From 329f8c4fa3a6a443f34e0e769677fc2e6451c56b Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 3 Sep 2013 14:30:42 -0400 Subject: [PATCH 001/247] init filesystem --- README.md | 22 +++- error/error.go | 2 + etcd.go | 4 +- etcd_handlers.go | 24 ++-- file_system/event.go | 123 +++++++++++++++++ file_system/event_test.go | 65 +++++++++ file_system/file_system.go | 139 ++++++++++++++++++++ file_system/file_system_test.go | 114 ++++++++++++++++ file_system/node.go | 225 ++++++++++++++++++++++++++++++++ file_system/watcher.go | 75 +++++++++++ file_system/watcher_test.go | 31 +++++ raft_server.go | 31 ++--- transporter.go | 66 ++++++++-- transporter_test.go | 36 +++++ util.go | 7 +- 15 files changed, 917 insertions(+), 47 deletions(-) create mode 100644 file_system/event.go create mode 100644 file_system/event_test.go create mode 100644 file_system/file_system.go create mode 100644 file_system/file_system_test.go create mode 100644 file_system/node.go create mode 100644 file_system/watcher.go create mode 100644 file_system/watcher_test.go create mode 100644 transporter_test.go diff --git a/README.md b/README.md index f6a4aa86bfc..cf883b0d1b6 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,5 @@ # etcd +README version 0.1.0 [![Build Status](https://travis-ci.org/coreos/etcd.png)](https://travis-ci.org/coreos/etcd) @@ -272,7 +273,7 @@ Next, lets configure etcd to use this keypair: You can now test the configuration using https: ```sh -curl --cacert fixtures/ca/ca.crt https://127.0.0.1:4001/v1/keys/foo -F value=bar +curl --cacert fixtures/ca/ca.crt https://127.0.0.1:4001/v1/keys/foo -d value=bar -v ``` You should be able to see the handshake succeed. @@ -302,7 +303,7 @@ We can also do authentication using CA certs. The clients will provide their cer Try the same request to this server: ```sh -curl --cacert fixtures/ca/ca.crt https://127.0.0.1:4001/v1/keys/foo -F value=bar +curl --cacert fixtures/ca/ca.crt https://127.0.0.1:4001/v1/keys/foo -d value=bar -v ``` The request should be rejected by the server. @@ -347,6 +348,9 @@ We use -s to specify server port and -c to specify client port and -d to specify ./etcd -s 127.0.0.1:7001 -c 127.0.0.1:4001 -d nodes/node1 -n node1 ``` +**Note:** If you want to run etcd on external IP address and still have access locally you need to add `-cl 0.0.0.0` so that it will listen on both external and localhost addresses. +A similar argument `-sl` is used to setup the listening address for the server port. + Let the join two more nodes to this cluster using the -C argument: ```sh @@ -363,7 +367,7 @@ curl -L http://127.0.0.1:4001/v1/machines We should see there are three nodes in the cluster ``` -0.0.0.0:4001,0.0.0.0:4002,0.0.0.0:4003 +http://127.0.0.1:4001, http://127.0.0.1:4002, http://127.0.0.1:4003 ``` The machine list is also available via this API: @@ -373,7 +377,7 @@ curl -L http://127.0.0.1:4001/v1/keys/_etcd/machines ``` ```json -[{"action":"GET","key":"/machines/node1","value":"0.0.0.0,7001,4001","index":4},{"action":"GET","key":"/machines/node3","value":"0.0.0.0,7002,4002","index":4},{"action":"GET","key":"/machines/node4","value":"0.0.0.0,7003,4003","index":4}] +[{"action":"GET","key":"/_etcd/machines/node1","value":"raft=http://127.0.0.1:7001&etcd=http://127.0.0.1:4001","index":4},{"action":"GET","key":"/_etcd/machines/node2","value":"raft=http://127.0.0.1:7002&etcd=http://127.0.0.1:4002","index":4},{"action":"GET","key":"/_etcd/machines/node3","value":"raft=http://127.0.0.1:7003&etcd=http://127.0.0.1:4003","index":4}] ``` The key of the machine is based on the ```commit index``` when it was added. The value of the machine is ```hostname```, ```raft port``` and ```client port```. @@ -386,7 +390,7 @@ curl -L http://127.0.0.1:4001/v1/leader The first server we set up should be the leader, if it has not dead during these commands. ``` -0.0.0.0:7001 +http://127.0.0.1:7001 ``` Now we can do normal SET and GET operations on keys as we explored earlier. @@ -414,7 +418,13 @@ curl -L http://127.0.0.1:4001/v1/leader ``` ``` -0.0.0.0:7002 or 0.0.0.0:7003 +http://127.0.0.1:7002 +``` + +or + +``` +http://127.0.0.1:7003 ``` You should be able to see this: diff --git a/error/error.go b/error/error.go index dc209f2081b..498f6e33e3f 100644 --- a/error/error.go +++ b/error/error.go @@ -17,6 +17,8 @@ func init() { errors[101] = "The given PrevValue is not equal to the value of the key" errors[102] = "Not A File" errors[103] = "Reached the max number of machines in the cluster" + errors[104] = "Not A Directory" + errors[105] = "Already exists" // Post form related errors errors[200] = "Value is Required in POST form" diff --git a/etcd.go b/etcd.go index 0bc3ef4ed3b..46546e8cc0d 100644 --- a/etcd.go +++ b/etcd.go @@ -52,8 +52,8 @@ func init() { flag.StringVar(&argInfo.Name, "n", "default-name", "the node name (required)") flag.StringVar(&argInfo.EtcdURL, "c", "127.0.0.1:4001", "the advertised public hostname:port for etcd client communication") flag.StringVar(&argInfo.RaftURL, "s", "127.0.0.1:7001", "the advertised public hostname:port for raft server communication") - flag.StringVar(&argInfo.EtcdListenHost, "cl", "127.0.0.1", "the listening hostname for etcd client communication") - flag.StringVar(&argInfo.RaftListenHost, "sl", "127.0.0.1", "the listening hostname for raft server communication") + flag.StringVar(&argInfo.EtcdListenHost, "cl", "", "the listening hostname for etcd client communication (defaults to advertised ip)") + flag.StringVar(&argInfo.RaftListenHost, "sl", "", "the listening hostname for raft server communication (defaults to advertised ip)") flag.StringVar(&argInfo.WebURL, "w", "", "the hostname:port of web interface") flag.StringVar(&argInfo.RaftTLS.CAFile, "serverCAFile", "", "the path of the CAFile") diff --git a/etcd_handlers.go b/etcd_handlers.go index 60e7b35b5ad..c0e82530a06 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -176,18 +176,18 @@ func dispatch(c Command, w http.ResponseWriter, req *http.Request, etcd bool) er //-------------------------------------- // Handler to return the current leader's raft address -func LeaderHttpHandler(w http.ResponseWriter, req *http.Request) error { - leader := r.Leader() - - if leader != "" { - w.WriteHeader(http.StatusOK) - raftURL, _ := nameToRaftURL(leader) - w.Write([]byte(raftURL)) - return nil - } else { - return etcdErr.NewError(301, "") - } -} +// func LeaderHttpHandler(w http.ResponseWriter, req *http.Request) error { +// leader := r.Leader() + +// if leader != "" { +// w.WriteHeader(http.StatusOK) +// raftURL, _ := nameToRaftURL(leader) +// w.Write([]byte(raftURL)) +// return nil +// } else { +// return etcdErr.NewError(301, "") +// } +// } // Handler to return all the known machines in the current cluster func MachinesHttpHandler(w http.ResponseWriter, req *http.Request) error { diff --git a/file_system/event.go b/file_system/event.go new file mode 100644 index 00000000000..a45d8feae88 --- /dev/null +++ b/file_system/event.go @@ -0,0 +1,123 @@ +package fileSystem + +import ( + "strings" + "sync" + "time" +) + +const ( + Set = "set" + Delete = "delete" + TestAndSet = "testAndSet" + TestIAndSet = "testiAndSet" +) + +type Event struct { + Action string `json:"action"` + Key string `json:"key"` + Dir bool `json:"dir,omitempty"` + PrevValue string `json:"prevValue,omitempty"` + Value string `json:"value,omitempty"` + Expiration *time.Time `json:"expiration,omitempty"` + TTL int64 `json:"ttl,omitempty"` // Time to live in second + // The command index of the raft machine when the command is executed + Index uint64 `json:"index"` + Term uint64 `json:"term"` +} + +func newEvent(action string, key string, index uint64, term uint64) *Event { + return &Event{ + Action: action, + Key: key, + Index: index, + Term: term, + } +} + +type eventQueue struct { + events []*Event + size int + front int + back int + capacity int +} + +func (eq *eventQueue) insert(e *Event) bool { + + eq.back = (eq.back + 1) % eq.capacity + eq.events[eq.back] = e + + if eq.size == eq.capacity { //dequeue + eq.front = (eq.back + 1) % eq.capacity + return true + } else { + eq.size++ + return false + } + +} + +type EventHistory struct { + Queue eventQueue + StartIndex uint64 + rwl sync.RWMutex +} + +func newEventHistory(capacity int) *EventHistory { + return &EventHistory{ + Queue: eventQueue{ + capacity: capacity, + events: make([]*Event, capacity), + back: -1, + }, + } +} + +// addEvent function adds event into the eventHistory +func (eh *EventHistory) addEvent(e *Event) { + eh.rwl.Lock() + defer eh.rwl.Unlock() + + if eh.Queue.insert(e) { + eh.StartIndex++ + } else { + eh.StartIndex = eh.Queue.events[eh.Queue.front].Index + } +} + +func (eh *EventHistory) scan(prefix string, index uint64) (*Event, error) { + eh.rwl.RLock() + defer eh.rwl.RUnlock() + + start := index - eh.StartIndex + + if start < 0 { + + // TODO: Add error type + return nil, nil + } + + if start >= uint64(eh.Queue.size) { + + return nil, nil + } + + i := int((start + uint64(eh.Queue.front)) % uint64(eh.Queue.capacity)) + + for { + + e := eh.Queue.events[i] + if strings.HasPrefix(e.Key, prefix) { + return e, nil + } + + i = (i + 1) % eh.Queue.capacity + + if i == eh.Queue.back { + // TODO: Add error type + return nil, nil + } + } + +} diff --git a/file_system/event_test.go b/file_system/event_test.go new file mode 100644 index 00000000000..2c9b694426b --- /dev/null +++ b/file_system/event_test.go @@ -0,0 +1,65 @@ +package fileSystem + +import ( + "testing" +) + +// TestEventQueue tests a queue with capacity = 100 +// Add 200 events into that queue, and test if the +// previous 100 events have been swapped out. +func TestEventQueue(t *testing.T) { + + eh := newEventHistory(100) + + // Add + for i := 0; i < 200; i++ { + e := newEvent(Set, "/foo", uint64(i), 0) + eh.addEvent(e) + } + + // Test + j := 100 + for i := eh.Queue.front; i != eh.Queue.back; i = (i + 1) % eh.Queue.capacity { + e := eh.Queue.events[i] + if e.Index != uint64(j) { + t.Fatalf("queue error!") + } + j++ + } + +} + +func TestScanHistory(t *testing.T) { + eh := newEventHistory(100) + + // Add + eh.addEvent(newEvent(Set, "/foo", 1, 0)) + eh.addEvent(newEvent(Set, "/foo/bar", 2, 0)) + eh.addEvent(newEvent(Set, "/foo/foo", 3, 0)) + eh.addEvent(newEvent(Set, "/foo/bar/bar", 4, 0)) + eh.addEvent(newEvent(Set, "/foo/foo/foo", 5, 0)) + + e, err := eh.scan("/foo", 1) + if err != nil || e.Index != 1 { + t.Fatalf("scan error [/foo] [1] %v", e.Index) + } + + e, err = eh.scan("/foo/bar", 1) + + if err != nil || e.Index != 2 { + t.Fatalf("scan error [/foo/bar] [2] %v", e.Index) + } + + e, err = eh.scan("/foo/bar", 3) + + if err != nil || e.Index != 4 { + t.Fatalf("scan error [/foo/bar/bar] [4] %v", e.Index) + } + + e, err = eh.scan("/foo/bar", 6) + + if e != nil { + t.Fatalf("bad index shoud reuturn nil") + } + +} diff --git a/file_system/file_system.go b/file_system/file_system.go new file mode 100644 index 00000000000..4ccc1e21ef8 --- /dev/null +++ b/file_system/file_system.go @@ -0,0 +1,139 @@ +package fileSystem + +import ( + "fmt" + "path/filepath" + "strings" + "time" + + etcdErr "github.com/coreos/etcd/error" +) + +type FileSystem struct { + Root *Node + EventHistory *EventHistory + WatcherHub *watcherHub + Index uint64 + Term uint64 +} + +func New() *FileSystem { + return &FileSystem{ + Root: newDir("/", 0, 0, nil, ""), + WatcherHub: newWatchHub(1000), + } + +} + +func (fs *FileSystem) InternalGet(path string, index uint64, term uint64) (*Node, error) { + fmt.Println("GET: ", path) + path = filepath.Clean("/" + path) + + // update file system known index and term + fs.Index, fs.Term = index, term + + walkFunc := func(parent *Node, dirName string) (*Node, error) { + child, ok := parent.Children[dirName] + if ok { + return child, nil + } + + return nil, etcdErr.NewError(100, "get") + } + + f, err := fs.walk(path, walkFunc) + + if err != nil { + return nil, err + } + + return f, nil +} + +func (fs *FileSystem) Set(path string, value string, expireTime time.Time, index uint64, term uint64) error { + path = filepath.Clean("/" + path) + + // update file system known index and term + fs.Index, fs.Term = index, term + + dir, name := filepath.Split(path) + + // walk through the path and get the last directory node + d, err := fs.walk(dir, fs.checkDir) + + if err != nil { + return err + } + + f := newFile(name, value, fs.Index, fs.Term, d, "", expireTime) + + err = d.Add(f) + + if err == nil { + if expireTime != Permanent { + go f.Expire() + } + } + + return err + +} + +func (fs *FileSystem) TestAndSet() { + +} + +func (fs *FileSystem) TestIndexAndSet() { + +} + +func (fs *FileSystem) Delete(path string, recurisive bool, index uint64, term uint64) error { + n, err := fs.InternalGet(path, index, term) + + if err != nil { + return err + } + + return n.Remove(recurisive) +} + +// walk function walks all the path and apply the walkFunc on each directory +func (fs *FileSystem) walk(path string, walkFunc func(prev *Node, component string) (*Node, error)) (*Node, error) { + components := strings.Split(path, "/") + + curr := fs.Root + + var err error + for i := 1; i < len(components); i++ { + if len(components[i]) == 0 { // ignore empty string + return curr, nil + } + + curr, err = walkFunc(curr, components[i]) + if err != nil { + return nil, err + } + + } + + return curr, nil +} + +// checkDir function will check whether the component is a directory under parent node. +// If it is a directory, this function will return the pointer to that node. +// If it does not exist, this function will create a new directory and return the pointer to that node. +// If it is a file, this function will return error. +func (fs *FileSystem) checkDir(parent *Node, dirName string) (*Node, error) { + + subDir, ok := parent.Children[dirName] + + if ok { + return subDir, nil + } + + n := newDir(filepath.Join(parent.Path, dirName), fs.Index, fs.Term, parent, parent.ACL) + + parent.Children[dirName] = n + + return n, nil +} diff --git a/file_system/file_system_test.go b/file_system/file_system_test.go new file mode 100644 index 00000000000..5c9ce9fd3b2 --- /dev/null +++ b/file_system/file_system_test.go @@ -0,0 +1,114 @@ +package fileSystem + +import ( + "testing" + "time" +) + +func TestSetAndGet(t *testing.T) { + fs := New() + setAndGet(fs, "/foobar", t) + setAndGet(fs, "/foo/bar", t) + setAndGet(fs, "/foo/foo/bar", t) +} + +func TestRemove(t *testing.T) { + fs := New() + + fs.Set("/foo", "bar", Permanent, 1, 1) + err := fs.Delete("/foo", false, 1, 1) + + if err != nil { + t.Fatalf("cannot delete %s [%s]", "/foo", err.Error()) + } + + _, err = fs.InternalGet("/foo", 1, 1) + + if err == nil || err.Error() != "Key Not Found" { + t.Fatalf("can get the node after deletion") + } + + fs.Set("/foo/bar", "bar", Permanent, 1, 1) + fs.Set("/foo/car", "car", Permanent, 1, 1) + fs.Set("/foo/dar/dar", "dar", Permanent, 1, 1) + + err = fs.Delete("/foo", false, 1, 1) + + if err == nil { + t.Fatalf("should not be able to delete a directory without recursive") + } + + err = fs.Delete("/foo", true, 1, 1) + + if err != nil { + t.Fatalf("cannot delete %s [%s]", "/foo", err.Error()) + } + + _, err = fs.InternalGet("/foo", 1, 1) + + if err == nil || err.Error() != "Key Not Found" { + t.Fatalf("can get the node after deletion ") + } + +} + +func TestExpire(t *testing.T) { + fs := New() + + expire := time.Now().Add(time.Second) + + fs.Set("/foo", "bar", expire, 1, 1) + + _, err := fs.InternalGet("/foo", 1, 1) + + if err != nil { + t.Fatalf("can not get the node") + } + + time.Sleep(time.Second * 2) + + _, err = fs.InternalGet("/foo", 1, 1) + + if err == nil { + t.Fatalf("can get the node after expiration time") + } + + fs.Set("/foo", "bar", expire, 1, 1) + + time.Sleep(time.Millisecond * 50) + _, err = fs.InternalGet("/foo", 1, 1) + + if err == nil { + t.Fatalf("can get the node after expiration time") + } + + expire = time.Now().Add(time.Second) + + fs.Set("/foo", "bar", expire, 1, 1) + fs.Delete("/foo", false, 1, 1) + +} + +func setAndGet(fs *FileSystem, path string, t *testing.T) { + err := fs.Set(path, "bar", Permanent, 1, 1) + + if err != nil { + t.Fatalf("cannot set %s=bar [%s]", path, err.Error()) + } + + n, err := fs.InternalGet(path, 1, 1) + + if err != nil { + t.Fatalf("cannot get %s [%s]", path, err.Error()) + } + + value, err := n.Read() + + if err != nil { + t.Fatalf("cannot read %s [%s]", path, err.Error()) + } + + if value != "bar" { + t.Fatalf("expect value of %s is bar [%s]", path, value) + } +} diff --git a/file_system/node.go b/file_system/node.go new file mode 100644 index 00000000000..00eb494c384 --- /dev/null +++ b/file_system/node.go @@ -0,0 +1,225 @@ +package fileSystem + +import ( + "fmt" + "path/filepath" + "sync" + "time" + + etcdErr "github.com/coreos/etcd/error" +) + +var ( + Permanent time.Time +) + +const ( + normal = iota + removed +) + +type Node struct { + Path string + CreateIndex uint64 + CreateTerm uint64 + Parent *Node + ExpireTime time.Time + ACL string + Value string // for key-value pair + Children map[string]*Node // for directory + status int + mu sync.Mutex + removeChan chan bool // remove channel +} + +func newFile(path string, value string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { + return &Node{ + Path: path, + CreateIndex: createIndex, + CreateTerm: createTerm, + Parent: parent, + ACL: ACL, + removeChan: make(chan bool, 1), + ExpireTime: expireTime, + Value: value, + } +} + +func newDir(path string, createIndex uint64, createTerm uint64, parent *Node, ACL string) *Node { + return &Node{ + Path: path, + CreateIndex: createIndex, + CreateTerm: createTerm, + Parent: parent, + ACL: ACL, + removeChan: make(chan bool, 1), + Children: make(map[string]*Node), + } +} + +// Remove function remove the node. +// If the node is a directory and recursive is true, the function will recursively remove +// add nodes under the receiver node. +func (n *Node) Remove(recursive bool) error { + n.mu.Lock() + defer n.mu.Unlock() + + if n.status == removed { + return nil + } + + if !n.IsDir() { // key-value pair + _, name := filepath.Split(n.Path) + + if n.Parent.Children[name] == n { + delete(n.Parent.Children, name) + n.removeChan <- true + n.status = removed + } + + return nil + } + + if !recursive { + return etcdErr.NewError(102, "") + } + + for _, n := range n.Children { // delete all children + n.Remove(true) + } + + // delete self + _, name := filepath.Split(n.Path) + if n.Parent.Children[name] == n { + delete(n.Parent.Children, name) + n.removeChan <- true + n.status = removed + } + + return nil +} + +// Get function gets the value of the node. +// If the receiver node is not a key-value pair, a "Not A File" error will be returned. +func (n *Node) Read() (string, error) { + if n.IsDir() { + return "", etcdErr.NewError(102, "") + } + + return n.Value, nil +} + +// Set function set the value of the node to the given value. +// If the receiver node is a directory, a "Not A File" error will be returned. +func (n *Node) Write(value string) error { + if n.IsDir() { + return etcdErr.NewError(102, "") + } + + n.Value = value + + return nil +} + +// List function return a slice of nodes under the receiver node. +// If the receiver node is not a directory, a "Not A Directory" error will be returned. +func (n *Node) List() ([]*Node, error) { + n.mu.Lock() + n.mu.Unlock() + if !n.IsDir() { + return nil, etcdErr.NewError(104, "") + } + + nodes := make([]*Node, len(n.Children)) + + i := 0 + for _, node := range n.Children { + nodes[i] = node + i++ + } + + return nodes, nil +} + +// Add function adds a node to the receiver node. +// If the receiver is not a directory, a "Not A Directory" error will be returned. +// If there is a existing node with the same name under the directory, a "Already Exist" +// error will be returned +func (n *Node) Add(child *Node) error { + n.mu.Lock() + n.mu.Unlock() + if n.status == removed { + return etcdErr.NewError(100, "") + } + + if !n.IsDir() { + return etcdErr.NewError(104, "") + } + + _, name := filepath.Split(child.Path) + + _, ok := n.Children[name] + + if ok { + return etcdErr.NewError(105, "") + } + + n.Children[name] = child + + return nil + +} + +// Clone function clone the node recursively and return the new node. +// If the node is a directory, it will clone all the content under this directory. +// If the node is a key-value pair, it will clone the pair. +func (n *Node) Clone() *Node { + n.mu.Lock() + n.mu.Unlock() + if !n.IsDir() { + return newFile(n.Path, n.Value, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime) + } + + clone := newDir(n.Path, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL) + + for key, child := range n.Children { + clone.Children[key] = child.Clone() + } + + return clone +} + +// IsDir function checks whether the node is a directory. +// If the node is a directory, the function will return true. +// Otherwise the function will return false. +func (n *Node) IsDir() bool { + + if n.Children == nil { // key-value pair + return false + } + + return true +} + +func (n *Node) Expire() { + for { + duration := n.ExpireTime.Sub(time.Now()) + if duration <= 0 { + n.Remove(true) + return + } + + select { + // if timeout, delete the node + case <-time.After(duration): + n.Remove(true) + return + + // if removed, return + case <-n.removeChan: + fmt.Println("node removed") + return + + } + } +} diff --git a/file_system/watcher.go b/file_system/watcher.go new file mode 100644 index 00000000000..dd6d5d8a5a6 --- /dev/null +++ b/file_system/watcher.go @@ -0,0 +1,75 @@ +package fileSystem + +import ( + "container/list" + "path/filepath" + "strings" +) + +type watcherHub struct { + watchers map[string]*list.List + count uint64 // current number of watchers + EventHistory *EventHistory +} + +func newWatchHub(capacity int) *watcherHub { + return &watcherHub{ + watchers: make(map[string]*list.List), + EventHistory: newEventHistory(capacity), + } +} + +func (wh *watcherHub) watch(prefix string, index uint64) (error, <-chan *Event) { + eventChan := make(chan *Event, 1) + + e, err := wh.EventHistory.scan(prefix, index) + + if err != nil { + return err, nil + } + + if e != nil { + eventChan <- e + return nil, eventChan + } + + l, ok := wh.watchers[prefix] + + if ok { + l.PushBack(eventChan) + } else { + l := list.New() + l.PushBack(eventChan) + wh.watchers[prefix] = l + } + + return nil, eventChan +} + +func (wh *watcherHub) notify(e *Event) { + + segments := strings.Split(e.Key, "/") + currPath := "/" + + // walk through all the paths + for _, segment := range segments { + currPath = filepath.Join(currPath, segment) + + l, ok := wh.watchers[currPath] + + if ok { + + for { + element := l.Front() + if element == nil { + delete(wh.watchers, currPath) + break + } + c, _ := element.Value.(chan *Event) + c <- e + l.Remove(element) + } + } + + } +} diff --git a/file_system/watcher_test.go b/file_system/watcher_test.go new file mode 100644 index 00000000000..fcd3fd9b354 --- /dev/null +++ b/file_system/watcher_test.go @@ -0,0 +1,31 @@ +package fileSystem + +import ( + "testing" +) + +func TestWatch(t *testing.T) { + wh := newWatchHub(100) + err, c := wh.watch("/foo", 0) + + if err != nil { + t.Fatal("%v", err) + } + + select { + case <-c: + t.Fatal("should not receive from channel before send the event") + default: + // do nothing + } + + e := newEvent(Set, "/foo/bar", 1, 0) + + wh.notify(e) + + re := <-c + + if e != re { + t.Fatal("recv != send") + } +} diff --git a/raft_server.go b/raft_server.go index b3b5d32776b..c8b86021c7d 100644 --- a/raft_server.go +++ b/raft_server.go @@ -16,13 +16,13 @@ import ( type raftServer struct { *raft.Server - version string - joinIndex uint64 - name string - url string + version string + joinIndex uint64 + name string + url string listenHost string - tlsConf *TLSConfig - tlsInfo *TLSInfo + tlsConf *TLSConfig + tlsInfo *TLSInfo } var r *raftServer @@ -30,7 +30,7 @@ var r *raftServer func newRaftServer(name string, url string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo) *raftServer { // Create transporter for raft - raftTransporter := newTransporter(tlsConf.Scheme, tlsConf.Client) + raftTransporter := newTransporter(tlsConf.Scheme, tlsConf.Client, ElectionTimeout) // Create raft server server, err := raft.NewServer(name, dirPath, raftTransporter, etcdStore, nil) @@ -38,13 +38,13 @@ func newRaftServer(name string, url string, listenHost string, tlsConf *TLSConfi check(err) return &raftServer{ - Server: server, - version: raftVersion, - name: name, - url: url, + Server: server, + version: raftVersion, + name: name, + url: url, listenHost: listenHost, - tlsConf: tlsConf, - tlsInfo: tlsInfo, + tlsConf: tlsConf, + tlsInfo: tlsInfo, } } @@ -169,7 +169,7 @@ func (r *raftServer) startTransport(scheme string, tlsConf tls.Config) { // getVersion fetches the raft version of a peer. This works for now but we // will need to do something more sophisticated later when we allow mixed // version clusters. -func getVersion(t transporter, versionURL url.URL) (string, error) { +func getVersion(t *transporter, versionURL url.URL) (string, error) { resp, err := t.Get(versionURL.String()) if err != nil { @@ -198,6 +198,7 @@ func joinCluster(cluster []string) bool { if _, ok := err.(etcdErr.Error); ok { fatal(err) } + debugf("cannot join to cluster via machine %s %s", machine, err) } } @@ -209,7 +210,7 @@ func joinByMachine(s *raft.Server, machine string, scheme string) error { var b bytes.Buffer // t must be ok - t, _ := r.Transporter().(transporter) + t, _ := r.Transporter().(*transporter) // Our version must match the leaders version versionURL := url.URL{Host: machine, Scheme: scheme, Path: "/version"} diff --git a/transporter.go b/transporter.go index c49479bc8ee..b4564742c2f 100644 --- a/transporter.go +++ b/transporter.go @@ -9,17 +9,25 @@ import ( "io" "net" "net/http" + "time" ) // Transporter layer for communication between raft nodes type transporter struct { - client *http.Client + client *http.Client + timeout time.Duration +} + +// response struct +type transporterResponse struct { + resp *http.Response + err error } // Create transporter using by raft server // Create http or https transporter based on // whether the user give the server cert and key -func newTransporter(scheme string, tlsConf tls.Config) transporter { +func newTransporter(scheme string, tlsConf tls.Config, timeout time.Duration) *transporter { t := transporter{} tr := &http.Transport{ @@ -32,8 +40,9 @@ func newTransporter(scheme string, tlsConf tls.Config) transporter { } t.client = &http.Client{Transport: tr} + t.timeout = timeout - return t + return &t } // Dial with timeout @@ -42,7 +51,7 @@ func dialTimeout(network, addr string) (net.Conn, error) { } // Sends AppendEntries RPCs to a peer when the server is the leader. -func (t transporter) SendAppendEntriesRequest(server *raft.Server, peer *raft.Peer, req *raft.AppendEntriesRequest) *raft.AppendEntriesResponse { +func (t *transporter) SendAppendEntriesRequest(server *raft.Server, peer *raft.Peer, req *raft.AppendEntriesRequest) *raft.AppendEntriesResponse { var aersp *raft.AppendEntriesResponse var b bytes.Buffer json.NewEncoder(&b).Encode(req) @@ -69,7 +78,7 @@ func (t transporter) SendAppendEntriesRequest(server *raft.Server, peer *raft.Pe } // Sends RequestVote RPCs to a peer when the server is the candidate. -func (t transporter) SendVoteRequest(server *raft.Server, peer *raft.Peer, req *raft.RequestVoteRequest) *raft.RequestVoteResponse { +func (t *transporter) SendVoteRequest(server *raft.Server, peer *raft.Peer, req *raft.RequestVoteRequest) *raft.RequestVoteResponse { var rvrsp *raft.RequestVoteResponse var b bytes.Buffer json.NewEncoder(&b).Encode(req) @@ -95,7 +104,7 @@ func (t transporter) SendVoteRequest(server *raft.Server, peer *raft.Peer, req * } // Sends SnapshotRequest RPCs to a peer when the server is the candidate. -func (t transporter) SendSnapshotRequest(server *raft.Server, peer *raft.Peer, req *raft.SnapshotRequest) *raft.SnapshotResponse { +func (t *transporter) SendSnapshotRequest(server *raft.Server, peer *raft.Peer, req *raft.SnapshotRequest) *raft.SnapshotResponse { var aersp *raft.SnapshotResponse var b bytes.Buffer json.NewEncoder(&b).Encode(req) @@ -123,7 +132,7 @@ func (t transporter) SendSnapshotRequest(server *raft.Server, peer *raft.Peer, r } // Sends SnapshotRecoveryRequest RPCs to a peer when the server is the candidate. -func (t transporter) SendSnapshotRecoveryRequest(server *raft.Server, peer *raft.Peer, req *raft.SnapshotRecoveryRequest) *raft.SnapshotRecoveryResponse { +func (t *transporter) SendSnapshotRecoveryRequest(server *raft.Server, peer *raft.Peer, req *raft.SnapshotRecoveryRequest) *raft.SnapshotRecoveryResponse { var aersp *raft.SnapshotRecoveryResponse var b bytes.Buffer json.NewEncoder(&b).Encode(req) @@ -150,11 +159,46 @@ func (t transporter) SendSnapshotRecoveryRequest(server *raft.Server, peer *raft } // Send server side POST request -func (t transporter) Post(path string, body io.Reader) (*http.Response, error) { - return t.client.Post(path, "application/json", body) +func (t *transporter) Post(path string, body io.Reader) (*http.Response, error) { + + c := make(chan *transporterResponse, 1) + + go func() { + tr := new(transporterResponse) + tr.resp, tr.err = t.client.Post(path, "application/json", body) + c <- tr + }() + + return t.waitResponse(c) + } // Send server side GET request -func (t transporter) Get(path string) (*http.Response, error) { - return t.client.Get(path) +func (t *transporter) Get(path string) (*http.Response, error) { + + c := make(chan *transporterResponse, 1) + + go func() { + tr := new(transporterResponse) + tr.resp, tr.err = t.client.Get(path) + c <- tr + }() + + return t.waitResponse(c) +} + +func (t *transporter) waitResponse(responseChan chan *transporterResponse) (*http.Response, error) { + + timeoutChan := time.After(t.timeout) + + select { + case <-timeoutChan: + return nil, fmt.Errorf("Wait Response Timeout: %v", t.timeout) + + case r := <-responseChan: + return r.resp, r.err + } + + // for complier + return nil, nil } diff --git a/transporter_test.go b/transporter_test.go new file mode 100644 index 00000000000..e440a094f1a --- /dev/null +++ b/transporter_test.go @@ -0,0 +1,36 @@ +package main + +import ( + "crypto/tls" + "testing" + "time" +) + +func TestTransporterTimeout(t *testing.T) { + + conf := tls.Config{} + + ts := newTransporter("http", conf, time.Second) + + ts.Get("http://google.com") + _, err := ts.Get("http://google.com:9999") // it doesn't exisit + if err == nil || err.Error() != "Wait Response Timeout: 1s" { + t.Fatal("timeout error: ", err.Error()) + } + + _, err = ts.Post("http://google.com:9999", nil) // it doesn't exisit + if err == nil || err.Error() != "Wait Response Timeout: 1s" { + t.Fatal("timeout error: ", err.Error()) + } + + _, err = ts.Get("http://www.google.com") + if err != nil { + t.Fatal("get error") + } + + _, err = ts.Post("http://www.google.com", nil) + if err != nil { + t.Fatal("post error") + } + +} diff --git a/util.go b/util.go index 8ddaa93cc57..22cbed6413c 100644 --- a/util.go +++ b/util.go @@ -114,11 +114,16 @@ func sanitizeListenHost(listen string, advertised string) string { fatal(err) } - _, aport, err := net.SplitHostPort(aurl.Host) + ahost, aport, err := net.SplitHostPort(aurl.Host) if err != nil { fatal(err) } + // If the listen host isn't set use the advertised host + if listen == "" { + listen = ahost + } + return net.JoinHostPort(listen, aport) } From 45c9ec9f29ba82f8ba3d905f7fde974e5c203741 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 3 Sep 2013 21:27:46 -0400 Subject: [PATCH 002/247] basic get --- file_system/event.go | 24 +++++++--- file_system/file_system.go | 97 ++++++++++++++++++++++++++++++-------- file_system/node.go | 22 +++++++++ 3 files changed, 116 insertions(+), 27 deletions(-) diff --git a/file_system/event.go b/file_system/event.go index a45d8feae88..56fc2859f2c 100644 --- a/file_system/event.go +++ b/file_system/event.go @@ -7,6 +7,7 @@ import ( ) const ( + Get = "get" Set = "set" Delete = "delete" TestAndSet = "testAndSet" @@ -14,18 +15,27 @@ const ( ) type Event struct { - Action string `json:"action"` - Key string `json:"key"` - Dir bool `json:"dir,omitempty"` - PrevValue string `json:"prevValue,omitempty"` - Value string `json:"value,omitempty"` - Expiration *time.Time `json:"expiration,omitempty"` - TTL int64 `json:"ttl,omitempty"` // Time to live in second + Action string `json:"action"` + Key string `json:"key, omitempty"` + Dir bool `json:"dir,omitempty"` + PrevValue string `json:"prevValue,omitempty"` + Value string `json:"value,omitempty"` + Pairs []KeyValuePair `json:"kvs,omitempty"` + Expiration *time.Time `json:"expiration,omitempty"` + TTL int64 `json:"ttl,omitempty"` // Time to live in second // The command index of the raft machine when the command is executed Index uint64 `json:"index"` Term uint64 `json:"term"` } +// When user list a directory, we add all the node into key-value pair slice +type KeyValuePair struct { + Key string `json:"key, omitempty"` + Value string `json:"value,omitempty"` + Dir bool `json:"dir,omitempty"` + Pairs []KeyValuePair `json:"kvs,omitempty"` +} + func newEvent(action string, key string, index uint64, term uint64) *Event { return &Event{ Action: action, diff --git a/file_system/file_system.go b/file_system/file_system.go index 4ccc1e21ef8..086f9ae2bea 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -25,29 +25,42 @@ func New() *FileSystem { } -func (fs *FileSystem) InternalGet(path string, index uint64, term uint64) (*Node, error) { - fmt.Println("GET: ", path) - path = filepath.Clean("/" + path) +func (fs *FileSystem) Get(path string, recusive bool, index uint64, term uint64) (*Event, error) { + // TODO: add recursive get + n, err := fs.InternalGet(path, index, term) - // update file system known index and term - fs.Index, fs.Term = index, term + if err != nil { + return nil, err + } - walkFunc := func(parent *Node, dirName string) (*Node, error) { - child, ok := parent.Children[dirName] - if ok { - return child, nil - } + e := newEvent(Get, path, index, term) - return nil, etcdErr.NewError(100, "get") - } + if n.IsDir() { // node is dir + e.Pairs = make([]KeyValuePair, len(n.Children)) - f, err := fs.walk(path, walkFunc) + i := 0 - if err != nil { - return nil, err + for _, subN := range n.Children { + + if subN.IsDir() { + e.Pairs[i] = KeyValuePair{ + Key: subN.Path, + Dir: true, + } + } else { + e.Pairs[i] = KeyValuePair{ + Key: subN.Path, + Value: subN.Value, + } + } + i++ + } + + } else { // node is file + e.Value = n.Value } - return f, nil + return e, nil } func (fs *FileSystem) Set(path string, value string, expireTime time.Time, index uint64, term uint64) error { @@ -66,17 +79,35 @@ func (fs *FileSystem) Set(path string, value string, expireTime time.Time, index } f := newFile(name, value, fs.Index, fs.Term, d, "", expireTime) + e := newEvent(Set, path, fs.Index, fs.Term) + e.Value = f.Value - err = d.Add(f) + // remove previous file if exist + oldFile, err := d.GetFile(name) if err == nil { - if expireTime != Permanent { - go f.Expire() + if oldFile != nil { + oldFile.Remove(false) + e.PrevValue = oldFile.Value } + } else { + return err } - return err + err = d.Add(f) + if err != nil { + return err + } + + // Node with TTL + if expireTime != Permanent { + go f.Expire() + e.Expiration = &f.ExpireTime + e.TTL = int64(expireTime.Sub(time.Now()) / time.Second) + } + + return nil } func (fs *FileSystem) TestAndSet() { @@ -119,6 +150,32 @@ func (fs *FileSystem) walk(path string, walkFunc func(prev *Node, component stri return curr, nil } +// InternalGet function get the node of the given path. +func (fs *FileSystem) InternalGet(path string, index uint64, term uint64) (*Node, error) { + fmt.Println("GET: ", path) + path = filepath.Clean("/" + path) + + // update file system known index and term + fs.Index, fs.Term = index, term + + walkFunc := func(parent *Node, dirName string) (*Node, error) { + child, ok := parent.Children[dirName] + if ok { + return child, nil + } + + return nil, etcdErr.NewError(100, "get") + } + + f, err := fs.walk(path, walkFunc) + + if err != nil { + return nil, err + } + + return f, nil +} + // checkDir function will check whether the component is a directory under parent node. // If it is a directory, this function will return the pointer to that node. // If it does not exist, this function will create a new directory and return the pointer to that node. diff --git a/file_system/node.go b/file_system/node.go index 00eb494c384..b695ba7877a 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -141,6 +141,28 @@ func (n *Node) List() ([]*Node, error) { return nodes, nil } +func (n *Node) GetFile(name string) (*Node, error) { + n.mu.Lock() + n.mu.Unlock() + + if !n.IsDir() { + return nil, etcdErr.NewError(104, n.Path) + } + + f, ok := n.Children[name] + + if ok { + if !f.IsDir() { + return f, nil + } else { + return nil, etcdErr.NewError(102, f.Path) + } + } + + return nil, nil + +} + // Add function adds a node to the receiver node. // If the receiver is not a directory, a "Not A Directory" error will be returned. // If there is a existing node with the same name under the directory, a "Already Exist" From b8967bc7d1ea39f8de7734d74d19e847916365fd Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 3 Sep 2013 22:35:25 -0400 Subject: [PATCH 003/247] support hidden node --- file_system/file_system.go | 37 ++++++++++++++++++++++++++++--------- file_system/node.go | 16 ++++++++++++++++ 2 files changed, 44 insertions(+), 9 deletions(-) diff --git a/file_system/file_system.go b/file_system/file_system.go index 086f9ae2bea..6c078916e3a 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -42,6 +42,10 @@ func (fs *FileSystem) Get(path string, recusive bool, index uint64, term uint64) for _, subN := range n.Children { + if subN.IsHidden() { // get will not list hidden node + continue + } + if subN.IsDir() { e.Pairs[i] = KeyValuePair{ Key: subN.Path, @@ -53,6 +57,7 @@ func (fs *FileSystem) Get(path string, recusive bool, index uint64, term uint64) Value: subN.Value, } } + i++ } @@ -63,7 +68,7 @@ func (fs *FileSystem) Get(path string, recusive bool, index uint64, term uint64) return e, nil } -func (fs *FileSystem) Set(path string, value string, expireTime time.Time, index uint64, term uint64) error { +func (fs *FileSystem) Set(path string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { path = filepath.Clean("/" + path) // update file system known index and term @@ -75,7 +80,7 @@ func (fs *FileSystem) Set(path string, value string, expireTime time.Time, index d, err := fs.walk(dir, fs.checkDir) if err != nil { - return err + return nil, err } f := newFile(name, value, fs.Index, fs.Term, d, "", expireTime) @@ -91,13 +96,13 @@ func (fs *FileSystem) Set(path string, value string, expireTime time.Time, index e.PrevValue = oldFile.Value } } else { - return err + return nil, err } err = d.Add(f) if err != nil { - return err + return nil, err } // Node with TTL @@ -107,10 +112,10 @@ func (fs *FileSystem) Set(path string, value string, expireTime time.Time, index e.TTL = int64(expireTime.Sub(time.Now()) / time.Second) } - return nil + return e, nil } -func (fs *FileSystem) TestAndSet() { +func (fs *FileSystem) TestAndSet(path string, recurisive bool, index uint64, term uint64) { } @@ -118,14 +123,28 @@ func (fs *FileSystem) TestIndexAndSet() { } -func (fs *FileSystem) Delete(path string, recurisive bool, index uint64, term uint64) error { +func (fs *FileSystem) Delete(path string, recurisive bool, index uint64, term uint64) (*Event, error) { n, err := fs.InternalGet(path, index, term) if err != nil { - return err + return nil, err + } + + err = n.Remove(recurisive) + + if err != nil { + return nil, err } - return n.Remove(recurisive) + e := newEvent(Delete, path, index, term) + + if n.IsDir() { + e.Dir = true + } else { + e.PrevValue = n.Value + } + + return e, nil } // walk function walks all the path and apply the walkFunc on each directory diff --git a/file_system/node.go b/file_system/node.go index b695ba7877a..d766bf54ff0 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -245,3 +245,19 @@ func (n *Node) Expire() { } } } + +// IsHidden function checks if the node is a hidden node. A hidden node +// will begin with '_' + +// A hidden node will not be shown via get command under a directory +// For example if we have /foo/_hidden and /foo/notHidden, get "/foo" +// will only return /foo/notHidden +func (n *Node) IsHidden() bool { + _, name := filepath.Split(n.Path) + + if name[0] == '_' { //hidden + return true + } + + return false +} From 23775dc776c0ccd706615e0f8c3e3d3760dc0416 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 3 Sep 2013 23:10:33 -0400 Subject: [PATCH 004/247] add hidden test --- file_system/event.go | 10 ++--- file_system/file_system.go | 11 ++++-- file_system/file_system_test.go | 66 +++++++++++++++++++++++++-------- 3 files changed, 63 insertions(+), 24 deletions(-) diff --git a/file_system/event.go b/file_system/event.go index 56fc2859f2c..26dc5f61fdd 100644 --- a/file_system/event.go +++ b/file_system/event.go @@ -20,7 +20,7 @@ type Event struct { Dir bool `json:"dir,omitempty"` PrevValue string `json:"prevValue,omitempty"` Value string `json:"value,omitempty"` - Pairs []KeyValuePair `json:"kvs,omitempty"` + KVPairs []KeyValuePair `json:"kvs,omitempty"` Expiration *time.Time `json:"expiration,omitempty"` TTL int64 `json:"ttl,omitempty"` // Time to live in second // The command index of the raft machine when the command is executed @@ -30,10 +30,10 @@ type Event struct { // When user list a directory, we add all the node into key-value pair slice type KeyValuePair struct { - Key string `json:"key, omitempty"` - Value string `json:"value,omitempty"` - Dir bool `json:"dir,omitempty"` - Pairs []KeyValuePair `json:"kvs,omitempty"` + Key string `json:"key, omitempty"` + Value string `json:"value,omitempty"` + Dir bool `json:"dir,omitempty"` + KVPairs []KeyValuePair `json:"kvs,omitempty"` } func newEvent(action string, key string, index uint64, term uint64) *Event { diff --git a/file_system/file_system.go b/file_system/file_system.go index 6c078916e3a..48986839488 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -36,7 +36,7 @@ func (fs *FileSystem) Get(path string, recusive bool, index uint64, term uint64) e := newEvent(Get, path, index, term) if n.IsDir() { // node is dir - e.Pairs = make([]KeyValuePair, len(n.Children)) + e.KVPairs = make([]KeyValuePair, len(n.Children)) i := 0 @@ -47,12 +47,12 @@ func (fs *FileSystem) Get(path string, recusive bool, index uint64, term uint64) } if subN.IsDir() { - e.Pairs[i] = KeyValuePair{ + e.KVPairs[i] = KeyValuePair{ Key: subN.Path, Dir: true, } } else { - e.Pairs[i] = KeyValuePair{ + e.KVPairs[i] = KeyValuePair{ Key: subN.Path, Value: subN.Value, } @@ -61,6 +61,9 @@ func (fs *FileSystem) Get(path string, recusive bool, index uint64, term uint64) i++ } + // eliminate hidden nodes + e.KVPairs = e.KVPairs[:i] + } else { // node is file e.Value = n.Value } @@ -83,7 +86,7 @@ func (fs *FileSystem) Set(path string, value string, expireTime time.Time, index return nil, err } - f := newFile(name, value, fs.Index, fs.Term, d, "", expireTime) + f := newFile(path, value, fs.Index, fs.Term, d, "", expireTime) e := newEvent(Set, path, fs.Index, fs.Term) e.Value = f.Value diff --git a/file_system/file_system_test.go b/file_system/file_system_test.go index 5c9ce9fd3b2..e43750d4ef2 100644 --- a/file_system/file_system_test.go +++ b/file_system/file_system_test.go @@ -12,17 +12,59 @@ func TestSetAndGet(t *testing.T) { setAndGet(fs, "/foo/foo/bar", t) } +func TestListDirectory(t *testing.T) { + fs := New() + + // create dir /foo + // set key-value /foo/foo=bar + fs.Set("/foo/foo", "bar", Permanent, 1, 1) + + // create dir /foo/fooDir + // set key-value /foo/fooDir/foo=bar + fs.Set("/foo/fooDir/foo", "bar", Permanent, 2, 1) + + e, err := fs.Get("/foo", false, 2, 1) + + if err != nil { + t.Fatalf("%v", err) + } + + if len(e.KVPairs) != 2 { + t.Fatalf("wrong number of kv pairs [%d/2]", len(e.KVPairs)) + } + + if e.KVPairs[0].Key != "/foo/foo" || e.KVPairs[0].Value != "bar" { + t.Fatalf("wrong kv [/foo/foo/ / %s] -> [bar / %s]", e.KVPairs[0].Key, e.KVPairs[0].Value) + } + + if e.KVPairs[1].Key != "/foo/fooDir" || e.KVPairs[1].Dir != true { + t.Fatalf("wrong kv [/foo/fooDir/ / %s] -> [true / %v]", e.KVPairs[1].Key, e.KVPairs[1].Dir) + } + + // test hidden node + + // create dir /foo/_hidden + // set key-value /foo/_hidden/foo -> bar + fs.Set("/foo/_hidden/foo", "bar", Permanent, 3, 1) + + e, _ = fs.Get("/foo", false, 2, 1) + + if len(e.KVPairs) != 2 { + t.Fatalf("hidden node is not hidden! %s", e.KVPairs[2].Key) + } +} + func TestRemove(t *testing.T) { fs := New() fs.Set("/foo", "bar", Permanent, 1, 1) - err := fs.Delete("/foo", false, 1, 1) + _, err := fs.Delete("/foo", false, 1, 1) if err != nil { t.Fatalf("cannot delete %s [%s]", "/foo", err.Error()) } - _, err = fs.InternalGet("/foo", 1, 1) + _, err = fs.Get("/foo", false, 1, 1) if err == nil || err.Error() != "Key Not Found" { t.Fatalf("can get the node after deletion") @@ -32,19 +74,19 @@ func TestRemove(t *testing.T) { fs.Set("/foo/car", "car", Permanent, 1, 1) fs.Set("/foo/dar/dar", "dar", Permanent, 1, 1) - err = fs.Delete("/foo", false, 1, 1) + _, err = fs.Delete("/foo", false, 1, 1) if err == nil { t.Fatalf("should not be able to delete a directory without recursive") } - err = fs.Delete("/foo", true, 1, 1) + _, err = fs.Delete("/foo", true, 1, 1) if err != nil { t.Fatalf("cannot delete %s [%s]", "/foo", err.Error()) } - _, err = fs.InternalGet("/foo", 1, 1) + _, err = fs.Get("/foo", false, 1, 1) if err == nil || err.Error() != "Key Not Found" { t.Fatalf("can get the node after deletion ") @@ -90,25 +132,19 @@ func TestExpire(t *testing.T) { } func setAndGet(fs *FileSystem, path string, t *testing.T) { - err := fs.Set(path, "bar", Permanent, 1, 1) + _, err := fs.Set(path, "bar", Permanent, 1, 1) if err != nil { t.Fatalf("cannot set %s=bar [%s]", path, err.Error()) } - n, err := fs.InternalGet(path, 1, 1) + e, err := fs.Get(path, false, 1, 1) if err != nil { t.Fatalf("cannot get %s [%s]", path, err.Error()) } - value, err := n.Read() - - if err != nil { - t.Fatalf("cannot read %s [%s]", path, err.Error()) - } - - if value != "bar" { - t.Fatalf("expect value of %s is bar [%s]", path, value) + if e.Value != "bar" { + t.Fatalf("expect value of %s is bar [%s]", path, e.Value) } } From c56312f09f8b1d5331756d5748b46f775f6d2ffe Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 3 Sep 2013 23:21:26 -0400 Subject: [PATCH 005/247] refactor --- file_system/file_system.go | 18 +++--------------- file_system/node.go | 15 +++++++++++++++ 2 files changed, 18 insertions(+), 15 deletions(-) diff --git a/file_system/file_system.go b/file_system/file_system.go index 48986839488..e5d580baaad 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -1,7 +1,6 @@ package fileSystem import ( - "fmt" "path/filepath" "strings" "time" @@ -40,23 +39,13 @@ func (fs *FileSystem) Get(path string, recusive bool, index uint64, term uint64) i := 0 - for _, subN := range n.Children { + for _, child := range n.Children { - if subN.IsHidden() { // get will not list hidden node + if child.IsHidden() { // get will not list hidden node continue } - if subN.IsDir() { - e.KVPairs[i] = KeyValuePair{ - Key: subN.Path, - Dir: true, - } - } else { - e.KVPairs[i] = KeyValuePair{ - Key: subN.Path, - Value: subN.Value, - } - } + e.KVPairs[i] = child.Pair() i++ } @@ -174,7 +163,6 @@ func (fs *FileSystem) walk(path string, walkFunc func(prev *Node, component stri // InternalGet function get the node of the given path. func (fs *FileSystem) InternalGet(path string, index uint64, term uint64) (*Node, error) { - fmt.Println("GET: ", path) path = filepath.Clean("/" + path) // update file system known index and term diff --git a/file_system/node.go b/file_system/node.go index d766bf54ff0..702ce645686 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -261,3 +261,18 @@ func (n *Node) IsHidden() bool { return false } + +func (n *Node) Pair() KeyValuePair { + + if n.IsDir() { + return KeyValuePair{ + Key: n.Path, + Dir: true, + } + } + + return KeyValuePair{ + Key: n.Path, + Value: n.Value, + } +} From 90f691fc2af66e0ae5f449f3d89760c03bb9584f Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Wed, 4 Sep 2013 18:22:07 -0400 Subject: [PATCH 006/247] node.go: fix defer and add doc --- file_system/node.go | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/file_system/node.go b/file_system/node.go index 702ce645686..2a7f5e83ba7 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -68,10 +68,12 @@ func (n *Node) Remove(recursive bool) error { return nil } - if !n.IsDir() { // key-value pair + if !n.IsDir() { // file node: key-value pair _, name := filepath.Split(n.Path) if n.Parent.Children[name] == n { + // This is the only pointer to Node object + // Handled by garbage collector delete(n.Parent.Children, name) n.removeChan <- true n.status = removed @@ -84,8 +86,8 @@ func (n *Node) Remove(recursive bool) error { return etcdErr.NewError(102, "") } - for _, n := range n.Children { // delete all children - n.Remove(true) + for _, child := range n.Children { // delete all children + child.Remove(true) } // delete self @@ -125,7 +127,7 @@ func (n *Node) Write(value string) error { // If the receiver node is not a directory, a "Not A Directory" error will be returned. func (n *Node) List() ([]*Node, error) { n.mu.Lock() - n.mu.Unlock() + defer n.mu.Unlock() if !n.IsDir() { return nil, etcdErr.NewError(104, "") } @@ -143,7 +145,7 @@ func (n *Node) List() ([]*Node, error) { func (n *Node) GetFile(name string) (*Node, error) { n.mu.Lock() - n.mu.Unlock() + defer n.mu.Unlock() if !n.IsDir() { return nil, etcdErr.NewError(104, n.Path) @@ -169,7 +171,7 @@ func (n *Node) GetFile(name string) (*Node, error) { // error will be returned func (n *Node) Add(child *Node) error { n.mu.Lock() - n.mu.Unlock() + defer n.mu.Unlock() if n.status == removed { return etcdErr.NewError(100, "") } @@ -197,7 +199,7 @@ func (n *Node) Add(child *Node) error { // If the node is a key-value pair, it will clone the pair. func (n *Node) Clone() *Node { n.mu.Lock() - n.mu.Unlock() + defer n.mu.Unlock() if !n.IsDir() { return newFile(n.Path, n.Value, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime) } From aed76d0e08f6c50911aeb3c927e84d553a8b02b0 Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Wed, 4 Sep 2013 18:23:44 -0400 Subject: [PATCH 007/247] gofmt --- file_system/node.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/file_system/node.go b/file_system/node.go index 2a7f5e83ba7..6e1c3246e78 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -68,12 +68,12 @@ func (n *Node) Remove(recursive bool) error { return nil } - if !n.IsDir() { // file node: key-value pair + if !n.IsDir() { // file node: key-value pair _, name := filepath.Split(n.Path) if n.Parent.Children[name] == n { - // This is the only pointer to Node object - // Handled by garbage collector + // This is the only pointer to Node object + // Handled by garbage collector delete(n.Parent.Children, name) n.removeChan <- true n.status = removed From 03af286b03d845e48e38ebeea21d868724b1cb6d Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Wed, 4 Sep 2013 20:50:04 -0400 Subject: [PATCH 008/247] change filepath to path and fix path namespace collision --- file_system/file_system.go | 44 +++++++++++++++++++------------------- file_system/node.go | 18 ++++++++-------- file_system/watcher.go | 4 ++-- 3 files changed, 33 insertions(+), 33 deletions(-) diff --git a/file_system/file_system.go b/file_system/file_system.go index e5d580baaad..e834d2d38da 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -1,7 +1,7 @@ package fileSystem import ( - "path/filepath" + "path" "strings" "time" @@ -24,15 +24,15 @@ func New() *FileSystem { } -func (fs *FileSystem) Get(path string, recusive bool, index uint64, term uint64) (*Event, error) { +func (fs *FileSystem) Get(key_path string, recusive bool, index uint64, term uint64) (*Event, error) { // TODO: add recursive get - n, err := fs.InternalGet(path, index, term) + n, err := fs.InternalGet(key_path, index, term) if err != nil { return nil, err } - e := newEvent(Get, path, index, term) + e := newEvent(Get, key_path, index, term) if n.IsDir() { // node is dir e.KVPairs = make([]KeyValuePair, len(n.Children)) @@ -60,23 +60,23 @@ func (fs *FileSystem) Get(path string, recusive bool, index uint64, term uint64) return e, nil } -func (fs *FileSystem) Set(path string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { - path = filepath.Clean("/" + path) +func (fs *FileSystem) Set(key_path string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { + key_path = path.Clean("/" + key_path) // update file system known index and term fs.Index, fs.Term = index, term - dir, name := filepath.Split(path) + dir, name := path.Split(key_path) - // walk through the path and get the last directory node + // walk through the key_path and get the last directory node d, err := fs.walk(dir, fs.checkDir) if err != nil { return nil, err } - f := newFile(path, value, fs.Index, fs.Term, d, "", expireTime) - e := newEvent(Set, path, fs.Index, fs.Term) + f := newFile(key_path, value, fs.Index, fs.Term, d, "", expireTime) + e := newEvent(Set, key_path, fs.Index, fs.Term) e.Value = f.Value // remove previous file if exist @@ -107,7 +107,7 @@ func (fs *FileSystem) Set(path string, value string, expireTime time.Time, index return e, nil } -func (fs *FileSystem) TestAndSet(path string, recurisive bool, index uint64, term uint64) { +func (fs *FileSystem) TestAndSet(key_path string, recurisive bool, index uint64, term uint64) { } @@ -115,8 +115,8 @@ func (fs *FileSystem) TestIndexAndSet() { } -func (fs *FileSystem) Delete(path string, recurisive bool, index uint64, term uint64) (*Event, error) { - n, err := fs.InternalGet(path, index, term) +func (fs *FileSystem) Delete(key_path string, recurisive bool, index uint64, term uint64) (*Event, error) { + n, err := fs.InternalGet(key_path, index, term) if err != nil { return nil, err @@ -128,7 +128,7 @@ func (fs *FileSystem) Delete(path string, recurisive bool, index uint64, term ui return nil, err } - e := newEvent(Delete, path, index, term) + e := newEvent(Delete, key_path, index, term) if n.IsDir() { e.Dir = true @@ -139,9 +139,9 @@ func (fs *FileSystem) Delete(path string, recurisive bool, index uint64, term ui return e, nil } -// walk function walks all the path and apply the walkFunc on each directory -func (fs *FileSystem) walk(path string, walkFunc func(prev *Node, component string) (*Node, error)) (*Node, error) { - components := strings.Split(path, "/") +// walk function walks all the key_path and apply the walkFunc on each directory +func (fs *FileSystem) walk(key_path string, walkFunc func(prev *Node, component string) (*Node, error)) (*Node, error) { + components := strings.Split(key_path, "/") curr := fs.Root @@ -161,9 +161,9 @@ func (fs *FileSystem) walk(path string, walkFunc func(prev *Node, component stri return curr, nil } -// InternalGet function get the node of the given path. -func (fs *FileSystem) InternalGet(path string, index uint64, term uint64) (*Node, error) { - path = filepath.Clean("/" + path) +// InternalGet function get the node of the given key_path. +func (fs *FileSystem) InternalGet(key_path string, index uint64, term uint64) (*Node, error) { + key_path = path.Clean("/" + key_path) // update file system known index and term fs.Index, fs.Term = index, term @@ -177,7 +177,7 @@ func (fs *FileSystem) InternalGet(path string, index uint64, term uint64) (*Node return nil, etcdErr.NewError(100, "get") } - f, err := fs.walk(path, walkFunc) + f, err := fs.walk(key_path, walkFunc) if err != nil { return nil, err @@ -198,7 +198,7 @@ func (fs *FileSystem) checkDir(parent *Node, dirName string) (*Node, error) { return subDir, nil } - n := newDir(filepath.Join(parent.Path, dirName), fs.Index, fs.Term, parent, parent.ACL) + n := newDir(path.Join(parent.Path, dirName), fs.Index, fs.Term, parent, parent.ACL) parent.Children[dirName] = n diff --git a/file_system/node.go b/file_system/node.go index 6e1c3246e78..0fbd8d1bd27 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -2,7 +2,7 @@ package fileSystem import ( "fmt" - "path/filepath" + "path" "sync" "time" @@ -32,9 +32,9 @@ type Node struct { removeChan chan bool // remove channel } -func newFile(path string, value string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { +func newFile(key_path string, value string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { return &Node{ - Path: path, + Path: key_path, CreateIndex: createIndex, CreateTerm: createTerm, Parent: parent, @@ -45,9 +45,9 @@ func newFile(path string, value string, createIndex uint64, createTerm uint64, p } } -func newDir(path string, createIndex uint64, createTerm uint64, parent *Node, ACL string) *Node { +func newDir(key_path string, createIndex uint64, createTerm uint64, parent *Node, ACL string) *Node { return &Node{ - Path: path, + Path: key_path, CreateIndex: createIndex, CreateTerm: createTerm, Parent: parent, @@ -69,7 +69,7 @@ func (n *Node) Remove(recursive bool) error { } if !n.IsDir() { // file node: key-value pair - _, name := filepath.Split(n.Path) + _, name := path.Split(n.Path) if n.Parent.Children[name] == n { // This is the only pointer to Node object @@ -91,7 +91,7 @@ func (n *Node) Remove(recursive bool) error { } // delete self - _, name := filepath.Split(n.Path) + _, name := path.Split(n.Path) if n.Parent.Children[name] == n { delete(n.Parent.Children, name) n.removeChan <- true @@ -180,7 +180,7 @@ func (n *Node) Add(child *Node) error { return etcdErr.NewError(104, "") } - _, name := filepath.Split(child.Path) + _, name := path.Split(child.Path) _, ok := n.Children[name] @@ -255,7 +255,7 @@ func (n *Node) Expire() { // For example if we have /foo/_hidden and /foo/notHidden, get "/foo" // will only return /foo/notHidden func (n *Node) IsHidden() bool { - _, name := filepath.Split(n.Path) + _, name := path.Split(n.Path) if name[0] == '_' { //hidden return true diff --git a/file_system/watcher.go b/file_system/watcher.go index dd6d5d8a5a6..237399dba02 100644 --- a/file_system/watcher.go +++ b/file_system/watcher.go @@ -2,7 +2,7 @@ package fileSystem import ( "container/list" - "path/filepath" + "path" "strings" ) @@ -53,7 +53,7 @@ func (wh *watcherHub) notify(e *Event) { // walk through all the paths for _, segment := range segments { - currPath = filepath.Join(currPath, segment) + currPath = path.Join(currPath, segment) l, ok := wh.watchers[currPath] From 197b9106f9c95fff3786ffd69321a0634b6edac0 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Wed, 4 Sep 2013 22:40:33 -0400 Subject: [PATCH 009/247] init file system spec --- Documentation/etcd-file-system.md | 51 ++++++++++++++++++++++++ Documentation/img/etcd_fs_structure.jpg | Bin 0 -> 26643 bytes 2 files changed, 51 insertions(+) create mode 100644 Documentation/etcd-file-system.md create mode 100644 Documentation/img/etcd_fs_structure.jpg diff --git a/Documentation/etcd-file-system.md b/Documentation/etcd-file-system.md new file mode 100644 index 00000000000..22ab061b2ac --- /dev/null +++ b/Documentation/etcd-file-system.md @@ -0,0 +1,51 @@ +#Etcd File System + +## Structure +[TODO] +![alt text](./img/etcd_fs_structure.jpg "etcd file system structure") + +## Node +The building ingredients of etcd file system are two kinds of nodes -- files and directories: +- File has data associated with it. +- Directory has children nodes associated with it. + +### Attributes: +####Expiration Time [optional] +The node will be deleted when it expires. +####ACL +The path of access control list of the node. + +### Operation: +#### Get (path, recursive) +Get the content of the node + - If the node is a file, the data of the file will be returned. + - If the node is a directory, the child nodes of the directory will be returned. + - If recursive is true, it will recursively get the nodes of the directory. + +####Set (path, value[optional], ttl [optional]) +Set the value to a file. Set operation will help to create intermediate directories with no expiration time. + - If the value is given, set will create a file + - If the value is not given, set will crate a directory + - If ttl is given, the node will be deleted when it expires. + +####Delete (path, recursive) +Delete the node of given path. + - If the node is a directory: + - If recursive is true, the operation will delete all nodes under the directory. + - If recursive is false, error will be returned. + +####TestAndSet (path, prevValue [prevIndex], value, ttl) +Atomically test and set value to a file. If test succeeds, this operation will change the previous value of the file to the given value. + - If the prevValue is given, it will test against previous value of the node. + - If the prevValue is empty, it will test if the node is not existing. + - If the prevValue is not empty, it will test if the prevValue is equal to the current value of the file. + - If the prevIndex is given, it will test if the create/last modified index of the node is equal to prevIndex. + +####Renew (path, ttl) +Set the node's expiration time to (current time + ttl) + +## ACL +[TODO] + +## User Group +[TODO] \ No newline at end of file diff --git a/Documentation/img/etcd_fs_structure.jpg b/Documentation/img/etcd_fs_structure.jpg new file mode 100644 index 0000000000000000000000000000000000000000..51140caa0485e487f0bb26f51edebc29d4748439 GIT binary patch literal 26643 zcmdSA2UrwM^DjCZ$YIGDmJEsniAq`!Ng|SiM3I~&3F5+%b4F27K|n+$NtT?!Oq3|X zE|NiZMPL4)$zKO?9a1?&_{zkH=HT3jo6<9eo`D0s#OB z_y-*0xcc<9v>c61jdb)cYya_rtKR9x4Id~20QmX^2b$_$;J3bN!%s2yw;X^7aN5Z^ z=!T}T@nztDdhJb|lnwx6k|$;TPsRTG7mO~hLC#h+FMfEqht`$9 z8S{ut9>UYL1c zi0O$PghATP)z9L@h9~?K7pIFSwgqhhdj#kJPCy{w3S0;LfFpn(xD03k7yrETgM2@b z=M4A*K41#uNBs4MY?*9?3`h2vY=Lb0FCLilPdUI7Z~^JRmGw{gLB1If4s3y?x&lFe z)duH+tHV{`m%+ORm}KGS;OGB(_i3|f^Dp|(wz`9PCI2k@!k?|b`OmF)^o$Aow>;PX zp5wo^NeB1_TEP7;37K-4HJQV|^8Z6)a&7X9ffsyg8Ws$WB|yvn#;rh4caLCxP0(Aqp5fQ` zbCx>8FDr9a4mk0>Co%xAWBP|rg|H3%gXcK`0E#u>{G9p+&mEj2EjR$6y8I8GNCh~x zK;QO;$~h$P=HJSLpZt&k)BpmU6`TMcAOwhm7Lf%M0Tn<4&;~96Mu0hR6>tE@um|7^ zwm1|(1JOVNkP2i0cY%DM5GVnj1C>B6&;+yr-M|3w9vB8b1GB&aumo%Xcwi4WfIuJ= z5IP7mgcE{e6><&Y4)KG8K+up_NGc=;k`F0@ynxg|njzhg zw~!IYGz1G-gZzN}hQgq~pV^hjy)9LqW5b7|+^oF_Yf;XLMi z+xcJ0ips&twaVKnk}BRRFI84m#Z+BYpQ7ZD>T1fkhu_eq2a>5maXht$~?Az5(v?>C1taTMVIwmke_a zzg!Wy;&-LV2r$w&$~BrZ7BdboZZ#n{F*bQj>-jSGlfwUVUvtVPk3Y%x2eC+xD*QlAVHGg589@ zn0=W2TL&%&ABT2Fgrk#V-8HgnR@Yu$J9IK~Dt6kvu7CZ}b-c5d^F8Mc7Y&y?E^Dr8 zt~stNZfb5hZmaHU?z!%39w?7Ik4;Z4&wNjUm!4Ol*S@!rcbWIGkCjigFQxA_-)281 zKX1Q2e?I>(|B)LKHxh0v1SkjG4ZsH)1U?Ib1lb3@4rT#2xgSErL*hfQH`Q-Gyh#kT z46O@e4D$>7aO?D~4@L+Rhgpi!jd~tU z9qk?cF-9gPC+26YRcuQfZ(LN|V!UpAc>;YxV8UdgN@8IWJjpZZQ?gw0gJe>QOUk=c znbf@0!!+l#cj;%-@24MUxMh6GRLm^MBG2;6n#e|FKhI&z3C~%~HOy_i!+$5`&dKiYrn@_4jBqoAsgw=lKv z_Y==2Q$@N(jm4tHc_ri}H%peET0R{tJy-hT8P~JaXGdjzWed+upZC2`dhy~V&&$km zXn9EaT7_N3aOK6y*Hw~LMb&K8sjq-nA+OeJuGLJ`8r1gGovW*=7pZ^Lz}%412x+|4 zh;Q;}T6lf+^+@xj=AIUnmWEcT*0MH%wukMk?U^0a9dRAUo#CCkT{pTmyWP7NdmMXa zd#!uN`%L;i^&9lR8PFN%9n>7`e53ZJ?XB|L=6C1bHN97S-}ph{L&Hafj}4y`KQ#_1 z4ZR*dKio2+I?^$U8toptI5se@KmPvnmCqv+786sGc9RQJE>o-1KGTGmn==QqQD4Zu zq|7nQ<$dM(T0DPhzIs7%p&fe>`vGT$n_YBX-1rvs?Qkh>nQl361-bHKRd%&)O=oR* z-FAIxi5b)=pn`7{i9PyEhGcdmt)`K;~P$aPJiqLVBj6<<^}*i%D}kjDgZDK z0{}Js-(sUbB*5)11e5`3ONbO?1KI}*f{Vb{$%@DwDFi5HDT}GRsAZ`SX@+PE>4NF6 zAWkumGvFBenVvBxv-q=GvT3tharkm3a6RE};Th%KL{jk!3up>n6N(UiDAFuCBli0g z_i1&B>t|vmUq}r}@0{h3MaenKrz+GajwpRUPp5oZMMw3zTBLeDs$S!h=E?=qMGkE# zor}6wdfxizOBn_Qm#YlhuY53?Fzx#{_-iy+k-{-uqnV-LZ%8k;1?!d1>2O%6c6++F!0&Zo6SKl5& zZ$~m<Fz&?d?y*(ArLyht5OJIoNxG|IY_{WRxe?)F_o-r0K=_ro4M z%I|)-@|dnbq0sJ0Y|+c&k&^vVo@bh6uFn%+JbyV@{;lG;ilbWQm2Qn)ZD3t;eL=&k z#DUZTaod9nPH^UF==!-OWAGy}Er=eee1c1~dl_2J7E=zZHBt^Dg`S zUnV#Bw@CC6p86`@szHL`W+2K6TQmeP0I?L>SF;pdM0&#>K&eF~z< zuSdU^52cPmNG-?5fAqf)m%q9Qw+t7K1@o?6wGrhA6N(2y4c$}UUSxQz2dImDdT<0SAs0zFBT{gEEakyTqg2D zv{I}_y#7?n>28VsGw&ovrKY6kWWJuAll>w$BR{F|S#eBh=-j*W{mR`ct*Z5EmFiDX z`5Kv;u@^$Mye`^nU(wOfJ*&sB&u|I2gg5wdx!+zE_-du=H0wi-w(H>Dgq*A9^piZQbrJ1KqrBkKb zq<@SsWuRf`XUt%7WIoFRXBlTLWea0Ba}Q|Sg^S9De+T>r$;18&)kwUloF64NO#L*pS>n4CHqUR2lNkSiX4iwN`>c~ z&I_K$Di^9asB)`Ls^zL1q3BSf8u^;e7tUPR(Q3UIr+rz6MQ2vG1oRiO`oAu986;jd zGGqhy6U9cJ#!|*VOY|=ASIGEzPait!AtXu3onhxA|_{XcuLF$$`z`8@LaM zbF#XA#u@57<5KUM<>u>d;345j|O7Z=Nsy0?XP@;`^Hhgd|-D_NpM1l=S{=V zvtevu2e)wH?{3$kA4bGS-oV&J=|?NZ2*onRLE;GU%Lz+~Yf0aef28cC{!SxhKr_j+ zsIuvD7;;(eu;1m%@L5OM%yZ&P zwsQFjlS<#Jlq=Bcn1-|nr9Y=mrQ{?zzYank>P z!^lq71ad_HC`Ew_0=RCN6$3y?8vux$1J{pI05GNl0Q2(z)I1!3e0>c-u0Htt`T+sp z040dMaDqsp9Eg@!g0XoF7)v$)BfvHU0XYjXhlGPM;5*1JlpA^h>IKb(c0ku*957v2 zDC`C73!DM21CIb>GCUZEd4RFk61fPu2YEU9HiZI31jPU)1En413(9>eJ*r1kKdAMo zi>Z%jY-nE7a?pm;exr6V%~LNfr43au84VC^NbN{T1tehN7Ze0GELb#%_lBg2t+~oOMX z)k|up)oIngqsBB|Yd*PK=68?qKBG;|~F!Mn{CkoIYnxR!ld~*3T8sM`6tu z`M%9B=dT*A9dA@@S#QJfRX?nM!gp)-oqn3<)!brNa7PB)Bp>6y5|UlBtkk$ok3Y z$Sui>!CqIRNT*n!l&4Il#8RnHJ)rtUZA;xnBSw=+^PAR#c8cy2T?f4)eKq0?qMSjJ zp_WmZv5(1w3CkSBLdjCZssi?Q7#Lx`;kd~u!a2p2&aKT&;%Vg#=TqS$A^Z5#1S|!G z1PMaD!g(UzqFQ3SV!y?wPqm&dkO)0vBzanjLTXj|txVO~JF*dSp7K@-x{BwOq|S++ z=U3)Y;Zo&RL#m6QBsG*YFJ3U$a=RF=ovD+hdq?lSe!-^;&ifk%1D}V`grEmtc4uUJ8F5N_nGry z4}U%WQMg_7qvY2!_;c2mrzUDmhHXvM~|V1 zk+?C;XOBs(X}X!WUqZgJ%ok&YaSy*yEV(UrtkSKSt>4}Fv<3gJxP2X;MtJpOd}n8u zX-{%rm+18C_V1j77l)lk6QuP&`X36;ekMQwocotRoHGU}22rC;2rEPrasyHfnE){! z6f^`{30;MW!dzj`VViI{I2!(jjE&5VteKpi+?BkOf}0{7+;gf^KBc0gx=r<++L`(b zjWx|Att~B%&X?|xJ_~UQF~H!-$jI2v+$i9C;(980Kif9@$o_N@$pd!7;Jn2fG(fE zox7X}lY?0Oqrnqupsy+F`Qg`QlHKkiT{3`P!zffofC8JwJwf|BwiQBl*; zP*KrRQ&Q3(XlUu^!HbH9fe}H^ctW2Pa*`d)MGl9P(^FAWok;wzUB_JjBPGNbN)3Yu z0#HT>j1h9&2OvRh$v_wsy#8u71PX(bkyC(eq5%aO7yu{)27|(3V0%Ga77_uz2jGlk zOaf;$$(c=@C>{fVeTzo=eQgTX8?wz}N_wGN)FD`jn z`mF5vi-ULX5O7uh>-w8!|E3oss23CthruaM^nyS`!31N3lL?$9XVNsGa0*}+l#QljxsY8{ z(@iBLXNqUN9ymhHCM=H?A)Ki8hi3noVln?)n*Cj|f9N#>(1Niplo7@Vpn#*p;uLY< z|Nm*RErI~+LDH(Tt`{f}CYn95#{k{hDi>}&@QVI9b8Sxdo2q2S`f6mrO@$>s72+B` z2z$4?iLl*QO%dkyllCZO!_=sOapT)5S;pHHIYTJHPhw_SDOCImiEy>sgHaeaL{uT2Z!U>3Gl7pyIZ36^Vth-d2qT1S z8^A^UfDY!c#1A=X@cDzy+C31)ctvEA*p8|nxB6`pb$3zHsNJ#6*?{dwRt-ev#j@_9 zHU+

C|Ad-m277mlZ$Ig|!q+l;K?zunm>Yzd_);XsvizB;W&I9&!wK~?nqC<`JPBGDH zOrek=as&hMg+%CH{>-6|D@4p@)M_e0e+Zl8RIo$=5Ei-2S7_LYb(u-5Ymo|gsb)_c zf^MDC=Ic6Chw_|D2>4Hx^%|v`HgHqkL>plFznoa{sMt;2t$+i&bw?)j#o~hnMDLu_ zrYg--Ic8rRohfs2gZ{mz*#nV!(eDbv8TtKPj;anrpC6)*ftU!i=6Jn;MA$vEhRlm2 zrb5YF%X`|5rilmdYUMJ6tIddJ_yYr?#eou00{b;1D^-m`KV|qW*B0|TcDc*C`%kml z66Zbw;dCq}8f2`?BqieGk}z(ZQZIt~>y+0_LEU0CgL-zU*0t=WEa&=y@3zTJEjX5p zFELg@e!-9{fog(L)I{$7{9wmg>8*g$!;AR!bh*7qV(iPx#PpxRZyNA>FQgf~Zal$> zKSc>Yandb1nsV&$9_l!QMvbYTCVebWB!-p?OL|s&%GpS~Yi77)CG&ANLn8+f%mP`J zxB7^wmCo2jBq8xeoryDf%`4K$!SAg`6rSE+)^4uNuDsuQa9PL-wLLj{2QblQ`M;Jd zY-cwll`g&R9QWI*@Og4!$IkAOP=*4p!Px~wCAykWw%0)n!k6`Bhe>P38WT$i*SEjD z!ebKHx0JMh!{nqaS%ypp+DzZ?-6F0X7jPNV$u<`s zAYNvu7*jk3M|gvD?|kYpz(<0k#fYKg<%@LC`9-6|!DO_FH-z*iiE1wLgQ%)XPG~?E zW1-j4_3DIUz-uVJMJxuhaDTEBUyhb2^$0jbeU?84nAc`P;$J6I);_k;5f17TwLB9T zX>%agJM;Y^E)!Ud6dieXa3#BZ4L?8i@fc7O$XCebCRmb3DT)`T0KZO8VR|sf09oF+ zUy_{1JXIvJ^r`zxIxE}z24aWwDh3=8gplo_iH1RuLY*s3_}XH3d3?}fxIPyFnnRGD;w*a4>Z7)aj3h?XtP zGl@-C7X@@raYHSJbnKQpN}oVe#Q#4|{RBkPPb3xb(vAZ`v8K$!M)IlacYpsf>*93X zz=003iB7M~6cnGT(d~tysh#dcld5X~VQjW`bgT??m}=gmB=Z)L#>u2}9V# zu5D;zJAu2O%|_nmthWo|L1@@fj=0tGO*L{dwC7R7fm!5aVJm*FqvSviP1Ve^-sFKu z+D%H{US%K^V;O*l8o*b_%-3c16h@JFa^|~8$TV+uv705;^G!1f+$*SCOOI;@)gScV zZeTGc<2AedYNAc<`RYt%h1HDxQAOTl-YD(U5xr7gM2)HD@5upV9a}2~Ux?F9ItE;- zdFDRNlqJjZ1_vB9KY6pc`qJoY-dt13^Hp*d-?aRLn{8r?on7r>i&oL~)e=^% z_|VKYRG9Rk2yRXd(mLv=gW#pPNQ5lDp^FNCE)V%WZq-ksIx6!&WX6@F5>O3S9#Hfm z_?|ON5nsH zaK<-+Q6j}DDYvkVb@|K79+g|T3qRfq#hV+;TIY5Ff%;FVym82xAcuBh9YLjczE(H7 z+TGr$&rMAW$5-k(w8t0EUEb1)XqD{s(KZ{Gp@qJoYF@<5+%_dNrEIHnM83<&-*Ys!`qi!RpSB$;G&Vaf)fo!^y;GP=7 ziuC?mb;U?bxI#aNQC8!vnsq*~_gMP!s}ZzZna7Mpg{886e8tU)n2f~VyHhZGg1B2n5sdko_dhK_|7%IH9_P9j^J+A)m{8C(2tlJGi)Zt2N6(v zXoB{9*H9{V+q9H?DL8B0UiLn2vg}>D7S0A!_((+UbrF<$GkK>E&mC2_@yyd6Y6~R= z9|N4zLI&6)yA?>=2&pu4-?sS)V))YO(u5-Cm#Y&q84qsxKDV~h5sq2@Y9bu?hX=A~ z0ZzQ?|Jl?LL7YA?M2iy}RnO_ZXK?h0HkFbay+Kx)*T*7>x_ z%Hn$e5uoOcD@>Vh^iTLHoy3HeBZ^@Uwlgs7kK1aVuIffSj>Uwpe?wO)j&FNR)K z;u-N~A7cJJ%dLX7*0x&YgQ)_2(I|B-{Ql;>#rd=j$vVR1R`Q}EA!v~Qp%F2p_}%cT z4dzB|RM3syX4L>k^lFP}CU%H+#-oM&fpdwcVolQ4<|fvwHyIqB%AZ;$^7VwWdM4&QD9#Oi zjE&B=^FQ1SI0k~tj{$G1`Ja$aMB5$*c04MkQuX}8He!{;G3=Yb|7uKWO<4dQ$N)g^Vd9{ zAU^Tx?u8N;4-~xbsf!)mQ8iRbX`^-g*g>?y-EPdGYuBrJ+?=#nT%cn08qIW6)xmpc zJ6YI0)8A2oAH=mkpM4&t()V`HyCI-kCa>#(+;C-ty#pPgde4t&L0IV>>0Pv^c|Ymf z;Vajt?ETR9Qum-vhHEcvIG;G|t4H-mk3*{iFDhp%3g?)6VcAp}n57t7jLfmcq+B`KGUeC9^2hd!hs;EWo4McORJ zkpI_-)nWbn#GqlpkG*9Y6+Pp&v5)*0({)Whgu%WHlcg-Gh7u6nDB2F>*RZm|4v|r` zbdjxqr&RTK__YOhYQ&j{1VeP_V(0AXoqj%4JJIvt6j9wjjii?RK`H;;vKob+&qDnq zFHiNpm4a9_%31$Vmq0y+bKf#rBXSGrNtV%U4#a}JIyADWH< z26Zv}oZp#gXnnDn>SsakeI&D)R*I|6IvUx`%wF{zp1C^P?deGkPvI~Y)G2ua+@NSM z`o|FZYbgEsiv%#A29f)hI`yXURGGjA$SWzOo%vse0lc4p?Fkojxdv zLa3MQCH}ECOzFVrDp%1e=OfJXpw%xflk+xiFP?4Kur|IZnV(+I_wAvGS)XvtInVM_ zAiBE(+@}z&dL!lW_S=c*tHquj0>j@IPX)L}K;Lu6Zr(oFt&VE&pDtSz+g!uZ46%{k zbe!@v==ms6XbjE(+q|3slh5&w%I@3pvX=sp877Xxha8}1CnG5J8!b4pO|3gpMM~ME zH+^L!$Q~73?~S<@6#qbf)qHYd{C_*@uo}X!qdff+9jR1xy|Gu1fyOpp!s%VjF*GaI zTQ1_|7dB4Qk!r?M8~6CLbBb#05(hD<1@gJLf=oPY!NBGtA=b8uJ^WdbP=SuzAa2C@^`xrPi^%=HNXgIEA75<9)zJPhkYMUI z2o$35*gt>nw@8ine%kuAsQZ_R%HXTHzsAh^FNKftFhK-vdefmbuDbu`z`4V zX#i87IZ#b+<5{43R8!?2Y(0zqO?EG;sHXPyd91L96yrCe{koskqxs^no&9bHwxgGw zZri}jDjx^Y8?S{MjH<{YtHI}@5_cn>pg&j|J7q0Cfw%Hx;M$J?EOP4RXE^^XYvA66TE7tfAcNiQK5mLEchKYSL5gT=Po z?MjwSn!IJH@^iD6KWMMrDo@kr0MeK!=-D@H{+p$_<1%rZ5Hyg3HX9)UXe#fk9WvfG zaogsfSCU;D@C6TAy}w>TkefZy<%Abd@jSUk--%n`Mz}-Mh;I*;(m9&6r{5``aTo zdhi&ZFfp8A7HY!y73FQuBwMsTgm^4CP~>7Xf%rlGu+5(C34F z&>^_?^D%Z|Ia}78^6yV+zKNq3zH=i`_p~``nvW7M7N^dAcX0v@yZ>%bAMLcCqJnvdK{c3tE+llwjf!F;IFgc4t+#kt>oy_n;@eOerb zyG4>BjUNM4z8?n%kTEmJz9H(apb(rIVD3ASg)=~FRn%Iut zUKHyk;EWO-L~96SF%0(t1WpCH3UK~diGM{vQ6#|5y?L|TF500(963-~w{lR*T{PS%2Y*xpM1CcN*?D{)GS!RczMlfC+6h2cBT z)#|=QUVTEBInCU~^(-4Ncz_$qU{c9UDP)n-O*+&Jwpxzi)Y(u`7?R*IuEnC_AKxoZ z`TgkuPe89fkIX>!#S~UXcIqcL0N9i2#I}e7`!H~zP!Y8HBH51aNAia?VH0~d-lzIA z5R3P18b-doW_#bV?oFc-y*$^;^(`5qor`P*=VYLXBTLxeUi$<;@fcWeOh8dZQV`2< zLn(*$3pDPzDdk?~enYL8V_}yUqVe?6BMp(}7D^XRkVp zI8uzhNECaC7A1CKjS@mhO2hK0TYhx5h7*;Ni`OnW*hY1@nBE_?uDLsRSL(I%QmqGu zJrZ)La`c#_YLC7}d_mBWCHd(~?)UTmwjk{owoKF&u=!Y}s8Tbhefa1+*IDd!imsr7kMz#zFB z&POTKfDSsKt8a1N%*0^;{WCdYQ5^!_1zd4@2LzBZQh)bv#|E%0 zV)UY#GkBLHRqm>r7 zXEp{&97ZX8Linc5UIMJYlU5z+U}qba*w5bbxhhkntI>Ya%=+pVgyMIONnt(eGxv|a zuD32CXZEdQ7mtDHNRC4hA`(}asxCnYjgoA0FU9mN&+6AV*5;eWeKnNv=NF&Rw+OuY z8JZKMf%G16gnM;1MP@!FeXzl#yAe?YYDaFdzBeR3?@q*G7^*>j%V}iJS!ZtPN*dgi z*IJ_DWuT09cxcbij8b2R#8GjGe{3g92fa6joODVXJ5SULPFGyCYx?XHavQ>*Jj4Bl zg+K2UFOy!%k^z*aiX$q*%uzIQ(gFIp!7(}y#db)|G>a?i`8C_8j%=pcF@k$Pe6RR@ z*jPdnfrHg%D!-c&ct!bw{#5Uf?*~?^MtrS5?7hE+FDbyhO{nYa>5xMWP+AS-#G#sOvyw=$_g7KG+bt8;>hw zm<}_*Rwt>NJ)d0n^M$krccm6^If3i}v)(yl(UQH5omgX}7DE9x(&PEZ-z* z+|6lnHIO$(91#d4TKA{|!Cvvblw8ml!O>XQHi|#$8ORR%JBezdspU(pySi zxue#cvKaT{n-u*5+v{T>$|j=*iy)o`U8jFEiHg`tKrSlvv?IDr9Ert7oZh-Ng}ur< zUn&Bw4n{71Q1_uZlL@QyaXBY+7vRP8ms=89!Cqn@h7jKC(s(XxCn*hwso+Ff&@~Oc zG4dOG$ABjX$Mn!e0K=)U&x(Ez`dlv*%Q4U`+aWU}d$yqWLWIN2#dBdUwXUu7WiPY8 z73f~q4d3%mAPHjFNuNkif=gBH`$}Tqph>8mQ`#Qko!%hFujjD(e{);^52v35L`El4 zl@3pVRS8S&zb8BAWhMR1qlGnVN4mi_wx^%gIY)nc*qB0>;Eh{}?x03;D+aD z=f55UH?@95wa4cF#$ctp9Qn|m#E)g9;m!tC#>OHTRU0FI=X@>x@ZyroV`cq79V)TX z2n)hK_KqX_G2o0$a-{n#cyOh%q{VYK$=u-eo4lz)ZDw6g_30;w>(zZjNu9NW1D(9{ z=&+H)(1Y`HXfI-JmA6eZ-m0&Y8EuBdXL{QB_B7Z3kY5Z~wjF}LN*7`~)E~;P*>NmG z)}i|42%&rD@MX#VX(M6JJ_$V~KATv$?(KVJobo%H#x2ti@F>0uZMUr4mLtAaw_Ke!Z&Rel;3E`KTCj6Ec`pyby zE|Elu$iqFL!e8=Q6Uoabi_07K9Q_KI%)f2w%jKVK zertR~Vs|P`rDqYh6L#ofbAwRg3Dfg+xu`zs6W4L0N%Ck&=H6f>xD15GA3FSog%w^H zYiJx%2n&hepX=k9{@TN#9e~`o*J?z7fMc`*B6gk=++0ehgSddI6(JPda36}JsG52r zthyq3M#Uz5L+nl0eT^?nD*b3LS3TR$ne43h9{dkI{7^C%>oBFg3Sx6HX~>bH<1|`m zl!PR@W4Duox10w$&Ka@TILr`J5BOs}dE@JKKS7tZGpu%UP~xX}9#J1Dftxtu(4N>a zU_{g+l=P46xZ*m~z6wnEOhrs2FK6(Z?{mgC|DJpvIfi6KVU0L9mDE1hrsqXDe6uEBOj?_yJfpL=Ae81q@ zd`;KbE$4z-jRs%$zac^t1CEW|T$mo|n6FZI;%o8M0~mxiyU!Q#bdfw+>pjjpL5^;d znP?2rt&jubY?Oqdzd0}&K9IFh_JE$%Z&A?OP3+Ylqp!kox)Aq(*m}SdX4cm<;L&F0 z&NzxT^S*Ibp|Q*h9!kTn`oayWfe>)9+tEPY2Tz4Ct{}vt(m^DY5V!Y&T8N(sVs;Y^ zK?~K%k)p(BC@kM-{cL=Y=Qq!R#ScvdWKLWu7UZPza$;wvIpK1jC&P++&pk&?cOgiMnV%R&P zzKRXkZ~4|6I`BH`DFsj{hQ3N*w9`GBG~SB+?p^u>@eqqgq?(ba*DwUl`X{P%(mi^? z!s8vE*_GbEG`KPR%eHN3*Qc`6?`yy#f}@^^1I58*v^lsKY8~(rtNU*1c%m(CBle8q zEBTjiS6*2;Yg^%U{(1$vheE~KCnyE@aSTiq9-**^flPMYkM~kYa$$*A>zwv&#PZKY zn|*O(TiI}Qv&sa)0gplo&?`A6V(2gRHHE8ZbFGuhQa5~wdmq@9@#Ld$-0kb_G5;E~ z5Z~JhX@gxyBnp{6ibf`@(>i$UwZ0}d@YS<_dj zxnn$%mH61dL9a5w^`1_viSKhkglU3YL9`Zgmpo*9o=e_{H|*Y?9x)f3#|g@##84bx z3AXjUKmEF|iq@X(V0@k!lenlXN7^~~i(dM>za+6k(->ST+7Jv`U5iO?*gI$mnenOt zC0-pU0w$GoTC>VK{AH2H_w;h^7c5<^GuU2v*ZblbqUWn;e8nur8s@3j@|QDj(HA{k zSInN9w!aXV+bfu;YxYGpOLwAP9QK15TbSDExxtItYsWkGR7bb?_o<0F5C?GvMhIeo zKkL;g+to>C10E|x1Z{;#m^0mSbq?>B@@MQz&Wg#!XxpPln1!bpDpeZ%WS0b?T~+cx z!E;6_*{4$Bwy*5EYpPi`*~dD2*#W&fOwR6hB}yuHHIIz(^RaEx3qL2>dw#P$E|RJ% zA1Ysu8Rd`pz86S4iCF(`?y}zx>;B4)WPdogcwsCtom-Df%Ue)jFV{t6C%)PL`>nVR zH4oU2?fy;##!r%FPmuVa3VjZ*P+B-BomR+rr4E-ara`bsVOO{9StOg|>EkqTdDd6Z_h7f!rQ_h@n_tk^LnKQw7u!B@XP=40?H z{=BQDWK$moy^?wW_UoA)&%u)}aFTs=qoW~F)e#1MW3!#_b>G|5XzrQ|qOq-BY%$3U z-y~75juC_Ql!yk{jna^o(L>JC=?^?kvD)rTJehVcsvlvzX-bKA31V3G&Cb&At*;4@ z?c{YhV%Cyxy#y6}^jATF&-0Wxyp61Kwxch=ssVnQ2k<_Jc>G5vOdWbBn1tt)I=fek zd#`_@vI<&@xQ#>3t*Uc3{K}hGjVNAA{Qdl1wOW7%FLI0VK%eNmpc}X8<6Qr+AH-in zntrHCE*zm&+Qy%$BJM;?b*~RbNr2}Nksk|0$0N^FZ&4f=*mvWG((ZY->Ww0XoUrGN z1ZhH6U0?~3%~iR3VMq6u@nR{vh}1&%hInE;VcXvRcV5j6rS8%fE1Qj8n9l?4vNBgB zHZT1F$`N&OAsNjFQlzn#$t@UGQuM%%h{C-UM6#e!PMo*uXR~fr)%kO92*H!@rR7-X zww^sI!DccOF$0m1(Ka)zy~Cgbo>b~rH#Kjg-*G-W5gyLh$HSlrhf-EHUCd+8xOi14 z(`sBDg(VSd2LP`g0@?F`Cbx&I|E5pVhJ8lg^+u zzAXBVNeXaannG+qd83%Z&Q#tT^q*eA4fP(>bL3hMSKM~^;U@EL7}Yrj6I zf!})wDcB?ZFwE|Y9p4vwVv?f|K=oLE*pen9g$YI8($S7}L;ZzG?0#qYDyycYqDutS zxVP9Tq?=x7Jjov3=?jgnEL8qJFF)B7}Ds; z1snpRy3$LZ9(3dqHO5LOyi*3;6S(T+X3H0NH^QJ;$r5~8R4sg`t(KZX*{Xxl(=XgD|*wt{s} z+cn~f1d;7AG>`u?C9dk;LI$Ca*hMIrMD4&KZ?1@gMe47sW(@I6*hDjdg*}s$X4V;!H3C;rWVl+kablnnzE~C@`Fi zQ#cC{j&@{6OV>_zdLd^pI!8sSwx#P_i0)&6S~V5l)Z0Y3^U&nA|J~h(aYC@@B&$c) zWQ06D6@wk8TL+OWhu+V^Cih-}gL+Z4=8@eRiawYutWBXzKlAQ0%U|2VcE`Z`%g=u$ z*k6&@x_B{#B}3kNQJeZ@2}ENR`1yq5-@m=GkTkgRYC=^F^K`xO=I?|D5k7m9d38_1 zR+7idOHM76pW@G0_&g&HTaZqSJk@%XO_IBMNKVYY6(ZK1?zTINO-%bfmoJI>5I8^0 zUdtHg>ZWcNxA1}&(uawJa%w>-h4?deKM>z{g9~Roe%2hHm>w*FU?lpRq`q%^)n@o+ z@jRR{rh<~}s-7!9gC;QfMsc1V<0(BoSJFgOD4#jJNi?B48j4Z1f1c{=ZfNvt$^AKF z%_0DlD^bg^>c2Op_)>2*Km~@f(i$JD`wJ5=TUpO(sJVQR*M6xrlRwUa%jaB<4Z}VtUzl z!|i7Ek59sG_O5hJ_+?VT#bu3z{d^dn?2DKaUBFMgFdot=ipzrwj+@6oRoInx3tNsH zXmg>R>r|B2_?$BqZ| z;XWL)jQfvhze}`tJpWHtjgkg5gm7;!o>1MFnKoHiH@I2rG~VV`_KpymUehtyQ^TCl z8+<#>om!a-SW{Ogjs9LL?0Zgvk%p9m)s=`VFGOdP@2%9}Kj_8=-RzCh_$Fp%Kfk$g zAn$>J|E$#Q3;$ZGl+hu*vGR$ao4QT9_5G>pecaIR*P09BTlQjI$mx}CM?jtH<1-Jz z2={)jJB1zu_T*he!GYCl)%u*~70%UmvfE4lipt_n#k)DdvCvH|Gcc{7ont zatx^OVCF#JVCM7w(Z4Vb8a+J3V7Tth)E>QnBa z*Iqse)E9eF9W9SwAPpDRpM0?s)`_JUn2MxBbBuua+c`L)WwZlRKs& z`vIjne^z%u!&O^l^)pBL7%yq=L&b?U5uDMisqGwvMT58ZsvyzeS=Xq}oJQEPmE+4S z4_F#8nUFhp3^e?YcD^&HskU7=h=58{RFE2!rh+fpQ78;xLS3Z#QSBD)A&Q0OXj8^+&;NH?SEjqf5Wzvcj+N=(;FrZg#pFnvB zve9Q-erlwbKE~g>iTGL7FW5(5MQuUsKuIt^1hEBSJ`&=3?_WhOZE#Geo{SBOUJOCe z=ws+gYSI1%9=4!Lw{U-IINmmYUh!S3s+zMc%>5Pn`K!5nbC;2S?+wLxbCA^OeF9a% z_y^w>$Cx71IQr&%QkiiLNUttf5x=-NAa+-lrnbXB4t;dYZ$H|4@4=YMB_S^AAKvpg z2;j3ft8t4Pl7asJFK7O2e~@Jv)&_t2X`o+V0F9Qe{Smuw6y!oMQLsO!7OEzmZ4v z8shn)Nky?=4}mVrWga6I-MTQwN zt-Z}HEY4xQZTc*bLEngIGAf7LTzO22GE*EncVw2n&xbT*hq!@I3(YdQ(A|CKfJ!{{ zY&*v;fp5-4LIOy#93Q-AH4t?vrZwtDgbFZc{bSMYLi_gJJ5*x=8Q}A7d{|j(=AHK= z)GnkMp{$VrD1krX>vAM{c?Av)$j9q}Nm|>l&&x8^S&(JQb!gZ>chmpKZx<@A4xQd~ zx}uei9&e;~47YOjGTcu!vz7uJxthuEd8i)`!n;K?PIoRfTyD_CJ_83rL%nSn-BG?b zBWiZI30~sb6%l=Vaw%h9_WdcmrI`8yFg)qHnbV|E1r%K3__9uMq`K`L75TPDY0B|b zE$e~Z^L^t{!iN%aK6WlWQ#Jk$`5o7)A_0Q1c5=LjG8>){l;_A6q}|NTy}&;I1saho z98%L`5ay%R9O}fJR>bN^#}UNeNy{ihKw`<_U6IxSLXilttp zwvSpk()%Ud+x3z|r_}zql4~tCacfA;Bu^2c*+P*!n)=CVHfHCdHbYMuJ8_a*sujx* zV@Suq4ztUa=uG7RW$kv{o~lq@#4Y72Z`YvZTkkrMCuk!PiUHkTH&2x2SidZmTH(HC z!;?{W2@===f@Bbl$p~vlxUJLdT1Vc+N>2nKmFh2tl+VU09iD6%9a0hEkdp-Je0laL z-r`s&w^#V;8~YwJDdEF6x=pv!zr0Ovetd>?&0pwIA|zJ-^a^q(<~Vg}kcauc1rfeQ zbmOZ7X*#r(_6Ws8$|e_-zJ5FFDC8((TLRfuGVg0Bv=REhEIcZ3&NO4bfNjt99p#5b z+@j@{$W&G(m3Eds|6I2d>mC*A^eb~=migcnML2bq?Bf_8nb$TQ6M_uyyjae6=0yMa z@zwORYhr|+k4Do2JH>;}V-WL@rgYJca6v4bUIw2j8HMR*o8UySQXb8jG4DSqj3y_m zL=P-=1vNT3!i`5RC*D8?{dkf;h}r)-P;>7c9UbrY$HgY*Bm9uJZU;a0G%c1qtKEr} zi%wJG^qhCziusgdR%pi*s zk`_8jC{Kjg(FL^mV!sM7H?yH4_W8O1~Bi-xg_+{K;m z3K6E+=Slc^;(IGI5u|+U_Ltrtd&4)V$O&zFQY$)@BnpH{#DUC=jDp$cas*_8#b z?RB5KGpmJ{tZpN&rRs|-@M1^4(|G5I&9y~}752%wu9_+B>dEa#yY05`J}nxf^-m&z zFqH)7pL2|xt|_CylP5~#B4H0$WhtUgRkMc_z112F5C>r6RgN>#_YDc-Jg&sh4bHu? zz~^6E%X%14<-o|w?#{osf3B@)HRL1&yKYQ7DvZ@6Yn*J?%X=<_ z+otW_=NxsicDFZtvQC}Yd{dftZL`qAfF0RwWh9+ibbPq!@b{#7p76(8p_;$N6$76U zxCxNx%l=`3xKg+2u}SMN?+}?@w1uv`hOJP}qs$%PF!r-z|F4-Mt<9GtBa8>Y#_r@J z*jjCb9ee|hQjsisJ|Tkc`qP#pxmnrVG_2V5)e^-T>pp~37rhXsa#_lTiKo>vkt}-# zE&;ZmLyX;m`1atFNP7ntI8`Nq^pAWS4etALTGO;|{o0hLyJ>x_EB<&*I{j@i^x03P zVHzHP#6mhG_O2;jLX^h$gBN)pTf-pNAJtJ6d6 z24)+FUdS3}_o1J~d{n~Ds<&>Bk-IAM>TDeK37Mr0PF)7vK|q8_$N=4^*it$&(OGbC z7!yz=8ap(<+POS^LC)ELMpaF?mOJ>+=lo^tW@X!9vBhI~)Rp^gd2Jbv)Lf`MY5x?U zcD`BGMQ|{!!x>dZo80}}T7&v{uHr$rRc@EW@sWmVe9=y@Df%JDOxztse<)xZBS$>1 zvTYmxnksFJCQZttN{LTS{F!v!bMiU=t+82ktU!pbG^~pYz>mAMK=&SGqw(*Ksm-AJ z)89?OrrYk(ZasYxzxHip(9!6xT*UHCBFwb`m;Xk0+ur*>p-UAd{v?sj8eEWG0!b!u z^c)cJ44~S7Hgr(v-mCl~=a?W*V&j5U@lu9F3}UWW=v)%SCk7S^8h4Jt^s${}DS?`7 zO4ZMdL&l%CUl(z2=t`ZlRE9h;gK#^dL@>U*y$QpD-C#j@WN`&>XPPA6jdBv+Rr7rC zbLoY zUV+Xp_lW8S6sdo+P;87(cpDKo8(}(L3)P(rPWL?UT2BsewBk&+v2d*+#w9?c)Bnb|6oG89M`@>>Lz)`mm^Y)5~2+6~c>;u{U z*M&^`MM>8Hd3LrfXQ6e4#^YWCM$tjjb@eqG9Sf;V$Z(b+NwYqusaMUpHtb^k&=h6I z8{2JGgNF~wbMYg+=ysY|%H47ujNP+Efoe_Ss^Uk~Pi@m>wM>S*tgDaHZObeZr_dYS zZ)10;895_&grCQy%92b7Z35+j&fF2n8>nhh!QOz~v|4oF-3IM>9UIck`}Ns)h zHzqTxC$y<@UyqC$x-(y4lmPQ+x_g>pPl(DEMDe@v>O_FZPIE-2V(7W_``ju?JBDNG z7DS2QZvs7px(%b~jrs6{T!{ zYW*CxW)l-F^nbU92evEiy)wFam-lNQC zHrlSKF&$PY8CrCF`=$Y7KRqOj5r~wYuz7jzDrBx||MvJd5E=KPb)m)f7!k}3-n~Ky zmNFR82JruYsnvi`E_ky5Ps~8~sn|VSA%29jExXmR4R8rY*$n+GLCOAC2ngIW`!reDL z9zHdHZw>Q0WZ{>GZ{8Nf!Pw>PMN6nTwriSQ4X1xaih~BfAn>Di=P8^TrxtQa*e?s7 z0`~}k&+_=URYo(8id_8zbuiXGJW%f$pwYM$J;x*RwFygA zcg+ET(KTGF(MTsd+>OU*;av4{ht}%dKTzmlk7|#BO;-BG{Ybw__NXE$@K2K1Vo6O$ zqGqH;Ji^^ApX^*erVuSPwe+s*Sf^@K_}QR@5~YKdySN>J1o80e{If^}sFa^pr*@gL zQRr}42bbgLwI=z7fPOr?@{8X*5f+j@Al5~w&wTu1OwjvPyz%YuQl)McNe|Gu<{+^P zZ|V>*csWWamsTuP6n7u#Hv%KlS;CYLS7l*p1M9fmS58Mr&Km#Dc@(yX8Y~fe=6~<0 z;QzU&YAlt|6zdr}TdvcNvHLB0TZX~{FWdw7%DBiqqOoyva(%&+YFb@LzA+7)1KHpy z0q8S`uV&%KaRH!fAs)`!f^c}7DdbIQmNJnwMMSH(ekE`<4GfNyVL+gK>0_-xPhlUWv9kOhw~ z)qDdLz0E4D9L9-2(RX|X!0kQ{pBpTx`&#=Isk5hYrH~BO_~4*WZcTW1eDZkQIL4Cg z^FNGUX&A9eFaJnEHjHqaerGV{Le)=DiafBef^daV?6grv>l@VECjz4!ayb!Sq$+Vc!vBS5@7d|AT}26 zOELbvkVxlQ?__7P!hE zZqztT)XVT;%E|glpSP1h$;Hn)%cv@4?uSaUiBOEwZCU*W^}MQ{4svZd$FT!W6Qi_u zinj321Hq`7HS}iAJkC4f2$m2qHIFxkqx$pWsd*cHD|z2i4-13NE_p!@ZEm@cZBBtp zVvjnq*FbGSPr543n}IOgN3I>wQ8?gPL|gsXUBC?4_Oap7fw8QerfOtBob7-W3m5B@ z1BY}_vZ27C;ni;0NKRQ=ouXH*+^D8udQbwwJ}R7<;iM*E6%(=kTan8Cd8Wz+k=ePGP_O$~!`6}_r2J0QsIaia~!J6-nlOl-$_AQaR=ktpF_5HcaE|RJ#+g8xr z{8;MblOK!!%G@HH%OPZ1KtVh8+)5fIf)OpD zE6ef~Rv36V15NeViId@*a`KWaLg?!s0$o1ewFj1q_a449xMF2HBacvFG}Uq|08sIj zG~!7C#F>7u8nffKAgEqnEiRSoJJE!%o$3h2YxSnxThT^OADAi%j_boxk+oX<6TVA{ zVDBQ+P0-NgQQc5D%lbLam)Yk!+OsEOXnl39kiSq|EbYJ{gKSzXR1&7@1~@XLJMfTj zJ<6ytXVU#!WFJwo?o-W~rQ{!qZ#1qve57>e&)+VUUFyN^O+z23GDm#xnL>oJ9I(EB z2awJafB@(Op^cybrw5zdC)eCf)0SQeraKblNxh+`z##_C3Pw#WpgP+#-gotheRG;4 z($)hj90bThaXYg7M8lq^SVN&XvNY)we;-i8h?|=GN}Ka;hWY+2o7NkeH)tJW+V$H1{L9` zx3|Sit%i=6O(Y#P*-ssNG~yIKuIx0JDDUdE=bNPxHP*rxe3)c#P%vlWcYUpTM&AtR z&|P}LGqDwTG+I!GdxRJ4#;n zg8#~2CkRaJ_`3bH4MBR+7Q{d$8?+NHZd+kpVG?#veuIIwagxH<0(1(fkNO1a)ls_! z-{pw0i2a!^dlGFH5xRdapGKYm#9CFL0E+F|?jg*ss~%gIlxo#kVw&o-+Ud>;<=yM4 z{hKYptnp6qPjJS8W{x;>y0ceGS&UJZTE-JCcve-ie>FVBHlCsQNA97EkH;c1yKhsp zZ@7LK{Xgud5OZ2tu;&|D5$A)#wGi`;z4(6O)s$c4^B`vjlM89{#0F<%J_;*kcJz~w ze3c{D3ut_axdezA;nNQ(gmoZyc#z_l8D=rgnZIRR`XIq?uQ#>|-wM0`;C*#TAQPJU z6#|-idEEYC7!0$WZ9}GrH-|Z#gj1wT|HYeSrVt3R^;bT(U61KwSx6X7e$aq!M<*v$AP6v)|nA z5WWVOyv5XW7sb?TnWkO4|g}I7v0O^(jQ!2{d%?{Q>3}MZN}lm_7Xtmu(8f(7@nH z^|VOF=cdXx>zUgR>6gq&HdP62BW3*y(L{TejEC;v2SPiP)xhH2a^GLNQ(0!rEg1-{hiy`3kouk!8a%1)R< zN86{$Rauml;}^!aSsD{Ob5y30FwE7>CxX9Ido0XFIF}j=Qy&-FEMTI=uN5dePnNeO zKE#a_f2(V4`FQDlZ$zv=v>qhrvg&CrDX_iRfs@-k17M9P&7{{e(Ssk9S|&q%dSnbH zLDl(BmrfkJfgI!rfE_u}co;Ugft>=^H}E_UhHIYt6~$qVJd0N2P6L^46HXkhHuN~p z@&PJrpz`^s(Nk!Xi9@nnW_rW*^HIAXTxbRo%9mu@w{aKTa+w~S+~=5TDZm$vc!IYlIl`4h+bj@2kVpgqbb11g7SD6xYEVL41Jlr#qOK*_)ok z8_qsf-@c$Oyk>?SrbCxT*(-ESawsDi3=g3>n(JVANHP4y(sTe(B=LYwhTx^NL%FD} zuxw}G;>rI@g#LdU!u`iM>VMz+zvg<(2vqqOfnat?TYcmLrC$kfZ*eo48hcmYJQ{j{ zbujnlYle}psw^!tcyw2^c3>M;mrc$TK>bNNY-F+_ejng@!8Qs!X5hmWR+ZD0hp8vC z4v0D{sT7a)!0V|?poc>;1vDA>(l_`z(trzcTt&md7ew%5IGV@3iKt|YYQ6rXTbRuW zBD66*sWGa zf?Z#l5R6mK?bZW^JqX>O%C=nZRB>0&Oa1DZlSg00A}k=#uETXdGSE5BaqD|^N>7Y zflR-o$Ko^%=gqSlhYbsN%>Qy9#!ih-ybnqD3N01^!Z=BEmyR9>XIuhkoV0*)wn-jm zI#@Y~Ii+$?Zk8Rw`CT+uZO>D;1BbNfj$={WqhJ*>04=|gP=TyF<_x$xMZZTDK+qA7 o Date: Wed, 4 Sep 2013 23:36:14 -0400 Subject: [PATCH 010/247] event.go: remove extra logic in addEvent() --- file_system/event.go | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/file_system/event.go b/file_system/event.go index 26dc5f61fdd..9faee369d63 100644 --- a/file_system/event.go +++ b/file_system/event.go @@ -53,17 +53,15 @@ type eventQueue struct { capacity int } -func (eq *eventQueue) insert(e *Event) bool { +func (eq *eventQueue) insert(e *Event) { eq.back = (eq.back + 1) % eq.capacity eq.events[eq.back] = e if eq.size == eq.capacity { //dequeue eq.front = (eq.back + 1) % eq.capacity - return true } else { eq.size++ - return false } } @@ -89,11 +87,9 @@ func (eh *EventHistory) addEvent(e *Event) { eh.rwl.Lock() defer eh.rwl.Unlock() - if eh.Queue.insert(e) { - eh.StartIndex++ - } else { - eh.StartIndex = eh.Queue.events[eh.Queue.front].Index - } + eh.Queue.insert(e) + + eh.StartIndex = eh.Queue.events[eh.Queue.front].Index } func (eh *EventHistory) scan(prefix string, index uint64) (*Event, error) { From cc77613fd952bcc08a8454267a83dd5a5a83ede5 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Wed, 4 Sep 2013 23:57:19 -0400 Subject: [PATCH 011/247] update file --- file_system/file_system.go | 30 ++++++++++++++++--------- file_system/file_system_test.go | 26 +++++++++++++++++++++ file_system/node.go | 40 ++++++++++++++++----------------- 3 files changed, 64 insertions(+), 32 deletions(-) diff --git a/file_system/file_system.go b/file_system/file_system.go index e834d2d38da..b7486642661 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -75,23 +75,31 @@ func (fs *FileSystem) Set(key_path string, value string, expireTime time.Time, i return nil, err } - f := newFile(key_path, value, fs.Index, fs.Term, d, "", expireTime) e := newEvent(Set, key_path, fs.Index, fs.Term) - e.Value = f.Value + e.Value = value - // remove previous file if exist - oldFile, err := d.GetFile(name) + f, err := d.GetFile(name) if err == nil { - if oldFile != nil { - oldFile.Remove(false) - e.PrevValue = oldFile.Value + + if f != nil { // update previous file if exist + e.PrevValue = f.Value + f.Write(e.Value) + + // if the previous ExpireTime is not Permanent and expireTime is given + // we stop the previous expire routine + if f.ExpireTime != Permanent && expireTime != Permanent { + f.stopExpire <- true + } + } else { // create new file + + f = newFile(key_path, value, fs.Index, fs.Term, d, "", expireTime) + + err = d.Add(f) + } - } else { - return nil, err - } - err = d.Add(f) + } if err != nil { return nil, err diff --git a/file_system/file_system_test.go b/file_system/file_system_test.go index e43750d4ef2..4d41f198405 100644 --- a/file_system/file_system_test.go +++ b/file_system/file_system_test.go @@ -12,6 +12,32 @@ func TestSetAndGet(t *testing.T) { setAndGet(fs, "/foo/foo/bar", t) } +func TestUpdateFile(t *testing.T) { + fs := New() + + _, err := fs.Set("/foo/bar", "bar", Permanent, 1, 1) + + if err != nil { + t.Fatalf("cannot set %s=bar [%s]", "/foo/bar", err.Error()) + } + + _, err = fs.Set("/foo/bar", "barbar", Permanent, 2, 1) + + if err != nil { + t.Fatalf("cannot set %s=barbar [%s]", "/foo/bar", err.Error()) + } + + e, err := fs.Get("/foo/bar", false, 2, 1) + + if err != nil { + t.Fatalf("cannot get %s [%s]", "/foo/bar", err.Error()) + } + + if e.Value != "barbar" { + t.Fatalf("expect value of %s is barbar [%s]", "/foo/bar", e.Value) + } +} + func TestListDirectory(t *testing.T) { fs := New() diff --git a/file_system/node.go b/file_system/node.go index 0fbd8d1bd27..771d8108c58 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -29,7 +29,7 @@ type Node struct { Children map[string]*Node // for directory status int mu sync.Mutex - removeChan chan bool // remove channel + stopExpire chan bool // stop expire routine channel } func newFile(key_path string, value string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { @@ -39,7 +39,7 @@ func newFile(key_path string, value string, createIndex uint64, createTerm uint6 CreateTerm: createTerm, Parent: parent, ACL: ACL, - removeChan: make(chan bool, 1), + stopExpire: make(chan bool, 1), ExpireTime: expireTime, Value: value, } @@ -52,7 +52,7 @@ func newDir(key_path string, createIndex uint64, createTerm uint64, parent *Node CreateTerm: createTerm, Parent: parent, ACL: ACL, - removeChan: make(chan bool, 1), + stopExpire: make(chan bool, 1), Children: make(map[string]*Node), } } @@ -75,7 +75,7 @@ func (n *Node) Remove(recursive bool) error { // This is the only pointer to Node object // Handled by garbage collector delete(n.Parent.Children, name) - n.removeChan <- true + n.stopExpire <- true n.status = removed } @@ -94,7 +94,7 @@ func (n *Node) Remove(recursive bool) error { _, name := path.Split(n.Path) if n.Parent.Children[name] == n { delete(n.Parent.Children, name) - n.removeChan <- true + n.stopExpire <- true n.status = removed } @@ -226,25 +226,23 @@ func (n *Node) IsDir() bool { } func (n *Node) Expire() { - for { - duration := n.ExpireTime.Sub(time.Now()) - if duration <= 0 { - n.Remove(true) - return - } + duration := n.ExpireTime.Sub(time.Now()) + if duration <= 0 { + n.Remove(true) + return + } - select { - // if timeout, delete the node - case <-time.After(duration): - n.Remove(true) - return + select { + // if timeout, delete the node + case <-time.After(duration): + n.Remove(true) + return - // if removed, return - case <-n.removeChan: - fmt.Println("node removed") - return + // if stopped, return + case <-n.stopExpire: + fmt.Println("expire stopped") + return - } } } From 4c286fde2349ca915fb561163dc86ffe7c18693d Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Thu, 5 Sep 2013 11:00:53 -0400 Subject: [PATCH 012/247] change key_path to keyPath --- file_system/file_system.go | 40 +++++++++++++++++++------------------- file_system/node.go | 8 ++++---- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/file_system/file_system.go b/file_system/file_system.go index b7486642661..3a9f179b512 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -24,15 +24,15 @@ func New() *FileSystem { } -func (fs *FileSystem) Get(key_path string, recusive bool, index uint64, term uint64) (*Event, error) { +func (fs *FileSystem) Get(keyPath string, recusive bool, index uint64, term uint64) (*Event, error) { // TODO: add recursive get - n, err := fs.InternalGet(key_path, index, term) + n, err := fs.InternalGet(keyPath, index, term) if err != nil { return nil, err } - e := newEvent(Get, key_path, index, term) + e := newEvent(Get, keyPath, index, term) if n.IsDir() { // node is dir e.KVPairs = make([]KeyValuePair, len(n.Children)) @@ -60,22 +60,22 @@ func (fs *FileSystem) Get(key_path string, recusive bool, index uint64, term uin return e, nil } -func (fs *FileSystem) Set(key_path string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { - key_path = path.Clean("/" + key_path) +func (fs *FileSystem) Set(keyPath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { + keyPath = path.Clean("/" + keyPath) // update file system known index and term fs.Index, fs.Term = index, term - dir, name := path.Split(key_path) + dir, name := path.Split(keyPath) - // walk through the key_path and get the last directory node + // walk through the keyPath and get the last directory node d, err := fs.walk(dir, fs.checkDir) if err != nil { return nil, err } - e := newEvent(Set, key_path, fs.Index, fs.Term) + e := newEvent(Set, keyPath, fs.Index, fs.Term) e.Value = value f, err := d.GetFile(name) @@ -93,7 +93,7 @@ func (fs *FileSystem) Set(key_path string, value string, expireTime time.Time, i } } else { // create new file - f = newFile(key_path, value, fs.Index, fs.Term, d, "", expireTime) + f = newFile(keyPath, value, fs.Index, fs.Term, d, "", expireTime) err = d.Add(f) @@ -115,7 +115,7 @@ func (fs *FileSystem) Set(key_path string, value string, expireTime time.Time, i return e, nil } -func (fs *FileSystem) TestAndSet(key_path string, recurisive bool, index uint64, term uint64) { +func (fs *FileSystem) TestAndSet(keyPath string, prevValue string, prevIndex uint64, index uint64, term uint64) { } @@ -123,8 +123,8 @@ func (fs *FileSystem) TestIndexAndSet() { } -func (fs *FileSystem) Delete(key_path string, recurisive bool, index uint64, term uint64) (*Event, error) { - n, err := fs.InternalGet(key_path, index, term) +func (fs *FileSystem) Delete(keyPath string, recurisive bool, index uint64, term uint64) (*Event, error) { + n, err := fs.InternalGet(keyPath, index, term) if err != nil { return nil, err @@ -136,7 +136,7 @@ func (fs *FileSystem) Delete(key_path string, recurisive bool, index uint64, ter return nil, err } - e := newEvent(Delete, key_path, index, term) + e := newEvent(Delete, keyPath, index, term) if n.IsDir() { e.Dir = true @@ -147,9 +147,9 @@ func (fs *FileSystem) Delete(key_path string, recurisive bool, index uint64, ter return e, nil } -// walk function walks all the key_path and apply the walkFunc on each directory -func (fs *FileSystem) walk(key_path string, walkFunc func(prev *Node, component string) (*Node, error)) (*Node, error) { - components := strings.Split(key_path, "/") +// walk function walks all the keyPath and apply the walkFunc on each directory +func (fs *FileSystem) walk(keyPath string, walkFunc func(prev *Node, component string) (*Node, error)) (*Node, error) { + components := strings.Split(keyPath, "/") curr := fs.Root @@ -169,9 +169,9 @@ func (fs *FileSystem) walk(key_path string, walkFunc func(prev *Node, component return curr, nil } -// InternalGet function get the node of the given key_path. -func (fs *FileSystem) InternalGet(key_path string, index uint64, term uint64) (*Node, error) { - key_path = path.Clean("/" + key_path) +// InternalGet function get the node of the given keyPath. +func (fs *FileSystem) InternalGet(keyPath string, index uint64, term uint64) (*Node, error) { + keyPath = path.Clean("/" + keyPath) // update file system known index and term fs.Index, fs.Term = index, term @@ -185,7 +185,7 @@ func (fs *FileSystem) InternalGet(key_path string, index uint64, term uint64) (* return nil, etcdErr.NewError(100, "get") } - f, err := fs.walk(key_path, walkFunc) + f, err := fs.walk(keyPath, walkFunc) if err != nil { return nil, err diff --git a/file_system/node.go b/file_system/node.go index 771d8108c58..7c9c969c404 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -32,9 +32,9 @@ type Node struct { stopExpire chan bool // stop expire routine channel } -func newFile(key_path string, value string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { +func newFile(keyPath string, value string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { return &Node{ - Path: key_path, + Path: keyPath, CreateIndex: createIndex, CreateTerm: createTerm, Parent: parent, @@ -45,9 +45,9 @@ func newFile(key_path string, value string, createIndex uint64, createTerm uint6 } } -func newDir(key_path string, createIndex uint64, createTerm uint64, parent *Node, ACL string) *Node { +func newDir(keyPath string, createIndex uint64, createTerm uint64, parent *Node, ACL string) *Node { return &Node{ - Path: key_path, + Path: keyPath, CreateIndex: createIndex, CreateTerm: createTerm, Parent: parent, From 227d79e2bff52fe4c9e7091341fad72f63e12274 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Thu, 5 Sep 2013 15:38:22 -0400 Subject: [PATCH 013/247] support testandset --- error/error.go | 2 +- file_system/event.go | 9 +++--- file_system/file_system.go | 51 ++++++++++++++++++++++++++++----- file_system/file_system_test.go | 34 ++++++++++++++++++++++ file_system/node.go | 46 ++++++++++++++++------------- 5 files changed, 109 insertions(+), 33 deletions(-) diff --git a/error/error.go b/error/error.go index 498f6e33e3f..baa395fb9cc 100644 --- a/error/error.go +++ b/error/error.go @@ -14,7 +14,7 @@ func init() { // command related errors errors[100] = "Key Not Found" - errors[101] = "The given PrevValue is not equal to the value of the key" + errors[101] = "Test Failed" errors[102] = "Not A File" errors[103] = "Reached the max number of machines in the cluster" errors[104] = "Not A Directory" diff --git a/file_system/event.go b/file_system/event.go index 9faee369d63..d539ed9dae3 100644 --- a/file_system/event.go +++ b/file_system/event.go @@ -7,11 +7,10 @@ import ( ) const ( - Get = "get" - Set = "set" - Delete = "delete" - TestAndSet = "testAndSet" - TestIAndSet = "testiAndSet" + Get = "get" + Set = "set" + Delete = "delete" + TestAndSet = "testAndSet" ) type Event struct { diff --git a/file_system/file_system.go b/file_system/file_system.go index 3a9f179b512..462745a90c7 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -1,6 +1,7 @@ package fileSystem import ( + "fmt" "path" "strings" "time" @@ -84,7 +85,7 @@ func (fs *FileSystem) Set(keyPath string, value string, expireTime time.Time, in if f != nil { // update previous file if exist e.PrevValue = f.Value - f.Write(e.Value) + f.Write(e.Value, index, term) // if the previous ExpireTime is not Permanent and expireTime is given // we stop the previous expire routine @@ -115,12 +116,43 @@ func (fs *FileSystem) Set(keyPath string, value string, expireTime time.Time, in return e, nil } -func (fs *FileSystem) TestAndSet(keyPath string, prevValue string, prevIndex uint64, index uint64, term uint64) { +func (fs *FileSystem) TestAndSet(keyPath string, prevValue string, prevIndex uint64, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { + f, err := fs.InternalGet(keyPath, index, term) -} + if err != nil { + + etcdError, _ := err.(etcdErr.Error) + if etcdError.ErrorCode == 100 { // file does not exist + + if prevValue == "" && prevIndex == 0 { // test against if prevValue is empty + fs.Set(keyPath, value, expireTime, index, term) + e := newEvent(TestAndSet, keyPath, index, term) + e.Value = value + return e, nil + } + + return nil, err + + } -func (fs *FileSystem) TestIndexAndSet() { + return nil, err + } + if f.IsDir() { // can only test and set file + return nil, etcdErr.NewError(102, keyPath) + } + + if f.Value == prevValue || f.ModifiedIndex == prevIndex { + // if test succeed, write the value + e := newEvent(TestAndSet, keyPath, index, term) + e.PrevValue = f.Value + e.Value = value + f.Write(value, index, term) + return e, nil + } + + cause := fmt.Sprintf("[%v/%v] [%v/%v]", prevValue, f.Value, prevIndex, f.ModifiedIndex) + return nil, etcdErr.NewError(101, cause) } func (fs *FileSystem) Delete(keyPath string, recurisive bool, index uint64, term uint64) (*Event, error) { @@ -176,13 +208,18 @@ func (fs *FileSystem) InternalGet(keyPath string, index uint64, term uint64) (*N // update file system known index and term fs.Index, fs.Term = index, term - walkFunc := func(parent *Node, dirName string) (*Node, error) { - child, ok := parent.Children[dirName] + walkFunc := func(parent *Node, name string) (*Node, error) { + + if !parent.IsDir() { + return nil, etcdErr.NewError(104, parent.Path) + } + + child, ok := parent.Children[name] if ok { return child, nil } - return nil, etcdErr.NewError(100, "get") + return nil, etcdErr.NewError(100, path.Join(parent.Path, name)) } f, err := fs.walk(keyPath, walkFunc) diff --git a/file_system/file_system_test.go b/file_system/file_system_test.go index 4d41f198405..b3777565074 100644 --- a/file_system/file_system_test.go +++ b/file_system/file_system_test.go @@ -157,6 +157,40 @@ func TestExpire(t *testing.T) { } +func TestTestAndSet(t *testing.T) { + fs := New() + fs.Set("/foo", "bar", Permanent, 1, 1) + + // test on wrong previous value + _, err := fs.TestAndSet("/foo", "barbar", 0, "car", Permanent, 2, 1) + if err == nil { + t.Fatal("test and set should fail barbar != bar") + } + + // test on value + e, err := fs.TestAndSet("/foo", "bar", 0, "car", Permanent, 3, 1) + + if err != nil { + t.Fatal("test and set should succeed bar == bar") + } + + if e.PrevValue != "bar" || e.Value != "car" { + t.Fatalf("[%v/%v] [%v/%v]", e.PrevValue, "bar", e.Value, "car") + } + + // test on index + e, err = fs.TestAndSet("/foo", "", 3, "bar", Permanent, 4, 1) + + if err != nil { + t.Fatal("test and set should succeed index 3 == 3") + } + + if e.PrevValue != "car" || e.Value != "bar" { + t.Fatalf("[%v/%v] [%v/%v]", e.PrevValue, "car", e.Value, "bar") + } + +} + func setAndGet(fs *FileSystem, path string, t *testing.T) { _, err := fs.Set(path, "bar", Permanent, 1, 1) diff --git a/file_system/node.go b/file_system/node.go index 7c9c969c404..02800480dae 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -19,29 +19,33 @@ const ( ) type Node struct { - Path string - CreateIndex uint64 - CreateTerm uint64 - Parent *Node - ExpireTime time.Time - ACL string - Value string // for key-value pair - Children map[string]*Node // for directory - status int - mu sync.Mutex - stopExpire chan bool // stop expire routine channel + Path string + CreateIndex uint64 + CreateTerm uint64 + ModifiedIndex uint64 + ModifiedTerm uint64 + Parent *Node + ExpireTime time.Time + ACL string + Value string // for key-value pair + Children map[string]*Node // for directory + status int + mu sync.Mutex + stopExpire chan bool // stop expire routine channel } func newFile(keyPath string, value string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { return &Node{ - Path: keyPath, - CreateIndex: createIndex, - CreateTerm: createTerm, - Parent: parent, - ACL: ACL, - stopExpire: make(chan bool, 1), - ExpireTime: expireTime, - Value: value, + Path: keyPath, + CreateIndex: createIndex, + CreateTerm: createTerm, + ModifiedIndex: createIndex, + ModifiedTerm: createTerm, + Parent: parent, + ACL: ACL, + stopExpire: make(chan bool, 1), + ExpireTime: expireTime, + Value: value, } } @@ -113,12 +117,14 @@ func (n *Node) Read() (string, error) { // Set function set the value of the node to the given value. // If the receiver node is a directory, a "Not A File" error will be returned. -func (n *Node) Write(value string) error { +func (n *Node) Write(value string, index uint64, term uint64) error { if n.IsDir() { return etcdErr.NewError(102, "") } n.Value = value + n.ModifiedIndex = index + n.ModifiedTerm = term return nil } From 6d27afd1c94276dec019e70b202922572cd9ea76 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Thu, 5 Sep 2013 15:48:53 -0400 Subject: [PATCH 014/247] more testandset test cases --- file_system/file_system_test.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/file_system/file_system_test.go b/file_system/file_system_test.go index b3777565074..ace5a5b8b3f 100644 --- a/file_system/file_system_test.go +++ b/file_system/file_system_test.go @@ -189,6 +189,15 @@ func TestTestAndSet(t *testing.T) { t.Fatalf("[%v/%v] [%v/%v]", e.PrevValue, "car", e.Value, "bar") } + // test on empty previous value + e, err = fs.TestAndSet("/fooDir/foo", "", 0, "bar", Permanent, 4, 1) + if err != nil { + t.Fatal("test on empty node should be succeeded") + } + + if e.Key != "/fooDir/foo" || e.PrevValue != "" || e.Value != "bar" { + t.Fatalf("[%v/%v] [%v/%v] [%v/%v]", e.Key, "/fooDir/foo", e.PrevValue, "", e.Value, "bar") + } } func setAndGet(fs *FileSystem, path string, t *testing.T) { From 450d0eb0daa69cf292ec05b0738cf14492c138f8 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Thu, 5 Sep 2013 23:10:41 -0400 Subject: [PATCH 015/247] support recursive get --- file_system/file_system.go | 10 +++++++--- file_system/file_system_test.go | 5 ++++- file_system/node.go | 30 ++++++++++++++++++++++++++++-- 3 files changed, 39 insertions(+), 6 deletions(-) diff --git a/file_system/file_system.go b/file_system/file_system.go index 462745a90c7..a0b7168b949 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -36,17 +36,21 @@ func (fs *FileSystem) Get(keyPath string, recusive bool, index uint64, term uint e := newEvent(Get, keyPath, index, term) if n.IsDir() { // node is dir - e.KVPairs = make([]KeyValuePair, len(n.Children)) + children, _ := n.List() + e.KVPairs = make([]KeyValuePair, len(children)) + + // we do not use the index in the children slice directly + // we need to skip the hidden one i := 0 - for _, child := range n.Children { + for _, child := range children { if child.IsHidden() { // get will not list hidden node continue } - e.KVPairs[i] = child.Pair() + e.KVPairs[i] = child.Pair(recusive) i++ } diff --git a/file_system/file_system_test.go b/file_system/file_system_test.go index ace5a5b8b3f..0392cde54ed 100644 --- a/file_system/file_system_test.go +++ b/file_system/file_system_test.go @@ -49,7 +49,7 @@ func TestListDirectory(t *testing.T) { // set key-value /foo/fooDir/foo=bar fs.Set("/foo/fooDir/foo", "bar", Permanent, 2, 1) - e, err := fs.Get("/foo", false, 2, 1) + e, err := fs.Get("/foo", true, 2, 1) if err != nil { t.Fatalf("%v", err) @@ -67,6 +67,9 @@ func TestListDirectory(t *testing.T) { t.Fatalf("wrong kv [/foo/fooDir/ / %s] -> [true / %v]", e.KVPairs[1].Key, e.KVPairs[1].Dir) } + if e.KVPairs[1].KVPairs[0].Key != "/foo/fooDir/foo" || e.KVPairs[1].KVPairs[0].Value != "bar" { + t.Fatalf("wrong kv [/foo/fooDir/foo / %s] -> [bar / %v]", e.KVPairs[1].KVPairs[0].Key, e.KVPairs[1].KVPairs[0].Value) + } // test hidden node // create dir /foo/_hidden diff --git a/file_system/node.go b/file_system/node.go index 02800480dae..8c676577316 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -268,13 +268,39 @@ func (n *Node) IsHidden() bool { return false } -func (n *Node) Pair() KeyValuePair { +func (n *Node) Pair(recurisive bool) KeyValuePair { if n.IsDir() { - return KeyValuePair{ + pair := KeyValuePair{ Key: n.Path, Dir: true, } + if !recurisive { + return pair + } + + children, _ := n.List() + pair.KVPairs = make([]KeyValuePair, len(children)) + + // we do not use the index in the children slice directly + // we need to skip the hidden one + i := 0 + + for _, child := range children { + + if child.IsHidden() { // get will not list hidden node + continue + } + + pair.KVPairs[i] = child.Pair(recurisive) + + i++ + } + + // eliminate hidden nodes + pair.KVPairs = pair.KVPairs[:i] + + return pair } return KeyValuePair{ From 9b80e1cd6441d4c3f33f7a2d8d8d2a1eca5b7e10 Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Fri, 6 Sep 2013 14:28:11 -0400 Subject: [PATCH 016/247] refine doc --- Documentation/etcd-file-system.md | 83 ++++++++++++++++++------------- 1 file changed, 48 insertions(+), 35 deletions(-) diff --git a/Documentation/etcd-file-system.md b/Documentation/etcd-file-system.md index 22ab061b2ac..c2fc218a8fe 100644 --- a/Documentation/etcd-file-system.md +++ b/Documentation/etcd-file-system.md @@ -5,47 +5,60 @@ ![alt text](./img/etcd_fs_structure.jpg "etcd file system structure") ## Node -The building ingredients of etcd file system are two kinds of nodes -- files and directories: -- File has data associated with it. -- Directory has children nodes associated with it. +In **Etcd**, the **Node** is the rudimentary element constructing the whole. +Currently **Etcd** file system is comprised in a Unix-like way of files and directories, and they are two kinds of nodes different in: + +- **File Node** has data associated with it. +- **Directory Node** has children nodes associated with it. + +Besides the file and directory difference, all nodes have common attributes and operations as follows: ### Attributes: -####Expiration Time [optional] -The node will be deleted when it expires. -####ACL -The path of access control list of the node. +- **Expiration Time** [optional] + + The node will be deleted when it expires. + +- **ACL** + + The path of access control list of the node. ### Operation: -#### Get (path, recursive) -Get the content of the node - - If the node is a file, the data of the file will be returned. - - If the node is a directory, the child nodes of the directory will be returned. - - If recursive is true, it will recursively get the nodes of the directory. - -####Set (path, value[optional], ttl [optional]) -Set the value to a file. Set operation will help to create intermediate directories with no expiration time. - - If the value is given, set will create a file - - If the value is not given, set will crate a directory - - If ttl is given, the node will be deleted when it expires. - -####Delete (path, recursive) -Delete the node of given path. - - If the node is a directory: - - If recursive is true, the operation will delete all nodes under the directory. - - If recursive is false, error will be returned. - -####TestAndSet (path, prevValue [prevIndex], value, ttl) -Atomically test and set value to a file. If test succeeds, this operation will change the previous value of the file to the given value. - - If the prevValue is given, it will test against previous value of the node. - - If the prevValue is empty, it will test if the node is not existing. - - If the prevValue is not empty, it will test if the prevValue is equal to the current value of the file. - - If the prevIndex is given, it will test if the create/last modified index of the node is equal to prevIndex. - -####Renew (path, ttl) -Set the node's expiration time to (current time + ttl) +- **Get** (path, recursive) + + Get the content of the node + - If the node is a file, the data of the file will be returned. + - If the node is a directory, the child nodes of the directory will be returned. + - If recursive is true, it will recursively get the nodes of the directory. + +- **Set** (path, value[optional], ttl [optional]) + + Set the value to a file. Set operation will help to create intermediate directories with no expiration time. + - If the value is given, set will create a file + - If the value is not given, set will crate a directory + - If ttl is given, the node will be deleted when it expires. + +- **Delete** (path, recursive) + + Delete the node of given path. + - If the node is a directory: + - If recursive is true, the operation will delete all nodes under the directory. + - If recursive is false, error will be returned. + +- **TestAndSet** (path, prevValue [prevIndex], value, ttl) + + Atomic *test and set* value to a file. If test succeeds, this operation will change the previous value of the file to the given value. + - If the prevValue is given, it will test against previous value of + the node. + - If the prevValue is empty, it will test if the node is not existing. + - If the prevValue is not empty, it will test if the prevValue is equal to the current value of the file. + - If the prevIndex is given, it will test if the create/last modified index of the node is equal to prevIndex. + +- **Renew** (path, ttl) + + Set the node's expiration time to (current time + ttl) ## ACL [TODO] ## User Group -[TODO] \ No newline at end of file +[TODO] From ea4ab2a429c579b6766479e91df117b71769346f Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Fri, 6 Sep 2013 22:05:11 -0400 Subject: [PATCH 017/247] recursive watch --- file_system/watcher.go | 61 ++++++++++++++++++++++++++++++------- file_system/watcher_test.go | 26 +++++++++++++++- 2 files changed, 75 insertions(+), 12 deletions(-) diff --git a/file_system/watcher.go b/file_system/watcher.go index 237399dba02..7cad8b15b72 100644 --- a/file_system/watcher.go +++ b/file_system/watcher.go @@ -12,6 +12,11 @@ type watcherHub struct { EventHistory *EventHistory } +type watcher struct { + eventChan chan *Event + recursive bool +} + func newWatchHub(capacity int) *watcherHub { return &watcherHub{ watchers: make(map[string]*list.List), @@ -19,7 +24,11 @@ func newWatchHub(capacity int) *watcherHub { } } -func (wh *watcherHub) watch(prefix string, index uint64) (error, <-chan *Event) { +// watch function returns an Event channel. +// If recursive is true, the first change after index under prefix will be sent to the event channel. +// If recursive is false, the first change after index at prefix will be sent to the event channel. +// If index is zero, watch will start from the current index + 1. +func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (error, <-chan *Event) { eventChan := make(chan *Event, 1) e, err := wh.EventHistory.scan(prefix, index) @@ -33,13 +42,19 @@ func (wh *watcherHub) watch(prefix string, index uint64) (error, <-chan *Event) return nil, eventChan } + w := &watcher{ + eventChan: eventChan, + recursive: recursive, + } + l, ok := wh.watchers[prefix] - if ok { - l.PushBack(eventChan) - } else { + if ok { // add the new watcher to the back of the list + l.PushBack(w) + + } else { // create a new list and add the new watcher l := list.New() - l.PushBack(eventChan) + l.PushBack(w) wh.watchers[prefix] = l } @@ -59,15 +74,39 @@ func (wh *watcherHub) notify(e *Event) { if ok { + curr := l.Front() + notifiedAll := true + for { - element := l.Front() - if element == nil { - delete(wh.watchers, currPath) + + if curr == nil { // we have reached the end of the list + + if notifiedAll { + // if we have notified all watcher in the list + // we can delete the list + delete(wh.watchers, currPath) + } break } - c, _ := element.Value.(chan *Event) - c <- e - l.Remove(element) + + next := curr.Next() // save the next + + w, _ := curr.Value.(*watcher) + + if w.recursive { + w.eventChan <- e + l.Remove(curr) + } else { + if e.Key == currPath { // only notify the same path + w.eventChan <- e + l.Remove(curr) + } else { // we do not notify all watcher in the list + notifiedAll = false + } + } + + curr = next // go to the next one + } } diff --git a/file_system/watcher_test.go b/file_system/watcher_test.go index fcd3fd9b354..c63a489d7d2 100644 --- a/file_system/watcher_test.go +++ b/file_system/watcher_test.go @@ -6,7 +6,7 @@ import ( func TestWatch(t *testing.T) { wh := newWatchHub(100) - err, c := wh.watch("/foo", 0) + err, c := wh.watch("/foo", true, 0) if err != nil { t.Fatal("%v", err) @@ -28,4 +28,28 @@ func TestWatch(t *testing.T) { if e != re { t.Fatal("recv != send") } + + _, c = wh.watch("/foo", false, 0) + + e = newEvent(Set, "/foo/bar", 1, 0) + + wh.notify(e) + + select { + case <-c: + t.Fatal("should not receive from channel if not recursive") + default: + // do nothing + } + + e = newEvent(Set, "/foo", 1, 0) + + wh.notify(e) + + re = <-c + + if e != re { + t.Fatal("recv != send") + } + } From 907e39edec707806132ecaf75f8e1760b9d1a8b6 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Fri, 6 Sep 2013 23:01:11 -0400 Subject: [PATCH 018/247] update operation --- Documentation/etcd-file-system.md | 15 +++-- file_system/file_system.go | 99 +++++++++++++++++++------------ 2 files changed, 71 insertions(+), 43 deletions(-) diff --git a/Documentation/etcd-file-system.md b/Documentation/etcd-file-system.md index c2fc218a8fe..e4bcd8992e1 100644 --- a/Documentation/etcd-file-system.md +++ b/Documentation/etcd-file-system.md @@ -30,13 +30,20 @@ Besides the file and directory difference, all nodes have common attributes and - If the node is a directory, the child nodes of the directory will be returned. - If recursive is true, it will recursively get the nodes of the directory. -- **Set** (path, value[optional], ttl [optional]) +- **Create** (path, value[optional], ttl [optional]) - Set the value to a file. Set operation will help to create intermediate directories with no expiration time. - - If the value is given, set will create a file - - If the value is not given, set will crate a directory + Create a file. Create operation will help to create intermediate directories with no expiration time. + - If the file already exists, create will fail. + - If the value is given, set will create a file. + - If the value is not given, set will crate a directory. - If ttl is given, the node will be deleted when it expires. +- **Update** (path, value[optional], ttl [optional]) + + Update the content of the node. + - If the value is given, the value of the key will be updated. + - If ttl is given, the expiration time of the node will be updated. + - **Delete** (path, recursive) Delete the node of given path. diff --git a/file_system/file_system.go b/file_system/file_system.go index a0b7168b949..9149b9834db 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -65,47 +65,82 @@ func (fs *FileSystem) Get(keyPath string, recusive bool, index uint64, term uint return e, nil } -func (fs *FileSystem) Set(keyPath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { - keyPath = path.Clean("/" + keyPath) - - // update file system known index and term - fs.Index, fs.Term = index, term - - dir, name := path.Split(keyPath) - - // walk through the keyPath and get the last directory node - d, err := fs.walk(dir, fs.checkDir) +func (fs *FileSystem) Update(keyPath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { + n, err := fs.InternalGet(keyPath, index, term) - if err != nil { + if err != nil { // if node does not exist, return error return nil, err } e := newEvent(Set, keyPath, fs.Index, fs.Term) - e.Value = value - f, err := d.GetFile(name) + if n.IsDir() { // if the node is a directory, we can only update ttl - if err == nil { + if len(value) != 0 { + return nil, etcdErr.NewError(102, keyPath) + } - if f != nil { // update previous file if exist - e.PrevValue = f.Value - f.Write(e.Value, index, term) + if n.ExpireTime != Permanent && expireTime != Permanent { + n.stopExpire <- true + } - // if the previous ExpireTime is not Permanent and expireTime is given - // we stop the previous expire routine - if f.ExpireTime != Permanent && expireTime != Permanent { - f.stopExpire <- true - } - } else { // create new file + } else { // if the node is a file, we can update value and ttl + e.PrevValue = n.Value - f = newFile(keyPath, value, fs.Index, fs.Term, d, "", expireTime) + if len(value) != 0 { + e.Value = value + } - err = d.Add(f) + n.Write(value, index, term) + if n.ExpireTime != Permanent && expireTime != Permanent { + n.stopExpire <- true } } + // update ttl + if expireTime != Permanent { + go n.Expire() + e.Expiration = &n.ExpireTime + e.TTL = int64(expireTime.Sub(time.Now()) / time.Second) + } + + return e, nil +} + +func (fs *FileSystem) Create(keyPath string, value string, expireTime time.Time, create bool, index uint64, term uint64) (*Event, error) { + keyPath = path.Clean("/" + keyPath) + + // make sure we can create the node + _, err := fs.InternalGet(keyPath, index, term) + + if err != nil { // key already exists + return nil, etcdErr.NewError(105, keyPath) + } + + etcdError, _ := err.(etcdErr.Error) + + if etcdError.ErrorCode == 104 { // we cannot create the key due to meet a file while walking + return nil, err + } + + dir, _ := path.Split(keyPath) + + // walk through the keyPath, create dirs and get the last directory node + d, err := fs.walk(dir, fs.checkDir) + + if err != nil { + return nil, err + } + + e := newEvent(Set, keyPath, fs.Index, fs.Term) + e.Value = value + + f := newFile(keyPath, value, fs.Index, fs.Term, d, "", expireTime) + + err = d.Add(f) + if err != nil { return nil, err } @@ -125,20 +160,6 @@ func (fs *FileSystem) TestAndSet(keyPath string, prevValue string, prevIndex uin if err != nil { - etcdError, _ := err.(etcdErr.Error) - if etcdError.ErrorCode == 100 { // file does not exist - - if prevValue == "" && prevIndex == 0 { // test against if prevValue is empty - fs.Set(keyPath, value, expireTime, index, term) - e := newEvent(TestAndSet, keyPath, index, term) - e.Value = value - return e, nil - } - - return nil, err - - } - return nil, err } From 4f99b6029183fe9f22089faf7d869a28a327f6f8 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Fri, 6 Sep 2013 23:24:01 -0400 Subject: [PATCH 019/247] update test --- file_system/file_system.go | 92 ++++++++++++++++----------------- file_system/file_system_test.go | 72 ++++++++++++++------------ 2 files changed, 86 insertions(+), 78 deletions(-) diff --git a/file_system/file_system.go b/file_system/file_system.go index 9149b9834db..3049d2fc77c 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -65,57 +65,13 @@ func (fs *FileSystem) Get(keyPath string, recusive bool, index uint64, term uint return e, nil } -func (fs *FileSystem) Update(keyPath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { - n, err := fs.InternalGet(keyPath, index, term) - - if err != nil { // if node does not exist, return error - return nil, err - } - - e := newEvent(Set, keyPath, fs.Index, fs.Term) - - if n.IsDir() { // if the node is a directory, we can only update ttl - - if len(value) != 0 { - return nil, etcdErr.NewError(102, keyPath) - } - - if n.ExpireTime != Permanent && expireTime != Permanent { - n.stopExpire <- true - } - - } else { // if the node is a file, we can update value and ttl - e.PrevValue = n.Value - - if len(value) != 0 { - e.Value = value - } - - n.Write(value, index, term) - - if n.ExpireTime != Permanent && expireTime != Permanent { - n.stopExpire <- true - } - - } - - // update ttl - if expireTime != Permanent { - go n.Expire() - e.Expiration = &n.ExpireTime - e.TTL = int64(expireTime.Sub(time.Now()) / time.Second) - } - - return e, nil -} - -func (fs *FileSystem) Create(keyPath string, value string, expireTime time.Time, create bool, index uint64, term uint64) (*Event, error) { +func (fs *FileSystem) Create(keyPath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { keyPath = path.Clean("/" + keyPath) // make sure we can create the node _, err := fs.InternalGet(keyPath, index, term) - if err != nil { // key already exists + if err == nil { // key already exists return nil, etcdErr.NewError(105, keyPath) } @@ -155,6 +111,50 @@ func (fs *FileSystem) Create(keyPath string, value string, expireTime time.Time, return e, nil } +func (fs *FileSystem) Update(keyPath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { + n, err := fs.InternalGet(keyPath, index, term) + + if err != nil { // if node does not exist, return error + return nil, err + } + + e := newEvent(Set, keyPath, fs.Index, fs.Term) + + if n.IsDir() { // if the node is a directory, we can only update ttl + + if len(value) != 0 { + return nil, etcdErr.NewError(102, keyPath) + } + + if n.ExpireTime != Permanent && expireTime != Permanent { + n.stopExpire <- true + } + + } else { // if the node is a file, we can update value and ttl + e.PrevValue = n.Value + + if len(value) != 0 { + e.Value = value + } + + n.Write(value, index, term) + + if n.ExpireTime != Permanent && expireTime != Permanent { + n.stopExpire <- true + } + + } + + // update ttl + if expireTime != Permanent { + go n.Expire() + e.Expiration = &n.ExpireTime + e.TTL = int64(expireTime.Sub(time.Now()) / time.Second) + } + + return e, nil +} + func (fs *FileSystem) TestAndSet(keyPath string, prevValue string, prevIndex uint64, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { f, err := fs.InternalGet(keyPath, index, term) diff --git a/file_system/file_system_test.go b/file_system/file_system_test.go index 0392cde54ed..8649c8e507c 100644 --- a/file_system/file_system_test.go +++ b/file_system/file_system_test.go @@ -5,26 +5,43 @@ import ( "time" ) -func TestSetAndGet(t *testing.T) { +func TestCreateAndGet(t *testing.T) { fs := New() - setAndGet(fs, "/foobar", t) - setAndGet(fs, "/foo/bar", t) - setAndGet(fs, "/foo/foo/bar", t) + + // this should create successfully + createAndGet(fs, "/foobar", t) + createAndGet(fs, "/foo/bar", t) + createAndGet(fs, "/foo/foo/bar", t) + + // already exist, create should fail + _, err := fs.Create("/foobar", "bar", Permanent, 1, 1) + + if err == nil { + t.Fatal("Create should fail") + } + + // meet file, create should fail + _, err = fs.Create("/foo/bar/bar", "bar", Permanent, 1, 1) + + if err == nil { + t.Fatal("Create should fail") + } + } func TestUpdateFile(t *testing.T) { fs := New() - _, err := fs.Set("/foo/bar", "bar", Permanent, 1, 1) + _, err := fs.Create("/foo/bar", "bar", Permanent, 1, 1) if err != nil { - t.Fatalf("cannot set %s=bar [%s]", "/foo/bar", err.Error()) + t.Fatalf("cannot update %s=bar [%s]", "/foo/bar", err.Error()) } - _, err = fs.Set("/foo/bar", "barbar", Permanent, 2, 1) + _, err = fs.Update("/foo/bar", "barbar", Permanent, 2, 1) if err != nil { - t.Fatalf("cannot set %s=barbar [%s]", "/foo/bar", err.Error()) + t.Fatalf("cannot update %s=barbar [%s]", "/foo/bar", err.Error()) } e, err := fs.Get("/foo/bar", false, 2, 1) @@ -43,11 +60,11 @@ func TestListDirectory(t *testing.T) { // create dir /foo // set key-value /foo/foo=bar - fs.Set("/foo/foo", "bar", Permanent, 1, 1) + fs.Create("/foo/foo", "bar", Permanent, 1, 1) // create dir /foo/fooDir // set key-value /foo/fooDir/foo=bar - fs.Set("/foo/fooDir/foo", "bar", Permanent, 2, 1) + fs.Create("/foo/fooDir/foo", "bar", Permanent, 2, 1) e, err := fs.Get("/foo", true, 2, 1) @@ -74,7 +91,7 @@ func TestListDirectory(t *testing.T) { // create dir /foo/_hidden // set key-value /foo/_hidden/foo -> bar - fs.Set("/foo/_hidden/foo", "bar", Permanent, 3, 1) + fs.Create("/foo/_hidden/foo", "bar", Permanent, 3, 1) e, _ = fs.Get("/foo", false, 2, 1) @@ -86,7 +103,7 @@ func TestListDirectory(t *testing.T) { func TestRemove(t *testing.T) { fs := New() - fs.Set("/foo", "bar", Permanent, 1, 1) + fs.Create("/foo", "bar", Permanent, 1, 1) _, err := fs.Delete("/foo", false, 1, 1) if err != nil { @@ -99,9 +116,9 @@ func TestRemove(t *testing.T) { t.Fatalf("can get the node after deletion") } - fs.Set("/foo/bar", "bar", Permanent, 1, 1) - fs.Set("/foo/car", "car", Permanent, 1, 1) - fs.Set("/foo/dar/dar", "dar", Permanent, 1, 1) + fs.Create("/foo/bar", "bar", Permanent, 1, 1) + fs.Create("/foo/car", "car", Permanent, 1, 1) + fs.Create("/foo/dar/dar", "dar", Permanent, 1, 1) _, err = fs.Delete("/foo", false, 1, 1) @@ -128,7 +145,7 @@ func TestExpire(t *testing.T) { expire := time.Now().Add(time.Second) - fs.Set("/foo", "bar", expire, 1, 1) + fs.Create("/foo", "bar", expire, 1, 1) _, err := fs.InternalGet("/foo", 1, 1) @@ -144,7 +161,7 @@ func TestExpire(t *testing.T) { t.Fatalf("can get the node after expiration time") } - fs.Set("/foo", "bar", expire, 1, 1) + fs.Create("/foo", "bar", expire, 1, 1) time.Sleep(time.Millisecond * 50) _, err = fs.InternalGet("/foo", 1, 1) @@ -155,14 +172,14 @@ func TestExpire(t *testing.T) { expire = time.Now().Add(time.Second) - fs.Set("/foo", "bar", expire, 1, 1) + fs.Create("/foo", "bar", expire, 1, 1) fs.Delete("/foo", false, 1, 1) } func TestTestAndSet(t *testing.T) { fs := New() - fs.Set("/foo", "bar", Permanent, 1, 1) + fs.Create("/foo", "bar", Permanent, 1, 1) // test on wrong previous value _, err := fs.TestAndSet("/foo", "barbar", 0, "car", Permanent, 2, 1) @@ -191,23 +208,13 @@ func TestTestAndSet(t *testing.T) { if e.PrevValue != "car" || e.Value != "bar" { t.Fatalf("[%v/%v] [%v/%v]", e.PrevValue, "car", e.Value, "bar") } - - // test on empty previous value - e, err = fs.TestAndSet("/fooDir/foo", "", 0, "bar", Permanent, 4, 1) - if err != nil { - t.Fatal("test on empty node should be succeeded") - } - - if e.Key != "/fooDir/foo" || e.PrevValue != "" || e.Value != "bar" { - t.Fatalf("[%v/%v] [%v/%v] [%v/%v]", e.Key, "/fooDir/foo", e.PrevValue, "", e.Value, "bar") - } } -func setAndGet(fs *FileSystem, path string, t *testing.T) { - _, err := fs.Set(path, "bar", Permanent, 1, 1) +func createAndGet(fs *FileSystem, path string, t *testing.T) { + _, err := fs.Create(path, "bar", Permanent, 1, 1) if err != nil { - t.Fatalf("cannot set %s=bar [%s]", path, err.Error()) + t.Fatalf("cannot create %s=bar [%s]", path, err.Error()) } e, err := fs.Get(path, false, 1, 1) @@ -219,4 +226,5 @@ func setAndGet(fs *FileSystem, path string, t *testing.T) { if e.Value != "bar" { t.Fatalf("expect value of %s is bar [%s]", path, e.Value) } + } From 948044093b69cbf9a6aa0aea79d1b7489e5f6ad5 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Fri, 6 Sep 2013 23:36:11 -0400 Subject: [PATCH 020/247] support create directory --- file_system/file_system.go | 27 +++++++++++++++++++-------- file_system/file_system_test.go | 22 +++++++++++++++++++++- file_system/node.go | 5 +++-- 3 files changed, 43 insertions(+), 11 deletions(-) diff --git a/file_system/file_system.go b/file_system/file_system.go index 3049d2fc77c..eb19e2ea8e5 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -19,7 +19,7 @@ type FileSystem struct { func New() *FileSystem { return &FileSystem{ - Root: newDir("/", 0, 0, nil, ""), + Root: newDir("/", 0, 0, nil, "", Permanent), WatcherHub: newWatchHub(1000), } @@ -36,6 +36,7 @@ func (fs *FileSystem) Get(keyPath string, recusive bool, index uint64, term uint e := newEvent(Get, keyPath, index, term) if n.IsDir() { // node is dir + e.Dir = true children, _ := n.List() e.KVPairs = make([]KeyValuePair, len(children)) @@ -57,7 +58,6 @@ func (fs *FileSystem) Get(keyPath string, recusive bool, index uint64, term uint // eliminate hidden nodes e.KVPairs = e.KVPairs[:i] - } else { // node is file e.Value = n.Value } @@ -91,11 +91,22 @@ func (fs *FileSystem) Create(keyPath string, value string, expireTime time.Time, } e := newEvent(Set, keyPath, fs.Index, fs.Term) - e.Value = value - f := newFile(keyPath, value, fs.Index, fs.Term, d, "", expireTime) + var n *Node + + if len(value) != 0 { // create file + e.Value = value + + n = newFile(keyPath, value, fs.Index, fs.Term, d, "", expireTime) + + } else { // create directory + e.Dir = true + + n = newDir(keyPath, fs.Index, fs.Term, d, "", expireTime) + + } - err = d.Add(f) + err = d.Add(n) if err != nil { return nil, err @@ -103,8 +114,8 @@ func (fs *FileSystem) Create(keyPath string, value string, expireTime time.Time, // Node with TTL if expireTime != Permanent { - go f.Expire() - e.Expiration = &f.ExpireTime + go n.Expire() + e.Expiration = &n.ExpireTime e.TTL = int64(expireTime.Sub(time.Now()) / time.Second) } @@ -268,7 +279,7 @@ func (fs *FileSystem) checkDir(parent *Node, dirName string) (*Node, error) { return subDir, nil } - n := newDir(path.Join(parent.Path, dirName), fs.Index, fs.Term, parent, parent.ACL) + n := newDir(path.Join(parent.Path, dirName), fs.Index, fs.Term, parent, parent.ACL, Permanent) parent.Children[dirName] = n diff --git a/file_system/file_system_test.go b/file_system/file_system_test.go index 8649c8e507c..04f5ad9b667 100644 --- a/file_system/file_system_test.go +++ b/file_system/file_system_test.go @@ -21,12 +21,32 @@ func TestCreateAndGet(t *testing.T) { } // meet file, create should fail - _, err = fs.Create("/foo/bar/bar", "bar", Permanent, 1, 1) + _, err = fs.Create("/foo/bar/bar", "bar", Permanent, 2, 1) if err == nil { t.Fatal("Create should fail") } + // create a directory + _, err = fs.Create("/fooDir", "", Permanent, 3, 1) + + if err != nil { + t.Fatal("Cannot create /fooDir") + } + + e, err := fs.Get("/fooDir", false, 3, 1) + + if err != nil || e.Dir != true { + t.Fatal("Cannot create /fooDir ") + } + + // create a file under directory + _, err = fs.Create("/fooDir/bar", "bar", Permanent, 4, 1) + + if err != nil { + t.Fatal("Cannot create /fooDir/bar = bar") + } + } func TestUpdateFile(t *testing.T) { diff --git a/file_system/node.go b/file_system/node.go index 8c676577316..4d933f55ae8 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -49,7 +49,7 @@ func newFile(keyPath string, value string, createIndex uint64, createTerm uint64 } } -func newDir(keyPath string, createIndex uint64, createTerm uint64, parent *Node, ACL string) *Node { +func newDir(keyPath string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { return &Node{ Path: keyPath, CreateIndex: createIndex, @@ -57,6 +57,7 @@ func newDir(keyPath string, createIndex uint64, createTerm uint64, parent *Node, Parent: parent, ACL: ACL, stopExpire: make(chan bool, 1), + ExpireTime: expireTime, Children: make(map[string]*Node), } } @@ -210,7 +211,7 @@ func (n *Node) Clone() *Node { return newFile(n.Path, n.Value, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime) } - clone := newDir(n.Path, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL) + clone := newDir(n.Path, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime) for key, child := range n.Children { clone.Children[key] = child.Clone() From bd8ec6d67b2a6223715cb454865f5090acc32dcd Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 7 Sep 2013 01:05:11 -0400 Subject: [PATCH 021/247] support watch delete --- file_system/file_system.go | 18 +++++--- file_system/file_system_test.go | 23 ++++++++++ file_system/node.go | 18 ++++++-- file_system/watcher.go | 76 ++++++++++++++++----------------- file_system/watcher_test.go | 4 +- 5 files changed, 88 insertions(+), 51 deletions(-) diff --git a/file_system/file_system.go b/file_system/file_system.go index eb19e2ea8e5..43f850275b7 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -198,12 +198,6 @@ func (fs *FileSystem) Delete(keyPath string, recurisive bool, index uint64, term return nil, err } - err = n.Remove(recurisive) - - if err != nil { - return nil, err - } - e := newEvent(Delete, keyPath, index, term) if n.IsDir() { @@ -212,6 +206,18 @@ func (fs *FileSystem) Delete(keyPath string, recurisive bool, index uint64, term e.PrevValue = n.Value } + callback := func(path string) { + fs.WatcherHub.notifyWithPath(e, path, true) + } + + err = n.Remove(recurisive, callback) + + if err != nil { + return nil, err + } + + fs.WatcherHub.notify(e) + return e, nil } diff --git a/file_system/file_system_test.go b/file_system/file_system_test.go index 04f5ad9b667..16666f61443 100644 --- a/file_system/file_system_test.go +++ b/file_system/file_system_test.go @@ -230,6 +230,29 @@ func TestTestAndSet(t *testing.T) { } } +func TestWatchRemove(t *testing.T) { + fs := New() + fs.Create("/foo/foo/foo", "bar", Permanent, 1, 1) + + // watch at a deeper path + c, _ := fs.WatcherHub.watch("/foo/foo/foo", false, 0) + fs.Delete("/foo", true, 2, 1) + e := <-c + if e.Key != "/foo" { + t.Fatal("watch for delete fails") + } + + fs.Create("/foo/foo/foo", "bar", Permanent, 3, 1) + // watch at a prefix + c, _ = fs.WatcherHub.watch("/foo", true, 0) + fs.Delete("/foo/foo/foo", false, 4, 1) + e = <-c + if e.Key != "/foo/foo/foo" { + t.Fatal("watch for delete fails") + } + +} + func createAndGet(fs *FileSystem, path string, t *testing.T) { _, err := fs.Create(path, "bar", Permanent, 1, 1) diff --git a/file_system/node.go b/file_system/node.go index 4d933f55ae8..76126676867 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -65,7 +65,7 @@ func newDir(keyPath string, createIndex uint64, createTerm uint64, parent *Node, // Remove function remove the node. // If the node is a directory and recursive is true, the function will recursively remove // add nodes under the receiver node. -func (n *Node) Remove(recursive bool) error { +func (n *Node) Remove(recursive bool, callback func(path string)) error { n.mu.Lock() defer n.mu.Unlock() @@ -80,6 +80,11 @@ func (n *Node) Remove(recursive bool) error { // This is the only pointer to Node object // Handled by garbage collector delete(n.Parent.Children, name) + + if callback != nil { + callback(n.Path) + } + n.stopExpire <- true n.status = removed } @@ -92,13 +97,18 @@ func (n *Node) Remove(recursive bool) error { } for _, child := range n.Children { // delete all children - child.Remove(true) + child.Remove(true, callback) } // delete self _, name := path.Split(n.Path) if n.Parent.Children[name] == n { delete(n.Parent.Children, name) + + if callback != nil { + callback(n.Path) + } + n.stopExpire <- true n.status = removed } @@ -235,14 +245,14 @@ func (n *Node) IsDir() bool { func (n *Node) Expire() { duration := n.ExpireTime.Sub(time.Now()) if duration <= 0 { - n.Remove(true) + n.Remove(true, nil) return } select { // if timeout, delete the node case <-time.After(duration): - n.Remove(true) + n.Remove(true, nil) return // if stopped, return diff --git a/file_system/watcher.go b/file_system/watcher.go index 7cad8b15b72..c17d0eb877a 100644 --- a/file_system/watcher.go +++ b/file_system/watcher.go @@ -28,18 +28,18 @@ func newWatchHub(capacity int) *watcherHub { // If recursive is true, the first change after index under prefix will be sent to the event channel. // If recursive is false, the first change after index at prefix will be sent to the event channel. // If index is zero, watch will start from the current index + 1. -func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (error, <-chan *Event) { +func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (<-chan *Event, error) { eventChan := make(chan *Event, 1) e, err := wh.EventHistory.scan(prefix, index) if err != nil { - return err, nil + return nil, err } if e != nil { eventChan <- e - return nil, eventChan + return eventChan, nil } w := &watcher{ @@ -58,57 +58,55 @@ func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (error, wh.watchers[prefix] = l } - return nil, eventChan + return eventChan, nil } -func (wh *watcherHub) notify(e *Event) { +func (wh *watcherHub) notifyWithPath(e *Event, path string, force bool) { + l, ok := wh.watchers[path] - segments := strings.Split(e.Key, "/") - currPath := "/" + if ok { - // walk through all the paths - for _, segment := range segments { - currPath = path.Join(currPath, segment) + curr := l.Front() + notifiedAll := true - l, ok := wh.watchers[currPath] + for { - if ok { + if curr == nil { // we have reached the end of the list - curr := l.Front() - notifiedAll := true + if notifiedAll { + // if we have notified all watcher in the list + // we can delete the list + delete(wh.watchers, path) + } + break + } - for { + next := curr.Next() // save the next - if curr == nil { // we have reached the end of the list + w, _ := curr.Value.(*watcher) - if notifiedAll { - // if we have notified all watcher in the list - // we can delete the list - delete(wh.watchers, currPath) - } - break - } + if w.recursive || force || e.Key == path { + w.eventChan <- e + l.Remove(curr) + } else { + notifiedAll = false + } - next := curr.Next() // save the next + curr = next // go to the next one - w, _ := curr.Value.(*watcher) + } + } +} - if w.recursive { - w.eventChan <- e - l.Remove(curr) - } else { - if e.Key == currPath { // only notify the same path - w.eventChan <- e - l.Remove(curr) - } else { // we do not notify all watcher in the list - notifiedAll = false - } - } +func (wh *watcherHub) notify(e *Event) { - curr = next // go to the next one + segments := strings.Split(e.Key, "/") - } - } + currPath := "/" + // walk through all the paths + for _, segment := range segments { + currPath = path.Join(currPath, segment) + wh.notifyWithPath(e, currPath, false) } } diff --git a/file_system/watcher_test.go b/file_system/watcher_test.go index c63a489d7d2..b817e64ec52 100644 --- a/file_system/watcher_test.go +++ b/file_system/watcher_test.go @@ -6,7 +6,7 @@ import ( func TestWatch(t *testing.T) { wh := newWatchHub(100) - err, c := wh.watch("/foo", true, 0) + c, err := wh.watch("/foo", true, 0) if err != nil { t.Fatal("%v", err) @@ -29,7 +29,7 @@ func TestWatch(t *testing.T) { t.Fatal("recv != send") } - _, c = wh.watch("/foo", false, 0) + c, _ = wh.watch("/foo", false, 0) e = newEvent(Set, "/foo/bar", 1, 0) From f50cf0497dc4f0a7cfb5e387dcbdc72e09160cb0 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 7 Sep 2013 08:54:58 -0400 Subject: [PATCH 022/247] add comments; event.go: add Create and Update const --- file_system/event.go | 3 +- file_system/event_test.go | 12 ++--- file_system/file_system.go | 96 ++++++++++++++++++++----------------- file_system/node.go | 8 ++-- file_system/watcher_test.go | 6 +-- 5 files changed, 68 insertions(+), 57 deletions(-) diff --git a/file_system/event.go b/file_system/event.go index d539ed9dae3..d8fef7da8e7 100644 --- a/file_system/event.go +++ b/file_system/event.go @@ -8,7 +8,8 @@ import ( const ( Get = "get" - Set = "set" + Create = "create" + Update = "update" Delete = "delete" TestAndSet = "testAndSet" ) diff --git a/file_system/event_test.go b/file_system/event_test.go index 2c9b694426b..53df55ee546 100644 --- a/file_system/event_test.go +++ b/file_system/event_test.go @@ -13,7 +13,7 @@ func TestEventQueue(t *testing.T) { // Add for i := 0; i < 200; i++ { - e := newEvent(Set, "/foo", uint64(i), 0) + e := newEvent(Create, "/foo", uint64(i), 0) eh.addEvent(e) } @@ -33,11 +33,11 @@ func TestScanHistory(t *testing.T) { eh := newEventHistory(100) // Add - eh.addEvent(newEvent(Set, "/foo", 1, 0)) - eh.addEvent(newEvent(Set, "/foo/bar", 2, 0)) - eh.addEvent(newEvent(Set, "/foo/foo", 3, 0)) - eh.addEvent(newEvent(Set, "/foo/bar/bar", 4, 0)) - eh.addEvent(newEvent(Set, "/foo/foo/foo", 5, 0)) + eh.addEvent(newEvent(Create, "/foo", 1, 0)) + eh.addEvent(newEvent(Create, "/foo/bar", 2, 0)) + eh.addEvent(newEvent(Create, "/foo/foo", 3, 0)) + eh.addEvent(newEvent(Create, "/foo/bar/bar", 4, 0)) + eh.addEvent(newEvent(Create, "/foo/foo/foo", 5, 0)) e, err := eh.scan("/foo", 1) if err != nil || e.Index != 1 { diff --git a/file_system/file_system.go b/file_system/file_system.go index 43f850275b7..8edf0a06841 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -25,15 +25,15 @@ func New() *FileSystem { } -func (fs *FileSystem) Get(keyPath string, recusive bool, index uint64, term uint64) (*Event, error) { +func (fs *FileSystem) Get(nodePath string, recusive bool, index uint64, term uint64) (*Event, error) { // TODO: add recursive get - n, err := fs.InternalGet(keyPath, index, term) + n, err := fs.InternalGet(nodePath, index, term) if err != nil { return nil, err } - e := newEvent(Get, keyPath, index, term) + e := newEvent(Get, nodePath, index, term) if n.IsDir() { // node is dir e.Dir = true @@ -65,14 +65,17 @@ func (fs *FileSystem) Get(keyPath string, recusive bool, index uint64, term uint return e, nil } -func (fs *FileSystem) Create(keyPath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { - keyPath = path.Clean("/" + keyPath) +// Create function creates the Node at nodePath. Create will help to create intermediate directories with no ttl. +// If the node has already existed, create will fail. +// If any node on the path is a file, create will fail. +func (fs *FileSystem) Create(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { + nodePath = path.Clean("/" + nodePath) // make sure we can create the node - _, err := fs.InternalGet(keyPath, index, term) + _, err := fs.InternalGet(nodePath, index, term) if err == nil { // key already exists - return nil, etcdErr.NewError(105, keyPath) + return nil, etcdErr.NewError(105, nodePath) } etcdError, _ := err.(etcdErr.Error) @@ -81,28 +84,28 @@ func (fs *FileSystem) Create(keyPath string, value string, expireTime time.Time, return nil, err } - dir, _ := path.Split(keyPath) + dir, _ := path.Split(nodePath) - // walk through the keyPath, create dirs and get the last directory node + // walk through the nodePath, create dirs and get the last directory node d, err := fs.walk(dir, fs.checkDir) if err != nil { return nil, err } - e := newEvent(Set, keyPath, fs.Index, fs.Term) + e := newEvent(Create, nodePath, fs.Index, fs.Term) var n *Node if len(value) != 0 { // create file e.Value = value - n = newFile(keyPath, value, fs.Index, fs.Term, d, "", expireTime) + n = newFile(nodePath, value, fs.Index, fs.Term, d, "", expireTime) } else { // create directory e.Dir = true - n = newDir(keyPath, fs.Index, fs.Term, d, "", expireTime) + n = newDir(nodePath, fs.Index, fs.Term, d, "", expireTime) } @@ -119,26 +122,26 @@ func (fs *FileSystem) Create(keyPath string, value string, expireTime time.Time, e.TTL = int64(expireTime.Sub(time.Now()) / time.Second) } + fs.WatcherHub.notify(e) return e, nil } -func (fs *FileSystem) Update(keyPath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { - n, err := fs.InternalGet(keyPath, index, term) +// Update function updates the value/ttl of the node. +// If the node is a file, the value and the ttl can be updated. +// If the node is a directory, only the ttl can be updated. +func (fs *FileSystem) Update(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { + n, err := fs.InternalGet(nodePath, index, term) - if err != nil { // if node does not exist, return error + if err != nil { // if the node does not exist, return error return nil, err } - e := newEvent(Set, keyPath, fs.Index, fs.Term) + e := newEvent(Update, nodePath, fs.Index, fs.Term) if n.IsDir() { // if the node is a directory, we can only update ttl if len(value) != 0 { - return nil, etcdErr.NewError(102, keyPath) - } - - if n.ExpireTime != Permanent && expireTime != Permanent { - n.stopExpire <- true + return nil, etcdErr.NewError(102, nodePath) } } else { // if the node is a file, we can update value and ttl @@ -149,25 +152,27 @@ func (fs *FileSystem) Update(keyPath string, value string, expireTime time.Time, } n.Write(value, index, term) - - if n.ExpireTime != Permanent && expireTime != Permanent { - n.stopExpire <- true - } - } // update ttl + if n.ExpireTime != Permanent && expireTime != Permanent { + n.stopExpire <- true + } + if expireTime != Permanent { go n.Expire() e.Expiration = &n.ExpireTime e.TTL = int64(expireTime.Sub(time.Now()) / time.Second) } + fs.WatcherHub.notify(e) return e, nil } -func (fs *FileSystem) TestAndSet(keyPath string, prevValue string, prevIndex uint64, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { - f, err := fs.InternalGet(keyPath, index, term) +func (fs *FileSystem) TestAndSet(nodePath string, prevValue string, prevIndex uint64, + value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { + + f, err := fs.InternalGet(nodePath, index, term) if err != nil { @@ -175,15 +180,18 @@ func (fs *FileSystem) TestAndSet(keyPath string, prevValue string, prevIndex uin } if f.IsDir() { // can only test and set file - return nil, etcdErr.NewError(102, keyPath) + return nil, etcdErr.NewError(102, nodePath) } if f.Value == prevValue || f.ModifiedIndex == prevIndex { // if test succeed, write the value - e := newEvent(TestAndSet, keyPath, index, term) + e := newEvent(TestAndSet, nodePath, index, term) e.PrevValue = f.Value e.Value = value f.Write(value, index, term) + + fs.WatcherHub.notify(e) + return e, nil } @@ -191,14 +199,16 @@ func (fs *FileSystem) TestAndSet(keyPath string, prevValue string, prevIndex uin return nil, etcdErr.NewError(101, cause) } -func (fs *FileSystem) Delete(keyPath string, recurisive bool, index uint64, term uint64) (*Event, error) { - n, err := fs.InternalGet(keyPath, index, term) +// Delete function deletes the node at the given path. +// If the node is a directory, recursive must be true to delete it. +func (fs *FileSystem) Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error) { + n, err := fs.InternalGet(nodePath, index, term) - if err != nil { + if err != nil { // if the node does not exist, return error return nil, err } - e := newEvent(Delete, keyPath, index, term) + e := newEvent(Delete, nodePath, index, term) if n.IsDir() { e.Dir = true @@ -206,11 +216,11 @@ func (fs *FileSystem) Delete(keyPath string, recurisive bool, index uint64, term e.PrevValue = n.Value } - callback := func(path string) { + callback := func(path string) { // notify function fs.WatcherHub.notifyWithPath(e, path, true) } - err = n.Remove(recurisive, callback) + err = n.Remove(recursive, callback) if err != nil { return nil, err @@ -221,9 +231,9 @@ func (fs *FileSystem) Delete(keyPath string, recurisive bool, index uint64, term return e, nil } -// walk function walks all the keyPath and apply the walkFunc on each directory -func (fs *FileSystem) walk(keyPath string, walkFunc func(prev *Node, component string) (*Node, error)) (*Node, error) { - components := strings.Split(keyPath, "/") +// walk function walks all the nodePath and apply the walkFunc on each directory +func (fs *FileSystem) walk(nodePath string, walkFunc func(prev *Node, component string) (*Node, error)) (*Node, error) { + components := strings.Split(nodePath, "/") curr := fs.Root @@ -243,9 +253,9 @@ func (fs *FileSystem) walk(keyPath string, walkFunc func(prev *Node, component s return curr, nil } -// InternalGet function get the node of the given keyPath. -func (fs *FileSystem) InternalGet(keyPath string, index uint64, term uint64) (*Node, error) { - keyPath = path.Clean("/" + keyPath) +// InternalGet function get the node of the given nodePath. +func (fs *FileSystem) InternalGet(nodePath string, index uint64, term uint64) (*Node, error) { + nodePath = path.Clean("/" + nodePath) // update file system known index and term fs.Index, fs.Term = index, term @@ -264,7 +274,7 @@ func (fs *FileSystem) InternalGet(keyPath string, index uint64, term uint64) (*N return nil, etcdErr.NewError(100, path.Join(parent.Path, name)) } - f, err := fs.walk(keyPath, walkFunc) + f, err := fs.walk(nodePath, walkFunc) if err != nil { return nil, err diff --git a/file_system/node.go b/file_system/node.go index 76126676867..3625847121c 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -34,9 +34,9 @@ type Node struct { stopExpire chan bool // stop expire routine channel } -func newFile(keyPath string, value string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { +func newFile(nodePath string, value string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { return &Node{ - Path: keyPath, + Path: nodePath, CreateIndex: createIndex, CreateTerm: createTerm, ModifiedIndex: createIndex, @@ -49,9 +49,9 @@ func newFile(keyPath string, value string, createIndex uint64, createTerm uint64 } } -func newDir(keyPath string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { +func newDir(nodePath string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { return &Node{ - Path: keyPath, + Path: nodePath, CreateIndex: createIndex, CreateTerm: createTerm, Parent: parent, diff --git a/file_system/watcher_test.go b/file_system/watcher_test.go index b817e64ec52..ea8eae78751 100644 --- a/file_system/watcher_test.go +++ b/file_system/watcher_test.go @@ -19,7 +19,7 @@ func TestWatch(t *testing.T) { // do nothing } - e := newEvent(Set, "/foo/bar", 1, 0) + e := newEvent(Create, "/foo/bar", 1, 0) wh.notify(e) @@ -31,7 +31,7 @@ func TestWatch(t *testing.T) { c, _ = wh.watch("/foo", false, 0) - e = newEvent(Set, "/foo/bar", 1, 0) + e = newEvent(Create, "/foo/bar", 1, 0) wh.notify(e) @@ -42,7 +42,7 @@ func TestWatch(t *testing.T) { // do nothing } - e = newEvent(Set, "/foo", 1, 0) + e = newEvent(Create, "/foo", 1, 0) wh.notify(e) From 08057fa6421374a31e2d8343cdc7ccfdcf5c1a87 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 7 Sep 2013 09:14:27 -0400 Subject: [PATCH 023/247] remove todo --- file_system/file_system.go | 1 - 1 file changed, 1 deletion(-) diff --git a/file_system/file_system.go b/file_system/file_system.go index 8edf0a06841..154f2d4648d 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -26,7 +26,6 @@ func New() *FileSystem { } func (fs *FileSystem) Get(nodePath string, recusive bool, index uint64, term uint64) (*Event, error) { - // TODO: add recursive get n, err := fs.InternalGet(nodePath, index, term) if err != nil { From 4f7011fc2b44ba6e215e05ca95df00ff75141473 Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Sat, 7 Sep 2013 10:40:19 -0400 Subject: [PATCH 024/247] add acl doc --- Documentation/etcd-file-system.md | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/Documentation/etcd-file-system.md b/Documentation/etcd-file-system.md index c2fc218a8fe..2be2c0eec80 100644 --- a/Documentation/etcd-file-system.md +++ b/Documentation/etcd-file-system.md @@ -58,7 +58,36 @@ Besides the file and directory difference, all nodes have common attributes and Set the node's expiration time to (current time + ttl) ## ACL + +### Theory +Etcd exports a Unix-like file system interface consisting of files and directories, collectively called nodes. +Each node has various meta-data, including three names of access control lists used to control reading, writing and changing (change ACL names for the node). + +We are storing the ACL names for nodes under a special *ACL* directory. +Each node has ACL name corresponding to one file within *ACL* dir. +Unless overridden, a node naturally inherits the ACL names of its parent directory on creation. + +For each ACL name, it has three children: *R (Reading)*, *W (Writing)*, *C (Changing)* + +Each permission is also a node. Under the node it contains the users who have this permission for the file refering to this ACL name. + +### Example +[TODO] +### Diagram [TODO] +### Interface + +Testing permissions: + +- (node *Node) get_perm() +- (node *Node) has_perm(perm string, user string) + +Setting/Changing permissions: + +- (node *Node) set_perm(perm string) +- (node *Node) change_ACLname(aclname string) + + ## User Group [TODO] From f41a9b9703f446859634d9604b5b401b98a9cda3 Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Sun, 8 Sep 2013 09:55:54 -0400 Subject: [PATCH 025/247] modify name. simplify test case. --- file_system/event.go | 23 ++++++++++++++--------- file_system/event_test.go | 6 +++++- 2 files changed, 19 insertions(+), 10 deletions(-) diff --git a/file_system/event.go b/file_system/event.go index d8fef7da8e7..1b2a00dbe43 100644 --- a/file_system/event.go +++ b/file_system/event.go @@ -49,17 +49,21 @@ type eventQueue struct { events []*Event size int front int - back int capacity int } +func (eq *eventQueue) back() int { + return (eq.front + eq.size - 1 + eq.capacity) % eq.capacity +} + func (eq *eventQueue) insert(e *Event) { - eq.back = (eq.back + 1) % eq.capacity - eq.events[eq.back] = e + index := (eq.back() + 1) % eq.capacity + + eq.events[index] = e if eq.size == eq.capacity { //dequeue - eq.front = (eq.back + 1) % eq.capacity + eq.front = (index + 1) % eq.capacity } else { eq.size++ } @@ -77,7 +81,6 @@ func newEventHistory(capacity int) *EventHistory { Queue: eventQueue{ capacity: capacity, events: make([]*Event, capacity), - back: -1, }, } } @@ -92,27 +95,29 @@ func (eh *EventHistory) addEvent(e *Event) { eh.StartIndex = eh.Queue.events[eh.Queue.front].Index } +// scan function is enumerating events from the index in history and +// stops till the first point where the key has identified prefix func (eh *EventHistory) scan(prefix string, index uint64) (*Event, error) { eh.rwl.RLock() defer eh.rwl.RUnlock() start := index - eh.StartIndex - if start < 0 { + // the index should locate after the event history's StartIndex + // and before its size + if start < 0 { // TODO: Add error type return nil, nil } if start >= uint64(eh.Queue.size) { - return nil, nil } i := int((start + uint64(eh.Queue.front)) % uint64(eh.Queue.capacity)) for { - e := eh.Queue.events[i] if strings.HasPrefix(e.Key, prefix) { return e, nil @@ -120,7 +125,7 @@ func (eh *EventHistory) scan(prefix string, index uint64) (*Event, error) { i = (i + 1) % eh.Queue.capacity - if i == eh.Queue.back { + if i == eh.Queue.back() { // TODO: Add error type return nil, nil } diff --git a/file_system/event_test.go b/file_system/event_test.go index 53df55ee546..fd069242842 100644 --- a/file_system/event_test.go +++ b/file_system/event_test.go @@ -19,12 +19,16 @@ func TestEventQueue(t *testing.T) { // Test j := 100 - for i := eh.Queue.front; i != eh.Queue.back; i = (i + 1) % eh.Queue.capacity { + i := eh.Queue.front + n := eh.Queue.size + for ; n > 0; n-- { e := eh.Queue.events[i] if e.Index != uint64(j) { t.Fatalf("queue error!") } j++ + i = (i + 1) % eh.Queue.capacity + } } From 9065c2e4f0875576bf6fbdbfff9b0d06d436c805 Mon Sep 17 00:00:00 2001 From: evan-gu Date: Sun, 8 Sep 2013 16:36:24 -0400 Subject: [PATCH 026/247] fix update dir ttl bug, add some test case --- file_system/file_system.go | 1 + file_system/file_system_test.go | 172 ++++++++++++++++++++++++++++---- file_system/watcher_test.go | 2 +- 3 files changed, 157 insertions(+), 18 deletions(-) diff --git a/file_system/file_system.go b/file_system/file_system.go index 154f2d4648d..d9e3e20a9c3 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -159,6 +159,7 @@ func (fs *FileSystem) Update(nodePath string, value string, expireTime time.Time } if expireTime != Permanent { + n.ExpireTime = expireTime go n.Expire() e.Expiration = &n.ExpireTime e.TTL = int64(expireTime.Sub(time.Now()) / time.Second) diff --git a/file_system/file_system_test.go b/file_system/file_system_test.go index 16666f61443..3956df0cdfc 100644 --- a/file_system/file_system_test.go +++ b/file_system/file_system_test.go @@ -55,7 +55,7 @@ func TestUpdateFile(t *testing.T) { _, err := fs.Create("/foo/bar", "bar", Permanent, 1, 1) if err != nil { - t.Fatalf("cannot update %s=bar [%s]", "/foo/bar", err.Error()) + t.Fatalf("cannot create %s=bar [%s]", "/foo/bar", err.Error()) } _, err = fs.Update("/foo/bar", "barbar", Permanent, 2, 1) @@ -73,6 +73,84 @@ func TestUpdateFile(t *testing.T) { if e.Value != "barbar" { t.Fatalf("expect value of %s is barbar [%s]", "/foo/bar", e.Value) } + + // create a directory, update its ttl, to see if it will be deleted + + _, err = fs.Create("/foo/foo", "", Permanent, 3, 1) + + if err != nil { + t.Fatalf("cannot create dir [%s] [%s]", "/foo/foo", err.Error()) + } + + _, err = fs.Create("/foo/foo/foo1", "bar1", Permanent, 4, 1) + + if err != nil { + t.Fatal("cannot create [%s]", err.Error()) + } + + _, err = fs.Create("/foo/foo/foo2", "", Permanent, 5, 1) + if err != nil { + t.Fatal("cannot create [%s]", err.Error()) + } + + _, err = fs.Create("/foo/foo/foo2/boo", "boo1", Permanent, 6, 1) + if err != nil { + t.Fatal("cannot create [%s]", err.Error()) + } + + expire := time.Now().Add(time.Second*2) + _, err = fs.Update("/foo/foo", "", expire, 7, 1) + if err != nil { + t.Fatalf("cannot update dir [%s] [%s]", "/foo/foo", err.Error()) + } + + // sleep 50ms, it should still reach the node + time.Sleep(time.Microsecond*50) + e, err = fs.Get("/foo/foo", true, 7, 1) + + if err != nil || e.Key != "/foo/foo" { + t.Fatalf("cannot get dir before expiration [%s]", err.Error()) + } + + if e.KVPairs[0].Key != "/foo/foo/foo1" || e.KVPairs[0].Value != "bar1" { + t.Fatalf("cannot get sub node before expiration [%s]", err.Error()) + } + + if e.KVPairs[1].Key != "/foo/foo/foo2" || e.KVPairs[1].Dir != true { + t.Fatalf("cannot get sub dir before expiration [%s]", err.Error()) + } + + /*if e.KVPairs[2].Key != "/foo/foo/foo2/boo" || e.KVPairs[2].Value != "boo1" { + t.Fatalf("cannot get sub node of sub dir before expiration [%s]", err.Error()) + }*/ + + // wait for expiration + time.Sleep(time.Second*3) + e, err = fs.Get("/foo/foo", true, 7, 1) + + if err == nil { + t.Fatal("still can get dir after expiration [%s]") + } + + _, err = fs.Get("/foo/foo/foo1", true, 7, 1) + if err == nil { + t.Fatal("still can get sub node after expiration [%s]") + } + + _, err = fs.Get("/foo/foo/foo2", true, 7, 1) + if err == nil { + t.Fatal("still can get sub dir after expiration [%s]") + } + + _, err = fs.Get("/foo/foo/foo2/boo", true, 7, 1) + if err == nil { + t.Fatalf("still can get sub node of sub dir after expiration [%s]", err.Error()) + } + + + + + } func TestListDirectory(t *testing.T) { @@ -181,23 +259,30 @@ func TestExpire(t *testing.T) { t.Fatalf("can get the node after expiration time") } + // test if we can reach the node before expiration + expire = time.Now().Add(time.Second) fs.Create("/foo", "bar", expire, 1, 1) time.Sleep(time.Millisecond * 50) _, err = fs.InternalGet("/foo", 1, 1) - if err == nil { - t.Fatalf("can get the node after expiration time") + if err != nil { + t.Fatalf("cannot get the node before expiration", err.Error()) } expire = time.Now().Add(time.Second) - + fs.Create("/foo", "bar", expire, 1, 1) - fs.Delete("/foo", false, 1, 1) + _, err = fs.Delete("/foo", false, 1, 1) + + if err != nil { + t.Fatalf("cannot delete the node before expiration", err.Error()) + } + } -func TestTestAndSet(t *testing.T) { +func TestTestAndSet(t *testing.T) { // TODO prevValue == nil ? fs := New() fs.Create("/foo", "bar", Permanent, 1, 1) @@ -228,29 +313,73 @@ func TestTestAndSet(t *testing.T) { if e.PrevValue != "car" || e.Value != "bar" { t.Fatalf("[%v/%v] [%v/%v]", e.PrevValue, "car", e.Value, "bar") } + + //e, err = fs.TestAndSet("/foo", ) } -func TestWatchRemove(t *testing.T) { +func TestWatch(t *testing.T) { fs := New() - fs.Create("/foo/foo/foo", "bar", Permanent, 1, 1) - // watch at a deeper path c, _ := fs.WatcherHub.watch("/foo/foo/foo", false, 0) - fs.Delete("/foo", true, 2, 1) - e := <-c + fs.Create("/foo/foo/foo", "bar", Permanent, 1, 1) + + e := nonblockingRetrive(c) + if e.Key != "/foo/foo/foo" { + t.Fatal("watch for Create node fails") + } + + c, _ = fs.WatcherHub.watch("/foo/foo/foo", false, 0) + fs.Update("/foo/foo/foo", "car", Permanent, 2, 1) + e = nonblockingRetrive(c) + if e.Key != "/foo/foo/foo" { + t.Fatal("watch for Update node fails") + } + + c, _ = fs.WatcherHub.watch("/foo/foo/foo", false, 0) + fs.TestAndSet("/foo/foo/foo", "car", 0, "bar", Permanent, 3, 1) + e = nonblockingRetrive(c) + if e.Key != "/foo/foo/foo" { + t.Fatal("watch for TestAndSet node fails") + } + + c, _ = fs.WatcherHub.watch("/foo/foo/foo", false, 0) + fs.Delete("/foo", true, 4, 1) //recursively delete + e = nonblockingRetrive(c) if e.Key != "/foo" { - t.Fatal("watch for delete fails") + t.Fatal("watch for Delete node fails") } - fs.Create("/foo/foo/foo", "bar", Permanent, 3, 1) + // watch at a prefix c, _ = fs.WatcherHub.watch("/foo", true, 0) - fs.Delete("/foo/foo/foo", false, 4, 1) - e = <-c - if e.Key != "/foo/foo/foo" { - t.Fatal("watch for delete fails") + fs.Create("/foo/foo/boo", "bar", Permanent, 5, 1) + e = nonblockingRetrive(c) + if e.Key != "/foo/foo/boo" { + t.Fatal("watch for Create subdirectory fails") } + c, _ = fs.WatcherHub.watch("/foo", true, 0) + fs.Update("/foo/foo/boo", "foo", Permanent, 6, 1) + e = nonblockingRetrive(c) + if e.Key != "/foo/foo/boo" { + t.Fatal("watch for Update subdirectory fails") + } + + c, _ = fs.WatcherHub.watch("/foo", true, 0) + fs.TestAndSet("/foo/foo/boo", "foo", 0, "bar", Permanent, 7, 1) + e = nonblockingRetrive(c) + if e.Key != "/foo/foo/boo" { + t.Fatal("watch for TestAndSet subdirectory fails") + } + + c, _ = fs.WatcherHub.watch("/foo", true, 0) + fs.Delete("/foo/foo/boo", false, 8, 1) + e = nonblockingRetrive(c) + if e.Key != "/foo/foo/boo" { + t.Fatal("watch for Delete subdirectory fails") + } + + } func createAndGet(fs *FileSystem, path string, t *testing.T) { @@ -271,3 +400,12 @@ func createAndGet(fs *FileSystem, path string, t *testing.T) { } } + +func nonblockingRetrive(c <-chan *Event) *Event{ + select { + case e := <-c: + return e + default: + return nil + } +} \ No newline at end of file diff --git a/file_system/watcher_test.go b/file_system/watcher_test.go index ea8eae78751..f874163bac4 100644 --- a/file_system/watcher_test.go +++ b/file_system/watcher_test.go @@ -4,7 +4,7 @@ import ( "testing" ) -func TestWatch(t *testing.T) { +func TestWatcher(t *testing.T) { wh := newWatchHub(100) c, err := wh.watch("/foo", true, 0) From 9a632351a653498c3a1527f74a2655623cc41240 Mon Sep 17 00:00:00 2001 From: evan-gu Date: Sun, 8 Sep 2013 16:37:24 -0400 Subject: [PATCH 027/247] gofmt --- file_system/file_system_test.go | 31 ++++++++++++------------------- 1 file changed, 12 insertions(+), 19 deletions(-) diff --git a/file_system/file_system_test.go b/file_system/file_system_test.go index 3956df0cdfc..4c70b35bb13 100644 --- a/file_system/file_system_test.go +++ b/file_system/file_system_test.go @@ -87,27 +87,27 @@ func TestUpdateFile(t *testing.T) { if err != nil { t.Fatal("cannot create [%s]", err.Error()) } - + _, err = fs.Create("/foo/foo/foo2", "", Permanent, 5, 1) if err != nil { t.Fatal("cannot create [%s]", err.Error()) } - + _, err = fs.Create("/foo/foo/foo2/boo", "boo1", Permanent, 6, 1) if err != nil { t.Fatal("cannot create [%s]", err.Error()) } - expire := time.Now().Add(time.Second*2) + expire := time.Now().Add(time.Second * 2) _, err = fs.Update("/foo/foo", "", expire, 7, 1) if err != nil { t.Fatalf("cannot update dir [%s] [%s]", "/foo/foo", err.Error()) } // sleep 50ms, it should still reach the node - time.Sleep(time.Microsecond*50) + time.Sleep(time.Microsecond * 50) e, err = fs.Get("/foo/foo", true, 7, 1) - + if err != nil || e.Key != "/foo/foo" { t.Fatalf("cannot get dir before expiration [%s]", err.Error()) } @@ -125,9 +125,9 @@ func TestUpdateFile(t *testing.T) { }*/ // wait for expiration - time.Sleep(time.Second*3) + time.Sleep(time.Second * 3) e, err = fs.Get("/foo/foo", true, 7, 1) - + if err == nil { t.Fatal("still can get dir after expiration [%s]") } @@ -147,10 +147,6 @@ func TestUpdateFile(t *testing.T) { t.Fatalf("still can get sub node of sub dir after expiration [%s]", err.Error()) } - - - - } func TestListDirectory(t *testing.T) { @@ -271,14 +267,13 @@ func TestExpire(t *testing.T) { } expire = time.Now().Add(time.Second) - + fs.Create("/foo", "bar", expire, 1, 1) _, err = fs.Delete("/foo", false, 1, 1) if err != nil { t.Fatalf("cannot delete the node before expiration", err.Error()) } - } @@ -322,12 +317,12 @@ func TestWatch(t *testing.T) { // watch at a deeper path c, _ := fs.WatcherHub.watch("/foo/foo/foo", false, 0) fs.Create("/foo/foo/foo", "bar", Permanent, 1, 1) - + e := nonblockingRetrive(c) if e.Key != "/foo/foo/foo" { t.Fatal("watch for Create node fails") } - + c, _ = fs.WatcherHub.watch("/foo/foo/foo", false, 0) fs.Update("/foo/foo/foo", "car", Permanent, 2, 1) e = nonblockingRetrive(c) @@ -349,7 +344,6 @@ func TestWatch(t *testing.T) { t.Fatal("watch for Delete node fails") } - // watch at a prefix c, _ = fs.WatcherHub.watch("/foo", true, 0) fs.Create("/foo/foo/boo", "bar", Permanent, 5, 1) @@ -379,7 +373,6 @@ func TestWatch(t *testing.T) { t.Fatal("watch for Delete subdirectory fails") } - } func createAndGet(fs *FileSystem, path string, t *testing.T) { @@ -401,11 +394,11 @@ func createAndGet(fs *FileSystem, path string, t *testing.T) { } -func nonblockingRetrive(c <-chan *Event) *Event{ +func nonblockingRetrive(c <-chan *Event) *Event { select { case e := <-c: return e default: return nil } -} \ No newline at end of file +} From effc8285f2f985a275d499b4acd04203076bba63 Mon Sep 17 00:00:00 2001 From: Hongchao Deng Date: Sun, 8 Sep 2013 18:46:16 -0400 Subject: [PATCH 028/247] New error-system for Etcd with docs --- Documentation/errorcode.md | 58 ++++++++++++++++++++++++++++++++++++++ command.go | 2 +- error/error.go | 32 ++++++++++++++++----- etcd_handlers.go | 18 ++++++------ file_system/event.go | 7 ++++- file_system/file_system.go | 12 ++++---- file_system/node.go | 28 ++++++++++-------- 7 files changed, 122 insertions(+), 35 deletions(-) create mode 100644 Documentation/errorcode.md diff --git a/Documentation/errorcode.md b/Documentation/errorcode.md new file mode 100644 index 00000000000..3a0443f8bc6 --- /dev/null +++ b/Documentation/errorcode.md @@ -0,0 +1,58 @@ +Error Code +====== + +This document describes the error code in **Etcd** project. + +It's categorized into four groups: + +- Command Related Error +- Post Form Related Error +- Raft Related Error +- Etcd Related Error + +Error code corresponding strerror +------ + + const ( + EcodeKeyNotFound = 100 + EcodeTestFailed = 101 + EcodeNotFile = 102 + EcodeNoMoreMachine = 103 + EcodeNotDir = 104 + EcodeNodeExist = 105 + EcodeKeyIsPreserved = 106 + + EcodeValueRequired = 200 + EcodePrevValueRequired = 201 + EcodeTTLNaN = 202 + EcodeIndexNaN = 203 + + EcodeRaftInternal = 300 + EcodeLeaderElect = 301 + + EcodeWatcherCleared = 400 + EcodeEventIndexCleared = 401 + ) + + // command related errors + errors[100] = "Key Not Found" + errors[101] = "Test Failed" //test and set + errors[102] = "Not A File" + errors[103] = "Reached the max number of machines in the cluster" + errors[104] = "Not A Directory" + errors[105] = "Already exists" // create + errors[106] = "The prefix of given key is a keyword in etcd" + + // Post form related errors + errors[200] = "Value is Required in POST form" + errors[201] = "PrevValue is Required in POST form" + errors[202] = "The given TTL in POST form is not a number" + errors[203] = "The given index in POST form is not a number" + + // raft related errors + errors[300] = "Raft Internal Error" + errors[301] = "During Leader Election" + + // etcd related errors + errors[400] = "watcher is cleared due to etcd recovery" + errors[401] = "The event in requested index is outdated and cleared" diff --git a/command.go b/command.go index b9c3a83f610..d03b4dc2eb9 100644 --- a/command.go +++ b/command.go @@ -156,7 +156,7 @@ func (c *JoinCommand) Apply(raftServer *raft.Server) (interface{}, error) { num := machineNum() if num == maxClusterSize { debug("Reject join request from ", c.Name) - return []byte{0}, etcdErr.NewError(103, "") + return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMoreMachine, "") } addNameToURL(c.Name, c.RaftVersion, c.RaftURL, c.EtcdURL) diff --git a/error/error.go b/error/error.go index baa395fb9cc..d97158cf1a9 100644 --- a/error/error.go +++ b/error/error.go @@ -7,18 +7,38 @@ import ( var errors map[int]string -const () +const ( + EcodeKeyNotFound = 100 + EcodeTestFailed = 101 + EcodeNotFile = 102 + EcodeNoMoreMachine = 103 + EcodeNotDir = 104 + EcodeNodeExist = 105 + EcodeKeyIsPreserved = 106 + + EcodeValueRequired = 200 + EcodePrevValueRequired = 201 + EcodeTTLNaN = 202 + EcodeIndexNaN = 203 + + EcodeRaftInternal = 300 + EcodeLeaderElect = 301 + + EcodeWatcherCleared = 400 + EcodeEventIndexCleared = 401 +) func init() { errors = make(map[int]string) // command related errors errors[100] = "Key Not Found" - errors[101] = "Test Failed" + errors[101] = "Test Failed" //test and set errors[102] = "Not A File" errors[103] = "Reached the max number of machines in the cluster" errors[104] = "Not A Directory" - errors[105] = "Already exists" + errors[105] = "Already exists" // create + errors[106] = "The prefix of given key is a keyword in etcd" // Post form related errors errors[200] = "Value is Required in POST form" @@ -30,11 +50,9 @@ func init() { errors[300] = "Raft Internal Error" errors[301] = "During Leader Election" - // keyword - errors[400] = "The prefix of the given key is a keyword in etcd" - // etcd related errors - errors[500] = "watcher is cleared due to etcd recovery" + errors[400] = "watcher is cleared due to etcd recovery" + errors[401] = "The event in requested index is outdated and cleared" } diff --git a/etcd_handlers.go b/etcd_handlers.go index c0e82530a06..7b8b0829ee9 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -68,7 +68,7 @@ func SetHttpHandler(w http.ResponseWriter, req *http.Request) error { key := req.URL.Path[len("/v1/keys/"):] if store.CheckKeyword(key) { - return etcdErr.NewError(400, "Set") + return etcdErr.NewError(etcdErr.EcodeKeyIsPreserved, "Set") } debugf("[recv] POST %v/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) @@ -76,7 +76,7 @@ func SetHttpHandler(w http.ResponseWriter, req *http.Request) error { value := req.FormValue("value") if len(value) == 0 { - return etcdErr.NewError(200, "Set") + return etcdErr.NewError(etcdErr.EcodeValueRequired, "Set") } prevValue := req.FormValue("prevValue") @@ -86,7 +86,7 @@ func SetHttpHandler(w http.ResponseWriter, req *http.Request) error { expireTime, err := durationToExpireTime(strDuration) if err != nil { - return etcdErr.NewError(202, "Set") + return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Set") } if len(prevValue) != 0 { @@ -131,7 +131,7 @@ func dispatch(c Command, w http.ResponseWriter, req *http.Request, etcd bool) er return err } else { if body == nil { - return etcdErr.NewError(300, "Empty result from raft") + return etcdErr.NewError(etcdErr.EcodeRaftInternal, "Empty result from raft") } else { body, _ := body.([]byte) w.WriteHeader(http.StatusOK) @@ -144,7 +144,7 @@ func dispatch(c Command, w http.ResponseWriter, req *http.Request, etcd bool) er leader := r.Leader() // current no leader if leader == "" { - return etcdErr.NewError(300, "") + return etcdErr.NewError(etcdErr.EcodeRaftInternal, "") } // tell the client where is the leader @@ -165,7 +165,7 @@ func dispatch(c Command, w http.ResponseWriter, req *http.Request, etcd bool) er http.Redirect(w, req, url, http.StatusTemporaryRedirect) return nil } - return etcdErr.NewError(300, "") + return etcdErr.NewError(etcdErr.EcodeRaftInternal, "") } //-------------------------------------- @@ -185,7 +185,7 @@ func dispatch(c Command, w http.ResponseWriter, req *http.Request, etcd bool) er // w.Write([]byte(raftURL)) // return nil // } else { -// return etcdErr.NewError(301, "") +// return etcdErr.NewError(etcdErr.EcodeLeaderElect, "") // } // } @@ -254,7 +254,7 @@ func WatchHttpHandler(w http.ResponseWriter, req *http.Request) error { sinceIndex, err := strconv.ParseUint(string(content), 10, 64) if err != nil { - return etcdErr.NewError(203, "Watch From Index") + return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index") } command.SinceIndex = sinceIndex @@ -264,7 +264,7 @@ func WatchHttpHandler(w http.ResponseWriter, req *http.Request) error { } if body, err := command.Apply(r.Server); err != nil { - return etcdErr.NewError(500, key) + return etcdErr.NewError(etcdErr.EcodeWatcherCleared, key) } else { w.WriteHeader(http.StatusOK) diff --git a/file_system/event.go b/file_system/event.go index 1b2a00dbe43..8bea67de2e8 100644 --- a/file_system/event.go +++ b/file_system/event.go @@ -1,6 +1,8 @@ package fileSystem import ( + "fmt" + etcdErr "github.com/coreos/etcd/error" "strings" "sync" "time" @@ -108,7 +110,10 @@ func (eh *EventHistory) scan(prefix string, index uint64) (*Event, error) { if start < 0 { // TODO: Add error type - return nil, nil + return nil, + etcdErr.NewError(etcdErr.EcodeEventIndexCleared, + fmt.Sprintf("prefix:%v index:%v", prefix, index), + ) } if start >= uint64(eh.Queue.size) { diff --git a/file_system/file_system.go b/file_system/file_system.go index 154f2d4648d..5f97a03d826 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -74,7 +74,7 @@ func (fs *FileSystem) Create(nodePath string, value string, expireTime time.Time _, err := fs.InternalGet(nodePath, index, term) if err == nil { // key already exists - return nil, etcdErr.NewError(105, nodePath) + return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath) } etcdError, _ := err.(etcdErr.Error) @@ -140,7 +140,7 @@ func (fs *FileSystem) Update(nodePath string, value string, expireTime time.Time if n.IsDir() { // if the node is a directory, we can only update ttl if len(value) != 0 { - return nil, etcdErr.NewError(102, nodePath) + return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath) } } else { // if the node is a file, we can update value and ttl @@ -179,7 +179,7 @@ func (fs *FileSystem) TestAndSet(nodePath string, prevValue string, prevIndex ui } if f.IsDir() { // can only test and set file - return nil, etcdErr.NewError(102, nodePath) + return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath) } if f.Value == prevValue || f.ModifiedIndex == prevIndex { @@ -195,7 +195,7 @@ func (fs *FileSystem) TestAndSet(nodePath string, prevValue string, prevIndex ui } cause := fmt.Sprintf("[%v/%v] [%v/%v]", prevValue, f.Value, prevIndex, f.ModifiedIndex) - return nil, etcdErr.NewError(101, cause) + return nil, etcdErr.NewError(etcdErr.EcodeTestFailed, cause) } // Delete function deletes the node at the given path. @@ -262,7 +262,7 @@ func (fs *FileSystem) InternalGet(nodePath string, index uint64, term uint64) (* walkFunc := func(parent *Node, name string) (*Node, error) { if !parent.IsDir() { - return nil, etcdErr.NewError(104, parent.Path) + return nil, etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path) } child, ok := parent.Children[name] @@ -270,7 +270,7 @@ func (fs *FileSystem) InternalGet(nodePath string, index uint64, term uint64) (* return child, nil } - return nil, etcdErr.NewError(100, path.Join(parent.Path, name)) + return nil, etcdErr.NewError(etcdErr.EcodeKeyNotFound, path.Join(parent.Path, name)) } f, err := fs.walk(nodePath, walkFunc) diff --git a/file_system/node.go b/file_system/node.go index 3625847121c..de39c5545cd 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -93,7 +93,7 @@ func (n *Node) Remove(recursive bool, callback func(path string)) error { } if !recursive { - return etcdErr.NewError(102, "") + return etcdErr.NewError(etcdErr.EcodeNotFile, "") } for _, child := range n.Children { // delete all children @@ -116,21 +116,21 @@ func (n *Node) Remove(recursive bool, callback func(path string)) error { return nil } -// Get function gets the value of the node. +// Read function gets the value of the node. // If the receiver node is not a key-value pair, a "Not A File" error will be returned. func (n *Node) Read() (string, error) { if n.IsDir() { - return "", etcdErr.NewError(102, "") + return "", etcdErr.NewError(etcdErr.EcodeNotFile, "") } return n.Value, nil } -// Set function set the value of the node to the given value. +// Write function set the value of the node to the given value. // If the receiver node is a directory, a "Not A File" error will be returned. func (n *Node) Write(value string, index uint64, term uint64) error { if n.IsDir() { - return etcdErr.NewError(102, "") + return etcdErr.NewError(etcdErr.EcodeNotFile, "") } n.Value = value @@ -146,7 +146,7 @@ func (n *Node) List() ([]*Node, error) { n.mu.Lock() defer n.mu.Unlock() if !n.IsDir() { - return nil, etcdErr.NewError(104, "") + return nil, etcdErr.NewError(etcdErr.EcodeNotDir, "") } nodes := make([]*Node, len(n.Children)) @@ -160,12 +160,18 @@ func (n *Node) List() ([]*Node, error) { return nodes, nil } +// GetFile function returns the file node under the directory node. +// On success, it returns the file node +// If the node that calls this function is not a directory, it returns +// Not Directory Error +// If the node corresponding to the name string is not file, it returns +// Not File Error func (n *Node) GetFile(name string) (*Node, error) { n.mu.Lock() defer n.mu.Unlock() if !n.IsDir() { - return nil, etcdErr.NewError(104, n.Path) + return nil, etcdErr.NewError(etcdErr.EcodeNotDir, n.Path) } f, ok := n.Children[name] @@ -174,7 +180,7 @@ func (n *Node) GetFile(name string) (*Node, error) { if !f.IsDir() { return f, nil } else { - return nil, etcdErr.NewError(102, f.Path) + return nil, etcdErr.NewError(etcdErr.EcodeNotFile, f.Path) } } @@ -190,11 +196,11 @@ func (n *Node) Add(child *Node) error { n.mu.Lock() defer n.mu.Unlock() if n.status == removed { - return etcdErr.NewError(100, "") + return etcdErr.NewError(etcdErr.EcodeKeyNotFound, "") } if !n.IsDir() { - return etcdErr.NewError(104, "") + return etcdErr.NewError(etcdErr.EcodeNotDir, "") } _, name := path.Split(child.Path) @@ -202,7 +208,7 @@ func (n *Node) Add(child *Node) error { _, ok := n.Children[name] if ok { - return etcdErr.NewError(105, "") + return etcdErr.NewError(etcdErr.EcodeNodeExist, "") } n.Children[name] = child From 1a7b3e8e089b5f0b71ab94c87c7e0c760a0696a1 Mon Sep 17 00:00:00 2001 From: evan-gu Date: Sun, 8 Sep 2013 21:14:31 -0400 Subject: [PATCH 029/247] add sorting and its test case --- file_system/event.go | 13 ++++++++ file_system/file_system.go | 6 +++- file_system/file_system_test.go | 54 +++++++++++++++++++++++++-------- store/test.go | 2 +- 4 files changed, 61 insertions(+), 14 deletions(-) diff --git a/file_system/event.go b/file_system/event.go index 1b2a00dbe43..16ea275a277 100644 --- a/file_system/event.go +++ b/file_system/event.go @@ -36,6 +36,19 @@ type KeyValuePair struct { KVPairs []KeyValuePair `json:"kvs,omitempty"` } +// interfaces for sort +func (e Event) Len() int { + return len(e.KVPairs) +} + +func (e Event) Less(i, j int) bool { + return e.KVPairs[i].Key < e.KVPairs[j].Key +} + +func (e Event) Swap(i, j int) { + e.KVPairs[i], e.KVPairs[j] = e.KVPairs[j], e.KVPairs[i] +} + func newEvent(action string, key string, index uint64, term uint64) *Event { return &Event{ Action: action, diff --git a/file_system/file_system.go b/file_system/file_system.go index d9e3e20a9c3..d2735966c89 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -3,6 +3,7 @@ package fileSystem import ( "fmt" "path" + "sort" "strings" "time" @@ -25,7 +26,7 @@ func New() *FileSystem { } -func (fs *FileSystem) Get(nodePath string, recusive bool, index uint64, term uint64) (*Event, error) { +func (fs *FileSystem) Get(nodePath string, recusive, sorting bool, index uint64, term uint64) (*Event, error) { n, err := fs.InternalGet(nodePath, index, term) if err != nil { @@ -57,6 +58,9 @@ func (fs *FileSystem) Get(nodePath string, recusive bool, index uint64, term uin // eliminate hidden nodes e.KVPairs = e.KVPairs[:i] + if sorting { + sort.Sort(e) + } } else { // node is file e.Value = n.Value } diff --git a/file_system/file_system_test.go b/file_system/file_system_test.go index 4c70b35bb13..caa10045142 100644 --- a/file_system/file_system_test.go +++ b/file_system/file_system_test.go @@ -1,6 +1,7 @@ package fileSystem import ( + "store" "testing" "time" ) @@ -34,7 +35,7 @@ func TestCreateAndGet(t *testing.T) { t.Fatal("Cannot create /fooDir") } - e, err := fs.Get("/fooDir", false, 3, 1) + e, err := fs.Get("/fooDir", false, false, 3, 1) if err != nil || e.Dir != true { t.Fatal("Cannot create /fooDir ") @@ -64,7 +65,7 @@ func TestUpdateFile(t *testing.T) { t.Fatalf("cannot update %s=barbar [%s]", "/foo/bar", err.Error()) } - e, err := fs.Get("/foo/bar", false, 2, 1) + e, err := fs.Get("/foo/bar", false, false, 2, 1) if err != nil { t.Fatalf("cannot get %s [%s]", "/foo/bar", err.Error()) @@ -106,7 +107,7 @@ func TestUpdateFile(t *testing.T) { // sleep 50ms, it should still reach the node time.Sleep(time.Microsecond * 50) - e, err = fs.Get("/foo/foo", true, 7, 1) + e, err = fs.Get("/foo/foo", true, false, 7, 1) if err != nil || e.Key != "/foo/foo" { t.Fatalf("cannot get dir before expiration [%s]", err.Error()) @@ -126,23 +127,23 @@ func TestUpdateFile(t *testing.T) { // wait for expiration time.Sleep(time.Second * 3) - e, err = fs.Get("/foo/foo", true, 7, 1) + e, err = fs.Get("/foo/foo", true, false, 7, 1) if err == nil { t.Fatal("still can get dir after expiration [%s]") } - _, err = fs.Get("/foo/foo/foo1", true, 7, 1) + _, err = fs.Get("/foo/foo/foo1", true, false, 7, 1) if err == nil { t.Fatal("still can get sub node after expiration [%s]") } - _, err = fs.Get("/foo/foo/foo2", true, 7, 1) + _, err = fs.Get("/foo/foo/foo2", true, false, 7, 1) if err == nil { t.Fatal("still can get sub dir after expiration [%s]") } - _, err = fs.Get("/foo/foo/foo2/boo", true, 7, 1) + _, err = fs.Get("/foo/foo/foo2/boo", true, false, 7, 1) if err == nil { t.Fatalf("still can get sub node of sub dir after expiration [%s]", err.Error()) } @@ -160,7 +161,7 @@ func TestListDirectory(t *testing.T) { // set key-value /foo/fooDir/foo=bar fs.Create("/foo/fooDir/foo", "bar", Permanent, 2, 1) - e, err := fs.Get("/foo", true, 2, 1) + e, err := fs.Get("/foo", true, false, 2, 1) if err != nil { t.Fatalf("%v", err) @@ -187,7 +188,7 @@ func TestListDirectory(t *testing.T) { // set key-value /foo/_hidden/foo -> bar fs.Create("/foo/_hidden/foo", "bar", Permanent, 3, 1) - e, _ = fs.Get("/foo", false, 2, 1) + e, _ = fs.Get("/foo", false, false, 2, 1) if len(e.KVPairs) != 2 { t.Fatalf("hidden node is not hidden! %s", e.KVPairs[2].Key) @@ -204,7 +205,7 @@ func TestRemove(t *testing.T) { t.Fatalf("cannot delete %s [%s]", "/foo", err.Error()) } - _, err = fs.Get("/foo", false, 1, 1) + _, err = fs.Get("/foo", false, false, 1, 1) if err == nil || err.Error() != "Key Not Found" { t.Fatalf("can get the node after deletion") @@ -226,7 +227,7 @@ func TestRemove(t *testing.T) { t.Fatalf("cannot delete %s [%s]", "/foo", err.Error()) } - _, err = fs.Get("/foo", false, 1, 1) + _, err = fs.Get("/foo", false, false, 1, 1) if err == nil || err.Error() != "Key Not Found" { t.Fatalf("can get the node after deletion ") @@ -382,7 +383,7 @@ func createAndGet(fs *FileSystem, path string, t *testing.T) { t.Fatalf("cannot create %s=bar [%s]", path, err.Error()) } - e, err := fs.Get(path, false, 1, 1) + e, err := fs.Get(path, false, false, 1, 1) if err != nil { t.Fatalf("cannot get %s [%s]", path, err.Error()) @@ -402,3 +403,32 @@ func nonblockingRetrive(c <-chan *Event) *Event { return nil } } + +func TestSort(t *testing.T) { + fs := New() + + // simulating random creation + keys := store.GenKeys(100, 5) + + //t.Log(keys) + i := uint64(1) + for _, k := range keys { + _, err := fs.Create(k, "bar", Permanent, i, 1) + if err != nil { + t.Fatalf("create node[%s] failed", k, err.Error()) + } + i++ + } + + e, err := fs.Get("/foo", true, true, i, 1) + if err != nil { + t.Fatalf("get dir nodes failed [%s]", err.Error()) + } + + for i, k := range e.KVPairs[:len(e.KVPairs)-1] { + //t.Log(k) + if k.Key >= e.KVPairs[i+1].Key { + t.Fatalf("sort failed, [%s] should be placed after [%s]", k.Key, e.KVPairs[i+1].Key) + } + } +} diff --git a/store/test.go b/store/test.go index ac23261bef6..5b922ffcb58 100644 --- a/store/test.go +++ b/store/test.go @@ -10,7 +10,7 @@ func GenKeys(num int, depth int) []string { keys := make([]string, num) for i := 0; i < num; i++ { - keys[i] = "/foo/" + keys[i] = "/foo" depth := rand.Intn(depth) + 1 for j := 0; j < depth; j++ { From 38489bd8460a161f80eb401f5d3f8819f026c03f Mon Sep 17 00:00:00 2001 From: evan-gu Date: Sun, 8 Sep 2013 21:21:57 -0400 Subject: [PATCH 030/247] add documentation about sorting, change the argument name from sorting to sorted --- Documentation/etcd-file-system.md | 3 ++- file_system/file_system.go | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/Documentation/etcd-file-system.md b/Documentation/etcd-file-system.md index c48cfecd4f9..092ecdee642 100644 --- a/Documentation/etcd-file-system.md +++ b/Documentation/etcd-file-system.md @@ -23,12 +23,13 @@ Besides the file and directory difference, all nodes have common attributes and The path of access control list of the node. ### Operation: -- **Get** (path, recursive) +- **Get** (path, recursive, sorted) Get the content of the node - If the node is a file, the data of the file will be returned. - If the node is a directory, the child nodes of the directory will be returned. - If recursive is true, it will recursively get the nodes of the directory. + - If sorted is true, the result will be sorted based on the path. - **Create** (path, value[optional], ttl [optional]) diff --git a/file_system/file_system.go b/file_system/file_system.go index d2735966c89..51023ef86c3 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -26,7 +26,7 @@ func New() *FileSystem { } -func (fs *FileSystem) Get(nodePath string, recusive, sorting bool, index uint64, term uint64) (*Event, error) { +func (fs *FileSystem) Get(nodePath string, recusive, sorted bool, index uint64, term uint64) (*Event, error) { n, err := fs.InternalGet(nodePath, index, term) if err != nil { @@ -58,7 +58,7 @@ func (fs *FileSystem) Get(nodePath string, recusive, sorting bool, index uint64, // eliminate hidden nodes e.KVPairs = e.KVPairs[:i] - if sorting { + if sorted { sort.Sort(e) } } else { // node is file From 643a92a4901eca282a299a1228d0afff2eb3f6c8 Mon Sep 17 00:00:00 2001 From: evan-gu Date: Sun, 8 Sep 2013 23:13:28 -0400 Subject: [PATCH 031/247] now it will sort recursively, and the sorting test case is better.... --- file_system/event.go | 14 ++++----- file_system/file_system.go | 12 ++++++-- file_system/file_system_test.go | 54 ++++++++++++++++++++++++++++++--- file_system/node.go | 9 ++++-- store/test.go | 2 +- 5 files changed, 73 insertions(+), 18 deletions(-) diff --git a/file_system/event.go b/file_system/event.go index 16ea275a277..363730bd884 100644 --- a/file_system/event.go +++ b/file_system/event.go @@ -36,17 +36,17 @@ type KeyValuePair struct { KVPairs []KeyValuePair `json:"kvs,omitempty"` } -// interfaces for sort -func (e Event) Len() int { - return len(e.KVPairs) +// interfaces for sorting +func (k KeyValuePair) Len() int { + return len(k.KVPairs) } -func (e Event) Less(i, j int) bool { - return e.KVPairs[i].Key < e.KVPairs[j].Key +func (k KeyValuePair) Less(i, j int) bool { + return k.KVPairs[i].Key < k.KVPairs[j].Key } -func (e Event) Swap(i, j int) { - e.KVPairs[i], e.KVPairs[j] = e.KVPairs[j], e.KVPairs[i] +func (k KeyValuePair) Swap(i, j int) { + k.KVPairs[i], k.KVPairs[j] = k.KVPairs[j], k.KVPairs[i] } func newEvent(action string, key string, index uint64, term uint64) *Event { diff --git a/file_system/file_system.go b/file_system/file_system.go index 51023ef86c3..c50311d365e 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -26,7 +26,7 @@ func New() *FileSystem { } -func (fs *FileSystem) Get(nodePath string, recusive, sorted bool, index uint64, term uint64) (*Event, error) { +func (fs *FileSystem) Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) { n, err := fs.InternalGet(nodePath, index, term) if err != nil { @@ -51,16 +51,22 @@ func (fs *FileSystem) Get(nodePath string, recusive, sorted bool, index uint64, continue } - e.KVPairs[i] = child.Pair(recusive) + e.KVPairs[i] = child.Pair(recursive, sorted) i++ } // eliminate hidden nodes e.KVPairs = e.KVPairs[:i] + + rootPairs := KeyValuePair{ + KVPairs: e.KVPairs, + } + if sorted { - sort.Sort(e) + sort.Sort(rootPairs) } + } else { // node is file e.Value = n.Value } diff --git a/file_system/file_system_test.go b/file_system/file_system_test.go index caa10045142..e887fd9da63 100644 --- a/file_system/file_system_test.go +++ b/file_system/file_system_test.go @@ -1,7 +1,8 @@ package fileSystem import ( - "store" + "math/rand" + "strconv" "testing" "time" ) @@ -408,16 +409,17 @@ func TestSort(t *testing.T) { fs := New() // simulating random creation - keys := store.GenKeys(100, 5) + keys := GenKeys(80, 4) //t.Log(keys) i := uint64(1) for _, k := range keys { _, err := fs.Create(k, "bar", Permanent, i, 1) if err != nil { - t.Fatalf("create node[%s] failed", k, err.Error()) + //t.Logf("create node[%s] failed %s", k, err.Error()) + } else { + i++ } - i++ } e, err := fs.Get("/foo", true, true, i, 1) @@ -426,9 +428,53 @@ func TestSort(t *testing.T) { } for i, k := range e.KVPairs[:len(e.KVPairs)-1] { + //t.Log("root:") //t.Log(k) if k.Key >= e.KVPairs[i+1].Key { t.Fatalf("sort failed, [%s] should be placed after [%s]", k.Key, e.KVPairs[i+1].Key) } + + if k.Dir { + recursiveTestSort(k, t) + } + + } + + if k := e.KVPairs[len(e.KVPairs)-1]; k.Dir { + recursiveTestSort(k, t) + } +} + +func recursiveTestSort(k KeyValuePair, t *testing.T) { + //t.Log("recursive in") + //t.Log(k) + for i, v := range k.KVPairs[:len(k.KVPairs)-1] { + if v.Key >= k.KVPairs[i+1].Key { + t.Fatalf("sort failed, [%s] should be placed after [%s]", v.Key, k.KVPairs[i+1].Key) + } + + if v.Dir { + recursiveTestSort(v, t) + } + + } + + if v := k.KVPairs[len(k.KVPairs)-1]; v.Dir { + recursiveTestSort(v, t) + } +} + +// GenKeys randomly generate num of keys with max depth +func GenKeys(num int, depth int) []string { + keys := make([]string, num) + for i := 0; i < num; i++ { + + keys[i] = "/foo" + depth := rand.Intn(depth) + 1 + + for j := 0; j < depth; j++ { + keys[i] += "/" + strconv.Itoa(rand.Int()%20) + } } + return keys } diff --git a/file_system/node.go b/file_system/node.go index 3625847121c..2da22b66589 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -3,6 +3,7 @@ package fileSystem import ( "fmt" "path" + "sort" "sync" "time" @@ -279,7 +280,7 @@ func (n *Node) IsHidden() bool { return false } -func (n *Node) Pair(recurisive bool) KeyValuePair { +func (n *Node) Pair(recurisive, sorted bool) KeyValuePair { if n.IsDir() { pair := KeyValuePair{ @@ -303,14 +304,16 @@ func (n *Node) Pair(recurisive bool) KeyValuePair { continue } - pair.KVPairs[i] = child.Pair(recurisive) + pair.KVPairs[i] = child.Pair(recurisive, sorted) i++ } // eliminate hidden nodes pair.KVPairs = pair.KVPairs[:i] - + if sorted { + sort.Sort(pair) + } return pair } diff --git a/store/test.go b/store/test.go index 5b922ffcb58..eaddaa69dbf 100644 --- a/store/test.go +++ b/store/test.go @@ -14,7 +14,7 @@ func GenKeys(num int, depth int) []string { depth := rand.Intn(depth) + 1 for j := 0; j < depth; j++ { - keys[i] += "/" + strconv.Itoa(rand.Int()) + keys[i] += "/" + strconv.Itoa(rand.Int()%20) } } return keys From 2c9c278e4d9a9cbc27ced845662bc3ff3521aa67 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Fri, 13 Sep 2013 17:10:40 -0400 Subject: [PATCH 032/247] refactor; add save and recover --- etcd_handlers.go | 24 +++--- file_system/event.go | 37 +++++---- file_system/event_test.go | 8 +- file_system/file_system.go | 51 +++++++++--- file_system/file_system_test.go | 140 ++++++++++++++++++++++---------- file_system/node.go | 78 ++++++++++++------ file_system/watcher.go | 3 +- 7 files changed, 226 insertions(+), 115 deletions(-) diff --git a/etcd_handlers.go b/etcd_handlers.go index 7b8b0829ee9..6b5203ec435 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -176,18 +176,18 @@ func dispatch(c Command, w http.ResponseWriter, req *http.Request, etcd bool) er //-------------------------------------- // Handler to return the current leader's raft address -// func LeaderHttpHandler(w http.ResponseWriter, req *http.Request) error { -// leader := r.Leader() - -// if leader != "" { -// w.WriteHeader(http.StatusOK) -// raftURL, _ := nameToRaftURL(leader) -// w.Write([]byte(raftURL)) -// return nil -// } else { -// return etcdErr.NewError(etcdErr.EcodeLeaderElect, "") -// } -// } +func LeaderHttpHandler(w http.ResponseWriter, req *http.Request) error { + leader := r.Leader() + + if leader != "" { + w.WriteHeader(http.StatusOK) + raftURL, _ := nameToRaftURL(leader) + w.Write([]byte(raftURL)) + return nil + } else { + return etcdErr.NewError(etcdErr.EcodeLeaderElect, "") + } +} // Handler to return all the known machines in the current cluster func MachinesHttpHandler(w http.ResponseWriter, req *http.Request) error { diff --git a/file_system/event.go b/file_system/event.go index f3d92ebb760..f5b3eeb858d 100644 --- a/file_system/event.go +++ b/file_system/event.go @@ -2,10 +2,11 @@ package fileSystem import ( "fmt" - etcdErr "github.com/coreos/etcd/error" "strings" "sync" "time" + + etcdErr "github.com/coreos/etcd/error" ) const ( @@ -61,26 +62,26 @@ func newEvent(action string, key string, index uint64, term uint64) *Event { } type eventQueue struct { - events []*Event - size int - front int - capacity int + Events []*Event + Size int + Front int + Capacity int } func (eq *eventQueue) back() int { - return (eq.front + eq.size - 1 + eq.capacity) % eq.capacity + return (eq.Front + eq.Size - 1 + eq.Capacity) % eq.Capacity } func (eq *eventQueue) insert(e *Event) { - index := (eq.back() + 1) % eq.capacity + index := (eq.back() + 1) % eq.Capacity - eq.events[index] = e + eq.Events[index] = e - if eq.size == eq.capacity { //dequeue - eq.front = (index + 1) % eq.capacity + if eq.Size == eq.Capacity { //dequeue + eq.Front = (index + 1) % eq.Capacity } else { - eq.size++ + eq.Size++ } } @@ -94,8 +95,8 @@ type EventHistory struct { func newEventHistory(capacity int) *EventHistory { return &EventHistory{ Queue: eventQueue{ - capacity: capacity, - events: make([]*Event, capacity), + Capacity: capacity, + Events: make([]*Event, capacity), }, } } @@ -107,7 +108,7 @@ func (eh *EventHistory) addEvent(e *Event) { eh.Queue.insert(e) - eh.StartIndex = eh.Queue.events[eh.Queue.front].Index + eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index } // scan function is enumerating events from the index in history and @@ -129,19 +130,19 @@ func (eh *EventHistory) scan(prefix string, index uint64) (*Event, error) { ) } - if start >= uint64(eh.Queue.size) { + if start >= uint64(eh.Queue.Size) { return nil, nil } - i := int((start + uint64(eh.Queue.front)) % uint64(eh.Queue.capacity)) + i := int((start + uint64(eh.Queue.Front)) % uint64(eh.Queue.Capacity)) for { - e := eh.Queue.events[i] + e := eh.Queue.Events[i] if strings.HasPrefix(e.Key, prefix) { return e, nil } - i = (i + 1) % eh.Queue.capacity + i = (i + 1) % eh.Queue.Capacity if i == eh.Queue.back() { // TODO: Add error type diff --git a/file_system/event_test.go b/file_system/event_test.go index fd069242842..146f8cd4db3 100644 --- a/file_system/event_test.go +++ b/file_system/event_test.go @@ -19,15 +19,15 @@ func TestEventQueue(t *testing.T) { // Test j := 100 - i := eh.Queue.front - n := eh.Queue.size + i := eh.Queue.Front + n := eh.Queue.Size for ; n > 0; n-- { - e := eh.Queue.events[i] + e := eh.Queue.Events[i] if e.Index != uint64(j) { t.Fatalf("queue error!") } j++ - i = (i + 1) % eh.Queue.capacity + i = (i + 1) % eh.Queue.Capacity } diff --git a/file_system/file_system.go b/file_system/file_system.go index d45c8d1bc31..5d11f8f20bb 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -1,6 +1,7 @@ package fileSystem import ( + "encoding/json" "fmt" "path" "sort" @@ -11,11 +12,10 @@ import ( ) type FileSystem struct { - Root *Node - EventHistory *EventHistory - WatcherHub *watcherHub - Index uint64 - Term uint64 + Root *Node + WatcherHub *watcherHub + Index uint64 + Term uint64 } func New() *FileSystem { @@ -126,7 +126,7 @@ func (fs *FileSystem) Create(nodePath string, value string, expireTime time.Time // Node with TTL if expireTime != Permanent { - go n.Expire() + n.Expire() e.Expiration = &n.ExpireTime e.TTL = int64(expireTime.Sub(time.Now()) / time.Second) } @@ -164,13 +164,13 @@ func (fs *FileSystem) Update(nodePath string, value string, expireTime time.Time } // update ttl - if n.ExpireTime != Permanent && expireTime != Permanent { + if !n.IsPermanent() && expireTime != Permanent { n.stopExpire <- true } - if expireTime != Permanent { + if expireTime.Sub(Permanent) != 0 { n.ExpireTime = expireTime - go n.Expire() + n.Expire() e.Expiration = &n.ExpireTime e.TTL = int64(expireTime.Sub(time.Now()) / time.Second) } @@ -298,7 +298,6 @@ func (fs *FileSystem) InternalGet(nodePath string, index uint64, term uint64) (* // If it does not exist, this function will create a new directory and return the pointer to that node. // If it is a file, this function will return error. func (fs *FileSystem) checkDir(parent *Node, dirName string) (*Node, error) { - subDir, ok := parent.Children[dirName] if ok { @@ -311,3 +310,35 @@ func (fs *FileSystem) checkDir(parent *Node, dirName string) (*Node, error) { return n, nil } + +// Save function saves the static state of the store system. +// Save function will not be able to save the state of watchers. +// Save function will not save the parent field of the node. Or there will +// be cyclic dependencies issue for the json package. +func (fs *FileSystem) Save() []byte { + cloneFs := New() + cloneFs.Root = fs.Root.Clone() + + b, err := json.Marshal(fs) + + if err != nil { + panic(err) + } + + return b +} + +// recovery function recovery the store system from a static state. +// It needs to recovery the parent field of the nodes. +// It needs to delete the expired nodes since the saved time and also +// need to create monitor go routines. +func (fs *FileSystem) Recover(state []byte) { + err := json.Unmarshal(state, fs) + + if err != nil { + panic(err) + } + + fs.Root.recoverAndclean() + +} diff --git a/file_system/file_system_test.go b/file_system/file_system_test.go index e887fd9da63..4da413da9e2 100644 --- a/file_system/file_system_test.go +++ b/file_system/file_system_test.go @@ -311,7 +311,6 @@ func TestTestAndSet(t *testing.T) { // TODO prevValue == nil ? t.Fatalf("[%v/%v] [%v/%v]", e.PrevValue, "car", e.Value, "bar") } - //e, err = fs.TestAndSet("/foo", ) } func TestWatch(t *testing.T) { @@ -377,46 +376,17 @@ func TestWatch(t *testing.T) { } -func createAndGet(fs *FileSystem, path string, t *testing.T) { - _, err := fs.Create(path, "bar", Permanent, 1, 1) - - if err != nil { - t.Fatalf("cannot create %s=bar [%s]", path, err.Error()) - } - - e, err := fs.Get(path, false, false, 1, 1) - - if err != nil { - t.Fatalf("cannot get %s [%s]", path, err.Error()) - } - - if e.Value != "bar" { - t.Fatalf("expect value of %s is bar [%s]", path, e.Value) - } - -} - -func nonblockingRetrive(c <-chan *Event) *Event { - select { - case e := <-c: - return e - default: - return nil - } -} - func TestSort(t *testing.T) { fs := New() // simulating random creation keys := GenKeys(80, 4) - //t.Log(keys) i := uint64(1) for _, k := range keys { _, err := fs.Create(k, "bar", Permanent, i, 1) if err != nil { - //t.Logf("create node[%s] failed %s", k, err.Error()) + panic(err) } else { i++ } @@ -428,8 +398,7 @@ func TestSort(t *testing.T) { } for i, k := range e.KVPairs[:len(e.KVPairs)-1] { - //t.Log("root:") - //t.Log(k) + if k.Key >= e.KVPairs[i+1].Key { t.Fatalf("sort failed, [%s] should be placed after [%s]", k.Key, e.KVPairs[i+1].Key) } @@ -445,23 +414,60 @@ func TestSort(t *testing.T) { } } -func recursiveTestSort(k KeyValuePair, t *testing.T) { - //t.Log("recursive in") - //t.Log(k) - for i, v := range k.KVPairs[:len(k.KVPairs)-1] { - if v.Key >= k.KVPairs[i+1].Key { - t.Fatalf("sort failed, [%s] should be placed after [%s]", v.Key, k.KVPairs[i+1].Key) +func TestSaveAndRecover(t *testing.T) { + fs := New() + + // simulating random creation + keys := GenKeys(8, 4) + + i := uint64(1) + for _, k := range keys { + _, err := fs.Create(k, "bar", Permanent, i, 1) + if err != nil { + panic(err) + } else { + i++ } + } - if v.Dir { - recursiveTestSort(v, t) + // create a node with expiration + // test if we can reach the node before expiration + + expire := time.Now().Add(time.Second) + fs.Create("/foo/foo", "bar", expire, 1, 1) + + b := fs.Save() + + cloneFs := New() + + time.Sleep(time.Second) + + cloneFs.Recover(b) + + for i, k := range keys { + _, err := cloneFs.Get(k, false, false, uint64(i), 1) + if err != nil { + panic(err) } + } + if fs.WatcherHub.EventHistory.StartIndex != cloneFs.WatcherHub.EventHistory.StartIndex { + t.Fatal("Error recovered event history start index") } - if v := k.KVPairs[len(k.KVPairs)-1]; v.Dir { - recursiveTestSort(v, t) + for i = 0; int(i) < fs.WatcherHub.EventHistory.Queue.Size; i++ { + if fs.WatcherHub.EventHistory.Queue.Events[i].Key != + cloneFs.WatcherHub.EventHistory.Queue.Events[i].Key { + t.Fatal("Error recovered event history") + } + } + + _, err := fs.Get("/foo/foo", false, false, 1, 1) + + if err == nil || err.Error() != "Key Not Found" { + t.Fatalf("can get the node after deletion ") } + } // GenKeys randomly generate num of keys with max depth @@ -473,8 +479,54 @@ func GenKeys(num int, depth int) []string { depth := rand.Intn(depth) + 1 for j := 0; j < depth; j++ { - keys[i] += "/" + strconv.Itoa(rand.Int()%20) + keys[i] += "/" + strconv.Itoa(rand.Int()) } } return keys } + +func createAndGet(fs *FileSystem, path string, t *testing.T) { + _, err := fs.Create(path, "bar", Permanent, 1, 1) + + if err != nil { + t.Fatalf("cannot create %s=bar [%s]", path, err.Error()) + } + + e, err := fs.Get(path, false, false, 1, 1) + + if err != nil { + t.Fatalf("cannot get %s [%s]", path, err.Error()) + } + + if e.Value != "bar" { + t.Fatalf("expect value of %s is bar [%s]", path, e.Value) + } + +} + +func recursiveTestSort(k KeyValuePair, t *testing.T) { + + for i, v := range k.KVPairs[:len(k.KVPairs)-1] { + if v.Key >= k.KVPairs[i+1].Key { + t.Fatalf("sort failed, [%s] should be placed after [%s]", v.Key, k.KVPairs[i+1].Key) + } + + if v.Dir { + recursiveTestSort(v, t) + } + + } + + if v := k.KVPairs[len(k.KVPairs)-1]; v.Dir { + recursiveTestSort(v, t) + } +} + +func nonblockingRetrive(c <-chan *Event) *Event { + select { + case e := <-c: + return e + default: + return nil + } +} diff --git a/file_system/node.go b/file_system/node.go index 406cc98d6d5..e4f42ef59f3 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -1,7 +1,6 @@ package fileSystem import ( - "fmt" "path" "sort" "sync" @@ -25,7 +24,7 @@ type Node struct { CreateTerm uint64 ModifiedIndex uint64 ModifiedTerm uint64 - Parent *Node + Parent *Node `json:"-"` ExpireTime time.Time ACL string Value string // for key-value pair @@ -237,62 +236,89 @@ func (n *Node) Clone() *Node { return clone } -// IsDir function checks whether the node is a directory. -// If the node is a directory, the function will return true. -// Otherwise the function will return false. -func (n *Node) IsDir() bool { - - if n.Children == nil { // key-value pair - return false +func (n *Node) recoverAndclean() { + if n.IsDir() { + for _, child := range n.Children { + child.Parent = n + child.recoverAndclean() + } } - return true + n.stopExpire = make(chan bool, 1) + + n.Expire() } func (n *Node) Expire() { - duration := n.ExpireTime.Sub(time.Now()) - if duration <= 0 { + expired, duration := n.IsExpired() + + if expired { // has been expired n.Remove(true, nil) return } - select { - // if timeout, delete the node - case <-time.After(duration): - n.Remove(true, nil) + if duration == 0 { // Permanent Node return + } - // if stopped, return - case <-n.stopExpire: - fmt.Println("expire stopped") - return + go func() { // do monitoring + select { + // if timeout, delete the node + case <-time.After(duration): + n.Remove(true, nil) + return - } + // if stopped, return + case <-n.stopExpire: + return + + } + }() } // IsHidden function checks if the node is a hidden node. A hidden node // will begin with '_' - // A hidden node will not be shown via get command under a directory // For example if we have /foo/_hidden and /foo/notHidden, get "/foo" // will only return /foo/notHidden func (n *Node) IsHidden() bool { _, name := path.Split(n.Path) - if name[0] == '_' { //hidden - return true + return name[0] == '_' + +} + +func (n *Node) IsPermanent() bool { + return n.ExpireTime.Sub(Permanent) == 0 +} + +func (n *Node) IsExpired() (bool, time.Duration) { + if n.IsPermanent() { + return false, 0 + } + + duration := n.ExpireTime.Sub(time.Now()) + if duration <= 0 { + return true, 0 } - return false + return false, duration } -func (n *Node) Pair(recurisive, sorted bool) KeyValuePair { +// IsDir function checks whether the node is a directory. +// If the node is a directory, the function will return true. +// Otherwise the function will return false. +func (n *Node) IsDir() bool { + return !(n.Children == nil) +} +func (n *Node) Pair(recurisive, sorted bool) KeyValuePair { if n.IsDir() { pair := KeyValuePair{ Key: n.Path, Dir: true, } + if !recurisive { return pair } diff --git a/file_system/watcher.go b/file_system/watcher.go index c17d0eb877a..728d11fd413 100644 --- a/file_system/watcher.go +++ b/file_system/watcher.go @@ -99,7 +99,6 @@ func (wh *watcherHub) notifyWithPath(e *Event, path string, force bool) { } func (wh *watcherHub) notify(e *Event) { - segments := strings.Split(e.Key, "/") currPath := "/" @@ -109,4 +108,6 @@ func (wh *watcherHub) notify(e *Event) { currPath = path.Join(currPath, segment) wh.notifyWithPath(e, currPath, false) } + + wh.EventHistory.addEvent(e) } From 3ff100321c98fbdaf0001b804f102b82c842a24b Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 14 Sep 2013 15:13:33 -0400 Subject: [PATCH 033/247] use new store system --- command.go | 110 ++++++++++++++++---- etcd.go | 9 +- etcd_handlers.go | 176 +++++++++++++++++++------------- etcd_test.go | 5 +- file_system/file_system.go | 31 ++++-- file_system/file_system_test.go | 12 ++- raft_handlers.go | 3 +- raft_server.go | 3 + util.go | 8 +- 9 files changed, 249 insertions(+), 108 deletions(-) diff --git a/command.go b/command.go index d03b4dc2eb9..0a7cc1c9703 100644 --- a/command.go +++ b/command.go @@ -4,12 +4,12 @@ import ( "encoding/binary" "encoding/json" "fmt" - etcdErr "github.com/coreos/etcd/error" - "github.com/coreos/etcd/store" - "github.com/coreos/go-raft" "os" "path" "time" + + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/go-raft" ) const commandPrefix = "etcd:" @@ -24,6 +24,54 @@ type Command interface { Apply(server *raft.Server) (interface{}, error) } +// Create command +type CreateCommand struct { + Key string `json:"key"` + Value string `json:"value"` + ExpireTime time.Time `json:"expireTime"` +} + +// The name of the create command in the log +func (c *CreateCommand) CommandName() string { + return commandName("create") +} + +// Create node +func (c *CreateCommand) Apply(server *raft.Server) (interface{}, error) { + e, err := etcdFs.Create(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) + + if err != nil { + debug(err) + return nil, err + } + + return json.Marshal(e) +} + +// Update command +type UpdateCommand struct { + Key string `json:"key"` + Value string `json:"value"` + ExpireTime time.Time `json:"expireTime"` +} + +// The name of the update command in the log +func (c *UpdateCommand) CommandName() string { + return commandName("update") +} + +// Update node +func (c *UpdateCommand) Apply(server *raft.Server) (interface{}, error) { + e, err := etcdFs.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) + + if err != nil { + debug(err) + return nil, err + } + + return json.Marshal(e) +} + // Set command type SetCommand struct { Key string `json:"key"` @@ -45,8 +93,9 @@ func (c *SetCommand) Apply(server *raft.Server) (interface{}, error) { type TestAndSetCommand struct { Key string `json:"key"` Value string `json:"value"` - PrevValue string `json: prevValue` ExpireTime time.Time `json:"expireTime"` + PrevValue string `json: prevValue` + PrevIndex uint64 `json: prevValue` } // The name of the testAndSet command in the log @@ -56,12 +105,22 @@ func (c *TestAndSetCommand) CommandName() string { // Set the key-value pair if the current value of the key equals to the given prevValue func (c *TestAndSetCommand) Apply(server *raft.Server) (interface{}, error) { - return etcdStore.TestAndSet(c.Key, c.PrevValue, c.Value, c.ExpireTime, server.CommitIndex()) + e, err := etcdFs.TestAndSet(c.Key, c.PrevValue, c.PrevIndex, + c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) + + if err != nil { + debug(err) + return nil, err + } + + return json.Marshal(e) } // Get command type GetCommand struct { - Key string `json:"key"` + Key string `json:"key"` + Recursive bool `json:"recursive"` + Sorted bool `json:"sorted"` } // The name of the get command in the log @@ -71,12 +130,20 @@ func (c *GetCommand) CommandName() string { // Get the value of key func (c *GetCommand) Apply(server *raft.Server) (interface{}, error) { - return etcdStore.Get(c.Key) + e, err := etcdFs.Get(c.Key, c.Recursive, c.Sorted, server.CommitIndex(), server.Term()) + + if err != nil { + debug(err) + return nil, err + } + + return json.Marshal(e) } // Delete command type DeleteCommand struct { - Key string `json:"key"` + Key string `json:"key"` + Recursive bool `json:"recursive"` } // The name of the delete command in the log @@ -86,13 +153,21 @@ func (c *DeleteCommand) CommandName() string { // Delete the key func (c *DeleteCommand) Apply(server *raft.Server) (interface{}, error) { - return etcdStore.Delete(c.Key, server.CommitIndex()) + e, err := etcdFs.Delete(c.Key, c.Recursive, server.CommitIndex(), server.Term()) + + if err != nil { + debug(err) + return nil, err + } + + return json.Marshal(e) } // Watch command type WatchCommand struct { Key string `json:"key"` SinceIndex uint64 `json:"sinceIndex"` + Recursive bool `json:"recursive"` } // The name of the watch command in the log @@ -101,20 +176,15 @@ func (c *WatchCommand) CommandName() string { } func (c *WatchCommand) Apply(server *raft.Server) (interface{}, error) { - // create a new watcher - watcher := store.NewWatcher() - - // add to the watchers list - etcdStore.AddWatcher(c.Key, watcher, c.SinceIndex) + eventChan, err := etcdFs.Watch(c.Key, c.Recursive, c.SinceIndex, server.CommitIndex(), server.Term()) - // wait for the notification for any changing - res := <-watcher.C - - if res == nil { - return nil, fmt.Errorf("Clearing watch") + if err != nil { + return nil, err } - return json.Marshal(res) + e := <-eventChan + + return json.Marshal(e) } // JoinCommand diff --git a/etcd.go b/etcd.go index 46546e8cc0d..bffb4e02e1b 100644 --- a/etcd.go +++ b/etcd.go @@ -3,12 +3,14 @@ package main import ( "crypto/tls" "flag" - "github.com/coreos/etcd/store" - "github.com/coreos/go-raft" "io/ioutil" "os" "strings" "time" + + "github.com/coreos/etcd/file_system" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" ) //------------------------------------------------------------------------------ @@ -129,6 +131,7 @@ type TLSConfig struct { //------------------------------------------------------------------------------ var etcdStore *store.Store +var etcdFs *fileSystem.FileSystem //------------------------------------------------------------------------------ // @@ -195,6 +198,8 @@ func main() { // Create etcd key-value store etcdStore = store.CreateStore(maxSize) + etcdFs = fileSystem.New() + snapConf = newSnapshotConf() // Create etcd and raft server diff --git a/etcd_handlers.go b/etcd_handlers.go index 6b5203ec435..e1c6a8b9e35 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -2,12 +2,12 @@ package main import ( "fmt" - etcdErr "github.com/coreos/etcd/error" - "github.com/coreos/etcd/store" - "github.com/coreos/go-raft" "net/http" "strconv" "strings" + + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/go-raft" ) //------------------------------------------------------------------- @@ -18,7 +18,6 @@ func NewEtcdMuxer() *http.ServeMux { // external commands etcdMux := http.NewServeMux() etcdMux.Handle("/"+version+"/keys/", errorHandler(Multiplexer)) - etcdMux.Handle("/"+version+"/watch/", errorHandler(WatchHttpHandler)) etcdMux.Handle("/"+version+"/leader", errorHandler(LeaderHttpHandler)) etcdMux.Handle("/"+version+"/machines", errorHandler(MachinesHttpHandler)) etcdMux.Handle("/"+version+"/stats", errorHandler(StatsHttpHandler)) @@ -47,15 +46,16 @@ func Multiplexer(w http.ResponseWriter, req *http.Request) error { case "GET": return GetHttpHandler(w, req) case "POST": - return SetHttpHandler(w, req) + return CreateHttpHandler(w, req) case "PUT": - return SetHttpHandler(w, req) + return UpdateHttpHandler(w, req) case "DELETE": return DeleteHttpHandler(w, req) default: w.WriteHeader(http.StatusMethodNotAllowed) return nil } + return nil } //-------------------------------------- @@ -63,63 +63,102 @@ func Multiplexer(w http.ResponseWriter, req *http.Request) error { // Set/Delete will dispatch to leader //-------------------------------------- -// Set Command Handler -func SetHttpHandler(w http.ResponseWriter, req *http.Request) error { - key := req.URL.Path[len("/v1/keys/"):] - - if store.CheckKeyword(key) { - return etcdErr.NewError(etcdErr.EcodeKeyIsPreserved, "Set") - } +func CreateHttpHandler(w http.ResponseWriter, req *http.Request) error { + key := req.URL.Path[len("/v2/keys"):] - debugf("[recv] POST %v/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) + debugf("recv.post[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) value := req.FormValue("value") - if len(value) == 0 { - return etcdErr.NewError(etcdErr.EcodeValueRequired, "Set") + ttl := req.FormValue("ttl") + + expireTime, err := durationToExpireTime(ttl) + + if err != nil { + return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create") } - prevValue := req.FormValue("prevValue") + command := &CreateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + } + + return dispatch(command, w, req, true) + +} - strDuration := req.FormValue("ttl") +func UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { + key := req.URL.Path[len("/v2/keys"):] - expireTime, err := durationToExpireTime(strDuration) + debugf("recv.put[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) + + value := req.FormValue("value") + + ttl := req.FormValue("ttl") + + expireTime, err := durationToExpireTime(ttl) if err != nil { - return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Set") + return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update") } - if len(prevValue) != 0 { - command := &TestAndSetCommand{ + // TODO: update should give at least one option + if value == "" && ttl == "" { + return nil + } + + prevValue := req.FormValue("prevValue") + + prevIndexStr := req.FormValue("prevIndex") + + if prevValue == "" && prevIndexStr == "" { // update without test + command := &UpdateCommand{ Key: key, Value: value, - PrevValue: prevValue, ExpireTime: expireTime, } return dispatch(command, w, req, true) - } else { - command := &SetCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, + } else { // update with test + var prevIndex uint64 + + if prevIndexStr != "" { + prevIndex, err = strconv.ParseUint(prevIndexStr, 10, 64) + } + + // TODO: add error type + if err != nil { + return nil + } + + command := &TestAndSetCommand{ + Key: key, + Value: value, + PrevValue: prevValue, + PrevIndex: prevIndex, } return dispatch(command, w, req, true) } + } // Delete Handler func DeleteHttpHandler(w http.ResponseWriter, req *http.Request) error { - key := req.URL.Path[len("/v1/keys/"):] + key := req.URL.Path[len("/v2/keys"):] - debugf("[recv] DELETE %v/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) + debugf("recv.delete[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) command := &DeleteCommand{ Key: key, } + if req.FormValue("recursive") == "true" { + command.Recursive = true + } + return dispatch(command, w, req, true) } @@ -212,64 +251,63 @@ func StatsHttpHandler(w http.ResponseWriter, req *http.Request) error { return nil } -// Get Handler func GetHttpHandler(w http.ResponseWriter, req *http.Request) error { - key := req.URL.Path[len("/v1/keys/"):] + var err error + var event interface{} + key := req.URL.Path[len("/v1/keys"):] - debugf("[recv] GET %s/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) + debugf("recv.get[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) - command := &GetCommand{ - Key: key, - } + recursive := req.FormValue("recursive") - if body, err := command.Apply(r.Server); err != nil { - return err - } else { - body, _ := body.([]byte) - w.WriteHeader(http.StatusOK) - w.Write(body) + if req.FormValue("wait") == "true" { + command := &WatchCommand{ + Key: key, + } - return nil - } + if recursive == "true" { + command.Recursive = true + } -} + indexStr := req.FormValue("wait_index") -// Watch handler -func WatchHttpHandler(w http.ResponseWriter, req *http.Request) error { - key := req.URL.Path[len("/v1/watch/"):] + if indexStr != "" { + sinceIndex, err := strconv.ParseUint(indexStr, 10, 64) - command := &WatchCommand{ - Key: key, - } + if err != nil { + return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index") + } - if req.Method == "GET" { - debugf("[recv] GET %s/watch/%s [%s]", e.url, key, req.RemoteAddr) - command.SinceIndex = 0 + command.SinceIndex = sinceIndex + } - } else if req.Method == "POST" { - // watch from a specific index + event, err = command.Apply(r.Server) - debugf("[recv] POST %s/watch/%s [%s]", e.url, key, req.RemoteAddr) - content := req.FormValue("index") + } else { + command := &GetCommand{ + Key: key, + } - sinceIndex, err := strconv.ParseUint(string(content), 10, 64) - if err != nil { - return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index") + sorted := req.FormValue("sorted") + + if sorted == "true" { + command.Sorted = true } - command.SinceIndex = sinceIndex - } else { - w.WriteHeader(http.StatusMethodNotAllowed) - return nil + if recursive == "true" { + command.Recursive = true + } + + event, err = command.Apply(r.Server) } - if body, err := command.Apply(r.Server); err != nil { - return etcdErr.NewError(etcdErr.EcodeWatcherCleared, key) + if err != nil { + return err } else { + event, _ := event.([]byte) w.WriteHeader(http.StatusOK) + w.Write(event) - body, _ := body.([]byte) - w.Write(body) return nil } diff --git a/etcd_test.go b/etcd_test.go index e61e7e4a845..caa6af84d22 100644 --- a/etcd_test.go +++ b/etcd_test.go @@ -2,8 +2,6 @@ package main import ( "fmt" - "github.com/coreos/etcd/test" - "github.com/coreos/go-etcd/etcd" "math/rand" "net/http" "net/http/httptest" @@ -13,6 +11,9 @@ import ( "strings" "testing" "time" + + "github.com/coreos/etcd/test" + "github.com/coreos/go-etcd/etcd" ) // Create a single node and try to set value diff --git a/file_system/file_system.go b/file_system/file_system.go index 5d11f8f20bb..ee67f9655d5 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -27,6 +27,8 @@ func New() *FileSystem { } func (fs *FileSystem) Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) { + nodePath = path.Clean(path.Join("/", nodePath)) + n, err := fs.InternalGet(nodePath, index, term) if err != nil { @@ -71,6 +73,11 @@ func (fs *FileSystem) Get(nodePath string, recursive, sorted bool, index uint64, e.Value = n.Value } + if n.ExpireTime.Sub(Permanent) != 0 { + e.Expiration = &n.ExpireTime + e.TTL = int64(n.ExpireTime.Sub(time.Now())/time.Second) + 1 + } + return e, nil } @@ -78,7 +85,7 @@ func (fs *FileSystem) Get(nodePath string, recursive, sorted bool, index uint64, // If the node has already existed, create will fail. // If any node on the path is a file, create will fail. func (fs *FileSystem) Create(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { - nodePath = path.Clean("/" + nodePath) + nodePath = path.Clean(path.Join("/", nodePath)) // make sure we can create the node _, err := fs.InternalGet(nodePath, index, term) @@ -125,10 +132,10 @@ func (fs *FileSystem) Create(nodePath string, value string, expireTime time.Time } // Node with TTL - if expireTime != Permanent { + if expireTime.Sub(Permanent) != 0 { n.Expire() e.Expiration = &n.ExpireTime - e.TTL = int64(expireTime.Sub(time.Now()) / time.Second) + e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 } fs.WatcherHub.notify(e) @@ -164,7 +171,7 @@ func (fs *FileSystem) Update(nodePath string, value string, expireTime time.Time } // update ttl - if !n.IsPermanent() && expireTime != Permanent { + if !n.IsPermanent() { n.stopExpire <- true } @@ -172,7 +179,7 @@ func (fs *FileSystem) Update(nodePath string, value string, expireTime time.Time n.ExpireTime = expireTime n.Expire() e.Expiration = &n.ExpireTime - e.TTL = int64(expireTime.Sub(time.Now()) / time.Second) + e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 } fs.WatcherHub.notify(e) @@ -205,7 +212,7 @@ func (fs *FileSystem) TestAndSet(nodePath string, prevValue string, prevIndex ui return e, nil } - cause := fmt.Sprintf("[%v/%v] [%v/%v]", prevValue, f.Value, prevIndex, f.ModifiedIndex) + cause := fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, f.Value, prevIndex, f.ModifiedIndex) return nil, etcdErr.NewError(etcdErr.EcodeTestFailed, cause) } @@ -241,6 +248,16 @@ func (fs *FileSystem) Delete(nodePath string, recursive bool, index uint64, term return e, nil } +func (fs *FileSystem) Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error) { + fs.Index, fs.Term = index, term + + if sinceIndex == 0 { + return fs.WatcherHub.watch(prefix, recursive, index+1) + } + + return fs.WatcherHub.watch(prefix, recursive, sinceIndex) +} + // walk function walks all the nodePath and apply the walkFunc on each directory func (fs *FileSystem) walk(nodePath string, walkFunc func(prev *Node, component string) (*Node, error)) (*Node, error) { components := strings.Split(nodePath, "/") @@ -265,7 +282,7 @@ func (fs *FileSystem) walk(nodePath string, walkFunc func(prev *Node, component // InternalGet function get the node of the given nodePath. func (fs *FileSystem) InternalGet(nodePath string, index uint64, term uint64) (*Node, error) { - nodePath = path.Clean("/" + nodePath) + nodePath = path.Clean(path.Join("/", nodePath)) // update file system known index and term fs.Index, fs.Term = index, term diff --git a/file_system/file_system_test.go b/file_system/file_system_test.go index 4da413da9e2..d36c86e2a55 100644 --- a/file_system/file_system_test.go +++ b/file_system/file_system_test.go @@ -10,10 +10,7 @@ import ( func TestCreateAndGet(t *testing.T) { fs := New() - // this should create successfully - createAndGet(fs, "/foobar", t) - createAndGet(fs, "/foo/bar", t) - createAndGet(fs, "/foo/foo/bar", t) + fs.Create("/foobar", "bar", Permanent, 1, 1) // already exist, create should fail _, err := fs.Create("/foobar", "bar", Permanent, 1, 1) @@ -22,6 +19,13 @@ func TestCreateAndGet(t *testing.T) { t.Fatal("Create should fail") } + fs.Delete("/foobar", true, 1, 1) + + // this should create successfully + createAndGet(fs, "/foobar", t) + createAndGet(fs, "/foo/bar", t) + createAndGet(fs, "/foo/foo/bar", t) + // meet file, create should fail _, err = fs.Create("/foo/bar/bar", "bar", Permanent, 2, 1) diff --git a/raft_handlers.go b/raft_handlers.go index 8ae9d2f8739..1a92560ec32 100644 --- a/raft_handlers.go +++ b/raft_handlers.go @@ -2,8 +2,9 @@ package main import ( "encoding/json" - "github.com/coreos/go-raft" "net/http" + + "github.com/coreos/go-raft" ) //------------------------------------------------------------- diff --git a/raft_server.go b/raft_server.go index c8b86021c7d..1175f4665b4 100644 --- a/raft_server.go +++ b/raft_server.go @@ -277,4 +277,7 @@ func registerCommands() { raft.RegisterCommand(&DeleteCommand{}) raft.RegisterCommand(&WatchCommand{}) raft.RegisterCommand(&TestAndSetCommand{}) + + raft.RegisterCommand(&CreateCommand{}) + raft.RegisterCommand(&UpdateCommand{}) } diff --git a/util.go b/util.go index 22cbed6413c..96b1cf39463 100644 --- a/util.go +++ b/util.go @@ -3,7 +3,6 @@ package main import ( "encoding/json" "fmt" - "github.com/coreos/etcd/web" "io" "log" "net" @@ -14,6 +13,9 @@ import ( "runtime/pprof" "strconv" "time" + + "github.com/coreos/etcd/file_system" + "github.com/coreos/etcd/web" ) //-------------------------------------- @@ -26,12 +28,12 @@ func durationToExpireTime(strDuration string) (time.Time, error) { duration, err := strconv.Atoi(strDuration) if err != nil { - return time.Unix(0, 0), err + return fileSystem.Permanent, err } return time.Now().Add(time.Second * (time.Duration)(duration)), nil } else { - return time.Unix(0, 0), nil + return fileSystem.Permanent, nil } } From a3545a7ffadf3b4cbb2ea3e426aae33e404baab5 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 15 Sep 2013 16:42:34 -0400 Subject: [PATCH 034/247] support consistent get --- etcd_handlers.go | 36 +++++++++++++++--------------------- util.go | 17 +++++++++++++++++ 2 files changed, 32 insertions(+), 21 deletions(-) diff --git a/etcd_handlers.go b/etcd_handlers.go index 2faebc641a9..79615630472 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -185,13 +185,12 @@ func DeleteHttpHandler(w http.ResponseWriter, req *http.Request) error { // Dispatch the command to leader func dispatch(c Command, w http.ResponseWriter, req *http.Request, etcd bool) error { - if r.State() == raft.Leader { if body, err := r.Do(c); err != nil { return err } else { if body == nil { - return etcdErr.NewError(etcdErr.EcodeRaftInternal, "Empty result from raft") + return etcdErr.NewError(300, "Empty result from raft") } else { body, _ := body.([]byte) w.WriteHeader(http.StatusOK) @@ -204,28 +203,14 @@ func dispatch(c Command, w http.ResponseWriter, req *http.Request, etcd bool) er leader := r.Leader() // current no leader if leader == "" { - return etcdErr.NewError(etcdErr.EcodeRaftInternal, "") - } - - // tell the client where is the leader - path := req.URL.Path - - var url string - - if etcd { - etcdAddr, _ := nameToEtcdURL(leader) - url = etcdAddr + path - } else { - raftAddr, _ := nameToRaftURL(leader) - url = raftAddr + path + return etcdErr.NewError(300, "") } - debugf("Redirect to %s", url) + redirect(leader, etcd, w, req) - http.Redirect(w, req, url, http.StatusTemporaryRedirect) return nil } - return etcdErr.NewError(etcdErr.EcodeRaftInternal, "") + return etcdErr.NewError(300, "") } //-------------------------------------- @@ -282,7 +267,7 @@ func GetHttpHandler(w http.ResponseWriter, req *http.Request) error { recursive := req.FormValue("recursive") - if req.FormValue("wait") == "true" { + if req.FormValue("wait") == "true" { // watch command := &WatchCommand{ Key: key, } @@ -305,7 +290,16 @@ func GetHttpHandler(w http.ResponseWriter, req *http.Request) error { event, err = command.Apply(r.Server) - } else { + } else { //get + + if req.FormValue("consistent") == "true" { + if r.State() != raft.Leader { + leader := r.Leader() + redirect(leader, true, w, req) + return nil + } + } + command := &GetCommand{ Key: key, } diff --git a/util.go b/util.go index 96b1cf39463..530b77ced40 100644 --- a/util.go +++ b/util.go @@ -65,6 +65,23 @@ func startWebInterface() { // HTTP Utilities //-------------------------------------- +func redirect(node string, etcd bool, w http.ResponseWriter, req *http.Request) { + var url string + path := req.URL.Path + + if etcd { + etcdAddr, _ := nameToEtcdURL(node) + url = etcdAddr + path + } else { + raftAddr, _ := nameToRaftURL(node) + url = raftAddr + path + } + + debugf("Redirect to %s", url) + + http.Redirect(w, req, url, http.StatusTemporaryRedirect) +} + func decodeJsonRequest(req *http.Request, data interface{}) error { decoder := json.NewDecoder(req.Body) if err := decoder.Decode(&data); err != nil && err != io.EOF { From e41ef9b733d3f108a798fdfb69769148b8c4b5a5 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 15 Sep 2013 17:46:38 -0400 Subject: [PATCH 035/247] refactor dispatch --- etcd_handlers.go | 40 ++++++++-------------------------------- raft_handlers.go | 9 ++++++--- util.go | 39 +++++++++++++++++++++++++++++++-------- 3 files changed, 45 insertions(+), 43 deletions(-) diff --git a/etcd_handlers.go b/etcd_handlers.go index 79615630472..a8d0508c11d 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -105,7 +105,7 @@ func CreateHttpHandler(w http.ResponseWriter, req *http.Request) error { ExpireTime: expireTime, } - return dispatch(command, w, req, true) + return dispatchEtcdCommand(command, w, req) } @@ -140,7 +140,7 @@ func UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { ExpireTime: expireTime, } - return dispatch(command, w, req, true) + return dispatchEtcdCommand(command, w, req) } else { // update with test var prevIndex uint64 @@ -161,7 +161,7 @@ func UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { PrevIndex: prevIndex, } - return dispatch(command, w, req, true) + return dispatchEtcdCommand(command, w, req) } } @@ -180,37 +180,12 @@ func DeleteHttpHandler(w http.ResponseWriter, req *http.Request) error { command.Recursive = true } - return dispatch(command, w, req, true) + return dispatchEtcdCommand(command, w, req) } // Dispatch the command to leader -func dispatch(c Command, w http.ResponseWriter, req *http.Request, etcd bool) error { - if r.State() == raft.Leader { - if body, err := r.Do(c); err != nil { - return err - } else { - if body == nil { - return etcdErr.NewError(300, "Empty result from raft") - } else { - body, _ := body.([]byte) - w.WriteHeader(http.StatusOK) - w.Write(body) - return nil - } - } - - } else { - leader := r.Leader() - // current no leader - if leader == "" { - return etcdErr.NewError(300, "") - } - - redirect(leader, etcd, w, req) - - return nil - } - return etcdErr.NewError(300, "") +func dispatchEtcdCommand(c Command, w http.ResponseWriter, req *http.Request) error { + return dispatch(c, w, req, nameToEtcdURL) } //-------------------------------------- @@ -295,7 +270,8 @@ func GetHttpHandler(w http.ResponseWriter, req *http.Request) error { if req.FormValue("consistent") == "true" { if r.State() != raft.Leader { leader := r.Leader() - redirect(leader, true, w, req) + url, _ := nameToEtcdURL(leader) + redirect(url, w, req) return nil } } diff --git a/raft_handlers.go b/raft_handlers.go index 4fff0780d42..6c95efd3f0a 100644 --- a/raft_handlers.go +++ b/raft_handlers.go @@ -104,7 +104,7 @@ func JoinHttpHandler(w http.ResponseWriter, req *http.Request) error { if err := decodeJsonRequest(req, command); err == nil { debugf("Receive Join Request from %s", command.Name) - return dispatch(command, w, req, false) + return dispatchRaftCommand(command, w, req) } else { w.WriteHeader(http.StatusInternalServerError) return nil @@ -125,8 +125,7 @@ func RemoveHttpHandler(w http.ResponseWriter, req *http.Request) { debugf("[recv] Remove Request [%s]", command.Name) - dispatch(command, w, req, false) - + dispatchRaftCommand(command, w, req) } // Response to the name request @@ -142,3 +141,7 @@ func RaftVersionHttpHandler(w http.ResponseWriter, req *http.Request) { w.WriteHeader(http.StatusOK) w.Write([]byte(r.version)) } + +func dispatchRaftCommand(c Command, w http.ResponseWriter, req *http.Request) error { + return dispatch(c, w, req, nameToRaftURL) +} diff --git a/util.go b/util.go index 530b77ced40..5d64e1fac94 100644 --- a/util.go +++ b/util.go @@ -65,17 +65,40 @@ func startWebInterface() { // HTTP Utilities //-------------------------------------- -func redirect(node string, etcd bool, w http.ResponseWriter, req *http.Request) { - var url string - path := req.URL.Path +func dispatch(c Command, w http.ResponseWriter, req *http.Request, toURL func(name string) (string, bool)) error { + if r.State() == raft.Leader { + if body, err := r.Do(c); err != nil { + return err + } else { + if body == nil { + return etcdErr.NewError(300, "Empty result from raft") + } else { + body, _ := body.([]byte) + w.WriteHeader(http.StatusOK) + w.Write(body) + return nil + } + } - if etcd { - etcdAddr, _ := nameToEtcdURL(node) - url = etcdAddr + path } else { - raftAddr, _ := nameToRaftURL(node) - url = raftAddr + path + leader := r.Leader() + // current no leader + if leader == "" { + return etcdErr.NewError(300, "") + } + url, _ := toURL(leader) + + redirect(url, w, req) + + return nil } + return etcdErr.NewError(300, "") +} + +func redirect(hostname string, w http.ResponseWriter, req *http.Request) { + path := req.URL.Path + + url := hostname + path debugf("Redirect to %s", url) From 232f83f99a9141d0a909d8592a1f416affea081f Mon Sep 17 00:00:00 2001 From: evan-gu Date: Sun, 15 Sep 2013 18:36:56 -0400 Subject: [PATCH 036/247] fix bug of deleting the root modified: etcd_handlers.go modified: file_system/file_system.go modified: file_system/node.go deleted: transporter_test.go --- etcd_handlers.go | 1 - file_system/file_system.go | 1 - file_system/node.go | 6 +++--- transporter_test.go | 36 ------------------------------------ 4 files changed, 3 insertions(+), 41 deletions(-) delete mode 100644 transporter_test.go diff --git a/etcd_handlers.go b/etcd_handlers.go index 79615630472..5c61c750ebd 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -169,7 +169,6 @@ func UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { // Delete Handler func DeleteHttpHandler(w http.ResponseWriter, req *http.Request) error { key := req.URL.Path[len("/v2/keys"):] - debugf("recv.delete[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) command := &DeleteCommand{ diff --git a/file_system/file_system.go b/file_system/file_system.go index ee67f9655d5..998c7cfaa65 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -244,7 +244,6 @@ func (fs *FileSystem) Delete(nodePath string, recursive bool, index uint64, term } fs.WatcherHub.notify(e) - return e, nil } diff --git a/file_system/node.go b/file_system/node.go index e4f42ef59f3..817f81f57a0 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -75,8 +75,8 @@ func (n *Node) Remove(recursive bool, callback func(path string)) error { if !n.IsDir() { // file node: key-value pair _, name := path.Split(n.Path) - - if n.Parent.Children[name] == n { + + if n.Parent != nil && n.Parent.Children[name] == n { // This is the only pointer to Node object // Handled by garbage collector delete(n.Parent.Children, name) @@ -102,7 +102,7 @@ func (n *Node) Remove(recursive bool, callback func(path string)) error { // delete self _, name := path.Split(n.Path) - if n.Parent.Children[name] == n { + if n.Parent != nil && n.Parent.Children[name] == n { delete(n.Parent.Children, name) if callback != nil { diff --git a/transporter_test.go b/transporter_test.go deleted file mode 100644 index e440a094f1a..00000000000 --- a/transporter_test.go +++ /dev/null @@ -1,36 +0,0 @@ -package main - -import ( - "crypto/tls" - "testing" - "time" -) - -func TestTransporterTimeout(t *testing.T) { - - conf := tls.Config{} - - ts := newTransporter("http", conf, time.Second) - - ts.Get("http://google.com") - _, err := ts.Get("http://google.com:9999") // it doesn't exisit - if err == nil || err.Error() != "Wait Response Timeout: 1s" { - t.Fatal("timeout error: ", err.Error()) - } - - _, err = ts.Post("http://google.com:9999", nil) // it doesn't exisit - if err == nil || err.Error() != "Wait Response Timeout: 1s" { - t.Fatal("timeout error: ", err.Error()) - } - - _, err = ts.Get("http://www.google.com") - if err != nil { - t.Fatal("get error") - } - - _, err = ts.Post("http://www.google.com", nil) - if err != nil { - t.Fatal("post error") - } - -} From 09414016c2cc5bf6cfc660d389da85a12fea0a41 Mon Sep 17 00:00:00 2001 From: evan-gu Date: Sun, 15 Sep 2013 18:47:53 -0400 Subject: [PATCH 037/247] fix bug of deleteing the root --- etcd_handlers.go | 1 + file_system/file_system.go | 1 + transporter_test.go | 36 ++++++++++++++++++++++++++++++++++++ 3 files changed, 38 insertions(+) create mode 100644 transporter_test.go diff --git a/etcd_handlers.go b/etcd_handlers.go index f21721638f7..82c06ece552 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -169,6 +169,7 @@ func UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { // Delete Handler func DeleteHttpHandler(w http.ResponseWriter, req *http.Request) error { key := req.URL.Path[len("/v2/keys"):] + debugf("recv.delete[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) command := &DeleteCommand{ diff --git a/file_system/file_system.go b/file_system/file_system.go index 998c7cfaa65..b46c6ffc9cd 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -244,6 +244,7 @@ func (fs *FileSystem) Delete(nodePath string, recursive bool, index uint64, term } fs.WatcherHub.notify(e) + return e, nil } diff --git a/transporter_test.go b/transporter_test.go new file mode 100644 index 00000000000..e440a094f1a --- /dev/null +++ b/transporter_test.go @@ -0,0 +1,36 @@ +package main + +import ( + "crypto/tls" + "testing" + "time" +) + +func TestTransporterTimeout(t *testing.T) { + + conf := tls.Config{} + + ts := newTransporter("http", conf, time.Second) + + ts.Get("http://google.com") + _, err := ts.Get("http://google.com:9999") // it doesn't exisit + if err == nil || err.Error() != "Wait Response Timeout: 1s" { + t.Fatal("timeout error: ", err.Error()) + } + + _, err = ts.Post("http://google.com:9999", nil) // it doesn't exisit + if err == nil || err.Error() != "Wait Response Timeout: 1s" { + t.Fatal("timeout error: ", err.Error()) + } + + _, err = ts.Get("http://www.google.com") + if err != nil { + t.Fatal("get error") + } + + _, err = ts.Post("http://www.google.com", nil) + if err != nil { + t.Fatal("post error") + } + +} From cd6ed3d15b400ea10a290387835cea5c5899599b Mon Sep 17 00:00:00 2001 From: evan-gu Date: Sun, 15 Sep 2013 18:51:25 -0400 Subject: [PATCH 038/247] gofmt --- etcd_handlers.go | 2 +- file_system/file_system.go | 2 +- file_system/node.go | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/etcd_handlers.go b/etcd_handlers.go index 82c06ece552..a8d0508c11d 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -169,7 +169,7 @@ func UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { // Delete Handler func DeleteHttpHandler(w http.ResponseWriter, req *http.Request) error { key := req.URL.Path[len("/v2/keys"):] - + debugf("recv.delete[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) command := &DeleteCommand{ diff --git a/file_system/file_system.go b/file_system/file_system.go index b46c6ffc9cd..ee67f9655d5 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -244,7 +244,7 @@ func (fs *FileSystem) Delete(nodePath string, recursive bool, index uint64, term } fs.WatcherHub.notify(e) - + return e, nil } diff --git a/file_system/node.go b/file_system/node.go index 817f81f57a0..872aacd4c60 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -75,7 +75,7 @@ func (n *Node) Remove(recursive bool, callback func(path string)) error { if !n.IsDir() { // file node: key-value pair _, name := path.Split(n.Path) - + if n.Parent != nil && n.Parent.Children[name] == n { // This is the only pointer to Node object // Handled by garbage collector From cd0201df065d0f675fe0d6c59c51437d85daae20 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 15 Sep 2013 20:50:35 -0400 Subject: [PATCH 039/247] fix import --- command.go | 7 ++++--- util.go | 2 ++ 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/command.go b/command.go index 0e4a58bc1bc..3c901893afc 100644 --- a/command.go +++ b/command.go @@ -9,6 +9,7 @@ import ( "time" etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/file_system" "github.com/coreos/go-raft" ) @@ -213,12 +214,12 @@ func (c *JoinCommand) CommandName() string { func (c *JoinCommand) Apply(raftServer *raft.Server) (interface{}, error) { // check if the join command is from a previous machine, who lost all its previous log. - response, _ := etcdStore.RawGet(path.Join("_etcd/machines", c.Name)) + e, _ := etcdFs.Get(path.Join("/_etcd/machines", c.Name), false, false, raftServer.CommitIndex(), raftServer.Term()) b := make([]byte, 8) binary.PutUvarint(b, raftServer.CommitIndex()) - if response != nil { + if e != nil { return b, nil } @@ -237,7 +238,7 @@ func (c *JoinCommand) Apply(raftServer *raft.Server) (interface{}, error) { // add machine in etcd storage key := path.Join("_etcd/machines", c.Name) value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", c.RaftURL, c.EtcdURL, c.RaftVersion) - etcdStore.Set(key, value, time.Unix(0, 0), raftServer.CommitIndex()) + etcdFs.Create(key, value, fileSystem.Permanent, raftServer.CommitIndex(), raftServer.Term()) if c.Name != r.Name() { r.peersStats[c.Name] = &raftPeerStats{MinLatency: 1 << 63} diff --git a/util.go b/util.go index 5d64e1fac94..80673d3b07d 100644 --- a/util.go +++ b/util.go @@ -14,8 +14,10 @@ import ( "strconv" "time" + etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/file_system" "github.com/coreos/etcd/web" + "github.com/coreos/go-raft" ) //-------------------------------------- From 1caf2a336432ed4d8947cb8b3abb99383c691780 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 15 Sep 2013 22:28:42 -0400 Subject: [PATCH 040/247] remove old store --- command.go | 21 +- etcd.go | 3 - etcd_handlers.go | 45 ++- file_system/file_system.go | 12 +- machines.go | 8 +- name_url_map.go | 10 +- raft_server.go | 3 +- snapshot.go | 10 +- store/keyword_test.go | 37 --- store/keywords.go | 33 -- store/stats.go | 33 -- store/store.go | 663 ------------------------------------- store/store_test.go | 258 --------------- store/test.go | 21 -- store/tree.go | 318 ------------------ store/tree_store_test.go | 247 -------------- store/watcher.go | 129 -------- store/watcher_test.go | 84 ----- util.go | 9 +- 19 files changed, 54 insertions(+), 1890 deletions(-) delete mode 100644 store/keyword_test.go delete mode 100644 store/keywords.go delete mode 100644 store/stats.go delete mode 100644 store/store.go delete mode 100644 store/store_test.go delete mode 100644 store/test.go delete mode 100644 store/tree.go delete mode 100644 store/tree_store_test.go delete mode 100644 store/watcher.go delete mode 100644 store/watcher_test.go diff --git a/command.go b/command.go index 3c901893afc..1c014e4d029 100644 --- a/command.go +++ b/command.go @@ -73,23 +73,6 @@ func (c *UpdateCommand) Apply(server *raft.Server) (interface{}, error) { return json.Marshal(e) } -// Set command -type SetCommand struct { - Key string `json:"key"` - Value string `json:"value"` - ExpireTime time.Time `json:"expireTime"` -} - -// The name of the set command in the log -func (c *SetCommand) CommandName() string { - return commandName("set") -} - -// Set the key-value pair -func (c *SetCommand) Apply(server *raft.Server) (interface{}, error) { - return etcdStore.Set(c.Key, c.Value, c.ExpireTime, server.CommitIndex()) -} - // TestAndSet command type TestAndSetCommand struct { Key string `json:"key"` @@ -240,7 +223,7 @@ func (c *JoinCommand) Apply(raftServer *raft.Server) (interface{}, error) { value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", c.RaftURL, c.EtcdURL, c.RaftVersion) etcdFs.Create(key, value, fileSystem.Permanent, raftServer.CommitIndex(), raftServer.Term()) - if c.Name != r.Name() { + if c.Name != r.Name() { // do not add self to the peer list r.peersStats[c.Name] = &raftPeerStats{MinLatency: 1 << 63} } @@ -267,7 +250,7 @@ func (c *RemoveCommand) Apply(raftServer *raft.Server) (interface{}, error) { // remove machine in etcd storage key := path.Join("_etcd/machines", c.Name) - _, err := etcdStore.Delete(key, raftServer.CommitIndex()) + _, err := etcdFs.Delete(key, false, raftServer.CommitIndex(), raftServer.Term()) delete(r.peersStats, c.Name) if err != nil { diff --git a/etcd.go b/etcd.go index bd2144183a6..cba7809f4d6 100644 --- a/etcd.go +++ b/etcd.go @@ -11,7 +11,6 @@ import ( "time" "github.com/coreos/etcd/file_system" - "github.com/coreos/etcd/store" "github.com/coreos/go-raft" ) @@ -137,7 +136,6 @@ type TLSConfig struct { // //------------------------------------------------------------------------------ -var etcdStore *store.Store var etcdFs *fileSystem.FileSystem //------------------------------------------------------------------------------ @@ -206,7 +204,6 @@ func main() { info := getInfo(dirPath) // Create etcd key-value store - etcdStore = store.CreateStore(maxSize) etcdFs = fileSystem.New() snapConf = newSnapshotConf() diff --git a/etcd_handlers.go b/etcd_handlers.go index a8d0508c11d..cde882c2ed7 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -7,6 +7,7 @@ import ( "strings" etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/file_system" "github.com/coreos/go-raft" ) @@ -83,17 +84,13 @@ func Multiplexer(w http.ResponseWriter, req *http.Request) error { //-------------------------------------- func CreateHttpHandler(w http.ResponseWriter, req *http.Request) error { - key := req.URL.Path[len("/v2/keys"):] + key := getNodePath(req.URL.Path) debugf("recv.post[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) - req.ParseForm() - - value := req.Form.Get("value") - - ttl := req.FormValue("ttl") + value := req.FormValue("value") - expireTime, err := durationToExpireTime(ttl) + expireTime, err := durationToExpireTime(req.FormValue("ttl")) if err != nil { return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create") @@ -110,22 +107,20 @@ func CreateHttpHandler(w http.ResponseWriter, req *http.Request) error { } func UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { - key := req.URL.Path[len("/v2/keys"):] + key := getNodePath(req.URL.Path) debugf("recv.put[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) value := req.FormValue("value") - ttl := req.FormValue("ttl") - - expireTime, err := durationToExpireTime(ttl) + expireTime, err := durationToExpireTime(req.FormValue("ttl")) if err != nil { return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update") } // TODO: update should give at least one option - if value == "" && ttl == "" { + if value == "" && expireTime.Sub(fileSystem.Permanent) == 0 { return nil } @@ -168,7 +163,7 @@ func UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { // Delete Handler func DeleteHttpHandler(w http.ResponseWriter, req *http.Request) error { - key := req.URL.Path[len("/v2/keys"):] + key := getNodePath(req.URL.Path) debugf("recv.delete[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) @@ -228,7 +223,7 @@ func VersionHttpHandler(w http.ResponseWriter, req *http.Request) error { // Handler to return the basic stats of etcd func StatsHttpHandler(w http.ResponseWriter, req *http.Request) error { w.WriteHeader(http.StatusOK) - w.Write(etcdStore.Stats()) + //w.Write(etcdStore.Stats()) w.Write(r.Stats()) return nil } @@ -236,10 +231,18 @@ func StatsHttpHandler(w http.ResponseWriter, req *http.Request) error { func GetHttpHandler(w http.ResponseWriter, req *http.Request) error { var err error var event interface{} - key := req.URL.Path[len("/v1/keys"):] - debugf("recv.get[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) + if req.FormValue("consistent") == "true" && r.State() != raft.Leader { + // help client to redirect the request to the current leader + leader := r.Leader() + url, _ := nameToEtcdURL(leader) + redirect(url, w, req) + return nil + } + + key := getNodePath(req.URL.Path) + recursive := req.FormValue("recursive") if req.FormValue("wait") == "true" { // watch @@ -267,15 +270,6 @@ func GetHttpHandler(w http.ResponseWriter, req *http.Request) error { } else { //get - if req.FormValue("consistent") == "true" { - if r.State() != raft.Leader { - leader := r.Leader() - url, _ := nameToEtcdURL(leader) - redirect(url, w, req) - return nil - } - } - command := &GetCommand{ Key: key, } @@ -295,6 +289,7 @@ func GetHttpHandler(w http.ResponseWriter, req *http.Request) error { if err != nil { return err + } else { event, _ := event.([]byte) w.WriteHeader(http.StatusOK) diff --git a/file_system/file_system.go b/file_system/file_system.go index ee67f9655d5..1de7830fb8d 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -332,30 +332,30 @@ func (fs *FileSystem) checkDir(parent *Node, dirName string) (*Node, error) { // Save function will not be able to save the state of watchers. // Save function will not save the parent field of the node. Or there will // be cyclic dependencies issue for the json package. -func (fs *FileSystem) Save() []byte { +func (fs *FileSystem) Save() ([]byte, error) { cloneFs := New() cloneFs.Root = fs.Root.Clone() b, err := json.Marshal(fs) if err != nil { - panic(err) + return nil, err } - return b + return b, nil } // recovery function recovery the store system from a static state. // It needs to recovery the parent field of the nodes. // It needs to delete the expired nodes since the saved time and also // need to create monitor go routines. -func (fs *FileSystem) Recover(state []byte) { +func (fs *FileSystem) Recovery(state []byte) error { err := json.Unmarshal(state, fs) if err != nil { - panic(err) + return err } fs.Root.recoverAndclean() - + return nil } diff --git a/machines.go b/machines.go index fbaa48d6f10..1988353d51e 100644 --- a/machines.go +++ b/machines.go @@ -2,9 +2,13 @@ package main // machineNum returns the number of machines in the cluster func machineNum() int { - response, _ := etcdStore.RawGet("_etcd/machines") + e, err := etcdFs.Get("/_etcd/machines", false, false, r.CommitIndex(), r.Term()) - return len(response) + if err != nil { + return 0 + } + + return len(e.KVPairs) } // getMachines gets the current machines in the cluster diff --git a/name_url_map.go b/name_url_map.go index 0e5abb1b74e..38e1ecc15db 100644 --- a/name_url_map.go +++ b/name_url_map.go @@ -49,16 +49,20 @@ func addNameToURL(name string, version string, raftURL string, etcdURL string) { } func readURL(nodeName string, urlName string) (string, bool) { - // if fails, try to recover from etcd storage + if nodeName == "" { + return "", false + } + + // convert nodeName to url from etcd storage key := path.Join("/_etcd/machines", nodeName) - resps, err := etcdStore.RawGet(key) + e, err := etcdFs.Get(key, false, false, r.CommitIndex(), r.Term()) if err != nil { return "", false } - m, err := url.ParseQuery(resps[0].Value) + m, err := url.ParseQuery(e.Value) if err != nil { panic("Failed to parse machines entry") diff --git a/raft_server.go b/raft_server.go index 628136479fc..d88f266f8b0 100644 --- a/raft_server.go +++ b/raft_server.go @@ -36,7 +36,7 @@ func newRaftServer(name string, url string, listenHost string, tlsConf *TLSConfi raftTransporter := newTransporter(tlsConf.Scheme, tlsConf.Client, ElectionTimeout) // Create raft server - server, err := raft.NewServer(name, dirPath, raftTransporter, etcdStore, nil) + server, err := raft.NewServer(name, dirPath, raftTransporter, etcdFs, nil) check(err) @@ -312,7 +312,6 @@ func (r *raftServer) Stats() []byte { func registerCommands() { raft.RegisterCommand(&JoinCommand{}) raft.RegisterCommand(&RemoveCommand{}) - raft.RegisterCommand(&SetCommand{}) raft.RegisterCommand(&GetCommand{}) raft.RegisterCommand(&DeleteCommand{}) raft.RegisterCommand(&WatchCommand{}) diff --git a/snapshot.go b/snapshot.go index 7b9da803437..a6caefd32b8 100644 --- a/snapshot.go +++ b/snapshot.go @@ -20,17 +20,17 @@ var snapConf *snapshotConf func newSnapshotConf() *snapshotConf { // check snapshot every 3 seconds and the threshold is 20K - return &snapshotConf{time.Second * 3, etcdStore.TotalWrites(), 20 * 1000} + return &snapshotConf{time.Second * 3, 0, 20 * 1000} } func monitorSnapshot() { for { time.Sleep(snapConf.checkingInterval) - currentWrites := etcdStore.TotalWrites() - snapConf.lastWrites - - if currentWrites > snapConf.writesThr { + //currentWrites := etcdStore.TotalWrites() - snapConf.lastWrites + currentWrites := 0 + if uint64(currentWrites) > snapConf.writesThr { r.TakeSnapshot() - snapConf.lastWrites = etcdStore.TotalWrites() + snapConf.lastWrites = 0 } } } diff --git a/store/keyword_test.go b/store/keyword_test.go deleted file mode 100644 index 7c54a9fde3a..00000000000 --- a/store/keyword_test.go +++ /dev/null @@ -1,37 +0,0 @@ -package store - -import ( - "testing" -) - -func TestKeywords(t *testing.T) { - keyword := CheckKeyword("_etcd") - if !keyword { - t.Fatal("_etcd should be keyword") - } - - keyword = CheckKeyword("/_etcd") - - if !keyword { - t.Fatal("/_etcd should be keyword") - } - - keyword = CheckKeyword("/_etcd/") - - if !keyword { - t.Fatal("/_etcd/ contains keyword prefix") - } - - keyword = CheckKeyword("/_etcd/node1") - - if !keyword { - t.Fatal("/_etcd/* contains keyword prefix") - } - - keyword = CheckKeyword("/nokeyword/_etcd/node1") - - if keyword { - t.Fatal("this does not contain keyword prefix") - } - -} diff --git a/store/keywords.go b/store/keywords.go deleted file mode 100644 index 2e4ceb75b6b..00000000000 --- a/store/keywords.go +++ /dev/null @@ -1,33 +0,0 @@ -package store - -import ( - "path" - "strings" -) - -// keywords for internal useage -// Key for string keyword; Value for only checking prefix -var keywords = map[string]bool{ - "/_etcd": true, - "/ephemeralNodes": true, -} - -// CheckKeyword will check if the key contains the keyword. -// For now, we only check for prefix. -func CheckKeyword(key string) bool { - key = path.Clean("/" + key) - - // find the second "/" - i := strings.Index(key[1:], "/") - - var prefix string - - if i == -1 { - prefix = key - } else { - prefix = key[:i+1] - } - _, ok := keywords[prefix] - - return ok -} diff --git a/store/stats.go b/store/stats.go deleted file mode 100644 index b57f4db3db9..00000000000 --- a/store/stats.go +++ /dev/null @@ -1,33 +0,0 @@ -package store - -import ( - "encoding/json" -) - -type EtcdStats struct { - // Number of get requests - Gets uint64 `json:"gets"` - - // Number of sets requests - Sets uint64 `json:"sets"` - - // Number of delete requests - Deletes uint64 `json:"deletes"` - - // Number of testAndSet requests - TestAndSets uint64 `json:"testAndSets"` -} - -// Stats returns the basic statistics information of etcd storage since its recent start -func (s *Store) Stats() []byte { - b, _ := json.Marshal(s.BasicStats) - return b -} - -// TotalWrites returns the total write operations -// It helps with snapshot -func (s *Store) TotalWrites() uint64 { - bs := s.BasicStats - - return bs.Deletes + bs.Sets + bs.TestAndSets -} diff --git a/store/store.go b/store/store.go deleted file mode 100644 index 916c1394eb9..00000000000 --- a/store/store.go +++ /dev/null @@ -1,663 +0,0 @@ -package store - -import ( - "encoding/json" - "fmt" - "path" - "strconv" - "sync" - "time" - - etcdErr "github.com/coreos/etcd/error" -) - -//------------------------------------------------------------------------------ -// -// Typedefs -// -//------------------------------------------------------------------------------ - -// The main struct of the Key-Value store -type Store struct { - - // key-value store structure - Tree *tree - - // This mutex protects everything except add watcher member. - // Add watch member does not depend on the current state of the store. - // And watch will return when other protected function is called and reach - // the watching condition. - // It is needed so that clone() can atomically replicate the Store - // and do the log snapshot in a go routine. - mutex sync.RWMutex - - // WatcherHub is where we register all the clients - // who issue a watch request - watcher *WatcherHub - - // The string channel to send messages to the outside world - // Now we use it to send changes to the hub of the web service - messager chan<- string - - // A map to keep the recent response to the clients - ResponseMap map[string]*Response - - // The max number of the recent responses we can record - ResponseMaxSize int - - // The current number of the recent responses we have recorded - ResponseCurrSize uint - - // The index of the first recent responses we have - ResponseStartIndex uint64 - - // Current index of the raft machine - Index uint64 - - // Basic statistics information of etcd storage - BasicStats EtcdStats -} - -// A Node represents a Value in the Key-Value pair in the store -// It has its value, expire time and a channel used to update the -// expire time (since we do countdown in a go routine, we need to -// communicate with it via channel) -type Node struct { - // The string value of the node - Value string `json:"value"` - - // If the node is a permanent one the ExprieTime will be Unix(0,0) - // Otherwise after the expireTime, the node will be deleted - ExpireTime time.Time `json:"expireTime"` - - // A channel to update the expireTime of the node - update chan time.Time `json:"-"` -} - -// The response from the store to the user who issue a command -type Response struct { - Action string `json:"action"` - Key string `json:"key"` - Dir bool `json:"dir,omitempty"` - PrevValue string `json:"prevValue,omitempty"` - Value string `json:"value,omitempty"` - - // If the key did not exist before the action, - // this field should be set to true - NewKey bool `json:"newKey,omitempty"` - - Expiration *time.Time `json:"expiration,omitempty"` - - // Time to live in second - TTL int64 `json:"ttl,omitempty"` - - // The command index of the raft machine when the command is executed - Index uint64 `json:"index"` -} - -// A listNode represent the simplest Key-Value pair with its type -// It is only used when do list opeartion -// We want to have a file system like store, thus we distingush "file" -// and "directory" -type ListNode struct { - Key string - Value string - Type string -} - -var PERMANENT = time.Unix(0, 0) - -//------------------------------------------------------------------------------ -// -// Methods -// -//------------------------------------------------------------------------------ - -// Create a new stroe -// Arguement max is the max number of response we want to record -func CreateStore(max int) *Store { - s := new(Store) - - s.messager = nil - - s.ResponseMap = make(map[string]*Response) - s.ResponseStartIndex = 0 - s.ResponseMaxSize = max - s.ResponseCurrSize = 0 - - s.Tree = &tree{ - &treeNode{ - Node{ - "/", - time.Unix(0, 0), - nil, - }, - true, - make(map[string]*treeNode), - }, - } - - s.watcher = newWatcherHub() - - return s -} - -// Set the messager of the store -func (s *Store) SetMessager(messager chan<- string) { - s.messager = messager -} - -func (s *Store) Set(key string, value string, expireTime time.Time, index uint64) ([]byte, error) { - s.mutex.Lock() - defer s.mutex.Unlock() - - return s.internalSet(key, value, expireTime, index) - -} - -// Set the key to value with expiration time -func (s *Store) internalSet(key string, value string, expireTime time.Time, index uint64) ([]byte, error) { - //Update index - s.Index = index - - //Update stats - s.BasicStats.Sets++ - - key = path.Clean("/" + key) - - isExpire := !expireTime.Equal(PERMANENT) - - // base response - resp := Response{ - Action: "SET", - Key: key, - Value: value, - Index: index, - } - - // When the slow follower receive the set command - // the key may be expired, we should not add the node - // also if the node exist, we need to delete the node - if isExpire && expireTime.Sub(time.Now()) < 0 { - return s.internalDelete(key, index) - } - - var TTL int64 - - // Update ttl - if isExpire { - TTL = int64(expireTime.Sub(time.Now()) / time.Second) - resp.Expiration = &expireTime - resp.TTL = TTL - } - - // Get the node - node, ok := s.Tree.get(key) - - if ok { - // Update when node exists - - // Node is not permanent - if !node.ExpireTime.Equal(PERMANENT) { - - // If node is not permanent - // Update its expireTime - node.update <- expireTime - - } else { - - // If we want the permanent node to have expire time - // We need to create a go routine with a channel - if isExpire { - node.update = make(chan time.Time) - go s.monitorExpiration(key, node.update, expireTime) - } - - } - - // Update the information of the node - s.Tree.set(key, Node{value, expireTime, node.update}) - - resp.PrevValue = node.Value - - s.watcher.notify(resp) - - msg, err := json.Marshal(resp) - - // Send to the messager - if s.messager != nil && err == nil { - s.messager <- string(msg) - } - - s.addToResponseMap(index, &resp) - - return msg, err - - // Add new node - } else { - - update := make(chan time.Time) - - ok := s.Tree.set(key, Node{value, expireTime, update}) - - if !ok { - return nil, etcdErr.NewError(102, "set: "+key) - } - - if isExpire { - go s.monitorExpiration(key, update, expireTime) - } - - resp.NewKey = true - - msg, err := json.Marshal(resp) - - // Nofity the watcher - s.watcher.notify(resp) - - // Send to the messager - if s.messager != nil && err == nil { - s.messager <- string(msg) - } - - s.addToResponseMap(index, &resp) - return msg, err - } - -} - -// Get the value of the key and return the raw response -func (s *Store) internalGet(key string) *Response { - - key = path.Clean("/" + key) - - node, ok := s.Tree.get(key) - - if ok { - var TTL int64 - var isExpire bool = false - - isExpire = !node.ExpireTime.Equal(PERMANENT) - - resp := &Response{ - Action: "GET", - Key: key, - Value: node.Value, - Index: s.Index, - } - - // Update ttl - if isExpire { - TTL = int64(node.ExpireTime.Sub(time.Now()) / time.Second) - resp.Expiration = &node.ExpireTime - resp.TTL = TTL - } - - return resp - - } else { - // we do not found the key - return nil - } -} - -// Get all the items under key -// If key is a file return the file -// If key is a directory reuturn an array of files -func (s *Store) Get(key string) ([]byte, error) { - s.mutex.RLock() - defer s.mutex.RUnlock() - - resps, err := s.RawGet(key) - - if err != nil { - return nil, err - } - - key = path.Clean("/" + key) - - // If the number of resps == 1 and the response key - // is the key we query, a signal key-value should - // be returned - if len(resps) == 1 && resps[0].Key == key { - return json.Marshal(resps[0]) - } - - return json.Marshal(resps) -} - -func (s *Store) rawGetNode(key string, node *Node) ([]*Response, error) { - resps := make([]*Response, 1) - - isExpire := !node.ExpireTime.Equal(PERMANENT) - - resps[0] = &Response{ - Action: "GET", - Index: s.Index, - Key: key, - Value: node.Value, - } - - // Update ttl - if isExpire { - TTL := int64(node.ExpireTime.Sub(time.Now()) / time.Second) - resps[0].Expiration = &node.ExpireTime - resps[0].TTL = TTL - } - - return resps, nil -} - -func (s *Store) rawGetNodeList(key string, keys []string, nodes []*Node) ([]*Response, error) { - resps := make([]*Response, len(nodes)) - - // TODO: check if nodes and keys are the same length - for i := 0; i < len(nodes); i++ { - var TTL int64 - var isExpire bool = false - - isExpire = !nodes[i].ExpireTime.Equal(PERMANENT) - - resps[i] = &Response{ - Action: "GET", - Index: s.Index, - Key: path.Join(key, keys[i]), - } - - if len(nodes[i].Value) != 0 { - resps[i].Value = nodes[i].Value - } else { - resps[i].Dir = true - } - - // Update ttl - if isExpire { - TTL = int64(nodes[i].ExpireTime.Sub(time.Now()) / time.Second) - resps[i].Expiration = &nodes[i].ExpireTime - resps[i].TTL = TTL - } - - } - - return resps, nil -} - -func (s *Store) RawGet(key string) ([]*Response, error) { - // Update stats - s.BasicStats.Gets++ - - key = path.Clean("/" + key) - - nodes, keys, ok := s.Tree.list(key) - if !ok { - return nil, etcdErr.NewError(100, "get: "+key) - } - - switch node := nodes.(type) { - case *Node: - return s.rawGetNode(key, node) - case []*Node: - return s.rawGetNodeList(key, keys, node) - default: - panic("invalid cast ") - } -} - -func (s *Store) Delete(key string, index uint64) ([]byte, error) { - s.mutex.Lock() - defer s.mutex.Unlock() - return s.internalDelete(key, index) -} - -// Delete the key -func (s *Store) internalDelete(key string, index uint64) ([]byte, error) { - - // Update stats - s.BasicStats.Deletes++ - - key = path.Clean("/" + key) - - // Update index - s.Index = index - - node, ok := s.Tree.get(key) - - if !ok { - return nil, etcdErr.NewError(100, "delete: "+key) - } - - resp := Response{ - Action: "DELETE", - Key: key, - PrevValue: node.Value, - Index: index, - } - - if node.ExpireTime.Equal(PERMANENT) { - - s.Tree.delete(key) - - } else { - resp.Expiration = &node.ExpireTime - // Kill the expire go routine - node.update <- PERMANENT - s.Tree.delete(key) - - } - - msg, err := json.Marshal(resp) - - s.watcher.notify(resp) - - // notify the messager - if s.messager != nil && err == nil { - s.messager <- string(msg) - } - - s.addToResponseMap(index, &resp) - - return msg, err -} - -// Set the value of the key to the value if the given prevValue is equal to the value of the key -func (s *Store) TestAndSet(key string, prevValue string, value string, expireTime time.Time, index uint64) ([]byte, error) { - s.mutex.Lock() - defer s.mutex.Unlock() - - // Update stats - s.BasicStats.TestAndSets++ - - resp := s.internalGet(key) - - if resp == nil { - if prevValue != "" { - errmsg := fmt.Sprintf("TestAndSet: key not found and previousValue is not empty %s:%s ", key, prevValue) - return nil, etcdErr.NewError(100, errmsg) - } - return s.internalSet(key, value, expireTime, index) - } - - if resp.Value == prevValue { - - // If test succeed, do set - return s.internalSet(key, value, expireTime, index) - } else { - - // If fails, return err - return nil, etcdErr.NewError(101, fmt.Sprintf("TestAndSet: %s!=%s", - resp.Value, prevValue)) - } - -} - -// Add a channel to the watchHub. -// The watchHub will send response to the channel when any key under the prefix -// changes [since the sinceIndex if given] -func (s *Store) AddWatcher(prefix string, watcher *Watcher, sinceIndex uint64) error { - return s.watcher.addWatcher(prefix, watcher, sinceIndex, s.ResponseStartIndex, s.Index, s.ResponseMap) -} - -// This function should be created as a go routine to delete the key-value pair -// when it reaches expiration time - -func (s *Store) monitorExpiration(key string, update chan time.Time, expireTime time.Time) { - - duration := expireTime.Sub(time.Now()) - - for { - select { - - // Timeout delete the node - case <-time.After(duration): - node, ok := s.Tree.get(key) - - if !ok { - return - - } else { - s.mutex.Lock() - - s.Tree.delete(key) - - resp := Response{ - Action: "DELETE", - Key: key, - PrevValue: node.Value, - Expiration: &node.ExpireTime, - Index: s.Index, - } - s.mutex.Unlock() - - msg, err := json.Marshal(resp) - - s.watcher.notify(resp) - - // notify the messager - if s.messager != nil && err == nil { - s.messager <- string(msg) - } - - return - - } - - case updateTime := <-update: - // Update duration - // If the node become a permanent one, the go routine is - // not needed - if updateTime.Equal(PERMANENT) { - return - } - - // Update duration - duration = updateTime.Sub(time.Now()) - } - } -} - -// When we receive a command that will change the state of the key-value store -// We will add the result of it to the ResponseMap for the use of watch command -// Also we may remove the oldest response when we add new one -func (s *Store) addToResponseMap(index uint64, resp *Response) { - - // zero case - if s.ResponseMaxSize == 0 { - return - } - - strIndex := strconv.FormatUint(index, 10) - s.ResponseMap[strIndex] = resp - - // unlimited - if s.ResponseMaxSize < 0 { - s.ResponseCurrSize++ - return - } - - // if we reach the max point, we need to delete the most latest - // response and update the startIndex - if s.ResponseCurrSize == uint(s.ResponseMaxSize) { - s.ResponseStartIndex++ - delete(s.ResponseMap, strconv.FormatUint(s.ResponseStartIndex, 10)) - } else { - s.ResponseCurrSize++ - } -} - -func (s *Store) clone() *Store { - newStore := &Store{ - ResponseMaxSize: s.ResponseMaxSize, - ResponseCurrSize: s.ResponseCurrSize, - ResponseStartIndex: s.ResponseStartIndex, - Index: s.Index, - BasicStats: s.BasicStats, - } - - newStore.Tree = s.Tree.clone() - newStore.ResponseMap = make(map[string]*Response) - - for index, response := range s.ResponseMap { - newStore.ResponseMap[index] = response - } - - return newStore -} - -// Save the current state of the storage system -func (s *Store) Save() ([]byte, error) { - // first we clone the store - // json is very slow, we cannot hold the lock for such a long time - s.mutex.Lock() - cloneStore := s.clone() - s.mutex.Unlock() - - b, err := json.Marshal(cloneStore) - if err != nil { - fmt.Println(err) - return nil, err - } - return b, nil -} - -// Recovery the state of the stroage system from a previous state -func (s *Store) Recovery(state []byte) error { - s.mutex.Lock() - defer s.mutex.Unlock() - // we need to stop all the current watchers - // recovery will clear watcherHub - s.watcher.stopWatchers() - - err := json.Unmarshal(state, s) - - // The only thing need to change after the recovery is the - // node with expiration time, we need to delete all the node - // that have been expired and setup go routines to monitor the - // other ones - s.checkExpiration() - - return err -} - -// Clean the expired nodes -// Set up go routines to mon -func (s *Store) checkExpiration() { - s.Tree.traverse(s.checkNode, false) -} - -// Check each node -func (s *Store) checkNode(key string, node *Node) { - - if node.ExpireTime.Equal(PERMANENT) { - return - } else { - if node.ExpireTime.Sub(time.Now()) >= time.Second { - - node.update = make(chan time.Time) - go s.monitorExpiration(key, node.update, node.ExpireTime) - - } else { - // we should delete this node - s.Tree.delete(key) - } - } -} diff --git a/store/store_test.go b/store/store_test.go deleted file mode 100644 index 6bd719008fc..00000000000 --- a/store/store_test.go +++ /dev/null @@ -1,258 +0,0 @@ -package store - -import ( - "encoding/json" - "testing" - "time" -) - -func TestStoreGetDelete(t *testing.T) { - - s := CreateStore(100) - s.Set("foo", "bar", time.Unix(0, 0), 1) - res, err := s.Get("foo") - - if err != nil { - t.Fatalf("Unknown error") - } - - var result Response - json.Unmarshal(res, &result) - - if result.Value != "bar" { - t.Fatalf("Cannot get stored value") - } - - s.Delete("foo", 2) - _, err = s.Get("foo") - - if err == nil { - t.Fatalf("Got deleted value") - } -} - -func TestTestAndSet(t *testing.T) { - s := CreateStore(100) - s.Set("foo", "bar", time.Unix(0, 0), 1) - - _, err := s.TestAndSet("foo", "barbar", "barbar", time.Unix(0, 0), 2) - - if err == nil { - t.Fatalf("test bar == barbar should fail") - } - - _, err = s.TestAndSet("foo", "bar", "barbar", time.Unix(0, 0), 3) - - if err != nil { - t.Fatalf("test bar == bar should succeed") - } - - _, err = s.TestAndSet("foo", "", "barbar", time.Unix(0, 0), 4) - - if err == nil { - t.Fatalf("test empty == bar should fail") - } - - _, err = s.TestAndSet("fooo", "bar", "barbar", time.Unix(0, 0), 5) - - if err == nil { - t.Fatalf("test bar == non-existing key should fail") - } - - _, err = s.TestAndSet("fooo", "", "bar", time.Unix(0, 0), 6) - - if err != nil { - t.Fatalf("test empty == non-existing key should succeed") - } - -} - -func TestSaveAndRecovery(t *testing.T) { - - s := CreateStore(100) - s.Set("foo", "bar", time.Unix(0, 0), 1) - s.Set("foo2", "bar2", time.Now().Add(time.Second*5), 2) - state, err := s.Save() - - if err != nil { - t.Fatalf("Cannot Save %s", err) - } - - newStore := CreateStore(100) - - // wait for foo2 expires - time.Sleep(time.Second * 6) - - newStore.Recovery(state) - - res, err := newStore.Get("foo") - - var result Response - json.Unmarshal(res, &result) - - if result.Value != "bar" { - t.Fatalf("Recovery Fail") - } - - res, err = newStore.Get("foo2") - - if err == nil { - t.Fatalf("Get expired value") - } - - s.Delete("foo", 3) - -} - -func TestExpire(t *testing.T) { - // test expire - s := CreateStore(100) - s.Set("foo", "bar", time.Now().Add(time.Second*1), 0) - time.Sleep(2 * time.Second) - - _, err := s.Get("foo") - - if err == nil { - t.Fatalf("Got expired value") - } - - //test change expire time - s.Set("foo", "bar", time.Now().Add(time.Second*10), 1) - - _, err = s.Get("foo") - - if err != nil { - t.Fatalf("Cannot get Value") - } - - s.Set("foo", "barbar", time.Now().Add(time.Second*1), 2) - - time.Sleep(2 * time.Second) - - _, err = s.Get("foo") - - if err == nil { - t.Fatalf("Got expired value") - } - - // test change expire to stable - s.Set("foo", "bar", time.Now().Add(time.Second*1), 3) - - s.Set("foo", "bar", time.Unix(0, 0), 4) - - time.Sleep(2 * time.Second) - - _, err = s.Get("foo") - - if err != nil { - t.Fatalf("Cannot get Value") - } - - // test stable to expire - s.Set("foo", "bar", time.Now().Add(time.Second*1), 5) - time.Sleep(2 * time.Second) - _, err = s.Get("foo") - - if err == nil { - t.Fatalf("Got expired value") - } - - // test set older node - s.Set("foo", "bar", time.Now().Add(-time.Second*1), 6) - _, err = s.Get("foo") - - if err == nil { - t.Fatalf("Got expired value") - } - -} - -func BenchmarkStoreSet(b *testing.B) { - s := CreateStore(100) - - keys := GenKeys(10000, 5) - - b.ResetTimer() - for i := 0; i < b.N; i++ { - - for i, key := range keys { - s.Set(key, "barbarbarbarbar", time.Unix(0, 0), uint64(i)) - } - - s = CreateStore(100) - } -} - -func BenchmarkStoreGet(b *testing.B) { - s := CreateStore(100) - - keys := GenKeys(10000, 5) - - for i, key := range keys { - s.Set(key, "barbarbarbarbar", time.Unix(0, 0), uint64(i)) - } - - b.ResetTimer() - for i := 0; i < b.N; i++ { - - for _, key := range keys { - s.Get(key) - } - - } -} - -func BenchmarkStoreSnapshotCopy(b *testing.B) { - s := CreateStore(100) - - keys := GenKeys(10000, 5) - - for i, key := range keys { - s.Set(key, "barbarbarbarbar", time.Unix(0, 0), uint64(i)) - } - - var state []byte - - b.ResetTimer() - for i := 0; i < b.N; i++ { - s.clone() - } - b.SetBytes(int64(len(state))) -} - -func BenchmarkSnapshotSaveJson(b *testing.B) { - s := CreateStore(100) - - keys := GenKeys(10000, 5) - - for i, key := range keys { - s.Set(key, "barbarbarbarbar", time.Unix(0, 0), uint64(i)) - } - - var state []byte - - b.ResetTimer() - for i := 0; i < b.N; i++ { - state, _ = s.Save() - } - b.SetBytes(int64(len(state))) -} - -func BenchmarkSnapshotRecovery(b *testing.B) { - s := CreateStore(100) - - keys := GenKeys(10000, 5) - - for i, key := range keys { - s.Set(key, "barbarbarbarbar", time.Unix(0, 0), uint64(i)) - } - - state, _ := s.Save() - - b.ResetTimer() - for i := 0; i < b.N; i++ { - newStore := CreateStore(100) - newStore.Recovery(state) - } - b.SetBytes(int64(len(state))) -} diff --git a/store/test.go b/store/test.go deleted file mode 100644 index eaddaa69dbf..00000000000 --- a/store/test.go +++ /dev/null @@ -1,21 +0,0 @@ -package store - -import ( - "math/rand" - "strconv" -) - -// GenKeys randomly generate num of keys with max depth -func GenKeys(num int, depth int) []string { - keys := make([]string, num) - for i := 0; i < num; i++ { - - keys[i] = "/foo" - depth := rand.Intn(depth) + 1 - - for j := 0; j < depth; j++ { - keys[i] += "/" + strconv.Itoa(rand.Int()%20) - } - } - return keys -} diff --git a/store/tree.go b/store/tree.go deleted file mode 100644 index 3d6d1bfa419..00000000000 --- a/store/tree.go +++ /dev/null @@ -1,318 +0,0 @@ -package store - -import ( - "path" - "sort" - "strings" - "time" -) - -//------------------------------------------------------------------------------ -// -// Typedefs -// -//------------------------------------------------------------------------------ - -// A file system like tree structure. Each non-leaf node of the tree has a hashmap to -// store its children nodes. Leaf nodes has no hashmap (a nil pointer) -type tree struct { - Root *treeNode -} - -// A treeNode wraps a Node. It has a hashmap to keep records of its children treeNodes. -type treeNode struct { - InternalNode Node - Dir bool - NodeMap map[string]*treeNode -} - -// TreeNode with its key. We use it when we need to sort the treeNodes. -type tnWithKey struct { - key string - tn *treeNode -} - -// Define type and functions to match sort interface -type tnWithKeySlice []tnWithKey - -func (s tnWithKeySlice) Len() int { return len(s) } -func (s tnWithKeySlice) Less(i, j int) bool { return s[i].key < s[j].key } -func (s tnWithKeySlice) Swap(i, j int) { s[i], s[j] = s[j], s[i] } - -// CONSTANT VARIABLE - -// Represent an empty node -var emptyNode = Node{"", PERMANENT, nil} - -//------------------------------------------------------------------------------ -// -// Methods -// -//------------------------------------------------------------------------------ - -// Set the key to the given value, return true if success -// If any intermidate path of the key is not a directory type, it will fail -// For example if the /foo = Node(bar) exists, set /foo/foo = Node(barbar) -// will fail. -func (t *tree) set(key string, value Node) bool { - - nodesName := split(key) - - // avoid set value to "/" - if len(nodesName) == 1 && len(nodesName[0]) == 0 { - return false - } - - nodeMap := t.Root.NodeMap - - i := 0 - newDir := false - - // go through all the path - for i = 0; i < len(nodesName)-1; i++ { - - // if we meet a new directory, all the directory after it must be new - if newDir { - tn := &treeNode{emptyNode, true, make(map[string]*treeNode)} - nodeMap[nodesName[i]] = tn - nodeMap = tn.NodeMap - continue - } - - // get the node from the nodeMap of the current level - tn, ok := nodeMap[nodesName[i]] - - if !ok { - // add a new directory and set newDir to true - newDir = true - tn := &treeNode{emptyNode, true, make(map[string]*treeNode)} - nodeMap[nodesName[i]] = tn - nodeMap = tn.NodeMap - - } else if ok && !tn.Dir { - - // if we meet a non-directory node, we cannot set the key - return false - } else { - - // update the nodeMap to next level - nodeMap = tn.NodeMap - } - - } - - // Add the last node - tn, ok := nodeMap[nodesName[i]] - - if !ok { - // we add a new treeNode - tn := &treeNode{value, false, nil} - nodeMap[nodesName[i]] = tn - - } else { - if tn.Dir { - return false - } - // we change the value of a old Treenode - tn.InternalNode = value - } - return true - -} - -// Get the tree node of the key -func (t *tree) internalGet(key string) (*treeNode, bool) { - nodesName := split(key) - - // should be able to get root - if len(nodesName) == 1 && nodesName[0] == "" { - return t.Root, true - } - - nodeMap := t.Root.NodeMap - - var i int - - for i = 0; i < len(nodesName)-1; i++ { - node, ok := nodeMap[nodesName[i]] - if !ok || !node.Dir { - return nil, false - } - nodeMap = node.NodeMap - } - - tn, ok := nodeMap[nodesName[i]] - if ok { - return tn, ok - } else { - return nil, ok - } -} - -// get the internalNode of the key -func (t *tree) get(key string) (Node, bool) { - tn, ok := t.internalGet(key) - - if ok { - if tn.Dir { - return emptyNode, false - } - return tn.InternalNode, ok - } else { - return emptyNode, ok - } -} - -// get the internalNode of the key -func (t *tree) list(directory string) (interface{}, []string, bool) { - treeNode, ok := t.internalGet(directory) - - if !ok { - return nil, nil, ok - - } else { - if !treeNode.Dir { - return &treeNode.InternalNode, nil, ok - } - length := len(treeNode.NodeMap) - nodes := make([]*Node, length) - keys := make([]string, length) - - i := 0 - for key, node := range treeNode.NodeMap { - nodes[i] = &node.InternalNode - keys[i] = key - i++ - } - - return nodes, keys, ok - } -} - -// delete the key, return true if success -func (t *tree) delete(key string) bool { - nodesName := split(key) - - nodeMap := t.Root.NodeMap - - var i int - - for i = 0; i < len(nodesName)-1; i++ { - node, ok := nodeMap[nodesName[i]] - if !ok || !node.Dir { - return false - } - nodeMap = node.NodeMap - } - - node, ok := nodeMap[nodesName[i]] - if ok && !node.Dir { - delete(nodeMap, nodesName[i]) - return true - } - return false -} - -// traverse wrapper -func (t *tree) traverse(f func(string, *Node), sort bool) { - if sort { - sortDfs("", t.Root, f) - } else { - dfs("", t.Root, f) - } -} - -// clone() will return a deep cloned tree -func (t *tree) clone() *tree { - newTree := new(tree) - newTree.Root = &treeNode{ - Node{ - "/", - time.Unix(0, 0), - nil, - }, - true, - make(map[string]*treeNode), - } - recursiveClone(t.Root, newTree.Root) - return newTree -} - -// recursiveClone is a helper function for clone() -func recursiveClone(tnSrc *treeNode, tnDes *treeNode) { - if !tnSrc.Dir { - tnDes.InternalNode = tnSrc.InternalNode - return - - } else { - tnDes.InternalNode = tnSrc.InternalNode - tnDes.Dir = true - tnDes.NodeMap = make(map[string]*treeNode) - - for key, tn := range tnSrc.NodeMap { - newTn := new(treeNode) - recursiveClone(tn, newTn) - tnDes.NodeMap[key] = newTn - } - - } -} - -// deep first search to traverse the tree -// apply the func f to each internal node -func dfs(key string, t *treeNode, f func(string, *Node)) { - - // base case - if len(t.NodeMap) == 0 { - f(key, &t.InternalNode) - - // recursion - } else { - for tnKey, tn := range t.NodeMap { - tnKey := key + "/" + tnKey - dfs(tnKey, tn, f) - } - } -} - -// sort deep first search to traverse the tree -// apply the func f to each internal node -func sortDfs(key string, t *treeNode, f func(string, *Node)) { - // base case - if len(t.NodeMap) == 0 { - f(key, &t.InternalNode) - - // recursion - } else { - - s := make(tnWithKeySlice, len(t.NodeMap)) - i := 0 - - // copy - for tnKey, tn := range t.NodeMap { - tnKey := key + "/" + tnKey - s[i] = tnWithKey{tnKey, tn} - i++ - } - - // sort - sort.Sort(s) - - // traverse - for i = 0; i < len(t.NodeMap); i++ { - sortDfs(s[i].key, s[i].tn, f) - } - } -} - -// split the key by '/', get the intermediate node name -func split(key string) []string { - key = "/" + key - key = path.Clean(key) - - // get the intermidate nodes name - nodesName := strings.Split(key, "/") - // we do not need the root node, since we start with it - nodesName = nodesName[1:] - return nodesName -} diff --git a/store/tree_store_test.go b/store/tree_store_test.go deleted file mode 100644 index ad8222ffb47..00000000000 --- a/store/tree_store_test.go +++ /dev/null @@ -1,247 +0,0 @@ -package store - -import ( - "fmt" - "math/rand" - "strconv" - "testing" - "time" -) - -func TestStoreGet(t *testing.T) { - - ts := &tree{ - &treeNode{ - NewTestNode("/"), - true, - make(map[string]*treeNode), - }, - } - - // create key - ts.set("/foo", NewTestNode("bar")) - // change value - ts.set("/foo", NewTestNode("barbar")) - // create key - ts.set("/hello/foo", NewTestNode("barbarbar")) - treeNode, ok := ts.get("/foo") - - if !ok { - t.Fatalf("Expect to get node, but not") - } - if treeNode.Value != "barbar" { - t.Fatalf("Expect value barbar, but got %s", treeNode.Value) - } - - // create key - treeNode, ok = ts.get("/hello/foo") - if !ok { - t.Fatalf("Expect to get node, but not") - } - if treeNode.Value != "barbarbar" { - t.Fatalf("Expect value barbarbar, but got %s", treeNode.Value) - } - - // create a key under other key - ok = ts.set("/foo/foo", NewTestNode("bar")) - if ok { - t.Fatalf("shoud not add key under a exisiting key") - } - - // delete a key - ok = ts.delete("/foo") - if !ok { - t.Fatalf("cannot delete key") - } - - // delete a directory - ok = ts.delete("/hello") - if ok { - t.Fatalf("Expect cannot delet /hello, but deleted! ") - } - - // test list - ts.set("/hello/fooo", NewTestNode("barbarbar")) - ts.set("/hello/foooo/foo", NewTestNode("barbarbar")) - - nodes, keys, ok := ts.list("/hello") - - if !ok { - t.Fatalf("cannot list!") - } else { - nodes, _ := nodes.([]*Node) - length := len(nodes) - - for i := 0; i < length; i++ { - fmt.Println(keys[i], "=", nodes[i].Value) - } - } - - keys = GenKeys(100, 10) - - for i := 0; i < 100; i++ { - value := strconv.Itoa(rand.Int()) - ts.set(keys[i], NewTestNode(value)) - treeNode, ok := ts.get(keys[i]) - - if !ok { - continue - } - if treeNode.Value != value { - t.Fatalf("Expect value %s, but got %s", value, treeNode.Value) - } - - } - ts.traverse(f, true) -} - -func TestTreeClone(t *testing.T) { - keys := GenKeys(10000, 10) - - ts := &tree{ - &treeNode{ - NewTestNode("/"), - true, - make(map[string]*treeNode), - }, - } - - backTs := &tree{ - &treeNode{ - NewTestNode("/"), - true, - make(map[string]*treeNode), - }, - } - - // generate the first tree - for _, key := range keys { - value := strconv.Itoa(rand.Int()) - ts.set(key, NewTestNode(value)) - backTs.set(key, NewTestNode(value)) - } - - copyTs := ts.clone() - - // test if they are identical - copyTs.traverse(ts.contain, false) - - // remove all the keys from first tree - for _, key := range keys { - ts.delete(key) - } - - // test if they are identical - // make sure changes in the first tree will affect the copy one - copyTs.traverse(backTs.contain, false) - -} - -func BenchmarkTreeStoreSet(b *testing.B) { - - keys := GenKeys(10000, 10) - - b.ResetTimer() - for i := 0; i < b.N; i++ { - - ts := &tree{ - &treeNode{ - NewTestNode("/"), - true, - make(map[string]*treeNode), - }, - } - - for _, key := range keys { - value := strconv.Itoa(rand.Int()) - ts.set(key, NewTestNode(value)) - } - } -} - -func BenchmarkTreeStoreGet(b *testing.B) { - - keys := GenKeys(10000, 10) - - ts := &tree{ - &treeNode{ - NewTestNode("/"), - true, - make(map[string]*treeNode), - }, - } - - for _, key := range keys { - value := strconv.Itoa(rand.Int()) - ts.set(key, NewTestNode(value)) - } - - b.ResetTimer() - for i := 0; i < b.N; i++ { - for _, key := range keys { - ts.get(key) - } - } -} - -func BenchmarkTreeStoreCopy(b *testing.B) { - keys := GenKeys(10000, 10) - - ts := &tree{ - &treeNode{ - NewTestNode("/"), - true, - make(map[string]*treeNode), - }, - } - - for _, key := range keys { - value := strconv.Itoa(rand.Int()) - ts.set(key, NewTestNode(value)) - } - - b.ResetTimer() - for i := 0; i < b.N; i++ { - ts.clone() - } -} - -func BenchmarkTreeStoreList(b *testing.B) { - - keys := GenKeys(10000, 10) - - ts := &tree{ - &treeNode{ - NewTestNode("/"), - true, - make(map[string]*treeNode), - }, - } - - for _, key := range keys { - value := strconv.Itoa(rand.Int()) - ts.set(key, NewTestNode(value)) - } - - b.ResetTimer() - for i := 0; i < b.N; i++ { - for _, key := range keys { - ts.list(key) - } - } -} - -func (t *tree) contain(key string, node *Node) { - _, ok := t.get(key) - if !ok { - panic("tree do not contain the given key") - } -} - -func f(key string, n *Node) { - return -} - -func NewTestNode(value string) Node { - return Node{value, time.Unix(0, 0), nil} -} diff --git a/store/watcher.go b/store/watcher.go deleted file mode 100644 index 17de27b21b6..00000000000 --- a/store/watcher.go +++ /dev/null @@ -1,129 +0,0 @@ -package store - -import ( - "path" - "strconv" - "strings" -) - -//------------------------------------------------------------------------------ -// -// Typedefs -// -//------------------------------------------------------------------------------ - -// WatcherHub is where the client register its watcher -type WatcherHub struct { - watchers map[string][]*Watcher -} - -// Currently watcher only contains a response channel -type Watcher struct { - C chan *Response -} - -// Create a new watcherHub -func newWatcherHub() *WatcherHub { - w := new(WatcherHub) - w.watchers = make(map[string][]*Watcher) - return w -} - -// Create a new watcher -func NewWatcher() *Watcher { - return &Watcher{C: make(chan *Response, 1)} -} - -// Add a watcher to the watcherHub -func (w *WatcherHub) addWatcher(prefix string, watcher *Watcher, sinceIndex uint64, - responseStartIndex uint64, currentIndex uint64, resMap map[string]*Response) error { - - prefix = path.Clean("/" + prefix) - - if sinceIndex != 0 && sinceIndex >= responseStartIndex { - for i := sinceIndex; i <= currentIndex; i++ { - if checkResponse(prefix, i, resMap) { - watcher.C <- resMap[strconv.FormatUint(i, 10)] - return nil - } - } - } - - _, ok := w.watchers[prefix] - - if !ok { - w.watchers[prefix] = make([]*Watcher, 0) - } - - w.watchers[prefix] = append(w.watchers[prefix], watcher) - - return nil -} - -// Check if the response has what we are watching -func checkResponse(prefix string, index uint64, resMap map[string]*Response) bool { - - resp, ok := resMap[strconv.FormatUint(index, 10)] - - if !ok { - // not storage system command - return false - } else { - path := resp.Key - if strings.HasPrefix(path, prefix) { - prefixLen := len(prefix) - if len(path) == prefixLen || path[prefixLen] == '/' { - return true - } - - } - } - - return false -} - -// Notify the watcher a action happened -func (w *WatcherHub) notify(resp Response) error { - resp.Key = path.Clean(resp.Key) - - segments := strings.Split(resp.Key, "/") - currPath := "/" - - // walk through all the pathes - for _, segment := range segments { - currPath = path.Join(currPath, segment) - - watchers, ok := w.watchers[currPath] - - if ok { - - newWatchers := make([]*Watcher, 0) - // notify all the watchers - for _, watcher := range watchers { - watcher.C <- &resp - } - - if len(newWatchers) == 0 { - // we have notified all the watchers at this path - // delete the map - delete(w.watchers, currPath) - } else { - w.watchers[currPath] = newWatchers - } - } - - } - - return nil -} - -// stopWatchers stops all the watchers -// This function is used when the etcd recovery from a snapshot at runtime -func (w *WatcherHub) stopWatchers() { - for _, subWatchers := range w.watchers { - for _, watcher := range subWatchers { - watcher.C <- nil - } - } - w.watchers = nil -} diff --git a/store/watcher_test.go b/store/watcher_test.go deleted file mode 100644 index b5730ed935a..00000000000 --- a/store/watcher_test.go +++ /dev/null @@ -1,84 +0,0 @@ -package store - -import ( - "testing" - "time" -) - -func TestWatch(t *testing.T) { - - s := CreateStore(100) - - watchers := make([]*Watcher, 10) - - for i, _ := range watchers { - - // create a new watcher - watchers[i] = NewWatcher() - // add to the watchers list - s.AddWatcher("foo", watchers[i], 0) - - } - - s.Set("/foo/foo", "bar", time.Unix(0, 0), 1) - - for _, watcher := range watchers { - - // wait for the notification for any changing - res := <-watcher.C - - if res == nil { - t.Fatal("watcher is cleared") - } - } - - for i, _ := range watchers { - - // create a new watcher - watchers[i] = NewWatcher() - // add to the watchers list - s.AddWatcher("foo/foo/foo", watchers[i], 0) - - } - - s.watcher.stopWatchers() - - for _, watcher := range watchers { - - // wait for the notification for any changing - res := <-watcher.C - - if res != nil { - t.Fatal("watcher is cleared") - } - } -} - -// BenchmarkWatch creates 10K watchers watch at /foo/[path] each time. -// Path is randomly chosen with max depth 10. -// It should take less than 15ms to wake up 10K watchers. -func BenchmarkWatch(b *testing.B) { - s := CreateStore(100) - - keys := GenKeys(10000, 10) - - b.ResetTimer() - for i := 0; i < b.N; i++ { - watchers := make([]*Watcher, 10000) - for i := 0; i < 10000; i++ { - // create a new watcher - watchers[i] = NewWatcher() - // add to the watchers list - s.AddWatcher(keys[i], watchers[i], 0) - } - - s.watcher.stopWatchers() - - for _, watcher := range watchers { - // wait for the notification for any changing - <-watcher.C - } - - s.watcher = newWatcherHub() - } -} diff --git a/util.go b/util.go index 80673d3b07d..960d12f4a64 100644 --- a/util.go +++ b/util.go @@ -47,7 +47,7 @@ var storeMsg chan string // Help to send msg from store to webHub func webHelper() { storeMsg = make(chan string) - etcdStore.SetMessager(storeMsg) + // etcdStore.SetMessager(storeMsg) for { // transfer the new msg to webHub web.Hub().Send(<-storeMsg) @@ -177,6 +177,11 @@ func check(err error) { } } +func getNodePath(urlPath string) string { + pathPrefixLen := len("/" + version + "/keys") + return urlPath[pathPrefixLen:] +} + //-------------------------------------- // Log //-------------------------------------- @@ -259,7 +264,7 @@ func directSet() { func send(c chan bool) { for i := 0; i < 10; i++ { - command := &SetCommand{} + command := &UpdateCommand{} command.Key = "foo" command.Value = "bar" command.ExpireTime = time.Unix(0, 0) From 2022c4bce6ad89cbac58854c717bd54014fc4199 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 16 Sep 2013 09:16:22 -0400 Subject: [PATCH 041/247] fix wait_index --- file_system/watcher.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/file_system/watcher.go b/file_system/watcher.go index 728d11fd413..20a679a64b0 100644 --- a/file_system/watcher.go +++ b/file_system/watcher.go @@ -13,8 +13,9 @@ type watcherHub struct { } type watcher struct { - eventChan chan *Event - recursive bool + eventChan chan *Event + recursive bool + sinceIndex uint64 } func newWatchHub(capacity int) *watcherHub { @@ -43,8 +44,9 @@ func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (<-chan } w := &watcher{ - eventChan: eventChan, - recursive: recursive, + eventChan: eventChan, + recursive: recursive, + sinceIndex: index, } l, ok := wh.watchers[prefix] @@ -85,7 +87,7 @@ func (wh *watcherHub) notifyWithPath(e *Event, path string, force bool) { w, _ := curr.Value.(*watcher) - if w.recursive || force || e.Key == path { + if (w.recursive || force || e.Key == path) && e.Index >= w.sinceIndex { w.eventChan <- e l.Remove(curr) } else { From a568c6dc75e674166864a30ca5e8c219578c7c42 Mon Sep 17 00:00:00 2001 From: evan-gu Date: Mon, 23 Sep 2013 01:55:22 -0400 Subject: [PATCH 042/247] add some basic stats, and test case --- etcd_handlers.go | 1 + file_system/event.go | 2 +- file_system/file_system.go | 42 +++++- file_system/file_system_test.go | 9 +- file_system/node.go | 2 +- file_system/stats.go | 146 +++++++++++++++++++++ file_system/stats_test.go | 221 ++++++++++++++++++++++++++++++++ file_system/watcher.go | 12 +- file_system/watcher_test.go | 3 +- 9 files changed, 421 insertions(+), 17 deletions(-) create mode 100644 file_system/stats.go create mode 100644 file_system/stats_test.go diff --git a/etcd_handlers.go b/etcd_handlers.go index cde882c2ed7..d8f42fc7ed7 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -224,6 +224,7 @@ func VersionHttpHandler(w http.ResponseWriter, req *http.Request) error { func StatsHttpHandler(w http.ResponseWriter, req *http.Request) error { w.WriteHeader(http.StatusOK) //w.Write(etcdStore.Stats()) + w.Write(etcdFs.Stats.GetStats()) w.Write(r.Stats()) return nil } diff --git a/file_system/event.go b/file_system/event.go index f5b3eeb858d..3053d85cd37 100644 --- a/file_system/event.go +++ b/file_system/event.go @@ -6,7 +6,7 @@ import ( "sync" "time" - etcdErr "github.com/coreos/etcd/error" + etcdErr "github.com/xiangli-cmu/etcd/error" ) const ( diff --git a/file_system/file_system.go b/file_system/file_system.go index 1de7830fb8d..0ed740253c4 100644 --- a/file_system/file_system.go +++ b/file_system/file_system.go @@ -8,7 +8,7 @@ import ( "strings" "time" - etcdErr "github.com/coreos/etcd/error" + etcdErr "github.com/xiangli-cmu/etcd/error" ) type FileSystem struct { @@ -16,13 +16,16 @@ type FileSystem struct { WatcherHub *watcherHub Index uint64 Term uint64 + Stats *EtcdStats } func New() *FileSystem { - return &FileSystem{ - Root: newDir("/", 0, 0, nil, "", Permanent), - WatcherHub: newWatchHub(1000), - } + fs := new(FileSystem) + fs.Root = newDir("/", 0, 0, nil, "", Permanent) + fs.Stats = newStats() + fs.WatcherHub = newWatchHub(1000, fs.Stats) + + return fs } @@ -32,6 +35,7 @@ func (fs *FileSystem) Get(nodePath string, recursive, sorted bool, index uint64, n, err := fs.InternalGet(nodePath, index, term) if err != nil { + fs.Stats.IncStats(StatsGetsMiss) return nil, err } @@ -78,6 +82,7 @@ func (fs *FileSystem) Get(nodePath string, recursive, sorted bool, index uint64, e.TTL = int64(n.ExpireTime.Sub(time.Now())/time.Second) + 1 } + fs.Stats.IncStats(StatsGetsHit) return e, nil } @@ -91,12 +96,14 @@ func (fs *FileSystem) Create(nodePath string, value string, expireTime time.Time _, err := fs.InternalGet(nodePath, index, term) if err == nil { // key already exists + fs.Stats.IncStats(StatsSetsMiss) return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath) } etcdError, _ := err.(etcdErr.Error) if etcdError.ErrorCode == 104 { // we cannot create the key due to meet a file while walking + fs.Stats.IncStats(StatsSetsMiss) return nil, err } @@ -106,6 +113,7 @@ func (fs *FileSystem) Create(nodePath string, value string, expireTime time.Time d, err := fs.walk(dir, fs.checkDir) if err != nil { + fs.Stats.IncStats(StatsSetsMiss) return nil, err } @@ -128,6 +136,7 @@ func (fs *FileSystem) Create(nodePath string, value string, expireTime time.Time err = d.Add(n) if err != nil { + fs.Stats.IncStats(StatsSetsMiss) return nil, err } @@ -139,6 +148,7 @@ func (fs *FileSystem) Create(nodePath string, value string, expireTime time.Time } fs.WatcherHub.notify(e) + fs.Stats.IncStats(StatsSetsHit) return e, nil } @@ -149,6 +159,7 @@ func (fs *FileSystem) Update(nodePath string, value string, expireTime time.Time n, err := fs.InternalGet(nodePath, index, term) if err != nil { // if the node does not exist, return error + fs.Stats.IncStats(StatsUpdatesMiss) return nil, err } @@ -157,6 +168,7 @@ func (fs *FileSystem) Update(nodePath string, value string, expireTime time.Time if n.IsDir() { // if the node is a directory, we can only update ttl if len(value) != 0 { + fs.Stats.IncStats(StatsUpdatesMiss) return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath) } @@ -183,6 +195,7 @@ func (fs *FileSystem) Update(nodePath string, value string, expireTime time.Time } fs.WatcherHub.notify(e) + fs.Stats.IncStats(StatsUpdatesHit) return e, nil } @@ -192,11 +205,12 @@ func (fs *FileSystem) TestAndSet(nodePath string, prevValue string, prevIndex ui f, err := fs.InternalGet(nodePath, index, term) if err != nil { - + fs.Stats.IncStats(StatsTestAndSetsMiss) return nil, err } if f.IsDir() { // can only test and set file + fs.Stats.IncStats(StatsTestAndSetsMiss) return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath) } @@ -208,11 +222,12 @@ func (fs *FileSystem) TestAndSet(nodePath string, prevValue string, prevIndex ui f.Write(value, index, term) fs.WatcherHub.notify(e) - + fs.Stats.IncStats(StatsTestAndSetsHit) return e, nil } cause := fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, f.Value, prevIndex, f.ModifiedIndex) + fs.Stats.IncStats(StatsTestAndSetsMiss) return nil, etcdErr.NewError(etcdErr.EcodeTestFailed, cause) } @@ -222,6 +237,7 @@ func (fs *FileSystem) Delete(nodePath string, recursive bool, index uint64, term n, err := fs.InternalGet(nodePath, index, term) if err != nil { // if the node does not exist, return error + fs.Stats.IncStats(StatsDeletesMiss) return nil, err } @@ -240,10 +256,12 @@ func (fs *FileSystem) Delete(nodePath string, recursive bool, index uint64, term err = n.Remove(recursive, callback) if err != nil { + fs.Stats.IncStats(StatsDeletesMiss) return nil, err } fs.WatcherHub.notify(e) + fs.Stats.IncStats(StatsDeletesHit) return e, nil } @@ -333,15 +351,23 @@ func (fs *FileSystem) checkDir(parent *Node, dirName string) (*Node, error) { // Save function will not save the parent field of the node. Or there will // be cyclic dependencies issue for the json package. func (fs *FileSystem) Save() ([]byte, error) { + + fs.Stats.IncStats(StatsSaveHit) cloneFs := New() cloneFs.Root = fs.Root.Clone() b, err := json.Marshal(fs) if err != nil { + fs.Stats.IncStats(StatsSaveMiss) + fs.Stats.rwlock.Lock() + fs.Stats.SaveHit-- // restore the savehit + fs.Stats.rwlock.Unlock() + return nil, err } + fs.Stats.IncStats(StatsSaveHit) return b, nil } @@ -353,9 +379,11 @@ func (fs *FileSystem) Recovery(state []byte) error { err := json.Unmarshal(state, fs) if err != nil { + fs.Stats.IncStats(StatsRecoveryMiss) return err } fs.Root.recoverAndclean() + fs.Stats.IncStats(StatsRecoveryHit) return nil } diff --git a/file_system/file_system_test.go b/file_system/file_system_test.go index d36c86e2a55..0478f9c3083 100644 --- a/file_system/file_system_test.go +++ b/file_system/file_system_test.go @@ -439,14 +439,12 @@ func TestSaveAndRecover(t *testing.T) { expire := time.Now().Add(time.Second) fs.Create("/foo/foo", "bar", expire, 1, 1) - - b := fs.Save() + b, err := fs.Save() cloneFs := New() - time.Sleep(time.Second) - cloneFs.Recover(b) + cloneFs.Recovery(b) for i, k := range keys { _, err := cloneFs.Get(k, false, false, uint64(i), 1) @@ -466,7 +464,7 @@ func TestSaveAndRecover(t *testing.T) { } } - _, err := fs.Get("/foo/foo", false, false, 1, 1) + _, err = fs.Get("/foo/foo", false, false, 1, 1) if err == nil || err.Error() != "Key Not Found" { t.Fatalf("can get the node after deletion ") @@ -476,6 +474,7 @@ func TestSaveAndRecover(t *testing.T) { // GenKeys randomly generate num of keys with max depth func GenKeys(num int, depth int) []string { + rand.Seed(time.Now().UnixNano()) keys := make([]string, num) for i := 0; i < num; i++ { diff --git a/file_system/node.go b/file_system/node.go index 872aacd4c60..587cfb43020 100644 --- a/file_system/node.go +++ b/file_system/node.go @@ -6,7 +6,7 @@ import ( "sync" "time" - etcdErr "github.com/coreos/etcd/error" + etcdErr "github.com/xiangli-cmu/etcd/error" ) var ( diff --git a/file_system/stats.go b/file_system/stats.go new file mode 100644 index 00000000000..c05655efc9f --- /dev/null +++ b/file_system/stats.go @@ -0,0 +1,146 @@ +package fileSystem + +import ( + "encoding/json" + "sync" +) + +const ( + // Operations that will be running serializely + StatsSetsHit = 100 + StatsSetsMiss = 101 + StatsDeletesHit = 102 + StatsDeletesMiss = 103 + StatsUpdatesHit = 104 + StatsUpdatesMiss = 105 + StatsTestAndSetsHit = 106 + StatsTestAndSetsMiss = 107 + StatsRecoveryHit = 108 + StatsRecoveryMiss = 109 + + // concurrent operations + StatsGetsHit = 200 + StatsGetsMiss = 201 + + StatsWatchHit = 300 + StatsWatchMiss = 301 + StatsInWatchingNum = 302 + + StatsSaveHit = 400 + StatsSaveMiss = 401 +) + +type EtcdStats struct { + + // Lock for synchronization + rwlock sync.RWMutex + + // Number of get requests + GetsHit uint64 `json:"gets_hits"` + GetsMiss uint64 `json:"gets_misses"` + + // Number of sets requests + SetsHit uint64 `json:"sets_hits"` + SetsMiss uint64 `json:"sets_misses"` + + // Number of delete requests + DeletesHit uint64 `json:"deletes_hits"` + DeletesMiss uint64 `json:"deletes_misses"` + + // Number of update requests + UpdatesHit uint64 `json:"updates_hits"` + UpdatesMiss uint64 `json:"updates_misses"` + + // Number of testAndSet requests + TestAndSetsHit uint64 `json:"testAndSets_hits"` + TestAndSetsMiss uint64 `json:"testAndSets_misses"` + + // Number of Watch requests + WatchHit uint64 `json:"watch_hit"` + WatchMiss uint64 `json:"watch_miss"` + InWatchingNum uint64 `json:"in_watching_number"` + + // Number of save requests + SaveHit uint64 `json:"save_hit"` + SaveMiss uint64 `json:"save_miss"` + + // Number of recovery requests + RecoveryHit uint64 `json:"recovery_hit"` + RecoveryMiss uint64 `json:"recovery_miss"` +} + +func newStats() *EtcdStats { + e := new(EtcdStats) + return e +} + +// Status() return the statistics info of etcd storage its recent start +func (e *EtcdStats) GetStats() []byte { + b, _ := json.Marshal(e) + return b +} + +func (e *EtcdStats) TotalReads() uint64 { + return e.GetsHit + e.GetsMiss +} + +func (e *EtcdStats) TotalWrites() uint64 { + return e.SetsHit + e.SetsMiss + + e.DeletesHit + e.DeletesMiss + + e.UpdatesHit + e.UpdatesMiss + + e.TestAndSetsHit + e.TestAndSetsMiss +} + +func (e *EtcdStats) IncStats(field int) { + if field >= 200 { + e.rwlock.Lock() + + switch field { + case StatsGetsHit: + e.GetsHit++ + case StatsGetsMiss: + e.GetsMiss++ + case StatsWatchHit: + e.WatchHit++ + case StatsWatchMiss: + e.WatchMiss++ + case StatsInWatchingNum: + e.InWatchingNum++ + case StatsSaveHit: + e.SaveHit++ + case StatsSaveMiss: + e.SaveMiss++ + } + + e.rwlock.Unlock() + + } else { + e.rwlock.RLock() + + switch field { + case StatsSetsHit: + e.SetsHit++ + case StatsSetsMiss: + e.SetsMiss++ + case StatsDeletesHit: + e.DeletesHit++ + case StatsDeletesMiss: + e.DeletesMiss++ + case StatsUpdatesHit: + e.UpdatesHit++ + case StatsUpdatesMiss: + e.UpdatesMiss++ + case StatsTestAndSetsHit: + e.TestAndSetsHit++ + case StatsTestAndSetsMiss: + e.TestAndSetsMiss++ + case StatsRecoveryHit: + e.RecoveryHit++ + case StatsRecoveryMiss: + e.RecoveryMiss++ + } + + e.rwlock.RUnlock() + } + +} diff --git a/file_system/stats_test.go b/file_system/stats_test.go new file mode 100644 index 00000000000..38bc42c4ff4 --- /dev/null +++ b/file_system/stats_test.go @@ -0,0 +1,221 @@ +package fileSystem + +import ( + "math/rand" + "testing" + "time" + //"fmt" +) + +func TestBasicStats(t *testing.T) { + fs := New() + keys := GenKeys(rand.Intn(100), 5) + + i := uint64(0) + GetsHit := uint64(0) + GetsMiss := uint64(0) + SetsHit := uint64(0) + SetsMiss := uint64(0) + DeletesHit := uint64(0) + DeletesMiss := uint64(0) + UpdatesHit := uint64(0) + UpdatesMiss := uint64(0) + TestAndSetsHit := uint64(0) + TestAndSetsMiss := uint64(0) + WatchHit := uint64(0) + WatchMiss := uint64(0) + InWatchingNum := uint64(0) + SaveHit := uint64(0) + SaveMiss := uint64(0) + RecoveryHit := uint64(0) + RecoveryMiss := uint64(0) + + for _, k := range keys { + i++ + _, err := fs.Create(k, "bar", time.Now().Add(time.Second*time.Duration(rand.Intn(10))), i, 1) + if err != nil { + SetsMiss++ + } else { + SetsHit++ + } + } + + for _, k := range keys { + _, err := fs.Get(k, false, false, i, 1) + if err != nil { + GetsMiss++ + } else { + GetsHit++ + } + } + + for _, k := range keys { + i++ + _, err := fs.Update(k, "foo", time.Now().Add(time.Second*time.Duration(rand.Intn(5))), i, 1) + if err != nil { + UpdatesMiss++ + } else { + UpdatesHit++ + } + } + + for _, k := range keys { + _, err := fs.Get(k, false, false, i, 1) + if err != nil { + GetsMiss++ + } else { + GetsHit++ + } + } + + for _, k := range keys { + i++ + _, err := fs.TestAndSet(k, "foo", 0, "bar", Permanent, i, 1) + if err != nil { + TestAndSetsMiss++ + } else { + TestAndSetsHit++ + } + } + + //fmt.Printf("#TestAndSet [%d]\n", TestAndSetsHit) + + for _, k := range keys { + _, err := fs.Watch(k, false, 0, i, 1) + if err != nil { + WatchMiss++ + } else { + WatchHit++ + InWatchingNum++ + } + } + + //fmt.Printf("#Watch [%d]\n", WatchHit) + + for _, k := range keys { + _, err := fs.Get(k, false, false, i, 1) + if err != nil { + GetsMiss++ + } else { + GetsHit++ + } + } + + //fmt.Println("fs.index ", fs.Index) + for j := 0; j < 5; j++ { + b := make([]byte, 10) + err := fs.Recovery(b) + if err != nil { + RecoveryMiss++ + } + + b, err = fs.Save() + if err != nil { + SaveMiss++ + } else { + SaveHit++ + } + + err = fs.Recovery(b) + if err != nil { + RecoveryMiss++ + } else { + RecoveryHit++ + } + } + //fmt.Println("fs.index after ", fs.Index) + //fmt.Println("stats.inwatching ", fs.Stats.InWatchingNum) + + for _, k := range keys { + i++ + _, err := fs.Delete(k, false, i, 1) + if err != nil { + DeletesMiss++ + } else { + InWatchingNum-- + DeletesHit++ + } + } + + //fmt.Printf("#Delete [%d] stats.deletehit [%d] \n", DeletesHit, fs.Stats.DeletesHit) + + for _, k := range keys { + _, err := fs.Get(k, false, false, i, 1) + if err != nil { + GetsMiss++ + } else { + GetsHit++ + } + } + + if GetsHit != fs.Stats.GetsHit { + t.Fatalf("GetsHit [%d] != Stats.GetsHit [%d]", GetsHit, fs.Stats.GetsHit) + } + + if GetsMiss != fs.Stats.GetsMiss { + t.Fatalf("GetsMiss [%d] != Stats.GetsMiss [%d]", GetsMiss, fs.Stats.GetsMiss) + } + + if SetsHit != fs.Stats.SetsHit { + t.Fatalf("SetsHit [%d] != Stats.SetsHit [%d]", SetsHit, fs.Stats.SetsHit) + } + + if SetsMiss != fs.Stats.SetsMiss { + t.Fatalf("SetsMiss [%d] != Stats.SetsMiss [%d]", SetsMiss, fs.Stats.SetsMiss) + } + + if DeletesHit != fs.Stats.DeletesHit { + t.Fatalf("DeletesHit [%d] != Stats.DeletesHit [%d]", DeletesHit, fs.Stats.DeletesHit) + } + + if DeletesMiss != fs.Stats.DeletesMiss { + t.Fatalf("DeletesMiss [%d] != Stats.DeletesMiss [%d]", DeletesMiss, fs.Stats.DeletesMiss) + } + + if UpdatesHit != fs.Stats.UpdatesHit { + t.Fatalf("UpdatesHit [%d] != Stats.UpdatesHit [%d]", UpdatesHit, fs.Stats.UpdatesHit) + } + + if UpdatesMiss != fs.Stats.UpdatesMiss { + t.Fatalf("UpdatesMiss [%d] != Stats.UpdatesMiss [%d]", UpdatesMiss, fs.Stats.UpdatesMiss) + } + + if TestAndSetsHit != fs.Stats.TestAndSetsHit { + t.Fatalf("TestAndSetsHit [%d] != Stats.TestAndSetsHit [%d]", TestAndSetsHit, fs.Stats.TestAndSetsHit) + } + + if TestAndSetsMiss != fs.Stats.TestAndSetsMiss { + t.Fatalf("TestAndSetsMiss [%d] != Stats.TestAndSetsMiss [%d]", TestAndSetsMiss, fs.Stats.TestAndSetsMiss) + } + + if SaveHit != fs.Stats.SaveHit { + t.Fatalf("SaveHit [%d] != Stats.SaveHit [%d]", SaveHit, fs.Stats.SaveHit) + } + + if SaveMiss != fs.Stats.SaveMiss { + t.Fatalf("SaveMiss [%d] != Stats.SaveMiss [%d]", SaveMiss, fs.Stats.SaveMiss) + } + + if WatchHit != fs.Stats.WatchHit { + t.Fatalf("WatchHit [%d] != Stats.WatchHit [%d]", WatchHit, fs.Stats.WatchHit) + } + + if WatchMiss != fs.Stats.WatchMiss { + t.Fatalf("WatchMiss [%d] != Stats.WatchMiss [%d]", WatchMiss, fs.Stats.WatchMiss) + } + + if InWatchingNum != fs.Stats.InWatchingNum { + t.Fatalf("InWatchingNum [%d] != Stats.InWatchingNum [%d]", InWatchingNum, fs.Stats.InWatchingNum) + } + + if RecoveryHit != fs.Stats.RecoveryHit { + t.Fatalf("RecoveryHit [%d] != Stats.RecoveryHit [%d]", RecoveryHit, fs.Stats.RecoveryHit) + } + + if RecoveryMiss != fs.Stats.RecoveryMiss { + t.Fatalf("RecoveryMiss [%d] != Stats.RecoveryMiss [%d]", RecoveryMiss, fs.Stats.RecoveryMiss) + } + + //fmt.Println(GetsHit, GetsMiss, SetsHit, SetsMiss, DeletesHit, DeletesMiss, UpdatesHit, UpdatesMiss, TestAndSetsHit, TestAndSetsMiss, WatchHit, WatchMiss, InWatchingNum, SaveHit, SaveMiss, RecoveryHit, RecoveryMiss) + +} diff --git a/file_system/watcher.go b/file_system/watcher.go index 20a679a64b0..30fc89da784 100644 --- a/file_system/watcher.go +++ b/file_system/watcher.go @@ -10,6 +10,7 @@ type watcherHub struct { watchers map[string]*list.List count uint64 // current number of watchers EventHistory *EventHistory + Stats *EtcdStats } type watcher struct { @@ -18,10 +19,11 @@ type watcher struct { sinceIndex uint64 } -func newWatchHub(capacity int) *watcherHub { +func newWatchHub(capacity int, stats *EtcdStats) *watcherHub { return &watcherHub{ watchers: make(map[string]*list.List), EventHistory: newEventHistory(capacity), + Stats: stats, } } @@ -35,9 +37,13 @@ func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (<-chan e, err := wh.EventHistory.scan(prefix, index) if err != nil { + wh.Stats.IncStats(StatsWatchMiss) return nil, err } + wh.Stats.IncStats(StatsWatchHit) + wh.Stats.IncStats(StatsInWatchingNum) + if e != nil { eventChan <- e return eventChan, nil @@ -67,7 +73,6 @@ func (wh *watcherHub) notifyWithPath(e *Event, path string, force bool) { l, ok := wh.watchers[path] if ok { - curr := l.Front() notifiedAll := true @@ -89,6 +94,9 @@ func (wh *watcherHub) notifyWithPath(e *Event, path string, force bool) { if (w.recursive || force || e.Key == path) && e.Index >= w.sinceIndex { w.eventChan <- e + wh.Stats.rwlock.Lock() // lock the InWatchingNum + wh.Stats.InWatchingNum-- + wh.Stats.rwlock.Unlock() l.Remove(curr) } else { notifiedAll = false diff --git a/file_system/watcher_test.go b/file_system/watcher_test.go index f874163bac4..132f367f3a9 100644 --- a/file_system/watcher_test.go +++ b/file_system/watcher_test.go @@ -5,7 +5,8 @@ import ( ) func TestWatcher(t *testing.T) { - wh := newWatchHub(100) + fs := New() + wh := fs.WatcherHub c, err := wh.watch("/foo", true, 0) if err != nil { From da83ee223b517c84c52a41f28972f43ee06d32ac Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 28 Sep 2013 16:26:19 -0700 Subject: [PATCH 043/247] clean up from yifan --- command.go | 20 +- error/error.go | 30 +-- etcd.go | 6 +- etcd_handlers.go | 7 +- file_system/stats.go | 146 ------------ file_system/stats_test.go | 221 ------------------ machines.go | 2 +- name_url_map.go | 2 +- raft_server.go | 2 +- {file_system => store}/event.go | 4 +- {file_system => store}/event_test.go | 2 +- {file_system => store}/node.go | 4 +- store/stats.go | 91 ++++++++ store/stats_test.go | 139 +++++++++++ file_system/file_system.go => store/store.go | 150 ++++++------ .../store_test.go | 172 +++++++------- {file_system => store}/watcher.go | 19 +- {file_system => store}/watcher_test.go | 6 +- util.go | 6 +- 19 files changed, 443 insertions(+), 586 deletions(-) delete mode 100644 file_system/stats.go delete mode 100644 file_system/stats_test.go rename {file_system => store}/event.go (98%) rename {file_system => store}/event_test.go (98%) rename {file_system => store}/node.go (99%) create mode 100644 store/stats.go create mode 100644 store/stats_test.go rename file_system/file_system.go => store/store.go (64%) rename file_system/file_system_test.go => store/store_test.go (68%) rename {file_system => store}/watcher.go (84%) rename {file_system => store}/watcher_test.go (92%) diff --git a/command.go b/command.go index 1c014e4d029..9ab35934bb8 100644 --- a/command.go +++ b/command.go @@ -9,7 +9,7 @@ import ( "time" etcdErr "github.com/coreos/etcd/error" - "github.com/coreos/etcd/file_system" + "github.com/coreos/etcd/store" "github.com/coreos/go-raft" ) @@ -39,7 +39,7 @@ func (c *CreateCommand) CommandName() string { // Create node func (c *CreateCommand) Apply(server *raft.Server) (interface{}, error) { - e, err := etcdFs.Create(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) + e, err := etcdStore.Create(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) if err != nil { debug(err) @@ -63,7 +63,7 @@ func (c *UpdateCommand) CommandName() string { // Update node func (c *UpdateCommand) Apply(server *raft.Server) (interface{}, error) { - e, err := etcdFs.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) + e, err := etcdStore.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) if err != nil { debug(err) @@ -89,7 +89,7 @@ func (c *TestAndSetCommand) CommandName() string { // Set the key-value pair if the current value of the key equals to the given prevValue func (c *TestAndSetCommand) Apply(server *raft.Server) (interface{}, error) { - e, err := etcdFs.TestAndSet(c.Key, c.PrevValue, c.PrevIndex, + e, err := etcdStore.TestAndSet(c.Key, c.PrevValue, c.PrevIndex, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) if err != nil { @@ -114,7 +114,7 @@ func (c *GetCommand) CommandName() string { // Get the value of key func (c *GetCommand) Apply(server *raft.Server) (interface{}, error) { - e, err := etcdFs.Get(c.Key, c.Recursive, c.Sorted, server.CommitIndex(), server.Term()) + e, err := etcdStore.Get(c.Key, c.Recursive, c.Sorted, server.CommitIndex(), server.Term()) if err != nil { debug(err) @@ -137,7 +137,7 @@ func (c *DeleteCommand) CommandName() string { // Delete the key func (c *DeleteCommand) Apply(server *raft.Server) (interface{}, error) { - e, err := etcdFs.Delete(c.Key, c.Recursive, server.CommitIndex(), server.Term()) + e, err := etcdStore.Delete(c.Key, c.Recursive, server.CommitIndex(), server.Term()) if err != nil { debug(err) @@ -160,7 +160,7 @@ func (c *WatchCommand) CommandName() string { } func (c *WatchCommand) Apply(server *raft.Server) (interface{}, error) { - eventChan, err := etcdFs.Watch(c.Key, c.Recursive, c.SinceIndex, server.CommitIndex(), server.Term()) + eventChan, err := etcdStore.Watch(c.Key, c.Recursive, c.SinceIndex, server.CommitIndex(), server.Term()) if err != nil { return nil, err @@ -197,7 +197,7 @@ func (c *JoinCommand) CommandName() string { func (c *JoinCommand) Apply(raftServer *raft.Server) (interface{}, error) { // check if the join command is from a previous machine, who lost all its previous log. - e, _ := etcdFs.Get(path.Join("/_etcd/machines", c.Name), false, false, raftServer.CommitIndex(), raftServer.Term()) + e, _ := etcdStore.Get(path.Join("/_etcd/machines", c.Name), false, false, raftServer.CommitIndex(), raftServer.Term()) b := make([]byte, 8) binary.PutUvarint(b, raftServer.CommitIndex()) @@ -221,7 +221,7 @@ func (c *JoinCommand) Apply(raftServer *raft.Server) (interface{}, error) { // add machine in etcd storage key := path.Join("_etcd/machines", c.Name) value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", c.RaftURL, c.EtcdURL, c.RaftVersion) - etcdFs.Create(key, value, fileSystem.Permanent, raftServer.CommitIndex(), raftServer.Term()) + etcdStore.Create(key, value, store.Permanent, raftServer.CommitIndex(), raftServer.Term()) if c.Name != r.Name() { // do not add self to the peer list r.peersStats[c.Name] = &raftPeerStats{MinLatency: 1 << 63} @@ -250,7 +250,7 @@ func (c *RemoveCommand) Apply(raftServer *raft.Server) (interface{}, error) { // remove machine in etcd storage key := path.Join("_etcd/machines", c.Name) - _, err := etcdFs.Delete(key, false, raftServer.CommitIndex(), raftServer.Term()) + _, err := etcdStore.Delete(key, false, raftServer.CommitIndex(), raftServer.Term()) delete(r.peersStats, c.Name) if err != nil { diff --git a/error/error.go b/error/error.go index d97158cf1a9..643c3244b61 100644 --- a/error/error.go +++ b/error/error.go @@ -32,27 +32,27 @@ func init() { errors = make(map[int]string) // command related errors - errors[100] = "Key Not Found" - errors[101] = "Test Failed" //test and set - errors[102] = "Not A File" - errors[103] = "Reached the max number of machines in the cluster" - errors[104] = "Not A Directory" - errors[105] = "Already exists" // create - errors[106] = "The prefix of given key is a keyword in etcd" + errors[EcodeKeyNotFound] = "Key Not Found" + errors[EcodeTestFailed] = "Test Failed" //test and set + errors[EcodeNotFile] = "Not A File" + errors[EcodeNoMoreMachine] = "Reached the max number of machines in the cluster" + errors[EcodeNotDir] = "Not A Directory" + errors[EcodeNodeExist] = "Already exists" // create + errors[EcodeKeyIsPreserved] = "The prefix of given key is a keyword in etcd" // Post form related errors - errors[200] = "Value is Required in POST form" - errors[201] = "PrevValue is Required in POST form" - errors[202] = "The given TTL in POST form is not a number" - errors[203] = "The given index in POST form is not a number" + errors[EcodeValueRequired] = "Value is Required in POST form" + errors[EcodePrevValueRequired] = "PrevValue is Required in POST form" + errors[EcodeTTLNaN] = "The given TTL in POST form is not a number" + errors[EcodeIndexNaN] = "The given index in POST form is not a number" // raft related errors - errors[300] = "Raft Internal Error" - errors[301] = "During Leader Election" + errors[EcodeRaftInternal] = "Raft Internal Error" + errors[EcodeLeaderElect] = "During Leader Election" // etcd related errors - errors[400] = "watcher is cleared due to etcd recovery" - errors[401] = "The event in requested index is outdated and cleared" + errors[EcodeWatcherCleared] = "watcher is cleared due to etcd recovery" + errors[EcodeEventIndexCleared] = "The event in requested index is outdated and cleared" } diff --git a/etcd.go b/etcd.go index cba7809f4d6..80c8dd12283 100644 --- a/etcd.go +++ b/etcd.go @@ -10,7 +10,7 @@ import ( "strings" "time" - "github.com/coreos/etcd/file_system" + "github.com/coreos/etcd/store" "github.com/coreos/go-raft" ) @@ -136,7 +136,7 @@ type TLSConfig struct { // //------------------------------------------------------------------------------ -var etcdFs *fileSystem.FileSystem +var etcdStore *store.Store //------------------------------------------------------------------------------ // @@ -204,7 +204,7 @@ func main() { info := getInfo(dirPath) // Create etcd key-value store - etcdFs = fileSystem.New() + etcdStore = store.New() snapConf = newSnapshotConf() diff --git a/etcd_handlers.go b/etcd_handlers.go index d8f42fc7ed7..09e5e8252b0 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -7,7 +7,7 @@ import ( "strings" etcdErr "github.com/coreos/etcd/error" - "github.com/coreos/etcd/file_system" + "github.com/coreos/etcd/store" "github.com/coreos/go-raft" ) @@ -120,7 +120,7 @@ func UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { } // TODO: update should give at least one option - if value == "" && expireTime.Sub(fileSystem.Permanent) == 0 { + if value == "" && expireTime.Sub(store.Permanent) == 0 { return nil } @@ -223,8 +223,7 @@ func VersionHttpHandler(w http.ResponseWriter, req *http.Request) error { // Handler to return the basic stats of etcd func StatsHttpHandler(w http.ResponseWriter, req *http.Request) error { w.WriteHeader(http.StatusOK) - //w.Write(etcdStore.Stats()) - w.Write(etcdFs.Stats.GetStats()) + w.Write(etcdStore.JsonStats()) w.Write(r.Stats()) return nil } diff --git a/file_system/stats.go b/file_system/stats.go deleted file mode 100644 index c05655efc9f..00000000000 --- a/file_system/stats.go +++ /dev/null @@ -1,146 +0,0 @@ -package fileSystem - -import ( - "encoding/json" - "sync" -) - -const ( - // Operations that will be running serializely - StatsSetsHit = 100 - StatsSetsMiss = 101 - StatsDeletesHit = 102 - StatsDeletesMiss = 103 - StatsUpdatesHit = 104 - StatsUpdatesMiss = 105 - StatsTestAndSetsHit = 106 - StatsTestAndSetsMiss = 107 - StatsRecoveryHit = 108 - StatsRecoveryMiss = 109 - - // concurrent operations - StatsGetsHit = 200 - StatsGetsMiss = 201 - - StatsWatchHit = 300 - StatsWatchMiss = 301 - StatsInWatchingNum = 302 - - StatsSaveHit = 400 - StatsSaveMiss = 401 -) - -type EtcdStats struct { - - // Lock for synchronization - rwlock sync.RWMutex - - // Number of get requests - GetsHit uint64 `json:"gets_hits"` - GetsMiss uint64 `json:"gets_misses"` - - // Number of sets requests - SetsHit uint64 `json:"sets_hits"` - SetsMiss uint64 `json:"sets_misses"` - - // Number of delete requests - DeletesHit uint64 `json:"deletes_hits"` - DeletesMiss uint64 `json:"deletes_misses"` - - // Number of update requests - UpdatesHit uint64 `json:"updates_hits"` - UpdatesMiss uint64 `json:"updates_misses"` - - // Number of testAndSet requests - TestAndSetsHit uint64 `json:"testAndSets_hits"` - TestAndSetsMiss uint64 `json:"testAndSets_misses"` - - // Number of Watch requests - WatchHit uint64 `json:"watch_hit"` - WatchMiss uint64 `json:"watch_miss"` - InWatchingNum uint64 `json:"in_watching_number"` - - // Number of save requests - SaveHit uint64 `json:"save_hit"` - SaveMiss uint64 `json:"save_miss"` - - // Number of recovery requests - RecoveryHit uint64 `json:"recovery_hit"` - RecoveryMiss uint64 `json:"recovery_miss"` -} - -func newStats() *EtcdStats { - e := new(EtcdStats) - return e -} - -// Status() return the statistics info of etcd storage its recent start -func (e *EtcdStats) GetStats() []byte { - b, _ := json.Marshal(e) - return b -} - -func (e *EtcdStats) TotalReads() uint64 { - return e.GetsHit + e.GetsMiss -} - -func (e *EtcdStats) TotalWrites() uint64 { - return e.SetsHit + e.SetsMiss + - e.DeletesHit + e.DeletesMiss + - e.UpdatesHit + e.UpdatesMiss + - e.TestAndSetsHit + e.TestAndSetsMiss -} - -func (e *EtcdStats) IncStats(field int) { - if field >= 200 { - e.rwlock.Lock() - - switch field { - case StatsGetsHit: - e.GetsHit++ - case StatsGetsMiss: - e.GetsMiss++ - case StatsWatchHit: - e.WatchHit++ - case StatsWatchMiss: - e.WatchMiss++ - case StatsInWatchingNum: - e.InWatchingNum++ - case StatsSaveHit: - e.SaveHit++ - case StatsSaveMiss: - e.SaveMiss++ - } - - e.rwlock.Unlock() - - } else { - e.rwlock.RLock() - - switch field { - case StatsSetsHit: - e.SetsHit++ - case StatsSetsMiss: - e.SetsMiss++ - case StatsDeletesHit: - e.DeletesHit++ - case StatsDeletesMiss: - e.DeletesMiss++ - case StatsUpdatesHit: - e.UpdatesHit++ - case StatsUpdatesMiss: - e.UpdatesMiss++ - case StatsTestAndSetsHit: - e.TestAndSetsHit++ - case StatsTestAndSetsMiss: - e.TestAndSetsMiss++ - case StatsRecoveryHit: - e.RecoveryHit++ - case StatsRecoveryMiss: - e.RecoveryMiss++ - } - - e.rwlock.RUnlock() - } - -} diff --git a/file_system/stats_test.go b/file_system/stats_test.go deleted file mode 100644 index 38bc42c4ff4..00000000000 --- a/file_system/stats_test.go +++ /dev/null @@ -1,221 +0,0 @@ -package fileSystem - -import ( - "math/rand" - "testing" - "time" - //"fmt" -) - -func TestBasicStats(t *testing.T) { - fs := New() - keys := GenKeys(rand.Intn(100), 5) - - i := uint64(0) - GetsHit := uint64(0) - GetsMiss := uint64(0) - SetsHit := uint64(0) - SetsMiss := uint64(0) - DeletesHit := uint64(0) - DeletesMiss := uint64(0) - UpdatesHit := uint64(0) - UpdatesMiss := uint64(0) - TestAndSetsHit := uint64(0) - TestAndSetsMiss := uint64(0) - WatchHit := uint64(0) - WatchMiss := uint64(0) - InWatchingNum := uint64(0) - SaveHit := uint64(0) - SaveMiss := uint64(0) - RecoveryHit := uint64(0) - RecoveryMiss := uint64(0) - - for _, k := range keys { - i++ - _, err := fs.Create(k, "bar", time.Now().Add(time.Second*time.Duration(rand.Intn(10))), i, 1) - if err != nil { - SetsMiss++ - } else { - SetsHit++ - } - } - - for _, k := range keys { - _, err := fs.Get(k, false, false, i, 1) - if err != nil { - GetsMiss++ - } else { - GetsHit++ - } - } - - for _, k := range keys { - i++ - _, err := fs.Update(k, "foo", time.Now().Add(time.Second*time.Duration(rand.Intn(5))), i, 1) - if err != nil { - UpdatesMiss++ - } else { - UpdatesHit++ - } - } - - for _, k := range keys { - _, err := fs.Get(k, false, false, i, 1) - if err != nil { - GetsMiss++ - } else { - GetsHit++ - } - } - - for _, k := range keys { - i++ - _, err := fs.TestAndSet(k, "foo", 0, "bar", Permanent, i, 1) - if err != nil { - TestAndSetsMiss++ - } else { - TestAndSetsHit++ - } - } - - //fmt.Printf("#TestAndSet [%d]\n", TestAndSetsHit) - - for _, k := range keys { - _, err := fs.Watch(k, false, 0, i, 1) - if err != nil { - WatchMiss++ - } else { - WatchHit++ - InWatchingNum++ - } - } - - //fmt.Printf("#Watch [%d]\n", WatchHit) - - for _, k := range keys { - _, err := fs.Get(k, false, false, i, 1) - if err != nil { - GetsMiss++ - } else { - GetsHit++ - } - } - - //fmt.Println("fs.index ", fs.Index) - for j := 0; j < 5; j++ { - b := make([]byte, 10) - err := fs.Recovery(b) - if err != nil { - RecoveryMiss++ - } - - b, err = fs.Save() - if err != nil { - SaveMiss++ - } else { - SaveHit++ - } - - err = fs.Recovery(b) - if err != nil { - RecoveryMiss++ - } else { - RecoveryHit++ - } - } - //fmt.Println("fs.index after ", fs.Index) - //fmt.Println("stats.inwatching ", fs.Stats.InWatchingNum) - - for _, k := range keys { - i++ - _, err := fs.Delete(k, false, i, 1) - if err != nil { - DeletesMiss++ - } else { - InWatchingNum-- - DeletesHit++ - } - } - - //fmt.Printf("#Delete [%d] stats.deletehit [%d] \n", DeletesHit, fs.Stats.DeletesHit) - - for _, k := range keys { - _, err := fs.Get(k, false, false, i, 1) - if err != nil { - GetsMiss++ - } else { - GetsHit++ - } - } - - if GetsHit != fs.Stats.GetsHit { - t.Fatalf("GetsHit [%d] != Stats.GetsHit [%d]", GetsHit, fs.Stats.GetsHit) - } - - if GetsMiss != fs.Stats.GetsMiss { - t.Fatalf("GetsMiss [%d] != Stats.GetsMiss [%d]", GetsMiss, fs.Stats.GetsMiss) - } - - if SetsHit != fs.Stats.SetsHit { - t.Fatalf("SetsHit [%d] != Stats.SetsHit [%d]", SetsHit, fs.Stats.SetsHit) - } - - if SetsMiss != fs.Stats.SetsMiss { - t.Fatalf("SetsMiss [%d] != Stats.SetsMiss [%d]", SetsMiss, fs.Stats.SetsMiss) - } - - if DeletesHit != fs.Stats.DeletesHit { - t.Fatalf("DeletesHit [%d] != Stats.DeletesHit [%d]", DeletesHit, fs.Stats.DeletesHit) - } - - if DeletesMiss != fs.Stats.DeletesMiss { - t.Fatalf("DeletesMiss [%d] != Stats.DeletesMiss [%d]", DeletesMiss, fs.Stats.DeletesMiss) - } - - if UpdatesHit != fs.Stats.UpdatesHit { - t.Fatalf("UpdatesHit [%d] != Stats.UpdatesHit [%d]", UpdatesHit, fs.Stats.UpdatesHit) - } - - if UpdatesMiss != fs.Stats.UpdatesMiss { - t.Fatalf("UpdatesMiss [%d] != Stats.UpdatesMiss [%d]", UpdatesMiss, fs.Stats.UpdatesMiss) - } - - if TestAndSetsHit != fs.Stats.TestAndSetsHit { - t.Fatalf("TestAndSetsHit [%d] != Stats.TestAndSetsHit [%d]", TestAndSetsHit, fs.Stats.TestAndSetsHit) - } - - if TestAndSetsMiss != fs.Stats.TestAndSetsMiss { - t.Fatalf("TestAndSetsMiss [%d] != Stats.TestAndSetsMiss [%d]", TestAndSetsMiss, fs.Stats.TestAndSetsMiss) - } - - if SaveHit != fs.Stats.SaveHit { - t.Fatalf("SaveHit [%d] != Stats.SaveHit [%d]", SaveHit, fs.Stats.SaveHit) - } - - if SaveMiss != fs.Stats.SaveMiss { - t.Fatalf("SaveMiss [%d] != Stats.SaveMiss [%d]", SaveMiss, fs.Stats.SaveMiss) - } - - if WatchHit != fs.Stats.WatchHit { - t.Fatalf("WatchHit [%d] != Stats.WatchHit [%d]", WatchHit, fs.Stats.WatchHit) - } - - if WatchMiss != fs.Stats.WatchMiss { - t.Fatalf("WatchMiss [%d] != Stats.WatchMiss [%d]", WatchMiss, fs.Stats.WatchMiss) - } - - if InWatchingNum != fs.Stats.InWatchingNum { - t.Fatalf("InWatchingNum [%d] != Stats.InWatchingNum [%d]", InWatchingNum, fs.Stats.InWatchingNum) - } - - if RecoveryHit != fs.Stats.RecoveryHit { - t.Fatalf("RecoveryHit [%d] != Stats.RecoveryHit [%d]", RecoveryHit, fs.Stats.RecoveryHit) - } - - if RecoveryMiss != fs.Stats.RecoveryMiss { - t.Fatalf("RecoveryMiss [%d] != Stats.RecoveryMiss [%d]", RecoveryMiss, fs.Stats.RecoveryMiss) - } - - //fmt.Println(GetsHit, GetsMiss, SetsHit, SetsMiss, DeletesHit, DeletesMiss, UpdatesHit, UpdatesMiss, TestAndSetsHit, TestAndSetsMiss, WatchHit, WatchMiss, InWatchingNum, SaveHit, SaveMiss, RecoveryHit, RecoveryMiss) - -} diff --git a/machines.go b/machines.go index 1988353d51e..b863c50fb3a 100644 --- a/machines.go +++ b/machines.go @@ -2,7 +2,7 @@ package main // machineNum returns the number of machines in the cluster func machineNum() int { - e, err := etcdFs.Get("/_etcd/machines", false, false, r.CommitIndex(), r.Term()) + e, err := etcdStore.Get("/_etcd/machines", false, false, r.CommitIndex(), r.Term()) if err != nil { return 0 diff --git a/name_url_map.go b/name_url_map.go index 38e1ecc15db..1192f3fda74 100644 --- a/name_url_map.go +++ b/name_url_map.go @@ -56,7 +56,7 @@ func readURL(nodeName string, urlName string) (string, bool) { // convert nodeName to url from etcd storage key := path.Join("/_etcd/machines", nodeName) - e, err := etcdFs.Get(key, false, false, r.CommitIndex(), r.Term()) + e, err := etcdStore.Get(key, false, false, r.CommitIndex(), r.Term()) if err != nil { return "", false diff --git a/raft_server.go b/raft_server.go index d88f266f8b0..506e4771a08 100644 --- a/raft_server.go +++ b/raft_server.go @@ -36,7 +36,7 @@ func newRaftServer(name string, url string, listenHost string, tlsConf *TLSConfi raftTransporter := newTransporter(tlsConf.Scheme, tlsConf.Client, ElectionTimeout) // Create raft server - server, err := raft.NewServer(name, dirPath, raftTransporter, etcdFs, nil) + server, err := raft.NewServer(name, dirPath, raftTransporter, etcdStore, nil) check(err) diff --git a/file_system/event.go b/store/event.go similarity index 98% rename from file_system/event.go rename to store/event.go index 3053d85cd37..ca7b0389b56 100644 --- a/file_system/event.go +++ b/store/event.go @@ -1,4 +1,4 @@ -package fileSystem +package store import ( "fmt" @@ -6,7 +6,7 @@ import ( "sync" "time" - etcdErr "github.com/xiangli-cmu/etcd/error" + etcdErr "github.com/coreos/etcd/error" ) const ( diff --git a/file_system/event_test.go b/store/event_test.go similarity index 98% rename from file_system/event_test.go rename to store/event_test.go index 146f8cd4db3..0d19dd52aee 100644 --- a/file_system/event_test.go +++ b/store/event_test.go @@ -1,4 +1,4 @@ -package fileSystem +package store import ( "testing" diff --git a/file_system/node.go b/store/node.go similarity index 99% rename from file_system/node.go rename to store/node.go index 587cfb43020..8502f70d15c 100644 --- a/file_system/node.go +++ b/store/node.go @@ -1,4 +1,4 @@ -package fileSystem +package store import ( "path" @@ -6,7 +6,7 @@ import ( "sync" "time" - etcdErr "github.com/xiangli-cmu/etcd/error" + etcdErr "github.com/coreos/etcd/error" ) var ( diff --git a/store/stats.go b/store/stats.go new file mode 100644 index 00000000000..1f98625a057 --- /dev/null +++ b/store/stats.go @@ -0,0 +1,91 @@ +package store + +import ( + "encoding/json" + "sync/atomic" +) + +const ( + SetSuccess = 100 + SetFail = 101 + DeleteSuccess = 102 + DeleteFail = 103 + UpdateSuccess = 104 + UpdateFail = 105 + TestAndSetSuccess = 106 + TestAndSetFail = 107 + GetSuccess = 110 + GetFail = 111 +) + +type Stats struct { + + // Number of get requests + GetSuccess uint64 `json:"getsSuccess"` + GetFail uint64 `json:"getsFail"` + + // Number of sets requests + SetSuccess uint64 `json:"setsSuccess"` + SetFail uint64 `json:"setsFail"` + + // Number of delete requests + DeleteSuccess uint64 `json:"deleteSuccess"` + DeleteFail uint64 `json:"deleteFail"` + + // Number of update requests + UpdateSuccess uint64 `json:"updateSuccess"` + UpdateFail uint64 `json:"updateFail"` + + // Number of testAndSet requests + TestAndSetSuccess uint64 `json:"testAndSetSuccess"` + TestAndSetFail uint64 `json:"testAndSetFail"` + + Watchers uint64 `json:"watchers"` +} + +func newStats() *Stats { + s := new(Stats) + return s +} + +// Status() return the statistics info of etcd storage its recent start +func (s *Stats) toJson() []byte { + b, _ := json.Marshal(s) + return b +} + +func (s *Stats) TotalReads() uint64 { + return s.GetSuccess + s.GetFail +} + +func (s *Stats) TotalWrites() uint64 { + return s.SetSuccess + s.SetFail + + s.DeleteSuccess + s.DeleteFail + + s.TestAndSetSuccess + s.TestAndSetFail + + s.UpdateSuccess + s.UpdateFail +} + +func (s *Stats) Inc(field int) { + switch field { + case SetSuccess: + atomic.AddUint64(&s.SetSuccess, 1) + case SetFail: + atomic.AddUint64(&s.SetFail, 1) + case DeleteSuccess: + atomic.AddUint64(&s.DeleteSuccess, 1) + case DeleteFail: + atomic.AddUint64(&s.DeleteFail, 1) + case GetSuccess: + atomic.AddUint64(&s.GetSuccess, 1) + case GetFail: + atomic.AddUint64(&s.GetFail, 1) + case UpdateSuccess: + atomic.AddUint64(&s.UpdateSuccess, 1) + case UpdateFail: + atomic.AddUint64(&s.UpdateFail, 1) + case TestAndSetSuccess: + atomic.AddUint64(&s.TestAndSetSuccess, 1) + case TestAndSetFail: + atomic.AddUint64(&s.TestAndSetFail, 1) + } +} diff --git a/store/stats_test.go b/store/stats_test.go new file mode 100644 index 00000000000..ff67f328a4e --- /dev/null +++ b/store/stats_test.go @@ -0,0 +1,139 @@ +package store + +import ( + "math/rand" + "testing" + "time" +) + +func TestBasicStats(t *testing.T) { + s := New() + keys := GenKeys(rand.Intn(100), 5) + + var i uint64 + var GetSuccess, GetFail, SetSuccess, SetFail, DeleteSuccess, DeleteFail uint64 + var UpdateSuccess, UpdateFail, TestAndSetSuccess, TestAndSetFail, watcher_number uint64 + + for _, k := range keys { + i++ + _, err := s.Create(k, "bar", time.Now().Add(time.Second*time.Duration(rand.Intn(10))), i, 1) + if err != nil { + SetFail++ + } else { + SetSuccess++ + } + } + + for _, k := range keys { + _, err := s.Get(k, false, false, i, 1) + if err != nil { + GetFail++ + } else { + GetSuccess++ + } + } + + for _, k := range keys { + i++ + _, err := s.Update(k, "foo", time.Now().Add(time.Second*time.Duration(rand.Intn(5))), i, 1) + if err != nil { + UpdateFail++ + } else { + UpdateSuccess++ + } + } + + for _, k := range keys { + _, err := s.Get(k, false, false, i, 1) + if err != nil { + GetFail++ + } else { + GetSuccess++ + } + } + + for _, k := range keys { + i++ + _, err := s.TestAndSet(k, "foo", 0, "bar", Permanent, i, 1) + if err != nil { + TestAndSetFail++ + } else { + TestAndSetSuccess++ + } + } + + for _, k := range keys { + s.Watch(k, false, 0, i, 1) + watcher_number++ + } + + for _, k := range keys { + _, err := s.Get(k, false, false, i, 1) + if err != nil { + GetFail++ + } else { + GetSuccess++ + } + } + + for _, k := range keys { + i++ + _, err := s.Delete(k, false, i, 1) + if err != nil { + DeleteFail++ + } else { + watcher_number-- + DeleteSuccess++ + } + } + + for _, k := range keys { + _, err := s.Get(k, false, false, i, 1) + if err != nil { + GetFail++ + } else { + GetSuccess++ + } + } + + if GetSuccess != s.Stats.GetSuccess { + t.Fatalf("GetSuccess [%d] != Stats.GetSuccess [%d]", GetSuccess, s.Stats.GetSuccess) + } + + if GetFail != s.Stats.GetFail { + t.Fatalf("GetFail [%d] != Stats.GetFail [%d]", GetFail, s.Stats.GetFail) + } + + if SetSuccess != s.Stats.SetSuccess { + t.Fatalf("SetSuccess [%d] != Stats.SetSuccess [%d]", SetSuccess, s.Stats.SetSuccess) + } + + if SetFail != s.Stats.SetFail { + t.Fatalf("SetFail [%d] != Stats.SetFail [%d]", SetFail, s.Stats.SetFail) + } + + if DeleteSuccess != s.Stats.DeleteSuccess { + t.Fatalf("DeleteSuccess [%d] != Stats.DeleteSuccess [%d]", DeleteSuccess, s.Stats.DeleteSuccess) + } + + if DeleteFail != s.Stats.DeleteFail { + t.Fatalf("DeleteFail [%d] != Stats.DeleteFail [%d]", DeleteFail, s.Stats.DeleteFail) + } + + if UpdateSuccess != s.Stats.UpdateSuccess { + t.Fatalf("UpdateSuccess [%d] != Stats.UpdateSuccess [%d]", UpdateSuccess, s.Stats.UpdateSuccess) + } + + if UpdateFail != s.Stats.UpdateFail { + t.Fatalf("UpdateFail [%d] != Stats.UpdateFail [%d]", UpdateFail, s.Stats.UpdateFail) + } + + if TestAndSetSuccess != s.Stats.TestAndSetSuccess { + t.Fatalf("TestAndSetSuccess [%d] != Stats.TestAndSetSuccess [%d]", TestAndSetSuccess, s.Stats.TestAndSetSuccess) + } + + if TestAndSetFail != s.Stats.TestAndSetFail { + t.Fatalf("TestAndSetFail [%d] != Stats.TestAndSetFail [%d]", TestAndSetFail, s.Stats.TestAndSetFail) + } + +} diff --git a/file_system/file_system.go b/store/store.go similarity index 64% rename from file_system/file_system.go rename to store/store.go index 0ed740253c4..73c6bf3ec6b 100644 --- a/file_system/file_system.go +++ b/store/store.go @@ -1,4 +1,4 @@ -package fileSystem +package store import ( "encoding/json" @@ -8,34 +8,34 @@ import ( "strings" "time" - etcdErr "github.com/xiangli-cmu/etcd/error" + etcdErr "github.com/coreos/etcd/error" ) -type FileSystem struct { +type Store struct { Root *Node WatcherHub *watcherHub Index uint64 Term uint64 - Stats *EtcdStats + Stats *Stats } -func New() *FileSystem { - fs := new(FileSystem) - fs.Root = newDir("/", 0, 0, nil, "", Permanent) - fs.Stats = newStats() - fs.WatcherHub = newWatchHub(1000, fs.Stats) +func New() *Store { + s := new(Store) + s.Root = newDir("/", 0, 0, nil, "", Permanent) + s.Stats = newStats() + s.WatcherHub = newWatchHub(1000) - return fs + return s } -func (fs *FileSystem) Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) { +func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) { nodePath = path.Clean(path.Join("/", nodePath)) - n, err := fs.InternalGet(nodePath, index, term) + n, err := s.InternalGet(nodePath, index, term) if err != nil { - fs.Stats.IncStats(StatsGetsMiss) + s.Stats.Inc(GetFail) return nil, err } @@ -82,61 +82,62 @@ func (fs *FileSystem) Get(nodePath string, recursive, sorted bool, index uint64, e.TTL = int64(n.ExpireTime.Sub(time.Now())/time.Second) + 1 } - fs.Stats.IncStats(StatsGetsHit) + s.Stats.Inc(GetSuccess) + return e, nil } // Create function creates the Node at nodePath. Create will help to create intermediate directories with no ttl. // If the node has already existed, create will fail. // If any node on the path is a file, create will fail. -func (fs *FileSystem) Create(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { +func (s *Store) Create(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { nodePath = path.Clean(path.Join("/", nodePath)) // make sure we can create the node - _, err := fs.InternalGet(nodePath, index, term) + _, err := s.InternalGet(nodePath, index, term) if err == nil { // key already exists - fs.Stats.IncStats(StatsSetsMiss) + s.Stats.Inc(SetFail) return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath) } etcdError, _ := err.(etcdErr.Error) if etcdError.ErrorCode == 104 { // we cannot create the key due to meet a file while walking - fs.Stats.IncStats(StatsSetsMiss) + s.Stats.Inc(SetFail) return nil, err } dir, _ := path.Split(nodePath) // walk through the nodePath, create dirs and get the last directory node - d, err := fs.walk(dir, fs.checkDir) + d, err := s.walk(dir, s.checkDir) if err != nil { - fs.Stats.IncStats(StatsSetsMiss) + s.Stats.Inc(SetFail) return nil, err } - e := newEvent(Create, nodePath, fs.Index, fs.Term) + e := newEvent(Create, nodePath, s.Index, s.Term) var n *Node if len(value) != 0 { // create file e.Value = value - n = newFile(nodePath, value, fs.Index, fs.Term, d, "", expireTime) + n = newFile(nodePath, value, s.Index, s.Term, d, "", expireTime) } else { // create directory e.Dir = true - n = newDir(nodePath, fs.Index, fs.Term, d, "", expireTime) + n = newDir(nodePath, s.Index, s.Term, d, "", expireTime) } err = d.Add(n) if err != nil { - fs.Stats.IncStats(StatsSetsMiss) + s.Stats.Inc(SetFail) return nil, err } @@ -147,28 +148,28 @@ func (fs *FileSystem) Create(nodePath string, value string, expireTime time.Time e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 } - fs.WatcherHub.notify(e) - fs.Stats.IncStats(StatsSetsHit) + s.WatcherHub.notify(e) + s.Stats.Inc(SetSuccess) return e, nil } // Update function updates the value/ttl of the node. // If the node is a file, the value and the ttl can be updated. // If the node is a directory, only the ttl can be updated. -func (fs *FileSystem) Update(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { - n, err := fs.InternalGet(nodePath, index, term) +func (s *Store) Update(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { + n, err := s.InternalGet(nodePath, index, term) if err != nil { // if the node does not exist, return error - fs.Stats.IncStats(StatsUpdatesMiss) + s.Stats.Inc(UpdateFail) return nil, err } - e := newEvent(Update, nodePath, fs.Index, fs.Term) + e := newEvent(Update, nodePath, s.Index, s.Term) if n.IsDir() { // if the node is a directory, we can only update ttl if len(value) != 0 { - fs.Stats.IncStats(StatsUpdatesMiss) + s.Stats.Inc(UpdateFail) return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath) } @@ -194,23 +195,23 @@ func (fs *FileSystem) Update(nodePath string, value string, expireTime time.Time e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 } - fs.WatcherHub.notify(e) - fs.Stats.IncStats(StatsUpdatesHit) + s.WatcherHub.notify(e) + s.Stats.Inc(UpdateSuccess) return e, nil } -func (fs *FileSystem) TestAndSet(nodePath string, prevValue string, prevIndex uint64, +func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { - f, err := fs.InternalGet(nodePath, index, term) + f, err := s.InternalGet(nodePath, index, term) if err != nil { - fs.Stats.IncStats(StatsTestAndSetsMiss) + s.Stats.Inc(TestAndSetFail) return nil, err } if f.IsDir() { // can only test and set file - fs.Stats.IncStats(StatsTestAndSetsMiss) + s.Stats.Inc(TestAndSetFail) return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath) } @@ -221,23 +222,23 @@ func (fs *FileSystem) TestAndSet(nodePath string, prevValue string, prevIndex ui e.Value = value f.Write(value, index, term) - fs.WatcherHub.notify(e) - fs.Stats.IncStats(StatsTestAndSetsHit) + s.WatcherHub.notify(e) + s.Stats.Inc(TestAndSetSuccess) return e, nil } cause := fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, f.Value, prevIndex, f.ModifiedIndex) - fs.Stats.IncStats(StatsTestAndSetsMiss) + s.Stats.Inc(TestAndSetFail) return nil, etcdErr.NewError(etcdErr.EcodeTestFailed, cause) } // Delete function deletes the node at the given path. // If the node is a directory, recursive must be true to delete it. -func (fs *FileSystem) Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error) { - n, err := fs.InternalGet(nodePath, index, term) +func (s *Store) Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error) { + n, err := s.InternalGet(nodePath, index, term) if err != nil { // if the node does not exist, return error - fs.Stats.IncStats(StatsDeletesMiss) + s.Stats.Inc(DeleteFail) return nil, err } @@ -250,37 +251,37 @@ func (fs *FileSystem) Delete(nodePath string, recursive bool, index uint64, term } callback := func(path string) { // notify function - fs.WatcherHub.notifyWithPath(e, path, true) + s.WatcherHub.notifyWithPath(e, path, true) } err = n.Remove(recursive, callback) if err != nil { - fs.Stats.IncStats(StatsDeletesMiss) + s.Stats.Inc(DeleteFail) return nil, err } - fs.WatcherHub.notify(e) - fs.Stats.IncStats(StatsDeletesHit) + s.WatcherHub.notify(e) + s.Stats.Inc(DeleteSuccess) return e, nil } -func (fs *FileSystem) Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error) { - fs.Index, fs.Term = index, term +func (s *Store) Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error) { + s.Index, s.Term = index, term if sinceIndex == 0 { - return fs.WatcherHub.watch(prefix, recursive, index+1) + return s.WatcherHub.watch(prefix, recursive, index+1) } - return fs.WatcherHub.watch(prefix, recursive, sinceIndex) + return s.WatcherHub.watch(prefix, recursive, sinceIndex) } // walk function walks all the nodePath and apply the walkFunc on each directory -func (fs *FileSystem) walk(nodePath string, walkFunc func(prev *Node, component string) (*Node, error)) (*Node, error) { +func (s *Store) walk(nodePath string, walkFunc func(prev *Node, component string) (*Node, error)) (*Node, error) { components := strings.Split(nodePath, "/") - curr := fs.Root + curr := s.Root var err error for i := 1; i < len(components); i++ { @@ -299,11 +300,11 @@ func (fs *FileSystem) walk(nodePath string, walkFunc func(prev *Node, component } // InternalGet function get the node of the given nodePath. -func (fs *FileSystem) InternalGet(nodePath string, index uint64, term uint64) (*Node, error) { +func (s *Store) InternalGet(nodePath string, index uint64, term uint64) (*Node, error) { nodePath = path.Clean(path.Join("/", nodePath)) // update file system known index and term - fs.Index, fs.Term = index, term + s.Index, s.Term = index, term walkFunc := func(parent *Node, name string) (*Node, error) { @@ -319,7 +320,7 @@ func (fs *FileSystem) InternalGet(nodePath string, index uint64, term uint64) (* return nil, etcdErr.NewError(etcdErr.EcodeKeyNotFound, path.Join(parent.Path, name)) } - f, err := fs.walk(nodePath, walkFunc) + f, err := s.walk(nodePath, walkFunc) if err != nil { return nil, err @@ -332,14 +333,14 @@ func (fs *FileSystem) InternalGet(nodePath string, index uint64, term uint64) (* // If it is a directory, this function will return the pointer to that node. // If it does not exist, this function will create a new directory and return the pointer to that node. // If it is a file, this function will return error. -func (fs *FileSystem) checkDir(parent *Node, dirName string) (*Node, error) { +func (s *Store) checkDir(parent *Node, dirName string) (*Node, error) { subDir, ok := parent.Children[dirName] if ok { return subDir, nil } - n := newDir(path.Join(parent.Path, dirName), fs.Index, fs.Term, parent, parent.ACL, Permanent) + n := newDir(path.Join(parent.Path, dirName), s.Index, s.Term, parent, parent.ACL, Permanent) parent.Children[dirName] = n @@ -350,24 +351,20 @@ func (fs *FileSystem) checkDir(parent *Node, dirName string) (*Node, error) { // Save function will not be able to save the state of watchers. // Save function will not save the parent field of the node. Or there will // be cyclic dependencies issue for the json package. -func (fs *FileSystem) Save() ([]byte, error) { - - fs.Stats.IncStats(StatsSaveHit) - cloneFs := New() - cloneFs.Root = fs.Root.Clone() +func (s *Store) Save() ([]byte, error) { + clonedStore := New() + clonedStore.Root = s.Root.Clone() + clonedStore.WatcherHub = s.WatcherHub + clonedStore.Index = s.Index + clonedStore.Term = s.Term + clonedStore.Stats = s.Stats - b, err := json.Marshal(fs) + b, err := json.Marshal(clonedStore) if err != nil { - fs.Stats.IncStats(StatsSaveMiss) - fs.Stats.rwlock.Lock() - fs.Stats.SaveHit-- // restore the savehit - fs.Stats.rwlock.Unlock() - return nil, err } - fs.Stats.IncStats(StatsSaveHit) return b, nil } @@ -375,15 +372,18 @@ func (fs *FileSystem) Save() ([]byte, error) { // It needs to recovery the parent field of the nodes. // It needs to delete the expired nodes since the saved time and also // need to create monitor go routines. -func (fs *FileSystem) Recovery(state []byte) error { - err := json.Unmarshal(state, fs) +func (s *Store) Recovery(state []byte) error { + err := json.Unmarshal(state, s) if err != nil { - fs.Stats.IncStats(StatsRecoveryMiss) return err } - fs.Root.recoverAndclean() - fs.Stats.IncStats(StatsRecoveryHit) + s.Root.recoverAndclean() return nil } + +func (s *Store) JsonStats() []byte { + s.Stats.Watchers = uint64(s.WatcherHub.count) + return s.Stats.toJson() +} diff --git a/file_system/file_system_test.go b/store/store_test.go similarity index 68% rename from file_system/file_system_test.go rename to store/store_test.go index 0478f9c3083..259310ff41a 100644 --- a/file_system/file_system_test.go +++ b/store/store_test.go @@ -1,4 +1,4 @@ -package fileSystem +package store import ( "math/rand" @@ -8,46 +8,46 @@ import ( ) func TestCreateAndGet(t *testing.T) { - fs := New() + s := New() - fs.Create("/foobar", "bar", Permanent, 1, 1) + s.Create("/foobar", "bar", Permanent, 1, 1) // already exist, create should fail - _, err := fs.Create("/foobar", "bar", Permanent, 1, 1) + _, err := s.Create("/foobar", "bar", Permanent, 1, 1) if err == nil { t.Fatal("Create should fail") } - fs.Delete("/foobar", true, 1, 1) + s.Delete("/foobar", true, 1, 1) // this should create successfully - createAndGet(fs, "/foobar", t) - createAndGet(fs, "/foo/bar", t) - createAndGet(fs, "/foo/foo/bar", t) + createAndGet(s, "/foobar", t) + createAndGet(s, "/foo/bar", t) + createAndGet(s, "/foo/foo/bar", t) // meet file, create should fail - _, err = fs.Create("/foo/bar/bar", "bar", Permanent, 2, 1) + _, err = s.Create("/foo/bar/bar", "bar", Permanent, 2, 1) if err == nil { t.Fatal("Create should fail") } // create a directory - _, err = fs.Create("/fooDir", "", Permanent, 3, 1) + _, err = s.Create("/fooDir", "", Permanent, 3, 1) if err != nil { t.Fatal("Cannot create /fooDir") } - e, err := fs.Get("/fooDir", false, false, 3, 1) + e, err := s.Get("/fooDir", false, false, 3, 1) if err != nil || e.Dir != true { t.Fatal("Cannot create /fooDir ") } // create a file under directory - _, err = fs.Create("/fooDir/bar", "bar", Permanent, 4, 1) + _, err = s.Create("/fooDir/bar", "bar", Permanent, 4, 1) if err != nil { t.Fatal("Cannot create /fooDir/bar = bar") @@ -56,21 +56,21 @@ func TestCreateAndGet(t *testing.T) { } func TestUpdateFile(t *testing.T) { - fs := New() + s := New() - _, err := fs.Create("/foo/bar", "bar", Permanent, 1, 1) + _, err := s.Create("/foo/bar", "bar", Permanent, 1, 1) if err != nil { t.Fatalf("cannot create %s=bar [%s]", "/foo/bar", err.Error()) } - _, err = fs.Update("/foo/bar", "barbar", Permanent, 2, 1) + _, err = s.Update("/foo/bar", "barbar", Permanent, 2, 1) if err != nil { t.Fatalf("cannot update %s=barbar [%s]", "/foo/bar", err.Error()) } - e, err := fs.Get("/foo/bar", false, false, 2, 1) + e, err := s.Get("/foo/bar", false, false, 2, 1) if err != nil { t.Fatalf("cannot get %s [%s]", "/foo/bar", err.Error()) @@ -82,37 +82,37 @@ func TestUpdateFile(t *testing.T) { // create a directory, update its ttl, to see if it will be deleted - _, err = fs.Create("/foo/foo", "", Permanent, 3, 1) + _, err = s.Create("/foo/foo", "", Permanent, 3, 1) if err != nil { t.Fatalf("cannot create dir [%s] [%s]", "/foo/foo", err.Error()) } - _, err = fs.Create("/foo/foo/foo1", "bar1", Permanent, 4, 1) + _, err = s.Create("/foo/foo/foo1", "bar1", Permanent, 4, 1) if err != nil { t.Fatal("cannot create [%s]", err.Error()) } - _, err = fs.Create("/foo/foo/foo2", "", Permanent, 5, 1) + _, err = s.Create("/foo/foo/foo2", "", Permanent, 5, 1) if err != nil { t.Fatal("cannot create [%s]", err.Error()) } - _, err = fs.Create("/foo/foo/foo2/boo", "boo1", Permanent, 6, 1) + _, err = s.Create("/foo/foo/foo2/boo", "boo1", Permanent, 6, 1) if err != nil { t.Fatal("cannot create [%s]", err.Error()) } expire := time.Now().Add(time.Second * 2) - _, err = fs.Update("/foo/foo", "", expire, 7, 1) + _, err = s.Update("/foo/foo", "", expire, 7, 1) if err != nil { t.Fatalf("cannot update dir [%s] [%s]", "/foo/foo", err.Error()) } // sleep 50ms, it should still reach the node time.Sleep(time.Microsecond * 50) - e, err = fs.Get("/foo/foo", true, false, 7, 1) + e, err = s.Get("/foo/foo", true, false, 7, 1) if err != nil || e.Key != "/foo/foo" { t.Fatalf("cannot get dir before expiration [%s]", err.Error()) @@ -132,23 +132,23 @@ func TestUpdateFile(t *testing.T) { // wait for expiration time.Sleep(time.Second * 3) - e, err = fs.Get("/foo/foo", true, false, 7, 1) + e, err = s.Get("/foo/foo", true, false, 7, 1) if err == nil { t.Fatal("still can get dir after expiration [%s]") } - _, err = fs.Get("/foo/foo/foo1", true, false, 7, 1) + _, err = s.Get("/foo/foo/foo1", true, false, 7, 1) if err == nil { t.Fatal("still can get sub node after expiration [%s]") } - _, err = fs.Get("/foo/foo/foo2", true, false, 7, 1) + _, err = s.Get("/foo/foo/foo2", true, false, 7, 1) if err == nil { t.Fatal("still can get sub dir after expiration [%s]") } - _, err = fs.Get("/foo/foo/foo2/boo", true, false, 7, 1) + _, err = s.Get("/foo/foo/foo2/boo", true, false, 7, 1) if err == nil { t.Fatalf("still can get sub node of sub dir after expiration [%s]", err.Error()) } @@ -156,17 +156,17 @@ func TestUpdateFile(t *testing.T) { } func TestListDirectory(t *testing.T) { - fs := New() + s := New() // create dir /foo // set key-value /foo/foo=bar - fs.Create("/foo/foo", "bar", Permanent, 1, 1) + s.Create("/foo/foo", "bar", Permanent, 1, 1) // create dir /foo/fooDir // set key-value /foo/fooDir/foo=bar - fs.Create("/foo/fooDir/foo", "bar", Permanent, 2, 1) + s.Create("/foo/fooDir/foo", "bar", Permanent, 2, 1) - e, err := fs.Get("/foo", true, false, 2, 1) + e, err := s.Get("/foo", true, false, 2, 1) if err != nil { t.Fatalf("%v", err) @@ -191,9 +191,9 @@ func TestListDirectory(t *testing.T) { // create dir /foo/_hidden // set key-value /foo/_hidden/foo -> bar - fs.Create("/foo/_hidden/foo", "bar", Permanent, 3, 1) + s.Create("/foo/_hidden/foo", "bar", Permanent, 3, 1) - e, _ = fs.Get("/foo", false, false, 2, 1) + e, _ = s.Get("/foo", false, false, 2, 1) if len(e.KVPairs) != 2 { t.Fatalf("hidden node is not hidden! %s", e.KVPairs[2].Key) @@ -201,38 +201,38 @@ func TestListDirectory(t *testing.T) { } func TestRemove(t *testing.T) { - fs := New() + s := New() - fs.Create("/foo", "bar", Permanent, 1, 1) - _, err := fs.Delete("/foo", false, 1, 1) + s.Create("/foo", "bar", Permanent, 1, 1) + _, err := s.Delete("/foo", false, 1, 1) if err != nil { t.Fatalf("cannot delete %s [%s]", "/foo", err.Error()) } - _, err = fs.Get("/foo", false, false, 1, 1) + _, err = s.Get("/foo", false, false, 1, 1) if err == nil || err.Error() != "Key Not Found" { t.Fatalf("can get the node after deletion") } - fs.Create("/foo/bar", "bar", Permanent, 1, 1) - fs.Create("/foo/car", "car", Permanent, 1, 1) - fs.Create("/foo/dar/dar", "dar", Permanent, 1, 1) + s.Create("/foo/bar", "bar", Permanent, 1, 1) + s.Create("/foo/car", "car", Permanent, 1, 1) + s.Create("/foo/dar/dar", "dar", Permanent, 1, 1) - _, err = fs.Delete("/foo", false, 1, 1) + _, err = s.Delete("/foo", false, 1, 1) if err == nil { t.Fatalf("should not be able to delete a directory without recursive") } - _, err = fs.Delete("/foo", true, 1, 1) + _, err = s.Delete("/foo", true, 1, 1) if err != nil { t.Fatalf("cannot delete %s [%s]", "/foo", err.Error()) } - _, err = fs.Get("/foo", false, false, 1, 1) + _, err = s.Get("/foo", false, false, 1, 1) if err == nil || err.Error() != "Key Not Found" { t.Fatalf("can get the node after deletion ") @@ -241,13 +241,13 @@ func TestRemove(t *testing.T) { } func TestExpire(t *testing.T) { - fs := New() + s := New() expire := time.Now().Add(time.Second) - fs.Create("/foo", "bar", expire, 1, 1) + s.Create("/foo", "bar", expire, 1, 1) - _, err := fs.InternalGet("/foo", 1, 1) + _, err := s.InternalGet("/foo", 1, 1) if err != nil { t.Fatalf("can not get the node") @@ -255,7 +255,7 @@ func TestExpire(t *testing.T) { time.Sleep(time.Second * 2) - _, err = fs.InternalGet("/foo", 1, 1) + _, err = s.InternalGet("/foo", 1, 1) if err == nil { t.Fatalf("can get the node after expiration time") @@ -263,10 +263,10 @@ func TestExpire(t *testing.T) { // test if we can reach the node before expiration expire = time.Now().Add(time.Second) - fs.Create("/foo", "bar", expire, 1, 1) + s.Create("/foo", "bar", expire, 1, 1) time.Sleep(time.Millisecond * 50) - _, err = fs.InternalGet("/foo", 1, 1) + _, err = s.InternalGet("/foo", 1, 1) if err != nil { t.Fatalf("cannot get the node before expiration", err.Error()) @@ -274,8 +274,8 @@ func TestExpire(t *testing.T) { expire = time.Now().Add(time.Second) - fs.Create("/foo", "bar", expire, 1, 1) - _, err = fs.Delete("/foo", false, 1, 1) + s.Create("/foo", "bar", expire, 1, 1) + _, err = s.Delete("/foo", false, 1, 1) if err != nil { t.Fatalf("cannot delete the node before expiration", err.Error()) @@ -284,17 +284,17 @@ func TestExpire(t *testing.T) { } func TestTestAndSet(t *testing.T) { // TODO prevValue == nil ? - fs := New() - fs.Create("/foo", "bar", Permanent, 1, 1) + s := New() + s.Create("/foo", "bar", Permanent, 1, 1) // test on wrong previous value - _, err := fs.TestAndSet("/foo", "barbar", 0, "car", Permanent, 2, 1) + _, err := s.TestAndSet("/foo", "barbar", 0, "car", Permanent, 2, 1) if err == nil { t.Fatal("test and set should fail barbar != bar") } // test on value - e, err := fs.TestAndSet("/foo", "bar", 0, "car", Permanent, 3, 1) + e, err := s.TestAndSet("/foo", "bar", 0, "car", Permanent, 3, 1) if err != nil { t.Fatal("test and set should succeed bar == bar") @@ -305,7 +305,7 @@ func TestTestAndSet(t *testing.T) { // TODO prevValue == nil ? } // test on index - e, err = fs.TestAndSet("/foo", "", 3, "bar", Permanent, 4, 1) + e, err = s.TestAndSet("/foo", "", 3, "bar", Permanent, 4, 1) if err != nil { t.Fatal("test and set should succeed index 3 == 3") @@ -318,61 +318,61 @@ func TestTestAndSet(t *testing.T) { // TODO prevValue == nil ? } func TestWatch(t *testing.T) { - fs := New() + s := New() // watch at a deeper path - c, _ := fs.WatcherHub.watch("/foo/foo/foo", false, 0) - fs.Create("/foo/foo/foo", "bar", Permanent, 1, 1) + c, _ := s.WatcherHub.watch("/foo/foo/foo", false, 0) + s.Create("/foo/foo/foo", "bar", Permanent, 1, 1) e := nonblockingRetrive(c) if e.Key != "/foo/foo/foo" { t.Fatal("watch for Create node fails") } - c, _ = fs.WatcherHub.watch("/foo/foo/foo", false, 0) - fs.Update("/foo/foo/foo", "car", Permanent, 2, 1) + c, _ = s.WatcherHub.watch("/foo/foo/foo", false, 0) + s.Update("/foo/foo/foo", "car", Permanent, 2, 1) e = nonblockingRetrive(c) if e.Key != "/foo/foo/foo" { t.Fatal("watch for Update node fails") } - c, _ = fs.WatcherHub.watch("/foo/foo/foo", false, 0) - fs.TestAndSet("/foo/foo/foo", "car", 0, "bar", Permanent, 3, 1) + c, _ = s.WatcherHub.watch("/foo/foo/foo", false, 0) + s.TestAndSet("/foo/foo/foo", "car", 0, "bar", Permanent, 3, 1) e = nonblockingRetrive(c) if e.Key != "/foo/foo/foo" { t.Fatal("watch for TestAndSet node fails") } - c, _ = fs.WatcherHub.watch("/foo/foo/foo", false, 0) - fs.Delete("/foo", true, 4, 1) //recursively delete + c, _ = s.WatcherHub.watch("/foo/foo/foo", false, 0) + s.Delete("/foo", true, 4, 1) //recursively delete e = nonblockingRetrive(c) if e.Key != "/foo" { t.Fatal("watch for Delete node fails") } // watch at a prefix - c, _ = fs.WatcherHub.watch("/foo", true, 0) - fs.Create("/foo/foo/boo", "bar", Permanent, 5, 1) + c, _ = s.WatcherHub.watch("/foo", true, 0) + s.Create("/foo/foo/boo", "bar", Permanent, 5, 1) e = nonblockingRetrive(c) if e.Key != "/foo/foo/boo" { t.Fatal("watch for Create subdirectory fails") } - c, _ = fs.WatcherHub.watch("/foo", true, 0) - fs.Update("/foo/foo/boo", "foo", Permanent, 6, 1) + c, _ = s.WatcherHub.watch("/foo", true, 0) + s.Update("/foo/foo/boo", "foo", Permanent, 6, 1) e = nonblockingRetrive(c) if e.Key != "/foo/foo/boo" { t.Fatal("watch for Update subdirectory fails") } - c, _ = fs.WatcherHub.watch("/foo", true, 0) - fs.TestAndSet("/foo/foo/boo", "foo", 0, "bar", Permanent, 7, 1) + c, _ = s.WatcherHub.watch("/foo", true, 0) + s.TestAndSet("/foo/foo/boo", "foo", 0, "bar", Permanent, 7, 1) e = nonblockingRetrive(c) if e.Key != "/foo/foo/boo" { t.Fatal("watch for TestAndSet subdirectory fails") } - c, _ = fs.WatcherHub.watch("/foo", true, 0) - fs.Delete("/foo/foo/boo", false, 8, 1) + c, _ = s.WatcherHub.watch("/foo", true, 0) + s.Delete("/foo/foo/boo", false, 8, 1) e = nonblockingRetrive(c) if e.Key != "/foo/foo/boo" { t.Fatal("watch for Delete subdirectory fails") @@ -381,14 +381,14 @@ func TestWatch(t *testing.T) { } func TestSort(t *testing.T) { - fs := New() + s := New() // simulating random creation keys := GenKeys(80, 4) i := uint64(1) for _, k := range keys { - _, err := fs.Create(k, "bar", Permanent, i, 1) + _, err := s.Create(k, "bar", Permanent, i, 1) if err != nil { panic(err) } else { @@ -396,7 +396,7 @@ func TestSort(t *testing.T) { } } - e, err := fs.Get("/foo", true, true, i, 1) + e, err := s.Get("/foo", true, true, i, 1) if err != nil { t.Fatalf("get dir nodes failed [%s]", err.Error()) } @@ -419,14 +419,14 @@ func TestSort(t *testing.T) { } func TestSaveAndRecover(t *testing.T) { - fs := New() + s := New() // simulating random creation keys := GenKeys(8, 4) i := uint64(1) for _, k := range keys { - _, err := fs.Create(k, "bar", Permanent, i, 1) + _, err := s.Create(k, "bar", Permanent, i, 1) if err != nil { panic(err) } else { @@ -438,8 +438,8 @@ func TestSaveAndRecover(t *testing.T) { // test if we can reach the node before expiration expire := time.Now().Add(time.Second) - fs.Create("/foo/foo", "bar", expire, 1, 1) - b, err := fs.Save() + s.Create("/foo/foo", "bar", expire, 1, 1) + b, err := s.Save() cloneFs := New() time.Sleep(time.Second) @@ -453,18 +453,18 @@ func TestSaveAndRecover(t *testing.T) { } } - if fs.WatcherHub.EventHistory.StartIndex != cloneFs.WatcherHub.EventHistory.StartIndex { + if s.WatcherHub.EventHistory.StartIndex != cloneFs.WatcherHub.EventHistory.StartIndex { t.Fatal("Error recovered event history start index") } - for i = 0; int(i) < fs.WatcherHub.EventHistory.Queue.Size; i++ { - if fs.WatcherHub.EventHistory.Queue.Events[i].Key != + for i = 0; int(i) < s.WatcherHub.EventHistory.Queue.Size; i++ { + if s.WatcherHub.EventHistory.Queue.Events[i].Key != cloneFs.WatcherHub.EventHistory.Queue.Events[i].Key { t.Fatal("Error recovered event history") } } - _, err = fs.Get("/foo/foo", false, false, 1, 1) + _, err = s.Get("/foo/foo", false, false, 1, 1) if err == nil || err.Error() != "Key Not Found" { t.Fatalf("can get the node after deletion ") @@ -488,14 +488,14 @@ func GenKeys(num int, depth int) []string { return keys } -func createAndGet(fs *FileSystem, path string, t *testing.T) { - _, err := fs.Create(path, "bar", Permanent, 1, 1) +func createAndGet(s *Store, path string, t *testing.T) { + _, err := s.Create(path, "bar", Permanent, 1, 1) if err != nil { t.Fatalf("cannot create %s=bar [%s]", path, err.Error()) } - e, err := fs.Get(path, false, false, 1, 1) + e, err := s.Get(path, false, false, 1, 1) if err != nil { t.Fatalf("cannot get %s [%s]", path, err.Error()) diff --git a/file_system/watcher.go b/store/watcher.go similarity index 84% rename from file_system/watcher.go rename to store/watcher.go index 30fc89da784..1e946e3c083 100644 --- a/file_system/watcher.go +++ b/store/watcher.go @@ -1,16 +1,16 @@ -package fileSystem +package store import ( "container/list" "path" "strings" + "sync/atomic" ) type watcherHub struct { watchers map[string]*list.List - count uint64 // current number of watchers + count int64 // current number of watchers. EventHistory *EventHistory - Stats *EtcdStats } type watcher struct { @@ -19,11 +19,10 @@ type watcher struct { sinceIndex uint64 } -func newWatchHub(capacity int, stats *EtcdStats) *watcherHub { +func newWatchHub(capacity int) *watcherHub { return &watcherHub{ watchers: make(map[string]*list.List), EventHistory: newEventHistory(capacity), - Stats: stats, } } @@ -37,13 +36,9 @@ func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (<-chan e, err := wh.EventHistory.scan(prefix, index) if err != nil { - wh.Stats.IncStats(StatsWatchMiss) return nil, err } - wh.Stats.IncStats(StatsWatchHit) - wh.Stats.IncStats(StatsInWatchingNum) - if e != nil { eventChan <- e return eventChan, nil @@ -66,6 +61,8 @@ func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (<-chan wh.watchers[prefix] = l } + atomic.AddInt64(&wh.count, 1) + return eventChan, nil } @@ -94,10 +91,8 @@ func (wh *watcherHub) notifyWithPath(e *Event, path string, force bool) { if (w.recursive || force || e.Key == path) && e.Index >= w.sinceIndex { w.eventChan <- e - wh.Stats.rwlock.Lock() // lock the InWatchingNum - wh.Stats.InWatchingNum-- - wh.Stats.rwlock.Unlock() l.Remove(curr) + atomic.AddInt64(&wh.count, -1) } else { notifiedAll = false } diff --git a/file_system/watcher_test.go b/store/watcher_test.go similarity index 92% rename from file_system/watcher_test.go rename to store/watcher_test.go index 132f367f3a9..90c23c59e93 100644 --- a/file_system/watcher_test.go +++ b/store/watcher_test.go @@ -1,12 +1,12 @@ -package fileSystem +package store import ( "testing" ) func TestWatcher(t *testing.T) { - fs := New() - wh := fs.WatcherHub + s := New() + wh := s.WatcherHub c, err := wh.watch("/foo", true, 0) if err != nil { diff --git a/util.go b/util.go index 960d12f4a64..434de1a2478 100644 --- a/util.go +++ b/util.go @@ -15,7 +15,7 @@ import ( "time" etcdErr "github.com/coreos/etcd/error" - "github.com/coreos/etcd/file_system" + "github.com/coreos/etcd/store" "github.com/coreos/etcd/web" "github.com/coreos/go-raft" ) @@ -30,12 +30,12 @@ func durationToExpireTime(strDuration string) (time.Time, error) { duration, err := strconv.Atoi(strDuration) if err != nil { - return fileSystem.Permanent, err + return store.Permanent, err } return time.Now().Add(time.Second * (time.Duration)(duration)), nil } else { - return fileSystem.Permanent, nil + return store.Permanent, nil } } From 784d286f370f38888582ec2bd19776cc77b2378b Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 28 Sep 2013 16:58:57 -0700 Subject: [PATCH 044/247] merge from master --- .gitignore | 3 +- README.md | 11 + build | 2 +- build.ps1 | 24 ++ command.go | 9 +- etcd.go | 7 +- etcd_handlers.go | 30 +- raft_server.go | 57 ++-- raft_stats.go | 90 +++--- scripts/release-version.ps1 | 7 + scripts/test-cluster | 19 ++ store/stats.go | 6 + store/store.go | 43 ++- store/watcher.go | 4 + .../p/goprotobuf/proto/all_test.go | 70 ++++- .../p/goprotobuf/proto/decode.go | 16 +- .../p/goprotobuf/proto/encode.go | 78 ++++-- .../p/goprotobuf/proto/extensions.go | 8 +- .../p/goprotobuf/proto/testdata/test.pb.go | 6 +- .../p/goprotobuf/proto/text_parser.go | 4 +- .../p/goprotobuf/proto/text_test.go | 2 +- .../protoc-gen-go/descriptor/descriptor.pb.go | 4 +- .../protoc-gen-go/generator/generator.go | 21 ++ .../protoc-gen-go/testdata/my_test/test.pb.go | 2 +- .../testdata/my_test/test.pb.go.golden | 2 +- .../protoc-gen-go/testdata/my_test/test.proto | 2 +- third_party/deps | 4 +- .../ccding/go-config-reader/.gitignore | 22 -- .../ccding/go-config-reader/README.md | 2 - .../ccding/go-config-reader/config/config.go | 199 -------------- .../ccding/go-config-reader/example.conf | 10 - .../ccding/go-config-reader/example.go | 32 --- .../github.com/ccding/go-logging/.gitignore | 22 -- .../github.com/ccding/go-logging/LICENSE | 191 ------------- .../github.com/ccding/go-logging/README.md | 217 --------------- .../github.com/ccding/go-logging/example.conf | 3 - .../github.com/ccding/go-logging/example.go | 45 --- .../ccding/go-logging/logging/commands.go | 98 ------- .../ccding/go-logging/logging/fields.go | 236 ---------------- .../ccding/go-logging/logging/fields_test.go | 60 ---- .../ccding/go-logging/logging/formater.go | 62 ----- .../ccding/go-logging/logging/get_go_id.c | 25 -- .../ccding/go-logging/logging/level.go | 68 ----- .../ccding/go-logging/logging/logging.go | 259 ------------------ .../ccding/go-logging/logging/logging_test.go | 71 ----- .../ccding/go-logging/logging/request.go | 24 -- .../ccding/go-logging/logging/writer.go | 130 --------- .../github.com/coreos/go-etcd/README.md | 47 +++- .../github.com/coreos/go-etcd/etcd/client.go | 10 +- .../github.com/coreos/go-etcd/etcd/debug.go | 18 +- .../github.com/coreos/go-etcd/etcd/version.go | 2 +- .../github.com/coreos/go-log/log/commands.go | 214 +++++++++++++++ .../github.com/coreos/go-log/log/fields.go | 69 +++++ .../github.com/coreos/go-log/log/logger.go | 73 +++++ .../github.com/coreos/go-log/log/priority.go | 54 ++++ .../github.com/coreos/go-log/log/sinks.go | 154 +++++++++++ .../coreos/go-raft/append_entries_request.go | 4 +- .../go-raft/append_entries_request_test.go | 6 +- .../coreos/go-raft/append_entries_response.go | 4 +- .../go-raft/append_entries_response_test.go | 6 +- .../coreos/go-raft/http_transporter.go | 16 +- third_party/github.com/coreos/go-raft/log.go | 30 +- third_party/github.com/coreos/go-raft/peer.go | 6 + .../coreos/go-raft/request_vote_request.go | 4 +- .../coreos/go-raft/request_vote_response.go | 4 +- .../github.com/coreos/go-raft/server.go | 18 +- .../github.com/coreos/go-raft/server_test.go | 2 +- .../go-raft/snapshot_recovery_request.go | 4 +- .../go-raft/snapshot_recovery_response.go | 4 +- .../coreos/go-raft/snapshot_request.go | 4 +- .../coreos/go-raft/snapshot_response.go | 4 +- third_party/github.com/coreos/go-raft/test.go | 4 +- .../go-systemd}/LICENSE | 0 .../github.com/coreos/go-systemd/README.md | 3 + .../coreos/go-systemd/activation/files.go | 29 ++ .../coreos/go-systemd/journal/send.go | 148 ++++++++++ transporter.go | 109 ++++---- transporter_test.go | 45 ++- util.go | 33 +-- 79 files changed, 1381 insertions(+), 2054 deletions(-) create mode 100644 build.ps1 create mode 100644 scripts/release-version.ps1 create mode 100755 scripts/test-cluster delete mode 100644 third_party/github.com/ccding/go-config-reader/.gitignore delete mode 100644 third_party/github.com/ccding/go-config-reader/README.md delete mode 100644 third_party/github.com/ccding/go-config-reader/config/config.go delete mode 100644 third_party/github.com/ccding/go-config-reader/example.conf delete mode 100644 third_party/github.com/ccding/go-config-reader/example.go delete mode 100644 third_party/github.com/ccding/go-logging/.gitignore delete mode 100644 third_party/github.com/ccding/go-logging/LICENSE delete mode 100644 third_party/github.com/ccding/go-logging/README.md delete mode 100644 third_party/github.com/ccding/go-logging/example.conf delete mode 100644 third_party/github.com/ccding/go-logging/example.go delete mode 100644 third_party/github.com/ccding/go-logging/logging/commands.go delete mode 100644 third_party/github.com/ccding/go-logging/logging/fields.go delete mode 100644 third_party/github.com/ccding/go-logging/logging/fields_test.go delete mode 100644 third_party/github.com/ccding/go-logging/logging/formater.go delete mode 100644 third_party/github.com/ccding/go-logging/logging/get_go_id.c delete mode 100644 third_party/github.com/ccding/go-logging/logging/level.go delete mode 100644 third_party/github.com/ccding/go-logging/logging/logging.go delete mode 100644 third_party/github.com/ccding/go-logging/logging/logging_test.go delete mode 100644 third_party/github.com/ccding/go-logging/logging/request.go delete mode 100644 third_party/github.com/ccding/go-logging/logging/writer.go create mode 100644 third_party/github.com/coreos/go-log/log/commands.go create mode 100644 third_party/github.com/coreos/go-log/log/fields.go create mode 100644 third_party/github.com/coreos/go-log/log/logger.go create mode 100644 third_party/github.com/coreos/go-log/log/priority.go create mode 100644 third_party/github.com/coreos/go-log/log/sinks.go rename third_party/github.com/{ccding/go-config-reader => coreos/go-systemd}/LICENSE (100%) create mode 100644 third_party/github.com/coreos/go-systemd/README.md create mode 100644 third_party/github.com/coreos/go-systemd/activation/files.go create mode 100644 third_party/github.com/coreos/go-systemd/journal/send.go diff --git a/.gitignore b/.gitignore index c16223f0906..d00d899e2e2 100644 --- a/.gitignore +++ b/.gitignore @@ -1,4 +1,5 @@ src/ pkg/ -./etcd +/etcd release_version.go +/machine* diff --git a/README.md b/README.md index 7dbd55ae79b..7ce98b95c82 100644 --- a/README.md +++ b/README.md @@ -474,6 +474,7 @@ If you are using SSL for server to server communication, you must use it on all **Python libraries** - [transitorykris/etcd-py](https://github.com/transitorykris/etcd-py) +- [jplana/python-etcd](https://github.com/jplana/python-etcd) **Node libraries** @@ -485,13 +486,23 @@ If you are using SSL for server to server communication, you must use it on all - [jpfuentes2/etcd-ruby](https://github.com/jpfuentes2/etcd-ruby) - [ranjib/etcd-ruby](https://github.com/ranjib/etcd-ruby) +**C libraries** + +- [jdarcy/etcd-api](https://github.com/jdarcy/etcd-api) + +**Chef Integration** + +- [coderanger/etcd-chef](https://github.com/coderanger/etcd-chef) + **Chef Cookbook** - [spheromak/etcd-cookbook](https://github.com/spheromak/etcd-cookbook) **Projects using etcd** +- [binocarlos/yoda](https://github.com/binocarlos/yoda) - etcd + ZeroMQ - [calavera/active-proxy](https://github.com/calavera/active-proxy) - HTTP Proxy configured with etcd +- [derekchiang/etcdplus](https://github.com/derekchiang/etcdplus) - A set of distributed synchronization primitives built upon etcd - [gleicon/goreman](https://github.com/gleicon/goreman/tree/etcd) - Branch of the Go Foreman clone with etcd support - [garethr/hiera-etcd](https://github.com/garethr/hiera-etcd) - Puppet hiera backend using etcd - [mattn/etcd-vim](https://github.com/mattn/etcd-vim) - SET and GET keys from inside vim diff --git a/build b/build index a4fe58737f9..b121ba30dff 100755 --- a/build +++ b/build @@ -1,4 +1,4 @@ -#!/bin/bash +#!/bin/sh ETCD_PACKAGE=github.com/coreos/etcd export GOPATH="${PWD}" diff --git a/build.ps1 b/build.ps1 new file mode 100644 index 00000000000..c75f608d7e8 --- /dev/null +++ b/build.ps1 @@ -0,0 +1,24 @@ + +$ETCD_PACKAGE="github.com/coreos/etcd" +$env:GOPATH=$pwd.Path +$SRC_DIR="$env:GOPATH/src" +$ETCD_DIR="$SRC_DIR/$ETCD_PACKAGE" +$env:ETCD_DIR="$SRC_DIR/$ETCD_PACKAGE" + +$ETCD_BASE=(Split-Path $ETCD_DIR -Parent) +if(-not(test-path $ETCD_DIR)){ + mkdir -force "$ETCD_BASE" > $null +} + +if(-not(test-path $ETCD_DIR )){ + cmd /c 'mklink /D "%ETCD_DIR%" ..\..\..\' +} + +foreach($i in (ls third_party/*)){ + if("$i" -eq "third_party/src") {continue} + + cp -Recurse -force "$i" src/ +} + +./scripts/release-version.ps1 | Out-File -Encoding UTF8 release_version.go +go build -v "${ETCD_PACKAGE}" diff --git a/command.go b/command.go index 9ab35934bb8..bfda5f2708c 100644 --- a/command.go +++ b/command.go @@ -223,8 +223,10 @@ func (c *JoinCommand) Apply(raftServer *raft.Server) (interface{}, error) { value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", c.RaftURL, c.EtcdURL, c.RaftVersion) etcdStore.Create(key, value, store.Permanent, raftServer.CommitIndex(), raftServer.Term()) - if c.Name != r.Name() { // do not add self to the peer list - r.peersStats[c.Name] = &raftPeerStats{MinLatency: 1 << 63} + // add peer stats + if c.Name != r.Name() { + r.followersStats.Followers[c.Name] = &raftFollowerStats{} + r.followersStats.Followers[c.Name].Latency.Minimum = 1 << 63 } return b, err @@ -251,7 +253,8 @@ func (c *RemoveCommand) Apply(raftServer *raft.Server) (interface{}, error) { key := path.Join("_etcd/machines", c.Name) _, err := etcdStore.Delete(key, false, raftServer.CommitIndex(), raftServer.Term()) - delete(r.peersStats, c.Name) + // delete from stats + delete(r.followersStats.Followers, c.Name) if err != nil { return []byte{0}, err diff --git a/etcd.go b/etcd.go index 80c8dd12283..49271260fef 100644 --- a/etcd.go +++ b/etcd.go @@ -90,12 +90,7 @@ func init() { const ( ElectionTimeout = 200 * time.Millisecond HeartbeatTimeout = 50 * time.Millisecond - - // Timeout for internal raft http connection - // The original timeout for http is 45 seconds - // which is too long for our usage. - HTTPTimeout = 10 * time.Second - RetryInterval = 10 + RetryInterval = 10 ) //------------------------------------------------------------------------------ diff --git a/etcd_handlers.go b/etcd_handlers.go index 09e5e8252b0..7dc4a0d3e04 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -21,7 +21,7 @@ func NewEtcdMuxer() *http.ServeMux { etcdMux.Handle("/"+version+"/keys/", errorHandler(Multiplexer)) etcdMux.Handle("/"+version+"/leader", errorHandler(LeaderHttpHandler)) etcdMux.Handle("/"+version+"/machines", errorHandler(MachinesHttpHandler)) - etcdMux.Handle("/"+version+"/stats", errorHandler(StatsHttpHandler)) + etcdMux.Handle("/"+version+"/stats/", errorHandler(StatsHttpHandler)) etcdMux.Handle("/version", errorHandler(VersionHttpHandler)) etcdMux.HandleFunc("/test/", TestHttpHandler) return etcdMux @@ -223,8 +223,28 @@ func VersionHttpHandler(w http.ResponseWriter, req *http.Request) error { // Handler to return the basic stats of etcd func StatsHttpHandler(w http.ResponseWriter, req *http.Request) error { w.WriteHeader(http.StatusOK) - w.Write(etcdStore.JsonStats()) - w.Write(r.Stats()) + + option := req.URL.Path[len("/v1/stats/"):] + + switch option { + case "self": + w.Write(r.Stats()) + case "leader": + if r.State() == raft.Leader { + w.Write(r.PeerStats()) + } else { + leader := r.Leader() + // current no leader + if leader == "" { + return etcdErr.NewError(300, "") + } + hostname, _ := nameToEtcdURL(leader) + redirect(hostname, w, req) + } + case "store": + w.Write(etcdStore.JsonStats()) + } + return nil } @@ -236,8 +256,8 @@ func GetHttpHandler(w http.ResponseWriter, req *http.Request) error { if req.FormValue("consistent") == "true" && r.State() != raft.Leader { // help client to redirect the request to the current leader leader := r.Leader() - url, _ := nameToEtcdURL(leader) - redirect(url, w, req) + hostname, _ := nameToEtcdURL(leader) + redirect(hostname, w, req) return nil } diff --git a/raft_server.go b/raft_server.go index 506e4771a08..efbf41acaff 100644 --- a/raft_server.go +++ b/raft_server.go @@ -17,15 +17,15 @@ import ( type raftServer struct { *raft.Server - version string - joinIndex uint64 - name string - url string - listenHost string - tlsConf *TLSConfig - tlsInfo *TLSInfo - peersStats map[string]*raftPeerStats - serverStats *raftServerStats + version string + joinIndex uint64 + name string + url string + listenHost string + tlsConf *TLSConfig + tlsInfo *TLSInfo + followersStats *raftFollowersStats + serverStats *raftServerStats } var r *raftServer @@ -33,10 +33,10 @@ var r *raftServer func newRaftServer(name string, url string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo) *raftServer { // Create transporter for raft - raftTransporter := newTransporter(tlsConf.Scheme, tlsConf.Client, ElectionTimeout) + raftTransporter := newTransporter(tlsConf.Scheme, tlsConf.Client) // Create raft server - server, err := raft.NewServer(name, dirPath, raftTransporter, etcdStore, nil) + server, err := raft.NewServer(name, dirPath, raftTransporter, etcdStore, nil, "") check(err) @@ -48,7 +48,10 @@ func newRaftServer(name string, url string, listenHost string, tlsConf *TLSConfi listenHost: listenHost, tlsConf: tlsConf, tlsInfo: tlsInfo, - peersStats: make(map[string]*raftPeerStats), + followersStats: &raftFollowersStats{ + Leader: name, + Followers: make(map[string]*raftFollowerStats), + }, serverStats: &raftServerStats{ StartTime: time.Now(), sendRateQueue: &statsQueue{ @@ -63,7 +66,6 @@ func newRaftServer(name string, url string, listenHost string, tlsConf *TLSConfi // Start the raft server func (r *raftServer) ListenAndServe() { - // Setup commands. registerCommands() @@ -183,13 +185,16 @@ func (r *raftServer) startTransport(scheme string, tlsConf tls.Config) { // will need to do something more sophisticated later when we allow mixed // version clusters. func getVersion(t *transporter, versionURL url.URL) (string, error) { - resp, err := t.Get(versionURL.String()) + resp, req, err := t.Get(versionURL.String()) if err != nil { return "", err } defer resp.Body.Close() + + t.CancelWhenTimeout(req) + body, err := ioutil.ReadAll(resp.Body) return string(body), nil @@ -244,7 +249,7 @@ func joinByMachine(s *raft.Server, machine string, scheme string) error { debugf("Send Join Request to %s", joinURL.String()) - resp, err := t.Post(joinURL.String(), &b) + resp, req, err := t.Post(joinURL.String(), &b) for { if err != nil { @@ -252,6 +257,9 @@ func joinByMachine(s *raft.Server, machine string, scheme string) error { } if resp != nil { defer resp.Body.Close() + + t.CancelWhenTimeout(req) + if resp.StatusCode == http.StatusOK { b, _ := ioutil.ReadAll(resp.Body) r.joinIndex, _ = binary.Uvarint(b) @@ -264,7 +272,7 @@ func joinByMachine(s *raft.Server, machine string, scheme string) error { json.NewEncoder(&b).Encode(newJoinCommand()) - resp, err = t.Post(address, &b) + resp, req, err = t.Post(address, &b) } else if resp.StatusCode == http.StatusBadRequest { debug("Reach max number machines in the cluster") @@ -282,7 +290,7 @@ func joinByMachine(s *raft.Server, machine string, scheme string) error { } func (r *raftServer) Stats() []byte { - r.serverStats.LeaderUptime = time.Now().Sub(r.serverStats.leaderStartTime).String() + r.serverStats.LeaderInfo.Uptime = time.Now().Sub(r.serverStats.LeaderInfo.startTime).String() queue := r.serverStats.sendRateQueue @@ -292,20 +300,17 @@ func (r *raftServer) Stats() []byte { r.serverStats.RecvingPkgRate, r.serverStats.RecvingBandwidthRate = queue.Rate() - sBytes, err := json.Marshal(r.serverStats) + b, _ := json.Marshal(r.serverStats) - if err != nil { - warn(err) - } + return b +} +func (r *raftServer) PeerStats() []byte { if r.State() == raft.Leader { - pBytes, _ := json.Marshal(r.peersStats) - - b := append(sBytes, pBytes...) + b, _ := json.Marshal(r.followersStats) return b } - - return sBytes + return nil } // Register commands to raft server diff --git a/raft_stats.go b/raft_stats.go index 175a1be55c0..23f97d41774 100644 --- a/raft_stats.go +++ b/raft_stats.go @@ -33,10 +33,14 @@ func (ps *packageStats) Time() time.Time { } type raftServerStats struct { - State string `json:"state"` - StartTime time.Time `json:"startTime"` - Leader string `json:"leader"` - LeaderUptime string `json:"leaderUptime"` + State string `json:"state"` + StartTime time.Time `json:"startTime"` + + LeaderInfo struct { + Name string `json:"leader"` + Uptime string `json:"uptime"` + startTime time.Time + } `json:"leaderInfo"` RecvAppendRequestCnt uint64 `json:"recvAppendRequestCnt,"` RecvingPkgRate float64 `json:"recvPkgRate,omitempty"` @@ -46,16 +50,15 @@ type raftServerStats struct { SendingPkgRate float64 `json:"sendPkgRate,omitempty"` SendingBandwidthRate float64 `json:"sendBandwidthRate,omitempty"` - leaderStartTime time.Time - sendRateQueue *statsQueue - recvRateQueue *statsQueue + sendRateQueue *statsQueue + recvRateQueue *statsQueue } func (ss *raftServerStats) RecvAppendReq(leaderName string, pkgSize int) { ss.State = raft.Follower - if leaderName != ss.Leader { - ss.Leader = leaderName - ss.leaderStartTime = time.Now() + if leaderName != ss.LeaderInfo.Name { + ss.LeaderInfo.Name = leaderName + ss.LeaderInfo.startTime = time.Now() } ss.recvRateQueue.Insert(NewPackageStats(time.Now(), pkgSize)) @@ -64,55 +67,66 @@ func (ss *raftServerStats) RecvAppendReq(leaderName string, pkgSize int) { func (ss *raftServerStats) SendAppendReq(pkgSize int) { now := time.Now() + if ss.State != raft.Leader { ss.State = raft.Leader - ss.Leader = r.Name() - ss.leaderStartTime = now + ss.LeaderInfo.Name = r.Name() + ss.LeaderInfo.startTime = now } - ss.sendRateQueue.Insert(NewPackageStats(time.Now(), pkgSize)) + ss.sendRateQueue.Insert(NewPackageStats(now, pkgSize)) ss.SendAppendRequestCnt++ } -type raftPeerStats struct { - Latency float64 `json:"latency"` - AvgLatency float64 `json:"averageLatency"` - avgLatencySquare float64 - SdvLatency float64 `json:"sdvLatency"` - MinLatency float64 `json:"minLatency"` - MaxLatency float64 `json:"maxLatency"` - FailCnt uint64 `json:"failsCount"` - SuccCnt uint64 `json:"successCount"` +type raftFollowersStats struct { + Leader string `json:"leader"` + Followers map[string]*raftFollowerStats `json:"followers"` +} + +type raftFollowerStats struct { + Latency struct { + Current float64 `json:"current"` + Average float64 `json:"average"` + averageSquare float64 + StandardDeviation float64 `json:"standardDeviation"` + Minimum float64 `json:"minimum"` + Maximum float64 `json:"maximum"` + } `json:"latency"` + + Counts struct { + Fail uint64 `json:"fail"` + Success uint64 `json:"success"` + } `json:"counts"` } -// Succ function update the raftPeerStats with a successful send -func (ps *raftPeerStats) Succ(d time.Duration) { - total := float64(ps.SuccCnt) * ps.AvgLatency - totalSquare := float64(ps.SuccCnt) * ps.avgLatencySquare +// Succ function update the raftFollowerStats with a successful send +func (ps *raftFollowerStats) Succ(d time.Duration) { + total := float64(ps.Counts.Success) * ps.Latency.Average + totalSquare := float64(ps.Counts.Success) * ps.Latency.averageSquare - ps.SuccCnt++ + ps.Counts.Success++ - ps.Latency = float64(d) / (1000000.0) + ps.Latency.Current = float64(d) / (1000000.0) - if ps.Latency > ps.MaxLatency { - ps.MaxLatency = ps.Latency + if ps.Latency.Current > ps.Latency.Maximum { + ps.Latency.Maximum = ps.Latency.Current } - if ps.Latency < ps.MinLatency { - ps.MinLatency = ps.Latency + if ps.Latency.Current < ps.Latency.Minimum { + ps.Latency.Minimum = ps.Latency.Current } - ps.AvgLatency = (total + ps.Latency) / float64(ps.SuccCnt) - ps.avgLatencySquare = (totalSquare + ps.Latency*ps.Latency) / float64(ps.SuccCnt) + ps.Latency.Average = (total + ps.Latency.Current) / float64(ps.Counts.Success) + ps.Latency.averageSquare = (totalSquare + ps.Latency.Current*ps.Latency.Current) / float64(ps.Counts.Success) // sdv = sqrt(avg(x^2) - avg(x)^2) - ps.SdvLatency = math.Sqrt(ps.avgLatencySquare - ps.AvgLatency*ps.AvgLatency) + ps.Latency.StandardDeviation = math.Sqrt(ps.Latency.averageSquare - ps.Latency.Average*ps.Latency.Average) } -// Fail function update the raftPeerStats with a unsuccessful send -func (ps *raftPeerStats) Fail() { - ps.FailCnt++ +// Fail function update the raftFollowerStats with a unsuccessful send +func (ps *raftFollowerStats) Fail() { + ps.Counts.Fail++ } type statsQueue struct { diff --git a/scripts/release-version.ps1 b/scripts/release-version.ps1 new file mode 100644 index 00000000000..54402844021 --- /dev/null +++ b/scripts/release-version.ps1 @@ -0,0 +1,7 @@ + +$VER=(git describe --tags HEAD) + +@" +package main +const releaseVersion = "$VER" +"@ \ No newline at end of file diff --git a/scripts/test-cluster b/scripts/test-cluster new file mode 100755 index 00000000000..ccdedd1b76f --- /dev/null +++ b/scripts/test-cluster @@ -0,0 +1,19 @@ +#!/bin/bash +SESSION=etcd-cluster + +tmux new-session -d -s $SESSION + +# Setup a window for tailing log files +tmux new-window -t $SESSION:1 -n 'machines' +tmux split-window -h +tmux select-pane -t 0 +tmux send-keys "./etcd -s 127.0.0.1:7001 -c 127.0.0.1:4001 -d machine1 -n machine1" C-m + +for i in 2 3; do + tmux select-pane -t 0 + tmux split-window -v + tmux send-keys "./etcd -cors='*' -s 127.0.0.1:700${i} -c 127.0.0.1:400${i} -C 127.0.0.1:7001 -d machine${i} -n machine${i}" C-m +done + +# Attach to session +tmux attach-session -t $SESSION diff --git a/store/stats.go b/store/stats.go index 1f98625a057..b276d5279d9 100644 --- a/store/stats.go +++ b/store/stats.go @@ -48,6 +48,12 @@ func newStats() *Stats { return s } +func (s *Stats) clone() *Stats { + return &Stats{s.GetSuccess, s.GetFail, s.SetSuccess, s.SetFail, + s.DeleteSuccess, s.DeleteFail, s.UpdateSuccess, s.UpdateFail, + s.TestAndSetSuccess, s.TestAndSetFail, s.Watchers} +} + // Status() return the statistics info of etcd storage its recent start func (s *Stats) toJson() []byte { b, _ := json.Marshal(s) diff --git a/store/store.go b/store/store.go index 73c6bf3ec6b..7c002c8be68 100644 --- a/store/store.go +++ b/store/store.go @@ -6,6 +6,7 @@ import ( "path" "sort" "strings" + "sync" "time" etcdErr "github.com/coreos/etcd/error" @@ -17,6 +18,7 @@ type Store struct { Index uint64 Term uint64 Stats *Stats + worldLock sync.RWMutex // stop the world lock. Used to do snapshot } func New() *Store { @@ -26,13 +28,15 @@ func New() *Store { s.WatcherHub = newWatchHub(1000) return s - } func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) { + s.worldLock.RLock() + defer s.worldLock.RUnlock() + nodePath = path.Clean(path.Join("/", nodePath)) - n, err := s.InternalGet(nodePath, index, term) + n, err := s.internalGet(nodePath, index, term) if err != nil { s.Stats.Inc(GetFail) @@ -91,10 +95,13 @@ func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term // If the node has already existed, create will fail. // If any node on the path is a file, create will fail. func (s *Store) Create(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { + s.worldLock.RLock() + defer s.worldLock.RUnlock() + nodePath = path.Clean(path.Join("/", nodePath)) // make sure we can create the node - _, err := s.InternalGet(nodePath, index, term) + _, err := s.internalGet(nodePath, index, term) if err == nil { // key already exists s.Stats.Inc(SetFail) @@ -157,7 +164,10 @@ func (s *Store) Create(nodePath string, value string, expireTime time.Time, inde // If the node is a file, the value and the ttl can be updated. // If the node is a directory, only the ttl can be updated. func (s *Store) Update(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { - n, err := s.InternalGet(nodePath, index, term) + s.worldLock.RLock() + defer s.worldLock.RUnlock() + + n, err := s.internalGet(nodePath, index, term) if err != nil { // if the node does not exist, return error s.Stats.Inc(UpdateFail) @@ -203,7 +213,10 @@ func (s *Store) Update(nodePath string, value string, expireTime time.Time, inde func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { - f, err := s.InternalGet(nodePath, index, term) + s.worldLock.RLock() + defer s.worldLock.RUnlock() + + f, err := s.internalGet(nodePath, index, term) if err != nil { s.Stats.Inc(TestAndSetFail) @@ -235,7 +248,10 @@ func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, // Delete function deletes the node at the given path. // If the node is a directory, recursive must be true to delete it. func (s *Store) Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error) { - n, err := s.InternalGet(nodePath, index, term) + s.worldLock.RLock() + defer s.worldLock.RUnlock() + + n, err := s.internalGet(nodePath, index, term) if err != nil { // if the node does not exist, return error s.Stats.Inc(DeleteFail) @@ -268,6 +284,9 @@ func (s *Store) Delete(nodePath string, recursive bool, index uint64, term uint6 } func (s *Store) Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error) { + s.worldLock.RLock() + defer s.worldLock.RUnlock() + s.Index, s.Term = index, term if sinceIndex == 0 { @@ -300,7 +319,7 @@ func (s *Store) walk(nodePath string, walkFunc func(prev *Node, component string } // InternalGet function get the node of the given nodePath. -func (s *Store) InternalGet(nodePath string, index uint64, term uint64) (*Node, error) { +func (s *Store) internalGet(nodePath string, index uint64, term uint64) (*Node, error) { nodePath = path.Clean(path.Join("/", nodePath)) // update file system known index and term @@ -352,12 +371,16 @@ func (s *Store) checkDir(parent *Node, dirName string) (*Node, error) { // Save function will not save the parent field of the node. Or there will // be cyclic dependencies issue for the json package. func (s *Store) Save() ([]byte, error) { + s.worldLock.Lock() + clonedStore := New() clonedStore.Root = s.Root.Clone() - clonedStore.WatcherHub = s.WatcherHub + clonedStore.WatcherHub = s.WatcherHub.clone() clonedStore.Index = s.Index clonedStore.Term = s.Term - clonedStore.Stats = s.Stats + clonedStore.Stats = s.Stats.clone() + + s.worldLock.Unlock() b, err := json.Marshal(clonedStore) @@ -373,6 +396,8 @@ func (s *Store) Save() ([]byte, error) { // It needs to delete the expired nodes since the saved time and also // need to create monitor go routines. func (s *Store) Recovery(state []byte) error { + s.worldLock.Lock() + defer s.worldLock.Unlock() err := json.Unmarshal(state, s) if err != nil { diff --git a/store/watcher.go b/store/watcher.go index 1e946e3c083..81ae4969a50 100644 --- a/store/watcher.go +++ b/store/watcher.go @@ -116,3 +116,7 @@ func (wh *watcherHub) notify(e *Event) { wh.EventHistory.addEvent(e) } + +func (wh *watcherHub) clone() *watcherHub { + return &watcherHub{} +} diff --git a/third_party/code.google.com/p/goprotobuf/proto/all_test.go b/third_party/code.google.com/p/goprotobuf/proto/all_test.go index 9d5115854f6..eec58410407 100644 --- a/third_party/code.google.com/p/goprotobuf/proto/all_test.go +++ b/third_party/code.google.com/p/goprotobuf/proto/all_test.go @@ -431,7 +431,7 @@ func TestRequiredBit(t *testing.T) { err := o.Marshal(pb) if err == nil { t.Error("did not catch missing required fields") - } else if strings.Index(err.Error(), "GoTest") < 0 { + } else if strings.Index(err.Error(), "Kind") < 0 { t.Error("wrong error type:", err) } } @@ -1205,7 +1205,7 @@ func TestRequiredFieldEnforcement(t *testing.T) { _, err := Marshal(pb) if err == nil { t.Error("marshal: expected error, got nil") - } else if strings.Index(err.Error(), "GoTestField") < 0 { + } else if strings.Index(err.Error(), "Label") < 0 { t.Errorf("marshal: bad error type: %v", err) } @@ -1216,7 +1216,7 @@ func TestRequiredFieldEnforcement(t *testing.T) { err = Unmarshal(buf, pb) if err == nil { t.Error("unmarshal: expected error, got nil") - } else if strings.Index(err.Error(), "GoTestField") < 0 { + } else if strings.Index(err.Error(), "{Unknown}") < 0 { t.Errorf("unmarshal: bad error type: %v", err) } } @@ -1670,6 +1670,70 @@ func TestEncodingSizes(t *testing.T) { } } +func TestErrRequiredNotSet(t *testing.T) { + pb := initGoTest(false) + pb.RequiredField.Label = nil + pb.F_Int32Required = nil + pb.F_Int64Required = nil + + expected := "0807" + // field 1, encoding 0, value 7 + "2206" + "120474797065" + // field 4, encoding 2 (GoTestField) + "5001" + // field 10, encoding 0, value 1 + "6d20000000" + // field 13, encoding 5, value 0x20 + "714000000000000000" + // field 14, encoding 1, value 0x40 + "78a019" + // field 15, encoding 0, value 0xca0 = 3232 + "8001c032" + // field 16, encoding 0, value 0x1940 = 6464 + "8d0100004a45" + // field 17, encoding 5, value 3232.0 + "9101000000000040b940" + // field 18, encoding 1, value 6464.0 + "9a0106" + "737472696e67" + // field 19, encoding 2, string "string" + "b304" + // field 70, encoding 3, start group + "ba0408" + "7265717569726564" + // field 71, encoding 2, string "required" + "b404" + // field 70, encoding 4, end group + "aa0605" + "6279746573" + // field 101, encoding 2, string "bytes" + "b0063f" + // field 102, encoding 0, 0x3f zigzag32 + "b8067f" // field 103, encoding 0, 0x7f zigzag64 + + o := old() + bytes, err := Marshal(pb) + if _, ok := err.(*ErrRequiredNotSet); !ok { + fmt.Printf("marshal-1 err = %v, want *ErrRequiredNotSet", err) + o.DebugPrint("", bytes) + t.Fatalf("expected = %s", expected) + } + if strings.Index(err.Error(), "RequiredField.Label") < 0 { + t.Errorf("marshal-1 wrong err msg: %v", err) + } + if !equal(bytes, expected, t) { + o.DebugPrint("neq 1", bytes) + t.Fatalf("expected = %s", expected) + } + + // Now test Unmarshal by recreating the original buffer. + pbd := new(GoTest) + err = Unmarshal(bytes, pbd) + if _, ok := err.(*ErrRequiredNotSet); !ok { + t.Fatalf("unmarshal err = %v, want *ErrRequiredNotSet", err) + o.DebugPrint("", bytes) + t.Fatalf("string = %s", expected) + } + if strings.Index(err.Error(), "RequiredField.{Unknown}") < 0 { + t.Errorf("unmarshal wrong err msg: %v", err) + } + bytes, err = Marshal(pbd) + if _, ok := err.(*ErrRequiredNotSet); !ok { + t.Errorf("marshal-2 err = %v, want *ErrRequiredNotSet", err) + o.DebugPrint("", bytes) + t.Fatalf("string = %s", expected) + } + if strings.Index(err.Error(), "RequiredField.Label") < 0 { + t.Errorf("marshal-2 wrong err msg: %v", err) + } + if !equal(bytes, expected, t) { + o.DebugPrint("neq 2", bytes) + t.Fatalf("string = %s", expected) + } +} + func fuzzUnmarshal(t *testing.T, data []byte) { defer func() { if e := recover(); e != nil { diff --git a/third_party/code.google.com/p/goprotobuf/proto/decode.go b/third_party/code.google.com/p/goprotobuf/proto/decode.go index f951c01a713..3c58cfb9b2a 100644 --- a/third_party/code.google.com/p/goprotobuf/proto/decode.go +++ b/third_party/code.google.com/p/goprotobuf/proto/decode.go @@ -46,7 +46,7 @@ import ( // ErrWrongType occurs when the wire encoding for the field disagrees with // that specified in the type being decoded. This is usually caused by attempting // to convert an encoded protocol buffer into a struct of the wrong type. -var ErrWrongType = errors.New("field/encoding mismatch: wrong type for field") +var ErrWrongType = errors.New("proto: field/encoding mismatch: wrong type for field") // errOverflow is returned when an integer is too large to be represented. var errOverflow = errors.New("proto: integer overflow") @@ -353,6 +353,7 @@ func (p *Buffer) Unmarshal(pb Message) error { // unmarshalType does the work of unmarshaling a structure. func (o *Buffer) unmarshalType(st reflect.Type, prop *StructProperties, is_group bool, base structPointer) error { + var state errorState required, reqFields := prop.reqCount, uint64(0) var err error @@ -406,7 +407,10 @@ func (o *Buffer) unmarshalType(st reflect.Type, prop *StructProperties, is_group continue } } - err = dec(o, p, base) + decErr := dec(o, p, base) + if decErr != nil && !state.shouldContinue(decErr, p) { + err = decErr + } if err == nil && p.Required { // Successfully decoded a required field. if tag <= 64 { @@ -430,8 +434,14 @@ func (o *Buffer) unmarshalType(st reflect.Type, prop *StructProperties, is_group if is_group { return io.ErrUnexpectedEOF } + if state.err != nil { + return state.err + } if required > 0 { - return &ErrRequiredNotSet{st} + // Not enough information to determine the exact field. If we use extra + // CPU, we could determine the field only if the missing required field + // has a tag <= 64 and we check reqFields. + return &ErrRequiredNotSet{"{Unknown}"} } } return err diff --git a/third_party/code.google.com/p/goprotobuf/proto/encode.go b/third_party/code.google.com/p/goprotobuf/proto/encode.go index 9d592cd5e1e..d49ab84d2f4 100644 --- a/third_party/code.google.com/p/goprotobuf/proto/encode.go +++ b/third_party/code.google.com/p/goprotobuf/proto/encode.go @@ -37,6 +37,7 @@ package proto import ( "errors" + "fmt" "reflect" "sort" ) @@ -46,12 +47,16 @@ import ( // all been initialized. It is also the error returned if Unmarshal is // called with an encoded protocol buffer that does not include all the // required fields. +// +// When printed, ErrRequiredNotSet reports the first unset required field in a +// message. If the field cannot be precisely determined, it is reported as +// "{Unknown}". type ErrRequiredNotSet struct { - t reflect.Type + field string } func (e *ErrRequiredNotSet) Error() string { - return "proto: required fields not set in " + e.t.String() + return fmt.Sprintf("proto: required field %q not set", e.field) } var ( @@ -175,7 +180,8 @@ func Marshal(pb Message) ([]byte, error) { } p := NewBuffer(nil) err := p.Marshal(pb) - if err != nil { + var state errorState + if err != nil && !state.shouldContinue(err, nil) { return nil, err } return p.buf, err @@ -274,6 +280,7 @@ func isNil(v reflect.Value) bool { // Encode a message struct. func (o *Buffer) enc_struct_message(p *Properties, base structPointer) error { + var state errorState structp := structPointer_GetStructPointer(base, p.field) if structPointer_IsNil(structp) { return ErrNil @@ -283,7 +290,7 @@ func (o *Buffer) enc_struct_message(p *Properties, base structPointer) error { if p.isMarshaler { m := structPointer_Interface(structp, p.stype).(Marshaler) data, err := m.Marshal() - if err != nil { + if err != nil && !state.shouldContinue(err, nil) { return err } o.buf = append(o.buf, p.tagcode...) @@ -300,18 +307,19 @@ func (o *Buffer) enc_struct_message(p *Properties, base structPointer) error { nbuf := o.buf o.buf = obuf - if err != nil { + if err != nil && !state.shouldContinue(err, nil) { o.buffree(nbuf) return err } o.buf = append(o.buf, p.tagcode...) o.EncodeRawBytes(nbuf) o.buffree(nbuf) - return nil + return state.err } // Encode a group struct. func (o *Buffer) enc_struct_group(p *Properties, base structPointer) error { + var state errorState b := structPointer_GetStructPointer(base, p.field) if structPointer_IsNil(b) { return ErrNil @@ -319,11 +327,11 @@ func (o *Buffer) enc_struct_group(p *Properties, base structPointer) error { o.EncodeVarint(uint64((p.Tag << 3) | WireStartGroup)) err := o.enc_struct(p.stype, p.sprop, b) - if err != nil { + if err != nil && !state.shouldContinue(err, nil) { return err } o.EncodeVarint(uint64((p.Tag << 3) | WireEndGroup)) - return nil + return state.err } // Encode a slice of bools ([]bool). @@ -470,6 +478,7 @@ func (o *Buffer) enc_slice_string(p *Properties, base structPointer) error { // Encode a slice of message structs ([]*struct). func (o *Buffer) enc_slice_struct_message(p *Properties, base structPointer) error { + var state errorState s := structPointer_StructPointerSlice(base, p.field) l := s.Len() @@ -483,7 +492,7 @@ func (o *Buffer) enc_slice_struct_message(p *Properties, base structPointer) err if p.isMarshaler { m := structPointer_Interface(structp, p.stype).(Marshaler) data, err := m.Marshal() - if err != nil { + if err != nil && !state.shouldContinue(err, nil) { return err } o.buf = append(o.buf, p.tagcode...) @@ -498,7 +507,7 @@ func (o *Buffer) enc_slice_struct_message(p *Properties, base structPointer) err nbuf := o.buf o.buf = obuf - if err != nil { + if err != nil && !state.shouldContinue(err, nil) { o.buffree(nbuf) if err == ErrNil { return ErrRepeatedHasNil @@ -510,11 +519,12 @@ func (o *Buffer) enc_slice_struct_message(p *Properties, base structPointer) err o.buffree(nbuf) } - return nil + return state.err } // Encode a slice of group structs ([]*struct). func (o *Buffer) enc_slice_struct_group(p *Properties, base structPointer) error { + var state errorState s := structPointer_StructPointerSlice(base, p.field) l := s.Len() @@ -528,7 +538,7 @@ func (o *Buffer) enc_slice_struct_group(p *Properties, base structPointer) error err := o.enc_struct(p.stype, p.sprop, b) - if err != nil { + if err != nil && !state.shouldContinue(err, nil) { if err == ErrNil { return ErrRepeatedHasNil } @@ -537,7 +547,7 @@ func (o *Buffer) enc_slice_struct_group(p *Properties, base structPointer) error o.EncodeVarint(uint64((p.Tag << 3) | WireEndGroup)) } - return nil + return state.err } // Encode an extension map. @@ -569,7 +579,7 @@ func (o *Buffer) enc_map(p *Properties, base structPointer) error { // Encode a struct. func (o *Buffer) enc_struct(t reflect.Type, prop *StructProperties, base structPointer) error { - required := prop.reqCount + var state errorState // Encode fields in tag order so that decoders may use optimizations // that depend on the ordering. // http://code.google.com/apis/protocolbuffers/docs/encoding.html#order @@ -577,19 +587,15 @@ func (o *Buffer) enc_struct(t reflect.Type, prop *StructProperties, base structP p := prop.Prop[i] if p.enc != nil { err := p.enc(o, p, base) - if err != nil { + if err != nil && !state.shouldContinue(err, p) { if err != ErrNil { return err + } else if p.Required && state.err == nil { + state.err = &ErrRequiredNotSet{p.Name} } - } else if p.Required { - required-- } } } - // See if we encoded all required fields. - if required > 0 { - return &ErrRequiredNotSet{t} - } // Add unrecognized fields at the end. if prop.unrecField.IsValid() { @@ -599,5 +605,33 @@ func (o *Buffer) enc_struct(t reflect.Type, prop *StructProperties, base structP } } - return nil + return state.err +} + +// errorState maintains the first error that occurs and updates that error +// with additional context. +type errorState struct { + err error +} + +// shouldContinue reports whether encoding should continue upon encountering the +// given error. If the error is ErrRequiredNotSet, shouldContinue returns true +// and, if this is the first appearance of that error, remembers it for future +// reporting. +// +// If prop is not nil, it may update any error with additional context about the +// field with the error. +func (s *errorState) shouldContinue(err error, prop *Properties) bool { + // Ignore unset required fields. + reqNotSet, ok := err.(*ErrRequiredNotSet) + if !ok { + return false + } + if s.err == nil { + if prop != nil { + err = &ErrRequiredNotSet{prop.Name + "." + reqNotSet.field} + } + s.err = err + } + return true } diff --git a/third_party/code.google.com/p/goprotobuf/proto/extensions.go b/third_party/code.google.com/p/goprotobuf/proto/extensions.go index 1e1e4dc7d97..e730b68ddf2 100644 --- a/third_party/code.google.com/p/goprotobuf/proto/extensions.go +++ b/third_party/code.google.com/p/goprotobuf/proto/extensions.go @@ -109,11 +109,11 @@ func isExtensionField(pb extendableProto, field int32) bool { func checkExtensionTypes(pb extendableProto, extension *ExtensionDesc) error { // Check the extended type. if a, b := reflect.TypeOf(pb), reflect.TypeOf(extension.ExtendedType); a != b { - return errors.New("bad extended type; " + b.String() + " does not extend " + a.String()) + return errors.New("proto: bad extended type; " + b.String() + " does not extend " + a.String()) } // Check the range. if !isExtensionField(pb, extension.Field) { - return errors.New("bad extension number; not in declared ranges") + return errors.New("proto: bad extension number; not in declared ranges") } return nil } @@ -272,7 +272,7 @@ func decodeExtension(b []byte, extension *ExtensionDesc) (interface{}, error) { func GetExtensions(pb Message, es []*ExtensionDesc) (extensions []interface{}, err error) { epb, ok := pb.(extendableProto) if !ok { - err = errors.New("not an extendable proto") + err = errors.New("proto: not an extendable proto") return } extensions = make([]interface{}, len(es)) @@ -292,7 +292,7 @@ func SetExtension(pb extendableProto, extension *ExtensionDesc, value interface{ } typ := reflect.TypeOf(extension.ExtensionType) if typ != reflect.TypeOf(value) { - return errors.New("bad extension value type") + return errors.New("proto: bad extension value type") } pb.ExtensionMap()[extension.Field] = Extension{desc: extension, value: value} diff --git a/third_party/code.google.com/p/goprotobuf/proto/testdata/test.pb.go b/third_party/code.google.com/p/goprotobuf/proto/testdata/test.pb.go index d5e2d5eda07..b8e40cfc7a0 100644 --- a/third_party/code.google.com/p/goprotobuf/proto/testdata/test.pb.go +++ b/third_party/code.google.com/p/goprotobuf/proto/testdata/test.pb.go @@ -244,7 +244,7 @@ func (m *GoEnum) GetFoo() FOO { if m != nil && m.Foo != nil { return *m.Foo } - return 0 + return FOO_FOO1 } type GoTestField struct { @@ -378,7 +378,7 @@ func (m *GoTest) GetKind() GoTest_KIND { if m != nil && m.Kind != nil { return *m.Kind } - return 0 + return GoTest_VOID } func (m *GoTest) GetTable() string { @@ -1289,7 +1289,7 @@ func (m *MyMessage) GetBikeshed() MyMessage_Color { if m != nil && m.Bikeshed != nil { return *m.Bikeshed } - return 0 + return MyMessage_RED } func (m *MyMessage) GetSomegroup() *MyMessage_SomeGroup { diff --git a/third_party/code.google.com/p/goprotobuf/proto/text_parser.go b/third_party/code.google.com/p/goprotobuf/proto/text_parser.go index f39b90885e6..13827f636e5 100644 --- a/third_party/code.google.com/p/goprotobuf/proto/text_parser.go +++ b/third_party/code.google.com/p/goprotobuf/proto/text_parser.go @@ -193,8 +193,8 @@ func (p *textParser) advance() { } var ( - errBadUTF8 = errors.New("bad UTF-8") - errBadHex = errors.New("bad hexadecimal") + errBadUTF8 = errors.New("proto: bad UTF-8") + errBadHex = errors.New("proto: bad hexadecimal") ) func unquoteC(s string, quote rune) (string, error) { diff --git a/third_party/code.google.com/p/goprotobuf/proto/text_test.go b/third_party/code.google.com/p/goprotobuf/proto/text_test.go index f5d0574629f..c64b073c763 100644 --- a/third_party/code.google.com/p/goprotobuf/proto/text_test.go +++ b/third_party/code.google.com/p/goprotobuf/proto/text_test.go @@ -303,7 +303,7 @@ type limitedWriter struct { limit int } -var outOfSpace = errors.New("insufficient space") +var outOfSpace = errors.New("proto: insufficient space") func (w *limitedWriter) Write(p []byte) (n int, err error) { var avail = w.limit - w.b.Len() diff --git a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/descriptor/descriptor.pb.go b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/descriptor/descriptor.pb.go index 3acbe29915c..0b34acb62db 100644 --- a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/descriptor/descriptor.pb.go +++ b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/descriptor/descriptor.pb.go @@ -487,14 +487,14 @@ func (m *FieldDescriptorProto) GetLabel() FieldDescriptorProto_Label { if m != nil && m.Label != nil { return *m.Label } - return 0 + return FieldDescriptorProto_LABEL_OPTIONAL } func (m *FieldDescriptorProto) GetType() FieldDescriptorProto_Type { if m != nil && m.Type != nil { return *m.Type } - return 0 + return FieldDescriptorProto_TYPE_DOUBLE } func (m *FieldDescriptorProto) GetTypeName() string { diff --git a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/generator/generator.go b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/generator/generator.go index 413f3614eb8..0b769d454b9 100644 --- a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/generator/generator.go +++ b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/generator/generator.go @@ -1664,6 +1664,27 @@ func (g *Generator) generateMessage(message *Descriptor) { g.P("return false") case descriptor.FieldDescriptorProto_TYPE_STRING: g.P(`return ""`) + case descriptor.FieldDescriptorProto_TYPE_ENUM: + // The default default for an enum is the first value in the enum, + // not zero. + obj := g.ObjectNamed(field.GetTypeName()) + var enum *EnumDescriptor + if id, ok := obj.(*ImportedDescriptor); ok { + // The enum type has been publicly imported. + enum, _ = id.o.(*EnumDescriptor) + } else { + enum, _ = obj.(*EnumDescriptor) + } + if enum == nil { + log.Printf("don't know how to generate getter for %s", field.GetName()) + continue + } + if len(enum.Value) == 0 { + g.P("return 0 // empty enum") + } else { + first := enum.Value[0].GetName() + g.P("return ", g.DefaultPackageName(obj)+enum.prefix()+first) + } default: g.P("return 0") } diff --git a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.pb.go b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.pb.go index a9d538d20cd..cfe977758b7 100644 --- a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.pb.go +++ b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.pb.go @@ -199,7 +199,7 @@ func (m *Request) GetHue() Request_Color { if m != nil && m.Hue != nil { return *m.Hue } - return 0 + return Request_RED } func (m *Request) GetHat() HatType { diff --git a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.pb.go.golden b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.pb.go.golden index a9d538d20cd..cfe977758b7 100644 --- a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.pb.go.golden +++ b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.pb.go.golden @@ -199,7 +199,7 @@ func (m *Request) GetHue() Request_Color { if m != nil && m.Hue != nil { return *m.Hue } - return 0 + return Request_RED } func (m *Request) GetHat() HatType { diff --git a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.proto b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.proto index 478e697c2be..551585d0cf7 100644 --- a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.proto +++ b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.proto @@ -58,7 +58,7 @@ message Request { } repeated int64 key = 1; // optional imp.ImportedMessage imported_message = 2; - optional Color hue = 3; + optional Color hue = 3; // no default optional HatType hat = 4 [default=FEDORA]; // optional imp.ImportedMessage.Owner owner = 6; optional float deadline = 7 [default=inf]; diff --git a/third_party/deps b/third_party/deps index 2b4ae07fa74..576081e484a 100755 --- a/third_party/deps +++ b/third_party/deps @@ -1,8 +1,8 @@ packages=" github.com/coreos/go-raft github.com/coreos/go-etcd - github.com/ccding/go-logging - github.com/ccding/go-config-reader + github.com/coreos/go-log/log + github.com/coreos/go-systemd bitbucket.org/kardianos/osext code.google.com/p/go.net code.google.com/p/goprotobuf diff --git a/third_party/github.com/ccding/go-config-reader/.gitignore b/third_party/github.com/ccding/go-config-reader/.gitignore deleted file mode 100644 index 00268614f04..00000000000 --- a/third_party/github.com/ccding/go-config-reader/.gitignore +++ /dev/null @@ -1,22 +0,0 @@ -# Compiled Object files, Static and Dynamic libs (Shared Objects) -*.o -*.a -*.so - -# Folders -_obj -_test - -# Architecture specific extensions/prefixes -*.[568vq] -[568vq].out - -*.cgo1.go -*.cgo2.c -_cgo_defun.c -_cgo_gotypes.go -_cgo_export.* - -_testmain.go - -*.exe diff --git a/third_party/github.com/ccding/go-config-reader/README.md b/third_party/github.com/ccding/go-config-reader/README.md deleted file mode 100644 index 846242cb20d..00000000000 --- a/third_party/github.com/ccding/go-config-reader/README.md +++ /dev/null @@ -1,2 +0,0 @@ -go-config-reader -================ diff --git a/third_party/github.com/ccding/go-config-reader/config/config.go b/third_party/github.com/ccding/go-config-reader/config/config.go deleted file mode 100644 index 36ca20dec43..00000000000 --- a/third_party/github.com/ccding/go-config-reader/config/config.go +++ /dev/null @@ -1,199 +0,0 @@ -// Copyright 2013, Cong Ding. All rights reserved. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -// -// author: Cong Ding - -package config - -import ( - "bufio" - "errors" - "fmt" - "io/ioutil" - "os" - "strings" -) - -var commentPrefix = []string{"//", "#", ";"} - -// Config struct constructs a new configuration handler. -type Config struct { - filename string - config map[string]map[string]string -} - -// NewConfig function cnstructs a new Config struct with filename. You have to -// call Read() function to let it read from the file. Otherwise you will get -// empty string (i.e., "") when you are calling Get() function. Another usage -// is that you call NewConfig() function and then call Add()/Set() function to -// add new key-values to the configuration. Finally you can call Write() -// function to write the new configuration to the file. -func NewConfig(filename string) *Config { - c := new(Config) - c.filename = filename - c.config = make(map[string]map[string]string) - return c -} - -// Filename function returns the filename of the configuration. -func (c *Config) Filename() string { - return c.filename -} - -// SetFilename function sets the filename of the configuration. -func (c *Config) SetFilename(filename string) { - c.filename = filename -} - -// Reset function reset the map in the configuration. -func (c *Config) Reset() { - c.config = make(map[string]map[string]string) -} - -// Read function reads configurations from the file defined in -// Config.filename. -func (c *Config) Read() error { - in, err := os.Open(c.filename) - if err != nil { - return err - } - defer in.Close() - scanner := bufio.NewScanner(in) - line := "" - section := "" - for scanner.Scan() { - if scanner.Text() == "" { - continue - } - if line == "" { - sec, ok := checkSection(scanner.Text()) - if ok { - section = sec - continue - } - } - if checkComment(scanner.Text()) { - continue - } - line += scanner.Text() - if strings.HasSuffix(line, "\\") { - line = line[:len(line)-1] - continue - } - key, value, ok := checkLine(line) - if !ok { - return errors.New("WRONG: " + line) - } - c.Set(section, key, value) - line = "" - } - return nil -} - -// Get function returns the value of a key in the configuration. If the key -// does not exist, it returns empty string (i.e., ""). -func (c *Config) Get(section string, key string) string { - value, ok := c.config[section][key] - if !ok { - return "" - } - return value -} - -// Set function updates the value of a key in the configuration. Function -// Set() is exactly the same as function Add(). -func (c *Config) Set(section string, key string, value string) { - _, ok := c.config[section] - if !ok { - c.config[section] = make(map[string]string) - } - c.config[section][key] = value -} - -// Add function adds a new key to the configuration. Function Add() is exactly -// the same as function Set(). -func (c *Config) Add(section string, key string, value string) { - c.Set(section, key, value) -} - -// Del function deletes a key from the configuration. -func (c *Config) Del(section string, key string) { - _, ok := c.config[section] - if ok { - delete(c.config[section], key) - if len(c.config[section]) == 0 { - delete(c.config, section) - } - } -} - -// Write function writes the updated configuration back. -func (c *Config) Write() error { - return nil -} - -// WriteTo function writes the configuration to a new file. This function -// re-organizes the configuration and deletes all the comments. -func (c *Config) WriteTo(filename string) error { - content := "" - for k, v := range c.config { - format := "%v = %v\n" - if k != "" { - content += fmt.Sprintf("[%v]\n", k) - format = "\t" + format - } - for key, value := range v { - content += fmt.Sprintf(format, key, value) - } - } - return ioutil.WriteFile(filename, []byte(content), 0644) -} - -// To check this line is a section or not. If it is not a section, it returns -// "". -func checkSection(line string) (string, bool) { - line = strings.TrimSpace(line) - lineLen := len(line) - if lineLen < 2 { - return "", false - } - if line[0] == '[' && line[lineLen-1] == ']' { - return line[1 : lineLen-1], true - } - return "", false -} - -// To check this line is a valid key-value pair or not. -func checkLine(line string) (string, string, bool) { - key := "" - value := "" - sp := strings.SplitN(line, "=", 2) - if len(sp) != 2 { - return key, value, false - } - key = strings.TrimSpace(sp[0]) - value = strings.TrimSpace(sp[1]) - return key, value, true -} - -// To check this line is a whole line comment or not. -func checkComment(line string) bool { - line = strings.TrimSpace(line) - for p := range commentPrefix { - if strings.HasPrefix(line, commentPrefix[p]) { - return true - } - } - return false -} diff --git a/third_party/github.com/ccding/go-config-reader/example.conf b/third_party/github.com/ccding/go-config-reader/example.conf deleted file mode 100644 index 706e7b1f110..00000000000 --- a/third_party/github.com/ccding/go-config-reader/example.conf +++ /dev/null @@ -1,10 +0,0 @@ -a = b -1 = 2 -cc = dd, 2 ejkl ijfadjfl -// adkfa -# 12jfiahdoif -dd = c \ - oadi - - [test] - a = c c d diff --git a/third_party/github.com/ccding/go-config-reader/example.go b/third_party/github.com/ccding/go-config-reader/example.go deleted file mode 100644 index 17587442b36..00000000000 --- a/third_party/github.com/ccding/go-config-reader/example.go +++ /dev/null @@ -1,32 +0,0 @@ -// Copyright 2013, Cong Ding. All rights reserved. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -// -// author: Cong Ding - -package main - -import ( - "fmt" - "github.com/ccding/go-config-reader/config" -) - -func main() { - c := config.NewConfig("example.conf") - err := c.Read() - fmt.Println(err) - fmt.Println(c) - fmt.Println(c.Get("test", "a")) - fmt.Println(c.Get("", "dd")) - c.WriteTo("example2.conf") -} diff --git a/third_party/github.com/ccding/go-logging/.gitignore b/third_party/github.com/ccding/go-logging/.gitignore deleted file mode 100644 index 00268614f04..00000000000 --- a/third_party/github.com/ccding/go-logging/.gitignore +++ /dev/null @@ -1,22 +0,0 @@ -# Compiled Object files, Static and Dynamic libs (Shared Objects) -*.o -*.a -*.so - -# Folders -_obj -_test - -# Architecture specific extensions/prefixes -*.[568vq] -[568vq].out - -*.cgo1.go -*.cgo2.c -_cgo_defun.c -_cgo_gotypes.go -_cgo_export.* - -_testmain.go - -*.exe diff --git a/third_party/github.com/ccding/go-logging/LICENSE b/third_party/github.com/ccding/go-logging/LICENSE deleted file mode 100644 index 37ec93a14fd..00000000000 --- a/third_party/github.com/ccding/go-logging/LICENSE +++ /dev/null @@ -1,191 +0,0 @@ -Apache License -Version 2.0, January 2004 -http://www.apache.org/licenses/ - -TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - -1. Definitions. - -"License" shall mean the terms and conditions for use, reproduction, and -distribution as defined by Sections 1 through 9 of this document. - -"Licensor" shall mean the copyright owner or entity authorized by the copyright -owner that is granting the License. - -"Legal Entity" shall mean the union of the acting entity and all other entities -that control, are controlled by, or are under common control with that entity. -For the purposes of this definition, "control" means (i) the power, direct or -indirect, to cause the direction or management of such entity, whether by -contract or otherwise, or (ii) ownership of fifty percent (50%) or more of the -outstanding shares, or (iii) beneficial ownership of such entity. - -"You" (or "Your") shall mean an individual or Legal Entity exercising -permissions granted by this License. - -"Source" form shall mean the preferred form for making modifications, including -but not limited to software source code, documentation source, and configuration -files. - -"Object" form shall mean any form resulting from mechanical transformation or -translation of a Source form, including but not limited to compiled object code, -generated documentation, and conversions to other media types. - -"Work" shall mean the work of authorship, whether in Source or Object form, made -available under the License, as indicated by a copyright notice that is included -in or attached to the work (an example is provided in the Appendix below). - -"Derivative Works" shall mean any work, whether in Source or Object form, that -is based on (or derived from) the Work and for which the editorial revisions, -annotations, elaborations, or other modifications represent, as a whole, an -original work of authorship. For the purposes of this License, Derivative Works -shall not include works that remain separable from, or merely link (or bind by -name) to the interfaces of, the Work and Derivative Works thereof. - -"Contribution" shall mean any work of authorship, including the original version -of the Work and any modifications or additions to that Work or Derivative Works -thereof, that is intentionally submitted to Licensor for inclusion in the Work -by the copyright owner or by an individual or Legal Entity authorized to submit -on behalf of the copyright owner. For the purposes of this definition, -"submitted" means any form of electronic, verbal, or written communication sent -to the Licensor or its representatives, including but not limited to -communication on electronic mailing lists, source code control systems, and -issue tracking systems that are managed by, or on behalf of, the Licensor for -the purpose of discussing and improving the Work, but excluding communication -that is conspicuously marked or otherwise designated in writing by the copyright -owner as "Not a Contribution." - -"Contributor" shall mean Licensor and any individual or Legal Entity on behalf -of whom a Contribution has been received by Licensor and subsequently -incorporated within the Work. - -2. Grant of Copyright License. - -Subject to the terms and conditions of this License, each Contributor hereby -grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, -irrevocable copyright license to reproduce, prepare Derivative Works of, -publicly display, publicly perform, sublicense, and distribute the Work and such -Derivative Works in Source or Object form. - -3. Grant of Patent License. - -Subject to the terms and conditions of this License, each Contributor hereby -grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, -irrevocable (except as stated in this section) patent license to make, have -made, use, offer to sell, sell, import, and otherwise transfer the Work, where -such license applies only to those patent claims licensable by such Contributor -that are necessarily infringed by their Contribution(s) alone or by combination -of their Contribution(s) with the Work to which such Contribution(s) was -submitted. If You institute patent litigation against any entity (including a -cross-claim or counterclaim in a lawsuit) alleging that the Work or a -Contribution incorporated within the Work constitutes direct or contributory -patent infringement, then any patent licenses granted to You under this License -for that Work shall terminate as of the date such litigation is filed. - -4. Redistribution. - -You may reproduce and distribute copies of the Work or Derivative Works thereof -in any medium, with or without modifications, and in Source or Object form, -provided that You meet the following conditions: - -You must give any other recipients of the Work or Derivative Works a copy of -this License; and -You must cause any modified files to carry prominent notices stating that You -changed the files; and -You must retain, in the Source form of any Derivative Works that You distribute, -all copyright, patent, trademark, and attribution notices from the Source form -of the Work, excluding those notices that do not pertain to any part of the -Derivative Works; and -If the Work includes a "NOTICE" text file as part of its distribution, then any -Derivative Works that You distribute must include a readable copy of the -attribution notices contained within such NOTICE file, excluding those notices -that do not pertain to any part of the Derivative Works, in at least one of the -following places: within a NOTICE text file distributed as part of the -Derivative Works; within the Source form or documentation, if provided along -with the Derivative Works; or, within a display generated by the Derivative -Works, if and wherever such third-party notices normally appear. The contents of -the NOTICE file are for informational purposes only and do not modify the -License. You may add Your own attribution notices within Derivative Works that -You distribute, alongside or as an addendum to the NOTICE text from the Work, -provided that such additional attribution notices cannot be construed as -modifying the License. -You may add Your own copyright statement to Your modifications and may provide -additional or different license terms and conditions for use, reproduction, or -distribution of Your modifications, or for any such Derivative Works as a whole, -provided Your use, reproduction, and distribution of the Work otherwise complies -with the conditions stated in this License. - -5. Submission of Contributions. - -Unless You explicitly state otherwise, any Contribution intentionally submitted -for inclusion in the Work by You to the Licensor shall be under the terms and -conditions of this License, without any additional terms or conditions. -Notwithstanding the above, nothing herein shall supersede or modify the terms of -any separate license agreement you may have executed with Licensor regarding -such Contributions. - -6. Trademarks. - -This License does not grant permission to use the trade names, trademarks, -service marks, or product names of the Licensor, except as required for -reasonable and customary use in describing the origin of the Work and -reproducing the content of the NOTICE file. - -7. Disclaimer of Warranty. - -Unless required by applicable law or agreed to in writing, Licensor provides the -Work (and each Contributor provides its Contributions) on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied, -including, without limitation, any warranties or conditions of TITLE, -NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A PARTICULAR PURPOSE. You are -solely responsible for determining the appropriateness of using or -redistributing the Work and assume any risks associated with Your exercise of -permissions under this License. - -8. Limitation of Liability. - -In no event and under no legal theory, whether in tort (including negligence), -contract, or otherwise, unless required by applicable law (such as deliberate -and grossly negligent acts) or agreed to in writing, shall any Contributor be -liable to You for damages, including any direct, indirect, special, incidental, -or consequential damages of any character arising as a result of this License or -out of the use or inability to use the Work (including but not limited to -damages for loss of goodwill, work stoppage, computer failure or malfunction, or -any and all other commercial damages or losses), even if such Contributor has -been advised of the possibility of such damages. - -9. Accepting Warranty or Additional Liability. - -While redistributing the Work or Derivative Works thereof, You may choose to -offer, and charge a fee for, acceptance of support, warranty, indemnity, or -other liability obligations and/or rights consistent with this License. However, -in accepting such obligations, You may act only on Your own behalf and on Your -sole responsibility, not on behalf of any other Contributor, and only if You -agree to indemnify, defend, and hold each Contributor harmless for any liability -incurred by, or claims asserted against, such Contributor by reason of your -accepting any such warranty or additional liability. - -END OF TERMS AND CONDITIONS - -APPENDIX: How to apply the Apache License to your work - -To apply the Apache License to your work, attach the following boilerplate -notice, with the fields enclosed by brackets "[]" replaced with your own -identifying information. (Don't include the brackets!) The text should be -enclosed in the appropriate comment syntax for the file format. We also -recommend that a file or class name and description of purpose be included on -the same "printed page" as the copyright notice for easier identification within -third-party archives. - - Copyright [yyyy] [name of copyright owner] - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. diff --git a/third_party/github.com/ccding/go-logging/README.md b/third_party/github.com/ccding/go-logging/README.md deleted file mode 100644 index cfcefb60330..00000000000 --- a/third_party/github.com/ccding/go-logging/README.md +++ /dev/null @@ -1,217 +0,0 @@ -#go-logging -go-logging is a high-performance logging library for golang. -* Simple: It supports only necessary operations and easy to get start. -* Fast: Asynchronous logging without runtime-related fields has an extremely - low delay of about 800 nano-seconds. - -## Getting Started -### Installation -The step below will download the library source code to -`${GOPATH}/src/github.com/ccding/go-logging`. -```bash -go get github.com/ccding/go-logging/logging -``` - -Given the source code downloaded, it makes you be able to run the examples, -tests, and benchmarks. -```bash -cd ${GOPATH}/src/github.com/ccding/go-logging/logging -go get -go run ../example.go -go test -v -bench . -``` - -### Example -go-logging is used like any other Go libraries. You can simply use the library -in this way. -```go -import "github.com/ccding/go-logging/logging" -``` - -Here is a simple example. -```go -package main - -import ( - "github.com/ccding/go-logging/logging" -) - -func main() { - logger, _ := logging.SimpleLogger("main") - logger.Error("this is a test from error") - logger.Destroy() -} -``` - -### Configuration -#### Construction Functions -It has the following functions to create a logger. -```go -// with BasicFormat and writing to stdout -SimpleLogger(name string) (*Logger, error) -// with BasicFormat and writing to DefaultFileName -BasicLogger(name string) (*Logger, error) -// with RichFormatand writing to DefaultFileName -RichLogger(name string) (*Logger, error) -// with detailed configuration and writing to file -FileLogger(name string, level Level, format string, timeFormat string, file string, sync bool) (*Logger, error) -// with detailed configuration and writing to a writer -WriterLogger(name string, level Level, format string, timeFormat string, out io.Writer, sync bool) (*Logger, error) -// read configurations from a config file -ConfigLogger(filename string) (*Logger, error) -``` -The meanings of these fields are -```go -name string // logger name -level Level // record level higher than this will be printed -format string // format configuration -timeFormat string // format for time -file string // file name for logging -out io.Writer // writer for logging -sync bool // use sync or async way to record logs -``` -The detailed description of these fields will be presented later. - -#### Logging Functions -It supports the following functions for logging. All of these functions are -thread-safe. -```go -(*Logger) Logf(level Level, format string, v ...interface{}) -(*Logger) Log(level Level, v ...interface{}) -(*Logger) Criticalf(format string, v ...interface{}) -(*Logger) Critical(v ...interface{}) -(*Logger) Fatalf(format string, v ...interface{}) -(*Logger) Fatal(v ...interface{}) -(*Logger) Errorf(format string, v ...interface{}) -(*Logger) Error(v ...interface{}) -(*Logger) Warningf(format string, v ...interface{}) -(*Logger) Warning(v ...interface{}) -(*Logger) Warnf(format string, v ...interface{}) -(*Logger) Warn(v ...interface{}) -(*Logger) Infof(format string, v ...interface{}) -(*Logger) Info(v ...interface{}) -(*Logger) Debugf(format string, v ...interface{}) -(*Logger) Debug(v ...interface{}) -(*Logger) Notsetf(format string, v ...interface{}) -(*Logger) Notset(v ...interface{}) -``` - -#### Logger Operations -The logger supports the following operations. In these functions, `SetWriter` -and `Destroy` are not thread-safe, while others are. All these functions are -running in a synchronous way. -```go -// Getter functions -(*Logger) Name() string // get name -(*Logger) TimeFormat() string // get time format -(*Logger) Level() Level // get level [this function is thread safe] -(*Logger) RecordFormat() string // get the first part of the format -(*Logger) RecordArgs() []string // get the second part of the format -(*Logger) Writer() io.Writer // get writer -(*Logger) Sync() bool // get sync or async - -// Setter functions -(*Logger) SetLevel(level Level) // set level [this function is thread safe] -(*Logger) SetWriter(out ...io.Writer) // set multiple writers - -// Other functions -(*Logger) Flush() // flush the writer -(*Logger) Destroy() // destroy the logger -``` - -#### Fields Description - -##### Name -Name field is a string, which can be written to the logging and used to -separate multiple loggers. It allows two logger having the same name. There -is not any default value for name. - -##### Logging Levels -There are these levels in logging. -```go -CRITICAL 50 -FATAL CRITICAL -ERROR 40 -WARNING 30 -WARN WARNING -INFO 20 -DEBUG 10 -NOTSET 0 -``` - -##### Record Format -The record format is described by a string, which has two parts separated by -`\n`. The first part describes the format of the log, and the second part -lists all the fields to be shown in the log. In other word, the first part is -the first parameter `format` of `fmt.Printf(format string, v ...interface{})`, -and the second part describes the second parameter `v` of it. It is not -allowed to have `\n` in the first part. The fields in the second part are -separated by comma `,`, while extra blank spaces are allowed. An example of -the format string is -```go -const BasicFormat = "%s [%6s] %30s - %s\n name,levelname,time,message" -``` -which is the pre-defined `BasicFormat` used by `BasicLogger()` and -`SimpleLogger()`. - -It supports the following fields for the second part of the format. -```go -"name" string %s // name of the logger -"seqid" uint64 %d // sequence number -"levelno" int32 %d // level number -"levelname" string %s // level name -"created" int64 %d // starting time of the logger -"nsecs" int64 %d // nanosecond of the starting time -"time" string %s // record created time -"timestamp" int64 %d // timestamp of record -"rtime" int64 %d // relative time since started -"filename" string %s // source filename of the caller -"pathname" string %s // filename with path -"module" string %s // executable filename -"lineno" int %d // line number in source code -"funcname" string %s // function name of the caller -"thread" int32 %d // thread id -"process" int %d // process id -"message" string %d // logger message -``` -The following runtime-related fields is extremely expensive and slow, please -be careful when using them. -```go -"filename" string %s // source filename of the caller -"pathname" string %s // filename with path -"lineno" int %d // line number in source code -"funcname" string %s // function name of the caller -"thread" int32 %d // thread id -``` - -There are a few pre-defined values for record format. -```go -BasicFormat = "%s [%6s] %30s - %s\n name,levelname,time,message" -RichFormat = "%s [%6s] %d %30s - %d - %s:%s:%d - %s\n name, levelname, seqid, time, thread, filename, funcname, lineno, message" -``` - -##### Time Format -We use the same time format as golang. The default time format is -```go -DefaultTimeFormat = "2006-01-02 15:04:05.999999999" // default time format -``` - -##### File Name, Writer, and Sync -The meaning of these fields are obvious. Filename is used to create writer. -We also allow the user create a writer by herself and pass it to the logger. -Sync describes whether the user would like to use synchronous or asynchronous -method to write logs. `true` value means synchronous method, and `false` value -means asynchronous way. We suggest you use asynchronous way because it causes -extremely low extra delay by the logging functions. - -## Contributors -In alphabetical order -* Cong Ding ([ccding][ccding]) -* Xiang Li ([xiangli-cmu][xiangli]) -* Zifei Tong ([5kg][5kg]) -[ccding]: //github.com/ccding -[xiangli]: //github.com/xiangli-cmu -[5kg]: //github.com/5kg - -## TODO List -1. logging server diff --git a/third_party/github.com/ccding/go-logging/example.conf b/third_party/github.com/ccding/go-logging/example.conf deleted file mode 100644 index ecc9a860ec0..00000000000 --- a/third_party/github.com/ccding/go-logging/example.conf +++ /dev/null @@ -1,3 +0,0 @@ -name = example -sync = 0 - diff --git a/third_party/github.com/ccding/go-logging/example.go b/third_party/github.com/ccding/go-logging/example.go deleted file mode 100644 index 1b82842fa1b..00000000000 --- a/third_party/github.com/ccding/go-logging/example.go +++ /dev/null @@ -1,45 +0,0 @@ -// Copyright 2013, Cong Ding. All rights reserved. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -// -// author: Cong Ding - -package main - -import ( - "github.com/ccding/go-logging/logging" - "time" -) - -func main() { - logger1, _ := logging.SimpleLogger("main") - logger1.SetLevel(logging.NOTSET) - logger1.Error("this is a test from error") - logger1.Debug("this is a test from debug") - logger1.Notset("orz", time.Now().UnixNano()) - logger1.Destroy() - - logger2, _ := logging.RichLogger("main") - logger2.SetLevel(logging.DEBUG) - logger2.Error("this is a test from error") - logger2.Debug("this is a test from debug") - logger2.Notset("orz", time.Now().UnixNano()) - logger2.Destroy() - - logger3, _ := logging.ConfigLogger("example.conf") - logger3.SetLevel(logging.DEBUG) - logger3.Error("this is a test from error") - logger3.Debug("this is a test from debug") - logger3.Notset("orz", time.Now().UnixNano()) - logger3.Destroy() -} diff --git a/third_party/github.com/ccding/go-logging/logging/commands.go b/third_party/github.com/ccding/go-logging/logging/commands.go deleted file mode 100644 index be19405988b..00000000000 --- a/third_party/github.com/ccding/go-logging/logging/commands.go +++ /dev/null @@ -1,98 +0,0 @@ -// Copyright 2013, Cong Ding. All rights reserved. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -// -// author: Cong Ding - -package logging - -// Logln receives log request from the client. The request includes a set of -// variables. -func (logger *Logger) Log(level Level, v ...interface{}) { - // Don't delete this calling. The calling is used to keep the same - // calldepth for all the logging functions. The calldepth is used to - // get runtime information such as line number, function name, etc. - logger.log(level, v...) -} - -// Logf receives log request from the client. The request has a string -// parameter to describe the format of output. -func (logger *Logger) Logf(level Level, format string, v ...interface{}) { - logger.logf(level, format, v...) -} - -// Other quick commands for different level - -func (logger *Logger) Critical(v ...interface{}) { - logger.log(CRITICAL, v...) -} - -func (logger *Logger) Fatal(v ...interface{}) { - logger.log(CRITICAL, v...) -} - -func (logger *Logger) Error(v ...interface{}) { - logger.log(ERROR, v...) -} - -func (logger *Logger) Warn(v ...interface{}) { - logger.log(WARNING, v...) -} - -func (logger *Logger) Warning(v ...interface{}) { - logger.log(WARNING, v...) -} - -func (logger *Logger) Info(v ...interface{}) { - logger.log(INFO, v...) -} - -func (logger *Logger) Debug(v ...interface{}) { - logger.log(DEBUG, v...) -} - -func (logger *Logger) Notset(v ...interface{}) { - logger.log(NOTSET, v...) -} - -func (logger *Logger) Criticalf(format string, v ...interface{}) { - logger.logf(CRITICAL, format, v...) -} - -func (logger *Logger) Fatalf(format string, v ...interface{}) { - logger.logf(CRITICAL, format, v...) -} - -func (logger *Logger) Errorf(format string, v ...interface{}) { - logger.logf(ERROR, format, v...) -} - -func (logger *Logger) Warnf(format string, v ...interface{}) { - logger.logf(WARNING, format, v...) -} - -func (logger *Logger) Warningf(format string, v ...interface{}) { - logger.logf(WARNING, format, v...) -} - -func (logger *Logger) Infof(format string, v ...interface{}) { - logger.logf(INFO, format, v...) -} - -func (logger *Logger) Debugf(format string, v ...interface{}) { - logger.logf(DEBUG, format, v...) -} - -func (logger *Logger) Notsetf(format string, v ...interface{}) { - logger.logf(NOTSET, format, v...) -} diff --git a/third_party/github.com/ccding/go-logging/logging/fields.go b/third_party/github.com/ccding/go-logging/logging/fields.go deleted file mode 100644 index aff57fca9ca..00000000000 --- a/third_party/github.com/ccding/go-logging/logging/fields.go +++ /dev/null @@ -1,236 +0,0 @@ -// Copyright 2013, Cong Ding. All rights reserved. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -// -// author: Cong Ding - -package logging - -import ( - "bitbucket.org/kardianos/osext" - "os" - "path" - "runtime" - "sync/atomic" - "time" -) - -// The struct for each log record -type record struct { - level Level - seqid uint64 - pathname string - filename string - module string - lineno int - funcname string - thread int - process int - message string - time time.Time -} - -// This variable maps fields in recordArgs to relavent function signatures -var fields = map[string]func(*Logger, *record) interface{}{ - "name": (*Logger).lname, // name of the logger - "seqid": (*Logger).nextSeqid, // sequence number - "levelno": (*Logger).levelno, // level number - "levelname": (*Logger).levelname, // level name - "created": (*Logger).created, // starting time of the logger - "nsecs": (*Logger).nsecs, // nanosecond of the starting time - "time": (*Logger).time, // record created time - "timestamp": (*Logger).timestamp, // timestamp of record - "rtime": (*Logger).rtime, // relative time since started - "filename": (*Logger).filename, // source filename of the caller - "pathname": (*Logger).pathname, // filename with path - "module": (*Logger).module, // executable filename - "lineno": (*Logger).lineno, // line number in source code - "funcname": (*Logger).funcname, // function name of the caller - "thread": (*Logger).thread, // thread id - "process": (*Logger).process, // process id - "message": (*Logger).message, // logger message -} - -var runtimeFields = map[string]bool{ - "name": false, - "seqid": false, - "levelno": false, - "levelname": false, - "created": false, - "nsecs": false, - "time": false, - "timestamp": false, - "rtime": false, - "filename": true, - "pathname": true, - "module": false, - "lineno": true, - "funcname": true, - "thread": true, - "process": false, - "message": false, -} - -// If it fails to get some fields with string type, these fields are set to -// errString value. -const errString = "???" - -// GetGoID returns the id of goroutine, which is defined in ./get_go_id.c -func GetGoID() int32 - -// genRuntime generates the runtime information, including pathname, function -// name, filename, line number. -func genRuntime(r *record) { - calldepth := 5 - pc, file, line, ok := runtime.Caller(calldepth) - if ok { - // Generate short function name - fname := runtime.FuncForPC(pc).Name() - fshort := fname - for i := len(fname) - 1; i > 0; i-- { - if fname[i] == '.' { - fshort = fname[i+1:] - break - } - } - - r.pathname = file - r.funcname = fshort - r.filename = path.Base(file) - r.lineno = line - } else { - r.pathname = errString - r.funcname = errString - r.filename = errString - // Here we uses -1 rather than 0, because the default value in - // golang is 0 and we should know the value is uninitialized - // or failed to get - r.lineno = -1 - } -} - -// Logger name -func (logger *Logger) lname(r *record) interface{} { - return logger.name -} - -// Next sequence number -func (logger *Logger) nextSeqid(r *record) interface{} { - if r.seqid == 0 { - r.seqid = atomic.AddUint64(&(logger.seqid), 1) - } - return r.seqid -} - -// Log level number -func (logger *Logger) levelno(r *record) interface{} { - return int32(r.level) -} - -// Log level name -func (logger *Logger) levelname(r *record) interface{} { - return levelNames[r.level] -} - -// File name of calling logger, with whole path -func (logger *Logger) pathname(r *record) interface{} { - if r.pathname == "" { - genRuntime(r) - } - return r.pathname -} - -// File name of calling logger -func (logger *Logger) filename(r *record) interface{} { - if r.filename == "" { - genRuntime(r) - } - return r.filename -} - -// module name -func (logger *Logger) module(r *record) interface{} { - module, _ := osext.Executable() - return path.Base(module) -} - -// Line number -func (logger *Logger) lineno(r *record) interface{} { - if r.lineno == 0 { - genRuntime(r) - } - return r.lineno -} - -// Function name -func (logger *Logger) funcname(r *record) interface{} { - if r.funcname == "" { - genRuntime(r) - } - return r.funcname -} - -// Timestamp of starting time -func (logger *Logger) created(r *record) interface{} { - return logger.startTime.UnixNano() -} - -// RFC3339Nano time -func (logger *Logger) time(r *record) interface{} { - if r.time.IsZero() { - r.time = time.Now() - } - return r.time.Format(logger.timeFormat) -} - -// Nanosecond of starting time -func (logger *Logger) nsecs(r *record) interface{} { - return logger.startTime.Nanosecond() -} - -// Nanosecond timestamp -func (logger *Logger) timestamp(r *record) interface{} { - if r.time.IsZero() { - r.time = time.Now() - } - return r.time.UnixNano() -} - -// Nanoseconds since logger created -func (logger *Logger) rtime(r *record) interface{} { - if r.time.IsZero() { - r.time = time.Now() - } - return r.time.Sub(logger.startTime).Nanoseconds() -} - -// Thread ID -func (logger *Logger) thread(r *record) interface{} { - if r.thread == 0 { - r.thread = int(GetGoID()) - } - return r.thread -} - -// Process ID -func (logger *Logger) process(r *record) interface{} { - if r.process == 0 { - r.process = os.Getpid() - } - return r.process -} - -// The log message -func (logger *Logger) message(r *record) interface{} { - return r.message -} diff --git a/third_party/github.com/ccding/go-logging/logging/fields_test.go b/third_party/github.com/ccding/go-logging/logging/fields_test.go deleted file mode 100644 index 7efae3b070b..00000000000 --- a/third_party/github.com/ccding/go-logging/logging/fields_test.go +++ /dev/null @@ -1,60 +0,0 @@ -// Copyright 2013, Cong Ding. All rights reserved. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -// -// author: Cong Ding - -package logging - -import ( - "fmt" - "strconv" - "testing" -) - -func empty() { -} - -func TestGetGoID(t *testing.T) { - for i := 0; i < 1000; i++ { - goid := int(GetGoID()) - go empty() - goid2 := int(GetGoID()) - if goid != goid2 { - t.Errorf("%v, %v\n", goid, goid2) - } - } -} - -func TestSeqid(t *testing.T) { - logger, _ := BasicLogger("test") - for i := 0; i < 1000; i++ { - r := new(record) - name := strconv.Itoa(i + 1) - seq := logger.nextSeqid(r) - if fmt.Sprintf("%d", seq) != name { - t.Errorf("%v, %v\n", seq, name) - } - } - logger.Destroy() -} - -func TestName(t *testing.T) { - name := "test" - logger, _ := BasicLogger(name) - r := new(record) - if logger.lname(r) != name { - t.Errorf("%v, %v\n", logger.lname(r), name) - } - logger.Destroy() -} diff --git a/third_party/github.com/ccding/go-logging/logging/formater.go b/third_party/github.com/ccding/go-logging/logging/formater.go deleted file mode 100644 index 8be2a31d29b..00000000000 --- a/third_party/github.com/ccding/go-logging/logging/formater.go +++ /dev/null @@ -1,62 +0,0 @@ -// Copyright 2013, Cong Ding. All rights reserved. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -// -// author: Cong Ding - -package logging - -import ( - "errors" - "fmt" - "strings" -) - -// pre-defined formats -const ( - BasicFormat = "%s [%6s] %30s - %s\n name,levelname,time,message" - RichFormat = "%s [%6s] %d %30s - %d - %s:%s:%d - %s\n name, levelname, seqid, time, thread, filename, funcname, lineno, message" -) - -// genLog generates log string from the format setting. -func (logger *Logger) genLog(level Level, message string) string { - fs := make([]interface{}, len(logger.recordArgs)) - r := new(record) - r.message = message - r.level = level - for k, v := range logger.recordArgs { - fs[k] = fields[v](logger, r) - } - return fmt.Sprintf(logger.recordFormat, fs...) -} - -// parseFormat checks the legality of format and parses it to recordFormat and recordArgs -func (logger *Logger) parseFormat(format string) error { - logger.runtime = false - fts := strings.Split(format, "\n") - if len(fts) != 2 { - return errors.New("logging format error") - } - logger.recordFormat = fts[0] - logger.recordArgs = strings.Split(fts[1], ",") - for k, v := range logger.recordArgs { - tv := strings.TrimSpace(v) - _, ok := fields[tv] - if ok == false { - return errors.New("logging format error") - } - logger.recordArgs[k] = tv - logger.runtime = logger.runtime || runtimeFields[tv] - } - return nil -} diff --git a/third_party/github.com/ccding/go-logging/logging/get_go_id.c b/third_party/github.com/ccding/go-logging/logging/get_go_id.c deleted file mode 100644 index f9b216f9f29..00000000000 --- a/third_party/github.com/ccding/go-logging/logging/get_go_id.c +++ /dev/null @@ -1,25 +0,0 @@ -// Copyright 2013, Cong Ding. All rights reserved. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -// -// author: Cong Ding - -// This file defines GetGoId function, which is used to get the id of the -// current goroutine. More details about this function are availeble in the -// runtime.c file of golang source code. -#include - -void ·GetGoID(int32 ret) { - ret = g->goid; - USED(&ret); -} diff --git a/third_party/github.com/ccding/go-logging/logging/level.go b/third_party/github.com/ccding/go-logging/logging/level.go deleted file mode 100644 index 4ada90a0366..00000000000 --- a/third_party/github.com/ccding/go-logging/logging/level.go +++ /dev/null @@ -1,68 +0,0 @@ -// Copyright 2013, Cong Ding. All rights reserved. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -// -// author: Cong Ding - -package logging - -// Level is the type of level. -type Level int32 - -// Values of level -const ( - CRITICAL Level = 50 - FATAL Level = CRITICAL - ERROR Level = 40 - WARNING Level = 30 - WARN Level = WARNING - INFO Level = 20 - DEBUG Level = 10 - NOTSET Level = 0 -) - -// The mapping from level to level name -var levelNames = map[Level]string{ - CRITICAL: "CRITICAL", - ERROR: "ERROR", - WARNING: "WARNING", - INFO: "INFO", - DEBUG: "DEBUG", - NOTSET: "NOTSET", -} - -// The mapping from level name to level -var levelValues = map[string]Level{ - "CRITICAL": CRITICAL, - "ERROR": ERROR, - "WARN": WARNING, - "WARNING": WARNING, - "INFO": INFO, - "DEBUG": DEBUG, - "NOTSET": NOTSET, -} - -// String function casts level value to string -func (level *Level) String() string { - return levelNames[*level] -} - -// GetLevelName lets users be able to get level name from level value. -func GetLevelName(levelValue Level) string { - return levelNames[levelValue] -} - -// GetLevelValue lets users be able to get level value from level name. -func GetLevelValue(levelName string) Level { - return levelValues[levelName] -} diff --git a/third_party/github.com/ccding/go-logging/logging/logging.go b/third_party/github.com/ccding/go-logging/logging/logging.go deleted file mode 100644 index 6467d94ef8c..00000000000 --- a/third_party/github.com/ccding/go-logging/logging/logging.go +++ /dev/null @@ -1,259 +0,0 @@ -// Copyright 2013, Cong Ding. All rights reserved. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -// -// author: Cong Ding - -// Package logging implements log library for other applications. It provides -// functions Debug, Info, Warning, Error, Critical, and formatting version -// Logf. -// -// Example: -// -// logger := logging.SimpleLogger("main") -// logger.SetLevel(logging.WARNING) -// logger.Error("test for error") -// logger.Warning("test for warning", "second parameter") -// logger.Debug("test for debug") -// -package logging - -import ( - "github.com/ccding/go-config-reader/config" - "io" - "os" - "strconv" - "sync" - "sync/atomic" - "time" -) - -// Pre-defined formats -const ( - DefaultFileName = "logging.log" // default logging filename - DefaultConfigFile = "logging.conf" // default logging configuration file - DefaultTimeFormat = "2006-01-02 15:04:05.999999999" // defaulttime format - bufSize = 1000 // buffer size for writer - queueSize = 10000 // chan queue size in async logging - reqSize = 10000 // chan queue size in async logging -) - -// Logger is the logging struct. -type Logger struct { - - // Be careful of the alignment issue of the variable seqid because it - // uses the sync/atomic.AddUint64() operation. If the alignment is - // wrong, it will cause a panic. To solve the alignment issue in an - // easy way, we put seqid to the beginning of the structure. - // seqid is only visiable internally. - seqid uint64 // last used sequence number in record - - // These variables can be configured by users. - name string // logger name - level Level // record level higher than this will be printed - recordFormat string // format of the record - recordArgs []string // arguments to be used in the recordFormat - out io.Writer // writer - sync bool // use sync or async way to record logs - timeFormat string // format for time - - // These variables are visible to users. - startTime time.Time // start time of the logger - - // Internally used variables, which don't have get and set functions. - wlock sync.Mutex // writer lock - queue chan string // queue used in async logging - request chan request // queue used in non-runtime logging - flush chan bool // flush signal for the watcher to write - quit chan bool // quit signal for the watcher to quit - fd *os.File // file handler, used to close the file on destroy - runtime bool // with runtime operation or not -} - -// SimpleLogger creates a new logger with simple configuration. -func SimpleLogger(name string) (*Logger, error) { - return createLogger(name, WARNING, BasicFormat, DefaultTimeFormat, os.Stdout, false) -} - -// BasicLogger creates a new logger with basic configuration. -func BasicLogger(name string) (*Logger, error) { - return FileLogger(name, WARNING, BasicFormat, DefaultTimeFormat, DefaultFileName, false) -} - -// RichLogger creates a new logger with simple configuration. -func RichLogger(name string) (*Logger, error) { - return FileLogger(name, NOTSET, RichFormat, DefaultTimeFormat, DefaultFileName, false) -} - -// FileLogger creates a new logger with file output. -func FileLogger(name string, level Level, format string, timeFormat string, file string, sync bool) (*Logger, error) { - out, err := os.Create(file) - if err != nil { - return nil, err - } - logger, err := createLogger(name, level, format, timeFormat, out, sync) - if err == nil { - logger.fd = out - return logger, nil - } else { - return nil, err - } -} - -// WriterLogger creates a new logger with a writer -func WriterLogger(name string, level Level, format string, timeFormat string, out io.Writer, sync bool) (*Logger, error) { - return createLogger(name, level, format, timeFormat, out, sync) -} - -// WriterLogger creates a new logger from a configuration file -func ConfigLogger(filename string) (*Logger, error) { - conf := config.NewConfig(filename) - err := conf.Read() - if err != nil { - return nil, err - } - name := conf.Get("", "name") - slevel := conf.Get("", "level") - if slevel == "" { - slevel = "0" - } - l, err := strconv.Atoi(slevel) - if err != nil { - return nil, err - } - level := Level(l) - format := conf.Get("", "format") - if format == "" { - format = BasicFormat - } - timeFormat := conf.Get("", "timeFormat") - if timeFormat == "" { - timeFormat = DefaultTimeFormat - } - ssync := conf.Get("", "sync") - if ssync == "" { - ssync = "0" - } - file := conf.Get("", "file") - if file == "" { - file = DefaultFileName - } - sync := true - if ssync == "0" { - sync = false - } else if ssync == "1" { - sync = true - } else { - return nil, err - } - return FileLogger(name, level, format, timeFormat, file, sync) -} - -// createLogger create a new logger -func createLogger(name string, level Level, format string, timeFormat string, out io.Writer, sync bool) (*Logger, error) { - logger := new(Logger) - - err := logger.parseFormat(format) - if err != nil { - return nil, err - } - - // asign values to logger - logger.name = name - logger.level = level - logger.out = out - logger.seqid = 0 - logger.sync = sync - logger.queue = make(chan string, queueSize) - logger.request = make(chan request, reqSize) - logger.flush = make(chan bool) - logger.quit = make(chan bool) - logger.startTime = time.Now() - logger.fd = nil - logger.timeFormat = timeFormat - - // start watcher to write logs if it is async or no runtime field - if !logger.sync { - go logger.watcher() - } - - return logger, nil -} - -// Destroy sends quit signal to watcher and releases all the resources. -func (logger *Logger) Destroy() { - if !logger.sync { - // quit watcher - logger.quit <- true - // wait for watcher quit - <-logger.quit - } - // clean up - if logger.fd != nil { - logger.fd.Close() - } -} - -// Flush the writer -func (logger *Logger) Flush() { - if !logger.sync { - // send flush signal - logger.flush <- true - // wait for flush finish - <-logger.flush - } -} - -// Getter functions - -func (logger *Logger) Name() string { - return logger.name -} - -func (logger *Logger) StartTime() int64 { - return logger.startTime.UnixNano() -} - -func (logger *Logger) TimeFormat() string { - return logger.timeFormat -} - -func (logger *Logger) Level() Level { - return Level(atomic.LoadInt32((*int32)(&logger.level))) -} - -func (logger *Logger) RecordFormat() string { - return logger.recordFormat -} - -func (logger *Logger) RecordArgs() []string { - return logger.recordArgs -} - -func (logger *Logger) Writer() io.Writer { - return logger.out -} - -func (logger *Logger) Sync() bool { - return logger.sync -} - -// Setter functions - -func (logger *Logger) SetLevel(level Level) { - atomic.StoreInt32((*int32)(&logger.level), int32(level)) -} - -func (logger *Logger) SetWriter(out ...io.Writer) { - logger.out = io.MultiWriter(out...) -} diff --git a/third_party/github.com/ccding/go-logging/logging/logging_test.go b/third_party/github.com/ccding/go-logging/logging/logging_test.go deleted file mode 100644 index fcf4bcce65d..00000000000 --- a/third_party/github.com/ccding/go-logging/logging/logging_test.go +++ /dev/null @@ -1,71 +0,0 @@ -// Copyright 2013, Cong Ding. All rights reserved. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -// -// author: Cong Ding - -package logging - -import ( - "fmt" - "os" - "testing" -) - -func BenchmarkSync(b *testing.B) { - logger, _ := RichLogger("main") - logger.SetLevel(NOTSET) - for i := 0; i < b.N; i++ { - logger.Error("this is a test from error") - } - logger.Flush() - logger.Destroy() -} - -func BenchmarkAsync(b *testing.B) { - logger, _ := RichLogger("main") - logger.SetLevel(NOTSET) - for i := 0; i < b.N; i++ { - logger.Error("this is a test from error") - } - logger.Flush() - logger.Destroy() -} - -func BenchmarkBasicSync(b *testing.B) { - logger, _ := BasicLogger("main") - logger.SetLevel(NOTSET) - for i := 0; i < b.N; i++ { - logger.Error("this is a test from error") - } - logger.Flush() - logger.Destroy() -} - -func BenchmarkBasicAsync(b *testing.B) { - logger, _ := BasicLogger("main") - logger.SetLevel(NOTSET) - for i := 0; i < b.N; i++ { - logger.Error("this is a test from error") - } - logger.Flush() - logger.Destroy() -} - -func BenchmarkPrintln(b *testing.B) { - out, _ := os.Create("logging.log") - for i := 0; i < b.N; i++ { - fmt.Fprintln(out, "this is a test from error") - } - out.Close() -} diff --git a/third_party/github.com/ccding/go-logging/logging/request.go b/third_party/github.com/ccding/go-logging/logging/request.go deleted file mode 100644 index 4b86410835f..00000000000 --- a/third_party/github.com/ccding/go-logging/logging/request.go +++ /dev/null @@ -1,24 +0,0 @@ -// Copyright 2013, Cong Ding. All rights reserved. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -// -// author: Cong Ding - -package logging - -// request struct stores the logger request -type request struct { - level Level - format string - v []interface{} -} diff --git a/third_party/github.com/ccding/go-logging/logging/writer.go b/third_party/github.com/ccding/go-logging/logging/writer.go deleted file mode 100644 index 9efeddf6072..00000000000 --- a/third_party/github.com/ccding/go-logging/logging/writer.go +++ /dev/null @@ -1,130 +0,0 @@ -// Copyright 2013, Cong Ding. All rights reserved. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. -// -// author: Cong Ding - -package logging - -import ( - "bytes" - "fmt" - "sync/atomic" - "time" -) - -// watcher watches the logger.queue channel, and writes the logs to output -func (logger *Logger) watcher() { - var buf bytes.Buffer - for { - timeout := time.After(time.Second / 10) - - for i := 0; i < bufSize; i++ { - select { - case msg := <-logger.queue: - fmt.Fprintln(&buf, msg) - case req := <-logger.request: - logger.flushReq(&buf, &req) - case <-timeout: - i = bufSize - case <-logger.flush: - logger.flushBuf(&buf) - logger.flush <- true - i = bufSize - case <-logger.quit: - // If quit signal received, cleans the channel - // and writes all of them to io.Writer. - for { - select { - case msg := <-logger.queue: - fmt.Fprintln(&buf, msg) - case req := <-logger.request: - logger.flushReq(&buf, &req) - case <-logger.flush: - // do nothing - default: - logger.flushBuf(&buf) - logger.quit <- true - return - } - } - } - } - logger.flushBuf(&buf) - } -} - -// flushBuf flushes the content of buffer to out and reset the buffer -func (logger *Logger) flushBuf(b *bytes.Buffer) { - if len(b.Bytes()) > 0 { - logger.out.Write(b.Bytes()) - b.Reset() - } -} - -// flushReq handles the request and writes the result to writer -func (logger *Logger) flushReq(b *bytes.Buffer, req *request) { - if req.format == "" { - msg := fmt.Sprint(req.v...) - msg = logger.genLog(req.level, msg) - fmt.Fprintln(b, msg) - } else { - msg := fmt.Sprintf(req.format, req.v...) - msg = logger.genLog(req.level, msg) - fmt.Fprintln(b, msg) - } -} - -// flushMsg is to print log to file, stdout, or others. -func (logger *Logger) flushMsg(message string) { - if logger.sync { - logger.wlock.Lock() - defer logger.wlock.Unlock() - fmt.Fprintln(logger.out, message) - } else { - logger.queue <- message - } -} - -// log records log v... with level `level'. -func (logger *Logger) log(level Level, v ...interface{}) { - if int32(level) >= atomic.LoadInt32((*int32)(&logger.level)) { - if logger.runtime || logger.sync { - message := fmt.Sprint(v...) - message = logger.genLog(level, message) - logger.flushMsg(message) - } else { - r := new(request) - r.level = level - r.v = v - logger.request <- *r - } - } -} - -// logf records log v... with level `level'. -func (logger *Logger) logf(level Level, format string, v ...interface{}) { - if int32(level) >= atomic.LoadInt32((*int32)(&logger.level)) { - if logger.runtime || logger.sync { - message := fmt.Sprintf(format, v...) - message = logger.genLog(level, message) - logger.flushMsg(message) - } else { - r := new(request) - r.level = level - r.format = format - r.v = v - logger.request <- *r - } - } -} diff --git a/third_party/github.com/coreos/go-etcd/README.md b/third_party/github.com/coreos/go-etcd/README.md index ac87acdfa6d..7a8f8b34a72 100644 --- a/third_party/github.com/coreos/go-etcd/README.md +++ b/third_party/github.com/coreos/go-etcd/README.md @@ -2,4 +2,49 @@ golang client library for etcd -This etcd client library is under heavy development. Check back soon for more docs. In the meantime, check out [etcd](https://github.com/coreos/etcd) for details on the client protocol. +This etcd client library is under heavy development. Check back soon for more +docs. In the meantime, check out [etcd](https://github.com/coreos/etcd) for +details on the client protocol. + +For usage see example below or look at godoc: [go-etcd/etcd](http://godoc.org/github.com/coreos/go-etcd/etcd) + +## Install + +```bash +go get github.com/coreos/go-etcd/etcd +``` + +## Examples + +Returning error values are not showed for the sake of simplicity, but you +should check them. + +```go +package main + +import ( + "fmt" + "github.com/coreos/go-etcd/etcd" +) + +func main() { + c := etcd.NewClient() // default binds to http://0.0.0.0:4001 + + // SET the value "bar" to the key "foo" with zero TTL + // returns a: *store.Response + res, _ := c.Set("foo", "bar", 0) + fmt.Printf("set response: %+v\n", res) + + // GET the value that is stored for the key "foo" + // return a slice: []*store.Response + values, _ := c.Get("foo") + for i, res := range values { // .. and print them out + fmt.Printf("[%d] get response: %+v\n", i, res) + } + + // DELETE the key "foo" + // returns a: *store.Response + res, _ = c.Delete("foo") + fmt.Printf("delete response: %+v\n", res) +} +``` diff --git a/third_party/github.com/coreos/go-etcd/etcd/client.go b/third_party/github.com/coreos/go-etcd/etcd/client.go index 723e0ebdcf3..abc4574fa5f 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/client.go +++ b/third_party/github.com/coreos/go-etcd/etcd/client.go @@ -40,10 +40,9 @@ func NewClient() *Client { // default leader and machines cluster := Cluster{ - Leader: "http://0.0.0.0:4001", - Machines: make([]string, 1), + Leader: "http://127.0.0.1:4001", + Machines: []string{"http://127.0.0.1:4001"}, } - cluster.Machines[0] = "http://0.0.0.0:4001" config := Config{ // default use http @@ -117,7 +116,7 @@ func (c *Client) SyncCluster() bool { // sync cluster information by providing machine list func (c *Client) internalSyncCluster(machines []string) bool { for _, machine := range machines { - httpPath := c.createHttpPath(machine, "v1/machines") + httpPath := c.createHttpPath(machine, version+"/machines") resp, err := c.httpClient.Get(httpPath) if err != nil { // try another machine in the cluster @@ -236,11 +235,12 @@ func (c *Client) sendRequest(method string, _path string, body string) (*http.Re // try to connect the leader continue } else if resp.StatusCode == http.StatusInternalServerError { + resp.Body.Close() + retry++ if retry > 2*len(c.cluster.Machines) { return nil, errors.New("Cannot reach servers") } - resp.Body.Close() continue } else { logger.Debug("send.return.response ", httpPath) diff --git a/third_party/github.com/coreos/go-etcd/etcd/debug.go b/third_party/github.com/coreos/go-etcd/etcd/debug.go index f35dfae76a0..d82d9b29284 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/debug.go +++ b/third_party/github.com/coreos/go-etcd/etcd/debug.go @@ -1,19 +1,27 @@ package etcd import ( - "github.com/ccding/go-logging/logging" + "github.com/coreos/go-log/log" + "os" ) -var logger, _ = logging.SimpleLogger("go-etcd") +var logger *log.Logger func init() { - logger.SetLevel(logging.FATAL) + setLogger(log.PriErr) } func OpenDebug() { - logger.SetLevel(logging.NOTSET) + setLogger(log.PriDebug) } func CloseDebug() { - logger.SetLevel(logging.FATAL) + setLogger(log.PriErr) +} + +func setLogger(priority log.Priority) { + logger = log.NewSimple( + log.PriorityFilter( + priority, + log.WriterSink(os.Stdout, log.BasicFormat, log.BasicFields))) } diff --git a/third_party/github.com/coreos/go-etcd/etcd/version.go b/third_party/github.com/coreos/go-etcd/etcd/version.go index b27956805ce..e84e7b5b765 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/version.go +++ b/third_party/github.com/coreos/go-etcd/etcd/version.go @@ -1,3 +1,3 @@ package etcd -var version = "v1" +const version = "v1" diff --git a/third_party/github.com/coreos/go-log/log/commands.go b/third_party/github.com/coreos/go-log/log/commands.go new file mode 100644 index 00000000000..f0bf757bf0c --- /dev/null +++ b/third_party/github.com/coreos/go-log/log/commands.go @@ -0,0 +1,214 @@ +package log +// Copyright 2013, David Fisher. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// +// author: David Fisher +// based on previous package by: Cong Ding + +import ( + "fmt" + "os" +) + +var BasicFormat = "%s [%9s] %s- %s\n" +var BasicFields = []string{"time", "priority", "prefix", "message"} +var RichFormat = "%s [%9s] %d %s - %s:%s:%d - %s\n" +var RichFields = []string{"full_time", "priority", "seq", "prefix", "filename", "funcname", "lineno", "message"} + +// This function has an unusual name to aid in finding it while walking the +// stack. We need to do some dead reckoning from this function to access the +// caller's stack, so there is a consistent call depth above this function. +func (logger *Logger) Log(priority Priority, v ...interface{}) { + fields := logger.fieldValues() + fields["priority"] = priority + fields["message"] = fmt.Sprint(v...) + for _, sink := range logger.sinks { + sink.Log(fields) + } +} + +func (logger *Logger) Logf(priority Priority, format string, v ...interface{}) { + logger.Log(priority, fmt.Sprintf(format, v...)) +} + + +func (logger *Logger) Emergency(v ...interface{}) { + logger.Log(PriEmerg, v...) +} +func (logger *Logger) Emergencyf(format string, v ...interface{}) { + logger.Log(PriEmerg, fmt.Sprintf(format, v...)) +} + +func (logger *Logger) Alert(v ...interface{}) { + logger.Log(PriAlert, v...) +} +func (logger *Logger) Alertf(format string, v ...interface{}) { + logger.Log(PriAlert, fmt.Sprintf(format, v...)) +} + +func (logger *Logger) Critical(v ...interface{}) { + logger.Log(PriCrit, v...) +} +func (logger *Logger) Criticalf(format string, v ...interface{}) { + logger.Log(PriCrit, fmt.Sprintf(format, v...)) +} + +func (logger *Logger) Error(v ...interface{}) { + logger.Log(PriErr, v...) +} +func (logger *Logger) Errorf(format string, v ...interface{}) { + logger.Log(PriErr, fmt.Sprintf(format, v...)) +} + +func (logger *Logger) Warning(v ...interface{}) { + logger.Log(PriWarning, v...) +} +func (logger *Logger) Warningf(format string, v ...interface{}) { + logger.Log(PriWarning, fmt.Sprintf(format, v...)) +} + +func (logger *Logger) Notice(v ...interface{}) { + logger.Log(PriNotice, v...) +} +func (logger *Logger) Noticef(format string, v ...interface{}) { + logger.Log(PriNotice, fmt.Sprintf(format, v...)) +} + +func (logger *Logger) Info(v ...interface{}) { + logger.Log(PriInfo, v...) +} +func (logger *Logger) Infof(format string, v ...interface{}) { + logger.Log(PriInfo, fmt.Sprintf(format, v...)) +} + +func (logger *Logger) Debug(v ...interface{}) { + logger.Log(PriDebug, v...) +} +func (logger *Logger) Debugf(format string, v ...interface{}) { + logger.Log(PriDebug, fmt.Sprintf(format, v...)) +} + + +func Emergency(v ...interface{}) { + defaultLogger.Log(PriEmerg, v...) +} +func Emergencyf(format string, v ...interface{}) { + defaultLogger.Log(PriEmerg, fmt.Sprintf(format, v...)) +} + +func Alert(v ...interface{}) { + defaultLogger.Log(PriAlert, v...) +} +func Alertf(format string, v ...interface{}) { + defaultLogger.Log(PriAlert, fmt.Sprintf(format, v...)) +} + +func Critical(v ...interface{}) { + defaultLogger.Log(PriCrit, v...) +} +func Criticalf(format string, v ...interface{}) { + defaultLogger.Log(PriCrit, fmt.Sprintf(format, v...)) +} + +func Error(v ...interface{}) { + defaultLogger.Log(PriErr, v...) +} +func Errorf(format string, v ...interface{}) { + defaultLogger.Log(PriErr, fmt.Sprintf(format, v...)) +} + +func Warning(v ...interface{}) { + defaultLogger.Log(PriWarning, v...) +} +func Warningf(format string, v ...interface{}) { + defaultLogger.Log(PriWarning, fmt.Sprintf(format, v...)) +} + +func Notice(v ...interface{}) { + defaultLogger.Log(PriNotice, v...) +} +func Noticef(format string, v ...interface{}) { + defaultLogger.Log(PriNotice, fmt.Sprintf(format, v...)) +} + +func Info(v ...interface{}) { + defaultLogger.Log(PriInfo, v...) +} +func Infof(format string, v ...interface{}) { + defaultLogger.Log(PriInfo, fmt.Sprintf(format, v...)) +} + +func Debug(v ...interface{}) { + defaultLogger.Log(PriDebug, v...) +} +func Debugf(format string, v ...interface{}) { + defaultLogger.Log(PriDebug, fmt.Sprintf(format, v...)) +} + +// Standard library log functions + +func (logger *Logger)Fatalln (v ...interface{}) { + logger.Log(PriCrit, v...) + os.Exit(1) +} +func (logger *Logger)Fatalf (format string, v ...interface{}) { + logger.Logf(PriCrit, format, v...) + os.Exit(1) +} + +func (logger *Logger)Panicln (v ...interface{}) { + s := fmt.Sprint(v...) + logger.Log(PriErr, s) + panic(s) +} +func (logger *Logger)Panicf (format string, v ...interface{}) { + s := fmt.Sprintf(format, v...) + logger.Log(PriErr, s) + panic(s) +} + +func (logger *Logger)Println (v ...interface{}) { + logger.Log(PriInfo, v...) +} +func (logger *Logger)Printf (format string, v ...interface{}) { + logger.Logf(PriInfo, format, v...) +} + + +func Fatalln (v ...interface{}) { + defaultLogger.Log(PriCrit, v...) + os.Exit(1) +} +func Fatalf (format string, v ...interface{}) { + defaultLogger.Logf(PriCrit, format, v...) + os.Exit(1) +} + +func Panicln (v ...interface{}) { + s := fmt.Sprint(v...) + defaultLogger.Log(PriErr, s) + panic(s) +} +func Panicf (format string, v ...interface{}) { + s := fmt.Sprintf(format, v...) + defaultLogger.Log(PriErr, s) + panic(s) +} + +func Println (v ...interface{}) { + defaultLogger.Log(PriInfo, v...) +} +func Printf (format string, v ...interface{}) { + defaultLogger.Logf(PriInfo, format, v...) +} diff --git a/third_party/github.com/coreos/go-log/log/fields.go b/third_party/github.com/coreos/go-log/log/fields.go new file mode 100644 index 00000000000..c060a225583 --- /dev/null +++ b/third_party/github.com/coreos/go-log/log/fields.go @@ -0,0 +1,69 @@ +package log +// Copyright 2013, David Fisher. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// +// author: David Fisher +// based on previous package by: Cong Ding + +import ( + "os" + "path" + "runtime" + "strings" + "sync/atomic" + "time" +) + +type Fields map[string]interface{} + +func (logger *Logger) fieldValues() Fields { + now := time.Now() + fields := Fields{ + "prefix": logger.prefix, // static field available to all sinks + "seq": logger.nextSeq(), // auto-incrementing sequence number + "start_time": logger.created, // start time of the logger + "time": now.Format(time.StampMilli), // formatted time of log entry + "full_time": now, // time of log entry + "rtime": time.Since(logger.created), // relative time of log entry since started + "pid": os.Getpid(), // process id + "executable": logger.executable, // executable filename + } + + if logger.verbose { + setVerboseFields(fields) + } + return fields +} + +func (logger *Logger) nextSeq() uint64 { + return atomic.AddUint64(&logger.seq, 1) +} + +func setVerboseFields(fields Fields) { + callers := make([]uintptr, 10) + n := runtime.Callers(3, callers) // starts in (*Logger).Log or similar + callers = callers[:n] + + for _, pc := range callers { + f := runtime.FuncForPC(pc) + if !strings.Contains(f.Name(), "logger.(*Logger)") { + fields["funcname"] = f.Name() + pathname, lineno := f.FileLine(pc) + fields["lineno"] = lineno + fields["pathname"] = pathname + fields["filename"] = path.Base(pathname) + return + } + } +} diff --git a/third_party/github.com/coreos/go-log/log/logger.go b/third_party/github.com/coreos/go-log/log/logger.go new file mode 100644 index 00000000000..59ecdabcb40 --- /dev/null +++ b/third_party/github.com/coreos/go-log/log/logger.go @@ -0,0 +1,73 @@ +package log + +// Copyright 2013, David Fisher. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// +// author: David Fisher +// based on previous package by: Cong Ding + +import ( + "bitbucket.org/kardianos/osext" + "os" + "path" + "time" +) + +// Logger is user-immutable immutable struct which can log to several outputs +type Logger struct { + sinks []Sink // the sinks this logger will log to + verbose bool // gather expensive logging data? + prefix string // static field available to all log sinks under this logger + + created time.Time // time when this logger was created + seq uint64 // sequential number of log message, starting at 1 + executable string // executable name +} + +// New creates a new Logger which logs to all the supplied sinks. The prefix +// argument is passed to all loggers under the field "prefix" with every log +// message. If verbose is true, more expensive runtime fields will be computed +// and passed to loggers. These fields are funcname, lineno, pathname, and +// filename. +func New(prefix string, verbose bool, sinks ...Sink) *Logger { + return &Logger{ + sinks: sinks, + verbose: verbose, + prefix: prefix, + + created: time.Now(), + seq: 0, + executable: getExecutableName(), + } +} + +func getExecutableName() string { + executablePath, err := osext.Executable() + if err != nil { + return "(UNKNOWN)" + } else { + return path.Base(executablePath) + } +} + +// NewSimple(sinks...) is equivalent to New("", false, sinks...) +func NewSimple(sinks ...Sink) *Logger { + return New("", false, sinks...) +} + +var defaultLogger *Logger + +func init() { + defaultLogger = NewSimple(CombinedSink(os.Stdout, BasicFormat, BasicFields)) +} diff --git a/third_party/github.com/coreos/go-log/log/priority.go b/third_party/github.com/coreos/go-log/log/priority.go new file mode 100644 index 00000000000..918dcaf44d7 --- /dev/null +++ b/third_party/github.com/coreos/go-log/log/priority.go @@ -0,0 +1,54 @@ +package log +// Copyright 2013, David Fisher. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// +// author: David Fisher +// based on previous package by: Cong Ding + +type Priority int + +const ( + PriEmerg Priority = iota + PriAlert + PriCrit + PriErr + PriWarning + PriNotice + PriInfo + PriDebug +) + +func (priority Priority) String() string { + switch priority { + case PriEmerg: + return "EMERGENCY" + case PriAlert: + return "ALERT" + case PriCrit: + return "CRITICAL" + case PriErr: + return "ERROR" + case PriWarning: + return "WARNING" + case PriNotice: + return "NOTICE" + case PriInfo: + return "INFO" + case PriDebug: + return "DEBUG" + + default: + return "UNKNOWN" + } +} diff --git a/third_party/github.com/coreos/go-log/log/sinks.go b/third_party/github.com/coreos/go-log/log/sinks.go new file mode 100644 index 00000000000..5cda993da84 --- /dev/null +++ b/third_party/github.com/coreos/go-log/log/sinks.go @@ -0,0 +1,154 @@ +package log + +// Copyright 2013, David Fisher. All rights reserved. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// +// author: David Fisher +// based on previous package by: Cong Ding + +import ( + "fmt" + "github.com/coreos/go-systemd/journal" + "io" + "strings" + "sync" +) + +const AsyncBuffer = 100 + +type Sink interface { + Log(Fields) +} + +type nullSink struct{} + +func (sink *nullSink) Log(fields Fields) {} + +func NullSink() Sink { + return &nullSink{} +} + +type writerSink struct { + lock sync.Mutex + out io.Writer + format string + fields []string +} + +func (sink *writerSink) Log(fields Fields) { + vals := make([]interface{}, len(sink.fields)) + for i, field := range sink.fields { + var ok bool + vals[i], ok = fields[field] + if !ok { + vals[i] = "???" + } + } + + sink.lock.Lock() + defer sink.lock.Unlock() + fmt.Fprintf(sink.out, sink.format, vals...) +} + +func WriterSink(out io.Writer, format string, fields []string) Sink { + return &writerSink{ + out: out, + format: format, + fields: fields, + } +} + +type journalSink struct{} + +func (sink *journalSink) Log(fields Fields) { + message := fields["message"].(string) + priority := toJournalPriority(fields["priority"].(Priority)) + journalFields := make(map[string]string) + for k, v := range fields { + if k == "message" || k == "priority" { + continue + } + journalFields[strings.ToUpper(k)] = fmt.Sprint(v) + } + journal.Send(message, priority, journalFields) +} + +func toJournalPriority(priority Priority) journal.Priority { + switch priority { + case PriEmerg: + return journal.PriEmerg + case PriAlert: + return journal.PriAlert + case PriCrit: + return journal.PriCrit + case PriErr: + return journal.PriErr + case PriWarning: + return journal.PriWarning + case PriNotice: + return journal.PriNotice + case PriInfo: + return journal.PriInfo + case PriDebug: + return journal.PriDebug + + default: + return journal.PriErr + } +} + +func JournalSink() Sink { + return &journalSink{} +} + +type combinedSink struct { + sinks []Sink +} + +func (sink *combinedSink) Log(fields Fields) { + for _, s := range sink.sinks { + s.Log(fields) + } +} + +func CombinedSink(writer io.Writer, format string, fields []string) Sink { + sinks := make([]Sink, 0) + sinks = append(sinks, WriterSink(writer, format, fields)) + if journal.Enabled() { + sinks = append(sinks, JournalSink()) + } + + return &combinedSink{ + sinks: sinks, + } +} + +type priorityFilter struct { + priority Priority + target Sink +} + +func (filter *priorityFilter) Log(fields Fields) { + // lower priority values indicate more important messages + if fields["priority"].(Priority) <= filter.priority { + filter.target.Log(fields) + } +} + +func PriorityFilter(priority Priority, target Sink) Sink { + return &priorityFilter{ + priority: priority, + target: target, + } +} diff --git a/third_party/github.com/coreos/go-raft/append_entries_request.go b/third_party/github.com/coreos/go-raft/append_entries_request.go index 78338e4eba6..af56f2a0c48 100644 --- a/third_party/github.com/coreos/go-raft/append_entries_request.go +++ b/third_party/github.com/coreos/go-raft/append_entries_request.go @@ -31,7 +31,7 @@ func newAppendEntriesRequest(term uint64, prevLogIndex uint64, prevLogTerm uint6 // Encodes the AppendEntriesRequest to a buffer. Returns the number of bytes // written and any error that may have occurred. -func (req *AppendEntriesRequest) encode(w io.Writer) (int, error) { +func (req *AppendEntriesRequest) Encode(w io.Writer) (int, error) { protoEntries := make([]*protobuf.ProtoAppendEntriesRequest_ProtoLogEntry, len(req.Entries)) @@ -63,7 +63,7 @@ func (req *AppendEntriesRequest) encode(w io.Writer) (int, error) { // Decodes the AppendEntriesRequest from a buffer. Returns the number of bytes read and // any error that occurs. -func (req *AppendEntriesRequest) decode(r io.Reader) (int, error) { +func (req *AppendEntriesRequest) Decode(r io.Reader) (int, error) { data, err := ioutil.ReadAll(r) if err != nil { diff --git a/third_party/github.com/coreos/go-raft/append_entries_request_test.go b/third_party/github.com/coreos/go-raft/append_entries_request_test.go index ef6732fc466..d8cbce735d0 100644 --- a/third_party/github.com/coreos/go-raft/append_entries_request_test.go +++ b/third_party/github.com/coreos/go-raft/append_entries_request_test.go @@ -10,7 +10,7 @@ func BenchmarkAppendEntriesRequestEncoding(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { var buf bytes.Buffer - req.encode(&buf) + req.Encode(&buf) } b.SetBytes(int64(len(tmp))) } @@ -19,7 +19,7 @@ func BenchmarkAppendEntriesRequestDecoding(b *testing.B) { req, buf := createTestAppendEntriesRequest(2000) b.ResetTimer() for i := 0; i < b.N; i++ { - req.decode(bytes.NewReader(buf)) + req.Decode(bytes.NewReader(buf)) } b.SetBytes(int64(len(buf))) } @@ -34,7 +34,7 @@ func createTestAppendEntriesRequest(entryCount int) (*AppendEntriesRequest, []by req := newAppendEntriesRequest(1, 1, 1, 1, "leader", entries) var buf bytes.Buffer - req.encode(&buf) + req.Encode(&buf) return req, buf.Bytes() } diff --git a/third_party/github.com/coreos/go-raft/append_entries_response.go b/third_party/github.com/coreos/go-raft/append_entries_response.go index e2b02ae87f6..363af46cdd3 100644 --- a/third_party/github.com/coreos/go-raft/append_entries_response.go +++ b/third_party/github.com/coreos/go-raft/append_entries_response.go @@ -30,7 +30,7 @@ func newAppendEntriesResponse(term uint64, success bool, index uint64, commitInd // Encodes the AppendEntriesResponse to a buffer. Returns the number of bytes // written and any error that may have occurred. -func (resp *AppendEntriesResponse) encode(w io.Writer) (int, error) { +func (resp *AppendEntriesResponse) Encode(w io.Writer) (int, error) { pb := &protobuf.ProtoAppendEntriesResponse{ Term: proto.Uint64(resp.Term), Index: proto.Uint64(resp.Index), @@ -47,7 +47,7 @@ func (resp *AppendEntriesResponse) encode(w io.Writer) (int, error) { // Decodes the AppendEntriesResponse from a buffer. Returns the number of bytes read and // any error that occurs. -func (resp *AppendEntriesResponse) decode(r io.Reader) (int, error) { +func (resp *AppendEntriesResponse) Decode(r io.Reader) (int, error) { data, err := ioutil.ReadAll(r) if err != nil { diff --git a/third_party/github.com/coreos/go-raft/append_entries_response_test.go b/third_party/github.com/coreos/go-raft/append_entries_response_test.go index 038dcda7646..f51ead1f8e0 100644 --- a/third_party/github.com/coreos/go-raft/append_entries_response_test.go +++ b/third_party/github.com/coreos/go-raft/append_entries_response_test.go @@ -10,7 +10,7 @@ func BenchmarkAppendEntriesResponseEncoding(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { var buf bytes.Buffer - req.encode(&buf) + req.Encode(&buf) } b.SetBytes(int64(len(tmp))) } @@ -19,7 +19,7 @@ func BenchmarkAppendEntriesResponseDecoding(b *testing.B) { req, buf := createTestAppendEntriesResponse(2000) b.ResetTimer() for i := 0; i < b.N; i++ { - req.decode(bytes.NewReader(buf)) + req.Decode(bytes.NewReader(buf)) } b.SetBytes(int64(len(buf))) } @@ -28,7 +28,7 @@ func createTestAppendEntriesResponse(entryCount int) (*AppendEntriesResponse, [] resp := newAppendEntriesResponse(1, true, 1, 1) var buf bytes.Buffer - resp.encode(&buf) + resp.Encode(&buf) return resp, buf.Bytes() } diff --git a/third_party/github.com/coreos/go-raft/http_transporter.go b/third_party/github.com/coreos/go-raft/http_transporter.go index 7dbcf5a400b..e41fd817f07 100644 --- a/third_party/github.com/coreos/go-raft/http_transporter.go +++ b/third_party/github.com/coreos/go-raft/http_transporter.go @@ -89,7 +89,7 @@ func (t *HTTPTransporter) Install(server *Server, mux HTTPMuxer) { // Sends an AppendEntries RPC to a peer. func (t *HTTPTransporter) SendAppendEntriesRequest(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { var b bytes.Buffer - if _, err := req.encode(&b); err != nil { + if _, err := req.Encode(&b); err != nil { traceln("transporter.ae.encoding.error:", err) return nil } @@ -106,7 +106,7 @@ func (t *HTTPTransporter) SendAppendEntriesRequest(server *Server, peer *Peer, r defer httpResp.Body.Close() resp := &AppendEntriesResponse{} - if _, err = resp.decode(httpResp.Body); err != nil && err != io.EOF { + if _, err = resp.Decode(httpResp.Body); err != nil && err != io.EOF { traceln("transporter.ae.decoding.error:", err) return nil } @@ -117,7 +117,7 @@ func (t *HTTPTransporter) SendAppendEntriesRequest(server *Server, peer *Peer, r // Sends a RequestVote RPC to a peer. func (t *HTTPTransporter) SendVoteRequest(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { var b bytes.Buffer - if _, err := req.encode(&b); err != nil { + if _, err := req.Encode(&b); err != nil { traceln("transporter.rv.encoding.error:", err) return nil } @@ -134,7 +134,7 @@ func (t *HTTPTransporter) SendVoteRequest(server *Server, peer *Peer, req *Reque defer httpResp.Body.Close() resp := &RequestVoteResponse{} - if _, err = resp.decode(httpResp.Body); err != nil && err != io.EOF { + if _, err = resp.Decode(httpResp.Body); err != nil && err != io.EOF { traceln("transporter.rv.decoding.error:", err) return nil } @@ -162,13 +162,13 @@ func (t *HTTPTransporter) appendEntriesHandler(server *Server) http.HandlerFunc traceln(server.Name(), "RECV /appendEntries") req := &AppendEntriesRequest{} - if _, err := req.decode(r.Body); err != nil { + if _, err := req.Decode(r.Body); err != nil { http.Error(w, "", http.StatusBadRequest) return } resp := server.AppendEntries(req) - if _, err := resp.encode(w); err != nil { + if _, err := resp.Encode(w); err != nil { http.Error(w, "", http.StatusInternalServerError) return } @@ -181,13 +181,13 @@ func (t *HTTPTransporter) requestVoteHandler(server *Server) http.HandlerFunc { traceln(server.Name(), "RECV /requestVote") req := &RequestVoteRequest{} - if _, err := req.decode(r.Body); err != nil { + if _, err := req.Decode(r.Body); err != nil { http.Error(w, "", http.StatusBadRequest) return } resp := server.RequestVote(req) - if _, err := resp.encode(w); err != nil { + if _, err := resp.Encode(w); err != nil { http.Error(w, "", http.StatusInternalServerError) return } diff --git a/third_party/github.com/coreos/go-raft/log.go b/third_party/github.com/coreos/go-raft/log.go index b686d317ca2..5733c2188f2 100644 --- a/third_party/github.com/coreos/go-raft/log.go +++ b/third_party/github.com/coreos/go-raft/log.go @@ -180,26 +180,23 @@ func (l *Log) open(path string) error { } break } - - // Append entry. - l.entries = append(l.entries, entry) - - if entry.Index <= l.commitIndex { - command, err := newCommand(entry.CommandName, entry.Command) - if err != nil { - continue + if entry.Index > l.startIndex { + // Append entry. + l.entries = append(l.entries, entry) + if entry.Index <= l.commitIndex { + command, err := newCommand(entry.CommandName, entry.Command) + if err != nil { + continue + } + l.ApplyFunc(command) } - l.ApplyFunc(command) + debugln("open.log.append log index ", entry.Index) } - debugln("open.log.append log index ", entry.Index) - readBytes += int64(n) } l.results = make([]*logResult, len(l.entries)) - l.compact(l.startIndex, l.startTerm) - debugln("open.log.recovery number of log ", len(l.entries)) return nil } @@ -273,6 +270,8 @@ func (l *Log) getEntriesAfter(index uint64, maxLogEntriesPerRequest uint64) ([]* entries := l.entries[index-l.startIndex:] length := len(entries) + traceln("log.entriesAfter: startIndex:", l.startIndex, " lenght", len(l.entries)) + if uint64(length) < maxLogEntriesPerRequest { // Determine the term at the given entry and return a subslice. return entries, l.entries[index-1-l.startIndex].Term @@ -353,7 +352,10 @@ func (l *Log) lastInfo() (index uint64, term uint64) { func (l *Log) updateCommitIndex(index uint64) { l.mutex.Lock() defer l.mutex.Unlock() - l.commitIndex = index + if index > l.commitIndex { + l.commitIndex = index + } + debugln("update.commit.index ", index) } // Updates the commit index and writes entries after that index to the stable storage. diff --git a/third_party/github.com/coreos/go-raft/peer.go b/third_party/github.com/coreos/go-raft/peer.go index 37b8c3fb73f..7b116edbb40 100644 --- a/third_party/github.com/coreos/go-raft/peer.go +++ b/third_party/github.com/coreos/go-raft/peer.go @@ -255,6 +255,12 @@ func (p *Peer) sendSnapshotRecoveryRequest() { req := newSnapshotRecoveryRequest(p.server.name, p.server.lastSnapshot) debugln("peer.snap.recovery.send: ", p.Name) resp := p.server.Transporter().SendSnapshotRecoveryRequest(p.server, p, req) + + if resp == nil { + debugln("peer.snap.recovery.timeout: ", p.Name) + return + } + if resp.Success { p.prevLogIndex = req.LastIndex } else { diff --git a/third_party/github.com/coreos/go-raft/request_vote_request.go b/third_party/github.com/coreos/go-raft/request_vote_request.go index a7571d8b3b6..c4e5ac697bf 100644 --- a/third_party/github.com/coreos/go-raft/request_vote_request.go +++ b/third_party/github.com/coreos/go-raft/request_vote_request.go @@ -28,7 +28,7 @@ func newRequestVoteRequest(term uint64, candidateName string, lastLogIndex uint6 // Encodes the RequestVoteRequest to a buffer. Returns the number of bytes // written and any error that may have occurred. -func (req *RequestVoteRequest) encode(w io.Writer) (int, error) { +func (req *RequestVoteRequest) Encode(w io.Writer) (int, error) { pb := &protobuf.ProtoRequestVoteRequest{ Term: proto.Uint64(req.Term), LastLogIndex: proto.Uint64(req.LastLogIndex), @@ -45,7 +45,7 @@ func (req *RequestVoteRequest) encode(w io.Writer) (int, error) { // Decodes the RequestVoteRequest from a buffer. Returns the number of bytes read and // any error that occurs. -func (req *RequestVoteRequest) decode(r io.Reader) (int, error) { +func (req *RequestVoteRequest) Decode(r io.Reader) (int, error) { data, err := ioutil.ReadAll(r) if err != nil { diff --git a/third_party/github.com/coreos/go-raft/request_vote_response.go b/third_party/github.com/coreos/go-raft/request_vote_response.go index 9ed1bc9b94b..1870245a1e4 100644 --- a/third_party/github.com/coreos/go-raft/request_vote_response.go +++ b/third_party/github.com/coreos/go-raft/request_vote_response.go @@ -24,7 +24,7 @@ func newRequestVoteResponse(term uint64, voteGranted bool) *RequestVoteResponse // Encodes the RequestVoteResponse to a buffer. Returns the number of bytes // written and any error that may have occurred. -func (resp *RequestVoteResponse) encode(w io.Writer) (int, error) { +func (resp *RequestVoteResponse) Encode(w io.Writer) (int, error) { pb := &protobuf.ProtoRequestVoteResponse{ Term: proto.Uint64(resp.Term), VoteGranted: proto.Bool(resp.VoteGranted), @@ -40,7 +40,7 @@ func (resp *RequestVoteResponse) encode(w io.Writer) (int, error) { // Decodes the RequestVoteResponse from a buffer. Returns the number of bytes read and // any error that occurs. -func (resp *RequestVoteResponse) decode(r io.Reader) (int, error) { +func (resp *RequestVoteResponse) Decode(r io.Reader) (int, error) { data, err := ioutil.ReadAll(r) if err != nil { diff --git a/third_party/github.com/coreos/go-raft/server.go b/third_party/github.com/coreos/go-raft/server.go index b4dab92ae2b..645a2e4a069 100644 --- a/third_party/github.com/coreos/go-raft/server.go +++ b/third_party/github.com/coreos/go-raft/server.go @@ -80,6 +80,8 @@ type Server struct { lastSnapshot *Snapshot stateMachine StateMachine maxLogEntriesPerRequest uint64 + + connectionString string } // An event to be processed by the server's event loop. @@ -96,7 +98,7 @@ type event struct { //------------------------------------------------------------------------------ // Creates a new server with a log at the given path. -func NewServer(name string, path string, transporter Transporter, stateMachine StateMachine, context interface{}) (*Server, error) { +func NewServer(name string, path string, transporter Transporter, stateMachine StateMachine, context interface{}, connectionString string) (*Server, error) { if name == "" { return nil, errors.New("raft.Server: Name cannot be blank") } @@ -117,6 +119,7 @@ func NewServer(name string, path string, transporter Transporter, stateMachine S electionTimeout: DefaultElectionTimeout, heartbeatTimeout: DefaultHeartbeatTimeout, maxLogEntriesPerRequest: MaxLogEntriesPerRequest, + connectionString: connectionString, } // Setup apply function. @@ -1009,10 +1012,17 @@ func (s *Server) TakeSnapshot() error { state = []byte{0} } - var peers []*Peer + peers := make([]*Peer, len(s.peers)+1) + i := 0 for _, peer := range s.peers { - peers = append(peers, peer.clone()) + peers[i] = peer.clone() + i++ + } + + peers[i] = &Peer{ + Name: s.Name(), + ConnectionString: s.connectionString, } s.currentSnapshot = &Snapshot{lastIndex, lastTerm, peers, state, path} @@ -1253,7 +1263,7 @@ func (s *Server) readConf() error { return err } - s.log.commitIndex = conf.CommitIndex + s.log.updateCommitIndex(conf.CommitIndex) return nil } diff --git a/third_party/github.com/coreos/go-raft/server_test.go b/third_party/github.com/coreos/go-raft/server_test.go index 2a15599701d..01bc96b7159 100644 --- a/third_party/github.com/coreos/go-raft/server_test.go +++ b/third_party/github.com/coreos/go-raft/server_test.go @@ -428,7 +428,7 @@ func TestServerRecoverFromPreviousLogAndConf(t *testing.T) { for _, name := range names { server := servers[name] if server.CommitIndex() != 17 { - t.Fatalf("%s commitIndex is invalid [%d/%d]", name, server.CommitIndex(), 16) + t.Fatalf("%s commitIndex is invalid [%d/%d]", name, server.CommitIndex(), 17) } server.Stop() } diff --git a/third_party/github.com/coreos/go-raft/snapshot_recovery_request.go b/third_party/github.com/coreos/go-raft/snapshot_recovery_request.go index 57b3e3a8891..a05f4310815 100644 --- a/third_party/github.com/coreos/go-raft/snapshot_recovery_request.go +++ b/third_party/github.com/coreos/go-raft/snapshot_recovery_request.go @@ -35,7 +35,7 @@ func newSnapshotRecoveryRequest(leaderName string, snapshot *Snapshot) *Snapshot // Encodes the SnapshotRecoveryRequest to a buffer. Returns the number of bytes // written and any error that may have occurred. -func (req *SnapshotRecoveryRequest) encode(w io.Writer) (int, error) { +func (req *SnapshotRecoveryRequest) Encode(w io.Writer) (int, error) { protoPeers := make([]*protobuf.ProtoSnapshotRecoveryRequest_ProtoPeer, len(req.Peers)) @@ -63,7 +63,7 @@ func (req *SnapshotRecoveryRequest) encode(w io.Writer) (int, error) { // Decodes the SnapshotRecoveryRequest from a buffer. Returns the number of bytes read and // any error that occurs. -func (req *SnapshotRecoveryRequest) decode(r io.Reader) (int, error) { +func (req *SnapshotRecoveryRequest) Decode(r io.Reader) (int, error) { data, err := ioutil.ReadAll(r) if err != nil { diff --git a/third_party/github.com/coreos/go-raft/snapshot_recovery_response.go b/third_party/github.com/coreos/go-raft/snapshot_recovery_response.go index 2b2f1cde176..7e4d86acee4 100644 --- a/third_party/github.com/coreos/go-raft/snapshot_recovery_response.go +++ b/third_party/github.com/coreos/go-raft/snapshot_recovery_response.go @@ -31,7 +31,7 @@ func newSnapshotRecoveryResponse(term uint64, success bool, commitIndex uint64) // Encodes the SnapshotRecoveryResponse to a buffer. Returns the number of bytes // written and any error that may have occurred. -func (req *SnapshotRecoveryResponse) encode(w io.Writer) (int, error) { +func (req *SnapshotRecoveryResponse) Encode(w io.Writer) (int, error) { pb := &protobuf.ProtoSnapshotRecoveryResponse{ Term: proto.Uint64(req.Term), Success: proto.Bool(req.Success), @@ -47,7 +47,7 @@ func (req *SnapshotRecoveryResponse) encode(w io.Writer) (int, error) { // Decodes the SnapshotRecoveryResponse from a buffer. Returns the number of bytes read and // any error that occurs. -func (req *SnapshotRecoveryResponse) decode(r io.Reader) (int, error) { +func (req *SnapshotRecoveryResponse) Decode(r io.Reader) (int, error) { data, err := ioutil.ReadAll(r) if err != nil { diff --git a/third_party/github.com/coreos/go-raft/snapshot_request.go b/third_party/github.com/coreos/go-raft/snapshot_request.go index c2f2cc768d6..3d75a52cc4f 100644 --- a/third_party/github.com/coreos/go-raft/snapshot_request.go +++ b/third_party/github.com/coreos/go-raft/snapshot_request.go @@ -31,7 +31,7 @@ func newSnapshotRequest(leaderName string, snapshot *Snapshot) *SnapshotRequest // Encodes the SnapshotRequest to a buffer. Returns the number of bytes // written and any error that may have occurred. -func (req *SnapshotRequest) encode(w io.Writer) (int, error) { +func (req *SnapshotRequest) Encode(w io.Writer) (int, error) { pb := &protobuf.ProtoSnapshotRequest{ LeaderName: proto.String(req.LeaderName), LastIndex: proto.Uint64(req.LastIndex), @@ -47,7 +47,7 @@ func (req *SnapshotRequest) encode(w io.Writer) (int, error) { // Decodes the SnapshotRequest from a buffer. Returns the number of bytes read and // any error that occurs. -func (req *SnapshotRequest) decode(r io.Reader) (int, error) { +func (req *SnapshotRequest) Decode(r io.Reader) (int, error) { data, err := ioutil.ReadAll(r) if err != nil { diff --git a/third_party/github.com/coreos/go-raft/snapshot_response.go b/third_party/github.com/coreos/go-raft/snapshot_response.go index 2e6c1c518b9..bd27f67afab 100644 --- a/third_party/github.com/coreos/go-raft/snapshot_response.go +++ b/third_party/github.com/coreos/go-raft/snapshot_response.go @@ -27,7 +27,7 @@ func newSnapshotResponse(success bool) *SnapshotResponse { // Encodes the SnapshotResponse to a buffer. Returns the number of bytes // written and any error that may have occurred. -func (resp *SnapshotResponse) encode(w io.Writer) (int, error) { +func (resp *SnapshotResponse) Encode(w io.Writer) (int, error) { pb := &protobuf.ProtoSnapshotResponse{ Success: proto.Bool(resp.Success), } @@ -41,7 +41,7 @@ func (resp *SnapshotResponse) encode(w io.Writer) (int, error) { // Decodes the SnapshotResponse from a buffer. Returns the number of bytes read and // any error that occurs. -func (resp *SnapshotResponse) decode(r io.Reader) (int, error) { +func (resp *SnapshotResponse) Decode(r io.Reader) (int, error) { data, err := ioutil.ReadAll(r) if err != nil { diff --git a/third_party/github.com/coreos/go-raft/test.go b/third_party/github.com/coreos/go-raft/test.go index 025cf0f5879..95a6c7168bd 100644 --- a/third_party/github.com/coreos/go-raft/test.go +++ b/third_party/github.com/coreos/go-raft/test.go @@ -65,12 +65,12 @@ func newTestServer(name string, transporter Transporter) *Server { if err := os.MkdirAll(p, 0644); err != nil { panic(err.Error()) } - server, _ := NewServer(name, p, transporter, nil, nil) + server, _ := NewServer(name, p, transporter, nil, nil, "") return server } func newTestServerWithPath(name string, transporter Transporter, p string) *Server { - server, _ := NewServer(name, p, transporter, nil, nil) + server, _ := NewServer(name, p, transporter, nil, nil, "") return server } diff --git a/third_party/github.com/ccding/go-config-reader/LICENSE b/third_party/github.com/coreos/go-systemd/LICENSE similarity index 100% rename from third_party/github.com/ccding/go-config-reader/LICENSE rename to third_party/github.com/coreos/go-systemd/LICENSE diff --git a/third_party/github.com/coreos/go-systemd/README.md b/third_party/github.com/coreos/go-systemd/README.md new file mode 100644 index 00000000000..9b8a2f83ff5 --- /dev/null +++ b/third_party/github.com/coreos/go-systemd/README.md @@ -0,0 +1,3 @@ +# go-systemd + +Go bindings to systemd socket activation, journal and D-BUS APIs. diff --git a/third_party/github.com/coreos/go-systemd/activation/files.go b/third_party/github.com/coreos/go-systemd/activation/files.go new file mode 100644 index 00000000000..83cd04fef4b --- /dev/null +++ b/third_party/github.com/coreos/go-systemd/activation/files.go @@ -0,0 +1,29 @@ +package activation + +import ( + "os" + "strconv" + "syscall" +) + +// based on: https://gist.github.com/alberts/4640792 +const ( + listenFdsStart = 3 +) + +func Files() []*os.File { + pid, err := strconv.Atoi(os.Getenv("LISTEN_PID")) + if err != nil || pid != os.Getpid() { + return nil + } + nfds, err := strconv.Atoi(os.Getenv("LISTEN_FDS")) + if err != nil || nfds == 0 { + return nil + } + files := []*os.File(nil) + for fd := listenFdsStart; fd < listenFdsStart+nfds; fd++ { + syscall.CloseOnExec(fd) + files = append(files, os.NewFile(uintptr(fd), "LISTEN_FD_" + strconv.Itoa(fd))) + } + return files +} diff --git a/third_party/github.com/coreos/go-systemd/journal/send.go b/third_party/github.com/coreos/go-systemd/journal/send.go new file mode 100644 index 00000000000..51caa11cf04 --- /dev/null +++ b/third_party/github.com/coreos/go-systemd/journal/send.go @@ -0,0 +1,148 @@ +// Package journal provides write bindings to the systemd journal +package journal + +import ( + "bytes" + "errors" + "fmt" + "io" + "io/ioutil" + "net" + "os" + "strconv" + "strings" + "syscall" + "encoding/binary" +) + +// Priority of a journal message +type Priority int + +const ( + PriEmerg Priority = iota + PriAlert + PriCrit + PriErr + PriWarning + PriNotice + PriInfo + PriDebug +) + +var conn net.Conn + +func init() { + conn, _ = net.Dial("unixgram", "/run/systemd/journal/socket") +} + +// Enabled returns true iff the systemd journal is available for logging +func Enabled() bool { + return conn != nil +} + +// Send a message to the systemd journal. vars is a map of journald fields to +// values. Fields must be composed of uppercase letters, numbers, and +// underscores, but must not start with an underscore. Within these +// restrictions, any arbitrary field name may be used. Some names have special +// significance: see the journalctl documentation +// (http://www.freedesktop.org/software/systemd/man/systemd.journal-fields.html) +// for more details. vars may be nil. +func Send(message string, priority Priority, vars map[string]string) error { + if conn == nil { + return journalError("could not connect to journald socket") + } + + data := new(bytes.Buffer) + appendVariable(data, "PRIORITY", strconv.Itoa(int(priority))) + appendVariable(data, "MESSAGE", message) + for k, v := range vars { + appendVariable(data, k, v) + } + + _, err := io.Copy(conn, data) + if err != nil && isSocketSpaceError(err) { + file, err := tempFd() + if err != nil { + return journalError(err.Error()) + } + _, err = io.Copy(file, data) + if err != nil { + return journalError(err.Error()) + } + + rights := syscall.UnixRights(int(file.Fd())) + + /* this connection should always be a UnixConn, but better safe than sorry */ + unixConn, ok := conn.(*net.UnixConn) + if !ok { + return journalError("can't send file through non-Unix connection") + } + unixConn.WriteMsgUnix([]byte{}, rights, nil) + } else if err != nil { + return journalError(err.Error()) + } + return nil +} + +func appendVariable(w io.Writer, name, value string) { + if !validVarName(name) { + journalError("variable name contains invalid character, ignoring") + } + if strings.ContainsRune(value, '\n') { + /* When the value contains a newline, we write: + * - the variable name, followed by a newline + * - the size (in 64bit little endian format) + * - the data, followed by a newline + */ + fmt.Fprintln(w, name) + binary.Write(w, binary.LittleEndian, uint64(len(value))) + fmt.Fprintln(w, value) + } else { + /* just write the variable and value all on one line */ + fmt.Fprintf(w, "%s=%s\n", name, value) + } +} + +func validVarName(name string) bool { + /* The variable name must be in uppercase and consist only of characters, + * numbers and underscores, and may not begin with an underscore. (from the docs) + */ + valid := true + valid = valid && name[0] != '_' + for _, c := range name { + valid = valid && ('A' <= c && c <= 'Z') || ('0' <= c && c <= '9') || c == '_' + } + return valid +} + +func isSocketSpaceError(err error) bool { + opErr, ok := err.(*net.OpError) + if !ok { + return false + } + + sysErr, ok := opErr.Err.(syscall.Errno) + if !ok { + return false + } + + return sysErr == syscall.EMSGSIZE || sysErr == syscall.ENOBUFS +} + +func tempFd() (*os.File, error) { + file, err := ioutil.TempFile("/dev/shm/", "journal.XXXXX") + if err != nil { + return nil, err + } + syscall.Unlink(file.Name()) + if err != nil { + return nil, err + } + return file, nil +} + +func journalError(s string) error { + s = "journal error: " + s + fmt.Fprintln(os.Stderr, s) + return errors.New(s) +} diff --git a/transporter.go b/transporter.go index 461741ce6f1..eb430837712 100644 --- a/transporter.go +++ b/transporter.go @@ -13,26 +13,33 @@ import ( "github.com/coreos/go-raft" ) +// Timeout for setup internal raft http connection +// This should not exceed 3 * RTT +var dailTimeout = 3 * HeartbeatTimeout + +// Timeout for setup internal raft http connection + receive response header +// This should not exceed 3 * RTT + RTT +var responseHeaderTimeout = 4 * HeartbeatTimeout + +// Timeout for receiving the response body from the server +// This should not exceed election timeout +var tranTimeout = ElectionTimeout + // Transporter layer for communication between raft nodes type transporter struct { - client *http.Client - timeout time.Duration -} - -// response struct -type transporterResponse struct { - resp *http.Response - err error + client *http.Client + transport *http.Transport } // Create transporter using by raft server // Create http or https transporter based on // whether the user give the server cert and key -func newTransporter(scheme string, tlsConf tls.Config, timeout time.Duration) *transporter { +func newTransporter(scheme string, tlsConf tls.Config) *transporter { t := transporter{} tr := &http.Transport{ - Dial: dialTimeout, + Dial: dialWithTimeout, + ResponseHeaderTimeout: responseHeaderTimeout, } if scheme == "https" { @@ -41,14 +48,14 @@ func newTransporter(scheme string, tlsConf tls.Config, timeout time.Duration) *t } t.client = &http.Client{Transport: tr} - t.timeout = timeout + t.transport = tr return &t } // Dial with timeout -func dialTimeout(network, addr string) (net.Conn, error) { - return net.DialTimeout(network, addr, HTTPTimeout) +func dialWithTimeout(network, addr string) (net.Conn, error) { + return net.DialTimeout(network, addr, dailTimeout) } // Sends AppendEntries RPCs to a peer when the server is the leader. @@ -66,29 +73,36 @@ func (t *transporter) SendAppendEntriesRequest(server *raft.Server, peer *raft.P debugf("Send LogEntries to %s ", u) - thisPeerStats, ok := r.peersStats[peer.Name] + thisFollowerStats, ok := r.followersStats.Followers[peer.Name] + + if !ok { //this is the first time this follower has been seen + thisFollowerStats = &raftFollowerStats{} + thisFollowerStats.Latency.Minimum = 1 << 63 + r.followersStats.Followers[peer.Name] = thisFollowerStats + } start := time.Now() - resp, err := t.Post(fmt.Sprintf("%s/log/append", u), &b) + resp, httpRequest, err := t.Post(fmt.Sprintf("%s/log/append", u), &b) end := time.Now() if err != nil { debugf("Cannot send AppendEntriesRequest to %s: %s", u, err) if ok { - thisPeerStats.Fail() + thisFollowerStats.Fail() } } else { if ok { - thisPeerStats.Succ(end.Sub(start)) + thisFollowerStats.Succ(end.Sub(start)) } } - r.peersStats[peer.Name] = thisPeerStats - if resp != nil { defer resp.Body.Close() + + t.CancelWhenTimeout(httpRequest) + aersp = &raft.AppendEntriesResponse{} if err := json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { return aersp @@ -108,7 +122,7 @@ func (t *transporter) SendVoteRequest(server *raft.Server, peer *raft.Peer, req u, _ := nameToRaftURL(peer.Name) debugf("Send Vote to %s", u) - resp, err := t.Post(fmt.Sprintf("%s/vote", u), &b) + resp, httpRequest, err := t.Post(fmt.Sprintf("%s/vote", u), &b) if err != nil { debugf("Cannot send VoteRequest to %s : %s", u, err) @@ -116,6 +130,9 @@ func (t *transporter) SendVoteRequest(server *raft.Server, peer *raft.Peer, req if resp != nil { defer resp.Body.Close() + + t.CancelWhenTimeout(httpRequest) + rvrsp := &raft.RequestVoteResponse{} if err := json.NewDecoder(resp.Body).Decode(&rvrsp); err == nil || err == io.EOF { return rvrsp @@ -135,7 +152,7 @@ func (t *transporter) SendSnapshotRequest(server *raft.Server, peer *raft.Peer, debugf("Send Snapshot to %s [Last Term: %d, LastIndex %d]", u, req.LastTerm, req.LastIndex) - resp, err := t.Post(fmt.Sprintf("%s/snapshot", u), &b) + resp, httpRequest, err := t.Post(fmt.Sprintf("%s/snapshot", u), &b) if err != nil { debugf("Cannot send SendSnapshotRequest to %s : %s", u, err) @@ -143,6 +160,9 @@ func (t *transporter) SendSnapshotRequest(server *raft.Server, peer *raft.Peer, if resp != nil { defer resp.Body.Close() + + t.CancelWhenTimeout(httpRequest) + aersp = &raft.SnapshotResponse{} if err = json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { @@ -163,7 +183,7 @@ func (t *transporter) SendSnapshotRecoveryRequest(server *raft.Server, peer *raf debugf("Send SnapshotRecovery to %s [Last Term: %d, LastIndex %d]", u, req.LastTerm, req.LastIndex) - resp, err := t.Post(fmt.Sprintf("%s/snapshotRecovery", u), &b) + resp, _, err := t.Post(fmt.Sprintf("%s/snapshotRecovery", u), &b) if err != nil { debugf("Cannot send SendSnapshotRecoveryRequest to %s : %s", u, err) @@ -172,6 +192,7 @@ func (t *transporter) SendSnapshotRecoveryRequest(server *raft.Server, peer *raf if resp != nil { defer resp.Body.Close() aersp = &raft.SnapshotRecoveryResponse{} + if err = json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { return aersp } @@ -181,46 +202,30 @@ func (t *transporter) SendSnapshotRecoveryRequest(server *raft.Server, peer *raf } // Send server side POST request -func (t *transporter) Post(path string, body io.Reader) (*http.Response, error) { +func (t *transporter) Post(urlStr string, body io.Reader) (*http.Response, *http.Request, error) { - c := make(chan *transporterResponse, 1) + req, _ := http.NewRequest("POST", urlStr, body) - go func() { - tr := new(transporterResponse) - tr.resp, tr.err = t.client.Post(path, "application/json", body) - c <- tr - }() + resp, err := t.client.Do(req) - return t.waitResponse(c) + return resp, req, err } // Send server side GET request -func (t *transporter) Get(path string) (*http.Response, error) { +func (t *transporter) Get(urlStr string) (*http.Response, *http.Request, error) { - c := make(chan *transporterResponse, 1) + req, _ := http.NewRequest("GET", urlStr, nil) - go func() { - tr := new(transporterResponse) - tr.resp, tr.err = t.client.Get(path) - c <- tr - }() + resp, err := t.client.Do(req) - return t.waitResponse(c) + return resp, req, err } -func (t *transporter) waitResponse(responseChan chan *transporterResponse) (*http.Response, error) { - - timeoutChan := time.After(t.timeout) - - select { - case <-timeoutChan: - return nil, fmt.Errorf("Wait Response Timeout: %v", t.timeout) - - case r := <-responseChan: - return r.resp, r.err - } - - // for complier - return nil, nil +// Cancel the on fly HTTP transaction when timeout happens +func (t *transporter) CancelWhenTimeout(req *http.Request) { + go func() { + time.Sleep(ElectionTimeout) + t.transport.CancelRequest(req) + }() } diff --git a/transporter_test.go b/transporter_test.go index e440a094f1a..8c71325c68e 100644 --- a/transporter_test.go +++ b/transporter_test.go @@ -2,33 +2,58 @@ package main import ( "crypto/tls" + "fmt" + "io/ioutil" + "net/http" "testing" "time" ) func TestTransporterTimeout(t *testing.T) { + http.HandleFunc("/timeout", func(w http.ResponseWriter, r *http.Request) { + fmt.Fprintf(w, "timeout") + w.(http.Flusher).Flush() // send headers and some body + time.Sleep(time.Second * 100) + }) + + go http.ListenAndServe(":8080", nil) + conf := tls.Config{} - ts := newTransporter("http", conf, time.Second) + ts := newTransporter("http", conf) ts.Get("http://google.com") - _, err := ts.Get("http://google.com:9999") // it doesn't exisit - if err == nil || err.Error() != "Wait Response Timeout: 1s" { - t.Fatal("timeout error: ", err.Error()) + _, _, err := ts.Get("http://google.com:9999") + if err == nil { + t.Fatal("timeout error") + } + + res, req, err := ts.Get("http://localhost:8080/timeout") + + if err != nil { + t.Fatal("should not timeout") + } + + ts.CancelWhenTimeout(req) + + body, err := ioutil.ReadAll(res.Body) + if err == nil { + fmt.Println(string(body)) + t.Fatal("expected an error reading the body") } - _, err = ts.Post("http://google.com:9999", nil) // it doesn't exisit - if err == nil || err.Error() != "Wait Response Timeout: 1s" { - t.Fatal("timeout error: ", err.Error()) + _, _, err = ts.Post("http://google.com:9999", nil) + if err == nil { + t.Fatal("timeout error") } - _, err = ts.Get("http://www.google.com") + _, _, err = ts.Get("http://www.google.com") if err != nil { - t.Fatal("get error") + t.Fatal("get error: ", err.Error()) } - _, err = ts.Post("http://www.google.com", nil) + _, _, err = ts.Post("http://www.google.com", nil) if err != nil { t.Fatal("post error") } diff --git a/util.go b/util.go index 434de1a2478..2eb360baf12 100644 --- a/util.go +++ b/util.go @@ -4,7 +4,6 @@ import ( "encoding/json" "fmt" "io" - "log" "net" "net/http" "net/url" @@ -17,6 +16,7 @@ import ( etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/store" "github.com/coreos/etcd/web" + "github.com/coreos/go-log/log" "github.com/coreos/go-raft" ) @@ -186,44 +186,39 @@ func getNodePath(urlPath string) string { // Log //-------------------------------------- -var logger *log.Logger +var logger *log.Logger = log.New("etcd", false, + log.CombinedSink(os.Stdout, "[%s] %s %-9s | %s\n", []string{"prefix", "time", "priority", "message"})) -func init() { - logger = log.New(os.Stdout, "[etcd] ", log.Lmicroseconds) +func infof(format string, v ...interface{}) { + logger.Infof(format, v...) } -func infof(msg string, v ...interface{}) { - logger.Printf("INFO "+msg+"\n", v...) -} - -func debugf(msg string, v ...interface{}) { +func debugf(format string, v ...interface{}) { if verbose { - logger.Printf("DEBUG "+msg+"\n", v...) + logger.Debugf(format, v...) } } func debug(v ...interface{}) { if verbose { - logger.Println("DEBUG " + fmt.Sprint(v...)) + logger.Debug(v...) } } -func warnf(msg string, v ...interface{}) { - logger.Printf("WARN "+msg+"\n", v...) +func warnf(format string, v ...interface{}) { + logger.Warningf(format, v...) } func warn(v ...interface{}) { - logger.Println("WARN " + fmt.Sprint(v...)) + logger.Warning(v...) } -func fatalf(msg string, v ...interface{}) { - logger.Printf("FATAL "+msg+"\n", v...) - os.Exit(1) +func fatalf(format string, v ...interface{}) { + logger.Fatalf(format, v...) } func fatal(v ...interface{}) { - logger.Println("FATAL " + fmt.Sprint(v...)) - os.Exit(1) + logger.Fatalln(v...) } //-------------------------------------- From 33e010ebd86591feb5bd1588109838194db14776 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 28 Sep 2013 17:41:02 -0700 Subject: [PATCH 045/247] add watchHistory clone --- store/event.go | 22 ++++++++++++++++++++++ store/store_test.go | 6 +++--- store/watcher.go | 6 +++++- 3 files changed, 30 insertions(+), 4 deletions(-) diff --git a/store/event.go b/store/event.go index ca7b0389b56..f4085358d49 100644 --- a/store/event.go +++ b/store/event.go @@ -151,3 +151,25 @@ func (eh *EventHistory) scan(prefix string, index uint64) (*Event, error) { } } + +// clone will be protected by a stop-world lock +// do not need to obtain internal lock +func (eh *EventHistory) clone() *EventHistory { + + clonedQueue := eventQueue{ + Capacity: eh.Queue.Capacity, + Events: make([]*Event, eh.Queue.Capacity), + Size: eh.Queue.Size, + Front: eh.Queue.Front, + } + + for i, e := range eh.Queue.Events { + clonedQueue.Events[i] = e + } + + return &EventHistory{ + StartIndex: eh.StartIndex, + Queue: clonedQueue, + } + +} diff --git a/store/store_test.go b/store/store_test.go index 259310ff41a..253063a2a00 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -247,7 +247,7 @@ func TestExpire(t *testing.T) { s.Create("/foo", "bar", expire, 1, 1) - _, err := s.InternalGet("/foo", 1, 1) + _, err := s.internalGet("/foo", 1, 1) if err != nil { t.Fatalf("can not get the node") @@ -255,7 +255,7 @@ func TestExpire(t *testing.T) { time.Sleep(time.Second * 2) - _, err = s.InternalGet("/foo", 1, 1) + _, err = s.internalGet("/foo", 1, 1) if err == nil { t.Fatalf("can get the node after expiration time") @@ -266,7 +266,7 @@ func TestExpire(t *testing.T) { s.Create("/foo", "bar", expire, 1, 1) time.Sleep(time.Millisecond * 50) - _, err = s.InternalGet("/foo", 1, 1) + _, err = s.internalGet("/foo", 1, 1) if err != nil { t.Fatalf("cannot get the node before expiration", err.Error()) diff --git a/store/watcher.go b/store/watcher.go index 81ae4969a50..3b3e43478c2 100644 --- a/store/watcher.go +++ b/store/watcher.go @@ -118,5 +118,9 @@ func (wh *watcherHub) notify(e *Event) { } func (wh *watcherHub) clone() *watcherHub { - return &watcherHub{} + clonedHistory := wh.EventHistory.clone() + + return &watcherHub{ + EventHistory: clonedHistory, + } } From 4b2e53f29efa272826850fd196655ff151f95143 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 28 Sep 2013 17:41:45 -0700 Subject: [PATCH 046/247] minor clean up --- store/store.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/store/store.go b/store/store.go index 7c002c8be68..6265d291c0c 100644 --- a/store/store.go +++ b/store/store.go @@ -374,10 +374,10 @@ func (s *Store) Save() ([]byte, error) { s.worldLock.Lock() clonedStore := New() - clonedStore.Root = s.Root.Clone() - clonedStore.WatcherHub = s.WatcherHub.clone() clonedStore.Index = s.Index clonedStore.Term = s.Term + clonedStore.Root = s.Root.Clone() + clonedStore.WatcherHub = s.WatcherHub.clone() clonedStore.Stats = s.Stats.clone() s.worldLock.Unlock() From 35724319c9c12a68cbb163e8496185bb6306428d Mon Sep 17 00:00:00 2001 From: evan-gu Date: Mon, 30 Sep 2013 01:06:18 -0400 Subject: [PATCH 047/247] add watcher for expiration, add expiration for TestAndSet, add related test case --- store/event.go | 19 +++++++++++++++++-- store/node.go | 22 ++++++++++++++++++---- store/store.go | 31 ++++++++++++++----------------- store/store_test.go | 41 +++++++++++++++++++++++++++++++++++++++-- store/watcher.go | 15 +++++++++++++++ 5 files changed, 103 insertions(+), 25 deletions(-) diff --git a/store/event.go b/store/event.go index f4085358d49..29cec3f1266 100644 --- a/store/event.go +++ b/store/event.go @@ -15,6 +15,7 @@ const ( Update = "update" Delete = "delete" TestAndSet = "testAndSet" + Expire = "expire" ) type Event struct { @@ -111,6 +112,21 @@ func (eh *EventHistory) addEvent(e *Event) { eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index } +// addEvent with the last event's index and term +func (eh *EventHistory) addEventWithouIndex(action, key string) (e *Event) { + eh.rwl.Lock() + defer eh.rwl.Unlock() + + LastEvent := eh.Queue.Events[eh.Queue.back()] + e = newEvent(action, key, LastEvent.Index, LastEvent.Term) + + eh.Queue.insert(e) + + eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index + + return e +} + // scan function is enumerating events from the index in history and // stops till the first point where the key has identified prefix func (eh *EventHistory) scan(prefix string, index uint64) (*Event, error) { @@ -126,8 +142,7 @@ func (eh *EventHistory) scan(prefix string, index uint64) (*Event, error) { // TODO: Add error type return nil, etcdErr.NewError(etcdErr.EcodeEventIndexCleared, - fmt.Sprintf("prefix:%v index:%v", prefix, index), - ) + fmt.Sprintf("prefix:%v index:%v", prefix, index)) } if start >= uint64(eh.Queue.Size) { diff --git a/store/node.go b/store/node.go index 8502f70d15c..051b622a226 100644 --- a/store/node.go +++ b/store/node.go @@ -236,20 +236,20 @@ func (n *Node) Clone() *Node { return clone } -func (n *Node) recoverAndclean() { +func (n *Node) recoverAndclean(WatcherHub *watcherHub) { if n.IsDir() { for _, child := range n.Children { child.Parent = n - child.recoverAndclean() + child.recoverAndclean(WatcherHub) } } n.stopExpire = make(chan bool, 1) - n.Expire() + n.Expire(WatcherHub) } -func (n *Node) Expire() { +func (n *Node) Expire(WatcherHub *watcherHub) { expired, duration := n.IsExpired() if expired { // has been expired @@ -266,6 +266,9 @@ func (n *Node) Expire() { // if timeout, delete the node case <-time.After(duration): n.Remove(true, nil) + + WatcherHub.notifyWithoutIndex(Expire, n.Path) + return // if stopped, return @@ -354,3 +357,14 @@ func (n *Node) Pair(recurisive, sorted bool) KeyValuePair { Value: n.Value, } } + +func (n *Node) UpdateTTL(expireTime time.Time, WatcherHub *watcherHub) { + if !n.IsPermanent() { + n.stopExpire <- true // suspend it to modify the expiration + } + + if expireTime.Sub(Permanent) != 0 { + n.ExpireTime = expireTime + n.Expire(WatcherHub) + } +} diff --git a/store/store.go b/store/store.go index 6265d291c0c..497ffbd8803 100644 --- a/store/store.go +++ b/store/store.go @@ -150,7 +150,7 @@ func (s *Store) Create(nodePath string, value string, expireTime time.Time, inde // Node with TTL if expireTime.Sub(Permanent) != 0 { - n.Expire() + n.Expire(s.WatcherHub) e.Expiration = &n.ExpireTime e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 } @@ -194,19 +194,14 @@ func (s *Store) Update(nodePath string, value string, expireTime time.Time, inde } // update ttl - if !n.IsPermanent() { - n.stopExpire <- true - } + n.UpdateTTL(expireTime, s.WatcherHub) - if expireTime.Sub(Permanent) != 0 { - n.ExpireTime = expireTime - n.Expire() - e.Expiration = &n.ExpireTime - e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 - } + e.Expiration = &n.ExpireTime + e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 s.WatcherHub.notify(e) s.Stats.Inc(UpdateSuccess) + return e, nil } @@ -216,31 +211,33 @@ func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, s.worldLock.RLock() defer s.worldLock.RUnlock() - f, err := s.internalGet(nodePath, index, term) + n, err := s.internalGet(nodePath, index, term) if err != nil { s.Stats.Inc(TestAndSetFail) return nil, err } - if f.IsDir() { // can only test and set file + if n.IsDir() { // can only test and set file s.Stats.Inc(TestAndSetFail) return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath) } - if f.Value == prevValue || f.ModifiedIndex == prevIndex { + if n.Value == prevValue || n.ModifiedIndex == prevIndex { // if test succeed, write the value e := newEvent(TestAndSet, nodePath, index, term) - e.PrevValue = f.Value + e.PrevValue = n.Value e.Value = value - f.Write(value, index, term) + n.Write(value, index, term) + + n.UpdateTTL(expireTime, s.WatcherHub) s.WatcherHub.notify(e) s.Stats.Inc(TestAndSetSuccess) return e, nil } - cause := fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, f.Value, prevIndex, f.ModifiedIndex) + cause := fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, n.Value, prevIndex, n.ModifiedIndex) s.Stats.Inc(TestAndSetFail) return nil, etcdErr.NewError(etcdErr.EcodeTestFailed, cause) } @@ -404,7 +401,7 @@ func (s *Store) Recovery(state []byte) error { return err } - s.Root.recoverAndclean() + s.Root.recoverAndclean(s.WatcherHub) return nil } diff --git a/store/store_test.go b/store/store_test.go index 253063a2a00..01a934d4add 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -378,6 +378,33 @@ func TestWatch(t *testing.T) { t.Fatal("watch for Delete subdirectory fails") } + // watch expire + s.Create("/foo/foo/boo", "foo", time.Now().Add(time.Second*1), 9, 1) + c, _ = s.WatcherHub.watch("/foo", true, 0) + time.Sleep(time.Second * 2) + e = nonblockingRetrive(c) + if e.Key != "/foo/foo/boo" || e.Index != 9 { + t.Fatal("watch for Expiration of Create() subdirectory fails ", e) + } + + s.Create("/foo/foo/boo", "foo", Permanent, 10, 1) + s.Update("/foo/foo/boo", "bar", time.Now().Add(time.Second*1), 11, 1) + c, _ = s.WatcherHub.watch("/foo", true, 0) + time.Sleep(time.Second * 2) + e = nonblockingRetrive(c) + if e.Key != "/foo/foo/boo" || e.Index != 11 { + t.Fatal("watch for Expiration of Update() subdirectory fails ", e) + } + + s.Create("/foo/foo/boo", "foo", Permanent, 12, 1) + s.TestAndSet("/foo/foo/boo", "foo", 0, "bar", time.Now().Add(time.Second*1), 13, 1) + c, _ = s.WatcherHub.watch("/foo", true, 0) + time.Sleep(time.Second * 2) + e = nonblockingRetrive(c) + if e.Key != "/foo/foo/boo" || e.Index != 13 { + t.Fatal("watch for Expiration of TestAndSet() subdirectory fails ", e) + } + } func TestSort(t *testing.T) { @@ -442,7 +469,7 @@ func TestSaveAndRecover(t *testing.T) { b, err := s.Save() cloneFs := New() - time.Sleep(time.Second) + time.Sleep(2 * time.Second) cloneFs.Recovery(b) @@ -457,7 +484,17 @@ func TestSaveAndRecover(t *testing.T) { t.Fatal("Error recovered event history start index") } - for i = 0; int(i) < s.WatcherHub.EventHistory.Queue.Size; i++ { + //t.Log("watcherhub.size: ", s.WatcherHub.EventHistory.Queue.Size) + //for i = 0; int(i) < s.WatcherHub.EventHistory.Queue.Size; i++ { + // t.Log(s.WatcherHub.EventHistory.Queue.Events[i]) + //} + // + //t.Log("ClonedWatcherhub.size: ", cloneFs.WatcherHub.EventHistory.Queue.Size) + //for i = 0; int(i) < cloneFs.WatcherHub.EventHistory.Queue.Size; i++ { + // t.Log(cloneFs.WatcherHub.EventHistory.Queue.Events[i]) + //} + + for i = 0; int(i) < cloneFs.WatcherHub.EventHistory.Queue.Size; i++ { if s.WatcherHub.EventHistory.Queue.Events[i].Key != cloneFs.WatcherHub.EventHistory.Queue.Events[i].Key { t.Fatal("Error recovered event history") diff --git a/store/watcher.go b/store/watcher.go index 3b3e43478c2..b039cd9a95e 100644 --- a/store/watcher.go +++ b/store/watcher.go @@ -117,6 +117,21 @@ func (wh *watcherHub) notify(e *Event) { wh.EventHistory.addEvent(e) } +// notify with last event's index and term +func (wh *watcherHub) notifyWithoutIndex(action, key string) { + e := wh.EventHistory.addEventWithouIndex(action, key) + + segments := strings.Split(e.Key, "/") + + currPath := "/" + + // walk through all the paths + for _, segment := range segments { + currPath = path.Join(currPath, segment) + wh.notifyWithPath(e, currPath, false) + } +} + func (wh *watcherHub) clone() *watcherHub { clonedHistory := wh.EventHistory.clone() From 3ae316ac380fc617d7cfd0a1063abdb9b99d3dac Mon Sep 17 00:00:00 2001 From: evan-gu Date: Mon, 30 Sep 2013 02:39:40 -0400 Subject: [PATCH 048/247] add ExpireCount and some test case --- store/node.go | 15 ++++++++------- store/stats.go | 6 +++++- store/stats_test.go | 30 ++++++++++++++++++++++++++++-- store/store.go | 8 ++++---- 4 files changed, 45 insertions(+), 14 deletions(-) diff --git a/store/node.go b/store/node.go index 051b622a226..f09c742c37e 100644 --- a/store/node.go +++ b/store/node.go @@ -236,20 +236,20 @@ func (n *Node) Clone() *Node { return clone } -func (n *Node) recoverAndclean(WatcherHub *watcherHub) { +func (n *Node) recoverAndclean(s *Store) { if n.IsDir() { for _, child := range n.Children { child.Parent = n - child.recoverAndclean(WatcherHub) + child.recoverAndclean(s) } } n.stopExpire = make(chan bool, 1) - n.Expire(WatcherHub) + n.Expire(s) } -func (n *Node) Expire(WatcherHub *watcherHub) { +func (n *Node) Expire(s *Store) { expired, duration := n.IsExpired() if expired { // has been expired @@ -267,7 +267,8 @@ func (n *Node) Expire(WatcherHub *watcherHub) { case <-time.After(duration): n.Remove(true, nil) - WatcherHub.notifyWithoutIndex(Expire, n.Path) + s.Stats.Inc(ExpireCount) + s.WatcherHub.notifyWithoutIndex(Expire, n.Path) return @@ -358,13 +359,13 @@ func (n *Node) Pair(recurisive, sorted bool) KeyValuePair { } } -func (n *Node) UpdateTTL(expireTime time.Time, WatcherHub *watcherHub) { +func (n *Node) UpdateTTL(expireTime time.Time, s *Store) { if !n.IsPermanent() { n.stopExpire <- true // suspend it to modify the expiration } if expireTime.Sub(Permanent) != 0 { n.ExpireTime = expireTime - n.Expire(WatcherHub) + n.Expire(s) } } diff --git a/store/stats.go b/store/stats.go index b276d5279d9..e2053ed4235 100644 --- a/store/stats.go +++ b/store/stats.go @@ -16,6 +16,7 @@ const ( TestAndSetFail = 107 GetSuccess = 110 GetFail = 111 + ExpireCount = 112 ) type Stats struct { @@ -39,6 +40,7 @@ type Stats struct { // Number of testAndSet requests TestAndSetSuccess uint64 `json:"testAndSetSuccess"` TestAndSetFail uint64 `json:"testAndSetFail"` + ExpireCount uint64 `json:"expireCount"` Watchers uint64 `json:"watchers"` } @@ -51,7 +53,7 @@ func newStats() *Stats { func (s *Stats) clone() *Stats { return &Stats{s.GetSuccess, s.GetFail, s.SetSuccess, s.SetFail, s.DeleteSuccess, s.DeleteFail, s.UpdateSuccess, s.UpdateFail, - s.TestAndSetSuccess, s.TestAndSetFail, s.Watchers} + s.TestAndSetSuccess, s.TestAndSetFail, s.Watchers, s.ExpireCount} } // Status() return the statistics info of etcd storage its recent start @@ -93,5 +95,7 @@ func (s *Stats) Inc(field int) { atomic.AddUint64(&s.TestAndSetSuccess, 1) case TestAndSetFail: atomic.AddUint64(&s.TestAndSetFail, 1) + case ExpireCount: + atomic.AddUint64(&s.ExpireCount, 1) } } diff --git a/store/stats_test.go b/store/stats_test.go index ff67f328a4e..207df825fd4 100644 --- a/store/stats_test.go +++ b/store/stats_test.go @@ -16,7 +16,7 @@ func TestBasicStats(t *testing.T) { for _, k := range keys { i++ - _, err := s.Create(k, "bar", time.Now().Add(time.Second*time.Duration(rand.Intn(10))), i, 1) + _, err := s.Create(k, "bar", time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1) if err != nil { SetFail++ } else { @@ -24,6 +24,8 @@ func TestBasicStats(t *testing.T) { } } + time.Sleep(time.Second * 3) + for _, k := range keys { _, err := s.Get(k, false, false, i, 1) if err != nil { @@ -35,7 +37,7 @@ func TestBasicStats(t *testing.T) { for _, k := range keys { i++ - _, err := s.Update(k, "foo", time.Now().Add(time.Second*time.Duration(rand.Intn(5))), i, 1) + _, err := s.Update(k, "foo", time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1) if err != nil { UpdateFail++ } else { @@ -43,6 +45,8 @@ func TestBasicStats(t *testing.T) { } } + time.Sleep(time.Second * 3) + for _, k := range keys { _, err := s.Get(k, false, false, i, 1) if err != nil { @@ -136,4 +140,26 @@ func TestBasicStats(t *testing.T) { t.Fatalf("TestAndSetFail [%d] != Stats.TestAndSetFail [%d]", TestAndSetFail, s.Stats.TestAndSetFail) } + s = New() + SetSuccess = 0 + SetFail = 0 + + for _, k := range keys { + i++ + _, err := s.Create(k, "bar", time.Now().Add(time.Second*3), i, 1) + if err != nil { + SetFail++ + } else { + SetSuccess++ + } + } + + time.Sleep(6 * time.Second) + + ExpireCount := SetSuccess + + if ExpireCount != s.Stats.ExpireCount { + t.Fatalf("ExpireCount [%d] != Stats.ExpireCount [%d]", ExpireCount, s.Stats.ExpireCount) + } + } diff --git a/store/store.go b/store/store.go index 497ffbd8803..51f49f51fe0 100644 --- a/store/store.go +++ b/store/store.go @@ -150,7 +150,7 @@ func (s *Store) Create(nodePath string, value string, expireTime time.Time, inde // Node with TTL if expireTime.Sub(Permanent) != 0 { - n.Expire(s.WatcherHub) + n.Expire(s) e.Expiration = &n.ExpireTime e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 } @@ -194,7 +194,7 @@ func (s *Store) Update(nodePath string, value string, expireTime time.Time, inde } // update ttl - n.UpdateTTL(expireTime, s.WatcherHub) + n.UpdateTTL(expireTime, s) e.Expiration = &n.ExpireTime e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 @@ -230,7 +230,7 @@ func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, e.Value = value n.Write(value, index, term) - n.UpdateTTL(expireTime, s.WatcherHub) + n.UpdateTTL(expireTime, s) s.WatcherHub.notify(e) s.Stats.Inc(TestAndSetSuccess) @@ -401,7 +401,7 @@ func (s *Store) Recovery(state []byte) error { return err } - s.Root.recoverAndclean(s.WatcherHub) + s.Root.recoverAndclean(s) return nil } From 0959448855580976939704cd6f72686cc7c3ebd0 Mon Sep 17 00:00:00 2001 From: evan-gu Date: Mon, 30 Sep 2013 12:18:28 -0400 Subject: [PATCH 049/247] add LastIndex and LastTerm in EventHistory --- store/event.go | 25 ++++++++++++++++++++----- store/node.go | 5 ++++- store/watcher.go | 17 +---------------- 3 files changed, 25 insertions(+), 22 deletions(-) diff --git a/store/event.go b/store/event.go index 29cec3f1266..4ca6f5e5882 100644 --- a/store/event.go +++ b/store/event.go @@ -90,6 +90,8 @@ func (eq *eventQueue) insert(e *Event) { type EventHistory struct { Queue eventQueue StartIndex uint64 + LastIndex uint64 + LastTerm uint64 rwl sync.RWMutex } @@ -103,29 +105,42 @@ func newEventHistory(capacity int) *EventHistory { } // addEvent function adds event into the eventHistory -func (eh *EventHistory) addEvent(e *Event) { +func (eh *EventHistory) addEvent(e *Event) *Event { eh.rwl.Lock() defer eh.rwl.Unlock() + if e.Index == 0 { + e.Index = eh.LastIndex + } + + if e.Term == 0 { + e.Term = eh.LastTerm + } + eh.Queue.insert(e) eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index + + eh.LastIndex = e.Index + eh.LastTerm = e.Term + + return e } // addEvent with the last event's index and term -func (eh *EventHistory) addEventWithouIndex(action, key string) (e *Event) { +/*func (eh *EventHistory) addEventWithouIndex(action, key string) (e *Event) { eh.rwl.Lock() defer eh.rwl.Unlock() LastEvent := eh.Queue.Events[eh.Queue.back()] - e = newEvent(action, key, LastEvent.Index, LastEvent.Term) + e = newEvent(action, key, LastEvent.Index, LastEvent.Term); eh.Queue.insert(e) eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index - return e -} + return e; +}*/ // scan function is enumerating events from the index in history and // stops till the first point where the key has identified prefix diff --git a/store/node.go b/store/node.go index f09c742c37e..214a33fa082 100644 --- a/store/node.go +++ b/store/node.go @@ -265,10 +265,13 @@ func (n *Node) Expire(s *Store) { select { // if timeout, delete the node case <-time.After(duration): + e := newEvent(Expire, n.Path, 0, 0) + n.Remove(true, nil) s.Stats.Inc(ExpireCount) - s.WatcherHub.notifyWithoutIndex(Expire, n.Path) + + s.WatcherHub.notify(e) return diff --git a/store/watcher.go b/store/watcher.go index b039cd9a95e..d67c713df95 100644 --- a/store/watcher.go +++ b/store/watcher.go @@ -104,22 +104,7 @@ func (wh *watcherHub) notifyWithPath(e *Event, path string, force bool) { } func (wh *watcherHub) notify(e *Event) { - segments := strings.Split(e.Key, "/") - - currPath := "/" - - // walk through all the paths - for _, segment := range segments { - currPath = path.Join(currPath, segment) - wh.notifyWithPath(e, currPath, false) - } - - wh.EventHistory.addEvent(e) -} - -// notify with last event's index and term -func (wh *watcherHub) notifyWithoutIndex(action, key string) { - e := wh.EventHistory.addEventWithouIndex(action, key) + e = wh.EventHistory.addEvent(e) segments := strings.Split(e.Key, "/") From b8ac1d082b7045aeffcfcba5825dbebc3e27a12a Mon Sep 17 00:00:00 2001 From: evan-gu Date: Mon, 30 Sep 2013 22:10:40 -0400 Subject: [PATCH 050/247] fix race between Expire() and others, fix UpdateTTL(), modified watcher to catch Expire() --- store/event.go | 29 ++++++++------------ store/event_test.go | 12 ++++----- store/node.go | 16 +++++++---- store/stats_test.go | 12 +++++++++ store/store.go | 5 ++-- store/store_test.go | 63 +++++++++++++++++++------------------------ store/watcher.go | 2 +- store/watcher_test.go | 15 +++++------ 8 files changed, 78 insertions(+), 76 deletions(-) diff --git a/store/event.go b/store/event.go index 4ca6f5e5882..e95d50f1acf 100644 --- a/store/event.go +++ b/store/event.go @@ -16,6 +16,8 @@ const ( Delete = "delete" TestAndSet = "testAndSet" Expire = "expire" + UndefIndex = 0 + UndefTerm = 0 ) type Event struct { @@ -92,6 +94,7 @@ type EventHistory struct { StartIndex uint64 LastIndex uint64 LastTerm uint64 + DupIndex uint64 rwl sync.RWMutex } @@ -109,12 +112,16 @@ func (eh *EventHistory) addEvent(e *Event) *Event { eh.rwl.Lock() defer eh.rwl.Unlock() - if e.Index == 0 { + DupIndex := uint64(0) + + if e.Index == UndefIndex { e.Index = eh.LastIndex + DupIndex = 1 } - if e.Term == 0 { + if e.Term == UndefTerm { e.Term = eh.LastTerm + DupIndex = 1 } eh.Queue.insert(e) @@ -123,32 +130,18 @@ func (eh *EventHistory) addEvent(e *Event) *Event { eh.LastIndex = e.Index eh.LastTerm = e.Term + eh.DupIndex += DupIndex return e } -// addEvent with the last event's index and term -/*func (eh *EventHistory) addEventWithouIndex(action, key string) (e *Event) { - eh.rwl.Lock() - defer eh.rwl.Unlock() - - LastEvent := eh.Queue.Events[eh.Queue.back()] - e = newEvent(action, key, LastEvent.Index, LastEvent.Term); - - eh.Queue.insert(e) - - eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index - - return e; -}*/ - // scan function is enumerating events from the index in history and // stops till the first point where the key has identified prefix func (eh *EventHistory) scan(prefix string, index uint64) (*Event, error) { eh.rwl.RLock() defer eh.rwl.RUnlock() - start := index - eh.StartIndex + start := index - eh.StartIndex + eh.DupIndex // the index should locate after the event history's StartIndex // and before its size diff --git a/store/event_test.go b/store/event_test.go index 0d19dd52aee..e5b35061ca2 100644 --- a/store/event_test.go +++ b/store/event_test.go @@ -13,7 +13,7 @@ func TestEventQueue(t *testing.T) { // Add for i := 0; i < 200; i++ { - e := newEvent(Create, "/foo", uint64(i), 0) + e := newEvent(Create, "/foo", uint64(i), 1) eh.addEvent(e) } @@ -37,11 +37,11 @@ func TestScanHistory(t *testing.T) { eh := newEventHistory(100) // Add - eh.addEvent(newEvent(Create, "/foo", 1, 0)) - eh.addEvent(newEvent(Create, "/foo/bar", 2, 0)) - eh.addEvent(newEvent(Create, "/foo/foo", 3, 0)) - eh.addEvent(newEvent(Create, "/foo/bar/bar", 4, 0)) - eh.addEvent(newEvent(Create, "/foo/foo/foo", 5, 0)) + eh.addEvent(newEvent(Create, "/foo", 1, 1)) + eh.addEvent(newEvent(Create, "/foo/bar", 2, 1)) + eh.addEvent(newEvent(Create, "/foo/foo", 3, 1)) + eh.addEvent(newEvent(Create, "/foo/bar/bar", 4, 1)) + eh.addEvent(newEvent(Create, "/foo/foo/foo", 5, 1)) e, err := eh.scan("/foo", 1) if err != nil || e.Index != 1 { diff --git a/store/node.go b/store/node.go index 214a33fa082..f82f01cd6ec 100644 --- a/store/node.go +++ b/store/node.go @@ -66,6 +66,7 @@ func newDir(nodePath string, createIndex uint64, createTerm uint64, parent *Node // If the node is a directory and recursive is true, the function will recursively remove // add nodes under the receiver node. func (n *Node) Remove(recursive bool, callback func(path string)) error { + n.mu.Lock() defer n.mu.Unlock() @@ -87,6 +88,7 @@ func (n *Node) Remove(recursive bool, callback func(path string)) error { n.stopExpire <- true n.status = removed + } return nil @@ -265,14 +267,14 @@ func (n *Node) Expire(s *Store) { select { // if timeout, delete the node case <-time.After(duration): - e := newEvent(Expire, n.Path, 0, 0) + s.worldLock.Lock() + e := newEvent(Expire, n.Path, UndefIndex, UndefTerm) + s.WatcherHub.notify(e) n.Remove(true, nil) - s.Stats.Inc(ExpireCount) - s.WatcherHub.notify(e) - + s.worldLock.Unlock() return // if stopped, return @@ -364,7 +366,11 @@ func (n *Node) Pair(recurisive, sorted bool) KeyValuePair { func (n *Node) UpdateTTL(expireTime time.Time, s *Store) { if !n.IsPermanent() { - n.stopExpire <- true // suspend it to modify the expiration + expired, _ := n.IsExpired() + + if !expired { + n.stopExpire <- true // suspend it to modify the expiration + } } if expireTime.Sub(Permanent) != 0 { diff --git a/store/stats_test.go b/store/stats_test.go index 207df825fd4..52cd1c8f8b0 100644 --- a/store/stats_test.go +++ b/store/stats_test.go @@ -24,6 +24,8 @@ func TestBasicStats(t *testing.T) { } } + //fmt.Println("create") + time.Sleep(time.Second * 3) for _, k := range keys { @@ -35,6 +37,8 @@ func TestBasicStats(t *testing.T) { } } + //fmt.Println("get") + for _, k := range keys { i++ _, err := s.Update(k, "foo", time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1) @@ -45,6 +49,8 @@ func TestBasicStats(t *testing.T) { } } + //fmt.Println("update") + time.Sleep(time.Second * 3) for _, k := range keys { @@ -66,11 +72,15 @@ func TestBasicStats(t *testing.T) { } } + //fmt.Println("get testAndSet") + for _, k := range keys { s.Watch(k, false, 0, i, 1) watcher_number++ } + //fmt.Println("watch") + for _, k := range keys { _, err := s.Get(k, false, false, i, 1) if err != nil { @@ -91,6 +101,8 @@ func TestBasicStats(t *testing.T) { } } + //fmt.Println("get delete") + for _, k := range keys { _, err := s.Get(k, false, false, i, 1) if err != nil { diff --git a/store/store.go b/store/store.go index 51f49f51fe0..e6b8cf1b9ef 100644 --- a/store/store.go +++ b/store/store.go @@ -164,6 +164,7 @@ func (s *Store) Create(nodePath string, value string, expireTime time.Time, inde // If the node is a file, the value and the ttl can be updated. // If the node is a directory, only the ttl can be updated. func (s *Store) Update(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { + s.worldLock.RLock() defer s.worldLock.RUnlock() @@ -171,15 +172,16 @@ func (s *Store) Update(nodePath string, value string, expireTime time.Time, inde if err != nil { // if the node does not exist, return error s.Stats.Inc(UpdateFail) + return nil, err } e := newEvent(Update, nodePath, s.Index, s.Term) if n.IsDir() { // if the node is a directory, we can only update ttl - if len(value) != 0 { s.Stats.Inc(UpdateFail) + return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath) } @@ -195,7 +197,6 @@ func (s *Store) Update(nodePath string, value string, expireTime time.Time, inde // update ttl n.UpdateTTL(expireTime, s) - e.Expiration = &n.ExpireTime e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 diff --git a/store/store_test.go b/store/store_test.go index 01a934d4add..a16e57032ca 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -320,88 +320,88 @@ func TestTestAndSet(t *testing.T) { // TODO prevValue == nil ? func TestWatch(t *testing.T) { s := New() // watch at a deeper path - c, _ := s.WatcherHub.watch("/foo/foo/foo", false, 0) + c, _ := s.Watch("/foo/foo/foo", false, 0, 0, 1) s.Create("/foo/foo/foo", "bar", Permanent, 1, 1) e := nonblockingRetrive(c) - if e.Key != "/foo/foo/foo" { - t.Fatal("watch for Create node fails") + if e.Key != "/foo/foo/foo" || e.Action != Create { + t.Fatal("watch for Create node fails ", e) } - c, _ = s.WatcherHub.watch("/foo/foo/foo", false, 0) + c, _ = s.Watch("/foo/foo/foo", false, 0, 1, 1) s.Update("/foo/foo/foo", "car", Permanent, 2, 1) e = nonblockingRetrive(c) - if e.Key != "/foo/foo/foo" { - t.Fatal("watch for Update node fails") + if e.Key != "/foo/foo/foo" || e.Action != Update { + t.Fatal("watch for Update node fails ", e) } - c, _ = s.WatcherHub.watch("/foo/foo/foo", false, 0) + c, _ = s.Watch("/foo/foo/foo", false, 0, 2, 1) s.TestAndSet("/foo/foo/foo", "car", 0, "bar", Permanent, 3, 1) e = nonblockingRetrive(c) - if e.Key != "/foo/foo/foo" { + if e.Key != "/foo/foo/foo" || e.Action != TestAndSet { t.Fatal("watch for TestAndSet node fails") } - c, _ = s.WatcherHub.watch("/foo/foo/foo", false, 0) + c, _ = s.Watch("/foo/foo/foo", false, 0, 3, 1) s.Delete("/foo", true, 4, 1) //recursively delete e = nonblockingRetrive(c) - if e.Key != "/foo" { - t.Fatal("watch for Delete node fails") + if e.Key != "/foo" || e.Action != Delete { + t.Fatal("watch for Delete node fails ", e) } // watch at a prefix - c, _ = s.WatcherHub.watch("/foo", true, 0) + c, _ = s.Watch("/foo", true, 0, 4, 1) s.Create("/foo/foo/boo", "bar", Permanent, 5, 1) e = nonblockingRetrive(c) - if e.Key != "/foo/foo/boo" { + if e.Key != "/foo/foo/boo" || e.Action != Create { t.Fatal("watch for Create subdirectory fails") } - c, _ = s.WatcherHub.watch("/foo", true, 0) + c, _ = s.Watch("/foo", true, 0, 5, 1) s.Update("/foo/foo/boo", "foo", Permanent, 6, 1) e = nonblockingRetrive(c) - if e.Key != "/foo/foo/boo" { + if e.Key != "/foo/foo/boo" || e.Action != Update { t.Fatal("watch for Update subdirectory fails") } - c, _ = s.WatcherHub.watch("/foo", true, 0) + c, _ = s.Watch("/foo", true, 0, 6, 1) s.TestAndSet("/foo/foo/boo", "foo", 0, "bar", Permanent, 7, 1) e = nonblockingRetrive(c) - if e.Key != "/foo/foo/boo" { + if e.Key != "/foo/foo/boo" || e.Action != TestAndSet { t.Fatal("watch for TestAndSet subdirectory fails") } - c, _ = s.WatcherHub.watch("/foo", true, 0) + c, _ = s.Watch("/foo", true, 0, 7, 1) s.Delete("/foo/foo/boo", false, 8, 1) e = nonblockingRetrive(c) - if e.Key != "/foo/foo/boo" { + if e.Key != "/foo/foo/boo" || e.Action != Delete { t.Fatal("watch for Delete subdirectory fails") } // watch expire s.Create("/foo/foo/boo", "foo", time.Now().Add(time.Second*1), 9, 1) - c, _ = s.WatcherHub.watch("/foo", true, 0) + c, _ = s.Watch("/foo", true, 0, 9, 1) time.Sleep(time.Second * 2) e = nonblockingRetrive(c) - if e.Key != "/foo/foo/boo" || e.Index != 9 { + if e.Key != "/foo/foo/boo" || e.Action != Expire || e.Index != 9 { t.Fatal("watch for Expiration of Create() subdirectory fails ", e) } s.Create("/foo/foo/boo", "foo", Permanent, 10, 1) s.Update("/foo/foo/boo", "bar", time.Now().Add(time.Second*1), 11, 1) - c, _ = s.WatcherHub.watch("/foo", true, 0) + c, _ = s.Watch("/foo", true, 0, 11, 1) time.Sleep(time.Second * 2) e = nonblockingRetrive(c) - if e.Key != "/foo/foo/boo" || e.Index != 11 { + if e.Key != "/foo/foo/boo" || e.Action != Expire || e.Index != 11 { t.Fatal("watch for Expiration of Update() subdirectory fails ", e) } s.Create("/foo/foo/boo", "foo", Permanent, 12, 1) s.TestAndSet("/foo/foo/boo", "foo", 0, "bar", time.Now().Add(time.Second*1), 13, 1) - c, _ = s.WatcherHub.watch("/foo", true, 0) + c, _ = s.Watch("/foo", true, 0, 13, 1) time.Sleep(time.Second * 2) e = nonblockingRetrive(c) - if e.Key != "/foo/foo/boo" || e.Index != 13 { + if e.Key != "/foo/foo/boo" || e.Action != Expire || e.Index != 13 { t.Fatal("watch for Expiration of TestAndSet() subdirectory fails ", e) } @@ -479,21 +479,12 @@ func TestSaveAndRecover(t *testing.T) { panic(err) } } - + s.worldLock.RLock() + defer s.worldLock.RUnlock() if s.WatcherHub.EventHistory.StartIndex != cloneFs.WatcherHub.EventHistory.StartIndex { t.Fatal("Error recovered event history start index") } - //t.Log("watcherhub.size: ", s.WatcherHub.EventHistory.Queue.Size) - //for i = 0; int(i) < s.WatcherHub.EventHistory.Queue.Size; i++ { - // t.Log(s.WatcherHub.EventHistory.Queue.Events[i]) - //} - // - //t.Log("ClonedWatcherhub.size: ", cloneFs.WatcherHub.EventHistory.Queue.Size) - //for i = 0; int(i) < cloneFs.WatcherHub.EventHistory.Queue.Size; i++ { - // t.Log(cloneFs.WatcherHub.EventHistory.Queue.Events[i]) - //} - for i = 0; int(i) < cloneFs.WatcherHub.EventHistory.Queue.Size; i++ { if s.WatcherHub.EventHistory.Queue.Events[i].Key != cloneFs.WatcherHub.EventHistory.Queue.Events[i].Key { diff --git a/store/watcher.go b/store/watcher.go index d67c713df95..7576a866b36 100644 --- a/store/watcher.go +++ b/store/watcher.go @@ -47,7 +47,7 @@ func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (<-chan w := &watcher{ eventChan: eventChan, recursive: recursive, - sinceIndex: index, + sinceIndex: index - 1, // to catch Expire() } l, ok := wh.watchers[prefix] diff --git a/store/watcher_test.go b/store/watcher_test.go index 90c23c59e93..e437422ad13 100644 --- a/store/watcher_test.go +++ b/store/watcher_test.go @@ -7,8 +7,7 @@ import ( func TestWatcher(t *testing.T) { s := New() wh := s.WatcherHub - c, err := wh.watch("/foo", true, 0) - + c, err := wh.watch("/foo", true, 1) if err != nil { t.Fatal("%v", err) } @@ -20,7 +19,7 @@ func TestWatcher(t *testing.T) { // do nothing } - e := newEvent(Create, "/foo/bar", 1, 0) + e := newEvent(Create, "/foo/bar", 1, 1) wh.notify(e) @@ -30,20 +29,20 @@ func TestWatcher(t *testing.T) { t.Fatal("recv != send") } - c, _ = wh.watch("/foo", false, 0) + c, _ = wh.watch("/foo", false, 2) - e = newEvent(Create, "/foo/bar", 1, 0) + e = newEvent(Create, "/foo/bar", 2, 1) wh.notify(e) select { - case <-c: - t.Fatal("should not receive from channel if not recursive") + case re = <-c: + t.Fatal("should not receive from channel if not recursive ", re) default: // do nothing } - e = newEvent(Create, "/foo", 1, 0) + e = newEvent(Create, "/foo", 3, 1) wh.notify(e) From 6fdffbcc851aa1db0c50289e05ec9d163c3189ab Mon Sep 17 00:00:00 2001 From: evan-gu Date: Mon, 30 Sep 2013 22:17:17 -0400 Subject: [PATCH 051/247] delete some debug comments in stats_test.go --- store/stats_test.go | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/store/stats_test.go b/store/stats_test.go index 52cd1c8f8b0..207df825fd4 100644 --- a/store/stats_test.go +++ b/store/stats_test.go @@ -24,8 +24,6 @@ func TestBasicStats(t *testing.T) { } } - //fmt.Println("create") - time.Sleep(time.Second * 3) for _, k := range keys { @@ -37,8 +35,6 @@ func TestBasicStats(t *testing.T) { } } - //fmt.Println("get") - for _, k := range keys { i++ _, err := s.Update(k, "foo", time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1) @@ -49,8 +45,6 @@ func TestBasicStats(t *testing.T) { } } - //fmt.Println("update") - time.Sleep(time.Second * 3) for _, k := range keys { @@ -72,15 +66,11 @@ func TestBasicStats(t *testing.T) { } } - //fmt.Println("get testAndSet") - for _, k := range keys { s.Watch(k, false, 0, i, 1) watcher_number++ } - //fmt.Println("watch") - for _, k := range keys { _, err := s.Get(k, false, false, i, 1) if err != nil { @@ -101,8 +91,6 @@ func TestBasicStats(t *testing.T) { } } - //fmt.Println("get delete") - for _, k := range keys { _, err := s.Get(k, false, false, i, 1) if err != nil { From 6f591032ef33b6e81311fb10957c3956974cd90e Mon Sep 17 00:00:00 2001 From: evan-gu Date: Tue, 1 Oct 2013 00:35:44 -0400 Subject: [PATCH 052/247] rename to DupCnt, duped; add some comments, maintained some format, add notification for immediate expiration --- store/event.go | 15 ++++++--------- store/event_test.go | 3 --- store/node.go | 22 +++++++++++++++++----- store/store.go | 4 ++-- store/store_test.go | 14 ++++---------- store/watcher.go | 5 +---- 6 files changed, 30 insertions(+), 33 deletions(-) diff --git a/store/event.go b/store/event.go index e95d50f1acf..34bdf7d9b1e 100644 --- a/store/event.go +++ b/store/event.go @@ -76,7 +76,6 @@ func (eq *eventQueue) back() int { } func (eq *eventQueue) insert(e *Event) { - index := (eq.back() + 1) % eq.Capacity eq.Events[index] = e @@ -94,7 +93,7 @@ type EventHistory struct { StartIndex uint64 LastIndex uint64 LastTerm uint64 - DupIndex uint64 + DupCnt uint64 // help to compute the watching point with duplicated indexes in the queue rwl sync.RWMutex } @@ -112,16 +111,16 @@ func (eh *EventHistory) addEvent(e *Event) *Event { eh.rwl.Lock() defer eh.rwl.Unlock() - DupIndex := uint64(0) + duped := uint64(0) if e.Index == UndefIndex { e.Index = eh.LastIndex - DupIndex = 1 + duped = 1 } if e.Term == UndefTerm { e.Term = eh.LastTerm - DupIndex = 1 + duped = 1 } eh.Queue.insert(e) @@ -130,7 +129,7 @@ func (eh *EventHistory) addEvent(e *Event) *Event { eh.LastIndex = e.Index eh.LastTerm = e.Term - eh.DupIndex += DupIndex + eh.DupCnt += duped return e } @@ -141,7 +140,7 @@ func (eh *EventHistory) scan(prefix string, index uint64) (*Event, error) { eh.rwl.RLock() defer eh.rwl.RUnlock() - start := index - eh.StartIndex + eh.DupIndex + start := index - eh.StartIndex + eh.DupCnt // the index should locate after the event history's StartIndex // and before its size @@ -172,13 +171,11 @@ func (eh *EventHistory) scan(prefix string, index uint64) (*Event, error) { return nil, nil } } - } // clone will be protected by a stop-world lock // do not need to obtain internal lock func (eh *EventHistory) clone() *EventHistory { - clonedQueue := eventQueue{ Capacity: eh.Queue.Capacity, Events: make([]*Event, eh.Queue.Capacity), diff --git a/store/event_test.go b/store/event_test.go index e5b35061ca2..c02a4d70ef0 100644 --- a/store/event_test.go +++ b/store/event_test.go @@ -28,9 +28,7 @@ func TestEventQueue(t *testing.T) { } j++ i = (i + 1) % eh.Queue.Capacity - } - } func TestScanHistory(t *testing.T) { @@ -65,5 +63,4 @@ func TestScanHistory(t *testing.T) { if e != nil { t.Fatalf("bad index shoud reuturn nil") } - } diff --git a/store/node.go b/store/node.go index f82f01cd6ec..cad85c75f93 100644 --- a/store/node.go +++ b/store/node.go @@ -66,7 +66,6 @@ func newDir(nodePath string, createIndex uint64, createTerm uint64, parent *Node // If the node is a directory and recursive is true, the function will recursively remove // add nodes under the receiver node. func (n *Node) Remove(recursive bool, callback func(path string)) error { - n.mu.Lock() defer n.mu.Unlock() @@ -187,7 +186,6 @@ func (n *Node) GetFile(name string) (*Node, error) { } return nil, nil - } // Add function adds a node to the receiver node. @@ -216,7 +214,6 @@ func (n *Node) Add(child *Node) error { n.Children[name] = child return nil - } // Clone function clone the node recursively and return the new node. @@ -251,11 +248,23 @@ func (n *Node) recoverAndclean(s *Store) { n.Expire(s) } +// Expire function will test if the node is expired. +// if the node is already expired, delete the node and return. +// if the node is permemant (this shouldn't happen), return at once. +// else wait for a period time, then remove the node. and notify the watchhub. func (n *Node) Expire(s *Store) { expired, duration := n.IsExpired() if expired { // has been expired + + // since the parent function of Expire() runs serially, + // there is no need for lock here + e := newEvent(Expire, n.Path, UndefIndex, UndefTerm) + s.WatcherHub.notify(e) + n.Remove(true, nil) + s.Stats.Inc(ExpireCount) + return } @@ -267,20 +276,23 @@ func (n *Node) Expire(s *Store) { select { // if timeout, delete the node case <-time.After(duration): + + // Lock to avoid race s.worldLock.Lock() e := newEvent(Expire, n.Path, UndefIndex, UndefTerm) s.WatcherHub.notify(e) + n.Remove(true, nil) s.Stats.Inc(ExpireCount) s.worldLock.Unlock() + return // if stopped, return case <-n.stopExpire: return - } }() } @@ -294,7 +306,6 @@ func (n *Node) IsHidden() bool { _, name := path.Split(n.Path) return name[0] == '_' - } func (n *Node) IsPermanent() bool { @@ -355,6 +366,7 @@ func (n *Node) Pair(recurisive, sorted bool) KeyValuePair { if sorted { sort.Sort(pair) } + return pair } diff --git a/store/store.go b/store/store.go index e6b8cf1b9ef..f77189172c9 100644 --- a/store/store.go +++ b/store/store.go @@ -164,7 +164,6 @@ func (s *Store) Create(nodePath string, value string, expireTime time.Time, inde // If the node is a file, the value and the ttl can be updated. // If the node is a directory, only the ttl can be updated. func (s *Store) Update(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { - s.worldLock.RLock() defer s.worldLock.RUnlock() @@ -197,10 +196,11 @@ func (s *Store) Update(nodePath string, value string, expireTime time.Time, inde // update ttl n.UpdateTTL(expireTime, s) + e.Expiration = &n.ExpireTime e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 - s.WatcherHub.notify(e) + s.Stats.Inc(UpdateSuccess) return e, nil diff --git a/store/store_test.go b/store/store_test.go index a16e57032ca..87010b1ee67 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -52,7 +52,6 @@ func TestCreateAndGet(t *testing.T) { if err != nil { t.Fatal("Cannot create /fooDir/bar = bar") } - } func TestUpdateFile(t *testing.T) { @@ -81,7 +80,6 @@ func TestUpdateFile(t *testing.T) { } // create a directory, update its ttl, to see if it will be deleted - _, err = s.Create("/foo/foo", "", Permanent, 3, 1) if err != nil { @@ -237,7 +235,6 @@ func TestRemove(t *testing.T) { if err == nil || err.Error() != "Key Not Found" { t.Fatalf("can get the node after deletion ") } - } func TestExpire(t *testing.T) { @@ -280,7 +277,6 @@ func TestExpire(t *testing.T) { if err != nil { t.Fatalf("cannot delete the node before expiration", err.Error()) } - } func TestTestAndSet(t *testing.T) { // TODO prevValue == nil ? @@ -314,7 +310,6 @@ func TestTestAndSet(t *testing.T) { // TODO prevValue == nil ? if e.PrevValue != "car" || e.Value != "bar" { t.Fatalf("[%v/%v] [%v/%v]", e.PrevValue, "car", e.Value, "bar") } - } func TestWatch(t *testing.T) { @@ -404,7 +399,6 @@ func TestWatch(t *testing.T) { if e.Key != "/foo/foo/boo" || e.Action != Expire || e.Index != 13 { t.Fatal("watch for Expiration of TestAndSet() subdirectory fails ", e) } - } func TestSort(t *testing.T) { @@ -479,8 +473,11 @@ func TestSaveAndRecover(t *testing.T) { panic(err) } } + + // lock to avoid racing with Expire() s.worldLock.RLock() defer s.worldLock.RUnlock() + if s.WatcherHub.EventHistory.StartIndex != cloneFs.WatcherHub.EventHistory.StartIndex { t.Fatal("Error recovered event history start index") } @@ -497,7 +494,6 @@ func TestSaveAndRecover(t *testing.T) { if err == nil || err.Error() != "Key Not Found" { t.Fatalf("can get the node after deletion ") } - } // GenKeys randomly generate num of keys with max depth @@ -513,6 +509,7 @@ func GenKeys(num int, depth int) []string { keys[i] += "/" + strconv.Itoa(rand.Int()) } } + return keys } @@ -532,11 +529,9 @@ func createAndGet(s *Store, path string, t *testing.T) { if e.Value != "bar" { t.Fatalf("expect value of %s is bar [%s]", path, e.Value) } - } func recursiveTestSort(k KeyValuePair, t *testing.T) { - for i, v := range k.KVPairs[:len(k.KVPairs)-1] { if v.Key >= k.KVPairs[i+1].Key { t.Fatalf("sort failed, [%s] should be placed after [%s]", v.Key, k.KVPairs[i+1].Key) @@ -545,7 +540,6 @@ func recursiveTestSort(k KeyValuePair, t *testing.T) { if v.Dir { recursiveTestSort(v, t) } - } if v := k.KVPairs[len(k.KVPairs)-1]; v.Dir { diff --git a/store/watcher.go b/store/watcher.go index 7576a866b36..5d96e45cd3e 100644 --- a/store/watcher.go +++ b/store/watcher.go @@ -74,21 +74,19 @@ func (wh *watcherHub) notifyWithPath(e *Event, path string, force bool) { notifiedAll := true for { - if curr == nil { // we have reached the end of the list - if notifiedAll { // if we have notified all watcher in the list // we can delete the list delete(wh.watchers, path) } + break } next := curr.Next() // save the next w, _ := curr.Value.(*watcher) - if (w.recursive || force || e.Key == path) && e.Index >= w.sinceIndex { w.eventChan <- e l.Remove(curr) @@ -98,7 +96,6 @@ func (wh *watcherHub) notifyWithPath(e *Event, path string, force bool) { } curr = next // go to the next one - } } } From 974d74befbd3ac882f012e564a5f9a3b7b35e489 Mon Sep 17 00:00:00 2001 From: evan-gu Date: Tue, 1 Oct 2013 01:25:45 -0400 Subject: [PATCH 053/247] add some comment and change a declaration form --- store/event.go | 2 +- store/node.go | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/store/event.go b/store/event.go index 34bdf7d9b1e..53d4568bd2d 100644 --- a/store/event.go +++ b/store/event.go @@ -111,7 +111,7 @@ func (eh *EventHistory) addEvent(e *Event) *Event { eh.rwl.Lock() defer eh.rwl.Unlock() - duped := uint64(0) + var duped uint64 if e.Index == UndefIndex { e.Index = eh.LastIndex diff --git a/store/node.go b/store/node.go index cad85c75f93..f5545fa6031 100644 --- a/store/node.go +++ b/store/node.go @@ -277,7 +277,8 @@ func (n *Node) Expire(s *Store) { // if timeout, delete the node case <-time.After(duration): - // Lock to avoid race + // Lock the worldLock to avoid race on s.WatchHub, + // and the race with other slibling nodes on their common parent. s.worldLock.Lock() e := newEvent(Expire, n.Path, UndefIndex, UndefTerm) From 558d30f33f785902e87eee062ba47f183656a59c Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 30 Sep 2013 23:18:52 -0700 Subject: [PATCH 054/247] simplify lock --- store/node.go | 26 ++------------------------ store/store.go | 18 +++++++++--------- store/store_test.go | 6 +++--- 3 files changed, 14 insertions(+), 36 deletions(-) diff --git a/store/node.go b/store/node.go index f5545fa6031..e7d71cdb3fb 100644 --- a/store/node.go +++ b/store/node.go @@ -3,7 +3,6 @@ package store import ( "path" "sort" - "sync" "time" etcdErr "github.com/coreos/etcd/error" @@ -30,7 +29,6 @@ type Node struct { Value string // for key-value pair Children map[string]*Node // for directory status int - mu sync.Mutex stopExpire chan bool // stop expire routine channel } @@ -66,10 +64,7 @@ func newDir(nodePath string, createIndex uint64, createTerm uint64, parent *Node // If the node is a directory and recursive is true, the function will recursively remove // add nodes under the receiver node. func (n *Node) Remove(recursive bool, callback func(path string)) error { - n.mu.Lock() - defer n.mu.Unlock() - - if n.status == removed { + if n.status == removed { // check race between remove and expire return nil } @@ -144,8 +139,6 @@ func (n *Node) Write(value string, index uint64, term uint64) error { // List function return a slice of nodes under the receiver node. // If the receiver node is not a directory, a "Not A Directory" error will be returned. func (n *Node) List() ([]*Node, error) { - n.mu.Lock() - defer n.mu.Unlock() if !n.IsDir() { return nil, etcdErr.NewError(etcdErr.EcodeNotDir, "") } @@ -168,9 +161,6 @@ func (n *Node) List() ([]*Node, error) { // If the node corresponding to the name string is not file, it returns // Not File Error func (n *Node) GetFile(name string) (*Node, error) { - n.mu.Lock() - defer n.mu.Unlock() - if !n.IsDir() { return nil, etcdErr.NewError(etcdErr.EcodeNotDir, n.Path) } @@ -193,12 +183,6 @@ func (n *Node) GetFile(name string) (*Node, error) { // If there is a existing node with the same name under the directory, a "Already Exist" // error will be returned func (n *Node) Add(child *Node) error { - n.mu.Lock() - defer n.mu.Unlock() - if n.status == removed { - return etcdErr.NewError(etcdErr.EcodeKeyNotFound, "") - } - if !n.IsDir() { return etcdErr.NewError(etcdErr.EcodeNotDir, "") } @@ -220,8 +204,6 @@ func (n *Node) Add(child *Node) error { // If the node is a directory, it will clone all the content under this directory. // If the node is a key-value pair, it will clone the pair. func (n *Node) Clone() *Node { - n.mu.Lock() - defer n.mu.Unlock() if !n.IsDir() { return newFile(n.Path, n.Value, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime) } @@ -256,7 +238,6 @@ func (n *Node) Expire(s *Store) { expired, duration := n.IsExpired() if expired { // has been expired - // since the parent function of Expire() runs serially, // there is no need for lock here e := newEvent(Expire, n.Path, UndefIndex, UndefTerm) @@ -277,9 +258,8 @@ func (n *Node) Expire(s *Store) { // if timeout, delete the node case <-time.After(duration): - // Lock the worldLock to avoid race on s.WatchHub, - // and the race with other slibling nodes on their common parent. s.worldLock.Lock() + defer s.worldLock.Unlock() e := newEvent(Expire, n.Path, UndefIndex, UndefTerm) s.WatcherHub.notify(e) @@ -287,8 +267,6 @@ func (n *Node) Expire(s *Store) { n.Remove(true, nil) s.Stats.Inc(ExpireCount) - s.worldLock.Unlock() - return // if stopped, return diff --git a/store/store.go b/store/store.go index f77189172c9..7755675780a 100644 --- a/store/store.go +++ b/store/store.go @@ -18,7 +18,7 @@ type Store struct { Index uint64 Term uint64 Stats *Stats - worldLock sync.RWMutex // stop the world lock. Used to do snapshot + worldLock sync.RWMutex // stop the world lock } func New() *Store { @@ -95,8 +95,8 @@ func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term // If the node has already existed, create will fail. // If any node on the path is a file, create will fail. func (s *Store) Create(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { - s.worldLock.RLock() - defer s.worldLock.RUnlock() + s.worldLock.Lock() + defer s.worldLock.Unlock() nodePath = path.Clean(path.Join("/", nodePath)) @@ -164,8 +164,8 @@ func (s *Store) Create(nodePath string, value string, expireTime time.Time, inde // If the node is a file, the value and the ttl can be updated. // If the node is a directory, only the ttl can be updated. func (s *Store) Update(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { - s.worldLock.RLock() - defer s.worldLock.RUnlock() + s.worldLock.Lock() + defer s.worldLock.Unlock() n, err := s.internalGet(nodePath, index, term) @@ -209,8 +209,8 @@ func (s *Store) Update(nodePath string, value string, expireTime time.Time, inde func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { - s.worldLock.RLock() - defer s.worldLock.RUnlock() + s.worldLock.Lock() + defer s.worldLock.Unlock() n, err := s.internalGet(nodePath, index, term) @@ -246,8 +246,8 @@ func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, // Delete function deletes the node at the given path. // If the node is a directory, recursive must be true to delete it. func (s *Store) Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error) { - s.worldLock.RLock() - defer s.worldLock.RUnlock() + s.worldLock.Lock() + defer s.worldLock.Unlock() n, err := s.internalGet(nodePath, index, term) diff --git a/store/store_test.go b/store/store_test.go index 87010b1ee67..37dba4f1499 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -244,7 +244,7 @@ func TestExpire(t *testing.T) { s.Create("/foo", "bar", expire, 1, 1) - _, err := s.internalGet("/foo", 1, 1) + _, err := s.Get("/foo", false, false, 1, 1) if err != nil { t.Fatalf("can not get the node") @@ -252,7 +252,7 @@ func TestExpire(t *testing.T) { time.Sleep(time.Second * 2) - _, err = s.internalGet("/foo", 1, 1) + _, err = s.Get("/foo", false, false, 1, 1) if err == nil { t.Fatalf("can get the node after expiration time") @@ -263,7 +263,7 @@ func TestExpire(t *testing.T) { s.Create("/foo", "bar", expire, 1, 1) time.Sleep(time.Millisecond * 50) - _, err = s.internalGet("/foo", 1, 1) + _, err = s.Get("/foo", false, false, 1, 1) if err != nil { t.Fatalf("cannot get the node before expiration", err.Error()) From 9412c86b97333bea5c65f0fb6307f065be75b6b7 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 30 Sep 2013 23:51:47 -0700 Subject: [PATCH 055/247] fix wrong logic in event.go --- store/event.go | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/store/event.go b/store/event.go index 53d4568bd2d..9eaa7084232 100644 --- a/store/event.go +++ b/store/event.go @@ -140,19 +140,18 @@ func (eh *EventHistory) scan(prefix string, index uint64) (*Event, error) { eh.rwl.RLock() defer eh.rwl.RUnlock() - start := index - eh.StartIndex + eh.DupCnt + start := index - eh.StartIndex // the index should locate after the event history's StartIndex - // and before its size - if start < 0 { - // TODO: Add error type return nil, etcdErr.NewError(etcdErr.EcodeEventIndexCleared, - fmt.Sprintf("prefix:%v index:%v", prefix, index)) + fmt.Sprintf("the requested history has been cleared [%v/%v]", + eh.StartIndex, index)) } - if start >= uint64(eh.Queue.Size) { + // the index should locate before the size of the queue minus the duplicate count + if start >= (uint64(eh.Queue.Size) - eh.DupCnt) { // future index return nil, nil } @@ -160,14 +159,13 @@ func (eh *EventHistory) scan(prefix string, index uint64) (*Event, error) { for { e := eh.Queue.Events[i] - if strings.HasPrefix(e.Key, prefix) { + if strings.HasPrefix(e.Key, prefix) && index <= e.Index { // make sure we bypass the smaller one return e, nil } i = (i + 1) % eh.Queue.Capacity - if i == eh.Queue.back() { - // TODO: Add error type + if i == eh.Queue.back() { // find nothing, return and watch from current index return nil, nil } } From 512dede9ce1bf5d97496149774fba4677c2b3dd0 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Wed, 2 Oct 2013 22:15:12 -0700 Subject: [PATCH 056/247] finish todo --- error/error.go | 10 ++-- etcd_handlers.go | 37 +++++++------ store/event.go | 3 ++ store/store.go | 135 +++++++++++++++++++++++++---------------------- 4 files changed, 101 insertions(+), 84 deletions(-) diff --git a/error/error.go b/error/error.go index 643c3244b61..8d342de143f 100644 --- a/error/error.go +++ b/error/error.go @@ -16,10 +16,11 @@ const ( EcodeNodeExist = 105 EcodeKeyIsPreserved = 106 - EcodeValueRequired = 200 - EcodePrevValueRequired = 201 - EcodeTTLNaN = 202 - EcodeIndexNaN = 203 + EcodeValueRequired = 200 + EcodePrevValueRequired = 201 + EcodeTTLNaN = 202 + EcodeIndexNaN = 203 + EcodeValueOrTTLRequired = 204 EcodeRaftInternal = 300 EcodeLeaderElect = 301 @@ -45,6 +46,7 @@ func init() { errors[EcodePrevValueRequired] = "PrevValue is Required in POST form" errors[EcodeTTLNaN] = "The given TTL in POST form is not a number" errors[EcodeIndexNaN] = "The given index in POST form is not a number" + errors[EcodeValueOrTTLRequired] = "Value or TTL is required in POST form" // raft related errors errors[EcodeRaftInternal] = "Raft Internal Error" diff --git a/etcd_handlers.go b/etcd_handlers.go index adbb56f0339..4e802a5a013 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -111,24 +111,26 @@ func UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { debugf("recv.put[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) - value := req.FormValue("value") + req.ParseForm() - expireTime, err := durationToExpireTime(req.FormValue("ttl")) + value := req.Form.Get("value") + + expireTime, err := durationToExpireTime(req.Form.Get("ttl")) if err != nil { return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update") } - // TODO: update should give at least one option + // update should give at least one option if value == "" && expireTime.Sub(store.Permanent) == 0 { - return nil + return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update") } - prevValue := req.FormValue("prevValue") + prevValue, valueOk := req.Form["prevValue"] - prevIndexStr := req.FormValue("prevIndex") + prevIndexStr, indexOk := req.Form["prevIndex"] - if prevValue == "" && prevIndexStr == "" { // update without test + if !valueOk && !indexOk { // update without test command := &UpdateCommand{ Key: key, Value: value, @@ -140,19 +142,21 @@ func UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { } else { // update with test var prevIndex uint64 - if prevIndexStr != "" { - prevIndex, err = strconv.ParseUint(prevIndexStr, 10, 64) - } + if indexOk { + prevIndex, err = strconv.ParseUint(prevIndexStr[0], 10, 64) - // TODO: add error type - if err != nil { - return nil + // bad previous index + if err != nil { + return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Update") + } + } else { + prevIndex = 0 } command := &TestAndSetCommand{ Key: key, Value: value, - PrevValue: prevValue, + PrevValue: prevValue[0], PrevIndex: prevIndex, } @@ -185,9 +189,8 @@ func dispatchEtcdCommand(c Command, w http.ResponseWriter, req *http.Request) er //-------------------------------------- // State non-sensitive handlers -// will not dispatch to leader -// TODO: add sensitive version for these -// command? +// command with consistent option will +// still dispatch to the leader //-------------------------------------- // Handler to return the current leader's raft address diff --git a/store/event.go b/store/event.go index 9eaa7084232..25562a9994d 100644 --- a/store/event.go +++ b/store/event.go @@ -16,6 +16,9 @@ const ( Delete = "delete" TestAndSet = "testAndSet" Expire = "expire" +) + +const ( UndefIndex = 0 UndefTerm = 0 ) diff --git a/store/store.go b/store/store.go index 7755675780a..0f6a425f7dd 100644 --- a/store/store.go +++ b/store/store.go @@ -97,67 +97,7 @@ func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term func (s *Store) Create(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { s.worldLock.Lock() defer s.worldLock.Unlock() - - nodePath = path.Clean(path.Join("/", nodePath)) - - // make sure we can create the node - _, err := s.internalGet(nodePath, index, term) - - if err == nil { // key already exists - s.Stats.Inc(SetFail) - return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath) - } - - etcdError, _ := err.(etcdErr.Error) - - if etcdError.ErrorCode == 104 { // we cannot create the key due to meet a file while walking - s.Stats.Inc(SetFail) - return nil, err - } - - dir, _ := path.Split(nodePath) - - // walk through the nodePath, create dirs and get the last directory node - d, err := s.walk(dir, s.checkDir) - - if err != nil { - s.Stats.Inc(SetFail) - return nil, err - } - - e := newEvent(Create, nodePath, s.Index, s.Term) - - var n *Node - - if len(value) != 0 { // create file - e.Value = value - - n = newFile(nodePath, value, s.Index, s.Term, d, "", expireTime) - - } else { // create directory - e.Dir = true - - n = newDir(nodePath, s.Index, s.Term, d, "", expireTime) - - } - - err = d.Add(n) - - if err != nil { - s.Stats.Inc(SetFail) - return nil, err - } - - // Node with TTL - if expireTime.Sub(Permanent) != 0 { - n.Expire(s) - e.Expiration = &n.ExpireTime - e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 - } - - s.WatcherHub.notify(e) - s.Stats.Inc(SetSuccess) - return e, nil + return s.internalCreate(nodePath, value, expireTime, index, term, Create) } // Update function updates the value/ttl of the node. @@ -197,8 +137,10 @@ func (s *Store) Update(nodePath string, value string, expireTime time.Time, inde // update ttl n.UpdateTTL(expireTime, s) - e.Expiration = &n.ExpireTime - e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 + if n.ExpireTime.Sub(Permanent) != 0 { + e.Expiration = &n.ExpireTime + e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 + } s.WatcherHub.notify(e) s.Stats.Inc(UpdateSuccess) @@ -212,6 +154,10 @@ func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, s.worldLock.Lock() defer s.worldLock.Unlock() + if prevValue == "" && prevIndex == 0 { // try create + return s.internalCreate(nodePath, value, expireTime, index, term, TestAndSet) + } + n, err := s.internalGet(nodePath, index, term) if err != nil { @@ -316,6 +262,69 @@ func (s *Store) walk(nodePath string, walkFunc func(prev *Node, component string return curr, nil } +func (s *Store) internalCreate(nodePath string, value string, expireTime time.Time, index uint64, term uint64, action string) (*Event, error) { + nodePath = path.Clean(path.Join("/", nodePath)) + + // make sure we can create the node + _, err := s.internalGet(nodePath, index, term) + + if err == nil { // key already exists + s.Stats.Inc(SetFail) + return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath) + } + + etcdError, _ := err.(etcdErr.Error) + + if etcdError.ErrorCode == 104 { // we cannot create the key due to meet a file while walking + s.Stats.Inc(SetFail) + return nil, err + } + + dir, _ := path.Split(nodePath) + + // walk through the nodePath, create dirs and get the last directory node + d, err := s.walk(dir, s.checkDir) + + if err != nil { + s.Stats.Inc(SetFail) + return nil, err + } + + e := newEvent(action, nodePath, s.Index, s.Term) + + var n *Node + + if len(value) != 0 { // create file + e.Value = value + + n = newFile(nodePath, value, s.Index, s.Term, d, "", expireTime) + + } else { // create directory + e.Dir = true + + n = newDir(nodePath, s.Index, s.Term, d, "", expireTime) + + } + + err = d.Add(n) + + if err != nil { + s.Stats.Inc(SetFail) + return nil, err + } + + // Node with TTL + if expireTime.Sub(Permanent) != 0 { + n.Expire(s) + e.Expiration = &n.ExpireTime + e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 + } + + s.WatcherHub.notify(e) + s.Stats.Inc(SetSuccess) + return e, nil +} + // InternalGet function get the node of the given nodePath. func (s *Store) internalGet(nodePath string, index uint64, term uint64) (*Node, error) { nodePath = path.Clean(path.Join("/", nodePath)) From b8b81d5b03798835e8519acba3876c6416664ac6 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Thu, 3 Oct 2013 08:59:05 -0700 Subject: [PATCH 057/247] feat(store) create node with incremental suffix. accept #190 in new API --- command.go | 11 +++++---- etcd_handlers.go | 11 +++++++-- store/stats_test.go | 4 +-- store/store.go | 15 +++++++++--- store/store_test.go | 60 ++++++++++++++++++++++----------------------- 5 files changed, 58 insertions(+), 43 deletions(-) diff --git a/command.go b/command.go index bfda5f2708c..3f007f13a17 100644 --- a/command.go +++ b/command.go @@ -27,9 +27,10 @@ type Command interface { // Create command type CreateCommand struct { - Key string `json:"key"` - Value string `json:"value"` - ExpireTime time.Time `json:"expireTime"` + Key string `json:"key"` + Value string `json:"value"` + ExpireTime time.Time `json:"expireTime"` + IncrementalSuffix bool `json:"incrementalSuffix"` } // The name of the create command in the log @@ -39,7 +40,7 @@ func (c *CreateCommand) CommandName() string { // Create node func (c *CreateCommand) Apply(server *raft.Server) (interface{}, error) { - e, err := etcdStore.Create(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) + e, err := etcdStore.Create(c.Key, c.Value, c.IncrementalSuffix, c.ExpireTime, server.CommitIndex(), server.Term()) if err != nil { debug(err) @@ -221,7 +222,7 @@ func (c *JoinCommand) Apply(raftServer *raft.Server) (interface{}, error) { // add machine in etcd storage key := path.Join("_etcd/machines", c.Name) value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", c.RaftURL, c.EtcdURL, c.RaftVersion) - etcdStore.Create(key, value, store.Permanent, raftServer.CommitIndex(), raftServer.Term()) + etcdStore.Create(key, value, false, store.Permanent, raftServer.CommitIndex(), raftServer.Term()) // add peer stats if c.Name != r.Name() { diff --git a/etcd_handlers.go b/etcd_handlers.go index 4e802a5a013..5681cf6ddae 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -75,6 +75,7 @@ func Multiplexer(w http.ResponseWriter, req *http.Request) error { w.WriteHeader(http.StatusMethodNotAllowed) return nil } + return nil } @@ -102,6 +103,10 @@ func CreateHttpHandler(w http.ResponseWriter, req *http.Request) error { ExpireTime: expireTime, } + if req.FormValue("incremental") == "true" { + command.IncrementalSuffix = true + } + return dispatchEtcdCommand(command, w, req) } @@ -201,6 +206,7 @@ func LeaderHttpHandler(w http.ResponseWriter, req *http.Request) error { w.WriteHeader(http.StatusOK) raftURL, _ := nameToRaftURL(leader) w.Write([]byte(raftURL)) + return nil } else { return etcdErr.NewError(etcdErr.EcodeLeaderElect, "") @@ -213,6 +219,7 @@ func MachinesHttpHandler(w http.ResponseWriter, req *http.Request) error { w.WriteHeader(http.StatusOK) w.Write([]byte(strings.Join(machines, ", "))) + return nil } @@ -220,6 +227,7 @@ func MachinesHttpHandler(w http.ResponseWriter, req *http.Request) error { func VersionHttpHandler(w http.ResponseWriter, req *http.Request) error { w.WriteHeader(http.StatusOK) fmt.Fprintf(w, "etcd %s", releaseVersion) + return nil } @@ -277,7 +285,6 @@ func GetHttpHandler(w http.ResponseWriter, req *http.Request) error { } indexStr := req.FormValue("wait_index") - if indexStr != "" { sinceIndex, err := strconv.ParseUint(indexStr, 10, 64) @@ -297,7 +304,6 @@ func GetHttpHandler(w http.ResponseWriter, req *http.Request) error { } sorted := req.FormValue("sorted") - if sorted == "true" { command.Sorted = true } @@ -330,6 +336,7 @@ func TestHttpHandler(w http.ResponseWriter, req *http.Request) { directSet() w.WriteHeader(http.StatusOK) w.Write([]byte("speed test success")) + return } diff --git a/store/stats_test.go b/store/stats_test.go index 207df825fd4..80f71ff88e9 100644 --- a/store/stats_test.go +++ b/store/stats_test.go @@ -16,7 +16,7 @@ func TestBasicStats(t *testing.T) { for _, k := range keys { i++ - _, err := s.Create(k, "bar", time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1) + _, err := s.Create(k, "bar", false, time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1) if err != nil { SetFail++ } else { @@ -146,7 +146,7 @@ func TestBasicStats(t *testing.T) { for _, k := range keys { i++ - _, err := s.Create(k, "bar", time.Now().Add(time.Second*3), i, 1) + _, err := s.Create(k, "bar", false, time.Now().Add(time.Second*3), i, 1) if err != nil { SetFail++ } else { diff --git a/store/store.go b/store/store.go index 0f6a425f7dd..0113e8b111d 100644 --- a/store/store.go +++ b/store/store.go @@ -5,6 +5,7 @@ import ( "fmt" "path" "sort" + "strconv" "strings" "sync" "time" @@ -94,10 +95,12 @@ func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term // Create function creates the Node at nodePath. Create will help to create intermediate directories with no ttl. // If the node has already existed, create will fail. // If any node on the path is a file, create will fail. -func (s *Store) Create(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { +func (s *Store) Create(nodePath string, value string, incrementalSuffix bool, + expireTime time.Time, index uint64, term uint64) (*Event, error) { + s.worldLock.Lock() defer s.worldLock.Unlock() - return s.internalCreate(nodePath, value, expireTime, index, term, Create) + return s.internalCreate(nodePath, value, incrementalSuffix, expireTime, index, term, Create) } // Update function updates the value/ttl of the node. @@ -155,7 +158,7 @@ func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, defer s.worldLock.Unlock() if prevValue == "" && prevIndex == 0 { // try create - return s.internalCreate(nodePath, value, expireTime, index, term, TestAndSet) + return s.internalCreate(nodePath, value, false, expireTime, index, term, TestAndSet) } n, err := s.internalGet(nodePath, index, term) @@ -262,7 +265,11 @@ func (s *Store) walk(nodePath string, walkFunc func(prev *Node, component string return curr, nil } -func (s *Store) internalCreate(nodePath string, value string, expireTime time.Time, index uint64, term uint64, action string) (*Event, error) { +func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix bool, expireTime time.Time, index uint64, term uint64, action string) (*Event, error) { + if incrementalSuffix { // append unique incremental suffix to the node path + nodePath += "_" + strconv.FormatUint(index, 10) + } + nodePath = path.Clean(path.Join("/", nodePath)) // make sure we can create the node diff --git a/store/store_test.go b/store/store_test.go index 37dba4f1499..5a672d9b05c 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -10,10 +10,10 @@ import ( func TestCreateAndGet(t *testing.T) { s := New() - s.Create("/foobar", "bar", Permanent, 1, 1) + s.Create("/foobar", "bar", false, Permanent, 1, 1) // already exist, create should fail - _, err := s.Create("/foobar", "bar", Permanent, 1, 1) + _, err := s.Create("/foobar", "bar", false, Permanent, 1, 1) if err == nil { t.Fatal("Create should fail") @@ -27,14 +27,14 @@ func TestCreateAndGet(t *testing.T) { createAndGet(s, "/foo/foo/bar", t) // meet file, create should fail - _, err = s.Create("/foo/bar/bar", "bar", Permanent, 2, 1) + _, err = s.Create("/foo/bar/bar", "bar", false, Permanent, 2, 1) if err == nil { t.Fatal("Create should fail") } // create a directory - _, err = s.Create("/fooDir", "", Permanent, 3, 1) + _, err = s.Create("/fooDir", "", false, Permanent, 3, 1) if err != nil { t.Fatal("Cannot create /fooDir") @@ -47,7 +47,7 @@ func TestCreateAndGet(t *testing.T) { } // create a file under directory - _, err = s.Create("/fooDir/bar", "bar", Permanent, 4, 1) + _, err = s.Create("/fooDir/bar", "bar", false, Permanent, 4, 1) if err != nil { t.Fatal("Cannot create /fooDir/bar = bar") @@ -57,7 +57,7 @@ func TestCreateAndGet(t *testing.T) { func TestUpdateFile(t *testing.T) { s := New() - _, err := s.Create("/foo/bar", "bar", Permanent, 1, 1) + _, err := s.Create("/foo/bar", "bar", false, Permanent, 1, 1) if err != nil { t.Fatalf("cannot create %s=bar [%s]", "/foo/bar", err.Error()) @@ -80,24 +80,24 @@ func TestUpdateFile(t *testing.T) { } // create a directory, update its ttl, to see if it will be deleted - _, err = s.Create("/foo/foo", "", Permanent, 3, 1) + _, err = s.Create("/foo/foo", "", false, Permanent, 3, 1) if err != nil { t.Fatalf("cannot create dir [%s] [%s]", "/foo/foo", err.Error()) } - _, err = s.Create("/foo/foo/foo1", "bar1", Permanent, 4, 1) + _, err = s.Create("/foo/foo/foo1", "bar1", false, Permanent, 4, 1) if err != nil { t.Fatal("cannot create [%s]", err.Error()) } - _, err = s.Create("/foo/foo/foo2", "", Permanent, 5, 1) + _, err = s.Create("/foo/foo/foo2", "", false, Permanent, 5, 1) if err != nil { t.Fatal("cannot create [%s]", err.Error()) } - _, err = s.Create("/foo/foo/foo2/boo", "boo1", Permanent, 6, 1) + _, err = s.Create("/foo/foo/foo2/boo", "boo1", false, Permanent, 6, 1) if err != nil { t.Fatal("cannot create [%s]", err.Error()) } @@ -158,11 +158,11 @@ func TestListDirectory(t *testing.T) { // create dir /foo // set key-value /foo/foo=bar - s.Create("/foo/foo", "bar", Permanent, 1, 1) + s.Create("/foo/foo", "bar", false, Permanent, 1, 1) // create dir /foo/fooDir // set key-value /foo/fooDir/foo=bar - s.Create("/foo/fooDir/foo", "bar", Permanent, 2, 1) + s.Create("/foo/fooDir/foo", "bar", false, Permanent, 2, 1) e, err := s.Get("/foo", true, false, 2, 1) @@ -189,7 +189,7 @@ func TestListDirectory(t *testing.T) { // create dir /foo/_hidden // set key-value /foo/_hidden/foo -> bar - s.Create("/foo/_hidden/foo", "bar", Permanent, 3, 1) + s.Create("/foo/_hidden/foo", "bar", false, Permanent, 3, 1) e, _ = s.Get("/foo", false, false, 2, 1) @@ -201,7 +201,7 @@ func TestListDirectory(t *testing.T) { func TestRemove(t *testing.T) { s := New() - s.Create("/foo", "bar", Permanent, 1, 1) + s.Create("/foo", "bar", false, Permanent, 1, 1) _, err := s.Delete("/foo", false, 1, 1) if err != nil { @@ -214,9 +214,9 @@ func TestRemove(t *testing.T) { t.Fatalf("can get the node after deletion") } - s.Create("/foo/bar", "bar", Permanent, 1, 1) - s.Create("/foo/car", "car", Permanent, 1, 1) - s.Create("/foo/dar/dar", "dar", Permanent, 1, 1) + s.Create("/foo/bar", "bar", false, Permanent, 1, 1) + s.Create("/foo/car", "car", false, Permanent, 1, 1) + s.Create("/foo/dar/dar", "dar", false, Permanent, 1, 1) _, err = s.Delete("/foo", false, 1, 1) @@ -242,7 +242,7 @@ func TestExpire(t *testing.T) { expire := time.Now().Add(time.Second) - s.Create("/foo", "bar", expire, 1, 1) + s.Create("/foo", "bar", false, expire, 1, 1) _, err := s.Get("/foo", false, false, 1, 1) @@ -260,7 +260,7 @@ func TestExpire(t *testing.T) { // test if we can reach the node before expiration expire = time.Now().Add(time.Second) - s.Create("/foo", "bar", expire, 1, 1) + s.Create("/foo", "bar", false, expire, 1, 1) time.Sleep(time.Millisecond * 50) _, err = s.Get("/foo", false, false, 1, 1) @@ -271,7 +271,7 @@ func TestExpire(t *testing.T) { expire = time.Now().Add(time.Second) - s.Create("/foo", "bar", expire, 1, 1) + s.Create("/foo", "bar", false, expire, 1, 1) _, err = s.Delete("/foo", false, 1, 1) if err != nil { @@ -281,7 +281,7 @@ func TestExpire(t *testing.T) { func TestTestAndSet(t *testing.T) { // TODO prevValue == nil ? s := New() - s.Create("/foo", "bar", Permanent, 1, 1) + s.Create("/foo", "bar", false, Permanent, 1, 1) // test on wrong previous value _, err := s.TestAndSet("/foo", "barbar", 0, "car", Permanent, 2, 1) @@ -316,7 +316,7 @@ func TestWatch(t *testing.T) { s := New() // watch at a deeper path c, _ := s.Watch("/foo/foo/foo", false, 0, 0, 1) - s.Create("/foo/foo/foo", "bar", Permanent, 1, 1) + s.Create("/foo/foo/foo", "bar", false, Permanent, 1, 1) e := nonblockingRetrive(c) if e.Key != "/foo/foo/foo" || e.Action != Create { @@ -346,7 +346,7 @@ func TestWatch(t *testing.T) { // watch at a prefix c, _ = s.Watch("/foo", true, 0, 4, 1) - s.Create("/foo/foo/boo", "bar", Permanent, 5, 1) + s.Create("/foo/foo/boo", "bar", false, Permanent, 5, 1) e = nonblockingRetrive(c) if e.Key != "/foo/foo/boo" || e.Action != Create { t.Fatal("watch for Create subdirectory fails") @@ -374,7 +374,7 @@ func TestWatch(t *testing.T) { } // watch expire - s.Create("/foo/foo/boo", "foo", time.Now().Add(time.Second*1), 9, 1) + s.Create("/foo/foo/boo", "foo", false, time.Now().Add(time.Second*1), 9, 1) c, _ = s.Watch("/foo", true, 0, 9, 1) time.Sleep(time.Second * 2) e = nonblockingRetrive(c) @@ -382,7 +382,7 @@ func TestWatch(t *testing.T) { t.Fatal("watch for Expiration of Create() subdirectory fails ", e) } - s.Create("/foo/foo/boo", "foo", Permanent, 10, 1) + s.Create("/foo/foo/boo", "foo", false, Permanent, 10, 1) s.Update("/foo/foo/boo", "bar", time.Now().Add(time.Second*1), 11, 1) c, _ = s.Watch("/foo", true, 0, 11, 1) time.Sleep(time.Second * 2) @@ -391,7 +391,7 @@ func TestWatch(t *testing.T) { t.Fatal("watch for Expiration of Update() subdirectory fails ", e) } - s.Create("/foo/foo/boo", "foo", Permanent, 12, 1) + s.Create("/foo/foo/boo", "foo", false, Permanent, 12, 1) s.TestAndSet("/foo/foo/boo", "foo", 0, "bar", time.Now().Add(time.Second*1), 13, 1) c, _ = s.Watch("/foo", true, 0, 13, 1) time.Sleep(time.Second * 2) @@ -409,7 +409,7 @@ func TestSort(t *testing.T) { i := uint64(1) for _, k := range keys { - _, err := s.Create(k, "bar", Permanent, i, 1) + _, err := s.Create(k, "bar", false, Permanent, i, 1) if err != nil { panic(err) } else { @@ -447,7 +447,7 @@ func TestSaveAndRecover(t *testing.T) { i := uint64(1) for _, k := range keys { - _, err := s.Create(k, "bar", Permanent, i, 1) + _, err := s.Create(k, "bar", false, Permanent, i, 1) if err != nil { panic(err) } else { @@ -459,7 +459,7 @@ func TestSaveAndRecover(t *testing.T) { // test if we can reach the node before expiration expire := time.Now().Add(time.Second) - s.Create("/foo/foo", "bar", expire, 1, 1) + s.Create("/foo/foo", "bar", false, expire, 1, 1) b, err := s.Save() cloneFs := New() @@ -514,7 +514,7 @@ func GenKeys(num int, depth int) []string { } func createAndGet(s *Store, path string, t *testing.T) { - _, err := s.Create(path, "bar", Permanent, 1, 1) + _, err := s.Create(path, "bar", false, Permanent, 1, 1) if err != nil { t.Fatalf("cannot create %s=bar [%s]", path, err.Error()) From baaaf24f709788568ed8ad13bbee1817989c8876 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 6 Oct 2013 11:23:52 -0700 Subject: [PATCH 058/247] (feat) v1 apt backward support --- command.go | 20 ++-- error/error.go | 11 +- etcd_handler_v1.go | 248 +++++++++++++++++++++++++++++++++++++++++++ etcd_handlers.go | 33 ++++-- etcd_test.go | 6 +- store/event.go | 15 ++- store/node.go | 40 +++---- store/response_v1.go | 26 +++++ store/stats_test.go | 4 +- store/store.go | 85 +++++++++++---- store/store_test.go | 63 +++++------ store/watcher.go | 4 +- util.go | 31 ++++-- version.go | 2 +- 14 files changed, 464 insertions(+), 124 deletions(-) create mode 100644 etcd_handler_v1.go create mode 100644 store/response_v1.go diff --git a/command.go b/command.go index 3f007f13a17..7fe0049c8fb 100644 --- a/command.go +++ b/command.go @@ -2,7 +2,6 @@ package main import ( "encoding/binary" - "encoding/json" "fmt" "os" "path" @@ -31,6 +30,7 @@ type CreateCommand struct { Value string `json:"value"` ExpireTime time.Time `json:"expireTime"` IncrementalSuffix bool `json:"incrementalSuffix"` + Force bool `json:"force"` } // The name of the create command in the log @@ -40,14 +40,14 @@ func (c *CreateCommand) CommandName() string { // Create node func (c *CreateCommand) Apply(server *raft.Server) (interface{}, error) { - e, err := etcdStore.Create(c.Key, c.Value, c.IncrementalSuffix, c.ExpireTime, server.CommitIndex(), server.Term()) + e, err := etcdStore.Create(c.Key, c.Value, c.IncrementalSuffix, c.Force, c.ExpireTime, server.CommitIndex(), server.Term()) if err != nil { debug(err) return nil, err } - return json.Marshal(e) + return e, nil } // Update command @@ -71,7 +71,7 @@ func (c *UpdateCommand) Apply(server *raft.Server) (interface{}, error) { return nil, err } - return json.Marshal(e) + return e, nil } // TestAndSet command @@ -98,7 +98,7 @@ func (c *TestAndSetCommand) Apply(server *raft.Server) (interface{}, error) { return nil, err } - return json.Marshal(e) + return e, nil } // Get command @@ -122,7 +122,7 @@ func (c *GetCommand) Apply(server *raft.Server) (interface{}, error) { return nil, err } - return json.Marshal(e) + return e, nil } // Delete command @@ -145,7 +145,7 @@ func (c *DeleteCommand) Apply(server *raft.Server) (interface{}, error) { return nil, err } - return json.Marshal(e) + return e, nil } // Watch command @@ -169,7 +169,7 @@ func (c *WatchCommand) Apply(server *raft.Server) (interface{}, error) { e := <-eventChan - return json.Marshal(e) + return e, nil } // JoinCommand @@ -211,7 +211,7 @@ func (c *JoinCommand) Apply(raftServer *raft.Server) (interface{}, error) { num := machineNum() if num == maxClusterSize { debug("Reject join request from ", c.Name) - return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMoreMachine, "") + return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMoreMachine, "", raftServer.CommitIndex(), raftServer.Term()) } addNameToURL(c.Name, c.RaftVersion, c.RaftURL, c.EtcdURL) @@ -222,7 +222,7 @@ func (c *JoinCommand) Apply(raftServer *raft.Server) (interface{}, error) { // add machine in etcd storage key := path.Join("_etcd/machines", c.Name) value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", c.RaftURL, c.EtcdURL, c.RaftVersion) - etcdStore.Create(key, value, false, store.Permanent, raftServer.CommitIndex(), raftServer.Term()) + etcdStore.Create(key, value, false, false, store.Permanent, raftServer.CommitIndex(), raftServer.Term()) // add peer stats if c.Name != r.Name() { diff --git a/error/error.go b/error/error.go index 8d342de143f..22b55906b04 100644 --- a/error/error.go +++ b/error/error.go @@ -2,6 +2,7 @@ package error import ( "encoding/json" + "fmt" "net/http" ) @@ -62,13 +63,17 @@ type Error struct { ErrorCode int `json:"errorCode"` Message string `json:"message"` Cause string `json:"cause,omitempty"` + Index uint64 `json:"index"` + Term uint64 `json:"term"` } -func NewError(errorCode int, cause string) Error { - return Error{ +func NewError(errorCode int, cause string, index uint64, term uint64) *Error { + return &Error{ ErrorCode: errorCode, Message: errors[errorCode], Cause: cause, + Index: index, + Term: term, } } @@ -87,6 +92,8 @@ func (e Error) toJsonString() string { } func (e Error) Write(w http.ResponseWriter) { + w.Header().Add("X-Etcd-Index", fmt.Sprint(e.Index)) + w.Header().Add("X-Etcd-Term", fmt.Sprint(e.Term)) // 3xx is reft internal error if e.ErrorCode/100 == 3 { http.Error(w, e.toJsonString(), http.StatusInternalServerError) diff --git a/etcd_handler_v1.go b/etcd_handler_v1.go new file mode 100644 index 00000000000..44cde5a860e --- /dev/null +++ b/etcd_handler_v1.go @@ -0,0 +1,248 @@ +package main + +import ( + "encoding/json" + "net/http" + "strconv" + + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" +) + +//------------------------------------------------------------------- +// Handlers to handle etcd-store related request via etcd url +//------------------------------------------------------------------- +// Multiplex GET/POST/DELETE request to corresponding handlers +func MultiplexerV1(w http.ResponseWriter, req *http.Request) error { + + switch req.Method { + case "GET": + return GetHttpHandlerV1(w, req) + case "POST": + return SetHttpHandlerV1(w, req) + case "PUT": + return SetHttpHandlerV1(w, req) + case "DELETE": + return DeleteHttpHandlerV1(w, req) + default: + w.WriteHeader(http.StatusMethodNotAllowed) + return nil + } +} + +//-------------------------------------- +// State sensitive handlers +// Set/Delete will dispatch to leader +//-------------------------------------- + +// Set Command Handler +func SetHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { + key := req.URL.Path[len("/v1/keys/"):] + + debugf("[recv] POST %v/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) + + req.ParseForm() + + value := req.Form.Get("value") + + if len(value) == 0 { + return etcdErr.NewError(200, "Set", store.UndefIndex, store.UndefTerm) + } + + strDuration := req.Form.Get("ttl") + + expireTime, err := durationToExpireTime(strDuration) + + if err != nil { + return etcdErr.NewError(202, "Set", store.UndefIndex, store.UndefTerm) + } + + if prevValueArr, ok := req.Form["prevValue"]; ok && len(prevValueArr) > 0 { + command := &TestAndSetCommand{ + Key: key, + Value: value, + PrevValue: prevValueArr[0], + ExpireTime: expireTime, + } + + return dispatchEtcdCommandV1(command, w, req) + + } else { + command := &CreateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + Force: true, + } + + return dispatchEtcdCommandV1(command, w, req) + } +} + +// Delete Handler +func DeleteHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { + key := req.URL.Path[len("/v1/keys/"):] + + debugf("[recv] DELETE %v/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) + + command := &DeleteCommand{ + Key: key, + } + + return dispatchEtcdCommandV1(command, w, req) +} + +//-------------------------------------- +// State non-sensitive handlers +// will not dispatch to leader +// TODO: add sensitive version for these +// command? +//-------------------------------------- + +// Get Handler +func GetHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { + key := req.URL.Path[len("/v1/keys/"):] + + debugf("[recv] GET %s/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) + + command := &GetCommand{ + Key: key, + } + + if event, err := command.Apply(r.Server); err != nil { + return err + } else { + event, _ := event.(*store.Event) + + response := eventToResponse(event) + bytes, _ := json.Marshal(response) + + w.WriteHeader(http.StatusOK) + + w.Write(bytes) + + return nil + } + +} + +// Watch handler +func WatchHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { + key := req.URL.Path[len("/v1/watch/"):] + + command := &WatchCommand{ + Key: key, + } + + if req.Method == "GET" { + debugf("[recv] GET %s/watch/%s [%s]", e.url, key, req.RemoteAddr) + command.SinceIndex = 0 + + } else if req.Method == "POST" { + // watch from a specific index + + debugf("[recv] POST %s/watch/%s [%s]", e.url, key, req.RemoteAddr) + content := req.FormValue("index") + + sinceIndex, err := strconv.ParseUint(string(content), 10, 64) + if err != nil { + return etcdErr.NewError(203, "Watch From Index", store.UndefIndex, store.UndefTerm) + } + command.SinceIndex = sinceIndex + + } else { + w.WriteHeader(http.StatusMethodNotAllowed) + return nil + } + + if event, err := command.Apply(r.Server); err != nil { + return etcdErr.NewError(500, key, store.UndefIndex, store.UndefTerm) + } else { + event, _ := event.(*store.Event) + + response := eventToResponse(event) + bytes, _ := json.Marshal(response) + + w.WriteHeader(http.StatusOK) + + w.Write(bytes) + return nil + } + +} + +// Dispatch the command to leader +func dispatchEtcdCommandV1(c Command, w http.ResponseWriter, req *http.Request) error { + return dispatchV1(c, w, req, nameToEtcdURL) +} + +func dispatchV1(c Command, w http.ResponseWriter, req *http.Request, toURL func(name string) (string, bool)) error { + if r.State() == raft.Leader { + if event, err := r.Do(c); err != nil { + return err + } else { + if event == nil { + return etcdErr.NewError(300, "Empty result from raft", store.UndefIndex, store.UndefTerm) + } + + event, _ := event.(*store.Event) + + response := eventToResponse(event) + bytes, _ := json.Marshal(response) + + w.WriteHeader(http.StatusOK) + w.Write(bytes) + return nil + + } + + } else { + leader := r.Leader() + // current no leader + if leader == "" { + return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) + } + url, _ := toURL(leader) + + redirect(url, w, req) + + return nil + } +} + +func eventToResponse(event *store.Event) interface{} { + if !event.Dir { + response := &store.Response{ + Action: event.Action, + Key: event.Key, + Value: event.Value, + PrevValue: event.PrevValue, + Index: event.Index, + TTL: event.TTL, + Expiration: event.Expiration, + } + + if response.Action == store.Create || response.Action == store.Update { + response.Action = "set" + if response.PrevValue == "" { + response.NewKey = true + } + } + + return response + } else { + responses := make([]*store.Response, len(event.KVPairs)) + + for i, kv := range event.KVPairs { + responses[i] = &store.Response{ + Action: event.Action, + Key: kv.Key, + Value: kv.Value, + Dir: kv.Dir, + Index: event.Index, + } + } + return responses + } +} diff --git a/etcd_handlers.go b/etcd_handlers.go index 5681cf6ddae..21521be0d1a 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -1,6 +1,7 @@ package main import ( + "encoding/json" "fmt" "net/http" "strconv" @@ -23,6 +24,11 @@ func NewEtcdMuxer() *http.ServeMux { etcdMux.Handle("/"+version+"/machines", errorHandler(MachinesHttpHandler)) etcdMux.Handle("/"+version+"/stats/", errorHandler(StatsHttpHandler)) etcdMux.Handle("/version", errorHandler(VersionHttpHandler)) + + etcdMux.Handle("/v1/keys/", errorHandler(MultiplexerV1)) + etcdMux.Handle("/v1/leader", errorHandler(LeaderHttpHandler)) + etcdMux.Handle("/v1/machines", errorHandler(MachinesHttpHandler)) + etcdMux.Handle("/v1/stats/", errorHandler(StatsHttpHandler)) etcdMux.HandleFunc("/test/", TestHttpHandler) return etcdMux } @@ -50,8 +56,8 @@ func addCorsHeader(w http.ResponseWriter, r *http.Request) { func (fn errorHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { addCorsHeader(w, r) if e := fn(w, r); e != nil { - if etcdErr, ok := e.(etcdErr.Error); ok { - debug("Return error: ", etcdErr.Error()) + if etcdErr, ok := e.(*etcdErr.Error); ok { + debug("Return error: ", (*etcdErr).Error()) etcdErr.Write(w) } else { http.Error(w, e.Error(), http.StatusInternalServerError) @@ -94,7 +100,7 @@ func CreateHttpHandler(w http.ResponseWriter, req *http.Request) error { expireTime, err := durationToExpireTime(req.FormValue("ttl")) if err != nil { - return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create") + return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create", store.UndefIndex, store.UndefTerm) } command := &CreateCommand{ @@ -123,12 +129,12 @@ func UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { expireTime, err := durationToExpireTime(req.Form.Get("ttl")) if err != nil { - return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update") + return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", store.UndefIndex, store.UndefTerm) } // update should give at least one option if value == "" && expireTime.Sub(store.Permanent) == 0 { - return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update") + return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", store.UndefIndex, store.UndefTerm) } prevValue, valueOk := req.Form["prevValue"] @@ -152,7 +158,7 @@ func UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { // bad previous index if err != nil { - return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Update") + return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Update", store.UndefIndex, store.UndefTerm) } } else { prevIndex = 0 @@ -209,7 +215,7 @@ func LeaderHttpHandler(w http.ResponseWriter, req *http.Request) error { return nil } else { - return etcdErr.NewError(etcdErr.EcodeLeaderElect, "") + return etcdErr.NewError(etcdErr.EcodeLeaderElect, "", store.UndefIndex, store.UndefTerm) } } @@ -246,7 +252,7 @@ func StatsHttpHandler(w http.ResponseWriter, req *http.Request) error { leader := r.Leader() // current no leader if leader == "" { - return etcdErr.NewError(300, "") + return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) } hostname, _ := nameToEtcdURL(leader) redirect(hostname, w, req) @@ -289,7 +295,7 @@ func GetHttpHandler(w http.ResponseWriter, req *http.Request) error { sinceIndex, err := strconv.ParseUint(indexStr, 10, 64) if err != nil { - return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index") + return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index", store.UndefIndex, store.UndefTerm) } command.SinceIndex = sinceIndex @@ -319,9 +325,14 @@ func GetHttpHandler(w http.ResponseWriter, req *http.Request) error { return err } else { - event, _ := event.([]byte) + event, _ := event.(*store.Event) + bytes, _ := json.Marshal(event) + + w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) + w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term)) w.WriteHeader(http.StatusOK) - w.Write(event) + + w.Write(bytes) return nil } diff --git a/etcd_test.go b/etcd_test.go index 32c320ad6be..a62119e3599 100644 --- a/etcd_test.go +++ b/etcd_test.go @@ -49,7 +49,7 @@ func TestSingleNode(t *testing.T) { result, err = c.Set("foo", "bar", 100) - if err != nil || result.Key != "/foo" || result.Value != "bar" || result.PrevValue != "bar" || result.TTL != 99 { + if err != nil || result.Key != "/foo" || result.Value != "bar" || result.PrevValue != "bar" || result.TTL != 100 { if err != nil { t.Fatal(err) } @@ -61,7 +61,7 @@ func TestSingleNode(t *testing.T) { // First, we'll test we can change the value if we get it write result, match, err := c.TestAndSet("foo", "bar", "foobar", 100) - if err != nil || result.Key != "/foo" || result.Value != "foobar" || result.PrevValue != "bar" || result.TTL != 99 || !match { + if err != nil || result.Key != "/foo" || result.Value != "foobar" || result.PrevValue != "bar" || result.TTL != 100 || !match { if err != nil { t.Fatal(err) } @@ -223,7 +223,7 @@ func templateTestSimpleMultiNode(t *testing.T, tls bool) { result, err = c.Set("foo", "bar", 100) - if err != nil || result.Key != "/foo" || result.Value != "bar" || result.PrevValue != "bar" || result.TTL != 99 { + if err != nil || result.Key != "/foo" || result.Value != "bar" || result.PrevValue != "bar" || result.TTL != 100 { if err != nil { t.Fatal(err) } diff --git a/store/event.go b/store/event.go index 25562a9994d..ba4149a79b5 100644 --- a/store/event.go +++ b/store/event.go @@ -118,28 +118,24 @@ func (eh *EventHistory) addEvent(e *Event) *Event { if e.Index == UndefIndex { e.Index = eh.LastIndex - duped = 1 - } - - if e.Term == UndefTerm { e.Term = eh.LastTerm duped = 1 } eh.Queue.insert(e) - eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index - eh.LastIndex = e.Index eh.LastTerm = e.Term eh.DupCnt += duped + eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index + return e } // scan function is enumerating events from the index in history and // stops till the first point where the key has identified prefix -func (eh *EventHistory) scan(prefix string, index uint64) (*Event, error) { +func (eh *EventHistory) scan(prefix string, index uint64) (*Event, *etcdErr.Error) { eh.rwl.RLock() defer eh.rwl.RUnlock() @@ -150,7 +146,7 @@ func (eh *EventHistory) scan(prefix string, index uint64) (*Event, error) { return nil, etcdErr.NewError(etcdErr.EcodeEventIndexCleared, fmt.Sprintf("the requested history has been cleared [%v/%v]", - eh.StartIndex, index)) + eh.StartIndex, index), UndefIndex, UndefTerm) } // the index should locate before the size of the queue minus the duplicate count @@ -191,6 +187,9 @@ func (eh *EventHistory) clone() *EventHistory { return &EventHistory{ StartIndex: eh.StartIndex, Queue: clonedQueue, + LastIndex: eh.LastIndex, + LastTerm: eh.LastTerm, + DupCnt: eh.DupCnt, } } diff --git a/store/node.go b/store/node.go index e7d71cdb3fb..7b6326ceda3 100644 --- a/store/node.go +++ b/store/node.go @@ -63,7 +63,7 @@ func newDir(nodePath string, createIndex uint64, createTerm uint64, parent *Node // Remove function remove the node. // If the node is a directory and recursive is true, the function will recursively remove // add nodes under the receiver node. -func (n *Node) Remove(recursive bool, callback func(path string)) error { +func (n *Node) Remove(recursive bool, callback func(path string)) *etcdErr.Error { if n.status == removed { // check race between remove and expire return nil } @@ -89,7 +89,7 @@ func (n *Node) Remove(recursive bool, callback func(path string)) error { } if !recursive { - return etcdErr.NewError(etcdErr.EcodeNotFile, "") + return etcdErr.NewError(etcdErr.EcodeNotFile, "", UndefIndex, UndefTerm) } for _, child := range n.Children { // delete all children @@ -114,9 +114,9 @@ func (n *Node) Remove(recursive bool, callback func(path string)) error { // Read function gets the value of the node. // If the receiver node is not a key-value pair, a "Not A File" error will be returned. -func (n *Node) Read() (string, error) { +func (n *Node) Read() (string, *etcdErr.Error) { if n.IsDir() { - return "", etcdErr.NewError(etcdErr.EcodeNotFile, "") + return "", etcdErr.NewError(etcdErr.EcodeNotFile, "", UndefIndex, UndefTerm) } return n.Value, nil @@ -124,9 +124,9 @@ func (n *Node) Read() (string, error) { // Write function set the value of the node to the given value. // If the receiver node is a directory, a "Not A File" error will be returned. -func (n *Node) Write(value string, index uint64, term uint64) error { +func (n *Node) Write(value string, index uint64, term uint64) *etcdErr.Error { if n.IsDir() { - return etcdErr.NewError(etcdErr.EcodeNotFile, "") + return etcdErr.NewError(etcdErr.EcodeNotFile, "", UndefIndex, UndefTerm) } n.Value = value @@ -138,9 +138,9 @@ func (n *Node) Write(value string, index uint64, term uint64) error { // List function return a slice of nodes under the receiver node. // If the receiver node is not a directory, a "Not A Directory" error will be returned. -func (n *Node) List() ([]*Node, error) { +func (n *Node) List() ([]*Node, *etcdErr.Error) { if !n.IsDir() { - return nil, etcdErr.NewError(etcdErr.EcodeNotDir, "") + return nil, etcdErr.NewError(etcdErr.EcodeNotDir, "", UndefIndex, UndefTerm) } nodes := make([]*Node, len(n.Children)) @@ -154,25 +154,17 @@ func (n *Node) List() ([]*Node, error) { return nodes, nil } -// GetFile function returns the file node under the directory node. +// GetChild function returns the child node under the directory node. // On success, it returns the file node -// If the node that calls this function is not a directory, it returns -// Not Directory Error -// If the node corresponding to the name string is not file, it returns -// Not File Error -func (n *Node) GetFile(name string) (*Node, error) { +func (n *Node) GetChild(name string) (*Node, *etcdErr.Error) { if !n.IsDir() { - return nil, etcdErr.NewError(etcdErr.EcodeNotDir, n.Path) + return nil, etcdErr.NewError(etcdErr.EcodeNotDir, n.Path, UndefIndex, UndefTerm) } - f, ok := n.Children[name] + child, ok := n.Children[name] if ok { - if !f.IsDir() { - return f, nil - } else { - return nil, etcdErr.NewError(etcdErr.EcodeNotFile, f.Path) - } + return child, nil } return nil, nil @@ -182,9 +174,9 @@ func (n *Node) GetFile(name string) (*Node, error) { // If the receiver is not a directory, a "Not A Directory" error will be returned. // If there is a existing node with the same name under the directory, a "Already Exist" // error will be returned -func (n *Node) Add(child *Node) error { +func (n *Node) Add(child *Node) *etcdErr.Error { if !n.IsDir() { - return etcdErr.NewError(etcdErr.EcodeNotDir, "") + return etcdErr.NewError(etcdErr.EcodeNotDir, "", UndefIndex, UndefTerm) } _, name := path.Split(child.Path) @@ -192,7 +184,7 @@ func (n *Node) Add(child *Node) error { _, ok := n.Children[name] if ok { - return etcdErr.NewError(etcdErr.EcodeNodeExist, "") + return etcdErr.NewError(etcdErr.EcodeNodeExist, "", UndefIndex, UndefTerm) } n.Children[name] = child diff --git a/store/response_v1.go b/store/response_v1.go new file mode 100644 index 00000000000..9ff8aa2ac79 --- /dev/null +++ b/store/response_v1.go @@ -0,0 +1,26 @@ +package store + +import ( + "time" +) + +// The response from the store to the user who issue a command +type Response struct { + Action string `json:"action"` + Key string `json:"key"` + Dir bool `json:"dir,omitempty"` + PrevValue string `json:"prevValue,omitempty"` + Value string `json:"value,omitempty"` + + // If the key did not exist before the action, + // this field should be set to true + NewKey bool `json:"newKey,omitempty"` + + Expiration *time.Time `json:"expiration,omitempty"` + + // Time to live in second + TTL int64 `json:"ttl,omitempty"` + + // The command index of the raft machine when the command is executed + Index uint64 `json:"index"` +} diff --git a/store/stats_test.go b/store/stats_test.go index 80f71ff88e9..deeb317dc1d 100644 --- a/store/stats_test.go +++ b/store/stats_test.go @@ -16,7 +16,7 @@ func TestBasicStats(t *testing.T) { for _, k := range keys { i++ - _, err := s.Create(k, "bar", false, time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1) + _, err := s.Create(k, "bar", false, false, time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1) if err != nil { SetFail++ } else { @@ -146,7 +146,7 @@ func TestBasicStats(t *testing.T) { for _, k := range keys { i++ - _, err := s.Create(k, "bar", false, time.Now().Add(time.Second*3), i, 1) + _, err := s.Create(k, "bar", false, false, time.Now().Add(time.Second*3), i, 1) if err != nil { SetFail++ } else { diff --git a/store/store.go b/store/store.go index 0113e8b111d..704ad26b75c 100644 --- a/store/store.go +++ b/store/store.go @@ -95,12 +95,13 @@ func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term // Create function creates the Node at nodePath. Create will help to create intermediate directories with no ttl. // If the node has already existed, create will fail. // If any node on the path is a file, create will fail. -func (s *Store) Create(nodePath string, value string, incrementalSuffix bool, +func (s *Store) Create(nodePath string, value string, incrementalSuffix bool, force bool, expireTime time.Time, index uint64, term uint64) (*Event, error) { + nodePath = path.Clean(path.Join("/", nodePath)) s.worldLock.Lock() defer s.worldLock.Unlock() - return s.internalCreate(nodePath, value, incrementalSuffix, expireTime, index, term, Create) + return s.internalCreate(nodePath, value, incrementalSuffix, force, expireTime, index, term, Create) } // Update function updates the value/ttl of the node. @@ -109,6 +110,7 @@ func (s *Store) Create(nodePath string, value string, incrementalSuffix bool, func (s *Store) Update(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { s.worldLock.Lock() defer s.worldLock.Unlock() + nodePath = path.Clean(path.Join("/", nodePath)) n, err := s.internalGet(nodePath, index, term) @@ -124,7 +126,8 @@ func (s *Store) Update(nodePath string, value string, expireTime time.Time, inde if len(value) != 0 { s.Stats.Inc(UpdateFail) - return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath) + err := etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term) + return nil, err } } else { // if the node is a file, we can update value and ttl @@ -154,11 +157,13 @@ func (s *Store) Update(nodePath string, value string, expireTime time.Time, inde func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { + nodePath = path.Clean(path.Join("/", nodePath)) + s.worldLock.Lock() defer s.worldLock.Unlock() if prevValue == "" && prevIndex == 0 { // try create - return s.internalCreate(nodePath, value, false, expireTime, index, term, TestAndSet) + return s.internalCreate(nodePath, value, false, false, expireTime, index, term, TestAndSet) } n, err := s.internalGet(nodePath, index, term) @@ -170,7 +175,7 @@ func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, if n.IsDir() { // can only test and set file s.Stats.Inc(TestAndSetFail) - return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath) + return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term) } if n.Value == prevValue || n.ModifiedIndex == prevIndex { @@ -182,6 +187,11 @@ func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, n.UpdateTTL(expireTime, s) + if n.ExpireTime.Sub(Permanent) != 0 { + e.Expiration = &n.ExpireTime + e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 + } + s.WatcherHub.notify(e) s.Stats.Inc(TestAndSetSuccess) return e, nil @@ -189,12 +199,14 @@ func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, cause := fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, n.Value, prevIndex, n.ModifiedIndex) s.Stats.Inc(TestAndSetFail) - return nil, etcdErr.NewError(etcdErr.EcodeTestFailed, cause) + return nil, etcdErr.NewError(etcdErr.EcodeTestFailed, cause, index, term) } // Delete function deletes the node at the given path. // If the node is a directory, recursive must be true to delete it. func (s *Store) Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error) { + nodePath = path.Clean(path.Join("/", nodePath)) + s.worldLock.Lock() defer s.worldLock.Unlock() @@ -231,25 +243,38 @@ func (s *Store) Delete(nodePath string, recursive bool, index uint64, term uint6 } func (s *Store) Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error) { + prefix = path.Clean(path.Join("/", prefix)) + s.worldLock.RLock() defer s.worldLock.RUnlock() s.Index, s.Term = index, term + var c <-chan *Event + var err *etcdErr.Error + if sinceIndex == 0 { - return s.WatcherHub.watch(prefix, recursive, index+1) + c, err = s.WatcherHub.watch(prefix, recursive, index+1) + + } else { + c, err = s.WatcherHub.watch(prefix, recursive, sinceIndex) } - return s.WatcherHub.watch(prefix, recursive, sinceIndex) + if err != nil { + err.Index = index + err.Term = term + } + + return c, err } // walk function walks all the nodePath and apply the walkFunc on each directory -func (s *Store) walk(nodePath string, walkFunc func(prev *Node, component string) (*Node, error)) (*Node, error) { +func (s *Store) walk(nodePath string, walkFunc func(prev *Node, component string) (*Node, *etcdErr.Error)) (*Node, *etcdErr.Error) { components := strings.Split(nodePath, "/") curr := s.Root + var err *etcdErr.Error - var err error for i := 1; i < len(components); i++ { if len(components[i]) == 0 { // ignore empty string return curr, nil @@ -265,7 +290,9 @@ func (s *Store) walk(nodePath string, walkFunc func(prev *Node, component string return curr, nil } -func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix bool, expireTime time.Time, index uint64, term uint64, action string) (*Event, error) { +func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix bool, force bool, + expireTime time.Time, index uint64, term uint64, action string) (*Event, error) { + if incrementalSuffix { // append unique incremental suffix to the node path nodePath += "_" + strconv.FormatUint(index, 10) } @@ -275,30 +302,42 @@ func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix // make sure we can create the node _, err := s.internalGet(nodePath, index, term) - if err == nil { // key already exists + if err == nil && !force { // key already exists s.Stats.Inc(SetFail) - return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath) + return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath, index, term) } - etcdError, _ := err.(etcdErr.Error) - - if etcdError.ErrorCode == 104 { // we cannot create the key due to meet a file while walking + if err != nil && err.ErrorCode == 104 { // we cannot create the key due to meet a file while walking s.Stats.Inc(SetFail) return nil, err } - dir, _ := path.Split(nodePath) + dir, newNodeName := path.Split(nodePath) // walk through the nodePath, create dirs and get the last directory node d, err := s.walk(dir, s.checkDir) if err != nil { s.Stats.Inc(SetFail) + fmt.Println("1: bad create") return nil, err } e := newEvent(action, nodePath, s.Index, s.Term) + if force { // force will try to replace a existing file + n, _ := d.GetChild(newNodeName) + if n != nil { + if n.IsDir() { + return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term) + } + e.PrevValue, _ = n.Read() + + n.Remove(false, nil) + + } + } + var n *Node if len(value) != 0 { // create file @@ -333,16 +372,17 @@ func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix } // InternalGet function get the node of the given nodePath. -func (s *Store) internalGet(nodePath string, index uint64, term uint64) (*Node, error) { +func (s *Store) internalGet(nodePath string, index uint64, term uint64) (*Node, *etcdErr.Error) { nodePath = path.Clean(path.Join("/", nodePath)) // update file system known index and term s.Index, s.Term = index, term - walkFunc := func(parent *Node, name string) (*Node, error) { + walkFunc := func(parent *Node, name string) (*Node, *etcdErr.Error) { if !parent.IsDir() { - return nil, etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path) + err := etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path, index, term) + return nil, err } child, ok := parent.Children[name] @@ -350,7 +390,7 @@ func (s *Store) internalGet(nodePath string, index uint64, term uint64) (*Node, return child, nil } - return nil, etcdErr.NewError(etcdErr.EcodeKeyNotFound, path.Join(parent.Path, name)) + return nil, etcdErr.NewError(etcdErr.EcodeKeyNotFound, path.Join(parent.Path, name), index, term) } f, err := s.walk(nodePath, walkFunc) @@ -358,7 +398,6 @@ func (s *Store) internalGet(nodePath string, index uint64, term uint64) (*Node, if err != nil { return nil, err } - return f, nil } @@ -366,7 +405,7 @@ func (s *Store) internalGet(nodePath string, index uint64, term uint64) (*Node, // If it is a directory, this function will return the pointer to that node. // If it does not exist, this function will create a new directory and return the pointer to that node. // If it is a file, this function will return error. -func (s *Store) checkDir(parent *Node, dirName string) (*Node, error) { +func (s *Store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) { subDir, ok := parent.Children[dirName] if ok { diff --git a/store/store_test.go b/store/store_test.go index 5a672d9b05c..0a0f971e6f0 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -10,10 +10,10 @@ import ( func TestCreateAndGet(t *testing.T) { s := New() - s.Create("/foobar", "bar", false, Permanent, 1, 1) + s.Create("/foobar", "bar", false, false, Permanent, 1, 1) // already exist, create should fail - _, err := s.Create("/foobar", "bar", false, Permanent, 1, 1) + _, err := s.Create("/foobar", "bar", false, false, Permanent, 1, 1) if err == nil { t.Fatal("Create should fail") @@ -27,14 +27,14 @@ func TestCreateAndGet(t *testing.T) { createAndGet(s, "/foo/foo/bar", t) // meet file, create should fail - _, err = s.Create("/foo/bar/bar", "bar", false, Permanent, 2, 1) + _, err = s.Create("/foo/bar/bar", "bar", false, false, Permanent, 2, 1) if err == nil { t.Fatal("Create should fail") } // create a directory - _, err = s.Create("/fooDir", "", false, Permanent, 3, 1) + _, err = s.Create("/fooDir", "", false, false, Permanent, 3, 1) if err != nil { t.Fatal("Cannot create /fooDir") @@ -47,7 +47,7 @@ func TestCreateAndGet(t *testing.T) { } // create a file under directory - _, err = s.Create("/fooDir/bar", "bar", false, Permanent, 4, 1) + _, err = s.Create("/fooDir/bar", "bar", false, false, Permanent, 4, 1) if err != nil { t.Fatal("Cannot create /fooDir/bar = bar") @@ -57,7 +57,7 @@ func TestCreateAndGet(t *testing.T) { func TestUpdateFile(t *testing.T) { s := New() - _, err := s.Create("/foo/bar", "bar", false, Permanent, 1, 1) + _, err := s.Create("/foo/bar", "bar", false, false, Permanent, 1, 1) if err != nil { t.Fatalf("cannot create %s=bar [%s]", "/foo/bar", err.Error()) @@ -80,24 +80,24 @@ func TestUpdateFile(t *testing.T) { } // create a directory, update its ttl, to see if it will be deleted - _, err = s.Create("/foo/foo", "", false, Permanent, 3, 1) + _, err = s.Create("/foo/foo", "", false, false, Permanent, 3, 1) if err != nil { t.Fatalf("cannot create dir [%s] [%s]", "/foo/foo", err.Error()) } - _, err = s.Create("/foo/foo/foo1", "bar1", false, Permanent, 4, 1) + _, err = s.Create("/foo/foo/foo1", "bar1", false, false, Permanent, 4, 1) if err != nil { t.Fatal("cannot create [%s]", err.Error()) } - _, err = s.Create("/foo/foo/foo2", "", false, Permanent, 5, 1) + _, err = s.Create("/foo/foo/foo2", "", false, false, Permanent, 5, 1) if err != nil { t.Fatal("cannot create [%s]", err.Error()) } - _, err = s.Create("/foo/foo/foo2/boo", "boo1", false, Permanent, 6, 1) + _, err = s.Create("/foo/foo/foo2/boo", "boo1", false, false, Permanent, 6, 1) if err != nil { t.Fatal("cannot create [%s]", err.Error()) } @@ -158,11 +158,11 @@ func TestListDirectory(t *testing.T) { // create dir /foo // set key-value /foo/foo=bar - s.Create("/foo/foo", "bar", false, Permanent, 1, 1) + s.Create("/foo/foo", "bar", false, false, Permanent, 1, 1) // create dir /foo/fooDir // set key-value /foo/fooDir/foo=bar - s.Create("/foo/fooDir/foo", "bar", false, Permanent, 2, 1) + s.Create("/foo/fooDir/foo", "bar", false, false, Permanent, 2, 1) e, err := s.Get("/foo", true, false, 2, 1) @@ -189,7 +189,7 @@ func TestListDirectory(t *testing.T) { // create dir /foo/_hidden // set key-value /foo/_hidden/foo -> bar - s.Create("/foo/_hidden/foo", "bar", false, Permanent, 3, 1) + s.Create("/foo/_hidden/foo", "bar", false, false, Permanent, 3, 1) e, _ = s.Get("/foo", false, false, 2, 1) @@ -201,7 +201,7 @@ func TestListDirectory(t *testing.T) { func TestRemove(t *testing.T) { s := New() - s.Create("/foo", "bar", false, Permanent, 1, 1) + s.Create("/foo", "bar", false, false, Permanent, 1, 1) _, err := s.Delete("/foo", false, 1, 1) if err != nil { @@ -214,9 +214,9 @@ func TestRemove(t *testing.T) { t.Fatalf("can get the node after deletion") } - s.Create("/foo/bar", "bar", false, Permanent, 1, 1) - s.Create("/foo/car", "car", false, Permanent, 1, 1) - s.Create("/foo/dar/dar", "dar", false, Permanent, 1, 1) + s.Create("/foo/bar", "bar", false, false, Permanent, 1, 1) + s.Create("/foo/car", "car", false, false, Permanent, 1, 1) + s.Create("/foo/dar/dar", "dar", false, false, Permanent, 1, 1) _, err = s.Delete("/foo", false, 1, 1) @@ -242,7 +242,7 @@ func TestExpire(t *testing.T) { expire := time.Now().Add(time.Second) - s.Create("/foo", "bar", false, expire, 1, 1) + s.Create("/foo", "bar", false, false, expire, 1, 1) _, err := s.Get("/foo", false, false, 1, 1) @@ -260,7 +260,7 @@ func TestExpire(t *testing.T) { // test if we can reach the node before expiration expire = time.Now().Add(time.Second) - s.Create("/foo", "bar", false, expire, 1, 1) + s.Create("/foo", "bar", false, false, expire, 1, 1) time.Sleep(time.Millisecond * 50) _, err = s.Get("/foo", false, false, 1, 1) @@ -271,7 +271,7 @@ func TestExpire(t *testing.T) { expire = time.Now().Add(time.Second) - s.Create("/foo", "bar", false, expire, 1, 1) + s.Create("/foo", "bar", false, false, expire, 1, 1) _, err = s.Delete("/foo", false, 1, 1) if err != nil { @@ -281,7 +281,7 @@ func TestExpire(t *testing.T) { func TestTestAndSet(t *testing.T) { // TODO prevValue == nil ? s := New() - s.Create("/foo", "bar", false, Permanent, 1, 1) + s.Create("/foo", "bar", false, false, Permanent, 1, 1) // test on wrong previous value _, err := s.TestAndSet("/foo", "barbar", 0, "car", Permanent, 2, 1) @@ -316,7 +316,7 @@ func TestWatch(t *testing.T) { s := New() // watch at a deeper path c, _ := s.Watch("/foo/foo/foo", false, 0, 0, 1) - s.Create("/foo/foo/foo", "bar", false, Permanent, 1, 1) + s.Create("/foo/foo/foo", "bar", false, false, Permanent, 1, 1) e := nonblockingRetrive(c) if e.Key != "/foo/foo/foo" || e.Action != Create { @@ -346,7 +346,7 @@ func TestWatch(t *testing.T) { // watch at a prefix c, _ = s.Watch("/foo", true, 0, 4, 1) - s.Create("/foo/foo/boo", "bar", false, Permanent, 5, 1) + s.Create("/foo/foo/boo", "bar", false, false, Permanent, 5, 1) e = nonblockingRetrive(c) if e.Key != "/foo/foo/boo" || e.Action != Create { t.Fatal("watch for Create subdirectory fails") @@ -374,7 +374,7 @@ func TestWatch(t *testing.T) { } // watch expire - s.Create("/foo/foo/boo", "foo", false, time.Now().Add(time.Second*1), 9, 1) + s.Create("/foo/foo/boo", "foo", false, false, time.Now().Add(time.Second*1), 9, 1) c, _ = s.Watch("/foo", true, 0, 9, 1) time.Sleep(time.Second * 2) e = nonblockingRetrive(c) @@ -382,7 +382,7 @@ func TestWatch(t *testing.T) { t.Fatal("watch for Expiration of Create() subdirectory fails ", e) } - s.Create("/foo/foo/boo", "foo", false, Permanent, 10, 1) + s.Create("/foo/foo/boo", "foo", false, false, Permanent, 10, 1) s.Update("/foo/foo/boo", "bar", time.Now().Add(time.Second*1), 11, 1) c, _ = s.Watch("/foo", true, 0, 11, 1) time.Sleep(time.Second * 2) @@ -391,7 +391,7 @@ func TestWatch(t *testing.T) { t.Fatal("watch for Expiration of Update() subdirectory fails ", e) } - s.Create("/foo/foo/boo", "foo", false, Permanent, 12, 1) + s.Create("/foo/foo/boo", "foo", false, false, Permanent, 12, 1) s.TestAndSet("/foo/foo/boo", "foo", 0, "bar", time.Now().Add(time.Second*1), 13, 1) c, _ = s.Watch("/foo", true, 0, 13, 1) time.Sleep(time.Second * 2) @@ -409,7 +409,7 @@ func TestSort(t *testing.T) { i := uint64(1) for _, k := range keys { - _, err := s.Create(k, "bar", false, Permanent, i, 1) + _, err := s.Create(k, "bar", false, false, Permanent, i, 1) if err != nil { panic(err) } else { @@ -447,7 +447,7 @@ func TestSaveAndRecover(t *testing.T) { i := uint64(1) for _, k := range keys { - _, err := s.Create(k, "bar", false, Permanent, i, 1) + _, err := s.Create(k, "bar", false, false, Permanent, i, 1) if err != nil { panic(err) } else { @@ -459,7 +459,7 @@ func TestSaveAndRecover(t *testing.T) { // test if we can reach the node before expiration expire := time.Now().Add(time.Second) - s.Create("/foo/foo", "bar", false, expire, 1, 1) + s.Create("/foo/foo", "bar", false, false, expire, 1, 1) b, err := s.Save() cloneFs := New() @@ -479,7 +479,8 @@ func TestSaveAndRecover(t *testing.T) { defer s.worldLock.RUnlock() if s.WatcherHub.EventHistory.StartIndex != cloneFs.WatcherHub.EventHistory.StartIndex { - t.Fatal("Error recovered event history start index") + t.Fatalf("Error recovered event history start index[%v/%v]", + s.WatcherHub.EventHistory.StartIndex, cloneFs.WatcherHub.EventHistory.StartIndex) } for i = 0; int(i) < cloneFs.WatcherHub.EventHistory.Queue.Size; i++ { @@ -514,7 +515,7 @@ func GenKeys(num int, depth int) []string { } func createAndGet(s *Store, path string, t *testing.T) { - _, err := s.Create(path, "bar", false, Permanent, 1, 1) + _, err := s.Create(path, "bar", false, false, Permanent, 1, 1) if err != nil { t.Fatalf("cannot create %s=bar [%s]", path, err.Error()) diff --git a/store/watcher.go b/store/watcher.go index 5d96e45cd3e..7d8d3418500 100644 --- a/store/watcher.go +++ b/store/watcher.go @@ -5,6 +5,8 @@ import ( "path" "strings" "sync/atomic" + + etcdErr "github.com/coreos/etcd/error" ) type watcherHub struct { @@ -30,7 +32,7 @@ func newWatchHub(capacity int) *watcherHub { // If recursive is true, the first change after index under prefix will be sent to the event channel. // If recursive is false, the first change after index at prefix will be sent to the event channel. // If index is zero, watch will start from the current index + 1. -func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (<-chan *Event, error) { +func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (<-chan *Event, *etcdErr.Error) { eventChan := make(chan *Event, 1) e, err := wh.EventHistory.scan(prefix, index) diff --git a/util.go b/util.go index 2eb360baf12..edecb97e49e 100644 --- a/util.go +++ b/util.go @@ -69,24 +69,40 @@ func startWebInterface() { func dispatch(c Command, w http.ResponseWriter, req *http.Request, toURL func(name string) (string, bool)) error { if r.State() == raft.Leader { - if body, err := r.Do(c); err != nil { + if response, err := r.Do(c); err != nil { return err } else { - if body == nil { - return etcdErr.NewError(300, "Empty result from raft") - } else { - body, _ := body.([]byte) + if response == nil { + return etcdErr.NewError(300, "Empty response from raft", store.UndefIndex, store.UndefTerm) + } + + event, ok := response.(*store.Event) + if ok { + bytes, err := json.Marshal(event) + if err != nil { + fmt.Println(err) + } + + w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) + w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term)) w.WriteHeader(http.StatusOK) - w.Write(body) + w.Write(bytes) + return nil } + + bytes, _ := response.([]byte) + w.WriteHeader(http.StatusOK) + w.Write(bytes) + + return nil } } else { leader := r.Leader() // current no leader if leader == "" { - return etcdErr.NewError(300, "") + return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) } url, _ := toURL(leader) @@ -94,7 +110,6 @@ func dispatch(c Command, w http.ResponseWriter, req *http.Request, toURL func(na return nil } - return etcdErr.NewError(300, "") } func redirect(hostname string, w http.ResponseWriter, req *http.Request) { diff --git a/version.go b/version.go index 2302ea205ef..20d31c91661 100644 --- a/version.go +++ b/version.go @@ -1,6 +1,6 @@ package main -const version = "v1" +const version = "v2" // TODO: The release version (generated from the git tag) will be the raft // protocol version for now. When things settle down we will fix it like the From a71838a59b67a3e2ccefe7ca92800517f36658e6 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 7 Oct 2013 22:17:58 -0700 Subject: [PATCH 059/247] refactor watcher.go --- etcd_handlers.go | 4 +- store/node.go | 3 + store/store.go | 3 +- store/store_test.go | 4 -- store/watcher.go | 130 ++++++--------------------------------- store/watcher_hub.go | 142 +++++++++++++++++++++++++++++++++++++++++++ 6 files changed, 169 insertions(+), 117 deletions(-) create mode 100644 store/watcher_hub.go diff --git a/etcd_handlers.go b/etcd_handlers.go index 21521be0d1a..fe64347a3aa 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -24,12 +24,14 @@ func NewEtcdMuxer() *http.ServeMux { etcdMux.Handle("/"+version+"/machines", errorHandler(MachinesHttpHandler)) etcdMux.Handle("/"+version+"/stats/", errorHandler(StatsHttpHandler)) etcdMux.Handle("/version", errorHandler(VersionHttpHandler)) + etcdMux.HandleFunc("/test/", TestHttpHandler) + // backward support etcdMux.Handle("/v1/keys/", errorHandler(MultiplexerV1)) etcdMux.Handle("/v1/leader", errorHandler(LeaderHttpHandler)) etcdMux.Handle("/v1/machines", errorHandler(MachinesHttpHandler)) etcdMux.Handle("/v1/stats/", errorHandler(StatsHttpHandler)) - etcdMux.HandleFunc("/test/", TestHttpHandler) + return etcdMux } diff --git a/store/node.go b/store/node.go index 7b6326ceda3..4117225cba3 100644 --- a/store/node.go +++ b/store/node.go @@ -349,6 +349,9 @@ func (n *Node) Pair(recurisive, sorted bool) KeyValuePair { func (n *Node) UpdateTTL(expireTime time.Time, s *Store) { if !n.IsPermanent() { + // check if the node has been expired + // if the node is not expired, we need to stop the go routine associated with + // that node. expired, _ := n.IsExpired() if !expired { diff --git a/store/store.go b/store/store.go index 704ad26b75c..b8830fe5eb1 100644 --- a/store/store.go +++ b/store/store.go @@ -226,7 +226,8 @@ func (s *Store) Delete(nodePath string, recursive bool, index uint64, term uint6 } callback := func(path string) { // notify function - s.WatcherHub.notifyWithPath(e, path, true) + // notify the watchers with delted set true + s.WatcherHub.notifyWatchers(e, path, true) } err = n.Remove(recursive, callback) diff --git a/store/store_test.go b/store/store_test.go index 0a0f971e6f0..3d5e9e97005 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -124,10 +124,6 @@ func TestUpdateFile(t *testing.T) { t.Fatalf("cannot get sub dir before expiration [%s]", err.Error()) } - /*if e.KVPairs[2].Key != "/foo/foo/foo2/boo" || e.KVPairs[2].Value != "boo1" { - t.Fatalf("cannot get sub node of sub dir before expiration [%s]", err.Error()) - }*/ - // wait for expiration time.Sleep(time.Second * 3) e, err = s.Get("/foo/foo", true, false, 7, 1) diff --git a/store/watcher.go b/store/watcher.go index 7d8d3418500..b9cb5499ca7 100644 --- a/store/watcher.go +++ b/store/watcher.go @@ -1,125 +1,33 @@ package store -import ( - "container/list" - "path" - "strings" - "sync/atomic" - - etcdErr "github.com/coreos/etcd/error" -) - -type watcherHub struct { - watchers map[string]*list.List - count int64 // current number of watchers. - EventHistory *EventHistory -} - type watcher struct { eventChan chan *Event recursive bool sinceIndex uint64 } -func newWatchHub(capacity int) *watcherHub { - return &watcherHub{ - watchers: make(map[string]*list.List), - EventHistory: newEventHistory(capacity), - } -} - -// watch function returns an Event channel. -// If recursive is true, the first change after index under prefix will be sent to the event channel. -// If recursive is false, the first change after index at prefix will be sent to the event channel. -// If index is zero, watch will start from the current index + 1. -func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (<-chan *Event, *etcdErr.Error) { - eventChan := make(chan *Event, 1) - - e, err := wh.EventHistory.scan(prefix, index) - - if err != nil { - return nil, err - } - - if e != nil { - eventChan <- e - return eventChan, nil - } - - w := &watcher{ - eventChan: eventChan, - recursive: recursive, - sinceIndex: index - 1, // to catch Expire() - } - - l, ok := wh.watchers[prefix] - - if ok { // add the new watcher to the back of the list - l.PushBack(w) - - } else { // create a new list and add the new watcher - l := list.New() - l.PushBack(w) - wh.watchers[prefix] = l - } - - atomic.AddInt64(&wh.count, 1) - - return eventChan, nil -} - -func (wh *watcherHub) notifyWithPath(e *Event, path string, force bool) { - l, ok := wh.watchers[path] - - if ok { - curr := l.Front() - notifiedAll := true - - for { - if curr == nil { // we have reached the end of the list - if notifiedAll { - // if we have notified all watcher in the list - // we can delete the list - delete(wh.watchers, path) - } - - break - } - - next := curr.Next() // save the next +// notify function notifies the watcher. If the watcher interests in the given path, +// the function will return true. +func (w *watcher) notify(e *Event, originalPath bool, deleted bool) bool { + // watcher is interested the path in three cases and under one condition + // the condition is that the event happens after the watcher's sinceIndex - w, _ := curr.Value.(*watcher) - if (w.recursive || force || e.Key == path) && e.Index >= w.sinceIndex { - w.eventChan <- e - l.Remove(curr) - atomic.AddInt64(&wh.count, -1) - } else { - notifiedAll = false - } + // 1. the path at which the event happens is the path the watcher is watching at. + // For example if the watcher is watching at "/foo" and the event happens at "/foo", + // the watcher must be interested in that event. - curr = next // go to the next one - } - } -} - -func (wh *watcherHub) notify(e *Event) { - e = wh.EventHistory.addEvent(e) - - segments := strings.Split(e.Key, "/") - - currPath := "/" - - // walk through all the paths - for _, segment := range segments { - currPath = path.Join(currPath, segment) - wh.notifyWithPath(e, currPath, false) - } -} + // 2. the watcher is a recursive watcher, it interests in the event happens after + // its watching path. For example if watcher A watches at "/foo" and it is a recursive + // one, it will interest in the event happens at "/foo/bar". -func (wh *watcherHub) clone() *watcherHub { - clonedHistory := wh.EventHistory.clone() + // 3. when we delete a directory, we need to force notify all the watchers who watches + // at the file we need to delete. + // For example a watcher is watching at "/foo/bar". And we deletes "/foo". The watcher + // should get notified even if "/foo" is not the path it is watching. - return &watcherHub{ - EventHistory: clonedHistory, + if (w.recursive || originalPath || deleted) && e.Index >= w.sinceIndex { + w.eventChan <- e + return true } + return false } diff --git a/store/watcher_hub.go b/store/watcher_hub.go new file mode 100644 index 00000000000..4c4bfd29d30 --- /dev/null +++ b/store/watcher_hub.go @@ -0,0 +1,142 @@ +package store + +import ( + "container/list" + "path" + "strings" + "sync/atomic" + + etcdErr "github.com/coreos/etcd/error" +) + +// A watcherHub contains all subscribed watchers +// watchers is a map with watched path as key and watcher as value +// EventHistory keeps the old events for watcherHub. It is used to help +// watcher to get a continuous event history. Or a watcher might miss the +// event happens between the end of the first watch command and the start +// of the second command. +type watcherHub struct { + watchers map[string]*list.List + count int64 // current number of watchers. + EventHistory *EventHistory +} + +// newWatchHub creates a watchHub. The capacity determines how many events we will +// keep in the eventHistory. +// Typically, we only need to keep a small size of history[smaller than 20K]. +// Ideally, it should smaller than 20K/s[max throughput] * 2 * 50ms[RTT] = 2000 +func newWatchHub(capacity int) *watcherHub { + return &watcherHub{ + watchers: make(map[string]*list.List), + EventHistory: newEventHistory(capacity), + } +} + +// watch function returns an Event channel. +// If recursive is true, the first change after index under prefix will be sent to the event channel. +// If recursive is false, the first change after index at prefix will be sent to the event channel. +// If index is zero, watch will start from the current index + 1. +func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (<-chan *Event, *etcdErr.Error) { + eventChan := make(chan *Event, 1) + + e, err := wh.EventHistory.scan(prefix, index) + + if err != nil { + return nil, err + } + + if e != nil { + eventChan <- e + return eventChan, nil + } + + w := &watcher{ + eventChan: eventChan, + recursive: recursive, + sinceIndex: index - 1, // to catch Expire() + } + + l, ok := wh.watchers[prefix] + + if ok { // add the new watcher to the back of the list + l.PushBack(w) + + } else { // create a new list and add the new watcher + l := list.New() + l.PushBack(w) + wh.watchers[prefix] = l + } + + atomic.AddInt64(&wh.count, 1) + + return eventChan, nil +} + +// notify function accepts an event and notify to the watchers. +func (wh *watcherHub) notify(e *Event) { + e = wh.EventHistory.addEvent(e) // add event into the eventHistory + + segments := strings.Split(e.Key, "/") + + currPath := "/" + + // walk through all the segments of the path and notify the watchers + // if the path is "/foo/bar", it will notify watchers with path "/", + // "/foo" and "/foo/bar" + + for _, segment := range segments { + currPath = path.Join(currPath, segment) + // notify the watchers who interests in the changes of current path + wh.notifyWatchers(e, currPath, false) + } +} + +func (wh *watcherHub) notifyWatchers(e *Event, path string, deleted bool) { + l, ok := wh.watchers[path] + + if ok { + curr := l.Front() + notifiedAll := true + + for { + if curr == nil { // we have reached the end of the list + if notifiedAll { + // if we have notified all watcher in the list + // we can delete the list + delete(wh.watchers, path) + } + + break + } + + next := curr.Next() // save reference to the next one in the list + + w, _ := curr.Value.(*watcher) + + if w.notify(e, e.Key == path, deleted) { + // if we successfully notify a watcher + // we need to remove the watcher from the list + // and decrease the counter + + l.Remove(curr) + atomic.AddInt64(&wh.count, -1) + } else { + // once there is a watcher in the list is not interested + // in the event, we should keep the list in the map + notifiedAll = false + } + + curr = next // update current to the next + } + } +} + +// clone function clones the watcherHub and return the cloned one. +// only clone the static content. do not clone the current watchers. +func (wh *watcherHub) clone() *watcherHub { + clonedHistory := wh.EventHistory.clone() + + return &watcherHub{ + EventHistory: clonedHistory, + } +} From a07802a34756ed218a5009240e520fdfd3b884b5 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 7 Oct 2013 23:12:13 -0700 Subject: [PATCH 060/247] refactor node.go; use once --- store/node.go | 218 +++++++++++++++++++++++++++---------------------- store/store.go | 2 +- 2 files changed, 122 insertions(+), 98 deletions(-) diff --git a/store/node.go b/store/node.go index 4117225cba3..547e1aed8c6 100644 --- a/store/node.go +++ b/store/node.go @@ -3,6 +3,7 @@ package store import ( "path" "sort" + "sync" "time" etcdErr "github.com/coreos/etcd/error" @@ -17,22 +18,37 @@ const ( removed ) +// Node is the basic element in the store system. +// A key-value pair will have a string value +// A directory will have a children map type Node struct { - Path string + Path string + CreateIndex uint64 CreateTerm uint64 ModifiedIndex uint64 ModifiedTerm uint64 - Parent *Node `json:"-"` - ExpireTime time.Time - ACL string - Value string // for key-value pair - Children map[string]*Node // for directory - status int - stopExpire chan bool // stop expire routine channel + + Parent *Node `json:"-"` // should not encode this field! avoid cyclical dependency. + + ExpireTime time.Time + ACL string + Value string // for key-value pair + Children map[string]*Node // for directory + + // a ttl node will have an expire routine associated with it. + // we need a channel to stop that routine when the expiration changes. + stopExpire chan bool + + // ensure we only delete the node once + // expire and remove may try to delete a node twice + once sync.Once } -func newFile(nodePath string, value string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { +// newKV creates a Key-Value pair +func newKV(nodePath string, value string, createIndex uint64, + createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { + return &Node{ Path: nodePath, CreateIndex: createIndex, @@ -47,7 +63,10 @@ func newFile(nodePath string, value string, createIndex uint64, createTerm uint6 } } -func newDir(nodePath string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { +// newDir creates a directory +func newDir(nodePath string, createIndex uint64, createTerm uint64, + parent *Node, ACL string, expireTime time.Time) *Node { + return &Node{ Path: nodePath, CreateIndex: createIndex, @@ -60,36 +79,79 @@ func newDir(nodePath string, createIndex uint64, createTerm uint64, parent *Node } } +// IsHidden function checks if the node is a hidden node. A hidden node +// will begin with '_' +// A hidden node will not be shown via get command under a directory +// For example if we have /foo/_hidden and /foo/notHidden, get "/foo" +// will only return /foo/notHidden +func (n *Node) IsHidden() bool { + _, name := path.Split(n.Path) + + return name[0] == '_' +} + +// IsPermanent function checks if the node is a permanent one. +func (n *Node) IsPermanent() bool { + return n.ExpireTime.Sub(Permanent) == 0 +} + +// IsExpired function checks if the node has been expired. +func (n *Node) IsExpired() (bool, time.Duration) { + if n.IsPermanent() { + return false, 0 + } + + duration := n.ExpireTime.Sub(time.Now()) + if duration <= 0 { + return true, 0 + } + + return false, duration +} + +// IsDir function checks whether the node is a directory. +// If the node is a directory, the function will return true. +// Otherwise the function will return false. +func (n *Node) IsDir() bool { + return !(n.Children == nil) +} + // Remove function remove the node. -// If the node is a directory and recursive is true, the function will recursively remove -// add nodes under the receiver node. func (n *Node) Remove(recursive bool, callback func(path string)) *etcdErr.Error { - if n.status == removed { // check race between remove and expire - return nil + + if n.IsDir() && !recursive { + // cannot delete a directory without set recursive to true + return etcdErr.NewError(etcdErr.EcodeNotFile, "", UndefIndex, UndefTerm) } - if !n.IsDir() { // file node: key-value pair - _, name := path.Split(n.Path) + onceBody := func() { + n.internalRemove(recursive, callback) + } - if n.Parent != nil && n.Parent.Children[name] == n { - // This is the only pointer to Node object - // Handled by garbage collector - delete(n.Parent.Children, name) + // this function might be entered multiple times by expire and delete + // every node will only be deleted once. + n.once.Do(onceBody) - if callback != nil { - callback(n.Path) - } + return nil +} - n.stopExpire <- true - n.status = removed +// internalRemove function will be called by remove() +func (n *Node) internalRemove(recursive bool, callback func(path string)) { + if !n.IsDir() { // key-value pair + _, name := path.Split(n.Path) + // find its parent and remove the node from the map + if n.Parent != nil && n.Parent.Children[name] == n { + delete(n.Parent.Children, name) } - return nil - } + if callback != nil { + callback(n.Path) + } - if !recursive { - return etcdErr.NewError(etcdErr.EcodeNotFile, "", UndefIndex, UndefTerm) + // the stop channel has a buffer. just send to it! + n.stopExpire <- true + return } for _, child := range n.Children { // delete all children @@ -106,10 +168,7 @@ func (n *Node) Remove(recursive bool, callback func(path string)) *etcdErr.Error } n.stopExpire <- true - n.status = removed } - - return nil } // Read function gets the value of the node. @@ -192,36 +251,6 @@ func (n *Node) Add(child *Node) *etcdErr.Error { return nil } -// Clone function clone the node recursively and return the new node. -// If the node is a directory, it will clone all the content under this directory. -// If the node is a key-value pair, it will clone the pair. -func (n *Node) Clone() *Node { - if !n.IsDir() { - return newFile(n.Path, n.Value, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime) - } - - clone := newDir(n.Path, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime) - - for key, child := range n.Children { - clone.Children[key] = child.Clone() - } - - return clone -} - -func (n *Node) recoverAndclean(s *Store) { - if n.IsDir() { - for _, child := range n.Children { - child.Parent = n - child.recoverAndclean(s) - } - } - - n.stopExpire = make(chan bool, 1) - - n.Expire(s) -} - // Expire function will test if the node is expired. // if the node is already expired, delete the node and return. // if the node is permemant (this shouldn't happen), return at once. @@ -268,41 +297,6 @@ func (n *Node) Expire(s *Store) { }() } -// IsHidden function checks if the node is a hidden node. A hidden node -// will begin with '_' -// A hidden node will not be shown via get command under a directory -// For example if we have /foo/_hidden and /foo/notHidden, get "/foo" -// will only return /foo/notHidden -func (n *Node) IsHidden() bool { - _, name := path.Split(n.Path) - - return name[0] == '_' -} - -func (n *Node) IsPermanent() bool { - return n.ExpireTime.Sub(Permanent) == 0 -} - -func (n *Node) IsExpired() (bool, time.Duration) { - if n.IsPermanent() { - return false, 0 - } - - duration := n.ExpireTime.Sub(time.Now()) - if duration <= 0 { - return true, 0 - } - - return false, duration -} - -// IsDir function checks whether the node is a directory. -// If the node is a directory, the function will return true. -// Otherwise the function will return false. -func (n *Node) IsDir() bool { - return !(n.Children == nil) -} - func (n *Node) Pair(recurisive, sorted bool) KeyValuePair { if n.IsDir() { pair := KeyValuePair{ @@ -364,3 +358,33 @@ func (n *Node) UpdateTTL(expireTime time.Time, s *Store) { n.Expire(s) } } + +// Clone function clone the node recursively and return the new node. +// If the node is a directory, it will clone all the content under this directory. +// If the node is a key-value pair, it will clone the pair. +func (n *Node) Clone() *Node { + if !n.IsDir() { + return newKV(n.Path, n.Value, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime) + } + + clone := newDir(n.Path, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime) + + for key, child := range n.Children { + clone.Children[key] = child.Clone() + } + + return clone +} + +func (n *Node) recoverAndclean(s *Store) { + if n.IsDir() { + for _, child := range n.Children { + child.Parent = n + child.recoverAndclean(s) + } + } + + n.stopExpire = make(chan bool, 1) + + n.Expire(s) +} diff --git a/store/store.go b/store/store.go index b8830fe5eb1..5d41e45eddb 100644 --- a/store/store.go +++ b/store/store.go @@ -344,7 +344,7 @@ func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix if len(value) != 0 { // create file e.Value = value - n = newFile(nodePath, value, s.Index, s.Term, d, "", expireTime) + n = newKV(nodePath, value, s.Index, s.Term, d, "", expireTime) } else { // create directory e.Dir = true From 48e6137f469f6130181cb4c3171b8ba8b4ac1358 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 7 Oct 2013 23:21:39 -0700 Subject: [PATCH 061/247] fix node.go race between expire and update --- store/node.go | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/store/node.go b/store/node.go index 547e1aed8c6..15837819975 100644 --- a/store/node.go +++ b/store/node.go @@ -279,14 +279,21 @@ func (n *Node) Expire(s *Store) { // if timeout, delete the node case <-time.After(duration): + // before expire get the lock, the expiration time + // of the node may be updated. + // we have to check again when get the lock s.worldLock.Lock() defer s.worldLock.Unlock() - e := newEvent(Expire, n.Path, UndefIndex, UndefTerm) - s.WatcherHub.notify(e) + expired, _ := n.IsExpired() - n.Remove(true, nil) - s.Stats.Inc(ExpireCount) + if expired { + e := newEvent(Expire, n.Path, UndefIndex, UndefTerm) + s.WatcherHub.notify(e) + + n.Remove(true, nil) + s.Stats.Inc(ExpireCount) + } return From 3c7f9215d1366c6811a0211a1c85d5dd3e72511c Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 7 Oct 2013 23:30:51 -0700 Subject: [PATCH 062/247] comments node.go --- store/node.go | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/store/node.go b/store/node.go index 15837819975..8e67eeb0e29 100644 --- a/store/node.go +++ b/store/node.go @@ -253,7 +253,7 @@ func (n *Node) Add(child *Node) *etcdErr.Error { // Expire function will test if the node is expired. // if the node is already expired, delete the node and return. -// if the node is permemant (this shouldn't happen), return at once. +// if the node is permanent (this shouldn't happen), return at once. // else wait for a period time, then remove the node. and notify the watchhub. func (n *Node) Expire(s *Store) { expired, duration := n.IsExpired() @@ -383,6 +383,13 @@ func (n *Node) Clone() *Node { return clone } +// recoverAndclean function help to do recovery. +// Two things need to be done: 1. recovery structure; 2. delete expired nodes + +// If the node is a directory, it will help recover children's parent pointer and recursively +// call this function on its children. +// We check the expire last since we need to recover the whole structure first and add all the +// notifications into the event history. func (n *Node) recoverAndclean(s *Store) { if n.IsDir() { for _, child := range n.Children { From c3e2332479ae1fc70cdb8cbc1ab70c59c9e21210 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 8 Oct 2013 21:25:56 -0700 Subject: [PATCH 063/247] refactor separate kvpair to kvpair.go; simplify sorting interface --- store/event.go | 37 ++++++++----------------------------- store/kv_pairs.go | 24 ++++++++++++++++++++++++ store/node.go | 2 +- store/store.go | 17 +++++++---------- 4 files changed, 40 insertions(+), 40 deletions(-) create mode 100644 store/kv_pairs.go diff --git a/store/event.go b/store/event.go index ba4149a79b5..15866d7450d 100644 --- a/store/event.go +++ b/store/event.go @@ -24,40 +24,19 @@ const ( ) type Event struct { - Action string `json:"action"` - Key string `json:"key, omitempty"` - Dir bool `json:"dir,omitempty"` - PrevValue string `json:"prevValue,omitempty"` - Value string `json:"value,omitempty"` - KVPairs []KeyValuePair `json:"kvs,omitempty"` - Expiration *time.Time `json:"expiration,omitempty"` - TTL int64 `json:"ttl,omitempty"` // Time to live in second + Action string `json:"action"` + Key string `json:"key, omitempty"` + Dir bool `json:"dir,omitempty"` + PrevValue string `json:"prevValue,omitempty"` + Value string `json:"value,omitempty"` + KVPairs kvPairs `json:"kvs,omitempty"` + Expiration *time.Time `json:"expiration,omitempty"` + TTL int64 `json:"ttl,omitempty"` // Time to live in second // The command index of the raft machine when the command is executed Index uint64 `json:"index"` Term uint64 `json:"term"` } -// When user list a directory, we add all the node into key-value pair slice -type KeyValuePair struct { - Key string `json:"key, omitempty"` - Value string `json:"value,omitempty"` - Dir bool `json:"dir,omitempty"` - KVPairs []KeyValuePair `json:"kvs,omitempty"` -} - -// interfaces for sorting -func (k KeyValuePair) Len() int { - return len(k.KVPairs) -} - -func (k KeyValuePair) Less(i, j int) bool { - return k.KVPairs[i].Key < k.KVPairs[j].Key -} - -func (k KeyValuePair) Swap(i, j int) { - k.KVPairs[i], k.KVPairs[j] = k.KVPairs[j], k.KVPairs[i] -} - func newEvent(action string, key string, index uint64, term uint64) *Event { return &Event{ Action: action, diff --git a/store/kv_pairs.go b/store/kv_pairs.go new file mode 100644 index 00000000000..90e5c3fd6df --- /dev/null +++ b/store/kv_pairs.go @@ -0,0 +1,24 @@ +package store + +// When user list a directory, we add all the node into key-value pair slice +type KeyValuePair struct { + Key string `json:"key, omitempty"` + Value string `json:"value,omitempty"` + Dir bool `json:"dir,omitempty"` + KVPairs kvPairs `json:"kvs,omitempty"` +} + +type kvPairs []KeyValuePair + +// interfaces for sorting +func (kvs kvPairs) Len() int { + return len(kvs) +} + +func (kvs kvPairs) Less(i, j int) bool { + return kvs[i].Key < kvs[j].Key +} + +func (kvs kvPairs) Swap(i, j int) { + kvs[i], kvs[j] = kvs[j], kvs[i] +} diff --git a/store/node.go b/store/node.go index 8e67eeb0e29..010a97dee7b 100644 --- a/store/node.go +++ b/store/node.go @@ -336,7 +336,7 @@ func (n *Node) Pair(recurisive, sorted bool) KeyValuePair { // eliminate hidden nodes pair.KVPairs = pair.KVPairs[:i] if sorted { - sort.Sort(pair) + sort.Sort(pair.KVPairs) } return pair diff --git a/store/store.go b/store/store.go index 5d41e45eddb..2c0c9cdb47c 100644 --- a/store/store.go +++ b/store/store.go @@ -24,13 +24,16 @@ type Store struct { func New() *Store { s := new(Store) - s.Root = newDir("/", 0, 0, nil, "", Permanent) + s.Root = newDir("/", UndefIndex, UndefTerm, nil, "", Permanent) s.Stats = newStats() s.WatcherHub = newWatchHub(1000) return s } +// get function returns a get event. +// If recursive is true, it will return all the content under the node path. +// If sorted is true, it will sort the content by keys. func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) { s.worldLock.RLock() defer s.worldLock.RUnlock() @@ -46,7 +49,7 @@ func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term e := newEvent(Get, nodePath, index, term) - if n.IsDir() { // node is dir + if n.IsDir() { // node is a directory e.Dir = true children, _ := n.List() @@ -57,25 +60,19 @@ func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term i := 0 for _, child := range children { - - if child.IsHidden() { // get will not list hidden node + if child.IsHidden() { // get will not return hidden nodes continue } e.KVPairs[i] = child.Pair(recursive, sorted) - i++ } // eliminate hidden nodes e.KVPairs = e.KVPairs[:i] - rootPairs := KeyValuePair{ - KVPairs: e.KVPairs, - } - if sorted { - sort.Sort(rootPairs) + sort.Sort(e.KVPairs) } } else { // node is file From 75959f994805f029bd169b16227b8dcf7b6344d4 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 8 Oct 2013 21:49:10 -0700 Subject: [PATCH 064/247] refactor add node.ExpirationAndTTL --- store/node.go | 117 ++++++++++++++++++++++++++----------------------- store/store.go | 23 +++------- 2 files changed, 69 insertions(+), 71 deletions(-) diff --git a/store/node.go b/store/node.go index 010a97dee7b..683e6038bf4 100644 --- a/store/node.go +++ b/store/node.go @@ -116,61 +116,6 @@ func (n *Node) IsDir() bool { return !(n.Children == nil) } -// Remove function remove the node. -func (n *Node) Remove(recursive bool, callback func(path string)) *etcdErr.Error { - - if n.IsDir() && !recursive { - // cannot delete a directory without set recursive to true - return etcdErr.NewError(etcdErr.EcodeNotFile, "", UndefIndex, UndefTerm) - } - - onceBody := func() { - n.internalRemove(recursive, callback) - } - - // this function might be entered multiple times by expire and delete - // every node will only be deleted once. - n.once.Do(onceBody) - - return nil -} - -// internalRemove function will be called by remove() -func (n *Node) internalRemove(recursive bool, callback func(path string)) { - if !n.IsDir() { // key-value pair - _, name := path.Split(n.Path) - - // find its parent and remove the node from the map - if n.Parent != nil && n.Parent.Children[name] == n { - delete(n.Parent.Children, name) - } - - if callback != nil { - callback(n.Path) - } - - // the stop channel has a buffer. just send to it! - n.stopExpire <- true - return - } - - for _, child := range n.Children { // delete all children - child.Remove(true, callback) - } - - // delete self - _, name := path.Split(n.Path) - if n.Parent != nil && n.Parent.Children[name] == n { - delete(n.Parent.Children, name) - - if callback != nil { - callback(n.Path) - } - - n.stopExpire <- true - } -} - // Read function gets the value of the node. // If the receiver node is not a key-value pair, a "Not A File" error will be returned. func (n *Node) Read() (string, *etcdErr.Error) { @@ -195,6 +140,13 @@ func (n *Node) Write(value string, index uint64, term uint64) *etcdErr.Error { return nil } +func (n *Node) ExpirationAndTTL() (*time.Time, int64) { + if n.ExpireTime.Sub(Permanent) != 0 { + return &n.ExpireTime, int64(n.ExpireTime.Sub(time.Now())/time.Second) + 1 + } + return nil, 0 +} + // List function return a slice of nodes under the receiver node. // If the receiver node is not a directory, a "Not A Directory" error will be returned. func (n *Node) List() ([]*Node, *etcdErr.Error) { @@ -251,6 +203,61 @@ func (n *Node) Add(child *Node) *etcdErr.Error { return nil } +// Remove function remove the node. +func (n *Node) Remove(recursive bool, callback func(path string)) *etcdErr.Error { + + if n.IsDir() && !recursive { + // cannot delete a directory without set recursive to true + return etcdErr.NewError(etcdErr.EcodeNotFile, "", UndefIndex, UndefTerm) + } + + onceBody := func() { + n.internalRemove(recursive, callback) + } + + // this function might be entered multiple times by expire and delete + // every node will only be deleted once. + n.once.Do(onceBody) + + return nil +} + +// internalRemove function will be called by remove() +func (n *Node) internalRemove(recursive bool, callback func(path string)) { + if !n.IsDir() { // key-value pair + _, name := path.Split(n.Path) + + // find its parent and remove the node from the map + if n.Parent != nil && n.Parent.Children[name] == n { + delete(n.Parent.Children, name) + } + + if callback != nil { + callback(n.Path) + } + + // the stop channel has a buffer. just send to it! + n.stopExpire <- true + return + } + + for _, child := range n.Children { // delete all children + child.Remove(true, callback) + } + + // delete self + _, name := path.Split(n.Path) + if n.Parent != nil && n.Parent.Children[name] == n { + delete(n.Parent.Children, name) + + if callback != nil { + callback(n.Path) + } + + n.stopExpire <- true + } +} + // Expire function will test if the node is expired. // if the node is already expired, delete the node and return. // if the node is permanent (this shouldn't happen), return at once. diff --git a/store/store.go b/store/store.go index 2c0c9cdb47c..966aa49c65d 100644 --- a/store/store.go +++ b/store/store.go @@ -75,14 +75,11 @@ func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term sort.Sort(e.KVPairs) } - } else { // node is file - e.Value = n.Value + } else { // node is a file + e.Value, _ = n.Read() } - if n.ExpireTime.Sub(Permanent) != 0 { - e.Expiration = &n.ExpireTime - e.TTL = int64(n.ExpireTime.Sub(time.Now())/time.Second) + 1 - } + e.Expiration, e.TTL = n.ExpirationAndTTL() s.Stats.Inc(GetSuccess) @@ -140,10 +137,8 @@ func (s *Store) Update(nodePath string, value string, expireTime time.Time, inde // update ttl n.UpdateTTL(expireTime, s) - if n.ExpireTime.Sub(Permanent) != 0 { - e.Expiration = &n.ExpireTime - e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 - } + e.Expiration, e.TTL = n.ExpirationAndTTL() + s.WatcherHub.notify(e) s.Stats.Inc(UpdateSuccess) @@ -184,10 +179,7 @@ func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, n.UpdateTTL(expireTime, s) - if n.ExpireTime.Sub(Permanent) != 0 { - e.Expiration = &n.ExpireTime - e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 - } + e.Expiration, e.TTL = n.ExpirationAndTTL() s.WatcherHub.notify(e) s.Stats.Inc(TestAndSetSuccess) @@ -360,8 +352,7 @@ func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix // Node with TTL if expireTime.Sub(Permanent) != 0 { n.Expire(s) - e.Expiration = &n.ExpireTime - e.TTL = int64(expireTime.Sub(time.Now())/time.Second) + 1 + e.Expiration, e.TTL = n.ExpirationAndTTL() } s.WatcherHub.notify(e) From a030a41153fb0dca59b8b7f35ac170362026b909 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 8 Oct 2013 22:01:22 -0700 Subject: [PATCH 065/247] refactor store.go remove unwanted print --- store/store.go | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/store/store.go b/store/store.go index 966aa49c65d..ec85a5d7018 100644 --- a/store/store.go +++ b/store/store.go @@ -31,7 +31,7 @@ func New() *Store { return s } -// get function returns a get event. +// Get function returns a get event. // If recursive is true, it will return all the content under the node path. // If sorted is true, it will sort the content by keys. func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) { @@ -110,7 +110,6 @@ func (s *Store) Update(nodePath string, value string, expireTime time.Time, inde if err != nil { // if the node does not exist, return error s.Stats.Inc(UpdateFail) - return nil, err } @@ -126,11 +125,6 @@ func (s *Store) Update(nodePath string, value string, expireTime time.Time, inde } else { // if the node is a file, we can update value and ttl e.PrevValue = n.Value - - if len(value) != 0 { - e.Value = value - } - n.Write(value, index, term) } @@ -172,13 +166,12 @@ func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, if n.Value == prevValue || n.ModifiedIndex == prevIndex { // if test succeed, write the value - e := newEvent(TestAndSet, nodePath, index, term) - e.PrevValue = n.Value - e.Value = value n.Write(value, index, term) - n.UpdateTTL(expireTime, s) + e := newEvent(TestAndSet, nodePath, index, term) + e.PrevValue = n.Value + e.Value = value e.Expiration, e.TTL = n.ExpirationAndTTL() s.WatcherHub.notify(e) @@ -309,7 +302,6 @@ func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix if err != nil { s.Stats.Inc(SetFail) - fmt.Println("1: bad create") return nil, err } From e597947bd8312889c6ec66a5c4cd3f8f0b862736 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 8 Oct 2013 22:10:18 -0700 Subject: [PATCH 066/247] refactor store.go update --- store/store.go | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/store/store.go b/store/store.go index ec85a5d7018..080d50695b4 100644 --- a/store/store.go +++ b/store/store.go @@ -101,7 +101,7 @@ func (s *Store) Create(nodePath string, value string, incrementalSuffix bool, fo // Update function updates the value/ttl of the node. // If the node is a file, the value and the ttl can be updated. // If the node is a directory, only the ttl can be updated. -func (s *Store) Update(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { +func (s *Store) Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) { s.worldLock.Lock() defer s.worldLock.Unlock() nodePath = path.Clean(path.Join("/", nodePath)) @@ -115,17 +115,17 @@ func (s *Store) Update(nodePath string, value string, expireTime time.Time, inde e := newEvent(Update, nodePath, s.Index, s.Term) - if n.IsDir() { // if the node is a directory, we can only update ttl - if len(value) != 0 { + if len(newValue) != 0 { + if n.IsDir() { + // if the node is a directory, we cannot update value s.Stats.Inc(UpdateFail) err := etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term) return nil, err } - } else { // if the node is a file, we can update value and ttl e.PrevValue = n.Value - n.Write(value, index, term) + n.Write(newValue, index, term) } // update ttl @@ -165,12 +165,13 @@ func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, } if n.Value == prevValue || n.ModifiedIndex == prevIndex { + e := newEvent(TestAndSet, nodePath, index, term) + e.PrevValue = n.Value + // if test succeed, write the value n.Write(value, index, term) n.UpdateTTL(expireTime, s) - e := newEvent(TestAndSet, nodePath, index, term) - e.PrevValue = n.Value e.Value = value e.Expiration, e.TTL = n.ExpirationAndTTL() From 4bf57537b5664c3bfb85146f4f04e1ff6996cb84 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 8 Oct 2013 22:12:00 -0700 Subject: [PATCH 067/247] refactor store.go remove extra assignment --- store/store.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/store/store.go b/store/store.go index 080d50695b4..2af95639dc3 100644 --- a/store/store.go +++ b/store/store.go @@ -119,9 +119,7 @@ func (s *Store) Update(nodePath string, newValue string, expireTime time.Time, i if n.IsDir() { // if the node is a directory, we cannot update value s.Stats.Inc(UpdateFail) - - err := etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term) - return nil, err + return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term) } e.PrevValue = n.Value From 61899d62c50b2220b8b240e7967b75fa5508ac37 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Wed, 9 Oct 2013 20:34:00 -0700 Subject: [PATCH 068/247] refactor command.go: commands do not rely on the etcdStore singleton. So we can seprate command into a package in the furture. --- command.go | 52 ++++++++++++------- .../github.com/coreos/go-raft/server.go | 5 ++ 2 files changed, 38 insertions(+), 19 deletions(-) diff --git a/command.go b/command.go index 7fe0049c8fb..79affd7d9f9 100644 --- a/command.go +++ b/command.go @@ -40,7 +40,9 @@ func (c *CreateCommand) CommandName() string { // Create node func (c *CreateCommand) Apply(server *raft.Server) (interface{}, error) { - e, err := etcdStore.Create(c.Key, c.Value, c.IncrementalSuffix, c.Force, c.ExpireTime, server.CommitIndex(), server.Term()) + s, _ := server.StateMachine().(*store.Store) + + e, err := s.Create(c.Key, c.Value, c.IncrementalSuffix, c.Force, c.ExpireTime, server.CommitIndex(), server.Term()) if err != nil { debug(err) @@ -64,7 +66,9 @@ func (c *UpdateCommand) CommandName() string { // Update node func (c *UpdateCommand) Apply(server *raft.Server) (interface{}, error) { - e, err := etcdStore.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) + s, _ := server.StateMachine().(*store.Store) + + e, err := s.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) if err != nil { debug(err) @@ -90,7 +94,9 @@ func (c *TestAndSetCommand) CommandName() string { // Set the key-value pair if the current value of the key equals to the given prevValue func (c *TestAndSetCommand) Apply(server *raft.Server) (interface{}, error) { - e, err := etcdStore.TestAndSet(c.Key, c.PrevValue, c.PrevIndex, + s, _ := server.StateMachine().(*store.Store) + + e, err := s.TestAndSet(c.Key, c.PrevValue, c.PrevIndex, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) if err != nil { @@ -115,7 +121,9 @@ func (c *GetCommand) CommandName() string { // Get the value of key func (c *GetCommand) Apply(server *raft.Server) (interface{}, error) { - e, err := etcdStore.Get(c.Key, c.Recursive, c.Sorted, server.CommitIndex(), server.Term()) + s, _ := server.StateMachine().(*store.Store) + + e, err := s.Get(c.Key, c.Recursive, c.Sorted, server.CommitIndex(), server.Term()) if err != nil { debug(err) @@ -138,7 +146,9 @@ func (c *DeleteCommand) CommandName() string { // Delete the key func (c *DeleteCommand) Apply(server *raft.Server) (interface{}, error) { - e, err := etcdStore.Delete(c.Key, c.Recursive, server.CommitIndex(), server.Term()) + s, _ := server.StateMachine().(*store.Store) + + e, err := s.Delete(c.Key, c.Recursive, server.CommitIndex(), server.Term()) if err != nil { debug(err) @@ -161,7 +171,9 @@ func (c *WatchCommand) CommandName() string { } func (c *WatchCommand) Apply(server *raft.Server) (interface{}, error) { - eventChan, err := etcdStore.Watch(c.Key, c.Recursive, c.SinceIndex, server.CommitIndex(), server.Term()) + s, _ := server.StateMachine().(*store.Store) + + eventChan, err := s.Watch(c.Key, c.Recursive, c.SinceIndex, server.CommitIndex(), server.Term()) if err != nil { return nil, err @@ -195,13 +207,14 @@ func (c *JoinCommand) CommandName() string { } // Join a server to the cluster -func (c *JoinCommand) Apply(raftServer *raft.Server) (interface{}, error) { +func (c *JoinCommand) Apply(server *raft.Server) (interface{}, error) { + s, _ := server.StateMachine().(*store.Store) // check if the join command is from a previous machine, who lost all its previous log. - e, _ := etcdStore.Get(path.Join("/_etcd/machines", c.Name), false, false, raftServer.CommitIndex(), raftServer.Term()) + e, _ := s.Get(path.Join("/_etcd/machines", c.Name), false, false, server.CommitIndex(), server.Term()) b := make([]byte, 8) - binary.PutUvarint(b, raftServer.CommitIndex()) + binary.PutUvarint(b, server.CommitIndex()) if e != nil { return b, nil @@ -211,18 +224,18 @@ func (c *JoinCommand) Apply(raftServer *raft.Server) (interface{}, error) { num := machineNum() if num == maxClusterSize { debug("Reject join request from ", c.Name) - return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMoreMachine, "", raftServer.CommitIndex(), raftServer.Term()) + return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMoreMachine, "", server.CommitIndex(), server.Term()) } addNameToURL(c.Name, c.RaftVersion, c.RaftURL, c.EtcdURL) // add peer in raft - err := raftServer.AddPeer(c.Name, "") + err := server.AddPeer(c.Name, "") // add machine in etcd storage key := path.Join("_etcd/machines", c.Name) value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", c.RaftURL, c.EtcdURL, c.RaftVersion) - etcdStore.Create(key, value, false, false, store.Permanent, raftServer.CommitIndex(), raftServer.Term()) + s.Create(key, value, false, false, store.Permanent, server.CommitIndex(), server.Term()) // add peer stats if c.Name != r.Name() { @@ -248,12 +261,13 @@ func (c *RemoveCommand) CommandName() string { } // Remove a server from the cluster -func (c *RemoveCommand) Apply(raftServer *raft.Server) (interface{}, error) { +func (c *RemoveCommand) Apply(server *raft.Server) (interface{}, error) { + s, _ := server.StateMachine().(*store.Store) // remove machine in etcd storage key := path.Join("_etcd/machines", c.Name) - _, err := etcdStore.Delete(key, false, raftServer.CommitIndex(), raftServer.Term()) + _, err := s.Delete(key, false, server.CommitIndex(), server.Term()) // delete from stats delete(r.followersStats.Followers, c.Name) @@ -262,21 +276,21 @@ func (c *RemoveCommand) Apply(raftServer *raft.Server) (interface{}, error) { } // remove peer in raft - err = raftServer.RemovePeer(c.Name) + err = server.RemovePeer(c.Name) if err != nil { return []byte{0}, err } - if c.Name == raftServer.Name() { + if c.Name == server.Name() { // the removed node is this node // if the node is not replaying the previous logs // and the node has sent out a join request in this // start. It is sure that this node received a new remove // command and need to be removed - if raftServer.CommitIndex() > r.joinIndex && r.joinIndex != 0 { - debugf("server [%s] is removed", raftServer.Name()) + if server.CommitIndex() > r.joinIndex && r.joinIndex != 0 { + debugf("server [%s] is removed", server.Name()) os.Exit(0) } else { // else ignore remove @@ -285,7 +299,7 @@ func (c *RemoveCommand) Apply(raftServer *raft.Server) (interface{}, error) { } b := make([]byte, 8) - binary.PutUvarint(b, raftServer.CommitIndex()) + binary.PutUvarint(b, server.CommitIndex()) return b, err } diff --git a/third_party/github.com/coreos/go-raft/server.go b/third_party/github.com/coreos/go-raft/server.go index 645a2e4a069..e42e3ad62fb 100644 --- a/third_party/github.com/coreos/go-raft/server.go +++ b/third_party/github.com/coreos/go-raft/server.go @@ -186,6 +186,11 @@ func (s *Server) Context() interface{} { return s.context } +// Retrieves the state machine passed into the constructor. +func (s *Server) StateMachine() StateMachine { + return s.stateMachine +} + // Retrieves the log path for the server. func (s *Server) LogPath() string { return path.Join(s.path, "log") From 255e14a5c416b06a84f6a4dd76f189e867ba01ea Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Wed, 9 Oct 2013 20:51:21 -0700 Subject: [PATCH 069/247] refactor command.go,server.go: add raftWrapper as context, totally get rid of reference in command.go --- command.go | 12 +++++++----- raft_server.go | 22 ++++++++++++---------- 2 files changed, 19 insertions(+), 15 deletions(-) diff --git a/command.go b/command.go index 79affd7d9f9..0a330c23269 100644 --- a/command.go +++ b/command.go @@ -192,12 +192,12 @@ type JoinCommand struct { EtcdURL string `json:"etcdURL"` } -func newJoinCommand() *JoinCommand { +func newJoinCommand(version, name, raftUrl, etcdUrl string) *JoinCommand { return &JoinCommand{ - RaftVersion: r.version, - Name: r.name, - RaftURL: r.url, - EtcdURL: e.url, + RaftVersion: version, + Name: name, + RaftURL: raftUrl, + EtcdURL: etcdUrl, } } @@ -209,6 +209,7 @@ func (c *JoinCommand) CommandName() string { // Join a server to the cluster func (c *JoinCommand) Apply(server *raft.Server) (interface{}, error) { s, _ := server.StateMachine().(*store.Store) + r, _ := server.Context().(*raftServer) // check if the join command is from a previous machine, who lost all its previous log. e, _ := s.Get(path.Join("/_etcd/machines", c.Name), false, false, server.CommitIndex(), server.Term()) @@ -263,6 +264,7 @@ func (c *RemoveCommand) CommandName() string { // Remove a server from the cluster func (c *RemoveCommand) Apply(server *raft.Server) (interface{}, error) { s, _ := server.StateMachine().(*store.Store) + r, _ := server.Context().(*raftServer) // remove machine in etcd storage key := path.Join("_etcd/machines", c.Name) diff --git a/raft_server.go b/raft_server.go index efbf41acaff..8e48c027eb4 100644 --- a/raft_server.go +++ b/raft_server.go @@ -35,13 +35,7 @@ func newRaftServer(name string, url string, listenHost string, tlsConf *TLSConfi // Create transporter for raft raftTransporter := newTransporter(tlsConf.Scheme, tlsConf.Client) - // Create raft server - server, err := raft.NewServer(name, dirPath, raftTransporter, etcdStore, nil, "") - - check(err) - - return &raftServer{ - Server: server, + raftWrapper := &raftServer{ version: raftVersion, name: name, url: url, @@ -62,6 +56,14 @@ func newRaftServer(name string, url string, listenHost string, tlsConf *TLSConfi }, }, } + + // Create raft server + server, err := raft.NewServer(name, dirPath, raftTransporter, etcdStore, raftWrapper, "") + check(err) + + raftWrapper.Server = server + + return raftWrapper } // Start the raft server @@ -127,7 +129,7 @@ func (r *raftServer) ListenAndServe() { func startAsLeader() { // leader need to join self as a peer for { - _, err := r.Do(newJoinCommand()) + _, err := r.Do(newJoinCommand(r.version, r.Name(), r.url, e.url)) if err == nil { break } @@ -243,7 +245,7 @@ func joinByMachine(s *raft.Server, machine string, scheme string) error { return fmt.Errorf("Unable to join: internal version mismatch, entire cluster must be running identical versions of etcd") } - json.NewEncoder(&b).Encode(newJoinCommand()) + json.NewEncoder(&b).Encode(newJoinCommand(r.version, r.Name(), r.url, e.url)) joinURL := url.URL{Host: machine, Scheme: scheme, Path: "/join"} @@ -270,7 +272,7 @@ func joinByMachine(s *raft.Server, machine string, scheme string) error { address := resp.Header.Get("Location") debugf("Send Join Request to %s", address) - json.NewEncoder(&b).Encode(newJoinCommand()) + json.NewEncoder(&b).Encode(newJoinCommand(r.version, r.Name(), r.url, e.url)) resp, req, err = t.Post(address, &b) From 40c520ca1b5b0e7b3ad942d1baf770e8098e06dd Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Thu, 10 Oct 2013 15:40:05 -0700 Subject: [PATCH 070/247] refactor remove raft singleton --- etcd.go | 9 +++---- etcd_handler_v1.go | 22 ++++++++------- etcd_handlers.go | 65 +++++++++++++++++++++++++-------------------- etcd_server.go | 25 +++++++++-------- machines.go | 5 ++-- name_url_map.go | 2 +- raft_handlers.go | 28 +++++++++---------- raft_server.go | 50 +++++++++++++++++----------------- raft_stats.go | 3 ++- snapshot.go | 4 +-- store/store.go | 4 ++- transporter.go | 14 +++++----- transporter_test.go | 2 +- util.go | 29 ++------------------ 14 files changed, 126 insertions(+), 136 deletions(-) diff --git a/etcd.go b/etcd.go index 49271260fef..6bc036724af 100644 --- a/etcd.go +++ b/etcd.go @@ -201,13 +201,12 @@ func main() { // Create etcd key-value store etcdStore = store.New() - snapConf = newSnapshotConf() - // Create etcd and raft server - e = newEtcdServer(info.Name, info.EtcdURL, info.EtcdListenHost, &etcdTLSConfig, &info.EtcdTLS) - r = newRaftServer(info.Name, info.RaftURL, info.RaftListenHost, &raftTLSConfig, &info.RaftTLS) + r := newRaftServer(info.Name, info.RaftURL, info.RaftListenHost, &raftTLSConfig, &info.RaftTLS) + snapConf = r.newSnapshotConf() + + e = newEtcdServer(info.Name, info.EtcdURL, info.EtcdListenHost, &etcdTLSConfig, &info.EtcdTLS, r) - startWebInterface() r.ListenAndServe() e.ListenAndServe() diff --git a/etcd_handler_v1.go b/etcd_handler_v1.go index 44cde5a860e..987b2e794fb 100644 --- a/etcd_handler_v1.go +++ b/etcd_handler_v1.go @@ -14,17 +14,17 @@ import ( // Handlers to handle etcd-store related request via etcd url //------------------------------------------------------------------- // Multiplex GET/POST/DELETE request to corresponding handlers -func MultiplexerV1(w http.ResponseWriter, req *http.Request) error { +func (e *etcdServer) MultiplexerV1(w http.ResponseWriter, req *http.Request) error { switch req.Method { case "GET": - return GetHttpHandlerV1(w, req) + return e.GetHttpHandlerV1(w, req) case "POST": - return SetHttpHandlerV1(w, req) + return e.SetHttpHandlerV1(w, req) case "PUT": - return SetHttpHandlerV1(w, req) + return e.SetHttpHandlerV1(w, req) case "DELETE": - return DeleteHttpHandlerV1(w, req) + return e.DeleteHttpHandlerV1(w, req) default: w.WriteHeader(http.StatusMethodNotAllowed) return nil @@ -37,7 +37,7 @@ func MultiplexerV1(w http.ResponseWriter, req *http.Request) error { //-------------------------------------- // Set Command Handler -func SetHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { +func (e *etcdServer) SetHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { key := req.URL.Path[len("/v1/keys/"):] debugf("[recv] POST %v/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) @@ -81,7 +81,7 @@ func SetHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { } // Delete Handler -func DeleteHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { +func (e *etcdServer) DeleteHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { key := req.URL.Path[len("/v1/keys/"):] debugf("[recv] DELETE %v/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) @@ -101,9 +101,10 @@ func DeleteHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { //-------------------------------------- // Get Handler -func GetHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { +func (e *etcdServer) GetHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { key := req.URL.Path[len("/v1/keys/"):] + r := e.raftServer debugf("[recv] GET %s/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) command := &GetCommand{ @@ -128,13 +129,13 @@ func GetHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { } // Watch handler -func WatchHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { +func (e *etcdServer) WatchHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { key := req.URL.Path[len("/v1/watch/"):] command := &WatchCommand{ Key: key, } - + r := e.raftServer if req.Method == "GET" { debugf("[recv] GET %s/watch/%s [%s]", e.url, key, req.RemoteAddr) command.SinceIndex = 0 @@ -178,6 +179,7 @@ func dispatchEtcdCommandV1(c Command, w http.ResponseWriter, req *http.Request) } func dispatchV1(c Command, w http.ResponseWriter, req *http.Request, toURL func(name string) (string, bool)) error { + r := e.raftServer if r.State() == raft.Leader { if event, err := r.Do(c); err != nil { return err diff --git a/etcd_handlers.go b/etcd_handlers.go index fe64347a3aa..487c1cef5db 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -19,18 +19,18 @@ import ( func NewEtcdMuxer() *http.ServeMux { // external commands etcdMux := http.NewServeMux() - etcdMux.Handle("/"+version+"/keys/", errorHandler(Multiplexer)) - etcdMux.Handle("/"+version+"/leader", errorHandler(LeaderHttpHandler)) - etcdMux.Handle("/"+version+"/machines", errorHandler(MachinesHttpHandler)) - etcdMux.Handle("/"+version+"/stats/", errorHandler(StatsHttpHandler)) - etcdMux.Handle("/version", errorHandler(VersionHttpHandler)) + etcdMux.Handle("/"+version+"/keys/", errorHandler(e.Multiplexer)) + etcdMux.Handle("/"+version+"/leader", errorHandler(e.LeaderHttpHandler)) + etcdMux.Handle("/"+version+"/machines", errorHandler(e.MachinesHttpHandler)) + etcdMux.Handle("/"+version+"/stats/", errorHandler(e.StatsHttpHandler)) + etcdMux.Handle("/version", errorHandler(e.VersionHttpHandler)) etcdMux.HandleFunc("/test/", TestHttpHandler) // backward support - etcdMux.Handle("/v1/keys/", errorHandler(MultiplexerV1)) - etcdMux.Handle("/v1/leader", errorHandler(LeaderHttpHandler)) - etcdMux.Handle("/v1/machines", errorHandler(MachinesHttpHandler)) - etcdMux.Handle("/v1/stats/", errorHandler(StatsHttpHandler)) + etcdMux.Handle("/v1/keys/", errorHandler(e.MultiplexerV1)) + etcdMux.Handle("/v1/leader", errorHandler(e.LeaderHttpHandler)) + etcdMux.Handle("/v1/machines", errorHandler(e.MachinesHttpHandler)) + etcdMux.Handle("/v1/stats/", errorHandler(e.StatsHttpHandler)) return etcdMux } @@ -68,17 +68,17 @@ func (fn errorHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { } // Multiplex GET/POST/DELETE request to corresponding handlers -func Multiplexer(w http.ResponseWriter, req *http.Request) error { +func (e *etcdServer) Multiplexer(w http.ResponseWriter, req *http.Request) error { switch req.Method { case "GET": - return GetHttpHandler(w, req) + return e.GetHttpHandler(w, req) case "POST": - return CreateHttpHandler(w, req) + return e.CreateHttpHandler(w, req) case "PUT": - return UpdateHttpHandler(w, req) + return e.UpdateHttpHandler(w, req) case "DELETE": - return DeleteHttpHandler(w, req) + return e.DeleteHttpHandler(w, req) default: w.WriteHeader(http.StatusMethodNotAllowed) return nil @@ -92,7 +92,7 @@ func Multiplexer(w http.ResponseWriter, req *http.Request) error { // Set/Delete will dispatch to leader //-------------------------------------- -func CreateHttpHandler(w http.ResponseWriter, req *http.Request) error { +func (e *etcdServer) CreateHttpHandler(w http.ResponseWriter, req *http.Request) error { key := getNodePath(req.URL.Path) debugf("recv.post[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) @@ -115,11 +115,11 @@ func CreateHttpHandler(w http.ResponseWriter, req *http.Request) error { command.IncrementalSuffix = true } - return dispatchEtcdCommand(command, w, req) + return e.dispatchEtcdCommand(command, w, req) } -func UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { +func (e *etcdServer) UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { key := getNodePath(req.URL.Path) debugf("recv.put[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) @@ -150,7 +150,7 @@ func UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { ExpireTime: expireTime, } - return dispatchEtcdCommand(command, w, req) + return e.dispatchEtcdCommand(command, w, req) } else { // update with test var prevIndex uint64 @@ -173,13 +173,13 @@ func UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { PrevIndex: prevIndex, } - return dispatchEtcdCommand(command, w, req) + return e.dispatchEtcdCommand(command, w, req) } } // Delete Handler -func DeleteHttpHandler(w http.ResponseWriter, req *http.Request) error { +func (e *etcdServer) DeleteHttpHandler(w http.ResponseWriter, req *http.Request) error { key := getNodePath(req.URL.Path) debugf("recv.delete[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) @@ -192,12 +192,12 @@ func DeleteHttpHandler(w http.ResponseWriter, req *http.Request) error { command.Recursive = true } - return dispatchEtcdCommand(command, w, req) + return e.dispatchEtcdCommand(command, w, req) } // Dispatch the command to leader -func dispatchEtcdCommand(c Command, w http.ResponseWriter, req *http.Request) error { - return dispatch(c, w, req, nameToEtcdURL) +func (e *etcdServer) dispatchEtcdCommand(c Command, w http.ResponseWriter, req *http.Request) error { + return e.raftServer.dispatch(c, w, req, nameToEtcdURL) } //-------------------------------------- @@ -207,7 +207,9 @@ func dispatchEtcdCommand(c Command, w http.ResponseWriter, req *http.Request) er //-------------------------------------- // Handler to return the current leader's raft address -func LeaderHttpHandler(w http.ResponseWriter, req *http.Request) error { +func (e *etcdServer) LeaderHttpHandler(w http.ResponseWriter, req *http.Request) error { + r := e.raftServer + leader := r.Leader() if leader != "" { @@ -222,8 +224,8 @@ func LeaderHttpHandler(w http.ResponseWriter, req *http.Request) error { } // Handler to return all the known machines in the current cluster -func MachinesHttpHandler(w http.ResponseWriter, req *http.Request) error { - machines := getMachines(nameToEtcdURL) +func (e *etcdServer) MachinesHttpHandler(w http.ResponseWriter, req *http.Request) error { + machines := e.raftServer.getMachines(nameToEtcdURL) w.WriteHeader(http.StatusOK) w.Write([]byte(strings.Join(machines, ", "))) @@ -232,7 +234,7 @@ func MachinesHttpHandler(w http.ResponseWriter, req *http.Request) error { } // Handler to return the current version of etcd -func VersionHttpHandler(w http.ResponseWriter, req *http.Request) error { +func (e *etcdServer) VersionHttpHandler(w http.ResponseWriter, req *http.Request) error { w.WriteHeader(http.StatusOK) fmt.Fprintf(w, "etcd %s", releaseVersion) @@ -240,10 +242,12 @@ func VersionHttpHandler(w http.ResponseWriter, req *http.Request) error { } // Handler to return the basic stats of etcd -func StatsHttpHandler(w http.ResponseWriter, req *http.Request) error { +func (e *etcdServer) StatsHttpHandler(w http.ResponseWriter, req *http.Request) error { option := req.URL.Path[len("/v1/stats/"):] w.WriteHeader(http.StatusOK) + r := e.raftServer + switch option { case "self": w.Write(r.Stats()) @@ -266,9 +270,12 @@ func StatsHttpHandler(w http.ResponseWriter, req *http.Request) error { return nil } -func GetHttpHandler(w http.ResponseWriter, req *http.Request) error { +func (e *etcdServer) GetHttpHandler(w http.ResponseWriter, req *http.Request) error { var err error var event interface{} + + r := e.raftServer + debugf("recv.get[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) if req.FormValue("consistent") == "true" && r.State() != raft.Leader { diff --git a/etcd_server.go b/etcd_server.go index 0139b03c273..657c4f9678e 100644 --- a/etcd_server.go +++ b/etcd_server.go @@ -6,26 +6,29 @@ import ( type etcdServer struct { http.Server - name string - url string - tlsConf *TLSConfig - tlsInfo *TLSInfo + raftServer *raftServer + name string + url string + tlsConf *TLSConfig + tlsInfo *TLSInfo } var e *etcdServer -func newEtcdServer(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo) *etcdServer { - return &etcdServer{ +func newEtcdServer(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, raftServer *raftServer) *etcdServer { + e = &etcdServer{ Server: http.Server{ - Handler: NewEtcdMuxer(), TLSConfig: &tlsConf.Server, Addr: listenHost, }, - name: name, - url: urlStr, - tlsConf: tlsConf, - tlsInfo: tlsInfo, + name: name, + url: urlStr, + tlsConf: tlsConf, + tlsInfo: tlsInfo, + raftServer: raftServer, } + e.Handler = NewEtcdMuxer() + return e } // Start to listen and response etcd client command diff --git a/machines.go b/machines.go index b863c50fb3a..b8b4a09d599 100644 --- a/machines.go +++ b/machines.go @@ -2,7 +2,7 @@ package main // machineNum returns the number of machines in the cluster func machineNum() int { - e, err := etcdStore.Get("/_etcd/machines", false, false, r.CommitIndex(), r.Term()) + e, err := etcdStore.Get("/_etcd/machines", false, false, 0, 0) if err != nil { return 0 @@ -12,8 +12,7 @@ func machineNum() int { } // getMachines gets the current machines in the cluster -func getMachines(toURL func(string) (string, bool)) []string { - +func (r *raftServer) getMachines(toURL func(string) (string, bool)) []string { peers := r.Peers() machines := make([]string, len(peers)+1) diff --git a/name_url_map.go b/name_url_map.go index 1192f3fda74..220963d5cf2 100644 --- a/name_url_map.go +++ b/name_url_map.go @@ -56,7 +56,7 @@ func readURL(nodeName string, urlName string) (string, bool) { // convert nodeName to url from etcd storage key := path.Join("/_etcd/machines", nodeName) - e, err := etcdStore.Get(key, false, false, r.CommitIndex(), r.Term()) + e, err := etcdStore.Get(key, false, false, 0, 0) if err != nil { return "", false diff --git a/raft_handlers.go b/raft_handlers.go index 6c95efd3f0a..a45fe496ccb 100644 --- a/raft_handlers.go +++ b/raft_handlers.go @@ -12,7 +12,7 @@ import ( //------------------------------------------------------------- // Get all the current logs -func GetLogHttpHandler(w http.ResponseWriter, req *http.Request) { +func (r *raftServer) GetLogHttpHandler(w http.ResponseWriter, req *http.Request) { debugf("[recv] GET %s/log", r.url) w.Header().Set("Content-Type", "application/json") w.WriteHeader(http.StatusOK) @@ -20,7 +20,7 @@ func GetLogHttpHandler(w http.ResponseWriter, req *http.Request) { } // Response to vote request -func VoteHttpHandler(w http.ResponseWriter, req *http.Request) { +func (r *raftServer) VoteHttpHandler(w http.ResponseWriter, req *http.Request) { rvreq := &raft.RequestVoteRequest{} err := decodeJsonRequest(req, rvreq) if err == nil { @@ -36,7 +36,7 @@ func VoteHttpHandler(w http.ResponseWriter, req *http.Request) { } // Response to append entries request -func AppendEntriesHttpHandler(w http.ResponseWriter, req *http.Request) { +func (r *raftServer) AppendEntriesHttpHandler(w http.ResponseWriter, req *http.Request) { aereq := &raft.AppendEntriesRequest{} err := decodeJsonRequest(req, aereq) @@ -59,7 +59,7 @@ func AppendEntriesHttpHandler(w http.ResponseWriter, req *http.Request) { } // Response to recover from snapshot request -func SnapshotHttpHandler(w http.ResponseWriter, req *http.Request) { +func (r *raftServer) SnapshotHttpHandler(w http.ResponseWriter, req *http.Request) { aereq := &raft.SnapshotRequest{} err := decodeJsonRequest(req, aereq) if err == nil { @@ -75,7 +75,7 @@ func SnapshotHttpHandler(w http.ResponseWriter, req *http.Request) { } // Response to recover from snapshot request -func SnapshotRecoveryHttpHandler(w http.ResponseWriter, req *http.Request) { +func (r *raftServer) SnapshotRecoveryHttpHandler(w http.ResponseWriter, req *http.Request) { aereq := &raft.SnapshotRecoveryRequest{} err := decodeJsonRequest(req, aereq) if err == nil { @@ -91,20 +91,20 @@ func SnapshotRecoveryHttpHandler(w http.ResponseWriter, req *http.Request) { } // Get the port that listening for etcd connecting of the server -func EtcdURLHttpHandler(w http.ResponseWriter, req *http.Request) { +func (r *raftServer) EtcdURLHttpHandler(w http.ResponseWriter, req *http.Request) { debugf("[recv] Get %s/etcdURL/ ", r.url) w.WriteHeader(http.StatusOK) w.Write([]byte(argInfo.EtcdURL)) } // Response to the join request -func JoinHttpHandler(w http.ResponseWriter, req *http.Request) error { +func (r *raftServer) JoinHttpHandler(w http.ResponseWriter, req *http.Request) error { command := &JoinCommand{} if err := decodeJsonRequest(req, command); err == nil { debugf("Receive Join Request from %s", command.Name) - return dispatchRaftCommand(command, w, req) + return r.dispatchRaftCommand(command, w, req) } else { w.WriteHeader(http.StatusInternalServerError) return nil @@ -112,7 +112,7 @@ func JoinHttpHandler(w http.ResponseWriter, req *http.Request) error { } // Response to remove request -func RemoveHttpHandler(w http.ResponseWriter, req *http.Request) { +func (r *raftServer) RemoveHttpHandler(w http.ResponseWriter, req *http.Request) { if req.Method != "DELETE" { w.WriteHeader(http.StatusMethodNotAllowed) return @@ -125,23 +125,23 @@ func RemoveHttpHandler(w http.ResponseWriter, req *http.Request) { debugf("[recv] Remove Request [%s]", command.Name) - dispatchRaftCommand(command, w, req) + r.dispatchRaftCommand(command, w, req) } // Response to the name request -func NameHttpHandler(w http.ResponseWriter, req *http.Request) { +func (r *raftServer) NameHttpHandler(w http.ResponseWriter, req *http.Request) { debugf("[recv] Get %s/name/ ", r.url) w.WriteHeader(http.StatusOK) w.Write([]byte(r.name)) } // Response to the name request -func RaftVersionHttpHandler(w http.ResponseWriter, req *http.Request) { +func (r *raftServer) RaftVersionHttpHandler(w http.ResponseWriter, req *http.Request) { debugf("[recv] Get %s/version/ ", r.url) w.WriteHeader(http.StatusOK) w.Write([]byte(r.version)) } -func dispatchRaftCommand(c Command, w http.ResponseWriter, req *http.Request) error { - return dispatch(c, w, req, nameToRaftURL) +func (r *raftServer) dispatchRaftCommand(c Command, w http.ResponseWriter, req *http.Request) error { + return r.dispatch(c, w, req, nameToRaftURL) } diff --git a/raft_server.go b/raft_server.go index 8e48c027eb4..146a2e84e4e 100644 --- a/raft_server.go +++ b/raft_server.go @@ -28,13 +28,10 @@ type raftServer struct { serverStats *raftServerStats } -var r *raftServer +//var r *raftServer func newRaftServer(name string, url string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo) *raftServer { - // Create transporter for raft - raftTransporter := newTransporter(tlsConf.Scheme, tlsConf.Client) - raftWrapper := &raftServer{ version: raftVersion, name: name, @@ -57,6 +54,9 @@ func newRaftServer(name string, url string, listenHost string, tlsConf *TLSConfi }, } + // Create transporter for raft + raftTransporter := newTransporter(tlsConf.Scheme, tlsConf.Client, raftWrapper) + // Create raft server server, err := raft.NewServer(name, dirPath, raftTransporter, etcdStore, raftWrapper, "") check(err) @@ -91,16 +91,16 @@ func (r *raftServer) ListenAndServe() { // start as a leader in a new cluster if len(cluster) == 0 { - startAsLeader() + r.startAsLeader() } else { - startAsFollower() + r.startAsFollower() } } else { // rejoin the previous cluster - cluster = getMachines(nameToRaftURL) + cluster = r.getMachines(nameToRaftURL) for i := 0; i < len(cluster); i++ { u, err := url.Parse(cluster[i]) if err != nil { @@ -108,7 +108,7 @@ func (r *raftServer) ListenAndServe() { } cluster[i] = u.Host } - ok := joinCluster(cluster) + ok := r.joinCluster(cluster) if !ok { warn("the entire cluster is down! this machine will restart the cluster.") } @@ -118,7 +118,7 @@ func (r *raftServer) ListenAndServe() { // open the snapshot if snapshot { - go monitorSnapshot() + go r.monitorSnapshot() } // start to response to raft requests @@ -126,7 +126,7 @@ func (r *raftServer) ListenAndServe() { } -func startAsLeader() { +func (r *raftServer) startAsLeader() { // leader need to join self as a peer for { _, err := r.Do(newJoinCommand(r.version, r.Name(), r.url, e.url)) @@ -137,10 +137,10 @@ func startAsLeader() { debugf("%s start as a leader", r.name) } -func startAsFollower() { +func (r *raftServer) startAsFollower() { // start as a follower in a existing cluster for i := 0; i < retryTimes; i++ { - ok := joinCluster(cluster) + ok := r.joinCluster(cluster) if ok { return } @@ -164,16 +164,16 @@ func (r *raftServer) startTransport(scheme string, tlsConf tls.Config) { } // internal commands - raftMux.HandleFunc("/name", NameHttpHandler) - raftMux.HandleFunc("/version", RaftVersionHttpHandler) - raftMux.Handle("/join", errorHandler(JoinHttpHandler)) - raftMux.HandleFunc("/remove/", RemoveHttpHandler) - raftMux.HandleFunc("/vote", VoteHttpHandler) - raftMux.HandleFunc("/log", GetLogHttpHandler) - raftMux.HandleFunc("/log/append", AppendEntriesHttpHandler) - raftMux.HandleFunc("/snapshot", SnapshotHttpHandler) - raftMux.HandleFunc("/snapshotRecovery", SnapshotRecoveryHttpHandler) - raftMux.HandleFunc("/etcdURL", EtcdURLHttpHandler) + raftMux.HandleFunc("/name", r.NameHttpHandler) + raftMux.HandleFunc("/version", r.RaftVersionHttpHandler) + raftMux.Handle("/join", errorHandler(r.JoinHttpHandler)) + raftMux.HandleFunc("/remove/", r.RemoveHttpHandler) + raftMux.HandleFunc("/vote", r.VoteHttpHandler) + raftMux.HandleFunc("/log", r.GetLogHttpHandler) + raftMux.HandleFunc("/log/append", r.AppendEntriesHttpHandler) + raftMux.HandleFunc("/snapshot", r.SnapshotHttpHandler) + raftMux.HandleFunc("/snapshotRecovery", r.SnapshotRecoveryHttpHandler) + raftMux.HandleFunc("/etcdURL", r.EtcdURLHttpHandler) if scheme == "http" { fatal(server.ListenAndServe()) @@ -202,14 +202,14 @@ func getVersion(t *transporter, versionURL url.URL) (string, error) { return string(body), nil } -func joinCluster(cluster []string) bool { +func (r *raftServer) joinCluster(cluster []string) bool { for _, machine := range cluster { if len(machine) == 0 { continue } - err := joinByMachine(r.Server, machine, r.tlsConf.Scheme) + err := r.joinByMachine(r.Server, machine, r.tlsConf.Scheme) if err == nil { debugf("%s success join to the cluster via machine %s", r.name, machine) return true @@ -226,7 +226,7 @@ func joinCluster(cluster []string) bool { } // Send join requests to machine. -func joinByMachine(s *raft.Server, machine string, scheme string) error { +func (r *raftServer) joinByMachine(s *raft.Server, machine string, scheme string) error { var b bytes.Buffer // t must be ok diff --git a/raft_stats.go b/raft_stats.go index 23f97d41774..45d21037f4a 100644 --- a/raft_stats.go +++ b/raft_stats.go @@ -33,6 +33,7 @@ func (ps *packageStats) Time() time.Time { } type raftServerStats struct { + Name string `json:"name"` State string `json:"state"` StartTime time.Time `json:"startTime"` @@ -70,7 +71,7 @@ func (ss *raftServerStats) SendAppendReq(pkgSize int) { if ss.State != raft.Leader { ss.State = raft.Leader - ss.LeaderInfo.Name = r.Name() + ss.LeaderInfo.Name = ss.Name ss.LeaderInfo.startTime = now } diff --git a/snapshot.go b/snapshot.go index a6caefd32b8..1b2bc728c95 100644 --- a/snapshot.go +++ b/snapshot.go @@ -18,12 +18,12 @@ type snapshotConf struct { var snapConf *snapshotConf -func newSnapshotConf() *snapshotConf { +func (r *raftServer) newSnapshotConf() *snapshotConf { // check snapshot every 3 seconds and the threshold is 20K return &snapshotConf{time.Second * 3, 0, 20 * 1000} } -func monitorSnapshot() { +func (r *raftServer) monitorSnapshot() { for { time.Sleep(snapConf.checkingInterval) //currentWrites := etcdStore.TotalWrites() - snapConf.lastWrites diff --git a/store/store.go b/store/store.go index 2af95639dc3..348aea02e4d 100644 --- a/store/store.go +++ b/store/store.go @@ -356,7 +356,9 @@ func (s *Store) internalGet(nodePath string, index uint64, term uint64) (*Node, nodePath = path.Clean(path.Join("/", nodePath)) // update file system known index and term - s.Index, s.Term = index, term + if index > s.Index { + s.Index, s.Term = index, term + } walkFunc := func(parent *Node, name string) (*Node, *etcdErr.Error) { diff --git a/transporter.go b/transporter.go index eb430837712..19a09c73e7c 100644 --- a/transporter.go +++ b/transporter.go @@ -27,14 +27,15 @@ var tranTimeout = ElectionTimeout // Transporter layer for communication between raft nodes type transporter struct { - client *http.Client - transport *http.Transport + client *http.Client + transport *http.Transport + raftServer *raftServer } // Create transporter using by raft server // Create http or https transporter based on // whether the user give the server cert and key -func newTransporter(scheme string, tlsConf tls.Config) *transporter { +func newTransporter(scheme string, tlsConf tls.Config, raftServer *raftServer) *transporter { t := transporter{} tr := &http.Transport{ @@ -49,6 +50,7 @@ func newTransporter(scheme string, tlsConf tls.Config) *transporter { t.client = &http.Client{Transport: tr} t.transport = tr + t.raftServer = raftServer return &t } @@ -67,18 +69,18 @@ func (t *transporter) SendAppendEntriesRequest(server *raft.Server, peer *raft.P size := b.Len() - r.serverStats.SendAppendReq(size) + t.raftServer.serverStats.SendAppendReq(size) u, _ := nameToRaftURL(peer.Name) debugf("Send LogEntries to %s ", u) - thisFollowerStats, ok := r.followersStats.Followers[peer.Name] + thisFollowerStats, ok := t.raftServer.followersStats.Followers[peer.Name] if !ok { //this is the first time this follower has been seen thisFollowerStats = &raftFollowerStats{} thisFollowerStats.Latency.Minimum = 1 << 63 - r.followersStats.Followers[peer.Name] = thisFollowerStats + t.raftServer.followersStats.Followers[peer.Name] = thisFollowerStats } start := time.Now() diff --git a/transporter_test.go b/transporter_test.go index 8c71325c68e..3d9655dbdd9 100644 --- a/transporter_test.go +++ b/transporter_test.go @@ -21,7 +21,7 @@ func TestTransporterTimeout(t *testing.T) { conf := tls.Config{} - ts := newTransporter("http", conf) + ts := newTransporter("http", conf, nil) ts.Get("http://google.com") _, _, err := ts.Get("http://google.com:9999") diff --git a/util.go b/util.go index edecb97e49e..318f728d10d 100644 --- a/util.go +++ b/util.go @@ -15,7 +15,6 @@ import ( etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/store" - "github.com/coreos/etcd/web" "github.com/coreos/go-log/log" "github.com/coreos/go-raft" ) @@ -39,35 +38,11 @@ func durationToExpireTime(strDuration string) (time.Time, error) { } } -//-------------------------------------- -// Web Helper -//-------------------------------------- -var storeMsg chan string - -// Help to send msg from store to webHub -func webHelper() { - storeMsg = make(chan string) - // etcdStore.SetMessager(storeMsg) - for { - // transfer the new msg to webHub - web.Hub().Send(<-storeMsg) - } -} - -// startWebInterface starts web interface if webURL is not empty -func startWebInterface() { - if argInfo.WebURL != "" { - // start web - go webHelper() - go web.Start(r.Server, argInfo.WebURL) - } -} - //-------------------------------------- // HTTP Utilities //-------------------------------------- -func dispatch(c Command, w http.ResponseWriter, req *http.Request, toURL func(name string) (string, bool)) error { +func (r *raftServer) dispatch(c Command, w http.ResponseWriter, req *http.Request, toURL func(name string) (string, bool)) error { if r.State() == raft.Leader { if response, err := r.Do(c); err != nil { return err @@ -278,7 +253,7 @@ func send(c chan bool) { command.Key = "foo" command.Value = "bar" command.ExpireTime = time.Unix(0, 0) - r.Do(command) + //r.Do(command) } c <- true } From a113a51d7392c67a26bd3da44f105da4d3737644 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Thu, 10 Oct 2013 22:42:45 -0600 Subject: [PATCH 071/247] Refactor v1 API into server/v1. --- etcd.go | 18 --- etcd_handler_v1.go | 250 -------------------------------- etcd_handlers.go | 10 +- etcd_server.go | 44 ------ server/server.go | 98 +++++++++++++ server/v1/delete_key_handler.go | 13 ++ server/v1/dispatch.go | 42 ++++++ server/v1/get_key_handler.go | 31 ++++ server/v1/install.go | 15 ++ server/v1/set_key_handler.go | 50 +++++++ server/v1/util.go | 38 +++++ server/v1/watch_key_handler.go | 39 +++++ 12 files changed, 331 insertions(+), 317 deletions(-) delete mode 100644 etcd_handler_v1.go delete mode 100644 etcd_server.go create mode 100644 server/server.go create mode 100644 server/v1/delete_key_handler.go create mode 100644 server/v1/dispatch.go create mode 100644 server/v1/get_key_handler.go create mode 100644 server/v1/install.go create mode 100644 server/v1/set_key_handler.go create mode 100644 server/v1/util.go create mode 100644 server/v1/watch_key_handler.go diff --git a/etcd.go b/etcd.go index 6bc036724af..775d6a18420 100644 --- a/etcd.go +++ b/etcd.go @@ -45,7 +45,6 @@ var ( cpuprofile string cors string - corsList map[string]bool ) func init() { @@ -212,20 +211,3 @@ func main() { } -// parseCorsFlag gathers up the cors whitelist and puts it into the corsList. -func parseCorsFlag() { - if cors != "" { - corsList = make(map[string]bool) - list := strings.Split(cors, ",") - for _, v := range list { - fmt.Println(v) - if v != "*" { - _, err := url.Parse(v) - if err != nil { - panic(fmt.Sprintf("bad cors url: %s", err)) - } - } - corsList[v] = true - } - } -} diff --git a/etcd_handler_v1.go b/etcd_handler_v1.go deleted file mode 100644 index 987b2e794fb..00000000000 --- a/etcd_handler_v1.go +++ /dev/null @@ -1,250 +0,0 @@ -package main - -import ( - "encoding/json" - "net/http" - "strconv" - - etcdErr "github.com/coreos/etcd/error" - "github.com/coreos/etcd/store" - "github.com/coreos/go-raft" -) - -//------------------------------------------------------------------- -// Handlers to handle etcd-store related request via etcd url -//------------------------------------------------------------------- -// Multiplex GET/POST/DELETE request to corresponding handlers -func (e *etcdServer) MultiplexerV1(w http.ResponseWriter, req *http.Request) error { - - switch req.Method { - case "GET": - return e.GetHttpHandlerV1(w, req) - case "POST": - return e.SetHttpHandlerV1(w, req) - case "PUT": - return e.SetHttpHandlerV1(w, req) - case "DELETE": - return e.DeleteHttpHandlerV1(w, req) - default: - w.WriteHeader(http.StatusMethodNotAllowed) - return nil - } -} - -//-------------------------------------- -// State sensitive handlers -// Set/Delete will dispatch to leader -//-------------------------------------- - -// Set Command Handler -func (e *etcdServer) SetHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { - key := req.URL.Path[len("/v1/keys/"):] - - debugf("[recv] POST %v/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) - - req.ParseForm() - - value := req.Form.Get("value") - - if len(value) == 0 { - return etcdErr.NewError(200, "Set", store.UndefIndex, store.UndefTerm) - } - - strDuration := req.Form.Get("ttl") - - expireTime, err := durationToExpireTime(strDuration) - - if err != nil { - return etcdErr.NewError(202, "Set", store.UndefIndex, store.UndefTerm) - } - - if prevValueArr, ok := req.Form["prevValue"]; ok && len(prevValueArr) > 0 { - command := &TestAndSetCommand{ - Key: key, - Value: value, - PrevValue: prevValueArr[0], - ExpireTime: expireTime, - } - - return dispatchEtcdCommandV1(command, w, req) - - } else { - command := &CreateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - Force: true, - } - - return dispatchEtcdCommandV1(command, w, req) - } -} - -// Delete Handler -func (e *etcdServer) DeleteHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { - key := req.URL.Path[len("/v1/keys/"):] - - debugf("[recv] DELETE %v/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) - - command := &DeleteCommand{ - Key: key, - } - - return dispatchEtcdCommandV1(command, w, req) -} - -//-------------------------------------- -// State non-sensitive handlers -// will not dispatch to leader -// TODO: add sensitive version for these -// command? -//-------------------------------------- - -// Get Handler -func (e *etcdServer) GetHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { - key := req.URL.Path[len("/v1/keys/"):] - - r := e.raftServer - debugf("[recv] GET %s/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) - - command := &GetCommand{ - Key: key, - } - - if event, err := command.Apply(r.Server); err != nil { - return err - } else { - event, _ := event.(*store.Event) - - response := eventToResponse(event) - bytes, _ := json.Marshal(response) - - w.WriteHeader(http.StatusOK) - - w.Write(bytes) - - return nil - } - -} - -// Watch handler -func (e *etcdServer) WatchHttpHandlerV1(w http.ResponseWriter, req *http.Request) error { - key := req.URL.Path[len("/v1/watch/"):] - - command := &WatchCommand{ - Key: key, - } - r := e.raftServer - if req.Method == "GET" { - debugf("[recv] GET %s/watch/%s [%s]", e.url, key, req.RemoteAddr) - command.SinceIndex = 0 - - } else if req.Method == "POST" { - // watch from a specific index - - debugf("[recv] POST %s/watch/%s [%s]", e.url, key, req.RemoteAddr) - content := req.FormValue("index") - - sinceIndex, err := strconv.ParseUint(string(content), 10, 64) - if err != nil { - return etcdErr.NewError(203, "Watch From Index", store.UndefIndex, store.UndefTerm) - } - command.SinceIndex = sinceIndex - - } else { - w.WriteHeader(http.StatusMethodNotAllowed) - return nil - } - - if event, err := command.Apply(r.Server); err != nil { - return etcdErr.NewError(500, key, store.UndefIndex, store.UndefTerm) - } else { - event, _ := event.(*store.Event) - - response := eventToResponse(event) - bytes, _ := json.Marshal(response) - - w.WriteHeader(http.StatusOK) - - w.Write(bytes) - return nil - } - -} - -// Dispatch the command to leader -func dispatchEtcdCommandV1(c Command, w http.ResponseWriter, req *http.Request) error { - return dispatchV1(c, w, req, nameToEtcdURL) -} - -func dispatchV1(c Command, w http.ResponseWriter, req *http.Request, toURL func(name string) (string, bool)) error { - r := e.raftServer - if r.State() == raft.Leader { - if event, err := r.Do(c); err != nil { - return err - } else { - if event == nil { - return etcdErr.NewError(300, "Empty result from raft", store.UndefIndex, store.UndefTerm) - } - - event, _ := event.(*store.Event) - - response := eventToResponse(event) - bytes, _ := json.Marshal(response) - - w.WriteHeader(http.StatusOK) - w.Write(bytes) - return nil - - } - - } else { - leader := r.Leader() - // current no leader - if leader == "" { - return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) - } - url, _ := toURL(leader) - - redirect(url, w, req) - - return nil - } -} - -func eventToResponse(event *store.Event) interface{} { - if !event.Dir { - response := &store.Response{ - Action: event.Action, - Key: event.Key, - Value: event.Value, - PrevValue: event.PrevValue, - Index: event.Index, - TTL: event.TTL, - Expiration: event.Expiration, - } - - if response.Action == store.Create || response.Action == store.Update { - response.Action = "set" - if response.PrevValue == "" { - response.NewKey = true - } - } - - return response - } else { - responses := make([]*store.Response, len(event.KVPairs)) - - for i, kv := range event.KVPairs { - responses[i] = &store.Response{ - Action: event.Action, - Key: kv.Key, - Value: kv.Value, - Dir: kv.Dir, - Index: event.Index, - } - } - return responses - } -} diff --git a/etcd_handlers.go b/etcd_handlers.go index 487c1cef5db..0d294de7d4a 100644 --- a/etcd_handlers.go +++ b/etcd_handlers.go @@ -18,11 +18,11 @@ import ( func NewEtcdMuxer() *http.ServeMux { // external commands - etcdMux := http.NewServeMux() - etcdMux.Handle("/"+version+"/keys/", errorHandler(e.Multiplexer)) - etcdMux.Handle("/"+version+"/leader", errorHandler(e.LeaderHttpHandler)) - etcdMux.Handle("/"+version+"/machines", errorHandler(e.MachinesHttpHandler)) - etcdMux.Handle("/"+version+"/stats/", errorHandler(e.StatsHttpHandler)) + router := mux.NewRouter() + etcdMux.Handle("/v2/keys/", errorHandler(e.Multiplexer)) + etcdMux.Handle("/v2/leader", errorHandler(e.LeaderHttpHandler)) + etcdMux.Handle("/v2/machines", errorHandler(e.MachinesHttpHandler)) + etcdMux.Handle("/v2/stats/", errorHandler(e.StatsHttpHandler)) etcdMux.Handle("/version", errorHandler(e.VersionHttpHandler)) etcdMux.HandleFunc("/test/", TestHttpHandler) diff --git a/etcd_server.go b/etcd_server.go deleted file mode 100644 index 657c4f9678e..00000000000 --- a/etcd_server.go +++ /dev/null @@ -1,44 +0,0 @@ -package main - -import ( - "net/http" -) - -type etcdServer struct { - http.Server - raftServer *raftServer - name string - url string - tlsConf *TLSConfig - tlsInfo *TLSInfo -} - -var e *etcdServer - -func newEtcdServer(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, raftServer *raftServer) *etcdServer { - e = &etcdServer{ - Server: http.Server{ - TLSConfig: &tlsConf.Server, - Addr: listenHost, - }, - name: name, - url: urlStr, - tlsConf: tlsConf, - tlsInfo: tlsInfo, - raftServer: raftServer, - } - e.Handler = NewEtcdMuxer() - return e -} - -// Start to listen and response etcd client command -func (e *etcdServer) ListenAndServe() { - - infof("etcd server [name %s, listen on %s, advertised url %s]", e.name, e.Server.Addr, e.url) - - if e.tlsConf.Scheme == "http" { - fatal(e.Server.ListenAndServe()) - } else { - fatal(e.Server.ListenAndServeTLS(e.tlsInfo.CertFile, e.tlsInfo.KeyFile)) - } -} diff --git a/server/server.go b/server/server.go new file mode 100644 index 00000000000..8deb2fc24ce --- /dev/null +++ b/server/server.go @@ -0,0 +1,98 @@ +package server + +import ( + "github.com/gorilla/mux" + "net/http" +) + +// The Server provides an HTTP interface to the underlying data store. +type Server struct { + http.Server + raftServer *raftServer + name string + url string + tlsConf *TLSConfig + tlsInfo *TLSInfo + corsOrigins map[string]bool +} + +// Creates a new Server. +func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, raftServer *raftServer) *Server { + s := &etcdServer{ + Server: http.Server{ + Handler: mux.NewRouter(), + TLSConfig: &tlsConf.Server, + Addr: listenHost, + }, + name: name, + url: urlStr, + tlsConf: tlsConf, + tlsInfo: tlsInfo, + raftServer: raftServer, + } + + // TODO: Move to main.go. + // Install the routes for each version of the API. + // v1.Install(s) + // v2.Install(s) + + return s +} + +// Adds a server handler to the router. +func (s *Server) HandleFunc(path string, f func(http.ResponseWriter, *http.Request, *server.Server) error) *mux.Route { + r := s.Handler.(*mux.Router) + + // Wrap the standard HandleFunc interface to pass in the server reference. + return r.HandleFunc(path, func(w http.ResponseWriter, req *http.Request) { + // Write CORS header. + if s.OriginAllowed("*") { + w.Header().Add("Access-Control-Allow-Origin", "*") + } else if s.OriginAllowed(r.Header.Get("Origin")) { + w.Header().Add("Access-Control-Allow-Origin", r.Header.Get("Origin")) + } + + // Execute handler function and return error if necessary. + if err := f(w, req, s); err != nil { + if etcdErr, ok := err.(*etcdErr.Error); ok { + debug("Return error: ", (*etcdErr).Error()) + etcdErr.Write(w) + } else { + http.Error(w, e.Error(), http.StatusInternalServerError) + } + } + }) +} + +// Start to listen and response etcd client command +func (s *Server) ListenAndServe() { + infof("etcd server [name %s, listen on %s, advertised url %s]", s.name, s.Server.Addr, s.url) + + if s.tlsConf.Scheme == "http" { + fatal(s.Server.ListenAndServe()) + } else { + fatal(s.Server.ListenAndServeTLS(s.tlsInfo.CertFile, s.tlsInfo.KeyFile)) + } +} + +// Sets a comma-delimited list of origins that are allowed. +func (s *Server) AllowOrigins(origins string) error { + // Construct a lookup of all origins. + m := make(map[string]bool) + for _, v := range strings.Split(cors, ",") { + if v != "*" { + if _, err := url.Parse(v); err != nil { + return fmt.Errorf("Invalid CORS origin: %s", err) + } + } + m[v] = true + } + s.origins = m + + return nil +} + +// Determines whether the server will allow a given CORS origin. +func (s *Server) OriginAllowed(origin string) { + return s.origins["*"] || s.origins[origin] +} diff --git a/server/v1/delete_key_handler.go b/server/v1/delete_key_handler.go new file mode 100644 index 00000000000..d2d450b6af5 --- /dev/null +++ b/server/v1/delete_key_handler.go @@ -0,0 +1,13 @@ +package v1 + +func deleteKeyHandler(w http.ResponseWriter, req *http.Request, e *etcdServer) error { + key := req.URL.Path[len("/v1/keys/"):] + + debugf("[recv] DELETE %v/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) + + command := &DeleteCommand{ + Key: key, + } + + return dispatchEtcdCommandV1(command, w, req) +} diff --git a/server/v1/dispatch.go b/server/v1/dispatch.go new file mode 100644 index 00000000000..e1aa0292234 --- /dev/null +++ b/server/v1/dispatch.go @@ -0,0 +1,42 @@ +package v1 + +// Dispatch the command to leader. +func dispatchCommand(c Command, w http.ResponseWriter, req *http.Request) error { + return dispatch(c, w, req, nameToEtcdURL) +} + +// Dispatches a command to a given URL. +func dispatch(c Command, w http.ResponseWriter, req *http.Request, toURL func(name string) (string, bool)) error { + r := e.raftServer + if r.State() == raft.Leader { + if event, err := r.Do(c); err != nil { + return err + } else { + if event == nil { + return etcdErr.NewError(300, "Empty result from raft", store.UndefIndex, store.UndefTerm) + } + + event, _ := event.(*store.Event) + + response := eventToResponse(event) + bytes, _ := json.Marshal(response) + + w.WriteHeader(http.StatusOK) + w.Write(bytes) + return nil + + } + + } else { + leader := r.Leader() + // current no leader + if leader == "" { + return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) + } + url, _ := toURL(leader) + + redirect(url, w, req) + + return nil + } +} diff --git a/server/v1/get_key_handler.go b/server/v1/get_key_handler.go new file mode 100644 index 00000000000..391a47a7ab4 --- /dev/null +++ b/server/v1/get_key_handler.go @@ -0,0 +1,31 @@ +package v1 + +import ( + "encoding/json" + "github.com/coreos/etcd/store" + "net/http" +) + +// Retrieves the value for a given key. +func getKeyHandler(w http.ResponseWriter, req *http.Request, e *etcdServer) error { + vars := mux.Vars(req) + key := "/" + vars["key"] + + debugf("[recv] GET %s/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) + + // Execute the command. + command := &GetCommand{Key: key} + event, err := command.Apply(e.raftServer.Server) + if err != nil { + return err + } + + // Convert event to a response and write to client. + event, _ := event.(*store.Event) + response := eventToResponse(event) + b, _ := json.Marshal(response) + w.WriteHeader(http.StatusOK) + w.Write(b) + + return nil +} diff --git a/server/v1/install.go b/server/v1/install.go new file mode 100644 index 00000000000..18b664cfea3 --- /dev/null +++ b/server/v1/install.go @@ -0,0 +1,15 @@ +package v1 + +import ( + "github.com/coreos/etcd/server" + "github.com/gorilla/mux" +) + +// Installs the routes for version 1 of the API on to a server. +func Install(s *server.Server) { + s.HandleFunc("/v1/keys/{key:.*}", getKeyHandler).Methods("GET") + s.HandleFunc("/v1/keys/{key:.*}", setKeyHandler).Methods("POST", "PUT") + s.HandleFunc("/v1/keys/{key:.*}", deleteKeyHandler).Methods("DELETE") + + s.HandleFunc("/v1/watch/{key:.*}", watchKeyHandler).Methods("GET", "POST") +} diff --git a/server/v1/set_key_handler.go b/server/v1/set_key_handler.go new file mode 100644 index 00000000000..26f6db01494 --- /dev/null +++ b/server/v1/set_key_handler.go @@ -0,0 +1,50 @@ +package v1 + +import ( + "encoding/json" + "github.com/coreos/etcd/store" + "net/http" +) + +// Sets the value for a given key. +func setKeyHandler(w http.ResponseWriter, req *http.Request, e *etcdServer) error { + vars := mux.Vars(req) + key := "/" + vars["key"] + + debugf("[recv] POST %v/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) + + req.ParseForm() + + // Parse non-blank value. + value := req.Form.Get("value") + if len(value) == 0 { + return error.NewError(200, "Set", store.UndefIndex, store.UndefTerm) + } + + // Convert time-to-live to an expiration time. + expireTime, err := durationToExpireTime(req.Form.Get("ttl")) + if err != nil { + return etcdErr.NewError(202, "Set", store.UndefIndex, store.UndefTerm) + } + + // If the "prevValue" is specified then test-and-set. Otherwise create a new key. + var c command.Command + if prevValueArr, ok := req.Form["prevValue"]; ok && len(prevValueArr) > 0 { + c = &TestAndSetCommand{ + Key: key, + Value: value, + PrevValue: prevValueArr[0], + ExpireTime: expireTime, + } + + } else { + c = &CreateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + Force: true, + } + } + + return dispatchEtcdCommand(command, w, req) +} diff --git a/server/v1/util.go b/server/v1/util.go new file mode 100644 index 00000000000..8bfc22f23e6 --- /dev/null +++ b/server/v1/util.go @@ -0,0 +1,38 @@ +package v1 + +// Converts an event object into a response object. +func eventToResponse(event *store.Event) interface{} { + if !event.Dir { + response := &store.Response{ + Action: event.Action, + Key: event.Key, + Value: event.Value, + PrevValue: event.PrevValue, + Index: event.Index, + TTL: event.TTL, + Expiration: event.Expiration, + } + + if response.Action == store.Create || response.Action == store.Update { + response.Action = "set" + if response.PrevValue == "" { + response.NewKey = true + } + } + + return response + } else { + responses := make([]*store.Response, len(event.KVPairs)) + + for i, kv := range event.KVPairs { + responses[i] = &store.Response{ + Action: event.Action, + Key: kv.Key, + Value: kv.Value, + Dir: kv.Dir, + Index: event.Index, + } + } + return responses + } +} diff --git a/server/v1/watch_key_handler.go b/server/v1/watch_key_handler.go new file mode 100644 index 00000000000..0b32314aa75 --- /dev/null +++ b/server/v1/watch_key_handler.go @@ -0,0 +1,39 @@ +package v1 + +import ( + "encoding/json" + "github.com/coreos/etcd/store" + "net/http" +) + +// Watches a given key prefix for changes. +func watchKeyHandler(w http.ResponseWriter, req *http.Request, e *etcdServer) error { + vars := mux.Vars(req) + key := "/" + vars["key"] + + debugf("[recv] %s %s/watch/%s [%s]", req.Method, e.url, key, req.RemoteAddr) + + // Create a command to watch from a given index (default 0). + command := &WatchCommand{Key: key} + if req.Method == "POST" { + sinceIndex, err := strconv.ParseUint(string(req.FormValue("index")), 10, 64) + if err != nil { + return etcdErr.NewError(203, "Watch From Index", store.UndefIndex, store.UndefTerm) + } + command.SinceIndex = sinceIndex + } + + // Apply the command and write the response. + event, err := command.Apply(e.raftServer.Server) + if err != nil { + return etcdErr.NewError(500, key, store.UndefIndex, store.UndefTerm) + } + + event, _ := event.(*store.Event) + response := eventToResponse(event) + b, _ := json.Marshal(response) + w.WriteHeader(http.StatusOK) + w.Write(b) + + return nil +} From 594c2cab47992d705e6d68240e8d368c19469b81 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Fri, 11 Oct 2013 00:07:22 -0600 Subject: [PATCH 072/247] Refactor v2 API into server/v2. --- command.go | 52 ----- etcd.go | 13 +- etcd_handlers.go | 364 -------------------------------- server/server.go | 60 ++++-- server/v1/delete_key_handler.go | 22 +- server/v1/dispatch.go | 38 ++-- server/v1/get_key_handler.go | 9 +- server/v1/install.go | 15 -- server/v1/set_key_handler.go | 6 +- server/v1/util.go | 38 ---- server/v1/v1.go | 50 +++++ server/v1/watch_key_handler.go | 15 +- server/v2/handlers.go | 364 ++++++++++++++++++++++++++++++++ 13 files changed, 509 insertions(+), 537 deletions(-) delete mode 100644 etcd_handlers.go delete mode 100644 server/v1/install.go delete mode 100644 server/v1/util.go create mode 100644 server/v1/v1.go create mode 100644 server/v2/handlers.go diff --git a/command.go b/command.go index 0a330c23269..dec76216c14 100644 --- a/command.go +++ b/command.go @@ -107,32 +107,6 @@ func (c *TestAndSetCommand) Apply(server *raft.Server) (interface{}, error) { return e, nil } -// Get command -type GetCommand struct { - Key string `json:"key"` - Recursive bool `json:"recursive"` - Sorted bool `json:"sorted"` -} - -// The name of the get command in the log -func (c *GetCommand) CommandName() string { - return commandName("get") -} - -// Get the value of key -func (c *GetCommand) Apply(server *raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(*store.Store) - - e, err := s.Get(c.Key, c.Recursive, c.Sorted, server.CommitIndex(), server.Term()) - - if err != nil { - debug(err) - return nil, err - } - - return e, nil -} - // Delete command type DeleteCommand struct { Key string `json:"key"` @@ -158,32 +132,6 @@ func (c *DeleteCommand) Apply(server *raft.Server) (interface{}, error) { return e, nil } -// Watch command -type WatchCommand struct { - Key string `json:"key"` - SinceIndex uint64 `json:"sinceIndex"` - Recursive bool `json:"recursive"` -} - -// The name of the watch command in the log -func (c *WatchCommand) CommandName() string { - return commandName("watch") -} - -func (c *WatchCommand) Apply(server *raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(*store.Store) - - eventChan, err := s.Watch(c.Key, c.Recursive, c.SinceIndex, server.CommitIndex(), server.Term()) - - if err != nil { - return nil, err - } - - e := <-eventChan - - return e, nil -} - // JoinCommand type JoinCommand struct { RaftVersion string `json:"raftVersion"` diff --git a/etcd.go b/etcd.go index 775d6a18420..8a1c6304269 100644 --- a/etcd.go +++ b/etcd.go @@ -3,14 +3,13 @@ package main import ( "crypto/tls" "flag" - "fmt" "io/ioutil" - "net/url" "os" "strings" "time" "github.com/coreos/etcd/store" + "github.com/coreos/etcd/server" "github.com/coreos/go-raft" ) @@ -154,8 +153,6 @@ func main() { raft.SetLogLevel(raft.Debug) } - parseCorsFlag() - if machines != "" { cluster = strings.Split(machines, ",") } else if machinesFile != "" { @@ -204,10 +201,12 @@ func main() { r := newRaftServer(info.Name, info.RaftURL, info.RaftListenHost, &raftTLSConfig, &info.RaftTLS) snapConf = r.newSnapshotConf() - e = newEtcdServer(info.Name, info.EtcdURL, info.EtcdListenHost, &etcdTLSConfig, &info.EtcdTLS, r) + s := server.New(info.Name, info.EtcdURL, info.EtcdListenHost, &etcdTLSConfig, &info.EtcdTLS, r) + if err := e.AllowOrigins(cors); err != nil { + panic(err) + } r.ListenAndServe() - e.ListenAndServe() - + s.ListenAndServe() } diff --git a/etcd_handlers.go b/etcd_handlers.go deleted file mode 100644 index 0d294de7d4a..00000000000 --- a/etcd_handlers.go +++ /dev/null @@ -1,364 +0,0 @@ -package main - -import ( - "encoding/json" - "fmt" - "net/http" - "strconv" - "strings" - - etcdErr "github.com/coreos/etcd/error" - "github.com/coreos/etcd/store" - "github.com/coreos/go-raft" -) - -//------------------------------------------------------------------- -// Handlers to handle etcd-store related request via etcd url -//------------------------------------------------------------------- - -func NewEtcdMuxer() *http.ServeMux { - // external commands - router := mux.NewRouter() - etcdMux.Handle("/v2/keys/", errorHandler(e.Multiplexer)) - etcdMux.Handle("/v2/leader", errorHandler(e.LeaderHttpHandler)) - etcdMux.Handle("/v2/machines", errorHandler(e.MachinesHttpHandler)) - etcdMux.Handle("/v2/stats/", errorHandler(e.StatsHttpHandler)) - etcdMux.Handle("/version", errorHandler(e.VersionHttpHandler)) - etcdMux.HandleFunc("/test/", TestHttpHandler) - - // backward support - etcdMux.Handle("/v1/keys/", errorHandler(e.MultiplexerV1)) - etcdMux.Handle("/v1/leader", errorHandler(e.LeaderHttpHandler)) - etcdMux.Handle("/v1/machines", errorHandler(e.MachinesHttpHandler)) - etcdMux.Handle("/v1/stats/", errorHandler(e.StatsHttpHandler)) - - return etcdMux -} - -type errorHandler func(http.ResponseWriter, *http.Request) error - -// addCorsHeader parses the request Origin header and loops through the user -// provided allowed origins and sets the Access-Control-Allow-Origin header if -// there is a match. -func addCorsHeader(w http.ResponseWriter, r *http.Request) { - val, ok := corsList["*"] - if val && ok { - w.Header().Add("Access-Control-Allow-Origin", "*") - return - } - - requestOrigin := r.Header.Get("Origin") - val, ok = corsList[requestOrigin] - if val && ok { - w.Header().Add("Access-Control-Allow-Origin", requestOrigin) - return - } -} - -func (fn errorHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { - addCorsHeader(w, r) - if e := fn(w, r); e != nil { - if etcdErr, ok := e.(*etcdErr.Error); ok { - debug("Return error: ", (*etcdErr).Error()) - etcdErr.Write(w) - } else { - http.Error(w, e.Error(), http.StatusInternalServerError) - } - } -} - -// Multiplex GET/POST/DELETE request to corresponding handlers -func (e *etcdServer) Multiplexer(w http.ResponseWriter, req *http.Request) error { - - switch req.Method { - case "GET": - return e.GetHttpHandler(w, req) - case "POST": - return e.CreateHttpHandler(w, req) - case "PUT": - return e.UpdateHttpHandler(w, req) - case "DELETE": - return e.DeleteHttpHandler(w, req) - default: - w.WriteHeader(http.StatusMethodNotAllowed) - return nil - } - - return nil -} - -//-------------------------------------- -// State sensitive handlers -// Set/Delete will dispatch to leader -//-------------------------------------- - -func (e *etcdServer) CreateHttpHandler(w http.ResponseWriter, req *http.Request) error { - key := getNodePath(req.URL.Path) - - debugf("recv.post[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) - - value := req.FormValue("value") - - expireTime, err := durationToExpireTime(req.FormValue("ttl")) - - if err != nil { - return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create", store.UndefIndex, store.UndefTerm) - } - - command := &CreateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - } - - if req.FormValue("incremental") == "true" { - command.IncrementalSuffix = true - } - - return e.dispatchEtcdCommand(command, w, req) - -} - -func (e *etcdServer) UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { - key := getNodePath(req.URL.Path) - - debugf("recv.put[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) - - req.ParseForm() - - value := req.Form.Get("value") - - expireTime, err := durationToExpireTime(req.Form.Get("ttl")) - - if err != nil { - return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", store.UndefIndex, store.UndefTerm) - } - - // update should give at least one option - if value == "" && expireTime.Sub(store.Permanent) == 0 { - return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", store.UndefIndex, store.UndefTerm) - } - - prevValue, valueOk := req.Form["prevValue"] - - prevIndexStr, indexOk := req.Form["prevIndex"] - - if !valueOk && !indexOk { // update without test - command := &UpdateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - } - - return e.dispatchEtcdCommand(command, w, req) - - } else { // update with test - var prevIndex uint64 - - if indexOk { - prevIndex, err = strconv.ParseUint(prevIndexStr[0], 10, 64) - - // bad previous index - if err != nil { - return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Update", store.UndefIndex, store.UndefTerm) - } - } else { - prevIndex = 0 - } - - command := &TestAndSetCommand{ - Key: key, - Value: value, - PrevValue: prevValue[0], - PrevIndex: prevIndex, - } - - return e.dispatchEtcdCommand(command, w, req) - } - -} - -// Delete Handler -func (e *etcdServer) DeleteHttpHandler(w http.ResponseWriter, req *http.Request) error { - key := getNodePath(req.URL.Path) - - debugf("recv.delete[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) - - command := &DeleteCommand{ - Key: key, - } - - if req.FormValue("recursive") == "true" { - command.Recursive = true - } - - return e.dispatchEtcdCommand(command, w, req) -} - -// Dispatch the command to leader -func (e *etcdServer) dispatchEtcdCommand(c Command, w http.ResponseWriter, req *http.Request) error { - return e.raftServer.dispatch(c, w, req, nameToEtcdURL) -} - -//-------------------------------------- -// State non-sensitive handlers -// command with consistent option will -// still dispatch to the leader -//-------------------------------------- - -// Handler to return the current leader's raft address -func (e *etcdServer) LeaderHttpHandler(w http.ResponseWriter, req *http.Request) error { - r := e.raftServer - - leader := r.Leader() - - if leader != "" { - w.WriteHeader(http.StatusOK) - raftURL, _ := nameToRaftURL(leader) - w.Write([]byte(raftURL)) - - return nil - } else { - return etcdErr.NewError(etcdErr.EcodeLeaderElect, "", store.UndefIndex, store.UndefTerm) - } -} - -// Handler to return all the known machines in the current cluster -func (e *etcdServer) MachinesHttpHandler(w http.ResponseWriter, req *http.Request) error { - machines := e.raftServer.getMachines(nameToEtcdURL) - - w.WriteHeader(http.StatusOK) - w.Write([]byte(strings.Join(machines, ", "))) - - return nil -} - -// Handler to return the current version of etcd -func (e *etcdServer) VersionHttpHandler(w http.ResponseWriter, req *http.Request) error { - w.WriteHeader(http.StatusOK) - fmt.Fprintf(w, "etcd %s", releaseVersion) - - return nil -} - -// Handler to return the basic stats of etcd -func (e *etcdServer) StatsHttpHandler(w http.ResponseWriter, req *http.Request) error { - option := req.URL.Path[len("/v1/stats/"):] - w.WriteHeader(http.StatusOK) - - r := e.raftServer - - switch option { - case "self": - w.Write(r.Stats()) - case "leader": - if r.State() == raft.Leader { - w.Write(r.PeerStats()) - } else { - leader := r.Leader() - // current no leader - if leader == "" { - return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) - } - hostname, _ := nameToEtcdURL(leader) - redirect(hostname, w, req) - } - case "store": - w.Write(etcdStore.JsonStats()) - } - - return nil -} - -func (e *etcdServer) GetHttpHandler(w http.ResponseWriter, req *http.Request) error { - var err error - var event interface{} - - r := e.raftServer - - debugf("recv.get[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) - - if req.FormValue("consistent") == "true" && r.State() != raft.Leader { - // help client to redirect the request to the current leader - leader := r.Leader() - hostname, _ := nameToEtcdURL(leader) - redirect(hostname, w, req) - return nil - } - - key := getNodePath(req.URL.Path) - - recursive := req.FormValue("recursive") - - if req.FormValue("wait") == "true" { // watch - command := &WatchCommand{ - Key: key, - } - - if recursive == "true" { - command.Recursive = true - } - - indexStr := req.FormValue("wait_index") - if indexStr != "" { - sinceIndex, err := strconv.ParseUint(indexStr, 10, 64) - - if err != nil { - return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index", store.UndefIndex, store.UndefTerm) - } - - command.SinceIndex = sinceIndex - } - - event, err = command.Apply(r.Server) - - } else { //get - - command := &GetCommand{ - Key: key, - } - - sorted := req.FormValue("sorted") - if sorted == "true" { - command.Sorted = true - } - - if recursive == "true" { - command.Recursive = true - } - - event, err = command.Apply(r.Server) - } - - if err != nil { - return err - - } else { - event, _ := event.(*store.Event) - bytes, _ := json.Marshal(event) - - w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) - w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term)) - w.WriteHeader(http.StatusOK) - - w.Write(bytes) - - return nil - } - -} - -// TestHandler -func TestHttpHandler(w http.ResponseWriter, req *http.Request) { - testType := req.URL.Path[len("/test/"):] - - if testType == "speed" { - directSet() - w.WriteHeader(http.StatusOK) - w.Write([]byte("speed test success")) - - return - } - - w.WriteHeader(http.StatusBadRequest) -} diff --git a/server/server.go b/server/server.go index 8deb2fc24ce..4bf1083018a 100644 --- a/server/server.go +++ b/server/server.go @@ -3,12 +3,20 @@ package server import ( "github.com/gorilla/mux" "net/http" + "net/url" ) -// The Server provides an HTTP interface to the underlying data store. -type Server struct { +// The Server provides an HTTP interface to the underlying store. +type Server interface { + CommitIndex() uint64 + Term() uint64 + Dispatch(Command, http.ResponseWriter, *http.Request) +} + +// This is the default implementation of the Server interface. +type server struct { http.Server - raftServer *raftServer + raftServer *raft.Server name string url string tlsConf *TLSConfig @@ -17,8 +25,8 @@ type Server struct { } // Creates a new Server. -func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, raftServer *raftServer) *Server { - s := &etcdServer{ +func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, raftServer *raft.Server) *Server { + s := &server{ Server: http.Server{ Handler: mux.NewRouter(), TLSConfig: &tlsConf.Server, @@ -31,20 +39,44 @@ func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsI raftServer: raftServer, } - // TODO: Move to main.go. // Install the routes for each version of the API. - // v1.Install(s) - // v2.Install(s) + s.installV1() return s } +// The current Raft committed index. +func (s *server) CommitIndex() uint64 { + return c.raftServer.CommitIndex() +} + +// The current Raft term. +func (s *server) Term() uint64 { + return c.raftServer.Term() +} + +// Executes a command against the Raft server. +func (s *server) Do(c Command, localOnly bool) (interface{}, error) { + return c.raftServer.Do(s.RaftServer().Server) +} + +func (s *server) installV1() { + s.handleFunc("/v1/keys/{key:.*}", v1.GetKeyHandler).Methods("GET") + s.handleFunc("/v1/keys/{key:.*}", v1.SetKeyHandler).Methods("POST", "PUT") + s.handleFunc("/v1/keys/{key:.*}", v1.DeleteKeyHandler).Methods("DELETE") + + s.handleFunc("/v1/watch/{key:.*}", v1.WatchKeyHandler).Methods("GET", "POST") +} + // Adds a server handler to the router. -func (s *Server) HandleFunc(path string, f func(http.ResponseWriter, *http.Request, *server.Server) error) *mux.Route { +func (s *server) handleFunc(path string, f func(http.ResponseWriter, *http.Request, Server) error) *mux.Route { r := s.Handler.(*mux.Router) // Wrap the standard HandleFunc interface to pass in the server reference. return r.HandleFunc(path, func(w http.ResponseWriter, req *http.Request) { + // Log request. + debugf("[recv] %s %s [%s]", req.Method, s.url, req.URL.Path, req.RemoteAddr) + // Write CORS header. if s.OriginAllowed("*") { w.Header().Add("Access-Control-Allow-Origin", "*") @@ -65,7 +97,7 @@ func (s *Server) HandleFunc(path string, f func(http.ResponseWriter, *http.Reque } // Start to listen and response etcd client command -func (s *Server) ListenAndServe() { +func (s *server) ListenAndServe() { infof("etcd server [name %s, listen on %s, advertised url %s]", s.name, s.Server.Addr, s.url) if s.tlsConf.Scheme == "http" { @@ -76,7 +108,7 @@ func (s *Server) ListenAndServe() { } // Sets a comma-delimited list of origins that are allowed. -func (s *Server) AllowOrigins(origins string) error { +func (s *server) AllowOrigins(origins string) error { // Construct a lookup of all origins. m := make(map[string]bool) for _, v := range strings.Split(cors, ",") { @@ -87,12 +119,12 @@ func (s *Server) AllowOrigins(origins string) error { } m[v] = true } - s.origins = m + s.corsOrigins = m return nil } // Determines whether the server will allow a given CORS origin. -func (s *Server) OriginAllowed(origin string) { - return s.origins["*"] || s.origins[origin] +func (s *server) OriginAllowed(origin string) { + return s.corsOrigins["*"] || s.corsOrigins[origin] } diff --git a/server/v1/delete_key_handler.go b/server/v1/delete_key_handler.go index d2d450b6af5..8b9e315bdd1 100644 --- a/server/v1/delete_key_handler.go +++ b/server/v1/delete_key_handler.go @@ -1,13 +1,15 @@ package v1 -func deleteKeyHandler(w http.ResponseWriter, req *http.Request, e *etcdServer) error { - key := req.URL.Path[len("/v1/keys/"):] - - debugf("[recv] DELETE %v/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) - - command := &DeleteCommand{ - Key: key, - } - - return dispatchEtcdCommandV1(command, w, req) +import ( + "encoding/json" + "github.com/coreos/etcd/store" + "net/http" +) + +// Removes a key from the store. +func DeleteKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { + vars := mux.Vars(req) + key := "/" + vars["key"] + command := &DeleteCommand{Key: key} + return s.Dispatch(command, w, req) } diff --git a/server/v1/dispatch.go b/server/v1/dispatch.go index e1aa0292234..7dd13861c99 100644 --- a/server/v1/dispatch.go +++ b/server/v1/dispatch.go @@ -1,40 +1,40 @@ package v1 // Dispatch the command to leader. -func dispatchCommand(c Command, w http.ResponseWriter, req *http.Request) error { - return dispatch(c, w, req, nameToEtcdURL) +func dispatchCommand(c Command, w http.ResponseWriter, req *http.Request, s *server.Server) error { + return dispatch(c, w, req, s, nameToEtcdURL) } // Dispatches a command to a given URL. -func dispatch(c Command, w http.ResponseWriter, req *http.Request, toURL func(name string) (string, bool)) error { - r := e.raftServer +func dispatch(c Command, w http.ResponseWriter, req *http.Request, s *server.Server, toURL func(name string) (string, bool)) error { + r := s.raftServer if r.State() == raft.Leader { - if event, err := r.Do(c); err != nil { + event, err := r.Do(c) + if err != nil { return err - } else { - if event == nil { - return etcdErr.NewError(300, "Empty result from raft", store.UndefIndex, store.UndefTerm) - } - - event, _ := event.(*store.Event) + } - response := eventToResponse(event) - bytes, _ := json.Marshal(response) + if event == nil { + return etcdErr.NewError(300, "Empty result from raft", store.UndefIndex, store.UndefTerm) + } - w.WriteHeader(http.StatusOK) - w.Write(bytes) - return nil + event, _ := event.(*store.Event) + response := eventToResponse(event) + b, _ := json.Marshal(response) + w.WriteHeader(http.StatusOK) + w.Write(b) - } + return nil } else { leader := r.Leader() - // current no leader + + // No leader available. if leader == "" { return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) } - url, _ := toURL(leader) + url, _ := toURL(leader) redirect(url, w, req) return nil diff --git a/server/v1/get_key_handler.go b/server/v1/get_key_handler.go index 391a47a7ab4..6469313a341 100644 --- a/server/v1/get_key_handler.go +++ b/server/v1/get_key_handler.go @@ -7,15 +7,12 @@ import ( ) // Retrieves the value for a given key. -func getKeyHandler(w http.ResponseWriter, req *http.Request, e *etcdServer) error { +func GetKeyHandler(w http.ResponseWriter, req *http.Request, e *etcdServer) error { vars := mux.Vars(req) key := "/" + vars["key"] - debugf("[recv] GET %s/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) - - // Execute the command. - command := &GetCommand{Key: key} - event, err := command.Apply(e.raftServer.Server) + // Retrieve the key from the store. + event, err := s.Store().Get(key, false, false, s.CommitIndex(), s.Term()) if err != nil { return err } diff --git a/server/v1/install.go b/server/v1/install.go deleted file mode 100644 index 18b664cfea3..00000000000 --- a/server/v1/install.go +++ /dev/null @@ -1,15 +0,0 @@ -package v1 - -import ( - "github.com/coreos/etcd/server" - "github.com/gorilla/mux" -) - -// Installs the routes for version 1 of the API on to a server. -func Install(s *server.Server) { - s.HandleFunc("/v1/keys/{key:.*}", getKeyHandler).Methods("GET") - s.HandleFunc("/v1/keys/{key:.*}", setKeyHandler).Methods("POST", "PUT") - s.HandleFunc("/v1/keys/{key:.*}", deleteKeyHandler).Methods("DELETE") - - s.HandleFunc("/v1/watch/{key:.*}", watchKeyHandler).Methods("GET", "POST") -} diff --git a/server/v1/set_key_handler.go b/server/v1/set_key_handler.go index 26f6db01494..ad629e48f26 100644 --- a/server/v1/set_key_handler.go +++ b/server/v1/set_key_handler.go @@ -7,12 +7,10 @@ import ( ) // Sets the value for a given key. -func setKeyHandler(w http.ResponseWriter, req *http.Request, e *etcdServer) error { +func SetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) key := "/" + vars["key"] - debugf("[recv] POST %v/v1/keys/%s [%s]", e.url, key, req.RemoteAddr) - req.ParseForm() // Parse non-blank value. @@ -46,5 +44,5 @@ func setKeyHandler(w http.ResponseWriter, req *http.Request, e *etcdServer) erro } } - return dispatchEtcdCommand(command, w, req) + return s.Dispatch(command, w, req) } diff --git a/server/v1/util.go b/server/v1/util.go deleted file mode 100644 index 8bfc22f23e6..00000000000 --- a/server/v1/util.go +++ /dev/null @@ -1,38 +0,0 @@ -package v1 - -// Converts an event object into a response object. -func eventToResponse(event *store.Event) interface{} { - if !event.Dir { - response := &store.Response{ - Action: event.Action, - Key: event.Key, - Value: event.Value, - PrevValue: event.PrevValue, - Index: event.Index, - TTL: event.TTL, - Expiration: event.Expiration, - } - - if response.Action == store.Create || response.Action == store.Update { - response.Action = "set" - if response.PrevValue == "" { - response.NewKey = true - } - } - - return response - } else { - responses := make([]*store.Response, len(event.KVPairs)) - - for i, kv := range event.KVPairs { - responses[i] = &store.Response{ - Action: event.Action, - Key: kv.Key, - Value: kv.Value, - Dir: kv.Dir, - Index: event.Index, - } - } - return responses - } -} diff --git a/server/v1/v1.go b/server/v1/v1.go new file mode 100644 index 00000000000..709463b0793 --- /dev/null +++ b/server/v1/v1.go @@ -0,0 +1,50 @@ +package v1 + +import ( + "github.com/coreos/etcd/server" + "github.com/gorilla/mux" +) + +// The Server interface provides all the methods required for the v1 API. +type Server interface { + CommitIndex() uint64 + Term() uint64 + Dispatch(http.ResponseWriter, *http.Request, Command) +} + +// Converts an event object into a response object. +func eventToResponse(event *store.Event) interface{} { + if !event.Dir { + response := &store.Response{ + Action: event.Action, + Key: event.Key, + Value: event.Value, + PrevValue: event.PrevValue, + Index: event.Index, + TTL: event.TTL, + Expiration: event.Expiration, + } + + if response.Action == store.Create || response.Action == store.Update { + response.Action = "set" + if response.PrevValue == "" { + response.NewKey = true + } + } + + return response + } else { + responses := make([]*store.Response, len(event.KVPairs)) + + for i, kv := range event.KVPairs { + responses[i] = &store.Response{ + Action: event.Action, + Key: kv.Key, + Value: kv.Value, + Dir: kv.Dir, + Index: event.Index, + } + } + return responses + } +} diff --git a/server/v1/watch_key_handler.go b/server/v1/watch_key_handler.go index 0b32314aa75..8ad103b8be4 100644 --- a/server/v1/watch_key_handler.go +++ b/server/v1/watch_key_handler.go @@ -7,27 +7,26 @@ import ( ) // Watches a given key prefix for changes. -func watchKeyHandler(w http.ResponseWriter, req *http.Request, e *etcdServer) error { +func WatchKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { + var err error vars := mux.Vars(req) key := "/" + vars["key"] - debugf("[recv] %s %s/watch/%s [%s]", req.Method, e.url, key, req.RemoteAddr) - // Create a command to watch from a given index (default 0). - command := &WatchCommand{Key: key} + sinceIndex := 0 if req.Method == "POST" { - sinceIndex, err := strconv.ParseUint(string(req.FormValue("index")), 10, 64) + sinceIndex, err = strconv.ParseUint(string(req.FormValue("index")), 10, 64) if err != nil { return etcdErr.NewError(203, "Watch From Index", store.UndefIndex, store.UndefTerm) } - command.SinceIndex = sinceIndex } - // Apply the command and write the response. - event, err := command.Apply(e.raftServer.Server) + // Start the watcher on the store. + c, err := s.Store().Watch(key, false, sinceIndex, s.CommitIndex(), s.Term()) if err != nil { return etcdErr.NewError(500, key, store.UndefIndex, store.UndefTerm) } + event := <-c event, _ := event.(*store.Event) response := eventToResponse(event) diff --git a/server/v2/handlers.go b/server/v2/handlers.go new file mode 100644 index 00000000000..7f102953109 --- /dev/null +++ b/server/v2/handlers.go @@ -0,0 +1,364 @@ +package main + +import ( + "encoding/json" + "fmt" + "net/http" + "strconv" + "strings" + + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" +) + +//------------------------------------------------------------------- +// Handlers to handle etcd-store related request via etcd url +//------------------------------------------------------------------- + +func NewEtcdMuxer() *http.ServeMux { + // external commands + router := mux.NewRouter() + etcdMux.Handle("/v2/keys/", errorHandler(e.Multiplexer)) + etcdMux.Handle("/v2/leader", errorHandler(e.LeaderHttpHandler)) + etcdMux.Handle("/v2/machines", errorHandler(e.MachinesHttpHandler)) + etcdMux.Handle("/v2/stats/", errorHandler(e.StatsHttpHandler)) + etcdMux.Handle("/version", errorHandler(e.VersionHttpHandler)) + etcdMux.HandleFunc("/test/", TestHttpHandler) + + // backward support + etcdMux.Handle("/v1/keys/", errorHandler(e.MultiplexerV1)) + etcdMux.Handle("/v1/leader", errorHandler(e.LeaderHttpHandler)) + etcdMux.Handle("/v1/machines", errorHandler(e.MachinesHttpHandler)) + etcdMux.Handle("/v1/stats/", errorHandler(e.StatsHttpHandler)) + + return etcdMux +} + +type errorHandler func(http.ResponseWriter, *http.Request) error + +// addCorsHeader parses the request Origin header and loops through the user +// provided allowed origins and sets the Access-Control-Allow-Origin header if +// there is a match. +func addCorsHeader(w http.ResponseWriter, r *http.Request) { + val, ok := corsList["*"] + if val && ok { + w.Header().Add("Access-Control-Allow-Origin", "*") + return + } + + requestOrigin := r.Header.Get("Origin") + val, ok = corsList[requestOrigin] + if val && ok { + w.Header().Add("Access-Control-Allow-Origin", requestOrigin) + return + } +} + +func (fn errorHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { + addCorsHeader(w, r) + if e := fn(w, r); e != nil { + if etcdErr, ok := e.(*etcdErr.Error); ok { + debug("Return error: ", (*etcdErr).Error()) + etcdErr.Write(w) + } else { + http.Error(w, e.Error(), http.StatusInternalServerError) + } + } +} + +// Multiplex GET/POST/DELETE request to corresponding handlers +func (e *etcdServer) Multiplexer(w http.ResponseWriter, req *http.Request) error { + + switch req.Method { + case "GET": + return e.GetHttpHandler(w, req) + case "POST": + return e.CreateHttpHandler(w, req) + case "PUT": + return e.UpdateHttpHandler(w, req) + case "DELETE": + return e.DeleteHttpHandler(w, req) + default: + w.WriteHeader(http.StatusMethodNotAllowed) + return nil + } + + return nil +} + +//-------------------------------------- +// State sensitive handlers +// Set/Delete will dispatch to leader +//-------------------------------------- + +func (e *etcdServer) CreateHttpHandler(w http.ResponseWriter, req *http.Request) error { + key := getNodePath(req.URL.Path) + + debugf("recv.post[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) + + value := req.FormValue("value") + + expireTime, err := durationToExpireTime(req.FormValue("ttl")) + + if err != nil { + return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create", store.UndefIndex, store.UndefTerm) + } + + command := &CreateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + } + + if req.FormValue("incremental") == "true" { + command.IncrementalSuffix = true + } + + return e.dispatchEtcdCommand(command, w, req) + +} + +func (e *etcdServer) UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { + key := getNodePath(req.URL.Path) + + debugf("recv.put[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) + + req.ParseForm() + + value := req.Form.Get("value") + + expireTime, err := durationToExpireTime(req.Form.Get("ttl")) + + if err != nil { + return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", store.UndefIndex, store.UndefTerm) + } + + // update should give at least one option + if value == "" && expireTime.Sub(store.Permanent) == 0 { + return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", store.UndefIndex, store.UndefTerm) + } + + prevValue, valueOk := req.Form["prevValue"] + + prevIndexStr, indexOk := req.Form["prevIndex"] + + if !valueOk && !indexOk { // update without test + command := &UpdateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + } + + return e.dispatchEtcdCommand(command, w, req) + + } else { // update with test + var prevIndex uint64 + + if indexOk { + prevIndex, err = strconv.ParseUint(prevIndexStr[0], 10, 64) + + // bad previous index + if err != nil { + return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Update", store.UndefIndex, store.UndefTerm) + } + } else { + prevIndex = 0 + } + + command := &TestAndSetCommand{ + Key: key, + Value: value, + PrevValue: prevValue[0], + PrevIndex: prevIndex, + } + + return e.dispatchEtcdCommand(command, w, req) + } + +} + +// Delete Handler +func (e *etcdServer) DeleteHttpHandler(w http.ResponseWriter, req *http.Request) error { + key := getNodePath(req.URL.Path) + + debugf("recv.delete[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) + + command := &DeleteCommand{ + Key: key, + } + + if req.FormValue("recursive") == "true" { + command.Recursive = true + } + + return e.dispatchEtcdCommand(command, w, req) +} + +// Dispatch the command to leader +func (e *etcdServer) dispatchEtcdCommand(c Command, w http.ResponseWriter, req *http.Request) error { + return e.raftServer.dispatch(c, w, req, nameToEtcdURL) +} + +//-------------------------------------- +// State non-sensitive handlers +// command with consistent option will +// still dispatch to the leader +//-------------------------------------- + +// Handler to return the current leader's raft address +func (e *etcdServer) LeaderHttpHandler(w http.ResponseWriter, req *http.Request) error { + r := e.raftServer + + leader := r.Leader() + + if leader != "" { + w.WriteHeader(http.StatusOK) + raftURL, _ := nameToRaftURL(leader) + w.Write([]byte(raftURL)) + + return nil + } else { + return etcdErr.NewError(etcdErr.EcodeLeaderElect, "", store.UndefIndex, store.UndefTerm) + } +} + +// Handler to return all the known machines in the current cluster +func (e *etcdServer) MachinesHttpHandler(w http.ResponseWriter, req *http.Request) error { + machines := e.raftServer.getMachines(nameToEtcdURL) + + w.WriteHeader(http.StatusOK) + w.Write([]byte(strings.Join(machines, ", "))) + + return nil +} + +// Handler to return the current version of etcd +func (e *etcdServer) VersionHttpHandler(w http.ResponseWriter, req *http.Request) error { + w.WriteHeader(http.StatusOK) + fmt.Fprintf(w, "etcd %s", releaseVersion) + + return nil +} + +// Handler to return the basic stats of etcd +func (e *etcdServer) StatsHttpHandler(w http.ResponseWriter, req *http.Request) error { + option := req.URL.Path[len("/v1/stats/"):] + w.WriteHeader(http.StatusOK) + + r := e.raftServer + + switch option { + case "self": + w.Write(r.Stats()) + case "leader": + if r.State() == raft.Leader { + w.Write(r.PeerStats()) + } else { + leader := r.Leader() + // current no leader + if leader == "" { + return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) + } + hostname, _ := nameToEtcdURL(leader) + redirect(hostname, w, req) + } + case "store": + w.Write(etcdStore.JsonStats()) + } + + return nil +} + +func (e *etcdServer) GetHttpHandler(w http.ResponseWriter, req *http.Request) error { + var err error + var event interface{} + + r := e.raftServer + + debugf("recv.get[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) + + if req.FormValue("consistent") == "true" && r.State() != raft.Leader { + // help client to redirect the request to the current leader + leader := r.Leader() + hostname, _ := nameToEtcdURL(leader) + redirect(hostname, w, req) + return nil + } + + key := getNodePath(req.URL.Path) + + recursive := req.FormValue("recursive") + + if req.FormValue("wait") == "true" { // watch + command := &WatchCommand{ + Key: key, + } + + if recursive == "true" { + command.Recursive = true + } + + indexStr := req.FormValue("wait_index") + if indexStr != "" { + sinceIndex, err := strconv.ParseUint(indexStr, 10, 64) + + if err != nil { + return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index", store.UndefIndex, store.UndefTerm) + } + + command.SinceIndex = sinceIndex + } + + event, err = command.Apply(r.Server) + + } else { //get + + command := &GetCommand{ + Key: key, + } + + sorted := req.FormValue("sorted") + if sorted == "true" { + command.Sorted = true + } + + if recursive == "true" { + command.Recursive = true + } + + event, err = command.Apply(r.Server) + } + + if err != nil { + return err + + } else { + event, _ := event.(*store.Event) + bytes, _ := json.Marshal(event) + + w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) + w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term)) + w.WriteHeader(http.StatusOK) + + w.Write(bytes) + + return nil + } + +} + +// TestHandler +func TestHttpHandler(w http.ResponseWriter, req *http.Request) { + testType := req.URL.Path[len("/test/"):] + + if testType == "speed" { + directSet() + w.WriteHeader(http.StatusOK) + w.Write([]byte("speed test success")) + + return + } + + w.WriteHeader(http.StatusBadRequest) +} From 89334df5aecc4461ed6e2b81236f6d95248bcd4c Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Fri, 11 Oct 2013 01:02:38 -0600 Subject: [PATCH 073/247] Refactor commands. --- command.go | 255 ------------------ command/command.go | 19 ++ command/create_command.go | 36 +++ command/delete_command.go | 32 +++ command/test_and_set_command.go | 38 +++ command/update_command.go | 35 +++ etcd.go | 22 +- log/log.go | 44 +++ machines.go | 11 - raft_server.go | 24 +- raft_stats.go | 210 --------------- server/join_command.go | 84 ++++++ server/package_stats.go | 25 ++ server/raft_follower_stats.go | 56 ++++ server/raft_server_stats.go | 55 ++++ server/remove_command.go | 68 +++++ server/server.go | 16 +- server/stats_queue.go | 88 ++++++ server/tls_config.go | 11 + server/tls_info.go | 7 + server/v1/delete_key_handler.go | 10 +- server/v1/v1.go | 66 ++--- server/v2/handlers.go | 460 ++++++++++++++++---------------- store/store.go | 10 + util.go | 40 --- 25 files changed, 894 insertions(+), 828 deletions(-) delete mode 100644 command.go create mode 100644 command/command.go create mode 100644 command/create_command.go create mode 100644 command/delete_command.go create mode 100644 command/test_and_set_command.go create mode 100644 command/update_command.go create mode 100644 log/log.go delete mode 100644 raft_stats.go create mode 100644 server/join_command.go create mode 100644 server/package_stats.go create mode 100644 server/raft_follower_stats.go create mode 100644 server/raft_server_stats.go create mode 100644 server/remove_command.go create mode 100644 server/stats_queue.go create mode 100644 server/tls_config.go create mode 100644 server/tls_info.go diff --git a/command.go b/command.go deleted file mode 100644 index dec76216c14..00000000000 --- a/command.go +++ /dev/null @@ -1,255 +0,0 @@ -package main - -import ( - "encoding/binary" - "fmt" - "os" - "path" - "time" - - etcdErr "github.com/coreos/etcd/error" - "github.com/coreos/etcd/store" - "github.com/coreos/go-raft" -) - -const commandPrefix = "etcd:" - -func commandName(name string) string { - return commandPrefix + name -} - -// A command represents an action to be taken on the replicated state machine. -type Command interface { - CommandName() string - Apply(server *raft.Server) (interface{}, error) -} - -// Create command -type CreateCommand struct { - Key string `json:"key"` - Value string `json:"value"` - ExpireTime time.Time `json:"expireTime"` - IncrementalSuffix bool `json:"incrementalSuffix"` - Force bool `json:"force"` -} - -// The name of the create command in the log -func (c *CreateCommand) CommandName() string { - return commandName("create") -} - -// Create node -func (c *CreateCommand) Apply(server *raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(*store.Store) - - e, err := s.Create(c.Key, c.Value, c.IncrementalSuffix, c.Force, c.ExpireTime, server.CommitIndex(), server.Term()) - - if err != nil { - debug(err) - return nil, err - } - - return e, nil -} - -// Update command -type UpdateCommand struct { - Key string `json:"key"` - Value string `json:"value"` - ExpireTime time.Time `json:"expireTime"` -} - -// The name of the update command in the log -func (c *UpdateCommand) CommandName() string { - return commandName("update") -} - -// Update node -func (c *UpdateCommand) Apply(server *raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(*store.Store) - - e, err := s.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) - - if err != nil { - debug(err) - return nil, err - } - - return e, nil -} - -// TestAndSet command -type TestAndSetCommand struct { - Key string `json:"key"` - Value string `json:"value"` - ExpireTime time.Time `json:"expireTime"` - PrevValue string `json: prevValue` - PrevIndex uint64 `json: prevValue` -} - -// The name of the testAndSet command in the log -func (c *TestAndSetCommand) CommandName() string { - return commandName("testAndSet") -} - -// Set the key-value pair if the current value of the key equals to the given prevValue -func (c *TestAndSetCommand) Apply(server *raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(*store.Store) - - e, err := s.TestAndSet(c.Key, c.PrevValue, c.PrevIndex, - c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) - - if err != nil { - debug(err) - return nil, err - } - - return e, nil -} - -// Delete command -type DeleteCommand struct { - Key string `json:"key"` - Recursive bool `json:"recursive"` -} - -// The name of the delete command in the log -func (c *DeleteCommand) CommandName() string { - return commandName("delete") -} - -// Delete the key -func (c *DeleteCommand) Apply(server *raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(*store.Store) - - e, err := s.Delete(c.Key, c.Recursive, server.CommitIndex(), server.Term()) - - if err != nil { - debug(err) - return nil, err - } - - return e, nil -} - -// JoinCommand -type JoinCommand struct { - RaftVersion string `json:"raftVersion"` - Name string `json:"name"` - RaftURL string `json:"raftURL"` - EtcdURL string `json:"etcdURL"` -} - -func newJoinCommand(version, name, raftUrl, etcdUrl string) *JoinCommand { - return &JoinCommand{ - RaftVersion: version, - Name: name, - RaftURL: raftUrl, - EtcdURL: etcdUrl, - } -} - -// The name of the join command in the log -func (c *JoinCommand) CommandName() string { - return commandName("join") -} - -// Join a server to the cluster -func (c *JoinCommand) Apply(server *raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(*store.Store) - r, _ := server.Context().(*raftServer) - - // check if the join command is from a previous machine, who lost all its previous log. - e, _ := s.Get(path.Join("/_etcd/machines", c.Name), false, false, server.CommitIndex(), server.Term()) - - b := make([]byte, 8) - binary.PutUvarint(b, server.CommitIndex()) - - if e != nil { - return b, nil - } - - // check machine number in the cluster - num := machineNum() - if num == maxClusterSize { - debug("Reject join request from ", c.Name) - return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMoreMachine, "", server.CommitIndex(), server.Term()) - } - - addNameToURL(c.Name, c.RaftVersion, c.RaftURL, c.EtcdURL) - - // add peer in raft - err := server.AddPeer(c.Name, "") - - // add machine in etcd storage - key := path.Join("_etcd/machines", c.Name) - value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", c.RaftURL, c.EtcdURL, c.RaftVersion) - s.Create(key, value, false, false, store.Permanent, server.CommitIndex(), server.Term()) - - // add peer stats - if c.Name != r.Name() { - r.followersStats.Followers[c.Name] = &raftFollowerStats{} - r.followersStats.Followers[c.Name].Latency.Minimum = 1 << 63 - } - - return b, err -} - -func (c *JoinCommand) NodeName() string { - return c.Name -} - -// RemoveCommand -type RemoveCommand struct { - Name string `json:"name"` -} - -// The name of the remove command in the log -func (c *RemoveCommand) CommandName() string { - return commandName("remove") -} - -// Remove a server from the cluster -func (c *RemoveCommand) Apply(server *raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(*store.Store) - r, _ := server.Context().(*raftServer) - - // remove machine in etcd storage - key := path.Join("_etcd/machines", c.Name) - - _, err := s.Delete(key, false, server.CommitIndex(), server.Term()) - // delete from stats - delete(r.followersStats.Followers, c.Name) - - if err != nil { - return []byte{0}, err - } - - // remove peer in raft - err = server.RemovePeer(c.Name) - - if err != nil { - return []byte{0}, err - } - - if c.Name == server.Name() { - // the removed node is this node - - // if the node is not replaying the previous logs - // and the node has sent out a join request in this - // start. It is sure that this node received a new remove - // command and need to be removed - if server.CommitIndex() > r.joinIndex && r.joinIndex != 0 { - debugf("server [%s] is removed", server.Name()) - os.Exit(0) - } else { - // else ignore remove - debugf("ignore previous remove command.") - } - } - - b := make([]byte, 8) - binary.PutUvarint(b, server.CommitIndex()) - - return b, err -} diff --git a/command/command.go b/command/command.go new file mode 100644 index 00000000000..50513d5f500 --- /dev/null +++ b/command/command.go @@ -0,0 +1,19 @@ +package command + +import ( + "github.com/coreos/go-raft" +) + +// A command represents an action to be taken on the replicated state machine. +type Command interface { + CommandName() string + Apply(server *raft.Server) (interface{}, error) +} + +// Registers commands to the Raft library. +func Register() { + raft.RegisterCommand(&DeleteCommand{}) + raft.RegisterCommand(&TestAndSetCommand{}) + raft.RegisterCommand(&CreateCommand{}) + raft.RegisterCommand(&UpdateCommand{}) +} diff --git a/command/create_command.go b/command/create_command.go new file mode 100644 index 00000000000..6dd2c5abaf0 --- /dev/null +++ b/command/create_command.go @@ -0,0 +1,36 @@ +package command + +import ( + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" + "time" +) + +// Create command +type CreateCommand struct { + Key string `json:"key"` + Value string `json:"value"` + ExpireTime time.Time `json:"expireTime"` + IncrementalSuffix bool `json:"incrementalSuffix"` + Force bool `json:"force"` +} + +// The name of the create command in the log +func (c *CreateCommand) CommandName() string { + return "etcd:create" +} + +// Create node +func (c *CreateCommand) Apply(server *raft.Server) (interface{}, error) { + s, _ := server.StateMachine().(*store.Store) + + e, err := s.Create(c.Key, c.Value, c.IncrementalSuffix, c.Force, c.ExpireTime, server.CommitIndex(), server.Term()) + + if err != nil { + log.Debug(err) + return nil, err + } + + return e, nil +} diff --git a/command/delete_command.go b/command/delete_command.go new file mode 100644 index 00000000000..a0d03c99d4c --- /dev/null +++ b/command/delete_command.go @@ -0,0 +1,32 @@ +package command + +import ( + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" +) + +// The DeleteCommand removes a key from the Store. +type DeleteCommand struct { + Key string `json:"key"` + Recursive bool `json:"recursive"` +} + +// The name of the delete command in the log +func (c *DeleteCommand) CommandName() string { + return "etcd:delete" +} + +// Delete the key +func (c *DeleteCommand) Apply(server *raft.Server) (interface{}, error) { + s, _ := server.StateMachine().(*store.Store) + + e, err := s.Delete(c.Key, c.Recursive, server.CommitIndex(), server.Term()) + + if err != nil { + log.Debug(err) + return nil, err + } + + return e, nil +} diff --git a/command/test_and_set_command.go b/command/test_and_set_command.go new file mode 100644 index 00000000000..4d723e22130 --- /dev/null +++ b/command/test_and_set_command.go @@ -0,0 +1,38 @@ +package command + +import ( + "time" + + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" +) + +// The TestAndSetCommand performs a conditional update on a key in the store. +type TestAndSetCommand struct { + Key string `json:"key"` + Value string `json:"value"` + ExpireTime time.Time `json:"expireTime"` + PrevValue string `json: prevValue` + PrevIndex uint64 `json: prevIndex` +} + +// The name of the testAndSet command in the log +func (c *TestAndSetCommand) CommandName() string { + return "etcd:testAndSet" +} + +// Set the key-value pair if the current value of the key equals to the given prevValue +func (c *TestAndSetCommand) Apply(server *raft.Server) (interface{}, error) { + s, _ := server.StateMachine().(*store.Store) + + e, err := s.TestAndSet(c.Key, c.PrevValue, c.PrevIndex, + c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) + + if err != nil { + log.Debug(err) + return nil, err + } + + return e, nil +} diff --git a/command/update_command.go b/command/update_command.go new file mode 100644 index 00000000000..245e3c1c7fc --- /dev/null +++ b/command/update_command.go @@ -0,0 +1,35 @@ +package command + +import ( + "time" + + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" +) + +// The UpdateCommand updates the value of a key in the Store. +type UpdateCommand struct { + Key string `json:"key"` + Value string `json:"value"` + ExpireTime time.Time `json:"expireTime"` +} + +// The name of the update command in the log +func (c *UpdateCommand) CommandName() string { + return "etcd:update" +} + +// Update node +func (c *UpdateCommand) Apply(server *raft.Server) (interface{}, error) { + s, _ := server.StateMachine().(*store.Store) + + e, err := s.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) + + if err != nil { + log.Debug(err) + return nil, err + } + + return e, nil +} diff --git a/etcd.go b/etcd.go index 8a1c6304269..79bda7b6779 100644 --- a/etcd.go +++ b/etcd.go @@ -8,8 +8,9 @@ import ( "strings" "time" - "github.com/coreos/etcd/store" + "github.com/coreos/etcd/log" "github.com/coreos/etcd/server" + "github.com/coreos/etcd/store" "github.com/coreos/go-raft" ) @@ -20,7 +21,6 @@ import ( //------------------------------------------------------------------------------ var ( - verbose bool veryVerbose bool machines string @@ -43,11 +43,11 @@ var ( cpuprofile string - cors string + cors string ) func init() { - flag.BoolVar(&verbose, "v", false, "verbose logging") + flag.BoolVar(&log.Verbose, "v", false, "verbose logging") flag.BoolVar(&veryVerbose, "vv", false, "very verbose logging") flag.StringVar(&machines, "C", "", "the ip address and port of a existing machines in the cluster, sepearate by comma") @@ -97,12 +97,6 @@ const ( // //------------------------------------------------------------------------------ -type TLSInfo struct { - CertFile string `json:"CertFile"` - KeyFile string `json:"KeyFile"` - CAFile string `json:"CAFile"` -} - type Info struct { Name string `json:"name"` @@ -117,12 +111,6 @@ type Info struct { EtcdTLS TLSInfo `json:"etcdTLS"` } -type TLSConfig struct { - Scheme string - Server tls.Config - Client tls.Config -} - //------------------------------------------------------------------------------ // // Variables @@ -199,6 +187,7 @@ func main() { // Create etcd and raft server r := newRaftServer(info.Name, info.RaftURL, info.RaftListenHost, &raftTLSConfig, &info.RaftTLS) + r.MaxClusterSize = maxClusterSize snapConf = r.newSnapshotConf() s := server.New(info.Name, info.EtcdURL, info.EtcdListenHost, &etcdTLSConfig, &info.EtcdTLS, r) @@ -209,4 +198,3 @@ func main() { r.ListenAndServe() s.ListenAndServe() } - diff --git a/log/log.go b/log/log.go new file mode 100644 index 00000000000..7f827b0ce08 --- /dev/null +++ b/log/log.go @@ -0,0 +1,44 @@ +package log + +import ( + golog "github.com/coreos/go-log/log" + "os" +) + +// The Verbose flag turns on verbose logging. +var Verbose bool = false + +var logger *golog.Logger = golog.New("etcd", false, + golog.CombinedSink(os.Stdout, "[%s] %s %-9s | %s\n", []string{"prefix", "time", "priority", "message"})) + +func Infof(format string, v ...interface{}) { + logger.Infof(format, v...) +} + +func Debugf(format string, v ...interface{}) { + if Verbose { + logger.Debugf(format, v...) + } +} + +func Debug(v ...interface{}) { + if Verbose { + logger.Debug(v...) + } +} + +func Warnf(format string, v ...interface{}) { + logger.Warningf(format, v...) +} + +func Warn(v ...interface{}) { + logger.Warning(v...) +} + +func Fatalf(format string, v ...interface{}) { + logger.Fatalf(format, v...) +} + +func Fatal(v ...interface{}) { + logger.Fatalln(v...) +} diff --git a/machines.go b/machines.go index b8b4a09d599..1da25ed757d 100644 --- a/machines.go +++ b/machines.go @@ -1,16 +1,5 @@ package main -// machineNum returns the number of machines in the cluster -func machineNum() int { - e, err := etcdStore.Get("/_etcd/machines", false, false, 0, 0) - - if err != nil { - return 0 - } - - return len(e.KVPairs) -} - // getMachines gets the current machines in the cluster func (r *raftServer) getMachines(toURL func(string) (string, bool)) []string { peers := r.Peers() diff --git a/raft_server.go b/raft_server.go index 146a2e84e4e..00cf4cbdbf3 100644 --- a/raft_server.go +++ b/raft_server.go @@ -11,10 +11,15 @@ import ( "net/url" "time" + "github.com/coreos/etcd/command" etcdErr "github.com/coreos/etcd/error" "github.com/coreos/go-raft" ) +func init() { + command.Register() +} + type raftServer struct { *raft.Server version string @@ -26,10 +31,9 @@ type raftServer struct { tlsInfo *TLSInfo followersStats *raftFollowersStats serverStats *raftServerStats + MaxClusterSize int } -//var r *raftServer - func newRaftServer(name string, url string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo) *raftServer { raftWrapper := &raftServer{ @@ -68,9 +72,6 @@ func newRaftServer(name string, url string, listenHost string, tlsConf *TLSConfi // Start the raft server func (r *raftServer) ListenAndServe() { - // Setup commands. - registerCommands() - // LoadSnapshot if snapshot { err := r.LoadSnapshot() @@ -314,16 +315,3 @@ func (r *raftServer) PeerStats() []byte { } return nil } - -// Register commands to raft server -func registerCommands() { - raft.RegisterCommand(&JoinCommand{}) - raft.RegisterCommand(&RemoveCommand{}) - raft.RegisterCommand(&GetCommand{}) - raft.RegisterCommand(&DeleteCommand{}) - raft.RegisterCommand(&WatchCommand{}) - raft.RegisterCommand(&TestAndSetCommand{}) - - raft.RegisterCommand(&CreateCommand{}) - raft.RegisterCommand(&UpdateCommand{}) -} diff --git a/raft_stats.go b/raft_stats.go deleted file mode 100644 index 45d21037f4a..00000000000 --- a/raft_stats.go +++ /dev/null @@ -1,210 +0,0 @@ -package main - -import ( - "math" - "sync" - "time" - - "github.com/coreos/go-raft" -) - -const ( - queueCapacity = 200 -) - -// packageStats represent the stats we need for a package. -// It has sending time and the size of the package. -type packageStats struct { - sendingTime time.Time - size int -} - -// NewPackageStats creates a pacakgeStats and return the pointer to it. -func NewPackageStats(now time.Time, size int) *packageStats { - return &packageStats{ - sendingTime: now, - size: size, - } -} - -// Time return the sending time of the package. -func (ps *packageStats) Time() time.Time { - return ps.sendingTime -} - -type raftServerStats struct { - Name string `json:"name"` - State string `json:"state"` - StartTime time.Time `json:"startTime"` - - LeaderInfo struct { - Name string `json:"leader"` - Uptime string `json:"uptime"` - startTime time.Time - } `json:"leaderInfo"` - - RecvAppendRequestCnt uint64 `json:"recvAppendRequestCnt,"` - RecvingPkgRate float64 `json:"recvPkgRate,omitempty"` - RecvingBandwidthRate float64 `json:"recvBandwidthRate,omitempty"` - - SendAppendRequestCnt uint64 `json:"sendAppendRequestCnt"` - SendingPkgRate float64 `json:"sendPkgRate,omitempty"` - SendingBandwidthRate float64 `json:"sendBandwidthRate,omitempty"` - - sendRateQueue *statsQueue - recvRateQueue *statsQueue -} - -func (ss *raftServerStats) RecvAppendReq(leaderName string, pkgSize int) { - ss.State = raft.Follower - if leaderName != ss.LeaderInfo.Name { - ss.LeaderInfo.Name = leaderName - ss.LeaderInfo.startTime = time.Now() - } - - ss.recvRateQueue.Insert(NewPackageStats(time.Now(), pkgSize)) - ss.RecvAppendRequestCnt++ -} - -func (ss *raftServerStats) SendAppendReq(pkgSize int) { - now := time.Now() - - if ss.State != raft.Leader { - ss.State = raft.Leader - ss.LeaderInfo.Name = ss.Name - ss.LeaderInfo.startTime = now - } - - ss.sendRateQueue.Insert(NewPackageStats(now, pkgSize)) - - ss.SendAppendRequestCnt++ -} - -type raftFollowersStats struct { - Leader string `json:"leader"` - Followers map[string]*raftFollowerStats `json:"followers"` -} - -type raftFollowerStats struct { - Latency struct { - Current float64 `json:"current"` - Average float64 `json:"average"` - averageSquare float64 - StandardDeviation float64 `json:"standardDeviation"` - Minimum float64 `json:"minimum"` - Maximum float64 `json:"maximum"` - } `json:"latency"` - - Counts struct { - Fail uint64 `json:"fail"` - Success uint64 `json:"success"` - } `json:"counts"` -} - -// Succ function update the raftFollowerStats with a successful send -func (ps *raftFollowerStats) Succ(d time.Duration) { - total := float64(ps.Counts.Success) * ps.Latency.Average - totalSquare := float64(ps.Counts.Success) * ps.Latency.averageSquare - - ps.Counts.Success++ - - ps.Latency.Current = float64(d) / (1000000.0) - - if ps.Latency.Current > ps.Latency.Maximum { - ps.Latency.Maximum = ps.Latency.Current - } - - if ps.Latency.Current < ps.Latency.Minimum { - ps.Latency.Minimum = ps.Latency.Current - } - - ps.Latency.Average = (total + ps.Latency.Current) / float64(ps.Counts.Success) - ps.Latency.averageSquare = (totalSquare + ps.Latency.Current*ps.Latency.Current) / float64(ps.Counts.Success) - - // sdv = sqrt(avg(x^2) - avg(x)^2) - ps.Latency.StandardDeviation = math.Sqrt(ps.Latency.averageSquare - ps.Latency.Average*ps.Latency.Average) -} - -// Fail function update the raftFollowerStats with a unsuccessful send -func (ps *raftFollowerStats) Fail() { - ps.Counts.Fail++ -} - -type statsQueue struct { - items [queueCapacity]*packageStats - size int - front int - back int - totalPkgSize int - rwl sync.RWMutex -} - -func (q *statsQueue) Len() int { - return q.size -} - -func (q *statsQueue) PkgSize() int { - return q.totalPkgSize -} - -// FrontAndBack gets the front and back elements in the queue -// We must grab front and back together with the protection of the lock -func (q *statsQueue) frontAndBack() (*packageStats, *packageStats) { - q.rwl.RLock() - defer q.rwl.RUnlock() - if q.size != 0 { - return q.items[q.front], q.items[q.back] - } - return nil, nil -} - -// Insert function insert a packageStats into the queue and update the records -func (q *statsQueue) Insert(p *packageStats) { - q.rwl.Lock() - defer q.rwl.Unlock() - - q.back = (q.back + 1) % queueCapacity - - if q.size == queueCapacity { //dequeue - q.totalPkgSize -= q.items[q.front].size - q.front = (q.back + 1) % queueCapacity - } else { - q.size++ - } - - q.items[q.back] = p - q.totalPkgSize += q.items[q.back].size - -} - -// Rate function returns the package rate and byte rate -func (q *statsQueue) Rate() (float64, float64) { - front, back := q.frontAndBack() - - if front == nil || back == nil { - return 0, 0 - } - - if time.Now().Sub(back.Time()) > time.Second { - q.Clear() - return 0, 0 - } - - sampleDuration := back.Time().Sub(front.Time()) - - pr := float64(q.Len()) / float64(sampleDuration) * float64(time.Second) - - br := float64(q.PkgSize()) / float64(sampleDuration) * float64(time.Second) - - return pr, br -} - -// Clear function clear up the statsQueue -func (q *statsQueue) Clear() { - q.rwl.Lock() - defer q.rwl.Unlock() - q.back = -1 - q.front = 0 - q.size = 0 - q.totalPkgSize = 0 -} diff --git a/server/join_command.go b/server/join_command.go new file mode 100644 index 00000000000..89a3a4e547e --- /dev/null +++ b/server/join_command.go @@ -0,0 +1,84 @@ +package server + +import ( + "encoding/binary" + "fmt" + "path" + + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" +) + +func init() { + raft.RegisterCommand(&JoinCommand{}) +} + +// The JoinCommand adds a node to the cluster. +type JoinCommand struct { + RaftVersion string `json:"raftVersion"` + Name string `json:"name"` + RaftURL string `json:"raftURL"` + EtcdURL string `json:"etcdURL"` + MaxClusterSize int `json:"maxClusterSize"` +} + +func NewJoinCommand(version, name, raftUrl, etcdUrl string, maxClusterSize int) *JoinCommand { + return &JoinCommand{ + RaftVersion: version, + Name: name, + RaftURL: raftUrl, + EtcdURL: etcdUrl, + MaxClusterSize: maxClusterSize, + } +} + +// The name of the join command in the log +func (c *JoinCommand) CommandName() string { + return "etcd:join" +} + +// Join a server to the cluster +func (c *JoinCommand) Apply(server *raft.Server) (interface{}, error) { + s, _ := server.StateMachine().(*store.Store) + r, _ := server.Context().(*RaftServer) + + // check if the join command is from a previous machine, who lost all its previous log. + e, _ := s.Get(path.Join("/_etcd/machines", c.Name), false, false, server.CommitIndex(), server.Term()) + + b := make([]byte, 8) + binary.PutUvarint(b, server.CommitIndex()) + + if e != nil { + return b, nil + } + + // check machine number in the cluster + if s.MachineCount() == c.MaxClusterSize { + log.Debug("Reject join request from ", c.Name) + return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMoreMachine, "", server.CommitIndex(), server.Term()) + } + + addNameToURL(c.Name, c.RaftVersion, c.RaftURL, c.EtcdURL) + + // add peer in raft + err := server.AddPeer(c.Name, "") + + // add machine in etcd storage + key := path.Join("_etcd/machines", c.Name) + value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", c.RaftURL, c.EtcdURL, c.RaftVersion) + s.Create(key, value, false, false, store.Permanent, server.CommitIndex(), server.Term()) + + // add peer stats + if c.Name != r.Name() { + r.followersStats.Followers[c.Name] = &raftFollowerStats{} + r.followersStats.Followers[c.Name].Latency.Minimum = 1 << 63 + } + + return b, err +} + +func (c *JoinCommand) NodeName() string { + return c.Name +} diff --git a/server/package_stats.go b/server/package_stats.go new file mode 100644 index 00000000000..519168033e0 --- /dev/null +++ b/server/package_stats.go @@ -0,0 +1,25 @@ +package server + +import ( + "time" +) + +// packageStats represent the stats we need for a package. +// It has sending time and the size of the package. +type packageStats struct { + sendingTime time.Time + size int +} + +// NewPackageStats creates a pacakgeStats and return the pointer to it. +func NewPackageStats(now time.Time, size int) *packageStats { + return &packageStats{ + sendingTime: now, + size: size, + } +} + +// Time return the sending time of the package. +func (ps *packageStats) Time() time.Time { + return ps.sendingTime +} diff --git a/server/raft_follower_stats.go b/server/raft_follower_stats.go new file mode 100644 index 00000000000..96b76c85b13 --- /dev/null +++ b/server/raft_follower_stats.go @@ -0,0 +1,56 @@ +package server + +import ( + "math" + "time" +) + +type raftFollowersStats struct { + Leader string `json:"leader"` + Followers map[string]*raftFollowerStats `json:"followers"` +} + +type raftFollowerStats struct { + Latency struct { + Current float64 `json:"current"` + Average float64 `json:"average"` + averageSquare float64 + StandardDeviation float64 `json:"standardDeviation"` + Minimum float64 `json:"minimum"` + Maximum float64 `json:"maximum"` + } `json:"latency"` + + Counts struct { + Fail uint64 `json:"fail"` + Success uint64 `json:"success"` + } `json:"counts"` +} + +// Succ function update the raftFollowerStats with a successful send +func (ps *raftFollowerStats) Succ(d time.Duration) { + total := float64(ps.Counts.Success) * ps.Latency.Average + totalSquare := float64(ps.Counts.Success) * ps.Latency.averageSquare + + ps.Counts.Success++ + + ps.Latency.Current = float64(d) / (1000000.0) + + if ps.Latency.Current > ps.Latency.Maximum { + ps.Latency.Maximum = ps.Latency.Current + } + + if ps.Latency.Current < ps.Latency.Minimum { + ps.Latency.Minimum = ps.Latency.Current + } + + ps.Latency.Average = (total + ps.Latency.Current) / float64(ps.Counts.Success) + ps.Latency.averageSquare = (totalSquare + ps.Latency.Current*ps.Latency.Current) / float64(ps.Counts.Success) + + // sdv = sqrt(avg(x^2) - avg(x)^2) + ps.Latency.StandardDeviation = math.Sqrt(ps.Latency.averageSquare - ps.Latency.Average*ps.Latency.Average) +} + +// Fail function update the raftFollowerStats with a unsuccessful send +func (ps *raftFollowerStats) Fail() { + ps.Counts.Fail++ +} diff --git a/server/raft_server_stats.go b/server/raft_server_stats.go new file mode 100644 index 00000000000..451578ab78e --- /dev/null +++ b/server/raft_server_stats.go @@ -0,0 +1,55 @@ +package server + +import ( + "time" + + "github.com/coreos/go-raft" +) + +type raftServerStats struct { + Name string `json:"name"` + State string `json:"state"` + StartTime time.Time `json:"startTime"` + + LeaderInfo struct { + Name string `json:"leader"` + Uptime string `json:"uptime"` + startTime time.Time + } `json:"leaderInfo"` + + RecvAppendRequestCnt uint64 `json:"recvAppendRequestCnt,"` + RecvingPkgRate float64 `json:"recvPkgRate,omitempty"` + RecvingBandwidthRate float64 `json:"recvBandwidthRate,omitempty"` + + SendAppendRequestCnt uint64 `json:"sendAppendRequestCnt"` + SendingPkgRate float64 `json:"sendPkgRate,omitempty"` + SendingBandwidthRate float64 `json:"sendBandwidthRate,omitempty"` + + sendRateQueue *statsQueue + recvRateQueue *statsQueue +} + +func (ss *raftServerStats) RecvAppendReq(leaderName string, pkgSize int) { + ss.State = raft.Follower + if leaderName != ss.LeaderInfo.Name { + ss.LeaderInfo.Name = leaderName + ss.LeaderInfo.startTime = time.Now() + } + + ss.recvRateQueue.Insert(NewPackageStats(time.Now(), pkgSize)) + ss.RecvAppendRequestCnt++ +} + +func (ss *raftServerStats) SendAppendReq(pkgSize int) { + now := time.Now() + + if ss.State != raft.Leader { + ss.State = raft.Leader + ss.LeaderInfo.Name = ss.Name + ss.LeaderInfo.startTime = now + } + + ss.sendRateQueue.Insert(NewPackageStats(now, pkgSize)) + + ss.SendAppendRequestCnt++ +} diff --git a/server/remove_command.go b/server/remove_command.go new file mode 100644 index 00000000000..a992de67c7e --- /dev/null +++ b/server/remove_command.go @@ -0,0 +1,68 @@ +package server + +import ( + "encoding/binary" + "path" + + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" +) + +func init() { + raft.RegisterCommand(&RemoveCommand{}) +} + +// The RemoveCommand removes a server from the cluster. +type RemoveCommand struct { + Name string `json:"name"` +} + +// The name of the remove command in the log +func (c *RemoveCommand) CommandName() string { + return "etcd:remove" +} + +// Remove a server from the cluster +func (c *RemoveCommand) Apply(server *raft.Server) (interface{}, error) { + s, _ := server.StateMachine().(*store.Store) + r, _ := server.Context().(*RaftServer) + + // remove machine in etcd storage + key := path.Join("_etcd/machines", c.Name) + + _, err := s.Delete(key, false, server.CommitIndex(), server.Term()) + // delete from stats + delete(r.followersStats.Followers, c.Name) + + if err != nil { + return []byte{0}, err + } + + // remove peer in raft + err = server.RemovePeer(c.Name) + + if err != nil { + return []byte{0}, err + } + + if c.Name == server.Name() { + // the removed node is this node + + // if the node is not replaying the previous logs + // and the node has sent out a join request in this + // start. It is sure that this node received a new remove + // command and need to be removed + if server.CommitIndex() > r.joinIndex && r.joinIndex != 0 { + debugf("server [%s] is removed", server.Name()) + os.Exit(0) + } else { + // else ignore remove + debugf("ignore previous remove command.") + } + } + + b := make([]byte, 8) + binary.PutUvarint(b, server.CommitIndex()) + + return b, err +} diff --git a/server/server.go b/server/server.go index 4bf1083018a..fad643aec40 100644 --- a/server/server.go +++ b/server/server.go @@ -1,16 +1,19 @@ package server import ( - "github.com/gorilla/mux" "net/http" "net/url" + + "github.com/coreos/etcd/command" + "github.com/coreos/go-raft" + "github.com/gorilla/mux" ) // The Server provides an HTTP interface to the underlying store. type Server interface { - CommitIndex() uint64 - Term() uint64 - Dispatch(Command, http.ResponseWriter, *http.Request) + CommitIndex() uint64 + Term() uint64 + Dispatch(command.Command, http.ResponseWriter, *http.Request) } // This is the default implementation of the Server interface. @@ -55,11 +58,6 @@ func (s *server) Term() uint64 { return c.raftServer.Term() } -// Executes a command against the Raft server. -func (s *server) Do(c Command, localOnly bool) (interface{}, error) { - return c.raftServer.Do(s.RaftServer().Server) -} - func (s *server) installV1() { s.handleFunc("/v1/keys/{key:.*}", v1.GetKeyHandler).Methods("GET") s.handleFunc("/v1/keys/{key:.*}", v1.SetKeyHandler).Methods("POST", "PUT") diff --git a/server/stats_queue.go b/server/stats_queue.go new file mode 100644 index 00000000000..5927eed15ec --- /dev/null +++ b/server/stats_queue.go @@ -0,0 +1,88 @@ +package server + +import ( + "sync" +) + +const ( + queueCapacity = 200 +) + +type statsQueue struct { + items [queueCapacity]*packageStats + size int + front int + back int + totalPkgSize int + rwl sync.RWMutex +} + +func (q *statsQueue) Len() int { + return q.size +} + +func (q *statsQueue) PkgSize() int { + return q.totalPkgSize +} + +// FrontAndBack gets the front and back elements in the queue +// We must grab front and back together with the protection of the lock +func (q *statsQueue) frontAndBack() (*packageStats, *packageStats) { + q.rwl.RLock() + defer q.rwl.RUnlock() + if q.size != 0 { + return q.items[q.front], q.items[q.back] + } + return nil, nil +} + +// Insert function insert a packageStats into the queue and update the records +func (q *statsQueue) Insert(p *packageStats) { + q.rwl.Lock() + defer q.rwl.Unlock() + + q.back = (q.back + 1) % queueCapacity + + if q.size == queueCapacity { //dequeue + q.totalPkgSize -= q.items[q.front].size + q.front = (q.back + 1) % queueCapacity + } else { + q.size++ + } + + q.items[q.back] = p + q.totalPkgSize += q.items[q.back].size + +} + +// Rate function returns the package rate and byte rate +func (q *statsQueue) Rate() (float64, float64) { + front, back := q.frontAndBack() + + if front == nil || back == nil { + return 0, 0 + } + + if time.Now().Sub(back.Time()) > time.Second { + q.Clear() + return 0, 0 + } + + sampleDuration := back.Time().Sub(front.Time()) + + pr := float64(q.Len()) / float64(sampleDuration) * float64(time.Second) + + br := float64(q.PkgSize()) / float64(sampleDuration) * float64(time.Second) + + return pr, br +} + +// Clear function clear up the statsQueue +func (q *statsQueue) Clear() { + q.rwl.Lock() + defer q.rwl.Unlock() + q.back = -1 + q.front = 0 + q.size = 0 + q.totalPkgSize = 0 +} diff --git a/server/tls_config.go b/server/tls_config.go new file mode 100644 index 00000000000..4b944626d16 --- /dev/null +++ b/server/tls_config.go @@ -0,0 +1,11 @@ +package server + +import ( + "crypto/tls" +) + +type TLSConfig struct { + Scheme string + Server tls.Config + Client tls.Config +} diff --git a/server/tls_info.go b/server/tls_info.go new file mode 100644 index 00000000000..91936b0906e --- /dev/null +++ b/server/tls_info.go @@ -0,0 +1,7 @@ +package server + +type TLSInfo struct { + CertFile string `json:"CertFile"` + KeyFile string `json:"KeyFile"` + CAFile string `json:"CAFile"` +} diff --git a/server/v1/delete_key_handler.go b/server/v1/delete_key_handler.go index 8b9e315bdd1..c9d695fdc3f 100644 --- a/server/v1/delete_key_handler.go +++ b/server/v1/delete_key_handler.go @@ -1,15 +1,15 @@ package v1 import ( - "encoding/json" - "github.com/coreos/etcd/store" - "net/http" + "encoding/json" + "github.com/coreos/etcd/store" + "net/http" ) // Removes a key from the store. func DeleteKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { - vars := mux.Vars(req) - key := "/" + vars["key"] + vars := mux.Vars(req) + key := "/" + vars["key"] command := &DeleteCommand{Key: key} return s.Dispatch(command, w, req) } diff --git a/server/v1/v1.go b/server/v1/v1.go index 709463b0793..7f9f80a0e54 100644 --- a/server/v1/v1.go +++ b/server/v1/v1.go @@ -7,44 +7,44 @@ import ( // The Server interface provides all the methods required for the v1 API. type Server interface { - CommitIndex() uint64 - Term() uint64 - Dispatch(http.ResponseWriter, *http.Request, Command) + CommitIndex() uint64 + Term() uint64 + Dispatch(http.ResponseWriter, *http.Request, Command) } // Converts an event object into a response object. func eventToResponse(event *store.Event) interface{} { - if !event.Dir { - response := &store.Response{ - Action: event.Action, - Key: event.Key, - Value: event.Value, - PrevValue: event.PrevValue, - Index: event.Index, - TTL: event.TTL, - Expiration: event.Expiration, - } + if !event.Dir { + response := &store.Response{ + Action: event.Action, + Key: event.Key, + Value: event.Value, + PrevValue: event.PrevValue, + Index: event.Index, + TTL: event.TTL, + Expiration: event.Expiration, + } - if response.Action == store.Create || response.Action == store.Update { - response.Action = "set" - if response.PrevValue == "" { - response.NewKey = true - } - } + if response.Action == store.Create || response.Action == store.Update { + response.Action = "set" + if response.PrevValue == "" { + response.NewKey = true + } + } - return response - } else { - responses := make([]*store.Response, len(event.KVPairs)) + return response + } else { + responses := make([]*store.Response, len(event.KVPairs)) - for i, kv := range event.KVPairs { - responses[i] = &store.Response{ - Action: event.Action, - Key: kv.Key, - Value: kv.Value, - Dir: kv.Dir, - Index: event.Index, - } - } - return responses - } + for i, kv := range event.KVPairs { + responses[i] = &store.Response{ + Action: event.Action, + Key: kv.Key, + Value: kv.Value, + Dir: kv.Dir, + Index: event.Index, + } + } + return responses + } } diff --git a/server/v2/handlers.go b/server/v2/handlers.go index 7f102953109..0d294de7d4a 100644 --- a/server/v2/handlers.go +++ b/server/v2/handlers.go @@ -1,15 +1,15 @@ package main import ( - "encoding/json" - "fmt" - "net/http" - "strconv" - "strings" - - etcdErr "github.com/coreos/etcd/error" - "github.com/coreos/etcd/store" - "github.com/coreos/go-raft" + "encoding/json" + "fmt" + "net/http" + "strconv" + "strings" + + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" ) //------------------------------------------------------------------- @@ -17,22 +17,22 @@ import ( //------------------------------------------------------------------- func NewEtcdMuxer() *http.ServeMux { - // external commands - router := mux.NewRouter() - etcdMux.Handle("/v2/keys/", errorHandler(e.Multiplexer)) - etcdMux.Handle("/v2/leader", errorHandler(e.LeaderHttpHandler)) - etcdMux.Handle("/v2/machines", errorHandler(e.MachinesHttpHandler)) - etcdMux.Handle("/v2/stats/", errorHandler(e.StatsHttpHandler)) - etcdMux.Handle("/version", errorHandler(e.VersionHttpHandler)) - etcdMux.HandleFunc("/test/", TestHttpHandler) - - // backward support - etcdMux.Handle("/v1/keys/", errorHandler(e.MultiplexerV1)) - etcdMux.Handle("/v1/leader", errorHandler(e.LeaderHttpHandler)) - etcdMux.Handle("/v1/machines", errorHandler(e.MachinesHttpHandler)) - etcdMux.Handle("/v1/stats/", errorHandler(e.StatsHttpHandler)) - - return etcdMux + // external commands + router := mux.NewRouter() + etcdMux.Handle("/v2/keys/", errorHandler(e.Multiplexer)) + etcdMux.Handle("/v2/leader", errorHandler(e.LeaderHttpHandler)) + etcdMux.Handle("/v2/machines", errorHandler(e.MachinesHttpHandler)) + etcdMux.Handle("/v2/stats/", errorHandler(e.StatsHttpHandler)) + etcdMux.Handle("/version", errorHandler(e.VersionHttpHandler)) + etcdMux.HandleFunc("/test/", TestHttpHandler) + + // backward support + etcdMux.Handle("/v1/keys/", errorHandler(e.MultiplexerV1)) + etcdMux.Handle("/v1/leader", errorHandler(e.LeaderHttpHandler)) + etcdMux.Handle("/v1/machines", errorHandler(e.MachinesHttpHandler)) + etcdMux.Handle("/v1/stats/", errorHandler(e.StatsHttpHandler)) + + return etcdMux } type errorHandler func(http.ResponseWriter, *http.Request) error @@ -41,50 +41,50 @@ type errorHandler func(http.ResponseWriter, *http.Request) error // provided allowed origins and sets the Access-Control-Allow-Origin header if // there is a match. func addCorsHeader(w http.ResponseWriter, r *http.Request) { - val, ok := corsList["*"] - if val && ok { - w.Header().Add("Access-Control-Allow-Origin", "*") - return - } - - requestOrigin := r.Header.Get("Origin") - val, ok = corsList[requestOrigin] - if val && ok { - w.Header().Add("Access-Control-Allow-Origin", requestOrigin) - return - } + val, ok := corsList["*"] + if val && ok { + w.Header().Add("Access-Control-Allow-Origin", "*") + return + } + + requestOrigin := r.Header.Get("Origin") + val, ok = corsList[requestOrigin] + if val && ok { + w.Header().Add("Access-Control-Allow-Origin", requestOrigin) + return + } } func (fn errorHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { - addCorsHeader(w, r) - if e := fn(w, r); e != nil { - if etcdErr, ok := e.(*etcdErr.Error); ok { - debug("Return error: ", (*etcdErr).Error()) - etcdErr.Write(w) - } else { - http.Error(w, e.Error(), http.StatusInternalServerError) - } - } + addCorsHeader(w, r) + if e := fn(w, r); e != nil { + if etcdErr, ok := e.(*etcdErr.Error); ok { + debug("Return error: ", (*etcdErr).Error()) + etcdErr.Write(w) + } else { + http.Error(w, e.Error(), http.StatusInternalServerError) + } + } } // Multiplex GET/POST/DELETE request to corresponding handlers func (e *etcdServer) Multiplexer(w http.ResponseWriter, req *http.Request) error { - switch req.Method { - case "GET": - return e.GetHttpHandler(w, req) - case "POST": - return e.CreateHttpHandler(w, req) - case "PUT": - return e.UpdateHttpHandler(w, req) - case "DELETE": - return e.DeleteHttpHandler(w, req) - default: - w.WriteHeader(http.StatusMethodNotAllowed) - return nil - } - - return nil + switch req.Method { + case "GET": + return e.GetHttpHandler(w, req) + case "POST": + return e.CreateHttpHandler(w, req) + case "PUT": + return e.UpdateHttpHandler(w, req) + case "DELETE": + return e.DeleteHttpHandler(w, req) + default: + w.WriteHeader(http.StatusMethodNotAllowed) + return nil + } + + return nil } //-------------------------------------- @@ -93,111 +93,111 @@ func (e *etcdServer) Multiplexer(w http.ResponseWriter, req *http.Request) error //-------------------------------------- func (e *etcdServer) CreateHttpHandler(w http.ResponseWriter, req *http.Request) error { - key := getNodePath(req.URL.Path) + key := getNodePath(req.URL.Path) - debugf("recv.post[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) + debugf("recv.post[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) - value := req.FormValue("value") + value := req.FormValue("value") - expireTime, err := durationToExpireTime(req.FormValue("ttl")) + expireTime, err := durationToExpireTime(req.FormValue("ttl")) - if err != nil { - return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create", store.UndefIndex, store.UndefTerm) - } + if err != nil { + return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create", store.UndefIndex, store.UndefTerm) + } - command := &CreateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - } + command := &CreateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + } - if req.FormValue("incremental") == "true" { - command.IncrementalSuffix = true - } + if req.FormValue("incremental") == "true" { + command.IncrementalSuffix = true + } - return e.dispatchEtcdCommand(command, w, req) + return e.dispatchEtcdCommand(command, w, req) } func (e *etcdServer) UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { - key := getNodePath(req.URL.Path) + key := getNodePath(req.URL.Path) - debugf("recv.put[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) + debugf("recv.put[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) - req.ParseForm() + req.ParseForm() - value := req.Form.Get("value") + value := req.Form.Get("value") - expireTime, err := durationToExpireTime(req.Form.Get("ttl")) + expireTime, err := durationToExpireTime(req.Form.Get("ttl")) - if err != nil { - return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", store.UndefIndex, store.UndefTerm) - } + if err != nil { + return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", store.UndefIndex, store.UndefTerm) + } - // update should give at least one option - if value == "" && expireTime.Sub(store.Permanent) == 0 { - return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", store.UndefIndex, store.UndefTerm) - } + // update should give at least one option + if value == "" && expireTime.Sub(store.Permanent) == 0 { + return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", store.UndefIndex, store.UndefTerm) + } - prevValue, valueOk := req.Form["prevValue"] + prevValue, valueOk := req.Form["prevValue"] - prevIndexStr, indexOk := req.Form["prevIndex"] + prevIndexStr, indexOk := req.Form["prevIndex"] - if !valueOk && !indexOk { // update without test - command := &UpdateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - } + if !valueOk && !indexOk { // update without test + command := &UpdateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + } - return e.dispatchEtcdCommand(command, w, req) + return e.dispatchEtcdCommand(command, w, req) - } else { // update with test - var prevIndex uint64 + } else { // update with test + var prevIndex uint64 - if indexOk { - prevIndex, err = strconv.ParseUint(prevIndexStr[0], 10, 64) + if indexOk { + prevIndex, err = strconv.ParseUint(prevIndexStr[0], 10, 64) - // bad previous index - if err != nil { - return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Update", store.UndefIndex, store.UndefTerm) - } - } else { - prevIndex = 0 - } + // bad previous index + if err != nil { + return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Update", store.UndefIndex, store.UndefTerm) + } + } else { + prevIndex = 0 + } - command := &TestAndSetCommand{ - Key: key, - Value: value, - PrevValue: prevValue[0], - PrevIndex: prevIndex, - } + command := &TestAndSetCommand{ + Key: key, + Value: value, + PrevValue: prevValue[0], + PrevIndex: prevIndex, + } - return e.dispatchEtcdCommand(command, w, req) - } + return e.dispatchEtcdCommand(command, w, req) + } } // Delete Handler func (e *etcdServer) DeleteHttpHandler(w http.ResponseWriter, req *http.Request) error { - key := getNodePath(req.URL.Path) + key := getNodePath(req.URL.Path) - debugf("recv.delete[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) + debugf("recv.delete[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) - command := &DeleteCommand{ - Key: key, - } + command := &DeleteCommand{ + Key: key, + } - if req.FormValue("recursive") == "true" { - command.Recursive = true - } + if req.FormValue("recursive") == "true" { + command.Recursive = true + } - return e.dispatchEtcdCommand(command, w, req) + return e.dispatchEtcdCommand(command, w, req) } // Dispatch the command to leader func (e *etcdServer) dispatchEtcdCommand(c Command, w http.ResponseWriter, req *http.Request) error { - return e.raftServer.dispatch(c, w, req, nameToEtcdURL) + return e.raftServer.dispatch(c, w, req, nameToEtcdURL) } //-------------------------------------- @@ -208,157 +208,157 @@ func (e *etcdServer) dispatchEtcdCommand(c Command, w http.ResponseWriter, req * // Handler to return the current leader's raft address func (e *etcdServer) LeaderHttpHandler(w http.ResponseWriter, req *http.Request) error { - r := e.raftServer + r := e.raftServer - leader := r.Leader() + leader := r.Leader() - if leader != "" { - w.WriteHeader(http.StatusOK) - raftURL, _ := nameToRaftURL(leader) - w.Write([]byte(raftURL)) + if leader != "" { + w.WriteHeader(http.StatusOK) + raftURL, _ := nameToRaftURL(leader) + w.Write([]byte(raftURL)) - return nil - } else { - return etcdErr.NewError(etcdErr.EcodeLeaderElect, "", store.UndefIndex, store.UndefTerm) - } + return nil + } else { + return etcdErr.NewError(etcdErr.EcodeLeaderElect, "", store.UndefIndex, store.UndefTerm) + } } // Handler to return all the known machines in the current cluster func (e *etcdServer) MachinesHttpHandler(w http.ResponseWriter, req *http.Request) error { - machines := e.raftServer.getMachines(nameToEtcdURL) + machines := e.raftServer.getMachines(nameToEtcdURL) - w.WriteHeader(http.StatusOK) - w.Write([]byte(strings.Join(machines, ", "))) + w.WriteHeader(http.StatusOK) + w.Write([]byte(strings.Join(machines, ", "))) - return nil + return nil } // Handler to return the current version of etcd func (e *etcdServer) VersionHttpHandler(w http.ResponseWriter, req *http.Request) error { - w.WriteHeader(http.StatusOK) - fmt.Fprintf(w, "etcd %s", releaseVersion) + w.WriteHeader(http.StatusOK) + fmt.Fprintf(w, "etcd %s", releaseVersion) - return nil + return nil } // Handler to return the basic stats of etcd func (e *etcdServer) StatsHttpHandler(w http.ResponseWriter, req *http.Request) error { - option := req.URL.Path[len("/v1/stats/"):] - w.WriteHeader(http.StatusOK) - - r := e.raftServer - - switch option { - case "self": - w.Write(r.Stats()) - case "leader": - if r.State() == raft.Leader { - w.Write(r.PeerStats()) - } else { - leader := r.Leader() - // current no leader - if leader == "" { - return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) - } - hostname, _ := nameToEtcdURL(leader) - redirect(hostname, w, req) - } - case "store": - w.Write(etcdStore.JsonStats()) - } - - return nil + option := req.URL.Path[len("/v1/stats/"):] + w.WriteHeader(http.StatusOK) + + r := e.raftServer + + switch option { + case "self": + w.Write(r.Stats()) + case "leader": + if r.State() == raft.Leader { + w.Write(r.PeerStats()) + } else { + leader := r.Leader() + // current no leader + if leader == "" { + return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) + } + hostname, _ := nameToEtcdURL(leader) + redirect(hostname, w, req) + } + case "store": + w.Write(etcdStore.JsonStats()) + } + + return nil } func (e *etcdServer) GetHttpHandler(w http.ResponseWriter, req *http.Request) error { - var err error - var event interface{} + var err error + var event interface{} - r := e.raftServer + r := e.raftServer - debugf("recv.get[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) + debugf("recv.get[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) - if req.FormValue("consistent") == "true" && r.State() != raft.Leader { - // help client to redirect the request to the current leader - leader := r.Leader() - hostname, _ := nameToEtcdURL(leader) - redirect(hostname, w, req) - return nil - } + if req.FormValue("consistent") == "true" && r.State() != raft.Leader { + // help client to redirect the request to the current leader + leader := r.Leader() + hostname, _ := nameToEtcdURL(leader) + redirect(hostname, w, req) + return nil + } - key := getNodePath(req.URL.Path) + key := getNodePath(req.URL.Path) - recursive := req.FormValue("recursive") + recursive := req.FormValue("recursive") - if req.FormValue("wait") == "true" { // watch - command := &WatchCommand{ - Key: key, - } + if req.FormValue("wait") == "true" { // watch + command := &WatchCommand{ + Key: key, + } - if recursive == "true" { - command.Recursive = true - } + if recursive == "true" { + command.Recursive = true + } - indexStr := req.FormValue("wait_index") - if indexStr != "" { - sinceIndex, err := strconv.ParseUint(indexStr, 10, 64) + indexStr := req.FormValue("wait_index") + if indexStr != "" { + sinceIndex, err := strconv.ParseUint(indexStr, 10, 64) - if err != nil { - return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index", store.UndefIndex, store.UndefTerm) - } + if err != nil { + return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index", store.UndefIndex, store.UndefTerm) + } - command.SinceIndex = sinceIndex - } + command.SinceIndex = sinceIndex + } - event, err = command.Apply(r.Server) + event, err = command.Apply(r.Server) - } else { //get + } else { //get - command := &GetCommand{ - Key: key, - } + command := &GetCommand{ + Key: key, + } - sorted := req.FormValue("sorted") - if sorted == "true" { - command.Sorted = true - } + sorted := req.FormValue("sorted") + if sorted == "true" { + command.Sorted = true + } - if recursive == "true" { - command.Recursive = true - } + if recursive == "true" { + command.Recursive = true + } - event, err = command.Apply(r.Server) - } + event, err = command.Apply(r.Server) + } - if err != nil { - return err + if err != nil { + return err - } else { - event, _ := event.(*store.Event) - bytes, _ := json.Marshal(event) + } else { + event, _ := event.(*store.Event) + bytes, _ := json.Marshal(event) - w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) - w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term)) - w.WriteHeader(http.StatusOK) + w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) + w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term)) + w.WriteHeader(http.StatusOK) - w.Write(bytes) + w.Write(bytes) - return nil - } + return nil + } } // TestHandler func TestHttpHandler(w http.ResponseWriter, req *http.Request) { - testType := req.URL.Path[len("/test/"):] + testType := req.URL.Path[len("/test/"):] - if testType == "speed" { - directSet() - w.WriteHeader(http.StatusOK) - w.Write([]byte("speed test success")) + if testType == "speed" { + directSet() + w.WriteHeader(http.StatusOK) + w.Write([]byte("speed test success")) - return - } + return + } - w.WriteHeader(http.StatusBadRequest) + w.WriteHeader(http.StatusBadRequest) } diff --git a/store/store.go b/store/store.go index 348aea02e4d..ddf0762805e 100644 --- a/store/store.go +++ b/store/store.go @@ -401,6 +401,16 @@ func (s *Store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) { return n, nil } +// Returns the number of machines in the cluster. +func (s *Store) MachineCount() int { + e, err := s.Get("/_etcd/machines", false, false, 0, 0) + if err != nil { + return 0 + } + + return len(e.KVPairs) +} + // Save function saves the static state of the store system. // Save function will not be able to save the state of watchers. // Save function will not save the parent field of the node. Or there will diff --git a/util.go b/util.go index 318f728d10d..fade01ec64e 100644 --- a/util.go +++ b/util.go @@ -15,7 +15,6 @@ import ( etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/store" - "github.com/coreos/go-log/log" "github.com/coreos/go-raft" ) @@ -172,45 +171,6 @@ func getNodePath(urlPath string) string { return urlPath[pathPrefixLen:] } -//-------------------------------------- -// Log -//-------------------------------------- - -var logger *log.Logger = log.New("etcd", false, - log.CombinedSink(os.Stdout, "[%s] %s %-9s | %s\n", []string{"prefix", "time", "priority", "message"})) - -func infof(format string, v ...interface{}) { - logger.Infof(format, v...) -} - -func debugf(format string, v ...interface{}) { - if verbose { - logger.Debugf(format, v...) - } -} - -func debug(v ...interface{}) { - if verbose { - logger.Debug(v...) - } -} - -func warnf(format string, v ...interface{}) { - logger.Warningf(format, v...) -} - -func warn(v ...interface{}) { - logger.Warning(v...) -} - -func fatalf(format string, v ...interface{}) { - logger.Fatalf(format, v...) -} - -func fatal(v ...interface{}) { - logger.Fatalln(v...) -} - //-------------------------------------- // CPU profile //-------------------------------------- From eb78d96a204704026497cc6aef4eb372d1946ff3 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sat, 12 Oct 2013 00:28:46 -0600 Subject: [PATCH 074/247] Intermediate commit. --- .gitignore | 2 +- build | 2 +- build.ps1 | 2 +- command/command.go | 19 - etcd.go | 18 +- name_url_map.go | 74 --- raft_handlers.go | 147 ------ raft_server.go | 317 ------------ server/join_command.go | 32 +- server/peer_server.go | 474 ++++++++++++++++++ server/registry.go | 167 ++++++ server/remove_command.go | 16 +- server/server.go | 3 +- server/timeout.go | 11 + server/transporter.go | 226 +++++++++ .../transporter_test.go | 2 +- server/util.go | 17 + server/v2/handlers.go | 4 - version.go => server/version.go | 6 +- snapshot.go | 36 -- {command => store}/create_command.go | 9 +- {command => store}/delete_command.go | 9 +- store/store.go | 10 - {command => store}/test_and_set_command.go | 9 +- {command => store}/update_command.go | 9 +- transporter.go | 233 --------- util.go | 22 - 27 files changed, 953 insertions(+), 923 deletions(-) delete mode 100644 command/command.go delete mode 100644 name_url_map.go delete mode 100644 raft_handlers.go delete mode 100644 raft_server.go create mode 100644 server/peer_server.go create mode 100644 server/registry.go create mode 100644 server/timeout.go create mode 100644 server/transporter.go rename transporter_test.go => server/transporter_test.go (98%) create mode 100644 server/util.go rename version.go => server/version.go (71%) delete mode 100644 snapshot.go rename {command => store}/create_command.go (86%) rename {command => store}/delete_command.go (83%) rename {command => store}/test_and_set_command.go (88%) rename {command => store}/update_command.go (85%) delete mode 100644 transporter.go diff --git a/.gitignore b/.gitignore index d00d899e2e2..4d8d3967c37 100644 --- a/.gitignore +++ b/.gitignore @@ -1,5 +1,5 @@ src/ pkg/ /etcd -release_version.go +/server/release_version.go /machine* diff --git a/build b/build index b121ba30dff..9c2f95f6ae5 100755 --- a/build +++ b/build @@ -21,5 +21,5 @@ for i in third_party/*; do cp -R "$i" src/ done -./scripts/release-version > release_version.go +./scripts/release-version > server/release_version.go go build "${ETCD_PACKAGE}" diff --git a/build.ps1 b/build.ps1 index c75f608d7e8..fb8a386ce22 100644 --- a/build.ps1 +++ b/build.ps1 @@ -20,5 +20,5 @@ foreach($i in (ls third_party/*)){ cp -Recurse -force "$i" src/ } -./scripts/release-version.ps1 | Out-File -Encoding UTF8 release_version.go +./scripts/release-version.ps1 | Out-File -Encoding UTF8 server/release_version.go go build -v "${ETCD_PACKAGE}" diff --git a/command/command.go b/command/command.go deleted file mode 100644 index 50513d5f500..00000000000 --- a/command/command.go +++ /dev/null @@ -1,19 +0,0 @@ -package command - -import ( - "github.com/coreos/go-raft" -) - -// A command represents an action to be taken on the replicated state machine. -type Command interface { - CommandName() string - Apply(server *raft.Server) (interface{}, error) -} - -// Registers commands to the Raft library. -func Register() { - raft.RegisterCommand(&DeleteCommand{}) - raft.RegisterCommand(&TestAndSetCommand{}) - raft.RegisterCommand(&CreateCommand{}) - raft.RegisterCommand(&UpdateCommand{}) -} diff --git a/etcd.go b/etcd.go index 79bda7b6779..82bc09649fb 100644 --- a/etcd.go +++ b/etcd.go @@ -85,12 +85,6 @@ func init() { flag.StringVar(&cors, "cors", "", "whitelist origins for cross-origin resource sharing (e.g. '*' or 'http://localhost:8001,etc')") } -const ( - ElectionTimeout = 200 * time.Millisecond - HeartbeatTimeout = 50 * time.Millisecond - RetryInterval = 10 -) - //------------------------------------------------------------------------------ // // Typedefs @@ -185,16 +179,18 @@ func main() { // Create etcd key-value store etcdStore = store.New() - // Create etcd and raft server - r := newRaftServer(info.Name, info.RaftURL, info.RaftListenHost, &raftTLSConfig, &info.RaftTLS) - r.MaxClusterSize = maxClusterSize - snapConf = r.newSnapshotConf() + // Create a shared node registry. + registry := server.NewRegistry() + + // Create peer server. + ps := NewPeerServer(info.Name, dirPath, info.RaftURL, info.RaftListenHost, &raftTLSConfig, &info.RaftTLS, registry) + ps.MaxClusterSize = maxClusterSize s := server.New(info.Name, info.EtcdURL, info.EtcdListenHost, &etcdTLSConfig, &info.EtcdTLS, r) if err := e.AllowOrigins(cors); err != nil { panic(err) } - r.ListenAndServe() + ps.ListenAndServe(snapshot) s.ListenAndServe() } diff --git a/name_url_map.go b/name_url_map.go deleted file mode 100644 index 220963d5cf2..00000000000 --- a/name_url_map.go +++ /dev/null @@ -1,74 +0,0 @@ -package main - -import ( - "net/url" - "path" -) - -// we map node name to url -type nodeInfo struct { - raftVersion string - raftURL string - etcdURL string -} - -var namesMap = make(map[string]*nodeInfo) - -// nameToEtcdURL maps node name to its etcd http address -func nameToEtcdURL(name string) (string, bool) { - - if info, ok := namesMap[name]; ok { - // first try to read from the map - return info.etcdURL, true - } - - // if fails, try to recover from etcd storage - return readURL(name, "etcd") - -} - -// nameToRaftURL maps node name to its raft http address -func nameToRaftURL(name string) (string, bool) { - if info, ok := namesMap[name]; ok { - // first try to read from the map - return info.raftURL, true - - } - - // if fails, try to recover from etcd storage - return readURL(name, "raft") -} - -// addNameToURL add a name that maps to raftURL and etcdURL -func addNameToURL(name string, version string, raftURL string, etcdURL string) { - namesMap[name] = &nodeInfo{ - raftVersion: raftVersion, - raftURL: raftURL, - etcdURL: etcdURL, - } -} - -func readURL(nodeName string, urlName string) (string, bool) { - if nodeName == "" { - return "", false - } - - // convert nodeName to url from etcd storage - key := path.Join("/_etcd/machines", nodeName) - - e, err := etcdStore.Get(key, false, false, 0, 0) - - if err != nil { - return "", false - } - - m, err := url.ParseQuery(e.Value) - - if err != nil { - panic("Failed to parse machines entry") - } - - url := m[urlName][0] - - return url, true -} diff --git a/raft_handlers.go b/raft_handlers.go deleted file mode 100644 index a45fe496ccb..00000000000 --- a/raft_handlers.go +++ /dev/null @@ -1,147 +0,0 @@ -package main - -import ( - "encoding/json" - "net/http" - - "github.com/coreos/go-raft" -) - -//------------------------------------------------------------- -// Handlers to handle raft related request via raft server port -//------------------------------------------------------------- - -// Get all the current logs -func (r *raftServer) GetLogHttpHandler(w http.ResponseWriter, req *http.Request) { - debugf("[recv] GET %s/log", r.url) - w.Header().Set("Content-Type", "application/json") - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(r.LogEntries()) -} - -// Response to vote request -func (r *raftServer) VoteHttpHandler(w http.ResponseWriter, req *http.Request) { - rvreq := &raft.RequestVoteRequest{} - err := decodeJsonRequest(req, rvreq) - if err == nil { - debugf("[recv] POST %s/vote [%s]", r.url, rvreq.CandidateName) - if resp := r.RequestVote(rvreq); resp != nil { - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(resp) - return - } - } - warnf("[vote] ERROR: %v", err) - w.WriteHeader(http.StatusInternalServerError) -} - -// Response to append entries request -func (r *raftServer) AppendEntriesHttpHandler(w http.ResponseWriter, req *http.Request) { - aereq := &raft.AppendEntriesRequest{} - err := decodeJsonRequest(req, aereq) - - if err == nil { - debugf("[recv] POST %s/log/append [%d]", r.url, len(aereq.Entries)) - - r.serverStats.RecvAppendReq(aereq.LeaderName, int(req.ContentLength)) - - if resp := r.AppendEntries(aereq); resp != nil { - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(resp) - if !resp.Success { - debugf("[Append Entry] Step back") - } - return - } - } - warnf("[Append Entry] ERROR: %v", err) - w.WriteHeader(http.StatusInternalServerError) -} - -// Response to recover from snapshot request -func (r *raftServer) SnapshotHttpHandler(w http.ResponseWriter, req *http.Request) { - aereq := &raft.SnapshotRequest{} - err := decodeJsonRequest(req, aereq) - if err == nil { - debugf("[recv] POST %s/snapshot/ ", r.url) - if resp := r.RequestSnapshot(aereq); resp != nil { - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(resp) - return - } - } - warnf("[Snapshot] ERROR: %v", err) - w.WriteHeader(http.StatusInternalServerError) -} - -// Response to recover from snapshot request -func (r *raftServer) SnapshotRecoveryHttpHandler(w http.ResponseWriter, req *http.Request) { - aereq := &raft.SnapshotRecoveryRequest{} - err := decodeJsonRequest(req, aereq) - if err == nil { - debugf("[recv] POST %s/snapshotRecovery/ ", r.url) - if resp := r.SnapshotRecoveryRequest(aereq); resp != nil { - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(resp) - return - } - } - warnf("[Snapshot] ERROR: %v", err) - w.WriteHeader(http.StatusInternalServerError) -} - -// Get the port that listening for etcd connecting of the server -func (r *raftServer) EtcdURLHttpHandler(w http.ResponseWriter, req *http.Request) { - debugf("[recv] Get %s/etcdURL/ ", r.url) - w.WriteHeader(http.StatusOK) - w.Write([]byte(argInfo.EtcdURL)) -} - -// Response to the join request -func (r *raftServer) JoinHttpHandler(w http.ResponseWriter, req *http.Request) error { - - command := &JoinCommand{} - - if err := decodeJsonRequest(req, command); err == nil { - debugf("Receive Join Request from %s", command.Name) - return r.dispatchRaftCommand(command, w, req) - } else { - w.WriteHeader(http.StatusInternalServerError) - return nil - } -} - -// Response to remove request -func (r *raftServer) RemoveHttpHandler(w http.ResponseWriter, req *http.Request) { - if req.Method != "DELETE" { - w.WriteHeader(http.StatusMethodNotAllowed) - return - } - - nodeName := req.URL.Path[len("/remove/"):] - command := &RemoveCommand{ - Name: nodeName, - } - - debugf("[recv] Remove Request [%s]", command.Name) - - r.dispatchRaftCommand(command, w, req) -} - -// Response to the name request -func (r *raftServer) NameHttpHandler(w http.ResponseWriter, req *http.Request) { - debugf("[recv] Get %s/name/ ", r.url) - w.WriteHeader(http.StatusOK) - w.Write([]byte(r.name)) -} - -// Response to the name request -func (r *raftServer) RaftVersionHttpHandler(w http.ResponseWriter, req *http.Request) { - debugf("[recv] Get %s/version/ ", r.url) - w.WriteHeader(http.StatusOK) - w.Write([]byte(r.version)) -} - -func (r *raftServer) dispatchRaftCommand(c Command, w http.ResponseWriter, req *http.Request) error { - return r.dispatch(c, w, req, nameToRaftURL) -} diff --git a/raft_server.go b/raft_server.go deleted file mode 100644 index 00cf4cbdbf3..00000000000 --- a/raft_server.go +++ /dev/null @@ -1,317 +0,0 @@ -package main - -import ( - "bytes" - "crypto/tls" - "encoding/binary" - "encoding/json" - "fmt" - "io/ioutil" - "net/http" - "net/url" - "time" - - "github.com/coreos/etcd/command" - etcdErr "github.com/coreos/etcd/error" - "github.com/coreos/go-raft" -) - -func init() { - command.Register() -} - -type raftServer struct { - *raft.Server - version string - joinIndex uint64 - name string - url string - listenHost string - tlsConf *TLSConfig - tlsInfo *TLSInfo - followersStats *raftFollowersStats - serverStats *raftServerStats - MaxClusterSize int -} - -func newRaftServer(name string, url string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo) *raftServer { - - raftWrapper := &raftServer{ - version: raftVersion, - name: name, - url: url, - listenHost: listenHost, - tlsConf: tlsConf, - tlsInfo: tlsInfo, - followersStats: &raftFollowersStats{ - Leader: name, - Followers: make(map[string]*raftFollowerStats), - }, - serverStats: &raftServerStats{ - StartTime: time.Now(), - sendRateQueue: &statsQueue{ - back: -1, - }, - recvRateQueue: &statsQueue{ - back: -1, - }, - }, - } - - // Create transporter for raft - raftTransporter := newTransporter(tlsConf.Scheme, tlsConf.Client, raftWrapper) - - // Create raft server - server, err := raft.NewServer(name, dirPath, raftTransporter, etcdStore, raftWrapper, "") - check(err) - - raftWrapper.Server = server - - return raftWrapper -} - -// Start the raft server -func (r *raftServer) ListenAndServe() { - // LoadSnapshot - if snapshot { - err := r.LoadSnapshot() - - if err == nil { - debugf("%s finished load snapshot", r.name) - } else { - debug(err) - } - } - - r.SetElectionTimeout(ElectionTimeout) - r.SetHeartbeatTimeout(HeartbeatTimeout) - - r.Start() - - if r.IsLogEmpty() { - - // start as a leader in a new cluster - if len(cluster) == 0 { - r.startAsLeader() - - } else { - r.startAsFollower() - } - - } else { - - // rejoin the previous cluster - cluster = r.getMachines(nameToRaftURL) - for i := 0; i < len(cluster); i++ { - u, err := url.Parse(cluster[i]) - if err != nil { - debug("rejoin cannot parse url: ", err) - } - cluster[i] = u.Host - } - ok := r.joinCluster(cluster) - if !ok { - warn("the entire cluster is down! this machine will restart the cluster.") - } - - debugf("%s restart as a follower", r.name) - } - - // open the snapshot - if snapshot { - go r.monitorSnapshot() - } - - // start to response to raft requests - go r.startTransport(r.tlsConf.Scheme, r.tlsConf.Server) - -} - -func (r *raftServer) startAsLeader() { - // leader need to join self as a peer - for { - _, err := r.Do(newJoinCommand(r.version, r.Name(), r.url, e.url)) - if err == nil { - break - } - } - debugf("%s start as a leader", r.name) -} - -func (r *raftServer) startAsFollower() { - // start as a follower in a existing cluster - for i := 0; i < retryTimes; i++ { - ok := r.joinCluster(cluster) - if ok { - return - } - warnf("cannot join to cluster via given machines, retry in %d seconds", RetryInterval) - time.Sleep(time.Second * RetryInterval) - } - - fatalf("Cannot join the cluster via given machines after %x retries", retryTimes) -} - -// Start to listen and response raft command -func (r *raftServer) startTransport(scheme string, tlsConf tls.Config) { - infof("raft server [name %s, listen on %s, advertised url %s]", r.name, r.listenHost, r.url) - - raftMux := http.NewServeMux() - - server := &http.Server{ - Handler: raftMux, - TLSConfig: &tlsConf, - Addr: r.listenHost, - } - - // internal commands - raftMux.HandleFunc("/name", r.NameHttpHandler) - raftMux.HandleFunc("/version", r.RaftVersionHttpHandler) - raftMux.Handle("/join", errorHandler(r.JoinHttpHandler)) - raftMux.HandleFunc("/remove/", r.RemoveHttpHandler) - raftMux.HandleFunc("/vote", r.VoteHttpHandler) - raftMux.HandleFunc("/log", r.GetLogHttpHandler) - raftMux.HandleFunc("/log/append", r.AppendEntriesHttpHandler) - raftMux.HandleFunc("/snapshot", r.SnapshotHttpHandler) - raftMux.HandleFunc("/snapshotRecovery", r.SnapshotRecoveryHttpHandler) - raftMux.HandleFunc("/etcdURL", r.EtcdURLHttpHandler) - - if scheme == "http" { - fatal(server.ListenAndServe()) - } else { - fatal(server.ListenAndServeTLS(r.tlsInfo.CertFile, r.tlsInfo.KeyFile)) - } - -} - -// getVersion fetches the raft version of a peer. This works for now but we -// will need to do something more sophisticated later when we allow mixed -// version clusters. -func getVersion(t *transporter, versionURL url.URL) (string, error) { - resp, req, err := t.Get(versionURL.String()) - - if err != nil { - return "", err - } - - defer resp.Body.Close() - - t.CancelWhenTimeout(req) - - body, err := ioutil.ReadAll(resp.Body) - - return string(body), nil -} - -func (r *raftServer) joinCluster(cluster []string) bool { - for _, machine := range cluster { - - if len(machine) == 0 { - continue - } - - err := r.joinByMachine(r.Server, machine, r.tlsConf.Scheme) - if err == nil { - debugf("%s success join to the cluster via machine %s", r.name, machine) - return true - - } else { - if _, ok := err.(etcdErr.Error); ok { - fatal(err) - } - - debugf("cannot join to cluster via machine %s %s", machine, err) - } - } - return false -} - -// Send join requests to machine. -func (r *raftServer) joinByMachine(s *raft.Server, machine string, scheme string) error { - var b bytes.Buffer - - // t must be ok - t, _ := r.Transporter().(*transporter) - - // Our version must match the leaders version - versionURL := url.URL{Host: machine, Scheme: scheme, Path: "/version"} - version, err := getVersion(t, versionURL) - if err != nil { - return fmt.Errorf("Unable to join: %v", err) - } - - // TODO: versioning of the internal protocol. See: - // Documentation/internatl-protocol-versioning.md - if version != r.version { - return fmt.Errorf("Unable to join: internal version mismatch, entire cluster must be running identical versions of etcd") - } - - json.NewEncoder(&b).Encode(newJoinCommand(r.version, r.Name(), r.url, e.url)) - - joinURL := url.URL{Host: machine, Scheme: scheme, Path: "/join"} - - debugf("Send Join Request to %s", joinURL.String()) - - resp, req, err := t.Post(joinURL.String(), &b) - - for { - if err != nil { - return fmt.Errorf("Unable to join: %v", err) - } - if resp != nil { - defer resp.Body.Close() - - t.CancelWhenTimeout(req) - - if resp.StatusCode == http.StatusOK { - b, _ := ioutil.ReadAll(resp.Body) - r.joinIndex, _ = binary.Uvarint(b) - return nil - } - if resp.StatusCode == http.StatusTemporaryRedirect { - - address := resp.Header.Get("Location") - debugf("Send Join Request to %s", address) - - json.NewEncoder(&b).Encode(newJoinCommand(r.version, r.Name(), r.url, e.url)) - - resp, req, err = t.Post(address, &b) - - } else if resp.StatusCode == http.StatusBadRequest { - debug("Reach max number machines in the cluster") - decoder := json.NewDecoder(resp.Body) - err := &etcdErr.Error{} - decoder.Decode(err) - return *err - } else { - return fmt.Errorf("Unable to join") - } - } - - } - return fmt.Errorf("Unable to join: %v", err) -} - -func (r *raftServer) Stats() []byte { - r.serverStats.LeaderInfo.Uptime = time.Now().Sub(r.serverStats.LeaderInfo.startTime).String() - - queue := r.serverStats.sendRateQueue - - r.serverStats.SendingPkgRate, r.serverStats.SendingBandwidthRate = queue.Rate() - - queue = r.serverStats.recvRateQueue - - r.serverStats.RecvingPkgRate, r.serverStats.RecvingBandwidthRate = queue.Rate() - - b, _ := json.Marshal(r.serverStats) - - return b -} - -func (r *raftServer) PeerStats() []byte { - if r.State() == raft.Leader { - b, _ := json.Marshal(r.followersStats) - return b - } - return nil -} diff --git a/server/join_command.go b/server/join_command.go index 89a3a4e547e..f338f3d390c 100644 --- a/server/join_command.go +++ b/server/join_command.go @@ -2,8 +2,6 @@ package server import ( "encoding/binary" - "fmt" - "path" etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/log" @@ -42,38 +40,32 @@ func (c *JoinCommand) CommandName() string { // Join a server to the cluster func (c *JoinCommand) Apply(server *raft.Server) (interface{}, error) { s, _ := server.StateMachine().(*store.Store) - r, _ := server.Context().(*RaftServer) - - // check if the join command is from a previous machine, who lost all its previous log. - e, _ := s.Get(path.Join("/_etcd/machines", c.Name), false, false, server.CommitIndex(), server.Term()) + ps, _ := server.Context().(*PeerServer) b := make([]byte, 8) binary.PutUvarint(b, server.CommitIndex()) - if e != nil { + // Check if the join command is from a previous machine, who lost all its previous log. + if _, ok := ps.registry.URL(c.Name); ok { return b, nil } - // check machine number in the cluster - if s.MachineCount() == c.MaxClusterSize { + // Check machine number in the cluster + if ps.registry.Count() == c.MaxClusterSize { log.Debug("Reject join request from ", c.Name) return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMoreMachine, "", server.CommitIndex(), server.Term()) } - addNameToURL(c.Name, c.RaftVersion, c.RaftURL, c.EtcdURL) + // Add to shared machine registry. + ps.registry.Register(c.Name, c.RaftVersion, c.RaftURL, c.EtcdURL, server.CommitIndex(), server.Term()) - // add peer in raft + // Add peer in raft err := server.AddPeer(c.Name, "") - // add machine in etcd storage - key := path.Join("_etcd/machines", c.Name) - value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", c.RaftURL, c.EtcdURL, c.RaftVersion) - s.Create(key, value, false, false, store.Permanent, server.CommitIndex(), server.Term()) - - // add peer stats - if c.Name != r.Name() { - r.followersStats.Followers[c.Name] = &raftFollowerStats{} - r.followersStats.Followers[c.Name].Latency.Minimum = 1 << 63 + // Add peer stats + if c.Name != ps.Name() { + ps.followersStats.Followers[c.Name] = &raftFollowerStats{} + ps.followersStats.Followers[c.Name].Latency.Minimum = 1 << 63 } return b, err diff --git a/server/peer_server.go b/server/peer_server.go new file mode 100644 index 00000000000..0c24c1521cb --- /dev/null +++ b/server/peer_server.go @@ -0,0 +1,474 @@ +package server + +import ( + "bytes" + "crypto/tls" + "encoding/binary" + "encoding/json" + "fmt" + "io/ioutil" + "net/http" + "net/url" + "time" + + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" +) + +type PeerServer struct { + *raft.Server + joinIndex uint64 + name string + url string + listenHost string + tlsConf *TLSConfig + tlsInfo *TLSInfo + followersStats *raftFollowersStats + serverStats *raftServerStats + registry *Registry + store *store.Store + snapConf *snapshotConf + MaxClusterSize int +} + +// TODO: find a good policy to do snapshot +type snapshotConf struct { + // Etcd will check if snapshot is need every checkingInterval + checkingInterval time.Duration + + // The number of writes when the last snapshot happened + lastWrites uint64 + + // If the incremental number of writes since the last snapshot + // exceeds the write Threshold, etcd will do a snapshot + writesThr uint64 +} + + +func NewPeerServer(name string, path string, url string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, registry *Registry, store *store.Store) *PeerServer { + s := &PeerServer{ + name: name, + url: url, + listenHost: listenHost, + tlsConf: tlsConf, + tlsInfo: tlsInfo, + registry: registry, + store: store, + snapConf: &snapshotConf{time.Second * 3, 0, 20 * 1000}, + followersStats: &raftFollowersStats{ + Leader: name, + Followers: make(map[string]*raftFollowerStats), + }, + serverStats: &raftServerStats{ + StartTime: time.Now(), + sendRateQueue: &statsQueue{ + back: -1, + }, + recvRateQueue: &statsQueue{ + back: -1, + }, + }, + } + + // Create transporter for raft + raftTransporter := newTransporter(tlsConf.Scheme, tlsConf.Client, s) + + // Create raft server + server, err := raft.NewServer(name, path, raftTransporter, s.store, s, "") + if err != nil { + log.Fatal(err) + } + + s.Server = server + + return s +} + +// Start the raft server +func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) { + // LoadSnapshot + if snapshot { + err := s.LoadSnapshot() + + if err == nil { + log.Debugf("%s finished load snapshot", s.name) + } else { + log.Debug(err) + } + } + + s.SetElectionTimeout(ElectionTimeout) + s.SetHeartbeatTimeout(HeartbeatTimeout) + + s.Start() + + if s.IsLogEmpty() { + // start as a leader in a new cluster + if len(cluster) == 0 { + s.startAsLeader() + } else { + s.startAsFollower(cluster) + } + + } else { + // Rejoin the previous cluster + cluster = s.registry.PeerURLs() + for i := 0; i < len(cluster); i++ { + u, err := url.Parse(cluster[i]) + if err != nil { + log.Debug("rejoin cannot parse url: ", err) + } + cluster[i] = u.Host + } + ok := s.joinCluster(cluster) + if !ok { + log.Warn("the entire cluster is down! this machine will restart the cluster.") + } + + log.Debugf("%s restart as a follower", s.name) + } + + // open the snapshot + if snapshot { + go s.monitorSnapshot() + } + + // start to response to raft requests + go s.startTransport(s.tlsConf.Scheme, s.tlsConf.Server) + +} + +// Get all the current logs +func (s *PeerServer) GetLogHttpHandler(w http.ResponseWriter, req *http.Request) { + log.Debugf("[recv] GET %s/log", s.url) + w.Header().Set("Content-Type", "application/json") + w.WriteHeader(http.StatusOK) + json.NewEncoder(w).Encode(s.LogEntries()) +} + +// Response to vote request +func (s *PeerServer) VoteHttpHandler(w http.ResponseWriter, req *http.Request) { + rvreq := &raft.RequestVoteRequest{} + err := decodeJsonRequest(req, rvreq) + if err == nil { + log.Debugf("[recv] POST %s/vote [%s]", s.url, rvreq.CandidateName) + if resp := s.RequestVote(rvreq); resp != nil { + w.WriteHeader(http.StatusOK) + json.NewEncoder(w).Encode(resp) + return + } + } + log.Warnf("[vote] ERROR: %v", err) + w.WriteHeader(http.StatusInternalServerError) +} + +// Response to append entries request +func (s *PeerServer) AppendEntriesHttpHandler(w http.ResponseWriter, req *http.Request) { + aereq := &raft.AppendEntriesRequest{} + err := decodeJsonRequest(req, aereq) + + if err == nil { + log.Debugf("[recv] POST %s/log/append [%d]", s.url, len(aereq.Entries)) + + s.serverStats.RecvAppendReq(aereq.LeaderName, int(req.ContentLength)) + + if resp := s.AppendEntries(aereq); resp != nil { + w.WriteHeader(http.StatusOK) + json.NewEncoder(w).Encode(resp) + if !resp.Success { + log.Debugf("[Append Entry] Step back") + } + return + } + } + log.Warnf("[Append Entry] ERROR: %v", err) + w.WriteHeader(http.StatusInternalServerError) +} + +// Response to recover from snapshot request +func (s *PeerServer) SnapshotHttpHandler(w http.ResponseWriter, req *http.Request) { + aereq := &raft.SnapshotRequest{} + err := decodeJsonRequest(req, aereq) + if err == nil { + log.Debugf("[recv] POST %s/snapshot/ ", s.url) + if resp := s.RequestSnapshot(aereq); resp != nil { + w.WriteHeader(http.StatusOK) + json.NewEncoder(w).Encode(resp) + return + } + } + log.Warnf("[Snapshot] ERROR: %v", err) + w.WriteHeader(http.StatusInternalServerError) +} + +// Response to recover from snapshot request +func (s *PeerServer) SnapshotRecoveryHttpHandler(w http.ResponseWriter, req *http.Request) { + aereq := &raft.SnapshotRecoveryRequest{} + err := decodeJsonRequest(req, aereq) + if err == nil { + log.Debugf("[recv] POST %s/snapshotRecovery/ ", s.url) + if resp := s.SnapshotRecoveryRequest(aereq); resp != nil { + w.WriteHeader(http.StatusOK) + json.NewEncoder(w).Encode(resp) + return + } + } + log.Warnf("[Snapshot] ERROR: %v", err) + w.WriteHeader(http.StatusInternalServerError) +} + +// Get the port that listening for etcd connecting of the server +func (s *PeerServer) EtcdURLHttpHandler(w http.ResponseWriter, req *http.Request) { + log.Debugf("[recv] Get %s/etcdURL/ ", s.url) + w.WriteHeader(http.StatusOK) + w.Write([]byte(argInfo.EtcdURL)) +} + +// Response to the join request +func (s *PeerServer) JoinHttpHandler(w http.ResponseWriter, req *http.Request) error { + command := &JoinCommand{} + + if err := decodeJsonRequest(req, command); err == nil { + log.Debugf("Receive Join Request from %s", command.Name) + return s.dispatchRaftCommand(command, w, req) + } else { + w.WriteHeader(http.StatusInternalServerError) + return nil + } +} + +// Response to remove request +func (s *PeerServer) RemoveHttpHandler(w http.ResponseWriter, req *http.Request) { + if req.Method != "DELETE" { + w.WriteHeader(http.StatusMethodNotAllowed) + return + } + + nodeName := req.URL.Path[len("/remove/"):] + command := &RemoveCommand{ + Name: nodeName, + } + + log.Debugf("[recv] Remove Request [%s]", command.Name) + + s.dispatchRaftCommand(command, w, req) +} + +// Response to the name request +func (s *PeerServer) NameHttpHandler(w http.ResponseWriter, req *http.Request) { + log.Debugf("[recv] Get %s/name/ ", s.url) + w.WriteHeader(http.StatusOK) + w.Write([]byte(s.name)) +} + +// Response to the name request +func (s *PeerServer) RaftVersionHttpHandler(w http.ResponseWriter, req *http.Request) { + log.Debugf("[recv] Get %s/version/ ", s.url) + w.WriteHeader(http.StatusOK) + w.Write([]byte(PeerVersion)) +} + +func (s *PeerServer) dispatchRaftCommand(c raft.Command, w http.ResponseWriter, req *http.Request) error { + return s.dispatch(c, w, req, nameToRaftURL) +} + +func (s *PeerServer) startAsLeader() { + // leader need to join self as a peer + for { + _, err := s.Do(newJoinCommand(PeerVersion, s.Name(), s.url, e.url)) + if err == nil { + break + } + } + log.Debugf("%s start as a leader", s.name) +} + +func (s *PeerServer) startAsFollower(cluster []string) { + // start as a follower in a existing cluster + for i := 0; i < retryTimes; i++ { + ok := s.joinCluster(cluster) + if ok { + return + } + log.Warnf("cannot join to cluster via given machines, retry in %d seconds", RetryInterval) + time.Sleep(time.Second * RetryInterval) + } + + fatalf("Cannot join the cluster via given machines after %x retries", retryTimes) +} + +// Start to listen and response raft command +func (s *PeerServer) startTransport(scheme string, tlsConf tls.Config) { + infof("raft server [name %s, listen on %s, advertised url %s]", s.name, s.listenHost, s.url) + + raftMux := http.NewServeMux() + + server := &http.Server{ + Handler: raftMux, + TLSConfig: &tlsConf, + Addr: s.listenHost, + } + + // internal commands + raftMux.HandleFunc("/name", s.NameHttpHandler) + raftMux.HandleFunc("/version", s.RaftVersionHttpHandler) + raftMux.Handle("/join", errorHandler(s.JoinHttpHandler)) + raftMux.HandleFunc("/remove/", s.RemoveHttpHandler) + raftMux.HandleFunc("/vote", s.VoteHttpHandler) + raftMux.HandleFunc("/log", s.GetLogHttpHandler) + raftMux.HandleFunc("/log/append", s.AppendEntriesHttpHandler) + raftMux.HandleFunc("/snapshot", s.SnapshotHttpHandler) + raftMux.HandleFunc("/snapshotRecovery", s.SnapshotRecoveryHttpHandler) + raftMux.HandleFunc("/etcdURL", s.EtcdURLHttpHandler) + + if scheme == "http" { + fatal(server.ListenAndServe()) + } else { + fatal(server.ListenAndServeTLS(s.tlsInfo.CertFile, s.tlsInfo.KeyFile)) + } + +} + +// getVersion fetches the raft version of a peer. This works for now but we +// will need to do something more sophisticated later when we allow mixed +// version clusters. +func getVersion(t *transporter, versionURL url.URL) (string, error) { + resp, req, err := t.Get(versionURL.String()) + + if err != nil { + return "", err + } + + defer resp.Body.Close() + + t.CancelWhenTimeout(req) + + body, err := ioutil.ReadAll(resp.Body) + + return string(body), nil +} + +func (s *PeerServer) joinCluster(cluster []string) bool { + for _, machine := range cluster { + if len(machine) == 0 { + continue + } + + err := s.joinByMachine(s.Server, machine, s.tlsConf.Scheme) + if err == nil { + log.Debugf("%s success join to the cluster via machine %s", s.name, machine) + return true + + } else { + if _, ok := err.(etcdErr.Error); ok { + fatal(err) + } + + log.Debugf("cannot join to cluster via machine %s %s", machine, err) + } + } + return false +} + +// Send join requests to machine. +func (s *PeerServer) joinByMachine(server *raft.Server, machine string, scheme string) error { + var b bytes.Buffer + + // t must be ok + t, _ := server.Transporter().(*transporter) + + // Our version must match the leaders version + versionURL := url.URL{Host: machine, Scheme: scheme, Path: "/version"} + version, err := getVersion(t, versionURL) + if err != nil { + return fmt.Errorf("Unable to join: %v", err) + } + + // TODO: versioning of the internal protocol. See: + // Documentation/internatl-protocol-versioning.md + if version != PeerVersion { + return fmt.Errorf("Unable to join: internal version mismatch, entire cluster must be running identical versions of etcd") + } + + json.NewEncoder(&b).Encode(newJoinCommand(PeerVersion, server.Name(), server.url, e.url)) + + joinURL := url.URL{Host: machine, Scheme: scheme, Path: "/join"} + + log.Debugf("Send Join Request to %s", joinURL.String()) + + resp, req, err := t.Post(joinURL.String(), &b) + + for { + if err != nil { + return fmt.Errorf("Unable to join: %v", err) + } + if resp != nil { + defer resp.Body.Close() + + t.CancelWhenTimeout(req) + + if resp.StatusCode == http.StatusOK { + b, _ := ioutil.ReadAll(resp.Body) + server.joinIndex, _ = binary.Uvarint(b) + return nil + } + if resp.StatusCode == http.StatusTemporaryRedirect { + + address := resp.Header.Get("Location") + log.Debugf("Send Join Request to %s", address) + + json.NewEncoder(&b).Encode(newJoinCommand(PeerVersion, server.Name(), server.url, e.url)) + + resp, req, err = t.Post(address, &b) + + } else if resp.StatusCode == http.StatusBadRequest { + debug("Reach max number machines in the cluster") + decoder := json.NewDecoder(resp.Body) + err := &etcdErr.Error{} + decoder.Decode(err) + return *err + } else { + return fmt.Errorf("Unable to join") + } + } + + } + return fmt.Errorf("Unable to join: %v", err) +} + +func (s *PeerServer) Stats() []byte { + s.serverStats.LeaderInfo.Uptime = time.Now().Sub(s.serverStats.LeaderInfo.startTime).String() + + queue := s.serverStats.sendRateQueue + + s.serverStats.SendingPkgRate, s.serverStats.SendingBandwidthRate = queue.Rate() + + queue = s.serverStats.recvRateQueue + + s.serverStats.RecvingPkgRate, s.serverStats.RecvingBandwidthRate = queue.Rate() + + b, _ := json.Marshal(s.serverStats) + + return b +} + +func (s *PeerServer) PeerStats() []byte { + if s.State() == raft.Leader { + b, _ := json.Marshal(s.followersStats) + return b + } + return nil +} + +func (s *PeerServer) monitorSnapshot() { + for { + time.Sleep(s.snapConf.checkingInterval) + currentWrites := 0 + if uint64(currentWrites) > s.snapConf.writesThr { + r.TakeSnapshot() + s.snapConf.lastWrites = 0 + } + } +} diff --git a/server/registry.go b/server/registry.go new file mode 100644 index 00000000000..cd7078d8120 --- /dev/null +++ b/server/registry.go @@ -0,0 +1,167 @@ +package server + +import ( + "sync" + + "github.com/coreos/etcd/store" +) + +// The location of the machine URL data. +const RegistryKey = "/_etcd/machines" + +// The Registry stores URL information for nodes. +type Registry struct { + sync.Mutex + store *store.Store + nodes map[string]*node +} + +// The internal storage format of the registry. +type node struct { + peerVersion string + peerURL string + url string +} + +// Creates a new Registry. +func NewRegistry(s *store.Store) *Registry { + return &Registry{ + store: s, + nodes: make(map[string]*node), + } +} + +// Adds a node to the registry. +func (r *Registry) Register(name string, peerVersion string, peerURL string, url string, commitIndex uint64, term uint64) { + r.Lock() + defer r.Unlock() + + // Write data to store. + key := path.Join(RegistryKey, name) + value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", peerURL, url, peerVersion) + r.store.Create(key, value, false, false, store.Permanent, commitIndex, term) +} + +// Removes a node from the registry. +func (r *Registry) Unregister(name string, commitIndex uint64, term uint64) error { + r.Lock() + defer r.Unlock() + + // Remove the key from the store. + _, err := s.Delete(path.Join(RegistryKey, name), false, commitIndex, term) + return err +} + +// Returns the number of nodes in the cluster. +func (r *Registry) Count() int { + e, err := s.Get(RegistryKey, false, false, 0, 0) + if err != nil { + return 0 + } + return len(e.KVPairs) +} + +// Retrieves the URL for a given node by name. +func (r *Registry) URL(name string) (string, bool) { + r.Lock() + defer r.Unlock() + return r.url(name) +} + +func (r *Registry) url(name string) (string, bool) { + if r.nodes[name] == nil { + r.load(name) + } + + if node := r.nodes[name]; node != nil { + return node.url, true + } + + return "", false +} + +// Retrieves the URLs for all nodes. +func (r *Registry) URLs() []string { + r.Lock() + defer r.Unlock() + + // Retrieve a list of all nodes. + e, err := s.Get(RegistryKey, false, false, 0, 0) + if err != nil { + return make([]string, 0) + } + + // Lookup the URL for each one. + urls := make([]string, 0) + for _, pair := range e.KVPairs { + urls = append(urls, r.url(pair.Key)) + } + + return urls +} + + +// Retrieves the peer URL for a given node by name. +func (r *Registry) PeerURL(name string) (string, bool) { + r.Lock() + defer r.Unlock() + return r.peerURL(name) +} + +func (r *Registry) peerURL(name string) (string, bool) { + if r.nodes[name] == nil { + r.load(name) + } + + if node := r.nodes[name]; node != nil { + return node.peerURL, true + } + + return "", false +} + +// Retrieves the peer URLs for all nodes. +func (r *Registry) PeerURLs() []string { + r.Lock() + defer r.Unlock() + + // Retrieve a list of all nodes. + e, err := s.Get(RegistryKey, false, false, 0, 0) + if err != nil { + return make([]string, 0) + } + + // Lookup the URL for each one. + urls := make([]string, 0) + for _, pair := range e.KVPairs { + urls = append(urls, r.peerURL(pair.Key)) + } + + return urls +} + +// Loads the given node by name from the store into the cache. +func (r *Registry) load(name string) { + if name == "" { + return + } + + // Retrieve from store. + e, err := etcdStore.Get(path.Join(RegistryKey, name), false, false, 0, 0) + if err != nil { + return + } + + // Parse as a query string. + m, err := url.ParseQuery(e.Value) + if err != nil { + panic(fmt.Sprintf("Failed to parse machines entry: %s", name)) + } + + // Create node. + r.nodes[name] := &node{ + url: m["etcd"][0], + peerURL: m["raft"][0], + peerVersion: m["raftVersion"][0], + } +} diff --git a/server/remove_command.go b/server/remove_command.go index a992de67c7e..43ff17d248d 100644 --- a/server/remove_command.go +++ b/server/remove_command.go @@ -25,22 +25,20 @@ func (c *RemoveCommand) CommandName() string { // Remove a server from the cluster func (c *RemoveCommand) Apply(server *raft.Server) (interface{}, error) { s, _ := server.StateMachine().(*store.Store) - r, _ := server.Context().(*RaftServer) + ps, _ := server.Context().(*PeerServer) - // remove machine in etcd storage - key := path.Join("_etcd/machines", c.Name) + // Remove node from the shared registry. + err := ps.registry.Unregister(c.Name, server.CommitIndex(), server.Term()) - _, err := s.Delete(key, false, server.CommitIndex(), server.Term()) - // delete from stats - delete(r.followersStats.Followers, c.Name) + // Delete from stats + delete(ps.followersStats.Followers, c.Name) if err != nil { return []byte{0}, err } - // remove peer in raft + // Remove peer in raft err = server.RemovePeer(c.Name) - if err != nil { return []byte{0}, err } @@ -52,7 +50,7 @@ func (c *RemoveCommand) Apply(server *raft.Server) (interface{}, error) { // and the node has sent out a join request in this // start. It is sure that this node received a new remove // command and need to be removed - if server.CommitIndex() > r.joinIndex && r.joinIndex != 0 { + if server.CommitIndex() > ps.joinIndex && ps.joinIndex != 0 { debugf("server [%s] is removed", server.Name()) os.Exit(0) } else { diff --git a/server/server.go b/server/server.go index fad643aec40..5835e1a121a 100644 --- a/server/server.go +++ b/server/server.go @@ -4,7 +4,6 @@ import ( "net/http" "net/url" - "github.com/coreos/etcd/command" "github.com/coreos/go-raft" "github.com/gorilla/mux" ) @@ -13,7 +12,7 @@ import ( type Server interface { CommitIndex() uint64 Term() uint64 - Dispatch(command.Command, http.ResponseWriter, *http.Request) + Dispatch(raft.Command, http.ResponseWriter, *http.Request) } // This is the default implementation of the Server interface. diff --git a/server/timeout.go b/server/timeout.go new file mode 100644 index 00000000000..35b49b63035 --- /dev/null +++ b/server/timeout.go @@ -0,0 +1,11 @@ +package server + +const ( + // The amount of time to elapse without a heartbeat before becoming a candidate. + ElectionTimeout = 200 * time.Millisecond + + // The frequency by which heartbeats are sent to followers. + HeartbeatTimeout = 50 * time.Millisecond + + RetryInterval = 10 +) diff --git a/server/transporter.go b/server/transporter.go new file mode 100644 index 00000000000..83f0f07fc2c --- /dev/null +++ b/server/transporter.go @@ -0,0 +1,226 @@ +package server + +import ( + "bytes" + "crypto/tls" + "encoding/json" + "fmt" + "io" + "net" + "net/http" + "time" + + "github.com/coreos/go-raft" +) + +// Timeout for setup internal raft http connection +// This should not exceed 3 * RTT +var dailTimeout = 3 * HeartbeatTimeout + +// Timeout for setup internal raft http connection + receive response header +// This should not exceed 3 * RTT + RTT +var responseHeaderTimeout = 4 * HeartbeatTimeout + +// Timeout for receiving the response body from the server +// This should not exceed election timeout +var tranTimeout = ElectionTimeout + +// Transporter layer for communication between raft nodes +type transporter struct { + client *http.Client + transport *http.Transport + raftServer *raftServer +} + +// Create transporter using by raft server +// Create http or https transporter based on +// whether the user give the server cert and key +func newTransporter(scheme string, tlsConf tls.Config, raftServer *raftServer) *transporter { + t := transporter{} + + tr := &http.Transport{ + Dial: dialWithTimeout, + ResponseHeaderTimeout: responseHeaderTimeout, + } + + if scheme == "https" { + tr.TLSClientConfig = &tlsConf + tr.DisableCompression = true + } + + t.client = &http.Client{Transport: tr} + t.transport = tr + t.raftServer = raftServer + + return &t +} + +// Dial with timeout +func dialWithTimeout(network, addr string) (net.Conn, error) { + return net.DialTimeout(network, addr, dailTimeout) +} + +// Sends AppendEntries RPCs to a peer when the server is the leader. +func (t *transporter) SendAppendEntriesRequest(server *raft.Server, peer *raft.Peer, req *raft.AppendEntriesRequest) *raft.AppendEntriesResponse { + var aersp *raft.AppendEntriesResponse + var b bytes.Buffer + + json.NewEncoder(&b).Encode(req) + + size := b.Len() + + t.raftServer.serverStats.SendAppendReq(size) + + u, _ := nameToRaftURL(peer.Name) + + debugf("Send LogEntries to %s ", u) + + thisFollowerStats, ok := t.raftServer.followersStats.Followers[peer.Name] + + if !ok { //this is the first time this follower has been seen + thisFollowerStats = &raftFollowerStats{} + thisFollowerStats.Latency.Minimum = 1 << 63 + t.raftServer.followersStats.Followers[peer.Name] = thisFollowerStats + } + + start := time.Now() + + resp, httpRequest, err := t.Post(fmt.Sprintf("%s/log/append", u), &b) + + end := time.Now() + + if err != nil { + debugf("Cannot send AppendEntriesRequest to %s: %s", u, err) + if ok { + thisFollowerStats.Fail() + } + } else { + if ok { + thisFollowerStats.Succ(end.Sub(start)) + } + } + + if resp != nil { + defer resp.Body.Close() + + t.CancelWhenTimeout(httpRequest) + + aersp = &raft.AppendEntriesResponse{} + if err := json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { + return aersp + } + + } + + return aersp +} + +// Sends RequestVote RPCs to a peer when the server is the candidate. +func (t *transporter) SendVoteRequest(server *raft.Server, peer *raft.Peer, req *raft.RequestVoteRequest) *raft.RequestVoteResponse { + var rvrsp *raft.RequestVoteResponse + var b bytes.Buffer + json.NewEncoder(&b).Encode(req) + + u, _ := nameToRaftURL(peer.Name) + debugf("Send Vote to %s", u) + + resp, httpRequest, err := t.Post(fmt.Sprintf("%s/vote", u), &b) + + if err != nil { + debugf("Cannot send VoteRequest to %s : %s", u, err) + } + + if resp != nil { + defer resp.Body.Close() + + t.CancelWhenTimeout(httpRequest) + + rvrsp := &raft.RequestVoteResponse{} + if err := json.NewDecoder(resp.Body).Decode(&rvrsp); err == nil || err == io.EOF { + return rvrsp + } + + } + return rvrsp +} + +// Sends SnapshotRequest RPCs to a peer when the server is the candidate. +func (t *transporter) SendSnapshotRequest(server *raft.Server, peer *raft.Peer, req *raft.SnapshotRequest) *raft.SnapshotResponse { + var aersp *raft.SnapshotResponse + var b bytes.Buffer + json.NewEncoder(&b).Encode(req) + + u, _ := nameToRaftURL(peer.Name) + debugf("Send Snapshot to %s [Last Term: %d, LastIndex %d]", u, + req.LastTerm, req.LastIndex) + + resp, httpRequest, err := t.Post(fmt.Sprintf("%s/snapshot", u), &b) + + if err != nil { + debugf("Cannot send SendSnapshotRequest to %s : %s", u, err) + } + + if resp != nil { + defer resp.Body.Close() + + t.CancelWhenTimeout(httpRequest) + + aersp = &raft.SnapshotResponse{} + if err = json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { + + return aersp + } + } + + return aersp +} + +// Sends SnapshotRecoveryRequest RPCs to a peer when the server is the candidate. +func (t *transporter) SendSnapshotRecoveryRequest(server *raft.Server, peer *raft.Peer, req *raft.SnapshotRecoveryRequest) *raft.SnapshotRecoveryResponse { + var aersp *raft.SnapshotRecoveryResponse + var b bytes.Buffer + json.NewEncoder(&b).Encode(req) + + u, _ := nameToRaftURL(peer.Name) + debugf("Send SnapshotRecovery to %s [Last Term: %d, LastIndex %d]", u, + req.LastTerm, req.LastIndex) + + resp, _, err := t.Post(fmt.Sprintf("%s/snapshotRecovery", u), &b) + + if err != nil { + debugf("Cannot send SendSnapshotRecoveryRequest to %s : %s", u, err) + } + + if resp != nil { + defer resp.Body.Close() + aersp = &raft.SnapshotRecoveryResponse{} + + if err = json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { + return aersp + } + } + + return aersp +} + +// Send server side POST request +func (t *transporter) Post(urlStr string, body io.Reader) (*http.Response, *http.Request, error) { + req, _ := http.NewRequest("POST", urlStr, body) + resp, err := t.client.Do(req) + return resp, req, err +} + +// Send server side GET request +func (t *transporter) Get(urlStr string) (*http.Response, *http.Request, error) { + req, _ := http.NewRequest("GET", urlStr, nil) + resp, err := t.client.Do(req) + return resp, req, err +} + +// Cancel the on fly HTTP transaction when timeout happens. +func (t *transporter) CancelWhenTimeout(req *http.Request) { + go func() { + time.Sleep(ElectionTimeout) + t.transport.CancelRequest(req) + }() +} diff --git a/transporter_test.go b/server/transporter_test.go similarity index 98% rename from transporter_test.go rename to server/transporter_test.go index 3d9655dbdd9..e83ea3400c4 100644 --- a/transporter_test.go +++ b/server/transporter_test.go @@ -1,4 +1,4 @@ -package main +package server import ( "crypto/tls" diff --git a/server/util.go b/server/util.go new file mode 100644 index 00000000000..bae347cd34a --- /dev/null +++ b/server/util.go @@ -0,0 +1,17 @@ +package server + +import ( + "fmt" + "net/http" + "github.com/coreos/etcd/log" +) + +func decodeJsonRequest(req *http.Request, data interface{}) error { + decoder := json.NewDecoder(req.Body) + if err := decoder.Decode(&data); err != nil && err != io.EOF { + log.Warnf("Malformed json request: %v", err) + return fmt.Errorf("Malformed json request: %v", err) + } + return nil +} + diff --git a/server/v2/handlers.go b/server/v2/handlers.go index 0d294de7d4a..067e5fd3801 100644 --- a/server/v2/handlers.go +++ b/server/v2/handlers.go @@ -12,10 +12,6 @@ import ( "github.com/coreos/go-raft" ) -//------------------------------------------------------------------- -// Handlers to handle etcd-store related request via etcd url -//------------------------------------------------------------------- - func NewEtcdMuxer() *http.ServeMux { // external commands router := mux.NewRouter() diff --git a/version.go b/server/version.go similarity index 71% rename from version.go rename to server/version.go index 20d31c91661..c9e2a48c095 100644 --- a/version.go +++ b/server/version.go @@ -1,8 +1,8 @@ -package main +package server -const version = "v2" +const Version = "v2" // TODO: The release version (generated from the git tag) will be the raft // protocol version for now. When things settle down we will fix it like the // client API above. -const raftVersion = releaseVersion +const PeerVersion = releaseVersion diff --git a/snapshot.go b/snapshot.go deleted file mode 100644 index 1b2bc728c95..00000000000 --- a/snapshot.go +++ /dev/null @@ -1,36 +0,0 @@ -package main - -import ( - "time" -) - -// basic conf. -// TODO: find a good policy to do snapshot -type snapshotConf struct { - // Etcd will check if snapshot is need every checkingInterval - checkingInterval time.Duration - // The number of writes when the last snapshot happened - lastWrites uint64 - // If the incremental number of writes since the last snapshot - // exceeds the write Threshold, etcd will do a snapshot - writesThr uint64 -} - -var snapConf *snapshotConf - -func (r *raftServer) newSnapshotConf() *snapshotConf { - // check snapshot every 3 seconds and the threshold is 20K - return &snapshotConf{time.Second * 3, 0, 20 * 1000} -} - -func (r *raftServer) monitorSnapshot() { - for { - time.Sleep(snapConf.checkingInterval) - //currentWrites := etcdStore.TotalWrites() - snapConf.lastWrites - currentWrites := 0 - if uint64(currentWrites) > snapConf.writesThr { - r.TakeSnapshot() - snapConf.lastWrites = 0 - } - } -} diff --git a/command/create_command.go b/store/create_command.go similarity index 86% rename from command/create_command.go rename to store/create_command.go index 6dd2c5abaf0..0263347a655 100644 --- a/command/create_command.go +++ b/store/create_command.go @@ -1,12 +1,15 @@ -package command +package store import ( "github.com/coreos/etcd/log" - "github.com/coreos/etcd/store" "github.com/coreos/go-raft" "time" ) +func init() { + raft.RegisterCommand(&CreateCommand{}) +} + // Create command type CreateCommand struct { Key string `json:"key"` @@ -23,7 +26,7 @@ func (c *CreateCommand) CommandName() string { // Create node func (c *CreateCommand) Apply(server *raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(*store.Store) + s, _ := server.StateMachine().(*Store) e, err := s.Create(c.Key, c.Value, c.IncrementalSuffix, c.Force, c.ExpireTime, server.CommitIndex(), server.Term()) diff --git a/command/delete_command.go b/store/delete_command.go similarity index 83% rename from command/delete_command.go rename to store/delete_command.go index a0d03c99d4c..3ac48bc3570 100644 --- a/command/delete_command.go +++ b/store/delete_command.go @@ -1,11 +1,14 @@ -package command +package store import ( "github.com/coreos/etcd/log" - "github.com/coreos/etcd/store" "github.com/coreos/go-raft" ) +func init() { + raft.RegisterCommand(&DeleteCommand{}) +} + // The DeleteCommand removes a key from the Store. type DeleteCommand struct { Key string `json:"key"` @@ -19,7 +22,7 @@ func (c *DeleteCommand) CommandName() string { // Delete the key func (c *DeleteCommand) Apply(server *raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(*store.Store) + s, _ := server.StateMachine().(*Store) e, err := s.Delete(c.Key, c.Recursive, server.CommitIndex(), server.Term()) diff --git a/store/store.go b/store/store.go index ddf0762805e..348aea02e4d 100644 --- a/store/store.go +++ b/store/store.go @@ -401,16 +401,6 @@ func (s *Store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) { return n, nil } -// Returns the number of machines in the cluster. -func (s *Store) MachineCount() int { - e, err := s.Get("/_etcd/machines", false, false, 0, 0) - if err != nil { - return 0 - } - - return len(e.KVPairs) -} - // Save function saves the static state of the store system. // Save function will not be able to save the state of watchers. // Save function will not save the parent field of the node. Or there will diff --git a/command/test_and_set_command.go b/store/test_and_set_command.go similarity index 88% rename from command/test_and_set_command.go rename to store/test_and_set_command.go index 4d723e22130..cf4167d1cdd 100644 --- a/command/test_and_set_command.go +++ b/store/test_and_set_command.go @@ -1,13 +1,16 @@ -package command +package store import ( "time" "github.com/coreos/etcd/log" - "github.com/coreos/etcd/store" "github.com/coreos/go-raft" ) +func init() { + raft.RegisterCommand(&TestAndSetCommand{}) +} + // The TestAndSetCommand performs a conditional update on a key in the store. type TestAndSetCommand struct { Key string `json:"key"` @@ -24,7 +27,7 @@ func (c *TestAndSetCommand) CommandName() string { // Set the key-value pair if the current value of the key equals to the given prevValue func (c *TestAndSetCommand) Apply(server *raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(*store.Store) + s, _ := server.StateMachine().(*Store) e, err := s.TestAndSet(c.Key, c.PrevValue, c.PrevIndex, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) diff --git a/command/update_command.go b/store/update_command.go similarity index 85% rename from command/update_command.go rename to store/update_command.go index 245e3c1c7fc..694be9844f3 100644 --- a/command/update_command.go +++ b/store/update_command.go @@ -1,13 +1,16 @@ -package command +package store import ( "time" "github.com/coreos/etcd/log" - "github.com/coreos/etcd/store" "github.com/coreos/go-raft" ) +func init() { + raft.RegisterCommand(&UpdateCommand{}) +} + // The UpdateCommand updates the value of a key in the Store. type UpdateCommand struct { Key string `json:"key"` @@ -22,7 +25,7 @@ func (c *UpdateCommand) CommandName() string { // Update node func (c *UpdateCommand) Apply(server *raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(*store.Store) + s, _ := server.StateMachine().(*Store) e, err := s.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) diff --git a/transporter.go b/transporter.go deleted file mode 100644 index 19a09c73e7c..00000000000 --- a/transporter.go +++ /dev/null @@ -1,233 +0,0 @@ -package main - -import ( - "bytes" - "crypto/tls" - "encoding/json" - "fmt" - "io" - "net" - "net/http" - "time" - - "github.com/coreos/go-raft" -) - -// Timeout for setup internal raft http connection -// This should not exceed 3 * RTT -var dailTimeout = 3 * HeartbeatTimeout - -// Timeout for setup internal raft http connection + receive response header -// This should not exceed 3 * RTT + RTT -var responseHeaderTimeout = 4 * HeartbeatTimeout - -// Timeout for receiving the response body from the server -// This should not exceed election timeout -var tranTimeout = ElectionTimeout - -// Transporter layer for communication between raft nodes -type transporter struct { - client *http.Client - transport *http.Transport - raftServer *raftServer -} - -// Create transporter using by raft server -// Create http or https transporter based on -// whether the user give the server cert and key -func newTransporter(scheme string, tlsConf tls.Config, raftServer *raftServer) *transporter { - t := transporter{} - - tr := &http.Transport{ - Dial: dialWithTimeout, - ResponseHeaderTimeout: responseHeaderTimeout, - } - - if scheme == "https" { - tr.TLSClientConfig = &tlsConf - tr.DisableCompression = true - } - - t.client = &http.Client{Transport: tr} - t.transport = tr - t.raftServer = raftServer - - return &t -} - -// Dial with timeout -func dialWithTimeout(network, addr string) (net.Conn, error) { - return net.DialTimeout(network, addr, dailTimeout) -} - -// Sends AppendEntries RPCs to a peer when the server is the leader. -func (t *transporter) SendAppendEntriesRequest(server *raft.Server, peer *raft.Peer, req *raft.AppendEntriesRequest) *raft.AppendEntriesResponse { - var aersp *raft.AppendEntriesResponse - var b bytes.Buffer - - json.NewEncoder(&b).Encode(req) - - size := b.Len() - - t.raftServer.serverStats.SendAppendReq(size) - - u, _ := nameToRaftURL(peer.Name) - - debugf("Send LogEntries to %s ", u) - - thisFollowerStats, ok := t.raftServer.followersStats.Followers[peer.Name] - - if !ok { //this is the first time this follower has been seen - thisFollowerStats = &raftFollowerStats{} - thisFollowerStats.Latency.Minimum = 1 << 63 - t.raftServer.followersStats.Followers[peer.Name] = thisFollowerStats - } - - start := time.Now() - - resp, httpRequest, err := t.Post(fmt.Sprintf("%s/log/append", u), &b) - - end := time.Now() - - if err != nil { - debugf("Cannot send AppendEntriesRequest to %s: %s", u, err) - if ok { - thisFollowerStats.Fail() - } - } else { - if ok { - thisFollowerStats.Succ(end.Sub(start)) - } - } - - if resp != nil { - defer resp.Body.Close() - - t.CancelWhenTimeout(httpRequest) - - aersp = &raft.AppendEntriesResponse{} - if err := json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { - return aersp - } - - } - - return aersp -} - -// Sends RequestVote RPCs to a peer when the server is the candidate. -func (t *transporter) SendVoteRequest(server *raft.Server, peer *raft.Peer, req *raft.RequestVoteRequest) *raft.RequestVoteResponse { - var rvrsp *raft.RequestVoteResponse - var b bytes.Buffer - json.NewEncoder(&b).Encode(req) - - u, _ := nameToRaftURL(peer.Name) - debugf("Send Vote to %s", u) - - resp, httpRequest, err := t.Post(fmt.Sprintf("%s/vote", u), &b) - - if err != nil { - debugf("Cannot send VoteRequest to %s : %s", u, err) - } - - if resp != nil { - defer resp.Body.Close() - - t.CancelWhenTimeout(httpRequest) - - rvrsp := &raft.RequestVoteResponse{} - if err := json.NewDecoder(resp.Body).Decode(&rvrsp); err == nil || err == io.EOF { - return rvrsp - } - - } - return rvrsp -} - -// Sends SnapshotRequest RPCs to a peer when the server is the candidate. -func (t *transporter) SendSnapshotRequest(server *raft.Server, peer *raft.Peer, req *raft.SnapshotRequest) *raft.SnapshotResponse { - var aersp *raft.SnapshotResponse - var b bytes.Buffer - json.NewEncoder(&b).Encode(req) - - u, _ := nameToRaftURL(peer.Name) - debugf("Send Snapshot to %s [Last Term: %d, LastIndex %d]", u, - req.LastTerm, req.LastIndex) - - resp, httpRequest, err := t.Post(fmt.Sprintf("%s/snapshot", u), &b) - - if err != nil { - debugf("Cannot send SendSnapshotRequest to %s : %s", u, err) - } - - if resp != nil { - defer resp.Body.Close() - - t.CancelWhenTimeout(httpRequest) - - aersp = &raft.SnapshotResponse{} - if err = json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { - - return aersp - } - } - - return aersp -} - -// Sends SnapshotRecoveryRequest RPCs to a peer when the server is the candidate. -func (t *transporter) SendSnapshotRecoveryRequest(server *raft.Server, peer *raft.Peer, req *raft.SnapshotRecoveryRequest) *raft.SnapshotRecoveryResponse { - var aersp *raft.SnapshotRecoveryResponse - var b bytes.Buffer - json.NewEncoder(&b).Encode(req) - - u, _ := nameToRaftURL(peer.Name) - debugf("Send SnapshotRecovery to %s [Last Term: %d, LastIndex %d]", u, - req.LastTerm, req.LastIndex) - - resp, _, err := t.Post(fmt.Sprintf("%s/snapshotRecovery", u), &b) - - if err != nil { - debugf("Cannot send SendSnapshotRecoveryRequest to %s : %s", u, err) - } - - if resp != nil { - defer resp.Body.Close() - aersp = &raft.SnapshotRecoveryResponse{} - - if err = json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { - return aersp - } - } - - return aersp -} - -// Send server side POST request -func (t *transporter) Post(urlStr string, body io.Reader) (*http.Response, *http.Request, error) { - - req, _ := http.NewRequest("POST", urlStr, body) - - resp, err := t.client.Do(req) - - return resp, req, err - -} - -// Send server side GET request -func (t *transporter) Get(urlStr string) (*http.Response, *http.Request, error) { - - req, _ := http.NewRequest("GET", urlStr, nil) - - resp, err := t.client.Do(req) - - return resp, req, err -} - -// Cancel the on fly HTTP transaction when timeout happens -func (t *transporter) CancelWhenTimeout(req *http.Request) { - go func() { - time.Sleep(ElectionTimeout) - t.transport.CancelRequest(req) - }() -} diff --git a/util.go b/util.go index fade01ec64e..e9b534085cf 100644 --- a/util.go +++ b/util.go @@ -88,33 +88,11 @@ func (r *raftServer) dispatch(c Command, w http.ResponseWriter, req *http.Reques func redirect(hostname string, w http.ResponseWriter, req *http.Request) { path := req.URL.Path - url := hostname + path - debugf("Redirect to %s", url) - http.Redirect(w, req, url, http.StatusTemporaryRedirect) } -func decodeJsonRequest(req *http.Request, data interface{}) error { - decoder := json.NewDecoder(req.Body) - if err := decoder.Decode(&data); err != nil && err != io.EOF { - warnf("Malformed json request: %v", err) - return fmt.Errorf("Malformed json request: %v", err) - } - return nil -} - -func encodeJsonResponse(w http.ResponseWriter, status int, data interface{}) { - w.Header().Set("Content-Type", "application/json") - w.WriteHeader(status) - - if data != nil { - encoder := json.NewEncoder(w) - encoder.Encode(data) - } -} - // sanitizeURL will cleanup a host string in the format hostname:port and // attach a schema. func sanitizeURL(host string, defaultScheme string) string { From bb9401544a2efc8f5c6717c3db2319fff1f39440 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sat, 12 Oct 2013 13:35:23 -0600 Subject: [PATCH 075/247] Intermediate commit. --- etcd.go | 3 ++ server/join_command.go | 6 +-- server/peer_server.go | 114 ++++++++++++++++++++++++++++++++++----- server/registry.go | 2 +- server/remove_command.go | 1 - server/server.go | 10 +++- server/util.go | 7 +++ util.go | 51 ------------------ 8 files changed, 122 insertions(+), 72 deletions(-) diff --git a/etcd.go b/etcd.go index 82bc09649fb..d307bf3061d 100644 --- a/etcd.go +++ b/etcd.go @@ -185,12 +185,15 @@ func main() { // Create peer server. ps := NewPeerServer(info.Name, dirPath, info.RaftURL, info.RaftListenHost, &raftTLSConfig, &info.RaftTLS, registry) ps.MaxClusterSize = maxClusterSize + ps.RetryTimes = retryTimes s := server.New(info.Name, info.EtcdURL, info.EtcdListenHost, &etcdTLSConfig, &info.EtcdTLS, r) if err := e.AllowOrigins(cors); err != nil { panic(err) } + ps.SetServer(server) + ps.ListenAndServe(snapshot) s.ListenAndServe() } diff --git a/server/join_command.go b/server/join_command.go index f338f3d390c..71caaa0ce59 100644 --- a/server/join_command.go +++ b/server/join_command.go @@ -19,16 +19,14 @@ type JoinCommand struct { Name string `json:"name"` RaftURL string `json:"raftURL"` EtcdURL string `json:"etcdURL"` - MaxClusterSize int `json:"maxClusterSize"` } -func NewJoinCommand(version, name, raftUrl, etcdUrl string, maxClusterSize int) *JoinCommand { +func NewJoinCommand(version, name, raftUrl, etcdUrl string) *JoinCommand { return &JoinCommand{ RaftVersion: version, Name: name, RaftURL: raftUrl, EtcdURL: etcdUrl, - MaxClusterSize: maxClusterSize, } } @@ -51,7 +49,7 @@ func (c *JoinCommand) Apply(server *raft.Server) (interface{}, error) { } // Check machine number in the cluster - if ps.registry.Count() == c.MaxClusterSize { + if ps.registry.Count() == ps.MaxClusterSize { log.Debug("Reject join request from ", c.Name) return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMoreMachine, "", server.CommitIndex(), server.Term()) } diff --git a/server/peer_server.go b/server/peer_server.go index 0c24c1521cb..de9316a4bc8 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -19,6 +19,7 @@ import ( type PeerServer struct { *raft.Server + server Server joinIndex uint64 name string url string @@ -31,6 +32,7 @@ type PeerServer struct { store *store.Store snapConf *snapshotConf MaxClusterSize int + RetryTimes int } // TODO: find a good policy to do snapshot @@ -140,6 +142,16 @@ func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) { } +// Retrieves the underlying Raft server. +func (s *PeerServer) RaftServer() *raft.Server { + return s.Server +} + +// Associates the client server with the peer server. +func (s *PeerServer) SetServer(server Server) { + s.server = server +} + // Get all the current logs func (s *PeerServer) GetLogHttpHandler(w http.ResponseWriter, req *http.Request) { log.Debugf("[recv] GET %s/log", s.url) @@ -223,7 +235,7 @@ func (s *PeerServer) SnapshotRecoveryHttpHandler(w http.ResponseWriter, req *htt func (s *PeerServer) EtcdURLHttpHandler(w http.ResponseWriter, req *http.Request) { log.Debugf("[recv] Get %s/etcdURL/ ", s.url) w.WriteHeader(http.StatusOK) - w.Write([]byte(argInfo.EtcdURL)) + w.Write([]byte(s.server.URL())) } // Response to the join request @@ -271,13 +283,13 @@ func (s *PeerServer) RaftVersionHttpHandler(w http.ResponseWriter, req *http.Req } func (s *PeerServer) dispatchRaftCommand(c raft.Command, w http.ResponseWriter, req *http.Request) error { - return s.dispatch(c, w, req, nameToRaftURL) + return s.dispatch(c, w, req) } func (s *PeerServer) startAsLeader() { // leader need to join self as a peer for { - _, err := s.Do(newJoinCommand(PeerVersion, s.Name(), s.url, e.url)) + _, err := s.Do(NewJoinCommand(PeerVersion, s.Name(), s.url, s.server.URL())) if err == nil { break } @@ -287,7 +299,7 @@ func (s *PeerServer) startAsLeader() { func (s *PeerServer) startAsFollower(cluster []string) { // start as a follower in a existing cluster - for i := 0; i < retryTimes; i++ { + for i := 0; i < s.RetryTimes; i++ { ok := s.joinCluster(cluster) if ok { return @@ -296,12 +308,12 @@ func (s *PeerServer) startAsFollower(cluster []string) { time.Sleep(time.Second * RetryInterval) } - fatalf("Cannot join the cluster via given machines after %x retries", retryTimes) + log.Fatalf("Cannot join the cluster via given machines after %x retries", s.RetryTimes) } // Start to listen and response raft command func (s *PeerServer) startTransport(scheme string, tlsConf tls.Config) { - infof("raft server [name %s, listen on %s, advertised url %s]", s.name, s.listenHost, s.url) + log.Infof("raft server [name %s, listen on %s, advertised url %s]", s.name, s.listenHost, s.url) raftMux := http.NewServeMux() @@ -324,9 +336,9 @@ func (s *PeerServer) startTransport(scheme string, tlsConf tls.Config) { raftMux.HandleFunc("/etcdURL", s.EtcdURLHttpHandler) if scheme == "http" { - fatal(server.ListenAndServe()) + log.Fatal(server.ListenAndServe()) } else { - fatal(server.ListenAndServeTLS(s.tlsInfo.CertFile, s.tlsInfo.KeyFile)) + log.Fatal(server.ListenAndServeTLS(s.tlsInfo.CertFile, s.tlsInfo.KeyFile)) } } @@ -336,11 +348,9 @@ func (s *PeerServer) startTransport(scheme string, tlsConf tls.Config) { // version clusters. func getVersion(t *transporter, versionURL url.URL) (string, error) { resp, req, err := t.Get(versionURL.String()) - if err != nil { return "", err } - defer resp.Body.Close() t.CancelWhenTimeout(req) @@ -363,7 +373,7 @@ func (s *PeerServer) joinCluster(cluster []string) bool { } else { if _, ok := err.(etcdErr.Error); ok { - fatal(err) + log.Fatal(err) } log.Debugf("cannot join to cluster via machine %s %s", machine, err) @@ -392,7 +402,7 @@ func (s *PeerServer) joinByMachine(server *raft.Server, machine string, scheme s return fmt.Errorf("Unable to join: internal version mismatch, entire cluster must be running identical versions of etcd") } - json.NewEncoder(&b).Encode(newJoinCommand(PeerVersion, server.Name(), server.url, e.url)) + json.NewEncoder(&b).Encode(NewJoinCommand(PeerVersion, server.Name(), s.url, s.server.URL())) joinURL := url.URL{Host: machine, Scheme: scheme, Path: "/join"} @@ -419,7 +429,7 @@ func (s *PeerServer) joinByMachine(server *raft.Server, machine string, scheme s address := resp.Header.Get("Location") log.Debugf("Send Join Request to %s", address) - json.NewEncoder(&b).Encode(newJoinCommand(PeerVersion, server.Name(), server.url, e.url)) + json.NewEncoder(&b).Encode(newJoinCommand(PeerVersion, server.Name(), s.url, s.server.URL())) resp, req, err = t.Post(address, &b) @@ -472,3 +482,81 @@ func (s *PeerServer) monitorSnapshot() { } } } + +func (s *PeerServer) dispatch(c raft.Command, w http.ResponseWriter, req *http.Request) error { + if r.State() == raft.Leader { + if response, err := r.Do(c); err != nil { + return err + } else { + if response == nil { + return etcdErr.NewError(300, "Empty response from raft", store.UndefIndex, store.UndefTerm) + } + + event, ok := response.(*store.Event) + if ok { + bytes, err := json.Marshal(event) + if err != nil { + fmt.Println(err) + } + + w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) + w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term)) + w.WriteHeader(http.StatusOK) + w.Write(bytes) + + return nil + } + + bytes, _ := response.([]byte) + w.WriteHeader(http.StatusOK) + w.Write(bytes) + + return nil + } + + } else { + leader := r.Leader() + // current no leader + if leader == "" { + return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) + } + url, _ := s.registry.PeerURL(leader) + + redirect(url, w, req) + + return nil + } +} + + +type errorHandler func(http.ResponseWriter, *http.Request) error + +// addCorsHeader parses the request Origin header and loops through the user +// provided allowed origins and sets the Access-Control-Allow-Origin header if +// there is a match. +func addCorsHeader(w http.ResponseWriter, r *http.Request) { + val, ok := corsList["*"] + if val && ok { + w.Header().Add("Access-Control-Allow-Origin", "*") + return + } + + requestOrigin := r.Header.Get("Origin") + val, ok = corsList[requestOrigin] + if val && ok { + w.Header().Add("Access-Control-Allow-Origin", requestOrigin) + return + } +} + +func (fn errorHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { + addCorsHeader(w, r) + if e := fn(w, r); e != nil { + if etcdErr, ok := e.(*etcdErr.Error); ok { + debug("Return error: ", (*etcdErr).Error()) + etcdErr.Write(w) + } else { + http.Error(w, e.Error(), http.StatusInternalServerError) + } + } +} diff --git a/server/registry.go b/server/registry.go index cd7078d8120..468b79179ac 100644 --- a/server/registry.go +++ b/server/registry.go @@ -159,7 +159,7 @@ func (r *Registry) load(name string) { } // Create node. - r.nodes[name] := &node{ + r.nodes[name] = &node{ url: m["etcd"][0], peerURL: m["raft"][0], peerVersion: m["raftVersion"][0], diff --git a/server/remove_command.go b/server/remove_command.go index 43ff17d248d..5e5feab5c50 100644 --- a/server/remove_command.go +++ b/server/remove_command.go @@ -2,7 +2,6 @@ package server import ( "encoding/binary" - "path" "github.com/coreos/etcd/store" "github.com/coreos/go-raft" diff --git a/server/server.go b/server/server.go index 5835e1a121a..f283f627ddc 100644 --- a/server/server.go +++ b/server/server.go @@ -12,6 +12,7 @@ import ( type Server interface { CommitIndex() uint64 Term() uint64 + URL() string Dispatch(raft.Command, http.ResponseWriter, *http.Request) } @@ -49,12 +50,17 @@ func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsI // The current Raft committed index. func (s *server) CommitIndex() uint64 { - return c.raftServer.CommitIndex() + return s.raftServer.CommitIndex() } // The current Raft term. func (s *server) Term() uint64 { - return c.raftServer.Term() + return s.raftServer.Term() +} + +// The server URL. +func (s *server) URL() string { + return s.url } func (s *server) installV1() { diff --git a/server/util.go b/server/util.go index bae347cd34a..0154e22bd2a 100644 --- a/server/util.go +++ b/server/util.go @@ -15,3 +15,10 @@ func decodeJsonRequest(req *http.Request, data interface{}) error { return nil } +func redirect(hostname string, w http.ResponseWriter, req *http.Request) { + path := req.URL.Path + url := hostname + path + debugf("Redirect to %s", url) + http.Redirect(w, req, url, http.StatusTemporaryRedirect) +} + diff --git a/util.go b/util.go index e9b534085cf..c97d7f77ddf 100644 --- a/util.go +++ b/util.go @@ -41,57 +41,6 @@ func durationToExpireTime(strDuration string) (time.Time, error) { // HTTP Utilities //-------------------------------------- -func (r *raftServer) dispatch(c Command, w http.ResponseWriter, req *http.Request, toURL func(name string) (string, bool)) error { - if r.State() == raft.Leader { - if response, err := r.Do(c); err != nil { - return err - } else { - if response == nil { - return etcdErr.NewError(300, "Empty response from raft", store.UndefIndex, store.UndefTerm) - } - - event, ok := response.(*store.Event) - if ok { - bytes, err := json.Marshal(event) - if err != nil { - fmt.Println(err) - } - - w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) - w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term)) - w.WriteHeader(http.StatusOK) - w.Write(bytes) - - return nil - } - - bytes, _ := response.([]byte) - w.WriteHeader(http.StatusOK) - w.Write(bytes) - - return nil - } - - } else { - leader := r.Leader() - // current no leader - if leader == "" { - return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) - } - url, _ := toURL(leader) - - redirect(url, w, req) - - return nil - } -} - -func redirect(hostname string, w http.ResponseWriter, req *http.Request) { - path := req.URL.Path - url := hostname + path - debugf("Redirect to %s", url) - http.Redirect(w, req, url, http.StatusTemporaryRedirect) -} // sanitizeURL will cleanup a host string in the format hostname:port and // attach a schema. From 8670e1b7aad03be94f5c22c21b80e179cdc0d99d Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sat, 12 Oct 2013 15:56:43 -0600 Subject: [PATCH 076/247] Refactored. --- config.go | 19 ++-- etcd.go | 40 ++++----- etcd_test.go | 7 +- machines.go | 34 ------- scripts/release-version | 2 +- server/join_command.go | 2 - server/peer_server.go | 82 +++++++---------- server/registry.go | 21 +++-- server/remove_command.go | 8 +- server/server.go | 113 ++++++++++++++++------- server/stats_queue.go | 1 + server/timeout.go | 4 + server/transporter.go | 37 ++++---- server/util.go | 5 +- server/v1/delete_key_handler.go | 8 +- server/v1/dispatch.go | 42 --------- server/v1/get_key_handler.go | 9 +- server/v1/set_key_handler.go | 19 ++-- server/v1/v1.go | 45 ++-------- server/v1/watch_key_handler.go | 12 +-- server/v2/handlers.go | 36 +++++++- store/event.go | 153 +++++++------------------------- store/event_history.go | 112 +++++++++++++++++++++++ store/event_queue.go | 26 ++++++ store/ttl.go | 21 +++++ util.go | 72 ++------------- 26 files changed, 453 insertions(+), 477 deletions(-) delete mode 100644 machines.go delete mode 100644 server/v1/dispatch.go create mode 100644 store/event_history.go create mode 100644 store/event_queue.go create mode 100644 store/ttl.go diff --git a/config.go b/config.go index 9580bcaf3b7..d1549c7becb 100644 --- a/config.go +++ b/config.go @@ -8,6 +8,9 @@ import ( "io/ioutil" "os" "path/filepath" + + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/server" ) //-------------------------------------- @@ -30,7 +33,7 @@ func getInfo(path string) *Info { os.Remove(confPath) os.RemoveAll(snapshotPath) } else if info := readInfo(infoPath); info != nil { - infof("Found node configuration in '%s'. Ignoring flags", infoPath) + log.Infof("Found node configuration in '%s'. Ignoring flags", infoPath) return info } @@ -41,10 +44,10 @@ func getInfo(path string) *Info { content, _ := json.MarshalIndent(info, "", " ") content = []byte(string(content) + "\n") if err := ioutil.WriteFile(infoPath, content, 0644); err != nil { - fatalf("Unable to write info to file: %v", err) + log.Fatalf("Unable to write info to file: %v", err) } - infof("Wrote node configuration to '%s'", infoPath) + log.Infof("Wrote node configuration to '%s'", infoPath) return info } @@ -57,7 +60,7 @@ func readInfo(path string) *Info { if os.IsNotExist(err) { return nil } - fatal(err) + log.Fatal(err) } defer file.Close() @@ -65,19 +68,19 @@ func readInfo(path string) *Info { content, err := ioutil.ReadAll(file) if err != nil { - fatalf("Unable to read info: %v", err) + log.Fatalf("Unable to read info: %v", err) return nil } if err = json.Unmarshal(content, &info); err != nil { - fatalf("Unable to parse info: %v", err) + log.Fatalf("Unable to parse info: %v", err) return nil } return info } -func tlsConfigFromInfo(info TLSInfo) (t TLSConfig, ok bool) { +func tlsConfigFromInfo(info server.TLSInfo) (t server.TLSConfig, ok bool) { var keyFile, certFile, CAFile string var tlsCert tls.Certificate var err error @@ -101,7 +104,7 @@ func tlsConfigFromInfo(info TLSInfo) (t TLSConfig, ok bool) { tlsCert, err = tls.LoadX509KeyPair(certFile, keyFile) if err != nil { - fatal(err) + log.Fatal(err) } t.Scheme = "https" diff --git a/etcd.go b/etcd.go index d307bf3061d..a224fa15486 100644 --- a/etcd.go +++ b/etcd.go @@ -1,12 +1,10 @@ package main import ( - "crypto/tls" "flag" "io/ioutil" "os" "strings" - "time" "github.com/coreos/etcd/log" "github.com/coreos/etcd/server" @@ -101,18 +99,10 @@ type Info struct { RaftListenHost string `json:"raftListenHost"` EtcdListenHost string `json:"etcdListenHost"` - RaftTLS TLSInfo `json:"raftTLS"` - EtcdTLS TLSInfo `json:"etcdTLS"` + RaftTLS server.TLSInfo `json:"raftTLS"` + EtcdTLS server.TLSInfo `json:"etcdTLS"` } -//------------------------------------------------------------------------------ -// -// Variables -// -//------------------------------------------------------------------------------ - -var etcdStore *store.Store - //------------------------------------------------------------------------------ // // Functions @@ -131,7 +121,7 @@ func main() { } if veryVerbose { - verbose = true + log.Verbose = true raft.SetLogLevel(raft.Debug) } @@ -140,7 +130,7 @@ func main() { } else if machinesFile != "" { b, err := ioutil.ReadFile(machinesFile) if err != nil { - fatalf("Unable to read the given machines file: %s", err) + log.Fatalf("Unable to read the given machines file: %s", err) } cluster = strings.Split(string(b), ",") } @@ -148,17 +138,17 @@ func main() { // Check TLS arguments raftTLSConfig, ok := tlsConfigFromInfo(argInfo.RaftTLS) if !ok { - fatal("Please specify cert and key file or cert and key file and CAFile or none of the three") + log.Fatal("Please specify cert and key file or cert and key file and CAFile or none of the three") } etcdTLSConfig, ok := tlsConfigFromInfo(argInfo.EtcdTLS) if !ok { - fatal("Please specify cert and key file or cert and key file and CAFile or none of the three") + log.Fatal("Please specify cert and key file or cert and key file and CAFile or none of the three") } argInfo.Name = strings.TrimSpace(argInfo.Name) if argInfo.Name == "" { - fatal("ERROR: server name required. e.g. '-n=server_name'") + log.Fatal("ERROR: server name required. e.g. '-n=server_name'") } // Check host name arguments @@ -171,29 +161,29 @@ func main() { // Read server info from file or grab it from user. if err := os.MkdirAll(dirPath, 0744); err != nil { - fatalf("Unable to create path: %s", err) + log.Fatalf("Unable to create path: %s", err) } info := getInfo(dirPath) // Create etcd key-value store - etcdStore = store.New() + store := store.New() // Create a shared node registry. - registry := server.NewRegistry() + registry := server.NewRegistry(store) // Create peer server. - ps := NewPeerServer(info.Name, dirPath, info.RaftURL, info.RaftListenHost, &raftTLSConfig, &info.RaftTLS, registry) + ps := server.NewPeerServer(info.Name, dirPath, info.RaftURL, info.RaftListenHost, &raftTLSConfig, &info.RaftTLS, registry, store) ps.MaxClusterSize = maxClusterSize ps.RetryTimes = retryTimes - s := server.New(info.Name, info.EtcdURL, info.EtcdListenHost, &etcdTLSConfig, &info.EtcdTLS, r) - if err := e.AllowOrigins(cors); err != nil { + s := server.New(info.Name, info.EtcdURL, info.EtcdListenHost, &etcdTLSConfig, &info.EtcdTLS, ps.Server, registry, store) + if err := s.AllowOrigins(cors); err != nil { panic(err) } - ps.SetServer(server) + ps.SetServer(s) - ps.ListenAndServe(snapshot) + ps.ListenAndServe(snapshot, cluster) s.ListenAndServe() } diff --git a/etcd_test.go b/etcd_test.go index a62119e3599..71222a24649 100644 --- a/etcd_test.go +++ b/etcd_test.go @@ -12,6 +12,7 @@ import ( "testing" "time" + "github.com/coreos/etcd/server" "github.com/coreos/etcd/test" "github.com/coreos/go-etcd/etcd" ) @@ -398,8 +399,8 @@ func TestKillLeader(t *testing.T) { totalTime += take avgTime := totalTime / (time.Duration)(i+1) - fmt.Println("Leader election time is ", take, "with election timeout", ElectionTimeout) - fmt.Println("Leader election time average is", avgTime, "with election timeout", ElectionTimeout) + fmt.Println("Leader election time is ", take, "with election timeout", server.ElectionTimeout) + fmt.Println("Leader election time average is", avgTime, "with election timeout", server.ElectionTimeout) etcds[num], err = os.StartProcess("etcd", argGroup[num], procAttr) } stop <- true @@ -456,7 +457,7 @@ func TestKillRandom(t *testing.T) { etcds[num].Wait() } - time.Sleep(ElectionTimeout) + time.Sleep(server.ElectionTimeout) <-leaderChan diff --git a/machines.go b/machines.go deleted file mode 100644 index 1da25ed757d..00000000000 --- a/machines.go +++ /dev/null @@ -1,34 +0,0 @@ -package main - -// getMachines gets the current machines in the cluster -func (r *raftServer) getMachines(toURL func(string) (string, bool)) []string { - peers := r.Peers() - - machines := make([]string, len(peers)+1) - - leader, ok := toURL(r.Leader()) - self, _ := toURL(r.Name()) - i := 1 - - if ok { - machines[0] = leader - if leader != self { - machines[1] = self - i = 2 - } - } else { - machines[0] = self - } - - // Add all peers to the slice - for peerName, _ := range peers { - if machine, ok := toURL(peerName); ok { - // do not add leader twice - if machine != leader { - machines[i] = machine - i++ - } - } - } - return machines -} diff --git a/scripts/release-version b/scripts/release-version index b1f68e4f7e6..c1fdaba0768 100755 --- a/scripts/release-version +++ b/scripts/release-version @@ -3,6 +3,6 @@ VER=$(git describe --tags HEAD) cat < s.snapConf.writesThr { - r.TakeSnapshot() + s.TakeSnapshot() s.snapConf.lastWrites = 0 } } } func (s *PeerServer) dispatch(c raft.Command, w http.ResponseWriter, req *http.Request) error { - if r.State() == raft.Leader { - if response, err := r.Do(c); err != nil { + if s.State() == raft.Leader { + if response, err := s.Do(c); err != nil { return err } else { if response == nil { @@ -515,7 +533,7 @@ func (s *PeerServer) dispatch(c raft.Command, w http.ResponseWriter, req *http.R } } else { - leader := r.Leader() + leader := s.Leader() // current no leader if leader == "" { return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) @@ -528,35 +546,3 @@ func (s *PeerServer) dispatch(c raft.Command, w http.ResponseWriter, req *http.R } } - -type errorHandler func(http.ResponseWriter, *http.Request) error - -// addCorsHeader parses the request Origin header and loops through the user -// provided allowed origins and sets the Access-Control-Allow-Origin header if -// there is a match. -func addCorsHeader(w http.ResponseWriter, r *http.Request) { - val, ok := corsList["*"] - if val && ok { - w.Header().Add("Access-Control-Allow-Origin", "*") - return - } - - requestOrigin := r.Header.Get("Origin") - val, ok = corsList[requestOrigin] - if val && ok { - w.Header().Add("Access-Control-Allow-Origin", requestOrigin) - return - } -} - -func (fn errorHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { - addCorsHeader(w, r) - if e := fn(w, r); e != nil { - if etcdErr, ok := e.(*etcdErr.Error); ok { - debug("Return error: ", (*etcdErr).Error()) - etcdErr.Write(w) - } else { - http.Error(w, e.Error(), http.StatusInternalServerError) - } - } -} diff --git a/server/registry.go b/server/registry.go index 468b79179ac..25ff936b7e9 100644 --- a/server/registry.go +++ b/server/registry.go @@ -1,6 +1,9 @@ package server import ( + "fmt" + "net/url" + "path" "sync" "github.com/coreos/etcd/store" @@ -48,13 +51,13 @@ func (r *Registry) Unregister(name string, commitIndex uint64, term uint64) erro defer r.Unlock() // Remove the key from the store. - _, err := s.Delete(path.Join(RegistryKey, name), false, commitIndex, term) + _, err := r.store.Delete(path.Join(RegistryKey, name), false, commitIndex, term) return err } // Returns the number of nodes in the cluster. func (r *Registry) Count() int { - e, err := s.Get(RegistryKey, false, false, 0, 0) + e, err := r.store.Get(RegistryKey, false, false, 0, 0) if err != nil { return 0 } @@ -86,7 +89,7 @@ func (r *Registry) URLs() []string { defer r.Unlock() // Retrieve a list of all nodes. - e, err := s.Get(RegistryKey, false, false, 0, 0) + e, err := r.store.Get(RegistryKey, false, false, 0, 0) if err != nil { return make([]string, 0) } @@ -94,7 +97,9 @@ func (r *Registry) URLs() []string { // Lookup the URL for each one. urls := make([]string, 0) for _, pair := range e.KVPairs { - urls = append(urls, r.url(pair.Key)) + if url, ok := r.url(pair.Key); ok { + urls = append(urls, url) + } } return urls @@ -126,7 +131,7 @@ func (r *Registry) PeerURLs() []string { defer r.Unlock() // Retrieve a list of all nodes. - e, err := s.Get(RegistryKey, false, false, 0, 0) + e, err := r.store.Get(RegistryKey, false, false, 0, 0) if err != nil { return make([]string, 0) } @@ -134,7 +139,9 @@ func (r *Registry) PeerURLs() []string { // Lookup the URL for each one. urls := make([]string, 0) for _, pair := range e.KVPairs { - urls = append(urls, r.peerURL(pair.Key)) + if url, ok := r.peerURL(pair.Key); ok { + urls = append(urls, url) + } } return urls @@ -147,7 +154,7 @@ func (r *Registry) load(name string) { } // Retrieve from store. - e, err := etcdStore.Get(path.Join(RegistryKey, name), false, false, 0, 0) + e, err := r.store.Get(path.Join(RegistryKey, name), false, false, 0, 0) if err != nil { return } diff --git a/server/remove_command.go b/server/remove_command.go index 5e5feab5c50..51140922b14 100644 --- a/server/remove_command.go +++ b/server/remove_command.go @@ -2,8 +2,9 @@ package server import ( "encoding/binary" + "os" - "github.com/coreos/etcd/store" + "github.com/coreos/etcd/log" "github.com/coreos/go-raft" ) @@ -23,7 +24,6 @@ func (c *RemoveCommand) CommandName() string { // Remove a server from the cluster func (c *RemoveCommand) Apply(server *raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(*store.Store) ps, _ := server.Context().(*PeerServer) // Remove node from the shared registry. @@ -50,11 +50,11 @@ func (c *RemoveCommand) Apply(server *raft.Server) (interface{}, error) { // start. It is sure that this node received a new remove // command and need to be removed if server.CommitIndex() > ps.joinIndex && ps.joinIndex != 0 { - debugf("server [%s] is removed", server.Name()) + log.Debugf("server [%s] is removed", server.Name()) os.Exit(0) } else { // else ignore remove - debugf("ignore previous remove command.") + log.Debugf("ignore previous remove command.") } } diff --git a/server/server.go b/server/server.go index f283f627ddc..586502378bb 100644 --- a/server/server.go +++ b/server/server.go @@ -1,25 +1,26 @@ package server import ( + "fmt" + "encoding/json" "net/http" "net/url" + "strings" + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/server/v1" + "github.com/coreos/etcd/store" "github.com/coreos/go-raft" "github.com/gorilla/mux" ) -// The Server provides an HTTP interface to the underlying store. -type Server interface { - CommitIndex() uint64 - Term() uint64 - URL() string - Dispatch(raft.Command, http.ResponseWriter, *http.Request) -} - // This is the default implementation of the Server interface. -type server struct { +type Server struct { http.Server raftServer *raft.Server + registry *Registry + store *store.Store name string url string tlsConf *TLSConfig @@ -28,14 +29,15 @@ type server struct { } // Creates a new Server. -func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, raftServer *raft.Server) *Server { - s := &server{ +func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, raftServer *raft.Server, registry *Registry, store *store.Store) *Server { + s := &Server{ Server: http.Server{ Handler: mux.NewRouter(), TLSConfig: &tlsConf.Server, Addr: listenHost, }, - name: name, + store: store, + registry: registry, url: urlStr, tlsConf: tlsConf, tlsInfo: tlsInfo, @@ -49,72 +51,117 @@ func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsI } // The current Raft committed index. -func (s *server) CommitIndex() uint64 { +func (s *Server) CommitIndex() uint64 { return s.raftServer.CommitIndex() } // The current Raft term. -func (s *server) Term() uint64 { +func (s *Server) Term() uint64 { return s.raftServer.Term() } // The server URL. -func (s *server) URL() string { +func (s *Server) URL() string { return s.url } -func (s *server) installV1() { - s.handleFunc("/v1/keys/{key:.*}", v1.GetKeyHandler).Methods("GET") - s.handleFunc("/v1/keys/{key:.*}", v1.SetKeyHandler).Methods("POST", "PUT") - s.handleFunc("/v1/keys/{key:.*}", v1.DeleteKeyHandler).Methods("DELETE") +// Returns a reference to the Store. +func (s *Server) Store() *store.Store { + return s.store +} + +func (s *Server) installV1() { + s.handleFuncV1("/v1/keys/{key:.*}", v1.GetKeyHandler).Methods("GET") + s.handleFuncV1("/v1/keys/{key:.*}", v1.SetKeyHandler).Methods("POST", "PUT") + s.handleFuncV1("/v1/keys/{key:.*}", v1.DeleteKeyHandler).Methods("DELETE") - s.handleFunc("/v1/watch/{key:.*}", v1.WatchKeyHandler).Methods("GET", "POST") + s.handleFuncV1("/v1/watch/{key:.*}", v1.WatchKeyHandler).Methods("GET", "POST") +} + +// Adds a v1 server handler to the router. +func (s *Server) handleFuncV1(path string, f func(http.ResponseWriter, *http.Request, v1.Server) error) *mux.Route { + return s.handleFunc(path, func(w http.ResponseWriter, req *http.Request, s *Server) error { + return f(w, req, s) + }) } // Adds a server handler to the router. -func (s *server) handleFunc(path string, f func(http.ResponseWriter, *http.Request, Server) error) *mux.Route { +func (s *Server) handleFunc(path string, f func(http.ResponseWriter, *http.Request, *Server) error) *mux.Route { r := s.Handler.(*mux.Router) // Wrap the standard HandleFunc interface to pass in the server reference. return r.HandleFunc(path, func(w http.ResponseWriter, req *http.Request) { // Log request. - debugf("[recv] %s %s [%s]", req.Method, s.url, req.URL.Path, req.RemoteAddr) + log.Debugf("[recv] %s %s [%s]", req.Method, s.url, req.URL.Path, req.RemoteAddr) // Write CORS header. if s.OriginAllowed("*") { w.Header().Add("Access-Control-Allow-Origin", "*") - } else if s.OriginAllowed(r.Header.Get("Origin")) { - w.Header().Add("Access-Control-Allow-Origin", r.Header.Get("Origin")) + } else if origin := req.Header.Get("Origin"); s.OriginAllowed(origin) { + w.Header().Add("Access-Control-Allow-Origin", origin) } // Execute handler function and return error if necessary. if err := f(w, req, s); err != nil { if etcdErr, ok := err.(*etcdErr.Error); ok { - debug("Return error: ", (*etcdErr).Error()) + log.Debug("Return error: ", (*etcdErr).Error()) etcdErr.Write(w) } else { - http.Error(w, e.Error(), http.StatusInternalServerError) + http.Error(w, err.Error(), http.StatusInternalServerError) } } }) } // Start to listen and response etcd client command -func (s *server) ListenAndServe() { - infof("etcd server [name %s, listen on %s, advertised url %s]", s.name, s.Server.Addr, s.url) +func (s *Server) ListenAndServe() { + log.Infof("etcd server [name %s, listen on %s, advertised url %s]", s.name, s.Server.Addr, s.url) if s.tlsConf.Scheme == "http" { - fatal(s.Server.ListenAndServe()) + log.Fatal(s.Server.ListenAndServe()) } else { - fatal(s.Server.ListenAndServeTLS(s.tlsInfo.CertFile, s.tlsInfo.KeyFile)) + log.Fatal(s.Server.ListenAndServeTLS(s.tlsInfo.CertFile, s.tlsInfo.KeyFile)) + } +} + +func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Request) error { + if s.raftServer.State() == raft.Leader { + event, err := s.raftServer.Do(c) + if err != nil { + return err + } + + if event == nil { + return etcdErr.NewError(300, "Empty result from raft", store.UndefIndex, store.UndefTerm) + } + + response := event.(*store.Event).Response() + b, _ := json.Marshal(response) + w.WriteHeader(http.StatusOK) + w.Write(b) + + return nil + + } else { + leader := s.raftServer.Leader() + + // No leader available. + if leader == "" { + return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) + } + + url, _ := s.registry.PeerURL(leader) + redirect(url, w, req) + + return nil } } // Sets a comma-delimited list of origins that are allowed. -func (s *server) AllowOrigins(origins string) error { +func (s *Server) AllowOrigins(origins string) error { // Construct a lookup of all origins. m := make(map[string]bool) - for _, v := range strings.Split(cors, ",") { + for _, v := range strings.Split(origins, ",") { if v != "*" { if _, err := url.Parse(v); err != nil { return fmt.Errorf("Invalid CORS origin: %s", err) @@ -128,6 +175,6 @@ func (s *server) AllowOrigins(origins string) error { } // Determines whether the server will allow a given CORS origin. -func (s *server) OriginAllowed(origin string) { +func (s *Server) OriginAllowed(origin string) bool { return s.corsOrigins["*"] || s.corsOrigins[origin] } diff --git a/server/stats_queue.go b/server/stats_queue.go index 5927eed15ec..8ee689c0a2a 100644 --- a/server/stats_queue.go +++ b/server/stats_queue.go @@ -2,6 +2,7 @@ package server import ( "sync" + "time" ) const ( diff --git a/server/timeout.go b/server/timeout.go index 35b49b63035..321e77b38f7 100644 --- a/server/timeout.go +++ b/server/timeout.go @@ -1,5 +1,9 @@ package server +import ( + "time" +) + const ( // The amount of time to elapse without a heartbeat before becoming a candidate. ElectionTimeout = 200 * time.Millisecond diff --git a/server/transporter.go b/server/transporter.go index 83f0f07fc2c..7397d45342b 100644 --- a/server/transporter.go +++ b/server/transporter.go @@ -10,6 +10,7 @@ import ( "net/http" "time" + "github.com/coreos/etcd/log" "github.com/coreos/go-raft" ) @@ -29,13 +30,13 @@ var tranTimeout = ElectionTimeout type transporter struct { client *http.Client transport *http.Transport - raftServer *raftServer + peerServer *PeerServer } // Create transporter using by raft server // Create http or https transporter based on // whether the user give the server cert and key -func newTransporter(scheme string, tlsConf tls.Config, raftServer *raftServer) *transporter { +func newTransporter(scheme string, tlsConf tls.Config, peerServer *PeerServer) *transporter { t := transporter{} tr := &http.Transport{ @@ -50,7 +51,7 @@ func newTransporter(scheme string, tlsConf tls.Config, raftServer *raftServer) * t.client = &http.Client{Transport: tr} t.transport = tr - t.raftServer = raftServer + t.peerServer = peerServer return &t } @@ -69,18 +70,18 @@ func (t *transporter) SendAppendEntriesRequest(server *raft.Server, peer *raft.P size := b.Len() - t.raftServer.serverStats.SendAppendReq(size) + t.peerServer.serverStats.SendAppendReq(size) - u, _ := nameToRaftURL(peer.Name) + u, _ := t.peerServer.registry.PeerURL(peer.Name) - debugf("Send LogEntries to %s ", u) + log.Debugf("Send LogEntries to %s ", u) - thisFollowerStats, ok := t.raftServer.followersStats.Followers[peer.Name] + thisFollowerStats, ok := t.peerServer.followersStats.Followers[peer.Name] if !ok { //this is the first time this follower has been seen thisFollowerStats = &raftFollowerStats{} thisFollowerStats.Latency.Minimum = 1 << 63 - t.raftServer.followersStats.Followers[peer.Name] = thisFollowerStats + t.peerServer.followersStats.Followers[peer.Name] = thisFollowerStats } start := time.Now() @@ -90,7 +91,7 @@ func (t *transporter) SendAppendEntriesRequest(server *raft.Server, peer *raft.P end := time.Now() if err != nil { - debugf("Cannot send AppendEntriesRequest to %s: %s", u, err) + log.Debugf("Cannot send AppendEntriesRequest to %s: %s", u, err) if ok { thisFollowerStats.Fail() } @@ -121,13 +122,13 @@ func (t *transporter) SendVoteRequest(server *raft.Server, peer *raft.Peer, req var b bytes.Buffer json.NewEncoder(&b).Encode(req) - u, _ := nameToRaftURL(peer.Name) - debugf("Send Vote to %s", u) + u, _ := t.peerServer.registry.PeerURL(peer.Name) + log.Debugf("Send Vote to %s", u) resp, httpRequest, err := t.Post(fmt.Sprintf("%s/vote", u), &b) if err != nil { - debugf("Cannot send VoteRequest to %s : %s", u, err) + log.Debugf("Cannot send VoteRequest to %s : %s", u, err) } if resp != nil { @@ -150,14 +151,14 @@ func (t *transporter) SendSnapshotRequest(server *raft.Server, peer *raft.Peer, var b bytes.Buffer json.NewEncoder(&b).Encode(req) - u, _ := nameToRaftURL(peer.Name) - debugf("Send Snapshot to %s [Last Term: %d, LastIndex %d]", u, + u, _ := t.peerServer.registry.PeerURL(peer.Name) + log.Debugf("Send Snapshot to %s [Last Term: %d, LastIndex %d]", u, req.LastTerm, req.LastIndex) resp, httpRequest, err := t.Post(fmt.Sprintf("%s/snapshot", u), &b) if err != nil { - debugf("Cannot send SendSnapshotRequest to %s : %s", u, err) + log.Debugf("Cannot send SendSnapshotRequest to %s : %s", u, err) } if resp != nil { @@ -181,14 +182,14 @@ func (t *transporter) SendSnapshotRecoveryRequest(server *raft.Server, peer *raf var b bytes.Buffer json.NewEncoder(&b).Encode(req) - u, _ := nameToRaftURL(peer.Name) - debugf("Send SnapshotRecovery to %s [Last Term: %d, LastIndex %d]", u, + u, _ := t.peerServer.registry.PeerURL(peer.Name) + log.Debugf("Send SnapshotRecovery to %s [Last Term: %d, LastIndex %d]", u, req.LastTerm, req.LastIndex) resp, _, err := t.Post(fmt.Sprintf("%s/snapshotRecovery", u), &b) if err != nil { - debugf("Cannot send SendSnapshotRecoveryRequest to %s : %s", u, err) + log.Debugf("Cannot send SendSnapshotRecoveryRequest to %s : %s", u, err) } if resp != nil { diff --git a/server/util.go b/server/util.go index 0154e22bd2a..95d93c17950 100644 --- a/server/util.go +++ b/server/util.go @@ -1,8 +1,11 @@ package server import ( + "encoding/json" "fmt" + "io" "net/http" + "github.com/coreos/etcd/log" ) @@ -18,7 +21,7 @@ func decodeJsonRequest(req *http.Request, data interface{}) error { func redirect(hostname string, w http.ResponseWriter, req *http.Request) { path := req.URL.Path url := hostname + path - debugf("Redirect to %s", url) + log.Debugf("Redirect to %s", url) http.Redirect(w, req, url, http.StatusTemporaryRedirect) } diff --git a/server/v1/delete_key_handler.go b/server/v1/delete_key_handler.go index c9d695fdc3f..a657e9f3d77 100644 --- a/server/v1/delete_key_handler.go +++ b/server/v1/delete_key_handler.go @@ -1,15 +1,15 @@ package v1 import ( - "encoding/json" - "github.com/coreos/etcd/store" "net/http" + "github.com/coreos/etcd/store" + "github.com/gorilla/mux" ) // Removes a key from the store. func DeleteKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) key := "/" + vars["key"] - command := &DeleteCommand{Key: key} - return s.Dispatch(command, w, req) + c := &store.DeleteCommand{Key: key} + return s.Dispatch(c, w, req) } diff --git a/server/v1/dispatch.go b/server/v1/dispatch.go deleted file mode 100644 index 7dd13861c99..00000000000 --- a/server/v1/dispatch.go +++ /dev/null @@ -1,42 +0,0 @@ -package v1 - -// Dispatch the command to leader. -func dispatchCommand(c Command, w http.ResponseWriter, req *http.Request, s *server.Server) error { - return dispatch(c, w, req, s, nameToEtcdURL) -} - -// Dispatches a command to a given URL. -func dispatch(c Command, w http.ResponseWriter, req *http.Request, s *server.Server, toURL func(name string) (string, bool)) error { - r := s.raftServer - if r.State() == raft.Leader { - event, err := r.Do(c) - if err != nil { - return err - } - - if event == nil { - return etcdErr.NewError(300, "Empty result from raft", store.UndefIndex, store.UndefTerm) - } - - event, _ := event.(*store.Event) - response := eventToResponse(event) - b, _ := json.Marshal(response) - w.WriteHeader(http.StatusOK) - w.Write(b) - - return nil - - } else { - leader := r.Leader() - - // No leader available. - if leader == "" { - return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) - } - - url, _ := toURL(leader) - redirect(url, w, req) - - return nil - } -} diff --git a/server/v1/get_key_handler.go b/server/v1/get_key_handler.go index 6469313a341..53558e142ff 100644 --- a/server/v1/get_key_handler.go +++ b/server/v1/get_key_handler.go @@ -2,12 +2,13 @@ package v1 import ( "encoding/json" - "github.com/coreos/etcd/store" "net/http" + + "github.com/gorilla/mux" ) // Retrieves the value for a given key. -func GetKeyHandler(w http.ResponseWriter, req *http.Request, e *etcdServer) error { +func GetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) key := "/" + vars["key"] @@ -18,9 +19,7 @@ func GetKeyHandler(w http.ResponseWriter, req *http.Request, e *etcdServer) erro } // Convert event to a response and write to client. - event, _ := event.(*store.Event) - response := eventToResponse(event) - b, _ := json.Marshal(response) + b, _ := json.Marshal(event.Response()) w.WriteHeader(http.StatusOK) w.Write(b) diff --git a/server/v1/set_key_handler.go b/server/v1/set_key_handler.go index ad629e48f26..03b6d7f9bef 100644 --- a/server/v1/set_key_handler.go +++ b/server/v1/set_key_handler.go @@ -1,9 +1,12 @@ package v1 import ( - "encoding/json" - "github.com/coreos/etcd/store" "net/http" + + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" + "github.com/gorilla/mux" ) // Sets the value for a given key. @@ -16,19 +19,19 @@ func SetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { // Parse non-blank value. value := req.Form.Get("value") if len(value) == 0 { - return error.NewError(200, "Set", store.UndefIndex, store.UndefTerm) + return etcdErr.NewError(200, "Set", store.UndefIndex, store.UndefTerm) } // Convert time-to-live to an expiration time. - expireTime, err := durationToExpireTime(req.Form.Get("ttl")) + expireTime, err := store.TTL(req.Form.Get("ttl")) if err != nil { return etcdErr.NewError(202, "Set", store.UndefIndex, store.UndefTerm) } // If the "prevValue" is specified then test-and-set. Otherwise create a new key. - var c command.Command + var c raft.Command if prevValueArr, ok := req.Form["prevValue"]; ok && len(prevValueArr) > 0 { - c = &TestAndSetCommand{ + c = &store.TestAndSetCommand{ Key: key, Value: value, PrevValue: prevValueArr[0], @@ -36,7 +39,7 @@ func SetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { } } else { - c = &CreateCommand{ + c = &store.CreateCommand{ Key: key, Value: value, ExpireTime: expireTime, @@ -44,5 +47,5 @@ func SetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { } } - return s.Dispatch(command, w, req) + return s.Dispatch(c, w, req) } diff --git a/server/v1/v1.go b/server/v1/v1.go index 7f9f80a0e54..586a08e67e9 100644 --- a/server/v1/v1.go +++ b/server/v1/v1.go @@ -1,50 +1,15 @@ package v1 import ( - "github.com/coreos/etcd/server" - "github.com/gorilla/mux" + "net/http" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" ) // The Server interface provides all the methods required for the v1 API. type Server interface { CommitIndex() uint64 Term() uint64 - Dispatch(http.ResponseWriter, *http.Request, Command) -} - -// Converts an event object into a response object. -func eventToResponse(event *store.Event) interface{} { - if !event.Dir { - response := &store.Response{ - Action: event.Action, - Key: event.Key, - Value: event.Value, - PrevValue: event.PrevValue, - Index: event.Index, - TTL: event.TTL, - Expiration: event.Expiration, - } - - if response.Action == store.Create || response.Action == store.Update { - response.Action = "set" - if response.PrevValue == "" { - response.NewKey = true - } - } - - return response - } else { - responses := make([]*store.Response, len(event.KVPairs)) - - for i, kv := range event.KVPairs { - responses[i] = &store.Response{ - Action: event.Action, - Key: kv.Key, - Value: kv.Value, - Dir: kv.Dir, - Index: event.Index, - } - } - return responses - } + Store() *store.Store + Dispatch(raft.Command, http.ResponseWriter, *http.Request) error } diff --git a/server/v1/watch_key_handler.go b/server/v1/watch_key_handler.go index 8ad103b8be4..e8db56c30b4 100644 --- a/server/v1/watch_key_handler.go +++ b/server/v1/watch_key_handler.go @@ -2,8 +2,12 @@ package v1 import ( "encoding/json" - "github.com/coreos/etcd/store" "net/http" + "strconv" + + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/store" + "github.com/gorilla/mux" ) // Watches a given key prefix for changes. @@ -13,7 +17,7 @@ func WatchKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { key := "/" + vars["key"] // Create a command to watch from a given index (default 0). - sinceIndex := 0 + var sinceIndex uint64 = 0 if req.Method == "POST" { sinceIndex, err = strconv.ParseUint(string(req.FormValue("index")), 10, 64) if err != nil { @@ -28,9 +32,7 @@ func WatchKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { } event := <-c - event, _ := event.(*store.Event) - response := eventToResponse(event) - b, _ := json.Marshal(response) + b, _ := json.Marshal(event.Response()) w.WriteHeader(http.StatusOK) w.Write(b) diff --git a/server/v2/handlers.go b/server/v2/handlers.go index 067e5fd3801..7fbd7e1d7e5 100644 --- a/server/v2/handlers.go +++ b/server/v2/handlers.go @@ -95,7 +95,7 @@ func (e *etcdServer) CreateHttpHandler(w http.ResponseWriter, req *http.Request) value := req.FormValue("value") - expireTime, err := durationToExpireTime(req.FormValue("ttl")) + expireTime, err := store.TTL(req.FormValue("ttl")) if err != nil { return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create", store.UndefIndex, store.UndefTerm) @@ -124,7 +124,7 @@ func (e *etcdServer) UpdateHttpHandler(w http.ResponseWriter, req *http.Request) value := req.Form.Get("value") - expireTime, err := durationToExpireTime(req.Form.Get("ttl")) + expireTime, err := store.TTL(req.Form.Get("ttl")) if err != nil { return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", store.UndefIndex, store.UndefTerm) @@ -344,6 +344,16 @@ func (e *etcdServer) GetHttpHandler(w http.ResponseWriter, req *http.Request) er } +func getNodePath(urlPath string) string { + pathPrefixLen := len("/" + version + "/keys") + return urlPath[pathPrefixLen:] +} + + +//-------------------------------------- +// Testing +//-------------------------------------- + // TestHandler func TestHttpHandler(w http.ResponseWriter, req *http.Request) { testType := req.URL.Path[len("/test/"):] @@ -358,3 +368,25 @@ func TestHttpHandler(w http.ResponseWriter, req *http.Request) { w.WriteHeader(http.StatusBadRequest) } + +func directSet() { + c := make(chan bool, 1000) + for i := 0; i < 1000; i++ { + go send(c) + } + + for i := 0; i < 1000; i++ { + <-c + } +} + +func send(c chan bool) { + for i := 0; i < 10; i++ { + command := &UpdateCommand{} + command.Key = "foo" + command.Value = "bar" + command.ExpireTime = time.Unix(0, 0) + //r.Do(command) + } + c <- true +} diff --git a/store/event.go b/store/event.go index 15866d7450d..48420186560 100644 --- a/store/event.go +++ b/store/event.go @@ -1,12 +1,7 @@ package store import ( - "fmt" - "strings" - "sync" "time" - - etcdErr "github.com/coreos/etcd/error" ) const ( @@ -46,129 +41,41 @@ func newEvent(action string, key string, index uint64, term uint64) *Event { } } -type eventQueue struct { - Events []*Event - Size int - Front int - Capacity int -} - -func (eq *eventQueue) back() int { - return (eq.Front + eq.Size - 1 + eq.Capacity) % eq.Capacity -} - -func (eq *eventQueue) insert(e *Event) { - index := (eq.back() + 1) % eq.Capacity - - eq.Events[index] = e - - if eq.Size == eq.Capacity { //dequeue - eq.Front = (index + 1) % eq.Capacity - } else { - eq.Size++ - } - -} - -type EventHistory struct { - Queue eventQueue - StartIndex uint64 - LastIndex uint64 - LastTerm uint64 - DupCnt uint64 // help to compute the watching point with duplicated indexes in the queue - rwl sync.RWMutex -} - -func newEventHistory(capacity int) *EventHistory { - return &EventHistory{ - Queue: eventQueue{ - Capacity: capacity, - Events: make([]*Event, capacity), - }, - } -} - -// addEvent function adds event into the eventHistory -func (eh *EventHistory) addEvent(e *Event) *Event { - eh.rwl.Lock() - defer eh.rwl.Unlock() - - var duped uint64 - - if e.Index == UndefIndex { - e.Index = eh.LastIndex - e.Term = eh.LastTerm - duped = 1 - } - - eh.Queue.insert(e) - - eh.LastIndex = e.Index - eh.LastTerm = e.Term - eh.DupCnt += duped - - eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index - - return e -} - -// scan function is enumerating events from the index in history and -// stops till the first point where the key has identified prefix -func (eh *EventHistory) scan(prefix string, index uint64) (*Event, *etcdErr.Error) { - eh.rwl.RLock() - defer eh.rwl.RUnlock() - - start := index - eh.StartIndex - - // the index should locate after the event history's StartIndex - if start < 0 { - return nil, - etcdErr.NewError(etcdErr.EcodeEventIndexCleared, - fmt.Sprintf("the requested history has been cleared [%v/%v]", - eh.StartIndex, index), UndefIndex, UndefTerm) - } - - // the index should locate before the size of the queue minus the duplicate count - if start >= (uint64(eh.Queue.Size) - eh.DupCnt) { // future index - return nil, nil - } - - i := int((start + uint64(eh.Queue.Front)) % uint64(eh.Queue.Capacity)) - - for { - e := eh.Queue.Events[i] - if strings.HasPrefix(e.Key, prefix) && index <= e.Index { // make sure we bypass the smaller one - return e, nil +// Converts an event object into a response object. +func (event *Event) Response() interface{} { + if !event.Dir { + response := &Response{ + Action: event.Action, + Key: event.Key, + Value: event.Value, + PrevValue: event.PrevValue, + Index: event.Index, + TTL: event.TTL, + Expiration: event.Expiration, } - i = (i + 1) % eh.Queue.Capacity + if response.Action == Create || response.Action == Update { + response.Action = "set" + if response.PrevValue == "" { + response.NewKey = true + } + } - if i == eh.Queue.back() { // find nothing, return and watch from current index - return nil, nil + return response + } else { + responses := make([]*Response, len(event.KVPairs)) + + for i, kv := range event.KVPairs { + responses[i] = &Response{ + Action: event.Action, + Key: kv.Key, + Value: kv.Value, + Dir: kv.Dir, + Index: event.Index, + } } + return responses } } -// clone will be protected by a stop-world lock -// do not need to obtain internal lock -func (eh *EventHistory) clone() *EventHistory { - clonedQueue := eventQueue{ - Capacity: eh.Queue.Capacity, - Events: make([]*Event, eh.Queue.Capacity), - Size: eh.Queue.Size, - Front: eh.Queue.Front, - } - - for i, e := range eh.Queue.Events { - clonedQueue.Events[i] = e - } - return &EventHistory{ - StartIndex: eh.StartIndex, - Queue: clonedQueue, - LastIndex: eh.LastIndex, - LastTerm: eh.LastTerm, - DupCnt: eh.DupCnt, - } - -} diff --git a/store/event_history.go b/store/event_history.go new file mode 100644 index 00000000000..73db5d87671 --- /dev/null +++ b/store/event_history.go @@ -0,0 +1,112 @@ +package store + +import ( + "fmt" + "strings" + "sync" + + etcdErr "github.com/coreos/etcd/error" +) + +type EventHistory struct { + Queue eventQueue + StartIndex uint64 + LastIndex uint64 + LastTerm uint64 + DupCnt uint64 // help to compute the watching point with duplicated indexes in the queue + rwl sync.RWMutex +} + +func newEventHistory(capacity int) *EventHistory { + return &EventHistory{ + Queue: eventQueue{ + Capacity: capacity, + Events: make([]*Event, capacity), + }, + } +} + +// addEvent function adds event into the eventHistory +func (eh *EventHistory) addEvent(e *Event) *Event { + eh.rwl.Lock() + defer eh.rwl.Unlock() + + var duped uint64 + + if e.Index == UndefIndex { + e.Index = eh.LastIndex + e.Term = eh.LastTerm + duped = 1 + } + + eh.Queue.insert(e) + + eh.LastIndex = e.Index + eh.LastTerm = e.Term + eh.DupCnt += duped + + eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index + + return e +} + +// scan function is enumerating events from the index in history and +// stops till the first point where the key has identified prefix +func (eh *EventHistory) scan(prefix string, index uint64) (*Event, *etcdErr.Error) { + eh.rwl.RLock() + defer eh.rwl.RUnlock() + + start := index - eh.StartIndex + + // the index should locate after the event history's StartIndex + if start < 0 { + return nil, + etcdErr.NewError(etcdErr.EcodeEventIndexCleared, + fmt.Sprintf("the requested history has been cleared [%v/%v]", + eh.StartIndex, index), UndefIndex, UndefTerm) + } + + // the index should locate before the size of the queue minus the duplicate count + if start >= (uint64(eh.Queue.Size) - eh.DupCnt) { // future index + return nil, nil + } + + i := int((start + uint64(eh.Queue.Front)) % uint64(eh.Queue.Capacity)) + + for { + e := eh.Queue.Events[i] + if strings.HasPrefix(e.Key, prefix) && index <= e.Index { // make sure we bypass the smaller one + return e, nil + } + + i = (i + 1) % eh.Queue.Capacity + + if i == eh.Queue.back() { // find nothing, return and watch from current index + return nil, nil + } + } +} + +// clone will be protected by a stop-world lock +// do not need to obtain internal lock +func (eh *EventHistory) clone() *EventHistory { + clonedQueue := eventQueue{ + Capacity: eh.Queue.Capacity, + Events: make([]*Event, eh.Queue.Capacity), + Size: eh.Queue.Size, + Front: eh.Queue.Front, + } + + for i, e := range eh.Queue.Events { + clonedQueue.Events[i] = e + } + + return &EventHistory{ + StartIndex: eh.StartIndex, + Queue: clonedQueue, + LastIndex: eh.LastIndex, + LastTerm: eh.LastTerm, + DupCnt: eh.DupCnt, + } + +} diff --git a/store/event_queue.go b/store/event_queue.go new file mode 100644 index 00000000000..7c520ffe4dc --- /dev/null +++ b/store/event_queue.go @@ -0,0 +1,26 @@ +package store + + +type eventQueue struct { + Events []*Event + Size int + Front int + Capacity int +} + +func (eq *eventQueue) back() int { + return (eq.Front + eq.Size - 1 + eq.Capacity) % eq.Capacity +} + +func (eq *eventQueue) insert(e *Event) { + index := (eq.back() + 1) % eq.Capacity + + eq.Events[index] = e + + if eq.Size == eq.Capacity { //dequeue + eq.Front = (index + 1) % eq.Capacity + } else { + eq.Size++ + } + +} diff --git a/store/ttl.go b/store/ttl.go new file mode 100644 index 00000000000..c73d95f8c4e --- /dev/null +++ b/store/ttl.go @@ -0,0 +1,21 @@ +package store + +import ( + "strconv" + "time" +) + +// Convert string duration to time format +func TTL(duration string) (time.Time, error) { + if duration != "" { + duration, err := strconv.Atoi(duration) + if err != nil { + return Permanent, err + } + return time.Now().Add(time.Second * (time.Duration)(duration)), nil + + } else { + return Permanent, nil + } +} + diff --git a/util.go b/util.go index c97d7f77ddf..d519fc1aaca 100644 --- a/util.go +++ b/util.go @@ -1,42 +1,15 @@ package main import ( - "encoding/json" - "fmt" - "io" "net" - "net/http" "net/url" "os" "os/signal" "runtime/pprof" - "strconv" - "time" - etcdErr "github.com/coreos/etcd/error" - "github.com/coreos/etcd/store" - "github.com/coreos/go-raft" + "github.com/coreos/etcd/log" ) -//-------------------------------------- -// etcd http Helper -//-------------------------------------- - -// Convert string duration to time format -func durationToExpireTime(strDuration string) (time.Time, error) { - if strDuration != "" { - duration, err := strconv.Atoi(strDuration) - - if err != nil { - return store.Permanent, err - } - return time.Now().Add(time.Second * (time.Duration)(duration)), nil - - } else { - return store.Permanent, nil - } -} - //-------------------------------------- // HTTP Utilities //-------------------------------------- @@ -52,13 +25,13 @@ func sanitizeURL(host string, defaultScheme string) string { p, err := url.Parse(host) if err != nil { - fatal(err) + log.Fatal(err) } // Make sure the host is in Host:Port format _, _, err = net.SplitHostPort(host) if err != nil { - fatal(err) + log.Fatal(err) } p = &url.URL{Host: host, Scheme: defaultScheme} @@ -71,12 +44,12 @@ func sanitizeURL(host string, defaultScheme string) string { func sanitizeListenHost(listen string, advertised string) string { aurl, err := url.Parse(advertised) if err != nil { - fatal(err) + log.Fatal(err) } ahost, aport, err := net.SplitHostPort(aurl.Host) if err != nil { - fatal(err) + log.Fatal(err) } // If the listen host isn't set use the advertised host @@ -89,15 +62,10 @@ func sanitizeListenHost(listen string, advertised string) string { func check(err error) { if err != nil { - fatal(err) + log.Fatal(err) } } -func getNodePath(urlPath string) string { - pathPrefixLen := len("/" + version + "/keys") - return urlPath[pathPrefixLen:] -} - //-------------------------------------- // CPU profile //-------------------------------------- @@ -105,7 +73,7 @@ func runCPUProfile() { f, err := os.Create(cpuprofile) if err != nil { - fatal(err) + log.Fatal(err) } pprof.StartCPUProfile(f) @@ -113,34 +81,10 @@ func runCPUProfile() { signal.Notify(c, os.Interrupt) go func() { for sig := range c { - infof("captured %v, stopping profiler and exiting..", sig) + log.Infof("captured %v, stopping profiler and exiting..", sig) pprof.StopCPUProfile() os.Exit(1) } }() } -//-------------------------------------- -// Testing -//-------------------------------------- -func directSet() { - c := make(chan bool, 1000) - for i := 0; i < 1000; i++ { - go send(c) - } - - for i := 0; i < 1000; i++ { - <-c - } -} - -func send(c chan bool) { - for i := 0; i < 10; i++ { - command := &UpdateCommand{} - command.Key = "foo" - command.Value = "bar" - command.ExpireTime = time.Unix(0, 0) - //r.Do(command) - } - c <- true -} From 34b99ee3430c310dbe131fff2515979c78c6d57f Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sat, 12 Oct 2013 20:34:31 -0600 Subject: [PATCH 077/247] bump(github.com/gorilla/mux): b08c5fcf14d01cb0c20ddf70157f54d4a2c3a38a --- third_party/github.com/gorilla/mux/LICENSE | 27 + third_party/github.com/gorilla/mux/README.md | 6 + .../github.com/gorilla/mux/bench_test.go | 21 + third_party/github.com/gorilla/mux/doc.go | 199 +++++ third_party/github.com/gorilla/mux/mux.go | 339 ++++++++ .../github.com/gorilla/mux/mux_test.go | 755 +++++++++++++++++ .../github.com/gorilla/mux/old_test.go | 758 ++++++++++++++++++ third_party/github.com/gorilla/mux/regexp.go | 247 ++++++ third_party/github.com/gorilla/mux/route.go | 499 ++++++++++++ 9 files changed, 2851 insertions(+) create mode 100644 third_party/github.com/gorilla/mux/LICENSE create mode 100644 third_party/github.com/gorilla/mux/README.md create mode 100644 third_party/github.com/gorilla/mux/bench_test.go create mode 100644 third_party/github.com/gorilla/mux/doc.go create mode 100644 third_party/github.com/gorilla/mux/mux.go create mode 100644 third_party/github.com/gorilla/mux/mux_test.go create mode 100644 third_party/github.com/gorilla/mux/old_test.go create mode 100644 third_party/github.com/gorilla/mux/regexp.go create mode 100644 third_party/github.com/gorilla/mux/route.go diff --git a/third_party/github.com/gorilla/mux/LICENSE b/third_party/github.com/gorilla/mux/LICENSE new file mode 100644 index 00000000000..0e5fb872800 --- /dev/null +++ b/third_party/github.com/gorilla/mux/LICENSE @@ -0,0 +1,27 @@ +Copyright (c) 2012 Rodrigo Moraes. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/third_party/github.com/gorilla/mux/README.md b/third_party/github.com/gorilla/mux/README.md new file mode 100644 index 00000000000..f6db41ad81d --- /dev/null +++ b/third_party/github.com/gorilla/mux/README.md @@ -0,0 +1,6 @@ +mux +=== + +gorilla/mux is a powerful URL router and dispatcher. + +Read the full documentation here: http://www.gorillatoolkit.org/pkg/mux diff --git a/third_party/github.com/gorilla/mux/bench_test.go b/third_party/github.com/gorilla/mux/bench_test.go new file mode 100644 index 00000000000..c5f97b2b2a8 --- /dev/null +++ b/third_party/github.com/gorilla/mux/bench_test.go @@ -0,0 +1,21 @@ +// Copyright 2012 The Gorilla Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package mux + +import ( + "net/http" + "testing" +) + +func BenchmarkMux(b *testing.B) { + router := new(Router) + handler := func(w http.ResponseWriter, r *http.Request) {} + router.HandleFunc("/v1/{v1}", handler) + + request, _ := http.NewRequest("GET", "/v1/anything", nil) + for i := 0; i < b.N; i++ { + router.ServeHTTP(nil, request) + } +} diff --git a/third_party/github.com/gorilla/mux/doc.go b/third_party/github.com/gorilla/mux/doc.go new file mode 100644 index 00000000000..8ee5540a4f9 --- /dev/null +++ b/third_party/github.com/gorilla/mux/doc.go @@ -0,0 +1,199 @@ +// Copyright 2012 The Gorilla Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +/* +Package gorilla/mux implements a request router and dispatcher. + +The name mux stands for "HTTP request multiplexer". Like the standard +http.ServeMux, mux.Router matches incoming requests against a list of +registered routes and calls a handler for the route that matches the URL +or other conditions. The main features are: + + * Requests can be matched based on URL host, path, path prefix, schemes, + header and query values, HTTP methods or using custom matchers. + * URL hosts and paths can have variables with an optional regular + expression. + * Registered URLs can be built, or "reversed", which helps maintaining + references to resources. + * Routes can be used as subrouters: nested routes are only tested if the + parent route matches. This is useful to define groups of routes that + share common conditions like a host, a path prefix or other repeated + attributes. As a bonus, this optimizes request matching. + * It implements the http.Handler interface so it is compatible with the + standard http.ServeMux. + +Let's start registering a couple of URL paths and handlers: + + func main() { + r := mux.NewRouter() + r.HandleFunc("/", HomeHandler) + r.HandleFunc("/products", ProductsHandler) + r.HandleFunc("/articles", ArticlesHandler) + http.Handle("/", r) + } + +Here we register three routes mapping URL paths to handlers. This is +equivalent to how http.HandleFunc() works: if an incoming request URL matches +one of the paths, the corresponding handler is called passing +(http.ResponseWriter, *http.Request) as parameters. + +Paths can have variables. They are defined using the format {name} or +{name:pattern}. If a regular expression pattern is not defined, the matched +variable will be anything until the next slash. For example: + + r := mux.NewRouter() + r.HandleFunc("/products/{key}", ProductHandler) + r.HandleFunc("/articles/{category}/", ArticlesCategoryHandler) + r.HandleFunc("/articles/{category}/{id:[0-9]+}", ArticleHandler) + +The names are used to create a map of route variables which can be retrieved +calling mux.Vars(): + + vars := mux.Vars(request) + category := vars["category"] + +And this is all you need to know about the basic usage. More advanced options +are explained below. + +Routes can also be restricted to a domain or subdomain. Just define a host +pattern to be matched. They can also have variables: + + r := mux.NewRouter() + // Only matches if domain is "www.domain.com". + r.Host("www.domain.com") + // Matches a dynamic subdomain. + r.Host("{subdomain:[a-z]+}.domain.com") + +There are several other matchers that can be added. To match path prefixes: + + r.PathPrefix("/products/") + +...or HTTP methods: + + r.Methods("GET", "POST") + +...or URL schemes: + + r.Schemes("https") + +...or header values: + + r.Headers("X-Requested-With", "XMLHttpRequest") + +...or query values: + + r.Queries("key", "value") + +...or to use a custom matcher function: + + r.MatcherFunc(func(r *http.Request, rm *RouteMatch) bool { + return r.ProtoMajor == 0 + }) + +...and finally, it is possible to combine several matchers in a single route: + + r.HandleFunc("/products", ProductsHandler). + Host("www.domain.com"). + Methods("GET"). + Schemes("http") + +Setting the same matching conditions again and again can be boring, so we have +a way to group several routes that share the same requirements. +We call it "subrouting". + +For example, let's say we have several URLs that should only match when the +host is "www.domain.com". Create a route for that host and get a "subrouter" +from it: + + r := mux.NewRouter() + s := r.Host("www.domain.com").Subrouter() + +Then register routes in the subrouter: + + s.HandleFunc("/products/", ProductsHandler) + s.HandleFunc("/products/{key}", ProductHandler) + s.HandleFunc("/articles/{category}/{id:[0-9]+}"), ArticleHandler) + +The three URL paths we registered above will only be tested if the domain is +"www.domain.com", because the subrouter is tested first. This is not +only convenient, but also optimizes request matching. You can create +subrouters combining any attribute matchers accepted by a route. + +Subrouters can be used to create domain or path "namespaces": you define +subrouters in a central place and then parts of the app can register its +paths relatively to a given subrouter. + +There's one more thing about subroutes. When a subrouter has a path prefix, +the inner routes use it as base for their paths: + + r := mux.NewRouter() + s := r.PathPrefix("/products").Subrouter() + // "/products/" + s.HandleFunc("/", ProductsHandler) + // "/products/{key}/" + s.HandleFunc("/{key}/", ProductHandler) + // "/products/{key}/details" + s.HandleFunc("/{key}/details"), ProductDetailsHandler) + +Now let's see how to build registered URLs. + +Routes can be named. All routes that define a name can have their URLs built, +or "reversed". We define a name calling Name() on a route. For example: + + r := mux.NewRouter() + r.HandleFunc("/articles/{category}/{id:[0-9]+}", ArticleHandler). + Name("article") + +To build a URL, get the route and call the URL() method, passing a sequence of +key/value pairs for the route variables. For the previous route, we would do: + + url, err := r.Get("article").URL("category", "technology", "id", "42") + +...and the result will be a url.URL with the following path: + + "/articles/technology/42" + +This also works for host variables: + + r := mux.NewRouter() + r.Host("{subdomain}.domain.com"). + Path("/articles/{category}/{id:[0-9]+}"). + HandlerFunc(ArticleHandler). + Name("article") + + // url.String() will be "http://news.domain.com/articles/technology/42" + url, err := r.Get("article").URL("subdomain", "news", + "category", "technology", + "id", "42") + +All variables defined in the route are required, and their values must +conform to the corresponding patterns. These requirements guarantee that a +generated URL will always match a registered route -- the only exception is +for explicitly defined "build-only" routes which never match. + +There's also a way to build only the URL host or path for a route: +use the methods URLHost() or URLPath() instead. For the previous route, +we would do: + + // "http://news.domain.com/" + host, err := r.Get("article").URLHost("subdomain", "news") + + // "/articles/technology/42" + path, err := r.Get("article").URLPath("category", "technology", "id", "42") + +And if you use subrouters, host and path defined separately can be built +as well: + + r := mux.NewRouter() + s := r.Host("{subdomain}.domain.com").Subrouter() + s.Path("/articles/{category}/{id:[0-9]+}"). + HandlerFunc(ArticleHandler). + Name("article") + + // "http://news.domain.com/articles/technology/42" + url, err := r.Get("article").URL("subdomain", "news", + "category", "technology", + "id", "42") +*/ +package mux diff --git a/third_party/github.com/gorilla/mux/mux.go b/third_party/github.com/gorilla/mux/mux.go new file mode 100644 index 00000000000..ddc1acc63f0 --- /dev/null +++ b/third_party/github.com/gorilla/mux/mux.go @@ -0,0 +1,339 @@ +// Copyright 2012 The Gorilla Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package mux + +import ( + "fmt" + "net/http" + "path" + + "github.com/gorilla/context" +) + +// NewRouter returns a new router instance. +func NewRouter() *Router { + return &Router{namedRoutes: make(map[string]*Route), KeepContext: false} +} + +// Router registers routes to be matched and dispatches a handler. +// +// It implements the http.Handler interface, so it can be registered to serve +// requests: +// +// var router = mux.NewRouter() +// +// func main() { +// http.Handle("/", router) +// } +// +// Or, for Google App Engine, register it in a init() function: +// +// func init() { +// http.Handle("/", router) +// } +// +// This will send all incoming requests to the router. +type Router struct { + // Configurable Handler to be used when no route matches. + NotFoundHandler http.Handler + // Parent route, if this is a subrouter. + parent parentRoute + // Routes to be matched, in order. + routes []*Route + // Routes by name for URL building. + namedRoutes map[string]*Route + // See Router.StrictSlash(). This defines the flag for new routes. + strictSlash bool + // If true, do not clear the request context after handling the request + KeepContext bool +} + +// Match matches registered routes against the request. +func (r *Router) Match(req *http.Request, match *RouteMatch) bool { + for _, route := range r.routes { + if route.Match(req, match) { + return true + } + } + return false +} + +// ServeHTTP dispatches the handler registered in the matched route. +// +// When there is a match, the route variables can be retrieved calling +// mux.Vars(request). +func (r *Router) ServeHTTP(w http.ResponseWriter, req *http.Request) { + // Clean path to canonical form and redirect. + if p := cleanPath(req.URL.Path); p != req.URL.Path { + w.Header().Set("Location", p) + w.WriteHeader(http.StatusMovedPermanently) + return + } + var match RouteMatch + var handler http.Handler + if r.Match(req, &match) { + handler = match.Handler + setVars(req, match.Vars) + setCurrentRoute(req, match.Route) + } + if handler == nil { + if r.NotFoundHandler == nil { + r.NotFoundHandler = http.NotFoundHandler() + } + handler = r.NotFoundHandler + } + if !r.KeepContext { + defer context.Clear(req) + } + handler.ServeHTTP(w, req) +} + +// Get returns a route registered with the given name. +func (r *Router) Get(name string) *Route { + return r.getNamedRoutes()[name] +} + +// GetRoute returns a route registered with the given name. This method +// was renamed to Get() and remains here for backwards compatibility. +func (r *Router) GetRoute(name string) *Route { + return r.getNamedRoutes()[name] +} + +// StrictSlash defines the slash behavior for new routes. +// +// When true, if the route path is "/path/", accessing "/path" will redirect +// to the former and vice versa. +// +// Special case: when a route sets a path prefix, strict slash is +// automatically set to false for that route because the redirect behavior +// can't be determined for prefixes. +func (r *Router) StrictSlash(value bool) *Router { + r.strictSlash = value + return r +} + +// ---------------------------------------------------------------------------- +// parentRoute +// ---------------------------------------------------------------------------- + +// getNamedRoutes returns the map where named routes are registered. +func (r *Router) getNamedRoutes() map[string]*Route { + if r.namedRoutes == nil { + if r.parent != nil { + r.namedRoutes = r.parent.getNamedRoutes() + } else { + r.namedRoutes = make(map[string]*Route) + } + } + return r.namedRoutes +} + +// getRegexpGroup returns regexp definitions from the parent route, if any. +func (r *Router) getRegexpGroup() *routeRegexpGroup { + if r.parent != nil { + return r.parent.getRegexpGroup() + } + return nil +} + +// ---------------------------------------------------------------------------- +// Route factories +// ---------------------------------------------------------------------------- + +// NewRoute registers an empty route. +func (r *Router) NewRoute() *Route { + route := &Route{parent: r, strictSlash: r.strictSlash} + r.routes = append(r.routes, route) + return route +} + +// Handle registers a new route with a matcher for the URL path. +// See Route.Path() and Route.Handler(). +func (r *Router) Handle(path string, handler http.Handler) *Route { + return r.NewRoute().Path(path).Handler(handler) +} + +// HandleFunc registers a new route with a matcher for the URL path. +// See Route.Path() and Route.HandlerFunc(). +func (r *Router) HandleFunc(path string, f func(http.ResponseWriter, + *http.Request)) *Route { + return r.NewRoute().Path(path).HandlerFunc(f) +} + +// Headers registers a new route with a matcher for request header values. +// See Route.Headers(). +func (r *Router) Headers(pairs ...string) *Route { + return r.NewRoute().Headers(pairs...) +} + +// Host registers a new route with a matcher for the URL host. +// See Route.Host(). +func (r *Router) Host(tpl string) *Route { + return r.NewRoute().Host(tpl) +} + +// MatcherFunc registers a new route with a custom matcher function. +// See Route.MatcherFunc(). +func (r *Router) MatcherFunc(f MatcherFunc) *Route { + return r.NewRoute().MatcherFunc(f) +} + +// Methods registers a new route with a matcher for HTTP methods. +// See Route.Methods(). +func (r *Router) Methods(methods ...string) *Route { + return r.NewRoute().Methods(methods...) +} + +// Path registers a new route with a matcher for the URL path. +// See Route.Path(). +func (r *Router) Path(tpl string) *Route { + return r.NewRoute().Path(tpl) +} + +// PathPrefix registers a new route with a matcher for the URL path prefix. +// See Route.PathPrefix(). +func (r *Router) PathPrefix(tpl string) *Route { + return r.NewRoute().PathPrefix(tpl) +} + +// Queries registers a new route with a matcher for URL query values. +// See Route.Queries(). +func (r *Router) Queries(pairs ...string) *Route { + return r.NewRoute().Queries(pairs...) +} + +// Schemes registers a new route with a matcher for URL schemes. +// See Route.Schemes(). +func (r *Router) Schemes(schemes ...string) *Route { + return r.NewRoute().Schemes(schemes...) +} + +// ---------------------------------------------------------------------------- +// Context +// ---------------------------------------------------------------------------- + +// RouteMatch stores information about a matched route. +type RouteMatch struct { + Route *Route + Handler http.Handler + Vars map[string]string +} + +type contextKey int + +const ( + varsKey contextKey = iota + routeKey +) + +// Vars returns the route variables for the current request, if any. +func Vars(r *http.Request) map[string]string { + if rv := context.Get(r, varsKey); rv != nil { + return rv.(map[string]string) + } + return nil +} + +// CurrentRoute returns the matched route for the current request, if any. +func CurrentRoute(r *http.Request) *Route { + if rv := context.Get(r, routeKey); rv != nil { + return rv.(*Route) + } + return nil +} + +func setVars(r *http.Request, val interface{}) { + context.Set(r, varsKey, val) +} + +func setCurrentRoute(r *http.Request, val interface{}) { + context.Set(r, routeKey, val) +} + +// ---------------------------------------------------------------------------- +// Helpers +// ---------------------------------------------------------------------------- + +// cleanPath returns the canonical path for p, eliminating . and .. elements. +// Borrowed from the net/http package. +func cleanPath(p string) string { + if p == "" { + return "/" + } + if p[0] != '/' { + p = "/" + p + } + np := path.Clean(p) + // path.Clean removes trailing slash except for root; + // put the trailing slash back if necessary. + if p[len(p)-1] == '/' && np != "/" { + np += "/" + } + return np +} + +// uniqueVars returns an error if two slices contain duplicated strings. +func uniqueVars(s1, s2 []string) error { + for _, v1 := range s1 { + for _, v2 := range s2 { + if v1 == v2 { + return fmt.Errorf("mux: duplicated route variable %q", v2) + } + } + } + return nil +} + +// mapFromPairs converts variadic string parameters to a string map. +func mapFromPairs(pairs ...string) (map[string]string, error) { + length := len(pairs) + if length%2 != 0 { + return nil, fmt.Errorf( + "mux: number of parameters must be multiple of 2, got %v", pairs) + } + m := make(map[string]string, length/2) + for i := 0; i < length; i += 2 { + m[pairs[i]] = pairs[i+1] + } + return m, nil +} + +// matchInArray returns true if the given string value is in the array. +func matchInArray(arr []string, value string) bool { + for _, v := range arr { + if v == value { + return true + } + } + return false +} + +// matchMap returns true if the given key/value pairs exist in a given map. +func matchMap(toCheck map[string]string, toMatch map[string][]string, + canonicalKey bool) bool { + for k, v := range toCheck { + // Check if key exists. + if canonicalKey { + k = http.CanonicalHeaderKey(k) + } + if values := toMatch[k]; values == nil { + return false + } else if v != "" { + // If value was defined as an empty string we only check that the + // key exists. Otherwise we also check for equality. + valueExists := false + for _, value := range values { + if v == value { + valueExists = true + break + } + } + if !valueExists { + return false + } + } + } + return true +} diff --git a/third_party/github.com/gorilla/mux/mux_test.go b/third_party/github.com/gorilla/mux/mux_test.go new file mode 100644 index 00000000000..8789697f513 --- /dev/null +++ b/third_party/github.com/gorilla/mux/mux_test.go @@ -0,0 +1,755 @@ +// Copyright 2012 The Gorilla Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package mux + +import ( + "fmt" + "net/http" + "testing" + + "github.com/gorilla/context" +) + +type routeTest struct { + title string // title of the test + route *Route // the route being tested + request *http.Request // a request to test the route + vars map[string]string // the expected vars of the match + host string // the expected host of the match + path string // the expected path of the match + shouldMatch bool // whether the request is expected to match the route at all +} + +func TestHost(t *testing.T) { + // newRequestHost a new request with a method, url, and host header + newRequestHost := func(method, url, host string) *http.Request { + req, err := http.NewRequest(method, url, nil) + if err != nil { + panic(err) + } + req.Host = host + return req + } + + tests := []routeTest{ + { + title: "Host route match", + route: new(Route).Host("aaa.bbb.ccc"), + request: newRequest("GET", "http://aaa.bbb.ccc/111/222/333"), + vars: map[string]string{}, + host: "aaa.bbb.ccc", + path: "", + shouldMatch: true, + }, + { + title: "Host route, wrong host in request URL", + route: new(Route).Host("aaa.bbb.ccc"), + request: newRequest("GET", "http://aaa.222.ccc/111/222/333"), + vars: map[string]string{}, + host: "aaa.bbb.ccc", + path: "", + shouldMatch: false, + }, + { + title: "Host route with port, match", + route: new(Route).Host("aaa.bbb.ccc:1234"), + request: newRequest("GET", "http://aaa.bbb.ccc:1234/111/222/333"), + vars: map[string]string{}, + host: "aaa.bbb.ccc:1234", + path: "", + shouldMatch: true, + }, + { + title: "Host route with port, wrong port in request URL", + route: new(Route).Host("aaa.bbb.ccc:1234"), + request: newRequest("GET", "http://aaa.bbb.ccc:9999/111/222/333"), + vars: map[string]string{}, + host: "aaa.bbb.ccc:1234", + path: "", + shouldMatch: false, + }, + { + title: "Host route, match with host in request header", + route: new(Route).Host("aaa.bbb.ccc"), + request: newRequestHost("GET", "/111/222/333", "aaa.bbb.ccc"), + vars: map[string]string{}, + host: "aaa.bbb.ccc", + path: "", + shouldMatch: true, + }, + { + title: "Host route, wrong host in request header", + route: new(Route).Host("aaa.bbb.ccc"), + request: newRequestHost("GET", "/111/222/333", "aaa.222.ccc"), + vars: map[string]string{}, + host: "aaa.bbb.ccc", + path: "", + shouldMatch: false, + }, + // BUG {new(Route).Host("aaa.bbb.ccc:1234"), newRequestHost("GET", "/111/222/333", "aaa.bbb.ccc:1234"), map[string]string{}, "aaa.bbb.ccc:1234", "", true}, + { + title: "Host route with port, wrong host in request header", + route: new(Route).Host("aaa.bbb.ccc:1234"), + request: newRequestHost("GET", "/111/222/333", "aaa.bbb.ccc:9999"), + vars: map[string]string{}, + host: "aaa.bbb.ccc:1234", + path: "", + shouldMatch: false, + }, + { + title: "Host route with pattern, match", + route: new(Route).Host("aaa.{v1:[a-z]{3}}.ccc"), + request: newRequest("GET", "http://aaa.bbb.ccc/111/222/333"), + vars: map[string]string{"v1": "bbb"}, + host: "aaa.bbb.ccc", + path: "", + shouldMatch: true, + }, + { + title: "Host route with pattern, wrong host in request URL", + route: new(Route).Host("aaa.{v1:[a-z]{3}}.ccc"), + request: newRequest("GET", "http://aaa.222.ccc/111/222/333"), + vars: map[string]string{"v1": "bbb"}, + host: "aaa.bbb.ccc", + path: "", + shouldMatch: false, + }, + { + title: "Host route with multiple patterns, match", + route: new(Route).Host("{v1:[a-z]{3}}.{v2:[a-z]{3}}.{v3:[a-z]{3}}"), + request: newRequest("GET", "http://aaa.bbb.ccc/111/222/333"), + vars: map[string]string{"v1": "aaa", "v2": "bbb", "v3": "ccc"}, + host: "aaa.bbb.ccc", + path: "", + shouldMatch: true, + }, + { + title: "Host route with multiple patterns, wrong host in request URL", + route: new(Route).Host("{v1:[a-z]{3}}.{v2:[a-z]{3}}.{v3:[a-z]{3}}"), + request: newRequest("GET", "http://aaa.222.ccc/111/222/333"), + vars: map[string]string{"v1": "aaa", "v2": "bbb", "v3": "ccc"}, + host: "aaa.bbb.ccc", + path: "", + shouldMatch: false, + }, + } + for _, test := range tests { + testRoute(t, test) + } +} + +func TestPath(t *testing.T) { + tests := []routeTest{ + { + title: "Path route, match", + route: new(Route).Path("/111/222/333"), + request: newRequest("GET", "http://localhost/111/222/333"), + vars: map[string]string{}, + host: "", + path: "/111/222/333", + shouldMatch: true, + }, + { + title: "Path route, wrong path in request in request URL", + route: new(Route).Path("/111/222/333"), + request: newRequest("GET", "http://localhost/1/2/3"), + vars: map[string]string{}, + host: "", + path: "/111/222/333", + shouldMatch: false, + }, + { + title: "Path route with pattern, match", + route: new(Route).Path("/111/{v1:[0-9]{3}}/333"), + request: newRequest("GET", "http://localhost/111/222/333"), + vars: map[string]string{"v1": "222"}, + host: "", + path: "/111/222/333", + shouldMatch: true, + }, + { + title: "Path route with pattern, URL in request does not match", + route: new(Route).Path("/111/{v1:[0-9]{3}}/333"), + request: newRequest("GET", "http://localhost/111/aaa/333"), + vars: map[string]string{"v1": "222"}, + host: "", + path: "/111/222/333", + shouldMatch: false, + }, + { + title: "Path route with multiple patterns, match", + route: new(Route).Path("/{v1:[0-9]{3}}/{v2:[0-9]{3}}/{v3:[0-9]{3}}"), + request: newRequest("GET", "http://localhost/111/222/333"), + vars: map[string]string{"v1": "111", "v2": "222", "v3": "333"}, + host: "", + path: "/111/222/333", + shouldMatch: true, + }, + { + title: "Path route with multiple patterns, URL in request does not match", + route: new(Route).Path("/{v1:[0-9]{3}}/{v2:[0-9]{3}}/{v3:[0-9]{3}}"), + request: newRequest("GET", "http://localhost/111/aaa/333"), + vars: map[string]string{"v1": "111", "v2": "222", "v3": "333"}, + host: "", + path: "/111/222/333", + shouldMatch: false, + }, + } + + for _, test := range tests { + testRoute(t, test) + } +} + +func TestPathPrefix(t *testing.T) { + tests := []routeTest{ + { + title: "PathPrefix route, match", + route: new(Route).PathPrefix("/111"), + request: newRequest("GET", "http://localhost/111/222/333"), + vars: map[string]string{}, + host: "", + path: "/111", + shouldMatch: true, + }, + { + title: "PathPrefix route, URL prefix in request does not match", + route: new(Route).PathPrefix("/111"), + request: newRequest("GET", "http://localhost/1/2/3"), + vars: map[string]string{}, + host: "", + path: "/111", + shouldMatch: false, + }, + { + title: "PathPrefix route with pattern, match", + route: new(Route).PathPrefix("/111/{v1:[0-9]{3}}"), + request: newRequest("GET", "http://localhost/111/222/333"), + vars: map[string]string{"v1": "222"}, + host: "", + path: "/111/222", + shouldMatch: true, + }, + { + title: "PathPrefix route with pattern, URL prefix in request does not match", + route: new(Route).PathPrefix("/111/{v1:[0-9]{3}}"), + request: newRequest("GET", "http://localhost/111/aaa/333"), + vars: map[string]string{"v1": "222"}, + host: "", + path: "/111/222", + shouldMatch: false, + }, + { + title: "PathPrefix route with multiple patterns, match", + route: new(Route).PathPrefix("/{v1:[0-9]{3}}/{v2:[0-9]{3}}"), + request: newRequest("GET", "http://localhost/111/222/333"), + vars: map[string]string{"v1": "111", "v2": "222"}, + host: "", + path: "/111/222", + shouldMatch: true, + }, + { + title: "PathPrefix route with multiple patterns, URL prefix in request does not match", + route: new(Route).PathPrefix("/{v1:[0-9]{3}}/{v2:[0-9]{3}}"), + request: newRequest("GET", "http://localhost/111/aaa/333"), + vars: map[string]string{"v1": "111", "v2": "222"}, + host: "", + path: "/111/222", + shouldMatch: false, + }, + } + + for _, test := range tests { + testRoute(t, test) + } +} + +func TestHostPath(t *testing.T) { + tests := []routeTest{ + { + title: "Host and Path route, match", + route: new(Route).Host("aaa.bbb.ccc").Path("/111/222/333"), + request: newRequest("GET", "http://aaa.bbb.ccc/111/222/333"), + vars: map[string]string{}, + host: "", + path: "", + shouldMatch: true, + }, + { + title: "Host and Path route, wrong host in request URL", + route: new(Route).Host("aaa.bbb.ccc").Path("/111/222/333"), + request: newRequest("GET", "http://aaa.222.ccc/111/222/333"), + vars: map[string]string{}, + host: "", + path: "", + shouldMatch: false, + }, + { + title: "Host and Path route with pattern, match", + route: new(Route).Host("aaa.{v1:[a-z]{3}}.ccc").Path("/111/{v2:[0-9]{3}}/333"), + request: newRequest("GET", "http://aaa.bbb.ccc/111/222/333"), + vars: map[string]string{"v1": "bbb", "v2": "222"}, + host: "aaa.bbb.ccc", + path: "/111/222/333", + shouldMatch: true, + }, + { + title: "Host and Path route with pattern, URL in request does not match", + route: new(Route).Host("aaa.{v1:[a-z]{3}}.ccc").Path("/111/{v2:[0-9]{3}}/333"), + request: newRequest("GET", "http://aaa.222.ccc/111/222/333"), + vars: map[string]string{"v1": "bbb", "v2": "222"}, + host: "aaa.bbb.ccc", + path: "/111/222/333", + shouldMatch: false, + }, + { + title: "Host and Path route with multiple patterns, match", + route: new(Route).Host("{v1:[a-z]{3}}.{v2:[a-z]{3}}.{v3:[a-z]{3}}").Path("/{v4:[0-9]{3}}/{v5:[0-9]{3}}/{v6:[0-9]{3}}"), + request: newRequest("GET", "http://aaa.bbb.ccc/111/222/333"), + vars: map[string]string{"v1": "aaa", "v2": "bbb", "v3": "ccc", "v4": "111", "v5": "222", "v6": "333"}, + host: "aaa.bbb.ccc", + path: "/111/222/333", + shouldMatch: true, + }, + { + title: "Host and Path route with multiple patterns, URL in request does not match", + route: new(Route).Host("{v1:[a-z]{3}}.{v2:[a-z]{3}}.{v3:[a-z]{3}}").Path("/{v4:[0-9]{3}}/{v5:[0-9]{3}}/{v6:[0-9]{3}}"), + request: newRequest("GET", "http://aaa.222.ccc/111/222/333"), + vars: map[string]string{"v1": "aaa", "v2": "bbb", "v3": "ccc", "v4": "111", "v5": "222", "v6": "333"}, + host: "aaa.bbb.ccc", + path: "/111/222/333", + shouldMatch: false, + }, + } + + for _, test := range tests { + testRoute(t, test) + } +} + +func TestHeaders(t *testing.T) { + // newRequestHeaders creates a new request with a method, url, and headers + newRequestHeaders := func(method, url string, headers map[string]string) *http.Request { + req, err := http.NewRequest(method, url, nil) + if err != nil { + panic(err) + } + for k, v := range headers { + req.Header.Add(k, v) + } + return req + } + + tests := []routeTest{ + { + title: "Headers route, match", + route: new(Route).Headers("foo", "bar", "baz", "ding"), + request: newRequestHeaders("GET", "http://localhost", map[string]string{"foo": "bar", "baz": "ding"}), + vars: map[string]string{}, + host: "", + path: "", + shouldMatch: true, + }, + { + title: "Headers route, bad header values", + route: new(Route).Headers("foo", "bar", "baz", "ding"), + request: newRequestHeaders("GET", "http://localhost", map[string]string{"foo": "bar", "baz": "dong"}), + vars: map[string]string{}, + host: "", + path: "", + shouldMatch: false, + }, + } + + for _, test := range tests { + testRoute(t, test) + } + +} + +func TestMethods(t *testing.T) { + tests := []routeTest{ + { + title: "Methods route, match GET", + route: new(Route).Methods("GET", "POST"), + request: newRequest("GET", "http://localhost"), + vars: map[string]string{}, + host: "", + path: "", + shouldMatch: true, + }, + { + title: "Methods route, match POST", + route: new(Route).Methods("GET", "POST"), + request: newRequest("POST", "http://localhost"), + vars: map[string]string{}, + host: "", + path: "", + shouldMatch: true, + }, + { + title: "Methods route, bad method", + route: new(Route).Methods("GET", "POST"), + request: newRequest("PUT", "http://localhost"), + vars: map[string]string{}, + host: "", + path: "", + shouldMatch: false, + }, + } + + for _, test := range tests { + testRoute(t, test) + } +} + +func TestQueries(t *testing.T) { + tests := []routeTest{ + { + title: "Queries route, match", + route: new(Route).Queries("foo", "bar", "baz", "ding"), + request: newRequest("GET", "http://localhost?foo=bar&baz=ding"), + vars: map[string]string{}, + host: "", + path: "", + shouldMatch: true, + }, + { + title: "Queries route, bad query", + route: new(Route).Queries("foo", "bar", "baz", "ding"), + request: newRequest("GET", "http://localhost?foo=bar&baz=dong"), + vars: map[string]string{}, + host: "", + path: "", + shouldMatch: false, + }, + } + + for _, test := range tests { + testRoute(t, test) + } +} + +func TestSchemes(t *testing.T) { + tests := []routeTest{ + // Schemes + { + title: "Schemes route, match https", + route: new(Route).Schemes("https", "ftp"), + request: newRequest("GET", "https://localhost"), + vars: map[string]string{}, + host: "", + path: "", + shouldMatch: true, + }, + { + title: "Schemes route, match ftp", + route: new(Route).Schemes("https", "ftp"), + request: newRequest("GET", "ftp://localhost"), + vars: map[string]string{}, + host: "", + path: "", + shouldMatch: true, + }, + { + title: "Schemes route, bad scheme", + route: new(Route).Schemes("https", "ftp"), + request: newRequest("GET", "http://localhost"), + vars: map[string]string{}, + host: "", + path: "", + shouldMatch: false, + }, + } + for _, test := range tests { + testRoute(t, test) + } +} + +func TestMatcherFunc(t *testing.T) { + m := func(r *http.Request, m *RouteMatch) bool { + if r.URL.Host == "aaa.bbb.ccc" { + return true + } + return false + } + + tests := []routeTest{ + { + title: "MatchFunc route, match", + route: new(Route).MatcherFunc(m), + request: newRequest("GET", "http://aaa.bbb.ccc"), + vars: map[string]string{}, + host: "", + path: "", + shouldMatch: true, + }, + { + title: "MatchFunc route, non-match", + route: new(Route).MatcherFunc(m), + request: newRequest("GET", "http://aaa.222.ccc"), + vars: map[string]string{}, + host: "", + path: "", + shouldMatch: false, + }, + } + + for _, test := range tests { + testRoute(t, test) + } +} + +func TestSubRouter(t *testing.T) { + subrouter1 := new(Route).Host("{v1:[a-z]+}.google.com").Subrouter() + subrouter2 := new(Route).PathPrefix("/foo/{v1}").Subrouter() + + tests := []routeTest{ + { + route: subrouter1.Path("/{v2:[a-z]+}"), + request: newRequest("GET", "http://aaa.google.com/bbb"), + vars: map[string]string{"v1": "aaa", "v2": "bbb"}, + host: "aaa.google.com", + path: "/bbb", + shouldMatch: true, + }, + { + route: subrouter1.Path("/{v2:[a-z]+}"), + request: newRequest("GET", "http://111.google.com/111"), + vars: map[string]string{"v1": "aaa", "v2": "bbb"}, + host: "aaa.google.com", + path: "/bbb", + shouldMatch: false, + }, + { + route: subrouter2.Path("/baz/{v2}"), + request: newRequest("GET", "http://localhost/foo/bar/baz/ding"), + vars: map[string]string{"v1": "bar", "v2": "ding"}, + host: "", + path: "/foo/bar/baz/ding", + shouldMatch: true, + }, + { + route: subrouter2.Path("/baz/{v2}"), + request: newRequest("GET", "http://localhost/foo/bar"), + vars: map[string]string{"v1": "bar", "v2": "ding"}, + host: "", + path: "/foo/bar/baz/ding", + shouldMatch: false, + }, + } + + for _, test := range tests { + testRoute(t, test) + } +} + +func TestNamedRoutes(t *testing.T) { + r1 := NewRouter() + r1.NewRoute().Name("a") + r1.NewRoute().Name("b") + r1.NewRoute().Name("c") + + r2 := r1.NewRoute().Subrouter() + r2.NewRoute().Name("d") + r2.NewRoute().Name("e") + r2.NewRoute().Name("f") + + r3 := r2.NewRoute().Subrouter() + r3.NewRoute().Name("g") + r3.NewRoute().Name("h") + r3.NewRoute().Name("i") + + if r1.namedRoutes == nil || len(r1.namedRoutes) != 9 { + t.Errorf("Expected 9 named routes, got %v", r1.namedRoutes) + } else if r1.Get("i") == nil { + t.Errorf("Subroute name not registered") + } +} + +func TestStrictSlash(t *testing.T) { + var r *Router + var req *http.Request + var route *Route + var match *RouteMatch + var matched bool + + // StrictSlash should be ignored for path prefix. + // So we register a route ending in slash but it doesn't attempt to add + // the slash for a path not ending in slash. + r = NewRouter() + r.StrictSlash(true) + route = r.NewRoute().PathPrefix("/static/") + req, _ = http.NewRequest("GET", "http://localhost/static/logo.png", nil) + match = new(RouteMatch) + matched = r.Match(req, match) + if !matched { + t.Errorf("Should match request %q -- %v", req.URL.Path, getRouteTemplate(route)) + } + if match.Handler != nil { + t.Errorf("Should not redirect") + } +} + +// ---------------------------------------------------------------------------- +// Helpers +// ---------------------------------------------------------------------------- + +func getRouteTemplate(route *Route) string { + host, path := "none", "none" + if route.regexp != nil { + if route.regexp.host != nil { + host = route.regexp.host.template + } + if route.regexp.path != nil { + path = route.regexp.path.template + } + } + return fmt.Sprintf("Host: %v, Path: %v", host, path) +} + +func testRoute(t *testing.T, test routeTest) { + request := test.request + route := test.route + vars := test.vars + shouldMatch := test.shouldMatch + host := test.host + path := test.path + url := test.host + test.path + + var match RouteMatch + ok := route.Match(request, &match) + if ok != shouldMatch { + msg := "Should match" + if !shouldMatch { + msg = "Should not match" + } + t.Errorf("(%v) %v:\nRoute: %#v\nRequest: %#v\nVars: %v\n", test.title, msg, route, request, vars) + return + } + if shouldMatch { + if test.vars != nil && !stringMapEqual(test.vars, match.Vars) { + t.Errorf("(%v) Vars not equal: expected %v, got %v", test.title, vars, match.Vars) + return + } + if host != "" { + u, _ := test.route.URLHost(mapToPairs(match.Vars)...) + if host != u.Host { + t.Errorf("(%v) URLHost not equal: expected %v, got %v -- %v", test.title, host, u.Host, getRouteTemplate(route)) + return + } + } + if path != "" { + u, _ := route.URLPath(mapToPairs(match.Vars)...) + if path != u.Path { + t.Errorf("(%v) URLPath not equal: expected %v, got %v -- %v", test.title, path, u.Path, getRouteTemplate(route)) + return + } + } + if url != "" { + u, _ := route.URL(mapToPairs(match.Vars)...) + if url != u.Host+u.Path { + t.Errorf("(%v) URL not equal: expected %v, got %v -- %v", test.title, url, u.Host+u.Path, getRouteTemplate(route)) + return + } + } + } +} + +// Tests that the context is cleared or not cleared properly depending on +// the configuration of the router +func TestKeepContext(t *testing.T) { + func1 := func(w http.ResponseWriter, r *http.Request) {} + + r := NewRouter() + r.HandleFunc("/", func1).Name("func1") + + req, _ := http.NewRequest("GET", "http://localhost/", nil) + context.Set(req, "t", 1) + + res := new(http.ResponseWriter) + r.ServeHTTP(*res, req) + + if _, ok := context.GetOk(req, "t"); ok { + t.Error("Context should have been cleared at end of request") + } + + r.KeepContext = true + + req, _ = http.NewRequest("GET", "http://localhost/", nil) + context.Set(req, "t", 1) + + r.ServeHTTP(*res, req) + if _, ok := context.GetOk(req, "t"); !ok { + t.Error("Context should NOT have been cleared at end of request") + } + +} + +// https://plus.google.com/101022900381697718949/posts/eWy6DjFJ6uW +func TestSubrouterHeader(t *testing.T) { + expected := "func1 response" + func1 := func(w http.ResponseWriter, r *http.Request) { + fmt.Fprint(w, expected) + } + func2 := func(http.ResponseWriter, *http.Request) {} + + r := NewRouter() + s := r.Headers("SomeSpecialHeader", "").Subrouter() + s.HandleFunc("/", func1).Name("func1") + r.HandleFunc("/", func2).Name("func2") + + req, _ := http.NewRequest("GET", "http://localhost/", nil) + req.Header.Add("SomeSpecialHeader", "foo") + match := new(RouteMatch) + matched := r.Match(req, match) + if !matched { + t.Errorf("Should match request") + } + if match.Route.GetName() != "func1" { + t.Errorf("Expecting func1 handler, got %s", match.Route.GetName()) + } + resp := NewRecorder() + match.Handler.ServeHTTP(resp, req) + if resp.Body.String() != expected { + t.Errorf("Expecting %q", expected) + } +} + +// mapToPairs converts a string map to a slice of string pairs +func mapToPairs(m map[string]string) []string { + var i int + p := make([]string, len(m)*2) + for k, v := range m { + p[i] = k + p[i+1] = v + i += 2 + } + return p +} + +// stringMapEqual checks the equality of two string maps +func stringMapEqual(m1, m2 map[string]string) bool { + nil1 := m1 == nil + nil2 := m2 == nil + if nil1 != nil2 || len(m1) != len(m2) { + return false + } + for k, v := range m1 { + if v != m2[k] { + return false + } + } + return true +} + +// newRequest is a helper function to create a new request with a method and url +func newRequest(method, url string) *http.Request { + req, err := http.NewRequest(method, url, nil) + if err != nil { + panic(err) + } + return req +} diff --git a/third_party/github.com/gorilla/mux/old_test.go b/third_party/github.com/gorilla/mux/old_test.go new file mode 100644 index 00000000000..7e266bb6954 --- /dev/null +++ b/third_party/github.com/gorilla/mux/old_test.go @@ -0,0 +1,758 @@ +// Old tests ported to Go1. This is a mess. Want to drop it one day. + +// Copyright 2011 Gorilla Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package mux + +import ( + "bytes" + "net/http" + "testing" +) + +// ---------------------------------------------------------------------------- +// ResponseRecorder +// ---------------------------------------------------------------------------- +// Copyright 2009 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// ResponseRecorder is an implementation of http.ResponseWriter that +// records its mutations for later inspection in tests. +type ResponseRecorder struct { + Code int // the HTTP response code from WriteHeader + HeaderMap http.Header // the HTTP response headers + Body *bytes.Buffer // if non-nil, the bytes.Buffer to append written data to + Flushed bool +} + +// NewRecorder returns an initialized ResponseRecorder. +func NewRecorder() *ResponseRecorder { + return &ResponseRecorder{ + HeaderMap: make(http.Header), + Body: new(bytes.Buffer), + } +} + +// DefaultRemoteAddr is the default remote address to return in RemoteAddr if +// an explicit DefaultRemoteAddr isn't set on ResponseRecorder. +const DefaultRemoteAddr = "1.2.3.4" + +// Header returns the response headers. +func (rw *ResponseRecorder) Header() http.Header { + return rw.HeaderMap +} + +// Write always succeeds and writes to rw.Body, if not nil. +func (rw *ResponseRecorder) Write(buf []byte) (int, error) { + if rw.Body != nil { + rw.Body.Write(buf) + } + if rw.Code == 0 { + rw.Code = http.StatusOK + } + return len(buf), nil +} + +// WriteHeader sets rw.Code. +func (rw *ResponseRecorder) WriteHeader(code int) { + rw.Code = code +} + +// Flush sets rw.Flushed to true. +func (rw *ResponseRecorder) Flush() { + rw.Flushed = true +} + +// ---------------------------------------------------------------------------- + +func TestRouteMatchers(t *testing.T) { + var scheme, host, path, query, method string + var headers map[string]string + var resultVars map[bool]map[string]string + + router := NewRouter() + router.NewRoute().Host("{var1}.google.com"). + Path("/{var2:[a-z]+}/{var3:[0-9]+}"). + Queries("foo", "bar"). + Methods("GET"). + Schemes("https"). + Headers("x-requested-with", "XMLHttpRequest") + router.NewRoute().Host("www.{var4}.com"). + PathPrefix("/foo/{var5:[a-z]+}/{var6:[0-9]+}"). + Queries("baz", "ding"). + Methods("POST"). + Schemes("http"). + Headers("Content-Type", "application/json") + + reset := func() { + // Everything match. + scheme = "https" + host = "www.google.com" + path = "/product/42" + query = "?foo=bar" + method = "GET" + headers = map[string]string{"X-Requested-With": "XMLHttpRequest"} + resultVars = map[bool]map[string]string{ + true: map[string]string{"var1": "www", "var2": "product", "var3": "42"}, + false: map[string]string{}, + } + } + + reset2 := func() { + // Everything match. + scheme = "http" + host = "www.google.com" + path = "/foo/product/42/path/that/is/ignored" + query = "?baz=ding" + method = "POST" + headers = map[string]string{"Content-Type": "application/json"} + resultVars = map[bool]map[string]string{ + true: map[string]string{"var4": "google", "var5": "product", "var6": "42"}, + false: map[string]string{}, + } + } + + match := func(shouldMatch bool) { + url := scheme + "://" + host + path + query + request, _ := http.NewRequest(method, url, nil) + for key, value := range headers { + request.Header.Add(key, value) + } + + var routeMatch RouteMatch + matched := router.Match(request, &routeMatch) + if matched != shouldMatch { + // Need better messages. :) + if matched { + t.Errorf("Should match.") + } else { + t.Errorf("Should not match.") + } + } + + if matched { + currentRoute := routeMatch.Route + if currentRoute == nil { + t.Errorf("Expected a current route.") + } + vars := routeMatch.Vars + expectedVars := resultVars[shouldMatch] + if len(vars) != len(expectedVars) { + t.Errorf("Expected vars: %v Got: %v.", expectedVars, vars) + } + for name, value := range vars { + if expectedVars[name] != value { + t.Errorf("Expected vars: %v Got: %v.", expectedVars, vars) + } + } + } + } + + // 1st route -------------------------------------------------------------- + + // Everything match. + reset() + match(true) + + // Scheme doesn't match. + reset() + scheme = "http" + match(false) + + // Host doesn't match. + reset() + host = "www.mygoogle.com" + match(false) + + // Path doesn't match. + reset() + path = "/product/notdigits" + match(false) + + // Query doesn't match. + reset() + query = "?foo=baz" + match(false) + + // Method doesn't match. + reset() + method = "POST" + match(false) + + // Header doesn't match. + reset() + headers = map[string]string{} + match(false) + + // Everything match, again. + reset() + match(true) + + // 2nd route -------------------------------------------------------------- + + // Everything match. + reset2() + match(true) + + // Scheme doesn't match. + reset2() + scheme = "https" + match(false) + + // Host doesn't match. + reset2() + host = "sub.google.com" + match(false) + + // Path doesn't match. + reset2() + path = "/bar/product/42" + match(false) + + // Query doesn't match. + reset2() + query = "?foo=baz" + match(false) + + // Method doesn't match. + reset2() + method = "GET" + match(false) + + // Header doesn't match. + reset2() + headers = map[string]string{} + match(false) + + // Everything match, again. + reset2() + match(true) +} + +type headerMatcherTest struct { + matcher headerMatcher + headers map[string]string + result bool +} + +var headerMatcherTests = []headerMatcherTest{ + { + matcher: headerMatcher(map[string]string{"x-requested-with": "XMLHttpRequest"}), + headers: map[string]string{"X-Requested-With": "XMLHttpRequest"}, + result: true, + }, + { + matcher: headerMatcher(map[string]string{"x-requested-with": ""}), + headers: map[string]string{"X-Requested-With": "anything"}, + result: true, + }, + { + matcher: headerMatcher(map[string]string{"x-requested-with": "XMLHttpRequest"}), + headers: map[string]string{}, + result: false, + }, +} + +type hostMatcherTest struct { + matcher *Route + url string + vars map[string]string + result bool +} + +var hostMatcherTests = []hostMatcherTest{ + { + matcher: NewRouter().NewRoute().Host("{foo:[a-z][a-z][a-z]}.{bar:[a-z][a-z][a-z]}.{baz:[a-z][a-z][a-z]}"), + url: "http://abc.def.ghi/", + vars: map[string]string{"foo": "abc", "bar": "def", "baz": "ghi"}, + result: true, + }, + { + matcher: NewRouter().NewRoute().Host("{foo:[a-z][a-z][a-z]}.{bar:[a-z][a-z][a-z]}.{baz:[a-z][a-z][a-z]}"), + url: "http://a.b.c/", + vars: map[string]string{"foo": "abc", "bar": "def", "baz": "ghi"}, + result: false, + }, +} + +type methodMatcherTest struct { + matcher methodMatcher + method string + result bool +} + +var methodMatcherTests = []methodMatcherTest{ + { + matcher: methodMatcher([]string{"GET", "POST", "PUT"}), + method: "GET", + result: true, + }, + { + matcher: methodMatcher([]string{"GET", "POST", "PUT"}), + method: "POST", + result: true, + }, + { + matcher: methodMatcher([]string{"GET", "POST", "PUT"}), + method: "PUT", + result: true, + }, + { + matcher: methodMatcher([]string{"GET", "POST", "PUT"}), + method: "DELETE", + result: false, + }, +} + +type pathMatcherTest struct { + matcher *Route + url string + vars map[string]string + result bool +} + +var pathMatcherTests = []pathMatcherTest{ + { + matcher: NewRouter().NewRoute().Path("/{foo:[0-9][0-9][0-9]}/{bar:[0-9][0-9][0-9]}/{baz:[0-9][0-9][0-9]}"), + url: "http://localhost:8080/123/456/789", + vars: map[string]string{"foo": "123", "bar": "456", "baz": "789"}, + result: true, + }, + { + matcher: NewRouter().NewRoute().Path("/{foo:[0-9][0-9][0-9]}/{bar:[0-9][0-9][0-9]}/{baz:[0-9][0-9][0-9]}"), + url: "http://localhost:8080/1/2/3", + vars: map[string]string{"foo": "123", "bar": "456", "baz": "789"}, + result: false, + }, +} + +type queryMatcherTest struct { + matcher queryMatcher + url string + result bool +} + +var queryMatcherTests = []queryMatcherTest{ + { + matcher: queryMatcher(map[string]string{"foo": "bar", "baz": "ding"}), + url: "http://localhost:8080/?foo=bar&baz=ding", + result: true, + }, + { + matcher: queryMatcher(map[string]string{"foo": "", "baz": ""}), + url: "http://localhost:8080/?foo=anything&baz=anything", + result: true, + }, + { + matcher: queryMatcher(map[string]string{"foo": "ding", "baz": "bar"}), + url: "http://localhost:8080/?foo=bar&baz=ding", + result: false, + }, + { + matcher: queryMatcher(map[string]string{"bar": "foo", "ding": "baz"}), + url: "http://localhost:8080/?foo=bar&baz=ding", + result: false, + }, +} + +type schemeMatcherTest struct { + matcher schemeMatcher + url string + result bool +} + +var schemeMatcherTests = []schemeMatcherTest{ + { + matcher: schemeMatcher([]string{"http", "https"}), + url: "http://localhost:8080/", + result: true, + }, + { + matcher: schemeMatcher([]string{"http", "https"}), + url: "https://localhost:8080/", + result: true, + }, + { + matcher: schemeMatcher([]string{"https"}), + url: "http://localhost:8080/", + result: false, + }, + { + matcher: schemeMatcher([]string{"http"}), + url: "https://localhost:8080/", + result: false, + }, +} + +type urlBuildingTest struct { + route *Route + vars []string + url string +} + +var urlBuildingTests = []urlBuildingTest{ + { + route: new(Route).Host("foo.domain.com"), + vars: []string{}, + url: "http://foo.domain.com", + }, + { + route: new(Route).Host("{subdomain}.domain.com"), + vars: []string{"subdomain", "bar"}, + url: "http://bar.domain.com", + }, + { + route: new(Route).Host("foo.domain.com").Path("/articles"), + vars: []string{}, + url: "http://foo.domain.com/articles", + }, + { + route: new(Route).Path("/articles"), + vars: []string{}, + url: "/articles", + }, + { + route: new(Route).Path("/articles/{category}/{id:[0-9]+}"), + vars: []string{"category", "technology", "id", "42"}, + url: "/articles/technology/42", + }, + { + route: new(Route).Host("{subdomain}.domain.com").Path("/articles/{category}/{id:[0-9]+}"), + vars: []string{"subdomain", "foo", "category", "technology", "id", "42"}, + url: "http://foo.domain.com/articles/technology/42", + }, +} + +func TestHeaderMatcher(t *testing.T) { + for _, v := range headerMatcherTests { + request, _ := http.NewRequest("GET", "http://localhost:8080/", nil) + for key, value := range v.headers { + request.Header.Add(key, value) + } + var routeMatch RouteMatch + result := v.matcher.Match(request, &routeMatch) + if result != v.result { + if v.result { + t.Errorf("%#v: should match %v.", v.matcher, request.Header) + } else { + t.Errorf("%#v: should not match %v.", v.matcher, request.Header) + } + } + } +} + +func TestHostMatcher(t *testing.T) { + for _, v := range hostMatcherTests { + request, _ := http.NewRequest("GET", v.url, nil) + var routeMatch RouteMatch + result := v.matcher.Match(request, &routeMatch) + vars := routeMatch.Vars + if result != v.result { + if v.result { + t.Errorf("%#v: should match %v.", v.matcher, v.url) + } else { + t.Errorf("%#v: should not match %v.", v.matcher, v.url) + } + } + if result { + if len(vars) != len(v.vars) { + t.Errorf("%#v: vars length should be %v, got %v.", v.matcher, len(v.vars), len(vars)) + } + for name, value := range vars { + if v.vars[name] != value { + t.Errorf("%#v: expected value %v for key %v, got %v.", v.matcher, v.vars[name], name, value) + } + } + } else { + if len(vars) != 0 { + t.Errorf("%#v: vars length should be 0, got %v.", v.matcher, len(vars)) + } + } + } +} + +func TestMethodMatcher(t *testing.T) { + for _, v := range methodMatcherTests { + request, _ := http.NewRequest(v.method, "http://localhost:8080/", nil) + var routeMatch RouteMatch + result := v.matcher.Match(request, &routeMatch) + if result != v.result { + if v.result { + t.Errorf("%#v: should match %v.", v.matcher, v.method) + } else { + t.Errorf("%#v: should not match %v.", v.matcher, v.method) + } + } + } +} + +func TestPathMatcher(t *testing.T) { + for _, v := range pathMatcherTests { + request, _ := http.NewRequest("GET", v.url, nil) + var routeMatch RouteMatch + result := v.matcher.Match(request, &routeMatch) + vars := routeMatch.Vars + if result != v.result { + if v.result { + t.Errorf("%#v: should match %v.", v.matcher, v.url) + } else { + t.Errorf("%#v: should not match %v.", v.matcher, v.url) + } + } + if result { + if len(vars) != len(v.vars) { + t.Errorf("%#v: vars length should be %v, got %v.", v.matcher, len(v.vars), len(vars)) + } + for name, value := range vars { + if v.vars[name] != value { + t.Errorf("%#v: expected value %v for key %v, got %v.", v.matcher, v.vars[name], name, value) + } + } + } else { + if len(vars) != 0 { + t.Errorf("%#v: vars length should be 0, got %v.", v.matcher, len(vars)) + } + } + } +} + +func TestQueryMatcher(t *testing.T) { + for _, v := range queryMatcherTests { + request, _ := http.NewRequest("GET", v.url, nil) + var routeMatch RouteMatch + result := v.matcher.Match(request, &routeMatch) + if result != v.result { + if v.result { + t.Errorf("%#v: should match %v.", v.matcher, v.url) + } else { + t.Errorf("%#v: should not match %v.", v.matcher, v.url) + } + } + } +} + +func TestSchemeMatcher(t *testing.T) { + for _, v := range queryMatcherTests { + request, _ := http.NewRequest("GET", v.url, nil) + var routeMatch RouteMatch + result := v.matcher.Match(request, &routeMatch) + if result != v.result { + if v.result { + t.Errorf("%#v: should match %v.", v.matcher, v.url) + } else { + t.Errorf("%#v: should not match %v.", v.matcher, v.url) + } + } + } +} + +func TestUrlBuilding(t *testing.T) { + + for _, v := range urlBuildingTests { + u, _ := v.route.URL(v.vars...) + url := u.String() + if url != v.url { + t.Errorf("expected %v, got %v", v.url, url) + /* + reversePath := "" + reverseHost := "" + if v.route.pathTemplate != nil { + reversePath = v.route.pathTemplate.Reverse + } + if v.route.hostTemplate != nil { + reverseHost = v.route.hostTemplate.Reverse + } + + t.Errorf("%#v:\nexpected: %q\ngot: %q\nreverse path: %q\nreverse host: %q", v.route, v.url, url, reversePath, reverseHost) + */ + } + } + + ArticleHandler := func(w http.ResponseWriter, r *http.Request) { + } + + router := NewRouter() + router.HandleFunc("/articles/{category}/{id:[0-9]+}", ArticleHandler).Name("article") + + url, _ := router.Get("article").URL("category", "technology", "id", "42") + expected := "/articles/technology/42" + if url.String() != expected { + t.Errorf("Expected %v, got %v", expected, url.String()) + } +} + +func TestMatchedRouteName(t *testing.T) { + routeName := "stock" + router := NewRouter() + route := router.NewRoute().Path("/products/").Name(routeName) + + url := "http://www.domain.com/products/" + request, _ := http.NewRequest("GET", url, nil) + var rv RouteMatch + ok := router.Match(request, &rv) + + if !ok || rv.Route != route { + t.Errorf("Expected same route, got %+v.", rv.Route) + } + + retName := rv.Route.GetName() + if retName != routeName { + t.Errorf("Expected %q, got %q.", routeName, retName) + } +} + +func TestSubRouting(t *testing.T) { + // Example from docs. + router := NewRouter() + subrouter := router.NewRoute().Host("www.domain.com").Subrouter() + route := subrouter.NewRoute().Path("/products/").Name("products") + + url := "http://www.domain.com/products/" + request, _ := http.NewRequest("GET", url, nil) + var rv RouteMatch + ok := router.Match(request, &rv) + + if !ok || rv.Route != route { + t.Errorf("Expected same route, got %+v.", rv.Route) + } + + u, _ := router.Get("products").URL() + builtUrl := u.String() + // Yay, subroute aware of the domain when building! + if builtUrl != url { + t.Errorf("Expected %q, got %q.", url, builtUrl) + } +} + +func TestVariableNames(t *testing.T) { + route := new(Route).Host("{arg1}.domain.com").Path("/{arg1}/{arg2:[0-9]+}") + if route.err == nil { + t.Errorf("Expected error for duplicated variable names") + } +} + +func TestRedirectSlash(t *testing.T) { + var route *Route + var routeMatch RouteMatch + r := NewRouter() + + r.StrictSlash(false) + route = r.NewRoute() + if route.strictSlash != false { + t.Errorf("Expected false redirectSlash.") + } + + r.StrictSlash(true) + route = r.NewRoute() + if route.strictSlash != true { + t.Errorf("Expected true redirectSlash.") + } + + route = new(Route) + route.strictSlash = true + route.Path("/{arg1}/{arg2:[0-9]+}/") + request, _ := http.NewRequest("GET", "http://localhost/foo/123", nil) + routeMatch = RouteMatch{} + _ = route.Match(request, &routeMatch) + vars := routeMatch.Vars + if vars["arg1"] != "foo" { + t.Errorf("Expected foo.") + } + if vars["arg2"] != "123" { + t.Errorf("Expected 123.") + } + rsp := NewRecorder() + routeMatch.Handler.ServeHTTP(rsp, request) + if rsp.HeaderMap.Get("Location") != "http://localhost/foo/123/" { + t.Errorf("Expected redirect header.") + } + + route = new(Route) + route.strictSlash = true + route.Path("/{arg1}/{arg2:[0-9]+}") + request, _ = http.NewRequest("GET", "http://localhost/foo/123/", nil) + routeMatch = RouteMatch{} + _ = route.Match(request, &routeMatch) + vars = routeMatch.Vars + if vars["arg1"] != "foo" { + t.Errorf("Expected foo.") + } + if vars["arg2"] != "123" { + t.Errorf("Expected 123.") + } + rsp = NewRecorder() + routeMatch.Handler.ServeHTTP(rsp, request) + if rsp.HeaderMap.Get("Location") != "http://localhost/foo/123" { + t.Errorf("Expected redirect header.") + } +} + +// Test for the new regexp library, still not available in stable Go. +func TestNewRegexp(t *testing.T) { + var p *routeRegexp + var matches []string + + tests := map[string]map[string][]string{ + "/{foo:a{2}}": { + "/a": nil, + "/aa": {"aa"}, + "/aaa": nil, + "/aaaa": nil, + }, + "/{foo:a{2,}}": { + "/a": nil, + "/aa": {"aa"}, + "/aaa": {"aaa"}, + "/aaaa": {"aaaa"}, + }, + "/{foo:a{2,3}}": { + "/a": nil, + "/aa": {"aa"}, + "/aaa": {"aaa"}, + "/aaaa": nil, + }, + "/{foo:[a-z]{3}}/{bar:[a-z]{2}}": { + "/a": nil, + "/ab": nil, + "/abc": nil, + "/abcd": nil, + "/abc/ab": {"abc", "ab"}, + "/abc/abc": nil, + "/abcd/ab": nil, + }, + `/{foo:\w{3,}}/{bar:\d{2,}}`: { + "/a": nil, + "/ab": nil, + "/abc": nil, + "/abc/1": nil, + "/abc/12": {"abc", "12"}, + "/abcd/12": {"abcd", "12"}, + "/abcd/123": {"abcd", "123"}, + }, + } + + for pattern, paths := range tests { + p, _ = newRouteRegexp(pattern, false, false, false) + for path, result := range paths { + matches = p.regexp.FindStringSubmatch(path) + if result == nil { + if matches != nil { + t.Errorf("%v should not match %v.", pattern, path) + } + } else { + if len(matches) != len(result)+1 { + t.Errorf("Expected %v matches, got %v.", len(result)+1, len(matches)) + } else { + for k, v := range result { + if matches[k+1] != v { + t.Errorf("Expected %v, got %v.", v, matches[k+1]) + } + } + } + } + } + } +} diff --git a/third_party/github.com/gorilla/mux/regexp.go b/third_party/github.com/gorilla/mux/regexp.go new file mode 100644 index 00000000000..4c3482bfbdd --- /dev/null +++ b/third_party/github.com/gorilla/mux/regexp.go @@ -0,0 +1,247 @@ +// Copyright 2012 The Gorilla Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package mux + +import ( + "bytes" + "fmt" + "net/http" + "net/url" + "regexp" + "strings" +) + +// newRouteRegexp parses a route template and returns a routeRegexp, +// used to match a host or path. +// +// It will extract named variables, assemble a regexp to be matched, create +// a "reverse" template to build URLs and compile regexps to validate variable +// values used in URL building. +// +// Previously we accepted only Python-like identifiers for variable +// names ([a-zA-Z_][a-zA-Z0-9_]*), but currently the only restriction is that +// name and pattern can't be empty, and names can't contain a colon. +func newRouteRegexp(tpl string, matchHost, matchPrefix, strictSlash bool) (*routeRegexp, error) { + // Check if it is well-formed. + idxs, errBraces := braceIndices(tpl) + if errBraces != nil { + return nil, errBraces + } + // Backup the original. + template := tpl + // Now let's parse it. + defaultPattern := "[^/]+" + if matchHost { + defaultPattern = "[^.]+" + matchPrefix, strictSlash = false, false + } + if matchPrefix { + strictSlash = false + } + // Set a flag for strictSlash. + endSlash := false + if strictSlash && strings.HasSuffix(tpl, "/") { + tpl = tpl[:len(tpl)-1] + endSlash = true + } + varsN := make([]string, len(idxs)/2) + varsR := make([]*regexp.Regexp, len(idxs)/2) + pattern := bytes.NewBufferString("^") + reverse := bytes.NewBufferString("") + var end int + var err error + for i := 0; i < len(idxs); i += 2 { + // Set all values we are interested in. + raw := tpl[end:idxs[i]] + end = idxs[i+1] + parts := strings.SplitN(tpl[idxs[i]+1:end-1], ":", 2) + name := parts[0] + patt := defaultPattern + if len(parts) == 2 { + patt = parts[1] + } + // Name or pattern can't be empty. + if name == "" || patt == "" { + return nil, fmt.Errorf("mux: missing name or pattern in %q", + tpl[idxs[i]:end]) + } + // Build the regexp pattern. + fmt.Fprintf(pattern, "%s(%s)", regexp.QuoteMeta(raw), patt) + // Build the reverse template. + fmt.Fprintf(reverse, "%s%%s", raw) + // Append variable name and compiled pattern. + varsN[i/2] = name + varsR[i/2], err = regexp.Compile(fmt.Sprintf("^%s$", patt)) + if err != nil { + return nil, err + } + } + // Add the remaining. + raw := tpl[end:] + pattern.WriteString(regexp.QuoteMeta(raw)) + if strictSlash { + pattern.WriteString("[/]?") + } + if !matchPrefix { + pattern.WriteByte('$') + } + reverse.WriteString(raw) + if endSlash { + reverse.WriteByte('/') + } + // Compile full regexp. + reg, errCompile := regexp.Compile(pattern.String()) + if errCompile != nil { + return nil, errCompile + } + // Done! + return &routeRegexp{ + template: template, + matchHost: matchHost, + regexp: reg, + reverse: reverse.String(), + varsN: varsN, + varsR: varsR, + }, nil +} + +// routeRegexp stores a regexp to match a host or path and information to +// collect and validate route variables. +type routeRegexp struct { + // The unmodified template. + template string + // True for host match, false for path match. + matchHost bool + // Expanded regexp. + regexp *regexp.Regexp + // Reverse template. + reverse string + // Variable names. + varsN []string + // Variable regexps (validators). + varsR []*regexp.Regexp +} + +// Match matches the regexp against the URL host or path. +func (r *routeRegexp) Match(req *http.Request, match *RouteMatch) bool { + if !r.matchHost { + return r.regexp.MatchString(req.URL.Path) + } + return r.regexp.MatchString(getHost(req)) +} + +// url builds a URL part using the given values. +func (r *routeRegexp) url(pairs ...string) (string, error) { + values, err := mapFromPairs(pairs...) + if err != nil { + return "", err + } + urlValues := make([]interface{}, len(r.varsN)) + for k, v := range r.varsN { + value, ok := values[v] + if !ok { + return "", fmt.Errorf("mux: missing route variable %q", v) + } + urlValues[k] = value + } + rv := fmt.Sprintf(r.reverse, urlValues...) + if !r.regexp.MatchString(rv) { + // The URL is checked against the full regexp, instead of checking + // individual variables. This is faster but to provide a good error + // message, we check individual regexps if the URL doesn't match. + for k, v := range r.varsN { + if !r.varsR[k].MatchString(values[v]) { + return "", fmt.Errorf( + "mux: variable %q doesn't match, expected %q", values[v], + r.varsR[k].String()) + } + } + } + return rv, nil +} + +// braceIndices returns the first level curly brace indices from a string. +// It returns an error in case of unbalanced braces. +func braceIndices(s string) ([]int, error) { + var level, idx int + idxs := make([]int, 0) + for i := 0; i < len(s); i++ { + switch s[i] { + case '{': + if level++; level == 1 { + idx = i + } + case '}': + if level--; level == 0 { + idxs = append(idxs, idx, i+1) + } else if level < 0 { + return nil, fmt.Errorf("mux: unbalanced braces in %q", s) + } + } + } + if level != 0 { + return nil, fmt.Errorf("mux: unbalanced braces in %q", s) + } + return idxs, nil +} + +// ---------------------------------------------------------------------------- +// routeRegexpGroup +// ---------------------------------------------------------------------------- + +// routeRegexpGroup groups the route matchers that carry variables. +type routeRegexpGroup struct { + host *routeRegexp + path *routeRegexp +} + +// setMatch extracts the variables from the URL once a route matches. +func (v *routeRegexpGroup) setMatch(req *http.Request, m *RouteMatch, r *Route) { + // Store host variables. + if v.host != nil { + hostVars := v.host.regexp.FindStringSubmatch(getHost(req)) + if hostVars != nil { + for k, v := range v.host.varsN { + m.Vars[v] = hostVars[k+1] + } + } + } + // Store path variables. + if v.path != nil { + pathVars := v.path.regexp.FindStringSubmatch(req.URL.Path) + if pathVars != nil { + for k, v := range v.path.varsN { + m.Vars[v] = pathVars[k+1] + } + // Check if we should redirect. + if r.strictSlash { + p1 := strings.HasSuffix(req.URL.Path, "/") + p2 := strings.HasSuffix(v.path.template, "/") + if p1 != p2 { + u, _ := url.Parse(req.URL.String()) + if p1 { + u.Path = u.Path[:len(u.Path)-1] + } else { + u.Path += "/" + } + m.Handler = http.RedirectHandler(u.String(), 301) + } + } + } + } +} + +// getHost tries its best to return the request host. +func getHost(r *http.Request) string { + if !r.URL.IsAbs() { + host := r.Host + // Slice off any port information. + if i := strings.Index(host, ":"); i != -1 { + host = host[:i] + } + return host + } + return r.URL.Host +} diff --git a/third_party/github.com/gorilla/mux/route.go b/third_party/github.com/gorilla/mux/route.go new file mode 100644 index 00000000000..7766254d527 --- /dev/null +++ b/third_party/github.com/gorilla/mux/route.go @@ -0,0 +1,499 @@ +// Copyright 2012 The Gorilla Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package mux + +import ( + "errors" + "fmt" + "net/http" + "net/url" + "strings" +) + +// Route stores information to match a request and build URLs. +type Route struct { + // Parent where the route was registered (a Router). + parent parentRoute + // Request handler for the route. + handler http.Handler + // List of matchers. + matchers []matcher + // Manager for the variables from host and path. + regexp *routeRegexpGroup + // If true, when the path pattern is "/path/", accessing "/path" will + // redirect to the former and vice versa. + strictSlash bool + // If true, this route never matches: it is only used to build URLs. + buildOnly bool + // The name used to build URLs. + name string + // Error resulted from building a route. + err error +} + +// Match matches the route against the request. +func (r *Route) Match(req *http.Request, match *RouteMatch) bool { + if r.buildOnly || r.err != nil { + return false + } + // Match everything. + for _, m := range r.matchers { + if matched := m.Match(req, match); !matched { + return false + } + } + // Yay, we have a match. Let's collect some info about it. + if match.Route == nil { + match.Route = r + } + if match.Handler == nil { + match.Handler = r.handler + } + if match.Vars == nil { + match.Vars = make(map[string]string) + } + // Set variables. + if r.regexp != nil { + r.regexp.setMatch(req, match, r) + } + return true +} + +// ---------------------------------------------------------------------------- +// Route attributes +// ---------------------------------------------------------------------------- + +// GetError returns an error resulted from building the route, if any. +func (r *Route) GetError() error { + return r.err +} + +// BuildOnly sets the route to never match: it is only used to build URLs. +func (r *Route) BuildOnly() *Route { + r.buildOnly = true + return r +} + +// Handler -------------------------------------------------------------------- + +// Handler sets a handler for the route. +func (r *Route) Handler(handler http.Handler) *Route { + if r.err == nil { + r.handler = handler + } + return r +} + +// HandlerFunc sets a handler function for the route. +func (r *Route) HandlerFunc(f func(http.ResponseWriter, *http.Request)) *Route { + return r.Handler(http.HandlerFunc(f)) +} + +// GetHandler returns the handler for the route, if any. +func (r *Route) GetHandler() http.Handler { + return r.handler +} + +// Name ----------------------------------------------------------------------- + +// Name sets the name for the route, used to build URLs. +// If the name was registered already it will be overwritten. +func (r *Route) Name(name string) *Route { + if r.name != "" { + r.err = fmt.Errorf("mux: route already has name %q, can't set %q", + r.name, name) + } + if r.err == nil { + r.name = name + r.getNamedRoutes()[name] = r + } + return r +} + +// GetName returns the name for the route, if any. +func (r *Route) GetName() string { + return r.name +} + +// ---------------------------------------------------------------------------- +// Matchers +// ---------------------------------------------------------------------------- + +// matcher types try to match a request. +type matcher interface { + Match(*http.Request, *RouteMatch) bool +} + +// addMatcher adds a matcher to the route. +func (r *Route) addMatcher(m matcher) *Route { + if r.err == nil { + r.matchers = append(r.matchers, m) + } + return r +} + +// addRegexpMatcher adds a host or path matcher and builder to a route. +func (r *Route) addRegexpMatcher(tpl string, matchHost, matchPrefix bool) error { + if r.err != nil { + return r.err + } + r.regexp = r.getRegexpGroup() + if !matchHost { + if len(tpl) == 0 || tpl[0] != '/' { + return fmt.Errorf("mux: path must start with a slash, got %q", tpl) + } + if r.regexp.path != nil { + tpl = strings.TrimRight(r.regexp.path.template, "/") + tpl + } + } + rr, err := newRouteRegexp(tpl, matchHost, matchPrefix, r.strictSlash) + if err != nil { + return err + } + if matchHost { + if r.regexp.path != nil { + if err = uniqueVars(rr.varsN, r.regexp.path.varsN); err != nil { + return err + } + } + r.regexp.host = rr + } else { + if r.regexp.host != nil { + if err = uniqueVars(rr.varsN, r.regexp.host.varsN); err != nil { + return err + } + } + r.regexp.path = rr + } + r.addMatcher(rr) + return nil +} + +// Headers -------------------------------------------------------------------- + +// headerMatcher matches the request against header values. +type headerMatcher map[string]string + +func (m headerMatcher) Match(r *http.Request, match *RouteMatch) bool { + return matchMap(m, r.Header, true) +} + +// Headers adds a matcher for request header values. +// It accepts a sequence of key/value pairs to be matched. For example: +// +// r := mux.NewRouter() +// r.Headers("Content-Type", "application/json", +// "X-Requested-With", "XMLHttpRequest") +// +// The above route will only match if both request header values match. +// +// It the value is an empty string, it will match any value if the key is set. +func (r *Route) Headers(pairs ...string) *Route { + if r.err == nil { + var headers map[string]string + headers, r.err = mapFromPairs(pairs...) + return r.addMatcher(headerMatcher(headers)) + } + return r +} + +// Host ----------------------------------------------------------------------- + +// Host adds a matcher for the URL host. +// It accepts a template with zero or more URL variables enclosed by {}. +// Variables can define an optional regexp pattern to me matched: +// +// - {name} matches anything until the next dot. +// +// - {name:pattern} matches the given regexp pattern. +// +// For example: +// +// r := mux.NewRouter() +// r.Host("www.domain.com") +// r.Host("{subdomain}.domain.com") +// r.Host("{subdomain:[a-z]+}.domain.com") +// +// Variable names must be unique in a given route. They can be retrieved +// calling mux.Vars(request). +func (r *Route) Host(tpl string) *Route { + r.err = r.addRegexpMatcher(tpl, true, false) + return r +} + +// MatcherFunc ---------------------------------------------------------------- + +// MatcherFunc is the function signature used by custom matchers. +type MatcherFunc func(*http.Request, *RouteMatch) bool + +func (m MatcherFunc) Match(r *http.Request, match *RouteMatch) bool { + return m(r, match) +} + +// MatcherFunc adds a custom function to be used as request matcher. +func (r *Route) MatcherFunc(f MatcherFunc) *Route { + return r.addMatcher(f) +} + +// Methods -------------------------------------------------------------------- + +// methodMatcher matches the request against HTTP methods. +type methodMatcher []string + +func (m methodMatcher) Match(r *http.Request, match *RouteMatch) bool { + return matchInArray(m, r.Method) +} + +// Methods adds a matcher for HTTP methods. +// It accepts a sequence of one or more methods to be matched, e.g.: +// "GET", "POST", "PUT". +func (r *Route) Methods(methods ...string) *Route { + for k, v := range methods { + methods[k] = strings.ToUpper(v) + } + return r.addMatcher(methodMatcher(methods)) +} + +// Path ----------------------------------------------------------------------- + +// Path adds a matcher for the URL path. +// It accepts a template with zero or more URL variables enclosed by {}. +// Variables can define an optional regexp pattern to me matched: +// +// - {name} matches anything until the next slash. +// +// - {name:pattern} matches the given regexp pattern. +// +// For example: +// +// r := mux.NewRouter() +// r.Path("/products/").Handler(ProductsHandler) +// r.Path("/products/{key}").Handler(ProductsHandler) +// r.Path("/articles/{category}/{id:[0-9]+}"). +// Handler(ArticleHandler) +// +// Variable names must be unique in a given route. They can be retrieved +// calling mux.Vars(request). +func (r *Route) Path(tpl string) *Route { + r.err = r.addRegexpMatcher(tpl, false, false) + return r +} + +// PathPrefix ----------------------------------------------------------------- + +// PathPrefix adds a matcher for the URL path prefix. +func (r *Route) PathPrefix(tpl string) *Route { + r.strictSlash = false + r.err = r.addRegexpMatcher(tpl, false, true) + return r +} + +// Query ---------------------------------------------------------------------- + +// queryMatcher matches the request against URL queries. +type queryMatcher map[string]string + +func (m queryMatcher) Match(r *http.Request, match *RouteMatch) bool { + return matchMap(m, r.URL.Query(), false) +} + +// Queries adds a matcher for URL query values. +// It accepts a sequence of key/value pairs. For example: +// +// r := mux.NewRouter() +// r.Queries("foo", "bar", "baz", "ding") +// +// The above route will only match if the URL contains the defined queries +// values, e.g.: ?foo=bar&baz=ding. +// +// It the value is an empty string, it will match any value if the key is set. +func (r *Route) Queries(pairs ...string) *Route { + if r.err == nil { + var queries map[string]string + queries, r.err = mapFromPairs(pairs...) + return r.addMatcher(queryMatcher(queries)) + } + return r +} + +// Schemes -------------------------------------------------------------------- + +// schemeMatcher matches the request against URL schemes. +type schemeMatcher []string + +func (m schemeMatcher) Match(r *http.Request, match *RouteMatch) bool { + return matchInArray(m, r.URL.Scheme) +} + +// Schemes adds a matcher for URL schemes. +// It accepts a sequence of schemes to be matched, e.g.: "http", "https". +func (r *Route) Schemes(schemes ...string) *Route { + for k, v := range schemes { + schemes[k] = strings.ToLower(v) + } + return r.addMatcher(schemeMatcher(schemes)) +} + +// Subrouter ------------------------------------------------------------------ + +// Subrouter creates a subrouter for the route. +// +// It will test the inner routes only if the parent route matched. For example: +// +// r := mux.NewRouter() +// s := r.Host("www.domain.com").Subrouter() +// s.HandleFunc("/products/", ProductsHandler) +// s.HandleFunc("/products/{key}", ProductHandler) +// s.HandleFunc("/articles/{category}/{id:[0-9]+}"), ArticleHandler) +// +// Here, the routes registered in the subrouter won't be tested if the host +// doesn't match. +func (r *Route) Subrouter() *Router { + router := &Router{parent: r, strictSlash: r.strictSlash} + r.addMatcher(router) + return router +} + +// ---------------------------------------------------------------------------- +// URL building +// ---------------------------------------------------------------------------- + +// URL builds a URL for the route. +// +// It accepts a sequence of key/value pairs for the route variables. For +// example, given this route: +// +// r := mux.NewRouter() +// r.HandleFunc("/articles/{category}/{id:[0-9]+}", ArticleHandler). +// Name("article") +// +// ...a URL for it can be built using: +// +// url, err := r.Get("article").URL("category", "technology", "id", "42") +// +// ...which will return an url.URL with the following path: +// +// "/articles/technology/42" +// +// This also works for host variables: +// +// r := mux.NewRouter() +// r.Host("{subdomain}.domain.com"). +// HandleFunc("/articles/{category}/{id:[0-9]+}", ArticleHandler). +// Name("article") +// +// // url.String() will be "http://news.domain.com/articles/technology/42" +// url, err := r.Get("article").URL("subdomain", "news", +// "category", "technology", +// "id", "42") +// +// All variables defined in the route are required, and their values must +// conform to the corresponding patterns. +func (r *Route) URL(pairs ...string) (*url.URL, error) { + if r.err != nil { + return nil, r.err + } + if r.regexp == nil { + return nil, errors.New("mux: route doesn't have a host or path") + } + var scheme, host, path string + var err error + if r.regexp.host != nil { + // Set a default scheme. + scheme = "http" + if host, err = r.regexp.host.url(pairs...); err != nil { + return nil, err + } + } + if r.regexp.path != nil { + if path, err = r.regexp.path.url(pairs...); err != nil { + return nil, err + } + } + return &url.URL{ + Scheme: scheme, + Host: host, + Path: path, + }, nil +} + +// URLHost builds the host part of the URL for a route. See Route.URL(). +// +// The route must have a host defined. +func (r *Route) URLHost(pairs ...string) (*url.URL, error) { + if r.err != nil { + return nil, r.err + } + if r.regexp == nil || r.regexp.host == nil { + return nil, errors.New("mux: route doesn't have a host") + } + host, err := r.regexp.host.url(pairs...) + if err != nil { + return nil, err + } + return &url.URL{ + Scheme: "http", + Host: host, + }, nil +} + +// URLPath builds the path part of the URL for a route. See Route.URL(). +// +// The route must have a path defined. +func (r *Route) URLPath(pairs ...string) (*url.URL, error) { + if r.err != nil { + return nil, r.err + } + if r.regexp == nil || r.regexp.path == nil { + return nil, errors.New("mux: route doesn't have a path") + } + path, err := r.regexp.path.url(pairs...) + if err != nil { + return nil, err + } + return &url.URL{ + Path: path, + }, nil +} + +// ---------------------------------------------------------------------------- +// parentRoute +// ---------------------------------------------------------------------------- + +// parentRoute allows routes to know about parent host and path definitions. +type parentRoute interface { + getNamedRoutes() map[string]*Route + getRegexpGroup() *routeRegexpGroup +} + +// getNamedRoutes returns the map where named routes are registered. +func (r *Route) getNamedRoutes() map[string]*Route { + if r.parent == nil { + // During tests router is not always set. + r.parent = NewRouter() + } + return r.parent.getNamedRoutes() +} + +// getRegexpGroup returns regexp definitions from this route. +func (r *Route) getRegexpGroup() *routeRegexpGroup { + if r.regexp == nil { + if r.parent == nil { + // During tests router is not always set. + r.parent = NewRouter() + } + regexp := r.parent.getRegexpGroup() + if regexp == nil { + r.regexp = new(routeRegexpGroup) + } else { + // Copy. + r.regexp = &routeRegexpGroup{ + host: regexp.host, + path: regexp.path, + } + } + } + return r.regexp +} From 52b67ca30784e94c1a4106c8a343291e2adbf335 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sat, 12 Oct 2013 20:35:10 -0600 Subject: [PATCH 078/247] Add gorilla/mux to third_party. --- etcd_test.go | 3 ++- third_party/deps | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/etcd_test.go b/etcd_test.go index 71222a24649..6ba6978ee0b 100644 --- a/etcd_test.go +++ b/etcd_test.go @@ -21,7 +21,7 @@ import ( func TestSingleNode(t *testing.T) { procAttr := new(os.ProcAttr) procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} - args := []string{"etcd", "-n=node1", "-f", "-d=/tmp/node1"} + args := []string{"etcd", "-vv", "-n=node1", "-f", "-d=/tmp/node1"} process, err := os.StartProcess("etcd", args, procAttr) if err != nil { @@ -32,6 +32,7 @@ func TestSingleNode(t *testing.T) { time.Sleep(time.Second) + etcd.OpenDebug() c := etcd.NewClient() c.SyncCluster() diff --git a/third_party/deps b/third_party/deps index 576081e484a..156c0afc8e4 100755 --- a/third_party/deps +++ b/third_party/deps @@ -3,6 +3,7 @@ packages=" github.com/coreos/go-etcd github.com/coreos/go-log/log github.com/coreos/go-systemd + github.com/gorilla/mux bitbucket.org/kardianos/osext code.google.com/p/go.net code.google.com/p/goprotobuf From b0793e2dd9de1ea4b99e4cf01d371a8859af5a2c Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 12 Oct 2013 21:48:29 -0700 Subject: [PATCH 079/247] refactor create do not need to check existence first --- store/store.go | 44 ++++++++++++++++++++------------------------ store/store_test.go | 13 ++++++++++++- 2 files changed, 32 insertions(+), 25 deletions(-) diff --git a/store/store.go b/store/store.go index 348aea02e4d..ade44acd992 100644 --- a/store/store.go +++ b/store/store.go @@ -245,9 +245,10 @@ func (s *Store) Watch(prefix string, recursive bool, sinceIndex uint64, index ui if err != nil { err.Index = index err.Term = term + return nil, err } - return c, err + return c, nil } // walk function walks all the nodePath and apply the walkFunc on each directory @@ -275,25 +276,14 @@ func (s *Store) walk(nodePath string, walkFunc func(prev *Node, component string func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix bool, force bool, expireTime time.Time, index uint64, term uint64, action string) (*Event, error) { + s.Index, s.Term = index, term + if incrementalSuffix { // append unique incremental suffix to the node path nodePath += "_" + strconv.FormatUint(index, 10) } nodePath = path.Clean(path.Join("/", nodePath)) - // make sure we can create the node - _, err := s.internalGet(nodePath, index, term) - - if err == nil && !force { // key already exists - s.Stats.Inc(SetFail) - return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath, index, term) - } - - if err != nil && err.ErrorCode == 104 { // we cannot create the key due to meet a file while walking - s.Stats.Inc(SetFail) - return nil, err - } - dir, newNodeName := path.Split(nodePath) // walk through the nodePath, create dirs and get the last directory node @@ -301,35 +291,37 @@ func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix if err != nil { s.Stats.Inc(SetFail) + err.Index, err.Term = s.Index, s.Term return nil, err } e := newEvent(action, nodePath, s.Index, s.Term) - if force { // force will try to replace a existing file - n, _ := d.GetChild(newNodeName) - if n != nil { + n, _ := d.GetChild(newNodeName) + + // force will try to replace a existing file + if n != nil { + if force { if n.IsDir() { return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term) } e.PrevValue, _ = n.Read() n.Remove(false, nil) - + } else { + return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath, index, term) } } - var n *Node - if len(value) != 0 { // create file e.Value = value - n = newKV(nodePath, value, s.Index, s.Term, d, "", expireTime) + n = newKV(nodePath, value, index, term, d, "", expireTime) } else { // create directory e.Dir = true - n = newDir(nodePath, s.Index, s.Term, d, "", expireTime) + n = newDir(nodePath, index, term, d, "", expireTime) } @@ -388,10 +380,14 @@ func (s *Store) internalGet(nodePath string, index uint64, term uint64) (*Node, // If it does not exist, this function will create a new directory and return the pointer to that node. // If it is a file, this function will return error. func (s *Store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) { - subDir, ok := parent.Children[dirName] + node, ok := parent.Children[dirName] if ok { - return subDir, nil + if node.IsDir() { + return node, nil + } + + return nil, etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path, UndefIndex, UndefTerm) } n := newDir(path.Join(parent.Path, dirName), s.Index, s.Term, parent, parent.ACL, Permanent) diff --git a/store/store_test.go b/store/store_test.go index 3d5e9e97005..74add4424d7 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -21,6 +21,17 @@ func TestCreateAndGet(t *testing.T) { s.Delete("/foobar", true, 1, 1) + s.Create("/foobar/foo", "bar", false, false, Permanent, 1, 1) + + // already exist, create should fail + _, err = s.Create("/foobar", "bar", false, false, Permanent, 1, 1) + + if err == nil { + t.Fatal("Create should fail") + } + + s.Delete("/foobar", true, 1, 1) + // this should create successfully createAndGet(s, "/foobar", t) createAndGet(s, "/foo/bar", t) @@ -365,7 +376,7 @@ func TestWatch(t *testing.T) { c, _ = s.Watch("/foo", true, 0, 7, 1) s.Delete("/foo/foo/boo", false, 8, 1) e = nonblockingRetrive(c) - if e.Key != "/foo/foo/boo" || e.Action != Delete { + if e == nil || e.Key != "/foo/foo/boo" || e.Action != Delete { t.Fatal("watch for Delete subdirectory fails") } From 7416d2fdcc98c9a7143c91404c53fd4afdc3b4f2 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sat, 12 Oct 2013 23:39:34 -0600 Subject: [PATCH 080/247] Mostly working. --- etcd.go | 2 +- etcd_test.go | 6 +- server/peer_server.go | 2 +- server/registry.go | 57 +++++++++----- server/server.go | 78 +++++++++++++++---- server/v2/handlers.go | 56 ------------- .../github.com/coreos/go-etcd/etcd/client.go | 1 - 7 files changed, 105 insertions(+), 97 deletions(-) diff --git a/etcd.go b/etcd.go index a224fa15486..05720fb357b 100644 --- a/etcd.go +++ b/etcd.go @@ -177,7 +177,7 @@ func main() { ps.MaxClusterSize = maxClusterSize ps.RetryTimes = retryTimes - s := server.New(info.Name, info.EtcdURL, info.EtcdListenHost, &etcdTLSConfig, &info.EtcdTLS, ps.Server, registry, store) + s := server.New(info.Name, info.EtcdURL, info.EtcdListenHost, &etcdTLSConfig, &info.EtcdTLS, ps, registry, store) if err := s.AllowOrigins(cors); err != nil { panic(err) } diff --git a/etcd_test.go b/etcd_test.go index 6ba6978ee0b..d868be89667 100644 --- a/etcd_test.go +++ b/etcd_test.go @@ -41,7 +41,7 @@ func TestSingleNode(t *testing.T) { if err != nil || result.Key != "/foo" || result.Value != "bar" || result.TTL < 95 { if err != nil { - t.Fatal(err) + t.Fatal("Set 1: ", err) } t.Fatalf("Set 1 failed with %s %s %v", result.Key, result.Value, result.TTL) @@ -53,7 +53,7 @@ func TestSingleNode(t *testing.T) { if err != nil || result.Key != "/foo" || result.Value != "bar" || result.PrevValue != "bar" || result.TTL != 100 { if err != nil { - t.Fatal(err) + t.Fatal("Set 2: ", err) } t.Fatalf("Set 2 failed with %s %s %v", result.Key, result.Value, result.TTL) } @@ -295,7 +295,7 @@ func TestMultiNodeKillAllAndRecovery(t *testing.T) { result, err := c.Set("foo", "bar", 0) if err != nil { - panic(err) + t.Fatalf("Recovery error: %s", err) } if result.Index != 18 { diff --git a/server/peer_server.go b/server/peer_server.go index ba698b55303..d016717b17f 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -116,7 +116,7 @@ func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) { } else { // Rejoin the previous cluster - cluster = s.registry.PeerURLs() + cluster = s.registry.PeerURLs(s.Leader(), s.name) for i := 0; i < len(cluster); i++ { u, err := url.Parse(cluster[i]) if err != nil { diff --git a/server/registry.go b/server/registry.go index 25ff936b7e9..8a62dff4081 100644 --- a/server/registry.go +++ b/server/registry.go @@ -4,8 +4,10 @@ import ( "fmt" "net/url" "path" + "strings" "sync" + "github.com/coreos/etcd/log" "github.com/coreos/etcd/store" ) @@ -84,28 +86,34 @@ func (r *Registry) url(name string) (string, bool) { } // Retrieves the URLs for all nodes. -func (r *Registry) URLs() []string { +func (r *Registry) URLs(leaderName, selfName string) []string { r.Lock() defer r.Unlock() - // Retrieve a list of all nodes. - e, err := r.store.Get(RegistryKey, false, false, 0, 0) - if err != nil { - return make([]string, 0) + // Build list including the leader and self. + urls := make([]string, 0) + if url, _ := r.url(leaderName); len(url) > 0 { + urls = append(urls, url) + } + if url, _ := r.url(selfName); len(url) > 0 { + urls = append(urls, url) } - // Lookup the URL for each one. - urls := make([]string, 0) - for _, pair := range e.KVPairs { - if url, ok := r.url(pair.Key); ok { - urls = append(urls, url) + // Retrieve a list of all nodes. + if e, _ := r.store.Get(RegistryKey, false, false, 0, 0); e != nil { + // Lookup the URL for each one. + for _, pair := range e.KVPairs { + if url, _ := r.url(pair.Key); len(url) > 0 { + urls = append(urls, url) + } } } + log.Infof("URLs: %s / %s (%s", leaderName, selfName, strings.Join(urls, ",")) + return urls } - // Retrieves the peer URL for a given node by name. func (r *Registry) PeerURL(name string) (string, bool) { r.Lock() @@ -126,24 +134,31 @@ func (r *Registry) peerURL(name string) (string, bool) { } // Retrieves the peer URLs for all nodes. -func (r *Registry) PeerURLs() []string { +func (r *Registry) PeerURLs(leaderName, selfName string) []string { r.Lock() defer r.Unlock() - // Retrieve a list of all nodes. - e, err := r.store.Get(RegistryKey, false, false, 0, 0) - if err != nil { - return make([]string, 0) + // Build list including the leader and self. + urls := make([]string, 0) + if url, _ := r.peerURL(leaderName); len(url) > 0 { + urls = append(urls, url) + } + if url, _ := r.peerURL(selfName); len(url) > 0 { + urls = append(urls, url) } - // Lookup the URL for each one. - urls := make([]string, 0) - for _, pair := range e.KVPairs { - if url, ok := r.peerURL(pair.Key); ok { - urls = append(urls, url) + // Retrieve a list of all nodes. + if e, _ := r.store.Get(RegistryKey, false, false, 0, 0); e != nil { + // Lookup the URL for each one. + for _, pair := range e.KVPairs { + if url, _ := r.peerURL(pair.Key); len(url) > 0 { + urls = append(urls, url) + } } } + log.Infof("PeerURLs: %s / %s (%s", leaderName, selfName, strings.Join(urls, ",")) + return urls } diff --git a/server/server.go b/server/server.go index 586502378bb..f1adcbf86e8 100644 --- a/server/server.go +++ b/server/server.go @@ -18,9 +18,9 @@ import ( // This is the default implementation of the Server interface. type Server struct { http.Server - raftServer *raft.Server - registry *Registry - store *store.Store + peerServer *PeerServer + registry *Registry + store *store.Store name string url string tlsConf *TLSConfig @@ -29,7 +29,7 @@ type Server struct { } // Creates a new Server. -func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, raftServer *raft.Server, registry *Registry, store *store.Store) *Server { +func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, peerServer *PeerServer, registry *Registry, store *store.Store) *Server { s := &Server{ Server: http.Server{ Handler: mux.NewRouter(), @@ -41,7 +41,7 @@ func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsI url: urlStr, tlsConf: tlsConf, tlsInfo: tlsInfo, - raftServer: raftServer, + peerServer: peerServer, } // Install the routes for each version of the API. @@ -52,12 +52,12 @@ func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsI // The current Raft committed index. func (s *Server) CommitIndex() uint64 { - return s.raftServer.CommitIndex() + return s.peerServer.CommitIndex() } // The current Raft term. func (s *Server) Term() uint64 { - return s.raftServer.Term() + return s.peerServer.Term() } // The server URL. @@ -74,19 +74,21 @@ func (s *Server) installV1() { s.handleFuncV1("/v1/keys/{key:.*}", v1.GetKeyHandler).Methods("GET") s.handleFuncV1("/v1/keys/{key:.*}", v1.SetKeyHandler).Methods("POST", "PUT") s.handleFuncV1("/v1/keys/{key:.*}", v1.DeleteKeyHandler).Methods("DELETE") - s.handleFuncV1("/v1/watch/{key:.*}", v1.WatchKeyHandler).Methods("GET", "POST") + s.handleFunc("/v1/leader", s.GetLeaderHandler).Methods("GET") + s.handleFunc("/v1/machines", s.GetMachinesHandler).Methods("GET") + s.handleFunc("/v1/stats", s.GetStatsHandler).Methods("GET") } // Adds a v1 server handler to the router. func (s *Server) handleFuncV1(path string, f func(http.ResponseWriter, *http.Request, v1.Server) error) *mux.Route { - return s.handleFunc(path, func(w http.ResponseWriter, req *http.Request, s *Server) error { + return s.handleFunc(path, func(w http.ResponseWriter, req *http.Request) error { return f(w, req, s) }) } // Adds a server handler to the router. -func (s *Server) handleFunc(path string, f func(http.ResponseWriter, *http.Request, *Server) error) *mux.Route { +func (s *Server) handleFunc(path string, f func(http.ResponseWriter, *http.Request) error) *mux.Route { r := s.Handler.(*mux.Router) // Wrap the standard HandleFunc interface to pass in the server reference. @@ -102,7 +104,7 @@ func (s *Server) handleFunc(path string, f func(http.ResponseWriter, *http.Reque } // Execute handler function and return error if necessary. - if err := f(w, req, s); err != nil { + if err := f(w, req); err != nil { if etcdErr, ok := err.(*etcdErr.Error); ok { log.Debug("Return error: ", (*etcdErr).Error()) etcdErr.Write(w) @@ -125,8 +127,8 @@ func (s *Server) ListenAndServe() { } func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Request) error { - if s.raftServer.State() == raft.Leader { - event, err := s.raftServer.Do(c) + if s.peerServer.State() == raft.Leader { + event, err := s.peerServer.Do(c) if err != nil { return err } @@ -143,7 +145,7 @@ func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Reque return nil } else { - leader := s.raftServer.Leader() + leader := s.peerServer.Leader() // No leader available. if leader == "" { @@ -178,3 +180,51 @@ func (s *Server) AllowOrigins(origins string) error { func (s *Server) OriginAllowed(origin string) bool { return s.corsOrigins["*"] || s.corsOrigins[origin] } + +// Handler to return the current leader's raft address +func (s *Server) GetLeaderHandler(w http.ResponseWriter, req *http.Request) error { + leader := s.peerServer.Leader() + if leader == "" { + return etcdErr.NewError(etcdErr.EcodeLeaderElect, "", store.UndefIndex, store.UndefTerm) + } + w.WriteHeader(http.StatusOK) + url, _ := s.registry.PeerURL(leader) + w.Write([]byte(url)) + return nil +} + +// Handler to return all the known machines in the current cluster. +func (s *Server) GetMachinesHandler(w http.ResponseWriter, req *http.Request) error { + machines := s.registry.URLs(s.peerServer.Leader(), s.name) + w.WriteHeader(http.StatusOK) + w.Write([]byte(strings.Join(machines, ", "))) + return nil +} + +// Retrieves stats on the Raft server. +func (s *Server) GetStatsHandler(w http.ResponseWriter, req *http.Request) error { + w.Write(s.peerServer.Stats()) + return nil +} + +// Retrieves stats on the leader. +func (s *Server) GetLeaderStatsHandler(w http.ResponseWriter, req *http.Request) error { + if s.peerServer.State() == raft.Leader { + w.Write(s.peerServer.PeerStats()) + return nil + } + + leader := s.peerServer.Leader() + if leader == "" { + return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) + } + hostname, _ := s.registry.URL(leader) + redirect(hostname, w, req) + return nil +} + +// Retrieves stats on the leader. +func (s *Server) GetStoreStatsHandler(w http.ResponseWriter, req *http.Request) error { + w.Write(s.store.JsonStats()) + return nil +} diff --git a/server/v2/handlers.go b/server/v2/handlers.go index 7fbd7e1d7e5..d495420052e 100644 --- a/server/v2/handlers.go +++ b/server/v2/handlers.go @@ -202,33 +202,6 @@ func (e *etcdServer) dispatchEtcdCommand(c Command, w http.ResponseWriter, req * // still dispatch to the leader //-------------------------------------- -// Handler to return the current leader's raft address -func (e *etcdServer) LeaderHttpHandler(w http.ResponseWriter, req *http.Request) error { - r := e.raftServer - - leader := r.Leader() - - if leader != "" { - w.WriteHeader(http.StatusOK) - raftURL, _ := nameToRaftURL(leader) - w.Write([]byte(raftURL)) - - return nil - } else { - return etcdErr.NewError(etcdErr.EcodeLeaderElect, "", store.UndefIndex, store.UndefTerm) - } -} - -// Handler to return all the known machines in the current cluster -func (e *etcdServer) MachinesHttpHandler(w http.ResponseWriter, req *http.Request) error { - machines := e.raftServer.getMachines(nameToEtcdURL) - - w.WriteHeader(http.StatusOK) - w.Write([]byte(strings.Join(machines, ", "))) - - return nil -} - // Handler to return the current version of etcd func (e *etcdServer) VersionHttpHandler(w http.ResponseWriter, req *http.Request) error { w.WriteHeader(http.StatusOK) @@ -237,35 +210,6 @@ func (e *etcdServer) VersionHttpHandler(w http.ResponseWriter, req *http.Request return nil } -// Handler to return the basic stats of etcd -func (e *etcdServer) StatsHttpHandler(w http.ResponseWriter, req *http.Request) error { - option := req.URL.Path[len("/v1/stats/"):] - w.WriteHeader(http.StatusOK) - - r := e.raftServer - - switch option { - case "self": - w.Write(r.Stats()) - case "leader": - if r.State() == raft.Leader { - w.Write(r.PeerStats()) - } else { - leader := r.Leader() - // current no leader - if leader == "" { - return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) - } - hostname, _ := nameToEtcdURL(leader) - redirect(hostname, w, req) - } - case "store": - w.Write(etcdStore.JsonStats()) - } - - return nil -} - func (e *etcdServer) GetHttpHandler(w http.ResponseWriter, req *http.Request) error { var err error var event interface{} diff --git a/third_party/github.com/coreos/go-etcd/etcd/client.go b/third_party/github.com/coreos/go-etcd/etcd/client.go index abc4574fa5f..cbbd7ad9c5a 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/client.go +++ b/third_party/github.com/coreos/go-etcd/etcd/client.go @@ -136,7 +136,6 @@ func (c *Client) internalSyncCluster(machines []string) bool { logger.Debugf("update.leader[%s,%s]", c.cluster.Leader, c.cluster.Machines[0]) c.cluster.Leader = c.cluster.Machines[0] - logger.Debug("sync.machines ", c.cluster.Machines) return true } } From d3b064c2e9775374012070cb7d2a5ec015df2e48 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sun, 13 Oct 2013 00:29:58 -0600 Subject: [PATCH 081/247] Refactor v2 routes. --- server/server.go | 78 +++++++- server/v2/create_key_handler.go | 29 +++ server/v2/delete_key_handler.go | 20 ++ server/v2/get_key_handler.go | 69 +++++++ server/v2/handlers.go | 336 -------------------------------- server/v2/update_key_handler.go | 64 ++++++ server/v2/v2.go | 18 ++ 7 files changed, 276 insertions(+), 338 deletions(-) create mode 100644 server/v2/create_key_handler.go create mode 100644 server/v2/delete_key_handler.go create mode 100644 server/v2/get_key_handler.go delete mode 100644 server/v2/handlers.go create mode 100644 server/v2/update_key_handler.go create mode 100644 server/v2/v2.go diff --git a/server/server.go b/server/server.go index f1adcbf86e8..98492711020 100644 --- a/server/server.go +++ b/server/server.go @@ -6,10 +6,12 @@ import ( "net/http" "net/url" "strings" + "time" etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/log" "github.com/coreos/etcd/server/v1" + "github.com/coreos/etcd/server/v2" "github.com/coreos/etcd/store" "github.com/coreos/go-raft" "github.com/gorilla/mux" @@ -44,12 +46,24 @@ func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsI peerServer: peerServer, } - // Install the routes for each version of the API. + // Install the routes. + s.handleFunc("/version", s.GetVersionHandler).Methods("GET") s.installV1() + s.installV2() return s } +// The current state of the server in the cluster. +func (s *Server) State() string { + return s.peerServer.State() +} + +// The node name of the leader in the cluster. +func (s *Server) Leader() string { + return s.peerServer.Leader() +} + // The current Raft committed index. func (s *Server) CommitIndex() uint64 { return s.peerServer.CommitIndex() @@ -65,6 +79,11 @@ func (s *Server) URL() string { return s.url } +// Retrives the Peer URL for a given node name. +func (s *Server) PeerURL(name string) (string, bool) { + return s.registry.PeerURL(name) +} + // Returns a reference to the Store. func (s *Server) Store() *store.Store { return s.store @@ -77,7 +96,21 @@ func (s *Server) installV1() { s.handleFuncV1("/v1/watch/{key:.*}", v1.WatchKeyHandler).Methods("GET", "POST") s.handleFunc("/v1/leader", s.GetLeaderHandler).Methods("GET") s.handleFunc("/v1/machines", s.GetMachinesHandler).Methods("GET") - s.handleFunc("/v1/stats", s.GetStatsHandler).Methods("GET") + s.handleFunc("/v1/stats/self", s.GetStatsHandler).Methods("GET") + s.handleFunc("/v1/stats/leader", s.GetLeaderStatsHandler).Methods("GET") + s.handleFunc("/v1/stats/store", s.GetStoreStatsHandler).Methods("GET") +} + +func (s *Server) installV2() { + s.handleFuncV2("/v2/keys/{key:.*}", v2.GetKeyHandler).Methods("GET") + s.handleFuncV2("/v2/keys/{key:.*}", v2.CreateKeyHandler).Methods("POST") + s.handleFuncV2("/v2/keys/{key:.*}", v2.UpdateKeyHandler).Methods("PUT") + s.handleFuncV2("/v2/keys/{key:.*}", v2.DeleteKeyHandler).Methods("DELETE") + s.handleFunc("/v2/leader", s.GetLeaderHandler).Methods("GET") + s.handleFunc("/v2/machines", s.GetMachinesHandler).Methods("GET") + s.handleFunc("/v2/stats/self", s.GetStatsHandler).Methods("GET") + s.handleFunc("/v2/stats/leader", s.GetLeaderStatsHandler).Methods("GET") + s.handleFunc("/v2/stats/store", s.GetStoreStatsHandler).Methods("GET") } // Adds a v1 server handler to the router. @@ -87,6 +120,13 @@ func (s *Server) handleFuncV1(path string, f func(http.ResponseWriter, *http.Req }) } +// Adds a v2 server handler to the router. +func (s *Server) handleFuncV2(path string, f func(http.ResponseWriter, *http.Request, v2.Server) error) *mux.Route { + return s.handleFunc(path, func(w http.ResponseWriter, req *http.Request) error { + return f(w, req, s) + }) +} + // Adds a server handler to the router. func (s *Server) handleFunc(path string, f func(http.ResponseWriter, *http.Request) error) *mux.Route { r := s.Handler.(*mux.Router) @@ -181,6 +221,13 @@ func (s *Server) OriginAllowed(origin string) bool { return s.corsOrigins["*"] || s.corsOrigins[origin] } +// Handler to return the current version of etcd. +func (s *Server) GetVersionHandler(w http.ResponseWriter, req *http.Request) error { + w.WriteHeader(http.StatusOK) + fmt.Fprintf(w, "etcd %s", releaseVersion) + return nil +} + // Handler to return the current leader's raft address func (s *Server) GetLeaderHandler(w http.ResponseWriter, req *http.Request) error { leader := s.peerServer.Leader() @@ -228,3 +275,30 @@ func (s *Server) GetStoreStatsHandler(w http.ResponseWriter, req *http.Request) w.Write(s.store.JsonStats()) return nil } + +// Executes a speed test to evaluate the performance of update replication. +func (s *Server) SpeedTestHandler(w http.ResponseWriter, req *http.Request) error { + count := 1000 + c := make(chan bool, count) + for i := 0; i < count; i++ { + go func() { + for j := 0; j < 10; j++ { + c := &store.UpdateCommand{ + Key: "foo", + Value: "bar", + ExpireTime: time.Unix(0, 0), + } + s.peerServer.Do(c) + } + c <- true + }() + } + + for i := 0; i < count; i++ { + <-c + } + + w.WriteHeader(http.StatusOK) + w.Write([]byte("speed test success")) + return nil +} diff --git a/server/v2/create_key_handler.go b/server/v2/create_key_handler.go new file mode 100644 index 00000000000..8a9fbf50e85 --- /dev/null +++ b/server/v2/create_key_handler.go @@ -0,0 +1,29 @@ +package v2 + +import ( + "net/http" + + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/store" + "github.com/gorilla/mux" +) + +func CreateKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { + vars := mux.Vars(req) + key := "/" + vars["key"] + + value := req.FormValue("value") + expireTime, err := store.TTL(req.FormValue("ttl")) + if err != nil { + return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create", store.UndefIndex, store.UndefTerm) + } + + c := &store.CreateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + IncrementalSuffix: (req.FormValue("incremental") == "true"), + } + + return s.Dispatch(c, w, req) +} diff --git a/server/v2/delete_key_handler.go b/server/v2/delete_key_handler.go new file mode 100644 index 00000000000..e3bdf2b2d27 --- /dev/null +++ b/server/v2/delete_key_handler.go @@ -0,0 +1,20 @@ +package v2 + +import ( + "net/http" + + "github.com/coreos/etcd/store" + "github.com/gorilla/mux" +) + +func DeleteKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { + vars := mux.Vars(req) + key := "/" + vars["key"] + + c := &store.DeleteCommand{ + Key: key, + Recursive: (req.FormValue("recursive") == "true"), + } + + return s.Dispatch(c, w, req) +} diff --git a/server/v2/get_key_handler.go b/server/v2/get_key_handler.go new file mode 100644 index 00000000000..e4d9b7207e3 --- /dev/null +++ b/server/v2/get_key_handler.go @@ -0,0 +1,69 @@ +package v2 + +import ( + "encoding/json" + "fmt" + "net/http" + "strconv" + + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" + "github.com/gorilla/mux" +) + +func GetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { + var err error + var event *store.Event + + vars := mux.Vars(req) + key := "/" + vars["key"] + + // Help client to redirect the request to the current leader + if req.FormValue("consistent") == "true" && s.State() != raft.Leader { + leader := s.Leader() + hostname, _ := s.PeerURL(leader) + url := hostname + req.URL.Path + log.Debugf("Redirect to %s", url) + http.Redirect(w, req, url, http.StatusTemporaryRedirect) + return nil + } + + recursive := (req.FormValue("recursive") == "true") + sorted := (req.FormValue("sorted") == "true") + + if req.FormValue("wait") == "true" { // watch + // Create a command to watch from a given index (default 0). + var sinceIndex uint64 = 0 + if req.Method == "POST" { + sinceIndex, err = strconv.ParseUint(string(req.FormValue("wait_index")), 10, 64) + if err != nil { + return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index", store.UndefIndex, store.UndefTerm) + } + } + + // Start the watcher on the store. + c, err := s.Store().Watch(key, recursive, sinceIndex, s.CommitIndex(), s.Term()) + if err != nil { + return etcdErr.NewError(500, key, store.UndefIndex, store.UndefTerm) + } + event = <-c + + } else { //get + // Retrieve the key from the store. + event, err = s.Store().Get(key, recursive, sorted, s.CommitIndex(), s.Term()) + if err != nil { + return err + } + } + + w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) + w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term)) + w.WriteHeader(http.StatusOK) + + b, _ := json.Marshal(event) + w.Write(b) + + return nil +} diff --git a/server/v2/handlers.go b/server/v2/handlers.go deleted file mode 100644 index d495420052e..00000000000 --- a/server/v2/handlers.go +++ /dev/null @@ -1,336 +0,0 @@ -package main - -import ( - "encoding/json" - "fmt" - "net/http" - "strconv" - "strings" - - etcdErr "github.com/coreos/etcd/error" - "github.com/coreos/etcd/store" - "github.com/coreos/go-raft" -) - -func NewEtcdMuxer() *http.ServeMux { - // external commands - router := mux.NewRouter() - etcdMux.Handle("/v2/keys/", errorHandler(e.Multiplexer)) - etcdMux.Handle("/v2/leader", errorHandler(e.LeaderHttpHandler)) - etcdMux.Handle("/v2/machines", errorHandler(e.MachinesHttpHandler)) - etcdMux.Handle("/v2/stats/", errorHandler(e.StatsHttpHandler)) - etcdMux.Handle("/version", errorHandler(e.VersionHttpHandler)) - etcdMux.HandleFunc("/test/", TestHttpHandler) - - // backward support - etcdMux.Handle("/v1/keys/", errorHandler(e.MultiplexerV1)) - etcdMux.Handle("/v1/leader", errorHandler(e.LeaderHttpHandler)) - etcdMux.Handle("/v1/machines", errorHandler(e.MachinesHttpHandler)) - etcdMux.Handle("/v1/stats/", errorHandler(e.StatsHttpHandler)) - - return etcdMux -} - -type errorHandler func(http.ResponseWriter, *http.Request) error - -// addCorsHeader parses the request Origin header and loops through the user -// provided allowed origins and sets the Access-Control-Allow-Origin header if -// there is a match. -func addCorsHeader(w http.ResponseWriter, r *http.Request) { - val, ok := corsList["*"] - if val && ok { - w.Header().Add("Access-Control-Allow-Origin", "*") - return - } - - requestOrigin := r.Header.Get("Origin") - val, ok = corsList[requestOrigin] - if val && ok { - w.Header().Add("Access-Control-Allow-Origin", requestOrigin) - return - } -} - -func (fn errorHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { - addCorsHeader(w, r) - if e := fn(w, r); e != nil { - if etcdErr, ok := e.(*etcdErr.Error); ok { - debug("Return error: ", (*etcdErr).Error()) - etcdErr.Write(w) - } else { - http.Error(w, e.Error(), http.StatusInternalServerError) - } - } -} - -// Multiplex GET/POST/DELETE request to corresponding handlers -func (e *etcdServer) Multiplexer(w http.ResponseWriter, req *http.Request) error { - - switch req.Method { - case "GET": - return e.GetHttpHandler(w, req) - case "POST": - return e.CreateHttpHandler(w, req) - case "PUT": - return e.UpdateHttpHandler(w, req) - case "DELETE": - return e.DeleteHttpHandler(w, req) - default: - w.WriteHeader(http.StatusMethodNotAllowed) - return nil - } - - return nil -} - -//-------------------------------------- -// State sensitive handlers -// Set/Delete will dispatch to leader -//-------------------------------------- - -func (e *etcdServer) CreateHttpHandler(w http.ResponseWriter, req *http.Request) error { - key := getNodePath(req.URL.Path) - - debugf("recv.post[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) - - value := req.FormValue("value") - - expireTime, err := store.TTL(req.FormValue("ttl")) - - if err != nil { - return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create", store.UndefIndex, store.UndefTerm) - } - - command := &CreateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - } - - if req.FormValue("incremental") == "true" { - command.IncrementalSuffix = true - } - - return e.dispatchEtcdCommand(command, w, req) - -} - -func (e *etcdServer) UpdateHttpHandler(w http.ResponseWriter, req *http.Request) error { - key := getNodePath(req.URL.Path) - - debugf("recv.put[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) - - req.ParseForm() - - value := req.Form.Get("value") - - expireTime, err := store.TTL(req.Form.Get("ttl")) - - if err != nil { - return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", store.UndefIndex, store.UndefTerm) - } - - // update should give at least one option - if value == "" && expireTime.Sub(store.Permanent) == 0 { - return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", store.UndefIndex, store.UndefTerm) - } - - prevValue, valueOk := req.Form["prevValue"] - - prevIndexStr, indexOk := req.Form["prevIndex"] - - if !valueOk && !indexOk { // update without test - command := &UpdateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - } - - return e.dispatchEtcdCommand(command, w, req) - - } else { // update with test - var prevIndex uint64 - - if indexOk { - prevIndex, err = strconv.ParseUint(prevIndexStr[0], 10, 64) - - // bad previous index - if err != nil { - return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Update", store.UndefIndex, store.UndefTerm) - } - } else { - prevIndex = 0 - } - - command := &TestAndSetCommand{ - Key: key, - Value: value, - PrevValue: prevValue[0], - PrevIndex: prevIndex, - } - - return e.dispatchEtcdCommand(command, w, req) - } - -} - -// Delete Handler -func (e *etcdServer) DeleteHttpHandler(w http.ResponseWriter, req *http.Request) error { - key := getNodePath(req.URL.Path) - - debugf("recv.delete[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) - - command := &DeleteCommand{ - Key: key, - } - - if req.FormValue("recursive") == "true" { - command.Recursive = true - } - - return e.dispatchEtcdCommand(command, w, req) -} - -// Dispatch the command to leader -func (e *etcdServer) dispatchEtcdCommand(c Command, w http.ResponseWriter, req *http.Request) error { - return e.raftServer.dispatch(c, w, req, nameToEtcdURL) -} - -//-------------------------------------- -// State non-sensitive handlers -// command with consistent option will -// still dispatch to the leader -//-------------------------------------- - -// Handler to return the current version of etcd -func (e *etcdServer) VersionHttpHandler(w http.ResponseWriter, req *http.Request) error { - w.WriteHeader(http.StatusOK) - fmt.Fprintf(w, "etcd %s", releaseVersion) - - return nil -} - -func (e *etcdServer) GetHttpHandler(w http.ResponseWriter, req *http.Request) error { - var err error - var event interface{} - - r := e.raftServer - - debugf("recv.get[%v] [%v%v]\n", req.RemoteAddr, req.Host, req.URL) - - if req.FormValue("consistent") == "true" && r.State() != raft.Leader { - // help client to redirect the request to the current leader - leader := r.Leader() - hostname, _ := nameToEtcdURL(leader) - redirect(hostname, w, req) - return nil - } - - key := getNodePath(req.URL.Path) - - recursive := req.FormValue("recursive") - - if req.FormValue("wait") == "true" { // watch - command := &WatchCommand{ - Key: key, - } - - if recursive == "true" { - command.Recursive = true - } - - indexStr := req.FormValue("wait_index") - if indexStr != "" { - sinceIndex, err := strconv.ParseUint(indexStr, 10, 64) - - if err != nil { - return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index", store.UndefIndex, store.UndefTerm) - } - - command.SinceIndex = sinceIndex - } - - event, err = command.Apply(r.Server) - - } else { //get - - command := &GetCommand{ - Key: key, - } - - sorted := req.FormValue("sorted") - if sorted == "true" { - command.Sorted = true - } - - if recursive == "true" { - command.Recursive = true - } - - event, err = command.Apply(r.Server) - } - - if err != nil { - return err - - } else { - event, _ := event.(*store.Event) - bytes, _ := json.Marshal(event) - - w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) - w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term)) - w.WriteHeader(http.StatusOK) - - w.Write(bytes) - - return nil - } - -} - -func getNodePath(urlPath string) string { - pathPrefixLen := len("/" + version + "/keys") - return urlPath[pathPrefixLen:] -} - - -//-------------------------------------- -// Testing -//-------------------------------------- - -// TestHandler -func TestHttpHandler(w http.ResponseWriter, req *http.Request) { - testType := req.URL.Path[len("/test/"):] - - if testType == "speed" { - directSet() - w.WriteHeader(http.StatusOK) - w.Write([]byte("speed test success")) - - return - } - - w.WriteHeader(http.StatusBadRequest) -} - -func directSet() { - c := make(chan bool, 1000) - for i := 0; i < 1000; i++ { - go send(c) - } - - for i := 0; i < 1000; i++ { - <-c - } -} - -func send(c chan bool) { - for i := 0; i < 10; i++ { - command := &UpdateCommand{} - command.Key = "foo" - command.Value = "bar" - command.ExpireTime = time.Unix(0, 0) - //r.Do(command) - } - c <- true -} diff --git a/server/v2/update_key_handler.go b/server/v2/update_key_handler.go new file mode 100644 index 00000000000..64e60cca5f1 --- /dev/null +++ b/server/v2/update_key_handler.go @@ -0,0 +1,64 @@ +package v2 + +import ( + "net/http" + "strconv" + + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" + "github.com/gorilla/mux" +) + +func UpdateKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { + vars := mux.Vars(req) + key := "/" + vars["key"] + + req.ParseForm() + + value := req.Form.Get("value") + expireTime, err := store.TTL(req.Form.Get("ttl")) + if err != nil { + return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", store.UndefIndex, store.UndefTerm) + } + + // Update should give at least one option + if value == "" && expireTime.Sub(store.Permanent) == 0 { + return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", store.UndefIndex, store.UndefTerm) + } + + prevValue, valueOk := req.Form["prevValue"] + prevIndexStr, indexOk := req.Form["prevIndex"] + + var c raft.Command + if !valueOk && !indexOk { // update without test + c = &store.UpdateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + } + + } else { // update with test + var prevIndex uint64 + + if indexOk { + prevIndex, err = strconv.ParseUint(prevIndexStr[0], 10, 64) + + // bad previous index + if err != nil { + return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Update", store.UndefIndex, store.UndefTerm) + } + } else { + prevIndex = 0 + } + + c = &store.TestAndSetCommand{ + Key: key, + Value: value, + PrevValue: prevValue[0], + PrevIndex: prevIndex, + } + } + + return s.Dispatch(c, w, req) +} diff --git a/server/v2/v2.go b/server/v2/v2.go new file mode 100644 index 00000000000..439f6078e42 --- /dev/null +++ b/server/v2/v2.go @@ -0,0 +1,18 @@ +package v2 + +import ( + "net/http" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" +) + +// The Server interface provides all the methods required for the v2 API. +type Server interface { + State() string + Leader() string + CommitIndex() uint64 + Term() uint64 + PeerURL(string) (string, bool) + Store() *store.Store + Dispatch(raft.Command, http.ResponseWriter, *http.Request) error +} From bd893986b26729960752e2551c3e526951d936ac Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Sun, 13 Oct 2013 10:05:00 -0700 Subject: [PATCH 082/247] bump(code.google.com/p/goprotobuf): 61664b8425f3 --- .../p/goprotobuf/proto/all_test.go | 160 +++++--- .../p/goprotobuf/proto/decode.go | 2 +- .../p/goprotobuf/proto/encode.go | 345 +++++++++++++++++- .../p/goprotobuf/proto/extensions.go | 24 ++ .../code.google.com/p/goprotobuf/proto/lib.go | 5 +- .../p/goprotobuf/proto/properties.go | 51 ++- .../p/goprotobuf/proto/size.go | 193 ---------- .../p/goprotobuf/proto/testdata/test.pb.go | 15 - .../p/goprotobuf/proto/text.go | 2 +- .../protoc-gen-go/descriptor/descriptor.pb.go | 12 - .../protoc-gen-go/generator/generator.go | 6 - .../protoc-gen-go/generator/name_test.go | 56 +++ .../protoc-gen-go/testdata/my_test/test.pb.go | 12 - .../testdata/my_test/test.pb.go.golden | 12 - 14 files changed, 563 insertions(+), 332 deletions(-) delete mode 100644 third_party/code.google.com/p/goprotobuf/proto/size.go create mode 100644 third_party/code.google.com/p/goprotobuf/protoc-gen-go/generator/name_test.go diff --git a/third_party/code.google.com/p/goprotobuf/proto/all_test.go b/third_party/code.google.com/p/goprotobuf/proto/all_test.go index eec58410407..73179e4dfe2 100644 --- a/third_party/code.google.com/p/goprotobuf/proto/all_test.go +++ b/third_party/code.google.com/p/goprotobuf/proto/all_test.go @@ -1382,7 +1382,7 @@ func TestJSON(t *testing.T) { }, Bikeshed: MyMessage_GREEN.Enum(), } - const expected = `{"count":4,"pet":["bunny","kitty"],"inner":{"host":"cauchy"},"bikeshed":"GREEN"}` + const expected = `{"count":4,"pet":["bunny","kitty"],"inner":{"host":"cauchy"},"bikeshed":1}` b, err := json.Marshal(m) if err != nil { @@ -1401,8 +1401,8 @@ func TestJSON(t *testing.T) { t.Fatalf("got %s, want %s", received, m) } - // Test unmarshalling of older json wire format. - const old = `{"count":4,"pet":["bunny","kitty"],"inner":{"host":"cauchy"},"bikeshed":1}` + // Test unmarshalling of JSON with symbolic enum name. + const old = `{"count":4,"pet":["bunny","kitty"],"inner":{"host":"cauchy"},"bikeshed":"GREEN"}` received.Reset() if err := json.Unmarshal([]byte(old), received); err != nil { t.Fatalf("json.Unmarshal failed: %v", err) @@ -1670,7 +1670,7 @@ func TestEncodingSizes(t *testing.T) { } } -func TestErrRequiredNotSet(t *testing.T) { +func TestRequiredNotSetError(t *testing.T) { pb := initGoTest(false) pb.RequiredField.Label = nil pb.F_Int32Required = nil @@ -1695,8 +1695,8 @@ func TestErrRequiredNotSet(t *testing.T) { o := old() bytes, err := Marshal(pb) - if _, ok := err.(*ErrRequiredNotSet); !ok { - fmt.Printf("marshal-1 err = %v, want *ErrRequiredNotSet", err) + if _, ok := err.(*RequiredNotSetError); !ok { + fmt.Printf("marshal-1 err = %v, want *RequiredNotSetError", err) o.DebugPrint("", bytes) t.Fatalf("expected = %s", expected) } @@ -1711,8 +1711,8 @@ func TestErrRequiredNotSet(t *testing.T) { // Now test Unmarshal by recreating the original buffer. pbd := new(GoTest) err = Unmarshal(bytes, pbd) - if _, ok := err.(*ErrRequiredNotSet); !ok { - t.Fatalf("unmarshal err = %v, want *ErrRequiredNotSet", err) + if _, ok := err.(*RequiredNotSetError); !ok { + t.Fatalf("unmarshal err = %v, want *RequiredNotSetError", err) o.DebugPrint("", bytes) t.Fatalf("string = %s", expected) } @@ -1720,8 +1720,8 @@ func TestErrRequiredNotSet(t *testing.T) { t.Errorf("unmarshal wrong err msg: %v", err) } bytes, err = Marshal(pbd) - if _, ok := err.(*ErrRequiredNotSet); !ok { - t.Errorf("marshal-2 err = %v, want *ErrRequiredNotSet", err) + if _, ok := err.(*RequiredNotSetError); !ok { + t.Errorf("marshal-2 err = %v, want *RequiredNotSetError", err) o.DebugPrint("", bytes) t.Fatalf("string = %s", expected) } @@ -1747,78 +1747,122 @@ func fuzzUnmarshal(t *testing.T, data []byte) { Unmarshal(data, pb) } -func benchmarkMsg(bytes bool) *GoTest { +// Benchmarks + +func testMsg() *GoTest { pb := initGoTest(true) - if bytes { - buf := make([]byte, 4000) - for i := range buf { - buf[i] = byte(i) - } - pb.F_BytesDefaulted = buf - } else { - const N = 1000 // Internally the library starts much smaller. - pb.F_Int32Repeated = make([]int32, N) - pb.F_DoubleRepeated = make([]float64, N) - for i := 0; i < N; i++ { - pb.F_Int32Repeated[i] = int32(i) - pb.F_DoubleRepeated[i] = float64(i) - } + const N = 1000 // Internally the library starts much smaller. + pb.F_Int32Repeated = make([]int32, N) + pb.F_DoubleRepeated = make([]float64, N) + for i := 0; i < N; i++ { + pb.F_Int32Repeated[i] = int32(i) + pb.F_DoubleRepeated[i] = float64(i) } return pb } -func BenchmarkMarshal(b *testing.B) { - pb := benchmarkMsg(false) - p := NewBuffer(nil) +func bytesMsg() *GoTest { + pb := initGoTest(true) + buf := make([]byte, 4000) + for i := range buf { + buf[i] = byte(i) + } + pb.F_BytesDefaulted = buf + return pb +} +func benchmarkMarshal(b *testing.B, pb Message, marshal func(Message) ([]byte, error)) { + d, _ := marshal(pb) + b.SetBytes(int64(len(d))) b.ResetTimer() for i := 0; i < b.N; i++ { - p.Reset() - p.Marshal(pb) + marshal(pb) } - b.SetBytes(int64(len(p.Bytes()))) } -func BenchmarkUnmarshal(b *testing.B) { - pb := benchmarkMsg(false) +func benchmarkBufferMarshal(b *testing.B, pb Message) { p := NewBuffer(nil) - p.Marshal(pb) - b.SetBytes(int64(len(p.Bytes()))) - p2 := NewBuffer(nil) - pbd := new(GoTest) + benchmarkMarshal(b, pb, func(pb0 Message) ([]byte, error) { + p.Reset() + err := p.Marshal(pb0) + return p.Bytes(), err + }) +} - b.ResetTimer() - for i := 0; i < b.N; i++ { - p2.SetBuf(p.Bytes()) - p2.Unmarshal(pbd) +func benchmarkSize(b *testing.B, pb Message) { + benchmarkMarshal(b, pb, func(pb0 Message) ([]byte, error) { + Size(pb) + return nil, nil + }) +} + +func newOf(pb Message) Message { + in := reflect.ValueOf(pb) + if in.IsNil() { + return pb } + return reflect.New(in.Type().Elem()).Interface().(Message) } -func BenchmarkMarshalBytes(b *testing.B) { - pb := benchmarkMsg(true) - p := NewBuffer(nil) +func benchmarkUnmarshal(b *testing.B, pb Message, unmarshal func([]byte, Message) error) { + d, _ := Marshal(pb) + b.SetBytes(int64(len(d))) + pbd := newOf(pb) b.ResetTimer() for i := 0; i < b.N; i++ { - p.Reset() - p.Marshal(pb) + unmarshal(d, pbd) } - b.SetBytes(int64(len(p.Bytes()))) } -func BenchmarkUnmarshalBytes(b *testing.B) { - pb := benchmarkMsg(true) +func benchmarkBufferUnmarshal(b *testing.B, pb Message) { p := NewBuffer(nil) - p.Marshal(pb) - b.SetBytes(int64(len(p.Bytes()))) - p2 := NewBuffer(nil) - pbd := new(GoTest) + benchmarkUnmarshal(b, pb, func(d []byte, pb0 Message) error { + p.SetBuf(d) + return p.Unmarshal(pb0) + }) +} - b.ResetTimer() - for i := 0; i < b.N; i++ { - p2.SetBuf(p.Bytes()) - p2.Unmarshal(pbd) - } +// Benchmark{Marshal,BufferMarshal,Size,Unmarshal,BufferUnmarshal}{,Bytes} + +func BenchmarkMarshal(b *testing.B) { + benchmarkMarshal(b, testMsg(), Marshal) +} + +func BenchmarkBufferMarshal(b *testing.B) { + benchmarkBufferMarshal(b, testMsg()) +} + +func BenchmarkSize(b *testing.B) { + benchmarkSize(b, testMsg()) +} + +func BenchmarkUnmarshal(b *testing.B) { + benchmarkUnmarshal(b, testMsg(), Unmarshal) +} + +func BenchmarkBufferUnmarshal(b *testing.B) { + benchmarkBufferUnmarshal(b, testMsg()) +} + +func BenchmarkMarshalBytes(b *testing.B) { + benchmarkMarshal(b, bytesMsg(), Marshal) +} + +func BenchmarkBufferMarshalBytes(b *testing.B) { + benchmarkBufferMarshal(b, bytesMsg()) +} + +func BenchmarkSizeBytes(b *testing.B) { + benchmarkSize(b, bytesMsg()) +} + +func BenchmarkUnmarshalBytes(b *testing.B) { + benchmarkUnmarshal(b, bytesMsg(), Unmarshal) +} + +func BenchmarkBufferUnmarshalBytes(b *testing.B) { + benchmarkBufferUnmarshal(b, bytesMsg()) } func BenchmarkUnmarshalUnrecognizedFields(b *testing.B) { diff --git a/third_party/code.google.com/p/goprotobuf/proto/decode.go b/third_party/code.google.com/p/goprotobuf/proto/decode.go index 3c58cfb9b2a..59189d76705 100644 --- a/third_party/code.google.com/p/goprotobuf/proto/decode.go +++ b/third_party/code.google.com/p/goprotobuf/proto/decode.go @@ -441,7 +441,7 @@ func (o *Buffer) unmarshalType(st reflect.Type, prop *StructProperties, is_group // Not enough information to determine the exact field. If we use extra // CPU, we could determine the field only if the missing required field // has a tag <= 64 and we check reqFields. - return &ErrRequiredNotSet{"{Unknown}"} + return &RequiredNotSetError{"{Unknown}"} } } return err diff --git a/third_party/code.google.com/p/goprotobuf/proto/encode.go b/third_party/code.google.com/p/goprotobuf/proto/encode.go index d49ab84d2f4..d75711027d3 100644 --- a/third_party/code.google.com/p/goprotobuf/proto/encode.go +++ b/third_party/code.google.com/p/goprotobuf/proto/encode.go @@ -42,20 +42,20 @@ import ( "sort" ) -// ErrRequiredNotSet is the error returned if Marshal is called with +// RequiredNotSetError is the error returned if Marshal is called with // a protocol buffer struct whose required fields have not // all been initialized. It is also the error returned if Unmarshal is // called with an encoded protocol buffer that does not include all the // required fields. // -// When printed, ErrRequiredNotSet reports the first unset required field in a +// When printed, RequiredNotSetError reports the first unset required field in a // message. If the field cannot be precisely determined, it is reported as // "{Unknown}". -type ErrRequiredNotSet struct { +type RequiredNotSetError struct { field string } -func (e *ErrRequiredNotSet) Error() string { +func (e *RequiredNotSetError) Error() string { return fmt.Sprintf("proto: required field %q not set", e.field) } @@ -105,6 +105,17 @@ func (p *Buffer) EncodeVarint(x uint64) error { return nil } +func sizeVarint(x uint64) (n int) { + for { + n++ + x >>= 7 + if x == 0 { + break + } + } + return n +} + // EncodeFixed64 writes a 64-bit integer to the Buffer. // This is the format for the // fixed64, sfixed64, and double protocol buffer types. @@ -121,6 +132,10 @@ func (p *Buffer) EncodeFixed64(x uint64) error { return nil } +func sizeFixed64(x uint64) int { + return 8 +} + // EncodeFixed32 writes a 32-bit integer to the Buffer. // This is the format for the // fixed32, sfixed32, and float protocol buffer types. @@ -133,6 +148,10 @@ func (p *Buffer) EncodeFixed32(x uint64) error { return nil } +func sizeFixed32(x uint64) int { + return 4 +} + // EncodeZigzag64 writes a zigzag-encoded 64-bit integer // to the Buffer. // This is the format used for the sint64 protocol buffer type. @@ -141,6 +160,10 @@ func (p *Buffer) EncodeZigzag64(x uint64) error { return p.EncodeVarint(uint64((x << 1) ^ uint64((int64(x) >> 63)))) } +func sizeZigzag64(x uint64) int { + return sizeVarint(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} + // EncodeZigzag32 writes a zigzag-encoded 32-bit integer // to the Buffer. // This is the format used for the sint32 protocol buffer type. @@ -149,6 +172,10 @@ func (p *Buffer) EncodeZigzag32(x uint64) error { return p.EncodeVarint(uint64((uint32(x) << 1) ^ uint32((int32(x) >> 31)))) } +func sizeZigzag32(x uint64) int { + return sizeVarint(uint64((uint32(x) << 1) ^ uint32((int32(x) >> 31)))) +} + // EncodeRawBytes writes a count-delimited byte buffer to the Buffer. // This is the format used for the bytes protocol buffer // type and for embedded messages. @@ -158,6 +185,11 @@ func (p *Buffer) EncodeRawBytes(b []byte) error { return nil } +func sizeRawBytes(b []byte) int { + return sizeVarint(uint64(len(b))) + + len(b) +} + // EncodeStringBytes writes an encoded string to the Buffer. // This is the format used for the proto2 string type. func (p *Buffer) EncodeStringBytes(s string) error { @@ -166,6 +198,11 @@ func (p *Buffer) EncodeStringBytes(s string) error { return nil } +func sizeStringBytes(s string) int { + return sizeVarint(uint64(len(s))) + + len(s) +} + // Marshaler is the interface representing objects that can marshal themselves. type Marshaler interface { Marshal() ([]byte, error) @@ -216,6 +253,30 @@ func (p *Buffer) Marshal(pb Message) error { return err } +// Size returns the encoded size of a protocol buffer. +func Size(pb Message) (n int) { + // Can the object marshal itself? If so, Size is slow. + // TODO: add Size to Marshaler, or add a Sizer interface. + if m, ok := pb.(Marshaler); ok { + b, _ := m.Marshal() + return len(b) + } + + t, base, err := getbase(pb) + if structPointer_IsNil(base) { + return 0 + } + if err == nil { + n = size_struct(t.Elem(), GetProperties(t.Elem()), base) + } + + if collectStats { + stats.Size++ + } + + return +} + // Individual type encoders. // Encode a bool. @@ -233,6 +294,14 @@ func (o *Buffer) enc_bool(p *Properties, base structPointer) error { return nil } +func size_bool(p *Properties, base structPointer) int { + v := *structPointer_Bool(base, p.field) + if v == nil { + return 0 + } + return len(p.tagcode) + 1 // each bool takes exactly one byte +} + // Encode an int32. func (o *Buffer) enc_int32(p *Properties, base structPointer) error { v := structPointer_Word32(base, p.field) @@ -245,6 +314,17 @@ func (o *Buffer) enc_int32(p *Properties, base structPointer) error { return nil } +func size_int32(p *Properties, base structPointer) (n int) { + v := structPointer_Word32(base, p.field) + if word32_IsNil(v) { + return 0 + } + x := word32_Get(v) + n += len(p.tagcode) + n += p.valSize(uint64(x)) + return +} + // Encode an int64. func (o *Buffer) enc_int64(p *Properties, base structPointer) error { v := structPointer_Word64(base, p.field) @@ -257,6 +337,17 @@ func (o *Buffer) enc_int64(p *Properties, base structPointer) error { return nil } +func size_int64(p *Properties, base structPointer) (n int) { + v := structPointer_Word64(base, p.field) + if word64_IsNil(v) { + return 0 + } + x := word64_Get(v) + n += len(p.tagcode) + n += p.valSize(x) + return +} + // Encode a string. func (o *Buffer) enc_string(p *Properties, base structPointer) error { v := *structPointer_String(base, p.field) @@ -269,6 +360,17 @@ func (o *Buffer) enc_string(p *Properties, base structPointer) error { return nil } +func size_string(p *Properties, base structPointer) (n int) { + v := *structPointer_String(base, p.field) + if v == nil { + return 0 + } + x := *v + n += len(p.tagcode) + n += sizeStringBytes(x) + return +} + // All protocol buffer fields are nillable, but be careful. func isNil(v reflect.Value) bool { switch v.Kind() { @@ -317,6 +419,27 @@ func (o *Buffer) enc_struct_message(p *Properties, base structPointer) error { return state.err } +func size_struct_message(p *Properties, base structPointer) int { + structp := structPointer_GetStructPointer(base, p.field) + if structPointer_IsNil(structp) { + return 0 + } + + // Can the object marshal itself? + if p.isMarshaler { + m := structPointer_Interface(structp, p.stype).(Marshaler) + data, _ := m.Marshal() + n0 := len(p.tagcode) + n1 := sizeRawBytes(data) + return n0 + n1 + } + + n0 := len(p.tagcode) + n1 := size_struct(p.stype, p.sprop, structp) + n2 := sizeVarint(uint64(n1)) // size of encoded length + return n0 + n1 + n2 +} + // Encode a group struct. func (o *Buffer) enc_struct_group(p *Properties, base structPointer) error { var state errorState @@ -334,6 +457,18 @@ func (o *Buffer) enc_struct_group(p *Properties, base structPointer) error { return state.err } +func size_struct_group(p *Properties, base structPointer) (n int) { + b := structPointer_GetStructPointer(base, p.field) + if structPointer_IsNil(b) { + return 0 + } + + n += sizeVarint(uint64((p.Tag << 3) | WireStartGroup)) + n += size_struct(p.stype, p.sprop, b) + n += sizeVarint(uint64((p.Tag << 3) | WireEndGroup)) + return +} + // Encode a slice of bools ([]bool). func (o *Buffer) enc_slice_bool(p *Properties, base structPointer) error { s := *structPointer_BoolSlice(base, p.field) @@ -352,6 +487,15 @@ func (o *Buffer) enc_slice_bool(p *Properties, base structPointer) error { return nil } +func size_slice_bool(p *Properties, base structPointer) int { + s := *structPointer_BoolSlice(base, p.field) + l := len(s) + if l == 0 { + return 0 + } + return l * (len(p.tagcode) + 1) // each bool takes exactly one byte +} + // Encode a slice of bools ([]bool) in packed format. func (o *Buffer) enc_slice_packed_bool(p *Properties, base structPointer) error { s := *structPointer_BoolSlice(base, p.field) @@ -371,6 +515,18 @@ func (o *Buffer) enc_slice_packed_bool(p *Properties, base structPointer) error return nil } +func size_slice_packed_bool(p *Properties, base structPointer) (n int) { + s := *structPointer_BoolSlice(base, p.field) + l := len(s) + if l == 0 { + return 0 + } + n += len(p.tagcode) + n += sizeVarint(uint64(l)) + n += l // each bool takes exactly one byte + return +} + // Encode a slice of bytes ([]byte). func (o *Buffer) enc_slice_byte(p *Properties, base structPointer) error { s := *structPointer_Bytes(base, p.field) @@ -382,6 +538,16 @@ func (o *Buffer) enc_slice_byte(p *Properties, base structPointer) error { return nil } +func size_slice_byte(p *Properties, base structPointer) (n int) { + s := *structPointer_Bytes(base, p.field) + if s == nil { + return 0 + } + n += len(p.tagcode) + n += sizeRawBytes(s) + return +} + // Encode a slice of int32s ([]int32). func (o *Buffer) enc_slice_int32(p *Properties, base structPointer) error { s := structPointer_Word32Slice(base, p.field) @@ -397,6 +563,20 @@ func (o *Buffer) enc_slice_int32(p *Properties, base structPointer) error { return nil } +func size_slice_int32(p *Properties, base structPointer) (n int) { + s := structPointer_Word32Slice(base, p.field) + l := s.Len() + if l == 0 { + return 0 + } + for i := 0; i < l; i++ { + n += len(p.tagcode) + x := s.Index(i) + n += p.valSize(uint64(x)) + } + return +} + // Encode a slice of int32s ([]int32) in packed format. func (o *Buffer) enc_slice_packed_int32(p *Properties, base structPointer) error { s := structPointer_Word32Slice(base, p.field) @@ -416,6 +596,23 @@ func (o *Buffer) enc_slice_packed_int32(p *Properties, base structPointer) error return nil } +func size_slice_packed_int32(p *Properties, base structPointer) (n int) { + s := structPointer_Word32Slice(base, p.field) + l := s.Len() + if l == 0 { + return 0 + } + var bufSize int + for i := 0; i < l; i++ { + bufSize += p.valSize(uint64(s.Index(i))) + } + + n += len(p.tagcode) + n += sizeVarint(uint64(bufSize)) + n += bufSize + return +} + // Encode a slice of int64s ([]int64). func (o *Buffer) enc_slice_int64(p *Properties, base structPointer) error { s := structPointer_Word64Slice(base, p.field) @@ -430,6 +627,19 @@ func (o *Buffer) enc_slice_int64(p *Properties, base structPointer) error { return nil } +func size_slice_int64(p *Properties, base structPointer) (n int) { + s := structPointer_Word64Slice(base, p.field) + l := s.Len() + if l == 0 { + return 0 + } + for i := 0; i < l; i++ { + n += len(p.tagcode) + n += p.valSize(s.Index(i)) + } + return +} + // Encode a slice of int64s ([]int64) in packed format. func (o *Buffer) enc_slice_packed_int64(p *Properties, base structPointer) error { s := structPointer_Word64Slice(base, p.field) @@ -449,6 +659,23 @@ func (o *Buffer) enc_slice_packed_int64(p *Properties, base structPointer) error return nil } +func size_slice_packed_int64(p *Properties, base structPointer) (n int) { + s := structPointer_Word64Slice(base, p.field) + l := s.Len() + if l == 0 { + return 0 + } + var bufSize int + for i := 0; i < l; i++ { + bufSize += p.valSize(s.Index(i)) + } + + n += len(p.tagcode) + n += sizeVarint(uint64(bufSize)) + n += bufSize + return +} + // Encode a slice of slice of bytes ([][]byte). func (o *Buffer) enc_slice_slice_byte(p *Properties, base structPointer) error { ss := *structPointer_BytesSlice(base, p.field) @@ -458,24 +685,45 @@ func (o *Buffer) enc_slice_slice_byte(p *Properties, base structPointer) error { } for i := 0; i < l; i++ { o.buf = append(o.buf, p.tagcode...) - s := ss[i] - o.EncodeRawBytes(s) + o.EncodeRawBytes(ss[i]) } return nil } +func size_slice_slice_byte(p *Properties, base structPointer) (n int) { + ss := *structPointer_BytesSlice(base, p.field) + l := len(ss) + if l == 0 { + return 0 + } + n += l * len(p.tagcode) + for i := 0; i < l; i++ { + n += sizeRawBytes(ss[i]) + } + return +} + // Encode a slice of strings ([]string). func (o *Buffer) enc_slice_string(p *Properties, base structPointer) error { ss := *structPointer_StringSlice(base, p.field) l := len(ss) for i := 0; i < l; i++ { o.buf = append(o.buf, p.tagcode...) - s := ss[i] - o.EncodeStringBytes(s) + o.EncodeStringBytes(ss[i]) } return nil } +func size_slice_string(p *Properties, base structPointer) (n int) { + ss := *structPointer_StringSlice(base, p.field) + l := len(ss) + n += l * len(p.tagcode) + for i := 0; i < l; i++ { + n += sizeStringBytes(ss[i]) + } + return +} + // Encode a slice of message structs ([]*struct). func (o *Buffer) enc_slice_struct_message(p *Properties, base structPointer) error { var state errorState @@ -522,6 +770,32 @@ func (o *Buffer) enc_slice_struct_message(p *Properties, base structPointer) err return state.err } +func size_slice_struct_message(p *Properties, base structPointer) (n int) { + s := structPointer_StructPointerSlice(base, p.field) + l := s.Len() + n += l * len(p.tagcode) + for i := 0; i < l; i++ { + structp := s.Index(i) + if structPointer_IsNil(structp) { + return // return the size up to this point + } + + // Can the object marshal itself? + if p.isMarshaler { + m := structPointer_Interface(structp, p.stype).(Marshaler) + data, _ := m.Marshal() + n += len(p.tagcode) + n += sizeRawBytes(data) + continue + } + + n0 := size_struct(p.stype, p.sprop, structp) + n1 := sizeVarint(uint64(n0)) // size of encoded length + n += n0 + n1 + } + return +} + // Encode a slice of group structs ([]*struct). func (o *Buffer) enc_slice_struct_group(p *Properties, base structPointer) error { var state errorState @@ -550,6 +824,23 @@ func (o *Buffer) enc_slice_struct_group(p *Properties, base structPointer) error return state.err } +func size_slice_struct_group(p *Properties, base structPointer) (n int) { + s := structPointer_StructPointerSlice(base, p.field) + l := s.Len() + + n += l * sizeVarint(uint64((p.Tag<<3)|WireStartGroup)) + n += l * sizeVarint(uint64((p.Tag<<3)|WireEndGroup)) + for i := 0; i < l; i++ { + b := s.Index(i) + if structPointer_IsNil(b) { + return // return size up to this point + } + + n += size_struct(p.stype, p.sprop, b) + } + return +} + // Encode an extension map. func (o *Buffer) enc_map(p *Properties, base structPointer) error { v := *structPointer_ExtMap(base, p.field) @@ -577,6 +868,11 @@ func (o *Buffer) enc_map(p *Properties, base structPointer) error { return nil } +func size_map(p *Properties, base structPointer) int { + v := *structPointer_ExtMap(base, p.field) + return sizeExtensionMap(v) +} + // Encode a struct. func (o *Buffer) enc_struct(t reflect.Type, prop *StructProperties, base structPointer) error { var state errorState @@ -587,11 +883,13 @@ func (o *Buffer) enc_struct(t reflect.Type, prop *StructProperties, base structP p := prop.Prop[i] if p.enc != nil { err := p.enc(o, p, base) - if err != nil && !state.shouldContinue(err, p) { - if err != ErrNil { + if err != nil { + if err == ErrNil { + if p.Required && state.err == nil { + state.err = &RequiredNotSetError{p.Name} + } + } else if !state.shouldContinue(err, p) { return err - } else if p.Required && state.err == nil { - state.err = &ErrRequiredNotSet{p.Name} } } } @@ -608,6 +906,23 @@ func (o *Buffer) enc_struct(t reflect.Type, prop *StructProperties, base structP return state.err } +func size_struct(t reflect.Type, prop *StructProperties, base structPointer) (n int) { + for _, i := range prop.order { + p := prop.Prop[i] + if p.size != nil { + n += p.size(p, base) + } + } + + // Add unrecognized fields at the end. + if prop.unrecField.IsValid() { + v := *structPointer_Bytes(base, prop.unrecField) + n += len(v) + } + + return +} + // errorState maintains the first error that occurs and updates that error // with additional context. type errorState struct { @@ -615,7 +930,7 @@ type errorState struct { } // shouldContinue reports whether encoding should continue upon encountering the -// given error. If the error is ErrRequiredNotSet, shouldContinue returns true +// given error. If the error is RequiredNotSetError, shouldContinue returns true // and, if this is the first appearance of that error, remembers it for future // reporting. // @@ -623,13 +938,13 @@ type errorState struct { // field with the error. func (s *errorState) shouldContinue(err error, prop *Properties) bool { // Ignore unset required fields. - reqNotSet, ok := err.(*ErrRequiredNotSet) + reqNotSet, ok := err.(*RequiredNotSetError) if !ok { return false } if s.err == nil { if prop != nil { - err = &ErrRequiredNotSet{prop.Name + "." + reqNotSet.field} + err = &RequiredNotSetError{prop.Name + "." + reqNotSet.field} } s.err = err } diff --git a/third_party/code.google.com/p/goprotobuf/proto/extensions.go b/third_party/code.google.com/p/goprotobuf/proto/extensions.go index e730b68ddf2..50d72aaa95d 100644 --- a/third_party/code.google.com/p/goprotobuf/proto/extensions.go +++ b/third_party/code.google.com/p/goprotobuf/proto/extensions.go @@ -183,6 +183,30 @@ func encodeExtensionMap(m map[int32]Extension) error { return nil } +func sizeExtensionMap(m map[int32]Extension) (n int) { + for _, e := range m { + if e.value == nil || e.desc == nil { + // Extension is only in its encoded form. + n += len(e.enc) + continue + } + + // We don't skip extensions that have an encoded form set, + // because the extension value may have been mutated after + // the last time this function was called. + + et := reflect.TypeOf(e.desc.ExtensionType) + props := extensionProperties(e.desc) + + // If e.value has type T, the encoder expects a *struct{ X T }. + // Pass a *T with a zero field and hope it all works out. + x := reflect.New(et) + x.Elem().Set(reflect.ValueOf(e.value)) + n += props.size(props, toStructPointer(x)) + } + return +} + // HasExtension returns whether the given extension is present in pb. func HasExtension(pb extendableProto, extension *ExtensionDesc) bool { // TODO: Check types, field numbers, etc.? diff --git a/third_party/code.google.com/p/goprotobuf/proto/lib.go b/third_party/code.google.com/p/goprotobuf/proto/lib.go index fa6fe22ea25..61f1f22aa41 100644 --- a/third_party/code.google.com/p/goprotobuf/proto/lib.go +++ b/third_party/code.google.com/p/goprotobuf/proto/lib.go @@ -223,6 +223,7 @@ type Stats struct { Decode uint64 // number of decodes Chit uint64 // number of cache hits Cmiss uint64 // number of cache misses + Size uint64 // number of sizes } // Set to true to enable stats collection. @@ -403,9 +404,7 @@ func EnumName(m map[int32]string, v int32) string { // names to its int values, and a byte buffer containing the JSON-encoded // value, it returns an int32 that can be cast to the enum type by the caller. // -// The function can deal with older JSON representations, which represented -// enums directly by their int32 values, or with newer representations, which -// use the symbolic name as a string. +// The function can deal with both JSON representations, numeric and symbolic. func UnmarshalJSONEnum(m map[string]int32, data []byte, enumName string) (int32, error) { if data[0] == '"' { // New style: enums are strings. diff --git a/third_party/code.google.com/p/goprotobuf/proto/properties.go b/third_party/code.google.com/p/goprotobuf/proto/properties.go index 75b3e8d0813..7177cfc9650 100644 --- a/third_party/code.google.com/p/goprotobuf/proto/properties.go +++ b/third_party/code.google.com/p/goprotobuf/proto/properties.go @@ -59,7 +59,7 @@ const ( const startSize = 10 // initial slice/string sizes -// Encoders are defined in encoder.go +// Encoders are defined in encode.go // An encoder outputs the full representation of a field, including its // tag and encoder type. type encoder func(p *Buffer, prop *Properties, base structPointer) error @@ -67,6 +67,15 @@ type encoder func(p *Buffer, prop *Properties, base structPointer) error // A valueEncoder encodes a single integer in a particular encoding. type valueEncoder func(o *Buffer, x uint64) error +// Sizers are defined in encode.go +// A sizer returns the encoded size of a field, including its tag and encoder +// type. +type sizer func(prop *Properties, base structPointer) int + +// A valueSizer returns the encoded size of a single integer in a particular +// encoding. +type valueSizer func(x uint64) int + // Decoders are defined in decode.go // A decoder creates a value from its wire representation. // Unrecognized subelements are saved in unrec. @@ -126,7 +135,7 @@ type StructProperties struct { } // Implement the sorting interface so we can sort the fields in tag order, as recommended by the spec. -// See encoder.go, (*Buffer).enc_struct. +// See encode.go, (*Buffer).enc_struct. func (sp *StructProperties) Len() int { return len(sp.order) } func (sp *StructProperties) Less(i, j int) bool { @@ -159,6 +168,9 @@ type Properties struct { isMarshaler bool isUnmarshaler bool + size sizer + valSize valueSizer // set for bool and numeric types only + dec decoder valDec valueDecoder // set for bool and numeric types only @@ -210,22 +222,27 @@ func (p *Properties) Parse(s string) { p.WireType = WireVarint p.valEnc = (*Buffer).EncodeVarint p.valDec = (*Buffer).DecodeVarint + p.valSize = sizeVarint case "fixed32": p.WireType = WireFixed32 p.valEnc = (*Buffer).EncodeFixed32 p.valDec = (*Buffer).DecodeFixed32 + p.valSize = sizeFixed32 case "fixed64": p.WireType = WireFixed64 p.valEnc = (*Buffer).EncodeFixed64 p.valDec = (*Buffer).DecodeFixed64 + p.valSize = sizeFixed64 case "zigzag32": p.WireType = WireVarint p.valEnc = (*Buffer).EncodeZigzag32 p.valDec = (*Buffer).DecodeZigzag32 + p.valSize = sizeZigzag32 case "zigzag64": p.WireType = WireVarint p.valEnc = (*Buffer).EncodeZigzag64 p.valDec = (*Buffer).DecodeZigzag64 + p.valSize = sizeZigzag64 case "bytes", "group": p.WireType = WireBytes // no numeric converter for non-numeric types @@ -276,6 +293,7 @@ var protoMessageType = reflect.TypeOf((*Message)(nil)).Elem() func (p *Properties) setEncAndDec(typ reflect.Type, lockGetProp bool) { p.enc = nil p.dec = nil + p.size = nil switch t1 := typ; t1.Kind() { default: @@ -289,21 +307,27 @@ func (p *Properties) setEncAndDec(typ reflect.Type, lockGetProp bool) { case reflect.Bool: p.enc = (*Buffer).enc_bool p.dec = (*Buffer).dec_bool + p.size = size_bool case reflect.Int32, reflect.Uint32: p.enc = (*Buffer).enc_int32 p.dec = (*Buffer).dec_int32 + p.size = size_int32 case reflect.Int64, reflect.Uint64: p.enc = (*Buffer).enc_int64 p.dec = (*Buffer).dec_int64 + p.size = size_int64 case reflect.Float32: p.enc = (*Buffer).enc_int32 // can just treat them as bits p.dec = (*Buffer).dec_int32 + p.size = size_int32 case reflect.Float64: p.enc = (*Buffer).enc_int64 // can just treat them as bits p.dec = (*Buffer).dec_int64 + p.size = size_int64 case reflect.String: p.enc = (*Buffer).enc_string p.dec = (*Buffer).dec_string + p.size = size_string case reflect.Struct: p.stype = t1.Elem() p.isMarshaler = isMarshaler(t1) @@ -311,9 +335,11 @@ func (p *Properties) setEncAndDec(typ reflect.Type, lockGetProp bool) { if p.Wire == "bytes" { p.enc = (*Buffer).enc_struct_message p.dec = (*Buffer).dec_struct_message + p.size = size_struct_message } else { p.enc = (*Buffer).enc_struct_group p.dec = (*Buffer).dec_struct_group + p.size = size_struct_group } } @@ -325,8 +351,10 @@ func (p *Properties) setEncAndDec(typ reflect.Type, lockGetProp bool) { case reflect.Bool: if p.Packed { p.enc = (*Buffer).enc_slice_packed_bool + p.size = size_slice_packed_bool } else { p.enc = (*Buffer).enc_slice_bool + p.size = size_slice_bool } p.dec = (*Buffer).dec_slice_bool p.packedDec = (*Buffer).dec_slice_packed_bool @@ -335,16 +363,20 @@ func (p *Properties) setEncAndDec(typ reflect.Type, lockGetProp bool) { case 32: if p.Packed { p.enc = (*Buffer).enc_slice_packed_int32 + p.size = size_slice_packed_int32 } else { p.enc = (*Buffer).enc_slice_int32 + p.size = size_slice_int32 } p.dec = (*Buffer).dec_slice_int32 p.packedDec = (*Buffer).dec_slice_packed_int32 case 64: if p.Packed { p.enc = (*Buffer).enc_slice_packed_int64 + p.size = size_slice_packed_int64 } else { p.enc = (*Buffer).enc_slice_int64 + p.size = size_slice_int64 } p.dec = (*Buffer).dec_slice_int64 p.packedDec = (*Buffer).dec_slice_packed_int64 @@ -352,6 +384,7 @@ func (p *Properties) setEncAndDec(typ reflect.Type, lockGetProp bool) { if t2.Kind() == reflect.Uint8 { p.enc = (*Buffer).enc_slice_byte p.dec = (*Buffer).dec_slice_byte + p.size = size_slice_byte } default: logNoSliceEnc(t1, t2) @@ -363,8 +396,10 @@ func (p *Properties) setEncAndDec(typ reflect.Type, lockGetProp bool) { // can just treat them as bits if p.Packed { p.enc = (*Buffer).enc_slice_packed_int32 + p.size = size_slice_packed_int32 } else { p.enc = (*Buffer).enc_slice_int32 + p.size = size_slice_int32 } p.dec = (*Buffer).dec_slice_int32 p.packedDec = (*Buffer).dec_slice_packed_int32 @@ -372,8 +407,10 @@ func (p *Properties) setEncAndDec(typ reflect.Type, lockGetProp bool) { // can just treat them as bits if p.Packed { p.enc = (*Buffer).enc_slice_packed_int64 + p.size = size_slice_packed_int64 } else { p.enc = (*Buffer).enc_slice_int64 + p.size = size_slice_int64 } p.dec = (*Buffer).dec_slice_int64 p.packedDec = (*Buffer).dec_slice_packed_int64 @@ -384,6 +421,7 @@ func (p *Properties) setEncAndDec(typ reflect.Type, lockGetProp bool) { case reflect.String: p.enc = (*Buffer).enc_slice_string p.dec = (*Buffer).dec_slice_string + p.size = size_slice_string case reflect.Ptr: switch t3 := t2.Elem(); t3.Kind() { default: @@ -393,11 +431,14 @@ func (p *Properties) setEncAndDec(typ reflect.Type, lockGetProp bool) { p.stype = t2.Elem() p.isMarshaler = isMarshaler(t2) p.isUnmarshaler = isUnmarshaler(t2) - p.enc = (*Buffer).enc_slice_struct_group - p.dec = (*Buffer).dec_slice_struct_group if p.Wire == "bytes" { p.enc = (*Buffer).enc_slice_struct_message p.dec = (*Buffer).dec_slice_struct_message + p.size = size_slice_struct_message + } else { + p.enc = (*Buffer).enc_slice_struct_group + p.dec = (*Buffer).dec_slice_struct_group + p.size = size_slice_struct_group } } case reflect.Slice: @@ -408,6 +449,7 @@ func (p *Properties) setEncAndDec(typ reflect.Type, lockGetProp bool) { case reflect.Uint8: p.enc = (*Buffer).enc_slice_slice_byte p.dec = (*Buffer).dec_slice_slice_byte + p.size = size_slice_slice_byte } } } @@ -525,6 +567,7 @@ func getPropertiesLocked(t reflect.Type) *StructProperties { if f.Name == "XXX_extensions" { // special case p.enc = (*Buffer).enc_map p.dec = nil // not needed + p.size = size_map } if f.Name == "XXX_unrecognized" { // special case prop.unrecField = toField(&f) diff --git a/third_party/code.google.com/p/goprotobuf/proto/size.go b/third_party/code.google.com/p/goprotobuf/proto/size.go deleted file mode 100644 index ebdc893ef39..00000000000 --- a/third_party/code.google.com/p/goprotobuf/proto/size.go +++ /dev/null @@ -1,193 +0,0 @@ -// Go support for Protocol Buffers - Google's data interchange format -// -// Copyright 2012 The Go Authors. All rights reserved. -// http://code.google.com/p/goprotobuf/ -// -// Redistribution and use in source and binary forms, with or without -// modification, are permitted provided that the following conditions are -// met: -// -// * Redistributions of source code must retain the above copyright -// notice, this list of conditions and the following disclaimer. -// * Redistributions in binary form must reproduce the above -// copyright notice, this list of conditions and the following disclaimer -// in the documentation and/or other materials provided with the -// distribution. -// * Neither the name of Google Inc. nor the names of its -// contributors may be used to endorse or promote products derived from -// this software without specific prior written permission. -// -// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - -// Functions to determine the size of an encoded protocol buffer. - -package proto - -import ( - "log" - "reflect" - "strings" -) - -// Size returns the encoded size of a protocol buffer. -// This function is expensive enough to be avoided unless proven worthwhile with instrumentation. -func Size(pb Message) int { - in := reflect.ValueOf(pb) - if in.IsNil() { - return 0 - } - return sizeStruct(in.Elem()) -} - -func sizeStruct(x reflect.Value) (n int) { - sprop := GetProperties(x.Type()) - for _, prop := range sprop.Prop { - if strings.HasPrefix(prop.Name, "XXX_") { // handled below - continue - } - fi, _ := sprop.decoderTags.get(prop.Tag) - f := x.Field(fi) - switch f.Kind() { - case reflect.Ptr: - if f.IsNil() { - continue - } - n += len(prop.tagcode) - f = f.Elem() // avoid a recursion in sizeField - case reflect.Slice: - if f.IsNil() { - continue - } - if f.Len() == 0 && f.Type().Elem().Kind() != reflect.Uint8 { - // short circuit for empty repeated fields. - // []byte isn't a repeated field. - continue - } - default: - log.Printf("proto: unknown struct field type %v", f.Type()) - continue - } - n += sizeField(f, prop) - } - - if em, ok := x.Addr().Interface().(extendableProto); ok { - for _, ext := range em.ExtensionMap() { - ms := len(ext.enc) - if ext.enc == nil { - props := new(Properties) - props.Init(reflect.TypeOf(ext.desc.ExtensionType), "x", ext.desc.Tag, nil) - ms = len(props.tagcode) + sizeField(reflect.ValueOf(ext.value), props) - } - n += ms - } - } - - if uf := x.FieldByName("XXX_unrecognized"); uf.IsValid() { - n += uf.Len() - } - - return n -} - -func sizeField(x reflect.Value, prop *Properties) (n int) { - if x.Type().Kind() == reflect.Slice { - n := x.Len() - et := x.Type().Elem() - if et.Kind() == reflect.Uint8 { - // []byte is easy. - return len(prop.tagcode) + sizeVarint(uint64(n)) + n - } - - var nb int - - // []bool and repeated fixed integer types are easy. - switch { - case et.Kind() == reflect.Bool: - nb += n - case prop.WireType == WireFixed64: - nb += n * 8 - case prop.WireType == WireFixed32: - nb += n * 4 - default: - for i := 0; i < n; i++ { - nb += sizeField(x.Index(i), prop) - } - } - // Non-packed repeated fields have a per-element header of the tagcode. - // Packed repeated fields only have a single header: the tag code plus a varint of the number of bytes. - if !prop.Packed { - nb += len(prop.tagcode) * n - } else { - nb += len(prop.tagcode) + sizeVarint(uint64(nb)) - } - return nb - } - - // easy scalars - switch prop.WireType { - case WireFixed64: - return 8 - case WireFixed32: - return 4 - } - - switch x.Kind() { - case reflect.Bool: - return 1 - case reflect.Int32, reflect.Int64: - if prop.Wire == "varint" { - return sizeVarint(uint64(x.Int())) - } else if prop.Wire == "zigzag32" || prop.Wire == "zigzag64" { - return sizeZigZag(uint64(x.Int())) - } - case reflect.Ptr: - return sizeField(x.Elem(), prop) - case reflect.String: - n := x.Len() - return sizeVarint(uint64(n)) + n - case reflect.Struct: - nb := sizeStruct(x) - if prop.Wire == "group" { - // Groups have start and end tags instead of a start tag and a length. - return nb + len(prop.tagcode) - } - return sizeVarint(uint64(nb)) + nb - case reflect.Uint32, reflect.Uint64: - if prop.Wire == "varint" { - return sizeVarint(uint64(x.Uint())) - } else if prop.Wire == "zigzag32" || prop.Wire == "zigzag64" { - return sizeZigZag(uint64(x.Int())) - } - default: - log.Printf("proto.sizeField: unhandled kind %v", x.Kind()) - } - - // unknown type, so not a protocol buffer - log.Printf("proto: don't know size of %v", x.Type()) - return 0 -} - -func sizeVarint(x uint64) (n int) { - for { - n++ - x >>= 7 - if x == 0 { - break - } - } - return n -} - -func sizeZigZag(x uint64) (n int) { - return sizeVarint(uint64((x << 1) ^ uint64((int64(x) >> 63)))) -} diff --git a/third_party/code.google.com/p/goprotobuf/proto/testdata/test.pb.go b/third_party/code.google.com/p/goprotobuf/proto/testdata/test.pb.go index b8e40cfc7a0..eb3f65a1fce 100644 --- a/third_party/code.google.com/p/goprotobuf/proto/testdata/test.pb.go +++ b/third_party/code.google.com/p/goprotobuf/proto/testdata/test.pb.go @@ -34,9 +34,6 @@ func (x FOO) Enum() *FOO { func (x FOO) String() string { return proto.EnumName(FOO_name, int32(x)) } -func (x FOO) MarshalJSON() ([]byte, error) { - return json.Marshal(x.String()) -} func (x *FOO) UnmarshalJSON(data []byte) error { value, err := proto.UnmarshalJSONEnum(FOO_value, data, "FOO") if err != nil { @@ -108,9 +105,6 @@ func (x GoTest_KIND) Enum() *GoTest_KIND { func (x GoTest_KIND) String() string { return proto.EnumName(GoTest_KIND_name, int32(x)) } -func (x GoTest_KIND) MarshalJSON() ([]byte, error) { - return json.Marshal(x.String()) -} func (x *GoTest_KIND) UnmarshalJSON(data []byte) error { value, err := proto.UnmarshalJSONEnum(GoTest_KIND_value, data, "GoTest_KIND") if err != nil { @@ -147,9 +141,6 @@ func (x MyMessage_Color) Enum() *MyMessage_Color { func (x MyMessage_Color) String() string { return proto.EnumName(MyMessage_Color_name, int32(x)) } -func (x MyMessage_Color) MarshalJSON() ([]byte, error) { - return json.Marshal(x.String()) -} func (x *MyMessage_Color) UnmarshalJSON(data []byte) error { value, err := proto.UnmarshalJSONEnum(MyMessage_Color_value, data, "MyMessage_Color") if err != nil { @@ -186,9 +177,6 @@ func (x Defaults_Color) Enum() *Defaults_Color { func (x Defaults_Color) String() string { return proto.EnumName(Defaults_Color_name, int32(x)) } -func (x Defaults_Color) MarshalJSON() ([]byte, error) { - return json.Marshal(x.String()) -} func (x *Defaults_Color) UnmarshalJSON(data []byte) error { value, err := proto.UnmarshalJSONEnum(Defaults_Color_value, data, "Defaults_Color") if err != nil { @@ -219,9 +207,6 @@ func (x RepeatedEnum_Color) Enum() *RepeatedEnum_Color { func (x RepeatedEnum_Color) String() string { return proto.EnumName(RepeatedEnum_Color_name, int32(x)) } -func (x RepeatedEnum_Color) MarshalJSON() ([]byte, error) { - return json.Marshal(x.String()) -} func (x *RepeatedEnum_Color) UnmarshalJSON(data []byte) error { value, err := proto.UnmarshalJSONEnum(RepeatedEnum_Color_value, data, "RepeatedEnum_Color") if err != nil { diff --git a/third_party/code.google.com/p/goprotobuf/proto/text.go b/third_party/code.google.com/p/goprotobuf/proto/text.go index ff8a0bb1e8c..f8cb9c94a74 100644 --- a/third_party/code.google.com/p/goprotobuf/proto/text.go +++ b/third_party/code.google.com/p/goprotobuf/proto/text.go @@ -477,7 +477,7 @@ func writeUnknownStruct(w *textWriter, data []byte) (err error) { switch wire { case WireBytes: buf, e := b.DecodeRawBytes(false) - if err == nil { + if e == nil { _, err = fmt.Fprintf(w, "%q", buf) } else { _, err = fmt.Fprintf(w, "/* %v */", e) diff --git a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/descriptor/descriptor.pb.go b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/descriptor/descriptor.pb.go index 0b34acb62db..a48174f72b3 100644 --- a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/descriptor/descriptor.pb.go +++ b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/descriptor/descriptor.pb.go @@ -92,9 +92,6 @@ func (x FieldDescriptorProto_Type) Enum() *FieldDescriptorProto_Type { func (x FieldDescriptorProto_Type) String() string { return proto.EnumName(FieldDescriptorProto_Type_name, int32(x)) } -func (x FieldDescriptorProto_Type) MarshalJSON() ([]byte, error) { - return json.Marshal(x.String()) -} func (x *FieldDescriptorProto_Type) UnmarshalJSON(data []byte) error { value, err := proto.UnmarshalJSONEnum(FieldDescriptorProto_Type_value, data, "FieldDescriptorProto_Type") if err != nil { @@ -132,9 +129,6 @@ func (x FieldDescriptorProto_Label) Enum() *FieldDescriptorProto_Label { func (x FieldDescriptorProto_Label) String() string { return proto.EnumName(FieldDescriptorProto_Label_name, int32(x)) } -func (x FieldDescriptorProto_Label) MarshalJSON() ([]byte, error) { - return json.Marshal(x.String()) -} func (x *FieldDescriptorProto_Label) UnmarshalJSON(data []byte) error { value, err := proto.UnmarshalJSONEnum(FieldDescriptorProto_Label_value, data, "FieldDescriptorProto_Label") if err != nil { @@ -173,9 +167,6 @@ func (x FileOptions_OptimizeMode) Enum() *FileOptions_OptimizeMode { func (x FileOptions_OptimizeMode) String() string { return proto.EnumName(FileOptions_OptimizeMode_name, int32(x)) } -func (x FileOptions_OptimizeMode) MarshalJSON() ([]byte, error) { - return json.Marshal(x.String()) -} func (x *FileOptions_OptimizeMode) UnmarshalJSON(data []byte) error { value, err := proto.UnmarshalJSONEnum(FileOptions_OptimizeMode_value, data, "FileOptions_OptimizeMode") if err != nil { @@ -213,9 +204,6 @@ func (x FieldOptions_CType) Enum() *FieldOptions_CType { func (x FieldOptions_CType) String() string { return proto.EnumName(FieldOptions_CType_name, int32(x)) } -func (x FieldOptions_CType) MarshalJSON() ([]byte, error) { - return json.Marshal(x.String()) -} func (x *FieldOptions_CType) UnmarshalJSON(data []byte) error { value, err := proto.UnmarshalJSONEnum(FieldOptions_CType_value, data, "FieldOptions_CType") if err != nil { diff --git a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/generator/generator.go b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/generator/generator.go index 0b769d454b9..9f38cad8e27 100644 --- a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/generator/generator.go +++ b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/generator/generator.go @@ -1201,12 +1201,6 @@ func (g *Generator) generateEnum(enum *EnumDescriptor) { g.Out() g.P("}") - g.P("func (x ", ccTypeName, ") MarshalJSON() ([]byte, error) {") - g.In() - g.P("return json.Marshal(x.String())") - g.Out() - g.P("}") - g.P("func (x *", ccTypeName, ") UnmarshalJSON(data []byte) error {") g.In() g.P("value, err := ", g.Pkg["proto"], ".UnmarshalJSONEnum(", ccTypeName, `_value, data, "`, ccTypeName, `")`) diff --git a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/generator/name_test.go b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/generator/name_test.go new file mode 100644 index 00000000000..9262b22230e --- /dev/null +++ b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/generator/name_test.go @@ -0,0 +1,56 @@ +// Go support for Protocol Buffers - Google's data interchange format +// +// Copyright 2013 The Go Authors. All rights reserved. +// http://code.google.com/p/goprotobuf/ +// +// Redistribution and use in source and binary forms, with or without +// modification, are permitted provided that the following conditions are +// met: +// +// * Redistributions of source code must retain the above copyright +// notice, this list of conditions and the following disclaimer. +// * Redistributions in binary form must reproduce the above +// copyright notice, this list of conditions and the following disclaimer +// in the documentation and/or other materials provided with the +// distribution. +// * Neither the name of Google Inc. nor the names of its +// contributors may be used to endorse or promote products derived from +// this software without specific prior written permission. +// +// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +package generator + +import ( + "testing" +) + +func TestCamelCase(t *testing.T) { + tests := []struct { + in, want string + }{ + {"one", "One"}, + {"one_two", "OneTwo"}, + {"_my_field_name_2", "XMyFieldName_2"}, + {"Something_Capped", "Something_Capped"}, + {"my_Name", "My_Name"}, + {"OneTwo", "OneTwo"}, + {"_", "X"}, + {"_a_", "XA_"}, + } + for _, tc := range tests { + if got := CamelCase(tc.in); got != tc.want { + t.Errorf("CamelCase(%q) = %q, want %q", tc.in, got, tc.want) + } + } +} diff --git a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.pb.go b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.pb.go index cfe977758b7..12abf2b924a 100644 --- a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.pb.go +++ b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.pb.go @@ -40,9 +40,6 @@ func (x HatType) Enum() *HatType { func (x HatType) String() string { return proto.EnumName(HatType_name, int32(x)) } -func (x HatType) MarshalJSON() ([]byte, error) { - return json.Marshal(x.String()) -} func (x *HatType) UnmarshalJSON(data []byte) error { value, err := proto.UnmarshalJSONEnum(HatType_value, data, "HatType") if err != nil { @@ -80,9 +77,6 @@ func (x Days) Enum() *Days { func (x Days) String() string { return proto.EnumName(Days_name, int32(x)) } -func (x Days) MarshalJSON() ([]byte, error) { - return json.Marshal(x.String()) -} func (x *Days) UnmarshalJSON(data []byte) error { value, err := proto.UnmarshalJSONEnum(Days_value, data, "Days") if err != nil { @@ -119,9 +113,6 @@ func (x Request_Color) Enum() *Request_Color { func (x Request_Color) String() string { return proto.EnumName(Request_Color_name, int32(x)) } -func (x Request_Color) MarshalJSON() ([]byte, error) { - return json.Marshal(x.String()) -} func (x *Request_Color) UnmarshalJSON(data []byte) error { value, err := proto.UnmarshalJSONEnum(Request_Color_value, data, "Request_Color") if err != nil { @@ -155,9 +146,6 @@ func (x Reply_Entry_Game) Enum() *Reply_Entry_Game { func (x Reply_Entry_Game) String() string { return proto.EnumName(Reply_Entry_Game_name, int32(x)) } -func (x Reply_Entry_Game) MarshalJSON() ([]byte, error) { - return json.Marshal(x.String()) -} func (x *Reply_Entry_Game) UnmarshalJSON(data []byte) error { value, err := proto.UnmarshalJSONEnum(Reply_Entry_Game_value, data, "Reply_Entry_Game") if err != nil { diff --git a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.pb.go.golden b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.pb.go.golden index cfe977758b7..12abf2b924a 100644 --- a/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.pb.go.golden +++ b/third_party/code.google.com/p/goprotobuf/protoc-gen-go/testdata/my_test/test.pb.go.golden @@ -40,9 +40,6 @@ func (x HatType) Enum() *HatType { func (x HatType) String() string { return proto.EnumName(HatType_name, int32(x)) } -func (x HatType) MarshalJSON() ([]byte, error) { - return json.Marshal(x.String()) -} func (x *HatType) UnmarshalJSON(data []byte) error { value, err := proto.UnmarshalJSONEnum(HatType_value, data, "HatType") if err != nil { @@ -80,9 +77,6 @@ func (x Days) Enum() *Days { func (x Days) String() string { return proto.EnumName(Days_name, int32(x)) } -func (x Days) MarshalJSON() ([]byte, error) { - return json.Marshal(x.String()) -} func (x *Days) UnmarshalJSON(data []byte) error { value, err := proto.UnmarshalJSONEnum(Days_value, data, "Days") if err != nil { @@ -119,9 +113,6 @@ func (x Request_Color) Enum() *Request_Color { func (x Request_Color) String() string { return proto.EnumName(Request_Color_name, int32(x)) } -func (x Request_Color) MarshalJSON() ([]byte, error) { - return json.Marshal(x.String()) -} func (x *Request_Color) UnmarshalJSON(data []byte) error { value, err := proto.UnmarshalJSONEnum(Request_Color_value, data, "Request_Color") if err != nil { @@ -155,9 +146,6 @@ func (x Reply_Entry_Game) Enum() *Reply_Entry_Game { func (x Reply_Entry_Game) String() string { return proto.EnumName(Reply_Entry_Game_name, int32(x)) } -func (x Reply_Entry_Game) MarshalJSON() ([]byte, error) { - return json.Marshal(x.String()) -} func (x *Reply_Entry_Game) UnmarshalJSON(data []byte) error { value, err := proto.UnmarshalJSONEnum(Reply_Entry_Game_value, data, "Reply_Entry_Game") if err != nil { From 0c5808eeec047fc9075b4e37beccd3e802971721 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 13 Oct 2013 10:44:40 -0700 Subject: [PATCH 083/247] fix server/server.go accept name when creating a when server; gofmt --- etcd_test.go | 1 - server/join_command.go | 16 +- server/peer_server.go | 823 ++++++++++++++++---------------- server/registry.go | 252 +++++----- server/server.go | 31 +- server/timeout.go | 12 +- server/transporter.go | 256 +++++----- server/util.go | 31 +- server/v1/delete_key_handler.go | 4 +- server/v1/v1.go | 2 +- server/v2/create_key_handler.go | 36 +- server/v2/delete_key_handler.go | 20 +- server/v2/get_key_handler.go | 104 ++-- server/v2/update_key_handler.go | 92 ++-- server/v2/v2.go | 20 +- store/create_command.go | 2 +- store/delete_command.go | 2 +- store/event.go | 2 - store/event_history.go | 160 +++---- store/event_queue.go | 25 +- store/test_and_set_command.go | 2 +- store/ttl.go | 23 +- store/update_command.go | 2 +- util.go | 2 - 24 files changed, 955 insertions(+), 965 deletions(-) diff --git a/etcd_test.go b/etcd_test.go index d868be89667..95e4c3c525e 100644 --- a/etcd_test.go +++ b/etcd_test.go @@ -32,7 +32,6 @@ func TestSingleNode(t *testing.T) { time.Sleep(time.Second) - etcd.OpenDebug() c := etcd.NewClient() c.SyncCluster() diff --git a/server/join_command.go b/server/join_command.go index c8ad6760b3a..d1a0dcc4250 100644 --- a/server/join_command.go +++ b/server/join_command.go @@ -14,18 +14,18 @@ func init() { // The JoinCommand adds a node to the cluster. type JoinCommand struct { - RaftVersion string `json:"raftVersion"` - Name string `json:"name"` - RaftURL string `json:"raftURL"` - EtcdURL string `json:"etcdURL"` + RaftVersion string `json:"raftVersion"` + Name string `json:"name"` + RaftURL string `json:"raftURL"` + EtcdURL string `json:"etcdURL"` } func NewJoinCommand(version, name, raftUrl, etcdUrl string) *JoinCommand { return &JoinCommand{ - RaftVersion: version, - Name: name, - RaftURL: raftUrl, - EtcdURL: etcdUrl, + RaftVersion: version, + Name: name, + RaftURL: raftUrl, + EtcdURL: etcdUrl, } } diff --git a/server/peer_server.go b/server/peer_server.go index d016717b17f..0b16f98e8ef 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -1,363 +1,362 @@ package server import ( - "bytes" - "crypto/tls" - "encoding/binary" - "encoding/json" - "fmt" - "io/ioutil" - "net/http" - "net/url" - "time" - - etcdErr "github.com/coreos/etcd/error" - "github.com/coreos/etcd/log" - "github.com/coreos/etcd/store" - "github.com/coreos/go-raft" + "bytes" + "crypto/tls" + "encoding/binary" + "encoding/json" + "fmt" + "io/ioutil" + "net/http" + "net/url" + "time" + + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" ) type PeerServer struct { - *raft.Server - server *Server - joinIndex uint64 - name string - url string - listenHost string - tlsConf *TLSConfig - tlsInfo *TLSInfo - followersStats *raftFollowersStats - serverStats *raftServerStats - registry *Registry - store *store.Store - snapConf *snapshotConf - MaxClusterSize int - RetryTimes int + *raft.Server + server *Server + joinIndex uint64 + name string + url string + listenHost string + tlsConf *TLSConfig + tlsInfo *TLSInfo + followersStats *raftFollowersStats + serverStats *raftServerStats + registry *Registry + store *store.Store + snapConf *snapshotConf + MaxClusterSize int + RetryTimes int } // TODO: find a good policy to do snapshot type snapshotConf struct { - // Etcd will check if snapshot is need every checkingInterval - checkingInterval time.Duration + // Etcd will check if snapshot is need every checkingInterval + checkingInterval time.Duration - // The number of writes when the last snapshot happened - lastWrites uint64 + // The number of writes when the last snapshot happened + lastWrites uint64 - // If the incremental number of writes since the last snapshot - // exceeds the write Threshold, etcd will do a snapshot - writesThr uint64 + // If the incremental number of writes since the last snapshot + // exceeds the write Threshold, etcd will do a snapshot + writesThr uint64 } - func NewPeerServer(name string, path string, url string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, registry *Registry, store *store.Store) *PeerServer { - s := &PeerServer{ - name: name, - url: url, - listenHost: listenHost, - tlsConf: tlsConf, - tlsInfo: tlsInfo, - registry: registry, - store: store, - snapConf: &snapshotConf{time.Second * 3, 0, 20 * 1000}, - followersStats: &raftFollowersStats{ - Leader: name, - Followers: make(map[string]*raftFollowerStats), - }, - serverStats: &raftServerStats{ - StartTime: time.Now(), - sendRateQueue: &statsQueue{ - back: -1, - }, - recvRateQueue: &statsQueue{ - back: -1, - }, - }, - } - - // Create transporter for raft - raftTransporter := newTransporter(tlsConf.Scheme, tlsConf.Client, s) - - // Create raft server - server, err := raft.NewServer(name, path, raftTransporter, s.store, s, "") - if err != nil { - log.Fatal(err) - } - - s.Server = server - - return s + s := &PeerServer{ + name: name, + url: url, + listenHost: listenHost, + tlsConf: tlsConf, + tlsInfo: tlsInfo, + registry: registry, + store: store, + snapConf: &snapshotConf{time.Second * 3, 0, 20 * 1000}, + followersStats: &raftFollowersStats{ + Leader: name, + Followers: make(map[string]*raftFollowerStats), + }, + serverStats: &raftServerStats{ + StartTime: time.Now(), + sendRateQueue: &statsQueue{ + back: -1, + }, + recvRateQueue: &statsQueue{ + back: -1, + }, + }, + } + + // Create transporter for raft + raftTransporter := newTransporter(tlsConf.Scheme, tlsConf.Client, s) + + // Create raft server + server, err := raft.NewServer(name, path, raftTransporter, s.store, s, "") + if err != nil { + log.Fatal(err) + } + + s.Server = server + + return s } // Start the raft server func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) { - // LoadSnapshot - if snapshot { - err := s.LoadSnapshot() - - if err == nil { - log.Debugf("%s finished load snapshot", s.name) - } else { - log.Debug(err) - } - } - - s.SetElectionTimeout(ElectionTimeout) - s.SetHeartbeatTimeout(HeartbeatTimeout) - - s.Start() - - if s.IsLogEmpty() { - // start as a leader in a new cluster - if len(cluster) == 0 { - s.startAsLeader() - } else { - s.startAsFollower(cluster) - } - - } else { - // Rejoin the previous cluster - cluster = s.registry.PeerURLs(s.Leader(), s.name) - for i := 0; i < len(cluster); i++ { - u, err := url.Parse(cluster[i]) - if err != nil { - log.Debug("rejoin cannot parse url: ", err) - } - cluster[i] = u.Host - } - ok := s.joinCluster(cluster) - if !ok { - log.Warn("the entire cluster is down! this machine will restart the cluster.") - } - - log.Debugf("%s restart as a follower", s.name) - } - - // open the snapshot - if snapshot { - go s.monitorSnapshot() - } - - // start to response to raft requests - go s.startTransport(s.tlsConf.Scheme, s.tlsConf.Server) + // LoadSnapshot + if snapshot { + err := s.LoadSnapshot() + + if err == nil { + log.Debugf("%s finished load snapshot", s.name) + } else { + log.Debug(err) + } + } + + s.SetElectionTimeout(ElectionTimeout) + s.SetHeartbeatTimeout(HeartbeatTimeout) + + s.Start() + + if s.IsLogEmpty() { + // start as a leader in a new cluster + if len(cluster) == 0 { + s.startAsLeader() + } else { + s.startAsFollower(cluster) + } + + } else { + // Rejoin the previous cluster + cluster = s.registry.PeerURLs(s.Leader(), s.name) + for i := 0; i < len(cluster); i++ { + u, err := url.Parse(cluster[i]) + if err != nil { + log.Debug("rejoin cannot parse url: ", err) + } + cluster[i] = u.Host + } + ok := s.joinCluster(cluster) + if !ok { + log.Warn("the entire cluster is down! this machine will restart the cluster.") + } + + log.Debugf("%s restart as a follower", s.name) + } + + // open the snapshot + if snapshot { + go s.monitorSnapshot() + } + + // start to response to raft requests + go s.startTransport(s.tlsConf.Scheme, s.tlsConf.Server) } // Retrieves the underlying Raft server. func (s *PeerServer) RaftServer() *raft.Server { - return s.Server + return s.Server } // Associates the client server with the peer server. func (s *PeerServer) SetServer(server *Server) { - s.server = server + s.server = server } // Get all the current logs func (s *PeerServer) GetLogHttpHandler(w http.ResponseWriter, req *http.Request) { - log.Debugf("[recv] GET %s/log", s.url) - w.Header().Set("Content-Type", "application/json") - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(s.LogEntries()) + log.Debugf("[recv] GET %s/log", s.url) + w.Header().Set("Content-Type", "application/json") + w.WriteHeader(http.StatusOK) + json.NewEncoder(w).Encode(s.LogEntries()) } // Response to vote request func (s *PeerServer) VoteHttpHandler(w http.ResponseWriter, req *http.Request) { - rvreq := &raft.RequestVoteRequest{} - err := decodeJsonRequest(req, rvreq) - if err == nil { - log.Debugf("[recv] POST %s/vote [%s]", s.url, rvreq.CandidateName) - if resp := s.RequestVote(rvreq); resp != nil { - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(resp) - return - } - } - log.Warnf("[vote] ERROR: %v", err) - w.WriteHeader(http.StatusInternalServerError) + rvreq := &raft.RequestVoteRequest{} + err := decodeJsonRequest(req, rvreq) + if err == nil { + log.Debugf("[recv] POST %s/vote [%s]", s.url, rvreq.CandidateName) + if resp := s.RequestVote(rvreq); resp != nil { + w.WriteHeader(http.StatusOK) + json.NewEncoder(w).Encode(resp) + return + } + } + log.Warnf("[vote] ERROR: %v", err) + w.WriteHeader(http.StatusInternalServerError) } // Response to append entries request func (s *PeerServer) AppendEntriesHttpHandler(w http.ResponseWriter, req *http.Request) { - aereq := &raft.AppendEntriesRequest{} - err := decodeJsonRequest(req, aereq) - - if err == nil { - log.Debugf("[recv] POST %s/log/append [%d]", s.url, len(aereq.Entries)) - - s.serverStats.RecvAppendReq(aereq.LeaderName, int(req.ContentLength)) - - if resp := s.AppendEntries(aereq); resp != nil { - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(resp) - if !resp.Success { - log.Debugf("[Append Entry] Step back") - } - return - } - } - log.Warnf("[Append Entry] ERROR: %v", err) - w.WriteHeader(http.StatusInternalServerError) + aereq := &raft.AppendEntriesRequest{} + err := decodeJsonRequest(req, aereq) + + if err == nil { + log.Debugf("[recv] POST %s/log/append [%d]", s.url, len(aereq.Entries)) + + s.serverStats.RecvAppendReq(aereq.LeaderName, int(req.ContentLength)) + + if resp := s.AppendEntries(aereq); resp != nil { + w.WriteHeader(http.StatusOK) + json.NewEncoder(w).Encode(resp) + if !resp.Success { + log.Debugf("[Append Entry] Step back") + } + return + } + } + log.Warnf("[Append Entry] ERROR: %v", err) + w.WriteHeader(http.StatusInternalServerError) } // Response to recover from snapshot request func (s *PeerServer) SnapshotHttpHandler(w http.ResponseWriter, req *http.Request) { - aereq := &raft.SnapshotRequest{} - err := decodeJsonRequest(req, aereq) - if err == nil { - log.Debugf("[recv] POST %s/snapshot/ ", s.url) - if resp := s.RequestSnapshot(aereq); resp != nil { - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(resp) - return - } - } - log.Warnf("[Snapshot] ERROR: %v", err) - w.WriteHeader(http.StatusInternalServerError) + aereq := &raft.SnapshotRequest{} + err := decodeJsonRequest(req, aereq) + if err == nil { + log.Debugf("[recv] POST %s/snapshot/ ", s.url) + if resp := s.RequestSnapshot(aereq); resp != nil { + w.WriteHeader(http.StatusOK) + json.NewEncoder(w).Encode(resp) + return + } + } + log.Warnf("[Snapshot] ERROR: %v", err) + w.WriteHeader(http.StatusInternalServerError) } // Response to recover from snapshot request func (s *PeerServer) SnapshotRecoveryHttpHandler(w http.ResponseWriter, req *http.Request) { - aereq := &raft.SnapshotRecoveryRequest{} - err := decodeJsonRequest(req, aereq) - if err == nil { - log.Debugf("[recv] POST %s/snapshotRecovery/ ", s.url) - if resp := s.SnapshotRecoveryRequest(aereq); resp != nil { - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(resp) - return - } - } - log.Warnf("[Snapshot] ERROR: %v", err) - w.WriteHeader(http.StatusInternalServerError) + aereq := &raft.SnapshotRecoveryRequest{} + err := decodeJsonRequest(req, aereq) + if err == nil { + log.Debugf("[recv] POST %s/snapshotRecovery/ ", s.url) + if resp := s.SnapshotRecoveryRequest(aereq); resp != nil { + w.WriteHeader(http.StatusOK) + json.NewEncoder(w).Encode(resp) + return + } + } + log.Warnf("[Snapshot] ERROR: %v", err) + w.WriteHeader(http.StatusInternalServerError) } // Get the port that listening for etcd connecting of the server func (s *PeerServer) EtcdURLHttpHandler(w http.ResponseWriter, req *http.Request) { - log.Debugf("[recv] Get %s/etcdURL/ ", s.url) - w.WriteHeader(http.StatusOK) - w.Write([]byte(s.server.URL())) + log.Debugf("[recv] Get %s/etcdURL/ ", s.url) + w.WriteHeader(http.StatusOK) + w.Write([]byte(s.server.URL())) } // Response to the join request func (s *PeerServer) JoinHttpHandler(w http.ResponseWriter, req *http.Request) { - command := &JoinCommand{} - - // Write CORS header. - if s.server.OriginAllowed("*") { - w.Header().Add("Access-Control-Allow-Origin", "*") - } else if s.server.OriginAllowed(req.Header.Get("Origin")) { - w.Header().Add("Access-Control-Allow-Origin", req.Header.Get("Origin")) - } - - err := decodeJsonRequest(req, command) - if err != nil { - w.WriteHeader(http.StatusInternalServerError) - return - } - - log.Debugf("Receive Join Request from %s", command.Name) - err = s.dispatchRaftCommand(command, w, req) - - // Return status. - if err != nil { - if etcdErr, ok := err.(*etcdErr.Error); ok { - log.Debug("Return error: ", (*etcdErr).Error()) - etcdErr.Write(w) - } else { - http.Error(w, err.Error(), http.StatusInternalServerError) - } - } + command := &JoinCommand{} + + // Write CORS header. + if s.server.OriginAllowed("*") { + w.Header().Add("Access-Control-Allow-Origin", "*") + } else if s.server.OriginAllowed(req.Header.Get("Origin")) { + w.Header().Add("Access-Control-Allow-Origin", req.Header.Get("Origin")) + } + + err := decodeJsonRequest(req, command) + if err != nil { + w.WriteHeader(http.StatusInternalServerError) + return + } + + log.Debugf("Receive Join Request from %s", command.Name) + err = s.dispatchRaftCommand(command, w, req) + + // Return status. + if err != nil { + if etcdErr, ok := err.(*etcdErr.Error); ok { + log.Debug("Return error: ", (*etcdErr).Error()) + etcdErr.Write(w) + } else { + http.Error(w, err.Error(), http.StatusInternalServerError) + } + } } // Response to remove request func (s *PeerServer) RemoveHttpHandler(w http.ResponseWriter, req *http.Request) { - if req.Method != "DELETE" { - w.WriteHeader(http.StatusMethodNotAllowed) - return - } + if req.Method != "DELETE" { + w.WriteHeader(http.StatusMethodNotAllowed) + return + } - nodeName := req.URL.Path[len("/remove/"):] - command := &RemoveCommand{ - Name: nodeName, - } + nodeName := req.URL.Path[len("/remove/"):] + command := &RemoveCommand{ + Name: nodeName, + } - log.Debugf("[recv] Remove Request [%s]", command.Name) + log.Debugf("[recv] Remove Request [%s]", command.Name) - s.dispatchRaftCommand(command, w, req) + s.dispatchRaftCommand(command, w, req) } // Response to the name request func (s *PeerServer) NameHttpHandler(w http.ResponseWriter, req *http.Request) { - log.Debugf("[recv] Get %s/name/ ", s.url) - w.WriteHeader(http.StatusOK) - w.Write([]byte(s.name)) + log.Debugf("[recv] Get %s/name/ ", s.url) + w.WriteHeader(http.StatusOK) + w.Write([]byte(s.name)) } // Response to the name request func (s *PeerServer) RaftVersionHttpHandler(w http.ResponseWriter, req *http.Request) { - log.Debugf("[recv] Get %s/version/ ", s.url) - w.WriteHeader(http.StatusOK) - w.Write([]byte(PeerVersion)) + log.Debugf("[recv] Get %s/version/ ", s.url) + w.WriteHeader(http.StatusOK) + w.Write([]byte(PeerVersion)) } func (s *PeerServer) dispatchRaftCommand(c raft.Command, w http.ResponseWriter, req *http.Request) error { - return s.dispatch(c, w, req) + return s.dispatch(c, w, req) } func (s *PeerServer) startAsLeader() { - // leader need to join self as a peer - for { - _, err := s.Do(NewJoinCommand(PeerVersion, s.Name(), s.url, s.server.URL())) - if err == nil { - break - } - } - log.Debugf("%s start as a leader", s.name) + // leader need to join self as a peer + for { + _, err := s.Do(NewJoinCommand(PeerVersion, s.Name(), s.url, s.server.URL())) + if err == nil { + break + } + } + log.Debugf("%s start as a leader", s.name) } func (s *PeerServer) startAsFollower(cluster []string) { - // start as a follower in a existing cluster - for i := 0; i < s.RetryTimes; i++ { - ok := s.joinCluster(cluster) - if ok { - return - } - log.Warnf("cannot join to cluster via given machines, retry in %d seconds", RetryInterval) - time.Sleep(time.Second * RetryInterval) - } - - log.Fatalf("Cannot join the cluster via given machines after %x retries", s.RetryTimes) + // start as a follower in a existing cluster + for i := 0; i < s.RetryTimes; i++ { + ok := s.joinCluster(cluster) + if ok { + return + } + log.Warnf("cannot join to cluster via given machines, retry in %d seconds", RetryInterval) + time.Sleep(time.Second * RetryInterval) + } + + log.Fatalf("Cannot join the cluster via given machines after %x retries", s.RetryTimes) } // Start to listen and response raft command func (s *PeerServer) startTransport(scheme string, tlsConf tls.Config) { - log.Infof("raft server [name %s, listen on %s, advertised url %s]", s.name, s.listenHost, s.url) - - raftMux := http.NewServeMux() - - server := &http.Server{ - Handler: raftMux, - TLSConfig: &tlsConf, - Addr: s.listenHost, - } - - // internal commands - raftMux.HandleFunc("/name", s.NameHttpHandler) - raftMux.HandleFunc("/version", s.RaftVersionHttpHandler) - raftMux.HandleFunc("/join", s.JoinHttpHandler) - raftMux.HandleFunc("/remove/", s.RemoveHttpHandler) - raftMux.HandleFunc("/vote", s.VoteHttpHandler) - raftMux.HandleFunc("/log", s.GetLogHttpHandler) - raftMux.HandleFunc("/log/append", s.AppendEntriesHttpHandler) - raftMux.HandleFunc("/snapshot", s.SnapshotHttpHandler) - raftMux.HandleFunc("/snapshotRecovery", s.SnapshotRecoveryHttpHandler) - raftMux.HandleFunc("/etcdURL", s.EtcdURLHttpHandler) - - if scheme == "http" { - log.Fatal(server.ListenAndServe()) - } else { - log.Fatal(server.ListenAndServeTLS(s.tlsInfo.CertFile, s.tlsInfo.KeyFile)) - } + log.Infof("raft server [name %s, listen on %s, advertised url %s]", s.name, s.listenHost, s.url) + + raftMux := http.NewServeMux() + + server := &http.Server{ + Handler: raftMux, + TLSConfig: &tlsConf, + Addr: s.listenHost, + } + + // internal commands + raftMux.HandleFunc("/name", s.NameHttpHandler) + raftMux.HandleFunc("/version", s.RaftVersionHttpHandler) + raftMux.HandleFunc("/join", s.JoinHttpHandler) + raftMux.HandleFunc("/remove/", s.RemoveHttpHandler) + raftMux.HandleFunc("/vote", s.VoteHttpHandler) + raftMux.HandleFunc("/log", s.GetLogHttpHandler) + raftMux.HandleFunc("/log/append", s.AppendEntriesHttpHandler) + raftMux.HandleFunc("/snapshot", s.SnapshotHttpHandler) + raftMux.HandleFunc("/snapshotRecovery", s.SnapshotRecoveryHttpHandler) + raftMux.HandleFunc("/etcdURL", s.EtcdURLHttpHandler) + + if scheme == "http" { + log.Fatal(server.ListenAndServe()) + } else { + log.Fatal(server.ListenAndServeTLS(s.tlsInfo.CertFile, s.tlsInfo.KeyFile)) + } } @@ -365,184 +364,182 @@ func (s *PeerServer) startTransport(scheme string, tlsConf tls.Config) { // will need to do something more sophisticated later when we allow mixed // version clusters. func getVersion(t *transporter, versionURL url.URL) (string, error) { - resp, req, err := t.Get(versionURL.String()) - if err != nil { - return "", err - } - defer resp.Body.Close() + resp, req, err := t.Get(versionURL.String()) + if err != nil { + return "", err + } + defer resp.Body.Close() - t.CancelWhenTimeout(req) + t.CancelWhenTimeout(req) - body, err := ioutil.ReadAll(resp.Body) + body, err := ioutil.ReadAll(resp.Body) - return string(body), nil + return string(body), nil } func (s *PeerServer) joinCluster(cluster []string) bool { - for _, machine := range cluster { - if len(machine) == 0 { - continue - } - - err := s.joinByMachine(s.Server, machine, s.tlsConf.Scheme) - if err == nil { - log.Debugf("%s success join to the cluster via machine %s", s.name, machine) - return true - - } else { - if _, ok := err.(etcdErr.Error); ok { - log.Fatal(err) - } - - log.Debugf("cannot join to cluster via machine %s %s", machine, err) - } - } - return false + for _, machine := range cluster { + if len(machine) == 0 { + continue + } + + err := s.joinByMachine(s.Server, machine, s.tlsConf.Scheme) + if err == nil { + log.Debugf("%s success join to the cluster via machine %s", s.name, machine) + return true + + } else { + if _, ok := err.(etcdErr.Error); ok { + log.Fatal(err) + } + + log.Debugf("cannot join to cluster via machine %s %s", machine, err) + } + } + return false } // Send join requests to machine. func (s *PeerServer) joinByMachine(server *raft.Server, machine string, scheme string) error { - var b bytes.Buffer + var b bytes.Buffer - // t must be ok - t, _ := server.Transporter().(*transporter) + // t must be ok + t, _ := server.Transporter().(*transporter) - // Our version must match the leaders version - versionURL := url.URL{Host: machine, Scheme: scheme, Path: "/version"} - version, err := getVersion(t, versionURL) - if err != nil { - return fmt.Errorf("Unable to join: %v", err) - } + // Our version must match the leaders version + versionURL := url.URL{Host: machine, Scheme: scheme, Path: "/version"} + version, err := getVersion(t, versionURL) + if err != nil { + return fmt.Errorf("Unable to join: %v", err) + } - // TODO: versioning of the internal protocol. See: - // Documentation/internatl-protocol-versioning.md - if version != PeerVersion { - return fmt.Errorf("Unable to join: internal version mismatch, entire cluster must be running identical versions of etcd") - } + // TODO: versioning of the internal protocol. See: + // Documentation/internatl-protocol-versioning.md + if version != PeerVersion { + return fmt.Errorf("Unable to join: internal version mismatch, entire cluster must be running identical versions of etcd") + } - json.NewEncoder(&b).Encode(NewJoinCommand(PeerVersion, server.Name(), s.url, s.server.URL())) + json.NewEncoder(&b).Encode(NewJoinCommand(PeerVersion, server.Name(), s.url, s.server.URL())) - joinURL := url.URL{Host: machine, Scheme: scheme, Path: "/join"} + joinURL := url.URL{Host: machine, Scheme: scheme, Path: "/join"} - log.Debugf("Send Join Request to %s", joinURL.String()) + log.Debugf("Send Join Request to %s", joinURL.String()) - resp, req, err := t.Post(joinURL.String(), &b) + resp, req, err := t.Post(joinURL.String(), &b) - for { - if err != nil { - return fmt.Errorf("Unable to join: %v", err) - } - if resp != nil { - defer resp.Body.Close() + for { + if err != nil { + return fmt.Errorf("Unable to join: %v", err) + } + if resp != nil { + defer resp.Body.Close() - t.CancelWhenTimeout(req) + t.CancelWhenTimeout(req) - if resp.StatusCode == http.StatusOK { - b, _ := ioutil.ReadAll(resp.Body) - s.joinIndex, _ = binary.Uvarint(b) - return nil - } - if resp.StatusCode == http.StatusTemporaryRedirect { + if resp.StatusCode == http.StatusOK { + b, _ := ioutil.ReadAll(resp.Body) + s.joinIndex, _ = binary.Uvarint(b) + return nil + } + if resp.StatusCode == http.StatusTemporaryRedirect { - address := resp.Header.Get("Location") - log.Debugf("Send Join Request to %s", address) + address := resp.Header.Get("Location") + log.Debugf("Send Join Request to %s", address) - json.NewEncoder(&b).Encode(NewJoinCommand(PeerVersion, server.Name(), s.url, s.server.URL())) + json.NewEncoder(&b).Encode(NewJoinCommand(PeerVersion, server.Name(), s.url, s.server.URL())) - resp, req, err = t.Post(address, &b) + resp, req, err = t.Post(address, &b) - } else if resp.StatusCode == http.StatusBadRequest { - log.Debug("Reach max number machines in the cluster") - decoder := json.NewDecoder(resp.Body) - err := &etcdErr.Error{} - decoder.Decode(err) - return *err - } else { - return fmt.Errorf("Unable to join") - } - } + } else if resp.StatusCode == http.StatusBadRequest { + log.Debug("Reach max number machines in the cluster") + decoder := json.NewDecoder(resp.Body) + err := &etcdErr.Error{} + decoder.Decode(err) + return *err + } else { + return fmt.Errorf("Unable to join") + } + } - } - return fmt.Errorf("Unable to join: %v", err) + } } func (s *PeerServer) Stats() []byte { - s.serverStats.LeaderInfo.Uptime = time.Now().Sub(s.serverStats.LeaderInfo.startTime).String() + s.serverStats.LeaderInfo.Uptime = time.Now().Sub(s.serverStats.LeaderInfo.startTime).String() - queue := s.serverStats.sendRateQueue + queue := s.serverStats.sendRateQueue - s.serverStats.SendingPkgRate, s.serverStats.SendingBandwidthRate = queue.Rate() + s.serverStats.SendingPkgRate, s.serverStats.SendingBandwidthRate = queue.Rate() - queue = s.serverStats.recvRateQueue + queue = s.serverStats.recvRateQueue - s.serverStats.RecvingPkgRate, s.serverStats.RecvingBandwidthRate = queue.Rate() + s.serverStats.RecvingPkgRate, s.serverStats.RecvingBandwidthRate = queue.Rate() - b, _ := json.Marshal(s.serverStats) + b, _ := json.Marshal(s.serverStats) - return b + return b } func (s *PeerServer) PeerStats() []byte { - if s.State() == raft.Leader { - b, _ := json.Marshal(s.followersStats) - return b - } - return nil + if s.State() == raft.Leader { + b, _ := json.Marshal(s.followersStats) + return b + } + return nil } func (s *PeerServer) monitorSnapshot() { - for { - time.Sleep(s.snapConf.checkingInterval) - currentWrites := 0 - if uint64(currentWrites) > s.snapConf.writesThr { - s.TakeSnapshot() - s.snapConf.lastWrites = 0 - } - } + for { + time.Sleep(s.snapConf.checkingInterval) + currentWrites := 0 + if uint64(currentWrites) > s.snapConf.writesThr { + s.TakeSnapshot() + s.snapConf.lastWrites = 0 + } + } } func (s *PeerServer) dispatch(c raft.Command, w http.ResponseWriter, req *http.Request) error { - if s.State() == raft.Leader { - if response, err := s.Do(c); err != nil { - return err - } else { - if response == nil { - return etcdErr.NewError(300, "Empty response from raft", store.UndefIndex, store.UndefTerm) - } - - event, ok := response.(*store.Event) - if ok { - bytes, err := json.Marshal(event) - if err != nil { - fmt.Println(err) - } - - w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) - w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term)) - w.WriteHeader(http.StatusOK) - w.Write(bytes) - - return nil - } - - bytes, _ := response.([]byte) - w.WriteHeader(http.StatusOK) - w.Write(bytes) - - return nil - } - - } else { - leader := s.Leader() - // current no leader - if leader == "" { - return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) - } - url, _ := s.registry.PeerURL(leader) - - redirect(url, w, req) - - return nil - } + if s.State() == raft.Leader { + if response, err := s.Do(c); err != nil { + return err + } else { + if response == nil { + return etcdErr.NewError(300, "Empty response from raft", store.UndefIndex, store.UndefTerm) + } + + event, ok := response.(*store.Event) + if ok { + bytes, err := json.Marshal(event) + if err != nil { + fmt.Println(err) + } + + w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) + w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term)) + w.WriteHeader(http.StatusOK) + w.Write(bytes) + + return nil + } + + bytes, _ := response.([]byte) + w.WriteHeader(http.StatusOK) + w.Write(bytes) + + return nil + } + + } else { + leader := s.Leader() + // current no leader + if leader == "" { + return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) + } + url, _ := s.registry.PeerURL(leader) + + redirect(url, w, req) + + return nil + } } - diff --git a/server/registry.go b/server/registry.go index 8a62dff4081..82d3b42a288 100644 --- a/server/registry.go +++ b/server/registry.go @@ -1,14 +1,14 @@ package server import ( - "fmt" - "net/url" - "path" - "strings" - "sync" - - "github.com/coreos/etcd/log" - "github.com/coreos/etcd/store" + "fmt" + "net/url" + "path" + "strings" + "sync" + + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" ) // The location of the machine URL data. @@ -16,174 +16,174 @@ const RegistryKey = "/_etcd/machines" // The Registry stores URL information for nodes. type Registry struct { - sync.Mutex - store *store.Store - nodes map[string]*node + sync.Mutex + store *store.Store + nodes map[string]*node } // The internal storage format of the registry. type node struct { - peerVersion string - peerURL string - url string + peerVersion string + peerURL string + url string } // Creates a new Registry. func NewRegistry(s *store.Store) *Registry { - return &Registry{ - store: s, - nodes: make(map[string]*node), - } + return &Registry{ + store: s, + nodes: make(map[string]*node), + } } // Adds a node to the registry. func (r *Registry) Register(name string, peerVersion string, peerURL string, url string, commitIndex uint64, term uint64) { - r.Lock() - defer r.Unlock() + r.Lock() + defer r.Unlock() - // Write data to store. - key := path.Join(RegistryKey, name) - value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", peerURL, url, peerVersion) - r.store.Create(key, value, false, false, store.Permanent, commitIndex, term) + // Write data to store. + key := path.Join(RegistryKey, name) + value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", peerURL, url, peerVersion) + r.store.Create(key, value, false, false, store.Permanent, commitIndex, term) } // Removes a node from the registry. func (r *Registry) Unregister(name string, commitIndex uint64, term uint64) error { - r.Lock() - defer r.Unlock() + r.Lock() + defer r.Unlock() - // Remove the key from the store. - _, err := r.store.Delete(path.Join(RegistryKey, name), false, commitIndex, term) - return err + // Remove the key from the store. + _, err := r.store.Delete(path.Join(RegistryKey, name), false, commitIndex, term) + return err } // Returns the number of nodes in the cluster. func (r *Registry) Count() int { - e, err := r.store.Get(RegistryKey, false, false, 0, 0) - if err != nil { - return 0 - } - return len(e.KVPairs) + e, err := r.store.Get(RegistryKey, false, false, 0, 0) + if err != nil { + return 0 + } + return len(e.KVPairs) } // Retrieves the URL for a given node by name. func (r *Registry) URL(name string) (string, bool) { - r.Lock() - defer r.Unlock() - return r.url(name) + r.Lock() + defer r.Unlock() + return r.url(name) } func (r *Registry) url(name string) (string, bool) { - if r.nodes[name] == nil { - r.load(name) - } + if r.nodes[name] == nil { + r.load(name) + } - if node := r.nodes[name]; node != nil { - return node.url, true - } + if node := r.nodes[name]; node != nil { + return node.url, true + } - return "", false + return "", false } // Retrieves the URLs for all nodes. func (r *Registry) URLs(leaderName, selfName string) []string { - r.Lock() - defer r.Unlock() - - // Build list including the leader and self. - urls := make([]string, 0) - if url, _ := r.url(leaderName); len(url) > 0 { - urls = append(urls, url) - } - if url, _ := r.url(selfName); len(url) > 0 { - urls = append(urls, url) - } - - // Retrieve a list of all nodes. - if e, _ := r.store.Get(RegistryKey, false, false, 0, 0); e != nil { - // Lookup the URL for each one. - for _, pair := range e.KVPairs { - if url, _ := r.url(pair.Key); len(url) > 0 { - urls = append(urls, url) - } - } - } - - log.Infof("URLs: %s / %s (%s", leaderName, selfName, strings.Join(urls, ",")) - - return urls + r.Lock() + defer r.Unlock() + + // Build list including the leader and self. + urls := make([]string, 0) + if url, _ := r.url(leaderName); len(url) > 0 { + urls = append(urls, url) + } + if url, _ := r.url(selfName); len(url) > 0 { + urls = append(urls, url) + } + + // Retrieve a list of all nodes. + if e, _ := r.store.Get(RegistryKey, false, false, 0, 0); e != nil { + // Lookup the URL for each one. + for _, pair := range e.KVPairs { + if url, _ := r.url(pair.Key); len(url) > 0 { + urls = append(urls, url) + } + } + } + + log.Infof("URLs: %s / %s (%s", leaderName, selfName, strings.Join(urls, ",")) + + return urls } // Retrieves the peer URL for a given node by name. func (r *Registry) PeerURL(name string) (string, bool) { - r.Lock() - defer r.Unlock() - return r.peerURL(name) + r.Lock() + defer r.Unlock() + return r.peerURL(name) } func (r *Registry) peerURL(name string) (string, bool) { - if r.nodes[name] == nil { - r.load(name) - } + if r.nodes[name] == nil { + r.load(name) + } - if node := r.nodes[name]; node != nil { - return node.peerURL, true - } + if node := r.nodes[name]; node != nil { + return node.peerURL, true + } - return "", false + return "", false } // Retrieves the peer URLs for all nodes. func (r *Registry) PeerURLs(leaderName, selfName string) []string { - r.Lock() - defer r.Unlock() - - // Build list including the leader and self. - urls := make([]string, 0) - if url, _ := r.peerURL(leaderName); len(url) > 0 { - urls = append(urls, url) - } - if url, _ := r.peerURL(selfName); len(url) > 0 { - urls = append(urls, url) - } - - // Retrieve a list of all nodes. - if e, _ := r.store.Get(RegistryKey, false, false, 0, 0); e != nil { - // Lookup the URL for each one. - for _, pair := range e.KVPairs { - if url, _ := r.peerURL(pair.Key); len(url) > 0 { - urls = append(urls, url) - } - } - } - - log.Infof("PeerURLs: %s / %s (%s", leaderName, selfName, strings.Join(urls, ",")) - - return urls + r.Lock() + defer r.Unlock() + + // Build list including the leader and self. + urls := make([]string, 0) + if url, _ := r.peerURL(leaderName); len(url) > 0 { + urls = append(urls, url) + } + if url, _ := r.peerURL(selfName); len(url) > 0 { + urls = append(urls, url) + } + + // Retrieve a list of all nodes. + if e, _ := r.store.Get(RegistryKey, false, false, 0, 0); e != nil { + // Lookup the URL for each one. + for _, pair := range e.KVPairs { + if url, _ := r.peerURL(pair.Key); len(url) > 0 { + urls = append(urls, url) + } + } + } + + log.Infof("PeerURLs: %s / %s (%s", leaderName, selfName, strings.Join(urls, ",")) + + return urls } // Loads the given node by name from the store into the cache. func (r *Registry) load(name string) { - if name == "" { - return - } - - // Retrieve from store. - e, err := r.store.Get(path.Join(RegistryKey, name), false, false, 0, 0) - if err != nil { - return - } - - // Parse as a query string. - m, err := url.ParseQuery(e.Value) - if err != nil { - panic(fmt.Sprintf("Failed to parse machines entry: %s", name)) - } - - // Create node. - r.nodes[name] = &node{ - url: m["etcd"][0], - peerURL: m["raft"][0], - peerVersion: m["raftVersion"][0], - } + if name == "" { + return + } + + // Retrieve from store. + e, err := r.store.Get(path.Join(RegistryKey, name), false, false, 0, 0) + if err != nil { + return + } + + // Parse as a query string. + m, err := url.ParseQuery(e.Value) + if err != nil { + panic(fmt.Sprintf("Failed to parse machines entry: %s", name)) + } + + // Create node. + r.nodes[name] = &node{ + url: m["etcd"][0], + peerURL: m["raft"][0], + peerVersion: m["raftVersion"][0], + } } diff --git a/server/server.go b/server/server.go index 98492711020..76a62fdce70 100644 --- a/server/server.go +++ b/server/server.go @@ -1,8 +1,8 @@ package server import ( - "fmt" "encoding/json" + "fmt" "net/http" "net/url" "strings" @@ -38,6 +38,7 @@ func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsI TLSConfig: &tlsConf.Server, Addr: listenHost, }, + name: name, store: store, registry: registry, url: urlStr, @@ -134,7 +135,7 @@ func (s *Server) handleFunc(path string, f func(http.ResponseWriter, *http.Reque // Wrap the standard HandleFunc interface to pass in the server reference. return r.HandleFunc(path, func(w http.ResponseWriter, req *http.Request) { // Log request. - log.Debugf("[recv] %s %s [%s]", req.Method, s.url, req.URL.Path, req.RemoteAddr) + log.Debugf("[recv] %s %s %s [%s]", req.Method, s.url, req.URL.Path, req.RemoteAddr) // Write CORS header. if s.OriginAllowed("*") { @@ -242,28 +243,28 @@ func (s *Server) GetLeaderHandler(w http.ResponseWriter, req *http.Request) erro // Handler to return all the known machines in the current cluster. func (s *Server) GetMachinesHandler(w http.ResponseWriter, req *http.Request) error { - machines := s.registry.URLs(s.peerServer.Leader(), s.name) - w.WriteHeader(http.StatusOK) - w.Write([]byte(strings.Join(machines, ", "))) - return nil + machines := s.registry.URLs(s.peerServer.Leader(), s.name) + w.WriteHeader(http.StatusOK) + w.Write([]byte(strings.Join(machines, ", "))) + return nil } // Retrieves stats on the Raft server. func (s *Server) GetStatsHandler(w http.ResponseWriter, req *http.Request) error { - w.Write(s.peerServer.Stats()) - return nil + w.Write(s.peerServer.Stats()) + return nil } // Retrieves stats on the leader. func (s *Server) GetLeaderStatsHandler(w http.ResponseWriter, req *http.Request) error { if s.peerServer.State() == raft.Leader { - w.Write(s.peerServer.PeerStats()) - return nil + w.Write(s.peerServer.PeerStats()) + return nil } leader := s.peerServer.Leader() if leader == "" { - return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) + return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) } hostname, _ := s.registry.URL(leader) redirect(hostname, w, req) @@ -272,8 +273,8 @@ func (s *Server) GetLeaderStatsHandler(w http.ResponseWriter, req *http.Request) // Retrieves stats on the leader. func (s *Server) GetStoreStatsHandler(w http.ResponseWriter, req *http.Request) error { - w.Write(s.store.JsonStats()) - return nil + w.Write(s.store.JsonStats()) + return nil } // Executes a speed test to evaluate the performance of update replication. @@ -284,8 +285,8 @@ func (s *Server) SpeedTestHandler(w http.ResponseWriter, req *http.Request) erro go func() { for j := 0; j < 10; j++ { c := &store.UpdateCommand{ - Key: "foo", - Value: "bar", + Key: "foo", + Value: "bar", ExpireTime: time.Unix(0, 0), } s.peerServer.Do(c) diff --git a/server/timeout.go b/server/timeout.go index 321e77b38f7..fa48c316215 100644 --- a/server/timeout.go +++ b/server/timeout.go @@ -1,15 +1,15 @@ package server import ( - "time" + "time" ) const ( - // The amount of time to elapse without a heartbeat before becoming a candidate. - ElectionTimeout = 200 * time.Millisecond + // The amount of time to elapse without a heartbeat before becoming a candidate. + ElectionTimeout = 200 * time.Millisecond - // The frequency by which heartbeats are sent to followers. - HeartbeatTimeout = 50 * time.Millisecond + // The frequency by which heartbeats are sent to followers. + HeartbeatTimeout = 50 * time.Millisecond - RetryInterval = 10 + RetryInterval = 10 ) diff --git a/server/transporter.go b/server/transporter.go index 7397d45342b..03928e2b3ba 100644 --- a/server/transporter.go +++ b/server/transporter.go @@ -1,17 +1,17 @@ package server import ( - "bytes" - "crypto/tls" - "encoding/json" - "fmt" - "io" - "net" - "net/http" - "time" - - "github.com/coreos/etcd/log" - "github.com/coreos/go-raft" + "bytes" + "crypto/tls" + "encoding/json" + "fmt" + "io" + "net" + "net/http" + "time" + + "github.com/coreos/etcd/log" + "github.com/coreos/go-raft" ) // Timeout for setup internal raft http connection @@ -28,200 +28,200 @@ var tranTimeout = ElectionTimeout // Transporter layer for communication between raft nodes type transporter struct { - client *http.Client - transport *http.Transport - peerServer *PeerServer + client *http.Client + transport *http.Transport + peerServer *PeerServer } // Create transporter using by raft server // Create http or https transporter based on // whether the user give the server cert and key func newTransporter(scheme string, tlsConf tls.Config, peerServer *PeerServer) *transporter { - t := transporter{} + t := transporter{} - tr := &http.Transport{ - Dial: dialWithTimeout, - ResponseHeaderTimeout: responseHeaderTimeout, - } + tr := &http.Transport{ + Dial: dialWithTimeout, + ResponseHeaderTimeout: responseHeaderTimeout, + } - if scheme == "https" { - tr.TLSClientConfig = &tlsConf - tr.DisableCompression = true - } + if scheme == "https" { + tr.TLSClientConfig = &tlsConf + tr.DisableCompression = true + } - t.client = &http.Client{Transport: tr} - t.transport = tr - t.peerServer = peerServer + t.client = &http.Client{Transport: tr} + t.transport = tr + t.peerServer = peerServer - return &t + return &t } // Dial with timeout func dialWithTimeout(network, addr string) (net.Conn, error) { - return net.DialTimeout(network, addr, dailTimeout) + return net.DialTimeout(network, addr, dailTimeout) } // Sends AppendEntries RPCs to a peer when the server is the leader. func (t *transporter) SendAppendEntriesRequest(server *raft.Server, peer *raft.Peer, req *raft.AppendEntriesRequest) *raft.AppendEntriesResponse { - var aersp *raft.AppendEntriesResponse - var b bytes.Buffer + var aersp *raft.AppendEntriesResponse + var b bytes.Buffer - json.NewEncoder(&b).Encode(req) + json.NewEncoder(&b).Encode(req) - size := b.Len() + size := b.Len() - t.peerServer.serverStats.SendAppendReq(size) + t.peerServer.serverStats.SendAppendReq(size) - u, _ := t.peerServer.registry.PeerURL(peer.Name) + u, _ := t.peerServer.registry.PeerURL(peer.Name) - log.Debugf("Send LogEntries to %s ", u) + log.Debugf("Send LogEntries to %s ", u) - thisFollowerStats, ok := t.peerServer.followersStats.Followers[peer.Name] + thisFollowerStats, ok := t.peerServer.followersStats.Followers[peer.Name] - if !ok { //this is the first time this follower has been seen - thisFollowerStats = &raftFollowerStats{} - thisFollowerStats.Latency.Minimum = 1 << 63 - t.peerServer.followersStats.Followers[peer.Name] = thisFollowerStats - } + if !ok { //this is the first time this follower has been seen + thisFollowerStats = &raftFollowerStats{} + thisFollowerStats.Latency.Minimum = 1 << 63 + t.peerServer.followersStats.Followers[peer.Name] = thisFollowerStats + } - start := time.Now() + start := time.Now() - resp, httpRequest, err := t.Post(fmt.Sprintf("%s/log/append", u), &b) + resp, httpRequest, err := t.Post(fmt.Sprintf("%s/log/append", u), &b) - end := time.Now() + end := time.Now() - if err != nil { - log.Debugf("Cannot send AppendEntriesRequest to %s: %s", u, err) - if ok { - thisFollowerStats.Fail() - } - } else { - if ok { - thisFollowerStats.Succ(end.Sub(start)) - } - } + if err != nil { + log.Debugf("Cannot send AppendEntriesRequest to %s: %s", u, err) + if ok { + thisFollowerStats.Fail() + } + } else { + if ok { + thisFollowerStats.Succ(end.Sub(start)) + } + } - if resp != nil { - defer resp.Body.Close() + if resp != nil { + defer resp.Body.Close() - t.CancelWhenTimeout(httpRequest) + t.CancelWhenTimeout(httpRequest) - aersp = &raft.AppendEntriesResponse{} - if err := json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { - return aersp - } + aersp = &raft.AppendEntriesResponse{} + if err := json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { + return aersp + } - } + } - return aersp + return aersp } // Sends RequestVote RPCs to a peer when the server is the candidate. func (t *transporter) SendVoteRequest(server *raft.Server, peer *raft.Peer, req *raft.RequestVoteRequest) *raft.RequestVoteResponse { - var rvrsp *raft.RequestVoteResponse - var b bytes.Buffer - json.NewEncoder(&b).Encode(req) + var rvrsp *raft.RequestVoteResponse + var b bytes.Buffer + json.NewEncoder(&b).Encode(req) - u, _ := t.peerServer.registry.PeerURL(peer.Name) - log.Debugf("Send Vote to %s", u) + u, _ := t.peerServer.registry.PeerURL(peer.Name) + log.Debugf("Send Vote to %s", u) - resp, httpRequest, err := t.Post(fmt.Sprintf("%s/vote", u), &b) + resp, httpRequest, err := t.Post(fmt.Sprintf("%s/vote", u), &b) - if err != nil { - log.Debugf("Cannot send VoteRequest to %s : %s", u, err) - } + if err != nil { + log.Debugf("Cannot send VoteRequest to %s : %s", u, err) + } - if resp != nil { - defer resp.Body.Close() + if resp != nil { + defer resp.Body.Close() - t.CancelWhenTimeout(httpRequest) + t.CancelWhenTimeout(httpRequest) - rvrsp := &raft.RequestVoteResponse{} - if err := json.NewDecoder(resp.Body).Decode(&rvrsp); err == nil || err == io.EOF { - return rvrsp - } + rvrsp := &raft.RequestVoteResponse{} + if err := json.NewDecoder(resp.Body).Decode(&rvrsp); err == nil || err == io.EOF { + return rvrsp + } - } - return rvrsp + } + return rvrsp } // Sends SnapshotRequest RPCs to a peer when the server is the candidate. func (t *transporter) SendSnapshotRequest(server *raft.Server, peer *raft.Peer, req *raft.SnapshotRequest) *raft.SnapshotResponse { - var aersp *raft.SnapshotResponse - var b bytes.Buffer - json.NewEncoder(&b).Encode(req) + var aersp *raft.SnapshotResponse + var b bytes.Buffer + json.NewEncoder(&b).Encode(req) - u, _ := t.peerServer.registry.PeerURL(peer.Name) - log.Debugf("Send Snapshot to %s [Last Term: %d, LastIndex %d]", u, - req.LastTerm, req.LastIndex) + u, _ := t.peerServer.registry.PeerURL(peer.Name) + log.Debugf("Send Snapshot to %s [Last Term: %d, LastIndex %d]", u, + req.LastTerm, req.LastIndex) - resp, httpRequest, err := t.Post(fmt.Sprintf("%s/snapshot", u), &b) + resp, httpRequest, err := t.Post(fmt.Sprintf("%s/snapshot", u), &b) - if err != nil { - log.Debugf("Cannot send SendSnapshotRequest to %s : %s", u, err) - } + if err != nil { + log.Debugf("Cannot send SendSnapshotRequest to %s : %s", u, err) + } - if resp != nil { - defer resp.Body.Close() + if resp != nil { + defer resp.Body.Close() - t.CancelWhenTimeout(httpRequest) + t.CancelWhenTimeout(httpRequest) - aersp = &raft.SnapshotResponse{} - if err = json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { + aersp = &raft.SnapshotResponse{} + if err = json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { - return aersp - } - } + return aersp + } + } - return aersp + return aersp } // Sends SnapshotRecoveryRequest RPCs to a peer when the server is the candidate. func (t *transporter) SendSnapshotRecoveryRequest(server *raft.Server, peer *raft.Peer, req *raft.SnapshotRecoveryRequest) *raft.SnapshotRecoveryResponse { - var aersp *raft.SnapshotRecoveryResponse - var b bytes.Buffer - json.NewEncoder(&b).Encode(req) + var aersp *raft.SnapshotRecoveryResponse + var b bytes.Buffer + json.NewEncoder(&b).Encode(req) - u, _ := t.peerServer.registry.PeerURL(peer.Name) - log.Debugf("Send SnapshotRecovery to %s [Last Term: %d, LastIndex %d]", u, - req.LastTerm, req.LastIndex) + u, _ := t.peerServer.registry.PeerURL(peer.Name) + log.Debugf("Send SnapshotRecovery to %s [Last Term: %d, LastIndex %d]", u, + req.LastTerm, req.LastIndex) - resp, _, err := t.Post(fmt.Sprintf("%s/snapshotRecovery", u), &b) + resp, _, err := t.Post(fmt.Sprintf("%s/snapshotRecovery", u), &b) - if err != nil { - log.Debugf("Cannot send SendSnapshotRecoveryRequest to %s : %s", u, err) - } + if err != nil { + log.Debugf("Cannot send SendSnapshotRecoveryRequest to %s : %s", u, err) + } - if resp != nil { - defer resp.Body.Close() - aersp = &raft.SnapshotRecoveryResponse{} + if resp != nil { + defer resp.Body.Close() + aersp = &raft.SnapshotRecoveryResponse{} - if err = json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { - return aersp - } - } + if err = json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { + return aersp + } + } - return aersp + return aersp } // Send server side POST request func (t *transporter) Post(urlStr string, body io.Reader) (*http.Response, *http.Request, error) { - req, _ := http.NewRequest("POST", urlStr, body) - resp, err := t.client.Do(req) - return resp, req, err + req, _ := http.NewRequest("POST", urlStr, body) + resp, err := t.client.Do(req) + return resp, req, err } // Send server side GET request func (t *transporter) Get(urlStr string) (*http.Response, *http.Request, error) { - req, _ := http.NewRequest("GET", urlStr, nil) - resp, err := t.client.Do(req) - return resp, req, err + req, _ := http.NewRequest("GET", urlStr, nil) + resp, err := t.client.Do(req) + return resp, req, err } // Cancel the on fly HTTP transaction when timeout happens. func (t *transporter) CancelWhenTimeout(req *http.Request) { - go func() { - time.Sleep(ElectionTimeout) - t.transport.CancelRequest(req) - }() + go func() { + time.Sleep(ElectionTimeout) + t.transport.CancelRequest(req) + }() } diff --git a/server/util.go b/server/util.go index 95d93c17950..6a53884baa2 100644 --- a/server/util.go +++ b/server/util.go @@ -1,27 +1,26 @@ package server import ( - "encoding/json" - "fmt" - "io" - "net/http" + "encoding/json" + "fmt" + "io" + "net/http" - "github.com/coreos/etcd/log" + "github.com/coreos/etcd/log" ) func decodeJsonRequest(req *http.Request, data interface{}) error { - decoder := json.NewDecoder(req.Body) - if err := decoder.Decode(&data); err != nil && err != io.EOF { - log.Warnf("Malformed json request: %v", err) - return fmt.Errorf("Malformed json request: %v", err) - } - return nil + decoder := json.NewDecoder(req.Body) + if err := decoder.Decode(&data); err != nil && err != io.EOF { + log.Warnf("Malformed json request: %v", err) + return fmt.Errorf("Malformed json request: %v", err) + } + return nil } func redirect(hostname string, w http.ResponseWriter, req *http.Request) { - path := req.URL.Path - url := hostname + path - log.Debugf("Redirect to %s", url) - http.Redirect(w, req, url, http.StatusTemporaryRedirect) + path := req.URL.Path + url := hostname + path + log.Debugf("Redirect to %s", url) + http.Redirect(w, req, url, http.StatusTemporaryRedirect) } - diff --git a/server/v1/delete_key_handler.go b/server/v1/delete_key_handler.go index a657e9f3d77..1288d6597e1 100644 --- a/server/v1/delete_key_handler.go +++ b/server/v1/delete_key_handler.go @@ -1,9 +1,9 @@ package v1 import ( + "github.com/coreos/etcd/store" + "github.com/gorilla/mux" "net/http" - "github.com/coreos/etcd/store" - "github.com/gorilla/mux" ) // Removes a key from the store. diff --git a/server/v1/v1.go b/server/v1/v1.go index 586a08e67e9..f71ed06220e 100644 --- a/server/v1/v1.go +++ b/server/v1/v1.go @@ -1,9 +1,9 @@ package v1 import ( - "net/http" "github.com/coreos/etcd/store" "github.com/coreos/go-raft" + "net/http" ) // The Server interface provides all the methods required for the v1 API. diff --git a/server/v2/create_key_handler.go b/server/v2/create_key_handler.go index 8a9fbf50e85..fab9bde1f9c 100644 --- a/server/v2/create_key_handler.go +++ b/server/v2/create_key_handler.go @@ -1,29 +1,29 @@ package v2 import ( - "net/http" + "net/http" - etcdErr "github.com/coreos/etcd/error" - "github.com/coreos/etcd/store" - "github.com/gorilla/mux" + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/store" + "github.com/gorilla/mux" ) func CreateKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { - vars := mux.Vars(req) - key := "/" + vars["key"] + vars := mux.Vars(req) + key := "/" + vars["key"] - value := req.FormValue("value") - expireTime, err := store.TTL(req.FormValue("ttl")) - if err != nil { - return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create", store.UndefIndex, store.UndefTerm) - } + value := req.FormValue("value") + expireTime, err := store.TTL(req.FormValue("ttl")) + if err != nil { + return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create", store.UndefIndex, store.UndefTerm) + } - c := &store.CreateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - IncrementalSuffix: (req.FormValue("incremental") == "true"), - } + c := &store.CreateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + IncrementalSuffix: (req.FormValue("incremental") == "true"), + } - return s.Dispatch(c, w, req) + return s.Dispatch(c, w, req) } diff --git a/server/v2/delete_key_handler.go b/server/v2/delete_key_handler.go index e3bdf2b2d27..c53e7245983 100644 --- a/server/v2/delete_key_handler.go +++ b/server/v2/delete_key_handler.go @@ -1,20 +1,20 @@ package v2 import ( - "net/http" + "net/http" - "github.com/coreos/etcd/store" - "github.com/gorilla/mux" + "github.com/coreos/etcd/store" + "github.com/gorilla/mux" ) func DeleteKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { - vars := mux.Vars(req) - key := "/" + vars["key"] + vars := mux.Vars(req) + key := "/" + vars["key"] - c := &store.DeleteCommand{ - Key: key, - Recursive: (req.FormValue("recursive") == "true"), - } + c := &store.DeleteCommand{ + Key: key, + Recursive: (req.FormValue("recursive") == "true"), + } - return s.Dispatch(c, w, req) + return s.Dispatch(c, w, req) } diff --git a/server/v2/get_key_handler.go b/server/v2/get_key_handler.go index e4d9b7207e3..e6cce6e8bd2 100644 --- a/server/v2/get_key_handler.go +++ b/server/v2/get_key_handler.go @@ -1,69 +1,69 @@ package v2 import ( - "encoding/json" - "fmt" - "net/http" - "strconv" + "encoding/json" + "fmt" + "net/http" + "strconv" - etcdErr "github.com/coreos/etcd/error" - "github.com/coreos/etcd/log" - "github.com/coreos/etcd/store" - "github.com/coreos/go-raft" - "github.com/gorilla/mux" + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" + "github.com/gorilla/mux" ) func GetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { - var err error - var event *store.Event + var err error + var event *store.Event - vars := mux.Vars(req) - key := "/" + vars["key"] + vars := mux.Vars(req) + key := "/" + vars["key"] - // Help client to redirect the request to the current leader - if req.FormValue("consistent") == "true" && s.State() != raft.Leader { - leader := s.Leader() - hostname, _ := s.PeerURL(leader) - url := hostname + req.URL.Path - log.Debugf("Redirect to %s", url) - http.Redirect(w, req, url, http.StatusTemporaryRedirect) - return nil - } + // Help client to redirect the request to the current leader + if req.FormValue("consistent") == "true" && s.State() != raft.Leader { + leader := s.Leader() + hostname, _ := s.PeerURL(leader) + url := hostname + req.URL.Path + log.Debugf("Redirect to %s", url) + http.Redirect(w, req, url, http.StatusTemporaryRedirect) + return nil + } - recursive := (req.FormValue("recursive") == "true") - sorted := (req.FormValue("sorted") == "true") + recursive := (req.FormValue("recursive") == "true") + sorted := (req.FormValue("sorted") == "true") - if req.FormValue("wait") == "true" { // watch - // Create a command to watch from a given index (default 0). - var sinceIndex uint64 = 0 - if req.Method == "POST" { - sinceIndex, err = strconv.ParseUint(string(req.FormValue("wait_index")), 10, 64) - if err != nil { - return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index", store.UndefIndex, store.UndefTerm) - } - } + if req.FormValue("wait") == "true" { // watch + // Create a command to watch from a given index (default 0). + var sinceIndex uint64 = 0 + if req.Method == "POST" { + sinceIndex, err = strconv.ParseUint(string(req.FormValue("wait_index")), 10, 64) + if err != nil { + return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index", store.UndefIndex, store.UndefTerm) + } + } - // Start the watcher on the store. - c, err := s.Store().Watch(key, recursive, sinceIndex, s.CommitIndex(), s.Term()) - if err != nil { - return etcdErr.NewError(500, key, store.UndefIndex, store.UndefTerm) - } - event = <-c + // Start the watcher on the store. + c, err := s.Store().Watch(key, recursive, sinceIndex, s.CommitIndex(), s.Term()) + if err != nil { + return etcdErr.NewError(500, key, store.UndefIndex, store.UndefTerm) + } + event = <-c - } else { //get - // Retrieve the key from the store. - event, err = s.Store().Get(key, recursive, sorted, s.CommitIndex(), s.Term()) - if err != nil { - return err - } - } + } else { //get + // Retrieve the key from the store. + event, err = s.Store().Get(key, recursive, sorted, s.CommitIndex(), s.Term()) + if err != nil { + return err + } + } - w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) - w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term)) - w.WriteHeader(http.StatusOK) + w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) + w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term)) + w.WriteHeader(http.StatusOK) - b, _ := json.Marshal(event) - w.Write(b) + b, _ := json.Marshal(event) + w.Write(b) - return nil + return nil } diff --git a/server/v2/update_key_handler.go b/server/v2/update_key_handler.go index 64e60cca5f1..841c8828de4 100644 --- a/server/v2/update_key_handler.go +++ b/server/v2/update_key_handler.go @@ -1,64 +1,64 @@ package v2 import ( - "net/http" - "strconv" + "net/http" + "strconv" - etcdErr "github.com/coreos/etcd/error" - "github.com/coreos/etcd/store" - "github.com/coreos/go-raft" - "github.com/gorilla/mux" + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" + "github.com/gorilla/mux" ) func UpdateKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { - vars := mux.Vars(req) - key := "/" + vars["key"] + vars := mux.Vars(req) + key := "/" + vars["key"] - req.ParseForm() + req.ParseForm() - value := req.Form.Get("value") - expireTime, err := store.TTL(req.Form.Get("ttl")) - if err != nil { - return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", store.UndefIndex, store.UndefTerm) - } + value := req.Form.Get("value") + expireTime, err := store.TTL(req.Form.Get("ttl")) + if err != nil { + return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", store.UndefIndex, store.UndefTerm) + } - // Update should give at least one option - if value == "" && expireTime.Sub(store.Permanent) == 0 { - return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", store.UndefIndex, store.UndefTerm) - } + // Update should give at least one option + if value == "" && expireTime.Sub(store.Permanent) == 0 { + return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", store.UndefIndex, store.UndefTerm) + } - prevValue, valueOk := req.Form["prevValue"] - prevIndexStr, indexOk := req.Form["prevIndex"] + prevValue, valueOk := req.Form["prevValue"] + prevIndexStr, indexOk := req.Form["prevIndex"] - var c raft.Command - if !valueOk && !indexOk { // update without test - c = &store.UpdateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - } + var c raft.Command + if !valueOk && !indexOk { // update without test + c = &store.UpdateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + } - } else { // update with test - var prevIndex uint64 + } else { // update with test + var prevIndex uint64 - if indexOk { - prevIndex, err = strconv.ParseUint(prevIndexStr[0], 10, 64) + if indexOk { + prevIndex, err = strconv.ParseUint(prevIndexStr[0], 10, 64) - // bad previous index - if err != nil { - return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Update", store.UndefIndex, store.UndefTerm) - } - } else { - prevIndex = 0 - } + // bad previous index + if err != nil { + return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Update", store.UndefIndex, store.UndefTerm) + } + } else { + prevIndex = 0 + } - c = &store.TestAndSetCommand{ - Key: key, - Value: value, - PrevValue: prevValue[0], - PrevIndex: prevIndex, - } - } + c = &store.TestAndSetCommand{ + Key: key, + Value: value, + PrevValue: prevValue[0], + PrevIndex: prevIndex, + } + } - return s.Dispatch(c, w, req) + return s.Dispatch(c, w, req) } diff --git a/server/v2/v2.go b/server/v2/v2.go index 439f6078e42..e412e859b23 100644 --- a/server/v2/v2.go +++ b/server/v2/v2.go @@ -1,18 +1,18 @@ package v2 import ( - "net/http" - "github.com/coreos/etcd/store" - "github.com/coreos/go-raft" + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" + "net/http" ) // The Server interface provides all the methods required for the v2 API. type Server interface { - State() string - Leader() string - CommitIndex() uint64 - Term() uint64 - PeerURL(string) (string, bool) - Store() *store.Store - Dispatch(raft.Command, http.ResponseWriter, *http.Request) error + State() string + Leader() string + CommitIndex() uint64 + Term() uint64 + PeerURL(string) (string, bool) + Store() *store.Store + Dispatch(raft.Command, http.ResponseWriter, *http.Request) error } diff --git a/store/create_command.go b/store/create_command.go index 0263347a655..2ccddd1035c 100644 --- a/store/create_command.go +++ b/store/create_command.go @@ -7,7 +7,7 @@ import ( ) func init() { - raft.RegisterCommand(&CreateCommand{}) + raft.RegisterCommand(&CreateCommand{}) } // Create command diff --git a/store/delete_command.go b/store/delete_command.go index 3ac48bc3570..324410192eb 100644 --- a/store/delete_command.go +++ b/store/delete_command.go @@ -6,7 +6,7 @@ import ( ) func init() { - raft.RegisterCommand(&DeleteCommand{}) + raft.RegisterCommand(&DeleteCommand{}) } // The DeleteCommand removes a key from the Store. diff --git a/store/event.go b/store/event.go index 48420186560..0d9ec0a37de 100644 --- a/store/event.go +++ b/store/event.go @@ -77,5 +77,3 @@ func (event *Event) Response() interface{} { return responses } } - - diff --git a/store/event_history.go b/store/event_history.go index 73db5d87671..3ddd3820690 100644 --- a/store/event_history.go +++ b/store/event_history.go @@ -1,112 +1,112 @@ package store import ( - "fmt" - "strings" - "sync" + "fmt" + "strings" + "sync" - etcdErr "github.com/coreos/etcd/error" + etcdErr "github.com/coreos/etcd/error" ) type EventHistory struct { - Queue eventQueue - StartIndex uint64 - LastIndex uint64 - LastTerm uint64 - DupCnt uint64 // help to compute the watching point with duplicated indexes in the queue - rwl sync.RWMutex + Queue eventQueue + StartIndex uint64 + LastIndex uint64 + LastTerm uint64 + DupCnt uint64 // help to compute the watching point with duplicated indexes in the queue + rwl sync.RWMutex } func newEventHistory(capacity int) *EventHistory { - return &EventHistory{ - Queue: eventQueue{ - Capacity: capacity, - Events: make([]*Event, capacity), - }, - } + return &EventHistory{ + Queue: eventQueue{ + Capacity: capacity, + Events: make([]*Event, capacity), + }, + } } // addEvent function adds event into the eventHistory func (eh *EventHistory) addEvent(e *Event) *Event { - eh.rwl.Lock() - defer eh.rwl.Unlock() + eh.rwl.Lock() + defer eh.rwl.Unlock() - var duped uint64 + var duped uint64 - if e.Index == UndefIndex { - e.Index = eh.LastIndex - e.Term = eh.LastTerm - duped = 1 - } + if e.Index == UndefIndex { + e.Index = eh.LastIndex + e.Term = eh.LastTerm + duped = 1 + } - eh.Queue.insert(e) + eh.Queue.insert(e) - eh.LastIndex = e.Index - eh.LastTerm = e.Term - eh.DupCnt += duped + eh.LastIndex = e.Index + eh.LastTerm = e.Term + eh.DupCnt += duped - eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index + eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index - return e + return e } // scan function is enumerating events from the index in history and // stops till the first point where the key has identified prefix func (eh *EventHistory) scan(prefix string, index uint64) (*Event, *etcdErr.Error) { - eh.rwl.RLock() - defer eh.rwl.RUnlock() - - start := index - eh.StartIndex - - // the index should locate after the event history's StartIndex - if start < 0 { - return nil, - etcdErr.NewError(etcdErr.EcodeEventIndexCleared, - fmt.Sprintf("the requested history has been cleared [%v/%v]", - eh.StartIndex, index), UndefIndex, UndefTerm) - } - - // the index should locate before the size of the queue minus the duplicate count - if start >= (uint64(eh.Queue.Size) - eh.DupCnt) { // future index - return nil, nil - } - - i := int((start + uint64(eh.Queue.Front)) % uint64(eh.Queue.Capacity)) - - for { - e := eh.Queue.Events[i] - if strings.HasPrefix(e.Key, prefix) && index <= e.Index { // make sure we bypass the smaller one - return e, nil - } - - i = (i + 1) % eh.Queue.Capacity - - if i == eh.Queue.back() { // find nothing, return and watch from current index - return nil, nil - } - } + eh.rwl.RLock() + defer eh.rwl.RUnlock() + + start := index - eh.StartIndex + + // the index should locate after the event history's StartIndex + if start < 0 { + return nil, + etcdErr.NewError(etcdErr.EcodeEventIndexCleared, + fmt.Sprintf("the requested history has been cleared [%v/%v]", + eh.StartIndex, index), UndefIndex, UndefTerm) + } + + // the index should locate before the size of the queue minus the duplicate count + if start >= (uint64(eh.Queue.Size) - eh.DupCnt) { // future index + return nil, nil + } + + i := int((start + uint64(eh.Queue.Front)) % uint64(eh.Queue.Capacity)) + + for { + e := eh.Queue.Events[i] + if strings.HasPrefix(e.Key, prefix) && index <= e.Index { // make sure we bypass the smaller one + return e, nil + } + + i = (i + 1) % eh.Queue.Capacity + + if i == eh.Queue.back() { // find nothing, return and watch from current index + return nil, nil + } + } } // clone will be protected by a stop-world lock // do not need to obtain internal lock func (eh *EventHistory) clone() *EventHistory { - clonedQueue := eventQueue{ - Capacity: eh.Queue.Capacity, - Events: make([]*Event, eh.Queue.Capacity), - Size: eh.Queue.Size, - Front: eh.Queue.Front, - } - - for i, e := range eh.Queue.Events { - clonedQueue.Events[i] = e - } - - return &EventHistory{ - StartIndex: eh.StartIndex, - Queue: clonedQueue, - LastIndex: eh.LastIndex, - LastTerm: eh.LastTerm, - DupCnt: eh.DupCnt, - } + clonedQueue := eventQueue{ + Capacity: eh.Queue.Capacity, + Events: make([]*Event, eh.Queue.Capacity), + Size: eh.Queue.Size, + Front: eh.Queue.Front, + } + + for i, e := range eh.Queue.Events { + clonedQueue.Events[i] = e + } + + return &EventHistory{ + StartIndex: eh.StartIndex, + Queue: clonedQueue, + LastIndex: eh.LastIndex, + LastTerm: eh.LastTerm, + DupCnt: eh.DupCnt, + } } diff --git a/store/event_queue.go b/store/event_queue.go index 7c520ffe4dc..0852956b1b8 100644 --- a/store/event_queue.go +++ b/store/event_queue.go @@ -1,26 +1,25 @@ package store - type eventQueue struct { - Events []*Event - Size int - Front int - Capacity int + Events []*Event + Size int + Front int + Capacity int } func (eq *eventQueue) back() int { - return (eq.Front + eq.Size - 1 + eq.Capacity) % eq.Capacity + return (eq.Front + eq.Size - 1 + eq.Capacity) % eq.Capacity } func (eq *eventQueue) insert(e *Event) { - index := (eq.back() + 1) % eq.Capacity + index := (eq.back() + 1) % eq.Capacity - eq.Events[index] = e + eq.Events[index] = e - if eq.Size == eq.Capacity { //dequeue - eq.Front = (index + 1) % eq.Capacity - } else { - eq.Size++ - } + if eq.Size == eq.Capacity { //dequeue + eq.Front = (index + 1) % eq.Capacity + } else { + eq.Size++ + } } diff --git a/store/test_and_set_command.go b/store/test_and_set_command.go index cf4167d1cdd..3370fed154e 100644 --- a/store/test_and_set_command.go +++ b/store/test_and_set_command.go @@ -8,7 +8,7 @@ import ( ) func init() { - raft.RegisterCommand(&TestAndSetCommand{}) + raft.RegisterCommand(&TestAndSetCommand{}) } // The TestAndSetCommand performs a conditional update on a key in the store. diff --git a/store/ttl.go b/store/ttl.go index c73d95f8c4e..fec0ec3bfb1 100644 --- a/store/ttl.go +++ b/store/ttl.go @@ -1,21 +1,20 @@ package store import ( - "strconv" - "time" + "strconv" + "time" ) // Convert string duration to time format func TTL(duration string) (time.Time, error) { - if duration != "" { - duration, err := strconv.Atoi(duration) - if err != nil { - return Permanent, err - } - return time.Now().Add(time.Second * (time.Duration)(duration)), nil + if duration != "" { + duration, err := strconv.Atoi(duration) + if err != nil { + return Permanent, err + } + return time.Now().Add(time.Second * (time.Duration)(duration)), nil - } else { - return Permanent, nil - } + } else { + return Permanent, nil + } } - diff --git a/store/update_command.go b/store/update_command.go index 694be9844f3..9ffd6c8686d 100644 --- a/store/update_command.go +++ b/store/update_command.go @@ -8,7 +8,7 @@ import ( ) func init() { - raft.RegisterCommand(&UpdateCommand{}) + raft.RegisterCommand(&UpdateCommand{}) } // The UpdateCommand updates the value of a key in the Store. diff --git a/util.go b/util.go index d519fc1aaca..089eba40cc2 100644 --- a/util.go +++ b/util.go @@ -14,7 +14,6 @@ import ( // HTTP Utilities //-------------------------------------- - // sanitizeURL will cleanup a host string in the format hostname:port and // attach a schema. func sanitizeURL(host string, defaultScheme string) string { @@ -87,4 +86,3 @@ func runCPUProfile() { } }() } - From ec24e769594b50c322803a26484ba96c08d2d2bc Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sun, 13 Oct 2013 14:56:17 -0600 Subject: [PATCH 084/247] Fix duplicates in registry. --- server/registry.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/registry.go b/server/registry.go index 8a62dff4081..8ee2406ed70 100644 --- a/server/registry.go +++ b/server/registry.go @@ -103,7 +103,7 @@ func (r *Registry) URLs(leaderName, selfName string) []string { if e, _ := r.store.Get(RegistryKey, false, false, 0, 0); e != nil { // Lookup the URL for each one. for _, pair := range e.KVPairs { - if url, _ := r.url(pair.Key); len(url) > 0 { + if url, _ := r.url(pair.Key); len(url) > 0 && pair.Key != leaderName && pair.Key != selfName { urls = append(urls, url) } } @@ -151,7 +151,7 @@ func (r *Registry) PeerURLs(leaderName, selfName string) []string { if e, _ := r.store.Get(RegistryKey, false, false, 0, 0); e != nil { // Lookup the URL for each one. for _, pair := range e.KVPairs { - if url, _ := r.peerURL(pair.Key); len(url) > 0 { + if url, _ := r.peerURL(pair.Key); len(url) > 0 && pair.Key != leaderName && pair.Key != selfName { urls = append(urls, url) } } From e954d3d41fae5a18fbc18dcd4deeaca2b2ac94e1 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 13 Oct 2013 15:25:07 -0700 Subject: [PATCH 085/247] fix registry.go: use the correct node name; self is already in the list --- server/registry.go | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/server/registry.go b/server/registry.go index 7335b88ea10..9f30d7854b6 100644 --- a/server/registry.go +++ b/server/registry.go @@ -4,6 +4,7 @@ import ( "fmt" "net/url" "path" + "path/filepath" "strings" "sync" @@ -95,21 +96,19 @@ func (r *Registry) URLs(leaderName, selfName string) []string { if url, _ := r.url(leaderName); len(url) > 0 { urls = append(urls, url) } - if url, _ := r.url(selfName); len(url) > 0 { - urls = append(urls, url) - } // Retrieve a list of all nodes. if e, _ := r.store.Get(RegistryKey, false, false, 0, 0); e != nil { // Lookup the URL for each one. for _, pair := range e.KVPairs { - if url, _ := r.url(pair.Key); len(url) > 0 && pair.Key != leaderName && pair.Key != selfName { + _, name := filepath.Split(pair.Key) + if url, _ := r.url(name); len(url) > 0 && name != leaderName { urls = append(urls, url) } } } - log.Infof("URLs: %s / %s (%s", leaderName, selfName, strings.Join(urls, ",")) + log.Infof("URLs: %s / %s (%s)", leaderName, selfName, strings.Join(urls, ",")) return urls } @@ -143,21 +142,19 @@ func (r *Registry) PeerURLs(leaderName, selfName string) []string { if url, _ := r.peerURL(leaderName); len(url) > 0 { urls = append(urls, url) } - if url, _ := r.peerURL(selfName); len(url) > 0 { - urls = append(urls, url) - } // Retrieve a list of all nodes. if e, _ := r.store.Get(RegistryKey, false, false, 0, 0); e != nil { // Lookup the URL for each one. for _, pair := range e.KVPairs { - if url, _ := r.peerURL(pair.Key); len(url) > 0 && pair.Key != leaderName && pair.Key != selfName { + _, name := filepath.Split(pair.Key) + if url, _ := r.peerURL(name); len(url) > 0 && name != leaderName { urls = append(urls, url) } } } - log.Infof("PeerURLs: %s / %s (%s", leaderName, selfName, strings.Join(urls, ",")) + log.Infof("PeerURLs: %s / %s (%s)", leaderName, selfName, strings.Join(urls, ",")) return urls } From 013d07bc2a8b662bd48925f3a14f549956c7b015 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sun, 13 Oct 2013 16:58:36 -0600 Subject: [PATCH 086/247] Fix server dispatch redirection. --- etcd_test.go | 9 +++------ server/peer_server.go | 6 ++---- server/registry.go | 10 ++++++++-- server/server.go | 8 +++++++- test/test.go | 4 ++-- 5 files changed, 22 insertions(+), 15 deletions(-) diff --git a/etcd_test.go b/etcd_test.go index 95e4c3c525e..4ba0f5fa9ce 100644 --- a/etcd_test.go +++ b/etcd_test.go @@ -21,7 +21,7 @@ import ( func TestSingleNode(t *testing.T) { procAttr := new(os.ProcAttr) procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} - args := []string{"etcd", "-vv", "-n=node1", "-f", "-d=/tmp/node1"} + args := []string{"etcd", "-n=node1", "-f", "-d=/tmp/node1"} process, err := os.StartProcess("etcd", args, procAttr) if err != nil { @@ -249,6 +249,7 @@ func TestMultiNodeKillAllAndRecovery(t *testing.T) { clusterSize := 5 argGroup, etcds, err := test.CreateCluster(clusterSize, procAttr, false) + defer test.DestroyCluster(etcds) if err != nil { t.Fatal("cannot create cluster") @@ -300,9 +301,6 @@ func TestMultiNodeKillAllAndRecovery(t *testing.T) { if result.Index != 18 { t.Fatalf("recovery failed! [%d/18]", result.Index) } - - // kill all - test.DestroyCluster(etcds) } // Create a five nodes @@ -479,6 +477,7 @@ func TestRemoveNode(t *testing.T) { clusterSize := 3 argGroup, etcds, _ := test.CreateCluster(clusterSize, procAttr, false) + defer test.DestroyCluster(etcds) time.Sleep(time.Second) @@ -572,8 +571,6 @@ func TestRemoveNode(t *testing.T) { } } } - test.DestroyCluster(etcds) - } func templateBenchmarkEtcdDirectCall(b *testing.B, tls bool) { diff --git a/server/peer_server.go b/server/peer_server.go index 0b16f98e8ef..62ec06c419e 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -410,7 +410,7 @@ func (s *PeerServer) joinByMachine(server *raft.Server, machine string, scheme s versionURL := url.URL{Host: machine, Scheme: scheme, Path: "/version"} version, err := getVersion(t, versionURL) if err != nil { - return fmt.Errorf("Unable to join: %v", err) + return fmt.Errorf("Error during join version check: %v", err) } // TODO: versioning of the internal protocol. See: @@ -442,12 +442,9 @@ func (s *PeerServer) joinByMachine(server *raft.Server, machine string, scheme s return nil } if resp.StatusCode == http.StatusTemporaryRedirect { - address := resp.Header.Get("Location") log.Debugf("Send Join Request to %s", address) - json.NewEncoder(&b).Encode(NewJoinCommand(PeerVersion, server.Name(), s.url, s.server.URL())) - resp, req, err = t.Post(address, &b) } else if resp.StatusCode == http.StatusBadRequest { @@ -538,6 +535,7 @@ func (s *PeerServer) dispatch(c raft.Command, w http.ResponseWriter, req *http.R } url, _ := s.registry.PeerURL(leader) + log.Debugf("Not leader; Current leader: %s; redirect: %s", leader, url) redirect(url, w, req) return nil diff --git a/server/registry.go b/server/registry.go index 9f30d7854b6..d2fd67ec49d 100644 --- a/server/registry.go +++ b/server/registry.go @@ -38,14 +38,16 @@ func NewRegistry(s *store.Store) *Registry { } // Adds a node to the registry. -func (r *Registry) Register(name string, peerVersion string, peerURL string, url string, commitIndex uint64, term uint64) { +func (r *Registry) Register(name string, peerVersion string, peerURL string, url string, commitIndex uint64, term uint64) error { r.Lock() defer r.Unlock() // Write data to store. key := path.Join(RegistryKey, name) value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", peerURL, url, peerVersion) - r.store.Create(key, value, false, false, store.Permanent, commitIndex, term) + _, err := r.store.Create(key, value, false, false, store.Permanent, commitIndex, term) + log.Debugf("Register: %s (%v)", name, err) + return err } // Removes a node from the registry. @@ -53,8 +55,12 @@ func (r *Registry) Unregister(name string, commitIndex uint64, term uint64) erro r.Lock() defer r.Unlock() + // Remove from cache. + delete(r.nodes, name) + // Remove the key from the store. _, err := r.store.Delete(path.Join(RegistryKey, name), false, commitIndex, term) + log.Debugf("Unregister: %s (%v)", name, err) return err } diff --git a/server/server.go b/server/server.go index 76a62fdce70..8cf837420ee 100644 --- a/server/server.go +++ b/server/server.go @@ -193,7 +193,13 @@ func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Reque return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) } - url, _ := s.registry.PeerURL(leader) + var url string + switch c.(type) { + case *JoinCommand, *RemoveCommand: + url, _ = s.registry.PeerURL(leader) + default: + url, _ = s.registry.URL(leader) + } redirect(url, w, req) return nil diff --git a/test/test.go b/test/test.go index acb212d485a..f4e5fad2e82 100644 --- a/test/test.go +++ b/test/test.go @@ -69,13 +69,13 @@ func CreateCluster(size int, procAttr *os.ProcAttr, ssl bool) ([][]string, []*os for i := 0; i < size; i++ { if i == 0 { - argGroup[i] = []string{"etcd", "-d=/tmp/node1", "-n=node1"} + argGroup[i] = []string{"etcd", "-v", "-d=/tmp/node1", "-n=node1"} if ssl { argGroup[i] = append(argGroup[i], sslServer1...) } } else { strI := strconv.Itoa(i + 1) - argGroup[i] = []string{"etcd", "-n=node" + strI, "-c=127.0.0.1:400" + strI, "-s=127.0.0.1:700" + strI, "-d=/tmp/node" + strI, "-C=127.0.0.1:7001"} + argGroup[i] = []string{"etcd", "-v", "-n=node" + strI, "-c=127.0.0.1:400" + strI, "-s=127.0.0.1:700" + strI, "-d=/tmp/node" + strI, "-C=127.0.0.1:7001"} if ssl { argGroup[i] = append(argGroup[i], sslServer2...) } From d44fd6661a2532314ceb20e42639ee0c62493e7a Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sun, 13 Oct 2013 21:09:56 -0600 Subject: [PATCH 087/247] Fix registry cache issues. --- etcd_test.go | 6 +++--- server/join_command.go | 3 +++ server/registry.go | 7 ++++++- server/remove_command.go | 4 +++- server/server.go | 13 +++++++++++-- server/transporter.go | 2 +- 6 files changed, 27 insertions(+), 8 deletions(-) diff --git a/etcd_test.go b/etcd_test.go index 4ba0f5fa9ce..8aaa2735d98 100644 --- a/etcd_test.go +++ b/etcd_test.go @@ -106,7 +106,7 @@ func TestInternalVersionFail(t *testing.T) { procAttr := new(os.ProcAttr) procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} - args := []string{"etcd", "-n=node1", "-f", "-d=/tmp/node1", "-vv", "-C=" + fakeURL.Host} + args := []string{"etcd", "-n=node1", "-f", "-d=/tmp/node1", "-C=" + fakeURL.Host} process, err := os.StartProcess("etcd", args, procAttr) if err != nil { @@ -525,7 +525,7 @@ func TestRemoveNode(t *testing.T) { } if len(resp) != 3 { - t.Fatal("add machine fails") + t.Fatalf("add machine fails #1 (%d != 3)", len(resp)) } } @@ -567,7 +567,7 @@ func TestRemoveNode(t *testing.T) { } if len(resp) != 3 { - t.Fatal("add machine fails") + t.Fatalf("add machine fails #2 (%d != 3)", len(resp)) } } } diff --git a/server/join_command.go b/server/join_command.go index d1a0dcc4250..d7a9aa599f4 100644 --- a/server/join_command.go +++ b/server/join_command.go @@ -41,6 +41,9 @@ func (c *JoinCommand) Apply(server *raft.Server) (interface{}, error) { b := make([]byte, 8) binary.PutUvarint(b, server.CommitIndex()) + // Make sure we're not getting a cached value from the registry. + ps.registry.Invalidate(c.Name) + // Check if the join command is from a previous machine, who lost all its previous log. if _, ok := ps.registry.URL(c.Name); ok { return b, nil diff --git a/server/registry.go b/server/registry.go index d2fd67ec49d..20f0f289a3b 100644 --- a/server/registry.go +++ b/server/registry.go @@ -56,7 +56,7 @@ func (r *Registry) Unregister(name string, commitIndex uint64, term uint64) erro defer r.Unlock() // Remove from cache. - delete(r.nodes, name) + // delete(r.nodes, name) // Remove the key from the store. _, err := r.store.Delete(path.Join(RegistryKey, name), false, commitIndex, term) @@ -165,6 +165,11 @@ func (r *Registry) PeerURLs(leaderName, selfName string) []string { return urls } +// Removes a node from the cache. +func (r *Registry) Invalidate(name string) { + delete(r.nodes, name) +} + // Loads the given node by name from the store into the cache. func (r *Registry) load(name string) { if name == "" { diff --git a/server/remove_command.go b/server/remove_command.go index 51140922b14..852774b0801 100644 --- a/server/remove_command.go +++ b/server/remove_command.go @@ -33,12 +33,14 @@ func (c *RemoveCommand) Apply(server *raft.Server) (interface{}, error) { delete(ps.followersStats.Followers, c.Name) if err != nil { + log.Debugf("Error while unregistering: %s (%v)", c.Name, err) return []byte{0}, err } // Remove peer in raft err = server.RemovePeer(c.Name) if err != nil { + log.Debugf("Unable to remove peer: %s (%v)", c.Name, err) return []byte{0}, err } @@ -57,7 +59,7 @@ func (c *RemoveCommand) Apply(server *raft.Server) (interface{}, error) { log.Debugf("ignore previous remove command.") } } - + b := make([]byte, 8) binary.PutUvarint(b, server.CommitIndex()) diff --git a/server/server.go b/server/server.go index 8cf837420ee..7fd918c6d69 100644 --- a/server/server.go +++ b/server/server.go @@ -178,8 +178,17 @@ func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Reque return etcdErr.NewError(300, "Empty result from raft", store.UndefIndex, store.UndefTerm) } - response := event.(*store.Event).Response() - b, _ := json.Marshal(response) + if b, ok := event.([]byte); ok { + w.WriteHeader(http.StatusOK) + w.Write(b) + } + + var b []byte + if strings.HasPrefix(req.URL.Path, "/v1") { + b, _ = json.Marshal(event.(*store.Event).Response()) + } else { + b, _ = json.Marshal(event.(*store.Event)) + } w.WriteHeader(http.StatusOK) w.Write(b) diff --git a/server/transporter.go b/server/transporter.go index 03928e2b3ba..56d1b1edc9d 100644 --- a/server/transporter.go +++ b/server/transporter.go @@ -123,7 +123,7 @@ func (t *transporter) SendVoteRequest(server *raft.Server, peer *raft.Peer, req json.NewEncoder(&b).Encode(req) u, _ := t.peerServer.registry.PeerURL(peer.Name) - log.Debugf("Send Vote to %s", u) + log.Debugf("Send Vote from %s to %s", server.Name(), u) resp, httpRequest, err := t.Post(fmt.Sprintf("%s/vote", u), &b) From 8475d9878e62a492461a03fbefc91c101664312f Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sun, 13 Oct 2013 21:14:45 -0600 Subject: [PATCH 088/247] bump(github.com/gorilla/context): 708054d61e5a2918b9f4e9700000ee611dcf03f5 --- .../github.com/gorilla/context/LICENSE | 27 +++++ .../github.com/gorilla/context/README.md | 6 + .../github.com/gorilla/context/context.go | 112 ++++++++++++++++++ .../gorilla/context/context_test.go | 66 +++++++++++ third_party/github.com/gorilla/context/doc.go | 82 +++++++++++++ 5 files changed, 293 insertions(+) create mode 100644 third_party/github.com/gorilla/context/LICENSE create mode 100644 third_party/github.com/gorilla/context/README.md create mode 100644 third_party/github.com/gorilla/context/context.go create mode 100644 third_party/github.com/gorilla/context/context_test.go create mode 100644 third_party/github.com/gorilla/context/doc.go diff --git a/third_party/github.com/gorilla/context/LICENSE b/third_party/github.com/gorilla/context/LICENSE new file mode 100644 index 00000000000..0e5fb872800 --- /dev/null +++ b/third_party/github.com/gorilla/context/LICENSE @@ -0,0 +1,27 @@ +Copyright (c) 2012 Rodrigo Moraes. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/third_party/github.com/gorilla/context/README.md b/third_party/github.com/gorilla/context/README.md new file mode 100644 index 00000000000..8ee62b42634 --- /dev/null +++ b/third_party/github.com/gorilla/context/README.md @@ -0,0 +1,6 @@ +context +======= + +gorilla/context is a general purpose registry for global request variables. + +Read the full documentation here: http://www.gorillatoolkit.org/pkg/context diff --git a/third_party/github.com/gorilla/context/context.go b/third_party/github.com/gorilla/context/context.go new file mode 100644 index 00000000000..35d65561f3c --- /dev/null +++ b/third_party/github.com/gorilla/context/context.go @@ -0,0 +1,112 @@ +// Copyright 2012 The Gorilla Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package context + +import ( + "net/http" + "sync" + "time" +) + +var ( + mutex sync.Mutex + data = make(map[*http.Request]map[interface{}]interface{}) + datat = make(map[*http.Request]int64) +) + +// Set stores a value for a given key in a given request. +func Set(r *http.Request, key, val interface{}) { + mutex.Lock() + defer mutex.Unlock() + if data[r] == nil { + data[r] = make(map[interface{}]interface{}) + datat[r] = time.Now().Unix() + } + data[r][key] = val +} + +// Get returns a value stored for a given key in a given request. +func Get(r *http.Request, key interface{}) interface{} { + mutex.Lock() + defer mutex.Unlock() + if data[r] != nil { + return data[r][key] + } + return nil +} + +// GetOk returns stored value and presence state like multi-value return of map access. +func GetOk(r *http.Request, key interface{}) (interface{}, bool) { + mutex.Lock() + defer mutex.Unlock() + if _, ok := data[r]; ok { + value, ok := data[r][key] + return value, ok + } + return nil, false +} + +// Delete removes a value stored for a given key in a given request. +func Delete(r *http.Request, key interface{}) { + mutex.Lock() + defer mutex.Unlock() + if data[r] != nil { + delete(data[r], key) + } +} + +// Clear removes all values stored for a given request. +// +// This is usually called by a handler wrapper to clean up request +// variables at the end of a request lifetime. See ClearHandler(). +func Clear(r *http.Request) { + mutex.Lock() + defer mutex.Unlock() + clear(r) +} + +// clear is Clear without the lock. +func clear(r *http.Request) { + delete(data, r) + delete(datat, r) +} + +// Purge removes request data stored for longer than maxAge, in seconds. +// It returns the amount of requests removed. +// +// If maxAge <= 0, all request data is removed. +// +// This is only used for sanity check: in case context cleaning was not +// properly set some request data can be kept forever, consuming an increasing +// amount of memory. In case this is detected, Purge() must be called +// periodically until the problem is fixed. +func Purge(maxAge int) int { + mutex.Lock() + defer mutex.Unlock() + count := 0 + if maxAge <= 0 { + count = len(data) + data = make(map[*http.Request]map[interface{}]interface{}) + datat = make(map[*http.Request]int64) + } else { + min := time.Now().Unix() - int64(maxAge) + for r, _ := range data { + if datat[r] < min { + clear(r) + count++ + } + } + } + return count +} + +// ClearHandler wraps an http.Handler and clears request values at the end +// of a request lifetime. +func ClearHandler(h http.Handler) http.Handler { + return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + defer Clear(r) + h.ServeHTTP(w, r) + }) +} diff --git a/third_party/github.com/gorilla/context/context_test.go b/third_party/github.com/gorilla/context/context_test.go new file mode 100644 index 00000000000..ff9e2ad5fc3 --- /dev/null +++ b/third_party/github.com/gorilla/context/context_test.go @@ -0,0 +1,66 @@ +// Copyright 2012 The Gorilla Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +package context + +import ( + "net/http" + "testing" +) + +type keyType int + +const ( + key1 keyType = iota + key2 +) + +func TestContext(t *testing.T) { + assertEqual := func(val interface{}, exp interface{}) { + if val != exp { + t.Errorf("Expected %v, got %v.", exp, val) + } + } + + r, _ := http.NewRequest("GET", "http://localhost:8080/", nil) + + // Get() + assertEqual(Get(r, key1), nil) + + // Set() + Set(r, key1, "1") + assertEqual(Get(r, key1), "1") + assertEqual(len(data[r]), 1) + + Set(r, key2, "2") + assertEqual(Get(r, key2), "2") + assertEqual(len(data[r]), 2) + + //GetOk + value, ok := GetOk(r, key1) + assertEqual(value, "1") + assertEqual(ok, true) + + value, ok = GetOk(r, "not exists") + assertEqual(value, nil) + assertEqual(ok, false) + + Set(r, "nil value", nil) + value, ok = GetOk(r, "nil value") + assertEqual(value, nil) + assertEqual(ok, true) + + // Delete() + Delete(r, key1) + assertEqual(Get(r, key1), nil) + assertEqual(len(data[r]), 2) + + Delete(r, key2) + assertEqual(Get(r, key2), nil) + assertEqual(len(data[r]), 1) + + // Clear() + Clear(r) + assertEqual(len(data), 0) +} diff --git a/third_party/github.com/gorilla/context/doc.go b/third_party/github.com/gorilla/context/doc.go new file mode 100644 index 00000000000..297606455ce --- /dev/null +++ b/third_party/github.com/gorilla/context/doc.go @@ -0,0 +1,82 @@ +// Copyright 2012 The Gorilla Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +/* +Package gorilla/context stores values shared during a request lifetime. + +For example, a router can set variables extracted from the URL and later +application handlers can access those values, or it can be used to store +sessions values to be saved at the end of a request. There are several +others common uses. + +The idea was posted by Brad Fitzpatrick to the go-nuts mailing list: + + http://groups.google.com/group/golang-nuts/msg/e2d679d303aa5d53 + +Here's the basic usage: first define the keys that you will need. The key +type is interface{} so a key can be of any type that supports equality. +Here we define a key using a custom int type to avoid name collisions: + + package foo + + import ( + "github.com/gorilla/context" + ) + + type key int + + const MyKey key = 0 + +Then set a variable. Variables are bound to an http.Request object, so you +need a request instance to set a value: + + context.Set(r, MyKey, "bar") + +The application can later access the variable using the same key you provided: + + func MyHandler(w http.ResponseWriter, r *http.Request) { + // val is "bar". + val := context.Get(r, foo.MyKey) + + // returns ("bar", true) + val, ok := context.GetOk(r, foo.MyKey) + // ... + } + +And that's all about the basic usage. We discuss some other ideas below. + +Any type can be stored in the context. To enforce a given type, make the key +private and wrap Get() and Set() to accept and return values of a specific +type: + + type key int + + const mykey key = 0 + + // GetMyKey returns a value for this package from the request values. + func GetMyKey(r *http.Request) SomeType { + if rv := context.Get(r, mykey); rv != nil { + return rv.(SomeType) + } + return nil + } + + // SetMyKey sets a value for this package in the request values. + func SetMyKey(r *http.Request, val SomeType) { + context.Set(r, mykey, val) + } + +Variables must be cleared at the end of a request, to remove all values +that were stored. This can be done in an http.Handler, after a request was +served. Just call Clear() passing the request: + + context.Clear(r) + +...or use ClearHandler(), which conveniently wraps an http.Handler to clear +variables at the end of a request lifetime. + +The Routers from the packages gorilla/mux and gorilla/pat call Clear() +so if you are using either of them you don't need to clear the context manually. +*/ +package context From 1a27c9de6709f844a3c9fc0c6d77b19e0f14874a Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sun, 13 Oct 2013 21:15:13 -0600 Subject: [PATCH 089/247] Add github.com/gorilla/context. --- third_party/deps | 1 + 1 file changed, 1 insertion(+) diff --git a/third_party/deps b/third_party/deps index 156c0afc8e4..1007477dde6 100755 --- a/third_party/deps +++ b/third_party/deps @@ -3,6 +3,7 @@ packages=" github.com/coreos/go-etcd github.com/coreos/go-log/log github.com/coreos/go-systemd + github.com/gorilla/context github.com/gorilla/mux bitbucket.org/kardianos/osext code.google.com/p/go.net From 0e49c49dce0fbacb461d8f8ddcdccee3ff1fa033 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sun, 13 Oct 2013 21:36:42 -0600 Subject: [PATCH 090/247] Remove verbosity from test. --- test/test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/test/test.go b/test/test.go index f4e5fad2e82..acb212d485a 100644 --- a/test/test.go +++ b/test/test.go @@ -69,13 +69,13 @@ func CreateCluster(size int, procAttr *os.ProcAttr, ssl bool) ([][]string, []*os for i := 0; i < size; i++ { if i == 0 { - argGroup[i] = []string{"etcd", "-v", "-d=/tmp/node1", "-n=node1"} + argGroup[i] = []string{"etcd", "-d=/tmp/node1", "-n=node1"} if ssl { argGroup[i] = append(argGroup[i], sslServer1...) } } else { strI := strconv.Itoa(i + 1) - argGroup[i] = []string{"etcd", "-v", "-n=node" + strI, "-c=127.0.0.1:400" + strI, "-s=127.0.0.1:700" + strI, "-d=/tmp/node" + strI, "-C=127.0.0.1:7001"} + argGroup[i] = []string{"etcd", "-n=node" + strI, "-c=127.0.0.1:400" + strI, "-s=127.0.0.1:700" + strI, "-d=/tmp/node" + strI, "-C=127.0.0.1:7001"} if ssl { argGroup[i] = append(argGroup[i], sslServer2...) } From dc59bd8d77610070cd45552c33b1b8de1fc40a53 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sun, 13 Oct 2013 21:37:45 -0600 Subject: [PATCH 091/247] gofmt --- server/registry.go | 2 +- server/remove_command.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/server/registry.go b/server/registry.go index 20f0f289a3b..55ffb10e03e 100644 --- a/server/registry.go +++ b/server/registry.go @@ -167,7 +167,7 @@ func (r *Registry) PeerURLs(leaderName, selfName string) []string { // Removes a node from the cache. func (r *Registry) Invalidate(name string) { - delete(r.nodes, name) + delete(r.nodes, name) } // Loads the given node by name from the store into the cache. diff --git a/server/remove_command.go b/server/remove_command.go index 852774b0801..6a8f7517125 100644 --- a/server/remove_command.go +++ b/server/remove_command.go @@ -59,7 +59,7 @@ func (c *RemoveCommand) Apply(server *raft.Server) (interface{}, error) { log.Debugf("ignore previous remove command.") } } - + b := make([]byte, 8) binary.PutUvarint(b, server.CommitIndex()) From a635f6b17cc9cab2c428d2b723cb3572b124ed73 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 13 Oct 2013 22:13:20 -0700 Subject: [PATCH 092/247] refactor remove the extra function --- server/join_command.go | 2 +- server/peer_server.go | 2 +- server/registry.go | 45 +++++++++--------------------------------- server/server.go | 6 +++--- 4 files changed, 14 insertions(+), 41 deletions(-) diff --git a/server/join_command.go b/server/join_command.go index d7a9aa599f4..83d2efb7393 100644 --- a/server/join_command.go +++ b/server/join_command.go @@ -45,7 +45,7 @@ func (c *JoinCommand) Apply(server *raft.Server) (interface{}, error) { ps.registry.Invalidate(c.Name) // Check if the join command is from a previous machine, who lost all its previous log. - if _, ok := ps.registry.URL(c.Name); ok { + if _, ok := ps.registry.ClientURL(c.Name); ok { return b, nil } diff --git a/server/peer_server.go b/server/peer_server.go index 62ec06c419e..4c77585f944 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -115,7 +115,7 @@ func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) { } else { // Rejoin the previous cluster - cluster = s.registry.PeerURLs(s.Leader(), s.name) + cluster = s.registry.URLs(s.Leader(), s.name, s.registry.peerURL) for i := 0; i < len(cluster); i++ { u, err := url.Parse(cluster[i]) if err != nil { diff --git a/server/registry.go b/server/registry.go index 55ffb10e03e..4b13108713b 100644 --- a/server/registry.go +++ b/server/registry.go @@ -73,14 +73,14 @@ func (r *Registry) Count() int { return len(e.KVPairs) } -// Retrieves the URL for a given node by name. -func (r *Registry) URL(name string) (string, bool) { +// Retrieves the client URL for a given node by name. +func (r *Registry) ClientURL(name string) (string, bool) { r.Lock() defer r.Unlock() - return r.url(name) + return r.clientURL(name) } -func (r *Registry) url(name string) (string, bool) { +func (r *Registry) clientURL(name string) (string, bool) { if r.nodes[name] == nil { r.load(name) } @@ -92,33 +92,6 @@ func (r *Registry) url(name string) (string, bool) { return "", false } -// Retrieves the URLs for all nodes. -func (r *Registry) URLs(leaderName, selfName string) []string { - r.Lock() - defer r.Unlock() - - // Build list including the leader and self. - urls := make([]string, 0) - if url, _ := r.url(leaderName); len(url) > 0 { - urls = append(urls, url) - } - - // Retrieve a list of all nodes. - if e, _ := r.store.Get(RegistryKey, false, false, 0, 0); e != nil { - // Lookup the URL for each one. - for _, pair := range e.KVPairs { - _, name := filepath.Split(pair.Key) - if url, _ := r.url(name); len(url) > 0 && name != leaderName { - urls = append(urls, url) - } - } - } - - log.Infof("URLs: %s / %s (%s)", leaderName, selfName, strings.Join(urls, ",")) - - return urls -} - // Retrieves the peer URL for a given node by name. func (r *Registry) PeerURL(name string) (string, bool) { r.Lock() @@ -138,14 +111,14 @@ func (r *Registry) peerURL(name string) (string, bool) { return "", false } -// Retrieves the peer URLs for all nodes. -func (r *Registry) PeerURLs(leaderName, selfName string) []string { +// Retrieves the URLs for all nodes using url function. +func (r *Registry) URLs(leaderName, selfName string, url func(name string) (string, bool)) []string { r.Lock() defer r.Unlock() // Build list including the leader and self. urls := make([]string, 0) - if url, _ := r.peerURL(leaderName); len(url) > 0 { + if url, _ := url(leaderName); len(url) > 0 { urls = append(urls, url) } @@ -154,13 +127,13 @@ func (r *Registry) PeerURLs(leaderName, selfName string) []string { // Lookup the URL for each one. for _, pair := range e.KVPairs { _, name := filepath.Split(pair.Key) - if url, _ := r.peerURL(name); len(url) > 0 && name != leaderName { + if url, _ := url(name); len(url) > 0 && name != leaderName { urls = append(urls, url) } } } - log.Infof("PeerURLs: %s / %s (%s)", leaderName, selfName, strings.Join(urls, ",")) + log.Infof("URLs: %s / %s (%s)", leaderName, selfName, strings.Join(urls, ",")) return urls } diff --git a/server/server.go b/server/server.go index 7fd918c6d69..9aae925e7e8 100644 --- a/server/server.go +++ b/server/server.go @@ -207,7 +207,7 @@ func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Reque case *JoinCommand, *RemoveCommand: url, _ = s.registry.PeerURL(leader) default: - url, _ = s.registry.URL(leader) + url, _ = s.registry.ClientURL(leader) } redirect(url, w, req) @@ -258,7 +258,7 @@ func (s *Server) GetLeaderHandler(w http.ResponseWriter, req *http.Request) erro // Handler to return all the known machines in the current cluster. func (s *Server) GetMachinesHandler(w http.ResponseWriter, req *http.Request) error { - machines := s.registry.URLs(s.peerServer.Leader(), s.name) + machines := s.registry.URLs(s.peerServer.Leader(), s.name, s.registry.clientURL) w.WriteHeader(http.StatusOK) w.Write([]byte(strings.Join(machines, ", "))) return nil @@ -281,7 +281,7 @@ func (s *Server) GetLeaderStatsHandler(w http.ResponseWriter, req *http.Request) if leader == "" { return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) } - hostname, _ := s.registry.URL(leader) + hostname, _ := s.registry.ClientURL(leader) redirect(hostname, w, req) return nil } From 7565313290edff9fc4b870b44bb7398df3e0f006 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 13 Oct 2013 22:20:23 -0700 Subject: [PATCH 093/247] refactor add wrapper function --- server/peer_server.go | 2 +- server/registry.go | 14 ++++++++++++-- server/server.go | 2 +- 3 files changed, 14 insertions(+), 4 deletions(-) diff --git a/server/peer_server.go b/server/peer_server.go index 4c77585f944..62ec06c419e 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -115,7 +115,7 @@ func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) { } else { // Rejoin the previous cluster - cluster = s.registry.URLs(s.Leader(), s.name, s.registry.peerURL) + cluster = s.registry.PeerURLs(s.Leader(), s.name) for i := 0; i < len(cluster); i++ { u, err := url.Parse(cluster[i]) if err != nil { diff --git a/server/registry.go b/server/registry.go index 4b13108713b..b9197436bd5 100644 --- a/server/registry.go +++ b/server/registry.go @@ -111,8 +111,18 @@ func (r *Registry) peerURL(name string) (string, bool) { return "", false } -// Retrieves the URLs for all nodes using url function. -func (r *Registry) URLs(leaderName, selfName string, url func(name string) (string, bool)) []string { +// Retrieves the Client URLs for all nodes. +func (r *Registry) ClientURLs(leaderName, selfName string) []string { + return r.urls(leaderName, selfName, r.clientURL) +} + +// Retrieves the Peer URLs for all nodes. +func (r *Registry) PeerURLs(leaderName, selfName string) []string { + return r.urls(leaderName, selfName, r.peerURL) +} + +// Retrieves the URLs for all nodes using url function. +func (r *Registry) urls(leaderName, selfName string, url func(name string) (string, bool)) []string { r.Lock() defer r.Unlock() diff --git a/server/server.go b/server/server.go index 9aae925e7e8..7f55c8a4762 100644 --- a/server/server.go +++ b/server/server.go @@ -258,7 +258,7 @@ func (s *Server) GetLeaderHandler(w http.ResponseWriter, req *http.Request) erro // Handler to return all the known machines in the current cluster. func (s *Server) GetMachinesHandler(w http.ResponseWriter, req *http.Request) error { - machines := s.registry.URLs(s.peerServer.Leader(), s.name, s.registry.clientURL) + machines := s.registry.ClientURLs(s.peerServer.Leader(), s.name) w.WriteHeader(http.StatusOK) w.Write([]byte(strings.Join(machines, ", "))) return nil From 090d049b81b39ba92f8622c545b4e89f6cb8fb51 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 13 Oct 2013 22:50:51 -0700 Subject: [PATCH 094/247] refactor move raft http handlers to peer_server_handlers.go --- server/peer_server.go | 152 ------------------------------- server/peer_server_handlers.go | 162 +++++++++++++++++++++++++++++++++ 2 files changed, 162 insertions(+), 152 deletions(-) create mode 100644 server/peer_server_handlers.go diff --git a/server/peer_server.go b/server/peer_server.go index 62ec06c419e..5edcb200539 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -151,158 +151,6 @@ func (s *PeerServer) SetServer(server *Server) { s.server = server } -// Get all the current logs -func (s *PeerServer) GetLogHttpHandler(w http.ResponseWriter, req *http.Request) { - log.Debugf("[recv] GET %s/log", s.url) - w.Header().Set("Content-Type", "application/json") - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(s.LogEntries()) -} - -// Response to vote request -func (s *PeerServer) VoteHttpHandler(w http.ResponseWriter, req *http.Request) { - rvreq := &raft.RequestVoteRequest{} - err := decodeJsonRequest(req, rvreq) - if err == nil { - log.Debugf("[recv] POST %s/vote [%s]", s.url, rvreq.CandidateName) - if resp := s.RequestVote(rvreq); resp != nil { - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(resp) - return - } - } - log.Warnf("[vote] ERROR: %v", err) - w.WriteHeader(http.StatusInternalServerError) -} - -// Response to append entries request -func (s *PeerServer) AppendEntriesHttpHandler(w http.ResponseWriter, req *http.Request) { - aereq := &raft.AppendEntriesRequest{} - err := decodeJsonRequest(req, aereq) - - if err == nil { - log.Debugf("[recv] POST %s/log/append [%d]", s.url, len(aereq.Entries)) - - s.serverStats.RecvAppendReq(aereq.LeaderName, int(req.ContentLength)) - - if resp := s.AppendEntries(aereq); resp != nil { - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(resp) - if !resp.Success { - log.Debugf("[Append Entry] Step back") - } - return - } - } - log.Warnf("[Append Entry] ERROR: %v", err) - w.WriteHeader(http.StatusInternalServerError) -} - -// Response to recover from snapshot request -func (s *PeerServer) SnapshotHttpHandler(w http.ResponseWriter, req *http.Request) { - aereq := &raft.SnapshotRequest{} - err := decodeJsonRequest(req, aereq) - if err == nil { - log.Debugf("[recv] POST %s/snapshot/ ", s.url) - if resp := s.RequestSnapshot(aereq); resp != nil { - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(resp) - return - } - } - log.Warnf("[Snapshot] ERROR: %v", err) - w.WriteHeader(http.StatusInternalServerError) -} - -// Response to recover from snapshot request -func (s *PeerServer) SnapshotRecoveryHttpHandler(w http.ResponseWriter, req *http.Request) { - aereq := &raft.SnapshotRecoveryRequest{} - err := decodeJsonRequest(req, aereq) - if err == nil { - log.Debugf("[recv] POST %s/snapshotRecovery/ ", s.url) - if resp := s.SnapshotRecoveryRequest(aereq); resp != nil { - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(resp) - return - } - } - log.Warnf("[Snapshot] ERROR: %v", err) - w.WriteHeader(http.StatusInternalServerError) -} - -// Get the port that listening for etcd connecting of the server -func (s *PeerServer) EtcdURLHttpHandler(w http.ResponseWriter, req *http.Request) { - log.Debugf("[recv] Get %s/etcdURL/ ", s.url) - w.WriteHeader(http.StatusOK) - w.Write([]byte(s.server.URL())) -} - -// Response to the join request -func (s *PeerServer) JoinHttpHandler(w http.ResponseWriter, req *http.Request) { - command := &JoinCommand{} - - // Write CORS header. - if s.server.OriginAllowed("*") { - w.Header().Add("Access-Control-Allow-Origin", "*") - } else if s.server.OriginAllowed(req.Header.Get("Origin")) { - w.Header().Add("Access-Control-Allow-Origin", req.Header.Get("Origin")) - } - - err := decodeJsonRequest(req, command) - if err != nil { - w.WriteHeader(http.StatusInternalServerError) - return - } - - log.Debugf("Receive Join Request from %s", command.Name) - err = s.dispatchRaftCommand(command, w, req) - - // Return status. - if err != nil { - if etcdErr, ok := err.(*etcdErr.Error); ok { - log.Debug("Return error: ", (*etcdErr).Error()) - etcdErr.Write(w) - } else { - http.Error(w, err.Error(), http.StatusInternalServerError) - } - } -} - -// Response to remove request -func (s *PeerServer) RemoveHttpHandler(w http.ResponseWriter, req *http.Request) { - if req.Method != "DELETE" { - w.WriteHeader(http.StatusMethodNotAllowed) - return - } - - nodeName := req.URL.Path[len("/remove/"):] - command := &RemoveCommand{ - Name: nodeName, - } - - log.Debugf("[recv] Remove Request [%s]", command.Name) - - s.dispatchRaftCommand(command, w, req) -} - -// Response to the name request -func (s *PeerServer) NameHttpHandler(w http.ResponseWriter, req *http.Request) { - log.Debugf("[recv] Get %s/name/ ", s.url) - w.WriteHeader(http.StatusOK) - w.Write([]byte(s.name)) -} - -// Response to the name request -func (s *PeerServer) RaftVersionHttpHandler(w http.ResponseWriter, req *http.Request) { - log.Debugf("[recv] Get %s/version/ ", s.url) - w.WriteHeader(http.StatusOK) - w.Write([]byte(PeerVersion)) -} - -func (s *PeerServer) dispatchRaftCommand(c raft.Command, w http.ResponseWriter, req *http.Request) error { - return s.dispatch(c, w, req) -} - func (s *PeerServer) startAsLeader() { // leader need to join self as a peer for { diff --git a/server/peer_server_handlers.go b/server/peer_server_handlers.go new file mode 100644 index 00000000000..d18fdc21bdf --- /dev/null +++ b/server/peer_server_handlers.go @@ -0,0 +1,162 @@ +package server + +import ( + "encoding/json" + "net/http" + + etcdErr "github.com/coreos/etcd/error" + "github.com/coreos/etcd/log" + "github.com/coreos/go-raft" +) + +// Get all the current logs +func (s *PeerServer) GetLogHttpHandler(w http.ResponseWriter, req *http.Request) { + log.Debugf("[recv] GET %s/log", s.url) + w.Header().Set("Content-Type", "application/json") + w.WriteHeader(http.StatusOK) + json.NewEncoder(w).Encode(s.LogEntries()) +} + +// Response to vote request +func (s *PeerServer) VoteHttpHandler(w http.ResponseWriter, req *http.Request) { + rvreq := &raft.RequestVoteRequest{} + err := decodeJsonRequest(req, rvreq) + if err == nil { + log.Debugf("[recv] POST %s/vote [%s]", s.url, rvreq.CandidateName) + if resp := s.RequestVote(rvreq); resp != nil { + w.WriteHeader(http.StatusOK) + json.NewEncoder(w).Encode(resp) + return + } + } + log.Warnf("[vote] ERROR: %v", err) + w.WriteHeader(http.StatusInternalServerError) +} + +// Response to append entries request +func (s *PeerServer) AppendEntriesHttpHandler(w http.ResponseWriter, req *http.Request) { + aereq := &raft.AppendEntriesRequest{} + err := decodeJsonRequest(req, aereq) + + if err == nil { + log.Debugf("[recv] POST %s/log/append [%d]", s.url, len(aereq.Entries)) + + s.serverStats.RecvAppendReq(aereq.LeaderName, int(req.ContentLength)) + + if resp := s.AppendEntries(aereq); resp != nil { + w.WriteHeader(http.StatusOK) + json.NewEncoder(w).Encode(resp) + if !resp.Success { + log.Debugf("[Append Entry] Step back") + } + return + } + } + log.Warnf("[Append Entry] ERROR: %v", err) + w.WriteHeader(http.StatusInternalServerError) +} + +// Response to recover from snapshot request +func (s *PeerServer) SnapshotHttpHandler(w http.ResponseWriter, req *http.Request) { + aereq := &raft.SnapshotRequest{} + err := decodeJsonRequest(req, aereq) + if err == nil { + log.Debugf("[recv] POST %s/snapshot/ ", s.url) + if resp := s.RequestSnapshot(aereq); resp != nil { + w.WriteHeader(http.StatusOK) + json.NewEncoder(w).Encode(resp) + return + } + } + log.Warnf("[Snapshot] ERROR: %v", err) + w.WriteHeader(http.StatusInternalServerError) +} + +// Response to recover from snapshot request +func (s *PeerServer) SnapshotRecoveryHttpHandler(w http.ResponseWriter, req *http.Request) { + aereq := &raft.SnapshotRecoveryRequest{} + err := decodeJsonRequest(req, aereq) + if err == nil { + log.Debugf("[recv] POST %s/snapshotRecovery/ ", s.url) + if resp := s.SnapshotRecoveryRequest(aereq); resp != nil { + w.WriteHeader(http.StatusOK) + json.NewEncoder(w).Encode(resp) + return + } + } + log.Warnf("[Snapshot] ERROR: %v", err) + w.WriteHeader(http.StatusInternalServerError) +} + +// Get the port that listening for etcd connecting of the server +func (s *PeerServer) EtcdURLHttpHandler(w http.ResponseWriter, req *http.Request) { + log.Debugf("[recv] Get %s/etcdURL/ ", s.url) + w.WriteHeader(http.StatusOK) + w.Write([]byte(s.server.URL())) +} + +// Response to the join request +func (s *PeerServer) JoinHttpHandler(w http.ResponseWriter, req *http.Request) { + command := &JoinCommand{} + + // Write CORS header. + if s.server.OriginAllowed("*") { + w.Header().Add("Access-Control-Allow-Origin", "*") + } else if s.server.OriginAllowed(req.Header.Get("Origin")) { + w.Header().Add("Access-Control-Allow-Origin", req.Header.Get("Origin")) + } + + err := decodeJsonRequest(req, command) + if err != nil { + w.WriteHeader(http.StatusInternalServerError) + return + } + + log.Debugf("Receive Join Request from %s", command.Name) + err = s.dispatchRaftCommand(command, w, req) + + // Return status. + if err != nil { + if etcdErr, ok := err.(*etcdErr.Error); ok { + log.Debug("Return error: ", (*etcdErr).Error()) + etcdErr.Write(w) + } else { + http.Error(w, err.Error(), http.StatusInternalServerError) + } + } +} + +// Response to remove request +func (s *PeerServer) RemoveHttpHandler(w http.ResponseWriter, req *http.Request) { + if req.Method != "DELETE" { + w.WriteHeader(http.StatusMethodNotAllowed) + return + } + + nodeName := req.URL.Path[len("/remove/"):] + command := &RemoveCommand{ + Name: nodeName, + } + + log.Debugf("[recv] Remove Request [%s]", command.Name) + + s.dispatchRaftCommand(command, w, req) +} + +// Response to the name request +func (s *PeerServer) NameHttpHandler(w http.ResponseWriter, req *http.Request) { + log.Debugf("[recv] Get %s/name/ ", s.url) + w.WriteHeader(http.StatusOK) + w.Write([]byte(s.name)) +} + +// Response to the name request +func (s *PeerServer) RaftVersionHttpHandler(w http.ResponseWriter, req *http.Request) { + log.Debugf("[recv] Get %s/version/ ", s.url) + w.WriteHeader(http.StatusOK) + w.Write([]byte(PeerVersion)) +} + +func (s *PeerServer) dispatchRaftCommand(c raft.Command, w http.ResponseWriter, req *http.Request) error { + return s.dispatch(c, w, req) +} From ef74464aea88a7a6574595d2511f037f6dcbecb7 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 13 Oct 2013 23:09:52 -0700 Subject: [PATCH 095/247] refactor remove extra function --- server/peer_server.go | 55 ++++++++++++++++++---------------- server/peer_server_handlers.go | 8 ++--- server/server.go | 47 +---------------------------- 3 files changed, 33 insertions(+), 77 deletions(-) diff --git a/server/peer_server.go b/server/peer_server.go index 5edcb200539..942d9e062cf 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -9,6 +9,7 @@ import ( "io/ioutil" "net/http" "net/url" + "strings" "time" etcdErr "github.com/coreos/etcd/error" @@ -346,44 +347,48 @@ func (s *PeerServer) monitorSnapshot() { func (s *PeerServer) dispatch(c raft.Command, w http.ResponseWriter, req *http.Request) error { if s.State() == raft.Leader { - if response, err := s.Do(c); err != nil { + result, err := s.Do(c) + if err != nil { return err - } else { - if response == nil { - return etcdErr.NewError(300, "Empty response from raft", store.UndefIndex, store.UndefTerm) - } - - event, ok := response.(*store.Event) - if ok { - bytes, err := json.Marshal(event) - if err != nil { - fmt.Println(err) - } - - w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) - w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term)) - w.WriteHeader(http.StatusOK) - w.Write(bytes) + } - return nil - } + if result == nil { + return etcdErr.NewError(300, "Empty result from raft", store.UndefIndex, store.UndefTerm) + } - bytes, _ := response.([]byte) + // response for raft related commands[join/remove] + if b, ok := result.([]byte); ok { w.WriteHeader(http.StatusOK) - w.Write(bytes) - + w.Write(b) return nil } + var b []byte + if strings.HasPrefix(req.URL.Path, "/v1") { + b, _ = json.Marshal(result.(*store.Event).Response()) + } else { + b, _ = json.Marshal(result.(*store.Event)) + } + w.WriteHeader(http.StatusOK) + w.Write(b) + + return nil + } else { leader := s.Leader() - // current no leader + + // No leader available. if leader == "" { return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) } - url, _ := s.registry.PeerURL(leader) - log.Debugf("Not leader; Current leader: %s; redirect: %s", leader, url) + var url string + switch c.(type) { + case *JoinCommand, *RemoveCommand: + url, _ = s.registry.PeerURL(leader) + default: + url, _ = s.registry.ClientURL(leader) + } redirect(url, w, req) return nil diff --git a/server/peer_server_handlers.go b/server/peer_server_handlers.go index d18fdc21bdf..ad055623eb8 100644 --- a/server/peer_server_handlers.go +++ b/server/peer_server_handlers.go @@ -113,7 +113,7 @@ func (s *PeerServer) JoinHttpHandler(w http.ResponseWriter, req *http.Request) { } log.Debugf("Receive Join Request from %s", command.Name) - err = s.dispatchRaftCommand(command, w, req) + err = s.dispatch(command, w, req) // Return status. if err != nil { @@ -140,7 +140,7 @@ func (s *PeerServer) RemoveHttpHandler(w http.ResponseWriter, req *http.Request) log.Debugf("[recv] Remove Request [%s]", command.Name) - s.dispatchRaftCommand(command, w, req) + s.dispatch(command, w, req) } // Response to the name request @@ -156,7 +156,3 @@ func (s *PeerServer) RaftVersionHttpHandler(w http.ResponseWriter, req *http.Req w.WriteHeader(http.StatusOK) w.Write([]byte(PeerVersion)) } - -func (s *PeerServer) dispatchRaftCommand(c raft.Command, w http.ResponseWriter, req *http.Request) error { - return s.dispatch(c, w, req) -} diff --git a/server/server.go b/server/server.go index 7f55c8a4762..24064fa4e86 100644 --- a/server/server.go +++ b/server/server.go @@ -1,7 +1,6 @@ package server import ( - "encoding/json" "fmt" "net/http" "net/url" @@ -168,51 +167,7 @@ func (s *Server) ListenAndServe() { } func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Request) error { - if s.peerServer.State() == raft.Leader { - event, err := s.peerServer.Do(c) - if err != nil { - return err - } - - if event == nil { - return etcdErr.NewError(300, "Empty result from raft", store.UndefIndex, store.UndefTerm) - } - - if b, ok := event.([]byte); ok { - w.WriteHeader(http.StatusOK) - w.Write(b) - } - - var b []byte - if strings.HasPrefix(req.URL.Path, "/v1") { - b, _ = json.Marshal(event.(*store.Event).Response()) - } else { - b, _ = json.Marshal(event.(*store.Event)) - } - w.WriteHeader(http.StatusOK) - w.Write(b) - - return nil - - } else { - leader := s.peerServer.Leader() - - // No leader available. - if leader == "" { - return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) - } - - var url string - switch c.(type) { - case *JoinCommand, *RemoveCommand: - url, _ = s.registry.PeerURL(leader) - default: - url, _ = s.registry.ClientURL(leader) - } - redirect(url, w, req) - - return nil - } + return s.peerServer.dispatch(c, w, req) } // Sets a comma-delimited list of origins that are allowed. From 63e128670ef00cc0fb3440deaf910913b68b4d73 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 14 Oct 2013 09:55:57 -0600 Subject: [PATCH 096/247] Fix etcd.NewClient(). --- etcd_test.go | 12 ++++++------ test/test.go | 2 +- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/etcd_test.go b/etcd_test.go index 8aaa2735d98..30df557fdeb 100644 --- a/etcd_test.go +++ b/etcd_test.go @@ -32,7 +32,7 @@ func TestSingleNode(t *testing.T) { time.Sleep(time.Second) - c := etcd.NewClient() + c := etcd.NewClient(nil) c.SyncCluster() // Test Set @@ -145,7 +145,7 @@ func TestSingleNodeRecovery(t *testing.T) { time.Sleep(time.Second) - c := etcd.NewClient() + c := etcd.NewClient(nil) c.SyncCluster() // Test Set @@ -205,7 +205,7 @@ func templateTestSimpleMultiNode(t *testing.T, tls bool) { time.Sleep(time.Second) - c := etcd.NewClient() + c := etcd.NewClient(nil) c.SyncCluster() @@ -255,7 +255,7 @@ func TestMultiNodeKillAllAndRecovery(t *testing.T) { t.Fatal("cannot create cluster") } - c := etcd.NewClient() + c := etcd.NewClient(nil) c.SyncCluster() @@ -320,7 +320,7 @@ func TestMultiNodeKillOne(t *testing.T) { time.Sleep(2 * time.Second) - c := etcd.NewClient() + c := etcd.NewClient(nil) c.SyncCluster() @@ -481,7 +481,7 @@ func TestRemoveNode(t *testing.T) { time.Sleep(time.Second) - c := etcd.NewClient() + c := etcd.NewClient(nil) c.SyncCluster() diff --git a/test/test.go b/test/test.go index acb212d485a..ea54584514c 100644 --- a/test/test.go +++ b/test/test.go @@ -22,7 +22,7 @@ func Set(stop chan bool) { stopSet := false i := 0 - c := etcd.NewClient() + c := etcd.NewClient(nil) for { key := fmt.Sprintf("%s_%v", "foo", i) From 375f7a73b9ea102986ae78bd3de31d30c13dfe19 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 14 Oct 2013 10:05:38 -0600 Subject: [PATCH 097/247] bump(github.com/coreos/go-raft): 9fed6b9e77a96f1ab3a4c02bc8dbed72fbac9eb0 --- third_party/github.com/coreos/go-raft/server.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/third_party/github.com/coreos/go-raft/server.go b/third_party/github.com/coreos/go-raft/server.go index e42e3ad62fb..b9b00e697d7 100644 --- a/third_party/github.com/coreos/go-raft/server.go +++ b/third_party/github.com/coreos/go-raft/server.go @@ -854,7 +854,7 @@ func (s *Server) processAppendEntriesResponse(resp *AppendEntriesResponse) { for _, peer := range s.peers { indices = append(indices, peer.getPrevLogIndex()) } - sort.Sort(uint64Slice(indices)) + sort.Sort(sort.Reverse(uint64Slice(indices))) // We can commit up to the index which the majority of the members have appended. commitIndex := indices[s.QuorumSize()-1] From 1843f7bda59c45fb101ec24f986777b775da545e Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 14 Oct 2013 10:05:39 -0600 Subject: [PATCH 098/247] bump(github.com/coreos/go-etcd): 0cc84e9bc81c45e074864360adc549e61a3a7f83 --- .../github.com/coreos/go-etcd/etcd/client.go | 21 ++++++-- .../coreos/go-etcd/etcd/client_test.go | 22 +++++++- .../coreos/go-etcd/etcd/delete_test.go | 2 +- .../coreos/go-etcd/etcd/get_test.go | 2 +- .../coreos/go-etcd/etcd/list_test.go | 2 +- .../coreos/go-etcd/etcd/set_test.go | 2 +- .../coreos/go-etcd/etcd/testAndSet_test.go | 2 +- .../github.com/coreos/go-etcd/etcd/watch.go | 7 ++- .../coreos/go-etcd/etcd/watch_test.go | 11 ++-- .../go-etcd/examples/sync-cluster/README.md | 3 ++ .../examples/sync-cluster/sync-cluster.go | 51 +++++++++++++++++++ 11 files changed, 110 insertions(+), 15 deletions(-) create mode 100644 third_party/github.com/coreos/go-etcd/examples/sync-cluster/README.md create mode 100644 third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go diff --git a/third_party/github.com/coreos/go-etcd/etcd/client.go b/third_party/github.com/coreos/go-etcd/etcd/client.go index cbbd7ad9c5a..31d3c2a3a76 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/client.go +++ b/third_party/github.com/coreos/go-etcd/etcd/client.go @@ -36,12 +36,16 @@ type Client struct { } // Setup a basic conf and cluster -func NewClient() *Client { +func NewClient(machines []string) *Client { + // if an empty slice was sent in then just assume localhost + if len(machines) == 0 { + machines = []string{"http://127.0.0.1:4001"} + } // default leader and machines cluster := Cluster{ - Leader: "http://127.0.0.1:4001", - Machines: []string{"http://127.0.0.1:4001"}, + Leader: machines[0], + Machines: machines, } config := Config{ @@ -107,6 +111,10 @@ func (c *Client) SetCluster(machines []string) bool { return success } +func (c *Client) GetCluster() []string { + return c.cluster.Machines +} + // sycn cluster information using the existing machine list func (c *Client) SyncCluster() bool { success := c.internalSyncCluster(c.cluster.Machines) @@ -128,14 +136,16 @@ func (c *Client) internalSyncCluster(machines []string) bool { // try another machine in the cluster continue } + // update Machines List - c.cluster.Machines = strings.Split(string(b), ",") + c.cluster.Machines = strings.Split(string(b), ", ") // update leader // the first one in the machine list is the leader logger.Debugf("update.leader[%s,%s]", c.cluster.Leader, c.cluster.Machines[0]) c.cluster.Leader = c.cluster.Machines[0] + logger.Debug("sync.machines ", c.cluster.Machines) return true } } @@ -146,6 +156,9 @@ func (c *Client) internalSyncCluster(machines []string) bool { func (c *Client) createHttpPath(serverName string, _path string) string { u, _ := url.Parse(serverName) u.Path = path.Join(u.Path, "/", _path) + if u.Scheme == "" { + u.Scheme = "http" + } return u.String() } diff --git a/third_party/github.com/coreos/go-etcd/etcd/client_test.go b/third_party/github.com/coreos/go-etcd/etcd/client_test.go index 45a99e96c1d..29f138113ce 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/client_test.go +++ b/third_party/github.com/coreos/go-etcd/etcd/client_test.go @@ -3,6 +3,8 @@ package etcd import ( "fmt" "testing" + "net/url" + "net" ) // To pass this test, we need to create a cluster of 3 machines @@ -10,13 +12,31 @@ import ( func TestSync(t *testing.T) { fmt.Println("Make sure there are three nodes at 0.0.0.0:4001-4003") - c := NewClient() + c := NewClient(nil) success := c.SyncCluster() if !success { t.Fatal("cannot sync machines") } + for _, m := range(c.GetCluster()) { + u, err := url.Parse(m) + if err != nil { + t.Fatal(err) + } + if u.Scheme != "http" { + t.Fatal("scheme must be http") + } + + host, _, err := net.SplitHostPort(u.Host) + if err != nil { + t.Fatal(err) + } + if host != "127.0.0.1" { + t.Fatal("Host must be 127.0.0.1") + } + } + badMachines := []string{"abc", "edef"} success = c.SetCluster(badMachines) diff --git a/third_party/github.com/coreos/go-etcd/etcd/delete_test.go b/third_party/github.com/coreos/go-etcd/etcd/delete_test.go index a5f9801676a..52756d09fcd 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/delete_test.go +++ b/third_party/github.com/coreos/go-etcd/etcd/delete_test.go @@ -6,7 +6,7 @@ import ( func TestDelete(t *testing.T) { - c := NewClient() + c := NewClient(nil) c.Set("foo", "bar", 100) result, err := c.Delete("foo") diff --git a/third_party/github.com/coreos/go-etcd/etcd/get_test.go b/third_party/github.com/coreos/go-etcd/etcd/get_test.go index 8e3852c9fcf..ff81374003c 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/get_test.go +++ b/third_party/github.com/coreos/go-etcd/etcd/get_test.go @@ -7,7 +7,7 @@ import ( func TestGet(t *testing.T) { - c := NewClient() + c := NewClient(nil) c.Set("foo", "bar", 100) diff --git a/third_party/github.com/coreos/go-etcd/etcd/list_test.go b/third_party/github.com/coreos/go-etcd/etcd/list_test.go index 1e98e76451a..382bb356d91 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/list_test.go +++ b/third_party/github.com/coreos/go-etcd/etcd/list_test.go @@ -6,7 +6,7 @@ import ( ) func TestList(t *testing.T) { - c := NewClient() + c := NewClient(nil) c.Set("foo_list/foo", "bar", 100) c.Set("foo_list/fooo", "barbar", 100) diff --git a/third_party/github.com/coreos/go-etcd/etcd/set_test.go b/third_party/github.com/coreos/go-etcd/etcd/set_test.go index dc46608d78b..3809ee952a9 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/set_test.go +++ b/third_party/github.com/coreos/go-etcd/etcd/set_test.go @@ -6,7 +6,7 @@ import ( ) func TestSet(t *testing.T) { - c := NewClient() + c := NewClient(nil) result, err := c.Set("foo", "bar", 100) diff --git a/third_party/github.com/coreos/go-etcd/etcd/testAndSet_test.go b/third_party/github.com/coreos/go-etcd/etcd/testAndSet_test.go index ba6d0e8f5c3..5dbd854b515 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/testAndSet_test.go +++ b/third_party/github.com/coreos/go-etcd/etcd/testAndSet_test.go @@ -6,7 +6,7 @@ import ( ) func TestTestAndSet(t *testing.T) { - c := NewClient() + c := NewClient(nil) c.Set("foo_testAndSet", "bar", 100) diff --git a/third_party/github.com/coreos/go-etcd/etcd/watch.go b/third_party/github.com/coreos/go-etcd/etcd/watch.go index 5da5565c66a..7f59ed065a3 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/watch.go +++ b/third_party/github.com/coreos/go-etcd/etcd/watch.go @@ -16,6 +16,11 @@ type respAndErr struct { err error } +// Errors introduced by the Watch command. +var ( + ErrWatchStoppedByUser = errors.New("Watch stopped by the user via stop channel") +) + // Watch any change under the given prefix. // When a sinceIndex is given, watch will try to scan from that index to the last index // and will return any changes under the given prefix during the history @@ -66,7 +71,7 @@ func (c *Client) watchOnce(key string, sinceIndex uint64, stop chan bool) (*stor resp, err = res.resp, res.err case <-stop: - resp, err = nil, errors.New("User stoped watch") + resp, err = nil, ErrWatchStoppedByUser } } else { resp, err = c.sendWatchRequest(key, sinceIndex) diff --git a/third_party/github.com/coreos/go-etcd/etcd/watch_test.go b/third_party/github.com/coreos/go-etcd/etcd/watch_test.go index 5e18a2b2909..0d9348518ca 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/watch_test.go +++ b/third_party/github.com/coreos/go-etcd/etcd/watch_test.go @@ -8,7 +8,7 @@ import ( ) func TestWatch(t *testing.T) { - c := NewClient() + c := NewClient(nil) go setHelper("bar", c) @@ -35,9 +35,12 @@ func TestWatch(t *testing.T) { go setLoop("bar", c) - go reciver(ch, stop) + go receiver(ch, stop) - c.Watch("watch_foo", 0, ch, stop) + _, err = c.Watch("watch_foo", 0, ch, stop) + if err != ErrWatchStoppedByUser { + t.Fatalf("Watch returned a non-user stop error") + } } func setHelper(value string, c *Client) { @@ -54,7 +57,7 @@ func setLoop(value string, c *Client) { } } -func reciver(c chan *store.Response, stop chan bool) { +func receiver(c chan *store.Response, stop chan bool) { for i := 0; i < 10; i++ { <-c } diff --git a/third_party/github.com/coreos/go-etcd/examples/sync-cluster/README.md b/third_party/github.com/coreos/go-etcd/examples/sync-cluster/README.md new file mode 100644 index 00000000000..145744febf6 --- /dev/null +++ b/third_party/github.com/coreos/go-etcd/examples/sync-cluster/README.md @@ -0,0 +1,3 @@ +Example script from the sync-cluster bug https://github.com/coreos/go-etcd/issues/27 + +TODO: turn this into a test case diff --git a/third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go b/third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go new file mode 100644 index 00000000000..8249b4bdcb2 --- /dev/null +++ b/third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go @@ -0,0 +1,51 @@ + +package main + +import ( + "fmt" + "github.com/coreos/go-etcd/etcd" + "strconv" + "time" +) + +func main() { + fmt.Println("etcd-client started") + c := etcd.NewClient(nil) + c.SetCluster([]string{ + "http://127.0.0.1:4001", + "http://127.0.0.1:4002", + "http://127.0.0.1:4003", + }) + + ticker := time.NewTicker(time.Second * 3) + + for { + select { + case d := <-ticker.C: + n := d.Second() + if n <= 0 { + n = 60 + } + + for ok := c.SyncCluster(); ok == false; { + fmt.Println("SyncCluster failed, trying again") + time.Sleep(100 * time.Millisecond) + } + + result, err := c.Set("foo", "exp_"+strconv.Itoa(n), 0) + if err != nil { + fmt.Println("set error", err) + } else { + fmt.Printf("set %+v\n", result) + } + + ss, err := c.Get("foo") + if err != nil { + fmt.Println("get error", err) + } else { + fmt.Println(len(ss)) + } + + } + } +} From 13b86f536019946b45ed65a798160d9d8afd000e Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 14 Oct 2013 10:05:43 -0600 Subject: [PATCH 099/247] bump(github.com/coreos/go-log/log): --- .../github.com/coreos/go-log/log/commands.go | 30 ++++++++++--------- .../github.com/coreos/go-log/log/fields.go | 3 +- .../github.com/coreos/go-log/log/logger.go | 3 +- .../github.com/coreos/go-log/log/priority.go | 3 +- .../github.com/coreos/go-log/log/sinks.go | 3 +- 5 files changed, 20 insertions(+), 22 deletions(-) diff --git a/third_party/github.com/coreos/go-log/log/commands.go b/third_party/github.com/coreos/go-log/log/commands.go index 9c661272d69..94dc9e152d7 100644 --- a/third_party/github.com/coreos/go-log/log/commands.go +++ b/third_party/github.com/coreos/go-log/log/commands.go @@ -1,6 +1,5 @@ package log - -// Copyright 2013, David Fisher. All rights reserved. +// Copyright 2013, CoreOS, Inc. All rights reserved. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -43,6 +42,7 @@ func (logger *Logger) Logf(priority Priority, format string, v ...interface{}) { logger.Log(priority, fmt.Sprintf(format, v...)) } + func (logger *Logger) Emergency(v ...interface{}) { logger.Log(PriEmerg, v...) } @@ -99,6 +99,7 @@ func (logger *Logger) Debugf(format string, v ...interface{}) { logger.Log(PriDebug, fmt.Sprintf(format, v...)) } + func Emergency(v ...interface{}) { defaultLogger.Log(PriEmerg, v...) } @@ -157,56 +158,57 @@ func Debugf(format string, v ...interface{}) { // Standard library log functions -func (logger *Logger) Fatalln(v ...interface{}) { +func (logger *Logger)Fatalln (v ...interface{}) { logger.Log(PriCrit, v...) os.Exit(1) } -func (logger *Logger) Fatalf(format string, v ...interface{}) { +func (logger *Logger)Fatalf (format string, v ...interface{}) { logger.Logf(PriCrit, format, v...) os.Exit(1) } -func (logger *Logger) Panicln(v ...interface{}) { +func (logger *Logger)Panicln (v ...interface{}) { s := fmt.Sprint(v...) logger.Log(PriErr, s) panic(s) } -func (logger *Logger) Panicf(format string, v ...interface{}) { +func (logger *Logger)Panicf (format string, v ...interface{}) { s := fmt.Sprintf(format, v...) logger.Log(PriErr, s) panic(s) } -func (logger *Logger) Println(v ...interface{}) { +func (logger *Logger)Println (v ...interface{}) { logger.Log(PriInfo, v...) } -func (logger *Logger) Printf(format string, v ...interface{}) { +func (logger *Logger)Printf (format string, v ...interface{}) { logger.Logf(PriInfo, format, v...) } -func Fatalln(v ...interface{}) { + +func Fatalln (v ...interface{}) { defaultLogger.Log(PriCrit, v...) os.Exit(1) } -func Fatalf(format string, v ...interface{}) { +func Fatalf (format string, v ...interface{}) { defaultLogger.Logf(PriCrit, format, v...) os.Exit(1) } -func Panicln(v ...interface{}) { +func Panicln (v ...interface{}) { s := fmt.Sprint(v...) defaultLogger.Log(PriErr, s) panic(s) } -func Panicf(format string, v ...interface{}) { +func Panicf (format string, v ...interface{}) { s := fmt.Sprintf(format, v...) defaultLogger.Log(PriErr, s) panic(s) } -func Println(v ...interface{}) { +func Println (v ...interface{}) { defaultLogger.Log(PriInfo, v...) } -func Printf(format string, v ...interface{}) { +func Printf (format string, v ...interface{}) { defaultLogger.Logf(PriInfo, format, v...) } diff --git a/third_party/github.com/coreos/go-log/log/fields.go b/third_party/github.com/coreos/go-log/log/fields.go index 547d941d3a2..e8d9698a08a 100644 --- a/third_party/github.com/coreos/go-log/log/fields.go +++ b/third_party/github.com/coreos/go-log/log/fields.go @@ -1,6 +1,5 @@ package log - -// Copyright 2013, David Fisher. All rights reserved. +// Copyright 2013, CoreOS, Inc. All rights reserved. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/third_party/github.com/coreos/go-log/log/logger.go b/third_party/github.com/coreos/go-log/log/logger.go index 59ecdabcb40..2089a11f898 100644 --- a/third_party/github.com/coreos/go-log/log/logger.go +++ b/third_party/github.com/coreos/go-log/log/logger.go @@ -1,6 +1,5 @@ package log - -// Copyright 2013, David Fisher. All rights reserved. +// Copyright 2013, CoreOS, Inc. All rights reserved. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/third_party/github.com/coreos/go-log/log/priority.go b/third_party/github.com/coreos/go-log/log/priority.go index 93183887726..ac73fc8a42c 100644 --- a/third_party/github.com/coreos/go-log/log/priority.go +++ b/third_party/github.com/coreos/go-log/log/priority.go @@ -1,6 +1,5 @@ package log - -// Copyright 2013, David Fisher. All rights reserved. +// Copyright 2013, CoreOS, Inc. All rights reserved. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/third_party/github.com/coreos/go-log/log/sinks.go b/third_party/github.com/coreos/go-log/log/sinks.go index 5cda993da84..a41f3365dfd 100644 --- a/third_party/github.com/coreos/go-log/log/sinks.go +++ b/third_party/github.com/coreos/go-log/log/sinks.go @@ -1,6 +1,5 @@ package log - -// Copyright 2013, David Fisher. All rights reserved. +// Copyright 2013, CoreOS, Inc. All rights reserved. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. From 6b55a470902029c494dba4dad4e969545fb1e108 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 14 Oct 2013 10:05:43 -0600 Subject: [PATCH 100/247] bump(github.com/coreos/go-systemd): af0e6cd015e5fad8d9d090ad7aa343a1f680318d --- .../coreos/go-systemd/activation/files.go | 3 +- .../github.com/coreos/go-systemd/dbus/dbus.go | 100 +++++++++ .../coreos/go-systemd/dbus/methods.go | 166 +++++++++++++++ .../coreos/go-systemd/dbus/properties.go | 193 ++++++++++++++++++ .../coreos/go-systemd/dbus/subscription.go | 182 +++++++++++++++++ .../coreos/go-systemd/journal/send.go | 6 +- 6 files changed, 648 insertions(+), 2 deletions(-) create mode 100644 third_party/github.com/coreos/go-systemd/dbus/dbus.go create mode 100644 third_party/github.com/coreos/go-systemd/dbus/methods.go create mode 100644 third_party/github.com/coreos/go-systemd/dbus/properties.go create mode 100644 third_party/github.com/coreos/go-systemd/dbus/subscription.go diff --git a/third_party/github.com/coreos/go-systemd/activation/files.go b/third_party/github.com/coreos/go-systemd/activation/files.go index eb60e77dfe0..4b854237077 100644 --- a/third_party/github.com/coreos/go-systemd/activation/files.go +++ b/third_party/github.com/coreos/go-systemd/activation/files.go @@ -1,3 +1,4 @@ +// Package activation implements primitives for systemd socket activation. package activation import ( @@ -23,7 +24,7 @@ func Files() []*os.File { files := []*os.File(nil) for fd := listenFdsStart; fd < listenFdsStart+nfds; fd++ { syscall.CloseOnExec(fd) - files = append(files, os.NewFile(uintptr(fd), "LISTEN_FD_"+strconv.Itoa(fd))) + files = append(files, os.NewFile(uintptr(fd), "LISTEN_FD_" + strconv.Itoa(fd))) } return files } diff --git a/third_party/github.com/coreos/go-systemd/dbus/dbus.go b/third_party/github.com/coreos/go-systemd/dbus/dbus.go new file mode 100644 index 00000000000..317f36552e7 --- /dev/null +++ b/third_party/github.com/coreos/go-systemd/dbus/dbus.go @@ -0,0 +1,100 @@ +// Integration with the systemd D-Bus API. See http://www.freedesktop.org/wiki/Software/systemd/dbus/ +package dbus + +import ( + "github.com/guelfey/go.dbus" + "sync" +) + +const signalBuffer = 100 + +type Conn struct { + sysconn *dbus.Conn + sysobj *dbus.Object + jobListener struct { + jobs map[dbus.ObjectPath]chan string + sync.Mutex + } + subscriber struct { + updateCh chan<- *SubStateUpdate + errCh chan<- error + sync.Mutex + ignore map[dbus.ObjectPath]int64 + cleanIgnore int64 + } + dispatch map[string]func(dbus.Signal) +} + +func New() *Conn { + c := new(Conn) + c.initConnection() + c.initJobs() + c.initSubscription() + c.initDispatch() + return c +} + +func (c *Conn) initConnection() { + var err error + c.sysconn, err = dbus.SystemBusPrivate() + if err != nil { + return + } + + err = c.sysconn.Auth(nil) + if err != nil { + c.sysconn.Close() + return + } + + err = c.sysconn.Hello() + if err != nil { + c.sysconn.Close() + return + } + + c.sysobj = c.sysconn.Object("org.freedesktop.systemd1", dbus.ObjectPath("/org/freedesktop/systemd1")) + + c.sysconn.BusObject().Call("org.freedesktop.DBus.AddMatch", 0, + "type='signal',interface='org.freedesktop.systemd1.Manager',member='JobRemoved'") + c.sysconn.BusObject().Call("org.freedesktop.DBus.AddMatch", 0, + "type='signal',interface='org.freedesktop.systemd1.Manager',member='UnitNew'") + c.sysconn.BusObject().Call("org.freedesktop.DBus.AddMatch", 0, + "type='signal',interface='org.freedesktop.DBus.Properties',member='PropertiesChanged'") + + err = c.sysobj.Call("org.freedesktop.systemd1.Manager.Subscribe", 0).Store() + if err != nil { + c.sysconn.Close() + return + } +} + +func (c *Conn) initDispatch() { + ch := make(chan *dbus.Signal, signalBuffer) + + c.sysconn.Signal(ch) + + go func() { + for { + signal := <-ch + switch signal.Name { + case "org.freedesktop.systemd1.Manager.JobRemoved": + c.jobComplete(signal) + + unitName := signal.Body[2].(string) + var unitPath dbus.ObjectPath + c.sysobj.Call("GetUnit", 0, unitName).Store(&unitPath) + if unitPath != dbus.ObjectPath("") { + c.sendSubStateUpdate(unitPath) + } + case "org.freedesktop.systemd1.Manager.UnitNew": + c.sendSubStateUpdate(signal.Body[1].(dbus.ObjectPath)) + case "org.freedesktop.DBus.Properties.PropertiesChanged": + if signal.Body[0].(string) == "org.freedesktop.systemd1.Unit" { + // we only care about SubState updates, which are a Unit property + c.sendSubStateUpdate(signal.Path) + } + } + } + }() +} diff --git a/third_party/github.com/coreos/go-systemd/dbus/methods.go b/third_party/github.com/coreos/go-systemd/dbus/methods.go new file mode 100644 index 00000000000..0e99cf5a775 --- /dev/null +++ b/third_party/github.com/coreos/go-systemd/dbus/methods.go @@ -0,0 +1,166 @@ +package dbus + +import ( + "github.com/guelfey/go.dbus" +) + +func (c *Conn) initJobs() { + c.jobListener.jobs = make(map[dbus.ObjectPath]chan string) +} + +func (c *Conn) jobComplete(signal *dbus.Signal) { + var id uint32 + var job dbus.ObjectPath + var unit string + var result string + dbus.Store(signal.Body, &id, &job, &unit, &result) + c.jobListener.Lock() + out, ok := c.jobListener.jobs[job] + if ok { + out <- result + } + c.jobListener.Unlock() +} + +func (c *Conn) startJob(job string, args ...interface{}) (<-chan string, error) { + c.jobListener.Lock() + defer c.jobListener.Unlock() + + ch := make(chan string, 1) + var path dbus.ObjectPath + err := c.sysobj.Call(job, 0, args...).Store(&path) + if err != nil { + return nil, err + } + c.jobListener.jobs[path] = ch + return ch, nil +} + +func (c *Conn) runJob(job string, args ...interface{}) (string, error) { + respCh, err := c.startJob(job, args...) + if err != nil { + return "", err + } + return <-respCh, nil +} + +// StartUnit enqeues a start job and depending jobs, if any (unless otherwise +// specified by the mode string). +// +// Takes the unit to activate, plus a mode string. The mode needs to be one of +// replace, fail, isolate, ignore-dependencies, ignore-requirements. If +// "replace" the call will start the unit and its dependencies, possibly +// replacing already queued jobs that conflict with this. If "fail" the call +// will start the unit and its dependencies, but will fail if this would change +// an already queued job. If "isolate" the call will start the unit in question +// and terminate all units that aren't dependencies of it. If +// "ignore-dependencies" it will start a unit but ignore all its dependencies. +// If "ignore-requirements" it will start a unit but only ignore the +// requirement dependencies. It is not recommended to make use of the latter +// two options. +// +// Result string: one of done, canceled, timeout, failed, dependency, skipped. +// done indicates successful execution of a job. canceled indicates that a job +// has been canceled before it finished execution. timeout indicates that the +// job timeout was reached. failed indicates that the job failed. dependency +// indicates that a job this job has been depending on failed and the job hence +// has been removed too. skipped indicates that a job was skipped because it +// didn't apply to the units current state. +func (c *Conn) StartUnit(name string, mode string) (string, error) { + return c.runJob("StartUnit", name, mode) +} + +// StopUnit is similar to StartUnit but stops the specified unit rather +// than starting it. +func (c *Conn) StopUnit(name string, mode string) (string, error) { + return c.runJob("StopUnit", name, mode) +} + +// ReloadUnit reloads a unit. Reloading is done only if the unit is already running and fails otherwise. +func (c *Conn) ReloadUnit(name string, mode string) (string, error) { + return c.runJob("ReloadUnit", name, mode) +} + +// RestartUnit restarts a service. If a service is restarted that isn't +// running it will be started. +func (c *Conn) RestartUnit(name string, mode string) (string, error) { + return c.runJob("RestartUnit", name, mode) +} + +// TryRestartUnit is like RestartUnit, except that a service that isn't running +// is not affected by the restart. +func (c *Conn) TryRestartUnit(name string, mode string) (string, error) { + return c.runJob("TryRestartUnit", name, mode) +} + +// ReloadOrRestart attempts a reload if the unit supports it and use a restart +// otherwise. +func (c *Conn) ReloadOrRestartUnit(name string, mode string) (string, error) { + return c.runJob("ReloadOrRestartUnit", name, mode) +} + +// ReloadOrTryRestart attempts a reload if the unit supports it and use a "Try" +// flavored restart otherwise. +func (c *Conn) ReloadOrTryRestartUnit(name string, mode string) (string, error) { + return c.runJob("ReloadOrTryRestartUnit", name, mode) +} + +// StartTransientUnit() may be used to create and start a transient unit, which +// will be released as soon as it is not running or referenced anymore or the +// system is rebooted. name is the unit name including suffix, and must be +// unique. mode is the same as in StartUnit(), properties contains properties +// of the unit. +func (c *Conn) StartTransientUnit(name string, mode string, properties ...Property) (string, error) { + // the dbus interface for this method does not use the last argument and + // should simply be given an empty list. We use a concrete type here + // (instead of the more appropriate interface{}) to satisfy the dbus library. + return c.runJob("StartTransientUnit", name, mode, properties, make([]string, 0)) +} + +// KillUnit takes the unit name and a UNIX signal number to send. All of the unit's +// processes are killed. +func (c *Conn) KillUnit(name string, signal int32) { + c.sysobj.Call("KillUnit", 0, name, "all", signal).Store() +} + +// ListUnits returns an array with all currently loaded units. Note that +// units may be known by multiple names at the same time, and hence there might +// be more unit names loaded than actual units behind them. +func (c *Conn) ListUnits() ([]UnitStatus, error) { + result := make([][]interface{}, 0) + err := c.sysobj.Call("ListUnits", 0).Store(&result) + if err != nil { + return nil, err + } + + resultInterface := make([]interface{}, len(result)) + for i := range result { + resultInterface[i] = result[i] + } + + status := make([]UnitStatus, len(result)) + statusInterface := make([]interface{}, len(status)) + for i := range status { + statusInterface[i] = &status[i] + } + + err = dbus.Store(resultInterface, statusInterface...) + if err != nil { + return nil, err + } + + return status, nil +} + +type UnitStatus struct { + Name string // The primary unit name as string + Description string // The human readable description string + LoadState string // The load state (i.e. whether the unit file has been loaded successfully) + ActiveState string // The active state (i.e. whether the unit is currently started or not) + SubState string // The sub state (a more fine-grained version of the active state that is specific to the unit type, which the active state is not) + Followed string // A unit that is being followed in its state by this unit, if there is any, otherwise the empty string. + Path dbus.ObjectPath // The unit object path + JobId uint32 // If there is a job queued for the job unit the numeric job id, 0 otherwise + JobType string // The job type as string + JobPath dbus.ObjectPath // The job object path +} diff --git a/third_party/github.com/coreos/go-systemd/dbus/properties.go b/third_party/github.com/coreos/go-systemd/dbus/properties.go new file mode 100644 index 00000000000..36acf866013 --- /dev/null +++ b/third_party/github.com/coreos/go-systemd/dbus/properties.go @@ -0,0 +1,193 @@ +package dbus + +import ( + "github.com/guelfey/go.dbus" +) + +// From the systemd docs: +// +// The properties array of StartTransientUnit() may take many of the settings +// that may also be configured in unit files. Not all parameters are currently +// accepted though, but we plan to cover more properties with future release. +// Currently you may set the Description, Slice and all dependency types of +// units, as well as RemainAfterExit, ExecStart for service units, +// TimeoutStopUSec and PIDs for scope units, and CPUAccounting, CPUShares, +// BlockIOAccounting, BlockIOWeight, BlockIOReadBandwidth, +// BlockIOWriteBandwidth, BlockIODeviceWeight, MemoryAccounting, MemoryLimit, +// DevicePolicy, DeviceAllow for services/scopes/slices. These fields map +// directly to their counterparts in unit files and as normal D-Bus object +// properties. The exception here is the PIDs field of scope units which is +// used for construction of the scope only and specifies the initial PIDs to +// add to the scope object. + +type Property property + +type property struct { + Name string + Value dbus.Variant +} + +type execStart struct { + Path string // the binary path to execute + Args []string // an array with all arguments to pass to the executed command, starting with argument 0 + UncleanIsFailure bool // a boolean whether it should be considered a failure if the process exits uncleanly +} + +// PropExecStart sets the ExecStart service property. The first argument is a +// slice with the binary path to execute followed by the arguments to pass to +// the executed command. See +// http://www.freedesktop.org/software/systemd/man/systemd.service.html#ExecStart= +func PropExecStart(command []string, uncleanIsFailure bool) Property { + return Property( + property{ + Name: "ExecStart", + Value: dbus.MakeVariant( + []execStart{ + execStart{ + Path: command[0], + Args: command, + UncleanIsFailure: uncleanIsFailure, + }})}) +} + +// PropRemainAfterExit sets the RemainAfterExit service property. See +// http://www.freedesktop.org/software/systemd/man/systemd.service.html#RemainAfterExit= +func PropRemainAfterExit(b bool) Property { + return Property( + property{ + Name: "RemainAfterExit", + Value: dbus.MakeVariant(b), + }) +} + +// PropDescription sets the Description unit property. See +// http://www.freedesktop.org/software/systemd/man/systemd.unit#Description= +func PropDescription(desc string) Property { + return Property( + property{ + Name: "Description", + Value: dbus.MakeVariant(desc), + }) +} + +func propDependency(name string, units []string) Property { + return Property( + property{ + Name: name, + Value: dbus.MakeVariant(units), + }) +} + +// PropRequires sets the Requires unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#Requires= +func PropRequires(units ...string) Property { + return propDependency("Requires", units) +} + +// PropRequiresOverridable sets the RequiresOverridable unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#RequiresOverridable= +func PropRequiresOverridable(units ...string) Property { + return propDependency("RequiresOverridable", units) +} + +// PropRequisite sets the Requisite unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#Requisite= +func PropRequisite(units ...string) Property { + return propDependency("Requisite", units) +} + +// PropRequisiteOverridable sets the RequisiteOverridable unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#RequisiteOverridable= +func PropRequisiteOverridable(units ...string) Property { + return propDependency("RequisiteOverridable", units) +} + +// PropWants sets the Wants unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#Wants= +func PropWants(units ...string) Property { + return propDependency("Wants", units) +} + +// PropBindsTo sets the BindsTo unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#BindsTo= +func PropBindsTo(units ...string) Property { + return propDependency("BindsTo", units) +} + +// PropRequiredBy sets the RequiredBy unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#RequiredBy= +func PropRequiredBy(units ...string) Property { + return propDependency("RequiredBy", units) +} + +// PropRequiredByOverridable sets the RequiredByOverridable unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#RequiredByOverridable= +func PropRequiredByOverridable(units ...string) Property { + return propDependency("RequiredByOverridable", units) +} + +// PropWantedBy sets the WantedBy unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#WantedBy= +func PropWantedBy(units ...string) Property { + return propDependency("WantedBy", units) +} + +// PropBoundBy sets the BoundBy unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#BoundBy= +func PropBoundBy(units ...string) Property { + return propDependency("BoundBy", units) +} + +// PropConflicts sets the Conflicts unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#Conflicts= +func PropConflicts(units ...string) Property { + return propDependency("Conflicts", units) +} + +// PropConflictedBy sets the ConflictedBy unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#ConflictedBy= +func PropConflictedBy(units ...string) Property { + return propDependency("ConflictedBy", units) +} + +// PropBefore sets the Before unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#Before= +func PropBefore(units ...string) Property { + return propDependency("Before", units) +} + +// PropAfter sets the After unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#After= +func PropAfter(units ...string) Property { + return propDependency("After", units) +} + +// PropOnFailure sets the OnFailure unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#OnFailure= +func PropOnFailure(units ...string) Property { + return propDependency("OnFailure", units) +} + +// PropTriggers sets the Triggers unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#Triggers= +func PropTriggers(units ...string) Property { + return propDependency("Triggers", units) +} + +// PropTriggeredBy sets the TriggeredBy unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#TriggeredBy= +func PropTriggeredBy(units ...string) Property { + return propDependency("TriggeredBy", units) +} + +// PropPropagatesReloadTo sets the PropagatesReloadTo unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#PropagatesReloadTo= +func PropPropagatesReloadTo(units ...string) Property { + return propDependency("PropagatesReloadTo", units) +} + +// PropRequiresMountsFor sets the RequiresMountsFor unit property. See +// http://www.freedesktop.org/software/systemd/main/systemd.unit.html#RequiresMountsFor= +func PropRequiresMountsFor(units ...string) Property { + return propDependency("RequiresMountsFor", units) +} diff --git a/third_party/github.com/coreos/go-systemd/dbus/subscription.go b/third_party/github.com/coreos/go-systemd/dbus/subscription.go new file mode 100644 index 00000000000..606a5736d7c --- /dev/null +++ b/third_party/github.com/coreos/go-systemd/dbus/subscription.go @@ -0,0 +1,182 @@ +package dbus + +import ( + "github.com/guelfey/go.dbus" + "time" +) + +const ( + cleanIgnoreInterval = int64(10 * time.Second) + ignoreInterval = int64(30 * time.Millisecond) +) + +func (c *Conn) initSubscription() { + c.subscriber.ignore = make(map[dbus.ObjectPath]int64) +} + +// Returns two unbuffered channels which will receive all changed units every +// @interval@ seconds. Deleted units are sent as nil. +func (c *Conn) SubscribeUnits(interval time.Duration) (<-chan map[string]*UnitStatus, <-chan error) { + return c.SubscribeUnitsCustom(interval, 0, func(u1, u2 *UnitStatus) bool { return *u1 != *u2 }) +} + +// SubscribeUnitsCustom is like SubscribeUnits but lets you specify the buffer +// size of the channels and the comparison function for detecting changes. +func (c *Conn) SubscribeUnitsCustom(interval time.Duration, buffer int, isChanged func(*UnitStatus, *UnitStatus) bool) (<-chan map[string]*UnitStatus, <-chan error) { + old := make(map[string]*UnitStatus) + statusChan := make(chan map[string]*UnitStatus, buffer) + errChan := make(chan error, buffer) + + go func() { + for { + timerChan := time.After(interval) + + units, err := c.ListUnits() + if err == nil { + cur := make(map[string]*UnitStatus) + for i := range units { + cur[units[i].Name] = &units[i] + } + + // add all new or changed units + changed := make(map[string]*UnitStatus) + for n, u := range cur { + if oldU, ok := old[n]; !ok || isChanged(oldU, u) { + changed[n] = u + } + delete(old, n) + } + + // add all deleted units + for oldN := range old { + changed[oldN] = nil + } + + old = cur + + statusChan <- changed + } else { + errChan <- err + } + + <-timerChan + } + }() + + return statusChan, errChan +} + +type SubStateUpdate struct { + UnitName string + SubState string +} + +type Error string + +func (e Error) Error() string { + return string(e) +} + +// SetSubStateSubscriber writes to updateCh when any unit's substate changes. +// Althrough this writes to updateCh on every state change, the reported state +// may be more recent than the change that generated it (due to an unavoidable +// race in the systemd dbus interface). That is, this method provides a good +// way to keep a current view of all units' states, but is not guaranteed to +// show every state transition they go through. Furthermore, state changes +// will only be written to the channel with non-blocking writes. If updateCh +// is full, it attempts to write an error to errCh; if errCh is full, the error +// passes silently. +func (c *Conn) SetSubStateSubscriber(updateCh chan<- *SubStateUpdate, errCh chan<- error) { + c.subscriber.Lock() + defer c.subscriber.Unlock() + c.subscriber.updateCh = updateCh + c.subscriber.errCh = errCh +} + +func (c *Conn) sendSubStateUpdate(path dbus.ObjectPath) { + c.subscriber.Lock() + defer c.subscriber.Unlock() + if c.subscriber.updateCh == nil { + return + } + + if c.shouldIgnore(path) { + return + } + + info, err := c.getUnitInfo(path) + if err != nil { + select { + case c.subscriber.errCh <- err: + default: + } + } + + name := info["Id"].Value().(string) + substate := info["SubState"].Value().(string) + + update := &SubStateUpdate{name, substate} + select { + case c.subscriber.updateCh <- update: + default: + select { + case c.subscriber.errCh <- Error("update channel full!"): + default: + } + } + + c.updateIgnore(path, info) +} + +func (c *Conn) getUnitInfo(path dbus.ObjectPath) (map[string]dbus.Variant, error) { + var err error + var props map[string]dbus.Variant + obj := c.sysconn.Object("org.freedesktop.systemd1", path) + err = obj.Call("GetAll", 0, "org.freedesktop.systemd1.Unit").Store(&props) + if err != nil { + return nil, err + } + return props, nil +} + +// The ignore functions work around a wart in the systemd dbus interface. +// Requesting the properties of an unloaded unit will cause systemd to send a +// pair of UnitNew/UnitRemoved signals. Because we need to get a unit's +// properties on UnitNew (as that's the only indication of a new unit coming up +// for the first time), we would enter an infinite loop if we did not attempt +// to detect and ignore these spurious signals. The signal themselves are +// indistinguishable from relevant ones, so we (somewhat hackishly) ignore an +// unloaded unit's signals for a short time after requesting its properties. +// This means that we will miss e.g. a transient unit being restarted +// *immediately* upon failure and also a transient unit being started +// immediately after requesting its status (with systemctl status, for example, +// because this causes a UnitNew signal to be sent which then causes us to fetch +// the properties). + +func (c *Conn) shouldIgnore(path dbus.ObjectPath) bool { + t, ok := c.subscriber.ignore[path] + return ok && t >= time.Now().UnixNano() +} + +func (c *Conn) updateIgnore(path dbus.ObjectPath, info map[string]dbus.Variant) { + c.cleanIgnore() + + // unit is unloaded - it will trigger bad systemd dbus behavior + if info["LoadState"].Value().(string) == "not-found" { + c.subscriber.ignore[path] = time.Now().UnixNano() + ignoreInterval + } +} + +// without this, ignore would grow unboundedly over time +func (c *Conn) cleanIgnore() { + now := time.Now().UnixNano() + if c.subscriber.cleanIgnore < now { + c.subscriber.cleanIgnore = now + cleanIgnoreInterval + + for p, t := range c.subscriber.ignore { + if t < now { + delete(c.subscriber.ignore, p) + } + } + } +} diff --git a/third_party/github.com/coreos/go-systemd/journal/send.go b/third_party/github.com/coreos/go-systemd/journal/send.go index 89bcab164d7..e288bb0395a 100644 --- a/third_party/github.com/coreos/go-systemd/journal/send.go +++ b/third_party/github.com/coreos/go-systemd/journal/send.go @@ -32,7 +32,11 @@ const ( var conn net.Conn func init() { - conn, _ = net.Dial("unixgram", "/run/systemd/journal/socket") + var err error + conn, err = net.Dial("unixgram", "/run/systemd/journal/socket") + if err != nil { + conn = nil + } } // Enabled returns true iff the systemd journal is available for logging From e1003a86234df5c88556ce21c3a69ad8eae51299 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 14 Oct 2013 10:05:46 -0600 Subject: [PATCH 101/247] bump(bitbucket.org/kardianos/osext): 364fb577de68 --- .../kardianos/osext/osext_sysctl.go | 40 ++++++++++++++----- 1 file changed, 29 insertions(+), 11 deletions(-) diff --git a/third_party/bitbucket.org/kardianos/osext/osext_sysctl.go b/third_party/bitbucket.org/kardianos/osext/osext_sysctl.go index d7646462862..e4d228ed1ec 100644 --- a/third_party/bitbucket.org/kardianos/osext/osext_sysctl.go +++ b/third_party/bitbucket.org/kardianos/osext/osext_sysctl.go @@ -8,6 +8,7 @@ package osext import ( "os" + "path/filepath" "runtime" "syscall" "unsafe" @@ -47,18 +48,35 @@ func executable() (string, error) { break } } + var strpath string if buf[0] != '/' { - if getwdError != nil { - return string(buf), getwdError - } else { - if buf[0] == '.' { - buf = buf[1:] - } - if startUpcwd[len(startUpcwd)-1] != '/' { - return startUpcwd + "/" + string(buf), nil - } - return startUpcwd + string(buf), nil + var e error + if strpath, e = getAbs(buf); e != nil { + return strpath, e } + } else { + strpath = string(buf) + } + // darwin KERN_PROCARGS may return the path to a symlink rather than the + // actual executable + if runtime.GOOS == "darwin" { + if strpath, err := filepath.EvalSymlinks(strpath); err != nil { + return strpath, err + } + } + return strpath, nil +} + +func getAbs(buf []byte) (string, error) { + if getwdError != nil { + return string(buf), getwdError + } else { + if buf[0] == '.' { + buf = buf[1:] + } + if startUpcwd[len(startUpcwd)-1] != '/' && buf[0] != '/' { + return startUpcwd + "/" + string(buf), nil + } + return startUpcwd + string(buf), nil } - return string(buf), nil } From 56192b7f8547111a77ad4fcf973341174d0d76c4 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 14 Oct 2013 10:05:49 -0600 Subject: [PATCH 102/247] bump(code.google.com/p/go.net): 355ff0aa1b2e --- .../code.google.com/p/go.net/ipv4/gen.go | 46 ++++---- .../code.google.com/p/go.net/ipv4/gentest.go | 47 ++++---- .../code.google.com/p/go.net/ipv4/header.go | 59 ++++++---- .../p/go.net/ipv4/header_test.go | 9 +- .../p/go.net/ipv4/iana_test.go | 4 +- .../code.google.com/p/go.net/ipv6/gen.go | 46 ++++---- .../code.google.com/p/go.net/ipv6/gentest.go | 47 ++++---- .../code.google.com/p/go.net/ipv6/iana.go | 4 +- .../p/go.net/ipv6/iana_test.go | 4 +- .../p/go.net/websocket/client.go | 18 +-- .../p/go.net/websocket/hybi.go | 26 +---- .../p/go.net/websocket/hybi_test.go | 108 ------------------ .../p/go.net/websocket/server.go | 8 -- .../p/go.net/websocket/websocket.go | 6 +- .../p/go.net/websocket/websocket_test.go | 14 +++ 15 files changed, 153 insertions(+), 293 deletions(-) diff --git a/third_party/code.google.com/p/go.net/ipv4/gen.go b/third_party/code.google.com/p/go.net/ipv4/gen.go index d2ed17d7da8..9a2af1e6801 100644 --- a/third_party/code.google.com/p/go.net/ipv4/gen.go +++ b/third_party/code.google.com/p/go.net/ipv4/gen.go @@ -97,20 +97,16 @@ func parseICMPv4Parameters(w io.Writer, r io.Reader) error { } type icmpv4Parameters struct { - XMLName xml.Name `xml:"registry"` - Title string `xml:"title"` - Updated string `xml:"updated"` - Registries []icmpv4ParamRegistry `xml:"registry"` -} - -type icmpv4ParamRegistry struct { - Title string `xml:"title"` - Records []icmpv4ParamRecord `xml:"record"` -} - -type icmpv4ParamRecord struct { - Value string `xml:"value"` - Descr string `xml:"description"` + XMLName xml.Name `xml:"registry"` + Title string `xml:"title"` + Updated string `xml:"updated"` + Registries []struct { + Title string `xml:"title"` + Records []struct { + Value string `xml:"value"` + Descr string `xml:"description"` + } `xml:"record"` + } `xml:"registry"` } type canonICMPv4ParamRecord struct { @@ -193,18 +189,16 @@ func parseProtocolNumbers(w io.Writer, r io.Reader) error { } type protocolNumbers struct { - XMLName xml.Name `xml:"registry"` - Title string `xml:"title"` - Updated string `xml:"updated"` - RegTitle string `xml:"registry>title"` - Note string `xml:"registry>note"` - Records []protocolRecord `xml:"registry>record"` -} - -type protocolRecord struct { - Value string `xml:"value"` - Name string `xml:"name"` - Descr string `xml:"description"` + XMLName xml.Name `xml:"registry"` + Title string `xml:"title"` + Updated string `xml:"updated"` + RegTitle string `xml:"registry>title"` + Note string `xml:"registry>note"` + Records []struct { + Value string `xml:"value"` + Name string `xml:"name"` + Descr string `xml:"description"` + } `xml:"registry>record"` } type canonProtocolRecord struct { diff --git a/third_party/code.google.com/p/go.net/ipv4/gentest.go b/third_party/code.google.com/p/go.net/ipv4/gentest.go index cc35225cc9d..3b21d7a6b37 100644 --- a/third_party/code.google.com/p/go.net/ipv4/gentest.go +++ b/third_party/code.google.com/p/go.net/ipv4/gentest.go @@ -39,7 +39,7 @@ var registries = []struct { func main() { var bb bytes.Buffer - fmt.Fprintf(&bb, "// go run gentv.go\n") + fmt.Fprintf(&bb, "// go run gentest.go\n") fmt.Fprintf(&bb, "// GENERATED BY THE COMMAND ABOVE; DO NOT EDIT\n\n") fmt.Fprintf(&bb, "package ipv4_test\n\n") for _, r := range registries { @@ -85,18 +85,19 @@ func parseDSCPRegistry(w io.Writer, r io.Reader) error { } type dscpRegistry struct { - XMLName xml.Name `xml:"registry"` - Title string `xml:"title"` - Updated string `xml:"updated"` - Note string `xml:"note"` - RegTitle string `xml:"registry>title"` - PoolRecords []dscpRecord `xml:"registry>record"` - Records []dscpRecord `xml:"registry>registry>record"` -} - -type dscpRecord struct { - Name string `xml:"name"` - Space string `xml:"space"` + XMLName xml.Name `xml:"registry"` + Title string `xml:"title"` + Updated string `xml:"updated"` + Note string `xml:"note"` + RegTitle string `xml:"registry>title"` + PoolRecords []struct { + Name string `xml:"name"` + Space string `xml:"space"` + } `xml:"registry>record"` + Records []struct { + Name string `xml:"name"` + Space string `xml:"space"` + } `xml:"registry>registry>record"` } type canonDSCPRecord struct { @@ -145,17 +146,15 @@ func parseTOSTCByte(w io.Writer, r io.Reader) error { } type tosTCByte struct { - XMLName xml.Name `xml:"registry"` - Title string `xml:"title"` - Updated string `xml:"updated"` - Note string `xml:"note"` - RegTitle string `xml:"registry>title"` - Records []tosTCByteRecord `xml:"registry>record"` -} - -type tosTCByteRecord struct { - Binary string `xml:"binary"` - Keyword string `xml:"keyword"` + XMLName xml.Name `xml:"registry"` + Title string `xml:"title"` + Updated string `xml:"updated"` + Note string `xml:"note"` + RegTitle string `xml:"registry>title"` + Records []struct { + Binary string `xml:"binary"` + Keyword string `xml:"keyword"` + } `xml:"registry>record"` } type canonTOSTCByteRecord struct { diff --git a/third_party/code.google.com/p/go.net/ipv4/header.go b/third_party/code.google.com/p/go.net/ipv4/header.go index 45bd27a174a..e15db01d5e9 100644 --- a/third_party/code.google.com/p/go.net/ipv4/header.go +++ b/third_party/code.google.com/p/go.net/ipv4/header.go @@ -36,41 +36,47 @@ const ( maxHeaderLen = 60 // sensible default, revisit if later RFCs define new usage of version and header length fields ) -type headerField int +const ( + posTOS = 1 // type-of-service + posTotalLen = 2 // packet total length + posID = 4 // identification + posFragOff = 6 // fragment offset + posTTL = 8 // time-to-live + posProtocol = 9 // next protocol + posChecksum = 10 // checksum + posSrc = 12 // source address + posDst = 16 // destination address +) + +type HeaderFlags int const ( - posTOS headerField = 1 // type-of-service - posTotalLen = 2 // packet total length - posID = 4 // identification - posFragOff = 6 // fragment offset - posTTL = 8 // time-to-live - posProtocol = 9 // next protocol - posChecksum = 10 // checksum - posSrc = 12 // source address - posDst = 16 // destination address + MoreFragments HeaderFlags = 1 << iota // more fragments flag + DontFragment // don't fragment flag ) // A Header represents an IPv4 header. type Header struct { - Version int // protocol version - Len int // header length - TOS int // type-of-service - TotalLen int // packet total length - ID int // identification - FragOff int // fragment offset - TTL int // time-to-live - Protocol int // next protocol - Checksum int // checksum - Src net.IP // source address - Dst net.IP // destination address - Options []byte // options, extension headers + Version int // protocol version + Len int // header length + TOS int // type-of-service + TotalLen int // packet total length + ID int // identification + Flags HeaderFlags // flags + FragOff int // fragment offset + TTL int // time-to-live + Protocol int // next protocol + Checksum int // checksum + Src net.IP // source address + Dst net.IP // destination address + Options []byte // options, extension headers } func (h *Header) String() string { if h == nil { return "" } - return fmt.Sprintf("ver: %v, hdrlen: %v, tos: %#x, totallen: %v, id: %#x, fragoff: %#x, ttl: %v, proto: %v, cksum: %#x, src: %v, dst: %v", h.Version, h.Len, h.TOS, h.TotalLen, h.ID, h.FragOff, h.TTL, h.Protocol, h.Checksum, h.Src, h.Dst) + return fmt.Sprintf("ver: %v, hdrlen: %v, tos: %#x, totallen: %v, id: %#x, flags: %#x, fragoff: %#x, ttl: %v, proto: %v, cksum: %#x, src: %v, dst: %v", h.Version, h.Len, h.TOS, h.TotalLen, h.ID, h.Flags, h.FragOff, h.TTL, h.Protocol, h.Checksum, h.Src, h.Dst) } // Please refer to the online manual; IP(4) on Darwin, FreeBSD and @@ -89,12 +95,13 @@ func (h *Header) Marshal() ([]byte, error) { b := make([]byte, hdrlen) b[0] = byte(Version<<4 | (hdrlen >> 2 & 0x0f)) b[posTOS] = byte(h.TOS) + flagsAndFragOff := (h.FragOff & 0x1fff) | int(h.Flags<<13) if supportsNewIPInput { b[posTotalLen], b[posTotalLen+1] = byte(h.TotalLen>>8), byte(h.TotalLen) - b[posFragOff], b[posFragOff+1] = byte(h.FragOff>>8), byte(h.FragOff) + b[posFragOff], b[posFragOff+1] = byte(flagsAndFragOff>>8), byte(flagsAndFragOff) } else { *(*uint16)(unsafe.Pointer(&b[posTotalLen : posTotalLen+1][0])) = uint16(h.TotalLen) - *(*uint16)(unsafe.Pointer(&b[posFragOff : posFragOff+1][0])) = uint16(h.FragOff) + *(*uint16)(unsafe.Pointer(&b[posFragOff : posFragOff+1][0])) = uint16(flagsAndFragOff) } b[posID], b[posID+1] = byte(h.ID>>8), byte(h.ID) b[posTTL] = byte(h.TTL) @@ -135,6 +142,8 @@ func ParseHeader(b []byte) (*Header, error) { h.TotalLen += hdrlen h.FragOff = int(*(*uint16)(unsafe.Pointer(&b[posFragOff : posFragOff+1][0]))) } + h.Flags = HeaderFlags(h.FragOff&0xe000) >> 13 + h.FragOff = h.FragOff & 0x1fff h.ID = int(b[posID])<<8 | int(b[posID+1]) h.TTL = int(b[posTTL]) h.Protocol = int(b[posProtocol]) diff --git a/third_party/code.google.com/p/go.net/ipv4/header_test.go b/third_party/code.google.com/p/go.net/ipv4/header_test.go index cc2fd8e09b7..121d1f20dc0 100644 --- a/third_party/code.google.com/p/go.net/ipv4/header_test.go +++ b/third_party/code.google.com/p/go.net/ipv4/header_test.go @@ -16,28 +16,28 @@ import ( var ( wireHeaderFromKernel = [ipv4.HeaderLen]byte{ 0x45, 0x01, 0xbe, 0xef, - 0xca, 0xfe, 0x05, 0xdc, + 0xca, 0xfe, 0x45, 0xdc, 0xff, 0x01, 0xde, 0xad, 172, 16, 254, 254, 192, 168, 0, 1, } wireHeaderToKernel = [ipv4.HeaderLen]byte{ 0x45, 0x01, 0xbe, 0xef, - 0xca, 0xfe, 0x05, 0xdc, + 0xca, 0xfe, 0x45, 0xdc, 0xff, 0x01, 0xde, 0xad, 172, 16, 254, 254, 192, 168, 0, 1, } wireHeaderFromTradBSDKernel = [ipv4.HeaderLen]byte{ 0x45, 0x01, 0xdb, 0xbe, - 0xca, 0xfe, 0xdc, 0x05, + 0xca, 0xfe, 0xdc, 0x45, 0xff, 0x01, 0xde, 0xad, 172, 16, 254, 254, 192, 168, 0, 1, } wireHeaderToTradBSDKernel = [ipv4.HeaderLen]byte{ 0x45, 0x01, 0xef, 0xbe, - 0xca, 0xfe, 0xdc, 0x05, + 0xca, 0xfe, 0xdc, 0x45, 0xff, 0x01, 0xde, 0xad, 172, 16, 254, 254, 192, 168, 0, 1, @@ -51,6 +51,7 @@ var ( TOS: 1, TotalLen: 0xbeef, ID: 0xcafe, + Flags: ipv4.DontFragment, FragOff: 1500, TTL: 255, Protocol: 1, diff --git a/third_party/code.google.com/p/go.net/ipv4/iana_test.go b/third_party/code.google.com/p/go.net/ipv4/iana_test.go index 276230d627c..8f9e7bae877 100644 --- a/third_party/code.google.com/p/go.net/ipv4/iana_test.go +++ b/third_party/code.google.com/p/go.net/ipv4/iana_test.go @@ -1,9 +1,9 @@ -// go run gentv.go +// go run gentest.go // GENERATED BY THE COMMAND ABOVE; DO NOT EDIT package ipv4_test -// Differentiated Services Field Codepoints, Updated: 2010-05-11 +// Differentiated Services Field Codepoints (DSCP), Updated: 2013-06-25 const ( DiffServCS0 = 0x0 // CS0 DiffServCS1 = 0x20 // CS1 diff --git a/third_party/code.google.com/p/go.net/ipv6/gen.go b/third_party/code.google.com/p/go.net/ipv6/gen.go index 36517b2461f..5680bac761d 100644 --- a/third_party/code.google.com/p/go.net/ipv6/gen.go +++ b/third_party/code.google.com/p/go.net/ipv6/gen.go @@ -97,20 +97,16 @@ func parseICMPv6Parameters(w io.Writer, r io.Reader) error { } type icmpv6Parameters struct { - XMLName xml.Name `xml:"registry"` - Title string `xml:"title"` - Updated string `xml:"updated"` - Registries []icmpv6ParamRegistry `xml:"registry"` -} - -type icmpv6ParamRegistry struct { - Title string `xml:"title"` - Records []icmpv6ParamRecord `xml:"record"` -} - -type icmpv6ParamRecord struct { - Value string `xml:"value"` - Name string `xml:"name"` + XMLName xml.Name `xml:"registry"` + Title string `xml:"title"` + Updated string `xml:"updated"` + Registries []struct { + Title string `xml:"title"` + Records []struct { + Value string `xml:"value"` + Name string `xml:"name"` + } `xml:"record"` + } `xml:"registry"` } type canonICMPv6ParamRecord struct { @@ -188,18 +184,16 @@ func parseProtocolNumbers(w io.Writer, r io.Reader) error { } type protocolNumbers struct { - XMLName xml.Name `xml:"registry"` - Title string `xml:"title"` - Updated string `xml:"updated"` - RegTitle string `xml:"registry>title"` - Note string `xml:"registry>note"` - Records []protocolRecord `xml:"registry>record"` -} - -type protocolRecord struct { - Value string `xml:"value"` - Name string `xml:"name"` - Descr string `xml:"description"` + XMLName xml.Name `xml:"registry"` + Title string `xml:"title"` + Updated string `xml:"updated"` + RegTitle string `xml:"registry>title"` + Note string `xml:"registry>note"` + Records []struct { + Value string `xml:"value"` + Name string `xml:"name"` + Descr string `xml:"description"` + } `xml:"registry>record"` } type canonProtocolRecord struct { diff --git a/third_party/code.google.com/p/go.net/ipv6/gentest.go b/third_party/code.google.com/p/go.net/ipv6/gentest.go index fddbfd1efe2..606930f63df 100644 --- a/third_party/code.google.com/p/go.net/ipv6/gentest.go +++ b/third_party/code.google.com/p/go.net/ipv6/gentest.go @@ -39,7 +39,7 @@ var registries = []struct { func main() { var bb bytes.Buffer - fmt.Fprintf(&bb, "// go run gentv.go\n") + fmt.Fprintf(&bb, "// go run gentest.go\n") fmt.Fprintf(&bb, "// GENERATED BY THE COMMAND ABOVE; DO NOT EDIT\n\n") fmt.Fprintf(&bb, "package ipv6_test\n\n") for _, r := range registries { @@ -85,18 +85,19 @@ func parseDSCPRegistry(w io.Writer, r io.Reader) error { } type dscpRegistry struct { - XMLName xml.Name `xml:"registry"` - Title string `xml:"title"` - Updated string `xml:"updated"` - Note string `xml:"note"` - RegTitle string `xml:"registry>title"` - PoolRecords []dscpRecord `xml:"registry>record"` - Records []dscpRecord `xml:"registry>registry>record"` -} - -type dscpRecord struct { - Name string `xml:"name"` - Space string `xml:"space"` + XMLName xml.Name `xml:"registry"` + Title string `xml:"title"` + Updated string `xml:"updated"` + Note string `xml:"note"` + RegTitle string `xml:"registry>title"` + PoolRecords []struct { + Name string `xml:"name"` + Space string `xml:"space"` + } `xml:"registry>record"` + Records []struct { + Name string `xml:"name"` + Space string `xml:"space"` + } `xml:"registry>registry>record"` } type canonDSCPRecord struct { @@ -145,17 +146,15 @@ func parseTOSTCByte(w io.Writer, r io.Reader) error { } type tosTCByte struct { - XMLName xml.Name `xml:"registry"` - Title string `xml:"title"` - Updated string `xml:"updated"` - Note string `xml:"note"` - RegTitle string `xml:"registry>title"` - Records []tosTCByteRecord `xml:"registry>record"` -} - -type tosTCByteRecord struct { - Binary string `xml:"binary"` - Keyword string `xml:"keyword"` + XMLName xml.Name `xml:"registry"` + Title string `xml:"title"` + Updated string `xml:"updated"` + Note string `xml:"note"` + RegTitle string `xml:"registry>title"` + Records []struct { + Binary string `xml:"binary"` + Keyword string `xml:"keyword"` + } `xml:"registry>record"` } type canonTOSTCByteRecord struct { diff --git a/third_party/code.google.com/p/go.net/ipv6/iana.go b/third_party/code.google.com/p/go.net/ipv6/iana.go index c888cf2564d..429087c5fdb 100644 --- a/third_party/code.google.com/p/go.net/ipv6/iana.go +++ b/third_party/code.google.com/p/go.net/ipv6/iana.go @@ -3,7 +3,7 @@ package ipv6 -// Internet Control Message Protocol version 6 (ICMPv6) Parameters, Updated: 2012-11-12 +// Internet Control Message Protocol version 6 (ICMPv6) Parameters, Updated: 2013-07-03 const ( ICMPTypeDestinationUnreachable ICMPType = 1 // Destination Unreachable ICMPTypePacketTooBig ICMPType = 2 // Packet Too Big @@ -41,7 +41,7 @@ const ( ICMPTypeDuplicateAddressConfirmation ICMPType = 158 // Duplicate Address Confirmation ) -// Internet Control Message Protocol version 6 (ICMPv6) Parameters, Updated: 2012-11-12 +// Internet Control Message Protocol version 6 (ICMPv6) Parameters, Updated: 2013-07-03 var icmpTypes = map[ICMPType]string{ 1: "destination unreachable", 2: "packet too big", diff --git a/third_party/code.google.com/p/go.net/ipv6/iana_test.go b/third_party/code.google.com/p/go.net/ipv6/iana_test.go index 7b6bb85ef16..03e8bfe5ac9 100644 --- a/third_party/code.google.com/p/go.net/ipv6/iana_test.go +++ b/third_party/code.google.com/p/go.net/ipv6/iana_test.go @@ -1,9 +1,9 @@ -// go run gentv.go +// go run gentest.go // GENERATED BY THE COMMAND ABOVE; DO NOT EDIT package ipv6_test -// Differentiated Services Field Codepoints, Updated: 2010-05-11 +// Differentiated Services Field Codepoints (DSCP), Updated: 2013-06-25 const ( DiffServCS0 = 0x0 // CS0 DiffServCS1 = 0x20 // CS1 diff --git a/third_party/code.google.com/p/go.net/websocket/client.go b/third_party/code.google.com/p/go.net/websocket/client.go index df54a688101..a861bb92c6c 100644 --- a/third_party/code.google.com/p/go.net/websocket/client.go +++ b/third_party/code.google.com/p/go.net/websocket/client.go @@ -43,26 +43,12 @@ func NewConfig(server, origin string) (config *Config, err error) { func NewClient(config *Config, rwc io.ReadWriteCloser) (ws *Conn, err error) { br := bufio.NewReader(rwc) bw := bufio.NewWriter(rwc) - switch config.Version { - case ProtocolVersionHixie75: - err = hixie75ClientHandshake(config, br, bw) - case ProtocolVersionHixie76, ProtocolVersionHybi00: - err = hixie76ClientHandshake(config, br, bw) - case ProtocolVersionHybi08, ProtocolVersionHybi13: - err = hybiClientHandshake(config, br, bw) - default: - err = ErrBadProtocolVersion - } + err = hybiClientHandshake(config, br, bw) if err != nil { return } buf := bufio.NewReadWriter(br, bw) - switch config.Version { - case ProtocolVersionHixie75, ProtocolVersionHixie76, ProtocolVersionHybi00: - ws = newHixieClientConn(config, buf, rwc) - case ProtocolVersionHybi08, ProtocolVersionHybi13: - ws = newHybiClientConn(config, buf, rwc) - } + ws = newHybiClientConn(config, buf, rwc) return } diff --git a/third_party/code.google.com/p/go.net/websocket/hybi.go b/third_party/code.google.com/p/go.net/websocket/hybi.go index 90f5d9ca01b..f8c0b2e2994 100644 --- a/third_party/code.google.com/p/go.net/websocket/hybi.go +++ b/third_party/code.google.com/p/go.net/websocket/hybi.go @@ -385,21 +385,8 @@ func getNonceAccept(nonce []byte) (expected []byte, err error) { return } -func isHybiVersion(version int) bool { - switch version { - case ProtocolVersionHybi08, ProtocolVersionHybi13: - return true - default: - } - return false -} - // Client handshake described in draft-ietf-hybi-thewebsocket-protocol-17 func hybiClientHandshake(config *Config, br *bufio.Reader, bw *bufio.Writer) (err error) { - if !isHybiVersion(config.Version) { - panic("wrong protocol version.") - } - bw.WriteString("GET " + config.Location.RequestURI() + " HTTP/1.1\r\n") bw.WriteString("Host: " + config.Location.Host + "\r\n") @@ -410,11 +397,12 @@ func hybiClientHandshake(config *Config, br *bufio.Reader, bw *bufio.Writer) (er nonce = []byte(config.handshakeData["key"]) } bw.WriteString("Sec-WebSocket-Key: " + string(nonce) + "\r\n") - if config.Version == ProtocolVersionHybi13 { - bw.WriteString("Origin: " + strings.ToLower(config.Origin.String()) + "\r\n") - } else if config.Version == ProtocolVersionHybi08 { - bw.WriteString("Sec-WebSocket-Origin: " + strings.ToLower(config.Origin.String()) + "\r\n") + bw.WriteString("Origin: " + strings.ToLower(config.Origin.String()) + "\r\n") + + if config.Version != ProtocolVersionHybi13 { + return ErrBadProtocolVersion } + bw.WriteString("Sec-WebSocket-Version: " + fmt.Sprintf("%d", config.Version) + "\r\n") if len(config.Protocol) > 0 { bw.WriteString("Sec-WebSocket-Protocol: " + strings.Join(config.Protocol, ", ") + "\r\n") @@ -500,8 +488,6 @@ func (c *hybiServerHandshaker) ReadHandshake(buf *bufio.Reader, req *http.Reques switch version { case "13": c.Version = ProtocolVersionHybi13 - case "8": - c.Version = ProtocolVersionHybi08 default: return http.StatusBadRequest, ErrBadWebSocketVersion } @@ -536,8 +522,6 @@ func Origin(config *Config, req *http.Request) (*url.URL, error) { switch config.Version { case ProtocolVersionHybi13: origin = req.Header.Get("Origin") - case ProtocolVersionHybi08: - origin = req.Header.Get("Sec-Websocket-Origin") } if origin == "null" { return nil, nil diff --git a/third_party/code.google.com/p/go.net/websocket/hybi_test.go b/third_party/code.google.com/p/go.net/websocket/hybi_test.go index 9db0ef97bb9..d6a19108a6d 100644 --- a/third_party/code.google.com/p/go.net/websocket/hybi_test.go +++ b/third_party/code.google.com/p/go.net/websocket/hybi_test.go @@ -157,68 +157,6 @@ Sec-WebSocket-Protocol: chat } } -func TestHybiClientHandshakeHybi08(t *testing.T) { - b := bytes.NewBuffer([]byte{}) - bw := bufio.NewWriter(b) - br := bufio.NewReader(strings.NewReader(`HTTP/1.1 101 Switching Protocols -Upgrade: websocket -Connection: Upgrade -Sec-WebSocket-Accept: s3pPLMBiTxaQ9kYGzzhZRbK+xOo= -Sec-WebSocket-Protocol: chat - -`)) - var err error - config := new(Config) - config.Location, err = url.ParseRequestURI("ws://server.example.com/chat") - if err != nil { - t.Fatal("location url", err) - } - config.Origin, err = url.ParseRequestURI("http://example.com") - if err != nil { - t.Fatal("origin url", err) - } - config.Protocol = append(config.Protocol, "chat") - config.Protocol = append(config.Protocol, "superchat") - config.Version = ProtocolVersionHybi08 - - config.handshakeData = map[string]string{ - "key": "dGhlIHNhbXBsZSBub25jZQ==", - } - err = hybiClientHandshake(config, br, bw) - if err != nil { - t.Errorf("handshake failed: %v", err) - } - req, err := http.ReadRequest(bufio.NewReader(b)) - if err != nil { - t.Fatalf("read request: %v", err) - } - if req.Method != "GET" { - t.Errorf("request method expected GET, but got %q", req.Method) - } - if req.URL.Path != "/chat" { - t.Errorf("request path expected /demo, but got %q", req.URL.Path) - } - if req.Proto != "HTTP/1.1" { - t.Errorf("request proto expected HTTP/1.1, but got %q", req.Proto) - } - if req.Host != "server.example.com" { - t.Errorf("request Host expected example.com, but got %v", req.Host) - } - var expectedHeader = map[string]string{ - "Connection": "Upgrade", - "Upgrade": "websocket", - "Sec-Websocket-Key": config.handshakeData["key"], - "Sec-Websocket-Origin": config.Origin.String(), - "Sec-Websocket-Protocol": "chat, superchat", - "Sec-Websocket-Version": fmt.Sprintf("%d", ProtocolVersionHybi08), - } - for k, v := range expectedHeader { - if req.Header.Get(k) != v { - t.Errorf(fmt.Sprintf("%s expected %q but got %q", k, v, req.Header.Get(k))) - } - } -} - func TestHybiServerHandshake(t *testing.T) { config := new(Config) handshaker := &hybiServerHandshaker{Config: config} @@ -314,52 +252,6 @@ Sec-WebSocket-Version: 13 } } -func TestHybiServerHandshakeHybi08(t *testing.T) { - config := new(Config) - handshaker := &hybiServerHandshaker{Config: config} - br := bufio.NewReader(strings.NewReader(`GET /chat HTTP/1.1 -Host: server.example.com -Upgrade: websocket -Connection: Upgrade -Sec-WebSocket-Key: dGhlIHNhbXBsZSBub25jZQ== -Sec-WebSocket-Origin: http://example.com -Sec-WebSocket-Protocol: chat, superchat -Sec-WebSocket-Version: 8 - -`)) - req, err := http.ReadRequest(br) - if err != nil { - t.Fatal("request", err) - } - code, err := handshaker.ReadHandshake(br, req) - if err != nil { - t.Errorf("handshake failed: %v", err) - } - if code != http.StatusSwitchingProtocols { - t.Errorf("status expected %q but got %q", http.StatusSwitchingProtocols, code) - } - b := bytes.NewBuffer([]byte{}) - bw := bufio.NewWriter(b) - - config.Protocol = []string{"chat"} - - err = handshaker.AcceptHandshake(bw) - if err != nil { - t.Errorf("handshake response failed: %v", err) - } - expectedResponse := strings.Join([]string{ - "HTTP/1.1 101 Switching Protocols", - "Upgrade: websocket", - "Connection: Upgrade", - "Sec-WebSocket-Accept: s3pPLMBiTxaQ9kYGzzhZRbK+xOo=", - "Sec-WebSocket-Protocol: chat", - "", ""}, "\r\n") - - if b.String() != expectedResponse { - t.Errorf("handshake expected %q but got %q", expectedResponse, b.String()) - } -} - func TestHybiServerHandshakeHybiBadVersion(t *testing.T) { config := new(Config) handshaker := &hybiServerHandshaker{Config: config} diff --git a/third_party/code.google.com/p/go.net/websocket/server.go b/third_party/code.google.com/p/go.net/websocket/server.go index 54e05b43008..70322133c49 100644 --- a/third_party/code.google.com/p/go.net/websocket/server.go +++ b/third_party/code.google.com/p/go.net/websocket/server.go @@ -22,14 +22,6 @@ func newServerConn(rwc io.ReadWriteCloser, buf *bufio.ReadWriter, req *http.Requ buf.Flush() return } - if err != nil { - hs = &hixie76ServerHandshaker{Config: config} - code, err = hs.ReadHandshake(buf.Reader, req) - } - if err != nil { - hs = &hixie75ServerHandshaker{Config: config} - code, err = hs.ReadHandshake(buf.Reader, req) - } if err != nil { fmt.Fprintf(buf, "HTTP/1.1 %03d %s\r\n", code, http.StatusText(code)) buf.WriteString("\r\n") diff --git a/third_party/code.google.com/p/go.net/websocket/websocket.go b/third_party/code.google.com/p/go.net/websocket/websocket.go index 861b3c68f2a..067f5b268b7 100644 --- a/third_party/code.google.com/p/go.net/websocket/websocket.go +++ b/third_party/code.google.com/p/go.net/websocket/websocket.go @@ -21,13 +21,9 @@ import ( ) const ( - ProtocolVersionHixie75 = -75 - ProtocolVersionHixie76 = -76 - ProtocolVersionHybi00 = 0 - ProtocolVersionHybi08 = 8 ProtocolVersionHybi13 = 13 ProtocolVersionHybi = ProtocolVersionHybi13 - SupportedProtocolVersion = "13, 8" + SupportedProtocolVersion = "13" ContinuationFrame = 0 TextFrame = 1 diff --git a/third_party/code.google.com/p/go.net/websocket/websocket_test.go b/third_party/code.google.com/p/go.net/websocket/websocket_test.go index 53e445be380..add89bce025 100644 --- a/third_party/code.google.com/p/go.net/websocket/websocket_test.go +++ b/third_party/code.google.com/p/go.net/websocket/websocket_test.go @@ -286,6 +286,20 @@ func TestTrailingSpaces(t *testing.T) { } } +func TestDialConfigBadVersion(t *testing.T) { + once.Do(startServer) + config := newConfig(t, "/echo") + config.Version = 1234 + + _, err := DialConfig(config) + + if dialerr, ok := err.(*DialError); ok { + if dialerr.Err != ErrBadProtocolVersion { + t.Errorf("dial expected err %q but got %q", ErrBadProtocolVersion, dialerr.Err) + } + } +} + func TestSmallBuffer(t *testing.T) { // http://code.google.com/p/go/issues/detail?id=1145 // Read should be able to handle reading a fragment of a frame. From 1321c63f3b70368ff75b358d01c649172a72f368 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 14 Oct 2013 11:12:30 -0600 Subject: [PATCH 103/247] Extract Store into an interface. --- server/peer_server.go | 4 +-- server/registry.go | 4 +-- server/server.go | 6 ++-- server/v1/v1.go | 2 +- server/v2/v2.go | 2 +- store/create_command.go | 2 +- store/delete_command.go | 2 +- store/node.go | 38 +++++++++++--------- store/stats_test.go | 4 +-- store/store.go | 68 ++++++++++++++++++++++------------- store/store_test.go | 22 ++++++------ store/test_and_set_command.go | 2 +- store/update_command.go | 2 +- store/watcher_test.go | 2 +- 14 files changed, 92 insertions(+), 68 deletions(-) diff --git a/server/peer_server.go b/server/peer_server.go index 942d9e062cf..97a7757a375 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -30,7 +30,7 @@ type PeerServer struct { followersStats *raftFollowersStats serverStats *raftServerStats registry *Registry - store *store.Store + store store.Store snapConf *snapshotConf MaxClusterSize int RetryTimes int @@ -49,7 +49,7 @@ type snapshotConf struct { writesThr uint64 } -func NewPeerServer(name string, path string, url string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, registry *Registry, store *store.Store) *PeerServer { +func NewPeerServer(name string, path string, url string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, registry *Registry, store store.Store) *PeerServer { s := &PeerServer{ name: name, url: url, diff --git a/server/registry.go b/server/registry.go index b9197436bd5..23ef9ddbbdb 100644 --- a/server/registry.go +++ b/server/registry.go @@ -18,7 +18,7 @@ const RegistryKey = "/_etcd/machines" // The Registry stores URL information for nodes. type Registry struct { sync.Mutex - store *store.Store + store store.Store nodes map[string]*node } @@ -30,7 +30,7 @@ type node struct { } // Creates a new Registry. -func NewRegistry(s *store.Store) *Registry { +func NewRegistry(s store.Store) *Registry { return &Registry{ store: s, nodes: make(map[string]*node), diff --git a/server/server.go b/server/server.go index 24064fa4e86..4e21aa17911 100644 --- a/server/server.go +++ b/server/server.go @@ -21,7 +21,7 @@ type Server struct { http.Server peerServer *PeerServer registry *Registry - store *store.Store + store store.Store name string url string tlsConf *TLSConfig @@ -30,7 +30,7 @@ type Server struct { } // Creates a new Server. -func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, peerServer *PeerServer, registry *Registry, store *store.Store) *Server { +func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, peerServer *PeerServer, registry *Registry, store store.Store) *Server { s := &Server{ Server: http.Server{ Handler: mux.NewRouter(), @@ -85,7 +85,7 @@ func (s *Server) PeerURL(name string) (string, bool) { } // Returns a reference to the Store. -func (s *Server) Store() *store.Store { +func (s *Server) Store() store.Store { return s.store } diff --git a/server/v1/v1.go b/server/v1/v1.go index f71ed06220e..6b155b7cc12 100644 --- a/server/v1/v1.go +++ b/server/v1/v1.go @@ -10,6 +10,6 @@ import ( type Server interface { CommitIndex() uint64 Term() uint64 - Store() *store.Store + Store() store.Store Dispatch(raft.Command, http.ResponseWriter, *http.Request) error } diff --git a/server/v2/v2.go b/server/v2/v2.go index e412e859b23..019297331ef 100644 --- a/server/v2/v2.go +++ b/server/v2/v2.go @@ -13,6 +13,6 @@ type Server interface { CommitIndex() uint64 Term() uint64 PeerURL(string) (string, bool) - Store() *store.Store + Store() store.Store Dispatch(raft.Command, http.ResponseWriter, *http.Request) error } diff --git a/store/create_command.go b/store/create_command.go index 2ccddd1035c..f13b9179064 100644 --- a/store/create_command.go +++ b/store/create_command.go @@ -26,7 +26,7 @@ func (c *CreateCommand) CommandName() string { // Create node func (c *CreateCommand) Apply(server *raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(*Store) + s, _ := server.StateMachine().(Store) e, err := s.Create(c.Key, c.Value, c.IncrementalSuffix, c.Force, c.ExpireTime, server.CommitIndex(), server.Term()) diff --git a/store/delete_command.go b/store/delete_command.go index 324410192eb..bc84dfc9970 100644 --- a/store/delete_command.go +++ b/store/delete_command.go @@ -22,7 +22,7 @@ func (c *DeleteCommand) CommandName() string { // Delete the key func (c *DeleteCommand) Apply(server *raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(*Store) + s, _ := server.StateMachine().(Store) e, err := s.Delete(c.Key, c.Recursive, server.CommitIndex(), server.Term()) diff --git a/store/node.go b/store/node.go index 683e6038bf4..bbb971bd46f 100644 --- a/store/node.go +++ b/store/node.go @@ -36,6 +36,9 @@ type Node struct { Value string // for key-value pair Children map[string]*Node // for directory + // A reference to the store this node is attached to. + store *store + // a ttl node will have an expire routine associated with it. // we need a channel to stop that routine when the expiration changes. stopExpire chan bool @@ -46,7 +49,7 @@ type Node struct { } // newKV creates a Key-Value pair -func newKV(nodePath string, value string, createIndex uint64, +func newKV(store *store, nodePath string, value string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { return &Node{ @@ -57,6 +60,7 @@ func newKV(nodePath string, value string, createIndex uint64, ModifiedTerm: createTerm, Parent: parent, ACL: ACL, + store: store, stopExpire: make(chan bool, 1), ExpireTime: expireTime, Value: value, @@ -64,7 +68,7 @@ func newKV(nodePath string, value string, createIndex uint64, } // newDir creates a directory -func newDir(nodePath string, createIndex uint64, createTerm uint64, +func newDir(store *store, nodePath string, createIndex uint64, createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { return &Node{ @@ -76,6 +80,7 @@ func newDir(nodePath string, createIndex uint64, createTerm uint64, stopExpire: make(chan bool, 1), ExpireTime: expireTime, Children: make(map[string]*Node), + store: store, } } @@ -262,17 +267,17 @@ func (n *Node) internalRemove(recursive bool, callback func(path string)) { // if the node is already expired, delete the node and return. // if the node is permanent (this shouldn't happen), return at once. // else wait for a period time, then remove the node. and notify the watchhub. -func (n *Node) Expire(s *Store) { +func (n *Node) Expire() { expired, duration := n.IsExpired() if expired { // has been expired // since the parent function of Expire() runs serially, // there is no need for lock here e := newEvent(Expire, n.Path, UndefIndex, UndefTerm) - s.WatcherHub.notify(e) + n.store.WatcherHub.notify(e) n.Remove(true, nil) - s.Stats.Inc(ExpireCount) + n.store.Stats.Inc(ExpireCount) return } @@ -289,17 +294,17 @@ func (n *Node) Expire(s *Store) { // before expire get the lock, the expiration time // of the node may be updated. // we have to check again when get the lock - s.worldLock.Lock() - defer s.worldLock.Unlock() + n.store.worldLock.Lock() + defer n.store.worldLock.Unlock() expired, _ := n.IsExpired() if expired { e := newEvent(Expire, n.Path, UndefIndex, UndefTerm) - s.WatcherHub.notify(e) + n.store.WatcherHub.notify(e) n.Remove(true, nil) - s.Stats.Inc(ExpireCount) + n.store.Stats.Inc(ExpireCount) } return @@ -355,7 +360,7 @@ func (n *Node) Pair(recurisive, sorted bool) KeyValuePair { } } -func (n *Node) UpdateTTL(expireTime time.Time, s *Store) { +func (n *Node) UpdateTTL(expireTime time.Time) { if !n.IsPermanent() { // check if the node has been expired // if the node is not expired, we need to stop the go routine associated with @@ -369,7 +374,7 @@ func (n *Node) UpdateTTL(expireTime time.Time, s *Store) { if expireTime.Sub(Permanent) != 0 { n.ExpireTime = expireTime - n.Expire(s) + n.Expire() } } @@ -378,10 +383,10 @@ func (n *Node) UpdateTTL(expireTime time.Time, s *Store) { // If the node is a key-value pair, it will clone the pair. func (n *Node) Clone() *Node { if !n.IsDir() { - return newKV(n.Path, n.Value, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime) + return newKV(n.store, n.Path, n.Value, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime) } - clone := newDir(n.Path, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime) + clone := newDir(n.store, n.Path, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime) for key, child := range n.Children { clone.Children[key] = child.Clone() @@ -397,15 +402,16 @@ func (n *Node) Clone() *Node { // call this function on its children. // We check the expire last since we need to recover the whole structure first and add all the // notifications into the event history. -func (n *Node) recoverAndclean(s *Store) { +func (n *Node) recoverAndclean() { if n.IsDir() { for _, child := range n.Children { child.Parent = n - child.recoverAndclean(s) + child.store = n.store + child.recoverAndclean() } } n.stopExpire = make(chan bool, 1) - n.Expire(s) + n.Expire() } diff --git a/store/stats_test.go b/store/stats_test.go index deeb317dc1d..44d1a899925 100644 --- a/store/stats_test.go +++ b/store/stats_test.go @@ -7,7 +7,7 @@ import ( ) func TestBasicStats(t *testing.T) { - s := New() + s := newStore() keys := GenKeys(rand.Intn(100), 5) var i uint64 @@ -140,7 +140,7 @@ func TestBasicStats(t *testing.T) { t.Fatalf("TestAndSetFail [%d] != Stats.TestAndSetFail [%d]", TestAndSetFail, s.Stats.TestAndSetFail) } - s = New() + s = newStore() SetSuccess = 0 SetFail = 0 diff --git a/store/store.go b/store/store.go index ade44acd992..60eb35da5bd 100644 --- a/store/store.go +++ b/store/store.go @@ -13,7 +13,21 @@ import ( etcdErr "github.com/coreos/etcd/error" ) -type Store struct { +type Store interface { + Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) + Create(nodePath string, value string, incrementalSuffix bool, force bool, + expireTime time.Time, index uint64, term uint64) (*Event, error) + Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) + TestAndSet(nodePath string, prevValue string, prevIndex uint64, + value string, expireTime time.Time, index uint64, term uint64) (*Event, error) + Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error) + Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error) + Save() ([]byte, error) + Recovery(state []byte) error + JsonStats() []byte +} + +type store struct { Root *Node WatcherHub *watcherHub Index uint64 @@ -22,9 +36,13 @@ type Store struct { worldLock sync.RWMutex // stop the world lock } -func New() *Store { - s := new(Store) - s.Root = newDir("/", UndefIndex, UndefTerm, nil, "", Permanent) +func New() Store { + return newStore() +} + +func newStore() *store { + s := new(store) + s.Root = newDir(s, "/", UndefIndex, UndefTerm, nil, "", Permanent) s.Stats = newStats() s.WatcherHub = newWatchHub(1000) @@ -34,7 +52,7 @@ func New() *Store { // Get function returns a get event. // If recursive is true, it will return all the content under the node path. // If sorted is true, it will sort the content by keys. -func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) { +func (s *store) Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) { s.worldLock.RLock() defer s.worldLock.RUnlock() @@ -89,7 +107,7 @@ func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term // Create function creates the Node at nodePath. Create will help to create intermediate directories with no ttl. // If the node has already existed, create will fail. // If any node on the path is a file, create will fail. -func (s *Store) Create(nodePath string, value string, incrementalSuffix bool, force bool, +func (s *store) Create(nodePath string, value string, incrementalSuffix bool, force bool, expireTime time.Time, index uint64, term uint64) (*Event, error) { nodePath = path.Clean(path.Join("/", nodePath)) @@ -101,7 +119,7 @@ func (s *Store) Create(nodePath string, value string, incrementalSuffix bool, fo // Update function updates the value/ttl of the node. // If the node is a file, the value and the ttl can be updated. // If the node is a directory, only the ttl can be updated. -func (s *Store) Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) { +func (s *store) Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) { s.worldLock.Lock() defer s.worldLock.Unlock() nodePath = path.Clean(path.Join("/", nodePath)) @@ -127,7 +145,7 @@ func (s *Store) Update(nodePath string, newValue string, expireTime time.Time, i } // update ttl - n.UpdateTTL(expireTime, s) + n.UpdateTTL(expireTime) e.Expiration, e.TTL = n.ExpirationAndTTL() @@ -138,7 +156,7 @@ func (s *Store) Update(nodePath string, newValue string, expireTime time.Time, i return e, nil } -func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, +func (s *store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { nodePath = path.Clean(path.Join("/", nodePath)) @@ -168,7 +186,7 @@ func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, // if test succeed, write the value n.Write(value, index, term) - n.UpdateTTL(expireTime, s) + n.UpdateTTL(expireTime) e.Value = value e.Expiration, e.TTL = n.ExpirationAndTTL() @@ -185,7 +203,7 @@ func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, // Delete function deletes the node at the given path. // If the node is a directory, recursive must be true to delete it. -func (s *Store) Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error) { +func (s *store) Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error) { nodePath = path.Clean(path.Join("/", nodePath)) s.worldLock.Lock() @@ -224,7 +242,7 @@ func (s *Store) Delete(nodePath string, recursive bool, index uint64, term uint6 return e, nil } -func (s *Store) Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error) { +func (s *store) Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error) { prefix = path.Clean(path.Join("/", prefix)) s.worldLock.RLock() @@ -252,7 +270,7 @@ func (s *Store) Watch(prefix string, recursive bool, sinceIndex uint64, index ui } // walk function walks all the nodePath and apply the walkFunc on each directory -func (s *Store) walk(nodePath string, walkFunc func(prev *Node, component string) (*Node, *etcdErr.Error)) (*Node, *etcdErr.Error) { +func (s *store) walk(nodePath string, walkFunc func(prev *Node, component string) (*Node, *etcdErr.Error)) (*Node, *etcdErr.Error) { components := strings.Split(nodePath, "/") curr := s.Root @@ -273,7 +291,7 @@ func (s *Store) walk(nodePath string, walkFunc func(prev *Node, component string return curr, nil } -func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix bool, force bool, +func (s *store) internalCreate(nodePath string, value string, incrementalSuffix bool, force bool, expireTime time.Time, index uint64, term uint64, action string) (*Event, error) { s.Index, s.Term = index, term @@ -316,12 +334,12 @@ func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix if len(value) != 0 { // create file e.Value = value - n = newKV(nodePath, value, index, term, d, "", expireTime) + n = newKV(s, nodePath, value, index, term, d, "", expireTime) } else { // create directory e.Dir = true - n = newDir(nodePath, index, term, d, "", expireTime) + n = newDir(s, nodePath, index, term, d, "", expireTime) } @@ -334,7 +352,7 @@ func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix // Node with TTL if expireTime.Sub(Permanent) != 0 { - n.Expire(s) + n.Expire() e.Expiration, e.TTL = n.ExpirationAndTTL() } @@ -344,7 +362,7 @@ func (s *Store) internalCreate(nodePath string, value string, incrementalSuffix } // InternalGet function get the node of the given nodePath. -func (s *Store) internalGet(nodePath string, index uint64, term uint64) (*Node, *etcdErr.Error) { +func (s *store) internalGet(nodePath string, index uint64, term uint64) (*Node, *etcdErr.Error) { nodePath = path.Clean(path.Join("/", nodePath)) // update file system known index and term @@ -379,7 +397,7 @@ func (s *Store) internalGet(nodePath string, index uint64, term uint64) (*Node, // If it is a directory, this function will return the pointer to that node. // If it does not exist, this function will create a new directory and return the pointer to that node. // If it is a file, this function will return error. -func (s *Store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) { +func (s *store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) { node, ok := parent.Children[dirName] if ok { @@ -390,7 +408,7 @@ func (s *Store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) { return nil, etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path, UndefIndex, UndefTerm) } - n := newDir(path.Join(parent.Path, dirName), s.Index, s.Term, parent, parent.ACL, Permanent) + n := newDir(s, path.Join(parent.Path, dirName), s.Index, s.Term, parent, parent.ACL, Permanent) parent.Children[dirName] = n @@ -401,10 +419,10 @@ func (s *Store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) { // Save function will not be able to save the state of watchers. // Save function will not save the parent field of the node. Or there will // be cyclic dependencies issue for the json package. -func (s *Store) Save() ([]byte, error) { +func (s *store) Save() ([]byte, error) { s.worldLock.Lock() - clonedStore := New() + clonedStore := newStore() clonedStore.Index = s.Index clonedStore.Term = s.Term clonedStore.Root = s.Root.Clone() @@ -426,7 +444,7 @@ func (s *Store) Save() ([]byte, error) { // It needs to recovery the parent field of the nodes. // It needs to delete the expired nodes since the saved time and also // need to create monitor go routines. -func (s *Store) Recovery(state []byte) error { +func (s *store) Recovery(state []byte) error { s.worldLock.Lock() defer s.worldLock.Unlock() err := json.Unmarshal(state, s) @@ -435,11 +453,11 @@ func (s *Store) Recovery(state []byte) error { return err } - s.Root.recoverAndclean(s) + s.Root.recoverAndclean() return nil } -func (s *Store) JsonStats() []byte { +func (s *store) JsonStats() []byte { s.Stats.Watchers = uint64(s.WatcherHub.count) return s.Stats.toJson() } diff --git a/store/store_test.go b/store/store_test.go index 74add4424d7..958d99007b1 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -8,7 +8,7 @@ import ( ) func TestCreateAndGet(t *testing.T) { - s := New() + s := newStore() s.Create("/foobar", "bar", false, false, Permanent, 1, 1) @@ -66,7 +66,7 @@ func TestCreateAndGet(t *testing.T) { } func TestUpdateFile(t *testing.T) { - s := New() + s := newStore() _, err := s.Create("/foo/bar", "bar", false, false, Permanent, 1, 1) @@ -161,7 +161,7 @@ func TestUpdateFile(t *testing.T) { } func TestListDirectory(t *testing.T) { - s := New() + s := newStore() // create dir /foo // set key-value /foo/foo=bar @@ -206,7 +206,7 @@ func TestListDirectory(t *testing.T) { } func TestRemove(t *testing.T) { - s := New() + s := newStore() s.Create("/foo", "bar", false, false, Permanent, 1, 1) _, err := s.Delete("/foo", false, 1, 1) @@ -245,7 +245,7 @@ func TestRemove(t *testing.T) { } func TestExpire(t *testing.T) { - s := New() + s := newStore() expire := time.Now().Add(time.Second) @@ -287,7 +287,7 @@ func TestExpire(t *testing.T) { } func TestTestAndSet(t *testing.T) { // TODO prevValue == nil ? - s := New() + s := newStore() s.Create("/foo", "bar", false, false, Permanent, 1, 1) // test on wrong previous value @@ -320,7 +320,7 @@ func TestTestAndSet(t *testing.T) { // TODO prevValue == nil ? } func TestWatch(t *testing.T) { - s := New() + s := newStore() // watch at a deeper path c, _ := s.Watch("/foo/foo/foo", false, 0, 0, 1) s.Create("/foo/foo/foo", "bar", false, false, Permanent, 1, 1) @@ -409,7 +409,7 @@ func TestWatch(t *testing.T) { } func TestSort(t *testing.T) { - s := New() + s := newStore() // simulating random creation keys := GenKeys(80, 4) @@ -447,7 +447,7 @@ func TestSort(t *testing.T) { } func TestSaveAndRecover(t *testing.T) { - s := New() + s := newStore() // simulating random creation keys := GenKeys(8, 4) @@ -469,7 +469,7 @@ func TestSaveAndRecover(t *testing.T) { s.Create("/foo/foo", "bar", false, false, expire, 1, 1) b, err := s.Save() - cloneFs := New() + cloneFs := newStore() time.Sleep(2 * time.Second) cloneFs.Recovery(b) @@ -521,7 +521,7 @@ func GenKeys(num int, depth int) []string { return keys } -func createAndGet(s *Store, path string, t *testing.T) { +func createAndGet(s *store, path string, t *testing.T) { _, err := s.Create(path, "bar", false, false, Permanent, 1, 1) if err != nil { diff --git a/store/test_and_set_command.go b/store/test_and_set_command.go index 3370fed154e..811f713e508 100644 --- a/store/test_and_set_command.go +++ b/store/test_and_set_command.go @@ -27,7 +27,7 @@ func (c *TestAndSetCommand) CommandName() string { // Set the key-value pair if the current value of the key equals to the given prevValue func (c *TestAndSetCommand) Apply(server *raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(*Store) + s, _ := server.StateMachine().(Store) e, err := s.TestAndSet(c.Key, c.PrevValue, c.PrevIndex, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) diff --git a/store/update_command.go b/store/update_command.go index 9ffd6c8686d..582fb42fe5a 100644 --- a/store/update_command.go +++ b/store/update_command.go @@ -25,7 +25,7 @@ func (c *UpdateCommand) CommandName() string { // Update node func (c *UpdateCommand) Apply(server *raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(*Store) + s, _ := server.StateMachine().(Store) e, err := s.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) diff --git a/store/watcher_test.go b/store/watcher_test.go index e437422ad13..61da92f7c63 100644 --- a/store/watcher_test.go +++ b/store/watcher_test.go @@ -5,7 +5,7 @@ import ( ) func TestWatcher(t *testing.T) { - s := New() + s := newStore() wh := s.WatcherHub c, err := wh.watch("/foo", true, 1) if err != nil { From e7598075ac85512c419bb0c27e055799f6bbb2e9 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 14 Oct 2013 13:05:55 -0600 Subject: [PATCH 104/247] Use raft.Server interface. --- server/join_command.go | 4 +- server/peer_server.go | 38 +-- server/peer_server_handlers.go | 10 +- server/remove_command.go | 2 +- server/server.go | 18 +- server/transporter.go | 8 +- store/create_command.go | 2 +- store/delete_command.go | 2 +- store/test_and_set_command.go | 2 +- store/update_command.go | 2 +- .../github.com/coreos/go-etcd/README.md | 6 +- .../github.com/coreos/go-etcd/etcd/delete.go | 5 +- .../github.com/coreos/go-etcd/etcd/get.go | 13 +- .../coreos/go-etcd/etcd/response.go | 26 ++ .../github.com/coreos/go-etcd/etcd/set.go | 9 +- .../coreos/go-etcd/etcd/testAndSet.go | 5 +- .../github.com/coreos/go-etcd/etcd/watch.go | 7 +- .../coreos/go-etcd/etcd/watch_test.go | 5 +- .../github.com/coreos/go-raft/README.md | 2 +- .../github.com/coreos/go-raft/command.go | 2 +- .../coreos/go-raft/http_transporter.go | 18 +- .../coreos/go-raft/http_transporter_test.go | 18 +- .../github.com/coreos/go-raft/join_command.go | 4 +- .../coreos/go-raft/leave_command.go | 4 +- .../github.com/coreos/go-raft/nop_command.go | 2 +- third_party/github.com/coreos/go-raft/peer.go | 4 +- .../github.com/coreos/go-raft/server.go | 159 ++++++---- .../github.com/coreos/go-raft/server_test.go | 289 +++++++++--------- .../github.com/coreos/go-raft/snapshot.go | 6 +- .../go-raft/snapshot_recovery_request.go | 2 +- third_party/github.com/coreos/go-raft/test.go | 28 +- .../github.com/coreos/go-raft/transporter.go | 8 +- web/web.go | 2 +- 33 files changed, 384 insertions(+), 328 deletions(-) create mode 100644 third_party/github.com/coreos/go-etcd/etcd/response.go diff --git a/server/join_command.go b/server/join_command.go index 83d2efb7393..7bebbe70463 100644 --- a/server/join_command.go +++ b/server/join_command.go @@ -35,7 +35,7 @@ func (c *JoinCommand) CommandName() string { } // Join a server to the cluster -func (c *JoinCommand) Apply(server *raft.Server) (interface{}, error) { +func (c *JoinCommand) Apply(server raft.Server) (interface{}, error) { ps, _ := server.Context().(*PeerServer) b := make([]byte, 8) @@ -62,7 +62,7 @@ func (c *JoinCommand) Apply(server *raft.Server) (interface{}, error) { err := server.AddPeer(c.Name, "") // Add peer stats - if c.Name != ps.Name() { + if c.Name != ps.RaftServer().Name() { ps.followersStats.Followers[c.Name] = &raftFollowerStats{} ps.followersStats.Followers[c.Name].Latency.Minimum = 1 << 63 } diff --git a/server/peer_server.go b/server/peer_server.go index 97a7757a375..d85ed9a2194 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -19,7 +19,7 @@ import ( ) type PeerServer struct { - *raft.Server + raftServer raft.Server server *Server joinIndex uint64 name string @@ -78,12 +78,12 @@ func NewPeerServer(name string, path string, url string, listenHost string, tlsC raftTransporter := newTransporter(tlsConf.Scheme, tlsConf.Client, s) // Create raft server - server, err := raft.NewServer(name, path, raftTransporter, s.store, s, "") + raftServer, err := raft.NewServer(name, path, raftTransporter, s.store, s, "") if err != nil { log.Fatal(err) } - s.Server = server + s.raftServer = raftServer return s } @@ -92,7 +92,7 @@ func NewPeerServer(name string, path string, url string, listenHost string, tlsC func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) { // LoadSnapshot if snapshot { - err := s.LoadSnapshot() + err := s.raftServer.LoadSnapshot() if err == nil { log.Debugf("%s finished load snapshot", s.name) @@ -101,12 +101,12 @@ func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) { } } - s.SetElectionTimeout(ElectionTimeout) - s.SetHeartbeatTimeout(HeartbeatTimeout) + s.raftServer.SetElectionTimeout(ElectionTimeout) + s.raftServer.SetHeartbeatTimeout(HeartbeatTimeout) - s.Start() + s.raftServer.Start() - if s.IsLogEmpty() { + if s.raftServer.IsLogEmpty() { // start as a leader in a new cluster if len(cluster) == 0 { s.startAsLeader() @@ -116,7 +116,7 @@ func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) { } else { // Rejoin the previous cluster - cluster = s.registry.PeerURLs(s.Leader(), s.name) + cluster = s.registry.PeerURLs(s.raftServer.Leader(), s.name) for i := 0; i < len(cluster); i++ { u, err := url.Parse(cluster[i]) if err != nil { @@ -143,8 +143,8 @@ func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) { } // Retrieves the underlying Raft server. -func (s *PeerServer) RaftServer() *raft.Server { - return s.Server +func (s *PeerServer) RaftServer() raft.Server { + return s.raftServer } // Associates the client server with the peer server. @@ -155,7 +155,7 @@ func (s *PeerServer) SetServer(server *Server) { func (s *PeerServer) startAsLeader() { // leader need to join self as a peer for { - _, err := s.Do(NewJoinCommand(PeerVersion, s.Name(), s.url, s.server.URL())) + _, err := s.raftServer.Do(NewJoinCommand(PeerVersion, s.raftServer.Name(), s.url, s.server.URL())) if err == nil { break } @@ -232,7 +232,7 @@ func (s *PeerServer) joinCluster(cluster []string) bool { continue } - err := s.joinByMachine(s.Server, machine, s.tlsConf.Scheme) + err := s.joinByMachine(s.raftServer, machine, s.tlsConf.Scheme) if err == nil { log.Debugf("%s success join to the cluster via machine %s", s.name, machine) return true @@ -249,7 +249,7 @@ func (s *PeerServer) joinCluster(cluster []string) bool { } // Send join requests to machine. -func (s *PeerServer) joinByMachine(server *raft.Server, machine string, scheme string) error { +func (s *PeerServer) joinByMachine(server raft.Server, machine string, scheme string) error { var b bytes.Buffer // t must be ok @@ -327,7 +327,7 @@ func (s *PeerServer) Stats() []byte { } func (s *PeerServer) PeerStats() []byte { - if s.State() == raft.Leader { + if s.raftServer.State() == raft.Leader { b, _ := json.Marshal(s.followersStats) return b } @@ -339,15 +339,15 @@ func (s *PeerServer) monitorSnapshot() { time.Sleep(s.snapConf.checkingInterval) currentWrites := 0 if uint64(currentWrites) > s.snapConf.writesThr { - s.TakeSnapshot() + s.raftServer.TakeSnapshot() s.snapConf.lastWrites = 0 } } } func (s *PeerServer) dispatch(c raft.Command, w http.ResponseWriter, req *http.Request) error { - if s.State() == raft.Leader { - result, err := s.Do(c) + if s.raftServer.State() == raft.Leader { + result, err := s.raftServer.Do(c) if err != nil { return err } @@ -375,7 +375,7 @@ func (s *PeerServer) dispatch(c raft.Command, w http.ResponseWriter, req *http.R return nil } else { - leader := s.Leader() + leader := s.raftServer.Leader() // No leader available. if leader == "" { diff --git a/server/peer_server_handlers.go b/server/peer_server_handlers.go index ad055623eb8..adb192e606f 100644 --- a/server/peer_server_handlers.go +++ b/server/peer_server_handlers.go @@ -14,7 +14,7 @@ func (s *PeerServer) GetLogHttpHandler(w http.ResponseWriter, req *http.Request) log.Debugf("[recv] GET %s/log", s.url) w.Header().Set("Content-Type", "application/json") w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(s.LogEntries()) + json.NewEncoder(w).Encode(s.raftServer.LogEntries()) } // Response to vote request @@ -23,7 +23,7 @@ func (s *PeerServer) VoteHttpHandler(w http.ResponseWriter, req *http.Request) { err := decodeJsonRequest(req, rvreq) if err == nil { log.Debugf("[recv] POST %s/vote [%s]", s.url, rvreq.CandidateName) - if resp := s.RequestVote(rvreq); resp != nil { + if resp := s.raftServer.RequestVote(rvreq); resp != nil { w.WriteHeader(http.StatusOK) json.NewEncoder(w).Encode(resp) return @@ -43,7 +43,7 @@ func (s *PeerServer) AppendEntriesHttpHandler(w http.ResponseWriter, req *http.R s.serverStats.RecvAppendReq(aereq.LeaderName, int(req.ContentLength)) - if resp := s.AppendEntries(aereq); resp != nil { + if resp := s.raftServer.AppendEntries(aereq); resp != nil { w.WriteHeader(http.StatusOK) json.NewEncoder(w).Encode(resp) if !resp.Success { @@ -62,7 +62,7 @@ func (s *PeerServer) SnapshotHttpHandler(w http.ResponseWriter, req *http.Reques err := decodeJsonRequest(req, aereq) if err == nil { log.Debugf("[recv] POST %s/snapshot/ ", s.url) - if resp := s.RequestSnapshot(aereq); resp != nil { + if resp := s.raftServer.RequestSnapshot(aereq); resp != nil { w.WriteHeader(http.StatusOK) json.NewEncoder(w).Encode(resp) return @@ -78,7 +78,7 @@ func (s *PeerServer) SnapshotRecoveryHttpHandler(w http.ResponseWriter, req *htt err := decodeJsonRequest(req, aereq) if err == nil { log.Debugf("[recv] POST %s/snapshotRecovery/ ", s.url) - if resp := s.SnapshotRecoveryRequest(aereq); resp != nil { + if resp := s.raftServer.SnapshotRecoveryRequest(aereq); resp != nil { w.WriteHeader(http.StatusOK) json.NewEncoder(w).Encode(resp) return diff --git a/server/remove_command.go b/server/remove_command.go index 6a8f7517125..42e2c507867 100644 --- a/server/remove_command.go +++ b/server/remove_command.go @@ -23,7 +23,7 @@ func (c *RemoveCommand) CommandName() string { } // Remove a server from the cluster -func (c *RemoveCommand) Apply(server *raft.Server) (interface{}, error) { +func (c *RemoveCommand) Apply(server raft.Server) (interface{}, error) { ps, _ := server.Context().(*PeerServer) // Remove node from the shared registry. diff --git a/server/server.go b/server/server.go index 4e21aa17911..d1b1abf0f59 100644 --- a/server/server.go +++ b/server/server.go @@ -56,22 +56,22 @@ func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsI // The current state of the server in the cluster. func (s *Server) State() string { - return s.peerServer.State() + return s.peerServer.RaftServer().State() } // The node name of the leader in the cluster. func (s *Server) Leader() string { - return s.peerServer.Leader() + return s.peerServer.RaftServer().Leader() } // The current Raft committed index. func (s *Server) CommitIndex() uint64 { - return s.peerServer.CommitIndex() + return s.peerServer.RaftServer().CommitIndex() } // The current Raft term. func (s *Server) Term() uint64 { - return s.peerServer.Term() + return s.peerServer.RaftServer().Term() } // The server URL. @@ -201,7 +201,7 @@ func (s *Server) GetVersionHandler(w http.ResponseWriter, req *http.Request) err // Handler to return the current leader's raft address func (s *Server) GetLeaderHandler(w http.ResponseWriter, req *http.Request) error { - leader := s.peerServer.Leader() + leader := s.peerServer.RaftServer().Leader() if leader == "" { return etcdErr.NewError(etcdErr.EcodeLeaderElect, "", store.UndefIndex, store.UndefTerm) } @@ -213,7 +213,7 @@ func (s *Server) GetLeaderHandler(w http.ResponseWriter, req *http.Request) erro // Handler to return all the known machines in the current cluster. func (s *Server) GetMachinesHandler(w http.ResponseWriter, req *http.Request) error { - machines := s.registry.ClientURLs(s.peerServer.Leader(), s.name) + machines := s.registry.ClientURLs(s.peerServer.RaftServer().Leader(), s.name) w.WriteHeader(http.StatusOK) w.Write([]byte(strings.Join(machines, ", "))) return nil @@ -227,12 +227,12 @@ func (s *Server) GetStatsHandler(w http.ResponseWriter, req *http.Request) error // Retrieves stats on the leader. func (s *Server) GetLeaderStatsHandler(w http.ResponseWriter, req *http.Request) error { - if s.peerServer.State() == raft.Leader { + if s.peerServer.RaftServer().State() == raft.Leader { w.Write(s.peerServer.PeerStats()) return nil } - leader := s.peerServer.Leader() + leader := s.peerServer.RaftServer().Leader() if leader == "" { return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) } @@ -259,7 +259,7 @@ func (s *Server) SpeedTestHandler(w http.ResponseWriter, req *http.Request) erro Value: "bar", ExpireTime: time.Unix(0, 0), } - s.peerServer.Do(c) + s.peerServer.RaftServer().Do(c) } c <- true }() diff --git a/server/transporter.go b/server/transporter.go index 56d1b1edc9d..3c5002f1ba2 100644 --- a/server/transporter.go +++ b/server/transporter.go @@ -62,7 +62,7 @@ func dialWithTimeout(network, addr string) (net.Conn, error) { } // Sends AppendEntries RPCs to a peer when the server is the leader. -func (t *transporter) SendAppendEntriesRequest(server *raft.Server, peer *raft.Peer, req *raft.AppendEntriesRequest) *raft.AppendEntriesResponse { +func (t *transporter) SendAppendEntriesRequest(server raft.Server, peer *raft.Peer, req *raft.AppendEntriesRequest) *raft.AppendEntriesResponse { var aersp *raft.AppendEntriesResponse var b bytes.Buffer @@ -117,7 +117,7 @@ func (t *transporter) SendAppendEntriesRequest(server *raft.Server, peer *raft.P } // Sends RequestVote RPCs to a peer when the server is the candidate. -func (t *transporter) SendVoteRequest(server *raft.Server, peer *raft.Peer, req *raft.RequestVoteRequest) *raft.RequestVoteResponse { +func (t *transporter) SendVoteRequest(server raft.Server, peer *raft.Peer, req *raft.RequestVoteRequest) *raft.RequestVoteResponse { var rvrsp *raft.RequestVoteResponse var b bytes.Buffer json.NewEncoder(&b).Encode(req) @@ -146,7 +146,7 @@ func (t *transporter) SendVoteRequest(server *raft.Server, peer *raft.Peer, req } // Sends SnapshotRequest RPCs to a peer when the server is the candidate. -func (t *transporter) SendSnapshotRequest(server *raft.Server, peer *raft.Peer, req *raft.SnapshotRequest) *raft.SnapshotResponse { +func (t *transporter) SendSnapshotRequest(server raft.Server, peer *raft.Peer, req *raft.SnapshotRequest) *raft.SnapshotResponse { var aersp *raft.SnapshotResponse var b bytes.Buffer json.NewEncoder(&b).Encode(req) @@ -177,7 +177,7 @@ func (t *transporter) SendSnapshotRequest(server *raft.Server, peer *raft.Peer, } // Sends SnapshotRecoveryRequest RPCs to a peer when the server is the candidate. -func (t *transporter) SendSnapshotRecoveryRequest(server *raft.Server, peer *raft.Peer, req *raft.SnapshotRecoveryRequest) *raft.SnapshotRecoveryResponse { +func (t *transporter) SendSnapshotRecoveryRequest(server raft.Server, peer *raft.Peer, req *raft.SnapshotRecoveryRequest) *raft.SnapshotRecoveryResponse { var aersp *raft.SnapshotRecoveryResponse var b bytes.Buffer json.NewEncoder(&b).Encode(req) diff --git a/store/create_command.go b/store/create_command.go index f13b9179064..43c09f99841 100644 --- a/store/create_command.go +++ b/store/create_command.go @@ -25,7 +25,7 @@ func (c *CreateCommand) CommandName() string { } // Create node -func (c *CreateCommand) Apply(server *raft.Server) (interface{}, error) { +func (c *CreateCommand) Apply(server raft.Server) (interface{}, error) { s, _ := server.StateMachine().(Store) e, err := s.Create(c.Key, c.Value, c.IncrementalSuffix, c.Force, c.ExpireTime, server.CommitIndex(), server.Term()) diff --git a/store/delete_command.go b/store/delete_command.go index bc84dfc9970..6ff3c5c6fab 100644 --- a/store/delete_command.go +++ b/store/delete_command.go @@ -21,7 +21,7 @@ func (c *DeleteCommand) CommandName() string { } // Delete the key -func (c *DeleteCommand) Apply(server *raft.Server) (interface{}, error) { +func (c *DeleteCommand) Apply(server raft.Server) (interface{}, error) { s, _ := server.StateMachine().(Store) e, err := s.Delete(c.Key, c.Recursive, server.CommitIndex(), server.Term()) diff --git a/store/test_and_set_command.go b/store/test_and_set_command.go index 811f713e508..03cb1879a39 100644 --- a/store/test_and_set_command.go +++ b/store/test_and_set_command.go @@ -26,7 +26,7 @@ func (c *TestAndSetCommand) CommandName() string { } // Set the key-value pair if the current value of the key equals to the given prevValue -func (c *TestAndSetCommand) Apply(server *raft.Server) (interface{}, error) { +func (c *TestAndSetCommand) Apply(server raft.Server) (interface{}, error) { s, _ := server.StateMachine().(Store) e, err := s.TestAndSet(c.Key, c.PrevValue, c.PrevIndex, diff --git a/store/update_command.go b/store/update_command.go index 582fb42fe5a..3152006bc65 100644 --- a/store/update_command.go +++ b/store/update_command.go @@ -24,7 +24,7 @@ func (c *UpdateCommand) CommandName() string { } // Update node -func (c *UpdateCommand) Apply(server *raft.Server) (interface{}, error) { +func (c *UpdateCommand) Apply(server raft.Server) (interface{}, error) { s, _ := server.StateMachine().(Store) e, err := s.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) diff --git a/third_party/github.com/coreos/go-etcd/README.md b/third_party/github.com/coreos/go-etcd/README.md index 7a8f8b34a72..9496dace47f 100644 --- a/third_party/github.com/coreos/go-etcd/README.md +++ b/third_party/github.com/coreos/go-etcd/README.md @@ -31,19 +31,19 @@ func main() { c := etcd.NewClient() // default binds to http://0.0.0.0:4001 // SET the value "bar" to the key "foo" with zero TTL - // returns a: *store.Response + // returns a: *Response res, _ := c.Set("foo", "bar", 0) fmt.Printf("set response: %+v\n", res) // GET the value that is stored for the key "foo" - // return a slice: []*store.Response + // return a slice: []*Response values, _ := c.Get("foo") for i, res := range values { // .. and print them out fmt.Printf("[%d] get response: %+v\n", i, res) } // DELETE the key "foo" - // returns a: *store.Response + // returns a: *Response res, _ = c.Delete("foo") fmt.Printf("delete response: %+v\n", res) } diff --git a/third_party/github.com/coreos/go-etcd/etcd/delete.go b/third_party/github.com/coreos/go-etcd/etcd/delete.go index fea16956091..91f6df87a5a 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/delete.go +++ b/third_party/github.com/coreos/go-etcd/etcd/delete.go @@ -2,13 +2,12 @@ package etcd import ( "encoding/json" - "github.com/coreos/etcd/store" "io/ioutil" "net/http" "path" ) -func (c *Client) Delete(key string) (*store.Response, error) { +func (c *Client) Delete(key string) (*Response, error) { resp, err := c.sendRequest("DELETE", path.Join("keys", key), "") @@ -28,7 +27,7 @@ func (c *Client) Delete(key string) (*store.Response, error) { return nil, handleError(b) } - var result store.Response + var result Response err = json.Unmarshal(b, &result) diff --git a/third_party/github.com/coreos/go-etcd/etcd/get.go b/third_party/github.com/coreos/go-etcd/etcd/get.go index b0d16fe20b0..3288621e7c4 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/get.go +++ b/third_party/github.com/coreos/go-etcd/etcd/get.go @@ -2,13 +2,12 @@ package etcd import ( "encoding/json" - "github.com/coreos/etcd/store" "io/ioutil" "net/http" "path" ) -func (c *Client) Get(key string) ([]*store.Response, error) { +func (c *Client) Get(key string) ([]*Response, error) { logger.Debugf("get %s [%s]", key, c.cluster.Leader) resp, err := c.sendRequest("GET", path.Join("keys", key), "") @@ -36,7 +35,7 @@ func (c *Client) Get(key string) ([]*store.Response, error) { // GetTo gets the value of the key from a given machine address. // If the given machine is not available it returns an error. // Mainly use for testing purpose -func (c *Client) GetFrom(key string, addr string) ([]*store.Response, error) { +func (c *Client) GetFrom(key string, addr string) ([]*Response, error) { httpPath := c.createHttpPath(addr, path.Join(version, "keys", key)) resp, err := c.httpClient.Get(httpPath) @@ -61,10 +60,10 @@ func (c *Client) GetFrom(key string, addr string) ([]*store.Response, error) { } // Convert byte stream to response. -func convertGetResponse(b []byte) ([]*store.Response, error) { +func convertGetResponse(b []byte) ([]*Response, error) { - var results []*store.Response - var result *store.Response + var results []*Response + var result *Response err := json.Unmarshal(b, &result) @@ -76,7 +75,7 @@ func convertGetResponse(b []byte) ([]*store.Response, error) { } } else { - results = make([]*store.Response, 1) + results = make([]*Response, 1) results[0] = result } return results, nil diff --git a/third_party/github.com/coreos/go-etcd/etcd/response.go b/third_party/github.com/coreos/go-etcd/etcd/response.go new file mode 100644 index 00000000000..d2311a675bd --- /dev/null +++ b/third_party/github.com/coreos/go-etcd/etcd/response.go @@ -0,0 +1,26 @@ +package etcd + +import ( + "time" +) + +// The response object from the server. +type Response struct { + Action string `json:"action"` + Key string `json:"key"` + Dir bool `json:"dir,omitempty"` + PrevValue string `json:"prevValue,omitempty"` + Value string `json:"value,omitempty"` + + // If the key did not exist before the action, + // this field should be set to true + NewKey bool `json:"newKey,omitempty"` + + Expiration *time.Time `json:"expiration,omitempty"` + + // Time to live in second + TTL int64 `json:"ttl,omitempty"` + + // The command index of the raft machine when the command is executed + Index uint64 `json:"index"` +} diff --git a/third_party/github.com/coreos/go-etcd/etcd/set.go b/third_party/github.com/coreos/go-etcd/etcd/set.go index 78acb90815b..17fc415f2f8 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/set.go +++ b/third_party/github.com/coreos/go-etcd/etcd/set.go @@ -3,14 +3,13 @@ package etcd import ( "encoding/json" "fmt" - "github.com/coreos/etcd/store" "io/ioutil" "net/http" "net/url" "path" ) -func (c *Client) Set(key string, value string, ttl uint64) (*store.Response, error) { +func (c *Client) Set(key string, value string, ttl uint64) (*Response, error) { logger.Debugf("set %s, %s, ttl: %d, [%s]", key, value, ttl, c.cluster.Leader) v := url.Values{} v.Set("value", value) @@ -45,7 +44,7 @@ func (c *Client) Set(key string, value string, ttl uint64) (*store.Response, err // SetTo sets the value of the key to a given machine address. // If the given machine is not available or is not leader it returns an error // Mainly use for testing purpose. -func (c *Client) SetTo(key string, value string, ttl uint64, addr string) (*store.Response, error) { +func (c *Client) SetTo(key string, value string, ttl uint64, addr string) (*Response, error) { v := url.Values{} v.Set("value", value) @@ -77,8 +76,8 @@ func (c *Client) SetTo(key string, value string, ttl uint64, addr string) (*stor } // Convert byte stream to response. -func convertSetResponse(b []byte) (*store.Response, error) { - var result store.Response +func convertSetResponse(b []byte) (*Response, error) { + var result Response err := json.Unmarshal(b, &result) diff --git a/third_party/github.com/coreos/go-etcd/etcd/testAndSet.go b/third_party/github.com/coreos/go-etcd/etcd/testAndSet.go index 0bd8672ecef..bdd8ecb4fd8 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/testAndSet.go +++ b/third_party/github.com/coreos/go-etcd/etcd/testAndSet.go @@ -3,14 +3,13 @@ package etcd import ( "encoding/json" "fmt" - "github.com/coreos/etcd/store" "io/ioutil" "net/http" "net/url" "path" ) -func (c *Client) TestAndSet(key string, prevValue string, value string, ttl uint64) (*store.Response, bool, error) { +func (c *Client) TestAndSet(key string, prevValue string, value string, ttl uint64) (*Response, bool, error) { logger.Debugf("set %s, %s[%s], ttl: %d, [%s]", key, value, prevValue, ttl, c.cluster.Leader) v := url.Values{} v.Set("value", value) @@ -39,7 +38,7 @@ func (c *Client) TestAndSet(key string, prevValue string, value string, ttl uint return nil, false, handleError(b) } - var result store.Response + var result Response err = json.Unmarshal(b, &result) diff --git a/third_party/github.com/coreos/go-etcd/etcd/watch.go b/third_party/github.com/coreos/go-etcd/etcd/watch.go index 7f59ed065a3..18fcfdc1235 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/watch.go +++ b/third_party/github.com/coreos/go-etcd/etcd/watch.go @@ -4,7 +4,6 @@ import ( "encoding/json" "errors" "fmt" - "github.com/coreos/etcd/store" "io/ioutil" "net/http" "net/url" @@ -28,7 +27,7 @@ var ( // channel. And after someone receive the channel, it will go on to watch that prefix. // If a stop channel is given, client can close long-term watch using the stop channel -func (c *Client) Watch(prefix string, sinceIndex uint64, receiver chan *store.Response, stop chan bool) (*store.Response, error) { +func (c *Client) Watch(prefix string, sinceIndex uint64, receiver chan *Response, stop chan bool) (*Response, error) { logger.Debugf("watch %s [%s]", prefix, c.cluster.Leader) if receiver == nil { return c.watchOnce(prefix, sinceIndex, stop) @@ -50,7 +49,7 @@ func (c *Client) Watch(prefix string, sinceIndex uint64, receiver chan *store.Re // helper func // return when there is change under the given prefix -func (c *Client) watchOnce(key string, sinceIndex uint64, stop chan bool) (*store.Response, error) { +func (c *Client) watchOnce(key string, sinceIndex uint64, stop chan bool) (*Response, error) { var resp *http.Response var err error @@ -94,7 +93,7 @@ func (c *Client) watchOnce(key string, sinceIndex uint64, stop chan bool) (*stor return nil, handleError(b) } - var result store.Response + var result Response err = json.Unmarshal(b, &result) diff --git a/third_party/github.com/coreos/go-etcd/etcd/watch_test.go b/third_party/github.com/coreos/go-etcd/etcd/watch_test.go index 0d9348518ca..a3d33a4f162 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/watch_test.go +++ b/third_party/github.com/coreos/go-etcd/etcd/watch_test.go @@ -2,7 +2,6 @@ package etcd import ( "fmt" - "github.com/coreos/etcd/store" "testing" "time" ) @@ -30,7 +29,7 @@ func TestWatch(t *testing.T) { t.Fatalf("Watch with Index failed with %s %s %v %v", result.Key, result.Value, result.TTL, result.Index) } - ch := make(chan *store.Response, 10) + ch := make(chan *Response, 10) stop := make(chan bool, 1) go setLoop("bar", c) @@ -57,7 +56,7 @@ func setLoop(value string, c *Client) { } } -func receiver(c chan *store.Response, stop chan bool) { +func receiver(c chan *Response, stop chan bool) { for i := 0; i < 10; i++ { <-c } diff --git a/third_party/github.com/coreos/go-raft/README.md b/third_party/github.com/coreos/go-raft/README.md index 6ecc74866f4..9d189ff6db3 100644 --- a/third_party/github.com/coreos/go-raft/README.md +++ b/third_party/github.com/coreos/go-raft/README.md @@ -57,7 +57,7 @@ A distributed consensus protocol is used for maintaining a consistent state acro Many distributed systems are built upon the Paxos protocol but Paxos can be difficult to understand and there are many gaps between Paxos and real world implementation. An alternative is the [Raft distributed consensus protocol][raft-paper] by Diego Ongaro and John Ousterhout. -Raft is a protocol built with understandability as a primary tenant and it centers around two things: +Raft is a protocol built with understandability as a primary tenet and it centers around two things: 1. Leader Election 2. Replicated Log diff --git a/third_party/github.com/coreos/go-raft/command.go b/third_party/github.com/coreos/go-raft/command.go index 2c0495171a4..9188881b87a 100644 --- a/third_party/github.com/coreos/go-raft/command.go +++ b/third_party/github.com/coreos/go-raft/command.go @@ -29,7 +29,7 @@ func init() { // A command represents an action to be taken on the replicated state machine. type Command interface { CommandName() string - Apply(server *Server) (interface{}, error) + Apply(server Server) (interface{}, error) } type CommandEncoder interface { diff --git a/third_party/github.com/coreos/go-raft/http_transporter.go b/third_party/github.com/coreos/go-raft/http_transporter.go index e41fd817f07..80aa3e74e79 100644 --- a/third_party/github.com/coreos/go-raft/http_transporter.go +++ b/third_party/github.com/coreos/go-raft/http_transporter.go @@ -77,7 +77,7 @@ func (t *HTTPTransporter) RequestVotePath() string { //-------------------------------------- // Applies Raft routes to an HTTP router for a given server. -func (t *HTTPTransporter) Install(server *Server, mux HTTPMuxer) { +func (t *HTTPTransporter) Install(server Server, mux HTTPMuxer) { mux.HandleFunc(t.AppendEntriesPath(), t.appendEntriesHandler(server)) mux.HandleFunc(t.RequestVotePath(), t.requestVoteHandler(server)) } @@ -87,14 +87,14 @@ func (t *HTTPTransporter) Install(server *Server, mux HTTPMuxer) { //-------------------------------------- // Sends an AppendEntries RPC to a peer. -func (t *HTTPTransporter) SendAppendEntriesRequest(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { +func (t *HTTPTransporter) SendAppendEntriesRequest(server Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { var b bytes.Buffer if _, err := req.Encode(&b); err != nil { traceln("transporter.ae.encoding.error:", err) return nil } - url := fmt.Sprintf("http://%s%s", peer.Name, t.AppendEntriesPath()) + url := fmt.Sprintf("%s%s", peer.ConnectionString, t.AppendEntriesPath()) traceln(server.Name(), "POST", url) client := &http.Client{Transport: &http.Transport{DisableKeepAlives: t.DisableKeepAlives}} @@ -115,14 +115,14 @@ func (t *HTTPTransporter) SendAppendEntriesRequest(server *Server, peer *Peer, r } // Sends a RequestVote RPC to a peer. -func (t *HTTPTransporter) SendVoteRequest(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { +func (t *HTTPTransporter) SendVoteRequest(server Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { var b bytes.Buffer if _, err := req.Encode(&b); err != nil { traceln("transporter.rv.encoding.error:", err) return nil } - url := fmt.Sprintf("http://%s%s", peer.Name, t.RequestVotePath()) + url := fmt.Sprintf("%s%s", peer.ConnectionString, t.RequestVotePath()) traceln(server.Name(), "POST", url) client := &http.Client{Transport: &http.Transport{DisableKeepAlives: t.DisableKeepAlives}} @@ -143,12 +143,12 @@ func (t *HTTPTransporter) SendVoteRequest(server *Server, peer *Peer, req *Reque } // Sends a SnapshotRequest RPC to a peer. -func (t *HTTPTransporter) SendSnapshotRequest(server *Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse { +func (t *HTTPTransporter) SendSnapshotRequest(server Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse { return nil } // Sends a SnapshotRequest RPC to a peer. -func (t *HTTPTransporter) SendSnapshotRecoveryRequest(server *Server, peer *Peer, req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse { +func (t *HTTPTransporter) SendSnapshotRecoveryRequest(server Server, peer *Peer, req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse { return nil } @@ -157,7 +157,7 @@ func (t *HTTPTransporter) SendSnapshotRecoveryRequest(server *Server, peer *Peer //-------------------------------------- // Handles incoming AppendEntries requests. -func (t *HTTPTransporter) appendEntriesHandler(server *Server) http.HandlerFunc { +func (t *HTTPTransporter) appendEntriesHandler(server Server) http.HandlerFunc { return func(w http.ResponseWriter, r *http.Request) { traceln(server.Name(), "RECV /appendEntries") @@ -176,7 +176,7 @@ func (t *HTTPTransporter) appendEntriesHandler(server *Server) http.HandlerFunc } // Handles incoming RequestVote requests. -func (t *HTTPTransporter) requestVoteHandler(server *Server) http.HandlerFunc { +func (t *HTTPTransporter) requestVoteHandler(server Server) http.HandlerFunc { return func(w http.ResponseWriter, r *http.Request) { traceln(server.Name(), "RECV /requestVote") diff --git a/third_party/github.com/coreos/go-raft/http_transporter_test.go b/third_party/github.com/coreos/go-raft/http_transporter_test.go index 3bd4a6d7435..ab44a523a0f 100644 --- a/third_party/github.com/coreos/go-raft/http_transporter_test.go +++ b/third_party/github.com/coreos/go-raft/http_transporter_test.go @@ -14,8 +14,8 @@ func TestHTTPTransporter(t *testing.T) { transporter := NewHTTPTransporter("/raft") transporter.DisableKeepAlives = true - servers := []*Server{} - f0 := func(server *Server, httpServer *http.Server) { + servers := []Server{} + f0 := func(server Server, httpServer *http.Server) { // Stop the leader and wait for an election. server.Stop() time.Sleep(testElectionTimeout * 2) @@ -25,15 +25,15 @@ func TestHTTPTransporter(t *testing.T) { } server.Start() } - f1 := func(server *Server, httpServer *http.Server) { + f1 := func(server Server, httpServer *http.Server) { } - f2 := func(server *Server, httpServer *http.Server) { + f2 := func(server Server, httpServer *http.Server) { } runTestHttpServers(t, &servers, transporter, f0, f1, f2) } // Starts multiple independent Raft servers wrapped with HTTP servers. -func runTestHttpServers(t *testing.T, servers *[]*Server, transporter *HTTPTransporter, callbacks ...func(*Server, *http.Server)) { +func runTestHttpServers(t *testing.T, servers *[]Server, transporter *HTTPTransporter, callbacks ...func(Server, *http.Server)) { var wg sync.WaitGroup httpServers := []*http.Server{} listeners := []net.Listener{} @@ -68,7 +68,7 @@ func runTestHttpServers(t *testing.T, servers *[]*Server, transporter *HTTPTrans // Setup configuration. for _, server := range *servers { - if _, err := (*servers)[0].Do(&DefaultJoinCommand{Name: server.Name()}); err != nil { + if _, err := (*servers)[0].Do(&DefaultJoinCommand{Name: server.Name(), ConnectionString: fmt.Sprintf("http://%s", server.Name())}); err != nil { t.Fatalf("Server %s unable to join: %v", server.Name(), err) } } @@ -94,7 +94,7 @@ func BenchmarkSpeed(b *testing.B) { transporter := NewHTTPTransporter("/raft") transporter.DisableKeepAlives = true - servers := []*Server{} + servers := []Server{} for i := 0; i < 3; i++ { port := 9000 + i @@ -125,7 +125,7 @@ func BenchmarkSpeed(b *testing.B) { // Setup configuration. for _, server := range servers { - (servers)[0].Do(&DefaultJoinCommand{Name: server.Name()}) + (servers)[0].Do(&DefaultJoinCommand{Name: server.Name(), ConnectionString: fmt.Sprintf("http://%s", server.Name())}) } c := make(chan bool) @@ -145,7 +145,7 @@ func BenchmarkSpeed(b *testing.B) { } } -func send(c chan bool, s *Server) { +func send(c chan bool, s Server) { for i := 0; i < 20; i++ { s.Do(&NOPCommand{}) } diff --git a/third_party/github.com/coreos/go-raft/join_command.go b/third_party/github.com/coreos/go-raft/join_command.go index 949eaae7697..1a43db2c331 100644 --- a/third_party/github.com/coreos/go-raft/join_command.go +++ b/third_party/github.com/coreos/go-raft/join_command.go @@ -3,7 +3,7 @@ package raft // Join command interface type JoinCommand interface { CommandName() string - Apply(server *Server) (interface{}, error) + Apply(server Server) (interface{}, error) NodeName() string } @@ -18,7 +18,7 @@ func (c *DefaultJoinCommand) CommandName() string { return "raft:join" } -func (c *DefaultJoinCommand) Apply(server *Server) (interface{}, error) { +func (c *DefaultJoinCommand) Apply(server Server) (interface{}, error) { err := server.AddPeer(c.Name, c.ConnectionString) return []byte("join"), err diff --git a/third_party/github.com/coreos/go-raft/leave_command.go b/third_party/github.com/coreos/go-raft/leave_command.go index c2a4923a0a2..ca2f148cf2b 100644 --- a/third_party/github.com/coreos/go-raft/leave_command.go +++ b/third_party/github.com/coreos/go-raft/leave_command.go @@ -3,7 +3,7 @@ package raft // Leave command interface type LeaveCommand interface { CommandName() string - Apply(server *Server) (interface{}, error) + Apply(server Server) (interface{}, error) NodeName() string } @@ -17,7 +17,7 @@ func (c *DefaultLeaveCommand) CommandName() string { return "raft:leave" } -func (c *DefaultLeaveCommand) Apply(server *Server) (interface{}, error) { +func (c *DefaultLeaveCommand) Apply(server Server) (interface{}, error) { err := server.RemovePeer(c.Name) return []byte("leave"), err diff --git a/third_party/github.com/coreos/go-raft/nop_command.go b/third_party/github.com/coreos/go-raft/nop_command.go index e3183cdd884..1e567142809 100644 --- a/third_party/github.com/coreos/go-raft/nop_command.go +++ b/third_party/github.com/coreos/go-raft/nop_command.go @@ -13,7 +13,7 @@ func (c NOPCommand) CommandName() string { return "raft:nop" } -func (c NOPCommand) Apply(server *Server) (interface{}, error) { +func (c NOPCommand) Apply(server Server) (interface{}, error) { return nil, nil } diff --git a/third_party/github.com/coreos/go-raft/peer.go b/third_party/github.com/coreos/go-raft/peer.go index 7b116edbb40..516b535e49f 100644 --- a/third_party/github.com/coreos/go-raft/peer.go +++ b/third_party/github.com/coreos/go-raft/peer.go @@ -13,7 +13,7 @@ import ( // A peer is a reference to another server involved in the consensus protocol. type Peer struct { - server *Server + server *server Name string `json:"name"` ConnectionString string `json:"connectionString"` prevLogIndex uint64 @@ -29,7 +29,7 @@ type Peer struct { //------------------------------------------------------------------------------ // Creates a new peer. -func newPeer(server *Server, name string, connectionString string, heartbeatTimeout time.Duration) *Peer { +func newPeer(server *server, name string, connectionString string, heartbeatTimeout time.Duration) *Peer { return &Peer{ server: server, Name: name, diff --git a/third_party/github.com/coreos/go-raft/server.go b/third_party/github.com/coreos/go-raft/server.go index b9b00e697d7..d7dec5b23e1 100644 --- a/third_party/github.com/coreos/go-raft/server.go +++ b/third_party/github.com/coreos/go-raft/server.go @@ -57,7 +57,46 @@ var CommandTimeoutError = errors.New("raft: Command timeout") // A server is involved in the consensus protocol and can act as a follower, // candidate or a leader. -type Server struct { +type Server interface { + Name() string + Context() interface{} + StateMachine() StateMachine + Leader() string + State() string + Path() string + LogPath() string + SnapshotPath(lastIndex uint64, lastTerm uint64) string + Term() uint64 + CommitIndex() uint64 + VotedFor() string + MemberCount() int + QuorumSize() int + IsLogEmpty() bool + LogEntries() []*LogEntry + LastCommandName() string + GetState() string + ElectionTimeout() time.Duration + SetElectionTimeout(duration time.Duration) + HeartbeatTimeout() time.Duration + SetHeartbeatTimeout(duration time.Duration) + Transporter() Transporter + SetTransporter(t Transporter) + AppendEntries(req *AppendEntriesRequest) *AppendEntriesResponse + RequestVote(req *RequestVoteRequest) *RequestVoteResponse + RequestSnapshot(req *SnapshotRequest) *SnapshotResponse + SnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse + AddPeer(name string, connectiongString string) error + RemovePeer(name string) error + Peers() map[string]*Peer + Start() error + Stop() + Running() bool + Do(command Command) (interface{}, error) + TakeSnapshot() error + LoadSnapshot() error +} + +type server struct { name string path string state string @@ -98,7 +137,7 @@ type event struct { //------------------------------------------------------------------------------ // Creates a new server with a log at the given path. -func NewServer(name string, path string, transporter Transporter, stateMachine StateMachine, context interface{}, connectionString string) (*Server, error) { +func NewServer(name string, path string, transporter Transporter, stateMachine StateMachine, context interface{}, connectionString string) (Server, error) { if name == "" { return nil, errors.New("raft.Server: Name cannot be blank") } @@ -106,7 +145,7 @@ func NewServer(name string, path string, transporter Transporter, stateMachine S panic("raft: Transporter required") } - s := &Server{ + s := &server{ name: name, path: path, transporter: transporter, @@ -142,22 +181,22 @@ func NewServer(name string, path string, transporter Transporter, stateMachine S //-------------------------------------- // Retrieves the name of the server. -func (s *Server) Name() string { +func (s *server) Name() string { return s.name } // Retrieves the storage path for the server. -func (s *Server) Path() string { +func (s *server) Path() string { return s.path } // The name of the current leader. -func (s *Server) Leader() string { +func (s *server) Leader() string { return s.leader } // Retrieves a copy of the peer data. -func (s *Server) Peers() map[string]*Peer { +func (s *server) Peers() map[string]*Peer { s.mutex.Lock() defer s.mutex.Unlock() @@ -169,42 +208,42 @@ func (s *Server) Peers() map[string]*Peer { } // Retrieves the object that transports requests. -func (s *Server) Transporter() Transporter { +func (s *server) Transporter() Transporter { s.mutex.RLock() defer s.mutex.RUnlock() return s.transporter } -func (s *Server) SetTransporter(t Transporter) { +func (s *server) SetTransporter(t Transporter) { s.mutex.Lock() defer s.mutex.Unlock() s.transporter = t } // Retrieves the context passed into the constructor. -func (s *Server) Context() interface{} { +func (s *server) Context() interface{} { return s.context } // Retrieves the state machine passed into the constructor. -func (s *Server) StateMachine() StateMachine { +func (s *server) StateMachine() StateMachine { return s.stateMachine } // Retrieves the log path for the server. -func (s *Server) LogPath() string { +func (s *server) LogPath() string { return path.Join(s.path, "log") } // Retrieves the current state of the server. -func (s *Server) State() string { +func (s *server) State() string { s.mutex.RLock() defer s.mutex.RUnlock() return s.state } // Sets the state of the server. -func (s *Server) setState(state string) { +func (s *server) setState(state string) { s.mutex.Lock() defer s.mutex.Unlock() s.state = state @@ -214,44 +253,44 @@ func (s *Server) setState(state string) { } // Retrieves the current term of the server. -func (s *Server) Term() uint64 { +func (s *server) Term() uint64 { return s.currentTerm } // Retrieves the current commit index of the server. -func (s *Server) CommitIndex() uint64 { +func (s *server) CommitIndex() uint64 { return s.log.commitIndex } // Retrieves the name of the candidate this server voted for in this term. -func (s *Server) VotedFor() string { +func (s *server) VotedFor() string { return s.votedFor } // Retrieves whether the server's log has no entries. -func (s *Server) IsLogEmpty() bool { +func (s *server) IsLogEmpty() bool { return s.log.isEmpty() } // A list of all the log entries. This should only be used for debugging purposes. -func (s *Server) LogEntries() []*LogEntry { +func (s *server) LogEntries() []*LogEntry { return s.log.entries } // A reference to the command name of the last entry. -func (s *Server) LastCommandName() string { +func (s *server) LastCommandName() string { return s.log.lastCommandName() } // Get the state of the server for debugging -func (s *Server) GetState() string { +func (s *server) GetState() string { s.mutex.RLock() defer s.mutex.RUnlock() return fmt.Sprintf("Name: %s, State: %s, Term: %v, CommitedIndex: %v ", s.name, s.state, s.currentTerm, s.log.commitIndex) } // Check if the server is promotable -func (s *Server) promotable() bool { +func (s *server) promotable() bool { return s.log.currentIndex() > 0 } @@ -260,14 +299,14 @@ func (s *Server) promotable() bool { //-------------------------------------- // Retrieves the number of member servers in the consensus. -func (s *Server) MemberCount() int { +func (s *server) MemberCount() int { s.mutex.Lock() defer s.mutex.Unlock() return len(s.peers) + 1 } // Retrieves the number of servers required to make a quorum. -func (s *Server) QuorumSize() int { +func (s *server) QuorumSize() int { return (s.MemberCount() / 2) + 1 } @@ -276,14 +315,14 @@ func (s *Server) QuorumSize() int { //-------------------------------------- // Retrieves the election timeout. -func (s *Server) ElectionTimeout() time.Duration { +func (s *server) ElectionTimeout() time.Duration { s.mutex.RLock() defer s.mutex.RUnlock() return s.electionTimeout } // Sets the election timeout. -func (s *Server) SetElectionTimeout(duration time.Duration) { +func (s *server) SetElectionTimeout(duration time.Duration) { s.mutex.Lock() defer s.mutex.Unlock() s.electionTimeout = duration @@ -294,14 +333,14 @@ func (s *Server) SetElectionTimeout(duration time.Duration) { //-------------------------------------- // Retrieves the heartbeat timeout. -func (s *Server) HeartbeatTimeout() time.Duration { +func (s *server) HeartbeatTimeout() time.Duration { s.mutex.RLock() defer s.mutex.RUnlock() return s.heartbeatTimeout } // Sets the heartbeat timeout. -func (s *Server) SetHeartbeatTimeout(duration time.Duration) { +func (s *server) SetHeartbeatTimeout(duration time.Duration) { s.mutex.Lock() defer s.mutex.Unlock() @@ -334,7 +373,7 @@ func init() { // If no log entries exist and a self-join command is issued then // immediately become leader and commit entry. -func (s *Server) Start() error { +func (s *server) Start() error { // Exit if the server is already running. if s.state != Stopped { return errors.New("raft.Server: Server already running") @@ -380,7 +419,7 @@ func (s *Server) Start() error { } // Shuts down the server. -func (s *Server) Stop() { +func (s *server) Stop() { s.send(&stopValue) s.mutex.Lock() defer s.mutex.Unlock() @@ -388,7 +427,7 @@ func (s *Server) Stop() { } // Checks if the server is currently running. -func (s *Server) Running() bool { +func (s *server) Running() bool { s.mutex.RLock() defer s.mutex.RUnlock() return s.state != Stopped @@ -400,7 +439,7 @@ func (s *Server) Running() bool { // Sets the current term for the server. This is only used when an external // current term is found. -func (s *Server) setCurrentTerm(term uint64, leaderName string, append bool) { +func (s *server) setCurrentTerm(term uint64, leaderName string, append bool) { s.mutex.Lock() defer s.mutex.Unlock() @@ -439,7 +478,7 @@ func (s *Server) setCurrentTerm(term uint64, leaderName string, append bool) { // | new leader | | // |_______________________|____________________________________ | // The main event loop for the server -func (s *Server) loop() { +func (s *server) loop() { defer s.debugln("server.loop.end") for { @@ -467,13 +506,13 @@ func (s *Server) loop() { // Sends an event to the event loop to be processed. The function will wait // until the event is actually processed before returning. -func (s *Server) send(value interface{}) (interface{}, error) { +func (s *server) send(value interface{}) (interface{}, error) { event := s.sendAsync(value) err := <-event.c return event.returnValue, err } -func (s *Server) sendAsync(value interface{}) *event { +func (s *server) sendAsync(value interface{}) *event { event := &event{target: value, c: make(chan error, 1)} s.c <- event return event @@ -484,7 +523,7 @@ func (s *Server) sendAsync(value interface{}) *event { // Converts to candidate if election timeout elapses without either: // 1.Receiving valid AppendEntries RPC, or // 2.Granting vote to candidate -func (s *Server) followerLoop() { +func (s *server) followerLoop() { s.setState(Follower) timeoutChan := afterBetween(s.ElectionTimeout(), s.ElectionTimeout()*2) @@ -547,7 +586,7 @@ func (s *Server) followerLoop() { } // The event loop that is run when the server is in a Candidate state. -func (s *Server) candidateLoop() { +func (s *server) candidateLoop() { lastLogIndex, lastLogTerm := s.log.lastInfo() s.leader = "" @@ -630,7 +669,7 @@ func (s *Server) candidateLoop() { } // The event loop that is run when the server is in a Leader state. -func (s *Server) leaderLoop() { +func (s *server) leaderLoop() { s.setState(Leader) s.syncedPeer = make(map[string]bool) logIndex, _ := s.log.lastInfo() @@ -682,7 +721,7 @@ func (s *Server) leaderLoop() { s.syncedPeer = nil } -func (s *Server) snapshotLoop() { +func (s *server) snapshotLoop() { s.setState(Snapshotting) for { @@ -721,12 +760,12 @@ func (s *Server) snapshotLoop() { // Attempts to execute a command and replicate it. The function will return // when the command has been successfully committed or an error has occurred. -func (s *Server) Do(command Command) (interface{}, error) { +func (s *server) Do(command Command) (interface{}, error) { return s.send(command) } // Processes a command. -func (s *Server) processCommand(command Command, e *event) { +func (s *server) processCommand(command Command, e *event) { s.debugln("server.command.process") // Create an entry for the command in the log. @@ -779,14 +818,14 @@ func (s *Server) processCommand(command Command, e *event) { //-------------------------------------- // Appends zero or more log entry from the leader to this server. -func (s *Server) AppendEntries(req *AppendEntriesRequest) *AppendEntriesResponse { +func (s *server) AppendEntries(req *AppendEntriesRequest) *AppendEntriesResponse { ret, _ := s.send(req) resp, _ := ret.(*AppendEntriesResponse) return resp } // Processes the "append entries" request. -func (s *Server) processAppendEntriesRequest(req *AppendEntriesRequest) (*AppendEntriesResponse, bool) { +func (s *server) processAppendEntriesRequest(req *AppendEntriesRequest) (*AppendEntriesResponse, bool) { s.traceln("server.ae.process") @@ -824,7 +863,7 @@ func (s *Server) processAppendEntriesRequest(req *AppendEntriesRequest) (*Append // Processes the "append entries" response from the peer. This is only // processed when the server is a leader. Responses received during other // states are dropped. -func (s *Server) processAppendEntriesResponse(resp *AppendEntriesResponse) { +func (s *server) processAppendEntriesResponse(resp *AppendEntriesResponse) { // If we find a higher term then change to a follower and exit. if resp.Term > s.currentTerm { @@ -888,14 +927,14 @@ func (s *Server) processAppendEntriesResponse(resp *AppendEntriesResponse) { // Requests a vote from a server. A vote can be obtained if the vote's term is // at the server's current term and the server has not made a vote yet. A vote // can also be obtained if the term is greater than the server's current term. -func (s *Server) RequestVote(req *RequestVoteRequest) *RequestVoteResponse { +func (s *server) RequestVote(req *RequestVoteRequest) *RequestVoteResponse { ret, _ := s.send(req) resp, _ := ret.(*RequestVoteResponse) return resp } // Processes a "request vote" request. -func (s *Server) processRequestVoteRequest(req *RequestVoteRequest) (*RequestVoteResponse, bool) { +func (s *server) processRequestVoteRequest(req *RequestVoteRequest) (*RequestVoteResponse, bool) { // If the request is coming from an old term then reject it. if req.Term < s.currentTerm { @@ -933,7 +972,7 @@ func (s *Server) processRequestVoteRequest(req *RequestVoteRequest) (*RequestVot //-------------------------------------- // Adds a peer to the server. -func (s *Server) AddPeer(name string, connectiongString string) error { +func (s *server) AddPeer(name string, connectiongString string) error { s.debugln("server.peer.add: ", name, len(s.peers)) // Do not allow peers to be added twice. @@ -959,7 +998,7 @@ func (s *Server) AddPeer(name string, connectiongString string) error { } // Removes a peer from the server. -func (s *Server) RemovePeer(name string) error { +func (s *server) RemovePeer(name string) error { s.debugln("server.peer.remove: ", name, len(s.peers)) // Skip the Peer if it has the same name as the Server @@ -988,7 +1027,7 @@ func (s *Server) RemovePeer(name string) error { // Log compaction //-------------------------------------- -func (s *Server) TakeSnapshot() error { +func (s *server) TakeSnapshot() error { //TODO put a snapshot mutex s.debugln("take Snapshot") if s.currentSnapshot != nil { @@ -1047,7 +1086,7 @@ func (s *Server) TakeSnapshot() error { } // Retrieves the log path for the server. -func (s *Server) saveSnapshot() error { +func (s *server) saveSnapshot() error { if s.currentSnapshot == nil { return errors.New("no snapshot to save") @@ -1071,17 +1110,17 @@ func (s *Server) saveSnapshot() error { } // Retrieves the log path for the server. -func (s *Server) SnapshotPath(lastIndex uint64, lastTerm uint64) string { +func (s *server) SnapshotPath(lastIndex uint64, lastTerm uint64) string { return path.Join(s.path, "snapshot", fmt.Sprintf("%v_%v.ss", lastTerm, lastIndex)) } -func (s *Server) RequestSnapshot(req *SnapshotRequest) *SnapshotResponse { +func (s *server) RequestSnapshot(req *SnapshotRequest) *SnapshotResponse { ret, _ := s.send(req) resp, _ := ret.(*SnapshotResponse) return resp } -func (s *Server) processSnapshotRequest(req *SnapshotRequest) *SnapshotResponse { +func (s *server) processSnapshotRequest(req *SnapshotRequest) *SnapshotResponse { // If the follower’s log contains an entry at the snapshot’s last index with a term // that matches the snapshot’s last term @@ -1099,13 +1138,13 @@ func (s *Server) processSnapshotRequest(req *SnapshotRequest) *SnapshotResponse return newSnapshotResponse(true) } -func (s *Server) SnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse { +func (s *server) SnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse { ret, _ := s.send(req) resp, _ := ret.(*SnapshotRecoveryResponse) return resp } -func (s *Server) processSnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse { +func (s *server) processSnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse { s.stateMachine.Recovery(req.State) @@ -1136,7 +1175,7 @@ func (s *Server) processSnapshotRecoveryRequest(req *SnapshotRecoveryRequest) *S } // Load a snapshot at restart -func (s *Server) LoadSnapshot() error { +func (s *server) LoadSnapshot() error { dir, err := os.OpenFile(path.Join(s.path, "snapshot"), os.O_RDONLY, 0) if err != nil { @@ -1221,7 +1260,7 @@ func (s *Server) LoadSnapshot() error { // Config File //-------------------------------------- -func (s *Server) writeConf() { +func (s *server) writeConf() { peers := make([]*Peer, len(s.peers)) @@ -1251,7 +1290,7 @@ func (s *Server) writeConf() { } // Read the configuration for the server. -func (s *Server) readConf() error { +func (s *server) readConf() error { confPath := path.Join(s.path, "conf") s.debugln("readConf.open ", confPath) @@ -1277,10 +1316,10 @@ func (s *Server) readConf() error { // Debugging //-------------------------------------- -func (s *Server) debugln(v ...interface{}) { +func (s *server) debugln(v ...interface{}) { debugf("[%s Term:%d] %s", s.name, s.currentTerm, fmt.Sprintln(v...)) } -func (s *Server) traceln(v ...interface{}) { +func (s *server) traceln(v ...interface{}) { tracef("[%s] %s", s.name, fmt.Sprintln(v...)) } diff --git a/third_party/github.com/coreos/go-raft/server_test.go b/third_party/github.com/coreos/go-raft/server_test.go index 01bc96b7159..792ed90f361 100644 --- a/third_party/github.com/coreos/go-raft/server_test.go +++ b/third_party/github.com/coreos/go-raft/server_test.go @@ -37,40 +37,40 @@ func TestServerRequestVote(t *testing.T) { // // Ensure that a vote request is denied if it comes from an old term. func TestServerRequestVoteDeniedForStaleTerm(t *testing.T) { - server := newTestServer("1", &testTransporter{}) + s := newTestServer("1", &testTransporter{}) - server.Start() - if _, err := server.Do(&DefaultJoinCommand{Name: server.Name()}); err != nil { - t.Fatalf("Server %s unable to join: %v", server.Name(), err) + s.Start() + if _, err := s.Do(&DefaultJoinCommand{Name: s.Name()}); err != nil { + t.Fatalf("Server %s unable to join: %v", s.Name(), err) } - server.currentTerm = 2 - defer server.Stop() - resp := server.RequestVote(newRequestVoteRequest(1, "foo", 1, 0)) + s.(*server).currentTerm = 2 + defer s.Stop() + resp := s.RequestVote(newRequestVoteRequest(1, "foo", 1, 0)) if resp.Term != 2 || resp.VoteGranted { t.Fatalf("Invalid request vote response: %v/%v", resp.Term, resp.VoteGranted) } - if server.currentTerm != 2 && server.State() != Follower { - t.Fatalf("Server did not update term and demote: %v / %v", server.currentTerm, server.State()) + if s.Term() != 2 && s.State() != Follower { + t.Fatalf("Server did not update term and demote: %v / %v", s.Term(), s.State()) } } // Ensure that a vote request is denied if we've already voted for a different candidate. func TestServerRequestVoteDeniedIfAlreadyVoted(t *testing.T) { - server := newTestServer("1", &testTransporter{}) + s := newTestServer("1", &testTransporter{}) - server.Start() - if _, err := server.Do(&DefaultJoinCommand{Name: server.Name()}); err != nil { - t.Fatalf("Server %s unable to join: %v", server.Name(), err) + s.Start() + if _, err := s.Do(&DefaultJoinCommand{Name: s.Name()}); err != nil { + t.Fatalf("Server %s unable to join: %v", s.Name(), err) } - server.currentTerm = 2 - defer server.Stop() - resp := server.RequestVote(newRequestVoteRequest(2, "foo", 1, 0)) + s.(*server).currentTerm = 2 + defer s.Stop() + resp := s.RequestVote(newRequestVoteRequest(2, "foo", 1, 0)) if resp.Term != 2 || !resp.VoteGranted { t.Fatalf("First vote should not have been denied") } - resp = server.RequestVote(newRequestVoteRequest(2, "bar", 1, 0)) + resp = s.RequestVote(newRequestVoteRequest(2, "bar", 1, 0)) if resp.Term != 2 || resp.VoteGranted { t.Fatalf("Second vote should have been denied") } @@ -78,24 +78,24 @@ func TestServerRequestVoteDeniedIfAlreadyVoted(t *testing.T) { // Ensure that a vote request is approved if vote occurs in a new term. func TestServerRequestVoteApprovedIfAlreadyVotedInOlderTerm(t *testing.T) { - server := newTestServer("1", &testTransporter{}) + s := newTestServer("1", &testTransporter{}) - server.Start() - if _, err := server.Do(&DefaultJoinCommand{Name: server.Name()}); err != nil { - t.Fatalf("Server %s unable to join: %v", server.Name(), err) + s.Start() + if _, err := s.Do(&DefaultJoinCommand{Name: s.Name()}); err != nil { + t.Fatalf("Server %s unable to join: %v", s.Name(), err) } time.Sleep(time.Millisecond * 100) - server.currentTerm = 2 - defer server.Stop() - resp := server.RequestVote(newRequestVoteRequest(2, "foo", 2, 1)) - if resp.Term != 2 || !resp.VoteGranted || server.VotedFor() != "foo" { + s.(*server).currentTerm = 2 + defer s.Stop() + resp := s.RequestVote(newRequestVoteRequest(2, "foo", 2, 1)) + if resp.Term != 2 || !resp.VoteGranted || s.VotedFor() != "foo" { t.Fatalf("First vote should not have been denied") } - resp = server.RequestVote(newRequestVoteRequest(3, "bar", 2, 1)) + resp = s.RequestVote(newRequestVoteRequest(3, "bar", 2, 1)) - if resp.Term != 3 || !resp.VoteGranted || server.VotedFor() != "bar" { + if resp.Term != 3 || !resp.VoteGranted || s.VotedFor() != "bar" { t.Fatalf("Second vote should have been approved") } } @@ -106,33 +106,32 @@ func TestServerRequestVoteDenyIfCandidateLogIsBehind(t *testing.T) { e0, _ := newLogEntry(tmpLog, 1, 1, &testCommand1{Val: "foo", I: 20}) e1, _ := newLogEntry(tmpLog, 2, 1, &testCommand2{X: 100}) e2, _ := newLogEntry(tmpLog, 3, 2, &testCommand1{Val: "bar", I: 0}) - server := newTestServerWithLog("1", &testTransporter{}, []*LogEntry{e0, e1, e2}) + s := newTestServerWithLog("1", &testTransporter{}, []*LogEntry{e0, e1, e2}) // start as a follower with term 2 and index 3 - server.Start() - - defer server.Stop() + s.Start() + defer s.Stop() // request vote from term 3 with last log entry 2, 2 - resp := server.RequestVote(newRequestVoteRequest(3, "foo", 2, 2)) + resp := s.RequestVote(newRequestVoteRequest(3, "foo", 2, 2)) if resp.Term != 3 || resp.VoteGranted { t.Fatalf("Stale index vote should have been denied [%v/%v]", resp.Term, resp.VoteGranted) } // request vote from term 2 with last log entry 2, 3 - resp = server.RequestVote(newRequestVoteRequest(2, "foo", 3, 2)) + resp = s.RequestVote(newRequestVoteRequest(2, "foo", 3, 2)) if resp.Term != 3 || resp.VoteGranted { t.Fatalf("Stale term vote should have been denied [%v/%v]", resp.Term, resp.VoteGranted) } // request vote from term 3 with last log entry 2, 3 - resp = server.RequestVote(newRequestVoteRequest(3, "foo", 3, 2)) + resp = s.RequestVote(newRequestVoteRequest(3, "foo", 3, 2)) if resp.Term != 3 || !resp.VoteGranted { t.Fatalf("Matching log vote should have been granted") } // request vote from term 3 with last log entry 2, 4 - resp = server.RequestVote(newRequestVoteRequest(3, "foo", 4, 2)) + resp = s.RequestVote(newRequestVoteRequest(3, "foo", 4, 2)) if resp.Term != 3 || !resp.VoteGranted { t.Fatalf("Ahead-of-log vote should have been granted") } @@ -145,28 +144,27 @@ func TestServerRequestVoteDenyIfCandidateLogIsBehind(t *testing.T) { // // Ensure that we can self-promote a server to candidate, obtain votes and become a fearless leader. func TestServerPromoteSelf(t *testing.T) { e0, _ := newLogEntry(newLog(), 1, 1, &testCommand1{Val: "foo", I: 20}) - server := newTestServerWithLog("1", &testTransporter{}, []*LogEntry{e0}) + s := newTestServerWithLog("1", &testTransporter{}, []*LogEntry{e0}) // start as a follower - server.Start() - - defer server.Stop() + s.Start() + defer s.Stop() time.Sleep(2 * testElectionTimeout) - if server.State() != Leader { - t.Fatalf("Server self-promotion failed: %v", server.State()) + if s.State() != Leader { + t.Fatalf("Server self-promotion failed: %v", s.State()) } } //Ensure that we can promote a server within a cluster to a leader. func TestServerPromote(t *testing.T) { - lookup := map[string]*Server{} + lookup := map[string]Server{} transporter := &testTransporter{} - transporter.sendVoteRequestFunc = func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { + transporter.sendVoteRequestFunc = func(s Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { return lookup[peer.Name].RequestVote(req) } - transporter.sendAppendEntriesRequestFunc = func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { + transporter.sendAppendEntriesRequestFunc = func(s Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { return lookup[peer.Name].AppendEntries(req) } servers := newTestCluster([]string{"1", "2", "3"}, transporter, lookup) @@ -180,8 +178,8 @@ func TestServerPromote(t *testing.T) { if servers[0].State() != Leader && servers[1].State() != Leader && servers[2].State() != Leader { t.Fatalf("No leader elected: (%s, %s, %s)", servers[0].State(), servers[1].State(), servers[2].State()) } - for _, server := range servers { - server.Stop() + for _, s := range servers { + s.Stop() } } @@ -191,20 +189,20 @@ func TestServerPromote(t *testing.T) { // Ensure we can append entries to a server. func TestServerAppendEntries(t *testing.T) { - server := newTestServer("1", &testTransporter{}) + s := newTestServer("1", &testTransporter{}) - server.SetHeartbeatTimeout(time.Second * 10) - server.Start() - defer server.Stop() + s.SetHeartbeatTimeout(time.Second * 10) + s.Start() + defer s.Stop() // Append single entry. e, _ := newLogEntry(nil, 1, 1, &testCommand1{Val: "foo", I: 10}) entries := []*LogEntry{e} - resp := server.AppendEntries(newAppendEntriesRequest(1, 0, 0, 0, "ldr", entries)) + resp := s.AppendEntries(newAppendEntriesRequest(1, 0, 0, 0, "ldr", entries)) if resp.Term != 1 || !resp.Success { t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success) } - if index, term := server.log.commitInfo(); index != 0 || term != 0 { + if index, term := s.(*server).log.commitInfo(); index != 0 || term != 0 { t.Fatalf("Invalid commit info [IDX=%v, TERM=%v]", index, term) } @@ -212,57 +210,56 @@ func TestServerAppendEntries(t *testing.T) { e1, _ := newLogEntry(nil, 2, 1, &testCommand1{Val: "bar", I: 20}) e2, _ := newLogEntry(nil, 3, 1, &testCommand1{Val: "baz", I: 30}) entries = []*LogEntry{e1, e2} - resp = server.AppendEntries(newAppendEntriesRequest(1, 1, 1, 1, "ldr", entries)) + resp = s.AppendEntries(newAppendEntriesRequest(1, 1, 1, 1, "ldr", entries)) if resp.Term != 1 || !resp.Success { t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success) } - if index, term := server.log.commitInfo(); index != 1 || term != 1 { + if index, term := s.(*server).log.commitInfo(); index != 1 || term != 1 { t.Fatalf("Invalid commit info [IDX=%v, TERM=%v]", index, term) } // Send zero entries and commit everything. - resp = server.AppendEntries(newAppendEntriesRequest(2, 3, 1, 3, "ldr", []*LogEntry{})) + resp = s.AppendEntries(newAppendEntriesRequest(2, 3, 1, 3, "ldr", []*LogEntry{})) if resp.Term != 2 || !resp.Success { t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success) } - if index, term := server.log.commitInfo(); index != 3 || term != 1 { + if index, term := s.(*server).log.commitInfo(); index != 3 || term != 1 { t.Fatalf("Invalid commit info [IDX=%v, TERM=%v]", index, term) } } //Ensure that entries with stale terms are rejected. func TestServerAppendEntriesWithStaleTermsAreRejected(t *testing.T) { - server := newTestServer("1", &testTransporter{}) + s := newTestServer("1", &testTransporter{}) - server.Start() + s.Start() - defer server.Stop() - server.currentTerm = 2 + defer s.Stop() + s.(*server).currentTerm = 2 // Append single entry. e, _ := newLogEntry(nil, 1, 1, &testCommand1{Val: "foo", I: 10}) entries := []*LogEntry{e} - resp := server.AppendEntries(newAppendEntriesRequest(1, 0, 0, 0, "ldr", entries)) + resp := s.AppendEntries(newAppendEntriesRequest(1, 0, 0, 0, "ldr", entries)) if resp.Term != 2 || resp.Success { t.Fatalf("AppendEntries should have failed: %v/%v", resp.Term, resp.Success) } - if index, term := server.log.commitInfo(); index != 0 || term != 0 { + if index, term := s.(*server).log.commitInfo(); index != 0 || term != 0 { t.Fatalf("Invalid commit info [IDX=%v, TERM=%v]", index, term) } } // Ensure that we reject entries if the commit log is different. func TestServerAppendEntriesRejectedIfAlreadyCommitted(t *testing.T) { - server := newTestServer("1", &testTransporter{}) - server.Start() - - defer server.Stop() + s := newTestServer("1", &testTransporter{}) + s.Start() + defer s.Stop() // Append single entry + commit. e1, _ := newLogEntry(nil, 1, 1, &testCommand1{Val: "foo", I: 10}) e2, _ := newLogEntry(nil, 2, 1, &testCommand1{Val: "foo", I: 15}) entries := []*LogEntry{e1, e2} - resp := server.AppendEntries(newAppendEntriesRequest(1, 0, 0, 2, "ldr", entries)) + resp := s.AppendEntries(newAppendEntriesRequest(1, 0, 0, 2, "ldr", entries)) if resp.Term != 1 || !resp.Success { t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success) } @@ -270,7 +267,7 @@ func TestServerAppendEntriesRejectedIfAlreadyCommitted(t *testing.T) { // Append entry again (post-commit). e, _ := newLogEntry(nil, 2, 1, &testCommand1{Val: "bar", I: 20}) entries = []*LogEntry{e} - resp = server.AppendEntries(newAppendEntriesRequest(1, 2, 1, 1, "ldr", entries)) + resp = s.AppendEntries(newAppendEntriesRequest(1, 2, 1, 1, "ldr", entries)) if resp.Term != 1 || resp.Success { t.Fatalf("AppendEntries should have failed: %v/%v", resp.Term, resp.Success) } @@ -278,9 +275,9 @@ func TestServerAppendEntriesRejectedIfAlreadyCommitted(t *testing.T) { // Ensure that we uncommitted entries are rolled back if new entries overwrite them. func TestServerAppendEntriesOverwritesUncommittedEntries(t *testing.T) { - server := newTestServer("1", &testTransporter{}) - server.Start() - defer server.Stop() + s := newTestServer("1", &testTransporter{}) + s.Start() + defer s.Stop() entry1, _ := newLogEntry(nil, 1, 1, &testCommand1{Val: "foo", I: 10}) entry2, _ := newLogEntry(nil, 2, 1, &testCommand1{Val: "foo", I: 15}) @@ -288,15 +285,15 @@ func TestServerAppendEntriesOverwritesUncommittedEntries(t *testing.T) { // Append single entry + commit. entries := []*LogEntry{entry1, entry2} - resp := server.AppendEntries(newAppendEntriesRequest(1, 0, 0, 1, "ldr", entries)) - if resp.Term != 1 || !resp.Success || server.log.commitIndex != 1 || !reflect.DeepEqual(server.log.entries, []*LogEntry{entry1, entry2}) { + resp := s.AppendEntries(newAppendEntriesRequest(1, 0, 0, 1, "ldr", entries)) + if resp.Term != 1 || !resp.Success || s.(*server).log.commitIndex != 1 || !reflect.DeepEqual(s.(*server).log.entries, []*LogEntry{entry1, entry2}) { t.Fatalf("AppendEntries failed: %v/%v", resp.Term, resp.Success) } // Append entry that overwrites the second (uncommitted) entry. entries = []*LogEntry{entry3} - resp = server.AppendEntries(newAppendEntriesRequest(2, 1, 1, 2, "ldr", entries)) - if resp.Term != 2 || !resp.Success || server.log.commitIndex != 2 || !reflect.DeepEqual(server.log.entries, []*LogEntry{entry1, entry3}) { + resp = s.AppendEntries(newAppendEntriesRequest(2, 1, 1, 2, "ldr", entries)) + if resp.Term != 2 || !resp.Success || s.(*server).log.commitIndex != 2 || !reflect.DeepEqual(s.(*server).log.entries, []*LogEntry{entry1, entry3}) { t.Fatalf("AppendEntries should have succeeded: %v/%v", resp.Term, resp.Success) } } @@ -307,11 +304,11 @@ func TestServerAppendEntriesOverwritesUncommittedEntries(t *testing.T) { // Ensure that a follower cannot execute a command. func TestServerDenyCommandExecutionWhenFollower(t *testing.T) { - server := newTestServer("1", &testTransporter{}) - server.Start() - defer server.Stop() + s := newTestServer("1", &testTransporter{}) + s.Start() + defer s.Stop() var err error - if _, err = server.Do(&testCommand1{Val: "foo", I: 10}); err != NotLeaderError { + if _, err = s.Do(&testCommand1{Val: "foo", I: 10}); err != NotLeaderError { t.Fatalf("Expected error: %v, got: %v", NotLeaderError, err) } } @@ -324,27 +321,27 @@ func TestServerDenyCommandExecutionWhenFollower(t *testing.T) { func TestServerRecoverFromPreviousLogAndConf(t *testing.T) { // Initialize the servers. var mutex sync.RWMutex - servers := map[string]*Server{} + servers := map[string]Server{} transporter := &testTransporter{} - transporter.sendVoteRequestFunc = func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { + transporter.sendVoteRequestFunc = func(s Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { mutex.RLock() - s := servers[peer.Name] + target := servers[peer.Name] mutex.RUnlock() - return s.RequestVote(req) + return target.RequestVote(req) } - transporter.sendAppendEntriesRequestFunc = func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { + transporter.sendAppendEntriesRequestFunc = func(s Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { mutex.RLock() - s := servers[peer.Name] + target := servers[peer.Name] mutex.RUnlock() - return s.AppendEntries(req) + return target.AppendEntries(req) } disTransporter := &testTransporter{} - disTransporter.sendVoteRequestFunc = func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { + disTransporter.sendVoteRequestFunc = func(s Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { return nil } - disTransporter.sendAppendEntriesRequestFunc = func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { + disTransporter.sendAppendEntriesRequestFunc = func(s Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { return nil } @@ -358,22 +355,22 @@ func TestServerRecoverFromPreviousLogAndConf(t *testing.T) { names = append(names, strconv.Itoa(i)) } - var leader *Server + var leader Server for _, name := range names { - server := newTestServer(name, transporter) + s := newTestServer(name, transporter) - servers[name] = server - paths[name] = server.Path() + servers[name] = s + paths[name] = s.Path() if name == "1" { - leader = server - server.SetHeartbeatTimeout(testHeartbeatTimeout) - server.Start() + leader = s + s.SetHeartbeatTimeout(testHeartbeatTimeout) + s.Start() time.Sleep(testHeartbeatTimeout) } else { - server.SetElectionTimeout(testElectionTimeout) - server.SetHeartbeatTimeout(testHeartbeatTimeout) - server.Start() + s.SetElectionTimeout(testElectionTimeout) + s.SetHeartbeatTimeout(testHeartbeatTimeout) + s.Start() time.Sleep(testHeartbeatTimeout) } if _, err := leader.Do(&DefaultJoinCommand{Name: name}); err != nil { @@ -385,35 +382,35 @@ func TestServerRecoverFromPreviousLogAndConf(t *testing.T) { // commit some commands for i := 0; i < 10; i++ { if _, err := leader.Do(&testCommand2{X: 1}); err != nil { - t.Fatalf("cannot commit command:", err.Error()) + t.Fatalf("cannot commit command: %s", err.Error()) } } time.Sleep(2 * testHeartbeatTimeout) for _, name := range names { - server := servers[name] - if server.CommitIndex() != 16 { - t.Fatalf("%s commitIndex is invalid [%d/%d]", name, server.CommitIndex(), 16) + s := servers[name] + if s.CommitIndex() != 16 { + t.Fatalf("%s commitIndex is invalid [%d/%d]", name, s.CommitIndex(), 16) } - server.Stop() + s.Stop() } for _, name := range names { // with old path and disable transportation - server := newTestServerWithPath(name, disTransporter, paths[name]) - servers[name] = server + s := newTestServerWithPath(name, disTransporter, paths[name]) + servers[name] = s - server.Start() + s.Start() // should only commit to the last join command - if server.CommitIndex() != 6 { - t.Fatalf("%s recover phase 1 commitIndex is invalid [%d/%d]", name, server.CommitIndex(), 6) + if s.CommitIndex() != 6 { + t.Fatalf("%s recover phase 1 commitIndex is invalid [%d/%d]", name, s.CommitIndex(), 6) } // peer conf should be recovered - if len(server.Peers()) != 4 { - t.Fatalf("%s recover phase 1 peer failed! [%d/%d]", name, len(server.Peers()), 4) + if len(s.Peers()) != 4 { + t.Fatalf("%s recover phase 1 peer failed! [%d/%d]", name, len(s.Peers()), 4) } } @@ -426,11 +423,11 @@ func TestServerRecoverFromPreviousLogAndConf(t *testing.T) { // should commit to the previous index + 1(nop command when new leader elected) for _, name := range names { - server := servers[name] - if server.CommitIndex() != 17 { - t.Fatalf("%s commitIndex is invalid [%d/%d]", name, server.CommitIndex(), 17) + s := servers[name] + if s.CommitIndex() != 17 { + t.Fatalf("%s commitIndex is invalid [%d/%d]", name, s.CommitIndex(), 17) } - server.Stop() + s.Stop() } } @@ -440,29 +437,29 @@ func TestServerRecoverFromPreviousLogAndConf(t *testing.T) { // Ensure that we can start a single server and append to its log. func TestServerSingleNode(t *testing.T) { - server := newTestServer("1", &testTransporter{}) - if server.State() != Stopped { - t.Fatalf("Unexpected server state: %v", server.State()) + s := newTestServer("1", &testTransporter{}) + if s.State() != Stopped { + t.Fatalf("Unexpected server state: %v", s.State()) } - server.Start() + s.Start() time.Sleep(testHeartbeatTimeout) // Join the server to itself. - if _, err := server.Do(&DefaultJoinCommand{Name: "1"}); err != nil { + if _, err := s.Do(&DefaultJoinCommand{Name: "1"}); err != nil { t.Fatalf("Unable to join: %v", err) } debugln("finish command") - if server.State() != Leader { - t.Fatalf("Unexpected server state: %v", server.State()) + if s.State() != Leader { + t.Fatalf("Unexpected server state: %v", s.State()) } - server.Stop() + s.Stop() - if server.State() != Stopped { - t.Fatalf("Unexpected server state: %v", server.State()) + if s.State() != Stopped { + t.Fatalf("Unexpected server state: %v", s.State()) } } @@ -470,27 +467,27 @@ func TestServerSingleNode(t *testing.T) { func TestServerMultiNode(t *testing.T) { // Initialize the servers. var mutex sync.RWMutex - servers := map[string]*Server{} + servers := map[string]Server{} transporter := &testTransporter{} - transporter.sendVoteRequestFunc = func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { + transporter.sendVoteRequestFunc = func(s Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { mutex.RLock() - s := servers[peer.Name] + target := servers[peer.Name] mutex.RUnlock() - return s.RequestVote(req) + return target.RequestVote(req) } - transporter.sendAppendEntriesRequestFunc = func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { + transporter.sendAppendEntriesRequestFunc = func(s Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { mutex.RLock() - s := servers[peer.Name] + target := servers[peer.Name] mutex.RUnlock() - return s.AppendEntries(req) + return target.AppendEntries(req) } disTransporter := &testTransporter{} - disTransporter.sendVoteRequestFunc = func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { + disTransporter.sendVoteRequestFunc = func(s Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { return nil } - disTransporter.sendAppendEntriesRequestFunc = func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { + disTransporter.sendAppendEntriesRequestFunc = func(s Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { return nil } @@ -503,24 +500,24 @@ func TestServerMultiNode(t *testing.T) { names = append(names, strconv.Itoa(i)) } - var leader *Server + var leader Server for _, name := range names { - server := newTestServer(name, transporter) - defer server.Stop() + s := newTestServer(name, transporter) + defer s.Stop() mutex.Lock() - servers[name] = server + servers[name] = s mutex.Unlock() if name == "1" { - leader = server - server.SetHeartbeatTimeout(testHeartbeatTimeout) - server.Start() + leader = s + s.SetHeartbeatTimeout(testHeartbeatTimeout) + s.Start() time.Sleep(testHeartbeatTimeout) } else { - server.SetElectionTimeout(testElectionTimeout) - server.SetHeartbeatTimeout(testHeartbeatTimeout) - server.Start() + s.SetElectionTimeout(testElectionTimeout) + s.SetHeartbeatTimeout(testHeartbeatTimeout) + s.Start() time.Sleep(testHeartbeatTimeout) } if _, err := leader.Do(&DefaultJoinCommand{Name: name}); err != nil { @@ -536,7 +533,7 @@ func TestServerMultiNode(t *testing.T) { t.Fatalf("Expected member count to be %v, got %v", n, leader.MemberCount()) } if servers["2"].State() == Leader || servers["3"].State() == Leader { - t.Fatalf("Expected leader should be 1: 2=%v, 3=%v\n", servers["2"].state, servers["3"].state) + t.Fatalf("Expected leader should be 1: 2=%v, 3=%v\n", servers["2"].State(), servers["3"].State()) } mutex.RUnlock() @@ -573,7 +570,7 @@ func TestServerMultiNode(t *testing.T) { } debugln("[Test] Done") } - debugln("Leader is ", value.Name(), " Index ", value.log.commitIndex) + debugln("Leader is ", value.Name(), " Index ", value.(*server).log.commitIndex) } debugln("Not Found leader") } @@ -584,7 +581,7 @@ func TestServerMultiNode(t *testing.T) { if value.State() == Leader { leader++ } - debugln(value.Name(), " ", value.currentTerm, " ", value.state) + debugln(value.Name(), " ", value.(*server).Term(), " ", value.State()) } } diff --git a/third_party/github.com/coreos/go-raft/snapshot.go b/third_party/github.com/coreos/go-raft/snapshot.go index 93b1a97cd7b..4f416f741a4 100644 --- a/third_party/github.com/coreos/go-raft/snapshot.go +++ b/third_party/github.com/coreos/go-raft/snapshot.go @@ -20,9 +20,9 @@ type Snapshot struct { LastIndex uint64 `json:"lastIndex"` LastTerm uint64 `json:"lastTerm"` // cluster configuration. - Peers []*Peer `json: "peers"` - State []byte `json: "state"` - Path string `json: "path"` + Peers []*Peer `json:"peers"` + State []byte `json:"state"` + Path string `json:"path"` } // Save the snapshot to a file diff --git a/third_party/github.com/coreos/go-raft/snapshot_recovery_request.go b/third_party/github.com/coreos/go-raft/snapshot_recovery_request.go index a05f4310815..275ff41f3c7 100644 --- a/third_party/github.com/coreos/go-raft/snapshot_recovery_request.go +++ b/third_party/github.com/coreos/go-raft/snapshot_recovery_request.go @@ -80,7 +80,7 @@ func (req *SnapshotRecoveryRequest) Decode(r io.Reader) (int, error) { req.LeaderName = pb.GetLeaderName() req.LastIndex = pb.GetLastIndex() req.LastTerm = pb.GetLastTerm() - req.State = req.State + req.State = pb.GetState() req.Peers = make([]*Peer, len(pb.Peers)) diff --git a/third_party/github.com/coreos/go-raft/test.go b/third_party/github.com/coreos/go-raft/test.go index 95a6c7168bd..5b323f749bb 100644 --- a/third_party/github.com/coreos/go-raft/test.go +++ b/third_party/github.com/coreos/go-raft/test.go @@ -60,7 +60,7 @@ func setupLog(entries []*LogEntry) (*Log, string) { // Servers //-------------------------------------- -func newTestServer(name string, transporter Transporter) *Server { +func newTestServer(name string, transporter Transporter) Server { p, _ := ioutil.TempDir("", "raft-server-") if err := os.MkdirAll(p, 0644); err != nil { panic(err.Error()) @@ -69,12 +69,12 @@ func newTestServer(name string, transporter Transporter) *Server { return server } -func newTestServerWithPath(name string, transporter Transporter, p string) *Server { +func newTestServerWithPath(name string, transporter Transporter, p string) Server { server, _ := NewServer(name, p, transporter, nil, nil, "") return server } -func newTestServerWithLog(name string, transporter Transporter, entries []*LogEntry) *Server { +func newTestServerWithLog(name string, transporter Transporter, entries []*LogEntry) Server { server := newTestServer(name, transporter) f, err := os.Create(server.LogPath()) if err != nil { @@ -88,8 +88,8 @@ func newTestServerWithLog(name string, transporter Transporter, entries []*LogEn return server } -func newTestCluster(names []string, transporter Transporter, lookup map[string]*Server) []*Server { - servers := []*Server{} +func newTestCluster(names []string, transporter Transporter, lookup map[string]Server) []Server { + servers := []Server{} e0, _ := newLogEntry(newLog(), 1, 1, &testCommand1{Val: "foo", I: 20}) for _, name := range names { @@ -116,24 +116,24 @@ func newTestCluster(names []string, transporter Transporter, lookup map[string]* //-------------------------------------- type testTransporter struct { - sendVoteRequestFunc func(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse - sendAppendEntriesRequestFunc func(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse - sendSnapshotRequestFunc func(server *Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse + sendVoteRequestFunc func(server Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse + sendAppendEntriesRequestFunc func(server Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse + sendSnapshotRequestFunc func(server Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse } -func (t *testTransporter) SendVoteRequest(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { +func (t *testTransporter) SendVoteRequest(server Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse { return t.sendVoteRequestFunc(server, peer, req) } -func (t *testTransporter) SendAppendEntriesRequest(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { +func (t *testTransporter) SendAppendEntriesRequest(server Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse { return t.sendAppendEntriesRequestFunc(server, peer, req) } -func (t *testTransporter) SendSnapshotRequest(server *Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse { +func (t *testTransporter) SendSnapshotRequest(server Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse { return t.sendSnapshotRequestFunc(server, peer, req) } -func (t *testTransporter) SendSnapshotRecoveryRequest(server *Server, peer *Peer, req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse { +func (t *testTransporter) SendSnapshotRecoveryRequest(server Server, peer *Peer, req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse { return t.SendSnapshotRecoveryRequest(server, peer, req) } @@ -163,7 +163,7 @@ func (c *testCommand1) CommandName() string { return "cmd_1" } -func (c *testCommand1) Apply(server *Server) (interface{}, error) { +func (c *testCommand1) Apply(server Server) (interface{}, error) { return nil, nil } @@ -179,6 +179,6 @@ func (c *testCommand2) CommandName() string { return "cmd_2" } -func (c *testCommand2) Apply(server *Server) (interface{}, error) { +func (c *testCommand2) Apply(server Server) (interface{}, error) { return nil, nil } diff --git a/third_party/github.com/coreos/go-raft/transporter.go b/third_party/github.com/coreos/go-raft/transporter.go index f7d51e527fd..faf95edd78d 100644 --- a/third_party/github.com/coreos/go-raft/transporter.go +++ b/third_party/github.com/coreos/go-raft/transporter.go @@ -9,8 +9,8 @@ package raft // Transporter is the interface for allowing the host application to transport // requests to other nodes. type Transporter interface { - SendVoteRequest(server *Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse - SendAppendEntriesRequest(server *Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse - SendSnapshotRequest(server *Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse - SendSnapshotRecoveryRequest(server *Server, peer *Peer, req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse + SendVoteRequest(server Server, peer *Peer, req *RequestVoteRequest) *RequestVoteResponse + SendAppendEntriesRequest(server Server, peer *Peer, req *AppendEntriesRequest) *AppendEntriesResponse + SendSnapshotRequest(server Server, peer *Peer, req *SnapshotRequest) *SnapshotResponse + SendSnapshotRecoveryRequest(server Server, peer *Peer, req *SnapshotRecoveryRequest) *SnapshotRecoveryResponse } diff --git a/web/web.go b/web/web.go index 1ce9d3fe582..723eb05c8a9 100644 --- a/web/web.go +++ b/web/web.go @@ -23,7 +23,7 @@ func mainHandler(c http.ResponseWriter, req *http.Request) { mainTempl.Execute(c, p) } -func Start(raftServer *raft.Server, webURL string) { +func Start(raftServer raft.Server, webURL string) { u, _ := url.Parse(webURL) webMux := http.NewServeMux() From c87a7a039e3fad6f5e9d9bde8f38ea297c47bc4f Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 14 Oct 2013 12:57:35 -0700 Subject: [PATCH 105/247] feat test both of the condidtion --- store/store.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/store/store.go b/store/store.go index ade44acd992..2ce2fc902ec 100644 --- a/store/store.go +++ b/store/store.go @@ -162,7 +162,9 @@ func (s *Store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term) } - if n.Value == prevValue || n.ModifiedIndex == prevIndex { + // If both of the prevValue and prevIndex are given, we will test both of them. + // Command will be executed, only if both of the tests are successful. + if (prevValue == "" || n.Value == prevValue) && (prevIndex == 0 || n.ModifiedIndex == prevIndex) { e := newEvent(TestAndSet, nodePath, index, term) e.PrevValue = n.Value From fb7a91739ad9d7764d4959ecd797293a810d1731 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 14 Oct 2013 14:20:50 -0600 Subject: [PATCH 106/247] Remove functional tests. --- .travis.yml | 2 +- etcd_test.go | 601 --------------------------------------------------- test.sh | 4 + test/test.go | 205 ------------------ 4 files changed, 5 insertions(+), 807 deletions(-) delete mode 100644 etcd_test.go delete mode 100644 test/test.go diff --git a/.travis.yml b/.travis.yml index 50eca08753d..1c07835fddb 100644 --- a/.travis.yml +++ b/.travis.yml @@ -2,7 +2,7 @@ language: go go: 1.1 install: - - echo "Skip install" + - go get -u github.com/coreos/etcd-test-runner script: - ./test.sh diff --git a/etcd_test.go b/etcd_test.go deleted file mode 100644 index 30df557fdeb..00000000000 --- a/etcd_test.go +++ /dev/null @@ -1,601 +0,0 @@ -package main - -import ( - "fmt" - "math/rand" - "net/http" - "net/http/httptest" - "net/url" - "os" - "strconv" - "strings" - "testing" - "time" - - "github.com/coreos/etcd/server" - "github.com/coreos/etcd/test" - "github.com/coreos/go-etcd/etcd" -) - -// Create a single node and try to set value -func TestSingleNode(t *testing.T) { - procAttr := new(os.ProcAttr) - procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} - args := []string{"etcd", "-n=node1", "-f", "-d=/tmp/node1"} - - process, err := os.StartProcess("etcd", args, procAttr) - if err != nil { - t.Fatal("start process failed:" + err.Error()) - return - } - defer process.Kill() - - time.Sleep(time.Second) - - c := etcd.NewClient(nil) - - c.SyncCluster() - // Test Set - result, err := c.Set("foo", "bar", 100) - - if err != nil || result.Key != "/foo" || result.Value != "bar" || result.TTL < 95 { - if err != nil { - t.Fatal("Set 1: ", err) - } - - t.Fatalf("Set 1 failed with %s %s %v", result.Key, result.Value, result.TTL) - } - - time.Sleep(time.Second) - - result, err = c.Set("foo", "bar", 100) - - if err != nil || result.Key != "/foo" || result.Value != "bar" || result.PrevValue != "bar" || result.TTL != 100 { - if err != nil { - t.Fatal("Set 2: ", err) - } - t.Fatalf("Set 2 failed with %s %s %v", result.Key, result.Value, result.TTL) - } - - // Add a test-and-set test - - // First, we'll test we can change the value if we get it write - result, match, err := c.TestAndSet("foo", "bar", "foobar", 100) - - if err != nil || result.Key != "/foo" || result.Value != "foobar" || result.PrevValue != "bar" || result.TTL != 100 || !match { - if err != nil { - t.Fatal(err) - } - t.Fatalf("Set 3 failed with %s %s %v", result.Key, result.Value, result.TTL) - } - - // Next, we'll make sure we can't set it without the correct prior value - _, _, err = c.TestAndSet("foo", "bar", "foofoo", 100) - - if err == nil { - t.Fatalf("Set 4 expecting error when setting key with incorrect previous value") - } - - // Finally, we'll make sure a blank previous value still counts as a test-and-set and still has to match - _, _, err = c.TestAndSet("foo", "", "barbar", 100) - - if err == nil { - t.Fatalf("Set 5 expecting error when setting key with blank (incorrect) previous value") - } -} - -// TestInternalVersionFail will ensure that etcd does not come up if the internal raft -// versions do not match. -func TestInternalVersionFail(t *testing.T) { - checkedVersion := false - testMux := http.NewServeMux() - - testMux.HandleFunc("/version", func(w http.ResponseWriter, r *http.Request) { - fmt.Fprintln(w, "This is not a version number") - checkedVersion = true - }) - - testMux.HandleFunc("/join", func(w http.ResponseWriter, r *http.Request) { - t.Fatal("should not attempt to join!") - }) - - ts := httptest.NewServer(testMux) - defer ts.Close() - - fakeURL, _ := url.Parse(ts.URL) - - procAttr := new(os.ProcAttr) - procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} - args := []string{"etcd", "-n=node1", "-f", "-d=/tmp/node1", "-C=" + fakeURL.Host} - - process, err := os.StartProcess("etcd", args, procAttr) - if err != nil { - t.Fatal("start process failed:" + err.Error()) - return - } - defer process.Kill() - - time.Sleep(time.Second) - - _, err = http.Get("http://127.0.0.1:4001") - - if err == nil { - t.Fatal("etcd node should not be up") - return - } - - if checkedVersion == false { - t.Fatal("etcd did not check the version") - return - } -} - -// This test creates a single node and then set a value to it. -// Then this test kills the node and restart it and tries to get the value again. -func TestSingleNodeRecovery(t *testing.T) { - procAttr := new(os.ProcAttr) - procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} - args := []string{"etcd", "-n=node1", "-d=/tmp/node1"} - - process, err := os.StartProcess("etcd", append(args, "-f"), procAttr) - if err != nil { - t.Fatal("start process failed:" + err.Error()) - return - } - - time.Sleep(time.Second) - - c := etcd.NewClient(nil) - - c.SyncCluster() - // Test Set - result, err := c.Set("foo", "bar", 100) - - if err != nil || result.Key != "/foo" || result.Value != "bar" || result.TTL < 95 { - if err != nil { - t.Fatal(err) - } - - t.Fatalf("Set 1 failed with %s %s %v", result.Key, result.Value, result.TTL) - } - - time.Sleep(time.Second) - - process.Kill() - - process, err = os.StartProcess("etcd", args, procAttr) - defer process.Kill() - if err != nil { - t.Fatal("start process failed:" + err.Error()) - return - } - - time.Sleep(time.Second) - - results, err := c.Get("foo") - if err != nil { - t.Fatal("get fail: " + err.Error()) - return - } - - result = results[0] - - if err != nil || result.Key != "/foo" || result.Value != "bar" || result.TTL > 99 { - if err != nil { - t.Fatal(err) - } - t.Fatalf("Recovery Get failed with %s %s %v", result.Key, result.Value, result.TTL) - } -} - -// Create a three nodes and try to set value -func templateTestSimpleMultiNode(t *testing.T, tls bool) { - procAttr := new(os.ProcAttr) - procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} - - clusterSize := 3 - - _, etcds, err := test.CreateCluster(clusterSize, procAttr, tls) - - if err != nil { - t.Fatal("cannot create cluster") - } - - defer test.DestroyCluster(etcds) - - time.Sleep(time.Second) - - c := etcd.NewClient(nil) - - c.SyncCluster() - - // Test Set - result, err := c.Set("foo", "bar", 100) - - if err != nil || result.Key != "/foo" || result.Value != "bar" || result.TTL < 95 { - if err != nil { - t.Fatal(err) - } - - t.Fatalf("Set 1 failed with %s %s %v", result.Key, result.Value, result.TTL) - } - - time.Sleep(time.Second) - - result, err = c.Set("foo", "bar", 100) - - if err != nil || result.Key != "/foo" || result.Value != "bar" || result.PrevValue != "bar" || result.TTL != 100 { - if err != nil { - t.Fatal(err) - } - t.Fatalf("Set 2 failed with %s %s %v", result.Key, result.Value, result.TTL) - } - -} - -func TestSimpleMultiNode(t *testing.T) { - templateTestSimpleMultiNode(t, false) -} - -func TestSimpleMultiNodeTls(t *testing.T) { - templateTestSimpleMultiNode(t, true) -} - -// Create a five nodes -// Kill all the nodes and restart -func TestMultiNodeKillAllAndRecovery(t *testing.T) { - procAttr := new(os.ProcAttr) - procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} - - clusterSize := 5 - argGroup, etcds, err := test.CreateCluster(clusterSize, procAttr, false) - defer test.DestroyCluster(etcds) - - if err != nil { - t.Fatal("cannot create cluster") - } - - c := etcd.NewClient(nil) - - c.SyncCluster() - - time.Sleep(time.Second) - - // send 10 commands - for i := 0; i < 10; i++ { - // Test Set - _, err := c.Set("foo", "bar", 0) - if err != nil { - panic(err) - } - } - - time.Sleep(time.Second) - - // kill all - test.DestroyCluster(etcds) - - time.Sleep(time.Second) - - stop := make(chan bool) - leaderChan := make(chan string, 1) - all := make(chan bool, 1) - - time.Sleep(time.Second) - - for i := 0; i < clusterSize; i++ { - etcds[i], err = os.StartProcess("etcd", argGroup[i], procAttr) - } - - go test.Monitor(clusterSize, 1, leaderChan, all, stop) - - <-all - <-leaderChan - - result, err := c.Set("foo", "bar", 0) - - if err != nil { - t.Fatalf("Recovery error: %s", err) - } - - if result.Index != 18 { - t.Fatalf("recovery failed! [%d/18]", result.Index) - } -} - -// Create a five nodes -// Randomly kill one of the node and keep on sending set command to the cluster -func TestMultiNodeKillOne(t *testing.T) { - procAttr := new(os.ProcAttr) - procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} - - clusterSize := 5 - argGroup, etcds, err := test.CreateCluster(clusterSize, procAttr, false) - - if err != nil { - t.Fatal("cannot create cluster") - } - - defer test.DestroyCluster(etcds) - - time.Sleep(2 * time.Second) - - c := etcd.NewClient(nil) - - c.SyncCluster() - - stop := make(chan bool) - // Test Set - go test.Set(stop) - - for i := 0; i < 10; i++ { - num := rand.Int() % clusterSize - fmt.Println("kill node", num+1) - - // kill - etcds[num].Kill() - etcds[num].Release() - time.Sleep(time.Second) - - // restart - etcds[num], err = os.StartProcess("etcd", argGroup[num], procAttr) - if err != nil { - panic(err) - } - time.Sleep(time.Second) - } - fmt.Println("stop") - stop <- true - <-stop -} - -// This test will kill the current leader and wait for the etcd cluster to elect a new leader for 200 times. -// It will print out the election time and the average election time. -func TestKillLeader(t *testing.T) { - procAttr := new(os.ProcAttr) - procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} - - clusterSize := 5 - argGroup, etcds, err := test.CreateCluster(clusterSize, procAttr, false) - - if err != nil { - t.Fatal("cannot create cluster") - } - - defer test.DestroyCluster(etcds) - - stop := make(chan bool) - leaderChan := make(chan string, 1) - all := make(chan bool, 1) - - time.Sleep(time.Second) - - go test.Monitor(clusterSize, 1, leaderChan, all, stop) - - var totalTime time.Duration - - leader := "http://127.0.0.1:7001" - - for i := 0; i < clusterSize; i++ { - fmt.Println("leader is ", leader) - port, _ := strconv.Atoi(strings.Split(leader, ":")[2]) - num := port - 7001 - fmt.Println("kill server ", num) - etcds[num].Kill() - etcds[num].Release() - - start := time.Now() - for { - newLeader := <-leaderChan - if newLeader != leader { - leader = newLeader - break - } - } - take := time.Now().Sub(start) - - totalTime += take - avgTime := totalTime / (time.Duration)(i+1) - - fmt.Println("Leader election time is ", take, "with election timeout", server.ElectionTimeout) - fmt.Println("Leader election time average is", avgTime, "with election timeout", server.ElectionTimeout) - etcds[num], err = os.StartProcess("etcd", argGroup[num], procAttr) - } - stop <- true -} - -// TestKillRandom kills random machines in the cluster and -// restart them after all other machines agree on the same leader -func TestKillRandom(t *testing.T) { - procAttr := new(os.ProcAttr) - procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} - - clusterSize := 9 - argGroup, etcds, err := test.CreateCluster(clusterSize, procAttr, false) - - if err != nil { - t.Fatal("cannot create cluster") - } - - defer test.DestroyCluster(etcds) - - stop := make(chan bool) - leaderChan := make(chan string, 1) - all := make(chan bool, 1) - - time.Sleep(3 * time.Second) - - go test.Monitor(clusterSize, 4, leaderChan, all, stop) - - toKill := make(map[int]bool) - - for i := 0; i < 20; i++ { - fmt.Printf("TestKillRandom Round[%d/20]\n", i) - - j := 0 - for { - - r := rand.Int31n(9) - if _, ok := toKill[int(r)]; !ok { - j++ - toKill[int(r)] = true - } - - if j > 3 { - break - } - - } - - for num, _ := range toKill { - err := etcds[num].Kill() - if err != nil { - panic(err) - } - etcds[num].Wait() - } - - time.Sleep(server.ElectionTimeout) - - <-leaderChan - - for num, _ := range toKill { - etcds[num], err = os.StartProcess("etcd", argGroup[num], procAttr) - } - - toKill = make(map[int]bool) - <-all - } - - stop <- true -} - -// remove the node and node rejoin with previous log -func TestRemoveNode(t *testing.T) { - procAttr := new(os.ProcAttr) - procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} - - clusterSize := 3 - argGroup, etcds, _ := test.CreateCluster(clusterSize, procAttr, false) - defer test.DestroyCluster(etcds) - - time.Sleep(time.Second) - - c := etcd.NewClient(nil) - - c.SyncCluster() - - rmReq, _ := http.NewRequest("DELETE", "http://127.0.0.1:7001/remove/node3", nil) - - client := &http.Client{} - for i := 0; i < 2; i++ { - for i := 0; i < 2; i++ { - client.Do(rmReq) - - etcds[2].Wait() - - resp, err := c.Get("_etcd/machines") - - if err != nil { - panic(err) - } - - if len(resp) != 2 { - t.Fatal("cannot remove machine") - } - - if i == 1 { - // rejoin with log - etcds[2], err = os.StartProcess("etcd", argGroup[2], procAttr) - } else { - // rejoin without log - etcds[2], err = os.StartProcess("etcd", append(argGroup[2], "-f"), procAttr) - } - - if err != nil { - panic(err) - } - - time.Sleep(time.Second) - - resp, err = c.Get("_etcd/machines") - - if err != nil { - panic(err) - } - - if len(resp) != 3 { - t.Fatalf("add machine fails #1 (%d != 3)", len(resp)) - } - } - - // first kill the node, then remove it, then add it back - for i := 0; i < 2; i++ { - etcds[2].Kill() - etcds[2].Wait() - - client.Do(rmReq) - - resp, err := c.Get("_etcd/machines") - - if err != nil { - panic(err) - } - - if len(resp) != 2 { - t.Fatal("cannot remove machine") - } - - if i == 1 { - // rejoin with log - etcds[2], err = os.StartProcess("etcd", append(argGroup[2]), procAttr) - } else { - // rejoin without log - etcds[2], err = os.StartProcess("etcd", append(argGroup[2], "-f"), procAttr) - } - - if err != nil { - panic(err) - } - - time.Sleep(time.Second) - - resp, err = c.Get("_etcd/machines") - - if err != nil { - panic(err) - } - - if len(resp) != 3 { - t.Fatalf("add machine fails #2 (%d != 3)", len(resp)) - } - } - } -} - -func templateBenchmarkEtcdDirectCall(b *testing.B, tls bool) { - procAttr := new(os.ProcAttr) - procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} - - clusterSize := 3 - _, etcds, _ := test.CreateCluster(clusterSize, procAttr, tls) - - defer test.DestroyCluster(etcds) - - time.Sleep(time.Second) - - b.ResetTimer() - for i := 0; i < b.N; i++ { - resp, _ := http.Get("http://127.0.0.1:4001/test/speed") - resp.Body.Close() - } - -} - -func BenchmarkEtcdDirectCall(b *testing.B) { - templateBenchmarkEtcdDirectCall(b, false) -} - -func BenchmarkEtcdDirectCallTls(b *testing.B) { - templateBenchmarkEtcdDirectCall(b, true) -} diff --git a/test.sh b/test.sh index 8022fead93b..c40f0c7e191 100755 --- a/test.sh +++ b/test.sh @@ -6,3 +6,7 @@ # Run the tests! go test -i go test -v + +# Run the functional tests! +go test -i github.com/coreos/etcd-test-runner +ETCD_BIN_PATH=$(pwd)/etcd go test -v github.com/coreos/etcd-test-runner diff --git a/test/test.go b/test/test.go deleted file mode 100644 index ea54584514c..00000000000 --- a/test/test.go +++ /dev/null @@ -1,205 +0,0 @@ -package test - -import ( - "fmt" - "github.com/coreos/go-etcd/etcd" - "io/ioutil" - "net" - "net/http" - "os" - "strconv" - "time" -) - -var client = http.Client{ - Transport: &http.Transport{ - Dial: dialTimeoutFast, - }, -} - -// Sending set commands -func Set(stop chan bool) { - - stopSet := false - i := 0 - c := etcd.NewClient(nil) - for { - key := fmt.Sprintf("%s_%v", "foo", i) - - result, err := c.Set(key, "bar", 0) - - if err != nil || result.Key != "/"+key || result.Value != "bar" { - select { - case <-stop: - stopSet = true - - default: - } - } - - select { - case <-stop: - stopSet = true - - default: - } - - if stopSet { - break - } - - i++ - } - stop <- true -} - -// Create a cluster of etcd nodes -func CreateCluster(size int, procAttr *os.ProcAttr, ssl bool) ([][]string, []*os.Process, error) { - argGroup := make([][]string, size) - - sslServer1 := []string{"-serverCAFile=./fixtures/ca/ca.crt", - "-serverCert=./fixtures/ca/server.crt", - "-serverKey=./fixtures/ca/server.key.insecure", - } - - sslServer2 := []string{"-serverCAFile=./fixtures/ca/ca.crt", - "-serverCert=./fixtures/ca/server2.crt", - "-serverKey=./fixtures/ca/server2.key.insecure", - } - - for i := 0; i < size; i++ { - if i == 0 { - argGroup[i] = []string{"etcd", "-d=/tmp/node1", "-n=node1"} - if ssl { - argGroup[i] = append(argGroup[i], sslServer1...) - } - } else { - strI := strconv.Itoa(i + 1) - argGroup[i] = []string{"etcd", "-n=node" + strI, "-c=127.0.0.1:400" + strI, "-s=127.0.0.1:700" + strI, "-d=/tmp/node" + strI, "-C=127.0.0.1:7001"} - if ssl { - argGroup[i] = append(argGroup[i], sslServer2...) - } - } - } - - etcds := make([]*os.Process, size) - - for i, _ := range etcds { - var err error - etcds[i], err = os.StartProcess("etcd", append(argGroup[i], "-f"), procAttr) - if err != nil { - return nil, nil, err - } - - // TODOBP: Change this sleep to wait until the master is up. - // The problem is that if the master isn't up then the children - // have to retry. This retry can take upwards of 15 seconds - // which slows tests way down and some of them fail. - if i == 0 { - time.Sleep(time.Second * 2) - } - } - - return argGroup, etcds, nil -} - -// Destroy all the nodes in the cluster -func DestroyCluster(etcds []*os.Process) error { - for _, etcd := range etcds { - err := etcd.Kill() - if err != nil { - panic(err.Error()) - } - etcd.Release() - } - return nil -} - -// -func Monitor(size int, allowDeadNum int, leaderChan chan string, all chan bool, stop chan bool) { - leaderMap := make(map[int]string) - baseAddrFormat := "http://0.0.0.0:400%d" - - for { - knownLeader := "unknown" - dead := 0 - var i int - - for i = 0; i < size; i++ { - leader, err := getLeader(fmt.Sprintf(baseAddrFormat, i+1)) - - if err == nil { - leaderMap[i] = leader - - if knownLeader == "unknown" { - knownLeader = leader - } else { - if leader != knownLeader { - break - } - - } - - } else { - dead++ - if dead > allowDeadNum { - break - } - } - - } - - if i == size { - select { - case <-stop: - return - case <-leaderChan: - leaderChan <- knownLeader - default: - leaderChan <- knownLeader - } - - } - if dead == 0 { - select { - case <-all: - all <- true - default: - all <- true - } - } - - time.Sleep(time.Millisecond * 10) - } - -} - -func getLeader(addr string) (string, error) { - - resp, err := client.Get(addr + "/v1/leader") - - if err != nil { - return "", err - } - - if resp.StatusCode != http.StatusOK { - resp.Body.Close() - return "", fmt.Errorf("no leader") - } - - b, err := ioutil.ReadAll(resp.Body) - - resp.Body.Close() - - if err != nil { - return "", err - } - - return string(b), nil - -} - -// Dial with timeout -func dialTimeoutFast(network, addr string) (net.Conn, error) { - return net.DialTimeout(network, addr, time.Millisecond*10) -} From 811d172a545c0d6d8f3b0e07068d2435523739f4 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 14 Oct 2013 21:22:20 -0700 Subject: [PATCH 107/247] fix change wait_index to waitIndex; we do not use post in 0.2 --- server/v2/get_key_handler.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/server/v2/get_key_handler.go b/server/v2/get_key_handler.go index e6cce6e8bd2..a7f8e56a1e6 100644 --- a/server/v2/get_key_handler.go +++ b/server/v2/get_key_handler.go @@ -25,7 +25,7 @@ func GetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { leader := s.Leader() hostname, _ := s.PeerURL(leader) url := hostname + req.URL.Path - log.Debugf("Redirect to %s", url) + log.Debugf("Redirect consistent get to %s", url) http.Redirect(w, req, url, http.StatusTemporaryRedirect) return nil } @@ -36,8 +36,10 @@ func GetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { if req.FormValue("wait") == "true" { // watch // Create a command to watch from a given index (default 0). var sinceIndex uint64 = 0 - if req.Method == "POST" { - sinceIndex, err = strconv.ParseUint(string(req.FormValue("wait_index")), 10, 64) + + waitIndex := req.FormValue("waitIndex") + if waitIndex != "" { + sinceIndex, err = strconv.ParseUint(string(req.FormValue("waitIndex")), 10, 64) if err != nil { return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index", store.UndefIndex, store.UndefTerm) } From 545f8ed6a183c48e156a0084ed2951a3c3ae34a1 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 14 Oct 2013 22:22:23 -0700 Subject: [PATCH 108/247] fix update PUT handler --- server/server.go | 3 +- server/v1/set_key_handler.go | 2 +- server/v2/update_key_handler.go | 57 ++++++++---- ...command.go => compare_and_swap_command.go} | 14 +-- store/stats_test.go | 4 +- store/store.go | 89 +++++++++---------- store/store_test.go | 24 ++--- store/update_command.go | 38 -------- 8 files changed, 108 insertions(+), 123 deletions(-) rename store/{test_and_set_command.go => compare_and_swap_command.go} (61%) delete mode 100644 store/update_command.go diff --git a/server/server.go b/server/server.go index d1b1abf0f59..0aeb5dbc41e 100644 --- a/server/server.go +++ b/server/server.go @@ -254,10 +254,11 @@ func (s *Server) SpeedTestHandler(w http.ResponseWriter, req *http.Request) erro for i := 0; i < count; i++ { go func() { for j := 0; j < 10; j++ { - c := &store.UpdateCommand{ + c := &store.CreateCommand{ Key: "foo", Value: "bar", ExpireTime: time.Unix(0, 0), + Force: true, } s.peerServer.RaftServer().Do(c) } diff --git a/server/v1/set_key_handler.go b/server/v1/set_key_handler.go index 03b6d7f9bef..acd4037c1e4 100644 --- a/server/v1/set_key_handler.go +++ b/server/v1/set_key_handler.go @@ -31,7 +31,7 @@ func SetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { // If the "prevValue" is specified then test-and-set. Otherwise create a new key. var c raft.Command if prevValueArr, ok := req.Form["prevValue"]; ok && len(prevValueArr) > 0 { - c = &store.TestAndSetCommand{ + c = &store.CompareAndSwapCommand{ Key: key, Value: value, PrevValue: prevValueArr[0], diff --git a/server/v2/update_key_handler.go b/server/v2/update_key_handler.go index 841c8828de4..60c260afca8 100644 --- a/server/v2/update_key_handler.go +++ b/server/v2/update_key_handler.go @@ -29,36 +29,59 @@ func UpdateKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error prevValue, valueOk := req.Form["prevValue"] prevIndexStr, indexOk := req.Form["prevIndex"] + prevExist, existOk := req.Form["prevExist"] var c raft.Command - if !valueOk && !indexOk { // update without test - c = &store.UpdateCommand{ + + // Set command: create a new node or replace the old one. + if !valueOk && !indexOk && !existOk { + c = &store.CreateCommand{ Key: key, Value: value, ExpireTime: expireTime, + Force: true, + } + return s.Dispatch(c, w, req) + } + + // update with test + if existOk { + if prevExist[0] == "false" { + // Create command: create a new node. Fail, if a node already exists + // Ignore prevIndex and prevValue + c = &store.CreateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + } } + } - } else { // update with test - var prevIndex uint64 + var prevIndex uint64 - if indexOk { - prevIndex, err = strconv.ParseUint(prevIndexStr[0], 10, 64) + if indexOk { + prevIndex, err = strconv.ParseUint(prevIndexStr[0], 10, 64) - // bad previous index - if err != nil { - return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Update", store.UndefIndex, store.UndefTerm) - } - } else { - prevIndex = 0 + // bad previous index + if err != nil { + return etcdErr.NewError(etcdErr.EcodeIndexNaN, "CompareAndSwap", store.UndefIndex, store.UndefTerm) } + } else { + prevIndex = 0 + } - c = &store.TestAndSetCommand{ - Key: key, - Value: value, - PrevValue: prevValue[0], - PrevIndex: prevIndex, + if valueOk { + if prevValue[0] == "" { + return etcdErr.NewError(etcdErr.EcodePrevValueRequired, "CompareAndSwap", store.UndefIndex, store.UndefTerm) } } + c = &store.CompareAndSwapCommand{ + Key: key, + Value: value, + PrevValue: prevValue[0], + PrevIndex: prevIndex, + } + return s.Dispatch(c, w, req) } diff --git a/store/test_and_set_command.go b/store/compare_and_swap_command.go similarity index 61% rename from store/test_and_set_command.go rename to store/compare_and_swap_command.go index 03cb1879a39..c4bfee569bf 100644 --- a/store/test_and_set_command.go +++ b/store/compare_and_swap_command.go @@ -8,11 +8,11 @@ import ( ) func init() { - raft.RegisterCommand(&TestAndSetCommand{}) + raft.RegisterCommand(&CompareAndSwapCommand{}) } -// The TestAndSetCommand performs a conditional update on a key in the store. -type TestAndSetCommand struct { +// The CompareAndSwap performs a conditional update on a key in the store. +type CompareAndSwapCommand struct { Key string `json:"key"` Value string `json:"value"` ExpireTime time.Time `json:"expireTime"` @@ -21,15 +21,15 @@ type TestAndSetCommand struct { } // The name of the testAndSet command in the log -func (c *TestAndSetCommand) CommandName() string { - return "etcd:testAndSet" +func (c *CompareAndSwapCommand) CommandName() string { + return "etcd:compareAndSwap" } // Set the key-value pair if the current value of the key equals to the given prevValue -func (c *TestAndSetCommand) Apply(server raft.Server) (interface{}, error) { +func (c *CompareAndSwapCommand) Apply(server raft.Server) (interface{}, error) { s, _ := server.StateMachine().(Store) - e, err := s.TestAndSet(c.Key, c.PrevValue, c.PrevIndex, + e, err := s.CompareAndSwap(c.Key, c.PrevValue, c.PrevIndex, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) if err != nil { diff --git a/store/stats_test.go b/store/stats_test.go index 44d1a899925..fa6483ac2ca 100644 --- a/store/stats_test.go +++ b/store/stats_test.go @@ -37,7 +37,7 @@ func TestBasicStats(t *testing.T) { for _, k := range keys { i++ - _, err := s.Update(k, "foo", time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1) + _, err := s.update(k, "foo", time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1) if err != nil { UpdateFail++ } else { @@ -58,7 +58,7 @@ func TestBasicStats(t *testing.T) { for _, k := range keys { i++ - _, err := s.TestAndSet(k, "foo", 0, "bar", Permanent, i, 1) + _, err := s.CompareAndSwap(k, "foo", 0, "bar", Permanent, i, 1) if err != nil { TestAndSetFail++ } else { diff --git a/store/store.go b/store/store.go index d063aa6bacf..f25b4345e6f 100644 --- a/store/store.go +++ b/store/store.go @@ -17,8 +17,7 @@ type Store interface { Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) Create(nodePath string, value string, incrementalSuffix bool, force bool, expireTime time.Time, index uint64, term uint64) (*Event, error) - Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) - TestAndSet(nodePath string, prevValue string, prevIndex uint64, + CompareAndSwap(nodePath string, prevValue string, prevIndex uint64, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error) Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error) @@ -116,47 +115,7 @@ func (s *store) Create(nodePath string, value string, incrementalSuffix bool, fo return s.internalCreate(nodePath, value, incrementalSuffix, force, expireTime, index, term, Create) } -// Update function updates the value/ttl of the node. -// If the node is a file, the value and the ttl can be updated. -// If the node is a directory, only the ttl can be updated. -func (s *store) Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) { - s.worldLock.Lock() - defer s.worldLock.Unlock() - nodePath = path.Clean(path.Join("/", nodePath)) - - n, err := s.internalGet(nodePath, index, term) - - if err != nil { // if the node does not exist, return error - s.Stats.Inc(UpdateFail) - return nil, err - } - - e := newEvent(Update, nodePath, s.Index, s.Term) - - if len(newValue) != 0 { - if n.IsDir() { - // if the node is a directory, we cannot update value - s.Stats.Inc(UpdateFail) - return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term) - } - - e.PrevValue = n.Value - n.Write(newValue, index, term) - } - - // update ttl - n.UpdateTTL(expireTime) - - e.Expiration, e.TTL = n.ExpirationAndTTL() - - s.WatcherHub.notify(e) - - s.Stats.Inc(UpdateSuccess) - - return e, nil -} - -func (s *store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, +func (s *store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint64, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { nodePath = path.Clean(path.Join("/", nodePath)) @@ -164,8 +123,8 @@ func (s *store) TestAndSet(nodePath string, prevValue string, prevIndex uint64, s.worldLock.Lock() defer s.worldLock.Unlock() - if prevValue == "" && prevIndex == 0 { // try create - return s.internalCreate(nodePath, value, false, false, expireTime, index, term, TestAndSet) + if prevValue == "" && prevIndex == 0 { // try just update + return s.update(nodePath, value, expireTime, index, term) } n, err := s.internalGet(nodePath, index, term) @@ -293,6 +252,46 @@ func (s *store) walk(nodePath string, walkFunc func(prev *Node, component string return curr, nil } +// Update function updates the value/ttl of the node. +// If the node is a file, the value and the ttl can be updated. +// If the node is a directory, only the ttl can be updated. +func (s *store) update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) { + s.worldLock.Lock() + defer s.worldLock.Unlock() + nodePath = path.Clean(path.Join("/", nodePath)) + + n, err := s.internalGet(nodePath, index, term) + + if err != nil { // if the node does not exist, return error + s.Stats.Inc(UpdateFail) + return nil, err + } + + e := newEvent(Update, nodePath, s.Index, s.Term) + + if len(newValue) != 0 { + if n.IsDir() { + // if the node is a directory, we cannot update value + s.Stats.Inc(UpdateFail) + return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term) + } + + e.PrevValue = n.Value + n.Write(newValue, index, term) + } + + // update ttl + n.UpdateTTL(expireTime) + + e.Expiration, e.TTL = n.ExpirationAndTTL() + + s.WatcherHub.notify(e) + + s.Stats.Inc(UpdateSuccess) + + return e, nil +} + func (s *store) internalCreate(nodePath string, value string, incrementalSuffix bool, force bool, expireTime time.Time, index uint64, term uint64, action string) (*Event, error) { diff --git a/store/store_test.go b/store/store_test.go index 958d99007b1..457a9b5f584 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -74,7 +74,7 @@ func TestUpdateFile(t *testing.T) { t.Fatalf("cannot create %s=bar [%s]", "/foo/bar", err.Error()) } - _, err = s.Update("/foo/bar", "barbar", Permanent, 2, 1) + _, err = s.update("/foo/bar", "barbar", Permanent, 2, 1) if err != nil { t.Fatalf("cannot update %s=barbar [%s]", "/foo/bar", err.Error()) @@ -114,7 +114,7 @@ func TestUpdateFile(t *testing.T) { } expire := time.Now().Add(time.Second * 2) - _, err = s.Update("/foo/foo", "", expire, 7, 1) + _, err = s.update("/foo/foo", "", expire, 7, 1) if err != nil { t.Fatalf("cannot update dir [%s] [%s]", "/foo/foo", err.Error()) } @@ -286,18 +286,18 @@ func TestExpire(t *testing.T) { } } -func TestTestAndSet(t *testing.T) { // TODO prevValue == nil ? +func TestCompareAndSwap(t *testing.T) { // TODO prevValue == nil ? s := newStore() s.Create("/foo", "bar", false, false, Permanent, 1, 1) // test on wrong previous value - _, err := s.TestAndSet("/foo", "barbar", 0, "car", Permanent, 2, 1) + _, err := s.CompareAndSwap("/foo", "barbar", 0, "car", Permanent, 2, 1) if err == nil { t.Fatal("test and set should fail barbar != bar") } // test on value - e, err := s.TestAndSet("/foo", "bar", 0, "car", Permanent, 3, 1) + e, err := s.CompareAndSwap("/foo", "bar", 0, "car", Permanent, 3, 1) if err != nil { t.Fatal("test and set should succeed bar == bar") @@ -308,7 +308,7 @@ func TestTestAndSet(t *testing.T) { // TODO prevValue == nil ? } // test on index - e, err = s.TestAndSet("/foo", "", 3, "bar", Permanent, 4, 1) + e, err = s.CompareAndSwap("/foo", "", 3, "bar", Permanent, 4, 1) if err != nil { t.Fatal("test and set should succeed index 3 == 3") @@ -331,14 +331,14 @@ func TestWatch(t *testing.T) { } c, _ = s.Watch("/foo/foo/foo", false, 0, 1, 1) - s.Update("/foo/foo/foo", "car", Permanent, 2, 1) + s.update("/foo/foo/foo", "car", Permanent, 2, 1) e = nonblockingRetrive(c) if e.Key != "/foo/foo/foo" || e.Action != Update { t.Fatal("watch for Update node fails ", e) } c, _ = s.Watch("/foo/foo/foo", false, 0, 2, 1) - s.TestAndSet("/foo/foo/foo", "car", 0, "bar", Permanent, 3, 1) + s.CompareAndSwap("/foo/foo/foo", "car", 0, "bar", Permanent, 3, 1) e = nonblockingRetrive(c) if e.Key != "/foo/foo/foo" || e.Action != TestAndSet { t.Fatal("watch for TestAndSet node fails") @@ -360,14 +360,14 @@ func TestWatch(t *testing.T) { } c, _ = s.Watch("/foo", true, 0, 5, 1) - s.Update("/foo/foo/boo", "foo", Permanent, 6, 1) + s.update("/foo/foo/boo", "foo", Permanent, 6, 1) e = nonblockingRetrive(c) if e.Key != "/foo/foo/boo" || e.Action != Update { t.Fatal("watch for Update subdirectory fails") } c, _ = s.Watch("/foo", true, 0, 6, 1) - s.TestAndSet("/foo/foo/boo", "foo", 0, "bar", Permanent, 7, 1) + s.CompareAndSwap("/foo/foo/boo", "foo", 0, "bar", Permanent, 7, 1) e = nonblockingRetrive(c) if e.Key != "/foo/foo/boo" || e.Action != TestAndSet { t.Fatal("watch for TestAndSet subdirectory fails") @@ -390,7 +390,7 @@ func TestWatch(t *testing.T) { } s.Create("/foo/foo/boo", "foo", false, false, Permanent, 10, 1) - s.Update("/foo/foo/boo", "bar", time.Now().Add(time.Second*1), 11, 1) + s.update("/foo/foo/boo", "bar", time.Now().Add(time.Second*1), 11, 1) c, _ = s.Watch("/foo", true, 0, 11, 1) time.Sleep(time.Second * 2) e = nonblockingRetrive(c) @@ -399,7 +399,7 @@ func TestWatch(t *testing.T) { } s.Create("/foo/foo/boo", "foo", false, false, Permanent, 12, 1) - s.TestAndSet("/foo/foo/boo", "foo", 0, "bar", time.Now().Add(time.Second*1), 13, 1) + s.CompareAndSwap("/foo/foo/boo", "foo", 0, "bar", time.Now().Add(time.Second*1), 13, 1) c, _ = s.Watch("/foo", true, 0, 13, 1) time.Sleep(time.Second * 2) e = nonblockingRetrive(c) diff --git a/store/update_command.go b/store/update_command.go deleted file mode 100644 index 3152006bc65..00000000000 --- a/store/update_command.go +++ /dev/null @@ -1,38 +0,0 @@ -package store - -import ( - "time" - - "github.com/coreos/etcd/log" - "github.com/coreos/go-raft" -) - -func init() { - raft.RegisterCommand(&UpdateCommand{}) -} - -// The UpdateCommand updates the value of a key in the Store. -type UpdateCommand struct { - Key string `json:"key"` - Value string `json:"value"` - ExpireTime time.Time `json:"expireTime"` -} - -// The name of the update command in the log -func (c *UpdateCommand) CommandName() string { - return "etcd:update" -} - -// Update node -func (c *UpdateCommand) Apply(server raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(Store) - - e, err := s.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) - - if err != nil { - log.Debug(err) - return nil, err - } - - return e, nil -} From 9ebdcb8ae33a64816a82c71fa6a3d9541bfad6f5 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 14 Oct 2013 22:32:22 -0700 Subject: [PATCH 109/247] refactor change testAndSet to CompareAndSwap --- store/event.go | 12 +++--- store/stats.go | 41 ++++++++++--------- store/stats_test.go | 14 +++---- store/store.go | 10 ++--- store/store_test.go | 10 ++--- .../coreos/go-etcd/etcd/client_test.go | 8 ++-- .../examples/sync-cluster/sync-cluster.go | 1 - .../github.com/coreos/go-log/log/commands.go | 28 ++++++------- .../github.com/coreos/go-log/log/fields.go | 1 + .../github.com/coreos/go-log/log/logger.go | 1 + .../github.com/coreos/go-log/log/priority.go | 1 + .../github.com/coreos/go-log/log/sinks.go | 1 + .../coreos/go-systemd/activation/files.go | 2 +- 13 files changed, 66 insertions(+), 64 deletions(-) diff --git a/store/event.go b/store/event.go index 0d9ec0a37de..f9ae0938baf 100644 --- a/store/event.go +++ b/store/event.go @@ -5,12 +5,12 @@ import ( ) const ( - Get = "get" - Create = "create" - Update = "update" - Delete = "delete" - TestAndSet = "testAndSet" - Expire = "expire" + Get = "get" + Create = "create" + Update = "update" + Delete = "delete" + CompareAndSwap = "compareAndSwap" + Expire = "expire" ) const ( diff --git a/store/stats.go b/store/stats.go index e2053ed4235..4c89b93e38c 100644 --- a/store/stats.go +++ b/store/stats.go @@ -6,17 +6,17 @@ import ( ) const ( - SetSuccess = 100 - SetFail = 101 - DeleteSuccess = 102 - DeleteFail = 103 - UpdateSuccess = 104 - UpdateFail = 105 - TestAndSetSuccess = 106 - TestAndSetFail = 107 - GetSuccess = 110 - GetFail = 111 - ExpireCount = 112 + SetSuccess = 100 + SetFail = 101 + DeleteSuccess = 102 + DeleteFail = 103 + UpdateSuccess = 104 + UpdateFail = 105 + CompareAndSwapSuccess = 106 + CompareAndSwapFail = 107 + GetSuccess = 110 + GetFail = 111 + ExpireCount = 112 ) type Stats struct { @@ -38,9 +38,10 @@ type Stats struct { UpdateFail uint64 `json:"updateFail"` // Number of testAndSet requests - TestAndSetSuccess uint64 `json:"testAndSetSuccess"` - TestAndSetFail uint64 `json:"testAndSetFail"` - ExpireCount uint64 `json:"expireCount"` + CompareAndSwapSuccess uint64 `json:"compareAndSwapSuccess"` + CompareAndSwapFail uint64 `json:"compareAndSwapFail"` + + ExpireCount uint64 `json:"expireCount"` Watchers uint64 `json:"watchers"` } @@ -53,7 +54,7 @@ func newStats() *Stats { func (s *Stats) clone() *Stats { return &Stats{s.GetSuccess, s.GetFail, s.SetSuccess, s.SetFail, s.DeleteSuccess, s.DeleteFail, s.UpdateSuccess, s.UpdateFail, - s.TestAndSetSuccess, s.TestAndSetFail, s.Watchers, s.ExpireCount} + s.CompareAndSwapSuccess, s.CompareAndSwapFail, s.Watchers, s.ExpireCount} } // Status() return the statistics info of etcd storage its recent start @@ -69,7 +70,7 @@ func (s *Stats) TotalReads() uint64 { func (s *Stats) TotalWrites() uint64 { return s.SetSuccess + s.SetFail + s.DeleteSuccess + s.DeleteFail + - s.TestAndSetSuccess + s.TestAndSetFail + + s.CompareAndSwapSuccess + s.CompareAndSwapFail + s.UpdateSuccess + s.UpdateFail } @@ -91,10 +92,10 @@ func (s *Stats) Inc(field int) { atomic.AddUint64(&s.UpdateSuccess, 1) case UpdateFail: atomic.AddUint64(&s.UpdateFail, 1) - case TestAndSetSuccess: - atomic.AddUint64(&s.TestAndSetSuccess, 1) - case TestAndSetFail: - atomic.AddUint64(&s.TestAndSetFail, 1) + case CompareAndSwapSuccess: + atomic.AddUint64(&s.CompareAndSwapSuccess, 1) + case CompareAndSwapFail: + atomic.AddUint64(&s.CompareAndSwapFail, 1) case ExpireCount: atomic.AddUint64(&s.ExpireCount, 1) } diff --git a/store/stats_test.go b/store/stats_test.go index fa6483ac2ca..7c2296fb085 100644 --- a/store/stats_test.go +++ b/store/stats_test.go @@ -12,7 +12,7 @@ func TestBasicStats(t *testing.T) { var i uint64 var GetSuccess, GetFail, SetSuccess, SetFail, DeleteSuccess, DeleteFail uint64 - var UpdateSuccess, UpdateFail, TestAndSetSuccess, TestAndSetFail, watcher_number uint64 + var UpdateSuccess, UpdateFail, CompareAndSwapSuccess, CompareAndSwapFail, watcher_number uint64 for _, k := range keys { i++ @@ -60,9 +60,9 @@ func TestBasicStats(t *testing.T) { i++ _, err := s.CompareAndSwap(k, "foo", 0, "bar", Permanent, i, 1) if err != nil { - TestAndSetFail++ + CompareAndSwapFail++ } else { - TestAndSetSuccess++ + CompareAndSwapSuccess++ } } @@ -132,12 +132,12 @@ func TestBasicStats(t *testing.T) { t.Fatalf("UpdateFail [%d] != Stats.UpdateFail [%d]", UpdateFail, s.Stats.UpdateFail) } - if TestAndSetSuccess != s.Stats.TestAndSetSuccess { - t.Fatalf("TestAndSetSuccess [%d] != Stats.TestAndSetSuccess [%d]", TestAndSetSuccess, s.Stats.TestAndSetSuccess) + if CompareAndSwapSuccess != s.Stats.CompareAndSwapSuccess { + t.Fatalf("TestAndSetSuccess [%d] != Stats.CompareAndSwapSuccess [%d]", CompareAndSwapSuccess, s.Stats.CompareAndSwapSuccess) } - if TestAndSetFail != s.Stats.TestAndSetFail { - t.Fatalf("TestAndSetFail [%d] != Stats.TestAndSetFail [%d]", TestAndSetFail, s.Stats.TestAndSetFail) + if CompareAndSwapFail != s.Stats.CompareAndSwapFail { + t.Fatalf("TestAndSetFail [%d] != Stats.TestAndSetFail [%d]", CompareAndSwapFail, s.Stats.CompareAndSwapFail) } s = newStore() diff --git a/store/store.go b/store/store.go index f25b4345e6f..6e0fe4d68de 100644 --- a/store/store.go +++ b/store/store.go @@ -130,19 +130,19 @@ func (s *store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint n, err := s.internalGet(nodePath, index, term) if err != nil { - s.Stats.Inc(TestAndSetFail) + s.Stats.Inc(CompareAndSwapFail) return nil, err } if n.IsDir() { // can only test and set file - s.Stats.Inc(TestAndSetFail) + s.Stats.Inc(CompareAndSwapFail) return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term) } // If both of the prevValue and prevIndex are given, we will test both of them. // Command will be executed, only if both of the tests are successful. if (prevValue == "" || n.Value == prevValue) && (prevIndex == 0 || n.ModifiedIndex == prevIndex) { - e := newEvent(TestAndSet, nodePath, index, term) + e := newEvent(CompareAndSwap, nodePath, index, term) e.PrevValue = n.Value // if test succeed, write the value @@ -153,12 +153,12 @@ func (s *store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint e.Expiration, e.TTL = n.ExpirationAndTTL() s.WatcherHub.notify(e) - s.Stats.Inc(TestAndSetSuccess) + s.Stats.Inc(CompareAndSwapSuccess) return e, nil } cause := fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, n.Value, prevIndex, n.ModifiedIndex) - s.Stats.Inc(TestAndSetFail) + s.Stats.Inc(CompareAndSwapFail) return nil, etcdErr.NewError(etcdErr.EcodeTestFailed, cause, index, term) } diff --git a/store/store_test.go b/store/store_test.go index 457a9b5f584..f26d3d6164f 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -340,8 +340,8 @@ func TestWatch(t *testing.T) { c, _ = s.Watch("/foo/foo/foo", false, 0, 2, 1) s.CompareAndSwap("/foo/foo/foo", "car", 0, "bar", Permanent, 3, 1) e = nonblockingRetrive(c) - if e.Key != "/foo/foo/foo" || e.Action != TestAndSet { - t.Fatal("watch for TestAndSet node fails") + if e.Key != "/foo/foo/foo" || e.Action != CompareAndSwap { + t.Fatal("watch for CompareAndSwap node fails") } c, _ = s.Watch("/foo/foo/foo", false, 0, 3, 1) @@ -369,8 +369,8 @@ func TestWatch(t *testing.T) { c, _ = s.Watch("/foo", true, 0, 6, 1) s.CompareAndSwap("/foo/foo/boo", "foo", 0, "bar", Permanent, 7, 1) e = nonblockingRetrive(c) - if e.Key != "/foo/foo/boo" || e.Action != TestAndSet { - t.Fatal("watch for TestAndSet subdirectory fails") + if e.Key != "/foo/foo/boo" || e.Action != CompareAndSwap { + t.Fatal("watch for CompareAndSwap subdirectory fails") } c, _ = s.Watch("/foo", true, 0, 7, 1) @@ -404,7 +404,7 @@ func TestWatch(t *testing.T) { time.Sleep(time.Second * 2) e = nonblockingRetrive(c) if e.Key != "/foo/foo/boo" || e.Action != Expire || e.Index != 13 { - t.Fatal("watch for Expiration of TestAndSet() subdirectory fails ", e) + t.Fatal("watch for Expiration of CompareAndSwap() subdirectory fails ", e) } } diff --git a/third_party/github.com/coreos/go-etcd/etcd/client_test.go b/third_party/github.com/coreos/go-etcd/etcd/client_test.go index 29f138113ce..bf75d894796 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/client_test.go +++ b/third_party/github.com/coreos/go-etcd/etcd/client_test.go @@ -2,9 +2,9 @@ package etcd import ( "fmt" - "testing" - "net/url" "net" + "net/url" + "testing" ) // To pass this test, we need to create a cluster of 3 machines @@ -19,7 +19,7 @@ func TestSync(t *testing.T) { t.Fatal("cannot sync machines") } - for _, m := range(c.GetCluster()) { + for _, m := range c.GetCluster() { u, err := url.Parse(m) if err != nil { t.Fatal(err) @@ -27,7 +27,7 @@ func TestSync(t *testing.T) { if u.Scheme != "http" { t.Fatal("scheme must be http") } - + host, _, err := net.SplitHostPort(u.Host) if err != nil { t.Fatal(err) diff --git a/third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go b/third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go index 8249b4bdcb2..8c7e375c5af 100644 --- a/third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go +++ b/third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go @@ -1,4 +1,3 @@ - package main import ( diff --git a/third_party/github.com/coreos/go-log/log/commands.go b/third_party/github.com/coreos/go-log/log/commands.go index 94dc9e152d7..f39fdef9717 100644 --- a/third_party/github.com/coreos/go-log/log/commands.go +++ b/third_party/github.com/coreos/go-log/log/commands.go @@ -1,4 +1,5 @@ package log + // Copyright 2013, CoreOS, Inc. All rights reserved. // // Licensed under the Apache License, Version 2.0 (the "License"); @@ -42,7 +43,6 @@ func (logger *Logger) Logf(priority Priority, format string, v ...interface{}) { logger.Log(priority, fmt.Sprintf(format, v...)) } - func (logger *Logger) Emergency(v ...interface{}) { logger.Log(PriEmerg, v...) } @@ -99,7 +99,6 @@ func (logger *Logger) Debugf(format string, v ...interface{}) { logger.Log(PriDebug, fmt.Sprintf(format, v...)) } - func Emergency(v ...interface{}) { defaultLogger.Log(PriEmerg, v...) } @@ -158,57 +157,56 @@ func Debugf(format string, v ...interface{}) { // Standard library log functions -func (logger *Logger)Fatalln (v ...interface{}) { +func (logger *Logger) Fatalln(v ...interface{}) { logger.Log(PriCrit, v...) os.Exit(1) } -func (logger *Logger)Fatalf (format string, v ...interface{}) { +func (logger *Logger) Fatalf(format string, v ...interface{}) { logger.Logf(PriCrit, format, v...) os.Exit(1) } -func (logger *Logger)Panicln (v ...interface{}) { +func (logger *Logger) Panicln(v ...interface{}) { s := fmt.Sprint(v...) logger.Log(PriErr, s) panic(s) } -func (logger *Logger)Panicf (format string, v ...interface{}) { +func (logger *Logger) Panicf(format string, v ...interface{}) { s := fmt.Sprintf(format, v...) logger.Log(PriErr, s) panic(s) } -func (logger *Logger)Println (v ...interface{}) { +func (logger *Logger) Println(v ...interface{}) { logger.Log(PriInfo, v...) } -func (logger *Logger)Printf (format string, v ...interface{}) { +func (logger *Logger) Printf(format string, v ...interface{}) { logger.Logf(PriInfo, format, v...) } - -func Fatalln (v ...interface{}) { +func Fatalln(v ...interface{}) { defaultLogger.Log(PriCrit, v...) os.Exit(1) } -func Fatalf (format string, v ...interface{}) { +func Fatalf(format string, v ...interface{}) { defaultLogger.Logf(PriCrit, format, v...) os.Exit(1) } -func Panicln (v ...interface{}) { +func Panicln(v ...interface{}) { s := fmt.Sprint(v...) defaultLogger.Log(PriErr, s) panic(s) } -func Panicf (format string, v ...interface{}) { +func Panicf(format string, v ...interface{}) { s := fmt.Sprintf(format, v...) defaultLogger.Log(PriErr, s) panic(s) } -func Println (v ...interface{}) { +func Println(v ...interface{}) { defaultLogger.Log(PriInfo, v...) } -func Printf (format string, v ...interface{}) { +func Printf(format string, v ...interface{}) { defaultLogger.Logf(PriInfo, format, v...) } diff --git a/third_party/github.com/coreos/go-log/log/fields.go b/third_party/github.com/coreos/go-log/log/fields.go index e8d9698a08a..b04edc8eb77 100644 --- a/third_party/github.com/coreos/go-log/log/fields.go +++ b/third_party/github.com/coreos/go-log/log/fields.go @@ -1,4 +1,5 @@ package log + // Copyright 2013, CoreOS, Inc. All rights reserved. // // Licensed under the Apache License, Version 2.0 (the "License"); diff --git a/third_party/github.com/coreos/go-log/log/logger.go b/third_party/github.com/coreos/go-log/log/logger.go index 2089a11f898..8c3b86c1dff 100644 --- a/third_party/github.com/coreos/go-log/log/logger.go +++ b/third_party/github.com/coreos/go-log/log/logger.go @@ -1,4 +1,5 @@ package log + // Copyright 2013, CoreOS, Inc. All rights reserved. // // Licensed under the Apache License, Version 2.0 (the "License"); diff --git a/third_party/github.com/coreos/go-log/log/priority.go b/third_party/github.com/coreos/go-log/log/priority.go index ac73fc8a42c..c169d686918 100644 --- a/third_party/github.com/coreos/go-log/log/priority.go +++ b/third_party/github.com/coreos/go-log/log/priority.go @@ -1,4 +1,5 @@ package log + // Copyright 2013, CoreOS, Inc. All rights reserved. // // Licensed under the Apache License, Version 2.0 (the "License"); diff --git a/third_party/github.com/coreos/go-log/log/sinks.go b/third_party/github.com/coreos/go-log/log/sinks.go index a41f3365dfd..bdf1e41f111 100644 --- a/third_party/github.com/coreos/go-log/log/sinks.go +++ b/third_party/github.com/coreos/go-log/log/sinks.go @@ -1,4 +1,5 @@ package log + // Copyright 2013, CoreOS, Inc. All rights reserved. // // Licensed under the Apache License, Version 2.0 (the "License"); diff --git a/third_party/github.com/coreos/go-systemd/activation/files.go b/third_party/github.com/coreos/go-systemd/activation/files.go index 4b854237077..a0a56f9e6b1 100644 --- a/third_party/github.com/coreos/go-systemd/activation/files.go +++ b/third_party/github.com/coreos/go-systemd/activation/files.go @@ -24,7 +24,7 @@ func Files() []*os.File { files := []*os.File(nil) for fd := listenFdsStart; fd < listenFdsStart+nfds; fd++ { syscall.CloseOnExec(fd) - files = append(files, os.NewFile(uintptr(fd), "LISTEN_FD_" + strconv.Itoa(fd))) + files = append(files, os.NewFile(uintptr(fd), "LISTEN_FD_"+strconv.Itoa(fd))) } return files } From 2aeb25e80ca73fbb6307da6c447933ff63603c16 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 14 Oct 2013 22:38:07 -0700 Subject: [PATCH 110/247] refactor change handler name to its method --- server/server.go | 8 ++++---- server/v2/{delete_key_handler.go => delete_handler.go} | 2 +- server/v2/{get_key_handler.go => get_handler.go} | 2 +- server/v2/{create_key_handler.go => post_handler.go} | 2 +- server/v2/{update_key_handler.go => put_handler.go} | 2 +- 5 files changed, 8 insertions(+), 8 deletions(-) rename server/v2/{delete_key_handler.go => delete_handler.go} (77%) rename server/v2/{get_key_handler.go => get_handler.go} (95%) rename server/v2/{create_key_handler.go => post_handler.go} (87%) rename server/v2/{update_key_handler.go => put_handler.go} (95%) diff --git a/server/server.go b/server/server.go index 0aeb5dbc41e..786947c81be 100644 --- a/server/server.go +++ b/server/server.go @@ -102,10 +102,10 @@ func (s *Server) installV1() { } func (s *Server) installV2() { - s.handleFuncV2("/v2/keys/{key:.*}", v2.GetKeyHandler).Methods("GET") - s.handleFuncV2("/v2/keys/{key:.*}", v2.CreateKeyHandler).Methods("POST") - s.handleFuncV2("/v2/keys/{key:.*}", v2.UpdateKeyHandler).Methods("PUT") - s.handleFuncV2("/v2/keys/{key:.*}", v2.DeleteKeyHandler).Methods("DELETE") + s.handleFuncV2("/v2/keys/{key:.*}", v2.GetHandler).Methods("GET") + s.handleFuncV2("/v2/keys/{key:.*}", v2.PostHandler).Methods("POST") + s.handleFuncV2("/v2/keys/{key:.*}", v2.PutHandler).Methods("PUT") + s.handleFuncV2("/v2/keys/{key:.*}", v2.DeleteHandler).Methods("DELETE") s.handleFunc("/v2/leader", s.GetLeaderHandler).Methods("GET") s.handleFunc("/v2/machines", s.GetMachinesHandler).Methods("GET") s.handleFunc("/v2/stats/self", s.GetStatsHandler).Methods("GET") diff --git a/server/v2/delete_key_handler.go b/server/v2/delete_handler.go similarity index 77% rename from server/v2/delete_key_handler.go rename to server/v2/delete_handler.go index c53e7245983..9012498cf2c 100644 --- a/server/v2/delete_key_handler.go +++ b/server/v2/delete_handler.go @@ -7,7 +7,7 @@ import ( "github.com/gorilla/mux" ) -func DeleteKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { +func DeleteHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) key := "/" + vars["key"] diff --git a/server/v2/get_key_handler.go b/server/v2/get_handler.go similarity index 95% rename from server/v2/get_key_handler.go rename to server/v2/get_handler.go index a7f8e56a1e6..d0e8042737d 100644 --- a/server/v2/get_key_handler.go +++ b/server/v2/get_handler.go @@ -13,7 +13,7 @@ import ( "github.com/gorilla/mux" ) -func GetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { +func GetHandler(w http.ResponseWriter, req *http.Request, s Server) error { var err error var event *store.Event diff --git a/server/v2/create_key_handler.go b/server/v2/post_handler.go similarity index 87% rename from server/v2/create_key_handler.go rename to server/v2/post_handler.go index fab9bde1f9c..dda146e5e14 100644 --- a/server/v2/create_key_handler.go +++ b/server/v2/post_handler.go @@ -8,7 +8,7 @@ import ( "github.com/gorilla/mux" ) -func CreateKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { +func PostHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) key := "/" + vars["key"] diff --git a/server/v2/update_key_handler.go b/server/v2/put_handler.go similarity index 95% rename from server/v2/update_key_handler.go rename to server/v2/put_handler.go index 60c260afca8..6a5856f9227 100644 --- a/server/v2/update_key_handler.go +++ b/server/v2/put_handler.go @@ -10,7 +10,7 @@ import ( "github.com/gorilla/mux" ) -func UpdateKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { +func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) key := "/" + vars["key"] From 53a9bd06185ee875f75ddcd4d11400699c600314 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 14 Oct 2013 22:44:17 -0700 Subject: [PATCH 111/247] feat add set command --- server/server.go | 3 +-- server/v1/set_key_handler.go | 3 +-- server/v2/put_handler.go | 3 +-- store/create_command.go | 3 +-- store/set_command.go | 38 ++++++++++++++++++++++++++++++++++++ 5 files changed, 42 insertions(+), 8 deletions(-) create mode 100644 store/set_command.go diff --git a/server/server.go b/server/server.go index 786947c81be..36df79f6fc1 100644 --- a/server/server.go +++ b/server/server.go @@ -254,11 +254,10 @@ func (s *Server) SpeedTestHandler(w http.ResponseWriter, req *http.Request) erro for i := 0; i < count; i++ { go func() { for j := 0; j < 10; j++ { - c := &store.CreateCommand{ + c := &store.SetCommand{ Key: "foo", Value: "bar", ExpireTime: time.Unix(0, 0), - Force: true, } s.peerServer.RaftServer().Do(c) } diff --git a/server/v1/set_key_handler.go b/server/v1/set_key_handler.go index acd4037c1e4..887c8c22e65 100644 --- a/server/v1/set_key_handler.go +++ b/server/v1/set_key_handler.go @@ -39,11 +39,10 @@ func SetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { } } else { - c = &store.CreateCommand{ + c = &store.SetCommand{ Key: key, Value: value, ExpireTime: expireTime, - Force: true, } } diff --git a/server/v2/put_handler.go b/server/v2/put_handler.go index 6a5856f9227..a0580f840e0 100644 --- a/server/v2/put_handler.go +++ b/server/v2/put_handler.go @@ -35,11 +35,10 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { // Set command: create a new node or replace the old one. if !valueOk && !indexOk && !existOk { - c = &store.CreateCommand{ + c = &store.SetCommand{ Key: key, Value: value, ExpireTime: expireTime, - Force: true, } return s.Dispatch(c, w, req) } diff --git a/store/create_command.go b/store/create_command.go index 43c09f99841..c1f57910a15 100644 --- a/store/create_command.go +++ b/store/create_command.go @@ -16,7 +16,6 @@ type CreateCommand struct { Value string `json:"value"` ExpireTime time.Time `json:"expireTime"` IncrementalSuffix bool `json:"incrementalSuffix"` - Force bool `json:"force"` } // The name of the create command in the log @@ -28,7 +27,7 @@ func (c *CreateCommand) CommandName() string { func (c *CreateCommand) Apply(server raft.Server) (interface{}, error) { s, _ := server.StateMachine().(Store) - e, err := s.Create(c.Key, c.Value, c.IncrementalSuffix, c.Force, c.ExpireTime, server.CommitIndex(), server.Term()) + e, err := s.Create(c.Key, c.Value, c.IncrementalSuffix, false, c.ExpireTime, server.CommitIndex(), server.Term()) if err != nil { log.Debug(err) diff --git a/store/set_command.go b/store/set_command.go new file mode 100644 index 00000000000..ac8e2cf58fa --- /dev/null +++ b/store/set_command.go @@ -0,0 +1,38 @@ +package store + +import ( + "github.com/coreos/etcd/log" + "github.com/coreos/go-raft" + "time" +) + +func init() { + raft.RegisterCommand(&CreateCommand{}) +} + +// Create command +type SetCommand struct { + Key string `json:"key"` + Value string `json:"value"` + ExpireTime time.Time `json:"expireTime"` +} + +// The name of the create command in the log +func (c *SetCommand) CommandName() string { + return "etcd:set" +} + +// Create node +func (c *SetCommand) Apply(server raft.Server) (interface{}, error) { + s, _ := server.StateMachine().(Store) + + // create a new node or replace the old node. + e, err := s.Create(c.Key, c.Value, false, true, c.ExpireTime, server.CommitIndex(), server.Term()) + + if err != nil { + log.Debug(err) + return nil, err + } + + return e, nil +} From 278a0899083fff7ea20d63fd8f6d11b45a7dd201 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 14 Oct 2013 22:45:29 -0700 Subject: [PATCH 112/247] fix set should register set rather than create --- store/set_command.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/store/set_command.go b/store/set_command.go index ac8e2cf58fa..6c2bd6885c1 100644 --- a/store/set_command.go +++ b/store/set_command.go @@ -7,7 +7,7 @@ import ( ) func init() { - raft.RegisterCommand(&CreateCommand{}) + raft.RegisterCommand(&SetCommand{}) } // Create command From fbf40fb74a1c700fe786bc251eef1cc3d89738d5 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 14 Oct 2013 23:04:21 -0700 Subject: [PATCH 113/247] refactor store.go add set function --- server/registry.go | 2 +- store/create_command.go | 2 +- store/event.go | 3 ++- store/set_command.go | 2 +- store/store.go | 18 ++++++++++++++---- 5 files changed, 19 insertions(+), 8 deletions(-) diff --git a/server/registry.go b/server/registry.go index 23ef9ddbbdb..fa63b50274a 100644 --- a/server/registry.go +++ b/server/registry.go @@ -45,7 +45,7 @@ func (r *Registry) Register(name string, peerVersion string, peerURL string, url // Write data to store. key := path.Join(RegistryKey, name) value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", peerURL, url, peerVersion) - _, err := r.store.Create(key, value, false, false, store.Permanent, commitIndex, term) + _, err := r.store.Create(key, value, false, store.Permanent, commitIndex, term) log.Debugf("Register: %s (%v)", name, err) return err } diff --git a/store/create_command.go b/store/create_command.go index c1f57910a15..b9f1aced557 100644 --- a/store/create_command.go +++ b/store/create_command.go @@ -27,7 +27,7 @@ func (c *CreateCommand) CommandName() string { func (c *CreateCommand) Apply(server raft.Server) (interface{}, error) { s, _ := server.StateMachine().(Store) - e, err := s.Create(c.Key, c.Value, c.IncrementalSuffix, false, c.ExpireTime, server.CommitIndex(), server.Term()) + e, err := s.Create(c.Key, c.Value, c.IncrementalSuffix, c.ExpireTime, server.CommitIndex(), server.Term()) if err != nil { log.Debug(err) diff --git a/store/event.go b/store/event.go index f9ae0938baf..14ba1e529d3 100644 --- a/store/event.go +++ b/store/event.go @@ -7,6 +7,7 @@ import ( const ( Get = "get" Create = "create" + Set = "set" Update = "update" Delete = "delete" CompareAndSwap = "compareAndSwap" @@ -54,7 +55,7 @@ func (event *Event) Response() interface{} { Expiration: event.Expiration, } - if response.Action == Create || response.Action == Update { + if response.Action == Create || response.Action == Set { response.Action = "set" if response.PrevValue == "" { response.NewKey = true diff --git a/store/set_command.go b/store/set_command.go index 6c2bd6885c1..55635cd9904 100644 --- a/store/set_command.go +++ b/store/set_command.go @@ -27,7 +27,7 @@ func (c *SetCommand) Apply(server raft.Server) (interface{}, error) { s, _ := server.StateMachine().(Store) // create a new node or replace the old node. - e, err := s.Create(c.Key, c.Value, false, true, c.ExpireTime, server.CommitIndex(), server.Term()) + e, err := s.Set(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) if err != nil { log.Debug(err) diff --git a/store/store.go b/store/store.go index 6e0fe4d68de..35dfd32c533 100644 --- a/store/store.go +++ b/store/store.go @@ -15,8 +15,9 @@ import ( type Store interface { Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) - Create(nodePath string, value string, incrementalSuffix bool, force bool, - expireTime time.Time, index uint64, term uint64) (*Event, error) + Set(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) + Create(nodePath string, value string, incrementalSuffix bool, expireTime time.Time, + index uint64, term uint64) (*Event, error) CompareAndSwap(nodePath string, prevValue string, prevIndex uint64, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error) @@ -106,13 +107,22 @@ func (s *store) Get(nodePath string, recursive, sorted bool, index uint64, term // Create function creates the Node at nodePath. Create will help to create intermediate directories with no ttl. // If the node has already existed, create will fail. // If any node on the path is a file, create will fail. -func (s *store) Create(nodePath string, value string, incrementalSuffix bool, force bool, +func (s *store) Create(nodePath string, value string, incrementalSuffix bool, expireTime time.Time, index uint64, term uint64) (*Event, error) { nodePath = path.Clean(path.Join("/", nodePath)) s.worldLock.Lock() defer s.worldLock.Unlock() - return s.internalCreate(nodePath, value, incrementalSuffix, force, expireTime, index, term, Create) + return s.internalCreate(nodePath, value, incrementalSuffix, false, expireTime, index, term, Create) +} + +// Set function creates or replace the Node at nodePath. +func (s *store) Set(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { + nodePath = path.Clean(path.Join("/", nodePath)) + + s.worldLock.Lock() + defer s.worldLock.Unlock() + return s.internalCreate(nodePath, value, false, true, expireTime, index, term, Set) } func (s *store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint64, From c5f9afa0e87cceeb8235def4a92166db6a06ad91 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 14 Oct 2013 23:15:31 -0700 Subject: [PATCH 114/247] fix store test --- store/stats_test.go | 4 +-- store/store.go | 4 +-- store/store_test.go | 64 ++++++++++++++++++++++----------------------- 3 files changed, 36 insertions(+), 36 deletions(-) diff --git a/store/stats_test.go b/store/stats_test.go index 7c2296fb085..6f3b342428b 100644 --- a/store/stats_test.go +++ b/store/stats_test.go @@ -16,7 +16,7 @@ func TestBasicStats(t *testing.T) { for _, k := range keys { i++ - _, err := s.Create(k, "bar", false, false, time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1) + _, err := s.Create(k, "bar", false, time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1) if err != nil { SetFail++ } else { @@ -146,7 +146,7 @@ func TestBasicStats(t *testing.T) { for _, k := range keys { i++ - _, err := s.Create(k, "bar", false, false, time.Now().Add(time.Second*3), i, 1) + _, err := s.Create(k, "bar", false, time.Now().Add(time.Second*3), i, 1) if err != nil { SetFail++ } else { diff --git a/store/store.go b/store/store.go index 35dfd32c533..8cecfb1787b 100644 --- a/store/store.go +++ b/store/store.go @@ -302,7 +302,7 @@ func (s *store) update(nodePath string, newValue string, expireTime time.Time, i return e, nil } -func (s *store) internalCreate(nodePath string, value string, incrementalSuffix bool, force bool, +func (s *store) internalCreate(nodePath string, value string, incrementalSuffix bool, replace bool, expireTime time.Time, index uint64, term uint64, action string) (*Event, error) { s.Index, s.Term = index, term @@ -330,7 +330,7 @@ func (s *store) internalCreate(nodePath string, value string, incrementalSuffix // force will try to replace a existing file if n != nil { - if force { + if replace { if n.IsDir() { return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term) } diff --git a/store/store_test.go b/store/store_test.go index f26d3d6164f..dd6a2f81868 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -10,10 +10,10 @@ import ( func TestCreateAndGet(t *testing.T) { s := newStore() - s.Create("/foobar", "bar", false, false, Permanent, 1, 1) + s.Create("/foobar", "bar", false, Permanent, 1, 1) // already exist, create should fail - _, err := s.Create("/foobar", "bar", false, false, Permanent, 1, 1) + _, err := s.Create("/foobar", "bar", false, Permanent, 1, 1) if err == nil { t.Fatal("Create should fail") @@ -21,10 +21,10 @@ func TestCreateAndGet(t *testing.T) { s.Delete("/foobar", true, 1, 1) - s.Create("/foobar/foo", "bar", false, false, Permanent, 1, 1) + s.Create("/foobar/foo", "bar", false, Permanent, 1, 1) // already exist, create should fail - _, err = s.Create("/foobar", "bar", false, false, Permanent, 1, 1) + _, err = s.Create("/foobar", "bar", false, Permanent, 1, 1) if err == nil { t.Fatal("Create should fail") @@ -38,14 +38,14 @@ func TestCreateAndGet(t *testing.T) { createAndGet(s, "/foo/foo/bar", t) // meet file, create should fail - _, err = s.Create("/foo/bar/bar", "bar", false, false, Permanent, 2, 1) + _, err = s.Create("/foo/bar/bar", "bar", false, Permanent, 2, 1) if err == nil { t.Fatal("Create should fail") } // create a directory - _, err = s.Create("/fooDir", "", false, false, Permanent, 3, 1) + _, err = s.Create("/fooDir", "", false, Permanent, 3, 1) if err != nil { t.Fatal("Cannot create /fooDir") @@ -58,7 +58,7 @@ func TestCreateAndGet(t *testing.T) { } // create a file under directory - _, err = s.Create("/fooDir/bar", "bar", false, false, Permanent, 4, 1) + _, err = s.Create("/fooDir/bar", "bar", false, Permanent, 4, 1) if err != nil { t.Fatal("Cannot create /fooDir/bar = bar") @@ -68,7 +68,7 @@ func TestCreateAndGet(t *testing.T) { func TestUpdateFile(t *testing.T) { s := newStore() - _, err := s.Create("/foo/bar", "bar", false, false, Permanent, 1, 1) + _, err := s.Create("/foo/bar", "bar", false, Permanent, 1, 1) if err != nil { t.Fatalf("cannot create %s=bar [%s]", "/foo/bar", err.Error()) @@ -91,24 +91,24 @@ func TestUpdateFile(t *testing.T) { } // create a directory, update its ttl, to see if it will be deleted - _, err = s.Create("/foo/foo", "", false, false, Permanent, 3, 1) + _, err = s.Create("/foo/foo", "", false, Permanent, 3, 1) if err != nil { t.Fatalf("cannot create dir [%s] [%s]", "/foo/foo", err.Error()) } - _, err = s.Create("/foo/foo/foo1", "bar1", false, false, Permanent, 4, 1) + _, err = s.Create("/foo/foo/foo1", "bar1", false, Permanent, 4, 1) if err != nil { t.Fatal("cannot create [%s]", err.Error()) } - _, err = s.Create("/foo/foo/foo2", "", false, false, Permanent, 5, 1) + _, err = s.Create("/foo/foo/foo2", "", false, Permanent, 5, 1) if err != nil { t.Fatal("cannot create [%s]", err.Error()) } - _, err = s.Create("/foo/foo/foo2/boo", "boo1", false, false, Permanent, 6, 1) + _, err = s.Create("/foo/foo/foo2/boo", "boo1", false, Permanent, 6, 1) if err != nil { t.Fatal("cannot create [%s]", err.Error()) } @@ -165,11 +165,11 @@ func TestListDirectory(t *testing.T) { // create dir /foo // set key-value /foo/foo=bar - s.Create("/foo/foo", "bar", false, false, Permanent, 1, 1) + s.Create("/foo/foo", "bar", false, Permanent, 1, 1) // create dir /foo/fooDir // set key-value /foo/fooDir/foo=bar - s.Create("/foo/fooDir/foo", "bar", false, false, Permanent, 2, 1) + s.Create("/foo/fooDir/foo", "bar", false, Permanent, 2, 1) e, err := s.Get("/foo", true, false, 2, 1) @@ -196,7 +196,7 @@ func TestListDirectory(t *testing.T) { // create dir /foo/_hidden // set key-value /foo/_hidden/foo -> bar - s.Create("/foo/_hidden/foo", "bar", false, false, Permanent, 3, 1) + s.Create("/foo/_hidden/foo", "bar", false, Permanent, 3, 1) e, _ = s.Get("/foo", false, false, 2, 1) @@ -208,7 +208,7 @@ func TestListDirectory(t *testing.T) { func TestRemove(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, false, Permanent, 1, 1) + s.Create("/foo", "bar", false, Permanent, 1, 1) _, err := s.Delete("/foo", false, 1, 1) if err != nil { @@ -221,9 +221,9 @@ func TestRemove(t *testing.T) { t.Fatalf("can get the node after deletion") } - s.Create("/foo/bar", "bar", false, false, Permanent, 1, 1) - s.Create("/foo/car", "car", false, false, Permanent, 1, 1) - s.Create("/foo/dar/dar", "dar", false, false, Permanent, 1, 1) + s.Create("/foo/bar", "bar", false, Permanent, 1, 1) + s.Create("/foo/car", "car", false, Permanent, 1, 1) + s.Create("/foo/dar/dar", "dar", false, Permanent, 1, 1) _, err = s.Delete("/foo", false, 1, 1) @@ -249,7 +249,7 @@ func TestExpire(t *testing.T) { expire := time.Now().Add(time.Second) - s.Create("/foo", "bar", false, false, expire, 1, 1) + s.Create("/foo", "bar", false, expire, 1, 1) _, err := s.Get("/foo", false, false, 1, 1) @@ -267,7 +267,7 @@ func TestExpire(t *testing.T) { // test if we can reach the node before expiration expire = time.Now().Add(time.Second) - s.Create("/foo", "bar", false, false, expire, 1, 1) + s.Create("/foo", "bar", false, expire, 1, 1) time.Sleep(time.Millisecond * 50) _, err = s.Get("/foo", false, false, 1, 1) @@ -278,7 +278,7 @@ func TestExpire(t *testing.T) { expire = time.Now().Add(time.Second) - s.Create("/foo", "bar", false, false, expire, 1, 1) + s.Create("/foo", "bar", false, expire, 1, 1) _, err = s.Delete("/foo", false, 1, 1) if err != nil { @@ -288,7 +288,7 @@ func TestExpire(t *testing.T) { func TestCompareAndSwap(t *testing.T) { // TODO prevValue == nil ? s := newStore() - s.Create("/foo", "bar", false, false, Permanent, 1, 1) + s.Create("/foo", "bar", false, Permanent, 1, 1) // test on wrong previous value _, err := s.CompareAndSwap("/foo", "barbar", 0, "car", Permanent, 2, 1) @@ -323,7 +323,7 @@ func TestWatch(t *testing.T) { s := newStore() // watch at a deeper path c, _ := s.Watch("/foo/foo/foo", false, 0, 0, 1) - s.Create("/foo/foo/foo", "bar", false, false, Permanent, 1, 1) + s.Create("/foo/foo/foo", "bar", false, Permanent, 1, 1) e := nonblockingRetrive(c) if e.Key != "/foo/foo/foo" || e.Action != Create { @@ -353,7 +353,7 @@ func TestWatch(t *testing.T) { // watch at a prefix c, _ = s.Watch("/foo", true, 0, 4, 1) - s.Create("/foo/foo/boo", "bar", false, false, Permanent, 5, 1) + s.Create("/foo/foo/boo", "bar", false, Permanent, 5, 1) e = nonblockingRetrive(c) if e.Key != "/foo/foo/boo" || e.Action != Create { t.Fatal("watch for Create subdirectory fails") @@ -381,7 +381,7 @@ func TestWatch(t *testing.T) { } // watch expire - s.Create("/foo/foo/boo", "foo", false, false, time.Now().Add(time.Second*1), 9, 1) + s.Create("/foo/foo/boo", "foo", false, time.Now().Add(time.Second*1), 9, 1) c, _ = s.Watch("/foo", true, 0, 9, 1) time.Sleep(time.Second * 2) e = nonblockingRetrive(c) @@ -389,7 +389,7 @@ func TestWatch(t *testing.T) { t.Fatal("watch for Expiration of Create() subdirectory fails ", e) } - s.Create("/foo/foo/boo", "foo", false, false, Permanent, 10, 1) + s.Create("/foo/foo/boo", "foo", false, Permanent, 10, 1) s.update("/foo/foo/boo", "bar", time.Now().Add(time.Second*1), 11, 1) c, _ = s.Watch("/foo", true, 0, 11, 1) time.Sleep(time.Second * 2) @@ -398,7 +398,7 @@ func TestWatch(t *testing.T) { t.Fatal("watch for Expiration of Update() subdirectory fails ", e) } - s.Create("/foo/foo/boo", "foo", false, false, Permanent, 12, 1) + s.Create("/foo/foo/boo", "foo", false, Permanent, 12, 1) s.CompareAndSwap("/foo/foo/boo", "foo", 0, "bar", time.Now().Add(time.Second*1), 13, 1) c, _ = s.Watch("/foo", true, 0, 13, 1) time.Sleep(time.Second * 2) @@ -416,7 +416,7 @@ func TestSort(t *testing.T) { i := uint64(1) for _, k := range keys { - _, err := s.Create(k, "bar", false, false, Permanent, i, 1) + _, err := s.Create(k, "bar", false, Permanent, i, 1) if err != nil { panic(err) } else { @@ -454,7 +454,7 @@ func TestSaveAndRecover(t *testing.T) { i := uint64(1) for _, k := range keys { - _, err := s.Create(k, "bar", false, false, Permanent, i, 1) + _, err := s.Create(k, "bar", false, Permanent, i, 1) if err != nil { panic(err) } else { @@ -466,7 +466,7 @@ func TestSaveAndRecover(t *testing.T) { // test if we can reach the node before expiration expire := time.Now().Add(time.Second) - s.Create("/foo/foo", "bar", false, false, expire, 1, 1) + s.Create("/foo/foo", "bar", false, expire, 1, 1) b, err := s.Save() cloneFs := newStore() @@ -522,7 +522,7 @@ func GenKeys(num int, depth int) []string { } func createAndGet(s *store, path string, t *testing.T) { - _, err := s.Create(path, "bar", false, false, Permanent, 1, 1) + _, err := s.Create(path, "bar", false, Permanent, 1, 1) if err != nil { t.Fatalf("cannot create %s=bar [%s]", path, err.Error()) From 01bbad31c773a51c2433d727ddbc47d08c346f29 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 14 Oct 2013 23:16:48 -0700 Subject: [PATCH 115/247] refactor remove web pacakge --- web/conn.go | 30 ------------------- web/file2gostring.sh | 28 ------------------ web/hub.go | 61 -------------------------------------- web/index.go | 5 ---- web/index.html | 70 -------------------------------------------- web/web.go | 50 ------------------------------- 6 files changed, 244 deletions(-) delete mode 100644 web/conn.go delete mode 100755 web/file2gostring.sh delete mode 100644 web/hub.go delete mode 100644 web/index.go delete mode 100644 web/index.html delete mode 100644 web/web.go diff --git a/web/conn.go b/web/conn.go deleted file mode 100644 index 25e871635fd..00000000000 --- a/web/conn.go +++ /dev/null @@ -1,30 +0,0 @@ -package web - -import ( - "code.google.com/p/go.net/websocket" -) - -type connection struct { - // The websocket connection. - ws *websocket.Conn - - // Buffered channel of outbound messages. - send chan string -} - -func (c *connection) writer() { - for message := range c.send { - err := websocket.Message.Send(c.ws, message) - if err != nil { - break - } - } - c.ws.Close() -} - -func wsHandler(ws *websocket.Conn) { - c := &connection{send: make(chan string, 256), ws: ws} - h.register <- c - defer func() { h.unregister <- c }() - c.writer() -} diff --git a/web/file2gostring.sh b/web/file2gostring.sh deleted file mode 100755 index 483b5fd3143..00000000000 --- a/web/file2gostring.sh +++ /dev/null @@ -1,28 +0,0 @@ -#!/bin/sh - -# this file is copied from doozerd. - -set -e - -munge() { - printf %s "$1" | tr . _ | tr -d -c '[:alnum:]_' -} - -quote() { - sed 's/\\/\\\\/g' | sed 's/"/\\"/g' | sed 's/$/\\n/' | tr -d '\n' -} - -pkg_path=$1 ; shift -file=$1 ; shift - -pkg=`basename $pkg_path` - -printf 'package %s\n' "$pkg" -printf '\n' -printf '// This file was generated from %s.\n' "$file" -printf '\n' -printf 'var ' -munge "`basename $file`" -printf ' string = "' -quote -printf '"\n' \ No newline at end of file diff --git a/web/hub.go b/web/hub.go deleted file mode 100644 index 47f203f725a..00000000000 --- a/web/hub.go +++ /dev/null @@ -1,61 +0,0 @@ -package web - -type hub struct { - // status - open bool - - // Registered connections. - connections map[*connection]bool - - // Inbound messages from the connections. - broadcast chan string - - // Register requests from the connections. - register chan *connection - - // Unregister requests from connections. - unregister chan *connection -} - -var h = hub{ - open: false, - broadcast: make(chan string), - register: make(chan *connection), - unregister: make(chan *connection), - connections: make(map[*connection]bool), -} - -func Hub() *hub { - return &h -} - -func HubOpen() bool { - return h.open -} - -func (h *hub) run() { - h.open = true - for { - select { - case c := <-h.register: - h.connections[c] = true - case c := <-h.unregister: - delete(h.connections, c) - close(c.send) - case m := <-h.broadcast: - for c := range h.connections { - select { - case c.send <- m: - default: - delete(h.connections, c) - close(c.send) - go c.ws.Close() - } - } - } - } -} - -func (h *hub) Send(msg string) { - h.broadcast <- msg -} diff --git a/web/index.go b/web/index.go deleted file mode 100644 index 2e30b6d77e1..00000000000 --- a/web/index.go +++ /dev/null @@ -1,5 +0,0 @@ -package web - -// This file was generated from index.html. - -var index_html string = "\n\netcd Web Interface\n\n\n\n\n

\n
\n\n\n" diff --git a/web/index.html b/web/index.html deleted file mode 100644 index 919bc98b26a..00000000000 --- a/web/index.html +++ /dev/null @@ -1,70 +0,0 @@ - - -etcd Web Interface - - - - -
Leader: {{.Leader}}
-
- - diff --git a/web/web.go b/web/web.go deleted file mode 100644 index 723eb05c8a9..00000000000 --- a/web/web.go +++ /dev/null @@ -1,50 +0,0 @@ -package web - -import ( - "code.google.com/p/go.net/websocket" - "fmt" - "github.com/coreos/go-raft" - "html/template" - "net/http" - "net/url" -) - -var mainTempl *template.Template -var mainPage *MainPage - -type MainPage struct { - Leader string - Address string -} - -func mainHandler(c http.ResponseWriter, req *http.Request) { - p := mainPage - - mainTempl.Execute(c, p) -} - -func Start(raftServer raft.Server, webURL string) { - u, _ := url.Parse(webURL) - - webMux := http.NewServeMux() - - server := &http.Server{ - Handler: webMux, - Addr: u.Host, - } - - mainPage = &MainPage{ - Leader: raftServer.Leader(), - Address: u.Host, - } - - mainTempl = template.Must(template.New("index.html").Parse(index_html)) - - go h.run() - webMux.HandleFunc("/", mainHandler) - webMux.Handle("/ws", websocket.Handler(wsHandler)) - - fmt.Printf("etcd web server [%s] listening on %s\n", raftServer.Name(), u) - - server.ListenAndServe() -} From baa683b48443fd1fc4012aa3ef3370a4a6a37687 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 15 Oct 2013 22:21:55 -0700 Subject: [PATCH 116/247] feat POST-create unique node under given path --- server/v2/post_handler.go | 8 ++++---- store/create_command.go | 10 +++++----- store/store.go | 10 +++++----- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/server/v2/post_handler.go b/server/v2/post_handler.go index dda146e5e14..b55eddba4bf 100644 --- a/server/v2/post_handler.go +++ b/server/v2/post_handler.go @@ -19,10 +19,10 @@ func PostHandler(w http.ResponseWriter, req *http.Request, s Server) error { } c := &store.CreateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - IncrementalSuffix: (req.FormValue("incremental") == "true"), + Key: key, + Value: value, + ExpireTime: expireTime, + Unique: true, } return s.Dispatch(c, w, req) diff --git a/store/create_command.go b/store/create_command.go index b9f1aced557..6a2487cf0ee 100644 --- a/store/create_command.go +++ b/store/create_command.go @@ -12,10 +12,10 @@ func init() { // Create command type CreateCommand struct { - Key string `json:"key"` - Value string `json:"value"` - ExpireTime time.Time `json:"expireTime"` - IncrementalSuffix bool `json:"incrementalSuffix"` + Key string `json:"key"` + Value string `json:"value"` + ExpireTime time.Time `json:"expireTime"` + Unique bool `json:"unique"` } // The name of the create command in the log @@ -27,7 +27,7 @@ func (c *CreateCommand) CommandName() string { func (c *CreateCommand) Apply(server raft.Server) (interface{}, error) { s, _ := server.StateMachine().(Store) - e, err := s.Create(c.Key, c.Value, c.IncrementalSuffix, c.ExpireTime, server.CommitIndex(), server.Term()) + e, err := s.Create(c.Key, c.Value, c.Unique, c.ExpireTime, server.CommitIndex(), server.Term()) if err != nil { log.Debug(err) diff --git a/store/store.go b/store/store.go index 8cecfb1787b..73d56bee5ac 100644 --- a/store/store.go +++ b/store/store.go @@ -107,13 +107,13 @@ func (s *store) Get(nodePath string, recursive, sorted bool, index uint64, term // Create function creates the Node at nodePath. Create will help to create intermediate directories with no ttl. // If the node has already existed, create will fail. // If any node on the path is a file, create will fail. -func (s *store) Create(nodePath string, value string, incrementalSuffix bool, +func (s *store) Create(nodePath string, value string, unique bool, expireTime time.Time, index uint64, term uint64) (*Event, error) { nodePath = path.Clean(path.Join("/", nodePath)) s.worldLock.Lock() defer s.worldLock.Unlock() - return s.internalCreate(nodePath, value, incrementalSuffix, false, expireTime, index, term, Create) + return s.internalCreate(nodePath, value, unique, false, expireTime, index, term, Create) } // Set function creates or replace the Node at nodePath. @@ -302,13 +302,13 @@ func (s *store) update(nodePath string, newValue string, expireTime time.Time, i return e, nil } -func (s *store) internalCreate(nodePath string, value string, incrementalSuffix bool, replace bool, +func (s *store) internalCreate(nodePath string, value string, unique bool, replace bool, expireTime time.Time, index uint64, term uint64, action string) (*Event, error) { s.Index, s.Term = index, term - if incrementalSuffix { // append unique incremental suffix to the node path - nodePath += "_" + strconv.FormatUint(index, 10) + if unique { // append unique item under the node path + nodePath += "/" + strconv.FormatUint(index, 10) } nodePath = path.Clean(path.Join("/", nodePath)) From 0392c187945888e9d29d45e21293d1f85b11008f Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 15 Oct 2013 23:18:03 -0700 Subject: [PATCH 117/247] refactor put_handler.go --- server/v2/put_handler.go | 47 ++++++++++++++++++++++++++++++---------- store/stats.go | 36 +++++++++++++++++++----------- store/stats_test.go | 26 +++++++++++----------- store/store.go | 36 ++++++++++++++++++------------ store/store_test.go | 10 ++++----- store/update_command.go | 37 +++++++++++++++++++++++++++++++ 6 files changed, 135 insertions(+), 57 deletions(-) create mode 100644 store/update_command.go diff --git a/server/v2/put_handler.go b/server/v2/put_handler.go index a0580f840e0..5dcf061c2b8 100644 --- a/server/v2/put_handler.go +++ b/server/v2/put_handler.go @@ -3,6 +3,7 @@ package v2 import ( "net/http" "strconv" + "time" etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/store" @@ -33,14 +34,9 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { var c raft.Command - // Set command: create a new node or replace the old one. + // Set handler: create a new node or replace the old one. if !valueOk && !indexOk && !existOk { - c = &store.SetCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - } - return s.Dispatch(c, w, req) + return SetHandler(w, req, s, key, value, expireTime) } // update with test @@ -48,11 +44,11 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { if prevExist[0] == "false" { // Create command: create a new node. Fail, if a node already exists // Ignore prevIndex and prevValue - c = &store.CreateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - } + return CreateHandler(w, req, s, key, value, expireTime) + } + + if prevExist[0] == "true" && !indexOk && !valueOk { + return UpdateHandler(w, req, s, key, value, expireTime) } } @@ -84,3 +80,30 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { return s.Dispatch(c, w, req) } + +func SetHandler(w http.ResponseWriter, req *http.Request, s Server, key, value string, expireTime time.Time) error { + c := &store.SetCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + } + return s.Dispatch(c, w, req) +} + +func CreateHandler(w http.ResponseWriter, req *http.Request, s Server, key, value string, expireTime time.Time) error { + c := &store.CreateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + } + return s.Dispatch(c, w, req) +} + +func UpdateHandler(w http.ResponseWriter, req *http.Request, s Server, key, value string, expireTime time.Time) error { + c := &store.UpdateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + } + return s.Dispatch(c, w, req) +} diff --git a/store/stats.go b/store/stats.go index 4c89b93e38c..5f4d26d91a2 100644 --- a/store/stats.go +++ b/store/stats.go @@ -6,17 +6,19 @@ import ( ) const ( - SetSuccess = 100 - SetFail = 101 - DeleteSuccess = 102 - DeleteFail = 103 - UpdateSuccess = 104 - UpdateFail = 105 - CompareAndSwapSuccess = 106 - CompareAndSwapFail = 107 - GetSuccess = 110 - GetFail = 111 - ExpireCount = 112 + SetSuccess = iota + SetFail + DeleteSuccess + DeleteFail + CreateSuccess + CreateFail + UpdateSuccess + UpdateFail + CompareAndSwapSuccess + CompareAndSwapFail + GetSuccess + GetFail + ExpireCount ) type Stats struct { @@ -37,6 +39,10 @@ type Stats struct { UpdateSuccess uint64 `json:"updateSuccess"` UpdateFail uint64 `json:"updateFail"` + // Number of create requests + CreateSuccess uint64 `json:"createSuccess"` + CreateFail uint64 `json:createFail` + // Number of testAndSet requests CompareAndSwapSuccess uint64 `json:"compareAndSwapSuccess"` CompareAndSwapFail uint64 `json:"compareAndSwapFail"` @@ -53,8 +59,8 @@ func newStats() *Stats { func (s *Stats) clone() *Stats { return &Stats{s.GetSuccess, s.GetFail, s.SetSuccess, s.SetFail, - s.DeleteSuccess, s.DeleteFail, s.UpdateSuccess, s.UpdateFail, - s.CompareAndSwapSuccess, s.CompareAndSwapFail, s.Watchers, s.ExpireCount} + s.DeleteSuccess, s.DeleteFail, s.UpdateSuccess, s.UpdateFail, s.CreateSuccess, + s.CreateFail, s.CompareAndSwapSuccess, s.CompareAndSwapFail, s.Watchers, s.ExpireCount} } // Status() return the statistics info of etcd storage its recent start @@ -80,6 +86,10 @@ func (s *Stats) Inc(field int) { atomic.AddUint64(&s.SetSuccess, 1) case SetFail: atomic.AddUint64(&s.SetFail, 1) + case CreateSuccess: + atomic.AddUint64(&s.CreateSuccess, 1) + case CreateFail: + atomic.AddUint64(&s.CreateFail, 1) case DeleteSuccess: atomic.AddUint64(&s.DeleteSuccess, 1) case DeleteFail: diff --git a/store/stats_test.go b/store/stats_test.go index 6f3b342428b..41c032ea57d 100644 --- a/store/stats_test.go +++ b/store/stats_test.go @@ -11,16 +11,16 @@ func TestBasicStats(t *testing.T) { keys := GenKeys(rand.Intn(100), 5) var i uint64 - var GetSuccess, GetFail, SetSuccess, SetFail, DeleteSuccess, DeleteFail uint64 + var GetSuccess, GetFail, CreateSuccess, CreateFail, DeleteSuccess, DeleteFail uint64 var UpdateSuccess, UpdateFail, CompareAndSwapSuccess, CompareAndSwapFail, watcher_number uint64 for _, k := range keys { i++ _, err := s.Create(k, "bar", false, time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1) if err != nil { - SetFail++ + CreateFail++ } else { - SetSuccess++ + CreateSuccess++ } } @@ -37,7 +37,7 @@ func TestBasicStats(t *testing.T) { for _, k := range keys { i++ - _, err := s.update(k, "foo", time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1) + _, err := s.Update(k, "foo", time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1) if err != nil { UpdateFail++ } else { @@ -108,12 +108,12 @@ func TestBasicStats(t *testing.T) { t.Fatalf("GetFail [%d] != Stats.GetFail [%d]", GetFail, s.Stats.GetFail) } - if SetSuccess != s.Stats.SetSuccess { - t.Fatalf("SetSuccess [%d] != Stats.SetSuccess [%d]", SetSuccess, s.Stats.SetSuccess) + if CreateSuccess != s.Stats.CreateSuccess { + t.Fatalf("CreateSuccess [%d] != Stats.CreateSuccess [%d]", CreateSuccess, s.Stats.CreateSuccess) } - if SetFail != s.Stats.SetFail { - t.Fatalf("SetFail [%d] != Stats.SetFail [%d]", SetFail, s.Stats.SetFail) + if CreateFail != s.Stats.CreateFail { + t.Fatalf("CreateFail [%d] != Stats.CreateFail [%d]", CreateFail, s.Stats.CreateFail) } if DeleteSuccess != s.Stats.DeleteSuccess { @@ -141,22 +141,22 @@ func TestBasicStats(t *testing.T) { } s = newStore() - SetSuccess = 0 - SetFail = 0 + CreateSuccess = 0 + CreateFail = 0 for _, k := range keys { i++ _, err := s.Create(k, "bar", false, time.Now().Add(time.Second*3), i, 1) if err != nil { - SetFail++ + CreateFail++ } else { - SetSuccess++ + CreateSuccess++ } } time.Sleep(6 * time.Second) - ExpireCount := SetSuccess + ExpireCount := CreateSuccess if ExpireCount != s.Stats.ExpireCount { t.Fatalf("ExpireCount [%d] != Stats.ExpireCount [%d]", ExpireCount, s.Stats.ExpireCount) diff --git a/store/store.go b/store/store.go index 73d56bee5ac..008ac26d5f3 100644 --- a/store/store.go +++ b/store/store.go @@ -16,6 +16,7 @@ import ( type Store interface { Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) Set(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) + Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) Create(nodePath string, value string, incrementalSuffix bool, expireTime time.Time, index uint64, term uint64) (*Event, error) CompareAndSwap(nodePath string, prevValue string, prevIndex uint64, @@ -113,7 +114,15 @@ func (s *store) Create(nodePath string, value string, unique bool, s.worldLock.Lock() defer s.worldLock.Unlock() - return s.internalCreate(nodePath, value, unique, false, expireTime, index, term, Create) + e, err := s.internalCreate(nodePath, value, unique, false, expireTime, index, term, Create) + + if err == nil { + s.Stats.Inc(CreateSuccess) + } else { + s.Stats.Inc(CreateFail) + } + + return e, err } // Set function creates or replace the Node at nodePath. @@ -122,7 +131,15 @@ func (s *store) Set(nodePath string, value string, expireTime time.Time, index u s.worldLock.Lock() defer s.worldLock.Unlock() - return s.internalCreate(nodePath, value, false, true, expireTime, index, term, Set) + e, err := s.internalCreate(nodePath, value, false, true, expireTime, index, term, Set) + + if err == nil { + s.Stats.Inc(SetSuccess) + } else { + s.Stats.Inc(SetFail) + } + + return e, err } func (s *store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint64, @@ -133,10 +150,6 @@ func (s *store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint s.worldLock.Lock() defer s.worldLock.Unlock() - if prevValue == "" && prevIndex == 0 { // try just update - return s.update(nodePath, value, expireTime, index, term) - } - n, err := s.internalGet(nodePath, index, term) if err != nil { @@ -265,7 +278,7 @@ func (s *store) walk(nodePath string, walkFunc func(prev *Node, component string // Update function updates the value/ttl of the node. // If the node is a file, the value and the ttl can be updated. // If the node is a directory, only the ttl can be updated. -func (s *store) update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) { +func (s *store) Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) { s.worldLock.Lock() defer s.worldLock.Unlock() nodePath = path.Clean(path.Join("/", nodePath)) @@ -354,12 +367,8 @@ func (s *store) internalCreate(nodePath string, value string, unique bool, repla } - err = d.Add(n) - - if err != nil { - s.Stats.Inc(SetFail) - return nil, err - } + // we are sure d is a directory and does not have the children with name n.Name + d.Add(n) // Node with TTL if expireTime.Sub(Permanent) != 0 { @@ -368,7 +377,6 @@ func (s *store) internalCreate(nodePath string, value string, unique bool, repla } s.WatcherHub.notify(e) - s.Stats.Inc(SetSuccess) return e, nil } diff --git a/store/store_test.go b/store/store_test.go index dd6a2f81868..20b1131c3d4 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -74,7 +74,7 @@ func TestUpdateFile(t *testing.T) { t.Fatalf("cannot create %s=bar [%s]", "/foo/bar", err.Error()) } - _, err = s.update("/foo/bar", "barbar", Permanent, 2, 1) + _, err = s.Update("/foo/bar", "barbar", Permanent, 2, 1) if err != nil { t.Fatalf("cannot update %s=barbar [%s]", "/foo/bar", err.Error()) @@ -114,7 +114,7 @@ func TestUpdateFile(t *testing.T) { } expire := time.Now().Add(time.Second * 2) - _, err = s.update("/foo/foo", "", expire, 7, 1) + _, err = s.Update("/foo/foo", "", expire, 7, 1) if err != nil { t.Fatalf("cannot update dir [%s] [%s]", "/foo/foo", err.Error()) } @@ -331,7 +331,7 @@ func TestWatch(t *testing.T) { } c, _ = s.Watch("/foo/foo/foo", false, 0, 1, 1) - s.update("/foo/foo/foo", "car", Permanent, 2, 1) + s.Update("/foo/foo/foo", "car", Permanent, 2, 1) e = nonblockingRetrive(c) if e.Key != "/foo/foo/foo" || e.Action != Update { t.Fatal("watch for Update node fails ", e) @@ -360,7 +360,7 @@ func TestWatch(t *testing.T) { } c, _ = s.Watch("/foo", true, 0, 5, 1) - s.update("/foo/foo/boo", "foo", Permanent, 6, 1) + s.Update("/foo/foo/boo", "foo", Permanent, 6, 1) e = nonblockingRetrive(c) if e.Key != "/foo/foo/boo" || e.Action != Update { t.Fatal("watch for Update subdirectory fails") @@ -390,7 +390,7 @@ func TestWatch(t *testing.T) { } s.Create("/foo/foo/boo", "foo", false, Permanent, 10, 1) - s.update("/foo/foo/boo", "bar", time.Now().Add(time.Second*1), 11, 1) + s.Update("/foo/foo/boo", "bar", time.Now().Add(time.Second*1), 11, 1) c, _ = s.Watch("/foo", true, 0, 11, 1) time.Sleep(time.Second * 2) e = nonblockingRetrive(c) diff --git a/store/update_command.go b/store/update_command.go new file mode 100644 index 00000000000..8e353cdd068 --- /dev/null +++ b/store/update_command.go @@ -0,0 +1,37 @@ +package store + +import ( + "github.com/coreos/etcd/log" + "github.com/coreos/go-raft" + "time" +) + +func init() { + raft.RegisterCommand(&UpdateCommand{}) +} + +// Update command +type UpdateCommand struct { + Key string `json:"key"` + Value string `json:"value"` + ExpireTime time.Time `json:"expireTime"` +} + +// The name of the update command in the log +func (c *UpdateCommand) CommandName() string { + return "etcd:update" +} + +// Create node +func (c *UpdateCommand) Apply(server raft.Server) (interface{}, error) { + s, _ := server.StateMachine().(Store) + + e, err := s.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) + + if err != nil { + log.Debug(err) + return nil, err + } + + return e, nil +} From e680f28c2f02ce819f2d6e7f1a4f499aa14d8f86 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 15 Oct 2013 23:25:12 -0700 Subject: [PATCH 118/247] fix move update check to update handler --- server/v2/put_handler.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/server/v2/put_handler.go b/server/v2/put_handler.go index 5dcf061c2b8..336366d38eb 100644 --- a/server/v2/put_handler.go +++ b/server/v2/put_handler.go @@ -23,11 +23,6 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", store.UndefIndex, store.UndefTerm) } - // Update should give at least one option - if value == "" && expireTime.Sub(store.Permanent) == 0 { - return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", store.UndefIndex, store.UndefTerm) - } - prevValue, valueOk := req.Form["prevValue"] prevIndexStr, indexOk := req.Form["prevIndex"] prevExist, existOk := req.Form["prevExist"] @@ -100,6 +95,11 @@ func CreateHandler(w http.ResponseWriter, req *http.Request, s Server, key, valu } func UpdateHandler(w http.ResponseWriter, req *http.Request, s Server, key, value string, expireTime time.Time) error { + // Update should give at least one option + if value == "" && expireTime.Sub(store.Permanent) == 0 { + return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", store.UndefIndex, store.UndefTerm) + } + c := &store.UpdateCommand{ Key: key, Value: value, From b4d311d6a1a04c70803c04f228c99946a02ec12b Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Wed, 16 Oct 2013 13:52:35 -0600 Subject: [PATCH 119/247] Reintegrate functional tests into etcd. --- .travis.yml | 3 - test.sh | 10 +- tests/functional/etcd_direct_call.go | 34 +++ tests/functional/init.go | 17 ++ tests/functional/internal_version_test.go | 57 +++++ tests/functional/kill_leader_test.go | 63 ++++++ tests/functional/kill_random_test.go | 76 +++++++ .../multi_node_kill_all_and_recovery_test.go | 72 ++++++ tests/functional/multi_node_kill_one_test.go | 58 +++++ tests/functional/remove_node_test.go | 113 ++++++++++ tests/functional/simple_multi_node_test.go | 62 ++++++ tests/functional/single_node_recovery_test.go | 68 ++++++ tests/functional/single_node_test.go | 77 +++++++ tests/functional/util.go | 205 ++++++++++++++++++ 14 files changed, 906 insertions(+), 9 deletions(-) create mode 100644 tests/functional/etcd_direct_call.go create mode 100644 tests/functional/init.go create mode 100644 tests/functional/internal_version_test.go create mode 100644 tests/functional/kill_leader_test.go create mode 100644 tests/functional/kill_random_test.go create mode 100644 tests/functional/multi_node_kill_all_and_recovery_test.go create mode 100644 tests/functional/multi_node_kill_one_test.go create mode 100644 tests/functional/remove_node_test.go create mode 100644 tests/functional/simple_multi_node_test.go create mode 100644 tests/functional/single_node_recovery_test.go create mode 100644 tests/functional/single_node_test.go create mode 100644 tests/functional/util.go diff --git a/.travis.yml b/.travis.yml index 1c07835fddb..708ec2d505f 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,8 +1,5 @@ language: go go: 1.1 -install: - - go get -u github.com/coreos/etcd-test-runner - script: - ./test.sh diff --git a/test.sh b/test.sh index c40f0c7e191..9695e70c2c2 100755 --- a/test.sh +++ b/test.sh @@ -3,10 +3,8 @@ # Get GOPATH, etc from build . ./build -# Run the tests! -go test -i -go test -v +# Unit tests +go test -v ./store -# Run the functional tests! -go test -i github.com/coreos/etcd-test-runner -ETCD_BIN_PATH=$(pwd)/etcd go test -v github.com/coreos/etcd-test-runner +# Functional tests +ETCD_BIN_PATH=$(pwd)/etcd go test -v ./tests/functional diff --git a/tests/functional/etcd_direct_call.go b/tests/functional/etcd_direct_call.go new file mode 100644 index 00000000000..6f629b4fd88 --- /dev/null +++ b/tests/functional/etcd_direct_call.go @@ -0,0 +1,34 @@ +package test + +import ( + "net/http" + "os" + "testing" + "time" +) + +func BenchmarkEtcdDirectCall(b *testing.B) { + templateBenchmarkEtcdDirectCall(b, false) +} + +func BenchmarkEtcdDirectCallTls(b *testing.B) { + templateBenchmarkEtcdDirectCall(b, true) +} + +func templateBenchmarkEtcdDirectCall(b *testing.B, tls bool) { + procAttr := new(os.ProcAttr) + procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} + + clusterSize := 3 + _, etcds, _ := CreateCluster(clusterSize, procAttr, tls) + + defer DestroyCluster(etcds) + + time.Sleep(time.Second) + + b.ResetTimer() + for i := 0; i < b.N; i++ { + resp, _ := http.Get("http://127.0.0.1:4001/test/speed") + resp.Body.Close() + } +} diff --git a/tests/functional/init.go b/tests/functional/init.go new file mode 100644 index 00000000000..48e4284b1e3 --- /dev/null +++ b/tests/functional/init.go @@ -0,0 +1,17 @@ +package test + +import ( + "go/build" + "os" + "path/filepath" +) + +var EtcdBinPath string + +func init() { + // Initialize the 'etcd' binary path or default it to the etcd diretory. + EtcdBinPath = os.Getenv("ETCD_BIN_PATH") + if EtcdBinPath == "" { + EtcdBinPath = filepath.Join(build.Default.GOPATH, "src", "github.com", "coreos", "etcd", "etcd") + } +} diff --git a/tests/functional/internal_version_test.go b/tests/functional/internal_version_test.go new file mode 100644 index 00000000000..3413928c15c --- /dev/null +++ b/tests/functional/internal_version_test.go @@ -0,0 +1,57 @@ +package test + +import ( + "fmt" + "net/http" + "net/http/httptest" + "net/url" + "os" + "testing" + "time" +) + +// Ensure that etcd does not come up if the internal raft versions do not match. +func TestInternalVersion(t *testing.T) { + checkedVersion := false + testMux := http.NewServeMux() + + testMux.HandleFunc("/version", func(w http.ResponseWriter, r *http.Request) { + fmt.Fprintln(w, "This is not a version number") + checkedVersion = true + }) + + testMux.HandleFunc("/join", func(w http.ResponseWriter, r *http.Request) { + t.Fatal("should not attempt to join!") + }) + + ts := httptest.NewServer(testMux) + defer ts.Close() + + fakeURL, _ := url.Parse(ts.URL) + + procAttr := new(os.ProcAttr) + procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} + args := []string{"etcd", "-n=node1", "-f", "-d=/tmp/node1", "-C=" + fakeURL.Host} + + process, err := os.StartProcess(EtcdBinPath, args, procAttr) + if err != nil { + t.Fatal("start process failed:" + err.Error()) + return + } + defer process.Kill() + + time.Sleep(time.Second) + + _, err = http.Get("http://127.0.0.1:4001") + + if err == nil { + t.Fatal("etcd node should not be up") + return + } + + if checkedVersion == false { + t.Fatal("etcd did not check the version") + return + } +} + diff --git a/tests/functional/kill_leader_test.go b/tests/functional/kill_leader_test.go new file mode 100644 index 00000000000..ad9113695bc --- /dev/null +++ b/tests/functional/kill_leader_test.go @@ -0,0 +1,63 @@ +package test + +import ( + "fmt" + "os" + "strconv" + "strings" + "testing" + "time" +) + +// This test will kill the current leader and wait for the etcd cluster to elect a new leader for 200 times. +// It will print out the election time and the average election time. +func TestKillLeader(t *testing.T) { + procAttr := new(os.ProcAttr) + procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} + + clusterSize := 5 + argGroup, etcds, err := CreateCluster(clusterSize, procAttr, false) + if err != nil { + t.Fatal("cannot create cluster") + } + defer DestroyCluster(etcds) + + stop := make(chan bool) + leaderChan := make(chan string, 1) + all := make(chan bool, 1) + + time.Sleep(time.Second) + + go Monitor(clusterSize, 1, leaderChan, all, stop) + + var totalTime time.Duration + + leader := "http://127.0.0.1:7001" + + for i := 0; i < clusterSize; i++ { + fmt.Println("leader is ", leader) + port, _ := strconv.Atoi(strings.Split(leader, ":")[2]) + num := port - 7001 + fmt.Println("kill server ", num) + etcds[num].Kill() + etcds[num].Release() + + start := time.Now() + for { + newLeader := <-leaderChan + if newLeader != leader { + leader = newLeader + break + } + } + take := time.Now().Sub(start) + + totalTime += take + avgTime := totalTime / (time.Duration)(i+1) + fmt.Println("Total time:", totalTime, "; Avg time:", avgTime) + + etcds[num], err = os.StartProcess(EtcdBinPath, argGroup[num], procAttr) + } + stop <- true +} + diff --git a/tests/functional/kill_random_test.go b/tests/functional/kill_random_test.go new file mode 100644 index 00000000000..4fc3c11713e --- /dev/null +++ b/tests/functional/kill_random_test.go @@ -0,0 +1,76 @@ +package test + +import ( + "fmt" + "math/rand" + "os" + "testing" + "time" +) + +// TestKillRandom kills random machines in the cluster and +// restart them after all other machines agree on the same leader +func TestKillRandom(t *testing.T) { + procAttr := new(os.ProcAttr) + procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} + + clusterSize := 9 + argGroup, etcds, err := CreateCluster(clusterSize, procAttr, false) + + if err != nil { + t.Fatal("cannot create cluster") + } + + defer DestroyCluster(etcds) + + stop := make(chan bool) + leaderChan := make(chan string, 1) + all := make(chan bool, 1) + + time.Sleep(3 * time.Second) + + go Monitor(clusterSize, 4, leaderChan, all, stop) + + toKill := make(map[int]bool) + + for i := 0; i < 20; i++ { + fmt.Printf("TestKillRandom Round[%d/20]\n", i) + + j := 0 + for { + + r := rand.Int31n(9) + if _, ok := toKill[int(r)]; !ok { + j++ + toKill[int(r)] = true + } + + if j > 3 { + break + } + + } + + for num, _ := range toKill { + err := etcds[num].Kill() + if err != nil { + panic(err) + } + etcds[num].Wait() + } + + time.Sleep(1 * time.Second) + + <-leaderChan + + for num, _ := range toKill { + etcds[num], err = os.StartProcess(EtcdBinPath, argGroup[num], procAttr) + } + + toKill = make(map[int]bool) + <-all + } + + stop <- true +} + diff --git a/tests/functional/multi_node_kill_all_and_recovery_test.go b/tests/functional/multi_node_kill_all_and_recovery_test.go new file mode 100644 index 00000000000..454b9648c21 --- /dev/null +++ b/tests/functional/multi_node_kill_all_and_recovery_test.go @@ -0,0 +1,72 @@ +package test + +import ( + "os" + "testing" + "time" + + "github.com/coreos/go-etcd/etcd" +) + +// Create a five nodes +// Kill all the nodes and restart +func TestMultiNodeKillAllAndRecovery(t *testing.T) { + procAttr := new(os.ProcAttr) + procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} + + clusterSize := 5 + argGroup, etcds, err := CreateCluster(clusterSize, procAttr, false) + defer DestroyCluster(etcds) + + if err != nil { + t.Fatal("cannot create cluster") + } + + c := etcd.NewClient(nil) + + c.SyncCluster() + + time.Sleep(time.Second) + + // send 10 commands + for i := 0; i < 10; i++ { + // Test Set + _, err := c.Set("foo", "bar", 0) + if err != nil { + panic(err) + } + } + + time.Sleep(time.Second) + + // kill all + DestroyCluster(etcds) + + time.Sleep(time.Second) + + stop := make(chan bool) + leaderChan := make(chan string, 1) + all := make(chan bool, 1) + + time.Sleep(time.Second) + + for i := 0; i < clusterSize; i++ { + etcds[i], err = os.StartProcess(EtcdBinPath, argGroup[i], procAttr) + } + + go Monitor(clusterSize, 1, leaderChan, all, stop) + + <-all + <-leaderChan + + result, err := c.Set("foo", "bar", 0) + + if err != nil { + t.Fatalf("Recovery error: %s", err) + } + + if result.Index != 18 { + t.Fatalf("recovery failed! [%d/18]", result.Index) + } +} + diff --git a/tests/functional/multi_node_kill_one_test.go b/tests/functional/multi_node_kill_one_test.go new file mode 100644 index 00000000000..725d54b6cfb --- /dev/null +++ b/tests/functional/multi_node_kill_one_test.go @@ -0,0 +1,58 @@ +package test + +import ( + "fmt" + "math/rand" + "os" + "testing" + "time" + + "github.com/coreos/go-etcd/etcd" +) + +// Create a five nodes +// Randomly kill one of the node and keep on sending set command to the cluster +func TestMultiNodeKillOne(t *testing.T) { + procAttr := new(os.ProcAttr) + procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} + + clusterSize := 5 + argGroup, etcds, err := CreateCluster(clusterSize, procAttr, false) + + if err != nil { + t.Fatal("cannot create cluster") + } + + defer DestroyCluster(etcds) + + time.Sleep(2 * time.Second) + + c := etcd.NewClient(nil) + + c.SyncCluster() + + stop := make(chan bool) + // Test Set + go Set(stop) + + for i := 0; i < 10; i++ { + num := rand.Int() % clusterSize + fmt.Println("kill node", num+1) + + // kill + etcds[num].Kill() + etcds[num].Release() + time.Sleep(time.Second) + + // restart + etcds[num], err = os.StartProcess(EtcdBinPath, argGroup[num], procAttr) + if err != nil { + panic(err) + } + time.Sleep(time.Second) + } + fmt.Println("stop") + stop <- true + <-stop +} + diff --git a/tests/functional/remove_node_test.go b/tests/functional/remove_node_test.go new file mode 100644 index 00000000000..09fa747aae8 --- /dev/null +++ b/tests/functional/remove_node_test.go @@ -0,0 +1,113 @@ +package test + +import ( + "net/http" + "os" + "testing" + "time" + + "github.com/coreos/go-etcd/etcd" +) + +// remove the node and node rejoin with previous log +func TestRemoveNode(t *testing.T) { + procAttr := new(os.ProcAttr) + procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} + + clusterSize := 3 + argGroup, etcds, _ := CreateCluster(clusterSize, procAttr, false) + defer DestroyCluster(etcds) + + time.Sleep(time.Second) + + c := etcd.NewClient(nil) + + c.SyncCluster() + + rmReq, _ := http.NewRequest("DELETE", "http://127.0.0.1:7001/remove/node3", nil) + + client := &http.Client{} + for i := 0; i < 2; i++ { + for i := 0; i < 2; i++ { + client.Do(rmReq) + + etcds[2].Wait() + + resp, err := c.Get("_etcd/machines") + + if err != nil { + panic(err) + } + + if len(resp) != 2 { + t.Fatal("cannot remove machine") + } + + if i == 1 { + // rejoin with log + etcds[2], err = os.StartProcess(EtcdBinPath, argGroup[2], procAttr) + } else { + // rejoin without log + etcds[2], err = os.StartProcess(EtcdBinPath, append(argGroup[2], "-f"), procAttr) + } + + if err != nil { + panic(err) + } + + time.Sleep(time.Second) + + resp, err = c.Get("_etcd/machines") + + if err != nil { + panic(err) + } + + if len(resp) != 3 { + t.Fatalf("add machine fails #1 (%d != 3)", len(resp)) + } + } + + // first kill the node, then remove it, then add it back + for i := 0; i < 2; i++ { + etcds[2].Kill() + etcds[2].Wait() + + client.Do(rmReq) + + resp, err := c.Get("_etcd/machines") + + if err != nil { + panic(err) + } + + if len(resp) != 2 { + t.Fatal("cannot remove machine") + } + + if i == 1 { + // rejoin with log + etcds[2], err = os.StartProcess(EtcdBinPath, append(argGroup[2]), procAttr) + } else { + // rejoin without log + etcds[2], err = os.StartProcess(EtcdBinPath, append(argGroup[2], "-f"), procAttr) + } + + if err != nil { + panic(err) + } + + time.Sleep(time.Second) + + resp, err = c.Get("_etcd/machines") + + if err != nil { + panic(err) + } + + if len(resp) != 3 { + t.Fatalf("add machine fails #2 (%d != 3)", len(resp)) + } + } + } +} diff --git a/tests/functional/simple_multi_node_test.go b/tests/functional/simple_multi_node_test.go new file mode 100644 index 00000000000..5d93cbf5c18 --- /dev/null +++ b/tests/functional/simple_multi_node_test.go @@ -0,0 +1,62 @@ +package test + +import ( + "os" + "testing" + "time" + + "github.com/coreos/go-etcd/etcd" +) + +func TestSimpleMultiNode(t *testing.T) { + templateTestSimpleMultiNode(t, false) +} + +func TestSimpleMultiNodeTls(t *testing.T) { + templateTestSimpleMultiNode(t, true) +} + +// Create a three nodes and try to set value +func templateTestSimpleMultiNode(t *testing.T, tls bool) { + procAttr := new(os.ProcAttr) + procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} + + clusterSize := 3 + + _, etcds, err := CreateCluster(clusterSize, procAttr, tls) + + if err != nil { + t.Fatal("cannot create cluster") + } + + defer DestroyCluster(etcds) + + time.Sleep(time.Second) + + c := etcd.NewClient(nil) + + c.SyncCluster() + + // Test Set + result, err := c.Set("foo", "bar", 100) + + if err != nil || result.Key != "/foo" || result.Value != "bar" || result.TTL < 95 { + if err != nil { + t.Fatal(err) + } + + t.Fatalf("Set 1 failed with %s %s %v", result.Key, result.Value, result.TTL) + } + + time.Sleep(time.Second) + + result, err = c.Set("foo", "bar", 100) + + if err != nil || result.Key != "/foo" || result.Value != "bar" || result.PrevValue != "bar" || result.TTL != 100 { + if err != nil { + t.Fatal(err) + } + t.Fatalf("Set 2 failed with %s %s %v", result.Key, result.Value, result.TTL) + } + +} diff --git a/tests/functional/single_node_recovery_test.go b/tests/functional/single_node_recovery_test.go new file mode 100644 index 00000000000..1b0d7f878ef --- /dev/null +++ b/tests/functional/single_node_recovery_test.go @@ -0,0 +1,68 @@ +package test + +import ( + "os" + "testing" + "time" + + "github.com/coreos/go-etcd/etcd" +) + +// This test creates a single node and then set a value to it. +// Then this test kills the node and restart it and tries to get the value again. +func TestSingleNodeRecovery(t *testing.T) { + procAttr := new(os.ProcAttr) + procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} + args := []string{"etcd", "-n=node1", "-d=/tmp/node1"} + + process, err := os.StartProcess(EtcdBinPath, append(args, "-f"), procAttr) + if err != nil { + t.Fatal("start process failed:" + err.Error()) + return + } + + time.Sleep(time.Second) + + c := etcd.NewClient(nil) + + c.SyncCluster() + // Test Set + result, err := c.Set("foo", "bar", 100) + + if err != nil || result.Key != "/foo" || result.Value != "bar" || result.TTL < 95 { + if err != nil { + t.Fatal(err) + } + + t.Fatalf("Set 1 failed with %s %s %v", result.Key, result.Value, result.TTL) + } + + time.Sleep(time.Second) + + process.Kill() + + process, err = os.StartProcess(EtcdBinPath, args, procAttr) + defer process.Kill() + if err != nil { + t.Fatal("start process failed:" + err.Error()) + return + } + + time.Sleep(time.Second) + + results, err := c.Get("foo") + if err != nil { + t.Fatal("get fail: " + err.Error()) + return + } + + result = results[0] + + if err != nil || result.Key != "/foo" || result.Value != "bar" || result.TTL > 99 { + if err != nil { + t.Fatal(err) + } + t.Fatalf("Recovery Get failed with %s %s %v", result.Key, result.Value, result.TTL) + } +} + diff --git a/tests/functional/single_node_test.go b/tests/functional/single_node_test.go new file mode 100644 index 00000000000..04362e328ce --- /dev/null +++ b/tests/functional/single_node_test.go @@ -0,0 +1,77 @@ +package test + +import ( + "os" + "testing" + "time" + + "github.com/coreos/go-etcd/etcd" +) + +// Create a single node and try to set value +func TestSingleNode(t *testing.T) { + procAttr := new(os.ProcAttr) + procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} + args := []string{"etcd", "-n=node1", "-f", "-d=/tmp/node1"} + + process, err := os.StartProcess(EtcdBinPath, args, procAttr) + if err != nil { + t.Fatal("start process failed:" + err.Error()) + return + } + defer process.Kill() + + time.Sleep(time.Second) + + c := etcd.NewClient(nil) + + c.SyncCluster() + // Test Set + result, err := c.Set("foo", "bar", 100) + + if err != nil || result.Key != "/foo" || result.Value != "bar" || result.TTL < 95 { + if err != nil { + t.Fatal("Set 1: ", err) + } + + t.Fatalf("Set 1 failed with %s %s %v", result.Key, result.Value, result.TTL) + } + + time.Sleep(time.Second) + + result, err = c.Set("foo", "bar", 100) + + if err != nil || result.Key != "/foo" || result.Value != "bar" || result.PrevValue != "bar" || result.TTL != 100 { + if err != nil { + t.Fatal("Set 2: ", err) + } + t.Fatalf("Set 2 failed with %s %s %v", result.Key, result.Value, result.TTL) + } + + // Add a test-and-set test + + // First, we'll test we can change the value if we get it write + result, match, err := c.TestAndSet("foo", "bar", "foobar", 100) + + if err != nil || result.Key != "/foo" || result.Value != "foobar" || result.PrevValue != "bar" || result.TTL != 100 || !match { + if err != nil { + t.Fatal(err) + } + t.Fatalf("Set 3 failed with %s %s %v", result.Key, result.Value, result.TTL) + } + + // Next, we'll make sure we can't set it without the correct prior value + _, _, err = c.TestAndSet("foo", "bar", "foofoo", 100) + + if err == nil { + t.Fatalf("Set 4 expecting error when setting key with incorrect previous value") + } + + // Finally, we'll make sure a blank previous value still counts as a test-and-set and still has to match + _, _, err = c.TestAndSet("foo", "", "barbar", 100) + + if err == nil { + t.Fatalf("Set 5 expecting error when setting key with blank (incorrect) previous value") + } +} + diff --git a/tests/functional/util.go b/tests/functional/util.go new file mode 100644 index 00000000000..981ff1be683 --- /dev/null +++ b/tests/functional/util.go @@ -0,0 +1,205 @@ +package test + +import ( + "fmt" + "github.com/coreos/go-etcd/etcd" + "io/ioutil" + "net" + "net/http" + "os" + "strconv" + "time" +) + +var client = http.Client{ + Transport: &http.Transport{ + Dial: dialTimeoutFast, + }, +} + +// Sending set commands +func Set(stop chan bool) { + + stopSet := false + i := 0 + c := etcd.NewClient(nil) + for { + key := fmt.Sprintf("%s_%v", "foo", i) + + result, err := c.Set(key, "bar", 0) + + if err != nil || result.Key != "/"+key || result.Value != "bar" { + select { + case <-stop: + stopSet = true + + default: + } + } + + select { + case <-stop: + stopSet = true + + default: + } + + if stopSet { + break + } + + i++ + } + stop <- true +} + +// Create a cluster of etcd nodes +func CreateCluster(size int, procAttr *os.ProcAttr, ssl bool) ([][]string, []*os.Process, error) { + argGroup := make([][]string, size) + + sslServer1 := []string{"-serverCAFile=./fixtures/ca/ca.crt", + "-serverCert=./fixtures/ca/server.crt", + "-serverKey=./fixtures/ca/server.key.insecure", + } + + sslServer2 := []string{"-serverCAFile=./fixtures/ca/ca.crt", + "-serverCert=./fixtures/ca/server2.crt", + "-serverKey=./fixtures/ca/server2.key.insecure", + } + + for i := 0; i < size; i++ { + if i == 0 { + argGroup[i] = []string{"etcd", "-d=/tmp/node1", "-n=node1"} + if ssl { + argGroup[i] = append(argGroup[i], sslServer1...) + } + } else { + strI := strconv.Itoa(i + 1) + argGroup[i] = []string{"etcd", "-n=node" + strI, "-c=127.0.0.1:400" + strI, "-s=127.0.0.1:700" + strI, "-d=/tmp/node" + strI, "-C=127.0.0.1:7001"} + if ssl { + argGroup[i] = append(argGroup[i], sslServer2...) + } + } + } + + etcds := make([]*os.Process, size) + + for i, _ := range etcds { + var err error + etcds[i], err = os.StartProcess(EtcdBinPath, append(argGroup[i], "-f"), procAttr) + if err != nil { + return nil, nil, err + } + + // TODOBP: Change this sleep to wait until the master is up. + // The problem is that if the master isn't up then the children + // have to retry. This retry can take upwards of 15 seconds + // which slows tests way down and some of them fail. + if i == 0 { + time.Sleep(time.Second * 2) + } + } + + return argGroup, etcds, nil +} + +// Destroy all the nodes in the cluster +func DestroyCluster(etcds []*os.Process) error { + for _, etcd := range etcds { + err := etcd.Kill() + if err != nil { + panic(err.Error()) + } + etcd.Release() + } + return nil +} + +// +func Monitor(size int, allowDeadNum int, leaderChan chan string, all chan bool, stop chan bool) { + leaderMap := make(map[int]string) + baseAddrFormat := "http://0.0.0.0:400%d" + + for { + knownLeader := "unknown" + dead := 0 + var i int + + for i = 0; i < size; i++ { + leader, err := getLeader(fmt.Sprintf(baseAddrFormat, i+1)) + + if err == nil { + leaderMap[i] = leader + + if knownLeader == "unknown" { + knownLeader = leader + } else { + if leader != knownLeader { + break + } + + } + + } else { + dead++ + if dead > allowDeadNum { + break + } + } + + } + + if i == size { + select { + case <-stop: + return + case <-leaderChan: + leaderChan <- knownLeader + default: + leaderChan <- knownLeader + } + + } + if dead == 0 { + select { + case <-all: + all <- true + default: + all <- true + } + } + + time.Sleep(time.Millisecond * 10) + } + +} + +func getLeader(addr string) (string, error) { + + resp, err := client.Get(addr + "/v1/leader") + + if err != nil { + return "", err + } + + if resp.StatusCode != http.StatusOK { + resp.Body.Close() + return "", fmt.Errorf("no leader") + } + + b, err := ioutil.ReadAll(resp.Body) + + resp.Body.Close() + + if err != nil { + return "", err + } + + return string(b), nil + +} + +// Dial with timeout +func dialTimeoutFast(network, addr string) (net.Conn, error) { + return net.DialTimeout(network, addr, time.Millisecond*10) +} From bc8c338beddf27c6450a7b39750bb3f6352c1d47 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Wed, 16 Oct 2013 15:23:11 -0600 Subject: [PATCH 120/247] Travis fix. --- .travis.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.travis.yml b/.travis.yml index 708ec2d505f..50eca08753d 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,5 +1,8 @@ language: go go: 1.1 +install: + - echo "Skip install" + script: - ./test.sh From f998a19c3d6c68697aa91546a85216f42c9031fd Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Wed, 9 Oct 2013 14:26:49 -0700 Subject: [PATCH 121/247] feat(etcd): add version flag print the version and exit, very simple :) Conflicts: etcd.go --- etcd.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/etcd.go b/etcd.go index 05720fb357b..057b0b9b37a 100644 --- a/etcd.go +++ b/etcd.go @@ -31,6 +31,8 @@ var ( force bool + printVersion bool + maxSize int snapshot bool @@ -45,6 +47,8 @@ var ( ) func init() { + flag.BoolVar(&printVersion, "version", false, "print the version and exit") + flag.BoolVar(&log.Verbose, "v", false, "verbose logging") flag.BoolVar(&veryVerbose, "vv", false, "very verbose logging") @@ -116,6 +120,11 @@ type Info struct { func main() { flag.Parse() + if printVersion { + fmt.Println(releaseVersion) + os.Exit(0) + } + if cpuprofile != "" { runCPUProfile() } From 7670c85d70f87d1749d6a49a202c5067a55d2748 Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Wed, 16 Oct 2013 12:38:32 -0700 Subject: [PATCH 122/247] feat(server): export ReleaseVersion releaseVersion is needed in etcd.go to print it out in the -version flag. --- etcd.go | 3 ++- scripts/release-version | 2 +- server/server.go | 2 +- server/version.go | 2 +- 4 files changed, 5 insertions(+), 4 deletions(-) diff --git a/etcd.go b/etcd.go index 057b0b9b37a..a0c6c5f3912 100644 --- a/etcd.go +++ b/etcd.go @@ -2,6 +2,7 @@ package main import ( "flag" + "fmt" "io/ioutil" "os" "strings" @@ -121,7 +122,7 @@ func main() { flag.Parse() if printVersion { - fmt.Println(releaseVersion) + fmt.Println(server.ReleaseVersion) os.Exit(0) } diff --git a/scripts/release-version b/scripts/release-version index c1fdaba0768..77db798378c 100755 --- a/scripts/release-version +++ b/scripts/release-version @@ -4,5 +4,5 @@ VER=$(git describe --tags HEAD) cat < Date: Wed, 16 Oct 2013 23:57:42 -0700 Subject: [PATCH 123/247] fix v1 handler: use create command when test against prevexistence --- server/v1/set_key_handler.go | 20 ++++++++++++++------ store/event.go | 7 +++++-- 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/server/v1/set_key_handler.go b/server/v1/set_key_handler.go index 887c8c22e65..d12d931ed78 100644 --- a/server/v1/set_key_handler.go +++ b/server/v1/set_key_handler.go @@ -30,12 +30,20 @@ func SetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { // If the "prevValue" is specified then test-and-set. Otherwise create a new key. var c raft.Command - if prevValueArr, ok := req.Form["prevValue"]; ok && len(prevValueArr) > 0 { - c = &store.CompareAndSwapCommand{ - Key: key, - Value: value, - PrevValue: prevValueArr[0], - ExpireTime: expireTime, + if prevValueArr, ok := req.Form["prevValue"]; ok { + if len(prevValueArr) > 0 { // test against previous value + c = &store.CompareAndSwapCommand{ + Key: key, + Value: value, + PrevValue: prevValueArr[0], + ExpireTime: expireTime, + } + } else { + c = &store.CreateCommand{ // test against existence + Key: key, + Value: value, + ExpireTime: expireTime, + } } } else { diff --git a/store/event.go b/store/event.go index 14ba1e529d3..92c0b96471a 100644 --- a/store/event.go +++ b/store/event.go @@ -55,13 +55,16 @@ func (event *Event) Response() interface{} { Expiration: event.Expiration, } - if response.Action == Create || response.Action == Set { - response.Action = "set" + if response.Action == Set { if response.PrevValue == "" { response.NewKey = true } } + if response.Action == CompareAndSwap || response.Action == Create { + response.Action = "testAndSet" + } + return response } else { responses := make([]*Response, len(event.KVPairs)) From 4479d72e938ee69c6de78229d3581c6bea06651b Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Thu, 17 Oct 2013 00:12:39 -0700 Subject: [PATCH 124/247] fix hmm... another stupid mistake --- server/v1/set_key_handler.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/v1/set_key_handler.go b/server/v1/set_key_handler.go index d12d931ed78..3ce179b699c 100644 --- a/server/v1/set_key_handler.go +++ b/server/v1/set_key_handler.go @@ -1,6 +1,7 @@ package v1 import ( + "fmt" "net/http" etcdErr "github.com/coreos/etcd/error" @@ -31,7 +32,7 @@ func SetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { // If the "prevValue" is specified then test-and-set. Otherwise create a new key. var c raft.Command if prevValueArr, ok := req.Form["prevValue"]; ok { - if len(prevValueArr) > 0 { // test against previous value + if len(prevValueArr[0]) > 0 { // test against previous value c = &store.CompareAndSwapCommand{ Key: key, Value: value, From 4fb3a01f25480030d3dc413993f41fc8dbd5c394 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Thu, 17 Oct 2013 10:02:59 -0600 Subject: [PATCH 125/247] Update build script to fail on error. --- build | 1 + test.sh | 1 + 2 files changed, 2 insertions(+) diff --git a/build b/build index 9c2f95f6ae5..8e76b4c86ea 100755 --- a/build +++ b/build @@ -1,4 +1,5 @@ #!/bin/sh +set -e ETCD_PACKAGE=github.com/coreos/etcd export GOPATH="${PWD}" diff --git a/test.sh b/test.sh index 9695e70c2c2..31196f7eccc 100755 --- a/test.sh +++ b/test.sh @@ -1,4 +1,5 @@ #!/bin/sh +set -e # Get GOPATH, etc from build . ./build From 87b2a0d9c5e2b95d2cf2d409338aeb1f91953ccf Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Thu, 17 Oct 2013 10:50:00 -0700 Subject: [PATCH 126/247] fix remove unused package --- server/v1/set_key_handler.go | 1 - 1 file changed, 1 deletion(-) diff --git a/server/v1/set_key_handler.go b/server/v1/set_key_handler.go index 3ce179b699c..5e29fafe6bb 100644 --- a/server/v1/set_key_handler.go +++ b/server/v1/set_key_handler.go @@ -1,7 +1,6 @@ package v1 import ( - "fmt" "net/http" etcdErr "github.com/coreos/etcd/error" From dcef04b796d34e81c55dbe6318c292c463d7e8cb Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Thu, 17 Oct 2013 18:11:11 -0600 Subject: [PATCH 127/247] Add v2 unit tests. --- etcd.go | 6 +- server/peer_server.go | 67 ++- server/server.go | 62 ++- server/v2/put_handler.go | 23 +- server/v2/tests/get_handler_test.go | 152 ++++++ server/v2/tests/post_handler_test.go | 38 ++ server/v2/tests/put_handler_test.go | 280 +++++++++++ test.sh | 12 +- tests/functional/internal_version_test.go | 3 +- tests/functional/util.go | 12 +- tests/http_utils.go | 67 +++ tests/mock/mock_store.go | 66 +++ tests/mock/server_v2.go | 55 +++ tests/server_utils.go | 53 ++ third_party/deps | 2 + .../coreos/go-etcd/etcd/client_test.go | 8 +- .../examples/sync-cluster/sync-cluster.go | 1 + .../github.com/coreos/go-log/log/commands.go | 28 +- .../github.com/coreos/go-log/log/fields.go | 1 - .../github.com/coreos/go-log/log/logger.go | 1 - .../github.com/coreos/go-log/log/priority.go | 1 - .../github.com/coreos/go-log/log/sinks.go | 1 - .../coreos/go-systemd/activation/files.go | 2 +- .../stretchr/testify/assert/assertions.go | 465 ++++++++++++++++++ .../testify/assert/assertions_test.go | 383 +++++++++++++++ .../github.com/stretchr/testify/assert/doc.go | 74 +++ .../stretchr/testify/assert/errors.go | 10 + .../github.com/stretchr/testify/mock/doc.go | 43 ++ .../github.com/stretchr/testify/mock/mock.go | 465 ++++++++++++++++++ .../stretchr/testify/mock/mock_test.go | 418 ++++++++++++++++ 30 files changed, 2745 insertions(+), 54 deletions(-) create mode 100644 server/v2/tests/get_handler_test.go create mode 100644 server/v2/tests/post_handler_test.go create mode 100644 server/v2/tests/put_handler_test.go create mode 100644 tests/http_utils.go create mode 100644 tests/mock/mock_store.go create mode 100644 tests/mock/server_v2.go create mode 100644 tests/server_utils.go create mode 100644 third_party/github.com/stretchr/testify/assert/assertions.go create mode 100644 third_party/github.com/stretchr/testify/assert/assertions_test.go create mode 100644 third_party/github.com/stretchr/testify/assert/doc.go create mode 100644 third_party/github.com/stretchr/testify/assert/errors.go create mode 100644 third_party/github.com/stretchr/testify/mock/doc.go create mode 100644 third_party/github.com/stretchr/testify/mock/mock.go create mode 100644 third_party/github.com/stretchr/testify/mock/mock_test.go diff --git a/etcd.go b/etcd.go index a0c6c5f3912..f9df25c6dc6 100644 --- a/etcd.go +++ b/etcd.go @@ -194,6 +194,8 @@ func main() { ps.SetServer(s) - ps.ListenAndServe(snapshot, cluster) - s.ListenAndServe() + go func() { + log.Fatal(ps.ListenAndServe(snapshot, cluster)) + }() + log.Fatal(s.ListenAndServe()) } diff --git a/server/peer_server.go b/server/peer_server.go index d85ed9a2194..058ddbf75e4 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -7,6 +7,7 @@ import ( "encoding/json" "fmt" "io/ioutil" + "net" "net/http" "net/url" "strings" @@ -21,6 +22,8 @@ import ( type PeerServer struct { raftServer raft.Server server *Server + httpServer *http.Server + listener net.Listener joinIndex uint64 name string url string @@ -89,7 +92,7 @@ func NewPeerServer(name string, path string, url string, listenHost string, tlsC } // Start the raft server -func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) { +func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) error { // LoadSnapshot if snapshot { err := s.raftServer.LoadSnapshot() @@ -138,8 +141,60 @@ func (s *PeerServer) ListenAndServe(snapshot bool, cluster []string) { } // start to response to raft requests - go s.startTransport(s.tlsConf.Scheme, s.tlsConf.Server) + return s.startTransport(s.tlsConf.Scheme, s.tlsConf.Server) +} + +// Overridden version of net/http added so we can manage the listener. +func (s *PeerServer) listenAndServe() error { + addr := s.httpServer.Addr + if addr == "" { + addr = ":http" + } + l, e := net.Listen("tcp", addr) + if e != nil { + return e + } + s.listener = l + return s.httpServer.Serve(l) +} + +// Overridden version of net/http added so we can manage the listener. +func (s *PeerServer) listenAndServeTLS(certFile, keyFile string) error { + addr := s.httpServer.Addr + if addr == "" { + addr = ":https" + } + config := &tls.Config{} + if s.httpServer.TLSConfig != nil { + *config = *s.httpServer.TLSConfig + } + if config.NextProtos == nil { + config.NextProtos = []string{"http/1.1"} + } + var err error + config.Certificates = make([]tls.Certificate, 1) + config.Certificates[0], err = tls.LoadX509KeyPair(certFile, keyFile) + if err != nil { + return err + } + + conn, err := net.Listen("tcp", addr) + if err != nil { + return err + } + + tlsListener := tls.NewListener(conn, config) + s.listener = tlsListener + return s.httpServer.Serve(tlsListener) +} + +// Stops the server. +func (s *PeerServer) Close() { + if s.listener != nil { + s.listener.Close() + s.listener = nil + } } // Retrieves the underlying Raft server. @@ -178,12 +233,12 @@ func (s *PeerServer) startAsFollower(cluster []string) { } // Start to listen and response raft command -func (s *PeerServer) startTransport(scheme string, tlsConf tls.Config) { +func (s *PeerServer) startTransport(scheme string, tlsConf tls.Config) error { log.Infof("raft server [name %s, listen on %s, advertised url %s]", s.name, s.listenHost, s.url) raftMux := http.NewServeMux() - server := &http.Server{ + s.httpServer = &http.Server{ Handler: raftMux, TLSConfig: &tlsConf, Addr: s.listenHost, @@ -202,9 +257,9 @@ func (s *PeerServer) startTransport(scheme string, tlsConf tls.Config) { raftMux.HandleFunc("/etcdURL", s.EtcdURLHttpHandler) if scheme == "http" { - log.Fatal(server.ListenAndServe()) + return s.listenAndServe() } else { - log.Fatal(server.ListenAndServeTLS(s.tlsInfo.CertFile, s.tlsInfo.KeyFile)) + return s.listenAndServeTLS(s.tlsInfo.CertFile, s.tlsInfo.KeyFile) } } diff --git a/server/server.go b/server/server.go index 16844d492d7..1a3e2a8d2c4 100644 --- a/server/server.go +++ b/server/server.go @@ -1,7 +1,9 @@ package server import ( + "crypto/tls" "fmt" + "net" "net/http" "net/url" "strings" @@ -21,6 +23,7 @@ type Server struct { http.Server peerServer *PeerServer registry *Registry + listener net.Listener store store.Store name string url string @@ -156,13 +159,66 @@ func (s *Server) handleFunc(path string, f func(http.ResponseWriter, *http.Reque } // Start to listen and response etcd client command -func (s *Server) ListenAndServe() { +func (s *Server) ListenAndServe() error { log.Infof("etcd server [name %s, listen on %s, advertised url %s]", s.name, s.Server.Addr, s.url) if s.tlsConf.Scheme == "http" { - log.Fatal(s.Server.ListenAndServe()) + return s.listenAndServe() } else { - log.Fatal(s.Server.ListenAndServeTLS(s.tlsInfo.CertFile, s.tlsInfo.KeyFile)) + return s.listenAndServeTLS(s.tlsInfo.CertFile, s.tlsInfo.KeyFile) + } +} + +// Overridden version of net/http added so we can manage the listener. +func (s *Server) listenAndServe() error { + addr := s.Server.Addr + if addr == "" { + addr = ":http" + } + l, e := net.Listen("tcp", addr) + if e != nil { + return e + } + s.listener = l + return s.Server.Serve(l) +} + +// Overridden version of net/http added so we can manage the listener. +func (s *Server) listenAndServeTLS(certFile, keyFile string) error { + addr := s.Server.Addr + if addr == "" { + addr = ":https" + } + config := &tls.Config{} + if s.Server.TLSConfig != nil { + *config = *s.Server.TLSConfig + } + if config.NextProtos == nil { + config.NextProtos = []string{"http/1.1"} + } + + var err error + config.Certificates = make([]tls.Certificate, 1) + config.Certificates[0], err = tls.LoadX509KeyPair(certFile, keyFile) + if err != nil { + return err + } + + conn, err := net.Listen("tcp", addr) + if err != nil { + return err + } + + tlsListener := tls.NewListener(conn, config) + s.listener = tlsListener + return s.Server.Serve(tlsListener) +} + +// Stops the server. +func (s *Server) Close() { + if s.listener != nil { + s.listener.Close() + s.listener = nil } } diff --git a/server/v2/put_handler.go b/server/v2/put_handler.go index 336366d38eb..536bbe21698 100644 --- a/server/v2/put_handler.go +++ b/server/v2/put_handler.go @@ -12,6 +12,8 @@ import ( ) func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { + var c raft.Command + vars := mux.Vars(req) key := "/" + vars["key"] @@ -23,11 +25,14 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", store.UndefIndex, store.UndefTerm) } - prevValue, valueOk := req.Form["prevValue"] - prevIndexStr, indexOk := req.Form["prevIndex"] - prevExist, existOk := req.Form["prevExist"] + _, valueOk := req.Form["prevValue"] + prevValue := req.Form.Get("prevValue") - var c raft.Command + _, indexOk := req.Form["prevIndex"] + prevIndexStr := req.Form.Get("prevIndex") + + _, existOk := req.Form["prevExist"] + prevExist := req.Form.Get("prevExist") // Set handler: create a new node or replace the old one. if !valueOk && !indexOk && !existOk { @@ -36,13 +41,13 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { // update with test if existOk { - if prevExist[0] == "false" { + if prevExist == "false" { // Create command: create a new node. Fail, if a node already exists // Ignore prevIndex and prevValue return CreateHandler(w, req, s, key, value, expireTime) } - if prevExist[0] == "true" && !indexOk && !valueOk { + if prevExist == "true" && !indexOk && !valueOk { return UpdateHandler(w, req, s, key, value, expireTime) } } @@ -50,7 +55,7 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { var prevIndex uint64 if indexOk { - prevIndex, err = strconv.ParseUint(prevIndexStr[0], 10, 64) + prevIndex, err = strconv.ParseUint(prevIndexStr, 10, 64) // bad previous index if err != nil { @@ -61,7 +66,7 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { } if valueOk { - if prevValue[0] == "" { + if prevValue == "" { return etcdErr.NewError(etcdErr.EcodePrevValueRequired, "CompareAndSwap", store.UndefIndex, store.UndefTerm) } } @@ -69,7 +74,7 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { c = &store.CompareAndSwapCommand{ Key: key, Value: value, - PrevValue: prevValue[0], + PrevValue: prevValue, PrevIndex: prevIndex, } diff --git a/server/v2/tests/get_handler_test.go b/server/v2/tests/get_handler_test.go new file mode 100644 index 00000000000..a248d33f423 --- /dev/null +++ b/server/v2/tests/get_handler_test.go @@ -0,0 +1,152 @@ +package v2 + +import ( + "fmt" + "net/url" + "testing" + "time" + + "github.com/coreos/etcd/server" + "github.com/coreos/etcd/tests" + "github.com/stretchr/testify/assert" +) + +// Ensures that a value can be retrieve for a given key. +// +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX +// $ curl localhost:4001/v2/keys/foo/bar +// +func TestV2GetKey(t *testing.T) { + tests.RunServer(func(s *server.Server) { + v := url.Values{} + v.Set("value", "XXX") + resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + tests.ReadBody(resp) + resp, _ = tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar")) + body := tests.ReadBodyJSON(resp) + assert.Equal(t, body["action"], "get", "") + assert.Equal(t, body["key"], "/foo/bar", "") + assert.Equal(t, body["value"], "XXX", "") + assert.Equal(t, body["index"], 3, "") + assert.Equal(t, body["term"], 0, "") + }) +} + +// Ensures that a directory of values can be recursively retrieved for a given key. +// +// $ curl -X PUT localhost:4001/v2/keys/foo/x -d value=XXX +// $ curl -X PUT localhost:4001/v2/keys/foo/y/z -d value=YYY +// $ curl localhost:4001/v2/keys/foo -d recursive=true +// +func TestV2GetKeyRecursively(t *testing.T) { + tests.RunServer(func(s *server.Server) { + v := url.Values{} + v.Set("value", "XXX") + resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/x"), v) + tests.ReadBody(resp) + + v.Set("value", "YYY") + resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/y/z"), v) + tests.ReadBody(resp) + + resp, _ = tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo?recursive=true")) + body := tests.ReadBodyJSON(resp) + assert.Equal(t, body["action"], "get", "") + assert.Equal(t, body["key"], "/foo", "") + assert.Equal(t, body["dir"], true, "") + assert.Equal(t, body["index"], 4, "") + assert.Equal(t, len(body["kvs"].([]interface{})), 2, "") + + kv0 := body["kvs"].([]interface{})[0].(map[string]interface{}) + assert.Equal(t, kv0["key"], "/foo/x", "") + assert.Equal(t, kv0["value"], "XXX", "") + + kv1 := body["kvs"].([]interface{})[1].(map[string]interface{}) + assert.Equal(t, kv1["key"], "/foo/y", "") + assert.Equal(t, kv1["dir"], true, "") + + kvs2 := kv1["kvs"].([]interface{})[0].(map[string]interface{}) + assert.Equal(t, kvs2["key"], "/foo/y/z", "") + assert.Equal(t, kvs2["value"], "YYY", "") + }) +} + +// Ensures that a watcher can wait for a value to be set and return it to the client. +// +// $ curl localhost:4001/v2/keys/foo/bar?wait=true +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX +// +func TestV2WatchKey(t *testing.T) { + tests.RunServer(func(s *server.Server) { + var body map[string]interface{} + go func() { + resp, _ := tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar?wait=true")) + body = tests.ReadBodyJSON(resp) + }() + + // Make sure response didn't fire early. + time.Sleep(1 * time.Millisecond) + assert.Nil(t, body, "") + + // Set a value. + v := url.Values{} + v.Set("value", "XXX") + resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + tests.ReadBody(resp) + + // A response should follow from the GET above. + time.Sleep(1 * time.Millisecond) + assert.NotNil(t, body, "") + assert.Equal(t, body["action"], "set", "") + assert.Equal(t, body["key"], "/foo/bar", "") + assert.Equal(t, body["value"], "XXX", "") + assert.Equal(t, body["index"], 3, "") + assert.Equal(t, body["term"], 0, "") + }) +} + + +// Ensures that a watcher can wait for a value to be set after a given index. +// +// $ curl localhost:4001/v2/keys/foo/bar?wait=true&waitIndex=4 +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY +// +func TestV2WatchKeyWithIndex(t *testing.T) { + tests.RunServer(func(s *server.Server) { + var body map[string]interface{} + go func() { + resp, _ := tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar?wait=true&waitIndex=5")) + body = tests.ReadBodyJSON(resp) + }() + + // Make sure response didn't fire early. + time.Sleep(1 * time.Millisecond) + assert.Nil(t, body, "") + + // Set a value (before given index). + v := url.Values{} + v.Set("value", "XXX") + resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + tests.ReadBody(resp) + + // Make sure response didn't fire early. + time.Sleep(1 * time.Millisecond) + assert.Nil(t, body, "") + + // Set a value (before given index). + v.Set("value", "YYY") + resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + tests.ReadBody(resp) + + // A response should follow from the GET above. + time.Sleep(1 * time.Millisecond) + assert.NotNil(t, body, "") + assert.Equal(t, body["action"], "set", "") + assert.Equal(t, body["key"], "/foo/bar", "") + assert.Equal(t, body["value"], "YYY", "") + assert.Equal(t, body["index"], 4, "") + assert.Equal(t, body["term"], 0, "") + }) +} + diff --git a/server/v2/tests/post_handler_test.go b/server/v2/tests/post_handler_test.go new file mode 100644 index 00000000000..8e7e23a8ce0 --- /dev/null +++ b/server/v2/tests/post_handler_test.go @@ -0,0 +1,38 @@ +package v2 + +import ( + "fmt" + "testing" + + "github.com/coreos/etcd/server" + "github.com/coreos/etcd/tests" + "github.com/stretchr/testify/assert" +) + +// Ensures a unique value is added to the key's children. +// +// $ curl -X POST localhost:4001/v2/keys/foo/bar +// $ curl -X POST localhost:4001/v2/keys/foo/bar +// $ curl -X POST localhost:4001/v2/keys/foo/baz +// +func TestV2CreateUnique(t *testing.T) { + tests.RunServer(func(s *server.Server) { + // POST should add index to list. + resp, _ := tests.PostForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), nil) + body := tests.ReadBodyJSON(resp) + assert.Equal(t, body["action"], "create", "") + assert.Equal(t, body["key"], "/foo/bar/3", "") + assert.Equal(t, body["dir"], true, "") + assert.Equal(t, body["index"], 3, "") + + // Second POST should add next index to list. + resp, _ = tests.PostForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), nil) + body = tests.ReadBodyJSON(resp) + assert.Equal(t, body["key"], "/foo/bar/4", "") + + // POST to a different key should add index to that list. + resp, _ = tests.PostForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/baz"), nil) + body = tests.ReadBodyJSON(resp) + assert.Equal(t, body["key"], "/foo/baz/5", "") + }) +} diff --git a/server/v2/tests/put_handler_test.go b/server/v2/tests/put_handler_test.go new file mode 100644 index 00000000000..1341d03b302 --- /dev/null +++ b/server/v2/tests/put_handler_test.go @@ -0,0 +1,280 @@ +package v2 + +import ( + "fmt" + "net/url" + "testing" + "time" + + "github.com/coreos/etcd/server" + "github.com/coreos/etcd/tests" + "github.com/stretchr/testify/assert" +) + +// Ensures that a key is set to a given value. +// +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX +// +func TestV2SetKey(t *testing.T) { + tests.RunServer(func(s *server.Server) { + v := url.Values{} + v.Set("value", "XXX") + resp, err := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + body := tests.ReadBody(resp) + assert.Nil(t, err, "") + assert.Equal(t, string(body), `{"action":"set","key":"/foo/bar","value":"XXX","index":3,"term":0}`, "") + }) +} + +// Ensures that a time-to-live is added to a key. +// +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX -d ttl=20 +// +func TestV2SetKeyWithTTL(t *testing.T) { + tests.RunServer(func(s *server.Server) { + t0 := time.Now() + v := url.Values{} + v.Set("value", "XXX") + v.Set("ttl", "20") + resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + body := tests.ReadBodyJSON(resp) + assert.Equal(t, body["ttl"], 20, "") + + // Make sure the expiration date is correct. + expiration, _ := time.Parse(time.RFC3339Nano, body["expiration"].(string)) + assert.Equal(t, expiration.Sub(t0) / time.Second, 20, "") + }) +} + +// Ensures that an invalid time-to-live is returned as an error. +// +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX -d ttl=bad_ttl +// +func TestV2SetKeyWithBadTTL(t *testing.T) { + tests.RunServer(func(s *server.Server) { + v := url.Values{} + v.Set("value", "XXX") + v.Set("ttl", "bad_ttl") + resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + body := tests.ReadBodyJSON(resp) + assert.Equal(t, body["errorCode"], 202, "") + assert.Equal(t, body["message"], "The given TTL in POST form is not a number", "") + assert.Equal(t, body["cause"], "Update", "") + }) +} + +// Ensures that a key is conditionally set only if it previously did not exist. +// +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX -d prevExist=false +// +func TestV2CreateKeySuccess(t *testing.T) { + tests.RunServer(func(s *server.Server) { + v := url.Values{} + v.Set("value", "XXX") + v.Set("prevExist", "false") + resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + body := tests.ReadBodyJSON(resp) + assert.Equal(t, body["value"], "XXX", "") + }) +} + +// Ensures that a key is not conditionally because it previously existed. +// +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX -d prevExist=false +// +func TestV2CreateKeyFail(t *testing.T) { + tests.RunServer(func(s *server.Server) { + v := url.Values{} + v.Set("value", "XXX") + v.Set("prevExist", "false") + resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + tests.ReadBody(resp) + resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + body := tests.ReadBodyJSON(resp) + assert.Equal(t, body["errorCode"], 105, "") + assert.Equal(t, body["message"], "Already exists", "") + assert.Equal(t, body["cause"], "/foo/bar", "") + }) +} + +// Ensures that a key is conditionally set only if it previously did exist. +// +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY -d prevExist=true +// +func TestV2UpdateKeySuccess(t *testing.T) { + tests.RunServer(func(s *server.Server) { + v := url.Values{} + + v.Set("value", "XXX") + resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + tests.ReadBody(resp) + + v.Set("value", "YYY") + v.Set("prevExist", "true") + resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + body := tests.ReadBodyJSON(resp) + assert.Equal(t, body["action"], "update", "") + assert.Equal(t, body["prevValue"], "XXX", "") + }) +} + +// Ensures that a key is not conditionally set if it previously did not exist. +// +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX -d prevExist=true +// +func TestV2UpdateKeyFailOnValue(t *testing.T) { + tests.RunServer(func(s *server.Server) { + v := url.Values{} + resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo"), v) + + v.Set("value", "YYY") + v.Set("prevExist", "true") + resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + body := tests.ReadBodyJSON(resp) + assert.Equal(t, body["errorCode"], 100, "") + assert.Equal(t, body["message"], "Key Not Found", "") + assert.Equal(t, body["cause"], "/foo/bar", "") + }) +} + +// Ensures that a key is not conditionally set if it previously did not exist. +// +// $ curl -X PUT localhost:4001/v2/keys/foo -d value=XXX -d prevExist=true +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX -d prevExist=true +// +func TestV2UpdateKeyFailOnMissingDirectory(t *testing.T) { + tests.RunServer(func(s *server.Server) { + v := url.Values{} + v.Set("value", "YYY") + v.Set("prevExist", "true") + resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + body := tests.ReadBodyJSON(resp) + assert.Equal(t, body["errorCode"], 100, "") + assert.Equal(t, body["message"], "Key Not Found", "") + assert.Equal(t, body["cause"], "/foo", "") + }) +} + +// Ensures that a key is set only if the previous index matches. +// +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY -d prevIndex=3 +// +func TestV2SetKeyCASOnIndexSuccess(t *testing.T) { + tests.RunServer(func(s *server.Server) { + v := url.Values{} + v.Set("value", "XXX") + resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + tests.ReadBody(resp) + v.Set("value", "YYY") + v.Set("prevIndex", "3") + resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + body := tests.ReadBodyJSON(resp) + assert.Equal(t, body["action"], "compareAndSwap", "") + assert.Equal(t, body["prevValue"], "XXX", "") + assert.Equal(t, body["value"], "YYY", "") + assert.Equal(t, body["index"], 4, "") + }) +} + +// Ensures that a key is not set if the previous index does not match. +// +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY -d prevIndex=10 +// +func TestV2SetKeyCASOnIndexFail(t *testing.T) { + tests.RunServer(func(s *server.Server) { + v := url.Values{} + v.Set("value", "XXX") + resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + tests.ReadBody(resp) + v.Set("value", "YYY") + v.Set("prevIndex", "10") + resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + body := tests.ReadBodyJSON(resp) + assert.Equal(t, body["errorCode"], 101, "") + assert.Equal(t, body["message"], "Test Failed", "") + assert.Equal(t, body["cause"], "[ != XXX] [10 != 3]", "") + assert.Equal(t, body["index"], 4, "") + }) +} + +// Ensures that an error is thrown if an invalid previous index is provided. +// +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY -d prevIndex=bad_index +// +func TestV2SetKeyCASWithInvalidIndex(t *testing.T) { + tests.RunServer(func(s *server.Server) { + v := url.Values{} + v.Set("value", "YYY") + v.Set("prevIndex", "bad_index") + resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + body := tests.ReadBodyJSON(resp) + assert.Equal(t, body["errorCode"], 203, "") + assert.Equal(t, body["message"], "The given index in POST form is not a number", "") + assert.Equal(t, body["cause"], "CompareAndSwap", "") + }) +} + +// Ensures that a key is set only if the previous value matches. +// +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY -d prevValue=XXX +// +func TestV2SetKeyCASOnValueSuccess(t *testing.T) { + tests.RunServer(func(s *server.Server) { + v := url.Values{} + v.Set("value", "XXX") + resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + tests.ReadBody(resp) + v.Set("value", "YYY") + v.Set("prevValue", "XXX") + resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + body := tests.ReadBodyJSON(resp) + assert.Equal(t, body["action"], "compareAndSwap", "") + assert.Equal(t, body["prevValue"], "XXX", "") + assert.Equal(t, body["value"], "YYY", "") + assert.Equal(t, body["index"], 4, "") + }) +} + +// Ensures that a key is not set if the previous value does not match. +// +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY -d prevValue=AAA +// +func TestV2SetKeyCASOnValueFail(t *testing.T) { + tests.RunServer(func(s *server.Server) { + v := url.Values{} + v.Set("value", "XXX") + resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + tests.ReadBody(resp) + v.Set("value", "YYY") + v.Set("prevValue", "AAA") + resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + body := tests.ReadBodyJSON(resp) + assert.Equal(t, body["errorCode"], 101, "") + assert.Equal(t, body["message"], "Test Failed", "") + assert.Equal(t, body["cause"], "[AAA != XXX] [0 != 3]", "") + assert.Equal(t, body["index"], 4, "") + }) +} + +// Ensures that an error is returned if a blank prevValue is set. +// +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX -d prevValue= +// +func TestV2SetKeyCASWithMissingValueFails(t *testing.T) { + tests.RunServer(func(s *server.Server) { + v := url.Values{} + v.Set("value", "XXX") + v.Set("prevValue", "") + resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + body := tests.ReadBodyJSON(resp) + assert.Equal(t, body["errorCode"], 201, "") + assert.Equal(t, body["message"], "PrevValue is Required in POST form", "") + assert.Equal(t, body["cause"], "CompareAndSwap", "") + }) +} diff --git a/test.sh b/test.sh index 31196f7eccc..a99e3f1789e 100755 --- a/test.sh +++ b/test.sh @@ -1,11 +1,15 @@ #!/bin/sh set -e -# Get GOPATH, etc from build -. ./build - # Unit tests +echo "-- UNIT TESTS --" +go test -v ./server/v2/tests go test -v ./store +# Get GOPATH, etc from build +echo "-- BUILDING BINARY --" +. ./build + # Functional tests -ETCD_BIN_PATH=$(pwd)/etcd go test -v ./tests/functional +echo "-- FUNCTIONAL TESTS --" +ETCD_BIN_PATH=$(PWD)/etcd go test -v ./tests/functional diff --git a/tests/functional/internal_version_test.go b/tests/functional/internal_version_test.go index 3413928c15c..37ac0d532ad 100644 --- a/tests/functional/internal_version_test.go +++ b/tests/functional/internal_version_test.go @@ -38,12 +38,11 @@ func TestInternalVersion(t *testing.T) { t.Fatal("start process failed:" + err.Error()) return } - defer process.Kill() time.Sleep(time.Second) + process.Kill() _, err = http.Get("http://127.0.0.1:4001") - if err == nil { t.Fatal("etcd node should not be up") return diff --git a/tests/functional/util.go b/tests/functional/util.go index 981ff1be683..3ef0028cdde 100644 --- a/tests/functional/util.go +++ b/tests/functional/util.go @@ -57,14 +57,14 @@ func Set(stop chan bool) { func CreateCluster(size int, procAttr *os.ProcAttr, ssl bool) ([][]string, []*os.Process, error) { argGroup := make([][]string, size) - sslServer1 := []string{"-serverCAFile=./fixtures/ca/ca.crt", - "-serverCert=./fixtures/ca/server.crt", - "-serverKey=./fixtures/ca/server.key.insecure", + sslServer1 := []string{"-serverCAFile=../../fixtures/ca/ca.crt", + "-serverCert=../../fixtures/ca/server.crt", + "-serverKey=../../fixtures/ca/server.key.insecure", } - sslServer2 := []string{"-serverCAFile=./fixtures/ca/ca.crt", - "-serverCert=./fixtures/ca/server2.crt", - "-serverKey=./fixtures/ca/server2.key.insecure", + sslServer2 := []string{"-serverCAFile=../../fixtures/ca/ca.crt", + "-serverCert=../../fixtures/ca/server2.crt", + "-serverKey=../../fixtures/ca/server2.key.insecure", } for i := 0; i < size; i++ { diff --git a/tests/http_utils.go b/tests/http_utils.go new file mode 100644 index 00000000000..b1395373d03 --- /dev/null +++ b/tests/http_utils.go @@ -0,0 +1,67 @@ +package tests + +import ( + "encoding/json" + "fmt" + "io" + "io/ioutil" + "net/http" + "net/url" + "strings" +) + +// Creates a new HTTP client with KeepAlive disabled. +func NewHTTPClient() *http.Client { + return &http.Client{Transport: &http.Transport{DisableKeepAlives: true}} +} + +// Reads the body from the response and closes it. +func ReadBody(resp *http.Response) []byte { + if resp == nil { + return []byte{} + } + body, _ := ioutil.ReadAll(resp.Body) + resp.Body.Close() + return body +} + +// Reads the body from the response and parses it as JSON. +func ReadBodyJSON(resp *http.Response) map[string]interface{} { + m := make(map[string]interface{}) + b := ReadBody(resp) + if err := json.Unmarshal(b, &m); err != nil { + panic(fmt.Sprintf("HTTP body JSON parse error: %v", err)) + } + return m +} + +func Get(url string) (*http.Response, error) { + return send("GET", url, "application/json", nil) +} + +func Post(url string, bodyType string, body io.Reader) (*http.Response, error) { + return send("POST", url, bodyType, body) +} + +func PostForm(url string, data url.Values) (*http.Response, error) { + return Post(url, "application/x-www-form-urlencoded", strings.NewReader(data.Encode())) +} + +func Put(url string, bodyType string, body io.Reader) (*http.Response, error) { + return send("PUT", url, bodyType, body) +} + +func PutForm(url string, data url.Values) (*http.Response, error) { + return Put(url, "application/x-www-form-urlencoded", strings.NewReader(data.Encode())) +} + + +func send(method string, url string, bodyType string, body io.Reader) (*http.Response, error) { + c := NewHTTPClient() + req, err := http.NewRequest(method, url, body) + if err != nil { + return nil, err + } + req.Header.Set("Content-Type", bodyType) + return c.Do(req) +} diff --git a/tests/mock/mock_store.go b/tests/mock/mock_store.go new file mode 100644 index 00000000000..a4b66236873 --- /dev/null +++ b/tests/mock/mock_store.go @@ -0,0 +1,66 @@ +package mock + +import ( + "github.com/coreos/etcd/store" + "github.com/stretchr/testify/mock" + "time" +) + +// A mock Store object used for testing. +type Store struct { + mock.Mock +} + +func NewStore() *Store { + return &Store{} +} + +func (s *Store) Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*store.Event, error) { + args := s.Called(nodePath, recursive, sorted, index, term) + return args.Get(0).(*store.Event), args.Error(1) +} + +func (s *Store) Set(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*store.Event, error) { + args := s.Called(nodePath, value, expireTime, index, term) + return args.Get(0).(*store.Event), args.Error(1) +} + +func (s *Store) Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*store.Event, error) { + args := s.Called(nodePath, newValue, expireTime, index, term) + return args.Get(0).(*store.Event), args.Error(1) +} + +func (s *Store) Create(nodePath string, value string, incrementalSuffix bool, expireTime time.Time, index uint64, term uint64) (*store.Event, error) { + args := s.Called(nodePath, value, incrementalSuffix, expireTime, index, term) + return args.Get(0).(*store.Event), args.Error(1) +} + +func (s *Store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint64, value string, expireTime time.Time, index uint64, term uint64) (*store.Event, error) { + args := s.Called(nodePath, prevValue, prevIndex, value, expireTime, index, term) + return args.Get(0).(*store.Event), args.Error(1) +} + +func (s *Store) Delete(nodePath string, recursive bool, index uint64, term uint64) (*store.Event, error) { + args := s.Called(nodePath, recursive, index, term) + return args.Get(0).(*store.Event), args.Error(1) +} + +func (s *Store) Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *store.Event, error) { + args := s.Called(prefix, recursive, sinceIndex, index, term) + return args.Get(0).(<-chan *store.Event), args.Error(1) +} + +func (s *Store) Save() ([]byte, error) { + args := s.Called() + return args.Get(0).([]byte), args.Error(1) +} + +func (s *Store) Recovery(b []byte) error { + args := s.Called(b) + return args.Error(1) +} + +func (s *Store) JsonStats() []byte { + args := s.Called() + return args.Get(0).([]byte) +} diff --git a/tests/mock/server_v2.go b/tests/mock/server_v2.go new file mode 100644 index 00000000000..2aa28185b5a --- /dev/null +++ b/tests/mock/server_v2.go @@ -0,0 +1,55 @@ +package mock + +import ( + "net/http" + + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" + "github.com/stretchr/testify/mock" +) + +// A mock Server for the v2 handlers. +type ServerV2 struct { + mock.Mock + store store.Store +} + +func NewServerV2(store store.Store) *ServerV2 { + return &ServerV2{ + store: store, + } +} + +func (s *ServerV2) State() string { + args := s.Called() + return args.String(0) +} + +func (s *ServerV2) Leader() string { + args := s.Called() + return args.String(0) +} + +func (s *ServerV2) CommitIndex() uint64 { + args := s.Called() + return args.Get(0).(uint64) +} + +func (s *ServerV2) Term() uint64 { + args := s.Called() + return args.Get(0).(uint64) +} + +func (s *ServerV2) PeerURL(name string) (string, bool) { + args := s.Called(name) + return args.String(0), args.Bool(1) +} + +func (s *ServerV2) Store() store.Store { + return s.store +} + +func (s *ServerV2) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Request) error { + args := s.Called(c, w, req) + return args.Error(0) +} diff --git a/tests/server_utils.go b/tests/server_utils.go new file mode 100644 index 00000000000..adddf336bcf --- /dev/null +++ b/tests/server_utils.go @@ -0,0 +1,53 @@ +package tests + +import ( + "io/ioutil" + "os" + "time" + + "github.com/coreos/etcd/store" + "github.com/coreos/etcd/server" +) + +const ( + testName = "ETCDTEST" + testClientURL = "localhost:4401" + testRaftURL = "localhost:7701" +) + +// Starts a server in a temporary directory. +func RunServer(f func(*server.Server)) { + path, _ := ioutil.TempDir("", "etcd-") + defer os.RemoveAll(path) + + store := store.New() + registry := server.NewRegistry(store) + ps := server.NewPeerServer(testName, path, testRaftURL, testRaftURL, &server.TLSConfig{Scheme:"http"}, &server.TLSInfo{}, registry, store) + s := server.New(testName, testClientURL, testClientURL, &server.TLSConfig{Scheme:"http"}, &server.TLSInfo{}, ps, registry, store) + ps.SetServer(s) + + // Start up peer server. + c := make(chan bool) + go func() { + c <- true + ps.ListenAndServe(false, []string{}) + }() + <- c + + // Start up etcd server. + go func() { + c <- true + s.ListenAndServe() + }() + <- c + + // Wait to make sure servers have started. + time.Sleep(5 * time.Millisecond) + + // Execute the function passed in. + f(s) + + // Clean up servers. + ps.Close() + s.Close() +} diff --git a/third_party/deps b/third_party/deps index 1007477dde6..0cb7ffea3df 100755 --- a/third_party/deps +++ b/third_party/deps @@ -5,6 +5,8 @@ packages=" github.com/coreos/go-systemd github.com/gorilla/context github.com/gorilla/mux + github.com/stretchr/testify/assert + github.com/stretchr/testify/mock bitbucket.org/kardianos/osext code.google.com/p/go.net code.google.com/p/goprotobuf diff --git a/third_party/github.com/coreos/go-etcd/etcd/client_test.go b/third_party/github.com/coreos/go-etcd/etcd/client_test.go index bf75d894796..29f138113ce 100644 --- a/third_party/github.com/coreos/go-etcd/etcd/client_test.go +++ b/third_party/github.com/coreos/go-etcd/etcd/client_test.go @@ -2,9 +2,9 @@ package etcd import ( "fmt" - "net" - "net/url" "testing" + "net/url" + "net" ) // To pass this test, we need to create a cluster of 3 machines @@ -19,7 +19,7 @@ func TestSync(t *testing.T) { t.Fatal("cannot sync machines") } - for _, m := range c.GetCluster() { + for _, m := range(c.GetCluster()) { u, err := url.Parse(m) if err != nil { t.Fatal(err) @@ -27,7 +27,7 @@ func TestSync(t *testing.T) { if u.Scheme != "http" { t.Fatal("scheme must be http") } - + host, _, err := net.SplitHostPort(u.Host) if err != nil { t.Fatal(err) diff --git a/third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go b/third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go index 8c7e375c5af..8249b4bdcb2 100644 --- a/third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go +++ b/third_party/github.com/coreos/go-etcd/examples/sync-cluster/sync-cluster.go @@ -1,3 +1,4 @@ + package main import ( diff --git a/third_party/github.com/coreos/go-log/log/commands.go b/third_party/github.com/coreos/go-log/log/commands.go index f39fdef9717..94dc9e152d7 100644 --- a/third_party/github.com/coreos/go-log/log/commands.go +++ b/third_party/github.com/coreos/go-log/log/commands.go @@ -1,5 +1,4 @@ package log - // Copyright 2013, CoreOS, Inc. All rights reserved. // // Licensed under the Apache License, Version 2.0 (the "License"); @@ -43,6 +42,7 @@ func (logger *Logger) Logf(priority Priority, format string, v ...interface{}) { logger.Log(priority, fmt.Sprintf(format, v...)) } + func (logger *Logger) Emergency(v ...interface{}) { logger.Log(PriEmerg, v...) } @@ -99,6 +99,7 @@ func (logger *Logger) Debugf(format string, v ...interface{}) { logger.Log(PriDebug, fmt.Sprintf(format, v...)) } + func Emergency(v ...interface{}) { defaultLogger.Log(PriEmerg, v...) } @@ -157,56 +158,57 @@ func Debugf(format string, v ...interface{}) { // Standard library log functions -func (logger *Logger) Fatalln(v ...interface{}) { +func (logger *Logger)Fatalln (v ...interface{}) { logger.Log(PriCrit, v...) os.Exit(1) } -func (logger *Logger) Fatalf(format string, v ...interface{}) { +func (logger *Logger)Fatalf (format string, v ...interface{}) { logger.Logf(PriCrit, format, v...) os.Exit(1) } -func (logger *Logger) Panicln(v ...interface{}) { +func (logger *Logger)Panicln (v ...interface{}) { s := fmt.Sprint(v...) logger.Log(PriErr, s) panic(s) } -func (logger *Logger) Panicf(format string, v ...interface{}) { +func (logger *Logger)Panicf (format string, v ...interface{}) { s := fmt.Sprintf(format, v...) logger.Log(PriErr, s) panic(s) } -func (logger *Logger) Println(v ...interface{}) { +func (logger *Logger)Println (v ...interface{}) { logger.Log(PriInfo, v...) } -func (logger *Logger) Printf(format string, v ...interface{}) { +func (logger *Logger)Printf (format string, v ...interface{}) { logger.Logf(PriInfo, format, v...) } -func Fatalln(v ...interface{}) { + +func Fatalln (v ...interface{}) { defaultLogger.Log(PriCrit, v...) os.Exit(1) } -func Fatalf(format string, v ...interface{}) { +func Fatalf (format string, v ...interface{}) { defaultLogger.Logf(PriCrit, format, v...) os.Exit(1) } -func Panicln(v ...interface{}) { +func Panicln (v ...interface{}) { s := fmt.Sprint(v...) defaultLogger.Log(PriErr, s) panic(s) } -func Panicf(format string, v ...interface{}) { +func Panicf (format string, v ...interface{}) { s := fmt.Sprintf(format, v...) defaultLogger.Log(PriErr, s) panic(s) } -func Println(v ...interface{}) { +func Println (v ...interface{}) { defaultLogger.Log(PriInfo, v...) } -func Printf(format string, v ...interface{}) { +func Printf (format string, v ...interface{}) { defaultLogger.Logf(PriInfo, format, v...) } diff --git a/third_party/github.com/coreos/go-log/log/fields.go b/third_party/github.com/coreos/go-log/log/fields.go index b04edc8eb77..e8d9698a08a 100644 --- a/third_party/github.com/coreos/go-log/log/fields.go +++ b/third_party/github.com/coreos/go-log/log/fields.go @@ -1,5 +1,4 @@ package log - // Copyright 2013, CoreOS, Inc. All rights reserved. // // Licensed under the Apache License, Version 2.0 (the "License"); diff --git a/third_party/github.com/coreos/go-log/log/logger.go b/third_party/github.com/coreos/go-log/log/logger.go index 8c3b86c1dff..2089a11f898 100644 --- a/third_party/github.com/coreos/go-log/log/logger.go +++ b/third_party/github.com/coreos/go-log/log/logger.go @@ -1,5 +1,4 @@ package log - // Copyright 2013, CoreOS, Inc. All rights reserved. // // Licensed under the Apache License, Version 2.0 (the "License"); diff --git a/third_party/github.com/coreos/go-log/log/priority.go b/third_party/github.com/coreos/go-log/log/priority.go index c169d686918..ac73fc8a42c 100644 --- a/third_party/github.com/coreos/go-log/log/priority.go +++ b/third_party/github.com/coreos/go-log/log/priority.go @@ -1,5 +1,4 @@ package log - // Copyright 2013, CoreOS, Inc. All rights reserved. // // Licensed under the Apache License, Version 2.0 (the "License"); diff --git a/third_party/github.com/coreos/go-log/log/sinks.go b/third_party/github.com/coreos/go-log/log/sinks.go index bdf1e41f111..a41f3365dfd 100644 --- a/third_party/github.com/coreos/go-log/log/sinks.go +++ b/third_party/github.com/coreos/go-log/log/sinks.go @@ -1,5 +1,4 @@ package log - // Copyright 2013, CoreOS, Inc. All rights reserved. // // Licensed under the Apache License, Version 2.0 (the "License"); diff --git a/third_party/github.com/coreos/go-systemd/activation/files.go b/third_party/github.com/coreos/go-systemd/activation/files.go index a0a56f9e6b1..4b854237077 100644 --- a/third_party/github.com/coreos/go-systemd/activation/files.go +++ b/third_party/github.com/coreos/go-systemd/activation/files.go @@ -24,7 +24,7 @@ func Files() []*os.File { files := []*os.File(nil) for fd := listenFdsStart; fd < listenFdsStart+nfds; fd++ { syscall.CloseOnExec(fd) - files = append(files, os.NewFile(uintptr(fd), "LISTEN_FD_"+strconv.Itoa(fd))) + files = append(files, os.NewFile(uintptr(fd), "LISTEN_FD_" + strconv.Itoa(fd))) } return files } diff --git a/third_party/github.com/stretchr/testify/assert/assertions.go b/third_party/github.com/stretchr/testify/assert/assertions.go new file mode 100644 index 00000000000..cc027cb06b9 --- /dev/null +++ b/third_party/github.com/stretchr/testify/assert/assertions.go @@ -0,0 +1,465 @@ +package assert + +import ( + "fmt" + "reflect" + "runtime" + "strings" + "testing" + "time" +) + +// Comparison a custom function that returns true on success and false on failure +type Comparison func() (success bool) + +/* + Helper functions +*/ + +// ObjectsAreEqual determines if two objects are considered equal. +// +// This function does no assertion of any kind. +func ObjectsAreEqual(a, b interface{}) bool { + + if reflect.DeepEqual(a, b) { + return true + } + + if reflect.ValueOf(a) == reflect.ValueOf(b) { + return true + } + + // Last ditch effort + if fmt.Sprintf("%#v", a) == fmt.Sprintf("%#v", b) { + return true + } + + return false + +} + +/* CallerInfo is necessary because the assert functions use the testing object +internally, causing it to print the file:line of the assert method, rather than where +the problem actually occured in calling code.*/ + +// CallerInfo returns a string containing the file and line number of the assert call +// that failed. +func CallerInfo() string { + + file := "" + line := 0 + ok := false + + for i := 0; ; i++ { + _, file, line, ok = runtime.Caller(i) + if !ok { + return "" + } + parts := strings.Split(file, "/") + dir := parts[len(parts)-2] + file = parts[len(parts)-1] + if (dir != "assert" && dir != "mock") || file == "mock_test.go" { + break + } + } + + return fmt.Sprintf("%s:%d", file, line) +} + +// getWhitespaceString returns a string that is long enough to overwrite the default +// output from the go testing framework. +func getWhitespaceString() string { + + _, file, line, ok := runtime.Caller(1) + if !ok { + return "" + } + parts := strings.Split(file, "/") + file = parts[len(parts)-1] + + return strings.Repeat(" ", len(fmt.Sprintf("%s:%d: ", file, line))) + +} + +func messageFromMsgAndArgs(msgAndArgs ...interface{}) string { + if len(msgAndArgs) == 0 || msgAndArgs == nil { + return "" + } + if len(msgAndArgs) == 1 { + return msgAndArgs[0].(string) + } + if len(msgAndArgs) > 1 { + return fmt.Sprintf(msgAndArgs[0].(string), msgAndArgs[1:]...) + } + return "" +} + +// Fail reports a failure through +func Fail(t *testing.T, failureMessage string, msgAndArgs ...interface{}) bool { + + message := messageFromMsgAndArgs(msgAndArgs...) + + if len(message) > 0 { + t.Errorf("\r%s\r\tLocation:\t%s\n\r\tError:\t\t%s\n\r\tMessages:\t%s\n\r", getWhitespaceString(), CallerInfo(), failureMessage, message) + } else { + t.Errorf("\r%s\r\tLocation:\t%s\n\r\tError:\t\t%s\n\r", getWhitespaceString(), CallerInfo(), failureMessage) + } + + return false +} + +// Implements asserts that an object is implemented by the specified interface. +// +// assert.Implements(t, (*MyInterface)(nil), new(MyObject), "MyObject") +func Implements(t *testing.T, interfaceObject interface{}, object interface{}, msgAndArgs ...interface{}) bool { + + interfaceType := reflect.TypeOf(interfaceObject).Elem() + + if !reflect.TypeOf(object).Implements(interfaceType) { + return Fail(t, fmt.Sprintf("Object must implement %v", interfaceType), msgAndArgs...) + } + + return true + +} + +// IsType asserts that the specified objects are of the same type. +func IsType(t *testing.T, expectedType interface{}, object interface{}, msgAndArgs ...interface{}) bool { + + if !ObjectsAreEqual(reflect.TypeOf(object), reflect.TypeOf(expectedType)) { + return Fail(t, fmt.Sprintf("Object expected to be of type %v, but was %v", reflect.TypeOf(expectedType), reflect.TypeOf(object)), msgAndArgs...) + } + + return true +} + +// Equal asserts that two objects are equal. +// +// assert.Equal(t, 123, 123, "123 and 123 should be equal") +// +// Returns whether the assertion was successful (true) or not (false). +func Equal(t *testing.T, a, b interface{}, msgAndArgs ...interface{}) bool { + + if !ObjectsAreEqual(a, b) { + return Fail(t, fmt.Sprintf("Not equal: %#v != %#v", a, b), msgAndArgs...) + } + + return true + +} + +// Exactly asserts that two objects are equal is value and type. +// +// assert.Exactly(t, int32(123), int64(123), "123 and 123 should NOT be equal") +// +// Returns whether the assertion was successful (true) or not (false). +func Exactly(t *testing.T, a, b interface{}, msgAndArgs ...interface{}) bool { + + aType := reflect.TypeOf(a) + bType := reflect.TypeOf(b) + + if aType != bType { + return Fail(t, "Types expected to match exactly", "%v != %v", aType, bType) + } + + return Equal(t, a, b, msgAndArgs...) + +} + +// NotNil asserts that the specified object is not nil. +// +// assert.NotNil(t, err, "err should be something") +// +// Returns whether the assertion was successful (true) or not (false). +func NotNil(t *testing.T, object interface{}, msgAndArgs ...interface{}) bool { + + var success bool = true + + if object == nil { + success = false + } else { + value := reflect.ValueOf(object) + kind := value.Kind() + if kind >= reflect.Chan && kind <= reflect.Slice && value.IsNil() { + success = false + } + } + + if !success { + Fail(t, "Expected not to be nil.", msgAndArgs...) + } + + return success +} + +// Nil asserts that the specified object is nil. +// +// assert.Nil(t, err, "err should be nothing") +// +// Returns whether the assertion was successful (true) or not (false). +func Nil(t *testing.T, object interface{}, msgAndArgs ...interface{}) bool { + + if object == nil { + return true + } else { + value := reflect.ValueOf(object) + kind := value.Kind() + if kind >= reflect.Chan && kind <= reflect.Slice && value.IsNil() { + return true + } + } + + return Fail(t, fmt.Sprintf("Expected nil, but got: %#v", object), msgAndArgs...) +} + +// isEmpty gets whether the specified object is considered empty or not. +func isEmpty(object interface{}) bool { + + if object == nil { + return true + } else if object == "" { + return true + } else if object == 0 { + return true + } else if object == false { + return true + } + + objValue := reflect.ValueOf(object) + switch objValue.Kind() { + case reflect.Slice: + { + return (objValue.Len() == 0) + } + } + + return false + +} + +// Empty asserts that the specified object is empty. I.e. nil, "", false, 0 or a +// slice with len == 0. +// +// assert.Empty(t, obj) +// +// Returns whether the assertion was successful (true) or not (false). +func Empty(t *testing.T, object interface{}, msgAndArgs ...interface{}) bool { + + pass := isEmpty(object) + if !pass { + Fail(t, fmt.Sprintf("Should be empty, but was %v", object), msgAndArgs...) + } + + return pass + +} + +// Empty asserts that the specified object is NOT empty. I.e. not nil, "", false, 0 or a +// slice with len == 0. +// +// if assert.NotEmpty(t, obj) { +// assert.Equal(t, "two", obj[1]) +// } +// +// Returns whether the assertion was successful (true) or not (false). +func NotEmpty(t *testing.T, object interface{}, msgAndArgs ...interface{}) bool { + + pass := !isEmpty(object) + if !pass { + Fail(t, fmt.Sprintf("Should NOT be empty, but was %v", object), msgAndArgs...) + } + + return pass + +} + +// True asserts that the specified value is true. +// +// assert.True(t, myBool, "myBool should be true") +// +// Returns whether the assertion was successful (true) or not (false). +func True(t *testing.T, value bool, msgAndArgs ...interface{}) bool { + + if value != true { + return Fail(t, "Should be true", msgAndArgs...) + } + + return true + +} + +// False asserts that the specified value is true. +// +// assert.False(t, myBool, "myBool should be false") +// +// Returns whether the assertion was successful (true) or not (false). +func False(t *testing.T, value bool, msgAndArgs ...interface{}) bool { + + if value != false { + return Fail(t, "Should be false", msgAndArgs...) + } + + return true + +} + +// NotEqual asserts that the specified values are NOT equal. +// +// assert.NotEqual(t, obj1, obj2, "two objects shouldn't be equal") +// +// Returns whether the assertion was successful (true) or not (false). +func NotEqual(t *testing.T, a, b interface{}, msgAndArgs ...interface{}) bool { + + if ObjectsAreEqual(a, b) { + return Fail(t, "Should not be equal", msgAndArgs...) + } + + return true + +} + +// Contains asserts that the specified string contains the specified substring. +// +// assert.Contains(t, "Hello World", "World", "But 'Hello World' does contain 'World'") +// +// Returns whether the assertion was successful (true) or not (false). +func Contains(t *testing.T, s, contains string, msgAndArgs ...interface{}) bool { + + if !strings.Contains(s, contains) { + return Fail(t, fmt.Sprintf("\"%s\" does not contain \"%s\"", s, contains), msgAndArgs...) + } + + return true + +} + +// NotContains asserts that the specified string does NOT contain the specified substring. +// +// assert.NotContains(t, "Hello World", "Earth", "But 'Hello World' does NOT contain 'Earth'") +// +// Returns whether the assertion was successful (true) or not (false). +func NotContains(t *testing.T, s, contains string, msgAndArgs ...interface{}) bool { + + if strings.Contains(s, contains) { + return Fail(t, fmt.Sprintf("\"%s\" should not contain \"%s\"", s, contains), msgAndArgs...) + } + + return true + +} + +// Uses a Comparison to assert a complex condition. +func Condition(t *testing.T, comp Comparison, msgAndArgs ...interface{}) bool { + result := comp() + if !result { + Fail(t, "Condition failed!", msgAndArgs...) + } + return result +} + +// PanicTestFunc defines a func that should be passed to the assert.Panics and assert.NotPanics +// methods, and represents a simple func that takes no arguments, and returns nothing. +type PanicTestFunc func() + +// didPanic returns true if the function passed to it panics. Otherwise, it returns false. +func didPanic(f PanicTestFunc) (bool, interface{}) { + + var didPanic bool = false + var message interface{} + func() { + + defer func() { + if message = recover(); message != nil { + didPanic = true + } + }() + + // call the target function + f() + + }() + + return didPanic, message + +} + +// Panics asserts that the code inside the specified PanicTestFunc panics. +// +// assert.Panics(t, func(){ +// GoCrazy() +// }, "Calling GoCrazy() should panic") +// +// Returns whether the assertion was successful (true) or not (false). +func Panics(t *testing.T, f PanicTestFunc, msgAndArgs ...interface{}) bool { + + if funcDidPanic, panicValue := didPanic(f); !funcDidPanic { + return Fail(t, fmt.Sprintf("func %#v should panic\n\r\tPanic value:\t%v", f, panicValue), msgAndArgs...) + } + + return true +} + +// NotPanics asserts that the code inside the specified PanicTestFunc does NOT panic. +// +// assert.NotPanics(t, func(){ +// RemainCalm() +// }, "Calling RemainCalm() should NOT panic") +// +// Returns whether the assertion was successful (true) or not (false). +func NotPanics(t *testing.T, f PanicTestFunc, msgAndArgs ...interface{}) bool { + + if funcDidPanic, panicValue := didPanic(f); funcDidPanic { + return Fail(t, fmt.Sprintf("func %#v should not panic\n\r\tPanic value:\t%v", f, panicValue), msgAndArgs...) + } + + return true +} + +// WithinDuration asserts that the two times are within duration delta of each other. +// +// assert.WithinDuration(t, time.Now(), time.Now(), 10*time.Second, "The difference should not be more than 10s") +// +// Returns whether the assertion was successful (true) or not (false). +func WithinDuration(t *testing.T, a, b time.Time, delta time.Duration, msgAndArgs ...interface{}) bool { + + dt := a.Sub(b) + if dt < -delta || dt > delta { + return Fail(t, fmt.Sprintf("Max difference between %v and %v allowed is %v, but difference was %v", a, b, dt, delta), msgAndArgs...) + } + + return true +} + +/* + Errors +*/ + +// NoError asserts that a function returned no error (i.e. `nil`). +// +// actualObj, err := SomeFunction() +// if assert.NoError(t, err) { +// assert.Equal(t, actualObj, expectedObj) +// } +// +// Returns whether the assertion was successful (true) or not (false). +func NoError(t *testing.T, theError error, msgAndArgs ...interface{}) bool { + + message := messageFromMsgAndArgs(msgAndArgs...) + return Nil(t, theError, "No error is expected but got %v %s", theError, message) + +} + +// Error asserts that a function returned an error (i.e. not `nil`). +// +// actualObj, err := SomeFunction() +// if assert.Error(t, err, "An error was expected") { +// assert.Equal(t, err, expectedError) +// } +// +// Returns whether the assertion was successful (true) or not (false). +func Error(t *testing.T, theError error, msgAndArgs ...interface{}) bool { + + message := messageFromMsgAndArgs(msgAndArgs...) + return NotNil(t, theError, "An error is expected but got nil. %s", message) + +} diff --git a/third_party/github.com/stretchr/testify/assert/assertions_test.go b/third_party/github.com/stretchr/testify/assert/assertions_test.go new file mode 100644 index 00000000000..4dda01489e7 --- /dev/null +++ b/third_party/github.com/stretchr/testify/assert/assertions_test.go @@ -0,0 +1,383 @@ +package assert + +import ( + "errors" + "testing" + "time" +) + +// AssertionTesterInterface defines an interface to be used for testing assertion methods +type AssertionTesterInterface interface { + TestMethod() +} + +// AssertionTesterConformingObject is an object that conforms to the AssertionTesterInterface interface +type AssertionTesterConformingObject struct { +} + +func (a *AssertionTesterConformingObject) TestMethod() { +} + +// AssertionTesterNonConformingObject is an object that does not conform to the AssertionTesterInterface interface +type AssertionTesterNonConformingObject struct { +} + +func TestObjectsAreEqual(t *testing.T) { + + if !ObjectsAreEqual("Hello World", "Hello World") { + t.Error("objectsAreEqual should return true") + } + if !ObjectsAreEqual(123, 123) { + t.Error("objectsAreEqual should return true") + } + if !ObjectsAreEqual(123.5, 123.5) { + t.Error("objectsAreEqual should return true") + } + if !ObjectsAreEqual([]byte("Hello World"), []byte("Hello World")) { + t.Error("objectsAreEqual should return true") + } + if !ObjectsAreEqual(nil, nil) { + t.Error("objectsAreEqual should return true") + } + +} + +func TestImplements(t *testing.T) { + + mockT := new(testing.T) + + if !Implements(mockT, (*AssertionTesterInterface)(nil), new(AssertionTesterConformingObject)) { + t.Error("Implements method should return true: AssertionTesterConformingObject implements AssertionTesterInterface") + } + if Implements(mockT, (*AssertionTesterInterface)(nil), new(AssertionTesterNonConformingObject)) { + t.Error("Implements method should return false: AssertionTesterNonConformingObject does not implements AssertionTesterInterface") + } + +} + +func TestIsType(t *testing.T) { + + mockT := new(testing.T) + + if !IsType(mockT, new(AssertionTesterConformingObject), new(AssertionTesterConformingObject)) { + t.Error("IsType should return true: AssertionTesterConformingObject is the same type as AssertionTesterConformingObject") + } + if IsType(mockT, new(AssertionTesterConformingObject), new(AssertionTesterNonConformingObject)) { + t.Error("IsType should return false: AssertionTesterConformingObject is not the same type as AssertionTesterNonConformingObject") + } + +} + +func TestEqual(t *testing.T) { + + mockT := new(testing.T) + + if !Equal(mockT, "Hello World", "Hello World") { + t.Error("Equal should return true") + } + if !Equal(mockT, 123, 123) { + t.Error("Equal should return true") + } + if !Equal(mockT, 123.5, 123.5) { + t.Error("Equal should return true") + } + if !Equal(mockT, []byte("Hello World"), []byte("Hello World")) { + t.Error("Equal should return true") + } + if !Equal(mockT, nil, nil) { + t.Error("Equal should return true") + } + +} + +func TestNotNil(t *testing.T) { + + mockT := new(testing.T) + + if !NotNil(mockT, new(AssertionTesterConformingObject)) { + t.Error("NotNil should return true: object is not nil") + } + if NotNil(mockT, nil) { + t.Error("NotNil should return false: object is nil") + } + +} + +func TestNil(t *testing.T) { + + mockT := new(testing.T) + + if !Nil(mockT, nil) { + t.Error("Nil should return true: object is nil") + } + if Nil(mockT, new(AssertionTesterConformingObject)) { + t.Error("Nil should return false: object is not nil") + } + +} + +func TestTrue(t *testing.T) { + + mockT := new(testing.T) + + if !True(mockT, true) { + t.Error("True should return true") + } + if True(mockT, false) { + t.Error("True should return false") + } + +} + +func TestFalse(t *testing.T) { + + mockT := new(testing.T) + + if !False(mockT, false) { + t.Error("False should return true") + } + if False(mockT, true) { + t.Error("False should return false") + } + +} + +func TestExactly(t *testing.T) { + + mockT := new(testing.T) + + a := float32(1) + b := float64(1) + c := float32(1) + d := float32(2) + + if Exactly(mockT, a, b) { + t.Error("Exactly should return false") + } + if Exactly(mockT, a, d) { + t.Error("Exactly should return false") + } + if !Exactly(mockT, a, c) { + t.Error("Exactly should return true") + } + + if Exactly(mockT, nil, a) { + t.Error("Exactly should return false") + } + if Exactly(mockT, a, nil) { + t.Error("Exactly should return false") + } + +} + +func TestNotEqual(t *testing.T) { + + mockT := new(testing.T) + + if !NotEqual(mockT, "Hello World", "Hello World!") { + t.Error("NotEqual should return true") + } + if !NotEqual(mockT, 123, 1234) { + t.Error("NotEqual should return true") + } + if !NotEqual(mockT, 123.5, 123.55) { + t.Error("NotEqual should return true") + } + if !NotEqual(mockT, []byte("Hello World"), []byte("Hello World!")) { + t.Error("NotEqual should return true") + } + if !NotEqual(mockT, nil, new(AssertionTesterConformingObject)) { + t.Error("NotEqual should return true") + } +} + +func TestContains(t *testing.T) { + + mockT := new(testing.T) + + if !Contains(mockT, "Hello World", "Hello") { + t.Error("Contains should return true: \"Hello World\" contains \"Hello\"") + } + if Contains(mockT, "Hello World", "Salut") { + t.Error("Contains should return false: \"Hello World\" does not contain \"Salut\"") + } + +} + +func TestNotContains(t *testing.T) { + + mockT := new(testing.T) + + if !NotContains(mockT, "Hello World", "Hello!") { + t.Error("NotContains should return true: \"Hello World\" does not contain \"Hello!\"") + } + if NotContains(mockT, "Hello World", "Hello") { + t.Error("NotContains should return false: \"Hello World\" contains \"Hello\"") + } + +} + +func TestDidPanic(t *testing.T) { + + if funcDidPanic, _ := didPanic(func() { + panic("Panic!") + }); !funcDidPanic { + t.Error("didPanic should return true") + } + + if funcDidPanic, _ := didPanic(func() { + }); funcDidPanic { + t.Error("didPanic should return false") + } + +} + +func TestPanics(t *testing.T) { + + mockT := new(testing.T) + + if !Panics(mockT, func() { + panic("Panic!") + }) { + t.Error("Panics should return true") + } + + if Panics(mockT, func() { + }) { + t.Error("Panics should return false") + } + +} + +func TestNotPanics(t *testing.T) { + + mockT := new(testing.T) + + if !NotPanics(mockT, func() { + }) { + t.Error("NotPanics should return true") + } + + if NotPanics(mockT, func() { + panic("Panic!") + }) { + t.Error("NotPanics should return false") + } + +} + +func TestEqual_Funcs(t *testing.T) { + + type f func() int + var f1 f = func() int { return 1 } + var f2 f = func() int { return 2 } + + var f1_copy f = f1 + + Equal(t, f1_copy, f1, "Funcs are the same and should be considered equal") + NotEqual(t, f1, f2, "f1 and f2 are different") + +} + +func TestNoError(t *testing.T) { + + mockT := new(testing.T) + + // start with a nil error + var err error = nil + + True(t, NoError(mockT, err), "NoError should return True for nil arg") + + // now set an error + err = errors.New("Some error") + + False(t, NoError(mockT, err), "NoError with error should return False") + +} + +func TestError(t *testing.T) { + + mockT := new(testing.T) + + // start with a nil error + var err error = nil + + False(t, Error(mockT, err), "Error should return False for nil arg") + + // now set an error + err = errors.New("Some error") + + True(t, Error(mockT, err), "Error with error should return True") + +} + +func Test_isEmpty(t *testing.T) { + + True(t, isEmpty("")) + True(t, isEmpty(nil)) + True(t, isEmpty([]string{})) + True(t, isEmpty(0)) + True(t, isEmpty(false)) + + False(t, isEmpty("something")) + False(t, isEmpty(errors.New("something"))) + False(t, isEmpty([]string{"something"})) + False(t, isEmpty(1)) + False(t, isEmpty(true)) + +} + +func TestEmpty(t *testing.T) { + + mockT := new(testing.T) + + True(t, Empty(mockT, ""), "Empty string is empty") + True(t, Empty(mockT, nil), "Nil is empty") + True(t, Empty(mockT, []string{}), "Empty string array is empty") + True(t, Empty(mockT, 0), "Zero int value is empty") + True(t, Empty(mockT, false), "False value is empty") + + False(t, Empty(mockT, "something"), "Non Empty string is not empty") + False(t, Empty(mockT, errors.New("something")), "Non nil object is not empty") + False(t, Empty(mockT, []string{"something"}), "Non empty string array is not empty") + False(t, Empty(mockT, 1), "Non-zero int value is not empty") + False(t, Empty(mockT, true), "True value is not empty") + +} + +func TestNotEmpty(t *testing.T) { + + mockT := new(testing.T) + + False(t, NotEmpty(mockT, ""), "Empty string is empty") + False(t, NotEmpty(mockT, nil), "Nil is empty") + False(t, NotEmpty(mockT, []string{}), "Empty string array is empty") + False(t, NotEmpty(mockT, 0), "Zero int value is empty") + False(t, NotEmpty(mockT, false), "False value is empty") + + True(t, NotEmpty(mockT, "something"), "Non Empty string is not empty") + True(t, NotEmpty(mockT, errors.New("something")), "Non nil object is not empty") + True(t, NotEmpty(mockT, []string{"something"}), "Non empty string array is not empty") + True(t, NotEmpty(mockT, 1), "Non-zero int value is not empty") + True(t, NotEmpty(mockT, true), "True value is not empty") + +} + +func TestWithinDuration(t *testing.T) { + + mockT := new(testing.T) + a := time.Now() + b := a.Add(10 * time.Second) + + True(t, WithinDuration(mockT, a, b, 10*time.Second), "A 10s difference is within a 10s time difference") + True(t, WithinDuration(mockT, b, a, 10*time.Second), "A 10s difference is within a 10s time difference") + + False(t, WithinDuration(mockT, a, b, 9*time.Second), "A 10s difference is not within a 9s time difference") + False(t, WithinDuration(mockT, b, a, 9*time.Second), "A 10s difference is not within a 9s time difference") + + False(t, WithinDuration(mockT, a, b, -9*time.Second), "A 10s difference is not within a 9s time difference") + False(t, WithinDuration(mockT, b, a, -9*time.Second), "A 10s difference is not within a 9s time difference") + + False(t, WithinDuration(mockT, a, b, -11*time.Second), "A 10s difference is not within a 9s time difference") + False(t, WithinDuration(mockT, b, a, -11*time.Second), "A 10s difference is not within a 9s time difference") +} diff --git a/third_party/github.com/stretchr/testify/assert/doc.go b/third_party/github.com/stretchr/testify/assert/doc.go new file mode 100644 index 00000000000..25f699bd6ed --- /dev/null +++ b/third_party/github.com/stretchr/testify/assert/doc.go @@ -0,0 +1,74 @@ +// A set of comprehensive testing tools for use with the normal Go testing system. +// +// Example Usage +// +// The following is a complete example using assert in a standard test function: +// import ( +// "testing" +// "github.com/stretchr/testify/assert" +// ) +// +// func TestSomething(t *testing.T) { +// +// var a string = "Hello" +// var b string = "Hello" +// +// assert.Equal(t, a, b, "The two words should be the same.") +// +// } +// +// Assertions +// +// Assertions allow you to easily write test code, and are global funcs in the `assert` package. +// All assertion functions take, as the first argument, the `*testing.T` object provided by the +// testing framework. This allows the assertion funcs to write the failings and other details to +// the correct place. +// +// Every assertion function also takes an optional string message as the final argument, +// allowing custom error messages to be appended to the message the assertion method outputs. +// +// Here is an overview of the assert functions: +// +// assert.Equal(t, expected, actual [, message [, format-args]) +// +// assert.NotEqual(t, notExpected, actual [, message [, format-args]]) +// +// assert.True(t, actualBool [, message [, format-args]]) +// +// assert.False(t, actualBool [, message [, format-args]]) +// +// assert.Nil(t, actualObject [, message [, format-args]]) +// +// assert.NotNil(t, actualObject [, message [, format-args]]) +// +// assert.Empty(t, actualObject [, message [, format-args]]) +// +// assert.NotEmpty(t, actualObject [, message [, format-args]]) +// +// assert.Error(t, errorObject [, message [, format-args]]) +// +// assert.NoError(t, errorObject [, message [, format-args]]) +// +// assert.Implements(t, (*MyInterface)(nil), new(MyObject) [,message [, format-args]]) +// +// assert.IsType(t, expectedObject, actualObject [, message [, format-args]]) +// +// assert.Contains(t, string, substring [, message [, format-args]]) +// +// assert.NotContains(t, string, substring [, message [, format-args]]) +// +// assert.Panics(t, func(){ +// +// // call code that should panic +// +// } [, message [, format-args]]) +// +// assert.NotPanics(t, func(){ +// +// // call code that should not panic +// +// } [, message [, format-args]]) +// +// assert.WithinDuration(t, timeA, timeB, deltaTime, [, message [, format-args]]) + +package assert diff --git a/third_party/github.com/stretchr/testify/assert/errors.go b/third_party/github.com/stretchr/testify/assert/errors.go new file mode 100644 index 00000000000..fa2e393bf90 --- /dev/null +++ b/third_party/github.com/stretchr/testify/assert/errors.go @@ -0,0 +1,10 @@ +package assert + +import ( + "errors" +) + +// AnError is an erorr instance useful for testing. If the code does not care +// about error specifics, and only needs to return the error for example, this +// error should be used to make the test code more readable. +var AnError error = errors.New("assert.AnError general error for testing.") diff --git a/third_party/github.com/stretchr/testify/mock/doc.go b/third_party/github.com/stretchr/testify/mock/doc.go new file mode 100644 index 00000000000..7d4e7b8d391 --- /dev/null +++ b/third_party/github.com/stretchr/testify/mock/doc.go @@ -0,0 +1,43 @@ +// Provides a system by which it is possible to mock your objects and verify calls are happening as expected. +// +// Example Usage +// +// The mock package provides an object, Mock, that tracks activity on another object. It is usually +// embedded into a test object as shown below: +// +// type MyTestObject struct { +// // add a Mock object instance +// mock.Mock +// +// // other fields go here as normal +// } +// +// When implementing the methods of an interface, you wire your functions up +// to call the Mock.Called(args...) method, and return the appropriate values. +// +// For example, to mock a method that saves the name and age of a person and returns +// the year of their birth or an error, you might write this: +// +// func (o *MyTestObject) SavePersonDetails(firstname, lastname string, age int) (int, error) { +// args := o.Mock.Called(firstname, lastname, age) +// return args.Int(0), args.Error(1) +// } +// +// The Int, Error and Bool methods are examples of strongly typed getters that take the argument +// index position. Given this argument list: +// +// (12, true, "Something") +// +// You could read them out strongly typed like this: +// +// args.Int(0) +// args.Bool(1) +// args.String(2) +// +// For objects of your own type, use the generic Arguments.Get(index) method and make a type assertion: +// +// return args.Get(0).(*MyObject), args.Get(1).(*AnotherObjectOfMine) +// +// This may cause a panic if the object you are getting is nil (the type assertion will fail), in those +// cases you should check for nil first. +package mock diff --git a/third_party/github.com/stretchr/testify/mock/mock.go b/third_party/github.com/stretchr/testify/mock/mock.go new file mode 100644 index 00000000000..bead86d8db4 --- /dev/null +++ b/third_party/github.com/stretchr/testify/mock/mock.go @@ -0,0 +1,465 @@ +package mock + +import ( + "fmt" + "github.com/stretchr/objx" + "github.com/stretchr/testify/assert" + "reflect" + "runtime" + "strings" + "testing" +) + +/* + Call +*/ + +// Call represents a method call and is used for setting expectations, +// as well as recording activity. +type Call struct { + + // The name of the method that was or will be called. + Method string + + // Holds the arguments of the method. + Arguments Arguments + + // Holds the arguments that should be returned when + // this method is called. + ReturnArguments Arguments +} + +// Mock is the workhorse used to track activity on another object. +// For an example of its usage, refer to the "Example Usage" section at the top of this document. +type Mock struct { + + // The method name that is currently + // being referred to by the On method. + onMethodName string + + // An array of the arguments that are + // currently being referred to by the On method. + onMethodArguments Arguments + + // Represents the calls that are expected of + // an object. + ExpectedCalls []Call + + // Holds the calls that were made to this mocked object. + Calls []Call + + // TestData holds any data that might be useful for testing. Testify ignores + // this data completely allowing you to do whatever you like with it. + testData objx.Map +} + +// TestData holds any data that might be useful for testing. Testify ignores +// this data completely allowing you to do whatever you like with it. +func (m *Mock) TestData() objx.Map { + + if m.testData == nil { + m.testData = make(objx.Map) + } + + return m.testData +} + +/* + Setting expectations +*/ + +// On starts a description of an expectation of the specified method +// being called. +// +// Mock.On("MyMethod", arg1, arg2) +func (m *Mock) On(methodName string, arguments ...interface{}) *Mock { + m.onMethodName = methodName + m.onMethodArguments = arguments + return m +} + +// Return finishes a description of an expectation of the method (and arguments) +// specified in the most recent On method call. +// +// Mock.On("MyMethod", arg1, arg2).Return(returnArg1, returnArg2) +func (m *Mock) Return(returnArguments ...interface{}) *Mock { + m.ExpectedCalls = append(m.ExpectedCalls, Call{m.onMethodName, m.onMethodArguments, returnArguments}) + return m +} + +/* + Recording and responding to activity +*/ + +func (m *Mock) findExpectedCall(method string, arguments ...interface{}) (bool, *Call) { + for _, call := range m.ExpectedCalls { + if call.Method == method { + + _, diffCount := call.Arguments.Diff(arguments) + if diffCount == 0 { + return true, &call + } + + } + } + return false, nil +} + +func (m *Mock) findClosestCall(method string, arguments ...interface{}) (bool, *Call) { + + diffCount := 0 + var closestCall *Call = nil + + for _, call := range m.ExpectedCalls { + if call.Method == method { + + _, tempDiffCount := call.Arguments.Diff(arguments) + if tempDiffCount < diffCount || diffCount == 0 { + diffCount = tempDiffCount + closestCall = &call + } + + } + } + + if closestCall == nil { + return false, nil + } + + return true, closestCall +} + +func callString(method string, arguments Arguments, includeArgumentValues bool) string { + + var argValsString string = "" + if includeArgumentValues { + var argVals []string + for argIndex, arg := range arguments { + argVals = append(argVals, fmt.Sprintf("%d: %v", argIndex, arg)) + } + argValsString = fmt.Sprintf("\n\t\t%s", strings.Join(argVals, "\n\t\t")) + } + + return fmt.Sprintf("%s(%s)%s", method, arguments.String(), argValsString) +} + +// Called tells the mock object that a method has been called, and gets an array +// of arguments to return. Panics if the call is unexpected (i.e. not preceeded by +// appropriate .On .Return() calls) +func (m *Mock) Called(arguments ...interface{}) Arguments { + + // get the calling function's name + pc, _, _, ok := runtime.Caller(1) + if !ok { + panic("Couldn't get the caller information") + } + functionPath := runtime.FuncForPC(pc).Name() + parts := strings.Split(functionPath, ".") + functionName := parts[len(parts)-1] + + found, call := m.findExpectedCall(functionName, arguments...) + + if !found { + + // we have to fail here - because we don't know what to do + // as the return arguments. This is because: + // + // a) this is a totally unexpected call to this method, + // b) the arguments are not what was expected, or + // c) the developer has forgotten to add an accompanying On...Return pair. + + closestFound, closestCall := m.findClosestCall(functionName, arguments...) + + if closestFound { + panic(fmt.Sprintf("\n\nmock: Unexpected Method Call\n-----------------------------\n\n%s\n\nThe closest call I have is: \n\n%s\n", callString(functionName, arguments, true), callString(functionName, closestCall.Arguments, true))) + } else { + panic(fmt.Sprintf("\nassert: mock: I don't know what to return because the method call was unexpected.\n\tEither do Mock.On(\"%s\").Return(...) first, or remove the %s() call.\n\tThis method was unexpected:\n\t\t%s\n\tat: %s", functionName, functionName, callString(functionName, arguments, true), assert.CallerInfo())) + } + + } + + // add the call + m.Calls = append(m.Calls, Call{functionName, arguments, make([]interface{}, 0)}) + + return call.ReturnArguments + +} + +/* + Assertions +*/ + +// AssertExpectationsForObjects asserts that everything specified with On and Return +// of the specified objects was in fact called as expected. +// +// Calls may have occurred in any order. +func AssertExpectationsForObjects(t *testing.T, testObjects ...interface{}) bool { + var success bool = true + for _, obj := range testObjects { + mockObj := obj.(Mock) + success = success && mockObj.AssertExpectations(t) + } + return success +} + +// AssertExpectations asserts that everything specified with On and Return was +// in fact called as expected. Calls may have occurred in any order. +func (m *Mock) AssertExpectations(t *testing.T) bool { + + var somethingMissing bool = false + var failedExpectations int = 0 + + // iterate through each expectation + for _, expectedCall := range m.ExpectedCalls { + if !m.methodWasCalled(expectedCall.Method, expectedCall.Arguments) { + somethingMissing = true + failedExpectations++ + t.Logf("\u274C\t%s(%s)", expectedCall.Method, expectedCall.Arguments.String()) + } else { + t.Logf("\u2705\t%s(%s)", expectedCall.Method, expectedCall.Arguments.String()) + } + } + + if somethingMissing { + t.Errorf("FAIL: %d out of %d expectation(s) were met.\n\tThe code you are testing needs to make %d more call(s).\n\tat: %s", len(m.ExpectedCalls)-failedExpectations, len(m.ExpectedCalls), failedExpectations, assert.CallerInfo()) + } + + return !somethingMissing +} + +// AssertNumberOfCalls asserts that the method was called expectedCalls times. +func (m *Mock) AssertNumberOfCalls(t *testing.T, methodName string, expectedCalls int) bool { + var actualCalls int = 0 + for _, call := range m.Calls { + if call.Method == methodName { + actualCalls++ + } + } + return assert.Equal(t, actualCalls, expectedCalls, fmt.Sprintf("Expected number of calls (%d) does not match the actual number of calls (%d).", expectedCalls, actualCalls)) +} + +// AssertCalled asserts that the method was called. +func (m *Mock) AssertCalled(t *testing.T, methodName string, arguments ...interface{}) bool { + if !assert.True(t, m.methodWasCalled(methodName, arguments), fmt.Sprintf("The \"%s\" method should have been called with %d argument(s), but was not.", methodName, len(arguments))) { + t.Logf("%s", m.ExpectedCalls) + return false + } + return true +} + +// AssertNotCalled asserts that the method was not called. +func (m *Mock) AssertNotCalled(t *testing.T, methodName string, arguments ...interface{}) bool { + if !assert.False(t, m.methodWasCalled(methodName, arguments), fmt.Sprintf("The \"%s\" method was called with %d argument(s), but should NOT have been.", methodName, len(arguments))) { + t.Logf("%s", m.ExpectedCalls) + return false + } + return true +} + +func (m *Mock) methodWasCalled(methodName string, arguments []interface{}) bool { + for _, call := range m.Calls { + if call.Method == methodName { + + _, differences := call.Arguments.Diff(arguments) + + if differences == 0 { + // found the expected call + return true + } + + } + } + // we didn't find the expected call + return false +} + +/* + Arguments +*/ + +// Arguments holds an array of method arguments or return values. +type Arguments []interface{} + +const ( + // The "any" argument. Used in Diff and Assert when + // the argument being tested shouldn't be taken into consideration. + Anything string = "mock.Anything" +) + +// AnythingOfTypeArgument is a string that contains the type of an argument +// for use when type checking. Used in Diff and Assert. +type AnythingOfTypeArgument string + +// AnythingOfType returns an AnythingOfTypeArgument object containing the +// name of the type to check for. Used in Diff and Assert. +// +// For example: +// Assert(t, AnythingOfType("string"), AnythingOfType("int")) +func AnythingOfType(t string) AnythingOfTypeArgument { + return AnythingOfTypeArgument(t) +} + +// Get Returns the argument at the specified index. +func (args Arguments) Get(index int) interface{} { + if index+1 > len(args) { + panic(fmt.Sprintf("assert: arguments: Cannot call Get(%d) because there are %d argument(s).", index, len(args))) + } + return args[index] +} + +// Is gets whether the objects match the arguments specified. +func (args Arguments) Is(objects ...interface{}) bool { + for i, obj := range args { + if obj != objects[i] { + return false + } + } + return true +} + +// Diff gets a string describing the differences between the arguments +// and the specified objects. +// +// Returns the diff string and number of differences found. +func (args Arguments) Diff(objects []interface{}) (string, int) { + + var output string = "\n" + var differences int + + var maxArgCount int = len(args) + if len(objects) > maxArgCount { + maxArgCount = len(objects) + } + + for i := 0; i < maxArgCount; i++ { + var actual, expected interface{} + + if len(args) <= i { + actual = "(Missing)" + } else { + actual = args[i] + } + + if len(objects) <= i { + expected = "(Missing)" + } else { + expected = objects[i] + } + + if reflect.TypeOf(expected) == reflect.TypeOf((*AnythingOfTypeArgument)(nil)).Elem() { + + // type checking + if reflect.TypeOf(actual).Name() != string(expected.(AnythingOfTypeArgument)) { + // not match + differences++ + output = fmt.Sprintf("%s\t%d: \u274C type %s != type %s - %s\n", output, i, expected, reflect.TypeOf(actual).Name(), actual) + } + + } else { + + // normal checking + + if assert.ObjectsAreEqual(expected, Anything) || assert.ObjectsAreEqual(actual, Anything) || assert.ObjectsAreEqual(actual, expected) { + // match + output = fmt.Sprintf("%s\t%d: \u2705 %s == %s\n", output, i, expected, actual) + } else { + // not match + differences++ + output = fmt.Sprintf("%s\t%d: \u274C %s != %s\n", output, i, expected, actual) + } + } + + } + + if differences == 0 { + return "No differences.", differences + } + + return output, differences + +} + +// Assert compares the arguments with the specified objects and fails if +// they do not exactly match. +func (args Arguments) Assert(t *testing.T, objects ...interface{}) bool { + + // get the differences + diff, diffCount := args.Diff(objects) + + if diffCount == 0 { + return true + } + + // there are differences... report them... + t.Logf(diff) + t.Errorf("%sArguments do not match.", assert.CallerInfo()) + + return false + +} + +// String gets the argument at the specified index. Panics if there is no argument, or +// if the argument is of the wrong type. +// +// If no index is provided, String() returns a complete string representation +// of the arguments. +func (args Arguments) String(indexOrNil ...int) string { + + if len(indexOrNil) == 0 { + // normal String() method - return a string representation of the args + var argsStr []string + for _, arg := range args { + argsStr = append(argsStr, fmt.Sprintf("%s", reflect.TypeOf(arg))) + } + return strings.Join(argsStr, ",") + } else if len(indexOrNil) == 1 { + // Index has been specified - get the argument at that index + var index int = indexOrNil[0] + var s string + var ok bool + if s, ok = args.Get(index).(string); !ok { + panic(fmt.Sprintf("assert: arguments: String(%d) failed because object wasn't correct type: %s", index, args.Get(index))) + } + return s + } + + panic(fmt.Sprintf("assert: arguments: Wrong number of arguments passed to String. Must be 0 or 1, not %d", len(indexOrNil))) + +} + +// Int gets the argument at the specified index. Panics if there is no argument, or +// if the argument is of the wrong type. +func (args Arguments) Int(index int) int { + var s int + var ok bool + if s, ok = args.Get(index).(int); !ok { + panic(fmt.Sprintf("assert: arguments: Int(%d) failed because object wasn't correct type: %s", index, args.Get(index))) + } + return s +} + +// Error gets the argument at the specified index. Panics if there is no argument, or +// if the argument is of the wrong type. +func (args Arguments) Error(index int) error { + obj := args.Get(index) + var s error + var ok bool + if obj == nil { + return nil + } + if s, ok = obj.(error); !ok { + panic(fmt.Sprintf("assert: arguments: Error(%d) failed because object wasn't correct type: %s", index, args.Get(index))) + } + return s +} + +// Bool gets the argument at the specified index. Panics if there is no argument, or +// if the argument is of the wrong type. +func (args Arguments) Bool(index int) bool { + var s bool + var ok bool + if s, ok = args.Get(index).(bool); !ok { + panic(fmt.Sprintf("assert: arguments: Bool(%d) failed because object wasn't correct type: %s", index, args.Get(index))) + } + return s +} diff --git a/third_party/github.com/stretchr/testify/mock/mock_test.go b/third_party/github.com/stretchr/testify/mock/mock_test.go new file mode 100644 index 00000000000..f50b3b1ae50 --- /dev/null +++ b/third_party/github.com/stretchr/testify/mock/mock_test.go @@ -0,0 +1,418 @@ +package mock + +import ( + "errors" + "github.com/stretchr/testify/assert" + "testing" +) + +/* + Test objects +*/ + +// ExampleInterface represents an example interface. +type ExampleInterface interface { + TheExampleMethod(a, b, c int) (int, error) +} + +// TestExampleImplementation is a test implementation of ExampleInterface +type TestExampleImplementation struct { + Mock +} + +func (i *TestExampleImplementation) TheExampleMethod(a, b, c int) (int, error) { + args := i.Mock.Called(a, b, c) + return args.Int(0), args.Error(1) +} + +func (i *TestExampleImplementation) TheExampleMethod2(yesorno bool) { + i.Mock.Called(yesorno) +} + +/* + Mock +*/ + +func Test_Mock_TestData(t *testing.T) { + + var mockedService *TestExampleImplementation = new(TestExampleImplementation) + + if assert.NotNil(t, mockedService.TestData()) { + + mockedService.TestData().Set("something", 123) + assert.Equal(t, 123, mockedService.TestData().Get("something").Data()) + + } + +} + +func Test_Mock_On(t *testing.T) { + + // make a test impl object + var mockedService *TestExampleImplementation = new(TestExampleImplementation) + + assert.Equal(t, mockedService.Mock.On("TheExampleMethod"), &mockedService.Mock) + assert.Equal(t, "TheExampleMethod", mockedService.Mock.onMethodName) + +} + +func Test_Mock_On_WithArgs(t *testing.T) { + + // make a test impl object + var mockedService *TestExampleImplementation = new(TestExampleImplementation) + + assert.Equal(t, mockedService.Mock.On("TheExampleMethod", 1, 2, 3), &mockedService.Mock) + assert.Equal(t, "TheExampleMethod", mockedService.Mock.onMethodName) + assert.Equal(t, 1, mockedService.Mock.onMethodArguments[0]) + assert.Equal(t, 2, mockedService.Mock.onMethodArguments[1]) + assert.Equal(t, 3, mockedService.Mock.onMethodArguments[2]) + +} + +func Test_Mock_Return(t *testing.T) { + + // make a test impl object + var mockedService *TestExampleImplementation = new(TestExampleImplementation) + + assert.Equal(t, mockedService.Mock.On("TheExampleMethod", "A", "B", true).Return(1, "two", true), &mockedService.Mock) + + // ensure the call was created + if assert.Equal(t, 1, len(mockedService.Mock.ExpectedCalls)) { + call := mockedService.Mock.ExpectedCalls[0] + + assert.Equal(t, "TheExampleMethod", call.Method) + assert.Equal(t, "A", call.Arguments[0]) + assert.Equal(t, "B", call.Arguments[1]) + assert.Equal(t, true, call.Arguments[2]) + assert.Equal(t, 1, call.ReturnArguments[0]) + assert.Equal(t, "two", call.ReturnArguments[1]) + assert.Equal(t, true, call.ReturnArguments[2]) + + } + +} + +func Test_Mock_Return_Nothing(t *testing.T) { + + // make a test impl object + var mockedService *TestExampleImplementation = new(TestExampleImplementation) + + assert.Equal(t, mockedService.Mock.On("TheExampleMethod", "A", "B", true).Return(), &mockedService.Mock) + + // ensure the call was created + if assert.Equal(t, 1, len(mockedService.Mock.ExpectedCalls)) { + call := mockedService.Mock.ExpectedCalls[0] + + assert.Equal(t, "TheExampleMethod", call.Method) + assert.Equal(t, "A", call.Arguments[0]) + assert.Equal(t, "B", call.Arguments[1]) + assert.Equal(t, true, call.Arguments[2]) + assert.Equal(t, 0, len(call.ReturnArguments)) + + } + +} + +func Test_Mock_findExpectedCall(t *testing.T) { + + m := new(Mock) + m.On("One", 1).Return("one") + m.On("Two", 2).Return("two") + m.On("Two", 3).Return("three") + + f, c := m.findExpectedCall("Two", 3) + + if assert.True(t, f) { + if assert.NotNil(t, c) { + assert.Equal(t, "Two", c.Method) + assert.Equal(t, 3, c.Arguments[0]) + assert.Equal(t, "three", c.ReturnArguments[0]) + } + } + +} + +func Test_callString(t *testing.T) { + + assert.Equal(t, `Method(int,bool,string)`, callString("Method", []interface{}{1, true, "something"}, false)) + +} + +func Test_Mock_Called(t *testing.T) { + + var mockedService *TestExampleImplementation = new(TestExampleImplementation) + + mockedService.Mock.On("Test_Mock_Called", 1, 2, 3).Return(5, "6", true) + + returnArguments := mockedService.Mock.Called(1, 2, 3) + + if assert.Equal(t, 1, len(mockedService.Mock.Calls)) { + assert.Equal(t, "Test_Mock_Called", mockedService.Mock.Calls[0].Method) + assert.Equal(t, 1, mockedService.Mock.Calls[0].Arguments[0]) + assert.Equal(t, 2, mockedService.Mock.Calls[0].Arguments[1]) + assert.Equal(t, 3, mockedService.Mock.Calls[0].Arguments[2]) + } + + if assert.Equal(t, 3, len(returnArguments)) { + assert.Equal(t, 5, returnArguments[0]) + assert.Equal(t, "6", returnArguments[1]) + assert.Equal(t, true, returnArguments[2]) + } + +} + +func Test_Mock_Called_Unexpected(t *testing.T) { + + var mockedService *TestExampleImplementation = new(TestExampleImplementation) + + // make sure it panics if no expectation was made + assert.Panics(t, func() { + mockedService.Mock.Called(1, 2, 3) + }, "Calling unexpected method should panic") + +} + +func Test_AssertExpectationsForObjects_Helper(t *testing.T) { + + var mockedService1 *TestExampleImplementation = new(TestExampleImplementation) + var mockedService2 *TestExampleImplementation = new(TestExampleImplementation) + var mockedService3 *TestExampleImplementation = new(TestExampleImplementation) + + mockedService1.Mock.On("Test_AssertExpectationsForObjects_Helper", 1).Return() + mockedService2.Mock.On("Test_AssertExpectationsForObjects_Helper", 2).Return() + mockedService3.Mock.On("Test_AssertExpectationsForObjects_Helper", 3).Return() + + mockedService1.Called(1) + mockedService2.Called(2) + mockedService3.Called(3) + + assert.True(t, AssertExpectationsForObjects(t, mockedService1.Mock, mockedService2.Mock, mockedService3.Mock)) + +} + +func Test_AssertExpectationsForObjects_Helper_Failed(t *testing.T) { + + var mockedService1 *TestExampleImplementation = new(TestExampleImplementation) + var mockedService2 *TestExampleImplementation = new(TestExampleImplementation) + var mockedService3 *TestExampleImplementation = new(TestExampleImplementation) + + mockedService1.Mock.On("Test_AssertExpectationsForObjects_Helper_Failed", 1).Return() + mockedService2.Mock.On("Test_AssertExpectationsForObjects_Helper_Failed", 2).Return() + mockedService3.Mock.On("Test_AssertExpectationsForObjects_Helper_Failed", 3).Return() + + mockedService1.Called(1) + mockedService3.Called(3) + + tt := new(testing.T) + assert.False(t, AssertExpectationsForObjects(tt, mockedService1.Mock, mockedService2.Mock, mockedService3.Mock)) + +} + +func Test_Mock_AssertExpectations(t *testing.T) { + + var mockedService *TestExampleImplementation = new(TestExampleImplementation) + + mockedService.Mock.On("Test_Mock_AssertExpectations", 1, 2, 3).Return(5, 6, 7) + + tt := new(testing.T) + assert.False(t, mockedService.AssertExpectations(tt)) + + // make the call now + mockedService.Mock.Called(1, 2, 3) + + // now assert expectations + assert.True(t, mockedService.AssertExpectations(tt)) + +} + +func Test_Mock_AssertNumberOfCalls(t *testing.T) { + + var mockedService *TestExampleImplementation = new(TestExampleImplementation) + + mockedService.Mock.On("Test_Mock_AssertNumberOfCalls", 1, 2, 3).Return(5, 6, 7) + + mockedService.Mock.Called(1, 2, 3) + assert.True(t, mockedService.AssertNumberOfCalls(t, "Test_Mock_AssertNumberOfCalls", 1)) + + mockedService.Mock.Called(1, 2, 3) + assert.True(t, mockedService.AssertNumberOfCalls(t, "Test_Mock_AssertNumberOfCalls", 2)) + +} + +func Test_Mock_AssertCalled(t *testing.T) { + + var mockedService *TestExampleImplementation = new(TestExampleImplementation) + + mockedService.Mock.On("Test_Mock_AssertCalled", 1, 2, 3).Return(5, 6, 7) + + mockedService.Mock.Called(1, 2, 3) + + assert.True(t, mockedService.AssertCalled(t, "Test_Mock_AssertCalled", 1, 2, 3)) + +} + +func Test_Mock_AssertCalled_WithArguments(t *testing.T) { + + var mockedService *TestExampleImplementation = new(TestExampleImplementation) + + mockedService.Mock.On("Test_Mock_AssertCalled_WithArguments", 1, 2, 3).Return(5, 6, 7) + + mockedService.Mock.Called(1, 2, 3) + + tt := new(testing.T) + assert.True(t, mockedService.AssertCalled(tt, "Test_Mock_AssertCalled_WithArguments", 1, 2, 3)) + assert.False(t, mockedService.AssertCalled(tt, "Test_Mock_AssertCalled_WithArguments", 2, 3, 4)) + +} + +func Test_Mock_AssertNotCalled(t *testing.T) { + + var mockedService *TestExampleImplementation = new(TestExampleImplementation) + + mockedService.Mock.On("Test_Mock_AssertNotCalled", 1, 2, 3).Return(5, 6, 7) + + mockedService.Mock.Called(1, 2, 3) + + assert.True(t, mockedService.AssertNotCalled(t, "Test_Mock_NotCalled")) + +} + +/* + Arguments helper methods +*/ +func Test_Arguments_Get(t *testing.T) { + + var args Arguments = []interface{}{"string", 123, true} + + assert.Equal(t, "string", args.Get(0).(string)) + assert.Equal(t, 123, args.Get(1).(int)) + assert.Equal(t, true, args.Get(2).(bool)) + +} + +func Test_Arguments_Is(t *testing.T) { + + var args Arguments = []interface{}{"string", 123, true} + + assert.True(t, args.Is("string", 123, true)) + assert.False(t, args.Is("wrong", 456, false)) + +} + +func Test_Arguments_Diff(t *testing.T) { + + var args Arguments = []interface{}{"Hello World", 123, true} + var diff string + var count int + diff, count = args.Diff([]interface{}{"Hello World", 456, "false"}) + + assert.Equal(t, 2, count) + assert.Contains(t, diff, `%!s(int=456) != %!s(int=123)`) + assert.Contains(t, diff, `false != %!s(bool=true)`) + +} + +func Test_Arguments_Diff_DifferentNumberOfArgs(t *testing.T) { + + var args Arguments = []interface{}{"string", 123, true} + var diff string + var count int + diff, count = args.Diff([]interface{}{"string", 456, "false", "extra"}) + + assert.Equal(t, 3, count) + assert.Contains(t, diff, `extra != (Missing)`) + +} + +func Test_Arguments_Diff_WithAnythingArgument(t *testing.T) { + + var args Arguments = []interface{}{"string", 123, true} + var count int + _, count = args.Diff([]interface{}{"string", Anything, true}) + + assert.Equal(t, 0, count) + +} + +func Test_Arguments_Diff_WithAnythingArgument_InActualToo(t *testing.T) { + + var args Arguments = []interface{}{"string", Anything, true} + var count int + _, count = args.Diff([]interface{}{"string", 123, true}) + + assert.Equal(t, 0, count) + +} + +func Test_Arguments_Diff_WithAnythingOfTypeArgument(t *testing.T) { + + var args Arguments = []interface{}{"string", 123, true} + var count int + _, count = args.Diff([]interface{}{"string", AnythingOfType("int"), true}) + + assert.Equal(t, 0, count) + +} + +func Test_Arguments_Diff_WithAnythingOfTypeArgument_Failing(t *testing.T) { + + var args Arguments = []interface{}{"string", 123, true} + var count int + var diff string + diff, count = args.Diff([]interface{}{"string", AnythingOfType("string"), true}) + + assert.Equal(t, 1, count) + assert.Contains(t, diff, `string != type int - %!s(int=123)`) + +} + +func Test_Arguments_Assert(t *testing.T) { + + var args Arguments = []interface{}{"string", 123, true} + + assert.True(t, args.Assert(t, "string", 123, true)) + +} + +func Test_Arguments_String_Representation(t *testing.T) { + + var args Arguments = []interface{}{"string", 123, true} + assert.Equal(t, `string,int,bool`, args.String()) + +} + +func Test_Arguments_String(t *testing.T) { + + var args Arguments = []interface{}{"string", 123, true} + assert.Equal(t, "string", args.String(0)) + +} + +func Test_Arguments_Error(t *testing.T) { + + var err error = errors.New("An Error") + var args Arguments = []interface{}{"string", 123, true, err} + assert.Equal(t, err, args.Error(3)) + +} + +func Test_Arguments_Error_Nil(t *testing.T) { + + var args Arguments = []interface{}{"string", 123, true, nil} + assert.Equal(t, nil, args.Error(3)) + +} + +func Test_Arguments_Int(t *testing.T) { + + var args Arguments = []interface{}{"string", 123, true} + assert.Equal(t, 123, args.Int(1)) + +} + +func Test_Arguments_Bool(t *testing.T) { + + var args Arguments = []interface{}{"string", 123, true} + assert.Equal(t, true, args.Bool(2)) + +} From 4135f425d30ce06503c98d02fec3e141f20104ea Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Thu, 17 Oct 2013 20:53:30 -0700 Subject: [PATCH 128/247] fix bad cert path --- tests/functional/util.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/functional/util.go b/tests/functional/util.go index 981ff1be683..3ef0028cdde 100644 --- a/tests/functional/util.go +++ b/tests/functional/util.go @@ -57,14 +57,14 @@ func Set(stop chan bool) { func CreateCluster(size int, procAttr *os.ProcAttr, ssl bool) ([][]string, []*os.Process, error) { argGroup := make([][]string, size) - sslServer1 := []string{"-serverCAFile=./fixtures/ca/ca.crt", - "-serverCert=./fixtures/ca/server.crt", - "-serverKey=./fixtures/ca/server.key.insecure", + sslServer1 := []string{"-serverCAFile=../../fixtures/ca/ca.crt", + "-serverCert=../../fixtures/ca/server.crt", + "-serverKey=../../fixtures/ca/server.key.insecure", } - sslServer2 := []string{"-serverCAFile=./fixtures/ca/ca.crt", - "-serverCert=./fixtures/ca/server2.crt", - "-serverKey=./fixtures/ca/server2.key.insecure", + sslServer2 := []string{"-serverCAFile=../../fixtures/ca/ca.crt", + "-serverCert=../../fixtures/ca/server2.crt", + "-serverKey=../../fixtures/ca/server2.key.insecure", } for i := 0; i < size; i++ { From a6f706926438bbbfaf539cee323d89d5d7d66fe0 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Thu, 17 Oct 2013 20:57:00 -0700 Subject: [PATCH 129/247] fix use right gopath --- test.sh | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/test.sh b/test.sh index 31196f7eccc..f48faffd32e 100755 --- a/test.sh +++ b/test.sh @@ -4,8 +4,13 @@ set -e # Get GOPATH, etc from build . ./build +# use right GOPATH +export GOPATH="${PWD}" + # Unit tests +go test -i ./store go test -v ./store # Functional tests +go test -i ./tests/functional ETCD_BIN_PATH=$(pwd)/etcd go test -v ./tests/functional From 7b6e305d4032c4bb06a0ffdf6b756cf912418774 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 19 Oct 2013 11:51:41 -0700 Subject: [PATCH 130/247] fix update also return newValue --- store/store.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/store/store.go b/store/store.go index 008ac26d5f3..20cfe0c87bc 100644 --- a/store/store.go +++ b/store/store.go @@ -306,6 +306,8 @@ func (s *store) Update(nodePath string, newValue string, expireTime time.Time, i // update ttl n.UpdateTTL(expireTime) + e.Value = newValue + e.Expiration, e.TTL = n.ExpirationAndTTL() s.WatcherHub.notify(e) From 6684186033661d6cdf62bb1e801fab5354e25887 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 19 Oct 2013 15:06:35 -0700 Subject: [PATCH 131/247] fix append rawqurey to the redirect location. --- server/util.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/util.go b/server/util.go index 6a53884baa2..baa7c2845b3 100644 --- a/server/util.go +++ b/server/util.go @@ -20,7 +20,8 @@ func decodeJsonRequest(req *http.Request, data interface{}) error { func redirect(hostname string, w http.ResponseWriter, req *http.Request) { path := req.URL.Path - url := hostname + path + query := req.URL.RawQuery + url := hostname + path + "?" + query log.Debugf("Redirect to %s", url) http.Redirect(w, req, url, http.StatusTemporaryRedirect) } From acbc0c8846ef4b3212477c3834172faeb9eef130 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 19 Oct 2013 19:57:36 -0700 Subject: [PATCH 132/247] refactor return http status 201 Created when creating a new node --- server/peer_server.go | 51 ----------------------------- server/peer_server_handlers.go | 60 +++++++++++++++++----------------- server/server.go | 53 +++++++++++++++++++++++++++++- store/event.go | 13 ++++++++ 4 files changed, 95 insertions(+), 82 deletions(-) diff --git a/server/peer_server.go b/server/peer_server.go index d85ed9a2194..b165205bb2a 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -9,7 +9,6 @@ import ( "io/ioutil" "net/http" "net/url" - "strings" "time" etcdErr "github.com/coreos/etcd/error" @@ -344,53 +343,3 @@ func (s *PeerServer) monitorSnapshot() { } } } - -func (s *PeerServer) dispatch(c raft.Command, w http.ResponseWriter, req *http.Request) error { - if s.raftServer.State() == raft.Leader { - result, err := s.raftServer.Do(c) - if err != nil { - return err - } - - if result == nil { - return etcdErr.NewError(300, "Empty result from raft", store.UndefIndex, store.UndefTerm) - } - - // response for raft related commands[join/remove] - if b, ok := result.([]byte); ok { - w.WriteHeader(http.StatusOK) - w.Write(b) - return nil - } - - var b []byte - if strings.HasPrefix(req.URL.Path, "/v1") { - b, _ = json.Marshal(result.(*store.Event).Response()) - } else { - b, _ = json.Marshal(result.(*store.Event)) - } - w.WriteHeader(http.StatusOK) - w.Write(b) - - return nil - - } else { - leader := s.raftServer.Leader() - - // No leader available. - if leader == "" { - return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) - } - - var url string - switch c.(type) { - case *JoinCommand, *RemoveCommand: - url, _ = s.registry.PeerURL(leader) - default: - url, _ = s.registry.ClientURL(leader) - } - redirect(url, w, req) - - return nil - } -} diff --git a/server/peer_server_handlers.go b/server/peer_server_handlers.go index adb192e606f..26ce7a5cd1a 100644 --- a/server/peer_server_handlers.go +++ b/server/peer_server_handlers.go @@ -10,20 +10,20 @@ import ( ) // Get all the current logs -func (s *PeerServer) GetLogHttpHandler(w http.ResponseWriter, req *http.Request) { - log.Debugf("[recv] GET %s/log", s.url) +func (ps *PeerServer) GetLogHttpHandler(w http.ResponseWriter, req *http.Request) { + log.Debugf("[recv] GET %s/log", ps.url) w.Header().Set("Content-Type", "application/json") w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(s.raftServer.LogEntries()) + json.NewEncoder(w).Encode(ps.raftServer.LogEntries()) } // Response to vote request -func (s *PeerServer) VoteHttpHandler(w http.ResponseWriter, req *http.Request) { +func (ps *PeerServer) VoteHttpHandler(w http.ResponseWriter, req *http.Request) { rvreq := &raft.RequestVoteRequest{} err := decodeJsonRequest(req, rvreq) if err == nil { - log.Debugf("[recv] POST %s/vote [%s]", s.url, rvreq.CandidateName) - if resp := s.raftServer.RequestVote(rvreq); resp != nil { + log.Debugf("[recv] POST %s/vote [%s]", ps.url, rvreq.CandidateName) + if resp := ps.raftServer.RequestVote(rvreq); resp != nil { w.WriteHeader(http.StatusOK) json.NewEncoder(w).Encode(resp) return @@ -34,16 +34,16 @@ func (s *PeerServer) VoteHttpHandler(w http.ResponseWriter, req *http.Request) { } // Response to append entries request -func (s *PeerServer) AppendEntriesHttpHandler(w http.ResponseWriter, req *http.Request) { +func (ps *PeerServer) AppendEntriesHttpHandler(w http.ResponseWriter, req *http.Request) { aereq := &raft.AppendEntriesRequest{} err := decodeJsonRequest(req, aereq) if err == nil { - log.Debugf("[recv] POST %s/log/append [%d]", s.url, len(aereq.Entries)) + log.Debugf("[recv] POST %s/log/append [%d]", ps.url, len(aereq.Entries)) - s.serverStats.RecvAppendReq(aereq.LeaderName, int(req.ContentLength)) + ps.serverStats.RecvAppendReq(aereq.LeaderName, int(req.ContentLength)) - if resp := s.raftServer.AppendEntries(aereq); resp != nil { + if resp := ps.raftServer.AppendEntries(aereq); resp != nil { w.WriteHeader(http.StatusOK) json.NewEncoder(w).Encode(resp) if !resp.Success { @@ -57,12 +57,12 @@ func (s *PeerServer) AppendEntriesHttpHandler(w http.ResponseWriter, req *http.R } // Response to recover from snapshot request -func (s *PeerServer) SnapshotHttpHandler(w http.ResponseWriter, req *http.Request) { +func (ps *PeerServer) SnapshotHttpHandler(w http.ResponseWriter, req *http.Request) { aereq := &raft.SnapshotRequest{} err := decodeJsonRequest(req, aereq) if err == nil { - log.Debugf("[recv] POST %s/snapshot/ ", s.url) - if resp := s.raftServer.RequestSnapshot(aereq); resp != nil { + log.Debugf("[recv] POST %s/snapshot/ ", ps.url) + if resp := ps.raftServer.RequestSnapshot(aereq); resp != nil { w.WriteHeader(http.StatusOK) json.NewEncoder(w).Encode(resp) return @@ -73,12 +73,12 @@ func (s *PeerServer) SnapshotHttpHandler(w http.ResponseWriter, req *http.Reques } // Response to recover from snapshot request -func (s *PeerServer) SnapshotRecoveryHttpHandler(w http.ResponseWriter, req *http.Request) { +func (ps *PeerServer) SnapshotRecoveryHttpHandler(w http.ResponseWriter, req *http.Request) { aereq := &raft.SnapshotRecoveryRequest{} err := decodeJsonRequest(req, aereq) if err == nil { - log.Debugf("[recv] POST %s/snapshotRecovery/ ", s.url) - if resp := s.raftServer.SnapshotRecoveryRequest(aereq); resp != nil { + log.Debugf("[recv] POST %s/snapshotRecovery/ ", ps.url) + if resp := ps.raftServer.SnapshotRecoveryRequest(aereq); resp != nil { w.WriteHeader(http.StatusOK) json.NewEncoder(w).Encode(resp) return @@ -89,20 +89,20 @@ func (s *PeerServer) SnapshotRecoveryHttpHandler(w http.ResponseWriter, req *htt } // Get the port that listening for etcd connecting of the server -func (s *PeerServer) EtcdURLHttpHandler(w http.ResponseWriter, req *http.Request) { - log.Debugf("[recv] Get %s/etcdURL/ ", s.url) +func (ps *PeerServer) EtcdURLHttpHandler(w http.ResponseWriter, req *http.Request) { + log.Debugf("[recv] Get %s/etcdURL/ ", ps.url) w.WriteHeader(http.StatusOK) - w.Write([]byte(s.server.URL())) + w.Write([]byte(ps.server.URL())) } // Response to the join request -func (s *PeerServer) JoinHttpHandler(w http.ResponseWriter, req *http.Request) { +func (ps *PeerServer) JoinHttpHandler(w http.ResponseWriter, req *http.Request) { command := &JoinCommand{} // Write CORS header. - if s.server.OriginAllowed("*") { + if ps.server.OriginAllowed("*") { w.Header().Add("Access-Control-Allow-Origin", "*") - } else if s.server.OriginAllowed(req.Header.Get("Origin")) { + } else if ps.server.OriginAllowed(req.Header.Get("Origin")) { w.Header().Add("Access-Control-Allow-Origin", req.Header.Get("Origin")) } @@ -113,7 +113,7 @@ func (s *PeerServer) JoinHttpHandler(w http.ResponseWriter, req *http.Request) { } log.Debugf("Receive Join Request from %s", command.Name) - err = s.dispatch(command, w, req) + err = ps.server.Dispatch(command, w, req) // Return status. if err != nil { @@ -127,7 +127,7 @@ func (s *PeerServer) JoinHttpHandler(w http.ResponseWriter, req *http.Request) { } // Response to remove request -func (s *PeerServer) RemoveHttpHandler(w http.ResponseWriter, req *http.Request) { +func (ps *PeerServer) RemoveHttpHandler(w http.ResponseWriter, req *http.Request) { if req.Method != "DELETE" { w.WriteHeader(http.StatusMethodNotAllowed) return @@ -140,19 +140,19 @@ func (s *PeerServer) RemoveHttpHandler(w http.ResponseWriter, req *http.Request) log.Debugf("[recv] Remove Request [%s]", command.Name) - s.dispatch(command, w, req) + ps.server.Dispatch(command, w, req) } // Response to the name request -func (s *PeerServer) NameHttpHandler(w http.ResponseWriter, req *http.Request) { - log.Debugf("[recv] Get %s/name/ ", s.url) +func (ps *PeerServer) NameHttpHandler(w http.ResponseWriter, req *http.Request) { + log.Debugf("[recv] Get %s/name/ ", ps.url) w.WriteHeader(http.StatusOK) - w.Write([]byte(s.name)) + w.Write([]byte(ps.name)) } // Response to the name request -func (s *PeerServer) RaftVersionHttpHandler(w http.ResponseWriter, req *http.Request) { - log.Debugf("[recv] Get %s/version/ ", s.url) +func (ps *PeerServer) RaftVersionHttpHandler(w http.ResponseWriter, req *http.Request) { + log.Debugf("[recv] Get %s/version/ ", ps.url) w.WriteHeader(http.StatusOK) w.Write([]byte(PeerVersion)) } diff --git a/server/server.go b/server/server.go index 16844d492d7..3d02f7584ab 100644 --- a/server/server.go +++ b/server/server.go @@ -1,6 +1,7 @@ package server import ( + "encoding/json" "fmt" "net/http" "net/url" @@ -167,7 +168,57 @@ func (s *Server) ListenAndServe() { } func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Request) error { - return s.peerServer.dispatch(c, w, req) + ps := s.peerServer + if ps.raftServer.State() == raft.Leader { + result, err := ps.raftServer.Do(c) + if err != nil { + return err + } + + if result == nil { + return etcdErr.NewError(300, "Empty result from raft", store.UndefIndex, store.UndefTerm) + } + + // response for raft related commands[join/remove] + if b, ok := result.([]byte); ok { + w.WriteHeader(http.StatusOK) + w.Write(b) + return nil + } + + var b []byte + if strings.HasPrefix(req.URL.Path, "/v1") { + b, _ = json.Marshal(result.(*store.Event).Response()) + w.WriteHeader(http.StatusOK) + } else { + e, _ := result.(*store.Event) + b, _ = json.Marshal(e) + w.WriteHeader(e.HttpStatusCode()) + } + + w.Write(b) + + return nil + + } else { + leader := ps.raftServer.Leader() + + // No leader available. + if leader == "" { + return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) + } + + var url string + switch c.(type) { + case *JoinCommand, *RemoveCommand: + url, _ = ps.registry.PeerURL(leader) + default: + url, _ = ps.registry.ClientURL(leader) + } + redirect(url, w, req) + + return nil + } } // Sets a comma-delimited list of origins that are allowed. diff --git a/store/event.go b/store/event.go index 92c0b96471a..136ca48b6f1 100644 --- a/store/event.go +++ b/store/event.go @@ -1,6 +1,7 @@ package store import ( + "net/http" "time" ) @@ -42,6 +43,18 @@ func newEvent(action string, key string, index uint64, term uint64) *Event { } } +func (e *Event) HttpStatusCode() int { + if e.Action == Create { + return http.StatusCreated + } + + if e.Action == Set && e.PrevValue == "" { + return http.StatusCreated + } + + return http.StatusOK +} + // Converts an event object into a response object. func (event *Event) Response() interface{} { if !event.Dir { From 5767b4e2455b4195b8b70986498ea0e43a02e46c Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 20 Oct 2013 11:10:24 -0700 Subject: [PATCH 133/247] use net/url pacakge --- server/util.go | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/server/util.go b/server/util.go index baa7c2845b3..7c992e873fa 100644 --- a/server/util.go +++ b/server/util.go @@ -5,6 +5,7 @@ import ( "fmt" "io" "net/http" + "net/url" "github.com/coreos/etcd/log" ) @@ -19,9 +20,13 @@ func decodeJsonRequest(req *http.Request, data interface{}) error { } func redirect(hostname string, w http.ResponseWriter, req *http.Request) { - path := req.URL.Path - query := req.URL.RawQuery - url := hostname + path + "?" + query - log.Debugf("Redirect to %s", url) - http.Redirect(w, req, url, http.StatusTemporaryRedirect) + originalURL, _ := url.Parse(req.URL.String()) + redirectURL, _ := url.Parse(hostname) + + // we need the original path and raw query + redirectURL.Path = originalURL.Path + redirectURL.RawQuery = originalURL.RawQuery + + log.Debugf("Redirect to %s", redirectURL.String()) + http.Redirect(w, req, redirectURL.String(), http.StatusTemporaryRedirect) } From 1119a86cc337e686514ea87d9a0c34beeb1dcabe Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 20 Oct 2013 11:25:24 -0700 Subject: [PATCH 134/247] fix add fragment --- server/util.go | 1 + 1 file changed, 1 insertion(+) diff --git a/server/util.go b/server/util.go index 7c992e873fa..3a898e53ee7 100644 --- a/server/util.go +++ b/server/util.go @@ -26,6 +26,7 @@ func redirect(hostname string, w http.ResponseWriter, req *http.Request) { // we need the original path and raw query redirectURL.Path = originalURL.Path redirectURL.RawQuery = originalURL.RawQuery + redirectURL.Fragment = originalURL.Fragment log.Debugf("Redirect to %s", redirectURL.String()) http.Redirect(w, req, redirectURL.String(), http.StatusTemporaryRedirect) From 2cafdcd657bbe974f8c583b385509b32a9229c69 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 20 Oct 2013 11:27:40 -0700 Subject: [PATCH 135/247] fix do not need to parse the original url --- server/util.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/util.go b/server/util.go index 3a898e53ee7..578c048cc8d 100644 --- a/server/util.go +++ b/server/util.go @@ -20,7 +20,7 @@ func decodeJsonRequest(req *http.Request, data interface{}) error { } func redirect(hostname string, w http.ResponseWriter, req *http.Request) { - originalURL, _ := url.Parse(req.URL.String()) + originalURL := req.URL redirectURL, _ := url.Parse(hostname) // we need the original path and raw query From 23119d1672e2d65a761b7e43517db433d91e8bea Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sun, 20 Oct 2013 13:39:29 -0600 Subject: [PATCH 136/247] Refactor store tests. --- store/stats_test.go | 221 ++++-------- store/store_test.go | 821 ++++++++++++++++++-------------------------- 2 files changed, 404 insertions(+), 638 deletions(-) diff --git a/store/stats_test.go b/store/stats_test.go index 41c032ea57d..b6247326450 100644 --- a/store/stats_test.go +++ b/store/stats_test.go @@ -1,165 +1,94 @@ package store import ( - "math/rand" "testing" "time" + + "github.com/stretchr/testify/assert" ) -func TestBasicStats(t *testing.T) { +// Ensure that a successful Get is recorded in the stats. +func TestStoreStatsGetSuccess(t *testing.T) { s := newStore() - keys := GenKeys(rand.Intn(100), 5) - - var i uint64 - var GetSuccess, GetFail, CreateSuccess, CreateFail, DeleteSuccess, DeleteFail uint64 - var UpdateSuccess, UpdateFail, CompareAndSwapSuccess, CompareAndSwapFail, watcher_number uint64 - - for _, k := range keys { - i++ - _, err := s.Create(k, "bar", false, time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1) - if err != nil { - CreateFail++ - } else { - CreateSuccess++ - } - } - - time.Sleep(time.Second * 3) - - for _, k := range keys { - _, err := s.Get(k, false, false, i, 1) - if err != nil { - GetFail++ - } else { - GetSuccess++ - } - } - - for _, k := range keys { - i++ - _, err := s.Update(k, "foo", time.Now().Add(time.Second*time.Duration(rand.Intn(6))), i, 1) - if err != nil { - UpdateFail++ - } else { - UpdateSuccess++ - } - } - - time.Sleep(time.Second * 3) - - for _, k := range keys { - _, err := s.Get(k, false, false, i, 1) - if err != nil { - GetFail++ - } else { - GetSuccess++ - } - } - - for _, k := range keys { - i++ - _, err := s.CompareAndSwap(k, "foo", 0, "bar", Permanent, i, 1) - if err != nil { - CompareAndSwapFail++ - } else { - CompareAndSwapSuccess++ - } - } - - for _, k := range keys { - s.Watch(k, false, 0, i, 1) - watcher_number++ - } - - for _, k := range keys { - _, err := s.Get(k, false, false, i, 1) - if err != nil { - GetFail++ - } else { - GetSuccess++ - } - } - - for _, k := range keys { - i++ - _, err := s.Delete(k, false, i, 1) - if err != nil { - DeleteFail++ - } else { - watcher_number-- - DeleteSuccess++ - } - } - - for _, k := range keys { - _, err := s.Get(k, false, false, i, 1) - if err != nil { - GetFail++ - } else { - GetSuccess++ - } - } - - if GetSuccess != s.Stats.GetSuccess { - t.Fatalf("GetSuccess [%d] != Stats.GetSuccess [%d]", GetSuccess, s.Stats.GetSuccess) - } - - if GetFail != s.Stats.GetFail { - t.Fatalf("GetFail [%d] != Stats.GetFail [%d]", GetFail, s.Stats.GetFail) - } - - if CreateSuccess != s.Stats.CreateSuccess { - t.Fatalf("CreateSuccess [%d] != Stats.CreateSuccess [%d]", CreateSuccess, s.Stats.CreateSuccess) - } - - if CreateFail != s.Stats.CreateFail { - t.Fatalf("CreateFail [%d] != Stats.CreateFail [%d]", CreateFail, s.Stats.CreateFail) - } - - if DeleteSuccess != s.Stats.DeleteSuccess { - t.Fatalf("DeleteSuccess [%d] != Stats.DeleteSuccess [%d]", DeleteSuccess, s.Stats.DeleteSuccess) - } - - if DeleteFail != s.Stats.DeleteFail { - t.Fatalf("DeleteFail [%d] != Stats.DeleteFail [%d]", DeleteFail, s.Stats.DeleteFail) - } + s.Create("/foo", "bar", false, Permanent, 3, 1) + s.Get("/foo", false, false, 3, 1) + assert.Equal(t, uint64(1), s.Stats.GetSuccess, "") +} - if UpdateSuccess != s.Stats.UpdateSuccess { - t.Fatalf("UpdateSuccess [%d] != Stats.UpdateSuccess [%d]", UpdateSuccess, s.Stats.UpdateSuccess) - } +// Ensure that a failed Get is recorded in the stats. +func TestStoreStatsGetFail(t *testing.T) { + s := newStore() + s.Create("/foo", "bar", false, Permanent, 3, 1) + s.Get("/no_such_key", false, false, 3, 1) + assert.Equal(t, uint64(1), s.Stats.GetFail, "") +} - if UpdateFail != s.Stats.UpdateFail { - t.Fatalf("UpdateFail [%d] != Stats.UpdateFail [%d]", UpdateFail, s.Stats.UpdateFail) - } +// Ensure that a successful Create is recorded in the stats. +func TestStoreStatsCreateSuccess(t *testing.T) { + s := newStore() + s.Create("/foo", "bar", false, Permanent, 3, 1) + assert.Equal(t, uint64(1), s.Stats.CreateSuccess, "") +} - if CompareAndSwapSuccess != s.Stats.CompareAndSwapSuccess { - t.Fatalf("TestAndSetSuccess [%d] != Stats.CompareAndSwapSuccess [%d]", CompareAndSwapSuccess, s.Stats.CompareAndSwapSuccess) - } +// Ensure that a failed Create is recorded in the stats. +func TestStoreStatsCreateFail(t *testing.T) { + s := newStore() + s.Create("/foo", "", false, Permanent, 3, 1) + s.Create("/foo", "bar", false, Permanent, 4, 1) + assert.Equal(t, uint64(1), s.Stats.CreateFail, "") +} - if CompareAndSwapFail != s.Stats.CompareAndSwapFail { - t.Fatalf("TestAndSetFail [%d] != Stats.TestAndSetFail [%d]", CompareAndSwapFail, s.Stats.CompareAndSwapFail) - } +// Ensure that a successful Update is recorded in the stats. +func TestStoreStatsUpdateSuccess(t *testing.T) { + s := newStore() + s.Create("/foo", "bar", false, Permanent, 3, 1) + s.Update("/foo", "baz", Permanent, 4, 1) + assert.Equal(t, uint64(1), s.Stats.UpdateSuccess, "") +} - s = newStore() - CreateSuccess = 0 - CreateFail = 0 +// Ensure that a failed Update is recorded in the stats. +func TestStoreStatsUpdateFail(t *testing.T) { + s := newStore() + s.Update("/foo", "bar", Permanent, 4, 1) + assert.Equal(t, uint64(1), s.Stats.UpdateFail, "") +} - for _, k := range keys { - i++ - _, err := s.Create(k, "bar", false, time.Now().Add(time.Second*3), i, 1) - if err != nil { - CreateFail++ - } else { - CreateSuccess++ - } - } +// Ensure that a successful CAS is recorded in the stats. +func TestStoreStatsCompareAndSwapSuccess(t *testing.T) { + s := newStore() + s.Create("/foo", "bar", false, Permanent, 3, 1) + s.CompareAndSwap("/foo", "bar", 0, "baz", Permanent, 4, 1) + assert.Equal(t, uint64(1), s.Stats.CompareAndSwapSuccess, "") +} - time.Sleep(6 * time.Second) +// Ensure that a failed CAS is recorded in the stats. +func TestStoreStatsCompareAndSwapFail(t *testing.T) { + s := newStore() + s.Create("/foo", "bar", false, Permanent, 3, 1) + s.CompareAndSwap("/foo", "wrong_value", 0, "baz", Permanent, 4, 1) + assert.Equal(t, uint64(1), s.Stats.CompareAndSwapFail, "") +} - ExpireCount := CreateSuccess +// Ensure that a successful Delete is recorded in the stats. +func TestStoreStatsDeleteSuccess(t *testing.T) { + s := newStore() + s.Create("/foo", "bar", false, Permanent, 3, 1) + s.Delete("/foo", false, 4, 1) + assert.Equal(t, uint64(1), s.Stats.DeleteSuccess, "") +} - if ExpireCount != s.Stats.ExpireCount { - t.Fatalf("ExpireCount [%d] != Stats.ExpireCount [%d]", ExpireCount, s.Stats.ExpireCount) - } +// Ensure that a failed Delete is recorded in the stats. +func TestStoreStatsDeleteFail(t *testing.T) { + s := newStore() + s.Delete("/foo", false, 4, 1) + assert.Equal(t, uint64(1), s.Stats.DeleteFail, "") +} +// Ensure that the number of expirations is recorded in the stats. +func TestStoreStatsExpireCount(t *testing.T) { + s := newStore() + s.Create("/foo", "bar", false, time.Now().Add(5 * time.Millisecond), 3, 1) + assert.Equal(t, uint64(0), s.Stats.ExpireCount, "") + time.Sleep(10 * time.Millisecond) + assert.Equal(t, uint64(1), s.Stats.ExpireCount, "") } diff --git a/store/store_test.go b/store/store_test.go index 20b1131c3d4..2328c4be940 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -1,561 +1,398 @@ package store import ( - "math/rand" - "strconv" "testing" "time" + + etcdErr "github.com/coreos/etcd/error" + "github.com/stretchr/testify/assert" ) -func TestCreateAndGet(t *testing.T) { +// Ensure that the store can retrieve an existing value. +func TestStoreGetValue(t *testing.T) { s := newStore() - - s.Create("/foobar", "bar", false, Permanent, 1, 1) - - // already exist, create should fail - _, err := s.Create("/foobar", "bar", false, Permanent, 1, 1) - - if err == nil { - t.Fatal("Create should fail") - } - - s.Delete("/foobar", true, 1, 1) - - s.Create("/foobar/foo", "bar", false, Permanent, 1, 1) - - // already exist, create should fail - _, err = s.Create("/foobar", "bar", false, Permanent, 1, 1) - - if err == nil { - t.Fatal("Create should fail") - } - - s.Delete("/foobar", true, 1, 1) - - // this should create successfully - createAndGet(s, "/foobar", t) - createAndGet(s, "/foo/bar", t) - createAndGet(s, "/foo/foo/bar", t) - - // meet file, create should fail - _, err = s.Create("/foo/bar/bar", "bar", false, Permanent, 2, 1) - - if err == nil { - t.Fatal("Create should fail") - } - - // create a directory - _, err = s.Create("/fooDir", "", false, Permanent, 3, 1) - - if err != nil { - t.Fatal("Cannot create /fooDir") - } - - e, err := s.Get("/fooDir", false, false, 3, 1) - - if err != nil || e.Dir != true { - t.Fatal("Cannot create /fooDir ") - } - - // create a file under directory - _, err = s.Create("/fooDir/bar", "bar", false, Permanent, 4, 1) - - if err != nil { - t.Fatal("Cannot create /fooDir/bar = bar") - } + s.Create("/foo", "bar", false, Permanent, 2, 1) + e, err := s.Get("/foo", false, false, 2, 1) + assert.Nil(t, err, "") + assert.Equal(t, e.Action, "get", "") + assert.Equal(t, e.Key, "/foo", "") + assert.Equal(t, e.Value, "bar", "") } -func TestUpdateFile(t *testing.T) { +// Ensure that the store can recrusively retrieve a directory listing. +// Note that hidden files should not be returned. +func TestStoreGetDirectory(t *testing.T) { s := newStore() - - _, err := s.Create("/foo/bar", "bar", false, Permanent, 1, 1) - - if err != nil { - t.Fatalf("cannot create %s=bar [%s]", "/foo/bar", err.Error()) - } - - _, err = s.Update("/foo/bar", "barbar", Permanent, 2, 1) - - if err != nil { - t.Fatalf("cannot update %s=barbar [%s]", "/foo/bar", err.Error()) - } - - e, err := s.Get("/foo/bar", false, false, 2, 1) - - if err != nil { - t.Fatalf("cannot get %s [%s]", "/foo/bar", err.Error()) - } - - if e.Value != "barbar" { - t.Fatalf("expect value of %s is barbar [%s]", "/foo/bar", e.Value) - } - - // create a directory, update its ttl, to see if it will be deleted - _, err = s.Create("/foo/foo", "", false, Permanent, 3, 1) - - if err != nil { - t.Fatalf("cannot create dir [%s] [%s]", "/foo/foo", err.Error()) - } - - _, err = s.Create("/foo/foo/foo1", "bar1", false, Permanent, 4, 1) - - if err != nil { - t.Fatal("cannot create [%s]", err.Error()) - } - - _, err = s.Create("/foo/foo/foo2", "", false, Permanent, 5, 1) - if err != nil { - t.Fatal("cannot create [%s]", err.Error()) - } - - _, err = s.Create("/foo/foo/foo2/boo", "boo1", false, Permanent, 6, 1) - if err != nil { - t.Fatal("cannot create [%s]", err.Error()) - } - - expire := time.Now().Add(time.Second * 2) - _, err = s.Update("/foo/foo", "", expire, 7, 1) - if err != nil { - t.Fatalf("cannot update dir [%s] [%s]", "/foo/foo", err.Error()) - } - - // sleep 50ms, it should still reach the node - time.Sleep(time.Microsecond * 50) - e, err = s.Get("/foo/foo", true, false, 7, 1) - - if err != nil || e.Key != "/foo/foo" { - t.Fatalf("cannot get dir before expiration [%s]", err.Error()) - } - - if e.KVPairs[0].Key != "/foo/foo/foo1" || e.KVPairs[0].Value != "bar1" { - t.Fatalf("cannot get sub node before expiration [%s]", err.Error()) - } - - if e.KVPairs[1].Key != "/foo/foo/foo2" || e.KVPairs[1].Dir != true { - t.Fatalf("cannot get sub dir before expiration [%s]", err.Error()) - } - - // wait for expiration - time.Sleep(time.Second * 3) - e, err = s.Get("/foo/foo", true, false, 7, 1) - - if err == nil { - t.Fatal("still can get dir after expiration [%s]") - } - - _, err = s.Get("/foo/foo/foo1", true, false, 7, 1) - if err == nil { - t.Fatal("still can get sub node after expiration [%s]") - } - - _, err = s.Get("/foo/foo/foo2", true, false, 7, 1) - if err == nil { - t.Fatal("still can get sub dir after expiration [%s]") - } - - _, err = s.Get("/foo/foo/foo2/boo", true, false, 7, 1) - if err == nil { - t.Fatalf("still can get sub node of sub dir after expiration [%s]", err.Error()) - } - + s.Create("/foo", "", false, Permanent, 2, 1) + s.Create("/foo/bar", "X", false, Permanent, 3, 1) + s.Create("/foo/_hidden", "*", false, Permanent, 4, 1) + s.Create("/foo/baz", "", false, Permanent, 5, 1) + s.Create("/foo/baz/bat", "Y", false, Permanent, 6, 1) + s.Create("/foo/baz/_hidden", "*", false, Permanent, 7, 1) + e, err := s.Get("/foo", true, false, 8, 1) + assert.Nil(t, err, "") + assert.Equal(t, e.Action, "get", "") + assert.Equal(t, e.Key, "/foo", "") + assert.Equal(t, len(e.KVPairs), 2, "") + assert.Equal(t, e.KVPairs[0].Key, "/foo/bar", "") + assert.Equal(t, e.KVPairs[0].Value, "X", "") + assert.Equal(t, e.KVPairs[0].Dir, false, "") + assert.Equal(t, e.KVPairs[1].Key, "/foo/baz", "") + assert.Equal(t, e.KVPairs[1].Dir, true, "") + assert.Equal(t, len(e.KVPairs[1].KVPairs), 1, "") + assert.Equal(t, e.KVPairs[1].KVPairs[0].Key, "/foo/baz/bat", "") + assert.Equal(t, e.KVPairs[1].KVPairs[0].Value, "Y", "") + assert.Equal(t, e.KVPairs[1].KVPairs[0].Dir, false, "") } -func TestListDirectory(t *testing.T) { +// Ensure that the store can retrieve a directory in sorted order. +func TestStoreGetSorted(t *testing.T) { s := newStore() + s.Create("/foo", "", false, Permanent, 2, 1) + s.Create("/foo/x", "0", false, Permanent, 3, 1) + s.Create("/foo/z", "0", false, Permanent, 4, 1) + s.Create("/foo/y", "", false, Permanent, 5, 1) + s.Create("/foo/y/a", "0", false, Permanent, 6, 1) + s.Create("/foo/y/b", "0", false, Permanent, 7, 1) + e, err := s.Get("/foo", true, true, 8, 1) + assert.Nil(t, err, "") + assert.Equal(t, e.KVPairs[0].Key, "/foo/x", "") + assert.Equal(t, e.KVPairs[1].Key, "/foo/y", "") + assert.Equal(t, e.KVPairs[1].KVPairs[0].Key, "/foo/y/a", "") + assert.Equal(t, e.KVPairs[1].KVPairs[1].Key, "/foo/y/b", "") + assert.Equal(t, e.KVPairs[2].Key, "/foo/z", "") +} - // create dir /foo - // set key-value /foo/foo=bar - s.Create("/foo/foo", "bar", false, Permanent, 1, 1) - - // create dir /foo/fooDir - // set key-value /foo/fooDir/foo=bar - s.Create("/foo/fooDir/foo", "bar", false, Permanent, 2, 1) - - e, err := s.Get("/foo", true, false, 2, 1) - - if err != nil { - t.Fatalf("%v", err) - } - - if len(e.KVPairs) != 2 { - t.Fatalf("wrong number of kv pairs [%d/2]", len(e.KVPairs)) - } - - if e.KVPairs[0].Key != "/foo/foo" || e.KVPairs[0].Value != "bar" { - t.Fatalf("wrong kv [/foo/foo/ / %s] -> [bar / %s]", e.KVPairs[0].Key, e.KVPairs[0].Value) - } - - if e.KVPairs[1].Key != "/foo/fooDir" || e.KVPairs[1].Dir != true { - t.Fatalf("wrong kv [/foo/fooDir/ / %s] -> [true / %v]", e.KVPairs[1].Key, e.KVPairs[1].Dir) - } - - if e.KVPairs[1].KVPairs[0].Key != "/foo/fooDir/foo" || e.KVPairs[1].KVPairs[0].Value != "bar" { - t.Fatalf("wrong kv [/foo/fooDir/foo / %s] -> [bar / %v]", e.KVPairs[1].KVPairs[0].Key, e.KVPairs[1].KVPairs[0].Value) - } - // test hidden node - - // create dir /foo/_hidden - // set key-value /foo/_hidden/foo -> bar - s.Create("/foo/_hidden/foo", "bar", false, Permanent, 3, 1) - - e, _ = s.Get("/foo", false, false, 2, 1) - if len(e.KVPairs) != 2 { - t.Fatalf("hidden node is not hidden! %s", e.KVPairs[2].Key) - } +// Ensure that the store can create a new key if it doesn't already exist. +func TestStoreCreateValue(t *testing.T) { + s := newStore() + e, err := s.Create("/foo", "bar", false, Permanent, 2, 1) + assert.Nil(t, err, "") + assert.Equal(t, e.Action, "create", "") + assert.Equal(t, e.Key, "/foo", "") + assert.False(t, e.Dir, "") + assert.Equal(t, e.PrevValue, "", "") + assert.Equal(t, e.Value, "bar", "") + assert.Nil(t, e.KVPairs, "") + assert.Nil(t, e.Expiration, "") + assert.Equal(t, e.TTL, 0, "") + assert.Equal(t, e.Index, uint64(2), "") + assert.Equal(t, e.Term, uint64(1), "") } -func TestRemove(t *testing.T) { +// Ensure that the store can create a new directory if it doesn't already exist. +func TestStoreCreateDirectory(t *testing.T) { s := newStore() - - s.Create("/foo", "bar", false, Permanent, 1, 1) - _, err := s.Delete("/foo", false, 1, 1) - - if err != nil { - t.Fatalf("cannot delete %s [%s]", "/foo", err.Error()) - } - - _, err = s.Get("/foo", false, false, 1, 1) - - if err == nil || err.Error() != "Key Not Found" { - t.Fatalf("can get the node after deletion") - } - - s.Create("/foo/bar", "bar", false, Permanent, 1, 1) - s.Create("/foo/car", "car", false, Permanent, 1, 1) - s.Create("/foo/dar/dar", "dar", false, Permanent, 1, 1) - - _, err = s.Delete("/foo", false, 1, 1) - - if err == nil { - t.Fatalf("should not be able to delete a directory without recursive") - } - - _, err = s.Delete("/foo", true, 1, 1) - - if err != nil { - t.Fatalf("cannot delete %s [%s]", "/foo", err.Error()) - } - - _, err = s.Get("/foo", false, false, 1, 1) - - if err == nil || err.Error() != "Key Not Found" { - t.Fatalf("can get the node after deletion ") - } + e, err := s.Create("/foo", "", false, Permanent, 2, 1) + assert.Nil(t, err, "") + assert.Equal(t, e.Action, "create", "") + assert.Equal(t, e.Key, "/foo", "") + assert.True(t, e.Dir, "") } -func TestExpire(t *testing.T) { +// Ensure that the store fails to create a key if it already exists. +func TestStoreCreateFailsIfExists(t *testing.T) { s := newStore() - - expire := time.Now().Add(time.Second) - - s.Create("/foo", "bar", false, expire, 1, 1) - - _, err := s.Get("/foo", false, false, 1, 1) - - if err != nil { - t.Fatalf("can not get the node") - } - - time.Sleep(time.Second * 2) - - _, err = s.Get("/foo", false, false, 1, 1) - - if err == nil { - t.Fatalf("can get the node after expiration time") - } - - // test if we can reach the node before expiration - expire = time.Now().Add(time.Second) - s.Create("/foo", "bar", false, expire, 1, 1) - - time.Sleep(time.Millisecond * 50) - _, err = s.Get("/foo", false, false, 1, 1) - - if err != nil { - t.Fatalf("cannot get the node before expiration", err.Error()) - } - - expire = time.Now().Add(time.Second) - - s.Create("/foo", "bar", false, expire, 1, 1) - _, err = s.Delete("/foo", false, 1, 1) - - if err != nil { - t.Fatalf("cannot delete the node before expiration", err.Error()) - } + s.Create("/foo", "", false, Permanent, 2, 1) + e, _err := s.Create("/foo", "", false, Permanent, 3, 1) + err := _err.(*etcdErr.Error) + assert.Equal(t, err.ErrorCode, etcdErr.EcodeNodeExist, "") + assert.Equal(t, err.Message, "Already exists", "") + assert.Equal(t, err.Cause, "/foo", "") + assert.Equal(t, err.Index, uint64(3), "") + assert.Equal(t, err.Term, uint64(1), "") + assert.Nil(t, e, 0, "") } -func TestCompareAndSwap(t *testing.T) { // TODO prevValue == nil ? +// Ensure that the store can update a key if it already exists. +func TestStoreUpdateValue(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 1, 1) - - // test on wrong previous value - _, err := s.CompareAndSwap("/foo", "barbar", 0, "car", Permanent, 2, 1) - if err == nil { - t.Fatal("test and set should fail barbar != bar") - } - - // test on value - e, err := s.CompareAndSwap("/foo", "bar", 0, "car", Permanent, 3, 1) - - if err != nil { - t.Fatal("test and set should succeed bar == bar") - } - - if e.PrevValue != "bar" || e.Value != "car" { - t.Fatalf("[%v/%v] [%v/%v]", e.PrevValue, "bar", e.Value, "car") - } - - // test on index - e, err = s.CompareAndSwap("/foo", "", 3, "bar", Permanent, 4, 1) - - if err != nil { - t.Fatal("test and set should succeed index 3 == 3") - } - - if e.PrevValue != "car" || e.Value != "bar" { - t.Fatalf("[%v/%v] [%v/%v]", e.PrevValue, "car", e.Value, "bar") - } + s.Create("/foo", "bar", false, Permanent, 2, 1) + e, err := s.Update("/foo", "baz", Permanent, 3, 1) + assert.Nil(t, err, "") + assert.Equal(t, e.Action, "update", "") + assert.Equal(t, e.Key, "/foo", "") + assert.False(t, e.Dir, "") + assert.Equal(t, e.PrevValue, "bar", "") + assert.Equal(t, e.Value, "baz", "") + assert.Equal(t, e.TTL, 0, "") + assert.Equal(t, e.Index, uint64(3), "") + assert.Equal(t, e.Term, uint64(1), "") + e, _ = s.Get("/foo", false, false, 3, 1) + assert.Equal(t, e.Value, "baz", "") } -func TestWatch(t *testing.T) { +// Ensure that the store cannot update a directory. +func TestStoreUpdateFailsIfDirectory(t *testing.T) { s := newStore() - // watch at a deeper path - c, _ := s.Watch("/foo/foo/foo", false, 0, 0, 1) - s.Create("/foo/foo/foo", "bar", false, Permanent, 1, 1) - - e := nonblockingRetrive(c) - if e.Key != "/foo/foo/foo" || e.Action != Create { - t.Fatal("watch for Create node fails ", e) - } - - c, _ = s.Watch("/foo/foo/foo", false, 0, 1, 1) - s.Update("/foo/foo/foo", "car", Permanent, 2, 1) - e = nonblockingRetrive(c) - if e.Key != "/foo/foo/foo" || e.Action != Update { - t.Fatal("watch for Update node fails ", e) - } - - c, _ = s.Watch("/foo/foo/foo", false, 0, 2, 1) - s.CompareAndSwap("/foo/foo/foo", "car", 0, "bar", Permanent, 3, 1) - e = nonblockingRetrive(c) - if e.Key != "/foo/foo/foo" || e.Action != CompareAndSwap { - t.Fatal("watch for CompareAndSwap node fails") - } - - c, _ = s.Watch("/foo/foo/foo", false, 0, 3, 1) - s.Delete("/foo", true, 4, 1) //recursively delete - e = nonblockingRetrive(c) - if e.Key != "/foo" || e.Action != Delete { - t.Fatal("watch for Delete node fails ", e) - } - - // watch at a prefix - c, _ = s.Watch("/foo", true, 0, 4, 1) - s.Create("/foo/foo/boo", "bar", false, Permanent, 5, 1) - e = nonblockingRetrive(c) - if e.Key != "/foo/foo/boo" || e.Action != Create { - t.Fatal("watch for Create subdirectory fails") - } - - c, _ = s.Watch("/foo", true, 0, 5, 1) - s.Update("/foo/foo/boo", "foo", Permanent, 6, 1) - e = nonblockingRetrive(c) - if e.Key != "/foo/foo/boo" || e.Action != Update { - t.Fatal("watch for Update subdirectory fails") - } - - c, _ = s.Watch("/foo", true, 0, 6, 1) - s.CompareAndSwap("/foo/foo/boo", "foo", 0, "bar", Permanent, 7, 1) - e = nonblockingRetrive(c) - if e.Key != "/foo/foo/boo" || e.Action != CompareAndSwap { - t.Fatal("watch for CompareAndSwap subdirectory fails") - } - - c, _ = s.Watch("/foo", true, 0, 7, 1) - s.Delete("/foo/foo/boo", false, 8, 1) - e = nonblockingRetrive(c) - if e == nil || e.Key != "/foo/foo/boo" || e.Action != Delete { - t.Fatal("watch for Delete subdirectory fails") - } - - // watch expire - s.Create("/foo/foo/boo", "foo", false, time.Now().Add(time.Second*1), 9, 1) - c, _ = s.Watch("/foo", true, 0, 9, 1) - time.Sleep(time.Second * 2) - e = nonblockingRetrive(c) - if e.Key != "/foo/foo/boo" || e.Action != Expire || e.Index != 9 { - t.Fatal("watch for Expiration of Create() subdirectory fails ", e) - } - - s.Create("/foo/foo/boo", "foo", false, Permanent, 10, 1) - s.Update("/foo/foo/boo", "bar", time.Now().Add(time.Second*1), 11, 1) - c, _ = s.Watch("/foo", true, 0, 11, 1) - time.Sleep(time.Second * 2) - e = nonblockingRetrive(c) - if e.Key != "/foo/foo/boo" || e.Action != Expire || e.Index != 11 { - t.Fatal("watch for Expiration of Update() subdirectory fails ", e) - } - - s.Create("/foo/foo/boo", "foo", false, Permanent, 12, 1) - s.CompareAndSwap("/foo/foo/boo", "foo", 0, "bar", time.Now().Add(time.Second*1), 13, 1) - c, _ = s.Watch("/foo", true, 0, 13, 1) - time.Sleep(time.Second * 2) - e = nonblockingRetrive(c) - if e.Key != "/foo/foo/boo" || e.Action != Expire || e.Index != 13 { - t.Fatal("watch for Expiration of CompareAndSwap() subdirectory fails ", e) - } + s.Create("/foo", "", false, Permanent, 2, 1) + e, _err := s.Update("/foo", "baz", Permanent, 3, 1) + err := _err.(*etcdErr.Error) + assert.Equal(t, err.ErrorCode, etcdErr.EcodeNotFile, "") + assert.Equal(t, err.Message, "Not A File", "") + assert.Equal(t, err.Cause, "/foo", "") + assert.Nil(t, e, "") } -func TestSort(t *testing.T) { +// Ensure that the store can update the TTL on a value. +func TestStoreUpdateValueTTL(t *testing.T) { s := newStore() + s.Create("/foo", "bar", false, Permanent, 2, 1) + _, err := s.Update("/foo", "baz", time.Now().Add(1 * time.Millisecond), 3, 1) + e, _ := s.Get("/foo", false, false, 3, 1) + assert.Equal(t, e.Value, "baz", "") + + time.Sleep(2 * time.Millisecond) + e, err = s.Get("/foo", false, false, 3, 1) + assert.Nil(t, e, "") + assert.Equal(t, err.(*etcdErr.Error).ErrorCode, etcdErr.EcodeKeyNotFound, "") +} - // simulating random creation - keys := GenKeys(80, 4) - - i := uint64(1) - for _, k := range keys { - _, err := s.Create(k, "bar", false, Permanent, i, 1) - if err != nil { - panic(err) - } else { - i++ - } - } - - e, err := s.Get("/foo", true, true, i, 1) - if err != nil { - t.Fatalf("get dir nodes failed [%s]", err.Error()) - } +// Ensure that the store can update the TTL on a directory. +func TestStoreUpdateDirTTL(t *testing.T) { + s := newStore() + s.Create("/foo", "", false, Permanent, 2, 1) + s.Create("/foo/bar", "baz", false, Permanent, 3, 1) + _, err := s.Update("/foo", "", time.Now().Add(1 * time.Millisecond), 3, 1) + e, _ := s.Get("/foo/bar", false, false, 3, 1) + assert.Equal(t, e.Value, "baz", "") + + time.Sleep(2 * time.Millisecond) + e, err = s.Get("/foo/bar", false, false, 3, 1) + assert.Nil(t, e, "") + assert.Equal(t, err.(*etcdErr.Error).ErrorCode, etcdErr.EcodeKeyNotFound, "") +} - for i, k := range e.KVPairs[:len(e.KVPairs)-1] { +// Ensure that the store can delete a value. +func TestStoreDeleteValue(t *testing.T) { + s := newStore() + s.Create("/foo", "bar", false, Permanent, 2, 1) + e, err := s.Delete("/foo", false, 3, 1) + assert.Nil(t, err, "") + assert.Equal(t, e.Action, "delete", "") +} - if k.Key >= e.KVPairs[i+1].Key { - t.Fatalf("sort failed, [%s] should be placed after [%s]", k.Key, e.KVPairs[i+1].Key) - } +// Ensure that the store can delete a directory if recursive is specified. +func TestStoreDeleteDiretory(t *testing.T) { + s := newStore() + s.Create("/foo", "", false, Permanent, 2, 1) + e, err := s.Delete("/foo", true, 3, 1) + assert.Nil(t, err, "") + assert.Equal(t, e.Action, "delete", "") +} - if k.Dir { - recursiveTestSort(k, t) - } +// Ensure that the store cannot delete a directory if recursive is not specified. +func TestStoreDeleteDiretoryFailsIfNonRecursive(t *testing.T) { + s := newStore() + s.Create("/foo", "", false, Permanent, 2, 1) + e, _err := s.Delete("/foo", false, 3, 1) + err := _err.(*etcdErr.Error) + assert.Equal(t, err.ErrorCode, etcdErr.EcodeNotFile, "") + assert.Equal(t, err.Message, "Not A File", "") + assert.Nil(t, e, "") +} - } - if k := e.KVPairs[len(e.KVPairs)-1]; k.Dir { - recursiveTestSort(k, t) - } +// Ensure that the store can conditionally update a key if it has a previous value. +func TestStoreCompareAndSwapPrevValue(t *testing.T) { + s := newStore() + s.Create("/foo", "bar", false, Permanent, 2, 1) + e, err := s.CompareAndSwap("/foo", "bar", 0, "baz", Permanent, 3, 1) + assert.Nil(t, err, "") + assert.Equal(t, e.Action, "compareAndSwap", "") + assert.Equal(t, e.PrevValue, "bar", "") + assert.Equal(t, e.Value, "baz", "") + e, _ = s.Get("/foo", false, false, 3, 1) + assert.Equal(t, e.Value, "baz", "") } -func TestSaveAndRecover(t *testing.T) { +// Ensure that the store cannot conditionally update a key if it has the wrong previous value. +func TestStoreCompareAndSwapPrevValueFailsIfNotMatch(t *testing.T) { s := newStore() + s.Create("/foo", "bar", false, Permanent, 2, 1) + e, _err := s.CompareAndSwap("/foo", "wrong_value", 0, "baz", Permanent, 3, 1) + err := _err.(*etcdErr.Error) + assert.Equal(t, err.ErrorCode, etcdErr.EcodeTestFailed, "") + assert.Equal(t, err.Message, "Test Failed", "") + assert.Nil(t, e, "") + e, _ = s.Get("/foo", false, false, 3, 1) + assert.Equal(t, e.Value, "bar", "") +} - // simulating random creation - keys := GenKeys(8, 4) - - i := uint64(1) - for _, k := range keys { - _, err := s.Create(k, "bar", false, Permanent, i, 1) - if err != nil { - panic(err) - } else { - i++ - } - } +// Ensure that the store can conditionally update a key if it has a previous index. +func TestStoreCompareAndSwapPrevIndex(t *testing.T) { + s := newStore() + s.Create("/foo", "bar", false, Permanent, 2, 1) + e, err := s.CompareAndSwap("/foo", "", 2, "baz", Permanent, 3, 1) + assert.Nil(t, err, "") + assert.Equal(t, e.Action, "compareAndSwap", "") + assert.Equal(t, e.PrevValue, "bar", "") + assert.Equal(t, e.Value, "baz", "") + e, _ = s.Get("/foo", false, false, 3, 1) + assert.Equal(t, e.Value, "baz", "") +} - // create a node with expiration - // test if we can reach the node before expiration +// Ensure that the store cannot conditionally update a key if it has the wrong previous index. +func TestStoreCompareAndSwapPrevIndexFailsIfNotMatch(t *testing.T) { + s := newStore() + s.Create("/foo", "bar", false, Permanent, 2, 1) + e, _err := s.CompareAndSwap("/foo", "", 100, "baz", Permanent, 3, 1) + err := _err.(*etcdErr.Error) + assert.Equal(t, err.ErrorCode, etcdErr.EcodeTestFailed, "") + assert.Equal(t, err.Message, "Test Failed", "") + assert.Nil(t, e, "") + e, _ = s.Get("/foo", false, false, 3, 1) + assert.Equal(t, e.Value, "bar", "") +} - expire := time.Now().Add(time.Second) - s.Create("/foo/foo", "bar", false, expire, 1, 1) - b, err := s.Save() +// Ensure that the store can watch for key creation. +func TestStoreWatchCreate(t *testing.T) { + s := newStore() + c, _ := s.Watch("/foo", false, 0, 0, 1) + s.Create("/foo", "bar", false, Permanent, 2, 1) + e := nbselect(c) + assert.Equal(t, e.Action, "create", "") + assert.Equal(t, e.Key, "/foo", "") + e = nbselect(c) + assert.Nil(t, e, "") +} - cloneFs := newStore() - time.Sleep(2 * time.Second) +// Ensure that the store can watch for recursive key creation. +func TestStoreWatchRecursiveCreate(t *testing.T) { + s := newStore() + c, _ := s.Watch("/foo", true, 0, 0, 1) + s.Create("/foo/bar", "baz", false, Permanent, 2, 1) + e := nbselect(c) + assert.Equal(t, e.Action, "create", "") + assert.Equal(t, e.Key, "/foo/bar", "") +} - cloneFs.Recovery(b) +// Ensure that the store can watch for key updates. +func TestStoreWatchUpdate(t *testing.T) { + s := newStore() + s.Create("/foo", "bar", false, Permanent, 2, 1) + c, _ := s.Watch("/foo", false, 0, 0, 1) + s.Update("/foo", "baz", Permanent, 3, 1) + e := nbselect(c) + assert.Equal(t, e.Action, "update", "") + assert.Equal(t, e.Key, "/foo", "") +} - for i, k := range keys { - _, err := cloneFs.Get(k, false, false, uint64(i), 1) - if err != nil { - panic(err) - } - } +// Ensure that the store can watch for recursive key updates. +func TestStoreWatchRecursiveUpdate(t *testing.T) { + s := newStore() + s.Create("/foo/bar", "baz", false, Permanent, 2, 1) + c, _ := s.Watch("/foo", true, 0, 0, 1) + s.Update("/foo/bar", "baz", Permanent, 3, 1) + e := nbselect(c) + assert.Equal(t, e.Action, "update", "") + assert.Equal(t, e.Key, "/foo/bar", "") +} - // lock to avoid racing with Expire() - s.worldLock.RLock() - defer s.worldLock.RUnlock() +// Ensure that the store can watch for key deletions. +func TestStoreWatchDelete(t *testing.T) { + s := newStore() + s.Create("/foo", "bar", false, Permanent, 2, 1) + c, _ := s.Watch("/foo", false, 0, 0, 1) + s.Delete("/foo", false, 3, 1) + e := nbselect(c) + assert.Equal(t, e.Action, "delete", "") + assert.Equal(t, e.Key, "/foo", "") +} - if s.WatcherHub.EventHistory.StartIndex != cloneFs.WatcherHub.EventHistory.StartIndex { - t.Fatalf("Error recovered event history start index[%v/%v]", - s.WatcherHub.EventHistory.StartIndex, cloneFs.WatcherHub.EventHistory.StartIndex) - } +// Ensure that the store can watch for recursive key deletions. +func TestStoreWatchRecursiveDelete(t *testing.T) { + s := newStore() + s.Create("/foo/bar", "baz", false, Permanent, 2, 1) + c, _ := s.Watch("/foo", true, 0, 0, 1) + s.Delete("/foo/bar", false, 3, 1) + e := nbselect(c) + assert.Equal(t, e.Action, "delete", "") + assert.Equal(t, e.Key, "/foo/bar", "") +} - for i = 0; int(i) < cloneFs.WatcherHub.EventHistory.Queue.Size; i++ { - if s.WatcherHub.EventHistory.Queue.Events[i].Key != - cloneFs.WatcherHub.EventHistory.Queue.Events[i].Key { - t.Fatal("Error recovered event history") - } - } +// Ensure that the store can watch for CAS updates. +func TestStoreWatchCompareAndSwap(t *testing.T) { + s := newStore() + s.Create("/foo", "bar", false, Permanent, 2, 1) + c, _ := s.Watch("/foo", false, 0, 0, 1) + s.CompareAndSwap("/foo", "bar", 0, "baz", Permanent, 3, 1) + e := nbselect(c) + assert.Equal(t, e.Action, "compareAndSwap", "") + assert.Equal(t, e.Key, "/foo", "") +} - _, err = s.Get("/foo/foo", false, false, 1, 1) +// Ensure that the store can watch for recursive CAS updates. +func TestStoreWatchRecursiveCompareAndSwap(t *testing.T) { + s := newStore() + s.Create("/foo/bar", "baz", false, Permanent, 2, 1) + c, _ := s.Watch("/foo", true, 0, 0, 1) + s.CompareAndSwap("/foo/bar", "baz", 0, "bat", Permanent, 3, 1) + e := nbselect(c) + assert.Equal(t, e.Action, "compareAndSwap", "") + assert.Equal(t, e.Key, "/foo/bar", "") +} - if err == nil || err.Error() != "Key Not Found" { - t.Fatalf("can get the node after deletion ") - } +// Ensure that the store can watch for key expiration. +func TestStoreWatchExpire(t *testing.T) { + s := newStore() + s.Create("/foo", "bar", false, time.Now().Add(1 * time.Millisecond), 2, 1) + c, _ := s.Watch("/foo", false, 0, 0, 1) + e := nbselect(c) + assert.Nil(t, e, "") + time.Sleep(2 * time.Millisecond) + e = nbselect(c) + assert.Equal(t, e.Action, "expire", "") + assert.Equal(t, e.Key, "/foo", "") } -// GenKeys randomly generate num of keys with max depth -func GenKeys(num int, depth int) []string { - rand.Seed(time.Now().UnixNano()) - keys := make([]string, num) - for i := 0; i < num; i++ { +// Ensure that the store can recover from a previously saved state. +func TestStoreRecover(t *testing.T) { + s := newStore() + s.Create("/foo", "", false, Permanent, 2, 1) + s.Create("/foo/x", "bar", false, Permanent, 3, 1) + s.Create("/foo/y", "baz", false, Permanent, 4, 1) + b, err := s.Save() - keys[i] = "/foo" - depth := rand.Intn(depth) + 1 + s2 := newStore() + s2.Recovery(b) - for j := 0; j < depth; j++ { - keys[i] += "/" + strconv.Itoa(rand.Int()) - } - } + e, err := s.Get("/foo/x", false, false, 4, 1) + assert.Nil(t, err, "") + assert.Equal(t, e.Value, "bar", "") - return keys + e, err = s.Get("/foo/y", false, false, 4, 1) + assert.Nil(t, err, "") + assert.Equal(t, e.Value, "baz", "") } -func createAndGet(s *store, path string, t *testing.T) { - _, err := s.Create(path, "bar", false, Permanent, 1, 1) +// Ensure that the store can recover from a previously saved state that includes an expiring key. +func TestStoreRecoverWithExpiration(t *testing.T) { + s := newStore() + s.Create("/foo", "", false, Permanent, 2, 1) + s.Create("/foo/x", "bar", false, Permanent, 3, 1) + s.Create("/foo/y", "baz", false, time.Now().Add(5 * time.Millisecond), 4, 1) + b, err := s.Save() - if err != nil { - t.Fatalf("cannot create %s=bar [%s]", path, err.Error()) - } + time.Sleep(10 * time.Millisecond) - e, err := s.Get(path, false, false, 1, 1) + s2 := newStore() + s2.Recovery(b) - if err != nil { - t.Fatalf("cannot get %s [%s]", path, err.Error()) - } + e, err := s.Get("/foo/x", false, false, 4, 1) + assert.Nil(t, err, "") + assert.Equal(t, e.Value, "bar", "") - if e.Value != "bar" { - t.Fatalf("expect value of %s is bar [%s]", path, e.Value) - } + e, err = s.Get("/foo/y", false, false, 4, 1) + assert.NotNil(t, err, "") + assert.Nil(t, e, "") } -func recursiveTestSort(k KeyValuePair, t *testing.T) { - for i, v := range k.KVPairs[:len(k.KVPairs)-1] { - if v.Key >= k.KVPairs[i+1].Key { - t.Fatalf("sort failed, [%s] should be placed after [%s]", v.Key, k.KVPairs[i+1].Key) - } - - if v.Dir { - recursiveTestSort(v, t) - } - } - if v := k.KVPairs[len(k.KVPairs)-1]; v.Dir { - recursiveTestSort(v, t) - } -} -func nonblockingRetrive(c <-chan *Event) *Event { +// Performs a non-blocking select on an event channel. +func nbselect(c <-chan *Event) *Event { select { case e := <-c: return e From 94accfcb232188dc0628130f0587e12ad870d79e Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 21 Oct 2013 12:35:02 -0700 Subject: [PATCH 137/247] refactor remove extra print parameter --- server/registry.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/registry.go b/server/registry.go index fa63b50274a..75b0e98c969 100644 --- a/server/registry.go +++ b/server/registry.go @@ -46,7 +46,7 @@ func (r *Registry) Register(name string, peerVersion string, peerURL string, url key := path.Join(RegistryKey, name) value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", peerURL, url, peerVersion) _, err := r.store.Create(key, value, false, store.Permanent, commitIndex, term) - log.Debugf("Register: %s (%v)", name, err) + log.Debugf("Register: %s", name) return err } @@ -60,7 +60,7 @@ func (r *Registry) Unregister(name string, commitIndex uint64, term uint64) erro // Remove the key from the store. _, err := r.store.Delete(path.Join(RegistryKey, name), false, commitIndex, term) - log.Debugf("Unregister: %s (%v)", name, err) + log.Debugf("Unregister: %s", name) return err } From 6803d077b99b270488932d7df79eeb516f0c031a Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 21 Oct 2013 12:37:22 -0700 Subject: [PATCH 138/247] fix store pacakge pass go vet --- store/compare_and_swap_command.go | 4 ++-- store/stats.go | 2 +- store/watcher_test.go | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/store/compare_and_swap_command.go b/store/compare_and_swap_command.go index c4bfee569bf..8ee119f36fe 100644 --- a/store/compare_and_swap_command.go +++ b/store/compare_and_swap_command.go @@ -16,8 +16,8 @@ type CompareAndSwapCommand struct { Key string `json:"key"` Value string `json:"value"` ExpireTime time.Time `json:"expireTime"` - PrevValue string `json: prevValue` - PrevIndex uint64 `json: prevIndex` + PrevValue string `json:"prevValue"` + PrevIndex uint64 `json:"prevIndex"` } // The name of the testAndSet command in the log diff --git a/store/stats.go b/store/stats.go index 5f4d26d91a2..c18f5cbf094 100644 --- a/store/stats.go +++ b/store/stats.go @@ -41,7 +41,7 @@ type Stats struct { // Number of create requests CreateSuccess uint64 `json:"createSuccess"` - CreateFail uint64 `json:createFail` + CreateFail uint64 `json:"createFail"` // Number of testAndSet requests CompareAndSwapSuccess uint64 `json:"compareAndSwapSuccess"` diff --git a/store/watcher_test.go b/store/watcher_test.go index 61da92f7c63..c3da475fca7 100644 --- a/store/watcher_test.go +++ b/store/watcher_test.go @@ -9,7 +9,7 @@ func TestWatcher(t *testing.T) { wh := s.WatcherHub c, err := wh.watch("/foo", true, 1) if err != nil { - t.Fatal("%v", err) + t.Fatalf("%v", err) } select { From 48b3e7f2abc1cf339fff318021cfbb30bfffa884 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 21 Oct 2013 16:00:20 -0600 Subject: [PATCH 139/247] Add basic config. --- Documentation/external-documentation.md | 105 ++++++++++++++++++++++++ server/config.go | 45 ++++++++++ server/config_test.go | 66 +++++++++++++++ server/tls_config.go | 1 + server/tls_info.go | 1 + server/util.go | 13 +++ 6 files changed, 231 insertions(+) create mode 100644 Documentation/external-documentation.md create mode 100644 server/config.go create mode 100644 server/config_test.go diff --git a/Documentation/external-documentation.md b/Documentation/external-documentation.md new file mode 100644 index 00000000000..1051e867407 --- /dev/null +++ b/Documentation/external-documentation.md @@ -0,0 +1,105 @@ +# Etcd Configuration + +Configuration options can be set in three places: + + 1. Command line flags + 2. Environment variables + 3. Configuration file + +Options set on the command line take precedence over all other sources. +Options set in environment variables take precedence over options set in +configuration files. + +## Command Line Flags + +### Required + +* `-n` - The node name. Defaults to `default-name`. + +### Optional + +* `-c` - The advertised public hostname:port for client communication. Defaults to `127.0.0.1:4001`. +* `-cl` - The listening hostname for client communication. Defaults to advertised ip. +* `-C` - A comma separated list of machines in the cluster (i.e `"203.0.113.101:7001,203.0.113.102:7001"`). +* `-CF` - The file path containing a comma separated list of machines in the cluster. +* `-clientCAFile` - The path of the client CAFile. Enables client cert authentication when present. +* `-clientCert` - The cert file of the client. +* `-clientKey` - The key file of the client. +* `-configfile` - The path of the etcd config file. Defaults to `/etc/etcd/etcd.toml`. +* `-cors` - A comma separated white list of origins for cross-origin resource sharing. +* `-cpuprofile` - The path to a file to output cpu profile data. Enables cpu profiling when present. +* `-d` - The directory to store log and snapshot. Defaults to the current working directory. +* `-m` - The max size of result buffer. Defaults to `1024`. +* `-maxsize` - The max size of the cluster. Defaults to `9`. +* `-r` - The max retry attempts when trying to join a cluster. Defaults to `3`. +* `-s` - The advertised public hostname:port for server communication. Defaults to `127.0.0.1:7001`. +* `-sl` - The listening hostname for server communication. Defaults to advertised ip. +* `-serverCAFile` - The path of the CAFile. Enables client/peer cert authentication when present. +* `-serverCert` - The cert file of the server. +* `-serverKey` - The key file of the server. +* `-snapshot` - Open or close snapshot. Defaults to `false`. +* `-v` - Enable verbose logging. Defaults to `false`. +* `-vv` - Enable very verbose logging. Defaults to `false`. +* `-version` - Print the version and exit. +* `-w` - The hostname:port of web interface. + +## Configuration File + +The etcd configuration file is written in [TOML](https://github.com/mojombo/toml) +and read from `/etc/etcd/etcd.toml` by default. + +```TOML +[etcd] + advertised_url = "127.0.0.1:4001" + ca_file = "" + cert_file = "" + cors = [] + cpu_profile_file = "" + datadir = "." + key_file = "" + listen_host = "127.0.0.1:4001" + machines = [] + machines_file = "" + max_cluster_size = 9 + max_result_buffer = 1024 + max_retry_attempts = 3 + name = "default-name" + snapshot = false + verbose = false + very_verbose = false + web_url = "" + +[raft] + advertised_url = "127.0.0.1:7001" + ca_file = "" + cert_file = "" + key_file = "" + listen_host = "127.0.0.1:7001" +``` + +## Environment Variables + + * `ETCD_ADVERTISED_URL` + * `ETCD_CA_FILE` + * `ETCD_CERT_FILE` + * `ETCD_CORS` + * `ETCD_CONFIG_FILE` + * `ETCD_CPU_PROFILE_FILE` + * `ETCD_DATADIR` + * `ETCD_KEY_FILE` + * `ETCD_LISTEN_HOST` + * `ETCD_MACHINES` + * `ETCD_MACHINES_FILE` + * `ETCD_MAX_RETRY_ATTEMPTS` + * `ETCD_MAX_CLUSTER_SIZE` + * `ETCD_MAX_RESULTS_BUFFER` + * `ETCD_NAME` + * `ETCD_SNAPSHOT` + * `ETCD_VERBOSE` + * `ETCD_VERY_VERBOSE` + * `ETCD_WEB_URL` + * `ETCD_RAFT_ADVERTISED_URL` + * `ETCD_RAFT_CA_FILE` + * `ETCD_RAFT_CERT_FILE` + * `ETCD_RAFT_KEY_FILE` + * `ETCD_RAFT_LISTEN_HOST` diff --git a/server/config.go b/server/config.go new file mode 100644 index 00000000000..cf243dedef5 --- /dev/null +++ b/server/config.go @@ -0,0 +1,45 @@ +package server + +// Config represents the server configuration. +type Config struct { + AdvertisedUrl string `toml:"advertised_url"` + CAFile string `toml:"ca_file"` + CertFile string `toml:"cert_file"` + CPUProfileFile string `toml:"cpu_profile_file"` + Cors []string `toml:"cors"` + CorsWhiteList map[string]bool + DataDir string `toml:"datadir"` + KeyFile string `toml:"key_file"` + ListenHost string `toml:"listen_host"` + Machines []string `toml:"machines"` + MachinesFile string `toml:"machines_file"` + MaxClusterSize int `toml:"max_cluster_size"` + MaxResultBuffer int `toml:"max_result_buffer"` + MaxRetryAttempts int `toml:"max_retry_attempts"` + Name string `toml:"name"` + Snapshot bool `toml:"snapshot"` + Verbose bool `toml:"verbose"` + VeryVerbose bool `toml:"very_verbose"` + WebURL string `toml:"web_url"` + + Peer struct { + AdvertisedUrl string `toml:"advertised_url"` + CAFile string `toml:"ca_file"` + CertFile string `toml:"cert_file"` + KeyFile string `toml:"key_file"` + ListenHost string `toml:"listen_host"` + } +} + +// NewConfig returns a Config initialized with default values. +func NewConfig() *Config { + c := new(Config) + c.AdvertisedUrl = "127.0.0.1:4001" + c.CorsWhiteList = make(map[string]bool) + c.DataDir = "." + c.MaxClusterSize = 9 + c.MaxResultBuffer = 1024 + c.MaxRetryAttempts = 3 + c.Peer.AdvertisedUrl = "127.0.0.1:7001" + return c +} diff --git a/server/config_test.go b/server/config_test.go new file mode 100644 index 00000000000..550bffa7e35 --- /dev/null +++ b/server/config_test.go @@ -0,0 +1,66 @@ +package server + +import ( + "testing" + + "github.com/BurntSushi/toml" + "github.com/stretchr/testify/assert" +) + + +// Ensures that a configuration can be deserialized from TOML. +func TestConfigTOML(t *testing.T) { + content := ` + advertised_url = "127.0.0.1:4002" + ca_file = "/tmp/file.ca" + cert_file = "/tmp/file.cert" + cors = ["*"] + cpu_profile_file = "XXX" + datadir = "/tmp/data" + key_file = "/tmp/file.key" + listen_host = "127.0.0.1:4003" + machines = ["coreos.com:4001", "coreos.com:4002"] + machines_file = "/tmp/machines" + max_cluster_size = 10 + max_result_buffer = 512 + max_retry_attempts = 5 + name = "test-name" + snapshot = true + verbose = true + very_verbose = true + web_url = "/web" + + [peer] + advertised_url = "127.0.0.1:7002" + ca_file = "/tmp/peer/file.ca" + cert_file = "/tmp/peer/file.cert" + key_file = "/tmp/peer/file.key" + listen_host = "127.0.0.1:7003" + ` + c := NewConfig() + _, err := toml.Decode(content, &c) + assert.Nil(t, err, "") + assert.Equal(t, c.AdvertisedUrl, "127.0.0.1:4002", "") + assert.Equal(t, c.CAFile, "/tmp/file.ca", "") + assert.Equal(t, c.CertFile, "/tmp/file.cert", "") + assert.Equal(t, c.CPUProfileFile, "XXX", "") + assert.Equal(t, c.Cors, []string{"*"}, "") + assert.Equal(t, c.DataDir, "/tmp/data", "") + assert.Equal(t, c.KeyFile, "/tmp/file.key", "") + assert.Equal(t, c.ListenHost, "127.0.0.1:4003", "") + assert.Equal(t, c.Machines, []string{"coreos.com:4001", "coreos.com:4002"}, "") + assert.Equal(t, c.MachinesFile, "/tmp/machines", "") + assert.Equal(t, c.MaxClusterSize, 10, "") + assert.Equal(t, c.MaxResultBuffer, 512, "") + assert.Equal(t, c.MaxRetryAttempts, 5, "") + assert.Equal(t, c.Name, "test-name", "") + assert.Equal(t, c.Snapshot, true, "") + assert.Equal(t, c.Verbose, true, "") + assert.Equal(t, c.VeryVerbose, true, "") + assert.Equal(t, c.WebURL, "/web", "") + assert.Equal(t, c.Peer.AdvertisedUrl, "127.0.0.1:7002", "") + assert.Equal(t, c.Peer.CAFile, "/tmp/peer/file.ca", "") + assert.Equal(t, c.Peer.CertFile, "/tmp/peer/file.cert", "") + assert.Equal(t, c.Peer.KeyFile, "/tmp/peer/file.key", "") + assert.Equal(t, c.Peer.ListenHost, "127.0.0.1:7003", "") +} diff --git a/server/tls_config.go b/server/tls_config.go index 4b944626d16..29b8fa5e4ff 100644 --- a/server/tls_config.go +++ b/server/tls_config.go @@ -4,6 +4,7 @@ import ( "crypto/tls" ) +// TLSConfig holds the TLS configuration. type TLSConfig struct { Scheme string Server tls.Config diff --git a/server/tls_info.go b/server/tls_info.go index 91936b0906e..89ed6f05506 100644 --- a/server/tls_info.go +++ b/server/tls_info.go @@ -1,5 +1,6 @@ package server +// TLSInfo holds the SSL certificates paths. type TLSInfo struct { CertFile string `json:"CertFile"` KeyFile string `json:"KeyFile"` diff --git a/server/util.go b/server/util.go index 578c048cc8d..0fa36b69a8d 100644 --- a/server/util.go +++ b/server/util.go @@ -6,6 +6,7 @@ import ( "io" "net/http" "net/url" + "strings" "github.com/coreos/etcd/log" ) @@ -31,3 +32,15 @@ func redirect(hostname string, w http.ResponseWriter, req *http.Request) { log.Debugf("Redirect to %s", redirectURL.String()) http.Redirect(w, req, redirectURL.String(), http.StatusTemporaryRedirect) } + +// trimsplit slices s into all substrings separated by sep and returns a +// slice of the substrings between the separator with all leading and trailing +// white space removed, as defined by Unicode. +func trimsplit(s, sep string) []string { + raw := strings.Split(s, ",") + trimmed := make([]string, len(raw)) + for _, r := range raw { + trimmed = append(trimmed, strings.TrimSpace(r)) + } + return trimmed +} From a659dec00b73cc9c62916b654a6b5940770e8305 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 21 Oct 2013 17:33:23 -0700 Subject: [PATCH 140/247] fix headerTimeout should be longer and body should be shorter --- server/transporter.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/server/transporter.go b/server/transporter.go index 3c5002f1ba2..f50dd0b5f53 100644 --- a/server/transporter.go +++ b/server/transporter.go @@ -18,13 +18,15 @@ import ( // This should not exceed 3 * RTT var dailTimeout = 3 * HeartbeatTimeout -// Timeout for setup internal raft http connection + receive response header -// This should not exceed 3 * RTT + RTT -var responseHeaderTimeout = 4 * HeartbeatTimeout +// Timeout for setup internal raft http connection + receive all post body +// The raft server will not send back response header until it received all the +// post body. +// This should not exceed dailTimeout + electionTimeout +var responseHeaderTimeout = 3*HeartbeatTimeout + ElectionTimeout // Timeout for receiving the response body from the server -// This should not exceed election timeout -var tranTimeout = ElectionTimeout +// This should not exceed heartbeatTimeout +var tranTimeout = HeartbeatTimeout // Transporter layer for communication between raft nodes type transporter struct { From 91cad59adabceeed1ac5ce8f6bf3492d6849dd21 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 21 Oct 2013 17:53:24 -0700 Subject: [PATCH 141/247] refactor use transtiemout --- server/transporter.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/transporter.go b/server/transporter.go index f50dd0b5f53..95545a82752 100644 --- a/server/transporter.go +++ b/server/transporter.go @@ -223,7 +223,7 @@ func (t *transporter) Get(urlStr string) (*http.Response, *http.Request, error) // Cancel the on fly HTTP transaction when timeout happens. func (t *transporter) CancelWhenTimeout(req *http.Request) { go func() { - time.Sleep(ElectionTimeout) + time.Sleep(tranTimeout) t.transport.CancelRequest(req) }() } From bc0e07bca275599728ec5b721d5a58c5315454bc Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 21 Oct 2013 16:00:20 -0600 Subject: [PATCH 142/247] Add tiered configuration. --- Documentation/external-documentation.md | 105 +++ config.go | 143 ---- etcd.go | 229 ++---- server/config.go | 402 ++++++++++ server/config_test.go | 479 +++++++++++ server/info.go | 19 + server/server.go | 4 +- server/tls_config.go | 1 + server/tls_info.go | 69 ++ server/transporter_test.go | 2 + server/util.go | 14 + test.sh | 3 + third_party/deps | 1 + .../github.com/BurntSushi/toml/.gitignore | 4 + .../github.com/BurntSushi/toml/COMPATIBLE | 3 + .../github.com/BurntSushi/toml/COPYING | 14 + .../github.com/BurntSushi/toml/Makefile | 14 + .../github.com/BurntSushi/toml/README.md | 163 ++++ .../BurntSushi/toml/_examples/example.go | 59 ++ .../BurntSushi/toml/_examples/example.toml | 35 + .../BurntSushi/toml/_examples/hard.toml | 22 + .../BurntSushi/toml/_examples/implicit.toml | 4 + .../toml/_examples/invalid-apples.toml | 6 + .../BurntSushi/toml/_examples/invalid.toml | 35 + .../BurntSushi/toml/_examples/readme1.toml | 5 + .../BurntSushi/toml/_examples/readme2.toml | 1 + .../github.com/BurntSushi/toml/decode.go | 429 ++++++++++ .../github.com/BurntSushi/toml/decode_test.go | 343 ++++++++ third_party/github.com/BurntSushi/toml/doc.go | 10 + .../github.com/BurntSushi/toml/encode.go | 99 +++ .../github.com/BurntSushi/toml/encode_test.go | 25 + third_party/github.com/BurntSushi/toml/lex.go | 741 ++++++++++++++++++ .../github.com/BurntSushi/toml/lex_test.go | 59 ++ .../github.com/BurntSushi/toml/out_test.go | 19 + .../github.com/BurntSushi/toml/parse.go | 388 +++++++++ .../github.com/BurntSushi/toml/parse_test.go | 61 ++ .../github.com/BurntSushi/toml/session.vim | 1 + .../BurntSushi/toml/toml-test-go/COPYING | 14 + .../BurntSushi/toml/toml-test-go/README.md | 14 + .../BurntSushi/toml/toml-test-go/main.go | 89 +++ .../github.com/BurntSushi/toml/tomlv/COPYING | 14 + .../BurntSushi/toml/tomlv/README.md | 22 + .../github.com/BurntSushi/toml/tomlv/main.go | 60 ++ .../github.com/BurntSushi/toml/type-check.go | 78 ++ util.go | 88 --- 45 files changed, 4000 insertions(+), 390 deletions(-) create mode 100644 Documentation/external-documentation.md delete mode 100644 config.go create mode 100644 server/config.go create mode 100644 server/config_test.go create mode 100644 server/info.go create mode 100644 third_party/github.com/BurntSushi/toml/.gitignore create mode 100644 third_party/github.com/BurntSushi/toml/COMPATIBLE create mode 100644 third_party/github.com/BurntSushi/toml/COPYING create mode 100644 third_party/github.com/BurntSushi/toml/Makefile create mode 100644 third_party/github.com/BurntSushi/toml/README.md create mode 100644 third_party/github.com/BurntSushi/toml/_examples/example.go create mode 100644 third_party/github.com/BurntSushi/toml/_examples/example.toml create mode 100644 third_party/github.com/BurntSushi/toml/_examples/hard.toml create mode 100644 third_party/github.com/BurntSushi/toml/_examples/implicit.toml create mode 100644 third_party/github.com/BurntSushi/toml/_examples/invalid-apples.toml create mode 100644 third_party/github.com/BurntSushi/toml/_examples/invalid.toml create mode 100644 third_party/github.com/BurntSushi/toml/_examples/readme1.toml create mode 100644 third_party/github.com/BurntSushi/toml/_examples/readme2.toml create mode 100644 third_party/github.com/BurntSushi/toml/decode.go create mode 100644 third_party/github.com/BurntSushi/toml/decode_test.go create mode 100644 third_party/github.com/BurntSushi/toml/doc.go create mode 100644 third_party/github.com/BurntSushi/toml/encode.go create mode 100644 third_party/github.com/BurntSushi/toml/encode_test.go create mode 100644 third_party/github.com/BurntSushi/toml/lex.go create mode 100644 third_party/github.com/BurntSushi/toml/lex_test.go create mode 100644 third_party/github.com/BurntSushi/toml/out_test.go create mode 100644 third_party/github.com/BurntSushi/toml/parse.go create mode 100644 third_party/github.com/BurntSushi/toml/parse_test.go create mode 100644 third_party/github.com/BurntSushi/toml/session.vim create mode 100644 third_party/github.com/BurntSushi/toml/toml-test-go/COPYING create mode 100644 third_party/github.com/BurntSushi/toml/toml-test-go/README.md create mode 100644 third_party/github.com/BurntSushi/toml/toml-test-go/main.go create mode 100644 third_party/github.com/BurntSushi/toml/tomlv/COPYING create mode 100644 third_party/github.com/BurntSushi/toml/tomlv/README.md create mode 100644 third_party/github.com/BurntSushi/toml/tomlv/main.go create mode 100644 third_party/github.com/BurntSushi/toml/type-check.go delete mode 100644 util.go diff --git a/Documentation/external-documentation.md b/Documentation/external-documentation.md new file mode 100644 index 00000000000..a9be38ff0eb --- /dev/null +++ b/Documentation/external-documentation.md @@ -0,0 +1,105 @@ +# Etcd Configuration + +Configuration options can be set in three places: + + 1. Command line flags + 2. Environment variables + 3. Configuration file + +Options set on the command line take precedence over all other sources. +Options set in environment variables take precedence over options set in +configuration files. + +## Command Line Flags + +### Required + +* `-n` - The node name. Defaults to `default-name`. + +### Optional + +* `-c` - The advertised public hostname:port for client communication. Defaults to `127.0.0.1:4001`. +* `-cl` - The listening hostname for client communication. Defaults to advertised ip. +* `-C` - A comma separated list of machines in the cluster (i.e `"203.0.113.101:7001,203.0.113.102:7001"`). +* `-CF` - The file path containing a comma separated list of machines in the cluster. +* `-clientCAFile` - The path of the client CAFile. Enables client cert authentication when present. +* `-clientCert` - The cert file of the client. +* `-clientKey` - The key file of the client. +* `-configfile` - The path of the etcd config file. Defaults to `/etc/etcd/etcd.toml`. +* `-cors` - A comma separated white list of origins for cross-origin resource sharing. +* `-cpuprofile` - The path to a file to output cpu profile data. Enables cpu profiling when present. +* `-d` - The directory to store log and snapshot. Defaults to the current working directory. +* `-m` - The max size of result buffer. Defaults to `1024`. +* `-maxsize` - The max size of the cluster. Defaults to `9`. +* `-r` - The max retry attempts when trying to join a cluster. Defaults to `3`. +* `-s` - The advertised public hostname:port for server communication. Defaults to `127.0.0.1:7001`. +* `-sl` - The listening hostname for server communication. Defaults to advertised ip. +* `-serverCAFile` - The path of the CAFile. Enables client/peer cert authentication when present. +* `-serverCert` - The cert file of the server. +* `-serverKey` - The key file of the server. +* `-snapshot` - Open or close snapshot. Defaults to `false`. +* `-v` - Enable verbose logging. Defaults to `false`. +* `-vv` - Enable very verbose logging. Defaults to `false`. +* `-version` - Print the version and exit. +* `-w` - The hostname:port of web interface. + +## Configuration File + +The etcd configuration file is written in [TOML](https://github.com/mojombo/toml) +and read from `/etc/etcd/etcd.toml` by default. + +```TOML +[etcd] + advertised_url = "127.0.0.1:4001" + ca_file = "" + cert_file = "" + cors = [] + cpu_profile_file = "" + datadir = "." + key_file = "" + listen_host = "127.0.0.1:4001" + machines = [] + machines_file = "" + max_cluster_size = 9 + max_result_buffer = 1024 + max_retry_attempts = 3 + name = "default-name" + snapshot = false + verbose = false + very_verbose = false + web_url = "" + +[raft] + advertised_url = "127.0.0.1:7001" + ca_file = "" + cert_file = "" + key_file = "" + listen_host = "127.0.0.1:7001" +``` + +## Environment Variables + + * `ETCD_ADVERTISED_URL` + * `ETCD_CA_FILE` + * `ETCD_CERT_FILE` + * `ETCD_CORS` + * `ETCD_CONFIG_FILE` + * `ETCD_CPU_PROFILE_FILE` + * `ETCD_DATADIR` + * `ETCD_KEY_FILE` + * `ETCD_LISTEN_HOST` + * `ETCD_MACHINES` + * `ETCD_MACHINES_FILE` + * `ETCD_MAX_RETRY_ATTEMPTS` + * `ETCD_MAX_CLUSTER_SIZE` + * `ETCD_MAX_RESULT_BUFFER` + * `ETCD_NAME` + * `ETCD_SNAPSHOT` + * `ETCD_VERBOSE` + * `ETCD_VERY_VERBOSE` + * `ETCD_WEB_URL` + * `ETCD_RAFT_ADVERTISED_URL` + * `ETCD_RAFT_CA_FILE` + * `ETCD_RAFT_CERT_FILE` + * `ETCD_RAFT_KEY_FILE` + * `ETCD_RAFT_LISTEN_HOST` diff --git a/config.go b/config.go deleted file mode 100644 index d1549c7becb..00000000000 --- a/config.go +++ /dev/null @@ -1,143 +0,0 @@ -package main - -import ( - "crypto/tls" - "crypto/x509" - "encoding/json" - "encoding/pem" - "io/ioutil" - "os" - "path/filepath" - - "github.com/coreos/etcd/log" - "github.com/coreos/etcd/server" -) - -//-------------------------------------- -// Config -//-------------------------------------- - -// Get the server info from previous conf file -// or from the user -func getInfo(path string) *Info { - - infoPath := filepath.Join(path, "info") - - if force { - // Delete the old configuration if exist - logPath := filepath.Join(path, "log") - confPath := filepath.Join(path, "conf") - snapshotPath := filepath.Join(path, "snapshot") - os.Remove(infoPath) - os.Remove(logPath) - os.Remove(confPath) - os.RemoveAll(snapshotPath) - } else if info := readInfo(infoPath); info != nil { - log.Infof("Found node configuration in '%s'. Ignoring flags", infoPath) - return info - } - - // Read info from command line - info := &argInfo - - // Write to file. - content, _ := json.MarshalIndent(info, "", " ") - content = []byte(string(content) + "\n") - if err := ioutil.WriteFile(infoPath, content, 0644); err != nil { - log.Fatalf("Unable to write info to file: %v", err) - } - - log.Infof("Wrote node configuration to '%s'", infoPath) - - return info -} - -// readInfo reads from info file and decode to Info struct -func readInfo(path string) *Info { - file, err := os.Open(path) - - if err != nil { - if os.IsNotExist(err) { - return nil - } - log.Fatal(err) - } - defer file.Close() - - info := &Info{} - - content, err := ioutil.ReadAll(file) - if err != nil { - log.Fatalf("Unable to read info: %v", err) - return nil - } - - if err = json.Unmarshal(content, &info); err != nil { - log.Fatalf("Unable to parse info: %v", err) - return nil - } - - return info -} - -func tlsConfigFromInfo(info server.TLSInfo) (t server.TLSConfig, ok bool) { - var keyFile, certFile, CAFile string - var tlsCert tls.Certificate - var err error - - t.Scheme = "http" - - keyFile = info.KeyFile - certFile = info.CertFile - CAFile = info.CAFile - - // If the user do not specify key file, cert file and - // CA file, the type will be HTTP - if keyFile == "" && certFile == "" && CAFile == "" { - return t, true - } - - // both the key and cert must be present - if keyFile == "" || certFile == "" { - return t, false - } - - tlsCert, err = tls.LoadX509KeyPair(certFile, keyFile) - if err != nil { - log.Fatal(err) - } - - t.Scheme = "https" - t.Server.ClientAuth, t.Server.ClientCAs = newCertPool(CAFile) - - // The client should trust the RootCA that the Server uses since - // everyone is a peer in the network. - t.Client.Certificates = []tls.Certificate{tlsCert} - t.Client.RootCAs = t.Server.ClientCAs - - return t, true -} - -// newCertPool creates x509 certPool and corresponding Auth Type. -// If the given CAfile is valid, add the cert into the pool and verify the clients' -// certs against the cert in the pool. -// If the given CAfile is empty, do not verify the clients' cert. -// If the given CAfile is not valid, fatal. -func newCertPool(CAFile string) (tls.ClientAuthType, *x509.CertPool) { - if CAFile == "" { - return tls.NoClientCert, nil - } - pemByte, err := ioutil.ReadFile(CAFile) - check(err) - - block, pemByte := pem.Decode(pemByte) - - cert, err := x509.ParseCertificate(block.Bytes) - check(err) - - certPool := x509.NewCertPool() - - certPool.AddCert(cert) - - return tls.RequireAndVerifyClientCert, certPool -} diff --git a/etcd.go b/etcd.go index f9df25c6dc6..3c01356f182 100644 --- a/etcd.go +++ b/etcd.go @@ -4,8 +4,9 @@ import ( "flag" "fmt" "io/ioutil" + "runtime/pprof" "os" - "strings" + "os/signal" "github.com/coreos/etcd/log" "github.com/coreos/etcd/server" @@ -13,189 +14,103 @@ import ( "github.com/coreos/go-raft" ) -//------------------------------------------------------------------------------ -// -// Initialization -// -//------------------------------------------------------------------------------ - -var ( - veryVerbose bool - - machines string - machinesFile string - - cluster []string - - argInfo Info - dirPath string - - force bool - - printVersion bool - - maxSize int - - snapshot bool - - retryTimes int - - maxClusterSize int - - cpuprofile string - - cors string -) - -func init() { - flag.BoolVar(&printVersion, "version", false, "print the version and exit") - - flag.BoolVar(&log.Verbose, "v", false, "verbose logging") - flag.BoolVar(&veryVerbose, "vv", false, "very verbose logging") - - flag.StringVar(&machines, "C", "", "the ip address and port of a existing machines in the cluster, sepearate by comma") - flag.StringVar(&machinesFile, "CF", "", "the file contains a list of existing machines in the cluster, seperate by comma") - - flag.StringVar(&argInfo.Name, "n", "default-name", "the node name (required)") - flag.StringVar(&argInfo.EtcdURL, "c", "127.0.0.1:4001", "the advertised public hostname:port for etcd client communication") - flag.StringVar(&argInfo.RaftURL, "s", "127.0.0.1:7001", "the advertised public hostname:port for raft server communication") - flag.StringVar(&argInfo.EtcdListenHost, "cl", "", "the listening hostname for etcd client communication (defaults to advertised ip)") - flag.StringVar(&argInfo.RaftListenHost, "sl", "", "the listening hostname for raft server communication (defaults to advertised ip)") - flag.StringVar(&argInfo.WebURL, "w", "", "the hostname:port of web interface") - - flag.StringVar(&argInfo.RaftTLS.CAFile, "serverCAFile", "", "the path of the CAFile") - flag.StringVar(&argInfo.RaftTLS.CertFile, "serverCert", "", "the cert file of the server") - flag.StringVar(&argInfo.RaftTLS.KeyFile, "serverKey", "", "the key file of the server") - - flag.StringVar(&argInfo.EtcdTLS.CAFile, "clientCAFile", "", "the path of the client CAFile") - flag.StringVar(&argInfo.EtcdTLS.CertFile, "clientCert", "", "the cert file of the client") - flag.StringVar(&argInfo.EtcdTLS.KeyFile, "clientKey", "", "the key file of the client") - - flag.StringVar(&dirPath, "d", ".", "the directory to store log and snapshot") - - flag.BoolVar(&force, "f", false, "force new node configuration if existing is found (WARNING: data loss!)") - - flag.BoolVar(&snapshot, "snapshot", false, "open or close snapshot") - - flag.IntVar(&maxSize, "m", 1024, "the max size of result buffer") - - flag.IntVar(&retryTimes, "r", 3, "the max retry attempts when trying to join a cluster") - - flag.IntVar(&maxClusterSize, "maxsize", 9, "the max size of the cluster") - - flag.StringVar(&cpuprofile, "cpuprofile", "", "write cpu profile to file") - - flag.StringVar(&cors, "cors", "", "whitelist origins for cross-origin resource sharing (e.g. '*' or 'http://localhost:8001,etc')") -} - -//------------------------------------------------------------------------------ -// -// Typedefs -// -//------------------------------------------------------------------------------ - -type Info struct { - Name string `json:"name"` - - RaftURL string `json:"raftURL"` - EtcdURL string `json:"etcdURL"` - WebURL string `json:"webURL"` - - RaftListenHost string `json:"raftListenHost"` - EtcdListenHost string `json:"etcdListenHost"` - - RaftTLS server.TLSInfo `json:"raftTLS"` - EtcdTLS server.TLSInfo `json:"etcdTLS"` -} - -//------------------------------------------------------------------------------ -// -// Functions -// -//------------------------------------------------------------------------------ - -//-------------------------------------- -// Main -//-------------------------------------- - func main() { - flag.Parse() + parseFlags() - if printVersion { - fmt.Println(server.ReleaseVersion) - os.Exit(0) + // Load configuration. + var config = server.NewConfig() + if err := config.Load(os.Args[1:]); err != nil { + log.Fatal("Configuration error:", err) } - if cpuprofile != "" { - runCPUProfile() - } - - if veryVerbose { + // Turn on logging. + if config.VeryVerbose { log.Verbose = true raft.SetLogLevel(raft.Debug) + } else if config.Verbose { + log.Verbose = true } - if machines != "" { - cluster = strings.Split(machines, ",") - } else if machinesFile != "" { - b, err := ioutil.ReadFile(machinesFile) - if err != nil { - log.Fatalf("Unable to read the given machines file: %s", err) - } - cluster = strings.Split(string(b), ",") - } - - // Check TLS arguments - raftTLSConfig, ok := tlsConfigFromInfo(argInfo.RaftTLS) - if !ok { - log.Fatal("Please specify cert and key file or cert and key file and CAFile or none of the three") + // Create data directory if it doesn't already exist. + if err := os.MkdirAll(config.DataDir, 0744); err != nil { + log.Fatalf("Unable to create path: %s", err) } - etcdTLSConfig, ok := tlsConfigFromInfo(argInfo.EtcdTLS) - if !ok { - log.Fatal("Please specify cert and key file or cert and key file and CAFile or none of the three") + // Load info object. + info, err := config.Info() + if err != nil { + log.Fatal("info:", err) } - - argInfo.Name = strings.TrimSpace(argInfo.Name) - if argInfo.Name == "" { + if info.Name == "" { log.Fatal("ERROR: server name required. e.g. '-n=server_name'") } - // Check host name arguments - argInfo.RaftURL = sanitizeURL(argInfo.RaftURL, raftTLSConfig.Scheme) - argInfo.EtcdURL = sanitizeURL(argInfo.EtcdURL, etcdTLSConfig.Scheme) - argInfo.WebURL = sanitizeURL(argInfo.WebURL, "http") - - argInfo.RaftListenHost = sanitizeListenHost(argInfo.RaftListenHost, argInfo.RaftURL) - argInfo.EtcdListenHost = sanitizeListenHost(argInfo.EtcdListenHost, argInfo.EtcdURL) - - // Read server info from file or grab it from user. - if err := os.MkdirAll(dirPath, 0744); err != nil { - log.Fatalf("Unable to create path: %s", err) + // Retrieve TLS configuration. + tlsConfig, err := info.EtcdTLS.Config() + if err != nil { + log.Fatal("Client TLS:", err) + } + peerTLSConfig, err := info.RaftTLS.Config() + if err != nil { + log.Fatal("Peer TLS:", err) } - info := getInfo(dirPath) - - // Create etcd key-value store + // Create etcd key-value store and registry. store := store.New() - - // Create a shared node registry. registry := server.NewRegistry(store) // Create peer server. - ps := server.NewPeerServer(info.Name, dirPath, info.RaftURL, info.RaftListenHost, &raftTLSConfig, &info.RaftTLS, registry, store) - ps.MaxClusterSize = maxClusterSize - ps.RetryTimes = retryTimes + ps := server.NewPeerServer(info.Name, config.DataDir, info.RaftURL, info.RaftListenHost, &peerTLSConfig, &info.RaftTLS, registry, store) + ps.MaxClusterSize = config.MaxClusterSize + ps.RetryTimes = config.MaxRetryAttempts - s := server.New(info.Name, info.EtcdURL, info.EtcdListenHost, &etcdTLSConfig, &info.EtcdTLS, ps, registry, store) - if err := s.AllowOrigins(cors); err != nil { + // Create client server. + s := server.New(info.Name, info.EtcdURL, info.EtcdListenHost, &tlsConfig, &info.EtcdTLS, ps, registry, store) + if err := s.AllowOrigins(config.Cors); err != nil { panic(err) } ps.SetServer(s) + // Run peer server in separate thread while the client server blocks. go func() { - log.Fatal(ps.ListenAndServe(snapshot, cluster)) + log.Fatal(ps.ListenAndServe(config.Snapshot, config.Machines)) }() log.Fatal(s.ListenAndServe()) } + +// Parses non-configuration flags. +func parseFlags() { + var versionFlag bool + var cpuprofile string + + f := flag.NewFlagSet(os.Args[0], -1) + f.SetOutput(ioutil.Discard) + f.BoolVar(&versionFlag, "version", false, "print the version and exit") + f.StringVar(&cpuprofile, "cpuprofile", "", "write cpu profile to file") + f.Parse(os.Args[1:]) + + // Print version if necessary. + if versionFlag { + fmt.Println(server.ReleaseVersion) + os.Exit(0) + } + + // Begin CPU profiling if specified. + if cpuprofile != "" { + f, err := os.Create(cpuprofile) + if err != nil { + log.Fatal(err) + } + pprof.StartCPUProfile(f) + + c := make(chan os.Signal, 1) + signal.Notify(c, os.Interrupt) + go func() { + sig := <-c + log.Infof("captured %v, stopping profiler and exiting..", sig) + pprof.StopCPUProfile() + os.Exit(1) + }() + } +} diff --git a/server/config.go b/server/config.go new file mode 100644 index 00000000000..b86a433eeaa --- /dev/null +++ b/server/config.go @@ -0,0 +1,402 @@ +package server + +import ( + "encoding/json" + "fmt" + "flag" + "io/ioutil" + "net" + "net/url" + "os" + "path/filepath" + "reflect" + "strconv" + "strings" + + "github.com/BurntSushi/toml" +) + +// The default location for the etcd configuration file. +const DefaultSystemConfigPath = "/etc/etcd/etcd.toml" + +// Config represents the server configuration. +type Config struct { + SystemPath string + + AdvertisedUrl string `toml:"advertised_url" env:"ETCD_ADVERTISED_URL"` + CAFile string `toml:"ca_file" env:"ETCD_CA_FILE"` + CertFile string `toml:"cert_file" env:"ETCD_CERT_FILE"` + Cors []string `toml:"cors" env:"ETCD_CORS"` + DataDir string `toml:"datadir" env:"ETCD_DATADIR"` + KeyFile string `toml:"key_file" env:"ETCD_KEY_FILE"` + ListenHost string `toml:"listen_host" env:"ETCD_LISTEN_HOST"` + Machines []string `toml:"machines" env:"ETCD_MACHINES"` + MachinesFile string `toml:"machines_file" env:"ETCD_MACHINES_FILE"` + MaxClusterSize int `toml:"max_cluster_size" env:"ETCD_MAX_CLUSTER_SIZE"` + MaxResultBuffer int `toml:"max_result_buffer" env:"ETCD_MAX_RESULT_BUFFER"` + MaxRetryAttempts int `toml:"max_retry_attempts" env:"ETCD_MAX_RETRY_ATTEMPTS"` + Name string `toml:"name" env:"ETCD_NAME"` + Snapshot bool `toml:"snapshot" env:"ETCD_SNAPSHOT"` + Verbose bool `toml:"verbose" env:"ETCD_VERBOSE"` + VeryVerbose bool `toml:"very_verbose" env:"ETCD_VERY_VERBOSE"` + WebURL string `toml:"web_url" env:"ETCD_WEB_URL"` + + Peer struct { + AdvertisedUrl string `toml:"advertised_url" env:"ETCD_PEER_ADVERTISED_URL"` + CAFile string `toml:"ca_file" env:"ETCD_PEER_CA_FILE"` + CertFile string `toml:"cert_file" env:"ETCD_PEER_CERT_FILE"` + KeyFile string `toml:"key_file" env:"ETCD_PEER_KEY_FILE"` + ListenHost string `toml:"listen_host" env:"ETCD_PEER_LISTEN_HOST"` + } +} + +// NewConfig returns a Config initialized with default values. +func NewConfig() *Config { + c := new(Config) + c.SystemPath = DefaultSystemConfigPath + c.AdvertisedUrl = "127.0.0.1:4001" + c.DataDir = "." + c.MaxClusterSize = 9 + c.MaxResultBuffer = 1024 + c.MaxRetryAttempts = 3 + c.Peer.AdvertisedUrl = "127.0.0.1:7001" + return c +} + +// Loads the configuration from the system config, command line config, +// environment variables, and finally command line arguments. +func (c *Config) Load(arguments []string) error { + var path string + f := flag.NewFlagSet("etcd", -1) + f.SetOutput(ioutil.Discard) + f.StringVar(&path, "config", "", "path to config file") + f.Parse(arguments) + + // Load from system file. + if err := c.LoadSystemFile(); err != nil { + return err + } + + // Load from config file specified in arguments. + if path != "" { + if err := c.LoadFile(path); err != nil { + return err + } + } + + // Load from the environment variables next. + if err := c.LoadEnv(); err != nil { + return err + } + + // Load from command line flags. + if err := c.LoadFlags(arguments); err != nil { + return err + } + + // Loads machines if a machine file was specified. + if err := c.LoadMachineFile(); err != nil { + return err + } + + // Sanitize all the input fields. + if err := c.Sanitize(); err != nil { + return fmt.Errorf("sanitize:", err) + } + + return nil +} + +// Loads from the system etcd configuration file if it exists. +func (c *Config) LoadSystemFile() error { + if _, err := os.Stat(c.SystemPath); os.IsNotExist(err) { + return nil + } + return c.LoadFile(c.SystemPath) +} + +// Loads configuration from a file. +func (c *Config) LoadFile(path string) error { + _, err := toml.DecodeFile(path, &c) + return err +} + +// LoadEnv loads the configuration via environment variables. +func (c *Config) LoadEnv() error { + if err := c.loadEnv(c); err != nil { + return err + } + if err := c.loadEnv(&c.Peer); err != nil { + return err + } + return nil +} + +func (c *Config) loadEnv(target interface{}) error { + value := reflect.Indirect(reflect.ValueOf(target)) + typ := value.Type() + for i := 0; i < typ.NumField(); i++ { + field := typ.Field(i) + + // Retrieve environment variable. + v := strings.TrimSpace(os.Getenv(field.Tag.Get("env"))) + if v == "" { + continue + } + + // Set the appropriate type. + switch field.Type.Kind() { + case reflect.Bool: + value.Field(i).SetBool(v != "0" && v != "false") + case reflect.Int: + newValue, err := strconv.ParseInt(v, 10, 0) + if err != nil { + return fmt.Errorf("Parse error: %s: %s", field.Tag.Get("env"), err) + } + value.Field(i).SetInt(newValue) + case reflect.String: + value.Field(i).SetString(v) + case reflect.Slice: + value.Field(i).Set(reflect.ValueOf(trimsplit(v, ","))) + } + } + return nil +} + +// Loads configuration from command line flags. +func (c *Config) LoadFlags(arguments []string) error { + var machines, cors string + var force bool + + f := flag.NewFlagSet(os.Args[0], flag.ContinueOnError) + + f.BoolVar(&force, "f", false, "force new node configuration if existing is found (WARNING: data loss!)") + + f.BoolVar(&c.Verbose, "v", c.Verbose, "verbose logging") + f.BoolVar(&c.VeryVerbose, "vv", c.Verbose, "very verbose logging") + + f.StringVar(&machines, "C", "", "the ip address and port of a existing machines in the cluster, sepearate by comma") + f.StringVar(&c.MachinesFile, "CF", c.MachinesFile, "the file contains a list of existing machines in the cluster, seperate by comma") + + f.StringVar(&c.Name, "n", c.Name, "the node name (required)") + f.StringVar(&c.AdvertisedUrl, "c", c.AdvertisedUrl, "the advertised public hostname:port for etcd client communication") + f.StringVar(&c.Peer.AdvertisedUrl, "s", c.Peer.AdvertisedUrl, "the advertised public hostname:port for raft server communication") + f.StringVar(&c.ListenHost, "cl", c.ListenHost, "the listening hostname for etcd client communication (defaults to advertised ip)") + f.StringVar(&c.Peer.ListenHost, "sl", c.Peer.ListenHost, "the listening hostname for raft server communication (defaults to advertised ip)") + f.StringVar(&c.WebURL, "w", c.WebURL, "the hostname:port of web interface") + + f.StringVar(&c.Peer.CAFile, "serverCAFile", c.Peer.CAFile, "the path of the CAFile") + f.StringVar(&c.Peer.CertFile, "serverCert", c.Peer.CertFile, "the cert file of the server") + f.StringVar(&c.Peer.KeyFile, "serverKey", c.Peer.KeyFile, "the key file of the server") + + f.StringVar(&c.CAFile, "clientCAFile", c.CAFile, "the path of the client CAFile") + f.StringVar(&c.CertFile, "clientCert", c.CertFile, "the cert file of the client") + f.StringVar(&c.KeyFile, "clientKey", c.KeyFile, "the key file of the client") + + f.StringVar(&c.DataDir, "d", c.DataDir, "the directory to store log and snapshot") + f.BoolVar(&c.Snapshot, "snapshot", c.Snapshot, "open or close snapshot") + f.IntVar(&c.MaxResultBuffer, "m", c.MaxResultBuffer, "the max size of result buffer") + f.IntVar(&c.MaxRetryAttempts, "r", c.MaxRetryAttempts, "the max retry attempts when trying to join a cluster") + f.IntVar(&c.MaxClusterSize, "maxsize", c.MaxClusterSize, "the max size of the cluster") + f.StringVar(&cors, "cors", "", "whitelist origins for cross-origin resource sharing (e.g. '*' or 'http://localhost:8001,etc')") + + // These flags are ignored since they were already parsed. + var path string + f.StringVar(&path, "config", "", "path to config file") + + f.Parse(arguments) + + // Convert some parameters to lists. + if machines != "" { + c.Machines = trimsplit(machines, ",") + } + if cors != "" { + c.Cors = trimsplit(cors, ",") + } + + // Force remove server configuration if specified. + if force { + c.Reset() + } + + return nil +} + +// LoadMachineFile loads the machines listed in the machine file. +func (c *Config) LoadMachineFile() error { + if c.MachinesFile == "" { + return nil + } + + b, err := ioutil.ReadFile(c.MachinesFile) + if err != nil { + return fmt.Errorf("Machines file error: %s", err) + } + c.Machines = trimsplit(string(b), ",") + + return nil +} + +// Reset removes all server configuration files. +func (c *Config) Reset() error { + if err := os.RemoveAll(filepath.Join(c.DataDir, "info")); err != nil { + return err + } + if err := os.RemoveAll(filepath.Join(c.DataDir, "log")); err != nil { + return err + } + if err := os.RemoveAll(filepath.Join(c.DataDir, "conf")); err != nil { + return err + } + if err := os.RemoveAll(filepath.Join(c.DataDir, "snapshot")); err != nil { + return err + } + + return nil +} + +// Reads the info file from the file system or initializes it based on the config. +func (c *Config) Info() (*Info, error) { + info := &Info{} + path := filepath.Join(c.DataDir, "info") + + // Open info file and read it out. + f, err := os.Open(path) + if err != nil && !os.IsNotExist(err) { + return nil, err + } else if f != nil { + defer f.Close() + if err := json.NewDecoder(f).Decode(&info); err != nil { + return nil, err + } + return info, nil + } + + // If the file doesn't exist then initialize it. + info.Name = strings.TrimSpace(c.Name) + info.EtcdURL = c.AdvertisedUrl + info.EtcdListenHost = c.ListenHost + info.RaftURL = c.Peer.AdvertisedUrl + info.RaftListenHost = c.Peer.ListenHost + info.WebURL = c.WebURL + info.EtcdTLS = c.TLSInfo() + info.RaftTLS = c.PeerTLSInfo() + + // Write to file. + f, err = os.Create(path) + if err != nil { + return nil, err + } + defer f.Close() + + if err := json.NewEncoder(f).Encode(info); err != nil { + return nil, err + } + + return info, nil +} + +// Sanitize cleans the input fields. +func (c *Config) Sanitize() error { + tlsConfig, err := c.TLSConfig() + if err != nil { + return err + } + + peerTlsConfig, err := c.PeerTLSConfig() + if err != nil { + return err + } + + // Sanitize the URLs first. + if c.AdvertisedUrl, err = sanitizeURL(c.AdvertisedUrl, tlsConfig.Scheme); err != nil { + return fmt.Errorf("Advertised URL: %s", err) + } + if c.ListenHost, err = sanitizeListenHost(c.ListenHost, c.AdvertisedUrl); err != nil { + return fmt.Errorf("Listen Host: %s", err) + } + if c.WebURL, err = sanitizeURL(c.WebURL, "http"); err != nil { + return fmt.Errorf("Web URL: %s", err) + } + if c.Peer.AdvertisedUrl, err = sanitizeURL(c.Peer.AdvertisedUrl, peerTlsConfig.Scheme); err != nil { + return fmt.Errorf("Peer Advertised URL: %s", err) + } + if c.Peer.ListenHost, err = sanitizeListenHost(c.Peer.ListenHost, c.Peer.AdvertisedUrl); err != nil { + return fmt.Errorf("Peer Listen Host: %s", err) + } + + return nil +} + + +// TLSInfo retrieves a TLSInfo object for the client server. +func (c *Config) TLSInfo() TLSInfo { + return TLSInfo{ + CAFile: c.CAFile, + CertFile: c.CertFile, + KeyFile: c.KeyFile, + } +} + +// ClientTLSConfig generates the TLS configuration for the client server. +func (c *Config) TLSConfig() (TLSConfig, error) { + return c.TLSInfo().Config() +} + +// PeerTLSInfo retrieves a TLSInfo object for the peer server. +func (c *Config) PeerTLSInfo() TLSInfo { + return TLSInfo{ + CAFile: c.Peer.CAFile, + CertFile: c.Peer.CertFile, + KeyFile: c.Peer.KeyFile, + } +} + +// PeerTLSConfig generates the TLS configuration for the peer server. +func (c *Config) PeerTLSConfig() (TLSConfig, error) { + return c.PeerTLSInfo().Config() +} + +// sanitizeURL will cleanup a host string in the format hostname:port and +// attach a schema. +func sanitizeURL(host string, defaultScheme string) (string, error) { + // Blank URLs are fine input, just return it + if len(host) == 0 { + return host, nil + } + + p, err := url.Parse(host) + if err != nil { + return "", err + } + + // Make sure the host is in Host:Port format + _, _, err = net.SplitHostPort(host) + if err != nil { + return "", err + } + + p = &url.URL{Host: host, Scheme: defaultScheme} + return p.String(), nil +} + +// sanitizeListenHost cleans up the ListenHost parameter and appends a port +// if necessary based on the advertised port. +func sanitizeListenHost(listen string, advertised string) (string, error) { + aurl, err := url.Parse(advertised) + if err != nil { + return "", err + } + + ahost, aport, err := net.SplitHostPort(aurl.Host) + if err != nil { + return "", err + } + + // If the listen host isn't set use the advertised host + if listen == "" { + listen = ahost + } + + return net.JoinHostPort(listen, aport), nil +} diff --git a/server/config_test.go b/server/config_test.go new file mode 100644 index 00000000000..7205348887f --- /dev/null +++ b/server/config_test.go @@ -0,0 +1,479 @@ +package server + +import ( + "io/ioutil" + "os" + "testing" + + "github.com/BurntSushi/toml" + "github.com/stretchr/testify/assert" +) + + +// Ensures that a configuration can be deserialized from TOML. +func TestConfigTOML(t *testing.T) { + content := ` + advertised_url = "127.0.0.1:4002" + ca_file = "/tmp/file.ca" + cert_file = "/tmp/file.cert" + cors = ["*"] + cpu_profile_file = "XXX" + datadir = "/tmp/data" + key_file = "/tmp/file.key" + listen_host = "127.0.0.1:4003" + machines = ["coreos.com:4001", "coreos.com:4002"] + machines_file = "/tmp/machines" + max_cluster_size = 10 + max_result_buffer = 512 + max_retry_attempts = 5 + name = "test-name" + snapshot = true + verbose = true + very_verbose = true + web_url = "/web" + + [peer] + advertised_url = "127.0.0.1:7002" + ca_file = "/tmp/peer/file.ca" + cert_file = "/tmp/peer/file.cert" + key_file = "/tmp/peer/file.key" + listen_host = "127.0.0.1:7003" + ` + c := NewConfig() + _, err := toml.Decode(content, &c) + assert.Nil(t, err, "") + assert.Equal(t, c.AdvertisedUrl, "127.0.0.1:4002", "") + assert.Equal(t, c.CAFile, "/tmp/file.ca", "") + assert.Equal(t, c.CertFile, "/tmp/file.cert", "") + assert.Equal(t, c.Cors, []string{"*"}, "") + assert.Equal(t, c.DataDir, "/tmp/data", "") + assert.Equal(t, c.KeyFile, "/tmp/file.key", "") + assert.Equal(t, c.ListenHost, "127.0.0.1:4003", "") + assert.Equal(t, c.Machines, []string{"coreos.com:4001", "coreos.com:4002"}, "") + assert.Equal(t, c.MachinesFile, "/tmp/machines", "") + assert.Equal(t, c.MaxClusterSize, 10, "") + assert.Equal(t, c.MaxResultBuffer, 512, "") + assert.Equal(t, c.MaxRetryAttempts, 5, "") + assert.Equal(t, c.Name, "test-name", "") + assert.Equal(t, c.Snapshot, true, "") + assert.Equal(t, c.Verbose, true, "") + assert.Equal(t, c.VeryVerbose, true, "") + assert.Equal(t, c.WebURL, "/web", "") + assert.Equal(t, c.Peer.AdvertisedUrl, "127.0.0.1:7002", "") + assert.Equal(t, c.Peer.CAFile, "/tmp/peer/file.ca", "") + assert.Equal(t, c.Peer.CertFile, "/tmp/peer/file.cert", "") + assert.Equal(t, c.Peer.KeyFile, "/tmp/peer/file.key", "") + assert.Equal(t, c.Peer.ListenHost, "127.0.0.1:7003", "") +} + +// Ensures that a configuration can be retrieved from environment variables. +func TestConfigEnv(t *testing.T) { + os.Setenv("ETCD_CA_FILE", "/tmp/file.ca") + os.Setenv("ETCD_CERT_FILE", "/tmp/file.cert") + os.Setenv("ETCD_CPU_PROFILE_FILE", "XXX") + os.Setenv("ETCD_CORS", "localhost:4001,localhost:4002") + os.Setenv("ETCD_DATADIR", "/tmp/data") + os.Setenv("ETCD_KEY_FILE", "/tmp/file.key") + os.Setenv("ETCD_LISTEN_HOST", "127.0.0.1:4003") + os.Setenv("ETCD_MACHINES", "coreos.com:4001,coreos.com:4002") + os.Setenv("ETCD_MACHINES_FILE", "/tmp/machines") + os.Setenv("ETCD_MAX_CLUSTER_SIZE", "10") + os.Setenv("ETCD_MAX_RESULT_BUFFER", "512") + os.Setenv("ETCD_MAX_RETRY_ATTEMPTS", "5") + os.Setenv("ETCD_NAME", "test-name") + os.Setenv("ETCD_SNAPSHOT", "true") + os.Setenv("ETCD_VERBOSE", "1") + os.Setenv("ETCD_VERY_VERBOSE", "yes") + os.Setenv("ETCD_WEB_URL", "/web") + os.Setenv("ETCD_PEER_ADVERTISED_URL", "127.0.0.1:7002") + os.Setenv("ETCD_PEER_CA_FILE", "/tmp/peer/file.ca") + os.Setenv("ETCD_PEER_CERT_FILE", "/tmp/peer/file.cert") + os.Setenv("ETCD_PEER_KEY_FILE", "/tmp/peer/file.key") + os.Setenv("ETCD_PEER_LISTEN_HOST", "127.0.0.1:7003") + + c := NewConfig() + c.LoadEnv() + assert.Equal(t, c.CAFile, "/tmp/file.ca", "") + assert.Equal(t, c.CertFile, "/tmp/file.cert", "") + assert.Equal(t, c.Cors, []string{"localhost:4001", "localhost:4002"}, "") + assert.Equal(t, c.DataDir, "/tmp/data", "") + assert.Equal(t, c.KeyFile, "/tmp/file.key", "") + assert.Equal(t, c.ListenHost, "127.0.0.1:4003", "") + assert.Equal(t, c.Machines, []string{"coreos.com:4001", "coreos.com:4002"}, "") + assert.Equal(t, c.MachinesFile, "/tmp/machines", "") + assert.Equal(t, c.MaxClusterSize, 10, "") + assert.Equal(t, c.MaxResultBuffer, 512, "") + assert.Equal(t, c.MaxRetryAttempts, 5, "") + assert.Equal(t, c.Name, "test-name", "") + assert.Equal(t, c.Snapshot, true, "") + assert.Equal(t, c.Verbose, true, "") + assert.Equal(t, c.VeryVerbose, true, "") + assert.Equal(t, c.WebURL, "/web", "") + assert.Equal(t, c.Peer.AdvertisedUrl, "127.0.0.1:7002", "") + assert.Equal(t, c.Peer.CAFile, "/tmp/peer/file.ca", "") + assert.Equal(t, c.Peer.CertFile, "/tmp/peer/file.cert", "") + assert.Equal(t, c.Peer.KeyFile, "/tmp/peer/file.key", "") + assert.Equal(t, c.Peer.ListenHost, "127.0.0.1:7003", "") +} + +// Ensures that a the advertised url can be parsed from the environment. +func TestConfigAdvertisedUrlEnv(t *testing.T) { + withEnv("ETCD_ADVERTISED_URL", "127.0.0.1:4002", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.AdvertisedUrl, "127.0.0.1:4002", "") + }) +} + +// Ensures that a the advertised flag can be parsed. +func TestConfigAdvertisedUrlFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-c", "127.0.0.1:4002"}), "") + assert.Equal(t, c.AdvertisedUrl, "127.0.0.1:4002", "") +} + +// Ensures that a the CA file can be parsed from the environment. +func TestConfigCAFileEnv(t *testing.T) { + withEnv("ETCD_CA_FILE", "/tmp/file.ca", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.CAFile, "/tmp/file.ca", "") + }) +} + +// Ensures that a the CA file flag can be parsed. +func TestConfigCAFileFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-clientCAFile", "/tmp/file.ca"}), "") + assert.Equal(t, c.CAFile, "/tmp/file.ca", "") +} + +// Ensures that a the CA file can be parsed from the environment. +func TestConfigCertFileEnv(t *testing.T) { + withEnv("ETCD_CERT_FILE", "/tmp/file.cert", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.CertFile, "/tmp/file.cert", "") + }) +} + +// Ensures that a the Cert file flag can be parsed. +func TestConfigCertFileFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-clientCert", "/tmp/file.cert"}), "") + assert.Equal(t, c.CertFile, "/tmp/file.cert", "") +} + +// Ensures that a the Key file can be parsed from the environment. +func TestConfigKeyFileEnv(t *testing.T) { + withEnv("ETCD_KEY_FILE", "/tmp/file.key", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.KeyFile, "/tmp/file.key", "") + }) +} + +// Ensures that a the Key file flag can be parsed. +func TestConfigKeyFileFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-clientKey", "/tmp/file.key"}), "") + assert.Equal(t, c.KeyFile, "/tmp/file.key", "") +} + +// Ensures that a the Listen Host can be parsed from the environment. +func TestConfigListenHostEnv(t *testing.T) { + withEnv("ETCD_LISTEN_HOST", "127.0.0.1:4003", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.ListenHost, "127.0.0.1:4003", "") + }) +} + +// Ensures that a the Listen Host file flag can be parsed. +func TestConfigListenHostFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-cl", "127.0.0.1:4003"}), "") + assert.Equal(t, c.ListenHost, "127.0.0.1:4003", "") +} + +// Ensures that the Machines can be parsed from the environment. +func TestConfigMachinesEnv(t *testing.T) { + withEnv("ETCD_MACHINES", "coreos.com:4001,coreos.com:4002", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Machines, []string{"coreos.com:4001", "coreos.com:4002"}, "") + }) +} + +// Ensures that a the Machines flag can be parsed. +func TestConfigMachinesFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-C", "coreos.com:4001,coreos.com:4002"}), "") + assert.Equal(t, c.Machines, []string{"coreos.com:4001", "coreos.com:4002"}, "") +} + +// Ensures that the Machines File can be parsed from the environment. +func TestConfigMachinesFileEnv(t *testing.T) { + withEnv("ETCD_MACHINES_FILE", "/tmp/machines", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.MachinesFile, "/tmp/machines", "") + }) +} + +// Ensures that a the Machines File flag can be parsed. +func TestConfigMachinesFileFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-CF", "/tmp/machines"}), "") + assert.Equal(t, c.MachinesFile, "/tmp/machines", "") +} + +// Ensures that the Max Cluster Size can be parsed from the environment. +func TestConfigMaxClusterSizeEnv(t *testing.T) { + withEnv("ETCD_MAX_CLUSTER_SIZE", "5", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.MaxClusterSize, 5, "") + }) +} + +// Ensures that a the Max Cluster Size flag can be parsed. +func TestConfigMaxClusterSizeFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-maxsize", "5"}), "") + assert.Equal(t, c.MaxClusterSize, 5, "") +} + +// Ensures that the Max Result Buffer can be parsed from the environment. +func TestConfigMaxResultBufferEnv(t *testing.T) { + withEnv("ETCD_MAX_RESULT_BUFFER", "512", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.MaxResultBuffer, 512, "") + }) +} + +// Ensures that a the Max Result Buffer flag can be parsed. +func TestConfigMaxResultBufferFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-m", "512"}), "") + assert.Equal(t, c.MaxResultBuffer, 512, "") +} + +// Ensures that the Max Retry Attempts can be parsed from the environment. +func TestConfigMaxRetryAttemptsEnv(t *testing.T) { + withEnv("ETCD_MAX_RETRY_ATTEMPTS", "10", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.MaxRetryAttempts, 10, "") + }) +} + +// Ensures that a the Max Retry Attempts flag can be parsed. +func TestConfigMaxRetryAttemptsFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-r", "10"}), "") + assert.Equal(t, c.MaxRetryAttempts, 10, "") +} + +// Ensures that the Name can be parsed from the environment. +func TestConfigNameEnv(t *testing.T) { + withEnv("ETCD_NAME", "test-name", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Name, "test-name", "") + }) +} + +// Ensures that a the Name flag can be parsed. +func TestConfigNameFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-n", "test-name"}), "") + assert.Equal(t, c.Name, "test-name", "") +} + +// Ensures that Snapshot can be parsed from the environment. +func TestConfigSnapshotEnv(t *testing.T) { + withEnv("ETCD_SNAPSHOT", "1", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Snapshot, true, "") + }) +} + +// Ensures that a the Snapshot flag can be parsed. +func TestConfigSnapshotFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-snapshot"}), "") + assert.Equal(t, c.Snapshot, true, "") +} + +// Ensures that Verbose can be parsed from the environment. +func TestConfigVerboseEnv(t *testing.T) { + withEnv("ETCD_VERBOSE", "true", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Verbose, true, "") + }) +} + +// Ensures that a the Verbose flag can be parsed. +func TestConfigVerboseFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-v"}), "") + assert.Equal(t, c.Verbose, true, "") +} + +// Ensures that Very Verbose can be parsed from the environment. +func TestConfigVeryVerboseEnv(t *testing.T) { + withEnv("ETCD_VERY_VERBOSE", "true", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.VeryVerbose, true, "") + }) +} + +// Ensures that a the Very Verbose flag can be parsed. +func TestConfigVeryVerboseFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-vv"}), "") + assert.Equal(t, c.VeryVerbose, true, "") +} + +// Ensures that Web URL can be parsed from the environment. +func TestConfigWebURLEnv(t *testing.T) { + withEnv("ETCD_WEB_URL", "/web", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.WebURL, "/web", "") + }) +} + +// Ensures that a the Web URL flag can be parsed. +func TestConfigWebURLFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-w", "/web"}), "") + assert.Equal(t, c.WebURL, "/web", "") +} + +// Ensures that the Peer Advertised URL can be parsed from the environment. +func TestConfigPeerAdvertisedUrlEnv(t *testing.T) { + withEnv("ETCD_PEER_ADVERTISED_URL", "localhost:7002", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Peer.AdvertisedUrl, "localhost:7002", "") + }) +} + +// Ensures that a the Peer Advertised URL flag can be parsed. +func TestConfigPeerAdvertisedUrlFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-s", "localhost:7002"}), "") + assert.Equal(t, c.Peer.AdvertisedUrl, "localhost:7002", "") +} + +// Ensures that the Peer CA File can be parsed from the environment. +func TestConfigPeerCAFileEnv(t *testing.T) { + withEnv("ETCD_PEER_CA_FILE", "/tmp/peer/file.ca", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Peer.CAFile, "/tmp/peer/file.ca", "") + }) +} + +// Ensures that a the Peer CA file flag can be parsed. +func TestConfigPeerCAFileFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-serverCAFile", "/tmp/peer/file.ca"}), "") + assert.Equal(t, c.Peer.CAFile, "/tmp/peer/file.ca", "") +} + +// Ensures that the Peer Cert File can be parsed from the environment. +func TestConfigPeerCertFileEnv(t *testing.T) { + withEnv("ETCD_PEER_CERT_FILE", "/tmp/peer/file.cert", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Peer.CertFile, "/tmp/peer/file.cert", "") + }) +} + +// Ensures that a the Cert file flag can be parsed. +func TestConfigPeerCertFileFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-serverCert", "/tmp/peer/file.cert"}), "") + assert.Equal(t, c.Peer.CertFile, "/tmp/peer/file.cert", "") +} + +// Ensures that the Peer Key File can be parsed from the environment. +func TestConfigPeerKeyFileEnv(t *testing.T) { + withEnv("ETCD_PEER_KEY_FILE", "/tmp/peer/file.key", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Peer.KeyFile, "/tmp/peer/file.key", "") + }) +} + +// Ensures that a the Peer Key file flag can be parsed. +func TestConfigPeerKeyFileFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-serverKey", "/tmp/peer/file.key"}), "") + assert.Equal(t, c.Peer.KeyFile, "/tmp/peer/file.key", "") +} + +// Ensures that the Peer Listen Host can be parsed from the environment. +func TestConfigPeerListenHostEnv(t *testing.T) { + withEnv("ETCD_PEER_LISTEN_HOST", "localhost:7004", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Peer.ListenHost, "localhost:7004", "") + }) +} + +// Ensures that a the Peer Listen Host file flag can be parsed. +func TestConfigPeerListenHostFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-sl", "127.0.0.1:4003"}), "") + assert.Equal(t, c.Peer.ListenHost, "127.0.0.1:4003", "") +} + + +// Ensures that a system config field is overridden by a custom config field. +func TestConfigCustomConfigOverrideSystemConfig(t *testing.T) { + system := `advertised_url = "127.0.0.1:5000"` + custom := `advertised_url = "127.0.0.1:6000"` + withTempFile(system, func(p1 string) { + withTempFile(custom, func(p2 string) { + c := NewConfig() + c.SystemPath = p1 + assert.Nil(t, c.Load([]string{"-config", p2}), "") + assert.Equal(t, c.AdvertisedUrl, "http://127.0.0.1:6000", "") + }) + }) +} + +// Ensures that a custom config field is overridden by an environment variable. +func TestConfigEnvVarOverrideCustomConfig(t *testing.T) { + os.Setenv("ETCD_PEER_ADVERTISED_URL", "127.0.0.1:8000") + defer os.Setenv("ETCD_PEER_ADVERTISED_URL", "") + + custom := `[peer]`+"\n"+`advertised_url = "127.0.0.1:9000"` + withTempFile(custom, func(path string) { + c := NewConfig() + c.SystemPath = "" + assert.Nil(t, c.Load([]string{"-config", path}), "") + assert.Equal(t, c.Peer.AdvertisedUrl, "http://127.0.0.1:8000", "") + }) +} + +// Ensures that an environment variable field is overridden by a command line argument. +func TestConfigCLIArgsOverrideEnvVar(t *testing.T) { + os.Setenv("ETCD_ADVERTISED_URL", "127.0.0.1:1000") + defer os.Setenv("ETCD_ADVERTISED_URL", "") + + c := NewConfig() + c.SystemPath = "" + assert.Nil(t, c.Load([]string{"-c", "127.0.0.1:2000"}), "") + assert.Equal(t, c.AdvertisedUrl, "http://127.0.0.1:2000", "") +} + + +//-------------------------------------- +// Helpers +//-------------------------------------- + +// Sets up the environment with a given environment variable set. +func withEnv(key, value string, f func(c *Config)) { + os.Setenv(key, value) + defer os.Setenv(key, "") + c := NewConfig() + f(c) +} + +// Creates a temp file and calls a function with the context. +func withTempFile(content string, fn func(string)) { + f, _ := ioutil.TempFile("", "") + f.WriteString(content) + f.Close() + defer os.Remove(f.Name()) + fn(f.Name()) +} diff --git a/server/info.go b/server/info.go new file mode 100644 index 00000000000..434f24a03e9 --- /dev/null +++ b/server/info.go @@ -0,0 +1,19 @@ +package server + +// Info describes the non-mutable state of the server upon initialization. +// These fields cannot be changed without deleting the server fields and +// reinitializing. +type Info struct { + Name string `json:"name"` + + RaftURL string `json:"raftURL"` + EtcdURL string `json:"etcdURL"` + WebURL string `json:"webURL"` + + RaftListenHost string `json:"raftListenHost"` + EtcdListenHost string `json:"etcdListenHost"` + + RaftTLS TLSInfo `json:"raftTLS"` + EtcdTLS TLSInfo `json:"etcdTLS"` +} + diff --git a/server/server.go b/server/server.go index f6a8779a2ac..8566d2ebaf5 100644 --- a/server/server.go +++ b/server/server.go @@ -283,10 +283,10 @@ func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Reque } // Sets a comma-delimited list of origins that are allowed. -func (s *Server) AllowOrigins(origins string) error { +func (s *Server) AllowOrigins(origins []string) error { // Construct a lookup of all origins. m := make(map[string]bool) - for _, v := range strings.Split(origins, ",") { + for _, v := range origins { if v != "*" { if _, err := url.Parse(v); err != nil { return fmt.Errorf("Invalid CORS origin: %s", err) diff --git a/server/tls_config.go b/server/tls_config.go index 4b944626d16..29b8fa5e4ff 100644 --- a/server/tls_config.go +++ b/server/tls_config.go @@ -4,6 +4,7 @@ import ( "crypto/tls" ) +// TLSConfig holds the TLS configuration. type TLSConfig struct { Scheme string Server tls.Config diff --git a/server/tls_info.go b/server/tls_info.go index 91936b0906e..1867ef44f09 100644 --- a/server/tls_info.go +++ b/server/tls_info.go @@ -1,7 +1,76 @@ package server +import ( + "crypto/tls" + "crypto/x509" + "encoding/pem" + "errors" + "io/ioutil" +) + +// TLSInfo holds the SSL certificates paths. type TLSInfo struct { CertFile string `json:"CertFile"` KeyFile string `json:"KeyFile"` CAFile string `json:"CAFile"` } + +// Generates a TLS configuration from the given files. +func (info TLSInfo) Config() (TLSConfig, error) { + var t TLSConfig + t.Scheme = "http" + + // If the user do not specify key file, cert file and CA file, the type will be HTTP + if info.KeyFile == "" && info.CertFile == "" && info.CAFile == "" { + return t, nil + } + + // Both the key and cert must be present. + if info.KeyFile == "" || info.CertFile == "" { + return t, errors.New("KeyFile and CertFile must both be present") + } + + tlsCert, err := tls.LoadX509KeyPair(info.CertFile, info.KeyFile) + if err != nil { + return t, err + } + + t.Scheme = "https" + t.Server.ClientAuth, t.Server.ClientCAs, err = newCertPool(info.CAFile) + if err != nil { + return t, err + } + + // The client should trust the RootCA that the Server uses since + // everyone is a peer in the network. + t.Client.Certificates = []tls.Certificate{tlsCert} + t.Client.RootCAs = t.Server.ClientCAs + + return t, nil +} + +// newCertPool creates x509 certPool and corresponding Auth Type. +// If the given CAfile is valid, add the cert into the pool and verify the clients' +// certs against the cert in the pool. +// If the given CAfile is empty, do not verify the clients' cert. +// If the given CAfile is not valid, fatal. +func newCertPool(CAFile string) (tls.ClientAuthType, *x509.CertPool, error) { + if CAFile == "" { + return tls.NoClientCert, nil, nil + } + pemByte, err := ioutil.ReadFile(CAFile) + if err != nil { + return 0, nil, err + } + + block, pemByte := pem.Decode(pemByte) + cert, err := x509.ParseCertificate(block.Bytes) + if err != nil { + return 0, nil, err + } + + certPool := x509.NewCertPool() + certPool.AddCert(cert) + + return tls.RequireAndVerifyClientCert, certPool, nil +} diff --git a/server/transporter_test.go b/server/transporter_test.go index e83ea3400c4..989b2e25020 100644 --- a/server/transporter_test.go +++ b/server/transporter_test.go @@ -1,5 +1,6 @@ package server +/* import ( "crypto/tls" "fmt" @@ -59,3 +60,4 @@ func TestTransporterTimeout(t *testing.T) { } } +*/ diff --git a/server/util.go b/server/util.go index 578c048cc8d..ee22c298204 100644 --- a/server/util.go +++ b/server/util.go @@ -6,6 +6,7 @@ import ( "io" "net/http" "net/url" + "strings" "github.com/coreos/etcd/log" ) @@ -31,3 +32,16 @@ func redirect(hostname string, w http.ResponseWriter, req *http.Request) { log.Debugf("Redirect to %s", redirectURL.String()) http.Redirect(w, req, redirectURL.String(), http.StatusTemporaryRedirect) } + +// trimsplit slices s into all substrings separated by sep and returns a +// slice of the substrings between the separator with all leading and trailing +// white space removed, as defined by Unicode. +func trimsplit(s, sep string) []string { + raw := strings.Split(s, ",") + trimmed := make([]string, 0) + for _, r := range raw { + trimmed = append(trimmed, strings.TrimSpace(r)) + } + return trimmed +} + diff --git a/test.sh b/test.sh index 66f745ed49f..5cc633975f4 100755 --- a/test.sh +++ b/test.sh @@ -8,6 +8,9 @@ set -e export GOPATH="${PWD}" # Unit tests +go test -i ./server +go test -v ./server + go test -i ./server/v2/tests go test -v ./server/v2/tests diff --git a/third_party/deps b/third_party/deps index 0cb7ffea3df..76464bdfc9c 100755 --- a/third_party/deps +++ b/third_party/deps @@ -1,4 +1,5 @@ packages=" + github.com/BurntSushi/toml github.com/coreos/go-raft github.com/coreos/go-etcd github.com/coreos/go-log/log diff --git a/third_party/github.com/BurntSushi/toml/.gitignore b/third_party/github.com/BurntSushi/toml/.gitignore new file mode 100644 index 00000000000..55e90a1e5c7 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/.gitignore @@ -0,0 +1,4 @@ +TAGS +tags +.*.swp +tomlcheck/tomlcheck diff --git a/third_party/github.com/BurntSushi/toml/COMPATIBLE b/third_party/github.com/BurntSushi/toml/COMPATIBLE new file mode 100644 index 00000000000..21e0938caef --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/COMPATIBLE @@ -0,0 +1,3 @@ +Compatible with TOML version +[v0.2.0](https://github.com/mojombo/toml/blob/master/versions/toml-v0.2.0.md) + diff --git a/third_party/github.com/BurntSushi/toml/COPYING b/third_party/github.com/BurntSushi/toml/COPYING new file mode 100644 index 00000000000..5a8e332545f --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/COPYING @@ -0,0 +1,14 @@ + DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE + Version 2, December 2004 + + Copyright (C) 2004 Sam Hocevar + + Everyone is permitted to copy and distribute verbatim or modified + copies of this license document, and changing it is allowed as long + as the name is changed. + + DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. You just DO WHAT THE FUCK YOU WANT TO. + diff --git a/third_party/github.com/BurntSushi/toml/Makefile b/third_party/github.com/BurntSushi/toml/Makefile new file mode 100644 index 00000000000..e6adc3e9efc --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/Makefile @@ -0,0 +1,14 @@ +install: + go install + +fmt: + gofmt -w *.go */*.go + colcheck *.go */*.go + +tags: + find ./ -name '*.go' -print0 | xargs -0 gotags > TAGS + +push: + git push origin master + git push github master + diff --git a/third_party/github.com/BurntSushi/toml/README.md b/third_party/github.com/BurntSushi/toml/README.md new file mode 100644 index 00000000000..f8c847d66dd --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/README.md @@ -0,0 +1,163 @@ +# TOML parser for Go with reflection + +TOML stands for Tom's Obvious, Minimal Language. + +Spec: https://github.com/mojombo/toml + +Compatible with TOML version +[v0.2.0](https://github.com/mojombo/toml/blob/master/versions/toml-v0.2.0.md) + +Documentation: http://godoc.org/github.com/BurntSushi/toml + +Installation: + +```bash +go get github.com/BurntSushi/toml +``` + +Try the toml validator: + +```bash +go get github.com/BurntSushi/toml/tomlv +tomlv some-toml-file.toml +``` + + +## Testing + +This package passes all tests in +[toml-test](https://github.com/BurntSushi/toml-test). + +## Examples + +This package works similarly to how the Go standard library handles `XML` +and `JSON`. Namely, data is loaded into Go values via reflection. + +For the simplest example, consider some TOML file as just a list of keys +and values: + +```toml +Age = 25 +Cats = [ "Cauchy", "Plato" ] +Pi = 3.14 +Perfection = [ 6, 28, 496, 8128 ] +DOB = 1987-07-05T05:45:00Z +``` + +Which could be defined in Go as: + +```go +type Config struct { + Age int + Cats []string + Pi float64 + Perfection []int + DOB time.Time // requires `import time` +} +``` + +And then decoded with: + +```go +var conf Config +if _, err := toml.Decode(tomlData, &conf); err != nil { + // handle error +} +``` + +You can also use struct tags if your struct field name doesn't map to a TOML +key value directly: + +```toml +some_key_NAME = "wat" +``` + +```go +type TOML struct { + ObscureKey string `toml:"some_key_NAME"` +} +``` + +## More complex usage + +Here's an example of how to load the example from the official spec page: + +```toml +# This is a TOML document. Boom. + +title = "TOML Example" + +[owner] +name = "Tom Preston-Werner" +organization = "GitHub" +bio = "GitHub Cofounder & CEO\nLikes tater tots and beer." +dob = 1979-05-27T07:32:00Z # First class dates? Why not? + +[database] +server = "192.168.1.1" +ports = [ 8001, 8001, 8002 ] +connection_max = 5000 +enabled = true + +[servers] + + # You can indent as you please. Tabs or spaces. TOML don't care. + [servers.alpha] + ip = "10.0.0.1" + dc = "eqdc10" + + [servers.beta] + ip = "10.0.0.2" + dc = "eqdc10" + +[clients] +data = [ ["gamma", "delta"], [1, 2] ] # just an update to make sure parsers support it + +# Line breaks are OK when inside arrays +hosts = [ + "alpha", + "omega" +] +``` + +And the corresponding Go types are: + +```go +type tomlConfig struct { + Title string + Owner ownerInfo + DB database `toml:"database"` + Servers map[string]server + Clients clients +} + +type ownerInfo struct { + Name string + Org string `toml:"organization"` + Bio string + DOB time.Time +} + +type database struct { + Server string + Ports []int + ConnMax int `toml:"connection_max"` + Enabled bool +} + +type server struct { + IP string + DC string +} + +type clients struct { + Data [][]interface{} + Hosts []string +} +``` + +Note that a case insensitive match will be tried if an exact match can't be +found. + +A working example of the above can be found in `_examples/example.{go,toml}`. + diff --git a/third_party/github.com/BurntSushi/toml/_examples/example.go b/third_party/github.com/BurntSushi/toml/_examples/example.go new file mode 100644 index 00000000000..c81d25a5248 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/_examples/example.go @@ -0,0 +1,59 @@ +package main + +import ( + "fmt" + "time" + + "github.com/BurntSushi/toml" +) + +type tomlConfig struct { + Title string + Owner ownerInfo + DB database `toml:"database"` + Servers map[string]server + Clients clients +} + +type ownerInfo struct { + Name string + Org string `toml:"organization"` + Bio string + DOB time.Time +} + +type database struct { + Server string + Ports []int + ConnMax int `toml:"connection_max"` + Enabled bool +} + +type server struct { + IP string + DC string +} + +type clients struct { + Data [][]interface{} + Hosts []string +} + +func main() { + var config tomlConfig + if _, err := toml.DecodeFile("example.toml", &config); err != nil { + fmt.Println(err) + return + } + + fmt.Printf("Title: %s\n", config.Title) + fmt.Printf("Owner: %s (%s, %s), Born: %s\n", + config.Owner.Name, config.Owner.Org, config.Owner.Bio, config.Owner.DOB) + fmt.Printf("Database: %s %v (Max conn. %d), Enabled? %v\n", + config.DB.Server, config.DB.Ports, config.DB.ConnMax, config.DB.Enabled) + for serverName, server := range config.Servers { + fmt.Printf("Server: %s (%s, %s)\n", serverName, server.IP, server.DC) + } + fmt.Printf("Client data: %v\n", config.Clients.Data) + fmt.Printf("Client hosts: %v\n", config.Clients.Hosts) +} diff --git a/third_party/github.com/BurntSushi/toml/_examples/example.toml b/third_party/github.com/BurntSushi/toml/_examples/example.toml new file mode 100644 index 00000000000..32c7a4faa46 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/_examples/example.toml @@ -0,0 +1,35 @@ +# This is a TOML document. Boom. + +title = "TOML Example" + +[owner] +name = "Tom Preston-Werner" +organization = "GitHub" +bio = "GitHub Cofounder & CEO\nLikes tater tots and beer." +dob = 1979-05-27T07:32:00Z # First class dates? Why not? + +[database] +server = "192.168.1.1" +ports = [ 8001, 8001, 8002 ] +connection_max = 5000 +enabled = true + +[servers] + + # You can indent as you please. Tabs or spaces. TOML don't care. + [servers.alpha] + ip = "10.0.0.1" + dc = "eqdc10" + + [servers.beta] + ip = "10.0.0.2" + dc = "eqdc10" + +[clients] +data = [ ["gamma", "delta"], [1, 2] ] # just an update to make sure parsers support it + +# Line breaks are OK when inside arrays +hosts = [ + "alpha", + "omega" +] diff --git a/third_party/github.com/BurntSushi/toml/_examples/hard.toml b/third_party/github.com/BurntSushi/toml/_examples/hard.toml new file mode 100644 index 00000000000..26145d2b42f --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/_examples/hard.toml @@ -0,0 +1,22 @@ +# Test file for TOML +# Only this one tries to emulate a TOML file written by a user of the kind of parser writers probably hate +# This part you'll really hate + +[the] +test_string = "You'll hate me after this - #" # " Annoying, isn't it? + + [the.hard] + test_array = [ "] ", " # "] # ] There you go, parse this! + test_array2 = [ "Test #11 ]proved that", "Experiment #9 was a success" ] + # You didn't think it'd as easy as chucking out the last #, did you? + another_test_string = " Same thing, but with a string #" + harder_test_string = " And when \"'s are in the string, along with # \"" # "and comments are there too" + # Things will get harder + + [the.hard.bit#] + what? = "You don't think some user won't do that?" + multi_line_array = [ + "]", + # ] Oh yes I did + ] + diff --git a/third_party/github.com/BurntSushi/toml/_examples/implicit.toml b/third_party/github.com/BurntSushi/toml/_examples/implicit.toml new file mode 100644 index 00000000000..1dea5ceb445 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/_examples/implicit.toml @@ -0,0 +1,4 @@ +# [x] you +# [x.y] don't +# [x.y.z] need these +[x.y.z.w] # for this to work diff --git a/third_party/github.com/BurntSushi/toml/_examples/invalid-apples.toml b/third_party/github.com/BurntSushi/toml/_examples/invalid-apples.toml new file mode 100644 index 00000000000..74e9e337edd --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/_examples/invalid-apples.toml @@ -0,0 +1,6 @@ +# DO NOT WANT +[fruit] +type = "apple" + +[fruit.type] +apple = "yes" diff --git a/third_party/github.com/BurntSushi/toml/_examples/invalid.toml b/third_party/github.com/BurntSushi/toml/_examples/invalid.toml new file mode 100644 index 00000000000..beb1dba54dd --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/_examples/invalid.toml @@ -0,0 +1,35 @@ +# This is an INVALID TOML document. Boom. +# Can you spot the error without help? + +title = "TOML Example" + +[owner] +name = "Tom Preston-Werner" +organization = "GitHub" +bio = "GitHub Cofounder & CEO\nLikes tater tots and beer." +dob = 1979-05-27T7:32:00Z # First class dates? Why not? + +[database] +server = "192.168.1.1" +ports = [ 8001, 8001, 8002 ] +connection_max = 5000 +enabled = true + +[servers] + # You can indent as you please. Tabs or spaces. TOML don't care. + [servers.alpha] + ip = "10.0.0.1" + dc = "eqdc10" + + [servers.beta] + ip = "10.0.0.2" + dc = "eqdc10" + +[clients] +data = [ ["gamma", "delta"], [1, 2] ] # just an update to make sure parsers support it + +# Line breaks are OK when inside arrays +hosts = [ + "alpha", + "omega" +] diff --git a/third_party/github.com/BurntSushi/toml/_examples/readme1.toml b/third_party/github.com/BurntSushi/toml/_examples/readme1.toml new file mode 100644 index 00000000000..3e1261d4c23 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/_examples/readme1.toml @@ -0,0 +1,5 @@ +Age = 25 +Cats = [ "Cauchy", "Plato" ] +Pi = 3.14 +Perfection = [ 6, 28, 496, 8128 ] +DOB = 1987-07-05T05:45:00Z diff --git a/third_party/github.com/BurntSushi/toml/_examples/readme2.toml b/third_party/github.com/BurntSushi/toml/_examples/readme2.toml new file mode 100644 index 00000000000..b51cd93408a --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/_examples/readme2.toml @@ -0,0 +1 @@ +some_key_NAME = "wat" diff --git a/third_party/github.com/BurntSushi/toml/decode.go b/third_party/github.com/BurntSushi/toml/decode.go new file mode 100644 index 00000000000..ddb26cf2be5 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/decode.go @@ -0,0 +1,429 @@ +package toml + +import ( + "fmt" + "io" + "io/ioutil" + "reflect" + "strings" + "time" +) + +var e = fmt.Errorf + +// Primitive is a TOML value that hasn't been decoded into a Go value. +// When using the various `Decode*` functions, the type `Primitive` may +// be given to any value, and its decoding will be delayed. +// +// A `Primitive` value can be decoded using the `PrimitiveDecode` function. +// +// The underlying representation of a `Primitive` value is subject to change. +// Do not rely on it. +// +// N.B. Primitive values are still parsed, so using them will only avoid +// the overhead of reflection. They can be useful when you don't know the +// exact type of TOML data until run time. +type Primitive interface{} + +// PrimitiveDecode is just like the other `Decode*` functions, except it +// decodes a TOML value that has already been parsed. Valid primitive values +// can *only* be obtained from values filled by the decoder functions, +// including `PrimitiveDecode`. (i.e., `v` may contain more `Primitive` +// values.) +// +// Meta data for primitive values is included in the meta data returned by +// the `Decode*` functions. +func PrimitiveDecode(primValue Primitive, v interface{}) error { + return unify(primValue, rvalue(v)) +} + +// Decode will decode the contents of `data` in TOML format into a pointer +// `v`. +// +// TOML hashes correspond to Go structs or maps. (Dealer's choice. They can be +// used interchangeably.) +// +// TOML datetimes correspond to Go `time.Time` values. +// +// All other TOML types (float, string, int, bool and array) correspond +// to the obvious Go types. +// +// TOML keys can map to either keys in a Go map or field names in a Go +// struct. The special `toml` struct tag may be used to map TOML keys to +// struct fields that don't match the key name exactly. (See the example.) +// A case insensitive match to struct names will be tried if an exact match +// can't be found. +// +// The mapping between TOML values and Go values is loose. That is, there +// may exist TOML values that cannot be placed into your representation, and +// there may be parts of your representation that do not correspond to +// TOML values. +// +// This decoder will not handle cyclic types. If a cyclic type is passed, +// `Decode` will not terminate. +func Decode(data string, v interface{}) (MetaData, error) { + p, err := parse(data) + if err != nil { + return MetaData{}, err + } + return MetaData{p.mapping, p.types, p.ordered}, unify(p.mapping, rvalue(v)) +} + +// DecodeFile is just like Decode, except it will automatically read the +// contents of the file at `fpath` and decode it for you. +func DecodeFile(fpath string, v interface{}) (MetaData, error) { + bs, err := ioutil.ReadFile(fpath) + if err != nil { + return MetaData{}, err + } + return Decode(string(bs), v) +} + +// DecodeReader is just like Decode, except it will consume all bytes +// from the reader and decode it for you. +func DecodeReader(r io.Reader, v interface{}) (MetaData, error) { + bs, err := ioutil.ReadAll(r) + if err != nil { + return MetaData{}, err + } + return Decode(string(bs), v) +} + +// unify performs a sort of type unification based on the structure of `rv`, +// which is the client representation. +// +// Any type mismatch produces an error. Finding a type that we don't know +// how to handle produces an unsupported type error. +func unify(data interface{}, rv reflect.Value) error { + // Special case. Look for a `Primitive` value. + if rv.Type() == reflect.TypeOf((*Primitive)(nil)).Elem() { + return unifyAnything(data, rv) + } + + // Special case. Go's `time.Time` is a struct, which we don't want + // to confuse with a user struct. + if rv.Type().AssignableTo(rvalue(time.Time{}).Type()) { + return unifyDatetime(data, rv) + } + + k := rv.Kind() + + // laziness + if k >= reflect.Int && k <= reflect.Uint64 { + return unifyInt(data, rv) + } + switch k { + case reflect.Ptr: + elem := reflect.New(rv.Type().Elem()) + err := unify(data, reflect.Indirect(elem)) + if err != nil { + return err + } + rv.Set(elem) + return nil + case reflect.Struct: + return unifyStruct(data, rv) + case reflect.Map: + return unifyMap(data, rv) + case reflect.Slice: + return unifySlice(data, rv) + case reflect.String: + return unifyString(data, rv) + case reflect.Bool: + return unifyBool(data, rv) + case reflect.Interface: + // we only support empty interfaces. + if rv.NumMethod() > 0 { + return e("Unsupported type '%s'.", rv.Kind()) + } + return unifyAnything(data, rv) + case reflect.Float32: + fallthrough + case reflect.Float64: + return unifyFloat64(data, rv) + } + return e("Unsupported type '%s'.", rv.Kind()) +} + +func unifyStruct(mapping interface{}, rv reflect.Value) error { + tmap, ok := mapping.(map[string]interface{}) + if !ok { + return mismatch(rv, "map", mapping) + } + + rt := rv.Type() + for i := 0; i < rt.NumField(); i++ { + // A little tricky. We want to use the special `toml` name in the + // struct tag if it exists. In particular, we need to make sure that + // this struct field is in the current map before trying to unify it. + sft := rt.Field(i) + kname := sft.Tag.Get("toml") + if len(kname) == 0 { + kname = sft.Name + } + if datum, ok := insensitiveGet(tmap, kname); ok { + sf := indirect(rv.Field(i)) + + // Don't try to mess with unexported types and other such things. + if sf.CanSet() { + if err := unify(datum, sf); err != nil { + return e("Type mismatch for '%s.%s': %s", + rt.String(), sft.Name, err) + } + } else if len(sft.Tag.Get("toml")) > 0 { + // Bad user! No soup for you! + return e("Field '%s.%s' is unexported, and therefore cannot "+ + "be loaded with reflection.", rt.String(), sft.Name) + } + } + } + return nil +} + +func unifyMap(mapping interface{}, rv reflect.Value) error { + tmap, ok := mapping.(map[string]interface{}) + if !ok { + return badtype("map", mapping) + } + if rv.IsNil() { + rv.Set(reflect.MakeMap(rv.Type())) + } + for k, v := range tmap { + rvkey := indirect(reflect.New(rv.Type().Key())) + rvval := reflect.Indirect(reflect.New(rv.Type().Elem())) + if err := unify(v, rvval); err != nil { + return err + } + + rvkey.SetString(k) + rv.SetMapIndex(rvkey, rvval) + } + return nil +} + +func unifySlice(data interface{}, rv reflect.Value) error { + datav := reflect.ValueOf(data) + if datav.Kind() != reflect.Slice { + return badtype("slice", data) + } + sliceLen := datav.Len() + if rv.IsNil() { + rv.Set(reflect.MakeSlice(rv.Type(), sliceLen, sliceLen)) + } + for i := 0; i < sliceLen; i++ { + v := datav.Index(i).Interface() + sliceval := indirect(rv.Index(i)) + if err := unify(v, sliceval); err != nil { + return err + } + } + return nil +} + +func unifyDatetime(data interface{}, rv reflect.Value) error { + if _, ok := data.(time.Time); ok { + rv.Set(reflect.ValueOf(data)) + return nil + } + return badtype("time.Time", data) +} + +func unifyString(data interface{}, rv reflect.Value) error { + if s, ok := data.(string); ok { + rv.SetString(s) + return nil + } + return badtype("string", data) +} + +func unifyFloat64(data interface{}, rv reflect.Value) error { + if num, ok := data.(float64); ok { + switch rv.Kind() { + case reflect.Float32: + fallthrough + case reflect.Float64: + rv.SetFloat(num) + default: + panic("bug") + } + return nil + } + return badtype("float", data) +} + +func unifyInt(data interface{}, rv reflect.Value) error { + if num, ok := data.(int64); ok { + switch rv.Kind() { + case reflect.Int: + fallthrough + case reflect.Int8: + fallthrough + case reflect.Int16: + fallthrough + case reflect.Int32: + fallthrough + case reflect.Int64: + rv.SetInt(int64(num)) + case reflect.Uint: + fallthrough + case reflect.Uint8: + fallthrough + case reflect.Uint16: + fallthrough + case reflect.Uint32: + fallthrough + case reflect.Uint64: + rv.SetUint(uint64(num)) + default: + panic("bug") + } + return nil + } + return badtype("integer", data) +} + +func unifyBool(data interface{}, rv reflect.Value) error { + if b, ok := data.(bool); ok { + rv.SetBool(b) + return nil + } + return badtype("integer", data) +} + +func unifyAnything(data interface{}, rv reflect.Value) error { + // too awesome to fail + rv.Set(reflect.ValueOf(data)) + return nil +} + +// rvalue returns a reflect.Value of `v`. All pointers are resolved. +func rvalue(v interface{}) reflect.Value { + return indirect(reflect.ValueOf(v)) +} + +// indirect returns the value pointed to by a pointer. +// Pointers are followed until the value is not a pointer. +// New values are allocated for each nil pointer. +func indirect(v reflect.Value) reflect.Value { + if v.Kind() != reflect.Ptr { + return v + } + if v.IsNil() { + v.Set(reflect.New(v.Type().Elem())) + } + return indirect(reflect.Indirect(v)) +} + +func tstring(rv reflect.Value) string { + return rv.Type().String() +} + +func badtype(expected string, data interface{}) error { + return e("Expected %s but found '%T'.", expected, data) +} + +func mismatch(user reflect.Value, expected string, data interface{}) error { + return e("Type mismatch for %s. Expected %s but found '%T'.", + tstring(user), expected, data) +} + +func insensitiveGet( + tmap map[string]interface{}, kname string) (interface{}, bool) { + + if datum, ok := tmap[kname]; ok { + return datum, true + } + for k, v := range tmap { + if strings.EqualFold(kname, k) { + return v, true + } + } + return nil, false +} + +// MetaData allows access to meta information about TOML data that may not +// be inferrable via reflection. In particular, whether a key has been defined +// and the TOML type of a key. +// +// (XXX: If TOML gets NULL values, that information will be added here too.) +type MetaData struct { + mapping map[string]interface{} + types map[string]tomlType + keys []Key +} + +// IsDefined returns true if the key given exists in the TOML data. The key +// should be specified hierarchially. e.g., +// +// // access the TOML key 'a.b.c' +// IsDefined("a", "b", "c") +// +// IsDefined will return false if an empty key given. Keys are case sensitive. +func (md MetaData) IsDefined(key ...string) bool { + var hashOrVal interface{} + var hash map[string]interface{} + var ok bool + + if len(key) == 0 { + return false + } + + hashOrVal = md.mapping + for _, k := range key { + if hash, ok = hashOrVal.(map[string]interface{}); !ok { + return false + } + if hashOrVal, ok = hash[k]; !ok { + return false + } + } + return true +} + +// Type returns a string representation of the type of the key specified. +// +// Type will return the empty string if given an empty key or a key that +// does not exist. Keys are case sensitive. +func (md MetaData) Type(key ...string) string { + fullkey := strings.Join(key, ".") + if typ, ok := md.types[fullkey]; ok { + return typ.typeString() + } + return "" +} + +// Key is the type of any TOML key, including key groups. Use (MetaData).Keys +// to get values of this type. +type Key []string + +func (k Key) String() string { + return strings.Join(k, ".") +} + +func (k Key) add(piece string) Key { + newKey := make(Key, len(k)) + copy(newKey, k) + return append(newKey, piece) +} + +// Keys returns a slice of every key in the TOML data, including key groups. +// Each key is itself a slice, where the first element is the top of the +// hierarchy and the last is the most specific. +// +// The list will have the same order as the keys appeared in the TOML data. +// +// All keys returned are non-empty. +func (md MetaData) Keys() []Key { + return md.keys +} + +func allKeys(m map[string]interface{}, context Key) []Key { + keys := make([]Key, 0, len(m)) + for k, v := range m { + keys = append(keys, context.add(k)) + if t, ok := v.(map[string]interface{}); ok { + keys = append(keys, allKeys(t, context.add(k))...) + } + } + return keys +} diff --git a/third_party/github.com/BurntSushi/toml/decode_test.go b/third_party/github.com/BurntSushi/toml/decode_test.go new file mode 100644 index 00000000000..ad80adee376 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/decode_test.go @@ -0,0 +1,343 @@ +package toml + +import ( + "fmt" + "log" + "reflect" + "testing" + "time" +) + +func init() { + log.SetFlags(0) +} + +var testSimple = ` +age = 250 + +andrew = "gallant" +kait = "brady" +now = 1987-07-05T05:45:00Z +yesOrNo = true +pi = 3.14 +colors = [ + ["red", "green", "blue"], + ["cyan", "magenta", "yellow", "black"], +] + +[Annoying.Cats] +plato = "smelly" +cauchy = "stupido" + +` + +type kitties struct { + Plato string + Cauchy string +} + +type simple struct { + Age int + Colors [][]string + Pi float64 + YesOrNo bool + Now time.Time + Andrew string + Kait string + Annoying map[string]kitties +} + +func TestDecode(t *testing.T) { + var val simple + + md, err := Decode(testSimple, &val) + if err != nil { + t.Fatal(err) + } + + testf("Is 'Annoying.Cats.plato' defined? %v\n", + md.IsDefined("Annoying", "Cats", "plato")) + testf("Is 'Cats.Stinky' defined? %v\n", md.IsDefined("Cats", "Stinky")) + testf("Type of 'colors'? %s\n\n", md.Type("colors")) + + testf("%v\n", val) +} + +var tomlTableArrays = ` +[[albums]] +name = "Born to Run" + + [[albums.songs]] + name = "Jungleland" + + [[albums.songs]] + name = "Meeting Across the River" + +[[albums]] +name = "Born in the USA" + + [[albums.songs]] + name = "Glory Days" + + [[albums.songs]] + name = "Dancing in the Dark" +` + +type Music struct { + Albums []Album +} + +type Album struct { + Name string + Songs []Song +} + +type Song struct { + Name string +} + +func TestTableArrays(t *testing.T) { + expected := Music{[]Album{ + {"Born to Run", []Song{{"Jungleland"}, {"Meeting Across the River"}}}, + {"Born in the USA", []Song{{"Glory Days"}, {"Dancing in the Dark"}}}, + }} + var got Music + if _, err := Decode(tomlTableArrays, &got); err != nil { + t.Fatal(err) + } + if !reflect.DeepEqual(expected, got) { + t.Fatalf("\n%#v\n!=\n%#v\n", expected, got) + } +} + +// Case insensitive matching tests. +// A bit more comprehensive than needed given the current implementation, +// but implementations change. +// Probably still missing demonstrations of some ugly corner cases regarding +// case insensitive matching and multiple fields. +var caseToml = ` +tOpString = "string" +tOpInt = 1 +tOpFloat = 1.1 +tOpBool = true +tOpdate = 2006-01-02T15:04:05Z +tOparray = [ "array" ] +Match = "i should be in Match only" +MatcH = "i should be in MatcH only" +Field = "neat" +FielD = "messy" +once = "just once" +[nEst.eD] +nEstedString = "another string" +` + +type Insensitive struct { + TopString string + TopInt int + TopFloat float64 + TopBool bool + TopDate time.Time + TopArray []string + Match string + MatcH string + Field string + Once string + OncE string + Nest InsensitiveNest +} + +type InsensitiveNest struct { + Ed InsensitiveEd +} + +type InsensitiveEd struct { + NestedString string +} + +func TestCase(t *testing.T) { + tme, err := time.Parse(time.RFC3339, time.RFC3339[:len(time.RFC3339)-5]) + if err != nil { + panic(err) + } + expected := Insensitive{ + TopString: "string", + TopInt: 1, + TopFloat: 1.1, + TopBool: true, + TopDate: tme, + TopArray: []string{"array"}, + MatcH: "i should be in MatcH only", + Match: "i should be in Match only", + Field: "neat", // encoding/json would store "messy" here + Once: "just once", + OncE: "just once", // wait, what? + Nest: InsensitiveNest{ + Ed: InsensitiveEd{NestedString: "another string"}, + }, + } + var got Insensitive + _, err = Decode(caseToml, &got) + if err != nil { + t.Fatal(err) + } + if !reflect.DeepEqual(expected, got) { + t.Fatalf("\n%#v\n!=\n%#v\n", expected, got) + } +} + +func TestPointers(t *testing.T) { + type Object struct { + Type string + Description string + } + + type Dict struct { + NamedObject map[string]*Object + BaseObject *Object + Strptr *string + Strptrs []*string + } + s1, s2, s3 := "blah", "abc", "def" + expected := &Dict{ + Strptr: &s1, + Strptrs: []*string{&s2, &s3}, + NamedObject: map[string]*Object{ + "foo": {"FOO", "fooooo!!!"}, + "bar": {"BAR", "ba-ba-ba-ba-barrrr!!!"}, + }, + BaseObject: &Object{"BASE", "da base"}, + } + + ex1 := ` +Strptr = "blah" +Strptrs = ["abc", "def"] + +[NamedObject.foo] +Type = "FOO" +Description = "fooooo!!!" + +[NamedObject.bar] +Type = "BAR" +Description = "ba-ba-ba-ba-barrrr!!!" + +[BaseObject] +Type = "BASE" +Description = "da base" +` + dict := new(Dict) + _, err := Decode(ex1, dict) + if err != nil { + t.Errorf("Decode error: %v", err) + } + if !reflect.DeepEqual(expected, dict) { + t.Fatalf("\n%#v\n!=\n%#v\n", expected, dict) + } +} + +func ExamplePrimitiveDecode() { + var md MetaData + var err error + + var tomlBlob = ` +ranking = ["Springsteen", "J Geils"] + +[bands.Springsteen] +started = 1973 +albums = ["Greetings", "WIESS", "Born to Run", "Darkness"] + +[bands.J Geils] +started = 1970 +albums = ["The J. Geils Band", "Full House", "Blow Your Face Out"] +` + + type band struct { + Started int + Albums []string + } + + type classics struct { + Ranking []string + Bands map[string]Primitive + } + + // Do the initial decode. Reflection is delayed on Primitive values. + var music classics + if md, err = Decode(tomlBlob, &music); err != nil { + log.Fatal(err) + } + + // MetaData still includes information on Primitive values. + fmt.Printf("Is `bands.Springsteen` defined? %v\n", + md.IsDefined("bands", "Springsteen")) + + // Decode primitive data into Go values. + for _, artist := range music.Ranking { + // A band is a primitive value, so we need to decode it to get a + // real `band` value. + primValue := music.Bands[artist] + + var aBand band + if err = PrimitiveDecode(primValue, &aBand); err != nil { + log.Fatal(err) + } + fmt.Printf("%s started in %d.\n", artist, aBand.Started) + } + + // Output: + // Is `bands.Springsteen` defined? true + // Springsteen started in 1973. + // J Geils started in 1970. +} + +func ExampleDecode() { + var tomlBlob = ` +# Some comments. +[alpha] +ip = "10.0.0.1" + + [alpha.config] + Ports = [ 8001, 8002 ] + Location = "Toronto" + Created = 1987-07-05T05:45:00Z + +[beta] +ip = "10.0.0.2" + + [beta.config] + Ports = [ 9001, 9002 ] + Location = "New Jersey" + Created = 1887-01-05T05:55:00Z +` + + type serverConfig struct { + Ports []int + Location string + Created time.Time + } + + type server struct { + IP string `toml:"ip"` + Config serverConfig `toml:"config"` + } + + type servers map[string]server + + var config servers + if _, err := Decode(tomlBlob, &config); err != nil { + log.Fatal(err) + } + + for _, name := range []string{"alpha", "beta"} { + s := config[name] + fmt.Printf("Server: %s (ip: %s) in %s created on %s\n", + name, s.IP, s.Config.Location, + s.Config.Created.Format("2006-01-02")) + fmt.Printf("Ports: %v\n", s.Config.Ports) + } + + // // Output: + // Server: alpha (ip: 10.0.0.1) in Toronto created on 1987-07-05 + // Ports: [8001 8002] + // Server: beta (ip: 10.0.0.2) in New Jersey created on 1887-01-05 + // Ports: [9001 9002] +} diff --git a/third_party/github.com/BurntSushi/toml/doc.go b/third_party/github.com/BurntSushi/toml/doc.go new file mode 100644 index 00000000000..1c2d7dffc21 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/doc.go @@ -0,0 +1,10 @@ +/* +Package toml provides facilities for decoding TOML configuration files +via reflection. + +Specification: https://github.com/mojombo/toml + +Use github.com/BurntSushi/toml/tomlv to check whether a file is valid +TOML or not, with helpful error messages. +*/ +package toml diff --git a/third_party/github.com/BurntSushi/toml/encode.go b/third_party/github.com/BurntSushi/toml/encode.go new file mode 100644 index 00000000000..7b080f414c6 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/encode.go @@ -0,0 +1,99 @@ +package toml + +// TODO: Build a decent encoder. +// Interestingly, this isn't as trivial as recursing down the type of the +// value given and outputting the corresponding TOML. In particular, multiple +// TOML types (especially if tuples are added) can map to a single Go type, so +// that the reverse correspondence isn't clear. +// +// One possible avenue is to choose a reasonable default (like structs map +// to hashes), but allow the user to override with struct tags. But this seems +// like a mess. +// +// The other possibility is to scrap an encoder altogether. After all, TOML +// is a configuration file format, and not a data exchange format. + +import ( + "bufio" + "fmt" + "io" + "reflect" + "strings" +) + +type encoder struct { + // A single indentation level. By default it is two spaces. + Indent string + + w *bufio.Writer +} + +func newEncoder(w io.Writer) *encoder { + return &encoder{ + w: bufio.NewWriter(w), + Indent: " ", + } +} + +func (enc *encoder) Encode(v interface{}) error { + rv := eindirect(reflect.ValueOf(v)) + if err := enc.encode(Key([]string{}), rv); err != nil { + return err + } + return enc.w.Flush() +} + +func (enc *encoder) encode(key Key, rv reflect.Value) error { + k := rv.Kind() + switch k { + case reflect.Struct: + return enc.eStruct(key, rv) + case reflect.String: + return enc.eString(key, rv) + } + return e("Unsupported type for key '%s': %s", key, k) +} + +func (enc *encoder) eStruct(key Key, rv reflect.Value) error { + rt := rv.Type() + for i := 0; i < rt.NumField(); i++ { + sft := rt.Field(i) + sf := rv.Field(i) + if err := enc.encode(key.add(sft.Name), sf); err != nil { + return err + } + } + return nil +} + +func (enc *encoder) eString(key Key, rv reflect.Value) error { + s := rv.String() + s = strings.NewReplacer( + "\t", "\\t", + "\n", "\\n", + "\r", "\\r", + "\"", "\\\"", + "\\", "\\\\", + ).Replace(s) + s = "\"" + s + "\"" + if err := enc.eKeyVal(key, s); err != nil { + return err + } + return nil +} + +func (enc *encoder) eKeyVal(key Key, value string) error { + out := fmt.Sprintf("%s%s = %s", + strings.Repeat(enc.Indent, len(key)-1), key[len(key)-1], value) + if _, err := fmt.Fprintln(enc.w, out); err != nil { + return err + } + return nil +} + +func eindirect(v reflect.Value) reflect.Value { + if v.Kind() != reflect.Ptr { + return v + } + return eindirect(reflect.Indirect(v)) +} diff --git a/third_party/github.com/BurntSushi/toml/encode_test.go b/third_party/github.com/BurntSushi/toml/encode_test.go new file mode 100644 index 00000000000..3f7fad8b4b1 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/encode_test.go @@ -0,0 +1,25 @@ +package toml + +import ( + "bytes" + "testing" +) + +type encodeSimple struct { + Location string + // Ages []int + // DOB time.Time +} + +func TestEncode(t *testing.T) { + v := encodeSimple{ + Location: "Westborough, MA", + } + + buf := new(bytes.Buffer) + e := newEncoder(buf) + if err := e.Encode(v); err != nil { + t.Fatal(err) + } + testf(buf.String()) +} diff --git a/third_party/github.com/BurntSushi/toml/lex.go b/third_party/github.com/BurntSushi/toml/lex.go new file mode 100644 index 00000000000..3dcae4923c6 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/lex.go @@ -0,0 +1,741 @@ +package toml + +import ( + "fmt" + "unicode/utf8" +) + +type itemType int + +const ( + itemError itemType = iota + itemNIL // used in the parser to indicate no type + itemEOF + itemText + itemString + itemBool + itemInteger + itemFloat + itemDatetime + itemArray // the start of an array + itemArrayEnd + itemTableStart + itemTableEnd + itemArrayTableStart + itemArrayTableEnd + itemKeyStart + itemCommentStart +) + +const ( + eof = 0 + tableStart = '[' + tableEnd = ']' + arrayTableStart = '[' + arrayTableEnd = ']' + tableSep = '.' + keySep = '=' + arrayStart = '[' + arrayEnd = ']' + arrayValTerm = ',' + commentStart = '#' + stringStart = '"' + stringEnd = '"' +) + +type stateFn func(lx *lexer) stateFn + +type lexer struct { + input string + start int + pos int + width int + line int + state stateFn + items chan item + + // A stack of state functions used to maintain context. + // The idea is to reuse parts of the state machine in various places. + // For example, values can appear at the top level or within arbitrarily + // nested arrays. The last state on the stack is used after a value has + // been lexed. Similarly for comments. + stack []stateFn +} + +type item struct { + typ itemType + val string + line int +} + +func (lx *lexer) nextItem() item { + for { + select { + case item := <-lx.items: + return item + default: + lx.state = lx.state(lx) + } + } + panic("not reached") +} + +func lex(input string) *lexer { + lx := &lexer{ + input: input, + state: lexTop, + line: 1, + items: make(chan item, 10), + stack: make([]stateFn, 0, 10), + } + return lx +} + +func (lx *lexer) push(state stateFn) { + lx.stack = append(lx.stack, state) +} + +func (lx *lexer) pop() stateFn { + if len(lx.stack) == 0 { + return lx.errorf("BUG in lexer: no states to pop.") + } + last := lx.stack[len(lx.stack)-1] + lx.stack = lx.stack[0 : len(lx.stack)-1] + return last +} + +func (lx *lexer) current() string { + return lx.input[lx.start:lx.pos] +} + +func (lx *lexer) emit(typ itemType) { + lx.items <- item{typ, lx.current(), lx.line} + lx.start = lx.pos +} + +func (lx *lexer) next() (r rune) { + if lx.pos >= len(lx.input) { + lx.width = 0 + return eof + } + + if lx.input[lx.pos] == '\n' { + lx.line++ + } + r, lx.width = utf8.DecodeRuneInString(lx.input[lx.pos:]) + lx.pos += lx.width + return r +} + +// ignore skips over the pending input before this point. +func (lx *lexer) ignore() { + lx.start = lx.pos +} + +// backup steps back one rune. Can be called only once per call of next. +func (lx *lexer) backup() { + lx.pos -= lx.width + if lx.pos < len(lx.input) && lx.input[lx.pos] == '\n' { + lx.line-- + } +} + +// accept consumes the next rune if it's equal to `valid`. +func (lx *lexer) accept(valid rune) bool { + if lx.next() == valid { + return true + } + lx.backup() + return false +} + +// peek returns but does not consume the next rune in the input. +func (lx *lexer) peek() rune { + r := lx.next() + lx.backup() + return r +} + +// errorf stops all lexing by emitting an error and returning `nil`. +// Note that any value that is a character is escaped if it's a special +// character (new lines, tabs, etc.). +func (lx *lexer) errorf(format string, values ...interface{}) stateFn { + for i, value := range values { + if v, ok := value.(rune); ok { + values[i] = escapeSpecial(v) + } + } + lx.items <- item{ + itemError, + fmt.Sprintf(format, values...), + lx.line, + } + return nil +} + +// lexTop consumes elements at the top level of TOML data. +func lexTop(lx *lexer) stateFn { + r := lx.next() + if isWhitespace(r) || isNL(r) { + return lexSkip(lx, lexTop) + } + + switch r { + case commentStart: + lx.push(lexTop) + return lexCommentStart + case tableStart: + return lexTableStart + case eof: + if lx.pos > lx.start { + return lx.errorf("Unexpected EOF.") + } + lx.emit(itemEOF) + return nil + } + + // At this point, the only valid item can be a key, so we back up + // and let the key lexer do the rest. + lx.backup() + lx.push(lexTopEnd) + return lexKeyStart +} + +// lexTopEnd is entered whenever a top-level item has been consumed. (A value +// or a table.) It must see only whitespace, and will turn back to lexTop +// upon a new line. If it sees EOF, it will quit the lexer successfully. +func lexTopEnd(lx *lexer) stateFn { + r := lx.next() + switch { + case r == commentStart: + // a comment will read to a new line for us. + lx.push(lexTop) + return lexCommentStart + case isWhitespace(r): + return lexTopEnd + case isNL(r): + lx.ignore() + return lexTop + case r == eof: + lx.ignore() + return lexTop + } + return lx.errorf("Expected a top-level item to end with a new line, "+ + "comment or EOF, but got '%s' instead.", r) +} + +// lexTable lexes the beginning of a table. Namely, it makes sure that +// it starts with a character other than '.' and ']'. +// It assumes that '[' has already been consumed. +// It also handles the case that this is an item in an array of tables. +// e.g., '[[name]]'. +func lexTableStart(lx *lexer) stateFn { + if lx.peek() == arrayTableStart { + lx.next() + lx.emit(itemArrayTableStart) + lx.push(lexArrayTableEnd) + } else { + lx.emit(itemTableStart) + lx.push(lexTableEnd) + } + return lexTableNameStart +} + +func lexTableEnd(lx *lexer) stateFn { + lx.emit(itemTableEnd) + return lexTopEnd +} + +func lexArrayTableEnd(lx *lexer) stateFn { + if r := lx.next(); r != arrayTableEnd { + return lx.errorf("Expected end of table array name delimiter '%s', "+ + "but got '%s' instead.", arrayTableEnd, r) + } + lx.emit(itemArrayTableEnd) + return lexTopEnd +} + +func lexTableNameStart(lx *lexer) stateFn { + switch lx.next() { + case tableEnd: + return lx.errorf("Unexpected end of table. (Tables cannot " + + "be empty.)") + case tableSep: + return lx.errorf("Unexpected table separator. (Tables cannot " + + "be empty.)") + } + return lexTableName +} + +// lexTableName lexes the name of a table. It assumes that at least one +// valid character for the table has already been read. +func lexTableName(lx *lexer) stateFn { + switch lx.peek() { + case tableStart: + return lx.errorf("Table names cannot contain '%s' or '%s'.", + tableStart, tableEnd) + case tableEnd: + lx.emit(itemText) + lx.next() + return lx.pop() + case tableSep: + lx.emit(itemText) + lx.next() + lx.ignore() + return lexTableNameStart + } + lx.next() + return lexTableName +} + +// lexKeyStart consumes a key name up until the first non-whitespace character. +// lexKeyStart will ignore whitespace. +func lexKeyStart(lx *lexer) stateFn { + r := lx.peek() + switch { + case r == keySep: + return lx.errorf("Unexpected key separator '%s'.", keySep) + case isWhitespace(r) || isNL(r): + lx.next() + return lexSkip(lx, lexKeyStart) + } + + lx.ignore() + lx.emit(itemKeyStart) + lx.next() + return lexKey +} + +// lexKey consumes the text of a key. Assumes that the first character (which +// is not whitespace) has already been consumed. +func lexKey(lx *lexer) stateFn { + r := lx.peek() + + // XXX: Possible divergence from spec? + // "Keys start with the first non-whitespace character and end with the + // last non-whitespace character before the equals sign." + // Note here that whitespace is either a tab or a space. + // But we'll call it quits if we see a new line too. + if isWhitespace(r) || isNL(r) { + lx.emit(itemText) + return lexKeyEnd + } + + // Let's also call it quits if we see an equals sign. + if r == keySep { + lx.emit(itemText) + return lexKeyEnd + } + + lx.next() + return lexKey +} + +// lexKeyEnd consumes the end of a key (up to the key separator). +// Assumes that the first whitespace character after a key (or the '=' +// separator) has NOT been consumed. +func lexKeyEnd(lx *lexer) stateFn { + r := lx.next() + switch { + case isWhitespace(r) || isNL(r): + return lexSkip(lx, lexKeyEnd) + case r == keySep: + return lexSkip(lx, lexValue) + } + return lx.errorf("Expected key separator '%s', but got '%s' instead.", + keySep, r) +} + +// lexValue starts the consumption of a value anywhere a value is expected. +// lexValue will ignore whitespace. +// After a value is lexed, the last state on the next is popped and returned. +func lexValue(lx *lexer) stateFn { + // We allow whitespace to precede a value, but NOT new lines. + // In array syntax, the array states are responsible for ignoring new lines. + r := lx.next() + if isWhitespace(r) { + return lexSkip(lx, lexValue) + } + + switch { + case r == arrayStart: + lx.ignore() + lx.emit(itemArray) + return lexArrayValue + case r == stringStart: + lx.ignore() // ignore the '"' + return lexString + case r == 't': + return lexTrue + case r == 'f': + return lexFalse + case r == '-': + return lexNumberStart + case isDigit(r): + lx.backup() // avoid an extra state and use the same as above + return lexNumberOrDateStart + case r == '.': // special error case, be kind to users + return lx.errorf("Floats must start with a digit, not '.'.") + } + return lx.errorf("Expected value but found '%s' instead.", r) +} + +// lexArrayValue consumes one value in an array. It assumes that '[' or ',' +// have already been consumed. All whitespace and new lines are ignored. +func lexArrayValue(lx *lexer) stateFn { + r := lx.next() + switch { + case isWhitespace(r) || isNL(r): + return lexSkip(lx, lexArrayValue) + case r == commentStart: + lx.push(lexArrayValue) + return lexCommentStart + case r == arrayValTerm: + return lx.errorf("Unexpected array value terminator '%s'.", + arrayValTerm) + case r == arrayEnd: + return lexArrayEnd + } + + lx.backup() + lx.push(lexArrayValueEnd) + return lexValue +} + +// lexArrayValueEnd consumes the cruft between values of an array. Namely, +// it ignores whitespace and expects either a ',' or a ']'. +func lexArrayValueEnd(lx *lexer) stateFn { + r := lx.next() + switch { + case isWhitespace(r) || isNL(r): + return lexSkip(lx, lexArrayValueEnd) + case r == commentStart: + lx.push(lexArrayValueEnd) + return lexCommentStart + case r == arrayValTerm: + return lexArrayValue // move on to the next value + case r == arrayEnd: + return lexArrayEnd + } + return lx.errorf("Expected an array value terminator '%s' or an array "+ + "terminator '%s', but got '%s' instead.", arrayValTerm, arrayEnd, r) +} + +// lexArrayEnd finishes the lexing of an array. It assumes that a ']' has +// just been consumed. +func lexArrayEnd(lx *lexer) stateFn { + lx.ignore() + lx.emit(itemArrayEnd) + return lx.pop() +} + +// lexString consumes the inner contents of a string. It assumes that the +// beginning '"' has already been consumed and ignored. +func lexString(lx *lexer) stateFn { + r := lx.next() + switch { + case isNL(r): + return lx.errorf("Strings cannot contain new lines.") + case r == '\\': + return lexStringEscape + case r == stringEnd: + lx.backup() + lx.emit(itemString) + lx.next() + lx.ignore() + return lx.pop() + } + return lexString +} + +// lexStringEscape consumes an escaped character. It assumes that the preceding +// '\\' has already been consumed. +func lexStringEscape(lx *lexer) stateFn { + r := lx.next() + switch r { + case 'b': + fallthrough + case 't': + fallthrough + case 'n': + fallthrough + case 'f': + fallthrough + case 'r': + fallthrough + case '"': + fallthrough + case '/': + fallthrough + case '\\': + return lexString + case 'u': + return lexStringUnicode + } + return lx.errorf("Invalid escape character '%s'. Only the following "+ + "escape characters are allowed: "+ + "\\b, \\t, \\n, \\f, \\r, \\\", \\/, \\\\, and \\uXXXX.", r) +} + +// lexStringBinary consumes two hexadecimal digits following '\x'. It assumes +// that the '\x' has already been consumed. +func lexStringUnicode(lx *lexer) stateFn { + var r rune + + for i := 0; i < 4; i++ { + r = lx.next() + if !isHexadecimal(r) { + return lx.errorf("Expected four hexadecimal digits after '\\x', "+ + "but got '%s' instead.", lx.current()) + } + } + return lexString +} + +// lexNumberOrDateStart consumes either a (positive) integer, float or datetime. +// It assumes that NO negative sign has been consumed. +func lexNumberOrDateStart(lx *lexer) stateFn { + r := lx.next() + if !isDigit(r) { + if r == '.' { + return lx.errorf("Floats must start with a digit, not '.'.") + } else { + return lx.errorf("Expected a digit but got '%s'.", r) + } + } + return lexNumberOrDate +} + +// lexNumberOrDate consumes either a (positive) integer, float or datetime. +func lexNumberOrDate(lx *lexer) stateFn { + r := lx.next() + switch { + case r == '-': + if lx.pos-lx.start != 5 { + return lx.errorf("All ISO8601 dates must be in full Zulu form.") + } + return lexDateAfterYear + case isDigit(r): + return lexNumberOrDate + case r == '.': + return lexFloatStart + } + + lx.backup() + lx.emit(itemInteger) + return lx.pop() +} + +// lexDateAfterYear consumes a full Zulu Datetime in ISO8601 format. +// It assumes that "YYYY-" has already been consumed. +func lexDateAfterYear(lx *lexer) stateFn { + formats := []rune{ + // digits are '0'. + // everything else is direct equality. + '0', '0', '-', '0', '0', + 'T', + '0', '0', ':', '0', '0', ':', '0', '0', + 'Z', + } + for _, f := range formats { + r := lx.next() + if f == '0' { + if !isDigit(r) { + return lx.errorf("Expected digit in ISO8601 datetime, "+ + "but found '%s' instead.", r) + } + } else if f != r { + return lx.errorf("Expected '%s' in ISO8601 datetime, "+ + "but found '%s' instead.", f, r) + } + } + lx.emit(itemDatetime) + return lx.pop() +} + +// lexNumberStart consumes either an integer or a float. It assumes that a +// negative sign has already been read, but that *no* digits have been consumed. +// lexNumberStart will move to the appropriate integer or float states. +func lexNumberStart(lx *lexer) stateFn { + // we MUST see a digit. Even floats have to start with a digit. + r := lx.next() + if !isDigit(r) { + if r == '.' { + return lx.errorf("Floats must start with a digit, not '.'.") + } else { + return lx.errorf("Expected a digit but got '%s'.", r) + } + } + return lexNumber +} + +// lexNumber consumes an integer or a float after seeing the first digit. +func lexNumber(lx *lexer) stateFn { + r := lx.next() + switch { + case isDigit(r): + return lexNumber + case r == '.': + return lexFloatStart + } + + lx.backup() + lx.emit(itemInteger) + return lx.pop() +} + +// lexFloatStart starts the consumption of digits of a float after a '.'. +// Namely, at least one digit is required. +func lexFloatStart(lx *lexer) stateFn { + r := lx.next() + if !isDigit(r) { + return lx.errorf("Floats must have a digit after the '.', but got "+ + "'%s' instead.", r) + } + return lexFloat +} + +// lexFloat consumes the digits of a float after a '.'. +// Assumes that one digit has been consumed after a '.' already. +func lexFloat(lx *lexer) stateFn { + r := lx.next() + if isDigit(r) { + return lexFloat + } + + lx.backup() + lx.emit(itemFloat) + return lx.pop() +} + +// lexTrue consumes the "rue" in "true". It assumes that 't' has already +// been consumed. +func lexTrue(lx *lexer) stateFn { + if r := lx.next(); r != 'r' { + return lx.errorf("Expected 'tr', but found 't%s' instead.", r) + } + if r := lx.next(); r != 'u' { + return lx.errorf("Expected 'tru', but found 'tr%s' instead.", r) + } + if r := lx.next(); r != 'e' { + return lx.errorf("Expected 'true', but found 'tru%s' instead.", r) + } + lx.emit(itemBool) + return lx.pop() +} + +// lexFalse consumes the "alse" in "false". It assumes that 'f' has already +// been consumed. +func lexFalse(lx *lexer) stateFn { + if r := lx.next(); r != 'a' { + return lx.errorf("Expected 'fa', but found 'f%s' instead.", r) + } + if r := lx.next(); r != 'l' { + return lx.errorf("Expected 'fal', but found 'fa%s' instead.", r) + } + if r := lx.next(); r != 's' { + return lx.errorf("Expected 'fals', but found 'fal%s' instead.", r) + } + if r := lx.next(); r != 'e' { + return lx.errorf("Expected 'false', but found 'fals%s' instead.", r) + } + lx.emit(itemBool) + return lx.pop() +} + +// lexCommentStart begins the lexing of a comment. It will emit +// itemCommentStart and consume no characters, passing control to lexComment. +func lexCommentStart(lx *lexer) stateFn { + lx.ignore() + lx.emit(itemCommentStart) + return lexComment +} + +// lexComment lexes an entire comment. It assumes that '#' has been consumed. +// It will consume *up to* the first new line character, and pass control +// back to the last state on the stack. +func lexComment(lx *lexer) stateFn { + r := lx.peek() + if isNL(r) || r == eof { + lx.emit(itemText) + return lx.pop() + } + lx.next() + return lexComment +} + +// lexSkip ignores all slurped input and moves on to the next state. +func lexSkip(lx *lexer, nextState stateFn) stateFn { + return func(lx *lexer) stateFn { + lx.ignore() + return nextState + } +} + +// isWhitespace returns true if `r` is a whitespace character according +// to the spec. +func isWhitespace(r rune) bool { + return r == '\t' || r == ' ' +} + +func isNL(r rune) bool { + return r == '\n' || r == '\r' +} + +func isDigit(r rune) bool { + return r >= '0' && r <= '9' +} + +func isHexadecimal(r rune) bool { + return (r >= '0' && r <= '9') || + (r >= 'a' && r <= 'f') || + (r >= 'A' && r <= 'F') +} + +func (itype itemType) String() string { + switch itype { + case itemError: + return "Error" + case itemNIL: + return "NIL" + case itemEOF: + return "EOF" + case itemText: + return "Text" + case itemString: + return "String" + case itemBool: + return "Bool" + case itemInteger: + return "Integer" + case itemFloat: + return "Float" + case itemDatetime: + return "DateTime" + case itemTableStart: + return "TableStart" + case itemTableEnd: + return "TableEnd" + case itemKeyStart: + return "KeyStart" + case itemArray: + return "Array" + case itemArrayEnd: + return "ArrayEnd" + case itemCommentStart: + return "CommentStart" + } + panic(fmt.Sprintf("BUG: Unknown type '%s'.", itype)) +} + +func (item item) String() string { + return fmt.Sprintf("(%s, %s)", item.typ.String(), item.val) +} + +func escapeSpecial(c rune) string { + switch c { + case '\n': + return "\\n" + } + return string(c) +} diff --git a/third_party/github.com/BurntSushi/toml/lex_test.go b/third_party/github.com/BurntSushi/toml/lex_test.go new file mode 100644 index 00000000000..6cfa21088f9 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/lex_test.go @@ -0,0 +1,59 @@ +package toml + +import ( + "log" + "testing" +) + +func init() { + log.SetFlags(0) +} + +var testSmall = ` +# This is a TOML document. Boom. + +[owner] +[owner] # Whoa there. +andrew = "gallant # poopy" # weeeee +predicate = false +num = -5192 +f = -0.5192 +zulu = 1979-05-27T07:32:00Z +whoop = "poop" +arrs = [ + 1987-07-05T05:45:00Z, + 5, + "wat?", + "hehe \n\r kewl", + [6], [], + 5.0, + # sweetness +] # more comments +# hehe +` + +var testSmaller = ` +[a.b] # Do you ignore me? +andrew = "ga# ll\"ant" # what about me? +kait = "brady" +awesomeness = true +pi = 3.14 +dob = 1987-07-05T17:45:00Z +perfection = [ + [6, 28], + [496, 8128] +] +` + +func TestLexer(t *testing.T) { + lx := lex(testSmaller) + for { + item := lx.nextItem() + if item.typ == itemEOF { + break + } else if item.typ == itemError { + t.Fatal(item.val) + } + testf("%s\n", item) + } +} diff --git a/third_party/github.com/BurntSushi/toml/out_test.go b/third_party/github.com/BurntSushi/toml/out_test.go new file mode 100644 index 00000000000..ab121e37520 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/out_test.go @@ -0,0 +1,19 @@ +package toml + +import ( + "flag" + "fmt" +) + +var flagOut = false + +func init() { + flag.BoolVar(&flagOut, "out", flagOut, "Print debug output.") + flag.Parse() +} + +func testf(format string, v ...interface{}) { + if flagOut { + fmt.Printf(format, v...) + } +} diff --git a/third_party/github.com/BurntSushi/toml/parse.go b/third_party/github.com/BurntSushi/toml/parse.go new file mode 100644 index 00000000000..57e4a70a959 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/parse.go @@ -0,0 +1,388 @@ +package toml + +import ( + "fmt" + "log" + "strconv" + "strings" + "time" + "unicode/utf8" +) + +type parser struct { + mapping map[string]interface{} + types map[string]tomlType + lx *lexer + + // A list of keys in the order that they appear in the TOML data. + ordered []Key + + // the full key for the current hash in scope + context Key + + // the base key name for everything except hashes + currentKey string + + // rough approximation of line number + approxLine int + + // A map of 'key.group.names' to whether they were created implicitly. + implicits map[string]bool +} + +type parseError string + +func (pe parseError) Error() string { + return string(pe) +} + +func parse(data string) (p *parser, err error) { + defer func() { + if r := recover(); r != nil { + var ok bool + if err, ok = r.(parseError); ok { + return + } + panic(r) + } + }() + + p = &parser{ + mapping: make(map[string]interface{}), + types: make(map[string]tomlType), + lx: lex(data), + ordered: make([]Key, 0), + implicits: make(map[string]bool), + } + for { + item := p.next() + if item.typ == itemEOF { + break + } + p.topLevel(item) + } + + return p, nil +} + +func (p *parser) panic(format string, v ...interface{}) { + msg := fmt.Sprintf("Near line %d, key '%s': %s", + p.approxLine, p.current(), fmt.Sprintf(format, v...)) + panic(parseError(msg)) +} + +func (p *parser) next() item { + it := p.lx.nextItem() + if it.typ == itemError { + p.panic("Near line %d: %s", it.line, it.val) + } + return it +} + +func (p *parser) bug(format string, v ...interface{}) { + log.Fatalf("BUG: %s\n\n", fmt.Sprintf(format, v...)) +} + +func (p *parser) expect(typ itemType) item { + it := p.next() + p.assertEqual(typ, it.typ) + return it +} + +func (p *parser) assertEqual(expected, got itemType) { + if expected != got { + p.bug("Expected '%s' but got '%s'.", expected, got) + } +} + +func (p *parser) topLevel(item item) { + switch item.typ { + case itemCommentStart: + p.approxLine = item.line + p.expect(itemText) + case itemTableStart: + kg := p.expect(itemText) + p.approxLine = kg.line + + key := make(Key, 0) + for ; kg.typ == itemText; kg = p.next() { + key = append(key, kg.val) + } + p.assertEqual(itemTableEnd, kg.typ) + + p.establishContext(key, false) + p.setType("", tomlHash) + p.ordered = append(p.ordered, key) + case itemArrayTableStart: + kg := p.expect(itemText) + p.approxLine = kg.line + + key := make(Key, 0) + for ; kg.typ == itemText; kg = p.next() { + key = append(key, kg.val) + } + p.assertEqual(itemArrayTableEnd, kg.typ) + + p.establishContext(key, true) + p.setType("", tomlArrayHash) + p.ordered = append(p.ordered, key) + case itemKeyStart: + kname := p.expect(itemText) + p.currentKey = kname.val + p.approxLine = kname.line + + val, typ := p.value(p.next()) + p.setValue(p.currentKey, val) + p.setType(p.currentKey, typ) + p.ordered = append(p.ordered, p.context.add(p.currentKey)) + + p.currentKey = "" + default: + p.bug("Unexpected type at top level: %s", item.typ) + } +} + +// value translates an expected value from the lexer into a Go value wrapped +// as an empty interface. +func (p *parser) value(it item) (interface{}, tomlType) { + switch it.typ { + case itemString: + return p.replaceUnicode(replaceEscapes(it.val)), p.typeOfPrimitive(it) + case itemBool: + switch it.val { + case "true": + return true, p.typeOfPrimitive(it) + case "false": + return false, p.typeOfPrimitive(it) + } + p.bug("Expected boolean value, but got '%s'.", it.val) + case itemInteger: + num, err := strconv.ParseInt(it.val, 10, 64) + if err != nil { + if e, ok := err.(*strconv.NumError); ok && + e.Err == strconv.ErrRange { + + p.panic("Integer '%s' is out of the range of 64-bit "+ + "signed integers.", it.val) + } else { + p.bug("Expected integer value, but got '%s'.", it.val) + } + } + return num, p.typeOfPrimitive(it) + case itemFloat: + num, err := strconv.ParseFloat(it.val, 64) + if err != nil { + if e, ok := err.(*strconv.NumError); ok && + e.Err == strconv.ErrRange { + + p.panic("Float '%s' is out of the range of 64-bit "+ + "IEEE-754 floating-point numbers.", it.val) + } else { + p.bug("Expected float value, but got '%s'.", it.val) + } + } + return num, p.typeOfPrimitive(it) + case itemDatetime: + t, err := time.Parse("2006-01-02T15:04:05Z", it.val) + if err != nil { + p.bug("Expected Zulu formatted DateTime, but got '%s'.", it.val) + } + return t, p.typeOfPrimitive(it) + case itemArray: + array := make([]interface{}, 0) + types := make([]tomlType, 0) + + for it = p.next(); it.typ != itemArrayEnd; it = p.next() { + if it.typ == itemCommentStart { + p.expect(itemText) + continue + } + + val, typ := p.value(it) + array = append(array, val) + types = append(types, typ) + } + return array, p.typeOfArray(types) + } + p.bug("Unexpected value type: %s", it.typ) + panic("unreachable") +} + +// establishContext sets the current context of the parser, +// where the context is the hash currently in scope. +// +// Establishing the context also makes sure that the key isn't a duplicate, and +// will create implicit hashes automatically. +func (p *parser) establishContext(key Key, array bool) { + var ok bool + + // Always start at the top level and drill down for our context. + hashContext := p.mapping + keyContext := make(Key, 0) + + // We only need implicit hashes for key[0:-1] + for _, k := range key[0 : len(key)-1] { + _, ok = hashContext[k] + keyContext = append(keyContext, k) + + // No key? Make an implicit hash and move on. + if !ok { + p.addImplicit(keyContext) + hashContext[k] = make(map[string]interface{}) + } + + // If the hash context is actually an array of tables, then set + // the hash context to the last element in that array. + // + // Otherwise, it better be a table, since this MUST be a key group (by + // virtue of it not being the last element in a key). + switch t := hashContext[k].(type) { + case []map[string]interface{}: + hashContext = t[len(t)-1] + case map[string]interface{}: + hashContext = t + default: + p.panic("Key '%s' was already created as a hash.", keyContext) + } + } + + p.context = keyContext + if array { + k := key[len(key)-1] + if _, ok := hashContext[k]; !ok { + hashContext[k] = make([]map[string]interface{}, 0, 5) + } + if hash, ok := hashContext[k].([]map[string]interface{}); ok { + hashContext[k] = append(hash, make(map[string]interface{})) + } else { + p.panic("Key '%s' was already created and cannot be used as "+ + "an array.", keyContext) + } + } else { + p.setValue(key[len(key)-1], make(map[string]interface{})) + } + p.context = append(p.context, key[len(key)-1]) +} + +// setValue sets the given key to the given value in the current context. +// It will make sure that the key hasn't already been defined, account for +// implicit key groups. +func (p *parser) setValue(key string, value interface{}) { + var tmpHash interface{} + var ok bool + + hash := p.mapping + keyContext := make(Key, 0) + for _, k := range p.context { + keyContext = append(keyContext, k) + if tmpHash, ok = hash[k]; !ok { + p.bug("Context for key '%s' has not been established.", keyContext) + } + switch t := tmpHash.(type) { + case []map[string]interface{}: + hash = t[len(t)-1] + case map[string]interface{}: + hash = t + default: + p.bug("Expected hash to have type 'map[string]interface{}', but "+ + "it has '%T' instead.", tmpHash) + } + } + keyContext = append(keyContext, key) + + if _, ok := hash[key]; ok { + // We need to do some fancy footwork here. If `hash[key]` was implcitly + // created AND `value` is a hash, then let this go through and stop + // tagging this table as implicit. + if p.isImplicit(keyContext) { + p.removeImplicit(keyContext) + return + } + + // Otherwise, we have a concrete key trying to override a previous + // key, which is *always* wrong. + p.panic("Key '%s' has already been defined.", keyContext) + } + hash[key] = value +} + +// setType sets the type of a particular value at a given key. +// It should be called immediately AFTER setValue. +func (p *parser) setType(key string, typ tomlType) { + keyContext := make(Key, 0, len(p.context)+1) + for _, k := range p.context { + keyContext = append(keyContext, k) + } + if len(key) > 0 { // allow type setting for hashes + keyContext = append(keyContext, key) + } + p.types[keyContext.String()] = typ +} + +// addImplicit sets the given Key as having been created implicitly. +func (p *parser) addImplicit(key Key) { + p.implicits[key.String()] = true +} + +// removeImplicit stops tagging the given key as having been implicitly created. +func (p *parser) removeImplicit(key Key) { + p.implicits[key.String()] = false +} + +// isImplicit returns true if the key group pointed to by the key was created +// implicitly. +func (p *parser) isImplicit(key Key) bool { + return p.implicits[key.String()] +} + +// current returns the full key name of the current context. +func (p *parser) current() string { + if len(p.currentKey) == 0 { + return p.context.String() + } + if len(p.context) == 0 { + return p.currentKey + } + return fmt.Sprintf("%s.%s", p.context, p.currentKey) +} + +func replaceEscapes(s string) string { + return strings.NewReplacer( + "\\b", "\u0008", + "\\t", "\u0009", + "\\n", "\u000A", + "\\f", "\u000C", + "\\r", "\u000D", + "\\\"", "\u0022", + "\\/", "\u002F", + "\\\\", "\u005C", + ).Replace(s) +} + +func (p *parser) replaceUnicode(s string) string { + indexEsc := func() int { + return strings.Index(s, "\\u") + } + for i := indexEsc(); i != -1; i = indexEsc() { + asciiBytes := s[i+2 : i+6] + s = strings.Replace(s, s[i:i+6], p.asciiEscapeToUnicode(asciiBytes), -1) + } + return s +} + +func (p *parser) asciiEscapeToUnicode(s string) string { + hex, err := strconv.ParseUint(strings.ToLower(s), 16, 32) + if err != nil { + p.bug("Could not parse '%s' as a hexadecimal number, but the "+ + "lexer claims it's OK: %s", s, err) + } + + // I honestly don't understand how this works. I can't seem to find + // a way to make this fail. I figured this would fail on invalid UTF-8 + // characters like U+DCFF, but it doesn't. + r := string(rune(hex)) + if !utf8.ValidString(r) { + p.panic("Escaped character '\\u%s' is not valid UTF-8.", s) + } + return string(r) +} diff --git a/third_party/github.com/BurntSushi/toml/parse_test.go b/third_party/github.com/BurntSushi/toml/parse_test.go new file mode 100644 index 00000000000..d6f0d32fbd9 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/parse_test.go @@ -0,0 +1,61 @@ +package toml + +import ( + "strings" + "testing" +) + +var testParseSmall = ` +# This is a TOML document. Boom. + +wat = "chipper" + +[owner.andrew.gallant] +hmm = "hi" + +[owner] # Whoa there. +andreW = "gallant # poopy" # weeeee +predicate = false +num = -5192 +f = -0.5192 +zulu = 1979-05-27T07:32:00Z +whoop = "poop" +tests = [ [1, 2, 3], ["abc", "xyz"] ] +arrs = [ # hmm + # more comments are awesome. + 1987-07-05T05:45:00Z, + # say wat? + 1987-07-05T05:45:00Z, + 1987-07-05T05:45:00Z, + # sweetness +] # more comments +# hehe +` + +var testParseSmall2 = ` +[a] +better = 43 + +[a.b.c] +answer = 42 +` + +func TestParse(t *testing.T) { + m, err := parse(testParseSmall) + if err != nil { + t.Fatal(err) + } + printMap(m.mapping, 0) +} + +func printMap(m map[string]interface{}, depth int) { + for k, v := range m { + testf("%s%s\n", strings.Repeat(" ", depth), k) + switch subm := v.(type) { + case map[string]interface{}: + printMap(subm, depth+1) + default: + testf("%s%v\n", strings.Repeat(" ", depth+1), v) + } + } +} diff --git a/third_party/github.com/BurntSushi/toml/session.vim b/third_party/github.com/BurntSushi/toml/session.vim new file mode 100644 index 00000000000..562164be060 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/session.vim @@ -0,0 +1 @@ +au BufWritePost *.go silent!make tags > /dev/null 2>&1 diff --git a/third_party/github.com/BurntSushi/toml/toml-test-go/COPYING b/third_party/github.com/BurntSushi/toml/toml-test-go/COPYING new file mode 100644 index 00000000000..5a8e332545f --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/toml-test-go/COPYING @@ -0,0 +1,14 @@ + DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE + Version 2, December 2004 + + Copyright (C) 2004 Sam Hocevar + + Everyone is permitted to copy and distribute verbatim or modified + copies of this license document, and changing it is allowed as long + as the name is changed. + + DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. You just DO WHAT THE FUCK YOU WANT TO. + diff --git a/third_party/github.com/BurntSushi/toml/toml-test-go/README.md b/third_party/github.com/BurntSushi/toml/toml-test-go/README.md new file mode 100644 index 00000000000..24421eb7034 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/toml-test-go/README.md @@ -0,0 +1,14 @@ +# Implements the TOML test suite interface + +This is an implementation of the interface expected by +[toml-test](https://github.com/BurntSushi/toml-test) for my +[toml parser written in Go](https://github.com/BurntSushi/toml). +In particular, it maps TOML data on `stdin` to a JSON format on `stdout`. + + +Compatible with TOML version +[v0.2.0](https://github.com/mojombo/toml/blob/master/versions/toml-v0.2.0.md) + +Compatible with `toml-test` version +[v0.2.0](https://github.com/BurntSushi/toml-test/tree/v0.2.0) + diff --git a/third_party/github.com/BurntSushi/toml/toml-test-go/main.go b/third_party/github.com/BurntSushi/toml/toml-test-go/main.go new file mode 100644 index 00000000000..79a3c212f79 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/toml-test-go/main.go @@ -0,0 +1,89 @@ +package main + +import ( + "encoding/json" + "flag" + "fmt" + "log" + "os" + "path" + "time" + + "github.com/BurntSushi/toml" +) + +func init() { + log.SetFlags(0) + + flag.Usage = usage + flag.Parse() +} + +func usage() { + log.Printf("Usage: %s < toml-file\n", path.Base(os.Args[0])) + flag.PrintDefaults() + + os.Exit(1) +} + +func main() { + if flag.NArg() != 0 { + flag.Usage() + } + + var tmp interface{} + if _, err := toml.DecodeReader(os.Stdin, &tmp); err != nil { + log.Fatalf("Error decoding TOML: %s", err) + } + + typedTmp := translate(tmp) + if err := json.NewEncoder(os.Stdout).Encode(typedTmp); err != nil { + log.Fatalf("Error encoding JSON: %s", err) + } +} + +func translate(tomlData interface{}) interface{} { + + switch orig := tomlData.(type) { + case map[string]interface{}: + typed := make(map[string]interface{}, len(orig)) + for k, v := range orig { + typed[k] = translate(v) + } + return typed + case []map[string]interface{}: + typed := make([]map[string]interface{}, len(orig)) + for i, v := range orig { + typed[i] = translate(v).(map[string]interface{}) + } + return typed + case []interface{}: + typed := make([]interface{}, len(orig)) + for i, v := range orig { + typed[i] = translate(v) + } + + // We don't really need to tag arrays, but let's be future proof. + // (If TOML ever supports tuples, we'll need this.) + return tag("array", typed) + case time.Time: + return tag("datetime", orig.Format("2006-01-02T15:04:05Z")) + case bool: + return tag("bool", fmt.Sprintf("%v", orig)) + case int64: + return tag("integer", fmt.Sprintf("%d", orig)) + case float64: + return tag("float", fmt.Sprintf("%v", orig)) + case string: + return tag("string", orig) + } + + panic(fmt.Sprintf("Unknown type: %T", tomlData)) +} + +func tag(typeName string, data interface{}) map[string]interface{} { + return map[string]interface{}{ + "type": typeName, + "value": data, + } +} diff --git a/third_party/github.com/BurntSushi/toml/tomlv/COPYING b/third_party/github.com/BurntSushi/toml/tomlv/COPYING new file mode 100644 index 00000000000..5a8e332545f --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/tomlv/COPYING @@ -0,0 +1,14 @@ + DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE + Version 2, December 2004 + + Copyright (C) 2004 Sam Hocevar + + Everyone is permitted to copy and distribute verbatim or modified + copies of this license document, and changing it is allowed as long + as the name is changed. + + DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. You just DO WHAT THE FUCK YOU WANT TO. + diff --git a/third_party/github.com/BurntSushi/toml/tomlv/README.md b/third_party/github.com/BurntSushi/toml/tomlv/README.md new file mode 100644 index 00000000000..bcc3f47b1a4 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/tomlv/README.md @@ -0,0 +1,22 @@ +# TOML Validator + +If Go is installed, it's simple to try it out: + +```bash +go get github.com/BurntSushi/toml/tomlv +tomlv some-toml-file.toml +``` + +You can see the types of every key in a TOML file with: + +```bash +tomlv -types some-toml-file.toml +``` + +At the moment, only one error message is reported at a time. Error messages +include line numbers. No output means that the files given are valid TOML, or +there is a bug in `tomlv`. + +Compatible with TOML version +[v0.1.0](https://github.com/mojombo/toml/blob/master/versions/toml-v0.1.0.md) + diff --git a/third_party/github.com/BurntSushi/toml/tomlv/main.go b/third_party/github.com/BurntSushi/toml/tomlv/main.go new file mode 100644 index 00000000000..a52086ef5cf --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/tomlv/main.go @@ -0,0 +1,60 @@ +package main + +import ( + "flag" + "fmt" + "log" + "os" + "path" + "strings" + "text/tabwriter" + + "github.com/BurntSushi/toml" +) + +var ( + flagTypes = false +) + +func init() { + log.SetFlags(0) + + flag.BoolVar(&flagTypes, "types", flagTypes, + "When set, the types of every defined key will be shown.") + + flag.Usage = usage + flag.Parse() +} + +func usage() { + log.Printf("Usage: %s toml-file [ toml-file ... ]\n", + path.Base(os.Args[0])) + flag.PrintDefaults() + + os.Exit(1) +} + +func main() { + if flag.NArg() < 1 { + flag.Usage() + } + for _, f := range flag.Args() { + var tmp interface{} + md, err := toml.DecodeFile(f, &tmp) + if err != nil { + log.Fatalf("Error in '%s': %s", f, err) + } + if flagTypes { + printTypes(md) + } + } +} + +func printTypes(md toml.MetaData) { + tabw := tabwriter.NewWriter(os.Stdout, 0, 0, 2, ' ', 0) + for _, key := range md.Keys() { + fmt.Fprintf(tabw, "%s%s\t%s\n", + strings.Repeat(" ", len(key)-1), key, md.Type(key...)) + } + tabw.Flush() +} diff --git a/third_party/github.com/BurntSushi/toml/type-check.go b/third_party/github.com/BurntSushi/toml/type-check.go new file mode 100644 index 00000000000..22f188d4202 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/type-check.go @@ -0,0 +1,78 @@ +package toml + +// tomlType represents any Go type that corresponds to a TOML type. +// While the first draft of the TOML spec has a simplistic type system that +// probably doesn't need this level of sophistication, we seem to be militating +// toward adding real composite types. +type tomlType interface { + typeString() string +} + +// typeEqual accepts any two types and returns true if they are equal. +func typeEqual(t1, t2 tomlType) bool { + return t1.typeString() == t2.typeString() +} + +type tomlBaseType string + +func (btype tomlBaseType) typeString() string { + return string(btype) +} + +func (btype tomlBaseType) String() string { + return btype.typeString() +} + +var ( + tomlInteger tomlBaseType = "Integer" + tomlFloat tomlBaseType = "Float" + tomlDatetime tomlBaseType = "Datetime" + tomlString tomlBaseType = "String" + tomlBool tomlBaseType = "Bool" + tomlArray tomlBaseType = "Array" + tomlHash tomlBaseType = "Hash" + tomlArrayHash tomlBaseType = "ArrayHash" +) + +// typeOfPrimitive returns a tomlType of any primitive value in TOML. +// Primitive values are: Integer, Float, Datetime, String and Bool. +// +// Passing a lexer item other than the following will cause a BUG message +// to occur: itemString, itemBool, itemInteger, itemFloat, itemDatetime. +func (p *parser) typeOfPrimitive(lexItem item) tomlType { + switch lexItem.typ { + case itemInteger: + return tomlInteger + case itemFloat: + return tomlFloat + case itemDatetime: + return tomlDatetime + case itemString: + return tomlString + case itemBool: + return tomlBool + } + p.bug("Cannot infer primitive type of lex item '%s'.", lexItem) + panic("unreachable") +} + +// typeOfArray returns a tomlType for an array given a list of types of its +// values. +// +// In the current spec, if an array is homogeneous, then its type is always +// "Array". If the array is not homogeneous, an error is generated. +func (p *parser) typeOfArray(types []tomlType) tomlType { + // Empty arrays are cool. + if len(types) == 0 { + return tomlArray + } + + theType := types[0] + for _, t := range types[1:] { + if !typeEqual(theType, t) { + p.panic("Array contains values of type '%s' and '%s', but arrays "+ + "must be homogeneous.", theType, t) + } + } + return tomlArray +} diff --git a/util.go b/util.go deleted file mode 100644 index 089eba40cc2..00000000000 --- a/util.go +++ /dev/null @@ -1,88 +0,0 @@ -package main - -import ( - "net" - "net/url" - "os" - "os/signal" - "runtime/pprof" - - "github.com/coreos/etcd/log" -) - -//-------------------------------------- -// HTTP Utilities -//-------------------------------------- - -// sanitizeURL will cleanup a host string in the format hostname:port and -// attach a schema. -func sanitizeURL(host string, defaultScheme string) string { - // Blank URLs are fine input, just return it - if len(host) == 0 { - return host - } - - p, err := url.Parse(host) - if err != nil { - log.Fatal(err) - } - - // Make sure the host is in Host:Port format - _, _, err = net.SplitHostPort(host) - if err != nil { - log.Fatal(err) - } - - p = &url.URL{Host: host, Scheme: defaultScheme} - - return p.String() -} - -// sanitizeListenHost cleans up the ListenHost parameter and appends a port -// if necessary based on the advertised port. -func sanitizeListenHost(listen string, advertised string) string { - aurl, err := url.Parse(advertised) - if err != nil { - log.Fatal(err) - } - - ahost, aport, err := net.SplitHostPort(aurl.Host) - if err != nil { - log.Fatal(err) - } - - // If the listen host isn't set use the advertised host - if listen == "" { - listen = ahost - } - - return net.JoinHostPort(listen, aport) -} - -func check(err error) { - if err != nil { - log.Fatal(err) - } -} - -//-------------------------------------- -// CPU profile -//-------------------------------------- -func runCPUProfile() { - - f, err := os.Create(cpuprofile) - if err != nil { - log.Fatal(err) - } - pprof.StartCPUProfile(f) - - c := make(chan os.Signal, 1) - signal.Notify(c, os.Interrupt) - go func() { - for sig := range c { - log.Infof("captured %v, stopping profiler and exiting..", sig) - pprof.StopCPUProfile() - os.Exit(1) - } - }() -} From 98fad7b86aba6173fb62a34766edafd71b0faa70 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 21 Oct 2013 21:51:15 -0700 Subject: [PATCH 143/247] update readme --- README.md | 186 ++++++++++++++++++++++++++++++++++++------------------ 1 file changed, 124 insertions(+), 62 deletions(-) diff --git a/README.md b/README.md index 562407e1446..a4d60b735ef 100644 --- a/README.md +++ b/README.md @@ -1,5 +1,5 @@ # etcd -README version 0.1.0 +README version 0.2.0 [![Build Status](https://travis-ci.org/coreos/etcd.png)](https://travis-ci.org/coreos/etcd) @@ -62,73 +62,70 @@ The `-n node0` tells the rest of the cluster that this node is named node0. Let’s set the first key-value pair to the node. In this case the key is `/message` and the value is `Hello world`. ```sh -curl -L http://127.0.0.1:4001/v1/keys/message -d value="Hello world" +curl -L http://127.0.0.1:4001/v2/keys/message -XPUT -d value="Hello world" ``` ```json -{"action":"SET","key":"/message","value":"Hello world","newKey":true,"index":3} +{"action":"set","key":"/message","value":"Hello world","index":3,"term":0} ``` This response contains five fields. We will introduce three more fields as we try more commands. -1. The action of the request; we set the value via a POST request, thus the action is `SET`. +1. The action of the request; we set the value via a PUT request, thus the action is `set`. 2. The key of the request; we set `/message` to `Hello world!`, so the key field is `/message`. Notice we use a file system like structure to represent the key-value pairs. So each key starts with `/`. 3. The current value of the key; we set the value to`Hello world`. -4. If we set a new key; `/message` did not exist before, so this is a new key. - -5. Index is the unique internal log index of the set request. Requests that change the log index include `SET`, `DELETE` and `TESTANDSET`. The `GET`, `LIST` and `WATCH` commands do not change state in the store and so they do not change the index. You may notice that in this example the index is 3, although it is the first request you sent to the server. This is because there are internal commands that also change the state like adding and syncing servers. +4. Index is the unique internal log index of the set request. Requests that change the log index include `set`, `delete`, `update`, `create` and `compareAndSwap`. The `get` and `watch` commands do not change state in the store and so they do not change the index. You may notice that in this example the index is 3, although it is the first request you sent to the server. This is because there are internal commands that also change the state like adding and syncing servers. ### Get the value of a key Get the value that we just set in `/message` by issuing a GET: ```sh -curl -L http://127.0.0.1:4001/v1/keys/message +curl -L http://127.0.0.1:4001/v2/keys/message ``` ```json -{"action":"GET","key":"/message","value":"Hello world","index":3} +{"action":"get","key":"/message","value":"Hello world","index":3,"term":0} ``` ### Change the value of a key -Change the value of `/message` from `Hello world` to `Hello etcd` with another POST to the key: +Change the value of `/message` from `Hello world` to `Hello etcd` with another PUT request to the key: ```sh -curl -L http://127.0.0.1:4001/v1/keys/message -d value="Hello etcd" +curl -L http://127.0.0.1:4001/v1/keys/message -XPUT -d value="Hello etcd" ``` ```json -{"action":"SET","key":"/message","prevValue":"Hello world","value":"Hello etcd","index":4} +{"action":"set","key":"/message","prevValue":"Hello world","value":"Hello etcd","index":4,"term":0} ``` -Notice that the `prevValue` is set to `Hello world`. - +Notice that the `prevValue` is set to the previous value of the key - `Hello world`. It is useful when you want to atomically set a value to a key and get its old value. ### Delete a key Remove the `/message` key with a DELETE: ```sh -curl -L http://127.0.0.1:4001/v1/keys/message -X DELETE +curl -L http://127.0.0.1:4001/v2/keys/message -XDELETE ``` ```json -{"action":"DELETE","key":"/message","prevValue":"Hello etcd","index":5} +{"action":"delete","key":"/message","prevValue":"Hello etcd","index":5,"term":0} ``` -### Using key TTL +### Use key TTL Keys in etcd can be set to expire after a specified number of seconds. That is done by setting a TTL (time to live) on the key when you POST: ```sh -curl -L http://127.0.0.1:4001/v1/keys/foo -d value=bar -d ttl=5 +curl -L http://127.0.0.1:4001/v2/keys/foo -XPUT -d value=bar -d ttl=5 ``` ```json -{"action":"SET","key":"/foo","value":"bar","newKey":true,"expiration":"2013-07-11T20:31:12.156146039-07:00","ttl":4,"index":6} +{"action":"set","key":"/foo","value":"bar","expiration":"2013-10-19T18:44:04.528757176-07:00","ttl":5,"index":6,"term":0} ``` Note the last two new fields in response: @@ -140,115 +137,180 @@ Note the last two new fields in response: Now you can try to get the key by sending: ```sh -curl -L http://127.0.0.1:4001/v1/keys/foo +curl -L http://127.0.0.1:4001/v2/keys/foo ``` If the TTL has expired, the key will be deleted, and you will be returned a 100. ```json -{"errorCode":100,"message":"Key Not Found","cause":"/foo"} +{"errorCode":100,"message":"Key Not Found","cause":"/foo","index":6,"term":0} ``` -### Watching a prefix +### Wait for a change -We can watch a path prefix and get notifications if any key change under that prefix. +We can watch for a change and get notification just at the given path or under the given path -In one terminal, we send a watch request: +In one terminal, we send a get request with `wait=true` : ```sh -curl -L http://127.0.0.1:4001/v1/watch/foo +curl -L http://127.0.0.1:4001/v2/keys/foo?wait=true ``` -Now, we are watching at the path prefix `/foo` and wait for any changes under this path. +Now, we are waitting for any changes at path `/foo`. -In another terminal, we set a key `/foo/foo` to `barbar` to see what will happen: +In another terminal, we set a key `/foo` with value `bar`: ```sh -curl -L http://127.0.0.1:4001/v1/keys/foo/foo -d value=barbar +curl -L http://127.0.0.1:4001/v2/keys/foo -XPUT -d value=bar ``` The first terminal should get the notification and return with the same response as the set request. ```json -{"action":"SET","key":"/foo/foo","value":"barbar","newKey":true,"index":7} +{"action":"set","key":"/foo","value":"bar","index":7,"term":0} ``` However, the watch command can do more than this. Using the the index we can watch for commands that has happened in the past. This is useful for ensuring you don't miss events between watch commands. -Let's try to watch for the set command of index 6 again: +Let's try to watch for the set command of index 7 again: ```sh -curl -L http://127.0.0.1:4001/v1/watch/foo -d index=7 +curl -L http://127.0.0.1:4001/v2/keys/foo?wait=true\&waitIndex=7 ``` The watch command returns immediately with the same response as previous. -### Atomic Test and Set +### Atomic Compare and Swap + +Etcd can be used as a centralized coordination service in a cluster and `CompareAndSwap` is the most basic operation to build distributed lock service. + +This command will set the value to the key only if the client provided conditions are equal to the current conditions. + +The current comparable conditions are: +1. `prevValue` previous value of the key: + +2. `prevIndex` previous index of the key -Etcd can be used as a centralized coordination service in a cluster and `TestAndSet` is the most basic operation to build distributed lock service. This command will set the value only if the client provided `prevValue` is equal the current key value. +3. `prevExist` previous existence of the key: if `prevExist` is true, it is a `update` request; if prevExist is `false`, it is a `create` request. Here is a simple example. Let's create a key-value pair first: `foo=one`. ```sh -curl -L http://127.0.0.1:4001/v1/keys/foo -d value=one +curl -L http://127.0.0.1:4001/v1/keys/foo -XPUT -d value=one ``` -Let's try an invalid `TestAndSet` command. -We can give another parameter prevValue to set command to make it a TestAndSet command. +Let's try an invalid `CompareAndSwap` command. +We can give another parameter prevValue to set command to make it a `CompareAndSwap` command. ```sh -curl -L http://127.0.0.1:4001/v1/keys/foo -d prevValue=two -d value=three +curl -L http://127.0.0.1:4001/v1/keys/foo?prevValue=two -XPUT -d value=three ``` -This will try to test if the previous of the key is two, it is change it to three. +This will try to compare the previous value of the key and the previous value we provided. If they are equal, the value of the key will change to three. ```json -{"errorCode":101,"message":"The given PrevValue is not equal to the value of the key","cause":"TestAndSet: one!=two"} +{"errorCode":101,"message":"Test Failed","cause":"[two != one] [0 != 8]","index":9,"term":0} ``` -which means `testAndSet` failed. +which means `compareAndSwap` failed. Let us try a valid one. ```sh -curl -L http://127.0.0.1:4001/v1/keys/foo -d prevValue=one -d value=two +curl -L http://127.0.0.1:4001/v2/keys/foo?prevValue=one -XPUT -d value=two ``` The response should be ```json -{"action":"SET","key":"/foo","prevValue":"one","value":"two","index":10} +{"action":"compareAndSwap","key":"/foo","prevValue":"one","value":"two","index":10,"term":0} ``` We successfully changed the value from “one” to “two”, since we give the correct previous value. -### Listing a directory - -Last we provide a simple List command to list all the keys under a prefix path. +### Get a directory Let us create some keys first. -We already have `/foo/foo=barbar` +We already have `/foo=two` -We create another one `/foo/foo_dir/foo=barbarbar` +We create another one `/foo_dir/foo=bar` ```sh -curl -L http://127.0.0.1:4001/v1/keys/foo/foo_dir/bar -d value=barbarbar +curl -L http://127.0.0.1:4001/v2/keys/foo_dir/foo -XPUT -d value=bar +``` + +```json +{"action":"set","key":"/foo_dir/foo","value":"bar","index":11,"term":0} ``` -Now list the keys under `/foo` +Now list the keys under root `/` ```sh -curl -L http://127.0.0.1:4001/v1/keys/foo/ +curl -L http://127.0.0.1:4001/v2/keys/ ``` We should see the response as an array of items ```json -[{"action":"GET","key":"/foo/foo","value":"barbar","index":10},{"action":"GET","key":"/foo/foo_dir","dir":true,"index":10}] +{"action":"get","key":"/","dir":true,"kvs":[{"key":"/foo","value":"two"},{"key":"/foo_dir","dir":true}],"index":11,"term":0} +``` + +which meas `/foo=two` is a key-value pair under `/ and `/foo_dir` is a directory. + +Also we can recursively get all the content under a directory by add `recursive=true`. + +```sh +curl -L http://127.0.0.1:4001/v2/keys/?recursive=true +``` + +```json +{"action":"get","key":"/","dir":true,"kvs":[{"key":"/foo","value":"two"},{"key":"/foo_dir","dir":true,"kvs":[{"key":"/foo_dir/foo","value":"bar"}]}],"index":11,"term":0} +``` + +### Delete a directory +Let try to delete the directory `/foo_dir`. + +To delete a directory, we must add `recursive=true`. + +```sh +curl -L http://127.0.0.1:4001/v2/keys/foo_dir?recursive=true -XDELETE +``` + +```json +{"action":"delete","key":"/foo_dir","dir":true,"index":12,"term":0} +``` + +### Create a hidden node +We can create a hidden key-value pair or directory by add `_` prefix. The hidden item will not be list when using get for a directory. + +```sh +curl -L http://127.0.0.1:4001/v2/keys/_message -XPUT -d value="Hello hidden world" +``` + +```json +{"action":"set","key":"/_message","value":"Hello hidden world","index":13,"term":0} +``` + +```sh +curl -L http://127.0.0.1:4001/v2/keys/message -XPUT -d value="Hello world" +``` + +```json +{"action":"set","key":"/message","value":"Hello world","index":14,"term":0} +``` + +Let us try to get the root `/` + +```sh +curl -L http://127.0.0.1:4001/v2/keys/ +``` + +```json +{"action":"get","key":"/","dir":true,"kvs":[{"key":"/foo","value":"two"},{"key":"/message","value":"Hello world"}],"index":15,"term":0} ``` -which meas `foo=barbar` is a key-value pair under `/foo` and `foo_dir` is a directory. +We can only get `/message`, but cannot get `/_message`. ## Advanced Usage @@ -273,7 +335,7 @@ Next, lets configure etcd to use this keypair: You can now test the configuration using https: ```sh -curl --cacert fixtures/ca/ca.crt https://127.0.0.1:4001/v1/keys/foo -d value=bar -v +curl --cacert fixtures/ca/ca.crt https://127.0.0.1:4001/v2/keys/foo -XPUT -d value=bar -v ``` You should be able to see the handshake succeed. @@ -287,7 +349,7 @@ SSLv3, TLS handshake, Finished (20): And also the response from the etcd server. ```json -{"action":"SET","key":"/foo","value":"bar","newKey":true,"index":3} +{"action":"set","key":"/foo","value":"bar","index":3, "term: 0"} ``` ### Authentication with HTTPS client certificates @@ -303,7 +365,7 @@ We can also do authentication using CA certs. The clients will provide their cer Try the same request to this server: ```sh -curl --cacert fixtures/ca/ca.crt https://127.0.0.1:4001/v1/keys/foo -d value=bar -v +curl --cacert fixtures/ca/ca.crt https://127.0.0.1:4001/v2/keys/foo -XPUT -d value=bar -v ``` The request should be rejected by the server. @@ -317,7 +379,7 @@ routines:SSL3_READ_BYTES:sslv3 alert bad certificate We need to give the CA signed cert to the server. ```sh -curl -L https://127.0.0.1:4001/v1/keys/foo -d value=bar -v --key myclient.key --cert myclient.crt -cacert clientCA.crt +curl -L https://127.0.0.1:4001/v1/keys/foo -XPUT -d value=bar -v --key myclient.key --cert myclient.crt -cacert clientCA.crt ``` You should able to see @@ -331,7 +393,7 @@ TLS handshake, Finished (20) And also the response from the server: ```json -{"action":"SET","key":"/foo","value":"bar","newKey":true,"index":3} +{"action":"set","key":"/foo","value":"bar","index":3,"term:0"} ``` ## Clustering @@ -377,7 +439,7 @@ curl -L http://127.0.0.1:4001/v1/keys/_etcd/machines ``` ```json -[{"action":"GET","key":"/_etcd/machines/node1","value":"raft=http://127.0.0.1:7001&etcd=http://127.0.0.1:4001","index":4},{"action":"GET","key":"/_etcd/machines/node2","value":"raft=http://127.0.0.1:7002&etcd=http://127.0.0.1:4002","index":4},{"action":"GET","key":"/_etcd/machines/node3","value":"raft=http://127.0.0.1:7003&etcd=http://127.0.0.1:4003","index":4}] +[{"action":"get","key":"/_etcd/machines/node1","value":"raft=http://127.0.0.1:7001&etcd=http://127.0.0.1:4001&raftVersion=v0.1.1-311-g91cad59","index":4},{"action":"get","key":"/_etcd/machines/node2","value":"raft=http://127.0.0.1:7002&etcd=http://127.0.0.1:4002&raftVersion=v0.1.1-311-g91cad59","index":4},{"action":"get","key":"/_etcd/machines/node3","value":"raft=http://127.0.0.1:7003&etcd=http://127.0.0.1:4003&raftVersion=v0.1.1-311-g91cad59","index":4}] ``` The key of the machine is based on the ```commit index``` when it was added. The value of the machine is ```hostname```, ```raft port``` and ```client port```. @@ -385,7 +447,7 @@ The key of the machine is based on the ```commit index``` when it was added. The Also try to get the current leader in the cluster ``` -curl -L http://127.0.0.1:4001/v1/leader +curl -L http://127.0.0.1:4001/v2/leader ``` The first server we set up should be the leader, if it has not died during these commands. @@ -396,11 +458,11 @@ http://127.0.0.1:7001 Now we can do normal SET and GET operations on keys as we explored earlier. ```sh -curl -L http://127.0.0.1:4001/v1/keys/foo -d value=bar +curl -L http://127.0.0.1:4001/v2/keys/foo -XPUT -d value=bar ``` ```json -{"action":"SET","key":"/foo","value":"bar","newKey":true,"index":5} +{"action":"set","key":"/foo","value":"bar","index":5,"term:0"} ``` ### Killing Nodes in the Cluster @@ -430,7 +492,7 @@ http://127.0.0.1:7003 You should be able to see this: ```json -{"action":"GET","key":"/foo","value":"bar","index":5} +{"action":"get","key":"/foo","value":"bar","index":5,"term:1"} ``` It succeeded! From 63456b5c4be1160e6e59840bc45fcbb937f89999 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 22 Oct 2013 00:17:58 -0700 Subject: [PATCH 144/247] fix close connection when client leaves --- server/v2/get_handler.go | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/server/v2/get_handler.go b/server/v2/get_handler.go index d0e8042737d..39e932fe09b 100644 --- a/server/v2/get_handler.go +++ b/server/v2/get_handler.go @@ -46,11 +46,19 @@ func GetHandler(w http.ResponseWriter, req *http.Request, s Server) error { } // Start the watcher on the store. - c, err := s.Store().Watch(key, recursive, sinceIndex, s.CommitIndex(), s.Term()) + eventChan, err := s.Store().Watch(key, recursive, sinceIndex, s.CommitIndex(), s.Term()) if err != nil { return etcdErr.NewError(500, key, store.UndefIndex, store.UndefTerm) } - event = <-c + + cn, _ := w.(http.CloseNotifier) + closeChan := cn.CloseNotify() + + select { + case <-closeChan: + return nil + case event = <-eventChan: + } } else { //get // Retrieve the key from the store. From 53b638558b7c505a292ae8f60b2231a4a8bb7db4 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 22 Oct 2013 14:30:55 -0700 Subject: [PATCH 145/247] Update readme --- README.md | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/README.md b/README.md index a4d60b735ef..2034ab31464 100644 --- a/README.md +++ b/README.md @@ -91,7 +91,7 @@ curl -L http://127.0.0.1:4001/v2/keys/message ```json {"action":"get","key":"/message","value":"Hello world","index":3,"term":0} ``` -### Change the value of a key +### Changing the value of a key Change the value of `/message` from `Hello world` to `Hello etcd` with another PUT request to the key: @@ -104,7 +104,7 @@ curl -L http://127.0.0.1:4001/v1/keys/message -XPUT -d value="Hello etcd" ``` Notice that the `prevValue` is set to the previous value of the key - `Hello world`. It is useful when you want to atomically set a value to a key and get its old value. -### Delete a key +### Deleting a key Remove the `/message` key with a DELETE: @@ -116,7 +116,7 @@ curl -L http://127.0.0.1:4001/v2/keys/message -XDELETE {"action":"delete","key":"/message","prevValue":"Hello etcd","index":5,"term":0} ``` -### Use key TTL +### Using key TTL Keys in etcd can be set to expire after a specified number of seconds. That is done by setting a TTL (time to live) on the key when you POST: @@ -146,9 +146,9 @@ If the TTL has expired, the key will be deleted, and you will be returned a 100. {"errorCode":100,"message":"Key Not Found","cause":"/foo","index":6,"term":0} ``` -### Wait for a change +### Waiting for a change -We can watch for a change and get notification just at the given path or under the given path +We can watch for a change and get a notification at a given path or any keys underneath it. In one terminal, we send a get request with `wait=true` : @@ -228,7 +228,7 @@ The response should be We successfully changed the value from “one” to “two”, since we give the correct previous value. -### Get a directory +### Listing a directory Let us create some keys first. @@ -268,7 +268,7 @@ curl -L http://127.0.0.1:4001/v2/keys/?recursive=true {"action":"get","key":"/","dir":true,"kvs":[{"key":"/foo","value":"two"},{"key":"/foo_dir","dir":true,"kvs":[{"key":"/foo_dir/foo","value":"bar"}]}],"index":11,"term":0} ``` -### Delete a directory +### Deleting a directory Let try to delete the directory `/foo_dir`. To delete a directory, we must add `recursive=true`. @@ -281,7 +281,7 @@ curl -L http://127.0.0.1:4001/v2/keys/foo_dir?recursive=true -XDELETE {"action":"delete","key":"/foo_dir","dir":true,"index":12,"term":0} ``` -### Create a hidden node +### Creating a hidden node We can create a hidden key-value pair or directory by add `_` prefix. The hidden item will not be list when using get for a directory. ```sh From f99d24310f6cd8b37851294105fb714ad37aadfd Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Wed, 23 Oct 2013 13:44:53 -0600 Subject: [PATCH 146/247] Fix conf extension and documentation. --- Documentation/external-documentation.md | 63 ++++++++++++------------- server/config.go | 2 +- 2 files changed, 32 insertions(+), 33 deletions(-) diff --git a/Documentation/external-documentation.md b/Documentation/external-documentation.md index a9be38ff0eb..a7d957aefdc 100644 --- a/Documentation/external-documentation.md +++ b/Documentation/external-documentation.md @@ -25,7 +25,7 @@ configuration files. * `-clientCAFile` - The path of the client CAFile. Enables client cert authentication when present. * `-clientCert` - The cert file of the client. * `-clientKey` - The key file of the client. -* `-configfile` - The path of the etcd config file. Defaults to `/etc/etcd/etcd.toml`. +* `-configfile` - The path of the etcd config file. Defaults to `/etc/etcd/etcd.conf`. * `-cors` - A comma separated white list of origins for cross-origin resource sharing. * `-cpuprofile` - The path to a file to output cpu profile data. Enables cpu profiling when present. * `-d` - The directory to store log and snapshot. Defaults to the current working directory. @@ -46,35 +46,34 @@ configuration files. ## Configuration File The etcd configuration file is written in [TOML](https://github.com/mojombo/toml) -and read from `/etc/etcd/etcd.toml` by default. +and read from `/etc/etcd/etcd.conf` by default. ```TOML -[etcd] - advertised_url = "127.0.0.1:4001" - ca_file = "" - cert_file = "" - cors = [] - cpu_profile_file = "" - datadir = "." - key_file = "" - listen_host = "127.0.0.1:4001" - machines = [] - machines_file = "" - max_cluster_size = 9 - max_result_buffer = 1024 - max_retry_attempts = 3 - name = "default-name" - snapshot = false - verbose = false - very_verbose = false - web_url = "" +advertised_url = "127.0.0.1:4001" +ca_file = "" +cert_file = "" +cors = [] +cpu_profile_file = "" +datadir = "." +key_file = "" +listen_host = "127.0.0.1:4001" +machines = [] +machines_file = "" +max_cluster_size = 9 +max_result_buffer = 1024 +max_retry_attempts = 3 +name = "default-name" +snapshot = false +verbose = false +very_verbose = false +web_url = "" -[raft] - advertised_url = "127.0.0.1:7001" - ca_file = "" - cert_file = "" - key_file = "" - listen_host = "127.0.0.1:7001" +[peer] +advertised_url = "127.0.0.1:7001" +ca_file = "" +cert_file = "" +key_file = "" +listen_host = "127.0.0.1:7001" ``` ## Environment Variables @@ -98,8 +97,8 @@ and read from `/etc/etcd/etcd.toml` by default. * `ETCD_VERBOSE` * `ETCD_VERY_VERBOSE` * `ETCD_WEB_URL` - * `ETCD_RAFT_ADVERTISED_URL` - * `ETCD_RAFT_CA_FILE` - * `ETCD_RAFT_CERT_FILE` - * `ETCD_RAFT_KEY_FILE` - * `ETCD_RAFT_LISTEN_HOST` + * `ETCD_PEER_ADVERTISED_URL` + * `ETCD_PEER_CA_FILE` + * `ETCD_PEER_CERT_FILE` + * `ETCD_PEER_KEY_FILE` + * `ETCD_PEER_LISTEN_HOST` diff --git a/server/config.go b/server/config.go index b86a433eeaa..39e7ca44e64 100644 --- a/server/config.go +++ b/server/config.go @@ -17,7 +17,7 @@ import ( ) // The default location for the etcd configuration file. -const DefaultSystemConfigPath = "/etc/etcd/etcd.toml" +const DefaultSystemConfigPath = "/etc/etcd/etcd.conf" // Config represents the server configuration. type Config struct { From 1fb3799118ba2669e478b9a312b50539e512f60d Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 26 Oct 2013 21:04:44 -0700 Subject: [PATCH 147/247] fix put ttl --- server/v2/put_handler.go | 9 +++++---- store/node.go | 2 +- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/server/v2/put_handler.go b/server/v2/put_handler.go index 536bbe21698..71c5401f5f4 100644 --- a/server/v2/put_handler.go +++ b/server/v2/put_handler.go @@ -72,10 +72,11 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { } c = &store.CompareAndSwapCommand{ - Key: key, - Value: value, - PrevValue: prevValue, - PrevIndex: prevIndex, + Key: key, + Value: value, + PrevValue: prevValue, + PrevIndex: prevIndex, + ExpireTime: expireTime, } return s.Dispatch(c, w, req) diff --git a/store/node.go b/store/node.go index bbb971bd46f..7203edd5f28 100644 --- a/store/node.go +++ b/store/node.go @@ -372,8 +372,8 @@ func (n *Node) UpdateTTL(expireTime time.Time) { } } + n.ExpireTime = expireTime if expireTime.Sub(Permanent) != 0 { - n.ExpireTime = expireTime n.Expire() } } From 7b60f8bdc378d99d80aecfdb84e0e1877cab4a2f Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 26 Oct 2013 21:19:51 -0700 Subject: [PATCH 148/247] feat kvpair include ttl --- store/kv_pairs.go | 14 ++++++++++---- store/node.go | 5 ++++- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/store/kv_pairs.go b/store/kv_pairs.go index 90e5c3fd6df..f2496d36335 100644 --- a/store/kv_pairs.go +++ b/store/kv_pairs.go @@ -1,11 +1,17 @@ package store +import ( + "time" +) + // When user list a directory, we add all the node into key-value pair slice type KeyValuePair struct { - Key string `json:"key, omitempty"` - Value string `json:"value,omitempty"` - Dir bool `json:"dir,omitempty"` - KVPairs kvPairs `json:"kvs,omitempty"` + Key string `json:"key, omitempty"` + Value string `json:"value,omitempty"` + Dir bool `json:"dir,omitempty"` + Expiration *time.Time `json:"expiration,omitempty"` + TTL int64 `json:"ttl,omitempty"` // Time to live in second + KVPairs kvPairs `json:"kvs,omitempty"` } type kvPairs []KeyValuePair diff --git a/store/node.go b/store/node.go index bbb971bd46f..2dfccc1ed37 100644 --- a/store/node.go +++ b/store/node.go @@ -322,6 +322,7 @@ func (n *Node) Pair(recurisive, sorted bool) KeyValuePair { Key: n.Path, Dir: true, } + pair.Expiration, pair.TTL = n.ExpirationAndTTL() if !recurisive { return pair @@ -354,10 +355,12 @@ func (n *Node) Pair(recurisive, sorted bool) KeyValuePair { return pair } - return KeyValuePair{ + pair := KeyValuePair{ Key: n.Path, Value: n.Value, } + pair.Expiration, pair.TTL = n.ExpirationAndTTL() + return pair } func (n *Node) UpdateTTL(expireTime time.Time) { From aa9ae329989c8a018cc5f2dcc967ba44419dd694 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sun, 27 Oct 2013 12:47:00 -0600 Subject: [PATCH 149/247] Internal versioning. --- server/join_command.go | 10 ++-- server/peer_server.go | 32 +++++----- server/peer_server_handlers.go | 5 +- server/registry.go | 5 +- server/server.go | 7 +-- server/v1/delete_key_handler.go | 3 +- server/v1/set_key_handler.go | 23 ++------ server/v2/delete_handler.go | 8 +-- server/v2/post_handler.go | 8 +-- server/v2/put_handler.go | 26 ++------- server/v2/tests/delete_handler_test.go | 29 +++++++++ server/version.go | 5 -- store/command_factory.go | 57 ++++++++++++++++++ store/store.go | 20 ++++++- store/v2/command_factory.go | 68 ++++++++++++++++++++++ store/{ => v2}/compare_and_swap_command.go | 7 ++- store/{ => v2}/create_command.go | 10 ++-- store/{ => v2}/delete_command.go | 7 ++- store/{ => v2}/set_command.go | 10 ++-- store/{ => v2}/update_command.go | 7 ++- tests/http_utils.go | 8 +++ 21 files changed, 248 insertions(+), 107 deletions(-) create mode 100644 server/v2/tests/delete_handler_test.go create mode 100644 store/command_factory.go create mode 100644 store/v2/command_factory.go rename store/{ => v2}/compare_and_swap_command.go (88%) rename store/{ => v2}/create_command.go (85%) rename store/{ => v2}/delete_command.go (84%) rename store/{ => v2}/set_command.go (85%) rename store/{ => v2}/update_command.go (85%) diff --git a/server/join_command.go b/server/join_command.go index 7bebbe70463..49dab4f2eaa 100644 --- a/server/join_command.go +++ b/server/join_command.go @@ -14,15 +14,17 @@ func init() { // The JoinCommand adds a node to the cluster. type JoinCommand struct { - RaftVersion string `json:"raftVersion"` + MinVersion int `json:"minVersion"` + MaxVersion int `json:"maxVersion"` Name string `json:"name"` RaftURL string `json:"raftURL"` EtcdURL string `json:"etcdURL"` } -func NewJoinCommand(version, name, raftUrl, etcdUrl string) *JoinCommand { +func NewJoinCommand(minVersion int, maxVersion int, name, raftUrl, etcdUrl string) *JoinCommand { return &JoinCommand{ - RaftVersion: version, + MinVersion: minVersion, + MaxVersion: maxVersion, Name: name, RaftURL: raftUrl, EtcdURL: etcdUrl, @@ -56,7 +58,7 @@ func (c *JoinCommand) Apply(server raft.Server) (interface{}, error) { } // Add to shared machine registry. - ps.registry.Register(c.Name, c.RaftVersion, c.RaftURL, c.EtcdURL, server.CommitIndex(), server.Term()) + ps.registry.Register(c.Name, c.RaftURL, c.EtcdURL, server.CommitIndex(), server.Term()) // Add peer in raft err := server.AddPeer(c.Name, "") diff --git a/server/peer_server.go b/server/peer_server.go index 2f53d7863c1..97b88a93d46 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -10,6 +10,7 @@ import ( "net" "net/http" "net/url" + "strconv" "time" etcdErr "github.com/coreos/etcd/error" @@ -209,7 +210,7 @@ func (s *PeerServer) SetServer(server *Server) { func (s *PeerServer) startAsLeader() { // leader need to join self as a peer for { - _, err := s.raftServer.Do(NewJoinCommand(PeerVersion, s.raftServer.Name(), s.url, s.server.URL())) + _, err := s.raftServer.Do(NewJoinCommand(store.MinVersion(), store.MaxVersion(), s.raftServer.Name(), s.url, s.server.URL())) if err == nil { break } @@ -245,7 +246,7 @@ func (s *PeerServer) startTransport(scheme string, tlsConf tls.Config) error { // internal commands raftMux.HandleFunc("/name", s.NameHttpHandler) - raftMux.HandleFunc("/version", s.RaftVersionHttpHandler) + raftMux.HandleFunc("/version", s.VersionHttpHandler) raftMux.HandleFunc("/join", s.JoinHttpHandler) raftMux.HandleFunc("/remove/", s.RemoveHttpHandler) raftMux.HandleFunc("/vote", s.VoteHttpHandler) @@ -263,21 +264,23 @@ func (s *PeerServer) startTransport(scheme string, tlsConf tls.Config) error { } -// getVersion fetches the raft version of a peer. This works for now but we -// will need to do something more sophisticated later when we allow mixed -// version clusters. -func getVersion(t *transporter, versionURL url.URL) (string, error) { +// getVersion fetches the peer version of a cluster. +func getVersion(t *transporter, versionURL url.URL) (int, error) { resp, req, err := t.Get(versionURL.String()) if err != nil { - return "", err + return 0, err } defer resp.Body.Close() t.CancelWhenTimeout(req) - body, err := ioutil.ReadAll(resp.Body) + if err != nil { + return 0, err + } - return string(body), nil + // Parse version number. + version, _ := strconv.Atoi(string(body)) + return version, nil } func (s *PeerServer) joinCluster(cluster []string) bool { @@ -315,14 +318,11 @@ func (s *PeerServer) joinByMachine(server raft.Server, machine string, scheme st if err != nil { return fmt.Errorf("Error during join version check: %v", err) } - - // TODO: versioning of the internal protocol. See: - // Documentation/internatl-protocol-versioning.md - if version != PeerVersion { - return fmt.Errorf("Unable to join: internal version mismatch, entire cluster must be running identical versions of etcd") + if version < store.MinVersion() || version > store.MaxVersion() { + return fmt.Errorf("Unable to join: cluster version is %d; version compatibility is %d - %d", version, store.MinVersion(), store.MaxVersion()) } - json.NewEncoder(&b).Encode(NewJoinCommand(PeerVersion, server.Name(), s.url, s.server.URL())) + json.NewEncoder(&b).Encode(NewJoinCommand(store.MinVersion(), store.MaxVersion(), server.Name(), s.url, s.server.URL())) joinURL := url.URL{Host: machine, Scheme: scheme, Path: "/join"} @@ -347,7 +347,7 @@ func (s *PeerServer) joinByMachine(server raft.Server, machine string, scheme st if resp.StatusCode == http.StatusTemporaryRedirect { address := resp.Header.Get("Location") log.Debugf("Send Join Request to %s", address) - json.NewEncoder(&b).Encode(NewJoinCommand(PeerVersion, server.Name(), s.url, s.server.URL())) + json.NewEncoder(&b).Encode(NewJoinCommand(store.MinVersion(), store.MaxVersion(), server.Name(), s.url, s.server.URL())) resp, req, err = t.Post(address, &b) } else if resp.StatusCode == http.StatusBadRequest { diff --git a/server/peer_server_handlers.go b/server/peer_server_handlers.go index 26ce7a5cd1a..b4b0a8b17dc 100644 --- a/server/peer_server_handlers.go +++ b/server/peer_server_handlers.go @@ -3,6 +3,7 @@ package server import ( "encoding/json" "net/http" + "strconv" etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/log" @@ -151,8 +152,8 @@ func (ps *PeerServer) NameHttpHandler(w http.ResponseWriter, req *http.Request) } // Response to the name request -func (ps *PeerServer) RaftVersionHttpHandler(w http.ResponseWriter, req *http.Request) { +func (ps *PeerServer) VersionHttpHandler(w http.ResponseWriter, req *http.Request) { log.Debugf("[recv] Get %s/version/ ", ps.url) w.WriteHeader(http.StatusOK) - w.Write([]byte(PeerVersion)) + w.Write([]byte(strconv.Itoa(ps.store.Version()))) } diff --git a/server/registry.go b/server/registry.go index 75b0e98c969..05cccc10c11 100644 --- a/server/registry.go +++ b/server/registry.go @@ -38,13 +38,13 @@ func NewRegistry(s store.Store) *Registry { } // Adds a node to the registry. -func (r *Registry) Register(name string, peerVersion string, peerURL string, url string, commitIndex uint64, term uint64) error { +func (r *Registry) Register(name string, peerURL string, url string, commitIndex uint64, term uint64) error { r.Lock() defer r.Unlock() // Write data to store. key := path.Join(RegistryKey, name) - value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", peerURL, url, peerVersion) + value := fmt.Sprintf("raft=%s&etcd=%s", peerURL, url) _, err := r.store.Create(key, value, false, store.Permanent, commitIndex, term) log.Debugf("Register: %s", name) return err @@ -175,6 +175,5 @@ func (r *Registry) load(name string) { r.nodes[name] = &node{ url: m["etcd"][0], peerURL: m["raft"][0], - peerVersion: m["raftVersion"][0], } } diff --git a/server/server.go b/server/server.go index 8566d2ebaf5..b88d5ac0ab3 100644 --- a/server/server.go +++ b/server/server.go @@ -15,6 +15,7 @@ import ( "github.com/coreos/etcd/server/v1" "github.com/coreos/etcd/server/v2" "github.com/coreos/etcd/store" + _ "github.com/coreos/etcd/store/v2" "github.com/coreos/go-raft" "github.com/gorilla/mux" ) @@ -366,11 +367,7 @@ func (s *Server) SpeedTestHandler(w http.ResponseWriter, req *http.Request) erro for i := 0; i < count; i++ { go func() { for j := 0; j < 10; j++ { - c := &store.SetCommand{ - Key: "foo", - Value: "bar", - ExpireTime: time.Unix(0, 0), - } + c := s.Store().CommandFactory().CreateSetCommand("foo", "bar", time.Unix(0, 0)) s.peerServer.RaftServer().Do(c) } c <- true diff --git a/server/v1/delete_key_handler.go b/server/v1/delete_key_handler.go index 1288d6597e1..44fe459e494 100644 --- a/server/v1/delete_key_handler.go +++ b/server/v1/delete_key_handler.go @@ -1,7 +1,6 @@ package v1 import ( - "github.com/coreos/etcd/store" "github.com/gorilla/mux" "net/http" ) @@ -10,6 +9,6 @@ import ( func DeleteKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) key := "/" + vars["key"] - c := &store.DeleteCommand{Key: key} + c := s.Store().CommandFactory().CreateDeleteCommand(key, false) return s.Dispatch(c, w, req) } diff --git a/server/v1/set_key_handler.go b/server/v1/set_key_handler.go index 5e29fafe6bb..7acfe7ecb21 100644 --- a/server/v1/set_key_handler.go +++ b/server/v1/set_key_handler.go @@ -31,27 +31,16 @@ func SetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { // If the "prevValue" is specified then test-and-set. Otherwise create a new key. var c raft.Command if prevValueArr, ok := req.Form["prevValue"]; ok { - if len(prevValueArr[0]) > 0 { // test against previous value - c = &store.CompareAndSwapCommand{ - Key: key, - Value: value, - PrevValue: prevValueArr[0], - ExpireTime: expireTime, - } + if len(prevValueArr[0]) > 0 { + // test against previous value + c = s.Store().CommandFactory().CreateCompareAndSwapCommand(key, value, prevValueArr[0], 0, expireTime) } else { - c = &store.CreateCommand{ // test against existence - Key: key, - Value: value, - ExpireTime: expireTime, - } + // test against existence + c = s.Store().CommandFactory().CreateCreateCommand(key, value, expireTime, false) } } else { - c = &store.SetCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - } + c = s.Store().CommandFactory().CreateSetCommand(key, value, expireTime) } return s.Dispatch(c, w, req) diff --git a/server/v2/delete_handler.go b/server/v2/delete_handler.go index 9012498cf2c..7afc02f0412 100644 --- a/server/v2/delete_handler.go +++ b/server/v2/delete_handler.go @@ -3,18 +3,14 @@ package v2 import ( "net/http" - "github.com/coreos/etcd/store" "github.com/gorilla/mux" ) func DeleteHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) key := "/" + vars["key"] + recursive := (req.FormValue("recursive") == "true") - c := &store.DeleteCommand{ - Key: key, - Recursive: (req.FormValue("recursive") == "true"), - } - + c := s.Store().CommandFactory().CreateDeleteCommand(key, recursive) return s.Dispatch(c, w, req) } diff --git a/server/v2/post_handler.go b/server/v2/post_handler.go index b55eddba4bf..4dc98b92510 100644 --- a/server/v2/post_handler.go +++ b/server/v2/post_handler.go @@ -18,12 +18,6 @@ func PostHandler(w http.ResponseWriter, req *http.Request, s Server) error { return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create", store.UndefIndex, store.UndefTerm) } - c := &store.CreateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - Unique: true, - } - + c := s.Store().CommandFactory().CreateCreateCommand(key, value, expireTime, true) return s.Dispatch(c, w, req) } diff --git a/server/v2/put_handler.go b/server/v2/put_handler.go index 536bbe21698..44eff345bab 100644 --- a/server/v2/put_handler.go +++ b/server/v2/put_handler.go @@ -71,31 +71,17 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { } } - c = &store.CompareAndSwapCommand{ - Key: key, - Value: value, - PrevValue: prevValue, - PrevIndex: prevIndex, - } - + c = s.Store().CommandFactory().CreateCompareAndSwapCommand(key, value, prevValue, prevIndex, store.Permanent) return s.Dispatch(c, w, req) } func SetHandler(w http.ResponseWriter, req *http.Request, s Server, key, value string, expireTime time.Time) error { - c := &store.SetCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - } + c := s.Store().CommandFactory().CreateSetCommand(key, value, expireTime) return s.Dispatch(c, w, req) } func CreateHandler(w http.ResponseWriter, req *http.Request, s Server, key, value string, expireTime time.Time) error { - c := &store.CreateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - } + c := s.Store().CommandFactory().CreateCreateCommand(key, value, expireTime, false) return s.Dispatch(c, w, req) } @@ -105,10 +91,6 @@ func UpdateHandler(w http.ResponseWriter, req *http.Request, s Server, key, valu return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", store.UndefIndex, store.UndefTerm) } - c := &store.UpdateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - } + c := s.Store().CommandFactory().CreateUpdateCommand(key, value, expireTime) return s.Dispatch(c, w, req) } diff --git a/server/v2/tests/delete_handler_test.go b/server/v2/tests/delete_handler_test.go new file mode 100644 index 00000000000..ab710c5ac32 --- /dev/null +++ b/server/v2/tests/delete_handler_test.go @@ -0,0 +1,29 @@ +package v2 + +import ( + "fmt" + "net/url" + "testing" + + "github.com/coreos/etcd/server" + "github.com/coreos/etcd/tests" + "github.com/stretchr/testify/assert" +) + +// Ensures that a key is deleted. +// +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX +// $ curl -X DELETE localhost:4001/v2/keys/foo/bar +// +func TestV2DeleteKey(t *testing.T) { + tests.RunServer(func(s *server.Server) { + v := url.Values{} + v.Set("value", "XXX") + resp, err := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + tests.ReadBody(resp) + resp, err = tests.DeleteForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), url.Values{}) + body := tests.ReadBody(resp) + assert.Nil(t, err, "") + assert.Equal(t, string(body), `{"action":"delete","key":"/foo/bar","prevValue":"XXX","index":4,"term":0}`, "") + }) +} diff --git a/server/version.go b/server/version.go index 354bda3a5bb..1d0fc6a4b4e 100644 --- a/server/version.go +++ b/server/version.go @@ -1,8 +1,3 @@ package server const Version = "v2" - -// TODO: The release version (generated from the git tag) will be the raft -// protocol version for now. When things settle down we will fix it like the -// client API above. -const PeerVersion = ReleaseVersion diff --git a/store/command_factory.go b/store/command_factory.go new file mode 100644 index 00000000000..07b169d49e6 --- /dev/null +++ b/store/command_factory.go @@ -0,0 +1,57 @@ +package store + +import ( + "fmt" + "time" + + "github.com/coreos/go-raft" +) + +// A lookup of factories by version. +var factories = make(map[int]CommandFactory) +var minVersion, maxVersion int + +// The CommandFactory provides a way to create different types of commands +// depending on the current version of the store. +type CommandFactory interface { + Version() int + CreateSetCommand(key string, value string, expireTime time.Time) raft.Command + CreateCreateCommand(key string, value string, expireTime time.Time, unique bool) raft.Command + CreateUpdateCommand(key string, value string, expireTime time.Time) raft.Command + CreateDeleteCommand(key string, recursive bool) raft.Command + CreateCompareAndSwapCommand(key string, value string, prevValue string, prevIndex uint64, expireTime time.Time) raft.Command +} + +// RegisterCommandFactory adds a command factory to the global registry. +func RegisterCommandFactory(factory CommandFactory) { + version := factory.Version() + + if GetCommandFactory(version) != nil { + panic(fmt.Sprintf("Command factory already registered for version: %d", factory.Version())) + } + + factories[version] = factory + + // Update compatibility versions. + if minVersion == 0 || version > minVersion { + minVersion = version + } + if maxVersion == 0 || version > maxVersion { + maxVersion = version + } +} + +// GetCommandFactory retrieves a command factory for a given command version. +func GetCommandFactory(version int) CommandFactory { + return factories[version] +} + +// MinVersion returns the minimum compatible store version. +func MinVersion() int { + return minVersion +} + +// MaxVersion returns the maximum compatible store version. +func MaxVersion() int { + return maxVersion +} diff --git a/store/store.go b/store/store.go index 20cfe0c87bc..22d1e9017a1 100644 --- a/store/store.go +++ b/store/store.go @@ -13,7 +13,12 @@ import ( etcdErr "github.com/coreos/etcd/error" ) +// The default version to set when the store is first initialized. +const defaultVersion = 2 + type Store interface { + Version() int + CommandFactory() CommandFactory Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) Set(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) @@ -34,6 +39,7 @@ type store struct { Index uint64 Term uint64 Stats *Stats + CurrentVersion int worldLock sync.RWMutex // stop the world lock } @@ -43,13 +49,23 @@ func New() Store { func newStore() *store { s := new(store) + s.CurrentVersion = defaultVersion s.Root = newDir(s, "/", UndefIndex, UndefTerm, nil, "", Permanent) s.Stats = newStats() s.WatcherHub = newWatchHub(1000) - return s } +// Version retrieves current version of the store. +func (s *store) Version() int { + return s.CurrentVersion +} + +// CommandFactory retrieves the command factory for the current version of the store. +func (s *store) CommandFactory() CommandFactory { + return GetCommandFactory(s.Version()) +} + // Get function returns a get event. // If recursive is true, it will return all the content under the node path. // If sorted is true, it will sort the content by keys. @@ -449,6 +465,7 @@ func (s *store) Save() ([]byte, error) { clonedStore.Root = s.Root.Clone() clonedStore.WatcherHub = s.WatcherHub.clone() clonedStore.Stats = s.Stats.clone() + clonedStore.CurrentVersion = s.CurrentVersion s.worldLock.Unlock() @@ -482,3 +499,4 @@ func (s *store) JsonStats() []byte { s.Stats.Watchers = uint64(s.WatcherHub.count) return s.Stats.toJson() } + diff --git a/store/v2/command_factory.go b/store/v2/command_factory.go new file mode 100644 index 00000000000..f69bd7f1b18 --- /dev/null +++ b/store/v2/command_factory.go @@ -0,0 +1,68 @@ +package v2 + +import ( + "time" + + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" +) + +func init() { + store.RegisterCommandFactory(&CommandFactory{}) +} + +// CommandFactory provides a pluggable way to create version 2 commands. +type CommandFactory struct { +} + +// Version returns the version of this factory. +func (f *CommandFactory) Version() int { + return 2 +} + +// CreateSetCommand creates a version 2 command to set a key to a given value in the store. +func (f *CommandFactory) CreateSetCommand(key string, value string, expireTime time.Time) raft.Command { + return &SetCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + } +} + +// CreateCreateCommand creates a version 2 command to create a new key in the store. +func (f *CommandFactory) CreateCreateCommand(key string, value string, expireTime time.Time, unique bool) raft.Command { + return &CreateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + Unique: unique, + } +} + +// CreateUpdateCommand creates a version 2 command to update a key to a given value in the store. +func (f *CommandFactory) CreateUpdateCommand(key string, value string, expireTime time.Time) raft.Command { + return &UpdateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + } +} + +// CreateDeleteCommand creates a version 2 command to delete a key from the store. +func (f *CommandFactory) CreateDeleteCommand(key string, recursive bool) raft.Command { + return &DeleteCommand{ + Key: key, + Recursive: recursive, + } +} + +// CreateCompareAndSwapCommand creates a version 2 command to conditionally set a key in the store. +func (f *CommandFactory) CreateCompareAndSwapCommand(key string, value string, prevValue string, prevIndex uint64, expireTime time.Time) raft.Command { + return &CompareAndSwapCommand{ + Key: key, + Value: value, + PrevValue: prevValue, + PrevIndex: prevIndex, + ExpireTime: expireTime, + } +} diff --git a/store/compare_and_swap_command.go b/store/v2/compare_and_swap_command.go similarity index 88% rename from store/compare_and_swap_command.go rename to store/v2/compare_and_swap_command.go index 8ee119f36fe..be1bcf826fe 100644 --- a/store/compare_and_swap_command.go +++ b/store/v2/compare_and_swap_command.go @@ -1,9 +1,10 @@ -package store +package v2 import ( "time" "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" "github.com/coreos/go-raft" ) @@ -22,12 +23,12 @@ type CompareAndSwapCommand struct { // The name of the testAndSet command in the log func (c *CompareAndSwapCommand) CommandName() string { - return "etcd:compareAndSwap" + return "etcd:v2:compareAndSwap" } // Set the key-value pair if the current value of the key equals to the given prevValue func (c *CompareAndSwapCommand) Apply(server raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(Store) + s, _ := server.StateMachine().(store.Store) e, err := s.CompareAndSwap(c.Key, c.PrevValue, c.PrevIndex, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) diff --git a/store/create_command.go b/store/v2/create_command.go similarity index 85% rename from store/create_command.go rename to store/v2/create_command.go index 6a2487cf0ee..8772e11f528 100644 --- a/store/create_command.go +++ b/store/v2/create_command.go @@ -1,9 +1,11 @@ -package store +package v2 import ( + "time" + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" "github.com/coreos/go-raft" - "time" ) func init() { @@ -20,12 +22,12 @@ type CreateCommand struct { // The name of the create command in the log func (c *CreateCommand) CommandName() string { - return "etcd:create" + return "etcd:v2:create" } // Create node func (c *CreateCommand) Apply(server raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(Store) + s, _ := server.StateMachine().(store.Store) e, err := s.Create(c.Key, c.Value, c.Unique, c.ExpireTime, server.CommitIndex(), server.Term()) diff --git a/store/delete_command.go b/store/v2/delete_command.go similarity index 84% rename from store/delete_command.go rename to store/v2/delete_command.go index 6ff3c5c6fab..861f919716d 100644 --- a/store/delete_command.go +++ b/store/v2/delete_command.go @@ -1,6 +1,7 @@ -package store +package v2 import ( + "github.com/coreos/etcd/store" "github.com/coreos/etcd/log" "github.com/coreos/go-raft" ) @@ -17,12 +18,12 @@ type DeleteCommand struct { // The name of the delete command in the log func (c *DeleteCommand) CommandName() string { - return "etcd:delete" + return "etcd:v2:delete" } // Delete the key func (c *DeleteCommand) Apply(server raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(Store) + s, _ := server.StateMachine().(store.Store) e, err := s.Delete(c.Key, c.Recursive, server.CommitIndex(), server.Term()) diff --git a/store/set_command.go b/store/v2/set_command.go similarity index 85% rename from store/set_command.go rename to store/v2/set_command.go index 55635cd9904..3d9a92f514c 100644 --- a/store/set_command.go +++ b/store/v2/set_command.go @@ -1,9 +1,11 @@ -package store +package v2 import ( + "time" + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" "github.com/coreos/go-raft" - "time" ) func init() { @@ -19,12 +21,12 @@ type SetCommand struct { // The name of the create command in the log func (c *SetCommand) CommandName() string { - return "etcd:set" + return "etcd:v2:set" } // Create node func (c *SetCommand) Apply(server raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(Store) + s, _ := server.StateMachine().(store.Store) // create a new node or replace the old node. e, err := s.Set(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) diff --git a/store/update_command.go b/store/v2/update_command.go similarity index 85% rename from store/update_command.go rename to store/v2/update_command.go index 8e353cdd068..e1136ddacb1 100644 --- a/store/update_command.go +++ b/store/v2/update_command.go @@ -1,7 +1,8 @@ -package store +package v2 import ( "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" "github.com/coreos/go-raft" "time" ) @@ -19,12 +20,12 @@ type UpdateCommand struct { // The name of the update command in the log func (c *UpdateCommand) CommandName() string { - return "etcd:update" + return "etcd:v2:update" } // Create node func (c *UpdateCommand) Apply(server raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(Store) + s, _ := server.StateMachine().(store.Store) e, err := s.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) diff --git a/tests/http_utils.go b/tests/http_utils.go index b1395373d03..ebe6f527304 100644 --- a/tests/http_utils.go +++ b/tests/http_utils.go @@ -55,6 +55,14 @@ func PutForm(url string, data url.Values) (*http.Response, error) { return Put(url, "application/x-www-form-urlencoded", strings.NewReader(data.Encode())) } +func Delete(url string, bodyType string, body io.Reader) (*http.Response, error) { + return send("DELETE", url, bodyType, body) +} + +func DeleteForm(url string, data url.Values) (*http.Response, error) { + return Delete(url, "application/x-www-form-urlencoded", strings.NewReader(data.Encode())) +} + func send(method string, url string, bodyType string, body io.Reader) (*http.Response, error) { c := NewHTTPClient() From ec175d4c47f8ca293eb66d734dee56e818ed5915 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 28 Oct 2013 09:42:54 -0700 Subject: [PATCH 150/247] test add test for ttl in kv pair --- store/store_test.go | 23 ++++++++++++----------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/store/store_test.go b/store/store_test.go index 2328c4be940..263e628ac3f 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -29,6 +29,7 @@ func TestStoreGetDirectory(t *testing.T) { s.Create("/foo/baz", "", false, Permanent, 5, 1) s.Create("/foo/baz/bat", "Y", false, Permanent, 6, 1) s.Create("/foo/baz/_hidden", "*", false, Permanent, 7, 1) + s.Create("/foo/baz/ttl", "Y", false, time.Now().Add(time.Second*3), 8, 1) e, err := s.Get("/foo", true, false, 8, 1) assert.Nil(t, err, "") assert.Equal(t, e.Action, "get", "") @@ -39,10 +40,14 @@ func TestStoreGetDirectory(t *testing.T) { assert.Equal(t, e.KVPairs[0].Dir, false, "") assert.Equal(t, e.KVPairs[1].Key, "/foo/baz", "") assert.Equal(t, e.KVPairs[1].Dir, true, "") - assert.Equal(t, len(e.KVPairs[1].KVPairs), 1, "") + assert.Equal(t, len(e.KVPairs[1].KVPairs), 2, "") assert.Equal(t, e.KVPairs[1].KVPairs[0].Key, "/foo/baz/bat", "") assert.Equal(t, e.KVPairs[1].KVPairs[0].Value, "Y", "") assert.Equal(t, e.KVPairs[1].KVPairs[0].Dir, false, "") + assert.Equal(t, e.KVPairs[1].KVPairs[1].Key, "/foo/baz/ttl", "") + assert.Equal(t, e.KVPairs[1].KVPairs[1].Value, "Y", "") + assert.Equal(t, e.KVPairs[1].KVPairs[1].Dir, false, "") + assert.Equal(t, e.KVPairs[1].KVPairs[1].TTL, 3, "") } // Ensure that the store can retrieve a directory in sorted order. @@ -63,7 +68,6 @@ func TestStoreGetSorted(t *testing.T) { assert.Equal(t, e.KVPairs[2].Key, "/foo/z", "") } - // Ensure that the store can create a new key if it doesn't already exist. func TestStoreCreateValue(t *testing.T) { s := newStore() @@ -139,10 +143,10 @@ func TestStoreUpdateFailsIfDirectory(t *testing.T) { func TestStoreUpdateValueTTL(t *testing.T) { s := newStore() s.Create("/foo", "bar", false, Permanent, 2, 1) - _, err := s.Update("/foo", "baz", time.Now().Add(1 * time.Millisecond), 3, 1) + _, err := s.Update("/foo", "baz", time.Now().Add(1*time.Millisecond), 3, 1) e, _ := s.Get("/foo", false, false, 3, 1) assert.Equal(t, e.Value, "baz", "") - + time.Sleep(2 * time.Millisecond) e, err = s.Get("/foo", false, false, 3, 1) assert.Nil(t, e, "") @@ -154,10 +158,10 @@ func TestStoreUpdateDirTTL(t *testing.T) { s := newStore() s.Create("/foo", "", false, Permanent, 2, 1) s.Create("/foo/bar", "baz", false, Permanent, 3, 1) - _, err := s.Update("/foo", "", time.Now().Add(1 * time.Millisecond), 3, 1) + _, err := s.Update("/foo", "", time.Now().Add(1*time.Millisecond), 3, 1) e, _ := s.Get("/foo/bar", false, false, 3, 1) assert.Equal(t, e.Value, "baz", "") - + time.Sleep(2 * time.Millisecond) e, err = s.Get("/foo/bar", false, false, 3, 1) assert.Nil(t, e, "") @@ -193,7 +197,6 @@ func TestStoreDeleteDiretoryFailsIfNonRecursive(t *testing.T) { assert.Nil(t, e, "") } - // Ensure that the store can conditionally update a key if it has a previous value. func TestStoreCompareAndSwapPrevValue(t *testing.T) { s := newStore() @@ -337,7 +340,7 @@ func TestStoreWatchRecursiveCompareAndSwap(t *testing.T) { // Ensure that the store can watch for key expiration. func TestStoreWatchExpire(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, time.Now().Add(1 * time.Millisecond), 2, 1) + s.Create("/foo", "bar", false, time.Now().Add(1*time.Millisecond), 2, 1) c, _ := s.Watch("/foo", false, 0, 0, 1) e := nbselect(c) assert.Nil(t, e, "") @@ -372,7 +375,7 @@ func TestStoreRecoverWithExpiration(t *testing.T) { s := newStore() s.Create("/foo", "", false, Permanent, 2, 1) s.Create("/foo/x", "bar", false, Permanent, 3, 1) - s.Create("/foo/y", "baz", false, time.Now().Add(5 * time.Millisecond), 4, 1) + s.Create("/foo/y", "baz", false, time.Now().Add(5*time.Millisecond), 4, 1) b, err := s.Save() time.Sleep(10 * time.Millisecond) @@ -389,8 +392,6 @@ func TestStoreRecoverWithExpiration(t *testing.T) { assert.Nil(t, e, "") } - - // Performs a non-blocking select on an event channel. func nbselect(c <-chan *Event) *Event { select { From ddf527e09207e4a12699b00c7fda0ea778a877ca Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Tue, 29 Oct 2013 15:21:22 -0600 Subject: [PATCH 151/247] Add version upgrade endpoint. --- server/peer_server.go | 50 +++++++++++++++++++------- server/peer_server_handlers.go | 38 ++++++++++++++++++-- store/command_factory.go | 1 + store/v2/command_factory.go | 5 +++ store/v2/compare_and_swap_command.go | 2 +- store/v2/create_command.go | 2 +- store/v2/delete_command.go | 2 +- store/v2/set_command.go | 2 +- store/v2/update_command.go | 2 +- tests/functional/version_check_test.go | 46 ++++++++++++++++++++++++ 10 files changed, 131 insertions(+), 19 deletions(-) create mode 100644 tests/functional/version_check_test.go diff --git a/server/peer_server.go b/server/peer_server.go index 97b88a93d46..dbcba981431 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -17,6 +17,7 @@ import ( "github.com/coreos/etcd/log" "github.com/coreos/etcd/store" "github.com/coreos/go-raft" + "github.com/gorilla/mux" ) type PeerServer struct { @@ -236,25 +237,27 @@ func (s *PeerServer) startAsFollower(cluster []string) { func (s *PeerServer) startTransport(scheme string, tlsConf tls.Config) error { log.Infof("raft server [name %s, listen on %s, advertised url %s]", s.name, s.listenHost, s.url) - raftMux := http.NewServeMux() + router := mux.NewRouter() s.httpServer = &http.Server{ - Handler: raftMux, + Handler: router, TLSConfig: &tlsConf, Addr: s.listenHost, } // internal commands - raftMux.HandleFunc("/name", s.NameHttpHandler) - raftMux.HandleFunc("/version", s.VersionHttpHandler) - raftMux.HandleFunc("/join", s.JoinHttpHandler) - raftMux.HandleFunc("/remove/", s.RemoveHttpHandler) - raftMux.HandleFunc("/vote", s.VoteHttpHandler) - raftMux.HandleFunc("/log", s.GetLogHttpHandler) - raftMux.HandleFunc("/log/append", s.AppendEntriesHttpHandler) - raftMux.HandleFunc("/snapshot", s.SnapshotHttpHandler) - raftMux.HandleFunc("/snapshotRecovery", s.SnapshotRecoveryHttpHandler) - raftMux.HandleFunc("/etcdURL", s.EtcdURLHttpHandler) + router.HandleFunc("/name", s.NameHttpHandler) + router.HandleFunc("/version", s.VersionHttpHandler) + router.HandleFunc("/version/{version:[0-9]+}/check", s.VersionCheckHttpHandler) + router.HandleFunc("/upgrade", s.UpgradeHttpHandler) + router.HandleFunc("/join", s.JoinHttpHandler) + router.HandleFunc("/remove/{name:.+}", s.RemoveHttpHandler) + router.HandleFunc("/vote", s.VoteHttpHandler) + router.HandleFunc("/log", s.GetLogHttpHandler) + router.HandleFunc("/log/append", s.AppendEntriesHttpHandler) + router.HandleFunc("/snapshot", s.SnapshotHttpHandler) + router.HandleFunc("/snapshotRecovery", s.SnapshotRecoveryHttpHandler) + router.HandleFunc("/etcdURL", s.EtcdURLHttpHandler) if scheme == "http" { return s.listenAndServe() @@ -283,6 +286,29 @@ func getVersion(t *transporter, versionURL url.URL) (int, error) { return version, nil } +// Upgradable checks whether all peers in a cluster support an upgrade to the next store version. +func (s *PeerServer) Upgradable() error { + nextVersion := s.store.Version() + 1 + for _, peerURL := range s.registry.PeerURLs(s.raftServer.Leader(), s.name) { + u, err := url.Parse(peerURL) + if err != nil { + return fmt.Errorf("PeerServer: Cannot parse URL: '%s' (%s)", peerURL, err) + } + + t, _ := s.raftServer.Transporter().(*transporter) + checkURL := (&url.URL{Host: u.Host, Scheme: s.tlsConf.Scheme, Path: fmt.Sprintf("/version/%d/check", nextVersion)}).String() + resp, _, err := t.Get(checkURL) + if err != nil { + return fmt.Errorf("PeerServer: Cannot check version compatibility: %s", u.Host) + } + if resp.StatusCode != 200 { + return fmt.Errorf("PeerServer: Version %d is not compatible with peer: %s", nextVersion, u.Host) + } + } + + return nil +} + func (s *PeerServer) joinCluster(cluster []string) bool { for _, machine := range cluster { if len(machine) == 0 { diff --git a/server/peer_server_handlers.go b/server/peer_server_handlers.go index b4b0a8b17dc..be665dbf530 100644 --- a/server/peer_server_handlers.go +++ b/server/peer_server_handlers.go @@ -7,7 +7,9 @@ import ( etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" "github.com/coreos/go-raft" + "github.com/gorilla/mux" ) // Get all the current logs @@ -134,9 +136,9 @@ func (ps *PeerServer) RemoveHttpHandler(w http.ResponseWriter, req *http.Request return } - nodeName := req.URL.Path[len("/remove/"):] + vars := mux.Vars(req) command := &RemoveCommand{ - Name: nodeName, + Name: vars["name"], } log.Debugf("[recv] Remove Request [%s]", command.Name) @@ -157,3 +159,35 @@ func (ps *PeerServer) VersionHttpHandler(w http.ResponseWriter, req *http.Reques w.WriteHeader(http.StatusOK) w.Write([]byte(strconv.Itoa(ps.store.Version()))) } + +// Checks whether a given version is supported. +func (ps *PeerServer) VersionCheckHttpHandler(w http.ResponseWriter, req *http.Request) { + log.Debugf("[recv] Get %s%s ", ps.url, req.URL.Path) + vars := mux.Vars(req) + version, _ := strconv.Atoi(vars["version"]) + if version >= store.MinVersion() && version <= store.MaxVersion() { + w.WriteHeader(http.StatusOK) + } else { + w.WriteHeader(http.StatusForbidden) + } +} + +// Upgrades the current store version to the next version. +func (ps *PeerServer) UpgradeHttpHandler(w http.ResponseWriter, req *http.Request) { + log.Debugf("[recv] Get %s/version", ps.url) + + // Check if upgrade is possible for all nodes. + if err := ps.Upgradable(); err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + + // Create an upgrade command from the current version. + c := ps.store.CommandFactory().CreateUpgradeCommand() + if err := ps.server.Dispatch(c, w, req); err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + + w.WriteHeader(http.StatusOK) +} diff --git a/store/command_factory.go b/store/command_factory.go index 07b169d49e6..9b52f42f9cd 100644 --- a/store/command_factory.go +++ b/store/command_factory.go @@ -15,6 +15,7 @@ var minVersion, maxVersion int // depending on the current version of the store. type CommandFactory interface { Version() int + CreateUpgradeCommand() raft.Command CreateSetCommand(key string, value string, expireTime time.Time) raft.Command CreateCreateCommand(key string, value string, expireTime time.Time, unique bool) raft.Command CreateUpdateCommand(key string, value string, expireTime time.Time) raft.Command diff --git a/store/v2/command_factory.go b/store/v2/command_factory.go index f69bd7f1b18..4f0e7260c81 100644 --- a/store/v2/command_factory.go +++ b/store/v2/command_factory.go @@ -20,6 +20,11 @@ func (f *CommandFactory) Version() int { return 2 } +// CreateUpgradeCommand is a no-op since version 2 is the first version to support store versioning. +func (f *CommandFactory) CreateUpgradeCommand() raft.Command { + return &raft.NOPCommand{} +} + // CreateSetCommand creates a version 2 command to set a key to a given value in the store. func (f *CommandFactory) CreateSetCommand(key string, value string, expireTime time.Time) raft.Command { return &SetCommand{ diff --git a/store/v2/compare_and_swap_command.go b/store/v2/compare_and_swap_command.go index be1bcf826fe..1de79fb1ff3 100644 --- a/store/v2/compare_and_swap_command.go +++ b/store/v2/compare_and_swap_command.go @@ -23,7 +23,7 @@ type CompareAndSwapCommand struct { // The name of the testAndSet command in the log func (c *CompareAndSwapCommand) CommandName() string { - return "etcd:v2:compareAndSwap" + return "etcd:compareAndSwap" } // Set the key-value pair if the current value of the key equals to the given prevValue diff --git a/store/v2/create_command.go b/store/v2/create_command.go index 8772e11f528..e187d99f7f1 100644 --- a/store/v2/create_command.go +++ b/store/v2/create_command.go @@ -22,7 +22,7 @@ type CreateCommand struct { // The name of the create command in the log func (c *CreateCommand) CommandName() string { - return "etcd:v2:create" + return "etcd:create" } // Create node diff --git a/store/v2/delete_command.go b/store/v2/delete_command.go index 861f919716d..6bd48368f44 100644 --- a/store/v2/delete_command.go +++ b/store/v2/delete_command.go @@ -18,7 +18,7 @@ type DeleteCommand struct { // The name of the delete command in the log func (c *DeleteCommand) CommandName() string { - return "etcd:v2:delete" + return "etcd:delete" } // Delete the key diff --git a/store/v2/set_command.go b/store/v2/set_command.go index 3d9a92f514c..4f6ecf59f1f 100644 --- a/store/v2/set_command.go +++ b/store/v2/set_command.go @@ -21,7 +21,7 @@ type SetCommand struct { // The name of the create command in the log func (c *SetCommand) CommandName() string { - return "etcd:v2:set" + return "etcd:set" } // Create node diff --git a/store/v2/update_command.go b/store/v2/update_command.go index e1136ddacb1..d080ecced44 100644 --- a/store/v2/update_command.go +++ b/store/v2/update_command.go @@ -20,7 +20,7 @@ type UpdateCommand struct { // The name of the update command in the log func (c *UpdateCommand) CommandName() string { - return "etcd:v2:update" + return "etcd:update" } // Create node diff --git a/tests/functional/version_check_test.go b/tests/functional/version_check_test.go new file mode 100644 index 00000000000..98a15539049 --- /dev/null +++ b/tests/functional/version_check_test.go @@ -0,0 +1,46 @@ +package test + +import ( + "net/http" + "os" + "testing" + "time" +) + +// Ensure that a node can reply to a version check appropriately. +func TestVersionCheck(t *testing.T) { + procAttr := new(os.ProcAttr) + procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} + args := []string{"etcd", "-n=node1", "-f", "-d=/tmp/version_check"} + + process, err := os.StartProcess(EtcdBinPath, args, procAttr) + if err != nil { + t.Fatal("start process failed:" + err.Error()) + return + } + defer process.Kill() + + time.Sleep(time.Second) + + // Check a version too small. + resp, _ := http.Get("http://localhost:7001/version/1/check") + resp.Body.Close() + if resp.StatusCode != http.StatusForbidden { + t.Fatal("Invalid version check: ", resp.StatusCode) + } + + // Check a version too large. + resp, _ = http.Get("http://localhost:7001/version/3/check") + resp.Body.Close() + if resp.StatusCode != http.StatusForbidden { + t.Fatal("Invalid version check: ", resp.StatusCode) + } + + // Check a version that's just right. + resp, _ = http.Get("http://localhost:7001/version/2/check") + resp.Body.Close() + if resp.StatusCode != http.StatusOK { + t.Fatal("Invalid version check: ", resp.StatusCode) + } +} + From 107762e82aaaeb7211900224698c23cefc32c8d7 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Wed, 30 Oct 2013 15:14:34 -0700 Subject: [PATCH 152/247] fix snapshot --- server/peer_server.go | 6 +++--- store/stats.go | 2 +- store/store.go | 5 +++++ 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/server/peer_server.go b/server/peer_server.go index 2f53d7863c1..538d3ac702e 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -60,7 +60,7 @@ func NewPeerServer(name string, path string, url string, listenHost string, tlsC tlsInfo: tlsInfo, registry: registry, store: store, - snapConf: &snapshotConf{time.Second * 3, 0, 20 * 1000}, + snapConf: &snapshotConf{time.Second * 3, 0, 2}, followersStats: &raftFollowersStats{ Leader: name, Followers: make(map[string]*raftFollowerStats), @@ -391,10 +391,10 @@ func (s *PeerServer) PeerStats() []byte { func (s *PeerServer) monitorSnapshot() { for { time.Sleep(s.snapConf.checkingInterval) - currentWrites := 0 + currentWrites := s.store.TotalTransactions() - s.snapConf.lastWrites if uint64(currentWrites) > s.snapConf.writesThr { s.raftServer.TakeSnapshot() - s.snapConf.lastWrites = 0 + s.snapConf.lastWrites = s.store.TotalTransactions() } } } diff --git a/store/stats.go b/store/stats.go index c18f5cbf094..40dff117537 100644 --- a/store/stats.go +++ b/store/stats.go @@ -73,7 +73,7 @@ func (s *Stats) TotalReads() uint64 { return s.GetSuccess + s.GetFail } -func (s *Stats) TotalWrites() uint64 { +func (s *Stats) TotalTranscations() uint64 { return s.SetSuccess + s.SetFail + s.DeleteSuccess + s.DeleteFail + s.CompareAndSwapSuccess + s.CompareAndSwapFail + diff --git a/store/store.go b/store/store.go index 20cfe0c87bc..7fa21a6389f 100644 --- a/store/store.go +++ b/store/store.go @@ -25,6 +25,7 @@ type Store interface { Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error) Save() ([]byte, error) Recovery(state []byte) error + TotalTransactions() uint64 JsonStats() []byte } @@ -482,3 +483,7 @@ func (s *store) JsonStats() []byte { s.Stats.Watchers = uint64(s.WatcherHub.count) return s.Stats.toJson() } + +func (s *store) TotalTransactions() uint64 { + return s.Stats.TotalTranscations() +} From bd651edf80279314dcd0b5bd335491608396e89a Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Wed, 30 Oct 2013 15:27:16 -0700 Subject: [PATCH 153/247] test add integration test --- server/v2/tests/get_handler_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/v2/tests/get_handler_test.go b/server/v2/tests/get_handler_test.go index a248d33f423..becca1ef2a1 100644 --- a/server/v2/tests/get_handler_test.go +++ b/server/v2/tests/get_handler_test.go @@ -42,6 +42,7 @@ func TestV2GetKeyRecursively(t *testing.T) { tests.RunServer(func(s *server.Server) { v := url.Values{} v.Set("value", "XXX") + v.Set("ttl", "10") resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/x"), v) tests.ReadBody(resp) @@ -60,6 +61,7 @@ func TestV2GetKeyRecursively(t *testing.T) { kv0 := body["kvs"].([]interface{})[0].(map[string]interface{}) assert.Equal(t, kv0["key"], "/foo/x", "") assert.Equal(t, kv0["value"], "XXX", "") + assert.Equal(t, kv0["ttl"], 10, "") kv1 := body["kvs"].([]interface{})[1].(map[string]interface{}) assert.Equal(t, kv1["key"], "/foo/y", "") @@ -105,7 +107,6 @@ func TestV2WatchKey(t *testing.T) { }) } - // Ensures that a watcher can wait for a value to be set after a given index. // // $ curl localhost:4001/v2/keys/foo/bar?wait=true&waitIndex=4 @@ -149,4 +150,3 @@ func TestV2WatchKeyWithIndex(t *testing.T) { assert.Equal(t, body["term"], 0, "") }) } - From f723651a6d65d0210ce39a84b983face2252ea3b Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Wed, 30 Oct 2013 15:46:01 -0700 Subject: [PATCH 154/247] fix test: should wait for watch response --- server/v2/tests/get_handler_test.go | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/server/v2/tests/get_handler_test.go b/server/v2/tests/get_handler_test.go index becca1ef2a1..d6ceae58a0f 100644 --- a/server/v2/tests/get_handler_test.go +++ b/server/v2/tests/get_handler_test.go @@ -116,9 +116,11 @@ func TestV2WatchKey(t *testing.T) { func TestV2WatchKeyWithIndex(t *testing.T) { tests.RunServer(func(s *server.Server) { var body map[string]interface{} + c := make(chan bool) go func() { resp, _ := tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar?wait=true&waitIndex=5")) body = tests.ReadBodyJSON(resp) + c <- true }() // Make sure response didn't fire early. @@ -142,6 +144,14 @@ func TestV2WatchKeyWithIndex(t *testing.T) { // A response should follow from the GET above. time.Sleep(1 * time.Millisecond) + + select { + case <-c: + + default: + t.Fatal("cannot get watch result") + } + assert.NotNil(t, body, "") assert.Equal(t, body["action"], "set", "") assert.Equal(t, body["key"], "/foo/bar", "") From 0cd78e210a9c62bca639845c4f41ba4890ebd2eb Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Wed, 30 Oct 2013 15:51:46 -0700 Subject: [PATCH 155/247] tests more delay for passing tests on travis --- tests/server_utils.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/server_utils.go b/tests/server_utils.go index adddf336bcf..04d9cadd712 100644 --- a/tests/server_utils.go +++ b/tests/server_utils.go @@ -5,14 +5,14 @@ import ( "os" "time" - "github.com/coreos/etcd/store" "github.com/coreos/etcd/server" + "github.com/coreos/etcd/store" ) const ( - testName = "ETCDTEST" + testName = "ETCDTEST" testClientURL = "localhost:4401" - testRaftURL = "localhost:7701" + testRaftURL = "localhost:7701" ) // Starts a server in a temporary directory. @@ -22,8 +22,8 @@ func RunServer(f func(*server.Server)) { store := store.New() registry := server.NewRegistry(store) - ps := server.NewPeerServer(testName, path, testRaftURL, testRaftURL, &server.TLSConfig{Scheme:"http"}, &server.TLSInfo{}, registry, store) - s := server.New(testName, testClientURL, testClientURL, &server.TLSConfig{Scheme:"http"}, &server.TLSInfo{}, ps, registry, store) + ps := server.NewPeerServer(testName, path, testRaftURL, testRaftURL, &server.TLSConfig{Scheme: "http"}, &server.TLSInfo{}, registry, store) + s := server.New(testName, testClientURL, testClientURL, &server.TLSConfig{Scheme: "http"}, &server.TLSInfo{}, ps, registry, store) ps.SetServer(s) // Start up peer server. @@ -32,17 +32,17 @@ func RunServer(f func(*server.Server)) { c <- true ps.ListenAndServe(false, []string{}) }() - <- c + <-c // Start up etcd server. go func() { c <- true s.ListenAndServe() }() - <- c + <-c // Wait to make sure servers have started. - time.Sleep(5 * time.Millisecond) + time.Sleep(50 * time.Millisecond) // Execute the function passed in. f(s) From 9d0de611a7a0d546deb8708c10f66ec3f8855fbf Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Wed, 30 Oct 2013 17:36:15 -0700 Subject: [PATCH 156/247] feat add snapCount parameter --- Documentation/external-documentation.md | 104 +++ config.go | 143 ---- etcd.go | 229 ++---- server/config.go | 406 ++++++++++ server/config_test.go | 479 +++++++++++ server/info.go | 19 + server/join_command.go | 10 +- server/peer_server.go | 84 +- server/peer_server_handlers.go | 43 +- server/registry.go | 5 +- server/server.go | 11 +- server/tls_config.go | 1 + server/tls_info.go | 69 ++ server/transporter.go | 14 +- server/transporter_test.go | 2 + server/util.go | 13 + server/v1/delete_key_handler.go | 3 +- server/v1/set_key_handler.go | 23 +- server/v2/delete_handler.go | 8 +- server/v2/post_handler.go | 8 +- server/v2/put_handler.go | 26 +- server/v2/tests/delete_handler_test.go | 29 + server/v2/tests/get_handler_test.go | 14 +- server/version.go | 5 - store/command_factory.go | 58 ++ store/node.go | 2 +- store/store.go | 31 +- store/v2/command_factory.go | 73 ++ store/{ => v2}/compare_and_swap_command.go | 5 +- store/{ => v2}/create_command.go | 8 +- store/{ => v2}/delete_command.go | 5 +- store/{ => v2}/set_command.go | 8 +- store/{ => v2}/update_command.go | 5 +- test.sh | 3 + tests/functional/version_check_test.go | 46 ++ tests/http_utils.go | 8 + tests/server_utils.go | 17 +- third_party/deps | 1 + .../github.com/BurntSushi/toml/.gitignore | 4 + .../github.com/BurntSushi/toml/COMPATIBLE | 3 + .../github.com/BurntSushi/toml/COPYING | 14 + .../github.com/BurntSushi/toml/Makefile | 14 + .../github.com/BurntSushi/toml/README.md | 163 ++++ .../BurntSushi/toml/_examples/example.go | 59 ++ .../BurntSushi/toml/_examples/example.toml | 35 + .../BurntSushi/toml/_examples/hard.toml | 22 + .../BurntSushi/toml/_examples/implicit.toml | 4 + .../toml/_examples/invalid-apples.toml | 6 + .../BurntSushi/toml/_examples/invalid.toml | 35 + .../BurntSushi/toml/_examples/readme1.toml | 5 + .../BurntSushi/toml/_examples/readme2.toml | 1 + .../github.com/BurntSushi/toml/decode.go | 429 ++++++++++ .../github.com/BurntSushi/toml/decode_test.go | 343 ++++++++ third_party/github.com/BurntSushi/toml/doc.go | 10 + .../github.com/BurntSushi/toml/encode.go | 99 +++ .../github.com/BurntSushi/toml/encode_test.go | 25 + third_party/github.com/BurntSushi/toml/lex.go | 741 ++++++++++++++++++ .../github.com/BurntSushi/toml/lex_test.go | 59 ++ .../github.com/BurntSushi/toml/out_test.go | 19 + .../github.com/BurntSushi/toml/parse.go | 388 +++++++++ .../github.com/BurntSushi/toml/parse_test.go | 61 ++ .../github.com/BurntSushi/toml/session.vim | 1 + .../BurntSushi/toml/toml-test-go/COPYING | 14 + .../BurntSushi/toml/toml-test-go/README.md | 14 + .../BurntSushi/toml/toml-test-go/main.go | 89 +++ .../github.com/BurntSushi/toml/tomlv/COPYING | 14 + .../BurntSushi/toml/tomlv/README.md | 22 + .../github.com/BurntSushi/toml/tomlv/main.go | 60 ++ .../github.com/BurntSushi/toml/type-check.go | 78 ++ util.go | 88 --- 70 files changed, 4407 insertions(+), 530 deletions(-) create mode 100644 Documentation/external-documentation.md delete mode 100644 config.go create mode 100644 server/config.go create mode 100644 server/config_test.go create mode 100644 server/info.go create mode 100644 server/v2/tests/delete_handler_test.go create mode 100644 store/command_factory.go create mode 100644 store/v2/command_factory.go rename store/{ => v2}/compare_and_swap_command.go (91%) rename store/{ => v2}/create_command.go (88%) rename store/{ => v2}/delete_command.go (87%) rename store/{ => v2}/set_command.go (88%) rename store/{ => v2}/update_command.go (88%) create mode 100644 tests/functional/version_check_test.go create mode 100644 third_party/github.com/BurntSushi/toml/.gitignore create mode 100644 third_party/github.com/BurntSushi/toml/COMPATIBLE create mode 100644 third_party/github.com/BurntSushi/toml/COPYING create mode 100644 third_party/github.com/BurntSushi/toml/Makefile create mode 100644 third_party/github.com/BurntSushi/toml/README.md create mode 100644 third_party/github.com/BurntSushi/toml/_examples/example.go create mode 100644 third_party/github.com/BurntSushi/toml/_examples/example.toml create mode 100644 third_party/github.com/BurntSushi/toml/_examples/hard.toml create mode 100644 third_party/github.com/BurntSushi/toml/_examples/implicit.toml create mode 100644 third_party/github.com/BurntSushi/toml/_examples/invalid-apples.toml create mode 100644 third_party/github.com/BurntSushi/toml/_examples/invalid.toml create mode 100644 third_party/github.com/BurntSushi/toml/_examples/readme1.toml create mode 100644 third_party/github.com/BurntSushi/toml/_examples/readme2.toml create mode 100644 third_party/github.com/BurntSushi/toml/decode.go create mode 100644 third_party/github.com/BurntSushi/toml/decode_test.go create mode 100644 third_party/github.com/BurntSushi/toml/doc.go create mode 100644 third_party/github.com/BurntSushi/toml/encode.go create mode 100644 third_party/github.com/BurntSushi/toml/encode_test.go create mode 100644 third_party/github.com/BurntSushi/toml/lex.go create mode 100644 third_party/github.com/BurntSushi/toml/lex_test.go create mode 100644 third_party/github.com/BurntSushi/toml/out_test.go create mode 100644 third_party/github.com/BurntSushi/toml/parse.go create mode 100644 third_party/github.com/BurntSushi/toml/parse_test.go create mode 100644 third_party/github.com/BurntSushi/toml/session.vim create mode 100644 third_party/github.com/BurntSushi/toml/toml-test-go/COPYING create mode 100644 third_party/github.com/BurntSushi/toml/toml-test-go/README.md create mode 100644 third_party/github.com/BurntSushi/toml/toml-test-go/main.go create mode 100644 third_party/github.com/BurntSushi/toml/tomlv/COPYING create mode 100644 third_party/github.com/BurntSushi/toml/tomlv/README.md create mode 100644 third_party/github.com/BurntSushi/toml/tomlv/main.go create mode 100644 third_party/github.com/BurntSushi/toml/type-check.go delete mode 100644 util.go diff --git a/Documentation/external-documentation.md b/Documentation/external-documentation.md new file mode 100644 index 00000000000..a7d957aefdc --- /dev/null +++ b/Documentation/external-documentation.md @@ -0,0 +1,104 @@ +# Etcd Configuration + +Configuration options can be set in three places: + + 1. Command line flags + 2. Environment variables + 3. Configuration file + +Options set on the command line take precedence over all other sources. +Options set in environment variables take precedence over options set in +configuration files. + +## Command Line Flags + +### Required + +* `-n` - The node name. Defaults to `default-name`. + +### Optional + +* `-c` - The advertised public hostname:port for client communication. Defaults to `127.0.0.1:4001`. +* `-cl` - The listening hostname for client communication. Defaults to advertised ip. +* `-C` - A comma separated list of machines in the cluster (i.e `"203.0.113.101:7001,203.0.113.102:7001"`). +* `-CF` - The file path containing a comma separated list of machines in the cluster. +* `-clientCAFile` - The path of the client CAFile. Enables client cert authentication when present. +* `-clientCert` - The cert file of the client. +* `-clientKey` - The key file of the client. +* `-configfile` - The path of the etcd config file. Defaults to `/etc/etcd/etcd.conf`. +* `-cors` - A comma separated white list of origins for cross-origin resource sharing. +* `-cpuprofile` - The path to a file to output cpu profile data. Enables cpu profiling when present. +* `-d` - The directory to store log and snapshot. Defaults to the current working directory. +* `-m` - The max size of result buffer. Defaults to `1024`. +* `-maxsize` - The max size of the cluster. Defaults to `9`. +* `-r` - The max retry attempts when trying to join a cluster. Defaults to `3`. +* `-s` - The advertised public hostname:port for server communication. Defaults to `127.0.0.1:7001`. +* `-sl` - The listening hostname for server communication. Defaults to advertised ip. +* `-serverCAFile` - The path of the CAFile. Enables client/peer cert authentication when present. +* `-serverCert` - The cert file of the server. +* `-serverKey` - The key file of the server. +* `-snapshot` - Open or close snapshot. Defaults to `false`. +* `-v` - Enable verbose logging. Defaults to `false`. +* `-vv` - Enable very verbose logging. Defaults to `false`. +* `-version` - Print the version and exit. +* `-w` - The hostname:port of web interface. + +## Configuration File + +The etcd configuration file is written in [TOML](https://github.com/mojombo/toml) +and read from `/etc/etcd/etcd.conf` by default. + +```TOML +advertised_url = "127.0.0.1:4001" +ca_file = "" +cert_file = "" +cors = [] +cpu_profile_file = "" +datadir = "." +key_file = "" +listen_host = "127.0.0.1:4001" +machines = [] +machines_file = "" +max_cluster_size = 9 +max_result_buffer = 1024 +max_retry_attempts = 3 +name = "default-name" +snapshot = false +verbose = false +very_verbose = false +web_url = "" + +[peer] +advertised_url = "127.0.0.1:7001" +ca_file = "" +cert_file = "" +key_file = "" +listen_host = "127.0.0.1:7001" +``` + +## Environment Variables + + * `ETCD_ADVERTISED_URL` + * `ETCD_CA_FILE` + * `ETCD_CERT_FILE` + * `ETCD_CORS` + * `ETCD_CONFIG_FILE` + * `ETCD_CPU_PROFILE_FILE` + * `ETCD_DATADIR` + * `ETCD_KEY_FILE` + * `ETCD_LISTEN_HOST` + * `ETCD_MACHINES` + * `ETCD_MACHINES_FILE` + * `ETCD_MAX_RETRY_ATTEMPTS` + * `ETCD_MAX_CLUSTER_SIZE` + * `ETCD_MAX_RESULT_BUFFER` + * `ETCD_NAME` + * `ETCD_SNAPSHOT` + * `ETCD_VERBOSE` + * `ETCD_VERY_VERBOSE` + * `ETCD_WEB_URL` + * `ETCD_PEER_ADVERTISED_URL` + * `ETCD_PEER_CA_FILE` + * `ETCD_PEER_CERT_FILE` + * `ETCD_PEER_KEY_FILE` + * `ETCD_PEER_LISTEN_HOST` diff --git a/config.go b/config.go deleted file mode 100644 index d1549c7becb..00000000000 --- a/config.go +++ /dev/null @@ -1,143 +0,0 @@ -package main - -import ( - "crypto/tls" - "crypto/x509" - "encoding/json" - "encoding/pem" - "io/ioutil" - "os" - "path/filepath" - - "github.com/coreos/etcd/log" - "github.com/coreos/etcd/server" -) - -//-------------------------------------- -// Config -//-------------------------------------- - -// Get the server info from previous conf file -// or from the user -func getInfo(path string) *Info { - - infoPath := filepath.Join(path, "info") - - if force { - // Delete the old configuration if exist - logPath := filepath.Join(path, "log") - confPath := filepath.Join(path, "conf") - snapshotPath := filepath.Join(path, "snapshot") - os.Remove(infoPath) - os.Remove(logPath) - os.Remove(confPath) - os.RemoveAll(snapshotPath) - } else if info := readInfo(infoPath); info != nil { - log.Infof("Found node configuration in '%s'. Ignoring flags", infoPath) - return info - } - - // Read info from command line - info := &argInfo - - // Write to file. - content, _ := json.MarshalIndent(info, "", " ") - content = []byte(string(content) + "\n") - if err := ioutil.WriteFile(infoPath, content, 0644); err != nil { - log.Fatalf("Unable to write info to file: %v", err) - } - - log.Infof("Wrote node configuration to '%s'", infoPath) - - return info -} - -// readInfo reads from info file and decode to Info struct -func readInfo(path string) *Info { - file, err := os.Open(path) - - if err != nil { - if os.IsNotExist(err) { - return nil - } - log.Fatal(err) - } - defer file.Close() - - info := &Info{} - - content, err := ioutil.ReadAll(file) - if err != nil { - log.Fatalf("Unable to read info: %v", err) - return nil - } - - if err = json.Unmarshal(content, &info); err != nil { - log.Fatalf("Unable to parse info: %v", err) - return nil - } - - return info -} - -func tlsConfigFromInfo(info server.TLSInfo) (t server.TLSConfig, ok bool) { - var keyFile, certFile, CAFile string - var tlsCert tls.Certificate - var err error - - t.Scheme = "http" - - keyFile = info.KeyFile - certFile = info.CertFile - CAFile = info.CAFile - - // If the user do not specify key file, cert file and - // CA file, the type will be HTTP - if keyFile == "" && certFile == "" && CAFile == "" { - return t, true - } - - // both the key and cert must be present - if keyFile == "" || certFile == "" { - return t, false - } - - tlsCert, err = tls.LoadX509KeyPair(certFile, keyFile) - if err != nil { - log.Fatal(err) - } - - t.Scheme = "https" - t.Server.ClientAuth, t.Server.ClientCAs = newCertPool(CAFile) - - // The client should trust the RootCA that the Server uses since - // everyone is a peer in the network. - t.Client.Certificates = []tls.Certificate{tlsCert} - t.Client.RootCAs = t.Server.ClientCAs - - return t, true -} - -// newCertPool creates x509 certPool and corresponding Auth Type. -// If the given CAfile is valid, add the cert into the pool and verify the clients' -// certs against the cert in the pool. -// If the given CAfile is empty, do not verify the clients' cert. -// If the given CAfile is not valid, fatal. -func newCertPool(CAFile string) (tls.ClientAuthType, *x509.CertPool) { - if CAFile == "" { - return tls.NoClientCert, nil - } - pemByte, err := ioutil.ReadFile(CAFile) - check(err) - - block, pemByte := pem.Decode(pemByte) - - cert, err := x509.ParseCertificate(block.Bytes) - check(err) - - certPool := x509.NewCertPool() - - certPool.AddCert(cert) - - return tls.RequireAndVerifyClientCert, certPool -} diff --git a/etcd.go b/etcd.go index f9df25c6dc6..ec79e519d8b 100644 --- a/etcd.go +++ b/etcd.go @@ -5,7 +5,8 @@ import ( "fmt" "io/ioutil" "os" - "strings" + "os/signal" + "runtime/pprof" "github.com/coreos/etcd/log" "github.com/coreos/etcd/server" @@ -13,189 +14,103 @@ import ( "github.com/coreos/go-raft" ) -//------------------------------------------------------------------------------ -// -// Initialization -// -//------------------------------------------------------------------------------ - -var ( - veryVerbose bool - - machines string - machinesFile string - - cluster []string - - argInfo Info - dirPath string - - force bool - - printVersion bool - - maxSize int - - snapshot bool - - retryTimes int - - maxClusterSize int - - cpuprofile string - - cors string -) - -func init() { - flag.BoolVar(&printVersion, "version", false, "print the version and exit") - - flag.BoolVar(&log.Verbose, "v", false, "verbose logging") - flag.BoolVar(&veryVerbose, "vv", false, "very verbose logging") - - flag.StringVar(&machines, "C", "", "the ip address and port of a existing machines in the cluster, sepearate by comma") - flag.StringVar(&machinesFile, "CF", "", "the file contains a list of existing machines in the cluster, seperate by comma") - - flag.StringVar(&argInfo.Name, "n", "default-name", "the node name (required)") - flag.StringVar(&argInfo.EtcdURL, "c", "127.0.0.1:4001", "the advertised public hostname:port for etcd client communication") - flag.StringVar(&argInfo.RaftURL, "s", "127.0.0.1:7001", "the advertised public hostname:port for raft server communication") - flag.StringVar(&argInfo.EtcdListenHost, "cl", "", "the listening hostname for etcd client communication (defaults to advertised ip)") - flag.StringVar(&argInfo.RaftListenHost, "sl", "", "the listening hostname for raft server communication (defaults to advertised ip)") - flag.StringVar(&argInfo.WebURL, "w", "", "the hostname:port of web interface") - - flag.StringVar(&argInfo.RaftTLS.CAFile, "serverCAFile", "", "the path of the CAFile") - flag.StringVar(&argInfo.RaftTLS.CertFile, "serverCert", "", "the cert file of the server") - flag.StringVar(&argInfo.RaftTLS.KeyFile, "serverKey", "", "the key file of the server") - - flag.StringVar(&argInfo.EtcdTLS.CAFile, "clientCAFile", "", "the path of the client CAFile") - flag.StringVar(&argInfo.EtcdTLS.CertFile, "clientCert", "", "the cert file of the client") - flag.StringVar(&argInfo.EtcdTLS.KeyFile, "clientKey", "", "the key file of the client") - - flag.StringVar(&dirPath, "d", ".", "the directory to store log and snapshot") - - flag.BoolVar(&force, "f", false, "force new node configuration if existing is found (WARNING: data loss!)") - - flag.BoolVar(&snapshot, "snapshot", false, "open or close snapshot") - - flag.IntVar(&maxSize, "m", 1024, "the max size of result buffer") - - flag.IntVar(&retryTimes, "r", 3, "the max retry attempts when trying to join a cluster") - - flag.IntVar(&maxClusterSize, "maxsize", 9, "the max size of the cluster") - - flag.StringVar(&cpuprofile, "cpuprofile", "", "write cpu profile to file") - - flag.StringVar(&cors, "cors", "", "whitelist origins for cross-origin resource sharing (e.g. '*' or 'http://localhost:8001,etc')") -} - -//------------------------------------------------------------------------------ -// -// Typedefs -// -//------------------------------------------------------------------------------ - -type Info struct { - Name string `json:"name"` - - RaftURL string `json:"raftURL"` - EtcdURL string `json:"etcdURL"` - WebURL string `json:"webURL"` - - RaftListenHost string `json:"raftListenHost"` - EtcdListenHost string `json:"etcdListenHost"` - - RaftTLS server.TLSInfo `json:"raftTLS"` - EtcdTLS server.TLSInfo `json:"etcdTLS"` -} - -//------------------------------------------------------------------------------ -// -// Functions -// -//------------------------------------------------------------------------------ - -//-------------------------------------- -// Main -//-------------------------------------- - func main() { - flag.Parse() + parseFlags() - if printVersion { - fmt.Println(server.ReleaseVersion) - os.Exit(0) + // Load configuration. + var config = server.NewConfig() + if err := config.Load(os.Args[1:]); err != nil { + log.Fatal("Configuration error:", err) } - if cpuprofile != "" { - runCPUProfile() - } - - if veryVerbose { + // Turn on logging. + if config.VeryVerbose { log.Verbose = true raft.SetLogLevel(raft.Debug) + } else if config.Verbose { + log.Verbose = true } - if machines != "" { - cluster = strings.Split(machines, ",") - } else if machinesFile != "" { - b, err := ioutil.ReadFile(machinesFile) - if err != nil { - log.Fatalf("Unable to read the given machines file: %s", err) - } - cluster = strings.Split(string(b), ",") - } - - // Check TLS arguments - raftTLSConfig, ok := tlsConfigFromInfo(argInfo.RaftTLS) - if !ok { - log.Fatal("Please specify cert and key file or cert and key file and CAFile or none of the three") + // Create data directory if it doesn't already exist. + if err := os.MkdirAll(config.DataDir, 0744); err != nil { + log.Fatalf("Unable to create path: %s", err) } - etcdTLSConfig, ok := tlsConfigFromInfo(argInfo.EtcdTLS) - if !ok { - log.Fatal("Please specify cert and key file or cert and key file and CAFile or none of the three") + // Load info object. + info, err := config.Info() + if err != nil { + log.Fatal("info:", err) } - - argInfo.Name = strings.TrimSpace(argInfo.Name) - if argInfo.Name == "" { + if info.Name == "" { log.Fatal("ERROR: server name required. e.g. '-n=server_name'") } - // Check host name arguments - argInfo.RaftURL = sanitizeURL(argInfo.RaftURL, raftTLSConfig.Scheme) - argInfo.EtcdURL = sanitizeURL(argInfo.EtcdURL, etcdTLSConfig.Scheme) - argInfo.WebURL = sanitizeURL(argInfo.WebURL, "http") - - argInfo.RaftListenHost = sanitizeListenHost(argInfo.RaftListenHost, argInfo.RaftURL) - argInfo.EtcdListenHost = sanitizeListenHost(argInfo.EtcdListenHost, argInfo.EtcdURL) - - // Read server info from file or grab it from user. - if err := os.MkdirAll(dirPath, 0744); err != nil { - log.Fatalf("Unable to create path: %s", err) + // Retrieve TLS configuration. + tlsConfig, err := info.EtcdTLS.Config() + if err != nil { + log.Fatal("Client TLS:", err) + } + peerTLSConfig, err := info.RaftTLS.Config() + if err != nil { + log.Fatal("Peer TLS:", err) } - info := getInfo(dirPath) - - // Create etcd key-value store + // Create etcd key-value store and registry. store := store.New() - - // Create a shared node registry. registry := server.NewRegistry(store) // Create peer server. - ps := server.NewPeerServer(info.Name, dirPath, info.RaftURL, info.RaftListenHost, &raftTLSConfig, &info.RaftTLS, registry, store) - ps.MaxClusterSize = maxClusterSize - ps.RetryTimes = retryTimes + ps := server.NewPeerServer(info.Name, config.DataDir, info.RaftURL, info.RaftListenHost, &peerTLSConfig, &info.RaftTLS, registry, store, config.SnapCount) + ps.MaxClusterSize = config.MaxClusterSize + ps.RetryTimes = config.MaxRetryAttempts - s := server.New(info.Name, info.EtcdURL, info.EtcdListenHost, &etcdTLSConfig, &info.EtcdTLS, ps, registry, store) - if err := s.AllowOrigins(cors); err != nil { + // Create client server. + s := server.New(info.Name, info.EtcdURL, info.EtcdListenHost, &tlsConfig, &info.EtcdTLS, ps, registry, store) + if err := s.AllowOrigins(config.Cors); err != nil { panic(err) } ps.SetServer(s) + // Run peer server in separate thread while the client server blocks. go func() { - log.Fatal(ps.ListenAndServe(snapshot, cluster)) + log.Fatal(ps.ListenAndServe(config.Snapshot, config.Machines)) }() log.Fatal(s.ListenAndServe()) } + +// Parses non-configuration flags. +func parseFlags() { + var versionFlag bool + var cpuprofile string + + f := flag.NewFlagSet(os.Args[0], -1) + f.SetOutput(ioutil.Discard) + f.BoolVar(&versionFlag, "version", false, "print the version and exit") + f.StringVar(&cpuprofile, "cpuprofile", "", "write cpu profile to file") + f.Parse(os.Args[1:]) + + // Print version if necessary. + if versionFlag { + fmt.Println(server.ReleaseVersion) + os.Exit(0) + } + + // Begin CPU profiling if specified. + if cpuprofile != "" { + f, err := os.Create(cpuprofile) + if err != nil { + log.Fatal(err) + } + pprof.StartCPUProfile(f) + + c := make(chan os.Signal, 1) + signal.Notify(c, os.Interrupt) + go func() { + sig := <-c + log.Infof("captured %v, stopping profiler and exiting..", sig) + pprof.StopCPUProfile() + os.Exit(1) + }() + } +} diff --git a/server/config.go b/server/config.go new file mode 100644 index 00000000000..95ec59737d0 --- /dev/null +++ b/server/config.go @@ -0,0 +1,406 @@ +package server + +import ( + "encoding/json" + "flag" + "fmt" + "io/ioutil" + "net" + "net/url" + "os" + "path/filepath" + "reflect" + "strconv" + "strings" + + "github.com/BurntSushi/toml" +) + +// The default location for the etcd configuration file. +const DefaultSystemConfigPath = "/etc/etcd/etcd.conf" + +// Config represents the server configuration. +type Config struct { + SystemPath string + + AdvertisedUrl string `toml:"advertised_url" env:"ETCD_ADVERTISED_URL"` + CAFile string `toml:"ca_file" env:"ETCD_CA_FILE"` + CertFile string `toml:"cert_file" env:"ETCD_CERT_FILE"` + Cors []string `toml:"cors" env:"ETCD_CORS"` + DataDir string `toml:"datadir" env:"ETCD_DATADIR"` + KeyFile string `toml:"key_file" env:"ETCD_KEY_FILE"` + ListenHost string `toml:"listen_host" env:"ETCD_LISTEN_HOST"` + Machines []string `toml:"machines" env:"ETCD_MACHINES"` + MachinesFile string `toml:"machines_file" env:"ETCD_MACHINES_FILE"` + MaxClusterSize int `toml:"max_cluster_size" env:"ETCD_MAX_CLUSTER_SIZE"` + MaxResultBuffer int `toml:"max_result_buffer" env:"ETCD_MAX_RESULT_BUFFER"` + MaxRetryAttempts int `toml:"max_retry_attempts" env:"ETCD_MAX_RETRY_ATTEMPTS"` + Name string `toml:"name" env:"ETCD_NAME"` + Snapshot bool `toml:"snapshot" env:"ETCD_SNAPSHOT"` + SnapCount int `toml:"snap_count" env:"ETCD_SNAPCOUNT"` + Verbose bool `toml:"verbose" env:"ETCD_VERBOSE"` + VeryVerbose bool `toml:"very_verbose" env:"ETCD_VERY_VERBOSE"` + WebURL string `toml:"web_url" env:"ETCD_WEB_URL"` + + Peer struct { + AdvertisedUrl string `toml:"advertised_url" env:"ETCD_PEER_ADVERTISED_URL"` + CAFile string `toml:"ca_file" env:"ETCD_PEER_CA_FILE"` + CertFile string `toml:"cert_file" env:"ETCD_PEER_CERT_FILE"` + KeyFile string `toml:"key_file" env:"ETCD_PEER_KEY_FILE"` + ListenHost string `toml:"listen_host" env:"ETCD_PEER_LISTEN_HOST"` + } +} + +// NewConfig returns a Config initialized with default values. +func NewConfig() *Config { + c := new(Config) + c.SystemPath = DefaultSystemConfigPath + c.AdvertisedUrl = "127.0.0.1:4001" + c.AdvertisedUrl = "127.0.0.1:4001" + c.DataDir = "." + c.MaxClusterSize = 9 + c.MaxResultBuffer = 1024 + c.MaxRetryAttempts = 3 + c.Peer.AdvertisedUrl = "127.0.0.1:7001" + c.SnapCount = 10000 + return c +} + +// Loads the configuration from the system config, command line config, +// environment variables, and finally command line arguments. +func (c *Config) Load(arguments []string) error { + var path string + f := flag.NewFlagSet("etcd", -1) + f.SetOutput(ioutil.Discard) + f.StringVar(&path, "config", "", "path to config file") + f.Parse(arguments) + + // Load from system file. + if err := c.LoadSystemFile(); err != nil { + return err + } + + // Load from config file specified in arguments. + if path != "" { + if err := c.LoadFile(path); err != nil { + return err + } + } + + // Load from the environment variables next. + if err := c.LoadEnv(); err != nil { + return err + } + + // Load from command line flags. + if err := c.LoadFlags(arguments); err != nil { + return err + } + + // Loads machines if a machine file was specified. + if err := c.LoadMachineFile(); err != nil { + return err + } + + // Sanitize all the input fields. + if err := c.Sanitize(); err != nil { + return fmt.Errorf("sanitize:", err) + } + + return nil +} + +// Loads from the system etcd configuration file if it exists. +func (c *Config) LoadSystemFile() error { + if _, err := os.Stat(c.SystemPath); os.IsNotExist(err) { + return nil + } + return c.LoadFile(c.SystemPath) +} + +// Loads configuration from a file. +func (c *Config) LoadFile(path string) error { + _, err := toml.DecodeFile(path, &c) + return err +} + +// LoadEnv loads the configuration via environment variables. +func (c *Config) LoadEnv() error { + if err := c.loadEnv(c); err != nil { + return err + } + if err := c.loadEnv(&c.Peer); err != nil { + return err + } + return nil +} + +func (c *Config) loadEnv(target interface{}) error { + value := reflect.Indirect(reflect.ValueOf(target)) + typ := value.Type() + for i := 0; i < typ.NumField(); i++ { + field := typ.Field(i) + + // Retrieve environment variable. + v := strings.TrimSpace(os.Getenv(field.Tag.Get("env"))) + if v == "" { + continue + } + + // Set the appropriate type. + switch field.Type.Kind() { + case reflect.Bool: + value.Field(i).SetBool(v != "0" && v != "false") + case reflect.Int: + newValue, err := strconv.ParseInt(v, 10, 0) + if err != nil { + return fmt.Errorf("Parse error: %s: %s", field.Tag.Get("env"), err) + } + value.Field(i).SetInt(newValue) + case reflect.String: + value.Field(i).SetString(v) + case reflect.Slice: + value.Field(i).Set(reflect.ValueOf(trimsplit(v, ","))) + } + } + return nil +} + +// Loads configuration from command line flags. +func (c *Config) LoadFlags(arguments []string) error { + var machines, cors string + var force bool + + f := flag.NewFlagSet(os.Args[0], flag.ContinueOnError) + + f.BoolVar(&force, "f", false, "force new node configuration if existing is found (WARNING: data loss!)") + + f.BoolVar(&c.Verbose, "v", c.Verbose, "verbose logging") + f.BoolVar(&c.VeryVerbose, "vv", c.Verbose, "very verbose logging") + + f.StringVar(&machines, "C", "", "the ip address and port of a existing machines in the cluster, sepearate by comma") + f.StringVar(&c.MachinesFile, "CF", c.MachinesFile, "the file contains a list of existing machines in the cluster, seperate by comma") + + f.StringVar(&c.Name, "n", c.Name, "the node name (required)") + f.StringVar(&c.AdvertisedUrl, "c", c.AdvertisedUrl, "the advertised public hostname:port for etcd client communication") + f.StringVar(&c.Peer.AdvertisedUrl, "s", c.Peer.AdvertisedUrl, "the advertised public hostname:port for raft server communication") + f.StringVar(&c.ListenHost, "cl", c.ListenHost, "the listening hostname for etcd client communication (defaults to advertised ip)") + f.StringVar(&c.Peer.ListenHost, "sl", c.Peer.ListenHost, "the listening hostname for raft server communication (defaults to advertised ip)") + f.StringVar(&c.WebURL, "w", c.WebURL, "the hostname:port of web interface") + + f.StringVar(&c.Peer.CAFile, "serverCAFile", c.Peer.CAFile, "the path of the CAFile") + f.StringVar(&c.Peer.CertFile, "serverCert", c.Peer.CertFile, "the cert file of the server") + f.StringVar(&c.Peer.KeyFile, "serverKey", c.Peer.KeyFile, "the key file of the server") + + f.StringVar(&c.CAFile, "clientCAFile", c.CAFile, "the path of the client CAFile") + f.StringVar(&c.CertFile, "clientCert", c.CertFile, "the cert file of the client") + f.StringVar(&c.KeyFile, "clientKey", c.KeyFile, "the key file of the client") + + f.StringVar(&c.DataDir, "d", c.DataDir, "the directory to store log and snapshot") + f.IntVar(&c.MaxResultBuffer, "m", c.MaxResultBuffer, "the max size of result buffer") + f.IntVar(&c.MaxRetryAttempts, "r", c.MaxRetryAttempts, "the max retry attempts when trying to join a cluster") + f.IntVar(&c.MaxClusterSize, "maxsize", c.MaxClusterSize, "the max size of the cluster") + f.StringVar(&cors, "cors", "", "whitelist origins for cross-origin resource sharing (e.g. '*' or 'http://localhost:8001,etc')") + + f.BoolVar(&c.Snapshot, "snapshot", c.Snapshot, "open or close snapshot") + f.IntVar(&c.SnapCount, "snapCount", c.SnapCount, "save the in memory logs and states to a snapshot file after snapCount transactions") + + // These flags are ignored since they were already parsed. + var path string + f.StringVar(&path, "config", "", "path to config file") + + f.Parse(arguments) + + // Convert some parameters to lists. + if machines != "" { + c.Machines = trimsplit(machines, ",") + } + if cors != "" { + c.Cors = trimsplit(cors, ",") + } + + // Force remove server configuration if specified. + if force { + c.Reset() + } + + return nil +} + +// LoadMachineFile loads the machines listed in the machine file. +func (c *Config) LoadMachineFile() error { + if c.MachinesFile == "" { + return nil + } + + b, err := ioutil.ReadFile(c.MachinesFile) + if err != nil { + return fmt.Errorf("Machines file error: %s", err) + } + c.Machines = trimsplit(string(b), ",") + + return nil +} + +// Reset removes all server configuration files. +func (c *Config) Reset() error { + if err := os.RemoveAll(filepath.Join(c.DataDir, "info")); err != nil { + return err + } + if err := os.RemoveAll(filepath.Join(c.DataDir, "log")); err != nil { + return err + } + if err := os.RemoveAll(filepath.Join(c.DataDir, "conf")); err != nil { + return err + } + if err := os.RemoveAll(filepath.Join(c.DataDir, "snapshot")); err != nil { + return err + } + + return nil +} + +// Reads the info file from the file system or initializes it based on the config. +func (c *Config) Info() (*Info, error) { + info := &Info{} + path := filepath.Join(c.DataDir, "info") + + // Open info file and read it out. + f, err := os.Open(path) + if err != nil && !os.IsNotExist(err) { + return nil, err + } else if f != nil { + defer f.Close() + if err := json.NewDecoder(f).Decode(&info); err != nil { + return nil, err + } + return info, nil + } + + // If the file doesn't exist then initialize it. + info.Name = strings.TrimSpace(c.Name) + info.EtcdURL = c.AdvertisedUrl + info.EtcdListenHost = c.ListenHost + info.RaftURL = c.Peer.AdvertisedUrl + info.RaftListenHost = c.Peer.ListenHost + info.WebURL = c.WebURL + info.EtcdTLS = c.TLSInfo() + info.RaftTLS = c.PeerTLSInfo() + + // Write to file. + f, err = os.Create(path) + if err != nil { + return nil, err + } + defer f.Close() + + if err := json.NewEncoder(f).Encode(info); err != nil { + return nil, err + } + + return info, nil +} + +// Sanitize cleans the input fields. +func (c *Config) Sanitize() error { + tlsConfig, err := c.TLSConfig() + if err != nil { + return err + } + + peerTlsConfig, err := c.PeerTLSConfig() + if err != nil { + return err + } + + // Sanitize the URLs first. + if c.AdvertisedUrl, err = sanitizeURL(c.AdvertisedUrl, tlsConfig.Scheme); err != nil { + return fmt.Errorf("Advertised URL: %s", err) + } + if c.ListenHost, err = sanitizeListenHost(c.ListenHost, c.AdvertisedUrl); err != nil { + return fmt.Errorf("Listen Host: %s", err) + } + if c.WebURL, err = sanitizeURL(c.WebURL, "http"); err != nil { + return fmt.Errorf("Web URL: %s", err) + } + if c.Peer.AdvertisedUrl, err = sanitizeURL(c.Peer.AdvertisedUrl, peerTlsConfig.Scheme); err != nil { + return fmt.Errorf("Peer Advertised URL: %s", err) + } + if c.Peer.ListenHost, err = sanitizeListenHost(c.Peer.ListenHost, c.Peer.AdvertisedUrl); err != nil { + return fmt.Errorf("Peer Listen Host: %s", err) + } + + return nil +} + +// TLSInfo retrieves a TLSInfo object for the client server. +func (c *Config) TLSInfo() TLSInfo { + return TLSInfo{ + CAFile: c.CAFile, + CertFile: c.CertFile, + KeyFile: c.KeyFile, + } +} + +// ClientTLSConfig generates the TLS configuration for the client server. +func (c *Config) TLSConfig() (TLSConfig, error) { + return c.TLSInfo().Config() +} + +// PeerTLSInfo retrieves a TLSInfo object for the peer server. +func (c *Config) PeerTLSInfo() TLSInfo { + return TLSInfo{ + CAFile: c.Peer.CAFile, + CertFile: c.Peer.CertFile, + KeyFile: c.Peer.KeyFile, + } +} + +// PeerTLSConfig generates the TLS configuration for the peer server. +func (c *Config) PeerTLSConfig() (TLSConfig, error) { + return c.PeerTLSInfo().Config() +} + +// sanitizeURL will cleanup a host string in the format hostname:port and +// attach a schema. +func sanitizeURL(host string, defaultScheme string) (string, error) { + // Blank URLs are fine input, just return it + if len(host) == 0 { + return host, nil + } + + p, err := url.Parse(host) + if err != nil { + return "", err + } + + // Make sure the host is in Host:Port format + _, _, err = net.SplitHostPort(host) + if err != nil { + return "", err + } + + p = &url.URL{Host: host, Scheme: defaultScheme} + return p.String(), nil +} + +// sanitizeListenHost cleans up the ListenHost parameter and appends a port +// if necessary based on the advertised port. +func sanitizeListenHost(listen string, advertised string) (string, error) { + aurl, err := url.Parse(advertised) + if err != nil { + return "", err + } + + ahost, aport, err := net.SplitHostPort(aurl.Host) + if err != nil { + return "", err + } + + // If the listen host isn't set use the advertised host + if listen == "" { + listen = ahost + } + + return net.JoinHostPort(listen, aport), nil +} diff --git a/server/config_test.go b/server/config_test.go new file mode 100644 index 00000000000..7205348887f --- /dev/null +++ b/server/config_test.go @@ -0,0 +1,479 @@ +package server + +import ( + "io/ioutil" + "os" + "testing" + + "github.com/BurntSushi/toml" + "github.com/stretchr/testify/assert" +) + + +// Ensures that a configuration can be deserialized from TOML. +func TestConfigTOML(t *testing.T) { + content := ` + advertised_url = "127.0.0.1:4002" + ca_file = "/tmp/file.ca" + cert_file = "/tmp/file.cert" + cors = ["*"] + cpu_profile_file = "XXX" + datadir = "/tmp/data" + key_file = "/tmp/file.key" + listen_host = "127.0.0.1:4003" + machines = ["coreos.com:4001", "coreos.com:4002"] + machines_file = "/tmp/machines" + max_cluster_size = 10 + max_result_buffer = 512 + max_retry_attempts = 5 + name = "test-name" + snapshot = true + verbose = true + very_verbose = true + web_url = "/web" + + [peer] + advertised_url = "127.0.0.1:7002" + ca_file = "/tmp/peer/file.ca" + cert_file = "/tmp/peer/file.cert" + key_file = "/tmp/peer/file.key" + listen_host = "127.0.0.1:7003" + ` + c := NewConfig() + _, err := toml.Decode(content, &c) + assert.Nil(t, err, "") + assert.Equal(t, c.AdvertisedUrl, "127.0.0.1:4002", "") + assert.Equal(t, c.CAFile, "/tmp/file.ca", "") + assert.Equal(t, c.CertFile, "/tmp/file.cert", "") + assert.Equal(t, c.Cors, []string{"*"}, "") + assert.Equal(t, c.DataDir, "/tmp/data", "") + assert.Equal(t, c.KeyFile, "/tmp/file.key", "") + assert.Equal(t, c.ListenHost, "127.0.0.1:4003", "") + assert.Equal(t, c.Machines, []string{"coreos.com:4001", "coreos.com:4002"}, "") + assert.Equal(t, c.MachinesFile, "/tmp/machines", "") + assert.Equal(t, c.MaxClusterSize, 10, "") + assert.Equal(t, c.MaxResultBuffer, 512, "") + assert.Equal(t, c.MaxRetryAttempts, 5, "") + assert.Equal(t, c.Name, "test-name", "") + assert.Equal(t, c.Snapshot, true, "") + assert.Equal(t, c.Verbose, true, "") + assert.Equal(t, c.VeryVerbose, true, "") + assert.Equal(t, c.WebURL, "/web", "") + assert.Equal(t, c.Peer.AdvertisedUrl, "127.0.0.1:7002", "") + assert.Equal(t, c.Peer.CAFile, "/tmp/peer/file.ca", "") + assert.Equal(t, c.Peer.CertFile, "/tmp/peer/file.cert", "") + assert.Equal(t, c.Peer.KeyFile, "/tmp/peer/file.key", "") + assert.Equal(t, c.Peer.ListenHost, "127.0.0.1:7003", "") +} + +// Ensures that a configuration can be retrieved from environment variables. +func TestConfigEnv(t *testing.T) { + os.Setenv("ETCD_CA_FILE", "/tmp/file.ca") + os.Setenv("ETCD_CERT_FILE", "/tmp/file.cert") + os.Setenv("ETCD_CPU_PROFILE_FILE", "XXX") + os.Setenv("ETCD_CORS", "localhost:4001,localhost:4002") + os.Setenv("ETCD_DATADIR", "/tmp/data") + os.Setenv("ETCD_KEY_FILE", "/tmp/file.key") + os.Setenv("ETCD_LISTEN_HOST", "127.0.0.1:4003") + os.Setenv("ETCD_MACHINES", "coreos.com:4001,coreos.com:4002") + os.Setenv("ETCD_MACHINES_FILE", "/tmp/machines") + os.Setenv("ETCD_MAX_CLUSTER_SIZE", "10") + os.Setenv("ETCD_MAX_RESULT_BUFFER", "512") + os.Setenv("ETCD_MAX_RETRY_ATTEMPTS", "5") + os.Setenv("ETCD_NAME", "test-name") + os.Setenv("ETCD_SNAPSHOT", "true") + os.Setenv("ETCD_VERBOSE", "1") + os.Setenv("ETCD_VERY_VERBOSE", "yes") + os.Setenv("ETCD_WEB_URL", "/web") + os.Setenv("ETCD_PEER_ADVERTISED_URL", "127.0.0.1:7002") + os.Setenv("ETCD_PEER_CA_FILE", "/tmp/peer/file.ca") + os.Setenv("ETCD_PEER_CERT_FILE", "/tmp/peer/file.cert") + os.Setenv("ETCD_PEER_KEY_FILE", "/tmp/peer/file.key") + os.Setenv("ETCD_PEER_LISTEN_HOST", "127.0.0.1:7003") + + c := NewConfig() + c.LoadEnv() + assert.Equal(t, c.CAFile, "/tmp/file.ca", "") + assert.Equal(t, c.CertFile, "/tmp/file.cert", "") + assert.Equal(t, c.Cors, []string{"localhost:4001", "localhost:4002"}, "") + assert.Equal(t, c.DataDir, "/tmp/data", "") + assert.Equal(t, c.KeyFile, "/tmp/file.key", "") + assert.Equal(t, c.ListenHost, "127.0.0.1:4003", "") + assert.Equal(t, c.Machines, []string{"coreos.com:4001", "coreos.com:4002"}, "") + assert.Equal(t, c.MachinesFile, "/tmp/machines", "") + assert.Equal(t, c.MaxClusterSize, 10, "") + assert.Equal(t, c.MaxResultBuffer, 512, "") + assert.Equal(t, c.MaxRetryAttempts, 5, "") + assert.Equal(t, c.Name, "test-name", "") + assert.Equal(t, c.Snapshot, true, "") + assert.Equal(t, c.Verbose, true, "") + assert.Equal(t, c.VeryVerbose, true, "") + assert.Equal(t, c.WebURL, "/web", "") + assert.Equal(t, c.Peer.AdvertisedUrl, "127.0.0.1:7002", "") + assert.Equal(t, c.Peer.CAFile, "/tmp/peer/file.ca", "") + assert.Equal(t, c.Peer.CertFile, "/tmp/peer/file.cert", "") + assert.Equal(t, c.Peer.KeyFile, "/tmp/peer/file.key", "") + assert.Equal(t, c.Peer.ListenHost, "127.0.0.1:7003", "") +} + +// Ensures that a the advertised url can be parsed from the environment. +func TestConfigAdvertisedUrlEnv(t *testing.T) { + withEnv("ETCD_ADVERTISED_URL", "127.0.0.1:4002", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.AdvertisedUrl, "127.0.0.1:4002", "") + }) +} + +// Ensures that a the advertised flag can be parsed. +func TestConfigAdvertisedUrlFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-c", "127.0.0.1:4002"}), "") + assert.Equal(t, c.AdvertisedUrl, "127.0.0.1:4002", "") +} + +// Ensures that a the CA file can be parsed from the environment. +func TestConfigCAFileEnv(t *testing.T) { + withEnv("ETCD_CA_FILE", "/tmp/file.ca", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.CAFile, "/tmp/file.ca", "") + }) +} + +// Ensures that a the CA file flag can be parsed. +func TestConfigCAFileFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-clientCAFile", "/tmp/file.ca"}), "") + assert.Equal(t, c.CAFile, "/tmp/file.ca", "") +} + +// Ensures that a the CA file can be parsed from the environment. +func TestConfigCertFileEnv(t *testing.T) { + withEnv("ETCD_CERT_FILE", "/tmp/file.cert", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.CertFile, "/tmp/file.cert", "") + }) +} + +// Ensures that a the Cert file flag can be parsed. +func TestConfigCertFileFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-clientCert", "/tmp/file.cert"}), "") + assert.Equal(t, c.CertFile, "/tmp/file.cert", "") +} + +// Ensures that a the Key file can be parsed from the environment. +func TestConfigKeyFileEnv(t *testing.T) { + withEnv("ETCD_KEY_FILE", "/tmp/file.key", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.KeyFile, "/tmp/file.key", "") + }) +} + +// Ensures that a the Key file flag can be parsed. +func TestConfigKeyFileFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-clientKey", "/tmp/file.key"}), "") + assert.Equal(t, c.KeyFile, "/tmp/file.key", "") +} + +// Ensures that a the Listen Host can be parsed from the environment. +func TestConfigListenHostEnv(t *testing.T) { + withEnv("ETCD_LISTEN_HOST", "127.0.0.1:4003", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.ListenHost, "127.0.0.1:4003", "") + }) +} + +// Ensures that a the Listen Host file flag can be parsed. +func TestConfigListenHostFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-cl", "127.0.0.1:4003"}), "") + assert.Equal(t, c.ListenHost, "127.0.0.1:4003", "") +} + +// Ensures that the Machines can be parsed from the environment. +func TestConfigMachinesEnv(t *testing.T) { + withEnv("ETCD_MACHINES", "coreos.com:4001,coreos.com:4002", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Machines, []string{"coreos.com:4001", "coreos.com:4002"}, "") + }) +} + +// Ensures that a the Machines flag can be parsed. +func TestConfigMachinesFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-C", "coreos.com:4001,coreos.com:4002"}), "") + assert.Equal(t, c.Machines, []string{"coreos.com:4001", "coreos.com:4002"}, "") +} + +// Ensures that the Machines File can be parsed from the environment. +func TestConfigMachinesFileEnv(t *testing.T) { + withEnv("ETCD_MACHINES_FILE", "/tmp/machines", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.MachinesFile, "/tmp/machines", "") + }) +} + +// Ensures that a the Machines File flag can be parsed. +func TestConfigMachinesFileFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-CF", "/tmp/machines"}), "") + assert.Equal(t, c.MachinesFile, "/tmp/machines", "") +} + +// Ensures that the Max Cluster Size can be parsed from the environment. +func TestConfigMaxClusterSizeEnv(t *testing.T) { + withEnv("ETCD_MAX_CLUSTER_SIZE", "5", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.MaxClusterSize, 5, "") + }) +} + +// Ensures that a the Max Cluster Size flag can be parsed. +func TestConfigMaxClusterSizeFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-maxsize", "5"}), "") + assert.Equal(t, c.MaxClusterSize, 5, "") +} + +// Ensures that the Max Result Buffer can be parsed from the environment. +func TestConfigMaxResultBufferEnv(t *testing.T) { + withEnv("ETCD_MAX_RESULT_BUFFER", "512", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.MaxResultBuffer, 512, "") + }) +} + +// Ensures that a the Max Result Buffer flag can be parsed. +func TestConfigMaxResultBufferFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-m", "512"}), "") + assert.Equal(t, c.MaxResultBuffer, 512, "") +} + +// Ensures that the Max Retry Attempts can be parsed from the environment. +func TestConfigMaxRetryAttemptsEnv(t *testing.T) { + withEnv("ETCD_MAX_RETRY_ATTEMPTS", "10", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.MaxRetryAttempts, 10, "") + }) +} + +// Ensures that a the Max Retry Attempts flag can be parsed. +func TestConfigMaxRetryAttemptsFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-r", "10"}), "") + assert.Equal(t, c.MaxRetryAttempts, 10, "") +} + +// Ensures that the Name can be parsed from the environment. +func TestConfigNameEnv(t *testing.T) { + withEnv("ETCD_NAME", "test-name", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Name, "test-name", "") + }) +} + +// Ensures that a the Name flag can be parsed. +func TestConfigNameFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-n", "test-name"}), "") + assert.Equal(t, c.Name, "test-name", "") +} + +// Ensures that Snapshot can be parsed from the environment. +func TestConfigSnapshotEnv(t *testing.T) { + withEnv("ETCD_SNAPSHOT", "1", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Snapshot, true, "") + }) +} + +// Ensures that a the Snapshot flag can be parsed. +func TestConfigSnapshotFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-snapshot"}), "") + assert.Equal(t, c.Snapshot, true, "") +} + +// Ensures that Verbose can be parsed from the environment. +func TestConfigVerboseEnv(t *testing.T) { + withEnv("ETCD_VERBOSE", "true", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Verbose, true, "") + }) +} + +// Ensures that a the Verbose flag can be parsed. +func TestConfigVerboseFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-v"}), "") + assert.Equal(t, c.Verbose, true, "") +} + +// Ensures that Very Verbose can be parsed from the environment. +func TestConfigVeryVerboseEnv(t *testing.T) { + withEnv("ETCD_VERY_VERBOSE", "true", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.VeryVerbose, true, "") + }) +} + +// Ensures that a the Very Verbose flag can be parsed. +func TestConfigVeryVerboseFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-vv"}), "") + assert.Equal(t, c.VeryVerbose, true, "") +} + +// Ensures that Web URL can be parsed from the environment. +func TestConfigWebURLEnv(t *testing.T) { + withEnv("ETCD_WEB_URL", "/web", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.WebURL, "/web", "") + }) +} + +// Ensures that a the Web URL flag can be parsed. +func TestConfigWebURLFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-w", "/web"}), "") + assert.Equal(t, c.WebURL, "/web", "") +} + +// Ensures that the Peer Advertised URL can be parsed from the environment. +func TestConfigPeerAdvertisedUrlEnv(t *testing.T) { + withEnv("ETCD_PEER_ADVERTISED_URL", "localhost:7002", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Peer.AdvertisedUrl, "localhost:7002", "") + }) +} + +// Ensures that a the Peer Advertised URL flag can be parsed. +func TestConfigPeerAdvertisedUrlFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-s", "localhost:7002"}), "") + assert.Equal(t, c.Peer.AdvertisedUrl, "localhost:7002", "") +} + +// Ensures that the Peer CA File can be parsed from the environment. +func TestConfigPeerCAFileEnv(t *testing.T) { + withEnv("ETCD_PEER_CA_FILE", "/tmp/peer/file.ca", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Peer.CAFile, "/tmp/peer/file.ca", "") + }) +} + +// Ensures that a the Peer CA file flag can be parsed. +func TestConfigPeerCAFileFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-serverCAFile", "/tmp/peer/file.ca"}), "") + assert.Equal(t, c.Peer.CAFile, "/tmp/peer/file.ca", "") +} + +// Ensures that the Peer Cert File can be parsed from the environment. +func TestConfigPeerCertFileEnv(t *testing.T) { + withEnv("ETCD_PEER_CERT_FILE", "/tmp/peer/file.cert", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Peer.CertFile, "/tmp/peer/file.cert", "") + }) +} + +// Ensures that a the Cert file flag can be parsed. +func TestConfigPeerCertFileFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-serverCert", "/tmp/peer/file.cert"}), "") + assert.Equal(t, c.Peer.CertFile, "/tmp/peer/file.cert", "") +} + +// Ensures that the Peer Key File can be parsed from the environment. +func TestConfigPeerKeyFileEnv(t *testing.T) { + withEnv("ETCD_PEER_KEY_FILE", "/tmp/peer/file.key", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Peer.KeyFile, "/tmp/peer/file.key", "") + }) +} + +// Ensures that a the Peer Key file flag can be parsed. +func TestConfigPeerKeyFileFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-serverKey", "/tmp/peer/file.key"}), "") + assert.Equal(t, c.Peer.KeyFile, "/tmp/peer/file.key", "") +} + +// Ensures that the Peer Listen Host can be parsed from the environment. +func TestConfigPeerListenHostEnv(t *testing.T) { + withEnv("ETCD_PEER_LISTEN_HOST", "localhost:7004", func(c *Config) { + assert.Nil(t, c.LoadEnv(), "") + assert.Equal(t, c.Peer.ListenHost, "localhost:7004", "") + }) +} + +// Ensures that a the Peer Listen Host file flag can be parsed. +func TestConfigPeerListenHostFlag(t *testing.T) { + c := NewConfig() + assert.Nil(t, c.LoadFlags([]string{"-sl", "127.0.0.1:4003"}), "") + assert.Equal(t, c.Peer.ListenHost, "127.0.0.1:4003", "") +} + + +// Ensures that a system config field is overridden by a custom config field. +func TestConfigCustomConfigOverrideSystemConfig(t *testing.T) { + system := `advertised_url = "127.0.0.1:5000"` + custom := `advertised_url = "127.0.0.1:6000"` + withTempFile(system, func(p1 string) { + withTempFile(custom, func(p2 string) { + c := NewConfig() + c.SystemPath = p1 + assert.Nil(t, c.Load([]string{"-config", p2}), "") + assert.Equal(t, c.AdvertisedUrl, "http://127.0.0.1:6000", "") + }) + }) +} + +// Ensures that a custom config field is overridden by an environment variable. +func TestConfigEnvVarOverrideCustomConfig(t *testing.T) { + os.Setenv("ETCD_PEER_ADVERTISED_URL", "127.0.0.1:8000") + defer os.Setenv("ETCD_PEER_ADVERTISED_URL", "") + + custom := `[peer]`+"\n"+`advertised_url = "127.0.0.1:9000"` + withTempFile(custom, func(path string) { + c := NewConfig() + c.SystemPath = "" + assert.Nil(t, c.Load([]string{"-config", path}), "") + assert.Equal(t, c.Peer.AdvertisedUrl, "http://127.0.0.1:8000", "") + }) +} + +// Ensures that an environment variable field is overridden by a command line argument. +func TestConfigCLIArgsOverrideEnvVar(t *testing.T) { + os.Setenv("ETCD_ADVERTISED_URL", "127.0.0.1:1000") + defer os.Setenv("ETCD_ADVERTISED_URL", "") + + c := NewConfig() + c.SystemPath = "" + assert.Nil(t, c.Load([]string{"-c", "127.0.0.1:2000"}), "") + assert.Equal(t, c.AdvertisedUrl, "http://127.0.0.1:2000", "") +} + + +//-------------------------------------- +// Helpers +//-------------------------------------- + +// Sets up the environment with a given environment variable set. +func withEnv(key, value string, f func(c *Config)) { + os.Setenv(key, value) + defer os.Setenv(key, "") + c := NewConfig() + f(c) +} + +// Creates a temp file and calls a function with the context. +func withTempFile(content string, fn func(string)) { + f, _ := ioutil.TempFile("", "") + f.WriteString(content) + f.Close() + defer os.Remove(f.Name()) + fn(f.Name()) +} diff --git a/server/info.go b/server/info.go new file mode 100644 index 00000000000..434f24a03e9 --- /dev/null +++ b/server/info.go @@ -0,0 +1,19 @@ +package server + +// Info describes the non-mutable state of the server upon initialization. +// These fields cannot be changed without deleting the server fields and +// reinitializing. +type Info struct { + Name string `json:"name"` + + RaftURL string `json:"raftURL"` + EtcdURL string `json:"etcdURL"` + WebURL string `json:"webURL"` + + RaftListenHost string `json:"raftListenHost"` + EtcdListenHost string `json:"etcdListenHost"` + + RaftTLS TLSInfo `json:"raftTLS"` + EtcdTLS TLSInfo `json:"etcdTLS"` +} + diff --git a/server/join_command.go b/server/join_command.go index 7bebbe70463..49dab4f2eaa 100644 --- a/server/join_command.go +++ b/server/join_command.go @@ -14,15 +14,17 @@ func init() { // The JoinCommand adds a node to the cluster. type JoinCommand struct { - RaftVersion string `json:"raftVersion"` + MinVersion int `json:"minVersion"` + MaxVersion int `json:"maxVersion"` Name string `json:"name"` RaftURL string `json:"raftURL"` EtcdURL string `json:"etcdURL"` } -func NewJoinCommand(version, name, raftUrl, etcdUrl string) *JoinCommand { +func NewJoinCommand(minVersion int, maxVersion int, name, raftUrl, etcdUrl string) *JoinCommand { return &JoinCommand{ - RaftVersion: version, + MinVersion: minVersion, + MaxVersion: maxVersion, Name: name, RaftURL: raftUrl, EtcdURL: etcdUrl, @@ -56,7 +58,7 @@ func (c *JoinCommand) Apply(server raft.Server) (interface{}, error) { } // Add to shared machine registry. - ps.registry.Register(c.Name, c.RaftVersion, c.RaftURL, c.EtcdURL, server.CommitIndex(), server.Term()) + ps.registry.Register(c.Name, c.RaftURL, c.EtcdURL, server.CommitIndex(), server.Term()) // Add peer in raft err := server.AddPeer(c.Name, "") diff --git a/server/peer_server.go b/server/peer_server.go index 538d3ac702e..bfa2ea7415b 100644 --- a/server/peer_server.go +++ b/server/peer_server.go @@ -10,12 +10,14 @@ import ( "net" "net/http" "net/url" + "strconv" "time" etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/log" "github.com/coreos/etcd/store" "github.com/coreos/go-raft" + "github.com/gorilla/mux" ) type PeerServer struct { @@ -51,7 +53,7 @@ type snapshotConf struct { writesThr uint64 } -func NewPeerServer(name string, path string, url string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, registry *Registry, store store.Store) *PeerServer { +func NewPeerServer(name string, path string, url string, listenHost string, tlsConf *TLSConfig, tlsInfo *TLSInfo, registry *Registry, store store.Store, snapCount int) *PeerServer { s := &PeerServer{ name: name, url: url, @@ -60,7 +62,7 @@ func NewPeerServer(name string, path string, url string, listenHost string, tlsC tlsInfo: tlsInfo, registry: registry, store: store, - snapConf: &snapshotConf{time.Second * 3, 0, 2}, + snapConf: &snapshotConf{time.Second * 3, 0, uint64(snapCount)}, followersStats: &raftFollowersStats{ Leader: name, Followers: make(map[string]*raftFollowerStats), @@ -209,7 +211,7 @@ func (s *PeerServer) SetServer(server *Server) { func (s *PeerServer) startAsLeader() { // leader need to join self as a peer for { - _, err := s.raftServer.Do(NewJoinCommand(PeerVersion, s.raftServer.Name(), s.url, s.server.URL())) + _, err := s.raftServer.Do(NewJoinCommand(store.MinVersion(), store.MaxVersion(), s.raftServer.Name(), s.url, s.server.URL())) if err == nil { break } @@ -235,25 +237,27 @@ func (s *PeerServer) startAsFollower(cluster []string) { func (s *PeerServer) startTransport(scheme string, tlsConf tls.Config) error { log.Infof("raft server [name %s, listen on %s, advertised url %s]", s.name, s.listenHost, s.url) - raftMux := http.NewServeMux() + router := mux.NewRouter() s.httpServer = &http.Server{ - Handler: raftMux, + Handler: router, TLSConfig: &tlsConf, Addr: s.listenHost, } // internal commands - raftMux.HandleFunc("/name", s.NameHttpHandler) - raftMux.HandleFunc("/version", s.RaftVersionHttpHandler) - raftMux.HandleFunc("/join", s.JoinHttpHandler) - raftMux.HandleFunc("/remove/", s.RemoveHttpHandler) - raftMux.HandleFunc("/vote", s.VoteHttpHandler) - raftMux.HandleFunc("/log", s.GetLogHttpHandler) - raftMux.HandleFunc("/log/append", s.AppendEntriesHttpHandler) - raftMux.HandleFunc("/snapshot", s.SnapshotHttpHandler) - raftMux.HandleFunc("/snapshotRecovery", s.SnapshotRecoveryHttpHandler) - raftMux.HandleFunc("/etcdURL", s.EtcdURLHttpHandler) + router.HandleFunc("/name", s.NameHttpHandler) + router.HandleFunc("/version", s.VersionHttpHandler) + router.HandleFunc("/version/{version:[0-9]+}/check", s.VersionCheckHttpHandler) + router.HandleFunc("/upgrade", s.UpgradeHttpHandler) + router.HandleFunc("/join", s.JoinHttpHandler) + router.HandleFunc("/remove/{name:.+}", s.RemoveHttpHandler) + router.HandleFunc("/vote", s.VoteHttpHandler) + router.HandleFunc("/log", s.GetLogHttpHandler) + router.HandleFunc("/log/append", s.AppendEntriesHttpHandler) + router.HandleFunc("/snapshot", s.SnapshotHttpHandler) + router.HandleFunc("/snapshotRecovery", s.SnapshotRecoveryHttpHandler) + router.HandleFunc("/etcdURL", s.EtcdURLHttpHandler) if scheme == "http" { return s.listenAndServe() @@ -263,21 +267,46 @@ func (s *PeerServer) startTransport(scheme string, tlsConf tls.Config) error { } -// getVersion fetches the raft version of a peer. This works for now but we -// will need to do something more sophisticated later when we allow mixed -// version clusters. -func getVersion(t *transporter, versionURL url.URL) (string, error) { +// getVersion fetches the peer version of a cluster. +func getVersion(t *transporter, versionURL url.URL) (int, error) { resp, req, err := t.Get(versionURL.String()) if err != nil { - return "", err + return 0, err } defer resp.Body.Close() t.CancelWhenTimeout(req) - body, err := ioutil.ReadAll(resp.Body) + if err != nil { + return 0, err + } - return string(body), nil + // Parse version number. + version, _ := strconv.Atoi(string(body)) + return version, nil +} + +// Upgradable checks whether all peers in a cluster support an upgrade to the next store version. +func (s *PeerServer) Upgradable() error { + nextVersion := s.store.Version() + 1 + for _, peerURL := range s.registry.PeerURLs(s.raftServer.Leader(), s.name) { + u, err := url.Parse(peerURL) + if err != nil { + return fmt.Errorf("PeerServer: Cannot parse URL: '%s' (%s)", peerURL, err) + } + + t, _ := s.raftServer.Transporter().(*transporter) + checkURL := (&url.URL{Host: u.Host, Scheme: s.tlsConf.Scheme, Path: fmt.Sprintf("/version/%d/check", nextVersion)}).String() + resp, _, err := t.Get(checkURL) + if err != nil { + return fmt.Errorf("PeerServer: Cannot check version compatibility: %s", u.Host) + } + if resp.StatusCode != 200 { + return fmt.Errorf("PeerServer: Version %d is not compatible with peer: %s", nextVersion, u.Host) + } + } + + return nil } func (s *PeerServer) joinCluster(cluster []string) bool { @@ -315,14 +344,11 @@ func (s *PeerServer) joinByMachine(server raft.Server, machine string, scheme st if err != nil { return fmt.Errorf("Error during join version check: %v", err) } - - // TODO: versioning of the internal protocol. See: - // Documentation/internatl-protocol-versioning.md - if version != PeerVersion { - return fmt.Errorf("Unable to join: internal version mismatch, entire cluster must be running identical versions of etcd") + if version < store.MinVersion() || version > store.MaxVersion() { + return fmt.Errorf("Unable to join: cluster version is %d; version compatibility is %d - %d", version, store.MinVersion(), store.MaxVersion()) } - json.NewEncoder(&b).Encode(NewJoinCommand(PeerVersion, server.Name(), s.url, s.server.URL())) + json.NewEncoder(&b).Encode(NewJoinCommand(store.MinVersion(), store.MaxVersion(), server.Name(), s.url, s.server.URL())) joinURL := url.URL{Host: machine, Scheme: scheme, Path: "/join"} @@ -347,7 +373,7 @@ func (s *PeerServer) joinByMachine(server raft.Server, machine string, scheme st if resp.StatusCode == http.StatusTemporaryRedirect { address := resp.Header.Get("Location") log.Debugf("Send Join Request to %s", address) - json.NewEncoder(&b).Encode(NewJoinCommand(PeerVersion, server.Name(), s.url, s.server.URL())) + json.NewEncoder(&b).Encode(NewJoinCommand(store.MinVersion(), store.MaxVersion(), server.Name(), s.url, s.server.URL())) resp, req, err = t.Post(address, &b) } else if resp.StatusCode == http.StatusBadRequest { diff --git a/server/peer_server_handlers.go b/server/peer_server_handlers.go index 26ce7a5cd1a..be665dbf530 100644 --- a/server/peer_server_handlers.go +++ b/server/peer_server_handlers.go @@ -3,10 +3,13 @@ package server import ( "encoding/json" "net/http" + "strconv" etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" "github.com/coreos/go-raft" + "github.com/gorilla/mux" ) // Get all the current logs @@ -133,9 +136,9 @@ func (ps *PeerServer) RemoveHttpHandler(w http.ResponseWriter, req *http.Request return } - nodeName := req.URL.Path[len("/remove/"):] + vars := mux.Vars(req) command := &RemoveCommand{ - Name: nodeName, + Name: vars["name"], } log.Debugf("[recv] Remove Request [%s]", command.Name) @@ -151,8 +154,40 @@ func (ps *PeerServer) NameHttpHandler(w http.ResponseWriter, req *http.Request) } // Response to the name request -func (ps *PeerServer) RaftVersionHttpHandler(w http.ResponseWriter, req *http.Request) { +func (ps *PeerServer) VersionHttpHandler(w http.ResponseWriter, req *http.Request) { log.Debugf("[recv] Get %s/version/ ", ps.url) w.WriteHeader(http.StatusOK) - w.Write([]byte(PeerVersion)) + w.Write([]byte(strconv.Itoa(ps.store.Version()))) +} + +// Checks whether a given version is supported. +func (ps *PeerServer) VersionCheckHttpHandler(w http.ResponseWriter, req *http.Request) { + log.Debugf("[recv] Get %s%s ", ps.url, req.URL.Path) + vars := mux.Vars(req) + version, _ := strconv.Atoi(vars["version"]) + if version >= store.MinVersion() && version <= store.MaxVersion() { + w.WriteHeader(http.StatusOK) + } else { + w.WriteHeader(http.StatusForbidden) + } +} + +// Upgrades the current store version to the next version. +func (ps *PeerServer) UpgradeHttpHandler(w http.ResponseWriter, req *http.Request) { + log.Debugf("[recv] Get %s/version", ps.url) + + // Check if upgrade is possible for all nodes. + if err := ps.Upgradable(); err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + + // Create an upgrade command from the current version. + c := ps.store.CommandFactory().CreateUpgradeCommand() + if err := ps.server.Dispatch(c, w, req); err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + + w.WriteHeader(http.StatusOK) } diff --git a/server/registry.go b/server/registry.go index 75b0e98c969..05cccc10c11 100644 --- a/server/registry.go +++ b/server/registry.go @@ -38,13 +38,13 @@ func NewRegistry(s store.Store) *Registry { } // Adds a node to the registry. -func (r *Registry) Register(name string, peerVersion string, peerURL string, url string, commitIndex uint64, term uint64) error { +func (r *Registry) Register(name string, peerURL string, url string, commitIndex uint64, term uint64) error { r.Lock() defer r.Unlock() // Write data to store. key := path.Join(RegistryKey, name) - value := fmt.Sprintf("raft=%s&etcd=%s&raftVersion=%s", peerURL, url, peerVersion) + value := fmt.Sprintf("raft=%s&etcd=%s", peerURL, url) _, err := r.store.Create(key, value, false, store.Permanent, commitIndex, term) log.Debugf("Register: %s", name) return err @@ -175,6 +175,5 @@ func (r *Registry) load(name string) { r.nodes[name] = &node{ url: m["etcd"][0], peerURL: m["raft"][0], - peerVersion: m["raftVersion"][0], } } diff --git a/server/server.go b/server/server.go index f6a8779a2ac..b88d5ac0ab3 100644 --- a/server/server.go +++ b/server/server.go @@ -15,6 +15,7 @@ import ( "github.com/coreos/etcd/server/v1" "github.com/coreos/etcd/server/v2" "github.com/coreos/etcd/store" + _ "github.com/coreos/etcd/store/v2" "github.com/coreos/go-raft" "github.com/gorilla/mux" ) @@ -283,10 +284,10 @@ func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Reque } // Sets a comma-delimited list of origins that are allowed. -func (s *Server) AllowOrigins(origins string) error { +func (s *Server) AllowOrigins(origins []string) error { // Construct a lookup of all origins. m := make(map[string]bool) - for _, v := range strings.Split(origins, ",") { + for _, v := range origins { if v != "*" { if _, err := url.Parse(v); err != nil { return fmt.Errorf("Invalid CORS origin: %s", err) @@ -366,11 +367,7 @@ func (s *Server) SpeedTestHandler(w http.ResponseWriter, req *http.Request) erro for i := 0; i < count; i++ { go func() { for j := 0; j < 10; j++ { - c := &store.SetCommand{ - Key: "foo", - Value: "bar", - ExpireTime: time.Unix(0, 0), - } + c := s.Store().CommandFactory().CreateSetCommand("foo", "bar", time.Unix(0, 0)) s.peerServer.RaftServer().Do(c) } c <- true diff --git a/server/tls_config.go b/server/tls_config.go index 4b944626d16..29b8fa5e4ff 100644 --- a/server/tls_config.go +++ b/server/tls_config.go @@ -4,6 +4,7 @@ import ( "crypto/tls" ) +// TLSConfig holds the TLS configuration. type TLSConfig struct { Scheme string Server tls.Config diff --git a/server/tls_info.go b/server/tls_info.go index 91936b0906e..1867ef44f09 100644 --- a/server/tls_info.go +++ b/server/tls_info.go @@ -1,7 +1,76 @@ package server +import ( + "crypto/tls" + "crypto/x509" + "encoding/pem" + "errors" + "io/ioutil" +) + +// TLSInfo holds the SSL certificates paths. type TLSInfo struct { CertFile string `json:"CertFile"` KeyFile string `json:"KeyFile"` CAFile string `json:"CAFile"` } + +// Generates a TLS configuration from the given files. +func (info TLSInfo) Config() (TLSConfig, error) { + var t TLSConfig + t.Scheme = "http" + + // If the user do not specify key file, cert file and CA file, the type will be HTTP + if info.KeyFile == "" && info.CertFile == "" && info.CAFile == "" { + return t, nil + } + + // Both the key and cert must be present. + if info.KeyFile == "" || info.CertFile == "" { + return t, errors.New("KeyFile and CertFile must both be present") + } + + tlsCert, err := tls.LoadX509KeyPair(info.CertFile, info.KeyFile) + if err != nil { + return t, err + } + + t.Scheme = "https" + t.Server.ClientAuth, t.Server.ClientCAs, err = newCertPool(info.CAFile) + if err != nil { + return t, err + } + + // The client should trust the RootCA that the Server uses since + // everyone is a peer in the network. + t.Client.Certificates = []tls.Certificate{tlsCert} + t.Client.RootCAs = t.Server.ClientCAs + + return t, nil +} + +// newCertPool creates x509 certPool and corresponding Auth Type. +// If the given CAfile is valid, add the cert into the pool and verify the clients' +// certs against the cert in the pool. +// If the given CAfile is empty, do not verify the clients' cert. +// If the given CAfile is not valid, fatal. +func newCertPool(CAFile string) (tls.ClientAuthType, *x509.CertPool, error) { + if CAFile == "" { + return tls.NoClientCert, nil, nil + } + pemByte, err := ioutil.ReadFile(CAFile) + if err != nil { + return 0, nil, err + } + + block, pemByte := pem.Decode(pemByte) + cert, err := x509.ParseCertificate(block.Bytes) + if err != nil { + return 0, nil, err + } + + certPool := x509.NewCertPool() + certPool.AddCert(cert) + + return tls.RequireAndVerifyClientCert, certPool, nil +} diff --git a/server/transporter.go b/server/transporter.go index 3c5002f1ba2..95545a82752 100644 --- a/server/transporter.go +++ b/server/transporter.go @@ -18,13 +18,15 @@ import ( // This should not exceed 3 * RTT var dailTimeout = 3 * HeartbeatTimeout -// Timeout for setup internal raft http connection + receive response header -// This should not exceed 3 * RTT + RTT -var responseHeaderTimeout = 4 * HeartbeatTimeout +// Timeout for setup internal raft http connection + receive all post body +// The raft server will not send back response header until it received all the +// post body. +// This should not exceed dailTimeout + electionTimeout +var responseHeaderTimeout = 3*HeartbeatTimeout + ElectionTimeout // Timeout for receiving the response body from the server -// This should not exceed election timeout -var tranTimeout = ElectionTimeout +// This should not exceed heartbeatTimeout +var tranTimeout = HeartbeatTimeout // Transporter layer for communication between raft nodes type transporter struct { @@ -221,7 +223,7 @@ func (t *transporter) Get(urlStr string) (*http.Response, *http.Request, error) // Cancel the on fly HTTP transaction when timeout happens. func (t *transporter) CancelWhenTimeout(req *http.Request) { go func() { - time.Sleep(ElectionTimeout) + time.Sleep(tranTimeout) t.transport.CancelRequest(req) }() } diff --git a/server/transporter_test.go b/server/transporter_test.go index e83ea3400c4..989b2e25020 100644 --- a/server/transporter_test.go +++ b/server/transporter_test.go @@ -1,5 +1,6 @@ package server +/* import ( "crypto/tls" "fmt" @@ -59,3 +60,4 @@ func TestTransporterTimeout(t *testing.T) { } } +*/ diff --git a/server/util.go b/server/util.go index 578c048cc8d..bae2eaa722a 100644 --- a/server/util.go +++ b/server/util.go @@ -6,6 +6,7 @@ import ( "io" "net/http" "net/url" + "strings" "github.com/coreos/etcd/log" ) @@ -31,3 +32,15 @@ func redirect(hostname string, w http.ResponseWriter, req *http.Request) { log.Debugf("Redirect to %s", redirectURL.String()) http.Redirect(w, req, redirectURL.String(), http.StatusTemporaryRedirect) } + +// trimsplit slices s into all substrings separated by sep and returns a +// slice of the substrings between the separator with all leading and trailing +// white space removed, as defined by Unicode. +func trimsplit(s, sep string) []string { + raw := strings.Split(s, ",") + trimmed := make([]string, 0) + for _, r := range raw { + trimmed = append(trimmed, strings.TrimSpace(r)) + } + return trimmed +} diff --git a/server/v1/delete_key_handler.go b/server/v1/delete_key_handler.go index 1288d6597e1..44fe459e494 100644 --- a/server/v1/delete_key_handler.go +++ b/server/v1/delete_key_handler.go @@ -1,7 +1,6 @@ package v1 import ( - "github.com/coreos/etcd/store" "github.com/gorilla/mux" "net/http" ) @@ -10,6 +9,6 @@ import ( func DeleteKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) key := "/" + vars["key"] - c := &store.DeleteCommand{Key: key} + c := s.Store().CommandFactory().CreateDeleteCommand(key, false) return s.Dispatch(c, w, req) } diff --git a/server/v1/set_key_handler.go b/server/v1/set_key_handler.go index 5e29fafe6bb..7acfe7ecb21 100644 --- a/server/v1/set_key_handler.go +++ b/server/v1/set_key_handler.go @@ -31,27 +31,16 @@ func SetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { // If the "prevValue" is specified then test-and-set. Otherwise create a new key. var c raft.Command if prevValueArr, ok := req.Form["prevValue"]; ok { - if len(prevValueArr[0]) > 0 { // test against previous value - c = &store.CompareAndSwapCommand{ - Key: key, - Value: value, - PrevValue: prevValueArr[0], - ExpireTime: expireTime, - } + if len(prevValueArr[0]) > 0 { + // test against previous value + c = s.Store().CommandFactory().CreateCompareAndSwapCommand(key, value, prevValueArr[0], 0, expireTime) } else { - c = &store.CreateCommand{ // test against existence - Key: key, - Value: value, - ExpireTime: expireTime, - } + // test against existence + c = s.Store().CommandFactory().CreateCreateCommand(key, value, expireTime, false) } } else { - c = &store.SetCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - } + c = s.Store().CommandFactory().CreateSetCommand(key, value, expireTime) } return s.Dispatch(c, w, req) diff --git a/server/v2/delete_handler.go b/server/v2/delete_handler.go index 9012498cf2c..7afc02f0412 100644 --- a/server/v2/delete_handler.go +++ b/server/v2/delete_handler.go @@ -3,18 +3,14 @@ package v2 import ( "net/http" - "github.com/coreos/etcd/store" "github.com/gorilla/mux" ) func DeleteHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) key := "/" + vars["key"] + recursive := (req.FormValue("recursive") == "true") - c := &store.DeleteCommand{ - Key: key, - Recursive: (req.FormValue("recursive") == "true"), - } - + c := s.Store().CommandFactory().CreateDeleteCommand(key, recursive) return s.Dispatch(c, w, req) } diff --git a/server/v2/post_handler.go b/server/v2/post_handler.go index b55eddba4bf..4dc98b92510 100644 --- a/server/v2/post_handler.go +++ b/server/v2/post_handler.go @@ -18,12 +18,6 @@ func PostHandler(w http.ResponseWriter, req *http.Request, s Server) error { return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create", store.UndefIndex, store.UndefTerm) } - c := &store.CreateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - Unique: true, - } - + c := s.Store().CommandFactory().CreateCreateCommand(key, value, expireTime, true) return s.Dispatch(c, w, req) } diff --git a/server/v2/put_handler.go b/server/v2/put_handler.go index 536bbe21698..3afb018b726 100644 --- a/server/v2/put_handler.go +++ b/server/v2/put_handler.go @@ -71,31 +71,17 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { } } - c = &store.CompareAndSwapCommand{ - Key: key, - Value: value, - PrevValue: prevValue, - PrevIndex: prevIndex, - } - + c = s.Store().CommandFactory().CreateCompareAndSwapCommand(key, value, prevValue, prevIndex, expireTime) return s.Dispatch(c, w, req) } func SetHandler(w http.ResponseWriter, req *http.Request, s Server, key, value string, expireTime time.Time) error { - c := &store.SetCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - } + c := s.Store().CommandFactory().CreateSetCommand(key, value, expireTime) return s.Dispatch(c, w, req) } func CreateHandler(w http.ResponseWriter, req *http.Request, s Server, key, value string, expireTime time.Time) error { - c := &store.CreateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - } + c := s.Store().CommandFactory().CreateCreateCommand(key, value, expireTime, false) return s.Dispatch(c, w, req) } @@ -105,10 +91,6 @@ func UpdateHandler(w http.ResponseWriter, req *http.Request, s Server, key, valu return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", store.UndefIndex, store.UndefTerm) } - c := &store.UpdateCommand{ - Key: key, - Value: value, - ExpireTime: expireTime, - } + c := s.Store().CommandFactory().CreateUpdateCommand(key, value, expireTime) return s.Dispatch(c, w, req) } diff --git a/server/v2/tests/delete_handler_test.go b/server/v2/tests/delete_handler_test.go new file mode 100644 index 00000000000..ab710c5ac32 --- /dev/null +++ b/server/v2/tests/delete_handler_test.go @@ -0,0 +1,29 @@ +package v2 + +import ( + "fmt" + "net/url" + "testing" + + "github.com/coreos/etcd/server" + "github.com/coreos/etcd/tests" + "github.com/stretchr/testify/assert" +) + +// Ensures that a key is deleted. +// +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX +// $ curl -X DELETE localhost:4001/v2/keys/foo/bar +// +func TestV2DeleteKey(t *testing.T) { + tests.RunServer(func(s *server.Server) { + v := url.Values{} + v.Set("value", "XXX") + resp, err := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) + tests.ReadBody(resp) + resp, err = tests.DeleteForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), url.Values{}) + body := tests.ReadBody(resp) + assert.Nil(t, err, "") + assert.Equal(t, string(body), `{"action":"delete","key":"/foo/bar","prevValue":"XXX","index":4,"term":0}`, "") + }) +} diff --git a/server/v2/tests/get_handler_test.go b/server/v2/tests/get_handler_test.go index a248d33f423..d6ceae58a0f 100644 --- a/server/v2/tests/get_handler_test.go +++ b/server/v2/tests/get_handler_test.go @@ -42,6 +42,7 @@ func TestV2GetKeyRecursively(t *testing.T) { tests.RunServer(func(s *server.Server) { v := url.Values{} v.Set("value", "XXX") + v.Set("ttl", "10") resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/x"), v) tests.ReadBody(resp) @@ -60,6 +61,7 @@ func TestV2GetKeyRecursively(t *testing.T) { kv0 := body["kvs"].([]interface{})[0].(map[string]interface{}) assert.Equal(t, kv0["key"], "/foo/x", "") assert.Equal(t, kv0["value"], "XXX", "") + assert.Equal(t, kv0["ttl"], 10, "") kv1 := body["kvs"].([]interface{})[1].(map[string]interface{}) assert.Equal(t, kv1["key"], "/foo/y", "") @@ -105,7 +107,6 @@ func TestV2WatchKey(t *testing.T) { }) } - // Ensures that a watcher can wait for a value to be set after a given index. // // $ curl localhost:4001/v2/keys/foo/bar?wait=true&waitIndex=4 @@ -115,9 +116,11 @@ func TestV2WatchKey(t *testing.T) { func TestV2WatchKeyWithIndex(t *testing.T) { tests.RunServer(func(s *server.Server) { var body map[string]interface{} + c := make(chan bool) go func() { resp, _ := tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar?wait=true&waitIndex=5")) body = tests.ReadBodyJSON(resp) + c <- true }() // Make sure response didn't fire early. @@ -141,6 +144,14 @@ func TestV2WatchKeyWithIndex(t *testing.T) { // A response should follow from the GET above. time.Sleep(1 * time.Millisecond) + + select { + case <-c: + + default: + t.Fatal("cannot get watch result") + } + assert.NotNil(t, body, "") assert.Equal(t, body["action"], "set", "") assert.Equal(t, body["key"], "/foo/bar", "") @@ -149,4 +160,3 @@ func TestV2WatchKeyWithIndex(t *testing.T) { assert.Equal(t, body["term"], 0, "") }) } - diff --git a/server/version.go b/server/version.go index 354bda3a5bb..1d0fc6a4b4e 100644 --- a/server/version.go +++ b/server/version.go @@ -1,8 +1,3 @@ package server const Version = "v2" - -// TODO: The release version (generated from the git tag) will be the raft -// protocol version for now. When things settle down we will fix it like the -// client API above. -const PeerVersion = ReleaseVersion diff --git a/store/command_factory.go b/store/command_factory.go new file mode 100644 index 00000000000..9b52f42f9cd --- /dev/null +++ b/store/command_factory.go @@ -0,0 +1,58 @@ +package store + +import ( + "fmt" + "time" + + "github.com/coreos/go-raft" +) + +// A lookup of factories by version. +var factories = make(map[int]CommandFactory) +var minVersion, maxVersion int + +// The CommandFactory provides a way to create different types of commands +// depending on the current version of the store. +type CommandFactory interface { + Version() int + CreateUpgradeCommand() raft.Command + CreateSetCommand(key string, value string, expireTime time.Time) raft.Command + CreateCreateCommand(key string, value string, expireTime time.Time, unique bool) raft.Command + CreateUpdateCommand(key string, value string, expireTime time.Time) raft.Command + CreateDeleteCommand(key string, recursive bool) raft.Command + CreateCompareAndSwapCommand(key string, value string, prevValue string, prevIndex uint64, expireTime time.Time) raft.Command +} + +// RegisterCommandFactory adds a command factory to the global registry. +func RegisterCommandFactory(factory CommandFactory) { + version := factory.Version() + + if GetCommandFactory(version) != nil { + panic(fmt.Sprintf("Command factory already registered for version: %d", factory.Version())) + } + + factories[version] = factory + + // Update compatibility versions. + if minVersion == 0 || version > minVersion { + minVersion = version + } + if maxVersion == 0 || version > maxVersion { + maxVersion = version + } +} + +// GetCommandFactory retrieves a command factory for a given command version. +func GetCommandFactory(version int) CommandFactory { + return factories[version] +} + +// MinVersion returns the minimum compatible store version. +func MinVersion() int { + return minVersion +} + +// MaxVersion returns the maximum compatible store version. +func MaxVersion() int { + return maxVersion +} diff --git a/store/node.go b/store/node.go index 2dfccc1ed37..0905d4d1be2 100644 --- a/store/node.go +++ b/store/node.go @@ -375,8 +375,8 @@ func (n *Node) UpdateTTL(expireTime time.Time) { } } + n.ExpireTime = expireTime if expireTime.Sub(Permanent) != 0 { - n.ExpireTime = expireTime n.Expire() } } diff --git a/store/store.go b/store/store.go index 7fa21a6389f..f752f6935b4 100644 --- a/store/store.go +++ b/store/store.go @@ -13,7 +13,12 @@ import ( etcdErr "github.com/coreos/etcd/error" ) +// The default version to set when the store is first initialized. +const defaultVersion = 2 + type Store interface { + Version() int + CommandFactory() CommandFactory Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) Set(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) @@ -30,12 +35,13 @@ type Store interface { } type store struct { - Root *Node - WatcherHub *watcherHub - Index uint64 - Term uint64 - Stats *Stats - worldLock sync.RWMutex // stop the world lock + Root *Node + WatcherHub *watcherHub + Index uint64 + Term uint64 + Stats *Stats + CurrentVersion int + worldLock sync.RWMutex // stop the world lock } func New() Store { @@ -44,13 +50,23 @@ func New() Store { func newStore() *store { s := new(store) + s.CurrentVersion = defaultVersion s.Root = newDir(s, "/", UndefIndex, UndefTerm, nil, "", Permanent) s.Stats = newStats() s.WatcherHub = newWatchHub(1000) - return s } +// Version retrieves current version of the store. +func (s *store) Version() int { + return s.CurrentVersion +} + +// CommandFactory retrieves the command factory for the current version of the store. +func (s *store) CommandFactory() CommandFactory { + return GetCommandFactory(s.Version()) +} + // Get function returns a get event. // If recursive is true, it will return all the content under the node path. // If sorted is true, it will sort the content by keys. @@ -450,6 +466,7 @@ func (s *store) Save() ([]byte, error) { clonedStore.Root = s.Root.Clone() clonedStore.WatcherHub = s.WatcherHub.clone() clonedStore.Stats = s.Stats.clone() + clonedStore.CurrentVersion = s.CurrentVersion s.worldLock.Unlock() diff --git a/store/v2/command_factory.go b/store/v2/command_factory.go new file mode 100644 index 00000000000..4f0e7260c81 --- /dev/null +++ b/store/v2/command_factory.go @@ -0,0 +1,73 @@ +package v2 + +import ( + "time" + + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" +) + +func init() { + store.RegisterCommandFactory(&CommandFactory{}) +} + +// CommandFactory provides a pluggable way to create version 2 commands. +type CommandFactory struct { +} + +// Version returns the version of this factory. +func (f *CommandFactory) Version() int { + return 2 +} + +// CreateUpgradeCommand is a no-op since version 2 is the first version to support store versioning. +func (f *CommandFactory) CreateUpgradeCommand() raft.Command { + return &raft.NOPCommand{} +} + +// CreateSetCommand creates a version 2 command to set a key to a given value in the store. +func (f *CommandFactory) CreateSetCommand(key string, value string, expireTime time.Time) raft.Command { + return &SetCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + } +} + +// CreateCreateCommand creates a version 2 command to create a new key in the store. +func (f *CommandFactory) CreateCreateCommand(key string, value string, expireTime time.Time, unique bool) raft.Command { + return &CreateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + Unique: unique, + } +} + +// CreateUpdateCommand creates a version 2 command to update a key to a given value in the store. +func (f *CommandFactory) CreateUpdateCommand(key string, value string, expireTime time.Time) raft.Command { + return &UpdateCommand{ + Key: key, + Value: value, + ExpireTime: expireTime, + } +} + +// CreateDeleteCommand creates a version 2 command to delete a key from the store. +func (f *CommandFactory) CreateDeleteCommand(key string, recursive bool) raft.Command { + return &DeleteCommand{ + Key: key, + Recursive: recursive, + } +} + +// CreateCompareAndSwapCommand creates a version 2 command to conditionally set a key in the store. +func (f *CommandFactory) CreateCompareAndSwapCommand(key string, value string, prevValue string, prevIndex uint64, expireTime time.Time) raft.Command { + return &CompareAndSwapCommand{ + Key: key, + Value: value, + PrevValue: prevValue, + PrevIndex: prevIndex, + ExpireTime: expireTime, + } +} diff --git a/store/compare_and_swap_command.go b/store/v2/compare_and_swap_command.go similarity index 91% rename from store/compare_and_swap_command.go rename to store/v2/compare_and_swap_command.go index 8ee119f36fe..1de79fb1ff3 100644 --- a/store/compare_and_swap_command.go +++ b/store/v2/compare_and_swap_command.go @@ -1,9 +1,10 @@ -package store +package v2 import ( "time" "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" "github.com/coreos/go-raft" ) @@ -27,7 +28,7 @@ func (c *CompareAndSwapCommand) CommandName() string { // Set the key-value pair if the current value of the key equals to the given prevValue func (c *CompareAndSwapCommand) Apply(server raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(Store) + s, _ := server.StateMachine().(store.Store) e, err := s.CompareAndSwap(c.Key, c.PrevValue, c.PrevIndex, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) diff --git a/store/create_command.go b/store/v2/create_command.go similarity index 88% rename from store/create_command.go rename to store/v2/create_command.go index 6a2487cf0ee..e187d99f7f1 100644 --- a/store/create_command.go +++ b/store/v2/create_command.go @@ -1,9 +1,11 @@ -package store +package v2 import ( + "time" + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" "github.com/coreos/go-raft" - "time" ) func init() { @@ -25,7 +27,7 @@ func (c *CreateCommand) CommandName() string { // Create node func (c *CreateCommand) Apply(server raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(Store) + s, _ := server.StateMachine().(store.Store) e, err := s.Create(c.Key, c.Value, c.Unique, c.ExpireTime, server.CommitIndex(), server.Term()) diff --git a/store/delete_command.go b/store/v2/delete_command.go similarity index 87% rename from store/delete_command.go rename to store/v2/delete_command.go index 6ff3c5c6fab..6bd48368f44 100644 --- a/store/delete_command.go +++ b/store/v2/delete_command.go @@ -1,6 +1,7 @@ -package store +package v2 import ( + "github.com/coreos/etcd/store" "github.com/coreos/etcd/log" "github.com/coreos/go-raft" ) @@ -22,7 +23,7 @@ func (c *DeleteCommand) CommandName() string { // Delete the key func (c *DeleteCommand) Apply(server raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(Store) + s, _ := server.StateMachine().(store.Store) e, err := s.Delete(c.Key, c.Recursive, server.CommitIndex(), server.Term()) diff --git a/store/set_command.go b/store/v2/set_command.go similarity index 88% rename from store/set_command.go rename to store/v2/set_command.go index 55635cd9904..4f6ecf59f1f 100644 --- a/store/set_command.go +++ b/store/v2/set_command.go @@ -1,9 +1,11 @@ -package store +package v2 import ( + "time" + "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" "github.com/coreos/go-raft" - "time" ) func init() { @@ -24,7 +26,7 @@ func (c *SetCommand) CommandName() string { // Create node func (c *SetCommand) Apply(server raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(Store) + s, _ := server.StateMachine().(store.Store) // create a new node or replace the old node. e, err := s.Set(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) diff --git a/store/update_command.go b/store/v2/update_command.go similarity index 88% rename from store/update_command.go rename to store/v2/update_command.go index 8e353cdd068..d080ecced44 100644 --- a/store/update_command.go +++ b/store/v2/update_command.go @@ -1,7 +1,8 @@ -package store +package v2 import ( "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" "github.com/coreos/go-raft" "time" ) @@ -24,7 +25,7 @@ func (c *UpdateCommand) CommandName() string { // Create node func (c *UpdateCommand) Apply(server raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(Store) + s, _ := server.StateMachine().(store.Store) e, err := s.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) diff --git a/test.sh b/test.sh index 66f745ed49f..5cc633975f4 100755 --- a/test.sh +++ b/test.sh @@ -8,6 +8,9 @@ set -e export GOPATH="${PWD}" # Unit tests +go test -i ./server +go test -v ./server + go test -i ./server/v2/tests go test -v ./server/v2/tests diff --git a/tests/functional/version_check_test.go b/tests/functional/version_check_test.go new file mode 100644 index 00000000000..98a15539049 --- /dev/null +++ b/tests/functional/version_check_test.go @@ -0,0 +1,46 @@ +package test + +import ( + "net/http" + "os" + "testing" + "time" +) + +// Ensure that a node can reply to a version check appropriately. +func TestVersionCheck(t *testing.T) { + procAttr := new(os.ProcAttr) + procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} + args := []string{"etcd", "-n=node1", "-f", "-d=/tmp/version_check"} + + process, err := os.StartProcess(EtcdBinPath, args, procAttr) + if err != nil { + t.Fatal("start process failed:" + err.Error()) + return + } + defer process.Kill() + + time.Sleep(time.Second) + + // Check a version too small. + resp, _ := http.Get("http://localhost:7001/version/1/check") + resp.Body.Close() + if resp.StatusCode != http.StatusForbidden { + t.Fatal("Invalid version check: ", resp.StatusCode) + } + + // Check a version too large. + resp, _ = http.Get("http://localhost:7001/version/3/check") + resp.Body.Close() + if resp.StatusCode != http.StatusForbidden { + t.Fatal("Invalid version check: ", resp.StatusCode) + } + + // Check a version that's just right. + resp, _ = http.Get("http://localhost:7001/version/2/check") + resp.Body.Close() + if resp.StatusCode != http.StatusOK { + t.Fatal("Invalid version check: ", resp.StatusCode) + } +} + diff --git a/tests/http_utils.go b/tests/http_utils.go index b1395373d03..ebe6f527304 100644 --- a/tests/http_utils.go +++ b/tests/http_utils.go @@ -55,6 +55,14 @@ func PutForm(url string, data url.Values) (*http.Response, error) { return Put(url, "application/x-www-form-urlencoded", strings.NewReader(data.Encode())) } +func Delete(url string, bodyType string, body io.Reader) (*http.Response, error) { + return send("DELETE", url, bodyType, body) +} + +func DeleteForm(url string, data url.Values) (*http.Response, error) { + return Delete(url, "application/x-www-form-urlencoded", strings.NewReader(data.Encode())) +} + func send(method string, url string, bodyType string, body io.Reader) (*http.Response, error) { c := NewHTTPClient() diff --git a/tests/server_utils.go b/tests/server_utils.go index adddf336bcf..e7ed347a3b9 100644 --- a/tests/server_utils.go +++ b/tests/server_utils.go @@ -5,14 +5,15 @@ import ( "os" "time" - "github.com/coreos/etcd/store" "github.com/coreos/etcd/server" + "github.com/coreos/etcd/store" ) const ( - testName = "ETCDTEST" + testName = "ETCDTEST" testClientURL = "localhost:4401" - testRaftURL = "localhost:7701" + testRaftURL = "localhost:7701" + testSnapCount = 10000 ) // Starts a server in a temporary directory. @@ -22,8 +23,8 @@ func RunServer(f func(*server.Server)) { store := store.New() registry := server.NewRegistry(store) - ps := server.NewPeerServer(testName, path, testRaftURL, testRaftURL, &server.TLSConfig{Scheme:"http"}, &server.TLSInfo{}, registry, store) - s := server.New(testName, testClientURL, testClientURL, &server.TLSConfig{Scheme:"http"}, &server.TLSInfo{}, ps, registry, store) + ps := server.NewPeerServer(testName, path, testRaftURL, testRaftURL, &server.TLSConfig{Scheme: "http"}, &server.TLSInfo{}, registry, store, testSnapCount) + s := server.New(testName, testClientURL, testClientURL, &server.TLSConfig{Scheme: "http"}, &server.TLSInfo{}, ps, registry, store) ps.SetServer(s) // Start up peer server. @@ -32,17 +33,17 @@ func RunServer(f func(*server.Server)) { c <- true ps.ListenAndServe(false, []string{}) }() - <- c + <-c // Start up etcd server. go func() { c <- true s.ListenAndServe() }() - <- c + <-c // Wait to make sure servers have started. - time.Sleep(5 * time.Millisecond) + time.Sleep(50 * time.Millisecond) // Execute the function passed in. f(s) diff --git a/third_party/deps b/third_party/deps index 0cb7ffea3df..76464bdfc9c 100755 --- a/third_party/deps +++ b/third_party/deps @@ -1,4 +1,5 @@ packages=" + github.com/BurntSushi/toml github.com/coreos/go-raft github.com/coreos/go-etcd github.com/coreos/go-log/log diff --git a/third_party/github.com/BurntSushi/toml/.gitignore b/third_party/github.com/BurntSushi/toml/.gitignore new file mode 100644 index 00000000000..55e90a1e5c7 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/.gitignore @@ -0,0 +1,4 @@ +TAGS +tags +.*.swp +tomlcheck/tomlcheck diff --git a/third_party/github.com/BurntSushi/toml/COMPATIBLE b/third_party/github.com/BurntSushi/toml/COMPATIBLE new file mode 100644 index 00000000000..21e0938caef --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/COMPATIBLE @@ -0,0 +1,3 @@ +Compatible with TOML version +[v0.2.0](https://github.com/mojombo/toml/blob/master/versions/toml-v0.2.0.md) + diff --git a/third_party/github.com/BurntSushi/toml/COPYING b/third_party/github.com/BurntSushi/toml/COPYING new file mode 100644 index 00000000000..5a8e332545f --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/COPYING @@ -0,0 +1,14 @@ + DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE + Version 2, December 2004 + + Copyright (C) 2004 Sam Hocevar + + Everyone is permitted to copy and distribute verbatim or modified + copies of this license document, and changing it is allowed as long + as the name is changed. + + DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. You just DO WHAT THE FUCK YOU WANT TO. + diff --git a/third_party/github.com/BurntSushi/toml/Makefile b/third_party/github.com/BurntSushi/toml/Makefile new file mode 100644 index 00000000000..e6adc3e9efc --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/Makefile @@ -0,0 +1,14 @@ +install: + go install + +fmt: + gofmt -w *.go */*.go + colcheck *.go */*.go + +tags: + find ./ -name '*.go' -print0 | xargs -0 gotags > TAGS + +push: + git push origin master + git push github master + diff --git a/third_party/github.com/BurntSushi/toml/README.md b/third_party/github.com/BurntSushi/toml/README.md new file mode 100644 index 00000000000..f8c847d66dd --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/README.md @@ -0,0 +1,163 @@ +# TOML parser for Go with reflection + +TOML stands for Tom's Obvious, Minimal Language. + +Spec: https://github.com/mojombo/toml + +Compatible with TOML version +[v0.2.0](https://github.com/mojombo/toml/blob/master/versions/toml-v0.2.0.md) + +Documentation: http://godoc.org/github.com/BurntSushi/toml + +Installation: + +```bash +go get github.com/BurntSushi/toml +``` + +Try the toml validator: + +```bash +go get github.com/BurntSushi/toml/tomlv +tomlv some-toml-file.toml +``` + + +## Testing + +This package passes all tests in +[toml-test](https://github.com/BurntSushi/toml-test). + +## Examples + +This package works similarly to how the Go standard library handles `XML` +and `JSON`. Namely, data is loaded into Go values via reflection. + +For the simplest example, consider some TOML file as just a list of keys +and values: + +```toml +Age = 25 +Cats = [ "Cauchy", "Plato" ] +Pi = 3.14 +Perfection = [ 6, 28, 496, 8128 ] +DOB = 1987-07-05T05:45:00Z +``` + +Which could be defined in Go as: + +```go +type Config struct { + Age int + Cats []string + Pi float64 + Perfection []int + DOB time.Time // requires `import time` +} +``` + +And then decoded with: + +```go +var conf Config +if _, err := toml.Decode(tomlData, &conf); err != nil { + // handle error +} +``` + +You can also use struct tags if your struct field name doesn't map to a TOML +key value directly: + +```toml +some_key_NAME = "wat" +``` + +```go +type TOML struct { + ObscureKey string `toml:"some_key_NAME"` +} +``` + +## More complex usage + +Here's an example of how to load the example from the official spec page: + +```toml +# This is a TOML document. Boom. + +title = "TOML Example" + +[owner] +name = "Tom Preston-Werner" +organization = "GitHub" +bio = "GitHub Cofounder & CEO\nLikes tater tots and beer." +dob = 1979-05-27T07:32:00Z # First class dates? Why not? + +[database] +server = "192.168.1.1" +ports = [ 8001, 8001, 8002 ] +connection_max = 5000 +enabled = true + +[servers] + + # You can indent as you please. Tabs or spaces. TOML don't care. + [servers.alpha] + ip = "10.0.0.1" + dc = "eqdc10" + + [servers.beta] + ip = "10.0.0.2" + dc = "eqdc10" + +[clients] +data = [ ["gamma", "delta"], [1, 2] ] # just an update to make sure parsers support it + +# Line breaks are OK when inside arrays +hosts = [ + "alpha", + "omega" +] +``` + +And the corresponding Go types are: + +```go +type tomlConfig struct { + Title string + Owner ownerInfo + DB database `toml:"database"` + Servers map[string]server + Clients clients +} + +type ownerInfo struct { + Name string + Org string `toml:"organization"` + Bio string + DOB time.Time +} + +type database struct { + Server string + Ports []int + ConnMax int `toml:"connection_max"` + Enabled bool +} + +type server struct { + IP string + DC string +} + +type clients struct { + Data [][]interface{} + Hosts []string +} +``` + +Note that a case insensitive match will be tried if an exact match can't be +found. + +A working example of the above can be found in `_examples/example.{go,toml}`. + diff --git a/third_party/github.com/BurntSushi/toml/_examples/example.go b/third_party/github.com/BurntSushi/toml/_examples/example.go new file mode 100644 index 00000000000..c81d25a5248 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/_examples/example.go @@ -0,0 +1,59 @@ +package main + +import ( + "fmt" + "time" + + "github.com/BurntSushi/toml" +) + +type tomlConfig struct { + Title string + Owner ownerInfo + DB database `toml:"database"` + Servers map[string]server + Clients clients +} + +type ownerInfo struct { + Name string + Org string `toml:"organization"` + Bio string + DOB time.Time +} + +type database struct { + Server string + Ports []int + ConnMax int `toml:"connection_max"` + Enabled bool +} + +type server struct { + IP string + DC string +} + +type clients struct { + Data [][]interface{} + Hosts []string +} + +func main() { + var config tomlConfig + if _, err := toml.DecodeFile("example.toml", &config); err != nil { + fmt.Println(err) + return + } + + fmt.Printf("Title: %s\n", config.Title) + fmt.Printf("Owner: %s (%s, %s), Born: %s\n", + config.Owner.Name, config.Owner.Org, config.Owner.Bio, config.Owner.DOB) + fmt.Printf("Database: %s %v (Max conn. %d), Enabled? %v\n", + config.DB.Server, config.DB.Ports, config.DB.ConnMax, config.DB.Enabled) + for serverName, server := range config.Servers { + fmt.Printf("Server: %s (%s, %s)\n", serverName, server.IP, server.DC) + } + fmt.Printf("Client data: %v\n", config.Clients.Data) + fmt.Printf("Client hosts: %v\n", config.Clients.Hosts) +} diff --git a/third_party/github.com/BurntSushi/toml/_examples/example.toml b/third_party/github.com/BurntSushi/toml/_examples/example.toml new file mode 100644 index 00000000000..32c7a4faa46 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/_examples/example.toml @@ -0,0 +1,35 @@ +# This is a TOML document. Boom. + +title = "TOML Example" + +[owner] +name = "Tom Preston-Werner" +organization = "GitHub" +bio = "GitHub Cofounder & CEO\nLikes tater tots and beer." +dob = 1979-05-27T07:32:00Z # First class dates? Why not? + +[database] +server = "192.168.1.1" +ports = [ 8001, 8001, 8002 ] +connection_max = 5000 +enabled = true + +[servers] + + # You can indent as you please. Tabs or spaces. TOML don't care. + [servers.alpha] + ip = "10.0.0.1" + dc = "eqdc10" + + [servers.beta] + ip = "10.0.0.2" + dc = "eqdc10" + +[clients] +data = [ ["gamma", "delta"], [1, 2] ] # just an update to make sure parsers support it + +# Line breaks are OK when inside arrays +hosts = [ + "alpha", + "omega" +] diff --git a/third_party/github.com/BurntSushi/toml/_examples/hard.toml b/third_party/github.com/BurntSushi/toml/_examples/hard.toml new file mode 100644 index 00000000000..26145d2b42f --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/_examples/hard.toml @@ -0,0 +1,22 @@ +# Test file for TOML +# Only this one tries to emulate a TOML file written by a user of the kind of parser writers probably hate +# This part you'll really hate + +[the] +test_string = "You'll hate me after this - #" # " Annoying, isn't it? + + [the.hard] + test_array = [ "] ", " # "] # ] There you go, parse this! + test_array2 = [ "Test #11 ]proved that", "Experiment #9 was a success" ] + # You didn't think it'd as easy as chucking out the last #, did you? + another_test_string = " Same thing, but with a string #" + harder_test_string = " And when \"'s are in the string, along with # \"" # "and comments are there too" + # Things will get harder + + [the.hard.bit#] + what? = "You don't think some user won't do that?" + multi_line_array = [ + "]", + # ] Oh yes I did + ] + diff --git a/third_party/github.com/BurntSushi/toml/_examples/implicit.toml b/third_party/github.com/BurntSushi/toml/_examples/implicit.toml new file mode 100644 index 00000000000..1dea5ceb445 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/_examples/implicit.toml @@ -0,0 +1,4 @@ +# [x] you +# [x.y] don't +# [x.y.z] need these +[x.y.z.w] # for this to work diff --git a/third_party/github.com/BurntSushi/toml/_examples/invalid-apples.toml b/third_party/github.com/BurntSushi/toml/_examples/invalid-apples.toml new file mode 100644 index 00000000000..74e9e337edd --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/_examples/invalid-apples.toml @@ -0,0 +1,6 @@ +# DO NOT WANT +[fruit] +type = "apple" + +[fruit.type] +apple = "yes" diff --git a/third_party/github.com/BurntSushi/toml/_examples/invalid.toml b/third_party/github.com/BurntSushi/toml/_examples/invalid.toml new file mode 100644 index 00000000000..beb1dba54dd --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/_examples/invalid.toml @@ -0,0 +1,35 @@ +# This is an INVALID TOML document. Boom. +# Can you spot the error without help? + +title = "TOML Example" + +[owner] +name = "Tom Preston-Werner" +organization = "GitHub" +bio = "GitHub Cofounder & CEO\nLikes tater tots and beer." +dob = 1979-05-27T7:32:00Z # First class dates? Why not? + +[database] +server = "192.168.1.1" +ports = [ 8001, 8001, 8002 ] +connection_max = 5000 +enabled = true + +[servers] + # You can indent as you please. Tabs or spaces. TOML don't care. + [servers.alpha] + ip = "10.0.0.1" + dc = "eqdc10" + + [servers.beta] + ip = "10.0.0.2" + dc = "eqdc10" + +[clients] +data = [ ["gamma", "delta"], [1, 2] ] # just an update to make sure parsers support it + +# Line breaks are OK when inside arrays +hosts = [ + "alpha", + "omega" +] diff --git a/third_party/github.com/BurntSushi/toml/_examples/readme1.toml b/third_party/github.com/BurntSushi/toml/_examples/readme1.toml new file mode 100644 index 00000000000..3e1261d4c23 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/_examples/readme1.toml @@ -0,0 +1,5 @@ +Age = 25 +Cats = [ "Cauchy", "Plato" ] +Pi = 3.14 +Perfection = [ 6, 28, 496, 8128 ] +DOB = 1987-07-05T05:45:00Z diff --git a/third_party/github.com/BurntSushi/toml/_examples/readme2.toml b/third_party/github.com/BurntSushi/toml/_examples/readme2.toml new file mode 100644 index 00000000000..b51cd93408a --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/_examples/readme2.toml @@ -0,0 +1 @@ +some_key_NAME = "wat" diff --git a/third_party/github.com/BurntSushi/toml/decode.go b/third_party/github.com/BurntSushi/toml/decode.go new file mode 100644 index 00000000000..ddb26cf2be5 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/decode.go @@ -0,0 +1,429 @@ +package toml + +import ( + "fmt" + "io" + "io/ioutil" + "reflect" + "strings" + "time" +) + +var e = fmt.Errorf + +// Primitive is a TOML value that hasn't been decoded into a Go value. +// When using the various `Decode*` functions, the type `Primitive` may +// be given to any value, and its decoding will be delayed. +// +// A `Primitive` value can be decoded using the `PrimitiveDecode` function. +// +// The underlying representation of a `Primitive` value is subject to change. +// Do not rely on it. +// +// N.B. Primitive values are still parsed, so using them will only avoid +// the overhead of reflection. They can be useful when you don't know the +// exact type of TOML data until run time. +type Primitive interface{} + +// PrimitiveDecode is just like the other `Decode*` functions, except it +// decodes a TOML value that has already been parsed. Valid primitive values +// can *only* be obtained from values filled by the decoder functions, +// including `PrimitiveDecode`. (i.e., `v` may contain more `Primitive` +// values.) +// +// Meta data for primitive values is included in the meta data returned by +// the `Decode*` functions. +func PrimitiveDecode(primValue Primitive, v interface{}) error { + return unify(primValue, rvalue(v)) +} + +// Decode will decode the contents of `data` in TOML format into a pointer +// `v`. +// +// TOML hashes correspond to Go structs or maps. (Dealer's choice. They can be +// used interchangeably.) +// +// TOML datetimes correspond to Go `time.Time` values. +// +// All other TOML types (float, string, int, bool and array) correspond +// to the obvious Go types. +// +// TOML keys can map to either keys in a Go map or field names in a Go +// struct. The special `toml` struct tag may be used to map TOML keys to +// struct fields that don't match the key name exactly. (See the example.) +// A case insensitive match to struct names will be tried if an exact match +// can't be found. +// +// The mapping between TOML values and Go values is loose. That is, there +// may exist TOML values that cannot be placed into your representation, and +// there may be parts of your representation that do not correspond to +// TOML values. +// +// This decoder will not handle cyclic types. If a cyclic type is passed, +// `Decode` will not terminate. +func Decode(data string, v interface{}) (MetaData, error) { + p, err := parse(data) + if err != nil { + return MetaData{}, err + } + return MetaData{p.mapping, p.types, p.ordered}, unify(p.mapping, rvalue(v)) +} + +// DecodeFile is just like Decode, except it will automatically read the +// contents of the file at `fpath` and decode it for you. +func DecodeFile(fpath string, v interface{}) (MetaData, error) { + bs, err := ioutil.ReadFile(fpath) + if err != nil { + return MetaData{}, err + } + return Decode(string(bs), v) +} + +// DecodeReader is just like Decode, except it will consume all bytes +// from the reader and decode it for you. +func DecodeReader(r io.Reader, v interface{}) (MetaData, error) { + bs, err := ioutil.ReadAll(r) + if err != nil { + return MetaData{}, err + } + return Decode(string(bs), v) +} + +// unify performs a sort of type unification based on the structure of `rv`, +// which is the client representation. +// +// Any type mismatch produces an error. Finding a type that we don't know +// how to handle produces an unsupported type error. +func unify(data interface{}, rv reflect.Value) error { + // Special case. Look for a `Primitive` value. + if rv.Type() == reflect.TypeOf((*Primitive)(nil)).Elem() { + return unifyAnything(data, rv) + } + + // Special case. Go's `time.Time` is a struct, which we don't want + // to confuse with a user struct. + if rv.Type().AssignableTo(rvalue(time.Time{}).Type()) { + return unifyDatetime(data, rv) + } + + k := rv.Kind() + + // laziness + if k >= reflect.Int && k <= reflect.Uint64 { + return unifyInt(data, rv) + } + switch k { + case reflect.Ptr: + elem := reflect.New(rv.Type().Elem()) + err := unify(data, reflect.Indirect(elem)) + if err != nil { + return err + } + rv.Set(elem) + return nil + case reflect.Struct: + return unifyStruct(data, rv) + case reflect.Map: + return unifyMap(data, rv) + case reflect.Slice: + return unifySlice(data, rv) + case reflect.String: + return unifyString(data, rv) + case reflect.Bool: + return unifyBool(data, rv) + case reflect.Interface: + // we only support empty interfaces. + if rv.NumMethod() > 0 { + return e("Unsupported type '%s'.", rv.Kind()) + } + return unifyAnything(data, rv) + case reflect.Float32: + fallthrough + case reflect.Float64: + return unifyFloat64(data, rv) + } + return e("Unsupported type '%s'.", rv.Kind()) +} + +func unifyStruct(mapping interface{}, rv reflect.Value) error { + tmap, ok := mapping.(map[string]interface{}) + if !ok { + return mismatch(rv, "map", mapping) + } + + rt := rv.Type() + for i := 0; i < rt.NumField(); i++ { + // A little tricky. We want to use the special `toml` name in the + // struct tag if it exists. In particular, we need to make sure that + // this struct field is in the current map before trying to unify it. + sft := rt.Field(i) + kname := sft.Tag.Get("toml") + if len(kname) == 0 { + kname = sft.Name + } + if datum, ok := insensitiveGet(tmap, kname); ok { + sf := indirect(rv.Field(i)) + + // Don't try to mess with unexported types and other such things. + if sf.CanSet() { + if err := unify(datum, sf); err != nil { + return e("Type mismatch for '%s.%s': %s", + rt.String(), sft.Name, err) + } + } else if len(sft.Tag.Get("toml")) > 0 { + // Bad user! No soup for you! + return e("Field '%s.%s' is unexported, and therefore cannot "+ + "be loaded with reflection.", rt.String(), sft.Name) + } + } + } + return nil +} + +func unifyMap(mapping interface{}, rv reflect.Value) error { + tmap, ok := mapping.(map[string]interface{}) + if !ok { + return badtype("map", mapping) + } + if rv.IsNil() { + rv.Set(reflect.MakeMap(rv.Type())) + } + for k, v := range tmap { + rvkey := indirect(reflect.New(rv.Type().Key())) + rvval := reflect.Indirect(reflect.New(rv.Type().Elem())) + if err := unify(v, rvval); err != nil { + return err + } + + rvkey.SetString(k) + rv.SetMapIndex(rvkey, rvval) + } + return nil +} + +func unifySlice(data interface{}, rv reflect.Value) error { + datav := reflect.ValueOf(data) + if datav.Kind() != reflect.Slice { + return badtype("slice", data) + } + sliceLen := datav.Len() + if rv.IsNil() { + rv.Set(reflect.MakeSlice(rv.Type(), sliceLen, sliceLen)) + } + for i := 0; i < sliceLen; i++ { + v := datav.Index(i).Interface() + sliceval := indirect(rv.Index(i)) + if err := unify(v, sliceval); err != nil { + return err + } + } + return nil +} + +func unifyDatetime(data interface{}, rv reflect.Value) error { + if _, ok := data.(time.Time); ok { + rv.Set(reflect.ValueOf(data)) + return nil + } + return badtype("time.Time", data) +} + +func unifyString(data interface{}, rv reflect.Value) error { + if s, ok := data.(string); ok { + rv.SetString(s) + return nil + } + return badtype("string", data) +} + +func unifyFloat64(data interface{}, rv reflect.Value) error { + if num, ok := data.(float64); ok { + switch rv.Kind() { + case reflect.Float32: + fallthrough + case reflect.Float64: + rv.SetFloat(num) + default: + panic("bug") + } + return nil + } + return badtype("float", data) +} + +func unifyInt(data interface{}, rv reflect.Value) error { + if num, ok := data.(int64); ok { + switch rv.Kind() { + case reflect.Int: + fallthrough + case reflect.Int8: + fallthrough + case reflect.Int16: + fallthrough + case reflect.Int32: + fallthrough + case reflect.Int64: + rv.SetInt(int64(num)) + case reflect.Uint: + fallthrough + case reflect.Uint8: + fallthrough + case reflect.Uint16: + fallthrough + case reflect.Uint32: + fallthrough + case reflect.Uint64: + rv.SetUint(uint64(num)) + default: + panic("bug") + } + return nil + } + return badtype("integer", data) +} + +func unifyBool(data interface{}, rv reflect.Value) error { + if b, ok := data.(bool); ok { + rv.SetBool(b) + return nil + } + return badtype("integer", data) +} + +func unifyAnything(data interface{}, rv reflect.Value) error { + // too awesome to fail + rv.Set(reflect.ValueOf(data)) + return nil +} + +// rvalue returns a reflect.Value of `v`. All pointers are resolved. +func rvalue(v interface{}) reflect.Value { + return indirect(reflect.ValueOf(v)) +} + +// indirect returns the value pointed to by a pointer. +// Pointers are followed until the value is not a pointer. +// New values are allocated for each nil pointer. +func indirect(v reflect.Value) reflect.Value { + if v.Kind() != reflect.Ptr { + return v + } + if v.IsNil() { + v.Set(reflect.New(v.Type().Elem())) + } + return indirect(reflect.Indirect(v)) +} + +func tstring(rv reflect.Value) string { + return rv.Type().String() +} + +func badtype(expected string, data interface{}) error { + return e("Expected %s but found '%T'.", expected, data) +} + +func mismatch(user reflect.Value, expected string, data interface{}) error { + return e("Type mismatch for %s. Expected %s but found '%T'.", + tstring(user), expected, data) +} + +func insensitiveGet( + tmap map[string]interface{}, kname string) (interface{}, bool) { + + if datum, ok := tmap[kname]; ok { + return datum, true + } + for k, v := range tmap { + if strings.EqualFold(kname, k) { + return v, true + } + } + return nil, false +} + +// MetaData allows access to meta information about TOML data that may not +// be inferrable via reflection. In particular, whether a key has been defined +// and the TOML type of a key. +// +// (XXX: If TOML gets NULL values, that information will be added here too.) +type MetaData struct { + mapping map[string]interface{} + types map[string]tomlType + keys []Key +} + +// IsDefined returns true if the key given exists in the TOML data. The key +// should be specified hierarchially. e.g., +// +// // access the TOML key 'a.b.c' +// IsDefined("a", "b", "c") +// +// IsDefined will return false if an empty key given. Keys are case sensitive. +func (md MetaData) IsDefined(key ...string) bool { + var hashOrVal interface{} + var hash map[string]interface{} + var ok bool + + if len(key) == 0 { + return false + } + + hashOrVal = md.mapping + for _, k := range key { + if hash, ok = hashOrVal.(map[string]interface{}); !ok { + return false + } + if hashOrVal, ok = hash[k]; !ok { + return false + } + } + return true +} + +// Type returns a string representation of the type of the key specified. +// +// Type will return the empty string if given an empty key or a key that +// does not exist. Keys are case sensitive. +func (md MetaData) Type(key ...string) string { + fullkey := strings.Join(key, ".") + if typ, ok := md.types[fullkey]; ok { + return typ.typeString() + } + return "" +} + +// Key is the type of any TOML key, including key groups. Use (MetaData).Keys +// to get values of this type. +type Key []string + +func (k Key) String() string { + return strings.Join(k, ".") +} + +func (k Key) add(piece string) Key { + newKey := make(Key, len(k)) + copy(newKey, k) + return append(newKey, piece) +} + +// Keys returns a slice of every key in the TOML data, including key groups. +// Each key is itself a slice, where the first element is the top of the +// hierarchy and the last is the most specific. +// +// The list will have the same order as the keys appeared in the TOML data. +// +// All keys returned are non-empty. +func (md MetaData) Keys() []Key { + return md.keys +} + +func allKeys(m map[string]interface{}, context Key) []Key { + keys := make([]Key, 0, len(m)) + for k, v := range m { + keys = append(keys, context.add(k)) + if t, ok := v.(map[string]interface{}); ok { + keys = append(keys, allKeys(t, context.add(k))...) + } + } + return keys +} diff --git a/third_party/github.com/BurntSushi/toml/decode_test.go b/third_party/github.com/BurntSushi/toml/decode_test.go new file mode 100644 index 00000000000..ad80adee376 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/decode_test.go @@ -0,0 +1,343 @@ +package toml + +import ( + "fmt" + "log" + "reflect" + "testing" + "time" +) + +func init() { + log.SetFlags(0) +} + +var testSimple = ` +age = 250 + +andrew = "gallant" +kait = "brady" +now = 1987-07-05T05:45:00Z +yesOrNo = true +pi = 3.14 +colors = [ + ["red", "green", "blue"], + ["cyan", "magenta", "yellow", "black"], +] + +[Annoying.Cats] +plato = "smelly" +cauchy = "stupido" + +` + +type kitties struct { + Plato string + Cauchy string +} + +type simple struct { + Age int + Colors [][]string + Pi float64 + YesOrNo bool + Now time.Time + Andrew string + Kait string + Annoying map[string]kitties +} + +func TestDecode(t *testing.T) { + var val simple + + md, err := Decode(testSimple, &val) + if err != nil { + t.Fatal(err) + } + + testf("Is 'Annoying.Cats.plato' defined? %v\n", + md.IsDefined("Annoying", "Cats", "plato")) + testf("Is 'Cats.Stinky' defined? %v\n", md.IsDefined("Cats", "Stinky")) + testf("Type of 'colors'? %s\n\n", md.Type("colors")) + + testf("%v\n", val) +} + +var tomlTableArrays = ` +[[albums]] +name = "Born to Run" + + [[albums.songs]] + name = "Jungleland" + + [[albums.songs]] + name = "Meeting Across the River" + +[[albums]] +name = "Born in the USA" + + [[albums.songs]] + name = "Glory Days" + + [[albums.songs]] + name = "Dancing in the Dark" +` + +type Music struct { + Albums []Album +} + +type Album struct { + Name string + Songs []Song +} + +type Song struct { + Name string +} + +func TestTableArrays(t *testing.T) { + expected := Music{[]Album{ + {"Born to Run", []Song{{"Jungleland"}, {"Meeting Across the River"}}}, + {"Born in the USA", []Song{{"Glory Days"}, {"Dancing in the Dark"}}}, + }} + var got Music + if _, err := Decode(tomlTableArrays, &got); err != nil { + t.Fatal(err) + } + if !reflect.DeepEqual(expected, got) { + t.Fatalf("\n%#v\n!=\n%#v\n", expected, got) + } +} + +// Case insensitive matching tests. +// A bit more comprehensive than needed given the current implementation, +// but implementations change. +// Probably still missing demonstrations of some ugly corner cases regarding +// case insensitive matching and multiple fields. +var caseToml = ` +tOpString = "string" +tOpInt = 1 +tOpFloat = 1.1 +tOpBool = true +tOpdate = 2006-01-02T15:04:05Z +tOparray = [ "array" ] +Match = "i should be in Match only" +MatcH = "i should be in MatcH only" +Field = "neat" +FielD = "messy" +once = "just once" +[nEst.eD] +nEstedString = "another string" +` + +type Insensitive struct { + TopString string + TopInt int + TopFloat float64 + TopBool bool + TopDate time.Time + TopArray []string + Match string + MatcH string + Field string + Once string + OncE string + Nest InsensitiveNest +} + +type InsensitiveNest struct { + Ed InsensitiveEd +} + +type InsensitiveEd struct { + NestedString string +} + +func TestCase(t *testing.T) { + tme, err := time.Parse(time.RFC3339, time.RFC3339[:len(time.RFC3339)-5]) + if err != nil { + panic(err) + } + expected := Insensitive{ + TopString: "string", + TopInt: 1, + TopFloat: 1.1, + TopBool: true, + TopDate: tme, + TopArray: []string{"array"}, + MatcH: "i should be in MatcH only", + Match: "i should be in Match only", + Field: "neat", // encoding/json would store "messy" here + Once: "just once", + OncE: "just once", // wait, what? + Nest: InsensitiveNest{ + Ed: InsensitiveEd{NestedString: "another string"}, + }, + } + var got Insensitive + _, err = Decode(caseToml, &got) + if err != nil { + t.Fatal(err) + } + if !reflect.DeepEqual(expected, got) { + t.Fatalf("\n%#v\n!=\n%#v\n", expected, got) + } +} + +func TestPointers(t *testing.T) { + type Object struct { + Type string + Description string + } + + type Dict struct { + NamedObject map[string]*Object + BaseObject *Object + Strptr *string + Strptrs []*string + } + s1, s2, s3 := "blah", "abc", "def" + expected := &Dict{ + Strptr: &s1, + Strptrs: []*string{&s2, &s3}, + NamedObject: map[string]*Object{ + "foo": {"FOO", "fooooo!!!"}, + "bar": {"BAR", "ba-ba-ba-ba-barrrr!!!"}, + }, + BaseObject: &Object{"BASE", "da base"}, + } + + ex1 := ` +Strptr = "blah" +Strptrs = ["abc", "def"] + +[NamedObject.foo] +Type = "FOO" +Description = "fooooo!!!" + +[NamedObject.bar] +Type = "BAR" +Description = "ba-ba-ba-ba-barrrr!!!" + +[BaseObject] +Type = "BASE" +Description = "da base" +` + dict := new(Dict) + _, err := Decode(ex1, dict) + if err != nil { + t.Errorf("Decode error: %v", err) + } + if !reflect.DeepEqual(expected, dict) { + t.Fatalf("\n%#v\n!=\n%#v\n", expected, dict) + } +} + +func ExamplePrimitiveDecode() { + var md MetaData + var err error + + var tomlBlob = ` +ranking = ["Springsteen", "J Geils"] + +[bands.Springsteen] +started = 1973 +albums = ["Greetings", "WIESS", "Born to Run", "Darkness"] + +[bands.J Geils] +started = 1970 +albums = ["The J. Geils Band", "Full House", "Blow Your Face Out"] +` + + type band struct { + Started int + Albums []string + } + + type classics struct { + Ranking []string + Bands map[string]Primitive + } + + // Do the initial decode. Reflection is delayed on Primitive values. + var music classics + if md, err = Decode(tomlBlob, &music); err != nil { + log.Fatal(err) + } + + // MetaData still includes information on Primitive values. + fmt.Printf("Is `bands.Springsteen` defined? %v\n", + md.IsDefined("bands", "Springsteen")) + + // Decode primitive data into Go values. + for _, artist := range music.Ranking { + // A band is a primitive value, so we need to decode it to get a + // real `band` value. + primValue := music.Bands[artist] + + var aBand band + if err = PrimitiveDecode(primValue, &aBand); err != nil { + log.Fatal(err) + } + fmt.Printf("%s started in %d.\n", artist, aBand.Started) + } + + // Output: + // Is `bands.Springsteen` defined? true + // Springsteen started in 1973. + // J Geils started in 1970. +} + +func ExampleDecode() { + var tomlBlob = ` +# Some comments. +[alpha] +ip = "10.0.0.1" + + [alpha.config] + Ports = [ 8001, 8002 ] + Location = "Toronto" + Created = 1987-07-05T05:45:00Z + +[beta] +ip = "10.0.0.2" + + [beta.config] + Ports = [ 9001, 9002 ] + Location = "New Jersey" + Created = 1887-01-05T05:55:00Z +` + + type serverConfig struct { + Ports []int + Location string + Created time.Time + } + + type server struct { + IP string `toml:"ip"` + Config serverConfig `toml:"config"` + } + + type servers map[string]server + + var config servers + if _, err := Decode(tomlBlob, &config); err != nil { + log.Fatal(err) + } + + for _, name := range []string{"alpha", "beta"} { + s := config[name] + fmt.Printf("Server: %s (ip: %s) in %s created on %s\n", + name, s.IP, s.Config.Location, + s.Config.Created.Format("2006-01-02")) + fmt.Printf("Ports: %v\n", s.Config.Ports) + } + + // // Output: + // Server: alpha (ip: 10.0.0.1) in Toronto created on 1987-07-05 + // Ports: [8001 8002] + // Server: beta (ip: 10.0.0.2) in New Jersey created on 1887-01-05 + // Ports: [9001 9002] +} diff --git a/third_party/github.com/BurntSushi/toml/doc.go b/third_party/github.com/BurntSushi/toml/doc.go new file mode 100644 index 00000000000..1c2d7dffc21 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/doc.go @@ -0,0 +1,10 @@ +/* +Package toml provides facilities for decoding TOML configuration files +via reflection. + +Specification: https://github.com/mojombo/toml + +Use github.com/BurntSushi/toml/tomlv to check whether a file is valid +TOML or not, with helpful error messages. +*/ +package toml diff --git a/third_party/github.com/BurntSushi/toml/encode.go b/third_party/github.com/BurntSushi/toml/encode.go new file mode 100644 index 00000000000..7b080f414c6 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/encode.go @@ -0,0 +1,99 @@ +package toml + +// TODO: Build a decent encoder. +// Interestingly, this isn't as trivial as recursing down the type of the +// value given and outputting the corresponding TOML. In particular, multiple +// TOML types (especially if tuples are added) can map to a single Go type, so +// that the reverse correspondence isn't clear. +// +// One possible avenue is to choose a reasonable default (like structs map +// to hashes), but allow the user to override with struct tags. But this seems +// like a mess. +// +// The other possibility is to scrap an encoder altogether. After all, TOML +// is a configuration file format, and not a data exchange format. + +import ( + "bufio" + "fmt" + "io" + "reflect" + "strings" +) + +type encoder struct { + // A single indentation level. By default it is two spaces. + Indent string + + w *bufio.Writer +} + +func newEncoder(w io.Writer) *encoder { + return &encoder{ + w: bufio.NewWriter(w), + Indent: " ", + } +} + +func (enc *encoder) Encode(v interface{}) error { + rv := eindirect(reflect.ValueOf(v)) + if err := enc.encode(Key([]string{}), rv); err != nil { + return err + } + return enc.w.Flush() +} + +func (enc *encoder) encode(key Key, rv reflect.Value) error { + k := rv.Kind() + switch k { + case reflect.Struct: + return enc.eStruct(key, rv) + case reflect.String: + return enc.eString(key, rv) + } + return e("Unsupported type for key '%s': %s", key, k) +} + +func (enc *encoder) eStruct(key Key, rv reflect.Value) error { + rt := rv.Type() + for i := 0; i < rt.NumField(); i++ { + sft := rt.Field(i) + sf := rv.Field(i) + if err := enc.encode(key.add(sft.Name), sf); err != nil { + return err + } + } + return nil +} + +func (enc *encoder) eString(key Key, rv reflect.Value) error { + s := rv.String() + s = strings.NewReplacer( + "\t", "\\t", + "\n", "\\n", + "\r", "\\r", + "\"", "\\\"", + "\\", "\\\\", + ).Replace(s) + s = "\"" + s + "\"" + if err := enc.eKeyVal(key, s); err != nil { + return err + } + return nil +} + +func (enc *encoder) eKeyVal(key Key, value string) error { + out := fmt.Sprintf("%s%s = %s", + strings.Repeat(enc.Indent, len(key)-1), key[len(key)-1], value) + if _, err := fmt.Fprintln(enc.w, out); err != nil { + return err + } + return nil +} + +func eindirect(v reflect.Value) reflect.Value { + if v.Kind() != reflect.Ptr { + return v + } + return eindirect(reflect.Indirect(v)) +} diff --git a/third_party/github.com/BurntSushi/toml/encode_test.go b/third_party/github.com/BurntSushi/toml/encode_test.go new file mode 100644 index 00000000000..3f7fad8b4b1 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/encode_test.go @@ -0,0 +1,25 @@ +package toml + +import ( + "bytes" + "testing" +) + +type encodeSimple struct { + Location string + // Ages []int + // DOB time.Time +} + +func TestEncode(t *testing.T) { + v := encodeSimple{ + Location: "Westborough, MA", + } + + buf := new(bytes.Buffer) + e := newEncoder(buf) + if err := e.Encode(v); err != nil { + t.Fatal(err) + } + testf(buf.String()) +} diff --git a/third_party/github.com/BurntSushi/toml/lex.go b/third_party/github.com/BurntSushi/toml/lex.go new file mode 100644 index 00000000000..3dcae4923c6 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/lex.go @@ -0,0 +1,741 @@ +package toml + +import ( + "fmt" + "unicode/utf8" +) + +type itemType int + +const ( + itemError itemType = iota + itemNIL // used in the parser to indicate no type + itemEOF + itemText + itemString + itemBool + itemInteger + itemFloat + itemDatetime + itemArray // the start of an array + itemArrayEnd + itemTableStart + itemTableEnd + itemArrayTableStart + itemArrayTableEnd + itemKeyStart + itemCommentStart +) + +const ( + eof = 0 + tableStart = '[' + tableEnd = ']' + arrayTableStart = '[' + arrayTableEnd = ']' + tableSep = '.' + keySep = '=' + arrayStart = '[' + arrayEnd = ']' + arrayValTerm = ',' + commentStart = '#' + stringStart = '"' + stringEnd = '"' +) + +type stateFn func(lx *lexer) stateFn + +type lexer struct { + input string + start int + pos int + width int + line int + state stateFn + items chan item + + // A stack of state functions used to maintain context. + // The idea is to reuse parts of the state machine in various places. + // For example, values can appear at the top level or within arbitrarily + // nested arrays. The last state on the stack is used after a value has + // been lexed. Similarly for comments. + stack []stateFn +} + +type item struct { + typ itemType + val string + line int +} + +func (lx *lexer) nextItem() item { + for { + select { + case item := <-lx.items: + return item + default: + lx.state = lx.state(lx) + } + } + panic("not reached") +} + +func lex(input string) *lexer { + lx := &lexer{ + input: input, + state: lexTop, + line: 1, + items: make(chan item, 10), + stack: make([]stateFn, 0, 10), + } + return lx +} + +func (lx *lexer) push(state stateFn) { + lx.stack = append(lx.stack, state) +} + +func (lx *lexer) pop() stateFn { + if len(lx.stack) == 0 { + return lx.errorf("BUG in lexer: no states to pop.") + } + last := lx.stack[len(lx.stack)-1] + lx.stack = lx.stack[0 : len(lx.stack)-1] + return last +} + +func (lx *lexer) current() string { + return lx.input[lx.start:lx.pos] +} + +func (lx *lexer) emit(typ itemType) { + lx.items <- item{typ, lx.current(), lx.line} + lx.start = lx.pos +} + +func (lx *lexer) next() (r rune) { + if lx.pos >= len(lx.input) { + lx.width = 0 + return eof + } + + if lx.input[lx.pos] == '\n' { + lx.line++ + } + r, lx.width = utf8.DecodeRuneInString(lx.input[lx.pos:]) + lx.pos += lx.width + return r +} + +// ignore skips over the pending input before this point. +func (lx *lexer) ignore() { + lx.start = lx.pos +} + +// backup steps back one rune. Can be called only once per call of next. +func (lx *lexer) backup() { + lx.pos -= lx.width + if lx.pos < len(lx.input) && lx.input[lx.pos] == '\n' { + lx.line-- + } +} + +// accept consumes the next rune if it's equal to `valid`. +func (lx *lexer) accept(valid rune) bool { + if lx.next() == valid { + return true + } + lx.backup() + return false +} + +// peek returns but does not consume the next rune in the input. +func (lx *lexer) peek() rune { + r := lx.next() + lx.backup() + return r +} + +// errorf stops all lexing by emitting an error and returning `nil`. +// Note that any value that is a character is escaped if it's a special +// character (new lines, tabs, etc.). +func (lx *lexer) errorf(format string, values ...interface{}) stateFn { + for i, value := range values { + if v, ok := value.(rune); ok { + values[i] = escapeSpecial(v) + } + } + lx.items <- item{ + itemError, + fmt.Sprintf(format, values...), + lx.line, + } + return nil +} + +// lexTop consumes elements at the top level of TOML data. +func lexTop(lx *lexer) stateFn { + r := lx.next() + if isWhitespace(r) || isNL(r) { + return lexSkip(lx, lexTop) + } + + switch r { + case commentStart: + lx.push(lexTop) + return lexCommentStart + case tableStart: + return lexTableStart + case eof: + if lx.pos > lx.start { + return lx.errorf("Unexpected EOF.") + } + lx.emit(itemEOF) + return nil + } + + // At this point, the only valid item can be a key, so we back up + // and let the key lexer do the rest. + lx.backup() + lx.push(lexTopEnd) + return lexKeyStart +} + +// lexTopEnd is entered whenever a top-level item has been consumed. (A value +// or a table.) It must see only whitespace, and will turn back to lexTop +// upon a new line. If it sees EOF, it will quit the lexer successfully. +func lexTopEnd(lx *lexer) stateFn { + r := lx.next() + switch { + case r == commentStart: + // a comment will read to a new line for us. + lx.push(lexTop) + return lexCommentStart + case isWhitespace(r): + return lexTopEnd + case isNL(r): + lx.ignore() + return lexTop + case r == eof: + lx.ignore() + return lexTop + } + return lx.errorf("Expected a top-level item to end with a new line, "+ + "comment or EOF, but got '%s' instead.", r) +} + +// lexTable lexes the beginning of a table. Namely, it makes sure that +// it starts with a character other than '.' and ']'. +// It assumes that '[' has already been consumed. +// It also handles the case that this is an item in an array of tables. +// e.g., '[[name]]'. +func lexTableStart(lx *lexer) stateFn { + if lx.peek() == arrayTableStart { + lx.next() + lx.emit(itemArrayTableStart) + lx.push(lexArrayTableEnd) + } else { + lx.emit(itemTableStart) + lx.push(lexTableEnd) + } + return lexTableNameStart +} + +func lexTableEnd(lx *lexer) stateFn { + lx.emit(itemTableEnd) + return lexTopEnd +} + +func lexArrayTableEnd(lx *lexer) stateFn { + if r := lx.next(); r != arrayTableEnd { + return lx.errorf("Expected end of table array name delimiter '%s', "+ + "but got '%s' instead.", arrayTableEnd, r) + } + lx.emit(itemArrayTableEnd) + return lexTopEnd +} + +func lexTableNameStart(lx *lexer) stateFn { + switch lx.next() { + case tableEnd: + return lx.errorf("Unexpected end of table. (Tables cannot " + + "be empty.)") + case tableSep: + return lx.errorf("Unexpected table separator. (Tables cannot " + + "be empty.)") + } + return lexTableName +} + +// lexTableName lexes the name of a table. It assumes that at least one +// valid character for the table has already been read. +func lexTableName(lx *lexer) stateFn { + switch lx.peek() { + case tableStart: + return lx.errorf("Table names cannot contain '%s' or '%s'.", + tableStart, tableEnd) + case tableEnd: + lx.emit(itemText) + lx.next() + return lx.pop() + case tableSep: + lx.emit(itemText) + lx.next() + lx.ignore() + return lexTableNameStart + } + lx.next() + return lexTableName +} + +// lexKeyStart consumes a key name up until the first non-whitespace character. +// lexKeyStart will ignore whitespace. +func lexKeyStart(lx *lexer) stateFn { + r := lx.peek() + switch { + case r == keySep: + return lx.errorf("Unexpected key separator '%s'.", keySep) + case isWhitespace(r) || isNL(r): + lx.next() + return lexSkip(lx, lexKeyStart) + } + + lx.ignore() + lx.emit(itemKeyStart) + lx.next() + return lexKey +} + +// lexKey consumes the text of a key. Assumes that the first character (which +// is not whitespace) has already been consumed. +func lexKey(lx *lexer) stateFn { + r := lx.peek() + + // XXX: Possible divergence from spec? + // "Keys start with the first non-whitespace character and end with the + // last non-whitespace character before the equals sign." + // Note here that whitespace is either a tab or a space. + // But we'll call it quits if we see a new line too. + if isWhitespace(r) || isNL(r) { + lx.emit(itemText) + return lexKeyEnd + } + + // Let's also call it quits if we see an equals sign. + if r == keySep { + lx.emit(itemText) + return lexKeyEnd + } + + lx.next() + return lexKey +} + +// lexKeyEnd consumes the end of a key (up to the key separator). +// Assumes that the first whitespace character after a key (or the '=' +// separator) has NOT been consumed. +func lexKeyEnd(lx *lexer) stateFn { + r := lx.next() + switch { + case isWhitespace(r) || isNL(r): + return lexSkip(lx, lexKeyEnd) + case r == keySep: + return lexSkip(lx, lexValue) + } + return lx.errorf("Expected key separator '%s', but got '%s' instead.", + keySep, r) +} + +// lexValue starts the consumption of a value anywhere a value is expected. +// lexValue will ignore whitespace. +// After a value is lexed, the last state on the next is popped and returned. +func lexValue(lx *lexer) stateFn { + // We allow whitespace to precede a value, but NOT new lines. + // In array syntax, the array states are responsible for ignoring new lines. + r := lx.next() + if isWhitespace(r) { + return lexSkip(lx, lexValue) + } + + switch { + case r == arrayStart: + lx.ignore() + lx.emit(itemArray) + return lexArrayValue + case r == stringStart: + lx.ignore() // ignore the '"' + return lexString + case r == 't': + return lexTrue + case r == 'f': + return lexFalse + case r == '-': + return lexNumberStart + case isDigit(r): + lx.backup() // avoid an extra state and use the same as above + return lexNumberOrDateStart + case r == '.': // special error case, be kind to users + return lx.errorf("Floats must start with a digit, not '.'.") + } + return lx.errorf("Expected value but found '%s' instead.", r) +} + +// lexArrayValue consumes one value in an array. It assumes that '[' or ',' +// have already been consumed. All whitespace and new lines are ignored. +func lexArrayValue(lx *lexer) stateFn { + r := lx.next() + switch { + case isWhitespace(r) || isNL(r): + return lexSkip(lx, lexArrayValue) + case r == commentStart: + lx.push(lexArrayValue) + return lexCommentStart + case r == arrayValTerm: + return lx.errorf("Unexpected array value terminator '%s'.", + arrayValTerm) + case r == arrayEnd: + return lexArrayEnd + } + + lx.backup() + lx.push(lexArrayValueEnd) + return lexValue +} + +// lexArrayValueEnd consumes the cruft between values of an array. Namely, +// it ignores whitespace and expects either a ',' or a ']'. +func lexArrayValueEnd(lx *lexer) stateFn { + r := lx.next() + switch { + case isWhitespace(r) || isNL(r): + return lexSkip(lx, lexArrayValueEnd) + case r == commentStart: + lx.push(lexArrayValueEnd) + return lexCommentStart + case r == arrayValTerm: + return lexArrayValue // move on to the next value + case r == arrayEnd: + return lexArrayEnd + } + return lx.errorf("Expected an array value terminator '%s' or an array "+ + "terminator '%s', but got '%s' instead.", arrayValTerm, arrayEnd, r) +} + +// lexArrayEnd finishes the lexing of an array. It assumes that a ']' has +// just been consumed. +func lexArrayEnd(lx *lexer) stateFn { + lx.ignore() + lx.emit(itemArrayEnd) + return lx.pop() +} + +// lexString consumes the inner contents of a string. It assumes that the +// beginning '"' has already been consumed and ignored. +func lexString(lx *lexer) stateFn { + r := lx.next() + switch { + case isNL(r): + return lx.errorf("Strings cannot contain new lines.") + case r == '\\': + return lexStringEscape + case r == stringEnd: + lx.backup() + lx.emit(itemString) + lx.next() + lx.ignore() + return lx.pop() + } + return lexString +} + +// lexStringEscape consumes an escaped character. It assumes that the preceding +// '\\' has already been consumed. +func lexStringEscape(lx *lexer) stateFn { + r := lx.next() + switch r { + case 'b': + fallthrough + case 't': + fallthrough + case 'n': + fallthrough + case 'f': + fallthrough + case 'r': + fallthrough + case '"': + fallthrough + case '/': + fallthrough + case '\\': + return lexString + case 'u': + return lexStringUnicode + } + return lx.errorf("Invalid escape character '%s'. Only the following "+ + "escape characters are allowed: "+ + "\\b, \\t, \\n, \\f, \\r, \\\", \\/, \\\\, and \\uXXXX.", r) +} + +// lexStringBinary consumes two hexadecimal digits following '\x'. It assumes +// that the '\x' has already been consumed. +func lexStringUnicode(lx *lexer) stateFn { + var r rune + + for i := 0; i < 4; i++ { + r = lx.next() + if !isHexadecimal(r) { + return lx.errorf("Expected four hexadecimal digits after '\\x', "+ + "but got '%s' instead.", lx.current()) + } + } + return lexString +} + +// lexNumberOrDateStart consumes either a (positive) integer, float or datetime. +// It assumes that NO negative sign has been consumed. +func lexNumberOrDateStart(lx *lexer) stateFn { + r := lx.next() + if !isDigit(r) { + if r == '.' { + return lx.errorf("Floats must start with a digit, not '.'.") + } else { + return lx.errorf("Expected a digit but got '%s'.", r) + } + } + return lexNumberOrDate +} + +// lexNumberOrDate consumes either a (positive) integer, float or datetime. +func lexNumberOrDate(lx *lexer) stateFn { + r := lx.next() + switch { + case r == '-': + if lx.pos-lx.start != 5 { + return lx.errorf("All ISO8601 dates must be in full Zulu form.") + } + return lexDateAfterYear + case isDigit(r): + return lexNumberOrDate + case r == '.': + return lexFloatStart + } + + lx.backup() + lx.emit(itemInteger) + return lx.pop() +} + +// lexDateAfterYear consumes a full Zulu Datetime in ISO8601 format. +// It assumes that "YYYY-" has already been consumed. +func lexDateAfterYear(lx *lexer) stateFn { + formats := []rune{ + // digits are '0'. + // everything else is direct equality. + '0', '0', '-', '0', '0', + 'T', + '0', '0', ':', '0', '0', ':', '0', '0', + 'Z', + } + for _, f := range formats { + r := lx.next() + if f == '0' { + if !isDigit(r) { + return lx.errorf("Expected digit in ISO8601 datetime, "+ + "but found '%s' instead.", r) + } + } else if f != r { + return lx.errorf("Expected '%s' in ISO8601 datetime, "+ + "but found '%s' instead.", f, r) + } + } + lx.emit(itemDatetime) + return lx.pop() +} + +// lexNumberStart consumes either an integer or a float. It assumes that a +// negative sign has already been read, but that *no* digits have been consumed. +// lexNumberStart will move to the appropriate integer or float states. +func lexNumberStart(lx *lexer) stateFn { + // we MUST see a digit. Even floats have to start with a digit. + r := lx.next() + if !isDigit(r) { + if r == '.' { + return lx.errorf("Floats must start with a digit, not '.'.") + } else { + return lx.errorf("Expected a digit but got '%s'.", r) + } + } + return lexNumber +} + +// lexNumber consumes an integer or a float after seeing the first digit. +func lexNumber(lx *lexer) stateFn { + r := lx.next() + switch { + case isDigit(r): + return lexNumber + case r == '.': + return lexFloatStart + } + + lx.backup() + lx.emit(itemInteger) + return lx.pop() +} + +// lexFloatStart starts the consumption of digits of a float after a '.'. +// Namely, at least one digit is required. +func lexFloatStart(lx *lexer) stateFn { + r := lx.next() + if !isDigit(r) { + return lx.errorf("Floats must have a digit after the '.', but got "+ + "'%s' instead.", r) + } + return lexFloat +} + +// lexFloat consumes the digits of a float after a '.'. +// Assumes that one digit has been consumed after a '.' already. +func lexFloat(lx *lexer) stateFn { + r := lx.next() + if isDigit(r) { + return lexFloat + } + + lx.backup() + lx.emit(itemFloat) + return lx.pop() +} + +// lexTrue consumes the "rue" in "true". It assumes that 't' has already +// been consumed. +func lexTrue(lx *lexer) stateFn { + if r := lx.next(); r != 'r' { + return lx.errorf("Expected 'tr', but found 't%s' instead.", r) + } + if r := lx.next(); r != 'u' { + return lx.errorf("Expected 'tru', but found 'tr%s' instead.", r) + } + if r := lx.next(); r != 'e' { + return lx.errorf("Expected 'true', but found 'tru%s' instead.", r) + } + lx.emit(itemBool) + return lx.pop() +} + +// lexFalse consumes the "alse" in "false". It assumes that 'f' has already +// been consumed. +func lexFalse(lx *lexer) stateFn { + if r := lx.next(); r != 'a' { + return lx.errorf("Expected 'fa', but found 'f%s' instead.", r) + } + if r := lx.next(); r != 'l' { + return lx.errorf("Expected 'fal', but found 'fa%s' instead.", r) + } + if r := lx.next(); r != 's' { + return lx.errorf("Expected 'fals', but found 'fal%s' instead.", r) + } + if r := lx.next(); r != 'e' { + return lx.errorf("Expected 'false', but found 'fals%s' instead.", r) + } + lx.emit(itemBool) + return lx.pop() +} + +// lexCommentStart begins the lexing of a comment. It will emit +// itemCommentStart and consume no characters, passing control to lexComment. +func lexCommentStart(lx *lexer) stateFn { + lx.ignore() + lx.emit(itemCommentStart) + return lexComment +} + +// lexComment lexes an entire comment. It assumes that '#' has been consumed. +// It will consume *up to* the first new line character, and pass control +// back to the last state on the stack. +func lexComment(lx *lexer) stateFn { + r := lx.peek() + if isNL(r) || r == eof { + lx.emit(itemText) + return lx.pop() + } + lx.next() + return lexComment +} + +// lexSkip ignores all slurped input and moves on to the next state. +func lexSkip(lx *lexer, nextState stateFn) stateFn { + return func(lx *lexer) stateFn { + lx.ignore() + return nextState + } +} + +// isWhitespace returns true if `r` is a whitespace character according +// to the spec. +func isWhitespace(r rune) bool { + return r == '\t' || r == ' ' +} + +func isNL(r rune) bool { + return r == '\n' || r == '\r' +} + +func isDigit(r rune) bool { + return r >= '0' && r <= '9' +} + +func isHexadecimal(r rune) bool { + return (r >= '0' && r <= '9') || + (r >= 'a' && r <= 'f') || + (r >= 'A' && r <= 'F') +} + +func (itype itemType) String() string { + switch itype { + case itemError: + return "Error" + case itemNIL: + return "NIL" + case itemEOF: + return "EOF" + case itemText: + return "Text" + case itemString: + return "String" + case itemBool: + return "Bool" + case itemInteger: + return "Integer" + case itemFloat: + return "Float" + case itemDatetime: + return "DateTime" + case itemTableStart: + return "TableStart" + case itemTableEnd: + return "TableEnd" + case itemKeyStart: + return "KeyStart" + case itemArray: + return "Array" + case itemArrayEnd: + return "ArrayEnd" + case itemCommentStart: + return "CommentStart" + } + panic(fmt.Sprintf("BUG: Unknown type '%s'.", itype)) +} + +func (item item) String() string { + return fmt.Sprintf("(%s, %s)", item.typ.String(), item.val) +} + +func escapeSpecial(c rune) string { + switch c { + case '\n': + return "\\n" + } + return string(c) +} diff --git a/third_party/github.com/BurntSushi/toml/lex_test.go b/third_party/github.com/BurntSushi/toml/lex_test.go new file mode 100644 index 00000000000..6cfa21088f9 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/lex_test.go @@ -0,0 +1,59 @@ +package toml + +import ( + "log" + "testing" +) + +func init() { + log.SetFlags(0) +} + +var testSmall = ` +# This is a TOML document. Boom. + +[owner] +[owner] # Whoa there. +andrew = "gallant # poopy" # weeeee +predicate = false +num = -5192 +f = -0.5192 +zulu = 1979-05-27T07:32:00Z +whoop = "poop" +arrs = [ + 1987-07-05T05:45:00Z, + 5, + "wat?", + "hehe \n\r kewl", + [6], [], + 5.0, + # sweetness +] # more comments +# hehe +` + +var testSmaller = ` +[a.b] # Do you ignore me? +andrew = "ga# ll\"ant" # what about me? +kait = "brady" +awesomeness = true +pi = 3.14 +dob = 1987-07-05T17:45:00Z +perfection = [ + [6, 28], + [496, 8128] +] +` + +func TestLexer(t *testing.T) { + lx := lex(testSmaller) + for { + item := lx.nextItem() + if item.typ == itemEOF { + break + } else if item.typ == itemError { + t.Fatal(item.val) + } + testf("%s\n", item) + } +} diff --git a/third_party/github.com/BurntSushi/toml/out_test.go b/third_party/github.com/BurntSushi/toml/out_test.go new file mode 100644 index 00000000000..ab121e37520 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/out_test.go @@ -0,0 +1,19 @@ +package toml + +import ( + "flag" + "fmt" +) + +var flagOut = false + +func init() { + flag.BoolVar(&flagOut, "out", flagOut, "Print debug output.") + flag.Parse() +} + +func testf(format string, v ...interface{}) { + if flagOut { + fmt.Printf(format, v...) + } +} diff --git a/third_party/github.com/BurntSushi/toml/parse.go b/third_party/github.com/BurntSushi/toml/parse.go new file mode 100644 index 00000000000..57e4a70a959 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/parse.go @@ -0,0 +1,388 @@ +package toml + +import ( + "fmt" + "log" + "strconv" + "strings" + "time" + "unicode/utf8" +) + +type parser struct { + mapping map[string]interface{} + types map[string]tomlType + lx *lexer + + // A list of keys in the order that they appear in the TOML data. + ordered []Key + + // the full key for the current hash in scope + context Key + + // the base key name for everything except hashes + currentKey string + + // rough approximation of line number + approxLine int + + // A map of 'key.group.names' to whether they were created implicitly. + implicits map[string]bool +} + +type parseError string + +func (pe parseError) Error() string { + return string(pe) +} + +func parse(data string) (p *parser, err error) { + defer func() { + if r := recover(); r != nil { + var ok bool + if err, ok = r.(parseError); ok { + return + } + panic(r) + } + }() + + p = &parser{ + mapping: make(map[string]interface{}), + types: make(map[string]tomlType), + lx: lex(data), + ordered: make([]Key, 0), + implicits: make(map[string]bool), + } + for { + item := p.next() + if item.typ == itemEOF { + break + } + p.topLevel(item) + } + + return p, nil +} + +func (p *parser) panic(format string, v ...interface{}) { + msg := fmt.Sprintf("Near line %d, key '%s': %s", + p.approxLine, p.current(), fmt.Sprintf(format, v...)) + panic(parseError(msg)) +} + +func (p *parser) next() item { + it := p.lx.nextItem() + if it.typ == itemError { + p.panic("Near line %d: %s", it.line, it.val) + } + return it +} + +func (p *parser) bug(format string, v ...interface{}) { + log.Fatalf("BUG: %s\n\n", fmt.Sprintf(format, v...)) +} + +func (p *parser) expect(typ itemType) item { + it := p.next() + p.assertEqual(typ, it.typ) + return it +} + +func (p *parser) assertEqual(expected, got itemType) { + if expected != got { + p.bug("Expected '%s' but got '%s'.", expected, got) + } +} + +func (p *parser) topLevel(item item) { + switch item.typ { + case itemCommentStart: + p.approxLine = item.line + p.expect(itemText) + case itemTableStart: + kg := p.expect(itemText) + p.approxLine = kg.line + + key := make(Key, 0) + for ; kg.typ == itemText; kg = p.next() { + key = append(key, kg.val) + } + p.assertEqual(itemTableEnd, kg.typ) + + p.establishContext(key, false) + p.setType("", tomlHash) + p.ordered = append(p.ordered, key) + case itemArrayTableStart: + kg := p.expect(itemText) + p.approxLine = kg.line + + key := make(Key, 0) + for ; kg.typ == itemText; kg = p.next() { + key = append(key, kg.val) + } + p.assertEqual(itemArrayTableEnd, kg.typ) + + p.establishContext(key, true) + p.setType("", tomlArrayHash) + p.ordered = append(p.ordered, key) + case itemKeyStart: + kname := p.expect(itemText) + p.currentKey = kname.val + p.approxLine = kname.line + + val, typ := p.value(p.next()) + p.setValue(p.currentKey, val) + p.setType(p.currentKey, typ) + p.ordered = append(p.ordered, p.context.add(p.currentKey)) + + p.currentKey = "" + default: + p.bug("Unexpected type at top level: %s", item.typ) + } +} + +// value translates an expected value from the lexer into a Go value wrapped +// as an empty interface. +func (p *parser) value(it item) (interface{}, tomlType) { + switch it.typ { + case itemString: + return p.replaceUnicode(replaceEscapes(it.val)), p.typeOfPrimitive(it) + case itemBool: + switch it.val { + case "true": + return true, p.typeOfPrimitive(it) + case "false": + return false, p.typeOfPrimitive(it) + } + p.bug("Expected boolean value, but got '%s'.", it.val) + case itemInteger: + num, err := strconv.ParseInt(it.val, 10, 64) + if err != nil { + if e, ok := err.(*strconv.NumError); ok && + e.Err == strconv.ErrRange { + + p.panic("Integer '%s' is out of the range of 64-bit "+ + "signed integers.", it.val) + } else { + p.bug("Expected integer value, but got '%s'.", it.val) + } + } + return num, p.typeOfPrimitive(it) + case itemFloat: + num, err := strconv.ParseFloat(it.val, 64) + if err != nil { + if e, ok := err.(*strconv.NumError); ok && + e.Err == strconv.ErrRange { + + p.panic("Float '%s' is out of the range of 64-bit "+ + "IEEE-754 floating-point numbers.", it.val) + } else { + p.bug("Expected float value, but got '%s'.", it.val) + } + } + return num, p.typeOfPrimitive(it) + case itemDatetime: + t, err := time.Parse("2006-01-02T15:04:05Z", it.val) + if err != nil { + p.bug("Expected Zulu formatted DateTime, but got '%s'.", it.val) + } + return t, p.typeOfPrimitive(it) + case itemArray: + array := make([]interface{}, 0) + types := make([]tomlType, 0) + + for it = p.next(); it.typ != itemArrayEnd; it = p.next() { + if it.typ == itemCommentStart { + p.expect(itemText) + continue + } + + val, typ := p.value(it) + array = append(array, val) + types = append(types, typ) + } + return array, p.typeOfArray(types) + } + p.bug("Unexpected value type: %s", it.typ) + panic("unreachable") +} + +// establishContext sets the current context of the parser, +// where the context is the hash currently in scope. +// +// Establishing the context also makes sure that the key isn't a duplicate, and +// will create implicit hashes automatically. +func (p *parser) establishContext(key Key, array bool) { + var ok bool + + // Always start at the top level and drill down for our context. + hashContext := p.mapping + keyContext := make(Key, 0) + + // We only need implicit hashes for key[0:-1] + for _, k := range key[0 : len(key)-1] { + _, ok = hashContext[k] + keyContext = append(keyContext, k) + + // No key? Make an implicit hash and move on. + if !ok { + p.addImplicit(keyContext) + hashContext[k] = make(map[string]interface{}) + } + + // If the hash context is actually an array of tables, then set + // the hash context to the last element in that array. + // + // Otherwise, it better be a table, since this MUST be a key group (by + // virtue of it not being the last element in a key). + switch t := hashContext[k].(type) { + case []map[string]interface{}: + hashContext = t[len(t)-1] + case map[string]interface{}: + hashContext = t + default: + p.panic("Key '%s' was already created as a hash.", keyContext) + } + } + + p.context = keyContext + if array { + k := key[len(key)-1] + if _, ok := hashContext[k]; !ok { + hashContext[k] = make([]map[string]interface{}, 0, 5) + } + if hash, ok := hashContext[k].([]map[string]interface{}); ok { + hashContext[k] = append(hash, make(map[string]interface{})) + } else { + p.panic("Key '%s' was already created and cannot be used as "+ + "an array.", keyContext) + } + } else { + p.setValue(key[len(key)-1], make(map[string]interface{})) + } + p.context = append(p.context, key[len(key)-1]) +} + +// setValue sets the given key to the given value in the current context. +// It will make sure that the key hasn't already been defined, account for +// implicit key groups. +func (p *parser) setValue(key string, value interface{}) { + var tmpHash interface{} + var ok bool + + hash := p.mapping + keyContext := make(Key, 0) + for _, k := range p.context { + keyContext = append(keyContext, k) + if tmpHash, ok = hash[k]; !ok { + p.bug("Context for key '%s' has not been established.", keyContext) + } + switch t := tmpHash.(type) { + case []map[string]interface{}: + hash = t[len(t)-1] + case map[string]interface{}: + hash = t + default: + p.bug("Expected hash to have type 'map[string]interface{}', but "+ + "it has '%T' instead.", tmpHash) + } + } + keyContext = append(keyContext, key) + + if _, ok := hash[key]; ok { + // We need to do some fancy footwork here. If `hash[key]` was implcitly + // created AND `value` is a hash, then let this go through and stop + // tagging this table as implicit. + if p.isImplicit(keyContext) { + p.removeImplicit(keyContext) + return + } + + // Otherwise, we have a concrete key trying to override a previous + // key, which is *always* wrong. + p.panic("Key '%s' has already been defined.", keyContext) + } + hash[key] = value +} + +// setType sets the type of a particular value at a given key. +// It should be called immediately AFTER setValue. +func (p *parser) setType(key string, typ tomlType) { + keyContext := make(Key, 0, len(p.context)+1) + for _, k := range p.context { + keyContext = append(keyContext, k) + } + if len(key) > 0 { // allow type setting for hashes + keyContext = append(keyContext, key) + } + p.types[keyContext.String()] = typ +} + +// addImplicit sets the given Key as having been created implicitly. +func (p *parser) addImplicit(key Key) { + p.implicits[key.String()] = true +} + +// removeImplicit stops tagging the given key as having been implicitly created. +func (p *parser) removeImplicit(key Key) { + p.implicits[key.String()] = false +} + +// isImplicit returns true if the key group pointed to by the key was created +// implicitly. +func (p *parser) isImplicit(key Key) bool { + return p.implicits[key.String()] +} + +// current returns the full key name of the current context. +func (p *parser) current() string { + if len(p.currentKey) == 0 { + return p.context.String() + } + if len(p.context) == 0 { + return p.currentKey + } + return fmt.Sprintf("%s.%s", p.context, p.currentKey) +} + +func replaceEscapes(s string) string { + return strings.NewReplacer( + "\\b", "\u0008", + "\\t", "\u0009", + "\\n", "\u000A", + "\\f", "\u000C", + "\\r", "\u000D", + "\\\"", "\u0022", + "\\/", "\u002F", + "\\\\", "\u005C", + ).Replace(s) +} + +func (p *parser) replaceUnicode(s string) string { + indexEsc := func() int { + return strings.Index(s, "\\u") + } + for i := indexEsc(); i != -1; i = indexEsc() { + asciiBytes := s[i+2 : i+6] + s = strings.Replace(s, s[i:i+6], p.asciiEscapeToUnicode(asciiBytes), -1) + } + return s +} + +func (p *parser) asciiEscapeToUnicode(s string) string { + hex, err := strconv.ParseUint(strings.ToLower(s), 16, 32) + if err != nil { + p.bug("Could not parse '%s' as a hexadecimal number, but the "+ + "lexer claims it's OK: %s", s, err) + } + + // I honestly don't understand how this works. I can't seem to find + // a way to make this fail. I figured this would fail on invalid UTF-8 + // characters like U+DCFF, but it doesn't. + r := string(rune(hex)) + if !utf8.ValidString(r) { + p.panic("Escaped character '\\u%s' is not valid UTF-8.", s) + } + return string(r) +} diff --git a/third_party/github.com/BurntSushi/toml/parse_test.go b/third_party/github.com/BurntSushi/toml/parse_test.go new file mode 100644 index 00000000000..d6f0d32fbd9 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/parse_test.go @@ -0,0 +1,61 @@ +package toml + +import ( + "strings" + "testing" +) + +var testParseSmall = ` +# This is a TOML document. Boom. + +wat = "chipper" + +[owner.andrew.gallant] +hmm = "hi" + +[owner] # Whoa there. +andreW = "gallant # poopy" # weeeee +predicate = false +num = -5192 +f = -0.5192 +zulu = 1979-05-27T07:32:00Z +whoop = "poop" +tests = [ [1, 2, 3], ["abc", "xyz"] ] +arrs = [ # hmm + # more comments are awesome. + 1987-07-05T05:45:00Z, + # say wat? + 1987-07-05T05:45:00Z, + 1987-07-05T05:45:00Z, + # sweetness +] # more comments +# hehe +` + +var testParseSmall2 = ` +[a] +better = 43 + +[a.b.c] +answer = 42 +` + +func TestParse(t *testing.T) { + m, err := parse(testParseSmall) + if err != nil { + t.Fatal(err) + } + printMap(m.mapping, 0) +} + +func printMap(m map[string]interface{}, depth int) { + for k, v := range m { + testf("%s%s\n", strings.Repeat(" ", depth), k) + switch subm := v.(type) { + case map[string]interface{}: + printMap(subm, depth+1) + default: + testf("%s%v\n", strings.Repeat(" ", depth+1), v) + } + } +} diff --git a/third_party/github.com/BurntSushi/toml/session.vim b/third_party/github.com/BurntSushi/toml/session.vim new file mode 100644 index 00000000000..562164be060 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/session.vim @@ -0,0 +1 @@ +au BufWritePost *.go silent!make tags > /dev/null 2>&1 diff --git a/third_party/github.com/BurntSushi/toml/toml-test-go/COPYING b/third_party/github.com/BurntSushi/toml/toml-test-go/COPYING new file mode 100644 index 00000000000..5a8e332545f --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/toml-test-go/COPYING @@ -0,0 +1,14 @@ + DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE + Version 2, December 2004 + + Copyright (C) 2004 Sam Hocevar + + Everyone is permitted to copy and distribute verbatim or modified + copies of this license document, and changing it is allowed as long + as the name is changed. + + DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. You just DO WHAT THE FUCK YOU WANT TO. + diff --git a/third_party/github.com/BurntSushi/toml/toml-test-go/README.md b/third_party/github.com/BurntSushi/toml/toml-test-go/README.md new file mode 100644 index 00000000000..24421eb7034 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/toml-test-go/README.md @@ -0,0 +1,14 @@ +# Implements the TOML test suite interface + +This is an implementation of the interface expected by +[toml-test](https://github.com/BurntSushi/toml-test) for my +[toml parser written in Go](https://github.com/BurntSushi/toml). +In particular, it maps TOML data on `stdin` to a JSON format on `stdout`. + + +Compatible with TOML version +[v0.2.0](https://github.com/mojombo/toml/blob/master/versions/toml-v0.2.0.md) + +Compatible with `toml-test` version +[v0.2.0](https://github.com/BurntSushi/toml-test/tree/v0.2.0) + diff --git a/third_party/github.com/BurntSushi/toml/toml-test-go/main.go b/third_party/github.com/BurntSushi/toml/toml-test-go/main.go new file mode 100644 index 00000000000..79a3c212f79 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/toml-test-go/main.go @@ -0,0 +1,89 @@ +package main + +import ( + "encoding/json" + "flag" + "fmt" + "log" + "os" + "path" + "time" + + "github.com/BurntSushi/toml" +) + +func init() { + log.SetFlags(0) + + flag.Usage = usage + flag.Parse() +} + +func usage() { + log.Printf("Usage: %s < toml-file\n", path.Base(os.Args[0])) + flag.PrintDefaults() + + os.Exit(1) +} + +func main() { + if flag.NArg() != 0 { + flag.Usage() + } + + var tmp interface{} + if _, err := toml.DecodeReader(os.Stdin, &tmp); err != nil { + log.Fatalf("Error decoding TOML: %s", err) + } + + typedTmp := translate(tmp) + if err := json.NewEncoder(os.Stdout).Encode(typedTmp); err != nil { + log.Fatalf("Error encoding JSON: %s", err) + } +} + +func translate(tomlData interface{}) interface{} { + + switch orig := tomlData.(type) { + case map[string]interface{}: + typed := make(map[string]interface{}, len(orig)) + for k, v := range orig { + typed[k] = translate(v) + } + return typed + case []map[string]interface{}: + typed := make([]map[string]interface{}, len(orig)) + for i, v := range orig { + typed[i] = translate(v).(map[string]interface{}) + } + return typed + case []interface{}: + typed := make([]interface{}, len(orig)) + for i, v := range orig { + typed[i] = translate(v) + } + + // We don't really need to tag arrays, but let's be future proof. + // (If TOML ever supports tuples, we'll need this.) + return tag("array", typed) + case time.Time: + return tag("datetime", orig.Format("2006-01-02T15:04:05Z")) + case bool: + return tag("bool", fmt.Sprintf("%v", orig)) + case int64: + return tag("integer", fmt.Sprintf("%d", orig)) + case float64: + return tag("float", fmt.Sprintf("%v", orig)) + case string: + return tag("string", orig) + } + + panic(fmt.Sprintf("Unknown type: %T", tomlData)) +} + +func tag(typeName string, data interface{}) map[string]interface{} { + return map[string]interface{}{ + "type": typeName, + "value": data, + } +} diff --git a/third_party/github.com/BurntSushi/toml/tomlv/COPYING b/third_party/github.com/BurntSushi/toml/tomlv/COPYING new file mode 100644 index 00000000000..5a8e332545f --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/tomlv/COPYING @@ -0,0 +1,14 @@ + DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE + Version 2, December 2004 + + Copyright (C) 2004 Sam Hocevar + + Everyone is permitted to copy and distribute verbatim or modified + copies of this license document, and changing it is allowed as long + as the name is changed. + + DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. You just DO WHAT THE FUCK YOU WANT TO. + diff --git a/third_party/github.com/BurntSushi/toml/tomlv/README.md b/third_party/github.com/BurntSushi/toml/tomlv/README.md new file mode 100644 index 00000000000..bcc3f47b1a4 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/tomlv/README.md @@ -0,0 +1,22 @@ +# TOML Validator + +If Go is installed, it's simple to try it out: + +```bash +go get github.com/BurntSushi/toml/tomlv +tomlv some-toml-file.toml +``` + +You can see the types of every key in a TOML file with: + +```bash +tomlv -types some-toml-file.toml +``` + +At the moment, only one error message is reported at a time. Error messages +include line numbers. No output means that the files given are valid TOML, or +there is a bug in `tomlv`. + +Compatible with TOML version +[v0.1.0](https://github.com/mojombo/toml/blob/master/versions/toml-v0.1.0.md) + diff --git a/third_party/github.com/BurntSushi/toml/tomlv/main.go b/third_party/github.com/BurntSushi/toml/tomlv/main.go new file mode 100644 index 00000000000..a52086ef5cf --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/tomlv/main.go @@ -0,0 +1,60 @@ +package main + +import ( + "flag" + "fmt" + "log" + "os" + "path" + "strings" + "text/tabwriter" + + "github.com/BurntSushi/toml" +) + +var ( + flagTypes = false +) + +func init() { + log.SetFlags(0) + + flag.BoolVar(&flagTypes, "types", flagTypes, + "When set, the types of every defined key will be shown.") + + flag.Usage = usage + flag.Parse() +} + +func usage() { + log.Printf("Usage: %s toml-file [ toml-file ... ]\n", + path.Base(os.Args[0])) + flag.PrintDefaults() + + os.Exit(1) +} + +func main() { + if flag.NArg() < 1 { + flag.Usage() + } + for _, f := range flag.Args() { + var tmp interface{} + md, err := toml.DecodeFile(f, &tmp) + if err != nil { + log.Fatalf("Error in '%s': %s", f, err) + } + if flagTypes { + printTypes(md) + } + } +} + +func printTypes(md toml.MetaData) { + tabw := tabwriter.NewWriter(os.Stdout, 0, 0, 2, ' ', 0) + for _, key := range md.Keys() { + fmt.Fprintf(tabw, "%s%s\t%s\n", + strings.Repeat(" ", len(key)-1), key, md.Type(key...)) + } + tabw.Flush() +} diff --git a/third_party/github.com/BurntSushi/toml/type-check.go b/third_party/github.com/BurntSushi/toml/type-check.go new file mode 100644 index 00000000000..22f188d4202 --- /dev/null +++ b/third_party/github.com/BurntSushi/toml/type-check.go @@ -0,0 +1,78 @@ +package toml + +// tomlType represents any Go type that corresponds to a TOML type. +// While the first draft of the TOML spec has a simplistic type system that +// probably doesn't need this level of sophistication, we seem to be militating +// toward adding real composite types. +type tomlType interface { + typeString() string +} + +// typeEqual accepts any two types and returns true if they are equal. +func typeEqual(t1, t2 tomlType) bool { + return t1.typeString() == t2.typeString() +} + +type tomlBaseType string + +func (btype tomlBaseType) typeString() string { + return string(btype) +} + +func (btype tomlBaseType) String() string { + return btype.typeString() +} + +var ( + tomlInteger tomlBaseType = "Integer" + tomlFloat tomlBaseType = "Float" + tomlDatetime tomlBaseType = "Datetime" + tomlString tomlBaseType = "String" + tomlBool tomlBaseType = "Bool" + tomlArray tomlBaseType = "Array" + tomlHash tomlBaseType = "Hash" + tomlArrayHash tomlBaseType = "ArrayHash" +) + +// typeOfPrimitive returns a tomlType of any primitive value in TOML. +// Primitive values are: Integer, Float, Datetime, String and Bool. +// +// Passing a lexer item other than the following will cause a BUG message +// to occur: itemString, itemBool, itemInteger, itemFloat, itemDatetime. +func (p *parser) typeOfPrimitive(lexItem item) tomlType { + switch lexItem.typ { + case itemInteger: + return tomlInteger + case itemFloat: + return tomlFloat + case itemDatetime: + return tomlDatetime + case itemString: + return tomlString + case itemBool: + return tomlBool + } + p.bug("Cannot infer primitive type of lex item '%s'.", lexItem) + panic("unreachable") +} + +// typeOfArray returns a tomlType for an array given a list of types of its +// values. +// +// In the current spec, if an array is homogeneous, then its type is always +// "Array". If the array is not homogeneous, an error is generated. +func (p *parser) typeOfArray(types []tomlType) tomlType { + // Empty arrays are cool. + if len(types) == 0 { + return tomlArray + } + + theType := types[0] + for _, t := range types[1:] { + if !typeEqual(theType, t) { + p.panic("Array contains values of type '%s' and '%s', but arrays "+ + "must be homogeneous.", theType, t) + } + } + return tomlArray +} diff --git a/util.go b/util.go deleted file mode 100644 index 089eba40cc2..00000000000 --- a/util.go +++ /dev/null @@ -1,88 +0,0 @@ -package main - -import ( - "net" - "net/url" - "os" - "os/signal" - "runtime/pprof" - - "github.com/coreos/etcd/log" -) - -//-------------------------------------- -// HTTP Utilities -//-------------------------------------- - -// sanitizeURL will cleanup a host string in the format hostname:port and -// attach a schema. -func sanitizeURL(host string, defaultScheme string) string { - // Blank URLs are fine input, just return it - if len(host) == 0 { - return host - } - - p, err := url.Parse(host) - if err != nil { - log.Fatal(err) - } - - // Make sure the host is in Host:Port format - _, _, err = net.SplitHostPort(host) - if err != nil { - log.Fatal(err) - } - - p = &url.URL{Host: host, Scheme: defaultScheme} - - return p.String() -} - -// sanitizeListenHost cleans up the ListenHost parameter and appends a port -// if necessary based on the advertised port. -func sanitizeListenHost(listen string, advertised string) string { - aurl, err := url.Parse(advertised) - if err != nil { - log.Fatal(err) - } - - ahost, aport, err := net.SplitHostPort(aurl.Host) - if err != nil { - log.Fatal(err) - } - - // If the listen host isn't set use the advertised host - if listen == "" { - listen = ahost - } - - return net.JoinHostPort(listen, aport) -} - -func check(err error) { - if err != nil { - log.Fatal(err) - } -} - -//-------------------------------------- -// CPU profile -//-------------------------------------- -func runCPUProfile() { - - f, err := os.Create(cpuprofile) - if err != nil { - log.Fatal(err) - } - pprof.StartCPUProfile(f) - - c := make(chan os.Signal, 1) - signal.Notify(c, os.Interrupt) - go func() { - for sig := range c { - log.Infof("captured %v, stopping profiler and exiting..", sig) - pprof.StopCPUProfile() - os.Exit(1) - } - }() -} From c88a556917419956c9c67ae08638ae2bf04a205a Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Wed, 30 Oct 2013 19:41:21 -0700 Subject: [PATCH 157/247] test add snapshot tests --- tests/functional/simple_snapshot_test.go | 89 ++++++++++++++++++++++++ 1 file changed, 89 insertions(+) create mode 100644 tests/functional/simple_snapshot_test.go diff --git a/tests/functional/simple_snapshot_test.go b/tests/functional/simple_snapshot_test.go new file mode 100644 index 00000000000..352dc3e62c8 --- /dev/null +++ b/tests/functional/simple_snapshot_test.go @@ -0,0 +1,89 @@ +package test + +import ( + "io/ioutil" + "os" + "testing" + "time" + + "github.com/coreos/go-etcd/etcd" +) + +// This test creates a single node and then set a value to it to trigger snapshot +func TestSimpleSnapshot(t *testing.T) { + procAttr := new(os.ProcAttr) + procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} + args := []string{"etcd", "-n=node1", "-d=/tmp/node1", "-snapshot=true", "-snapCount=500"} + + process, err := os.StartProcess(EtcdBinPath, append(args, "-f"), procAttr) + if err != nil { + t.Fatal("start process failed:" + err.Error()) + } + + time.Sleep(time.Second) + + c := etcd.NewClient(nil) + + c.SyncCluster() + // issue first 501 commands + for i := 0; i < 501; i++ { + result, err := c.Set("foo", "bar", 100) + + if err != nil || result.Key != "/foo" || result.Value != "bar" || result.TTL < 95 { + if err != nil { + t.Fatal(err) + } + + t.Fatalf("Set failed with %s %s %v", result.Key, result.Value, result.TTL) + } + } + + // wait for a snapshot interval + time.Sleep(3 * time.Second) + + snapshots, err := ioutil.ReadDir("/tmp/node1/snapshot") + + if err != nil { + t.Fatal("list snapshot failed:" + err.Error()) + } + + if len(snapshots) != 1 { + t.Fatal("wrong number of snapshot :[1/", len(snapshots), "]") + } + + if snapshots[0].Name() != "0_503.ss" { + t.Fatal("wrong name of snapshot :[0_503.ss/", snapshots[0].Name(), "]") + } + + // issue second 501 commands + for i := 0; i < 501; i++ { + result, err := c.Set("foo", "bar", 100) + + if err != nil || result.Key != "/foo" || result.Value != "bar" || result.TTL < 95 { + if err != nil { + t.Fatal(err) + } + + t.Fatalf("Set failed with %s %s %v", result.Key, result.Value, result.TTL) + } + } + + // wait for a snapshot interval + time.Sleep(3 * time.Second) + + snapshots, err = ioutil.ReadDir("/tmp/node1/snapshot") + + if err != nil { + t.Fatal("list snapshot failed:" + err.Error()) + } + + if len(snapshots) != 1 { + t.Fatal("wrong number of snapshot :[1/", len(snapshots), "]") + } + + if snapshots[0].Name() != "0_1004.ss" { + t.Fatal("wrong name of snapshot :[0_1004.ss/", snapshots[0].Name(), "]") + } + + process.Kill() +} From c10653ae0cf430633ee839be9e20c66a2dbd2a8d Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Thu, 31 Oct 2013 15:16:28 -0700 Subject: [PATCH 158/247] feat(etcd): use hostname as machine name by default I don't want etcd to require parameters so people can rapidly try it out. Try to use the hostname as the name instead. --- etcd.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/etcd.go b/etcd.go index 3c01356f182..c7a92042b5e 100644 --- a/etcd.go +++ b/etcd.go @@ -42,7 +42,12 @@ func main() { log.Fatal("info:", err) } if info.Name == "" { - log.Fatal("ERROR: server name required. e.g. '-n=server_name'") + host, err := os.Hostname() + if err != nil || host == "" { + log.Fatal("Machine name required and hostname not set. e.g. '-n=machine_name'") + } + log.Warnf("Using hostname %s as the machine name. You must ensure this name is unique among etcd machines.", host) + info.Name = host } // Retrieve TLS configuration. From ae615eda62ec067a6b08e59889677a328d2ca8ea Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Fri, 1 Nov 2013 15:56:38 -0700 Subject: [PATCH 159/247] fix typo --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 2034ab31464..adc0db9b283 100644 --- a/README.md +++ b/README.md @@ -156,7 +156,7 @@ In one terminal, we send a get request with `wait=true` : curl -L http://127.0.0.1:4001/v2/keys/foo?wait=true ``` -Now, we are waitting for any changes at path `/foo`. +Now, we are waiting for any changes at path `/foo`. In another terminal, we set a key `/foo` with value `bar`: From eb2fbcb575871e54accc664897c3d8e81eeb4bc8 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 2 Nov 2013 21:50:35 -0700 Subject: [PATCH 160/247] feat use-raft-protobuf --- server/peer_server_handlers.go | 135 ++++++++++++++++++++++----------- server/transporter.go | 85 ++++++++++++--------- 2 files changed, 139 insertions(+), 81 deletions(-) diff --git a/server/peer_server_handlers.go b/server/peer_server_handlers.go index be665dbf530..104447f90a2 100644 --- a/server/peer_server_handlers.go +++ b/server/peer_server_handlers.go @@ -23,72 +23,115 @@ func (ps *PeerServer) GetLogHttpHandler(w http.ResponseWriter, req *http.Request // Response to vote request func (ps *PeerServer) VoteHttpHandler(w http.ResponseWriter, req *http.Request) { rvreq := &raft.RequestVoteRequest{} - err := decodeJsonRequest(req, rvreq) - if err == nil { - log.Debugf("[recv] POST %s/vote [%s]", ps.url, rvreq.CandidateName) - if resp := ps.raftServer.RequestVote(rvreq); resp != nil { - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(resp) - return - } + + if _, err := rvreq.Decode(req.Body); err != nil { + http.Error(w, "", http.StatusBadRequest) + log.Warnf("[recv] BADREQUEST %s/vote [%v]", ps.url, err) + return + } + + log.Debugf("[recv] POST %s/vote [%s]", ps.url, rvreq.CandidateName) + + resp := ps.raftServer.RequestVote(rvreq) + + if resp == nil { + log.Warn("[vote] Error: nil response") + http.Error(w, "", http.StatusInternalServerError) + return + } + + if _, err := resp.Encode(w); err != nil { + log.Warn("[vote] Error: %v", err) + http.Error(w, "", http.StatusInternalServerError) + return } - log.Warnf("[vote] ERROR: %v", err) - w.WriteHeader(http.StatusInternalServerError) } // Response to append entries request func (ps *PeerServer) AppendEntriesHttpHandler(w http.ResponseWriter, req *http.Request) { aereq := &raft.AppendEntriesRequest{} - err := decodeJsonRequest(req, aereq) - if err == nil { - log.Debugf("[recv] POST %s/log/append [%d]", ps.url, len(aereq.Entries)) + if _, err := aereq.Decode(req.Body); err != nil { + http.Error(w, "", http.StatusBadRequest) + log.Warnf("[recv] BADREQUEST %s/log/append [%v]", ps.url, err) + return + } - ps.serverStats.RecvAppendReq(aereq.LeaderName, int(req.ContentLength)) + log.Debugf("[recv] POST %s/log/append [%d]", ps.url, len(aereq.Entries)) - if resp := ps.raftServer.AppendEntries(aereq); resp != nil { - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(resp) - if !resp.Success { - log.Debugf("[Append Entry] Step back") - } - return - } + ps.serverStats.RecvAppendReq(aereq.LeaderName, int(req.ContentLength)) + + resp := ps.raftServer.AppendEntries(aereq) + + if resp == nil { + log.Warn("[ae] Error: nil response") + http.Error(w, "", http.StatusInternalServerError) + return + } + + if !resp.Success { + log.Debugf("[Append Entry] Step back") + } + + if _, err := resp.Encode(w); err != nil { + log.Warn("[ae] Error: %v", err) + http.Error(w, "", http.StatusInternalServerError) + return } - log.Warnf("[Append Entry] ERROR: %v", err) - w.WriteHeader(http.StatusInternalServerError) } // Response to recover from snapshot request func (ps *PeerServer) SnapshotHttpHandler(w http.ResponseWriter, req *http.Request) { - aereq := &raft.SnapshotRequest{} - err := decodeJsonRequest(req, aereq) - if err == nil { - log.Debugf("[recv] POST %s/snapshot/ ", ps.url) - if resp := ps.raftServer.RequestSnapshot(aereq); resp != nil { - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(resp) - return - } + ssreq := &raft.SnapshotRequest{} + + if _, err := ssreq.Decode(req.Body); err != nil { + http.Error(w, "", http.StatusBadRequest) + log.Warnf("[recv] BADREQUEST %s/snapshot [%v]", ps.url, err) + return + } + + log.Debugf("[recv] POST %s/snapshot", ps.url) + + resp := ps.raftServer.RequestSnapshot(ssreq) + + if resp == nil { + log.Warn("[ss] Error: nil response") + http.Error(w, "", http.StatusInternalServerError) + return + } + + if _, err := resp.Encode(w); err != nil { + log.Warn("[ss] Error: %v", err) + http.Error(w, "", http.StatusInternalServerError) + return } - log.Warnf("[Snapshot] ERROR: %v", err) - w.WriteHeader(http.StatusInternalServerError) } // Response to recover from snapshot request func (ps *PeerServer) SnapshotRecoveryHttpHandler(w http.ResponseWriter, req *http.Request) { - aereq := &raft.SnapshotRecoveryRequest{} - err := decodeJsonRequest(req, aereq) - if err == nil { - log.Debugf("[recv] POST %s/snapshotRecovery/ ", ps.url) - if resp := ps.raftServer.SnapshotRecoveryRequest(aereq); resp != nil { - w.WriteHeader(http.StatusOK) - json.NewEncoder(w).Encode(resp) - return - } + ssrreq := &raft.SnapshotRecoveryRequest{} + + if _, err := ssrreq.Decode(req.Body); err != nil { + http.Error(w, "", http.StatusBadRequest) + log.Warnf("[recv] BADREQUEST %s/snapshotRecovery [%v]", ps.url, err) + return + } + + log.Debugf("[recv] POST %s/snapshotRecovery", ps.url) + + resp := ps.raftServer.SnapshotRecoveryRequest(ssrreq) + + if resp == nil { + log.Warn("[ssr] Error: nil response") + http.Error(w, "", http.StatusInternalServerError) + return + } + + if _, err := resp.Encode(w); err != nil { + log.Warn("[ssr] Error: %v", err) + http.Error(w, "", http.StatusInternalServerError) + return } - log.Warnf("[Snapshot] ERROR: %v", err) - w.WriteHeader(http.StatusInternalServerError) } // Get the port that listening for etcd connecting of the server diff --git a/server/transporter.go b/server/transporter.go index 95545a82752..d844dc33eae 100644 --- a/server/transporter.go +++ b/server/transporter.go @@ -3,7 +3,6 @@ package server import ( "bytes" "crypto/tls" - "encoding/json" "fmt" "io" "net" @@ -65,10 +64,12 @@ func dialWithTimeout(network, addr string) (net.Conn, error) { // Sends AppendEntries RPCs to a peer when the server is the leader. func (t *transporter) SendAppendEntriesRequest(server raft.Server, peer *raft.Peer, req *raft.AppendEntriesRequest) *raft.AppendEntriesResponse { - var aersp *raft.AppendEntriesResponse var b bytes.Buffer - json.NewEncoder(&b).Encode(req) + if _, err := req.Encode(&b); err != nil { + log.Warn("transporter.ae.encoding.error:", err) + return nil + } size := b.Len() @@ -97,6 +98,7 @@ func (t *transporter) SendAppendEntriesRequest(server raft.Server, peer *raft.Pe if ok { thisFollowerStats.Fail() } + return nil } else { if ok { thisFollowerStats.Succ(end.Sub(start)) @@ -108,21 +110,25 @@ func (t *transporter) SendAppendEntriesRequest(server raft.Server, peer *raft.Pe t.CancelWhenTimeout(httpRequest) - aersp = &raft.AppendEntriesResponse{} - if err := json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { - return aersp + aeresp := &raft.AppendEntriesResponse{} + if _, err = aeresp.Decode(resp.Body); err != nil && err != io.EOF { + log.Warn("transporter.ae.decoding.error:", err) + return nil } - + return aeresp } - return aersp + return nil } // Sends RequestVote RPCs to a peer when the server is the candidate. func (t *transporter) SendVoteRequest(server raft.Server, peer *raft.Peer, req *raft.RequestVoteRequest) *raft.RequestVoteResponse { - var rvrsp *raft.RequestVoteResponse var b bytes.Buffer - json.NewEncoder(&b).Encode(req) + + if _, err := req.Encode(&b); err != nil { + log.Warn("transporter.vr.encoding.error:", err) + return nil + } u, _ := t.peerServer.registry.PeerURL(peer.Name) log.Debugf("Send Vote from %s to %s", server.Name(), u) @@ -139,28 +145,31 @@ func (t *transporter) SendVoteRequest(server raft.Server, peer *raft.Peer, req * t.CancelWhenTimeout(httpRequest) rvrsp := &raft.RequestVoteResponse{} - if err := json.NewDecoder(resp.Body).Decode(&rvrsp); err == nil || err == io.EOF { - return rvrsp + if _, err = rvrsp.Decode(resp.Body); err != nil && err != io.EOF { + log.Warn("transporter.vr.decoding.error:", err) + return nil } - + return rvrsp } - return rvrsp + return nil } // Sends SnapshotRequest RPCs to a peer when the server is the candidate. func (t *transporter) SendSnapshotRequest(server raft.Server, peer *raft.Peer, req *raft.SnapshotRequest) *raft.SnapshotResponse { - var aersp *raft.SnapshotResponse var b bytes.Buffer - json.NewEncoder(&b).Encode(req) + + if _, err := req.Encode(&b); err != nil { + log.Warn("transporter.ss.encoding.error:", err) + return nil + } u, _ := t.peerServer.registry.PeerURL(peer.Name) - log.Debugf("Send Snapshot to %s [Last Term: %d, LastIndex %d]", u, - req.LastTerm, req.LastIndex) + log.Debugf("Send Snapshot Request from %s to %s", server.Name(), u) resp, httpRequest, err := t.Post(fmt.Sprintf("%s/snapshot", u), &b) if err != nil { - log.Debugf("Cannot send SendSnapshotRequest to %s : %s", u, err) + log.Debugf("Cannot send Snapshot Request to %s : %s", u, err) } if resp != nil { @@ -168,42 +177,48 @@ func (t *transporter) SendSnapshotRequest(server raft.Server, peer *raft.Peer, r t.CancelWhenTimeout(httpRequest) - aersp = &raft.SnapshotResponse{} - if err = json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { - - return aersp + ssrsp := &raft.SnapshotResponse{} + if _, err = ssrsp.Decode(resp.Body); err != nil && err != io.EOF { + log.Warn("transporter.ss.decoding.error:", err) + return nil } + return ssrsp } - - return aersp + return nil } // Sends SnapshotRecoveryRequest RPCs to a peer when the server is the candidate. func (t *transporter) SendSnapshotRecoveryRequest(server raft.Server, peer *raft.Peer, req *raft.SnapshotRecoveryRequest) *raft.SnapshotRecoveryResponse { - var aersp *raft.SnapshotRecoveryResponse var b bytes.Buffer - json.NewEncoder(&b).Encode(req) + + if _, err := req.Encode(&b); err != nil { + log.Warn("transporter.ss.encoding.error:", err) + return nil + } u, _ := t.peerServer.registry.PeerURL(peer.Name) - log.Debugf("Send SnapshotRecovery to %s [Last Term: %d, LastIndex %d]", u, - req.LastTerm, req.LastIndex) + log.Debugf("Send Snapshot Recovery from %s to %s", server.Name(), u) - resp, _, err := t.Post(fmt.Sprintf("%s/snapshotRecovery", u), &b) + resp, httpRequest, err := t.Post(fmt.Sprintf("%s/snapshotRecovery", u), &b) if err != nil { - log.Debugf("Cannot send SendSnapshotRecoveryRequest to %s : %s", u, err) + log.Debugf("Cannot send Snapshot Recovery to %s : %s", u, err) } if resp != nil { defer resp.Body.Close() - aersp = &raft.SnapshotRecoveryResponse{} - if err = json.NewDecoder(resp.Body).Decode(&aersp); err == nil || err == io.EOF { - return aersp + t.CancelWhenTimeout(httpRequest) + + ssrrsp := &raft.SnapshotRecoveryResponse{} + if _, err = ssrrsp.Decode(resp.Body); err != nil && err != io.EOF { + log.Warn("transporter.ssr.decoding.error:", err) + return nil } + return ssrrsp } + return nil - return aersp } // Send server side POST request From f3ce75f2b4c90183942404f7a5bcd3d1c5ab59b0 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 2 Nov 2013 22:05:44 -0700 Subject: [PATCH 161/247] feat add a simple speed test --- server/server.go | 1 + 1 file changed, 1 insertion(+) diff --git a/server/server.go b/server/server.go index b88d5ac0ab3..48757ad2ba5 100644 --- a/server/server.go +++ b/server/server.go @@ -116,6 +116,7 @@ func (s *Server) installV2() { s.handleFunc("/v2/stats/self", s.GetStatsHandler).Methods("GET") s.handleFunc("/v2/stats/leader", s.GetLeaderStatsHandler).Methods("GET") s.handleFunc("/v2/stats/store", s.GetStoreStatsHandler).Methods("GET") + s.handleFunc("/v2/speedTest", s.SpeedTestHandler).Methods("GET") } // Adds a v1 server handler to the router. From 02abbb6a6c7a7aef6e4ca4971fbbe00e841bd059 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sun, 3 Nov 2013 15:49:00 -0700 Subject: [PATCH 162/247] Migration test. --- release_version.go | 2 + tests/fixtures/v1/README | 14 +++++++ tests/fixtures/v1/complete.1.sh | 4 ++ tests/fixtures/v1/complete.2.sh | 3 ++ tests/fixtures/v1/complete.3.sh | 3 ++ tests/fixtures/v1/complete.4.sh | 12 ++++++ tests/fixtures/v1/complete/conf | 1 + tests/fixtures/v1/complete/info | 18 +++++++++ tests/fixtures/v1/complete/log | Bin 0 -> 1289 bytes tests/functional/v1_migration_test.go | 51 ++++++++++++++++++++++++++ 10 files changed, 108 insertions(+) create mode 100644 release_version.go create mode 100644 tests/fixtures/v1/README create mode 100644 tests/fixtures/v1/complete.1.sh create mode 100644 tests/fixtures/v1/complete.2.sh create mode 100644 tests/fixtures/v1/complete.3.sh create mode 100644 tests/fixtures/v1/complete.4.sh create mode 100644 tests/fixtures/v1/complete/conf create mode 100644 tests/fixtures/v1/complete/info create mode 100644 tests/fixtures/v1/complete/log create mode 100644 tests/functional/v1_migration_test.go diff --git a/release_version.go b/release_version.go new file mode 100644 index 00000000000..5285c9763c0 --- /dev/null +++ b/release_version.go @@ -0,0 +1,2 @@ +package main +const releaseVersion = "v0.1.2-33-g1a2a9d6" diff --git a/tests/fixtures/v1/README b/tests/fixtures/v1/README new file mode 100644 index 00000000000..0940522bb54 --- /dev/null +++ b/tests/fixtures/v1/README @@ -0,0 +1,14 @@ +README + +The scripts in this directory should be run from the project root: + +$ cd $GOPATH/src/github.com/coreos/etcd +$ tests/fixtures/v1/complete.1.sh + +Scripts with numbers should be run in separate terminal windows (in order): + +$ tests/fixtures/v1/complete.1.sh +$ tests/fixtures/v1/complete.2.sh +$ tests/fixtures/v1/complete.3.sh + +The resulting server state data can be found in tmp/node0. diff --git a/tests/fixtures/v1/complete.1.sh b/tests/fixtures/v1/complete.1.sh new file mode 100644 index 00000000000..ee77deaed26 --- /dev/null +++ b/tests/fixtures/v1/complete.1.sh @@ -0,0 +1,4 @@ +#!/bin/sh + +./build +./etcd -d tmp/node0 -n node0 diff --git a/tests/fixtures/v1/complete.2.sh b/tests/fixtures/v1/complete.2.sh new file mode 100644 index 00000000000..1b067eb2b83 --- /dev/null +++ b/tests/fixtures/v1/complete.2.sh @@ -0,0 +1,3 @@ +#!/bin/sh + +./etcd -s 127.0.0.1:7002 -c 127.0.0.1:4002 -C 127.0.0.1:7001 -d tmp/node2 -n node2 diff --git a/tests/fixtures/v1/complete.3.sh b/tests/fixtures/v1/complete.3.sh new file mode 100644 index 00000000000..a1c9c6b3e56 --- /dev/null +++ b/tests/fixtures/v1/complete.3.sh @@ -0,0 +1,3 @@ +#!/bin/sh + +./etcd -s 127.0.0.1:7003 -c 127.0.0.1:4003 -C 127.0.0.1:7001 -d tmp/node3 -n node3 diff --git a/tests/fixtures/v1/complete.4.sh b/tests/fixtures/v1/complete.4.sh new file mode 100644 index 00000000000..b228b1a1d7d --- /dev/null +++ b/tests/fixtures/v1/complete.4.sh @@ -0,0 +1,12 @@ +#!/bin/sh + +curl -L http://127.0.0.1:4001/v1/keys/message -d value="Hello world" +curl -L http://127.0.0.1:4001/v1/keys/message -d value="Hello etcd" +curl -L http://127.0.0.1:4001/v1/keys/message -X DELETE +curl -L http://127.0.0.1:4001/v1/keys/foo -d value=bar -d ttl=5 +curl -L http://127.0.0.1:4001/v1/keys/foo -d value=one +curl -L http://127.0.0.1:4001/v1/keys/foo -d prevValue=two -d value=three +curl -L http://127.0.0.1:4001/v1/keys/foo -d prevValue=one -d value=two +curl -L http://127.0.0.1:4001/v1/keys/bar -d prevValue= -d value=four +curl -L http://127.0.0.1:4001/v1/keys/bar -d prevValue= -d value=five +curl -X DELETE http://127.0.0.1:4001/v1/keys/_etcd/machines diff --git a/tests/fixtures/v1/complete/conf b/tests/fixtures/v1/complete/conf new file mode 100644 index 00000000000..827bb86d1b3 --- /dev/null +++ b/tests/fixtures/v1/complete/conf @@ -0,0 +1 @@ +{"commitIndex":13,"peers":[]} \ No newline at end of file diff --git a/tests/fixtures/v1/complete/info b/tests/fixtures/v1/complete/info new file mode 100644 index 00000000000..398c8e1e80a --- /dev/null +++ b/tests/fixtures/v1/complete/info @@ -0,0 +1,18 @@ +{ + "name": "node0", + "raftURL": "http://127.0.0.1:7001", + "etcdURL": "http://127.0.0.1:4001", + "webURL": "", + "raftListenHost": "127.0.0.1:7001", + "etcdListenHost": "127.0.0.1:4001", + "raftTLS": { + "CertFile": "", + "KeyFile": "", + "CAFile": "" + }, + "etcdTLS": { + "CertFile": "", + "KeyFile": "", + "CAFile": "" + } +} diff --git a/tests/fixtures/v1/complete/log b/tests/fixtures/v1/complete/log new file mode 100644 index 0000000000000000000000000000000000000000..5ed55603d845adc26e3651e53fabe6cc9cf59427 GIT binary patch literal 1289 zcmbu9&rjPh6vr8C!k^$hq!loeclV)6;9^=T3bHjLlU+JhL@m6sprNRgTEQ#n+#9v+u@;u;d z6%{dBGA`_JGHeuQYcyO7t-y(&V{r#hNE|>_CD(wAQIg-2sw2pA>pj>qPl^0E%t|=V zYQd^!dcHz8>Rd&!l=%3PUS0$k$=Syl(#dNi($pTLL(V=SXtTDILJkhZw|UkyFJYgv z&oi=qOhfzDHq8G?%6TnXs=!yWHXX)Y69-t(gFtsf6Gpx?P5U9LTA$$@F6P*RQrR)Tm4l4J(NYAjEzZXN?O$mfKDPeT^TwV7-c3XlCZmW$&;9pC?HKd7slV5Yi z8DE6Gm|`d8n(o}by{GCmxo_}_(9Jf`Y&4(a_E731Ul979PNEU}|4u>?v~`wu68& Date: Sun, 3 Nov 2013 15:58:16 -0700 Subject: [PATCH 163/247] Update migration. --- tests/fixtures/v1/README | 1 + tests/fixtures/v1/complete/conf | 2 +- tests/fixtures/v1/complete/log | Bin 1289 -> 1643 bytes 3 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/fixtures/v1/README b/tests/fixtures/v1/README index 0940522bb54..8e24d1bc6b8 100644 --- a/tests/fixtures/v1/README +++ b/tests/fixtures/v1/README @@ -10,5 +10,6 @@ Scripts with numbers should be run in separate terminal windows (in order): $ tests/fixtures/v1/complete.1.sh $ tests/fixtures/v1/complete.2.sh $ tests/fixtures/v1/complete.3.sh +$ tests/fixtures/v1/complete.4.sh The resulting server state data can be found in tmp/node0. diff --git a/tests/fixtures/v1/complete/conf b/tests/fixtures/v1/complete/conf index 827bb86d1b3..2c5bb4e812f 100644 --- a/tests/fixtures/v1/complete/conf +++ b/tests/fixtures/v1/complete/conf @@ -1 +1 @@ -{"commitIndex":13,"peers":[]} \ No newline at end of file +{"commitIndex":16,"peers":[{"name":"node3","connectionString":""}]} \ No newline at end of file diff --git a/tests/fixtures/v1/complete/log b/tests/fixtures/v1/complete/log index 5ed55603d845adc26e3651e53fabe6cc9cf59427..72bda1d35f6abf4cca845fd6b57c9b997c901b41 100644 GIT binary patch delta 351 zcmeC=dd)Mzh|5wpJ=N0GFv-$#qD?WYg(VjU^F(fSB;m z%WSJ{plfUxVrXh*VrFG*sApthY-D0;W@@f$U}j}tpj6AH00yR}Tpa9^mompPa!h7s zsbJ)sT*Fev$Tc~GRcEq1tH)#}7SYKytTC(>mRuY>6S>uq-S9Mq)yRa4gLkqgn-TyU C@Ff)h From 44356dc654f00252b935323cf9769f7c0c4574f2 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Sun, 3 Nov 2013 16:02:59 -0700 Subject: [PATCH 164/247] Update migration script. --- tests/fixtures/v1/complete.4.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/fixtures/v1/complete.4.sh b/tests/fixtures/v1/complete.4.sh index b228b1a1d7d..15c756eb091 100644 --- a/tests/fixtures/v1/complete.4.sh +++ b/tests/fixtures/v1/complete.4.sh @@ -3,10 +3,11 @@ curl -L http://127.0.0.1:4001/v1/keys/message -d value="Hello world" curl -L http://127.0.0.1:4001/v1/keys/message -d value="Hello etcd" curl -L http://127.0.0.1:4001/v1/keys/message -X DELETE -curl -L http://127.0.0.1:4001/v1/keys/foo -d value=bar -d ttl=5 +curl -L http://127.0.0.1:4001/v1/keys/message2 -d value="Hola" +curl -L http://127.0.0.1:4001/v1/keys/expiring -d value=bar -d ttl=5 curl -L http://127.0.0.1:4001/v1/keys/foo -d value=one curl -L http://127.0.0.1:4001/v1/keys/foo -d prevValue=two -d value=three curl -L http://127.0.0.1:4001/v1/keys/foo -d prevValue=one -d value=two curl -L http://127.0.0.1:4001/v1/keys/bar -d prevValue= -d value=four curl -L http://127.0.0.1:4001/v1/keys/bar -d prevValue= -d value=five -curl -X DELETE http://127.0.0.1:4001/v1/keys/_etcd/machines +curl -X DELETE http://127.0.0.1:7001/remove/node2 From 52d5e052012ad536c90087d63d03623c8e88fa77 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 3 Nov 2013 20:59:08 -0800 Subject: [PATCH 165/247] refactor use defer --- tests/functional/simple_snapshot_test.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/functional/simple_snapshot_test.go b/tests/functional/simple_snapshot_test.go index 352dc3e62c8..78fa90b5f78 100644 --- a/tests/functional/simple_snapshot_test.go +++ b/tests/functional/simple_snapshot_test.go @@ -19,6 +19,7 @@ func TestSimpleSnapshot(t *testing.T) { if err != nil { t.Fatal("start process failed:" + err.Error()) } + defer process.Kill() time.Sleep(time.Second) @@ -84,6 +85,4 @@ func TestSimpleSnapshot(t *testing.T) { if snapshots[0].Name() != "0_1004.ss" { t.Fatal("wrong name of snapshot :[0_1004.ss/", snapshots[0].Name(), "]") } - - process.Kill() } From 235ffc234de42b9eac7f8f94aa11a04dc1bb9abf Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 3 Nov 2013 21:06:51 -0800 Subject: [PATCH 166/247] refactor change cmd argument snapCount to snapshotCount --- server/config.go | 4 ++-- tests/functional/simple_snapshot_test.go | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/server/config.go b/server/config.go index 95ec59737d0..df87b98946c 100644 --- a/server/config.go +++ b/server/config.go @@ -37,7 +37,7 @@ type Config struct { MaxRetryAttempts int `toml:"max_retry_attempts" env:"ETCD_MAX_RETRY_ATTEMPTS"` Name string `toml:"name" env:"ETCD_NAME"` Snapshot bool `toml:"snapshot" env:"ETCD_SNAPSHOT"` - SnapCount int `toml:"snap_count" env:"ETCD_SNAPCOUNT"` + SnapCount int `toml:"snapshot_count" env:"ETCD_SNAPSHOTCOUNT"` Verbose bool `toml:"verbose" env:"ETCD_VERBOSE"` VeryVerbose bool `toml:"very_verbose" env:"ETCD_VERY_VERBOSE"` WebURL string `toml:"web_url" env:"ETCD_WEB_URL"` @@ -203,7 +203,7 @@ func (c *Config) LoadFlags(arguments []string) error { f.StringVar(&cors, "cors", "", "whitelist origins for cross-origin resource sharing (e.g. '*' or 'http://localhost:8001,etc')") f.BoolVar(&c.Snapshot, "snapshot", c.Snapshot, "open or close snapshot") - f.IntVar(&c.SnapCount, "snapCount", c.SnapCount, "save the in memory logs and states to a snapshot file after snapCount transactions") + f.IntVar(&c.SnapCount, "snapshotCount", c.SnapCount, "save the in memory logs and states to a snapshot file after snapCount transactions") // These flags are ignored since they were already parsed. var path string diff --git a/tests/functional/simple_snapshot_test.go b/tests/functional/simple_snapshot_test.go index 78fa90b5f78..e7cce08cc04 100644 --- a/tests/functional/simple_snapshot_test.go +++ b/tests/functional/simple_snapshot_test.go @@ -13,7 +13,7 @@ import ( func TestSimpleSnapshot(t *testing.T) { procAttr := new(os.ProcAttr) procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} - args := []string{"etcd", "-n=node1", "-d=/tmp/node1", "-snapshot=true", "-snapCount=500"} + args := []string{"etcd", "-n=node1", "-d=/tmp/node1", "-snapshot=true", "-snapshotCount=500"} process, err := os.StartProcess(EtcdBinPath, append(args, "-f"), procAttr) if err != nil { From 5394c8a894b2934b064b6eaf1b612290effc36a5 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 3 Nov 2013 23:34:47 -0800 Subject: [PATCH 167/247] feat add ttl heap --- store/heap_test.go | 36 +++++++++++++++++++++++++++++++ store/ttl_key_heap.go | 49 +++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 85 insertions(+) create mode 100644 store/heap_test.go create mode 100644 store/ttl_key_heap.go diff --git a/store/heap_test.go b/store/heap_test.go new file mode 100644 index 00000000000..02682b97395 --- /dev/null +++ b/store/heap_test.go @@ -0,0 +1,36 @@ +package store + +import ( + "container/heap" + "fmt" + "testing" + "time" +) + +func TestHeapPushPop(t *testing.T) { + h := &TTLKeyHeap{Map: make(map[*Node]int)} + heap.Init(h) + + kvs := make([]*Node, 10) + + // add from older expire time to earlier expire time + // the path is equal to ttl from now + for i, n := range kvs { + path := fmt.Sprintf("%v", 10-i) + m := time.Duration(10 - i) + n = newKV(nil, path, path, 0, 0, nil, "", time.Now().Add(time.Second*m)) + heap.Push(h, n) + } + + min := time.Now() + + for i := 0; i < 9; i++ { + iNode := heap.Pop(h) + node, _ := iNode.(*Node) + if node.ExpireTime.Before(min) { + t.Fatal("heap sort wrong!") + } + min = node.ExpireTime + } + +} diff --git a/store/ttl_key_heap.go b/store/ttl_key_heap.go new file mode 100644 index 00000000000..abb12fede52 --- /dev/null +++ b/store/ttl_key_heap.go @@ -0,0 +1,49 @@ +package store + +import ( + "container/heap" +) + +// An TTLKeyHeap is a min-heap of TTLKeys order by expiration time +type TTLKeyHeap struct { + Array []*Node + Map map[*Node]int +} + +func (h TTLKeyHeap) Len() int { + return len(h.Array) +} + +func (h TTLKeyHeap) Less(i, j int) bool { + return h.Array[i].ExpireTime.Before(h.Array[j].ExpireTime) +} + +func (h TTLKeyHeap) Swap(i, j int) { + // swap node + h.Array[i], h.Array[j] = h.Array[j], h.Array[i] + + // update map + h.Map[h.Array[i]] = i + h.Map[h.Array[j]] = j +} + +func (h *TTLKeyHeap) Push(x interface{}) { + n, _ := x.(*Node) + h.Map[n] = len(h.Array) + h.Array = append(h.Array, n) +} + +func (h *TTLKeyHeap) Pop() interface{} { + old := h.Array + n := len(old) + x := old[n-1] + h.Array = old[0 : n-1] + delete(h.Map, x) + return x +} + +func (h *TTLKeyHeap) Update(n *Node) { + index := h.Map[n] + heap.Remove(h, index) + heap.Push(h, n) +} From b9593b80ecd569453784e05481e944d180e344c1 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sun, 3 Nov 2013 23:51:48 -0800 Subject: [PATCH 168/247] feat add update heap test --- store/heap_test.go | 51 +++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 50 insertions(+), 1 deletion(-) diff --git a/store/heap_test.go b/store/heap_test.go index 02682b97395..2ae573724dd 100644 --- a/store/heap_test.go +++ b/store/heap_test.go @@ -11,6 +11,32 @@ func TestHeapPushPop(t *testing.T) { h := &TTLKeyHeap{Map: make(map[*Node]int)} heap.Init(h) + // add from older expire time to earlier expire time + // the path is equal to ttl from now + for i := 0; i < 10; i++ { + path := fmt.Sprintf("%v", 10-i) + m := time.Duration(10 - i) + n := newKV(nil, path, path, 0, 0, nil, "", time.Now().Add(time.Second*m)) + heap.Push(h, n) + } + + min := time.Now() + + for i := 0; i < 10; i++ { + iNode := heap.Pop(h) + node, _ := iNode.(*Node) + if node.ExpireTime.Before(min) { + t.Fatal("heap sort wrong!") + } + min = node.ExpireTime + } + +} + +func TestHeapUpdate(t *testing.T) { + h := &TTLKeyHeap{Map: make(map[*Node]int)} + heap.Init(h) + kvs := make([]*Node, 10) // add from older expire time to earlier expire time @@ -19,18 +45,41 @@ func TestHeapPushPop(t *testing.T) { path := fmt.Sprintf("%v", 10-i) m := time.Duration(10 - i) n = newKV(nil, path, path, 0, 0, nil, "", time.Now().Add(time.Second*m)) + kvs[i] = n heap.Push(h, n) } + // Path 7 + kvs[3].ExpireTime = time.Now().Add(time.Second * 11) + + // Path 5 + kvs[5].ExpireTime = time.Now().Add(time.Second * 12) + + h.Update(kvs[3]) + h.Update(kvs[5]) + min := time.Now() - for i := 0; i < 9; i++ { + for i := 0; i < 10; i++ { iNode := heap.Pop(h) node, _ := iNode.(*Node) if node.ExpireTime.Before(min) { t.Fatal("heap sort wrong!") } min = node.ExpireTime + + if i == 8 { + if node.Path != "7" { + t.Fatal("heap sort wrong!", node.Path) + } + } + + if i == 9 { + if node.Path != "5" { + t.Fatal("heap sort wrong!") + } + } + } } From bf76af8fd1ae74f16cfa055ee7328638da2c5bb7 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 4 Nov 2013 13:16:45 -0700 Subject: [PATCH 169/247] Update migration. --- tests/fixtures/v1/README | 2 +- tests/fixtures/v1/complete/conf | 1 - tests/fixtures/v1/complete/info | 18 -------- tests/fixtures/v1/complete/log | Bin 1643 -> 0 bytes tests/fixtures/v1/{complete.1.sh => run.1.sh} | 0 tests/fixtures/v1/{complete.2.sh => run.2.sh} | 0 tests/fixtures/v1/{complete.3.sh => run.3.sh} | 0 tests/fixtures/v1/{complete.4.sh => run.4.sh} | 0 tests/functional/v1_migration_test.go | 39 +++++++++++------- 9 files changed, 24 insertions(+), 36 deletions(-) delete mode 100644 tests/fixtures/v1/complete/conf delete mode 100644 tests/fixtures/v1/complete/info delete mode 100644 tests/fixtures/v1/complete/log rename tests/fixtures/v1/{complete.1.sh => run.1.sh} (100%) rename tests/fixtures/v1/{complete.2.sh => run.2.sh} (100%) rename tests/fixtures/v1/{complete.3.sh => run.3.sh} (100%) rename tests/fixtures/v1/{complete.4.sh => run.4.sh} (100%) diff --git a/tests/fixtures/v1/README b/tests/fixtures/v1/README index 8e24d1bc6b8..e930eda7d70 100644 --- a/tests/fixtures/v1/README +++ b/tests/fixtures/v1/README @@ -12,4 +12,4 @@ $ tests/fixtures/v1/complete.2.sh $ tests/fixtures/v1/complete.3.sh $ tests/fixtures/v1/complete.4.sh -The resulting server state data can be found in tmp/node0. +The resulting server state data can be found in tmp/node*. diff --git a/tests/fixtures/v1/complete/conf b/tests/fixtures/v1/complete/conf deleted file mode 100644 index 2c5bb4e812f..00000000000 --- a/tests/fixtures/v1/complete/conf +++ /dev/null @@ -1 +0,0 @@ -{"commitIndex":16,"peers":[{"name":"node3","connectionString":""}]} \ No newline at end of file diff --git a/tests/fixtures/v1/complete/info b/tests/fixtures/v1/complete/info deleted file mode 100644 index 398c8e1e80a..00000000000 --- a/tests/fixtures/v1/complete/info +++ /dev/null @@ -1,18 +0,0 @@ -{ - "name": "node0", - "raftURL": "http://127.0.0.1:7001", - "etcdURL": "http://127.0.0.1:4001", - "webURL": "", - "raftListenHost": "127.0.0.1:7001", - "etcdListenHost": "127.0.0.1:4001", - "raftTLS": { - "CertFile": "", - "KeyFile": "", - "CAFile": "" - }, - "etcdTLS": { - "CertFile": "", - "KeyFile": "", - "CAFile": "" - } -} diff --git a/tests/fixtures/v1/complete/log b/tests/fixtures/v1/complete/log deleted file mode 100644 index 72bda1d35f6abf4cca845fd6b57c9b997c901b41..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1643 zcmbuAO>dh(5Qh8R&8BVA9D72X3)y8^Fx+~GRH>DcG)}4>aacRFn6+Tzq_XSwj7JPIO*gX6rT*_u($r(_>k^VM0Zv4#TP&z64<_`UoK{4%cR diff --git a/tests/fixtures/v1/complete.1.sh b/tests/fixtures/v1/run.1.sh similarity index 100% rename from tests/fixtures/v1/complete.1.sh rename to tests/fixtures/v1/run.1.sh diff --git a/tests/fixtures/v1/complete.2.sh b/tests/fixtures/v1/run.2.sh similarity index 100% rename from tests/fixtures/v1/complete.2.sh rename to tests/fixtures/v1/run.2.sh diff --git a/tests/fixtures/v1/complete.3.sh b/tests/fixtures/v1/run.3.sh similarity index 100% rename from tests/fixtures/v1/complete.3.sh rename to tests/fixtures/v1/run.3.sh diff --git a/tests/fixtures/v1/complete.4.sh b/tests/fixtures/v1/run.4.sh similarity index 100% rename from tests/fixtures/v1/complete.4.sh rename to tests/fixtures/v1/run.4.sh diff --git a/tests/functional/v1_migration_test.go b/tests/functional/v1_migration_test.go index f6f892287b6..f5d76fee564 100644 --- a/tests/functional/v1_migration_test.go +++ b/tests/functional/v1_migration_test.go @@ -5,6 +5,7 @@ import ( "io/ioutil" "os" "os/exec" + "path/filepath" "testing" "time" @@ -18,23 +19,30 @@ func TestV1Migration(t *testing.T) { os.RemoveAll(path) defer os.RemoveAll(path) - // Copy over fixture files. - if err := exec.Command("cp", "-r", "../fixtures/v1/complete", path).Run(); err != nil { - panic("Fixture initialization error") + nodes := []string{"node0", "node1"} + for i, node := range nodes { + nodepath := filepath.Join(path, node) + + // Copy over fixture files. + if err := exec.Command("cp", "-r", "../fixtures/v1/" + node, nodepath).Run(); err != nil { + panic("Fixture initialization error") + } + + procAttr := new(os.ProcAttr) + procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} + + args := []string{"etcd", fmt.Sprintf("-d=%s", nodepath)} + args = append(args, "-c", fmt.Sprintf("127.0.0.1:%d", 4001 + i)) + args = append(args, "-s", fmt.Sprintf("127.0.0.1:%d", 7001 + i)) + process, err := os.StartProcess(EtcdBinPath, args, procAttr) + if err != nil { + t.Fatal("start process failed:" + err.Error()) + return + } + defer process.Kill() + time.Sleep(time.Second) } - procAttr := new(os.ProcAttr) - procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} - args := []string{"etcd", fmt.Sprintf("-d=%s", path)} - - process, err := os.StartProcess(EtcdBinPath, args, procAttr) - if err != nil { - t.Fatal("start process failed:" + err.Error()) - return - } - defer process.Kill() - time.Sleep(time.Second) - // Ensure deleted message is removed. resp, err := tests.Get("http://localhost:4001/v2/keys/message") @@ -48,4 +56,3 @@ func TestV1Migration(t *testing.T) { assert.Nil(t, err, "") assert.Equal(t, resp.StatusCode, 404, "") } - From 6b5d6ecd8b21c61a19e18b618a3378ce2c47d3d4 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 4 Nov 2013 13:46:43 -0700 Subject: [PATCH 170/247] Cluster migration test. --- tests/fixtures/v1/node0/conf | 1 + tests/fixtures/v1/node0/info | 18 ++++++++++++++++++ tests/fixtures/v1/node0/log | Bin 0 -> 1540 bytes tests/fixtures/v1/node2/conf | 1 + tests/fixtures/v1/node2/info | 18 ++++++++++++++++++ tests/fixtures/v1/node2/log | Bin 0 -> 1540 bytes tests/fixtures/v1/node3/conf | 1 + tests/fixtures/v1/node3/info | 18 ++++++++++++++++++ tests/fixtures/v1/node3/log | Bin 0 -> 1540 bytes tests/functional/v1_migration_test.go | 13 ++++++++++--- 10 files changed, 67 insertions(+), 3 deletions(-) create mode 100644 tests/fixtures/v1/node0/conf create mode 100644 tests/fixtures/v1/node0/info create mode 100644 tests/fixtures/v1/node0/log create mode 100644 tests/fixtures/v1/node2/conf create mode 100644 tests/fixtures/v1/node2/info create mode 100644 tests/fixtures/v1/node2/log create mode 100644 tests/fixtures/v1/node3/conf create mode 100644 tests/fixtures/v1/node3/info create mode 100644 tests/fixtures/v1/node3/log diff --git a/tests/fixtures/v1/node0/conf b/tests/fixtures/v1/node0/conf new file mode 100644 index 00000000000..8f401dbe448 --- /dev/null +++ b/tests/fixtures/v1/node0/conf @@ -0,0 +1 @@ +{"commitIndex":15,"peers":[{"name":"node2","connectionString":""}]} \ No newline at end of file diff --git a/tests/fixtures/v1/node0/info b/tests/fixtures/v1/node0/info new file mode 100644 index 00000000000..398c8e1e80a --- /dev/null +++ b/tests/fixtures/v1/node0/info @@ -0,0 +1,18 @@ +{ + "name": "node0", + "raftURL": "http://127.0.0.1:7001", + "etcdURL": "http://127.0.0.1:4001", + "webURL": "", + "raftListenHost": "127.0.0.1:7001", + "etcdListenHost": "127.0.0.1:4001", + "raftTLS": { + "CertFile": "", + "KeyFile": "", + "CAFile": "" + }, + "etcdTLS": { + "CertFile": "", + "KeyFile": "", + "CAFile": "" + } +} diff --git a/tests/fixtures/v1/node0/log b/tests/fixtures/v1/node0/log new file mode 100644 index 0000000000000000000000000000000000000000..de3e7075e9396556e207ce78f30e80d042202a7a GIT binary patch literal 1540 zcmbu9O>dh(5Qf{PADd0vBt7N+>CM`9wwTRz(!?up)P*$d{bGqSjSrmSF5} z3;PfJMOUU!mhl8I**wm!$hl+4GHqS!B1;HyGfN8?rj@BO^g~CY<8mm2NC>eLJ8?QQaYkPly0Q3yM9j%27}P|0#9S6Yn9~C0=XYadu1Q8mu#bEE?I!Tgd%zW>iG5VP z{sm|6Het_d?9#4!K*sut+-q_lQ7b~LYLd7fT_l$hl}Q4n|>Sr E08;vuA^-pY literal 0 HcmV?d00001 diff --git a/tests/fixtures/v1/node2/conf b/tests/fixtures/v1/node2/conf new file mode 100644 index 00000000000..19d6a9c9228 --- /dev/null +++ b/tests/fixtures/v1/node2/conf @@ -0,0 +1 @@ +{"commitIndex":15,"peers":[{"name":"node0","connectionString":""}]} \ No newline at end of file diff --git a/tests/fixtures/v1/node2/info b/tests/fixtures/v1/node2/info new file mode 100644 index 00000000000..85114a5f810 --- /dev/null +++ b/tests/fixtures/v1/node2/info @@ -0,0 +1,18 @@ +{ + "name": "node2", + "raftURL": "http://127.0.0.1:7002", + "etcdURL": "http://127.0.0.1:4002", + "webURL": "", + "raftListenHost": "127.0.0.1:7002", + "etcdListenHost": "127.0.0.1:4002", + "raftTLS": { + "CertFile": "", + "KeyFile": "", + "CAFile": "" + }, + "etcdTLS": { + "CertFile": "", + "KeyFile": "", + "CAFile": "" + } +} diff --git a/tests/fixtures/v1/node2/log b/tests/fixtures/v1/node2/log new file mode 100644 index 0000000000000000000000000000000000000000..de3e7075e9396556e207ce78f30e80d042202a7a GIT binary patch literal 1540 zcmbu9O>dh(5Qf{PADd0vBt7N+>CM`9wwTRz(!?up)P*$d{bGqSjSrmSF5} z3;PfJMOUU!mhl8I**wm!$hl+4GHqS!B1;HyGfN8?rj@BO^g~CY<8mm2NC>eLJ8?QQaYkPly0Q3yM9j%27}P|0#9S6Yn9~C0=XYadu1Q8mu#bEE?I!Tgd%zW>iG5VP z{sm|6Het_d?9#4!K*sut+-q_lQ7b~LYLd7fT_l$hl}Q4n|>Sr E08;vuA^-pY literal 0 HcmV?d00001 diff --git a/tests/fixtures/v1/node3/conf b/tests/fixtures/v1/node3/conf new file mode 100644 index 00000000000..d8a5840de89 --- /dev/null +++ b/tests/fixtures/v1/node3/conf @@ -0,0 +1 @@ +{"commitIndex":15,"peers":[{"name":"node0","connectionString":""},{"name":"node2","connectionString":""}]} \ No newline at end of file diff --git a/tests/fixtures/v1/node3/info b/tests/fixtures/v1/node3/info new file mode 100644 index 00000000000..5e5cb3f3a84 --- /dev/null +++ b/tests/fixtures/v1/node3/info @@ -0,0 +1,18 @@ +{ + "name": "node3", + "raftURL": "http://127.0.0.1:7003", + "etcdURL": "http://127.0.0.1:4003", + "webURL": "", + "raftListenHost": "127.0.0.1:7003", + "etcdListenHost": "127.0.0.1:4003", + "raftTLS": { + "CertFile": "", + "KeyFile": "", + "CAFile": "" + }, + "etcdTLS": { + "CertFile": "", + "KeyFile": "", + "CAFile": "" + } +} diff --git a/tests/fixtures/v1/node3/log b/tests/fixtures/v1/node3/log new file mode 100644 index 0000000000000000000000000000000000000000..de3e7075e9396556e207ce78f30e80d042202a7a GIT binary patch literal 1540 zcmbu9O>dh(5Qf{PADd0vBt7N+>CM`9wwTRz(!?up)P*$d{bGqSjSrmSF5} z3;PfJMOUU!mhl8I**wm!$hl+4GHqS!B1;HyGfN8?rj@BO^g~CY<8mm2NC>eLJ8?QQaYkPly0Q3yM9j%27}P|0#9S6Yn9~C0=XYadu1Q8mu#bEE?I!Tgd%zW>iG5VP z{sm|6Het_d?9#4!K*sut+-q_lQ7b~LYLd7fT_l$hl}Q4n|>Sr E08;vuA^-pY literal 0 HcmV?d00001 diff --git a/tests/functional/v1_migration_test.go b/tests/functional/v1_migration_test.go index f5d76fee564..b4c9a07bc2c 100644 --- a/tests/functional/v1_migration_test.go +++ b/tests/functional/v1_migration_test.go @@ -19,13 +19,19 @@ func TestV1Migration(t *testing.T) { os.RemoveAll(path) defer os.RemoveAll(path) - nodes := []string{"node0", "node1"} + nodes := []string{"node0", "node2"} for i, node := range nodes { nodepath := filepath.Join(path, node) + fixturepath, _ := filepath.Abs(filepath.Join("../fixtures/v1/", node)) + fmt.Println("FIXPATH =", fixturepath) + fmt.Println("NODEPATH =", nodepath) + os.MkdirAll(filepath.Dir(nodepath), 0777) // Copy over fixture files. - if err := exec.Command("cp", "-r", "../fixtures/v1/" + node, nodepath).Run(); err != nil { - panic("Fixture initialization error") + c := exec.Command("cp", "-rf", fixturepath, nodepath) + if out, err := c.CombinedOutput(); err != nil { + fmt.Println(">>>>>>\n", string(out), "<<<<<<") + panic("Fixture initialization error:" + err.Error()) } procAttr := new(os.ProcAttr) @@ -43,6 +49,7 @@ func TestV1Migration(t *testing.T) { time.Sleep(time.Second) } + time.Sleep(120 * time.Second) // Ensure deleted message is removed. resp, err := tests.Get("http://localhost:4001/v2/keys/message") From 35d971970790ff34c9eb07cb970124a5ff6d0239 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 4 Nov 2013 13:56:02 -0700 Subject: [PATCH 171/247] Add v1.solo migration fixture. --- tests/fixtures/{v1 => v1.cluster}/README | 10 +++++----- tests/fixtures/{v1 => v1.cluster}/node0/conf | 0 tests/fixtures/{v1 => v1.cluster}/node0/info | 0 tests/fixtures/{v1 => v1.cluster}/node0/log | Bin tests/fixtures/{v1 => v1.cluster}/node2/conf | 0 tests/fixtures/{v1 => v1.cluster}/node2/info | 0 tests/fixtures/{v1 => v1.cluster}/node2/log | Bin tests/fixtures/{v1 => v1.cluster}/node3/conf | 0 tests/fixtures/{v1 => v1.cluster}/node3/info | 0 tests/fixtures/{v1 => v1.cluster}/node3/log | Bin tests/fixtures/{v1 => v1.cluster}/run.1.sh | 0 tests/fixtures/{v1 => v1.cluster}/run.2.sh | 0 tests/fixtures/{v1 => v1.cluster}/run.3.sh | 0 tests/fixtures/{v1 => v1.cluster}/run.4.sh | 0 tests/fixtures/v1.solo/README | 13 +++++++++++++ tests/fixtures/v1.solo/node0/conf | 1 + tests/fixtures/v1.solo/node0/info | 18 ++++++++++++++++++ tests/fixtures/v1.solo/node0/log | Bin 0 -> 275 bytes tests/fixtures/v1.solo/run.1.sh | 4 ++++ tests/fixtures/v1.solo/run.2.sh | 3 +++ 20 files changed, 44 insertions(+), 5 deletions(-) rename tests/fixtures/{v1 => v1.cluster}/README (59%) rename tests/fixtures/{v1 => v1.cluster}/node0/conf (100%) rename tests/fixtures/{v1 => v1.cluster}/node0/info (100%) rename tests/fixtures/{v1 => v1.cluster}/node0/log (100%) rename tests/fixtures/{v1 => v1.cluster}/node2/conf (100%) rename tests/fixtures/{v1 => v1.cluster}/node2/info (100%) rename tests/fixtures/{v1 => v1.cluster}/node2/log (100%) rename tests/fixtures/{v1 => v1.cluster}/node3/conf (100%) rename tests/fixtures/{v1 => v1.cluster}/node3/info (100%) rename tests/fixtures/{v1 => v1.cluster}/node3/log (100%) rename tests/fixtures/{v1 => v1.cluster}/run.1.sh (100%) mode change 100644 => 100755 rename tests/fixtures/{v1 => v1.cluster}/run.2.sh (100%) mode change 100644 => 100755 rename tests/fixtures/{v1 => v1.cluster}/run.3.sh (100%) mode change 100644 => 100755 rename tests/fixtures/{v1 => v1.cluster}/run.4.sh (100%) mode change 100644 => 100755 create mode 100644 tests/fixtures/v1.solo/README create mode 100644 tests/fixtures/v1.solo/node0/conf create mode 100644 tests/fixtures/v1.solo/node0/info create mode 100644 tests/fixtures/v1.solo/node0/log create mode 100755 tests/fixtures/v1.solo/run.1.sh create mode 100755 tests/fixtures/v1.solo/run.2.sh diff --git a/tests/fixtures/v1/README b/tests/fixtures/v1.cluster/README similarity index 59% rename from tests/fixtures/v1/README rename to tests/fixtures/v1.cluster/README index e930eda7d70..8e144444a3e 100644 --- a/tests/fixtures/v1/README +++ b/tests/fixtures/v1.cluster/README @@ -3,13 +3,13 @@ README The scripts in this directory should be run from the project root: $ cd $GOPATH/src/github.com/coreos/etcd -$ tests/fixtures/v1/complete.1.sh +$ tests/fixtures/v1/run.1.sh Scripts with numbers should be run in separate terminal windows (in order): -$ tests/fixtures/v1/complete.1.sh -$ tests/fixtures/v1/complete.2.sh -$ tests/fixtures/v1/complete.3.sh -$ tests/fixtures/v1/complete.4.sh +$ tests/fixtures/v1/run.1.sh +$ tests/fixtures/v1/run.2.sh +$ tests/fixtures/v1/run.3.sh +$ tests/fixtures/v1/run.4.sh The resulting server state data can be found in tmp/node*. diff --git a/tests/fixtures/v1/node0/conf b/tests/fixtures/v1.cluster/node0/conf similarity index 100% rename from tests/fixtures/v1/node0/conf rename to tests/fixtures/v1.cluster/node0/conf diff --git a/tests/fixtures/v1/node0/info b/tests/fixtures/v1.cluster/node0/info similarity index 100% rename from tests/fixtures/v1/node0/info rename to tests/fixtures/v1.cluster/node0/info diff --git a/tests/fixtures/v1/node0/log b/tests/fixtures/v1.cluster/node0/log similarity index 100% rename from tests/fixtures/v1/node0/log rename to tests/fixtures/v1.cluster/node0/log diff --git a/tests/fixtures/v1/node2/conf b/tests/fixtures/v1.cluster/node2/conf similarity index 100% rename from tests/fixtures/v1/node2/conf rename to tests/fixtures/v1.cluster/node2/conf diff --git a/tests/fixtures/v1/node2/info b/tests/fixtures/v1.cluster/node2/info similarity index 100% rename from tests/fixtures/v1/node2/info rename to tests/fixtures/v1.cluster/node2/info diff --git a/tests/fixtures/v1/node2/log b/tests/fixtures/v1.cluster/node2/log similarity index 100% rename from tests/fixtures/v1/node2/log rename to tests/fixtures/v1.cluster/node2/log diff --git a/tests/fixtures/v1/node3/conf b/tests/fixtures/v1.cluster/node3/conf similarity index 100% rename from tests/fixtures/v1/node3/conf rename to tests/fixtures/v1.cluster/node3/conf diff --git a/tests/fixtures/v1/node3/info b/tests/fixtures/v1.cluster/node3/info similarity index 100% rename from tests/fixtures/v1/node3/info rename to tests/fixtures/v1.cluster/node3/info diff --git a/tests/fixtures/v1/node3/log b/tests/fixtures/v1.cluster/node3/log similarity index 100% rename from tests/fixtures/v1/node3/log rename to tests/fixtures/v1.cluster/node3/log diff --git a/tests/fixtures/v1/run.1.sh b/tests/fixtures/v1.cluster/run.1.sh old mode 100644 new mode 100755 similarity index 100% rename from tests/fixtures/v1/run.1.sh rename to tests/fixtures/v1.cluster/run.1.sh diff --git a/tests/fixtures/v1/run.2.sh b/tests/fixtures/v1.cluster/run.2.sh old mode 100644 new mode 100755 similarity index 100% rename from tests/fixtures/v1/run.2.sh rename to tests/fixtures/v1.cluster/run.2.sh diff --git a/tests/fixtures/v1/run.3.sh b/tests/fixtures/v1.cluster/run.3.sh old mode 100644 new mode 100755 similarity index 100% rename from tests/fixtures/v1/run.3.sh rename to tests/fixtures/v1.cluster/run.3.sh diff --git a/tests/fixtures/v1/run.4.sh b/tests/fixtures/v1.cluster/run.4.sh old mode 100644 new mode 100755 similarity index 100% rename from tests/fixtures/v1/run.4.sh rename to tests/fixtures/v1.cluster/run.4.sh diff --git a/tests/fixtures/v1.solo/README b/tests/fixtures/v1.solo/README new file mode 100644 index 00000000000..65d86d32381 --- /dev/null +++ b/tests/fixtures/v1.solo/README @@ -0,0 +1,13 @@ +README + +The scripts in this directory should be run from the project root: + +$ cd $GOPATH/src/github.com/coreos/etcd +$ tests/fixtures/v1.solo/run.1.sh + +Scripts with numbers should be run in separate terminal windows (in order): + +$ tests/fixtures/v1/run.1.sh +$ tests/fixtures/v1/run.2.sh + +The resulting server state data can be found in tmp/node0. diff --git a/tests/fixtures/v1.solo/node0/conf b/tests/fixtures/v1.solo/node0/conf new file mode 100644 index 00000000000..95106f8b126 --- /dev/null +++ b/tests/fixtures/v1.solo/node0/conf @@ -0,0 +1 @@ +{"commitIndex":1,"peers":[]} \ No newline at end of file diff --git a/tests/fixtures/v1.solo/node0/info b/tests/fixtures/v1.solo/node0/info new file mode 100644 index 00000000000..398c8e1e80a --- /dev/null +++ b/tests/fixtures/v1.solo/node0/info @@ -0,0 +1,18 @@ +{ + "name": "node0", + "raftURL": "http://127.0.0.1:7001", + "etcdURL": "http://127.0.0.1:4001", + "webURL": "", + "raftListenHost": "127.0.0.1:7001", + "etcdListenHost": "127.0.0.1:4001", + "raftTLS": { + "CertFile": "", + "KeyFile": "", + "CAFile": "" + }, + "etcdTLS": { + "CertFile": "", + "KeyFile": "", + "CAFile": "" + } +} diff --git a/tests/fixtures/v1.solo/node0/log b/tests/fixtures/v1.solo/node0/log new file mode 100644 index 0000000000000000000000000000000000000000..661d21d0e8daad5ddc3c14c5b7153ce0dbe55f4f GIT binary patch literal 275 zcmZ{fy>h}Z5QGUdjw@;^jcROk5NxC;NRb8x=nX!Ii7k0-@snYAckWD$PK9o@r~M=Z zuZ0eSk6~~0iGH7(lPj#e-DQ&<{1%(Ga8??5Q8C2RXr`kh>N)JRmSbtN3hBNr6dwrS2^^?L A4FCWD literal 0 HcmV?d00001 diff --git a/tests/fixtures/v1.solo/run.1.sh b/tests/fixtures/v1.solo/run.1.sh new file mode 100755 index 00000000000..ee77deaed26 --- /dev/null +++ b/tests/fixtures/v1.solo/run.1.sh @@ -0,0 +1,4 @@ +#!/bin/sh + +./build +./etcd -d tmp/node0 -n node0 diff --git a/tests/fixtures/v1.solo/run.2.sh b/tests/fixtures/v1.solo/run.2.sh new file mode 100755 index 00000000000..96bd3e862fc --- /dev/null +++ b/tests/fixtures/v1.solo/run.2.sh @@ -0,0 +1,3 @@ +#!/bin/sh + +curl -L http://127.0.0.1:4001/v1/keys/message -d value="Hello world" From d3bfc49b7cfe17b1d27f1b75cf14d205112f4964 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Mon, 4 Nov 2013 14:36:20 -0700 Subject: [PATCH 172/247] Remove leading slash from handlers. --- server/v1/delete_key_handler.go | 2 +- server/v1/get_key_handler.go | 2 +- server/v1/set_key_handler.go | 2 +- server/v1/watch_key_handler.go | 2 +- server/v2/delete_handler.go | 2 +- server/v2/get_handler.go | 2 +- server/v2/post_handler.go | 2 +- server/v2/put_handler.go | 2 +- tests/functional/v1_migration_test.go | 50 +++++++++++++++++++++++++-- 9 files changed, 56 insertions(+), 10 deletions(-) diff --git a/server/v1/delete_key_handler.go b/server/v1/delete_key_handler.go index 44fe459e494..561196b9599 100644 --- a/server/v1/delete_key_handler.go +++ b/server/v1/delete_key_handler.go @@ -8,7 +8,7 @@ import ( // Removes a key from the store. func DeleteKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) - key := "/" + vars["key"] + key := vars["key"] c := s.Store().CommandFactory().CreateDeleteCommand(key, false) return s.Dispatch(c, w, req) } diff --git a/server/v1/get_key_handler.go b/server/v1/get_key_handler.go index 53558e142ff..9a68212c6a7 100644 --- a/server/v1/get_key_handler.go +++ b/server/v1/get_key_handler.go @@ -10,7 +10,7 @@ import ( // Retrieves the value for a given key. func GetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) - key := "/" + vars["key"] + key := vars["key"] // Retrieve the key from the store. event, err := s.Store().Get(key, false, false, s.CommitIndex(), s.Term()) diff --git a/server/v1/set_key_handler.go b/server/v1/set_key_handler.go index 7acfe7ecb21..c047ebfb775 100644 --- a/server/v1/set_key_handler.go +++ b/server/v1/set_key_handler.go @@ -12,7 +12,7 @@ import ( // Sets the value for a given key. func SetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) - key := "/" + vars["key"] + key := vars["key"] req.ParseForm() diff --git a/server/v1/watch_key_handler.go b/server/v1/watch_key_handler.go index e8db56c30b4..8a617a50bd3 100644 --- a/server/v1/watch_key_handler.go +++ b/server/v1/watch_key_handler.go @@ -14,7 +14,7 @@ import ( func WatchKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { var err error vars := mux.Vars(req) - key := "/" + vars["key"] + key := vars["key"] // Create a command to watch from a given index (default 0). var sinceIndex uint64 = 0 diff --git a/server/v2/delete_handler.go b/server/v2/delete_handler.go index 7afc02f0412..17030c75715 100644 --- a/server/v2/delete_handler.go +++ b/server/v2/delete_handler.go @@ -8,7 +8,7 @@ import ( func DeleteHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) - key := "/" + vars["key"] + key := vars["key"] recursive := (req.FormValue("recursive") == "true") c := s.Store().CommandFactory().CreateDeleteCommand(key, recursive) diff --git a/server/v2/get_handler.go b/server/v2/get_handler.go index 39e932fe09b..640533b05bb 100644 --- a/server/v2/get_handler.go +++ b/server/v2/get_handler.go @@ -18,7 +18,7 @@ func GetHandler(w http.ResponseWriter, req *http.Request, s Server) error { var event *store.Event vars := mux.Vars(req) - key := "/" + vars["key"] + key := vars["key"] // Help client to redirect the request to the current leader if req.FormValue("consistent") == "true" && s.State() != raft.Leader { diff --git a/server/v2/post_handler.go b/server/v2/post_handler.go index 4dc98b92510..ca94b4a93da 100644 --- a/server/v2/post_handler.go +++ b/server/v2/post_handler.go @@ -10,7 +10,7 @@ import ( func PostHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) - key := "/" + vars["key"] + key := vars["key"] value := req.FormValue("value") expireTime, err := store.TTL(req.FormValue("ttl")) diff --git a/server/v2/put_handler.go b/server/v2/put_handler.go index 3afb018b726..cb44a723762 100644 --- a/server/v2/put_handler.go +++ b/server/v2/put_handler.go @@ -15,7 +15,7 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { var c raft.Command vars := mux.Vars(req) - key := "/" + vars["key"] + key := vars["key"] req.ParseForm() diff --git a/tests/functional/v1_migration_test.go b/tests/functional/v1_migration_test.go index b4c9a07bc2c..a01d00c2039 100644 --- a/tests/functional/v1_migration_test.go +++ b/tests/functional/v1_migration_test.go @@ -14,7 +14,53 @@ import ( ) // Ensure that we can start a v2 node from the log of a v1 node. -func TestV1Migration(t *testing.T) { +func TestV1SoloMigration(t *testing.T) { + path, _ := ioutil.TempDir("", "etcd-") + os.MkdirAll(path, 0777) + defer os.RemoveAll(path) + + nodepath := filepath.Join(path, "node0") + fixturepath, _ := filepath.Abs("../fixtures/v1.solo/node0") + + + // Copy over fixture files. + c := exec.Command("cp", "-rf", fixturepath, nodepath) + if out, err := c.CombinedOutput(); err != nil { + fmt.Println(">>>>>>\n", string(out), "<<<<<<") + panic("Fixture initialization error:" + err.Error()) + } + + procAttr := new(os.ProcAttr) + procAttr.Files = []*os.File{nil, os.Stdout, os.Stderr} + + args := []string{"etcd", fmt.Sprintf("-d=%s", nodepath)} + args = append(args, "-c", "127.0.0.1:4001") + args = append(args, "-s", "127.0.0.1:7001") + process, err := os.StartProcess(EtcdBinPath, args, procAttr) + if err != nil { + t.Fatal("start process failed:" + err.Error()) + return + } + defer process.Kill() + time.Sleep(time.Second) + + time.Sleep(120 * time.Second) + + // Ensure deleted message is removed. + resp, err := tests.Get("http://localhost:4001/v2/keys/message") + tests.ReadBody(resp) + assert.Nil(t, err, "") + assert.Equal(t, resp.StatusCode, 404, "") + + // Ensure TTL'd message is removed. + resp, err = tests.Get("http://localhost:4001/v2/keys/foo") + tests.ReadBody(resp) + assert.Nil(t, err, "") + assert.Equal(t, resp.StatusCode, 404, "") +} + +// Ensure that we can start a v2 cluster from the logs of a v1 cluster. +func TestV1ClusterMigration(t *testing.T) { path, _ := ioutil.TempDir("", "etcd-") os.RemoveAll(path) defer os.RemoveAll(path) @@ -22,7 +68,7 @@ func TestV1Migration(t *testing.T) { nodes := []string{"node0", "node2"} for i, node := range nodes { nodepath := filepath.Join(path, node) - fixturepath, _ := filepath.Abs(filepath.Join("../fixtures/v1/", node)) + fixturepath, _ := filepath.Abs(filepath.Join("../fixtures/v1.cluster/", node)) fmt.Println("FIXPATH =", fixturepath) fmt.Println("NODEPATH =", nodepath) os.MkdirAll(filepath.Dir(nodepath), 0777) From c05df9e3f5ef37e401a959a13e7b25955cd1d15e Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 4 Nov 2013 20:31:24 -0800 Subject: [PATCH 173/247] refactor add newTTLKeyHeap function --- store/heap_test.go | 3 +-- store/ttl_key_heap.go | 6 ++++++ 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/store/heap_test.go b/store/heap_test.go index 2ae573724dd..1da81d3120c 100644 --- a/store/heap_test.go +++ b/store/heap_test.go @@ -8,8 +8,7 @@ import ( ) func TestHeapPushPop(t *testing.T) { - h := &TTLKeyHeap{Map: make(map[*Node]int)} - heap.Init(h) + h := newTTLKeyHeap() // add from older expire time to earlier expire time // the path is equal to ttl from now diff --git a/store/ttl_key_heap.go b/store/ttl_key_heap.go index abb12fede52..23a9997f0c0 100644 --- a/store/ttl_key_heap.go +++ b/store/ttl_key_heap.go @@ -10,6 +10,12 @@ type TTLKeyHeap struct { Map map[*Node]int } +func newTTLKeyHeap() *TTLKeyHeap { + h := &TTLKeyHeap{Map: make(map[*Node]int)} + heap.Init(h) + return h +} + func (h TTLKeyHeap) Len() int { return len(h.Array) } From 3f6d6cf4c60f601162bc79e424c10cbdf90b9089 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 4 Nov 2013 20:56:41 -0800 Subject: [PATCH 174/247] refactor use time.IsZero --- store/node.go | 12 +++++------- store/store.go | 4 +++- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/store/node.go b/store/node.go index 0905d4d1be2..b95a46e87ca 100644 --- a/store/node.go +++ b/store/node.go @@ -9,15 +9,13 @@ import ( etcdErr "github.com/coreos/etcd/error" ) -var ( - Permanent time.Time -) - const ( normal = iota removed ) +var Permanent time.Time + // Node is the basic element in the store system. // A key-value pair will have a string value // A directory will have a children map @@ -97,7 +95,7 @@ func (n *Node) IsHidden() bool { // IsPermanent function checks if the node is a permanent one. func (n *Node) IsPermanent() bool { - return n.ExpireTime.Sub(Permanent) == 0 + return !n.ExpireTime.IsZero() } // IsExpired function checks if the node has been expired. @@ -146,7 +144,7 @@ func (n *Node) Write(value string, index uint64, term uint64) *etcdErr.Error { } func (n *Node) ExpirationAndTTL() (*time.Time, int64) { - if n.ExpireTime.Sub(Permanent) != 0 { + if n.IsPermanent() { return &n.ExpireTime, int64(n.ExpireTime.Sub(time.Now())/time.Second) + 1 } return nil, 0 @@ -376,7 +374,7 @@ func (n *Node) UpdateTTL(expireTime time.Time) { } n.ExpireTime = expireTime - if expireTime.Sub(Permanent) != 0 { + if !n.IsPermanent() { n.Expire() } } diff --git a/store/store.go b/store/store.go index f752f6935b4..93278bf31c6 100644 --- a/store/store.go +++ b/store/store.go @@ -37,6 +37,7 @@ type Store interface { type store struct { Root *Node WatcherHub *watcherHub + TTLKeyHeap *TTLKeyHeap Index uint64 Term uint64 Stats *Stats @@ -54,6 +55,7 @@ func newStore() *store { s.Root = newDir(s, "/", UndefIndex, UndefTerm, nil, "", Permanent) s.Stats = newStats() s.WatcherHub = newWatchHub(1000) + s.TTLKeyHeap = newTTLKeyHeap() return s } @@ -390,7 +392,7 @@ func (s *store) internalCreate(nodePath string, value string, unique bool, repla d.Add(n) // Node with TTL - if expireTime.Sub(Permanent) != 0 { + if !n.IsPermanent() { n.Expire() e.Expiration, e.TTL = n.ExpirationAndTTL() } From c5a6f9bb6b40e16fa9190947f75e3a1be8838d2d Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 4 Nov 2013 21:22:22 -0800 Subject: [PATCH 175/247] fix iszero --- store/heap_test.go | 4 ++-- store/node.go | 33 +++++++++++++++++++++++++++++++-- store/store.go | 3 +++ store/ttl_key_heap.go | 7 ++++++- 4 files changed, 42 insertions(+), 5 deletions(-) diff --git a/store/heap_test.go b/store/heap_test.go index 1da81d3120c..f55413ee114 100644 --- a/store/heap_test.go +++ b/store/heap_test.go @@ -54,8 +54,8 @@ func TestHeapUpdate(t *testing.T) { // Path 5 kvs[5].ExpireTime = time.Now().Add(time.Second * 12) - h.Update(kvs[3]) - h.Update(kvs[5]) + h.update(kvs[3]) + h.update(kvs[5]) min := time.Now() diff --git a/store/node.go b/store/node.go index b95a46e87ca..50518f92604 100644 --- a/store/node.go +++ b/store/node.go @@ -1,6 +1,7 @@ package store import ( + "container/heap" "path" "sort" "sync" @@ -95,7 +96,7 @@ func (n *Node) IsHidden() bool { // IsPermanent function checks if the node is a permanent one. func (n *Node) IsPermanent() bool { - return !n.ExpireTime.IsZero() + return n.ExpireTime.IsZero() } // IsExpired function checks if the node has been expired. @@ -144,7 +145,7 @@ func (n *Node) Write(value string, index uint64, term uint64) *etcdErr.Error { } func (n *Node) ExpirationAndTTL() (*time.Time, int64) { - if n.IsPermanent() { + if !n.IsPermanent() { return &n.ExpireTime, int64(n.ExpireTime.Sub(time.Now())/time.Second) + 1 } return nil, 0 @@ -239,6 +240,10 @@ func (n *Node) internalRemove(recursive bool, callback func(path string)) { callback(n.Path) } + if !n.IsPermanent() { + n.store.TTLKeyHeap.remove(n) + } + // the stop channel has a buffer. just send to it! n.stopExpire <- true return @@ -257,6 +262,10 @@ func (n *Node) internalRemove(recursive bool, callback func(path string)) { callback(n.Path) } + if !n.IsPermanent() { + n.store.TTLKeyHeap.remove(n) + } + n.stopExpire <- true } } @@ -362,6 +371,26 @@ func (n *Node) Pair(recurisive, sorted bool) KeyValuePair { } func (n *Node) UpdateTTL(expireTime time.Time) { + + if !n.IsPermanent() { + if expireTime.IsZero() { + // from ttl to permanent + // remove from ttl heap + n.store.TTLKeyHeap.remove(n) + } else { + // update ttl + // update ttl heap + n.store.TTLKeyHeap.update(n) + } + + } else { + if !expireTime.IsZero() { + // from permanent to ttl + // push into ttl heap + heap.Push(n.store.TTLKeyHeap, n) + } + } + if !n.IsPermanent() { // check if the node has been expired // if the node is not expired, we need to stop the go routine associated with diff --git a/store/store.go b/store/store.go index 93278bf31c6..aae06f92dfb 100644 --- a/store/store.go +++ b/store/store.go @@ -1,6 +1,7 @@ package store import ( + "container/heap" "encoding/json" "fmt" "path" @@ -393,6 +394,8 @@ func (s *store) internalCreate(nodePath string, value string, unique bool, repla // Node with TTL if !n.IsPermanent() { + heap.Push(s.TTLKeyHeap, n) + n.Expire() e.Expiration, e.TTL = n.ExpirationAndTTL() } diff --git a/store/ttl_key_heap.go b/store/ttl_key_heap.go index 23a9997f0c0..9694a6abe99 100644 --- a/store/ttl_key_heap.go +++ b/store/ttl_key_heap.go @@ -48,8 +48,13 @@ func (h *TTLKeyHeap) Pop() interface{} { return x } -func (h *TTLKeyHeap) Update(n *Node) { +func (h *TTLKeyHeap) update(n *Node) { index := h.Map[n] heap.Remove(h, index) heap.Push(h, n) } + +func (h *TTLKeyHeap) remove(n *Node) { + index := h.Map[n] + heap.Remove(h, index) +} From efe431ead0597b669b6e3c74c84806da51372b0f Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 4 Nov 2013 21:33:23 -0800 Subject: [PATCH 176/247] refactor add push/pop function --- store/heap_test.go | 10 ++++------ store/ttl_key_heap.go | 10 ++++++++++ 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/store/heap_test.go b/store/heap_test.go index f55413ee114..e9cda376ed1 100644 --- a/store/heap_test.go +++ b/store/heap_test.go @@ -16,14 +16,13 @@ func TestHeapPushPop(t *testing.T) { path := fmt.Sprintf("%v", 10-i) m := time.Duration(10 - i) n := newKV(nil, path, path, 0, 0, nil, "", time.Now().Add(time.Second*m)) - heap.Push(h, n) + h.push(n) } min := time.Now() for i := 0; i < 10; i++ { - iNode := heap.Pop(h) - node, _ := iNode.(*Node) + node := h.pop() if node.ExpireTime.Before(min) { t.Fatal("heap sort wrong!") } @@ -45,7 +44,7 @@ func TestHeapUpdate(t *testing.T) { m := time.Duration(10 - i) n = newKV(nil, path, path, 0, 0, nil, "", time.Now().Add(time.Second*m)) kvs[i] = n - heap.Push(h, n) + h.push(n) } // Path 7 @@ -60,8 +59,7 @@ func TestHeapUpdate(t *testing.T) { min := time.Now() for i := 0; i < 10; i++ { - iNode := heap.Pop(h) - node, _ := iNode.(*Node) + node := h.pop() if node.ExpireTime.Before(min) { t.Fatal("heap sort wrong!") } diff --git a/store/ttl_key_heap.go b/store/ttl_key_heap.go index 9694a6abe99..34fa4ba63d2 100644 --- a/store/ttl_key_heap.go +++ b/store/ttl_key_heap.go @@ -48,6 +48,16 @@ func (h *TTLKeyHeap) Pop() interface{} { return x } +func (h *TTLKeyHeap) pop() *Node { + x := heap.Pop(h) + n, _ := x.(*Node) + return n +} + +func (h *TTLKeyHeap) push(x interface{}) { + heap.Push(h, x) +} + func (h *TTLKeyHeap) update(n *Node) { index := h.Map[n] heap.Remove(h, index) From 0d8510df338e88c9e422636e3e395afbfce6cc23 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 4 Nov 2013 21:36:05 -0800 Subject: [PATCH 177/247] refactor use push --- store/node.go | 3 +-- store/store.go | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/store/node.go b/store/node.go index 50518f92604..72510260dcb 100644 --- a/store/node.go +++ b/store/node.go @@ -1,7 +1,6 @@ package store import ( - "container/heap" "path" "sort" "sync" @@ -387,7 +386,7 @@ func (n *Node) UpdateTTL(expireTime time.Time) { if !expireTime.IsZero() { // from permanent to ttl // push into ttl heap - heap.Push(n.store.TTLKeyHeap, n) + n.store.TTLKeyHeap.push(n) } } diff --git a/store/store.go b/store/store.go index aae06f92dfb..babb5a93288 100644 --- a/store/store.go +++ b/store/store.go @@ -1,7 +1,6 @@ package store import ( - "container/heap" "encoding/json" "fmt" "path" @@ -394,7 +393,7 @@ func (s *store) internalCreate(nodePath string, value string, unique bool, repla // Node with TTL if !n.IsPermanent() { - heap.Push(s.TTLKeyHeap, n) + s.TTLKeyHeap.push(n) n.Expire() e.Expiration, e.TTL = n.ExpirationAndTTL() From 07b52ee24c768b338c5b249e409c52870b8d8e7f Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 4 Nov 2013 21:51:14 -0800 Subject: [PATCH 178/247] fix save and recovery --- store/heap_test.go | 6 ++--- store/node.go | 14 +++++++----- store/store.go | 9 +++++--- store/ttl_key_heap.go | 52 +++++++++++++++++++++---------------------- 4 files changed, 43 insertions(+), 38 deletions(-) diff --git a/store/heap_test.go b/store/heap_test.go index e9cda376ed1..aa0b9caf613 100644 --- a/store/heap_test.go +++ b/store/heap_test.go @@ -1,14 +1,13 @@ package store import ( - "container/heap" "fmt" "testing" "time" ) func TestHeapPushPop(t *testing.T) { - h := newTTLKeyHeap() + h := newTtlKeyHeap() // add from older expire time to earlier expire time // the path is equal to ttl from now @@ -32,8 +31,7 @@ func TestHeapPushPop(t *testing.T) { } func TestHeapUpdate(t *testing.T) { - h := &TTLKeyHeap{Map: make(map[*Node]int)} - heap.Init(h) + h := newTtlKeyHeap() kvs := make([]*Node, 10) diff --git a/store/node.go b/store/node.go index 72510260dcb..38a9692038a 100644 --- a/store/node.go +++ b/store/node.go @@ -240,7 +240,7 @@ func (n *Node) internalRemove(recursive bool, callback func(path string)) { } if !n.IsPermanent() { - n.store.TTLKeyHeap.remove(n) + n.store.ttlKeyHeap.remove(n) } // the stop channel has a buffer. just send to it! @@ -262,7 +262,7 @@ func (n *Node) internalRemove(recursive bool, callback func(path string)) { } if !n.IsPermanent() { - n.store.TTLKeyHeap.remove(n) + n.store.ttlKeyHeap.remove(n) } n.stopExpire <- true @@ -375,18 +375,18 @@ func (n *Node) UpdateTTL(expireTime time.Time) { if expireTime.IsZero() { // from ttl to permanent // remove from ttl heap - n.store.TTLKeyHeap.remove(n) + n.store.ttlKeyHeap.remove(n) } else { // update ttl // update ttl heap - n.store.TTLKeyHeap.update(n) + n.store.ttlKeyHeap.update(n) } } else { if !expireTime.IsZero() { // from permanent to ttl // push into ttl heap - n.store.TTLKeyHeap.push(n) + n.store.ttlKeyHeap.push(n) } } @@ -442,5 +442,9 @@ func (n *Node) recoverAndclean() { n.stopExpire = make(chan bool, 1) + if !n.ExpireTime.IsZero() { + n.store.ttlKeyHeap.push(n) + } + n.Expire() } diff --git a/store/store.go b/store/store.go index babb5a93288..f7abf50df11 100644 --- a/store/store.go +++ b/store/store.go @@ -37,11 +37,11 @@ type Store interface { type store struct { Root *Node WatcherHub *watcherHub - TTLKeyHeap *TTLKeyHeap Index uint64 Term uint64 Stats *Stats CurrentVersion int + ttlKeyHeap *ttlKeyHeap // need to recovery manually worldLock sync.RWMutex // stop the world lock } @@ -55,7 +55,7 @@ func newStore() *store { s.Root = newDir(s, "/", UndefIndex, UndefTerm, nil, "", Permanent) s.Stats = newStats() s.WatcherHub = newWatchHub(1000) - s.TTLKeyHeap = newTTLKeyHeap() + s.ttlKeyHeap = newTtlKeyHeap() return s } @@ -393,7 +393,7 @@ func (s *store) internalCreate(nodePath string, value string, unique bool, repla // Node with TTL if !n.IsPermanent() { - s.TTLKeyHeap.push(n) + s.ttlKeyHeap.push(n) n.Expire() e.Expiration, e.TTL = n.ExpirationAndTTL() @@ -477,6 +477,7 @@ func (s *store) Save() ([]byte, error) { b, err := json.Marshal(clonedStore) if err != nil { + fmt.Println(err) return nil, err } @@ -496,6 +497,8 @@ func (s *store) Recovery(state []byte) error { return err } + s.ttlKeyHeap = newTtlKeyHeap() + s.Root.recoverAndclean() return nil } diff --git a/store/ttl_key_heap.go b/store/ttl_key_heap.go index 34fa4ba63d2..28c38ea3138 100644 --- a/store/ttl_key_heap.go +++ b/store/ttl_key_heap.go @@ -5,66 +5,66 @@ import ( ) // An TTLKeyHeap is a min-heap of TTLKeys order by expiration time -type TTLKeyHeap struct { - Array []*Node - Map map[*Node]int +type ttlKeyHeap struct { + array []*Node + keyMap map[*Node]int } -func newTTLKeyHeap() *TTLKeyHeap { - h := &TTLKeyHeap{Map: make(map[*Node]int)} +func newTtlKeyHeap() *ttlKeyHeap { + h := &ttlKeyHeap{keyMap: make(map[*Node]int)} heap.Init(h) return h } -func (h TTLKeyHeap) Len() int { - return len(h.Array) +func (h ttlKeyHeap) Len() int { + return len(h.array) } -func (h TTLKeyHeap) Less(i, j int) bool { - return h.Array[i].ExpireTime.Before(h.Array[j].ExpireTime) +func (h ttlKeyHeap) Less(i, j int) bool { + return h.array[i].ExpireTime.Before(h.array[j].ExpireTime) } -func (h TTLKeyHeap) Swap(i, j int) { +func (h ttlKeyHeap) Swap(i, j int) { // swap node - h.Array[i], h.Array[j] = h.Array[j], h.Array[i] + h.array[i], h.array[j] = h.array[j], h.array[i] // update map - h.Map[h.Array[i]] = i - h.Map[h.Array[j]] = j + h.keyMap[h.array[i]] = i + h.keyMap[h.array[j]] = j } -func (h *TTLKeyHeap) Push(x interface{}) { +func (h *ttlKeyHeap) Push(x interface{}) { n, _ := x.(*Node) - h.Map[n] = len(h.Array) - h.Array = append(h.Array, n) + h.keyMap[n] = len(h.array) + h.array = append(h.array, n) } -func (h *TTLKeyHeap) Pop() interface{} { - old := h.Array +func (h *ttlKeyHeap) Pop() interface{} { + old := h.array n := len(old) x := old[n-1] - h.Array = old[0 : n-1] - delete(h.Map, x) + h.array = old[0 : n-1] + delete(h.keyMap, x) return x } -func (h *TTLKeyHeap) pop() *Node { +func (h *ttlKeyHeap) pop() *Node { x := heap.Pop(h) n, _ := x.(*Node) return n } -func (h *TTLKeyHeap) push(x interface{}) { +func (h *ttlKeyHeap) push(x interface{}) { heap.Push(h, x) } -func (h *TTLKeyHeap) update(n *Node) { - index := h.Map[n] +func (h *ttlKeyHeap) update(n *Node) { + index := h.keyMap[n] heap.Remove(h, index) heap.Push(h, n) } -func (h *TTLKeyHeap) remove(n *Node) { - index := h.Map[n] +func (h *ttlKeyHeap) remove(n *Node) { + index := h.keyMap[n] heap.Remove(h, index) } From 1d49098954434c1d8edefe09b071b39a3df78cfc Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 4 Nov 2013 22:13:26 -0800 Subject: [PATCH 179/247] feat add heap top --- store/store.go | 4 ++++ store/ttl_key_heap.go | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/store/store.go b/store/store.go index f7abf50df11..3fc8aaf2e13 100644 --- a/store/store.go +++ b/store/store.go @@ -457,6 +457,10 @@ func (s *store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) { return n, nil } +func (s *store) MonitorTTLKeys() { + +} + // Save function saves the static state of the store system. // Save function will not be able to save the state of watchers. // Save function will not save the parent field of the node. Or there will diff --git a/store/ttl_key_heap.go b/store/ttl_key_heap.go index 28c38ea3138..feb2ad50030 100644 --- a/store/ttl_key_heap.go +++ b/store/ttl_key_heap.go @@ -48,6 +48,10 @@ func (h *ttlKeyHeap) Pop() interface{} { return x } +func (h *ttlKeyHeap) top() *Node { + return h.array[0] +} + func (h *ttlKeyHeap) pop() *Node { x := heap.Pop(h) n, _ := x.(*Node) From 8c6606ed121beef4fb81b5ace71cd7aa1f190dc5 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Tue, 5 Nov 2013 15:57:53 -0700 Subject: [PATCH 180/247] Fix TTL migration issue. --- store/store.go | 13 +++++++++++++ tests/functional/v1_migration_test.go | 12 ++---------- 2 files changed, 15 insertions(+), 10 deletions(-) diff --git a/store/store.go b/store/store.go index 22d1e9017a1..4e5ea79fa10 100644 --- a/store/store.go +++ b/store/store.go @@ -16,6 +16,12 @@ import ( // The default version to set when the store is first initialized. const defaultVersion = 2 +var minExpireTime time.Time + +func init() { + minExpireTime, _ = time.Parse(time.RFC3339, "2000-01-01T00:00:00Z") +} + type Store interface { Version() int CommandFactory() CommandFactory @@ -344,6 +350,13 @@ func (s *store) internalCreate(nodePath string, value string, unique bool, repla nodePath = path.Clean(path.Join("/", nodePath)) + // Assume expire times that are way in the past are not valid. + // This can occur when the time is serialized to JSON and read back in. + if expireTime.Before(minExpireTime) { + expireTime = Permanent + } + + dir, newNodeName := path.Split(nodePath) // walk through the nodePath, create dirs and get the last directory node diff --git a/tests/functional/v1_migration_test.go b/tests/functional/v1_migration_test.go index a01d00c2039..c99f7711a0e 100644 --- a/tests/functional/v1_migration_test.go +++ b/tests/functional/v1_migration_test.go @@ -21,7 +21,7 @@ func TestV1SoloMigration(t *testing.T) { nodepath := filepath.Join(path, "node0") fixturepath, _ := filepath.Abs("../fixtures/v1.solo/node0") - + fmt.Println("DATA_DIR =", nodepath) // Copy over fixture files. c := exec.Command("cp", "-rf", fixturepath, nodepath) @@ -44,19 +44,11 @@ func TestV1SoloMigration(t *testing.T) { defer process.Kill() time.Sleep(time.Second) - time.Sleep(120 * time.Second) - // Ensure deleted message is removed. resp, err := tests.Get("http://localhost:4001/v2/keys/message") tests.ReadBody(resp) assert.Nil(t, err, "") - assert.Equal(t, resp.StatusCode, 404, "") - - // Ensure TTL'd message is removed. - resp, err = tests.Get("http://localhost:4001/v2/keys/foo") - tests.ReadBody(resp) - assert.Nil(t, err, "") - assert.Equal(t, resp.StatusCode, 404, "") + assert.Equal(t, resp.StatusCode, 200, "") } // Ensure that we can start a v2 cluster from the logs of a v1 cluster. From 797d996535f28c988cbc9d7a0b570e6cc0622906 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 5 Nov 2013 21:47:25 -0800 Subject: [PATCH 181/247] init sync to delete expiring keys --- store/node.go | 108 +----------------------------------------- store/stats_test.go | 16 +++---- store/store.go | 23 +++++++-- store/store_test.go | 26 +++++++--- store/ttl_key_heap.go | 19 +++++--- 5 files changed, 61 insertions(+), 131 deletions(-) diff --git a/store/node.go b/store/node.go index 38a9692038a..47874514f7d 100644 --- a/store/node.go +++ b/store/node.go @@ -37,10 +37,6 @@ type Node struct { // A reference to the store this node is attached to. store *store - // a ttl node will have an expire routine associated with it. - // we need a channel to stop that routine when the expiration changes. - stopExpire chan bool - // ensure we only delete the node once // expire and remove may try to delete a node twice once sync.Once @@ -59,7 +55,6 @@ func newKV(store *store, nodePath string, value string, createIndex uint64, Parent: parent, ACL: ACL, store: store, - stopExpire: make(chan bool, 1), ExpireTime: expireTime, Value: value, } @@ -75,7 +70,6 @@ func newDir(store *store, nodePath string, createIndex uint64, createTerm uint64 CreateTerm: createTerm, Parent: parent, ACL: ACL, - stopExpire: make(chan bool, 1), ExpireTime: expireTime, Children: make(map[string]*Node), store: store, @@ -98,20 +92,6 @@ func (n *Node) IsPermanent() bool { return n.ExpireTime.IsZero() } -// IsExpired function checks if the node has been expired. -func (n *Node) IsExpired() (bool, time.Duration) { - if n.IsPermanent() { - return false, 0 - } - - duration := n.ExpireTime.Sub(time.Now()) - if duration <= 0 { - return true, 0 - } - - return false, duration -} - // IsDir function checks whether the node is a directory. // If the node is a directory, the function will return true. // Otherwise the function will return false. @@ -214,19 +194,6 @@ func (n *Node) Remove(recursive bool, callback func(path string)) *etcdErr.Error return etcdErr.NewError(etcdErr.EcodeNotFile, "", UndefIndex, UndefTerm) } - onceBody := func() { - n.internalRemove(recursive, callback) - } - - // this function might be entered multiple times by expire and delete - // every node will only be deleted once. - n.once.Do(onceBody) - - return nil -} - -// internalRemove function will be called by remove() -func (n *Node) internalRemove(recursive bool, callback func(path string)) { if !n.IsDir() { // key-value pair _, name := path.Split(n.Path) @@ -243,9 +210,7 @@ func (n *Node) internalRemove(recursive bool, callback func(path string)) { n.store.ttlKeyHeap.remove(n) } - // the stop channel has a buffer. just send to it! - n.stopExpire <- true - return + return nil } for _, child := range n.Children { // delete all children @@ -265,61 +230,9 @@ func (n *Node) internalRemove(recursive bool, callback func(path string)) { n.store.ttlKeyHeap.remove(n) } - n.stopExpire <- true - } -} - -// Expire function will test if the node is expired. -// if the node is already expired, delete the node and return. -// if the node is permanent (this shouldn't happen), return at once. -// else wait for a period time, then remove the node. and notify the watchhub. -func (n *Node) Expire() { - expired, duration := n.IsExpired() - - if expired { // has been expired - // since the parent function of Expire() runs serially, - // there is no need for lock here - e := newEvent(Expire, n.Path, UndefIndex, UndefTerm) - n.store.WatcherHub.notify(e) - - n.Remove(true, nil) - n.store.Stats.Inc(ExpireCount) - - return } - if duration == 0 { // Permanent Node - return - } - - go func() { // do monitoring - select { - // if timeout, delete the node - case <-time.After(duration): - - // before expire get the lock, the expiration time - // of the node may be updated. - // we have to check again when get the lock - n.store.worldLock.Lock() - defer n.store.worldLock.Unlock() - - expired, _ := n.IsExpired() - - if expired { - e := newEvent(Expire, n.Path, UndefIndex, UndefTerm) - n.store.WatcherHub.notify(e) - - n.Remove(true, nil) - n.store.Stats.Inc(ExpireCount) - } - - return - - // if stopped, return - case <-n.stopExpire: - return - } - }() + return nil } func (n *Node) Pair(recurisive, sorted bool) KeyValuePair { @@ -390,21 +303,7 @@ func (n *Node) UpdateTTL(expireTime time.Time) { } } - if !n.IsPermanent() { - // check if the node has been expired - // if the node is not expired, we need to stop the go routine associated with - // that node. - expired, _ := n.IsExpired() - - if !expired { - n.stopExpire <- true // suspend it to modify the expiration - } - } - n.ExpireTime = expireTime - if !n.IsPermanent() { - n.Expire() - } } // Clone function clone the node recursively and return the new node. @@ -440,11 +339,8 @@ func (n *Node) recoverAndclean() { } } - n.stopExpire = make(chan bool, 1) - if !n.ExpireTime.IsZero() { n.store.ttlKeyHeap.push(n) } - n.Expire() } diff --git a/store/stats_test.go b/store/stats_test.go index b6247326450..212c56c0a40 100644 --- a/store/stats_test.go +++ b/store/stats_test.go @@ -2,7 +2,7 @@ package store import ( "testing" - "time" + //"time" "github.com/stretchr/testify/assert" ) @@ -85,10 +85,10 @@ func TestStoreStatsDeleteFail(t *testing.T) { } // Ensure that the number of expirations is recorded in the stats. -func TestStoreStatsExpireCount(t *testing.T) { - s := newStore() - s.Create("/foo", "bar", false, time.Now().Add(5 * time.Millisecond), 3, 1) - assert.Equal(t, uint64(0), s.Stats.ExpireCount, "") - time.Sleep(10 * time.Millisecond) - assert.Equal(t, uint64(1), s.Stats.ExpireCount, "") -} +// func TestStoreStatsExpireCount(t *testing.T) { +// s := newStore() +// s.Create("/foo", "bar", false, time.Now().Add(5 * time.Millisecond), 3, 1) +// assert.Equal(t, uint64(0), s.Stats.ExpireCount, "") +// time.Sleep(10 * time.Millisecond) +// assert.Equal(t, uint64(1), s.Stats.ExpireCount, "") +// } diff --git a/store/store.go b/store/store.go index 3fc8aaf2e13..39eb1618c05 100644 --- a/store/store.go +++ b/store/store.go @@ -395,7 +395,6 @@ func (s *store) internalCreate(nodePath string, value string, unique bool, repla if !n.IsPermanent() { s.ttlKeyHeap.push(n) - n.Expire() e.Expiration, e.TTL = n.ExpirationAndTTL() } @@ -435,6 +434,24 @@ func (s *store) internalGet(nodePath string, index uint64, term uint64) (*Node, return f, nil } +// deleteExpiredKyes will delete all +func (s *store) deleteExpiredKeys(cutoff time.Time) { + s.worldLock.Lock() + defer s.worldLock.Unlock() + + for { + node := s.ttlKeyHeap.top() + if node == nil || node.ExpireTime.After(cutoff) { + return + } + + s.ttlKeyHeap.pop() + node.Remove(true, nil) + + s.WatcherHub.notify(newEvent(Expire, node.Path, s.Index, s.Term)) + } +} + // checkDir function will check whether the component is a directory under parent node. // If it is a directory, this function will return the pointer to that node. // If it does not exist, this function will create a new directory and return the pointer to that node. @@ -457,10 +474,6 @@ func (s *store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) { return n, nil } -func (s *store) MonitorTTLKeys() { - -} - // Save function saves the static state of the store system. // Save function will not be able to save the state of watchers. // Save function will not save the parent field of the node. Or there will diff --git a/store/store_test.go b/store/store_test.go index 263e628ac3f..013656c76c7 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -142,12 +142,13 @@ func TestStoreUpdateFailsIfDirectory(t *testing.T) { // Ensure that the store can update the TTL on a value. func TestStoreUpdateValueTTL(t *testing.T) { s := newStore() + go mockSyncService(s.deleteExpiredKeys) s.Create("/foo", "bar", false, Permanent, 2, 1) - _, err := s.Update("/foo", "baz", time.Now().Add(1*time.Millisecond), 3, 1) + _, err := s.Update("/foo", "baz", time.Now().Add(500*time.Millisecond), 3, 1) e, _ := s.Get("/foo", false, false, 3, 1) assert.Equal(t, e.Value, "baz", "") - time.Sleep(2 * time.Millisecond) + time.Sleep(600 * time.Millisecond) e, err = s.Get("/foo", false, false, 3, 1) assert.Nil(t, e, "") assert.Equal(t, err.(*etcdErr.Error).ErrorCode, etcdErr.EcodeKeyNotFound, "") @@ -156,13 +157,14 @@ func TestStoreUpdateValueTTL(t *testing.T) { // Ensure that the store can update the TTL on a directory. func TestStoreUpdateDirTTL(t *testing.T) { s := newStore() + go mockSyncService(s.deleteExpiredKeys) s.Create("/foo", "", false, Permanent, 2, 1) s.Create("/foo/bar", "baz", false, Permanent, 3, 1) - _, err := s.Update("/foo", "", time.Now().Add(1*time.Millisecond), 3, 1) + _, err := s.Update("/foo", "", time.Now().Add(500*time.Millisecond), 3, 1) e, _ := s.Get("/foo/bar", false, false, 3, 1) assert.Equal(t, e.Value, "baz", "") - time.Sleep(2 * time.Millisecond) + time.Sleep(600 * time.Millisecond) e, err = s.Get("/foo/bar", false, false, 3, 1) assert.Nil(t, e, "") assert.Equal(t, err.(*etcdErr.Error).ErrorCode, etcdErr.EcodeKeyNotFound, "") @@ -340,11 +342,12 @@ func TestStoreWatchRecursiveCompareAndSwap(t *testing.T) { // Ensure that the store can watch for key expiration. func TestStoreWatchExpire(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, time.Now().Add(1*time.Millisecond), 2, 1) + go mockSyncService(s.deleteExpiredKeys) + s.Create("/foo", "bar", false, time.Now().Add(500*time.Millisecond), 2, 1) c, _ := s.Watch("/foo", false, 0, 0, 1) e := nbselect(c) assert.Nil(t, e, "") - time.Sleep(2 * time.Millisecond) + time.Sleep(600 * time.Millisecond) e = nbselect(c) assert.Equal(t, e.Action, "expire", "") assert.Equal(t, e.Key, "/foo", "") @@ -373,6 +376,7 @@ func TestStoreRecover(t *testing.T) { // Ensure that the store can recover from a previously saved state that includes an expiring key. func TestStoreRecoverWithExpiration(t *testing.T) { s := newStore() + go mockSyncService(s.deleteExpiredKeys) s.Create("/foo", "", false, Permanent, 2, 1) s.Create("/foo/x", "bar", false, Permanent, 3, 1) s.Create("/foo/y", "baz", false, time.Now().Add(5*time.Millisecond), 4, 1) @@ -381,8 +385,11 @@ func TestStoreRecoverWithExpiration(t *testing.T) { time.Sleep(10 * time.Millisecond) s2 := newStore() + go mockSyncService(s2.deleteExpiredKeys) s2.Recovery(b) + time.Sleep(600 * time.Millisecond) + e, err := s.Get("/foo/x", false, false, 4, 1) assert.Nil(t, err, "") assert.Equal(t, e.Value, "bar", "") @@ -401,3 +408,10 @@ func nbselect(c <-chan *Event) *Event { return nil } } + +func mockSyncService(f func(now time.Time)) { + ticker := time.Tick(time.Millisecond * 500) + for now := range ticker { + f(now) + } +} diff --git a/store/ttl_key_heap.go b/store/ttl_key_heap.go index feb2ad50030..0cda91d8dd0 100644 --- a/store/ttl_key_heap.go +++ b/store/ttl_key_heap.go @@ -49,7 +49,10 @@ func (h *ttlKeyHeap) Pop() interface{} { } func (h *ttlKeyHeap) top() *Node { - return h.array[0] + if h.Len() != 0 { + return h.array[0] + } + return nil } func (h *ttlKeyHeap) pop() *Node { @@ -63,12 +66,16 @@ func (h *ttlKeyHeap) push(x interface{}) { } func (h *ttlKeyHeap) update(n *Node) { - index := h.keyMap[n] - heap.Remove(h, index) - heap.Push(h, n) + index, ok := h.keyMap[n] + if ok { + heap.Remove(h, index) + heap.Push(h, n) + } } func (h *ttlKeyHeap) remove(n *Node) { - index := h.keyMap[n] - heap.Remove(h, index) + index, ok := h.keyMap[n] + if ok { + heap.Remove(h, index) + } } From 49c55477e567695b068cf1d94f52e1d437d4bf51 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 5 Nov 2013 21:56:21 -0800 Subject: [PATCH 182/247] refactor clean up --- store/node.go | 5 ----- store/stats_test.go | 19 ++++++++++--------- store/store.go | 1 + 3 files changed, 11 insertions(+), 14 deletions(-) diff --git a/store/node.go b/store/node.go index 47874514f7d..b3daa1dfc27 100644 --- a/store/node.go +++ b/store/node.go @@ -3,7 +3,6 @@ package store import ( "path" "sort" - "sync" "time" etcdErr "github.com/coreos/etcd/error" @@ -36,10 +35,6 @@ type Node struct { // A reference to the store this node is attached to. store *store - - // ensure we only delete the node once - // expire and remove may try to delete a node twice - once sync.Once } // newKV creates a Key-Value pair diff --git a/store/stats_test.go b/store/stats_test.go index 212c56c0a40..9b3172ad8a7 100644 --- a/store/stats_test.go +++ b/store/stats_test.go @@ -2,7 +2,7 @@ package store import ( "testing" - //"time" + "time" "github.com/stretchr/testify/assert" ) @@ -84,11 +84,12 @@ func TestStoreStatsDeleteFail(t *testing.T) { assert.Equal(t, uint64(1), s.Stats.DeleteFail, "") } -// Ensure that the number of expirations is recorded in the stats. -// func TestStoreStatsExpireCount(t *testing.T) { -// s := newStore() -// s.Create("/foo", "bar", false, time.Now().Add(5 * time.Millisecond), 3, 1) -// assert.Equal(t, uint64(0), s.Stats.ExpireCount, "") -// time.Sleep(10 * time.Millisecond) -// assert.Equal(t, uint64(1), s.Stats.ExpireCount, "") -// } +//Ensure that the number of expirations is recorded in the stats. +func TestStoreStatsExpireCount(t *testing.T) { + s := newStore() + go mockSyncService(s.deleteExpiredKeys) + s.Create("/foo", "bar", false, time.Now().Add(500*time.Millisecond), 3, 1) + assert.Equal(t, uint64(0), s.Stats.ExpireCount, "") + time.Sleep(600 * time.Millisecond) + assert.Equal(t, uint64(1), s.Stats.ExpireCount, "") +} diff --git a/store/store.go b/store/store.go index 39eb1618c05..abffabf741e 100644 --- a/store/store.go +++ b/store/store.go @@ -448,6 +448,7 @@ func (s *store) deleteExpiredKeys(cutoff time.Time) { s.ttlKeyHeap.pop() node.Remove(true, nil) + s.Stats.Inc(ExpireCount) s.WatcherHub.notify(newEvent(Expire, node.Path, s.Index, s.Term)) } } From 779195eb4fd741a2f29ddbd46531d2fe8a5db553 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 5 Nov 2013 22:18:54 -0800 Subject: [PATCH 183/247] fix bug in update ttl --- store/node.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/store/node.go b/store/node.go index b3daa1dfc27..33ae7669252 100644 --- a/store/node.go +++ b/store/node.go @@ -286,6 +286,7 @@ func (n *Node) UpdateTTL(expireTime time.Time) { n.store.ttlKeyHeap.remove(n) } else { // update ttl + n.ExpireTime = expireTime // update ttl heap n.store.ttlKeyHeap.update(n) } @@ -293,12 +294,11 @@ func (n *Node) UpdateTTL(expireTime time.Time) { } else { if !expireTime.IsZero() { // from permanent to ttl + n.ExpireTime = expireTime // push into ttl heap n.store.ttlKeyHeap.push(n) } } - - n.ExpireTime = expireTime } // Clone function clone the node recursively and return the new node. From 55058c64f55a340b1b78a4cfa55aaae9c6775051 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Tue, 5 Nov 2013 23:30:48 -0800 Subject: [PATCH 184/247] feat wathch for expiring need to be pending --- store/store.go | 2 ++ store/watcher_hub.go | 26 ++++++++++++++++++++------ 2 files changed, 22 insertions(+), 6 deletions(-) diff --git a/store/store.go b/store/store.go index abffabf741e..14d480b9ce4 100644 --- a/store/store.go +++ b/store/store.go @@ -451,6 +451,8 @@ func (s *store) deleteExpiredKeys(cutoff time.Time) { s.Stats.Inc(ExpireCount) s.WatcherHub.notify(newEvent(Expire, node.Path, s.Index, s.Term)) } + + s.WatcherHub.clearPendingWatchers() } // checkDir function will check whether the component is a directory under parent node. diff --git a/store/watcher_hub.go b/store/watcher_hub.go index 4c4bfd29d30..9e0285b47c4 100644 --- a/store/watcher_hub.go +++ b/store/watcher_hub.go @@ -16,9 +16,10 @@ import ( // event happens between the end of the first watch command and the start // of the second command. type watcherHub struct { - watchers map[string]*list.List - count int64 // current number of watchers. - EventHistory *EventHistory + watchers map[string]*list.List + count int64 // current number of watchers. + EventHistory *EventHistory + pendingWatchers *list.List } // newWatchHub creates a watchHub. The capacity determines how many events we will @@ -27,8 +28,9 @@ type watcherHub struct { // Ideally, it should smaller than 20K/s[max throughput] * 2 * 50ms[RTT] = 2000 func newWatchHub(capacity int) *watcherHub { return &watcherHub{ - watchers: make(map[string]*list.List), - EventHistory: newEventHistory(capacity), + watchers: make(map[string]*list.List), + EventHistory: newEventHistory(capacity), + pendingWatchers: list.New(), } } @@ -117,9 +119,13 @@ func (wh *watcherHub) notifyWatchers(e *Event, path string, deleted bool) { // if we successfully notify a watcher // we need to remove the watcher from the list // and decrease the counter - l.Remove(curr) atomic.AddInt64(&wh.count, -1) + + if e.Action == Expire { + wh.pendingWatchers.PushBack(w) + } + } else { // once there is a watcher in the list is not interested // in the event, we should keep the list in the map @@ -131,6 +137,14 @@ func (wh *watcherHub) notifyWatchers(e *Event, path string, deleted bool) { } } +func (wh *watcherHub) clearPendingWatchers() { + for e := wh.pendingWatchers.Front(); e != nil; e = e.Next() { + w, _ := e.Value.(*watcher) + w.eventChan <- nil + } + wh.pendingWatchers = list.New() +} + // clone function clones the watcherHub and return the cloned one. // only clone the static content. do not clone the current watchers. func (wh *watcherHub) clone() *watcherHub { From c307b6abcad43b022a368659d901a3669c5d9601 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Wed, 6 Nov 2013 21:19:37 -0800 Subject: [PATCH 185/247] fix watcher_hub --- store/event_history.go | 30 ++++++++++++------- store/event_test.go | 12 ++++---- store/store.go | 5 ++-- store/store_test.go | 11 +++++-- store/watcher.go | 1 - store/watcher_hub.go | 65 ++++++++++++++++++++++++++---------------- 6 files changed, 77 insertions(+), 47 deletions(-) diff --git a/store/event_history.go b/store/event_history.go index 3ddd3820690..ce21ec1fc5e 100644 --- a/store/event_history.go +++ b/store/event_history.go @@ -31,19 +31,14 @@ func (eh *EventHistory) addEvent(e *Event) *Event { eh.rwl.Lock() defer eh.rwl.Unlock() - var duped uint64 - - if e.Index == UndefIndex { - e.Index = eh.LastIndex - e.Term = eh.LastTerm - duped = 1 + if e.Index == eh.LastIndex { + eh.DupCnt += 1 } eh.Queue.insert(e) eh.LastIndex = e.Index eh.LastTerm = e.Term - eh.DupCnt += duped eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index @@ -52,7 +47,7 @@ func (eh *EventHistory) addEvent(e *Event) *Event { // scan function is enumerating events from the index in history and // stops till the first point where the key has identified prefix -func (eh *EventHistory) scan(prefix string, index uint64) (*Event, *etcdErr.Error) { +func (eh *EventHistory) scan(prefix string, index uint64) ([]*Event, *etcdErr.Error) { eh.rwl.RLock() defer eh.rwl.RUnlock() @@ -73,16 +68,29 @@ func (eh *EventHistory) scan(prefix string, index uint64) (*Event, *etcdErr.Erro i := int((start + uint64(eh.Queue.Front)) % uint64(eh.Queue.Capacity)) + events := make([]*Event, 0) + var eventIndex uint64 + for { e := eh.Queue.Events[i] + + if eventIndex != 0 && eventIndex != e.Index { + return events, nil + } + if strings.HasPrefix(e.Key, prefix) && index <= e.Index { // make sure we bypass the smaller one - return e, nil + eventIndex = e.Index + events = append(events, e) } i = (i + 1) % eh.Queue.Capacity - if i == eh.Queue.back() { // find nothing, return and watch from current index - return nil, nil + if i == eh.Queue.back() { + if eventIndex == 0 { // find nothing, return and watch from current index + return nil, nil + } + + return events, nil } } } diff --git a/store/event_test.go b/store/event_test.go index c02a4d70ef0..aedf7f7da15 100644 --- a/store/event_test.go +++ b/store/event_test.go @@ -42,20 +42,20 @@ func TestScanHistory(t *testing.T) { eh.addEvent(newEvent(Create, "/foo/foo/foo", 5, 1)) e, err := eh.scan("/foo", 1) - if err != nil || e.Index != 1 { - t.Fatalf("scan error [/foo] [1] %v", e.Index) + if err != nil || e[0].Index != 1 { + t.Fatalf("scan error [/foo] [1] %v", e[0].Index) } e, err = eh.scan("/foo/bar", 1) - if err != nil || e.Index != 2 { - t.Fatalf("scan error [/foo/bar] [2] %v", e.Index) + if err != nil || e[0].Index != 2 { + t.Fatalf("scan error [/foo/bar] [2] %v", e[0].Index) } e, err = eh.scan("/foo/bar", 3) - if err != nil || e.Index != 4 { - t.Fatalf("scan error [/foo/bar/bar] [4] %v", e.Index) + if err != nil || e[0].Index != 4 { + t.Fatalf("scan error [/foo/bar/bar] [4] %v", e[0].Index) } e, err = eh.scan("/foo/bar", 6) diff --git a/store/store.go b/store/store.go index 14d480b9ce4..6190e354d70 100644 --- a/store/store.go +++ b/store/store.go @@ -435,10 +435,12 @@ func (s *store) internalGet(nodePath string, index uint64, term uint64) (*Node, } // deleteExpiredKyes will delete all -func (s *store) deleteExpiredKeys(cutoff time.Time) { +func (s *store) deleteExpiredKeys(cutoff time.Time, index uint64, term uint64) { s.worldLock.Lock() defer s.worldLock.Unlock() + s.Index, s.Term = index, term + for { node := s.ttlKeyHeap.top() if node == nil || node.ExpireTime.After(cutoff) { @@ -497,7 +499,6 @@ func (s *store) Save() ([]byte, error) { b, err := json.Marshal(clonedStore) if err != nil { - fmt.Println(err) return nil, err } diff --git a/store/store_test.go b/store/store_test.go index 013656c76c7..1fc242d914e 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -344,13 +344,18 @@ func TestStoreWatchExpire(t *testing.T) { s := newStore() go mockSyncService(s.deleteExpiredKeys) s.Create("/foo", "bar", false, time.Now().Add(500*time.Millisecond), 2, 1) - c, _ := s.Watch("/foo", false, 0, 0, 1) + s.Create("/foofoo", "barbarbar", false, time.Now().Add(500*time.Millisecond), 2, 1) + + c, _ := s.Watch("/", true, 0, 0, 1) e := nbselect(c) assert.Nil(t, e, "") time.Sleep(600 * time.Millisecond) e = nbselect(c) assert.Equal(t, e.Action, "expire", "") assert.Equal(t, e.Key, "/foo", "") + e = nbselect(c) + assert.Equal(t, e.Action, "expire", "") + assert.Equal(t, e.Key, "/foofoo", "") } // Ensure that the store can recover from a previously saved state. @@ -409,9 +414,9 @@ func nbselect(c <-chan *Event) *Event { } } -func mockSyncService(f func(now time.Time)) { +func mockSyncService(f func(now time.Time, index uint64, term uint64)) { ticker := time.Tick(time.Millisecond * 500) for now := range ticker { - f(now) + f(now, 2, 1) } } diff --git a/store/watcher.go b/store/watcher.go index b9cb5499ca7..2015d0072da 100644 --- a/store/watcher.go +++ b/store/watcher.go @@ -24,7 +24,6 @@ func (w *watcher) notify(e *Event, originalPath bool, deleted bool) bool { // at the file we need to delete. // For example a watcher is watching at "/foo/bar". And we deletes "/foo". The watcher // should get notified even if "/foo" is not the path it is watching. - if (w.recursive || originalPath || deleted) && e.Index >= w.sinceIndex { w.eventChan <- e return true diff --git a/store/watcher_hub.go b/store/watcher_hub.go index 9e0285b47c4..33eda248ef6 100644 --- a/store/watcher_hub.go +++ b/store/watcher_hub.go @@ -19,7 +19,8 @@ type watcherHub struct { watchers map[string]*list.List count int64 // current number of watchers. EventHistory *EventHistory - pendingWatchers *list.List + pendingWatchers map[*list.Element]*list.List + pendingList map[*list.List]string } // newWatchHub creates a watchHub. The capacity determines how many events we will @@ -30,7 +31,8 @@ func newWatchHub(capacity int) *watcherHub { return &watcherHub{ watchers: make(map[string]*list.List), EventHistory: newEventHistory(capacity), - pendingWatchers: list.New(), + pendingWatchers: make(map[*list.Element]*list.List), + pendingList: make(map[*list.List]string), } } @@ -39,23 +41,30 @@ func newWatchHub(capacity int) *watcherHub { // If recursive is false, the first change after index at prefix will be sent to the event channel. // If index is zero, watch will start from the current index + 1. func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (<-chan *Event, *etcdErr.Error) { - eventChan := make(chan *Event, 1) - - e, err := wh.EventHistory.scan(prefix, index) + events, err := wh.EventHistory.scan(prefix, index) if err != nil { return nil, err } - if e != nil { - eventChan <- e + eventChan := make(chan *Event, len(events)+5) // use a buffered channel + + if events != nil { + for _, e := range events { + eventChan <- e + } + + if len(events) > 1 { + eventChan <- nil + } + return eventChan, nil } w := &watcher{ eventChan: eventChan, recursive: recursive, - sinceIndex: index - 1, // to catch Expire() + sinceIndex: index, } l, ok := wh.watchers[prefix] @@ -95,19 +104,16 @@ func (wh *watcherHub) notify(e *Event) { func (wh *watcherHub) notifyWatchers(e *Event, path string, deleted bool) { l, ok := wh.watchers[path] - if ok { curr := l.Front() - notifiedAll := true for { if curr == nil { // we have reached the end of the list - if notifiedAll { + if l.Len() == 0 { // if we have notified all watcher in the list // we can delete the list delete(wh.watchers, path) } - break } @@ -116,20 +122,18 @@ func (wh *watcherHub) notifyWatchers(e *Event, path string, deleted bool) { w, _ := curr.Value.(*watcher) if w.notify(e, e.Key == path, deleted) { - // if we successfully notify a watcher - // we need to remove the watcher from the list - // and decrease the counter - l.Remove(curr) - atomic.AddInt64(&wh.count, -1) if e.Action == Expire { - wh.pendingWatchers.PushBack(w) + wh.pendingWatchers[curr] = l + wh.pendingList[l] = path + } else { + // if we successfully notify a watcher + // we need to remove the watcher from the list + // and decrease the counter + l.Remove(curr) + atomic.AddInt64(&wh.count, -1) } - } else { - // once there is a watcher in the list is not interested - // in the event, we should keep the list in the map - notifiedAll = false } curr = next // update current to the next @@ -138,11 +142,24 @@ func (wh *watcherHub) notifyWatchers(e *Event, path string, deleted bool) { } func (wh *watcherHub) clearPendingWatchers() { - for e := wh.pendingWatchers.Front(); e != nil; e = e.Next() { + if len(wh.pendingWatchers) == 0 { // avoid making new maps + return + } + + for e, l := range wh.pendingWatchers { + l.Remove(e) + + if l.Len() == 0 { + path := wh.pendingList[l] + delete(wh.watchers, path) + } + w, _ := e.Value.(*watcher) w.eventChan <- nil } - wh.pendingWatchers = list.New() + + wh.pendingWatchers = make(map[*list.Element]*list.List) + wh.pendingList = make(map[*list.List]string) } // clone function clones the watcherHub and return the cloned one. From 4c1d86409503d20b6661cfd78a05463c78057586 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Wed, 6 Nov 2013 21:27:39 -0800 Subject: [PATCH 186/247] fix cleanup --- store/store.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/store/store.go b/store/store.go index 6190e354d70..debf1192eaa 100644 --- a/store/store.go +++ b/store/store.go @@ -444,7 +444,7 @@ func (s *store) deleteExpiredKeys(cutoff time.Time, index uint64, term uint64) { for { node := s.ttlKeyHeap.top() if node == nil || node.ExpireTime.After(cutoff) { - return + break } s.ttlKeyHeap.pop() @@ -453,7 +453,6 @@ func (s *store) deleteExpiredKeys(cutoff time.Time, index uint64, term uint64) { s.Stats.Inc(ExpireCount) s.WatcherHub.notify(newEvent(Expire, node.Path, s.Index, s.Term)) } - s.WatcherHub.clearPendingWatchers() } From 28ac516f68be7d715eb47df889c8abea9bd721f0 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Thu, 7 Nov 2013 20:55:26 -0800 Subject: [PATCH 187/247] fix sinceIndex --- server/v2/tests/get_handler_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/v2/tests/get_handler_test.go b/server/v2/tests/get_handler_test.go index d6ceae58a0f..3b365872cb1 100644 --- a/server/v2/tests/get_handler_test.go +++ b/server/v2/tests/get_handler_test.go @@ -118,7 +118,7 @@ func TestV2WatchKeyWithIndex(t *testing.T) { var body map[string]interface{} c := make(chan bool) go func() { - resp, _ := tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar?wait=true&waitIndex=5")) + resp, _ := tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar?wait=true&waitIndex=4")) body = tests.ReadBodyJSON(resp) c <- true }() From b4f4528ef46bbf6dd291697375953b5fb9c3945c Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Thu, 7 Nov 2013 22:29:15 -0800 Subject: [PATCH 188/247] feat upgrade get_handler --- server/v2/get_handler.go | 26 +++++++++++++++++++++----- 1 file changed, 21 insertions(+), 5 deletions(-) diff --git a/server/v2/get_handler.go b/server/v2/get_handler.go index 39e932fe09b..afa44c9b858 100644 --- a/server/v2/get_handler.go +++ b/server/v2/get_handler.go @@ -16,6 +16,7 @@ import ( func GetHandler(w http.ResponseWriter, req *http.Request, s Server) error { var err error var event *store.Event + events := make([]*store.Event, 0) vars := mux.Vars(req) key := "/" + vars["key"] @@ -54,10 +55,17 @@ func GetHandler(w http.ResponseWriter, req *http.Request, s Server) error { cn, _ := w.(http.CloseNotifier) closeChan := cn.CloseNotify() - select { - case <-closeChan: - return nil - case event = <-eventChan: + for { + select { + case <-closeChan: + return nil + case event = <-eventChan: + if event != nil && event.Action == store.Expire { + events = append(events, event) + } else { + goto finish + } + } } } else { //get @@ -68,11 +76,19 @@ func GetHandler(w http.ResponseWriter, req *http.Request, s Server) error { } } +finish: + w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) w.Header().Add("X-Etcd-Term", fmt.Sprint(event.Term)) w.WriteHeader(http.StatusOK) - b, _ := json.Marshal(event) + var b []byte + + if len(events) == 0 { + b, _ = json.Marshal(event) + } else { + b, _ = json.Marshal(events) + } w.Write(b) return nil From 5a4e764d7a2f39ea5b0984dec24775c4d8bdc71b Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Fri, 8 Nov 2013 13:24:23 -0800 Subject: [PATCH 189/247] refactor add comments for receiving expire commands --- server/v2/get_handler.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/server/v2/get_handler.go b/server/v2/get_handler.go index afa44c9b858..212c3229797 100644 --- a/server/v2/get_handler.go +++ b/server/v2/get_handler.go @@ -60,6 +60,9 @@ func GetHandler(w http.ResponseWriter, req *http.Request, s Server) error { case <-closeChan: return nil case event = <-eventChan: + // for events other than expire, just one event for one watch + // for expire event, we might have a stream of events + // we use a nil item to terminate the expire event stream if event != nil && event.Action == store.Expire { events = append(events, event) } else { From acd940a450dc620d9b3484a8f0369a17d5a394ba Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Fri, 8 Nov 2013 13:37:30 -0800 Subject: [PATCH 190/247] refactor comments on IsPermanent --- store/node.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/store/node.go b/store/node.go index 33ae7669252..6b11727c88f 100644 --- a/store/node.go +++ b/store/node.go @@ -84,6 +84,9 @@ func (n *Node) IsHidden() bool { // IsPermanent function checks if the node is a permanent one. func (n *Node) IsPermanent() bool { + // we use a uninitialized time.Time to indicate the node is a + // permanent one. + // the uninitialized time.Time should equal zero. return n.ExpireTime.IsZero() } From 6a074d4f6f830abc4237425a9df58735126cbdbb Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Mon, 7 Oct 2013 09:44:51 -0700 Subject: [PATCH 191/247] chore(*.go): add copyright notice --- .header | 20 ++++++++++++++++++++ error/error.go | 16 ++++++++++++++++ etcd.go | 16 ++++++++++++++++ server/transporter_test.go | 16 ++++++++++++++++ store/stats.go | 16 ++++++++++++++++ store/store.go | 16 ++++++++++++++++ store/store_test.go | 16 ++++++++++++++++ store/watcher.go | 16 ++++++++++++++++ store/watcher_test.go | 16 ++++++++++++++++ tests/functional/util.go | 16 ++++++++++++++++ 10 files changed, 164 insertions(+) create mode 100644 .header diff --git a/.header b/.header new file mode 100644 index 00000000000..21b983b6246 --- /dev/null +++ b/.header @@ -0,0 +1,20 @@ +/* +Copyright 2013 CoreOS Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package x + +import ( +) diff --git a/error/error.go b/error/error.go index 22b55906b04..c70d84922b9 100644 --- a/error/error.go +++ b/error/error.go @@ -1,3 +1,19 @@ +/* +Copyright 2013 CoreOS Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package error import ( diff --git a/etcd.go b/etcd.go index 6f79d39e7b5..16124db6d6f 100644 --- a/etcd.go +++ b/etcd.go @@ -1,3 +1,19 @@ +/* +Copyright 2013 CoreOS Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package main import ( diff --git a/server/transporter_test.go b/server/transporter_test.go index 989b2e25020..394841729f0 100644 --- a/server/transporter_test.go +++ b/server/transporter_test.go @@ -1,3 +1,19 @@ +/* +Copyright 2013 CoreOS Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package server /* diff --git a/store/stats.go b/store/stats.go index 40dff117537..2eb271d73b8 100644 --- a/store/stats.go +++ b/store/stats.go @@ -1,3 +1,19 @@ +/* +Copyright 2013 CoreOS Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package store import ( diff --git a/store/store.go b/store/store.go index f752f6935b4..663edd31d4b 100644 --- a/store/store.go +++ b/store/store.go @@ -1,3 +1,19 @@ +/* +Copyright 2013 CoreOS Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package store import ( diff --git a/store/store_test.go b/store/store_test.go index 263e628ac3f..2cce09b54a6 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -1,3 +1,19 @@ +/* +Copyright 2013 CoreOS Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package store import ( diff --git a/store/watcher.go b/store/watcher.go index b9cb5499ca7..39933db187d 100644 --- a/store/watcher.go +++ b/store/watcher.go @@ -1,3 +1,19 @@ +/* +Copyright 2013 CoreOS Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package store type watcher struct { diff --git a/store/watcher_test.go b/store/watcher_test.go index c3da475fca7..102d20afb71 100644 --- a/store/watcher_test.go +++ b/store/watcher_test.go @@ -1,3 +1,19 @@ +/* +Copyright 2013 CoreOS Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package store import ( diff --git a/tests/functional/util.go b/tests/functional/util.go index 3ef0028cdde..fd2f3c8fe9e 100644 --- a/tests/functional/util.go +++ b/tests/functional/util.go @@ -1,3 +1,19 @@ +/* +Copyright 2013 CoreOS Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + package test import ( From bf5f0ab3f26b96e96483dd042f1e810297428ded Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Fri, 27 Sep 2013 16:49:03 -0700 Subject: [PATCH 192/247] feat(dashboard/README): initial commit --- dashboard/README.md | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 dashboard/README.md diff --git a/dashboard/README.md b/dashboard/README.md new file mode 100644 index 00000000000..a87b7eecbd1 --- /dev/null +++ b/dashboard/README.md @@ -0,0 +1,5 @@ +This directory holds the frontend for the etcd dashboard. To have etcd serve from this directory run: + +``` +ETCD_DASHBOARD_DIR=`pwd`/dashboard ./etcd +``` From 53678ad1345cf67d882f652fa1111836fe353b77 Mon Sep 17 00:00:00 2001 From: Rob Szumski Date: Mon, 7 Oct 2013 16:15:06 -0700 Subject: [PATCH 193/247] feat(dashboard): Initial commit of frontend code --- dashboard/.bowerrc | 3 + dashboard/.editorconfig | 21 + dashboard/.gitattributes | 1 + dashboard/.gitignore | 5 + dashboard/.jshintignore | 3 + dashboard/.jshintrc | 27 + dashboard/.travis.yml | 7 + dashboard/Gruntfile.js | 345 + dashboard/LICENSE | 202 + dashboard/README.md | 12 +- dashboard/app/.buildignore | 1 + dashboard/app/browser.html | 51 + dashboard/app/index.html | 134 + dashboard/app/scripts/common/services/etcd.js | 81 + dashboard/app/scripts/controllers/browser.js | 191 + dashboard/app/scripts/controllers/stats.js | 181 + dashboard/app/scripts/ng-time-relative.min.js | 1 + dashboard/app/scripts/vega.js | 6970 +++++++++++++++++ dashboard/app/stats.html | 51 + dashboard/app/styles/bootstrap.css | 6167 +++++++++++++++ dashboard/app/styles/etcd-widgets.css | 694 ++ dashboard/app/styles/main.css | 22 + dashboard/app/views/browser.html | 99 + dashboard/app/views/stats.html | 46 + dashboard/bower.json | 20 + dashboard/build | 10 + dashboard/karma-e2e.conf.js | 54 + dashboard/karma.conf.js | 52 + dashboard/package.json | 38 + dashboard/test/.jshintrc | 35 + dashboard/test/runner.html | 10 + dashboard/test/spec/controllers/main.js | 22 + 32 files changed, 15554 insertions(+), 2 deletions(-) create mode 100644 dashboard/.bowerrc create mode 100644 dashboard/.editorconfig create mode 100644 dashboard/.gitattributes create mode 100644 dashboard/.gitignore create mode 100644 dashboard/.jshintignore create mode 100644 dashboard/.jshintrc create mode 100644 dashboard/.travis.yml create mode 100644 dashboard/Gruntfile.js create mode 100644 dashboard/LICENSE create mode 100644 dashboard/app/.buildignore create mode 100644 dashboard/app/browser.html create mode 100644 dashboard/app/index.html create mode 100644 dashboard/app/scripts/common/services/etcd.js create mode 100644 dashboard/app/scripts/controllers/browser.js create mode 100644 dashboard/app/scripts/controllers/stats.js create mode 100644 dashboard/app/scripts/ng-time-relative.min.js create mode 100644 dashboard/app/scripts/vega.js create mode 100644 dashboard/app/stats.html create mode 100644 dashboard/app/styles/bootstrap.css create mode 100644 dashboard/app/styles/etcd-widgets.css create mode 100644 dashboard/app/styles/main.css create mode 100644 dashboard/app/views/browser.html create mode 100644 dashboard/app/views/stats.html create mode 100644 dashboard/bower.json create mode 100755 dashboard/build create mode 100644 dashboard/karma-e2e.conf.js create mode 100644 dashboard/karma.conf.js create mode 100644 dashboard/package.json create mode 100644 dashboard/test/.jshintrc create mode 100644 dashboard/test/runner.html create mode 100644 dashboard/test/spec/controllers/main.js diff --git a/dashboard/.bowerrc b/dashboard/.bowerrc new file mode 100644 index 00000000000..ba0accc5a30 --- /dev/null +++ b/dashboard/.bowerrc @@ -0,0 +1,3 @@ +{ + "directory": "app/bower_components" +} diff --git a/dashboard/.editorconfig b/dashboard/.editorconfig new file mode 100644 index 00000000000..c2cdfb8ada2 --- /dev/null +++ b/dashboard/.editorconfig @@ -0,0 +1,21 @@ +# EditorConfig helps developers define and maintain consistent +# coding styles between different editors and IDEs +# editorconfig.org + +root = true + + +[*] + +# Change these settings to your own preference +indent_style = space +indent_size = 2 + +# We recommend you to keep these unchanged +end_of_line = lf +charset = utf-8 +trim_trailing_whitespace = true +insert_final_newline = true + +[*.md] +trim_trailing_whitespace = false diff --git a/dashboard/.gitattributes b/dashboard/.gitattributes new file mode 100644 index 00000000000..2125666142e --- /dev/null +++ b/dashboard/.gitattributes @@ -0,0 +1 @@ +* text=auto \ No newline at end of file diff --git a/dashboard/.gitignore b/dashboard/.gitignore new file mode 100644 index 00000000000..7911b28d575 --- /dev/null +++ b/dashboard/.gitignore @@ -0,0 +1,5 @@ +node_modules +dist +.tmp +.sass-cache +app/bower_components diff --git a/dashboard/.jshintignore b/dashboard/.jshintignore new file mode 100644 index 00000000000..ddbd3d2bc29 --- /dev/null +++ b/dashboard/.jshintignore @@ -0,0 +1,3 @@ +app/scripts/vega.js +app/scripts/moment.min.js +app/scripts/ng-time-relative.min.js diff --git a/dashboard/.jshintrc b/dashboard/.jshintrc new file mode 100644 index 00000000000..950c65247a5 --- /dev/null +++ b/dashboard/.jshintrc @@ -0,0 +1,27 @@ +{ + "node": true, + "browser": true, + "esnext": true, + "bitwise": true, + "camelcase": true, + "curly": true, + "eqeqeq": true, + "immed": true, + "indent": 2, + "latedef": true, + "newcap": true, + "noarg": true, + "quotmark": "single", + "regexp": true, + "undef": true, + "unused": false, + "strict": true, + "trailing": true, + "smarttabs": true, + "globals": { + "angular": false, + "$": false, + "vg": false, + "moment": false + } +} diff --git a/dashboard/.travis.yml b/dashboard/.travis.yml new file mode 100644 index 00000000000..83f4e22f0af --- /dev/null +++ b/dashboard/.travis.yml @@ -0,0 +1,7 @@ +language: node_js +node_js: + - '0.8' + - '0.10' +before_script: + - 'npm install -g bower grunt-cli' + - 'bower install' diff --git a/dashboard/Gruntfile.js b/dashboard/Gruntfile.js new file mode 100644 index 00000000000..39870153ad9 --- /dev/null +++ b/dashboard/Gruntfile.js @@ -0,0 +1,345 @@ +// Generated on 2013-10-07 using generator-webapp 0.4.3 +'use strict'; + +// # Globbing +// for performance reasons we're only matching one level down: +// 'test/spec/{,*/}*.js' +// use this if you want to recursively match all subfolders: +// 'test/spec/**/*.js' + +module.exports = function (grunt) { + // show elapsed time at the end + require('time-grunt')(grunt); + // load all grunt tasks + require('load-grunt-tasks')(grunt); + + grunt.initConfig({ + // configurable paths + uglify: { + options: { + mangle: false + }, + }, + yeoman: { + app: 'app', + dist: 'dist' + }, + watch: { + compass: { + files: ['<%= yeoman.app %>/styles/{,*/}*.{scss,sass}'], + tasks: ['compass:server', 'autoprefixer'] + }, + styles: { + files: ['<%= yeoman.app %>/styles/{,*/}*.css'], + tasks: ['copy:styles', 'autoprefixer'] + }, + livereload: { + options: { + livereload: '<%= connect.options.livereload %>' + }, + files: [ + '<%= yeoman.app %>/*.html', + '.tmp/styles/{,*/}*.css', + '{.tmp,<%= yeoman.app %>}/scripts/{,*/}*.js', + '<%= yeoman.app %>/images/{,*/}*.{png,jpg,jpeg,gif,webp,svg}' + ] + } + }, + connect: { + options: { + port: 9000, + livereload: 35729, + // change this to '0.0.0.0' to access the server from outside + hostname: 'localhost' + }, + livereload: { + options: { + open: true, + base: [ + '.tmp', + '<%= yeoman.app %>' + ] + } + }, + test: { + options: { + base: [ + '.tmp', + 'test', + '<%= yeoman.app %>' + ] + } + }, + dist: { + options: { + open: true, + base: '<%= yeoman.dist %>' + } + } + }, + clean: { + dist: { + files: [{ + dot: true, + src: [ + '.tmp', + '<%= yeoman.dist %>/*', + '!<%= yeoman.dist %>/.git*' + ] + }] + }, + server: '.tmp' + }, + jshint: { + options: { + jshintrc: '.jshintrc' + }, + all: [ + '<%= yeoman.app %>/scripts/{,*/}*.js', + '!<%= yeoman.app %>/scripts/vendor/*', + ] + }, + mocha: { + all: { + options: { + run: true, + urls: ['http://<%= connect.test.options.hostname %>:<%= connect.test.options.port %>/index.html'] + } + } + }, + compass: { + options: { + sassDir: '<%= yeoman.app %>/styles', + cssDir: '.tmp/styles', + generatedImagesDir: '.tmp/images/generated', + imagesDir: '<%= yeoman.app %>/images', + javascriptsDir: '<%= yeoman.app %>/scripts', + fontsDir: '<%= yeoman.app %>/styles/fonts', + importPath: '<%= yeoman.app %>/bower_components', + httpImagesPath: '/images', + httpGeneratedImagesPath: '/images/generated', + httpFontsPath: '/styles/fonts', + relativeAssets: false, + assetCacheBuster: false + }, + dist: { + options: { + generatedImagesDir: '<%= yeoman.dist %>/images/generated' + } + }, + server: { + options: { + debugInfo: true + } + } + }, + autoprefixer: { + options: { + browsers: ['last 1 version'] + }, + dist: { + files: [{ + expand: true, + cwd: '.tmp/styles/', + src: '{,*/}*.css', + dest: '.tmp/styles/' + }] + } + }, + // not used since Uglify task does concat, + // but still available if needed + /*concat: { + dist: {} + },*/ + requirejs: { + dist: { + // Options: https://github.com/jrburke/r.js/blob/master/build/example.build.js + options: { + // `name` and `out` is set by grunt-usemin + baseUrl: '<%= yeoman.app %>/scripts', + optimize: 'none', + // TODO: Figure out how to make sourcemaps work with grunt-usemin + // https://github.com/yeoman/grunt-usemin/issues/30 + //generateSourceMaps: true, + // required to support SourceMaps + // http://requirejs.org/docs/errors.html#sourcemapcomments + preserveLicenseComments: false, + useStrict: true, + wrap: true + //uglify2: {} // https://github.com/mishoo/UglifyJS2 + } + } + }, + useminPrepare: { + options: { + dest: '<%= yeoman.dist %>' + }, + html: ['<%= yeoman.app %>/**/*.html'] + }, + usemin: { + options: { + dirs: ['<%= yeoman.dist %>'] + }, + html: ['<%= yeoman.dist %>/{,*/}*.html'], + css: ['<%= yeoman.dist %>/styles/{,*/}*.css'] + }, + imagemin: { + dist: { + files: [{ + expand: true, + cwd: '<%= yeoman.app %>/images', + src: '{,*/}*.{png,jpg,jpeg}', + dest: '<%= yeoman.dist %>/images' + }] + } + }, + svgmin: { + dist: { + files: [{ + expand: true, + cwd: '<%= yeoman.app %>/images', + src: '{,*/}*.svg', + dest: '<%= yeoman.dist %>/images' + }] + } + }, + cssmin: { + // This task is pre-configured if you do not wish to use Usemin + // blocks for your CSS. By default, the Usemin block from your + // `index.html` will take care of minification, e.g. + // + // + // + // dist: { + // files: { + // '<%= yeoman.dist %>/styles/main.css': [ + // '.tmp/styles/{,*/}*.css', + // '<%= yeoman.app %>/styles/{,*/}*.css' + // ] + // } + // } + }, + htmlmin: { + dist: { + options: { + /*removeCommentsFromCDATA: true, + // https://github.com/yeoman/grunt-usemin/issues/44 + //collapseWhitespace: true, + collapseBooleanAttributes: true, + removeAttributeQuotes: true, + removeRedundantAttributes: true, + useShortDoctype: true, + removeEmptyAttributes: true, + removeOptionalTags: true*/ + }, + files: [{ + expand: true, + cwd: '<%= yeoman.app %>', + src: '*.html', + dest: '<%= yeoman.dist %>' + }] + } + }, + // Put files not handled in other tasks here + copy: { + dist: { + files: [{ + expand: true, + dot: true, + cwd: '<%= yeoman.app %>', + dest: '<%= yeoman.dist %>', + src: [ + '*.{ico,png,txt}', + '.htaccess', + 'images/{,*/}*.{webp,gif}', + 'styles/fonts/{,*/}*.*', + 'views/*.*', + 'index.html', + 'bower_components/sass-bootstrap/fonts/*.*' + ] + }] + }, + styles: { + expand: true, + dot: true, + cwd: '<%= yeoman.app %>/styles', + dest: '.tmp/styles/', + src: '{,*/}*.css' + } + }, + modernizr: { + devFile: '<%= yeoman.app %>/bower_components/modernizr/modernizr.js', + outputFile: '<%= yeoman.dist %>/bower_components/modernizr/modernizr.js', + files: [ + '<%= yeoman.dist %>/scripts/{,*/}*.js', + '<%= yeoman.dist %>/styles/{,*/}*.css', + '!<%= yeoman.dist %>/scripts/vendor/*' + ], + uglify: true + }, + concurrent: { + server: [ + 'compass', + 'copy:styles' + ], + test: [ + 'copy:styles' + ], + dist: [ + 'compass', + 'copy:styles', + 'imagemin', + 'svgmin', + 'htmlmin' + ] + }, + bower: { + options: { + exclude: ['modernizr'] + }, + all: { + rjsConfig: '<%= yeoman.app %>/scripts/main.js' + } + } + }); + + grunt.registerTask('server', function (target) { + if (target === 'dist') { + return grunt.task.run(['build', 'connect:dist:keepalive']); + } + + grunt.task.run([ + 'clean:server', + 'concurrent:server', + 'autoprefixer', + 'connect:livereload', + 'watch' + ]); + }); + + grunt.registerTask('test', [ + 'clean:server', + 'concurrent:test', + 'autoprefixer', + 'connect:test', + 'mocha' + ]); + + grunt.registerTask('build', [ + 'clean:dist', + 'useminPrepare', + 'concurrent:dist', + 'autoprefixer', + 'concat', + 'cssmin', + 'uglify', + 'usemin', + 'copy:dist' + ]); + + grunt.registerTask('default', [ + 'jshint', + 'test', + 'build' + ]); +}; diff --git a/dashboard/LICENSE b/dashboard/LICENSE new file mode 100644 index 00000000000..d6456956733 --- /dev/null +++ b/dashboard/LICENSE @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/dashboard/README.md b/dashboard/README.md index a87b7eecbd1..45f27af950e 100644 --- a/dashboard/README.md +++ b/dashboard/README.md @@ -1,5 +1,13 @@ -This directory holds the frontend for the etcd dashboard. To have etcd serve from this directory run: +# etcd Dashboard + +## Developing + +### Install yeoman + +http://yeoman.io/ + +### Hacking ``` -ETCD_DASHBOARD_DIR=`pwd`/dashboard ./etcd +grunt server ``` diff --git a/dashboard/app/.buildignore b/dashboard/app/.buildignore new file mode 100644 index 00000000000..fc98b8eb541 --- /dev/null +++ b/dashboard/app/.buildignore @@ -0,0 +1 @@ +*.coffee \ No newline at end of file diff --git a/dashboard/app/browser.html b/dashboard/app/browser.html new file mode 100644 index 00000000000..27127833eb6 --- /dev/null +++ b/dashboard/app/browser.html @@ -0,0 +1,51 @@ + + + + + + + + + etcd Browser + + + + + + + + + + + + + + + + +
+
+ + + + + + + + + + + + + + + + + + + diff --git a/dashboard/app/index.html b/dashboard/app/index.html new file mode 100644 index 00000000000..e3995e6d160 --- /dev/null +++ b/dashboard/app/index.html @@ -0,0 +1,134 @@ + + + + + + + + + etcd dashboard + + + + + + + +

etcd Dashboard

+ + +
+ + diff --git a/dashboard/app/scripts/common/services/etcd.js b/dashboard/app/scripts/common/services/etcd.js new file mode 100644 index 00000000000..5a303bac8bb --- /dev/null +++ b/dashboard/app/scripts/common/services/etcd.js @@ -0,0 +1,81 @@ +'use strict'; + +angular.module('etcd', []) + +.factory('EtcdV1', ['$http', function($http) { + var keyPrefix = '/v1/keys/' + var statsPrefix = '/v1/stats/' + var baseURL = '/v1/' + + delete $http.defaults.headers.common['X-Requested-With']; + + function cleanupPath(path) { + var parts = path.split('/'); + if (parts.length === 0) { + return ''; + } + parts = parts.filter(function(v){return v!=='';}); + return parts.join('/'); + } + + function newKey(keyName) { + var self = {}; + self.name = cleanupPath(keyName); + + self.getParent = function() { + var parts = self.name.split('/'); + if (parts.length === 0) { + return newKey(''); + } + parts.pop(); + return newKey(parts.join('/')); + }; + + self.path = function() { + return '/' + cleanupPath(keyPrefix + self.name); + }; + + self.get = function() { + return $http.get(self.path()); + }; + + self.set = function(keyValue) { + return $http({ + url: self.path(), + data: $.param({value: keyValue}), + method: 'POST', + headers: {'Content-Type': 'application/x-www-form-urlencoded'} + }); + }; + + self.deleteKey = function(keyValue) { + return $http({ + url: self.path(), + method: 'DELETE', + headers: {'Content-Type': 'application/x-www-form-urlencoded'} + }); + }; + + return self; + } + + function newStat(statName) { + var self = {}; + self.name = cleanupPath(statName); + + self.path = function() { + return '/' + cleanupPath(statsPrefix + self.name); + }; + + self.get = function() { + return $http.get(self.path()); + }; + + return self + } + + return { + getStat: newStat, + getKey: newKey + } +}]); diff --git a/dashboard/app/scripts/controllers/browser.js b/dashboard/app/scripts/controllers/browser.js new file mode 100644 index 00000000000..ecb42022223 --- /dev/null +++ b/dashboard/app/scripts/controllers/browser.js @@ -0,0 +1,191 @@ +'use strict'; + +angular.module('etcdBrowser', ['ngRoute', 'etcd', 'timeRelative']) + +.constant('keyPrefix', '/v1/keys') + +.config(['$routeProvider', 'keyPrefix', function ($routeProvider, keyPrefix) { + //read localstorage + var previousPath = localStorage.getItem('etcd_path'); + + $routeProvider + .when('/', { + redirectTo: keyPrefix + }) + .otherwise({ + templateUrl: 'views/browser.html', + controller: 'MainCtrl' + }); +}]) + +.controller('MainCtrl', ['$scope', '$location', 'EtcdV1', 'keyPrefix', function ($scope, $location, EtcdV1, keyPrefix) { + $scope.save = 'etcd-save-hide'; + $scope.preview = 'etcd-preview-hide'; + $scope.enableBack = true; + $scope.writingNew = false; + + // etcdPath is the path to the key that is currenly being looked at. + $scope.etcdPath = $location.path(); + + $scope.$watch('etcdPath', function() { + function etcdPathKey() { + return pathKey($scope.etcdPath); + } + + function pathKey(path) { + var parts = path.split(keyPrefix); + if (parts.length === 1) { + return ''; + } + return parts[1]; + } + + // Notify everyone of the update + localStorage.setItem('etcdPath', $scope.etcdPath); + $scope.enableBack = true; + //disable back button if at root (/v1/keys/) + if($scope.etcdPath === '') { + $scope.enableBack = false; + } + + $scope.key = EtcdV1.getKey(etcdPathKey($scope.etcdPath)); + }); + + $scope.$watch('key', function() { + if ($scope.writingNew === true) { + return; + } + $scope.key.get().success(function (data, status, headers, config) { + //hide any errors + $('#etcd-browse-error').hide(); + // Looking at a directory if we got an array + if (data.length) { + $scope.list = data; + $scope.preview = 'etcd-preview-hide'; + } else { + $scope.singleValue = data.value; + $scope.preview = 'etcd-preview-reveal'; + $scope.key.getParent().get().success(function(data) { + $scope.list = data; + }); + } + $scope.previewMessage = 'No key selected.'; + }).error(function (data, status, headers, config) { + $scope.previewMessage = 'Key does not exist.'; + $scope.showBrowseError(data.message); + }); + }); + + //back button click + $scope.back = function() { + $scope.etcdPath = $scope.key.getParent().path(); + $scope.syncLocation(); + $scope.preview = 'etcd-preview-hide'; + $scope.writingNew = false; + }; + + $scope.syncLocation = function() { + $location.path($scope.etcdPath); + }; + + $scope.showSave = function() { + $scope.save = 'etcd-save-reveal'; + }; + + $scope.saveData = function() { + // TODO: fixup etcd to allow for empty values + $scope.key.set($scope.singleValue || ' ').success(function (data, status, headers, config) { + $scope.save = 'etcd-save-hide'; + $scope.preview = 'etcd-preview-hide'; + $scope.back(); + $scope.writingNew = false; + }).error(function (data, status, headers, config) { + $scope.showSaveError(data.message); + }); + }; + + $scope.deleteKey = function() { + $scope.key.deleteKey().success(function (data, status, headers, config) { + //TODO: remove loader + $scope.save = 'etcd-save-hide'; + $scope.preview = 'etcd-preview-hide'; + $scope.back(); + }).error(function (data, status, headers, config) { + //TODO: remove loader + //show errors + $scope.showBrowseError('Error: Could not delete the key'); + }); + }; + + $scope.add = function() { + $scope.save = 'etcd-save-reveal'; + $scope.preview = 'etcd-preview-reveal'; + $scope.singleValue = ''; + $('.etcd-browser-path').find('input').focus(); + $scope.writingNew = true; + }; + + $scope.showBrowseError = function(message) { + $('#etcd-browse-error').find('.etcd-popover-content').text('Error: ' + message); + $('#etcd-browse-error').addClass('etcd-popover-right').show(); + }; + + $scope.showSaveError = function(message) { + $('#etcd-save-error').find('.etcd-popover-content').text('Error: ' + message); + $('#etcd-save-error').addClass('etcd-popover-left').show(); + }; + + $scope.getHeight = function() { + return $(window).height(); + }; + $scope.$watch($scope.getHeight, function() { + $('.etcd-body').css('height', $scope.getHeight()-45); + }); + window.onresize = function(){ + $scope.$apply(); + }; + +}]) + +.directive('ngEnter', function() { + return function(scope, element, attrs) { + element.bind('keydown keypress', function(event) { + if(event.which === 13) { + scope.$apply(function(){ + scope.$eval(attrs.ngEnter); + }); + + event.preventDefault(); + } + }); + }; +}) + +.directive('highlight', function() { + return { + restrict: 'A', + link: function(scope, element, attrs) { + if('#' + scope.etcdPath === attrs.href) { + element.parent().parent().addClass('etcd-selected'); + } + } + }; +}); + +moment.lang('en', { + relativeTime : { + future: 'Expires in %s', + past: 'Expired %s ago', + s: 'seconds', + m: 'a minute', + mm: '%d minutes', + h: 'an hour', + hh: '%d hours', + d: 'a day', + dd: '%d days', + M: 'a month', + MM: '%d months', + y: 'a year', + yy: '%d years' + } +}); diff --git a/dashboard/app/scripts/controllers/stats.js b/dashboard/app/scripts/controllers/stats.js new file mode 100644 index 00000000000..e689147b093 --- /dev/null +++ b/dashboard/app/scripts/controllers/stats.js @@ -0,0 +1,181 @@ +'use strict'; + +angular.module('etcdStats', ['ngRoute', 'etcd']) + +.config(['$routeProvider', function ($routeProvider) { + $routeProvider + .when('/', { + templateUrl: 'views/stats.html', + controller: 'StatsCtrl' + }) + .otherwise({ + templateUrl: 'views/stats.html', + controller: 'StatsCtrl' + }); +}]) + +.controller('StatsCtrl', ['$scope', 'EtcdV1', 'statsVega', function ($scope, EtcdV1, statsVega) { + $scope.graphContainer = '#latency'; + $scope.graphVisibility = 'etcd-graph-show'; + $scope.tableVisibility = 'etcd-table-hide'; + + //make requests + function readStats() { + EtcdV1.getStat('leader').get().success(function(data) { + $scope.leaderStats = data; + $scope.followers = []; + $.each(data.followers, function(index, value) { + value.name = index; + $scope.followers.push(value); + }); + drawGraph(); + }); + } + + function drawGraph () { + //hardcoded padding from chart json + var vertPadding = 30; + var horzPadding = 15; + //fetch width and height of graph area + var width = $($scope.graphContainer).width() - horzPadding; + var height = $($scope.graphContainer).height() - vertPadding; + + // parse a spec and create a visualization view + function parse(spec) { + vg.parse.spec(spec, function(chart) { + chart({ + el: $scope.graphContainer, + data: { + 'stats': $scope.followers + } + }).width(width).height(height).update(); + }); + } + parse(statsVega); + } + + $scope.showTable = function() { + $scope.tableVisibility = 'etcd-table-reveal'; + }; + + $scope.showGraph = function() { + $scope.tableVisibility = 'etcd-table-hide'; + }; + + $scope.getHeight = function() { + return $(window).height(); + }; + $scope.getWidth = function() { + return $(window).width(); + }; + $scope.$watch($scope.getHeight, function() { + $('.etcd-body').css('height', $scope.getHeight()-5); + readStats(); + }); + $scope.$watch($scope.getWidth, function() { + readStats(); + }); + window.onresize = function(){ + $scope.$apply(); + }; + + // Update the graphs live + setInterval(function() { + readStats(); + $scope.$apply(); + }, 500); +}]) + + +/* statsVega returns the vega configuration for the stats dashboard */ +.factory('statsVega', function () { + return { + 'padding': {'top': 10, 'left': 5, 'bottom': 40, 'right': 10}, + 'data': [ + { + 'name': 'stats' + }, + { + 'name': 'thresholds', + 'values': [50, 100] + } + ], + 'scales': [ + { + 'name': 'y', + 'type': 'ordinal', + 'range': 'height', + 'domain': {'data': 'stats', 'field': 'index'} + }, + { + 'name': 'x', + 'range': 'width', + 'domainMin': 0, + 'domainMax': 100, + 'nice': true, + 'zero': true, + 'domain': {'data': 'stats', 'field': 'data.latency.current'} + }, + { + 'name': 'color', + 'type': 'linear', + 'domain': [10, 50, 100, 1000000000], + 'range': ['#00DB24', '#FFC000', '#c40022', '#c40022'] + } + ], + 'axes': [ + { + 'type': 'x', + 'scale': 'x', + 'ticks': 6, + 'name': 'Latency (ms)' + }, + { + 'type': 'y', + 'scale': 'y', + 'properties': { + 'ticks': { + 'stroke': {'value': 'transparent'} + }, + 'majorTicks': { + 'stroke': {'value': 'transparent'} + }, + 'labels': { + 'fill': {'value': 'transparent'} + }, + 'axis': { + 'stroke': {'value': '#333'}, + 'strokeWidth': {'value': 1} + } + } + } + ], + 'marks': [ + { + 'type': 'rect', + 'from': {'data': 'stats'}, + 'properties': { + 'enter': { + 'x': {'scale': 'x', 'value': 0}, + 'x2': {'scale': 'x', 'field': 'data.latency.current'}, + 'y': {'scale': 'y', 'field': 'index', 'offset': -1}, + 'height': {'value': 3}, + 'fill': {'scale':'color', 'field':'data.latency.current'} + } + } + }, + { + 'type': 'symbol', + 'from': {'data': 'stats'}, + 'properties': { + 'enter': { + 'x': {'scale': 'x', 'field': 'data.latency.current'}, + 'y': {'scale': 'y', 'field': 'index'}, + 'size': {'value': 50}, + 'fill': {'value': '#000'} + } + } + } + ] + }; +}); diff --git a/dashboard/app/scripts/ng-time-relative.min.js b/dashboard/app/scripts/ng-time-relative.min.js new file mode 100644 index 00000000000..3c09675bd4f --- /dev/null +++ b/dashboard/app/scripts/ng-time-relative.min.js @@ -0,0 +1 @@ +(function(e){if("function"==typeof bootstrap)bootstrap("ng-time-relative",e);else if("object"==typeof exports)module.exports=e();else if("function"==typeof define&&define.amd)define(e);else if("undefined"!=typeof ses){if(!ses.ok())return;ses.makeNgTimeRelative=e}else"undefined"!=typeof window?window.ngTimeRelative=e():global.ngTimeRelative=e()})(function(){var define,ses,bootstrap,module,exports;return function(e,t,n){function i(n,s){if(!t[n]){if(!e[n]){var o=typeof require=="function"&&require;if(!s&&o)return o(n,!0);if(r)return r(n,!0);throw new Error("Cannot find module '"+n+"'")}var u=t[n]={exports:{}};e[n][0](function(t){var r=e[n][1][t];return i(r?r:t)},u,u.exports)}return t[n].exports}var r=typeof require=="function"&&require;for(var s=0;s 1 + ? function(x) { return s.reduce(function(x,f) { return x[f]; }, x); } + : function(x) { return x[f]; }; +}; + +vg.comparator = function(sort) { + var sign = []; + if (sort === undefined) sort = []; + sort = vg.array(sort).map(function(f) { + var s = 1; + if (f[0] === "-") { s = -1; f = f.slice(1); } + else if (f[0] === "+") { s = +1; f = f.slice(1); } + sign.push(s); + return vg.accessor(f); + }); + return function(a,b) { + var i, n, f, x, y; + for (i=0, n=sort.length; i y) return sign[i]; + } + return 0; + }; +}; + +vg.cmp = function(a, b) { return ab ? 1 : 0; }; + +vg.numcmp = function(a, b) { return a - b; }; + +vg.array = function(x) { + return x != null ? (vg.isArray(x) ? x : [x]) : []; +}; + +vg.values = function(x) { + return (vg.isObject(x) && !vg.isArray(x) && x.values) ? x.values : x; +}; + +vg.str = function(x) { + return vg.isArray(x) ? "[" + x.map(vg.str) + "]" + : vg.isObject(x) ? JSON.stringify(x) + : vg.isString(x) ? ("'"+vg_escape_str(x)+"'") : x; +}; + +var escape_str_re = /(^|[^\\])'/g; + +function vg_escape_str(x) { + return x.replace(escape_str_re, "$1\\'"); +} + +vg.keys = function(x) { + var keys = []; + for (var key in x) keys.push(key); + return keys; +}; + +vg.unique = function(data, f, results) { + if (!vg.isArray(data) || data.length==0) return []; + f = f || vg.identity; + results = results || []; + for (var v, i=0, n=data.length; i max) { max = v; idx = i; } + } + return idx; +}; + +vg.truncate = function(s, length, pos, word, ellipsis) { + var len = s.length; + if (len <= length) return s; + ellipsis = ellipsis || "..."; + var l = Math.max(0, length - ellipsis.length); + + switch (pos) { + case "left": + return ellipsis + (word ? vg_truncateOnWord(s,l,1) : s.slice(len-l)); + case "middle": + case "center": + var l1 = Math.ceil(l/2), l2 = Math.floor(l/2); + return (word ? vg_truncateOnWord(s,l1) : s.slice(0,l1)) + ellipsis + + (word ? vg_truncateOnWord(s,l2,1) : s.slice(len-l2)); + default: + return (word ? vg_truncateOnWord(s,l) : s.slice(0,l)) + ellipsis; + } +} + +function vg_truncateOnWord(s, len, rev) { + var cnt = 0, tok = s.split(vg_truncate_word_re); + if (rev) { + s = (tok = tok.reverse()) + .filter(function(w) { cnt += w.length; return cnt <= len; }) + .reverse(); + } else { + s = tok.filter(function(w) { cnt += w.length; return cnt <= len; }); + } + return s.length ? s.join("").trim() : tok[0].slice(0, len); +} + +var vg_truncate_word_re = /([\u0009\u000A\u000B\u000C\u000D\u0020\u00A0\u1680\u180E\u2000\u2001\u2002\u2003\u2004\u2005\u2006\u2007\u2008\u2009\u200A\u202F\u205F\u2028\u2029\u3000\uFEFF])/; + +// Logging + +function vg_write(msg) { + vg.config.isNode + ? process.stderr.write(msg + "\n") + : console.log(msg); +} + +vg.log = function(msg) { + vg_write("[Vega Log] " + msg); +}; + +vg.error = function(msg) { + msg = "[Vega Err] " + msg; + vg_write(msg); + if (typeof alert !== "undefined") alert(msg); +};vg.config = {}; + +// are we running in node.js? +// via timetler.com/2012/10/13/environment-detection-in-javascript/ +vg.config.isNode = typeof exports !== 'undefined' && this.exports !== exports; + +// base url for loading external data files +// used only for server-side operation +vg.config.baseURL = ""; + +// version and namepsaces for exported svg +vg.config.svgNamespace = + 'version="1.1" xmlns="http://www.w3.org/2000/svg" ' + + 'xmlns:xlink="http://www.w3.org/1999/xlink"'; + +// inset padding for automatic padding calculation +vg.config.autopadInset = 5; + +// extensible scale lookup table +// all d3.scale.* instances also supported +vg.config.scale = { + time: d3.time.scale, + utc: d3.time.scale.utc +}; + +// default rendering settings +vg.config.render = { + lineWidth: 1, + lineCap: "butt", + font: "sans-serif", + fontSize: 11 +}; + +// default axis properties +vg.config.axis = { + orient: "bottom", + ticks: 10, + padding: 3, + axisColor: "#000", + gridColor: "#d8d8d8", + tickColor: "#000", + tickLabelColor: "#000", + axisWidth: 1, + tickWidth: 1, + tickSize: 6, + tickLabelFontSize: 11, + tickLabelFont: "sans-serif", + titleColor: "#000", + titleFont: "sans-serif", + titleFontSize: 11, + titleFontWeight: "bold", + titleOffset: 35 +}; + +// default legend properties +vg.config.legend = { + orient: "right", + offset: 10, + padding: 3, + gradientStrokeColor: "#888", + gradientStrokeWidth: 1, + gradientHeight: 16, + gradientWidth: 100, + labelColor: "#000", + labelFontSize: 10, + labelFont: "sans-serif", + labelAlign: "left", + labelBaseline: "middle", + labelOffset: 8, + symbolShape: "circle", + symbolSize: 50, + symbolColor: "#888", + symbolStrokeWidth: 1, + titleColor: "#000", + titleFont: "sans-serif", + titleFontSize: 11, + titleFontWeight: "bold" +}; + +// default color values +vg.config.color = { + rgb: [128, 128, 128], + lab: [50, 0, 0], + hcl: [0, 0, 50], + hsl: [0, 0, 0.5] +}; + +// default scale ranges +vg.config.range = { + category10: [ + "#1f77b4", + "#ff7f0e", + "#2ca02c", + "#d62728", + "#9467bd", + "#8c564b", + "#e377c2", + "#7f7f7f", + "#bcbd22", + "#17becf" + ], + category20: [ + "#1f77b4", + "#aec7e8", + "#ff7f0e", + "#ffbb78", + "#2ca02c", + "#98df8a", + "#d62728", + "#ff9896", + "#9467bd", + "#c5b0d5", + "#8c564b", + "#c49c94", + "#e377c2", + "#f7b6d2", + "#7f7f7f", + "#c7c7c7", + "#bcbd22", + "#dbdb8d", + "#17becf", + "#9edae5" + ], + shapes: [ + "circle", + "cross", + "diamond", + "square", + "triangle-down", + "triangle-up" + ] +};vg.Bounds = (function() { + var bounds = function(b) { + this.clear(); + if (b) this.union(b); + }; + + var prototype = bounds.prototype; + + prototype.clear = function() { + this.x1 = +Number.MAX_VALUE; + this.y1 = +Number.MAX_VALUE; + this.x2 = -Number.MAX_VALUE; + this.y2 = -Number.MAX_VALUE; + return this; + }; + + prototype.set = function(x1, y1, x2, y2) { + this.x1 = x1; + this.y1 = y1; + this.x2 = x2; + this.y2 = y2; + return this; + }; + + prototype.add = function(x, y) { + if (x < this.x1) this.x1 = x; + if (y < this.y1) this.y1 = y; + if (x > this.x2) this.x2 = x; + if (y > this.y2) this.y2 = y; + return this; + }; + + prototype.expand = function(d) { + this.x1 -= d; + this.y1 -= d; + this.x2 += d; + this.y2 += d; + return this; + }; + + prototype.round = function() { + this.x1 = Math.floor(this.x1); + this.y1 = Math.floor(this.y1); + this.x2 = Math.ceil(this.x2); + this.y2 = Math.ceil(this.y2); + return this; + }; + + prototype.translate = function(dx, dy) { + this.x1 += dx; + this.x2 += dx; + this.y1 += dy; + this.y2 += dy; + return this; + }; + + prototype.rotate = function(angle, x, y) { + var cos = Math.cos(angle), + sin = Math.sin(angle), + cx = x - x*cos + y*sin, + cy = y - x*sin - y*cos, + x1 = this.x1, x2 = this.x2, + y1 = this.y1, y2 = this.y2; + + return this.clear() + .add(cos*x1 - sin*y1 + cx, sin*x1 + cos*y1 + cy) + .add(cos*x1 - sin*y2 + cx, sin*x1 + cos*y2 + cy) + .add(cos*x2 - sin*y1 + cx, sin*x2 + cos*y1 + cy) + .add(cos*x2 - sin*y2 + cx, sin*x2 + cos*y2 + cy); + } + + prototype.union = function(b) { + if (b.x1 < this.x1) this.x1 = b.x1; + if (b.y1 < this.y1) this.y1 = b.y1; + if (b.x2 > this.x2) this.x2 = b.x2; + if (b.y2 > this.y2) this.y2 = b.y2; + return this; + }; + + prototype.encloses = function(b) { + return b && ( + this.x1 <= b.x1 && + this.x2 >= b.x2 && + this.y1 <= b.y1 && + this.y2 >= b.y2 + ); + }; + + prototype.intersects = function(b) { + return b && !( + this.x2 < b.x1 || + this.x1 > b.x2 || + this.y2 < b.y1 || + this.y1 > b.y2 + ); + }; + + prototype.contains = function(x, y) { + return !( + x < this.x1 || + x > this.x2 || + y < this.y1 || + y > this.y2 + ); + }; + + prototype.width = function() { + return this.x2 - this.x1; + }; + + prototype.height = function() { + return this.y2 - this.y1; + }; + + return bounds; +})();vg.Gradient = (function() { + + function gradient(type) { + this.id = "grad_" + (vg_gradient_id++); + this.type = type || "linear"; + this.stops = []; + this.x1 = 0; + this.x2 = 1; + this.y1 = 0; + this.y2 = 0; + }; + + var prototype = gradient.prototype; + + prototype.stop = function(offset, color) { + this.stops.push({ + offset: offset, + color: color + }); + return this; + }; + + return gradient; +})(); + +var vg_gradient_id = 0;vg.canvas = {};vg.canvas.path = (function() { + + // Path parsing and rendering code taken from fabric.js -- Thanks! + var cmdLength = { m:2, l:2, h:1, v:1, c:6, s:4, q:4, t:2, a:7 }, + re = [/([MLHVCSQTAZmlhvcsqtaz])/g, /###/, /(\d)-/g, /\s|,|###/]; + + function parse(path) { + var result = [], + currentPath, + chunks, + parsed; + + // First, break path into command sequence + path = path.slice().replace(re[0], '###$1').split(re[1]).slice(1); + + // Next, parse each command in turn + for (var i=0, j, chunksParsed, len=path.length; i commandLength) { + for (var k = 1, klen = chunksParsed.length; k < klen; k += commandLength) { + result.push([ chunksParsed[0] ].concat(chunksParsed.slice(k, k + commandLength))); + } + } + else { + result.push(chunksParsed); + } + } + + return result; + } + + function drawArc(g, x, y, coords, bounds, l, t) { + var rx = coords[0]; + var ry = coords[1]; + var rot = coords[2]; + var large = coords[3]; + var sweep = coords[4]; + var ex = coords[5]; + var ey = coords[6]; + var segs = arcToSegments(ex, ey, rx, ry, large, sweep, rot, x, y); + for (var i=0; i 1) { + pl = Math.sqrt(pl); + rx *= pl; + ry *= pl; + } + + var a00 = cos_th / rx; + var a01 = sin_th / rx; + var a10 = (-sin_th) / ry; + var a11 = (cos_th) / ry; + var x0 = a00 * ox + a01 * oy; + var y0 = a10 * ox + a11 * oy; + var x1 = a00 * x + a01 * y; + var y1 = a10 * x + a11 * y; + + var d = (x1-x0) * (x1-x0) + (y1-y0) * (y1-y0); + var sfactor_sq = 1 / d - 0.25; + if (sfactor_sq < 0) sfactor_sq = 0; + var sfactor = Math.sqrt(sfactor_sq); + if (sweep == large) sfactor = -sfactor; + var xc = 0.5 * (x0 + x1) - sfactor * (y1-y0); + var yc = 0.5 * (y0 + y1) + sfactor * (x1-x0); + + var th0 = Math.atan2(y0-yc, x0-xc); + var th1 = Math.atan2(y1-yc, x1-xc); + + var th_arc = th1-th0; + if (th_arc < 0 && sweep == 1){ + th_arc += 2*Math.PI; + } else if (th_arc > 0 && sweep == 0) { + th_arc -= 2 * Math.PI; + } + + var segments = Math.ceil(Math.abs(th_arc / (Math.PI * 0.5 + 0.001))); + var result = []; + for (var i=0; i 0) { + g.globalAlpha = opac * (o.strokeOpacity==null ? 1 : o.strokeOpacity); + g.strokeStyle = color(g, o, stroke); + g.lineWidth = lw; + g.lineCap = (lc = o.strokeCap) != null ? lc : vg.config.render.lineCap; + g.vgLineDash(o.strokeDash || null); + g.vgLineDashOffset(o.strokeDashOffset || 0); + g.stroke(); + } + } + } + + function drawPathAll(path, g, scene, bounds) { + var i, len, item; + for (i=0, len=scene.items.length; i 0) { + g.globalAlpha = opac * (o.strokeOpacity==null ? 1 : o.strokeOpacity); + g.strokeStyle = color(g, o, stroke); + g.lineWidth = lw; + g.lineCap = (lc = o.strokeCap) != null ? lc : vg.config.render.lineCap; + g.vgLineDash(o.strokeDash || null); + g.vgLineDashOffset(o.strokeDashOffset || 0); + g.strokeRect(x, y, w, h); + } + } + } + } + + function drawRule(g, scene, bounds) { + if (!scene.items.length) return; + var items = scene.items, + o, stroke, opac, lc, lw, x1, y1, x2, y2; + + for (var i=0, len=items.length; i 0) { + g.globalAlpha = opac * (o.strokeOpacity==null ? 1 : o.strokeOpacity); + g.strokeStyle = color(g, o, stroke); + g.lineWidth = lw; + g.lineCap = (lc = o.strokeCap) != null ? lc : vg.config.render.lineCap; + g.vgLineDash(o.strokeDash || null); + g.vgLineDashOffset(o.strokeDashOffset || 0); + g.beginPath(); + g.moveTo(x1, y1); + g.lineTo(x2, y2); + g.stroke(); + } + } + } + } + + function drawImage(g, scene, bounds) { + if (!scene.items.length) return; + var renderer = this, + items = scene.items, o; + + for (var i=0, len=items.length; i 0) { + g.globalAlpha = opac * (o.strokeOpacity==null ? 1 : o.strokeOpacity); + g.strokeStyle = color(o, stroke); + g.lineWidth = lw; + g.strokeText(o.text, x, y); + } + } + + if (o.angle) g.restore(); + } + } + + function drawAll(pathFunc) { + return function(g, scene, bounds) { + drawPathAll(pathFunc, g, scene, bounds); + } + } + + function drawOne(pathFunc) { + return function(g, scene, bounds) { + if (!scene.items.length) return; + if (bounds && !bounds.intersects(scene.items[0].bounds)) + return; // bounds check + drawPathOne(pathFunc, g, scene.items[0], scene.items); + } + } + + function drawGroup(g, scene, bounds) { + if (!scene.items.length) return; + var items = scene.items, group, axes, legends, + renderer = this, gx, gy, gb, i, n, j, m; + + drawRect(g, scene, bounds); + + for (i=0, n=items.length; i=0;) { + group = items[i]; + dx = group.x || 0; + dy = group.y || 0; + + g.save(); + g.translate(dx, dy); + for (j=group.items.length; --j >= 0;) { + subscene = group.items[j]; + if (subscene.interactive === false) continue; + hit = handler.pick(subscene, x, y, gx-dx, gy-dy); + if (hit) { + g.restore(); + return hit; + } + } + g.restore(); + } + + return scene.interactive + ? pickAll(hitTests.rect, g, scene, x, y, gx, gy) + : false; + } + + function pickAll(test, g, scene, x, y, gx, gy) { + if (!scene.items.length) return false; + var o, b, i; + + if (g._ratio !== 1) { + x *= g._ratio; + y *= g._ratio; + } + + for (i=scene.items.length; --i >= 0;) { + o = scene.items[i]; b = o.bounds; + // first hit test against bounding box + if ((b && !b.contains(gx, gy)) || !b) continue; + // if in bounding box, perform more careful test + if (test(g, o, x, y, gx, gy)) return o; + } + return false; + } + + function pickArea(g, scene, x, y, gx, gy) { + if (!scene.items.length) return false; + var items = scene.items, + o, b, i, di, dd, od, dx, dy; + + b = items[0].bounds; + if (b && !b.contains(gx, gy)) return false; + if (g._ratio !== 1) { + x *= g._ratio; + y *= g._ratio; + } + if (!hitTests.area(g, items, x, y)) return false; + return items[0]; + } + + function pickLine(g, scene, x, y, gx, gy) { + if (!scene.items.length) return false; + var items = scene.items, + o, b, i, di, dd, od, dx, dy; + + b = items[0].bounds; + if (b && !b.contains(gx, gy)) return false; + if (g._ratio !== 1) { + x *= g._ratio; + y *= g._ratio; + } + if (!hitTests.line(g, items, x, y)) return false; + return items[0]; + } + + function pick(test) { + return function (g, scene, x, y, gx, gy) { + return pickAll(test, g, scene, x, y, gx, gy); + }; + } + + function textHit(g, o, x, y, gx, gy) { + if (!o.fontSize) return false; + if (!o.angle) return true; // bounds sufficient if no rotation + + var b = vg.scene.bounds.text(o, tmpBounds, true), + a = -o.angle * Math.PI / 180, + cos = Math.cos(a), + sin = Math.sin(a), + x = o.x, + y = o.y, + px = cos*gx - sin*gy + (x - x*cos + y*sin), + py = sin*gx + cos*gy + (y - x*sin - y*cos); + + return b.contains(px, py); + } + + var hitTests = { + text: textHit, + rect: function(g,o,x,y) { return true; }, // bounds test is sufficient + image: function(g,o,x,y) { return true; }, // bounds test is sufficient + rule: function(g,o,x,y) { + if (!g.isPointInStroke) return false; + ruleStroke(g,o); return g.isPointInStroke(x,y); + }, + line: function(g,s,x,y) { + if (!g.isPointInStroke) return false; + lineStroke(g,s); return g.isPointInStroke(x,y); + }, + arc: function(g,o,x,y) { arcPath(g,o); return g.isPointInPath(x,y); }, + area: function(g,s,x,y) { areaPath(g,s); return g.isPointInPath(x,y); }, + path: function(g,o,x,y) { pathPath(g,o); return g.isPointInPath(x,y); }, + symbol: function(g,o,x,y) { symbolPath(g,o); return g.isPointInPath(x,y); } + }; + + return { + draw: { + group: drawGroup, + area: drawOne(areaPath), + line: drawOne(linePath), + arc: drawAll(arcPath), + path: drawAll(pathPath), + symbol: drawAll(symbolPath), + rect: drawRect, + rule: drawRule, + text: drawText, + image: drawImage, + drawOne: drawOne, // expose for extensibility + drawAll: drawAll // expose for extensibility + }, + pick: { + group: pickGroup, + area: pickArea, + line: pickLine, + arc: pick(hitTests.arc), + path: pick(hitTests.path), + symbol: pick(hitTests.symbol), + rect: pick(hitTests.rect), + rule: pick(hitTests.rule), + text: pick(hitTests.text), + image: pick(hitTests.image), + pickAll: pickAll // expose for extensibility + } + }; + +})();vg.canvas.Renderer = (function() { + var renderer = function() { + this._ctx = null; + this._el = null; + this._imgload = 0; + }; + + var prototype = renderer.prototype; + + prototype.initialize = function(el, width, height, pad) { + this._el = el; + + if (!el) return this; // early exit if no DOM element + + // select canvas element + var canvas = d3.select(el) + .selectAll("canvas.marks") + .data([1]); + + // create new canvas element if needed + canvas.enter() + .append("canvas") + .attr("class", "marks"); + + // remove extraneous canvas if needed + canvas.exit().remove(); + + return this.resize(width, height, pad); + }; + + prototype.resize = function(width, height, pad) { + this._width = width; + this._height = height; + this._padding = pad; + + if (this._el) { + var canvas = d3.select(this._el).select("canvas.marks"); + + // initialize canvas attributes + canvas + .attr("width", width + pad.left + pad.right) + .attr("height", height + pad.top + pad.bottom); + + // get the canvas graphics context + var s; + this._ctx = canvas.node().getContext("2d"); + this._ctx._ratio = (s = scaleCanvas(canvas.node(), this._ctx) || 1); + this._ctx.setTransform(s, 0, 0, s, s*pad.left, s*pad.top); + } + + initializeLineDash(this._ctx); + return this; + }; + + function scaleCanvas(canvas, ctx) { + // get canvas pixel data + var devicePixelRatio = window.devicePixelRatio || 1, + backingStoreRatio = ( + ctx.webkitBackingStorePixelRatio || + ctx.mozBackingStorePixelRatio || + ctx.msBackingStorePixelRatio || + ctx.oBackingStorePixelRatio || + ctx.backingStorePixelRatio) || 1, + ratio = devicePixelRatio / backingStoreRatio; + + if (devicePixelRatio !== backingStoreRatio) { + var w = canvas.width, h = canvas.height; + // set actual and visible canvas size + canvas.setAttribute("width", w * ratio); + canvas.setAttribute("height", h * ratio); + canvas.style.width = w + 'px'; + canvas.style.height = h + 'px'; + } + return ratio; + } + + function initializeLineDash(ctx) { + if (ctx.vgLineDash) return; // already set + + var NODASH = []; + if (ctx.setLineDash) { + ctx.vgLineDash = function(dash) { this.setLineDash(dash || NODASH); }; + ctx.vgLineDashOffset = function(off) { this.lineDashOffset = off; }; + } else if (ctx.webkitLineDash !== undefined) { + ctx.vgLineDash = function(dash) { this.webkitLineDash = dash || NODASH; }; + ctx.vgLineDashOffset = function(off) { this.webkitLineDashOffset = off; }; + } else if (ctx.mozDash !== undefined) { + ctx.vgLineDash = function(dash) { this.mozDash = dash; }; + ctx.vgLineDashOffset = function(off) { /* unsupported */ }; + } else { + ctx.vgLineDash = function(dash) { /* unsupported */ }; + ctx.vgLineDashOffset = function(off) { /* unsupported */ }; + } + } + + prototype.context = function(ctx) { + if (ctx) { this._ctx = ctx; return this; } + else return this._ctx; + }; + + prototype.element = function() { + return this._el; + }; + + prototype.pendingImages = function() { + return this._imgload; + }; + + function translatedBounds(item, bounds) { + var b = new vg.Bounds(bounds); + while ((item = item.mark.group) != null) { + b.translate(item.x || 0, item.y || 0); + } + return b; + } + + function getBounds(items) { + return !items ? null : + vg.array(items).reduce(function(b, item) { + return b.union(translatedBounds(item, item.bounds)) + .union(translatedBounds(item, item['bounds:prev'])); + }, new vg.Bounds()); + } + + function setBounds(g, bounds) { + var bbox = null; + if (bounds) { + bbox = (new vg.Bounds(bounds)).round(); + g.beginPath(); + g.rect(bbox.x1, bbox.y1, bbox.width(), bbox.height()); + g.clip(); + } + return bbox; + } + + prototype.render = function(scene, items) { + var g = this._ctx, + pad = this._padding, + w = this._width + pad.left + pad.right, + h = this._height + pad.top + pad.bottom, + bb = null, bb2; + + // setup + this._scene = scene; + g.save(); + bb = setBounds(g, getBounds(items)); + g.clearRect(-pad.left, -pad.top, w, h); + + // render + this.draw(g, scene, bb); + + // render again to handle possible bounds change + if (items) { + g.restore(); + g.save(); + bb2 = setBounds(g, getBounds(items)); + if (!bb.encloses(bb2)) { + g.clearRect(-pad.left, -pad.top, w, h); + this.draw(g, scene, bb2); + } + } + + // takedown + g.restore(); + this._scene = null; + }; + + prototype.draw = function(ctx, scene, bounds) { + var marktype = scene.marktype, + renderer = vg.canvas.marks.draw[marktype]; + renderer.call(this, ctx, scene, bounds); + + // compute mark-level bounds + scene.bounds = scene.items.reduce(function(b, item) { + return item.bounds ? b.union(item.bounds) : b; + }, scene.bounds || new vg.Bounds()); + }; + + prototype.renderAsync = function(scene) { + // TODO make safe for multiple scene rendering? + var renderer = this; + if (renderer._async_id) { + clearTimeout(renderer._async_id); + } + renderer._async_id = setTimeout(function() { + renderer.render(scene); + delete renderer._async_id; + }, 50); + }; + + prototype.loadImage = function(uri) { + var renderer = this, + scene = renderer._scene, + image = null, url; + + renderer._imgload += 1; + if (vg.config.isNode) { + image = new (require("canvas").Image)(); + vg.data.load(uri, function(err, data) { + if (err) { vg.error(err); return; } + image.src = data; + image.loaded = true; + renderer._imgload -= 1; + }); + } else { + image = new Image(); + url = vg.config.baseURL + uri; + image.onload = function() { + vg.log("LOAD IMAGE: "+url); + image.loaded = true; + renderer._imgload -= 1; + renderer.renderAsync(scene); + }; + image.src = url; + } + + return image; + }; + + return renderer; +})();vg.canvas.Handler = (function() { + var handler = function(el, model) { + this._active = null; + this._handlers = {}; + if (el) this.initialize(el); + if (model) this.model(model); + }; + + var prototype = handler.prototype; + + prototype.initialize = function(el, pad, obj) { + this._el = d3.select(el).node(); + this._canvas = d3.select(el).select("canvas.marks").node(); + this._padding = pad; + this._obj = obj || null; + + // add event listeners + var canvas = this._canvas, that = this; + events.forEach(function(type) { + canvas.addEventListener(type, function(evt) { + prototype[type].call(that, evt); + }); + }); + + return this; + }; + + prototype.padding = function(pad) { + this._padding = pad; + return this; + }; + + prototype.model = function(model) { + if (!arguments.length) return this._model; + this._model = model; + return this; + }; + + prototype.handlers = function() { + var h = this._handlers; + return vg.keys(h).reduce(function(a, k) { + return h[k].reduce(function(a, x) { return (a.push(x), a); }, a); + }, []); + }; + + // setup events + var events = [ + "mousedown", + "mouseup", + "click", + "dblclick", + "wheel", + "keydown", + "keypress", + "keyup", + "mousewheel" + ]; + events.forEach(function(type) { + prototype[type] = function(evt) { + this.fire(type, evt); + }; + }); + events.push("mousemove"); + events.push("mouseout"); + + function eventName(name) { + var i = name.indexOf("."); + return i < 0 ? name : name.slice(0,i); + } + + prototype.mousemove = function(evt) { + var pad = this._padding, + b = evt.target.getBoundingClientRect(), + x = evt.clientX - b.left, + y = evt.clientY - b.top, + a = this._active, + p = this.pick(this._model.scene(), x, y, x-pad.left, y-pad.top); + + if (p === a) { + this.fire("mousemove", evt); + return; + } else if (a) { + this.fire("mouseout", evt); + } + this._active = p; + if (p) { + this.fire("mouseover", evt); + } + }; + + prototype.mouseout = function(evt) { + if (this._active) { + this.fire("mouseout", evt); + } + this._active = null; + }; + + // to keep firefox happy + prototype.DOMMouseScroll = function(evt) { + this.fire("mousewheel", evt); + }; + + // fire an event + prototype.fire = function(type, evt) { + var a = this._active, + h = this._handlers[type]; + if (a && h) { + for (var i=0, len=h.length; i=0;) { + if (h[i].type !== type) continue; + if (!handler || h[i].handler === handler) h.splice(i, 1); + } + return this; + }; + + // retrieve the current canvas context + prototype.context = function() { + return this._canvas.getContext("2d"); + }; + + // find the scenegraph item at the current mouse position + // returns an array of scenegraph items, from leaf node up to the root + // x, y -- the absolute x, y mouse coordinates on the canvas element + // gx, gy -- the relative coordinates within the current group + prototype.pick = function(scene, x, y, gx, gy) { + var g = this.context(), + marktype = scene.marktype, + picker = vg.canvas.marks.pick[marktype]; + return picker.call(this, g, scene, x, y, gx, gy); + }; + + return handler; +})();vg.svg = {};vg.svg.marks = (function() { + + function x(o) { return o.x || 0; } + function y(o) { return o.y || 0; } + function yh(o) { return o.y + o.height || 0; } + function key(o) { return o.key; } + function size(o) { return o.size==null ? 100 : o.size; } + function shape(o) { return o.shape || "circle"; } + + var arc_path = d3.svg.arc(), + area_path = d3.svg.area().x(x).y1(y).y0(yh), + line_path = d3.svg.line().x(x).y(y), + symbol_path = d3.svg.symbol().type(shape).size(size); + + var mark_id = 0; + + var textAlign = { + "left": "start", + "center": "middle", + "right": "end" + }; + + var styles = { + "fill": "fill", + "fillOpacity": "fill-opacity", + "stroke": "stroke", + "strokeWidth": "stroke-width", + "strokeOpacity": "stroke-opacity", + "strokeCap": "stroke-linecap", + "strokeDash": "stroke-dasharray", + "strokeDashOffset": "stroke-dashoffset", + "opacity": "opacity" + }; + var styleProps = vg.keys(styles); + + function style(d) { + var i, n, prop, name, value, + o = d.mark ? d : d.length ? d[0] : null; + if (o === null) return; + + for (i=0, n=styleProps.length; i " + tag, + m = p.selectAll(s).data(data), + e = m.enter().append(tag); + + if (notG) { + p.style("pointer-events", evts); + e.each(function(d) { + if (d.mark) d._svg = this; + else if (d.length) d[0]._svg = this; + }); + } else { + e.append("rect").attr("class","background").style("pointer-events",evts); + } + + m.exit().remove(); + m.each(attr); + if (notG) m.each(style); + else p.selectAll(s+" > rect.background").each(group_bg).each(style); + + return p; + } + + function drawGroup(g, scene, index, prefix) { + var p = drawMark(g, scene, index, prefix || "group_", "g", group), + c = p.node().childNodes, n = c.length, i, j, m; + + for (i=0; i=0;) { + if (h[i].type !== type) continue; + if (!handler || h[i].handler === handler) { + dom.removeEventListener(name, h[i].svg); + h.splice(i, 1); + } + } + return this; + }; + + return handler; +})();vg.data = {}; + +vg.data.ingestAll = function(data) { + return vg.isTree(data) + ? vg_make_tree(vg.data.ingestTree(data[0], data.children)) + : data.map(vg.data.ingest); +}; + +vg.data.ingest = function(datum, index) { + return { + data: datum, + index: index + }; +}; + +vg.data.ingestTree = function(node, children) { + var d = vg.data.ingest(node), + c = node[children], n, i; + if (c && (n = c.length)) { + d.values = Array(n); + for (i=0; i= 0) file = file.slice(vg_load_fileProtocol.length); + require("fs").readFile(file, callback); +} + +function vg_load_http(url, callback) { + vg.log("LOAD HTTP: " + url); + var req = require("http").request(url, function(res) { + var pos=0, data = new Buffer(parseInt(res.headers['content-length'],10)); + res.on("error", function(err) { callback(err, null); }); + res.on("data", function(x) { x.copy(data, pos); pos += x.length; }); + res.on("end", function() { callback(null, data); }); + }); + req.on("error", function(err) { callback(err); }); + req.end(); +}vg.data.read = (function() { + var formats = {}, + parsers = { + "number": vg.number, + "boolean": vg.boolean, + "date": Date.parse + }; + + function read(data, format) { + var type = (format && format.type) || "json"; + data = formats[type](data, format); + if (format && format.parse) parseValues(data, format.parse); + return data; + } + + formats.json = function(data, format) { + var d = JSON.parse(data); + if (format && format.property) { + d = vg.accessor(format.property)(d); + } + return d; + }; + + formats.csv = function(data, format) { + var d = d3.csv.parse(data); + return d; + }; + + formats.tsv = function(data, format) { + var d = d3.tsv.parse(data); + return d; + }; + + formats.topojson = function(data, format) { + if (topojson == null) { + vg.error("TopoJSON library not loaded."); + return []; + } + var t = JSON.parse(data), obj = []; + + if (format && format.feature) { + obj = (obj = t.objects[format.feature]) + ? topojson.feature(t, obj).features + : (vg.error("Invalid TopoJSON object: "+format.feature), []); + } else if (format && format.mesh) { + obj = (obj = t.objects[format.mesh]) + ? [topojson.mesh(t, t.objects[format.mesh])] + : (vg.error("Invalid TopoJSON object: " + format.mesh), []); + } + else { vg.error("Missing TopoJSON feature or mesh parameter."); } + + return obj; + }; + + formats.treejson = function(data, format) { + var d = [JSON.parse(data)]; + d.__vgtree__ = true; + d.children = format.children || "children"; + return d; + }; + + function parseValues(data, types) { + var cols = vg.keys(types), + p = cols.map(function(col) { return parsers[types[col]]; }), + tree = vg.isTree(data); + vg_parseArray(tree ? [data] : data, cols, p, tree); + } + + function vg_parseArray(data, cols, p, tree) { + var d, i, j, len, clen; + for (i=0, len=data.length; i0 ? "|" : "") + String(kv); + } + obj = map[kstr]; + if (obj === undefined) { + vals.push(obj = map[kstr] = { + key: kstr, + keys: klist, + index: vals.length, + values: [] + }); + } + obj.values.push(data[i]); + } + + if (sort) { + for (i=0, len=vals.length; i b ? 1 : 0; + }); + data = [data[~~(list.length/2)]]; + } else { + var idx = vg.array(by); + data = data.slice(idx[0], idx[1]); + } + return data; + } + + slice.by = function(x) { + by = x; + return slice; + }; + + slice.field = function(f) { + field = vg.accessor(f); + return slice; + }; + + return slice; +};vg.data.sort = function() { + var by = null; + + function sort(data) { + data = (vg.isArray(data) ? data : data.values || []); + data.sort(by); + for (var i=0, n=data.length; ib.x ? 1 : (a.zb.z ? 1 : 0); + }); + + // emit data series for stack layout + for (x=points[0].x, i=0, j=0, k=0, n=points.length; k i) series[i++].push({x:j, y:0}); + p.x = j; + series[i++].push(p); + } + while (i < series.length) series[i++].push({x:j, y:0}); + + return series; + } + + stack.point = function(field) { + point = vg.accessor(field); + return stack; + }; + + stack.height = function(field) { + height = vg.accessor(field); + return stack; + }; + + params.forEach(function(name) { + stack[name] = function(x) { + layout[name](x); + return stack; + } + }); + + stack.output = function(map) { + d3.keys(output).forEach(function(k) { + if (map[k] !== undefined) { + output[k] = map[k]; + } + }); + return stack; + }; + + return stack; +};vg.data.stats = function() { + var value = vg.accessor("data"), + assign = false, + median = false, + output = { + "count": "count", + "min": "min", + "max": "max", + "sum": "sum", + "mean": "mean", + "variance": "variance", + "stdev": "stdev", + "median": "median" + }; + + function reduce(data) { + var min = +Infinity, + max = -Infinity, + sum = 0, + mean = 0, + M2 = 0, + i, len, v, delta; + + var list = (vg.isArray(data) ? data : data.values || []).map(value); + + // compute aggregates + for (i=0, len=list.length; i max) max = v; + sum += v; + delta = v - mean; + mean = mean + delta / (i+1); + M2 = M2 + delta * (v - mean); + } + M2 = M2 / (len - 1); + + var o = vg.isArray(data) ? {} : data; + if (median) { + list.sort(vg.numcmp); + i = list.length >> 1; + o[output.median] = list.length % 2 + ? list[i] + : (list[i-1] + list[i])/2; + } + o[output.count] = len; + o[output.min] = min; + o[output.max] = max; + o[output.sum] = sum; + o[output.mean] = mean; + o[output.variance] = M2; + o[output.stdev] = Math.sqrt(M2); + + if (assign) { + list = (vg.isArray(data) ? data : data.values); + v = {}; + v[output.count] = len; + v[output.min] = min; + v[output.max] = max; + v[output.sum] = sum; + v[output.mean] = mean; + v[output.variance] = M2; + v[output.stdev] = Math.sqrt(M2); + if (median) v[output.median] = o[output.median]; + for (i=0, len=list.length; i\~\&\|\?\:\+\-\/\*\%\!\^\,\;\[\]\{\}\(\) ]+)/; + + return function(x) { + var tokens = x.split(lexer), + t, v, i, n, sq, dq; + + for (sq=0, dq=0, i=0, n=tokens.length; i 0) ? "\n " : " "; + code += "o."+name+" = "+valueRef(name, ref)+";"; + vars[name] = true; + } + + if (vars.x2) { + if (vars.x) { + code += "\n if (o.x > o.x2) { " + + "var t = o.x; o.x = o.x2; o.x2 = t; };"; + code += "\n o.width = (o.x2 - o.x);"; + } else if (vars.width && !vars.x1) { + code += "\n o.x = (o.x2 - o.width);"; + } + } + + if (vars.y2) { + if (vars.y) { + code += "\n if (o.y > o.y2) { " + + "var t = o.y; o.y = o.y2; o.y2 = t; };"; + code += "\n o.height = (o.y2 - o.y);"; + } else if (vars.height && !vars.y1) { + code += "\n o.y = (o.y2 - o.height);"; + } + } + + if (hasPath(mark, vars)) { + code += "\n if (o['path:parsed']) o['path:parsed'] = null;" + } + code += "\n if (trans) trans.interpolate(item, o);"; + + try { + return Function("item", "group", "trans", code); + } catch (e) { + vg.error(e); + vg.log(code); + } + } + + function hasPath(mark, vars) { + return vars.path || + ((mark==="area" || mark==="line") && + (vars.x || vars.x2 || vars.width || + vars.y || vars.y2 || vars.height || + vars.tension || vars.interpolate)); + } + + var GROUP_VARS = { + "width": 1, + "height": 1, + "mark.group.width": 1, + "mark.group.height": 1 + }; + + function valueRef(name, ref) { + if (ref == null) return null; + var isColor = name==="fill" || name==="stroke"; + + if (isColor) { + if (ref.c) { + return colorRef("hcl", ref.h, ref.c, ref.l); + } else if (ref.h || ref.s) { + return colorRef("hsl", ref.h, ref.s, ref.l); + } else if (ref.l || ref.a) { + return colorRef("lab", ref.l, ref.a, ref.b); + } else if (ref.r || ref.g || ref.b) { + return colorRef("rgb", ref.r, ref.g, ref.b); + } + } + + // initialize value + var val = "item.datum.data"; + if (ref.value !== undefined) { + val = vg.str(ref.value); + } + + // get field reference for enclosing group + if (ref.group != null) { + var grp = ""; + if (vg.isString(ref.group)) { + grp = GROUP_VARS[ref.group] + ? "group." + ref.group + : "group.datum["+vg.field(ref.group).map(vg.str).join("][")+"]"; + } + } + + // get data field value + if (ref.field != null) { + if (vg.isString(ref.field)) { + val = "item.datum["+vg.field(ref.field).map(vg.str).join("][")+"]"; + if (ref.group != null) { val = grp+"["+val+"]"; } + } else { + val = "this.accessor(group.datum[" + + vg.field(ref.field.group).map(vg.str).join("][") + + "])(item.datum.data)"; + } + } else if (ref.group != null) { + val = grp; + } + + // run through scale function + if (ref.scale != null) { + var scale = vg.isString(ref.scale) + ? vg.str(ref.scale) + : (ref.scale.group ? "group" : "item") + + ".datum[" + vg.str(ref.scale.group || ref.scale.field) + "]"; + scale = "group.scales[" + scale + "]"; + val = scale + (ref.band ? ".rangeBand()" : "("+val+")"); + } + + // multiply, offset, return value + val = "(" + (ref.mult?(vg.number(ref.mult)+" * "):"") + val + ")" + + (ref.offset ? " + " + vg.number(ref.offset) : ""); + if (isColor) val = '('+val+')+""'; + return val; + } + + function colorRef(type, x, y, z) { + var xx = x ? valueRef("", x) : vg.config.color[type][0], + yy = y ? valueRef("", y) : vg.config.color[type][1], + zz = z ? valueRef("", z) : vg.config.color[type][2]; + return "(this.d3." + type + "(" + [xx,yy,zz].join(",") + ') + "")'; + } + + return compile; +})();vg.parse.scales = (function() { + var LINEAR = "linear", + ORDINAL = "ordinal", + LOG = "log", + POWER = "pow", + TIME = "time", + GROUP_PROPERTY = {width: 1, height: 1}; + + function scales(spec, scales, db, group) { + return (spec || []).reduce(function(o, def) { + var name = def.name, prev = name + ":prev"; + o[name] = scale(def, o[name], db, group); + o[prev] = o[prev] || o[name]; + return o; + }, scales || {}); + } + + function scale(def, scale, db, group) { + var s = instance(def, scale), + m = s.type===ORDINAL ? ordinal : quantitative, + rng = range(def, group), + data = vg.values(group.datum); + + m(def, s, rng, db, data); + return s; + } + + function instance(def, scale) { + var type = def.type || LINEAR; + if (!scale || type !== scale.type) { + var ctor = vg.config.scale[type] || d3.scale[type]; + if (!ctor) vg.error("Unrecognized scale type: " + type); + (scale = ctor()).type = scale.type || type; + scale.scaleName = def.name; + } + return scale; + } + + function ordinal(def, scale, rng, db, data) { + var domain, refs, values, str; + + // domain + domain = def.domain; + if (vg.isArray(domain)) { + scale.domain(domain); + } else if (vg.isObject(domain)) { + refs = def.domain.fields || vg.array(def.domain); + values = refs.reduce(function(values, r) { + var dat = vg.values(db[r.data] || data), + get = vg.accessor(vg.isString(r.field) + ? r.field : "data." + vg.accessor(r.field.group)(data)); + return vg.unique(dat, get, values); + }, []); + if (def.sort) values.sort(vg.cmp); + scale.domain(values); + } + + // range + str = typeof rng[0] === 'string'; + if (str || rng.length > 2) { + scale.range(rng); // color or shape values + } else if (def.points) { + scale.rangePoints(rng, def.padding||0); + } else if (def.round || def.round===undefined) { + scale.rangeRoundBands(rng, def.padding||0); + } else { + scale.rangeBands(rng, def.padding||0); + } + } + + function quantitative(def, scale, rng, db, data) { + var domain, refs, interval, z; + + // domain + domain = [null, null]; + function extract(ref, min, max, z) { + var dat = vg.values(db[ref.data] || data); + var fields = vg.array(ref.field).map(function(f) { + return vg.isString(f) ? f + : "data." + vg.accessor(f.group)(data); + }); + + fields.forEach(function(f,i) { + f = vg.accessor(f); + if (min) domain[0] = d3.min([domain[0], d3.min(dat, f)]); + if (max) domain[z] = d3.max([domain[z], d3.max(dat, f)]); + }); + } + if (def.domain !== undefined) { + if (vg.isArray(def.domain)) { + domain = def.domain.slice(); + } else if (vg.isObject(def.domain)) { + refs = def.domain.fields || vg.array(def.domain); + refs.forEach(function(r) { extract(r,1,1,1); }); + } else { + domain = def.domain; + } + } + z = domain.length - 1; + if (def.domainMin !== undefined) { + if (vg.isObject(def.domainMin)) { + domain[0] = null; + refs = def.domainMin.fields || vg.array(def.domainMin); + refs.forEach(function(r) { extract(r,1,0,z); }); + } else { + domain[0] = def.domainMin; + } + } + if (def.domainMax !== undefined) { + if (vg.isObject(def.domainMax)) { + domain[z] = null; + refs = def.domainMax.fields || vg.array(def.domainMax); + refs.forEach(function(r) { extract(r,0,1,z); }); + } else { + domain[z] = def.domainMax; + } + } + if (def.type !== LOG && def.type !== TIME && (def.zero || def.zero===undefined)) { + domain[0] = Math.min(0, domain[0]); + domain[z] = Math.max(0, domain[z]); + } + scale.domain(domain); + + // range + // vertical scales should flip by default, so use XOR here + if (def.range === "height") rng = rng.reverse(); + scale[def.round && scale.rangeRound ? "rangeRound" : "range"](rng); + + if (def.exponent && def.type===POWER) scale.exponent(def.exponent); + if (def.clamp) scale.clamp(true); + if (def.nice) { + if (def.type === TIME) { + interval = d3.time[def.nice]; + if (!interval) vg.error("Unrecognized interval: " + interval); + scale.nice(interval); + } else { + scale.nice(); + } + } + } + + function range(def, group) { + var rng = [null, null]; + + if (def.range !== undefined) { + if (typeof def.range === 'string') { + if (GROUP_PROPERTY[def.range]) { + rng = [0, group[def.range]]; + } else if (vg.config.range[def.range]) { + rng = vg.config.range[def.range]; + } else { + vg.error("Unrecogized range: "+def.range); + return rng; + } + } else if (vg.isArray(def.range)) { + rng = def.range; + } else { + rng = [0, def.range]; + } + } + if (def.rangeMin !== undefined) { + rng[0] = def.rangeMin; + } + if (def.rangeMax !== undefined) { + rng[rng.length-1] = def.rangeMax; + } + + if (def.reverse !== undefined) { + var rev = def.reverse; + if (vg.isObject(rev)) { + rev = vg.accessor(rev.field)(group.datum); + } + if (rev) rng = rng.reverse(); + } + + return rng; + } + + return scales; +})(); +vg.parse.spec = function(spec, callback, viewFactory) { + + viewFactory = viewFactory || vg.ViewFactory; + + function parse(spec) { + // protect against subsequent spec modification + spec = vg.duplicate(spec); + + var width = spec.width || 500, + height = spec.height || 500, + viewport = spec.viewport || null; + + var defs = { + width: width, + height: height, + viewport: viewport, + padding: vg.parse.padding(spec.padding), + marks: vg.parse.marks(spec, width, height), + data: vg.parse.data(spec.data, function() { callback(viewConstructor); }) + }; + + var viewConstructor = viewFactory(defs); + } + + vg.isObject(spec) ? parse(spec) : + d3.json(spec, function(error, json) { + error ? vg.error(error) : parse(json); + }); +};vg.parse.transform = function(def) { + var tx = vg.data[def.type](); + + vg.keys(def).forEach(function(k) { + if (k === 'type') return; + (tx[k])(def[k]); + }); + + return tx; +};vg.scene = {}; + +vg.scene.GROUP = "group", +vg.scene.ENTER = 0, +vg.scene.UPDATE = 1, +vg.scene.EXIT = 2; + +vg.scene.DEFAULT_DATA = {"sentinel":1} + +vg.scene.data = function(data, parentData) { + var DEFAULT = vg.scene.DEFAULT_DATA; + + // if data is undefined, inherit or use default + data = vg.values(data || parentData || [DEFAULT]); + + // if inheriting default data, ensure its in an array + if (data === DEFAULT) data = [DEFAULT]; + + return data; +}; + +vg.scene.fontString = function(o) { + return (o.fontStyle ? o.fontStyle + " " : "") + + (o.fontVariant ? o.fontVariant + " " : "") + + (o.fontWeight ? o.fontWeight + " " : "") + + (o.fontSize != null ? o.fontSize : vg.config.render.fontSize) + "px " + + (o.font || vg.config.render.font); +};vg.scene.Item = (function() { + function item(mark) { + this.mark = mark; + } + + var prototype = item.prototype; + + prototype.hasPropertySet = function(name) { + var props = this.mark.def.properties; + return props && props[name] != null; + }; + + prototype.cousin = function(offset, index) { + if (offset === 0) return this; + offset = offset || -1; + var mark = this.mark, + group = mark.group, + iidx = index==null ? mark.items.indexOf(this) : index, + midx = group.items.indexOf(mark) + offset; + return group.items[midx].items[iidx]; + }; + + prototype.sibling = function(offset) { + if (offset === 0) return this; + offset = offset || -1; + var mark = this.mark, + iidx = mark.items.indexOf(this) + offset; + return mark.items[iidx]; + }; + + prototype.remove = function() { + var item = this, + list = item.mark.items, + i = list.indexOf(item); + if (i >= 0) (i===list.length-1) ? list.pop() : list.splice(i, 1); + return item; + }; + + return item; +})(); + +vg.scene.item = function(mark) { + return new vg.scene.Item(mark); +};vg.scene.visit = function(node, func) { + var i, n, items; + if (func(node)) return true; + if (items = node.items) { + for (i=0, n=items.length; i0) s += "|"; + s += String(f[i](d)); + } + return s; + } + } + + return build; +})();vg.scene.bounds = (function() { + + var parse = vg.canvas.path.parse, + boundPath = vg.canvas.path.bounds, + areaPath = vg.canvas.path.area, + linePath = vg.canvas.path.line, + halfpi = Math.PI / 2, + sqrt3 = Math.sqrt(3), + tan30 = Math.tan(30 * Math.PI / 180), + gfx = null; + + function context() { + return gfx || (gfx = (vg.config.isNode + ? new (require("canvas"))(1,1) + : d3.select("body").append("canvas") + .attr("class", "vega_hidden") + .attr("width", 1) + .attr("height", 1) + .style("display", "none") + .node()) + .getContext("2d")); + } + + function pathBounds(o, path, bounds) { + if (path == null) { + bounds.set(0, 0, 0, 0); + } else { + boundPath(path, bounds); + if (o.stroke && o.opacity !== 0 && o.strokeWidth > 0) { + bounds.expand(o.strokeWidth); + } + } + return bounds; + } + + function path(o, bounds) { + var p = o.path + ? o["path:parsed"] || (o["path:parsed"] = parse(o.path)) + : null; + return pathBounds(o, p, bounds); + } + + function area(o, bounds) { + var items = o.mark.items, o = items[0]; + var p = o["path:parsed"] || (o["path:parsed"]=parse(areaPath(items))); + return pathBounds(items[0], p, bounds); + } + + function line(o, bounds) { + var items = o.mark.items, o = items[0]; + var p = o["path:parsed"] || (o["path:parsed"]=parse(linePath(items))); + return pathBounds(items[0], p, bounds); + } + + function rect(o, bounds) { + var x = o.x || 0, + y = o.y || 0, + w = (x + o.width) || 0, + h = (y + o.height) || 0; + bounds.set(x, y, w, h); + if (o.stroke && o.opacity !== 0 && o.strokeWidth > 0) { + bounds.expand(o.strokeWidth); + } + return bounds; + } + + function image(o, bounds) { + var w = o.width || 0, + h = o.height || 0, + x = (o.x||0) - (o.align === "center" + ? w/2 : (o.align === "right" ? w : 0)), + y = (o.y||0) - (o.baseline === "middle" + ? h/2 : (o.baseline === "bottom" ? h : 0)); + return bounds.set(x, y, x+w, y+h); + } + + function rule(o, bounds) { + var x1, y1; + bounds.set( + x1 = o.x || 0, + y1 = o.y || 0, + o.x2 != null ? o.x2 : x1, + o.y2 != null ? o.y2 : y1 + ); + if (o.stroke && o.opacity !== 0 && o.strokeWidth > 0) { + bounds.expand(o.strokeWidth); + } + return bounds; + } + + function arc(o, bounds) { + var cx = o.x || 0, + cy = o.y || 0, + ir = o.innerRadius || 0, + or = o.outerRadius || 0, + sa = (o.startAngle || 0) - halfpi, + ea = (o.endAngle || 0) - halfpi, + xmin = Infinity, xmax = -Infinity, + ymin = Infinity, ymax = -Infinity, + a, i, n, x, y, ix, iy, ox, oy; + + var angles = [sa, ea], + s = sa - (sa%halfpi); + for (i=0; i<4 && s 0) { + bounds.expand(o.strokeWidth); + } + return bounds; + } + + function symbol(o, bounds) { + var size = o.size != null ? o.size : 100, + x = o.x || 0, + y = o.y || 0, + r, t, rx, ry; + + switch (o.shape) { + case "cross": + r = Math.sqrt(size / 5) / 2; + t = 3*r; + bounds.set(x-t, y-t, x+y, y+t); + break; + + case "diamond": + ry = Math.sqrt(size / (2 * tan30)); + rx = ry * tan30; + bounds.set(x-rx, y-ry, x+rx, y+ry); + break; + + case "square": + t = Math.sqrt(size); + r = t / 2; + bounds.set(x-r, y-r, x+r, y+r); + break; + + case "triangle-down": + rx = Math.sqrt(size / sqrt3); + ry = rx * sqrt3 / 2; + bounds.set(x-rx, y-ry, x+rx, y+ry); + break; + + case "triangle-up": + rx = Math.sqrt(size / sqrt3); + ry = rx * sqrt3 / 2; + bounds.set(x-rx, y-ry, x+rx, y+ry); + break; + + default: + r = Math.sqrt(size/Math.PI); + bounds.set(x-r, y-r, x+r, y+r); + } + if (o.stroke && o.opacity !== 0 && o.strokeWidth > 0) { + bounds.expand(o.strokeWidth); + } + return bounds; + } + + function text(o, bounds, noRotate) { + var x = (o.x || 0) + (o.dx || 0), + y = (o.y || 0) + (o.dy || 0), + h = o.fontSize || vg.config.render.fontSize, + a = o.align, + b = o.baseline, + g = context(), w; + + g.font = vg.scene.fontString(o); + g.textAlign = a || "left"; + g.textBaseline = b || "alphabetic"; + w = g.measureText(o.text || "").width; + + // horizontal + if (a === "center") { + x = x - (w / 2); + } else if (a === "right") { + x = x - w; + } else { + // left by default, do nothing + } + + /// TODO find a robust solution for heights. + /// These offsets work for some but not all fonts. + + // vertical + if (b === "top") { + y = y + (h/5); + } else if (b === "bottom") { + y = y - h; + } else if (b === "middle") { + y = y - (h/2) + (h/10); + } else { + y = y - 4*h/5; // alphabetic by default + } + + bounds.set(x, y, x+w, y+h); + if (o.angle && !noRotate) { + bounds.rotate(o.angle*Math.PI/180, o.x||0, o.y||0); + } + return bounds.expand(noRotate ? 0 : 1); + } + + function group(g, bounds, includeLegends) { + var axes = g.axisItems || [], + legends = g.legendItems || [], j, m; + + for (j=0, m=axes.length; j 1) f = 1; + e = curr.ease(f); + + for (i=0, n=curr.length; i 1 ? +y : tickMajorSize, + end = n > 0 ? +arguments[n] : tickMajorSize; + + if (tickMajorSize !== major || + tickMinorSize !== minor || + tickEndSize !== end) { + reset(); + } + + tickMajorSize = major; + tickMinorSize = minor; + tickEndSize = end; + return axis; + }; + + axis.tickSubdivide = function(x) { + if (!arguments.length) return tickSubdivide; + tickSubdivide = +x; + return axis; + }; + + axis.offset = function(x) { + if (!arguments.length) return offset; + offset = vg.isObject(x) ? x : +x; + return axis; + }; + + axis.tickPadding = function(x) { + if (!arguments.length) return tickPadding; + if (tickPadding !== +x) { tickPadding = +x; reset(); } + return axis; + }; + + axis.titleOffset = function(x) { + if (!arguments.length) return titleOffset; + if (titleOffset !== +x) { titleOffset = +x; reset(); } + return axis; + }; + + axis.layer = function(x) { + if (!arguments.length) return layer; + if (layer !== x) { layer = x; reset(); } + return axis; + }; + + axis.grid = function(x) { + if (!arguments.length) return grid; + if (grid !== x) { grid = x; reset(); } + return axis; + }; + + axis.gridLineProperties = function(x) { + if (!arguments.length) return gridLineStyle; + if (gridLineStyle !== x) { gridLineStyle = x; } + return axis; + }; + + axis.majorTickProperties = function(x) { + if (!arguments.length) return majorTickStyle; + if (majorTickStyle !== x) { majorTickStyle = x; } + return axis; + }; + + axis.minorTickProperties = function(x) { + if (!arguments.length) return minorTickStyle; + if (minorTickStyle !== x) { minorTickStyle = x; } + return axis; + }; + + axis.tickLabelProperties = function(x) { + if (!arguments.length) return tickLabelStyle; + if (tickLabelStyle !== x) { tickLabelStyle = x; } + return axis; + }; + + axis.titleProperties = function(x) { + if (!arguments.length) return titleStyle; + if (titleStyle !== x) { titleStyle = x; } + return axis; + }; + + axis.domainProperties = function(x) { + if (!arguments.length) return domainStyle; + if (domainStyle !== x) { domainStyle = x; } + return axis; + }; + + axis.reset = function() { reset(); }; + + return axis; +}; + +var vg_axisOrients = {top: 1, right: 1, bottom: 1, left: 1}; + +function vg_axisSubdivide(scale, ticks, m) { + subticks = []; + if (m && ticks.length > 1) { + var extent = vg_axisScaleExtent(scale.domain()), + subticks, + i = -1, + n = ticks.length, + d = (ticks[1] - ticks[0]) / ++m, + j, + v; + while (++i < n) { + for (j = m; --j > 0;) { + if ((v = +ticks[i] - j * d) >= extent[0]) { + subticks.push(v); + } + } + } + for (--i, j = 0; ++j < m && (v = +ticks[i] + j * d) < extent[1];) { + subticks.push(v); + } + } + return subticks; +} + +function vg_axisScaleExtent(domain) { + var start = domain[0], stop = domain[domain.length - 1]; + return start < stop ? [start, stop] : [stop, start]; +} + +function vg_axisScaleRange(scale) { + return scale.rangeExtent + ? scale.rangeExtent() + : vg_axisScaleExtent(scale.range()); +} + +var vg_axisAlign = { + bottom: "center", + top: "center", + left: "right", + right: "left" +}; + +var vg_axisBaseline = { + bottom: "top", + top: "bottom", + left: "middle", + right: "middle" +}; + +function vg_axisLabelExtend(orient, labels, oldScale, newScale, size, pad) { + size = Math.max(size, 0) + pad; + if (orient === "left" || orient === "top") { + size *= -1; + } + if (orient === "top" || orient === "bottom") { + vg.extend(labels.properties.enter, { + x: oldScale, + y: {value: size}, + }); + vg.extend(labels.properties.update, { + x: newScale, + y: {value: size}, + align: {value: "center"}, + baseline: {value: vg_axisBaseline[orient]} + }); + } else { + vg.extend(labels.properties.enter, { + x: {value: size}, + y: oldScale, + }); + vg.extend(labels.properties.update, { + x: {value: size}, + y: newScale, + align: {value: vg_axisAlign[orient]}, + baseline: {value: "middle"} + }); + } +} + +function vg_axisTicksExtend(orient, ticks, oldScale, newScale, size) { + var sign = (orient === "left" || orient === "top") ? -1 : 1; + if (size === Infinity) { + size = (orient === "top" || orient === "bottom") + ? {group: "mark.group.height", mult: -sign} + : {group: "mark.group.width", mult: -sign}; + } else { + size = {value: sign * size}; + } + if (orient === "top" || orient === "bottom") { + vg.extend(ticks.properties.enter, { + x: oldScale, + y: {value: 0}, + y2: size + }); + vg.extend(ticks.properties.update, { + x: newScale, + y: {value: 0}, + y2: size + }); + vg.extend(ticks.properties.exit, { + x: newScale, + }); + } else { + vg.extend(ticks.properties.enter, { + x: {value: 0}, + x2: size, + y: oldScale + }); + vg.extend(ticks.properties.update, { + x: {value: 0}, + x2: size, + y: newScale + }); + vg.extend(ticks.properties.exit, { + y: newScale, + }); + } +} + +function vg_axisTitleExtend(orient, title, range, offset) { + var mid = ~~((range[1] - range[0]) / 2), + sign = (orient === "top" || orient === "left") ? -1 : 1; + + if (orient === "bottom" || orient === "top") { + vg.extend(title.properties.update, { + x: {value: mid}, + y: {value: sign*offset}, + angle: {value: 0} + }); + } else { + vg.extend(title.properties.update, { + x: {value: sign*offset}, + y: {value: mid}, + angle: {value: -90} + }); + } +} + +function vg_axisDomainExtend(orient, domain, range, size) { + var path; + if (orient === "top" || orient === "left") { + size = -1 * size; + } + if (orient === "bottom" || orient === "top") { + path = "M" + range[0] + "," + size + "V0H" + range[1] + "V" + size; + } else { + path = "M" + size + "," + range[0] + "H0V" + range[1] + "H" + size; + } + domain.properties.update.path = {value: path}; +} + +function vg_axisUpdate(item, group, trans) { + var o = trans ? {} : item, + offset = item.mark.def.offset, + orient = item.mark.def.orient, + width = group.width, + height = group.height; // TODO fallback to global w,h? + + if (vg.isObject(offset)) { + offset = -group.scales[offset.scale](offset.value); + } + + switch (orient) { + case "left": { o.x = -offset; o.y = 0; break; } + case "right": { o.x = width + offset; o.y = 0; break; } + case "bottom": { o.x = 0; o.y = height + offset; break; } + case "top": { o.x = 0; o.y = -offset; break; } + default: { o.x = 0; o.y = 0; } + } + + if (trans) trans.interpolate(item, o); +} + +function vg_axisTicks() { + return { + type: "rule", + interactive: false, + key: "data", + properties: { + enter: { + stroke: {value: vg.config.axis.tickColor}, + strokeWidth: {value: vg.config.axis.tickWidth}, + opacity: {value: 1e-6} + }, + exit: { opacity: {value: 1e-6} }, + update: { opacity: {value: 1} } + } + }; +} + +function vg_axisTickLabels() { + return { + type: "text", + interactive: true, + key: "data", + properties: { + enter: { + fill: {value: vg.config.axis.tickLabelColor}, + font: {value: vg.config.axis.tickLabelFont}, + fontSize: {value: vg.config.axis.tickLabelFontSize}, + opacity: {value: 1e-6}, + text: {field: "label"} + }, + exit: { opacity: {value: 1e-6} }, + update: { opacity: {value: 1} } + } + }; +} + +function vg_axisTitle() { + return { + type: "text", + interactive: true, + properties: { + enter: { + font: {value: vg.config.axis.titleFont}, + fontSize: {value: vg.config.axis.titleFontSize}, + fontWeight: {value: vg.config.axis.titleFontWeight}, + fill: {value: vg.config.axis.titleColor}, + align: {value: "center"}, + baseline: {value: "middle"}, + text: {field: "data"} + }, + update: {} + } + }; +} + +function vg_axisDomain() { + return { + type: "path", + interactive: false, + properties: { + enter: { + x: {value: 0.5}, + y: {value: 0.5}, + stroke: {value: vg.config.axis.axisColor}, + strokeWidth: {value: vg.config.axis.axisWidth} + }, + update: {} + } + }; +} +vg.scene.legend = function() { + var size = null, + shape = null, + fill = null, + stroke = null, + spacing = null, + values = null, + format = null, + title = undefined, + orient = "right", + offset = vg.config.legend.offset, + padding = vg.config.legend.padding, + legendDef, + tickArguments = [5], + legendStyle = {}, + symbolStyle = {}, + gradientStyle = {}, + titleStyle = {}, + labelStyle = {}; + + var legend = {}, + legendDef = null; + + function reset() { legendDef = null; } + + legend.def = function() { + var scale = size || shape || fill || stroke; + if (!legendDef) { + legendDef = (scale===fill || scale===stroke) && !discrete(scale.type) + ? quantDef(scale) + : ordinalDef(scale); + } + legendDef.orient = orient; + legendDef.offset = offset; + legendDef.padding = padding; + return legendDef; + }; + + function discrete(type) { + return type==="ordinal" || type==="quantize" + || type==="quantile" || type==="threshold"; + } + + function ordinalDef(scale) { + var def = o_legend_def(size, shape, fill, stroke); + + // generate data + var data = (values == null + ? (scale.ticks ? scale.ticks.apply(scale, tickArguments) : scale.domain()) + : values).map(vg.data.ingest); + var fmt = format==null ? (scale.tickFormat ? scale.tickFormat.apply(scale, tickArguments) : String) : format; + + // determine spacing between legend entries + var fs, range, offset, pad=5, domain = d3.range(data.length); + if (size) { + range = data.map(function(x) { return Math.sqrt(size(x.data)); }); + offset = d3.max(range); + range = range.reduce(function(a,b,i,z) { + if (i > 0) a[i] = a[i-1] + z[i-1]/2 + pad; + return (a[i] += b/2, a); }, [0]).map(Math.round); + } else { + offset = Math.round(Math.sqrt(vg.config.legend.symbolSize)); + range = spacing + || (fs = labelStyle.fontSize) && (fs.value + pad) + || (vg.config.legend.labelFontSize + pad); + range = domain.map(function(d,i) { + return Math.round(offset/2 + i*range); + }); + } + + // account for padding and title size + var sz = padding, ts; + if (title) { + ts = titleStyle.fontSize; + sz += 5 + ((ts && ts.value) || vg.config.legend.titleFontSize); + } + for (var i=0, n=range.length; i this._width ? Math.ceil(+b.x2 - this._width) + inset : 0, + b = b.y2 > this._height ? Math.ceil(+b.y2 - this._height) + inset : 0; + pad = {left:l, top:t, right:r, bottom:b}; + + if (this._strict) { + this._autopad = 0; + this._padding = pad; + this._width = Math.max(0, this.__width - (l+r)); + this._height = Math.max(0, this.__height - (t+b)); + this._model.width(this._width); + this._model.height(this._height); + if (this._el) this.initialize(this._el.parentNode); + this.update({props:"enter"}).update({props:"update"}); + } else { + this.padding(pad).update(opt); + } + return this; + }; + + prototype.viewport = function(size) { + if (!arguments.length) return this._viewport; + if (this._viewport !== size) { + this._viewport = size; + if (this._el) this.initialize(this._el.parentNode); + } + return this; + }; + + prototype.renderer = function(type) { + if (!arguments.length) return this._io; + if (type === "canvas") type = vg.canvas; + if (type === "svg") type = vg.svg; + if (this._io !== type) { + this._io = type; + this._renderer = null; + if (this._el) this.initialize(this._el.parentNode); + if (this._build) this.render(); + } + return this; + }; + + prototype.defs = function(defs) { + if (!arguments.length) return this._model.defs(); + this._model.defs(defs); + return this; + }; + + prototype.data = function(data) { + if (!arguments.length) return this._model.data(); + var ingest = vg.keys(data).reduce(function(d, k) { + return (d[k] = vg.data.ingestAll(data[k]), d); + }, {}); + this._model.data(ingest); + this._build = false; + return this; + }; + + prototype.model = function(model) { + if (!arguments.length) return this._model; + if (this._model !== model) { + this._model = model; + if (this._handler) this._handler.model(model); + } + return this; + }; + + prototype.initialize = function(el) { + var v = this, prevHandler, + w = v._width, h = v._height, pad = v._padding; + + // clear pre-existing container + d3.select(el).select("div.vega").remove(); + + // add div container + this._el = el = d3.select(el) + .append("div") + .attr("class", "vega") + .style("position", "relative") + .node(); + if (v._viewport) { + d3.select(el) + .style("width", (v._viewport[0] || w)+"px") + .style("height", (v._viewport[1] || h)+"px") + .style("overflow", "auto"); + } + + // renderer + v._renderer = (v._renderer || new this._io.Renderer()) + .initialize(el, w, h, pad); + + // input handler + prevHandler = v._handler; + v._handler = new this._io.Handler() + .initialize(el, pad, v) + .model(v._model); + + if (prevHandler) { + prevHandler.handlers().forEach(function(h) { + v._handler.on(h.type, h.handler); + }); + } + + return this; + }; + + prototype.render = function(items) { + this._renderer.render(this._model.scene(), items); + return this; + }; + + prototype.on = function() { + this._handler.on.apply(this._handler, arguments); + return this; + }; + + prototype.off = function() { + this._handler.off.apply(this._handler, arguments); + return this; + }; + + prototype.update = function(opt) { + opt = opt || {}; + var view = this, + trans = opt.duration + ? vg.scene.transition(opt.duration, opt.ease) + : null; + + view._build = view._build || (view._model.build(), true); + view._model.encode(trans, opt.props, opt.items); + + if (trans) { + trans.start(function(items) { + view._renderer.render(view._model.scene(), items); + }); + } + else view.render(opt.items); + + return view.autopad(opt); + }; + + return view; +})(); + +// view constructor factory +// takes definitions from parsed specification as input +// returns a view constructor +vg.ViewFactory = function(defs) { + return function(opt) { + opt = opt || {}; + var v = new vg.View() + .width(defs.width) + .height(defs.height) + .padding(defs.padding) + .viewport(defs.viewport) + .renderer(opt.renderer || "canvas") + .defs(defs); + + if (defs.data.load) v.data(defs.data.load); + if (opt.data) v.data(opt.data); + if (opt.el) v.initialize(opt.el); + + if (opt.hover !== false) { + v.on("mouseover", function(evt, item) { + if (item.hasPropertySet("hover")) { + this.update({props:"hover", items:item}); + } + }) + .on("mouseout", function(evt, item) { + if (item.hasPropertySet("hover")) { + this.update({props:"update", items:item}); + } + }); + } + + return v; + }; +}; +vg.Spec = (function() { + var spec = function(s) { + this.spec = { + width: 500, + height: 500, + padding: 0, + data: [], + scales: [], + axes: [], + marks: [] + }; + if (s) vg.extend(this.spec, s); + }; + + var prototype = spec.prototype; + + prototype.width = function(w) { + this.spec.width = w; + return this; + }; + + prototype.height = function(h) { + this.spec.height = h; + return this; + }; + + prototype.padding = function(p) { + this.spec.padding = p; + return this; + }; + + prototype.viewport = function(v) { + this.spec.viewport = v; + return this; + }; + + prototype.data = function(name, params) { + if (!params) params = vg.isString(name) ? {name: name} : name; + else params.name = name; + this.spec.data.push(params); + return this; + }; + + prototype.scale = function(name, params) { + if (!params) params = vg.isString(name) ? {name: name} : name; + else params.name = name; + this.spec.scales.push(params); + return this; + }; + + prototype.axis = function(params) { + this.spec.axes.push(params); + return this; + }; + + prototype.mark = function(type, mark) { + if (!mark) mark = {type: type}; + else mark.type = type; + mark.properties = {}; + this.spec.marks.push(mark); + + var that = this; + return { + from: function(name, obj) { + mark.from = obj + ? (obj.data = name, obj) + : vg.isString(name) ? {data: name} : name; + return this; + }, + prop: function(name, obj) { + mark.properties[name] = vg.keys(obj).reduce(function(o,k) { + var v = obj[k]; + return (o[k] = vg.isObject(v) ? v : {value: v}, o); + }, {}); + return this; + }, + done: function() { return that; } + }; + }; + + prototype.parse = function(callback) { + vg.parse.spec(this.spec, callback); + }; + + prototype.json = function() { + return this.spec; + }; + + return spec; +})(); + +vg.spec = function(s) { + return new vg.Spec(s); +}; +vg.headless = {};vg.headless.View = (function() { + + var view = function(width, height, pad, type) { + this._canvas = null; + this._type = type; + this._el = "body"; + this._build = false; + this._model = new vg.Model(); + this._width = this.__width = width || 500; + this._height = this.__height = height || 500; + this._autopad = 1; + this._padding = pad || {top:0, left:0, bottom:0, right:0}; + this._renderer = new vg[type].Renderer(); + this.initialize(); + }; + + var prototype = view.prototype; + + prototype.el = function(el) { + if (!arguments.length) return this._el; + if (this._el !== el) { + this._el = el; + this.initialize(); + } + return this; + }; + + prototype.width = function(width) { + if (!arguments.length) return this._width; + if (this._width !== width) { + this._width = width; + this.initialize(); + this._model.width(width); + } + return this; + }; + + prototype.height = function(height) { + if (!arguments.length) return this._height; + if (this._height !== height) { + this._height = height; + this.initialize(); + this._model.height(this._height); + } + return this; + }; + + prototype.padding = function(pad) { + if (!arguments.length) return this._padding; + if (this._padding !== pad) { + if (vg.isString(pad)) { + this._autopad = 1; + this._padding = {top:0, left:0, bottom:0, right:0}; + this._strict = (pad === "strict"); + } else { + this._autopad = 0; + this._padding = pad; + this._strict = false; + } + this.initialize(); + } + return this; + }; + + prototype.autopad = function(opt) { + if (this._autopad < 1) return this; + else this._autopad = 0; + + var pad = this._padding, + b = this._model.scene().bounds, + inset = vg.config.autopadInset, + l = b.x1 < 0 ? Math.ceil(-b.x1) + inset : 0, + t = b.y1 < 0 ? Math.ceil(-b.y1) + inset : 0, + r = b.x2 > this._width ? Math.ceil(+b.x2 - this._width) + inset : 0, + b = b.y2 > this._height ? Math.ceil(+b.y2 - this._height) + inset : 0; + pad = {left:l, top:t, right:r, bottom:b}; + + if (this._strict) { + this._autopad = 0; + this._padding = pad; + this._width = Math.max(0, this.__width - (l+r)); + this._height = Math.max(0, this.__height - (t+b)); + this._model.width(this._width); + this._model.height(this._height); + if (this._el) this.initialize(); + this.update({props:"enter"}).update({props:"update"}); + } else { + this.padding(pad).update(opt); + } + return this; + }; + + prototype.viewport = function() { + if (!arguments.length) return null; + return this; + }; + + prototype.defs = function(defs) { + if (!arguments.length) return this._model.defs(); + this._model.defs(defs); + return this; + }; + + prototype.data = function(data) { + if (!arguments.length) return this._model.data(); + var ingest = vg.keys(data).reduce(function(d, k) { + return (d[k] = vg.data.ingestAll(data[k]), d); + }, {}); + this._model.data(ingest); + this._build = false; + return this; + }; + + prototype.renderer = function() { + return this._renderer; + }; + + prototype.canvas = function() { + return this._canvas; + }; + + prototype.canvasAsync = function(callback) { + var r = this._renderer, view = this; + + function wait() { + if (r.pendingImages() === 0) { + view.render(); // re-render with all images + callback(view._canvas); + } else { + setTimeout(wait, 10); + } + } + + // if images loading, poll until ready + (r.pendingImages() > 0) ? wait() : callback(this._canvas); + }; + + prototype.svg = function() { + if (this._type !== "svg") return null; + + var p = this._padding, + w = this._width + (p ? p.left + p.right : 0), + h = this._height + (p ? p.top + p.bottom : 0); + + // build svg text + var svg = d3.select(this._el) + .select("svg").node().innerHTML + .replace(/ href=/g, " xlink:href="); // ns hack. sigh. + + return '' + svg + '' + }; + + prototype.initialize = function() { + var w = this._width, + h = this._height, + pad = this._padding; + + if (this._type === "svg") { + this.initSVG(w, h, pad); + } else { + this.initCanvas(w, h, pad); + } + + return this; + }; + + prototype.initCanvas = function(w, h, pad) { + var Canvas = require("canvas"), + tw = w + pad.left + pad.right, + th = h + pad.top + pad.bottom, + canvas = this._canvas = new Canvas(tw, th), + ctx = canvas.getContext("2d"); + + // setup canvas context + ctx.setTransform(1, 0, 0, 1, pad.left, pad.top); + + // configure renderer + this._renderer.context(ctx); + this._renderer.resize(w, h, pad); + }; + + prototype.initSVG = function(w, h, pad) { + var tw = w + pad.left + pad.right, + th = h + pad.top + pad.bottom; + + // configure renderer + this._renderer.initialize(this._el, w, h, pad); + } + + prototype.render = function(items) { + this._renderer.render(this._model.scene(), items); + return this; + }; + + prototype.update = function(opt) { + opt = opt || {}; + var view = this; + view._build = view._build || (view._model.build(), true); + view._model.encode(null, opt.props, opt.items); + view.render(opt.items); + return view.autopad(opt); + }; + + return view; +})(); + +// headless view constructor factory +// takes definitions from parsed specification as input +// returns a view constructor +vg.headless.View.Factory = function(defs) { + return function(opt) { + opt = opt || {}; + var w = defs.width, + h = defs.height, + p = defs.padding, + r = opt.renderer || "canvas", + v = new vg.headless.View(w, h, p, r).defs(defs); + if (defs.data.load) v.data(defs.data.load); + if (opt.data) v.data(opt.data); + return v; + }; +};vg.headless.render = function(opt, callback) { + function draw(chart) { + try { + // create and render view + var view = chart({ + data: opt.data, + renderer: opt.renderer + }).update(); + + if (opt.renderer === "svg") { + // extract rendered svg + callback(null, {svg: view.svg()}); + } else { + // extract rendered canvas, waiting for any images to load + view.canvasAsync(function(canvas) { + callback(null, {canvas: canvas}); + }); + } + } catch (err) { + callback(err, null); + } + } + + vg.parse.spec(opt.spec, draw, vg.headless.View.Factory); +}; return vg; +})(d3, typeof topojson === "undefined" ? null : topojson); +// assumes D3 and topojson in global namespace diff --git a/dashboard/app/stats.html b/dashboard/app/stats.html new file mode 100644 index 00000000000..eb324f1b22f --- /dev/null +++ b/dashboard/app/stats.html @@ -0,0 +1,51 @@ + + + + + + + + + etcd Browser + + + + + + + + + + + + + + + + +
+
+ + + + + + + + + + + + + + + + + + + diff --git a/dashboard/app/styles/bootstrap.css b/dashboard/app/styles/bootstrap.css new file mode 100644 index 00000000000..b725064aab2 --- /dev/null +++ b/dashboard/app/styles/bootstrap.css @@ -0,0 +1,6167 @@ +/*! + * Bootstrap v2.3.2 + * + * Copyright 2012 Twitter, Inc + * Licensed under the Apache License v2.0 + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Designed and built with all the love in the world @twitter by @mdo and @fat. + */ + +.clearfix { + *zoom: 1; +} + +.clearfix:before, +.clearfix:after { + display: table; + line-height: 0; + content: ""; +} + +.clearfix:after { + clear: both; +} + +.hide-text { + font: 0/0 a; + color: transparent; + text-shadow: none; + background-color: transparent; + border: 0; +} + +.input-block-level { + display: block; + width: 100%; + min-height: 30px; + -webkit-box-sizing: border-box; + -moz-box-sizing: border-box; + box-sizing: border-box; +} + +article, +aside, +details, +figcaption, +figure, +footer, +header, +hgroup, +nav, +section { + display: block; +} + +audio, +canvas, +video { + display: inline-block; + *display: inline; + *zoom: 1; +} + +audio:not([controls]) { + display: none; +} + +html { + font-size: 100%; + -webkit-text-size-adjust: 100%; + -ms-text-size-adjust: 100%; +} + +a:focus { + outline: thin dotted #333; + outline: 5px auto -webkit-focus-ring-color; + outline-offset: -2px; +} + +a:hover, +a:active { + outline: 0; +} + +sub, +sup { + position: relative; + font-size: 75%; + line-height: 0; + vertical-align: baseline; +} + +sup { + top: -0.5em; +} + +sub { + bottom: -0.25em; +} + +img { + width: auto\9; + height: auto; + max-width: 100%; + vertical-align: middle; + border: 0; + -ms-interpolation-mode: bicubic; +} + +#map_canvas img, +.google-maps img { + max-width: none; +} + +button, +input, +select, +textarea { + margin: 0; + font-size: 100%; + vertical-align: middle; +} + +button, +input { + *overflow: visible; + line-height: normal; +} + +button::-moz-focus-inner, +input::-moz-focus-inner { + padding: 0; + border: 0; +} + +button, +html input[type="button"], +input[type="reset"], +input[type="submit"] { + cursor: pointer; + -webkit-appearance: button; +} + +label, +select, +button, +input[type="button"], +input[type="reset"], +input[type="submit"], +input[type="radio"], +input[type="checkbox"] { + cursor: pointer; +} + +input[type="search"] { + -webkit-box-sizing: content-box; + -moz-box-sizing: content-box; + box-sizing: content-box; + -webkit-appearance: textfield; +} + +input[type="search"]::-webkit-search-decoration, +input[type="search"]::-webkit-search-cancel-button { + -webkit-appearance: none; +} + +textarea { + overflow: auto; + vertical-align: top; +} + +@media print { + * { + color: #000 !important; + text-shadow: none !important; + background: transparent !important; + box-shadow: none !important; + } + a, + a:visited { + text-decoration: underline; + } + a[href]:after { + content: " (" attr(href) ")"; + } + abbr[title]:after { + content: " (" attr(title) ")"; + } + .ir a:after, + a[href^="javascript:"]:after, + a[href^="#"]:after { + content: ""; + } + pre, + blockquote { + border: 1px solid #999; + page-break-inside: avoid; + } + thead { + display: table-header-group; + } + tr, + img { + page-break-inside: avoid; + } + img { + max-width: 100% !important; + } + @page { + margin: 0.5cm; + } + p, + h2, + h3 { + orphans: 3; + widows: 3; + } + h2, + h3 { + page-break-after: avoid; + } +} + +body { + margin: 0; + font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; + font-size: 14px; + line-height: 20px; + color: #333333; + background-color: #ffffff; +} + +a { + color: #0088cc; + text-decoration: none; +} + +a:hover, +a:focus { + color: #005580; + text-decoration: underline; +} + +.img-rounded { + -webkit-border-radius: 6px; + -moz-border-radius: 6px; + border-radius: 6px; +} + +.img-polaroid { + padding: 4px; + background-color: #fff; + border: 1px solid #ccc; + border: 1px solid rgba(0, 0, 0, 0.2); + -webkit-box-shadow: 0 1px 3px rgba(0, 0, 0, 0.1); + -moz-box-shadow: 0 1px 3px rgba(0, 0, 0, 0.1); + box-shadow: 0 1px 3px rgba(0, 0, 0, 0.1); +} + +.img-circle { + -webkit-border-radius: 500px; + -moz-border-radius: 500px; + border-radius: 500px; +} + +.row { + margin-left: -20px; + *zoom: 1; +} + +.row:before, +.row:after { + display: table; + line-height: 0; + content: ""; +} + +.row:after { + clear: both; +} + +[class*="span"] { + float: left; + min-height: 1px; + margin-left: 20px; +} + +.container, +.navbar-static-top .container, +.navbar-fixed-top .container, +.navbar-fixed-bottom .container { + width: 940px; +} + +.span12 { + width: 940px; +} + +.span11 { + width: 860px; +} + +.span10 { + width: 780px; +} + +.span9 { + width: 700px; +} + +.span8 { + width: 620px; +} + +.span7 { + width: 540px; +} + +.span6 { + width: 460px; +} + +.span5 { + width: 380px; +} + +.span4 { + width: 300px; +} + +.span3 { + width: 220px; +} + +.span2 { + width: 140px; +} + +.span1 { + width: 60px; +} + +.offset12 { + margin-left: 980px; +} + +.offset11 { + margin-left: 900px; +} + +.offset10 { + margin-left: 820px; +} + +.offset9 { + margin-left: 740px; +} + +.offset8 { + margin-left: 660px; +} + +.offset7 { + margin-left: 580px; +} + +.offset6 { + margin-left: 500px; +} + +.offset5 { + margin-left: 420px; +} + +.offset4 { + margin-left: 340px; +} + +.offset3 { + margin-left: 260px; +} + +.offset2 { + margin-left: 180px; +} + +.offset1 { + margin-left: 100px; +} + +.row-fluid { + width: 100%; + *zoom: 1; +} + +.row-fluid:before, +.row-fluid:after { + display: table; + line-height: 0; + content: ""; +} + +.row-fluid:after { + clear: both; +} + +.row-fluid [class*="span"] { + display: block; + float: left; + width: 100%; + min-height: 30px; + margin-left: 2.127659574468085%; + *margin-left: 2.074468085106383%; + -webkit-box-sizing: border-box; + -moz-box-sizing: border-box; + box-sizing: border-box; +} + +.row-fluid [class*="span"]:first-child { + margin-left: 0; +} + +.row-fluid .controls-row [class*="span"] + [class*="span"] { + margin-left: 2.127659574468085%; +} + +.row-fluid .span12 { + width: 100%; + *width: 99.94680851063829%; +} + +.row-fluid .span11 { + width: 91.48936170212765%; + *width: 91.43617021276594%; +} + +.row-fluid .span10 { + width: 82.97872340425532%; + *width: 82.92553191489361%; +} + +.row-fluid .span9 { + width: 74.46808510638297%; + *width: 74.41489361702126%; +} + +.row-fluid .span8 { + width: 65.95744680851064%; + *width: 65.90425531914893%; +} + +.row-fluid .span7 { + width: 57.44680851063829%; + *width: 57.39361702127659%; +} + +.row-fluid .span6 { + width: 48.93617021276595%; + *width: 48.88297872340425%; +} + +.row-fluid .span5 { + width: 40.42553191489362%; + *width: 40.37234042553192%; +} + +.row-fluid .span4 { + width: 31.914893617021278%; + *width: 31.861702127659576%; +} + +.row-fluid .span3 { + width: 23.404255319148934%; + *width: 23.351063829787233%; +} + +.row-fluid .span2 { + width: 14.893617021276595%; + *width: 14.840425531914894%; +} + +.row-fluid .span1 { + width: 6.382978723404255%; + *width: 6.329787234042553%; +} + +.row-fluid .offset12 { + margin-left: 104.25531914893617%; + *margin-left: 104.14893617021275%; +} + +.row-fluid .offset12:first-child { + margin-left: 102.12765957446808%; + *margin-left: 102.02127659574467%; +} + +.row-fluid .offset11 { + margin-left: 95.74468085106382%; + *margin-left: 95.6382978723404%; +} + +.row-fluid .offset11:first-child { + margin-left: 93.61702127659574%; + *margin-left: 93.51063829787232%; +} + +.row-fluid .offset10 { + margin-left: 87.23404255319149%; + *margin-left: 87.12765957446807%; +} + +.row-fluid .offset10:first-child { + margin-left: 85.1063829787234%; + *margin-left: 84.99999999999999%; +} + +.row-fluid .offset9 { + margin-left: 78.72340425531914%; + *margin-left: 78.61702127659572%; +} + +.row-fluid .offset9:first-child { + margin-left: 76.59574468085106%; + *margin-left: 76.48936170212764%; +} + +.row-fluid .offset8 { + margin-left: 70.2127659574468%; + *margin-left: 70.10638297872339%; +} + +.row-fluid .offset8:first-child { + margin-left: 68.08510638297872%; + *margin-left: 67.9787234042553%; +} + +.row-fluid .offset7 { + margin-left: 61.70212765957446%; + *margin-left: 61.59574468085106%; +} + +.row-fluid .offset7:first-child { + margin-left: 59.574468085106375%; + *margin-left: 59.46808510638297%; +} + +.row-fluid .offset6 { + margin-left: 53.191489361702125%; + *margin-left: 53.085106382978715%; +} + +.row-fluid .offset6:first-child { + margin-left: 51.063829787234035%; + *margin-left: 50.95744680851063%; +} + +.row-fluid .offset5 { + margin-left: 44.68085106382979%; + *margin-left: 44.57446808510638%; +} + +.row-fluid .offset5:first-child { + margin-left: 42.5531914893617%; + *margin-left: 42.4468085106383%; +} + +.row-fluid .offset4 { + margin-left: 36.170212765957444%; + *margin-left: 36.06382978723405%; +} + +.row-fluid .offset4:first-child { + margin-left: 34.04255319148936%; + *margin-left: 33.93617021276596%; +} + +.row-fluid .offset3 { + margin-left: 27.659574468085104%; + *margin-left: 27.5531914893617%; +} + +.row-fluid .offset3:first-child { + margin-left: 25.53191489361702%; + *margin-left: 25.425531914893618%; +} + +.row-fluid .offset2 { + margin-left: 19.148936170212764%; + *margin-left: 19.04255319148936%; +} + +.row-fluid .offset2:first-child { + margin-left: 17.02127659574468%; + *margin-left: 16.914893617021278%; +} + +.row-fluid .offset1 { + margin-left: 10.638297872340425%; + *margin-left: 10.53191489361702%; +} + +.row-fluid .offset1:first-child { + margin-left: 8.51063829787234%; + *margin-left: 8.404255319148938%; +} + +[class*="span"].hide, +.row-fluid [class*="span"].hide { + display: none; +} + +[class*="span"].pull-right, +.row-fluid [class*="span"].pull-right { + float: right; +} + +.container { + margin-right: auto; + margin-left: auto; + *zoom: 1; +} + +.container:before, +.container:after { + display: table; + line-height: 0; + content: ""; +} + +.container:after { + clear: both; +} + +.container-fluid { + padding-right: 20px; + padding-left: 20px; + *zoom: 1; +} + +.container-fluid:before, +.container-fluid:after { + display: table; + line-height: 0; + content: ""; +} + +.container-fluid:after { + clear: both; +} + +p { + margin: 0 0 10px; +} + +.lead { + margin-bottom: 20px; + font-size: 21px; + font-weight: 200; + line-height: 30px; +} + +small { + font-size: 85%; +} + +strong { + font-weight: bold; +} + +em { + font-style: italic; +} + +cite { + font-style: normal; +} + +.muted { + color: #999999; +} + +a.muted:hover, +a.muted:focus { + color: #808080; +} + +.text-warning { + color: #c09853; +} + +a.text-warning:hover, +a.text-warning:focus { + color: #a47e3c; +} + +.text-error { + color: #b94a48; +} + +a.text-error:hover, +a.text-error:focus { + color: #953b39; +} + +.text-info { + color: #3a87ad; +} + +a.text-info:hover, +a.text-info:focus { + color: #2d6987; +} + +.text-success { + color: #468847; +} + +a.text-success:hover, +a.text-success:focus { + color: #356635; +} + +.text-left { + text-align: left; +} + +.text-right { + text-align: right; +} + +.text-center { + text-align: center; +} + +h1, +h2, +h3, +h4, +h5, +h6 { + margin: 10px 0; + font-family: inherit; + font-weight: bold; + line-height: 20px; + color: inherit; + text-rendering: optimizelegibility; +} + +h1 small, +h2 small, +h3 small, +h4 small, +h5 small, +h6 small { + font-weight: normal; + line-height: 1; + color: #999999; +} + +h1, +h2, +h3 { + line-height: 40px; +} + +h1 { + font-size: 38.5px; +} + +h2 { + font-size: 31.5px; +} + +h3 { + font-size: 24.5px; +} + +h4 { + font-size: 17.5px; +} + +h5 { + font-size: 14px; +} + +h6 { + font-size: 11.9px; +} + +h1 small { + font-size: 24.5px; +} + +h2 small { + font-size: 17.5px; +} + +h3 small { + font-size: 14px; +} + +h4 small { + font-size: 14px; +} + +.page-header { + padding-bottom: 9px; + margin: 20px 0 30px; + border-bottom: 1px solid #eeeeee; +} + +ul, +ol { + padding: 0; + margin: 0 0 10px 25px; +} + +ul ul, +ul ol, +ol ol, +ol ul { + margin-bottom: 0; +} + +li { + line-height: 20px; +} + +ul.unstyled, +ol.unstyled { + margin-left: 0; + list-style: none; +} + +ul.inline, +ol.inline { + margin-left: 0; + list-style: none; +} + +ul.inline > li, +ol.inline > li { + display: inline-block; + *display: inline; + padding-right: 5px; + padding-left: 5px; + *zoom: 1; +} + +dl { + margin-bottom: 20px; +} + +dt, +dd { + line-height: 20px; +} + +dt { + font-weight: bold; +} + +dd { + margin-left: 10px; +} + +.dl-horizontal { + *zoom: 1; +} + +.dl-horizontal:before, +.dl-horizontal:after { + display: table; + line-height: 0; + content: ""; +} + +.dl-horizontal:after { + clear: both; +} + +.dl-horizontal dt { + float: left; + width: 160px; + overflow: hidden; + clear: left; + text-align: right; + text-overflow: ellipsis; + white-space: nowrap; +} + +.dl-horizontal dd { + margin-left: 180px; +} + +hr { + margin: 20px 0; + border: 0; + border-top: 1px solid #eeeeee; + border-bottom: 1px solid #ffffff; +} + +abbr[title], +abbr[data-original-title] { + cursor: help; + border-bottom: 1px dotted #999999; +} + +abbr.initialism { + font-size: 90%; + text-transform: uppercase; +} + +blockquote { + padding: 0 0 0 15px; + margin: 0 0 20px; + border-left: 5px solid #eeeeee; +} + +blockquote p { + margin-bottom: 0; + font-size: 17.5px; + font-weight: 300; + line-height: 1.25; +} + +blockquote small { + display: block; + line-height: 20px; + color: #999999; +} + +blockquote small:before { + content: '\2014 \00A0'; +} + +blockquote.pull-right { + float: right; + padding-right: 15px; + padding-left: 0; + border-right: 5px solid #eeeeee; + border-left: 0; +} + +blockquote.pull-right p, +blockquote.pull-right small { + text-align: right; +} + +blockquote.pull-right small:before { + content: ''; +} + +blockquote.pull-right small:after { + content: '\00A0 \2014'; +} + +q:before, +q:after, +blockquote:before, +blockquote:after { + content: ""; +} + +address { + display: block; + margin-bottom: 20px; + font-style: normal; + line-height: 20px; +} + +code, +pre { + padding: 0 3px 2px; + font-family: Monaco, Menlo, Consolas, "Courier New", monospace; + font-size: 12px; + color: #333333; + -webkit-border-radius: 3px; + -moz-border-radius: 3px; + border-radius: 3px; +} + +code { + padding: 2px 4px; + color: #d14; + white-space: nowrap; + background-color: #f7f7f9; + border: 1px solid #e1e1e8; +} + +pre { + display: block; + padding: 9.5px; + margin: 0 0 10px; + font-size: 13px; + line-height: 20px; + word-break: break-all; + word-wrap: break-word; + white-space: pre; + white-space: pre-wrap; + background-color: #f5f5f5; + border: 1px solid #ccc; + border: 1px solid rgba(0, 0, 0, 0.15); + -webkit-border-radius: 4px; + -moz-border-radius: 4px; + border-radius: 4px; +} + +pre.prettyprint { + margin-bottom: 20px; +} + +pre code { + padding: 0; + color: inherit; + white-space: pre; + white-space: pre-wrap; + background-color: transparent; + border: 0; +} + +.pre-scrollable { + max-height: 340px; + overflow-y: scroll; +} + +form { + margin: 0 0 20px; +} + +fieldset { + padding: 0; + margin: 0; + border: 0; +} + +legend { + display: block; + width: 100%; + padding: 0; + margin-bottom: 20px; + font-size: 21px; + line-height: 40px; + color: #333333; + border: 0; + border-bottom: 1px solid #e5e5e5; +} + +legend small { + font-size: 15px; + color: #999999; +} + +label, +input, +button, +select, +textarea { + font-size: 14px; + font-weight: normal; + line-height: 20px; +} + +input, +button, +select, +textarea { + font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; +} + +label { + display: block; + margin-bottom: 5px; +} + +select, +textarea, +input[type="text"], +input[type="password"], +input[type="datetime"], +input[type="datetime-local"], +input[type="date"], +input[type="month"], +input[type="time"], +input[type="week"], +input[type="number"], +input[type="email"], +input[type="url"], +input[type="search"], +input[type="tel"], +input[type="color"], +.uneditable-input { + display: inline-block; + height: 20px; + padding: 4px 6px; + margin-bottom: 10px; + font-size: 14px; + line-height: 20px; + color: #555555; + vertical-align: middle; + -webkit-border-radius: 4px; + -moz-border-radius: 4px; + border-radius: 4px; +} + +input, +textarea, +.uneditable-input { + width: 206px; +} + +textarea { + height: auto; +} + +textarea, +input[type="text"], +input[type="password"], +input[type="datetime"], +input[type="datetime-local"], +input[type="date"], +input[type="month"], +input[type="time"], +input[type="week"], +input[type="number"], +input[type="email"], +input[type="url"], +input[type="search"], +input[type="tel"], +input[type="color"], +.uneditable-input { + background-color: #ffffff; + border: 1px solid #cccccc; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); + -moz-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); + -webkit-transition: border linear 0.2s, box-shadow linear 0.2s; + -moz-transition: border linear 0.2s, box-shadow linear 0.2s; + -o-transition: border linear 0.2s, box-shadow linear 0.2s; + transition: border linear 0.2s, box-shadow linear 0.2s; +} + +textarea:focus, +input[type="text"]:focus, +input[type="password"]:focus, +input[type="datetime"]:focus, +input[type="datetime-local"]:focus, +input[type="date"]:focus, +input[type="month"]:focus, +input[type="time"]:focus, +input[type="week"]:focus, +input[type="number"]:focus, +input[type="email"]:focus, +input[type="url"]:focus, +input[type="search"]:focus, +input[type="tel"]:focus, +input[type="color"]:focus, +.uneditable-input:focus { + border-color: rgba(82, 168, 236, 0.8); + outline: 0; + outline: thin dotted \9; + /* IE6-9 */ + + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 8px rgba(82, 168, 236, 0.6); + -moz-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 8px rgba(82, 168, 236, 0.6); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 8px rgba(82, 168, 236, 0.6); +} + +input[type="radio"], +input[type="checkbox"] { + margin: 4px 0 0; + margin-top: 1px \9; + *margin-top: 0; + line-height: normal; +} + +input[type="file"], +input[type="image"], +input[type="submit"], +input[type="reset"], +input[type="button"], +input[type="radio"], +input[type="checkbox"] { + width: auto; +} + +select, +input[type="file"] { + height: 30px; + /* In IE7, the height of the select element cannot be changed by height, only font-size */ + + *margin-top: 4px; + /* For IE7, add top margin to align select with labels */ + + line-height: 30px; +} + +select { + width: 220px; + background-color: #ffffff; + border: 1px solid #cccccc; +} + +select[multiple], +select[size] { + height: auto; +} + +select:focus, +input[type="file"]:focus, +input[type="radio"]:focus, +input[type="checkbox"]:focus { + outline: thin dotted #333; + outline: 5px auto -webkit-focus-ring-color; + outline-offset: -2px; +} + +.uneditable-input, +.uneditable-textarea { + color: #999999; + cursor: not-allowed; + background-color: #fcfcfc; + border-color: #cccccc; + -webkit-box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.025); + -moz-box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.025); + box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.025); +} + +.uneditable-input { + overflow: hidden; + white-space: nowrap; +} + +.uneditable-textarea { + width: auto; + height: auto; +} + +input:-moz-placeholder, +textarea:-moz-placeholder { + color: #999999; +} + +input:-ms-input-placeholder, +textarea:-ms-input-placeholder { + color: #999999; +} + +input::-webkit-input-placeholder, +textarea::-webkit-input-placeholder { + color: #999999; +} + +.radio, +.checkbox { + min-height: 20px; + padding-left: 20px; +} + +.radio input[type="radio"], +.checkbox input[type="checkbox"] { + float: left; + margin-left: -20px; +} + +.controls > .radio:first-child, +.controls > .checkbox:first-child { + padding-top: 5px; +} + +.radio.inline, +.checkbox.inline { + display: inline-block; + padding-top: 5px; + margin-bottom: 0; + vertical-align: middle; +} + +.radio.inline + .radio.inline, +.checkbox.inline + .checkbox.inline { + margin-left: 10px; +} + +.input-mini { + width: 60px; +} + +.input-small { + width: 90px; +} + +.input-medium { + width: 150px; +} + +.input-large { + width: 210px; +} + +.input-xlarge { + width: 270px; +} + +.input-xxlarge { + width: 530px; +} + +input[class*="span"], +select[class*="span"], +textarea[class*="span"], +.uneditable-input[class*="span"], +.row-fluid input[class*="span"], +.row-fluid select[class*="span"], +.row-fluid textarea[class*="span"], +.row-fluid .uneditable-input[class*="span"] { + float: none; + margin-left: 0; +} + +.input-append input[class*="span"], +.input-append .uneditable-input[class*="span"], +.input-prepend input[class*="span"], +.input-prepend .uneditable-input[class*="span"], +.row-fluid input[class*="span"], +.row-fluid select[class*="span"], +.row-fluid textarea[class*="span"], +.row-fluid .uneditable-input[class*="span"], +.row-fluid .input-prepend [class*="span"], +.row-fluid .input-append [class*="span"] { + display: inline-block; +} + +input, +textarea, +.uneditable-input { + margin-left: 0; +} + +.controls-row [class*="span"] + [class*="span"] { + margin-left: 20px; +} + +input.span12, +textarea.span12, +.uneditable-input.span12 { + width: 926px; +} + +input.span11, +textarea.span11, +.uneditable-input.span11 { + width: 846px; +} + +input.span10, +textarea.span10, +.uneditable-input.span10 { + width: 766px; +} + +input.span9, +textarea.span9, +.uneditable-input.span9 { + width: 686px; +} + +input.span8, +textarea.span8, +.uneditable-input.span8 { + width: 606px; +} + +input.span7, +textarea.span7, +.uneditable-input.span7 { + width: 526px; +} + +input.span6, +textarea.span6, +.uneditable-input.span6 { + width: 446px; +} + +input.span5, +textarea.span5, +.uneditable-input.span5 { + width: 366px; +} + +input.span4, +textarea.span4, +.uneditable-input.span4 { + width: 286px; +} + +input.span3, +textarea.span3, +.uneditable-input.span3 { + width: 206px; +} + +input.span2, +textarea.span2, +.uneditable-input.span2 { + width: 126px; +} + +input.span1, +textarea.span1, +.uneditable-input.span1 { + width: 46px; +} + +.controls-row { + *zoom: 1; +} + +.controls-row:before, +.controls-row:after { + display: table; + line-height: 0; + content: ""; +} + +.controls-row:after { + clear: both; +} + +.controls-row [class*="span"], +.row-fluid .controls-row [class*="span"] { + float: left; +} + +.controls-row .checkbox[class*="span"], +.controls-row .radio[class*="span"] { + padding-top: 5px; +} + +input[disabled], +select[disabled], +textarea[disabled], +input[readonly], +select[readonly], +textarea[readonly] { + cursor: not-allowed; + background-color: #eeeeee; +} + +input[type="radio"][disabled], +input[type="checkbox"][disabled], +input[type="radio"][readonly], +input[type="checkbox"][readonly] { + background-color: transparent; +} + +.control-group.warning .control-label, +.control-group.warning .help-block, +.control-group.warning .help-inline { + color: #c09853; +} + +.control-group.warning .checkbox, +.control-group.warning .radio, +.control-group.warning input, +.control-group.warning select, +.control-group.warning textarea { + color: #c09853; +} + +.control-group.warning input, +.control-group.warning select, +.control-group.warning textarea { + border-color: #c09853; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); + -moz-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); +} + +.control-group.warning input:focus, +.control-group.warning select:focus, +.control-group.warning textarea:focus { + border-color: #a47e3c; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #dbc59e; + -moz-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #dbc59e; + box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #dbc59e; +} + +.control-group.warning .input-prepend .add-on, +.control-group.warning .input-append .add-on { + color: #c09853; + background-color: #fcf8e3; + border-color: #c09853; +} + +.control-group.error .control-label, +.control-group.error .help-block, +.control-group.error .help-inline { + color: #b94a48; +} + +.control-group.error .checkbox, +.control-group.error .radio, +.control-group.error input, +.control-group.error select, +.control-group.error textarea { + color: #b94a48; +} + +.control-group.error input, +.control-group.error select, +.control-group.error textarea { + border-color: #b94a48; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); + -moz-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); +} + +.control-group.error input:focus, +.control-group.error select:focus, +.control-group.error textarea:focus { + border-color: #953b39; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #d59392; + -moz-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #d59392; + box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #d59392; +} + +.control-group.error .input-prepend .add-on, +.control-group.error .input-append .add-on { + color: #b94a48; + background-color: #f2dede; + border-color: #b94a48; +} + +.control-group.success .control-label, +.control-group.success .help-block, +.control-group.success .help-inline { + color: #468847; +} + +.control-group.success .checkbox, +.control-group.success .radio, +.control-group.success input, +.control-group.success select, +.control-group.success textarea { + color: #468847; +} + +.control-group.success input, +.control-group.success select, +.control-group.success textarea { + border-color: #468847; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); + -moz-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); +} + +.control-group.success input:focus, +.control-group.success select:focus, +.control-group.success textarea:focus { + border-color: #356635; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #7aba7b; + -moz-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #7aba7b; + box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #7aba7b; +} + +.control-group.success .input-prepend .add-on, +.control-group.success .input-append .add-on { + color: #468847; + background-color: #dff0d8; + border-color: #468847; +} + +.control-group.info .control-label, +.control-group.info .help-block, +.control-group.info .help-inline { + color: #3a87ad; +} + +.control-group.info .checkbox, +.control-group.info .radio, +.control-group.info input, +.control-group.info select, +.control-group.info textarea { + color: #3a87ad; +} + +.control-group.info input, +.control-group.info select, +.control-group.info textarea { + border-color: #3a87ad; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); + -moz-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075); +} + +.control-group.info input:focus, +.control-group.info select:focus, +.control-group.info textarea:focus { + border-color: #2d6987; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #7ab5d3; + -moz-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #7ab5d3; + box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.075), 0 0 6px #7ab5d3; +} + +.control-group.info .input-prepend .add-on, +.control-group.info .input-append .add-on { + color: #3a87ad; + background-color: #d9edf7; + border-color: #3a87ad; +} + +input:focus:invalid, +textarea:focus:invalid, +select:focus:invalid { + color: #b94a48; + border-color: #ee5f5b; +} + +input:focus:invalid:focus, +textarea:focus:invalid:focus, +select:focus:invalid:focus { + border-color: #e9322d; + -webkit-box-shadow: 0 0 6px #f8b9b7; + -moz-box-shadow: 0 0 6px #f8b9b7; + box-shadow: 0 0 6px #f8b9b7; +} + +.form-actions { + padding: 19px 20px 20px; + margin-top: 20px; + margin-bottom: 20px; + background-color: #f5f5f5; + border-top: 1px solid #e5e5e5; + *zoom: 1; +} + +.form-actions:before, +.form-actions:after { + display: table; + line-height: 0; + content: ""; +} + +.form-actions:after { + clear: both; +} + +.help-block, +.help-inline { + color: #595959; +} + +.help-block { + display: block; + margin-bottom: 10px; +} + +.help-inline { + display: inline-block; + *display: inline; + padding-left: 5px; + vertical-align: middle; + *zoom: 1; +} + +.input-append, +.input-prepend { + display: inline-block; + margin-bottom: 10px; + font-size: 0; + white-space: nowrap; + vertical-align: middle; +} + +.input-append input, +.input-prepend input, +.input-append select, +.input-prepend select, +.input-append .uneditable-input, +.input-prepend .uneditable-input, +.input-append .dropdown-menu, +.input-prepend .dropdown-menu, +.input-append .popover, +.input-prepend .popover { + font-size: 14px; +} + +.input-append input, +.input-prepend input, +.input-append select, +.input-prepend select, +.input-append .uneditable-input, +.input-prepend .uneditable-input { + position: relative; + margin-bottom: 0; + *margin-left: 0; + vertical-align: top; + -webkit-border-radius: 0 4px 4px 0; + -moz-border-radius: 0 4px 4px 0; + border-radius: 0 4px 4px 0; +} + +.input-append input:focus, +.input-prepend input:focus, +.input-append select:focus, +.input-prepend select:focus, +.input-append .uneditable-input:focus, +.input-prepend .uneditable-input:focus { + z-index: 2; +} + +.input-append .add-on, +.input-prepend .add-on { + display: inline-block; + width: auto; + height: 20px; + min-width: 16px; + padding: 4px 5px; + font-size: 14px; + font-weight: normal; + line-height: 20px; + text-align: center; + text-shadow: 0 1px 0 #ffffff; + background-color: #eeeeee; + border: 1px solid #ccc; +} + +.input-append .add-on, +.input-prepend .add-on, +.input-append .btn, +.input-prepend .btn, +.input-append .btn-group > .dropdown-toggle, +.input-prepend .btn-group > .dropdown-toggle { + vertical-align: top; + -webkit-border-radius: 0; + -moz-border-radius: 0; + border-radius: 0; +} + +.input-append .active, +.input-prepend .active { + background-color: #a9dba9; + border-color: #46a546; +} + +.input-prepend .add-on, +.input-prepend .btn { + margin-right: -1px; +} + +.input-prepend .add-on:first-child, +.input-prepend .btn:first-child { + -webkit-border-radius: 4px 0 0 4px; + -moz-border-radius: 4px 0 0 4px; + border-radius: 4px 0 0 4px; +} + +.input-append input, +.input-append select, +.input-append .uneditable-input { + -webkit-border-radius: 4px 0 0 4px; + -moz-border-radius: 4px 0 0 4px; + border-radius: 4px 0 0 4px; +} + +.input-append input + .btn-group .btn:last-child, +.input-append select + .btn-group .btn:last-child, +.input-append .uneditable-input + .btn-group .btn:last-child { + -webkit-border-radius: 0 4px 4px 0; + -moz-border-radius: 0 4px 4px 0; + border-radius: 0 4px 4px 0; +} + +.input-append .add-on, +.input-append .btn, +.input-append .btn-group { + margin-left: -1px; +} + +.input-append .add-on:last-child, +.input-append .btn:last-child, +.input-append .btn-group:last-child > .dropdown-toggle { + -webkit-border-radius: 0 4px 4px 0; + -moz-border-radius: 0 4px 4px 0; + border-radius: 0 4px 4px 0; +} + +.input-prepend.input-append input, +.input-prepend.input-append select, +.input-prepend.input-append .uneditable-input { + -webkit-border-radius: 0; + -moz-border-radius: 0; + border-radius: 0; +} + +.input-prepend.input-append input + .btn-group .btn, +.input-prepend.input-append select + .btn-group .btn, +.input-prepend.input-append .uneditable-input + .btn-group .btn { + -webkit-border-radius: 0 4px 4px 0; + -moz-border-radius: 0 4px 4px 0; + border-radius: 0 4px 4px 0; +} + +.input-prepend.input-append .add-on:first-child, +.input-prepend.input-append .btn:first-child { + margin-right: -1px; + -webkit-border-radius: 4px 0 0 4px; + -moz-border-radius: 4px 0 0 4px; + border-radius: 4px 0 0 4px; +} + +.input-prepend.input-append .add-on:last-child, +.input-prepend.input-append .btn:last-child { + margin-left: -1px; + -webkit-border-radius: 0 4px 4px 0; + -moz-border-radius: 0 4px 4px 0; + border-radius: 0 4px 4px 0; +} + +.input-prepend.input-append .btn-group:first-child { + margin-left: 0; +} + +input.search-query { + padding-right: 14px; + padding-right: 4px \9; + padding-left: 14px; + padding-left: 4px \9; + /* IE7-8 doesn't have border-radius, so don't indent the padding */ + + margin-bottom: 0; + -webkit-border-radius: 15px; + -moz-border-radius: 15px; + border-radius: 15px; +} + +/* Allow for input prepend/append in search forms */ + +.form-search .input-append .search-query, +.form-search .input-prepend .search-query { + -webkit-border-radius: 0; + -moz-border-radius: 0; + border-radius: 0; +} + +.form-search .input-append .search-query { + -webkit-border-radius: 14px 0 0 14px; + -moz-border-radius: 14px 0 0 14px; + border-radius: 14px 0 0 14px; +} + +.form-search .input-append .btn { + -webkit-border-radius: 0 14px 14px 0; + -moz-border-radius: 0 14px 14px 0; + border-radius: 0 14px 14px 0; +} + +.form-search .input-prepend .search-query { + -webkit-border-radius: 0 14px 14px 0; + -moz-border-radius: 0 14px 14px 0; + border-radius: 0 14px 14px 0; +} + +.form-search .input-prepend .btn { + -webkit-border-radius: 14px 0 0 14px; + -moz-border-radius: 14px 0 0 14px; + border-radius: 14px 0 0 14px; +} + +.form-search input, +.form-inline input, +.form-horizontal input, +.form-search textarea, +.form-inline textarea, +.form-horizontal textarea, +.form-search select, +.form-inline select, +.form-horizontal select, +.form-search .help-inline, +.form-inline .help-inline, +.form-horizontal .help-inline, +.form-search .uneditable-input, +.form-inline .uneditable-input, +.form-horizontal .uneditable-input, +.form-search .input-prepend, +.form-inline .input-prepend, +.form-horizontal .input-prepend, +.form-search .input-append, +.form-inline .input-append, +.form-horizontal .input-append { + display: inline-block; + *display: inline; + margin-bottom: 0; + vertical-align: middle; + *zoom: 1; +} + +.form-search .hide, +.form-inline .hide, +.form-horizontal .hide { + display: none; +} + +.form-search label, +.form-inline label, +.form-search .btn-group, +.form-inline .btn-group { + display: inline-block; +} + +.form-search .input-append, +.form-inline .input-append, +.form-search .input-prepend, +.form-inline .input-prepend { + margin-bottom: 0; +} + +.form-search .radio, +.form-search .checkbox, +.form-inline .radio, +.form-inline .checkbox { + padding-left: 0; + margin-bottom: 0; + vertical-align: middle; +} + +.form-search .radio input[type="radio"], +.form-search .checkbox input[type="checkbox"], +.form-inline .radio input[type="radio"], +.form-inline .checkbox input[type="checkbox"] { + float: left; + margin-right: 3px; + margin-left: 0; +} + +.control-group { + margin-bottom: 10px; +} + +legend + .control-group { + margin-top: 20px; + -webkit-margin-top-collapse: separate; +} + +.form-horizontal .control-group { + margin-bottom: 20px; + *zoom: 1; +} + +.form-horizontal .control-group:before, +.form-horizontal .control-group:after { + display: table; + line-height: 0; + content: ""; +} + +.form-horizontal .control-group:after { + clear: both; +} + +.form-horizontal .control-label { + float: left; + width: 160px; + padding-top: 5px; + text-align: right; +} + +.form-horizontal .controls { + *display: inline-block; + *padding-left: 20px; + margin-left: 180px; + *margin-left: 0; +} + +.form-horizontal .controls:first-child { + *padding-left: 180px; +} + +.form-horizontal .help-block { + margin-bottom: 0; +} + +.form-horizontal input + .help-block, +.form-horizontal select + .help-block, +.form-horizontal textarea + .help-block, +.form-horizontal .uneditable-input + .help-block, +.form-horizontal .input-prepend + .help-block, +.form-horizontal .input-append + .help-block { + margin-top: 10px; +} + +.form-horizontal .form-actions { + padding-left: 180px; +} + +table { + max-width: 100%; + background-color: transparent; + border-collapse: collapse; + border-spacing: 0; +} + +.table { + width: 100%; + margin-bottom: 20px; +} + +.table th, +.table td { + padding: 8px; + line-height: 20px; + text-align: left; + vertical-align: top; + border-top: 1px solid #dddddd; +} + +.table th { + font-weight: bold; +} + +.table thead th { + vertical-align: bottom; +} + +.table caption + thead tr:first-child th, +.table caption + thead tr:first-child td, +.table colgroup + thead tr:first-child th, +.table colgroup + thead tr:first-child td, +.table thead:first-child tr:first-child th, +.table thead:first-child tr:first-child td { + border-top: 0; +} + +.table tbody + tbody { + border-top: 2px solid #dddddd; +} + +.table .table { + background-color: #ffffff; +} + +.table-condensed th, +.table-condensed td { + padding: 4px 5px; +} + +.table-bordered { + border: 1px solid #dddddd; + border-collapse: separate; + *border-collapse: collapse; + border-left: 0; + -webkit-border-radius: 4px; + -moz-border-radius: 4px; + border-radius: 4px; +} + +.table-bordered th, +.table-bordered td { + border-left: 1px solid #dddddd; +} + +.table-bordered caption + thead tr:first-child th, +.table-bordered caption + tbody tr:first-child th, +.table-bordered caption + tbody tr:first-child td, +.table-bordered colgroup + thead tr:first-child th, +.table-bordered colgroup + tbody tr:first-child th, +.table-bordered colgroup + tbody tr:first-child td, +.table-bordered thead:first-child tr:first-child th, +.table-bordered tbody:first-child tr:first-child th, +.table-bordered tbody:first-child tr:first-child td { + border-top: 0; +} + +.table-bordered thead:first-child tr:first-child > th:first-child, +.table-bordered tbody:first-child tr:first-child > td:first-child, +.table-bordered tbody:first-child tr:first-child > th:first-child { + -webkit-border-top-left-radius: 4px; + border-top-left-radius: 4px; + -moz-border-radius-topleft: 4px; +} + +.table-bordered thead:first-child tr:first-child > th:last-child, +.table-bordered tbody:first-child tr:first-child > td:last-child, +.table-bordered tbody:first-child tr:first-child > th:last-child { + -webkit-border-top-right-radius: 4px; + border-top-right-radius: 4px; + -moz-border-radius-topright: 4px; +} + +.table-bordered thead:last-child tr:last-child > th:first-child, +.table-bordered tbody:last-child tr:last-child > td:first-child, +.table-bordered tbody:last-child tr:last-child > th:first-child, +.table-bordered tfoot:last-child tr:last-child > td:first-child, +.table-bordered tfoot:last-child tr:last-child > th:first-child { + -webkit-border-bottom-left-radius: 4px; + border-bottom-left-radius: 4px; + -moz-border-radius-bottomleft: 4px; +} + +.table-bordered thead:last-child tr:last-child > th:last-child, +.table-bordered tbody:last-child tr:last-child > td:last-child, +.table-bordered tbody:last-child tr:last-child > th:last-child, +.table-bordered tfoot:last-child tr:last-child > td:last-child, +.table-bordered tfoot:last-child tr:last-child > th:last-child { + -webkit-border-bottom-right-radius: 4px; + border-bottom-right-radius: 4px; + -moz-border-radius-bottomright: 4px; +} + +.table-bordered tfoot + tbody:last-child tr:last-child td:first-child { + -webkit-border-bottom-left-radius: 0; + border-bottom-left-radius: 0; + -moz-border-radius-bottomleft: 0; +} + +.table-bordered tfoot + tbody:last-child tr:last-child td:last-child { + -webkit-border-bottom-right-radius: 0; + border-bottom-right-radius: 0; + -moz-border-radius-bottomright: 0; +} + +.table-bordered caption + thead tr:first-child th:first-child, +.table-bordered caption + tbody tr:first-child td:first-child, +.table-bordered colgroup + thead tr:first-child th:first-child, +.table-bordered colgroup + tbody tr:first-child td:first-child { + -webkit-border-top-left-radius: 4px; + border-top-left-radius: 4px; + -moz-border-radius-topleft: 4px; +} + +.table-bordered caption + thead tr:first-child th:last-child, +.table-bordered caption + tbody tr:first-child td:last-child, +.table-bordered colgroup + thead tr:first-child th:last-child, +.table-bordered colgroup + tbody tr:first-child td:last-child { + -webkit-border-top-right-radius: 4px; + border-top-right-radius: 4px; + -moz-border-radius-topright: 4px; +} + +.table-striped tbody > tr:nth-child(odd) > td, +.table-striped tbody > tr:nth-child(odd) > th { + background-color: #f9f9f9; +} + +.table-hover tbody tr:hover > td, +.table-hover tbody tr:hover > th { + background-color: #f5f5f5; +} + +table td[class*="span"], +table th[class*="span"], +.row-fluid table td[class*="span"], +.row-fluid table th[class*="span"] { + display: table-cell; + float: none; + margin-left: 0; +} + +.table td.span1, +.table th.span1 { + float: none; + width: 44px; + margin-left: 0; +} + +.table td.span2, +.table th.span2 { + float: none; + width: 124px; + margin-left: 0; +} + +.table td.span3, +.table th.span3 { + float: none; + width: 204px; + margin-left: 0; +} + +.table td.span4, +.table th.span4 { + float: none; + width: 284px; + margin-left: 0; +} + +.table td.span5, +.table th.span5 { + float: none; + width: 364px; + margin-left: 0; +} + +.table td.span6, +.table th.span6 { + float: none; + width: 444px; + margin-left: 0; +} + +.table td.span7, +.table th.span7 { + float: none; + width: 524px; + margin-left: 0; +} + +.table td.span8, +.table th.span8 { + float: none; + width: 604px; + margin-left: 0; +} + +.table td.span9, +.table th.span9 { + float: none; + width: 684px; + margin-left: 0; +} + +.table td.span10, +.table th.span10 { + float: none; + width: 764px; + margin-left: 0; +} + +.table td.span11, +.table th.span11 { + float: none; + width: 844px; + margin-left: 0; +} + +.table td.span12, +.table th.span12 { + float: none; + width: 924px; + margin-left: 0; +} + +.table tbody tr.success > td { + background-color: #dff0d8; +} + +.table tbody tr.error > td { + background-color: #f2dede; +} + +.table tbody tr.warning > td { + background-color: #fcf8e3; +} + +.table tbody tr.info > td { + background-color: #d9edf7; +} + +.table-hover tbody tr.success:hover > td { + background-color: #d0e9c6; +} + +.table-hover tbody tr.error:hover > td { + background-color: #ebcccc; +} + +.table-hover tbody tr.warning:hover > td { + background-color: #faf2cc; +} + +.table-hover tbody tr.info:hover > td { + background-color: #c4e3f3; +} + +[class^="icon-"], +[class*=" icon-"] { + display: inline-block; + width: 14px; + height: 14px; + margin-top: 1px; + *margin-right: .3em; + line-height: 14px; + vertical-align: text-top; + background-image: url("../img/glyphicons-halflings.png"); + background-position: 14px 14px; + background-repeat: no-repeat; +} + +/* White icons with optional class, or on hover/focus/active states of certain elements */ + +.icon-white, +.nav-pills > .active > a > [class^="icon-"], +.nav-pills > .active > a > [class*=" icon-"], +.nav-list > .active > a > [class^="icon-"], +.nav-list > .active > a > [class*=" icon-"], +.navbar-inverse .nav > .active > a > [class^="icon-"], +.navbar-inverse .nav > .active > a > [class*=" icon-"], +.dropdown-menu > li > a:hover > [class^="icon-"], +.dropdown-menu > li > a:focus > [class^="icon-"], +.dropdown-menu > li > a:hover > [class*=" icon-"], +.dropdown-menu > li > a:focus > [class*=" icon-"], +.dropdown-menu > .active > a > [class^="icon-"], +.dropdown-menu > .active > a > [class*=" icon-"], +.dropdown-submenu:hover > a > [class^="icon-"], +.dropdown-submenu:focus > a > [class^="icon-"], +.dropdown-submenu:hover > a > [class*=" icon-"], +.dropdown-submenu:focus > a > [class*=" icon-"] { + background-image: url("../img/glyphicons-halflings-white.png"); +} + +.icon-glass { + background-position: 0 0; +} + +.icon-music { + background-position: -24px 0; +} + +.icon-search { + background-position: -48px 0; +} + +.icon-envelope { + background-position: -72px 0; +} + +.icon-heart { + background-position: -96px 0; +} + +.icon-star { + background-position: -120px 0; +} + +.icon-star-empty { + background-position: -144px 0; +} + +.icon-user { + background-position: -168px 0; +} + +.icon-film { + background-position: -192px 0; +} + +.icon-th-large { + background-position: -216px 0; +} + +.icon-th { + background-position: -240px 0; +} + +.icon-th-list { + background-position: -264px 0; +} + +.icon-ok { + background-position: -288px 0; +} + +.icon-remove { + background-position: -312px 0; +} + +.icon-zoom-in { + background-position: -336px 0; +} + +.icon-zoom-out { + background-position: -360px 0; +} + +.icon-off { + background-position: -384px 0; +} + +.icon-signal { + background-position: -408px 0; +} + +.icon-cog { + background-position: -432px 0; +} + +.icon-trash { + background-position: -456px 0; +} + +.icon-home { + background-position: 0 -24px; +} + +.icon-file { + background-position: -24px -24px; +} + +.icon-time { + background-position: -48px -24px; +} + +.icon-road { + background-position: -72px -24px; +} + +.icon-download-alt { + background-position: -96px -24px; +} + +.icon-download { + background-position: -120px -24px; +} + +.icon-upload { + background-position: -144px -24px; +} + +.icon-inbox { + background-position: -168px -24px; +} + +.icon-play-circle { + background-position: -192px -24px; +} + +.icon-repeat { + background-position: -216px -24px; +} + +.icon-refresh { + background-position: -240px -24px; +} + +.icon-list-alt { + background-position: -264px -24px; +} + +.icon-lock { + background-position: -287px -24px; +} + +.icon-flag { + background-position: -312px -24px; +} + +.icon-headphones { + background-position: -336px -24px; +} + +.icon-volume-off { + background-position: -360px -24px; +} + +.icon-volume-down { + background-position: -384px -24px; +} + +.icon-volume-up { + background-position: -408px -24px; +} + +.icon-qrcode { + background-position: -432px -24px; +} + +.icon-barcode { + background-position: -456px -24px; +} + +.icon-tag { + background-position: 0 -48px; +} + +.icon-tags { + background-position: -25px -48px; +} + +.icon-book { + background-position: -48px -48px; +} + +.icon-bookmark { + background-position: -72px -48px; +} + +.icon-print { + background-position: -96px -48px; +} + +.icon-camera { + background-position: -120px -48px; +} + +.icon-font { + background-position: -144px -48px; +} + +.icon-bold { + background-position: -167px -48px; +} + +.icon-italic { + background-position: -192px -48px; +} + +.icon-text-height { + background-position: -216px -48px; +} + +.icon-text-width { + background-position: -240px -48px; +} + +.icon-align-left { + background-position: -264px -48px; +} + +.icon-align-center { + background-position: -288px -48px; +} + +.icon-align-right { + background-position: -312px -48px; +} + +.icon-align-justify { + background-position: -336px -48px; +} + +.icon-list { + background-position: -360px -48px; +} + +.icon-indent-left { + background-position: -384px -48px; +} + +.icon-indent-right { + background-position: -408px -48px; +} + +.icon-facetime-video { + background-position: -432px -48px; +} + +.icon-picture { + background-position: -456px -48px; +} + +.icon-pencil { + background-position: 0 -72px; +} + +.icon-map-marker { + background-position: -24px -72px; +} + +.icon-adjust { + background-position: -48px -72px; +} + +.icon-tint { + background-position: -72px -72px; +} + +.icon-edit { + background-position: -96px -72px; +} + +.icon-share { + background-position: -120px -72px; +} + +.icon-check { + background-position: -144px -72px; +} + +.icon-move { + background-position: -168px -72px; +} + +.icon-step-backward { + background-position: -192px -72px; +} + +.icon-fast-backward { + background-position: -216px -72px; +} + +.icon-backward { + background-position: -240px -72px; +} + +.icon-play { + background-position: -264px -72px; +} + +.icon-pause { + background-position: -288px -72px; +} + +.icon-stop { + background-position: -312px -72px; +} + +.icon-forward { + background-position: -336px -72px; +} + +.icon-fast-forward { + background-position: -360px -72px; +} + +.icon-step-forward { + background-position: -384px -72px; +} + +.icon-eject { + background-position: -408px -72px; +} + +.icon-chevron-left { + background-position: -432px -72px; +} + +.icon-chevron-right { + background-position: -456px -72px; +} + +.icon-plus-sign { + background-position: 0 -96px; +} + +.icon-minus-sign { + background-position: -24px -96px; +} + +.icon-remove-sign { + background-position: -48px -96px; +} + +.icon-ok-sign { + background-position: -72px -96px; +} + +.icon-question-sign { + background-position: -96px -96px; +} + +.icon-info-sign { + background-position: -120px -96px; +} + +.icon-screenshot { + background-position: -144px -96px; +} + +.icon-remove-circle { + background-position: -168px -96px; +} + +.icon-ok-circle { + background-position: -192px -96px; +} + +.icon-ban-circle { + background-position: -216px -96px; +} + +.icon-arrow-left { + background-position: -240px -96px; +} + +.icon-arrow-right { + background-position: -264px -96px; +} + +.icon-arrow-up { + background-position: -289px -96px; +} + +.icon-arrow-down { + background-position: -312px -96px; +} + +.icon-share-alt { + background-position: -336px -96px; +} + +.icon-resize-full { + background-position: -360px -96px; +} + +.icon-resize-small { + background-position: -384px -96px; +} + +.icon-plus { + background-position: -408px -96px; +} + +.icon-minus { + background-position: -433px -96px; +} + +.icon-asterisk { + background-position: -456px -96px; +} + +.icon-exclamation-sign { + background-position: 0 -120px; +} + +.icon-gift { + background-position: -24px -120px; +} + +.icon-leaf { + background-position: -48px -120px; +} + +.icon-fire { + background-position: -72px -120px; +} + +.icon-eye-open { + background-position: -96px -120px; +} + +.icon-eye-close { + background-position: -120px -120px; +} + +.icon-warning-sign { + background-position: -144px -120px; +} + +.icon-plane { + background-position: -168px -120px; +} + +.icon-calendar { + background-position: -192px -120px; +} + +.icon-random { + width: 16px; + background-position: -216px -120px; +} + +.icon-comment { + background-position: -240px -120px; +} + +.icon-magnet { + background-position: -264px -120px; +} + +.icon-chevron-up { + background-position: -288px -120px; +} + +.icon-chevron-down { + background-position: -313px -119px; +} + +.icon-retweet { + background-position: -336px -120px; +} + +.icon-shopping-cart { + background-position: -360px -120px; +} + +.icon-folder-close { + width: 16px; + background-position: -384px -120px; +} + +.icon-folder-open { + width: 16px; + background-position: -408px -120px; +} + +.icon-resize-vertical { + background-position: -432px -119px; +} + +.icon-resize-horizontal { + background-position: -456px -118px; +} + +.icon-hdd { + background-position: 0 -144px; +} + +.icon-bullhorn { + background-position: -24px -144px; +} + +.icon-bell { + background-position: -48px -144px; +} + +.icon-certificate { + background-position: -72px -144px; +} + +.icon-thumbs-up { + background-position: -96px -144px; +} + +.icon-thumbs-down { + background-position: -120px -144px; +} + +.icon-hand-right { + background-position: -144px -144px; +} + +.icon-hand-left { + background-position: -168px -144px; +} + +.icon-hand-up { + background-position: -192px -144px; +} + +.icon-hand-down { + background-position: -216px -144px; +} + +.icon-circle-arrow-right { + background-position: -240px -144px; +} + +.icon-circle-arrow-left { + background-position: -264px -144px; +} + +.icon-circle-arrow-up { + background-position: -288px -144px; +} + +.icon-circle-arrow-down { + background-position: -312px -144px; +} + +.icon-globe { + background-position: -336px -144px; +} + +.icon-wrench { + background-position: -360px -144px; +} + +.icon-tasks { + background-position: -384px -144px; +} + +.icon-filter { + background-position: -408px -144px; +} + +.icon-briefcase { + background-position: -432px -144px; +} + +.icon-fullscreen { + background-position: -456px -144px; +} + +.dropup, +.dropdown { + position: relative; +} + +.dropdown-toggle { + *margin-bottom: -3px; +} + +.dropdown-toggle:active, +.open .dropdown-toggle { + outline: 0; +} + +.caret { + display: inline-block; + width: 0; + height: 0; + vertical-align: top; + border-top: 4px solid #000000; + border-right: 4px solid transparent; + border-left: 4px solid transparent; + content: ""; +} + +.dropdown .caret { + margin-top: 8px; + margin-left: 2px; +} + +.dropdown-menu { + position: absolute; + top: 100%; + left: 0; + z-index: 1000; + display: none; + float: left; + min-width: 160px; + padding: 5px 0; + margin: 2px 0 0; + list-style: none; + background-color: #ffffff; + border: 1px solid #ccc; + border: 1px solid rgba(0, 0, 0, 0.2); + *border-right-width: 2px; + *border-bottom-width: 2px; + -webkit-border-radius: 6px; + -moz-border-radius: 6px; + border-radius: 6px; + -webkit-box-shadow: 0 5px 10px rgba(0, 0, 0, 0.2); + -moz-box-shadow: 0 5px 10px rgba(0, 0, 0, 0.2); + box-shadow: 0 5px 10px rgba(0, 0, 0, 0.2); + -webkit-background-clip: padding-box; + -moz-background-clip: padding; + background-clip: padding-box; +} + +.dropdown-menu.pull-right { + right: 0; + left: auto; +} + +.dropdown-menu .divider { + *width: 100%; + height: 1px; + margin: 9px 1px; + *margin: -5px 0 5px; + overflow: hidden; + background-color: #e5e5e5; + border-bottom: 1px solid #ffffff; +} + +.dropdown-menu > li > a { + display: block; + padding: 3px 20px; + clear: both; + font-weight: normal; + line-height: 20px; + color: #333333; + white-space: nowrap; +} + +.dropdown-menu > li > a:hover, +.dropdown-menu > li > a:focus, +.dropdown-submenu:hover > a, +.dropdown-submenu:focus > a { + color: #ffffff; + text-decoration: none; + background-color: #0081c2; + background-image: -moz-linear-gradient(top, #0088cc, #0077b3); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#0088cc), to(#0077b3)); + background-image: -webkit-linear-gradient(top, #0088cc, #0077b3); + background-image: -o-linear-gradient(top, #0088cc, #0077b3); + background-image: linear-gradient(to bottom, #0088cc, #0077b3); + background-repeat: repeat-x; + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc', endColorstr='#ff0077b3', GradientType=0); +} + +.dropdown-menu > .active > a, +.dropdown-menu > .active > a:hover, +.dropdown-menu > .active > a:focus { + color: #ffffff; + text-decoration: none; + background-color: #0081c2; + background-image: -moz-linear-gradient(top, #0088cc, #0077b3); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#0088cc), to(#0077b3)); + background-image: -webkit-linear-gradient(top, #0088cc, #0077b3); + background-image: -o-linear-gradient(top, #0088cc, #0077b3); + background-image: linear-gradient(to bottom, #0088cc, #0077b3); + background-repeat: repeat-x; + outline: 0; + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc', endColorstr='#ff0077b3', GradientType=0); +} + +.dropdown-menu > .disabled > a, +.dropdown-menu > .disabled > a:hover, +.dropdown-menu > .disabled > a:focus { + color: #999999; +} + +.dropdown-menu > .disabled > a:hover, +.dropdown-menu > .disabled > a:focus { + text-decoration: none; + cursor: default; + background-color: transparent; + background-image: none; + filter: progid:DXImageTransform.Microsoft.gradient(enabled=false); +} + +.open { + *z-index: 1000; +} + +.open > .dropdown-menu { + display: block; +} + +.dropdown-backdrop { + position: fixed; + top: 0; + right: 0; + bottom: 0; + left: 0; + z-index: 990; +} + +.pull-right > .dropdown-menu { + right: 0; + left: auto; +} + +.dropup .caret, +.navbar-fixed-bottom .dropdown .caret { + border-top: 0; + border-bottom: 4px solid #000000; + content: ""; +} + +.dropup .dropdown-menu, +.navbar-fixed-bottom .dropdown .dropdown-menu { + top: auto; + bottom: 100%; + margin-bottom: 1px; +} + +.dropdown-submenu { + position: relative; +} + +.dropdown-submenu > .dropdown-menu { + top: 0; + left: 100%; + margin-top: -6px; + margin-left: -1px; + -webkit-border-radius: 0 6px 6px 6px; + -moz-border-radius: 0 6px 6px 6px; + border-radius: 0 6px 6px 6px; +} + +.dropdown-submenu:hover > .dropdown-menu { + display: block; +} + +.dropup .dropdown-submenu > .dropdown-menu { + top: auto; + bottom: 0; + margin-top: 0; + margin-bottom: -2px; + -webkit-border-radius: 5px 5px 5px 0; + -moz-border-radius: 5px 5px 5px 0; + border-radius: 5px 5px 5px 0; +} + +.dropdown-submenu > a:after { + display: block; + float: right; + width: 0; + height: 0; + margin-top: 5px; + margin-right: -10px; + border-color: transparent; + border-left-color: #cccccc; + border-style: solid; + border-width: 5px 0 5px 5px; + content: " "; +} + +.dropdown-submenu:hover > a:after { + border-left-color: #ffffff; +} + +.dropdown-submenu.pull-left { + float: none; +} + +.dropdown-submenu.pull-left > .dropdown-menu { + left: -100%; + margin-left: 10px; + -webkit-border-radius: 6px 0 6px 6px; + -moz-border-radius: 6px 0 6px 6px; + border-radius: 6px 0 6px 6px; +} + +.dropdown .dropdown-menu .nav-header { + padding-right: 20px; + padding-left: 20px; +} + +.typeahead { + z-index: 1051; + margin-top: 2px; + -webkit-border-radius: 4px; + -moz-border-radius: 4px; + border-radius: 4px; +} + +.well { + min-height: 20px; + padding: 19px; + margin-bottom: 20px; + background-color: #f5f5f5; + border: 1px solid #e3e3e3; + -webkit-border-radius: 4px; + -moz-border-radius: 4px; + border-radius: 4px; + -webkit-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.05); + -moz-box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.05); + box-shadow: inset 0 1px 1px rgba(0, 0, 0, 0.05); +} + +.well blockquote { + border-color: #ddd; + border-color: rgba(0, 0, 0, 0.15); +} + +.well-large { + padding: 24px; + -webkit-border-radius: 6px; + -moz-border-radius: 6px; + border-radius: 6px; +} + +.well-small { + padding: 9px; + -webkit-border-radius: 3px; + -moz-border-radius: 3px; + border-radius: 3px; +} + +.fade { + opacity: 0; + -webkit-transition: opacity 0.15s linear; + -moz-transition: opacity 0.15s linear; + -o-transition: opacity 0.15s linear; + transition: opacity 0.15s linear; +} + +.fade.in { + opacity: 1; +} + +.collapse { + position: relative; + height: 0; + overflow: hidden; + -webkit-transition: height 0.35s ease; + -moz-transition: height 0.35s ease; + -o-transition: height 0.35s ease; + transition: height 0.35s ease; +} + +.collapse.in { + height: auto; +} + +.close { + float: right; + font-size: 20px; + font-weight: bold; + line-height: 20px; + color: #000000; + text-shadow: 0 1px 0 #ffffff; + opacity: 0.2; + filter: alpha(opacity=20); +} + +.close:hover, +.close:focus { + color: #000000; + text-decoration: none; + cursor: pointer; + opacity: 0.4; + filter: alpha(opacity=40); +} + +button.close { + padding: 0; + cursor: pointer; + background: transparent; + border: 0; + -webkit-appearance: none; +} + +.btn { + display: inline-block; + *display: inline; + padding: 4px 12px; + margin-bottom: 0; + *margin-left: .3em; + font-size: 14px; + line-height: 20px; + color: #333333; + text-align: center; + text-shadow: 0 1px 1px rgba(255, 255, 255, 0.75); + vertical-align: middle; + cursor: pointer; + background-color: #f5f5f5; + *background-color: #e6e6e6; + background-image: -moz-linear-gradient(top, #ffffff, #e6e6e6); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#ffffff), to(#e6e6e6)); + background-image: -webkit-linear-gradient(top, #ffffff, #e6e6e6); + background-image: -o-linear-gradient(top, #ffffff, #e6e6e6); + background-image: linear-gradient(to bottom, #ffffff, #e6e6e6); + background-repeat: repeat-x; + border: 1px solid #cccccc; + *border: 0; + border-color: #e6e6e6 #e6e6e6 #bfbfbf; + border-color: rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25); + border-bottom-color: #b3b3b3; + -webkit-border-radius: 4px; + -moz-border-radius: 4px; + border-radius: 4px; + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff', endColorstr='#ffe6e6e6', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled=false); + *zoom: 1; + -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.2), 0 1px 2px rgba(0, 0, 0, 0.05); + -moz-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.2), 0 1px 2px rgba(0, 0, 0, 0.05); + box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.2), 0 1px 2px rgba(0, 0, 0, 0.05); +} + +.btn:hover, +.btn:focus, +.btn:active, +.btn.active, +.btn.disabled, +.btn[disabled] { + color: #333333; + background-color: #e6e6e6; + *background-color: #d9d9d9; +} + +.btn:active, +.btn.active { + background-color: #cccccc \9; +} + +.btn:first-child { + *margin-left: 0; +} + +.btn:hover, +.btn:focus { + color: #333333; + text-decoration: none; + background-position: 0 -15px; + -webkit-transition: background-position 0.1s linear; + -moz-transition: background-position 0.1s linear; + -o-transition: background-position 0.1s linear; + transition: background-position 0.1s linear; +} + +.btn:focus { + outline: thin dotted #333; + outline: 5px auto -webkit-focus-ring-color; + outline-offset: -2px; +} + +.btn.active, +.btn:active { + background-image: none; + outline: 0; + -webkit-box-shadow: inset 0 2px 4px rgba(0, 0, 0, 0.15), 0 1px 2px rgba(0, 0, 0, 0.05); + -moz-box-shadow: inset 0 2px 4px rgba(0, 0, 0, 0.15), 0 1px 2px rgba(0, 0, 0, 0.05); + box-shadow: inset 0 2px 4px rgba(0, 0, 0, 0.15), 0 1px 2px rgba(0, 0, 0, 0.05); +} + +.btn.disabled, +.btn[disabled] { + cursor: default; + background-image: none; + opacity: 0.65; + filter: alpha(opacity=65); + -webkit-box-shadow: none; + -moz-box-shadow: none; + box-shadow: none; +} + +.btn-large { + padding: 11px 19px; + font-size: 17.5px; + -webkit-border-radius: 6px; + -moz-border-radius: 6px; + border-radius: 6px; +} + +.btn-large [class^="icon-"], +.btn-large [class*=" icon-"] { + margin-top: 4px; +} + +.btn-small { + padding: 2px 10px; + font-size: 11.9px; + -webkit-border-radius: 3px; + -moz-border-radius: 3px; + border-radius: 3px; +} + +.btn-small [class^="icon-"], +.btn-small [class*=" icon-"] { + margin-top: 0; +} + +.btn-mini [class^="icon-"], +.btn-mini [class*=" icon-"] { + margin-top: -1px; +} + +.btn-mini { + padding: 0 6px; + font-size: 10.5px; + -webkit-border-radius: 3px; + -moz-border-radius: 3px; + border-radius: 3px; +} + +.btn-block { + display: block; + width: 100%; + padding-right: 0; + padding-left: 0; + -webkit-box-sizing: border-box; + -moz-box-sizing: border-box; + box-sizing: border-box; +} + +.btn-block + .btn-block { + margin-top: 5px; +} + +input[type="submit"].btn-block, +input[type="reset"].btn-block, +input[type="button"].btn-block { + width: 100%; +} + +.btn-primary.active, +.btn-warning.active, +.btn-danger.active, +.btn-success.active, +.btn-info.active, +.btn-inverse.active { + color: rgba(255, 255, 255, 0.75); +} + +.btn-primary { + color: #ffffff; + text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25); + background-color: #006dcc; + *background-color: #0044cc; + background-image: -moz-linear-gradient(top, #0088cc, #0044cc); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#0088cc), to(#0044cc)); + background-image: -webkit-linear-gradient(top, #0088cc, #0044cc); + background-image: -o-linear-gradient(top, #0088cc, #0044cc); + background-image: linear-gradient(to bottom, #0088cc, #0044cc); + background-repeat: repeat-x; + border-color: #0044cc #0044cc #002a80; + border-color: rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc', endColorstr='#ff0044cc', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled=false); +} + +.btn-primary:hover, +.btn-primary:focus, +.btn-primary:active, +.btn-primary.active, +.btn-primary.disabled, +.btn-primary[disabled] { + color: #ffffff; + background-color: #0044cc; + *background-color: #003bb3; +} + +.btn-primary:active, +.btn-primary.active { + background-color: #003399 \9; +} + +.btn-warning { + color: #ffffff; + text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25); + background-color: #faa732; + *background-color: #f89406; + background-image: -moz-linear-gradient(top, #fbb450, #f89406); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#fbb450), to(#f89406)); + background-image: -webkit-linear-gradient(top, #fbb450, #f89406); + background-image: -o-linear-gradient(top, #fbb450, #f89406); + background-image: linear-gradient(to bottom, #fbb450, #f89406); + background-repeat: repeat-x; + border-color: #f89406 #f89406 #ad6704; + border-color: rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffbb450', endColorstr='#fff89406', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled=false); +} + +.btn-warning:hover, +.btn-warning:focus, +.btn-warning:active, +.btn-warning.active, +.btn-warning.disabled, +.btn-warning[disabled] { + color: #ffffff; + background-color: #f89406; + *background-color: #df8505; +} + +.btn-warning:active, +.btn-warning.active { + background-color: #c67605 \9; +} + +.btn-danger { + color: #ffffff; + text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25); + background-color: #da4f49; + *background-color: #bd362f; + background-image: -moz-linear-gradient(top, #ee5f5b, #bd362f); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#ee5f5b), to(#bd362f)); + background-image: -webkit-linear-gradient(top, #ee5f5b, #bd362f); + background-image: -o-linear-gradient(top, #ee5f5b, #bd362f); + background-image: linear-gradient(to bottom, #ee5f5b, #bd362f); + background-repeat: repeat-x; + border-color: #bd362f #bd362f #802420; + border-color: rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffee5f5b', endColorstr='#ffbd362f', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled=false); +} + +.btn-danger:hover, +.btn-danger:focus, +.btn-danger:active, +.btn-danger.active, +.btn-danger.disabled, +.btn-danger[disabled] { + color: #ffffff; + background-color: #bd362f; + *background-color: #a9302a; +} + +.btn-danger:active, +.btn-danger.active { + background-color: #942a25 \9; +} + +.btn-success { + color: #ffffff; + text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25); + background-color: #5bb75b; + *background-color: #51a351; + background-image: -moz-linear-gradient(top, #62c462, #51a351); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#62c462), to(#51a351)); + background-image: -webkit-linear-gradient(top, #62c462, #51a351); + background-image: -o-linear-gradient(top, #62c462, #51a351); + background-image: linear-gradient(to bottom, #62c462, #51a351); + background-repeat: repeat-x; + border-color: #51a351 #51a351 #387038; + border-color: rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff62c462', endColorstr='#ff51a351', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled=false); +} + +.btn-success:hover, +.btn-success:focus, +.btn-success:active, +.btn-success.active, +.btn-success.disabled, +.btn-success[disabled] { + color: #ffffff; + background-color: #51a351; + *background-color: #499249; +} + +.btn-success:active, +.btn-success.active { + background-color: #408140 \9; +} + +.btn-info { + color: #ffffff; + text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25); + background-color: #49afcd; + *background-color: #2f96b4; + background-image: -moz-linear-gradient(top, #5bc0de, #2f96b4); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#5bc0de), to(#2f96b4)); + background-image: -webkit-linear-gradient(top, #5bc0de, #2f96b4); + background-image: -o-linear-gradient(top, #5bc0de, #2f96b4); + background-image: linear-gradient(to bottom, #5bc0de, #2f96b4); + background-repeat: repeat-x; + border-color: #2f96b4 #2f96b4 #1f6377; + border-color: rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de', endColorstr='#ff2f96b4', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled=false); +} + +.btn-info:hover, +.btn-info:focus, +.btn-info:active, +.btn-info.active, +.btn-info.disabled, +.btn-info[disabled] { + color: #ffffff; + background-color: #2f96b4; + *background-color: #2a85a0; +} + +.btn-info:active, +.btn-info.active { + background-color: #24748c \9; +} + +.btn-inverse { + color: #ffffff; + text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25); + background-color: #363636; + *background-color: #222222; + background-image: -moz-linear-gradient(top, #444444, #222222); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#444444), to(#222222)); + background-image: -webkit-linear-gradient(top, #444444, #222222); + background-image: -o-linear-gradient(top, #444444, #222222); + background-image: linear-gradient(to bottom, #444444, #222222); + background-repeat: repeat-x; + border-color: #222222 #222222 #000000; + border-color: rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff444444', endColorstr='#ff222222', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled=false); +} + +.btn-inverse:hover, +.btn-inverse:focus, +.btn-inverse:active, +.btn-inverse.active, +.btn-inverse.disabled, +.btn-inverse[disabled] { + color: #ffffff; + background-color: #222222; + *background-color: #151515; +} + +.btn-inverse:active, +.btn-inverse.active { + background-color: #080808 \9; +} + +button.btn, +input[type="submit"].btn { + *padding-top: 3px; + *padding-bottom: 3px; +} + +button.btn::-moz-focus-inner, +input[type="submit"].btn::-moz-focus-inner { + padding: 0; + border: 0; +} + +button.btn.btn-large, +input[type="submit"].btn.btn-large { + *padding-top: 7px; + *padding-bottom: 7px; +} + +button.btn.btn-small, +input[type="submit"].btn.btn-small { + *padding-top: 3px; + *padding-bottom: 3px; +} + +button.btn.btn-mini, +input[type="submit"].btn.btn-mini { + *padding-top: 1px; + *padding-bottom: 1px; +} + +.btn-link, +.btn-link:active, +.btn-link[disabled] { + background-color: transparent; + background-image: none; + -webkit-box-shadow: none; + -moz-box-shadow: none; + box-shadow: none; +} + +.btn-link { + color: #0088cc; + cursor: pointer; + border-color: transparent; + -webkit-border-radius: 0; + -moz-border-radius: 0; + border-radius: 0; +} + +.btn-link:hover, +.btn-link:focus { + color: #005580; + text-decoration: underline; + background-color: transparent; +} + +.btn-link[disabled]:hover, +.btn-link[disabled]:focus { + color: #333333; + text-decoration: none; +} + +.btn-group { + position: relative; + display: inline-block; + *display: inline; + *margin-left: .3em; + font-size: 0; + white-space: nowrap; + vertical-align: middle; + *zoom: 1; +} + +.btn-group:first-child { + *margin-left: 0; +} + +.btn-group + .btn-group { + margin-left: 5px; +} + +.btn-toolbar { + margin-top: 10px; + margin-bottom: 10px; + font-size: 0; +} + +.btn-toolbar > .btn + .btn, +.btn-toolbar > .btn-group + .btn, +.btn-toolbar > .btn + .btn-group { + margin-left: 5px; +} + +.btn-group > .btn { + position: relative; + -webkit-border-radius: 0; + -moz-border-radius: 0; + border-radius: 0; +} + +.btn-group > .btn + .btn { + margin-left: -1px; +} + +.btn-group > .btn, +.btn-group > .dropdown-menu, +.btn-group > .popover { + font-size: 14px; +} + +.btn-group > .btn-mini { + font-size: 10.5px; +} + +.btn-group > .btn-small { + font-size: 11.9px; +} + +.btn-group > .btn-large { + font-size: 17.5px; +} + +.btn-group > .btn:first-child { + margin-left: 0; + -webkit-border-bottom-left-radius: 4px; + border-bottom-left-radius: 4px; + -webkit-border-top-left-radius: 4px; + border-top-left-radius: 4px; + -moz-border-radius-bottomleft: 4px; + -moz-border-radius-topleft: 4px; +} + +.btn-group > .btn:last-child, +.btn-group > .dropdown-toggle { + -webkit-border-top-right-radius: 4px; + border-top-right-radius: 4px; + -webkit-border-bottom-right-radius: 4px; + border-bottom-right-radius: 4px; + -moz-border-radius-topright: 4px; + -moz-border-radius-bottomright: 4px; +} + +.btn-group > .btn.large:first-child { + margin-left: 0; + -webkit-border-bottom-left-radius: 6px; + border-bottom-left-radius: 6px; + -webkit-border-top-left-radius: 6px; + border-top-left-radius: 6px; + -moz-border-radius-bottomleft: 6px; + -moz-border-radius-topleft: 6px; +} + +.btn-group > .btn.large:last-child, +.btn-group > .large.dropdown-toggle { + -webkit-border-top-right-radius: 6px; + border-top-right-radius: 6px; + -webkit-border-bottom-right-radius: 6px; + border-bottom-right-radius: 6px; + -moz-border-radius-topright: 6px; + -moz-border-radius-bottomright: 6px; +} + +.btn-group > .btn:hover, +.btn-group > .btn:focus, +.btn-group > .btn:active, +.btn-group > .btn.active { + z-index: 2; +} + +.btn-group .dropdown-toggle:active, +.btn-group.open .dropdown-toggle { + outline: 0; +} + +.btn-group > .btn + .dropdown-toggle { + *padding-top: 5px; + padding-right: 8px; + *padding-bottom: 5px; + padding-left: 8px; + -webkit-box-shadow: inset 1px 0 0 rgba(255, 255, 255, 0.125), inset 0 1px 0 rgba(255, 255, 255, 0.2), 0 1px 2px rgba(0, 0, 0, 0.05); + -moz-box-shadow: inset 1px 0 0 rgba(255, 255, 255, 0.125), inset 0 1px 0 rgba(255, 255, 255, 0.2), 0 1px 2px rgba(0, 0, 0, 0.05); + box-shadow: inset 1px 0 0 rgba(255, 255, 255, 0.125), inset 0 1px 0 rgba(255, 255, 255, 0.2), 0 1px 2px rgba(0, 0, 0, 0.05); +} + +.btn-group > .btn-mini + .dropdown-toggle { + *padding-top: 2px; + padding-right: 5px; + *padding-bottom: 2px; + padding-left: 5px; +} + +.btn-group > .btn-small + .dropdown-toggle { + *padding-top: 5px; + *padding-bottom: 4px; +} + +.btn-group > .btn-large + .dropdown-toggle { + *padding-top: 7px; + padding-right: 12px; + *padding-bottom: 7px; + padding-left: 12px; +} + +.btn-group.open .dropdown-toggle { + background-image: none; + -webkit-box-shadow: inset 0 2px 4px rgba(0, 0, 0, 0.15), 0 1px 2px rgba(0, 0, 0, 0.05); + -moz-box-shadow: inset 0 2px 4px rgba(0, 0, 0, 0.15), 0 1px 2px rgba(0, 0, 0, 0.05); + box-shadow: inset 0 2px 4px rgba(0, 0, 0, 0.15), 0 1px 2px rgba(0, 0, 0, 0.05); +} + +.btn-group.open .btn.dropdown-toggle { + background-color: #e6e6e6; +} + +.btn-group.open .btn-primary.dropdown-toggle { + background-color: #0044cc; +} + +.btn-group.open .btn-warning.dropdown-toggle { + background-color: #f89406; +} + +.btn-group.open .btn-danger.dropdown-toggle { + background-color: #bd362f; +} + +.btn-group.open .btn-success.dropdown-toggle { + background-color: #51a351; +} + +.btn-group.open .btn-info.dropdown-toggle { + background-color: #2f96b4; +} + +.btn-group.open .btn-inverse.dropdown-toggle { + background-color: #222222; +} + +.btn .caret { + margin-top: 8px; + margin-left: 0; +} + +.btn-large .caret { + margin-top: 6px; +} + +.btn-large .caret { + border-top-width: 5px; + border-right-width: 5px; + border-left-width: 5px; +} + +.btn-mini .caret, +.btn-small .caret { + margin-top: 8px; +} + +.dropup .btn-large .caret { + border-bottom-width: 5px; +} + +.btn-primary .caret, +.btn-warning .caret, +.btn-danger .caret, +.btn-info .caret, +.btn-success .caret, +.btn-inverse .caret { + border-top-color: #ffffff; + border-bottom-color: #ffffff; +} + +.btn-group-vertical { + display: inline-block; + *display: inline; + /* IE7 inline-block hack */ + + *zoom: 1; +} + +.btn-group-vertical > .btn { + display: block; + float: none; + max-width: 100%; + -webkit-border-radius: 0; + -moz-border-radius: 0; + border-radius: 0; +} + +.btn-group-vertical > .btn + .btn { + margin-top: -1px; + margin-left: 0; +} + +.btn-group-vertical > .btn:first-child { + -webkit-border-radius: 4px 4px 0 0; + -moz-border-radius: 4px 4px 0 0; + border-radius: 4px 4px 0 0; +} + +.btn-group-vertical > .btn:last-child { + -webkit-border-radius: 0 0 4px 4px; + -moz-border-radius: 0 0 4px 4px; + border-radius: 0 0 4px 4px; +} + +.btn-group-vertical > .btn-large:first-child { + -webkit-border-radius: 6px 6px 0 0; + -moz-border-radius: 6px 6px 0 0; + border-radius: 6px 6px 0 0; +} + +.btn-group-vertical > .btn-large:last-child { + -webkit-border-radius: 0 0 6px 6px; + -moz-border-radius: 0 0 6px 6px; + border-radius: 0 0 6px 6px; +} + +.alert { + padding: 8px 35px 8px 14px; + margin-bottom: 20px; + text-shadow: 0 1px 0 rgba(255, 255, 255, 0.5); + background-color: #fcf8e3; + border: 1px solid #fbeed5; + -webkit-border-radius: 4px; + -moz-border-radius: 4px; + border-radius: 4px; +} + +.alert, +.alert h4 { + color: #c09853; +} + +.alert h4 { + margin: 0; +} + +.alert .close { + position: relative; + top: -2px; + right: -21px; + line-height: 20px; +} + +.alert-success { + color: #468847; + background-color: #dff0d8; + border-color: #d6e9c6; +} + +.alert-success h4 { + color: #468847; +} + +.alert-danger, +.alert-error { + color: #b94a48; + background-color: #f2dede; + border-color: #eed3d7; +} + +.alert-danger h4, +.alert-error h4 { + color: #b94a48; +} + +.alert-info { + color: #3a87ad; + background-color: #d9edf7; + border-color: #bce8f1; +} + +.alert-info h4 { + color: #3a87ad; +} + +.alert-block { + padding-top: 14px; + padding-bottom: 14px; +} + +.alert-block > p, +.alert-block > ul { + margin-bottom: 0; +} + +.alert-block p + p { + margin-top: 5px; +} + +.nav { + margin-bottom: 20px; + margin-left: 0; + list-style: none; +} + +.nav > li > a { + display: block; +} + +.nav > li > a:hover, +.nav > li > a:focus { + text-decoration: none; + background-color: #eeeeee; +} + +.nav > li > a > img { + max-width: none; +} + +.nav > .pull-right { + float: right; +} + +.nav-header { + display: block; + padding: 3px 15px; + font-size: 11px; + font-weight: bold; + line-height: 20px; + color: #999999; + text-shadow: 0 1px 0 rgba(255, 255, 255, 0.5); + text-transform: uppercase; +} + +.nav li + .nav-header { + margin-top: 9px; +} + +.nav-list { + padding-right: 15px; + padding-left: 15px; + margin-bottom: 0; +} + +.nav-list > li > a, +.nav-list .nav-header { + margin-right: -15px; + margin-left: -15px; + text-shadow: 0 1px 0 rgba(255, 255, 255, 0.5); +} + +.nav-list > li > a { + padding: 3px 15px; +} + +.nav-list > .active > a, +.nav-list > .active > a:hover, +.nav-list > .active > a:focus { + color: #ffffff; + text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.2); + background-color: #0088cc; +} + +.nav-list [class^="icon-"], +.nav-list [class*=" icon-"] { + margin-right: 2px; +} + +.nav-list .divider { + *width: 100%; + height: 1px; + margin: 9px 1px; + *margin: -5px 0 5px; + overflow: hidden; + background-color: #e5e5e5; + border-bottom: 1px solid #ffffff; +} + +.nav-tabs, +.nav-pills { + *zoom: 1; +} + +.nav-tabs:before, +.nav-pills:before, +.nav-tabs:after, +.nav-pills:after { + display: table; + line-height: 0; + content: ""; +} + +.nav-tabs:after, +.nav-pills:after { + clear: both; +} + +.nav-tabs > li, +.nav-pills > li { + float: left; +} + +.nav-tabs > li > a, +.nav-pills > li > a { + padding-right: 12px; + padding-left: 12px; + margin-right: 2px; + line-height: 14px; +} + +.nav-tabs { + border-bottom: 1px solid #ddd; +} + +.nav-tabs > li { + margin-bottom: -1px; +} + +.nav-tabs > li > a { + padding-top: 8px; + padding-bottom: 8px; + line-height: 20px; + border: 1px solid transparent; + -webkit-border-radius: 4px 4px 0 0; + -moz-border-radius: 4px 4px 0 0; + border-radius: 4px 4px 0 0; +} + +.nav-tabs > li > a:hover, +.nav-tabs > li > a:focus { + border-color: #eeeeee #eeeeee #dddddd; +} + +.nav-tabs > .active > a, +.nav-tabs > .active > a:hover, +.nav-tabs > .active > a:focus { + color: #555555; + cursor: default; + background-color: #ffffff; + border: 1px solid #ddd; + border-bottom-color: transparent; +} + +.nav-pills > li > a { + padding-top: 8px; + padding-bottom: 8px; + margin-top: 2px; + margin-bottom: 2px; + -webkit-border-radius: 5px; + -moz-border-radius: 5px; + border-radius: 5px; +} + +.nav-pills > .active > a, +.nav-pills > .active > a:hover, +.nav-pills > .active > a:focus { + color: #ffffff; + background-color: #0088cc; +} + +.nav-stacked > li { + float: none; +} + +.nav-stacked > li > a { + margin-right: 0; +} + +.nav-tabs.nav-stacked { + border-bottom: 0; +} + +.nav-tabs.nav-stacked > li > a { + border: 1px solid #ddd; + -webkit-border-radius: 0; + -moz-border-radius: 0; + border-radius: 0; +} + +.nav-tabs.nav-stacked > li:first-child > a { + -webkit-border-top-right-radius: 4px; + border-top-right-radius: 4px; + -webkit-border-top-left-radius: 4px; + border-top-left-radius: 4px; + -moz-border-radius-topright: 4px; + -moz-border-radius-topleft: 4px; +} + +.nav-tabs.nav-stacked > li:last-child > a { + -webkit-border-bottom-right-radius: 4px; + border-bottom-right-radius: 4px; + -webkit-border-bottom-left-radius: 4px; + border-bottom-left-radius: 4px; + -moz-border-radius-bottomright: 4px; + -moz-border-radius-bottomleft: 4px; +} + +.nav-tabs.nav-stacked > li > a:hover, +.nav-tabs.nav-stacked > li > a:focus { + z-index: 2; + border-color: #ddd; +} + +.nav-pills.nav-stacked > li > a { + margin-bottom: 3px; +} + +.nav-pills.nav-stacked > li:last-child > a { + margin-bottom: 1px; +} + +.nav-tabs .dropdown-menu { + -webkit-border-radius: 0 0 6px 6px; + -moz-border-radius: 0 0 6px 6px; + border-radius: 0 0 6px 6px; +} + +.nav-pills .dropdown-menu { + -webkit-border-radius: 6px; + -moz-border-radius: 6px; + border-radius: 6px; +} + +.nav .dropdown-toggle .caret { + margin-top: 6px; + border-top-color: #0088cc; + border-bottom-color: #0088cc; +} + +.nav .dropdown-toggle:hover .caret, +.nav .dropdown-toggle:focus .caret { + border-top-color: #005580; + border-bottom-color: #005580; +} + +/* move down carets for tabs */ + +.nav-tabs .dropdown-toggle .caret { + margin-top: 8px; +} + +.nav .active .dropdown-toggle .caret { + border-top-color: #fff; + border-bottom-color: #fff; +} + +.nav-tabs .active .dropdown-toggle .caret { + border-top-color: #555555; + border-bottom-color: #555555; +} + +.nav > .dropdown.active > a:hover, +.nav > .dropdown.active > a:focus { + cursor: pointer; +} + +.nav-tabs .open .dropdown-toggle, +.nav-pills .open .dropdown-toggle, +.nav > li.dropdown.open.active > a:hover, +.nav > li.dropdown.open.active > a:focus { + color: #ffffff; + background-color: #999999; + border-color: #999999; +} + +.nav li.dropdown.open .caret, +.nav li.dropdown.open.active .caret, +.nav li.dropdown.open a:hover .caret, +.nav li.dropdown.open a:focus .caret { + border-top-color: #ffffff; + border-bottom-color: #ffffff; + opacity: 1; + filter: alpha(opacity=100); +} + +.tabs-stacked .open > a:hover, +.tabs-stacked .open > a:focus { + border-color: #999999; +} + +.tabbable { + *zoom: 1; +} + +.tabbable:before, +.tabbable:after { + display: table; + line-height: 0; + content: ""; +} + +.tabbable:after { + clear: both; +} + +.tab-content { + overflow: auto; +} + +.tabs-below > .nav-tabs, +.tabs-right > .nav-tabs, +.tabs-left > .nav-tabs { + border-bottom: 0; +} + +.tab-content > .tab-pane, +.pill-content > .pill-pane { + display: none; +} + +.tab-content > .active, +.pill-content > .active { + display: block; +} + +.tabs-below > .nav-tabs { + border-top: 1px solid #ddd; +} + +.tabs-below > .nav-tabs > li { + margin-top: -1px; + margin-bottom: 0; +} + +.tabs-below > .nav-tabs > li > a { + -webkit-border-radius: 0 0 4px 4px; + -moz-border-radius: 0 0 4px 4px; + border-radius: 0 0 4px 4px; +} + +.tabs-below > .nav-tabs > li > a:hover, +.tabs-below > .nav-tabs > li > a:focus { + border-top-color: #ddd; + border-bottom-color: transparent; +} + +.tabs-below > .nav-tabs > .active > a, +.tabs-below > .nav-tabs > .active > a:hover, +.tabs-below > .nav-tabs > .active > a:focus { + border-color: transparent #ddd #ddd #ddd; +} + +.tabs-left > .nav-tabs > li, +.tabs-right > .nav-tabs > li { + float: none; +} + +.tabs-left > .nav-tabs > li > a, +.tabs-right > .nav-tabs > li > a { + min-width: 74px; + margin-right: 0; + margin-bottom: 3px; +} + +.tabs-left > .nav-tabs { + float: left; + margin-right: 19px; + border-right: 1px solid #ddd; +} + +.tabs-left > .nav-tabs > li > a { + margin-right: -1px; + -webkit-border-radius: 4px 0 0 4px; + -moz-border-radius: 4px 0 0 4px; + border-radius: 4px 0 0 4px; +} + +.tabs-left > .nav-tabs > li > a:hover, +.tabs-left > .nav-tabs > li > a:focus { + border-color: #eeeeee #dddddd #eeeeee #eeeeee; +} + +.tabs-left > .nav-tabs .active > a, +.tabs-left > .nav-tabs .active > a:hover, +.tabs-left > .nav-tabs .active > a:focus { + border-color: #ddd transparent #ddd #ddd; + *border-right-color: #ffffff; +} + +.tabs-right > .nav-tabs { + float: right; + margin-left: 19px; + border-left: 1px solid #ddd; +} + +.tabs-right > .nav-tabs > li > a { + margin-left: -1px; + -webkit-border-radius: 0 4px 4px 0; + -moz-border-radius: 0 4px 4px 0; + border-radius: 0 4px 4px 0; +} + +.tabs-right > .nav-tabs > li > a:hover, +.tabs-right > .nav-tabs > li > a:focus { + border-color: #eeeeee #eeeeee #eeeeee #dddddd; +} + +.tabs-right > .nav-tabs .active > a, +.tabs-right > .nav-tabs .active > a:hover, +.tabs-right > .nav-tabs .active > a:focus { + border-color: #ddd #ddd #ddd transparent; + *border-left-color: #ffffff; +} + +.nav > .disabled > a { + color: #999999; +} + +.nav > .disabled > a:hover, +.nav > .disabled > a:focus { + text-decoration: none; + cursor: default; + background-color: transparent; +} + +.navbar { + *position: relative; + *z-index: 2; + margin-bottom: 20px; + overflow: visible; +} + +.navbar-inner { + min-height: 40px; + padding-right: 20px; + padding-left: 20px; + background-color: #fafafa; + background-image: -moz-linear-gradient(top, #ffffff, #f2f2f2); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#ffffff), to(#f2f2f2)); + background-image: -webkit-linear-gradient(top, #ffffff, #f2f2f2); + background-image: -o-linear-gradient(top, #ffffff, #f2f2f2); + background-image: linear-gradient(to bottom, #ffffff, #f2f2f2); + background-repeat: repeat-x; + border: 1px solid #d4d4d4; + -webkit-border-radius: 4px; + -moz-border-radius: 4px; + border-radius: 4px; + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff', endColorstr='#fff2f2f2', GradientType=0); + *zoom: 1; + -webkit-box-shadow: 0 1px 4px rgba(0, 0, 0, 0.065); + -moz-box-shadow: 0 1px 4px rgba(0, 0, 0, 0.065); + box-shadow: 0 1px 4px rgba(0, 0, 0, 0.065); +} + +.navbar-inner:before, +.navbar-inner:after { + display: table; + line-height: 0; + content: ""; +} + +.navbar-inner:after { + clear: both; +} + +.navbar .container { + width: auto; +} + +.nav-collapse.collapse { + height: auto; + overflow: visible; +} + +.navbar .brand { + display: block; + float: left; + padding: 10px 20px 10px; + margin-left: -20px; + font-size: 20px; + font-weight: 200; + color: #777777; + text-shadow: 0 1px 0 #ffffff; +} + +.navbar .brand:hover, +.navbar .brand:focus { + text-decoration: none; +} + +.navbar-text { + margin-bottom: 0; + line-height: 40px; + color: #777777; +} + +.navbar-link { + color: #777777; +} + +.navbar-link:hover, +.navbar-link:focus { + color: #333333; +} + +.navbar .divider-vertical { + height: 40px; + margin: 0 9px; + border-right: 1px solid #ffffff; + border-left: 1px solid #f2f2f2; +} + +.navbar .btn, +.navbar .btn-group { + margin-top: 5px; +} + +.navbar .btn-group .btn, +.navbar .input-prepend .btn, +.navbar .input-append .btn, +.navbar .input-prepend .btn-group, +.navbar .input-append .btn-group { + margin-top: 0; +} + +.navbar-form { + margin-bottom: 0; + *zoom: 1; +} + +.navbar-form:before, +.navbar-form:after { + display: table; + line-height: 0; + content: ""; +} + +.navbar-form:after { + clear: both; +} + +.navbar-form input, +.navbar-form select, +.navbar-form .radio, +.navbar-form .checkbox { + margin-top: 5px; +} + +.navbar-form input, +.navbar-form select, +.navbar-form .btn { + display: inline-block; + margin-bottom: 0; +} + +.navbar-form input[type="image"], +.navbar-form input[type="checkbox"], +.navbar-form input[type="radio"] { + margin-top: 3px; +} + +.navbar-form .input-append, +.navbar-form .input-prepend { + margin-top: 5px; + white-space: nowrap; +} + +.navbar-form .input-append input, +.navbar-form .input-prepend input { + margin-top: 0; +} + +.navbar-search { + position: relative; + float: left; + margin-top: 5px; + margin-bottom: 0; +} + +.navbar-search .search-query { + padding: 4px 14px; + margin-bottom: 0; + font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; + font-size: 13px; + font-weight: normal; + line-height: 1; + -webkit-border-radius: 15px; + -moz-border-radius: 15px; + border-radius: 15px; +} + +.navbar-static-top { + position: static; + margin-bottom: 0; +} + +.navbar-static-top .navbar-inner { + -webkit-border-radius: 0; + -moz-border-radius: 0; + border-radius: 0; +} + +.navbar-fixed-top, +.navbar-fixed-bottom { + position: fixed; + right: 0; + left: 0; + z-index: 1030; + margin-bottom: 0; +} + +.navbar-fixed-top .navbar-inner, +.navbar-static-top .navbar-inner { + border-width: 0 0 1px; +} + +.navbar-fixed-bottom .navbar-inner { + border-width: 1px 0 0; +} + +.navbar-fixed-top .navbar-inner, +.navbar-fixed-bottom .navbar-inner { + padding-right: 0; + padding-left: 0; + -webkit-border-radius: 0; + -moz-border-radius: 0; + border-radius: 0; +} + +.navbar-static-top .container, +.navbar-fixed-top .container, +.navbar-fixed-bottom .container { + width: 940px; +} + +.navbar-fixed-top { + top: 0; +} + +.navbar-fixed-top .navbar-inner, +.navbar-static-top .navbar-inner { + -webkit-box-shadow: 0 1px 10px rgba(0, 0, 0, 0.1); + -moz-box-shadow: 0 1px 10px rgba(0, 0, 0, 0.1); + box-shadow: 0 1px 10px rgba(0, 0, 0, 0.1); +} + +.navbar-fixed-bottom { + bottom: 0; +} + +.navbar-fixed-bottom .navbar-inner { + -webkit-box-shadow: 0 -1px 10px rgba(0, 0, 0, 0.1); + -moz-box-shadow: 0 -1px 10px rgba(0, 0, 0, 0.1); + box-shadow: 0 -1px 10px rgba(0, 0, 0, 0.1); +} + +.navbar .nav { + position: relative; + left: 0; + display: block; + float: left; + margin: 0 10px 0 0; +} + +.navbar .nav.pull-right { + float: right; + margin-right: 0; +} + +.navbar .nav > li { + float: left; +} + +.navbar .nav > li > a { + float: none; + padding: 10px 15px 10px; + color: #777777; + text-decoration: none; + text-shadow: 0 1px 0 #ffffff; +} + +.navbar .nav .dropdown-toggle .caret { + margin-top: 8px; +} + +.navbar .nav > li > a:focus, +.navbar .nav > li > a:hover { + color: #333333; + text-decoration: none; + background-color: transparent; +} + +.navbar .nav > .active > a, +.navbar .nav > .active > a:hover, +.navbar .nav > .active > a:focus { + color: #555555; + text-decoration: none; + background-color: #e5e5e5; + -webkit-box-shadow: inset 0 3px 8px rgba(0, 0, 0, 0.125); + -moz-box-shadow: inset 0 3px 8px rgba(0, 0, 0, 0.125); + box-shadow: inset 0 3px 8px rgba(0, 0, 0, 0.125); +} + +.navbar .btn-navbar { + display: none; + float: right; + padding: 7px 10px; + margin-right: 5px; + margin-left: 5px; + color: #ffffff; + text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25); + background-color: #ededed; + *background-color: #e5e5e5; + background-image: -moz-linear-gradient(top, #f2f2f2, #e5e5e5); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#f2f2f2), to(#e5e5e5)); + background-image: -webkit-linear-gradient(top, #f2f2f2, #e5e5e5); + background-image: -o-linear-gradient(top, #f2f2f2, #e5e5e5); + background-image: linear-gradient(to bottom, #f2f2f2, #e5e5e5); + background-repeat: repeat-x; + border-color: #e5e5e5 #e5e5e5 #bfbfbf; + border-color: rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff2f2f2', endColorstr='#ffe5e5e5', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled=false); + -webkit-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1), 0 1px 0 rgba(255, 255, 255, 0.075); + -moz-box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1), 0 1px 0 rgba(255, 255, 255, 0.075); + box-shadow: inset 0 1px 0 rgba(255, 255, 255, 0.1), 0 1px 0 rgba(255, 255, 255, 0.075); +} + +.navbar .btn-navbar:hover, +.navbar .btn-navbar:focus, +.navbar .btn-navbar:active, +.navbar .btn-navbar.active, +.navbar .btn-navbar.disabled, +.navbar .btn-navbar[disabled] { + color: #ffffff; + background-color: #e5e5e5; + *background-color: #d9d9d9; +} + +.navbar .btn-navbar:active, +.navbar .btn-navbar.active { + background-color: #cccccc \9; +} + +.navbar .btn-navbar .icon-bar { + display: block; + width: 18px; + height: 2px; + background-color: #f5f5f5; + -webkit-border-radius: 1px; + -moz-border-radius: 1px; + border-radius: 1px; + -webkit-box-shadow: 0 1px 0 rgba(0, 0, 0, 0.25); + -moz-box-shadow: 0 1px 0 rgba(0, 0, 0, 0.25); + box-shadow: 0 1px 0 rgba(0, 0, 0, 0.25); +} + +.btn-navbar .icon-bar + .icon-bar { + margin-top: 3px; +} + +.navbar .nav > li > .dropdown-menu:before { + position: absolute; + top: -7px; + left: 9px; + display: inline-block; + border-right: 7px solid transparent; + border-bottom: 7px solid #ccc; + border-left: 7px solid transparent; + border-bottom-color: rgba(0, 0, 0, 0.2); + content: ''; +} + +.navbar .nav > li > .dropdown-menu:after { + position: absolute; + top: -6px; + left: 10px; + display: inline-block; + border-right: 6px solid transparent; + border-bottom: 6px solid #ffffff; + border-left: 6px solid transparent; + content: ''; +} + +.navbar-fixed-bottom .nav > li > .dropdown-menu:before { + top: auto; + bottom: -7px; + border-top: 7px solid #ccc; + border-bottom: 0; + border-top-color: rgba(0, 0, 0, 0.2); +} + +.navbar-fixed-bottom .nav > li > .dropdown-menu:after { + top: auto; + bottom: -6px; + border-top: 6px solid #ffffff; + border-bottom: 0; +} + +.navbar .nav li.dropdown > a:hover .caret, +.navbar .nav li.dropdown > a:focus .caret { + border-top-color: #333333; + border-bottom-color: #333333; +} + +.navbar .nav li.dropdown.open > .dropdown-toggle, +.navbar .nav li.dropdown.active > .dropdown-toggle, +.navbar .nav li.dropdown.open.active > .dropdown-toggle { + color: #555555; + background-color: #e5e5e5; +} + +.navbar .nav li.dropdown > .dropdown-toggle .caret { + border-top-color: #777777; + border-bottom-color: #777777; +} + +.navbar .nav li.dropdown.open > .dropdown-toggle .caret, +.navbar .nav li.dropdown.active > .dropdown-toggle .caret, +.navbar .nav li.dropdown.open.active > .dropdown-toggle .caret { + border-top-color: #555555; + border-bottom-color: #555555; +} + +.navbar .pull-right > li > .dropdown-menu, +.navbar .nav > li > .dropdown-menu.pull-right { + right: 0; + left: auto; +} + +.navbar .pull-right > li > .dropdown-menu:before, +.navbar .nav > li > .dropdown-menu.pull-right:before { + right: 12px; + left: auto; +} + +.navbar .pull-right > li > .dropdown-menu:after, +.navbar .nav > li > .dropdown-menu.pull-right:after { + right: 13px; + left: auto; +} + +.navbar .pull-right > li > .dropdown-menu .dropdown-menu, +.navbar .nav > li > .dropdown-menu.pull-right .dropdown-menu { + right: 100%; + left: auto; + margin-right: -1px; + margin-left: 0; + -webkit-border-radius: 6px 0 6px 6px; + -moz-border-radius: 6px 0 6px 6px; + border-radius: 6px 0 6px 6px; +} + +.navbar-inverse .navbar-inner { + background-color: #1b1b1b; + background-image: -moz-linear-gradient(top, #222222, #111111); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#222222), to(#111111)); + background-image: -webkit-linear-gradient(top, #222222, #111111); + background-image: -o-linear-gradient(top, #222222, #111111); + background-image: linear-gradient(to bottom, #222222, #111111); + background-repeat: repeat-x; + border-color: #252525; + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff222222', endColorstr='#ff111111', GradientType=0); +} + +.navbar-inverse .brand, +.navbar-inverse .nav > li > a { + color: #999999; + text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25); +} + +.navbar-inverse .brand:hover, +.navbar-inverse .nav > li > a:hover, +.navbar-inverse .brand:focus, +.navbar-inverse .nav > li > a:focus { + color: #ffffff; +} + +.navbar-inverse .brand { + color: #999999; +} + +.navbar-inverse .navbar-text { + color: #999999; +} + +.navbar-inverse .nav > li > a:focus, +.navbar-inverse .nav > li > a:hover { + color: #ffffff; + background-color: transparent; +} + +.navbar-inverse .nav .active > a, +.navbar-inverse .nav .active > a:hover, +.navbar-inverse .nav .active > a:focus { + color: #ffffff; + background-color: #111111; +} + +.navbar-inverse .navbar-link { + color: #999999; +} + +.navbar-inverse .navbar-link:hover, +.navbar-inverse .navbar-link:focus { + color: #ffffff; +} + +.navbar-inverse .divider-vertical { + border-right-color: #222222; + border-left-color: #111111; +} + +.navbar-inverse .nav li.dropdown.open > .dropdown-toggle, +.navbar-inverse .nav li.dropdown.active > .dropdown-toggle, +.navbar-inverse .nav li.dropdown.open.active > .dropdown-toggle { + color: #ffffff; + background-color: #111111; +} + +.navbar-inverse .nav li.dropdown > a:hover .caret, +.navbar-inverse .nav li.dropdown > a:focus .caret { + border-top-color: #ffffff; + border-bottom-color: #ffffff; +} + +.navbar-inverse .nav li.dropdown > .dropdown-toggle .caret { + border-top-color: #999999; + border-bottom-color: #999999; +} + +.navbar-inverse .nav li.dropdown.open > .dropdown-toggle .caret, +.navbar-inverse .nav li.dropdown.active > .dropdown-toggle .caret, +.navbar-inverse .nav li.dropdown.open.active > .dropdown-toggle .caret { + border-top-color: #ffffff; + border-bottom-color: #ffffff; +} + +.navbar-inverse .navbar-search .search-query { + color: #ffffff; + background-color: #515151; + border-color: #111111; + -webkit-box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1), 0 1px 0 rgba(255, 255, 255, 0.15); + -moz-box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1), 0 1px 0 rgba(255, 255, 255, 0.15); + box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1), 0 1px 0 rgba(255, 255, 255, 0.15); + -webkit-transition: none; + -moz-transition: none; + -o-transition: none; + transition: none; +} + +.navbar-inverse .navbar-search .search-query:-moz-placeholder { + color: #cccccc; +} + +.navbar-inverse .navbar-search .search-query:-ms-input-placeholder { + color: #cccccc; +} + +.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder { + color: #cccccc; +} + +.navbar-inverse .navbar-search .search-query:focus, +.navbar-inverse .navbar-search .search-query.focused { + padding: 5px 15px; + color: #333333; + text-shadow: 0 1px 0 #ffffff; + background-color: #ffffff; + border: 0; + outline: 0; + -webkit-box-shadow: 0 0 3px rgba(0, 0, 0, 0.15); + -moz-box-shadow: 0 0 3px rgba(0, 0, 0, 0.15); + box-shadow: 0 0 3px rgba(0, 0, 0, 0.15); +} + +.navbar-inverse .btn-navbar { + color: #ffffff; + text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25); + background-color: #0e0e0e; + *background-color: #040404; + background-image: -moz-linear-gradient(top, #151515, #040404); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#151515), to(#040404)); + background-image: -webkit-linear-gradient(top, #151515, #040404); + background-image: -o-linear-gradient(top, #151515, #040404); + background-image: linear-gradient(to bottom, #151515, #040404); + background-repeat: repeat-x; + border-color: #040404 #040404 #000000; + border-color: rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25); + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff151515', endColorstr='#ff040404', GradientType=0); + filter: progid:DXImageTransform.Microsoft.gradient(enabled=false); +} + +.navbar-inverse .btn-navbar:hover, +.navbar-inverse .btn-navbar:focus, +.navbar-inverse .btn-navbar:active, +.navbar-inverse .btn-navbar.active, +.navbar-inverse .btn-navbar.disabled, +.navbar-inverse .btn-navbar[disabled] { + color: #ffffff; + background-color: #040404; + *background-color: #000000; +} + +.navbar-inverse .btn-navbar:active, +.navbar-inverse .btn-navbar.active { + background-color: #000000 \9; +} + +.breadcrumb { + padding: 8px 15px; + margin: 0 0 20px; + list-style: none; + background-color: #f5f5f5; + -webkit-border-radius: 4px; + -moz-border-radius: 4px; + border-radius: 4px; +} + +.breadcrumb > li { + display: inline-block; + *display: inline; + text-shadow: 0 1px 0 #ffffff; + *zoom: 1; +} + +.breadcrumb > li > .divider { + padding: 0 5px; + color: #ccc; +} + +.breadcrumb > .active { + color: #999999; +} + +.pagination { + margin: 20px 0; +} + +.pagination ul { + display: inline-block; + *display: inline; + margin-bottom: 0; + margin-left: 0; + -webkit-border-radius: 4px; + -moz-border-radius: 4px; + border-radius: 4px; + *zoom: 1; + -webkit-box-shadow: 0 1px 2px rgba(0, 0, 0, 0.05); + -moz-box-shadow: 0 1px 2px rgba(0, 0, 0, 0.05); + box-shadow: 0 1px 2px rgba(0, 0, 0, 0.05); +} + +.pagination ul > li { + display: inline; +} + +.pagination ul > li > a, +.pagination ul > li > span { + float: left; + padding: 4px 12px; + line-height: 20px; + text-decoration: none; + background-color: #ffffff; + border: 1px solid #dddddd; + border-left-width: 0; +} + +.pagination ul > li > a:hover, +.pagination ul > li > a:focus, +.pagination ul > .active > a, +.pagination ul > .active > span { + background-color: #f5f5f5; +} + +.pagination ul > .active > a, +.pagination ul > .active > span { + color: #999999; + cursor: default; +} + +.pagination ul > .disabled > span, +.pagination ul > .disabled > a, +.pagination ul > .disabled > a:hover, +.pagination ul > .disabled > a:focus { + color: #999999; + cursor: default; + background-color: transparent; +} + +.pagination ul > li:first-child > a, +.pagination ul > li:first-child > span { + border-left-width: 1px; + -webkit-border-bottom-left-radius: 4px; + border-bottom-left-radius: 4px; + -webkit-border-top-left-radius: 4px; + border-top-left-radius: 4px; + -moz-border-radius-bottomleft: 4px; + -moz-border-radius-topleft: 4px; +} + +.pagination ul > li:last-child > a, +.pagination ul > li:last-child > span { + -webkit-border-top-right-radius: 4px; + border-top-right-radius: 4px; + -webkit-border-bottom-right-radius: 4px; + border-bottom-right-radius: 4px; + -moz-border-radius-topright: 4px; + -moz-border-radius-bottomright: 4px; +} + +.pagination-centered { + text-align: center; +} + +.pagination-right { + text-align: right; +} + +.pagination-large ul > li > a, +.pagination-large ul > li > span { + padding: 11px 19px; + font-size: 17.5px; +} + +.pagination-large ul > li:first-child > a, +.pagination-large ul > li:first-child > span { + -webkit-border-bottom-left-radius: 6px; + border-bottom-left-radius: 6px; + -webkit-border-top-left-radius: 6px; + border-top-left-radius: 6px; + -moz-border-radius-bottomleft: 6px; + -moz-border-radius-topleft: 6px; +} + +.pagination-large ul > li:last-child > a, +.pagination-large ul > li:last-child > span { + -webkit-border-top-right-radius: 6px; + border-top-right-radius: 6px; + -webkit-border-bottom-right-radius: 6px; + border-bottom-right-radius: 6px; + -moz-border-radius-topright: 6px; + -moz-border-radius-bottomright: 6px; +} + +.pagination-mini ul > li:first-child > a, +.pagination-small ul > li:first-child > a, +.pagination-mini ul > li:first-child > span, +.pagination-small ul > li:first-child > span { + -webkit-border-bottom-left-radius: 3px; + border-bottom-left-radius: 3px; + -webkit-border-top-left-radius: 3px; + border-top-left-radius: 3px; + -moz-border-radius-bottomleft: 3px; + -moz-border-radius-topleft: 3px; +} + +.pagination-mini ul > li:last-child > a, +.pagination-small ul > li:last-child > a, +.pagination-mini ul > li:last-child > span, +.pagination-small ul > li:last-child > span { + -webkit-border-top-right-radius: 3px; + border-top-right-radius: 3px; + -webkit-border-bottom-right-radius: 3px; + border-bottom-right-radius: 3px; + -moz-border-radius-topright: 3px; + -moz-border-radius-bottomright: 3px; +} + +.pagination-small ul > li > a, +.pagination-small ul > li > span { + padding: 2px 10px; + font-size: 11.9px; +} + +.pagination-mini ul > li > a, +.pagination-mini ul > li > span { + padding: 0 6px; + font-size: 10.5px; +} + +.pager { + margin: 20px 0; + text-align: center; + list-style: none; + *zoom: 1; +} + +.pager:before, +.pager:after { + display: table; + line-height: 0; + content: ""; +} + +.pager:after { + clear: both; +} + +.pager li { + display: inline; +} + +.pager li > a, +.pager li > span { + display: inline-block; + padding: 5px 14px; + background-color: #fff; + border: 1px solid #ddd; + -webkit-border-radius: 15px; + -moz-border-radius: 15px; + border-radius: 15px; +} + +.pager li > a:hover, +.pager li > a:focus { + text-decoration: none; + background-color: #f5f5f5; +} + +.pager .next > a, +.pager .next > span { + float: right; +} + +.pager .previous > a, +.pager .previous > span { + float: left; +} + +.pager .disabled > a, +.pager .disabled > a:hover, +.pager .disabled > a:focus, +.pager .disabled > span { + color: #999999; + cursor: default; + background-color: #fff; +} + +.modal-backdrop { + position: fixed; + top: 0; + right: 0; + bottom: 0; + left: 0; + z-index: 1040; + background-color: #000000; +} + +.modal-backdrop.fade { + opacity: 0; +} + +.modal-backdrop, +.modal-backdrop.fade.in { + opacity: 0.8; + filter: alpha(opacity=80); +} + +.modal { + position: fixed; + top: 10%; + left: 50%; + z-index: 1050; + width: 560px; + margin-left: -280px; + background-color: #ffffff; + border: 1px solid #999; + border: 1px solid rgba(0, 0, 0, 0.3); + *border: 1px solid #999; + -webkit-border-radius: 6px; + -moz-border-radius: 6px; + border-radius: 6px; + outline: none; + -webkit-box-shadow: 0 3px 7px rgba(0, 0, 0, 0.3); + -moz-box-shadow: 0 3px 7px rgba(0, 0, 0, 0.3); + box-shadow: 0 3px 7px rgba(0, 0, 0, 0.3); + -webkit-background-clip: padding-box; + -moz-background-clip: padding-box; + background-clip: padding-box; +} + +.modal.fade { + top: -25%; + -webkit-transition: opacity 0.3s linear, top 0.3s ease-out; + -moz-transition: opacity 0.3s linear, top 0.3s ease-out; + -o-transition: opacity 0.3s linear, top 0.3s ease-out; + transition: opacity 0.3s linear, top 0.3s ease-out; +} + +.modal.fade.in { + top: 10%; +} + +.modal-header { + padding: 9px 15px; + border-bottom: 1px solid #eee; +} + +.modal-header .close { + margin-top: 2px; +} + +.modal-header h3 { + margin: 0; + line-height: 30px; +} + +.modal-body { + position: relative; + max-height: 400px; + padding: 15px; + overflow-y: auto; +} + +.modal-form { + margin-bottom: 0; +} + +.modal-footer { + padding: 14px 15px 15px; + margin-bottom: 0; + text-align: right; + background-color: #f5f5f5; + border-top: 1px solid #ddd; + -webkit-border-radius: 0 0 6px 6px; + -moz-border-radius: 0 0 6px 6px; + border-radius: 0 0 6px 6px; + *zoom: 1; + -webkit-box-shadow: inset 0 1px 0 #ffffff; + -moz-box-shadow: inset 0 1px 0 #ffffff; + box-shadow: inset 0 1px 0 #ffffff; +} + +.modal-footer:before, +.modal-footer:after { + display: table; + line-height: 0; + content: ""; +} + +.modal-footer:after { + clear: both; +} + +.modal-footer .btn + .btn { + margin-bottom: 0; + margin-left: 5px; +} + +.modal-footer .btn-group .btn + .btn { + margin-left: -1px; +} + +.modal-footer .btn-block + .btn-block { + margin-left: 0; +} + +.tooltip { + position: absolute; + z-index: 1030; + display: block; + font-size: 11px; + line-height: 1.4; + opacity: 0; + filter: alpha(opacity=0); + visibility: visible; +} + +.tooltip.in { + opacity: 0.8; + filter: alpha(opacity=80); +} + +.tooltip.top { + padding: 5px 0; + margin-top: -3px; +} + +.tooltip.right { + padding: 0 5px; + margin-left: 3px; +} + +.tooltip.bottom { + padding: 5px 0; + margin-top: 3px; +} + +.tooltip.left { + padding: 0 5px; + margin-left: -3px; +} + +.tooltip-inner { + max-width: 200px; + padding: 8px; + color: #ffffff; + text-align: center; + text-decoration: none; + background-color: #000000; + -webkit-border-radius: 4px; + -moz-border-radius: 4px; + border-radius: 4px; +} + +.tooltip-arrow { + position: absolute; + width: 0; + height: 0; + border-color: transparent; + border-style: solid; +} + +.tooltip.top .tooltip-arrow { + bottom: 0; + left: 50%; + margin-left: -5px; + border-top-color: #000000; + border-width: 5px 5px 0; +} + +.tooltip.right .tooltip-arrow { + top: 50%; + left: 0; + margin-top: -5px; + border-right-color: #000000; + border-width: 5px 5px 5px 0; +} + +.tooltip.left .tooltip-arrow { + top: 50%; + right: 0; + margin-top: -5px; + border-left-color: #000000; + border-width: 5px 0 5px 5px; +} + +.tooltip.bottom .tooltip-arrow { + top: 0; + left: 50%; + margin-left: -5px; + border-bottom-color: #000000; + border-width: 0 5px 5px; +} + +.popover { + position: absolute; + top: 0; + left: 0; + z-index: 1010; + display: none; + max-width: 276px; + padding: 1px; + text-align: left; + white-space: normal; + background-color: #ffffff; + border: 1px solid #ccc; + border: 1px solid rgba(0, 0, 0, 0.2); + -webkit-border-radius: 6px; + -moz-border-radius: 6px; + border-radius: 6px; + -webkit-box-shadow: 0 5px 10px rgba(0, 0, 0, 0.2); + -moz-box-shadow: 0 5px 10px rgba(0, 0, 0, 0.2); + box-shadow: 0 5px 10px rgba(0, 0, 0, 0.2); + -webkit-background-clip: padding-box; + -moz-background-clip: padding; + background-clip: padding-box; +} + +.popover.top { + margin-top: -10px; +} + +.popover.right { + margin-left: 10px; +} + +.popover.bottom { + margin-top: 10px; +} + +.popover.left { + margin-left: -10px; +} + +.popover-title { + padding: 8px 14px; + margin: 0; + font-size: 14px; + font-weight: normal; + line-height: 18px; + background-color: #f7f7f7; + border-bottom: 1px solid #ebebeb; + -webkit-border-radius: 5px 5px 0 0; + -moz-border-radius: 5px 5px 0 0; + border-radius: 5px 5px 0 0; +} + +.popover-title:empty { + display: none; +} + +.popover-content { + padding: 9px 14px; +} + +.popover .arrow, +.popover .arrow:after { + position: absolute; + display: block; + width: 0; + height: 0; + border-color: transparent; + border-style: solid; +} + +.popover .arrow { + border-width: 11px; +} + +.popover .arrow:after { + border-width: 10px; + content: ""; +} + +.popover.top .arrow { + bottom: -11px; + left: 50%; + margin-left: -11px; + border-top-color: #999; + border-top-color: rgba(0, 0, 0, 0.25); + border-bottom-width: 0; +} + +.popover.top .arrow:after { + bottom: 1px; + margin-left: -10px; + border-top-color: #ffffff; + border-bottom-width: 0; +} + +.popover.right .arrow { + top: 50%; + left: -11px; + margin-top: -11px; + border-right-color: #999; + border-right-color: rgba(0, 0, 0, 0.25); + border-left-width: 0; +} + +.popover.right .arrow:after { + bottom: -10px; + left: 1px; + border-right-color: #ffffff; + border-left-width: 0; +} + +.popover.bottom .arrow { + top: -11px; + left: 50%; + margin-left: -11px; + border-bottom-color: #999; + border-bottom-color: rgba(0, 0, 0, 0.25); + border-top-width: 0; +} + +.popover.bottom .arrow:after { + top: 1px; + margin-left: -10px; + border-bottom-color: #ffffff; + border-top-width: 0; +} + +.popover.left .arrow { + top: 50%; + right: -11px; + margin-top: -11px; + border-left-color: #999; + border-left-color: rgba(0, 0, 0, 0.25); + border-right-width: 0; +} + +.popover.left .arrow:after { + right: 1px; + bottom: -10px; + border-left-color: #ffffff; + border-right-width: 0; +} + +.thumbnails { + margin-left: -20px; + list-style: none; + *zoom: 1; +} + +.thumbnails:before, +.thumbnails:after { + display: table; + line-height: 0; + content: ""; +} + +.thumbnails:after { + clear: both; +} + +.row-fluid .thumbnails { + margin-left: 0; +} + +.thumbnails > li { + float: left; + margin-bottom: 20px; + margin-left: 20px; +} + +.thumbnail { + display: block; + padding: 4px; + line-height: 20px; + border: 1px solid #ddd; + -webkit-border-radius: 4px; + -moz-border-radius: 4px; + border-radius: 4px; + -webkit-box-shadow: 0 1px 3px rgba(0, 0, 0, 0.055); + -moz-box-shadow: 0 1px 3px rgba(0, 0, 0, 0.055); + box-shadow: 0 1px 3px rgba(0, 0, 0, 0.055); + -webkit-transition: all 0.2s ease-in-out; + -moz-transition: all 0.2s ease-in-out; + -o-transition: all 0.2s ease-in-out; + transition: all 0.2s ease-in-out; +} + +a.thumbnail:hover, +a.thumbnail:focus { + border-color: #0088cc; + -webkit-box-shadow: 0 1px 4px rgba(0, 105, 214, 0.25); + -moz-box-shadow: 0 1px 4px rgba(0, 105, 214, 0.25); + box-shadow: 0 1px 4px rgba(0, 105, 214, 0.25); +} + +.thumbnail > img { + display: block; + max-width: 100%; + margin-right: auto; + margin-left: auto; +} + +.thumbnail .caption { + padding: 9px; + color: #555555; +} + +.media, +.media-body { + overflow: hidden; + *overflow: visible; + zoom: 1; +} + +.media, +.media .media { + margin-top: 15px; +} + +.media:first-child { + margin-top: 0; +} + +.media-object { + display: block; +} + +.media-heading { + margin: 0 0 5px; +} + +.media > .pull-left { + margin-right: 10px; +} + +.media > .pull-right { + margin-left: 10px; +} + +.media-list { + margin-left: 0; + list-style: none; +} + +.label, +.badge { + display: inline-block; + padding: 2px 4px; + font-size: 11.844px; + font-weight: bold; + line-height: 14px; + color: #ffffff; + text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25); + white-space: nowrap; + vertical-align: baseline; + background-color: #999999; +} + +.label { + -webkit-border-radius: 3px; + -moz-border-radius: 3px; + border-radius: 3px; +} + +.badge { + padding-right: 9px; + padding-left: 9px; + -webkit-border-radius: 9px; + -moz-border-radius: 9px; + border-radius: 9px; +} + +.label:empty, +.badge:empty { + display: none; +} + +a.label:hover, +a.label:focus, +a.badge:hover, +a.badge:focus { + color: #ffffff; + text-decoration: none; + cursor: pointer; +} + +.label-important, +.badge-important { + background-color: #b94a48; +} + +.label-important[href], +.badge-important[href] { + background-color: #953b39; +} + +.label-warning, +.badge-warning { + background-color: #f89406; +} + +.label-warning[href], +.badge-warning[href] { + background-color: #c67605; +} + +.label-success, +.badge-success { + background-color: #468847; +} + +.label-success[href], +.badge-success[href] { + background-color: #356635; +} + +.label-info, +.badge-info { + background-color: #3a87ad; +} + +.label-info[href], +.badge-info[href] { + background-color: #2d6987; +} + +.label-inverse, +.badge-inverse { + background-color: #333333; +} + +.label-inverse[href], +.badge-inverse[href] { + background-color: #1a1a1a; +} + +.btn .label, +.btn .badge { + position: relative; + top: -1px; +} + +.btn-mini .label, +.btn-mini .badge { + top: 0; +} + +@-webkit-keyframes progress-bar-stripes { + from { + background-position: 40px 0; + } + to { + background-position: 0 0; + } +} + +@-moz-keyframes progress-bar-stripes { + from { + background-position: 40px 0; + } + to { + background-position: 0 0; + } +} + +@-ms-keyframes progress-bar-stripes { + from { + background-position: 40px 0; + } + to { + background-position: 0 0; + } +} + +@-o-keyframes progress-bar-stripes { + from { + background-position: 0 0; + } + to { + background-position: 40px 0; + } +} + +@keyframes progress-bar-stripes { + from { + background-position: 40px 0; + } + to { + background-position: 0 0; + } +} + +.progress { + height: 20px; + margin-bottom: 20px; + overflow: hidden; + background-color: #f7f7f7; + background-image: -moz-linear-gradient(top, #f5f5f5, #f9f9f9); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#f5f5f5), to(#f9f9f9)); + background-image: -webkit-linear-gradient(top, #f5f5f5, #f9f9f9); + background-image: -o-linear-gradient(top, #f5f5f5, #f9f9f9); + background-image: linear-gradient(to bottom, #f5f5f5, #f9f9f9); + background-repeat: repeat-x; + -webkit-border-radius: 4px; + -moz-border-radius: 4px; + border-radius: 4px; + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff5f5f5', endColorstr='#fff9f9f9', GradientType=0); + -webkit-box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1); + -moz-box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1); + box-shadow: inset 0 1px 2px rgba(0, 0, 0, 0.1); +} + +.progress .bar { + float: left; + width: 0; + height: 100%; + font-size: 12px; + color: #ffffff; + text-align: center; + text-shadow: 0 -1px 0 rgba(0, 0, 0, 0.25); + background-color: #0e90d2; + background-image: -moz-linear-gradient(top, #149bdf, #0480be); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#149bdf), to(#0480be)); + background-image: -webkit-linear-gradient(top, #149bdf, #0480be); + background-image: -o-linear-gradient(top, #149bdf, #0480be); + background-image: linear-gradient(to bottom, #149bdf, #0480be); + background-repeat: repeat-x; + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff149bdf', endColorstr='#ff0480be', GradientType=0); + -webkit-box-shadow: inset 0 -1px 0 rgba(0, 0, 0, 0.15); + -moz-box-shadow: inset 0 -1px 0 rgba(0, 0, 0, 0.15); + box-shadow: inset 0 -1px 0 rgba(0, 0, 0, 0.15); + -webkit-box-sizing: border-box; + -moz-box-sizing: border-box; + box-sizing: border-box; + -webkit-transition: width 0.6s ease; + -moz-transition: width 0.6s ease; + -o-transition: width 0.6s ease; + transition: width 0.6s ease; +} + +.progress .bar + .bar { + -webkit-box-shadow: inset 1px 0 0 rgba(0, 0, 0, 0.15), inset 0 -1px 0 rgba(0, 0, 0, 0.15); + -moz-box-shadow: inset 1px 0 0 rgba(0, 0, 0, 0.15), inset 0 -1px 0 rgba(0, 0, 0, 0.15); + box-shadow: inset 1px 0 0 rgba(0, 0, 0, 0.15), inset 0 -1px 0 rgba(0, 0, 0, 0.15); +} + +.progress-striped .bar { + background-color: #149bdf; + background-image: -webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255, 255, 255, 0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255, 255, 255, 0.15)), color-stop(0.75, rgba(255, 255, 255, 0.15)), color-stop(0.75, transparent), to(transparent)); + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); + background-image: -moz-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); + background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); + -webkit-background-size: 40px 40px; + -moz-background-size: 40px 40px; + -o-background-size: 40px 40px; + background-size: 40px 40px; +} + +.progress.active .bar { + -webkit-animation: progress-bar-stripes 2s linear infinite; + -moz-animation: progress-bar-stripes 2s linear infinite; + -ms-animation: progress-bar-stripes 2s linear infinite; + -o-animation: progress-bar-stripes 2s linear infinite; + animation: progress-bar-stripes 2s linear infinite; +} + +.progress-danger .bar, +.progress .bar-danger { + background-color: #dd514c; + background-image: -moz-linear-gradient(top, #ee5f5b, #c43c35); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#ee5f5b), to(#c43c35)); + background-image: -webkit-linear-gradient(top, #ee5f5b, #c43c35); + background-image: -o-linear-gradient(top, #ee5f5b, #c43c35); + background-image: linear-gradient(to bottom, #ee5f5b, #c43c35); + background-repeat: repeat-x; + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffee5f5b', endColorstr='#ffc43c35', GradientType=0); +} + +.progress-danger.progress-striped .bar, +.progress-striped .bar-danger { + background-color: #ee5f5b; + background-image: -webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255, 255, 255, 0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255, 255, 255, 0.15)), color-stop(0.75, rgba(255, 255, 255, 0.15)), color-stop(0.75, transparent), to(transparent)); + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); + background-image: -moz-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); + background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); +} + +.progress-success .bar, +.progress .bar-success { + background-color: #5eb95e; + background-image: -moz-linear-gradient(top, #62c462, #57a957); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#62c462), to(#57a957)); + background-image: -webkit-linear-gradient(top, #62c462, #57a957); + background-image: -o-linear-gradient(top, #62c462, #57a957); + background-image: linear-gradient(to bottom, #62c462, #57a957); + background-repeat: repeat-x; + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff62c462', endColorstr='#ff57a957', GradientType=0); +} + +.progress-success.progress-striped .bar, +.progress-striped .bar-success { + background-color: #62c462; + background-image: -webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255, 255, 255, 0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255, 255, 255, 0.15)), color-stop(0.75, rgba(255, 255, 255, 0.15)), color-stop(0.75, transparent), to(transparent)); + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); + background-image: -moz-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); + background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); +} + +.progress-info .bar, +.progress .bar-info { + background-color: #4bb1cf; + background-image: -moz-linear-gradient(top, #5bc0de, #339bb9); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#5bc0de), to(#339bb9)); + background-image: -webkit-linear-gradient(top, #5bc0de, #339bb9); + background-image: -o-linear-gradient(top, #5bc0de, #339bb9); + background-image: linear-gradient(to bottom, #5bc0de, #339bb9); + background-repeat: repeat-x; + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de', endColorstr='#ff339bb9', GradientType=0); +} + +.progress-info.progress-striped .bar, +.progress-striped .bar-info { + background-color: #5bc0de; + background-image: -webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255, 255, 255, 0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255, 255, 255, 0.15)), color-stop(0.75, rgba(255, 255, 255, 0.15)), color-stop(0.75, transparent), to(transparent)); + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); + background-image: -moz-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); + background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); +} + +.progress-warning .bar, +.progress .bar-warning { + background-color: #faa732; + background-image: -moz-linear-gradient(top, #fbb450, #f89406); + background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#fbb450), to(#f89406)); + background-image: -webkit-linear-gradient(top, #fbb450, #f89406); + background-image: -o-linear-gradient(top, #fbb450, #f89406); + background-image: linear-gradient(to bottom, #fbb450, #f89406); + background-repeat: repeat-x; + filter: progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffbb450', endColorstr='#fff89406', GradientType=0); +} + +.progress-warning.progress-striped .bar, +.progress-striped .bar-warning { + background-color: #fbb450; + background-image: -webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255, 255, 255, 0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255, 255, 255, 0.15)), color-stop(0.75, rgba(255, 255, 255, 0.15)), color-stop(0.75, transparent), to(transparent)); + background-image: -webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); + background-image: -moz-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); + background-image: -o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); + background-image: linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent); +} + +.accordion { + margin-bottom: 20px; +} + +.accordion-group { + margin-bottom: 2px; + border: 1px solid #e5e5e5; + -webkit-border-radius: 4px; + -moz-border-radius: 4px; + border-radius: 4px; +} + +.accordion-heading { + border-bottom: 0; +} + +.accordion-heading .accordion-toggle { + display: block; + padding: 8px 15px; +} + +.accordion-toggle { + cursor: pointer; +} + +.accordion-inner { + padding: 9px 15px; + border-top: 1px solid #e5e5e5; +} + +.carousel { + position: relative; + margin-bottom: 20px; + line-height: 1; +} + +.carousel-inner { + position: relative; + width: 100%; + overflow: hidden; +} + +.carousel-inner > .item { + position: relative; + display: none; + -webkit-transition: 0.6s ease-in-out left; + -moz-transition: 0.6s ease-in-out left; + -o-transition: 0.6s ease-in-out left; + transition: 0.6s ease-in-out left; +} + +.carousel-inner > .item > img, +.carousel-inner > .item > a > img { + display: block; + line-height: 1; +} + +.carousel-inner > .active, +.carousel-inner > .next, +.carousel-inner > .prev { + display: block; +} + +.carousel-inner > .active { + left: 0; +} + +.carousel-inner > .next, +.carousel-inner > .prev { + position: absolute; + top: 0; + width: 100%; +} + +.carousel-inner > .next { + left: 100%; +} + +.carousel-inner > .prev { + left: -100%; +} + +.carousel-inner > .next.left, +.carousel-inner > .prev.right { + left: 0; +} + +.carousel-inner > .active.left { + left: -100%; +} + +.carousel-inner > .active.right { + left: 100%; +} + +.carousel-control { + position: absolute; + top: 40%; + left: 15px; + width: 40px; + height: 40px; + margin-top: -20px; + font-size: 60px; + font-weight: 100; + line-height: 30px; + color: #ffffff; + text-align: center; + background: #222222; + border: 3px solid #ffffff; + -webkit-border-radius: 23px; + -moz-border-radius: 23px; + border-radius: 23px; + opacity: 0.5; + filter: alpha(opacity=50); +} + +.carousel-control.right { + right: 15px; + left: auto; +} + +.carousel-control:hover, +.carousel-control:focus { + color: #ffffff; + text-decoration: none; + opacity: 0.9; + filter: alpha(opacity=90); +} + +.carousel-indicators { + position: absolute; + top: 15px; + right: 15px; + z-index: 5; + margin: 0; + list-style: none; +} + +.carousel-indicators li { + display: block; + float: left; + width: 10px; + height: 10px; + margin-left: 5px; + text-indent: -999px; + background-color: #ccc; + background-color: rgba(255, 255, 255, 0.25); + border-radius: 5px; +} + +.carousel-indicators .active { + background-color: #fff; +} + +.carousel-caption { + position: absolute; + right: 0; + bottom: 0; + left: 0; + padding: 15px; + background: #333333; + background: rgba(0, 0, 0, 0.75); +} + +.carousel-caption h4, +.carousel-caption p { + line-height: 20px; + color: #ffffff; +} + +.carousel-caption h4 { + margin: 0 0 5px; +} + +.carousel-caption p { + margin-bottom: 0; +} + +.hero-unit { + padding: 60px; + margin-bottom: 30px; + font-size: 18px; + font-weight: 200; + line-height: 30px; + color: inherit; + background-color: #eeeeee; + -webkit-border-radius: 6px; + -moz-border-radius: 6px; + border-radius: 6px; +} + +.hero-unit h1 { + margin-bottom: 0; + font-size: 60px; + line-height: 1; + letter-spacing: -1px; + color: inherit; +} + +.hero-unit li { + line-height: 30px; +} + +.pull-right { + float: right; +} + +.pull-left { + float: left; +} + +.hide { + display: none; +} + +.show { + display: block; +} + +.invisible { + visibility: hidden; +} + +.affix { + position: fixed; +} diff --git a/dashboard/app/styles/etcd-widgets.css b/dashboard/app/styles/etcd-widgets.css new file mode 100644 index 00000000000..1cccf39f338 --- /dev/null +++ b/dashboard/app/styles/etcd-widgets.css @@ -0,0 +1,694 @@ +body { + margin: 0px; +} +.etcd-container { + background-color: #fff; + border: 1px solid #ddd; + border-radius: 5px; + box-shadow: rgba(0, 0, 0, 0.14902) 0px 1px 3px; + font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; + overflow: hidden; + box-sizing: border-box; + -moz-box-sizing: border-box; + position: relative; +} + + a { + color: #2176AC; + text-decoration: none; + } + + a:hover, a:active { + text-decoration: underline; + } + + input[type=text] { + box-shadow: inset 0 1px 2px rgba(0,0,0,.5); + border: none; + border-radius: 3px; + font-size: 13px; + padding-left: 5px; + padding-right: 5px; + height: 25px; + } + + input[type=text]:focus { + + } + + h2 { + font-size: 22px; + font-family: "Source Sans Pro", "Helvetica Neue", Helvetica, Arial, sans-serif; + font-weight: 500; + margin: 0 0 20px 0; + padding: 0; + } + + .etcd-button { + display:inline-block; + padding:6px 12px; + margin-bottom:0; + font-size:14px; + font-weight:normal; + line-height:1.428571429; + text-align:center; + white-space:nowrap; + vertical-align:middle; + cursor:pointer; + border:1px solid transparent; + border-radius:4px; + -webkit-user-select:none; + -moz-user-select:none; + -ms-user-select:none; + -o-user-select:none; + user-select:none; + margin: 0px; + border: none; + box-shadow: inset 0 1px 0 rgba(255,255,255,0.2),0 1px 2px rgba(0,0,0,0.25); + } + + .etcd-button.etcd-button-small { + height: 25px; + padding: 0 10px; + font-size: 13px; + } + + .etcd-button-primary { + background-color: #428BCA; + color: #fff; + text-shadow: 0 0 3px rgba(0,0,0,0.25); + } + + .etcd-button-primary:active { + background-color: #2276ad; + } + + .etcd-popover { + background: #333; + border-radius: 3px; + padding: 15px; + position: absolute; + top: 39px; + z-index: 9999; + color: #fff; + font-size: 13px; + box-shadow: 0px 2px 10px rgba(0,0,0,.5); + display: none; + } + + .etcd-popover-error .etcd-popover-content { + color: #FF3C43; + font-weight: bold; + } + + .etcd-popover-notch { + width: 14px; + height: 14px; + -webkit-transform: rotate(45deg); + -moz-transform: rotate(45deg); + -ms-transform: rotate(45deg); + position: absolute; + margin-top: -5px; + margin-left: 3px; + background: #333; + top: 0px; + right: 15px; + } + + .etcd-popover.etcd-popover-right { + left: 77px; + } + + .etcd-popover-right .etcd-popover-notch { + left: 15px; + } + + .etcd-popover.etcd-popover-left { + right: 10px; + } + + .etcd-popover-left .etcd-popover-notch { + right: 15px; + } + + .etcd-popover-confirm { + margin-top: 10px; + } + + .etcd-popover-confirm button { + + } + + .etcd-header { + width: 100%; + position: relative; + box-sizing: border-box; + -moz-box-sizing: border-box; + } + .etcd-header.solid { + background: #eeeeee; + background: -moz-linear-gradient(top, #eeeeee 0%, #dddddd 100%); + background: -webkit-gradient(linear, left top, left bottom, color-stop(0%,#eeeeee), color-stop(100%,#dddddd)); + background: -webkit-linear-gradient(top, #eeeeee 0%,#dddddd 100%); + background: -o-linear-gradient(top, #eeeeee 0%,#dddddd 100%); + background: -ms-linear-gradient(top, #eeeeee 0%,#dddddd 100%); + background: linear-gradient(to bottom, #eeeeee 0%,#dddddd 100%); + filter: progid:DXImageTransform.Microsoft.gradient( startColorstr='#eeeeee', endColorstr='#dddddd',GradientType=0 ); + } + + .etcd-body { + top: 0px; + left: 0px; + position: relative; + overflow-y: auto; + overflow-x: hidden; + height: 100%; + width: 100%; + box-sizing: border-box; + -moz-box-sizing: border-box; + } + + .etcd-body table { + width: 100%; + box-sizing: border-box; + -moz-box-sizing: border-box; + } + + .etcd-body table thead td { + text-transform: uppercase; + font-size: 11px; + line-height: 20px; + border-bottom: 1px solid #ddd; + padding-top: 0px; + padding-right: 10px; + padding-bottom: 0px; + padding-left: 0px; + color: #666; + } + + .etcd-body table tbody td { + line-height: 18px; + border-bottom: 1px solid #ddd; + padding-top: 6px; + padding-right: 10px; + padding-bottom: 6px; + padding-left: 0px; + vertical-align: text-top; + } + + .etcd-body table .etcd-ttl-header { + width: 33%; + } + + .etcd-body table tbody .etcd-ttl { + font-size: 13px; + } + + .etcd-body table tbody .etcd-ttl .etcd-ttl-none { + color: #999; + font-weight: 100; + } + + .etcd-body table .etcd-actions-header { + width: 30px; + } + + .etcd-body table thead td:first-child, .etcd-body table tbody td:first-child { + padding-left: 10px; + } + + .etcd-body table thead td:last-child, .etcd-body table tbody td:last-child { + padding-right: 10px; + } + + .etcd-container .etcd-preview .etcd-dialog { + background: #333; + position: absolute; + right: 0px; + left: 0px; + padding: 20px; + color: #fff; + font-size: 14px; + font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; + bottom: 0px; + opacity: 0; + min-height: 110px; /* REMOVE ME! */ + transition-property: all; + transition-duration: 150ms; + transition-timing-function: ease-in-out; + } + + .etcd-container .etcd-preview .etcd-dialog .etcd-dialog-message { + margin-bottom: 20px; + } + + .etcd-container .etcd-preview .etcd-dialog .etcd-dialog-buttons a { + line-height: 34px; + color: #fff; + vertical-align: middle; + margin-left: 10px; + } + + /*.etcd-container .etcd-preview .etcd-dialog.etcd-reveal { + opacity: 1; + } + + .etcd-container .etcd-preview .etcd-dialog.etcd-hide { + opacity: 0; + }*/ + + .etcd-body .etcd-list { + padding: 20px; + box-sizing: border-box; + -moz-box-sizing: border-box; + overflow: auto; + height: 100%; + position: absolute; + } + + .etcd-body .etcd-list .etcd-selected { + background-color: #EAF3FF; + } + + .etcd-body .etcd-list a.directory { + font-weight: bold; + } + + .etcd-body .etcd-list tr:hover .etcd-delete svg { + 1visibility: visible; + fill: #ff0000; + } + + .etcd-body .etcd-list .etcd-delete { + height: 20px; + width: 25px; + vertical-align: middle; + margin: 0px; + display: inline-block; + } + + .etcd-body .etcd-list .etcd-delete svg { + height: 20px; + fill: #eee; + } + + .etcd-body .etcd-list .etcd-selected .etcd-delete svg { + height: 20px; + fill: #ddd; + } + + .etcd-body .etcd-list .etcd-delete:hover svg { + cursor: pointer; + fill: #ff0000; + } + + +.etcd-container.etcd-browser { + +} + + .etcd-container.etcd-browser .etcd-header { + height: 37px; + } + + .etcd-back { + height: 37px; + width: 37px; + vertical-align: middle; + margin: 0px; + position: absolute; + top: 0px; + left: 3px; + display: none; + } + + .etcd-container.etcd-browser.etcd-preview-reveal .etcd-back { + display: block; + } + + .etcd-container.etcd-browser.etcd-preview-hide .etcd-back { + display: block; + } + + .etcd-back svg { + height: 20px; + padding: 8px 6px; + } + + .etcd-back:hover svg { + cursor: pointer; + fill: #428bca; + } + + .etcd-back.etcd-disabled svg { + fill: #bbb; + } + + .etcd-add { + height: 37px; + width: 37px; + vertical-align: middle; + margin: 0px; + position: absolute; + top: 0px; + left: 36px; + } + + .etcd-container.etcd-browser.etcd-preview-reveal .etcd-add { + + } + + .etcd-container.etcd-browser.etcd-preview-hide .etcd-add { + + } + + .etcd-add svg { + height: 22px; + padding: 7px 6px; + } + + .etcd-add:hover svg { + cursor: pointer; + fill: #428bca; + } + + .etcd-add.etcd-disabled svg { + fill: #bbb; + } + + .etcd-container.etcd-browser .etcd-header .etcd-browser-path { + position: absolute; + left: 72px; + right: 0px; + top: 0; + margin: 6px 5px 6px 5px; + } + + .etcd-container.etcd-browser .etcd-header .etcd-browser-path input { + width: 100%; + box-sizing: border-box; + -moz-box-sizing: border-box; + -webkit-box-sizing: border-box; + } + + .etcd-container.etcd-browser .etcd-header .etcd-save { + position: absolute; + width: 54px; + right: -55px; + margin: 6px 0; + } + + .etcd-container.etcd-browser.etcd-save-reveal .etcd-header .etcd-save { + right: 7px; + } + + .etcd-container.etcd-browser.etcd-save-reveal .etcd-header .etcd-browser-path { + right: 62px; + } + + .etcd-container.etcd-browser.etcd-save-hide .etcd-header .etcd-save { + right: -55px; + } + + .etcd-container.etcd-browser.etcd-save-hide .etcd-header .etcd-browser-path { + right: 0px; + } + + .etcd-container.etcd-browser .etcd-preview { + position: absolute; + left: 100%; + min-height: 100%; + overflow-y: auto; + overflow-x: hidden; + top: 0px; + box-sizing: border-box; + -moz-box-sizing: border-box; + background-color: #fff; + width: 100%; + border-left: 1px solid #ddd; + } + + .etcd-container.etcd-browser .etcd-preview pre, .etcd-container.etcd-browser .etcd-preview textarea { + padding: 20px 20px 20px 20px; + margin: 0px; + font-family: Consolas, "Liberation Mono", Courier, monospace; + height: 100%; + width: 100%; + white-space: pre-wrap; + position: absolute; + font-size: 13px; + border: 1px; + outline: none; + box-sizing: border-box; + -moz-box-sizing: border-box; + } + + .etcd-container.etcd-browser.etcd-preview-reveal .etcd-preview pre, .etcd-container.etcd-browser.etcd-preview-reveal .etcd-preview textarea { + display: block; + } + + .etcd-container.etcd-browser .etcd-preview .etcd-empty { + top: 0px; + bottom: 0px; + width: 100%; + text-align: center; + position: absolute; + } + + .etcd-container.etcd-browser.etcd-preview-reveal .etcd-empty { + display: none; + } + + .etcd-container.etcd-browser .etcd-preview .etcd-empty-message { + margin-top: 25%; + color: #999; + } + + /* Single Column Positioning */ + @media (max-width: 700px) { + .etcd-container.etcd-browser .etcd-list { + width: 100%; + } + + .etcd-container.etcd-browser.etcd-preview-reveal .etcd-list { + left: -100%; + transition-property: all; + transition-duration: 250ms; + transition-timing-function: ease-in-out; + } + + .etcd-container.etcd-browser.etcd-preview-hide .etcd-list { + left: 0%; + transition-property: all; + transition-duration: 250ms; + transition-timing-function: ease-in-out; + } + + .etcd-container.etcd-browser .etcd-preview { + left: 100%; + } + + .etcd-container.etcd-browser.etcd-preview-reveal .etcd-preview { + left: -1px; + transition-property: all; + transition-duration: 250ms; + transition-timing-function: ease-in-out; + } + + .etcd-container.etcd-browser.etcd-preview-hide .etcd-preview { + left: 100%; + transition-property: all; + transition-duration: 250ms; + transition-timing-function: ease-in-out; + } + } + + + /* Double Column Positioning */ + @media (min-width: 700px) { + .etcd-container.etcd-browser .etcd-list { + width: 50%; + } + + .etcd-container.etcd-browser .etcd-preview { + left: 50%; + width: 50%; + } + + .etcd-container.etcd-browser.etcd-preview-reveal .etcd-preview { + left: 50%; /* does nothing */ + } + + .etcd-container.etcd-browser.etcd-preview-reveal .etcd-preview .etcd-empty { + display: none; + } + + .etcd-container.etcd-browser.etcd-preview-hide .etcd-preview { + left: 50%; /* does nothing */ + } + + .etcd-container.etcd-browser.etcd-preview-hide .etcd-preview .etcd-empty { + display: block; + } + + .etcd-container.etcd-browser.etcd-preview-hide .etcd-preview pre, .etcd-container.etcd-browser.etcd-preview-hide .etcd-preview textarea { + display: none; + } + } + +.etcd-container.etcd-stats { + +} + + .etcd-container.etcd-stats h2 { + margin-top: -7px; + } + + .etcd-format-selector { + position: absolute; + top: 12px; + right: 16px; + z-index: 999; + } + + .etcd-format-selector .etcd-selector-item { + display: inline-block; + height: 12px; + width: 12px; + padding: 8px 4px; + } + + .etcd-format-selector .etcd-selector-item:hover { + cursor: pointer; + } + + .etcd-format-selector .etcd-selector-item svg { + fill: #333; + } + + .etcd-container.etcd-stats .etcd-graph { + box-sizing: border-box; + -moz-box-sizing: border-box; + position: absolute; + top: 0px; + bottom: 0px; + left: 0px; + right: 0px; + padding: 20px; + } + + .etcd-container.etcd-stats .etcd-graph .etcd-graph-container { + position: absolute; + top: 60px; + bottom: 20px; + left: 20px; + right: 20px; + box-sizing: border-box; + -moz-box-sizing: border-box; + } + + .etcd-container.etcd-stats table .etcd-latency { + width: 50%; + } + + .etcd-container.etcd-stats .etcd-list { + position: absolute; + left: 100%; + min-height: 100%; + overflow-y: auto; + overflow-x: hidden; + top: 0px; + box-sizing: border-box; + -moz-box-sizing: border-box; + background-color: #fff; + width: 100%; + border-left: 1px solid #ddd; + } + + .etcd-container.etcd-stats .etcd-list .etcd-square { + height: 10px; + width: 10px; + display: inline-block; + margin-right: 5px; + } + + .etcd-container.etcd-stats .etcd-list .etcd-square-red { + background-color: #c40022; + } + + .etcd-container.etcd-stats .etcd-list .etcd-square-orange { + background-color: #FFC000; + } + + .etcd-container.etcd-stats .etcd-list .etcd-square-green { + background-color: #00DB24; + } + + .etcd-container.etcd-stats .etcd-list .etcd-latency-value { + display: inline-block; + } + + /* Single Column Positioning */ + @media (max-width: 700px) { + .etcd-container.etcd-stats .etcd-list { + width: 100%; + left: 100%; + } + + .etcd-container.etcd-stats .etcd-graph { + left: 0%; + } + + .etcd-container.etcd-stats.etcd-table-reveal .etcd-graph { + left: -100%; + transition-property: all; + transition-duration: 250ms; + transition-timing-function: ease-in-out; + } + .etcd-container.etcd-stats.etcd-table-hide .etcd-graph { + left: 0%; + transition-property: all; + transition-duration: 250ms; + transition-timing-function: ease-in-out; + + } + .etcd-container.etcd-stats.etcd-table-hide .etcd-format-selector .etcd-selector-graph svg * { + fill: #428bca; + } + + .etcd-container.etcd-stats.etcd-table-hide .etcd-list { + left: 100%; + transition-property: all; + transition-duration: 250ms; + transition-timing-function: ease-in-out; + } + .etcd-container.etcd-stats.etcd-table-reveal .etcd-list { + left: 0%; + transition-property: all; + transition-duration: 250ms; + transition-timing-function: ease-in-out; + } + .etcd-container.etcd-stats.etcd-table-reveal .etcd-format-selector .etcd-selector-table svg * { + fill: #428bca; + } + + } + + + /* Double Column Positioning */ + @media (min-width: 700px) { + .etcd-container.etcd-stats .etcd-list { + width: 50%; + left: 50%; + } + + .etcd-container.etcd-stats .etcd-graph { + left: 0%; + width: 50%; + } + + .etcd-container.etcd-stats .etcd-format-selector { + display: none; + } + + } diff --git a/dashboard/app/styles/main.css b/dashboard/app/styles/main.css new file mode 100644 index 00000000000..c754fddc39b --- /dev/null +++ b/dashboard/app/styles/main.css @@ -0,0 +1,22 @@ +body { + background: #fafafa; + font-family: "Helvetica Neue", Helvetica, Arial, sans-serif; + color: #333; +} + +.hero-unit { + margin: 50px auto 0 auto; + width: 300px; + font-size: 18px; + font-weight: 200; + line-height: 30px; + background-color: #eee; + border-radius: 6px; + padding: 60px; +} + +.hero-unit h1 { + font-size: 60px; + line-height: 1; + letter-spacing: -1px; +} diff --git a/dashboard/app/views/browser.html b/dashboard/app/views/browser.html new file mode 100644 index 00000000000..8a2504cf890 --- /dev/null +++ b/dashboard/app/views/browser.html @@ -0,0 +1,99 @@ +
+ +
+
+
+ Error: +
+
+
+
+
+ Error: +
+
+
+ + + + + + + + + + + + +
+ +
+ +
+
+
+ + + + + + + + + + + + + +
NameTTL 
{{key.key}} +
+
+
+
+
+ + + + +
+
+
+
+
+
+ +
+
{{preview_message}}
+
+
+
+ Save and replicate this change? +
+
+ + Cancel +
+
+
+
+
diff --git a/dashboard/app/views/stats.html b/dashboard/app/views/stats.html new file mode 100644 index 00000000000..eb1a615477a --- /dev/null +++ b/dashboard/app/views/stats.html @@ -0,0 +1,46 @@ +
+
+
+
+ + + + +
+
+ + + + + +
+
+
+

Follower Latency

+
+
+
+
+

Follower List

+ + + + + + + + + + + +
Machine NameLatency
{{follower.name}} +
+
{{follower.latency.current | number:1 }} ms
+
+
+
+
diff --git a/dashboard/bower.json b/dashboard/bower.json new file mode 100644 index 00000000000..51ed4e5adce --- /dev/null +++ b/dashboard/bower.json @@ -0,0 +1,20 @@ +{ + "name": "etcdDashboard", + "version": "0.0.0", + "dependencies": { + "angular": "~1.2.0-rc.2", + "json3": "~3.2.4", + "jquery": "~1.9.1", + "bootstrap-sass": "~2.3.1", + "es5-shim": "~2.0.8", + "angular-route": "~1.2.0-rc.2", + "angular-resource": "~1.2.0-rc.2", + "angular-cookies": "~1.2.0-rc.2", + "angular-sanitize": "~1.2.0-rc.2" + }, + "devDependencies": { + "angular-mocks": "~1.2.0-rc.2", + "angular-scenario": "~1.2.0-rc.2", + "underscore": "~1.5.2" + } +} diff --git a/dashboard/build b/dashboard/build new file mode 100755 index 00000000000..60d6fcdcb2c --- /dev/null +++ b/dashboard/build @@ -0,0 +1,10 @@ +#!/bin/sh + +grunt build + +git clean -x -f dashboard/dist + +for i in `find dashboard/dist -type f`; do + go build github.com/jteeuwen/go-bindata + ./go-bindata -pkg "dist" -toc -prefix dashboard/dist $i +done diff --git a/dashboard/karma-e2e.conf.js b/dashboard/karma-e2e.conf.js new file mode 100644 index 00000000000..fa01484a0a9 --- /dev/null +++ b/dashboard/karma-e2e.conf.js @@ -0,0 +1,54 @@ +// Karma configuration +// http://karma-runner.github.io/0.10/config/configuration-file.html + +module.exports = function(config) { + config.set({ + // base path, that will be used to resolve files and exclude + basePath: '', + + // testing framework to use (jasmine/mocha/qunit/...) + frameworks: ['ng-scenario'], + + // list of files / patterns to load in the browser + files: [ + 'test/e2e/**/*.js' + ], + + // list of files / patterns to exclude + exclude: [], + + // web server port + port: 8080, + + // level of logging + // possible values: LOG_DISABLE || LOG_ERROR || LOG_WARN || LOG_INFO || LOG_DEBUG + logLevel: config.LOG_INFO, + + + // enable / disable watching file and executing tests whenever any file changes + autoWatch: false, + + + // Start these browsers, currently available: + // - Chrome + // - ChromeCanary + // - Firefox + // - Opera + // - Safari (only Mac) + // - PhantomJS + // - IE (only Windows) + browsers: ['Chrome'], + + + // Continuous Integration mode + // if true, it capture browsers, run tests and exit + singleRun: false + + // Uncomment the following lines if you are using grunt's server to run the tests + // proxies: { + // '/': 'http://localhost:9000/' + // }, + // URL root prevent conflicts with the site root + // urlRoot: '_karma_' + }); +}; diff --git a/dashboard/karma.conf.js b/dashboard/karma.conf.js new file mode 100644 index 00000000000..fae04e3aaae --- /dev/null +++ b/dashboard/karma.conf.js @@ -0,0 +1,52 @@ +// Karma configuration +// http://karma-runner.github.io/0.10/config/configuration-file.html + +module.exports = function(config) { + config.set({ + // base path, that will be used to resolve files and exclude + basePath: '', + + // testing framework to use (jasmine/mocha/qunit/...) + frameworks: ['jasmine'], + + // list of files / patterns to load in the browser + files: [ + 'app/bower_components/angular/angular.js', + 'app/bower_components/angular-mocks/angular-mocks.js', + 'app/scripts/*.js', + 'app/scripts/**/*.js', + 'test/mock/**/*.js', + 'test/spec/**/*.js' + ], + + // list of files / patterns to exclude + exclude: [], + + // web server port + port: 8080, + + // level of logging + // possible values: LOG_DISABLE || LOG_ERROR || LOG_WARN || LOG_INFO || LOG_DEBUG + logLevel: config.LOG_INFO, + + + // enable / disable watching file and executing tests whenever any file changes + autoWatch: false, + + + // Start these browsers, currently available: + // - Chrome + // - ChromeCanary + // - Firefox + // - Opera + // - Safari (only Mac) + // - PhantomJS + // - IE (only Windows) + browsers: ['Chrome'], + + + // Continuous Integration mode + // if true, it capture browsers, run tests and exit + singleRun: false + }); +}; diff --git a/dashboard/package.json b/dashboard/package.json new file mode 100644 index 00000000000..84d59176686 --- /dev/null +++ b/dashboard/package.json @@ -0,0 +1,38 @@ +{ + "name": "etcd-dashboard", + "version": "0.0.0", + "dependencies": {}, + "devDependencies": { + "grunt": "~0.4.1", + "grunt-contrib-copy": "~0.4.1", + "grunt-contrib-concat": "~0.3.0", + "grunt-contrib-coffee": "~0.7.0", + "grunt-contrib-uglify": "~0.2.0", + "grunt-contrib-compass": "~0.5.0", + "grunt-contrib-jshint": "~0.6.0", + "grunt-contrib-cssmin": "~0.6.0", + "grunt-contrib-connect": "~0.3.0", + "grunt-contrib-clean": "~0.5.0", + "grunt-contrib-htmlmin": "~0.1.3", + "grunt-contrib-imagemin": "~0.2.0", + "grunt-contrib-watch": "~0.5.2", + "grunt-autoprefixer": "~0.2.0", + "grunt-usemin": "~0.1.11", + "grunt-svgmin": "~0.2.0", + "grunt-rev": "~0.1.0", + "grunt-open": "~0.2.0", + "grunt-concurrent": "~0.3.0", + "load-grunt-tasks": "~0.1.0", + "connect-livereload": "~0.2.0", + "grunt-google-cdn": "~0.2.0", + "grunt-ngmin": "~0.0.2", + "time-grunt": "~0.1.0", + "grunt-karma": "~0.6.2" + }, + "engines": { + "node": ">=0.8.0" + }, + "scripts": { + "test": "grunt test" + } +} diff --git a/dashboard/test/.jshintrc b/dashboard/test/.jshintrc new file mode 100644 index 00000000000..aa37e7a4d09 --- /dev/null +++ b/dashboard/test/.jshintrc @@ -0,0 +1,35 @@ +{ + "node": true, + "browser": true, + "esnext": true, + "bitwise": true, + "camelcase": true, + "curly": true, + "eqeqeq": true, + "immed": true, + "indent": 2, + "latedef": true, + "newcap": true, + "noarg": true, + "quotmark": "single", + "regexp": true, + "undef": true, + "unused": true, + "strict": true, + "trailing": true, + "smarttabs": true, + "globals": { + "after": false, + "afterEach": false, + "angular": false, + "before": false, + "beforeEach": false, + "browser": false, + "describe": false, + "expect": false, + "inject": false, + "it": false, + "spyOn": false + } +} + diff --git a/dashboard/test/runner.html b/dashboard/test/runner.html new file mode 100644 index 00000000000..f4a00a12b0a --- /dev/null +++ b/dashboard/test/runner.html @@ -0,0 +1,10 @@ + + + + End2end Test Runner + + + + + + \ No newline at end of file diff --git a/dashboard/test/spec/controllers/main.js b/dashboard/test/spec/controllers/main.js new file mode 100644 index 00000000000..ef4dea5e5b9 --- /dev/null +++ b/dashboard/test/spec/controllers/main.js @@ -0,0 +1,22 @@ +'use strict'; + +describe('Controller: MainCtrl', function () { + + // load the controller's module + beforeEach(module('etcdDashboardApp')); + + var MainCtrl, + scope; + + // Initialize the controller and a mock scope + beforeEach(inject(function ($controller, $rootScope) { + scope = $rootScope.$new(); + MainCtrl = $controller('MainCtrl', { + $scope: scope + }); + })); + + it('should attach a list of awesomeThings to the scope', function () { + expect(scope.awesomeThings.length).toBe(3); + }); +}); From b2d72b3e552c38f0f55c9f3c2368813a12254024 Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Mon, 7 Oct 2013 16:49:50 -0700 Subject: [PATCH 194/247] bump(github.com/jteeuwen/go-bindata): 79847ab3e91ae5d2e1b18796c1795c78f29565d7 --- .gitignore | 1 + dashboard/.gitignore | 1 + third_party/deps | 1 + .../jteeuwen/go-bindata/CONTRIBUTORS | 10 + .../github.com/jteeuwen/go-bindata/LICENSE | 3 + .../github.com/jteeuwen/go-bindata/README.md | 173 ++ .../jteeuwen/go-bindata/lib/bytewriter.go | 37 + .../jteeuwen/go-bindata/lib/stringwriter.go | 30 + .../github.com/jteeuwen/go-bindata/lib/toc.go | 39 + .../jteeuwen/go-bindata/lib/translate.go | 187 ++ .../jteeuwen/go-bindata/lib/version.go | 31 + .../github.com/jteeuwen/go-bindata/main.go | 157 ++ .../go-bindata/testdata/bindata-toc.go | 7 + .../go-bindata/testdata/gophercolor.png | Bin 0 -> 21899 bytes .../go-bindata/testdata/memcpy-compressed.go | 1849 ++++++++++++++++ .../testdata/memcpy-uncompressed.go | 1832 ++++++++++++++++ .../testdata/nomemcpy-compressed.go | 1403 +++++++++++++ .../testdata/nomemcpy-uncompressed.go | 1392 +++++++++++++ .../jteeuwen/go-bindata/testdata/toc.go | 1853 +++++++++++++++++ 19 files changed, 9006 insertions(+) create mode 100644 third_party/github.com/jteeuwen/go-bindata/CONTRIBUTORS create mode 100644 third_party/github.com/jteeuwen/go-bindata/LICENSE create mode 100644 third_party/github.com/jteeuwen/go-bindata/README.md create mode 100644 third_party/github.com/jteeuwen/go-bindata/lib/bytewriter.go create mode 100644 third_party/github.com/jteeuwen/go-bindata/lib/stringwriter.go create mode 100644 third_party/github.com/jteeuwen/go-bindata/lib/toc.go create mode 100644 third_party/github.com/jteeuwen/go-bindata/lib/translate.go create mode 100644 third_party/github.com/jteeuwen/go-bindata/lib/version.go create mode 100644 third_party/github.com/jteeuwen/go-bindata/main.go create mode 100644 third_party/github.com/jteeuwen/go-bindata/testdata/bindata-toc.go create mode 100644 third_party/github.com/jteeuwen/go-bindata/testdata/gophercolor.png create mode 100644 third_party/github.com/jteeuwen/go-bindata/testdata/memcpy-compressed.go create mode 100644 third_party/github.com/jteeuwen/go-bindata/testdata/memcpy-uncompressed.go create mode 100644 third_party/github.com/jteeuwen/go-bindata/testdata/nomemcpy-compressed.go create mode 100644 third_party/github.com/jteeuwen/go-bindata/testdata/nomemcpy-uncompressed.go create mode 100644 third_party/github.com/jteeuwen/go-bindata/testdata/toc.go diff --git a/.gitignore b/.gitignore index 4d8d3967c37..a080789eff0 100644 --- a/.gitignore +++ b/.gitignore @@ -2,4 +2,5 @@ src/ pkg/ /etcd /server/release_version.go +/go-bindata /machine* diff --git a/dashboard/.gitignore b/dashboard/.gitignore index 7911b28d575..315e280ee42 100644 --- a/dashboard/.gitignore +++ b/dashboard/.gitignore @@ -3,3 +3,4 @@ dist .tmp .sass-cache app/bower_components +/go-bindata diff --git a/third_party/deps b/third_party/deps index 76464bdfc9c..1fa05339c37 100755 --- a/third_party/deps +++ b/third_party/deps @@ -11,4 +11,5 @@ packages=" bitbucket.org/kardianos/osext code.google.com/p/go.net code.google.com/p/goprotobuf + github.com/jteeuwen/go-bindata " diff --git a/third_party/github.com/jteeuwen/go-bindata/CONTRIBUTORS b/third_party/github.com/jteeuwen/go-bindata/CONTRIBUTORS new file mode 100644 index 00000000000..cec3a284e09 --- /dev/null +++ b/third_party/github.com/jteeuwen/go-bindata/CONTRIBUTORS @@ -0,0 +1,10 @@ +# This is the list of people who can contribute (or have contributed) to this +# project. This includes code, documentation, testing, content creation and +# bugfixes. +# +# Names should be added to this file like so: +# Name [] +# +# Please keep the list sorted. + +Jim Teeuwen diff --git a/third_party/github.com/jteeuwen/go-bindata/LICENSE b/third_party/github.com/jteeuwen/go-bindata/LICENSE new file mode 100644 index 00000000000..c07a9311f17 --- /dev/null +++ b/third_party/github.com/jteeuwen/go-bindata/LICENSE @@ -0,0 +1,3 @@ +This work is subject to the CC0 1.0 Universal (CC0 1.0) Public Domain Dedication +license. Its contents can be found at: +http://creativecommons.org/publicdomain/zero/1.0 diff --git a/third_party/github.com/jteeuwen/go-bindata/README.md b/third_party/github.com/jteeuwen/go-bindata/README.md new file mode 100644 index 00000000000..4db2921af1f --- /dev/null +++ b/third_party/github.com/jteeuwen/go-bindata/README.md @@ -0,0 +1,173 @@ +## bindata + +This tool converts any file into managable Go source code. Useful for embedding +binary data into a go program. The file data is optionally gzip compressed +before being converted to a raw byte slice. + +### Usage + +The simplest invocation is to pass it only the input file name. +The output file and code settings are inferred from this automatically. + + $ go-bindata testdata/gophercolor.png + [w] No output file specified. Using 'testdata/gophercolor.png.go'. + [w] No package name specified. Using 'main'. + [w] No function name specified. Using 'testdata_gophercolor_png'. + +This creates the `testdata/gophercolor.png.go` file which has a package +declaration with name `main` and one function named `testdata_gophercolor_png` with +the following signature: + +```go +func testdata_gophercolor_png() []byte +``` + +You can now simply include the new .go file in your program and call +`testdata_gophercolor_png()` to get the (uncompressed) image data. The function panics +if something went wrong during decompression. See the testdata directory for +example input and output files for various modes. + +Aternatively, you can pipe the input file data into stdin. `go-bindata` will +then spit out the generated Go code to stdout. This does require explicitly +naming the desired function name, as it can not be inferred from the +input data. The package name will still default to 'main'. + + $ cat testdata/gophercolor.png | go-bindata -f gophercolor_png | gofmt + +Invoke the program with the `-h` flag for more options. + +In order to strip off a part of the generated function name, we can use the `-prefix` flag. +In the above example, the input file `testdata/gophercolor.png` yields a function named +`testdata_gophercolor_png`. If we want the `testdata` component to be left out, we invoke +the program as follows: + + $ go-bindata -prefix "testdata/" testdata/gophercolor.png + + +### Lower memory footprint + +Using the `-nomemcopy` flag, will alter the way the output file is generated. +It will employ a hack that allows us to read the file data directly from +the compiled program's `.rodata` section. This ensures that when we call +call our generated function, we omit unnecessary memcopies. + +The downside of this, is that it requires dependencies on the `reflect` and +`unsafe` packages. These may be restricted on platforms like AppEngine and +thus prevent you from using this mode. + +Another disadvantage is that the byte slice we create, is strictly read-only. +For most use-cases this is not a problem, but if you ever try to alter the +returned byte slice, a runtime panic is thrown. Use this mode only on target +platforms where memory constraints are an issue. + +The default behaviour is to use the old code generation method. This +prevents the two previously mentioned issues, but will employ at least one +extra memcopy and thus increase memory requirements. + +For instance, consider the following two examples: + +This would be the default mode, using an extra memcopy but gives a safe +implementation without dependencies on `reflect` and `unsafe`: + +```go +func myfile() []byte { + return []byte{0x89, 0x50, 0x4e, 0x47, 0x0d, 0x0a, 0x1a} +} +``` + +Here is the same functionality, but uses the `.rodata` hack. +The byte slice returned from this example can not be written to without +generating a runtime error. + +```go +var _myfile = "\x89\x50\x4e\x47\x0d\x0a\x1a" + +func myfile() []byte { + var empty [0]byte + sx := (*reflect.StringHeader)(unsafe.Pointer(&_myfile)) + b := empty[:] + bx := (*reflect.SliceHeader)(unsafe.Pointer(&b)) + bx.Data = sx.Data + bx.Len = len(_myfile) + bx.Cap = bx.Len + return b +} +``` + + +### Optional compression + +When the `-uncompressed` flag is given, the supplied resource is *not* GZIP compressed +before being turned into Go code. The data should still be accessed through +a function call, so nothing changes in the usage of the generated file. + +This feature is useful if you do not care for compression, or the supplied +resource is already compressed. Doing it again would not add any value and may +even increase the size of the data. + +The default behaviour of the program is to use compression. + + +### Table of Contents + +With the `-toc` flag, we can have `go-bindata` create a table of contents for all the files +which have been generated by the tool. It does this by first generating a new file named +`bindata-toc.go`. This contains a global map of type `map[string] func() []byte`. It uses the +input filename as the key and the data function as the value. We can use this +to fetch all data for our files, matching a given pattern. + +It then appands an `init` function to each generated file, which simply makes the data +function append itself to the global `bindata` map. + +Once you have compiled your program with all these new files and run it, the map will +be populated by all generated data files. + +**Note**: The `bindata-toc.go` file will not be created when we run in `pipe` mode. +The reason being, that the tool does not write any files at all, as it has no idea +where to save them. The data file is written to `stdout` instead after all. + + +#### Table of Contents keys + +The keys used in the `go_bindata` map, are the same as the input file name passed to `go-bindata`. +This includes the fully qualified (absolute) path. In most cases, this is not desireable, as it +puts potentially sensitive information in your code base. For this purpose, the tool supplies +another command line flag `-prefix`. This accepts a portion of a path name, which should be +stripped off from the map keys and function names. + +For example, running without the `-prefix` flag, we get: + + $ go-bindata /path/to/templates/foo.html + + go_bindata["/path/to/templates/foo.html"] = path_to_templates_foo_html + +Running with the `-prefix` flag, we get: + + $ go-bindata -prefix "/path/to/" /path/to/templates/foo.html + + go_bindata["templates/foo.html"] = templates_foo_html + + +#### bindata-toc.go + +The `bindata-toc.go` file is very simple and looks as follows: + +```go +package $PACKAGENAME + +// Global Table of Contents map. Generated by go-bindata. +// After startup of the program, all generated data files will +// put themselves in this map. The key is the full filename, as +// supplied to go-bindata. +var go_bindata = make(map[string] func() []byte) +``` + +#### Build tags + +With the optional -tags flag, you can specify any go build tags that +must be fulfilled for the output file to be included in a build. This +is useful for including binary data in multiple formats, where the desired +format is specified at build time with the appropriate tag(s). + +The tags are appended to a `// +build` line in the beginning of the output file +and must follow the build tags syntax specified by the go tool. diff --git a/third_party/github.com/jteeuwen/go-bindata/lib/bytewriter.go b/third_party/github.com/jteeuwen/go-bindata/lib/bytewriter.go new file mode 100644 index 00000000000..c9b602c70e7 --- /dev/null +++ b/third_party/github.com/jteeuwen/go-bindata/lib/bytewriter.go @@ -0,0 +1,37 @@ +// This work is subject to the CC0 1.0 Universal (CC0 1.0) Public Domain Dedication +// license. Its contents can be found at: +// http://creativecommons.org/publicdomain/zero/1.0/ + +package bindata + +import ( + "fmt" + "io" +) + +var newline = []byte{'\n'} + +type ByteWriter struct { + io.Writer + c int +} + +func (w *ByteWriter) Write(p []byte) (n int, err error) { + if len(p) == 0 { + return + } + + for n = range p { + if w.c%12 == 0 { + w.Writer.Write(newline) + w.c = 0 + } + + fmt.Fprintf(w.Writer, "0x%02x,", p[n]) + w.c++ + } + + n++ + + return +} diff --git a/third_party/github.com/jteeuwen/go-bindata/lib/stringwriter.go b/third_party/github.com/jteeuwen/go-bindata/lib/stringwriter.go new file mode 100644 index 00000000000..278f35b3283 --- /dev/null +++ b/third_party/github.com/jteeuwen/go-bindata/lib/stringwriter.go @@ -0,0 +1,30 @@ +// This work is subject to the CC0 1.0 Universal (CC0 1.0) Public Domain Dedication +// license. Its contents can be found at: +// http://creativecommons.org/publicdomain/zero/1.0/ + +package bindata + +import ( + "fmt" + "io" +) + +type StringWriter struct { + io.Writer + c int +} + +func (w *StringWriter) Write(p []byte) (n int, err error) { + if len(p) == 0 { + return + } + + for n = range p { + fmt.Fprintf(w.Writer, "\\x%02x", p[n]) + w.c++ + } + + n++ + + return +} diff --git a/third_party/github.com/jteeuwen/go-bindata/lib/toc.go b/third_party/github.com/jteeuwen/go-bindata/lib/toc.go new file mode 100644 index 00000000000..a534d93472c --- /dev/null +++ b/third_party/github.com/jteeuwen/go-bindata/lib/toc.go @@ -0,0 +1,39 @@ +// This work is subject to the CC0 1.0 Universal (CC0 1.0) Public Domain Dedication +// license. Its contents can be found at: +// http://creativecommons.org/publicdomain/zero/1.0/ + +package bindata + +import ( + "fmt" + "io" + "io/ioutil" + "path/filepath" + "strings" +) + +// createTOC writes a table of contents file to the given location. +func CreateTOC(dir, pkgname string) error { + file := filepath.Join(dir, "bindata-toc.go") + code := fmt.Sprintf(`package %s + +// Global Table of Contents map. Generated by go-bindata. +// After startup of the program, all generated data files will +// put themselves in this map. The key is the full filename, as +// supplied to go-bindata. +var go_bindata = make(map[string]func() []byte)`, pkgname) + + return ioutil.WriteFile(file, []byte(code), 0600) +} + +// WriteTOCInit writes the TOC init function for a given data file +// replacing the prefix in the filename by "", funcname being the translated function name +func WriteTOCInit(output io.Writer, filename, prefix, funcname string) { + filename = strings.Replace(filename, prefix, "", 1) + fmt.Fprintf(output, ` + +func init() { + go_bindata[%q] = %s +} +`, filename, funcname) +} diff --git a/third_party/github.com/jteeuwen/go-bindata/lib/translate.go b/third_party/github.com/jteeuwen/go-bindata/lib/translate.go new file mode 100644 index 00000000000..4e8cb2f7616 --- /dev/null +++ b/third_party/github.com/jteeuwen/go-bindata/lib/translate.go @@ -0,0 +1,187 @@ +// This work is subject to the CC0 1.0 Universal (CC0 1.0) Public Domain Dedication +// license. Its contents can be found at: +// http://creativecommons.org/publicdomain/zero/1.0/ + +package bindata + +import ( + "compress/gzip" + "fmt" + "io" + "regexp" + "strings" + "unicode" +) + +var regFuncName = regexp.MustCompile(`[^a-zA-Z0-9_]`) + +// translate translates the input file to go source code. +func Translate(input io.Reader, output io.Writer, pkgname, funcname string, uncompressed, nomemcpy bool) { + if nomemcpy { + if uncompressed { + translate_nomemcpy_uncomp(input, output, pkgname, funcname) + } else { + translate_nomemcpy_comp(input, output, pkgname, funcname) + } + } else { + if uncompressed { + translate_memcpy_uncomp(input, output, pkgname, funcname) + } else { + translate_memcpy_comp(input, output, pkgname, funcname) + } + } +} + +// input -> gzip -> gowriter -> output. +func translate_memcpy_comp(input io.Reader, output io.Writer, pkgname, funcname string) { + fmt.Fprintf(output, `package %s + +import ( + "bytes" + "compress/gzip" + "io" +) + +// %s returns raw, uncompressed file data. +func %s() []byte { + gz, err := gzip.NewReader(bytes.NewBuffer([]byte{`, pkgname, funcname, funcname) + + gz := gzip.NewWriter(&ByteWriter{Writer: output}) + io.Copy(gz, input) + gz.Close() + + fmt.Fprint(output, ` + })) + + if err != nil { + panic("Decompression failed: " + err.Error()) + } + + var b bytes.Buffer + io.Copy(&b, gz) + gz.Close() + + return b.Bytes() +}`) +} + +// input -> gzip -> gowriter -> output. +func translate_memcpy_uncomp(input io.Reader, output io.Writer, pkgname, funcname string) { + fmt.Fprintf(output, `package %s + +// %s returns raw file data. +func %s() []byte { + return []byte{`, pkgname, funcname, funcname) + + io.Copy(&ByteWriter{Writer: output}, input) + + fmt.Fprint(output, ` + } +}`) +} + +// input -> gzip -> gowriter -> output. +func translate_nomemcpy_comp(input io.Reader, output io.Writer, pkgname, funcname string) { + fmt.Fprintf(output, `package %s + +import ( + "bytes" + "compress/gzip" + "io" + "reflect" + "unsafe" +) + +var _%s = "`, pkgname, funcname) + + gz := gzip.NewWriter(&StringWriter{Writer: output}) + io.Copy(gz, input) + gz.Close() + + fmt.Fprintf(output, `" + +// %s returns raw, uncompressed file data. +func %s() []byte { + var empty [0]byte + sx := (*reflect.StringHeader)(unsafe.Pointer(&_%s)) + b := empty[:] + bx := (*reflect.SliceHeader)(unsafe.Pointer(&b)) + bx.Data = sx.Data + bx.Len = len(_%s) + bx.Cap = bx.Len + + gz, err := gzip.NewReader(bytes.NewBuffer(b)) + + if err != nil { + panic("Decompression failed: " + err.Error()) + } + + var buf bytes.Buffer + io.Copy(&buf, gz) + gz.Close() + + return buf.Bytes() +} +`, funcname, funcname, funcname, funcname) +} + +// input -> gowriter -> output. +func translate_nomemcpy_uncomp(input io.Reader, output io.Writer, pkgname, funcname string) { + fmt.Fprintf(output, `package %s + +import ( + "reflect" + "unsafe" +) + +var _%s = "`, pkgname, funcname) + + io.Copy(&StringWriter{Writer: output}, input) + + fmt.Fprintf(output, `" + +// %s returns raw file data. +// +// WARNING: The returned byte slice is READ-ONLY. +// Attempting to alter the slice contents will yield a runtime panic. +func %s() []byte { + var empty [0]byte + sx := (*reflect.StringHeader)(unsafe.Pointer(&_%s)) + b := empty[:] + bx := (*reflect.SliceHeader)(unsafe.Pointer(&b)) + bx.Data = sx.Data + bx.Len = len(_%s) + bx.Cap = bx.Len + return b +} +`, funcname, funcname, funcname, funcname) +} + +// safeFuncname creates a safe function name from the input path. +func SafeFuncname(in, prefix string) string { + name := strings.Replace(in, prefix, "", 1) + + if len(name) == 0 { + name = in + } + + name = strings.ToLower(name) + name = regFuncName.ReplaceAllString(name, "_") + + if unicode.IsDigit(rune(name[0])) { + // Identifier can't start with a digit. + name = "_" + name + } + + // Get rid of "__" instances for niceness. + for strings.Index(name, "__") > -1 { + name = strings.Replace(name, "__", "_", -1) + } + + // Leading underscore is silly. + if name[0] == '_' { + name = name[1:] + } + + return name +} diff --git a/third_party/github.com/jteeuwen/go-bindata/lib/version.go b/third_party/github.com/jteeuwen/go-bindata/lib/version.go new file mode 100644 index 00000000000..38d44c2298c --- /dev/null +++ b/third_party/github.com/jteeuwen/go-bindata/lib/version.go @@ -0,0 +1,31 @@ +// This work is subject to the CC0 1.0 Universal (CC0 1.0) Public Domain Dedication +// license. Its contents can be found at: +// http://creativecommons.org/publicdomain/zero/1.0/ + +package bindata + +import ( + "fmt" + "runtime" +) + +const ( + AppName = "bindata" + AppVersionMajor = 2 + AppVersionMinor = 1 +) + +// revision part of the program version. +// This will be set automatically at build time like so: +// +// go build -ldflags "-X main.AppVersionRev `date -u +%s`" +var AppVersionRev string + +func Version() string { + if len(AppVersionRev) == 0 { + AppVersionRev = "0" + } + + return fmt.Sprintf("%s %d.%d.%s (Go runtime %s).\nCopyright (c) 2010-2013, Jim Teeuwen.", + AppName, AppVersionMajor, AppVersionMinor, AppVersionRev, runtime.Version()) +} diff --git a/third_party/github.com/jteeuwen/go-bindata/main.go b/third_party/github.com/jteeuwen/go-bindata/main.go new file mode 100644 index 00000000000..8682966b53f --- /dev/null +++ b/third_party/github.com/jteeuwen/go-bindata/main.go @@ -0,0 +1,157 @@ +// This work is subject to the CC0 1.0 Universal (CC0 1.0) Public Domain Dedication +// license. Its contents can be found at: +// http://creativecommons.org/publicdomain/zero/1.0/ + +package main + +import ( + "flag" + "fmt" + "github.com/jteeuwen/go-bindata/lib" + "os" + "path" + "path/filepath" + "unicode" +) + +var ( + pipe = false + in = "" + out = flag.String("out", "", "Optional path and name of the output file.") + pkgname = flag.String("pkg", "main", "Name of the package to generate.") + funcname = flag.String("func", "", "Optional name of the function to generate.") + prefix = flag.String("prefix", "", "Optional path prefix to strip off map keys and function names.") + uncompressed = flag.Bool("uncompressed", false, "The specified resource will /not/ be GZIP compressed when this flag is specified. This alters the generated output code.") + nomemcopy = flag.Bool("nomemcopy", false, "Use a .rodata hack to get rid of unnecessary memcopies. Refer to the documentation to see what implications this carries.") + tags = flag.String("tags", "", "Optional build tags") + toc = flag.Bool("toc", false, "Generate a table of contents for this and other files. The input filepath becomes the map key. This option is only useable in non-pipe mode.") + version = flag.Bool("version", false, "Display version information.") +) + +func main() { + parseArgs() + + if pipe { + bindata.Translate(os.Stdin, os.Stdout, *pkgname, *funcname, *uncompressed, *nomemcopy) + return + } + + fs, err := os.Open(in) + if err != nil { + fmt.Fprintf(os.Stderr, "[e] %s\n", err) + return + } + + defer fs.Close() + + fd, err := os.Create(*out) + if err != nil { + fmt.Fprintf(os.Stderr, "[e] %s\n", err) + return + } + + defer fd.Close() + + if *tags != "" { + fmt.Fprintf(fd, "// +build %s\n\n", *tags) + } + + // Translate binary to Go code. + bindata.Translate(fs, fd, *pkgname, *funcname, *uncompressed, *nomemcopy) + + // Append the TOC init function to the end of the output file and + // write the `bindata-toc.go` file, if applicable. + if *toc { + dir, _ := filepath.Split(*out) + err := bindata.CreateTOC(dir, *pkgname) + + if err != nil { + fmt.Fprintf(os.Stderr, "[e] %s\n", err) + return + } + + bindata.WriteTOCInit(fd, in, *prefix, *funcname) + } +} + +// parseArgs processes and verifies commandline arguments. +func parseArgs() { + flag.Usage = func() { + fmt.Printf("Usage: %s [options] \n\n", os.Args[0]) + flag.PrintDefaults() + } + flag.Parse() + + if *version { + fmt.Printf("%s\n", bindata.Version()) + os.Exit(0) + } + + pipe = flag.NArg() == 0 + + if !pipe { + *prefix, _ = filepath.Abs(filepath.Clean(*prefix)) + in, _ = filepath.Abs(filepath.Clean(flag.Args()[0])) + *out = safeFilename(*out, in) + } + + if len(*pkgname) == 0 { + fmt.Fprintln(os.Stderr, "[w] No package name specified. Using 'main'.") + *pkgname = "main" + } else { + if unicode.IsDigit(rune((*pkgname)[0])) { + // Identifier can't start with a digit. + *pkgname = "_" + *pkgname + } + } + + if len(*funcname) == 0 { + if pipe { + // Can't infer from input file name in this mode. + fmt.Fprintln(os.Stderr, "[e] No function name specified.") + os.Exit(1) + } + + *funcname = bindata.SafeFuncname(in, *prefix) + fmt.Fprintf(os.Stderr, "[w] No function name specified. Using %s.\n", *funcname) + } +} + +// safeFilename creates a safe output filename from the given +// output and input paths. +func safeFilename(out, in string) string { + var filename string + + if len(out) == 0 { + filename = in + ".go" + + _, err := os.Lstat(filename) + if err == nil { + // File already exists. Pad name with a sequential number until we + // find a name that is available. + count := 0 + + for { + filename = path.Join(out, fmt.Sprintf("%s.%d.go", in, count)) + _, err = os.Lstat(filename) + + if err != nil { + break + } + + count++ + } + } + } else { + filename, _ = filepath.Abs(filepath.Clean(out)) + } + + // Ensure output directory exists while we're here. + dir, _ := filepath.Split(filename) + _, err := os.Lstat(dir) + if err != nil { + os.MkdirAll(dir, 0755) + } + + return filename +} diff --git a/third_party/github.com/jteeuwen/go-bindata/testdata/bindata-toc.go b/third_party/github.com/jteeuwen/go-bindata/testdata/bindata-toc.go new file mode 100644 index 00000000000..65141e50b03 --- /dev/null +++ b/third_party/github.com/jteeuwen/go-bindata/testdata/bindata-toc.go @@ -0,0 +1,7 @@ +package main + +// Global Table of Contents map. Generated by go-bindata. +// After startup of the program, all generated data files will +// put themselves in this map. The key is the full filename, as +// supplied to go-bindata. +var go_bindata = make(map[string]func() []byte) \ No newline at end of file diff --git a/third_party/github.com/jteeuwen/go-bindata/testdata/gophercolor.png b/third_party/github.com/jteeuwen/go-bindata/testdata/gophercolor.png new file mode 100644 index 0000000000000000000000000000000000000000..c9697c77ac18494d3465006c8d1328c8fe7de5d6 GIT binary patch literal 21899 zcmXtXzQ{17rySuwJ=XcJADawCGgvWyifk22-l48oh->3f`u%Ch7p{LsXAP~HQrKqT)qJ_PSy|abA z1BsNVD2aoUy_uzrDG21ToS|Z_s&a(I4_UtzkqPxrkg-?Bge6fHiSWmWrJ^H+Ly-<8 z%~`@!>OmD3|A;@F9SR*4<^Km$iQ!8){6Cmo(%h)ud7&s#t9tQXoL`x9>sivkDb zHyL$PFx{|Uk|f!bIQ+kqh!CM}{vPTd+}dRl`UOwn078MSGa+?;C5Hk%`10}5lXb&% zf}lLW2(X}Tx%6%pf{=IQW06!{DE~mHZkO0^GO+#_AR(_PkwTD=1eAY93Y7{d`y=Sb zn6c3gNRbirgW7K&4D!!-Nc{-~(n%o2gvyQuk)WG~ih(S6LFH5G;o=}o77&J+Og}g1 zA0voGO2bSNRNVyXoY1;Ks+u_%QG(Skny1pOE$C-(parh+get~L0s=qphU=zx_< zt>tT>;1>U-1Iy?LqoKh>L^COcO^?B8_`xtuh^f~z6_@oFJM!kuI0%#-iwT_V&8x>G zQsv|%XH+AS0poGk=XWwAqn)?CsWJy45NONAcjk?Oxem{t_p`t4TQ24KM+<#~Y*$Ex zSuL_q11S4wS>wjxzqJw1iD_9{+S}dzC*3ciV>qhe^Jd;_(5>-m^XSX>0=e7l+@|=& zs`u;r$A`_{@mtvff~h#TK%>?D=pVv5duf#?bdN)F_%R-P;fl=(^3mXO0FI+TY^$)|C7E zh2ZP=Y)X*7nN*x32&6AY!K6A`CpZKP0*Ph+qN^6hzwNAO5^J%pQ;pGQspa?~O zuLz0=yoo=7(+}G601=wtiE2W+AIx(?gzVi)7QyijNKCzI%}D$X$WMl_nLRY^zu-O! z55uAxk}iir8HH<-N5NvJ20f7WNPa|%pdc9u#ZV%XiRF;|svNFLq9#dr&F}PqJ49Ea zHI{c6#2WAt-2VM*!mnB-{<$v=qAkT-c!3hR2-7AmU(=$oveT!X>hbvhaONCLmsub$ zM+=A!v2-8ez_9h(urLkwiPU^!fQvUCt061@QZ|TJtzHh@7ItSmRpb55sYVnZ2_^YI&^78T}vdAPoCD89XtAdn0yF*>;45X81wf~ zs5uR8It~nuIj%cSRf=pOE;s^rNR@@_>q*K%Dsrm5nkHQ%oi}FeD14%HB2^+j9kXgl zX<_MlDT3OK+L&r|iN1>7ie_CXQ^W869q22yLpvS8O=9Q{-^6HDH1bZsfGgl!T0r6oDsqHNM^ z8gROj3B`0Z?NtP*%LHYUimVK!#BX;XwOH;JhXD9B&ydZ zBwApYXqdbe7~3+#mp3mFDS`aQa!_}NV#jvSb}*0XGZiV7jNGx@d783ZOCfC`uPm?Z z(8Tye?!-bWM>_x4Mb?A#gY?Dpjiw*k9@-3z#f{CTQnkCiYT9~r z5)~O0lnZSYZAGX>E%Nd5Wtj`+ebrZuBaJ0ZxYc|AKCe_Z+BW(anj2x8Hw{$woyT{? z`?UH*zVgE$hAbhK5o#0qI@dd4@Tl{wr%nDn7xWppO*&#W*PKF``p34@h(1Q!Z<(Qz z4*qVAZQkPi`zACZOeSEADOrFi9e;3bxMNq^<<3!W%WL*7BfT!Y_pErO>YVf3@;qcU zYn(lcHdBkudAe^{bW>?>Mi3z@k;QtW+*d(V;g@;C0XefVbF4kNn-_-Rpe@ROC+yWey$oOZb=0hvx?` zZ_=(PL7o-U``V>5;Xm0e!kEI*0TlszJr$%D!>2J7q>zJ@$=FFtq##A#!-Z)W!+pbu zKHo3YK_NjJ!Gci^p_+IzUz0i9XI4d>H^iRAlEl{2&A%#eS20R+&S@A7%wX(Mg z4l2=(C&d6; z0mEKPAazMhyDXyYS~^cAR+<}$Eh|4y$JgI+u`!M!rKEE4jpoPZ`(|!q<)%TkZv%D_ zP0U1zFeBD=G3B4M7~va53J+DaX%^{VYRyeG%(^DRM!A#9GU+GI#_L8`Vz~ZZWWArE z^`dRXH5LTRXfP6L?st@1ao6%Wdal6lV1A+xR{vbi*BstRXFIP@cBYYB>Dxi&C1?Fh zzh^X{rvb|P1VIgcFUaR)b@5D3L6cc)qrvbgI0Ih*Gc3FkV+A`I)9)Y6s>dFFi2Z0+ z>S{uLNm+l3$&}fw+0p(wW2O#LQfcyjGLjwQ(sDiGhI;+k-M#nCr>dv#nl*yshKe5cDkcD-1bO<6_I)JmDz)rj0<;0 zL{3Dt$iIWz527u3gzf*J##AtKu=RJ$5S*|B z206md3G0U99?9b-QQZ%&dNgd=C0O^%i>(NV* z=#RRybWZ9U2hUUns5N+ z?2e*%;2AgvNiAm(NLS*&2NXtvs4(y(jEj_DvF6I(T2kYghy%&h@g7ghK9n@g@=aI(2+fHqO9feWLwmZa#AW0r9Wb{Ys^xqD@574CMo4OI{Gdx zIh4`2$D#C7zu74Z6H*Gp63|G+NUiDQJzkZi^AkD^0)=3OD|w@Z$pwBKWd0`LPm1-k znbt)iSLLuPGdvv^FgHavoajlRHMQG&$=dC5sh)O!MoMRVqExfJZCL48^=CMl*AYWz zu+aX{+F9pMu*c3wC$$JBikWp~#-Rx(Db)Qq#kU-hUTGUn_K~Q{x0fS7;V}>^rBF&e zDneI-D7}fg&SLMgNB{wWw3H}n01w=JbBCxko!IvrI(RdIt00tr?3h)C)t60o-Y<9F zZ_724Txt>&$;=rz_G7Bni)d==Y3CGnk{lCa2F2BF*$OMq6*X3@QU51ncy4y-vD86Y z;n6!}jJb=&_r;_0{p}e&%CmZC;>)QsZ>{2Ym1!a{JFwo2h0-aDgxVo&IHl(69~qJcY}XsPDF2Dx|^6p5C8d7b;E#I8>(N5iVfilLGB7J zoNrfJjPHU5Z&j~6c%RnIm8)MRFx?zU{%Be`bvNApPbGS)oOJqd52i;geWuTz!$N+?RvJ~dc0t^KsJ@$BiHesifuM{N;F-@+2wro z>QbTlHedpA309c|xZHC%1kF)9ZprAZgfzXLo^`s5m0OJJy?0td#9!%4GthAnhF9J) z&A+J+%lVzw4B_y5u^Sp1Vw;E~Opc9>82nKbz zofnvQDS_B!$|vUBLt-*^cuELA^__&1vB-rb2u$tT*lM?084E07>V;NMhlVnF;^MxW z(i#bkLi<&xjOP%iG>{GgWq zD|xZr0f&SnU>}fmuCFZJXtN~J=5{72Cr8QtEtJxrD=0TqW&J==num}WCmkp~tDJnD~?Z9m|b6*C8ULqI?x5*1trARRuE7WtSdz;Kq*4mf-eB+8gYOxHPQi z>`LMcBg+)=Tjju*CenlTpKWWJzKU5Z1*N6iHhEJ;D)*QtZ`)zm8^Y2Sv-xm02UFBc zOqF6f212!y8N6)v>wH$KCEds>`h6J(+g$U=nFiCxm?Vt zGRWs6!Teo%WPS~1%PAsHxs^_H+0Bh630C?uBED}v{MN$^%l*X9i#=cA_;rtP?W@Ef zMvI-N`a}u)R=umi6ALN%79* zcCjuuk;Vb_aJSL~hX}{B=jchq?9Rs%@EW2b{|EtgSEi0#%U+oJun2G;91YbMkE+>zJb1iA z9&Zi|l;fcLhlXx$!3xmR4HuVS?H_G&#tYy3Xt6PUN4wtBcR_9UMB5J1Pq-IZCzJFJ z1OW^ab)ov()hj+fyVN+I0K>BTm_tpFm zx2k|9;0Qg~@k}kuyu&@{kfu~4ieDxrf6rjW?5JA&HckW)BqFMl5OONpHi7>B_|N=d z`^?;N&og{t8J_gP@XG2P?~4EDJqiYfs?kDwHuO@PJF~E`FyB=_dJ!KJ2S+0CWd#9O zi{nW}(}$RhGakdO$&$>0vXgeM7dSIH0^dQm+m1G{RNIyOn))xxeF~i?3)GI3wObyL zh4v@$(3G5|r84DgCy9 zcgJrnXJEr_e<<(QyTva=eE;58s+)?k|K|97QnBJF9jx&lE{=r{Xy&8{;zGh-#Q zp`y-DAL*O9?xwJ#*X4&s!S^5xTs<@6Wc#U1CvV zK7%LiH%km(!2RrN&J1Vup}sxUOQn1Cra{k*_Vy<=ORKh6e*6+{T?G^3SHTN*%TNz) zxq4sz5b`!k4!NE!Cn5_xbiezIB{9A+R$C6$O&V5DQ};T>8ZtaSG5URXydzYoJ?NeR z>zm&zbV(V&VwUS@TcN(T=U7&N6z7}0vQ4EuO{CMy(6sn}^2HkFY#U7&zSmsYgMh{H zQ_6g{apV@7%H&VwcBYCJc%s3iS05*Cs|J^-P=oG?rTLkc!~%mZ`wprer#h^=X;|x= z70B8a%i?|cbc(HR#i}6wW3%cJhuA+x>hOQtHm!76AspsZq|F`tMtv3E`WQ}$NK0uj zsSOhH@OThURYrj4d0vk1o$@`LCS{k)b30q+yBTNeWI3qPJ7COeKBO`8JJs1far)_D zbJ!Lb^f~MD)bD~DmW0$Qrl16CW(v2bXW|-@+vV?AgCoC38FV}QrT*8W$Q}>Z*W5}m z`oC^cN(m887blf%=jB7P>6{Z;w2j%ep!qTln9U(dYk1RCes|R}jK1Eg^?Xkw!p4`5 zkQ1-h_`P$a$VB6&>;=Q~6P18mv^YZUhKN2)PVI9m^ef zH9Qa|$gR`dl=wnVs=D6Vr>b;YY6T^*vA=nvVxo@rYh;j<*= zU0GpJdK4PISkhLA-{!WKB3>I`!7E&>J#SzP(<3|hlu>-O2dIM@O|&Pf?tYz?iyrgi zSL%&pjv&Z}-}~RmbPnqUxJp;aYV%00Rfl!T&tZ+jd3+-)V zELX-{Y+R;Avq0FlF_x%ys999BQw;L=XA*P?BogXHAE&205qP~>XeZ$)zfx-4f$G1F z#pdrj9dP%ZV zf~;mI?uB5Jxr8Jv!GZMbENIs~&JC~m^xy9#NrlwZYMobKGTO8L>?{#XjrOp}<>rz7 zYk{DA-6JbT!8NO_)%8Sbh{JX{VQ(zixNb@Qn8s#+!0a$s7?Qs7-)!3XurY4|gN%>o z#kjO2Q?nAk&6mlT_4`1gsBiKov!L);zSmnMO=+&fsRx#f`rAl&gU*BFQ5u<^V4?R% zXrkr}>og7u9=g9b6BH``;`CfB7X2_^2ze4T^gZ`-M@y@RYE8$m7Ao~*TwK_tq@?-> z1~PVMUS4jdTn?vyDd-c+*7YwBtYfLE;c&mvcc!*|6o-57V`od(x@`#cMH3OJB&9yk zV6T|HYxQ-25_;91e>8hRcpIk<&)vN8NNuvf>>KiVi(bjIR4vzxjfp{N*JpNXBc!El ztu#)f;^yXV)@5R4dtaJ!G(Twkatw|nDcWgITw#$*bM*`WdRcs$c57L@%gI7yDx3NB zlFn3&YL!`-3tzCM(^r!Mq130sp< z!ob^@)%ei6RjfkIY`@|bF^DxYQ?GsawDIpu?flXMC}z1{;%0s?@cMcX`lDqz-iN6= z`^O5eF5g#=gA8|hC&Q_My*m#+cI$;dk@#F_yY+Pz$2bITE8OS}C**xn>wYPS_?$5* zDVlVqe-jhM#R9AF#zs>wwi$W=gN@DiAyH%fwd)0RqsM&ITe?|D&RpdBKQ=2JeE z*7opy%y3^P<>0^+S2~r9iCBcjX%Fw|MmU+IHjvrHn)UMq2nnw zYgZL6-uqJsekit*9!T_xMut;lw=KKS~Lnx+@B zdaq)?RGV;socG7K2J!6~zQ$!8wE{(i5*U{Szz59xtx1uz#F)MdoO7l6VGZf-EY6eO zjrJSzkn=U>l5#Hl_3?_Pb?J%+I$56sPp1426G1{;UjY21Rd2-qLwC!`_IWwi6{O=L zUJ|gfR`)j{HP2}5`{-hpk0c;Xd{3IJHw?{Qs=xT-vYAa6$RwkYdj2Yyn4H|-3LyrR zN@f%x?<}Bq&}(bkA?rUutGpI2u@YOZeLqH1ib`-)qDoYXt~cJ_AaNs49gy`r!@gi$ zr94(P>P{CaHwflE!};l4((t8X{&)vJ`)%vBqoXK;8I0SX%7R}+EvM-8VDy4@J=y=_5z>nTZg;6 zLVu5%X^FOnnsVRp`F=2&!CV@6FrM0H+q4#(OH-Z>f!e36-F;6QyTEU}I*f7j-{Bv| z|DnX&9eVE4*ng%URx06RjdQeS?tO^{H&36L{Sjam+2&ob_123~K?vAMnmU}e%VKqA z6T@S?K8dku9M*AZX`_1XYvMpn&y$E00jsJ{5gC~DmT%O?`!YvT^9~I>-JMvjHeZRy z-T|Fh?rgc!_4ESC?=R;Beqvgpms|QEv_iDu{1c(wbOo#2ul?4xHF~9lV=g=iFUXLW zdyd;nSb92v8~%kaCqnv!x)HFSx9_|x=Za@=rI4zv7n!u1>?Zod(2u}H(Nm0zi0kj$ zZDbo&jvZ+)*M4~=`Yj)TUSqRVThPD9XE{qDI6~Y}#J1Qd$)Gmoc*CdvwBd)J&8x=I z7%hGVBKJC35QGD--o%3qlaV*(uL6}Nm8Ij53AUOz(wnaa@nzWMy^l+Z!o6RR1z$kj z(SmQoz}kN9&eCzuDY1&ZIH+DpdFx^675!~FoehOfCA;~r#+W)1QM;x z+Pdb1nC|B?;p3ZBjnVA_e z35i?2MNlWU37E|Ya1^~nff1rql>iC+P#a8N2bm;u@$F>%dI*hHUL}Y zCwRYH4S~w_+LC~)<3Vp~WaG_4o&rt8KcPJBm1rlJKNxIy@Nl(5K}CgziuwyB0C-uo z&$An#02FZ50TyazM)eH2Fuy&V`Dep@_Ap4W*YJ4kg3}jI5Pd=DMC_vHw{x;94D6S+qc4=!99v#i>xOinl6FNe#S_~3W zRz~yhO4+LANT1?9#}wHFNJj@4~Ow6 zhm;)jBe2GUs831SR3CQZ6^6&gc7m|=L6eS0B^FZ^uvz zEs=(jgBAa=-zC6X;!v%)oxzas@U)7pfD2@>_s6-NX{!SWZ9dVEc$KMljOX7!TXe8cC_&J#`s>%`+>dilG)izloX z(-huqB%?yaQi!&pboTKnXcN8XPvWKY|i424?erb-9X&tWla*J^wtrP%Yki#F-nsr;MNKzE5(2699UR z%XbZZj?a$Q69%?6tq5+`SUmcvKz+#wi=Z)o%nuMAEmF=^miyIDW}-syr2VP;nH`xC z8fRkCBfv)g4+A~|W!-(ycoCRku52QN2~ASH4mkpZ^4Bs$LR=LZMnF8Ci-cx$gcE%=f~oSjg3;i-pBcwI4$z_CPL@ zPngX5Eb0J(4(zkz-SM0+Q@!-N_u!+f)Rixa2M!nW_wRTpD!F`V0FTW-7G-5+fCG+5 zB{g=(VXp@~LC&FdmmX`_;$HfY1x?PD8^+T(PO@tJw=URrClped@2}H#S&U4L;jr zbVu5zNw>+N-(bBVJxHH1WpeJB&TiDLRdDk{XL_~%sCQMg) zM|M5=+TR}usC+pt>LC_$#i;mPc6=9{oB*|*ramX_)c*7#@j_WT&E;CnzIMr~@y^Sg zu|XH2yys=g@!?q`*sRZT}acm!gOiabal^pYfI<53_Az{gz-SrLkLaEdHv956^Z6C2%H-`_? zcq}&Y%m~ui$DmV(Tcu$>k+pKP4519Z?+O7REu!Q}r6rxM4pptmUY17(j6D=XZgQHY zNSFsNv;9rHzxR2K{_T-t{IKgm?2gm(&!heq*}R|kgxos~U-~FIwF%N2DU%;Yn-O{d zX#wTHb82F;|2J_L6oJhwaJFIBRzr!r;fkOi9c7s2i&j?J8~=VZiX5@XVtcMU%UX-` zxSp(9fg>6*==#BKdXhyp(|1=ssr&`~5($*qWOOU@M>>%R4qB(slE_aPfNf7Nt}|t0 zuDh)_^0lG9C(c+M<2N2hZeR}T{_X~TYr=8Z7k`|2jw0@H{QtOerTFB^+(Ex!(`a4tu-5_MdI(>2@`np zX7}Hj$tS?*%BW<5f;e*}T6^VY9d6#3amUmwp4Yp|T9d;kceO4@%DH4BxXTxI>}N0f zQf2ffGw+?wW1Qr_($sb&OS8Hc2!ASJpgPpFTBf!09SVR-PGEw02_i88?t&^J=6q5S zcY6AXw?|P`4VAb{pjZ-9a{h(;>T^SB)}6pPA96J0ycYoMT5WE&CCVnGfqq#pukJ40 z332#UmR?cDcmbIiReN2??tSc2uS;LRyRAuK9vwVgFO5tj*o$+;Xa~r**Vp&|7;_C& zt&>Loa_5N3-gsaNt^RyQeAB%Cv#a>AjO*o-P=hpPxLD!-V>$tk`?+o@#$zPc`j3$~ z%Eua4^ZbOlaAvQ^iv_t8wu7Cm|Jp-9;jrvc#9kI2_g9kd8<^rOXC2trg4S68xeryG zmzEhO< zuv1o4mHRF47>|8N!6zJc7_fLd`^rOxVH1?i@TIul4(ynMOj%h+)^+#FS7jfV7kYC zUm$CWYp5meN1jxB^>NCwjak;sPhEL6CE(^7h%Bzz=%H|Nl&b9fVaiKD12lx~q>-JE z1rBclce9KJi zBaH3{KnVbFgk-e5t)kke^uk;k=`b7ei4oVE-Ow<9`_JSx^4>zs5g9DGf&x3+nmS36 zz9S4?Hm78Fhcj*y2%XX1$31mSxM$1m-~28F6pKUDCux~DlRJS%IYE|U<&YlFQ>seW>t1Eb5}Fy zl1QCuCZ7;Q!to;DNmF<&DK^X6urM*jt*nm4FZI3om4~7TANSMkK`Bq`C<)%Z9d|X+ z4~%$=Ii~SE1MHxJA5E6uPB{a96Xctl-K+&{Xtns_iw^n9Cx-+>cODHrW<`<>TQZ}s zXmw>=D!Fd2GU(p+h)+X-jfpCJ06ggMx;7OP z8!K#M!vHRn%QK(K9I%^#uT0L!kc+@#d^lm=u<1c~Jueilzyr*CIklyZDUD>G&EHLi zI0pugVqjHvHN&23#Fy$_8DJ{A`PQo*JCQd3s(XDCjv-Io=<$Zs-onhhOw*jcs`U9JAbAoH5A3%{(>v`Y?8-s7DSdk}x6+P=W6N*Et zokyFsV;-Cak-KKByDfk@AfSTxe#_}+Xn45M{&*_z>7ZyS1z&H`zef?|ST9!k+oQ$~ z<*BHn>IlRElZ<5Qfnq{v$uIe983`BKM9fcFYyj5&xmv+hpKsS&;{4fUZx^ejb|jU- z4jAD(J)X5B_}5_aNBAMDzrjZ)tIduC-yE`s7qvUQI8=({P5VQWQAIAt*`_w{yjqTy z&0en7KI)%ttaL$NY@BoYuc){{w4}rO?2KO$E*`9+>=FF{Vfz}!RH|Mf4b%}SSy{fV zzW}nEEmfnBgAf}Y>cm6Sk?yv*4U0Z+h&seTgZb9S%*tF$vjuM`J;9~g{DUm_1fOt# zhSBQYeqYZ_K|drO@-vIq(l8VyKu*xdd*M&X!S(g;-vsU7+`_wB5rpZuyoScf)EAZw^)ph_NVb&Ay$u}8?`0%=SnBEHa}9d25mBxaRzVL(g| zMH3%{&{ipt^3;yEoC^@DHvjClwpne|3Gm_Ys8mkCk!2eo^SbJ_3;14dr=11)ek-_h zt1Z3k$g}#qDwX8W_kw(NEK$?MK+l^z6w)zd0DQ&scj!%n`{WeEVH1T)dD5TIP6ok8 zPf7I_x-fcNz#xO^DZn#)wE5JQwcov^dVaEysbg>9DG|5A9ytQ`GrFIm$K~^2vG2HB zfcqW#pK%JHjW?5&qEXi_w=-=321uNxsXc7KFlQ)MRWfMZ6^F7c-{SOliqgWOwo8w$ zmN7)=uX15c8Wgw~5E5ecFdIY(oLr5m+ixejcXcNoP}1{ahpY_}Z^+T0Qr_*!X~4Si zNKTFC>Y9ARP330L?TiAjiHly5&k0qEGGJI^tCH(Vj||$5ta5p_`bP&R-GP@~*?{PH;=5^l|N996Yr0aeO=Z+(t<_c6b1%`v z!~_Ku)zPYol0M?{(xFr>-1NMNdNU8-Pa5*D@wLTLy7T~>EGA)gJn7)S;eB|=9dIf(ff&M6y8v93-R9l# zf7jU>G4UR?r(z7WKR~528;xyNo4*21D&V`UZ%Ae4ZyFljUTkD_z(m)+E5e&%rY^NG zr14O_r_(w+6*AA?5sFDPHihaF&}f{O{r;4d{L%?k7aE}kLpMEgtQVF14$Uz zQ!w9uMn7L1c5jTpwEWz(I4Am+=9u*~`>n}FIU1K3$@t72U9wm|3=lDYbI{fPH9L|6 zg=McO%a^A@7a?4r^P8KrP?e)#`Y9qfA98kf!>?;^EKUoNkQn>`R6v0;eY<+gLVml4HX)cAGy~I=p@{sD zj_*Nsh2RS&TVHztr~3KuDbu?E=|Y`kwWrE^_{QEhe9exRe5p+1-ZW{+{lTJ}ZawvF zp}DeZ;W>5HiS#3f)!ZhK(BXruIv4SuFX-4a)+M#G$f`IQG~vTrTTH~neAdtsW2W-x zcvcRCHHft@|4#~X-TMYNq`y&79M0i-A*lC9zCQSwvNpOLcs5s>8W+lPKg%itDVK+* zld6&7;co@yMBbOe07AVy_DZE{Mfo5Oo9Wfu_TKdOuWgpL7<1z{ek@ux9eU|N!Kter za?6*4rJDLvPXDQQ2m)!3{Ap)Lf9H6~qjZbW**+`xdl^0z;>m+sms~pMS$#%ELY&pS zw@j$@eCYrJb9?uADjP0`6^8q|C$)yInoC!KTn>#=6+R20 zmZe2t;k?jde9ekA_0dz#x{w^$MGwl0{F3rZu}&R4rqQUQUlV?b^4Byscm7B@uqMB` zcd9BNV#!N(^4O(q0EKKl=}}%b1h0+sb53DY7?`aOBD&SD_=<)UWl z=xJsq`PDfQ9-)95dh^Q00J)rb4vGSIF!s4azheTZKyqAmGG+tBI=g-Dt{6h8Zd^eq zb8^Ll#C|V5g>vZ@)7d?=Bdq|A>l$$XXY0~-gjcE>o^x}jp~OUX8kpgN<^A}H_+)KB z!9{PXm6c-G&dI5>wf%0-$Bg~$N6f@A0qy=lz*C) zUFUe`N6VS^>%^mDzt7337YQ#tx5A;7d+|G!)jlY4FxEV!N)i#R!}-U=xFv}*+s#!m zFV%_<^V(VVMZ8|n=E2j>uqOhXWU?{i=L}-}*xVyVtO&js0+gUA?c$q)r4#u@HqDe&Zfdd!P1lGS-`4IkHCB?E!`Dp z@|t|kE!Qpy$Z!)zY7^v0G{iYvpn3Wf{p%0GJLbluPFz=o$k=B#_AdNy^H<8!v~Q0@ z+>{jgOXN#ST^>e8`lOV5;t_`j&}tRImPAN#v+-?K_Z~KBRxSBh;SJEPd!80+wj>}R z3^G@&^gAH|0}UPh=wA)FTB+*R{fhmN-Fs1tq_i~re@nJhYnoEne`G^&;Z~)91lN-E z_isV+sVCElYjki8wpzvc*FXFO0$zWt66rOUEps#(y{f^#CJMLij`Ir&wp=nh`n`Z? z3jhi}Z{~y$8O`QPwb?cvyE;21*|jB1Icp`yaQxv9Mj;(rPh*E3OVXi%`*L=G|NaO# zsZ)=u3IdNGi@>xU>wSKD#|O=)*5U}LkG*k;en7)aQrD(&TXoW713ld{YIlSLw0p0= z=q0+ZI)?c3VH9|~0+QI@X;1qNKBdxm7;M%fqGSkW?-ljY`L$-}9;^EXa4?*9=12L2K@BkVPO+t}hH^hRZzl=hGqA#YX zsPcupqhhGe+OZ{+CCQhqQ5K7`TG2_f-3eMD>H!3Rb6{YTl9bF=uh9O_u`a7@9qkIq z-hBV_=SyyGZhql_?A%?#5YwZk=#LJhmCu({@;>gcuIQ!O^m_~#S!R}ah&+5Qqh0fr zdOA@A+{)iFrQ z)Z2>!VL-|yagdN$d=XKV!{xVOpqtWs`w(JZ2K;)6aFq&8=@eG)y&&xxBsu9GX3jgI zl34RYV;7h0%Ea=7AM^rjd(if;)GRc-ZkB7jzAc~N*thmvRUd@gRyepSZjxG=y;0-^ zC~{e8$Scx!Q2=+rT9JtD#)D*h-_gZ2Z;r9TVU$Nj*m(5WE3^@d>TnBN2>mm7vmio) ztI#l1_RQY^D>@!XSqZZ$7mmX)dM+Ky&6>$IXwh)yJ{K;*cLELvqa7q|tnd}e{2HSG zJ{*nDb!F10SL54MK#-Bcnp`|dNZw`{$n(Z@Afo#x>GRi#ankI|iPeDxX>Npp4?scU zG0=G^(g8ji{1_4IZn~!oEFxTO{hUZmws6I9kgjLb1esS!^{G}BA%D_H(Z-vFmMg+Q zfV##4GeJWhO=6jj7&GR%`ZSD6;X4ImnO)8|fDVxMOp2)UzgmUl6$0VVu<2=e8tF(< zm}mlYAkk7=-Wp_diB{m}j6l-U44h!{f3G<`^~4!W`r_dIJX8Nz>!4uM)BBiB(!uFp zS}G#_Y6Z0MPm1%Su!iqO?>QJ(XYB<^YE2Hmu-5u5A(1_wU|^tGd&#_L{S67aw>+!_ zVAr2aczT5Ydkxf+NgBmm@fRU=b*vw`rEnZf2zl6t%-}-kMJ;E@qiM%#oqHAy#tiCTb_+U{ws@8-Ih*H-D<0|_M zv5U$xOU1U&@*y3nm_f2qypxd~S&U+Dn?DQ2U%Ql<--R?-lC8CdQMkkQW0HA~MOgJKg?s-`zfBweW zs+%bNDQAUhOo!C~&gl1GX9jXG_A{MZk>PHV*JCmqG3U2tD5k|Xm zs|~Gt>eJ3XU3O|O?-@|tJMQn>NsRvK;MCb00`c%a1Q_%%=n~kh)nJndzJ8@fxmVql zla%}cSOEW}k<%ySfqd|tf~}r(N@2c{O^b-F22%v;I%|XB$_f7^=|9P_;W*Ql%BO+g zgt*<;kf|A`r*%0%)Eyli<(ryZ9Y2GaIuIB>O5Ih_+}e0~MjRHPkdn$7gB({x`&i0n zd$A)4xLfR0)%dHN36Wp`oNBb}9w9w&GuM`@opW!ws42#5YB6g<%FRu$qSkOO2ih3d zv8uTw8#pL?UyU-v8PDK_wDSd|+qJ^cQbK>1ZQdo-_`MwDUgP;2*gLG{I_~_OIx=5& zsAYYi-07Nc8wRDs2T>Kph*Gq0sx-akL~5<3|8}{K_~&Ei9QHw>Mu2WaUr@& zsD?p|UY~#u%_?t@dePf854$!C{uEw{(B8e(VBk>F;qW@Knm(u>W;vAST=n(5~We5sT`ZxK8>ijUuao zUWwI}N#IB!mW7C)_aq@h`mrG@vy5g~@tD{m*|0sQSxI$2v+S+Tg7NLLzef}O7oBNt z{)WeCBiZ3~=W*Esqg87%VhOj=;%pAMuA0?9pwPksWvpl9;`_beg0lW)raYvI-_CwX zuMh}FyA#eFBIglc2_!E$N;2&N!IDY>Tk8|_V#2NRAaNm`hEa&eyB*dK1JB{U_MWgqW1Q3^K151 zpJD!1#h2P+$6CFNrt{qfqlF21!7I(VX}65iChsD~adqYFGm!pbo-0$!c+Bb)vGT&= z1Y+$Ju+c*imeOCq6vLM0hbdDJW|>pD)Kpa01K0*YBnY;9t1qNPvxY3342SE~ak;@t zmu^AuZ*$8)h_0R9cO>43?g#izMCA~DfIV$t0wBK%U+3j;-gY}o%np)DDaW>Qdgj{iJ3r)Je5z{&4m}pM z0kb~ppYnVpNjD^}BSQDVr7s(ZfjwT%3$xpLdtJxZ+InGA7qy!05d|>B<@0zrE=+xbQ`4976fX*>To^B$-m3=-*?)u3L+x%TXHkUEtK5NxKL#-^WC9Jb3%y;hXcM@li@PBd zIjsHeWIVG4^16fAIFS=rys#+2hSxV$C*^$3Sl4~LLG?RYkDq&1jer|`8V><&iiCiA z_lYFS^Nf5fR|n+%i57`S?j}|eZWOplsijM?0!jIrh6IR5^66JmVj$oIOL_W9msGC- zr|*3D#7V-h0p6DQUj-4h27zvVjq}?Gucjj~HH;s^k2{6=NmYv%Af zi1CwS!i0@i^jVt<{Dh)nUJD}W)}X>cD}_K6VIYbmW~S#~r2`sz9eE<&QZo?V+f(s!O=k_$RohQ&$> zQA0cP_&5$H8YRS{&jno^a}!#Tk^}18wC^;HjETBMz!k}h;ep$!5wGriG|RU%Z5`^8 z{t-GB;%2EqB{A)Og4gSzs(0={nW-fHo*L2nnJKhRrjC%c<(DWzzG4w)t8iwq7h=Kp zm8Z|Y36Br5zyk-=4Ao4TLi0{mJfG*dk`#&bY`$DUqibij%r&T5rbWJgG<>=hm>{7) zh|FrDeu&Zf^;?*eKe8UXz2Hovr20Yj=4U2*2&Lt+fZOB~o%XwTLFYwn!xYco3*M5& zZggvmh)eptQk0sebHFQD1^5bhWrR%}GBV8c;A*%%zyjm$G=fxMuMb6jwgOYhHNRK) z&UxLK2%B}?Hp-eJ{Q4gJKO#;+e>b>j$I3B1LWwvjeJcXN7|*G2%BGP+ORbfSONAXH zqQhJ}OD1kNSRHLTLE0{-o-~{xTsPRS7;MMFBi( zNd!WGmajGf9yQb>*3Hx#7tn>GbPn(X1UOP-pLvW5@oyhU0YmeTHiqZV?xo*_HjI33ckuY^4J?|%<_%a z!a2Dg&IzTW-1sG}+5>g!2AaLWmv2U|q?3lT2vu-yV+SiAi^6>p1+vrptv#TCrz!@;qli3>=kFlNsSc(M-HPcb`HJ z^WeC$lBouqAM3|>eFH!1_-GK!vM$9cPgGuB-eb2#rutQ06`{P7nJ({%d;tqOki0DP zcV3|zRm!TGV&ssMChmNV-%;h*8#%bP3<}yhK{zu$^!lP?>oM9rgfk_dqpyEveh4f! z3W2O0qvL+`F!N;ywSbz$`iduBsc{J~Vv( zV6uvc#&|2WJ&jUI%3G-cB4y!UwT7ZM`H4}3ip+EoiF4Q5e^}Ys!zPpLK~f6|GEGJU z$HUymqzL%QDOgjb=V!`9O(Qv&CG@qizp=OM4i-D7-Ro-t$c=Q?e|`{#pY4e*bzlQR z7VcOzm?2{<+e)RjD|il^}jZ{N)RXnp+vS6&C2i@#ZvgXn0( zoW6kp_2NA7o@*wI3Sd(b(xZlYBu`bZ88twBbH-k_*;d^L_5wnG(p^BooMUg&TJd`i zWbn~CzUow9R=m7*&Ivr>F8*x2n@SF9`Eb{>%8NzKkWl63YbI5D?)J@#ldTD9&tEOE zzi~V3pcdb}$^rKBx1oGmsySn8P;8qv=CQDRd&@2qBwG~k|53+gh1NbFjbISYJi_g_ z5Ar@pS`28un!Le6C@7qM=0eX!n(`i9+f{%SwTkekiD0i}B+7ib<%x9wDEA7_W3f{8 z;nr9sR4sK)5JoqxuTO|t)zf=>dlM27Re{VAre{agvQ}=Xxtr!0Soy$&tl;hR1H{Q3 zHH1DapprB%WW^Y40syX^Q`Ab)P!N8Hgz2vIk1vceR#&DM9}s#vIy_M51>a{eP}}Ll z;KvAW@{u+5YLa>X?_~e@(zRC_V~b~AcP;K#j&owpYQOHB8oCIE0$`WNq?i|W<>+9e zyd+l!(jGex!r83leYMn5xN{u|a(zvuxm-kwIbK z;~91qn$04Vy0;f@Xu-{NBog${8@wujwCLZx07zomNCVRsb9IkqrQLW|d=Nj1{{daf zxYGx!TpCd_Hv891D3cIYE#);R$FL4PKZoE#`Tg?CFEqTGUZ5JZ#%oCp1hc`V>OmnF z5EW%@J-XRa@3&cUGLVq4Q9z}WeYF|Af7c<$IhE1eyipqBGYj%GZkM2yi6kC$RMj8B z&qTzwE*OO7f28u-7PxpPZ^O56`Meq%XGUAYuW-Zp`L`OMya$h6o}JV4TO7lC`XIC= zxm~!_c=W}AvWVtkOyaLcNzOldc4_IkHT=i~lhMjk+vBC)xvQ&mfu-MjCf7Gc$1Fld zLDo_}(yGJCb82k3zqa~Ex{#J22^fysqT83bTJ?w=Ci?WCsKL5D)okYE%E$R~3xN+L zHe}Zu=Ruy^v1z|Zd(#Gh; zd*2D@#K}*>C9C z(1jBbf?kcmNVE2J?f1;cFU+myQKgZW%ye|!AV90`<0ASyb8Oa!i@>STA3{-D?C)58Nhu&AiIM_~{6b^rUsM6aqVm9yP zPp50g1i_^Hg*{~sp(+NnMylQGb}eem)wm(T^Zf^ zpy9_xoQ{%u$Vw-fokYTJ=vPTo_%Jb!ofh;F#9t!2PhluNEk@ z4efJ2eO7GGzWTOTeUy9DJPiGvcQl_C5rK|2E@`}7v*C?EicG#}7KKD^4t2T~xCx05 z>7Vj!jUDc+v&;Eh>V0xDCrZ-HZY(&I^nlrgi^ylYL9XU|g0Q5KLG1vxtrcrKA8 zpE@QKRxq-*LM590NTEKRWRSFGhvpN#gdh9;iuNeS{XF$StjZgILvC7`FXs1*Oss^1 zU~ggV#`3kEi7VvpqJIZoKej$3vya+!5muo2FO{|%kFH`z`OR0^9E29bWbC4UbZU(Z z_4DQ)r5ITI(fS8Xn<^KZP#iH<@Y6uGk$qIjBj@Bi0gOLI)?CF(U)AI1|p#-eB*VmkR{j#Xm?wfQb=Do~S4B=4N z!hWmUURWW_B0%MsvQpZrd~1CTYyWI8@r5(0o82GSC|eu+HanrxvRywQWaKbTzT8Fp zvZeOO&Tmhj^F(QsJ2FswR26y#VZ7S=;bVE~m^ty15GX3S6Zds@?On#|>lZhHulhG& zs-$>>^-I%~e)@@@OByx93#mIYlEhk`giwbo&iv{3n7aGzP4Ui7yW;Ly_^aW=>J1Ek z4K~nIZl@iXz6H>SzaXhsJd+ZLr=9yF7>QsZU+Rk4ot_wp4Sd+38Qm@PYyQG*NaxQL z_5nft3tqeE5x;7p4DktlFV{yW1lhJk=m6JQMe|7_k5I<2zjn;FU4Pbor(fadV&d>` z0krtb4JPg1#=c*0@vzAUkTQtx#XlXa%36`DBI~SC3CDjFzJI$4^Wf7pMJxBjzIkC9`xwY0B^xIGPbayT$EiEmX z^$|o`R<^LT6w+bd`bU5Gl0KPH>{{S4ZtLVW`(D>C4qk)sp7Xvhwpn^nAs!F9g+l>#s(5@S>3Xn z`UN+igzAkU(Ssn4ZzCgEwjnhq!pBsS7<$5ygZiCfEU z-tGOyi2D?&J9RLO4tKm64IK44Ryw92Pp$}Onj1%=TpmStx9W?TnQ}3=&@WCpj%Euk z`-HJ{+UbhhEl5d70Li_+y7q-g85tR5V3m`=5O(>#D!7$}+#XzM8*w#S5`NS5 ziAOh$@wFD(a&vH+FamXEmiyj?%}l1#9D%ooCMf%?5#<8$ziACO&gYxvF0;fdmgOzT zkJ=YclyitW1VuzX^)Jg&LX*_vk<`GLbjIG`!;<51zMKrjw>&dJwBi%uMYv!t@3La0 zMck^Vu2xX&?N}-KqsQ6#*IiklJJd!ic$f^x-S7&nK0ZCA1&9Id&4ur&Pv6PGrjBX=0~qXky5vPrKVBgb z^W2+m2|c#dk60R(ueTS<%F5pUTpvAaT<{>KCSjq`P;WM)1T2!8S6#cEsAq4I+i`@MHdxS)38^C;laLT0rk8degW-UV%8h&)#_=9_w-ulbd#Hg{vy1p2 D=Pi;N literal 0 HcmV?d00001 diff --git a/third_party/github.com/jteeuwen/go-bindata/testdata/memcpy-compressed.go b/third_party/github.com/jteeuwen/go-bindata/testdata/memcpy-compressed.go new file mode 100644 index 00000000000..5407a54167b --- /dev/null +++ b/third_party/github.com/jteeuwen/go-bindata/testdata/memcpy-compressed.go @@ -0,0 +1,1849 @@ +package main + +import ( + "bytes" + "compress/gzip" + "io" +) + +// gophercolor_png returns the raw, uncompressed file data data. +func gophercolor_png() []byte { + gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x00, 0xff, 0x34, 0x9b, + 0x65, 0x50, 0x1b, 0x5f, 0x17, 0xc6, 0x43, 0x69, 0x8b, 0xbb, 0xbb, 0xbb, + 0x16, 0x2d, 0x5e, 0xb4, 0xb8, 0xbb, 0x16, 0x77, 0x77, 0xa7, 0xb8, 0x14, + 0x77, 0x28, 0xee, 0xee, 0x4e, 0x70, 0x77, 0x08, 0x52, 0xdc, 0x09, 0xee, + 0xae, 0x6f, 0xfe, 0x1f, 0xde, 0xc9, 0x6c, 0x32, 0xb3, 0xb3, 0xb9, 0xbb, + 0xf7, 0xec, 0x39, 0xe7, 0x79, 0x7e, 0xd9, 0x9b, 0x08, 0x45, 0xf9, 0x9f, + 0x48, 0xf0, 0xf8, 0xf0, 0x00, 0x00, 0x00, 0x49, 0x4a, 0x52, 0x4c, 0x19, + 0xf2, 0xf9, 0xfc, 0xdf, 0x06, 0xfb, 0x15, 0xf2, 0x1e, 0x7e, 0xaa, 0x2d, + 0x00, 0x00, 0xc0, 0x2b, 0x58, 0x8a, 0x8a, 0x2a, 0x2a, 0x5a, 0xd8, 0xbb, + 0xd8, 0x3b, 0x5b, 0xd8, 0x3b, 0x90, 0x4a, 0x89, 0x8a, 0x92, 0x3a, 0x38, + 0xd9, 0x9b, 0x59, 0xda, 0x98, 0x02, 0x00, 0x1e, 0x2b, 0x59, 0x2a, 0xe6, + 0xaa, 0x2a, 0x47, 0xd8, 0x02, 0x0f, 0x1b, 0xb7, 0x22, 0x32, 0xe1, 0x01, + 0x49, 0x32, 0xf6, 0xca, 0x58, 0xb0, 0xa4, 0xca, 0x22, 0x51, 0x01, 0x98, + 0xf1, 0xd4, 0xf4, 0x64, 0x70, 0xa8, 0xd2, 0xe1, 0x64, 0x79, 0xcb, 0x58, + 0x4a, 0xbd, 0x68, 0xe2, 0xe2, 0x9f, 0x09, 0x46, 0x72, 0xc3, 0xa1, 0x63, + 0x62, 0x02, 0xfe, 0x60, 0x29, 0x31, 0x22, 0x46, 0xc2, 0xff, 0x83, 0xd9, + 0x23, 0xcb, 0x8f, 0x09, 0x2a, 0x08, 0x8f, 0x1c, 0x7b, 0xd9, 0xf6, 0x2e, + 0xb0, 0x5e, 0xec, 0x7c, 0x38, 0x98, 0x7c, 0x6f, 0x5a, 0x12, 0x04, 0x57, + 0xde, 0x64, 0x35, 0x4c, 0xc3, 0xf4, 0xc0, 0x22, 0xa6, 0x48, 0xb2, 0x2b, + 0x71, 0x06, 0x20, 0x96, 0x89, 0x10, 0xa3, 0x6d, 0x05, 0x0d, 0x0f, 0x0c, + 0x6d, 0xef, 0x31, 0xff, 0xf8, 0x0d, 0x4f, 0xe9, 0x00, 0x40, 0x85, 0x6d, + 0x30, 0x21, 0x73, 0x7e, 0xa3, 0x80, 0x02, 0x3c, 0xfa, 0xf1, 0xf3, 0x33, + 0x90, 0xf7, 0xc0, 0x74, 0x01, 0xa0, 0x3c, 0x40, 0x08, 0xb0, 0x80, 0x1e, + 0xb9, 0xcc, 0x1e, 0x56, 0xc2, 0xb0, 0x0f, 0x94, 0x63, 0x91, 0x74, 0x5d, + 0xa8, 0x80, 0x60, 0xa8, 0x1e, 0x97, 0xf8, 0xef, 0x32, 0xb0, 0x01, 0x98, + 0x80, 0x1f, 0x5e, 0x31, 0x22, 0xc5, 0x80, 0x1f, 0x12, 0x50, 0x01, 0x59, + 0x69, 0xd4, 0x2a, 0x80, 0xdc, 0xcf, 0x80, 0x5f, 0xe3, 0xc6, 0x46, 0xbb, + 0x00, 0x45, 0x26, 0xc0, 0x2f, 0x1a, 0xff, 0x03, 0x10, 0x20, 0x20, 0xeb, + 0x31, 0xdd, 0x1b, 0x0a, 0xa0, 0x93, 0x44, 0x86, 0x05, 0x95, 0x1b, 0x0f, + 0x20, 0xc5, 0x30, 0x0d, 0x17, 0x03, 0x58, 0xf0, 0x01, 0x2a, 0xa7, 0xd5, + 0x23, 0xc5, 0x01, 0x9a, 0xac, 0x00, 0x4c, 0x33, 0x99, 0x01, 0x1e, 0xc0, + 0x3f, 0x26, 0x00, 0xab, 0x94, 0x86, 0x99, 0x24, 0xa0, 0xb6, 0x05, 0xd0, + 0x33, 0x85, 0x8a, 0x50, 0x0f, 0x80, 0x45, 0x04, 0xb0, 0x2a, 0x85, 0x87, + 0xd0, 0x01, 0x3e, 0x7b, 0x03, 0x7e, 0x8d, 0x50, 0x50, 0x78, 0x00, 0x82, + 0xd3, 0x01, 0x98, 0x12, 0xd7, 0x1a, 0xfc, 0x57, 0x0c, 0xd5, 0xa8, 0x60, + 0x7a, 0x48, 0xa4, 0xd2, 0xeb, 0xf9, 0xdb, 0x28, 0x79, 0xc4, 0x7f, 0xeb, + 0xc0, 0x32, 0x39, 0xc2, 0x68, 0x68, 0x30, 0x13, 0xd3, 0x4e, 0x49, 0xe1, + 0x30, 0x60, 0x72, 0x19, 0x7e, 0x32, 0xcc, 0xf8, 0xc1, 0xdc, 0xe7, 0x99, + 0x8e, 0xc7, 0xf6, 0x9b, 0x03, 0x65, 0xeb, 0x66, 0x02, 0x00, 0xc8, 0x8d, + 0xc7, 0x82, 0xcc, 0xf6, 0xe6, 0xcd, 0x63, 0x0a, 0xb9, 0x7a, 0x6a, 0x8a, + 0x2b, 0xa6, 0x19, 0xd9, 0x80, 0xe9, 0xb8, 0xfb, 0xeb, 0x07, 0xb9, 0x91, + 0xd1, 0xee, 0xfb, 0xfe, 0x74, 0x85, 0xc3, 0x0f, 0x00, 0x60, 0xdb, 0xc5, + 0x6f, 0xf6, 0x9d, 0x91, 0xa5, 0x01, 0x3f, 0x80, 0xef, 0x6b, 0x80, 0xed, + 0x7b, 0x3e, 0xd5, 0xf9, 0x67, 0x0b, 0x7d, 0x84, 0x5c, 0xd7, 0x87, 0x28, + 0xb3, 0x7a, 0x94, 0x1f, 0x4d, 0x80, 0xdc, 0xa3, 0x15, 0x8d, 0x1b, 0x87, + 0xff, 0x62, 0x23, 0x9e, 0x17, 0xd7, 0xb6, 0xbc, 0xbc, 0xbf, 0xb7, 0xf7, + 0x4f, 0x7a, 0x40, 0x44, 0xc7, 0x70, 0x4c, 0xc3, 0xf7, 0xdd, 0xbc, 0xcf, + 0xa0, 0x47, 0xe3, 0xcd, 0xe6, 0xc9, 0x8f, 0xff, 0xf5, 0xe1, 0x6e, 0xab, + 0x6b, 0x87, 0xf2, 0x37, 0x9b, 0xde, 0xef, 0x9f, 0x9f, 0x1f, 0xb7, 0xfa, + 0x26, 0x6e, 0x65, 0x8b, 0x08, 0xa7, 0x13, 0xe0, 0x82, 0x8d, 0xd6, 0x0e, + 0x62, 0x7f, 0xbe, 0xb6, 0x20, 0x7c, 0xbc, 0x12, 0x81, 0xc8, 0x07, 0xa4, + 0x8c, 0xda, 0x34, 0xe9, 0x31, 0x77, 0x95, 0xd0, 0x1d, 0x63, 0xc4, 0x32, + 0xfe, 0xdc, 0xb0, 0x15, 0x91, 0x47, 0x88, 0x24, 0xd0, 0x0d, 0x52, 0xcd, + 0x7d, 0x98, 0xec, 0x6c, 0x32, 0xbe, 0x40, 0x0b, 0x6e, 0x72, 0x02, 0x6a, + 0xad, 0x04, 0xa1, 0xbc, 0xe3, 0x99, 0x4a, 0xf5, 0x8c, 0xa1, 0xef, 0x65, + 0x68, 0x50, 0x35, 0x5d, 0x7a, 0x76, 0x1e, 0x00, 0x80, 0xeb, 0x0e, 0xfb, + 0xde, 0x79, 0x5a, 0x58, 0xe8, 0x00, 0x8b, 0xa0, 0xed, 0x69, 0xf7, 0x8f, + 0x0b, 0xa1, 0xb7, 0x9f, 0xb9, 0x54, 0x80, 0x00, 0x33, 0xa9, 0x04, 0x47, + 0x00, 0x40, 0x5f, 0x8c, 0x92, 0x59, 0x75, 0xac, 0x41, 0x68, 0x18, 0x16, + 0x00, 0x10, 0xcb, 0xfd, 0x4d, 0x5f, 0x2b, 0x4c, 0x70, 0xdb, 0x8b, 0xcb, + 0xd8, 0xf3, 0x95, 0xb0, 0xf7, 0xcb, 0x5a, 0x2f, 0xc2, 0xeb, 0x2f, 0xae, + 0x60, 0x11, 0xca, 0x81, 0x3e, 0x11, 0x54, 0x11, 0x78, 0x93, 0x00, 0x42, + 0xa7, 0x5f, 0x74, 0x95, 0x81, 0x22, 0xb4, 0xa1, 0x93, 0xb5, 0x44, 0xf4, + 0xbf, 0x58, 0xe6, 0x7f, 0x10, 0x71, 0xf4, 0x28, 0x59, 0x84, 0x26, 0x3a, + 0x20, 0x33, 0xf7, 0xa9, 0xb5, 0x22, 0x0b, 0x38, 0xa0, 0x3c, 0x1b, 0xc2, + 0x66, 0xf7, 0xd2, 0x76, 0xfe, 0x86, 0xfb, 0x2c, 0x3c, 0x02, 0x8b, 0x61, + 0x48, 0xb6, 0x12, 0x0e, 0x65, 0x14, 0xa9, 0x49, 0x11, 0x03, 0x8b, 0x93, + 0x1e, 0xf2, 0x48, 0xd6, 0x2b, 0xf9, 0x19, 0x3d, 0x8a, 0x92, 0x74, 0x34, + 0x1c, 0x53, 0x89, 0x5c, 0x26, 0x9e, 0x53, 0xf2, 0x9b, 0x72, 0xa4, 0x2a, + 0xa9, 0x9a, 0x24, 0xd1, 0xb5, 0x80, 0xd3, 0x27, 0x9e, 0x30, 0x5d, 0x89, + 0xf6, 0x78, 0xbe, 0x11, 0x00, 0x5b, 0xe0, 0x6b, 0x68, 0xe7, 0xcf, 0x6f, + 0x49, 0xbf, 0xeb, 0x95, 0x04, 0xe6, 0x11, 0x9b, 0x44, 0xdb, 0x4a, 0xb9, + 0xf1, 0x83, 0x25, 0xf2, 0x11, 0x66, 0x4c, 0x5c, 0xbe, 0x65, 0xc4, 0x62, + 0xe7, 0x66, 0xce, 0x38, 0x35, 0xe2, 0x0b, 0xfc, 0xe1, 0xca, 0x03, 0xcf, + 0x54, 0x58, 0xa0, 0xac, 0xc4, 0x0a, 0x8a, 0x0e, 0xb3, 0xf6, 0x1c, 0xe2, + 0xc2, 0xb0, 0x0f, 0xd8, 0xb0, 0x32, 0x0f, 0xf5, 0x8b, 0xd4, 0x7d, 0x67, + 0x84, 0x4b, 0x34, 0x1e, 0xaf, 0x23, 0xaf, 0x44, 0xac, 0x18, 0xc2, 0xaf, + 0x55, 0xaf, 0x84, 0xee, 0x88, 0xb8, 0x33, 0x9e, 0xae, 0x73, 0xff, 0x8c, + 0xfe, 0x5b, 0xbc, 0x07, 0x8e, 0xeb, 0xb3, 0xbb, 0x08, 0xa9, 0x21, 0xb1, + 0xb0, 0xa4, 0xa6, 0xac, 0xa6, 0xfc, 0x3f, 0xc9, 0x52, 0x52, 0x42, 0x4a, + 0x0e, 0xb4, 0x15, 0x38, 0x6e, 0x38, 0xe8, 0x60, 0xa5, 0x3e, 0x1a, 0xf2, + 0xc4, 0x52, 0x99, 0x46, 0xea, 0x76, 0xb4, 0xa7, 0x01, 0x5c, 0x23, 0xb6, + 0x6f, 0x61, 0x9a, 0x92, 0x8c, 0x14, 0x8b, 0xe8, 0xb1, 0x18, 0x2c, 0xa8, + 0xde, 0xe1, 0xa2, 0x03, 0xdc, 0x94, 0xe8, 0x11, 0xa2, 0x6c, 0x34, 0x6d, + 0x71, 0x9f, 0xa4, 0xea, 0x0a, 0x55, 0xe7, 0xd0, 0xe7, 0x24, 0xe6, 0xec, + 0x94, 0x29, 0x8e, 0x18, 0xcb, 0x32, 0x39, 0x24, 0xf1, 0x29, 0x12, 0x47, + 0x57, 0x4c, 0xac, 0xeb, 0x61, 0x78, 0x92, 0x18, 0x49, 0x19, 0x46, 0x42, + 0xf6, 0xb2, 0xf6, 0xfa, 0xf7, 0xd8, 0xf7, 0x08, 0xf6, 0xc8, 0xae, 0x75, + 0xe0, 0xb3, 0x8c, 0x7e, 0x38, 0x16, 0x3e, 0xaa, 0xec, 0xd0, 0x2a, 0x8f, + 0x59, 0x49, 0xe3, 0xed, 0xa3, 0xee, 0x73, 0xec, 0x7f, 0x35, 0x4d, 0x63, + 0x13, 0x56, 0x2b, 0xc8, 0x50, 0x5e, 0x2f, 0xcd, 0x53, 0x45, 0xe3, 0x8c, + 0xd5, 0x16, 0xa9, 0xca, 0x6b, 0x56, 0xae, 0x51, 0x0a, 0x2b, 0xc5, 0xe0, + 0x70, 0xcd, 0xe0, 0x99, 0x0c, 0xc2, 0x96, 0x92, 0x29, 0xaa, 0xca, 0x4f, + 0x06, 0xe1, 0x5b, 0xe0, 0x58, 0x3c, 0x58, 0xd6, 0x1c, 0xba, 0xb4, 0xa3, + 0xad, 0x27, 0x13, 0xcb, 0xa7, 0xce, 0x68, 0xcc, 0x8d, 0x1f, 0xfd, 0x3c, + 0x4c, 0x3d, 0x84, 0xbe, 0x75, 0x47, 0x44, 0x0a, 0xc5, 0x0a, 0x1a, 0x0e, + 0x37, 0xa7, 0xc5, 0xcb, 0xc4, 0x85, 0xc1, 0x35, 0xc7, 0x73, 0xc7, 0xad, + 0x49, 0x93, 0x2d, 0xc6, 0x03, 0x45, 0xe1, 0x0d, 0xab, 0xb2, 0x72, 0x7f, + 0x3b, 0x49, 0x03, 0xa7, 0xa3, 0xa4, 0xdb, 0xab, 0x69, 0xd2, 0x37, 0xd3, + 0xfb, 0x60, 0xc5, 0x8f, 0xc1, 0x27, 0x4b, 0x27, 0x53, 0x27, 0x13, 0xd0, + 0xb3, 0xa8, 0x96, 0x95, 0x17, 0x97, 0x6f, 0x94, 0x23, 0xa8, 0xdd, 0xa8, + 0x8d, 0xab, 0xc6, 0x96, 0xe9, 0xab, 0xf8, 0x56, 0x4b, 0xa8, 0xb1, 0xab, + 0xec, 0x96, 0xc1, 0xcd, 0xa7, 0x54, 0xa8, 0x95, 0x67, 0x29, 0x9e, 0xa8, + 0x10, 0xa8, 0x98, 0x97, 0xf1, 0x95, 0x86, 0x95, 0x9b, 0x97, 0xda, 0xab, + 0x0a, 0x15, 0xfa, 0x69, 0xbd, 0x45, 0x7c, 0xd5, 0xea, 0x97, 0xed, 0x57, + 0x10, 0x2d, 0x54, 0xca, 0xad, 0x33, 0xd8, 0xed, 0x1c, 0x22, 0x45, 0x30, + 0xc6, 0x6b, 0xf0, 0x60, 0x24, 0xb3, 0x46, 0xaa, 0x08, 0x75, 0xfd, 0xcd, + 0x39, 0x40, 0x5d, 0x37, 0xb9, 0xec, 0xe3, 0x46, 0xdf, 0x91, 0xf6, 0x46, + 0xd4, 0x11, 0x25, 0x50, 0x5e, 0x96, 0xc7, 0x23, 0xca, 0x2e, 0xcd, 0xae, + 0xd1, 0xa0, 0x7c, 0x93, 0x64, 0x26, 0x04, 0xb3, 0x9b, 0xe6, 0xf8, 0xf8, + 0xd7, 0x45, 0x1e, 0x0e, 0x4c, 0x5b, 0x92, 0x7a, 0xe3, 0x17, 0xb9, 0x81, + 0x79, 0x41, 0xb7, 0x43, 0x47, 0x50, 0xcc, 0x21, 0xdb, 0xf6, 0xcd, 0x54, + 0xd7, 0xba, 0x3c, 0x25, 0x35, 0x45, 0xb3, 0x42, 0xb2, 0x42, 0xa9, 0xe2, + 0xe7, 0x09, 0x90, 0xab, 0xfa, 0xac, 0x24, 0xab, 0x18, 0x28, 0xde, 0xa1, + 0xf5, 0xf2, 0x1b, 0x59, 0x78, 0x58, 0x9a, 0x46, 0x97, 0xe6, 0xa7, 0x79, + 0xec, 0xd4, 0x97, 0x2c, 0xb0, 0xca, 0xb2, 0xca, 0x7a, 0x09, 0x7c, 0x09, + 0xb6, 0x9c, 0x00, 0xef, 0x5d, 0x35, 0x73, 0xb9, 0x75, 0x11, 0x41, 0x5e, + 0x87, 0x98, 0x7b, 0xb7, 0x5d, 0xc9, 0x79, 0x5a, 0x76, 0x97, 0x7e, 0x40, + 0xb6, 0xac, 0x7d, 0xc9, 0x86, 0x27, 0x10, 0xfc, 0x4c, 0xe8, 0xcb, 0xf2, + 0xa6, 0xfa, 0x66, 0xfb, 0x5c, 0x72, 0xf7, 0x13, 0x51, 0xf3, 0x6b, 0x03, + 0x22, 0xe5, 0xd7, 0x55, 0xc4, 0x29, 0x5c, 0xfe, 0x41, 0x4e, 0x47, 0x7b, + 0x7e, 0x22, 0x96, 0xe1, 0xd5, 0x2b, 0x25, 0x51, 0xbd, 0xfe, 0x1f, 0xa2, + 0x45, 0x86, 0xc9, 0x86, 0xa9, 0xdb, 0xc1, 0xf1, 0x6d, 0xb3, 0xfc, 0x05, + 0x0b, 0x12, 0xd1, 0x12, 0x28, 0x7f, 0x2c, 0x87, 0x74, 0x77, 0x29, 0x77, + 0x6d, 0x87, 0x6c, 0x87, 0x0a, 0xa8, 0xbf, 0x52, 0x93, 0x51, 0xcb, 0xc8, + 0x1d, 0xcb, 0x9d, 0x67, 0x28, 0xcb, 0xb5, 0x15, 0xd3, 0x15, 0xf3, 0xc9, + 0xf2, 0xc9, 0x0e, 0x4f, 0x4e, 0x4c, 0xe6, 0x4f, 0x2e, 0xa6, 0x73, 0x66, + 0x0a, 0x7c, 0x5b, 0x62, 0x03, 0x67, 0x82, 0x33, 0x97, 0x32, 0x37, 0x5b, + 0x7e, 0x69, 0x7b, 0x68, 0x33, 0x36, 0x97, 0x36, 0xb7, 0x9a, 0x4a, 0xd5, + 0xef, 0x69, 0x17, 0x6b, 0x9d, 0x37, 0x77, 0xda, 0x85, 0x68, 0xf3, 0x68, + 0x15, 0x69, 0xaa, 0x69, 0xeb, 0x35, 0x48, 0x54, 0x65, 0x55, 0x51, 0x2d, + 0x76, 0x54, 0x75, 0x94, 0xa0, 0x95, 0xb4, 0xc9, 0x27, 0xca, 0x57, 0x64, + 0x2f, 0x9a, 0xf7, 0xd7, 0x5e, 0x35, 0x8f, 0x36, 0x97, 0xb5, 0xe0, 0xd5, + 0xee, 0xff, 0xfb, 0xba, 0x5a, 0xdd, 0x6c, 0xdb, 0xec, 0x6b, 0x68, 0x6e, + 0x84, 0x63, 0xde, 0x32, 0x58, 0xdd, 0x7f, 0x9e, 0x08, 0x4c, 0xf4, 0x6d, + 0xf7, 0x8d, 0x7e, 0x13, 0x80, 0x41, 0x0a, 0x5b, 0x46, 0xae, 0x20, 0xd2, + 0x26, 0xf2, 0x73, 0x6e, 0x74, 0xc2, 0xe4, 0x55, 0xe7, 0xdd, 0xc8, 0x98, + 0xfa, 0x7b, 0x2e, 0xe4, 0x3b, 0x78, 0x9b, 0x72, 0xc4, 0x62, 0xae, 0x39, + 0x8d, 0x3a, 0xfd, 0x8f, 0x7d, 0xb7, 0x19, 0x63, 0x9c, 0x6e, 0xc0, 0x32, + 0x4b, 0x25, 0x13, 0xf4, 0xd3, 0x1c, 0xc7, 0xbc, 0xcd, 0x39, 0xe8, 0x26, + 0x3c, 0x2a, 0x42, 0x26, 0x70, 0x9c, 0x39, 0x55, 0x90, 0x39, 0x93, 0x60, + 0x68, 0x7d, 0x04, 0xb8, 0x27, 0xed, 0x72, 0xe7, 0xa8, 0xb7, 0xed, 0x35, + 0x77, 0x97, 0x95, 0xd9, 0x90, 0xd9, 0x77, 0x56, 0xba, 0x5a, 0x73, 0xce, + 0x75, 0x6e, 0x79, 0x1e, 0xb6, 0x96, 0x33, 0xc1, 0x91, 0x43, 0x97, 0xad, + 0xc5, 0xee, 0x3c, 0xd3, 0x3f, 0x22, 0xba, 0xa5, 0xb4, 0x3f, 0x2b, 0x84, + 0x90, 0x93, 0xcc, 0x6a, 0xbd, 0x59, 0xe9, 0xa7, 0x20, 0xaa, 0xf0, 0xdb, + 0xbc, 0x09, 0x4c, 0xc1, 0x62, 0xcc, 0x12, 0xaf, 0x51, 0xb5, 0x54, 0xdb, + 0x52, 0xe4, 0x5e, 0xfe, 0x60, 0x78, 0x81, 0x23, 0x78, 0x2f, 0xe8, 0x94, + 0xb3, 0x2b, 0xb8, 0xcb, 0xcf, 0xbd, 0xc1, 0xb6, 0xa1, 0x65, 0x30, 0xc1, + 0xdd, 0xd3, 0x32, 0xd2, 0x32, 0xae, 0xd0, 0x6f, 0x9f, 0x6d, 0x10, 0xb2, + 0x29, 0xea, 0x4f, 0xf1, 0x61, 0xf2, 0xe1, 0xf5, 0x31, 0x0b, 0x28, 0x09, + 0xa0, 0x82, 0x32, 0x87, 0x8e, 0xf8, 0x2d, 0xfc, 0x5b, 0xe7, 0xd3, 0xcb, + 0xa3, 0x97, 0x0f, 0x59, 0x77, 0x8c, 0x10, 0xef, 0xaa, 0xe9, 0x7d, 0xfd, + 0xf2, 0x99, 0xf0, 0x9f, 0xdc, 0x36, 0x61, 0x2c, 0xe1, 0xd8, 0xc0, 0xaa, + 0xc0, 0xfd, 0xde, 0x2a, 0x32, 0x8b, 0x91, 0xd3, 0xb8, 0x2a, 0xb2, 0x07, + 0x70, 0xda, 0x54, 0xfc, 0x94, 0x25, 0x72, 0x88, 0xa2, 0xdf, 0xe1, 0xe2, + 0x0c, 0x8c, 0xe1, 0x81, 0x61, 0x54, 0xbf, 0x1f, 0x22, 0x4d, 0x48, 0x58, + 0x88, 0x46, 0xa8, 0x50, 0x8c, 0x43, 0xb8, 0x26, 0xfe, 0xec, 0xb7, 0x54, + 0x4e, 0xf7, 0xd9, 0x35, 0x51, 0xe7, 0x4d, 0xb1, 0x67, 0xb1, 0x14, 0xb1, + 0x8d, 0x4c, 0xf3, 0x6f, 0x0a, 0x3c, 0x35, 0x4c, 0xd2, 0x9c, 0xf4, 0x82, + 0xe5, 0xcc, 0x56, 0xfc, 0x2b, 0xdc, 0xed, 0x1c, 0xed, 0x42, 0x43, 0xcc, + 0x66, 0x5a, 0x13, 0x0b, 0x9a, 0x59, 0x9a, 0x52, 0x36, 0x32, 0x84, 0x3c, + 0xcc, 0xf4, 0xd5, 0x5c, 0x76, 0x36, 0x5d, 0x9b, 0xaf, 0x67, 0x37, 0x61, + 0x3b, 0x59, 0x3b, 0x08, 0xbb, 0xde, 0x38, 0x69, 0x2c, 0x23, 0x7a, 0x0e, + 0x9e, 0x79, 0x53, 0x7d, 0xb7, 0x21, 0x25, 0xd8, 0x65, 0xcc, 0xe8, 0x8c, + 0xbc, 0x29, 0x3f, 0x64, 0xc3, 0xe5, 0xbf, 0x27, 0x5d, 0x0a, 0xa3, 0xd2, + 0x97, 0x62, 0xff, 0xa1, 0xba, 0xc6, 0x70, 0xa7, 0x8a, 0x81, 0xf7, 0xec, + 0x23, 0xea, 0x33, 0x27, 0x67, 0x99, 0xe5, 0x9a, 0x0d, 0xff, 0x27, 0xaa, + 0x4b, 0x4f, 0x2c, 0x7f, 0x2d, 0x49, 0x2f, 0xe7, 0x95, 0x70, 0x1a, 0xdf, + 0xc9, 0xe9, 0x49, 0x39, 0xc8, 0x5e, 0xc4, 0xb8, 0xdf, 0x16, 0x4c, 0x53, + 0x16, 0xd7, 0x29, 0x8b, 0x24, 0xbb, 0x9e, 0xc9, 0x9b, 0x8d, 0x9d, 0xe1, + 0x46, 0x6a, 0xcb, 0x26, 0xc0, 0x0b, 0xfc, 0x16, 0x94, 0x10, 0x1f, 0xe7, + 0x28, 0x52, 0x4e, 0x26, 0x97, 0xd8, 0x6c, 0x7e, 0x6c, 0x7e, 0x60, 0xe6, + 0x36, 0x5e, 0x69, 0x1a, 0x52, 0xff, 0x3e, 0x68, 0x17, 0xd5, 0xc2, 0x42, + 0x9c, 0x0c, 0x33, 0x6a, 0xdd, 0x10, 0x57, 0xf9, 0x45, 0x8b, 0x09, 0xbe, + 0x59, 0xa4, 0xf8, 0x50, 0x55, 0x9b, 0x76, 0x89, 0x1e, 0xa6, 0xde, 0xdc, + 0x54, 0xc3, 0xac, 0x7b, 0x32, 0x62, 0x8c, 0x27, 0xa5, 0x22, 0x9b, 0x61, + 0xf2, 0x6c, 0xa2, 0x61, 0x6c, 0x35, 0x8e, 0xfb, 0xef, 0x45, 0xce, 0xc7, + 0x19, 0x7a, 0x1f, 0xfa, 0x76, 0x42, 0x73, 0x51, 0xa8, 0x8c, 0x06, 0x13, + 0x91, 0xe6, 0x01, 0xb0, 0xd2, 0x8a, 0xa7, 0x9e, 0xdf, 0xd1, 0x73, 0x15, + 0x7e, 0x17, 0xeb, 0x0b, 0x43, 0xa8, 0xfa, 0xd7, 0x4a, 0xfe, 0xd6, 0xc8, + 0x66, 0x7a, 0xdb, 0x05, 0x85, 0x8a, 0x33, 0x8d, 0xd4, 0x6a, 0x3f, 0x20, + 0xb5, 0x97, 0x9c, 0xf5, 0x6f, 0xfd, 0xde, 0xb1, 0xc0, 0xde, 0x26, 0x28, + 0xeb, 0x13, 0x04, 0x35, 0xf8, 0xfd, 0x10, 0xdf, 0x93, 0xb5, 0xa5, 0x33, + 0x3d, 0x05, 0x4d, 0x16, 0xad, 0xcd, 0x26, 0xc3, 0xe7, 0xd0, 0x2c, 0x02, + 0x41, 0xac, 0x88, 0xc8, 0x6a, 0xcc, 0x55, 0x9c, 0x54, 0x2c, 0xff, 0x7f, + 0xb4, 0x6b, 0x1e, 0xfb, 0x04, 0x61, 0xf6, 0x63, 0xdd, 0xe9, 0x6b, 0x49, + 0x8d, 0x65, 0x15, 0x03, 0x6d, 0x26, 0xd3, 0x66, 0x73, 0x66, 0x47, 0x07, + 0x1b, 0x4c, 0xd9, 0x3a, 0xc8, 0x29, 0xe5, 0xa9, 0x07, 0xa9, 0xc8, 0x76, + 0x48, 0xcb, 0x2b, 0x8d, 0x48, 0x9b, 0xea, 0x8d, 0x67, 0x77, 0xf7, 0xe2, + 0xab, 0x2c, 0xa9, 0x93, 0xb4, 0xab, 0x1d, 0x41, 0xcd, 0xa2, 0x6d, 0x05, + 0xdd, 0xfc, 0xad, 0x9d, 0xa6, 0x25, 0xa3, 0x53, 0x93, 0x7c, 0x42, 0xaa, + 0x85, 0xaa, 0xb7, 0xaa, 0xcf, 0xaa, 0x5c, 0xe7, 0x73, 0xeb, 0x4d, 0x86, + 0x4b, 0x8f, 0x76, 0xb3, 0xe7, 0x04, 0xde, 0x9d, 0xf7, 0x1c, 0xa7, 0x0b, + 0xab, 0x3a, 0x9d, 0x0f, 0x6f, 0x82, 0xa3, 0x5d, 0xab, 0x6a, 0x50, 0x6a, + 0x72, 0x0b, 0x66, 0x0b, 0xa6, 0x2f, 0x52, 0x2f, 0x0d, 0x0f, 0x42, 0xd5, + 0x11, 0xe5, 0x47, 0x8f, 0x29, 0x2f, 0xb6, 0x7a, 0x55, 0xf2, 0xb6, 0x8e, + 0x67, 0xc3, 0xd0, 0x73, 0xd0, 0x6f, 0x30, 0x14, 0xc8, 0xcd, 0x4b, 0x6d, + 0xf3, 0x0b, 0x44, 0x73, 0x02, 0xaa, 0xca, 0xb1, 0x8b, 0x0c, 0x27, 0x3f, + 0x91, 0x87, 0x49, 0xba, 0xde, 0xb3, 0x2e, 0xdc, 0x99, 0x90, 0xb8, 0x90, + 0xd4, 0xa2, 0xff, 0x0e, 0xed, 0x50, 0x38, 0x88, 0x79, 0xee, 0x1d, 0x3d, + 0xc0, 0x6d, 0x3e, 0x10, 0x23, 0xf1, 0xae, 0xb8, 0xfd, 0xf1, 0xae, 0xdc, + 0x86, 0x01, 0x91, 0xb8, 0x11, 0x99, 0x13, 0xb1, 0x58, 0x89, 0x64, 0x5c, + 0x67, 0x48, 0x64, 0xaa, 0x64, 0xbc, 0xe8, 0x18, 0xe9, 0xb4, 0x2b, 0x4c, + 0x08, 0xce, 0x2e, 0x1e, 0x6f, 0x78, 0xbd, 0xfe, 0x5c, 0x25, 0x41, 0xf9, + 0xcb, 0xc1, 0xaf, 0xe5, 0x29, 0x7d, 0xa3, 0x3a, 0x6a, 0xb9, 0x72, 0x14, + 0x08, 0x4a, 0x34, 0xbf, 0x9e, 0xfe, 0x93, 0x95, 0x9c, 0xe5, 0xeb, 0xc7, + 0xfc, 0xb4, 0xf6, 0x6b, 0x35, 0xbf, 0xd9, 0x1b, 0x6c, 0x3c, 0xb5, 0x69, + 0x85, 0x62, 0x7f, 0xd1, 0x8e, 0xea, 0xb2, 0x3d, 0x27, 0xfe, 0x11, 0x05, + 0xe6, 0xeb, 0xe4, 0x9b, 0xd2, 0xbd, 0x6e, 0x75, 0xb9, 0xae, 0x5f, 0xdd, + 0xc7, 0xd0, 0x68, 0xd5, 0xc8, 0xa8, 0xe1, 0x35, 0xb0, 0x3d, 0x76, 0x39, + 0xe6, 0x7f, 0x4f, 0x04, 0x29, 0x0c, 0x77, 0xfc, 0xed, 0xc8, 0xa8, 0x6e, + 0xd3, 0xb5, 0x6e, 0x3b, 0xf7, 0x02, 0x36, 0x79, 0x9b, 0x3d, 0x1c, 0x11, + 0x64, 0x73, 0x4f, 0xad, 0xdf, 0x02, 0x3d, 0xaf, 0xb0, 0x3f, 0x16, 0x17, + 0xb2, 0x79, 0x04, 0xd6, 0x81, 0x37, 0x5e, 0xf6, 0x0f, 0x83, 0x0f, 0x77, + 0x9b, 0xd8, 0xeb, 0x7e, 0x4e, 0x34, 0x6f, 0x53, 0xe3, 0x1d, 0xcb, 0xee, + 0x34, 0xe7, 0x55, 0x9d, 0x0a, 0x9d, 0x06, 0xb7, 0x3f, 0xce, 0x84, 0x97, + 0x45, 0xd7, 0xc3, 0x16, 0x84, 0x74, 0xde, 0xd7, 0xfe, 0x5d, 0xac, 0xf9, + 0x5b, 0xf5, 0xb8, 0xe7, 0xde, 0xcc, 0xae, 0xfd, 0x7c, 0xf8, 0x1a, 0x12, + 0x34, 0xfd, 0x1e, 0x7f, 0x17, 0x8b, 0x22, 0xfa, 0x25, 0x36, 0x3d, 0x76, + 0x3c, 0x14, 0x29, 0xf4, 0x24, 0xfb, 0xbb, 0x20, 0x0f, 0xcf, 0x8e, 0x90, + 0xd3, 0xdb, 0xd1, 0x6b, 0x89, 0xe8, 0xb4, 0xc9, 0x34, 0x75, 0x36, 0xce, + 0x7b, 0xfe, 0xfb, 0x78, 0x35, 0x0e, 0x36, 0x8b, 0xdf, 0xe3, 0xc2, 0xc9, + 0xd2, 0xd4, 0x6c, 0xda, 0x6c, 0x5e, 0xea, 0x75, 0x6a, 0x62, 0xb7, 0xad, + 0x4f, 0xd1, 0xdb, 0x71, 0xcf, 0x09, 0x5e, 0xcd, 0xea, 0x50, 0xdf, 0x10, + 0x71, 0xd7, 0xa8, 0x07, 0xfb, 0x37, 0xfd, 0x5b, 0xf5, 0x97, 0xb3, 0x5e, + 0xf7, 0x35, 0x77, 0xa2, 0x8b, 0x82, 0x4d, 0x14, 0x85, 0x3d, 0x1f, 0x52, + 0x7f, 0xda, 0x0f, 0x8b, 0xa7, 0xba, 0x3b, 0x95, 0x99, 0x92, 0x97, 0x44, + 0xbc, 0xc4, 0xc4, 0x8f, 0xcf, 0xd0, 0x69, 0x37, 0xaf, 0xc6, 0xec, 0x4c, + 0xab, 0x10, 0x87, 0xf9, 0xd5, 0x48, 0xe6, 0xa7, 0x18, 0xe0, 0xe3, 0xbf, + 0x57, 0x2e, 0xb0, 0x24, 0x11, 0xb2, 0x07, 0xce, 0x41, 0x52, 0xcb, 0x19, + 0x00, 0xd0, 0x95, 0xf8, 0x6f, 0x83, 0xc2, 0x4c, 0x12, 0x15, 0x86, 0xec, + 0x84, 0x71, 0x91, 0x92, 0x13, 0x87, 0x59, 0x81, 0xc1, 0x81, 0xe2, 0x64, + 0x40, 0x99, 0x71, 0x27, 0x06, 0x00, 0x48, 0x01, 0x52, 0x62, 0xc2, 0xaa, + 0x1e, 0x2b, 0xa7, 0x59, 0xee, 0x1e, 0x98, 0x6a, 0x0f, 0xc3, 0x42, 0xee, + 0x66, 0xcd, 0x85, 0x34, 0x71, 0xc1, 0x33, 0x70, 0xf1, 0xc8, 0x64, 0x92, + 0x68, 0x8a, 0x62, 0x62, 0x4a, 0x74, 0x91, 0xb0, 0x81, 0x91, 0xd1, 0x1a, + 0x83, 0x22, 0x68, 0x3e, 0xb6, 0xe1, 0xe1, 0xb0, 0xf4, 0x91, 0xe1, 0x91, + 0xb4, 0xf4, 0xe4, 0xcf, 0x2a, 0x8a, 0xdb, 0x45, 0xb0, 0x9a, 0x47, 0xf0, + 0xb0, 0x92, 0xf1, 0xd4, 0x92, 0x70, 0x40, 0x4a, 0x4a, 0x32, 0x5b, 0xee, + 0x0e, 0x76, 0x67, 0x7f, 0x7f, 0x9b, 0xfa, 0xaf, 0x96, 0x50, 0xe8, 0xb0, + 0x99, 0x4f, 0x82, 0x28, 0x28, 0x1d, 0x99, 0x74, 0xee, 0x53, 0x7f, 0x2f, + 0x04, 0xc0, 0x6b, 0x4e, 0xc8, 0x9a, 0xb0, 0xdf, 0x4b, 0xad, 0x5e, 0x2d, + 0x3f, 0xd4, 0x2b, 0x52, 0x30, 0x23, 0xd1, 0x0d, 0x88, 0xfb, 0x30, 0x30, + 0xa9, 0x9a, 0x53, 0x50, 0x5d, 0x29, 0x18, 0x23, 0xbe, 0x08, 0x06, 0x53, + 0x05, 0x8e, 0x13, 0x70, 0xe9, 0x08, 0x74, 0x6e, 0x54, 0x85, 0xf8, 0x45, + 0x44, 0x63, 0x91, 0x9f, 0xd0, 0xd3, 0x99, 0xea, 0xb1, 0xe2, 0xc3, 0xe5, + 0x49, 0xc8, 0x1c, 0x95, 0x01, 0x38, 0x00, 0x20, 0xc7, 0x50, 0x86, 0xb7, + 0xe8, 0x19, 0x33, 0xf8, 0x09, 0x24, 0x91, 0xe0, 0x1a, 0xee, 0xc9, 0x24, + 0xa6, 0x2f, 0xfe, 0x11, 0x91, 0xed, 0xf1, 0xb7, 0xb7, 0x39, 0x51, 0x4d, + 0xeb, 0x91, 0xca, 0x48, 0xa0, 0x35, 0x6b, 0xe0, 0x8f, 0xa8, 0x43, 0xec, + 0xa9, 0x50, 0x6c, 0x26, 0xe4, 0xa8, 0x85, 0x8a, 0x13, 0x4e, 0xd4, 0xbe, + 0x56, 0x10, 0x87, 0x33, 0x6f, 0xa0, 0x0d, 0x3c, 0xf5, 0x02, 0xc3, 0xf0, + 0x8e, 0xcb, 0xbf, 0xba, 0x55, 0x9b, 0xe0, 0xcf, 0x7c, 0x11, 0x91, 0xb0, + 0x14, 0x75, 0x76, 0xfc, 0x48, 0xd6, 0x4b, 0xa4, 0x75, 0xfa, 0x72, 0x7d, + 0x22, 0x41, 0x8f, 0xd6, 0xc3, 0x82, 0x1c, 0x6a, 0x83, 0x86, 0x2d, 0x86, + 0x92, 0x89, 0x74, 0xec, 0x7e, 0x66, 0x4b, 0xd9, 0x41, 0x6c, 0x4b, 0xc9, + 0x34, 0x6b, 0xc4, 0x28, 0x99, 0x2d, 0x17, 0xd0, 0x36, 0x32, 0x21, 0x33, + 0x1b, 0x2a, 0x98, 0x2f, 0x12, 0xfd, 0x92, 0x9b, 0xea, 0xba, 0x67, 0xcd, + 0xab, 0x04, 0x61, 0x9e, 0xfc, 0xfe, 0xc3, 0x9f, 0xb8, 0x03, 0x5f, 0xa0, + 0x37, 0x6b, 0x56, 0x62, 0x45, 0xbe, 0x27, 0x60, 0x0d, 0x0a, 0x52, 0x50, + 0xee, 0xa5, 0x50, 0xa5, 0xa7, 0x7f, 0x80, 0xa1, 0x0e, 0x41, 0xc3, 0xe3, + 0x25, 0x39, 0x2b, 0x95, 0xa9, 0xb6, 0x68, 0xc0, 0x4a, 0x32, 0x6a, 0x11, + 0xa5, 0x3f, 0x62, 0x74, 0xeb, 0x2c, 0x52, 0xe9, 0xab, 0xc4, 0xb9, 0xba, + 0x26, 0x69, 0x95, 0xb8, 0x3a, 0xfa, 0x2e, 0x6d, 0x79, 0x54, 0xb1, 0xb7, + 0xc7, 0xe1, 0x0c, 0xd3, 0x41, 0x1c, 0x54, 0xc5, 0x49, 0x69, 0xc5, 0x11, + 0x49, 0xe8, 0x64, 0x62, 0x64, 0xd6, 0xf4, 0xf2, 0x1e, 0x6f, 0xca, 0xd2, + 0x85, 0x49, 0x5d, 0x06, 0xc1, 0x3f, 0x60, 0x85, 0xab, 0xf9, 0x62, 0x85, + 0x29, 0x04, 0x7f, 0x81, 0xb3, 0x83, 0x08, 0x05, 0x52, 0xe2, 0xfd, 0x6d, + 0x66, 0x74, 0x51, 0x29, 0x54, 0x39, 0xed, 0x2a, 0x0c, 0x4f, 0x27, 0x04, + 0xcd, 0x4d, 0x31, 0x0c, 0x27, 0x7b, 0xd3, 0xe8, 0x4c, 0xed, 0xf6, 0x5f, + 0x73, 0xf6, 0xba, 0x5d, 0x6a, 0x67, 0xdc, 0x5f, 0x90, 0x33, 0x99, 0x9e, + 0xa9, 0xd4, 0x6c, 0x6f, 0x9b, 0x56, 0x81, 0x6b, 0x01, 0x8c, 0x27, 0x0b, + 0x3a, 0x62, 0x28, 0x8c, 0x16, 0xda, 0x9f, 0xda, 0xbb, 0x1a, 0x02, 0x08, + 0xf7, 0x39, 0x98, 0xe8, 0xa5, 0x96, 0x40, 0xc4, 0x71, 0x73, 0xae, 0x2c, + 0xb8, 0xb4, 0x91, 0xa4, 0xc5, 0xba, 0x9f, 0xa7, 0xf9, 0x57, 0xa2, 0x32, + 0xab, 0x6d, 0x23, 0xc9, 0x3f, 0x13, 0xd7, 0xf8, 0xf5, 0x71, 0x16, 0x26, + 0x3d, 0x05, 0x63, 0xa7, 0x15, 0x4b, 0xa9, 0xc7, 0x44, 0xe9, 0x6a, 0x10, + 0x67, 0xb6, 0xd0, 0x34, 0x58, 0xec, 0x7f, 0x15, 0x09, 0x24, 0x24, 0x94, + 0x96, 0x12, 0x45, 0x0b, 0xe4, 0x85, 0x5b, 0x68, 0x05, 0x8a, 0x5a, 0xd3, + 0x8b, 0xfd, 0xcc, 0xa3, 0x87, 0x37, 0x13, 0xbc, 0x0a, 0x41, 0xfd, 0xc7, + 0x81, 0x55, 0x63, 0x58, 0x7b, 0xd9, 0x72, 0xe7, 0xf3, 0x7a, 0xe7, 0xf3, + 0xbe, 0x52, 0x87, 0x43, 0x27, 0x0c, 0x2a, 0xdf, 0xfc, 0x22, 0xcc, 0xee, + 0xfa, 0x56, 0xd3, 0xc9, 0xfe, 0x07, 0x1b, 0xf0, 0x95, 0x4b, 0x67, 0xf0, + 0x40, 0xdf, 0x7e, 0x8c, 0x4d, 0x8d, 0x93, 0x48, 0x09, 0x2d, 0x6f, 0x20, + 0x29, 0x76, 0x6f, 0xf1, 0xbe, 0xca, 0x72, 0xb9, 0x90, 0x19, 0xb9, 0x23, + 0x7a, 0xe6, 0x17, 0x4d, 0x77, 0xf5, 0xf2, 0x9f, 0x7e, 0x43, 0xf0, 0x99, + 0x14, 0x68, 0xd9, 0x9f, 0xb6, 0xc9, 0x41, 0x32, 0x9e, 0x5e, 0x4c, 0x55, + 0x8d, 0x2e, 0xb6, 0x9d, 0x69, 0xb2, 0x36, 0xdc, 0x6f, 0xb9, 0xff, 0xc8, + 0x26, 0x70, 0x15, 0xc8, 0x9c, 0xb3, 0x86, 0x26, 0x2f, 0x0e, 0xd7, 0x06, + 0x62, 0x8d, 0x91, 0x83, 0x22, 0x89, 0x3d, 0xe5, 0x8a, 0x26, 0xce, 0xaa, + 0xde, 0xef, 0x3d, 0xe7, 0x46, 0xc3, 0x51, 0x41, 0xee, 0xe7, 0x60, 0x9d, + 0xc4, 0x60, 0x72, 0x35, 0xdc, 0xdb, 0x6b, 0x7f, 0xba, 0x5d, 0x25, 0xd1, + 0x6a, 0x70, 0xe4, 0x23, 0x68, 0x3e, 0x65, 0x74, 0xe2, 0x2b, 0x57, 0x8f, + 0xdc, 0x2f, 0x82, 0x02, 0x5a, 0xea, 0x0a, 0x41, 0x65, 0x16, 0x67, 0xef, + 0x21, 0xa5, 0x15, 0x8f, 0xa2, 0xc8, 0x2c, 0xb2, 0x3a, 0xbd, 0x6a, 0xec, + 0xb5, 0x5f, 0x2b, 0x26, 0xa5, 0x21, 0x8b, 0xa4, 0x75, 0xe3, 0x4b, 0x94, + 0x9f, 0x6c, 0x2e, 0x59, 0x2d, 0x58, 0x61, 0x33, 0x68, 0x6c, 0x34, 0xc9, + 0x49, 0x6b, 0x9c, 0x45, 0xd6, 0x29, 0xf9, 0x08, 0xed, 0xbb, 0x8a, 0xbd, + 0x73, 0xca, 0x2f, 0xd9, 0x05, 0x24, 0x35, 0xee, 0x22, 0x12, 0xb1, 0xa7, + 0xab, 0x11, 0xbc, 0x8b, 0x48, 0x53, 0xc2, 0x1a, 0x36, 0x2c, 0x1c, 0xa2, + 0x13, 0x6f, 0xd8, 0x6d, 0xd4, 0x19, 0x3b, 0x53, 0x16, 0x6a, 0xc3, 0x1b, + 0xa3, 0xbd, 0x5d, 0x8b, 0x68, 0x6a, 0x5a, 0xd7, 0xd7, 0xa7, 0x14, 0x58, + 0x72, 0xa2, 0xf5, 0xad, 0xfe, 0x58, 0x52, 0x59, 0x94, 0x39, 0x5a, 0x04, + 0xe0, 0x61, 0xe3, 0x33, 0x0a, 0xfa, 0xd9, 0xa2, 0x36, 0xed, 0x30, 0x14, + 0x94, 0x1d, 0x5b, 0xc2, 0x29, 0xa7, 0xfd, 0x21, 0xae, 0x91, 0x1f, 0x86, + 0x09, 0x8d, 0xcb, 0xc8, 0x7e, 0xaa, 0xe9, 0x65, 0x09, 0x81, 0xd4, 0x8f, + 0x5d, 0xeb, 0xf6, 0xd7, 0x0c, 0x2e, 0xdb, 0xb1, 0x72, 0xb5, 0x8a, 0x7f, + 0xed, 0xae, 0x0c, 0xab, 0xcd, 0xb6, 0x7c, 0xbd, 0xae, 0x2c, 0x7f, 0x94, + 0x14, 0x1e, 0xf2, 0x54, 0x3a, 0xb0, 0x4b, 0x80, 0x06, 0x12, 0x66, 0x5c, + 0x4b, 0x77, 0xcf, 0xf6, 0xef, 0x35, 0xc2, 0x18, 0x67, 0xf7, 0x56, 0x6f, + 0x02, 0xd9, 0xb6, 0x1b, 0x22, 0x8b, 0x8f, 0x28, 0x97, 0x5c, 0x78, 0x97, + 0x5b, 0x67, 0x3b, 0xdb, 0xd9, 0x2d, 0xbf, 0xd2, 0x0e, 0x78, 0x46, 0x87, + 0xb2, 0xc0, 0x86, 0x5b, 0xe3, 0x18, 0x8f, 0xbe, 0x6f, 0x06, 0x4f, 0xdd, + 0x9f, 0x4b, 0x7e, 0xd8, 0x31, 0xfe, 0x90, 0x94, 0x24, 0x70, 0x72, 0x4a, + 0xd1, 0x82, 0x37, 0x41, 0x7b, 0x73, 0xaa, 0xdd, 0x41, 0x72, 0xe8, 0xe0, + 0x2e, 0xb0, 0x54, 0x6b, 0x3f, 0x7e, 0xfe, 0x5d, 0x23, 0xbc, 0x02, 0x95, + 0x7d, 0xee, 0x4d, 0x03, 0x6c, 0xd7, 0x0d, 0xcb, 0xb8, 0xf0, 0xfa, 0xc8, + 0x9c, 0xa0, 0x19, 0xee, 0xaa, 0x75, 0x9e, 0x26, 0xcb, 0xf6, 0x37, 0x46, + 0x5f, 0x75, 0x27, 0xf9, 0xfe, 0xbb, 0xb1, 0xfb, 0xf5, 0x1a, 0x61, 0x49, + 0xdf, 0x9b, 0x29, 0x32, 0x3f, 0xb7, 0xfc, 0x58, 0x0a, 0x9c, 0x51, 0x7c, + 0xf4, 0xf1, 0x30, 0xce, 0xd2, 0xcd, 0xea, 0xf7, 0xea, 0xd1, 0xf5, 0xf1, + 0xfe, 0x82, 0x11, 0xe3, 0x59, 0x3b, 0x3c, 0x89, 0x78, 0xea, 0xcc, 0x57, + 0xaf, 0xf8, 0x53, 0x65, 0x86, 0x18, 0xc4, 0x01, 0x89, 0x79, 0x0e, 0xae, + 0x90, 0x29, 0x76, 0x00, 0x26, 0xb9, 0x51, 0x40, 0xd2, 0xf0, 0x30, 0x83, + 0xf4, 0x1e, 0x4e, 0x71, 0xfa, 0xec, 0x0b, 0x98, 0x45, 0x02, 0x43, 0x45, + 0x70, 0xe7, 0xcf, 0x99, 0x10, 0x46, 0x61, 0x12, 0xe1, 0xf8, 0xc8, 0x5e, + 0xa8, 0x77, 0xf6, 0x64, 0xec, 0x26, 0x96, 0xbb, 0x89, 0x49, 0xec, 0xc8, + 0x9f, 0x3f, 0x35, 0x37, 0x8c, 0x48, 0xf5, 0xe1, 0xfa, 0xe5, 0x68, 0x38, + 0x0f, 0xfc, 0x61, 0x28, 0xdd, 0xa1, 0xc5, 0x0b, 0x9d, 0x4a, 0x4b, 0x89, + 0xdd, 0xd0, 0xa0, 0xed, 0xda, 0xd5, 0x47, 0x9f, 0xd3, 0x86, 0xf9, 0x4a, + 0xf5, 0xab, 0x24, 0xe6, 0x9b, 0x23, 0xc9, 0x5f, 0x2d, 0xab, 0x87, 0x7c, + 0x26, 0x84, 0xdd, 0xd5, 0x1b, 0x2e, 0x7d, 0x9b, 0xd9, 0xbe, 0xf7, 0x5a, + 0x12, 0x31, 0x84, 0x11, 0xc5, 0x96, 0x60, 0x59, 0xb6, 0xd8, 0xe1, 0xfe, + 0x8b, 0xee, 0x8f, 0xa9, 0x99, 0x3c, 0x51, 0xad, 0x76, 0x97, 0xb2, 0x8f, + 0x07, 0xcb, 0xdd, 0x30, 0x65, 0x96, 0x3c, 0xf8, 0x12, 0xd6, 0xf7, 0x3d, + 0xcf, 0xce, 0x5b, 0x8f, 0x57, 0xac, 0xb6, 0x87, 0x2d, 0xa5, 0x8f, 0xe8, + 0x0d, 0xfc, 0xe7, 0x52, 0x9b, 0x4d, 0xde, 0x32, 0x4d, 0x97, 0x09, 0x0e, + 0x39, 0x3f, 0xbd, 0x86, 0x74, 0xa8, 0xfe, 0x4d, 0x98, 0x60, 0x3c, 0xc2, + 0x35, 0x66, 0x41, 0x6e, 0xf7, 0x3c, 0x4e, 0x21, 0x73, 0x34, 0xe0, 0x2d, + 0x39, 0xbd, 0xd5, 0x0f, 0x4d, 0xaf, 0xe7, 0x17, 0x9d, 0x9e, 0x52, 0xe5, + 0x36, 0xa3, 0x46, 0x9f, 0xdd, 0xb6, 0x30, 0xf1, 0x37, 0x86, 0xf2, 0x2c, + 0x0c, 0x3c, 0x84, 0x91, 0xd5, 0xf7, 0x6c, 0xf3, 0xef, 0xea, 0x23, 0x72, + 0xfe, 0xa7, 0x75, 0xc3, 0xb8, 0x02, 0x5e, 0x1c, 0x86, 0x86, 0x86, 0x38, + 0x26, 0xe2, 0x08, 0x53, 0xe3, 0xe3, 0xe3, 0x05, 0xda, 0x4d, 0x56, 0x84, + 0x02, 0x5e, 0x98, 0x01, 0x99, 0xab, 0x5c, 0xbb, 0xeb, 0xa9, 0xd9, 0xa4, + 0xc3, 0xfb, 0x82, 0xdb, 0x2b, 0x5d, 0xbd, 0x42, 0x21, 0xdd, 0x6e, 0xe7, + 0x45, 0xe6, 0xdd, 0x4a, 0x00, 0x0e, 0x99, 0xca, 0xc9, 0x79, 0xf7, 0xb0, + 0xb8, 0xac, 0x5d, 0xde, 0x32, 0xa2, 0x2f, 0xfd, 0x27, 0x91, 0xe4, 0xac, + 0x72, 0x11, 0x49, 0x82, 0xd3, 0x7b, 0x36, 0xb6, 0xda, 0xed, 0x56, 0xc6, + 0x82, 0xcb, 0x58, 0x7a, 0xe1, 0xb5, 0x33, 0xe1, 0xe1, 0xcc, 0x9e, 0x78, + 0x78, 0x1f, 0xe6, 0x74, 0xc9, 0xe2, 0xb7, 0x32, 0xb1, 0x82, 0xbe, 0xfc, + 0x3e, 0xcf, 0x65, 0x44, 0x00, 0xb5, 0xe8, 0xd7, 0x19, 0x56, 0xb6, 0x67, + 0x64, 0x82, 0x99, 0xcc, 0x8a, 0x4c, 0xce, 0x42, 0xa7, 0x90, 0xeb, 0xc3, + 0x50, 0xc7, 0x29, 0x31, 0x59, 0xbe, 0x6c, 0xb9, 0xd4, 0x0f, 0x44, 0xaa, + 0xa5, 0xfd, 0x9b, 0x5f, 0x34, 0xf2, 0x7f, 0x25, 0x2f, 0x36, 0x80, 0x70, + 0xc8, 0xc8, 0x82, 0xf6, 0x81, 0x39, 0xe7, 0xfa, 0xca, 0xd2, 0xcd, 0x36, + 0xcb, 0x22, 0x1d, 0x6e, 0x67, 0x92, 0x72, 0x72, 0x54, 0x1c, 0xdf, 0xc3, + 0xa9, 0x0c, 0xba, 0x43, 0xf2, 0xc3, 0x55, 0x36, 0xc0, 0x8a, 0xd2, 0xbc, + 0x44, 0x24, 0xb8, 0x81, 0x49, 0x09, 0xc7, 0x6a, 0xf4, 0x97, 0x6f, 0x7b, + 0x4a, 0x90, 0x08, 0x3d, 0x4d, 0xd7, 0xc4, 0x25, 0x9e, 0x1c, 0xec, 0x5a, + 0x6e, 0x36, 0xb8, 0x0e, 0x13, 0x23, 0x97, 0x14, 0xaa, 0x30, 0xb3, 0xfd, + 0x25, 0x26, 0xaa, 0x29, 0x92, 0x79, 0xa9, 0xb8, 0xf4, 0xb8, 0x03, 0x35, + 0x03, 0xb7, 0xda, 0x42, 0x3b, 0x9f, 0xe8, 0x6e, 0x70, 0xaa, 0x12, 0x32, + 0xb0, 0x5f, 0x72, 0xab, 0x49, 0xb2, 0x8c, 0x2c, 0xd3, 0xc4, 0x6f, 0x95, + 0x83, 0xe3, 0x5a, 0xc8, 0xc0, 0x1b, 0xde, 0x1d, 0xeb, 0x9a, 0x7e, 0x8a, + 0x39, 0xab, 0x45, 0xe5, 0xe5, 0x3b, 0x36, 0x05, 0xd3, 0x25, 0x2a, 0xf7, + 0x58, 0x27, 0xef, 0x3b, 0x11, 0x38, 0x9b, 0xc2, 0xd2, 0x16, 0x73, 0x85, + 0x70, 0x37, 0xe0, 0x69, 0x1a, 0x66, 0xe6, 0x6a, 0x31, 0x1d, 0x83, 0x1f, + 0xf5, 0x53, 0x59, 0x7c, 0xec, 0xf6, 0x1b, 0xfc, 0x56, 0xaa, 0x65, 0x3d, + 0x28, 0x2a, 0xfa, 0xfd, 0x59, 0x60, 0x9e, 0xfd, 0xab, 0x30, 0xe5, 0xb9, + 0xb6, 0xfd, 0x77, 0x30, 0x54, 0xa9, 0x5a, 0x01, 0x9b, 0x90, 0xdf, 0x3b, + 0x7e, 0xbb, 0xe1, 0xe5, 0xbf, 0x19, 0x77, 0xad, 0xcb, 0xd5, 0x76, 0x91, + 0x5d, 0x4c, 0xf9, 0x35, 0xbb, 0xcd, 0x09, 0xad, 0xb3, 0x47, 0xfa, 0xd7, + 0xcd, 0x56, 0x70, 0xad, 0xe5, 0x31, 0xde, 0xfe, 0xce, 0xd5, 0x71, 0x63, + 0x6b, 0x15, 0xc9, 0xaf, 0x4c, 0xa4, 0x8d, 0xea, 0x14, 0x3b, 0x2d, 0xb1, + 0x6e, 0x13, 0x87, 0xef, 0x3b, 0xa3, 0xe3, 0xab, 0xd1, 0xe5, 0xea, 0x67, + 0x16, 0xb8, 0x2f, 0x5d, 0xb6, 0x0b, 0x85, 0xfa, 0x97, 0x5b, 0x61, 0xed, + 0xe3, 0xed, 0x70, 0x6a, 0x21, 0x76, 0xb8, 0xe0, 0x33, 0xe5, 0x8b, 0x35, + 0xef, 0xf0, 0xf3, 0xd8, 0xe8, 0xcb, 0xc1, 0x09, 0x50, 0xf1, 0x6c, 0x83, + 0xf9, 0xb4, 0x51, 0xd3, 0x96, 0x79, 0x87, 0xbf, 0xbe, 0x3e, 0x71, 0x32, + 0x83, 0x06, 0x35, 0x13, 0x53, 0xf4, 0xd4, 0x99, 0x0e, 0xdb, 0x71, 0x37, + 0x4a, 0xb6, 0xd3, 0x00, 0x31, 0x1d, 0x8f, 0xd3, 0xe9, 0x77, 0xee, 0xcd, + 0x0f, 0x7e, 0x65, 0x69, 0x91, 0xa0, 0x2c, 0x48, 0x68, 0x3c, 0x5c, 0x73, + 0x1e, 0xc8, 0x83, 0x7f, 0x48, 0x8f, 0x2e, 0x56, 0xe9, 0x40, 0x62, 0x4b, + 0x2d, 0x28, 0x08, 0x49, 0x74, 0xcb, 0x95, 0x52, 0x2e, 0x95, 0x21, 0x7e, + 0x5f, 0x09, 0x96, 0x9f, 0xd2, 0xa3, 0x0b, 0xeb, 0x30, 0xdb, 0x4e, 0x51, + 0x05, 0xca, 0xed, 0xf4, 0x9a, 0x2b, 0x6e, 0x6e, 0xb4, 0x29, 0xb0, 0xd2, + 0x5f, 0x91, 0x08, 0xfc, 0x0a, 0xe1, 0xb7, 0x1f, 0x67, 0x59, 0xef, 0x27, + 0xbd, 0x2f, 0xf6, 0xfd, 0x84, 0x13, 0x37, 0x3c, 0x22, 0xb5, 0xaf, 0x92, + 0x43, 0x1c, 0x89, 0xd5, 0x3b, 0x01, 0x32, 0x1a, 0xb6, 0xd7, 0x89, 0x3e, + 0x5b, 0x3d, 0xbf, 0x6f, 0x4f, 0x97, 0x41, 0x0a, 0x19, 0x51, 0x46, 0x58, + 0xc1, 0x3d, 0xdb, 0x0e, 0xb5, 0xee, 0x35, 0xf6, 0x88, 0x36, 0xe7, 0x81, + 0xc9, 0xaf, 0xad, 0x2e, 0x93, 0x65, 0x11, 0xcc, 0x9f, 0x1d, 0xbf, 0x29, + 0xf1, 0xcf, 0x00, 0xa1, 0x01, 0xcb, 0x6d, 0x76, 0x42, 0xa3, 0x50, 0xa7, + 0x6e, 0x7c, 0x3a, 0x2d, 0x76, 0x45, 0xfa, 0x40, 0xaf, 0x35, 0xc7, 0x61, + 0x22, 0xda, 0x48, 0x05, 0xac, 0x8f, 0x7f, 0xad, 0x59, 0x45, 0x15, 0xef, + 0xfc, 0x77, 0x9c, 0x08, 0x4d, 0x05, 0xc6, 0x1d, 0xd8, 0x92, 0x47, 0x91, + 0xae, 0xbc, 0x7c, 0x9a, 0x27, 0xc6, 0x4c, 0x51, 0x7e, 0x8e, 0x86, 0x0d, + 0x83, 0x5b, 0x77, 0xc5, 0xa4, 0x62, 0x16, 0x24, 0x74, 0x7a, 0x89, 0xe4, + 0x14, 0x81, 0x7b, 0x7f, 0x63, 0x7f, 0x86, 0x5f, 0xc6, 0xfd, 0xa2, 0x04, + 0xdb, 0xbb, 0x16, 0x25, 0xbd, 0x76, 0xf9, 0x11, 0xc4, 0x76, 0xb1, 0x4e, + 0x81, 0x77, 0x16, 0xab, 0xcf, 0x72, 0x3e, 0xba, 0x34, 0x29, 0x3f, 0xb8, + 0xdd, 0x2e, 0x36, 0xe4, 0x26, 0x33, 0x38, 0xa1, 0x1e, 0xef, 0x56, 0x5b, + 0xe0, 0x90, 0xe0, 0x78, 0xf7, 0x1d, 0x3d, 0x89, 0x59, 0xdc, 0xf9, 0x79, + 0x03, 0xdf, 0xc2, 0x54, 0xe4, 0x9f, 0x1e, 0x20, 0xd5, 0x95, 0xad, 0x63, + 0xd7, 0xb6, 0x1f, 0xbc, 0x69, 0xa5, 0x5e, 0x31, 0xf2, 0x3a, 0x5a, 0x20, + 0x4f, 0x80, 0x0d, 0x98, 0xd0, 0x79, 0xe4, 0x63, 0xf9, 0x93, 0x8e, 0xf8, + 0x79, 0x58, 0xf8, 0x1c, 0x4d, 0xe6, 0x85, 0x31, 0xca, 0x6f, 0x96, 0xb4, + 0xa8, 0x21, 0x03, 0x35, 0x86, 0x83, 0xae, 0x81, 0x46, 0x0f, 0x54, 0x99, + 0xad, 0x2b, 0xe8, 0xfb, 0x17, 0x16, 0x01, 0xee, 0x93, 0xa3, 0xa1, 0xfa, + 0xd2, 0x51, 0x2d, 0xfb, 0xaf, 0x47, 0x8f, 0x8f, 0x87, 0xa7, 0x1b, 0xb0, + 0x81, 0x72, 0x02, 0xf4, 0xc0, 0xf0, 0xf0, 0xcd, 0x2d, 0x48, 0x01, 0x7a, + 0xa6, 0xe9, 0xe2, 0x12, 0xa4, 0xfd, 0xab, 0x43, 0xce, 0x78, 0xf1, 0x67, + 0x3f, 0x1d, 0x0e, 0x96, 0xdf, 0x58, 0xf7, 0x47, 0xe6, 0x1e, 0xa0, 0xe3, + 0x9e, 0x78, 0xc7, 0x41, 0xfa, 0x19, 0xef, 0x22, 0xe7, 0x64, 0x8a, 0xc1, + 0x81, 0x30, 0x90, 0x71, 0xb2, 0x21, 0x5c, 0x7f, 0xa7, 0x76, 0xd5, 0xd7, + 0xbb, 0x5b, 0x8d, 0xf3, 0x59, 0x5e, 0x6f, 0xb3, 0x9b, 0x86, 0x7c, 0x02, + 0xfb, 0x69, 0x70, 0x75, 0xfa, 0x7a, 0xbb, 0xf4, 0xf7, 0x8e, 0x2c, 0xc5, + 0x08, 0xd8, 0x68, 0x92, 0xc6, 0xf7, 0xa6, 0xc0, 0x9f, 0xe9, 0xa5, 0x54, + 0xa9, 0x40, 0x52, 0x52, 0x92, 0xd6, 0x31, 0x01, 0x07, 0x55, 0x63, 0xc9, + 0xd5, 0x6c, 0x32, 0x63, 0xf0, 0x7a, 0x7f, 0x7c, 0xb9, 0xd1, 0x12, 0x63, + 0x74, 0x74, 0xf4, 0xdd, 0x38, 0xdb, 0xe7, 0x5a, 0x0a, 0xa0, 0x0c, 0x8a, + 0x49, 0x4c, 0xcf, 0xf2, 0xd6, 0xe6, 0xd6, 0x51, 0xc3, 0xf2, 0x7b, 0x7f, + 0x75, 0x3d, 0xa6, 0x56, 0x65, 0xba, 0x7e, 0xd3, 0x3b, 0x67, 0xaf, 0x09, + 0x73, 0xdc, 0x9f, 0xe6, 0x92, 0x3c, 0xe2, 0x64, 0x62, 0x6a, 0x84, 0x9c, + 0xf0, 0xd2, 0x63, 0x4e, 0x04, 0x8b, 0xca, 0xa5, 0xc4, 0xb0, 0xd0, 0x2f, + 0x4b, 0xb5, 0x9b, 0x9a, 0xe1, 0xe3, 0x21, 0xc6, 0x42, 0xb3, 0x63, 0x88, + 0x47, 0x21, 0x3d, 0x2b, 0xc6, 0x26, 0x32, 0xff, 0xf2, 0x9e, 0x06, 0x04, + 0xe5, 0xf0, 0x9d, 0x90, 0xff, 0x69, 0x2f, 0xd1, 0xfc, 0x8f, 0x41, 0x14, + 0x3b, 0x71, 0x0e, 0xab, 0xe1, 0x27, 0xae, 0xcc, 0xe8, 0x55, 0xfb, 0xee, + 0xac, 0x8a, 0x9a, 0xae, 0x0c, 0x89, 0x68, 0x7c, 0x34, 0x12, 0x12, 0x05, + 0x59, 0x59, 0xc4, 0xf1, 0x54, 0x96, 0x12, 0xe5, 0x22, 0xe9, 0xf7, 0x5d, + 0xbe, 0x37, 0xf3, 0x43, 0xbc, 0x9a, 0x22, 0xda, 0xc0, 0x23, 0x8c, 0x50, + 0xc7, 0x33, 0xb5, 0x08, 0x73, 0xa0, 0x7b, 0xc8, 0x21, 0x6d, 0xb9, 0x46, + 0x8c, 0x80, 0x4c, 0x5a, 0xe1, 0xbe, 0xc1, 0xaa, 0xd9, 0x51, 0xad, 0xf8, + 0x6d, 0x82, 0x08, 0x12, 0x40, 0x44, 0x54, 0x27, 0x29, 0xec, 0x54, 0xb6, + 0x63, 0x12, 0xe3, 0xe3, 0xe9, 0x9f, 0xf9, 0xe3, 0xce, 0x99, 0xb9, 0xe3, + 0xbe, 0x67, 0xe4, 0xe4, 0x0a, 0x3c, 0xd9, 0xa3, 0xe1, 0x95, 0xd5, 0x07, + 0xfe, 0x95, 0x80, 0xf7, 0x3d, 0x2a, 0x26, 0xa6, 0xaa, 0xd1, 0x62, 0x67, + 0x2e, 0xf4, 0x72, 0x87, 0x3b, 0x8b, 0xb0, 0xb0, 0x30, 0xff, 0xd5, 0x00, + 0x46, 0x09, 0x3f, 0x33, 0x27, 0x67, 0x32, 0xe4, 0x66, 0x85, 0x40, 0xb2, + 0xc5, 0xf1, 0xa4, 0xaa, 0xe5, 0x53, 0x5c, 0xd6, 0x19, 0xaf, 0xe1, 0xf6, + 0x54, 0x59, 0xf6, 0x60, 0xc5, 0x49, 0x27, 0xf7, 0x2b, 0x9c, 0x99, 0x1c, + 0xa1, 0xdf, 0x90, 0xdb, 0x2d, 0xb0, 0x03, 0x24, 0xd5, 0x71, 0x25, 0xbf, + 0xae, 0xfe, 0xdb, 0xf2, 0x80, 0x52, 0xe7, 0xd9, 0x02, 0x0d, 0x98, 0xb6, + 0x7e, 0xcb, 0x0b, 0xb0, 0xd0, 0xfe, 0x22, 0x1e, 0x9e, 0x96, 0xb7, 0xbc, + 0x5c, 0xa1, 0x7c, 0xd1, 0x5d, 0x20, 0xe0, 0x3f, 0x91, 0x45, 0x8a, 0xbf, + 0x10, 0x31, 0xcc, 0x3a, 0x61, 0x44, 0x95, 0x97, 0x94, 0x90, 0x67, 0x2c, + 0x04, 0x9d, 0xab, 0x51, 0xa3, 0xaf, 0x5a, 0xaa, 0x98, 0x27, 0xa6, 0xe7, + 0x46, 0xf6, 0x99, 0x41, 0xbd, 0x32, 0xd7, 0xeb, 0xee, 0xf8, 0xbd, 0xed, + 0x0c, 0x64, 0xd8, 0x13, 0x00, 0xe5, 0xf3, 0x76, 0xb7, 0x84, 0x48, 0xcc, + 0xff, 0xef, 0x63, 0xb5, 0xb6, 0xa5, 0x24, 0xf7, 0x6f, 0x9e, 0xef, 0x17, + 0x15, 0x2b, 0xce, 0xf2, 0xa1, 0x3a, 0x9f, 0x21, 0x3e, 0xd5, 0x3f, 0x4c, + 0xfe, 0x73, 0xdf, 0xd6, 0xed, 0xba, 0x42, 0xb3, 0xb3, 0x56, 0xb3, 0x9b, + 0xaa, 0x1a, 0xfc, 0xf5, 0x91, 0xbf, 0x9b, 0xeb, 0x8e, 0x39, 0x46, 0x8e, + 0xc3, 0xa5, 0x24, 0x45, 0x06, 0x0c, 0x0c, 0x44, 0xbf, 0x3e, 0x99, 0x17, + 0xf8, 0xa4, 0x7e, 0xcb, 0x4b, 0x58, 0xea, 0xf0, 0xd6, 0x70, 0x88, 0x9b, + 0xac, 0x3b, 0xac, 0xe0, 0x76, 0xfc, 0x7e, 0x4a, 0x65, 0xa6, 0x27, 0x07, + 0x2a, 0x5c, 0xae, 0x52, 0x20, 0xa5, 0xda, 0xdb, 0xdb, 0x43, 0x25, 0x24, + 0xd4, 0xcd, 0xff, 0xd6, 0x66, 0x7f, 0xd6, 0x49, 0xb4, 0x4b, 0xd8, 0x17, + 0x53, 0x5b, 0xcb, 0xba, 0xd2, 0x62, 0x3f, 0x5d, 0xa9, 0xd5, 0x00, 0xd0, + 0x14, 0xfa, 0xe8, 0x27, 0xdf, 0x54, 0x80, 0xed, 0x9d, 0xe7, 0x89, 0xc2, + 0x7d, 0xdb, 0x5e, 0x7a, 0xdf, 0x04, 0xfb, 0xb0, 0x05, 0x5a, 0x25, 0x1b, + 0x4d, 0x7b, 0xd9, 0x35, 0x30, 0x2b, 0x04, 0x94, 0xce, 0xac, 0xb2, 0x22, + 0xfb, 0x5a, 0x5a, 0x96, 0x9a, 0xce, 0x46, 0x7d, 0xf1, 0x7f, 0x5f, 0x0e, + 0x5f, 0x69, 0x4a, 0xb5, 0x40, 0x6d, 0x9f, 0x3f, 0x59, 0xf7, 0xfc, 0x65, + 0xf1, 0xd1, 0x41, 0xd8, 0x2d, 0x47, 0x42, 0x23, 0x13, 0xb9, 0xf7, 0x65, + 0x91, 0x85, 0x61, 0x29, 0x24, 0x69, 0xa6, 0xd6, 0xb0, 0x1f, 0x9d, 0xca, + 0xb3, 0xf7, 0x6d, 0x0e, 0x04, 0x9a, 0x8e, 0xd8, 0x03, 0x68, 0x69, 0x69, + 0x4d, 0xa4, 0x7d, 0xdb, 0x0d, 0xd6, 0xb3, 0x15, 0x07, 0x92, 0x48, 0x9e, + 0xb7, 0x37, 0x11, 0x21, 0x74, 0x7b, 0x81, 0x3e, 0x5e, 0x6c, 0x51, 0xba, + 0xc5, 0x62, 0xc6, 0x7d, 0x87, 0x4e, 0x3a, 0x6f, 0x96, 0x19, 0x11, 0x41, + 0x03, 0x1c, 0xeb, 0xce, 0x8c, 0x67, 0x8d, 0xe1, 0x7e, 0xbd, 0xb5, 0xcb, + 0xe5, 0xfb, 0x0c, 0xa6, 0x21, 0x2f, 0xcd, 0xf6, 0xf6, 0x27, 0x75, 0xcb, + 0x6b, 0xb6, 0xd8, 0xfe, 0x02, 0x12, 0x6e, 0xdd, 0x45, 0x93, 0xe2, 0x57, + 0xa1, 0x8b, 0x76, 0x2b, 0x50, 0x8f, 0x6e, 0xf9, 0x7a, 0x88, 0x7f, 0x88, + 0xf8, 0x9a, 0x25, 0x87, 0x5d, 0xcf, 0x56, 0x92, 0x51, 0x04, 0x1f, 0x7b, + 0x3e, 0x7c, 0xc7, 0x53, 0x98, 0xde, 0x99, 0x6a, 0x2d, 0x87, 0x1b, 0xa6, + 0x0c, 0x6b, 0x67, 0x68, 0xfa, 0x9c, 0xe2, 0x0d, 0x19, 0x9f, 0x9e, 0x99, + 0xfc, 0x7f, 0x3a, 0xde, 0x11, 0x29, 0xd4, 0x83, 0x7b, 0x66, 0x41, 0xfa, + 0xe6, 0xf7, 0x0a, 0xdd, 0x52, 0x06, 0xb0, 0x58, 0x95, 0x3a, 0xda, 0x56, + 0x68, 0x6f, 0x9d, 0x79, 0x96, 0x55, 0x00, 0xc5, 0x85, 0x56, 0xaf, 0x5c, + 0x53, 0x69, 0x4f, 0x13, 0xe9, 0xcc, 0x0a, 0x74, 0x2d, 0x01, 0x6f, 0xfe, + 0x78, 0xc3, 0xbc, 0x1d, 0xa3, 0x49, 0xc3, 0x8f, 0xba, 0x2b, 0x7b, 0x00, + 0x24, 0x38, 0xfe, 0x4a, 0xd9, 0x2f, 0x36, 0x8e, 0x3c, 0x3f, 0xa6, 0xb3, + 0x05, 0xd2, 0x79, 0x9c, 0xa9, 0x63, 0x05, 0x9f, 0x69, 0xb1, 0x18, 0xd4, + 0x27, 0x48, 0x3a, 0x6a, 0x41, 0x65, 0x2a, 0x34, 0x80, 0x7b, 0x12, 0x69, + 0x01, 0x96, 0x65, 0xb1, 0xe0, 0x90, 0xcb, 0x7e, 0x70, 0xcd, 0xd3, 0x34, + 0xd0, 0xba, 0x87, 0x96, 0xad, 0xc1, 0x59, 0x81, 0xbc, 0x63, 0xa9, 0x22, + 0xd1, 0x8f, 0x5f, 0xa7, 0xd4, 0xea, 0x56, 0xac, 0xe6, 0x21, 0x60, 0x7c, + 0x4e, 0x3d, 0xea, 0x10, 0xe7, 0xd7, 0x68, 0x03, 0x41, 0x90, 0x6d, 0xcb, + 0x2a, 0xd0, 0x0a, 0xe1, 0x30, 0xaf, 0x2a, 0x63, 0x1e, 0x1c, 0x84, 0xf6, + 0x26, 0xbe, 0xfd, 0x39, 0x92, 0x0a, 0x89, 0x8e, 0xca, 0x60, 0x4a, 0x1b, + 0xf0, 0xc3, 0xc3, 0xe3, 0x51, 0x3c, 0x5d, 0x19, 0x01, 0x9e, 0xf7, 0x65, + 0x25, 0xb1, 0x6f, 0xda, 0xef, 0x70, 0x86, 0x8c, 0x43, 0xae, 0xc0, 0xed, + 0x6c, 0x85, 0xff, 0x66, 0x82, 0xbd, 0x8b, 0x15, 0x5c, 0xa3, 0x07, 0x66, + 0xca, 0x69, 0x3d, 0xa4, 0x35, 0xf3, 0x3f, 0xd5, 0xd9, 0x39, 0x71, 0xf2, + 0xf6, 0xb0, 0x39, 0xec, 0x08, 0x0e, 0xf9, 0x9a, 0x73, 0x79, 0xea, 0x7f, + 0xc1, 0x03, 0x4b, 0x4a, 0x66, 0x15, 0x57, 0x54, 0x86, 0x3d, 0x3b, 0x8d, + 0xd7, 0xdb, 0x3b, 0x79, 0x8d, 0xc5, 0xe3, 0xf2, 0x77, 0xbc, 0xc9, 0x51, + 0xc0, 0xa3, 0x02, 0xba, 0x93, 0xe3, 0x52, 0xff, 0xdb, 0x51, 0x74, 0xaf, + 0x87, 0xeb, 0x5b, 0x7e, 0xb5, 0x18, 0xc3, 0x5f, 0x37, 0x29, 0xa5, 0xa4, + 0x28, 0xa7, 0x8b, 0x93, 0xea, 0x8e, 0xf3, 0xca, 0x61, 0xd9, 0x4c, 0xae, + 0xc9, 0x1c, 0xba, 0xe6, 0x5c, 0x5b, 0xc0, 0x42, 0x85, 0x06, 0xcc, 0xd6, + 0x30, 0x95, 0x35, 0xbc, 0x69, 0xba, 0x80, 0xbb, 0x6a, 0x05, 0x66, 0x7f, + 0x5f, 0xcd, 0x46, 0xa1, 0xa7, 0x11, 0x51, 0xf3, 0x2b, 0x30, 0xec, 0xc4, + 0xeb, 0x2d, 0x35, 0xbc, 0x36, 0xee, 0xd5, 0x1d, 0x5a, 0xc5, 0x9c, 0xe5, + 0xb3, 0x2a, 0xcb, 0x67, 0xad, 0xc5, 0x35, 0xf2, 0xe7, 0x36, 0x0a, 0xde, + 0xe7, 0x6a, 0x45, 0x87, 0x42, 0xc9, 0x74, 0x01, 0xd4, 0xb8, 0x26, 0x2b, + 0x55, 0xec, 0x1b, 0x41, 0xa2, 0x51, 0x22, 0x6e, 0x03, 0x51, 0xc7, 0x4a, + 0x07, 0x3e, 0x0d, 0x0f, 0x84, 0x49, 0x0a, 0xeb, 0x0c, 0x37, 0x25, 0xfe, + 0xf0, 0x93, 0x9a, 0xee, 0xf7, 0xce, 0xe9, 0x1a, 0xdd, 0xb6, 0x7a, 0x21, + 0x09, 0x1c, 0x9c, 0xef, 0x3e, 0x68, 0x58, 0x68, 0x63, 0x03, 0x1a, 0x39, + 0x26, 0x73, 0x87, 0xbd, 0x84, 0x52, 0x72, 0xe0, 0x3c, 0xf0, 0x10, 0x3a, + 0xfe, 0x8e, 0x8e, 0x10, 0xab, 0xff, 0x95, 0x15, 0x26, 0x95, 0x47, 0xb1, + 0xe1, 0xab, 0x25, 0x5d, 0x95, 0xf8, 0x6d, 0x7e, 0x7b, 0x39, 0xd2, 0xb5, + 0xf1, 0x15, 0xbc, 0xc2, 0xc5, 0xfa, 0xcb, 0x26, 0xe6, 0x08, 0x03, 0x8a, + 0x83, 0x6f, 0x05, 0xe5, 0x37, 0xfb, 0x6b, 0x49, 0x7a, 0x35, 0xda, 0x58, + 0x4f, 0x34, 0xdd, 0x83, 0x06, 0x2d, 0x17, 0x3d, 0x96, 0xa7, 0x6f, 0x34, + 0x9b, 0xc7, 0x9c, 0x21, 0x0f, 0x9b, 0xfe, 0x1f, 0x7f, 0xa7, 0x32, 0x39, + 0xad, 0x17, 0xe1, 0xaa, 0x5d, 0x25, 0x6b, 0x31, 0x22, 0x18, 0x43, 0x9b, + 0x61, 0x33, 0xb8, 0x2a, 0x07, 0x71, 0x8c, 0x73, 0x4c, 0xb7, 0xaf, 0xef, + 0x1a, 0x8f, 0x04, 0xc3, 0x35, 0x36, 0x73, 0x22, 0x0d, 0x71, 0x2d, 0xb4, + 0x77, 0x8c, 0xb1, 0x5d, 0x99, 0xf2, 0x71, 0xf0, 0x98, 0x97, 0xcc, 0x82, + 0x61, 0xdf, 0xc7, 0x59, 0x63, 0x3a, 0xa1, 0xe6, 0xd0, 0xd0, 0x4f, 0x31, + 0x29, 0x3e, 0x5e, 0x48, 0x31, 0x24, 0x90, 0x49, 0x69, 0x96, 0x7c, 0x9d, + 0x9e, 0x0b, 0x44, 0xe0, 0xf5, 0xe6, 0xa0, 0x9d, 0x84, 0x63, 0xfc, 0xcb, + 0x69, 0xf0, 0x06, 0x6b, 0xff, 0x36, 0x97, 0xaa, 0xbf, 0x5d, 0x0d, 0x93, + 0x94, 0x68, 0x35, 0x98, 0x99, 0x0b, 0x2c, 0xda, 0xb5, 0xbc, 0xb8, 0x67, + 0x63, 0x8d, 0x08, 0xc3, 0x0a, 0xef, 0xc3, 0xcf, 0x92, 0x46, 0x55, 0x3d, + 0xe4, 0x19, 0x9e, 0xe1, 0x23, 0x92, 0xb8, 0x90, 0x40, 0xf4, 0x38, 0xed, + 0x7e, 0xdd, 0xa8, 0x27, 0xfb, 0xf4, 0xea, 0x35, 0xc5, 0x4e, 0xe9, 0x61, + 0x2d, 0x37, 0x79, 0x3f, 0x0c, 0x87, 0xdc, 0x25, 0x2c, 0x69, 0x19, 0x0e, + 0xa0, 0x67, 0x67, 0x41, 0xb7, 0xe1, 0x71, 0x6e, 0x78, 0xcd, 0xaf, 0xff, + 0x71, 0x2f, 0x49, 0xf6, 0x43, 0x4d, 0xad, 0xde, 0x79, 0xed, 0x35, 0xab, + 0x33, 0xe7, 0xcf, 0xee, 0x32, 0xe1, 0xf4, 0x58, 0x2f, 0xab, 0x5c, 0x7e, + 0x01, 0xf9, 0xbf, 0xb6, 0x07, 0xd4, 0x6f, 0xfb, 0xe4, 0xa5, 0xe8, 0x38, + 0x72, 0x0b, 0xd2, 0xdf, 0x06, 0x06, 0xc5, 0xe2, 0xc1, 0x57, 0xca, 0x84, + 0xf2, 0x01, 0xe5, 0xf4, 0xc1, 0x5f, 0xf9, 0x46, 0xa7, 0x5c, 0x13, 0xb3, + 0xb3, 0xe5, 0x2d, 0x2c, 0x0e, 0xc5, 0xeb, 0x8e, 0x95, 0xa5, 0xd5, 0x6b, + 0xf4, 0x27, 0x3b, 0x3d, 0x6f, 0x3c, 0x9e, 0x8f, 0x4a, 0x96, 0xdb, 0x5d, + 0x4f, 0xa4, 0x86, 0x39, 0x6d, 0x57, 0x92, 0xf6, 0xc7, 0x53, 0x8d, 0x1b, + 0x96, 0xe5, 0x8f, 0x69, 0x6d, 0x06, 0x09, 0xcd, 0x0e, 0x43, 0x85, 0x1f, + 0x32, 0x57, 0xff, 0xf3, 0xa6, 0x5d, 0x8f, 0x9b, 0xd3, 0xe8, 0x9d, 0xc9, + 0x4c, 0x2b, 0xd9, 0xc3, 0x1d, 0x4c, 0x32, 0x28, 0x04, 0xf8, 0x5e, 0x4c, + 0xcb, 0xcb, 0x32, 0x75, 0xab, 0x89, 0x3b, 0xfc, 0xd9, 0x4c, 0x39, 0x41, + 0x60, 0x89, 0x92, 0xfe, 0x29, 0x6f, 0x16, 0x21, 0xe1, 0xf1, 0x42, 0xaf, + 0xf7, 0x68, 0xda, 0xf2, 0x7c, 0x87, 0x53, 0x0f, 0x6c, 0x19, 0xf5, 0xf7, + 0xd1, 0xc8, 0x26, 0x9d, 0x47, 0xc7, 0xa3, 0x0c, 0x72, 0x3d, 0xa1, 0x62, + 0x9f, 0xcf, 0xd0, 0xc4, 0xad, 0x59, 0xd6, 0x19, 0x9c, 0x94, 0xbc, 0xf4, + 0x7f, 0x6f, 0x26, 0x29, 0x55, 0x02, 0xc4, 0x19, 0xb8, 0x59, 0x2d, 0x06, + 0x60, 0x10, 0x89, 0x78, 0x93, 0x34, 0xf4, 0x3d, 0xf7, 0xf3, 0xc7, 0xca, + 0x6b, 0x87, 0xeb, 0x4d, 0xc7, 0xb1, 0x17, 0xab, 0xf5, 0x64, 0x5c, 0x5c, + 0x38, 0xa4, 0xa4, 0xa4, 0x06, 0x06, 0x07, 0xb3, 0xf6, 0x66, 0x5f, 0x5f, + 0x6f, 0xa6, 0x5d, 0x0e, 0x67, 0x7e, 0x2b, 0xe8, 0x93, 0xc8, 0x36, 0x0c, + 0xac, 0x0c, 0x6e, 0x60, 0xab, 0xa9, 0xe1, 0xf2, 0xbc, 0x33, 0x74, 0xa5, + 0x77, 0x7c, 0x16, 0x87, 0xfb, 0xe8, 0xe7, 0x60, 0xcf, 0xd4, 0xba, 0x6d, + 0x0a, 0xf7, 0x43, 0x17, 0x11, 0xa9, 0x26, 0xa3, 0x01, 0x6b, 0x70, 0x54, + 0xcd, 0xdd, 0xb5, 0xfb, 0x39, 0x40, 0xfd, 0x78, 0xd3, 0x5e, 0x78, 0x32, + 0x7b, 0x45, 0x78, 0x9f, 0xa0, 0x89, 0xcc, 0xdf, 0xba, 0xf2, 0x48, 0x6f, + 0xb1, 0xc0, 0xea, 0x1f, 0xf6, 0x7d, 0x8f, 0x5d, 0x2d, 0xb0, 0x54, 0xad, + 0xd4, 0x8c, 0x8f, 0x8b, 0x43, 0xed, 0xd4, 0x67, 0x71, 0xeb, 0x20, 0xa2, + 0xa3, 0x6a, 0xaf, 0x36, 0xce, 0xa0, 0xe6, 0xe1, 0xe1, 0x69, 0xd5, 0x65, + 0x66, 0x63, 0xff, 0x58, 0x9e, 0x77, 0x34, 0x07, 0x37, 0x23, 0x1e, 0x83, + 0xa2, 0x49, 0x4b, 0x76, 0x3b, 0x93, 0x57, 0x59, 0xe5, 0x32, 0x5c, 0x3d, + 0x03, 0x21, 0x5a, 0x96, 0x98, 0xa1, 0xdd, 0x5e, 0x91, 0xe8, 0x72, 0xb2, + 0x18, 0x9d, 0xce, 0x6e, 0x7e, 0xbd, 0xac, 0x33, 0x1d, 0xa7, 0x5a, 0x63, + 0x16, 0xe1, 0xc2, 0x1f, 0x6a, 0xe9, 0x74, 0x96, 0x15, 0x34, 0xea, 0xba, + 0x9f, 0x57, 0x1e, 0x9f, 0x88, 0x44, 0x51, 0xa0, 0x00, 0xf4, 0x7a, 0xf9, + 0xd7, 0xde, 0x54, 0x3e, 0x68, 0x12, 0x75, 0x88, 0x4a, 0xb0, 0x9a, 0x55, + 0xf0, 0xb3, 0xc4, 0xb8, 0x41, 0xcd, 0xb7, 0xef, 0x64, 0x9d, 0x5b, 0x90, + 0x91, 0x32, 0xfc, 0x88, 0x04, 0x91, 0x82, 0x99, 0x7a, 0xcc, 0x60, 0x91, + 0x96, 0x65, 0xba, 0x7f, 0xef, 0x48, 0xe8, 0x6a, 0x3b, 0xd5, 0xf9, 0x86, + 0x2f, 0x14, 0x55, 0x44, 0x54, 0xf4, 0x68, 0xa9, 0x86, 0x44, 0x34, 0x0c, + 0x99, 0xe9, 0x1a, 0x38, 0x14, 0x4b, 0x32, 0x0d, 0x51, 0xb2, 0xb1, 0x78, + 0x8a, 0x9f, 0xac, 0x72, 0x89, 0x0f, 0xa1, 0x0f, 0xbb, 0xc7, 0x66, 0x77, + 0x6f, 0x20, 0x45, 0xf0, 0xa7, 0xff, 0x6c, 0xd0, 0x3e, 0x1c, 0xc4, 0xd0, + 0x0b, 0xf9, 0xfd, 0x85, 0xf4, 0x60, 0xe9, 0x55, 0xea, 0x98, 0x40, 0x47, + 0x84, 0x12, 0x50, 0xc4, 0xf0, 0xee, 0x36, 0x98, 0x45, 0x9f, 0xa6, 0xd8, + 0xc7, 0x24, 0x62, 0x5d, 0x46, 0x09, 0x13, 0x7f, 0x73, 0x4e, 0xe3, 0x53, + 0xdf, 0x76, 0xa9, 0x0a, 0x0d, 0x8b, 0xfd, 0xaa, 0x5b, 0xd4, 0x10, 0xf6, + 0xec, 0x6c, 0xa3, 0xf6, 0xe1, 0xf3, 0xe6, 0xbf, 0x33, 0xb5, 0xf3, 0x4b, + 0x0f, 0x48, 0x33, 0xcb, 0xf7, 0x12, 0x37, 0xf3, 0x7f, 0x85, 0xd7, 0xd7, + 0x7b, 0x60, 0x38, 0x5a, 0x91, 0xe3, 0x7b, 0x9c, 0x6e, 0xb0, 0x7f, 0xaa, + 0xf2, 0xea, 0xf6, 0x7b, 0xf3, 0x00, 0x67, 0xb9, 0xcb, 0x3b, 0x19, 0x4e, + 0x0f, 0xee, 0xdf, 0x79, 0xf0, 0x73, 0x58, 0x2f, 0xfe, 0x89, 0x26, 0xe0, + 0x46, 0xdf, 0x6b, 0x6c, 0xb0, 0x38, 0xc6, 0x25, 0x74, 0x5b, 0xe5, 0xc1, + 0x68, 0x3a, 0xa1, 0xe8, 0x97, 0xda, 0xf0, 0x4f, 0x43, 0x22, 0xe0, 0x8a, + 0x4b, 0x4b, 0xd3, 0xa4, 0x37, 0xfd, 0x9b, 0x9c, 0x2c, 0x2e, 0x16, 0x5c, + 0x83, 0x3f, 0x3e, 0x96, 0x7e, 0xb1, 0xc3, 0xd8, 0x2b, 0x00, 0x00, 0xb0, + 0xfb, 0x3d, 0x26, 0x6b, 0x6c, 0xbc, 0x75, 0xbf, 0x3e, 0xe8, 0xf0, 0x89, + 0x2c, 0xb5, 0x98, 0x6b, 0xab, 0x32, 0x83, 0xc4, 0xdc, 0x01, 0x9e, 0xa5, + 0x56, 0x13, 0x89, 0xa1, 0xf1, 0xd7, 0x6e, 0xab, 0x15, 0xe8, 0x9e, 0xee, + 0x8f, 0x18, 0xef, 0xf8, 0x66, 0x22, 0x3d, 0x72, 0x0f, 0x02, 0x17, 0x0a, + 0x3a, 0x98, 0x38, 0x38, 0xc3, 0x79, 0xa2, 0x61, 0x91, 0x4d, 0x5f, 0x60, + 0x90, 0x09, 0x73, 0xa5, 0x93, 0xe9, 0xdf, 0x2f, 0xc6, 0xcd, 0x3b, 0x45, + 0x1d, 0x14, 0x67, 0xd7, 0x9f, 0x9a, 0x32, 0xdd, 0x66, 0xc0, 0xe9, 0xdf, + 0x5c, 0x8e, 0xe7, 0x23, 0xda, 0x9c, 0xc1, 0x29, 0xd2, 0x74, 0x42, 0x10, + 0xbd, 0x95, 0x4d, 0x63, 0xcd, 0x1b, 0x58, 0xbb, 0x57, 0xf1, 0x44, 0xcd, + 0xfa, 0xb8, 0x21, 0x87, 0x41, 0x2e, 0x91, 0x56, 0x8b, 0xe6, 0x37, 0xc1, + 0xc5, 0x4b, 0x0e, 0x0a, 0xb2, 0x27, 0xe5, 0x7e, 0xc0, 0x89, 0x9a, 0x2d, + 0x11, 0xee, 0x91, 0xaf, 0xb1, 0xd4, 0x16, 0x11, 0x1c, 0x2d, 0x58, 0x24, + 0x90, 0x18, 0x5b, 0x7d, 0x3b, 0xef, 0x90, 0xeb, 0xba, 0x99, 0xdb, 0xf5, + 0x4b, 0x90, 0xa1, 0x6b, 0x2b, 0xb7, 0xf6, 0x17, 0x1a, 0x85, 0x98, 0x77, + 0xcd, 0x53, 0xca, 0x54, 0x65, 0xe5, 0x36, 0x27, 0x53, 0xc4, 0xfe, 0x5f, + 0x10, 0x1d, 0x05, 0x34, 0x76, 0x19, 0x93, 0x12, 0xc7, 0xb6, 0x58, 0xc8, + 0x68, 0x53, 0x71, 0xd8, 0xaa, 0x22, 0xa8, 0xa9, 0xa5, 0x71, 0xdb, 0x13, + 0x5b, 0x5a, 0x5a, 0x16, 0xc9, 0x4f, 0xe9, 0xd7, 0xa4, 0xc2, 0x61, 0x50, + 0x03, 0x14, 0x85, 0xf2, 0xac, 0x22, 0x58, 0x16, 0x98, 0x34, 0x93, 0xb9, + 0xfd, 0xaf, 0x69, 0x6d, 0x32, 0xe8, 0x11, 0x29, 0x0a, 0x12, 0x8a, 0xd5, + 0xda, 0x97, 0xc9, 0x3d, 0xf1, 0x28, 0xbd, 0x24, 0x47, 0xce, 0x9e, 0xee, + 0x06, 0x17, 0x18, 0xfb, 0x25, 0xed, 0x5f, 0x4f, 0x0b, 0x80, 0x1e, 0x8f, + 0xd5, 0x31, 0xef, 0x4d, 0xc7, 0xed, 0x86, 0x73, 0xaf, 0xcd, 0x7e, 0xad, + 0x37, 0x3e, 0x0b, 0xa2, 0x10, 0x43, 0x3b, 0x4c, 0x11, 0xcb, 0xaf, 0x7c, + 0x2a, 0xad, 0xed, 0xe1, 0xf7, 0x94, 0xe5, 0xbe, 0x41, 0xc6, 0xc9, 0x89, + 0x25, 0xae, 0x74, 0x38, 0x6f, 0x4e, 0xdb, 0xfc, 0x9a, 0x61, 0x61, 0x7e, + 0xd3, 0x52, 0x98, 0xd4, 0x68, 0xa5, 0x4c, 0x12, 0x40, 0xc4, 0xe7, 0x96, + 0xc4, 0xab, 0x65, 0xbf, 0xda, 0x8c, 0x14, 0xe0, 0x56, 0x95, 0x58, 0x9b, + 0xf3, 0xd3, 0xc2, 0xa9, 0x0b, 0x92, 0x0b, 0x24, 0xc5, 0x9d, 0x5f, 0x49, + 0x5d, 0xdf, 0xb6, 0x1e, 0xd0, 0xd0, 0xb2, 0xc6, 0xd4, 0x4a, 0x67, 0x36, + 0x8d, 0x5c, 0xb4, 0x9c, 0xd7, 0x43, 0x1c, 0xaa, 0x13, 0xe7, 0xb2, 0x38, + 0xb6, 0x7e, 0xaf, 0x7a, 0x96, 0x2e, 0x31, 0xe1, 0x86, 0xc2, 0x49, 0xba, + 0xac, 0xe7, 0xaa, 0xb5, 0xfc, 0x7e, 0x2f, 0x6b, 0x74, 0x3d, 0x59, 0x8c, + 0x4f, 0x4a, 0x72, 0x99, 0xad, 0x2f, 0x37, 0x33, 0x35, 0x45, 0xc3, 0xc3, + 0x93, 0x1a, 0x76, 0x63, 0xf2, 0x80, 0xe0, 0xd5, 0x54, 0x06, 0x5b, 0x59, + 0x6e, 0x81, 0xc5, 0xca, 0xbe, 0x23, 0xbd, 0x6a, 0xb8, 0xf1, 0xc7, 0xfb, + 0x5b, 0x2a, 0x3f, 0xb4, 0xcd, 0xef, 0x96, 0x9d, 0xcc, 0xfb, 0x6b, 0x5b, + 0x85, 0x37, 0xd7, 0xe7, 0xef, 0x10, 0x27, 0x8e, 0x03, 0x24, 0x32, 0xa6, + 0x99, 0xdc, 0xba, 0xc1, 0x13, 0x82, 0x16, 0xde, 0xed, 0x31, 0xb5, 0x75, + 0xa1, 0x93, 0xc1, 0xd6, 0x0b, 0x1f, 0x99, 0x55, 0x6e, 0x4f, 0x3c, 0xbf, + 0xd8, 0xf8, 0xfd, 0xf3, 0xfc, 0x38, 0xad, 0xbb, 0x5e, 0x36, 0xbf, 0x8c, + 0x5e, 0x01, 0xf8, 0x1c, 0x9f, 0xb3, 0x27, 0x93, 0xca, 0x0c, 0x2c, 0xa5, + 0x84, 0x31, 0x90, 0x48, 0x89, 0xc0, 0xa0, 0x61, 0x73, 0xef, 0x95, 0xc6, + 0x66, 0x36, 0x30, 0x30, 0x51, 0x8e, 0xf6, 0x3b, 0x37, 0x44, 0x26, 0xfc, + 0xfe, 0x03, 0x72, 0x8e, 0xb7, 0xbc, 0xe1, 0xa2, 0xe4, 0xba, 0x5f, 0xaf, + 0x1e, 0x06, 0xf0, 0xc7, 0x29, 0x72, 0xc7, 0xe8, 0xe8, 0x7b, 0xad, 0x94, + 0x67, 0x1e, 0xe7, 0xb3, 0xd9, 0x4f, 0x78, 0x96, 0x6b, 0x0a, 0xe5, 0x56, + 0x0b, 0x69, 0xa5, 0xa5, 0x04, 0x8c, 0xea, 0x39, 0x9f, 0xac, 0x48, 0x05, + 0x64, 0x75, 0x09, 0xf7, 0xf6, 0xc4, 0xf8, 0x19, 0xf6, 0xaf, 0x52, 0xed, + 0xec, 0x56, 0x65, 0xbb, 0xd8, 0x1e, 0xfd, 0xfc, 0xc7, 0x4c, 0x33, 0x8b, + 0xd9, 0x7c, 0xd6, 0x70, 0x5e, 0x6b, 0x4d, 0x38, 0x01, 0xce, 0x07, 0x13, + 0xdb, 0x43, 0x04, 0xf6, 0x8c, 0xdf, 0x12, 0x72, 0x8e, 0xda, 0x21, 0x4d, + 0x84, 0x0a, 0x62, 0x97, 0x05, 0x41, 0x60, 0x96, 0x83, 0xf6, 0x94, 0x68, + 0x3a, 0x12, 0x2c, 0xbf, 0x45, 0xae, 0xf9, 0x72, 0xfd, 0x88, 0xa6, 0x4c, + 0x77, 0x56, 0xe7, 0x29, 0x9f, 0x66, 0xfb, 0x4d, 0xf9, 0x87, 0xf3, 0x75, + 0x96, 0xb2, 0x4a, 0x6e, 0xfb, 0x8d, 0x89, 0xaa, 0x96, 0x0d, 0xe9, 0xaa, + 0x47, 0x7a, 0x59, 0x5f, 0xb0, 0x27, 0x73, 0xe1, 0xa7, 0xc9, 0x90, 0x1f, + 0xdc, 0x88, 0x7d, 0x22, 0x41, 0x5a, 0x8d, 0x46, 0x02, 0xbf, 0x74, 0xb7, + 0x9d, 0x6c, 0x5f, 0x56, 0xf2, 0x5d, 0x01, 0x3a, 0x2e, 0xe2, 0x65, 0x81, + 0xab, 0xeb, 0x0d, 0x7f, 0x93, 0xd2, 0x0b, 0x8c, 0xc6, 0x7f, 0x8e, 0x2d, + 0xe5, 0xf8, 0x4a, 0x42, 0xd0, 0xc4, 0xef, 0xa4, 0xc5, 0x7a, 0xd3, 0xd0, + 0xec, 0xb5, 0x16, 0x31, 0x00, 0x8f, 0xdd, 0x6c, 0xa6, 0x48, 0x26, 0x15, + 0x9d, 0xcc, 0xf3, 0x77, 0xd1, 0xe4, 0xd4, 0xd4, 0xc1, 0x76, 0x18, 0x09, + 0x84, 0x54, 0xb2, 0x63, 0x88, 0xf8, 0xe6, 0x20, 0xbc, 0x83, 0x51, 0x5f, + 0xdf, 0xf9, 0xb0, 0xe1, 0x0d, 0x58, 0xe3, 0xb3, 0x70, 0x89, 0x97, 0xd8, + 0x76, 0xed, 0xff, 0x35, 0x96, 0x56, 0x52, 0x86, 0xab, 0x12, 0x53, 0xa6, + 0x52, 0x72, 0xbd, 0xf9, 0xf1, 0xfe, 0x90, 0x30, 0xfa, 0x0c, 0x7c, 0xd8, + 0x28, 0x30, 0xec, 0x0f, 0xd5, 0x55, 0x2a, 0x60, 0x63, 0xa7, 0xe9, 0x72, + 0x91, 0x72, 0x7b, 0xc0, 0xba, 0x67, 0x3c, 0x3f, 0x9d, 0x4f, 0x19, 0xb9, + 0x54, 0x2c, 0x9c, 0x00, 0x0a, 0x70, 0xdc, 0xb6, 0x5f, 0x3b, 0x3a, 0x8a, + 0x32, 0x9a, 0xc1, 0xb8, 0xfd, 0xa9, 0x2e, 0xfa, 0x4d, 0xdc, 0x76, 0x9a, + 0x59, 0x97, 0xff, 0x4b, 0x87, 0xe2, 0xb1, 0x3c, 0x84, 0xff, 0x72, 0xf3, + 0x5b, 0xe5, 0xca, 0x25, 0x0e, 0x4b, 0xf7, 0x2b, 0x5b, 0x7f, 0x28, 0x12, + 0x43, 0x5d, 0xab, 0x1a, 0x6c, 0xaf, 0x8a, 0x82, 0x80, 0xeb, 0x50, 0xa3, + 0xac, 0xf1, 0xd0, 0x9f, 0xed, 0xb2, 0x92, 0xec, 0x7c, 0xa3, 0xc1, 0x48, + 0x42, 0x41, 0x9f, 0x34, 0x43, 0x68, 0x2f, 0xd0, 0xd9, 0x81, 0x75, 0xa4, + 0xcb, 0x1b, 0xf4, 0xcf, 0x98, 0xd9, 0xb6, 0x58, 0x7b, 0xa8, 0x69, 0xf7, + 0x4d, 0xdf, 0x9f, 0xe0, 0xa9, 0x2c, 0x96, 0xf2, 0x60, 0xf0, 0x44, 0x7a, + 0xbf, 0x6d, 0xcb, 0x7a, 0x68, 0x3e, 0x6d, 0x65, 0xe6, 0x03, 0xd4, 0x01, + 0x55, 0xfd, 0xde, 0x47, 0xca, 0xf8, 0x05, 0x41, 0xf3, 0xd5, 0x61, 0x9c, + 0x63, 0xc0, 0xae, 0xc0, 0x08, 0xc1, 0x2f, 0x25, 0xbe, 0x9e, 0xf0, 0x73, + 0x97, 0xe6, 0x81, 0x33, 0xfd, 0x88, 0x72, 0x09, 0x5f, 0x6b, 0xdc, 0xa3, + 0x75, 0x73, 0x9f, 0x4b, 0x74, 0xb8, 0x05, 0x86, 0xec, 0xdc, 0xcf, 0xab, + 0xee, 0x24, 0xb9, 0xe6, 0xdd, 0xf1, 0x8d, 0xd6, 0x4b, 0x31, 0x21, 0x08, + 0x38, 0x29, 0x9a, 0x3a, 0x5c, 0xb6, 0x2b, 0x62, 0x0d, 0x66, 0x93, 0x23, + 0xe3, 0x7c, 0xbe, 0xc9, 0xf1, 0x19, 0x9c, 0xd6, 0x09, 0x19, 0x19, 0x63, + 0x7a, 0xee, 0xeb, 0xe2, 0x90, 0x9e, 0x5e, 0x20, 0x11, 0x2d, 0x02, 0x52, + 0x55, 0x9f, 0x26, 0x36, 0x30, 0xe9, 0xab, 0xdc, 0xac, 0x4f, 0xf4, 0x63, + 0x89, 0xa6, 0xe1, 0x1b, 0x1e, 0x7c, 0xbe, 0x3b, 0xb1, 0x5c, 0x6b, 0xbd, + 0x12, 0x8f, 0xbf, 0x83, 0x28, 0xb1, 0xdc, 0xd9, 0x4a, 0x97, 0xeb, 0xe9, + 0x05, 0xb2, 0xff, 0x5f, 0xae, 0x49, 0x82, 0xe4, 0x8c, 0x64, 0x2f, 0xb7, + 0xcc, 0x21, 0x68, 0x85, 0x58, 0xc3, 0x85, 0x67, 0x22, 0xbb, 0x99, 0x2a, + 0x36, 0xb9, 0xdf, 0x07, 0xed, 0x1d, 0xeb, 0x0c, 0x4a, 0x49, 0xc7, 0xdc, + 0xf8, 0x12, 0x5e, 0x0f, 0xc3, 0x5e, 0xee, 0x79, 0x6e, 0xaf, 0x11, 0x99, + 0x99, 0x84, 0x6e, 0x04, 0x17, 0x7e, 0x5c, 0x08, 0x99, 0x93, 0xea, 0x46, + 0x90, 0x7a, 0xde, 0xb9, 0xf3, 0xb2, 0x9c, 0x2f, 0x9d, 0xc5, 0x93, 0x42, + 0xae, 0xb5, 0x5e, 0x62, 0xd6, 0x6e, 0xb1, 0x9b, 0x1c, 0x88, 0xc0, 0x38, + 0x02, 0x95, 0xc4, 0x4e, 0x33, 0x2d, 0x21, 0x6d, 0x7c, 0xec, 0x74, 0x90, + 0x6f, 0xd6, 0x38, 0x02, 0x33, 0x5e, 0xaf, 0xfd, 0x0b, 0xca, 0xf4, 0xdb, + 0x3e, 0x41, 0xb4, 0xc6, 0x66, 0xb9, 0xbe, 0x68, 0x60, 0x9f, 0xdf, 0x72, + 0x8e, 0x52, 0x68, 0x94, 0x04, 0x58, 0xc2, 0xbe, 0xd4, 0x2c, 0xc9, 0xa8, + 0x36, 0xee, 0x78, 0xc3, 0xaf, 0xff, 0xbc, 0xe9, 0x4f, 0x90, 0xcb, 0xa7, + 0xc6, 0xd9, 0x1c, 0x2b, 0x7e, 0x06, 0xa0, 0xf0, 0x3a, 0x59, 0x14, 0x82, + 0x03, 0xad, 0xdd, 0x24, 0x82, 0x0c, 0x53, 0x51, 0x36, 0x17, 0xae, 0x82, + 0x95, 0x25, 0x95, 0xa5, 0x13, 0x50, 0x08, 0xb7, 0x5b, 0x38, 0x33, 0x5b, + 0xaf, 0x86, 0x08, 0x64, 0x38, 0xe4, 0x7d, 0x8e, 0xcb, 0x4a, 0x22, 0x7d, + 0x5e, 0x51, 0x84, 0x5e, 0x01, 0x3d, 0xb1, 0x42, 0xef, 0x23, 0x90, 0xd8, + 0x7e, 0xed, 0xc9, 0xd1, 0x71, 0xcf, 0x2b, 0xb3, 0x8a, 0xbf, 0x00, 0xd7, + 0xae, 0xa6, 0xbd, 0xf7, 0xb2, 0xf6, 0x89, 0x06, 0x59, 0xce, 0xe4, 0x42, + 0x61, 0x50, 0xcb, 0x6e, 0xfd, 0xab, 0x33, 0xa6, 0x89, 0xc6, 0xd2, 0xa5, + 0xed, 0x7a, 0xbd, 0x96, 0x09, 0x46, 0xe5, 0x5f, 0x79, 0x40, 0x6b, 0x5f, + 0x77, 0x42, 0xca, 0xf4, 0x7b, 0x4d, 0x83, 0x4c, 0x8e, 0xce, 0xb7, 0xba, + 0x0c, 0x55, 0x77, 0xbd, 0x11, 0x0a, 0xb9, 0x43, 0x0e, 0x5f, 0xe2, 0xb8, + 0x30, 0x2e, 0x0e, 0xa6, 0xef, 0xef, 0x14, 0x36, 0x76, 0x3f, 0x74, 0xc8, + 0x21, 0x83, 0xba, 0xcb, 0xcb, 0xd7, 0xad, 0x93, 0x2b, 0xd1, 0x10, 0x65, + 0xda, 0x54, 0xbc, 0xc7, 0x7b, 0x17, 0x7f, 0x76, 0x77, 0x38, 0xc0, 0xdb, + 0xf3, 0x1d, 0xc3, 0xac, 0x9b, 0xcf, 0xcb, 0x59, 0x0b, 0xd8, 0x67, 0x6a, + 0x6f, 0x34, 0xf1, 0x8d, 0x68, 0x20, 0xb4, 0x8a, 0xaf, 0xce, 0xf2, 0x98, + 0x87, 0xae, 0xb9, 0x98, 0x9e, 0xf1, 0xc9, 0x8c, 0x31, 0xaf, 0xb7, 0xeb, + 0x5a, 0x6a, 0x76, 0x76, 0x56, 0x4c, 0x42, 0xa2, 0xde, 0x71, 0x57, 0xa2, + 0xbe, 0xbe, 0x7e, 0xea, 0x84, 0xc7, 0x45, 0xcf, 0xfc, 0x64, 0xb1, 0x0a, + 0x42, 0x10, 0xb8, 0x99, 0x93, 0x12, 0x17, 0x1b, 0x40, 0x4b, 0xdd, 0x1a, + 0x6a, 0x02, 0xff, 0xd7, 0x34, 0xc3, 0x37, 0xef, 0x02, 0x82, 0x21, 0xaa, + 0x9c, 0x15, 0x3e, 0xa2, 0x7a, 0x27, 0xdd, 0xb5, 0xe5, 0xb6, 0x2a, 0x80, + 0xca, 0xc1, 0x9e, 0xcf, 0x53, 0xd0, 0x2a, 0x70, 0xcc, 0xc8, 0x83, 0x14, + 0x75, 0xdd, 0x2e, 0x36, 0x36, 0x8e, 0xad, 0x83, 0x8f, 0xb4, 0x42, 0xfd, + 0x8f, 0xe0, 0xd1, 0x45, 0x46, 0x79, 0xfd, 0xfe, 0x05, 0xf0, 0x89, 0xb9, + 0xab, 0x43, 0x6c, 0xd5, 0xe9, 0xf1, 0x72, 0xbb, 0x81, 0x98, 0xdd, 0xef, + 0xd5, 0xa3, 0x30, 0xc9, 0xe7, 0xf5, 0x6a, 0x18, 0x50, 0xa9, 0xd7, 0x91, + 0x02, 0xc9, 0x8e, 0x47, 0xbd, 0xe9, 0xd1, 0xcd, 0x77, 0x0f, 0x79, 0x4a, + 0x74, 0x91, 0x80, 0xa4, 0xca, 0x8c, 0x37, 0xe2, 0xdd, 0x50, 0x81, 0x50, + 0x90, 0x21, 0xf8, 0xf1, 0x6a, 0x97, 0x92, 0x9a, 0x1a, 0x1d, 0x0d, 0xed, + 0x37, 0x6a, 0x20, 0xe4, 0xb0, 0x5a, 0xdf, 0x17, 0x37, 0x08, 0x01, 0x28, + 0xe0, 0xa9, 0xa7, 0x15, 0x4b, 0x99, 0x99, 0x51, 0xbf, 0x3c, 0x5c, 0x98, + 0xdf, 0x1e, 0xce, 0xfe, 0xb3, 0xe1, 0x3a, 0x5e, 0xaa, 0x99, 0x86, 0x90, + 0x9e, 0x5c, 0x72, 0xab, 0xc5, 0xfb, 0xab, 0x21, 0xb9, 0xca, 0x86, 0x57, + 0x28, 0x7f, 0x6c, 0xcb, 0xfc, 0x49, 0x50, 0x52, 0x02, 0x0d, 0xc9, 0x69, + 0x28, 0x79, 0x30, 0xc0, 0xfe, 0xa4, 0xb6, 0x96, 0xc0, 0x4a, 0x1b, 0x13, + 0x05, 0x90, 0xef, 0xcd, 0x0f, 0xb3, 0x04, 0x05, 0x21, 0xfa, 0xa0, 0xef, + 0x0b, 0x5c, 0x8a, 0xe2, 0x4a, 0x92, 0xe3, 0x97, 0xb6, 0x36, 0x91, 0xb1, + 0xb1, 0x2c, 0x8e, 0x4b, 0x57, 0x36, 0xb4, 0xe1, 0xa3, 0x0c, 0xaa, 0xa5, + 0x80, 0x1f, 0xca, 0xca, 0xe8, 0x01, 0xdd, 0x69, 0xdb, 0xf5, 0x9c, 0x99, + 0xd3, 0x3c, 0xe7, 0x58, 0x22, 0x5b, 0x57, 0xc7, 0x8d, 0x40, 0xfa, 0x78, + 0xb5, 0x86, 0x61, 0x95, 0xba, 0xbe, 0x88, 0xbb, 0x09, 0xbf, 0xb7, 0x3b, + 0x8d, 0xe3, 0xf9, 0x32, 0xd1, 0x36, 0xa7, 0x7d, 0x07, 0xde, 0xac, 0xf7, + 0xa6, 0x9d, 0x2f, 0x1f, 0xaa, 0x6a, 0x54, 0xe4, 0xe4, 0x81, 0xe5, 0x6a, + 0x15, 0xd7, 0x6b, 0xce, 0xd3, 0x6f, 0xce, 0x1b, 0xb7, 0x2d, 0xe9, 0x77, + 0xb3, 0x97, 0xaf, 0x58, 0x21, 0xa9, 0xf1, 0x22, 0x0c, 0x81, 0xb3, 0xd3, + 0x42, 0xa7, 0x0d, 0x97, 0x5a, 0xe3, 0x95, 0x9e, 0xcd, 0x9f, 0x0a, 0xdc, + 0x97, 0xef, 0x15, 0xa5, 0x35, 0xbf, 0x80, 0x26, 0x22, 0x61, 0x3c, 0x2a, + 0x1f, 0xca, 0xc0, 0x03, 0x08, 0x6c, 0x89, 0x00, 0x9a, 0x53, 0xd2, 0x0e, + 0xd5, 0xc7, 0xbd, 0x44, 0x85, 0x91, 0xf1, 0xf1, 0xdd, 0x10, 0x1c, 0x7d, + 0xc0, 0xa5, 0xce, 0x8d, 0xf7, 0x79, 0x5b, 0x18, 0xc4, 0x8b, 0xe6, 0x20, + 0xe0, 0xad, 0xa2, 0x8b, 0xfc, 0xc2, 0xfd, 0xba, 0xa9, 0x40, 0xfc, 0xa5, + 0xe7, 0xdc, 0x37, 0xec, 0xf0, 0x6d, 0xef, 0x64, 0xaf, 0xf5, 0x3a, 0xd2, + 0xc5, 0xef, 0x66, 0x80, 0x57, 0xb5, 0x0a, 0x53, 0x74, 0x3d, 0xd8, 0xea, + 0xf6, 0xf0, 0xdf, 0xc1, 0x5f, 0xde, 0xda, 0x39, 0xec, 0x8a, 0x16, 0xd9, + 0xf2, 0xcc, 0x8f, 0xc9, 0x17, 0xc0, 0xa5, 0xa2, 0x48, 0xdf, 0xcd, 0xe9, + 0xda, 0x53, 0x86, 0xf8, 0xed, 0x38, 0x5a, 0xb1, 0x25, 0xf2, 0x70, 0x6a, + 0xa8, 0xe2, 0x3f, 0x8e, 0xbb, 0x24, 0x78, 0x4b, 0x5c, 0xea, 0x55, 0xb7, + 0x33, 0x18, 0x52, 0x5e, 0xde, 0x76, 0xb1, 0x55, 0x50, 0x91, 0x0c, 0xce, + 0xc1, 0x04, 0x8f, 0x33, 0x5d, 0x6d, 0x20, 0x91, 0xcd, 0x17, 0x5a, 0x52, + 0xaf, 0x0a, 0xf5, 0xae, 0x31, 0x56, 0x18, 0x14, 0x92, 0x7e, 0x33, 0x90, + 0xbc, 0xe4, 0x9d, 0x0f, 0x2f, 0x8e, 0x9a, 0x92, 0x8f, 0x96, 0x0e, 0x52, + 0xbf, 0x98, 0x1a, 0x96, 0xc1, 0x57, 0xec, 0x5d, 0x92, 0x16, 0xa1, 0x2e, + 0x2b, 0x71, 0xbf, 0x2c, 0x41, 0xe4, 0x75, 0x56, 0x56, 0xb9, 0xd1, 0xf8, + 0x79, 0x17, 0x0e, 0x35, 0x32, 0x36, 0x7e, 0x44, 0x35, 0x35, 0xd4, 0x4f, + 0x9f, 0xc9, 0xbb, 0x29, 0x30, 0x71, 0x92, 0xa2, 0x80, 0x8d, 0x64, 0x8c, + 0x8d, 0xb9, 0xf2, 0x59, 0x65, 0xf0, 0x68, 0xcf, 0x83, 0x53, 0x59, 0x35, + 0xeb, 0xd6, 0x5a, 0xb7, 0xe3, 0xa8, 0x79, 0x79, 0x1a, 0x3c, 0xf5, 0x0d, + 0xfe, 0xfe, 0x98, 0xb9, 0x1b, 0x58, 0x0b, 0xf2, 0x99, 0x40, 0xaa, 0xa7, + 0x9e, 0xf8, 0x28, 0xb3, 0xe4, 0xe7, 0x87, 0x0b, 0x7f, 0xf5, 0xfc, 0x0f, + 0x02, 0x4e, 0xeb, 0x6c, 0x5c, 0x91, 0xe2, 0x62, 0x92, 0xe1, 0xe2, 0xf2, + 0x12, 0xaa, 0xf6, 0x24, 0xf4, 0x8e, 0xc9, 0xa1, 0xb5, 0x90, 0x6e, 0x76, + 0xab, 0xd7, 0xd4, 0xd6, 0x1b, 0x7e, 0x8f, 0x4e, 0xde, 0x48, 0x72, 0x58, + 0x27, 0x5e, 0x0f, 0xd3, 0xcb, 0xa0, 0xc6, 0x6e, 0xde, 0x88, 0x3e, 0x2c, + 0x53, 0x78, 0xb7, 0xd1, 0xac, 0x2a, 0x11, 0xcb, 0x1f, 0x74, 0x4d, 0xe5, + 0x73, 0x4f, 0xcf, 0x29, 0x13, 0x7a, 0xad, 0xca, 0xd2, 0xe9, 0xbe, 0x15, + 0xfb, 0xad, 0x06, 0xa1, 0xdf, 0xda, 0x8b, 0x39, 0x49, 0xf9, 0xbc, 0x0b, + 0x36, 0xf9, 0xfd, 0xaf, 0xee, 0x8f, 0x10, 0x50, 0xb1, 0xf7, 0x24, 0x58, + 0x46, 0x2e, 0xd4, 0x13, 0x5a, 0xb7, 0x43, 0x09, 0x5e, 0xb1, 0xdb, 0x69, + 0x71, 0xd9, 0xcd, 0xfe, 0x08, 0x49, 0xc0, 0x18, 0x98, 0x2d, 0xfc, 0xd5, + 0xe5, 0x2e, 0x21, 0xb6, 0x46, 0x6d, 0x5d, 0xf3, 0xfc, 0xf3, 0xe2, 0x42, + 0xbd, 0x74, 0xeb, 0x91, 0x60, 0x6e, 0xf0, 0x91, 0x36, 0x71, 0xaa, 0xfb, + 0x31, 0xc5, 0x0d, 0x4a, 0xda, 0xdb, 0x6b, 0xc7, 0xe3, 0xbf, 0x86, 0x3e, + 0x27, 0xe0, 0xc0, 0xef, 0x8d, 0x00, 0x59, 0x9b, 0x56, 0x85, 0xb6, 0x6a, + 0x2c, 0x1d, 0x79, 0x2b, 0xd0, 0x5c, 0x32, 0xc1, 0x22, 0x68, 0x2c, 0x1c, + 0x0b, 0x30, 0x45, 0xf9, 0x69, 0x91, 0x71, 0xad, 0xb1, 0xde, 0xaf, 0x7d, + 0x31, 0x13, 0xfd, 0x01, 0x4f, 0x36, 0x90, 0x7e, 0x90, 0x0d, 0xcc, 0x8e, + 0xd2, 0x38, 0x13, 0x93, 0x1e, 0x85, 0x14, 0xfa, 0x7f, 0x3f, 0x03, 0x3e, + 0x41, 0xda, 0xba, 0xfb, 0x90, 0xf1, 0x12, 0xc4, 0xc3, 0x26, 0x52, 0xcb, + 0xc9, 0xe3, 0x6a, 0xd0, 0x6a, 0x54, 0xeb, 0xbe, 0x0f, 0xc5, 0xf9, 0x87, + 0xd1, 0x26, 0x74, 0xc0, 0x74, 0x0d, 0xa5, 0x74, 0x3f, 0xfc, 0xbd, 0xf7, + 0x3a, 0x5b, 0x6d, 0xa1, 0x61, 0x63, 0x4b, 0x80, 0xa8, 0x47, 0xed, 0x76, + 0x8d, 0x5e, 0xe5, 0x6a, 0x7d, 0x62, 0xe1, 0x33, 0x9c, 0xa9, 0x7a, 0x59, + 0x08, 0x2a, 0xa5, 0x6a, 0x85, 0x64, 0x74, 0x39, 0x96, 0xe4, 0x86, 0xfb, + 0x67, 0x10, 0xff, 0x2b, 0x9b, 0x62, 0x31, 0x77, 0x87, 0x2e, 0x36, 0xb6, + 0x00, 0xf5, 0x41, 0x1c, 0xb5, 0xd8, 0x23, 0x85, 0x48, 0x21, 0x51, 0x2a, + 0xdb, 0x8b, 0xa8, 0xc3, 0x03, 0x74, 0x70, 0xbf, 0x1d, 0xdf, 0xae, 0xe7, + 0x42, 0x85, 0xea, 0xa5, 0xff, 0xbd, 0x07, 0x3f, 0x55, 0xb5, 0xae, 0x7c, + 0x49, 0x22, 0x64, 0xb8, 0xb8, 0xbf, 0x7f, 0x27, 0xc2, 0xd1, 0x28, 0xe4, + 0xcb, 0x03, 0x79, 0x6d, 0xfe, 0xb1, 0x2a, 0x2b, 0x2b, 0x5b, 0xcc, 0x15, + 0x8e, 0x4a, 0xa5, 0x34, 0xf7, 0x24, 0xe0, 0x6c, 0x04, 0x3e, 0x3f, 0x9c, + 0x43, 0xeb, 0x2e, 0x3f, 0xd5, 0xb1, 0x27, 0xbc, 0xf6, 0xa3, 0x08, 0x99, + 0x9c, 0xad, 0x34, 0x4d, 0x64, 0x70, 0x9e, 0xe4, 0xd4, 0x05, 0x6c, 0x5f, + 0xb0, 0xef, 0x4d, 0x2a, 0xa4, 0x77, 0x5b, 0x60, 0xb5, 0x1b, 0xa2, 0xe2, + 0xa3, 0x0f, 0x6c, 0x76, 0xa0, 0xd4, 0x3a, 0xad, 0xab, 0x9b, 0xbd, 0x35, + 0x12, 0xa6, 0xf2, 0xa1, 0x64, 0xca, 0x31, 0x34, 0xdd, 0x6c, 0x0f, 0x42, + 0xc5, 0x6a, 0x98, 0xcf, 0xab, 0xa0, 0x1c, 0xb8, 0xf7, 0x65, 0x5b, 0x99, + 0xae, 0xe7, 0xa7, 0x4a, 0xe7, 0x4c, 0xd9, 0xd9, 0x47, 0x0c, 0x37, 0xa0, + 0xab, 0x31, 0x6d, 0x6e, 0xb2, 0xb2, 0x19, 0x7c, 0xb5, 0xc7, 0xec, 0x89, + 0xb6, 0xcd, 0x90, 0xee, 0x90, 0xc3, 0xb8, 0xc1, 0x7e, 0x47, 0x7e, 0x94, + 0x3e, 0x63, 0xce, 0xe6, 0x3a, 0x68, 0xac, 0x2e, 0x93, 0x03, 0x59, 0xcc, + 0x6c, 0x93, 0xf5, 0xa4, 0x66, 0x51, 0x91, 0x6d, 0xef, 0x68, 0xb6, 0xb1, + 0x9f, 0x45, 0xf2, 0xad, 0x6f, 0x8a, 0x88, 0x88, 0x8c, 0x4c, 0x4e, 0x9a, + 0xae, 0xf6, 0x8d, 0xee, 0x79, 0xf2, 0x77, 0x7e, 0x3c, 0x11, 0x52, 0xfb, + 0x71, 0x5e, 0x34, 0x3e, 0x58, 0xcc, 0x97, 0xa2, 0x11, 0x70, 0xdb, 0xf1, + 0x5f, 0x70, 0x71, 0xc9, 0x54, 0x4e, 0xce, 0xd0, 0x9c, 0x93, 0x75, 0x69, + 0x7f, 0xf9, 0x94, 0x7c, 0xa1, 0x2c, 0x9d, 0xe1, 0x72, 0xad, 0x66, 0x5f, + 0xbf, 0x6c, 0xd5, 0x7c, 0xe7, 0xe5, 0xce, 0xd4, 0xa4, 0xfb, 0xa0, 0xbc, + 0xef, 0x65, 0xca, 0xf9, 0xe9, 0x45, 0x23, 0xd4, 0xb1, 0xd2, 0xd0, 0x67, + 0xe1, 0xbb, 0xa7, 0xff, 0x03, 0xf2, 0x9a, 0xf3, 0xc9, 0x9d, 0x47, 0xa7, + 0x3e, 0x59, 0xf0, 0x56, 0x86, 0x19, 0x92, 0xe7, 0x7b, 0xb4, 0x94, 0x6a, + 0x19, 0xa6, 0xf3, 0xd0, 0x23, 0xc8, 0x4c, 0x05, 0x0e, 0xce, 0x29, 0x61, + 0xd7, 0x29, 0x31, 0x25, 0xc5, 0x6d, 0x36, 0xdc, 0xaf, 0x98, 0x2b, 0xeb, + 0xd8, 0x3e, 0x0c, 0x85, 0x84, 0x9a, 0x9b, 0x3b, 0x35, 0xee, 0x17, 0x4d, + 0x46, 0xf9, 0xe5, 0x62, 0xd2, 0xb2, 0x0f, 0x87, 0xeb, 0x3b, 0xde, 0x85, + 0x52, 0x97, 0xcf, 0x53, 0x43, 0xf3, 0x6e, 0xc6, 0x01, 0x52, 0xa9, 0x6e, + 0x1e, 0x3f, 0x96, 0xf1, 0x78, 0x7c, 0xf2, 0x99, 0xd1, 0x43, 0x57, 0x3f, + 0xa3, 0x4e, 0x03, 0x5e, 0x8d, 0x86, 0xf9, 0x64, 0xce, 0xea, 0xd1, 0xca, + 0x03, 0x55, 0xe8, 0x7d, 0x77, 0x18, 0x31, 0x51, 0x0d, 0x92, 0xe4, 0x49, + 0xb5, 0x25, 0xbd, 0xad, 0x0e, 0x75, 0xbd, 0xc9, 0x7e, 0x8e, 0x07, 0x10, + 0xc6, 0x1e, 0x8a, 0x31, 0x3f, 0x35, 0x43, 0x53, 0x24, 0xc2, 0x83, 0x8f, + 0xc5, 0xfe, 0x26, 0x31, 0xc0, 0xe7, 0xbc, 0x4e, 0xff, 0xf6, 0xc8, 0x31, + 0x71, 0xc4, 0x15, 0x1c, 0xbf, 0xeb, 0xf4, 0xe2, 0x7d, 0x34, 0xf0, 0x2a, + 0x5b, 0xe0, 0x7d, 0x4f, 0xc4, 0xb3, 0xdb, 0x84, 0xd8, 0x4f, 0xd9, 0xa5, + 0x4d, 0x98, 0xd9, 0x4c, 0x95, 0xfa, 0x38, 0xd6, 0x8a, 0xd0, 0x0b, 0x41, + 0x0b, 0x28, 0x07, 0xde, 0x53, 0x13, 0x93, 0x83, 0xbf, 0xc9, 0x7b, 0x80, + 0x28, 0x1c, 0x16, 0xc1, 0xb3, 0x4d, 0xbb, 0x8e, 0x3a, 0x2a, 0x94, 0x4d, + 0x57, 0x04, 0xfd, 0xd8, 0x18, 0xc3, 0x0c, 0x3f, 0xf4, 0x8a, 0xf2, 0xbe, + 0xc4, 0x8f, 0xa3, 0x31, 0x94, 0xa8, 0x25, 0x56, 0xdb, 0x73, 0xe5, 0x39, + 0xb6, 0x7a, 0xdc, 0xa7, 0x8d, 0x54, 0xd5, 0x2a, 0x5c, 0xd1, 0xa8, 0x21, + 0xd7, 0x4f, 0x76, 0x33, 0x53, 0xac, 0xb2, 0xd9, 0x7e, 0x7b, 0xf2, 0x29, + 0x95, 0xaf, 0x18, 0x97, 0xc8, 0x80, 0xec, 0x16, 0xa3, 0x5b, 0x99, 0x3f, + 0xd2, 0xc9, 0xc4, 0x70, 0xd0, 0x3a, 0xc5, 0x96, 0xc4, 0xfe, 0xca, 0x6a, + 0x58, 0x7e, 0xcf, 0xaf, 0xae, 0xb3, 0x15, 0xe3, 0xae, 0x7b, 0xed, 0x37, + 0xf2, 0xf5, 0xe1, 0x7e, 0x27, 0xce, 0xd8, 0xc0, 0x89, 0x1b, 0x8f, 0x23, + 0x37, 0x10, 0x27, 0xcd, 0x40, 0x4f, 0x11, 0xda, 0xf2, 0xf2, 0xf2, 0xbb, + 0xa2, 0xc9, 0x7e, 0x7b, 0x15, 0x8e, 0x4b, 0xd5, 0xcf, 0x8b, 0xa3, 0x01, + 0x60, 0x4b, 0xcb, 0xf0, 0xfe, 0xd8, 0xcd, 0x34, 0x2f, 0x66, 0x65, 0x20, + 0xbb, 0x5c, 0xc4, 0x70, 0x01, 0xa8, 0x5a, 0xcd, 0x58, 0x90, 0x80, 0x3d, + 0x4d, 0xa7, 0xda, 0x3a, 0x06, 0x75, 0xc6, 0x7d, 0xa4, 0x75, 0xdc, 0x28, + 0xe2, 0xe4, 0x29, 0x57, 0xc4, 0x45, 0xf8, 0x1b, 0x5c, 0xbf, 0x14, 0x9e, + 0x44, 0xd9, 0xcc, 0x7b, 0x6e, 0x2e, 0xf6, 0x3e, 0x0a, 0xf1, 0x2b, 0x25, + 0x46, 0xa7, 0x21, 0x67, 0x85, 0xda, 0x5e, 0x6c, 0x1b, 0x46, 0x78, 0xb4, + 0xa0, 0x6f, 0x53, 0x6c, 0xe7, 0x2b, 0x79, 0x78, 0xfc, 0x76, 0xde, 0xdf, + 0x2a, 0x17, 0x73, 0x90, 0x44, 0xd3, 0x2b, 0x76, 0x6d, 0x63, 0x7b, 0x09, + 0xa4, 0x36, 0x3e, 0x6d, 0x95, 0x17, 0xb6, 0xf3, 0x7f, 0xb3, 0x79, 0xf5, + 0x34, 0x28, 0xd1, 0x3b, 0xa1, 0xb4, 0x6b, 0xe5, 0x2d, 0x93, 0xdc, 0x0e, + 0x1a, 0x2e, 0xb8, 0x99, 0xc9, 0x22, 0xc2, 0x65, 0x1f, 0x68, 0xd1, 0xbd, + 0x8d, 0x27, 0xdb, 0xfd, 0xa4, 0xdc, 0x4b, 0xe4, 0xc6, 0x17, 0x7d, 0x3c, + 0x00, 0x4d, 0x9b, 0xa1, 0xfe, 0x44, 0x15, 0x75, 0x98, 0x0a, 0xbb, 0xbd, + 0xbd, 0x6e, 0xc3, 0xbc, 0x44, 0xf2, 0xb3, 0x27, 0x69, 0x12, 0x7f, 0xc1, + 0xec, 0xfe, 0x27, 0x0b, 0xc5, 0x24, 0x08, 0x43, 0x19, 0x8d, 0x5c, 0x48, + 0x08, 0x77, 0xbe, 0x4c, 0x6b, 0xbf, 0xd2, 0x0c, 0xe8, 0xe6, 0x83, 0x95, + 0x70, 0x4c, 0xc3, 0xea, 0x79, 0xbd, 0xa7, 0xac, 0x35, 0x35, 0x72, 0x72, + 0x57, 0xef, 0x72, 0xa4, 0x9c, 0x4f, 0x2e, 0x82, 0xb7, 0x72, 0x61, 0xc7, + 0x71, 0xf6, 0xaa, 0x2f, 0x55, 0xc1, 0x70, 0x32, 0xfb, 0xd1, 0xe5, 0x3c, + 0xce, 0x45, 0xf1, 0x3b, 0x43, 0x6d, 0x57, 0xb2, 0x3c, 0xa7, 0x67, 0x91, + 0xc8, 0x5b, 0x09, 0x13, 0xcd, 0xa1, 0xae, 0xdd, 0x32, 0xa3, 0x93, 0xff, + 0x50, 0x10, 0x50, 0x76, 0x02, 0x13, 0xca, 0x47, 0x18, 0x8d, 0x15, 0x75, + 0x88, 0x83, 0x26, 0x22, 0xe6, 0x7c, 0x52, 0x95, 0x70, 0x7a, 0xfa, 0x85, + 0xaf, 0x57, 0x51, 0x55, 0x0d, 0x8d, 0xa4, 0x5b, 0xb0, 0x54, 0x12, 0x4b, + 0x72, 0xe1, 0x95, 0xe7, 0xea, 0x6b, 0x53, 0x79, 0xce, 0x9d, 0xe0, 0x39, + 0x3f, 0x4a, 0xec, 0xc3, 0x79, 0xdf, 0x54, 0x16, 0x9f, 0x56, 0x87, 0x9b, + 0x6d, 0x99, 0xb2, 0x09, 0x59, 0xb0, 0x7f, 0xce, 0xeb, 0x9b, 0xbb, 0x4b, + 0x4f, 0x52, 0x02, 0x41, 0x8d, 0xa5, 0x57, 0x8c, 0x31, 0x7e, 0x60, 0x76, + 0x56, 0xcd, 0x7e, 0x37, 0x8a, 0x7b, 0x3f, 0xc7, 0xf4, 0xdb, 0x32, 0x22, + 0x68, 0xaf, 0x3d, 0x25, 0xc2, 0x03, 0xc8, 0xdb, 0xfd, 0xda, 0xcc, 0x4c, + 0x8a, 0xb3, 0x34, 0x5f, 0x8a, 0x0e, 0x46, 0xb9, 0xbd, 0xbe, 0xbe, 0xff, + 0xc7, 0x94, 0x6f, 0xa0, 0x6a, 0x9d, 0x32, 0xf6, 0x97, 0x27, 0x4f, 0x54, + 0xf6, 0xc6, 0x03, 0xeb, 0xc7, 0x9a, 0xf7, 0x0b, 0xd2, 0xbb, 0xe6, 0xf5, + 0x9f, 0x3d, 0xc5, 0xcf, 0x15, 0xdc, 0xaf, 0x5f, 0x7e, 0x34, 0x49, 0x63, + 0x45, 0x8a, 0x15, 0x1f, 0x3c, 0x65, 0x12, 0xf2, 0xba, 0x9f, 0xeb, 0x96, + 0x63, 0x3e, 0x45, 0x73, 0x6f, 0xfc, 0x1a, 0x4d, 0xa0, 0x7a, 0xaa, 0x73, + 0x35, 0x2f, 0x4c, 0x9a, 0x8f, 0x64, 0xf1, 0x7a, 0xba, 0x58, 0x94, 0x3b, + 0x61, 0x07, 0xef, 0x6e, 0xff, 0x27, 0x0f, 0x81, 0xc5, 0x23, 0xb2, 0x47, + 0x51, 0xfb, 0x39, 0xf8, 0x3c, 0xdf, 0x48, 0x7f, 0x6e, 0x62, 0x89, 0xb3, + 0x9e, 0x01, 0x71, 0xae, 0x85, 0xac, 0x73, 0xb4, 0xc4, 0xba, 0xc3, 0x15, + 0x17, 0xa4, 0xe5, 0x48, 0xcd, 0x93, 0x62, 0xa3, 0x02, 0x85, 0x3f, 0x8e, + 0xd4, 0x39, 0x18, 0xc2, 0x2f, 0x51, 0x08, 0x39, 0xe5, 0x2a, 0xd5, 0x0a, + 0x9d, 0x6d, 0x20, 0x15, 0xac, 0x1e, 0xcd, 0xe3, 0x72, 0x88, 0xda, 0xfd, + 0x76, 0xaf, 0x53, 0x86, 0xf6, 0x1e, 0xc5, 0xd2, 0x05, 0xdb, 0x73, 0x49, + 0x73, 0xd1, 0x9f, 0x05, 0xee, 0xe5, 0xa0, 0xaa, 0xaa, 0xa9, 0xf4, 0x5f, + 0xd9, 0x35, 0x1e, 0xff, 0x89, 0x4a, 0x40, 0x34, 0xba, 0x20, 0x47, 0x00, + 0x19, 0x14, 0x8c, 0xfb, 0x9a, 0xb3, 0xea, 0x08, 0xd6, 0xfb, 0x27, 0x42, + 0x4a, 0xf6, 0x59, 0xe7, 0xe6, 0xe1, 0x74, 0xa9, 0x54, 0x99, 0xec, 0x57, + 0xea, 0xf8, 0x62, 0xb9, 0x12, 0xdd, 0x66, 0xb2, 0xc2, 0x59, 0x19, 0x2d, + 0x1a, 0x99, 0x69, 0xf4, 0x61, 0xaa, 0x8d, 0xd7, 0x24, 0x07, 0xfb, 0xf9, + 0x34, 0x0a, 0x86, 0xb8, 0xec, 0xd3, 0x08, 0xb9, 0x0e, 0x25, 0xd9, 0x96, + 0x87, 0x3a, 0x33, 0x40, 0x58, 0xbc, 0x72, 0xa5, 0xec, 0xe4, 0xeb, 0xb4, + 0xe0, 0x4a, 0x94, 0x38, 0x03, 0x97, 0xc3, 0xcd, 0x24, 0x77, 0x8e, 0xfb, + 0x7d, 0xf2, 0x8e, 0x18, 0xa4, 0x83, 0x5e, 0x79, 0xf0, 0xbb, 0x74, 0xb4, + 0x21, 0xb9, 0xaf, 0xf9, 0x20, 0xb5, 0x1b, 0x3a, 0x6c, 0x39, 0xd0, 0x03, + 0x9f, 0x5f, 0x9a, 0xd3, 0xef, 0x4f, 0x5d, 0x98, 0x5a, 0x36, 0xe7, 0xf1, + 0xe1, 0x4a, 0x23, 0x73, 0x29, 0xdd, 0xd2, 0xb0, 0x34, 0x2f, 0x72, 0xfc, + 0xf5, 0x60, 0x7a, 0xc6, 0x83, 0x2e, 0x50, 0x34, 0xc5, 0x35, 0xd4, 0x92, + 0x0f, 0xa2, 0x4f, 0x6a, 0xbd, 0xfa, 0xb9, 0x64, 0xd9, 0xc7, 0x2d, 0x1b, + 0x16, 0xa6, 0xaf, 0xbc, 0x5a, 0x92, 0xf0, 0xf2, 0x0d, 0x44, 0x96, 0xd6, + 0x6d, 0xf4, 0x50, 0xb9, 0x39, 0xcb, 0x55, 0x76, 0xff, 0x30, 0x17, 0x24, + 0xa1, 0x43, 0x18, 0xc2, 0x76, 0xca, 0xc8, 0xce, 0xd7, 0x02, 0x97, 0x2f, + 0x01, 0x25, 0xb7, 0x8c, 0xde, 0x90, 0x4c, 0x5c, 0xe1, 0x5e, 0xeb, 0x1b, + 0x57, 0x64, 0x8b, 0xfe, 0xa1, 0x8c, 0x9b, 0x1b, 0x17, 0x84, 0x6c, 0xf9, + 0x85, 0x67, 0x59, 0xb8, 0xbc, 0x6e, 0x0f, 0x7b, 0x4f, 0x97, 0x1b, 0xda, + 0xe8, 0xcb, 0x88, 0x59, 0x50, 0xda, 0x71, 0xd6, 0x59, 0xbf, 0x67, 0x27, + 0xf7, 0x13, 0xc5, 0x02, 0x03, 0x51, 0x03, 0x15, 0xf4, 0x49, 0x24, 0x63, + 0xe5, 0x6d, 0x15, 0xd5, 0xbe, 0x64, 0x16, 0xe7, 0xd3, 0x4a, 0x47, 0xd8, + 0x50, 0x4c, 0x8e, 0xba, 0xde, 0xec, 0x85, 0x47, 0x04, 0xd8, 0x7f, 0xa5, + 0xa8, 0x93, 0xf7, 0xf9, 0xc1, 0x12, 0x95, 0xc5, 0x9a, 0x2f, 0x24, 0xb8, + 0xbb, 0xa3, 0xa9, 0x93, 0x22, 0xe9, 0x37, 0xca, 0xe8, 0x65, 0x73, 0x4a, + 0x7e, 0x77, 0x38, 0xeb, 0x36, 0x89, 0xd0, 0x15, 0xeb, 0x73, 0xdc, 0xdb, + 0xc0, 0x8c, 0x77, 0xb6, 0xe2, 0xfe, 0xdd, 0xff, 0x42, 0x50, 0xb1, 0x34, + 0x4c, 0x7d, 0x8a, 0x8e, 0x99, 0x53, 0x6e, 0xf4, 0xae, 0x2a, 0x71, 0x26, + 0x4b, 0x58, 0xed, 0xee, 0xff, 0x6b, 0x01, 0x38, 0x92, 0x27, 0x33, 0xb9, + 0x41, 0xcd, 0xb6, 0x3a, 0x29, 0x8c, 0x5a, 0x4e, 0xb3, 0x27, 0x69, 0x77, + 0xa7, 0xcb, 0x89, 0x10, 0x17, 0x0b, 0x7d, 0xd9, 0x3b, 0xf2, 0x7e, 0xd8, + 0xee, 0x28, 0x52, 0x7d, 0x6f, 0xfe, 0x7c, 0x3f, 0x92, 0xc3, 0x55, 0x38, + 0x40, 0x3f, 0x52, 0x1a, 0x19, 0x38, 0x59, 0x3c, 0xfb, 0xe6, 0x98, 0xf3, + 0x90, 0x4b, 0x0f, 0x7f, 0x43, 0x13, 0x4b, 0x8f, 0xc5, 0xb9, 0x2e, 0x8d, + 0xce, 0x95, 0x19, 0x9a, 0x0f, 0x6b, 0x98, 0x51, 0x6d, 0x95, 0xa1, 0x5e, + 0xd6, 0x5c, 0x4d, 0xb2, 0x66, 0xa7, 0x75, 0x8b, 0x2e, 0x66, 0x65, 0x75, + 0x91, 0x16, 0x38, 0x8d, 0xdc, 0x11, 0x7d, 0x39, 0x03, 0xd2, 0x31, 0xab, + 0xb1, 0x8b, 0x9b, 0xbf, 0x9a, 0xc5, 0x90, 0x44, 0xee, 0x52, 0x33, 0xf9, + 0x82, 0x20, 0x12, 0x89, 0x1f, 0x05, 0x4f, 0x96, 0xc6, 0x67, 0x49, 0xb9, + 0x55, 0xd1, 0x81, 0x8d, 0x85, 0x25, 0x6e, 0x65, 0x75, 0x2c, 0x7e, 0xa9, + 0xef, 0x23, 0xa0, 0x3c, 0x1c, 0x43, 0xf4, 0x74, 0x90, 0x69, 0x0f, 0x48, + 0x7b, 0xde, 0x40, 0x4d, 0xf2, 0xe9, 0x03, 0xde, 0xd5, 0xc5, 0x3e, 0x32, + 0xe1, 0x2f, 0xe5, 0x99, 0x26, 0xf2, 0x0e, 0x72, 0x00, 0x8a, 0x7e, 0xb5, + 0x58, 0x7e, 0x3f, 0xe5, 0x0a, 0x0c, 0x22, 0x2c, 0x34, 0x37, 0xbb, 0x59, + 0x0f, 0xdc, 0xd4, 0x6a, 0xf3, 0x23, 0x10, 0x1d, 0xf6, 0x93, 0x4f, 0x0d, + 0x0b, 0x85, 0xee, 0x3a, 0x1a, 0x7e, 0xca, 0x89, 0x26, 0x1f, 0xb1, 0x64, + 0xc1, 0x58, 0xd5, 0xea, 0xce, 0xba, 0x54, 0x7e, 0x67, 0xb8, 0xba, 0x6d, + 0x62, 0xf4, 0x8a, 0xe8, 0x77, 0x16, 0xef, 0xa5, 0xe9, 0x98, 0x0c, 0x75, + 0xcc, 0xb0, 0x24, 0x2b, 0xe9, 0x9f, 0xc2, 0xe8, 0xdf, 0xe0, 0x8b, 0x0e, + 0xb9, 0xb7, 0x3d, 0xe2, 0xd7, 0x50, 0x60, 0x67, 0xa6, 0x8e, 0x1c, 0x04, + 0x0d, 0x39, 0xbc, 0xad, 0x4f, 0xc7, 0xc5, 0xc7, 0x0b, 0xdb, 0xd8, 0x30, + 0x82, 0x8a, 0xe5, 0x0a, 0xcc, 0xa7, 0xb3, 0x07, 0xed, 0x66, 0xe1, 0xab, + 0x53, 0xb3, 0xb2, 0xe4, 0xa2, 0xb0, 0x99, 0x1e, 0x4f, 0x58, 0x36, 0x6d, + 0x7a, 0x11, 0xde, 0xce, 0x8b, 0x85, 0xab, 0xf0, 0xd3, 0x8a, 0x23, 0xe4, + 0xd4, 0x96, 0x81, 0xd3, 0x1a, 0xa9, 0xbe, 0x5b, 0x41, 0x2d, 0x86, 0xb8, + 0x60, 0x83, 0x27, 0x31, 0x90, 0x2a, 0x87, 0x9a, 0x61, 0xef, 0x35, 0xd2, + 0x6b, 0xed, 0x1e, 0x4a, 0x4c, 0x91, 0xb9, 0x51, 0x06, 0xb5, 0x16, 0x4f, + 0x75, 0xa2, 0x7a, 0xd7, 0x82, 0x4c, 0xa9, 0x44, 0x37, 0x8c, 0xdd, 0xee, + 0xb3, 0xba, 0x14, 0xc4, 0x6f, 0x98, 0x9a, 0x4d, 0x7b, 0x89, 0x34, 0x7d, + 0xaf, 0xb6, 0xcf, 0xa2, 0xc4, 0x66, 0xde, 0x64, 0xc5, 0xf5, 0xa0, 0x67, + 0x64, 0x0c, 0xad, 0xac, 0xc4, 0x8c, 0x3a, 0xa4, 0x82, 0x02, 0x4d, 0x94, + 0xcb, 0xbc, 0xff, 0x58, 0x0b, 0x72, 0xdf, 0x08, 0xf6, 0xc4, 0x1f, 0xef, + 0x7c, 0x9e, 0xb6, 0x7d, 0x6d, 0x36, 0xbb, 0xb1, 0x64, 0x5a, 0x7c, 0x0a, + 0x4d, 0x84, 0xb4, 0x35, 0x9b, 0xc7, 0x77, 0x51, 0x3f, 0xad, 0xe6, 0x14, + 0x61, 0x0a, 0xa1, 0x42, 0x8d, 0x1a, 0x7d, 0x9f, 0xbb, 0x05, 0x8d, 0x89, + 0x6c, 0x81, 0x0e, 0xcd, 0x1c, 0x15, 0xd8, 0xde, 0x9e, 0xdf, 0xd0, 0x9e, + 0x0f, 0xeb, 0xee, 0x6f, 0xdd, 0xbe, 0xb7, 0x2b, 0xa4, 0xda, 0x9c, 0x65, + 0x60, 0x03, 0x96, 0x5c, 0x82, 0xaf, 0xa3, 0x2f, 0xb1, 0xa8, 0xf8, 0xa7, + 0x94, 0x6c, 0xfc, 0x9f, 0x53, 0x5f, 0xd6, 0x24, 0x66, 0xbf, 0x74, 0xb5, + 0xdb, 0xa8, 0x74, 0xeb, 0xf4, 0x35, 0xeb, 0xbd, 0x48, 0x6d, 0xf8, 0xef, + 0x15, 0x5a, 0xaf, 0x79, 0xec, 0x46, 0xdb, 0xfc, 0xab, 0xfd, 0x5d, 0xf0, + 0x33, 0xf6, 0x93, 0x22, 0x77, 0x1e, 0xbd, 0x33, 0x75, 0x34, 0x29, 0xcd, + 0x92, 0x9c, 0xb0, 0x6f, 0x7b, 0x7d, 0x30, 0x98, 0xa4, 0x3b, 0x2a, 0x64, + 0x7c, 0x89, 0x22, 0xb8, 0x3f, 0x2b, 0x39, 0x79, 0x67, 0xbe, 0x91, 0xfe, + 0xf0, 0xd9, 0x6f, 0xdc, 0x95, 0xe5, 0x3e, 0x93, 0x1e, 0x59, 0xa1, 0xf7, + 0xc7, 0xda, 0x7a, 0x4b, 0x33, 0xdf, 0x99, 0x63, 0x58, 0x4c, 0x71, 0xd6, + 0xa0, 0x76, 0xb6, 0x3f, 0x01, 0x62, 0xff, 0x5e, 0xbb, 0x7a, 0xf4, 0x93, + 0xdd, 0x1c, 0x03, 0x87, 0x42, 0x65, 0xff, 0x5f, 0x1e, 0x67, 0x58, 0x78, + 0xf8, 0x9d, 0x8a, 0x9a, 0xae, 0xf6, 0x5f, 0x38, 0x44, 0x5b, 0x93, 0xdc, + 0x9b, 0x03, 0x76, 0x2d, 0x90, 0x73, 0x16, 0x80, 0x9a, 0x6c, 0x0c, 0xbe, + 0xa7, 0x63, 0x47, 0x9f, 0xfc, 0x55, 0xf4, 0x6c, 0xdc, 0x4a, 0x2a, 0x0d, + 0xaf, 0x75, 0x1e, 0x6b, 0xd5, 0x8e, 0x7b, 0x74, 0x1a, 0x42, 0xe9, 0x9e, + 0xb5, 0xee, 0xb1, 0x80, 0xf8, 0x80, 0xc0, 0x2a, 0xbe, 0xfb, 0x6d, 0x27, + 0xef, 0xf0, 0xc8, 0x48, 0xa3, 0x83, 0x89, 0x74, 0xc8, 0xd5, 0x41, 0xd1, + 0x51, 0x09, 0xbd, 0xdd, 0x62, 0xfc, 0x8c, 0x21, 0xcc, 0xb3, 0x5e, 0xaa, + 0x0e, 0xb0, 0x47, 0x8b, 0x1f, 0x2e, 0x50, 0x51, 0x47, 0xd3, 0x11, 0x4c, + 0x80, 0x38, 0x19, 0xc9, 0x74, 0xb0, 0x62, 0x52, 0x78, 0xd9, 0x6f, 0xf9, + 0x6b, 0x19, 0x09, 0x17, 0xba, 0x64, 0xac, 0x2f, 0x39, 0xec, 0x0d, 0xce, + 0x0a, 0xde, 0x6b, 0x55, 0xdc, 0x8d, 0x85, 0x76, 0x7d, 0x65, 0xce, 0x5f, + 0x4d, 0xf6, 0xf7, 0xb0, 0xdb, 0xea, 0x47, 0xd3, 0x19, 0xed, 0x20, 0x16, + 0xbe, 0xcb, 0xe3, 0x45, 0x4b, 0x42, 0x60, 0x1d, 0xa6, 0x70, 0x54, 0xe0, + 0x61, 0x2d, 0x08, 0x74, 0x64, 0xb2, 0xd6, 0xea, 0x48, 0xf8, 0xdd, 0x21, + 0x77, 0x64, 0x49, 0x1b, 0xe8, 0xc5, 0xa9, 0x52, 0x22, 0x6f, 0x3a, 0x10, + 0x9e, 0x8a, 0x26, 0x72, 0x39, 0xc1, 0x3e, 0xbd, 0x75, 0xe7, 0xd5, 0x76, + 0xb4, 0x62, 0xf6, 0x7a, 0xb5, 0xfe, 0x59, 0xff, 0x74, 0x73, 0xb5, 0x1b, + 0xfa, 0x62, 0xf3, 0x9c, 0x2b, 0x60, 0x55, 0xc5, 0xe5, 0x41, 0x4b, 0x3a, + 0xa2, 0xdf, 0xce, 0xf8, 0x35, 0xe9, 0xe2, 0xd1, 0x2a, 0xc6, 0x1e, 0xc9, + 0x1f, 0x02, 0xb6, 0x6f, 0x11, 0xcc, 0xe5, 0xea, 0x55, 0xd2, 0x90, 0x3e, + 0x22, 0x25, 0x2b, 0xcb, 0xbf, 0xfd, 0x37, 0x9d, 0xdd, 0x7c, 0xae, 0x5c, + 0x8d, 0x21, 0xe1, 0x81, 0xc4, 0xf0, 0x50, 0x27, 0x11, 0x9a, 0x9e, 0x6c, + 0xaf, 0xcd, 0x6d, 0x44, 0xf4, 0x65, 0x53, 0xd4, 0x21, 0x0e, 0x1a, 0xc4, + 0xbf, 0x31, 0x6e, 0x56, 0xe1, 0x62, 0x9a, 0x2b, 0x74, 0x43, 0xe5, 0x09, + 0x2a, 0xd7, 0x16, 0x18, 0x62, 0xbd, 0x27, 0xfc, 0x82, 0x0b, 0x11, 0x8c, + 0x76, 0xf7, 0xce, 0xfb, 0x46, 0x16, 0x4a, 0x86, 0x61, 0xf1, 0x30, 0xef, + 0x1c, 0x3e, 0x4b, 0xc3, 0x70, 0xd4, 0x40, 0x39, 0x21, 0x5f, 0x9f, 0xc5, + 0x3f, 0x65, 0xe0, 0xeb, 0xeb, 0xa0, 0x20, 0xc2, 0xce, 0xef, 0x6e, 0x91, + 0xdd, 0xed, 0x08, 0x3f, 0xf6, 0xc7, 0x92, 0x8f, 0x66, 0xf2, 0xa0, 0x89, + 0x63, 0xc3, 0x26, 0x26, 0x64, 0x9d, 0x9c, 0x9c, 0x4a, 0xd5, 0xab, 0xf0, + 0x17, 0xca, 0xd5, 0x76, 0x2f, 0xba, 0x3f, 0xcc, 0x7a, 0x03, 0xbf, 0xf8, + 0xa5, 0xe2, 0x92, 0x7e, 0x1f, 0x11, 0xce, 0xc4, 0x8e, 0x6e, 0xdd, 0x3e, + 0x66, 0x68, 0xdb, 0x4d, 0xb1, 0x1e, 0x99, 0x72, 0xcc, 0xe1, 0xe2, 0xb8, + 0xf2, 0xe4, 0x0e, 0x1c, 0xc8, 0xf4, 0xdd, 0x15, 0x71, 0x16, 0xc4, 0x8b, + 0x01, 0x33, 0x76, 0x8c, 0xc0, 0x4b, 0xbe, 0x0f, 0x0d, 0x1c, 0xbd, 0xb6, + 0x9b, 0xb1, 0xff, 0xb7, 0x02, 0xe0, 0xe6, 0x60, 0x92, 0x82, 0x63, 0x9e, + 0x5b, 0xd9, 0xa0, 0x48, 0x1a, 0x0b, 0x99, 0x98, 0x3f, 0x7c, 0x78, 0x97, + 0x87, 0xce, 0x37, 0x85, 0x45, 0xa5, 0x52, 0x18, 0x02, 0xd3, 0xc3, 0xb1, + 0x24, 0xe0, 0x30, 0xba, 0x1a, 0x25, 0x32, 0xde, 0xfa, 0x89, 0xb6, 0x73, + 0x41, 0x22, 0xd5, 0x56, 0xef, 0x1e, 0x6b, 0x9b, 0xb5, 0x66, 0x9d, 0x40, + 0x5f, 0x4e, 0x8f, 0x6a, 0xe5, 0x24, 0x5c, 0x59, 0xf6, 0x41, 0x72, 0xaf, + 0xab, 0x3e, 0xbb, 0xc0, 0x9f, 0x8d, 0x76, 0xda, 0x39, 0x0f, 0x3f, 0x6f, + 0x85, 0x76, 0x6f, 0xb5, 0xa5, 0x57, 0x1c, 0x79, 0xd7, 0x5e, 0xd6, 0xa4, + 0x52, 0x1c, 0xfa, 0x5f, 0x51, 0xae, 0x8e, 0x25, 0xea, 0x5a, 0x18, 0x19, + 0xf8, 0x52, 0x87, 0xc3, 0x80, 0xc3, 0x06, 0x90, 0x5b, 0xbc, 0x70, 0x87, + 0xd1, 0x32, 0x74, 0x40, 0x41, 0x39, 0x12, 0x31, 0x59, 0xac, 0x54, 0x20, + 0xfd, 0x05, 0xfd, 0x64, 0x08, 0x74, 0xf4, 0x9c, 0xd2, 0x68, 0xa1, 0x0b, + 0xa3, 0xc7, 0x0d, 0x31, 0x32, 0x58, 0x9e, 0x82, 0xbc, 0x23, 0x47, 0x5b, + 0xcf, 0x1d, 0x39, 0x07, 0x3d, 0xdb, 0xaa, 0xe7, 0x27, 0x8b, 0xd9, 0x3a, + 0xf6, 0x16, 0x9e, 0x12, 0x09, 0xab, 0x1c, 0xd1, 0xa3, 0x20, 0x7f, 0xa3, + 0xfb, 0x69, 0x06, 0x3c, 0x7e, 0x0f, 0x56, 0x7b, 0xe0, 0xdd, 0x22, 0x5c, + 0x50, 0xf8, 0x7f, 0x0f, 0x4e, 0x0d, 0xfb, 0x43, 0xdf, 0x4d, 0xd2, 0x4a, + 0xc6, 0x5c, 0xbb, 0xdd, 0xce, 0xb4, 0x33, 0x38, 0xad, 0x25, 0xce, 0x32, + 0xd4, 0x1e, 0x37, 0x61, 0x58, 0xb2, 0x14, 0x6b, 0xaa, 0x19, 0xb5, 0xee, + 0x4a, 0xc3, 0x59, 0x57, 0x6e, 0x71, 0x07, 0xba, 0x44, 0xe9, 0x60, 0xd1, + 0x3a, 0xa5, 0xc7, 0xd6, 0x2a, 0x18, 0xa3, 0xf4, 0xd7, 0xb8, 0xb1, 0xa7, + 0x0c, 0x8a, 0x5c, 0x0c, 0x88, 0x88, 0xb0, 0x7a, 0x59, 0x0c, 0x62, 0x04, + 0x4f, 0xae, 0xc6, 0xd5, 0xed, 0x77, 0x92, 0xdd, 0xbb, 0x7b, 0x92, 0xc1, + 0x54, 0x0c, 0x7c, 0x38, 0x61, 0xeb, 0x43, 0x24, 0x9b, 0x72, 0xb4, 0x68, + 0x54, 0x77, 0xfb, 0x72, 0x1a, 0xb0, 0x6e, 0xbc, 0x92, 0xd3, 0x13, 0xf9, + 0xba, 0x26, 0xdf, 0xf1, 0xd2, 0x79, 0x18, 0x75, 0xbb, 0x62, 0x46, 0x2e, + 0x15, 0x5d, 0xf4, 0x4a, 0x7c, 0x4f, 0xa8, 0xd3, 0x94, 0x77, 0xae, 0x8e, + 0x71, 0x54, 0x29, 0x74, 0xa5, 0x47, 0x87, 0x6c, 0x47, 0xd7, 0xb8, 0x3d, + 0x38, 0x32, 0xca, 0xaf, 0xcf, 0x82, 0xc9, 0x0e, 0x9f, 0xcb, 0xb8, 0xef, + 0xf7, 0xda, 0xd7, 0xfe, 0x78, 0x7d, 0x58, 0x37, 0x9f, 0xad, 0x4c, 0xe7, + 0x0a, 0x07, 0xc9, 0x6b, 0x05, 0xad, 0xb3, 0x22, 0x12, 0x22, 0xc3, 0xd8, + 0x77, 0x3d, 0x51, 0x07, 0x7d, 0x45, 0xe2, 0x3d, 0xf3, 0x35, 0xf0, 0xb9, + 0x12, 0x97, 0x88, 0x21, 0x1c, 0xfd, 0x83, 0x62, 0x90, 0x6f, 0xb6, 0xd0, + 0xca, 0x66, 0xb9, 0x00, 0xb5, 0xf1, 0xf7, 0x11, 0xc9, 0x60, 0x4a, 0x3f, + 0xad, 0x26, 0x71, 0x1b, 0x45, 0x32, 0x9d, 0x07, 0x12, 0xdb, 0x3a, 0x74, + 0xe0, 0x5a, 0x83, 0x72, 0x31, 0xb0, 0xba, 0x1c, 0xfe, 0x04, 0xec, 0xef, + 0xa1, 0xbd, 0xff, 0xfe, 0xb0, 0x05, 0x8a, 0x11, 0x08, 0x3b, 0x5c, 0x74, + 0xff, 0xce, 0x25, 0x67, 0xaa, 0xf7, 0xee, 0x99, 0xca, 0x67, 0x93, 0x31, + 0xe6, 0x6f, 0xf0, 0x16, 0x7e, 0x81, 0x53, 0x53, 0x6c, 0x24, 0x8b, 0x89, + 0xa1, 0x7f, 0x04, 0x3e, 0xe9, 0x09, 0xbe, 0xec, 0xce, 0x85, 0x40, 0xc7, + 0x33, 0xff, 0x4d, 0xf3, 0xc7, 0xdf, 0x2f, 0x08, 0xd8, 0x33, 0xd5, 0x7a, + 0x1d, 0x2a, 0x63, 0x36, 0xeb, 0xed, 0xae, 0xba, 0x9e, 0xfb, 0xc9, 0x26, + 0x26, 0x26, 0xa8, 0x68, 0x68, 0x8e, 0x56, 0x35, 0x54, 0x0c, 0x51, 0x97, + 0x97, 0x0e, 0xe5, 0x6a, 0x91, 0xa6, 0xe7, 0x25, 0x14, 0xd3, 0xb8, 0xfa, + 0x9f, 0xd9, 0xfa, 0x44, 0x02, 0x2d, 0x3c, 0x5f, 0x00, 0xca, 0x73, 0x31, + 0xd0, 0xf0, 0x64, 0x64, 0x02, 0x14, 0xd7, 0xd7, 0xea, 0xe6, 0xc7, 0xcf, + 0xc8, 0xd6, 0xac, 0xd5, 0x9a, 0x76, 0x95, 0xc1, 0xbe, 0x47, 0x0d, 0x3f, + 0xa8, 0xdf, 0x42, 0x81, 0xfe, 0xd2, 0x0f, 0x8f, 0x9b, 0xdf, 0xda, 0x2c, + 0xa5, 0xcb, 0xc1, 0x38, 0xe4, 0x71, 0x49, 0x73, 0x13, 0x29, 0xe0, 0xff, + 0xcc, 0xfb, 0xc8, 0xae, 0x3b, 0x1a, 0xfa, 0xe9, 0x96, 0xa2, 0x34, 0xef, + 0x74, 0xa7, 0x20, 0x5c, 0x8d, 0xdd, 0xd6, 0xdd, 0xf1, 0x7f, 0x79, 0xb6, + 0x4e, 0x42, 0xb2, 0x4f, 0xe3, 0xa9, 0x18, 0x07, 0x1d, 0x50, 0xa6, 0x52, + 0x92, 0xdd, 0x6c, 0x6c, 0xb3, 0xd6, 0xfa, 0x8d, 0x2b, 0x33, 0xfa, 0xa4, + 0xa6, 0x7b, 0x63, 0x53, 0x2a, 0xbb, 0x70, 0xcb, 0xd0, 0xf0, 0xf6, 0x62, + 0x33, 0x07, 0x08, 0x13, 0x5b, 0xff, 0xa1, 0x08, 0x6f, 0x8a, 0x95, 0xbe, + 0xdc, 0xc6, 0x98, 0xc1, 0x4b, 0xfd, 0x91, 0x49, 0xe7, 0xec, 0x54, 0xcc, + 0xb2, 0x70, 0x47, 0x24, 0x26, 0xd1, 0xdc, 0x12, 0xae, 0x4f, 0x48, 0xab, + 0x71, 0x5e, 0x11, 0xf4, 0x25, 0x27, 0xf5, 0xb5, 0x0b, 0xaa, 0x21, 0x3c, + 0x4a, 0xed, 0x94, 0x2b, 0x7a, 0x95, 0xb7, 0x51, 0x02, 0x2d, 0xf0, 0x87, + 0x56, 0x32, 0x53, 0x30, 0x29, 0x23, 0x47, 0x1a, 0xd6, 0x87, 0x77, 0xec, + 0xcb, 0xc5, 0x61, 0xcf, 0xe6, 0x28, 0x4c, 0x9b, 0xf7, 0x96, 0xc5, 0x39, + 0xf1, 0xf7, 0x0c, 0xc7, 0x9c, 0xe7, 0xb9, 0xef, 0x2d, 0x36, 0xca, 0xb1, + 0x78, 0x24, 0xc8, 0xc6, 0x67, 0xee, 0x18, 0x92, 0xa5, 0xfa, 0x11, 0x10, + 0x88, 0xf9, 0x9b, 0x87, 0xa1, 0xf6, 0xaf, 0xd5, 0x91, 0x02, 0xd2, 0x58, + 0xf6, 0x4b, 0x72, 0xfc, 0x78, 0xab, 0x74, 0x11, 0x22, 0x05, 0xbb, 0xbe, + 0xbb, 0x49, 0xff, 0xa8, 0x71, 0x14, 0xca, 0x3c, 0x45, 0x02, 0x15, 0x3e, + 0x9c, 0x9d, 0xdd, 0x08, 0xe8, 0xda, 0x1b, 0x8b, 0xd3, 0x3d, 0x48, 0x48, + 0x0c, 0x7d, 0x82, 0x74, 0x40, 0xc1, 0x0a, 0x7d, 0xbb, 0x46, 0xcb, 0x62, + 0x01, 0xa4, 0x70, 0x2f, 0x0d, 0x55, 0xe3, 0xe3, 0x0e, 0xd1, 0xc8, 0xf9, + 0x2d, 0x83, 0x94, 0x9a, 0xdd, 0xb2, 0x78, 0x8c, 0xe5, 0xeb, 0x1b, 0x4a, + 0xfe, 0x8d, 0x9e, 0x51, 0x57, 0x75, 0xd6, 0xd2, 0xe2, 0x93, 0xaf, 0x3f, + 0x63, 0x4c, 0x59, 0x8d, 0xdf, 0xd9, 0xe8, 0x35, 0x22, 0x0f, 0x9a, 0xef, + 0xab, 0x5e, 0x41, 0x0e, 0x80, 0x02, 0xa1, 0xd7, 0x8c, 0x32, 0x67, 0xec, + 0xf8, 0xc4, 0x44, 0x51, 0x07, 0x87, 0x4b, 0xcb, 0x1a, 0x0c, 0x7c, 0x7c, + 0x69, 0x79, 0xf9, 0x29, 0x3c, 0x29, 0x42, 0x7c, 0xfc, 0xc8, 0xfe, 0xe2, + 0x5a, 0x8f, 0x63, 0xdc, 0x09, 0x36, 0x69, 0x37, 0x3a, 0xaa, 0x8c, 0xf2, + 0xb9, 0xc3, 0x83, 0x9f, 0xaf, 0x53, 0x46, 0x8a, 0x33, 0xfd, 0xdb, 0xcc, + 0x4b, 0xc3, 0x5a, 0xbf, 0xaf, 0xff, 0xa4, 0x4e, 0xe4, 0x5b, 0x6b, 0xbb, + 0xca, 0xb2, 0x9f, 0x69, 0x89, 0x47, 0x3c, 0x22, 0xc5, 0xe9, 0xb8, 0x9f, + 0x9a, 0x6e, 0x47, 0xde, 0xd1, 0x24, 0x63, 0xf0, 0x1e, 0x15, 0x01, 0x34, + 0x54, 0x1f, 0xce, 0x98, 0xba, 0x66, 0x80, 0xff, 0x8a, 0x66, 0xdb, 0x03, + 0xec, 0x8f, 0x4b, 0x9c, 0x42, 0xae, 0x11, 0x3d, 0x6a, 0x19, 0xdc, 0xd7, + 0x5c, 0x26, 0xe1, 0x28, 0x53, 0x95, 0x9f, 0x4b, 0x28, 0xc0, 0x13, 0xe9, + 0xce, 0xd2, 0x5a, 0x58, 0xc1, 0xc2, 0x99, 0x5c, 0xb6, 0xe4, 0x5c, 0x5c, + 0x29, 0x7b, 0x7b, 0x7b, 0xa0, 0x31, 0x97, 0x9e, 0x00, 0x28, 0x13, 0xc2, + 0xe9, 0x21, 0x33, 0x10, 0x6d, 0x05, 0xaa, 0xc8, 0x2f, 0xe9, 0xc4, 0x5e, + 0x14, 0xd7, 0x07, 0x02, 0xa1, 0x54, 0xeb, 0x6b, 0x2f, 0xc8, 0x54, 0xbf, + 0x1e, 0x3e, 0x67, 0xfb, 0x14, 0x21, 0x5f, 0xa8, 0x93, 0xae, 0xf5, 0xd6, + 0xf8, 0x8c, 0xdc, 0xd8, 0xfb, 0x45, 0xae, 0xdb, 0xf1, 0x51, 0x2c, 0x57, + 0x68, 0xdc, 0x9b, 0xa6, 0x2c, 0xbb, 0x3f, 0x5a, 0xf4, 0xd0, 0x6f, 0xa4, + 0xbf, 0x2f, 0xb6, 0x56, 0xd7, 0x2c, 0xbe, 0xa4, 0xaf, 0x39, 0x91, 0x7d, + 0xe6, 0xb4, 0x9a, 0xdf, 0x9a, 0xc9, 0x13, 0xe5, 0x7f, 0x58, 0x73, 0x2e, + 0x11, 0x38, 0x5f, 0xd4, 0xb1, 0x67, 0x6a, 0x48, 0xe9, 0x64, 0x95, 0x55, + 0x71, 0x32, 0x68, 0x21, 0x80, 0xb7, 0xb6, 0x98, 0x8c, 0x8b, 0xfb, 0xaa, + 0xa1, 0x25, 0xc6, 0x42, 0xed, 0x01, 0x7c, 0x51, 0x0e, 0x86, 0x35, 0x88, + 0xb7, 0xaf, 0xfc, 0x0f, 0x05, 0xb9, 0xef, 0xfd, 0x42, 0x9b, 0x32, 0xdf, + 0xa9, 0x29, 0x0f, 0xe7, 0xf1, 0xbc, 0x10, 0xd4, 0x07, 0xa2, 0xdf, 0x1e, + 0xfd, 0x4d, 0xd9, 0x9a, 0x5d, 0xc0, 0x9e, 0xad, 0xd5, 0xa6, 0xc6, 0xc5, + 0xb2, 0x3c, 0x2f, 0xb2, 0x8a, 0x1d, 0x6e, 0x67, 0x8f, 0xcf, 0x27, 0x35, + 0xa3, 0x23, 0x23, 0xdf, 0x8b, 0x2a, 0x89, 0x7d, 0x2e, 0x85, 0xfb, 0x44, + 0x82, 0x2e, 0x8f, 0xbd, 0xd2, 0xa9, 0xdb, 0x51, 0x3f, 0x89, 0xc3, 0x66, + 0x7b, 0xb5, 0x76, 0x7c, 0xb4, 0xfc, 0xfd, 0x6b, 0x9b, 0xa3, 0x2d, 0xc6, + 0xd2, 0xfc, 0x5d, 0x00, 0x9b, 0x8e, 0x5d, 0x87, 0x41, 0x3a, 0x58, 0x68, + 0xfa, 0xaa, 0x97, 0xc2, 0xf2, 0x15, 0xbc, 0x5c, 0xd7, 0x78, 0xca, 0x15, + 0x30, 0xfd, 0xf1, 0x80, 0x40, 0xd6, 0x2b, 0x7f, 0xda, 0x05, 0x1c, 0xe8, + 0x9a, 0x28, 0x7b, 0x92, 0x6e, 0x33, 0x3a, 0x3b, 0x60, 0xe3, 0xf9, 0xa8, + 0x20, 0xa0, 0x46, 0x3a, 0x79, 0x74, 0xeb, 0x96, 0xcb, 0xe4, 0x3a, 0x6b, + 0xcc, 0xca, 0x4a, 0x4a, 0xb0, 0x5a, 0xf0, 0x91, 0x09, 0xb7, 0x5e, 0x28, + 0x1f, 0x44, 0x60, 0xe9, 0xec, 0x99, 0x48, 0x67, 0xc7, 0xe3, 0xb4, 0xc2, + 0x74, 0xdf, 0xf0, 0xa4, 0xd1, 0xd0, 0x75, 0xfc, 0xe2, 0xfd, 0x70, 0x7a, + 0x30, 0xee, 0x38, 0x9d, 0x03, 0xe2, 0x7d, 0xf9, 0x11, 0x62, 0x98, 0xe1, + 0xd2, 0x5d, 0x24, 0x97, 0x47, 0xab, 0x54, 0x43, 0xc0, 0x4a, 0xa4, 0x65, + 0x69, 0x11, 0x5c, 0xbc, 0xf0, 0x63, 0x29, 0xb1, 0x6e, 0xd5, 0x9a, 0xe6, + 0xe8, 0x39, 0x6f, 0xe3, 0x87, 0x1c, 0xc8, 0x45, 0x0f, 0xca, 0x8b, 0xe2, + 0x92, 0x0a, 0xb1, 0x54, 0xa7, 0x01, 0x1e, 0xcb, 0x68, 0xcc, 0xd1, 0xbf, + 0x85, 0xa0, 0x4c, 0xf9, 0x5b, 0x86, 0x9b, 0xbb, 0x40, 0xb4, 0x1c, 0x48, + 0x53, 0xc0, 0xdc, 0x27, 0x5d, 0x15, 0x02, 0xb1, 0x14, 0x65, 0xbb, 0x05, + 0x1c, 0xd2, 0x36, 0x10, 0x42, 0xb6, 0x9e, 0xf2, 0xa8, 0x2c, 0x2f, 0x2f, + 0xf9, 0x4b, 0x73, 0xe1, 0x0c, 0x9e, 0xe4, 0x5f, 0x2e, 0x21, 0x90, 0x40, + 0x63, 0xbb, 0x48, 0xc1, 0xb2, 0xc8, 0x90, 0xfe, 0x48, 0x1d, 0x6a, 0xb0, + 0x18, 0x0c, 0x7d, 0xbd, 0xa8, 0x9b, 0x3e, 0x7a, 0x9e, 0x9d, 0x4d, 0x7d, + 0x73, 0x4e, 0x86, 0x47, 0x28, 0xa2, 0x81, 0xde, 0xba, 0xb2, 0x31, 0x40, + 0xa9, 0x64, 0x99, 0x87, 0xa2, 0xc4, 0x1b, 0x8a, 0x7f, 0x01, 0xea, 0x03, + 0x27, 0x21, 0x1d, 0x48, 0x8e, 0xdb, 0x4e, 0xc6, 0x6c, 0x90, 0x44, 0x67, + 0xaf, 0xdf, 0xdd, 0x15, 0xf3, 0x47, 0xba, 0x1b, 0x77, 0x08, 0xea, 0xbc, + 0x5c, 0xe9, 0x10, 0x89, 0xff, 0x6b, 0x6f, 0xb1, 0x5c, 0xa6, 0xc5, 0x4c, + 0x6e, 0x2f, 0xdd, 0x68, 0xbb, 0xe3, 0xae, 0x44, 0xf7, 0xa0, 0xfb, 0x3f, + 0x5b, 0x97, 0x8e, 0x23, 0xfc, 0x72, 0xd5, 0xc8, 0xbc, 0x9b, 0x3b, 0x0d, + 0x25, 0x66, 0xf2, 0xd3, 0xe0, 0x2c, 0x17, 0x6c, 0x9f, 0xa7, 0x2f, 0x24, + 0x27, 0xeb, 0x83, 0x0a, 0x2e, 0x7a, 0xd3, 0xed, 0x6c, 0x94, 0x7b, 0x76, + 0x4e, 0x4e, 0x5d, 0xed, 0x9d, 0x77, 0xfb, 0x4e, 0x96, 0x16, 0x6d, 0x4e, + 0x02, 0x7d, 0x73, 0x73, 0x5d, 0x04, 0x88, 0xcf, 0x05, 0xb9, 0xc3, 0xfb, + 0xc3, 0xb7, 0x46, 0xfd, 0x0f, 0x18, 0x2c, 0x16, 0x70, 0xc7, 0xfb, 0x7f, + 0x36, 0x17, 0x6d, 0x0e, 0x22, 0x0c, 0x2f, 0x66, 0x6b, 0x0d, 0xfb, 0x55, + 0xd4, 0xd5, 0xf5, 0x1a, 0xcc, 0x90, 0x26, 0x33, 0x5f, 0xd8, 0xdc, 0x5c, + 0x3b, 0x9e, 0xf0, 0xf5, 0xfd, 0xef, 0x85, 0x5a, 0xd7, 0xb2, 0xd9, 0x18, + 0x6a, 0xf2, 0xdd, 0xf3, 0x67, 0x8a, 0x87, 0x7f, 0xb1, 0x35, 0x38, 0x53, + 0xfd, 0xfa, 0x94, 0x01, 0x5a, 0xfe, 0xda, 0xe2, 0x9d, 0x8e, 0x55, 0x90, + 0xca, 0x01, 0x19, 0x72, 0xf2, 0xd3, 0xc3, 0x78, 0xae, 0xc6, 0x7c, 0x63, + 0xda, 0x7b, 0xae, 0x0d, 0xcc, 0x3f, 0xe7, 0x4b, 0x8b, 0x74, 0xb5, 0x95, + 0x13, 0x1d, 0x1a, 0x47, 0x95, 0xd5, 0x03, 0x9b, 0x95, 0x46, 0x38, 0xb2, + 0xcf, 0x02, 0x5e, 0xb7, 0xbf, 0xce, 0xd7, 0xda, 0xfe, 0x10, 0x0b, 0x64, + 0xa4, 0xfd, 0xd3, 0xac, 0xee, 0x3e, 0xc7, 0x3f, 0x3f, 0x5a, 0xc9, 0xb6, + 0xdf, 0x20, 0x19, 0x1b, 0x0f, 0x3a, 0xa7, 0xa0, 0x59, 0x22, 0xba, 0xf4, + 0xdc, 0x8e, 0x84, 0x56, 0xee, 0x23, 0xe8, 0x52, 0x56, 0xfb, 0xa4, 0xc8, + 0xc9, 0x54, 0xf7, 0x9c, 0x2e, 0x49, 0x4c, 0xbc, 0x01, 0x17, 0x10, 0x17, + 0xe7, 0x96, 0x22, 0xce, 0x62, 0x37, 0x5f, 0xc3, 0xb2, 0x5c, 0x2f, 0x3e, + 0xc2, 0x67, 0x67, 0xd9, 0x1f, 0xf5, 0x76, 0x41, 0x57, 0x00, 0x4f, 0x37, + 0xcb, 0x91, 0xdc, 0x01, 0xd3, 0x35, 0xd5, 0x9c, 0xe5, 0x3d, 0x8b, 0x94, + 0x78, 0x39, 0x43, 0x4a, 0xde, 0x27, 0x82, 0xf7, 0x3a, 0x43, 0xb4, 0xe6, + 0x62, 0xdb, 0x10, 0xa0, 0x63, 0xed, 0x50, 0xf7, 0x2b, 0x83, 0x5a, 0xc3, + 0xcd, 0x8f, 0xff, 0x43, 0xe7, 0x50, 0x77, 0xe4, 0xfc, 0x3a, 0x76, 0x90, + 0x85, 0x84, 0x41, 0x4c, 0x32, 0xe6, 0x82, 0xa8, 0x70, 0x58, 0xfa, 0x28, + 0x42, 0x35, 0x29, 0x17, 0x17, 0x34, 0xc1, 0xf3, 0x89, 0xab, 0xa5, 0x38, + 0x59, 0xa8, 0xfc, 0xe4, 0x65, 0x8e, 0x29, 0x1f, 0x27, 0x33, 0x0a, 0xeb, + 0xb9, 0xbc, 0x7c, 0x4f, 0xb0, 0xc5, 0x6b, 0xea, 0x25, 0xbf, 0x52, 0xbb, + 0x2c, 0x30, 0xcb, 0x4d, 0x18, 0xb9, 0x43, 0x88, 0x33, 0xd9, 0x10, 0xf7, + 0x70, 0x91, 0xf6, 0xf9, 0x4b, 0xec, 0xef, 0x3f, 0x84, 0x1f, 0x58, 0x9b, + 0x53, 0x3a, 0x09, 0xdd, 0x55, 0x22, 0xe3, 0x5f, 0xd9, 0x39, 0xba, 0x09, + 0xbe, 0x2f, 0x5c, 0x29, 0x4b, 0xd3, 0xbd, 0x3f, 0x11, 0xf3, 0x50, 0x51, + 0x16, 0x2e, 0x53, 0x2c, 0x2f, 0x77, 0x7b, 0x18, 0x19, 0xe9, 0x93, 0x51, + 0xed, 0x8b, 0x47, 0x1d, 0x82, 0xa1, 0xd5, 0xaa, 0x42, 0x2d, 0x89, 0x91, + 0x13, 0xe6, 0x12, 0x3b, 0xac, 0xee, 0x3d, 0x6c, 0x32, 0xac, 0xda, 0x0a, + 0xb1, 0x23, 0x9b, 0xa0, 0x5d, 0xf7, 0x3d, 0x2d, 0xd6, 0x6d, 0x49, 0x01, + 0xc2, 0xc2, 0xc2, 0xf3, 0xa5, 0x4a, 0x41, 0x49, 0x49, 0x98, 0xe8, 0x18, + 0x18, 0x47, 0xff, 0xea, 0x28, 0xd4, 0xca, 0x55, 0xb7, 0xef, 0x57, 0xed, + 0x87, 0xed, 0x3e, 0x4a, 0xe2, 0x24, 0xa5, 0xa5, 0xe1, 0xff, 0x0b, 0xd9, + 0x72, 0xbd, 0x69, 0x5a, 0xf1, 0xc0, 0x91, 0x0d, 0xe1, 0x85, 0x5b, 0x8d, + 0x02, 0x32, 0x5c, 0x5b, 0xca, 0xdf, 0xbf, 0x45, 0xa9, 0xa7, 0x9e, 0xcc, + 0xab, 0xae, 0xb1, 0xa1, 0x75, 0x38, 0x6a, 0x55, 0xe7, 0xdf, 0xfe, 0x08, + 0x10, 0x0a, 0x7a, 0xfd, 0xb1, 0x4a, 0x66, 0xd0, 0x5c, 0xb1, 0xcc, 0xd3, + 0x64, 0xf2, 0xaa, 0x05, 0xfd, 0x9e, 0x2c, 0xde, 0xbe, 0x3b, 0x2e, 0x2c, + 0x2a, 0xda, 0x76, 0xc9, 0x06, 0x0e, 0x78, 0xc9, 0xca, 0x22, 0x46, 0xe1, + 0xb0, 0x7c, 0xdd, 0xca, 0x0b, 0x23, 0x36, 0x9a, 0x2b, 0x94, 0xea, 0xb0, + 0x6d, 0x3e, 0x76, 0x75, 0x76, 0x4a, 0xb1, 0xeb, 0x48, 0x9a, 0x76, 0xd6, + 0x4a, 0x8d, 0xe3, 0xfc, 0x23, 0x10, 0x8a, 0x4a, 0x36, 0xbe, 0x91, 0xc1, + 0x01, 0x3d, 0x9e, 0xa2, 0x43, 0x0b, 0x87, 0x78, 0x06, 0x26, 0xf8, 0x78, + 0x7a, 0xbd, 0x99, 0x9e, 0x7e, 0x5a, 0x53, 0x10, 0x7c, 0xfa, 0x5c, 0x02, + 0xa2, 0x03, 0x6e, 0xf4, 0xfb, 0xeb, 0x1d, 0x83, 0x5b, 0x4f, 0xad, 0xc5, + 0x11, 0xd0, 0x9e, 0xfa, 0x12, 0x92, 0xfd, 0x21, 0x82, 0x99, 0xa2, 0xae, + 0x4d, 0xeb, 0xb6, 0xe6, 0xa4, 0xc7, 0x0e, 0x78, 0xbe, 0x67, 0xd2, 0x06, + 0x86, 0x05, 0x76, 0xfa, 0x6c, 0xbc, 0xea, 0x25, 0xbb, 0xaf, 0x39, 0x86, + 0x05, 0xf0, 0x63, 0x16, 0xf9, 0xb8, 0x42, 0x90, 0x38, 0xc0, 0x69, 0x5f, + 0xfb, 0x1d, 0x3e, 0xad, 0xda, 0x79, 0x04, 0x64, 0xd6, 0x2b, 0xa2, 0x64, + 0x08, 0xe2, 0xe8, 0x13, 0xf5, 0xd5, 0xf9, 0xbd, 0x6e, 0xe7, 0x1e, 0xaf, + 0xfb, 0x1e, 0x2b, 0xf4, 0x8e, 0xdb, 0xb1, 0x2e, 0xaf, 0xac, 0x8c, 0x0e, + 0x99, 0xe8, 0xc8, 0x05, 0xc0, 0x25, 0x53, 0xfc, 0x16, 0xe1, 0x87, 0xd9, + 0x81, 0x04, 0x5c, 0x9a, 0x31, 0xcd, 0x64, 0x5c, 0x79, 0xbe, 0x66, 0x86, + 0x65, 0xde, 0x3f, 0xe3, 0x3b, 0x43, 0x68, 0x2f, 0xe8, 0x2e, 0xa8, 0xb6, + 0xb5, 0xb5, 0xb5, 0x06, 0xed, 0xdb, 0x9f, 0x24, 0xbe, 0x49, 0x11, 0x18, + 0x34, 0xf5, 0x86, 0xfd, 0x5f, 0xbd, 0xee, 0x8e, 0xbf, 0xd0, 0x4a, 0x7a, + 0xfa, 0x84, 0x41, 0xbe, 0x16, 0x30, 0xdc, 0x40, 0x44, 0x46, 0x3e, 0x34, + 0xf1, 0x6c, 0xd9, 0x36, 0x32, 0xe6, 0xaa, 0xa5, 0xbe, 0x2e, 0x52, 0x57, + 0xa0, 0x57, 0x41, 0xe9, 0x5f, 0x17, 0xb7, 0x3b, 0x1c, 0x9b, 0xd3, 0x64, + 0x1c, 0xa7, 0x7c, 0x7d, 0xd9, 0x8f, 0x82, 0xe3, 0x01, 0xa5, 0xac, 0x44, + 0x2e, 0xa1, 0xf5, 0xcd, 0xeb, 0x5b, 0xae, 0xdd, 0xfa, 0x1f, 0xb2, 0x80, + 0x6f, 0x36, 0xa7, 0x98, 0xa7, 0xa6, 0xdf, 0x7f, 0x07, 0x86, 0x24, 0xfc, + 0xae, 0xf1, 0x69, 0x3a, 0xa0, 0x79, 0x5e, 0xfe, 0x23, 0x25, 0x2c, 0x3c, + 0xcd, 0x77, 0xd1, 0x89, 0xf1, 0x23, 0x00, 0xb7, 0xac, 0x3c, 0x99, 0x31, + 0xdf, 0x80, 0xb6, 0xd9, 0x0e, 0x0b, 0x7d, 0xbe, 0x00, 0xc1, 0x4f, 0x67, + 0x38, 0xb4, 0xaa, 0x83, 0x69, 0xb2, 0x51, 0xb1, 0x32, 0x4e, 0x79, 0xae, + 0xd0, 0x90, 0x09, 0x90, 0xc2, 0x62, 0xa0, 0xbf, 0x77, 0x87, 0x7f, 0xb2, + 0x58, 0x05, 0x4d, 0x4a, 0x59, 0xd8, 0x30, 0xcc, 0x8a, 0xac, 0xbf, 0x82, + 0x4a, 0x89, 0xaa, 0x2f, 0xa6, 0xa8, 0xa8, 0x82, 0x48, 0x01, 0xac, 0x1a, + 0x6e, 0xb0, 0x76, 0x6c, 0x0b, 0x67, 0x4d, 0xf1, 0x63, 0x6f, 0x96, 0xc5, + 0xa6, 0xaa, 0x55, 0x74, 0xd2, 0xec, 0x74, 0x82, 0x56, 0x88, 0x79, 0x04, + 0x64, 0x4e, 0x4a, 0x40, 0x0c, 0x9b, 0x94, 0xa4, 0x64, 0xae, 0x7a, 0x95, + 0xf6, 0x7f, 0x4f, 0xac, 0x2b, 0xaa, 0xdb, 0xc7, 0xba, 0xc3, 0x72, 0xb7, + 0x3e, 0xfe, 0xfc, 0x41, 0xcc, 0xcf, 0xcf, 0x2f, 0x2c, 0x1e, 0x94, 0x9d, + 0xbf, 0x4b, 0x1a, 0x66, 0x7e, 0xd2, 0x14, 0xfd, 0x05, 0x44, 0x5e, 0xfd, + 0x7a, 0x49, 0x2d, 0xef, 0xeb, 0x0e, 0xeb, 0x8a, 0x51, 0xae, 0xcd, 0xb0, + 0xcf, 0x88, 0x99, 0x63, 0x66, 0x89, 0x8f, 0xc4, 0xcb, 0xef, 0x32, 0xd6, + 0xbd, 0x50, 0xad, 0xa7, 0x13, 0x43, 0xc8, 0xa3, 0xfc, 0x3d, 0x4b, 0x5e, + 0x44, 0x54, 0x34, 0x5d, 0xd0, 0xb7, 0xfd, 0x2f, 0xab, 0x61, 0x20, 0x60, + 0xb5, 0x39, 0x2e, 0xee, 0xf9, 0xce, 0xd9, 0x19, 0x25, 0x5d, 0x99, 0xb8, + 0x29, 0x5c, 0x5f, 0x77, 0xa6, 0x71, 0xcc, 0xd6, 0xf5, 0xad, 0x67, 0x27, + 0xd4, 0x75, 0xe7, 0xe9, 0x19, 0x88, 0x33, 0xe6, 0xe3, 0x45, 0x89, 0x30, + 0x28, 0xe5, 0x92, 0x3c, 0x44, 0x44, 0x22, 0xbe, 0x84, 0xa4, 0xca, 0xe9, + 0x12, 0x64, 0x18, 0xec, 0xa6, 0xb4, 0xd0, 0x39, 0x8c, 0xd4, 0x9f, 0x05, + 0xbf, 0x11, 0x26, 0x5c, 0xa3, 0x42, 0x5b, 0x3e, 0x8d, 0xdd, 0x77, 0x2f, + 0x44, 0xb7, 0xfe, 0x90, 0x9a, 0x72, 0x0f, 0x33, 0xef, 0x93, 0x54, 0x4d, + 0xa2, 0xf9, 0xa9, 0xb9, 0x9b, 0xdb, 0x76, 0x75, 0x72, 0x65, 0xd2, 0x1f, + 0x7a, 0x11, 0xb6, 0x33, 0x64, 0xe7, 0x77, 0x7a, 0x0e, 0x3a, 0x7e, 0x0f, + 0xeb, 0x35, 0x7e, 0xff, 0xb6, 0x6f, 0x68, 0xfc, 0x7b, 0x67, 0xbe, 0xaa, + 0x2f, 0xc2, 0x6d, 0x3b, 0x1c, 0x5c, 0x0d, 0x0f, 0x69, 0xbd, 0xcc, 0xfe, + 0x18, 0xca, 0x22, 0x14, 0x85, 0xac, 0xf4, 0xc3, 0x1b, 0xf2, 0x4f, 0x51, + 0xc6, 0xfc, 0x54, 0x58, 0x2b, 0x52, 0xb1, 0xbd, 0x44, 0x80, 0x8c, 0xdf, + 0x07, 0x86, 0xb8, 0xe6, 0xcd, 0x31, 0x76, 0x24, 0x8c, 0xf2, 0x98, 0x08, + 0x3b, 0xaf, 0x3e, 0x47, 0x95, 0x36, 0x52, 0x49, 0x03, 0x5e, 0x71, 0x1b, + 0x51, 0x40, 0x16, 0x5f, 0xce, 0xc3, 0x8f, 0xa1, 0x35, 0x68, 0x18, 0x6a, + 0x7b, 0x96, 0xef, 0x34, 0x2c, 0xf6, 0x89, 0xe2, 0xb7, 0x09, 0xc2, 0x6c, + 0xca, 0xc5, 0x13, 0x30, 0x4c, 0x9e, 0xcb, 0xc7, 0x72, 0x73, 0x66, 0x72, + 0x4d, 0x5a, 0xb4, 0x5c, 0xf9, 0x2f, 0xc5, 0xcb, 0xf0, 0x5d, 0xf8, 0x12, + 0xc7, 0x4c, 0x76, 0x0f, 0x19, 0xe3, 0xc5, 0x6f, 0x50, 0xe6, 0xd7, 0x98, + 0x45, 0xf0, 0x91, 0xe8, 0xfc, 0xae, 0xab, 0x53, 0xbe, 0x7d, 0x1a, 0xdf, + 0xa7, 0x03, 0x43, 0x8c, 0x1c, 0xd6, 0xe5, 0x12, 0x49, 0xc3, 0x0a, 0xb6, + 0x2a, 0x00, 0x9e, 0x37, 0xa6, 0x83, 0x51, 0x3d, 0xde, 0x64, 0x2f, 0xdf, + 0x26, 0x27, 0xc8, 0x72, 0x2f, 0x27, 0xd7, 0x06, 0x17, 0xa5, 0x79, 0x10, + 0x06, 0x3f, 0x41, 0x1a, 0x12, 0xaf, 0x81, 0xce, 0x63, 0x89, 0x0e, 0xe4, + 0xd2, 0xe0, 0xc7, 0x89, 0xd9, 0xf6, 0x32, 0x3d, 0x65, 0x60, 0x91, 0xe0, + 0xea, 0x1b, 0xe7, 0x45, 0xea, 0x72, 0x85, 0xab, 0x8e, 0x43, 0x74, 0x3d, + 0x5b, 0x26, 0xc9, 0xbd, 0xca, 0x6a, 0x4f, 0xb5, 0x6a, 0x88, 0x04, 0x52, + 0x46, 0x4b, 0x36, 0xdf, 0x2d, 0xb4, 0x5c, 0x11, 0x06, 0x03, 0xd5, 0x35, + 0x16, 0x59, 0x08, 0x9b, 0x0a, 0x68, 0x93, 0x2b, 0x74, 0xc4, 0x58, 0xb0, + 0xce, 0xd5, 0x67, 0x60, 0x54, 0x8a, 0x3f, 0x28, 0x99, 0x2a, 0xec, 0xf2, + 0xbe, 0xe7, 0x96, 0xd1, 0x93, 0xf5, 0x4e, 0x89, 0xa8, 0x0b, 0xbc, 0x59, + 0x85, 0x15, 0x14, 0x8b, 0xdf, 0xca, 0x44, 0xcc, 0xcc, 0xc8, 0xd3, 0x4a, + 0x47, 0x93, 0xc1, 0xc4, 0x12, 0x62, 0xa8, 0x98, 0xb3, 0xd4, 0x57, 0xb6, + 0x87, 0x18, 0x5d, 0xa2, 0x17, 0xf9, 0x3b, 0x23, 0x90, 0xf6, 0xb6, 0x42, + 0x26, 0x98, 0xfa, 0xdf, 0x31, 0xa7, 0xa7, 0x9e, 0xb8, 0x43, 0x29, 0x88, + 0x0e, 0x3e, 0x2f, 0xb3, 0x8d, 0x4f, 0xf5, 0x0e, 0x0a, 0x36, 0xbd, 0x7d, + 0x4f, 0xb9, 0xa4, 0x40, 0xae, 0x80, 0xf2, 0x72, 0x11, 0xe9, 0x37, 0x2b, + 0x88, 0x8c, 0x7f, 0x51, 0x5c, 0x18, 0x83, 0x35, 0xfc, 0x30, 0xfa, 0xc8, + 0xc3, 0xbc, 0x3a, 0xeb, 0x2c, 0x4a, 0x51, 0x6b, 0x71, 0xf8, 0x8d, 0xbd, + 0x3e, 0x60, 0xf9, 0x10, 0xdd, 0xfb, 0x05, 0x06, 0x06, 0x9a, 0xad, 0x8f, + 0xdc, 0x8b, 0x2e, 0xc0, 0x90, 0xb0, 0x67, 0xdb, 0xc3, 0x4a, 0x10, 0x76, + 0xe3, 0xd9, 0x84, 0xb7, 0x57, 0xf8, 0xbf, 0x63, 0xa0, 0x9e, 0x4d, 0x32, + 0x28, 0xe7, 0x4b, 0x11, 0x7f, 0xa8, 0xab, 0x63, 0xff, 0xca, 0x2f, 0x87, + 0xe3, 0x64, 0x46, 0x28, 0xc0, 0x39, 0x64, 0xf9, 0xb9, 0x84, 0x23, 0x23, + 0x0e, 0x27, 0x37, 0xe6, 0x6a, 0xba, 0x14, 0x41, 0x87, 0x43, 0x76, 0x3c, + 0x5f, 0x69, 0x74, 0x66, 0xd5, 0x21, 0x66, 0x57, 0x63, 0x87, 0x7b, 0x2a, + 0xe0, 0x90, 0x24, 0xfa, 0x33, 0x36, 0xde, 0x8d, 0x3f, 0x8a, 0x89, 0xef, + 0xbe, 0xfe, 0xc3, 0x81, 0x9e, 0xef, 0x81, 0xd0, 0xe4, 0x22, 0xaf, 0xc0, + 0x43, 0x80, 0x57, 0x3e, 0x2c, 0x2b, 0x8e, 0xd4, 0xbc, 0xac, 0xf2, 0x92, + 0x3e, 0xf8, 0xd7, 0xcb, 0xd4, 0x5b, 0xe5, 0x66, 0x7d, 0xc3, 0x8c, 0xf3, + 0xfe, 0xe4, 0xe7, 0x8b, 0xec, 0x39, 0xe6, 0x8f, 0x80, 0xa5, 0xc7, 0xdc, + 0x5c, 0x89, 0xbd, 0x7d, 0xa2, 0x08, 0xee, 0xc9, 0x82, 0x0e, 0x88, 0xa3, + 0x6b, 0x72, 0x16, 0xaa, 0xbd, 0x2f, 0x37, 0xf8, 0x64, 0x51, 0x3e, 0xa6, + 0xd6, 0x82, 0x02, 0x81, 0x52, 0xd7, 0x45, 0xe3, 0x1a, 0xfb, 0x2c, 0xb1, + 0x8b, 0xea, 0x33, 0x4b, 0xc5, 0x8e, 0xaf, 0xf2, 0x61, 0x40, 0x55, 0xac, + 0x10, 0x59, 0x29, 0xbe, 0xa9, 0x68, 0x60, 0x0e, 0x93, 0xd8, 0xfb, 0x96, + 0x77, 0xd1, 0xc4, 0x5b, 0xb7, 0x32, 0xcb, 0xc7, 0x0f, 0x0d, 0xd6, 0x54, + 0x6b, 0x8a, 0x1e, 0x3b, 0x32, 0x11, 0x6c, 0xef, 0x4e, 0x1e, 0xdb, 0x1c, + 0xcf, 0x37, 0xe3, 0xab, 0x63, 0x90, 0x2f, 0xe3, 0x81, 0x9e, 0x09, 0x04, + 0xbe, 0xf7, 0xf6, 0xda, 0x21, 0x5c, 0xf9, 0x9f, 0xf9, 0xf9, 0xef, 0x1f, + 0x0d, 0x1a, 0xd5, 0xb2, 0x0e, 0x0e, 0xac, 0x9e, 0x9e, 0x9e, 0x36, 0x9b, + 0x5d, 0x44, 0x74, 0x54, 0xc3, 0x31, 0x44, 0xc6, 0xb3, 0x91, 0x48, 0x8c, + 0xba, 0xe7, 0x15, 0xd9, 0x02, 0x85, 0x9b, 0xeb, 0x6b, 0x6e, 0xa2, 0x0c, + 0xa7, 0x5c, 0x55, 0xaa, 0x71, 0x40, 0xab, 0x41, 0xdc, 0xb1, 0x9f, 0xa1, + 0xce, 0x8c, 0x14, 0x20, 0x9c, 0x17, 0x7a, 0xb7, 0x12, 0xc3, 0xbd, 0x14, + 0xaf, 0xa7, 0x54, 0x38, 0x12, 0xae, 0xef, 0xad, 0x50, 0xcc, 0x4b, 0xbf, + 0xad, 0x94, 0xcd, 0x4e, 0xa1, 0x34, 0xbf, 0xdd, 0xe8, 0x05, 0x12, 0xba, + 0x45, 0xf9, 0xe2, 0x80, 0xf2, 0x2d, 0xe8, 0x91, 0xf8, 0xf8, 0x27, 0xe6, + 0xcb, 0x54, 0x8e, 0x16, 0x2a, 0x96, 0xe0, 0xd4, 0xf4, 0x14, 0xd4, 0xb6, + 0x5d, 0x21, 0x83, 0xeb, 0xfa, 0xbf, 0xdd, 0x9d, 0xae, 0xb6, 0xa0, 0xc8, + 0xa9, 0x68, 0x5d, 0xdf, 0x69, 0x06, 0x66, 0xdd, 0xcb, 0xc8, 0xbb, 0x91, + 0x6d, 0x75, 0xf5, 0xda, 0xd7, 0x1a, 0x5a, 0xf7, 0x36, 0x66, 0x9c, 0x7d, + 0xb9, 0xec, 0xd2, 0x5e, 0xd9, 0x37, 0xa0, 0xbe, 0x77, 0x3c, 0x38, 0xb0, + 0x93, 0x18, 0xfb, 0x03, 0xe4, 0xd2, 0xd9, 0x1f, 0x06, 0xf0, 0x0a, 0x3c, + 0x06, 0x1a, 0xf4, 0x32, 0xea, 0x12, 0xda, 0xac, 0xd7, 0xe1, 0x90, 0x0a, + 0xbd, 0x7d, 0xa3, 0x41, 0xbd, 0x57, 0xdd, 0x93, 0x93, 0x94, 0xfc, 0x75, + 0xb1, 0x01, 0xfc, 0x6f, 0x35, 0x72, 0xe6, 0xa4, 0x7c, 0xea, 0x07, 0xe2, + 0x9d, 0x82, 0xad, 0x9e, 0x74, 0x5a, 0x71, 0xa1, 0x91, 0x4d, 0x9b, 0x88, + 0xad, 0x06, 0x73, 0x14, 0xda, 0x06, 0x5b, 0x93, 0xe1, 0xea, 0x89, 0xc0, + 0x25, 0xd9, 0x3f, 0xc9, 0xf1, 0x91, 0x04, 0xd3, 0xb6, 0xea, 0xe7, 0xc1, + 0xd0, 0xa4, 0x84, 0xbd, 0xb7, 0xb0, 0xe9, 0x59, 0xa7, 0xe7, 0x0d, 0x39, + 0x08, 0xd4, 0x1d, 0x1d, 0x1d, 0x15, 0x9a, 0x9a, 0x5c, 0x1d, 0xbf, 0x80, + 0x98, 0x81, 0x08, 0x8c, 0x9f, 0xa5, 0xee, 0x6a, 0x68, 0x6f, 0x6d, 0xbc, + 0x3c, 0xa3, 0x0e, 0x8b, 0x50, 0xc9, 0xc8, 0xe4, 0x8c, 0x01, 0x8e, 0xab, + 0xa2, 0xfd, 0xac, 0x95, 0x73, 0x7d, 0x38, 0xd1, 0x84, 0x3c, 0x6d, 0x76, + 0xaa, 0x6a, 0x02, 0x35, 0xce, 0x44, 0xc8, 0x30, 0x89, 0xd4, 0x72, 0x46, + 0x2b, 0xee, 0x51, 0xd2, 0x83, 0x37, 0xf3, 0x1d, 0xf9, 0xda, 0x79, 0xf7, + 0x96, 0x4b, 0x9a, 0x62, 0x2c, 0xa6, 0x16, 0x73, 0x26, 0x64, 0x3c, 0x3c, + 0x0c, 0x2a, 0x6a, 0x4d, 0xe7, 0x72, 0x80, 0xcd, 0x09, 0x1d, 0xab, 0xba, + 0x4b, 0xf2, 0x41, 0x70, 0xc5, 0xc7, 0x5a, 0x8c, 0x21, 0xee, 0x44, 0x16, + 0xdf, 0x43, 0x27, 0x7f, 0x60, 0xa6, 0x5d, 0x3b, 0x1c, 0x1d, 0x15, 0xf4, + 0x4f, 0xd9, 0xd6, 0x3d, 0x32, 0x8d, 0xa0, 0x95, 0x10, 0xf7, 0x3a, 0xcf, + 0x20, 0x48, 0xed, 0xb0, 0x55, 0xea, 0xb8, 0x0b, 0x70, 0xe9, 0xc8, 0x20, + 0xee, 0xd6, 0xe4, 0x60, 0x52, 0xf9, 0xdc, 0x1c, 0xa2, 0x94, 0xc4, 0x83, + 0xaf, 0xb4, 0xeb, 0xbc, 0xe0, 0x44, 0xd5, 0x30, 0x21, 0x04, 0x4d, 0xc6, + 0xc6, 0x96, 0x30, 0x95, 0xc5, 0x87, 0xf3, 0xd4, 0x59, 0x59, 0xc9, 0x68, + 0xb5, 0x50, 0x1e, 0x82, 0x1a, 0x38, 0x96, 0xec, 0x8b, 0xa0, 0xa6, 0xfe, + 0x5a, 0xb7, 0x9f, 0x10, 0xa6, 0xab, 0x84, 0x66, 0x18, 0xc2, 0xe4, 0xf5, + 0x05, 0x72, 0xd1, 0x6c, 0xf2, 0x06, 0xe9, 0x17, 0x7d, 0xdd, 0x0b, 0xb0, + 0x5b, 0x43, 0x1b, 0xd3, 0x52, 0x20, 0x0e, 0xa6, 0xfa, 0x54, 0xd6, 0xeb, + 0x6b, 0x7e, 0x97, 0x93, 0xc5, 0xb3, 0xbf, 0x53, 0x70, 0x08, 0x08, 0xc6, + 0x03, 0xe1, 0x68, 0x03, 0xc3, 0xc3, 0xd0, 0x97, 0x10, 0x65, 0xda, 0x5d, + 0x40, 0x0a, 0x6e, 0x6d, 0x84, 0x2e, 0x2b, 0xb6, 0xdc, 0x99, 0xc8, 0x69, + 0x8a, 0x9f, 0xc3, 0x40, 0x86, 0x9b, 0xc5, 0x9e, 0x95, 0xa1, 0xab, 0xaf, + 0x27, 0x4e, 0xe9, 0xed, 0x8d, 0xb6, 0x36, 0x35, 0x03, 0xa7, 0xbd, 0xe9, + 0x7b, 0x6d, 0x8e, 0xb1, 0x68, 0x88, 0xc4, 0x29, 0xd7, 0x83, 0x7b, 0x94, + 0xdc, 0x4e, 0x79, 0x46, 0x1e, 0xaf, 0x76, 0x21, 0x92, 0xa6, 0x63, 0x0f, + 0x4c, 0xde, 0xdb, 0x33, 0xa1, 0x56, 0xc8, 0x96, 0x5d, 0x6d, 0xb6, 0x8d, + 0x57, 0x4e, 0xc8, 0xda, 0xed, 0x2a, 0x54, 0x5f, 0x67, 0x95, 0x25, 0xef, + 0x17, 0x45, 0x78, 0x4b, 0x50, 0x5e, 0x85, 0x14, 0x93, 0x84, 0xdd, 0x79, + 0x28, 0x51, 0xa4, 0xf2, 0xf9, 0x3a, 0x96, 0x75, 0x4b, 0xf8, 0xfa, 0x0b, + 0xb2, 0x8e, 0x3a, 0xf0, 0x33, 0xea, 0x9e, 0xcb, 0x7f, 0x6b, 0x9e, 0x7d, + 0x72, 0x82, 0x02, 0xa7, 0x24, 0x8f, 0xfd, 0x45, 0x61, 0x85, 0x25, 0x25, + 0x61, 0xff, 0x53, 0xe8, 0xad, 0x9e, 0xdf, 0x9d, 0x4f, 0x07, 0x99, 0xa5, + 0x91, 0x22, 0x16, 0xb5, 0x37, 0x09, 0x5d, 0x0c, 0x35, 0x1a, 0x25, 0x56, + 0x83, 0x7d, 0x12, 0x6b, 0xab, 0x53, 0x90, 0x91, 0xc2, 0x2c, 0x8b, 0x91, + 0xbe, 0xee, 0x4b, 0x3e, 0x0c, 0xeb, 0xc7, 0x87, 0xa9, 0xcc, 0x31, 0x99, + 0x61, 0x27, 0x62, 0xe1, 0x88, 0xe4, 0xc2, 0xf6, 0x9e, 0xe6, 0xa4, 0x34, + 0x78, 0xe7, 0xd8, 0x5b, 0x9d, 0x85, 0x12, 0xd8, 0xe5, 0xfe, 0x8c, 0x25, + 0xfe, 0x6f, 0xe9, 0x4c, 0x6b, 0xeb, 0x77, 0x7c, 0x2e, 0x1b, 0x49, 0xa0, + 0xd7, 0x9d, 0xc7, 0x65, 0x2f, 0x8c, 0x56, 0xbd, 0xc9, 0xa8, 0x25, 0xdc, + 0x66, 0x9b, 0xb3, 0x79, 0x7d, 0x7d, 0xbd, 0x66, 0xed, 0x2f, 0x28, 0xf4, + 0x88, 0x60, 0x19, 0xeb, 0x59, 0xb9, 0xc4, 0x01, 0x2f, 0xb8, 0x90, 0x9c, + 0x7f, 0xd9, 0x69, 0x8f, 0xe9, 0xe2, 0xb7, 0x73, 0x88, 0x99, 0x55, 0x82, + 0x91, 0xe8, 0x77, 0xc2, 0xd9, 0xc3, 0x14, 0xbc, 0x82, 0xac, 0xc1, 0x14, + 0x65, 0x47, 0x29, 0xa6, 0xdd, 0x00, 0x58, 0x32, 0x49, 0x48, 0x69, 0x1d, + 0xa2, 0x74, 0x9b, 0x8d, 0xc7, 0x00, 0xf9, 0xf9, 0x9e, 0x31, 0x82, 0x77, + 0x28, 0xf6, 0x03, 0xbf, 0x20, 0xa4, 0x09, 0x5d, 0xf0, 0xc9, 0x3e, 0x38, + 0x1e, 0x0f, 0x0f, 0x2b, 0x75, 0x48, 0x53, 0x0e, 0xc7, 0x12, 0xed, 0x37, + 0x4c, 0x0b, 0x16, 0xe8, 0x75, 0xb8, 0xad, 0xf9, 0x3e, 0xac, 0xbf, 0x3f, + 0x9b, 0x86, 0xd4, 0xd9, 0x2c, 0xb3, 0x7d, 0x80, 0x61, 0xce, 0x0f, 0x15, + 0xe9, 0x37, 0xa6, 0x89, 0x5a, 0x1e, 0xcd, 0x17, 0x28, 0xd3, 0xd4, 0xde, + 0x5f, 0x6f, 0x7e, 0xcf, 0xbd, 0xe9, 0x33, 0x2e, 0xdc, 0xaa, 0xf8, 0xd1, + 0x78, 0xec, 0x5a, 0xab, 0x57, 0xd0, 0x76, 0x7d, 0xd8, 0x60, 0xb6, 0x31, + 0xa3, 0x2c, 0x7a, 0x65, 0x38, 0xb7, 0xec, 0x55, 0x70, 0xd9, 0x14, 0x5c, + 0xc4, 0x39, 0xae, 0xad, 0x9e, 0x7d, 0xed, 0x7a, 0xdc, 0x64, 0x99, 0xaf, + 0x50, 0xcb, 0x7a, 0xca, 0xe9, 0x12, 0xb1, 0xf2, 0xc2, 0xe6, 0x92, 0xa3, + 0xe2, 0xa0, 0x84, 0x8d, 0x1d, 0x8e, 0xb2, 0x94, 0x46, 0x04, 0x51, 0x8e, + 0x58, 0x9a, 0x1f, 0xa6, 0x4d, 0x3f, 0x9a, 0x65, 0x4f, 0xe7, 0xd3, 0x50, + 0x53, 0x5f, 0x0f, 0xe2, 0x18, 0xc4, 0xa5, 0xa5, 0xc1, 0xf6, 0x6c, 0xf7, + 0x87, 0x95, 0x69, 0xd6, 0x91, 0x47, 0x92, 0xe3, 0x72, 0x9f, 0x3a, 0xae, + 0x34, 0x59, 0xe9, 0xd2, 0x2f, 0x0a, 0xfd, 0x6d, 0x6d, 0x1b, 0x6c, 0x46, + 0xec, 0x5c, 0x82, 0xd0, 0xc0, 0xfe, 0x4c, 0x63, 0xfb, 0x7c, 0xf9, 0x9d, + 0xf1, 0xcc, 0x62, 0x15, 0xa7, 0x93, 0xf0, 0xd8, 0xfb, 0x83, 0x3b, 0x96, + 0xef, 0xc5, 0x51, 0x70, 0xb5, 0x4e, 0xc7, 0xda, 0xd4, 0x42, 0x46, 0x4b, + 0x37, 0x18, 0x4f, 0xf9, 0xe6, 0xe6, 0x48, 0xa3, 0x0a, 0x77, 0xfc, 0x21, + 0x27, 0x7a, 0x82, 0x84, 0x0e, 0xc7, 0xfc, 0x3b, 0x8a, 0xba, 0xed, 0xbf, + 0xa9, 0x91, 0x87, 0x2e, 0x93, 0xe9, 0xdf, 0x14, 0x80, 0xef, 0x84, 0xc8, + 0xa7, 0x6e, 0x7c, 0x5a, 0x5a, 0x5a, 0xc1, 0x21, 0x5f, 0x6f, 0x06, 0x57, + 0xa7, 0x95, 0x81, 0xca, 0x36, 0x65, 0x5a, 0x93, 0x1e, 0x8c, 0x02, 0x89, + 0xf7, 0x0f, 0xfc, 0x2a, 0x5b, 0xb8, 0xf6, 0x5f, 0x1b, 0xf2, 0xc4, 0x6f, + 0xc7, 0x25, 0x94, 0x44, 0x05, 0x93, 0xd8, 0x4a, 0x58, 0x7a, 0xa2, 0xa0, + 0xbd, 0x40, 0x6f, 0x29, 0xd6, 0xb5, 0x9d, 0x8f, 0xf9, 0x3a, 0xce, 0x01, + 0xd4, 0x59, 0xc4, 0x14, 0xc5, 0xba, 0x9f, 0xf3, 0xc8, 0x7a, 0x27, 0x5e, + 0x6e, 0xb1, 0xae, 0xf9, 0x37, 0xc8, 0xe3, 0x55, 0xa4, 0xa5, 0xe2, 0x40, + 0xbd, 0x8d, 0x6d, 0x51, 0xe9, 0x82, 0xf6, 0x74, 0xc5, 0x49, 0x10, 0xfc, + 0x2a, 0x7e, 0x5b, 0x90, 0xc3, 0x7d, 0xdf, 0x03, 0x9a, 0x98, 0x81, 0x53, + 0xca, 0xa9, 0xc6, 0xb7, 0xf3, 0x7a, 0xe7, 0x3b, 0xef, 0xbb, 0x3e, 0xe5, + 0x5f, 0x7c, 0x69, 0x68, 0x87, 0x0c, 0x3d, 0x1e, 0xb2, 0x09, 0xb3, 0x31, + 0x16, 0x90, 0xf6, 0x72, 0x24, 0x49, 0x3f, 0x2c, 0xae, 0x23, 0x42, 0xff, + 0xc9, 0xa5, 0x3f, 0xf7, 0x2c, 0x2b, 0xe8, 0xe9, 0x72, 0x41, 0x98, 0xc3, + 0xd6, 0xe3, 0x4b, 0xa8, 0x65, 0xfe, 0x09, 0x4e, 0x0a, 0xbf, 0x88, 0x5a, + 0xb2, 0x72, 0x87, 0x2c, 0x0f, 0xc7, 0x3f, 0xa9, 0xa0, 0x4c, 0x62, 0x6a, + 0x96, 0x69, 0x04, 0x8b, 0x0a, 0x61, 0x7b, 0xa5, 0x8c, 0xcd, 0x58, 0xb5, + 0x90, 0x5a, 0xd4, 0xd3, 0xfb, 0xe7, 0x52, 0xab, 0xbd, 0xb8, 0x40, 0xe4, + 0xcb, 0x6d, 0xf0, 0xd0, 0x4e, 0xad, 0xf5, 0xe2, 0x96, 0x2e, 0x5a, 0x2d, + 0xfb, 0x13, 0xea, 0x6e, 0xa8, 0x80, 0x7c, 0x01, 0x2f, 0xe7, 0xa5, 0x87, + 0x2b, 0xe2, 0x6e, 0x5f, 0xa2, 0x78, 0xd2, 0x01, 0x10, 0xd6, 0x93, 0x98, + 0x25, 0xc2, 0x75, 0x07, 0x41, 0x5b, 0xfb, 0x58, 0x63, 0x18, 0x02, 0x6b, + 0x9f, 0x63, 0xcd, 0x11, 0x42, 0xe3, 0xaf, 0xbf, 0x94, 0xf2, 0x01, 0x90, + 0x89, 0xe4, 0x76, 0xe3, 0xc1, 0x43, 0x79, 0xb4, 0x29, 0x69, 0x53, 0xa1, + 0xbc, 0x70, 0x6a, 0x16, 0x1b, 0xad, 0xf2, 0x78, 0xd7, 0xbf, 0x64, 0x77, + 0x8c, 0xc6, 0xc1, 0x61, 0x3e, 0xf7, 0xfb, 0xd8, 0xb3, 0x2c, 0xa2, 0x16, + 0xda, 0x13, 0x14, 0x5f, 0x72, 0xfc, 0xf5, 0x1a, 0x22, 0xf4, 0x20, 0x89, + 0x2f, 0x52, 0xe7, 0x50, 0x1e, 0x1f, 0x12, 0x78, 0x15, 0xb2, 0x77, 0x79, + 0x59, 0xc1, 0x94, 0xff, 0x3f, 0x00, 0x5b, 0x14, 0xa4, 0xeb, 0x5f, 0x76, + 0xbd, 0xb5, 0x7e, 0x78, 0x11, 0x84, 0xff, 0x5f, 0xc1, 0x9b, 0x7c, 0xee, + 0xf5, 0x4f, 0x2c, 0xa1, 0x58, 0x38, 0x2d, 0x84, 0x50, 0xc3, 0x32, 0x33, + 0x7f, 0xb3, 0x88, 0x7e, 0xb9, 0x41, 0x55, 0xeb, 0x02, 0x99, 0xf7, 0x0a, + 0x61, 0x13, 0x00, 0xdb, 0x76, 0xed, 0xc3, 0xe5, 0xf6, 0x58, 0x82, 0xb1, + 0xf0, 0x8f, 0x2d, 0xcf, 0x9f, 0x25, 0xf0, 0x7d, 0xf4, 0x6a, 0x36, 0x5b, + 0x43, 0x8b, 0xe8, 0x97, 0x1b, 0x0c, 0xe3, 0xaf, 0xfc, 0x4f, 0x11, 0x37, + 0x14, 0xfc, 0x36, 0x7c, 0xdc, 0xe7, 0xd6, 0xa0, 0x9c, 0x85, 0x13, 0x88, + 0x6e, 0xf3, 0x25, 0xd2, 0x10, 0x40, 0x13, 0xaf, 0x77, 0x93, 0x45, 0xf4, + 0xcb, 0xf7, 0x65, 0xa6, 0x2e, 0x00, 0xc7, 0x34, 0x40, 0xfe, 0xf2, 0xdb, + 0x5a, 0x1c, 0x76, 0x9b, 0x45, 0x76, 0x0b, 0xa7, 0x70, 0xdd, 0x83, 0xbe, + 0x2d, 0xed, 0x6d, 0xb4, 0x88, 0x7e, 0xc6, 0x17, 0x18, 0x8d, 0xcd, 0x16, + 0x3b, 0x08, 0x10, 0x86, 0x61, 0xf2, 0xf3, 0xaf, 0xeb, 0xac, 0x3c, 0x73, + 0x16, 0xf2, 0xc3, 0x3b, 0x85, 0xb4, 0xe3, 0x59, 0xf9, 0x06, 0xc1, 0xdc, + 0x62, 0x11, 0xfd, 0xf2, 0xb6, 0xe8, 0x78, 0xbd, 0xfb, 0x0e, 0x0b, 0x11, + 0xf5, 0x05, 0x20, 0x9f, 0x79, 0xed, 0x23, 0xcb, 0xa2, 0x5b, 0x00, 0x7c, + 0x05, 0x29, 0x0e, 0x1e, 0x4d, 0xf3, 0x91, 0x48, 0x09, 0xca, 0xb0, 0x88, + 0x7e, 0xb9, 0xbf, 0x50, 0xad, 0x0e, 0x52, 0xf2, 0x28, 0x20, 0xf2, 0x3c, + 0x5e, 0x3c, 0x5e, 0xaf, 0x25, 0x14, 0x0b, 0x18, 0x86, 0x49, 0x79, 0x7f, + 0x4a, 0xa9, 0xb0, 0xdd, 0x16, 0xd1, 0x2f, 0x73, 0xe8, 0xfa, 0x36, 0x20, + 0x3d, 0x57, 0xd3, 0xe2, 0x46, 0x03, 0xf2, 0xf5, 0x0f, 0xbe, 0xb6, 0x84, + 0x62, 0x01, 0x21, 0x04, 0xde, 0xfc, 0x4d, 0x4f, 0xa6, 0x99, 0x61, 0xad, + 0x8c, 0x0b, 0x04, 0x44, 0x46, 0xb6, 0x41, 0xd7, 0x53, 0x5e, 0x06, 0x21, + 0x66, 0xfe, 0xf8, 0x9b, 0xb4, 0x6a, 0xb7, 0x59, 0xd0, 0x0d, 0xa3, 0x50, + 0x21, 0x87, 0x90, 0x83, 0x16, 0xd1, 0x03, 0x00, 0x19, 0x19, 0xcb, 0x00, + 0xc3, 0x25, 0x44, 0xf8, 0x1b, 0x80, 0x70, 0xb9, 0xf3, 0x2c, 0xa1, 0x58, + 0x16, 0x9d, 0x7f, 0xf4, 0xc0, 0x65, 0x59, 0xf4, 0x40, 0x81, 0xaa, 0x56, + 0x05, 0x8e, 0xbf, 0x00, 0xc8, 0x29, 0x33, 0x7e, 0xb1, 0x76, 0xb4, 0x95, + 0x75, 0x7d, 0x50, 0x04, 0x19, 0x59, 0x39, 0x00, 0x68, 0x9a, 0xd2, 0xc9, + 0x22, 0x7a, 0x80, 0xc0, 0x30, 0xf6, 0xa1, 0xaa, 0x09, 0x06, 0x08, 0xf9, + 0xc9, 0xb4, 0xf9, 0x64, 0xe5, 0xb8, 0x2c, 0xa1, 0x94, 0xe9, 0x8e, 0x5f, + 0xe5, 0x58, 0x7a, 0x26, 0x80, 0xd4, 0xf5, 0xf4, 0x04, 0x8b, 0xe8, 0x81, + 0x14, 0x97, 0xe9, 0x3b, 0x25, 0x04, 0x2d, 0x2e, 0x70, 0xdd, 0x2c, 0x94, + 0x5d, 0x98, 0xa6, 0x49, 0x48, 0x90, 0x03, 0x40, 0x80, 0x50, 0x2d, 0xa2, + 0x07, 0x18, 0x14, 0x25, 0xe4, 0x13, 0x00, 0xb7, 0xc7, 0x63, 0xcd, 0xa9, + 0x97, 0x61, 0x78, 0x75, 0x83, 0x6a, 0x95, 0x63, 0x0b, 0xfc, 0xbd, 0x3a, + 0x16, 0xd1, 0x03, 0x0c, 0x76, 0x7b, 0xad, 0x9f, 0x01, 0x16, 0xaf, 0xd8, + 0x80, 0xa2, 0x58, 0x56, 0xbd, 0xac, 0x42, 0x4a, 0x89, 0xd3, 0x6e, 0xcf, + 0xff, 0xa6, 0x25, 0x5b, 0x44, 0x0f, 0x20, 0x04, 0x05, 0xb5, 0xc6, 0xed, + 0x5e, 0x9e, 0x0a, 0x98, 0x6f, 0x7d, 0x32, 0xdd, 0x4a, 0x48, 0x51, 0xa6, + 0x3d, 0x3b, 0x85, 0xcc, 0xec, 0xdc, 0x02, 0xfb, 0x1e, 0x65, 0x11, 0x3d, + 0x80, 0xe0, 0x72, 0xf9, 0x13, 0x89, 0x18, 0xb9, 0xae, 0x3c, 0x0e, 0x1e, + 0x39, 0x66, 0xb9, 0xef, 0x65, 0xd8, 0xa2, 0x87, 0x85, 0x06, 0x03, 0x20, + 0x84, 0x23, 0xd3, 0x22, 0x7a, 0x60, 0x42, 0x07, 0xe4, 0xec, 0x9f, 0x57, + 0xa0, 0x5a, 0x56, 0xbd, 0xcc, 0xc2, 0x34, 0xcd, 0x7c, 0xd2, 0x7b, 0x6b, + 0x5a, 0x44, 0x0f, 0x30, 0xd8, 0x6c, 0xf5, 0x01, 0xc7, 0x6e, 0x40, 0x4c, + 0xf8, 0x62, 0x0e, 0x96, 0x3d, 0x2f, 0x9b, 0x10, 0x80, 0xc7, 0xab, 0x17, + 0x7c, 0xb5, 0x36, 0xb5, 0x04, 0x1a, 0xbc, 0xde, 0x2d, 0x80, 0x3a, 0xaf, + 0xe0, 0xfb, 0x5f, 0x3b, 0xf7, 0x59, 0xee, 0x7b, 0x59, 0x24, 0xba, 0x22, + 0xc8, 0x75, 0xb9, 0x01, 0x24, 0x08, 0xdd, 0x22, 0x7a, 0x00, 0x42, 0x51, + 0x42, 0x56, 0x17, 0x7c, 0x9e, 0xf5, 0xe3, 0x6f, 0x68, 0x9a, 0x6a, 0x09, + 0xa5, 0x2c, 0xc6, 0xe9, 0x7e, 0xe3, 0xae, 0xae, 0xb4, 0xd9, 0xea, 0x58, + 0x44, 0x0f, 0xbc, 0xd8, 0x2c, 0xc5, 0x9f, 0x51, 0x64, 0xea, 0x77, 0x8b, + 0x30, 0x4d, 0xcb, 0xa2, 0x97, 0xb9, 0x10, 0x4e, 0xd3, 0x48, 0xde, 0xe7, + 0x2b, 0x1f, 0x25, 0x84, 0xb6, 0xdf, 0xeb, 0xdd, 0x66, 0x11, 0x3d, 0x40, + 0x9d, 0x37, 0xff, 0xce, 0x96, 0x1d, 0x7b, 0x0e, 0x5a, 0xe2, 0x28, 0x6b, + 0xd6, 0x5c, 0x4a, 0xbf, 0x27, 0x27, 0xa5, 0x5e, 0xbd, 0xb4, 0xb7, 0xd7, + 0x22, 0xfa, 0xf9, 0x23, 0xa7, 0xe0, 0xc3, 0xae, 0xbd, 0x87, 0x2c, 0x69, + 0x94, 0x31, 0xa8, 0xaa, 0x42, 0x46, 0x66, 0x76, 0x81, 0x45, 0xaf, 0x66, + 0x11, 0x3d, 0x00, 0xe1, 0x74, 0x5e, 0x89, 0xa2, 0x84, 0x7d, 0x56, 0xf0, + 0xfd, 0xcf, 0xbf, 0x76, 0x59, 0xd3, 0x6c, 0x65, 0xd0, 0xa2, 0x3b, 0x1d, + 0xb6, 0x7c, 0xa2, 0xab, 0x5b, 0x2d, 0xa2, 0x07, 0x20, 0xdc, 0xee, 0xb5, + 0x98, 0x66, 0xe6, 0x53, 0x05, 0xdf, 0x97, 0xfe, 0xbe, 0xd1, 0x1a, 0x90, + 0x2b, 0x83, 0x31, 0x7a, 0x41, 0x05, 0x1f, 0x29, 0x75, 0x6b, 0x7a, 0x2d, + 0xc0, 0xe3, 0xf4, 0x14, 0x80, 0x3d, 0x07, 0x8e, 0x94, 0xea, 0x82, 0x80, + 0x16, 0x8a, 0x1f, 0x3b, 0xf6, 0x1c, 0x44, 0x9a, 0x12, 0x10, 0x48, 0x99, + 0xfb, 0x91, 0x45, 0xf4, 0x80, 0x86, 0x7d, 0x6a, 0xc1, 0xa7, 0xd5, 0x1b, + 0xb7, 0x5b, 0xf3, 0xe9, 0x65, 0x08, 0xbf, 0xaf, 0x2f, 0x28, 0xf2, 0xa1, + 0xae, 0xbd, 0x1c, 0xda, 0x6b, 0x11, 0xfd, 0x02, 0xa0, 0x69, 0x15, 0x37, + 0x14, 0x7c, 0xfe, 0xee, 0x97, 0x15, 0x96, 0xfb, 0x5e, 0x86, 0x90, 0x96, + 0xe1, 0xcb, 0xe9, 0xae, 0x28, 0x36, 0xc5, 0x22, 0x7a, 0x80, 0x43, 0xd7, + 0x93, 0x3f, 0x2d, 0xf8, 0xbc, 0x69, 0x7b, 0x32, 0x36, 0x8b, 0xe8, 0x65, + 0x02, 0x52, 0x4a, 0x92, 0xf7, 0x1f, 0x2e, 0xf8, 0xb8, 0xdd, 0x22, 0x7a, + 0xd9, 0x88, 0xd3, 0x0d, 0x80, 0xdd, 0x7b, 0x0f, 0xa3, 0x69, 0x9a, 0x25, + 0x8e, 0xb2, 0xe0, 0xc9, 0xa9, 0x2a, 0xfb, 0x0f, 0xa7, 0x02, 0x08, 0x90, + 0x3b, 0x2c, 0xa2, 0x97, 0x09, 0x28, 0x7b, 0x00, 0xbc, 0xba, 0x5e, 0x28, + 0x6e, 0xb3, 0x10, 0xd0, 0x5d, 0xbb, 0x10, 0xe8, 0xf9, 0x39, 0xdd, 0xa5, + 0x94, 0x1e, 0x8b, 0xe8, 0x65, 0x03, 0xfb, 0x0a, 0x3e, 0x04, 0x39, 0xec, + 0x96, 0x34, 0xca, 0x42, 0xd7, 0xae, 0x08, 0x72, 0x7c, 0x1b, 0x5a, 0x00, + 0x8e, 0x58, 0x44, 0x0f, 0x70, 0x44, 0x46, 0xde, 0x03, 0xd8, 0xfd, 0xd9, + 0x28, 0x72, 0x5d, 0x56, 0xbe, 0xf7, 0x32, 0xe1, 0xba, 0x6b, 0x2a, 0xa9, + 0x69, 0xc7, 0xf3, 0x49, 0x1f, 0xbe, 0xd3, 0x22, 0x7a, 0x80, 0x23, 0x23, + 0xe3, 0x4b, 0x84, 0x70, 0xfc, 0x56, 0xf0, 0x3d, 0x38, 0xd8, 0x69, 0x09, + 0xa5, 0x2c, 0xbc, 0xf7, 0xcc, 0x1c, 0xd2, 0x8f, 0xfb, 0x96, 0xbf, 0x9a, + 0xa6, 0xf8, 0xcb, 0x22, 0x7a, 0xd9, 0x40, 0x70, 0xc1, 0x87, 0xb4, 0xf4, + 0x4c, 0x4b, 0x1a, 0x65, 0x00, 0xfb, 0x0f, 0xa5, 0x14, 0x72, 0xe3, 0x8d, + 0x04, 0x8b, 0xe8, 0x65, 0x22, 0x5e, 0x8b, 0x5f, 0x5a, 0xf0, 0xf9, 0x70, + 0x6a, 0xba, 0x25, 0x90, 0x00, 0x87, 0x94, 0x92, 0x95, 0xeb, 0xfe, 0x59, + 0xda, 0xae, 0xaa, 0xc1, 0xe9, 0x16, 0xd1, 0xcb, 0x00, 0x0c, 0x63, 0x9d, + 0x7f, 0x30, 0x26, 0x24, 0xc8, 0x72, 0xdd, 0x03, 0x1d, 0xaa, 0xaa, 0x30, + 0x67, 0xc1, 0x4a, 0x3f, 0x7d, 0xbc, 0xde, 0x7d, 0x1b, 0x2c, 0xa2, 0x97, + 0x19, 0x88, 0xe3, 0x00, 0xbb, 0xf6, 0x59, 0xdb, 0x55, 0x03, 0x1d, 0x76, + 0x4d, 0xe3, 0x68, 0xaa, 0x6f, 0x0f, 0x8b, 0x10, 0xda, 0xee, 0xcb, 0xa5, + 0xdd, 0x16, 0xd1, 0x8b, 0x07, 0x39, 0x00, 0x91, 0xe1, 0xa1, 0x96, 0x24, + 0x02, 0xbd, 0x4b, 0x57, 0x04, 0x1e, 0xaf, 0xb7, 0x80, 0x3e, 0x0b, 0x2d, + 0xa2, 0x97, 0x2d, 0x84, 0x02, 0x24, 0xd5, 0xab, 0x69, 0x49, 0x22, 0xd0, + 0x5d, 0x77, 0x45, 0x25, 0xdb, 0x5f, 0x60, 0xd3, 0x31, 0x23, 0x32, 0xb2, + 0xbd, 0x45, 0xf4, 0xb2, 0x03, 0x25, 0x03, 0xa0, 0x7c, 0x54, 0xb8, 0x25, + 0x8a, 0x00, 0xc7, 0xcc, 0x9f, 0xfc, 0xb3, 0xa9, 0xd8, 0x6c, 0x6d, 0x96, + 0x65, 0x64, 0x2c, 0xb0, 0x88, 0x5e, 0x86, 0xb0, 0xc7, 0x12, 0x41, 0xe0, + 0x43, 0x4a, 0xc9, 0x8c, 0xef, 0x97, 0xe5, 0x7f, 0xd3, 0xf0, 0x78, 0xe6, + 0x66, 0x5b, 0xae, 0x7b, 0x19, 0xd3, 0x01, 0x80, 0x4a, 0x15, 0xca, 0x5b, + 0x92, 0x08, 0x64, 0xb7, 0x5d, 0x55, 0x0a, 0xcd, 0xa1, 0xcb, 0x23, 0x97, + 0x53, 0xdb, 0x2d, 0xa2, 0x17, 0x0b, 0x7c, 0x09, 0xfc, 0xad, 0x2c, 0x33, + 0x65, 0x20, 0x3e, 0xcf, 0x2d, 0x58, 0xe3, 0xae, 0x6e, 0xb7, 0x88, 0x5e, + 0xd6, 0x68, 0x2e, 0x94, 0x63, 0x60, 0xad, 0x75, 0x2f, 0x0b, 0x8e, 0x9b, + 0xf0, 0x55, 0xca, 0x96, 0xaa, 0x1a, 0xfe, 0x8d, 0x45, 0xf4, 0xb2, 0x17, + 0xbb, 0xe5, 0x01, 0xa4, 0x67, 0x66, 0x5b, 0xc2, 0x08, 0x60, 0x78, 0xbc, + 0x7a, 0xc1, 0x1a, 0x77, 0x61, 0x9a, 0xae, 0xed, 0x35, 0x6a, 0xbc, 0x69, + 0x11, 0xbd, 0x8c, 0x39, 0x75, 0x87, 0x7c, 0xae, 0xbb, 0x61, 0x89, 0x22, + 0x70, 0x3b, 0x73, 0x3e, 0x9a, 0x3a, 0xdf, 0xff, 0x5d, 0xd3, 0xe2, 0xff, + 0x48, 0x4e, 0x7e, 0xda, 0x22, 0x7a, 0x19, 0x53, 0x83, 0x4a, 0x00, 0x79, + 0x1e, 0xaf, 0x25, 0x8a, 0x00, 0x85, 0xa2, 0x08, 0xa6, 0x7d, 0xb7, 0x28, + 0xff, 0x73, 0xe8, 0xef, 0x5e, 0xef, 0xa6, 0x63, 0x96, 0xeb, 0x5e, 0xe6, + 0x60, 0xda, 0x7c, 0xbd, 0xbe, 0x25, 0x89, 0x80, 0xf5, 0xd9, 0x14, 0x15, + 0x97, 0xdb, 0x97, 0x4c, 0xc6, 0x34, 0x8d, 0xff, 0xbb, 0xec, 0x3a, 0x2a, + 0xeb, 0x15, 0x16, 0x1f, 0xc2, 0x42, 0x82, 0x2c, 0x21, 0x04, 0x28, 0x32, + 0xb3, 0x73, 0x70, 0xe7, 0x79, 0x00, 0xa4, 0xa2, 0x38, 0x0d, 0x8b, 0xe8, + 0x65, 0x10, 0x42, 0xd8, 0x36, 0x01, 0x6c, 0xdb, 0xb5, 0xcf, 0x12, 0x46, + 0x80, 0x62, 0xef, 0x41, 0xff, 0x1e, 0x74, 0x21, 0xa5, 0xfc, 0x31, 0x28, + 0xa8, 0xbd, 0x45, 0xf4, 0x32, 0x17, 0xa1, 0x4b, 0xaf, 0x0e, 0xe0, 0x74, + 0x38, 0x2c, 0x61, 0x04, 0xe4, 0xfb, 0x95, 0xfc, 0xb8, 0x74, 0x0d, 0x80, + 0x14, 0xc2, 0x81, 0x94, 0x19, 0xbb, 0x5c, 0xae, 0x05, 0x16, 0xd1, 0xcb, + 0x12, 0x9c, 0xce, 0x26, 0x08, 0xa1, 0xd4, 0x05, 0xc8, 0xc9, 0x75, 0x59, + 0x02, 0x09, 0x48, 0x8f, 0x4d, 0xf0, 0xd5, 0xec, 0x05, 0x00, 0x42, 0x55, + 0xa3, 0xae, 0xbc, 0x1c, 0x9f, 0xc1, 0x22, 0xfa, 0x05, 0xc2, 0xed, 0x5e, + 0x87, 0x94, 0x66, 0x1c, 0xc0, 0xf1, 0xac, 0x1c, 0x4b, 0x20, 0x01, 0x0a, + 0xaf, 0xee, 0x5b, 0xf5, 0x68, 0x9a, 0xca, 0x2e, 0x8b, 0xe8, 0x65, 0x16, + 0x46, 0x18, 0x80, 0x69, 0x0d, 0xbb, 0x07, 0xe6, 0xdb, 0x35, 0x0d, 0x4c, + 0xd3, 0x47, 0xf4, 0xb0, 0xb0, 0x17, 0xb3, 0x2d, 0xa2, 0x97, 0x5d, 0x54, + 0x01, 0xc8, 0xca, 0xce, 0xb5, 0x24, 0x11, 0x80, 0x18, 0x37, 0x69, 0xa6, + 0x3f, 0x5c, 0x3f, 0x7e, 0xfc, 0x31, 0x1d, 0x5f, 0xa2, 0x91, 0x74, 0xb0, + 0xfd, 0xa5, 0xaa, 0x71, 0x8f, 0x17, 0xfc, 0x31, 0x2c, 0xac, 0x93, 0x45, + 0xf4, 0x00, 0x8f, 0xe2, 0x2a, 0xfa, 0xdc, 0x3b, 0xc3, 0xaa, 0xa8, 0x1a, + 0x40, 0x90, 0x52, 0x62, 0x9a, 0x92, 0x2f, 0x66, 0xfe, 0x02, 0xc0, 0xb1, + 0x63, 0xc7, 0x84, 0xcb, 0xe5, 0xe2, 0xd8, 0xb1, 0x63, 0xc1, 0x9b, 0x36, + 0x6d, 0x8a, 0x7c, 0xfe, 0xf9, 0xa7, 0xea, 0x54, 0xae, 0x6c, 0x7b, 0x1f, + 0x90, 0xa0, 0x1e, 0xc8, 0xc9, 0xd9, 0xdc, 0xcc, 0x22, 0x7a, 0x60, 0x3b, + 0x77, 0x1a, 0xc0, 0xd1, 0xd4, 0x74, 0x14, 0x45, 0x58, 0xe2, 0x08, 0x10, + 0x28, 0x42, 0x30, 0xe6, 0xe3, 0x6f, 0x01, 0xa8, 0x53, 0xa7, 0x0e, 0xe5, + 0xca, 0x95, 0xc3, 0xe9, 0x74, 0x52, 0xae, 0x5c, 0x39, 0x1a, 0x34, 0x68, + 0xc0, 0xab, 0xaf, 0xbe, 0xc6, 0x9e, 0x3d, 0x7b, 0x59, 0xb8, 0x70, 0x21, + 0x51, 0x51, 0xe1, 0x15, 0x4d, 0x73, 0xcf, 0x6a, 0x21, 0x22, 0x3e, 0xb2, + 0x88, 0x1e, 0x80, 0x08, 0x09, 0x79, 0xc1, 0xbf, 0x4a, 0x26, 0x33, 0x3b, + 0x17, 0x45, 0x58, 0x22, 0x0d, 0x14, 0x6b, 0xee, 0xd1, 0x0d, 0xbe, 0x9a, + 0xed, 0x4b, 0x0b, 0xb7, 0x68, 0xd1, 0xa2, 0xd3, 0x1e, 0xdb, 0xae, 0x5d, + 0x3b, 0xd2, 0xd2, 0xd2, 0x44, 0xd7, 0xae, 0xb7, 0x49, 0x29, 0x8f, 0x3f, + 0x2c, 0x44, 0xd4, 0xbd, 0x10, 0x61, 0x11, 0x3d, 0x90, 0x90, 0x93, 0x33, + 0xca, 0x05, 0x3e, 0x2b, 0xde, 0xb4, 0x51, 0x2d, 0xff, 0xa0, 0x8d, 0x85, + 0xcb, 0x3c, 0x18, 0x13, 0x82, 0x8d, 0xdb, 0xfe, 0x49, 0xf2, 0x1a, 0x1d, + 0x1d, 0x7d, 0xd6, 0x8e, 0x61, 0xe6, 0xcc, 0x99, 0xa2, 0x53, 0xa7, 0x4e, + 0x52, 0xca, 0xf4, 0xcf, 0x21, 0xf4, 0x0a, 0x80, 0x7a, 0xf5, 0xf6, 0x5b, + 0x44, 0x0f, 0x34, 0xdc, 0xd9, 0xa9, 0x2d, 0x56, 0x84, 0x1e, 0x18, 0x30, + 0x4d, 0x49, 0xdf, 0xe7, 0xdf, 0x06, 0x60, 0xf0, 0xe0, 0xc1, 0xd8, 0x6c, + 0xb6, 0xb3, 0x76, 0x0c, 0x52, 0x4a, 0xe6, 0xce, 0x9d, 0x2b, 0x7a, 0xf7, + 0xee, 0x2d, 0xe1, 0xc0, 0xdf, 0x76, 0x7b, 0xdd, 0x26, 0x5b, 0xb7, 0x56, + 0x29, 0x1d, 0x1d, 0x97, 0xf5, 0x4a, 0x2f, 0x0c, 0xb1, 0xb1, 0x52, 0x39, + 0x7a, 0x54, 0x18, 0x03, 0x1e, 0xe8, 0xca, 0x43, 0x3d, 0x6f, 0xb1, 0xa6, + 0xd8, 0x02, 0x04, 0x1b, 0xff, 0xda, 0xcd, 0xbd, 0x83, 0x47, 0xfb, 0xad, + 0xb5, 0x94, 0x12, 0x21, 0xc4, 0x59, 0xad, 0xba, 0x10, 0x02, 0xb7, 0xdb, + 0x4d, 0x50, 0x50, 0x10, 0xe0, 0xdc, 0x07, 0xee, 0x6a, 0x96, 0x45, 0x0f, + 0x00, 0x1c, 0x3d, 0x2a, 0x4c, 0x10, 0x39, 0x6e, 0x8f, 0xd7, 0xb2, 0xe6, + 0x01, 0x12, 0x9b, 0x0b, 0x21, 0x18, 0xf4, 0xca, 0x87, 0x00, 0xf4, 0xe9, + 0xd3, 0xc7, 0x6f, 0xb1, 0x8b, 0xe2, 0xee, 0x3f, 0xf5, 0xd4, 0x53, 0x48, + 0x29, 0xb9, 0xe7, 0x9e, 0x7b, 0x24, 0xb8, 0xab, 0xda, 0x6c, 0x8d, 0x15, + 0x8b, 0xe8, 0x81, 0x83, 0x90, 0x3f, 0xb7, 0xee, 0xc2, 0xa6, 0xa9, 0x96, + 0x24, 0x2e, 0x73, 0x92, 0xab, 0xaa, 0xca, 0x07, 0xff, 0xf7, 0x1d, 0xa9, + 0xe9, 0xbe, 0xb2, 0xc8, 0x93, 0x26, 0x4d, 0x2a, 0xf2, 0x94, 0x69, 0x66, + 0x66, 0x26, 0x63, 0xc7, 0x8e, 0xe5, 0xa6, 0x9b, 0x6e, 0xa2, 0x79, 0xf3, + 0xe6, 0x02, 0x40, 0xd7, 0x77, 0x97, 0x8a, 0x3a, 0x5d, 0x9a, 0xf5, 0x7a, + 0x8b, 0x27, 0x02, 0xda, 0xb6, 0x6b, 0x3f, 0x9a, 0xaa, 0xe2, 0xf1, 0xed, + 0x6f, 0xb1, 0x70, 0x19, 0x42, 0x51, 0x04, 0x3f, 0xff, 0xba, 0x96, 0x0f, + 0xbf, 0x9c, 0xeb, 0xff, 0x6d, 0xf5, 0xea, 0xd5, 0x34, 0x6f, 0xde, 0xbc, + 0x88, 0xe7, 0xfb, 0xec, 0xe6, 0xb2, 0x65, 0xcb, 0x58, 0xb1, 0x62, 0x45, + 0x7e, 0xe7, 0x51, 0x3a, 0x92, 0x91, 0x58, 0x16, 0xbd, 0x78, 0x88, 0xbe, + 0x49, 0xd3, 0x54, 0x2b, 0x3a, 0xbf, 0xbc, 0xed, 0x39, 0x19, 0x99, 0x39, + 0x3c, 0x35, 0xf2, 0x43, 0xdf, 0x97, 0x7c, 0xb4, 0x68, 0xd1, 0x82, 0x15, + 0x2b, 0x96, 0xf3, 0xfb, 0xef, 0xbf, 0xcb, 0x23, 0x47, 0x8e, 0xe0, 0x76, + 0xbb, 0xfd, 0xd6, 0xff, 0xdf, 0x08, 0x0d, 0x0d, 0x65, 0xce, 0x9c, 0x39, + 0xf8, 0x2c, 0xb9, 0x9e, 0x4f, 0xfe, 0x60, 0xaf, 0x45, 0xf4, 0x80, 0xb1, + 0x04, 0xce, 0x55, 0x29, 0xc7, 0x32, 0x84, 0xaa, 0x58, 0xe2, 0xbc, 0x5c, + 0x5d, 0x76, 0xbb, 0xcd, 0xc6, 0x7f, 0x9e, 0x19, 0x5b, 0x40, 0x72, 0x21, + 0x44, 0xe4, 0x04, 0x40, 0xa8, 0x6a, 0x8b, 0x2b, 0xae, 0xb9, 0xe6, 0xe6, + 0xd5, 0x2d, 0x5b, 0xb6, 0x14, 0x71, 0x71, 0x71, 0x04, 0x05, 0x05, 0xb1, + 0x61, 0xc3, 0x86, 0xd3, 0xc6, 0xec, 0x2d, 0x5a, 0xb4, 0x38, 0xe1, 0xbb, + 0xc3, 0x11, 0x67, 0x58, 0x44, 0x0f, 0x00, 0xf4, 0xe9, 0xe3, 0x01, 0xec, + 0xc9, 0x00, 0x07, 0x8f, 0x1e, 0xb3, 0x04, 0x72, 0x19, 0x42, 0xd3, 0x54, + 0x9e, 0x7b, 0xe3, 0x53, 0x76, 0xec, 0x39, 0x00, 0x20, 0xec, 0xf6, 0xaa, + 0xe3, 0xa5, 0xcc, 0x78, 0x02, 0xc0, 0x30, 0x56, 0xed, 0x34, 0xcd, 0xac, + 0x16, 0x36, 0x5b, 0xfd, 0x56, 0xf9, 0x9d, 0x80, 0x4c, 0x4a, 0x4a, 0x22, + 0x39, 0x39, 0xf9, 0x24, 0xab, 0x2e, 0xa5, 0xc4, 0xe5, 0x72, 0x15, 0x74, + 0x16, 0x7a, 0xb9, 0x72, 0x37, 0x56, 0x70, 0xb9, 0x36, 0x9b, 0x16, 0xd1, + 0x03, 0x00, 0x9f, 0x7e, 0x6a, 0xc7, 0x34, 0xd3, 0x5f, 0x01, 0x58, 0xb5, + 0xfe, 0x2f, 0x6b, 0xad, 0xfb, 0x65, 0x18, 0x97, 0xbf, 0xfe, 0xc1, 0xd7, + 0xcc, 0xf9, 0x79, 0x39, 0x00, 0x36, 0x5b, 0xe5, 0x17, 0x3c, 0x9e, 0x7d, + 0xfd, 0x63, 0x62, 0x5e, 0x3e, 0xe1, 0x38, 0xaf, 0x77, 0xcb, 0x4a, 0x45, + 0xa9, 0x68, 0x53, 0xd5, 0xc8, 0xdf, 0x01, 0x5a, 0xb6, 0x6c, 0x79, 0x92, + 0x55, 0x17, 0x42, 0xd0, 0xb6, 0x6d, 0x5b, 0x00, 0x61, 0xb3, 0x55, 0x79, + 0x35, 0x2d, 0xed, 0xe7, 0xa3, 0xe5, 0xcb, 0x97, 0x8e, 0x8d, 0x2e, 0x16, + 0xd1, 0x8b, 0xcf, 0x2e, 0xe4, 0x2e, 0x5f, 0xbb, 0x05, 0x55, 0xb5, 0x44, + 0x7a, 0xb9, 0x40, 0x55, 0x15, 0xc6, 0x4e, 0xfa, 0x1f, 0x5f, 0xce, 0x5a, + 0x90, 0x4f, 0xf2, 0x2a, 0xaf, 0x7a, 0xbd, 0x07, 0x5e, 0x85, 0x1a, 0xa4, + 0xa4, 0xbc, 0x74, 0x0a, 0xcb, 0x5f, 0xdd, 0x30, 0x8c, 0x8c, 0x56, 0x42, + 0x38, 0x57, 0x1c, 0x39, 0x72, 0x84, 0x97, 0x5e, 0x7a, 0xe9, 0x84, 0x78, + 0x7d, 0xc5, 0x8a, 0x15, 0xec, 0xdd, 0xbb, 0x57, 0x82, 0x3d, 0xc3, 0xeb, + 0xdd, 0xff, 0x12, 0xc0, 0xb1, 0x63, 0xf3, 0x2c, 0x78, 0x4e, 0x71, 0x41, + 0x00, 0x00, 0x0a, 0xa5, 0x49, 0x44, 0x41, 0x54, 0xa2, 0x07, 0x12, 0x84, + 0x50, 0x56, 0x6d, 0xdb, 0xb5, 0x4f, 0xda, 0x34, 0x6b, 0x22, 0xe3, 0x72, + 0x80, 0xcd, 0xa6, 0x31, 0x6c, 0xec, 0x14, 0x3e, 0xfb, 0xf6, 0x47, 0x40, + 0x20, 0x44, 0x85, 0x36, 0x5e, 0xef, 0xfe, 0x17, 0xaa, 0x55, 0x7b, 0x1a, + 0x48, 0x3e, 0xe5, 0x39, 0x1e, 0xcf, 0x4a, 0x14, 0x25, 0x1e, 0x29, 0xdd, + 0xad, 0x35, 0xad, 0xfc, 0xac, 0x11, 0x23, 0x46, 0x70, 0xfb, 0xed, 0xb7, + 0xb3, 0x70, 0xe1, 0x42, 0x9a, 0x35, 0x6b, 0x26, 0x5b, 0xb7, 0x6e, 0x0d, + 0x38, 0x76, 0x85, 0x85, 0x5d, 0x5d, 0xd1, 0xe9, 0xbc, 0xb6, 0x74, 0xe9, + 0xa7, 0xf5, 0xca, 0x8b, 0x8b, 0xe8, 0xa1, 0x0f, 0x4a, 0x99, 0xfd, 0xe9, + 0x86, 0xef, 0x27, 0x62, 0x9a, 0x66, 0x91, 0x16, 0x58, 0x58, 0xb8, 0x34, + 0x08, 0x72, 0xda, 0xb9, 0x7b, 0xe0, 0x6b, 0xfc, 0xb1, 0xf1, 0x6f, 0x09, + 0x8a, 0xb0, 0xd9, 0xaa, 0xb6, 0xf3, 0x7a, 0xf7, 0x2c, 0x2e, 0xea, 0xf9, + 0x15, 0x2a, 0xdc, 0xcd, 0x91, 0x23, 0x5f, 0xa1, 0x69, 0x71, 0x8f, 0xeb, + 0x7a, 0xea, 0xfb, 0xa0, 0x23, 0x44, 0xe8, 0xf7, 0x52, 0xea, 0xcf, 0x81, + 0xfb, 0xcf, 0xa0, 0xa0, 0x06, 0xb8, 0x5c, 0x9b, 0x2d, 0xa2, 0x07, 0x22, + 0xec, 0xf6, 0xf8, 0x06, 0x1e, 0xcf, 0xee, 0x4d, 0x75, 0x13, 0xaa, 0x32, + 0xeb, 0xa3, 0xe1, 0xb8, 0xf3, 0xac, 0x62, 0x0e, 0xa5, 0x11, 0x4e, 0x87, + 0x8d, 0xde, 0x4f, 0x8e, 0x66, 0xcd, 0xc6, 0xed, 0x12, 0x14, 0x21, 0x44, + 0x74, 0x0b, 0x29, 0x8f, 0xae, 0x3e, 0xd7, 0xeb, 0xd4, 0xa8, 0x71, 0x9c, + 0xe4, 0xe4, 0x13, 0x77, 0xa8, 0xc5, 0xc6, 0xd6, 0xe6, 0xe8, 0xd1, 0xd2, + 0x59, 0x7b, 0xd1, 0x22, 0x7a, 0xf1, 0x3a, 0x84, 0x5b, 0xc0, 0x5b, 0x6f, + 0xd7, 0xd2, 0xcf, 0xc9, 0xb4, 0xb2, 0xcd, 0x94, 0x2a, 0x48, 0x29, 0x09, + 0x72, 0xda, 0xb9, 0x6b, 0xe0, 0x6b, 0xac, 0xdb, 0xb4, 0x43, 0x82, 0x9a, + 0x17, 0x14, 0x94, 0x78, 0x85, 0xcb, 0xb5, 0xee, 0x40, 0x99, 0x18, 0x8f, + 0xb0, 0x54, 0xa0, 0x38, 0xdd, 0x77, 0x47, 0x30, 0xe8, 0x37, 0xe6, 0xb8, + 0xdc, 0x34, 0x4f, 0xac, 0x63, 0xb9, 0xef, 0xa5, 0x49, 0xd1, 0x55, 0x95, + 0x6e, 0x8f, 0x8e, 0x60, 0xd3, 0xb6, 0x64, 0x84, 0x70, 0x6e, 0xa8, 0x52, + 0x65, 0x7c, 0x42, 0x5a, 0xda, 0xc7, 0x19, 0x65, 0xe5, 0xf9, 0xad, 0xc1, + 0xb8, 0xe2, 0x72, 0x09, 0x9d, 0x09, 0x48, 0x69, 0x6c, 0x00, 0x98, 0xf3, + 0xf3, 0x0a, 0x82, 0x9d, 0x56, 0x8e, 0xf7, 0xd2, 0x60, 0xc5, 0x15, 0x21, + 0x38, 0x78, 0xe4, 0x18, 0x57, 0xdf, 0xf1, 0xa4, 0xf4, 0x15, 0xd8, 0x70, + 0xcc, 0x94, 0xd2, 0xdd, 0xe4, 0xc8, 0x91, 0x91, 0x9e, 0x32, 0xd5, 0xd1, + 0x59, 0xea, 0x50, 0x3c, 0xd0, 0x75, 0xa2, 0x84, 0x30, 0x66, 0x82, 0x5e, + 0x2e, 0xc7, 0xe5, 0x16, 0xa6, 0x34, 0xb9, 0xb2, 0x61, 0x2d, 0xcb, 0xaa, + 0x5f, 0x52, 0x2b, 0xae, 0xf0, 0xed, 0xf7, 0xbf, 0xd2, 0x77, 0xc8, 0xdb, + 0xd2, 0xe3, 0xf5, 0x0a, 0x87, 0xa3, 0xda, 0xab, 0x86, 0x91, 0xd6, 0x57, + 0xd3, 0xae, 0x40, 0xd7, 0x93, 0xcb, 0x96, 0xb7, 0x69, 0xa9, 0x43, 0x71, + 0xb9, 0xed, 0xf6, 0xcd, 0x52, 0x7a, 0xea, 0x55, 0xa9, 0x52, 0x45, 0x1c, + 0x3a, 0x74, 0x88, 0xa8, 0x88, 0x50, 0x96, 0x7d, 0x33, 0xd6, 0xaa, 0xb0, + 0x7a, 0x09, 0xac, 0xb8, 0xaa, 0x2a, 0xec, 0xdc, 0x73, 0x88, 0x5e, 0xfd, + 0x5f, 0x95, 0x79, 0x1e, 0x8f, 0x00, 0x45, 0x57, 0xd5, 0x2b, 0x3a, 0x18, + 0xc6, 0xf6, 0x45, 0xd1, 0xd1, 0x23, 0x48, 0x4d, 0x1d, 0x56, 0xe6, 0xe4, + 0x62, 0xb9, 0xee, 0xc5, 0x62, 0x39, 0xe2, 0x6e, 0x94, 0xd2, 0x53, 0x7f, + 0xc0, 0x80, 0x01, 0x62, 0xdf, 0xbe, 0x7d, 0x4c, 0x99, 0x32, 0x85, 0xd4, + 0xb4, 0xe3, 0xfc, 0xb6, 0x66, 0xb3, 0xb5, 0x52, 0xee, 0xa2, 0x13, 0x1d, + 0xbe, 0x98, 0xb9, 0x80, 0xdb, 0xfb, 0x0e, 0x97, 0x79, 0x1e, 0x8f, 0x50, + 0x94, 0xa0, 0xdf, 0xc1, 0xb4, 0xd9, 0x6c, 0x21, 0x8b, 0x80, 0x32, 0x49, + 0x72, 0xcb, 0xa2, 0x17, 0x9b, 0x35, 0xd7, 0xf6, 0x37, 0x69, 0x92, 0x58, + 0xe9, 0x8f, 0x3f, 0xfe, 0xf0, 0xcb, 0xb3, 0x79, 0xf3, 0xe6, 0xec, 0xd9, + 0xbd, 0x83, 0x05, 0x5f, 0xbe, 0x61, 0xe5, 0x91, 0xbb, 0x18, 0xb1, 0xb8, + 0xa2, 0xb0, 0xff, 0x50, 0x0a, 0x5d, 0xfb, 0x0e, 0xc7, 0xeb, 0xd5, 0x01, + 0x21, 0x55, 0xb5, 0x4a, 0x0f, 0xc3, 0xd8, 0x37, 0xc3, 0x92, 0x90, 0x45, + 0xf4, 0x62, 0xd3, 0xb5, 0xd4, 0xd4, 0x54, 0xca, 0x95, 0x2b, 0xe7, 0xcf, + 0x1d, 0xe6, 0x72, 0xb9, 0x08, 0x09, 0x09, 0x61, 0xec, 0xd0, 0x47, 0xb9, + 0xbe, 0x75, 0x52, 0x91, 0x52, 0x11, 0x59, 0x38, 0x77, 0x82, 0x17, 0xc8, + 0xfb, 0xd9, 0xd1, 0x93, 0xf8, 0x69, 0xe9, 0x1a, 0x09, 0xa0, 0x28, 0xc1, + 0xdb, 0x4c, 0x33, 0xb7, 0x9e, 0xa2, 0x54, 0xc3, 0x34, 0xf7, 0x5a, 0x82, + 0xb2, 0x88, 0x5e, 0x6c, 0xc8, 0x4b, 0x49, 0x49, 0xb1, 0x97, 0x2f, 0x5f, + 0xfe, 0x04, 0x32, 0xdb, 0xed, 0x76, 0x4c, 0xd3, 0x60, 0xed, 0xbc, 0x0f, + 0x90, 0xa6, 0xe5, 0xc2, 0x97, 0x44, 0x2c, 0x3e, 0x74, 0xcc, 0x67, 0xcc, + 0xf9, 0x65, 0x05, 0x80, 0x14, 0x42, 0x15, 0x52, 0x46, 0xb4, 0x83, 0xb4, + 0xc5, 0x96, 0x84, 0xac, 0x18, 0xbd, 0xf8, 0x85, 0xa8, 0x84, 0xfc, 0x18, + 0x13, 0x13, 0xc3, 0x3b, 0xef, 0xbc, 0x43, 0x72, 0x72, 0x32, 0x00, 0xcf, + 0x3c, 0xf3, 0x0c, 0x5e, 0xaf, 0x17, 0xc3, 0x30, 0x69, 0xde, 0xe5, 0x09, + 0x2b, 0x56, 0x2f, 0x46, 0x82, 0x03, 0xfc, 0xfc, 0xeb, 0x5a, 0xae, 0xec, + 0xf4, 0x78, 0x01, 0xc9, 0x51, 0x94, 0xf0, 0x85, 0x52, 0x36, 0x51, 0x7c, + 0x24, 0xaf, 0x61, 0x09, 0xca, 0xb2, 0xe8, 0xc5, 0x0b, 0x4d, 0xab, 0x89, + 0xae, 0xef, 0x42, 0xd3, 0x62, 0x5f, 0xd5, 0xf5, 0xa3, 0xcf, 0x9f, 0x4e, + 0x3f, 0xc7, 0xbc, 0xd0, 0x57, 0xdc, 0x70, 0x75, 0x13, 0x4b, 0x60, 0x17, + 0x08, 0xd3, 0x94, 0x74, 0x7d, 0x64, 0x18, 0x7b, 0x0f, 0xa6, 0x00, 0x48, + 0xd0, 0xb2, 0x6c, 0xb6, 0x4a, 0x57, 0x7a, 0xbd, 0x7b, 0x77, 0xda, 0x6c, + 0xe5, 0xf1, 0x7a, 0xad, 0x9c, 0x00, 0x16, 0xd1, 0x4b, 0x08, 0x4e, 0x67, + 0x2b, 0xdc, 0xee, 0x15, 0x68, 0x5a, 0xd5, 0x70, 0x5d, 0x3f, 0x1a, 0xa7, + 0x69, 0x91, 0x59, 0xa1, 0xa1, 0x6f, 0x1d, 0x76, 0xb9, 0xfe, 0x4f, 0x95, + 0x72, 0xaf, 0x4d, 0xd7, 0x93, 0xd7, 0x99, 0x66, 0x6e, 0x9d, 0x99, 0x13, + 0x87, 0x13, 0x5f, 0x35, 0xce, 0x12, 0xd8, 0x39, 0x5a, 0x70, 0x45, 0x11, + 0x38, 0x1d, 0x0e, 0x5e, 0x79, 0xf7, 0x4b, 0xa6, 0x7e, 0xb7, 0x10, 0xc3, + 0x30, 0x01, 0x05, 0x4d, 0x8b, 0xfe, 0xff, 0xf6, 0xce, 0x36, 0x28, 0xaa, + 0xeb, 0x8c, 0xe3, 0xff, 0x73, 0xee, 0xcb, 0x5e, 0x70, 0x5f, 0x0c, 0x20, + 0x0c, 0x08, 0x68, 0x42, 0x08, 0x45, 0xab, 0xc6, 0x3a, 0xd4, 0x69, 0x4a, + 0x1b, 0x69, 0x52, 0xda, 0x66, 0xc6, 0xb6, 0x93, 0x92, 0xa4, 0x93, 0xb6, + 0x32, 0xed, 0x87, 0xaa, 0xfd, 0x52, 0x26, 0x9d, 0x44, 0x74, 0x1c, 0x07, + 0x94, 0xaa, 0x33, 0x4e, 0x1a, 0x13, 0x32, 0x99, 0x8e, 0x09, 0x1d, 0x51, + 0x74, 0xb0, 0x11, 0xa3, 0x4e, 0x8d, 0x89, 0x14, 0x0d, 0x49, 0xad, 0x1a, + 0x6b, 0x29, 0x83, 0x75, 0xc6, 0x26, 0xbe, 0x2c, 0x48, 0x95, 0x42, 0x79, + 0x87, 0xbd, 0xbb, 0x77, 0xef, 0x7d, 0xfa, 0x81, 0xdd, 0x0d, 0x49, 0x51, + 0x61, 0x9a, 0xa4, 0xab, 0x3c, 0xbf, 0x99, 0xfd, 0xc4, 0xee, 0xb2, 0xf3, + 0x9c, 0xf3, 0xbb, 0xcf, 0x39, 0xe7, 0x9e, 0xf3, 0xdc, 0x92, 0x82, 0x82, + 0xae, 0x86, 0x53, 0xa7, 0xb8, 0x1b, 0xb3, 0xe8, 0x71, 0x80, 0xa2, 0x2c, + 0xd6, 0x6d, 0xbb, 0x2d, 0x08, 0x84, 0xe9, 0xcc, 0xc1, 0x6a, 0xa1, 0x6b, + 0x2a, 0x2f, 0xcc, 0x4d, 0xf6, 0x22, 0xea, 0xd2, 0xf1, 0xec, 0xe6, 0x1d, + 0x38, 0x72, 0xfc, 0x4c, 0xa4, 0x66, 0xbe, 0x84, 0x94, 0x33, 0x7e, 0xe5, + 0x76, 0x97, 0x6d, 0x1f, 0x1c, 0xdc, 0xe4, 0x78, 0x3c, 0x4b, 0x31, 0x34, + 0x74, 0x86, 0x03, 0xc5, 0xa2, 0xc7, 0x07, 0x29, 0x29, 0x8f, 0xa7, 0xf4, + 0xf4, 0x1c, 0xe8, 0x9e, 0x33, 0x3b, 0x8d, 0x0e, 0xbd, 0x5a, 0x29, 0x78, + 0x15, 0xfe, 0xe6, 0x19, 0xdc, 0xa5, 0x6b, 0x18, 0x1c, 0x0e, 0xe0, 0xf9, + 0xd7, 0x5e, 0xc7, 0xa1, 0x63, 0x7f, 0x8e, 0x54, 0xd6, 0x15, 0x90, 0xd2, + 0x57, 0x91, 0x9e, 0xfe, 0x46, 0x55, 0x67, 0x67, 0x91, 0xad, 0xaa, 0x59, + 0x08, 0x87, 0x3b, 0x38, 0x60, 0x2c, 0x7a, 0x9c, 0x05, 0x5a, 0x24, 0x43, + 0xd3, 0x7c, 0x45, 0xa1, 0xd0, 0xe5, 0xe3, 0x0f, 0x2d, 0x99, 0x4f, 0x3b, + 0xb6, 0x94, 0x09, 0xcb, 0x0a, 0xb3, 0xec, 0x11, 0xa4, 0x14, 0x30, 0x74, + 0x1d, 0x35, 0xaf, 0xbf, 0x85, 0x63, 0xef, 0x9e, 0x43, 0xcb, 0xdf, 0x3f, + 0x8c, 0x75, 0x51, 0x21, 0x12, 0x36, 0xac, 0x58, 0x31, 0x52, 0x55, 0x5b, + 0x2b, 0x78, 0x45, 0x93, 0x45, 0xbf, 0x53, 0x3a, 0xb4, 0x6f, 0x8d, 0xe3, + 0x0c, 0x6c, 0x2d, 0xfa, 0xca, 0x83, 0x54, 0x5d, 0xf1, 0x0b, 0x61, 0x85, + 0xed, 0x69, 0x2b, 0x3b, 0x11, 0x41, 0x91, 0x12, 0xc3, 0x01, 0x13, 0xaf, + 0xec, 0x3e, 0x8c, 0xba, 0x37, 0x9a, 0x80, 0x48, 0x15, 0xd6, 0x88, 0xe0, + 0x75, 0x86, 0x91, 0xb3, 0x36, 0x10, 0x68, 0xbb, 0xc6, 0x3d, 0x87, 0x45, + 0xbf, 0xf3, 0xe6, 0x9d, 0x46, 0x5e, 0xb1, 0x69, 0x5e, 0x7c, 0xfb, 0x07, + 0xdf, 0x29, 0xa4, 0xca, 0xb2, 0x52, 0x61, 0x85, 0xa7, 0x4f, 0x66, 0x8f, + 0x9e, 0x28, 0xd3, 0x75, 0x0d, 0x97, 0xdb, 0xaf, 0xe3, 0xc8, 0x89, 0xf7, + 0xf1, 0xca, 0xee, 0xc3, 0xe3, 0x04, 0xd7, 0xcf, 0x08, 0xa1, 0x35, 0x10, + 0x8d, 0x6c, 0x1b, 0xfb, 0xc4, 0x5c, 0xdc, 0xac, 0xb4, 0x13, 0xc3, 0xa2, + 0xc7, 0x3d, 0x8a, 0x92, 0xba, 0xc6, 0xb6, 0xff, 0xb5, 0x75, 0x7e, 0xee, + 0x1c, 0xaa, 0x28, 0x5b, 0x21, 0xe6, 0xe5, 0x66, 0x23, 0x74, 0x97, 0x0f, + 0xe5, 0x35, 0x55, 0xc1, 0x8d, 0x9e, 0x7e, 0x9c, 0x3c, 0x7b, 0x1e, 0x3b, + 0xea, 0xdf, 0xc4, 0xb5, 0xeb, 0xdd, 0xe3, 0x04, 0x97, 0x03, 0xaa, 0x9a, + 0x56, 0x1a, 0x0e, 0x5f, 0x3f, 0xc4, 0xbd, 0x83, 0x45, 0xbf, 0xab, 0x50, + 0xd5, 0xac, 0x1f, 0x85, 0xc3, 0x1d, 0x75, 0x00, 0xe8, 0xa9, 0xe5, 0xcb, + 0xc4, 0xda, 0xd5, 0x3f, 0xbc, 0xeb, 0xb6, 0xca, 0x2a, 0x52, 0x62, 0xd4, + 0x0c, 0xe2, 0xc2, 0x07, 0x7e, 0xbc, 0xb4, 0xf3, 0x60, 0x74, 0xee, 0x4d, + 0x1f, 0xf5, 0x3d, 0xd1, 0x21, 0x84, 0x77, 0x2f, 0xd1, 0x40, 0x39, 0xf7, + 0x08, 0x16, 0xfd, 0xee, 0xcd, 0x72, 0x5a, 0xf6, 0x63, 0x96, 0xd5, 0x7e, + 0x04, 0x00, 0xa5, 0xa7, 0x26, 0x89, 0xba, 0xed, 0xe5, 0x98, 0x95, 0xe4, + 0x83, 0x6d, 0xff, 0x77, 0x81, 0xc9, 0xe8, 0x45, 0x40, 0x0a, 0x01, 0xcb, + 0xb6, 0xe1, 0xd8, 0x0e, 0x74, 0x4d, 0x8d, 0xab, 0xa7, 0xb8, 0x46, 0x7f, + 0xa3, 0x10, 0xc0, 0x95, 0x8e, 0x2e, 0xbc, 0x5c, 0x7b, 0x10, 0x7f, 0x3c, + 0xd9, 0xf2, 0xc9, 0x6e, 0xd7, 0x27, 0xc4, 0x8c, 0xa6, 0xec, 0xec, 0xbf, + 0x3c, 0xed, 0xf7, 0x7f, 0x81, 0xcf, 0xf1, 0xb2, 0xe8, 0xd3, 0x25, 0xb3, + 0xdf, 0x77, 0x6f, 0x38, 0xec, 0xbf, 0x00, 0xd8, 0x06, 0x00, 0x14, 0x7f, + 0x6d, 0x09, 0xb6, 0x3c, 0xf7, 0x33, 0x28, 0x8a, 0x12, 0xdb, 0xf2, 0x29, + 0x84, 0x80, 0x19, 0x0c, 0xe1, 0xc2, 0x87, 0xed, 0xd8, 0xd5, 0xd0, 0x88, + 0x77, 0x4e, 0xb7, 0x46, 0x86, 0xc3, 0x2a, 0xde, 0xdc, 0xf9, 0x6b, 0xcc, + 0x4a, 0xf2, 0xfd, 0xdf, 0x47, 0x03, 0xb6, 0x6d, 0x23, 0x60, 0x86, 0xd0, + 0x74, 0xb2, 0x05, 0x95, 0x2f, 0xd5, 0x4d, 0x70, 0x6a, 0x4f, 0xf5, 0x03, + 0x99, 0x5f, 0x05, 0xae, 0x76, 0x6a, 0xda, 0x3c, 0x58, 0xd6, 0x05, 0x6e, + 0x7c, 0x16, 0x7d, 0xfa, 0x60, 0x18, 0x5f, 0xc2, 0xf2, 0xe5, 0xe7, 0x94, + 0xfd, 0xfb, 0x13, 0x87, 0x89, 0x02, 0x06, 0x00, 0x12, 0x02, 0xe2, 0xb7, + 0x55, 0xbf, 0xc4, 0xd2, 0xc5, 0xf9, 0x20, 0x22, 0x1c, 0x3a, 0x76, 0x0a, + 0x15, 0x2f, 0xee, 0x8a, 0x64, 0xcb, 0xb1, 0x97, 0xe3, 0x38, 0xb1, 0x0b, + 0xc1, 0xb3, 0x3f, 0x7f, 0x02, 0x4f, 0x7f, 0xef, 0x1b, 0xb1, 0xd3, 0x5c, + 0x9f, 0x97, 0xf0, 0x8e, 0x43, 0x08, 0xdb, 0x36, 0xb6, 0xd7, 0x34, 0x60, + 0xef, 0xe1, 0x13, 0x13, 0xbe, 0x05, 0xd0, 0xbb, 0xa5, 0x9c, 0xfd, 0xb0, + 0xe3, 0x5c, 0xb9, 0xc8, 0xad, 0xcd, 0xa2, 0x33, 0x00, 0x34, 0x2d, 0x73, + 0xbd, 0x65, 0x5d, 0xdb, 0x14, 0xcd, 0xe2, 0x8d, 0xbb, 0xb7, 0xa2, 0xea, + 0xe5, 0xbd, 0x78, 0xe7, 0x74, 0x2b, 0xf6, 0xed, 0xdb, 0x87, 0x92, 0x92, + 0x92, 0x8f, 0x1a, 0x4f, 0x08, 0x34, 0x37, 0x37, 0xa3, 0xa8, 0xa8, 0x08, + 0xc0, 0x58, 0xbd, 0xf2, 0x4d, 0xcf, 0xfc, 0x14, 0x8f, 0x16, 0x2e, 0xfe, + 0x4c, 0x64, 0x1f, 0xff, 0x9d, 0x9d, 0x37, 0x7a, 0xf0, 0xbb, 0xdf, 0xbf, + 0x85, 0x03, 0x6f, 0xff, 0x09, 0x13, 0x9f, 0xd7, 0xd1, 0x2f, 0x4a, 0x39, + 0x73, 0xed, 0xfd, 0xf7, 0xff, 0xf8, 0x0f, 0x57, 0xaf, 0xb6, 0x5a, 0xa1, + 0x50, 0x13, 0x16, 0x2d, 0x6a, 0x43, 0x6b, 0xeb, 0x02, 0x6e, 0x64, 0x16, + 0x7d, 0xba, 0x67, 0xf6, 0x2f, 0xc2, 0x34, 0xcf, 0x43, 0x51, 0x52, 0x5d, + 0x9a, 0x96, 0xf8, 0x75, 0xd3, 0xec, 0x3c, 0x06, 0x58, 0x40, 0x64, 0xf1, + 0x2a, 0x18, 0x0c, 0x42, 0xd3, 0xb4, 0x98, 0x6c, 0x51, 0xf1, 0x4c, 0xd3, + 0xc4, 0xaa, 0x55, 0xab, 0x50, 0x5b, 0x5b, 0x0b, 0x00, 0xf0, 0xb8, 0x13, + 0x71, 0xfa, 0xc0, 0x8b, 0xb0, 0xc2, 0xf6, 0xa7, 0x22, 0xb7, 0x14, 0x02, + 0x9a, 0xae, 0xe1, 0xaf, 0xe7, 0x3f, 0xc0, 0x73, 0x9b, 0x5f, 0x45, 0x57, + 0x4f, 0x1f, 0xec, 0x09, 0x8b, 0x69, 0xb8, 0xb7, 0x09, 0x21, 0x37, 0x4b, + 0xa9, 0x9a, 0xb6, 0xdd, 0x6b, 0x72, 0x8b, 0xb2, 0xe8, 0xcc, 0x2d, 0xf0, + 0x78, 0x1e, 0xc4, 0xd0, 0xd0, 0xdf, 0xa2, 0xfa, 0xbf, 0x00, 0x98, 0x65, + 0x95, 0x95, 0x95, 0xd8, 0xb0, 0xe1, 0xd6, 0x65, 0x90, 0x46, 0x46, 0x46, + 0xb0, 0x72, 0xe5, 0x4a, 0xec, 0xd9, 0xb3, 0x07, 0x52, 0x4a, 0x6c, 0xdf, + 0xb0, 0x1a, 0x0f, 0x2f, 0x5d, 0x38, 0xe9, 0x27, 0xc7, 0xc4, 0x16, 0xfb, + 0xa4, 0x84, 0x4b, 0xd7, 0xf0, 0xcf, 0xae, 0x7f, 0x63, 0xff, 0xd1, 0x77, + 0xd1, 0x70, 0xf4, 0x3d, 0xf4, 0x0d, 0x0c, 0x8f, 0xaf, 0x7f, 0x17, 0xdb, + 0xd4, 0xa2, 0xaa, 0x69, 0x4f, 0x86, 0xc3, 0xbd, 0x4d, 0x2e, 0x57, 0x56, + 0x20, 0x18, 0xbc, 0x14, 0xe0, 0xd6, 0x63, 0xd1, 0x99, 0x29, 0xe2, 0x72, + 0x7d, 0x19, 0xc1, 0x60, 0x4b, 0x2f, 0x60, 0xdd, 0x33, 0x30, 0x30, 0x00, + 0x8f, 0xc7, 0x73, 0x4b, 0x61, 0xa3, 0xa2, 0x36, 0x36, 0x36, 0xa2, 0xb8, + 0xb8, 0x18, 0x00, 0xb0, 0x66, 0xf5, 0x53, 0x28, 0x7d, 0xfc, 0x9b, 0xb1, + 0x79, 0xbd, 0x19, 0x0c, 0x61, 0xd4, 0x0c, 0xe2, 0xe8, 0x89, 0xb3, 0x48, + 0x30, 0x74, 0xd8, 0xb6, 0x8d, 0xf3, 0xff, 0xb8, 0x8a, 0xb4, 0x59, 0x49, + 0x38, 0xdb, 0x7a, 0x11, 0x7d, 0x03, 0x43, 0x18, 0x19, 0x35, 0x71, 0xa3, + 0xbb, 0x17, 0x56, 0xd8, 0x1e, 0x77, 0x1b, 0xcc, 0xd5, 0x0e, 0xc8, 0x5d, + 0x80, 0xad, 0x28, 0x8a, 0xe7, 0x12, 0x51, 0xe8, 0xf8, 0xdc, 0xb9, 0xdb, + 0xfc, 0x97, 0x2f, 0xaf, 0xe2, 0x3a, 0x59, 0x0c, 0xf3, 0xbf, 0xe0, 0xf5, + 0x3e, 0x9a, 0x05, 0xc0, 0x29, 0x2c, 0x2c, 0xa4, 0xc9, 0xe2, 0x38, 0x0e, + 0x39, 0x8e, 0x43, 0x8a, 0xa2, 0x10, 0x00, 0xd2, 0x34, 0x95, 0xee, 0xcb, + 0x4e, 0xa7, 0xf4, 0xd4, 0x24, 0x9a, 0x95, 0x3c, 0x93, 0x22, 0xd9, 0x78, + 0x12, 0x2f, 0xd5, 0x2f, 0xa5, 0xfb, 0x79, 0x20, 0x71, 0xa9, 0xa6, 0x3d, + 0xe0, 0xfb, 0xe4, 0x6f, 0x4b, 0x48, 0x78, 0x88, 0x1b, 0x88, 0x33, 0x3a, + 0xf3, 0xe9, 0x90, 0x70, 0x04, 0x08, 0x3c, 0xd6, 0xdf, 0xdf, 0x0f, 0x9f, + 0xcf, 0x37, 0xe9, 0x39, 0xb5, 0x10, 0x02, 0xaa, 0xaa, 0xc2, 0xb6, 0x6d, + 0x02, 0x84, 0x00, 0xb4, 0xe3, 0x80, 0x33, 0x1f, 0x40, 0x1a, 0x20, 0xdb, + 0x00, 0x9c, 0x02, 0x30, 0x04, 0x60, 0x09, 0x40, 0x06, 0xa0, 0x9c, 0x03, + 0x9c, 0x4b, 0x42, 0x28, 0x3d, 0x00, 0x06, 0x14, 0x25, 0xa5, 0x2d, 0x1c, + 0xee, 0xb8, 0xc2, 0xf1, 0x67, 0x98, 0xcf, 0xf2, 0xea, 0x2b, 0xd2, 0xe1, + 0x76, 0x2f, 0x50, 0x00, 0x41, 0x25, 0x25, 0x25, 0x0e, 0x4d, 0x91, 0x8d, + 0x1b, 0x37, 0xc6, 0x32, 0xb3, 0xa2, 0xcc, 0xc9, 0x9b, 0xca, 0xff, 0x5e, + 0xb8, 0x70, 0x27, 0x37, 0x00, 0xc3, 0x7c, 0x5e, 0x48, 0x79, 0xcf, 0x0b, + 0x00, 0x9c, 0xe8, 0x70, 0x7c, 0xb2, 0xf4, 0xf6, 0xf6, 0x12, 0x00, 0x07, + 0x00, 0x49, 0xe9, 0xdd, 0xc5, 0x91, 0x64, 0x98, 0x38, 0x45, 0xd7, 0xf3, + 0xb3, 0x00, 0x50, 0x45, 0x45, 0xc5, 0x94, 0x32, 0xf9, 0x78, 0xc9, 0x01, + 0xa3, 0x05, 0x18, 0xdb, 0x8c, 0xc3, 0x30, 0x4c, 0x1c, 0x91, 0x9f, 0xdf, + 0x11, 0x19, 0xba, 0x1b, 0xdd, 0xa9, 0xa9, 0xa9, 0x93, 0xce, 0xe6, 0xd1, + 0xf7, 0xb8, 0xdd, 0x6e, 0x07, 0x10, 0x04, 0x18, 0x5b, 0xc6, 0x46, 0x05, + 0x19, 0x1c, 0x54, 0x86, 0x89, 0x47, 0x14, 0x25, 0xe3, 0x11, 0x00, 0xd4, + 0xdf, 0xdf, 0x3f, 0xa5, 0x21, 0xfb, 0xfa, 0xf5, 0xeb, 0x1d, 0x00, 0xa4, + 0xeb, 0xf3, 0xbe, 0x3b, 0x63, 0xc6, 0x42, 0x0e, 0x24, 0xc3, 0xc4, 0x23, + 0x1e, 0xcf, 0xb7, 0xe0, 0xf5, 0x6e, 0x13, 0x80, 0x4a, 0xe9, 0xe9, 0xe9, + 0x53, 0x1a, 0xb2, 0x77, 0x77, 0x77, 0x13, 0x00, 0x12, 0x22, 0xf1, 0x3d, + 0x8e, 0x24, 0xc3, 0xc4, 0x39, 0x52, 0x7a, 0x7f, 0x02, 0xc0, 0xa9, 0xae, + 0xae, 0xfe, 0x58, 0x36, 0xbf, 0x5d, 0x66, 0x6f, 0x6e, 0x6e, 0x26, 0x00, + 0x64, 0x18, 0x79, 0x3c, 0x21, 0x67, 0x98, 0xf8, 0x17, 0xdd, 0x30, 0x53, + 0x52, 0x52, 0x62, 0x02, 0x97, 0x95, 0x95, 0x91, 0xdf, 0xef, 0xbf, 0xed, + 0xdc, 0x3c, 0x39, 0x39, 0xd9, 0x01, 0x04, 0x3f, 0xb9, 0x80, 0x61, 0xe2, + 0x5f, 0xf2, 0xa4, 0x05, 0x00, 0xa8, 0xa0, 0xa0, 0x80, 0x88, 0x88, 0x06, + 0x07, 0x07, 0x63, 0xf7, 0xc2, 0x6f, 0x95, 0xd1, 0xeb, 0xeb, 0xeb, 0xa3, + 0xc3, 0xf6, 0xdf, 0x70, 0x14, 0x19, 0x26, 0xee, 0x99, 0x79, 0xef, 0x78, + 0xd1, 0x2d, 0xcb, 0x8a, 0x89, 0x3e, 0x3a, 0x3a, 0x7a, 0x53, 0xd9, 0xbd, + 0x5e, 0x2f, 0x8d, 0xad, 0xb4, 0x33, 0x0c, 0x13, 0xf7, 0xcc, 0x9e, 0xbd, + 0x57, 0x03, 0x40, 0x19, 0x19, 0x19, 0x31, 0x89, 0x73, 0x73, 0x73, 0x09, + 0x00, 0x95, 0x96, 0x96, 0x4e, 0x28, 0x79, 0x6d, 0x6d, 0x6d, 0xe4, 0xbe, + 0xb9, 0xf1, 0x3e, 0x47, 0x90, 0x61, 0xee, 0x98, 0xe1, 0xfb, 0xcc, 0x9d, + 0x00, 0x9c, 0xfa, 0xfa, 0x7a, 0x72, 0x1c, 0x27, 0xba, 0x01, 0xe6, 0x63, + 0x59, 0x3d, 0x9a, 0xd9, 0x6b, 0x6a, 0x6a, 0x62, 0x7f, 0x53, 0xd5, 0xfc, + 0x45, 0x86, 0x31, 0x8f, 0x03, 0xc8, 0x30, 0xf1, 0x8e, 0xa6, 0x3d, 0x00, + 0x4d, 0x5b, 0xec, 0x03, 0x5c, 0x7d, 0x8a, 0x22, 0xa9, 0xbc, 0xbc, 0x9c, + 0x86, 0x87, 0x87, 0x29, 0x27, 0x27, 0x87, 0x00, 0xd0, 0xb2, 0x65, 0xcb, + 0xa8, 0xa1, 0xa1, 0x81, 0xd6, 0xad, 0x5b, 0x47, 0x79, 0x79, 0x79, 0x51, + 0xc9, 0x1d, 0x21, 0xb4, 0x61, 0x8e, 0x1e, 0xc3, 0xdc, 0x81, 0x08, 0xe1, + 0x79, 0x06, 0x50, 0x26, 0x71, 0x94, 0x54, 0x92, 0x94, 0x99, 0x73, 0x39, + 0x62, 0xcc, 0x6d, 0xfb, 0x14, 0x87, 0x20, 0xbe, 0x30, 0x8c, 0x02, 0x98, + 0xe6, 0x59, 0x00, 0x80, 0xa2, 0x24, 0x7d, 0xdb, 0x71, 0x02, 0x39, 0x80, + 0xe8, 0x02, 0x44, 0x3b, 0x40, 0xdf, 0x27, 0x0a, 0xe5, 0x02, 0xc8, 0x91, + 0xd2, 0x75, 0x30, 0x33, 0xf3, 0x4a, 0x55, 0x7b, 0x7b, 0x2a, 0x17, 0x7e, + 0x60, 0x6e, 0xcb, 0x7f, 0x00, 0x4a, 0x3f, 0xff, 0x3a, 0x92, 0xd3, 0x63, + 0x31, 0x00, 0x00, 0x00, 0x00, 0x49, 0x45, 0x4e, 0x44, 0xae, 0x42, 0x60, + 0x82, 0x01, 0x00, 0x00, 0xff, 0xff, 0x25, 0x50, 0x56, 0x5e, 0x8b, 0x55, + 0x00, 0x00, + })) + + if err != nil { + panic("Decompression failed: " + err.Error()) + } + + var b bytes.Buffer + io.Copy(&b, gz) + gz.Close() + + return b.Bytes() +} diff --git a/third_party/github.com/jteeuwen/go-bindata/testdata/memcpy-uncompressed.go b/third_party/github.com/jteeuwen/go-bindata/testdata/memcpy-uncompressed.go new file mode 100644 index 00000000000..1b156c2122e --- /dev/null +++ b/third_party/github.com/jteeuwen/go-bindata/testdata/memcpy-uncompressed.go @@ -0,0 +1,1832 @@ +package main + +// gophercolor_png returns the raw file data data. +func gophercolor_png() []byte { + return []byte{ + 0x89, 0x50, 0x4e, 0x47, 0x0d, 0x0a, 0x1a, 0x0a, 0x00, 0x00, 0x00, 0x0d, + 0x49, 0x48, 0x44, 0x52, 0x00, 0x00, 0x00, 0xfa, 0x00, 0x00, 0x00, 0xfa, + 0x08, 0x06, 0x00, 0x00, 0x00, 0x88, 0xec, 0x5a, 0x3d, 0x00, 0x00, 0x0a, + 0x4f, 0x69, 0x43, 0x43, 0x50, 0x50, 0x68, 0x6f, 0x74, 0x6f, 0x73, 0x68, + 0x6f, 0x70, 0x20, 0x49, 0x43, 0x43, 0x20, 0x70, 0x72, 0x6f, 0x66, 0x69, + 0x6c, 0x65, 0x00, 0x00, 0x78, 0xda, 0x9d, 0x53, 0x67, 0x54, 0x53, 0xe9, + 0x16, 0x3d, 0xf7, 0xde, 0xf4, 0x42, 0x4b, 0x88, 0x80, 0x94, 0x4b, 0x6f, + 0x52, 0x15, 0x08, 0x20, 0x52, 0x42, 0x8b, 0x80, 0x14, 0x91, 0x26, 0x2a, + 0x21, 0x09, 0x10, 0x4a, 0x88, 0x21, 0xa1, 0xd9, 0x15, 0x51, 0xc1, 0x11, + 0x45, 0x45, 0x04, 0x1b, 0xc8, 0xa0, 0x88, 0x03, 0x8e, 0x8e, 0x80, 0x8c, + 0x15, 0x51, 0x2c, 0x0c, 0x8a, 0x0a, 0xd8, 0x07, 0xe4, 0x21, 0xa2, 0x8e, + 0x83, 0xa3, 0x88, 0x8a, 0xca, 0xfb, 0xe1, 0x7b, 0xa3, 0x6b, 0xd6, 0xbc, + 0xf7, 0xe6, 0xcd, 0xfe, 0xb5, 0xd7, 0x3e, 0xe7, 0xac, 0xf3, 0x9d, 0xb3, + 0xcf, 0x07, 0xc0, 0x08, 0x0c, 0x96, 0x48, 0x33, 0x51, 0x35, 0x80, 0x0c, + 0xa9, 0x42, 0x1e, 0x11, 0xe0, 0x83, 0xc7, 0xc4, 0xc6, 0xe1, 0xe4, 0x2e, + 0x40, 0x81, 0x0a, 0x24, 0x70, 0x00, 0x10, 0x08, 0xb3, 0x64, 0x21, 0x73, + 0xfd, 0x23, 0x01, 0x00, 0xf8, 0x7e, 0x3c, 0x3c, 0x2b, 0x22, 0xc0, 0x07, + 0xbe, 0x00, 0x01, 0x78, 0xd3, 0x0b, 0x08, 0x00, 0xc0, 0x4d, 0x9b, 0xc0, + 0x30, 0x1c, 0x87, 0xff, 0x0f, 0xea, 0x42, 0x99, 0x5c, 0x01, 0x80, 0x84, + 0x01, 0xc0, 0x74, 0x91, 0x38, 0x4b, 0x08, 0x80, 0x14, 0x00, 0x40, 0x7a, + 0x8e, 0x42, 0xa6, 0x00, 0x40, 0x46, 0x01, 0x80, 0x9d, 0x98, 0x26, 0x53, + 0x00, 0xa0, 0x04, 0x00, 0x60, 0xcb, 0x63, 0x62, 0xe3, 0x00, 0x50, 0x2d, + 0x00, 0x60, 0x27, 0x7f, 0xe6, 0xd3, 0x00, 0x80, 0x9d, 0xf8, 0x99, 0x7b, + 0x01, 0x00, 0x5b, 0x94, 0x21, 0x15, 0x01, 0xa0, 0x91, 0x00, 0x20, 0x13, + 0x65, 0x88, 0x44, 0x00, 0x68, 0x3b, 0x00, 0xac, 0xcf, 0x56, 0x8a, 0x45, + 0x00, 0x58, 0x30, 0x00, 0x14, 0x66, 0x4b, 0xc4, 0x39, 0x00, 0xd8, 0x2d, + 0x00, 0x30, 0x49, 0x57, 0x66, 0x48, 0x00, 0xb0, 0xb7, 0x00, 0xc0, 0xce, + 0x10, 0x0b, 0xb2, 0x00, 0x08, 0x0c, 0x00, 0x30, 0x51, 0x88, 0x85, 0x29, + 0x00, 0x04, 0x7b, 0x00, 0x60, 0xc8, 0x23, 0x23, 0x78, 0x00, 0x84, 0x99, + 0x00, 0x14, 0x46, 0xf2, 0x57, 0x3c, 0xf1, 0x2b, 0xae, 0x10, 0xe7, 0x2a, + 0x00, 0x00, 0x78, 0x99, 0xb2, 0x3c, 0xb9, 0x24, 0x39, 0x45, 0x81, 0x5b, + 0x08, 0x2d, 0x71, 0x07, 0x57, 0x57, 0x2e, 0x1e, 0x28, 0xce, 0x49, 0x17, + 0x2b, 0x14, 0x36, 0x61, 0x02, 0x61, 0x9a, 0x40, 0x2e, 0xc2, 0x79, 0x99, + 0x19, 0x32, 0x81, 0x34, 0x0f, 0xe0, 0xf3, 0xcc, 0x00, 0x00, 0xa0, 0x91, + 0x15, 0x11, 0xe0, 0x83, 0xf3, 0xfd, 0x78, 0xce, 0x0e, 0xae, 0xce, 0xce, + 0x36, 0x8e, 0xb6, 0x0e, 0x5f, 0x2d, 0xea, 0xbf, 0x06, 0xff, 0x22, 0x62, + 0x62, 0xe3, 0xfe, 0xe5, 0xcf, 0xab, 0x70, 0x40, 0x00, 0x00, 0xe1, 0x74, + 0x7e, 0xd1, 0xfe, 0x2c, 0x2f, 0xb3, 0x1a, 0x80, 0x3b, 0x06, 0x80, 0x6d, + 0xfe, 0xa2, 0x25, 0xee, 0x04, 0x68, 0x5e, 0x0b, 0xa0, 0x75, 0xf7, 0x8b, + 0x66, 0xb2, 0x0f, 0x40, 0xb5, 0x00, 0xa0, 0xe9, 0xda, 0x57, 0xf3, 0x70, + 0xf8, 0x7e, 0x3c, 0x3c, 0x45, 0xa1, 0x90, 0xb9, 0xd9, 0xd9, 0xe5, 0xe4, + 0xe4, 0xd8, 0x4a, 0xc4, 0x42, 0x5b, 0x61, 0xca, 0x57, 0x7d, 0xfe, 0x67, + 0xc2, 0x5f, 0xc0, 0x57, 0xfd, 0x6c, 0xf9, 0x7e, 0x3c, 0xfc, 0xf7, 0xf5, + 0xe0, 0xbe, 0xe2, 0x24, 0x81, 0x32, 0x5d, 0x81, 0x47, 0x04, 0xf8, 0xe0, + 0xc2, 0xcc, 0xf4, 0x4c, 0xa5, 0x1c, 0xcf, 0x92, 0x09, 0x84, 0x62, 0xdc, + 0xe6, 0x8f, 0x47, 0xfc, 0xb7, 0x0b, 0xff, 0xfc, 0x1d, 0xd3, 0x22, 0xc4, + 0x49, 0x62, 0xb9, 0x58, 0x2a, 0x14, 0xe3, 0x51, 0x12, 0x71, 0x8e, 0x44, + 0x9a, 0x8c, 0xf3, 0x32, 0xa5, 0x22, 0x89, 0x42, 0x92, 0x29, 0xc5, 0x25, + 0xd2, 0xff, 0x64, 0xe2, 0xdf, 0x2c, 0xfb, 0x03, 0x3e, 0xdf, 0x35, 0x00, + 0xb0, 0x6a, 0x3e, 0x01, 0x7b, 0x91, 0x2d, 0xa8, 0x5d, 0x63, 0x03, 0xf6, + 0x4b, 0x27, 0x10, 0x58, 0x74, 0xc0, 0xe2, 0xf7, 0x00, 0x00, 0xf2, 0xbb, + 0x6f, 0xc1, 0xd4, 0x28, 0x08, 0x03, 0x80, 0x68, 0x83, 0xe1, 0xcf, 0x77, + 0xff, 0xef, 0x3f, 0xfd, 0x47, 0xa0, 0x25, 0x00, 0x80, 0x66, 0x49, 0x92, + 0x71, 0x00, 0x00, 0x5e, 0x44, 0x24, 0x2e, 0x54, 0xca, 0xb3, 0x3f, 0xc7, + 0x08, 0x00, 0x00, 0x44, 0xa0, 0x81, 0x2a, 0xb0, 0x41, 0x1b, 0xf4, 0xc1, + 0x18, 0x2c, 0xc0, 0x06, 0x1c, 0xc1, 0x05, 0xdc, 0xc1, 0x0b, 0xfc, 0x60, + 0x36, 0x84, 0x42, 0x24, 0xc4, 0xc2, 0x42, 0x10, 0x42, 0x0a, 0x64, 0x80, + 0x1c, 0x72, 0x60, 0x29, 0xac, 0x82, 0x42, 0x28, 0x86, 0xcd, 0xb0, 0x1d, + 0x2a, 0x60, 0x2f, 0xd4, 0x40, 0x1d, 0x34, 0xc0, 0x51, 0x68, 0x86, 0x93, + 0x70, 0x0e, 0x2e, 0xc2, 0x55, 0xb8, 0x0e, 0x3d, 0x70, 0x0f, 0xfa, 0x61, + 0x08, 0x9e, 0xc1, 0x28, 0xbc, 0x81, 0x09, 0x04, 0x41, 0xc8, 0x08, 0x13, + 0x61, 0x21, 0xda, 0x88, 0x01, 0x62, 0x8a, 0x58, 0x23, 0x8e, 0x08, 0x17, + 0x99, 0x85, 0xf8, 0x21, 0xc1, 0x48, 0x04, 0x12, 0x8b, 0x24, 0x20, 0xc9, + 0x88, 0x14, 0x51, 0x22, 0x4b, 0x91, 0x35, 0x48, 0x31, 0x52, 0x8a, 0x54, + 0x20, 0x55, 0x48, 0x1d, 0xf2, 0x3d, 0x72, 0x02, 0x39, 0x87, 0x5c, 0x46, + 0xba, 0x91, 0x3b, 0xc8, 0x00, 0x32, 0x82, 0xfc, 0x86, 0xbc, 0x47, 0x31, + 0x94, 0x81, 0xb2, 0x51, 0x3d, 0xd4, 0x0c, 0xb5, 0x43, 0xb9, 0xa8, 0x37, + 0x1a, 0x84, 0x46, 0xa2, 0x0b, 0xd0, 0x64, 0x74, 0x31, 0x9a, 0x8f, 0x16, + 0xa0, 0x9b, 0xd0, 0x72, 0xb4, 0x1a, 0x3d, 0x8c, 0x36, 0xa1, 0xe7, 0xd0, + 0xab, 0x68, 0x0f, 0xda, 0x8f, 0x3e, 0x43, 0xc7, 0x30, 0xc0, 0xe8, 0x18, + 0x07, 0x33, 0xc4, 0x6c, 0x30, 0x2e, 0xc6, 0xc3, 0x42, 0xb1, 0x38, 0x2c, + 0x09, 0x93, 0x63, 0xcb, 0xb1, 0x22, 0xac, 0x0c, 0xab, 0xc6, 0x1a, 0xb0, + 0x56, 0xac, 0x03, 0xbb, 0x89, 0xf5, 0x63, 0xcf, 0xb1, 0x77, 0x04, 0x12, + 0x81, 0x45, 0xc0, 0x09, 0x36, 0x04, 0x77, 0x42, 0x20, 0x61, 0x1e, 0x41, + 0x48, 0x58, 0x4c, 0x58, 0x4e, 0xd8, 0x48, 0xa8, 0x20, 0x1c, 0x24, 0x34, + 0x11, 0xda, 0x09, 0x37, 0x09, 0x03, 0x84, 0x51, 0xc2, 0x27, 0x22, 0x93, + 0xa8, 0x4b, 0xb4, 0x26, 0xba, 0x11, 0xf9, 0xc4, 0x18, 0x62, 0x32, 0x31, + 0x87, 0x58, 0x48, 0x2c, 0x23, 0xd6, 0x12, 0x8f, 0x13, 0x2f, 0x10, 0x7b, + 0x88, 0x43, 0xc4, 0x37, 0x24, 0x12, 0x89, 0x43, 0x32, 0x27, 0xb9, 0x90, + 0x02, 0x49, 0xb1, 0xa4, 0x54, 0xd2, 0x12, 0xd2, 0x46, 0xd2, 0x6e, 0x52, + 0x23, 0xe9, 0x2c, 0xa9, 0x9b, 0x34, 0x48, 0x1a, 0x23, 0x93, 0xc9, 0xda, + 0x64, 0x6b, 0xb2, 0x07, 0x39, 0x94, 0x2c, 0x20, 0x2b, 0xc8, 0x85, 0xe4, + 0x9d, 0xe4, 0xc3, 0xe4, 0x33, 0xe4, 0x1b, 0xe4, 0x21, 0xf2, 0x5b, 0x0a, + 0x9d, 0x62, 0x40, 0x71, 0xa4, 0xf8, 0x53, 0xe2, 0x28, 0x52, 0xca, 0x6a, + 0x4a, 0x19, 0xe5, 0x10, 0xe5, 0x34, 0xe5, 0x06, 0x65, 0x98, 0x32, 0x41, + 0x55, 0xa3, 0x9a, 0x52, 0xdd, 0xa8, 0xa1, 0x54, 0x11, 0x35, 0x8f, 0x5a, + 0x42, 0xad, 0xa1, 0xb6, 0x52, 0xaf, 0x51, 0x87, 0xa8, 0x13, 0x34, 0x75, + 0x9a, 0x39, 0xcd, 0x83, 0x16, 0x49, 0x4b, 0xa5, 0xad, 0xa2, 0x95, 0xd3, + 0x1a, 0x68, 0x17, 0x68, 0xf7, 0x69, 0xaf, 0xe8, 0x74, 0xba, 0x11, 0xdd, + 0x95, 0x1e, 0x4e, 0x97, 0xd0, 0x57, 0xd2, 0xcb, 0xe9, 0x47, 0xe8, 0x97, + 0xe8, 0x03, 0xf4, 0x77, 0x0c, 0x0d, 0x86, 0x15, 0x83, 0xc7, 0x88, 0x67, + 0x28, 0x19, 0x9b, 0x18, 0x07, 0x18, 0x67, 0x19, 0x77, 0x18, 0xaf, 0x98, + 0x4c, 0xa6, 0x19, 0xd3, 0x8b, 0x19, 0xc7, 0x54, 0x30, 0x37, 0x31, 0xeb, + 0x98, 0xe7, 0x99, 0x0f, 0x99, 0x6f, 0x55, 0x58, 0x2a, 0xb6, 0x2a, 0x7c, + 0x15, 0x91, 0xca, 0x0a, 0x95, 0x4a, 0x95, 0x26, 0x95, 0x1b, 0x2a, 0x2f, + 0x54, 0xa9, 0xaa, 0xa6, 0xaa, 0xde, 0xaa, 0x0b, 0x55, 0xf3, 0x55, 0xcb, + 0x54, 0x8f, 0xa9, 0x5e, 0x53, 0x7d, 0xae, 0x46, 0x55, 0x33, 0x53, 0xe3, + 0xa9, 0x09, 0xd4, 0x96, 0xab, 0x55, 0xaa, 0x9d, 0x50, 0xeb, 0x53, 0x1b, + 0x53, 0x67, 0xa9, 0x3b, 0xa8, 0x87, 0xaa, 0x67, 0xa8, 0x6f, 0x54, 0x3f, + 0xa4, 0x7e, 0x59, 0xfd, 0x89, 0x06, 0x59, 0xc3, 0x4c, 0xc3, 0x4f, 0x43, + 0xa4, 0x51, 0xa0, 0xb1, 0x5f, 0xe3, 0xbc, 0xc6, 0x20, 0x0b, 0x63, 0x19, + 0xb3, 0x78, 0x2c, 0x21, 0x6b, 0x0d, 0xab, 0x86, 0x75, 0x81, 0x35, 0xc4, + 0x26, 0xb1, 0xcd, 0xd9, 0x7c, 0x76, 0x2a, 0xbb, 0x98, 0xfd, 0x1d, 0xbb, + 0x8b, 0x3d, 0xaa, 0xa9, 0xa1, 0x39, 0x43, 0x33, 0x4a, 0x33, 0x57, 0xb3, + 0x52, 0xf3, 0x94, 0x66, 0x3f, 0x07, 0xe3, 0x98, 0x71, 0xf8, 0x9c, 0x74, + 0x4e, 0x09, 0xe7, 0x28, 0xa7, 0x97, 0xf3, 0x7e, 0x8a, 0xde, 0x14, 0xef, + 0x29, 0xe2, 0x29, 0x1b, 0xa6, 0x34, 0x4c, 0xb9, 0x31, 0x65, 0x5c, 0x6b, + 0xaa, 0x96, 0x97, 0x96, 0x58, 0xab, 0x48, 0xab, 0x51, 0xab, 0x47, 0xeb, + 0xbd, 0x36, 0xae, 0xed, 0xa7, 0x9d, 0xa6, 0xbd, 0x45, 0xbb, 0x59, 0xfb, + 0x81, 0x0e, 0x41, 0xc7, 0x4a, 0x27, 0x5c, 0x27, 0x47, 0x67, 0x8f, 0xce, + 0x05, 0x9d, 0xe7, 0x53, 0xd9, 0x53, 0xdd, 0xa7, 0x0a, 0xa7, 0x16, 0x4d, + 0x3d, 0x3a, 0xf5, 0xae, 0x2e, 0xaa, 0x6b, 0xa5, 0x1b, 0xa1, 0xbb, 0x44, + 0x77, 0xbf, 0x6e, 0xa7, 0xee, 0x98, 0x9e, 0xbe, 0x5e, 0x80, 0x9e, 0x4c, + 0x6f, 0xa7, 0xde, 0x79, 0xbd, 0xe7, 0xfa, 0x1c, 0x7d, 0x2f, 0xfd, 0x54, + 0xfd, 0x6d, 0xfa, 0xa7, 0xf5, 0x47, 0x0c, 0x58, 0x06, 0xb3, 0x0c, 0x24, + 0x06, 0xdb, 0x0c, 0xce, 0x18, 0x3c, 0xc5, 0x35, 0x71, 0x6f, 0x3c, 0x1d, + 0x2f, 0xc7, 0xdb, 0xf1, 0x51, 0x43, 0x5d, 0xc3, 0x40, 0x43, 0xa5, 0x61, + 0x95, 0x61, 0x97, 0xe1, 0x84, 0x91, 0xb9, 0xd1, 0x3c, 0xa3, 0xd5, 0x46, + 0x8d, 0x46, 0x0f, 0x8c, 0x69, 0xc6, 0x5c, 0xe3, 0x24, 0xe3, 0x6d, 0xc6, + 0x6d, 0xc6, 0xa3, 0x26, 0x06, 0x26, 0x21, 0x26, 0x4b, 0x4d, 0xea, 0x4d, + 0xee, 0x9a, 0x52, 0x4d, 0xb9, 0xa6, 0x29, 0xa6, 0x3b, 0x4c, 0x3b, 0x4c, + 0xc7, 0xcd, 0xcc, 0xcd, 0xa2, 0xcd, 0xd6, 0x99, 0x35, 0x9b, 0x3d, 0x31, + 0xd7, 0x32, 0xe7, 0x9b, 0xe7, 0x9b, 0xd7, 0x9b, 0xdf, 0xb7, 0x60, 0x5a, + 0x78, 0x5a, 0x2c, 0xb6, 0xa8, 0xb6, 0xb8, 0x65, 0x49, 0xb2, 0xe4, 0x5a, + 0xa6, 0x59, 0xee, 0xb6, 0xbc, 0x6e, 0x85, 0x5a, 0x39, 0x59, 0xa5, 0x58, + 0x55, 0x5a, 0x5d, 0xb3, 0x46, 0xad, 0x9d, 0xad, 0x25, 0xd6, 0xbb, 0xad, + 0xbb, 0xa7, 0x11, 0xa7, 0xb9, 0x4e, 0x93, 0x4e, 0xab, 0x9e, 0xd6, 0x67, + 0xc3, 0xb0, 0xf1, 0xb6, 0xc9, 0xb6, 0xa9, 0xb7, 0x19, 0xb0, 0xe5, 0xd8, + 0x06, 0xdb, 0xae, 0xb6, 0x6d, 0xb6, 0x7d, 0x61, 0x67, 0x62, 0x17, 0x67, + 0xb7, 0xc5, 0xae, 0xc3, 0xee, 0x93, 0xbd, 0x93, 0x7d, 0xba, 0x7d, 0x8d, + 0xfd, 0x3d, 0x07, 0x0d, 0x87, 0xd9, 0x0e, 0xab, 0x1d, 0x5a, 0x1d, 0x7e, + 0x73, 0xb4, 0x72, 0x14, 0x3a, 0x56, 0x3a, 0xde, 0x9a, 0xce, 0x9c, 0xee, + 0x3f, 0x7d, 0xc5, 0xf4, 0x96, 0xe9, 0x2f, 0x67, 0x58, 0xcf, 0x10, 0xcf, + 0xd8, 0x33, 0xe3, 0xb6, 0x13, 0xcb, 0x29, 0xc4, 0x69, 0x9d, 0x53, 0x9b, + 0xd3, 0x47, 0x67, 0x17, 0x67, 0xb9, 0x73, 0x83, 0xf3, 0x88, 0x8b, 0x89, + 0x4b, 0x82, 0xcb, 0x2e, 0x97, 0x3e, 0x2e, 0x9b, 0x1b, 0xc6, 0xdd, 0xc8, + 0xbd, 0xe4, 0x4a, 0x74, 0xf5, 0x71, 0x5d, 0xe1, 0x7a, 0xd2, 0xf5, 0x9d, + 0x9b, 0xb3, 0x9b, 0xc2, 0xed, 0xa8, 0xdb, 0xaf, 0xee, 0x36, 0xee, 0x69, + 0xee, 0x87, 0xdc, 0x9f, 0xcc, 0x34, 0x9f, 0x29, 0x9e, 0x59, 0x33, 0x73, + 0xd0, 0xc3, 0xc8, 0x43, 0xe0, 0x51, 0xe5, 0xd1, 0x3f, 0x0b, 0x9f, 0x95, + 0x30, 0x6b, 0xdf, 0xac, 0x7e, 0x4f, 0x43, 0x4f, 0x81, 0x67, 0xb5, 0xe7, + 0x23, 0x2f, 0x63, 0x2f, 0x91, 0x57, 0xad, 0xd7, 0xb0, 0xb7, 0xa5, 0x77, + 0xaa, 0xf7, 0x61, 0xef, 0x17, 0x3e, 0xf6, 0x3e, 0x72, 0x9f, 0xe3, 0x3e, + 0xe3, 0x3c, 0x37, 0xde, 0x32, 0xde, 0x59, 0x5f, 0xcc, 0x37, 0xc0, 0xb7, + 0xc8, 0xb7, 0xcb, 0x4f, 0xc3, 0x6f, 0x9e, 0x5f, 0x85, 0xdf, 0x43, 0x7f, + 0x23, 0xff, 0x64, 0xff, 0x7a, 0xff, 0xd1, 0x00, 0xa7, 0x80, 0x25, 0x01, + 0x67, 0x03, 0x89, 0x81, 0x41, 0x81, 0x5b, 0x02, 0xfb, 0xf8, 0x7a, 0x7c, + 0x21, 0xbf, 0x8e, 0x3f, 0x3a, 0xdb, 0x65, 0xf6, 0xb2, 0xd9, 0xed, 0x41, + 0x8c, 0xa0, 0xb9, 0x41, 0x15, 0x41, 0x8f, 0x82, 0xad, 0x82, 0xe5, 0xc1, + 0xad, 0x21, 0x68, 0xc8, 0xec, 0x90, 0xad, 0x21, 0xf7, 0xe7, 0x98, 0xce, + 0x91, 0xce, 0x69, 0x0e, 0x85, 0x50, 0x7e, 0xe8, 0xd6, 0xd0, 0x07, 0x61, + 0xe6, 0x61, 0x8b, 0xc3, 0x7e, 0x0c, 0x27, 0x85, 0x87, 0x85, 0x57, 0x86, + 0x3f, 0x8e, 0x70, 0x88, 0x58, 0x1a, 0xd1, 0x31, 0x97, 0x35, 0x77, 0xd1, + 0xdc, 0x43, 0x73, 0xdf, 0x44, 0xfa, 0x44, 0x96, 0x44, 0xde, 0x9b, 0x67, + 0x31, 0x4f, 0x39, 0xaf, 0x2d, 0x4a, 0x35, 0x2a, 0x3e, 0xaa, 0x2e, 0x6a, + 0x3c, 0xda, 0x37, 0xba, 0x34, 0xba, 0x3f, 0xc6, 0x2e, 0x66, 0x59, 0xcc, + 0xd5, 0x58, 0x9d, 0x58, 0x49, 0x6c, 0x4b, 0x1c, 0x39, 0x2e, 0x2a, 0xae, + 0x36, 0x6e, 0x6c, 0xbe, 0xdf, 0xfc, 0xed, 0xf3, 0x87, 0xe2, 0x9d, 0xe2, + 0x0b, 0xe3, 0x7b, 0x17, 0x98, 0x2f, 0xc8, 0x5d, 0x70, 0x79, 0xa1, 0xce, + 0xc2, 0xf4, 0x85, 0xa7, 0x16, 0xa9, 0x2e, 0x12, 0x2c, 0x3a, 0x96, 0x40, + 0x4c, 0x88, 0x4e, 0x38, 0x94, 0xf0, 0x41, 0x10, 0x2a, 0xa8, 0x16, 0x8c, + 0x25, 0xf2, 0x13, 0x77, 0x25, 0x8e, 0x0a, 0x79, 0xc2, 0x1d, 0xc2, 0x67, + 0x22, 0x2f, 0xd1, 0x36, 0xd1, 0x88, 0xd8, 0x43, 0x5c, 0x2a, 0x1e, 0x4e, + 0xf2, 0x48, 0x2a, 0x4d, 0x7a, 0x92, 0xec, 0x91, 0xbc, 0x35, 0x79, 0x24, + 0xc5, 0x33, 0xa5, 0x2c, 0xe5, 0xb9, 0x84, 0x27, 0xa9, 0x90, 0xbc, 0x4c, + 0x0d, 0x4c, 0xdd, 0x9b, 0x3a, 0x9e, 0x16, 0x9a, 0x76, 0x20, 0x6d, 0x32, + 0x3d, 0x3a, 0xbd, 0x31, 0x83, 0x92, 0x91, 0x90, 0x71, 0x42, 0xaa, 0x21, + 0x4d, 0x93, 0xb6, 0x67, 0xea, 0x67, 0xe6, 0x66, 0x76, 0xcb, 0xac, 0x65, + 0x85, 0xb2, 0xfe, 0xc5, 0x6e, 0x8b, 0xb7, 0x2f, 0x1e, 0x95, 0x07, 0xc9, + 0x6b, 0xb3, 0x90, 0xac, 0x05, 0x59, 0x2d, 0x0a, 0xb6, 0x42, 0xa6, 0xe8, + 0x54, 0x5a, 0x28, 0xd7, 0x2a, 0x07, 0xb2, 0x67, 0x65, 0x57, 0x66, 0xbf, + 0xcd, 0x89, 0xca, 0x39, 0x96, 0xab, 0x9e, 0x2b, 0xcd, 0xed, 0xcc, 0xb3, + 0xca, 0xdb, 0x90, 0x37, 0x9c, 0xef, 0x9f, 0xff, 0xed, 0x12, 0xc2, 0x12, + 0xe1, 0x92, 0xb6, 0xa5, 0x86, 0x4b, 0x57, 0x2d, 0x1d, 0x58, 0xe6, 0xbd, + 0xac, 0x6a, 0x39, 0xb2, 0x3c, 0x71, 0x79, 0xdb, 0x0a, 0xe3, 0x15, 0x05, + 0x2b, 0x86, 0x56, 0x06, 0xac, 0x3c, 0xb8, 0x8a, 0xb6, 0x2a, 0x6d, 0xd5, + 0x4f, 0xab, 0xed, 0x57, 0x97, 0xae, 0x7e, 0xbd, 0x26, 0x7a, 0x4d, 0x6b, + 0x81, 0x5e, 0xc1, 0xca, 0x82, 0xc1, 0xb5, 0x01, 0x6b, 0xeb, 0x0b, 0x55, + 0x0a, 0xe5, 0x85, 0x7d, 0xeb, 0xdc, 0xd7, 0xed, 0x5d, 0x4f, 0x58, 0x2f, + 0x59, 0xdf, 0xb5, 0x61, 0xfa, 0x86, 0x9d, 0x1b, 0x3e, 0x15, 0x89, 0x8a, + 0xae, 0x14, 0xdb, 0x17, 0x97, 0x15, 0x7f, 0xd8, 0x28, 0xdc, 0x78, 0xe5, + 0x1b, 0x87, 0x6f, 0xca, 0xbf, 0x99, 0xdc, 0x94, 0xb4, 0xa9, 0xab, 0xc4, + 0xb9, 0x64, 0xcf, 0x66, 0xd2, 0x66, 0xe9, 0xe6, 0xde, 0x2d, 0x9e, 0x5b, + 0x0e, 0x96, 0xaa, 0x97, 0xe6, 0x97, 0x0e, 0x6e, 0x0d, 0xd9, 0xda, 0xb4, + 0x0d, 0xdf, 0x56, 0xb4, 0xed, 0xf5, 0xf6, 0x45, 0xdb, 0x2f, 0x97, 0xcd, + 0x28, 0xdb, 0xbb, 0x83, 0xb6, 0x43, 0xb9, 0xa3, 0xbf, 0x3c, 0xb8, 0xbc, + 0x65, 0xa7, 0xc9, 0xce, 0xcd, 0x3b, 0x3f, 0x54, 0xa4, 0x54, 0xf4, 0x54, + 0xfa, 0x54, 0x36, 0xee, 0xd2, 0xdd, 0xb5, 0x61, 0xd7, 0xf8, 0x6e, 0xd1, + 0xee, 0x1b, 0x7b, 0xbc, 0xf6, 0x34, 0xec, 0xd5, 0xdb, 0x5b, 0xbc, 0xf7, + 0xfd, 0x3e, 0xc9, 0xbe, 0xdb, 0x55, 0x01, 0x55, 0x4d, 0xd5, 0x66, 0xd5, + 0x65, 0xfb, 0x49, 0xfb, 0xb3, 0xf7, 0x3f, 0xae, 0x89, 0xaa, 0xe9, 0xf8, + 0x96, 0xfb, 0x6d, 0x5d, 0xad, 0x4e, 0x6d, 0x71, 0xed, 0xc7, 0x03, 0xd2, + 0x03, 0xfd, 0x07, 0x23, 0x0e, 0xb6, 0xd7, 0xb9, 0xd4, 0xd5, 0x1d, 0xd2, + 0x3d, 0x54, 0x52, 0x8f, 0xd6, 0x2b, 0xeb, 0x47, 0x0e, 0xc7, 0x1f, 0xbe, + 0xfe, 0x9d, 0xef, 0x77, 0x2d, 0x0d, 0x36, 0x0d, 0x55, 0x8d, 0x9c, 0xc6, + 0xe2, 0x23, 0x70, 0x44, 0x79, 0xe4, 0xe9, 0xf7, 0x09, 0xdf, 0xf7, 0x1e, + 0x0d, 0x3a, 0xda, 0x76, 0x8c, 0x7b, 0xac, 0xe1, 0x07, 0xd3, 0x1f, 0x76, + 0x1d, 0x67, 0x1d, 0x2f, 0x6a, 0x42, 0x9a, 0xf2, 0x9a, 0x46, 0x9b, 0x53, + 0x9a, 0xfb, 0x5b, 0x62, 0x5b, 0xba, 0x4f, 0xcc, 0x3e, 0xd1, 0xd6, 0xea, + 0xde, 0x7a, 0xfc, 0x47, 0xdb, 0x1f, 0x0f, 0x9c, 0x34, 0x3c, 0x59, 0x79, + 0x4a, 0xf3, 0x54, 0xc9, 0x69, 0xda, 0xe9, 0x82, 0xd3, 0x93, 0x67, 0xf2, + 0xcf, 0x8c, 0x9d, 0x95, 0x9d, 0x7d, 0x7e, 0x2e, 0xf9, 0xdc, 0x60, 0xdb, + 0xa2, 0xb6, 0x7b, 0xe7, 0x63, 0xce, 0xdf, 0x6a, 0x0f, 0x6f, 0xef, 0xba, + 0x10, 0x74, 0xe1, 0xd2, 0x45, 0xff, 0x8b, 0xe7, 0x3b, 0xbc, 0x3b, 0xce, + 0x5c, 0xf2, 0xb8, 0x74, 0xf2, 0xb2, 0xdb, 0xe5, 0x13, 0x57, 0xb8, 0x57, + 0x9a, 0xaf, 0x3a, 0x5f, 0x6d, 0xea, 0x74, 0xea, 0x3c, 0xfe, 0x93, 0xd3, + 0x4f, 0xc7, 0xbb, 0x9c, 0xbb, 0x9a, 0xae, 0xb9, 0x5c, 0x6b, 0xb9, 0xee, + 0x7a, 0xbd, 0xb5, 0x7b, 0x66, 0xf7, 0xe9, 0x1b, 0x9e, 0x37, 0xce, 0xdd, + 0xf4, 0xbd, 0x79, 0xf1, 0x16, 0xff, 0xd6, 0xd5, 0x9e, 0x39, 0x3d, 0xdd, + 0xbd, 0xf3, 0x7a, 0x6f, 0xf7, 0xc5, 0xf7, 0xf5, 0xdf, 0x16, 0xdd, 0x7e, + 0x72, 0x27, 0xfd, 0xce, 0xcb, 0xbb, 0xd9, 0x77, 0x27, 0xee, 0xad, 0xbc, + 0x4f, 0xbc, 0x5f, 0xf4, 0x40, 0xed, 0x41, 0xd9, 0x43, 0xdd, 0x87, 0xd5, + 0x3f, 0x5b, 0xfe, 0xdc, 0xd8, 0xef, 0xdc, 0x7f, 0x6a, 0xc0, 0x77, 0xa0, + 0xf3, 0xd1, 0xdc, 0x47, 0xf7, 0x06, 0x85, 0x83, 0xcf, 0xfe, 0x91, 0xf5, + 0x8f, 0x0f, 0x43, 0x05, 0x8f, 0x99, 0x8f, 0xcb, 0x86, 0x0d, 0x86, 0xeb, + 0x9e, 0x38, 0x3e, 0x39, 0x39, 0xe2, 0x3f, 0x72, 0xfd, 0xe9, 0xfc, 0xa7, + 0x43, 0xcf, 0x64, 0xcf, 0x26, 0x9e, 0x17, 0xfe, 0xa2, 0xfe, 0xcb, 0xae, + 0x17, 0x16, 0x2f, 0x7e, 0xf8, 0xd5, 0xeb, 0xd7, 0xce, 0xd1, 0x98, 0xd1, + 0xa1, 0x97, 0xf2, 0x97, 0x93, 0xbf, 0x6d, 0x7c, 0xa5, 0xfd, 0xea, 0xc0, + 0xeb, 0x19, 0xaf, 0xdb, 0xc6, 0xc2, 0xc6, 0x1e, 0xbe, 0xc9, 0x78, 0x33, + 0x31, 0x5e, 0xf4, 0x56, 0xfb, 0xed, 0xc1, 0x77, 0xdc, 0x77, 0x1d, 0xef, + 0xa3, 0xdf, 0x0f, 0x4f, 0xe4, 0x7c, 0x20, 0x7f, 0x28, 0xff, 0x68, 0xf9, + 0xb1, 0xf5, 0x53, 0xd0, 0xa7, 0xfb, 0x93, 0x19, 0x93, 0x93, 0xff, 0x04, + 0x03, 0x98, 0xf3, 0xfc, 0x63, 0x33, 0x2d, 0xdb, 0x00, 0x00, 0x00, 0x06, + 0x62, 0x4b, 0x47, 0x44, 0x00, 0xff, 0x00, 0xff, 0x00, 0xff, 0xa0, 0xbd, + 0xa7, 0x93, 0x00, 0x00, 0x00, 0x09, 0x70, 0x48, 0x59, 0x73, 0x00, 0x00, + 0x5c, 0x46, 0x00, 0x00, 0x5c, 0x46, 0x01, 0x14, 0x94, 0x43, 0x41, 0x00, + 0x00, 0x00, 0x07, 0x74, 0x49, 0x4d, 0x45, 0x07, 0xda, 0x07, 0x17, 0x01, + 0x35, 0x2b, 0x0f, 0xd0, 0x77, 0x1e, 0x00, 0x00, 0x20, 0x00, 0x49, 0x44, + 0x41, 0x54, 0x78, 0xda, 0xec, 0x9d, 0x77, 0x78, 0x14, 0x55, 0xf7, 0xc7, + 0x3f, 0x77, 0x66, 0xb6, 0xa4, 0x27, 0x90, 0x84, 0xd0, 0x09, 0x91, 0x0e, + 0x21, 0x48, 0x11, 0x50, 0x44, 0x44, 0x51, 0x29, 0x8a, 0x08, 0x82, 0x8a, + 0x8d, 0x57, 0xc5, 0x42, 0x11, 0x7c, 0x6d, 0x88, 0x88, 0x08, 0x2a, 0x8a, + 0x88, 0x8a, 0x28, 0x2a, 0x22, 0xfa, 0x53, 0x50, 0xe1, 0xa5, 0x08, 0x58, + 0xe9, 0x0a, 0x08, 0x48, 0x91, 0x26, 0x48, 0x09, 0xbd, 0x24, 0x24, 0x21, + 0x6d, 0x37, 0xbb, 0x33, 0x73, 0x7f, 0x7f, 0x6c, 0xb2, 0x06, 0x69, 0x01, + 0x12, 0x08, 0x9b, 0xf9, 0x3e, 0x0f, 0x0f, 0xbb, 0x9b, 0x29, 0x77, 0xce, + 0x9c, 0xef, 0x3d, 0xe7, 0xdc, 0x72, 0x0e, 0x58, 0x08, 0x38, 0xa8, 0x6a, + 0xfc, 0x69, 0xff, 0x56, 0xab, 0x96, 0x14, 0x8a, 0x12, 0x5f, 0x1e, 0xc2, + 0x13, 0x14, 0x25, 0xb6, 0x96, 0x10, 0x75, 0x23, 0x2c, 0x89, 0x05, 0x3e, + 0x84, 0x25, 0x82, 0xcb, 0x1b, 0x36, 0x5b, 0x3d, 0xbc, 0xde, 0xad, 0x85, + 0x7e, 0x89, 0x8d, 0x15, 0x22, 0xeb, 0x2a, 0x29, 0x65, 0x5d, 0x30, 0x1a, + 0x09, 0xa1, 0x46, 0x4b, 0xe9, 0xa9, 0x00, 0x34, 0x00, 0xd3, 0x71, 0x86, + 0x2b, 0xfd, 0x8d, 0xd0, 0x66, 0x0a, 0xcc, 0x0d, 0x42, 0x84, 0xaf, 0x37, + 0xcd, 0x94, 0x2d, 0x05, 0x7f, 0x89, 0x8a, 0xba, 0x91, 0xf4, 0xf4, 0x9f, + 0x8b, 0xb5, 0xdd, 0x8a, 0x52, 0x0d, 0xd3, 0xdc, 0x6b, 0xbd, 0x40, 0x8b, + 0xe8, 0x16, 0xce, 0x86, 0x16, 0x2d, 0x0e, 0x8b, 0xd5, 0xab, 0xeb, 0x35, + 0x10, 0xc2, 0xb8, 0xd3, 0x34, 0x73, 0x3a, 0x82, 0x6c, 0x0a, 0x26, 0xd5, + 0x2b, 0xc7, 0xe2, 0x74, 0xd8, 0xb1, 0xdb, 0x6c, 0x84, 0x04, 0x3b, 0x89, + 0x8a, 0x08, 0x23, 0xb1, 0x6e, 0x3c, 0x0d, 0x6b, 0xd7, 0x20, 0xb1, 0x5e, + 0x4d, 0xc2, 0x42, 0x83, 0xf8, 0x6b, 0xc7, 0x3e, 0x34, 0x55, 0xc5, 0x61, + 0xb7, 0x61, 0x48, 0x93, 0x29, 0x33, 0x7e, 0x66, 0xd7, 0x9e, 0x83, 0x32, + 0xd7, 0x95, 0x27, 0xdc, 0x1e, 0x0f, 0x9b, 0xb7, 0xef, 0x03, 0x6c, 0x4b, + 0x85, 0xd0, 0xde, 0x53, 0x14, 0xfb, 0x46, 0xc3, 0x48, 0xdf, 0x56, 0x5c, + 0xed, 0xb6, 0xdb, 0x1b, 0x87, 0x79, 0x3c, 0x7f, 0xc7, 0x02, 0x37, 0x82, + 0xfb, 0x03, 0xdf, 0xaf, 0xda, 0x8f, 0x42, 0x38, 0x92, 0x15, 0xc5, 0x3e, + 0x23, 0x24, 0xe4, 0x96, 0x25, 0x99, 0x99, 0xff, 0xe7, 0x01, 0xe8, 0xd3, + 0xc7, 0xcb, 0xa7, 0x9f, 0xda, 0xac, 0x97, 0x6d, 0x11, 0xbd, 0xac, 0x21, + 0x26, 0x42, 0x51, 0x8c, 0x44, 0x29, 0xdd, 0x2f, 0x49, 0x99, 0xdb, 0x1e, + 0xa0, 0x5c, 0x64, 0x98, 0xac, 0x18, 0x5b, 0x5e, 0x74, 0xb9, 0xa1, 0x25, + 0x8f, 0xf4, 0xea, 0x88, 0x2b, 0xcf, 0x83, 0x34, 0x25, 0xa6, 0x94, 0x98, + 0xa6, 0x89, 0x94, 0x12, 0x21, 0x44, 0x21, 0x6b, 0x2a, 0x4e, 0x78, 0xfd, + 0x52, 0x4a, 0xa4, 0x94, 0xbe, 0x5f, 0x84, 0x40, 0x08, 0x41, 0xae, 0x3b, + 0x8f, 0x41, 0x23, 0x3e, 0x60, 0xe7, 0x9e, 0x83, 0x1c, 0x3d, 0x96, 0x91, + 0x7f, 0x6c, 0xd0, 0x5c, 0x10, 0x23, 0x54, 0x35, 0x6e, 0xab, 0x61, 0xec, + 0xcc, 0x3e, 0x67, 0x65, 0x13, 0x61, 0xcd, 0xc1, 0x98, 0x29, 0x65, 0x6e, + 0xe5, 0xfc, 0x9f, 0xe4, 0xbf, 0x74, 0xb0, 0xd0, 0x77, 0xfb, 0x0e, 0x9b, + 0x2d, 0xfa, 0x25, 0x55, 0x6d, 0xf4, 0xb5, 0xdb, 0xbd, 0xdc, 0x80, 0x2c, + 0xeb, 0xd5, 0x5b, 0x44, 0x0f, 0x2c, 0x68, 0x5a, 0x02, 0xba, 0xbe, 0xb3, + 0x80, 0x1c, 0xe5, 0x34, 0x2d, 0x2a, 0x49, 0xd7, 0xd3, 0x1e, 0x90, 0xd2, + 0x75, 0x2f, 0x18, 0x28, 0x8a, 0x20, 0xa6, 0x5c, 0x04, 0xcf, 0x3c, 0xda, + 0x8b, 0x9b, 0xae, 0x6d, 0x8a, 0x22, 0x04, 0x1e, 0xaf, 0x7e, 0xc2, 0x35, + 0xa4, 0x94, 0xf9, 0x84, 0x16, 0x98, 0xa6, 0x49, 0x56, 0x8e, 0x8b, 0x5c, + 0x57, 0x1e, 0x9b, 0xb7, 0x27, 0xb3, 0x68, 0xe5, 0x06, 0x42, 0x82, 0x1c, + 0x1c, 0x4a, 0x49, 0x43, 0x11, 0x82, 0x3a, 0x09, 0xd5, 0xb8, 0xbd, 0x43, + 0x6b, 0x2a, 0x44, 0x47, 0xa1, 0x2a, 0x0a, 0x66, 0x3e, 0xf1, 0x85, 0x10, + 0xd8, 0x34, 0x15, 0xaf, 0x61, 0xb0, 0xf0, 0xb7, 0xf5, 0x7c, 0xfc, 0xf5, + 0x7c, 0xfe, 0xda, 0xb1, 0x17, 0x29, 0x41, 0xd3, 0xaa, 0xdf, 0x05, 0x41, + 0x33, 0x75, 0xfd, 0xaf, 0xbc, 0x33, 0x8c, 0x16, 0x00, 0x06, 0x36, 0x5b, + 0xc5, 0xe6, 0x5e, 0x6f, 0xca, 0x32, 0x55, 0x35, 0x1d, 0x51, 0x11, 0xa1, + 0xc4, 0x94, 0x8f, 0xe4, 0xd6, 0xf6, 0xad, 0x69, 0xd9, 0xa4, 0x2e, 0x0e, + 0xbb, 0x8d, 0xd0, 0x60, 0x27, 0xbf, 0xae, 0xd9, 0x8c, 0xc3, 0x61, 0xe7, + 0xed, 0x49, 0xd3, 0xd9, 0x7f, 0x28, 0xb5, 0x70, 0x48, 0x91, 0x2a, 0x44, + 0x54, 0x55, 0x29, 0x8f, 0xba, 0x2d, 0xcd, 0xb0, 0x88, 0x7e, 0xd9, 0xc3, + 0xe9, 0x6c, 0x82, 0xdb, 0xbd, 0x2e, 0x9f, 0xdc, 0x11, 0x4e, 0x45, 0x09, + 0xb9, 0xd3, 0x30, 0x8e, 0x4d, 0x01, 0x1f, 0x8f, 0xec, 0x36, 0x8d, 0x1e, + 0x9d, 0xae, 0xe5, 0xc1, 0xee, 0x37, 0x11, 0x17, 0x53, 0x0e, 0xc3, 0x34, + 0x30, 0xcd, 0x13, 0x2d, 0x75, 0x61, 0xe4, 0xe4, 0xba, 0xf9, 0xf4, 0xdb, + 0x1f, 0x59, 0xb3, 0x71, 0x3b, 0x1b, 0xb6, 0xec, 0x2c, 0x72, 0x3b, 0xaa, + 0x56, 0x8a, 0xe5, 0xa5, 0x01, 0xbd, 0x69, 0x5c, 0x3f, 0x01, 0x9b, 0xa6, + 0x9e, 0xe4, 0x0d, 0x98, 0xa6, 0xc9, 0x0f, 0x4b, 0xd6, 0x30, 0x72, 0xfc, + 0x57, 0xe4, 0xba, 0xf2, 0x50, 0x94, 0x98, 0x0e, 0x42, 0x98, 0xbf, 0x1a, + 0x46, 0xaa, 0xeb, 0xdf, 0xd7, 0x6a, 0xd5, 0x6a, 0x16, 0x2b, 0x57, 0xde, + 0xb5, 0x4d, 0x4a, 0x57, 0xed, 0x8a, 0xb1, 0x51, 0x7c, 0x3f, 0xe5, 0x75, + 0x44, 0xfe, 0x35, 0x4e, 0xd7, 0x6e, 0x80, 0x94, 0xb4, 0xe3, 0x0c, 0x1d, + 0x33, 0x99, 0x55, 0x1b, 0xfe, 0x42, 0x4a, 0x24, 0x08, 0x61, 0xb3, 0xd5, + 0xe8, 0x12, 0x1e, 0xde, 0x6e, 0xde, 0xb1, 0x63, 0x73, 0x24, 0xa4, 0x5a, + 0x0a, 0x63, 0x11, 0xfd, 0xf2, 0x42, 0x58, 0x58, 0x6b, 0xb2, 0xb2, 0x96, + 0xa3, 0x69, 0x35, 0x11, 0xc2, 0xdb, 0x40, 0xd7, 0x53, 0xd6, 0x4b, 0xe9, + 0xd6, 0x00, 0x19, 0x16, 0x1a, 0x2c, 0x3e, 0x7e, 0x6d, 0x10, 0xb5, 0xe2, + 0x2b, 0xa3, 0xa9, 0xea, 0x69, 0x09, 0x52, 0x98, 0x8c, 0x1e, 0xaf, 0x4e, + 0xc7, 0x07, 0x86, 0x90, 0x9a, 0x9e, 0xf9, 0xaf, 0xc1, 0x2f, 0x85, 0x88, + 0x88, 0x08, 0x6e, 0xb8, 0xe1, 0x06, 0x9a, 0x36, 0x6d, 0xca, 0xaa, 0x55, + 0xab, 0xd8, 0xba, 0x75, 0x2b, 0xdb, 0xb6, 0x6d, 0x3b, 0xc1, 0x75, 0x2f, + 0x8c, 0x51, 0x4f, 0xf7, 0xa1, 0x53, 0xbb, 0x16, 0xa7, 0xbd, 0x5f, 0x46, + 0x66, 0x36, 0xd7, 0xf5, 0xfa, 0x6f, 0xfe, 0xaf, 0x41, 0x13, 0xed, 0xf6, + 0x6a, 0xfd, 0x3d, 0x9e, 0x6d, 0xde, 0x42, 0xd6, 0xf8, 0x0f, 0xf0, 0x36, + 0x19, 0xf0, 0xe0, 0xed, 0xe2, 0xe1, 0x9e, 0xb7, 0x60, 0x98, 0xe6, 0x39, + 0xc9, 0xc6, 0x9d, 0xe7, 0x61, 0xe0, 0xcb, 0x13, 0xf8, 0x7d, 0xfd, 0x5f, + 0xf9, 0xbf, 0x04, 0xa7, 0x40, 0x6e, 0x2c, 0x40, 0x48, 0x48, 0x1b, 0x72, + 0x72, 0x96, 0x59, 0x0a, 0x64, 0x11, 0xfd, 0x72, 0xb0, 0xe2, 0x0d, 0x70, + 0xbb, 0x37, 0xa3, 0x69, 0x55, 0xba, 0xea, 0xfa, 0x81, 0xaf, 0x41, 0xda, + 0x01, 0x9e, 0xee, 0x7b, 0x27, 0xbd, 0xba, 0x5c, 0x87, 0x9a, 0xef, 0x7a, + 0xff, 0x9b, 0xcc, 0x27, 0xc7, 0xbe, 0xb0, 0x73, 0xcf, 0x21, 0x9e, 0x7f, + 0x63, 0x12, 0xdb, 0x77, 0x1f, 0x38, 0x81, 0xb4, 0xbf, 0xfc, 0xf2, 0x0b, + 0xd7, 0x5e, 0x7b, 0x2d, 0x8a, 0xa2, 0xa0, 0xaa, 0xea, 0x49, 0xe7, 0x9a, + 0xa6, 0xe9, 0xff, 0xf7, 0xcb, 0x2f, 0xbf, 0x30, 0x7e, 0xfc, 0x78, 0xbe, + 0xff, 0xfe, 0xfb, 0x13, 0x8e, 0x79, 0xb0, 0xc7, 0xcd, 0x0c, 0xec, 0x73, + 0x3b, 0xb2, 0x50, 0x47, 0x53, 0xd0, 0x1e, 0xd3, 0x34, 0xf9, 0xf4, 0xdb, + 0x9f, 0x18, 0x3f, 0x65, 0x16, 0x80, 0x14, 0x22, 0x62, 0x80, 0x94, 0xc7, + 0xc7, 0x2b, 0x4a, 0xe4, 0x17, 0xa6, 0x99, 0xd1, 0xfb, 0xe7, 0x2f, 0x46, + 0x13, 0x53, 0x3e, 0xe2, 0x8c, 0xed, 0x3f, 0x13, 0xa4, 0x94, 0x1c, 0xcb, + 0xc8, 0xe4, 0x86, 0x7b, 0x9e, 0xcd, 0x8f, 0xdf, 0x15, 0x77, 0x64, 0x64, + 0x8f, 0xc8, 0x8c, 0x8c, 0xaf, 0xf3, 0x2c, 0x0d, 0xb2, 0x88, 0x5e, 0xaa, + 0x11, 0x17, 0xf7, 0x3c, 0x87, 0x0f, 0xbf, 0x86, 0xa6, 0xd5, 0xbc, 0x51, + 0xd7, 0x93, 0xbf, 0x03, 0xd3, 0xe1, 0xb0, 0xdb, 0xf8, 0x7c, 0xec, 0xb3, + 0xd4, 0xac, 0x56, 0xf1, 0x94, 0x2e, 0xf3, 0xe9, 0x48, 0x60, 0xb7, 0xdb, + 0xe8, 0x3b, 0x64, 0x1c, 0xbf, 0xae, 0xde, 0x74, 0xc2, 0xdf, 0x9e, 0x7d, + 0xf6, 0x59, 0x46, 0x8e, 0x1c, 0x89, 0xa6, 0x69, 0xe7, 0x4c, 0x32, 0x8f, + 0xc7, 0xc3, 0xef, 0xbf, 0xff, 0xce, 0xd0, 0xa1, 0x43, 0x59, 0xba, 0x74, + 0xa9, 0xff, 0xf7, 0x69, 0xe3, 0x87, 0x52, 0x2f, 0xa1, 0x0a, 0xa7, 0x30, + 0xfe, 0xe4, 0x79, 0xbc, 0xf4, 0x78, 0xfc, 0x15, 0xb9, 0xf7, 0xe0, 0x51, + 0xff, 0x8d, 0xde, 0x1a, 0xfa, 0xa8, 0x6c, 0xdf, 0x3a, 0xa9, 0x58, 0x74, + 0xcc, 0x34, 0x4d, 0x7e, 0x5d, 0xb3, 0x99, 0x01, 0xc3, 0xdf, 0x07, 0x84, + 0x19, 0x1c, 0xdc, 0x2e, 0x3e, 0x37, 0x77, 0xa1, 0x35, 0x3f, 0x67, 0x11, + 0xbd, 0xf4, 0x22, 0x2a, 0x6a, 0x40, 0x58, 0x7a, 0xfa, 0xfb, 0x5b, 0xc0, + 0xa8, 0x52, 0xb9, 0x62, 0xb4, 0x7c, 0xe3, 0xb9, 0x87, 0x45, 0xfd, 0x2b, + 0xaa, 0x9d, 0x13, 0x19, 0x0b, 0xc8, 0xdb, 0xfe, 0x9e, 0x67, 0x38, 0x56, + 0xc8, 0x4d, 0x7f, 0xec, 0xb1, 0xc7, 0x18, 0x3d, 0x7a, 0x34, 0x61, 0x61, + 0x61, 0x17, 0x64, 0x45, 0x0b, 0xce, 0xcb, 0xcb, 0xcb, 0xa3, 0x5a, 0xb5, + 0x6a, 0x1c, 0x3d, 0x7a, 0x14, 0x80, 0x9b, 0xdb, 0x36, 0xe3, 0xdd, 0x97, + 0x9e, 0x20, 0xc7, 0xe5, 0x3e, 0xe1, 0xda, 0xbe, 0xc1, 0x3f, 0x85, 0xbf, + 0x76, 0xee, 0xa5, 0x67, 0xbf, 0x51, 0x00, 0x34, 0x4b, 0xac, 0xcd, 0xd4, + 0x77, 0x87, 0x90, 0x9d, 0xe3, 0x3a, 0xa9, 0x1d, 0x05, 0xc3, 0xeb, 0x8a, + 0x22, 0x30, 0x4d, 0x89, 0x94, 0x3e, 0xcf, 0xe4, 0x6c, 0x6d, 0x5a, 0xba, + 0x6a, 0x63, 0x3e, 0xd9, 0x15, 0x5d, 0x88, 0xb0, 0xd0, 0x88, 0x88, 0x2e, + 0x79, 0x19, 0x19, 0xff, 0x67, 0x29, 0x95, 0x45, 0xf4, 0x4b, 0x8f, 0x3e, + 0x7d, 0x3c, 0x7c, 0xfa, 0xa9, 0x1d, 0x00, 0x55, 0x8d, 0xfc, 0xd0, 0x30, + 0x32, 0xfa, 0x0e, 0x1b, 0xd0, 0x9b, 0xab, 0x9b, 0x35, 0xa4, 0x72, 0x85, + 0xf2, 0xe8, 0x86, 0x71, 0xce, 0x44, 0x4c, 0x3b, 0x9e, 0x4d, 0x97, 0xff, + 0x0c, 0x25, 0xd7, 0xe5, 0xf3, 0x60, 0x27, 0x4e, 0x9c, 0x48, 0xef, 0xde, + 0xbd, 0x09, 0x0e, 0x0e, 0x3e, 0x6f, 0x82, 0x9f, 0xee, 0x5e, 0x52, 0x4a, + 0xb6, 0x6c, 0xd9, 0x42, 0xbb, 0x76, 0xed, 0x48, 0x4d, 0x4d, 0x25, 0x34, + 0x38, 0x88, 0x25, 0x5f, 0xbf, 0x85, 0xa2, 0x88, 0x53, 0xde, 0xe7, 0x50, + 0x4a, 0x3a, 0x1d, 0x1f, 0x18, 0x82, 0x94, 0x92, 0xea, 0x55, 0x2a, 0xf0, + 0xfd, 0xe4, 0x51, 0xe4, 0x79, 0xbc, 0xf9, 0xcf, 0xaf, 0x90, 0x93, 0xeb, + 0xe6, 0xe3, 0x69, 0xdf, 0xb3, 0x75, 0xc7, 0x1e, 0x0e, 0xa7, 0xa4, 0x53, + 0x2e, 0x32, 0x9c, 0x1e, 0x1d, 0xaf, 0xa5, 0x4b, 0xfb, 0xab, 0xf0, 0x78, + 0xf5, 0xd3, 0xb6, 0xbd, 0xe0, 0xb9, 0x86, 0xbc, 0xf9, 0x29, 0xf3, 0x17, + 0xad, 0x92, 0x9a, 0x16, 0xfb, 0xa0, 0xae, 0x1f, 0x9d, 0x62, 0x69, 0x98, + 0x45, 0xf4, 0x52, 0x84, 0x90, 0xb7, 0x21, 0xe7, 0xde, 0x7b, 0xbb, 0xdd, + 0x58, 0x7e, 0x50, 0x9f, 0xdb, 0xa5, 0xaa, 0xaa, 0xe2, 0x6c, 0xa3, 0xcf, + 0xa7, 0x53, 0xf6, 0x15, 0xeb, 0xfe, 0xe2, 0x89, 0x17, 0xdf, 0x41, 0x4a, + 0x68, 0xd2, 0xa4, 0x09, 0xf3, 0xe7, 0xcf, 0x27, 0x2e, 0x2e, 0xae, 0x44, + 0x5b, 0x5f, 0x40, 0xb2, 0xce, 0x9d, 0x3b, 0x33, 0x6f, 0xde, 0x3c, 0x6a, + 0x54, 0xa9, 0xc0, 0x0f, 0x53, 0x5e, 0xc3, 0x9d, 0xe7, 0x39, 0xe5, 0xf1, + 0x87, 0x52, 0xd2, 0xb9, 0xe5, 0xfe, 0xe7, 0x01, 0xa8, 0x55, 0xa3, 0x32, + 0x3f, 0x7e, 0xfe, 0x1a, 0xba, 0x61, 0xf0, 0xd8, 0xd0, 0x77, 0x59, 0xf0, + 0xdb, 0xba, 0x42, 0xe3, 0x14, 0x4e, 0xdc, 0x6e, 0xdf, 0xcc, 0x59, 0xed, + 0xf8, 0x2a, 0xfc, 0xdf, 0xb8, 0xe7, 0xb0, 0x69, 0xea, 0x19, 0xe5, 0xe2, + 0xf1, 0xea, 0xb4, 0xb8, 0xad, 0x1f, 0x60, 0x9b, 0x0d, 0xde, 0xae, 0x96, + 0x6e, 0x59, 0x44, 0xbf, 0x64, 0x70, 0x38, 0xe2, 0xc9, 0xcb, 0xdb, 0x8d, + 0xaa, 0x56, 0xed, 0x68, 0x18, 0xfb, 0xbe, 0x6d, 0xd5, 0xa4, 0x5e, 0xf0, + 0xe0, 0x87, 0xba, 0xcb, 0xba, 0x09, 0x55, 0x85, 0x6e, 0x18, 0xe7, 0xed, + 0x52, 0xef, 0xdc, 0x7b, 0x88, 0xee, 0x8f, 0x8d, 0xf0, 0xc5, 0xcc, 0xd3, + 0xa6, 0xd1, 0xb3, 0x67, 0xcf, 0x62, 0xb5, 0xe0, 0x67, 0xbb, 0x7f, 0x5e, + 0x5e, 0x1e, 0x95, 0x2b, 0x57, 0x26, 0x2d, 0x2d, 0x8d, 0xce, 0xed, 0x5b, + 0x32, 0xea, 0xbf, 0x0f, 0x9e, 0x72, 0xc4, 0x1e, 0x29, 0x39, 0x72, 0xec, + 0x38, 0x37, 0xdf, 0xff, 0x3c, 0x52, 0x4a, 0x42, 0x83, 0x9d, 0xe4, 0x79, + 0xbc, 0x78, 0x75, 0x9f, 0xf7, 0x22, 0x84, 0x40, 0x4a, 0xc9, 0xd6, 0xad, + 0x5b, 0x09, 0x0e, 0x0e, 0x26, 0x3e, 0x3e, 0x1e, 0xd3, 0x34, 0x69, 0xda, + 0xa8, 0x36, 0x53, 0xc6, 0x3c, 0x7d, 0x46, 0x2f, 0x47, 0x4a, 0xc9, 0xd5, + 0xdd, 0x07, 0xe1, 0x72, 0x8b, 0xa3, 0x52, 0xba, 0x2a, 0x58, 0xda, 0x76, + 0x76, 0x28, 0x96, 0x08, 0x4a, 0x06, 0x0d, 0x1b, 0x7e, 0xa4, 0x0a, 0xe1, + 0xf8, 0xd1, 0x30, 0xf6, 0xcd, 0x7b, 0xef, 0xe5, 0x7e, 0x41, 0x93, 0xde, + 0x78, 0x8a, 0x5a, 0xf1, 0x95, 0x85, 0x71, 0x1e, 0x56, 0xbc, 0x00, 0x4b, + 0x57, 0x6d, 0xf2, 0x93, 0x7c, 0xe0, 0xc0, 0x81, 0xf4, 0xec, 0xd9, 0xd3, + 0x4f, 0x9a, 0x8b, 0x62, 0x15, 0x84, 0xc0, 0xe1, 0x70, 0xb0, 0x77, 0xaf, + 0x6f, 0x0c, 0x6c, 0xee, 0x82, 0x95, 0xfc, 0xb8, 0x74, 0xcd, 0xa9, 0x89, + 0x2e, 0x04, 0x71, 0x31, 0x51, 0x3c, 0xd0, 0xbd, 0x03, 0x00, 0xd9, 0xb9, + 0x6e, 0x3f, 0xc9, 0x01, 0xec, 0x76, 0x3b, 0x5b, 0xb7, 0x6e, 0xa5, 0x5e, + 0xbd, 0x7a, 0xdc, 0x71, 0xc7, 0x1d, 0x28, 0x8a, 0x4f, 0x15, 0xff, 0xd8, + 0xb8, 0x9d, 0xa5, 0xab, 0xfe, 0x3c, 0xf5, 0x35, 0x0b, 0xb5, 0xa3, 0x63, + 0xbb, 0x16, 0x48, 0xe9, 0x8a, 0x75, 0x3a, 0x3b, 0x58, 0xeb, 0x63, 0x2d, + 0x8b, 0x7e, 0x71, 0x61, 0xb3, 0xc5, 0xe0, 0xf5, 0xa6, 0x20, 0x44, 0x68, + 0x1f, 0x29, 0x5d, 0x93, 0x22, 0x23, 0x82, 0xe4, 0x9c, 0x8f, 0x47, 0x88, + 0xf0, 0x90, 0x60, 0x24, 0xe7, 0x6f, 0x75, 0xa5, 0x94, 0xfc, 0xbe, 0x7e, + 0x1b, 0x8f, 0xbe, 0x30, 0xce, 0xe7, 0xe2, 0xd6, 0xae, 0xed, 0x9f, 0xff, + 0xbe, 0x58, 0x24, 0xff, 0x37, 0x76, 0xef, 0xde, 0x4d, 0xcd, 0x9a, 0x35, + 0x01, 0xf8, 0xf5, 0xdb, 0xb7, 0x09, 0x0d, 0x09, 0x3a, 0xe5, 0x71, 0x79, + 0x1e, 0x2f, 0x77, 0x3c, 0x3a, 0x82, 0xfd, 0x87, 0x53, 0x4e, 0xf9, 0xf7, + 0xef, 0xbf, 0xff, 0x9e, 0x5b, 0x6e, 0xb9, 0xe5, 0x84, 0xdf, 0x6a, 0x56, + 0xab, 0xc8, 0xfc, 0xc9, 0xa3, 0x4e, 0x1b, 0x16, 0x00, 0xcc, 0x5b, 0xf8, + 0x3b, 0x2f, 0x8c, 0x99, 0x0c, 0x04, 0xc7, 0x41, 0xee, 0x11, 0x4b, 0xfb, + 0x2c, 0x8b, 0x7e, 0xd1, 0x20, 0xa5, 0xb3, 0x9a, 0x10, 0x8e, 0x34, 0x29, + 0xb3, 0x27, 0x5d, 0xd3, 0xac, 0x9e, 0x5c, 0x3e, 0x7d, 0x9c, 0x08, 0x0b, + 0x09, 0x02, 0x71, 0x61, 0x56, 0xd7, 0xe9, 0xb0, 0x33, 0x60, 0xf8, 0x78, + 0xff, 0xf7, 0xf9, 0xf3, 0xe7, 0x5f, 0x52, 0x92, 0x03, 0xc4, 0xc7, 0xc7, + 0xf3, 0xf4, 0xd3, 0x4f, 0x03, 0xd0, 0xb5, 0xef, 0xf0, 0xd3, 0x5a, 0x60, + 0xbb, 0x4d, 0x63, 0xd6, 0x47, 0xc3, 0x29, 0x17, 0x15, 0x7e, 0xca, 0xbf, + 0xff, 0x9b, 0xe4, 0x00, 0xbb, 0xf6, 0x1e, 0xe2, 0x70, 0x4a, 0xfa, 0x19, + 0xef, 0x9f, 0xeb, 0xce, 0x2b, 0x70, 0x1c, 0x82, 0x2c, 0xcd, 0xb3, 0x88, + 0x5e, 0xe2, 0xb0, 0xdb, 0x7d, 0x7b, 0xbf, 0x55, 0x35, 0xfa, 0x4e, 0x5d, + 0xdf, 0xbf, 0x27, 0x22, 0xcc, 0x16, 0xf9, 0xc5, 0xdb, 0xcf, 0xf2, 0xe1, + 0xa8, 0x81, 0xe2, 0x4c, 0x23, 0xc8, 0xe7, 0x62, 0xcd, 0x27, 0x7d, 0xf3, + 0xa3, 0x7f, 0x2d, 0xfb, 0xa8, 0x51, 0xa3, 0x48, 0x48, 0x48, 0x28, 0x71, + 0x92, 0x17, 0x10, 0x57, 0xd7, 0x75, 0x66, 0xcd, 0x9a, 0xc5, 0xf2, 0xe5, + 0xcb, 0xd9, 0xb4, 0x69, 0x13, 0xc9, 0xc9, 0xc9, 0xfe, 0x63, 0x9e, 0x7c, + 0xf2, 0x49, 0x00, 0x52, 0xd3, 0x8e, 0x93, 0x99, 0x9d, 0x7b, 0x5a, 0x37, + 0x5b, 0x55, 0x15, 0x7e, 0xfe, 0xfc, 0x75, 0xea, 0x26, 0x54, 0x2d, 0xf2, + 0xfd, 0x5d, 0xee, 0x33, 0xaf, 0x87, 0x71, 0xe5, 0xcf, 0x36, 0x48, 0xe9, + 0x35, 0x2d, 0x2d, 0xb4, 0x88, 0x5e, 0xe2, 0xf0, 0x78, 0xd2, 0x42, 0x15, + 0x25, 0x74, 0xa7, 0x61, 0xa4, 0x7e, 0x9d, 0x54, 0xbf, 0x26, 0x2b, 0xff, + 0xf7, 0x8e, 0x68, 0x58, 0xbb, 0xc6, 0x39, 0x4f, 0x99, 0x9d, 0x8e, 0x6c, + 0x8a, 0xa2, 0xf0, 0xf6, 0x27, 0xd3, 0x01, 0x70, 0x38, 0x1c, 0x3c, 0xf9, + 0xe4, 0x93, 0x67, 0x8c, 0x5f, 0x8b, 0x33, 0x1e, 0x9f, 0x30, 0x61, 0x02, + 0x36, 0x9b, 0x8d, 0xdb, 0x6f, 0xbf, 0x9d, 0xab, 0xaf, 0xbe, 0x9a, 0x46, + 0x8d, 0x1a, 0x11, 0x1f, 0x1f, 0x4f, 0x4c, 0x4c, 0x0c, 0xcb, 0x97, 0x2f, + 0xa7, 0x52, 0xa5, 0x4a, 0xfe, 0xe3, 0x3b, 0xfd, 0x67, 0xe8, 0x19, 0xaf, + 0xa5, 0x28, 0x82, 0xe9, 0x13, 0x86, 0x71, 0xed, 0x55, 0x89, 0x67, 0xbd, + 0x77, 0x85, 0xe8, 0x28, 0xaa, 0x57, 0x8e, 0x3d, 0x4b, 0x98, 0xa4, 0xe5, + 0x5f, 0xdb, 0x66, 0xe9, 0xb0, 0x45, 0xf4, 0x92, 0x42, 0x0d, 0x00, 0x42, + 0x43, 0x5b, 0x94, 0x87, 0xec, 0x4c, 0xbb, 0xcd, 0x13, 0xff, 0xf9, 0xd8, + 0x67, 0xf8, 0xe2, 0xed, 0x67, 0x71, 0xe5, 0x79, 0x8a, 0xcd, 0xda, 0x0a, + 0x21, 0xf8, 0x61, 0xf1, 0x6a, 0xff, 0x1a, 0xf1, 0x3d, 0x7b, 0xf6, 0x10, + 0x14, 0x14, 0x54, 0x62, 0xd6, 0xbc, 0xa0, 0x03, 0xd9, 0xbb, 0x77, 0x2f, + 0x41, 0x41, 0x41, 0x3c, 0xf1, 0xc4, 0x13, 0xa7, 0x3c, 0x2e, 0x35, 0x35, + 0x95, 0x5b, 0x6e, 0xb9, 0x85, 0x5e, 0xbd, 0x7a, 0x71, 0xeb, 0xad, 0xb7, + 0x02, 0x90, 0x9d, 0xed, 0x3a, 0x61, 0xe1, 0xce, 0xa9, 0x9e, 0xc5, 0xab, + 0xeb, 0xbc, 0x37, 0xfc, 0x09, 0x66, 0x4d, 0x1c, 0x7e, 0xc6, 0x76, 0xf4, + 0xbd, 0xbb, 0xd3, 0x49, 0xbb, 0xf1, 0x4e, 0xdd, 0x56, 0x81, 0x69, 0xe6, + 0x24, 0x5b, 0xfa, 0x68, 0x11, 0xbd, 0x98, 0xdd, 0xf4, 0x3a, 0x00, 0x68, + 0x5a, 0x05, 0x45, 0x88, 0x98, 0xa1, 0xd9, 0xd9, 0xab, 0x52, 0xef, 0xbf, + 0xa3, 0x3d, 0x7f, 0xcc, 0x9d, 0x20, 0x1a, 0xd5, 0x89, 0xc7, 0x30, 0xcc, + 0x62, 0x25, 0xa1, 0x94, 0x92, 0xa1, 0x63, 0x3f, 0x03, 0xa0, 0x57, 0xaf, + 0x5e, 0x54, 0xa8, 0x50, 0xa1, 0x44, 0x5d, 0x76, 0x21, 0x04, 0x2b, 0x56, + 0xac, 0xa0, 0x7a, 0xf5, 0xea, 0xfe, 0xb9, 0xed, 0xd3, 0x61, 0xc0, 0x80, + 0x01, 0x7c, 0xfd, 0xf5, 0xd7, 0x0c, 0x1e, 0x3c, 0xd8, 0xff, 0xdb, 0xb0, + 0xb7, 0xa7, 0xa0, 0x9c, 0xa1, 0x7d, 0x05, 0x53, 0x6a, 0x35, 0xaa, 0xc6, + 0xb1, 0x7c, 0xc6, 0x3b, 0x54, 0x8c, 0x2d, 0x7f, 0xd2, 0x31, 0xdd, 0x6e, + 0xbe, 0x86, 0x9e, 0x9d, 0xdb, 0x9e, 0xb5, 0xad, 0xb3, 0x7f, 0x5e, 0x0e, + 0x38, 0x67, 0x5c, 0xca, 0x71, 0x8a, 0xcb, 0x09, 0x96, 0x94, 0x8a, 0x80, + 0xc4, 0xc4, 0x8d, 0xfc, 0xf9, 0x67, 0xa3, 0x7c, 0x97, 0x31, 0xa1, 0x92, + 0xd7, 0xbb, 0x7b, 0x57, 0x70, 0x90, 0xcd, 0xb1, 0xe8, 0xab, 0x37, 0x71, + 0x38, 0xec, 0x25, 0x66, 0x5d, 0x4d, 0xd3, 0xa4, 0xd9, 0xad, 0x4f, 0x20, + 0x25, 0xe4, 0xe4, 0xe4, 0x10, 0x1c, 0x1c, 0x5c, 0xa2, 0x31, 0xb9, 0x6f, + 0xed, 0xbc, 0x1d, 0xe3, 0x1c, 0xc2, 0x8e, 0xb0, 0xb0, 0x30, 0xda, 0xb7, + 0x6f, 0xcf, 0xac, 0x59, 0xb3, 0x00, 0x58, 0x3f, 0xff, 0xc3, 0x22, 0xdf, + 0x4f, 0x08, 0xc1, 0xd4, 0x39, 0x8b, 0x18, 0xfd, 0xe1, 0xd7, 0xfe, 0xdf, + 0xe7, 0x7c, 0x32, 0x82, 0x6a, 0x95, 0x62, 0xcf, 0x7a, 0x6e, 0xb3, 0x2e, + 0x4f, 0x80, 0xa8, 0xd0, 0xdb, 0x30, 0x0e, 0x7d, 0x69, 0x69, 0xa8, 0x65, + 0xd1, 0x8b, 0x05, 0x7f, 0xfe, 0xd9, 0x88, 0xda, 0xb5, 0x97, 0x68, 0x10, + 0xba, 0xd4, 0xeb, 0xdd, 0x79, 0x60, 0x68, 0xff, 0xbb, 0x1c, 0xbf, 0x4d, + 0x1f, 0x27, 0x4b, 0x8a, 0xe4, 0x05, 0xd6, 0x2f, 0x2b, 0xd7, 0x85, 0x94, + 0xd0, 0xb0, 0x61, 0xc3, 0x12, 0x25, 0x79, 0xc1, 0xfd, 0xd2, 0xd3, 0xd3, + 0xcf, 0x89, 0xe4, 0x00, 0x59, 0x59, 0x59, 0xcc, 0x98, 0x31, 0xe3, 0x84, + 0x5d, 0x6d, 0x45, 0xbd, 0x1f, 0x48, 0xee, 0xba, 0xb5, 0x1d, 0x0b, 0xbf, + 0x7a, 0xd3, 0xff, 0xfb, 0x6d, 0x0f, 0xbf, 0x44, 0x8e, 0xcb, 0x7d, 0xc6, + 0xeb, 0xbc, 0xf3, 0xd9, 0x2c, 0x0c, 0xd3, 0xc4, 0x34, 0xdd, 0x73, 0x2c, + 0xed, 0xb4, 0x88, 0x7e, 0xc1, 0xb0, 0xd9, 0xaa, 0xe5, 0x2b, 0x65, 0x85, + 0xfb, 0xb6, 0x6f, 0x6f, 0xeb, 0xb1, 0xd9, 0xdc, 0x6d, 0x16, 0x7f, 0x3d, + 0x46, 0x76, 0xbf, 0xa5, 0xcd, 0x45, 0xf1, 0x86, 0xd6, 0x6e, 0xda, 0x01, + 0xf8, 0x76, 0xa2, 0x5d, 0x0c, 0x8c, 0x1d, 0x3b, 0xb6, 0x48, 0xc7, 0x75, + 0xed, 0xda, 0x95, 0x0f, 0x3e, 0xf8, 0xc0, 0xff, 0x7d, 0xcb, 0x96, 0x2d, + 0xfe, 0x2d, 0xb0, 0x69, 0xc7, 0xb3, 0xce, 0x61, 0xb0, 0xd0, 0x27, 0xc2, + 0x72, 0x91, 0x61, 0x2c, 0xf9, 0xfa, 0x2d, 0x7f, 0x47, 0x71, 0xf5, 0x1d, + 0x4f, 0xb2, 0xe7, 0xc0, 0xd1, 0xd3, 0x5e, 0x67, 0xf6, 0x4f, 0xbf, 0x49, + 0x5f, 0x08, 0x15, 0xac, 0x5b, 0x5a, 0x6a, 0x11, 0xfd, 0xbc, 0xa1, 0x69, + 0xad, 0x00, 0x50, 0xd5, 0xb8, 0x7a, 0xa0, 0x65, 0x4a, 0x79, 0x64, 0x4a, + 0x9b, 0xab, 0x12, 0x59, 0x3d, 0x7b, 0x3c, 0x91, 0x61, 0xa1, 0xe2, 0x62, + 0xcd, 0x61, 0xff, 0xb1, 0xf1, 0x6f, 0x00, 0xda, 0xb7, 0x6f, 0x7f, 0x51, + 0x9e, 0xfb, 0x6c, 0x71, 0x39, 0x40, 0xcf, 0x9e, 0x3d, 0x99, 0x39, 0x73, + 0x26, 0x8f, 0x3e, 0xfa, 0x28, 0x15, 0x2b, 0x56, 0xcc, 0x1f, 0xbb, 0xb0, + 0xd3, 0xa9, 0x53, 0x27, 0x00, 0xf6, 0x1f, 0x4a, 0x3d, 0x2f, 0xd9, 0x44, + 0x84, 0x85, 0xf0, 0xc3, 0xe7, 0xaf, 0xf9, 0xcf, 0xbd, 0x6b, 0xc0, 0x28, + 0x32, 0xb3, 0x73, 0x4f, 0x22, 0xbb, 0xd7, 0xab, 0x93, 0x7e, 0x3c, 0x5b, + 0xa8, 0x6a, 0xf4, 0x44, 0xaf, 0xf7, 0x80, 0xcb, 0xd2, 0x56, 0x8b, 0xe8, + 0x17, 0x60, 0xc9, 0xb3, 0x1b, 0x83, 0x6d, 0xb7, 0xdb, 0xbd, 0x6a, 0x0b, + 0xe8, 0xa1, 0xad, 0x9a, 0xd4, 0xe7, 0x83, 0x11, 0xfd, 0x45, 0xe1, 0x04, + 0x8a, 0x25, 0x0d, 0x29, 0x25, 0x5f, 0xce, 0x5a, 0x00, 0x40, 0x78, 0x78, + 0xf8, 0x45, 0x99, 0x52, 0x0b, 0x0a, 0x3a, 0xfb, 0xda, 0x93, 0xc2, 0xcf, + 0x7e, 0xe8, 0xd0, 0x21, 0x34, 0x4d, 0xa3, 0x76, 0xed, 0xda, 0x3c, 0xf3, + 0xcc, 0x33, 0xbe, 0x30, 0xe7, 0xaf, 0x5d, 0xe7, 0x2d, 0x9f, 0xb8, 0xe8, + 0x28, 0x66, 0x7f, 0xec, 0x5b, 0xe2, 0xeb, 0x72, 0x7b, 0x78, 0x6c, 0xe8, + 0xbb, 0x84, 0x85, 0x06, 0x9f, 0xf0, 0xec, 0x7f, 0xef, 0x39, 0x08, 0x20, + 0x21, 0x6a, 0x90, 0xa5, 0xa9, 0x16, 0xd1, 0xcf, 0x19, 0xc1, 0xc1, 0xcd, + 0xf2, 0x15, 0x39, 0x74, 0x9c, 0xcb, 0xb5, 0x71, 0x3d, 0x78, 0xab, 0x03, + 0xbc, 0x34, 0xf0, 0x5e, 0x31, 0xe9, 0x8d, 0xc1, 0x78, 0x75, 0xfd, 0xa2, + 0xae, 0x44, 0x2b, 0x9c, 0x76, 0x49, 0x51, 0x94, 0x8b, 0x72, 0xef, 0xeb, + 0xae, 0xbb, 0xee, 0xac, 0xc7, 0x4c, 0x9b, 0x36, 0xcd, 0x9f, 0x29, 0xb6, + 0xa0, 0x6d, 0x00, 0xd5, 0xab, 0x57, 0x07, 0xe0, 0xc7, 0x25, 0x6b, 0x0a, + 0x65, 0x99, 0x3d, 0x77, 0x54, 0xab, 0x14, 0xc3, 0xc2, 0xaf, 0xde, 0xa4, + 0x79, 0x62, 0x1d, 0xb6, 0xfc, 0xbd, 0x87, 0xeb, 0x7a, 0xfd, 0x97, 0x88, + 0xb0, 0x90, 0xfc, 0x77, 0x03, 0x53, 0x67, 0x2f, 0x04, 0x54, 0x2f, 0x04, + 0x59, 0xd6, 0xdc, 0x22, 0xfa, 0xb9, 0x23, 0x3a, 0xfa, 0xae, 0x50, 0x70, + 0xa4, 0x48, 0x99, 0x3d, 0x10, 0x90, 0xb5, 0x6a, 0x54, 0x16, 0xf3, 0x3e, + 0x1d, 0xc9, 0x1d, 0x37, 0x5f, 0x43, 0x71, 0xac, 0x70, 0x3b, 0x57, 0x78, + 0x0b, 0xcd, 0x23, 0x6b, 0x9a, 0x76, 0x51, 0x3c, 0x88, 0xeb, 0xaf, 0xbf, + 0xfe, 0xbc, 0xcf, 0xaf, 0x5c, 0xb9, 0xb2, 0x3f, 0x46, 0x17, 0x17, 0x38, + 0x7c, 0x11, 0x15, 0x11, 0xca, 0xc4, 0x57, 0x9f, 0x64, 0xd2, 0xe8, 0xc1, + 0x1c, 0x49, 0x4d, 0xe7, 0xa1, 0xe7, 0xc6, 0x12, 0x1a, 0xe2, 0x5b, 0x3f, + 0x30, 0x7f, 0xf1, 0x6a, 0x14, 0x25, 0x78, 0xa6, 0x61, 0xfc, 0x69, 0x29, + 0xad, 0x45, 0xf4, 0xa2, 0xba, 0xaa, 0x0d, 0xf2, 0x63, 0xf1, 0x0a, 0x4f, + 0xef, 0xdd, 0xfb, 0xdf, 0x14, 0xc8, 0x2b, 0x0f, 0x70, 0x7d, 0xab, 0x24, + 0x31, 0x6f, 0xf2, 0x48, 0x2a, 0x55, 0x28, 0x8f, 0x79, 0x11, 0x5c, 0xe6, + 0xb3, 0x59, 0x74, 0x5d, 0x2f, 0xf9, 0x31, 0x27, 0xdf, 0xea, 0x35, 0x85, + 0xf7, 0xdf, 0x7f, 0xff, 0x9c, 0xce, 0x9b, 0x35, 0x6b, 0xd6, 0x09, 0xae, + 0x75, 0x48, 0xb0, 0x13, 0x89, 0x2c, 0x86, 0xb6, 0x08, 0x9a, 0x36, 0xac, + 0xc5, 0x17, 0x63, 0x9f, 0x65, 0xe1, 0xf2, 0xf5, 0xb4, 0xe9, 0x3e, 0x88, + 0x57, 0xdf, 0x9f, 0x8a, 0x61, 0x18, 0x68, 0x5a, 0xe2, 0x63, 0x16, 0x75, + 0x2d, 0xa2, 0x17, 0x19, 0x2e, 0xd7, 0x66, 0x84, 0x08, 0xfe, 0xcb, 0x30, + 0x8e, 0xbc, 0x01, 0xd2, 0x11, 0x12, 0xec, 0x14, 0x23, 0xff, 0xfb, 0x20, + 0x13, 0x46, 0x0e, 0x20, 0x27, 0xd7, 0x7d, 0x72, 0xfa, 0xa3, 0x42, 0x0a, + 0x5d, 0xd2, 0x03, 0x72, 0x41, 0x8e, 0x7f, 0xaa, 0x27, 0x19, 0xc5, 0xb0, + 0x9c, 0xb6, 0xa8, 0x56, 0xfd, 0xf1, 0xc7, 0x1f, 0xa7, 0x59, 0xb3, 0x66, + 0x67, 0x3d, 0xd6, 0x6e, 0xb7, 0xfb, 0x77, 0x9e, 0x15, 0xc8, 0x41, 0x08, + 0x41, 0xe5, 0x0a, 0xd1, 0x20, 0x8b, 0xad, 0xf7, 0xa1, 0x61, 0xed, 0x1a, + 0x0c, 0x1f, 0x74, 0x1f, 0x87, 0x52, 0xd2, 0x98, 0xf6, 0xdd, 0x62, 0xc0, + 0x9e, 0xec, 0xf1, 0xfc, 0x96, 0x6e, 0x51, 0xf7, 0xdc, 0xa0, 0x95, 0xe5, + 0x87, 0x17, 0x22, 0x74, 0x87, 0x94, 0xd9, 0x09, 0x00, 0x5d, 0x6e, 0x68, + 0x29, 0xde, 0x78, 0xee, 0x61, 0xf2, 0x3c, 0x5e, 0xff, 0xf6, 0x48, 0x21, + 0x40, 0x55, 0x55, 0xb2, 0x73, 0xdc, 0xfc, 0x9d, 0xbc, 0x9f, 0x8c, 0xe3, + 0xd9, 0x1c, 0xcf, 0xca, 0xc1, 0x30, 0x4d, 0xa2, 0xa3, 0x22, 0xd8, 0xb9, + 0xf7, 0x10, 0x31, 0xe5, 0x22, 0xa8, 0x12, 0x17, 0x4d, 0xd5, 0x4a, 0x31, + 0xc4, 0xc5, 0x44, 0x91, 0xe7, 0xf1, 0x52, 0x1c, 0x4e, 0x80, 0xaa, 0x2a, + 0x84, 0x06, 0x3b, 0xc9, 0xce, 0x75, 0x93, 0x9e, 0x9e, 0x4e, 0x68, 0x68, + 0xe8, 0x45, 0xb1, 0xea, 0x52, 0x4a, 0x56, 0xaf, 0x5e, 0xcd, 0xbc, 0x79, + 0xf3, 0x78, 0xfa, 0xe9, 0xa7, 0xd9, 0xba, 0x75, 0xeb, 0x49, 0xc7, 0x35, + 0x6d, 0xda, 0x94, 0xe5, 0xcb, 0x97, 0x63, 0xb3, 0xd9, 0x4e, 0xea, 0x28, + 0x6c, 0xc5, 0x1c, 0x66, 0xe8, 0x86, 0x41, 0xf7, 0x9b, 0xdb, 0xb0, 0x73, + 0xcf, 0x21, 0xbe, 0xf8, 0xdf, 0xcf, 0x12, 0xbc, 0x95, 0x2d, 0xda, 0x9e, + 0xc7, 0xbb, 0x2d, 0x4b, 0x0f, 0x1b, 0x1a, 0x7a, 0x2d, 0xd9, 0xd9, 0x4b, + 0xb1, 0xdb, 0x93, 0xe2, 0x3c, 0x9e, 0x2d, 0x9f, 0x83, 0xe7, 0x46, 0xa7, + 0xc3, 0xce, 0x7b, 0x2f, 0x3f, 0x41, 0xcb, 0xa4, 0x7a, 0xfe, 0x8d, 0x28, + 0xaa, 0xa2, 0x70, 0xec, 0x78, 0x16, 0x4b, 0x56, 0xfe, 0xc9, 0x8a, 0xb5, + 0x5b, 0xf8, 0x71, 0xe9, 0x9a, 0x22, 0x5d, 0x3f, 0xa6, 0x7c, 0x04, 0x03, + 0x1e, 0xb8, 0x9d, 0x6b, 0x9a, 0x35, 0x24, 0x3a, 0x2a, 0x9c, 0xf3, 0xcd, + 0x24, 0x53, 0x80, 0x45, 0x2b, 0x37, 0x30, 0x68, 0xc4, 0x07, 0x0c, 0x1d, + 0x3a, 0x94, 0x57, 0x5e, 0x79, 0xe5, 0xa2, 0xca, 0xaa, 0xb0, 0xc7, 0xb2, + 0x65, 0xcb, 0x16, 0xd6, 0xae, 0x5d, 0x4b, 0x74, 0x74, 0x34, 0x49, 0x49, + 0x49, 0xc4, 0xc5, 0xc5, 0x9d, 0xe4, 0xd1, 0xfc, 0xfc, 0xf3, 0xcf, 0x74, + 0xe8, 0xd0, 0x81, 0x4f, 0x5e, 0x1f, 0x4c, 0xb3, 0xc4, 0xda, 0xc5, 0xde, + 0x16, 0x55, 0x55, 0x18, 0x39, 0xfe, 0x2b, 0xbe, 0x99, 0xbb, 0x04, 0x45, + 0x09, 0xff, 0xc3, 0x34, 0x33, 0x9b, 0x59, 0xf4, 0xb5, 0x88, 0x7e, 0x12, + 0x42, 0x42, 0xae, 0x21, 0x27, 0xe7, 0x57, 0x34, 0xad, 0xd2, 0xf5, 0xba, + 0x7e, 0x70, 0x01, 0x40, 0xfd, 0x5a, 0xd5, 0xf9, 0x66, 0xfc, 0x0b, 0xfe, + 0xcc, 0x27, 0x8a, 0xa2, 0xe0, 0xf1, 0x78, 0x99, 0xb7, 0x68, 0x15, 0xc3, + 0xc7, 0x7d, 0xfe, 0x8f, 0xdb, 0xa3, 0x69, 0x54, 0xac, 0x58, 0x91, 0x90, + 0x90, 0x10, 0xbc, 0x5e, 0x2f, 0x76, 0xbb, 0x1d, 0x29, 0x25, 0xba, 0xae, + 0x63, 0x9a, 0x26, 0x39, 0x39, 0x39, 0xb8, 0x5c, 0x2e, 0x32, 0x33, 0xff, + 0xd9, 0xd4, 0x71, 0x67, 0xe7, 0xb6, 0x0c, 0xea, 0xd3, 0x8d, 0x20, 0xa7, + 0xe3, 0xbc, 0x95, 0xdb, 0x30, 0x4d, 0x9a, 0x75, 0x79, 0x82, 0xf2, 0xe5, + 0xcb, 0x93, 0x9a, 0x5a, 0xba, 0xab, 0x93, 0x74, 0xeb, 0xd6, 0x8d, 0x99, + 0x33, 0x67, 0xf2, 0xd9, 0x5b, 0xcf, 0x90, 0x54, 0xaf, 0x66, 0x89, 0x74, + 0x3c, 0x86, 0x69, 0x72, 0xed, 0x9d, 0x83, 0xc9, 0x75, 0xe5, 0xa1, 0xaa, + 0x91, 0x93, 0x0d, 0x23, 0xa3, 0x4f, 0xbd, 0x7a, 0xfb, 0xd8, 0xba, 0xb5, + 0xaa, 0xc5, 0x64, 0x8b, 0xe8, 0x10, 0x1b, 0xdb, 0x9d, 0xa3, 0x47, 0xa7, + 0x63, 0xb3, 0x55, 0x7d, 0xc2, 0xeb, 0xdd, 0x37, 0x3e, 0x2c, 0x24, 0x88, + 0xe9, 0x1f, 0x0c, 0x23, 0x2e, 0x26, 0xca, 0x5f, 0xd6, 0x28, 0x2f, 0xcf, + 0xc3, 0xe4, 0xe9, 0x3f, 0xf1, 0xe1, 0x97, 0x73, 0xf3, 0x3b, 0x86, 0x10, + 0x42, 0x43, 0x43, 0xe9, 0xd7, 0xaf, 0x1f, 0x43, 0x87, 0x0e, 0x2d, 0xf2, + 0xbd, 0xc6, 0x8f, 0x1f, 0xcf, 0xab, 0xaf, 0xbe, 0xca, 0x91, 0x23, 0x47, + 0x30, 0x4d, 0x93, 0xf7, 0x86, 0xf7, 0xe3, 0xea, 0x66, 0xf5, 0xfd, 0xd3, + 0x50, 0xe7, 0x02, 0xd3, 0x34, 0x69, 0xda, 0xe5, 0x09, 0xa4, 0x94, 0x1c, + 0x3f, 0x7e, 0x9c, 0xb0, 0xb0, 0x30, 0x4a, 0xdb, 0x26, 0x8e, 0x82, 0x71, + 0x0b, 0xa7, 0xd3, 0x89, 0xc7, 0xe3, 0xe1, 0xe7, 0x2f, 0x5e, 0x27, 0xa6, + 0x7c, 0x64, 0x89, 0xdd, 0x4b, 0x51, 0x14, 0x1a, 0xdf, 0xd2, 0x57, 0x02, + 0xc2, 0xe1, 0xa8, 0x53, 0x27, 0x2f, 0x6f, 0xdb, 0x76, 0x8b, 0xc6, 0x16, + 0xd1, 0xd1, 0xb4, 0x5a, 0xe8, 0xfa, 0xdf, 0xd8, 0xed, 0x55, 0xee, 0xf0, + 0x78, 0xf6, 0x4f, 0xbf, 0xa2, 0x7a, 0x45, 0x66, 0x7f, 0xfc, 0x0a, 0x5e, + 0x5d, 0xf7, 0x2b, 0xe9, 0xda, 0x4d, 0x3b, 0xf8, 0xcf, 0xb3, 0x6f, 0xf9, + 0xad, 0x7a, 0xbf, 0x7e, 0xfd, 0x78, 0xe7, 0x9d, 0x77, 0x4e, 0x72, 0x61, + 0xcf, 0xc5, 0xe5, 0xf5, 0x78, 0x3c, 0x34, 0x6b, 0xd6, 0x8c, 0x8d, 0x1b, + 0x37, 0x12, 0xe4, 0xb4, 0xb3, 0x68, 0xea, 0x18, 0x1c, 0x76, 0xdb, 0x39, + 0x13, 0xb5, 0xeb, 0x23, 0xc3, 0x49, 0xde, 0x7f, 0x98, 0x0d, 0x1b, 0x36, + 0x90, 0x98, 0x98, 0x58, 0x2a, 0x65, 0x9c, 0x95, 0x95, 0x45, 0x44, 0x84, + 0xaf, 0x1a, 0xcb, 0xca, 0x99, 0xef, 0xe2, 0x2c, 0xc1, 0x3d, 0x00, 0x00, + 0x33, 0x7e, 0xf8, 0x95, 0x57, 0xde, 0xfd, 0xbf, 0xfc, 0xf7, 0x5b, 0x3b, + 0x42, 0xd7, 0xb7, 0x67, 0x5a, 0x54, 0x2e, 0xd3, 0x44, 0x77, 0x00, 0x79, + 0xa8, 0x6a, 0xcc, 0x93, 0x86, 0x91, 0xf2, 0x76, 0xdb, 0xab, 0x12, 0x79, + 0x77, 0xf8, 0x13, 0xfe, 0x1a, 0x66, 0x42, 0xc0, 0x4d, 0xf7, 0x3d, 0xef, + 0x4f, 0x5b, 0x14, 0x17, 0x17, 0xc7, 0xa1, 0x43, 0x87, 0x8a, 0xb5, 0x05, + 0x07, 0x0e, 0x1c, 0xa0, 0x4a, 0x95, 0x2a, 0xfe, 0xef, 0xcb, 0x67, 0xbc, + 0x43, 0x70, 0x50, 0xd1, 0xdd, 0xf9, 0xb5, 0x9b, 0x76, 0xd0, 0xe7, 0x99, + 0x31, 0x74, 0xea, 0xd4, 0x89, 0xb9, 0x73, 0xe7, 0x96, 0x4a, 0x29, 0x3f, + 0xf1, 0xc4, 0x13, 0x4c, 0x98, 0x30, 0xa1, 0xc4, 0xdc, 0xf6, 0x53, 0x79, + 0x10, 0x9d, 0xff, 0xf3, 0x22, 0x07, 0x0e, 0xa7, 0x4a, 0x55, 0x8d, 0x3c, + 0x64, 0x18, 0x19, 0x95, 0x83, 0x83, 0x6f, 0x20, 0x37, 0xf7, 0x17, 0x8b, + 0xd1, 0xa7, 0x41, 0xc0, 0x4e, 0xaf, 0x69, 0x5a, 0x42, 0x3e, 0xc9, 0xa3, + 0xd6, 0x1b, 0x46, 0xca, 0xdb, 0xfd, 0xee, 0xbb, 0x4d, 0xbe, 0xf3, 0xd2, + 0xe3, 0x7e, 0x92, 0x4b, 0x29, 0xb9, 0xaa, 0x6b, 0x7f, 0x3f, 0xc9, 0x27, + 0x4e, 0x9c, 0x58, 0xec, 0x24, 0x97, 0x52, 0x52, 0xb9, 0x72, 0x65, 0x0c, + 0xc3, 0x60, 0xc0, 0x80, 0x01, 0x00, 0xb4, 0xbe, 0x63, 0x20, 0x1e, 0x8f, + 0xb7, 0x68, 0x4b, 0x5a, 0x25, 0x34, 0x6d, 0x54, 0x0b, 0x55, 0x55, 0x98, + 0x37, 0x6f, 0x1e, 0x69, 0x69, 0x69, 0xa5, 0x4e, 0xce, 0x5e, 0xaf, 0x97, + 0x09, 0x13, 0x26, 0x00, 0x50, 0x3f, 0xa1, 0x6a, 0x89, 0x2f, 0xd5, 0x2d, + 0x58, 0x95, 0x37, 0x7f, 0xf2, 0x28, 0x6c, 0x9a, 0x2a, 0x0c, 0x23, 0xa3, + 0x92, 0xa6, 0x55, 0xba, 0xd9, 0x22, 0x79, 0x19, 0x24, 0x7a, 0x48, 0xc8, + 0xed, 0xf9, 0xf5, 0xc5, 0xd5, 0x2c, 0xc3, 0x48, 0x6f, 0xfc, 0xec, 0xa3, + 0xbd, 0x78, 0xf8, 0xae, 0x8e, 0xfe, 0xb5, 0xea, 0xba, 0x61, 0xd2, 0xfc, + 0xb6, 0x7e, 0xb8, 0xf3, 0x7c, 0xd5, 0x43, 0xd6, 0xad, 0x5b, 0xc7, 0x23, + 0x8f, 0x3c, 0x52, 0xec, 0x4a, 0x5a, 0x78, 0x7e, 0xf9, 0x9d, 0x77, 0xde, + 0x21, 0x35, 0x35, 0x15, 0x45, 0x51, 0xe8, 0xd4, 0x67, 0x28, 0xb6, 0xfc, + 0x9a, 0x68, 0x67, 0xf3, 0xb7, 0xa4, 0x94, 0xb4, 0x6a, 0x52, 0x1f, 0x80, + 0x1d, 0x3b, 0x76, 0x94, 0x3a, 0x59, 0x6f, 0xdb, 0xb6, 0x0d, 0x80, 0x76, + 0xad, 0x93, 0xb0, 0x9f, 0x47, 0x68, 0x72, 0xbe, 0x30, 0x4d, 0x93, 0x1f, + 0xa6, 0xbc, 0x06, 0x20, 0x75, 0xfd, 0xe0, 0xf7, 0x11, 0x11, 0x9d, 0xca, + 0x55, 0xa8, 0xd0, 0xdf, 0x62, 0x74, 0x59, 0x73, 0xdd, 0x85, 0x70, 0xae, + 0x93, 0xd2, 0x9d, 0x34, 0xe0, 0x81, 0xdb, 0x79, 0xa8, 0xd7, 0x2d, 0x18, + 0x86, 0x09, 0x48, 0x74, 0xdd, 0xa0, 0x55, 0xb7, 0x81, 0xfe, 0xa9, 0xb4, + 0x75, 0xeb, 0xd6, 0x91, 0x94, 0x94, 0x74, 0xd1, 0xb2, 0xaa, 0x66, 0x65, + 0x65, 0x11, 0x19, 0x19, 0x49, 0xc7, 0x76, 0x2d, 0x78, 0xe3, 0xb9, 0x87, + 0xce, 0x9a, 0x32, 0xa9, 0xa0, 0xa3, 0x68, 0xda, 0xe5, 0x71, 0x2a, 0x54, + 0x88, 0x63, 0xff, 0xfe, 0xfd, 0x97, 0x3c, 0x03, 0x6c, 0x81, 0xb7, 0xe2, + 0x9b, 0xf6, 0xf2, 0x6d, 0x4f, 0xfd, 0x75, 0xfa, 0x38, 0x42, 0x83, 0x9d, + 0x17, 0xbd, 0x1d, 0x63, 0x27, 0xcd, 0xe0, 0xf3, 0x19, 0x3f, 0x03, 0x41, + 0xe3, 0xc0, 0x65, 0x6d, 0x74, 0x29, 0x4b, 0x16, 0x5d, 0x88, 0xc8, 0xd1, + 0x52, 0xba, 0x93, 0xee, 0xef, 0xde, 0x81, 0x47, 0xee, 0xea, 0x98, 0xbf, + 0xb2, 0x4c, 0xa2, 0xa9, 0x2a, 0x4f, 0xbd, 0xfa, 0x91, 0x9f, 0xe4, 0x4b, + 0x97, 0x2e, 0xbd, 0xa8, 0x24, 0x07, 0x5f, 0x46, 0x96, 0x89, 0x13, 0x27, + 0x32, 0x77, 0xc1, 0x4a, 0x16, 0x2e, 0x5f, 0x5f, 0x64, 0x52, 0x8d, 0x7e, + 0xee, 0x61, 0x0e, 0x1c, 0x38, 0x40, 0xff, 0xfe, 0xfd, 0xfd, 0xa1, 0xc7, + 0xa5, 0x95, 0xb1, 0x60, 0xfc, 0x78, 0x5f, 0x0a, 0xea, 0x96, 0x4d, 0xea, + 0x12, 0x12, 0xe4, 0xb8, 0x24, 0xed, 0x78, 0xee, 0xd1, 0x9e, 0xf9, 0x19, + 0x69, 0xdc, 0x4f, 0xaa, 0x6a, 0xd5, 0x98, 0xa8, 0xa8, 0x1b, 0x2c, 0x56, + 0x9f, 0x02, 0x6a, 0x20, 0x3d, 0x4c, 0x5c, 0x1c, 0xe4, 0xe4, 0x44, 0x3c, + 0x2b, 0xe5, 0xf1, 0x97, 0x6e, 0x6e, 0xdb, 0x4c, 0xbe, 0x32, 0xf8, 0x7e, + 0x7f, 0xca, 0x65, 0x9b, 0xa6, 0x32, 0x7c, 0xdc, 0x17, 0xfc, 0xb0, 0x64, + 0x35, 0x00, 0x73, 0xe6, 0xcc, 0xe1, 0xc6, 0x1b, 0x6f, 0x2c, 0x31, 0x92, + 0x9f, 0xe9, 0xba, 0x57, 0x5e, 0x79, 0x25, 0xef, 0xbf, 0xff, 0x3e, 0xd3, + 0xe7, 0x2f, 0xe6, 0xba, 0x96, 0x8d, 0x29, 0x1f, 0x15, 0x7e, 0xd6, 0x36, + 0xd4, 0xaa, 0x5e, 0x89, 0xb5, 0x9b, 0x77, 0x30, 0x73, 0xce, 0x7c, 0xb6, + 0x6f, 0xdf, 0x4e, 0xf7, 0xee, 0xdd, 0x2f, 0xa9, 0xac, 0x37, 0x6f, 0xde, + 0xcc, 0xad, 0xb7, 0xde, 0x4a, 0xad, 0xf8, 0x2a, 0x4c, 0x7d, 0xe7, 0x79, + 0x2e, 0xa4, 0x02, 0xcd, 0x85, 0x40, 0x37, 0x0c, 0xc2, 0x42, 0x83, 0x59, + 0xb4, 0x62, 0x3d, 0x60, 0x5c, 0xe1, 0x72, 0x6d, 0xfb, 0xda, 0xa2, 0x75, + 0x00, 0x5b, 0x74, 0x45, 0xa9, 0x82, 0xdb, 0xdd, 0xb3, 0x9c, 0x94, 0x99, + 0xa3, 0x62, 0xcb, 0x47, 0xca, 0xd7, 0x9f, 0x7d, 0x48, 0xe4, 0x79, 0xbc, + 0x7e, 0xeb, 0xb7, 0x6b, 0xdf, 0x61, 0x66, 0xfc, 0xb0, 0x0c, 0x80, 0x19, + 0x33, 0x66, 0xd0, 0xa5, 0x4b, 0x97, 0x12, 0x21, 0x79, 0x81, 0xa5, 0xcd, + 0xce, 0xce, 0xe6, 0xe1, 0x87, 0x1f, 0x26, 0x3e, 0x3e, 0x9e, 0x8e, 0x1d, + 0x3b, 0xd2, 0xb3, 0x67, 0xcf, 0x13, 0xb2, 0xb2, 0xbc, 0xf7, 0xde, 0x7b, + 0x00, 0xdc, 0x3b, 0x68, 0x74, 0x91, 0x46, 0xe1, 0x75, 0xc3, 0x60, 0xca, + 0x98, 0xa7, 0xa9, 0x18, 0x53, 0x8e, 0xa9, 0x53, 0xa7, 0xf2, 0xdf, 0xff, + 0xfe, 0xf7, 0x92, 0xc9, 0xfa, 0xbd, 0xf7, 0xde, 0xa3, 0x61, 0xc3, 0x86, + 0x5c, 0x51, 0xa3, 0x32, 0x33, 0x27, 0xbe, 0x74, 0x49, 0x76, 0xf7, 0x15, + 0xf6, 0x2c, 0xee, 0xec, 0xd4, 0x96, 0xc8, 0xf0, 0x50, 0xa4, 0xcc, 0xbd, + 0x3d, 0x34, 0xf4, 0xba, 0xf2, 0x71, 0x71, 0x43, 0x2c, 0x66, 0x07, 0x76, + 0x8c, 0xae, 0xa5, 0x81, 0x1e, 0xb9, 0xec, 0x9b, 0xb1, 0xa2, 0x60, 0x5b, + 0x23, 0xf8, 0xaa, 0x85, 0x3c, 0xf0, 0xdf, 0x31, 0xac, 0xda, 0xf0, 0x17, + 0x2f, 0xbf, 0xfc, 0x32, 0xc3, 0x86, 0x0d, 0x2b, 0xb1, 0xb8, 0x55, 0x08, + 0xc1, 0x53, 0x4f, 0x3d, 0x75, 0xc6, 0xb4, 0x4c, 0x63, 0xc6, 0x8c, 0xe1, + 0xa9, 0xa7, 0x9e, 0xa2, 0x62, 0xc5, 0x8a, 0x1c, 0x3e, 0x7c, 0x98, 0x61, + 0x03, 0x7a, 0xd3, 0xed, 0xe6, 0x6b, 0x8a, 0x74, 0xfd, 0x03, 0x47, 0x8e, + 0xd1, 0xb9, 0x8f, 0x6f, 0x01, 0xcf, 0x77, 0xdf, 0x7d, 0x47, 0xe7, 0xce, + 0x9d, 0x2f, 0xaa, 0x84, 0xe7, 0xcc, 0x99, 0xc3, 0x6d, 0xb7, 0xdd, 0x86, + 0xa2, 0x28, 0xac, 0x9b, 0xf7, 0x01, 0xe6, 0x25, 0xb2, 0xe4, 0xff, 0x96, + 0xcb, 0xef, 0x1b, 0xb6, 0xf1, 0xe8, 0x90, 0x71, 0x80, 0xe3, 0x3d, 0xc8, + 0x1b, 0x60, 0x51, 0x3b, 0xc0, 0x88, 0xee, 0x74, 0xb6, 0xc4, 0xed, 0x5e, + 0x89, 0xaa, 0x46, 0x7d, 0x6b, 0x18, 0xe9, 0xdd, 0x67, 0x7c, 0xf0, 0x12, + 0x09, 0xd5, 0x2b, 0x9e, 0xa0, 0x04, 0xdb, 0x77, 0x1f, 0xa0, 0x67, 0xbf, + 0x91, 0xb4, 0x6b, 0xd7, 0x8e, 0x85, 0x0b, 0x17, 0x96, 0x58, 0x5b, 0x36, + 0x6d, 0xda, 0x44, 0xb3, 0x66, 0xcd, 0xc8, 0xcb, 0x3b, 0x7d, 0x95, 0x91, + 0x9a, 0x35, 0x6b, 0x92, 0x9a, 0x9a, 0xca, 0x5d, 0x77, 0xdd, 0x45, 0x74, + 0x74, 0x34, 0xa3, 0x46, 0x8d, 0x42, 0xd3, 0x54, 0x56, 0xcf, 0x1e, 0x5f, + 0x64, 0xc2, 0xac, 0xdf, 0xb2, 0x93, 0x7e, 0x2f, 0x8d, 0x27, 0x3b, 0xc7, + 0xc5, 0xfa, 0xf5, 0xeb, 0x69, 0xdc, 0xb8, 0xf1, 0x45, 0x91, 0xf5, 0xdf, + 0x7f, 0xff, 0x4d, 0xed, 0xda, 0xbe, 0x75, 0xec, 0xef, 0x0e, 0x7f, 0x9c, + 0x36, 0xcd, 0x1b, 0x95, 0x9a, 0x95, 0x7a, 0x76, 0x9b, 0xc6, 0x03, 0x4f, + 0x8f, 0x61, 0xd5, 0xfa, 0x1d, 0x6e, 0xd0, 0xad, 0x32, 0x4d, 0x81, 0xe6, + 0xba, 0xbb, 0xdd, 0x2b, 0x51, 0x94, 0xea, 0x37, 0x1a, 0x46, 0x7a, 0xf7, + 0xc7, 0x7a, 0x77, 0xa1, 0x76, 0xfc, 0x89, 0x9b, 0x9b, 0x1c, 0x76, 0x1b, + 0xef, 0x7e, 0x36, 0x0b, 0x9b, 0xcd, 0x56, 0x62, 0x24, 0x97, 0x52, 0xe2, + 0xf5, 0x7a, 0x69, 0xd4, 0xa8, 0xd1, 0x19, 0x49, 0x0e, 0xb0, 0x6b, 0xd7, + 0x2e, 0x5a, 0xb7, 0x6e, 0xcd, 0xc4, 0x89, 0x13, 0xe9, 0xd3, 0xa7, 0x8f, + 0xcf, 0x2d, 0xd7, 0x0d, 0xde, 0xff, 0xe2, 0xbb, 0x22, 0xdf, 0xaf, 0x71, + 0xbd, 0x9a, 0xfc, 0xf2, 0x7f, 0xa3, 0xa9, 0x5e, 0xb9, 0x02, 0x49, 0x49, + 0x49, 0x6c, 0xd9, 0xb2, 0xa5, 0xc4, 0xe5, 0x3c, 0x69, 0xd2, 0x24, 0x3f, + 0xc9, 0x1f, 0xbd, 0xa7, 0x33, 0xd7, 0xb6, 0x48, 0x2c, 0x55, 0xcb, 0x71, + 0xf3, 0x3c, 0x5e, 0xfa, 0xdf, 0x7f, 0x1b, 0xa0, 0x3b, 0x55, 0x35, 0xb6, + 0x8f, 0x45, 0xed, 0x00, 0x23, 0x7a, 0xeb, 0xd6, 0x3f, 0x09, 0xd3, 0xdc, + 0xf3, 0x93, 0xd3, 0x61, 0x97, 0x0f, 0xdf, 0xd5, 0xf1, 0x84, 0x52, 0x48, + 0x52, 0x4a, 0x92, 0x0f, 0x1c, 0xe1, 0xb7, 0x35, 0x9b, 0xb8, 0xf1, 0xc6, + 0x1b, 0x4b, 0x34, 0x4e, 0x7c, 0xea, 0xa9, 0xa7, 0x8a, 0x7c, 0xfc, 0x0f, + 0x3f, 0xfc, 0x00, 0xc0, 0x8f, 0x3f, 0xfe, 0xc8, 0x6d, 0xb7, 0xdd, 0x06, + 0xc0, 0x9f, 0x5b, 0x77, 0xa1, 0xa9, 0x6a, 0x91, 0xef, 0xe7, 0xb0, 0xdb, + 0x98, 0xfe, 0xc1, 0x30, 0xc2, 0x43, 0x83, 0x69, 0xd0, 0xa0, 0x01, 0x13, + 0x26, 0x4c, 0xe0, 0xd8, 0xb1, 0x63, 0x27, 0x8d, 0x15, 0x5c, 0x28, 0xbe, + 0xfc, 0xf2, 0x4b, 0x84, 0x10, 0x3c, 0xda, 0xf7, 0x11, 0xba, 0xdd, 0x72, + 0x0d, 0x9b, 0x7e, 0xfc, 0x98, 0xc7, 0x7a, 0x77, 0x29, 0x7d, 0xae, 0xa9, + 0x10, 0x5c, 0xdd, 0xb4, 0x01, 0x0e, 0xbb, 0x4d, 0x1a, 0x46, 0xea, 0xa4, + 0x90, 0x90, 0x07, 0xc2, 0x9c, 0xce, 0x16, 0x16, 0xc3, 0x03, 0x85, 0xe8, + 0x2b, 0x56, 0x74, 0xfb, 0x06, 0xe0, 0xeb, 0xf1, 0x43, 0x85, 0x52, 0x28, + 0x97, 0x19, 0xf8, 0xf6, 0x74, 0x7f, 0x33, 0x77, 0x09, 0x00, 0xfd, 0xfa, + 0xf5, 0x2b, 0xd1, 0x76, 0x7c, 0xfb, 0xed, 0xb7, 0xe7, 0x7c, 0xce, 0xe4, + 0xc9, 0x93, 0xfd, 0x1d, 0xc4, 0x86, 0xad, 0x3b, 0xb1, 0x69, 0xea, 0x39, + 0x29, 0xb6, 0xa6, 0x2a, 0x2c, 0xf9, 0x66, 0x2c, 0xa1, 0xc1, 0xbe, 0xf2, + 0x49, 0xd1, 0xd1, 0xd1, 0x44, 0x46, 0x46, 0xb2, 0x71, 0xe3, 0x46, 0xb2, + 0xb2, 0xb2, 0xce, 0xeb, 0x39, 0x74, 0x5d, 0x67, 0xeb, 0xd6, 0xad, 0x0c, + 0x1e, 0x3c, 0x18, 0x9b, 0xcd, 0x46, 0xef, 0xde, 0xbd, 0x69, 0x5c, 0xaf, + 0x26, 0x1b, 0x7f, 0xfc, 0x98, 0x61, 0xfd, 0x7b, 0xa3, 0x1b, 0xc6, 0x25, + 0x9f, 0xda, 0x3b, 0x1d, 0xb2, 0x72, 0x5c, 0xdc, 0xd9, 0xb9, 0xad, 0x00, + 0x53, 0xe6, 0xe4, 0x7c, 0xf9, 0x83, 0xdb, 0xbd, 0xca, 0x62, 0x78, 0x20, + 0x10, 0xdd, 0x6e, 0x8f, 0x8f, 0x90, 0x32, 0xbb, 0x3b, 0x20, 0xab, 0x56, + 0x8c, 0x3e, 0xe9, 0xef, 0x9a, 0xaa, 0xb2, 0xe5, 0xef, 0xbd, 0xf9, 0x2e, + 0xbe, 0xbb, 0x44, 0xdb, 0x72, 0xf8, 0xf0, 0xe1, 0xb3, 0x1e, 0x33, 0x7e, + 0xfc, 0x78, 0xa4, 0x94, 0x7c, 0xfc, 0xf1, 0xc7, 0x00, 0xac, 0x5d, 0xbb, + 0x96, 0x36, 0x6d, 0xda, 0xf8, 0x5d, 0xcf, 0xc9, 0xdf, 0xfe, 0x78, 0x4e, + 0x24, 0x12, 0x42, 0x80, 0x94, 0xac, 0x9a, 0xfd, 0x1e, 0xe3, 0x86, 0x3d, + 0x86, 0xd3, 0x61, 0xe7, 0xf8, 0xf1, 0xe3, 0x24, 0x26, 0x26, 0x12, 0x11, + 0x11, 0x81, 0x10, 0x82, 0x36, 0x6d, 0xda, 0xb0, 0x7d, 0xfb, 0x76, 0xd2, + 0xd3, 0xd3, 0x4f, 0x19, 0x56, 0x98, 0xa6, 0x49, 0x66, 0x66, 0x26, 0xfb, + 0xf7, 0xef, 0x67, 0xf4, 0xe8, 0xd1, 0xd8, 0x6c, 0x36, 0xea, 0xd7, 0xaf, + 0xcf, 0xdb, 0x6f, 0xbf, 0x4d, 0x95, 0xb8, 0x68, 0xfe, 0xfc, 0x61, 0x22, + 0x53, 0xde, 0x7a, 0x86, 0x3c, 0x8f, 0xb7, 0xd4, 0xeb, 0x83, 0x94, 0x92, + 0x27, 0x1f, 0xec, 0x86, 0x22, 0x84, 0x00, 0x6f, 0xeb, 0xb0, 0xb0, 0x1b, + 0x6a, 0x5a, 0x14, 0x0f, 0x00, 0xa2, 0x7b, 0x3c, 0x07, 0xd7, 0x01, 0x3c, + 0xd0, 0xbd, 0x83, 0x38, 0xd5, 0x36, 0x50, 0x45, 0x51, 0x48, 0xcb, 0xf0, + 0x6d, 0x6c, 0x8a, 0x8f, 0x8f, 0x2f, 0x71, 0xd7, 0xf1, 0x6c, 0x28, 0x88, + 0xc9, 0x2b, 0x54, 0xa8, 0x00, 0x40, 0x52, 0x52, 0x12, 0x80, 0xbf, 0x98, + 0xe1, 0xb2, 0x35, 0x9b, 0xcf, 0x39, 0xee, 0x15, 0x42, 0xe0, 0xf1, 0xea, + 0xb4, 0xbd, 0x2a, 0x91, 0x55, 0xb3, 0xc7, 0x53, 0xb1, 0xc2, 0x89, 0xf5, + 0xcc, 0x7e, 0xfd, 0xf5, 0x57, 0xea, 0xd4, 0xa9, 0x43, 0xb9, 0x72, 0xe5, + 0x70, 0x3a, 0x9d, 0xfe, 0xb5, 0xe2, 0x05, 0xff, 0x54, 0x55, 0x25, 0x22, + 0x22, 0x82, 0xaa, 0x55, 0xab, 0xf2, 0xdc, 0x73, 0xcf, 0xfd, 0x73, 0xde, + 0xf4, 0xb7, 0x99, 0xf5, 0xd1, 0xf0, 0xfc, 0x15, 0x85, 0x97, 0x91, 0x42, + 0x2b, 0x82, 0xd1, 0xcf, 0x3f, 0xec, 0xb3, 0xf0, 0x59, 0xcb, 0xac, 0x79, + 0xb6, 0x02, 0xa3, 0x77, 0xd9, 0xf6, 0x50, 0x4a, 0x58, 0x05, 0xd3, 0xcc, + 0x8a, 0x07, 0x78, 0xac, 0xf7, 0xa9, 0xe7, 0xc4, 0x0b, 0x32, 0x93, 0x00, + 0x27, 0xec, 0x20, 0xbb, 0x54, 0xf8, 0xe4, 0x93, 0x4f, 0xc8, 0xcb, 0xcb, + 0xe3, 0x85, 0x17, 0x5e, 0x00, 0xf0, 0x5b, 0xf3, 0x7b, 0xee, 0xb9, 0x87, + 0xf7, 0xdf, 0x7f, 0x9f, 0x0b, 0x19, 0xdb, 0x12, 0x42, 0x60, 0x18, 0x06, + 0xdf, 0x4f, 0x1e, 0x05, 0xc0, 0xee, 0x7d, 0x87, 0xe8, 0xfd, 0xe4, 0xeb, + 0xe4, 0xb8, 0xf2, 0x8a, 0x74, 0x7e, 0xf3, 0xc4, 0x3a, 0x54, 0xad, 0x14, + 0x43, 0xdd, 0x84, 0x6a, 0xf4, 0xe8, 0xd8, 0xe6, 0x9c, 0x3a, 0xb0, 0xd2, + 0x16, 0xab, 0xb7, 0x4c, 0xaa, 0x9b, 0xff, 0xcd, 0xfb, 0x00, 0xf0, 0x50, + 0x42, 0xc2, 0xf3, 0xec, 0xdc, 0xf9, 0x9a, 0x45, 0xf4, 0xcb, 0x11, 0xa6, + 0x69, 0x0e, 0xc7, 0x97, 0x86, 0x50, 0xd8, 0x34, 0xf5, 0x94, 0x0a, 0x69, + 0x18, 0x26, 0xdb, 0x76, 0xed, 0x07, 0x20, 0x3a, 0x3a, 0xba, 0x44, 0xdb, + 0xd3, 0xa5, 0x4b, 0x17, 0xe6, 0xcc, 0x39, 0x73, 0x29, 0xb0, 0x82, 0x1d, + 0x6c, 0x05, 0x28, 0x20, 0x7a, 0xab, 0x56, 0xbe, 0xca, 0x30, 0x07, 0x0f, + 0x1f, 0xc3, 0x66, 0xd3, 0x4e, 0x48, 0xf5, 0x7c, 0x3a, 0x17, 0x55, 0x51, + 0x7c, 0x59, 0x5b, 0x0d, 0xc3, 0x44, 0x55, 0x15, 0x5f, 0x06, 0x16, 0xe3, + 0x1f, 0xb7, 0x3f, 0xbe, 0x6a, 0x45, 0x7e, 0x9d, 0x3e, 0x0e, 0xdd, 0x30, + 0x30, 0x4d, 0xc9, 0x91, 0xd4, 0x74, 0x34, 0x55, 0x21, 0x32, 0x3c, 0x0c, + 0x55, 0x55, 0x10, 0x02, 0x04, 0x22, 0xbf, 0x23, 0x14, 0x17, 0x94, 0x8b, + 0xbd, 0xb4, 0x21, 0x2c, 0xb4, 0xa0, 0x4e, 0x9d, 0xa9, 0x0a, 0x11, 0xfa, + 0xc5, 0xce, 0x9d, 0xaf, 0xdd, 0x6b, 0x59, 0xf4, 0xcb, 0x10, 0xa1, 0xa1, + 0x57, 0x39, 0xb2, 0xb3, 0x7f, 0x7f, 0x14, 0xa0, 0xde, 0x15, 0xd5, 0x4e, + 0x9b, 0xbd, 0x25, 0xf9, 0xc0, 0x91, 0x8b, 0x66, 0x95, 0xfa, 0xf7, 0xef, + 0x7f, 0x56, 0xa2, 0xff, 0x1b, 0x35, 0x6b, 0x9e, 0x18, 0x42, 0xa6, 0xa6, + 0x1f, 0xc7, 0xa6, 0xaa, 0xa7, 0x25, 0xba, 0x94, 0x12, 0xbb, 0xcd, 0xc6, + 0xdc, 0x85, 0xbf, 0x33, 0x6a, 0xfc, 0x97, 0xb8, 0xf3, 0x3c, 0x78, 0xbc, + 0x3a, 0x8a, 0x22, 0x08, 0x72, 0x3a, 0x78, 0x65, 0xf0, 0x83, 0xb4, 0xbf, + 0x3a, 0x89, 0xc2, 0x15, 0x65, 0x0a, 0x76, 0xc9, 0x9d, 0xad, 0x42, 0x69, + 0x40, 0x29, 0xb5, 0xaa, 0xd2, 0xf9, 0xfa, 0x96, 0xcc, 0x5d, 0xb8, 0x52, + 0x4a, 0x99, 0x7d, 0xab, 0xe5, 0xb8, 0x5f, 0x86, 0x31, 0xba, 0xa6, 0x35, + 0x20, 0x3b, 0x7b, 0xa3, 0xdf, 0x3c, 0x7f, 0xf1, 0xf6, 0xe9, 0x0b, 0x10, + 0x16, 0xe4, 0x46, 0x2f, 0xc8, 0xef, 0x56, 0x92, 0xb8, 0xe1, 0x86, 0x1b, + 0xfc, 0x16, 0xba, 0x28, 0x18, 0x33, 0x66, 0x8c, 0x3f, 0x46, 0x07, 0x5f, + 0x66, 0xd5, 0x9c, 0x5c, 0x37, 0xa7, 0x1e, 0x6b, 0x10, 0xb8, 0xdc, 0x79, + 0x8c, 0xfb, 0x74, 0x26, 0xd7, 0xf4, 0x78, 0x92, 0x67, 0x5f, 0xff, 0x98, + 0xcc, 0xec, 0x5c, 0xff, 0xce, 0x37, 0xd3, 0x94, 0xe4, 0xe4, 0xba, 0x19, + 0x3c, 0xf2, 0x03, 0xee, 0x1b, 0x34, 0x1a, 0xc1, 0x89, 0xd3, 0x6b, 0x65, + 0xad, 0x86, 0xb8, 0x57, 0xd7, 0xe9, 0x7b, 0x4f, 0x27, 0xf0, 0x2d, 0x08, + 0x0b, 0x57, 0xd5, 0xea, 0x3d, 0x2d, 0x8b, 0x7e, 0x59, 0x21, 0x36, 0xdc, + 0x30, 0xf6, 0xdc, 0x05, 0x66, 0x43, 0x40, 0x0a, 0x21, 0xc4, 0x99, 0xe6, + 0x9e, 0xbd, 0x9e, 0x8b, 0x57, 0xed, 0x44, 0x4a, 0xc9, 0x8f, 0x3f, 0xfe, + 0x48, 0xef, 0xde, 0xbd, 0xf9, 0xdf, 0xff, 0xfe, 0x77, 0xc6, 0x63, 0xd7, + 0xae, 0x5d, 0x4b, 0x93, 0x26, 0x4d, 0x4e, 0x18, 0x57, 0x28, 0x57, 0xae, + 0x5c, 0xfe, 0xc6, 0x90, 0x7f, 0x87, 0x28, 0x92, 0xbb, 0x07, 0xbe, 0xc6, + 0x96, 0xbf, 0xf7, 0x9c, 0xf6, 0x7a, 0xed, 0xdb, 0xb7, 0x27, 0x32, 0x32, + 0x92, 0x19, 0x33, 0x66, 0xb0, 0xe1, 0xaf, 0x5d, 0xac, 0xdb, 0xb2, 0x93, + 0xa4, 0xfa, 0x09, 0x65, 0x56, 0xa9, 0x85, 0x10, 0x24, 0x54, 0xab, 0x48, + 0x8d, 0xaa, 0x15, 0x48, 0xde, 0x77, 0x04, 0xd3, 0x3c, 0xfc, 0x32, 0x50, + 0xa6, 0x37, 0xbb, 0x5c, 0x16, 0x16, 0x3d, 0x26, 0xe6, 0x90, 0x26, 0x44, + 0xf8, 0x23, 0x42, 0xa4, 0x1d, 0x97, 0x32, 0xfb, 0x43, 0x70, 0xf7, 0x03, + 0x84, 0xc3, 0x6e, 0x3b, 0xe3, 0x79, 0xd5, 0xab, 0x54, 0xf0, 0x7f, 0xf6, + 0x78, 0x3c, 0x25, 0xae, 0x5c, 0x4e, 0xa7, 0x93, 0x19, 0x33, 0x66, 0x90, + 0x9c, 0x9c, 0xcc, 0x88, 0x11, 0x23, 0x4e, 0xaa, 0x82, 0x3a, 0x6c, 0xd8, + 0x30, 0x52, 0x52, 0x52, 0x68, 0xd2, 0xa4, 0xc9, 0x49, 0x96, 0xb6, 0xc0, + 0x92, 0x17, 0xde, 0x82, 0xfa, 0xf7, 0xee, 0x03, 0x5c, 0xd9, 0xf9, 0xb1, + 0x33, 0x92, 0xfc, 0xc3, 0x0f, 0x3f, 0x64, 0xed, 0xda, 0xb5, 0xcc, 0x9a, + 0x35, 0xeb, 0x9f, 0xb1, 0x80, 0xe1, 0xef, 0x33, 0xe4, 0xcd, 0x4f, 0x99, + 0xbf, 0x68, 0x15, 0xba, 0x61, 0x10, 0x1a, 0x12, 0xc4, 0xdf, 0xbb, 0x0f, + 0xb0, 0x72, 0xdd, 0x56, 0x66, 0xfd, 0xb4, 0x1c, 0x97, 0x3b, 0x0f, 0x9b, + 0x4d, 0x2b, 0xb5, 0xf3, 0xe1, 0xc5, 0x01, 0x8f, 0x57, 0x67, 0xd4, 0x53, + 0x0f, 0xe6, 0x77, 0xc2, 0x9e, 0x6a, 0x65, 0xdd, 0xa2, 0x97, 0x4a, 0x9f, + 0xce, 0x6e, 0x6f, 0x89, 0xc7, 0xb3, 0x12, 0x55, 0x2d, 0xdf, 0xdf, 0x30, + 0x32, 0x5f, 0x06, 0x6f, 0x14, 0xc0, 0x8d, 0x6d, 0x9a, 0x4a, 0xbb, 0x4d, + 0x13, 0xf3, 0x16, 0xfe, 0x0e, 0xf8, 0x4a, 0xed, 0x2e, 0x9e, 0x36, 0xe6, + 0xb4, 0xae, 0xa9, 0x4d, 0xd3, 0x68, 0xd0, 0xe1, 0x21, 0xc0, 0x97, 0x9d, + 0x25, 0x21, 0xe1, 0xe2, 0x58, 0xb9, 0x33, 0xed, 0x8a, 0x3b, 0xdd, 0xdf, + 0x42, 0x42, 0x42, 0xc8, 0xcd, 0xcd, 0x65, 0xdb, 0xc2, 0xc9, 0xe4, 0x79, + 0x3c, 0xbc, 0xff, 0xf9, 0x1c, 0x26, 0x7e, 0x35, 0xef, 0xb4, 0xf7, 0x68, + 0xd4, 0xa8, 0x11, 0x1b, 0x37, 0x6e, 0x3c, 0xef, 0x36, 0x36, 0x4b, 0xac, + 0xcd, 0xd0, 0x27, 0xee, 0x21, 0xbe, 0x5a, 0x05, 0x02, 0x95, 0xef, 0x52, + 0x4a, 0x9a, 0x74, 0xf2, 0x55, 0x6f, 0xb2, 0xd9, 0x6a, 0xb6, 0xf5, 0x7a, + 0x77, 0x2d, 0xb5, 0x5c, 0xf7, 0x52, 0xe5, 0x7a, 0x65, 0x35, 0x02, 0xfb, + 0x57, 0x86, 0x71, 0xac, 0x61, 0x7c, 0xd5, 0x38, 0x79, 0x7f, 0xf7, 0x0e, + 0xdc, 0x73, 0xeb, 0xf5, 0x78, 0xbc, 0x5e, 0x21, 0x84, 0xe0, 0x9a, 0x66, + 0x0d, 0x79, 0xfe, 0x8d, 0x49, 0x54, 0xa9, 0x14, 0x73, 0xc6, 0xf8, 0xd3, + 0x66, 0x53, 0x09, 0x09, 0x72, 0x92, 0xe3, 0x72, 0x93, 0x96, 0x96, 0x76, + 0xd1, 0x88, 0x7e, 0xa6, 0x36, 0x9d, 0xea, 0x6f, 0x87, 0x0f, 0x1f, 0x26, + 0x37, 0x37, 0x97, 0x90, 0x60, 0x27, 0x9a, 0xaa, 0xf0, 0xd6, 0x94, 0xd9, + 0x7c, 0x34, 0x75, 0xfe, 0x19, 0xef, 0x51, 0xbe, 0x7c, 0xf9, 0xb3, 0xb6, + 0xe3, 0x9a, 0xe6, 0x0d, 0xa8, 0x5c, 0xa1, 0x3c, 0x15, 0x63, 0xcb, 0x91, + 0x95, 0xed, 0x62, 0xf7, 0xbe, 0xc3, 0x2c, 0x5b, 0xb3, 0x19, 0xaf, 0x57, + 0x67, 0xcd, 0x9f, 0xdb, 0xe9, 0xda, 0xf7, 0x25, 0x86, 0xf6, 0xbf, 0x87, + 0x1e, 0x1d, 0xaf, 0x0d, 0x48, 0xeb, 0xae, 0x69, 0x2a, 0x6d, 0x5b, 0x26, + 0xb2, 0x64, 0xe5, 0x9f, 0x78, 0xbd, 0x07, 0x6f, 0x01, 0x2c, 0xa2, 0x97, + 0x9a, 0x58, 0x42, 0x89, 0x78, 0x3b, 0x2f, 0x6f, 0xf3, 0x93, 0x80, 0x7c, + 0xee, 0xb1, 0x5e, 0xf4, 0xe9, 0x71, 0x93, 0xc8, 0x75, 0xe7, 0x91, 0xe3, + 0x72, 0xfb, 0x7b, 0xe9, 0xc4, 0xfc, 0x4c, 0xa3, 0x7b, 0xf6, 0x1d, 0x39, + 0xe3, 0xb5, 0x0c, 0xc3, 0x24, 0xbe, 0x5a, 0x1c, 0x9b, 0xb6, 0x25, 0x97, + 0xf8, 0xca, 0xb8, 0x0b, 0xc1, 0xf2, 0xe5, 0xcb, 0x01, 0x70, 0x3a, 0xec, + 0x64, 0x64, 0xe6, 0x9c, 0x95, 0xe4, 0x00, 0x8b, 0x17, 0x2f, 0x3e, 0xed, + 0xdf, 0x6e, 0xbb, 0xb1, 0x25, 0x0f, 0xdf, 0x75, 0x0b, 0x5e, 0x8f, 0x8e, + 0x61, 0x9a, 0x7e, 0x12, 0xab, 0xaa, 0xc2, 0x93, 0xff, 0xe9, 0xca, 0x0f, + 0x8b, 0xd7, 0x30, 0x6d, 0xee, 0x52, 0xd2, 0x32, 0xb2, 0x18, 0xf9, 0xde, + 0x97, 0x54, 0xad, 0x18, 0x43, 0x8b, 0xc6, 0x75, 0x02, 0x6e, 0xd0, 0xce, + 0x30, 0x4c, 0x9e, 0x7e, 0xe4, 0x4e, 0x96, 0xac, 0xfc, 0x13, 0xf0, 0x0e, + 0x00, 0x9e, 0xb7, 0x62, 0xf4, 0x4b, 0x8c, 0x4a, 0x95, 0x1e, 0x09, 0x03, + 0x5b, 0xa6, 0x69, 0x1e, 0x7f, 0x52, 0x55, 0x15, 0x7e, 0xfa, 0xfc, 0x75, + 0xd1, 0xab, 0xcb, 0x75, 0xe4, 0xba, 0xf3, 0x4e, 0xb2, 0x88, 0x7e, 0xeb, + 0x73, 0x16, 0xbd, 0xcc, 0xf3, 0x78, 0xe9, 0x76, 0xd3, 0x35, 0x27, 0xc4, + 0xc0, 0xa5, 0x11, 0xd9, 0xd9, 0xd9, 0xfe, 0x50, 0x64, 0xe5, 0xba, 0xad, + 0x17, 0x74, 0xad, 0x47, 0xef, 0xe9, 0xc4, 0xbd, 0xb7, 0xb7, 0xc7, 0xe5, + 0xca, 0xf3, 0xcf, 0x3a, 0x14, 0xac, 0x82, 0x33, 0x4d, 0x89, 0xc7, 0xa3, + 0xd3, 0xae, 0x55, 0x63, 0x3e, 0x1b, 0x33, 0x98, 0x5b, 0xae, 0x6b, 0x8e, + 0x10, 0xd0, 0x77, 0xc8, 0xb8, 0xcb, 0x62, 0x89, 0xeb, 0xf9, 0xa0, 0x42, + 0x74, 0x41, 0x31, 0x09, 0xc3, 0x49, 0x19, 0x46, 0xa9, 0xd0, 0xfe, 0xa0, + 0xa0, 0x16, 0xe5, 0x0f, 0x1e, 0xfc, 0x24, 0x13, 0xbc, 0x61, 0x35, 0xab, + 0x55, 0xe4, 0x8f, 0xb9, 0x13, 0x88, 0x8d, 0x3e, 0x7d, 0xb5, 0x8f, 0xbc, + 0xfc, 0x22, 0x88, 0x91, 0xe1, 0xa1, 0x9c, 0xad, 0x74, 0x67, 0xd3, 0x46, + 0xb5, 0xfc, 0x16, 0xb0, 0xb4, 0xba, 0xa7, 0x93, 0x26, 0x4d, 0x02, 0xe0, + 0xaa, 0xa4, 0xba, 0x3c, 0x31, 0x6c, 0xfc, 0x79, 0x5f, 0xa7, 0x5d, 0xeb, + 0x24, 0x6e, 0xb8, 0x3a, 0xa9, 0x48, 0xe1, 0x83, 0xc7, 0xa3, 0xf3, 0xd0, + 0x9d, 0x1d, 0x18, 0x33, 0xc4, 0xb7, 0x5c, 0xf4, 0x91, 0x21, 0xe3, 0x02, + 0x52, 0xc1, 0x1d, 0x76, 0x3b, 0x8d, 0xea, 0xc4, 0x03, 0x28, 0x9a, 0x56, + 0xf9, 0x25, 0x8b, 0xe8, 0x97, 0x08, 0xe1, 0xe1, 0xdd, 0x6c, 0x2e, 0xd7, + 0x1f, 0x47, 0xc0, 0x94, 0xcd, 0x1a, 0xd5, 0x66, 0xf6, 0x47, 0x2f, 0x23, + 0xcd, 0xd3, 0x13, 0x52, 0x11, 0x22, 0x3f, 0x3f, 0x18, 0xd4, 0xa9, 0x59, + 0xe5, 0xac, 0x66, 0xbd, 0x76, 0x7c, 0x15, 0x92, 0xea, 0x27, 0x30, 0x79, + 0xf2, 0xe4, 0x52, 0x59, 0xce, 0xc8, 0xeb, 0xf5, 0xb2, 0x74, 0xe9, 0x52, + 0xa2, 0x22, 0x42, 0x19, 0xda, 0xef, 0x6e, 0x34, 0xed, 0xfc, 0x5e, 0x49, + 0xab, 0x2b, 0xeb, 0xd1, 0xff, 0xbe, 0x73, 0xcb, 0x36, 0x23, 0x81, 0x9a, + 0x55, 0xe3, 0x48, 0xaa, 0x9f, 0xc0, 0xd6, 0x1d, 0x7b, 0x51, 0x14, 0x11, + 0x70, 0xb1, 0xba, 0x69, 0x9a, 0xbc, 0x3c, 0xe8, 0x3e, 0x00, 0xa9, 0xeb, + 0x07, 0x86, 0x3b, 0x1c, 0x8d, 0x15, 0x8b, 0xe8, 0x17, 0x11, 0x42, 0x44, + 0x73, 0xeb, 0xad, 0x92, 0xec, 0xec, 0x05, 0x3b, 0xc1, 0x50, 0x54, 0x55, + 0x11, 0x1f, 0xbf, 0x3e, 0xa8, 0x48, 0x15, 0x48, 0xd5, 0xfc, 0x39, 0xf1, + 0x06, 0xb5, 0xaa, 0x9f, 0xf5, 0x3e, 0xee, 0x3c, 0x0f, 0x8f, 0xf7, 0xee, + 0xc2, 0xce, 0x9d, 0x3b, 0x59, 0xbb, 0x76, 0x6d, 0xa9, 0x52, 0x64, 0x21, + 0x84, 0x7f, 0x9f, 0xfc, 0xfd, 0x77, 0x74, 0xc0, 0x94, 0x92, 0x1b, 0xaf, + 0x69, 0x7a, 0x8e, 0x63, 0x1a, 0x82, 0x9e, 0x9d, 0xaf, 0xe5, 0xbf, 0x0f, + 0x77, 0xc3, 0x34, 0xcf, 0xfd, 0xd9, 0x0c, 0xd3, 0xe4, 0xba, 0x96, 0x89, + 0x78, 0xbd, 0x3a, 0xbf, 0xfc, 0xb6, 0x2e, 0x20, 0x17, 0xd7, 0xd4, 0xa8, + 0x12, 0xe7, 0x0f, 0xf4, 0xf2, 0xf2, 0xf6, 0xd8, 0x2d, 0xa2, 0x5f, 0x54, + 0x6b, 0x96, 0xca, 0x9c, 0x39, 0xa1, 0x43, 0x4c, 0xf3, 0x78, 0x15, 0x40, + 0xdc, 0x7b, 0xfb, 0x0d, 0xfe, 0x58, 0xf2, 0x8c, 0xe4, 0x50, 0x04, 0xab, + 0x37, 0xfc, 0x05, 0x40, 0xb5, 0x4a, 0x15, 0x8a, 0x44, 0xa6, 0xe6, 0xf9, + 0x9b, 0x1c, 0x3a, 0x77, 0xee, 0x5c, 0xaa, 0x14, 0xf9, 0x8d, 0x37, 0xde, + 0x60, 0xc9, 0x92, 0x25, 0xf9, 0xb1, 0x75, 0x67, 0xa4, 0x94, 0xd4, 0x8a, + 0x2f, 0x7a, 0xf9, 0xef, 0xd6, 0x4d, 0xeb, 0x33, 0xe7, 0xe3, 0xe1, 0xf4, + 0xe8, 0xd8, 0xe6, 0x82, 0xa6, 0xc8, 0x4c, 0xe9, 0x8b, 0xe5, 0x9f, 0x1a, + 0x39, 0x31, 0x20, 0x47, 0xdf, 0x15, 0x45, 0x30, 0xed, 0xbd, 0x17, 0xf2, + 0x3f, 0x6b, 0x9f, 0x59, 0x44, 0xbf, 0x88, 0x50, 0xd5, 0x4a, 0x4d, 0x20, + 0x67, 0x94, 0x8f, 0x8b, 0x82, 0x41, 0xff, 0xe9, 0x56, 0x34, 0x2b, 0x88, + 0xf0, 0x0f, 0x1c, 0x35, 0x4d, 0xac, 0x55, 0xa4, 0x73, 0x6c, 0x9a, 0x4a, + 0xbb, 0x56, 0x8d, 0x39, 0x74, 0xe8, 0x10, 0xbf, 0xfd, 0xf6, 0x5b, 0xa9, + 0x11, 0xfe, 0x8b, 0x2f, 0xbe, 0x08, 0xc0, 0xf0, 0x27, 0xef, 0xc3, 0x9d, + 0xe7, 0xc1, 0x34, 0x25, 0xad, 0xaf, 0xac, 0x7f, 0xda, 0xe3, 0x63, 0xcb, + 0x47, 0x10, 0x1b, 0x1d, 0xc9, 0xd5, 0x4d, 0x1b, 0x30, 0xed, 0xbd, 0xe7, + 0x18, 0xfc, 0x9f, 0xdb, 0x71, 0xe7, 0x5d, 0xd8, 0x42, 0x20, 0x21, 0x04, + 0x5b, 0xf3, 0xf7, 0xec, 0xd7, 0xa8, 0x52, 0x21, 0x60, 0x97, 0xcb, 0xd6, + 0xaa, 0x51, 0x29, 0xdf, 0x95, 0x4f, 0xed, 0xa9, 0x28, 0x11, 0x21, 0x65, + 0x8d, 0xe8, 0x97, 0x6c, 0x7a, 0xcd, 0x34, 0x33, 0xee, 0xcf, 0x0f, 0x13, + 0x45, 0x4c, 0xf9, 0xc8, 0x22, 0x5b, 0x24, 0x21, 0xe0, 0x78, 0x56, 0x2e, + 0x00, 0x41, 0x45, 0xac, 0xda, 0xa9, 0xeb, 0x06, 0xcf, 0x3e, 0xda, 0x8b, + 0x45, 0x2b, 0x36, 0x70, 0xf3, 0xcd, 0x37, 0x9f, 0x77, 0xf6, 0x95, 0xe2, + 0x44, 0xd7, 0xae, 0x5d, 0xf1, 0x78, 0x3c, 0x74, 0xbb, 0xb9, 0x0d, 0x77, + 0xdc, 0x7c, 0x0d, 0xba, 0x61, 0x70, 0xe0, 0x70, 0x2a, 0xbd, 0xfa, 0xfb, + 0xb6, 0x99, 0xf6, 0xec, 0x74, 0x2d, 0xb7, 0xdf, 0xd4, 0x1a, 0x09, 0xa8, + 0x8a, 0xa0, 0x24, 0x76, 0x98, 0x15, 0x58, 0xef, 0x9f, 0x7f, 0x5d, 0x07, + 0xc0, 0xcb, 0x83, 0xef, 0x0f, 0x58, 0x45, 0x57, 0x55, 0x95, 0xe6, 0x8d, + 0xeb, 0xb0, 0x7a, 0xc3, 0x36, 0x4c, 0x33, 0xcb, 0x69, 0xb3, 0xd5, 0xcf, + 0xf1, 0x7a, 0xb7, 0x94, 0x19, 0xa2, 0x5f, 0x42, 0xd7, 0xdd, 0x6c, 0x5d, + 0x10, 0x37, 0x35, 0xaa, 0x53, 0xe3, 0x8c, 0x2e, 0xa3, 0x94, 0x12, 0x9b, + 0xa6, 0x32, 0x6d, 0xce, 0x62, 0x6e, 0x7d, 0x68, 0x18, 0x3b, 0x92, 0x0f, + 0xa0, 0xa9, 0x2a, 0x61, 0x21, 0x45, 0x4f, 0xf6, 0x59, 0x31, 0x36, 0x8a, + 0xb7, 0x5e, 0xe8, 0x4b, 0x76, 0x76, 0x36, 0x0d, 0x1b, 0x36, 0x3c, 0x41, + 0xd1, 0x2f, 0x36, 0x7a, 0xf4, 0xe8, 0xc1, 0xec, 0xd9, 0xb3, 0xb9, 0x2a, + 0xa9, 0x1e, 0x2f, 0x0f, 0xba, 0x17, 0xdd, 0x30, 0x38, 0x9e, 0x95, 0xc3, + 0x1d, 0x8f, 0xbd, 0x82, 0x10, 0x82, 0x4f, 0x5e, 0x1f, 0x48, 0x8f, 0x4e, + 0x6d, 0x50, 0x55, 0x05, 0x9b, 0xa6, 0xa2, 0x28, 0x4a, 0x89, 0x6c, 0x23, + 0xcd, 0xc9, 0x75, 0xf3, 0xe4, 0x88, 0x89, 0x80, 0x6f, 0x06, 0x23, 0xb1, + 0x4e, 0x7c, 0x40, 0x2f, 0x8b, 0x9d, 0x30, 0xa2, 0x3f, 0x3e, 0xe3, 0xe2, + 0x58, 0x5b, 0x96, 0x48, 0x7e, 0xc9, 0x2c, 0x7a, 0x6c, 0xec, 0x22, 0xf5, + 0xe8, 0xd1, 0x76, 0xcd, 0x0b, 0xbe, 0x8f, 0x7c, 0xea, 0xc1, 0xb3, 0x2e, + 0x19, 0xed, 0xda, 0x77, 0x38, 0x7f, 0xef, 0x3e, 0x50, 0xa8, 0x87, 0x56, + 0xce, 0x29, 0x2e, 0x35, 0x4d, 0xc9, 0xf5, 0xad, 0x93, 0xd0, 0x9d, 0x41, + 0x55, 0xf5, 0x00, 0x00, 0x20, 0x00, 0x49, 0x44, 0x41, 0x54, 0x34, 0x95, + 0xcd, 0x9b, 0x37, 0xd3, 0xb6, 0x6d, 0x5b, 0x96, 0x2c, 0x59, 0x72, 0xd1, + 0xeb, 0x98, 0xf5, 0xec, 0xd9, 0x93, 0xe9, 0xd3, 0xa7, 0x03, 0xf0, 0xc1, + 0xc8, 0xfe, 0xe8, 0xba, 0x71, 0x42, 0xae, 0xf6, 0x67, 0xfa, 0xf6, 0xc8, + 0x9f, 0x36, 0xa4, 0xc4, 0x2a, 0xc8, 0xa8, 0x8a, 0x82, 0xcd, 0xa6, 0xd1, + 0xfd, 0x71, 0x9f, 0xf7, 0xa0, 0x2a, 0x0a, 0xf3, 0x27, 0x8f, 0x2a, 0x15, + 0x35, 0xdd, 0x4a, 0x12, 0x36, 0x9b, 0x86, 0xa2, 0x08, 0x61, 0x9a, 0xae, + 0x6a, 0x9a, 0x56, 0xa9, 0xb6, 0xae, 0x1f, 0xdc, 0x6e, 0x59, 0xf4, 0x12, + 0x44, 0x6a, 0x6a, 0xef, 0x98, 0x82, 0xcf, 0x0e, 0xbb, 0x8d, 0xf0, 0xd0, + 0xd3, 0x5b, 0x66, 0xaf, 0x6e, 0x90, 0xd4, 0xf1, 0xd1, 0x13, 0x48, 0x0e, + 0xbe, 0x55, 0x64, 0x05, 0x0b, 0x42, 0x8a, 0x1a, 0x8b, 0x0a, 0x21, 0x98, + 0x3b, 0x69, 0x24, 0xe0, 0xab, 0xbb, 0x16, 0x15, 0x15, 0x45, 0x6a, 0x6a, + 0xea, 0x45, 0xf0, 0x5e, 0x7c, 0x3d, 0x52, 0xc7, 0x8e, 0x1d, 0xf9, 0xe6, + 0x9b, 0x6f, 0x00, 0x98, 0xfa, 0xde, 0x10, 0x94, 0x7c, 0xc2, 0xbd, 0xf5, + 0xb1, 0x8f, 0xf8, 0x2d, 0x1a, 0xd7, 0xa1, 0x65, 0x93, 0x3a, 0xc5, 0x34, + 0x00, 0xa5, 0x60, 0xb7, 0x69, 0x38, 0xec, 0x36, 0x82, 0x83, 0x1c, 0xa4, + 0x67, 0x66, 0xf3, 0xdd, 0x82, 0xdf, 0x59, 0xb9, 0x7e, 0x1b, 0x43, 0xc7, + 0x7e, 0x4e, 0x97, 0x87, 0x86, 0x03, 0xbe, 0xe9, 0xc7, 0xf9, 0x9f, 0x8d, + 0x22, 0xc8, 0x69, 0x2f, 0x13, 0xdb, 0x59, 0xbf, 0x9d, 0xf0, 0x52, 0xfe, + 0x2b, 0xf1, 0xf4, 0xb5, 0x2c, 0x7a, 0x89, 0xc3, 0x73, 0x45, 0xc1, 0x27, + 0xbb, 0xcd, 0x86, 0x71, 0x9a, 0x69, 0x21, 0xa7, 0xc3, 0xce, 0x13, 0xc3, + 0xde, 0x3b, 0x8d, 0x85, 0xf6, 0x6d, 0xe9, 0x3c, 0x57, 0x4f, 0x33, 0x2e, + 0x26, 0x8a, 0xc5, 0xd3, 0xc6, 0x70, 0xfd, 0xdd, 0xcf, 0x90, 0x91, 0x91, + 0x41, 0x6c, 0x6c, 0x2c, 0xd3, 0xa6, 0x4d, 0xa3, 0x67, 0xcf, 0x9e, 0xc5, + 0x6e, 0xd1, 0x0a, 0xae, 0x97, 0x9d, 0x9d, 0x4d, 0x8b, 0x16, 0x2d, 0xf8, + 0xeb, 0x2f, 0xdf, 0x6c, 0xc1, 0x0b, 0xfd, 0xee, 0xa6, 0x41, 0xad, 0x1a, + 0x98, 0xa6, 0x89, 0x4d, 0x55, 0xd9, 0xbd, 0xcf, 0x57, 0x97, 0x7d, 0xe0, + 0x83, 0xb7, 0x61, 0x18, 0xe7, 0x5f, 0xf9, 0x44, 0xd3, 0x54, 0x34, 0x55, + 0x61, 0xc1, 0xf2, 0x0d, 0xfc, 0xb0, 0xe4, 0x0f, 0x8e, 0xa5, 0x67, 0x62, + 0x9a, 0x26, 0x59, 0x39, 0xae, 0x93, 0x56, 0xbe, 0xd5, 0x4b, 0xa8, 0x46, + 0xbf, 0x07, 0x6e, 0xe3, 0xfa, 0x56, 0x49, 0xb8, 0xf3, 0x3c, 0x65, 0x66, + 0xcf, 0x7a, 0x42, 0xb5, 0x38, 0xae, 0x6f, 0x9d, 0x24, 0x16, 0x2e, 0xdf, + 0x30, 0x18, 0x78, 0x2a, 0x2c, 0x2c, 0x86, 0xac, 0xac, 0x14, 0x8b, 0xe8, + 0x25, 0x01, 0xd3, 0xcc, 0xaa, 0x4b, 0xfe, 0x40, 0xdc, 0x83, 0x77, 0xde, + 0x84, 0x79, 0x1a, 0xcb, 0xbc, 0xfa, 0xcf, 0x6d, 0xfc, 0xb6, 0x66, 0xf3, + 0x69, 0x2d, 0xd6, 0xf9, 0x86, 0x93, 0x91, 0xe1, 0xa1, 0x6c, 0xf8, 0xfe, + 0x43, 0x7e, 0x59, 0xb6, 0x96, 0x41, 0x23, 0x3f, 0xa4, 0x57, 0xaf, 0x5e, + 0x7c, 0xf5, 0xd5, 0x57, 0xcc, 0x9e, 0x3d, 0xbb, 0x58, 0x9f, 0x53, 0x08, + 0xc1, 0xc0, 0x81, 0x03, 0x79, 0xf7, 0xdd, 0x77, 0xfd, 0xbf, 0x7d, 0xf4, + 0xda, 0x20, 0x5a, 0x35, 0xa9, 0xe7, 0x5f, 0x2f, 0xa0, 0x1b, 0x06, 0xc9, + 0xfb, 0x8f, 0x10, 0x1a, 0xec, 0x24, 0x32, 0x3c, 0x04, 0x97, 0xfb, 0xdc, + 0x46, 0xd1, 0x05, 0xbe, 0xba, 0x6c, 0x53, 0xbf, 0x5b, 0xc2, 0xb6, 0x5d, + 0xfb, 0x49, 0xde, 0x7f, 0xe4, 0xa4, 0x6b, 0xdc, 0x78, 0xe3, 0x8d, 0x6c, + 0xd8, 0xb0, 0x81, 0xa3, 0x47, 0x8f, 0x02, 0x50, 0x37, 0xa1, 0x2a, 0x73, + 0x26, 0x8d, 0x20, 0x27, 0xd7, 0x4d, 0x41, 0x7d, 0xba, 0xb2, 0x84, 0xe7, + 0x1f, 0xbf, 0x8b, 0x85, 0xcb, 0xd7, 0x23, 0x84, 0xc3, 0x9d, 0x95, 0x95, + 0xe2, 0xd4, 0xb4, 0x2a, 0xe8, 0xfa, 0x7e, 0xcb, 0x75, 0x2f, 0xf6, 0x9b, + 0x2a, 0x0e, 0x4f, 0xc1, 0x40, 0xdc, 0xdd, 0xb7, 0xb6, 0x3b, 0xed, 0x71, + 0x87, 0x8e, 0xa6, 0x9d, 0xc5, 0x5a, 0x9e, 0x7f, 0x1b, 0x0c, 0xc3, 0xe4, + 0xba, 0x56, 0x8d, 0xf9, 0x6e, 0xd2, 0x2b, 0x34, 0x4f, 0xac, 0xc3, 0x9c, + 0x39, 0x73, 0x08, 0x0a, 0x0a, 0xe2, 0xab, 0xaf, 0xbe, 0xba, 0x60, 0x17, + 0x1d, 0xe0, 0xcd, 0x37, 0xdf, 0xc4, 0x6e, 0xb7, 0xfb, 0x49, 0xde, 0xa3, + 0xd3, 0xb5, 0x6c, 0x5f, 0x38, 0x99, 0x16, 0x8d, 0xeb, 0x9c, 0x50, 0x79, + 0xb4, 0xe0, 0x94, 0xa8, 0x88, 0xb0, 0x73, 0xca, 0xb8, 0x5a, 0x90, 0xf8, + 0x72, 0xc6, 0x0f, 0xbf, 0xd1, 0x6b, 0xc0, 0x68, 0x66, 0xfd, 0xb4, 0x82, + 0xad, 0x3b, 0xf6, 0xe1, 0x72, 0x7b, 0x88, 0x8a, 0x8a, 0x62, 0xe6, 0xcc, + 0x99, 0x6c, 0xd8, 0xb0, 0x01, 0x29, 0x25, 0x3f, 0xfd, 0xf4, 0x13, 0x47, + 0x8e, 0x1c, 0xa1, 0x6b, 0xd7, 0xae, 0x80, 0x6f, 0x11, 0x91, 0xc7, 0xa3, + 0x53, 0x56, 0x11, 0x5b, 0x3e, 0x92, 0xa4, 0xfa, 0x09, 0x48, 0x99, 0xe7, + 0x50, 0x94, 0x88, 0xa9, 0x81, 0x4e, 0xf2, 0x4b, 0x46, 0x74, 0x29, 0x95, + 0x15, 0x05, 0x9f, 0x33, 0xb3, 0x73, 0x4f, 0x7b, 0xdc, 0xad, 0x37, 0xb4, + 0xa4, 0x6e, 0xc2, 0xa9, 0x73, 0x06, 0x64, 0xe5, 0xe4, 0x16, 0xb9, 0xb2, + 0xc9, 0x99, 0x2c, 0x6e, 0x95, 0xb8, 0x68, 0xbe, 0x78, 0xfb, 0x59, 0x46, + 0x3d, 0xdd, 0x07, 0xa4, 0xc9, 0x3d, 0xf7, 0xdc, 0x83, 0xd3, 0xe9, 0x64, + 0xdc, 0xb8, 0x71, 0x1c, 0x38, 0x70, 0xa0, 0xc8, 0xd7, 0x5a, 0xbd, 0x7a, + 0x35, 0x53, 0xa7, 0x4e, 0x65, 0xc4, 0x88, 0x97, 0x11, 0x42, 0xf0, 0xcc, + 0x33, 0xcf, 0xe0, 0xf5, 0x7a, 0xb9, 0xe9, 0xda, 0x66, 0xfc, 0xf1, 0xdd, + 0x04, 0x5e, 0xec, 0xdf, 0xdb, 0xbf, 0x03, 0xef, 0xdf, 0xee, 0x36, 0x80, + 0xdb, 0x53, 0x74, 0xf7, 0x59, 0x4a, 0x89, 0xc3, 0x6e, 0x63, 0xfc, 0x94, + 0xef, 0xf8, 0x6a, 0x8e, 0x6f, 0x0d, 0x7f, 0xe7, 0xce, 0x9d, 0xfd, 0x89, + 0x2e, 0xaa, 0x56, 0xad, 0x4a, 0xd7, 0xae, 0x5d, 0x49, 0x4c, 0x4c, 0x3c, + 0xe1, 0x9c, 0x99, 0x33, 0x67, 0xd2, 0xaa, 0x55, 0x2b, 0x92, 0xf7, 0x1f, + 0x61, 0xe8, 0x5b, 0x93, 0x03, 0x2a, 0x21, 0xe4, 0xb9, 0x76, 0xc8, 0x43, + 0xfb, 0xdf, 0x43, 0x70, 0x90, 0x03, 0xd3, 0x3c, 0xde, 0xcb, 0x66, 0xab, + 0x74, 0x65, 0xa0, 0x3f, 0xf3, 0x25, 0x79, 0xd3, 0xaa, 0x5a, 0x3d, 0xc6, + 0x30, 0xf6, 0x1c, 0x05, 0x18, 0xda, 0xef, 0x6e, 0xba, 0x77, 0xbc, 0xf6, + 0xb4, 0x2f, 0x24, 0x2b, 0xc7, 0x45, 0x87, 0x7b, 0x9f, 0x3b, 0x69, 0x61, + 0x88, 0x10, 0x82, 0x4d, 0x3f, 0x7d, 0x7c, 0xd6, 0x8c, 0xa9, 0xe7, 0xf2, + 0xf2, 0x83, 0x83, 0x1c, 0xbc, 0x38, 0x76, 0x8a, 0xbf, 0xba, 0x0b, 0x40, + 0xe5, 0xca, 0x95, 0xe9, 0xd0, 0xa1, 0x03, 0x1e, 0x8f, 0x87, 0xcc, 0xcc, + 0x4c, 0x72, 0x72, 0x72, 0xa8, 0x56, 0xad, 0x1a, 0xd5, 0xaa, 0x55, 0xe3, + 0xef, 0xbf, 0xff, 0x66, 0xc1, 0x82, 0x05, 0x7e, 0x97, 0x18, 0x20, 0x38, + 0xc8, 0x41, 0x9b, 0x16, 0x8d, 0xb8, 0xe1, 0xea, 0x2b, 0xb9, 0xe3, 0x96, + 0x6b, 0xc8, 0xce, 0x71, 0x9f, 0x36, 0x34, 0xf1, 0x79, 0x37, 0x82, 0xc4, + 0x9b, 0x7d, 0xe3, 0x42, 0x73, 0x3e, 0x19, 0x8e, 0xe7, 0x2c, 0xbb, 0xc8, + 0x0a, 0x48, 0xfe, 0xc6, 0xc4, 0xe9, 0xfc, 0xba, 0x66, 0x33, 0xd7, 0x5e, + 0x7b, 0x2d, 0xf3, 0xe6, 0xcd, 0x23, 0x34, 0xd4, 0x37, 0x52, 0x5f, 0xa5, + 0x4a, 0x15, 0x0e, 0x1e, 0x3c, 0x88, 0xc7, 0xe3, 0x39, 0x29, 0x7d, 0x96, + 0x2f, 0x53, 0xac, 0x41, 0x74, 0x74, 0x34, 0xc7, 0x8f, 0x1f, 0xe7, 0x87, + 0x29, 0xaf, 0x51, 0x21, 0x3a, 0xb2, 0xcc, 0xb9, 0xee, 0x3e, 0x1d, 0x54, + 0xb8, 0x7b, 0xc0, 0x6b, 0x6c, 0xdc, 0xb6, 0x5b, 0x82, 0x7d, 0x35, 0x78, + 0xae, 0x52, 0x94, 0x18, 0x4c, 0x33, 0xc5, 0x22, 0x7a, 0xf1, 0xc2, 0x6e, + 0x82, 0x47, 0xb4, 0x6e, 0x5a, 0x9f, 0xf7, 0x47, 0xf4, 0x3f, 0xe3, 0xf4, + 0x5a, 0x4a, 0xda, 0x71, 0x3a, 0xdc, 0xfb, 0xdc, 0x49, 0x96, 0x70, 0xc3, + 0xfc, 0x0f, 0xf1, 0xea, 0x46, 0xb1, 0xb7, 0x2c, 0x2b, 0x3b, 0x97, 0xc7, + 0x87, 0xbd, 0xc7, 0x5f, 0x3b, 0xf6, 0xe1, 0xd5, 0xf5, 0x13, 0xb7, 0xc6, + 0xe6, 0x23, 0x24, 0xc8, 0x89, 0xcd, 0xa6, 0x51, 0xa3, 0x4a, 0x05, 0x12, + 0xeb, 0xc6, 0xd3, 0xe9, 0xfa, 0x96, 0xb4, 0x68, 0x5c, 0x07, 0x5d, 0x37, + 0xc8, 0x75, 0xe7, 0x15, 0x79, 0x3e, 0x3a, 0xc8, 0xe9, 0xe0, 0xfa, 0xbb, + 0x9f, 0xe6, 0xc0, 0xe1, 0x54, 0xee, 0xeb, 0xd6, 0x9e, 0x5b, 0x6f, 0x68, + 0x79, 0x46, 0x92, 0xdb, 0x34, 0x8d, 0xc9, 0xd3, 0x7f, 0x62, 0xf6, 0xcf, + 0x2b, 0x19, 0x3c, 0x78, 0x30, 0x6f, 0xbd, 0xf5, 0xd6, 0x09, 0x83, 0x88, + 0x5b, 0xb7, 0x6e, 0xa5, 0x61, 0xc3, 0x86, 0xfe, 0x64, 0x98, 0xa7, 0xca, + 0x75, 0xbf, 0x76, 0xed, 0x5a, 0x9a, 0x35, 0x6b, 0x46, 0xed, 0x9a, 0x55, + 0xf8, 0xdf, 0x07, 0x2f, 0x9d, 0x50, 0xaf, 0xae, 0x2c, 0x59, 0xf5, 0xa8, + 0x88, 0x30, 0xda, 0xf4, 0x18, 0xc4, 0xbe, 0x43, 0x29, 0x08, 0x11, 0xbc, + 0x4a, 0xca, 0xdc, 0xab, 0x2c, 0x8b, 0x5e, 0xdc, 0x37, 0x16, 0xce, 0x5f, + 0xa5, 0x74, 0x5f, 0x1d, 0x1d, 0x15, 0xc1, 0x2f, 0x5f, 0x8e, 0x3e, 0xeb, + 0xf1, 0xcb, 0x56, 0x6f, 0xe2, 0x95, 0x77, 0xbf, 0xc0, 0x95, 0xe7, 0x25, + 0x2b, 0x3b, 0x17, 0x87, 0xdd, 0xc6, 0x1f, 0xdf, 0x4d, 0x28, 0x11, 0x25, + 0xf5, 0xe5, 0x4d, 0x57, 0x08, 0x76, 0x3a, 0x48, 0xcf, 0xcc, 0xa2, 0x5c, + 0x64, 0x38, 0x19, 0x99, 0x39, 0x2c, 0x5c, 0xbe, 0x8e, 0xc8, 0xf0, 0x50, + 0x74, 0x5d, 0xa7, 0x7d, 0xeb, 0x26, 0x98, 0x52, 0xe2, 0xf1, 0xea, 0x17, + 0x54, 0x23, 0x5c, 0x4a, 0xc9, 0xc6, 0x6d, 0xc9, 0xdc, 0x37, 0x78, 0x34, + 0x9a, 0xaa, 0xf2, 0xed, 0x84, 0x21, 0xe8, 0xfa, 0xa9, 0x37, 0xf6, 0xe4, + 0xba, 0xdc, 0x8c, 0x7a, 0xff, 0x6b, 0xb6, 0xed, 0xda, 0xcf, 0xa0, 0x41, + 0x83, 0x4e, 0x59, 0x83, 0xdd, 0x30, 0x0c, 0x1c, 0x0e, 0x07, 0x6f, 0xbe, + 0xf9, 0x26, 0x83, 0x06, 0x0d, 0x3a, 0xed, 0x7d, 0x5f, 0x7c, 0xf1, 0x45, + 0x46, 0x8e, 0x1c, 0xc9, 0x8c, 0x0f, 0x5f, 0xa2, 0x66, 0xd5, 0xb8, 0x32, + 0x69, 0xd5, 0x01, 0xde, 0x9c, 0xf8, 0x0d, 0x5f, 0xce, 0x5e, 0x98, 0xaf, + 0x93, 0xe1, 0x0f, 0x48, 0x99, 0x39, 0xc5, 0x22, 0x7a, 0xb1, 0x12, 0xbd, + 0xdc, 0xb3, 0x52, 0xa6, 0xbd, 0xae, 0xaa, 0x0a, 0xeb, 0xe7, 0x7f, 0x78, + 0x5a, 0xe5, 0xfe, 0xf7, 0xe0, 0xd3, 0x8e, 0x3d, 0x87, 0xe8, 0xd6, 0x77, + 0x38, 0x36, 0x4d, 0x65, 0x5d, 0xfe, 0x79, 0x97, 0x3b, 0x6c, 0x9a, 0xca, + 0x7f, 0x5f, 0xfd, 0x88, 0xef, 0x17, 0xaf, 0xa6, 0x62, 0x4c, 0x14, 0x13, + 0x5e, 0xe9, 0xe7, 0xeb, 0xc0, 0x84, 0xf0, 0xbf, 0xa0, 0xcc, 0xec, 0x5c, + 0xfa, 0x3c, 0xf3, 0xb6, 0xff, 0x9c, 0x05, 0x0b, 0x16, 0xd0, 0xae, 0x5d, + 0xbb, 0x53, 0xca, 0x6c, 0xdd, 0xba, 0x75, 0x5c, 0x79, 0xe5, 0x95, 0x64, + 0x64, 0x64, 0x10, 0x11, 0x11, 0x71, 0x6a, 0xaf, 0x25, 0x2b, 0x8b, 0xf0, + 0xf0, 0x70, 0xaa, 0x55, 0x8a, 0x65, 0xee, 0xa7, 0x23, 0xcf, 0x18, 0x5e, + 0x04, 0x32, 0xc2, 0x42, 0x82, 0x68, 0x79, 0xfb, 0x00, 0x52, 0xd2, 0x8e, + 0x03, 0x0a, 0x21, 0x21, 0x3d, 0x23, 0xf2, 0xf2, 0x56, 0x67, 0xea, 0xfa, + 0x0e, 0x6b, 0x30, 0xae, 0x58, 0x6e, 0xac, 0x84, 0x7d, 0xe9, 0xb3, 0x40, + 0x26, 0xfd, 0x86, 0xbd, 0x7f, 0x4a, 0xf7, 0xf8, 0xdf, 0x31, 0xb9, 0x69, + 0x4a, 0xaa, 0xe7, 0x17, 0x22, 0x90, 0x94, 0xd2, 0xcc, 0x96, 0xe7, 0x01, + 0x8f, 0x57, 0x67, 0xc8, 0xe3, 0x77, 0x11, 0x12, 0xec, 0xe0, 0x50, 0x4a, + 0x3a, 0xcf, 0xbc, 0x3e, 0x09, 0xaf, 0x6e, 0xe0, 0xf5, 0xea, 0xfc, 0xba, + 0x66, 0x33, 0xdd, 0x1f, 0x1f, 0xe5, 0x27, 0x79, 0x50, 0x90, 0x03, 0x80, + 0xa9, 0x53, 0xa7, 0x9e, 0xb6, 0x63, 0x6c, 0xdc, 0xb8, 0x31, 0x36, 0x9b, + 0x8d, 0xeb, 0xaf, 0xbf, 0xde, 0xdf, 0x49, 0x9e, 0xa4, 0xe0, 0x61, 0x61, + 0xf4, 0xef, 0xdf, 0x9f, 0xbd, 0x07, 0x8f, 0xb2, 0xff, 0x50, 0x0a, 0x65, + 0x15, 0x99, 0xd9, 0xb9, 0x2c, 0x9a, 0x3a, 0x26, 0xff, 0x9b, 0x29, 0x73, + 0x72, 0xa6, 0x2f, 0xd5, 0xf5, 0x1d, 0x44, 0x46, 0xb6, 0xb7, 0x88, 0x5e, + 0x1c, 0x28, 0x57, 0xee, 0xab, 0x83, 0x05, 0x9f, 0x97, 0xfc, 0xbe, 0x01, + 0xb3, 0x88, 0x8b, 0x55, 0xec, 0x36, 0x8d, 0xdb, 0x3a, 0xb4, 0x46, 0x11, + 0x82, 0x40, 0x59, 0x95, 0x2d, 0x84, 0x20, 0x2c, 0x34, 0x98, 0x15, 0xff, + 0x7b, 0x8f, 0xfb, 0xef, 0xe8, 0xc0, 0xdf, 0xc9, 0x07, 0xb9, 0x7b, 0xe0, + 0x68, 0xee, 0x1e, 0x38, 0x9a, 0x71, 0x9f, 0xfa, 0xd2, 0x38, 0xb7, 0x6c, + 0x52, 0x8f, 0x19, 0x1f, 0x0e, 0x63, 0xed, 0x77, 0x13, 0x48, 0xa8, 0x5e, + 0x89, 0x19, 0x33, 0x66, 0x9c, 0xa1, 0x13, 0x55, 0xd8, 0xb8, 0x71, 0x23, + 0x6b, 0xd7, 0xae, 0xe5, 0xa7, 0x9f, 0x7e, 0x3a, 0xad, 0x5c, 0x0b, 0x8a, + 0x3e, 0xbe, 0x38, 0x76, 0x4a, 0x40, 0xaf, 0x71, 0x3f, 0x9b, 0xec, 0x0d, + 0xd3, 0xa4, 0xf7, 0xed, 0xed, 0xf3, 0x3d, 0x5c, 0x6f, 0x63, 0x45, 0x29, + 0xf7, 0x46, 0x46, 0xc6, 0x02, 0xcb, 0x75, 0x2f, 0x3e, 0xab, 0x5e, 0x6e, + 0xb4, 0x69, 0xa6, 0x3d, 0x0d, 0x88, 0x7a, 0x57, 0x54, 0x63, 0xea, 0xbb, + 0x43, 0x8a, 0xd4, 0xe0, 0x5f, 0x96, 0xaf, 0xe3, 0xa9, 0x91, 0x13, 0xd9, + 0xf2, 0xf3, 0x24, 0x3c, 0xde, 0xc0, 0xc9, 0x75, 0x56, 0x30, 0xa8, 0xa6, + 0xeb, 0x06, 0x47, 0x8e, 0x65, 0x30, 0x63, 0xfe, 0x32, 0x2a, 0x57, 0x8a, + 0xe6, 0xb6, 0xf6, 0xad, 0xfc, 0xd3, 0x70, 0x00, 0xa3, 0x3f, 0xfc, 0x9a, + 0xa9, 0x73, 0x16, 0x91, 0x93, 0x93, 0x43, 0x70, 0x70, 0xf0, 0x69, 0xaf, + 0x13, 0x1a, 0x1a, 0x4a, 0x4e, 0x4e, 0xce, 0x19, 0x49, 0x1c, 0x1a, 0x1a, + 0x8a, 0xc3, 0xa6, 0xb0, 0x78, 0xea, 0x18, 0xcc, 0x32, 0x4a, 0x76, 0x29, + 0x25, 0x9a, 0xaa, 0xd2, 0xe8, 0xe6, 0x47, 0xfc, 0xce, 0x62, 0x50, 0xd0, + 0xc3, 0xe1, 0x2e, 0xd7, 0xc7, 0x59, 0x81, 0xf2, 0x8c, 0x97, 0xcc, 0xa2, + 0x6b, 0x5a, 0x75, 0x4c, 0x33, 0xed, 0x59, 0x45, 0x89, 0xf8, 0x0d, 0x90, + 0x5b, 0x77, 0xec, 0x65, 0xe1, 0x8a, 0xf5, 0x27, 0x95, 0x13, 0x3a, 0xe9, + 0xa5, 0x00, 0x57, 0x54, 0xf7, 0xed, 0x2d, 0xbe, 0xd0, 0xbd, 0xd8, 0xa5, + 0xd1, 0xba, 0x80, 0x6f, 0xea, 0xa7, 0x72, 0x85, 0xf2, 0x0c, 0x78, 0xb0, + 0x2b, 0x77, 0xdc, 0x74, 0xcd, 0x09, 0x24, 0x97, 0x52, 0xfa, 0xa7, 0x23, + 0xe7, 0xcc, 0x99, 0x73, 0x4a, 0x59, 0x15, 0x84, 0x41, 0x9b, 0x36, 0x6d, + 0x22, 0x36, 0x36, 0x96, 0xe4, 0xe4, 0xe4, 0xd3, 0xca, 0x74, 0xc0, 0x80, + 0x01, 0x64, 0x1c, 0xcf, 0xc6, 0x66, 0xd3, 0x28, 0xab, 0x10, 0x42, 0x60, + 0x4a, 0x93, 0xc1, 0x0f, 0x75, 0xf7, 0x1b, 0x3f, 0x97, 0x6b, 0xf2, 0x7a, + 0xcb, 0x75, 0x2f, 0x06, 0xe8, 0xfa, 0x9e, 0x7c, 0xa5, 0x0e, 0xef, 0x56, + 0x20, 0xdc, 0xc1, 0xaf, 0x7c, 0xc8, 0xf3, 0x6f, 0x7e, 0x8a, 0xdd, 0x6e, + 0x3b, 0x23, 0xd9, 0xab, 0x57, 0xf6, 0x65, 0x96, 0xd9, 0x77, 0xf8, 0x68, + 0xc0, 0x2a, 0xde, 0x99, 0xfe, 0xd6, 0xb8, 0xae, 0xaf, 0xd6, 0xfb, 0x99, + 0xdc, 0x72, 0x21, 0x04, 0x35, 0x6a, 0xd4, 0xe0, 0xd0, 0xa1, 0x43, 0x3c, + 0xf7, 0xdc, 0x73, 0xa7, 0x3d, 0xee, 0xd6, 0x5b, 0x6f, 0x2d, 0xb3, 0x96, + 0xbc, 0x30, 0x4c, 0x53, 0x72, 0x5f, 0xb7, 0x1b, 0x0a, 0x6b, 0x68, 0xcd, + 0x90, 0x90, 0x06, 0x57, 0x59, 0x44, 0x2f, 0x26, 0x78, 0xbd, 0xfb, 0x52, + 0x84, 0x08, 0x5f, 0x91, 0x6f, 0xac, 0x99, 0xbf, 0x68, 0x15, 0x37, 0xf6, + 0x7e, 0x86, 0xb5, 0x9b, 0xfe, 0x26, 0x24, 0xe8, 0xd4, 0x19, 0x7a, 0x0b, + 0x56, 0xc4, 0x8d, 0xfd, 0xf8, 0x7f, 0x65, 0x32, 0xb6, 0x74, 0xe7, 0x79, + 0xb8, 0xae, 0x65, 0x63, 0xa6, 0x4c, 0x39, 0xfb, 0x4c, 0x50, 0xbb, 0x76, + 0xed, 0xf8, 0xfa, 0xeb, 0xaf, 0xc9, 0xc8, 0xc8, 0x38, 0xa5, 0xac, 0x1e, + 0x7c, 0xf0, 0x41, 0xc2, 0x42, 0x83, 0xf0, 0xea, 0x7a, 0x99, 0x26, 0xba, + 0x10, 0x02, 0x45, 0x08, 0x9e, 0x7a, 0xb8, 0xbb, 0xff, 0xb7, 0x9c, 0x9c, + 0x6d, 0x9f, 0x5a, 0x44, 0x2f, 0xb6, 0x38, 0x3d, 0x16, 0x29, 0x33, 0x5b, + 0x2b, 0x4a, 0x84, 0x3f, 0xcf, 0xf1, 0xc1, 0x23, 0x69, 0xfc, 0xe7, 0xd9, + 0xb1, 0xb4, 0xec, 0x36, 0x80, 0xcf, 0xff, 0xf7, 0x0b, 0x21, 0xc1, 0x4e, + 0xec, 0xbe, 0xbd, 0xc4, 0xbe, 0xcc, 0xa9, 0xf9, 0x4a, 0xb9, 0x62, 0xed, + 0xe6, 0x32, 0x39, 0xff, 0xab, 0x1b, 0x26, 0x0d, 0xeb, 0xf8, 0x76, 0xbf, + 0x4d, 0x9b, 0x36, 0xed, 0xb4, 0x9d, 0x9d, 0x94, 0x92, 0x6a, 0xd5, 0x7c, + 0x4b, 0x88, 0x6b, 0xd5, 0xaa, 0xc5, 0x0b, 0x2f, 0xbc, 0xc0, 0xcc, 0x99, + 0x33, 0x19, 0x35, 0x6a, 0x14, 0x77, 0xde, 0x79, 0x27, 0x57, 0x5c, 0x71, + 0x05, 0x7b, 0xf7, 0xec, 0xe6, 0xcb, 0x71, 0xcf, 0x9f, 0xd3, 0x3a, 0xfb, + 0x40, 0x85, 0x61, 0x9a, 0x74, 0xbf, 0xa5, 0x4d, 0xa1, 0x28, 0x51, 0xaf, + 0x1b, 0x30, 0x1d, 0x59, 0x69, 0x68, 0x84, 0xa6, 0xd5, 0x40, 0xd7, 0x93, + 0xb1, 0xdb, 0x6b, 0x27, 0xe9, 0xfa, 0xa1, 0xde, 0x40, 0x4d, 0xd3, 0x74, + 0x5d, 0x01, 0x7a, 0x23, 0xf0, 0x25, 0x0c, 0xa8, 0x5b, 0xb3, 0x0a, 0x15, + 0x62, 0xca, 0x71, 0x7f, 0xb7, 0x1b, 0xa9, 0x52, 0x31, 0x9a, 0x76, 0x77, + 0x3d, 0x8d, 0x4d, 0xd3, 0x58, 0x3d, 0x67, 0x7c, 0x99, 0x54, 0x4a, 0xbb, + 0x4d, 0x23, 0xa9, 0xe3, 0xa3, 0x34, 0x4a, 0x6c, 0xcc, 0x9a, 0x35, 0x6b, + 0xce, 0x78, 0xac, 0xaa, 0xaa, 0xa7, 0x9c, 0x27, 0xef, 0x73, 0xe7, 0xcd, + 0x3c, 0xd9, 0xa7, 0x1b, 0x46, 0x11, 0x32, 0xef, 0x96, 0x15, 0x68, 0x9a, + 0x4a, 0xff, 0x97, 0xc6, 0xb3, 0x68, 0xc5, 0x86, 0xfc, 0xef, 0xb1, 0xcf, + 0xe9, 0xfa, 0xd1, 0xd1, 0x97, 0xfd, 0x73, 0x95, 0x0a, 0x0b, 0xa5, 0x27, + 0x03, 0xe0, 0xf1, 0x6c, 0x5f, 0x0f, 0xac, 0x2f, 0xd4, 0x01, 0x24, 0x19, + 0xc6, 0x91, 0xfb, 0xbd, 0x5e, 0xbd, 0xcd, 0xc6, 0x6d, 0xc9, 0x4d, 0x37, + 0x6e, 0x4b, 0x66, 0xc5, 0x1f, 0x5b, 0xe4, 0xc3, 0x77, 0x75, 0x14, 0x40, + 0x99, 0x76, 0x37, 0x85, 0x10, 0xd4, 0x4d, 0xa8, 0xc6, 0x1f, 0x7f, 0xfc, + 0xc1, 0xa6, 0x4d, 0x9b, 0x68, 0xd0, 0xa0, 0xc1, 0x29, 0xc9, 0xba, 0x71, + 0xe3, 0x46, 0xbf, 0xc5, 0x77, 0xd8, 0x6d, 0x74, 0xbb, 0xe9, 0x1a, 0xaa, + 0x54, 0x8a, 0xa1, 0xf3, 0xf5, 0x57, 0x51, 0x2e, 0x22, 0xec, 0x84, 0x9d, + 0x74, 0x16, 0x7c, 0xf9, 0x05, 0x1f, 0xeb, 0xdd, 0xc5, 0x4f, 0x74, 0x5d, + 0xcf, 0xba, 0x32, 0x24, 0xe4, 0x6e, 0x72, 0x72, 0xbe, 0xba, 0xbc, 0xf5, + 0xe5, 0x72, 0x69, 0x68, 0xb9, 0x72, 0x3d, 0xc2, 0xd2, 0xd2, 0xbe, 0x1b, + 0x0c, 0xfa, 0xa3, 0xa0, 0xc7, 0xe5, 0xc7, 0xf4, 0x62, 0xc3, 0xf7, 0x13, + 0x2f, 0x68, 0x09, 0xea, 0xe5, 0x8c, 0xdf, 0xd6, 0x6c, 0xe6, 0x89, 0x61, + 0xef, 0xd1, 0xb0, 0x61, 0xc3, 0x53, 0x56, 0x56, 0x5d, 0xb3, 0x66, 0x0d, + 0xcd, 0x9b, 0xfb, 0x32, 0x76, 0x75, 0xbb, 0xf9, 0x1a, 0x5e, 0x7f, 0xf6, + 0x3f, 0xb8, 0xdc, 0x9e, 0x32, 0x2b, 0xaf, 0xa2, 0x77, 0xa2, 0xd0, 0xa6, + 0xc7, 0x60, 0x32, 0xb3, 0x73, 0x25, 0x60, 0x02, 0x9a, 0xd3, 0xd9, 0x06, + 0xb7, 0x7b, 0x99, 0x15, 0xa3, 0x97, 0x34, 0xd2, 0xd2, 0xbe, 0xcd, 0x02, + 0xf7, 0xcb, 0xa0, 0x57, 0xd4, 0xb4, 0x98, 0xfe, 0xa0, 0x6c, 0x07, 0xd8, + 0x73, 0xf0, 0x68, 0x99, 0x55, 0xda, 0xeb, 0x5b, 0x27, 0x71, 0x53, 0xdb, + 0xe6, 0x6c, 0xda, 0xb4, 0x09, 0x21, 0x04, 0x3d, 0x7a, 0xf4, 0x60, 0xee, + 0xdc, 0xb9, 0x8c, 0x1e, 0x3d, 0x9a, 0x98, 0xd8, 0x58, 0xae, 0xbf, 0xee, + 0x1a, 0xee, 0xe9, 0xda, 0x9e, 0x6f, 0xde, 0x1f, 0xca, 0xcb, 0x83, 0xee, + 0x23, 0x27, 0xd7, 0x1d, 0xf0, 0x79, 0xe1, 0x8a, 0x03, 0x52, 0xc2, 0x1b, + 0xbe, 0x52, 0x55, 0x02, 0x50, 0x35, 0x2d, 0xb1, 0xfa, 0x99, 0x48, 0x1e, + 0x1e, 0xde, 0x09, 0x80, 0x90, 0x90, 0x76, 0x96, 0x45, 0x2f, 0x6e, 0xd4, + 0xaf, 0x2f, 0xd9, 0xb2, 0x45, 0xc8, 0x3b, 0x6e, 0x69, 0xc3, 0x8b, 0xfd, + 0xef, 0x29, 0xa3, 0x0a, 0x29, 0xd1, 0x34, 0x95, 0xbb, 0x07, 0xbe, 0xce, + 0xb6, 0x9d, 0x7b, 0xd1, 0x0d, 0x93, 0xf0, 0xd0, 0x20, 0x22, 0xc2, 0x42, + 0x19, 0xfc, 0xd0, 0x1d, 0xdc, 0x74, 0x6d, 0xb3, 0x80, 0x5b, 0x6b, 0x70, + 0xb1, 0x60, 0x9a, 0x26, 0x57, 0x76, 0x7e, 0x3c, 0xff, 0x5b, 0xe8, 0x5c, + 0xc8, 0xee, 0xf2, 0x8f, 0xc5, 0x2f, 0x1f, 0x2b, 0x44, 0x48, 0x8e, 0xef, + 0x1d, 0xa4, 0xc7, 0x4a, 0xe9, 0x89, 0x54, 0x94, 0xa0, 0x18, 0xd3, 0xcc, + 0x39, 0x02, 0x36, 0xb7, 0xa6, 0x95, 0x3f, 0xaa, 0xeb, 0xfb, 0xd2, 0x2d, + 0xa2, 0x17, 0x9b, 0x8b, 0x15, 0xfa, 0xaa, 0xaa, 0xe4, 0x3e, 0xb7, 0x7a, + 0xce, 0xfb, 0xa2, 0xac, 0x5a, 0xa9, 0x82, 0x9d, 0x76, 0x41, 0x0e, 0xbb, + 0x3f, 0x35, 0x95, 0x61, 0x18, 0xe8, 0xd6, 0x28, 0xfa, 0x05, 0x8f, 0x81, + 0x8c, 0x1c, 0xff, 0x15, 0xdf, 0xce, 0x5b, 0x92, 0xbf, 0xad, 0x42, 0xcb, + 0x06, 0x33, 0x34, 0xbf, 0x1b, 0x38, 0xd5, 0xf1, 0x52, 0x4a, 0x29, 0xfe, + 0xf9, 0x1e, 0x39, 0x25, 0x24, 0xa4, 0xd9, 0x23, 0xd9, 0xd9, 0xbf, 0x78, + 0x62, 0x62, 0x5e, 0x21, 0x25, 0xe5, 0x45, 0x8b, 0xe8, 0xe7, 0x03, 0x55, + 0xad, 0x86, 0x69, 0x1e, 0x0e, 0x92, 0xd2, 0x93, 0xbb, 0x6a, 0xf6, 0x78, + 0x6c, 0x9a, 0x6a, 0xb9, 0xa4, 0x16, 0x8a, 0xb5, 0x03, 0x75, 0xe5, 0x79, + 0x68, 0xdd, 0x6d, 0x20, 0x00, 0x41, 0x41, 0x41, 0xd4, 0xa8, 0x51, 0x83, + 0x94, 0x94, 0x14, 0x12, 0x13, 0x13, 0xe9, 0xd8, 0xb1, 0x23, 0x55, 0xaa, + 0x54, 0xe1, 0xf6, 0xdb, 0x6f, 0xc7, 0x6e, 0xff, 0xa7, 0x90, 0x48, 0x4a, + 0x4a, 0x0a, 0x3d, 0x7a, 0xf4, 0x60, 0xd9, 0xb2, 0x65, 0x98, 0xa6, 0xc4, + 0xe9, 0x6c, 0x1c, 0xef, 0x76, 0xaf, 0x4f, 0x0e, 0x09, 0xb9, 0x96, 0x9c, + 0x9c, 0xa5, 0x97, 0xec, 0x79, 0x2e, 0xdb, 0x75, 0x8f, 0x86, 0xb1, 0x17, + 0x55, 0xad, 0xee, 0x31, 0x8c, 0x3d, 0x1c, 0x3e, 0x7a, 0x8c, 0x6a, 0x95, + 0x2b, 0x58, 0xda, 0x69, 0xa1, 0x58, 0x2d, 0x7a, 0xb0, 0xd3, 0x81, 0xcd, + 0xa6, 0xe1, 0xf5, 0xea, 0xa4, 0xa5, 0xa5, 0xe1, 0x74, 0x9e, 0xbd, 0xc4, + 0x7a, 0x4c, 0x4c, 0x0c, 0x8b, 0x17, 0x2f, 0x06, 0xe0, 0xa1, 0x87, 0x1e, + 0x62, 0xd2, 0xa4, 0x49, 0xbb, 0x6d, 0xb6, 0xea, 0x75, 0x73, 0x72, 0x96, + 0x6e, 0xbb, 0x94, 0xcf, 0x73, 0x59, 0x97, 0x90, 0x35, 0x8c, 0x3d, 0x86, + 0x10, 0x21, 0xcb, 0xde, 0x9a, 0x34, 0x03, 0xcb, 0x96, 0x5b, 0x28, 0x09, + 0x0c, 0xed, 0xe7, 0x1b, 0xff, 0xf9, 0xfc, 0xf3, 0xcf, 0xcf, 0xf9, 0xdc, + 0x4f, 0x3e, 0xf9, 0x04, 0xa7, 0xd3, 0x29, 0xbd, 0xde, 0xc3, 0x7f, 0x5d, + 0xea, 0xe7, 0xb8, 0xec, 0x6b, 0x45, 0x0b, 0x11, 0xf9, 0xc2, 0x92, 0x95, + 0x7f, 0xe2, 0xf1, 0xea, 0x96, 0x56, 0x5a, 0x28, 0x76, 0xdc, 0x72, 0x5d, + 0x33, 0x00, 0xfa, 0xf6, 0x2d, 0x5a, 0xbd, 0x87, 0x82, 0xbc, 0x7c, 0xde, + 0xfc, 0x5d, 0x95, 0x77, 0xdc, 0x71, 0x87, 0x80, 0x3c, 0x14, 0xa5, 0x7c, + 0x75, 0x8b, 0xe8, 0x17, 0x80, 0x72, 0xe5, 0x5a, 0xfe, 0x0a, 0x98, 0xae, + 0x73, 0xc8, 0xd3, 0x66, 0xc1, 0x42, 0x51, 0x61, 0xd3, 0x34, 0xc2, 0x43, + 0x7d, 0x5b, 0x81, 0xdd, 0x6e, 0x77, 0x91, 0x5c, 0xfe, 0x8f, 0x3f, 0xfe, + 0x18, 0xbb, 0xdd, 0x4e, 0x52, 0x52, 0x12, 0x3d, 0x7a, 0xf4, 0xc8, 0xef, + 0x00, 0x74, 0x9b, 0x45, 0xf4, 0x0b, 0x40, 0x66, 0xe6, 0x46, 0x09, 0x98, + 0x63, 0x27, 0xcd, 0x90, 0xaa, 0xa2, 0x58, 0x9a, 0x69, 0xa1, 0xd8, 0x63, + 0xf5, 0x61, 0x03, 0x7a, 0x03, 0xbe, 0x01, 0xb9, 0xdc, 0xdc, 0xdc, 0xb3, + 0x5a, 0xf4, 0x47, 0x1f, 0x7d, 0x94, 0x89, 0x13, 0x27, 0xb2, 0x61, 0xc3, + 0x06, 0x7a, 0xf5, 0xea, 0x05, 0x28, 0x48, 0x79, 0x7c, 0x87, 0x45, 0xf4, + 0x0b, 0x80, 0xc7, 0xb3, 0x1d, 0x21, 0x22, 0xc6, 0xcc, 0xfa, 0x69, 0xb9, + 0xc8, 0xca, 0x75, 0x59, 0x56, 0xdd, 0x42, 0xb1, 0xa3, 0x5d, 0xab, 0x24, + 0x7f, 0xb1, 0x90, 0xe3, 0xc7, 0x8f, 0x9f, 0xb5, 0x63, 0x90, 0x52, 0xf2, + 0xf0, 0xc3, 0x0f, 0x17, 0x78, 0x01, 0x52, 0x51, 0x22, 0x46, 0x59, 0x31, + 0x7a, 0x31, 0xa0, 0x6e, 0xdd, 0x8c, 0x21, 0x80, 0x31, 0x6c, 0xec, 0x14, + 0xec, 0x65, 0x38, 0x81, 0x82, 0x85, 0x92, 0x81, 0xaf, 0x7c, 0xb5, 0xe6, + 0x27, 0xfa, 0xd9, 0x8c, 0x49, 0x41, 0x41, 0xcf, 0x3b, 0xef, 0xbc, 0x13, + 0x40, 0x80, 0x18, 0xa9, 0xaa, 0x95, 0x2c, 0xa2, 0x5f, 0x28, 0xb6, 0x6e, + 0x15, 0x12, 0xd4, 0xa3, 0x0b, 0x7e, 0x5b, 0xc7, 0x86, 0xad, 0xbb, 0x2d, + 0xcd, 0xb4, 0x50, 0xac, 0x90, 0x52, 0xd2, 0xa4, 0x61, 0x2d, 0x00, 0x96, + 0x2f, 0x5f, 0x5e, 0xe4, 0xf5, 0x1a, 0xeb, 0xd6, 0xad, 0x03, 0x20, 0x24, + 0xa4, 0xb3, 0xc7, 0x30, 0x0e, 0x5e, 0xda, 0x10, 0x24, 0x10, 0x5e, 0x44, + 0x50, 0x50, 0x53, 0x0c, 0x23, 0xbd, 0xad, 0xc7, 0xb3, 0x6b, 0x71, 0xb9, + 0x88, 0x30, 0x96, 0x7e, 0x33, 0xb6, 0x4c, 0x16, 0x25, 0xb0, 0x50, 0x72, + 0x58, 0xbc, 0x72, 0x03, 0x4f, 0x8e, 0xf8, 0x00, 0x9b, 0xcd, 0x46, 0xf3, + 0xe6, 0xcd, 0x49, 0x48, 0x48, 0xa0, 0x56, 0xad, 0x5a, 0x74, 0xe8, 0xd0, + 0x81, 0xab, 0xae, 0xba, 0xca, 0xbf, 0x87, 0xa0, 0xe0, 0xff, 0x8c, 0x8c, + 0x0c, 0xa2, 0xa2, 0xa2, 0xa4, 0xa6, 0xc5, 0x4c, 0xd4, 0xf5, 0x94, 0xc7, + 0x2e, 0xf9, 0x58, 0x43, 0x60, 0xbd, 0x0e, 0xdb, 0x06, 0xf0, 0x26, 0x4e, + 0x7d, 0x77, 0x08, 0x75, 0x13, 0xaa, 0x5a, 0x2b, 0xe5, 0x2c, 0x14, 0x9f, + 0x66, 0x69, 0x1a, 0x0d, 0x3a, 0x3c, 0x74, 0xca, 0xbf, 0xd5, 0xae, 0x5d, + 0x5b, 0x8e, 0x1c, 0x39, 0x52, 0x38, 0x9d, 0x4e, 0x42, 0x43, 0x43, 0x99, + 0x3e, 0x7d, 0xba, 0x9c, 0x30, 0x61, 0x82, 0x00, 0xdb, 0xb6, 0x90, 0x90, + 0xfa, 0xf5, 0x73, 0x73, 0x0f, 0x99, 0x52, 0x1e, 0xb5, 0x88, 0x5e, 0x5c, + 0xd0, 0xb4, 0xca, 0x6d, 0x75, 0xfd, 0xc0, 0xe2, 0x86, 0x75, 0xe2, 0xf9, + 0xfa, 0xbd, 0x17, 0xca, 0x7c, 0x7a, 0x24, 0x0b, 0xc5, 0x49, 0x74, 0x95, + 0xc6, 0x1d, 0x1f, 0x45, 0xd7, 0x0d, 0x54, 0x35, 0x74, 0x83, 0x61, 0x84, + 0x76, 0x51, 0xd5, 0xbc, 0xc7, 0x0d, 0xc3, 0x9d, 0x0a, 0xde, 0x87, 0x41, + 0xaf, 0x53, 0x28, 0xaa, 0xcf, 0x16, 0xc2, 0xf6, 0x3f, 0x29, 0xdd, 0xf7, + 0x97, 0x96, 0xf6, 0x07, 0xd4, 0x7c, 0x94, 0xae, 0x1f, 0x58, 0x02, 0xb6, + 0xbf, 0x37, 0x6d, 0xdb, 0xcd, 0xda, 0xcd, 0x7f, 0x03, 0xd6, 0x08, 0xbc, + 0x85, 0xe2, 0x81, 0x99, 0x9f, 0x12, 0x1a, 0xc0, 0x30, 0xf2, 0x1a, 0xc3, + 0xe1, 0x7d, 0x86, 0x91, 0xfe, 0xbc, 0xa2, 0x54, 0x7d, 0x0b, 0xf4, 0xba, + 0x09, 0x09, 0xaf, 0x39, 0x20, 0xb2, 0x4b, 0xe5, 0xca, 0x0f, 0xd6, 0x01, + 0x23, 0x4c, 0x4a, 0xf7, 0xfd, 0x0e, 0x47, 0x43, 0x2c, 0xa2, 0x97, 0x08, + 0x6a, 0x20, 0x44, 0xe4, 0x93, 0x00, 0x4b, 0x7e, 0xff, 0x13, 0x45, 0x58, + 0xf3, 0xea, 0x16, 0x8a, 0x07, 0x52, 0xca, 0x42, 0x33, 0x3a, 0x5e, 0x34, + 0xad, 0x5a, 0x0d, 0xa7, 0xb3, 0x19, 0xa6, 0xb9, 0x1d, 0x80, 0x9d, 0x3b, + 0x9f, 0xf7, 0x40, 0xc6, 0xdc, 0x03, 0x07, 0x26, 0x6f, 0x2f, 0x38, 0x27, + 0x2f, 0x6f, 0x93, 0x45, 0xf4, 0x92, 0x41, 0x32, 0x52, 0xa6, 0xcc, 0x07, + 0x2d, 0x79, 0xd9, 0xea, 0x4d, 0xd2, 0x66, 0x4d, 0xb5, 0x59, 0x28, 0x36, + 0xa2, 0xfb, 0xa6, 0xd9, 0x0a, 0xbe, 0x1a, 0x46, 0xea, 0x2d, 0x6e, 0xf7, + 0x9a, 0xcb, 0xa6, 0xfd, 0x01, 0x67, 0xf2, 0x14, 0xa5, 0x0a, 0x8a, 0x12, + 0x3c, 0x75, 0xdb, 0xce, 0x7d, 0xc2, 0x57, 0x38, 0xcf, 0x82, 0x85, 0x62, + 0x70, 0xdd, 0x4d, 0x93, 0x20, 0x87, 0xa3, 0xe0, 0xab, 0x00, 0x79, 0xf3, + 0x65, 0xc5, 0x8b, 0xc0, 0x7b, 0x21, 0xfb, 0x31, 0xcd, 0xcc, 0x21, 0xa0, + 0xf0, 0xcd, 0xdc, 0xc5, 0xd6, 0x4a, 0x39, 0x0b, 0xc5, 0x02, 0x21, 0x04, + 0x3d, 0x3a, 0x5f, 0x5b, 0xf8, 0xa7, 0x5b, 0x2c, 0xa2, 0x97, 0x0a, 0xcb, + 0x1e, 0x32, 0xe4, 0x9b, 0x79, 0x4b, 0x08, 0x0d, 0x09, 0xb2, 0xb4, 0xd4, + 0x42, 0xb1, 0xa0, 0x41, 0xad, 0xea, 0x85, 0x5c, 0x79, 0xb7, 0xcd, 0x22, + 0x7a, 0xa9, 0xb0, 0xec, 0x59, 0xaf, 0x1d, 0x3d, 0x96, 0xc9, 0xa7, 0xdf, + 0xfe, 0x68, 0x59, 0x75, 0x0b, 0xc5, 0x82, 0x56, 0x57, 0xd6, 0x2f, 0x1c, + 0xb5, 0xa3, 0x28, 0x95, 0xab, 0x5b, 0x44, 0x2f, 0x15, 0xee, 0x56, 0xd0, + 0x07, 0x53, 0xa6, 0xff, 0x24, 0x2d, 0xab, 0x6e, 0xa1, 0x38, 0xa0, 0xa9, + 0x2a, 0x21, 0xc1, 0xce, 0x42, 0x56, 0x3d, 0xfd, 0x6a, 0x87, 0xa3, 0xa6, + 0x45, 0xf4, 0x4b, 0x89, 0xd0, 0xd0, 0x4e, 0x28, 0x4a, 0x8d, 0x21, 0x07, + 0x8f, 0x1c, 0x13, 0x53, 0x67, 0x2f, 0xb2, 0xac, 0xba, 0x85, 0x62, 0xf0, + 0x12, 0xa5, 0x7f, 0x73, 0x0b, 0x20, 0xc1, 0xb8, 0x27, 0x2f, 0x6f, 0x97, + 0x45, 0xf4, 0x4b, 0x89, 0xec, 0xec, 0x79, 0x18, 0xc6, 0x96, 0x0c, 0x70, + 0x7c, 0xfb, 0xd1, 0xb4, 0xf9, 0xb2, 0x70, 0x4f, 0x6c, 0xc1, 0xc2, 0xf9, + 0xa0, 0x20, 0xbd, 0x36, 0x80, 0xaa, 0xaa, 0x42, 0x4a, 0xfd, 0x6a, 0xcb, + 0x75, 0x2f, 0x05, 0x50, 0xd5, 0xca, 0x08, 0x61, 0xff, 0x62, 0xff, 0xa1, + 0x14, 0xf1, 0xed, 0xbc, 0xa5, 0x96, 0x55, 0xb7, 0x70, 0x81, 0x16, 0xdd, + 0xc4, 0x69, 0xf7, 0x8d, 0xc1, 0x05, 0x07, 0x07, 0x03, 0x32, 0xc2, 0x22, + 0x7a, 0x29, 0x80, 0x61, 0x1c, 0xc0, 0xe1, 0x78, 0x6a, 0x3e, 0x08, 0xde, + 0xfa, 0x64, 0x3a, 0xc1, 0x41, 0x96, 0x55, 0xb7, 0x70, 0x01, 0xfa, 0x64, + 0x9a, 0x24, 0xd4, 0xa8, 0x0c, 0x40, 0x56, 0x56, 0x16, 0x60, 0xa2, 0xaa, + 0x09, 0x35, 0x2e, 0x0b, 0xa3, 0x17, 0xe8, 0x2f, 0x47, 0xd7, 0x17, 0x4b, + 0x45, 0x09, 0x4d, 0xca, 0x75, 0x65, 0xd7, 0x89, 0x29, 0x17, 0x21, 0xea, + 0xd4, 0xac, 0x62, 0xed, 0x6a, 0xbb, 0x44, 0x6e, 0xaf, 0x6e, 0x18, 0xec, + 0x3d, 0x70, 0x94, 0x43, 0x47, 0x8f, 0x91, 0x76, 0x3c, 0x8b, 0x2d, 0x3b, + 0xf6, 0xb2, 0xff, 0x70, 0x2a, 0x3b, 0xf7, 0x1c, 0x64, 0xef, 0xa1, 0xa3, + 0x78, 0x3d, 0x3a, 0x4e, 0x87, 0x9d, 0x90, 0x20, 0x67, 0xa9, 0xac, 0xf0, + 0x2a, 0x84, 0x60, 0xfb, 0xce, 0xfd, 0xac, 0xdf, 0xb2, 0xb3, 0xd0, 0x73, + 0xe5, 0xcd, 0x04, 0xef, 0x9e, 0xd2, 0x2e, 0xff, 0x80, 0xd7, 0xf8, 0xa0, + 0xa0, 0x46, 0xe4, 0xe5, 0x1d, 0x89, 0x37, 0xcd, 0xa3, 0xbb, 0xec, 0x36, + 0x8d, 0xb5, 0x73, 0x3f, 0xb0, 0xf6, 0xaa, 0x5f, 0x02, 0x68, 0xaa, 0xca, + 0x55, 0xb7, 0x0f, 0x20, 0x27, 0xd7, 0x75, 0xd6, 0x63, 0xaf, 0x6f, 0x9d, + 0x44, 0xef, 0xae, 0xed, 0x69, 0x50, 0xbb, 0x06, 0x4e, 0x87, 0xbd, 0x54, + 0x15, 0x85, 0x4c, 0x49, 0x3b, 0xce, 0x8d, 0xbd, 0x9f, 0x2d, 0x44, 0xfe, + 0xe0, 0x7b, 0xa5, 0xcc, 0xfd, 0xbf, 0x52, 0x2f, 0xff, 0x40, 0x57, 0x30, + 0x97, 0x6b, 0x23, 0xc0, 0x6e, 0x21, 0x42, 0x16, 0x7b, 0xbc, 0x39, 0x6d, + 0x9f, 0x79, 0xfd, 0x63, 0xf1, 0xea, 0xd3, 0x7d, 0x2c, 0xe6, 0x5d, 0x64, + 0xb8, 0xf3, 0x3c, 0xe4, 0xe4, 0xba, 0x88, 0x8a, 0x8a, 0x22, 0x36, 0x36, + 0x96, 0x88, 0x88, 0x08, 0x2a, 0x57, 0xae, 0x4c, 0x70, 0x70, 0x30, 0x79, + 0x79, 0x79, 0x6c, 0xdf, 0xbe, 0x1d, 0x29, 0x25, 0xc7, 0x8e, 0x1d, 0x63, + 0xd1, 0x8a, 0x0d, 0x2c, 0x5c, 0xee, 0xab, 0x9e, 0x3d, 0xa4, 0xdf, 0xdd, + 0xdc, 0x76, 0x43, 0x2b, 0xec, 0x36, 0xad, 0x54, 0x90, 0xbd, 0x6a, 0xc5, + 0x18, 0xca, 0x47, 0x86, 0x73, 0x2c, 0x23, 0xd3, 0x17, 0xfb, 0x2a, 0x76, + 0xa7, 0x61, 0xe4, 0x96, 0x7a, 0xf9, 0x97, 0x09, 0x1f, 0x36, 0x38, 0xb8, + 0x01, 0x2e, 0xd7, 0x81, 0x6a, 0x52, 0x66, 0xec, 0x01, 0x58, 0x31, 0xf3, + 0x5d, 0x82, 0x1c, 0x76, 0x8b, 0x7d, 0x17, 0xd3, 0xa2, 0x68, 0x2a, 0x0d, + 0x3b, 0x3c, 0xcc, 0xd4, 0xa9, 0x53, 0xe9, 0xd5, 0xab, 0xd7, 0x09, 0x55, + 0x5d, 0x4f, 0x55, 0xe1, 0x75, 0xc7, 0x8e, 0x1d, 0x5c, 0x7f, 0xfd, 0xf5, + 0xec, 0xdb, 0xb7, 0x0f, 0x4d, 0x53, 0x59, 0xf2, 0xf5, 0x58, 0x82, 0x9d, + 0xf6, 0x4b, 0x4e, 0x76, 0x21, 0xe0, 0xbe, 0xc1, 0x6f, 0xb0, 0x61, 0x6b, + 0xc1, 0xb4, 0x9a, 0xed, 0x05, 0xf0, 0xbe, 0x5a, 0xda, 0xe5, 0x5f, 0x26, + 0xf6, 0x71, 0xe6, 0xe6, 0x6e, 0x46, 0xca, 0x8c, 0xbd, 0x36, 0x5b, 0xe5, + 0xc7, 0x00, 0x3a, 0x3d, 0xf8, 0x82, 0x5f, 0xc1, 0x2c, 0x5c, 0x1c, 0x6c, + 0xdd, 0xb1, 0x17, 0x20, 0x3f, 0xfd, 0x31, 0x27, 0x10, 0xf6, 0x54, 0xe4, + 0x4d, 0x48, 0x48, 0x60, 0xef, 0xde, 0xbd, 0x1c, 0x3d, 0x7a, 0x14, 0x9b, + 0xcd, 0x4e, 0x97, 0xff, 0x0c, 0xf5, 0x4f, 0x6d, 0x5d, 0x4a, 0x98, 0xa6, + 0xa4, 0x62, 0x6c, 0xb9, 0x42, 0x6d, 0x57, 0x2e, 0x8b, 0x11, 0xde, 0x32, + 0xb5, 0x61, 0xdb, 0xeb, 0x3d, 0xf0, 0x21, 0xd8, 0x48, 0xcb, 0xc8, 0x92, + 0x65, 0xb9, 0xae, 0xfa, 0xc5, 0x86, 0x94, 0x92, 0xe4, 0xfd, 0x87, 0xcf, + 0xd1, 0x72, 0xfa, 0xde, 0x4d, 0x74, 0x74, 0x34, 0xe9, 0xe9, 0xe9, 0xa4, + 0x65, 0x64, 0xf1, 0xea, 0xfb, 0xd3, 0x2e, 0xbd, 0x0b, 0x2c, 0x04, 0x49, + 0xf5, 0xaf, 0x28, 0xf4, 0x6c, 0x7a, 0x79, 0x8b, 0xe8, 0xa5, 0x10, 0x21, + 0x21, 0x4d, 0x63, 0x00, 0x71, 0xdb, 0x43, 0xc3, 0x30, 0xac, 0xd2, 0xc2, + 0x17, 0x8d, 0x1c, 0x39, 0xb9, 0x6e, 0x54, 0x55, 0x3d, 0xaf, 0x73, 0x1d, + 0x0e, 0x07, 0x93, 0x26, 0x4d, 0x62, 0xda, 0x77, 0x8b, 0x4a, 0xc5, 0xf3, + 0xd4, 0xbb, 0xa2, 0x5a, 0xa1, 0xf6, 0x69, 0xd7, 0x58, 0x44, 0x2f, 0x65, + 0x68, 0xd2, 0x64, 0x21, 0x39, 0x39, 0x2b, 0x53, 0x55, 0xb5, 0xee, 0x4d, + 0x00, 0xdf, 0xcc, 0x5b, 0x6a, 0xb1, 0xf0, 0x22, 0xc5, 0xe7, 0xab, 0xff, + 0xdc, 0x8e, 0x61, 0x18, 0xcc, 0x9d, 0x3b, 0xf7, 0xbc, 0x3c, 0x82, 0x9b, + 0x6e, 0xba, 0x09, 0x29, 0x25, 0x03, 0x47, 0x4c, 0xb8, 0xe4, 0x21, 0x57, + 0x83, 0xda, 0x85, 0x77, 0xb1, 0x79, 0x83, 0x2c, 0xa2, 0x97, 0x32, 0xac, + 0x5b, 0x77, 0x3d, 0x36, 0x5b, 0x4b, 0x0c, 0xe3, 0xaf, 0x9f, 0x14, 0x25, + 0x7c, 0xf3, 0xe8, 0x0f, 0xa7, 0x91, 0xe7, 0xf1, 0x5a, 0xb1, 0x7a, 0x09, + 0x43, 0x55, 0x14, 0x92, 0xf7, 0x1f, 0x21, 0x32, 0x32, 0x92, 0xce, 0x9d, + 0x3b, 0x17, 0xf9, 0xbc, 0xac, 0xac, 0x2c, 0x6a, 0xd5, 0xaa, 0x85, 0x10, + 0x82, 0xca, 0x95, 0x7d, 0x0b, 0x55, 0x56, 0xfc, 0xb1, 0xe5, 0x92, 0x87, + 0x5c, 0x51, 0x11, 0x61, 0x85, 0x2d, 0x7a, 0x05, 0x8b, 0xe8, 0xa5, 0x32, + 0x4e, 0x5f, 0x99, 0xef, 0xc2, 0xbf, 0xd5, 0x08, 0xe0, 0xc6, 0xde, 0xcf, + 0x49, 0xd3, 0x34, 0x2d, 0xb2, 0x97, 0x30, 0xf2, 0xf2, 0x3c, 0x74, 0xeb, + 0xd6, 0xed, 0x9c, 0xce, 0x09, 0x0b, 0x0b, 0x63, 0xc4, 0x88, 0x11, 0xc4, + 0xc7, 0xc7, 0x03, 0xf0, 0xfc, 0xf3, 0xcf, 0xe3, 0xd5, 0x0d, 0x84, 0xb8, + 0xb4, 0x03, 0xa9, 0xa6, 0x69, 0xe2, 0xcc, 0x9f, 0xb5, 0x91, 0xd2, 0x13, + 0x0e, 0x09, 0xd1, 0x16, 0xd1, 0x4b, 0x29, 0xb2, 0xb2, 0x1e, 0x96, 0xc1, + 0xc1, 0x8d, 0x6b, 0x65, 0x66, 0xe7, 0x98, 0xfd, 0x5e, 0x7a, 0xdf, 0xca, + 0x2f, 0x57, 0x42, 0x90, 0x52, 0xb2, 0xe7, 0xc0, 0x51, 0x76, 0xec, 0x39, + 0xc8, 0xf8, 0xf1, 0xe3, 0xcf, 0xf9, 0xdc, 0x5b, 0x6f, 0xbd, 0x95, 0xe4, + 0xe4, 0x64, 0x26, 0x4f, 0x9e, 0x4c, 0xdb, 0xb6, 0x6d, 0x91, 0x52, 0x92, + 0x9d, 0xe3, 0xbe, 0xa4, 0x56, 0xdd, 0x30, 0x4c, 0x22, 0xc3, 0x43, 0x0b, + 0xfd, 0x92, 0x52, 0xdb, 0x22, 0x7a, 0x29, 0x46, 0x6e, 0xee, 0x86, 0x1d, + 0x8a, 0x52, 0xee, 0xdd, 0x15, 0x6b, 0xb7, 0x88, 0xdd, 0xfb, 0x0e, 0x5b, + 0x56, 0xbd, 0x04, 0x10, 0xe4, 0x74, 0xf0, 0xd8, 0xd0, 0x77, 0x7c, 0x9f, + 0x83, 0x82, 0xce, 0x48, 0xea, 0x7f, 0x43, 0x08, 0x41, 0x48, 0x48, 0x08, + 0x3d, 0x7a, 0xf4, 0x60, 0xe0, 0xc0, 0x81, 0xbc, 0xf9, 0xe6, 0x9b, 0xa8, + 0x8a, 0x42, 0x68, 0xb0, 0xf3, 0x92, 0xbe, 0x2b, 0xaf, 0x57, 0xa7, 0x6a, + 0xc5, 0xc2, 0x46, 0xdc, 0xdb, 0xce, 0x22, 0x7a, 0x29, 0x87, 0x69, 0xa6, + 0x0d, 0x06, 0xe5, 0x48, 0xf7, 0xc7, 0x5e, 0x91, 0x87, 0x53, 0xd2, 0x2d, + 0x66, 0x16, 0x93, 0x15, 0x17, 0x42, 0xa0, 0x08, 0xc1, 0xec, 0x9f, 0x96, + 0xb3, 0x7b, 0x9f, 0x6f, 0x6a, 0xed, 0x86, 0x1b, 0x6e, 0xa0, 0x47, 0x8f, + 0x1e, 0x34, 0x6b, 0xd6, 0x8c, 0xb8, 0xb8, 0x38, 0x1a, 0x36, 0x6c, 0x48, + 0xbd, 0x7a, 0xf5, 0x78, 0xf0, 0xc1, 0x07, 0x59, 0xb2, 0x64, 0xc9, 0x69, + 0x09, 0xdf, 0xb9, 0x73, 0x67, 0xb2, 0xb2, 0xb2, 0x58, 0xb0, 0x60, 0x01, + 0x12, 0x89, 0x84, 0x4b, 0x6b, 0xd1, 0x4d, 0x93, 0xc4, 0x7a, 0x09, 0x85, + 0x9f, 0xd8, 0x9e, 0x98, 0xf8, 0x99, 0x45, 0xf4, 0xd2, 0x0c, 0x9b, 0xad, + 0x3e, 0x8a, 0x12, 0xf5, 0x41, 0x9e, 0xc7, 0x23, 0x3a, 0x3e, 0x30, 0x84, + 0x23, 0xa9, 0xe9, 0x96, 0x65, 0xbf, 0x00, 0x08, 0x21, 0x48, 0x49, 0x3b, + 0xce, 0xe8, 0x0f, 0xbf, 0x66, 0xcb, 0x8e, 0xbd, 0x3c, 0x3b, 0xfa, 0x13, + 0x84, 0xe2, 0x23, 0xe5, 0x82, 0x05, 0x0b, 0x98, 0x3f, 0xef, 0x3b, 0x4c, + 0xf7, 0x71, 0xea, 0xc7, 0xc7, 0x51, 0xbb, 0x4a, 0x24, 0xc7, 0x8f, 0x1d, + 0xe5, 0xb3, 0xcf, 0x3e, 0xa3, 0x5d, 0xbb, 0x76, 0xdc, 0x7d, 0xf7, 0xdd, + 0xfe, 0xfa, 0x65, 0x85, 0xb1, 0x6c, 0xd9, 0x32, 0xff, 0xe7, 0x07, 0xee, + 0xe8, 0x50, 0x2a, 0xde, 0xcf, 0x1d, 0xb7, 0xf8, 0x67, 0xd5, 0x24, 0x98, + 0x55, 0xfe, 0xfc, 0xf3, 0x81, 0xd2, 0xfd, 0x5e, 0x2c, 0xd5, 0xf4, 0x53, + 0x7e, 0x27, 0x78, 0xe3, 0x6b, 0x56, 0xab, 0x28, 0xbe, 0xff, 0x6c, 0x14, + 0xb9, 0x2e, 0x0f, 0xd6, 0x7a, 0x9a, 0x73, 0xb7, 0xe4, 0xab, 0x36, 0x6c, + 0xa3, 0xef, 0x90, 0x71, 0xdc, 0xdb, 0xed, 0x06, 0xbe, 0xf8, 0xdf, 0x2f, + 0xd4, 0xab, 0x55, 0x9d, 0xf9, 0x9f, 0xbe, 0x42, 0x6a, 0x7a, 0x16, 0x36, + 0x4d, 0x25, 0x34, 0x24, 0x08, 0x8f, 0xc7, 0x8b, 0x69, 0x4a, 0x0c, 0xd3, + 0x24, 0xc8, 0x69, 0x67, 0xe8, 0x98, 0xcf, 0xf8, 0x66, 0x9e, 0xcf, 0xa2, + 0x27, 0x26, 0x26, 0xf2, 0xc5, 0x17, 0x5f, 0x90, 0x98, 0x98, 0x08, 0xc0, + 0xe1, 0xc3, 0x87, 0xa9, 0x58, 0xb1, 0x22, 0x8a, 0x22, 0x18, 0x37, 0xec, + 0x71, 0xda, 0xb5, 0x6a, 0x5c, 0x2a, 0xd6, 0x3f, 0x9c, 0xb8, 0xb9, 0xc5, + 0xb6, 0x0c, 0xbc, 0xd7, 0x5a, 0x44, 0x2f, 0xe5, 0xd0, 0xb4, 0xba, 0xd4, + 0xaa, 0xf5, 0x63, 0xd0, 0xd6, 0xad, 0x35, 0x72, 0x41, 0xca, 0xfe, 0xf7, + 0x77, 0x15, 0x7d, 0xef, 0xe9, 0x84, 0xae, 0x5b, 0xbb, 0xdc, 0xce, 0xd5, + 0x9a, 0xb7, 0xbf, 0xe7, 0x19, 0x52, 0xf3, 0xf3, 0xe9, 0x57, 0xad, 0x18, + 0xcb, 0xf7, 0x9f, 0x8d, 0xcc, 0x1f, 0x29, 0x17, 0x67, 0x38, 0x0f, 0x56, + 0x6d, 0xd8, 0xce, 0xc8, 0xf7, 0xbe, 0x64, 0xcf, 0x81, 0x23, 0x00, 0x38, + 0x1c, 0x0e, 0xec, 0x76, 0x3b, 0x59, 0x59, 0x59, 0x84, 0x85, 0x06, 0xf3, + 0xc5, 0xdb, 0xcf, 0x52, 0xbd, 0x52, 0x6c, 0xa9, 0x59, 0xcd, 0x78, 0x2c, + 0x3d, 0x93, 0xf6, 0xf7, 0x3c, 0x53, 0xe0, 0x18, 0x6f, 0x06, 0xb3, 0xa1, + 0x45, 0xf4, 0xcb, 0x46, 0x51, 0x43, 0x3e, 0x94, 0x32, 0xa7, 0x2f, 0xc0, + 0x8b, 0x03, 0x7a, 0xd3, 0xfd, 0x96, 0x6b, 0xb0, 0xbc, 0xf8, 0xa2, 0x5b, + 0x73, 0x80, 0x26, 0x9d, 0x1e, 0x23, 0xa6, 0x5c, 0x04, 0xa1, 0x21, 0xc1, + 0xcc, 0xfb, 0xf4, 0x15, 0xf2, 0x3c, 0xde, 0x22, 0x91, 0x53, 0x4a, 0x49, + 0x90, 0xd3, 0xc1, 0xb4, 0xb9, 0x8b, 0x99, 0x3e, 0x6f, 0x29, 0xa6, 0x94, + 0x64, 0x1c, 0xcf, 0xa6, 0xfd, 0xd5, 0x4d, 0x18, 0xfe, 0xe4, 0xbd, 0x64, + 0x66, 0xe7, 0x96, 0xaa, 0x25, 0xcb, 0xe1, 0xa1, 0xc1, 0xd4, 0xbc, 0xf6, + 0xbe, 0x02, 0xa2, 0xef, 0x05, 0xb3, 0xba, 0x45, 0xf4, 0xcb, 0xcb, 0x85, + 0xdf, 0x07, 0xde, 0xca, 0x80, 0x98, 0xfb, 0xe9, 0x48, 0x2a, 0xc7, 0x45, + 0x5b, 0x42, 0x2a, 0x02, 0x74, 0xc3, 0xa0, 0xed, 0x9d, 0x83, 0xf9, 0xf0, + 0xd5, 0x41, 0x34, 0x6d, 0x78, 0x05, 0x86, 0x69, 0xa2, 0xeb, 0x17, 0x96, + 0x3c, 0x42, 0x55, 0x95, 0x52, 0xbb, 0x4c, 0x39, 0x34, 0xd8, 0x49, 0x83, + 0x9b, 0x1e, 0x26, 0x2f, 0xcf, 0x0b, 0x68, 0xab, 0x41, 0x6f, 0x51, 0x9a, + 0xdf, 0x8f, 0x55, 0x85, 0xb0, 0x10, 0xec, 0xf6, 0xfa, 0xa8, 0x6a, 0xe4, + 0x90, 0x82, 0x0e, 0xf0, 0xe1, 0xe7, 0xc6, 0xe2, 0xb0, 0x6b, 0xd6, 0xe0, + 0x5c, 0x11, 0xb0, 0x33, 0xf9, 0x10, 0xe3, 0x86, 0x3d, 0x4e, 0xa3, 0x3a, + 0x35, 0xf0, 0x78, 0x75, 0x0c, 0xe3, 0xc2, 0x93, 0x45, 0x94, 0xe6, 0xbd, + 0x08, 0x79, 0x1e, 0x2f, 0x89, 0x75, 0xe2, 0x0b, 0x5a, 0x5a, 0xea, 0x57, + 0xc7, 0x59, 0x44, 0x2f, 0x04, 0x8f, 0x67, 0x0b, 0x86, 0x91, 0xf2, 0x05, + 0xa8, 0x3b, 0x00, 0x0e, 0x1d, 0x4d, 0xe3, 0x91, 0xe7, 0xc6, 0xa1, 0x28, + 0x96, 0x98, 0xce, 0x86, 0x3a, 0x09, 0x55, 0x68, 0xde, 0xb8, 0x4e, 0x19, + 0xf2, 0x60, 0x4c, 0xe2, 0x62, 0xcb, 0xe7, 0x87, 0x7c, 0xf6, 0xe5, 0x16, + 0xd1, 0x2f, 0x43, 0x68, 0x5a, 0xcc, 0x23, 0x05, 0x9f, 0x7f, 0x5d, 0xb3, + 0x89, 0x5d, 0xfb, 0x0e, 0x5b, 0x42, 0xb1, 0x70, 0xaa, 0x91, 0x85, 0x82, + 0xf1, 0x85, 0xe4, 0xf0, 0xf0, 0xab, 0x2d, 0xa2, 0x5f, 0x76, 0xbd, 0xb5, + 0x7e, 0x78, 0x11, 0x84, 0xff, 0x5f, 0xc1, 0x9b, 0x7c, 0xee, 0xf5, 0x4f, + 0x2c, 0xa1, 0x58, 0x38, 0x2d, 0x84, 0x50, 0xc3, 0x32, 0x33, 0x7f, 0xb3, + 0x88, 0x7e, 0xb9, 0x41, 0x55, 0xeb, 0x02, 0x99, 0xf7, 0x0a, 0x61, 0x13, + 0x00, 0xdb, 0x76, 0xed, 0xc3, 0xe5, 0xf6, 0x58, 0x82, 0xb1, 0xf0, 0x8f, + 0x2d, 0xcf, 0x9f, 0x25, 0xf0, 0x7d, 0xf4, 0x6a, 0x36, 0x5b, 0x43, 0x8b, + 0xe8, 0x97, 0x1b, 0x0c, 0xe3, 0xaf, 0xfc, 0x4f, 0x11, 0x37, 0x14, 0xfc, + 0x36, 0x7c, 0xdc, 0xe7, 0xd6, 0xa0, 0x9c, 0x85, 0x13, 0x88, 0x6e, 0xf3, + 0x25, 0xd2, 0x10, 0x40, 0x13, 0xaf, 0x77, 0x93, 0x45, 0xf4, 0xcb, 0xf7, + 0x65, 0xa6, 0x2e, 0x00, 0xc7, 0x34, 0x40, 0xfe, 0xf2, 0xdb, 0x5a, 0x1c, + 0x76, 0x9b, 0x45, 0x76, 0x0b, 0xa7, 0x70, 0xdd, 0x83, 0xbe, 0x2d, 0xed, + 0x6d, 0xb4, 0x88, 0x7e, 0xc6, 0x17, 0x18, 0x8d, 0xcd, 0x16, 0x3b, 0x08, + 0x10, 0x86, 0x61, 0xf2, 0xf3, 0xaf, 0xeb, 0xac, 0x3c, 0x73, 0x16, 0xf2, + 0xc3, 0x3b, 0x85, 0xb4, 0xe3, 0x59, 0xf9, 0x06, 0xc1, 0xdc, 0x62, 0x11, + 0xfd, 0xf2, 0xb6, 0xe8, 0x78, 0xbd, 0xfb, 0x0e, 0x0b, 0x11, 0xf5, 0x05, + 0x20, 0x9f, 0x79, 0xed, 0x23, 0xcb, 0xa2, 0x5b, 0x00, 0x7c, 0x05, 0x29, + 0x0e, 0x1e, 0x4d, 0xf3, 0x91, 0x48, 0x09, 0xca, 0xb0, 0x88, 0x7e, 0xb9, + 0xbf, 0x50, 0xad, 0x0e, 0x52, 0xf2, 0x28, 0x20, 0xf2, 0x3c, 0x5e, 0x3c, + 0x5e, 0xaf, 0x25, 0x14, 0x0b, 0x18, 0x86, 0x49, 0x79, 0x7f, 0x4a, 0xa9, + 0xb0, 0xdd, 0x16, 0xd1, 0x2f, 0x73, 0xe8, 0xfa, 0x36, 0x20, 0x3d, 0x57, + 0xd3, 0xe2, 0x46, 0x03, 0xf2, 0xf5, 0x0f, 0xbe, 0xb6, 0x84, 0x62, 0x01, + 0x21, 0x04, 0xde, 0xfc, 0x4d, 0x4f, 0xa6, 0x99, 0x61, 0xad, 0x8c, 0x0b, + 0x04, 0x44, 0x46, 0xb6, 0x41, 0xd7, 0x53, 0x5e, 0x06, 0x21, 0x66, 0xfe, + 0xf8, 0x9b, 0xb4, 0x6a, 0xb7, 0x59, 0xd0, 0x0d, 0xa3, 0x50, 0x21, 0x87, + 0x90, 0x83, 0x16, 0xd1, 0x03, 0x00, 0x19, 0x19, 0xcb, 0x00, 0xc3, 0x25, + 0x44, 0xf8, 0x1b, 0x80, 0x70, 0xb9, 0xf3, 0x2c, 0xa1, 0x58, 0x16, 0x9d, + 0x7f, 0xf4, 0xc0, 0x65, 0x59, 0xf4, 0x40, 0x81, 0xaa, 0x56, 0x05, 0x8e, + 0xbf, 0x00, 0xc8, 0x29, 0x33, 0x7e, 0xb1, 0x76, 0xb4, 0x95, 0x75, 0x7d, + 0x50, 0x04, 0x19, 0x59, 0x39, 0x00, 0x68, 0x9a, 0xd2, 0xc9, 0x22, 0x7a, + 0x80, 0xc0, 0x30, 0xf6, 0xa1, 0xaa, 0x09, 0x06, 0x08, 0xf9, 0xc9, 0xb4, + 0xf9, 0x64, 0xe5, 0xb8, 0x2c, 0xa1, 0x94, 0xe9, 0x8e, 0x5f, 0xe5, 0x58, + 0x7a, 0x26, 0x80, 0xd4, 0xf5, 0xf4, 0x04, 0x8b, 0xe8, 0x81, 0x14, 0x97, + 0xe9, 0x3b, 0x25, 0x04, 0x2d, 0x2e, 0x70, 0xdd, 0x2c, 0x94, 0x5d, 0x98, + 0xa6, 0x49, 0x48, 0x90, 0x03, 0x40, 0x80, 0x50, 0x2d, 0xa2, 0x07, 0x18, + 0x14, 0x25, 0xe4, 0x13, 0x00, 0xb7, 0xc7, 0x63, 0xcd, 0xa9, 0x97, 0x61, + 0x78, 0x75, 0x83, 0x6a, 0x95, 0x63, 0x0b, 0xfc, 0xbd, 0x3a, 0x16, 0xd1, + 0x03, 0x0c, 0x76, 0x7b, 0xad, 0x9f, 0x01, 0x16, 0xaf, 0xd8, 0x80, 0xa2, + 0x58, 0x56, 0xbd, 0xac, 0x42, 0x4a, 0x89, 0xd3, 0x6e, 0xcf, 0xff, 0xa6, + 0x25, 0x5b, 0x44, 0x0f, 0x20, 0x04, 0x05, 0xb5, 0xc6, 0xed, 0x5e, 0x9e, + 0x0a, 0x98, 0x6f, 0x7d, 0x32, 0xdd, 0x4a, 0x48, 0x51, 0xa6, 0x3d, 0x3b, + 0x85, 0xcc, 0xec, 0xdc, 0x02, 0xfb, 0x1e, 0x65, 0x11, 0x3d, 0x80, 0xe0, + 0x72, 0xf9, 0x13, 0x89, 0x18, 0xb9, 0xae, 0x3c, 0x0e, 0x1e, 0x39, 0x66, + 0xb9, 0xef, 0x65, 0xd8, 0xa2, 0x87, 0x85, 0x06, 0x03, 0x20, 0x84, 0x23, + 0xd3, 0x22, 0x7a, 0x60, 0x42, 0x07, 0xe4, 0xec, 0x9f, 0x57, 0xa0, 0x5a, + 0x56, 0xbd, 0xcc, 0xc2, 0x34, 0xcd, 0x7c, 0xd2, 0x7b, 0x6b, 0x5a, 0x44, + 0x0f, 0x30, 0xd8, 0x6c, 0xf5, 0x01, 0xc7, 0x6e, 0x40, 0x4c, 0xf8, 0x62, + 0x0e, 0x96, 0x3d, 0x2f, 0x9b, 0x10, 0x80, 0xc7, 0xab, 0x17, 0x7c, 0xb5, + 0x36, 0xb5, 0x04, 0x1a, 0xbc, 0xde, 0x2d, 0x80, 0x3a, 0xaf, 0xe0, 0xfb, + 0x5f, 0x3b, 0xf7, 0x59, 0xee, 0x7b, 0x59, 0x24, 0xba, 0x22, 0xc8, 0x75, + 0xb9, 0x01, 0x24, 0x08, 0xdd, 0x22, 0x7a, 0x00, 0x42, 0x51, 0x42, 0x56, + 0x17, 0x7c, 0x9e, 0xf5, 0xe3, 0x6f, 0x68, 0x9a, 0x6a, 0x09, 0xa5, 0x2c, + 0xc6, 0xe9, 0x7e, 0xe3, 0xae, 0xae, 0xb4, 0xd9, 0xea, 0x58, 0x44, 0x0f, + 0xbc, 0xd8, 0x2c, 0xc5, 0x9f, 0x51, 0x64, 0xea, 0x77, 0x8b, 0x30, 0x4d, + 0xcb, 0xa2, 0x97, 0xb9, 0x10, 0x4e, 0xd3, 0x48, 0xde, 0xe7, 0x2b, 0x1f, + 0x25, 0x84, 0xb6, 0xdf, 0xeb, 0xdd, 0x66, 0x11, 0x3d, 0x40, 0x9d, 0x37, + 0xff, 0xce, 0x96, 0x1d, 0x7b, 0x0e, 0x5a, 0xe2, 0x28, 0x6b, 0xd6, 0x5c, + 0x4a, 0xbf, 0x27, 0x27, 0xa5, 0x5e, 0xbd, 0xb4, 0xb7, 0xd7, 0x22, 0xfa, + 0xf9, 0x23, 0xa7, 0xe0, 0xc3, 0xae, 0xbd, 0x87, 0x2c, 0x69, 0x94, 0x31, + 0xa8, 0xaa, 0x42, 0x46, 0x66, 0x76, 0x81, 0x45, 0xaf, 0x66, 0x11, 0x3d, + 0x00, 0xe1, 0x74, 0x5e, 0x89, 0xa2, 0x84, 0x7d, 0x56, 0xf0, 0xfd, 0xcf, + 0xbf, 0x76, 0x59, 0xd3, 0x6c, 0x65, 0xd0, 0xa2, 0x3b, 0x1d, 0xb6, 0x7c, + 0xa2, 0xab, 0x5b, 0x2d, 0xa2, 0x07, 0x20, 0xdc, 0xee, 0xb5, 0x98, 0x66, + 0xe6, 0x53, 0x05, 0xdf, 0x97, 0xfe, 0xbe, 0xd1, 0x1a, 0x90, 0x2b, 0x83, + 0x31, 0x7a, 0x41, 0x05, 0x1f, 0x29, 0x75, 0x6b, 0x7a, 0x2d, 0xc0, 0xe3, + 0xf4, 0x14, 0x80, 0x3d, 0x07, 0x8e, 0x94, 0xea, 0x82, 0x80, 0x16, 0x8a, + 0x1f, 0x3b, 0xf6, 0x1c, 0x44, 0x9a, 0x12, 0x10, 0x48, 0x99, 0xfb, 0x91, + 0x45, 0xf4, 0x80, 0x86, 0x7d, 0x6a, 0xc1, 0xa7, 0xd5, 0x1b, 0xb7, 0x5b, + 0xf3, 0xe9, 0x65, 0x08, 0xbf, 0xaf, 0x2f, 0x28, 0xf2, 0xa1, 0xae, 0xbd, + 0x1c, 0xda, 0x6b, 0x11, 0xfd, 0x02, 0xa0, 0x69, 0x15, 0x37, 0x14, 0x7c, + 0xfe, 0xee, 0x97, 0x15, 0x96, 0xfb, 0x5e, 0x86, 0x90, 0x96, 0xe1, 0xcb, + 0xe9, 0xae, 0x28, 0x36, 0xc5, 0x22, 0x7a, 0x80, 0x43, 0xd7, 0x93, 0x3f, + 0x2d, 0xf8, 0xbc, 0x69, 0x7b, 0x32, 0x36, 0x8b, 0xe8, 0x65, 0x02, 0x52, + 0x4a, 0x92, 0xf7, 0x1f, 0x2e, 0xf8, 0xb8, 0xdd, 0x22, 0x7a, 0xd9, 0x88, + 0xd3, 0x0d, 0x80, 0xdd, 0x7b, 0x0f, 0xa3, 0x69, 0x9a, 0x25, 0x8e, 0xb2, + 0xe0, 0xc9, 0xa9, 0x2a, 0xfb, 0x0f, 0xa7, 0x02, 0x08, 0x90, 0x3b, 0x2c, + 0xa2, 0x97, 0x09, 0x28, 0x7b, 0x00, 0xbc, 0xba, 0x5e, 0x28, 0x6e, 0xb3, + 0x10, 0xd0, 0x5d, 0xbb, 0x10, 0xe8, 0xf9, 0x39, 0xdd, 0xa5, 0x94, 0x1e, + 0x8b, 0xe8, 0x65, 0x03, 0xfb, 0x0a, 0x3e, 0x04, 0x39, 0xec, 0x96, 0x34, + 0xca, 0x42, 0xd7, 0xae, 0x08, 0x72, 0x7c, 0x1b, 0x5a, 0x00, 0x8e, 0x58, + 0x44, 0x0f, 0x70, 0x44, 0x46, 0xde, 0x03, 0xd8, 0xfd, 0xd9, 0x28, 0x72, + 0x5d, 0x56, 0xbe, 0xf7, 0x32, 0xe1, 0xba, 0x6b, 0x2a, 0xa9, 0x69, 0xc7, + 0xf3, 0x49, 0x1f, 0xbe, 0xd3, 0x22, 0x7a, 0x80, 0x23, 0x23, 0xe3, 0x4b, + 0x84, 0x70, 0xfc, 0x56, 0xf0, 0x3d, 0x38, 0xd8, 0x69, 0x09, 0xa5, 0x2c, + 0xbc, 0xf7, 0xcc, 0x1c, 0xd2, 0x8f, 0xfb, 0x96, 0xbf, 0x9a, 0xa6, 0xf8, + 0xcb, 0x22, 0x7a, 0xd9, 0x40, 0x70, 0xc1, 0x87, 0xb4, 0xf4, 0x4c, 0x4b, + 0x1a, 0x65, 0x00, 0xfb, 0x0f, 0xa5, 0x14, 0x72, 0xe3, 0x8d, 0x04, 0x8b, + 0xe8, 0x65, 0x22, 0x5e, 0x8b, 0x5f, 0x5a, 0xf0, 0xf9, 0x70, 0x6a, 0xba, + 0x25, 0x90, 0x00, 0x87, 0x94, 0x92, 0x95, 0xeb, 0xfe, 0x59, 0xda, 0xae, + 0xaa, 0xc1, 0xe9, 0x16, 0xd1, 0xcb, 0x00, 0x0c, 0x63, 0x9d, 0x7f, 0x30, + 0x26, 0x24, 0xc8, 0x72, 0xdd, 0x03, 0x1d, 0xaa, 0xaa, 0x30, 0x67, 0xc1, + 0x4a, 0x3f, 0x7d, 0xbc, 0xde, 0x7d, 0x1b, 0x2c, 0xa2, 0x97, 0x19, 0x88, + 0xe3, 0x00, 0xbb, 0xf6, 0x59, 0xdb, 0x55, 0x03, 0x1d, 0x76, 0x4d, 0xe3, + 0x68, 0xaa, 0x6f, 0x0f, 0x8b, 0x10, 0xda, 0xee, 0xcb, 0xa5, 0xdd, 0x16, + 0xd1, 0x8b, 0x07, 0x39, 0x00, 0x91, 0xe1, 0xa1, 0x96, 0x24, 0x02, 0xbd, + 0x4b, 0x57, 0x04, 0x1e, 0xaf, 0xb7, 0x80, 0x3e, 0x0b, 0x2d, 0xa2, 0x97, + 0x2d, 0x84, 0x02, 0x24, 0xd5, 0xab, 0x69, 0x49, 0x22, 0xd0, 0x5d, 0x77, + 0x45, 0x25, 0xdb, 0x5f, 0x60, 0xd3, 0x31, 0x23, 0x32, 0xb2, 0xbd, 0x45, + 0xf4, 0xb2, 0x03, 0x25, 0x03, 0xa0, 0x7c, 0x54, 0xb8, 0x25, 0x8a, 0x00, + 0xc7, 0xcc, 0x9f, 0xfc, 0xb3, 0xa9, 0xd8, 0x6c, 0x6d, 0x96, 0x65, 0x64, + 0x2c, 0xb0, 0x88, 0x5e, 0x86, 0xb0, 0xc7, 0x12, 0x41, 0xe0, 0x43, 0x4a, + 0xc9, 0x8c, 0xef, 0x97, 0xe5, 0x7f, 0xd3, 0xf0, 0x78, 0xe6, 0x66, 0x5b, + 0xae, 0x7b, 0x19, 0xd3, 0x01, 0x80, 0x4a, 0x15, 0xca, 0x5b, 0x92, 0x08, + 0x64, 0xb7, 0x5d, 0x55, 0x0a, 0xcd, 0xa1, 0xcb, 0x23, 0x97, 0x53, 0xdb, + 0x2d, 0xa2, 0x17, 0x0b, 0x7c, 0x09, 0xfc, 0xad, 0x2c, 0x33, 0x65, 0x20, + 0x3e, 0xcf, 0x2d, 0x58, 0xe3, 0xae, 0x6e, 0xb7, 0x88, 0x5e, 0xd6, 0x68, + 0x2e, 0x94, 0x63, 0x60, 0xad, 0x75, 0x2f, 0x0b, 0x8e, 0x9b, 0xf0, 0x55, + 0xca, 0x96, 0xaa, 0x1a, 0xfe, 0x8d, 0x45, 0xf4, 0xb2, 0x17, 0xbb, 0xe5, + 0x01, 0xa4, 0x67, 0x66, 0x5b, 0xc2, 0x08, 0x60, 0x78, 0xbc, 0x7a, 0xc1, + 0x1a, 0x77, 0x61, 0x9a, 0xae, 0xed, 0x35, 0x6a, 0xbc, 0x69, 0x11, 0xbd, + 0x8c, 0x39, 0x75, 0x87, 0x7c, 0xae, 0xbb, 0x61, 0x89, 0x22, 0x70, 0x3b, + 0x73, 0x3e, 0x9a, 0x3a, 0xdf, 0xff, 0x5d, 0xd3, 0xe2, 0xff, 0x48, 0x4e, + 0x7e, 0xda, 0x22, 0x7a, 0x19, 0x53, 0x83, 0x4a, 0x00, 0x79, 0x1e, 0xaf, + 0x25, 0x8a, 0x00, 0x85, 0xa2, 0x08, 0xa6, 0x7d, 0xb7, 0x28, 0xff, 0x73, + 0xe8, 0xef, 0x5e, 0xef, 0xa6, 0x63, 0x96, 0xeb, 0x5e, 0xe6, 0x60, 0xda, + 0x7c, 0xbd, 0xbe, 0x25, 0x89, 0x80, 0xf5, 0xd9, 0x14, 0x15, 0x97, 0xdb, + 0x97, 0x4c, 0xc6, 0x34, 0x8d, 0xff, 0xbb, 0xec, 0x3a, 0x2a, 0xeb, 0x15, + 0x16, 0x1f, 0xc2, 0x42, 0x82, 0x2c, 0x21, 0x04, 0x28, 0x32, 0xb3, 0x73, + 0x70, 0xe7, 0x79, 0x00, 0xa4, 0xa2, 0x38, 0x0d, 0x8b, 0xe8, 0x65, 0x10, + 0x42, 0xd8, 0x36, 0x01, 0x6c, 0xdb, 0xb5, 0xcf, 0x12, 0x46, 0x80, 0x62, + 0xef, 0x41, 0xff, 0x1e, 0x74, 0x21, 0xa5, 0xfc, 0x31, 0x28, 0xa8, 0xbd, + 0x45, 0xf4, 0x32, 0x17, 0xa1, 0x4b, 0xaf, 0x0e, 0xe0, 0x74, 0x38, 0x2c, + 0x61, 0x04, 0xe4, 0xfb, 0x95, 0xfc, 0xb8, 0x74, 0x0d, 0x80, 0x14, 0xc2, + 0x81, 0x94, 0x19, 0xbb, 0x5c, 0xae, 0x05, 0x16, 0xd1, 0xcb, 0x12, 0x9c, + 0xce, 0x26, 0x08, 0xa1, 0xd4, 0x05, 0xc8, 0xc9, 0x75, 0x59, 0x02, 0x09, + 0x48, 0x8f, 0x4d, 0xf0, 0xd5, 0xec, 0x05, 0x00, 0x42, 0x55, 0xa3, 0xae, + 0xbc, 0x1c, 0x9f, 0xc1, 0x22, 0xfa, 0x05, 0xc2, 0xed, 0x5e, 0x87, 0x94, + 0x66, 0x1c, 0xc0, 0xf1, 0xac, 0x1c, 0x4b, 0x20, 0x01, 0x0a, 0xaf, 0xee, + 0x5b, 0xf5, 0x68, 0x9a, 0xca, 0x2e, 0x8b, 0xe8, 0x65, 0x16, 0x46, 0x18, + 0x80, 0x69, 0x0d, 0xbb, 0x07, 0xe6, 0xdb, 0x35, 0x0d, 0x4c, 0xd3, 0x47, + 0xf4, 0xb0, 0xb0, 0x17, 0xb3, 0x2d, 0xa2, 0x97, 0x5d, 0x54, 0x01, 0xc8, + 0xca, 0xce, 0xb5, 0x24, 0x11, 0x80, 0x18, 0x37, 0x69, 0xa6, 0x3f, 0x5c, + 0x3f, 0x7e, 0xfc, 0x31, 0x1d, 0x5f, 0xa2, 0x91, 0x74, 0xb0, 0xfd, 0xa5, + 0xaa, 0x71, 0x8f, 0x17, 0xfc, 0x31, 0x2c, 0xac, 0x93, 0x45, 0xf4, 0x00, + 0x8f, 0xe2, 0x2a, 0xfa, 0xdc, 0x3b, 0xc3, 0xaa, 0xa8, 0x1a, 0x40, 0x90, + 0x52, 0x62, 0x9a, 0x92, 0x2f, 0x66, 0xfe, 0x02, 0xc0, 0xb1, 0x63, 0xc7, + 0x84, 0xcb, 0xe5, 0xe2, 0xd8, 0xb1, 0x63, 0xc1, 0x9b, 0x36, 0x6d, 0x8a, + 0x7c, 0xfe, 0xf9, 0xa7, 0xea, 0x54, 0xae, 0x6c, 0x7b, 0x1f, 0x90, 0xa0, + 0x1e, 0xc8, 0xc9, 0xd9, 0xdc, 0xcc, 0x22, 0x7a, 0x60, 0x3b, 0x77, 0x1a, + 0xc0, 0xd1, 0xd4, 0x74, 0x14, 0x45, 0x58, 0xe2, 0x08, 0x10, 0x28, 0x42, + 0x30, 0xe6, 0xe3, 0x6f, 0x01, 0xa8, 0x53, 0xa7, 0x0e, 0xe5, 0xca, 0x95, + 0xc3, 0xe9, 0x74, 0x52, 0xae, 0x5c, 0x39, 0x1a, 0x34, 0x68, 0xc0, 0xab, + 0xaf, 0xbe, 0xc6, 0x9e, 0x3d, 0x7b, 0x59, 0xb8, 0x70, 0x21, 0x51, 0x51, + 0xe1, 0x15, 0x4d, 0x73, 0xcf, 0x6a, 0x21, 0x22, 0x3e, 0xb2, 0x88, 0x1e, + 0x80, 0x08, 0x09, 0x79, 0xc1, 0xbf, 0x4a, 0x26, 0x33, 0x3b, 0x17, 0x45, + 0x58, 0x22, 0x0d, 0x14, 0x6b, 0xee, 0xd1, 0x0d, 0xbe, 0x9a, 0xed, 0x4b, + 0x0b, 0xb7, 0x68, 0xd1, 0xa2, 0xd3, 0x1e, 0xdb, 0xae, 0x5d, 0x3b, 0xd2, + 0xd2, 0xd2, 0x44, 0xd7, 0xae, 0xb7, 0x49, 0x29, 0x8f, 0x3f, 0x2c, 0x44, + 0xd4, 0xbd, 0x10, 0x61, 0x11, 0x3d, 0x90, 0x90, 0x93, 0x33, 0xca, 0x05, + 0x3e, 0x2b, 0xde, 0xb4, 0x51, 0x2d, 0xff, 0xa0, 0x8d, 0x85, 0xcb, 0x3c, + 0x18, 0x13, 0x82, 0x8d, 0xdb, 0xfe, 0x49, 0xf2, 0x1a, 0x1d, 0x1d, 0x7d, + 0xd6, 0x8e, 0x61, 0xe6, 0xcc, 0x99, 0xa2, 0x53, 0xa7, 0x4e, 0x52, 0xca, + 0xf4, 0xcf, 0x21, 0xf4, 0x0a, 0x80, 0x7a, 0xf5, 0xf6, 0x5b, 0x44, 0x0f, + 0x34, 0xdc, 0xd9, 0xa9, 0x2d, 0x56, 0x84, 0x1e, 0x18, 0x30, 0x4d, 0x49, + 0xdf, 0xe7, 0xdf, 0x06, 0x60, 0xf0, 0xe0, 0xc1, 0xd8, 0x6c, 0xb6, 0xb3, + 0x76, 0x0c, 0x52, 0x4a, 0xe6, 0xce, 0x9d, 0x2b, 0x7a, 0xf7, 0xee, 0x2d, + 0xe1, 0xc0, 0xdf, 0x76, 0x7b, 0xdd, 0x26, 0x5b, 0xb7, 0x56, 0x29, 0x1d, + 0x1d, 0x97, 0xf5, 0x4a, 0x2f, 0x0c, 0xb1, 0xb1, 0x52, 0x39, 0x7a, 0x54, + 0x18, 0x03, 0x1e, 0xe8, 0xca, 0x43, 0x3d, 0x6f, 0xb1, 0xa6, 0xd8, 0x02, + 0x04, 0x1b, 0xff, 0xda, 0xcd, 0xbd, 0x83, 0x47, 0xfb, 0xad, 0xb5, 0x94, + 0x12, 0x21, 0xc4, 0x59, 0xad, 0xba, 0x10, 0x02, 0xb7, 0xdb, 0x4d, 0x50, + 0x50, 0x10, 0xe0, 0xdc, 0x07, 0xee, 0x6a, 0x96, 0x45, 0x0f, 0x00, 0x1c, + 0x3d, 0x2a, 0x4c, 0x10, 0x39, 0x6e, 0x8f, 0xd7, 0xb2, 0xe6, 0x01, 0x12, + 0x9b, 0x0b, 0x21, 0x18, 0xf4, 0xca, 0x87, 0x00, 0xf4, 0xe9, 0xd3, 0xc7, + 0x6f, 0xb1, 0x8b, 0xe2, 0xee, 0x3f, 0xf5, 0xd4, 0x53, 0x48, 0x29, 0xb9, + 0xe7, 0x9e, 0x7b, 0x24, 0xb8, 0xab, 0xda, 0x6c, 0x8d, 0x15, 0x8b, 0xe8, + 0x81, 0x83, 0x90, 0x3f, 0xb7, 0xee, 0xc2, 0xa6, 0xa9, 0x96, 0x24, 0x2e, + 0x73, 0x92, 0xab, 0xaa, 0xca, 0x07, 0xff, 0xf7, 0x1d, 0xa9, 0xe9, 0xbe, + 0xb2, 0xc8, 0x93, 0x26, 0x4d, 0x2a, 0xf2, 0x94, 0x69, 0x66, 0x66, 0x26, + 0x63, 0xc7, 0x8e, 0xe5, 0xa6, 0x9b, 0x6e, 0xa2, 0x79, 0xf3, 0xe6, 0x02, + 0x40, 0xd7, 0x77, 0x97, 0x8a, 0x3a, 0x5d, 0x9a, 0xf5, 0x7a, 0x8b, 0x27, + 0x02, 0xda, 0xb6, 0x6b, 0x3f, 0x9a, 0xaa, 0xe2, 0xf1, 0xed, 0x6f, 0xb1, + 0x70, 0x19, 0x42, 0x51, 0x04, 0x3f, 0xff, 0xba, 0x96, 0x0f, 0xbf, 0x9c, + 0xeb, 0xff, 0x6d, 0xf5, 0xea, 0xd5, 0x34, 0x6f, 0xde, 0xbc, 0x88, 0xe7, + 0xfb, 0xec, 0xe6, 0xb2, 0x65, 0xcb, 0x58, 0xb1, 0x62, 0x45, 0x7e, 0xe7, + 0x51, 0x3a, 0x92, 0x91, 0x58, 0x16, 0xbd, 0x78, 0x88, 0xbe, 0x49, 0xd3, + 0x54, 0x2b, 0x3a, 0xbf, 0xbc, 0xed, 0x39, 0x19, 0x99, 0x39, 0x3c, 0x35, + 0xf2, 0x43, 0xdf, 0x97, 0x7c, 0xb4, 0x68, 0xd1, 0x82, 0x15, 0x2b, 0x96, + 0xf3, 0xfb, 0xef, 0xbf, 0xcb, 0x23, 0x47, 0x8e, 0xe0, 0x76, 0xbb, 0xfd, + 0xd6, 0xff, 0xdf, 0x08, 0x0d, 0x0d, 0x65, 0xce, 0x9c, 0x39, 0xf8, 0x2c, + 0xb9, 0x9e, 0x4f, 0xfe, 0x60, 0xaf, 0x45, 0xf4, 0x80, 0xb1, 0x04, 0xce, + 0x55, 0x29, 0xc7, 0x32, 0x84, 0xaa, 0x58, 0xe2, 0xbc, 0x5c, 0x5d, 0x76, + 0xbb, 0xcd, 0xc6, 0x7f, 0x9e, 0x19, 0x5b, 0x40, 0x72, 0x21, 0x44, 0xe4, + 0x04, 0x40, 0xa8, 0x6a, 0x8b, 0x2b, 0xae, 0xb9, 0xe6, 0xe6, 0xd5, 0x2d, + 0x5b, 0xb6, 0x14, 0x71, 0x71, 0x71, 0x04, 0x05, 0x05, 0xb1, 0x61, 0xc3, + 0x86, 0xd3, 0xc6, 0xec, 0x2d, 0x5a, 0xb4, 0x38, 0xe1, 0xbb, 0xc3, 0x11, + 0x67, 0x58, 0x44, 0x0f, 0x00, 0xf4, 0xe9, 0xe3, 0x01, 0xec, 0xc9, 0x00, + 0x07, 0x8f, 0x1e, 0xb3, 0x04, 0x72, 0x19, 0x42, 0xd3, 0x54, 0x9e, 0x7b, + 0xe3, 0x53, 0x76, 0xec, 0x39, 0x00, 0x20, 0xec, 0xf6, 0xaa, 0xe3, 0xa5, + 0xcc, 0x78, 0x02, 0xc0, 0x30, 0x56, 0xed, 0x34, 0xcd, 0xac, 0x16, 0x36, + 0x5b, 0xfd, 0x56, 0xf9, 0x9d, 0x80, 0x4c, 0x4a, 0x4a, 0x22, 0x39, 0x39, + 0xf9, 0x24, 0xab, 0x2e, 0xa5, 0xc4, 0xe5, 0x72, 0x15, 0x74, 0x16, 0x7a, + 0xb9, 0x72, 0x37, 0x56, 0x70, 0xb9, 0x36, 0x9b, 0x16, 0xd1, 0x03, 0x00, + 0x9f, 0x7e, 0x6a, 0xc7, 0x34, 0xd3, 0x5f, 0x01, 0x58, 0xb5, 0xfe, 0x2f, + 0x6b, 0xad, 0xfb, 0x65, 0x18, 0x97, 0xbf, 0xfe, 0xc1, 0xd7, 0xcc, 0xf9, + 0x79, 0x39, 0x00, 0x36, 0x5b, 0xe5, 0x17, 0x3c, 0x9e, 0x7d, 0xfd, 0x63, + 0x62, 0x5e, 0x3e, 0xe1, 0x38, 0xaf, 0x77, 0xcb, 0x4a, 0x45, 0xa9, 0x68, + 0x53, 0xd5, 0xc8, 0xdf, 0x01, 0x5a, 0xb6, 0x6c, 0x79, 0x92, 0x55, 0x17, + 0x42, 0xd0, 0xb6, 0x6d, 0x5b, 0x00, 0x61, 0xb3, 0x55, 0x79, 0x35, 0x2d, + 0xed, 0xe7, 0xa3, 0xe5, 0xcb, 0x97, 0x8e, 0x8d, 0x2e, 0x16, 0xd1, 0x8b, + 0xcf, 0x2e, 0xe4, 0x2e, 0x5f, 0xbb, 0x05, 0x55, 0xb5, 0x44, 0x7a, 0xb9, + 0x40, 0x55, 0x15, 0xc6, 0x4e, 0xfa, 0x1f, 0x5f, 0xce, 0x5a, 0x90, 0x4f, + 0xf2, 0x2a, 0xaf, 0x7a, 0xbd, 0x07, 0x5e, 0x85, 0x1a, 0xa4, 0xa4, 0xbc, + 0x74, 0x0a, 0xcb, 0x5f, 0xdd, 0x30, 0x8c, 0x8c, 0x56, 0x42, 0x38, 0x57, + 0x1c, 0x39, 0x72, 0x84, 0x97, 0x5e, 0x7a, 0xe9, 0x84, 0x78, 0x7d, 0xc5, + 0x8a, 0x15, 0xec, 0xdd, 0xbb, 0x57, 0x82, 0x3d, 0xc3, 0xeb, 0xdd, 0xff, + 0x12, 0xc0, 0xb1, 0x63, 0xf3, 0x2c, 0x78, 0x4e, 0x71, 0x41, 0x00, 0x00, + 0x0a, 0xa5, 0x49, 0x44, 0x41, 0x54, 0xa2, 0x07, 0x12, 0x84, 0x50, 0x56, + 0x6d, 0xdb, 0xb5, 0x4f, 0xda, 0x34, 0x6b, 0x22, 0xe3, 0x72, 0x80, 0xcd, + 0xa6, 0x31, 0x6c, 0xec, 0x14, 0x3e, 0xfb, 0xf6, 0x47, 0x40, 0x20, 0x44, + 0x85, 0x36, 0x5e, 0xef, 0xfe, 0x17, 0xaa, 0x55, 0x7b, 0x1a, 0x48, 0x3e, + 0xe5, 0x39, 0x1e, 0xcf, 0x4a, 0x14, 0x25, 0x1e, 0x29, 0xdd, 0xad, 0x35, + 0xad, 0xfc, 0xac, 0x11, 0x23, 0x46, 0x70, 0xfb, 0xed, 0xb7, 0xb3, 0x70, + 0xe1, 0x42, 0x9a, 0x35, 0x6b, 0x26, 0x5b, 0xb7, 0x6e, 0x0d, 0x38, 0x76, + 0x85, 0x85, 0x5d, 0x5d, 0xd1, 0xe9, 0xbc, 0xb6, 0x74, 0xe9, 0xa7, 0xf5, + 0xca, 0x8b, 0x8b, 0xe8, 0xa1, 0x0f, 0x4a, 0x99, 0xfd, 0xe9, 0x86, 0xef, + 0x27, 0x62, 0x9a, 0x66, 0x91, 0x16, 0x58, 0x58, 0xb8, 0x34, 0x08, 0x72, + 0xda, 0xb9, 0x7b, 0xe0, 0x6b, 0xfc, 0xb1, 0xf1, 0x6f, 0x09, 0x8a, 0xb0, + 0xd9, 0xaa, 0xb6, 0xf3, 0x7a, 0xf7, 0x2c, 0x2e, 0xea, 0xf9, 0x15, 0x2a, + 0xdc, 0xcd, 0x91, 0x23, 0x5f, 0xa1, 0x69, 0x71, 0x8f, 0xeb, 0x7a, 0xea, + 0xfb, 0xa0, 0x23, 0x44, 0xe8, 0xf7, 0x52, 0xea, 0xcf, 0x81, 0xfb, 0xcf, + 0xa0, 0xa0, 0x06, 0xb8, 0x5c, 0x9b, 0x2d, 0xa2, 0x07, 0x22, 0xec, 0xf6, + 0xf8, 0x06, 0x1e, 0xcf, 0xee, 0x4d, 0x75, 0x13, 0xaa, 0x32, 0xeb, 0xa3, + 0xe1, 0xb8, 0xf3, 0xac, 0x62, 0x0e, 0xa5, 0x11, 0x4e, 0x87, 0x8d, 0xde, + 0x4f, 0x8e, 0x66, 0xcd, 0xc6, 0xed, 0x12, 0x14, 0x21, 0x44, 0x74, 0x0b, + 0x29, 0x8f, 0xae, 0x3e, 0xd7, 0xeb, 0xd4, 0xa8, 0x71, 0x9c, 0xe4, 0xe4, + 0x13, 0x77, 0xa8, 0xc5, 0xc6, 0xd6, 0xe6, 0xe8, 0xd1, 0xd2, 0x59, 0x7b, + 0xd1, 0x22, 0x7a, 0xf1, 0x3a, 0x84, 0x5b, 0xc0, 0x5b, 0x6f, 0xd7, 0xd2, + 0xcf, 0xc9, 0xb4, 0xb2, 0xcd, 0x94, 0x2a, 0x48, 0x29, 0x09, 0x72, 0xda, + 0xb9, 0x6b, 0xe0, 0x6b, 0xac, 0xdb, 0xb4, 0x43, 0x82, 0x9a, 0x17, 0x14, + 0x94, 0x78, 0x85, 0xcb, 0xb5, 0xee, 0x40, 0x99, 0x18, 0x8f, 0xb0, 0x54, + 0xa0, 0x38, 0xdd, 0x77, 0x47, 0x30, 0xe8, 0x37, 0xe6, 0xb8, 0xdc, 0x34, + 0x4f, 0xac, 0x63, 0xb9, 0xef, 0xa5, 0x49, 0xd1, 0x55, 0x95, 0x6e, 0x8f, + 0x8e, 0x60, 0xd3, 0xb6, 0x64, 0x84, 0x70, 0x6e, 0xa8, 0x52, 0x65, 0x7c, + 0x42, 0x5a, 0xda, 0xc7, 0x19, 0x65, 0xe5, 0xf9, 0xad, 0xc1, 0xb8, 0xe2, + 0x72, 0x09, 0x9d, 0x09, 0x48, 0x69, 0x6c, 0x00, 0x98, 0xf3, 0xf3, 0x0a, + 0x82, 0x9d, 0x56, 0x8e, 0xf7, 0xd2, 0x60, 0xc5, 0x15, 0x21, 0x38, 0x78, + 0xe4, 0x18, 0x57, 0xdf, 0xf1, 0xa4, 0xf4, 0x15, 0xd8, 0x70, 0xcc, 0x94, + 0xd2, 0xdd, 0xe4, 0xc8, 0x91, 0x91, 0x9e, 0x32, 0xd5, 0xd1, 0x59, 0xea, + 0x50, 0x3c, 0xd0, 0x75, 0xa2, 0x84, 0x30, 0x66, 0x82, 0x5e, 0x2e, 0xc7, + 0xe5, 0x16, 0xa6, 0x34, 0xb9, 0xb2, 0x61, 0x2d, 0xcb, 0xaa, 0x5f, 0x52, + 0x2b, 0xae, 0xf0, 0xed, 0xf7, 0xbf, 0xd2, 0x77, 0xc8, 0xdb, 0xd2, 0xe3, + 0xf5, 0x0a, 0x87, 0xa3, 0xda, 0xab, 0x86, 0x91, 0xd6, 0x57, 0xd3, 0xae, + 0x40, 0xd7, 0x93, 0xcb, 0x96, 0xb7, 0x69, 0xa9, 0x43, 0x71, 0xb9, 0xed, + 0xf6, 0xcd, 0x52, 0x7a, 0xea, 0x55, 0xa9, 0x52, 0x45, 0x1c, 0x3a, 0x74, + 0x88, 0xa8, 0x88, 0x50, 0x96, 0x7d, 0x33, 0xd6, 0xaa, 0xb0, 0x7a, 0x09, + 0xac, 0xb8, 0xaa, 0x2a, 0xec, 0xdc, 0x73, 0x88, 0x5e, 0xfd, 0x5f, 0x95, + 0x79, 0x1e, 0x8f, 0x00, 0x45, 0x57, 0xd5, 0x2b, 0x3a, 0x18, 0xc6, 0xf6, + 0x45, 0xd1, 0xd1, 0x23, 0x48, 0x4d, 0x1d, 0x56, 0xe6, 0xe4, 0x62, 0xb9, + 0xee, 0xc5, 0x62, 0x39, 0xe2, 0x6e, 0x94, 0xd2, 0x53, 0x7f, 0xc0, 0x80, + 0x01, 0x62, 0xdf, 0xbe, 0x7d, 0x4c, 0x99, 0x32, 0x85, 0xd4, 0xb4, 0xe3, + 0xfc, 0xb6, 0x66, 0xb3, 0xb5, 0x52, 0xee, 0xa2, 0x13, 0x1d, 0xbe, 0x98, + 0xb9, 0x80, 0xdb, 0xfb, 0x0e, 0x97, 0x79, 0x1e, 0x8f, 0x50, 0x94, 0xa0, + 0xdf, 0xc1, 0xb4, 0xd9, 0x6c, 0x21, 0x8b, 0x80, 0x32, 0x49, 0x72, 0xcb, + 0xa2, 0x17, 0x9b, 0x35, 0xd7, 0xf6, 0x37, 0x69, 0x92, 0x58, 0xe9, 0x8f, + 0x3f, 0xfe, 0xf0, 0xcb, 0xb3, 0x79, 0xf3, 0xe6, 0xec, 0xd9, 0xbd, 0x83, + 0x05, 0x5f, 0xbe, 0x61, 0xe5, 0x91, 0xbb, 0x18, 0xb1, 0xb8, 0xa2, 0xb0, + 0xff, 0x50, 0x0a, 0x5d, 0xfb, 0x0e, 0xc7, 0xeb, 0xd5, 0x01, 0x21, 0x55, + 0xb5, 0x4a, 0x0f, 0xc3, 0xd8, 0x37, 0xc3, 0x92, 0x90, 0x45, 0xf4, 0x62, + 0xd3, 0xb5, 0xd4, 0xd4, 0x54, 0xca, 0x95, 0x2b, 0xe7, 0xcf, 0x1d, 0xe6, + 0x72, 0xb9, 0x08, 0x09, 0x09, 0x61, 0xec, 0xd0, 0x47, 0xb9, 0xbe, 0x75, + 0x52, 0x91, 0x52, 0x11, 0x59, 0x38, 0x77, 0x82, 0x17, 0xc8, 0xfb, 0xd9, + 0xd1, 0x93, 0xf8, 0x69, 0xe9, 0x1a, 0x09, 0xa0, 0x28, 0xc1, 0xdb, 0x4c, + 0x33, 0xb7, 0x9e, 0xa2, 0x54, 0xc3, 0x34, 0xf7, 0x5a, 0x82, 0xb2, 0x88, + 0x5e, 0x6c, 0xc8, 0x4b, 0x49, 0x49, 0xb1, 0x97, 0x2f, 0x5f, 0xfe, 0x04, + 0x32, 0xdb, 0xed, 0x76, 0x4c, 0xd3, 0x60, 0xed, 0xbc, 0x0f, 0x90, 0xa6, + 0xe5, 0xc2, 0x97, 0x44, 0x2c, 0x3e, 0x74, 0xcc, 0x67, 0xcc, 0xf9, 0x65, + 0x05, 0x80, 0x14, 0x42, 0x15, 0x52, 0x46, 0xb4, 0x83, 0xb4, 0xc5, 0x96, + 0x84, 0xac, 0x18, 0xbd, 0xf8, 0x85, 0xa8, 0x84, 0xfc, 0x18, 0x13, 0x13, + 0xc3, 0x3b, 0xef, 0xbc, 0x43, 0x72, 0x72, 0x32, 0x00, 0xcf, 0x3c, 0xf3, + 0x0c, 0x5e, 0xaf, 0x17, 0xc3, 0x30, 0x69, 0xde, 0xe5, 0x09, 0x2b, 0x56, + 0x2f, 0x46, 0x82, 0x03, 0xfc, 0xfc, 0xeb, 0x5a, 0xae, 0xec, 0xf4, 0x78, + 0x01, 0xc9, 0x51, 0x94, 0xf0, 0x85, 0x52, 0x36, 0x51, 0x7c, 0x24, 0xaf, + 0x61, 0x09, 0xca, 0xb2, 0xe8, 0xc5, 0x0b, 0x4d, 0xab, 0x89, 0xae, 0xef, + 0x42, 0xd3, 0x62, 0x5f, 0xd5, 0xf5, 0xa3, 0xcf, 0x9f, 0x4e, 0x3f, 0xc7, + 0xbc, 0xd0, 0x57, 0xdc, 0x70, 0x75, 0x13, 0x4b, 0x60, 0x17, 0x08, 0xd3, + 0x94, 0x74, 0x7d, 0x64, 0x18, 0x7b, 0x0f, 0xa6, 0x00, 0x48, 0xd0, 0xb2, + 0x6c, 0xb6, 0x4a, 0x57, 0x7a, 0xbd, 0x7b, 0x77, 0xda, 0x6c, 0xe5, 0xf1, + 0x7a, 0xad, 0x9c, 0x00, 0x16, 0xd1, 0x4b, 0x08, 0x4e, 0x67, 0x2b, 0xdc, + 0xee, 0x15, 0x68, 0x5a, 0xd5, 0x70, 0x5d, 0x3f, 0x1a, 0xa7, 0x69, 0x91, + 0x59, 0xa1, 0xa1, 0x6f, 0x1d, 0x76, 0xb9, 0xfe, 0x4f, 0x95, 0x72, 0xaf, + 0x4d, 0xd7, 0x93, 0xd7, 0x99, 0x66, 0x6e, 0x9d, 0x99, 0x13, 0x87, 0x13, + 0x5f, 0x35, 0xce, 0x12, 0xd8, 0x39, 0x5a, 0x70, 0x45, 0x11, 0x38, 0x1d, + 0x0e, 0x5e, 0x79, 0xf7, 0x4b, 0xa6, 0x7e, 0xb7, 0x10, 0xc3, 0x30, 0x01, + 0x05, 0x4d, 0x8b, 0xfe, 0xff, 0xf6, 0xce, 0x36, 0x28, 0xaa, 0xeb, 0x8c, + 0xe3, 0xff, 0x73, 0xee, 0xcb, 0x5e, 0x70, 0x5f, 0x0c, 0x20, 0x0c, 0x08, + 0x68, 0x42, 0x08, 0x45, 0xab, 0xc6, 0x3a, 0xd4, 0x69, 0x4a, 0x1b, 0x69, + 0x52, 0xda, 0x66, 0xc6, 0xb6, 0x93, 0x92, 0xa4, 0x93, 0xb6, 0x32, 0xed, + 0x87, 0xaa, 0xfd, 0x52, 0x26, 0x9d, 0x44, 0x74, 0x1c, 0x07, 0x94, 0xaa, + 0x33, 0x4e, 0x1a, 0x13, 0x32, 0x99, 0x8e, 0x09, 0x1d, 0x51, 0x74, 0xb0, + 0x11, 0xa3, 0x4e, 0x8d, 0x89, 0x14, 0x0d, 0x49, 0xad, 0x1a, 0x6b, 0x29, + 0x83, 0x75, 0xc6, 0x26, 0xbe, 0x2c, 0x48, 0x95, 0x42, 0x79, 0x87, 0xbd, + 0xbb, 0x77, 0xef, 0x7d, 0xfa, 0x81, 0xdd, 0x0d, 0x49, 0x51, 0x61, 0x9a, + 0xa4, 0xab, 0x3c, 0xbf, 0x99, 0xfd, 0xc4, 0xee, 0xb2, 0xf3, 0x9c, 0xf3, + 0xbb, 0xcf, 0x39, 0xe7, 0x9e, 0xf3, 0xdc, 0x92, 0x82, 0x82, 0xae, 0x86, + 0x53, 0xa7, 0xb8, 0x1b, 0xb3, 0xe8, 0x71, 0x80, 0xa2, 0x2c, 0xd6, 0x6d, + 0xbb, 0x2d, 0x08, 0x84, 0xe9, 0xcc, 0xc1, 0x6a, 0xa1, 0x6b, 0x2a, 0x2f, + 0xcc, 0x4d, 0xf6, 0x22, 0xea, 0xd2, 0xf1, 0xec, 0xe6, 0x1d, 0x38, 0x72, + 0xfc, 0x4c, 0xa4, 0x66, 0xbe, 0x84, 0x94, 0x33, 0x7e, 0xe5, 0x76, 0x97, + 0x6d, 0x1f, 0x1c, 0xdc, 0xe4, 0x78, 0x3c, 0x4b, 0x31, 0x34, 0x74, 0x86, + 0x03, 0xc5, 0xa2, 0xc7, 0x07, 0x29, 0x29, 0x8f, 0xa7, 0xf4, 0xf4, 0x1c, + 0xe8, 0x9e, 0x33, 0x3b, 0x8d, 0x0e, 0xbd, 0x5a, 0x29, 0x78, 0x15, 0xfe, + 0xe6, 0x19, 0xdc, 0xa5, 0x6b, 0x18, 0x1c, 0x0e, 0xe0, 0xf9, 0xd7, 0x5e, + 0xc7, 0xa1, 0x63, 0x7f, 0x8e, 0x54, 0xd6, 0x15, 0x90, 0xd2, 0x57, 0x91, + 0x9e, 0xfe, 0x46, 0x55, 0x67, 0x67, 0x91, 0xad, 0xaa, 0x59, 0x08, 0x87, + 0x3b, 0x38, 0x60, 0x2c, 0x7a, 0x9c, 0x05, 0x5a, 0x24, 0x43, 0xd3, 0x7c, + 0x45, 0xa1, 0xd0, 0xe5, 0xe3, 0x0f, 0x2d, 0x99, 0x4f, 0x3b, 0xb6, 0x94, + 0x09, 0xcb, 0x0a, 0xb3, 0xec, 0x11, 0xa4, 0x14, 0x30, 0x74, 0x1d, 0x35, + 0xaf, 0xbf, 0x85, 0x63, 0xef, 0x9e, 0x43, 0xcb, 0xdf, 0x3f, 0x8c, 0x75, + 0x51, 0x21, 0x12, 0x36, 0xac, 0x58, 0x31, 0x52, 0x55, 0x5b, 0x2b, 0x78, + 0x45, 0x93, 0x45, 0xbf, 0x53, 0x3a, 0xb4, 0x6f, 0x8d, 0xe3, 0x0c, 0x6c, + 0x2d, 0xfa, 0xca, 0x83, 0x54, 0x5d, 0xf1, 0x0b, 0x61, 0x85, 0xed, 0x69, + 0x2b, 0x3b, 0x11, 0x41, 0x91, 0x12, 0xc3, 0x01, 0x13, 0xaf, 0xec, 0x3e, + 0x8c, 0xba, 0x37, 0x9a, 0x80, 0x48, 0x15, 0xd6, 0x88, 0xe0, 0x75, 0x86, + 0x91, 0xb3, 0x36, 0x10, 0x68, 0xbb, 0xc6, 0x3d, 0x87, 0x45, 0xbf, 0xf3, + 0xe6, 0x9d, 0x46, 0x5e, 0xb1, 0x69, 0x5e, 0x7c, 0xfb, 0x07, 0xdf, 0x29, + 0xa4, 0xca, 0xb2, 0x52, 0x61, 0x85, 0xa7, 0x4f, 0x66, 0x8f, 0x9e, 0x28, + 0xd3, 0x75, 0x0d, 0x97, 0xdb, 0xaf, 0xe3, 0xc8, 0x89, 0xf7, 0xf1, 0xca, + 0xee, 0xc3, 0xe3, 0x04, 0xd7, 0xcf, 0x08, 0xa1, 0x35, 0x10, 0x8d, 0x6c, + 0x1b, 0xfb, 0xc4, 0x5c, 0xdc, 0xac, 0xb4, 0x13, 0xc3, 0xa2, 0xc7, 0x3d, + 0x8a, 0x92, 0xba, 0xc6, 0xb6, 0xff, 0xb5, 0x75, 0x7e, 0xee, 0x1c, 0xaa, + 0x28, 0x5b, 0x21, 0xe6, 0xe5, 0x66, 0x23, 0x74, 0x97, 0x0f, 0xe5, 0x35, + 0x55, 0xc1, 0x8d, 0x9e, 0x7e, 0x9c, 0x3c, 0x7b, 0x1e, 0x3b, 0xea, 0xdf, + 0xc4, 0xb5, 0xeb, 0xdd, 0xe3, 0x04, 0x97, 0x03, 0xaa, 0x9a, 0x56, 0x1a, + 0x0e, 0x5f, 0x3f, 0xc4, 0xbd, 0x83, 0x45, 0xbf, 0xab, 0x50, 0xd5, 0xac, + 0x1f, 0x85, 0xc3, 0x1d, 0x75, 0x00, 0xe8, 0xa9, 0xe5, 0xcb, 0xc4, 0xda, + 0xd5, 0x3f, 0xbc, 0xeb, 0xb6, 0xca, 0x2a, 0x52, 0x62, 0xd4, 0x0c, 0xe2, + 0xc2, 0x07, 0x7e, 0xbc, 0xb4, 0xf3, 0x60, 0x74, 0xee, 0x4d, 0x1f, 0xf5, + 0x3d, 0xd1, 0x21, 0x84, 0x77, 0x2f, 0xd1, 0x40, 0x39, 0xf7, 0x08, 0x16, + 0xfd, 0xee, 0xcd, 0x72, 0x5a, 0xf6, 0x63, 0x96, 0xd5, 0x7e, 0x04, 0x00, + 0xa5, 0xa7, 0x26, 0x89, 0xba, 0xed, 0xe5, 0x98, 0x95, 0xe4, 0x83, 0x6d, + 0xff, 0x77, 0x81, 0xc9, 0xe8, 0x45, 0x40, 0x0a, 0x01, 0xcb, 0xb6, 0xe1, + 0xd8, 0x0e, 0x74, 0x4d, 0x8d, 0xab, 0xa7, 0xb8, 0x46, 0x7f, 0xa3, 0x10, + 0xc0, 0x95, 0x8e, 0x2e, 0xbc, 0x5c, 0x7b, 0x10, 0x7f, 0x3c, 0xd9, 0xf2, + 0xc9, 0x6e, 0xd7, 0x27, 0xc4, 0x8c, 0xa6, 0xec, 0xec, 0xbf, 0x3c, 0xed, + 0xf7, 0x7f, 0x81, 0xcf, 0xf1, 0xb2, 0xe8, 0xd3, 0x25, 0xb3, 0xdf, 0x77, + 0x6f, 0x38, 0xec, 0xbf, 0x00, 0xd8, 0x06, 0x00, 0x14, 0x7f, 0x6d, 0x09, + 0xb6, 0x3c, 0xf7, 0x33, 0x28, 0x8a, 0x12, 0xdb, 0xf2, 0x29, 0x84, 0x80, + 0x19, 0x0c, 0xe1, 0xc2, 0x87, 0xed, 0xd8, 0xd5, 0xd0, 0x88, 0x77, 0x4e, + 0xb7, 0x46, 0x86, 0xc3, 0x2a, 0xde, 0xdc, 0xf9, 0x6b, 0xcc, 0x4a, 0xf2, + 0xfd, 0xdf, 0x47, 0x03, 0xb6, 0x6d, 0x23, 0x60, 0x86, 0xd0, 0x74, 0xb2, + 0x05, 0x95, 0x2f, 0xd5, 0x4d, 0x70, 0x6a, 0x4f, 0xf5, 0x03, 0x99, 0x5f, + 0x05, 0xae, 0x76, 0x6a, 0xda, 0x3c, 0x58, 0xd6, 0x05, 0x6e, 0x7c, 0x16, + 0x7d, 0xfa, 0x60, 0x18, 0x5f, 0xc2, 0xf2, 0xe5, 0xe7, 0x94, 0xfd, 0xfb, + 0x13, 0x87, 0x89, 0x02, 0x06, 0x00, 0x12, 0x02, 0xe2, 0xb7, 0x55, 0xbf, + 0xc4, 0xd2, 0xc5, 0xf9, 0x20, 0x22, 0x1c, 0x3a, 0x76, 0x0a, 0x15, 0x2f, + 0xee, 0x8a, 0x64, 0xcb, 0xb1, 0x97, 0xe3, 0x38, 0xb1, 0x0b, 0xc1, 0xb3, + 0x3f, 0x7f, 0x02, 0x4f, 0x7f, 0xef, 0x1b, 0xb1, 0xd3, 0x5c, 0x9f, 0x97, + 0xf0, 0x8e, 0x43, 0x08, 0xdb, 0x36, 0xb6, 0xd7, 0x34, 0x60, 0xef, 0xe1, + 0x13, 0x13, 0xbe, 0x05, 0xd0, 0xbb, 0xa5, 0x9c, 0xfd, 0xb0, 0xe3, 0x5c, + 0xb9, 0xc8, 0xad, 0xcd, 0xa2, 0x33, 0x00, 0x34, 0x2d, 0x73, 0xbd, 0x65, + 0x5d, 0xdb, 0x14, 0xcd, 0xe2, 0x8d, 0xbb, 0xb7, 0xa2, 0xea, 0xe5, 0xbd, + 0x78, 0xe7, 0x74, 0x2b, 0xf6, 0xed, 0xdb, 0x87, 0x92, 0x92, 0x92, 0x8f, + 0x1a, 0x4f, 0x08, 0x34, 0x37, 0x37, 0xa3, 0xa8, 0xa8, 0x08, 0xc0, 0x58, + 0xbd, 0xf2, 0x4d, 0xcf, 0xfc, 0x14, 0x8f, 0x16, 0x2e, 0xfe, 0x4c, 0x64, + 0x1f, 0xff, 0x9d, 0x9d, 0x37, 0x7a, 0xf0, 0xbb, 0xdf, 0xbf, 0x85, 0x03, + 0x6f, 0xff, 0x09, 0x13, 0x9f, 0xd7, 0xd1, 0x2f, 0x4a, 0x39, 0x73, 0xed, + 0xfd, 0xf7, 0xff, 0xf8, 0x0f, 0x57, 0xaf, 0xb6, 0x5a, 0xa1, 0x50, 0x13, + 0x16, 0x2d, 0x6a, 0x43, 0x6b, 0xeb, 0x02, 0x6e, 0x64, 0x16, 0x7d, 0xba, + 0x67, 0xf6, 0x2f, 0xc2, 0x34, 0xcf, 0x43, 0x51, 0x52, 0x5d, 0x9a, 0x96, + 0xf8, 0x75, 0xd3, 0xec, 0x3c, 0x06, 0x58, 0x40, 0x64, 0xf1, 0x2a, 0x18, + 0x0c, 0x42, 0xd3, 0xb4, 0x98, 0x6c, 0x51, 0xf1, 0x4c, 0xd3, 0xc4, 0xaa, + 0x55, 0xab, 0x50, 0x5b, 0x5b, 0x0b, 0x00, 0xf0, 0xb8, 0x13, 0x71, 0xfa, + 0xc0, 0x8b, 0xb0, 0xc2, 0xf6, 0xa7, 0x22, 0xb7, 0x14, 0x02, 0x9a, 0xae, + 0xe1, 0xaf, 0xe7, 0x3f, 0xc0, 0x73, 0x9b, 0x5f, 0x45, 0x57, 0x4f, 0x1f, + 0xec, 0x09, 0x8b, 0x69, 0xb8, 0xb7, 0x09, 0x21, 0x37, 0x4b, 0xa9, 0x9a, + 0xb6, 0xdd, 0x6b, 0x72, 0x8b, 0xb2, 0xe8, 0xcc, 0x2d, 0xf0, 0x78, 0x1e, + 0xc4, 0xd0, 0xd0, 0xdf, 0xa2, 0xfa, 0xbf, 0x00, 0x98, 0x65, 0x95, 0x95, + 0x95, 0xd8, 0xb0, 0xe1, 0xd6, 0x65, 0x90, 0x46, 0x46, 0x46, 0xb0, 0x72, + 0xe5, 0x4a, 0xec, 0xd9, 0xb3, 0x07, 0x52, 0x4a, 0x6c, 0xdf, 0xb0, 0x1a, + 0x0f, 0x2f, 0x5d, 0x38, 0xe9, 0x27, 0xc7, 0xc4, 0x16, 0xfb, 0xa4, 0x84, + 0x4b, 0xd7, 0xf0, 0xcf, 0xae, 0x7f, 0x63, 0xff, 0xd1, 0x77, 0xd1, 0x70, + 0xf4, 0x3d, 0xf4, 0x0d, 0x0c, 0x8f, 0xaf, 0x7f, 0x17, 0xdb, 0xd4, 0xa2, + 0xaa, 0x69, 0x4f, 0x86, 0xc3, 0xbd, 0x4d, 0x2e, 0x57, 0x56, 0x20, 0x18, + 0xbc, 0x14, 0xe0, 0xd6, 0x63, 0xd1, 0x99, 0x29, 0xe2, 0x72, 0x7d, 0x19, + 0xc1, 0x60, 0x4b, 0x2f, 0x60, 0xdd, 0x33, 0x30, 0x30, 0x00, 0x8f, 0xc7, + 0x73, 0x4b, 0x61, 0xa3, 0xa2, 0x36, 0x36, 0x36, 0xa2, 0xb8, 0xb8, 0x18, + 0x00, 0xb0, 0x66, 0xf5, 0x53, 0x28, 0x7d, 0xfc, 0x9b, 0xb1, 0x79, 0xbd, + 0x19, 0x0c, 0x61, 0xd4, 0x0c, 0xe2, 0xe8, 0x89, 0xb3, 0x48, 0x30, 0x74, + 0xd8, 0xb6, 0x8d, 0xf3, 0xff, 0xb8, 0x8a, 0xb4, 0x59, 0x49, 0x38, 0xdb, + 0x7a, 0x11, 0x7d, 0x03, 0x43, 0x18, 0x19, 0x35, 0x71, 0xa3, 0xbb, 0x17, + 0x56, 0xd8, 0x1e, 0x77, 0x1b, 0xcc, 0xd5, 0x0e, 0xc8, 0x5d, 0x80, 0xad, + 0x28, 0x8a, 0xe7, 0x12, 0x51, 0xe8, 0xf8, 0xdc, 0xb9, 0xdb, 0xfc, 0x97, + 0x2f, 0xaf, 0xe2, 0x3a, 0x59, 0x0c, 0xf3, 0xbf, 0xe0, 0xf5, 0x3e, 0x9a, + 0x05, 0xc0, 0x29, 0x2c, 0x2c, 0xa4, 0xc9, 0xe2, 0x38, 0x0e, 0x39, 0x8e, + 0x43, 0x8a, 0xa2, 0x10, 0x00, 0xd2, 0x34, 0x95, 0xee, 0xcb, 0x4e, 0xa7, + 0xf4, 0xd4, 0x24, 0x9a, 0x95, 0x3c, 0x93, 0x22, 0xd9, 0x78, 0x12, 0x2f, + 0xd5, 0x2f, 0xa5, 0xfb, 0x79, 0x20, 0x71, 0xa9, 0xa6, 0x3d, 0xe0, 0xfb, + 0xe4, 0x6f, 0x4b, 0x48, 0x78, 0x88, 0x1b, 0x88, 0x33, 0x3a, 0xf3, 0xe9, + 0x90, 0x70, 0x04, 0x08, 0x3c, 0xd6, 0xdf, 0xdf, 0x0f, 0x9f, 0xcf, 0x37, + 0xe9, 0x39, 0xb5, 0x10, 0x02, 0xaa, 0xaa, 0xc2, 0xb6, 0x6d, 0x02, 0x84, + 0x00, 0xb4, 0xe3, 0x80, 0x33, 0x1f, 0x40, 0x1a, 0x20, 0xdb, 0x00, 0x9c, + 0x02, 0x30, 0x04, 0x60, 0x09, 0x40, 0x06, 0xa0, 0x9c, 0x03, 0x9c, 0x4b, + 0x42, 0x28, 0x3d, 0x00, 0x06, 0x14, 0x25, 0xa5, 0x2d, 0x1c, 0xee, 0xb8, + 0xc2, 0xf1, 0x67, 0x98, 0xcf, 0xf2, 0xea, 0x2b, 0xd2, 0xe1, 0x76, 0x2f, + 0x50, 0x00, 0x41, 0x25, 0x25, 0x25, 0x0e, 0x4d, 0x91, 0x8d, 0x1b, 0x37, + 0xc6, 0x32, 0xb3, 0xa2, 0xcc, 0xc9, 0x9b, 0xca, 0xff, 0x5e, 0xb8, 0x70, + 0x27, 0x37, 0x00, 0xc3, 0x7c, 0x5e, 0x48, 0x79, 0xcf, 0x0b, 0x00, 0x9c, + 0xe8, 0x70, 0x7c, 0xb2, 0xf4, 0xf6, 0xf6, 0x12, 0x00, 0x07, 0x00, 0x49, + 0xe9, 0xdd, 0xc5, 0x91, 0x64, 0x98, 0x38, 0x45, 0xd7, 0xf3, 0xb3, 0x00, + 0x50, 0x45, 0x45, 0xc5, 0x94, 0x32, 0xf9, 0x78, 0xc9, 0x01, 0xa3, 0x05, + 0x18, 0xdb, 0x8c, 0xc3, 0x30, 0x4c, 0x1c, 0x91, 0x9f, 0xdf, 0x11, 0x19, + 0xba, 0x1b, 0xdd, 0xa9, 0xa9, 0xa9, 0x93, 0xce, 0xe6, 0xd1, 0xf7, 0xb8, + 0xdd, 0x6e, 0x07, 0x10, 0x04, 0x18, 0x5b, 0xc6, 0x46, 0x05, 0x19, 0x1c, + 0x54, 0x86, 0x89, 0x47, 0x14, 0x25, 0xe3, 0x11, 0x00, 0xd4, 0xdf, 0xdf, + 0x3f, 0xa5, 0x21, 0xfb, 0xfa, 0xf5, 0xeb, 0x1d, 0x00, 0xa4, 0xeb, 0xf3, + 0xbe, 0x3b, 0x63, 0xc6, 0x42, 0x0e, 0x24, 0xc3, 0xc4, 0x23, 0x1e, 0xcf, + 0xb7, 0xe0, 0xf5, 0x6e, 0x13, 0x80, 0x4a, 0xe9, 0xe9, 0xe9, 0x53, 0x1a, + 0xb2, 0x77, 0x77, 0x77, 0x13, 0x00, 0x12, 0x22, 0xf1, 0x3d, 0x8e, 0x24, + 0xc3, 0xc4, 0x39, 0x52, 0x7a, 0x7f, 0x02, 0xc0, 0xa9, 0xae, 0xae, 0xfe, + 0x58, 0x36, 0xbf, 0x5d, 0x66, 0x6f, 0x6e, 0x6e, 0x26, 0x00, 0x64, 0x18, + 0x79, 0x3c, 0x21, 0x67, 0x98, 0xf8, 0x17, 0xdd, 0x30, 0x53, 0x52, 0x52, + 0x62, 0x02, 0x97, 0x95, 0x95, 0x91, 0xdf, 0xef, 0xbf, 0xed, 0xdc, 0x3c, + 0x39, 0x39, 0xd9, 0x01, 0x04, 0x3f, 0xb9, 0x80, 0x61, 0xe2, 0x5f, 0xf2, + 0xa4, 0x05, 0x00, 0xa8, 0xa0, 0xa0, 0x80, 0x88, 0x88, 0x06, 0x07, 0x07, + 0x63, 0xf7, 0xc2, 0x6f, 0x95, 0xd1, 0xeb, 0xeb, 0xeb, 0xa3, 0xc3, 0xf6, + 0xdf, 0x70, 0x14, 0x19, 0x26, 0xee, 0x99, 0x79, 0xef, 0x78, 0xd1, 0x2d, + 0xcb, 0x8a, 0x89, 0x3e, 0x3a, 0x3a, 0x7a, 0x53, 0xd9, 0xbd, 0x5e, 0x2f, + 0x8d, 0xad, 0xb4, 0x33, 0x0c, 0x13, 0xf7, 0xcc, 0x9e, 0xbd, 0x57, 0x03, + 0x40, 0x19, 0x19, 0x19, 0x31, 0x89, 0x73, 0x73, 0x73, 0x09, 0x00, 0x95, + 0x96, 0x96, 0x4e, 0x28, 0x79, 0x6d, 0x6d, 0x6d, 0xe4, 0xbe, 0xb9, 0xf1, + 0x3e, 0x47, 0x90, 0x61, 0xee, 0x98, 0xe1, 0xfb, 0xcc, 0x9d, 0x00, 0x9c, + 0xfa, 0xfa, 0x7a, 0x72, 0x1c, 0x27, 0xba, 0x01, 0xe6, 0x63, 0x59, 0x3d, + 0x9a, 0xd9, 0x6b, 0x6a, 0x6a, 0x62, 0x7f, 0x53, 0xd5, 0xfc, 0x45, 0x86, + 0x31, 0x8f, 0x03, 0xc8, 0x30, 0xf1, 0x8e, 0xa6, 0x3d, 0x00, 0x4d, 0x5b, + 0xec, 0x03, 0x5c, 0x7d, 0x8a, 0x22, 0xa9, 0xbc, 0xbc, 0x9c, 0x86, 0x87, + 0x87, 0x29, 0x27, 0x27, 0x87, 0x00, 0xd0, 0xb2, 0x65, 0xcb, 0xa8, 0xa1, + 0xa1, 0x81, 0xd6, 0xad, 0x5b, 0x47, 0x79, 0x79, 0x79, 0x51, 0xc9, 0x1d, + 0x21, 0xb4, 0x61, 0x8e, 0x1e, 0xc3, 0xdc, 0x81, 0x08, 0xe1, 0x79, 0x06, + 0x50, 0x26, 0x71, 0x94, 0x54, 0x92, 0x94, 0x99, 0x73, 0x39, 0x62, 0xcc, + 0x6d, 0xfb, 0x14, 0x87, 0x20, 0xbe, 0x30, 0x8c, 0x02, 0x98, 0xe6, 0x59, + 0x00, 0x80, 0xa2, 0x24, 0x7d, 0xdb, 0x71, 0x02, 0x39, 0x80, 0xe8, 0x02, + 0x44, 0x3b, 0x40, 0xdf, 0x27, 0x0a, 0xe5, 0x02, 0xc8, 0x91, 0xd2, 0x75, + 0x30, 0x33, 0xf3, 0x4a, 0x55, 0x7b, 0x7b, 0x2a, 0x17, 0x7e, 0x60, 0x6e, + 0xcb, 0x7f, 0x00, 0x4a, 0x3f, 0xff, 0x3a, 0x92, 0xd3, 0x63, 0x31, 0x00, + 0x00, 0x00, 0x00, 0x49, 0x45, 0x4e, 0x44, 0xae, 0x42, 0x60, 0x82, + } +} diff --git a/third_party/github.com/jteeuwen/go-bindata/testdata/nomemcpy-compressed.go b/third_party/github.com/jteeuwen/go-bindata/testdata/nomemcpy-compressed.go new file mode 100644 index 00000000000..39bed17a0b2 --- /dev/null +++ b/third_party/github.com/jteeuwen/go-bindata/testdata/nomemcpy-compressed.go @@ -0,0 +1,1403 @@ +package main + +import ( + "bytes" + "compress/gzip" + "io" + "reflect" + "unsafe" +) + +var _gophercolor_png = "" + + "\x1f\x8b\x08\x00\x00\x09\x6e\x88\x00\xff\x34\x9b\x65\x50\x1b\x5f" + + "\x17\xc6\x43\x69\x8b\xbb\xbb\xbb\x16\x2d\x5e\xb4\xb8\xbb\x16\x77" + + "\x77\xa7\xb8\x14\x77\x28\xee\xee\x4e\x70\x77\x08\x52\xdc\x09\xee" + + "\xae\x6f\xfe\x1f\xde\xc9\x6c\x32\xb3\xb3\xb9\xbb\xf7\xec\x39\xe7" + + "\x79\x7e\xd9\x9b\x08\x45\xf9\x9f\x48\xf0\xf8\xf0\x00\x00\x00\x49" + + "\x4a\x52\x4c\x19\xf2\xf9\xfc\xdf\x06\xfb\x15\xf2\x1e\x7e\xaa\x2d" + + "\x00\x00\xc0\x2b\x58\x8a\x8a\x2a\x2a\x5a\xd8\xbb\xd8\x3b\x5b\xd8" + + "\x3b\x90\x4a\x89\x8a\x92\x3a\x38\xd9\x9b\x59\xda\x98\x02\x00\x1e" + + "\x2b\x59\x2a\xe6\xaa\x2a\x47\xd8\x02\x0f\x1b\xb7\x22\x32\xe1\x01" + + "\x49\x32\xf6\xca\x58\xb0\xa4\xca\x22\x51\x01\x98\xf1\xd4\xf4\x64" + + "\x70\xa8\xd2\xe1\x64\x79\xcb\x58\x4a\xbd\x68\xe2\xe2\x9f\x09\x46" + + "\x72\xc3\xa1\x63\x62\x02\xfe\x60\x29\x31\x22\x46\xc2\xff\x83\xd9" + + "\x23\xcb\x8f\x09\x2a\x08\x8f\x1c\x7b\xd9\xf6\x2e\xb0\x5e\xec\x7c" + + "\x38\x98\x7c\x6f\x5a\x12\x04\x57\xde\x64\x35\x4c\xc3\xf4\xc0\x22" + + "\xa6\x48\xb2\x2b\x71\x06\x20\x96\x89\x10\xa3\x6d\x05\x0d\x0f\x0c" + + "\x6d\xef\x31\xff\xf8\x0d\x4f\xe9\x00\x40\x85\x6d\x30\x21\x73\x7e" + + "\xa3\x80\x02\x3c\xfa\xf1\xf3\x33\x90\xf7\xc0\x74\x01\xa0\x3c\x40" + + "\x08\xb0\x80\x1e\xb9\xcc\x1e\x56\xc2\xb0\x0f\x94\x63\x91\x74\x5d" + + "\xa8\x80\x60\xa8\x1e\x97\xf8\xef\x32\xb0\x01\x98\x80\x1f\x5e\x31" + + "\x22\xc5\x80\x1f\x12\x50\x01\x59\x69\xd4\x2a\x80\xdc\xcf\x80\x5f" + + "\xe3\xc6\x46\xbb\x00\x45\x26\xc0\x2f\x1a\xff\x03\x10\x20\x20\xeb" + + "\x31\xdd\x1b\x0a\xa0\x93\x44\x86\x05\x95\x1b\x0f\x20\xc5\x30\x0d" + + "\x17\x03\x58\xf0\x01\x2a\xa7\xd5\x23\xc5\x01\x9a\xac\x00\x4c\x33" + + "\x99\x01\x1e\xc0\x3f\x26\x00\xab\x94\x86\x99\x24\xa0\xb6\x05\xd0" + + "\x33\x85\x8a\x50\x0f\x80\x45\x04\xb0\x2a\x85\x87\xd0\x01\x3e\x7b" + + "\x03\x7e\x8d\x50\x50\x78\x00\x82\xd3\x01\x98\x12\xd7\x1a\xfc\x57" + + "\x0c\xd5\xa8\x60\x7a\x48\xa4\xd2\xeb\xf9\xdb\x28\x79\xc4\x7f\xeb" + + "\xc0\x32\x39\xc2\x68\x68\x30\x13\xd3\x4e\x49\xe1\x30\x60\x72\x19" + + "\x7e\x32\xcc\xf8\xc1\xdc\xe7\x99\x8e\xc7\xf6\x9b\x03\x65\xeb\x66" + + "\x02\x00\xc8\x8d\xc7\x82\xcc\xf6\xe6\xcd\x63\x0a\xb9\x7a\x6a\x8a" + + "\x2b\xa6\x19\xd9\x80\xe9\xb8\xfb\xeb\x07\xb9\x91\xd1\xee\xfb\xfe" + + "\x74\x85\xc3\x0f\x00\x60\xdb\xc5\x6f\xf6\x9d\x91\xa5\x01\x3f\x80" + + "\xef\x6b\x80\xed\x7b\x3e\xd5\xf9\x67\x0b\x7d\x84\x5c\xd7\x87\x28" + + "\xb3\x7a\x94\x1f\x4d\x80\xdc\xa3\x15\x8d\x1b\x87\xff\x62\x23\x9e" + + "\x17\xd7\xb6\xbc\xbc\xbf\xb7\xf7\x4f\x7a\x40\x44\xc7\x70\x4c\xc3" + + "\xf7\xdd\xbc\xcf\xa0\x47\xe3\xcd\xe6\xc9\x8f\xff\xf5\xe1\x6e\xab" + + "\x6b\x87\xf2\x37\x9b\xde\xef\x9f\x9f\x1f\xb7\xfa\x26\x6e\x65\x8b" + + "\x08\xa7\x13\xe0\x82\x8d\xd6\x0e\x62\x7f\xbe\xb6\x20\x7c\xbc\x12" + + "\x81\xc8\x07\xa4\x8c\xda\x34\xe9\x31\x77\x95\xd0\x1d\x63\xc4\x32" + + "\xfe\xdc\xb0\x15\x91\x47\x88\x24\xd0\x0d\x52\xcd\x7d\x98\xec\x6c" + + "\x32\xbe\x40\x0b\x6e\x72\x02\x6a\xad\x04\xa1\xbc\xe3\x99\x4a\xf5" + + "\x8c\xa1\xef\x65\x68\x50\x35\x5d\x7a\x76\x1e\x00\x80\xeb\x0e\xfb" + + "\xde\x79\x5a\x58\xe8\x00\x8b\xa0\xed\x69\xf7\x8f\x0b\xa1\xb7\x9f" + + "\xb9\x54\x80\x00\x33\xa9\x04\x47\x00\x40\x5f\x8c\x92\x59\x75\xac" + + "\x41\x68\x18\x16\x00\x10\xcb\xfd\x4d\x5f\x2b\x4c\x70\xdb\x8b\xcb" + + "\xd8\xf3\x95\xb0\xf7\xcb\x5a\x2f\xc2\xeb\x2f\xae\x60\x11\xca\x81" + + "\x3e\x11\x54\x11\x78\x93\x00\x42\xa7\x5f\x74\x95\x81\x22\xb4\xa1" + + "\x93\xb5\x44\xf4\xbf\x58\xe6\x7f\x10\x71\xf4\x28\x59\x84\x26\x3a" + + "\x20\x33\xf7\xa9\xb5\x22\x0b\x38\xa0\x3c\x1b\xc2\x66\xf7\xd2\x76" + + "\xfe\x86\xfb\x2c\x3c\x02\x8b\x61\x48\xb6\x12\x0e\x65\x14\xa9\x49" + + "\x11\x03\x8b\x93\x1e\xf2\x48\xd6\x2b\xf9\x19\x3d\x8a\x92\x74\x34" + + "\x1c\x53\x89\x5c\x26\x9e\x53\xf2\x9b\x72\xa4\x2a\xa9\x9a\x24\xd1" + + "\xb5\x80\xd3\x27\x9e\x30\x5d\x89\xf6\x78\xbe\x11\x00\x5b\xe0\x6b" + + "\x68\xe7\xcf\x6f\x49\xbf\xeb\x95\x04\xe6\x11\x9b\x44\xdb\x4a\xb9" + + "\xf1\x83\x25\xf2\x11\x66\x4c\x5c\xbe\x65\xc4\x62\xe7\x66\xce\x38" + + "\x35\xe2\x0b\xfc\xe1\xca\x03\xcf\x54\x58\xa0\xac\xc4\x0a\x8a\x0e" + + "\xb3\xf6\x1c\xe2\xc2\xb0\x0f\xd8\xb0\x32\x0f\xf5\x8b\xd4\x7d\x67" + + "\x84\x4b\x34\x1e\xaf\x23\xaf\x44\xac\x18\xc2\xaf\x55\xaf\x84\xee" + + "\x88\xb8\x33\x9e\xae\x73\xff\x8c\xfe\x5b\xbc\x07\x8e\xeb\xb3\xbb" + + "\x08\xa9\x21\xb1\xb0\xa4\xa6\xac\xa6\xfc\x3f\xc9\x52\x52\x42\x4a" + + "\x0e\xb4\x15\x38\x6e\x38\xe8\x60\xa5\x3e\x1a\xf2\xc4\x52\x99\x46" + + "\xea\x76\xb4\xa7\x01\x5c\x23\xb6\x6f\x61\x9a\x92\x8c\x14\x8b\xe8" + + "\xb1\x18\x2c\xa8\xde\xe1\xa2\x03\xdc\x94\xe8\x11\xa2\x6c\x34\x6d" + + "\x71\x9f\xa4\xea\x0a\x55\xe7\xd0\xe7\x24\xe6\xec\x94\x29\x8e\x18" + + "\xcb\x32\x39\x24\xf1\x29\x12\x47\x57\x4c\xac\xeb\x61\x78\x92\x18" + + "\x49\x19\x46\x42\xf6\xb2\xf6\xfa\xf7\xd8\xf7\x08\xf6\xc8\xae\x75" + + "\xe0\xb3\x8c\x7e\x38\x16\x3e\xaa\xec\xd0\x2a\x8f\x59\x49\xe3\xed" + + "\xa3\xee\x73\xec\x7f\x35\x4d\x63\x13\x56\x2b\xc8\x50\x5e\x2f\xcd" + + "\x53\x45\xe3\x8c\xd5\x16\xa9\xca\x6b\x56\xae\x51\x0a\x2b\xc5\xe0" + + "\x70\xcd\xe0\x99\x0c\xc2\x96\x92\x29\xaa\xca\x4f\x06\xe1\x5b\xe0" + + "\x58\x3c\x58\xd6\x1c\xba\xb4\xa3\xad\x27\x13\xcb\xa7\xce\x68\xcc" + + "\x8d\x1f\xfd\x3c\x4c\x3d\x84\xbe\x75\x47\x44\x0a\xc5\x0a\x1a\x0e" + + "\x37\xa7\xc5\xcb\xc4\x85\xc1\x35\xc7\x73\xc7\xad\x49\x93\x2d\xc6" + + "\x03\x45\xe1\x0d\xab\xb2\x72\x7f\x3b\x49\x03\xa7\xa3\xa4\xdb\xab" + + "\x69\xd2\x37\xd3\xfb\x60\xc5\x8f\xc1\x27\x4b\x27\x53\x27\x13\xd0" + + "\xb3\xa8\x96\x95\x17\x97\x6f\x94\x23\xa8\xdd\xa8\x8d\xab\xc6\x96" + + "\xe9\xab\xf8\x56\x4b\xa8\xb1\xab\xec\x96\xc1\xcd\xa7\x54\xa8\x95" + + "\x67\x29\x9e\xa8\x10\xa8\x98\x97\xf1\x95\x86\x95\x9b\x97\xda\xab" + + "\x0a\x15\xfa\x69\xbd\x45\x7c\xd5\xea\x97\xed\x57\x10\x2d\x54\xca" + + "\xad\x33\xd8\xed\x1c\x22\x45\x30\xc6\x6b\xf0\x60\x24\xb3\x46\xaa" + + "\x08\x75\xfd\xcd\x39\x40\x5d\x37\xb9\xec\xe3\x46\xdf\x91\xf6\x46" + + "\xd4\x11\x25\x50\x5e\x96\xc7\x23\xca\x2e\xcd\xae\xd1\xa0\x7c\x93" + + "\x64\x26\x04\xb3\x9b\xe6\xf8\xf8\xd7\x45\x1e\x0e\x4c\x5b\x92\x7a" + + "\xe3\x17\xb9\x81\x79\x41\xb7\x43\x47\x50\xcc\x21\xdb\xf6\xcd\x54" + + "\xd7\xba\x3c\x25\x35\x45\xb3\x42\xb2\x42\xa9\xe2\xe7\x09\x90\xab" + + "\xfa\xac\x24\xab\x18\x28\xde\xa1\xf5\xf2\x1b\x59\x78\x58\x9a\x46" + + "\x97\xe6\xa7\x79\xec\xd4\x97\x2c\xb0\xca\xb2\xca\x7a\x09\x7c\x09" + + "\xb6\x9c\x00\xef\x5d\x35\x73\xb9\x75\x11\x41\x5e\x87\x98\x7b\xb7" + + "\x5d\xc9\x79\x5a\x76\x97\x7e\x40\xb6\xac\x7d\xc9\x86\x27\x10\xfc" + + "\x4c\xe8\xcb\xf2\xa6\xfa\x66\xfb\x5c\x72\xf7\x13\x51\xf3\x6b\x03" + + "\x22\xe5\xd7\x55\xc4\x29\x5c\xfe\x41\x4e\x47\x7b\x7e\x22\x96\xe1" + + "\xd5\x2b\x25\x51\xbd\xfe\x1f\xa2\x45\x86\xc9\x86\xa9\xdb\xc1\xf1" + + "\x6d\xb3\xfc\x05\x0b\x12\xd1\x12\x28\x7f\x2c\x87\x74\x77\x29\x77" + + "\x6d\x87\x6c\x87\x0a\xa8\xbf\x52\x93\x51\xcb\xc8\x1d\xcb\x9d\x67" + + "\x28\xcb\xb5\x15\xd3\x15\xf3\xc9\xf2\xc9\x0e\x4f\x4e\x4c\xe6\x4f" + + "\x2e\xa6\x73\x66\x0a\x7c\x5b\x62\x03\x67\x82\x33\x97\x32\x37\x5b" + + "\x7e\x69\x7b\x68\x33\x36\x97\x36\xb7\x9a\x4a\xd5\xef\x69\x17\x6b" + + "\x9d\x37\x77\xda\x85\x68\xf3\x68\x15\x69\xaa\x69\xeb\x35\x48\x54" + + "\x65\x55\x51\x2d\x76\x54\x75\x94\xa0\x95\xb4\xc9\x27\xca\x57\x64" + + "\x2f\x9a\xf7\xd7\x5e\x35\x8f\x36\x97\xb5\xe0\xd5\xee\xff\xfb\xba" + + "\x5a\xdd\x6c\xdb\xec\x6b\x68\x6e\x84\x63\xde\x32\x58\xdd\x7f\x9e" + + "\x08\x4c\xf4\x6d\xf7\x8d\x7e\x13\x80\x41\x0a\x5b\x46\xae\x20\xd2" + + "\x26\xf2\x73\x6e\x74\xc2\xe4\x55\xe7\xdd\xc8\x98\xfa\x7b\x2e\xe4" + + "\x3b\x78\x9b\x72\xc4\x62\xae\x39\x8d\x3a\xfd\x8f\x7d\xb7\x19\x63" + + "\x9c\x6e\xc0\x32\x4b\x25\x13\xf4\xd3\x1c\xc7\xbc\xcd\x39\xe8\x26" + + "\x3c\x2a\x42\x26\x70\x9c\x39\x55\x90\x39\x93\x60\x68\x7d\x04\xb8" + + "\x27\xed\x72\xe7\xa8\xb7\xed\x35\x77\x97\x95\xd9\x90\xd9\x77\x56" + + "\xba\x5a\x73\xce\x75\x6e\x79\x1e\xb6\x96\x33\xc1\x91\x43\x97\xad" + + "\xc5\xee\x3c\xd3\x3f\x22\xba\xa5\xb4\x3f\x2b\x84\x90\x93\xcc\x6a" + + "\xbd\x59\xe9\xa7\x20\xaa\xf0\xdb\xbc\x09\x4c\xc1\x62\xcc\x12\xaf" + + "\x51\xb5\x54\xdb\x52\xe4\x5e\xfe\x60\x78\x81\x23\x78\x2f\xe8\x94" + + "\xb3\x2b\xb8\xcb\xcf\xbd\xc1\xb6\xa1\x65\x30\xc1\xdd\xd3\x32\xd2" + + "\x32\xae\xd0\x6f\x9f\x6d\x10\xb2\x29\xea\x4f\xf1\x61\xf2\xe1\xf5" + + "\x31\x0b\x28\x09\xa0\x82\x32\x87\x8e\xf8\x2d\xfc\x5b\xe7\xd3\xcb" + + "\xa3\x97\x0f\x59\x77\x8c\x10\xef\xaa\xe9\x7d\xfd\xf2\x99\xf0\x9f" + + "\xdc\x36\x61\x2c\xe1\xd8\xc0\xaa\xc0\xfd\xde\x2a\x32\x8b\x91\xd3" + + "\xb8\x2a\xb2\x07\x70\xda\x54\xfc\x94\x25\x72\x88\xa2\xdf\xe1\xe2" + + "\x0c\x8c\xe1\x81\x61\x54\xbf\x1f\x22\x4d\x48\x58\x88\x46\xa8\x50" + + "\x8c\x43\xb8\x26\xfe\xec\xb7\x54\x4e\xf7\xd9\x35\x51\xe7\x4d\xb1" + + "\x67\xb1\x14\xb1\x8d\x4c\xf3\x6f\x0a\x3c\x35\x4c\xd2\x9c\xf4\x82" + + "\xe5\xcc\x56\xfc\x2b\xdc\xed\x1c\xed\x42\x43\xcc\x66\x5a\x13\x0b" + + "\x9a\x59\x9a\x52\x36\x32\x84\x3c\xcc\xf4\xd5\x5c\x76\x36\x5d\x9b" + + "\xaf\x67\x37\x61\x3b\x59\x3b\x08\xbb\xde\x38\x69\x2c\x23\x7a\x0e" + + "\x9e\x79\x53\x7d\xb7\x21\x25\xd8\x65\xcc\xe8\x8c\xbc\x29\x3f\x64" + + "\xc3\xe5\xbf\x27\x5d\x0a\xa3\xd2\x97\x62\xff\xa1\xba\xc6\x70\xa7" + + "\x8a\x81\xf7\xec\x23\xea\x33\x27\x67\x99\xe5\x9a\x0d\xff\x27\xaa" + + "\x4b\x4f\x2c\x7f\x2d\x49\x2f\xe7\x95\x70\x1a\xdf\xc9\xe9\x49\x39" + + "\xc8\x5e\xc4\xb8\xdf\x16\x4c\x53\x16\xd7\x29\x8b\x24\xbb\x9e\xc9" + + "\x9b\x8d\x9d\xe1\x46\x6a\xcb\x26\xc0\x0b\xfc\x16\x94\x10\x1f\xe7" + + "\x28\x52\x4e\x26\x97\xd8\x6c\x7e\x6c\x7e\x60\xe6\x36\x5e\x69\x1a" + + "\x52\xff\x3e\x68\x17\xd5\xc2\x42\x9c\x0c\x33\x6a\xdd\x10\x57\xf9" + + "\x45\x8b\x09\xbe\x59\xa4\xf8\x50\x55\x9b\x76\x89\x1e\xa6\xde\xdc" + + "\x54\xc3\xac\x7b\x32\x62\x8c\x27\xa5\x22\x9b\x61\xf2\x6c\xa2\x61" + + "\x6c\x35\x8e\xfb\xef\x45\xce\xc7\x19\x7a\x1f\xfa\x76\x42\x73\x51" + + "\xa8\x8c\x06\x13\x91\xe6\x01\xb0\xd2\x8a\xa7\x9e\xdf\xd1\x73\x15" + + "\x7e\x17\xeb\x0b\x43\xa8\xfa\xd7\x4a\xfe\xd6\xc8\x66\x7a\xdb\x05" + + "\x85\x8a\x33\x8d\xd4\x6a\x3f\x20\xb5\x97\x9c\xf5\x6f\xfd\xde\xb1" + + "\xc0\xde\x26\x28\xeb\x13\x04\x35\xf8\xfd\x10\xdf\x93\xb5\xa5\x33" + + "\x3d\x05\x4d\x16\xad\xcd\x26\xc3\xe7\xd0\x2c\x02\x41\xac\x88\xc8" + + "\x6a\xcc\x55\x9c\x54\x2c\xff\x7f\xb4\x6b\x1e\xfb\x04\x61\xf6\x63" + + "\xdd\xe9\x6b\x49\x8d\x65\x15\x03\x6d\x26\xd3\x66\x73\x66\x47\x07" + + "\x1b\x4c\xd9\x3a\xc8\x29\xe5\xa9\x07\xa9\xc8\x76\x48\xcb\x2b\x8d" + + "\x48\x9b\xea\x8d\x67\x77\xf7\xe2\xab\x2c\xa9\x93\xb4\xab\x1d\x41" + + "\xcd\xa2\x6d\x05\xdd\xfc\xad\x9d\xa6\x25\xa3\x53\x93\x7c\x42\xaa" + + "\x85\xaa\xb7\xaa\xcf\xaa\x5c\xe7\x73\xeb\x4d\x86\x4b\x8f\x76\xb3" + + "\xe7\x04\xde\x9d\xf7\x1c\xa7\x0b\xab\x3a\x9d\x0f\x6f\x82\xa3\x5d" + + "\xab\x6a\x50\x6a\x72\x0b\x66\x0b\xa6\x2f\x52\x2f\x0d\x0f\x42\xd5" + + "\x11\xe5\x47\x8f\x29\x2f\xb6\x7a\x55\xf2\xb6\x8e\x67\xc3\xd0\x73" + + "\xd0\x6f\x30\x14\xc8\xcd\x4b\x6d\xf3\x0b\x44\x73\x02\xaa\xca\xb1" + + "\x8b\x0c\x27\x3f\x91\x87\x49\xba\xde\xb3\x2e\xdc\x99\x90\xb8\x90" + + "\xd4\xa2\xff\x0e\xed\x50\x38\x88\x79\xee\x1d\x3d\xc0\x6d\x3e\x10" + + "\x23\xf1\xae\xb8\xfd\xf1\xae\xdc\x86\x01\x91\xb8\x11\x99\x13\xb1" + + "\x58\x89\x64\x5c\x67\x48\x64\xaa\x64\xbc\xe8\x18\xe9\xb4\x2b\x4c" + + "\x08\xce\x2e\x1e\x6f\x78\xbd\xfe\x5c\x25\x41\xf9\xcb\xc1\xaf\xe5" + + "\x29\x7d\xa3\x3a\x6a\xb9\x72\x14\x08\x4a\x34\xbf\x9e\xfe\x93\x95" + + "\x9c\xe5\xeb\xc7\xfc\xb4\xf6\x6b\x35\xbf\xd9\x1b\x6c\x3c\xb5\x69" + + "\x85\x62\x7f\xd1\x8e\xea\xb2\x3d\x27\xfe\x11\x05\xe6\xeb\xe4\x9b" + + "\xd2\xbd\x6e\x75\xb9\xae\x5f\xdd\xc7\xd0\x68\xd5\xc8\xa8\xe1\x35" + + "\xb0\x3d\x76\x39\xe6\x7f\x4f\x04\x29\x0c\x77\xfc\xed\xc8\xa8\x6e" + + "\xd3\xb5\x6e\x3b\xf7\x02\x36\x79\x9b\x3d\x1c\x11\x64\x73\x4f\xad" + + "\xdf\x02\x3d\xaf\xb0\x3f\x16\x17\xb2\x79\x04\xd6\x81\x37\x5e\xf6" + + "\x0f\x83\x0f\x77\x9b\xd8\xeb\x7e\x4e\x34\x6f\x53\xe3\x1d\xcb\xee" + + "\x34\xe7\x55\x9d\x0a\x9d\x06\xb7\x3f\xce\x84\x97\x45\xd7\xc3\x16" + + "\x84\x74\xde\xd7\xfe\x5d\xac\xf9\x5b\xf5\xb8\xe7\xde\xcc\xae\xfd" + + "\x7c\xf8\x1a\x12\x34\xfd\x1e\x7f\x17\x8b\x22\xfa\x25\x36\x3d\x76" + + "\x3c\x14\x29\xf4\x24\xfb\xbb\x20\x0f\xcf\x8e\x90\xd3\xdb\xd1\x6b" + + "\x89\xe8\xb4\xc9\x34\x75\x36\xce\x7b\xfe\xfb\x78\x35\x0e\x36\x8b" + + "\xdf\xe3\xc2\xc9\xd2\xd4\x6c\xda\x6c\x5e\xea\x75\x6a\x62\xb7\xad" + + "\x4f\xd1\xdb\x71\xcf\x09\x5e\xcd\xea\x50\xdf\x10\x71\xd7\xa8\x07" + + "\xfb\x37\xfd\x5b\xf5\x97\xb3\x5e\xf7\x35\x77\xa2\x8b\x82\x4d\x14" + + "\x85\x3d\x1f\x52\x7f\xda\x0f\x8b\xa7\xba\x3b\x95\x99\x92\x97\x44" + + "\xbc\xc4\xc4\x8f\xcf\xd0\x69\x37\xaf\xc6\xec\x4c\xab\x10\x87\xf9" + + "\xd5\x48\xe6\xa7\x18\xe0\xe3\xbf\x57\x2e\xb0\x24\x11\xb2\x07\xce" + + "\x41\x52\xcb\x19\x00\xd0\x95\xf8\x6f\x83\xc2\x4c\x12\x15\x86\xec" + + "\x84\x71\x91\x92\x13\x87\x59\x81\xc1\x81\xe2\x64\x40\x99\x71\x27" + + "\x06\x00\x48\x01\x52\x62\xc2\xaa\x1e\x2b\xa7\x59\xee\x1e\x98\x6a" + + "\x0f\xc3\x42\xee\x66\xcd\x85\x34\x71\xc1\x33\x70\xf1\xc8\x64\x92" + + "\x68\x8a\x62\x62\x4a\x74\x91\xb0\x81\x91\xd1\x1a\x83\x22\x68\x3e" + + "\xb6\xe1\xe1\xb0\xf4\x91\xe1\x91\xb4\xf4\xe4\xcf\x2a\x8a\xdb\x45" + + "\xb0\x9a\x47\xf0\xb0\x92\xf1\xd4\x92\x70\x40\x4a\x4a\x32\x5b\xee" + + "\x0e\x76\x67\x7f\x7f\x9b\xfa\xaf\x96\x50\xe8\xb0\x99\x4f\x82\x28" + + "\x28\x1d\x99\x74\xee\x53\x7f\x2f\x04\xc0\x6b\x4e\xc8\x9a\xb0\xdf" + + "\x4b\xad\x5e\x2d\x3f\xd4\x2b\x52\x30\x23\xd1\x0d\x88\xfb\x30\x30" + + "\xa9\x9a\x53\x50\x5d\x29\x18\x23\xbe\x08\x06\x53\x05\x8e\x13\x70" + + "\xe9\x08\x74\x6e\x54\x85\xf8\x45\x44\x63\x91\x9f\xd0\xd3\x99\xea" + + "\xb1\xe2\xc3\xe5\x49\xc8\x1c\x95\x01\x38\x00\x20\xc7\x50\x86\xb7" + + "\xe8\x19\x33\xf8\x09\x24\x91\xe0\x1a\xee\xc9\x24\xa6\x2f\xfe\x11" + + "\x91\xed\xf1\xb7\xb7\x39\x51\x4d\xeb\x91\xca\x48\xa0\x35\x6b\xe0" + + "\x8f\xa8\x43\xec\xa9\x50\x6c\x26\xe4\xa8\x85\x8a\x13\x4e\xd4\xbe" + + "\x56\x10\x87\x33\x6f\xa0\x0d\x3c\xf5\x02\xc3\xf0\x8e\xcb\xbf\xba" + + "\x55\x9b\xe0\xcf\x7c\x11\x91\xb0\x14\x75\x76\xfc\x48\xd6\x4b\xa4" + + "\x75\xfa\x72\x7d\x22\x41\x8f\xd6\xc3\x82\x1c\x6a\x83\x86\x2d\x86" + + "\x92\x89\x74\xec\x7e\x66\x4b\xd9\x41\x6c\x4b\xc9\x34\x6b\xc4\x28" + + "\x99\x2d\x17\xd0\x36\x32\x21\x33\x1b\x2a\x98\x2f\x12\xfd\x92\x9b" + + "\xea\xba\x67\xcd\xab\x04\x61\x9e\xfc\xfe\xc3\x9f\xb8\x03\x5f\xa0" + + "\x37\x6b\x56\x62\x45\xbe\x27\x60\x0d\x0a\x52\x50\xee\xa5\x50\xa5" + + "\xa7\x7f\x80\xa1\x0e\x41\xc3\xe3\x25\x39\x2b\x95\xa9\xb6\x68\xc0" + + "\x4a\x32\x6a\x11\xa5\x3f\x62\x74\xeb\x2c\x52\xe9\xab\xc4\xb9\xba" + + "\x26\x69\x95\xb8\x3a\xfa\x2e\x6d\x79\x54\xb1\xb7\xc7\xe1\x0c\xd3" + + "\x41\x1c\x54\xc5\x49\x69\xc5\x11\x49\xe8\x64\x62\x64\xd6\xf4\xf2" + + "\x1e\x6f\xca\xd2\x85\x49\x5d\x06\xc1\x3f\x60\x85\xab\xf9\x62\x85" + + "\x29\x04\x7f\x81\xb3\x83\x08\x05\x52\xe2\xfd\x6d\x66\x74\x51\x29" + + "\x54\x39\xed\x2a\x0c\x4f\x27\x04\xcd\x4d\x31\x0c\x27\x7b\xd3\xe8" + + "\x4c\xed\xf6\x5f\x73\xf6\xba\x5d\x6a\x67\xdc\x5f\x90\x33\x99\x9e" + + "\xa9\xd4\x6c\x6f\x9b\x56\x81\x6b\x01\x8c\x27\x0b\x3a\x62\x28\x8c" + + "\x16\xda\x9f\xda\xbb\x1a\x02\x08\xf7\x39\x98\xe8\xa5\x96\x40\xc4" + + "\x71\x73\xae\x2c\xb8\xb4\x91\xa4\xc5\xba\x9f\xa7\xf9\x57\xa2\x32" + + "\xab\x6d\x23\xc9\x3f\x13\xd7\xf8\xf5\x71\x16\x26\x3d\x05\x63\xa7" + + "\x15\x4b\xa9\xc7\x44\xe9\x6a\x10\x67\xb6\xd0\x34\x58\xec\x7f\x15" + + "\x09\x24\x24\x94\x96\x12\x45\x0b\xe4\x85\x5b\x68\x05\x8a\x5a\xd3" + + "\x8b\xfd\xcc\xa3\x87\x37\x13\xbc\x0a\x41\xfd\xc7\x81\x55\x63\x58" + + "\x7b\xd9\x72\xe7\xf3\x7a\xe7\xf3\xbe\x52\x87\x43\x27\x0c\x2a\xdf" + + "\xfc\x22\xcc\xee\xfa\x56\xd3\xc9\xfe\x07\x1b\xf0\x95\x4b\x67\xf0" + + "\x40\xdf\x7e\x8c\x4d\x8d\x93\x48\x09\x2d\x6f\x20\x29\x76\x6f\xf1" + + "\xbe\xca\x72\xb9\x90\x19\xb9\x23\x7a\xe6\x17\x4d\x77\xf5\xf2\x9f" + + "\x7e\x43\xf0\x99\x14\x68\xd9\x9f\xb6\xc9\x41\x32\x9e\x5e\x4c\x55" + + "\x8d\x2e\xb6\x9d\x69\xb2\x36\xdc\x6f\xb9\xff\xc8\x26\x70\x15\xc8" + + "\x9c\xb3\x86\x26\x2f\x0e\xd7\x06\x62\x8d\x91\x83\x22\x89\x3d\xe5" + + "\x8a\x26\xce\xaa\xde\xef\x3d\xe7\x46\xc3\x51\x41\xee\xe7\x60\x9d" + + "\xc4\x60\x72\x35\xdc\xdb\x6b\x7f\xba\x5d\x25\xd1\x6a\x70\xe4\x23" + + "\x68\x3e\x65\x74\xe2\x2b\x57\x8f\xdc\x2f\x82\x02\x5a\xea\x0a\x41" + + "\x65\x16\x67\xef\x21\xa5\x15\x8f\xa2\xc8\x2c\xb2\x3a\xbd\x6a\xec" + + "\xb5\x5f\x2b\x26\xa5\x21\x8b\xa4\x75\xe3\x4b\x94\x9f\x6c\x2e\x59" + + "\x2d\x58\x61\x33\x68\x6c\x34\xc9\x49\x6b\x9c\x45\xd6\x29\xf9\x08" + + "\xed\xbb\x8a\xbd\x73\xca\x2f\xd9\x05\x24\x35\xee\x22\x12\xb1\xa7" + + "\xab\x11\xbc\x8b\x48\x53\xc2\x1a\x36\x2c\x1c\xa2\x13\x6f\xd8\x6d" + + "\xd4\x19\x3b\x53\x16\x6a\xc3\x1b\xa3\xbd\x5d\x8b\x68\x6a\x5a\xd7" + + "\xd7\xa7\x14\x58\x72\xa2\xf5\xad\xfe\x58\x52\x59\x94\x39\x5a\x04" + + "\xe0\x61\xe3\x33\x0a\xfa\xd9\xa2\x36\xed\x30\x14\x94\x1d\x5b\xc2" + + "\x29\xa7\xfd\x21\xae\x91\x1f\x86\x09\x8d\xcb\xc8\x7e\xaa\xe9\x65" + + "\x09\x81\xd4\x8f\x5d\xeb\xf6\xd7\x0c\x2e\xdb\xb1\x72\xb5\x8a\x7f" + + "\xed\xae\x0c\xab\xcd\xb6\x7c\xbd\xae\x2c\x7f\x94\x14\x1e\xf2\x54" + + "\x3a\xb0\x4b\x80\x06\x12\x66\x5c\x4b\x77\xcf\xf6\xef\x35\xc2\x18" + + "\x67\xf7\x56\x6f\x02\xd9\xb6\x1b\x22\x8b\x8f\x28\x97\x5c\x78\x97" + + "\x5b\x67\x3b\xdb\xd9\x2d\xbf\xd2\x0e\x78\x46\x87\xb2\xc0\x86\x5b" + + "\xe3\x18\x8f\xbe\x6f\x06\x4f\xdd\x9f\x4b\x7e\xd8\x31\xfe\x90\x94" + + "\x24\x70\x72\x4a\xd1\x82\x37\x41\x7b\x73\xaa\xdd\x41\x72\xe8\xe0" + + "\x2e\xb0\x54\x6b\x3f\x7e\xfe\x5d\x23\xbc\x02\x95\x7d\xee\x4d\x03" + + "\x6c\xd7\x0d\xcb\xb8\xf0\xfa\xc8\x9c\xa0\x19\xee\xaa\x75\x9e\x26" + + "\xcb\xf6\x37\x46\x5f\x75\x27\xf9\xfe\xbb\xb1\xfb\xf5\x1a\x61\x49" + + "\xdf\x9b\x29\x32\x3f\xb7\xfc\x58\x0a\x9c\x51\x7c\xf4\xf1\x30\xce" + + "\xd2\xcd\xea\xf7\xea\xd1\xf5\xf1\xfe\x82\x11\xe3\x59\x3b\x3c\x89" + + "\x78\xea\xcc\x57\xaf\xf8\x53\x65\x86\x18\xc4\x01\x89\x79\x0e\xae" + + "\x90\x29\x76\x00\x26\xb9\x51\x40\xd2\xf0\x30\x83\xf4\x1e\x4e\x71" + + "\xfa\xec\x0b\x98\x45\x02\x43\x45\x70\xe7\xcf\x99\x10\x46\x61\x12" + + "\xe1\xf8\xc8\x5e\xa8\x77\xf6\x64\xec\x26\x96\xbb\x89\x49\xec\xc8" + + "\x9f\x3f\x35\x37\x8c\x48\xf5\xe1\xfa\xe5\x68\x38\x0f\xfc\x61\x28" + + "\xdd\xa1\xc5\x0b\x9d\x4a\x4b\x89\xdd\xd0\xa0\xed\xda\xd5\x47\x9f" + + "\xd3\x86\xf9\x4a\xf5\xab\x24\xe6\x9b\x23\xc9\x5f\x2d\xab\x87\x7c" + + "\x26\x84\xdd\xd5\x1b\x2e\x7d\x9b\xd9\xbe\xf7\x5a\x12\x31\x84\x11" + + "\xc5\x96\x60\x59\xb6\xd8\xe1\xfe\x8b\xee\x8f\xa9\x99\x3c\x51\xad" + + "\x76\x97\xb2\x8f\x07\xcb\xdd\x30\x65\x96\x3c\xf8\x12\xd6\xf7\x3d" + + "\xcf\xce\x5b\x8f\x57\xac\xb6\x87\x2d\xa5\x8f\xe8\x0d\xfc\xe7\x52" + + "\x9b\x4d\xde\x32\x4d\x97\x09\x0e\x39\x3f\xbd\x86\x74\xa8\xfe\x4d" + + "\x98\x60\x3c\xc2\x35\x66\x41\x6e\xf7\x3c\x4e\x21\x73\x34\xe0\x2d" + + "\x39\xbd\xd5\x0f\x4d\xaf\xe7\x17\x9d\x9e\x52\xe5\x36\xa3\x46\x9f" + + "\xdd\xb6\x30\xf1\x37\x86\xf2\x2c\x0c\x3c\x84\x91\xd5\xf7\x6c\xf3" + + "\xef\xea\x23\x72\xfe\xa7\x75\xc3\xb8\x02\x5e\x1c\x86\x86\x86\x38" + + "\x26\xe2\x08\x53\xe3\xe3\xe3\x05\xda\x4d\x56\x84\x02\x5e\x98\x01" + + "\x99\xab\x5c\xbb\xeb\xa9\xd9\xa4\xc3\xfb\x82\xdb\x2b\x5d\xbd\x42" + + "\x21\xdd\x6e\xe7\x45\xe6\xdd\x4a\x00\x0e\x99\xca\xc9\x79\xf7\xb0" + + "\xb8\xac\x5d\xde\x32\xa2\x2f\xfd\x27\x91\xe4\xac\x72\x11\x49\x82" + + "\xd3\x7b\x36\xb6\xda\xed\x56\xc6\x82\xcb\x58\x7a\xe1\xb5\x33\xe1" + + "\xe1\xcc\x9e\x78\x78\x1f\xe6\x74\xc9\xe2\xb7\x32\xb1\x82\xbe\xfc" + + "\x3e\xcf\x65\x44\x00\xb5\xe8\xd7\x19\x56\xb6\x67\x64\x82\x99\xcc" + + "\x8a\x4c\xce\x42\xa7\x90\xeb\xc3\x50\xc7\x29\x31\x59\xbe\x6c\xb9" + + "\xd4\x0f\x44\xaa\xa5\xfd\x9b\x5f\x34\xf2\x7f\x25\x2f\x36\x80\x70" + + "\xc8\xc8\x82\xf6\x81\x39\xe7\xfa\xca\xd2\xcd\x36\xcb\x22\x1d\x6e" + + "\x67\x92\x72\x72\x54\x1c\xdf\xc3\xa9\x0c\xba\x43\xf2\xc3\x55\x36" + + "\xc0\x8a\xd2\xbc\x44\x24\xb8\x81\x49\x09\xc7\x6a\xf4\x97\x6f\x7b" + + "\x4a\x90\x08\x3d\x4d\xd7\xc4\x25\x9e\x1c\xec\x5a\x6e\x36\xb8\x0e" + + "\x13\x23\x97\x14\xaa\x30\xb3\xfd\x25\x26\xaa\x29\x92\x79\xa9\xb8" + + "\xf4\xb8\x03\x35\x03\xb7\xda\x42\x3b\x9f\xe8\x6e\x70\xaa\x12\x32" + + "\xb0\x5f\x72\xab\x49\xb2\x8c\x2c\xd3\xc4\x6f\x95\x83\xe3\x5a\xc8" + + "\xc0\x1b\xde\x1d\xeb\x9a\x7e\x8a\x39\xab\x45\xe5\xe5\x3b\x36\x05" + + "\xd3\x25\x2a\xf7\x58\x27\xef\x3b\x11\x38\x9b\xc2\xd2\x16\x73\x85" + + "\x70\x37\xe0\x69\x1a\x66\xe6\x6a\x31\x1d\x83\x1f\xf5\x53\x59\x7c" + + "\xec\xf6\x1b\xfc\x56\xaa\x65\x3d\x28\x2a\xfa\xfd\x59\x60\x9e\xfd" + + "\xab\x30\xe5\xb9\xb6\xfd\x77\x30\x54\xa9\x5a\x01\x9b\x90\xdf\x3b" + + "\x7e\xbb\xe1\xe5\xbf\x19\x77\xad\xcb\xd5\x76\x91\x5d\x4c\xf9\x35" + + "\xbb\xcd\x09\xad\xb3\x47\xfa\xd7\xcd\x56\x70\xad\xe5\x31\xde\xfe" + + "\xce\xd5\x71\x63\x6b\x15\xc9\xaf\x4c\xa4\x8d\xea\x14\x3b\x2d\xb1" + + "\x6e\x13\x87\xef\x3b\xa3\xe3\xab\xd1\xe5\xea\x67\x16\xb8\x2f\x5d" + + "\xb6\x0b\x85\xfa\x97\x5b\x61\xed\xe3\xed\x70\x6a\x21\x76\xb8\xe0" + + "\x33\xe5\x8b\x35\xef\xf0\xf3\xd8\xe8\xcb\xc1\x09\x50\xf1\x6c\x83" + + "\xf9\xb4\x51\xd3\x96\x79\x87\xbf\xbe\x3e\x71\x32\x83\x06\x35\x13" + + "\x53\xf4\xd4\x99\x0e\xdb\x71\x37\x4a\xb6\xd3\x00\x31\x1d\x8f\xd3" + + "\xe9\x77\xee\xcd\x0f\x7e\x65\x69\x91\xa0\x2c\x48\x68\x3c\x5c\x73" + + "\x1e\xc8\x83\x7f\x48\x8f\x2e\x56\xe9\x40\x62\x4b\x2d\x28\x08\x49" + + "\x74\xcb\x95\x52\x2e\x95\x21\x7e\x5f\x09\x96\x9f\xd2\xa3\x0b\xeb" + + "\x30\xdb\x4e\x51\x05\xca\xed\xf4\x9a\x2b\x6e\x6e\xb4\x29\xb0\xd2" + + "\x5f\x91\x08\xfc\x0a\xe1\xb7\x1f\x67\x59\xef\x27\xbd\x2f\xf6\xfd" + + "\x84\x13\x37\x3c\x22\xb5\xaf\x92\x43\x1c\x89\xd5\x3b\x01\x32\x1a" + + "\xb6\xd7\x89\x3e\x5b\x3d\xbf\x6f\x4f\x97\x41\x0a\x19\x51\x46\x58" + + "\xc1\x3d\xdb\x0e\xb5\xee\x35\xf6\x88\x36\xe7\x81\xc9\xaf\xad\x2e" + + "\x93\x65\x11\xcc\x9f\x1d\xbf\x29\xf1\xcf\x00\xa1\x01\xcb\x6d\x76" + + "\x42\xa3\x50\xa7\x6e\x7c\x3a\x2d\x76\x45\xfa\x40\xaf\x35\xc7\x61" + + "\x22\xda\x48\x05\xac\x8f\x7f\xad\x59\x45\x15\xef\xfc\x77\x9c\x08" + + "\x4d\x05\xc6\x1d\xd8\x92\x47\x91\xae\xbc\x7c\x9a\x27\xc6\x4c\x51" + + "\x7e\x8e\x86\x0d\x83\x5b\x77\xc5\xa4\x62\x16\x24\x74\x7a\x89\xe4" + + "\x14\x81\x7b\x7f\x63\x7f\x86\x5f\xc6\xfd\xa2\x04\xdb\xbb\x16\x25" + + "\xbd\x76\xf9\x11\xc4\x76\xb1\x4e\x81\x77\x16\xab\xcf\x72\x3e\xba" + + "\x34\x29\x3f\xb8\xdd\x2e\x36\xe4\x26\x33\x38\xa1\x1e\xef\x56\x5b" + + "\xe0\x90\xe0\x78\xf7\x1d\x3d\x89\x59\xdc\xf9\x79\x03\xdf\xc2\x54" + + "\xe4\x9f\x1e\x20\xd5\x95\xad\x63\xd7\xb6\x1f\xbc\x69\xa5\x5e\x31" + + "\xf2\x3a\x5a\x20\x4f\x80\x0d\x98\xd0\x79\xe4\x63\xf9\x93\x8e\xf8" + + "\x79\x58\xf8\x1c\x4d\xe6\x85\x31\xca\x6f\x96\xb4\xa8\x21\x03\x35" + + "\x86\x83\xae\x81\x46\x0f\x54\x99\xad\x2b\xe8\xfb\x17\x16\x01\xee" + + "\x93\xa3\xa1\xfa\xd2\x51\x2d\xfb\xaf\x47\x8f\x8f\x87\xa7\x1b\xb0" + + "\x81\x72\x02\xf4\xc0\xf0\xf0\xcd\x2d\x48\x01\x7a\xa6\xe9\xe2\x12" + + "\xa4\xfd\xab\x43\xce\x78\xf1\x67\x3f\x1d\x0e\x96\xdf\x58\xf7\x47" + + "\xe6\x1e\xa0\xe3\x9e\x78\xc7\x41\xfa\x19\xef\x22\xe7\x64\x8a\xc1" + + "\x81\x30\x90\x71\xb2\x21\x5c\x7f\xa7\x76\xd5\xd7\xbb\x5b\x8d\xf3" + + "\x59\x5e\x6f\xb3\x9b\x86\x7c\x02\xfb\x69\x70\x75\xfa\x7a\xbb\xf4" + + "\xf7\x8e\x2c\xc5\x08\xd8\x68\x92\xc6\xf7\xa6\xc0\x9f\xe9\xa5\x54" + + "\xa9\x40\x52\x52\x92\xd6\x31\x01\x07\x55\x63\xc9\xd5\x6c\x32\x63" + + "\xf0\x7a\x7f\x7c\xb9\xd1\x12\x63\x74\x74\xf4\xdd\x38\xdb\xe7\x5a" + + "\x0a\xa0\x0c\x8a\x49\x4c\xcf\xf2\xd6\xe6\xd6\x51\xc3\xf2\x7b\x7f" + + "\x75\x3d\xa6\x56\x65\xba\x7e\xd3\x3b\x67\xaf\x09\x73\xdc\x9f\xe6" + + "\x92\x3c\xe2\x64\x62\x6a\x84\x9c\xf0\xd2\x63\x4e\x04\x8b\xca\xa5" + + "\xc4\xb0\xd0\x2f\x4b\xb5\x9b\x9a\xe1\xe3\x21\xc6\x42\xb3\x63\x88" + + "\x47\x21\x3d\x2b\xc6\x26\x32\xff\xf2\x9e\x06\x04\xe5\xf0\x9d\x90" + + "\xff\x69\x2f\xd1\xfc\x8f\x41\x14\x3b\x71\x0e\xab\xe1\x27\xae\xcc" + + "\xe8\x55\xfb\xee\xac\x8a\x9a\xae\x0c\x89\x68\x7c\x34\x12\x12\x05" + + "\x59\x59\xc4\xf1\x54\x96\x12\xe5\x22\xe9\xf7\x5d\xbe\x37\xf3\x43" + + "\xbc\x9a\x22\xda\xc0\x23\x8c\x50\xc7\x33\xb5\x08\x73\xa0\x7b\xc8" + + "\x21\x6d\xb9\x46\x8c\x80\x4c\x5a\xe1\xbe\xc1\xaa\xd9\x51\xad\xf8" + + "\x6d\x82\x08\x12\x40\x44\x54\x27\x29\xec\x54\xb6\x63\x12\xe3\xe3" + + "\xe9\x9f\xf9\xe3\xce\x99\xb9\xe3\xbe\x67\xe4\xe4\x0a\x3c\xd9\xa3" + + "\xe1\x95\xd5\x07\xfe\x95\x80\xf7\x3d\x2a\x26\xa6\xaa\xd1\x62\x67" + + "\x2e\xf4\x72\x87\x3b\x8b\xb0\xb0\x30\xff\xd5\x00\x46\x09\x3f\x33" + + "\x27\x67\x32\xe4\x66\x85\x40\xb2\xc5\xf1\xa4\xaa\xe5\x53\x5c\xd6" + + "\x19\xaf\xe1\xf6\x54\x59\xf6\x60\xc5\x49\x27\xf7\x2b\x9c\x99\x1c" + + "\xa1\xdf\x90\xdb\x2d\xb0\x03\x24\xd5\x71\x25\xbf\xae\xfe\xdb\xf2" + + "\x80\x52\xe7\xd9\x02\x0d\x98\xb6\x7e\xcb\x0b\xb0\xd0\xfe\x22\x1e" + + "\x9e\x96\xb7\xbc\x5c\xa1\x7c\xd1\x5d\x20\xe0\x3f\x91\x45\x8a\xbf" + + "\x10\x31\xcc\x3a\x61\x44\x95\x97\x94\x90\x67\x2c\x04\x9d\xab\x51" + + "\xa3\xaf\x5a\xaa\x98\x27\xa6\xe7\x46\xf6\x99\x41\xbd\x32\xd7\xeb" + + "\xee\xf8\xbd\xed\x0c\x64\xd8\x13\x00\xe5\xf3\x76\xb7\x84\x48\xcc" + + "\xff\xef\x63\xb5\xb6\xa5\x24\xf7\x6f\x9e\xef\x17\x15\x2b\xce\xf2" + + "\xa1\x3a\x9f\x21\x3e\xd5\x3f\x4c\xfe\x73\xdf\xd6\xed\xba\x42\xb3" + + "\xb3\x56\xb3\x9b\xaa\x1a\xfc\xf5\x91\xbf\x9b\xeb\x8e\x39\x46\x8e" + + "\xc3\xa5\x24\x45\x06\x0c\x0c\x44\xbf\x3e\x99\x17\xf8\xa4\x7e\xcb" + + "\x4b\x58\xea\xf0\xd6\x70\x88\x9b\xac\x3b\xac\xe0\x76\xfc\x7e\x4a" + + "\x65\xa6\x27\x07\x2a\x5c\xae\x52\x20\xa5\xda\xdb\xdb\x43\x25\x24" + + "\xd4\xcd\xff\xd6\x66\x7f\xd6\x49\xb4\x4b\xd8\x17\x53\x5b\xcb\xba" + + "\xd2\x62\x3f\x5d\xa9\xd5\x00\xd0\x14\xfa\xe8\x27\xdf\x54\x80\xed" + + "\x9d\xe7\x89\xc2\x7d\xdb\x5e\x7a\xdf\x04\xfb\xb0\x05\x5a\x25\x1b" + + "\x4d\x7b\xd9\x35\x30\x2b\x04\x94\xce\xac\xb2\x22\xfb\x5a\x5a\x96" + + "\x9a\xce\x46\x7d\xf1\x7f\x5f\x0e\x5f\x69\x4a\xb5\x40\x6d\x9f\x3f" + + "\x59\xf7\xfc\x65\xf1\xd1\x41\xd8\x2d\x47\x42\x23\x13\xb9\xf7\x65" + + "\x91\x85\x61\x29\x24\x69\xa6\xd6\xb0\x1f\x9d\xca\xb3\xf7\x6d\x0e" + + "\x04\x9a\x8e\xd8\x03\x68\x69\x69\x4d\xa4\x7d\xdb\x0d\xd6\xb3\x15" + + "\x07\x92\x48\x9e\xb7\x37\x11\x21\x74\x7b\x81\x3e\x5e\x6c\x51\xba" + + "\xc5\x62\xc6\x7d\x87\x4e\x3a\x6f\x96\x19\x11\x41\x03\x1c\xeb\xce" + + "\x8c\x67\x8d\xe1\x7e\xbd\xb5\xcb\xe5\xfb\x0c\xa6\x21\x2f\xcd\xf6" + + "\xf6\x27\x75\xcb\x6b\xb6\xd8\xfe\x02\x12\x6e\xdd\x45\x93\xe2\x57" + + "\xa1\x8b\x76\x2b\x50\x8f\x6e\xf9\x7a\x88\x7f\x88\xf8\x9a\x25\x87" + + "\x5d\xcf\x56\x92\x51\x04\x1f\x7b\x3e\x7c\xc7\x53\x98\xde\x99\x6a" + + "\x2d\x87\x1b\xa6\x0c\x6b\x67\x68\xfa\x9c\xe2\x0d\x19\x9f\x9e\x99" + + "\xfc\x7f\x3a\xde\x11\x29\xd4\x83\x7b\x66\x41\xfa\xe6\xf7\x0a\xdd" + + "\x52\x06\xb0\x58\x95\x3a\xda\x56\x68\x6f\x9d\x79\x96\x55\x00\xc5" + + "\x85\x56\xaf\x5c\x53\x69\x4f\x13\xe9\xcc\x0a\x74\x2d\x01\x6f\xfe" + + "\x78\xc3\xbc\x1d\xa3\x49\xc3\x8f\xba\x2b\x7b\x00\x24\x38\xfe\x4a" + + "\xd9\x2f\x36\x8e\x3c\x3f\xa6\xb3\x05\xd2\x79\x9c\xa9\x63\x05\x9f" + + "\x69\xb1\x18\xd4\x27\x48\x3a\x6a\x41\x65\x2a\x34\x80\x7b\x12\x69" + + "\x01\x96\x65\xb1\xe0\x90\xcb\x7e\x70\xcd\xd3\x34\xd0\xba\x87\x96" + + "\xad\xc1\x59\x81\xbc\x63\xa9\x22\xd1\x8f\x5f\xa7\xd4\xea\x56\xac" + + "\xe6\x21\x60\x7c\x4e\x3d\xea\x10\xe7\xd7\x68\x03\x41\x90\x6d\xcb" + + "\x2a\xd0\x0a\xe1\x30\xaf\x2a\x63\x1e\x1c\x84\xf6\x26\xbe\xfd\x39" + + "\x92\x0a\x89\x8e\xca\x60\x4a\x1b\xf0\xc3\xc3\xe3\x51\x3c\x5d\x19" + + "\x01\x9e\xf7\x65\x25\xb1\x6f\xda\xef\x70\x86\x8c\x43\xae\xc0\xed" + + "\x6c\x85\xff\x66\x82\xbd\x8b\x15\x5c\xa3\x07\x66\xca\x69\x3d\xa4" + + "\x35\xf3\x3f\xd5\xd9\x39\x71\xf2\xf6\xb0\x39\xec\x08\x0e\xf9\x9a" + + "\x73\x79\xea\x7f\xc1\x03\x4b\x4a\x66\x15\x57\x54\x86\x3d\x3b\x8d" + + "\xd7\xdb\x3b\x79\x8d\xc5\xe3\xf2\x77\xbc\xc9\x51\xc0\xa3\x02\xba" + + "\x93\xe3\x52\xff\xdb\x51\x74\xaf\x87\xeb\x5b\x7e\xb5\x18\xc3\x5f" + + "\x37\x29\xa5\xa4\x28\xa7\x8b\x93\xea\x8e\xf3\xca\x61\xd9\x4c\xae" + + "\xc9\x1c\xba\xe6\x5c\x5b\xc0\x42\x85\x06\xcc\xd6\x30\x95\x35\xbc" + + "\x69\xba\x80\xbb\x6a\x05\x66\x7f\x5f\xcd\x46\xa1\xa7\x11\x51\xf3" + + "\x2b\x30\xec\xc4\xeb\x2d\x35\xbc\x36\xee\xd5\x1d\x5a\xc5\x9c\xe5" + + "\xb3\x2a\xcb\x67\xad\xc5\x35\xf2\xe7\x36\x0a\xde\xe7\x6a\x45\x87" + + "\x42\xc9\x74\x01\xd4\xb8\x26\x2b\x55\xec\x1b\x41\xa2\x51\x22\x6e" + + "\x03\x51\xc7\x4a\x07\x3e\x0d\x0f\x84\x49\x0a\xeb\x0c\x37\x25\xfe" + + "\xf0\x93\x9a\xee\xf7\xce\xe9\x1a\xdd\xb6\x7a\x21\x09\x1c\x9c\xef" + + "\x3e\x68\x58\x68\x63\x03\x1a\x39\x26\x73\x87\xbd\x84\x52\x72\xe0" + + "\x3c\xf0\x10\x3a\xfe\x8e\x8e\x10\xab\xff\x95\x15\x26\x95\x47\xb1" + + "\xe1\xab\x25\x5d\x95\xf8\x6d\x7e\x7b\x39\xd2\xb5\xf1\x15\xbc\xc2" + + "\xc5\xfa\xcb\x26\xe6\x08\x03\x8a\x83\x6f\x05\xe5\x37\xfb\x6b\x49" + + "\x7a\x35\xda\x58\x4f\x34\xdd\x83\x06\x2d\x17\x3d\x96\xa7\x6f\x34" + + "\x9b\xc7\x9c\x21\x0f\x9b\xfe\x1f\x7f\xa7\x32\x39\xad\x17\xe1\xaa" + + "\x5d\x25\x6b\x31\x22\x18\x43\x9b\x61\x33\xb8\x2a\x07\x71\x8c\x73" + + "\x4c\xb7\xaf\xef\x1a\x8f\x04\xc3\x35\x36\x73\x22\x0d\x71\x2d\xb4" + + "\x77\x8c\xb1\x5d\x99\xf2\x71\xf0\x98\x97\xcc\x82\x61\xdf\xc7\x59" + + "\x63\x3a\xa1\xe6\xd0\xd0\x4f\x31\x29\x3e\x5e\x48\x31\x24\x90\x49" + + "\x69\x96\x7c\x9d\x9e\x0b\x44\xe0\xf5\xe6\xa0\x9d\x84\x63\xfc\xcb" + + "\x69\xf0\x06\x6b\xff\x36\x97\xaa\xbf\x5d\x0d\x93\x94\x68\x35\x98" + + "\x99\x0b\x2c\xda\xb5\xbc\xb8\x67\x63\x8d\x08\xc3\x0a\xef\xc3\xcf" + + "\x92\x46\x55\x3d\xe4\x19\x9e\xe1\x23\x92\xb8\x90\x40\xf4\x38\xed" + + "\x7e\xdd\xa8\x27\xfb\xf4\xea\x35\xc5\x4e\xe9\x61\x2d\x37\x79\x3f" + + "\x0c\x87\xdc\x25\x2c\x69\x19\x0e\xa0\x67\x67\x41\xb7\xe1\x71\x6e" + + "\x78\xcd\xaf\xff\x71\x2f\x49\xf6\x43\x4d\xad\xde\x79\xed\x35\xab" + + "\x33\xe7\xcf\xee\x32\xe1\xf4\x58\x2f\xab\x5c\x7e\x01\xf9\xbf\xb6" + + "\x07\xd4\x6f\xfb\xe4\xa5\xe8\x38\x72\x0b\xd2\xdf\x06\x06\xc5\xe2" + + "\xc1\x57\xca\x84\xf2\x01\xe5\xf4\xc1\x5f\xf9\x46\xa7\x5c\x13\xb3" + + "\xb3\xe5\x2d\x2c\x0e\xc5\xeb\x8e\x95\xa5\xd5\x6b\xf4\x27\x3b\x3d" + + "\x6f\x3c\x9e\x8f\x4a\x96\xdb\x5d\x4f\xa4\x86\x39\x6d\x57\x92\xf6" + + "\xc7\x53\x8d\x1b\x96\xe5\x8f\x69\x6d\x06\x09\xcd\x0e\x43\x85\x1f" + + "\x32\x57\xff\xf3\xa6\x5d\x8f\x9b\xd3\xe8\x9d\xc9\x4c\x2b\xd9\xc3" + + "\x1d\x4c\x32\x28\x04\xf8\x5e\x4c\xcb\xcb\x32\x75\xab\x89\x3b\xfc" + + "\xd9\x4c\x39\x41\x60\x89\x92\xfe\x29\x6f\x16\x21\xe1\xf1\x42\xaf" + + "\xf7\x68\xda\xf2\x7c\x87\x53\x0f\x6c\x19\xf5\xf7\xd1\xc8\x26\x9d" + + "\x47\xc7\xa3\x0c\x72\x3d\xa1\x62\x9f\xcf\xd0\xc4\xad\x59\xd6\x19" + + "\x9c\x94\xbc\xf4\x7f\x6f\x26\x29\x55\x02\xc4\x19\xb8\x59\x2d\x06" + + "\x60\x10\x89\x78\x93\x34\xf4\x3d\xf7\xf3\xc7\xca\x6b\x87\xeb\x4d" + + "\xc7\xb1\x17\xab\xf5\x64\x5c\x5c\x38\xa4\xa4\xa4\x06\x06\x07\xb3" + + "\xf6\x66\x5f\x5f\x6f\xa6\x5d\x0e\x67\x7e\x2b\xe8\x93\xc8\x36\x0c" + + "\xac\x0c\x6e\x60\xab\xa9\xe1\xf2\xbc\x33\x74\xa5\x77\x7c\x16\x87" + + "\xfb\xe8\xe7\x60\xcf\xd4\xba\x6d\x0a\xf7\x43\x17\x11\xa9\x26\xa3" + + "\x01\x6b\x70\x54\xcd\xdd\xb5\xfb\x39\x40\xfd\x78\xd3\x5e\x78\x32" + + "\x7b\x45\x78\x9f\xa0\x89\xcc\xdf\xba\xf2\x48\x6f\xb1\xc0\xea\x1f" + + "\xf6\x7d\x8f\x5d\x2d\xb0\x54\xad\xd4\x8c\x8f\x8b\x43\xed\xd4\x67" + + "\x71\xeb\x20\xa2\xa3\x6a\xaf\x36\xce\xa0\xe6\xe1\xe1\x69\xd5\x65" + + "\x66\x63\xff\x58\x9e\x77\x34\x07\x37\x23\x1e\x83\xa2\x49\x4b\x76" + + "\x3b\x93\x57\x59\xe5\x32\x5c\x3d\x03\x21\x5a\x96\x98\xa1\xdd\x5e" + + "\x91\xe8\x72\xb2\x18\x9d\xce\x6e\x7e\xbd\xac\x33\x1d\xa7\x5a\x63" + + "\x16\xe1\xc2\x1f\x6a\xe9\x74\x96\x15\x34\xea\xba\x9f\x57\x1e\x9f" + + "\x88\x44\x51\xa0\x00\xf4\x7a\xf9\xd7\xde\x54\x3e\x68\x12\x75\x88" + + "\x4a\xb0\x9a\x55\xf0\xb3\xc4\xb8\x41\xcd\xb7\xef\x64\x9d\x5b\x90" + + "\x91\x32\xfc\x88\x04\x91\x82\x99\x7a\xcc\x60\x91\x96\x65\xba\x7f" + + "\xef\x48\xe8\x6a\x3b\xd5\xf9\x86\x2f\x14\x55\x44\x54\xf4\x68\xa9" + + "\x86\x44\x34\x0c\x99\xe9\x1a\x38\x14\x4b\x32\x0d\x51\xb2\xb1\x78" + + "\x8a\x9f\xac\x72\x89\x0f\xa1\x0f\xbb\xc7\x66\x77\x6f\x20\x45\xf0" + + "\xa7\xff\x6c\xd0\x3e\x1c\xc4\xd0\x0b\xf9\xfd\x85\xf4\x60\xe9\x55" + + "\xea\x98\x40\x47\x84\x12\x50\xc4\xf0\xee\x36\x98\x45\x9f\xa6\xd8" + + "\xc7\x24\x62\x5d\x46\x09\x13\x7f\x73\x4e\xe3\x53\xdf\x76\xa9\x0a" + + "\x0d\x8b\xfd\xaa\x5b\xd4\x10\xf6\xec\x6c\xa3\xf6\xe1\xf3\xe6\xbf" + + "\x33\xb5\xf3\x4b\x0f\x48\x33\xcb\xf7\x12\x37\xf3\x7f\x85\xd7\xd7" + + "\x7b\x60\x38\x5a\x91\xe3\x7b\x9c\x6e\xb0\x7f\xaa\xf2\xea\xf6\x7b" + + "\xf3\x00\x67\xb9\xcb\x3b\x19\x4e\x0f\xee\xdf\x79\xf0\x73\x58\x2f" + + "\xfe\x89\x26\xe0\x46\xdf\x6b\x6c\xb0\x38\xc6\x25\x74\x5b\xe5\xc1" + + "\x68\x3a\xa1\xe8\x97\xda\xf0\x4f\x43\x22\xe0\x8a\x4b\x4b\xd3\xa4" + + "\x37\xfd\x9b\x9c\x2c\x2e\x16\x5c\x83\x3f\x3e\x96\x7e\xb1\xc3\xd8" + + "\x2b\x00\x00\xb0\xfb\x3d\x26\x6b\x6c\xbc\x75\xbf\x3e\xe8\xf0\x89" + + "\x2c\xb5\x98\x6b\xab\x32\x83\xc4\xdc\x01\x9e\xa5\x56\x13\x89\xa1" + + "\xf1\xd7\x6e\xab\x15\xe8\x9e\xee\x8f\x18\xef\xf8\x66\x22\x3d\x72" + + "\x0f\x02\x17\x0a\x3a\x98\x38\x38\xc3\x79\xa2\x61\x91\x4d\x5f\x60" + + "\x90\x09\x73\xa5\x93\xe9\xdf\x2f\xc6\xcd\x3b\x45\x1d\x14\x67\xd7" + + "\x9f\x9a\x32\xdd\x66\xc0\xe9\xdf\x5c\x8e\xe7\x23\xda\x9c\xc1\x29" + + "\xd2\x74\x42\x10\xbd\x95\x4d\x63\xcd\x1b\x58\xbb\x57\xf1\x44\xcd" + + "\xfa\xb8\x21\x87\x41\x2e\x91\x56\x8b\xe6\x37\xc1\xc5\x4b\x0e\x0a" + + "\xb2\x27\xe5\x7e\xc0\x89\x9a\x2d\x11\xee\x91\xaf\xb1\xd4\x16\x11" + + "\x1c\x2d\x58\x24\x90\x18\x5b\x7d\x3b\xef\x90\xeb\xba\x99\xdb\xf5" + + "\x4b\x90\xa1\x6b\x2b\xb7\xf6\x17\x1a\x85\x98\x77\xcd\x53\xca\x54" + + "\x65\xe5\x36\x27\x53\xc4\xfe\x5f\x10\x1d\x05\x34\x76\x19\x93\x12" + + "\xc7\xb6\x58\xc8\x68\x53\x71\xd8\xaa\x22\xa8\xa9\xa5\x71\xdb\x13" + + "\x5b\x5a\x5a\x16\xc9\x4f\xe9\xd7\xa4\xc2\x61\x50\x03\x14\x85\xf2" + + "\xac\x22\x58\x16\x98\x34\x93\xb9\xfd\xaf\x69\x6d\x32\xe8\x11\x29" + + "\x0a\x12\x8a\xd5\xda\x97\xc9\x3d\xf1\x28\xbd\x24\x47\xce\x9e\xee" + + "\x06\x17\x18\xfb\x25\xed\x5f\x4f\x0b\x80\x1e\x8f\xd5\x31\xef\x4d" + + "\xc7\xed\x86\x73\xaf\xcd\x7e\xad\x37\x3e\x0b\xa2\x10\x43\x3b\x4c" + + "\x11\xcb\xaf\x7c\x2a\xad\xed\xe1\xf7\x94\xe5\xbe\x41\xc6\xc9\x89" + + "\x25\xae\x74\x38\x6f\x4e\xdb\xfc\x9a\x61\x61\x7e\xd3\x52\x98\xd4" + + "\x68\xa5\x4c\x12\x40\xc4\xe7\x96\xc4\xab\x65\xbf\xda\x8c\x14\xe0" + + "\x56\x95\x58\x9b\xf3\xd3\xc2\xa9\x0b\x92\x0b\x24\xc5\x9d\x5f\x49" + + "\x5d\xdf\xb6\x1e\xd0\xd0\xb2\xc6\xd4\x4a\x67\x36\x8d\x5c\xb4\x9c" + + "\xd7\x43\x1c\xaa\x13\xe7\xb2\x38\xb6\x7e\xaf\x7a\x96\x2e\x31\xe1" + + "\x86\xc2\x49\xba\xac\xe7\xaa\xb5\xfc\x7e\x2f\x6b\x74\x3d\x59\x8c" + + "\x4f\x4a\x72\x99\xad\x2f\x37\x33\x35\x45\xc3\xc3\x93\x1a\x76\x63" + + "\xf2\x80\xe0\xd5\x54\x06\x5b\x59\x6e\x81\xc5\xca\xbe\x23\xbd\x6a" + + "\xb8\xf1\xc7\xfb\x5b\x2a\x3f\xb4\xcd\xef\x96\x9d\xcc\xfb\x6b\x5b" + + "\x85\x37\xd7\xe7\xef\x10\x27\x8e\x03\x24\x32\xa6\x99\xdc\xba\xc1" + + "\x13\x82\x16\xde\xed\x31\xb5\x75\xa1\x93\xc1\xd6\x0b\x1f\x99\x55" + + "\x6e\x4f\x3c\xbf\xd8\xf8\xfd\xf3\xfc\x38\xad\xbb\x5e\x36\xbf\x8c" + + "\x5e\x01\xf8\x1c\x9f\xb3\x27\x93\xca\x0c\x2c\xa5\x84\x31\x90\x48" + + "\x89\xc0\xa0\x61\x73\xef\x95\xc6\x66\x36\x30\x30\x51\x8e\xf6\x3b" + + "\x37\x44\x26\xfc\xfe\x03\x72\x8e\xb7\xbc\xe1\xa2\xe4\xba\x5f\xaf" + + "\x1e\x06\xf0\xc7\x29\x72\xc7\xe8\xe8\x7b\xad\x94\x67\x1e\xe7\xb3" + + "\xd9\x4f\x78\x96\x6b\x0a\xe5\x56\x0b\x69\xa5\xa5\x04\x8c\xea\x39" + + "\x9f\xac\x48\x05\x64\x75\x09\xf7\xf6\xc4\xf8\x19\xf6\xaf\x52\xed" + + "\xec\x56\x65\xbb\xd8\x1e\xfd\xfc\xc7\x4c\x33\x8b\xd9\x7c\xd6\x70" + + "\x5e\x6b\x4d\x38\x01\xce\x07\x13\xdb\x43\x04\xf6\x8c\xdf\x12\x72" + + "\x8e\xda\x21\x4d\x84\x0a\x62\x97\x05\x41\x60\x96\x83\xf6\x94\x68" + + "\x3a\x12\x2c\xbf\x45\xae\xf9\x72\xfd\x88\xa6\x4c\x77\x56\xe7\x29" + + "\x9f\x66\xfb\x4d\xf9\x87\xf3\x75\x96\xb2\x4a\x6e\xfb\x8d\x89\xaa" + + "\x96\x0d\xe9\xaa\x47\x7a\x59\x5f\xb0\x27\x73\xe1\xa7\xc9\x90\x1f" + + "\xdc\x88\x7d\x22\x41\x5a\x8d\x46\x02\xbf\x74\xb7\x9d\x6c\x5f\x56" + + "\xf2\x5d\x01\x3a\x2e\xe2\x65\x81\xab\xeb\x0d\x7f\x93\xd2\x0b\x8c" + + "\xc6\x7f\x8e\x2d\xe5\xf8\x4a\x42\xd0\xc4\xef\xa4\xc5\x7a\xd3\xd0" + + "\xec\xb5\x16\x31\x00\x8f\xdd\x6c\xa6\x48\x26\x15\x9d\xcc\xf3\x77" + + "\xd1\xe4\xd4\xd4\xc1\x76\x18\x09\x84\x54\xb2\x63\x88\xf8\xe6\x20" + + "\xbc\x83\x51\x5f\xdf\xf9\xb0\xe1\x0d\x58\xe3\xb3\x70\x89\x97\xd8" + + "\x76\xed\xff\x35\x96\x56\x52\x86\xab\x12\x53\xa6\x52\x72\xbd\xf9" + + "\xf1\xfe\x90\x30\xfa\x0c\x7c\xd8\x28\x30\xec\x0f\xd5\x55\x2a\x60" + + "\x63\xa7\xe9\x72\x91\x72\x7b\xc0\xba\x67\x3c\x3f\x9d\x4f\x19\xb9" + + "\x54\x2c\x9c\x00\x0a\x70\xdc\xb6\x5f\x3b\x3a\x8a\x32\x9a\xc1\xb8" + + "\xfd\xa9\x2e\xfa\x4d\xdc\x76\x9a\x59\x97\xff\x4b\x87\xe2\xb1\x3c" + + "\x84\xff\x72\xf3\x5b\xe5\xca\x25\x0e\x4b\xf7\x2b\x5b\x7f\x28\x12" + + "\x43\x5d\xab\x1a\x6c\xaf\x8a\x82\x80\xeb\x50\xa3\xac\xf1\xd0\x9f" + + "\xed\xb2\x92\xec\x7c\xa3\xc1\x48\x42\x41\x9f\x34\x43\x68\x2f\xd0" + + "\xd9\x81\x75\xa4\xcb\x1b\xf4\xcf\x98\xd9\xb6\x58\x7b\xa8\x69\xf7" + + "\x4d\xdf\x9f\xe0\xa9\x2c\x96\xf2\x60\xf0\x44\x7a\xbf\x6d\xcb\x7a" + + "\x68\x3e\x6d\x65\xe6\x03\xd4\x01\x55\xfd\xde\x47\xca\xf8\x05\x41" + + "\xf3\xd5\x61\x9c\x63\xc0\xae\xc0\x08\xc1\x2f\x25\xbe\x9e\xf0\x73" + + "\x97\xe6\x81\x33\xfd\x88\x72\x09\x5f\x6b\xdc\xa3\x75\x73\x9f\x4b" + + "\x74\xb8\x05\x86\xec\xdc\xcf\xab\xee\x24\xb9\xe6\xdd\xf1\x8d\xd6" + + "\x4b\x31\x21\x08\x38\x29\x9a\x3a\x5c\xb6\x2b\x62\x0d\x66\x93\x23" + + "\xe3\x7c\xbe\xc9\xf1\x19\x9c\xd6\x09\x19\x19\x63\x7a\xee\xeb\xe2" + + "\x90\x9e\x5e\x20\x11\x2d\x02\x52\x55\x9f\x26\x36\x30\xe9\xab\xdc" + + "\xac\x4f\xf4\x63\x89\xa6\xe1\x1b\x1e\x7c\xbe\x3b\xb1\x5c\x6b\xbd" + + "\x12\x8f\xbf\x83\x28\xb1\xdc\xd9\x4a\x97\xeb\xe9\x05\xb2\xff\x5f" + + "\xae\x49\x82\xe4\x8c\x64\x2f\xb7\xcc\x21\x68\x85\x58\xc3\x85\x67" + + "\x22\xbb\x99\x2a\x36\xb9\xdf\x07\xed\x1d\xeb\x0c\x4a\x49\xc7\xdc" + + "\xf8\x12\x5e\x0f\xc3\x5e\xee\x79\x6e\xaf\x11\x99\x99\x84\x6e\x04" + + "\x17\x7e\x5c\x08\x99\x93\xea\x46\x90\x7a\xde\xb9\xf3\xb2\x9c\x2f" + + "\x9d\xc5\x93\x42\xae\xb5\x5e\x62\xd6\x6e\xb1\x9b\x1c\x88\xc0\x38" + + "\x02\x95\xc4\x4e\x33\x2d\x21\x6d\x7c\xec\x74\x90\x6f\xd6\x38\x02" + + "\x33\x5e\xaf\xfd\x0b\xca\xf4\xdb\x3e\x41\xb4\xc6\x66\xb9\xbe\x68" + + "\x60\x9f\xdf\x72\x8e\x52\x68\x94\x04\x58\xc2\xbe\xd4\x2c\xc9\xa8" + + "\x36\xee\x78\xc3\xaf\xff\xbc\xe9\x4f\x90\xcb\xa7\xc6\xd9\x1c\x2b" + + "\x7e\x06\xa0\xf0\x3a\x59\x14\x82\x03\xad\xdd\x24\x82\x0c\x53\x51" + + "\x36\x17\xae\x82\x95\x25\x95\xa5\x13\x50\x08\xb7\x5b\x38\x33\x5b" + + "\xaf\x86\x08\x64\x38\xe4\x7d\x8e\xcb\x4a\x22\x7d\x5e\x51\x84\x5e" + + "\x01\x3d\xb1\x42\xef\x23\x90\xd8\x7e\xed\xc9\xd1\x71\xcf\x2b\xb3" + + "\x8a\xbf\x00\xd7\xae\xa6\xbd\xf7\xb2\xf6\x89\x06\x59\xce\xe4\x42" + + "\x61\x50\xcb\x6e\xfd\xab\x33\xa6\x89\xc6\xd2\xa5\xed\x7a\xbd\x96" + + "\x09\x46\xe5\x5f\x79\x40\x6b\x5f\x77\x42\xca\xf4\x7b\x4d\x83\x4c" + + "\x8e\xce\xb7\xba\x0c\x55\x77\xbd\x11\x0a\xb9\x43\x0e\x5f\xe2\xb8" + + "\x30\x2e\x0e\xa6\xef\xef\x14\x36\x76\x3f\x74\xc8\x21\x83\xba\xcb" + + "\xcb\xd7\xad\x93\x2b\xd1\x10\x65\xda\x54\xbc\xc7\x7b\x17\x7f\x76" + + "\x77\x38\xc0\xdb\xf3\x1d\xc3\xac\x9b\xcf\xcb\x59\x0b\xd8\x67\x6a" + + "\x6f\x34\xf1\x8d\x68\x20\xb4\x8a\xaf\xce\xf2\x98\x87\xae\xb9\x98" + + "\x9e\xf1\xc9\x8c\x31\xaf\xb7\xeb\x5a\x6a\x76\x76\x56\x4c\x42\xa2" + + "\xde\x71\x57\xa2\xbe\xbe\x7e\xea\x84\xc7\x45\xcf\xfc\x64\xb1\x0a" + + "\x42\x10\xb8\x99\x93\x12\x17\x1b\x40\x4b\xdd\x1a\x6a\x02\xff\xd7" + + "\x34\xc3\x37\xef\x02\x82\x21\xaa\x9c\x15\x3e\xa2\x7a\x27\xdd\xb5" + + "\xe5\xb6\x2a\x80\xca\xc1\x9e\xcf\x53\xd0\x2a\x70\xcc\xc8\x83\x14" + + "\x75\xdd\x2e\x36\x36\x8e\xad\x83\x8f\xb4\x42\xfd\x8f\xe0\xd1\x45" + + "\x46\x79\xfd\xfe\x05\xf0\x89\xb9\xab\x43\x6c\xd5\xe9\xf1\x72\xbb" + + "\x81\x98\xdd\xef\xd5\xa3\x30\xc9\xe7\xf5\x6a\x18\x50\xa9\xd7\x91" + + "\x02\xc9\x8e\x47\xbd\xe9\xd1\xcd\x77\x0f\x79\x4a\x74\x91\x80\xa4" + + "\xca\x8c\x37\xe2\xdd\x50\x81\x50\x90\x21\xf8\xf1\x6a\x97\x92\x9a" + + "\x1a\x1d\x0d\xed\x37\x6a\x20\xe4\xb0\x5a\xdf\x17\x37\x08\x01\x28" + + "\xe0\xa9\xa7\x15\x4b\x99\x99\x51\xbf\x3c\x5c\x98\xdf\x1e\xce\xfe" + + "\xb3\xe1\x3a\x5e\xaa\x99\x86\x90\x9e\x5c\x72\xab\xc5\xfb\xab\x21" + + "\xb9\xca\x86\x57\x28\x7f\x6c\xcb\xfc\x49\x50\x52\x02\x0d\xc9\x69" + + "\x28\x79\x30\xc0\xfe\xa4\xb6\x96\xc0\x4a\x1b\x13\x05\x90\xef\xcd" + + "\x0f\xb3\x04\x05\x21\xfa\xa0\xef\x0b\x5c\x8a\xe2\x4a\x92\xe3\x97" + + "\xb6\x36\x91\xb1\xb1\x2c\x8e\x4b\x57\x36\xb4\xe1\xa3\x0c\xaa\xa5" + + "\x80\x1f\xca\xca\xe8\x01\xdd\x69\xdb\xf5\x9c\x99\xd3\x3c\xe7\x58" + + "\x22\x5b\x57\xc7\x8d\x40\xfa\x78\xb5\x86\x61\x95\xba\xbe\x88\xbb" + + "\x09\xbf\xb7\x3b\x8d\xe3\xf9\x32\xd1\x36\xa7\x7d\x07\xde\xac\xf7" + + "\xa6\x9d\x2f\x1f\xaa\x6a\x54\xe4\xe4\x81\xe5\x6a\x15\xd7\x6b\xce" + + "\xd3\x6f\xce\x1b\xb7\x2d\xe9\x77\xb3\x97\xaf\x58\x21\xa9\xf1\x22" + + "\x0c\x81\xb3\xd3\x42\xa7\x0d\x97\x5a\xe3\x95\x9e\xcd\x9f\x0a\xdc" + + "\x97\xef\x15\xa5\x35\xbf\x80\x26\x22\x61\x3c\x2a\x1f\xca\xc0\x03" + + "\x08\x6c\x89\x00\x9a\x53\xd2\x0e\xd5\xc7\xbd\x44\x85\x91\xf1\xf1" + + "\xdd\x10\x1c\x7d\xc0\xa5\xce\x8d\xf7\x79\x5b\x18\xc4\x8b\xe6\x20" + + "\xe0\xad\xa2\x8b\xfc\xc2\xfd\xba\xa9\x40\xfc\xa5\xe7\xdc\x37\xec" + + "\xf0\x6d\xef\x64\xaf\xf5\x3a\xd2\xc5\xef\x66\x80\x57\xb5\x0a\x53" + + "\x74\x3d\xd8\xea\xf6\xf0\xdf\xc1\x5f\xde\xda\x39\xec\x8a\x16\xd9" + + "\xf2\xcc\x8f\xc9\x17\xc0\xa5\xa2\x48\xdf\xcd\xe9\xda\x53\x86\xf8" + + "\xed\x38\x5a\xb1\x25\xf2\x70\x6a\xa8\xe2\x3f\x8e\xbb\x24\x78\x4b" + + "\x5c\xea\x55\xb7\x33\x18\x52\x5e\xde\x76\xb1\x55\x50\x91\x0c\xce" + + "\xc1\x04\x8f\x33\x5d\x6d\x20\x91\xcd\x17\x5a\x52\xaf\x0a\xf5\xae" + + "\x31\x56\x18\x14\x92\x7e\x33\x90\xbc\xe4\x9d\x0f\x2f\x8e\x9a\x92" + + "\x8f\x96\x0e\x52\xbf\x98\x1a\x96\xc1\x57\xec\x5d\x92\x16\xa1\x2e" + + "\x2b\x71\xbf\x2c\x41\xe4\x75\x56\x56\xb9\xd1\xf8\x79\x17\x0e\x35" + + "\x32\x36\x7e\x44\x35\x35\xd4\x4f\x9f\xc9\xbb\x29\x30\x71\x92\xa2" + + "\x80\x8d\x64\x8c\x8d\xb9\xf2\x59\x65\xf0\x68\xcf\x83\x53\x59\x35" + + "\xeb\xd6\x5a\xb7\xe3\xa8\x79\x79\x1a\x3c\xf5\x0d\xfe\xfe\x98\xb9" + + "\x1b\x58\x0b\xf2\x99\x40\xaa\xa7\x9e\xf8\x28\xb3\xe4\xe7\x87\x0b" + + "\x7f\xf5\xfc\x0f\x02\x4e\xeb\x6c\x5c\x91\xe2\x62\x92\xe1\xe2\xf2" + + "\x12\xaa\xf6\x24\xf4\x8e\xc9\xa1\xb5\x90\x6e\x76\xab\xd7\xd4\xd6" + + "\x1b\x7e\x8f\x4e\xde\x48\x72\x58\x27\x5e\x0f\xd3\xcb\xa0\xc6\x6e" + + "\xde\x88\x3e\x2c\x53\x78\xb7\xd1\xac\x2a\x11\xcb\x1f\x74\x4d\xe5" + + "\x73\x4f\xcf\x29\x13\x7a\xad\xca\xd2\xe9\xbe\x15\xfb\xad\x06\xa1" + + "\xdf\xda\x8b\x39\x49\xf9\xbc\x0b\x36\xf9\xfd\xaf\xee\x8f\x10\x50" + + "\xb1\xf7\x24\x58\x46\x2e\xd4\x13\x5a\xb7\x43\x09\x5e\xb1\xdb\x69" + + "\x71\xd9\xcd\xfe\x08\x49\xc0\x18\x98\x2d\xfc\xd5\xe5\x2e\x21\xb6" + + "\x46\x6d\x5d\xf3\xfc\xf3\xe2\x42\xbd\x74\xeb\x91\x60\x6e\xf0\x91" + + "\x36\x71\xaa\xfb\x31\xc5\x0d\x4a\xda\xdb\x6b\xc7\xe3\xbf\x86\x3e" + + "\x27\xe0\xc0\xef\x8d\x00\x59\x9b\x56\x85\xb6\x6a\x2c\x1d\x79\x2b" + + "\xd0\x5c\x32\xc1\x22\x68\x2c\x1c\x0b\x30\x45\xf9\x69\x91\x71\xad" + + "\xb1\xde\xaf\x7d\x31\x13\xfd\x01\x4f\x36\x90\x7e\x90\x0d\xcc\x8e" + + "\xd2\x38\x13\x93\x1e\x85\x14\xfa\x7f\x3f\x03\x3e\x41\xda\xba\xfb" + + "\x90\xf1\x12\xc4\xc3\x26\x52\xcb\xc9\xe3\x6a\xd0\x6a\x54\xeb\xbe" + + "\x0f\xc5\xf9\x87\xd1\x26\x74\xc0\x74\x0d\xa5\x74\x3f\xfc\xbd\xf7" + + "\x3a\x5b\x6d\xa1\x61\x63\x4b\x80\xa8\x47\xed\x76\x8d\x5e\xe5\x6a" + + "\x7d\x62\xe1\x33\x9c\xa9\x7a\x59\x08\x2a\xa5\x6a\x85\x64\x74\x39" + + "\x96\xe4\x86\xfb\x67\x10\xff\x2b\x9b\x62\x31\x77\x87\x2e\x36\xb6" + + "\x00\xf5\x41\x1c\xb5\xd8\x23\x85\x48\x21\x51\x2a\xdb\x8b\xa8\xc3" + + "\x03\x74\x70\xbf\x1d\xdf\xae\xe7\x42\x85\xea\xa5\xff\xbd\x07\x3f" + + "\x55\xb5\xae\x7c\x49\x22\x64\xb8\xb8\xbf\x7f\x27\xc2\xd1\x28\xe4" + + "\xcb\x03\x79\x6d\xfe\xb1\x2a\x2b\x2b\x5b\xcc\x15\x8e\x4a\xa5\x34" + + "\xf7\x24\xe0\x6c\x04\x3e\x3f\x9c\x43\xeb\x2e\x3f\xd5\xb1\x27\xbc" + + "\xf6\xa3\x08\x99\x9c\xad\x34\x4d\x64\x70\x9e\xe4\xd4\x05\x6c\x5f" + + "\xb0\xef\x4d\x2a\xa4\x77\x5b\x60\xb5\x1b\xa2\xe2\xa3\x0f\x6c\x76" + + "\xa0\xd4\x3a\xad\xab\x9b\xbd\x35\x12\xa6\xf2\xa1\x64\xca\x31\x34" + + "\xdd\x6c\x0f\x42\xc5\x6a\x98\xcf\xab\xa0\x1c\xb8\xf7\x65\x5b\x99" + + "\xae\xe7\xa7\x4a\xe7\x4c\xd9\xd9\x47\x0c\x37\xa0\xab\x31\x6d\x6e" + + "\xb2\xb2\x19\x7c\xb5\xc7\xec\x89\xb6\xcd\x90\xee\x90\xc3\xb8\xc1" + + "\x7e\x47\x7e\x94\x3e\x63\xce\xe6\x3a\x68\xac\x2e\x93\x03\x59\xcc" + + "\x6c\x93\xf5\xa4\x66\x51\x91\x6d\xef\x68\xb6\xb1\x9f\x45\xf2\xad" + + "\x6f\x8a\x88\x88\x8c\x4c\x4e\x9a\xae\xf6\x8d\xee\x79\xf2\x77\x7e" + + "\x3c\x11\x52\xfb\x71\x5e\x34\x3e\x58\xcc\x97\xa2\x11\x70\xdb\xf1" + + "\x5f\x70\x71\xc9\x54\x4e\xce\xd0\x9c\x93\x75\x69\x7f\xf9\x94\x7c" + + "\xa1\x2c\x9d\xe1\x72\xad\x66\x5f\xbf\x6c\xd5\x7c\xe7\xe5\xce\xd4" + + "\xa4\xfb\xa0\xbc\xef\x65\xca\xf9\xe9\x45\x23\xd4\xb1\xd2\xd0\x67" + + "\xe1\xbb\xa7\xff\x03\xf2\x9a\xf3\xc9\x9d\x47\xa7\x3e\x59\xf0\x56" + + "\x86\x19\x92\xe7\x7b\xb4\x94\x6a\x19\xa6\xf3\xd0\x23\xc8\x4c\x05" + + "\x0e\xce\x29\x61\xd7\x29\x31\x25\xc5\x6d\x36\xdc\xaf\x98\x2b\xeb" + + "\xd8\x3e\x0c\x85\x84\x9a\x9b\x3b\x35\xee\x17\x4d\x46\xf9\xe5\x62" + + "\xd2\xb2\x0f\x87\xeb\x3b\xde\x85\x52\x97\xcf\x53\x43\xf3\x6e\xc6" + + "\x01\x52\xa9\x6e\x1e\x3f\x96\xf1\x78\x7c\xf2\x99\xd1\x43\x57\x3f" + + "\xa3\x4e\x03\x5e\x8d\x86\xf9\x64\xce\xea\xd1\xca\x03\x55\xe8\x7d" + + "\x77\x18\x31\x51\x0d\x92\xe4\x49\xb5\x25\xbd\xad\x0e\x75\xbd\xc9" + + "\x7e\x8e\x07\x10\xc6\x1e\x8a\x31\x3f\x35\x43\x53\x24\xc2\x83\x8f" + + "\xc5\xfe\x26\x31\xc0\xe7\xbc\x4e\xff\xf6\xc8\x31\x71\xc4\x15\x1c" + + "\xbf\xeb\xf4\xe2\x7d\x34\xf0\x2a\x5b\xe0\x7d\x4f\xc4\xb3\xdb\x84" + + "\xd8\x4f\xd9\xa5\x4d\x98\xd9\x4c\x95\xfa\x38\xd6\x8a\xd0\x0b\x41" + + "\x0b\x28\x07\xde\x53\x13\x93\x83\xbf\xc9\x7b\x80\x28\x1c\x16\xc1" + + "\xb3\x4d\xbb\x8e\x3a\x2a\x94\x4d\x57\x04\xfd\xd8\x18\xc3\x0c\x3f" + + "\xf4\x8a\xf2\xbe\xc4\x8f\xa3\x31\x94\xa8\x25\x56\xdb\x73\xe5\x39" + + "\xb6\x7a\xdc\xa7\x8d\x54\xd5\x2a\x5c\xd1\xa8\x21\xd7\x4f\x76\x33" + + "\x53\xac\xb2\xd9\x7e\x7b\xf2\x29\x95\xaf\x18\x97\xc8\x80\xec\x16" + + "\xa3\x5b\x99\x3f\xd2\xc9\xc4\x70\xd0\x3a\xc5\x96\xc4\xfe\xca\x6a" + + "\x58\x7e\xcf\xaf\xae\xb3\x15\xe3\xae\x7b\xed\x37\xf2\xf5\xe1\x7e" + + "\x27\xce\xd8\xc0\x89\x1b\x8f\x23\x37\x10\x27\xcd\x40\x4f\x11\xda" + + "\xf2\xf2\xf2\xbb\xa2\xc9\x7e\x7b\x15\x8e\x4b\xd5\xcf\x8b\xa3\x01" + + "\x60\x4b\xcb\xf0\xfe\xd8\xcd\x34\x2f\x66\x65\x20\xbb\x5c\xc4\x70" + + "\x01\xa8\x5a\xcd\x58\x90\x80\x3d\x4d\xa7\xda\x3a\x06\x75\xc6\x7d" + + "\xa4\x75\xdc\x28\xe2\xe4\x29\x57\xc4\x45\xf8\x1b\x5c\xbf\x14\x9e" + + "\x44\xd9\xcc\x7b\x6e\x2e\xf6\x3e\x0a\xf1\x2b\x25\x46\xa7\x21\x67" + + "\x85\xda\x5e\x6c\x1b\x46\x78\xb4\xa0\x6f\x53\x6c\xe7\x2b\x79\x78" + + "\xfc\x76\xde\xdf\x2a\x17\x73\x90\x44\xd3\x2b\x76\x6d\x63\x7b\x09" + + "\xa4\x36\x3e\x6d\x95\x17\xb6\xf3\x7f\xb3\x79\xf5\x34\x28\xd1\x3b" + + "\xa1\xb4\x6b\xe5\x2d\x93\xdc\x0e\x1a\x2e\xb8\x99\xc9\x22\xc2\x65" + + "\x1f\x68\xd1\xbd\x8d\x27\xdb\xfd\xa4\xdc\x4b\xe4\xc6\x17\x7d\x3c" + + "\x00\x4d\x9b\xa1\xfe\x44\x15\x75\x98\x0a\xbb\xbd\xbd\x6e\xc3\xbc" + + "\x44\xf2\xb3\x27\x69\x12\x7f\xc1\xec\xfe\x27\x0b\xc5\x24\x08\x43" + + "\x19\x8d\x5c\x48\x08\x77\xbe\x4c\x6b\xbf\xd2\x0c\xe8\xe6\x83\x95" + + "\x70\x4c\xc3\xea\x79\xbd\xa7\xac\x35\x35\x72\x72\x57\xef\x72\xa4" + + "\x9c\x4f\x2e\x82\xb7\x72\x61\xc7\x71\xf6\xaa\x2f\x55\xc1\x70\x32" + + "\xfb\xd1\xe5\x3c\xce\x45\xf1\x3b\x43\x6d\x57\xb2\x3c\xa7\x67\x91" + + "\xc8\x5b\x09\x13\xcd\xa1\xae\xdd\x32\xa3\x93\xff\x50\x10\x50\x76" + + "\x02\x13\xca\x47\x18\x8d\x15\x75\x88\x83\x26\x22\xe6\x7c\x52\x95" + + "\x70\x7a\xfa\x85\xaf\x57\x51\x55\x0d\x8d\xa4\x5b\xb0\x54\x12\x4b" + + "\x72\xe1\x95\xe7\xea\x6b\x53\x79\xce\x9d\xe0\x39\x3f\x4a\xec\xc3" + + "\x79\xdf\x54\x16\x9f\x56\x87\x9b\x6d\x99\xb2\x09\x59\xb0\x7f\xce" + + "\xeb\x9b\xbb\x4b\x4f\x52\x02\x41\x8d\xa5\x57\x8c\x31\x7e\x60\x76" + + "\x56\xcd\x7e\x37\x8a\x7b\x3f\xc7\xf4\xdb\x32\x22\x68\xaf\x3d\x25" + + "\xc2\x03\xc8\xdb\xfd\xda\xcc\x4c\x8a\xb3\x34\x5f\x8a\x0e\x46\xb9" + + "\xbd\xbe\xbe\xff\xc7\x94\x6f\xa0\x6a\x9d\x32\xf6\x97\x27\x4f\x54" + + "\xf6\xc6\x03\xeb\xc7\x9a\xf7\x0b\xd2\xbb\xe6\xf5\x9f\x3d\xc5\xcf" + + "\x15\xdc\xaf\x5f\x7e\x34\x49\x63\x45\x8a\x15\x1f\x3c\x65\x12\xf2" + + "\xba\x9f\xeb\x96\x63\x3e\x45\x73\x6f\xfc\x1a\x4d\xa0\x7a\xaa\x73" + + "\x35\x2f\x4c\x9a\x8f\x64\xf1\x7a\xba\x58\x94\x3b\x61\x07\xef\x6e" + + "\xff\x27\x0f\x81\xc5\x23\xb2\x47\x51\xfb\x39\xf8\x3c\xdf\x48\x7f" + + "\x6e\x62\x89\xb3\x9e\x01\x71\xae\x85\xac\x73\xb4\xc4\xba\xc3\x15" + + "\x17\xa4\xe5\x48\xcd\x93\x62\xa3\x02\x85\x3f\x8e\xd4\x39\x18\xc2" + + "\x2f\x51\x08\x39\xe5\x2a\xd5\x0a\x9d\x6d\x20\x15\xac\x1e\xcd\xe3" + + "\x72\x88\xda\xfd\x76\xaf\x53\x86\xf6\x1e\xc5\xd2\x05\xdb\x73\x49" + + "\x73\xd1\x9f\x05\xee\xe5\xa0\xaa\xaa\xa9\xf4\x5f\xd9\x35\x1e\xff" + + "\x89\x4a\x40\x34\xba\x20\x47\x00\x19\x14\x8c\xfb\x9a\xb3\xea\x08" + + "\xd6\xfb\x27\x42\x4a\xf6\x59\xe7\xe6\xe1\x74\xa9\x54\x99\xec\x57" + + "\xea\xf8\x62\xb9\x12\xdd\x66\xb2\xc2\x59\x19\x2d\x1a\x99\x69\xf4" + + "\x61\xaa\x8d\xd7\x24\x07\xfb\xf9\x34\x0a\x86\xb8\xec\xd3\x08\xb9" + + "\x0e\x25\xd9\x96\x87\x3a\x33\x40\x58\xbc\x72\xa5\xec\xe4\xeb\xb4" + + "\xe0\x4a\x94\x38\x03\x97\xc3\xcd\x24\x77\x8e\xfb\x7d\xf2\x8e\x18" + + "\xa4\x83\x5e\x79\xf0\xbb\x74\xb4\x21\xb9\xaf\xf9\x20\xb5\x1b\x3a" + + "\x6c\x39\xd0\x03\x9f\x5f\x9a\xd3\xef\x4f\x5d\x98\x5a\x36\xe7\xf1" + + "\xe1\x4a\x23\x73\x29\xdd\xd2\xb0\x34\x2f\x72\xfc\xf5\x60\x7a\xc6" + + "\x83\x2e\x50\x34\xc5\x35\xd4\x92\x0f\xa2\x4f\x6a\xbd\xfa\xb9\x64" + + "\xd9\xc7\x2d\x1b\x16\xa6\xaf\xbc\x5a\x92\xf0\xf2\x0d\x44\x96\xd6" + + "\x6d\xf4\x50\xb9\x39\xcb\x55\x76\xff\x30\x17\x24\xa1\x43\x18\xc2" + + "\x76\xca\xc8\xce\xd7\x02\x97\x2f\x01\x25\xb7\x8c\xde\x90\x4c\x5c" + + "\xe1\x5e\xeb\x1b\x57\x64\x8b\xfe\xa1\x8c\x9b\x1b\x17\x84\x6c\xf9" + + "\x85\x67\x59\xb8\xbc\x6e\x0f\x7b\x4f\x97\x1b\xda\xe8\xcb\x88\x59" + + "\x50\xda\x71\xd6\x59\xbf\x67\x27\xf7\x13\xc5\x02\x03\x51\x03\x15" + + "\xf4\x49\x24\x63\xe5\x6d\x15\xd5\xbe\x64\x16\xe7\xd3\x4a\x47\xd8" + + "\x50\x4c\x8e\xba\xde\xec\x85\x47\x04\xd8\x7f\xa5\xa8\x93\xf7\xf9" + + "\xc1\x12\x95\xc5\x9a\x2f\x24\xb8\xbb\xa3\xa9\x93\x22\xe9\x37\xca" + + "\xe8\x65\x73\x4a\x7e\x77\x38\xeb\x36\x89\xd0\x15\xeb\x73\xdc\xdb" + + "\xc0\x8c\x77\xb6\xe2\xfe\xdd\xff\x42\x50\xb1\x34\x4c\x7d\x8a\x8e" + + "\x99\x53\x6e\xf4\xae\x2a\x71\x26\x4b\x58\xed\xee\xff\x6b\x01\x38" + + "\x92\x27\x33\xb9\x41\xcd\xb6\x3a\x29\x8c\x5a\x4e\xb3\x27\x69\x77" + + "\xa7\xcb\x89\x10\x17\x0b\x7d\xd9\x3b\xf2\x7e\xd8\xee\x28\x52\x7d" + + "\x6f\xfe\x7c\x3f\x92\xc3\x55\x38\x40\x3f\x52\x1a\x19\x38\x59\x3c" + + "\xfb\xe6\x98\xf3\x90\x4b\x0f\x7f\x43\x13\x4b\x8f\xc5\xb9\x2e\x8d" + + "\xce\x95\x19\x9a\x0f\x6b\x98\x51\x6d\x95\xa1\x5e\xd6\x5c\x4d\xb2" + + "\x66\xa7\x75\x8b\x2e\x66\x65\x75\x91\x16\x38\x8d\xdc\x11\x7d\x39" + + "\x03\xd2\x31\xab\xb1\x8b\x9b\xbf\x9a\xc5\x90\x44\xee\x52\x33\xf9" + + "\x82\x20\x12\x89\x1f\x05\x4f\x96\xc6\x67\x49\xb9\x55\xd1\x81\x8d" + + "\x85\x25\x6e\x65\x75\x2c\x7e\xa9\xef\x23\xa0\x3c\x1c\x43\xf4\x74" + + "\x90\x69\x0f\x48\x7b\xde\x40\x4d\xf2\xe9\x03\xde\xd5\xc5\x3e\x32" + + "\xe1\x2f\xe5\x99\x26\xf2\x0e\x72\x00\x8a\x7e\xb5\x58\x7e\x3f\xe5" + + "\x0a\x0c\x22\x2c\x34\x37\xbb\x59\x0f\xdc\xd4\x6a\xf3\x23\x10\x1d" + + "\xf6\x93\x4f\x0d\x0b\x85\xee\x3a\x1a\x7e\xca\x89\x26\x1f\xb1\x64" + + "\xc1\x58\xd5\xea\xce\xba\x54\x7e\x67\xb8\xba\x6d\x62\xf4\x8a\xe8" + + "\x77\x16\xef\xa5\xe9\x98\x0c\x75\xcc\xb0\x24\x2b\xe9\x9f\xc2\xe8" + + "\xdf\xe0\x8b\x0e\xb9\xb7\x3d\xe2\xd7\x50\x60\x67\xa6\x8e\x1c\x04" + + "\x0d\x39\xbc\xad\x4f\xc7\xc5\xc7\x0b\xdb\xd8\x30\x82\x8a\xe5\x0a" + + "\xcc\xa7\xb3\x07\xed\x66\xe1\xab\x53\xb3\xb2\xe4\xa2\xb0\x99\x1e" + + "\x4f\x58\x36\x6d\x7a\x11\xde\xce\x8b\x85\xab\xf0\xd3\x8a\x23\xe4" + + "\xd4\x96\x81\xd3\x1a\xa9\xbe\x5b\x41\x2d\x86\xb8\x60\x83\x27\x31" + + "\x90\x2a\x87\x9a\x61\xef\x35\xd2\x6b\xed\x1e\x4a\x4c\x91\xb9\x51" + + "\x06\xb5\x16\x4f\x75\xa2\x7a\xd7\x82\x4c\xa9\x44\x37\x8c\xdd\xee" + + "\xb3\xba\x14\xc4\x6f\x98\x9a\x4d\x7b\x89\x34\x7d\xaf\xb6\xcf\xa2" + + "\xc4\x66\xde\x64\xc5\xf5\xa0\x67\x64\x0c\xad\xac\xc4\x8c\x3a\xa4" + + "\x82\x02\x4d\x94\xcb\xbc\xff\x58\x0b\x72\xdf\x08\xf6\xc4\x1f\xef" + + "\x7c\x9e\xb6\x7d\x6d\x36\xbb\xb1\x64\x5a\x7c\x0a\x4d\x84\xb4\x35" + + "\x9b\xc7\x77\x51\x3f\xad\xe6\x14\x61\x0a\xa1\x42\x8d\x1a\x7d\x9f" + + "\xbb\x05\x8d\x89\x6c\x81\x0e\xcd\x1c\x15\xd8\xde\x9e\xdf\xd0\x9e" + + "\x0f\xeb\xee\x6f\xdd\xbe\xb7\x2b\xa4\xda\x9c\x65\x60\x03\x96\x5c" + + "\x82\xaf\xa3\x2f\xb1\xa8\xf8\xa7\x94\x6c\xfc\x9f\x53\x5f\xd6\x24" + + "\x66\xbf\x74\xb5\xdb\xa8\x74\xeb\xf4\x35\xeb\xbd\x48\x6d\xf8\xef" + + "\x15\x5a\xaf\x79\xec\x46\xdb\xfc\xab\xfd\x5d\xf0\x33\xf6\x93\x22" + + "\x77\x1e\xbd\x33\x75\x34\x29\xcd\x92\x9c\xb0\x6f\x7b\x7d\x30\x98" + + "\xa4\x3b\x2a\x64\x7c\x89\x22\xb8\x3f\x2b\x39\x79\x67\xbe\x91\xfe" + + "\xf0\xd9\x6f\xdc\x95\xe5\x3e\x93\x1e\x59\xa1\xf7\xc7\xda\x7a\x4b" + + "\x33\xdf\x99\x63\x58\x4c\x71\xd6\xa0\x76\xb6\x3f\x01\x62\xff\x5e" + + "\xbb\x7a\xf4\x93\xdd\x1c\x03\x87\x42\x65\xff\x5f\x1e\x67\x58\x78" + + "\xf8\x9d\x8a\x9a\xae\xf6\x5f\x38\x44\x5b\x93\xdc\x9b\x03\x76\x2d" + + "\x90\x73\x16\x80\x9a\x6c\x0c\xbe\xa7\x63\x47\x9f\xfc\x55\xf4\x6c" + + "\xdc\x4a\x2a\x0d\xaf\x75\x1e\x6b\xd5\x8e\x7b\x74\x1a\x42\xe9\x9e" + + "\xb5\xee\xb1\x80\xf8\x80\xc0\x2a\xbe\xfb\x6d\x27\xef\xf0\xc8\x48" + + "\xa3\x83\x89\x74\xc8\xd5\x41\xd1\x51\x09\xbd\xdd\x62\xfc\x8c\x21" + + "\xcc\xb3\x5e\xaa\x0e\xb0\x47\x8b\x1f\x2e\x50\x51\x47\xd3\x11\x4c" + + "\x80\x38\x19\xc9\x74\xb0\x62\x52\x78\xd9\x6f\xf9\x6b\x19\x09\x17" + + "\xba\x64\xac\x2f\x39\xec\x0d\xce\x0a\xde\x6b\x55\xdc\x8d\x85\x76" + + "\x7d\x65\xce\x5f\x4d\xf6\xf7\xb0\xdb\xea\x47\xd3\x19\xed\x20\x16" + + "\xbe\xcb\xe3\x45\x4b\x42\x60\x1d\xa6\x70\x54\xe0\x61\x2d\x08\x74" + + "\x64\xb2\xd6\xea\x48\xf8\xdd\x21\x77\x64\x49\x1b\xe8\xc5\xa9\x52" + + "\x22\x6f\x3a\x10\x9e\x8a\x26\x72\x39\xc1\x3e\xbd\x75\xe7\xd5\x76" + + "\xb4\x62\xf6\x7a\xb5\xfe\x59\xff\x74\x73\xb5\x1b\xfa\x62\xf3\x9c" + + "\x2b\x60\x55\xc5\xe5\x41\x4b\x3a\xa2\xdf\xce\xf8\x35\xe9\xe2\xd1" + + "\x2a\xc6\x1e\xc9\x1f\x02\xb6\x6f\x11\xcc\xe5\xea\x55\xd2\x90\x3e" + + "\x22\x25\x2b\xcb\xbf\xfd\x37\x9d\xdd\x7c\xae\x5c\x8d\x21\xe1\x81" + + "\xc4\xf0\x50\x27\x11\x9a\x9e\x6c\xaf\xcd\x6d\x44\xf4\x65\x53\xd4" + + "\x21\x0e\x1a\xc4\xbf\x31\x6e\x56\xe1\x62\x9a\x2b\x74\x43\xe5\x09" + + "\x2a\xd7\x16\x18\x62\xbd\x27\xfc\x82\x0b\x11\x8c\x76\xf7\xce\xfb" + + "\x46\x16\x4a\x86\x61\xf1\x30\xef\x1c\x3e\x4b\xc3\x70\xd4\x40\x39" + + "\x21\x5f\x9f\xc5\x3f\x65\xe0\xeb\xeb\xa0\x20\xc2\xce\xef\x6e\x91" + + "\xdd\xed\x08\x3f\xf6\xc7\x92\x8f\x66\xf2\xa0\x89\x63\xc3\x26\x26" + + "\x64\x9d\x9c\x9c\x4a\xd5\xab\xf0\x17\xca\xd5\x76\x2f\xba\x3f\xcc" + + "\x7a\x03\xbf\xf8\xa5\xe2\x92\x7e\x1f\x11\xce\xc4\x8e\x6e\xdd\x3e" + + "\x66\x68\xdb\x4d\xb1\x1e\x99\x72\xcc\xe1\xe2\xb8\xf2\xe4\x0e\x1c" + + "\xc8\xf4\xdd\x15\x71\x16\xc4\x8b\x01\x33\x76\x8c\xc0\x4b\xbe\x0f" + + "\x0d\x1c\xbd\xb6\x9b\xb1\xff\xb7\x02\xe0\xe6\x60\x92\x82\x63\x9e" + + "\x5b\xd9\xa0\x48\x1a\x0b\x99\x98\x3f\x7c\x78\x97\x87\xce\x37\x85" + + "\x45\xa5\x52\x18\x02\xd3\xc3\xb1\x24\xe0\x30\xba\x1a\x25\x32\xde" + + "\xfa\x89\xb6\x73\x41\x22\xd5\x56\xef\x1e\x6b\x9b\xb5\x66\x9d\x40" + + "\x5f\x4e\x8f\x6a\xe5\x24\x5c\x59\xf6\x41\x72\xaf\xab\x3e\xbb\xc0" + + "\x9f\x8d\x76\xda\x39\x0f\x3f\x6f\x85\x76\x6f\xb5\xa5\x57\x1c\x79" + + "\xd7\x5e\xd6\xa4\x52\x1c\xfa\x5f\x51\xae\x8e\x25\xea\x5a\x18\x19" + + "\xf8\x52\x87\xc3\x80\xc3\x06\x90\x5b\xbc\x70\x87\xd1\x32\x74\x40" + + "\x41\x39\x12\x31\x59\xac\x54\x20\xfd\x05\xfd\x64\x08\x74\xf4\x9c" + + "\xd2\x68\xa1\x0b\xa3\xc7\x0d\x31\x32\x58\x9e\x82\xbc\x23\x47\x5b" + + "\xcf\x1d\x39\x07\x3d\xdb\xaa\xe7\x27\x8b\xd9\x3a\xf6\x16\x9e\x12" + + "\x09\xab\x1c\xd1\xa3\x20\x7f\xa3\xfb\x69\x06\x3c\x7e\x0f\x56\x7b" + + "\xe0\xdd\x22\x5c\x50\xf8\x7f\x0f\x4e\x0d\xfb\x43\xdf\x4d\xd2\x4a" + + "\xc6\x5c\xbb\xdd\xce\xb4\x33\x38\xad\x25\xce\x32\xd4\x1e\x37\x61" + + "\x58\xb2\x14\x6b\xaa\x19\xb5\xee\x4a\xc3\x59\x57\x6e\x71\x07\xba" + + "\x44\xe9\x60\xd1\x3a\xa5\xc7\xd6\x2a\x18\xa3\xf4\xd7\xb8\xb1\xa7" + + "\x0c\x8a\x5c\x0c\x88\x88\xb0\x7a\x59\x0c\x62\x04\x4f\xae\xc6\xd5" + + "\xed\x77\x92\xdd\xbb\x7b\x92\xc1\x54\x0c\x7c\x38\x61\xeb\x43\x24" + + "\x9b\x72\xb4\x68\x54\x77\xfb\x72\x1a\xb0\x6e\xbc\x92\xd3\x13\xf9" + + "\xba\x26\xdf\xf1\xd2\x79\x18\x75\xbb\x62\x46\x2e\x15\x5d\xf4\x4a" + + "\x7c\x4f\xa8\xd3\x94\x77\xae\x8e\x71\x54\x29\x74\xa5\x47\x87\x6c" + + "\x47\xd7\xb8\x3d\x38\x32\xca\xaf\xcf\x82\xc9\x0e\x9f\xcb\xb8\xef" + + "\xf7\xda\xd7\xfe\x78\x7d\x58\x37\x9f\xad\x4c\xe7\x0a\x07\xc9\x6b" + + "\x05\xad\xb3\x22\x12\x22\xc3\xd8\x77\x3d\x51\x07\x7d\x45\xe2\x3d" + + "\xf3\x35\xf0\xb9\x12\x97\x88\x21\x1c\xfd\x83\x62\x90\x6f\xb6\xd0" + + "\xca\x66\xb9\x00\xb5\xf1\xf7\x11\xc9\x60\x4a\x3f\xad\x26\x71\x1b" + + "\x45\x32\x9d\x07\x12\xdb\x3a\x74\xe0\x5a\x83\x72\x31\xb0\xba\x1c" + + "\xfe\x04\xec\xef\xa1\xbd\xff\xfe\xb0\x05\x8a\x11\x08\x3b\x5c\x74" + + "\xff\xce\x25\x67\xaa\xf7\xee\x99\xca\x67\x93\x31\xe6\x6f\xf0\x16" + + "\x7e\x81\x53\x53\x6c\x24\x8b\x89\xa1\x7f\x04\x3e\xe9\x09\xbe\xec" + + "\xce\x85\x40\xc7\x33\xff\x4d\xf3\xc7\xdf\x2f\x08\xd8\x33\xd5\x7a" + + "\x1d\x2a\x63\x36\xeb\xed\xae\xba\x9e\xfb\xc9\x26\x26\x26\xa8\x68" + + "\x68\x8e\x56\x35\x54\x0c\x51\x97\x97\x0e\xe5\x6a\x91\xa6\xe7\x25" + + "\x14\xd3\xb8\xfa\x9f\xd9\xfa\x44\x02\x2d\x3c\x5f\x00\xca\x73\x31" + + "\xd0\xf0\x64\x64\x02\x14\xd7\xd7\xea\xe6\xc7\xcf\xc8\xd6\xac\xd5" + + "\x9a\x76\x95\xc1\xbe\x47\x0d\x3f\xa8\xdf\x42\x81\xfe\xd2\x0f\x8f" + + "\x9b\xdf\xda\x2c\xa5\xcb\xc1\x38\xe4\x71\x49\x73\x13\x29\xe0\xff" + + "\xcc\xfb\xc8\xae\x3b\x1a\xfa\xe9\x96\xa2\x34\xef\x74\xa7\x20\x5c" + + "\x8d\xdd\xd6\xdd\xf1\x7f\x79\xb6\x4e\x42\xb2\x4f\xe3\xa9\x18\x07" + + "\x1d\x50\xa6\x52\x92\xdd\x6c\x6c\xb3\xd6\xfa\x8d\x2b\x33\xfa\xa4" + + "\xa6\x7b\x63\x53\x2a\xbb\x70\xcb\xd0\xf0\xf6\x62\x33\x07\x08\x13" + + "\x5b\xff\xa1\x08\x6f\x8a\x95\xbe\xdc\xc6\x98\xc1\x4b\xfd\x91\x49" + + "\xe7\xec\x54\xcc\xb2\x70\x47\x24\x26\xd1\xdc\x12\xae\x4f\x48\xab" + + "\x71\x5e\x11\xf4\x25\x27\xf5\xb5\x0b\xaa\x21\x3c\x4a\xed\x94\x2b" + + "\x7a\x95\xb7\x51\x02\x2d\xf0\x87\x56\x32\x53\x30\x29\x23\x47\x1a" + + "\xd6\x87\x77\xec\xcb\xc5\x61\xcf\xe6\x28\x4c\x9b\xf7\x96\xc5\x39" + + "\xf1\xf7\x0c\xc7\x9c\xe7\xb9\xef\x2d\x36\xca\xb1\x78\x24\xc8\xc6" + + "\x67\xee\x18\x92\xa5\xfa\x11\x10\x88\xf9\x9b\x87\xa1\xf6\xaf\xd5" + + "\x91\x02\xd2\x58\xf6\x4b\x72\xfc\x78\xab\x74\x11\x22\x05\xbb\xbe" + + "\xbb\x49\xff\xa8\x71\x14\xca\x3c\x45\x02\x15\x3e\x9c\x9d\xdd\x08" + + "\xe8\xda\x1b\x8b\xd3\x3d\x48\x48\x0c\x7d\x82\x74\x40\xc1\x0a\x7d" + + "\xbb\x46\xcb\x62\x01\xa4\x70\x2f\x0d\x55\xe3\xe3\x0e\xd1\xc8\xf9" + + "\x2d\x83\x94\x9a\xdd\xb2\x78\x8c\xe5\xeb\x1b\x4a\xfe\x8d\x9e\x51" + + "\x57\x75\xd6\xd2\xe2\x93\xaf\x3f\x63\x4c\x59\x8d\xdf\xd9\xe8\x35" + + "\x22\x0f\x9a\xef\xab\x5e\x41\x0e\x80\x02\xa1\xd7\x8c\x32\x67\xec" + + "\xf8\xc4\x44\x51\x07\x87\x4b\xcb\x1a\x0c\x7c\x7c\x69\x79\xf9\x29" + + "\x3c\x29\x42\x7c\xfc\xc8\xfe\xe2\x5a\x8f\x63\xdc\x09\x36\x69\x37" + + "\x3a\xaa\x8c\xf2\xb9\xc3\x83\x9f\xaf\x53\x46\x8a\x33\xfd\xdb\xcc" + + "\x4b\xc3\x5a\xbf\xaf\xff\xa4\x4e\xe4\x5b\x6b\xbb\xca\xb2\x9f\x69" + + "\x89\x47\x3c\x22\xc5\xe9\xb8\x9f\x9a\x6e\x47\xde\xd1\x24\x63\xf0" + + "\x1e\x15\x01\x34\x54\x1f\xce\x98\xba\x66\x80\xff\x8a\x66\xdb\x03" + + "\xec\x8f\x4b\x9c\x42\xae\x11\x3d\x6a\x19\xdc\xd7\x5c\x26\xe1\x28" + + "\x53\x95\x9f\x4b\x28\xc0\x13\xe9\xce\xd2\x5a\x58\xc1\xc2\x99\x5c" + + "\xb6\xe4\x5c\x5c\x29\x7b\x7b\x7b\xa0\x31\x97\x9e\x00\x28\x13\xc2" + + "\xe9\x21\x33\x10\x6d\x05\xaa\xc8\x2f\xe9\xc4\x5e\x14\xd7\x07\x02" + + "\xa1\x54\xeb\x6b\x2f\xc8\x54\xbf\x1e\x3e\x67\xfb\x14\x21\x5f\xa8" + + "\x93\xae\xf5\xd6\xf8\x8c\xdc\xd8\xfb\x45\xae\xdb\xf1\x51\x2c\x57" + + "\x68\xdc\x9b\xa6\x2c\xbb\x3f\x5a\xf4\xd0\x6f\xa4\xbf\x2f\xb6\x56" + + "\xd7\x2c\xbe\xa4\xaf\x39\x91\x7d\xe6\xb4\x9a\xdf\x9a\xc9\x13\xe5" + + "\x7f\x58\x73\x2e\x11\x38\x5f\xd4\xb1\x67\x6a\x48\xe9\x64\x95\x55" + + "\x71\x32\x68\x21\x80\xb7\xb6\x98\x8c\x8b\xfb\xaa\xa1\x25\xc6\x42" + + "\xed\x01\x7c\x51\x0e\x86\x35\x88\xb7\xaf\xfc\x0f\x05\xb9\xef\xfd" + + "\x42\x9b\x32\xdf\xa9\x29\x0f\xe7\xf1\xbc\x10\xd4\x07\xa2\xdf\x1e" + + "\xfd\x4d\xd9\x9a\x5d\xc0\x9e\xad\xd5\xa6\xc6\xc5\xb2\x3c\x2f\xb2" + + "\x8a\x1d\x6e\x67\x8f\xcf\x27\x35\xa3\x23\x23\xdf\x8b\x2a\x89\x7d" + + "\x2e\x85\xfb\x44\x82\x2e\x8f\xbd\xd2\xa9\xdb\x51\x3f\x89\xc3\x66" + + "\x7b\xb5\x76\x7c\xb4\xfc\xfd\x6b\x9b\xa3\x2d\xc6\xd2\xfc\x5d\x00" + + "\x9b\x8e\x5d\x87\x41\x3a\x58\x68\xfa\xaa\x97\xc2\xf2\x15\xbc\x5c" + + "\xd7\x78\xca\x15\x30\xfd\xf1\x80\x40\xd6\x2b\x7f\xda\x05\x1c\xe8" + + "\x9a\x28\x7b\x92\x6e\x33\x3a\x3b\x60\xe3\xf9\xa8\x20\xa0\x46\x3a" + + "\x79\x74\xeb\x96\xcb\xe4\x3a\x6b\xcc\xca\x4a\x4a\xb0\x5a\xf0\x91" + + "\x09\xb7\x5e\x28\x1f\x44\x60\xe9\xec\x99\x48\x67\xc7\xe3\xb4\xc2" + + "\x74\xdf\xf0\xa4\xd1\xd0\x75\xfc\xe2\xfd\x70\x7a\x30\xee\x38\x9d" + + "\x03\xe2\x7d\xf9\x11\x62\x98\xe1\xd2\x5d\x24\x97\x47\xab\x54\x43" + + "\xc0\x4a\xa4\x65\x69\x11\x5c\xbc\xf0\x63\x29\xb1\x6e\xd5\x9a\xe6" + + "\xe8\x39\x6f\xe3\x87\x1c\xc8\x45\x0f\xca\x8b\xe2\x92\x0a\xb1\x54" + + "\xa7\x01\x1e\xcb\x68\xcc\xd1\xbf\x85\xa0\x4c\xf9\x5b\x86\x9b\xbb" + + "\x40\xb4\x1c\x48\x53\xc0\xdc\x27\x5d\x15\x02\xb1\x14\x65\xbb\x05" + + "\x1c\xd2\x36\x10\x42\xb6\x9e\xf2\xa8\x2c\x2f\x2f\xf9\x4b\x73\xe1" + + "\x0c\x9e\xe4\x5f\x2e\x21\x90\x40\x63\xbb\x48\xc1\xb2\xc8\x90\xfe" + + "\x48\x1d\x6a\xb0\x18\x0c\x7d\xbd\xa8\x9b\x3e\x7a\x9e\x9d\x4d\x7d" + + "\x73\x4e\x86\x47\x28\xa2\x81\xde\xba\xb2\x31\x40\xa9\x64\x99\x87" + + "\xa2\xc4\x1b\x8a\x7f\x01\xea\x03\x27\x21\x1d\x48\x8e\xdb\x4e\xc6" + + "\x6c\x90\x44\x67\xaf\xdf\xdd\x15\xf3\x47\xba\x1b\x77\x08\xea\xbc" + + "\x5c\xe9\x10\x89\xff\x6b\x6f\xb1\x5c\xa6\xc5\x4c\x6e\x2f\xdd\x68" + + "\xbb\xe3\xae\x44\xf7\xa0\xfb\x3f\x5b\x97\x8e\x23\xfc\x72\xd5\xc8" + + "\xbc\x9b\x3b\x0d\x25\x66\xf2\xd3\xe0\x2c\x17\x6c\x9f\xa7\x2f\x24" + + "\x27\xeb\x83\x0a\x2e\x7a\xd3\xed\x6c\x94\x7b\x76\x4e\x4e\x5d\xed" + + "\x9d\x77\xfb\x4e\x96\x16\x6d\x4e\x02\x7d\x73\x73\x5d\x04\x88\xcf" + + "\x05\xb9\xc3\xfb\xc3\xb7\x46\xfd\x0f\x18\x2c\x16\x70\xc7\xfb\x7f" + + "\x36\x17\x6d\x0e\x22\x0c\x2f\x66\x6b\x0d\xfb\x55\xd4\xd5\xf5\x1a" + + "\xcc\x90\x26\x33\x5f\xd8\xdc\x5c\x3b\x9e\xf0\xf5\xfd\xef\x85\x5a" + + "\xd7\xb2\xd9\x18\x6a\xf2\xdd\xf3\x67\x8a\x87\x7f\xb1\x35\x38\x53" + + "\xfd\xfa\x94\x01\x5a\xfe\xda\xe2\x9d\x8e\x55\x90\xca\x01\x19\x72" + + "\xf2\xd3\xc3\x78\xae\xc6\x7c\x63\xda\x7b\xae\x0d\xcc\x3f\xe7\x4b" + + "\x8b\x74\xb5\x95\x13\x1d\x1a\x47\x95\xd5\x03\x9b\x95\x46\x38\xb2" + + "\xcf\x02\x5e\xb7\xbf\xce\xd7\xda\xfe\x10\x0b\x64\xa4\xfd\xd3\xac" + + "\xee\x3e\xc7\x3f\x3f\x5a\xc9\xb6\xdf\x20\x19\x1b\x0f\x3a\xa7\xa0" + + "\x59\x22\xba\xf4\xdc\x8e\x84\x56\xee\x23\xe8\x52\x56\xfb\xa4\xc8" + + "\xc9\x54\xf7\x9c\x2e\x49\x4c\xbc\x01\x17\x10\x17\xe7\x96\x22\xce" + + "\x62\x37\x5f\xc3\xb2\x5c\x2f\x3e\xc2\x67\x67\xd9\x1f\xf5\x76\x41" + + "\x57\x00\x4f\x37\xcb\x91\xdc\x01\xd3\x35\xd5\x9c\xe5\x3d\x8b\x94" + + "\x78\x39\x43\x4a\xde\x27\x82\xf7\x3a\x43\xb4\xe6\x62\xdb\x10\xa0" + + "\x63\xed\x50\xf7\x2b\x83\x5a\xc3\xcd\x8f\xff\x43\xe7\x50\x77\xe4" + + "\xfc\x3a\x76\x90\x85\x84\x41\x4c\x32\xe6\x82\xa8\x70\x58\xfa\x28" + + "\x42\x35\x29\x17\x17\x34\xc1\xf3\x89\xab\xa5\x38\x59\xa8\xfc\xe4" + + "\x65\x8e\x29\x1f\x27\x33\x0a\xeb\xb9\xbc\x7c\x4f\xb0\xc5\x6b\xea" + + "\x25\xbf\x52\xbb\x2c\x30\xcb\x4d\x18\xb9\x43\x88\x33\xd9\x10\xf7" + + "\x70\x91\xf6\xf9\x4b\xec\xef\x3f\x84\x1f\x58\x9b\x53\x3a\x09\xdd" + + "\x55\x22\xe3\x5f\xd9\x39\xba\x09\xbe\x2f\x5c\x29\x4b\xd3\xbd\x3f" + + "\x11\xf3\x50\x51\x16\x2e\x53\x2c\x2f\x77\x7b\x18\x19\xe9\x93\x51" + + "\xed\x8b\x47\x1d\x82\xa1\xd5\xaa\x42\x2d\x89\x91\x13\xe6\x12\x3b" + + "\xac\xee\x3d\x6c\x32\xac\xda\x0a\xb1\x23\x9b\xa0\x5d\xf7\x3d\x2d" + + "\xd6\x6d\x49\x01\xc2\xc2\xc2\xf3\xa5\x4a\x41\x49\x49\x98\xe8\x18" + + "\x18\x47\xff\xea\x28\xd4\xca\x55\xb7\xef\x57\xed\x87\xed\x3e\x4a" + + "\xe2\x24\xa5\xa5\xe1\xff\x0b\xd9\x72\xbd\x69\x5a\xf1\xc0\x91\x0d" + + "\xe1\x85\x5b\x8d\x02\x32\x5c\x5b\xca\xdf\xbf\x45\xa9\xa7\x9e\xcc" + + "\xab\xae\xb1\xa1\x75\x38\x6a\x55\xe7\xdf\xfe\x08\x10\x0a\x7a\xfd" + + "\xb1\x4a\x66\xd0\x5c\xb1\xcc\xd3\x64\xf2\xaa\x05\xfd\x9e\x2c\xde" + + "\xbe\x3b\x2e\x2c\x2a\xda\x76\xc9\x06\x0e\x78\xc9\xca\x22\x46\xe1" + + "\xb0\x7c\xdd\xca\x0b\x23\x36\x9a\x2b\x94\xea\xb0\x6d\x3e\x76\x75" + + "\x76\x4a\xb1\xeb\x48\x9a\x76\xd6\x4a\x8d\xe3\xfc\x23\x10\x8a\x4a" + + "\x36\xbe\x91\xc1\x01\x3d\x9e\xa2\x43\x0b\x87\x78\x06\x26\xf8\x78" + + "\x7a\xbd\x99\x9e\x7e\x5a\x53\x10\x7c\xfa\x5c\x02\xa2\x03\x6e\xf4" + + "\xfb\xeb\x1d\x83\x5b\x4f\xad\xc5\x11\xd0\x9e\xfa\x12\x92\xfd\x21" + + "\x82\x99\xa2\xae\x4d\xeb\xb6\xe6\xa4\xc7\x0e\x78\xbe\x67\xd2\x06" + + "\x86\x05\x76\xfa\x6c\xbc\xea\x25\xbb\xaf\x39\x86\x05\xf0\x63\x16" + + "\xf9\xb8\x42\x90\x38\xc0\x69\x5f\xfb\x1d\x3e\xad\xda\x79\x04\x64" + + "\xd6\x2b\xa2\x64\x08\xe2\xe8\x13\xf5\xd5\xf9\xbd\x6e\xe7\x1e\xaf" + + "\xfb\x1e\x2b\xf4\x8e\xdb\xb1\x2e\xaf\xac\x8c\x0e\x99\xe8\xc8\x05" + + "\xc0\x25\x53\xfc\x16\xe1\x87\xd9\x81\x04\x5c\x9a\x31\xcd\x64\x5c" + + "\x79\xbe\x66\x86\x65\xde\x3f\xe3\x3b\x43\x68\x2f\xe8\x2e\xa8\xb6" + + "\xb5\xb5\xb5\x06\xed\xdb\x9f\x24\xbe\x49\x11\x18\x34\xf5\x86\xfd" + + "\x5f\xbd\xee\x8e\xbf\xd0\x4a\x7a\xfa\x84\x41\xbe\x16\x30\xdc\x40" + + "\x44\x46\x3e\x34\xf1\x6c\xd9\x36\x32\xe6\xaa\xa5\xbe\x2e\x52\x57" + + "\xa0\x57\x41\xe9\x5f\x17\xb7\x3b\x1c\x9b\xd3\x64\x1c\xa7\x7c\x7d" + + "\xd9\x8f\x82\xe3\x01\xa5\xac\x44\x2e\xa1\xf5\xcd\xeb\x5b\xae\xdd" + + "\xfa\x1f\xb2\x80\x6f\x36\xa7\x98\xa7\xa6\xdf\x7f\x07\x86\x24\xfc" + + "\xae\xf1\x69\x3a\xa0\x79\x5e\xfe\x23\x25\x2c\x3c\xcd\x77\xd1\x89" + + "\xf1\x23\x00\xb7\xac\x3c\x99\x31\xdf\x80\xb6\xd9\x0e\x0b\x7d\xbe" + + "\x00\xc1\x4f\x67\x38\xb4\xaa\x83\x69\xb2\x51\xb1\x32\x4e\x79\xae" + + "\xd0\x90\x09\x90\xc2\x62\xa0\xbf\x77\x87\x7f\xb2\x58\x05\x4d\x4a" + + "\x59\xd8\x30\xcc\x8a\xac\xbf\x82\x4a\x89\xaa\x2f\xa6\xa8\xa8\x82" + + "\x48\x01\xac\x1a\x6e\xb0\x76\x6c\x0b\x67\x4d\xf1\x63\x6f\x96\xc5" + + "\xa6\xaa\x55\x74\xd2\xec\x74\x82\x56\x88\x79\x04\x64\x4e\x4a\x40" + + "\x0c\x9b\x94\xa4\x64\xae\x7a\x95\xf6\x7f\x4f\xac\x2b\xaa\xdb\xc7" + + "\xba\xc3\x72\xb7\x3e\xfe\xfc\x41\xcc\xcf\xcf\x2f\x2c\x1e\x94\x9d" + + "\xbf\x4b\x1a\x66\x7e\xd2\x14\xfd\x05\x44\x5e\xfd\x7a\x49\x2d\xef" + + "\xeb\x0e\xeb\x8a\x51\xae\xcd\xb0\xcf\x88\x99\x63\x66\x89\x8f\xc4" + + "\xcb\xef\x32\xd6\xbd\x50\xad\xa7\x13\x43\xc8\xa3\xfc\x3d\x4b\x5e" + + "\x44\x54\x34\x5d\xd0\xb7\xfd\x2f\xab\x61\x20\x60\xb5\x39\x2e\xee" + + "\xf9\xce\xd9\x19\x25\x5d\x99\xb8\x29\x5c\x5f\x77\xa6\x71\xcc\xd6" + + "\xf5\xad\x67\x27\xd4\x75\xe7\xe9\x19\x88\x33\xe6\xe3\x45\x89\x30" + + "\x28\xe5\x92\x3c\x44\x44\x22\xbe\x84\xa4\xca\xe9\x12\x64\x18\xec" + + "\xa6\xb4\xd0\x39\x8c\xd4\x9f\x05\xbf\x11\x26\x5c\xa3\x42\x5b\x3e" + + "\x8d\xdd\x77\x2f\x44\xb7\xfe\x90\x9a\x72\x0f\x33\xef\x93\x54\x4d" + + "\xa2\xf9\xa9\xb9\x9b\xdb\x76\x75\x72\x65\xd2\x1f\x7a\x11\xb6\x33" + + "\x64\xe7\x77\x7a\x0e\x3a\x7e\x0f\xeb\x35\x7e\xff\xb6\x6f\x68\xfc" + + "\x7b\x67\xbe\xaa\x2f\xc2\x6d\x3b\x1c\x5c\x0d\x0f\x69\xbd\xcc\xfe" + + "\x18\xca\x22\x14\x85\xac\xf4\xc3\x1b\xf2\x4f\x51\xc6\xfc\x54\x58" + + "\x2b\x52\xb1\xbd\x44\x80\x8c\xdf\x07\x86\xb8\xe6\xcd\x31\x76\x24" + + "\x8c\xf2\x98\x08\x3b\xaf\x3e\x47\x95\x36\x52\x49\x03\x5e\x71\x1b" + + "\x51\x40\x16\x5f\xce\xc3\x8f\xa1\x35\x68\x18\x6a\x7b\x96\xef\x34" + + "\x2c\xf6\x89\xe2\xb7\x09\xc2\x6c\xca\xc5\x13\x30\x4c\x9e\xcb\xc7" + + "\x72\x73\x66\x72\x4d\x5a\xb4\x5c\xf9\x2f\xc5\xcb\xf0\x5d\xf8\x12" + + "\xc7\x4c\x76\x0f\x19\xe3\xc5\x6f\x50\xe6\xd7\x98\x45\xf0\x91\xe8" + + "\xfc\xae\xab\x53\xbe\x7d\x1a\xdf\xa7\x03\x43\x8c\x1c\xd6\xe5\x12" + + "\x49\xc3\x0a\xb6\x2a\x00\x9e\x37\xa6\x83\x51\x3d\xde\x64\x2f\xdf" + + "\x26\x27\xc8\x72\x2f\x27\xd7\x06\x17\xa5\x79\x10\x06\x3f\x41\x1a" + + "\x12\xaf\x81\xce\x63\x89\x0e\xe4\xd2\xe0\xc7\x89\xd9\xf6\x32\x3d" + + "\x65\x60\x91\xe0\xea\x1b\xe7\x45\xea\x72\x85\xab\x8e\x43\x74\x3d" + + "\x5b\x26\xc9\xbd\xca\x6a\x4f\xb5\x6a\x88\x04\x52\x46\x4b\x36\xdf" + + "\x2d\xb4\x5c\x11\x06\x03\xd5\x35\x16\x59\x08\x9b\x0a\x68\x93\x2b" + + "\x74\xc4\x58\xb0\xce\xd5\x67\x60\x54\x8a\x3f\x28\x99\x2a\xec\xf2" + + "\xbe\xe7\x96\xd1\x93\xf5\x4e\x89\xa8\x0b\xbc\x59\x85\x15\x14\x8b" + + "\xdf\xca\x44\xcc\xcc\xc8\xd3\x4a\x47\x93\xc1\xc4\x12\x62\xa8\x98" + + "\xb3\xd4\x57\xb6\x87\x18\x5d\xa2\x17\xf9\x3b\x23\x90\xf6\xb6\x42" + + "\x26\x98\xfa\xdf\x31\xa7\xa7\x9e\xb8\x43\x29\x88\x0e\x3e\x2f\xb3" + + "\x8d\x4f\xf5\x0e\x0a\x36\xbd\x7d\x4f\xb9\xa4\x40\xae\x80\xf2\x72" + + "\x11\xe9\x37\x2b\x88\x8c\x7f\x51\x5c\x18\x83\x35\xfc\x30\xfa\xc8" + + "\xc3\xbc\x3a\xeb\x2c\x4a\x51\x6b\x71\xf8\x8d\xbd\x3e\x60\xf9\x10" + + "\xdd\xfb\x05\x06\x06\x9a\xad\x8f\xdc\x8b\x2e\xc0\x90\xb0\x67\xdb" + + "\xc3\x4a\x10\x76\xe3\xd9\x84\xb7\x57\xf8\xbf\x63\xa0\x9e\x4d\x32" + + "\x28\xe7\x4b\x11\x7f\xa8\xab\x63\xff\xca\x2f\x87\xe3\x64\x46\x28" + + "\xc0\x39\x64\xf9\xb9\x84\x23\x23\x0e\x27\x37\xe6\x6a\xba\x14\x41" + + "\x87\x43\x76\x3c\x5f\x69\x74\x66\xd5\x21\x66\x57\x63\x87\x7b\x2a" + + "\xe0\x90\x24\xfa\x33\x36\xde\x8d\x3f\x8a\x89\xef\xbe\xfe\xc3\x81" + + "\x9e\xef\x81\xd0\xe4\x22\xaf\xc0\x43\x80\x57\x3e\x2c\x2b\x8e\xd4" + + "\xbc\xac\xf2\x92\x3e\xf8\xd7\xcb\xd4\x5b\xe5\x66\x7d\xc3\x8c\xf3" + + "\xfe\xe4\xe7\x8b\xec\x39\xe6\x8f\x80\xa5\xc7\xdc\x5c\x89\xbd\x7d" + + "\xa2\x08\xee\xc9\x82\x0e\x88\xa3\x6b\x72\x16\xaa\xbd\x2f\x37\xf8" + + "\x64\x51\x3e\xa6\xd6\x82\x02\x81\x52\xd7\x45\xe3\x1a\xfb\x2c\xb1" + + "\x8b\xea\x33\x4b\xc5\x8e\xaf\xf2\x61\x40\x55\xac\x10\x59\x29\xbe" + + "\xa9\x68\x60\x0e\x93\xd8\xfb\x96\x77\xd1\xc4\x5b\xb7\x32\xcb\xc7" + + "\x0f\x0d\xd6\x54\x6b\x8a\x1e\x3b\x32\x11\x6c\xef\x4e\x1e\xdb\x1c" + + "\xcf\x37\xe3\xab\x63\x90\x2f\xe3\x81\x9e\x09\x04\xbe\xf7\xf6\xda" + + "\x21\x5c\xf9\x9f\xf9\xf9\xef\x1f\x0d\x1a\xd5\xb2\x0e\x0e\xac\x9e" + + "\x9e\x9e\x36\x9b\x5d\x44\x74\x54\xc3\x31\x44\xc6\xb3\x91\x48\x8c" + + "\xba\xe7\x15\xd9\x02\x85\x9b\xeb\x6b\x6e\xa2\x0c\xa7\x5c\x55\xaa" + + "\x71\x40\xab\x41\xdc\xb1\x9f\xa1\xce\x8c\x14\x20\x9c\x17\x7a\xb7" + + "\x12\xc3\xbd\x14\xaf\xa7\x54\x38\x12\xae\xef\xad\x50\xcc\x4b\xbf" + + "\xad\x94\xcd\x4e\xa1\x34\xbf\xdd\xe8\x05\x12\xba\x45\xf9\xe2\x80" + + "\xf2\x2d\xe8\x91\xf8\xf8\x27\xe6\xcb\x54\x8e\x16\x2a\x96\xe0\xd4" + + "\xf4\x14\xd4\xb6\x5d\x21\x83\xeb\xfa\xbf\xdd\x9d\xae\xb6\xa0\xc8" + + "\xa9\x68\x5d\xdf\x69\x06\x66\xdd\xcb\xc8\xbb\x91\x6d\x75\xf5\xda" + + "\xd7\x1a\x5a\xf7\x36\x66\x9c\x7d\xb9\xec\xd2\x5e\xd9\x37\xa0\xbe" + + "\x77\x3c\x38\xb0\x93\x18\xfb\x03\xe4\xd2\xd9\x1f\x06\xf0\x0a\x3c" + + "\x06\x1a\xf4\x32\xea\x12\xda\xac\xd7\xe1\x90\x0a\xbd\x7d\xa3\x41" + + "\xbd\x57\xdd\x93\x93\x94\xfc\x75\xb1\x01\xfc\x6f\x35\x72\xe6\xa4" + + "\x7c\xea\x07\xe2\x9d\x82\xad\x9e\x74\x5a\x71\xa1\x91\x4d\x9b\x88" + + "\xad\x06\x73\x14\xda\x06\x5b\x93\xe1\xea\x89\xc0\x25\xd9\x3f\xc9" + + "\xf1\x91\x04\xd3\xb6\xea\xe7\xc1\xd0\xa4\x84\xbd\xb7\xb0\xe9\x59" + + "\xa7\xe7\x0d\x39\x08\xd4\x1d\x1d\x1d\x15\x9a\x9a\x5c\x1d\xbf\x80" + + "\x98\x81\x08\x8c\x9f\xa5\xee\x6a\x68\x6f\x6d\xbc\x3c\xa3\x0e\x8b" + + "\x50\xc9\xc8\xe4\x8c\x01\x8e\xab\xa2\xfd\xac\x95\x73\x7d\x38\xd1" + + "\x84\x3c\x6d\x76\xaa\x6a\x02\x35\xce\x44\xc8\x30\x89\xd4\x72\x46" + + "\x2b\xee\x51\xd2\x83\x37\xf3\x1d\xf9\xda\x79\xf7\x96\x4b\x9a\x62" + + "\x2c\xa6\x16\x73\x26\x64\x3c\x3c\x0c\x2a\x6a\x4d\xe7\x72\x80\xcd" + + "\x09\x1d\xab\xba\x4b\xf2\x41\x70\xc5\xc7\x5a\x8c\x21\xee\x44\x16" + + "\xdf\x43\x27\x7f\x60\xa6\x5d\x3b\x1c\x1d\x15\xf4\x4f\xd9\xd6\x3d" + + "\x32\x8d\xa0\x95\x10\xf7\x3a\xcf\x20\x48\xed\xb0\x55\xea\xb8\x0b" + + "\x70\xe9\xc8\x20\xee\xd6\xe4\x60\x52\xf9\xdc\x1c\xa2\x94\xc4\x83" + + "\xaf\xb4\xeb\xbc\xe0\x44\xd5\x30\x21\x04\x4d\xc6\xc6\x96\x30\x95" + + "\xc5\x87\xf3\xd4\x59\x59\xc9\x68\xb5\x50\x1e\x82\x1a\x38\x96\xec" + + "\x8b\xa0\xa6\xfe\x5a\xb7\x9f\x10\xa6\xab\x84\x66\x18\xc2\xe4\xf5" + + "\x05\x72\xd1\x6c\xf2\x06\xe9\x17\x7d\xdd\x0b\xb0\x5b\x43\x1b\xd3" + + "\x52\x20\x0e\xa6\xfa\x54\xd6\xeb\x6b\x7e\x97\x93\xc5\xb3\xbf\x53" + + "\x70\x08\x08\xc6\x03\xe1\x68\x03\xc3\xc3\xd0\x97\x10\x65\xda\x5d" + + "\x40\x0a\x6e\x6d\x84\x2e\x2b\xb6\xdc\x99\xc8\x69\x8a\x9f\xc3\x40" + + "\x86\x9b\xc5\x9e\x95\xa1\xab\xaf\x27\x4e\xe9\xed\x8d\xb6\x36\x35" + + "\x03\xa7\xbd\xe9\x7b\x6d\x8e\xb1\x68\x88\xc4\x29\xd7\x83\x7b\x94" + + "\xdc\x4e\x79\x46\x1e\xaf\x76\x21\x92\xa6\x63\x0f\x4c\xde\xdb\x33" + + "\xa1\x56\xc8\x96\x5d\x6d\xb6\x8d\x57\x4e\xc8\xda\xed\x2a\x54\x5f" + + "\x67\x95\x25\xef\x17\x45\x78\x4b\x50\x5e\x85\x14\x93\x84\xdd\x79" + + "\x28\x51\xa4\xf2\xf9\x3a\x96\x75\x4b\xf8\xfa\x0b\xb2\x8e\x3a\xf0" + + "\x33\xea\x9e\xcb\x7f\x6b\x9e\x7d\x72\x82\x02\xa7\x24\x8f\xfd\x45" + + "\x61\x85\x25\x25\x61\xff\x53\xe8\xad\x9e\xdf\x9d\x4f\x07\x99\xa5" + + "\x91\x22\x16\xb5\x37\x09\x5d\x0c\x35\x1a\x25\x56\x83\x7d\x12\x6b" + + "\xab\x53\x90\x91\xc2\x2c\x8b\x91\xbe\xee\x4b\x3e\x0c\xeb\xc7\x87" + + "\xa9\xcc\x31\x99\x61\x27\x62\xe1\x88\xe4\xc2\xf6\x9e\xe6\xa4\x34" + + "\x78\xe7\xd8\x5b\x9d\x85\x12\xd8\xe5\xfe\x8c\x25\xfe\x6f\xe9\x4c" + + "\x6b\xeb\x77\x7c\x2e\x1b\x49\xa0\xd7\x9d\xc7\x65\x2f\x8c\x56\xbd" + + "\xc9\xa8\x25\xdc\x66\x9b\xb3\x79\x7d\x7d\xbd\x66\xed\x2f\x28\xf4" + + "\x88\x60\x19\xeb\x59\xb9\xc4\x01\x2f\xb8\x90\x9c\x7f\xd9\x69\x8f" + + "\xe9\xe2\xb7\x73\x88\x99\x55\x82\x91\xe8\x77\xc2\xd9\xc3\x14\xbc" + + "\x82\xac\xc1\x14\x65\x47\x29\xa6\xdd\x00\x58\x32\x49\x48\x69\x1d" + + "\xa2\x74\x9b\x8d\xc7\x00\xf9\xf9\x9e\x31\x82\x77\x28\xf6\x03\xbf" + + "\x20\xa4\x09\x5d\xf0\xc9\x3e\x38\x1e\x0f\x0f\x2b\x75\x48\x53\x0e" + + "\xc7\x12\xed\x37\x4c\x0b\x16\xe8\x75\xb8\xad\xf9\x3e\xac\xbf\x3f" + + "\x9b\x86\xd4\xd9\x2c\xb3\x7d\x80\x61\xce\x0f\x15\xe9\x37\xa6\x89" + + "\x5a\x1e\xcd\x17\x28\xd3\xd4\xde\x5f\x6f\x7e\xcf\xbd\xe9\x33\x2e" + + "\xdc\xaa\xf8\xd1\x78\xec\x5a\xab\x57\xd0\x76\x7d\xd8\x60\xb6\x31" + + "\xa3\x2c\x7a\x65\x38\xb7\xec\x55\x70\xd9\x14\x5c\xc4\x39\xae\xad" + + "\x9e\x7d\xed\x7a\xdc\x64\x99\xaf\x50\xcb\x7a\xca\xe9\x12\xb1\xf2" + + "\xc2\xe6\x92\xa3\xe2\xa0\x84\x8d\x1d\x8e\xb2\x94\x46\x04\x51\x8e" + + "\x58\x9a\x1f\xa6\x4d\x3f\x9a\x65\x4f\xe7\xd3\x50\x53\x5f\x0f\xe2" + + "\x18\xc4\xa5\xa5\xc1\xf6\x6c\xf7\x87\x95\x69\xd6\x91\x47\x92\xe3" + + "\x72\x9f\x3a\xae\x34\x59\xe9\xd2\x2f\x0a\xfd\x6d\x6d\x1b\x6c\x46" + + "\xec\x5c\x82\xd0\xc0\xfe\x4c\x63\xfb\x7c\xf9\x9d\xf1\xcc\x62\x15" + + "\xa7\x93\xf0\xd8\xfb\x83\x3b\x96\xef\xc5\x51\x70\xb5\x4e\xc7\xda" + + "\xd4\x42\x46\x4b\x37\x18\x4f\xf9\xe6\xe6\x48\xa3\x0a\x77\xfc\x21" + + "\x27\x7a\x82\x84\x0e\xc7\xfc\x3b\x8a\xba\xed\xbf\xa9\x91\x87\x2e" + + "\x93\xe9\xdf\x14\x80\xef\x84\xc8\xa7\x6e\x7c\x5a\x5a\x5a\xc1\x21" + + "\x5f\x6f\x06\x57\xa7\x95\x81\xca\x36\x65\x5a\x93\x1e\x8c\x02\x89" + + "\xf7\x0f\xfc\x2a\x5b\xb8\xf6\x5f\x1b\xf2\xc4\x6f\xc7\x25\x94\x44" + + "\x05\x93\xd8\x4a\x58\x7a\xa2\xa0\xbd\x40\x6f\x29\xd6\xb5\x9d\x8f" + + "\xf9\x3a\xce\x01\xd4\x59\xc4\x14\xc5\xba\x9f\xf3\xc8\x7a\x27\x5e" + + "\x6e\xb1\xae\xf9\x37\xc8\xe3\x55\xa4\xa5\xe2\x40\xbd\x8d\x6d\x51" + + "\xe9\x82\xf6\x74\xc5\x49\x10\xfc\x2a\x7e\x5b\x90\xc3\x7d\xdf\x03" + + "\x9a\x98\x81\x53\xca\xa9\xc6\xb7\xf3\x7a\xe7\x3b\xef\xbb\x3e\xe5" + + "\x5f\x7c\x69\x68\x87\x0c\x3d\x1e\xb2\x09\xb3\x31\x16\x90\xf6\x72" + + "\x24\x49\x3f\x2c\xae\x23\x42\xff\xc9\xa5\x3f\xf7\x2c\x2b\xe8\xe9" + + "\x72\x41\x98\xc3\xd6\xe3\x4b\xa8\x65\xfe\x09\x4e\x0a\xbf\x88\x5a" + + "\xb2\x72\x87\x2c\x0f\xc7\x3f\xa9\xa0\x4c\x62\x6a\x96\x69\x04\x8b" + + "\x0a\x61\x7b\xa5\x8c\xcd\x58\xb5\x90\x5a\xd4\xd3\xfb\xe7\x52\xab" + + "\xbd\xb8\x40\xe4\xcb\x6d\xf0\xd0\x4e\xad\xf5\xe2\x96\x2e\x5a\x2d" + + "\xfb\x13\xea\x6e\xa8\x80\x7c\x01\x2f\xe7\xa5\x87\x2b\xe2\x6e\x5f" + + "\xa2\x78\xd2\x01\x10\xd6\x93\x98\x25\xc2\x75\x07\x41\x5b\xfb\x58" + + "\x63\x18\x02\x6b\x9f\x63\xcd\x11\x42\xe3\xaf\xbf\x94\xf2\x01\x90" + + "\x89\xe4\x76\xe3\xc1\x43\x79\xb4\x29\x69\x53\xa1\xbc\x70\x6a\x16" + + "\x1b\xad\xf2\x78\xd7\xbf\x64\x77\x8c\xc6\xc1\x61\x3e\xf7\xfb\xd8" + + "\xb3\x2c\xa2\x16\xda\x13\x14\x5f\x72\xfc\xf5\x1a\x22\xf4\x20\x89" + + "\x2f\x52\xe7\x50\x1e\x1f\x12\x78\x15\xb2\x77\x79\x59\xc1\x94\xff" + + "\x3f\x00\x5b\x14\xa4\xeb\x5f\x76\xbd\xb5\x7e\x78\x11\x84\xff\x5f" + + "\xc1\x9b\x7c\xee\xf5\x4f\x2c\xa1\x58\x38\x2d\x84\x50\xc3\x32\x33" + + "\x7f\xb3\x88\x7e\xb9\x41\x55\xeb\x02\x99\xf7\x0a\x61\x13\x00\xdb" + + "\x76\xed\xc3\xe5\xf6\x58\x82\xb1\xf0\x8f\x2d\xcf\x9f\x25\xf0\x7d" + + "\xf4\x6a\x36\x5b\x43\x8b\xe8\x97\x1b\x0c\xe3\xaf\xfc\x4f\x11\x37" + + "\x14\xfc\x36\x7c\xdc\xe7\xd6\xa0\x9c\x85\x13\x88\x6e\xf3\x25\xd2" + + "\x10\x40\x13\xaf\x77\x93\x45\xf4\xcb\xf7\x65\xa6\x2e\x00\xc7\x34" + + "\x40\xfe\xf2\xdb\x5a\x1c\x76\x9b\x45\x76\x0b\xa7\x70\xdd\x83\xbe" + + "\x2d\xed\x6d\xb4\x88\x7e\xc6\x17\x18\x8d\xcd\x16\x3b\x08\x10\x86" + + "\x61\xf2\xf3\xaf\xeb\xac\x3c\x73\x16\xf2\xc3\x3b\x85\xb4\xe3\x59" + + "\xf9\x06\xc1\xdc\x62\x11\xfd\xf2\xb6\xe8\x78\xbd\xfb\x0e\x0b\x11" + + "\xf5\x05\x20\x9f\x79\xed\x23\xcb\xa2\x5b\x00\x7c\x05\x29\x0e\x1e" + + "\x4d\xf3\x91\x48\x09\xca\xb0\x88\x7e\xb9\xbf\x50\xad\x0e\x52\xf2" + + "\x28\x20\xf2\x3c\x5e\x3c\x5e\xaf\x25\x14\x0b\x18\x86\x49\x79\x7f" + + "\x4a\xa9\xb0\xdd\x16\xd1\x2f\x73\xe8\xfa\x36\x20\x3d\x57\xd3\xe2" + + "\x46\x03\xf2\xf5\x0f\xbe\xb6\x84\x62\x01\x21\x04\xde\xfc\x4d\x4f" + + "\xa6\x99\x61\xad\x8c\x0b\x04\x44\x46\xb6\x41\xd7\x53\x5e\x06\x21" + + "\x66\xfe\xf8\x9b\xb4\x6a\xb7\x59\xd0\x0d\xa3\x50\x21\x87\x90\x83" + + "\x16\xd1\x03\x00\x19\x19\xcb\x00\xc3\x25\x44\xf8\x1b\x80\x70\xb9" + + "\xf3\x2c\xa1\x58\x16\x9d\x7f\xf4\xc0\x65\x59\xf4\x40\x81\xaa\x56" + + "\x05\x8e\xbf\x00\xc8\x29\x33\x7e\xb1\x76\xb4\x95\x75\x7d\x50\x04" + + "\x19\x59\x39\x00\x68\x9a\xd2\xc9\x22\x7a\x80\xc0\x30\xf6\xa1\xaa" + + "\x09\x06\x08\xf9\xc9\xb4\xf9\x64\xe5\xb8\x2c\xa1\x94\xe9\x8e\x5f" + + "\xe5\x58\x7a\x26\x80\xd4\xf5\xf4\x04\x8b\xe8\x81\x14\x97\xe9\x3b" + + "\x25\x04\x2d\x2e\x70\xdd\x2c\x94\x5d\x98\xa6\x49\x48\x90\x03\x40" + + "\x80\x50\x2d\xa2\x07\x18\x14\x25\xe4\x13\x00\xb7\xc7\x63\xcd\xa9" + + "\x97\x61\x78\x75\x83\x6a\x95\x63\x0b\xfc\xbd\x3a\x16\xd1\x03\x0c" + + "\x76\x7b\xad\x9f\x01\x16\xaf\xd8\x80\xa2\x58\x56\xbd\xac\x42\x4a" + + "\x89\xd3\x6e\xcf\xff\xa6\x25\x5b\x44\x0f\x20\x04\x05\xb5\xc6\xed" + + "\x5e\x9e\x0a\x98\x6f\x7d\x32\xdd\x4a\x48\x51\xa6\x3d\x3b\x85\xcc" + + "\xec\xdc\x02\xfb\x1e\x65\x11\x3d\x80\xe0\x72\xf9\x13\x89\x18\xb9" + + "\xae\x3c\x0e\x1e\x39\x66\xb9\xef\x65\xd8\xa2\x87\x85\x06\x03\x20" + + "\x84\x23\xd3\x22\x7a\x60\x42\x07\xe4\xec\x9f\x57\xa0\x5a\x56\xbd" + + "\xcc\xc2\x34\xcd\x7c\xd2\x7b\x6b\x5a\x44\x0f\x30\xd8\x6c\xf5\x01" + + "\xc7\x6e\x40\x4c\xf8\x62\x0e\x96\x3d\x2f\x9b\x10\x80\xc7\xab\x17" + + "\x7c\xb5\x36\xb5\x04\x1a\xbc\xde\x2d\x80\x3a\xaf\xe0\xfb\x5f\x3b" + + "\xf7\x59\xee\x7b\x59\x24\xba\x22\xc8\x75\xb9\x01\x24\x08\xdd\x22" + + "\x7a\x00\x42\x51\x42\x56\x17\x7c\x9e\xf5\xe3\x6f\x68\x9a\x6a\x09" + + "\xa5\x2c\xc6\xe9\x7e\xe3\xae\xae\xb4\xd9\xea\x58\x44\x0f\xbc\xd8" + + "\x2c\xc5\x9f\x51\x64\xea\x77\x8b\x30\x4d\xcb\xa2\x97\xb9\x10\x4e" + + "\xd3\x48\xde\xe7\x2b\x1f\x25\x84\xb6\xdf\xeb\xdd\x66\x11\x3d\x40" + + "\x9d\x37\xff\xce\x96\x1d\x7b\x0e\x5a\xe2\x28\x6b\xd6\x5c\x4a\xbf" + + "\x27\x27\xa5\x5e\xbd\xb4\xb7\xd7\x22\xfa\xf9\x23\xa7\xe0\xc3\xae" + + "\xbd\x87\x2c\x69\x94\x31\xa8\xaa\x42\x46\x66\x76\x81\x45\xaf\x66" + + "\x11\x3d\x00\xe1\x74\x5e\x89\xa2\x84\x7d\x56\xf0\xfd\xcf\xbf\x76" + + "\x59\xd3\x6c\x65\xd0\xa2\x3b\x1d\xb6\x7c\xa2\xab\x5b\x2d\xa2\x07" + + "\x20\xdc\xee\xb5\x98\x66\xe6\x53\x05\xdf\x97\xfe\xbe\xd1\x1a\x90" + + "\x2b\x83\x31\x7a\x41\x05\x1f\x29\x75\x6b\x7a\x2d\xc0\xe3\xf4\x14" + + "\x80\x3d\x07\x8e\x94\xea\x82\x80\x16\x8a\x1f\x3b\xf6\x1c\x44\x9a" + + "\x12\x10\x48\x99\xfb\x91\x45\xf4\x80\x86\x7d\x6a\xc1\xa7\xd5\x1b" + + "\xb7\x5b\xf3\xe9\x65\x08\xbf\xaf\x2f\x28\xf2\xa1\xae\xbd\x1c\xda" + + "\x6b\x11\xfd\x02\xa0\x69\x15\x37\x14\x7c\xfe\xee\x97\x15\x96\xfb" + + "\x5e\x86\x90\x96\xe1\xcb\xe9\xae\x28\x36\xc5\x22\x7a\x80\x43\xd7" + + "\x93\x3f\x2d\xf8\xbc\x69\x7b\x32\x36\x8b\xe8\x65\x02\x52\x4a\x92" + + "\xf7\x1f\x2e\xf8\xb8\xdd\x22\x7a\xd9\x88\xd3\x0d\x80\xdd\x7b\x0f" + + "\xa3\x69\x9a\x25\x8e\xb2\xe0\xc9\xa9\x2a\xfb\x0f\xa7\x02\x08\x90" + + "\x3b\x2c\xa2\x97\x09\x28\x7b\x00\xbc\xba\x5e\x28\x6e\xb3\x10\xd0" + + "\x5d\xbb\x10\xe8\xf9\x39\xdd\xa5\x94\x1e\x8b\xe8\x65\x03\xfb\x0a" + + "\x3e\x04\x39\xec\x96\x34\xca\x42\xd7\xae\x08\x72\x7c\x1b\x5a\x00" + + "\x8e\x58\x44\x0f\x70\x44\x46\xde\x03\xd8\xfd\xd9\x28\x72\x5d\x56" + + "\xbe\xf7\x32\xe1\xba\x6b\x2a\xa9\x69\xc7\xf3\x49\x1f\xbe\xd3\x22" + + "\x7a\x80\x23\x23\xe3\x4b\x84\x70\xfc\x56\xf0\x3d\x38\xd8\x69\x09" + + "\xa5\x2c\xbc\xf7\xcc\x1c\xd2\x8f\xfb\x96\xbf\x9a\xa6\xf8\xcb\x22" + + "\x7a\xd9\x40\x70\xc1\x87\xb4\xf4\x4c\x4b\x1a\x65\x00\xfb\x0f\xa5" + + "\x14\x72\xe3\x8d\x04\x8b\xe8\x65\x22\x5e\x8b\x5f\x5a\xf0\xf9\x70" + + "\x6a\xba\x25\x90\x00\x87\x94\x92\x95\xeb\xfe\x59\xda\xae\xaa\xc1" + + "\xe9\x16\xd1\xcb\x00\x0c\x63\x9d\x7f\x30\x26\x24\xc8\x72\xdd\x03" + + "\x1d\xaa\xaa\x30\x67\xc1\x4a\x3f\x7d\xbc\xde\x7d\x1b\x2c\xa2\x97" + + "\x19\x88\xe3\x00\xbb\xf6\x59\xdb\x55\x03\x1d\x76\x4d\xe3\x68\xaa" + + "\x6f\x0f\x8b\x10\xda\xee\xcb\xa5\xdd\x16\xd1\x8b\x07\x39\x00\x91" + + "\xe1\xa1\x96\x24\x02\xbd\x4b\x57\x04\x1e\xaf\xb7\x80\x3e\x0b\x2d" + + "\xa2\x97\x2d\x84\x02\x24\xd5\xab\x69\x49\x22\xd0\x5d\x77\x45\x25" + + "\xdb\x5f\x60\xd3\x31\x23\x32\xb2\xbd\x45\xf4\xb2\x03\x25\x03\xa0" + + "\x7c\x54\xb8\x25\x8a\x00\xc7\xcc\x9f\xfc\xb3\xa9\xd8\x6c\x6d\x96" + + "\x65\x64\x2c\xb0\x88\x5e\x86\xb0\xc7\x12\x41\xe0\x43\x4a\xc9\x8c" + + "\xef\x97\xe5\x7f\xd3\xf0\x78\xe6\x66\x5b\xae\x7b\x19\xd3\x01\x80" + + "\x4a\x15\xca\x5b\x92\x08\x64\xb7\x5d\x55\x0a\xcd\xa1\xcb\x23\x97" + + "\x53\xdb\x2d\xa2\x17\x0b\x7c\x09\xfc\xad\x2c\x33\x65\x20\x3e\xcf" + + "\x2d\x58\xe3\xae\x6e\xb7\x88\x5e\xd6\x68\x2e\x94\x63\x60\xad\x75" + + "\x2f\x0b\x8e\x9b\xf0\x55\xca\x96\xaa\x1a\xfe\x8d\x45\xf4\xb2\x17" + + "\xbb\xe5\x01\xa4\x67\x66\x5b\xc2\x08\x60\x78\xbc\x7a\xc1\x1a\x77" + + "\x61\x9a\xae\xed\x35\x6a\xbc\x69\x11\xbd\x8c\x39\x75\x87\x7c\xae" + + "\xbb\x61\x89\x22\x70\x3b\x73\x3e\x9a\x3a\xdf\xff\x5d\xd3\xe2\xff" + + "\x48\x4e\x7e\xda\x22\x7a\x19\x53\x83\x4a\x00\x79\x1e\xaf\x25\x8a" + + "\x00\x85\xa2\x08\xa6\x7d\xb7\x28\xff\x73\xe8\xef\x5e\xef\xa6\x63" + + "\x96\xeb\x5e\xe6\x60\xda\x7c\xbd\xbe\x25\x89\x80\xf5\xd9\x14\x15" + + "\x97\xdb\x97\x4c\xc6\x34\x8d\xff\xbb\xec\x3a\x2a\xeb\x15\x16\x1f" + + "\xc2\x42\x82\x2c\x21\x04\x28\x32\xb3\x73\x70\xe7\x79\x00\xa4\xa2" + + "\x38\x0d\x8b\xe8\x65\x10\x42\xd8\x36\x01\x6c\xdb\xb5\xcf\x12\x46" + + "\x80\x62\xef\x41\xff\x1e\x74\x21\xa5\xfc\x31\x28\xa8\xbd\x45\xf4" + + "\x32\x17\xa1\x4b\xaf\x0e\xe0\x74\x38\x2c\x61\x04\xe4\xfb\x95\xfc" + + "\xb8\x74\x0d\x80\x14\xc2\x81\x94\x19\xbb\x5c\xae\x05\x16\xd1\xcb" + + "\x12\x9c\xce\x26\x08\xa1\xd4\x05\xc8\xc9\x75\x59\x02\x09\x48\x8f" + + "\x4d\xf0\xd5\xec\x05\x00\x42\x55\xa3\xae\xbc\x1c\x9f\xc1\x22\xfa" + + "\x05\xc2\xed\x5e\x87\x94\x66\x1c\xc0\xf1\xac\x1c\x4b\x20\x01\x0a" + + "\xaf\xee\x5b\xf5\x68\x9a\xca\x2e\x8b\xe8\x65\x16\x46\x18\x80\x69" + + "\x0d\xbb\x07\xe6\xdb\x35\x0d\x4c\xd3\x47\xf4\xb0\xb0\x17\xb3\x2d" + + "\xa2\x97\x5d\x54\x01\xc8\xca\xce\xb5\x24\x11\x80\x18\x37\x69\xa6" + + "\x3f\x5c\x3f\x7e\xfc\x31\x1d\x5f\xa2\x91\x74\xb0\xfd\xa5\xaa\x71" + + "\x8f\x17\xfc\x31\x2c\xac\x93\x45\xf4\x00\x8f\xe2\x2a\xfa\xdc\x3b" + + "\xc3\xaa\xa8\x1a\x40\x90\x52\x62\x9a\x92\x2f\x66\xfe\x02\xc0\xb1" + + "\x63\xc7\x84\xcb\xe5\xe2\xd8\xb1\x63\xc1\x9b\x36\x6d\x8a\x7c\xfe" + + "\xf9\xa7\xea\x54\xae\x6c\x7b\x1f\x90\xa0\x1e\xc8\xc9\xd9\xdc\xcc" + + "\x22\x7a\x60\x3b\x77\x1a\xc0\xd1\xd4\x74\x14\x45\x58\xe2\x08\x10" + + "\x28\x42\x30\xe6\xe3\x6f\x01\xa8\x53\xa7\x0e\xe5\xca\x95\xc3\xe9" + + "\x74\x52\xae\x5c\x39\x1a\x34\x68\xc0\xab\xaf\xbe\xc6\x9e\x3d\x7b" + + "\x59\xb8\x70\x21\x51\x51\xe1\x15\x4d\x73\xcf\x6a\x21\x22\x3e\xb2" + + "\x88\x1e\x80\x08\x09\x79\xc1\xbf\x4a\x26\x33\x3b\x17\x45\x58\x22" + + "\x0d\x14\x6b\xee\xd1\x0d\xbe\x9a\xed\x4b\x0b\xb7\x68\xd1\xa2\xd3" + + "\x1e\xdb\xae\x5d\x3b\xd2\xd2\xd2\x44\xd7\xae\xb7\x49\x29\x8f\x3f" + + "\x2c\x44\xd4\xbd\x10\x61\x11\x3d\x90\x90\x93\x33\xca\x05\x3e\x2b" + + "\xde\xb4\x51\x2d\xff\xa0\x8d\x85\xcb\x3c\x18\x13\x82\x8d\xdb\xfe" + + "\x49\xf2\x1a\x1d\x1d\x7d\xd6\x8e\x61\xe6\xcc\x99\xa2\x53\xa7\x4e" + + "\x52\xca\xf4\xcf\x21\xf4\x0a\x80\x7a\xf5\xf6\x5b\x44\x0f\x34\xdc" + + "\xd9\xa9\x2d\x56\x84\x1e\x18\x30\x4d\x49\xdf\xe7\xdf\x06\x60\xf0" + + "\xe0\xc1\xd8\x6c\xb6\xb3\x76\x0c\x52\x4a\xe6\xce\x9d\x2b\x7a\xf7" + + "\xee\x2d\xe1\xc0\xdf\x76\x7b\xdd\x26\x5b\xb7\x56\x29\x1d\x1d\x97" + + "\xf5\x4a\x2f\x0c\xb1\xb1\x52\x39\x7a\x54\x18\x03\x1e\xe8\xca\x43" + + "\x3d\x6f\xb1\xa6\xd8\x02\x04\x1b\xff\xda\xcd\xbd\x83\x47\xfb\xad" + + "\xb5\x94\x12\x21\xc4\x59\xad\xba\x10\x02\xb7\xdb\x4d\x50\x50\x10" + + "\xe0\xdc\x07\xee\x6a\x96\x45\x0f\x00\x1c\x3d\x2a\x4c\x10\x39\x6e" + + "\x8f\xd7\xb2\xe6\x01\x12\x9b\x0b\x21\x18\xf4\xca\x87\x00\xf4\xe9" + + "\xd3\xc7\x6f\xb1\x8b\xe2\xee\x3f\xf5\xd4\x53\x48\x29\xb9\xe7\x9e" + + "\x7b\x24\xb8\xab\xda\x6c\x8d\x15\x8b\xe8\x81\x83\x90\x3f\xb7\xee" + + "\xc2\xa6\xa9\x96\x24\x2e\x73\x92\xab\xaa\xca\x07\xff\xf7\x1d\xa9" + + "\xe9\xbe\xb2\xc8\x93\x26\x4d\x2a\xf2\x94\x69\x66\x66\x26\x63\xc7" + + "\x8e\xe5\xa6\x9b\x6e\xa2\x79\xf3\xe6\x02\x40\xd7\x77\x97\x8a\x3a" + + "\x5d\x9a\xf5\x7a\x8b\x27\x02\xda\xb6\x6b\x3f\x9a\xaa\xe2\xf1\xed" + + "\x6f\xb1\x70\x19\x42\x51\x04\x3f\xff\xba\x96\x0f\xbf\x9c\xeb\xff" + + "\x6d\xf5\xea\xd5\x34\x6f\xde\xbc\x88\xe7\xfb\xec\xe6\xb2\x65\xcb" + + "\x58\xb1\x62\x45\x7e\xe7\x51\x3a\x92\x91\x58\x16\xbd\x78\x88\xbe" + + "\x49\xd3\x54\x2b\x3a\xbf\xbc\xed\x39\x19\x99\x39\x3c\x35\xf2\x43" + + "\xdf\x97\x7c\xb4\x68\xd1\x82\x15\x2b\x96\xf3\xfb\xef\xbf\xcb\x23" + + "\x47\x8e\xe0\x76\xbb\xfd\xd6\xff\xdf\x08\x0d\x0d\x65\xce\x9c\x39" + + "\xf8\x2c\xb9\x9e\x4f\xfe\x60\xaf\x45\xf4\x80\xb1\x04\xce\x55\x29" + + "\xc7\x32\x84\xaa\x58\xe2\xbc\x5c\x5d\x76\xbb\xcd\xc6\x7f\x9e\x19" + + "\x5b\x40\x72\x21\x44\xe4\x04\x40\xa8\x6a\x8b\x2b\xae\xb9\xe6\xe6" + + "\xd5\x2d\x5b\xb6\x14\x71\x71\x71\x04\x05\x05\xb1\x61\xc3\x86\xd3" + + "\xc6\xec\x2d\x5a\xb4\x38\xe1\xbb\xc3\x11\x67\x58\x44\x0f\x00\xf4" + + "\xe9\xe3\x01\xec\xc9\x00\x07\x8f\x1e\xb3\x04\x72\x19\x42\xd3\x54" + + "\x9e\x7b\xe3\x53\x76\xec\x39\x00\x20\xec\xf6\xaa\xe3\xa5\xcc\x78" + + "\x02\xc0\x30\x56\xed\x34\xcd\xac\x16\x36\x5b\xfd\x56\xf9\x9d\x80" + + "\x4c\x4a\x4a\x22\x39\x39\xf9\x24\xab\x2e\xa5\xc4\xe5\x72\x15\x74" + + "\x16\x7a\xb9\x72\x37\x56\x70\xb9\x36\x9b\x16\xd1\x03\x00\x9f\x7e" + + "\x6a\xc7\x34\xd3\x5f\x01\x58\xb5\xfe\x2f\x6b\xad\xfb\x65\x18\x97" + + "\xbf\xfe\xc1\xd7\xcc\xf9\x79\x39\x00\x36\x5b\xe5\x17\x3c\x9e\x7d" + + "\xfd\x63\x62\x5e\x3e\xe1\x38\xaf\x77\xcb\x4a\x45\xa9\x68\x53\xd5" + + "\xc8\xdf\x01\x5a\xb6\x6c\x79\x92\x55\x17\x42\xd0\xb6\x6d\x5b\x00" + + "\x61\xb3\x55\x79\x35\x2d\xed\xe7\xa3\xe5\xcb\x97\x8e\x8d\x2e\x16" + + "\xd1\x8b\xcf\x2e\xe4\x2e\x5f\xbb\x05\x55\xb5\x44\x7a\xb9\x40\x55" + + "\x15\xc6\x4e\xfa\x1f\x5f\xce\x5a\x90\x4f\xf2\x2a\xaf\x7a\xbd\x07" + + "\x5e\x85\x1a\xa4\xa4\xbc\x74\x0a\xcb\x5f\xdd\x30\x8c\x8c\x56\x42" + + "\x38\x57\x1c\x39\x72\x84\x97\x5e\x7a\xe9\x84\x78\x7d\xc5\x8a\x15" + + "\xec\xdd\xbb\x57\x82\x3d\xc3\xeb\xdd\xff\x12\xc0\xb1\x63\xf3\x2c" + + "\x78\x4e\x71\x41\x00\x00\x0a\xa5\x49\x44\x41\x54\xa2\x07\x12\x84" + + "\x50\x56\x6d\xdb\xb5\x4f\xda\x34\x6b\x22\xe3\x72\x80\xcd\xa6\x31" + + "\x6c\xec\x14\x3e\xfb\xf6\x47\x40\x20\x44\x85\x36\x5e\xef\xfe\x17" + + "\xaa\x55\x7b\x1a\x48\x3e\xe5\x39\x1e\xcf\x4a\x14\x25\x1e\x29\xdd" + + "\xad\x35\xad\xfc\xac\x11\x23\x46\x70\xfb\xed\xb7\xb3\x70\xe1\x42" + + "\x9a\x35\x6b\x26\x5b\xb7\x6e\x0d\x38\x76\x85\x85\x5d\x5d\xd1\xe9" + + "\xbc\xb6\x74\xe9\xa7\xf5\xca\x8b\x8b\xe8\xa1\x0f\x4a\x99\xfd\xe9" + + "\x86\xef\x27\x62\x9a\x66\x91\x16\x58\x58\xb8\x34\x08\x72\xda\xb9" + + "\x7b\xe0\x6b\xfc\xb1\xf1\x6f\x09\x8a\xb0\xd9\xaa\xb6\xf3\x7a\xf7" + + "\x2c\x2e\xea\xf9\x15\x2a\xdc\xcd\x91\x23\x5f\xa1\x69\x71\x8f\xeb" + + "\x7a\xea\xfb\xa0\x23\x44\xe8\xf7\x52\xea\xcf\x81\xfb\xcf\xa0\xa0" + + "\x06\xb8\x5c\x9b\x2d\xa2\x07\x22\xec\xf6\xf8\x06\x1e\xcf\xee\x4d" + + "\x75\x13\xaa\x32\xeb\xa3\xe1\xb8\xf3\xac\x62\x0e\xa5\x11\x4e\x87" + + "\x8d\xde\x4f\x8e\x66\xcd\xc6\xed\x12\x14\x21\x44\x74\x0b\x29\x8f" + + "\xae\x3e\xd7\xeb\xd4\xa8\x71\x9c\xe4\xe4\x13\x77\xa8\xc5\xc6\xd6" + + "\xe6\xe8\xd1\xd2\x59\x7b\xd1\x22\x7a\xf1\x3a\x84\x5b\xc0\x5b\x6f" + + "\xd7\xd2\xcf\xc9\xb4\xb2\xcd\x94\x2a\x48\x29\x09\x72\xda\xb9\x6b" + + "\xe0\x6b\xac\xdb\xb4\x43\x82\x9a\x17\x14\x94\x78\x85\xcb\xb5\xee" + + "\x40\x99\x18\x8f\xb0\x54\xa0\x38\xdd\x77\x47\x30\xe8\x37\xe6\xb8" + + "\xdc\x34\x4f\xac\x63\xb9\xef\xa5\x49\xd1\x55\x95\x6e\x8f\x8e\x60" + + "\xd3\xb6\x64\x84\x70\x6e\xa8\x52\x65\x7c\x42\x5a\xda\xc7\x19\x65" + + "\xe5\xf9\xad\xc1\xb8\xe2\x72\x09\x9d\x09\x48\x69\x6c\x00\x98\xf3" + + "\xf3\x0a\x82\x9d\x56\x8e\xf7\xd2\x60\xc5\x15\x21\x38\x78\xe4\x18" + + "\x57\xdf\xf1\xa4\xf4\x15\xd8\x70\xcc\x94\xd2\xdd\xe4\xc8\x91\x91" + + "\x9e\x32\xd5\xd1\x59\xea\x50\x3c\xd0\x75\xa2\x84\x30\x66\x82\x5e" + + "\x2e\xc7\xe5\x16\xa6\x34\xb9\xb2\x61\x2d\xcb\xaa\x5f\x52\x2b\xae" + + "\xf0\xed\xf7\xbf\xd2\x77\xc8\xdb\xd2\xe3\xf5\x0a\x87\xa3\xda\xab" + + "\x86\x91\xd6\x57\xd3\xae\x40\xd7\x93\xcb\x96\xb7\x69\xa9\x43\x71" + + "\xb9\xed\xf6\xcd\x52\x7a\xea\x55\xa9\x52\x45\x1c\x3a\x74\x88\xa8" + + "\x88\x50\x96\x7d\x33\xd6\xaa\xb0\x7a\x09\xac\xb8\xaa\x2a\xec\xdc" + + "\x73\x88\x5e\xfd\x5f\x95\x79\x1e\x8f\x00\x45\x57\xd5\x2b\x3a\x18" + + "\xc6\xf6\x45\xd1\xd1\x23\x48\x4d\x1d\x56\xe6\xe4\x62\xb9\xee\xc5" + + "\x62\x39\xe2\x6e\x94\xd2\x53\x7f\xc0\x80\x01\x62\xdf\xbe\x7d\x4c" + + "\x99\x32\x85\xd4\xb4\xe3\xfc\xb6\x66\xb3\xb5\x52\xee\xa2\x13\x1d" + + "\xbe\x98\xb9\x80\xdb\xfb\x0e\x97\x79\x1e\x8f\x50\x94\xa0\xdf\xc1" + + "\xb4\xd9\x6c\x21\x8b\x80\x32\x49\x72\xcb\xa2\x17\x9b\x35\xd7\xf6" + + "\x37\x69\x92\x58\xe9\x8f\x3f\xfe\xf0\xcb\xb3\x79\xf3\xe6\xec\xd9" + + "\xbd\x83\x05\x5f\xbe\x61\xe5\x91\xbb\x18\xb1\xb8\xa2\xb0\xff\x50" + + "\x0a\x5d\xfb\x0e\xc7\xeb\xd5\x01\x21\x55\xb5\x4a\x0f\xc3\xd8\x37" + + "\xc3\x92\x90\x45\xf4\x62\xd3\xb5\xd4\xd4\x54\xca\x95\x2b\xe7\xcf" + + "\x1d\xe6\x72\xb9\x08\x09\x09\x61\xec\xd0\x47\xb9\xbe\x75\x52\x91" + + "\x52\x11\x59\x38\x77\x82\x17\xc8\xfb\xd9\xd1\x93\xf8\x69\xe9\x1a" + + "\x09\xa0\x28\xc1\xdb\x4c\x33\xb7\x9e\xa2\x54\xc3\x34\xf7\x5a\x82" + + "\xb2\x88\x5e\x6c\xc8\x4b\x49\x49\xb1\x97\x2f\x5f\xfe\x04\x32\xdb" + + "\xed\x76\x4c\xd3\x60\xed\xbc\x0f\x90\xa6\xe5\xc2\x97\x44\x2c\x3e" + + "\x74\xcc\x67\xcc\xf9\x65\x05\x80\x14\x42\x15\x52\x46\xb4\x83\xb4" + + "\xc5\x96\x84\xac\x18\xbd\xf8\x85\xa8\x84\xfc\x18\x13\x13\xc3\x3b" + + "\xef\xbc\x43\x72\x72\x32\x00\xcf\x3c\xf3\x0c\x5e\xaf\x17\xc3\x30" + + "\x69\xde\xe5\x09\x2b\x56\x2f\x46\x82\x03\xfc\xfc\xeb\x5a\xae\xec" + + "\xf4\x78\x01\xc9\x51\x94\xf0\x85\x52\x36\x51\x7c\x24\xaf\x61\x09" + + "\xca\xb2\xe8\xc5\x0b\x4d\xab\x89\xae\xef\x42\xd3\x62\x5f\xd5\xf5" + + "\xa3\xcf\x9f\x4e\x3f\xc7\xbc\xd0\x57\xdc\x70\x75\x13\x4b\x60\x17" + + "\x08\xd3\x94\x74\x7d\x64\x18\x7b\x0f\xa6\x00\x48\xd0\xb2\x6c\xb6" + + "\x4a\x57\x7a\xbd\x7b\x77\xda\x6c\xe5\xf1\x7a\xad\x9c\x00\x16\xd1" + + "\x4b\x08\x4e\x67\x2b\xdc\xee\x15\x68\x5a\xd5\x70\x5d\x3f\x1a\xa7" + + "\x69\x91\x59\xa1\xa1\x6f\x1d\x76\xb9\xfe\x4f\x95\x72\xaf\x4d\xd7" + + "\x93\xd7\x99\x66\x6e\x9d\x99\x13\x87\x13\x5f\x35\xce\x12\xd8\x39" + + "\x5a\x70\x45\x11\x38\x1d\x0e\x5e\x79\xf7\x4b\xa6\x7e\xb7\x10\xc3" + + "\x30\x01\x05\x4d\x8b\xfe\xff\xf6\xce\x36\x28\xaa\xeb\x8c\xe3\xff" + + "\x73\xee\xcb\x5e\x70\x5f\x0c\x20\x0c\x08\x68\x42\x08\x45\xab\xc6" + + "\x3a\xd4\x69\x4a\x1b\x69\x52\xda\x66\xc6\xb6\x93\x92\xa4\x93\xb6" + + "\x32\xed\x87\xaa\xfd\x52\x26\x9d\x44\x74\x1c\x07\x94\xaa\x33\x4e" + + "\x1a\x13\x32\x99\x8e\x09\x1d\x51\x74\xb0\x11\xa3\x4e\x8d\x89\x14" + + "\x0d\x49\xad\x1a\x6b\x29\x83\x75\xc6\x26\xbe\x2c\x48\x95\x42\x79" + + "\x87\xbd\xbb\x77\xef\x7d\xfa\x81\xdd\x0d\x49\x51\x61\x9a\xa4\xab" + + "\x3c\xbf\x99\xfd\xc4\xee\xb2\xf3\x9c\xf3\xbb\xcf\x39\xe7\x9e\xf3" + + "\xdc\x92\x82\x82\xae\x86\x53\xa7\xb8\x1b\xb3\xe8\x71\x80\xa2\x2c" + + "\xd6\x6d\xbb\x2d\x08\x84\xe9\xcc\xc1\x6a\xa1\x6b\x2a\x2f\xcc\x4d" + + "\xf6\x22\xea\xd2\xf1\xec\xe6\x1d\x38\x72\xfc\x4c\xa4\x66\xbe\x84" + + "\x94\x33\x7e\xe5\x76\x97\x6d\x1f\x1c\xdc\xe4\x78\x3c\x4b\x31\x34" + + "\x74\x86\x03\xc5\xa2\xc7\x07\x29\x29\x8f\xa7\xf4\xf4\x1c\xe8\x9e" + + "\x33\x3b\x8d\x0e\xbd\x5a\x29\x78\x15\xfe\xe6\x19\xdc\xa5\x6b\x18" + + "\x1c\x0e\xe0\xf9\xd7\x5e\xc7\xa1\x63\x7f\x8e\x54\xd6\x15\x90\xd2" + + "\x57\x91\x9e\xfe\x46\x55\x67\x67\x91\xad\xaa\x59\x08\x87\x3b\x38" + + "\x60\x2c\x7a\x9c\x05\x5a\x24\x43\xd3\x7c\x45\xa1\xd0\xe5\xe3\x0f" + + "\x2d\x99\x4f\x3b\xb6\x94\x09\xcb\x0a\xb3\xec\x11\xa4\x14\x30\x74" + + "\x1d\x35\xaf\xbf\x85\x63\xef\x9e\x43\xcb\xdf\x3f\x8c\x75\x51\x21" + + "\x12\x36\xac\x58\x31\x52\x55\x5b\x2b\x78\x45\x93\x45\xbf\x53\x3a" + + "\xb4\x6f\x8d\xe3\x0c\x6c\x2d\xfa\xca\x83\x54\x5d\xf1\x0b\x61\x85" + + "\xed\x69\x2b\x3b\x11\x41\x91\x12\xc3\x01\x13\xaf\xec\x3e\x8c\xba" + + "\x37\x9a\x80\x48\x15\xd6\x88\xe0\x75\x86\x91\xb3\x36\x10\x68\xbb" + + "\xc6\x3d\x87\x45\xbf\xf3\xe6\x9d\x46\x5e\xb1\x69\x5e\x7c\xfb\x07" + + "\xdf\x29\xa4\xca\xb2\x52\x61\x85\xa7\x4f\x66\x8f\x9e\x28\xd3\x75" + + "\x0d\x97\xdb\xaf\xe3\xc8\x89\xf7\xf1\xca\xee\xc3\xe3\x04\xd7\xcf" + + "\x08\xa1\x35\x10\x8d\x6c\x1b\xfb\xc4\x5c\xdc\xac\xb4\x13\xc3\xa2" + + "\xc7\x3d\x8a\x92\xba\xc6\xb6\xff\xb5\x75\x7e\xee\x1c\xaa\x28\x5b" + + "\x21\xe6\xe5\x66\x23\x74\x97\x0f\xe5\x35\x55\xc1\x8d\x9e\x7e\x9c" + + "\x3c\x7b\x1e\x3b\xea\xdf\xc4\xb5\xeb\xdd\xe3\x04\x97\x03\xaa\x9a" + + "\x56\x1a\x0e\x5f\x3f\xc4\xbd\x83\x45\xbf\xab\x50\xd5\xac\x1f\x85" + + "\xc3\x1d\x75\x00\xe8\xa9\xe5\xcb\xc4\xda\xd5\x3f\xbc\xeb\xb6\xca" + + "\x2a\x52\x62\xd4\x0c\xe2\xc2\x07\x7e\xbc\xb4\xf3\x60\x74\xee\x4d" + + "\x1f\xf5\x3d\xd1\x21\x84\x77\x2f\xd1\x40\x39\xf7\x08\x16\xfd\xee" + + "\xcd\x72\x5a\xf6\x63\x96\xd5\x7e\x04\x00\xa5\xa7\x26\x89\xba\xed" + + "\xe5\x98\x95\xe4\x83\x6d\xff\x77\x81\xc9\xe8\x45\x40\x0a\x01\xcb" + + "\xb6\xe1\xd8\x0e\x74\x4d\x8d\xab\xa7\xb8\x46\x7f\xa3\x10\xc0\x95" + + "\x8e\x2e\xbc\x5c\x7b\x10\x7f\x3c\xd9\xf2\xc9\x6e\xd7\x27\xc4\x8c" + + "\xa6\xec\xec\xbf\x3c\xed\xf7\x7f\x81\xcf\xf1\xb2\xe8\xd3\x25\xb3" + + "\xdf\x77\x6f\x38\xec\xbf\x00\xd8\x06\x00\x14\x7f\x6d\x09\xb6\x3c" + + "\xf7\x33\x28\x8a\x12\xdb\xf2\x29\x84\x80\x19\x0c\xe1\xc2\x87\xed" + + "\xd8\xd5\xd0\x88\x77\x4e\xb7\x46\x86\xc3\x2a\xde\xdc\xf9\x6b\xcc" + + "\x4a\xf2\xfd\xdf\x47\x03\xb6\x6d\x23\x60\x86\xd0\x74\xb2\x05\x95" + + "\x2f\xd5\x4d\x70\x6a\x4f\xf5\x03\x99\x5f\x05\xae\x76\x6a\xda\x3c" + + "\x58\xd6\x05\x6e\x7c\x16\x7d\xfa\x60\x18\x5f\xc2\xf2\xe5\xe7\x94" + + "\xfd\xfb\x13\x87\x89\x02\x06\x00\x12\x02\xe2\xb7\x55\xbf\xc4\xd2" + + "\xc5\xf9\x20\x22\x1c\x3a\x76\x0a\x15\x2f\xee\x8a\x64\xcb\xb1\x97" + + "\xe3\x38\xb1\x0b\xc1\xb3\x3f\x7f\x02\x4f\x7f\xef\x1b\xb1\xd3\x5c" + + "\x9f\x97\xf0\x8e\x43\x08\xdb\x36\xb6\xd7\x34\x60\xef\xe1\x13\x13" + + "\xbe\x05\xd0\xbb\xa5\x9c\xfd\xb0\xe3\x5c\xb9\xc8\xad\xcd\xa2\x33" + + "\x00\x34\x2d\x73\xbd\x65\x5d\xdb\x14\xcd\xe2\x8d\xbb\xb7\xa2\xea" + + "\xe5\xbd\x78\xe7\x74\x2b\xf6\xed\xdb\x87\x92\x92\x92\x8f\x1a\x4f" + + "\x08\x34\x37\x37\xa3\xa8\xa8\x08\xc0\x58\xbd\xf2\x4d\xcf\xfc\x14" + + "\x8f\x16\x2e\xfe\x4c\x64\x1f\xff\x9d\x9d\x37\x7a\xf0\xbb\xdf\xbf" + + "\x85\x03\x6f\xff\x09\x13\x9f\xd7\xd1\x2f\x4a\x39\x73\xed\xfd\xf7" + + "\xff\xf8\x0f\x57\xaf\xb6\x5a\xa1\x50\x13\x16\x2d\x6a\x43\x6b\xeb" + + "\x02\x6e\x64\x16\x7d\xba\x67\xf6\x2f\xc2\x34\xcf\x43\x51\x52\x5d" + + "\x9a\x96\xf8\x75\xd3\xec\x3c\x06\x58\x40\x64\xf1\x2a\x18\x0c\x42" + + "\xd3\xb4\x98\x6c\x51\xf1\x4c\xd3\xc4\xaa\x55\xab\x50\x5b\x5b\x0b" + + "\x00\xf0\xb8\x13\x71\xfa\xc0\x8b\xb0\xc2\xf6\xa7\x22\xb7\x14\x02" + + "\x9a\xae\xe1\xaf\xe7\x3f\xc0\x73\x9b\x5f\x45\x57\x4f\x1f\xec\x09" + + "\x8b\x69\xb8\xb7\x09\x21\x37\x4b\xa9\x9a\xb6\xdd\x6b\x72\x8b\xb2" + + "\xe8\xcc\x2d\xf0\x78\x1e\xc4\xd0\xd0\xdf\xa2\xfa\xbf\x00\x98\x65" + + "\x95\x95\x95\xd8\xb0\xe1\xd6\x65\x90\x46\x46\x46\xb0\x72\xe5\x4a" + + "\xec\xd9\xb3\x07\x52\x4a\x6c\xdf\xb0\x1a\x0f\x2f\x5d\x38\xe9\x27" + + "\xc7\xc4\x16\xfb\xa4\x84\x4b\xd7\xf0\xcf\xae\x7f\x63\xff\xd1\x77" + + "\xd1\x70\xf4\x3d\xf4\x0d\x0c\x8f\xaf\x7f\x17\xdb\xd4\xa2\xaa\x69" + + "\x4f\x86\xc3\xbd\x4d\x2e\x57\x56\x20\x18\xbc\x14\xe0\xd6\x63\xd1" + + "\x99\x29\xe2\x72\x7d\x19\xc1\x60\x4b\x2f\x60\xdd\x33\x30\x30\x00" + + "\x8f\xc7\x73\x4b\x61\xa3\xa2\x36\x36\x36\xa2\xb8\xb8\x18\x00\xb0" + + "\x66\xf5\x53\x28\x7d\xfc\x9b\xb1\x79\xbd\x19\x0c\x61\xd4\x0c\xe2" + + "\xe8\x89\xb3\x48\x30\x74\xd8\xb6\x8d\xf3\xff\xb8\x8a\xb4\x59\x49" + + "\x38\xdb\x7a\x11\x7d\x03\x43\x18\x19\x35\x71\xa3\xbb\x17\x56\xd8" + + "\x1e\x77\x1b\xcc\xd5\x0e\xc8\x5d\x80\xad\x28\x8a\xe7\x12\x51\xe8" + + "\xf8\xdc\xb9\xdb\xfc\x97\x2f\xaf\xe2\x3a\x59\x0c\xf3\xbf\xe0\xf5" + + "\x3e\x9a\x05\xc0\x29\x2c\x2c\xa4\xc9\xe2\x38\x0e\x39\x8e\x43\x8a" + + "\xa2\x10\x00\xd2\x34\x95\xee\xcb\x4e\xa7\xf4\xd4\x24\x9a\x95\x3c" + + "\x93\x22\xd9\x78\x12\x2f\xd5\x2f\xa5\xfb\x79\x20\x71\xa9\xa6\x3d" + + "\xe0\xfb\xe4\x6f\x4b\x48\x78\x88\x1b\x88\x33\x3a\xf3\xe9\x90\x70" + + "\x04\x08\x3c\xd6\xdf\xdf\x0f\x9f\xcf\x37\xe9\x39\xb5\x10\x02\xaa" + + "\xaa\xc2\xb6\x6d\x02\x84\x00\xb4\xe3\x80\x33\x1f\x40\x1a\x20\xdb" + + "\x00\x9c\x02\x30\x04\x60\x09\x40\x06\xa0\x9c\x03\x9c\x4b\x42\x28" + + "\x3d\x00\x06\x14\x25\xa5\x2d\x1c\xee\xb8\xc2\xf1\x67\x98\xcf\xf2" + + "\xea\x2b\xd2\xe1\x76\x2f\x50\x00\x41\x25\x25\x25\x0e\x4d\x91\x8d" + + "\x1b\x37\xc6\x32\xb3\xa2\xcc\xc9\x9b\xca\xff\x5e\xb8\x70\x27\x37" + + "\x00\xc3\x7c\x5e\x48\x79\xcf\x0b\x00\x9c\xe8\x70\x7c\xb2\xf4\xf6" + + "\xf6\x12\x00\x07\x00\x49\xe9\xdd\xc5\x91\x64\x98\x38\x45\xd7\xf3" + + "\xb3\x00\x50\x45\x45\xc5\x94\x32\xf9\x78\xc9\x01\xa3\x05\x18\xdb" + + "\x8c\xc3\x30\x4c\x1c\x91\x9f\xdf\x11\x19\xba\x1b\xdd\xa9\xa9\xa9" + + "\x93\xce\xe6\xd1\xf7\xb8\xdd\x6e\x07\x10\x04\x18\x5b\xc6\x46\x05" + + "\x19\x1c\x54\x86\x89\x47\x14\x25\xe3\x11\x00\xd4\xdf\xdf\x3f\xa5" + + "\x21\xfb\xfa\xf5\xeb\x1d\x00\xa4\xeb\xf3\xbe\x3b\x63\xc6\x42\x0e" + + "\x24\xc3\xc4\x23\x1e\xcf\xb7\xe0\xf5\x6e\x13\x80\x4a\xe9\xe9\xe9" + + "\x53\x1a\xb2\x77\x77\x77\x13\x00\x12\x22\xf1\x3d\x8e\x24\xc3\xc4" + + "\x39\x52\x7a\x7f\x02\xc0\xa9\xae\xae\xfe\x58\x36\xbf\x5d\x66\x6f" + + "\x6e\x6e\x26\x00\x64\x18\x79\x3c\x21\x67\x98\xf8\x17\xdd\x30\x53" + + "\x52\x52\x62\x02\x97\x95\x95\x91\xdf\xef\xbf\xed\xdc\x3c\x39\x39" + + "\xd9\x01\x04\x3f\xb9\x80\x61\xe2\x5f\xf2\xa4\x05\x00\xa8\xa0\xa0" + + "\x80\x88\x88\x06\x07\x07\x63\xf7\xc2\x6f\x95\xd1\xeb\xeb\xeb\xa3" + + "\xc3\xf6\xdf\x70\x14\x19\x26\xee\x99\x79\xef\x78\xd1\x2d\xcb\x8a" + + "\x89\x3e\x3a\x3a\x7a\x53\xd9\xbd\x5e\x2f\x8d\xad\xb4\x33\x0c\x13" + + "\xf7\xcc\x9e\xbd\x57\x03\x40\x19\x19\x19\x31\x89\x73\x73\x73\x09" + + "\x00\x95\x96\x96\x4e\x28\x79\x6d\x6d\x6d\xe4\xbe\xb9\xf1\x3e\x47" + + "\x90\x61\xee\x98\xe1\xfb\xcc\x9d\x00\x9c\xfa\xfa\x7a\x72\x1c\x27" + + "\xba\x01\xe6\x63\x59\x3d\x9a\xd9\x6b\x6a\x6a\x62\x7f\x53\xd5\xfc" + + "\x45\x86\x31\x8f\x03\xc8\x30\xf1\x8e\xa6\x3d\x00\x4d\x5b\xec\x03" + + "\x5c\x7d\x8a\x22\xa9\xbc\xbc\x9c\x86\x87\x87\x29\x27\x27\x87\x00" + + "\xd0\xb2\x65\xcb\xa8\xa1\xa1\x81\xd6\xad\x5b\x47\x79\x79\x79\x51" + + "\xc9\x1d\x21\xb4\x61\x8e\x1e\xc3\xdc\x81\x08\xe1\x79\x06\x50\x26" + + "\x71\x94\x54\x92\x94\x99\x73\x39\x62\xcc\x6d\xfb\x14\x87\x20\xbe" + + "\x30\x8c\x02\x98\xe6\x59\x00\x80\xa2\x24\x7d\xdb\x71\x02\x39\x80" + + "\xe8\x02\x44\x3b\x40\xdf\x27\x0a\xe5\x02\xc8\x91\xd2\x75\x30\x33" + + "\xf3\x4a\x55\x7b\x7b\x2a\x17\x7e\x60\x6e\xcb\x7f\x00\x4a\x3f\xff" + + "\x3a\x92\xd3\x63\x31\x00\x00\x00\x00\x49\x45\x4e\x44\xae\x42\x60" + + "\x82\x01\x00\x00\xff\xff\x25\x50\x56\x5e\x8b\x55\x00\x00" + +// gophercolor_png returns the raw, uncompressed file data data. +func gophercolor_png() []byte { + var empty [0]byte + sx := (*reflect.StringHeader)(unsafe.Pointer(&_gophercolor_png)) + b := empty[:] + bx := (*reflect.SliceHeader)(unsafe.Pointer(&b)) + bx.Data = sx.Data + bx.Len = len(_gophercolor_png) + bx.Cap = bx.Len + + gz, err := gzip.NewReader(bytes.NewBuffer(b)) + + if err != nil { + panic("Decompression failed: " + err.Error()) + } + + var buf bytes.Buffer + io.Copy(&buf, gz) + gz.Close() + + return buf.Bytes() +} diff --git a/third_party/github.com/jteeuwen/go-bindata/testdata/nomemcpy-uncompressed.go b/third_party/github.com/jteeuwen/go-bindata/testdata/nomemcpy-uncompressed.go new file mode 100644 index 00000000000..5aa3333a767 --- /dev/null +++ b/third_party/github.com/jteeuwen/go-bindata/testdata/nomemcpy-uncompressed.go @@ -0,0 +1,1392 @@ +package main + +import ( + "reflect" + "unsafe" +) + +var _gophercolor_png = "" + + "\x89\x50\x4e\x47\x0d\x0a\x1a\x0a\x00\x00\x00\x0d\x49\x48\x44\x52" + + "\x00\x00\x00\xfa\x00\x00\x00\xfa\x08\x06\x00\x00\x00\x88\xec\x5a" + + "\x3d\x00\x00\x0a\x4f\x69\x43\x43\x50\x50\x68\x6f\x74\x6f\x73\x68" + + "\x6f\x70\x20\x49\x43\x43\x20\x70\x72\x6f\x66\x69\x6c\x65\x00\x00" + + "\x78\xda\x9d\x53\x67\x54\x53\xe9\x16\x3d\xf7\xde\xf4\x42\x4b\x88" + + "\x80\x94\x4b\x6f\x52\x15\x08\x20\x52\x42\x8b\x80\x14\x91\x26\x2a" + + "\x21\x09\x10\x4a\x88\x21\xa1\xd9\x15\x51\xc1\x11\x45\x45\x04\x1b" + + "\xc8\xa0\x88\x03\x8e\x8e\x80\x8c\x15\x51\x2c\x0c\x8a\x0a\xd8\x07" + + "\xe4\x21\xa2\x8e\x83\xa3\x88\x8a\xca\xfb\xe1\x7b\xa3\x6b\xd6\xbc" + + "\xf7\xe6\xcd\xfe\xb5\xd7\x3e\xe7\xac\xf3\x9d\xb3\xcf\x07\xc0\x08" + + "\x0c\x96\x48\x33\x51\x35\x80\x0c\xa9\x42\x1e\x11\xe0\x83\xc7\xc4" + + "\xc6\xe1\xe4\x2e\x40\x81\x0a\x24\x70\x00\x10\x08\xb3\x64\x21\x73" + + "\xfd\x23\x01\x00\xf8\x7e\x3c\x3c\x2b\x22\xc0\x07\xbe\x00\x01\x78" + + "\xd3\x0b\x08\x00\xc0\x4d\x9b\xc0\x30\x1c\x87\xff\x0f\xea\x42\x99" + + "\x5c\x01\x80\x84\x01\xc0\x74\x91\x38\x4b\x08\x80\x14\x00\x40\x7a" + + "\x8e\x42\xa6\x00\x40\x46\x01\x80\x9d\x98\x26\x53\x00\xa0\x04\x00" + + "\x60\xcb\x63\x62\xe3\x00\x50\x2d\x00\x60\x27\x7f\xe6\xd3\x00\x80" + + "\x9d\xf8\x99\x7b\x01\x00\x5b\x94\x21\x15\x01\xa0\x91\x00\x20\x13" + + "\x65\x88\x44\x00\x68\x3b\x00\xac\xcf\x56\x8a\x45\x00\x58\x30\x00" + + "\x14\x66\x4b\xc4\x39\x00\xd8\x2d\x00\x30\x49\x57\x66\x48\x00\xb0" + + "\xb7\x00\xc0\xce\x10\x0b\xb2\x00\x08\x0c\x00\x30\x51\x88\x85\x29" + + "\x00\x04\x7b\x00\x60\xc8\x23\x23\x78\x00\x84\x99\x00\x14\x46\xf2" + + "\x57\x3c\xf1\x2b\xae\x10\xe7\x2a\x00\x00\x78\x99\xb2\x3c\xb9\x24" + + "\x39\x45\x81\x5b\x08\x2d\x71\x07\x57\x57\x2e\x1e\x28\xce\x49\x17" + + "\x2b\x14\x36\x61\x02\x61\x9a\x40\x2e\xc2\x79\x99\x19\x32\x81\x34" + + "\x0f\xe0\xf3\xcc\x00\x00\xa0\x91\x15\x11\xe0\x83\xf3\xfd\x78\xce" + + "\x0e\xae\xce\xce\x36\x8e\xb6\x0e\x5f\x2d\xea\xbf\x06\xff\x22\x62" + + "\x62\xe3\xfe\xe5\xcf\xab\x70\x40\x00\x00\xe1\x74\x7e\xd1\xfe\x2c" + + "\x2f\xb3\x1a\x80\x3b\x06\x80\x6d\xfe\xa2\x25\xee\x04\x68\x5e\x0b" + + "\xa0\x75\xf7\x8b\x66\xb2\x0f\x40\xb5\x00\xa0\xe9\xda\x57\xf3\x70" + + "\xf8\x7e\x3c\x3c\x45\xa1\x90\xb9\xd9\xd9\xe5\xe4\xe4\xd8\x4a\xc4" + + "\x42\x5b\x61\xca\x57\x7d\xfe\x67\xc2\x5f\xc0\x57\xfd\x6c\xf9\x7e" + + "\x3c\xfc\xf7\xf5\xe0\xbe\xe2\x24\x81\x32\x5d\x81\x47\x04\xf8\xe0" + + "\xc2\xcc\xf4\x4c\xa5\x1c\xcf\x92\x09\x84\x62\xdc\xe6\x8f\x47\xfc" + + "\xb7\x0b\xff\xfc\x1d\xd3\x22\xc4\x49\x62\xb9\x58\x2a\x14\xe3\x51" + + "\x12\x71\x8e\x44\x9a\x8c\xf3\x32\xa5\x22\x89\x42\x92\x29\xc5\x25" + + "\xd2\xff\x64\xe2\xdf\x2c\xfb\x03\x3e\xdf\x35\x00\xb0\x6a\x3e\x01" + + "\x7b\x91\x2d\xa8\x5d\x63\x03\xf6\x4b\x27\x10\x58\x74\xc0\xe2\xf7" + + "\x00\x00\xf2\xbb\x6f\xc1\xd4\x28\x08\x03\x80\x68\x83\xe1\xcf\x77" + + "\xff\xef\x3f\xfd\x47\xa0\x25\x00\x80\x66\x49\x92\x71\x00\x00\x5e" + + "\x44\x24\x2e\x54\xca\xb3\x3f\xc7\x08\x00\x00\x44\xa0\x81\x2a\xb0" + + "\x41\x1b\xf4\xc1\x18\x2c\xc0\x06\x1c\xc1\x05\xdc\xc1\x0b\xfc\x60" + + "\x36\x84\x42\x24\xc4\xc2\x42\x10\x42\x0a\x64\x80\x1c\x72\x60\x29" + + "\xac\x82\x42\x28\x86\xcd\xb0\x1d\x2a\x60\x2f\xd4\x40\x1d\x34\xc0" + + "\x51\x68\x86\x93\x70\x0e\x2e\xc2\x55\xb8\x0e\x3d\x70\x0f\xfa\x61" + + "\x08\x9e\xc1\x28\xbc\x81\x09\x04\x41\xc8\x08\x13\x61\x21\xda\x88" + + "\x01\x62\x8a\x58\x23\x8e\x08\x17\x99\x85\xf8\x21\xc1\x48\x04\x12" + + "\x8b\x24\x20\xc9\x88\x14\x51\x22\x4b\x91\x35\x48\x31\x52\x8a\x54" + + "\x20\x55\x48\x1d\xf2\x3d\x72\x02\x39\x87\x5c\x46\xba\x91\x3b\xc8" + + "\x00\x32\x82\xfc\x86\xbc\x47\x31\x94\x81\xb2\x51\x3d\xd4\x0c\xb5" + + "\x43\xb9\xa8\x37\x1a\x84\x46\xa2\x0b\xd0\x64\x74\x31\x9a\x8f\x16" + + "\xa0\x9b\xd0\x72\xb4\x1a\x3d\x8c\x36\xa1\xe7\xd0\xab\x68\x0f\xda" + + "\x8f\x3e\x43\xc7\x30\xc0\xe8\x18\x07\x33\xc4\x6c\x30\x2e\xc6\xc3" + + "\x42\xb1\x38\x2c\x09\x93\x63\xcb\xb1\x22\xac\x0c\xab\xc6\x1a\xb0" + + "\x56\xac\x03\xbb\x89\xf5\x63\xcf\xb1\x77\x04\x12\x81\x45\xc0\x09" + + "\x36\x04\x77\x42\x20\x61\x1e\x41\x48\x58\x4c\x58\x4e\xd8\x48\xa8" + + "\x20\x1c\x24\x34\x11\xda\x09\x37\x09\x03\x84\x51\xc2\x27\x22\x93" + + "\xa8\x4b\xb4\x26\xba\x11\xf9\xc4\x18\x62\x32\x31\x87\x58\x48\x2c" + + "\x23\xd6\x12\x8f\x13\x2f\x10\x7b\x88\x43\xc4\x37\x24\x12\x89\x43" + + "\x32\x27\xb9\x90\x02\x49\xb1\xa4\x54\xd2\x12\xd2\x46\xd2\x6e\x52" + + "\x23\xe9\x2c\xa9\x9b\x34\x48\x1a\x23\x93\xc9\xda\x64\x6b\xb2\x07" + + "\x39\x94\x2c\x20\x2b\xc8\x85\xe4\x9d\xe4\xc3\xe4\x33\xe4\x1b\xe4" + + "\x21\xf2\x5b\x0a\x9d\x62\x40\x71\xa4\xf8\x53\xe2\x28\x52\xca\x6a" + + "\x4a\x19\xe5\x10\xe5\x34\xe5\x06\x65\x98\x32\x41\x55\xa3\x9a\x52" + + "\xdd\xa8\xa1\x54\x11\x35\x8f\x5a\x42\xad\xa1\xb6\x52\xaf\x51\x87" + + "\xa8\x13\x34\x75\x9a\x39\xcd\x83\x16\x49\x4b\xa5\xad\xa2\x95\xd3" + + "\x1a\x68\x17\x68\xf7\x69\xaf\xe8\x74\xba\x11\xdd\x95\x1e\x4e\x97" + + "\xd0\x57\xd2\xcb\xe9\x47\xe8\x97\xe8\x03\xf4\x77\x0c\x0d\x86\x15" + + "\x83\xc7\x88\x67\x28\x19\x9b\x18\x07\x18\x67\x19\x77\x18\xaf\x98" + + "\x4c\xa6\x19\xd3\x8b\x19\xc7\x54\x30\x37\x31\xeb\x98\xe7\x99\x0f" + + "\x99\x6f\x55\x58\x2a\xb6\x2a\x7c\x15\x91\xca\x0a\x95\x4a\x95\x26" + + "\x95\x1b\x2a\x2f\x54\xa9\xaa\xa6\xaa\xde\xaa\x0b\x55\xf3\x55\xcb" + + "\x54\x8f\xa9\x5e\x53\x7d\xae\x46\x55\x33\x53\xe3\xa9\x09\xd4\x96" + + "\xab\x55\xaa\x9d\x50\xeb\x53\x1b\x53\x67\xa9\x3b\xa8\x87\xaa\x67" + + "\xa8\x6f\x54\x3f\xa4\x7e\x59\xfd\x89\x06\x59\xc3\x4c\xc3\x4f\x43" + + "\xa4\x51\xa0\xb1\x5f\xe3\xbc\xc6\x20\x0b\x63\x19\xb3\x78\x2c\x21" + + "\x6b\x0d\xab\x86\x75\x81\x35\xc4\x26\xb1\xcd\xd9\x7c\x76\x2a\xbb" + + "\x98\xfd\x1d\xbb\x8b\x3d\xaa\xa9\xa1\x39\x43\x33\x4a\x33\x57\xb3" + + "\x52\xf3\x94\x66\x3f\x07\xe3\x98\x71\xf8\x9c\x74\x4e\x09\xe7\x28" + + "\xa7\x97\xf3\x7e\x8a\xde\x14\xef\x29\xe2\x29\x1b\xa6\x34\x4c\xb9" + + "\x31\x65\x5c\x6b\xaa\x96\x97\x96\x58\xab\x48\xab\x51\xab\x47\xeb" + + "\xbd\x36\xae\xed\xa7\x9d\xa6\xbd\x45\xbb\x59\xfb\x81\x0e\x41\xc7" + + "\x4a\x27\x5c\x27\x47\x67\x8f\xce\x05\x9d\xe7\x53\xd9\x53\xdd\xa7" + + "\x0a\xa7\x16\x4d\x3d\x3a\xf5\xae\x2e\xaa\x6b\xa5\x1b\xa1\xbb\x44" + + "\x77\xbf\x6e\xa7\xee\x98\x9e\xbe\x5e\x80\x9e\x4c\x6f\xa7\xde\x79" + + "\xbd\xe7\xfa\x1c\x7d\x2f\xfd\x54\xfd\x6d\xfa\xa7\xf5\x47\x0c\x58" + + "\x06\xb3\x0c\x24\x06\xdb\x0c\xce\x18\x3c\xc5\x35\x71\x6f\x3c\x1d" + + "\x2f\xc7\xdb\xf1\x51\x43\x5d\xc3\x40\x43\xa5\x61\x95\x61\x97\xe1" + + "\x84\x91\xb9\xd1\x3c\xa3\xd5\x46\x8d\x46\x0f\x8c\x69\xc6\x5c\xe3" + + "\x24\xe3\x6d\xc6\x6d\xc6\xa3\x26\x06\x26\x21\x26\x4b\x4d\xea\x4d" + + "\xee\x9a\x52\x4d\xb9\xa6\x29\xa6\x3b\x4c\x3b\x4c\xc7\xcd\xcc\xcd" + + "\xa2\xcd\xd6\x99\x35\x9b\x3d\x31\xd7\x32\xe7\x9b\xe7\x9b\xd7\x9b" + + "\xdf\xb7\x60\x5a\x78\x5a\x2c\xb6\xa8\xb6\xb8\x65\x49\xb2\xe4\x5a" + + "\xa6\x59\xee\xb6\xbc\x6e\x85\x5a\x39\x59\xa5\x58\x55\x5a\x5d\xb3" + + "\x46\xad\x9d\xad\x25\xd6\xbb\xad\xbb\xa7\x11\xa7\xb9\x4e\x93\x4e" + + "\xab\x9e\xd6\x67\xc3\xb0\xf1\xb6\xc9\xb6\xa9\xb7\x19\xb0\xe5\xd8" + + "\x06\xdb\xae\xb6\x6d\xb6\x7d\x61\x67\x62\x17\x67\xb7\xc5\xae\xc3" + + "\xee\x93\xbd\x93\x7d\xba\x7d\x8d\xfd\x3d\x07\x0d\x87\xd9\x0e\xab" + + "\x1d\x5a\x1d\x7e\x73\xb4\x72\x14\x3a\x56\x3a\xde\x9a\xce\x9c\xee" + + "\x3f\x7d\xc5\xf4\x96\xe9\x2f\x67\x58\xcf\x10\xcf\xd8\x33\xe3\xb6" + + "\x13\xcb\x29\xc4\x69\x9d\x53\x9b\xd3\x47\x67\x17\x67\xb9\x73\x83" + + "\xf3\x88\x8b\x89\x4b\x82\xcb\x2e\x97\x3e\x2e\x9b\x1b\xc6\xdd\xc8" + + "\xbd\xe4\x4a\x74\xf5\x71\x5d\xe1\x7a\xd2\xf5\x9d\x9b\xb3\x9b\xc2" + + "\xed\xa8\xdb\xaf\xee\x36\xee\x69\xee\x87\xdc\x9f\xcc\x34\x9f\x29" + + "\x9e\x59\x33\x73\xd0\xc3\xc8\x43\xe0\x51\xe5\xd1\x3f\x0b\x9f\x95" + + "\x30\x6b\xdf\xac\x7e\x4f\x43\x4f\x81\x67\xb5\xe7\x23\x2f\x63\x2f" + + "\x91\x57\xad\xd7\xb0\xb7\xa5\x77\xaa\xf7\x61\xef\x17\x3e\xf6\x3e" + + "\x72\x9f\xe3\x3e\xe3\x3c\x37\xde\x32\xde\x59\x5f\xcc\x37\xc0\xb7" + + "\xc8\xb7\xcb\x4f\xc3\x6f\x9e\x5f\x85\xdf\x43\x7f\x23\xff\x64\xff" + + "\x7a\xff\xd1\x00\xa7\x80\x25\x01\x67\x03\x89\x81\x41\x81\x5b\x02" + + "\xfb\xf8\x7a\x7c\x21\xbf\x8e\x3f\x3a\xdb\x65\xf6\xb2\xd9\xed\x41" + + "\x8c\xa0\xb9\x41\x15\x41\x8f\x82\xad\x82\xe5\xc1\xad\x21\x68\xc8" + + "\xec\x90\xad\x21\xf7\xe7\x98\xce\x91\xce\x69\x0e\x85\x50\x7e\xe8" + + "\xd6\xd0\x07\x61\xe6\x61\x8b\xc3\x7e\x0c\x27\x85\x87\x85\x57\x86" + + "\x3f\x8e\x70\x88\x58\x1a\xd1\x31\x97\x35\x77\xd1\xdc\x43\x73\xdf" + + "\x44\xfa\x44\x96\x44\xde\x9b\x67\x31\x4f\x39\xaf\x2d\x4a\x35\x2a" + + "\x3e\xaa\x2e\x6a\x3c\xda\x37\xba\x34\xba\x3f\xc6\x2e\x66\x59\xcc" + + "\xd5\x58\x9d\x58\x49\x6c\x4b\x1c\x39\x2e\x2a\xae\x36\x6e\x6c\xbe" + + "\xdf\xfc\xed\xf3\x87\xe2\x9d\xe2\x0b\xe3\x7b\x17\x98\x2f\xc8\x5d" + + "\x70\x79\xa1\xce\xc2\xf4\x85\xa7\x16\xa9\x2e\x12\x2c\x3a\x96\x40" + + "\x4c\x88\x4e\x38\x94\xf0\x41\x10\x2a\xa8\x16\x8c\x25\xf2\x13\x77" + + "\x25\x8e\x0a\x79\xc2\x1d\xc2\x67\x22\x2f\xd1\x36\xd1\x88\xd8\x43" + + "\x5c\x2a\x1e\x4e\xf2\x48\x2a\x4d\x7a\x92\xec\x91\xbc\x35\x79\x24" + + "\xc5\x33\xa5\x2c\xe5\xb9\x84\x27\xa9\x90\xbc\x4c\x0d\x4c\xdd\x9b" + + "\x3a\x9e\x16\x9a\x76\x20\x6d\x32\x3d\x3a\xbd\x31\x83\x92\x91\x90" + + "\x71\x42\xaa\x21\x4d\x93\xb6\x67\xea\x67\xe6\x66\x76\xcb\xac\x65" + + "\x85\xb2\xfe\xc5\x6e\x8b\xb7\x2f\x1e\x95\x07\xc9\x6b\xb3\x90\xac" + + "\x05\x59\x2d\x0a\xb6\x42\xa6\xe8\x54\x5a\x28\xd7\x2a\x07\xb2\x67" + + "\x65\x57\x66\xbf\xcd\x89\xca\x39\x96\xab\x9e\x2b\xcd\xed\xcc\xb3" + + "\xca\xdb\x90\x37\x9c\xef\x9f\xff\xed\x12\xc2\x12\xe1\x92\xb6\xa5" + + "\x86\x4b\x57\x2d\x1d\x58\xe6\xbd\xac\x6a\x39\xb2\x3c\x71\x79\xdb" + + "\x0a\xe3\x15\x05\x2b\x86\x56\x06\xac\x3c\xb8\x8a\xb6\x2a\x6d\xd5" + + "\x4f\xab\xed\x57\x97\xae\x7e\xbd\x26\x7a\x4d\x6b\x81\x5e\xc1\xca" + + "\x82\xc1\xb5\x01\x6b\xeb\x0b\x55\x0a\xe5\x85\x7d\xeb\xdc\xd7\xed" + + "\x5d\x4f\x58\x2f\x59\xdf\xb5\x61\xfa\x86\x9d\x1b\x3e\x15\x89\x8a" + + "\xae\x14\xdb\x17\x97\x15\x7f\xd8\x28\xdc\x78\xe5\x1b\x87\x6f\xca" + + "\xbf\x99\xdc\x94\xb4\xa9\xab\xc4\xb9\x64\xcf\x66\xd2\x66\xe9\xe6" + + "\xde\x2d\x9e\x5b\x0e\x96\xaa\x97\xe6\x97\x0e\x6e\x0d\xd9\xda\xb4" + + "\x0d\xdf\x56\xb4\xed\xf5\xf6\x45\xdb\x2f\x97\xcd\x28\xdb\xbb\x83" + + "\xb6\x43\xb9\xa3\xbf\x3c\xb8\xbc\x65\xa7\xc9\xce\xcd\x3b\x3f\x54" + + "\xa4\x54\xf4\x54\xfa\x54\x36\xee\xd2\xdd\xb5\x61\xd7\xf8\x6e\xd1" + + "\xee\x1b\x7b\xbc\xf6\x34\xec\xd5\xdb\x5b\xbc\xf7\xfd\x3e\xc9\xbe" + + "\xdb\x55\x01\x55\x4d\xd5\x66\xd5\x65\xfb\x49\xfb\xb3\xf7\x3f\xae" + + "\x89\xaa\xe9\xf8\x96\xfb\x6d\x5d\xad\x4e\x6d\x71\xed\xc7\x03\xd2" + + "\x03\xfd\x07\x23\x0e\xb6\xd7\xb9\xd4\xd5\x1d\xd2\x3d\x54\x52\x8f" + + "\xd6\x2b\xeb\x47\x0e\xc7\x1f\xbe\xfe\x9d\xef\x77\x2d\x0d\x36\x0d" + + "\x55\x8d\x9c\xc6\xe2\x23\x70\x44\x79\xe4\xe9\xf7\x09\xdf\xf7\x1e" + + "\x0d\x3a\xda\x76\x8c\x7b\xac\xe1\x07\xd3\x1f\x76\x1d\x67\x1d\x2f" + + "\x6a\x42\x9a\xf2\x9a\x46\x9b\x53\x9a\xfb\x5b\x62\x5b\xba\x4f\xcc" + + "\x3e\xd1\xd6\xea\xde\x7a\xfc\x47\xdb\x1f\x0f\x9c\x34\x3c\x59\x79" + + "\x4a\xf3\x54\xc9\x69\xda\xe9\x82\xd3\x93\x67\xf2\xcf\x8c\x9d\x95" + + "\x9d\x7d\x7e\x2e\xf9\xdc\x60\xdb\xa2\xb6\x7b\xe7\x63\xce\xdf\x6a" + + "\x0f\x6f\xef\xba\x10\x74\xe1\xd2\x45\xff\x8b\xe7\x3b\xbc\x3b\xce" + + "\x5c\xf2\xb8\x74\xf2\xb2\xdb\xe5\x13\x57\xb8\x57\x9a\xaf\x3a\x5f" + + "\x6d\xea\x74\xea\x3c\xfe\x93\xd3\x4f\xc7\xbb\x9c\xbb\x9a\xae\xb9" + + "\x5c\x6b\xb9\xee\x7a\xbd\xb5\x7b\x66\xf7\xe9\x1b\x9e\x37\xce\xdd" + + "\xf4\xbd\x79\xf1\x16\xff\xd6\xd5\x9e\x39\x3d\xdd\xbd\xf3\x7a\x6f" + + "\xf7\xc5\xf7\xf5\xdf\x16\xdd\x7e\x72\x27\xfd\xce\xcb\xbb\xd9\x77" + + "\x27\xee\xad\xbc\x4f\xbc\x5f\xf4\x40\xed\x41\xd9\x43\xdd\x87\xd5" + + "\x3f\x5b\xfe\xdc\xd8\xef\xdc\x7f\x6a\xc0\x77\xa0\xf3\xd1\xdc\x47" + + "\xf7\x06\x85\x83\xcf\xfe\x91\xf5\x8f\x0f\x43\x05\x8f\x99\x8f\xcb" + + "\x86\x0d\x86\xeb\x9e\x38\x3e\x39\x39\xe2\x3f\x72\xfd\xe9\xfc\xa7" + + "\x43\xcf\x64\xcf\x26\x9e\x17\xfe\xa2\xfe\xcb\xae\x17\x16\x2f\x7e" + + "\xf8\xd5\xeb\xd7\xce\xd1\x98\xd1\xa1\x97\xf2\x97\x93\xbf\x6d\x7c" + + "\xa5\xfd\xea\xc0\xeb\x19\xaf\xdb\xc6\xc2\xc6\x1e\xbe\xc9\x78\x33" + + "\x31\x5e\xf4\x56\xfb\xed\xc1\x77\xdc\x77\x1d\xef\xa3\xdf\x0f\x4f" + + "\xe4\x7c\x20\x7f\x28\xff\x68\xf9\xb1\xf5\x53\xd0\xa7\xfb\x93\x19" + + "\x93\x93\xff\x04\x03\x98\xf3\xfc\x63\x33\x2d\xdb\x00\x00\x00\x06" + + "\x62\x4b\x47\x44\x00\xff\x00\xff\x00\xff\xa0\xbd\xa7\x93\x00\x00" + + "\x00\x09\x70\x48\x59\x73\x00\x00\x5c\x46\x00\x00\x5c\x46\x01\x14" + + "\x94\x43\x41\x00\x00\x00\x07\x74\x49\x4d\x45\x07\xda\x07\x17\x01" + + "\x35\x2b\x0f\xd0\x77\x1e\x00\x00\x20\x00\x49\x44\x41\x54\x78\xda" + + "\xec\x9d\x77\x78\x14\x55\xf7\xc7\x3f\x77\x66\xb6\xa4\x27\x90\x84" + + "\xd0\x09\x91\x0e\x21\x48\x11\x50\x44\x44\x51\x29\x8a\x08\x82\x8a" + + "\x8d\x57\xc5\x42\x11\x7c\x6d\x88\x88\x08\x2a\x8a\x88\x8a\x28\x2a" + + "\x22\xfa\x53\x50\xe1\xa5\x08\x58\xe9\x0a\x08\x48\x91\x26\x48\x09" + + "\xbd\x24\x24\x21\x6d\x37\xbb\x33\x73\x7f\x7f\x6c\xb2\x06\x69\x01" + + "\x12\x08\x9b\xf9\x3e\x0f\x0f\xbb\x9b\x29\x77\xce\x9c\xef\x3d\xe7" + + "\xdc\x72\x0e\x58\x08\x38\xa8\x6a\xfc\x69\xff\x56\xab\x96\x14\x8a" + + "\x12\x5f\x1e\xc2\x13\x14\x25\xb6\x96\x10\x75\x23\x2c\x89\x05\x3e" + + "\x84\x25\x82\xcb\x1b\x36\x5b\x3d\xbc\xde\xad\x85\x7e\x89\x8d\x15" + + "\x22\xeb\x2a\x29\x65\x5d\x30\x1a\x09\xa1\x46\x4b\xe9\xa9\x00\x34" + + "\x00\xd3\x71\x86\x2b\xfd\x8d\xd0\x66\x0a\xcc\x0d\x42\x84\xaf\x37" + + "\xcd\x94\x2d\x05\x7f\x89\x8a\xba\x91\xf4\xf4\x9f\x8b\xb5\xdd\x8a" + + "\x52\x0d\xd3\xdc\x6b\xbd\x40\x8b\xe8\x16\xce\x86\x16\x2d\x0e\x8b" + + "\xd5\xab\xeb\x35\x10\xc2\xb8\xd3\x34\x73\x3a\x82\x6c\x0a\x26\xd5" + + "\x2b\xc7\xe2\x74\xd8\xb1\xdb\x6c\x84\x04\x3b\x89\x8a\x08\x23\xb1" + + "\x6e\x3c\x0d\x6b\xd7\x20\xb1\x5e\x4d\xc2\x42\x83\xf8\x6b\xc7\x3e" + + "\x34\x55\xc5\x61\xb7\x61\x48\x93\x29\x33\x7e\x66\xd7\x9e\x83\x32" + + "\xd7\x95\x27\xdc\x1e\x0f\x9b\xb7\xef\x03\x6c\x4b\x85\xd0\xde\x53" + + "\x14\xfb\x46\xc3\x48\xdf\x56\x5c\xed\xb6\xdb\x1b\x87\x79\x3c\x7f" + + "\xc7\x02\x37\x82\xfb\x03\xdf\xaf\xda\x8f\x42\x38\x92\x15\xc5\x3e" + + "\x23\x24\xe4\x96\x25\x99\x99\xff\xe7\x01\xe8\xd3\xc7\xcb\xa7\x9f" + + "\xda\xac\x97\x6d\x11\xbd\xac\x21\x26\x42\x51\x8c\x44\x29\xdd\x2f" + + "\x49\x99\xdb\x1e\xa0\x5c\x64\x98\xac\x18\x5b\x5e\x74\xb9\xa1\x25" + + "\x8f\xf4\xea\x88\x2b\xcf\x83\x34\x25\xa6\x94\x98\xa6\x89\x94\x12" + + "\x21\x44\x21\x6b\x2a\x4e\x78\xfd\x52\x4a\xa4\x94\xbe\x5f\x84\x40" + + "\x08\x41\xae\x3b\x8f\x41\x23\x3e\x60\xe7\x9e\x83\x1c\x3d\x96\x91" + + "\x7f\x6c\xd0\x5c\x10\x23\x54\x35\x6e\xab\x61\xec\xcc\x3e\x67\x65" + + "\x13\x61\xcd\xc1\x98\x29\x65\x6e\xe5\xfc\x9f\xe4\xbf\x74\xb0\xd0" + + "\x77\xfb\x0e\x9b\x2d\xfa\x25\x55\x6d\xf4\xb5\xdb\xbd\xdc\x80\x2c" + + "\xeb\xd5\x5b\x44\x0f\x2c\x68\x5a\x02\xba\xbe\xb3\x80\x1c\xe5\x34" + + "\x2d\x2a\x49\xd7\xd3\x1e\x90\xd2\x75\x2f\x18\x28\x8a\x20\xa6\x5c" + + "\x04\xcf\x3c\xda\x8b\x9b\xae\x6d\x8a\x22\x04\x1e\xaf\x7e\xc2\x35" + + "\xa4\x94\xf9\x84\x16\x98\xa6\x49\x56\x8e\x8b\x5c\x57\x1e\x9b\xb7" + + "\x27\xb3\x68\xe5\x06\x42\x82\x1c\x1c\x4a\x49\x43\x11\x82\x3a\x09" + + "\xd5\xb8\xbd\x43\x6b\x2a\x44\x47\xa1\x2a\x0a\x66\x3e\xf1\x85\x10" + + "\xd8\x34\x15\xaf\x61\xb0\xf0\xb7\xf5\x7c\xfc\xf5\x7c\xfe\xda\xb1" + + "\x17\x29\x41\xd3\xaa\xdf\x05\x41\x33\x75\xfd\xaf\xbc\x33\x8c\x16" + + "\x00\x06\x36\x5b\xc5\xe6\x5e\x6f\xca\x32\x55\x35\x1d\x51\x11\xa1" + + "\xc4\x94\x8f\xe4\xd6\xf6\xad\x69\xd9\xa4\x2e\x0e\xbb\x8d\xd0\x60" + + "\x27\xbf\xae\xd9\x8c\xc3\x61\xe7\xed\x49\xd3\xd9\x7f\x28\xb5\x70" + + "\x48\x91\x2a\x44\x54\x55\x29\x8f\xba\x2d\xcd\xb0\x88\x7e\xd9\xc3" + + "\xe9\x6c\x82\xdb\xbd\x2e\x9f\xdc\x11\x4e\x45\x09\xb9\xd3\x30\x8e" + + "\x4d\x01\x1f\x8f\xec\x36\x8d\x1e\x9d\xae\xe5\xc1\xee\x37\x11\x17" + + "\x53\x0e\xc3\x34\x30\xcd\x13\x2d\x75\x61\xe4\xe4\xba\xf9\xf4\xdb" + + "\x1f\x59\xb3\x71\x3b\x1b\xb6\xec\x2c\x72\x3b\xaa\x56\x8a\xe5\xa5" + + "\x01\xbd\x69\x5c\x3f\x01\x9b\xa6\x9e\xe4\x0d\x98\xa6\xc9\x0f\x4b" + + "\xd6\x30\x72\xfc\x57\xe4\xba\xf2\x50\x94\x98\x0e\x42\x98\xbf\x1a" + + "\x46\xaa\xeb\xdf\xd7\x6a\xd5\x6a\x16\x2b\x57\xde\xb5\x4d\x4a\x57" + + "\xed\x8a\xb1\x51\x7c\x3f\xe5\x75\x44\xfe\x35\x4e\xd7\x6e\x80\x94" + + "\xb4\xe3\x0c\x1d\x33\x99\x55\x1b\xfe\x42\x4a\x24\x08\x61\xb3\xd5" + + "\xe8\x12\x1e\xde\x6e\xde\xb1\x63\x73\x24\xa4\x5a\x0a\x63\x11\xfd" + + "\xf2\x42\x58\x58\x6b\xb2\xb2\x96\xa3\x69\x35\x11\xc2\xdb\x40\xd7" + + "\x53\xd6\x4b\xe9\xd6\x00\x19\x16\x1a\x2c\x3e\x7e\x6d\x10\xb5\xe2" + + "\x2b\xa3\xa9\xea\x69\x09\x52\x98\x8c\x1e\xaf\x4e\xc7\x07\x86\x90" + + "\x9a\x9e\xf9\xaf\xc1\x2f\x85\x88\x88\x08\x6e\xb8\xe1\x06\x9a\x36" + + "\x6d\xca\xaa\x55\xab\xd8\xba\x75\x2b\xdb\xb6\x6d\x3b\xc1\x75\x2f" + + "\x8c\x51\x4f\xf7\xa1\x53\xbb\x16\xa7\xbd\x5f\x46\x66\x36\xd7\xf5" + + "\xfa\x6f\xfe\xaf\x41\x13\xed\xf6\x6a\xfd\x3d\x9e\x6d\xde\x42\xd6" + + "\xf8\x0f\xf0\x36\x19\xf0\xe0\xed\xe2\xe1\x9e\xb7\x60\x98\xe6\x39" + + "\xc9\xc6\x9d\xe7\x61\xe0\xcb\x13\xf8\x7d\xfd\x5f\xf9\xbf\x04\xa7" + + "\x40\x6e\x2c\x40\x48\x48\x1b\x72\x72\x96\x59\x0a\x64\x11\xfd\x72" + + "\xb0\xe2\x0d\x70\xbb\x37\xa3\x69\x55\xba\xea\xfa\x81\xaf\x41\xda" + + "\x01\x9e\xee\x7b\x27\xbd\xba\x5c\x87\x9a\xef\x7a\xff\x9b\xcc\x27" + + "\xc7\xbe\xb0\x73\xcf\x21\x9e\x7f\x63\x12\xdb\x77\x1f\x38\x81\xb4" + + "\xbf\xfc\xf2\x0b\xd7\x5e\x7b\x2d\x8a\xa2\xa0\xaa\xea\x49\xe7\x9a" + + "\xa6\xe9\xff\xf7\xcb\x2f\xbf\x30\x7e\xfc\x78\xbe\xff\xfe\xfb\x13" + + "\x8e\x79\xb0\xc7\xcd\x0c\xec\x73\x3b\xb2\x50\x47\x53\xd0\x1e\xd3" + + "\x34\xf9\xf4\xdb\x9f\x18\x3f\x65\x16\x80\x14\x22\x62\x80\x94\xc7" + + "\xc7\x2b\x4a\xe4\x17\xa6\x99\xd1\xfb\xe7\x2f\x46\x13\x53\x3e\xe2" + + "\x8c\xed\x3f\x13\xa4\x94\x1c\xcb\xc8\xe4\x86\x7b\x9e\xcd\x8f\xdf" + + "\x15\x77\x64\x64\x8f\xc8\x8c\x8c\xaf\xf3\x2c\x0d\xb2\x88\x5e\xaa" + + "\x11\x17\xf7\x3c\x87\x0f\xbf\x86\xa6\xd5\xbc\x51\xd7\x93\xbf\x03" + + "\xd3\xe1\xb0\xdb\xf8\x7c\xec\xb3\xd4\xac\x56\xf1\x94\x2e\xf3\xe9" + + "\x48\x60\xb7\xdb\xe8\x3b\x64\x1c\xbf\xae\xde\x74\xc2\xdf\x9e\x7d" + + "\xf6\x59\x46\x8e\x1c\x89\xa6\x69\xe7\x4c\x32\x8f\xc7\xc3\xef\xbf" + + "\xff\xce\xd0\xa1\x43\x59\xba\x74\xa9\xff\xf7\x69\xe3\x87\x52\x2f" + + "\xa1\x0a\xa7\x30\xfe\xe4\x79\xbc\xf4\x78\xfc\x15\xb9\xf7\xe0\x51" + + "\xff\x8d\xde\x1a\xfa\xa8\x6c\xdf\x3a\xa9\x58\x74\xcc\x34\x4d\x7e" + + "\x5d\xb3\x99\x01\xc3\xdf\x07\x84\x19\x1c\xdc\x2e\x3e\x37\x77\xa1" + + "\x35\x3f\x67\x11\xbd\xf4\x22\x2a\x6a\x40\x58\x7a\xfa\xfb\x5b\xc0" + + "\xa8\x52\xb9\x62\xb4\x7c\xe3\xb9\x87\x45\xfd\x2b\xaa\x9d\x13\x19" + + "\x0b\xc8\xdb\xfe\x9e\x67\x38\x56\xc8\x4d\x7f\xec\xb1\xc7\x18\x3d" + + "\x7a\x34\x61\x61\x61\x17\x64\x45\x0b\xce\xcb\xcb\xcb\xa3\x5a\xb5" + + "\x6a\x1c\x3d\x7a\x14\x80\x9b\xdb\x36\xe3\xdd\x97\x9e\x20\xc7\xe5" + + "\x3e\xe1\xda\xbe\xc1\x3f\x85\xbf\x76\xee\xa5\x67\xbf\x51\x00\x34" + + "\x4b\xac\xcd\xd4\x77\x87\x90\x9d\xe3\x3a\xa9\x1d\x05\xc3\xeb\x8a" + + "\x22\x30\x4d\x89\x94\x3e\xcf\xe4\x6c\x6d\x5a\xba\x6a\x63\x3e\xd9" + + "\x15\x5d\x88\xb0\xd0\x88\x88\x2e\x79\x19\x19\xff\x67\x29\x95\x45" + + "\xf4\x4b\x8f\x3e\x7d\x3c\x7c\xfa\xa9\x1d\x00\x55\x8d\xfc\xd0\x30" + + "\x32\xfa\x0e\x1b\xd0\x9b\xab\x9b\x35\xa4\x72\x85\xf2\xe8\x86\x71" + + "\xce\x44\x4c\x3b\x9e\x4d\x97\xff\x0c\x25\xd7\xe5\xf3\x60\x27\x4e" + + "\x9c\x48\xef\xde\xbd\x09\x0e\x0e\x3e\x6f\x82\x9f\xee\x5e\x52\x4a" + + "\xb6\x6c\xd9\x42\xbb\x76\xed\x48\x4d\x4d\x25\x34\x38\x88\x25\x5f" + + "\xbf\x85\xa2\x88\x53\xde\xe7\x50\x4a\x3a\x1d\x1f\x18\x82\x94\x92" + + "\xea\x55\x2a\xf0\xfd\xe4\x51\xe4\x79\xbc\xf9\xcf\xaf\x90\x93\xeb" + + "\xe6\xe3\x69\xdf\xb3\x75\xc7\x1e\x0e\xa7\xa4\x53\x2e\x32\x9c\x1e" + + "\x1d\xaf\xa5\x4b\xfb\xab\xf0\x78\xf5\xd3\xb6\xbd\xe0\xb9\x86\xbc" + + "\xf9\x29\xf3\x17\xad\x92\x9a\x16\xfb\xa0\xae\x1f\x9d\x62\x69\x98" + + "\x45\xf4\x52\x84\x90\xb7\x21\xe7\xde\x7b\xbb\xdd\x58\x7e\x50\x9f" + + "\xdb\xa5\xaa\xaa\xe2\x6c\xa3\xcf\xa7\x53\xf6\x15\xeb\xfe\xe2\x89" + + "\x17\xdf\x41\x4a\x68\xd2\xa4\x09\xf3\xe7\xcf\x27\x2e\x2e\xae\x44" + + "\x5b\x5f\x40\xb2\xce\x9d\x3b\x33\x6f\xde\x3c\x6a\x54\xa9\xc0\x0f" + + "\x53\x5e\xc3\x9d\xe7\x39\xe5\xf1\x87\x52\xd2\xb9\xe5\xfe\xe7\x01" + + "\xa8\x55\xa3\x32\x3f\x7e\xfe\x1a\xba\x61\xf0\xd8\xd0\x77\x59\xf0" + + "\xdb\xba\x42\xe3\x14\x4e\xdc\x6e\xdf\xcc\x59\xed\xf8\x2a\xfc\xdf" + + "\xb8\xe7\xb0\x69\xea\x19\xe5\xe2\xf1\xea\xb4\xb8\xad\x1f\x60\x9b" + + "\x0d\xde\xae\x96\x6e\x59\x44\xbf\x64\x70\x38\xe2\xc9\xcb\xdb\x8d" + + "\xaa\x56\xed\x68\x18\xfb\xbe\x6d\xd5\xa4\x5e\xf0\xe0\x87\xba\xcb" + + "\xba\x09\x55\x85\x6e\x18\xe7\xed\x52\xef\xdc\x7b\x88\xee\x8f\x8d" + + "\xf0\xc5\xcc\xd3\xa6\xd1\xb3\x67\xcf\x62\xb5\xe0\x67\xbb\x7f\x5e" + + "\x5e\x1e\x95\x2b\x57\x26\x2d\x2d\x8d\xce\xed\x5b\x32\xea\xbf\x0f" + + "\x9e\x72\xc4\x1e\x29\x39\x72\xec\x38\x37\xdf\xff\x3c\x52\x4a\x42" + + "\x83\x9d\xe4\x79\xbc\x78\x75\x9f\xf7\x22\x84\x40\x4a\xc9\xd6\xad" + + "\x5b\x09\x0e\x0e\x26\x3e\x3e\x1e\xd3\x34\x69\xda\xa8\x36\x53\xc6" + + "\x3c\x7d\x46\x2f\x47\x4a\xc9\xd5\xdd\x07\xe1\x72\x8b\xa3\x52\xba" + + "\x2a\x58\xda\x76\x76\x28\x96\x08\x4a\x06\x0d\x1b\x7e\xa4\x0a\xe1" + + "\xf8\xd1\x30\xf6\xcd\x7b\xef\xe5\x7e\x41\x93\xde\x78\x8a\x5a\xf1" + + "\x95\x85\x71\x1e\x56\xbc\x00\x4b\x57\x6d\xf2\x93\x7c\xe0\xc0\x81" + + "\xf4\xec\xd9\xd3\x4f\x9a\x8b\x62\x15\x84\xc0\xe1\x70\xb0\x77\xaf" + + "\x6f\x0c\x6c\xee\x82\x95\xfc\xb8\x74\xcd\xa9\x89\x2e\x04\x71\x31" + + "\x51\x3c\xd0\xbd\x03\x00\xd9\xb9\x6e\x3f\xc9\x01\xec\x76\x3b\x5b" + + "\xb7\x6e\xa5\x5e\xbd\x7a\xdc\x71\xc7\x1d\x28\x8a\x4f\x15\xff\xd8" + + "\xb8\x9d\xa5\xab\xfe\x3c\xf5\x35\x0b\xb5\xa3\x63\xbb\x16\x48\xe9" + + "\x8a\x75\x3a\x3b\x58\xeb\x63\x2d\x8b\x7e\x71\x61\xb3\xc5\xe0\xf5" + + "\xa6\x20\x44\x68\x1f\x29\x5d\x93\x22\x23\x82\xe4\x9c\x8f\x47\x88" + + "\xf0\x90\x60\x24\xe7\x6f\x75\xa5\x94\xfc\xbe\x7e\x1b\x8f\xbe\x30" + + "\xce\xe7\xe2\xd6\xae\xed\x9f\xff\xbe\x58\x24\xff\x37\x76\xef\xde" + + "\x4d\xcd\x9a\x35\x01\xf8\xf5\xdb\xb7\x09\x0d\x09\x3a\xe5\x71\x79" + + "\x1e\x2f\x77\x3c\x3a\x82\xfd\x87\x53\x4e\xf9\xf7\xef\xbf\xff\x9e" + + "\x5b\x6e\xb9\xe5\x84\xdf\x6a\x56\xab\xc8\xfc\xc9\xa3\x4e\x1b\x16" + + "\x00\xcc\x5b\xf8\x3b\x2f\x8c\x99\x0c\x04\xc7\x41\xee\x11\x4b\xfb" + + "\x2c\x8b\x7e\xd1\x20\xa5\xb3\x9a\x10\x8e\x34\x29\xb3\x27\x5d\xd3" + + "\xac\x9e\x5c\x3e\x7d\x9c\x08\x0b\x09\x02\x71\x61\x56\xd7\xe9\xb0" + + "\x33\x60\xf8\x78\xff\xf7\xf9\xf3\xe7\x5f\x52\x92\x03\xc4\xc7\xc7" + + "\xf3\xf4\xd3\x4f\x03\xd0\xb5\xef\xf0\xd3\x5a\x60\xbb\x4d\x63\xd6" + + "\x47\xc3\x29\x17\x15\x7e\xca\xbf\xff\x9b\xe4\x00\xbb\xf6\x1e\xe2" + + "\x70\x4a\xfa\x19\xef\x9f\xeb\xce\x2b\x70\x1c\x82\x2c\xcd\xb3\x88" + + "\x5e\xe2\xb0\xdb\x7d\x7b\xbf\x55\x35\xfa\x4e\x5d\xdf\xbf\x27\x22" + + "\xcc\x16\xf9\xc5\xdb\xcf\xf2\xe1\xa8\x81\xe2\x4c\x23\xc8\xe7\x62" + + "\xcd\x27\x7d\xf3\xa3\x7f\x2d\xfb\xa8\x51\xa3\x48\x48\x48\x28\x71" + + "\x92\x17\x10\x57\xd7\x75\x66\xcd\x9a\xc5\xf2\xe5\xcb\xd9\xb4\x69" + + "\x13\xc9\xc9\xc9\xfe\x63\x9e\x7c\xf2\x49\x00\x52\xd3\x8e\x93\x99" + + "\x9d\x7b\x5a\x37\x5b\x55\x15\x7e\xfe\xfc\x75\xea\x26\x54\x2d\xf2" + + "\xfd\x5d\xee\x33\xaf\x87\x71\xe5\xcf\x36\x48\xe9\x35\x2d\x2d\xb4" + + "\x88\x5e\xe2\xf0\x78\xd2\x42\x15\x25\x74\xa7\x61\xa4\x7e\x9d\x54" + + "\xbf\x26\x2b\xff\xf7\x8e\x68\x58\xbb\xc6\x39\x4f\x99\x9d\x8e\x6c" + + "\x8a\xa2\xf0\xf6\x27\xd3\x01\x70\x38\x1c\x3c\xf9\xe4\x93\x67\x8c" + + "\x5f\x8b\x33\x1e\x9f\x30\x61\x02\x36\x9b\x8d\xdb\x6f\xbf\x9d\xab" + + "\xaf\xbe\x9a\x46\x8d\x1a\x11\x1f\x1f\x4f\x4c\x4c\x0c\xcb\x97\x2f" + + "\xa7\x52\xa5\x4a\xfe\xe3\x3b\xfd\x67\xe8\x19\xaf\xa5\x28\x82\xe9" + + "\x13\x86\x71\xed\x55\x89\x67\xbd\x77\x85\xe8\x28\xaa\x57\x8e\x3d" + + "\x4b\x98\xa4\xe5\x5f\xdb\x66\xe9\xb0\x45\xf4\x92\x42\x0d\x00\x42" + + "\x43\x5b\x94\x87\xec\x4c\xbb\xcd\x13\xff\xf9\xd8\x67\xf8\xe2\xed" + + "\x67\x71\xe5\x79\x8a\xcd\xda\x0a\x21\xf8\x61\xf1\x6a\xff\x1a\xf1" + + "\x3d\x7b\xf6\x10\x14\x14\x54\x62\xd6\xbc\xa0\x03\xd9\xbb\x77\x2f" + + "\x41\x41\x41\x3c\xf1\xc4\x13\xa7\x3c\x2e\x35\x35\x95\x5b\x6e\xb9" + + "\x85\x5e\xbd\x7a\x71\xeb\xad\xb7\x02\x90\x9d\xed\x3a\x61\xe1\xce" + + "\xa9\x9e\xc5\xab\xeb\xbc\x37\xfc\x09\x66\x4d\x1c\x7e\xc6\x76\xf4" + + "\xbd\xbb\xd3\x49\xbb\xf1\x4e\xdd\x56\x81\x69\xe6\x24\x5b\xfa\x68" + + "\x11\xbd\x98\xdd\xf4\x3a\x00\x68\x5a\x05\x45\x88\x98\xa1\xd9\xd9" + + "\xab\x52\xef\xbf\xa3\x3d\x7f\xcc\x9d\x20\x1a\xd5\x89\xc7\x30\xcc" + + "\x62\x25\xa1\x94\x92\xa1\x63\x3f\x03\xa0\x57\xaf\x5e\x54\xa8\x50" + + "\xa1\x44\x5d\x76\x21\x04\x2b\x56\xac\xa0\x7a\xf5\xea\xfe\xb9\xed" + + "\xd3\x61\xc0\x80\x01\x7c\xfd\xf5\xd7\x0c\x1e\x3c\xd8\xff\xdb\xb0" + + "\xb7\xa7\xa0\x9c\xa1\x7d\x05\x53\x6a\x35\xaa\xc6\xb1\x7c\xc6\x3b" + + "\x54\x8c\x2d\x7f\xd2\x31\xdd\x6e\xbe\x86\x9e\x9d\xdb\x9e\xb5\xad" + + "\xb3\x7f\x5e\x0e\x38\x67\x5c\xca\x71\x8a\xcb\x09\x96\x94\x8a\x80" + + "\xc4\xc4\x8d\xfc\xf9\x67\xa3\x7c\x97\x31\xa1\x92\xd7\xbb\x7b\x57" + + "\x70\x90\xcd\xb1\xe8\xab\x37\x71\x38\xec\x25\x66\x5d\x4d\xd3\xa4" + + "\xd9\xad\x4f\x20\x25\xe4\xe4\xe4\x10\x1c\x1c\x5c\xa2\x31\xb9\x6f" + + "\xed\xbc\x1d\xe3\x1c\xc2\x8e\xb0\xb0\x30\xda\xb7\x6f\xcf\xac\x59" + + "\xb3\x00\x58\x3f\xff\xc3\x22\xdf\x4f\x08\xc1\xd4\x39\x8b\x18\xfd" + + "\xe1\xd7\xfe\xdf\xe7\x7c\x32\x82\x6a\x95\x62\xcf\x7a\x6e\xb3\x2e" + + "\x4f\x80\xa8\xd0\xdb\x30\x0e\x7d\x69\x69\xa8\x65\xd1\x8b\x05\x7f" + + "\xfe\xd9\x88\xda\xb5\x97\x68\x10\xba\xd4\xeb\xdd\x79\x60\x68\xff" + + "\xbb\x1c\xbf\x4d\x1f\x27\x4b\x8a\xe4\x05\xd6\x2f\x2b\xd7\x85\x94" + + "\xd0\xb0\x61\xc3\x12\x25\x79\xc1\xfd\xd2\xd3\xd3\xcf\x89\xe4\x00" + + "\x59\x59\x59\xcc\x98\x31\xe3\x84\x5d\x6d\x45\xbd\x1f\x48\xee\xba" + + "\xb5\x1d\x0b\xbf\x7a\xd3\xff\xfb\x6d\x0f\xbf\x44\x8e\xcb\x7d\xc6" + + "\xeb\xbc\xf3\xd9\x2c\x0c\xd3\xc4\x34\xdd\x73\x2c\xed\xb4\x88\x7e" + + "\xc1\xb0\xd9\xaa\xe5\x2b\x65\x85\xfb\xb6\x6f\x6f\xeb\xb1\xd9\xdc" + + "\x6d\x16\x7f\x3d\x46\x76\xbf\xa5\xcd\x45\xf1\x86\xd6\x6e\xda\x01" + + "\xf8\x76\xa2\x5d\x0c\x8c\x1d\x3b\xb6\x48\xc7\x75\xed\xda\x95\x0f" + + "\x3e\xf8\xc0\xff\x7d\xcb\x96\x2d\xfe\x2d\xb0\x69\xc7\xb3\xce\x61" + + "\xb0\xd0\x27\xc2\x72\x91\x61\x2c\xf9\xfa\x2d\x7f\x47\x71\xf5\x1d" + + "\x4f\xb2\xe7\xc0\xd1\xd3\x5e\x67\xf6\x4f\xbf\x49\x5f\x08\x15\xac" + + "\x5b\x5a\x6a\x11\xfd\xbc\xa1\x69\xad\x00\x50\xd5\xb8\x7a\xa0\x65" + + "\x4a\x79\x64\x4a\x9b\xab\x12\x59\x3d\x7b\x3c\x91\x61\xa1\xe2\x62" + + "\xcd\x61\xff\xb1\xf1\x6f\x00\xda\xb7\x6f\x7f\x51\x9e\xfb\x6c\x71" + + "\x39\x40\xcf\x9e\x3d\x99\x39\x73\x26\x8f\x3e\xfa\x28\x15\x2b\x56" + + "\xcc\x1f\xbb\xb0\xd3\xa9\x53\x27\x00\xf6\x1f\x4a\x3d\x2f\xd9\x44" + + "\x84\x85\xf0\xc3\xe7\xaf\xf9\xcf\xbd\x6b\xc0\x28\x32\xb3\x73\x4f" + + "\x22\xbb\xd7\xab\x93\x7e\x3c\x5b\xa8\x6a\xf4\x44\xaf\xf7\x80\xcb" + + "\xd2\x56\x8b\xe8\x17\x60\xc9\xb3\x1b\x83\x6d\xb7\xdb\xbd\x6a\x0b" + + "\xe8\xa1\xad\x9a\xd4\xe7\x83\x11\xfd\x45\xe1\x04\x8a\x25\x0d\x29" + + "\x25\x5f\xce\x5a\x00\x40\x78\x78\xf8\x45\x99\x52\x0b\x0a\x3a\xfb" + + "\xda\x93\xc2\xcf\x7e\xe8\xd0\x21\x34\x4d\xa3\x76\xed\xda\x3c\xf3" + + "\xcc\x33\xbe\x30\xe7\xaf\x5d\xe7\x2d\x9f\xb8\xe8\x28\x66\x7f\xec" + + "\x5b\xe2\xeb\x72\x7b\x78\x6c\xe8\xbb\x84\x85\x06\x9f\xf0\xec\x7f" + + "\xef\x39\x08\x20\x21\x6a\x90\xa5\xa9\x16\xd1\xcf\x19\xc1\xc1\xcd" + + "\xf2\x15\x39\x74\x9c\xcb\xb5\x71\x3d\x78\xab\x03\xbc\x34\xf0\x5e" + + "\x31\xe9\x8d\xc1\x78\x75\xfd\xa2\xae\x44\x2b\x9c\x76\x49\x51\x94" + + "\x8b\x72\xef\xeb\xae\xbb\xee\xac\xc7\x4c\x9b\x36\xcd\x9f\x29\xb6" + + "\xa0\x6d\x00\xd5\xab\x57\x07\xe0\xc7\x25\x6b\x0a\x65\x99\x3d\x77" + + "\x54\xab\x14\xc3\xc2\xaf\xde\xa4\x79\x62\x1d\xb6\xfc\xbd\x87\xeb" + + "\x7a\xfd\x97\x88\xb0\x90\xfc\x77\x03\x53\x67\x2f\x04\x54\x2f\x04" + + "\x59\xd6\xdc\x22\xfa\xb9\x23\x3a\xfa\xae\x50\x70\xa4\x48\x99\x3d" + + "\x10\x90\xb5\x6a\x54\x16\xf3\x3e\x1d\xc9\x1d\x37\x5f\x43\x71\xac" + + "\x70\x3b\x57\x78\x0b\xcd\x23\x6b\x9a\x76\x51\x3c\x88\xeb\xaf\xbf" + + "\xfe\xbc\xcf\xaf\x5c\xb9\xb2\x3f\x46\x17\x17\x38\x7c\x11\x15\x11" + + "\xca\xc4\x57\x9f\x64\xd2\xe8\xc1\x1c\x49\x4d\xe7\xa1\xe7\xc6\x12" + + "\x1a\xe2\x5b\x3f\x30\x7f\xf1\x6a\x14\x25\x78\xa6\x61\xfc\x69\x29" + + "\xad\x45\xf4\xa2\xba\xaa\x0d\xf2\x63\xf1\x0a\x4f\xef\xdd\xfb\xdf" + + "\x14\xc8\x2b\x0f\x70\x7d\xab\x24\x31\x6f\xf2\x48\x2a\x55\x28\x8f" + + "\x79\x11\x5c\xe6\xb3\x59\x74\x5d\x2f\xf9\x31\x27\xdf\xea\x35\x85" + + "\xf7\xdf\x7f\xff\x9c\xce\x9b\x35\x6b\xd6\x09\xae\x75\x48\xb0\x13" + + "\x89\x2c\x86\xb6\x08\x9a\x36\xac\xc5\x17\x63\x9f\x65\xe1\xf2\xf5" + + "\xb4\xe9\x3e\x88\x57\xdf\x9f\x8a\x61\x18\x68\x5a\xe2\x63\x16\x75" + + "\x2d\xa2\x17\x19\x2e\xd7\x66\x84\x08\xfe\xcb\x30\x8e\xbc\x01\xd2" + + "\x11\x12\xec\x14\x23\xff\xfb\x20\x13\x46\x0e\x20\x27\xd7\x7d\x72" + + "\xfa\xa3\x42\x0a\x5d\xd2\x03\x72\x41\x8e\x7f\xaa\x27\x19\xc5\xb0" + + "\x9c\xb6\xa8\x56\xfd\xf1\xc7\x1f\xa7\x59\xb3\x66\x67\x3d\xd6\x6e" + + "\xb7\xfb\x77\x9e\x15\xc8\x41\x08\x41\xe5\x0a\xd1\x20\x8b\xad\xf7" + + "\xa1\x61\xed\x1a\x0c\x1f\x74\x1f\x87\x52\xd2\x98\xf6\xdd\x62\xc0" + + "\x9e\xec\xf1\xfc\x96\x6e\x51\xf7\xdc\xa0\x95\xe5\x87\x17\x22\x74" + + "\x87\x94\xd9\x09\x00\x5d\x6e\x68\x29\xde\x78\xee\x61\xf2\x3c\x5e" + + "\xff\xf6\x48\x21\x40\x55\x55\xb2\x73\xdc\xfc\x9d\xbc\x9f\x8c\xe3" + + "\xd9\x1c\xcf\xca\xc1\x30\x4d\xa2\xa3\x22\xd8\xb9\xf7\x10\x31\xe5" + + "\x22\xa8\x12\x17\x4d\xd5\x4a\x31\xc4\xc5\x44\x91\xe7\xf1\x52\x1c" + + "\x4e\x80\xaa\x2a\x84\x06\x3b\xc9\xce\x75\x93\x9e\x9e\x4e\x68\x68" + + "\xe8\x45\xb1\xea\x52\x4a\x56\xaf\x5e\xcd\xbc\x79\xf3\x78\xfa\xe9" + + "\xa7\xd9\xba\x75\xeb\x49\xc7\x35\x6d\xda\x94\xe5\xcb\x97\x63\xb3" + + "\xd9\x4e\xea\x28\x6c\xc5\x1c\x66\xe8\x86\x41\xf7\x9b\xdb\xb0\x73" + + "\xcf\x21\xbe\xf8\xdf\xcf\x12\xbc\x95\x2d\xda\x9e\xc7\xbb\x2d\x4b" + + "\x0f\x1b\x1a\x7a\x2d\xd9\xd9\x4b\xb1\xdb\x93\xe2\x3c\x9e\x2d\x9f" + + "\x83\xe7\x46\xa7\xc3\xce\x7b\x2f\x3f\x41\xcb\xa4\x7a\xfe\x8d\x28" + + "\xaa\xa2\x70\xec\x78\x16\x4b\x56\xfe\xc9\x8a\xb5\x5b\xf8\x71\xe9" + + "\x9a\x22\x5d\x3f\xa6\x7c\x04\x03\x1e\xb8\x9d\x6b\x9a\x35\x24\x3a" + + "\x2a\x9c\xf3\xcd\x24\x53\x80\x45\x2b\x37\x30\x68\xc4\x07\x0c\x1d" + + "\x3a\x94\x57\x5e\x79\xe5\xa2\xca\xaa\xb0\xc7\xb2\x65\xcb\x16\xd6" + + "\xae\x5d\x4b\x74\x74\x34\x49\x49\x49\xc4\xc5\xc5\x9d\xe4\xd1\xfc" + + "\xfc\xf3\xcf\x74\xe8\xd0\x81\x4f\x5e\x1f\x4c\xb3\xc4\xda\xc5\xde" + + "\x16\x55\x55\x18\x39\xfe\x2b\xbe\x99\xbb\x04\x45\x09\xff\xc3\x34" + + "\x33\x9b\x59\xf4\xb5\x88\x7e\x12\x42\x42\xae\x21\x27\xe7\x57\x34" + + "\xad\xd2\xf5\xba\x7e\x70\x01\x40\xfd\x5a\xd5\xf9\x66\xfc\x0b\xfe" + + "\xcc\x27\x8a\xa2\xe0\xf1\x78\x99\xb7\x68\x15\xc3\xc7\x7d\xfe\x8f" + + "\xdb\xa3\x69\x54\xac\x58\x91\x90\x90\x10\xbc\x5e\x2f\x76\xbb\x1d" + + "\x29\x25\xba\xae\x63\x9a\x26\x39\x39\x39\xb8\x5c\x2e\x32\x33\xff" + + "\xd9\xd4\x71\x67\xe7\xb6\x0c\xea\xd3\x8d\x20\xa7\xe3\xbc\x95\xdb" + + "\x30\x4d\x9a\x75\x79\x82\xf2\xe5\xcb\x93\x9a\x5a\xba\xab\x93\x74" + + "\xeb\xd6\x8d\x99\x33\x67\xf2\xd9\x5b\xcf\x90\x54\xaf\x66\x89\x74" + + "\x3c\x86\x69\x72\xed\x9d\x83\xc9\x75\xe5\xa1\xaa\x91\x93\x0d\x23" + + "\xa3\x4f\xbd\x7a\xfb\xd8\xba\xb5\xaa\xc5\x64\x8b\xe8\x10\x1b\xdb" + + "\x9d\xa3\x47\xa7\x63\xb3\x55\x7d\xc2\xeb\xdd\x37\x3e\x2c\x24\x88" + + "\xe9\x1f\x0c\x23\x2e\x26\xca\x5f\xd6\x28\x2f\xcf\xc3\xe4\xe9\x3f" + + "\xf1\xe1\x97\x73\xf3\x3b\x86\x10\x42\x43\x43\xe9\xd7\xaf\x1f\x43" + + "\x87\x0e\x2d\xf2\xbd\xc6\x8f\x1f\xcf\xab\xaf\xbe\xca\x91\x23\x47" + + "\x30\x4d\x93\xf7\x86\xf7\xe3\xea\x66\xf5\xfd\xd3\x50\xe7\x02\xd3" + + "\x34\x69\xda\xe5\x09\xa4\x94\x1c\x3f\x7e\x9c\xb0\xb0\x30\x4a\xdb" + + "\x26\x8e\x82\x71\x0b\xa7\xd3\x89\xc7\xe3\xe1\xe7\x2f\x5e\x27\xa6" + + "\x7c\x64\x89\xdd\x4b\x51\x14\x1a\xdf\xd2\x57\x02\xc2\xe1\xa8\x53" + + "\x27\x2f\x6f\xdb\x76\x8b\xc6\x16\xd1\xd1\xb4\x5a\xe8\xfa\xdf\xd8" + + "\xed\x55\xee\xf0\x78\xf6\x4f\xbf\xa2\x7a\x45\x66\x7f\xfc\x0a\x5e" + + "\x5d\xf7\x2b\xe9\xda\x4d\x3b\xf8\xcf\xb3\x6f\xf9\xad\x7a\xbf\x7e" + + "\xfd\x78\xe7\x9d\x77\x4e\x72\x61\xcf\xc5\xe5\xf5\x78\x3c\x34\x6b" + + "\xd6\x8c\x8d\x1b\x37\x12\xe4\xb4\xb3\x68\xea\x18\x1c\x76\xdb\x39" + + "\x13\xb5\xeb\x23\xc3\x49\xde\x7f\x98\x0d\x1b\x36\x90\x98\x98\x58" + + "\x2a\x65\x9c\x95\x95\x45\x44\x84\xaf\x1a\xcb\xca\x99\xef\xe2\x2c" + + "\xc1\x3d\x00\x00\x33\x7e\xf8\x95\x57\xde\xfd\xbf\xfc\xf7\x5b\x3b" + + "\x42\xd7\xb7\x67\x5a\x54\x2e\xd3\x44\x77\x00\x79\xa8\x6a\xcc\x93" + + "\x86\x91\xf2\x76\xdb\xab\x12\x79\x77\xf8\x13\xfe\x1a\x66\x42\xc0" + + "\x4d\xf7\x3d\xef\x4f\x5b\x14\x17\x17\xc7\xa1\x43\x87\x8a\xb5\x05" + + "\x07\x0e\x1c\xa0\x4a\x95\x2a\xfe\xef\xcb\x67\xbc\x43\x70\x50\xd1" + + "\xdd\xf9\xb5\x9b\x76\xd0\xe7\x99\x31\x74\xea\xd4\x89\xb9\x73\xe7" + + "\x96\x4a\x29\x3f\xf1\xc4\x13\x4c\x98\x30\xa1\xc4\xdc\xf6\x53\x79" + + "\x10\x9d\xff\xf3\x22\x07\x0e\xa7\x4a\x55\x8d\x3c\x64\x18\x19\x95" + + "\x83\x83\x6f\x20\x37\xf7\x17\x8b\xd1\xa7\x41\xc0\x4e\xaf\x69\x5a" + + "\x42\x3e\xc9\xa3\xd6\x1b\x46\xca\xdb\xfd\xee\xbb\x4d\xbe\xf3\xd2" + + "\xe3\x7e\x92\x4b\x29\xb9\xaa\x6b\x7f\x3f\xc9\x27\x4e\x9c\x58\xec" + + "\x24\x97\x52\x52\xb9\x72\x65\x0c\xc3\x60\xc0\x80\x01\x00\xb4\xbe" + + "\x63\x20\x1e\x8f\xb7\x68\x4b\x5a\x25\x34\x6d\x54\x0b\x55\x55\x98" + + "\x37\x6f\x1e\x69\x69\x69\xa5\x4e\xce\x5e\xaf\x97\x09\x13\x26\x00" + + "\x50\x3f\xa1\x6a\x89\x2f\xd5\x2d\x58\x95\x37\x7f\xf2\x28\x6c\x9a" + + "\x2a\x0c\x23\xa3\x92\xa6\x55\xba\xd9\x22\x79\x19\x24\x7a\x48\xc8" + + "\xed\xf9\xf5\xc5\xd5\x2c\xc3\x48\x6f\xfc\xec\xa3\xbd\x78\xf8\xae" + + "\x8e\xfe\xb5\xea\xba\x61\xd2\xfc\xb6\x7e\xb8\xf3\x7c\xd5\x43\xd6" + + "\xad\x5b\xc7\x23\x8f\x3c\x52\xec\x4a\x5a\x78\x7e\xf9\x9d\x77\xde" + + "\x21\x35\x35\x15\x45\x51\xe8\xd4\x67\x28\xb6\xfc\x9a\x68\x67\xf3" + + "\xb7\xa4\x94\xb4\x6a\x52\x1f\x80\x1d\x3b\x76\x94\x3a\x59\x6f\xdb" + + "\xb6\x0d\x80\x76\xad\x93\xb0\x9f\x47\x68\x72\xbe\x30\x4d\x93\x1f" + + "\xa6\xbc\x06\x20\x75\xfd\xe0\xf7\x11\x11\x9d\xca\x55\xa8\xd0\xdf" + + "\x62\x74\x59\x73\xdd\x85\x70\xae\x93\xd2\x9d\x34\xe0\x81\xdb\x79" + + "\xa8\xd7\x2d\x18\x86\x09\x48\x74\xdd\xa0\x55\xb7\x81\xfe\xa9\xb4" + + "\x75\xeb\xd6\x91\x94\x94\x74\xd1\xb2\xaa\x66\x65\x65\x11\x19\x19" + + "\x49\xc7\x76\x2d\x78\xe3\xb9\x87\xce\x9a\x32\xa9\xa0\xa3\x68\xda" + + "\xe5\x71\x2a\x54\x88\x63\xff\xfe\xfd\x97\x3c\x03\x6c\x81\xb7\xe2" + + "\x9b\xf6\xf2\x6d\x4f\xfd\x75\xfa\x38\x42\x83\x9d\x17\xbd\x1d\x63" + + "\x27\xcd\xe0\xf3\x19\x3f\x03\x41\xe3\xc0\x65\x6d\x74\x29\x4b\x16" + + "\x5d\x88\xc8\xd1\x52\xba\x93\xee\xef\xde\x81\x47\xee\xea\x98\xbf" + + "\xb2\x4c\xa2\xa9\x2a\x4f\xbd\xfa\x91\x9f\xe4\x4b\x97\x2e\xbd\xa8" + + "\x24\x07\x5f\x46\x96\x89\x13\x27\x32\x77\xc1\x4a\x16\x2e\x5f\x5f" + + "\x64\x52\x8d\x7e\xee\x61\x0e\x1c\x38\x40\xff\xfe\xfd\xfd\xa1\xc7" + + "\xa5\x95\xb1\x60\xfc\x78\x5f\x0a\xea\x96\x4d\xea\x12\x12\xe4\xb8" + + "\x24\xed\x78\xee\xd1\x9e\xf9\x19\x69\xdc\x4f\xaa\x6a\xd5\x98\xa8" + + "\xa8\x1b\x2c\x56\x9f\x02\x6a\x20\x3d\x4c\x5c\x1c\xe4\xe4\x44\x3c" + + "\x2b\xe5\xf1\x97\x6e\x6e\xdb\x4c\xbe\x32\xf8\x7e\x7f\xca\x65\x9b" + + "\xa6\x32\x7c\xdc\x17\xfc\xb0\x64\x35\x00\x73\xe6\xcc\xe1\xc6\x1b" + + "\x6f\x2c\x31\x92\x9f\xe9\xba\x57\x5e\x79\x25\xef\xbf\xff\x3e\xd3" + + "\xe7\x2f\xe6\xba\x96\x8d\x29\x1f\x15\x7e\xd6\x36\xd4\xaa\x5e\x89" + + "\xb5\x9b\x77\x30\x73\xce\x7c\xb6\x6f\xdf\x4e\xf7\xee\xdd\x2f\xa9" + + "\xac\x37\x6f\xde\xcc\xad\xb7\xde\x4a\xad\xf8\x2a\x4c\x7d\xe7\x79" + + "\x2e\xa4\x02\xcd\x85\x40\x37\x0c\xc2\x42\x83\x59\xb4\x62\x3d\x60" + + "\x5c\xe1\x72\x6d\xfb\xda\xa2\x75\x00\x5b\x74\x45\xa9\x82\xdb\xdd" + + "\xb3\x9c\x94\x99\xa3\x62\xcb\x47\xca\xd7\x9f\x7d\x48\xe4\x79\xbc" + + "\x7e\xeb\xb7\x6b\xdf\x61\x66\xfc\xb0\x0c\x80\x19\x33\x66\xd0\xa5" + + "\x4b\x97\x12\x21\x79\x81\xa5\xcd\xce\xce\xe6\xe1\x87\x1f\x26\x3e" + + "\x3e\x9e\x8e\x1d\x3b\xd2\xb3\x67\xcf\x13\xb2\xb2\xbc\xf7\xde\x7b" + + "\x00\xdc\x3b\x68\x74\x91\x46\xe1\x75\xc3\x60\xca\x98\xa7\xa9\x18" + + "\x53\x8e\xa9\x53\xa7\xf2\xdf\xff\xfe\xf7\x92\xc9\xfa\xbd\xf7\xde" + + "\xa3\x61\xc3\x86\x5c\x51\xa3\x32\x33\x27\xbe\x74\x49\x76\xf7\x15" + + "\xf6\x2c\xee\xec\xd4\x96\xc8\xf0\x50\xa4\xcc\xbd\x3d\x34\xf4\xba" + + "\xf2\x71\x71\x43\x2c\x66\x07\x76\x8c\xae\xa5\x81\x1e\xb9\xec\x9b" + + "\xb1\xa2\x60\x5b\x23\xf8\xaa\x85\x3c\xf0\xdf\x31\xac\xda\xf0\x17" + + "\x2f\xbf\xfc\x32\xc3\x86\x0d\x2b\xb1\xb8\x55\x08\xc1\x53\x4f\x3d" + + "\x75\xc6\xb4\x4c\x63\xc6\x8c\xe1\xa9\xa7\x9e\xa2\x62\xc5\x8a\x1c" + + "\x3e\x7c\x98\x61\x03\x7a\xd3\xed\xe6\x6b\x8a\x74\xfd\x03\x47\x8e" + + "\xd1\xb9\x8f\x6f\x01\xcf\x77\xdf\x7d\x47\xe7\xce\x9d\x2f\xaa\x84" + + "\xe7\xcc\x99\xc3\x6d\xb7\xdd\x86\xa2\x28\xac\x9b\xf7\x01\xe6\x25" + + "\xb2\xe4\xff\x96\xcb\xef\x1b\xb6\xf1\xe8\x90\x71\x80\xe3\x3d\xc8" + + "\x1b\x60\x51\x3b\xc0\x88\xee\x74\xb6\xc4\xed\x5e\x89\xaa\x46\x7d" + + "\x6b\x18\xe9\xdd\x67\x7c\xf0\x12\x09\xd5\x2b\x9e\xa0\x04\xdb\x77" + + "\x1f\xa0\x67\xbf\x91\xb4\x6b\xd7\x8e\x85\x0b\x17\x96\x58\x5b\x36" + + "\x6d\xda\x44\xb3\x66\xcd\xc8\xcb\x3b\x7d\x95\x91\x9a\x35\x6b\x92" + + "\x9a\x9a\xca\x5d\x77\xdd\x45\x74\x74\x34\xa3\x46\x8d\x42\xd3\x54" + + "\x56\xcf\x1e\x5f\x64\xc2\xac\xdf\xb2\x93\x7e\x2f\x8d\x27\x3b\xc7" + + "\xc5\xfa\xf5\xeb\x69\xdc\xb8\xf1\x45\x91\xf5\xdf\x7f\xff\x4d\xed" + + "\xda\xbe\x75\xec\xef\x0e\x7f\x9c\x36\xcd\x1b\x95\x9a\x95\x7a\x76" + + "\x9b\xc6\x03\x4f\x8f\x61\xd5\xfa\x1d\x6e\xd0\xad\x32\x4d\x81\xe6" + + "\xba\xbb\xdd\x2b\x51\x94\xea\x37\x1a\x46\x7a\xf7\xc7\x7a\x77\xa1" + + "\x76\xfc\x89\x9b\x9b\x1c\x76\x1b\xef\x7e\x36\x0b\x9b\xcd\x56\x62" + + "\x24\x97\x52\xe2\xf5\x7a\x69\xd4\xa8\xd1\x19\x49\x0e\xb0\x6b\xd7" + + "\x2e\x5a\xb7\x6e\xcd\xc4\x89\x13\xe9\xd3\xa7\x8f\xcf\x2d\xd7\x0d" + + "\xde\xff\xe2\xbb\x22\xdf\xaf\x71\xbd\x9a\xfc\xf2\x7f\xa3\xa9\x5e" + + "\xb9\x02\x49\x49\x49\x6c\xd9\xb2\xa5\xc4\xe5\x3c\x69\xd2\x24\x3f" + + "\xc9\x1f\xbd\xa7\x33\xd7\xb6\x48\x2c\x55\xcb\x71\xf3\x3c\x5e\xfa" + + "\xdf\x7f\x1b\xa0\x3b\x55\x35\xb6\x8f\x45\xed\x00\x23\x7a\xeb\xd6" + + "\x3f\x09\xd3\xdc\xf3\x93\xd3\x61\x97\x0f\xdf\xd5\xf1\x84\x52\x48" + + "\x52\x4a\x92\x0f\x1c\xe1\xb7\x35\x9b\xb8\xf1\xc6\x1b\x4b\x34\x4e" + + "\x7c\xea\xa9\xa7\x8a\x7c\xfc\x0f\x3f\xfc\x00\xc0\x8f\x3f\xfe\xc8" + + "\x6d\xb7\xdd\x06\xc0\x9f\x5b\x77\xa1\xa9\x6a\x91\xef\xe7\xb0\xdb" + + "\x98\xfe\xc1\x30\xc2\x43\x83\x69\xd0\xa0\x01\x13\x26\x4c\xe0\xd8" + + "\xb1\x63\x27\x8d\x15\x5c\x28\xbe\xfc\xf2\x4b\x84\x10\x3c\xda\xf7" + + "\x11\xba\xdd\x72\x0d\x9b\x7e\xfc\x98\xc7\x7a\x77\x29\x7d\xae\xa9" + + "\x10\x5c\xdd\xb4\x01\x0e\xbb\x4d\x1a\x46\xea\xa4\x90\x90\x07\xc2" + + "\x9c\xce\x16\x16\xc3\x03\x85\xe8\x2b\x56\x74\xfb\x06\xe0\xeb\xf1" + + "\x43\x85\x52\x28\x97\x19\xf8\xf6\x74\x7f\x33\x77\x09\x00\xfd\xfa" + + "\xf5\x2b\xd1\x76\x7c\xfb\xed\xb7\xe7\x7c\xce\xe4\xc9\x93\xfd\x1d" + + "\xc4\x86\xad\x3b\xb1\x69\xea\x39\x29\xb6\xa6\x2a\x2c\xf9\x66\x2c" + + "\xa1\xc1\xbe\xf2\x49\xd1\xd1\xd1\x44\x46\x46\xb2\x71\xe3\x46\xb2" + + "\xb2\xb2\xce\xeb\x39\x74\x5d\x67\xeb\xd6\xad\x0c\x1e\x3c\x18\x9b" + + "\xcd\x46\xef\xde\xbd\x69\x5c\xaf\x26\x1b\x7f\xfc\x98\x61\xfd\x7b" + + "\xa3\x1b\xc6\x25\x9f\xda\x3b\x1d\xb2\x72\x5c\xdc\xd9\xb9\xad\x00" + + "\x53\xe6\xe4\x7c\xf9\x83\xdb\xbd\xca\x62\x78\x20\x10\xdd\x6e\x8f" + + "\x8f\x90\x32\xbb\x3b\x20\xab\x56\x8c\x3e\xe9\xef\x9a\xaa\xb2\xe5" + + "\xef\xbd\xf9\x2e\xbe\xbb\x44\xdb\x72\xf8\xf0\xe1\xb3\x1e\x33\x7e" + + "\xfc\x78\xa4\x94\x7c\xfc\xf1\xc7\x00\xac\x5d\xbb\x96\x36\x6d\xda" + + "\xf8\x5d\xcf\xc9\xdf\xfe\x78\x4e\x24\x12\x42\x80\x94\xac\x9a\xfd" + + "\x1e\xe3\x86\x3d\x86\xd3\x61\xe7\xf8\xf1\xe3\x24\x26\x26\x12\x11" + + "\x11\x81\x10\x82\x36\x6d\xda\xb0\x7d\xfb\x76\xd2\xd3\xd3\x4f\x19" + + "\x56\x98\xa6\x49\x66\x66\x26\xfb\xf7\xef\x67\xf4\xe8\xd1\xd8\x6c" + + "\x36\xea\xd7\xaf\xcf\xdb\x6f\xbf\x4d\x95\xb8\x68\xfe\xfc\x61\x22" + + "\x53\xde\x7a\x86\x3c\x8f\xb7\xd4\xeb\x83\x94\x92\x27\x1f\xec\x86" + + "\x22\x84\x00\x6f\xeb\xb0\xb0\x1b\x6a\x5a\x14\x0f\x00\xa2\x7b\x3c" + + "\x07\xd7\x01\x3c\xd0\xbd\x83\x38\xd5\x36\x50\x45\x51\x48\xcb\xf0" + + "\x6d\x6c\x8a\x8f\x8f\x2f\x71\xd7\xf1\x6c\x28\x88\xc9\x2b\x54\xa8" + + "\x00\x40\x52\x52\x12\x80\xbf\x98\xe1\xb2\x35\x9b\xcf\x39\xee\x15" + + "\x42\xe0\xf1\xea\xb4\xbd\x2a\x91\x55\xb3\xc7\x53\xb1\xc2\x89\xf5" + + "\xcc\x7e\xfd\xf5\x57\xea\xd4\xa9\x43\xb9\x72\xe5\x70\x3a\x9d\xfe" + + "\xb5\xe2\x05\xff\x54\x55\x25\x22\x22\x82\xaa\x55\xab\xf2\xdc\x73" + + "\xcf\xfd\x73\xde\xf4\xb7\x99\xf5\xd1\xf0\xfc\x15\x85\x97\x91\x42" + + "\x2b\x82\xd1\xcf\x3f\xec\xb3\xf0\x59\xcb\xac\x79\xb6\x02\xa3\x77" + + "\xd9\xf6\x50\x4a\x58\x05\xd3\xcc\x8a\x07\x78\xac\xf7\xa9\xe7\xc4" + + "\x0b\x32\x93\x00\x27\xec\x20\xbb\x54\xf8\xe4\x93\x4f\xc8\xcb\xcb" + + "\xe3\x85\x17\x5e\x00\xf0\x5b\xf3\x7b\xee\xb9\x87\xf7\xdf\x7f\x9f" + + "\x0b\x19\xdb\x12\x42\x60\x18\x06\xdf\x4f\x1e\x05\xc0\xee\x7d\x87" + + "\xe8\xfd\xe4\xeb\xe4\xb8\xf2\x8a\x74\x7e\xf3\xc4\x3a\x54\xad\x14" + + "\x43\xdd\x84\x6a\xf4\xe8\xd8\xe6\x9c\x3a\xb0\xd2\x16\xab\xb7\x4c" + + "\xaa\x9b\xff\xcd\xfb\x00\xf0\x50\x42\xc2\xf3\xec\xdc\xf9\x9a\x45" + + "\xf4\xcb\x11\xa6\x69\x0e\xc7\x97\x86\x50\xd8\x34\xf5\x94\x0a\x69" + + "\x18\x26\xdb\x76\xed\x07\x20\x3a\x3a\xba\x44\xdb\xd3\xa5\x4b\x17" + + "\xe6\xcc\x39\x73\x29\xb0\x82\x1d\x6c\x05\x28\x20\x7a\xab\x56\xbe" + + "\xca\x30\x07\x0f\x1f\xc3\x66\xd3\x4e\x48\xf5\x7c\x3a\x17\x55\x51" + + "\x7c\x59\x5b\x0d\xc3\x44\x55\x15\x5f\x06\x16\xe3\x1f\xb7\x3f\xbe" + + "\x6a\x45\x7e\x9d\x3e\x0e\xdd\x30\x30\x4d\xc9\x91\xd4\x74\x34\x55" + + "\x21\x32\x3c\x0c\x55\x55\x10\x02\x04\x22\xbf\x23\x14\x17\x94\x8b" + + "\xbd\xb4\x21\x2c\xb4\xa0\x4e\x9d\xa9\x0a\x11\xfa\xc5\xce\x9d\xaf" + + "\xdd\x6b\x59\xf4\xcb\x10\xa1\xa1\x57\x39\xb2\xb3\x7f\x7f\x14\xa0" + + "\xde\x15\xd5\x4e\x9b\xbd\x25\xf9\xc0\x91\x8b\x66\x95\xfa\xf7\xef" + + "\x7f\x56\xa2\xff\x1b\x35\x6b\x9e\x18\x42\xa6\xa6\x1f\xc7\xa6\xaa" + + "\xa7\x25\xba\x94\x12\xbb\xcd\xc6\xdc\x85\xbf\x33\x6a\xfc\x97\xb8" + + "\xf3\x3c\x78\xbc\x3a\x8a\x22\x08\x72\x3a\x78\x65\xf0\x83\xb4\xbf" + + "\x3a\x89\xc2\x15\x65\x0a\x76\xc9\x9d\xad\x42\x69\x40\x29\xb5\xaa" + + "\xd2\xf9\xfa\x96\xcc\x5d\xb8\x52\x4a\x99\x7d\xab\xe5\xb8\x5f\x86" + + "\x31\xba\xa6\x35\x20\x3b\x7b\xa3\xdf\x3c\x7f\xf1\xf6\xe9\x0b\x10" + + "\x16\xe4\x46\x2f\xc8\xef\x56\x92\xb8\xe1\x86\x1b\xfc\x16\xba\x28" + + "\x18\x33\x66\x8c\x3f\x46\x07\x5f\x66\xd5\x9c\x5c\x37\xa7\x1e\x6b" + + "\x10\xb8\xdc\x79\x8c\xfb\x74\x26\xd7\xf4\x78\x92\x67\x5f\xff\x98" + + "\xcc\xec\x5c\xff\xce\x37\xd3\x94\xe4\xe4\xba\x19\x3c\xf2\x03\xee" + + "\x1b\x34\x1a\xc1\x89\xd3\x6b\x65\xad\x86\xb8\x57\xd7\xe9\x7b\x4f" + + "\x27\xf0\x2d\x08\x0b\x57\xd5\xea\x3d\x2d\x8b\x7e\x59\x21\x36\xdc" + + "\x30\xf6\xdc\x05\x66\x43\x40\x0a\x21\xc4\x99\xe6\x9e\xbd\x9e\x8b" + + "\x57\xed\x44\x4a\xc9\x8f\x3f\xfe\x48\xef\xde\xbd\xf9\xdf\xff\xfe" + + "\x77\xc6\x63\xd7\xae\x5d\x4b\x93\x26\x4d\x4e\x18\x57\x28\x57\xae" + + "\x5c\xfe\xc6\x90\x7f\x87\x28\x92\xbb\x07\xbe\xc6\x96\xbf\xf7\x9c" + + "\xf6\x7a\xed\xdb\xb7\x27\x32\x32\x92\x19\x33\x66\xb0\xe1\xaf\x5d" + + "\xac\xdb\xb2\x93\xa4\xfa\x09\x65\x56\xa9\x85\x10\x24\x54\xab\x48" + + "\x8d\xaa\x15\x48\xde\x77\x04\xd3\x3c\xfc\x32\x50\xa6\x37\xbb\x5c" + + "\x16\x16\x3d\x26\xe6\x90\x26\x44\xf8\x23\x42\xa4\x1d\x97\x32\xfb" + + "\x43\x70\xf7\x03\x84\xc3\x6e\x3b\xe3\x79\xd5\xab\x54\xf0\x7f\xf6" + + "\x78\x3c\x25\xae\x5c\x4e\xa7\x93\x19\x33\x66\x90\x9c\x9c\xcc\x88" + + "\x11\x23\x4e\xaa\x82\x3a\x6c\xd8\x30\x52\x52\x52\x68\xd2\xa4\xc9" + + "\x49\x96\xb6\xc0\x92\x17\xde\x82\xfa\xf7\xee\x03\x5c\xd9\xf9\xb1" + + "\x33\x92\xfc\xc3\x0f\x3f\x64\xed\xda\xb5\xcc\x9a\x35\xeb\x9f\xb1" + + "\x80\xe1\xef\x33\xe4\xcd\x4f\x99\xbf\x68\x15\xba\x61\x10\x1a\x12" + + "\xc4\xdf\xbb\x0f\xb0\x72\xdd\x56\x66\xfd\xb4\x1c\x97\x3b\x0f\x9b" + + "\x4d\x2b\xb5\xf3\xe1\xc5\x01\x8f\x57\x67\xd4\x53\x0f\xe6\x77\xc2" + + "\x9e\x6a\x65\xdd\xa2\x97\x4a\x9f\xce\x6e\x6f\x89\xc7\xb3\x12\x55" + + "\x2d\xdf\xdf\x30\x32\x5f\x06\x6f\x14\xc0\x8d\x6d\x9a\x4a\xbb\x4d" + + "\x13\xf3\x16\xfe\x0e\xf8\x4a\xed\x2e\x9e\x36\xe6\xb4\xae\xa9\x4d" + + "\xd3\x68\xd0\xe1\x21\xc0\x97\x9d\x25\x21\xe1\xe2\x58\xb9\x33\xed" + + "\x8a\x3b\xdd\xdf\x42\x42\x42\xc8\xcd\xcd\x65\xdb\xc2\xc9\xe4\x79" + + "\x3c\xbc\xff\xf9\x1c\x26\x7e\x35\xef\xb4\xf7\x68\xd4\xa8\x11\x1b" + + "\x37\x6e\x3c\xef\x36\x36\x4b\xac\xcd\xd0\x27\xee\x21\xbe\x5a\x05" + + "\x02\x95\xef\x52\x4a\x9a\x74\xf2\x55\x6f\xb2\xd9\x6a\xb6\xf5\x7a" + + "\x77\x2d\xb5\x5c\xf7\x52\xe5\x7a\x65\x35\x02\xfb\x57\x86\x71\xac" + + "\x61\x7c\xd5\x38\x79\x7f\xf7\x0e\xdc\x73\xeb\xf5\x78\xbc\x5e\x21" + + "\x84\xe0\x9a\x66\x0d\x79\xfe\x8d\x49\x54\xa9\x14\x73\xc6\xf8\xd3" + + "\x66\x53\x09\x09\x72\x92\xe3\x72\x93\x96\x96\x76\xd1\x88\x7e\xa6" + + "\x36\x9d\xea\x6f\x87\x0f\x1f\x26\x37\x37\x97\x90\x60\x27\x9a\xaa" + + "\xf0\xd6\x94\xd9\x7c\x34\x75\xfe\x19\xef\x51\xbe\x7c\xf9\xb3\xb6" + + "\xe3\x9a\xe6\x0d\xa8\x5c\xa1\x3c\x15\x63\xcb\x91\x95\xed\x62\xf7" + + "\xbe\xc3\x2c\x5b\xb3\x19\xaf\x57\x67\xcd\x9f\xdb\xe9\xda\xf7\x25" + + "\x86\xf6\xbf\x87\x1e\x1d\xaf\x0d\x48\xeb\xae\x69\x2a\x6d\x5b\x26" + + "\xb2\x64\xe5\x9f\x78\xbd\x07\x6f\x01\x2c\xa2\x97\x9a\x58\x42\x89" + + "\x78\x3b\x2f\x6f\xf3\x93\x80\x7c\xee\xb1\x5e\xf4\xe9\x71\x93\xc8" + + "\x75\xe7\x91\xe3\x72\xfb\x7b\xe9\xc4\xfc\x4c\xa3\x7b\xf6\x1d\x39" + + "\xe3\xb5\x0c\xc3\x24\xbe\x5a\x1c\x9b\xb6\x25\x97\xf8\xca\xb8\x0b" + + "\xc1\xf2\xe5\xcb\x01\x70\x3a\xec\x64\x64\xe6\x9c\x95\xe4\x00\x8b" + + "\x17\x2f\x3e\xed\xdf\x6e\xbb\xb1\x25\x0f\xdf\x75\x0b\x5e\x8f\x8e" + + "\x61\x9a\x7e\x12\xab\xaa\xc2\x93\xff\xe9\xca\x0f\x8b\xd7\x30\x6d" + + "\xee\x52\xd2\x32\xb2\x18\xf9\xde\x97\x54\xad\x18\x43\x8b\xc6\x75" + + "\x02\x6e\xd0\xce\x30\x4c\x9e\x7e\xe4\x4e\x96\xac\xfc\x13\xf0\x0e" + + "\x00\x9e\xb7\x62\xf4\x4b\x8c\x4a\x95\x1e\x09\x03\x5b\xa6\x69\x1e" + + "\x7f\x52\x55\x15\x7e\xfa\xfc\x75\xd1\xab\xcb\x75\xe4\xba\xf3\x4e" + + "\xb2\x88\x7e\xeb\x73\x16\xbd\xcc\xf3\x78\xe9\x76\xd3\x35\x27\xc4" + + "\xc0\xa5\x11\xd9\xd9\xd9\xfe\x50\x64\xe5\xba\xad\x17\x74\xad\x47" + + "\xef\xe9\xc4\xbd\xb7\xb7\xc7\xe5\xca\xf3\xcf\x3a\x14\xac\x82\x33" + + "\x4d\x89\xc7\xa3\xd3\xae\x55\x63\x3e\x1b\x33\x98\x5b\xae\x6b\x8e" + + "\x10\xd0\x77\xc8\xb8\xcb\x62\x89\xeb\xf9\xa0\x42\x74\x41\x31\x09" + + "\xc3\x49\x19\x46\xa9\xd0\xfe\xa0\xa0\x16\xe5\x0f\x1e\xfc\x24\x13" + + "\xbc\x61\x35\xab\x55\xe4\x8f\xb9\x13\x88\x8d\x3e\x7d\xb5\x8f\xbc" + + "\xfc\x22\x88\x91\xe1\xa1\x9c\xad\x74\x67\xd3\x46\xb5\xfc\x16\xb0" + + "\xb4\xba\xa7\x93\x26\x4d\x02\xe0\xaa\xa4\xba\x3c\x31\x6c\xfc\x79" + + "\x5f\xa7\x5d\xeb\x24\x6e\xb8\x3a\xa9\x48\xe1\x83\xc7\xa3\xf3\xd0" + + "\x9d\x1d\x18\x33\xc4\xb7\x5c\xf4\x91\x21\xe3\x02\x52\xc1\x1d\x76" + + "\x3b\x8d\xea\xc4\x03\x28\x9a\x56\xf9\x25\x8b\xe8\x97\x08\xe1\xe1" + + "\xdd\x6c\x2e\xd7\x1f\x47\xc0\x94\xcd\x1a\xd5\x66\xf6\x47\x2f\x23" + + "\xcd\xd3\x13\x52\x11\x22\x3f\x3f\x18\xd4\xa9\x59\xe5\xac\x66\xbd" + + "\x76\x7c\x15\x92\xea\x27\x30\x79\xf2\xe4\x52\x59\xce\xc8\xeb\xf5" + + "\xb2\x74\xe9\x52\xa2\x22\x42\x19\xda\xef\x6e\x34\xed\xfc\x5e\x49" + + "\xab\x2b\xeb\xd1\xff\xbe\x73\xcb\x36\x23\x81\x9a\x55\xe3\x48\xaa" + + "\x9f\xc0\xd6\x1d\x7b\x51\x14\x11\x70\xb1\xba\x69\x9a\xbc\x3c\xe8" + + "\x3e\x00\xa9\xeb\x07\x86\x3b\x1c\x8d\x15\x8b\xe8\x17\x11\x42\x44" + + "\x73\xeb\xad\x92\xec\xec\x05\x3b\xc1\x50\x54\x55\x11\x1f\xbf\x3e" + + "\xa8\x48\x15\x48\xd5\xfc\x39\xf1\x06\xb5\xaa\x9f\xf5\x3e\xee\x3c" + + "\x0f\x8f\xf7\xee\xc2\xce\x9d\x3b\x59\xbb\x76\x6d\xa9\x52\x64\x21" + + "\x84\x7f\x9f\xfc\xfd\x77\x74\xc0\x94\x92\x1b\xaf\x69\x7a\x8e\x63" + + "\x1a\x82\x9e\x9d\xaf\xe5\xbf\x0f\x77\xc3\x34\xcf\xfd\xd9\x0c\xd3" + + "\xe4\xba\x96\x89\x78\xbd\x3a\xbf\xfc\xb6\x2e\x20\x17\xd7\xd4\xa8" + + "\x12\xe7\x0f\xf4\xf2\xf2\xf6\xd8\x2d\xa2\x5f\x54\x6b\x96\xca\x9c" + + "\x39\xa1\x43\x4c\xf3\x78\x15\x40\xdc\x7b\xfb\x0d\xfe\x58\xf2\x8c" + + "\xe4\x50\x04\xab\x37\xfc\x05\x40\xb5\x4a\x15\x8a\x44\xa6\xe6\xf9" + + "\x9b\x1c\x3a\x77\xee\x5c\xaa\x14\xf9\x8d\x37\xde\x60\xc9\x92\x25" + + "\xf9\xb1\x75\x67\xa4\x94\xd4\x8a\x2f\x7a\xf9\xef\xd6\x4d\xeb\x33" + + "\xe7\xe3\xe1\xf4\xe8\xd8\xe6\x82\xa6\xc8\x4c\xe9\x8b\xe5\x9f\x1a" + + "\x39\x31\x20\x47\xdf\x15\x45\x30\xed\xbd\x17\xf2\x3f\x6b\x9f\x59" + + "\x44\xbf\x88\x50\xd5\x4a\x4d\x20\x67\x94\x8f\x8b\x82\x41\xff\xe9" + + "\x56\x34\x2b\x88\xf0\x0f\x1c\x35\x4d\xac\x55\xa4\x73\x6c\x9a\x4a" + + "\xbb\x56\x8d\x39\x74\xe8\x10\xbf\xfd\xf6\x5b\xa9\x11\xfe\x8b\x2f" + + "\xbe\x08\xc0\xf0\x27\xef\xc3\x9d\xe7\xc1\x34\x25\xad\xaf\xac\x7f" + + "\xda\xe3\x63\xcb\x47\x10\x1b\x1d\xc9\xd5\x4d\x1b\x30\xed\xbd\xe7" + + "\x18\xfc\x9f\xdb\x71\xe7\x5d\xd8\x42\x20\x21\x04\x5b\xf3\xf7\xec" + + "\xd7\xa8\x52\x21\x60\x97\xcb\xd6\xaa\x51\x29\xdf\x95\x4f\xed\xa9" + + "\x28\x11\x21\x65\x8d\xe8\x97\x6c\x7a\xcd\x34\x33\xee\xcf\x0f\x13" + + "\x45\x4c\xf9\xc8\x22\x5b\x24\x21\xe0\x78\x56\x2e\x00\x41\x45\xac" + + "\xda\xa9\xeb\x06\xcf\x3e\xda\x8b\x45\x2b\x36\x70\xf3\xcd\x37\x9f" + + "\x77\xf6\x95\xe2\x44\xd7\xae\x5d\xf1\x78\x3c\x74\xbb\xb9\x0d\x77" + + "\xdc\x7c\x0d\xba\x61\x70\xe0\x70\x2a\xbd\xfa\xfb\xb6\x99\xf6\xec" + + "\x74\x2d\xb7\xdf\xd4\x1a\x09\xa8\x8a\xa0\x24\x76\x98\x15\x58\xef" + + "\x9f\x7f\x5d\x07\xc0\xcb\x83\xef\x0f\x58\x45\x57\x55\x95\xe6\x8d" + + "\xeb\xb0\x7a\xc3\x36\x4c\x33\xcb\x69\xb3\xd5\xcf\xf1\x7a\xb7\x94" + + "\x19\xa2\x5f\x42\xd7\xdd\x6c\x5d\x10\x37\x35\xaa\x53\xe3\x8c\x2e" + + "\xa3\x94\x12\x9b\xa6\x32\x6d\xce\x62\x6e\x7d\x68\x18\x3b\x92\x0f" + + "\xa0\xa9\x2a\x61\x21\x45\x4f\xf6\x59\x31\x36\x8a\xb7\x5e\xe8\x4b" + + "\x76\x76\x36\x0d\x1b\x36\x3c\x41\xd1\x2f\x36\x7a\xf4\xe8\xc1\xec" + + "\xd9\xb3\xb9\x2a\xa9\x1e\x2f\x0f\xba\x17\xdd\x30\x38\x9e\x95\xc3" + + "\x1d\x8f\xbd\x82\x10\x82\x4f\x5e\x1f\x48\x8f\x4e\x6d\x50\x55\x05" + + "\x9b\xa6\xa2\x28\x4a\x89\x6c\x23\xcd\xc9\x75\xf3\xe4\x88\x89\x80" + + "\x6f\x06\x23\xb1\x4e\x7c\x40\x2f\x8b\x9d\x30\xa2\x3f\x3e\xe3\xe2" + + "\x58\x5b\x96\x48\x7e\xc9\x2c\x7a\x6c\xec\x22\xf5\xe8\xd1\x76\xcd" + + "\x0b\xbe\x8f\x7c\xea\xc1\xb3\x2e\x19\xed\xda\x77\x38\x7f\xef\x3e" + + "\x50\xa8\x87\x56\xce\x29\x2e\x35\x4d\xc9\xf5\xad\x93\xd0\x9d\x41" + + "\x55\xf5\x00\x00\x20\x00\x49\x44\x41\x54\x34\x95\xcd\x9b\x37\xd3" + + "\xb6\x6d\x5b\x96\x2c\x59\x72\xd1\xeb\x98\xf5\xec\xd9\x93\xe9\xd3" + + "\xa7\x03\xf0\xc1\xc8\xfe\xe8\xba\x71\x42\xae\xf6\x67\xfa\xf6\xc8" + + "\x9f\x36\xa4\xc4\x2a\xc8\xa8\x8a\x82\xcd\xa6\xd1\xfd\x71\x9f\xf7" + + "\xa0\x2a\x0a\xf3\x27\x8f\x2a\x15\x35\xdd\x4a\x12\x36\x9b\x86\xa2" + + "\x08\x61\x9a\xae\x6a\x9a\x56\xa9\xb6\xae\x1f\xdc\x6e\x59\xf4\x12" + + "\x44\x6a\x6a\xef\x98\x82\xcf\x0e\xbb\x8d\xf0\xd0\xd3\x5b\x66\xaf" + + "\x6e\x90\xd4\xf1\xd1\x13\x48\x0e\xbe\x55\x64\x05\x0b\x42\x8a\x1a" + + "\x8b\x0a\x21\x98\x3b\x69\x24\xe0\xab\xbb\x16\x15\x15\x45\x6a\x6a" + + "\xea\x45\xf0\x5e\x7c\x3d\x52\xc7\x8e\x1d\xf9\xe6\x9b\x6f\x00\x98" + + "\xfa\xde\x10\x94\x7c\xc2\xbd\xf5\xb1\x8f\xf8\x2d\x1a\xd7\xa1\x65" + + "\x93\x3a\xc5\x34\x00\xa5\x60\xb7\x69\x38\xec\x36\x82\x83\x1c\xa4" + + "\x67\x66\xf3\xdd\x82\xdf\x59\xb9\x7e\x1b\x43\xc7\x7e\x4e\x97\x87" + + "\x86\x03\xbe\xe9\xc7\xf9\x9f\x8d\x22\xc8\x69\x2f\x13\xdb\x59\xbf" + + "\x9d\xf0\x52\xfe\x2b\xf1\xf4\xb5\x2c\x7a\x89\xc3\x73\x45\xc1\x27" + + "\xbb\xcd\x86\x71\x9a\x69\x21\xa7\xc3\xce\x13\xc3\xde\x3b\x8d\x85" + + "\xf6\x6d\xe9\x3c\x57\x4f\x33\x2e\x26\x8a\xc5\xd3\xc6\x70\xfd\xdd" + + "\xcf\x90\x91\x91\x41\x6c\x6c\x2c\xd3\xa6\x4d\xa3\x67\xcf\x9e\xc5" + + "\x6e\xd1\x0a\xae\x97\x9d\x9d\x4d\x8b\x16\x2d\xf8\xeb\x2f\xdf\x6c" + + "\xc1\x0b\xfd\xee\xa6\x41\xad\x1a\x98\xa6\x89\x4d\x55\xd9\xbd\xcf" + + "\x57\x97\x7d\xe0\x83\xb7\x61\x18\xe7\x5f\xf9\x44\xd3\x54\x34\x55" + + "\x61\xc1\xf2\x0d\xfc\xb0\xe4\x0f\x8e\xa5\x67\x62\x9a\x26\x59\x39" + + "\xae\x93\x56\xbe\xd5\x4b\xa8\x46\xbf\x07\x6e\xe3\xfa\x56\x49\xb8" + + "\xf3\x3c\x65\x66\xcf\x7a\x42\xb5\x38\xae\x6f\x9d\x24\x16\x2e\xdf" + + "\x30\x18\x78\x2a\x2c\x2c\x86\xac\xac\x14\x8b\xe8\x25\x01\xd3\xcc" + + "\xaa\x4b\xfe\x40\xdc\x83\x77\xde\x84\x79\x1a\xcb\xbc\xfa\xcf\x6d" + + "\xfc\xb6\x66\xf3\x69\x2d\xd6\xf9\x86\x93\x91\xe1\xa1\x6c\xf8\xfe" + + "\x43\x7e\x59\xb6\x96\x41\x23\x3f\xa4\x57\xaf\x5e\x7c\xf5\xd5\x57" + + "\xcc\x9e\x3d\xbb\x58\x9f\x53\x08\xc1\xc0\x81\x03\x79\xf7\xdd\x77" + + "\xfd\xbf\x7d\xf4\xda\x20\x5a\x35\xa9\xe7\x5f\x2f\xa0\x1b\x06\xc9" + + "\xfb\x8f\x10\x1a\xec\x24\x32\x3c\x04\x97\xfb\xdc\x46\xd1\x05\xbe" + + "\xba\x6c\x53\xbf\x5b\xc2\xb6\x5d\xfb\x49\xde\x7f\xe4\xa4\x6b\xdc" + + "\x78\xe3\x8d\x6c\xd8\xb0\x81\xa3\x47\x8f\x02\x50\x37\xa1\x2a\x73" + + "\x26\x8d\x20\x27\xd7\x4d\x41\x7d\xba\xb2\x84\xe7\x1f\xbf\x8b\x85" + + "\xcb\xd7\x23\x84\xc3\x9d\x95\x95\xe2\xd4\xb4\x2a\xe8\xfa\x7e\xcb" + + "\x75\x2f\xf6\x9b\x2a\x0e\x4f\xc1\x40\xdc\xdd\xb7\xb6\x3b\xed\x71" + + "\x87\x8e\xa6\x9d\xc5\x5a\x9e\x7f\x1b\x0c\xc3\xe4\xba\x56\x8d\xf9" + + "\x6e\xd2\x2b\x34\x4f\xac\xc3\x9c\x39\x73\x08\x0a\x0a\xe2\xab\xaf" + + "\xbe\xba\x60\x17\x1d\xe0\xcd\x37\xdf\xc4\x6e\xb7\xfb\x49\xde\xa3" + + "\xd3\xb5\x6c\x5f\x38\x99\x16\x8d\xeb\x9c\x50\x79\xb4\xe0\x94\xa8" + + "\x88\xb0\x73\xca\xb8\x5a\x90\xf8\x72\xc6\x0f\xbf\xd1\x6b\xc0\x68" + + "\x66\xfd\xb4\x82\xad\x3b\xf6\xe1\x72\x7b\x88\x8a\x8a\x62\xe6\xcc" + + "\x99\x6c\xd8\xb0\x01\x29\x25\x3f\xfd\xf4\x13\x47\x8e\x1c\xa1\x6b" + + "\xd7\xae\x80\x6f\x11\x91\xc7\xa3\x53\x56\x11\x5b\x3e\x92\xa4\xfa" + + "\x09\x48\x99\xe7\x50\x94\x88\xa9\x81\x4e\xf2\x4b\x46\x74\x29\x95" + + "\x15\x05\x9f\x33\xb3\x73\x4f\x7b\xdc\xad\x37\xb4\xa4\x6e\xc2\xa9" + + "\x73\x06\x64\xe5\xe4\x16\xb9\xb2\xc9\x99\x2c\x6e\x95\xb8\x68\xbe" + + "\x78\xfb\x59\x46\x3d\xdd\x07\xa4\xc9\x3d\xf7\xdc\x83\xd3\xe9\x64" + + "\xdc\xb8\x71\x1c\x38\x70\xa0\xc8\xd7\x5a\xbd\x7a\x35\x53\xa7\x4e" + + "\x65\xc4\x88\x97\x11\x42\xf0\xcc\x33\xcf\xe0\xf5\x7a\xb9\xe9\xda" + + "\x66\xfc\xf1\xdd\x04\x5e\xec\xdf\xdb\xbf\x03\xef\xdf\xee\x36\x80" + + "\xdb\x53\x74\xf7\x59\x4a\x89\xc3\x6e\x63\xfc\x94\xef\xf8\x6a\x8e" + + "\x6f\x0d\x7f\xe7\xce\x9d\xfd\x89\x2e\xaa\x56\xad\x4a\xd7\xae\x5d" + + "\x49\x4c\x4c\x3c\xe1\x9c\x99\x33\x67\xd2\xaa\x55\x2b\x92\xf7\x1f" + + "\x61\xe8\x5b\x93\x03\x2a\x21\xe4\xb9\x76\xc8\x43\xfb\xdf\x43\x70" + + "\x90\x03\xd3\x3c\xde\xcb\x66\xab\x74\x65\xa0\x3f\xf3\x25\x79\xd3" + + "\xaa\x5a\x3d\xc6\x30\xf6\x1c\x05\x18\xda\xef\x6e\xba\x77\xbc\xf6" + + "\xb4\x2f\x24\x2b\xc7\x45\x87\x7b\x9f\x3b\x69\x61\x88\x10\x82\x4d" + + "\x3f\x7d\x7c\xd6\x8c\xa9\xe7\xf2\xf2\x83\x83\x1c\xbc\x38\x76\x8a" + + "\xbf\xba\x0b\x40\xe5\xca\x95\xe9\xd0\xa1\x03\x1e\x8f\x87\xcc\xcc" + + "\x4c\x72\x72\x72\xa8\x56\xad\x1a\xd5\xaa\x55\xe3\xef\xbf\xff\x66" + + "\xc1\x82\x05\x7e\x97\x18\x20\x38\xc8\x41\x9b\x16\x8d\xb8\xe1\xea" + + "\x2b\xb9\xe3\x96\x6b\xc8\xce\x71\x9f\x36\x34\xf1\x79\x37\x82\xc4" + + "\x9b\x7d\xe3\x42\x73\x3e\x19\x8e\xe7\x2c\xbb\xc8\x0a\x48\xfe\xc6" + + "\xc4\xe9\xfc\xba\x66\x33\xd7\x5e\x7b\x2d\xf3\xe6\xcd\x23\x34\xd4" + + "\x37\x52\x5f\xa5\x4a\x15\x0e\x1e\x3c\x88\xc7\xe3\x39\x29\x7d\x96" + + "\x2f\x53\xac\x41\x74\x74\x34\xc7\x8f\x1f\xe7\x87\x29\xaf\x51\x21" + + "\x3a\xb2\xcc\xb9\xee\x3e\x1d\x54\xb8\x7b\xc0\x6b\x6c\xdc\xb6\x5b" + + "\x82\x7d\x35\x78\xae\x52\x94\x18\x4c\x33\xc5\x22\x7a\xf1\xc2\x6e" + + "\x82\x47\xb4\x6e\x5a\x9f\xf7\x47\xf4\x3f\xe3\xf4\x5a\x4a\xda\x71" + + "\x3a\xdc\xfb\xdc\x49\x96\x70\xc3\xfc\x0f\xf1\xea\x46\xb1\xb7\x2c" + + "\x2b\x3b\x97\xc7\x87\xbd\xc7\x5f\x3b\xf6\xe1\xd5\xf5\x13\xb7\xc6" + + "\xe6\x23\x24\xc8\x89\xcd\xa6\x51\xa3\x4a\x05\x12\xeb\xc6\xd3\xe9" + + "\xfa\x96\xb4\x68\x5c\x07\x5d\x37\xc8\x75\xe7\x15\x79\x3e\x3a\xc8" + + "\xe9\xe0\xfa\xbb\x9f\xe6\xc0\xe1\x54\xee\xeb\xd6\x9e\x5b\x6f\x68" + + "\x79\x46\x92\xdb\x34\x8d\xc9\xd3\x7f\x62\xf6\xcf\x2b\x19\x3c\x78" + + "\x30\x6f\xbd\xf5\xd6\x09\x83\x88\x5b\xb7\x6e\xa5\x61\xc3\x86\xfe" + + "\x64\x98\xa7\xca\x75\xbf\x76\xed\x5a\x9a\x35\x6b\x46\xed\x9a\x55" + + "\xf8\xdf\x07\x2f\x9d\x50\xaf\xae\x2c\x59\xf5\xa8\x88\x30\xda\xf4" + + "\x18\xc4\xbe\x43\x29\x08\x11\xbc\x4a\xca\xdc\xab\x2c\x8b\x5e\xdc" + + "\x37\x16\xce\x5f\xa5\x74\x5f\x1d\x1d\x15\xc1\x2f\x5f\x8e\x3e\xeb" + + "\xf1\xcb\x56\x6f\xe2\x95\x77\xbf\xc0\x95\xe7\x25\x2b\x3b\x17\x87" + + "\xdd\xc6\x1f\xdf\x4d\x28\x11\x25\xf5\xe5\x4d\x57\x08\x76\x3a\x48" + + "\xcf\xcc\xa2\x5c\x64\x38\x19\x99\x39\x2c\x5c\xbe\x8e\xc8\xf0\x50" + + "\x74\x5d\xa7\x7d\xeb\x26\x98\x52\xe2\xf1\xea\x17\x54\x23\x5c\x4a" + + "\xc9\xc6\x6d\xc9\xdc\x37\x78\x34\x9a\xaa\xf2\xed\x84\x21\xe8\xfa" + + "\xa9\x37\xf6\xe4\xba\xdc\x8c\x7a\xff\x6b\xb6\xed\xda\xcf\xa0\x41" + + "\x83\x4e\x59\x83\xdd\x30\x0c\x1c\x0e\x07\x6f\xbe\xf9\x26\x83\x06" + + "\x0d\x3a\xed\x7d\x5f\x7c\xf1\x45\x46\x8e\x1c\xc9\x8c\x0f\x5f\xa2" + + "\x66\xd5\xb8\x32\x69\xd5\x01\xde\x9c\xf8\x0d\x5f\xce\x5e\x98\xaf" + + "\x93\xe1\x0f\x48\x99\x39\xc5\x22\x7a\xb1\x12\xbd\xdc\xb3\x52\xa6" + + "\xbd\xae\xaa\x0a\xeb\xe7\x7f\x78\x5a\xe5\xfe\xf7\xe0\xd3\x8e\x3d" + + "\x87\xe8\xd6\x77\x38\x36\x4d\x65\x5d\xfe\x79\x97\x3b\x6c\x9a\xca" + + "\x7f\x5f\xfd\x88\xef\x17\xaf\xa6\x62\x4c\x14\x13\x5e\xe9\xe7\xeb" + + "\xc0\x84\xf0\xbf\xa0\xcc\xec\x5c\xfa\x3c\xf3\xb6\xff\x9c\x05\x0b" + + "\x16\xd0\xae\x5d\xbb\x53\xca\x6c\xdd\xba\x75\x5c\x79\xe5\x95\x64" + + "\x64\x64\x10\x11\x11\x71\x6a\xaf\x25\x2b\x8b\xf0\xf0\x70\xaa\x55" + + "\x8a\x65\xee\xa7\x23\xcf\x18\x5e\x04\x32\xc2\x42\x82\x68\x79\xfb" + + "\x00\x52\xd2\x8e\x03\x0a\x21\x21\x3d\x23\xf2\xf2\x56\x67\xea\xfa" + + "\x0e\x6b\x30\xae\x58\x6e\xac\x84\x7d\xe9\xb3\x40\x26\xfd\x86\xbd" + + "\x7f\x4a\xf7\xf8\xdf\x31\xb9\x69\x4a\xaa\xe7\x17\x22\x90\x94\xd2" + + "\xcc\x96\xe7\x01\x8f\x57\x67\xc8\xe3\x77\x11\x12\xec\xe0\x50\x4a" + + "\x3a\xcf\xbc\x3e\x09\xaf\x6e\xe0\xf5\xea\xfc\xba\x66\x33\xdd\x1f" + + "\x1f\xe5\x27\x79\x50\x90\x03\x80\xa9\x53\xa7\x9e\xb6\x63\x6c\xdc" + + "\xb8\x31\x36\x9b\x8d\xeb\xaf\xbf\xde\xdf\x49\x9e\xa4\xe0\x61\x61" + + "\xf4\xef\xdf\x9f\xbd\x07\x8f\xb2\xff\x50\x0a\x65\x15\x99\xd9\xb9" + + "\x2c\x9a\x3a\x26\xff\x9b\x29\x73\x72\xa6\x2f\xd5\xf5\x1d\x44\x46" + + "\xb6\xb7\x88\x5e\x1c\x28\x57\xee\xab\x83\x05\x9f\x97\xfc\xbe\x01" + + "\xb3\x88\x8b\x55\xec\x36\x8d\xdb\x3a\xb4\x46\x11\x82\x40\x59\x95" + + "\x2d\x84\x20\x2c\x34\x98\x15\xff\x7b\x8f\xfb\xef\xe8\xc0\xdf\xc9" + + "\x07\xb9\x7b\xe0\x68\xee\x1e\x38\x9a\x71\x9f\xfa\xd2\x38\xb7\x6c" + + "\x52\x8f\x19\x1f\x0e\x63\xed\x77\x13\x48\xa8\x5e\x89\x19\x33\x66" + + "\x9c\xa1\x13\x55\xd8\xb8\x71\x23\x6b\xd7\xae\xe5\xa7\x9f\x7e\x3a" + + "\xad\x5c\x0b\x8a\x3e\xbe\x38\x76\x4a\x40\xaf\x71\x3f\x9b\xec\x0d" + + "\xd3\xa4\xf7\xed\xed\xf3\x3d\x5c\x6f\x63\x45\x29\xf7\x46\x46\xc6" + + "\x02\xcb\x75\x2f\x3e\xab\x5e\x6e\xb4\x69\xa6\x3d\x0d\x88\x7a\x57" + + "\x54\x63\xea\xbb\x43\x8a\xd4\xe0\x5f\x96\xaf\xe3\xa9\x91\x13\xd9" + + "\xf2\xf3\x24\x3c\xde\xc0\xc9\x75\x56\x30\xa8\xa6\xeb\x06\x47\x8e" + + "\x65\x30\x63\xfe\x32\x2a\x57\x8a\xe6\xb6\xf6\xad\xfc\xd3\x70\x00" + + "\xa3\x3f\xfc\x9a\xa9\x73\x16\x91\x93\x93\x43\x70\x70\xf0\x69\xaf" + + "\x13\x1a\x1a\x4a\x4e\x4e\xce\x19\x49\x1c\x1a\x1a\x8a\xc3\xa6\xb0" + + "\x78\xea\x18\xcc\x32\x4a\x76\x29\x25\x9a\xaa\xd2\xe8\xe6\x47\xfc" + + "\xce\x62\x50\xd0\xc3\xe1\x2e\xd7\xc7\x59\x81\xf2\x8c\x97\xcc\xa2" + + "\x6b\x5a\x75\x4c\x33\xed\x59\x45\x89\xf8\x0d\x90\x5b\x77\xec\x65" + + "\xe1\x8a\xf5\x27\x95\x13\x3a\xe9\xa5\x00\x57\x54\xf7\xed\x2d\xbe" + + "\xd0\xbd\xd8\xa5\xd1\xba\x80\x6f\xea\xa7\x72\x85\xf2\x0c\x78\xb0" + + "\x2b\x77\xdc\x74\xcd\x09\x24\x97\x52\xfa\xa7\x23\xe7\xcc\x99\x73" + + "\x4a\x59\x15\x84\x41\x9b\x36\x6d\x22\x36\x36\x96\xe4\xe4\xe4\xd3" + + "\xca\x74\xc0\x80\x01\x64\x1c\xcf\xc6\x66\xd3\x28\xab\x10\x42\x60" + + "\x4a\x93\xc1\x0f\x75\xf7\x1b\x3f\x97\x6b\xf2\x7a\xcb\x75\x2f\x06" + + "\xe8\xfa\x9e\x7c\xa5\x0e\xef\x56\x20\xdc\xc1\xaf\x7c\xc8\xf3\x6f" + + "\x7e\x8a\xdd\x6e\x3b\x23\xd9\xab\x57\xf6\x65\x96\xd9\x77\xf8\x68" + + "\xc0\x2a\xde\x99\xfe\xd6\xb8\xae\xaf\xd6\xfb\x99\xdc\x72\x21\x04" + + "\x35\x6a\xd4\xe0\xd0\xa1\x43\x3c\xf7\xdc\x73\xa7\x3d\xee\xd6\x5b" + + "\x6f\x2d\xb3\x96\xbc\x30\x4c\x53\x72\x5f\xb7\x1b\x0a\x6b\x68\xcd" + + "\x90\x90\x06\x57\x59\x44\x2f\x26\x78\xbd\xfb\x52\x84\x08\x5f\x91" + + "\x6f\xac\x99\xbf\x68\x15\x37\xf6\x7e\x86\xb5\x9b\xfe\x26\x24\xe8" + + "\xd4\x19\x7a\x0b\x56\xc4\x8d\xfd\xf8\x7f\x65\x32\xb6\x74\xe7\x79" + + "\xb8\xae\x65\x63\xa6\x4c\x39\xfb\x4c\x50\xbb\x76\xed\xf8\xfa\xeb" + + "\xaf\xc9\xc8\xc8\x38\xa5\xac\x1e\x7c\xf0\x41\xc2\x42\x83\xf0\xea" + + "\x7a\x99\x26\xba\x10\x02\x45\x08\x9e\x7a\xb8\xbb\xff\xb7\x9c\x9c" + + "\x6d\x9f\x5a\x44\x2f\xb6\x38\x3d\x16\x29\x33\x5b\x2b\x4a\x84\x3f" + + "\xcf\xf1\xc1\x23\x69\xfc\xe7\xd9\xb1\xb4\xec\x36\x80\xcf\xff\xf7" + + "\x0b\x21\xc1\x4e\xec\xbe\xbd\xc4\xbe\xcc\xa9\xf9\x4a\xb9\x62\xed" + + "\xe6\x32\x39\xff\xab\x1b\x26\x0d\xeb\xf8\x76\xbf\x4d\x9b\x36\xed" + + "\xb4\x9d\x9d\x94\x92\x6a\xd5\x7c\x4b\x88\x6b\xd5\xaa\xc5\x0b\x2f" + + "\xbc\xc0\xcc\x99\x33\x19\x35\x6a\x14\x77\xde\x79\x27\x57\x5c\x71" + + "\x05\x7b\xf7\xec\xe6\xcb\x71\xcf\x9f\xd3\x3a\xfb\x40\x85\x61\x9a" + + "\x74\xbf\xa5\x4d\xa1\x28\x51\xaf\x1b\x30\x1d\x59\x69\x68\x84\xa6" + + "\xd5\x40\xd7\x93\xb1\xdb\x6b\x27\xe9\xfa\xa1\xde\x40\x4d\xd3\x74" + + "\x5d\x01\x7a\x23\xf0\x25\x0c\xa8\x5b\xb3\x0a\x15\x62\xca\x71\x7f" + + "\xb7\x1b\xa9\x52\x31\x9a\x76\x77\x3d\x8d\x4d\xd3\x58\x3d\x67\x7c" + + "\x99\x54\x4a\xbb\x4d\x23\xa9\xe3\xa3\x34\x4a\x6c\xcc\x9a\x35\x6b" + + "\xce\x78\xac\xaa\xaa\xa7\x9c\x27\xef\x73\xe7\xcd\x3c\xd9\xa7\x1b" + + "\x46\x11\x32\xef\x96\x15\x68\x9a\x4a\xff\x97\xc6\xb3\x68\xc5\x86" + + "\xfc\xef\xb1\xcf\xe9\xfa\xd1\xd1\x97\xfd\x73\x95\x0a\x0b\xa5\x27" + + "\x03\xe0\xf1\x6c\x5f\x0f\xac\x2f\xd4\x01\x24\x19\xc6\x91\xfb\xbd" + + "\x5e\xbd\xcd\xc6\x6d\xc9\x4d\x37\x6e\x4b\x66\xc5\x1f\x5b\xe4\xc3" + + "\x77\x75\x14\x40\x99\x76\x37\x85\x10\xd4\x4d\xa8\xc6\x1f\x7f\xfc" + + "\xc1\xa6\x4d\x9b\x68\xd0\xa0\xc1\x29\xc9\xba\x71\xe3\x46\xbf\xc5" + + "\x77\xd8\x6d\x74\xbb\xe9\x1a\xaa\x54\x8a\xa1\xf3\xf5\x57\x51\x2e" + + "\x22\xec\x84\x9d\x74\x16\x7c\xf9\x05\x1f\xeb\xdd\xc5\x4f\x74\x5d" + + "\xcf\xba\x32\x24\xe4\x6e\x72\x72\xbe\xba\xbc\xf5\xe5\x72\x69\x68" + + "\xb9\x72\x3d\xc2\xd2\xd2\xbe\x1b\x0c\xfa\xa3\xa0\xc7\xe5\xc7\xf4" + + "\x62\xc3\xf7\x13\x2f\x68\x09\xea\xe5\x8c\xdf\xd6\x6c\xe6\x89\x61" + + "\xef\xd1\xb0\x61\xc3\x53\x56\x56\x5d\xb3\x66\x0d\xcd\x9b\xfb\x32" + + "\x76\x75\xbb\xf9\x1a\x5e\x7f\xf6\x3f\xb8\xdc\x9e\x32\x2b\xaf\xa2" + + "\x77\xa2\xd0\xa6\xc7\x60\x32\xb3\x73\x25\x60\x02\x9a\xd3\xd9\x06" + + "\xb7\x7b\x99\x15\xa3\x97\x34\xd2\xd2\xbe\xcd\x02\xf7\xcb\xa0\x57" + + "\xd4\xb4\x98\xfe\xa0\x6c\x07\xd8\x73\xf0\x68\x99\x55\xda\xeb\x5b" + + "\x27\x71\x53\xdb\xe6\x6c\xda\xb4\x09\x21\x04\x3d\x7a\xf4\x60\xee" + + "\xdc\xb9\x8c\x1e\x3d\x9a\x98\xd8\x58\xae\xbf\xee\x1a\xee\xe9\xda" + + "\x9e\x6f\xde\x1f\xca\xcb\x83\xee\x23\x27\xd7\x1d\xf0\x79\xe1\x8a" + + "\x03\x52\xc2\x1b\xbe\x52\x55\x02\x50\x35\x2d\xb1\xfa\x99\x48\x1e" + + "\x1e\xde\x09\x80\x90\x90\x76\x96\x45\x2f\x6e\xd4\xaf\x2f\xd9\xb2" + + "\x45\xc8\x3b\x6e\x69\xc3\x8b\xfd\xef\x29\xa3\x0a\x29\xd1\x34\x95" + + "\xbb\x07\xbe\xce\xb6\x9d\x7b\xd1\x0d\x93\xf0\xd0\x20\x22\xc2\x42" + + "\x19\xfc\xd0\x1d\xdc\x74\x6d\xb3\x80\x5b\x6b\x70\xb1\x60\x9a\x26" + + "\x57\x76\x7e\x3c\xff\x5b\xe8\x5c\xc8\xee\xf2\x8f\xc5\x2f\x1f\x2b" + + "\x44\x48\x8e\xef\x1d\xa4\xc7\x4a\xe9\x89\x54\x94\xa0\x18\xd3\xcc" + + "\x39\x02\x36\xb7\xa6\x95\x3f\xaa\xeb\xfb\xd2\x2d\xa2\x17\x9b\x8b" + + "\x15\xfa\xaa\xaa\xe4\x3e\xb7\x7a\xce\xfb\xa2\xac\x5a\xa9\x82\x9d" + + "\x76\x41\x0e\xbb\x3f\x35\x95\x61\x18\xe8\xd6\x28\xfa\x05\x8f\x81" + + "\x8c\x1c\xff\x15\xdf\xce\x5b\x92\xbf\xad\x42\xcb\x06\x33\x34\xbf" + + "\x1b\x38\xd5\xf1\x52\x4a\x29\xfe\xf9\x1e\x39\x25\x24\xa4\xd9\x23" + + "\xd9\xd9\xbf\x78\x62\x62\x5e\x21\x25\xe5\x45\x8b\xe8\xe7\x03\x55" + + "\xad\x86\x69\x1e\x0e\x92\xd2\x93\xbb\x6a\xf6\x78\x6c\x9a\x6a\xb9" + + "\xa4\x16\x8a\xb5\x03\x75\xe5\x79\x68\xdd\x6d\x20\x00\x41\x41\x41" + + "\xd4\xa8\x51\x83\x94\x94\x14\x12\x13\x13\xe9\xd8\xb1\x23\x55\xaa" + + "\x54\xe1\xf6\xdb\x6f\xc7\x6e\xff\xa7\x90\x48\x4a\x4a\x0a\x3d\x7a" + + "\xf4\x60\xd9\xb2\x65\x98\xa6\xc4\xe9\x6c\x1c\xef\x76\xaf\x4f\x0e" + + "\x09\xb9\x96\x9c\x9c\xa5\x97\xec\x79\x2e\xdb\x75\x8f\x86\xb1\x17" + + "\x55\xad\xee\x31\x8c\x3d\x1c\x3e\x7a\x8c\x6a\x95\x2b\x58\xda\x69" + + "\xa1\x58\x2d\x7a\xb0\xd3\x81\xcd\xa6\xe1\xf5\xea\xa4\xa5\xa5\xe1" + + "\x74\x9e\xbd\xc4\x7a\x4c\x4c\x0c\x8b\x17\x2f\x06\xe0\xa1\x87\x1e" + + "\x62\xd2\xa4\x49\xbb\x6d\xb6\xea\x75\x73\x72\x96\x6e\xbb\x94\xcf" + + "\x73\x59\x97\x90\x35\x8c\x3d\x86\x10\x21\xcb\xde\x9a\x34\x03\xcb" + + "\x96\x5b\x28\x09\x0c\xed\xe7\x1b\xff\xf9\xfc\xf3\xcf\xcf\xf9\xdc" + + "\x4f\x3e\xf9\x04\xa7\xd3\x29\xbd\xde\xc3\x7f\x5d\xea\xe7\xb8\xec" + + "\x6b\x45\x0b\x11\xf9\xc2\x92\x95\x7f\xe2\xf1\xea\x96\x56\x5a\x28" + + "\x76\xdc\x72\x5d\x33\x00\xfa\xf6\x2d\x5a\xbd\x87\x82\xbc\x7c\xde" + + "\xfc\x5d\x95\x77\xdc\x71\x87\x80\x3c\x14\xa5\x7c\x75\x8b\xe8\x17" + + "\x80\x72\xe5\x5a\xfe\x0a\x98\xae\x73\xc8\xd3\x66\xc1\x42\x51\x61" + + "\xd3\x34\xc2\x43\x7d\x5b\x81\xdd\x6e\x77\x91\x5c\xfe\x8f\x3f\xfe" + + "\x18\xbb\xdd\x4e\x52\x52\x12\x3d\x7a\xf4\xc8\xef\x00\x74\x9b\x45" + + "\xf4\x0b\x40\x66\xe6\x46\x09\x98\x63\x27\xcd\x90\xaa\xa2\x58\x9a" + + "\x69\xa1\xd8\x63\xf5\x61\x03\x7a\x03\xbe\x01\xb9\xdc\xdc\xdc\xb3" + + "\x5a\xf4\x47\x1f\x7d\x94\x89\x13\x27\xb2\x61\xc3\x06\x7a\xf5\xea" + + "\x05\x28\x48\x79\x7c\x87\x45\xf4\x0b\x80\xc7\xb3\x1d\x21\x22\xc6" + + "\xcc\xfa\x69\xb9\xc8\xca\x75\x59\x56\xdd\x42\xb1\xa3\x5d\xab\x24" + + "\x7f\xb1\x90\xe3\xc7\x8f\x9f\xb5\x63\x90\x52\xf2\xf0\xc3\x0f\x17" + + "\x78\x01\x52\x51\x22\x46\x59\x31\x7a\x31\xa0\x6e\xdd\x8c\x21\x80" + + "\x31\x6c\xec\x14\xec\x65\x38\x81\x82\x85\x92\x81\xaf\x7c\xb5\xe6" + + "\x27\xfa\xd9\x8c\x49\x41\x41\xcf\x3b\xef\xbc\x13\x40\x80\x18\xa9" + + "\xaa\x95\x2c\xa2\x5f\x28\xb6\x6e\x15\x12\xd4\xa3\x0b\x7e\x5b\xc7" + + "\x86\xad\xbb\x2d\xcd\xb4\x50\xac\x90\x52\xd2\xa4\x61\x2d\x00\x96" + + "\x2f\x5f\x5e\xe4\xf5\x1a\xeb\xd6\xad\x03\x20\x24\xa4\xb3\xc7\x30" + + "\x0e\x5e\xda\x10\x24\x10\x5e\x44\x50\x50\x53\x0c\x23\xbd\xad\xc7" + + "\xb3\x6b\x71\xb9\x88\x30\x96\x7e\x33\xb6\x4c\x16\x25\xb0\x50\x72" + + "\x58\xbc\x72\x03\x4f\x8e\xf8\x00\x9b\xcd\x46\xf3\xe6\xcd\x49\x48" + + "\x48\xa0\x56\xad\x5a\x74\xe8\xd0\x81\xab\xae\xba\xca\xbf\x87\xa0" + + "\xe0\xff\x8c\x8c\x0c\xa2\xa2\xa2\xa4\xa6\xc5\x4c\xd4\xf5\x94\xc7" + + "\x2e\xf9\x58\x43\x60\xbd\x0e\xdb\x06\xf0\x26\x4e\x7d\x77\x08\x75" + + "\x13\xaa\x5a\x2b\xe5\x2c\x14\x9f\x66\x69\x1a\x0d\x3a\x3c\x74\xca" + + "\xbf\xd5\xae\x5d\x5b\x8e\x1c\x39\x52\x38\x9d\x4e\x42\x43\x43\x99" + + "\x3e\x7d\xba\x9c\x30\x61\x82\x00\xdb\xb6\x90\x90\xfa\xf5\x73\x73" + + "\x0f\x99\x52\x1e\xb5\x88\x5e\x5c\xd0\xb4\xca\x6d\x75\xfd\xc0\xe2" + + "\x86\x75\xe2\xf9\xfa\xbd\x17\xca\x7c\x7a\x24\x0b\xc5\x49\x74\x95" + + "\xc6\x1d\x1f\x45\xd7\x0d\x54\x35\x74\x83\x61\x84\x76\x51\xd5\xbc" + + "\xc7\x0d\xc3\x9d\x0a\xde\x87\x41\xaf\x53\x28\xaa\xcf\x16\xc2\xf6" + + "\x3f\x29\xdd\xf7\x97\x96\xf6\x07\xd4\x7c\x94\xae\x1f\x58\x02\xb6" + + "\xbf\x37\x6d\xdb\xcd\xda\xcd\x7f\x03\xd6\x08\xbc\x85\xe2\x81\x99" + + "\x9f\x12\x1a\xc0\x30\xf2\x1a\xc3\xe1\x7d\x86\x91\xfe\xbc\xa2\x54" + + "\x7d\x0b\xf4\xba\x09\x09\xaf\x39\x20\xb2\x4b\xe5\xca\x0f\xd6\x01" + + "\x23\x4c\x4a\xf7\xfd\x0e\x47\x43\x2c\xa2\x97\x08\x6a\x20\x44\xe4" + + "\x93\x00\x4b\x7e\xff\x13\x45\x58\xf3\xea\x16\x8a\x07\x52\xca\x42" + + "\x33\x3a\x5e\x34\xad\x5a\x0d\xa7\xb3\x19\xa6\xb9\x1d\x80\x9d\x3b" + + "\x9f\xf7\x40\xc6\xdc\x03\x07\x26\x6f\x2f\x38\x27\x2f\x6f\x93\x45" + + "\xf4\x92\x41\x32\x52\xa6\xcc\x07\x2d\x79\xd9\xea\x4d\xd2\x66\x4d" + + "\xb5\x59\x28\x36\xa2\xfb\xa6\xd9\x0a\xbe\x1a\x46\xea\x2d\x6e\xf7" + + "\x9a\xcb\xa6\xfd\x01\x67\xf2\x14\xa5\x0a\x8a\x12\x3c\x75\xdb\xce" + + "\x7d\xc2\x57\x38\xcf\x82\x85\x62\x70\xdd\x4d\x93\x20\x87\xa3\xe0" + + "\xab\x00\x79\xf3\x65\xc5\x8b\xc0\x7b\x21\xfb\x31\xcd\xcc\x21\xa0" + + "\xf0\xcd\xdc\xc5\xd6\x4a\x39\x0b\xc5\x02\x21\x04\x3d\x3a\x5f\x5b" + + "\xf8\xa7\x5b\x2c\xa2\x97\x0a\xcb\x1e\x32\xe4\x9b\x79\x4b\x08\x0d" + + "\x09\xb2\xb4\xd4\x42\xb1\xa0\x41\xad\xea\x85\x5c\x79\xb7\xcd\x22" + + "\x7a\xa9\xb0\xec\x59\xaf\x1d\x3d\x96\xc9\xa7\xdf\xfe\x68\x59\x75" + + "\x0b\xc5\x82\x56\x57\xd6\x2f\x1c\xb5\xa3\x28\x95\xab\x5b\x44\x2f" + + "\x15\xee\x56\xd0\x07\x53\xa6\xff\x24\x2d\xab\x6e\xa1\x38\xa0\xa9" + + "\x2a\x21\xc1\xce\x42\x56\x3d\xfd\x6a\x87\xa3\xa6\x45\xf4\x4b\x89" + + "\xd0\xd0\x4e\x28\x4a\x8d\x21\x07\x8f\x1c\x13\x53\x67\x2f\xb2\xac" + + "\xba\x85\x62\xf0\x12\xa5\x7f\x73\x0b\x20\xc1\xb8\x27\x2f\x6f\x97" + + "\x45\xf4\x4b\x89\xec\xec\x79\x18\xc6\x96\x0c\x70\x7c\xfb\xd1\xb4" + + "\xf9\xb2\x70\x4f\x6c\xc1\xc2\xf9\xa0\x20\xbd\x36\x80\xaa\xaa\x42" + + "\x4a\xfd\x6a\xcb\x75\x2f\x05\x50\xd5\xca\x08\x61\xff\x62\xff\xa1" + + "\x14\xf1\xed\xbc\xa5\x96\x55\xb7\x70\x81\x16\xdd\xc4\x69\xf7\x8d" + + "\xc1\x05\x07\x07\x03\x32\xc2\x22\x7a\x29\x80\x61\x1c\xc0\xe1\x78" + + "\x6a\x3e\x08\xde\xfa\x64\x3a\xc1\x41\x96\x55\xb7\x70\x01\xfa\x64" + + "\x9a\x24\xd4\xa8\x0c\x40\x56\x56\x16\x60\xa2\xaa\x09\x35\x2e\x0b" + + "\xa3\x17\xe8\x2f\x47\xd7\x17\x4b\x45\x09\x4d\xca\x75\x65\xd7\x89" + + "\x29\x17\x21\xea\xd4\xac\x62\xed\x6a\xbb\x44\x6e\xaf\x6e\x18\xec" + + "\x3d\x70\x94\x43\x47\x8f\x91\x76\x3c\x8b\x2d\x3b\xf6\xb2\xff\x70" + + "\x2a\x3b\xf7\x1c\x64\xef\xa1\xa3\x78\x3d\x3a\x4e\x87\x9d\x90\x20" + + "\x67\xa9\xac\xf0\x2a\x84\x60\xfb\xce\xfd\xac\xdf\xb2\xb3\xd0\x73" + + "\xe5\xcd\x04\xef\x9e\xd2\x2e\xff\x80\xd7\xf8\xa0\xa0\x46\xe4\xe5" + + "\x1d\x89\x37\xcd\xa3\xbb\xec\x36\x8d\xb5\x73\x3f\xb0\xf6\xaa\x5f" + + "\x02\x68\xaa\xca\x55\xb7\x0f\x20\x27\xd7\x75\xd6\x63\xaf\x6f\x9d" + + "\x44\xef\xae\xed\x69\x50\xbb\x06\x4e\x87\xbd\x54\x15\x85\x4c\x49" + + "\x3b\xce\x8d\xbd\x9f\x2d\x44\xfe\xe0\x7b\xa5\xcc\xfd\xbf\x52\x2f" + + "\xff\x40\x57\x30\x97\x6b\x23\xc0\x6e\x21\x42\x16\x7b\xbc\x39\x6d" + + "\x9f\x79\xfd\x63\xf1\xea\xd3\x7d\x2c\xe6\x5d\x64\xb8\xf3\x3c\xe4" + + "\xe4\xba\x88\x8a\x8a\x22\x36\x36\x96\x88\x88\x08\x2a\x57\xae\x4c" + + "\x70\x70\x30\x79\x79\x79\x6c\xdf\xbe\x1d\x29\x25\xc7\x8e\x1d\x63" + + "\xd1\x8a\x0d\x2c\x5c\xee\xab\x9e\x3d\xa4\xdf\xdd\xdc\x76\x43\x2b" + + "\xec\x36\xad\x54\x90\xbd\x6a\xc5\x18\xca\x47\x86\x73\x2c\x23\xd3" + + "\x17\xfb\x2a\x76\xa7\x61\xe4\x96\x7a\xf9\x97\x09\x1f\x36\x38\xb8" + + "\x01\x2e\xd7\x81\x6a\x52\x66\xec\x01\x58\x31\xf3\x5d\x82\x1c\x76" + + "\x8b\x7d\x17\xd3\xa2\x68\x2a\x0d\x3b\x3c\xcc\xd4\xa9\x53\xe9\xd5" + + "\xab\xd7\x09\x55\x5d\x4f\x55\xe1\x75\xc7\x8e\x1d\x5c\x7f\xfd\xf5" + + "\xec\xdb\xb7\x0f\x4d\x53\x59\xf2\xf5\x58\x82\x9d\xf6\x4b\x4e\x76" + + "\x21\xe0\xbe\xc1\x6f\xb0\x61\x6b\xc1\xb4\x9a\xed\x05\xf0\xbe\x5a" + + "\xda\xe5\x5f\x26\xf6\x71\xe6\xe6\x6e\x46\xca\x8c\xbd\x36\x5b\xe5" + + "\xc7\x00\x3a\x3d\xf8\x82\x5f\xc1\x2c\x5c\x1c\x6c\xdd\xb1\x17\x20" + + "\x3f\xfd\x31\x27\x10\xf6\x54\xe4\x4d\x48\x48\x60\xef\xde\xbd\x1c" + + "\x3d\x7a\x14\x9b\xcd\x4e\x97\xff\x0c\xf5\x4f\x6d\x5d\x4a\x98\xa6" + + "\xa4\x62\x6c\xb9\x42\x6d\x57\x2e\x8b\x11\xde\x32\xb5\x61\xdb\xeb" + + "\x3d\xf0\x21\xd8\x48\xcb\xc8\x92\x65\xb9\xae\xfa\xc5\x86\x94\x92" + + "\xe4\xfd\x87\xcf\xd1\x72\xfa\xde\x4d\x74\x74\x34\xe9\xe9\xe9\xa4" + + "\x65\x64\xf1\xea\xfb\xd3\x2e\xbd\x0b\x2c\x04\x49\xf5\xaf\x28\xf4" + + "\x6c\x7a\x79\x8b\xe8\xa5\x10\x21\x21\x4d\x63\x00\x71\xdb\x43\xc3" + + "\x30\xac\xd2\xc2\x17\x8d\x1c\x39\xb9\x6e\x54\x55\x3d\xaf\x73\x1d" + + "\x0e\x07\x93\x26\x4d\x62\xda\x77\x8b\x4a\xc5\xf3\xd4\xbb\xa2\x5a" + + "\xa1\xf6\x69\xd7\x58\x44\x2f\x65\x68\xd2\x64\x21\x39\x39\x2b\x53" + + "\x55\xb5\xee\x4d\x00\xdf\xcc\x5b\x6a\xb1\xf0\x22\xc5\xe7\xab\xff" + + "\xdc\x8e\x61\x18\xcc\x9d\x3b\xf7\xbc\x3c\x82\x9b\x6e\xba\x09\x29" + + "\x25\x03\x47\x4c\xb8\xe4\x21\x57\x83\xda\x85\x77\xb1\x79\x83\x2c" + + "\xa2\x97\x32\xac\x5b\x77\x3d\x36\x5b\x4b\x0c\xe3\xaf\x9f\x14\x25" + + "\x7c\xf3\xe8\x0f\xa7\x91\xe7\xf1\x5a\xb1\x7a\x09\x43\x55\x14\x92" + + "\xf7\x1f\x21\x32\x32\x92\xce\x9d\x3b\x17\xf9\xbc\xac\xac\x2c\x6a" + + "\xd5\xaa\x85\x10\x82\xca\x95\x7d\x0b\x55\x56\xfc\xb1\xe5\x92\x87" + + "\x5c\x51\x11\x61\x85\x2d\x7a\x05\x8b\xe8\xa5\x32\x4e\x5f\x99\xef" + + "\xc2\xbf\xd5\x08\xe0\xc6\xde\xcf\x49\xd3\x34\x2d\xb2\x97\x30\xf2" + + "\xf2\x3c\x74\xeb\xd6\xed\x9c\xce\x09\x0b\x0b\x63\xc4\x88\x11\xc4" + + "\xc7\xc7\x03\xf0\xfc\xf3\xcf\xe3\xd5\x0d\x84\xb8\xb4\x03\xa9\xa6" + + "\x69\xe2\xcc\x9f\xb5\x91\xd2\x13\x0e\x09\xd1\x16\xd1\x4b\x29\xb2" + + "\xb2\x1e\x96\xc1\xc1\x8d\x6b\x65\x66\xe7\x98\xfd\x5e\x7a\xdf\xca" + + "\x2f\x57\x42\x90\x52\xb2\xe7\xc0\x51\x76\xec\x39\xc8\xf8\xf1\xe3" + + "\xcf\xf9\xdc\x5b\x6f\xbd\x95\xe4\xe4\x64\x26\x4f\x9e\x4c\xdb\xb6" + + "\x6d\x91\x52\x92\x9d\xe3\xbe\xa4\x56\xdd\x30\x4c\x22\xc3\x43\x0b" + + "\xfd\x92\x52\xdb\x22\x7a\x29\x46\x6e\xee\x86\x1d\x8a\x52\xee\xdd" + + "\x15\x6b\xb7\x88\xdd\xfb\x0e\x5b\x56\xbd\x04\x10\xe4\x74\xf0\xd8" + + "\xd0\x77\x7c\x9f\x83\x82\xce\x48\xea\x7f\x43\x08\x41\x48\x48\x08" + + "\x3d\x7a\xf4\x60\xe0\xc0\x81\xbc\xf9\xe6\x9b\xa8\x8a\x42\x68\xb0" + + "\xf3\x92\xbe\x2b\xaf\x57\xa7\x6a\xc5\xc2\x46\xdc\xdb\xce\x22\x7a" + + "\x29\x87\x69\xa6\x0d\x06\xe5\x48\xf7\xc7\x5e\x91\x87\x53\xd2\x2d" + + "\x66\x16\x93\x15\x17\x42\xa0\x08\xc1\xec\x9f\x96\xb3\x7b\x9f\x6f" + + "\x6a\xed\x86\x1b\x6e\xa0\x47\x8f\x1e\x34\x6b\xd6\x8c\xb8\xb8\x38" + + "\x1a\x36\x6c\x48\xbd\x7a\xf5\x78\xf0\xc1\x07\x59\xb2\x64\xc9\x69" + + "\x09\xdf\xb9\x73\x67\xb2\xb2\xb2\x58\xb0\x60\x01\x12\x89\x84\x4b" + + "\x6b\xd1\x4d\x93\xc4\x7a\x09\x85\x9f\xd8\x9e\x98\xf8\x99\x45\xf4" + + "\xd2\x0c\x9b\xad\x3e\x8a\x12\xf5\x41\x9e\xc7\x23\x3a\x3e\x30\x84" + + "\x23\xa9\xe9\x96\x65\xbf\x00\x08\x21\x48\x49\x3b\xce\xe8\x0f\xbf" + + "\x66\xcb\x8e\xbd\x3c\x3b\xfa\x13\x84\xe2\x23\xe5\x82\x05\x0b\x98" + + "\x3f\xef\x3b\x4c\xf7\x71\xea\xc7\xc7\x51\xbb\x4a\x24\xc7\x8f\x1d" + + "\xe5\xb3\xcf\x3e\xa3\x5d\xbb\x76\xdc\x7d\xf7\xdd\xfe\xfa\x65\x85" + + "\xb1\x6c\xd9\x32\xff\xe7\x07\xee\xe8\x50\x2a\xde\xcf\x1d\xb7\xf8" + + "\x67\xd5\x24\x98\x55\xfe\xfc\xf3\x81\xd2\xfd\x5e\x2c\xd5\xf4\x53" + + "\x7e\x27\x78\xe3\x6b\x56\xab\x28\xbe\xff\x6c\x14\xb9\x2e\x0f\xd6" + + "\x7a\x9a\x73\xb7\xe4\xab\x36\x6c\xa3\xef\x90\x71\xdc\xdb\xed\x06" + + "\xbe\xf8\xdf\x2f\xd4\xab\x55\x9d\xf9\x9f\xbe\x42\x6a\x7a\x16\x36" + + "\x4d\x25\x34\x24\x08\x8f\xc7\x8b\x69\x4a\x0c\xd3\x24\xc8\x69\x67" + + "\xe8\x98\xcf\xf8\x66\x9e\xcf\xa2\x27\x26\x26\xf2\xc5\x17\x5f\x90" + + "\x98\x98\x08\xc0\xe1\xc3\x87\xa9\x58\xb1\x22\x8a\x22\x18\x37\xec" + + "\x71\xda\xb5\x6a\x5c\x2a\xd6\x3f\x9c\xb8\xb9\xc5\xb6\x0c\xbc\xd7" + + "\x5a\x44\x2f\xe5\xd0\xb4\xba\xd4\xaa\xf5\x63\xd0\xd6\xad\x35\x72" + + "\x41\xca\xfe\xf7\x77\x15\x7d\xef\xe9\x84\xae\x5b\xbb\xdc\xce\xd5" + + "\x9a\xb7\xbf\xe7\x19\x52\xf3\xf3\xe9\x57\xad\x18\xcb\xf7\x9f\x8d" + + "\xcc\x1f\x29\x17\x67\x38\x0f\x56\x6d\xd8\xce\xc8\xf7\xbe\x64\xcf" + + "\x81\x23\x00\x38\x1c\x0e\xec\x76\x3b\x59\x59\x59\x84\x85\x06\xf3" + + "\xc5\xdb\xcf\x52\xbd\x52\x6c\xa9\x59\xcd\x78\x2c\x3d\x93\xf6\xf7" + + "\x3c\x53\xe0\x18\x6f\x06\xb3\xa1\x45\xf4\xcb\x46\x51\x43\x3e\x94" + + "\x32\xa7\x2f\xc0\x8b\x03\x7a\xd3\xfd\x96\x6b\xb0\xbc\xf8\xa2\x5b" + + "\x73\x80\x26\x9d\x1e\x23\xa6\x5c\x04\xa1\x21\xc1\xcc\xfb\xf4\x15" + + "\xf2\x3c\xde\x22\x91\x53\x4a\x49\x90\xd3\xc1\xb4\xb9\x8b\x99\x3e" + + "\x6f\x29\xa6\x94\x64\x1c\xcf\xa6\xfd\xd5\x4d\x18\xfe\xe4\xbd\x64" + + "\x66\xe7\x96\xaa\x25\xcb\xe1\xa1\xc1\xd4\xbc\xf6\xbe\x02\xa2\xef" + + "\x05\xb3\xba\x45\xf4\xcb\xcb\x85\xdf\x07\xde\xca\x80\x98\xfb\xe9" + + "\x48\x2a\xc7\x45\x5b\x42\x2a\x02\x74\xc3\xa0\xed\x9d\x83\xf9\xf0" + + "\xd5\x41\x34\x6d\x78\x05\x86\x69\xa2\xeb\x17\x96\x3c\x42\x55\x95" + + "\x52\xbb\x4c\x39\x34\xd8\x49\x83\x9b\x1e\x26\x2f\xcf\x0b\x68\xab" + + "\x41\x6f\x51\x9a\xdf\x8f\x55\x85\xb0\x10\xec\xf6\xfa\xa8\x6a\xe4" + + "\x90\x82\x0e\xf0\xe1\xe7\xc6\xe2\xb0\x6b\xd6\xe0\x5c\x11\xb0\x33" + + "\xf9\x10\xe3\x86\x3d\x4e\xa3\x3a\x35\xf0\x78\x75\x0c\xe3\xc2\x93" + + "\x45\x94\xe6\xbd\x08\x79\x1e\x2f\x89\x75\xe2\x0b\x5a\x5a\xea\x57" + + "\xc7\x59\x44\x2f\x04\x8f\x67\x0b\x86\x91\xf2\x05\xa8\x3b\x00\x0e" + + "\x1d\x4d\xe3\x91\xe7\xc6\xa1\x28\x96\x98\xce\x86\x3a\x09\x55\x68" + + "\xde\xb8\x4e\x19\xf2\x60\x4c\xe2\x62\xcb\xe7\x87\x7c\xf6\xe5\x16" + + "\xd1\x2f\x43\x68\x5a\xcc\x23\x05\x9f\x7f\x5d\xb3\x89\x5d\xfb\x0e" + + "\x5b\x42\xb1\x70\xaa\x91\x85\x82\xf1\x85\xe4\xf0\xf0\xab\x2d\xa2" + + "\x5f\x76\xbd\xb5\x7e\x78\x11\x84\xff\x5f\xc1\x9b\x7c\xee\xf5\x4f" + + "\x2c\xa1\x58\x38\x2d\x84\x50\xc3\x32\x33\x7f\xb3\x88\x7e\xb9\x41" + + "\x55\xeb\x02\x99\xf7\x0a\x61\x13\x00\xdb\x76\xed\xc3\xe5\xf6\x58" + + "\x82\xb1\xf0\x8f\x2d\xcf\x9f\x25\xf0\x7d\xf4\x6a\x36\x5b\x43\x8b" + + "\xe8\x97\x1b\x0c\xe3\xaf\xfc\x4f\x11\x37\x14\xfc\x36\x7c\xdc\xe7" + + "\xd6\xa0\x9c\x85\x13\x88\x6e\xf3\x25\xd2\x10\x40\x13\xaf\x77\x93" + + "\x45\xf4\xcb\xf7\x65\xa6\x2e\x00\xc7\x34\x40\xfe\xf2\xdb\x5a\x1c" + + "\x76\x9b\x45\x76\x0b\xa7\x70\xdd\x83\xbe\x2d\xed\x6d\xb4\x88\x7e" + + "\xc6\x17\x18\x8d\xcd\x16\x3b\x08\x10\x86\x61\xf2\xf3\xaf\xeb\xac" + + "\x3c\x73\x16\xf2\xc3\x3b\x85\xb4\xe3\x59\xf9\x06\xc1\xdc\x62\x11" + + "\xfd\xf2\xb6\xe8\x78\xbd\xfb\x0e\x0b\x11\xf5\x05\x20\x9f\x79\xed" + + "\x23\xcb\xa2\x5b\x00\x7c\x05\x29\x0e\x1e\x4d\xf3\x91\x48\x09\xca" + + "\xb0\x88\x7e\xb9\xbf\x50\xad\x0e\x52\xf2\x28\x20\xf2\x3c\x5e\x3c" + + "\x5e\xaf\x25\x14\x0b\x18\x86\x49\x79\x7f\x4a\xa9\xb0\xdd\x16\xd1" + + "\x2f\x73\xe8\xfa\x36\x20\x3d\x57\xd3\xe2\x46\x03\xf2\xf5\x0f\xbe" + + "\xb6\x84\x62\x01\x21\x04\xde\xfc\x4d\x4f\xa6\x99\x61\xad\x8c\x0b" + + "\x04\x44\x46\xb6\x41\xd7\x53\x5e\x06\x21\x66\xfe\xf8\x9b\xb4\x6a" + + "\xb7\x59\xd0\x0d\xa3\x50\x21\x87\x90\x83\x16\xd1\x03\x00\x19\x19" + + "\xcb\x00\xc3\x25\x44\xf8\x1b\x80\x70\xb9\xf3\x2c\xa1\x58\x16\x9d" + + "\x7f\xf4\xc0\x65\x59\xf4\x40\x81\xaa\x56\x05\x8e\xbf\x00\xc8\x29" + + "\x33\x7e\xb1\x76\xb4\x95\x75\x7d\x50\x04\x19\x59\x39\x00\x68\x9a" + + "\xd2\xc9\x22\x7a\x80\xc0\x30\xf6\xa1\xaa\x09\x06\x08\xf9\xc9\xb4" + + "\xf9\x64\xe5\xb8\x2c\xa1\x94\xe9\x8e\x5f\xe5\x58\x7a\x26\x80\xd4" + + "\xf5\xf4\x04\x8b\xe8\x81\x14\x97\xe9\x3b\x25\x04\x2d\x2e\x70\xdd" + + "\x2c\x94\x5d\x98\xa6\x49\x48\x90\x03\x40\x80\x50\x2d\xa2\x07\x18" + + "\x14\x25\xe4\x13\x00\xb7\xc7\x63\xcd\xa9\x97\x61\x78\x75\x83\x6a" + + "\x95\x63\x0b\xfc\xbd\x3a\x16\xd1\x03\x0c\x76\x7b\xad\x9f\x01\x16" + + "\xaf\xd8\x80\xa2\x58\x56\xbd\xac\x42\x4a\x89\xd3\x6e\xcf\xff\xa6" + + "\x25\x5b\x44\x0f\x20\x04\x05\xb5\xc6\xed\x5e\x9e\x0a\x98\x6f\x7d" + + "\x32\xdd\x4a\x48\x51\xa6\x3d\x3b\x85\xcc\xec\xdc\x02\xfb\x1e\x65" + + "\x11\x3d\x80\xe0\x72\xf9\x13\x89\x18\xb9\xae\x3c\x0e\x1e\x39\x66" + + "\xb9\xef\x65\xd8\xa2\x87\x85\x06\x03\x20\x84\x23\xd3\x22\x7a\x60" + + "\x42\x07\xe4\xec\x9f\x57\xa0\x5a\x56\xbd\xcc\xc2\x34\xcd\x7c\xd2" + + "\x7b\x6b\x5a\x44\x0f\x30\xd8\x6c\xf5\x01\xc7\x6e\x40\x4c\xf8\x62" + + "\x0e\x96\x3d\x2f\x9b\x10\x80\xc7\xab\x17\x7c\xb5\x36\xb5\x04\x1a" + + "\xbc\xde\x2d\x80\x3a\xaf\xe0\xfb\x5f\x3b\xf7\x59\xee\x7b\x59\x24" + + "\xba\x22\xc8\x75\xb9\x01\x24\x08\xdd\x22\x7a\x00\x42\x51\x42\x56" + + "\x17\x7c\x9e\xf5\xe3\x6f\x68\x9a\x6a\x09\xa5\x2c\xc6\xe9\x7e\xe3" + + "\xae\xae\xb4\xd9\xea\x58\x44\x0f\xbc\xd8\x2c\xc5\x9f\x51\x64\xea" + + "\x77\x8b\x30\x4d\xcb\xa2\x97\xb9\x10\x4e\xd3\x48\xde\xe7\x2b\x1f" + + "\x25\x84\xb6\xdf\xeb\xdd\x66\x11\x3d\x40\x9d\x37\xff\xce\x96\x1d" + + "\x7b\x0e\x5a\xe2\x28\x6b\xd6\x5c\x4a\xbf\x27\x27\xa5\x5e\xbd\xb4" + + "\xb7\xd7\x22\xfa\xf9\x23\xa7\xe0\xc3\xae\xbd\x87\x2c\x69\x94\x31" + + "\xa8\xaa\x42\x46\x66\x76\x81\x45\xaf\x66\x11\x3d\x00\xe1\x74\x5e" + + "\x89\xa2\x84\x7d\x56\xf0\xfd\xcf\xbf\x76\x59\xd3\x6c\x65\xd0\xa2" + + "\x3b\x1d\xb6\x7c\xa2\xab\x5b\x2d\xa2\x07\x20\xdc\xee\xb5\x98\x66" + + "\xe6\x53\x05\xdf\x97\xfe\xbe\xd1\x1a\x90\x2b\x83\x31\x7a\x41\x05" + + "\x1f\x29\x75\x6b\x7a\x2d\xc0\xe3\xf4\x14\x80\x3d\x07\x8e\x94\xea" + + "\x82\x80\x16\x8a\x1f\x3b\xf6\x1c\x44\x9a\x12\x10\x48\x99\xfb\x91" + + "\x45\xf4\x80\x86\x7d\x6a\xc1\xa7\xd5\x1b\xb7\x5b\xf3\xe9\x65\x08" + + "\xbf\xaf\x2f\x28\xf2\xa1\xae\xbd\x1c\xda\x6b\x11\xfd\x02\xa0\x69" + + "\x15\x37\x14\x7c\xfe\xee\x97\x15\x96\xfb\x5e\x86\x90\x96\xe1\xcb" + + "\xe9\xae\x28\x36\xc5\x22\x7a\x80\x43\xd7\x93\x3f\x2d\xf8\xbc\x69" + + "\x7b\x32\x36\x8b\xe8\x65\x02\x52\x4a\x92\xf7\x1f\x2e\xf8\xb8\xdd" + + "\x22\x7a\xd9\x88\xd3\x0d\x80\xdd\x7b\x0f\xa3\x69\x9a\x25\x8e\xb2" + + "\xe0\xc9\xa9\x2a\xfb\x0f\xa7\x02\x08\x90\x3b\x2c\xa2\x97\x09\x28" + + "\x7b\x00\xbc\xba\x5e\x28\x6e\xb3\x10\xd0\x5d\xbb\x10\xe8\xf9\x39" + + "\xdd\xa5\x94\x1e\x8b\xe8\x65\x03\xfb\x0a\x3e\x04\x39\xec\x96\x34" + + "\xca\x42\xd7\xae\x08\x72\x7c\x1b\x5a\x00\x8e\x58\x44\x0f\x70\x44" + + "\x46\xde\x03\xd8\xfd\xd9\x28\x72\x5d\x56\xbe\xf7\x32\xe1\xba\x6b" + + "\x2a\xa9\x69\xc7\xf3\x49\x1f\xbe\xd3\x22\x7a\x80\x23\x23\xe3\x4b" + + "\x84\x70\xfc\x56\xf0\x3d\x38\xd8\x69\x09\xa5\x2c\xbc\xf7\xcc\x1c" + + "\xd2\x8f\xfb\x96\xbf\x9a\xa6\xf8\xcb\x22\x7a\xd9\x40\x70\xc1\x87" + + "\xb4\xf4\x4c\x4b\x1a\x65\x00\xfb\x0f\xa5\x14\x72\xe3\x8d\x04\x8b" + + "\xe8\x65\x22\x5e\x8b\x5f\x5a\xf0\xf9\x70\x6a\xba\x25\x90\x00\x87" + + "\x94\x92\x95\xeb\xfe\x59\xda\xae\xaa\xc1\xe9\x16\xd1\xcb\x00\x0c" + + "\x63\x9d\x7f\x30\x26\x24\xc8\x72\xdd\x03\x1d\xaa\xaa\x30\x67\xc1" + + "\x4a\x3f\x7d\xbc\xde\x7d\x1b\x2c\xa2\x97\x19\x88\xe3\x00\xbb\xf6" + + "\x59\xdb\x55\x03\x1d\x76\x4d\xe3\x68\xaa\x6f\x0f\x8b\x10\xda\xee" + + "\xcb\xa5\xdd\x16\xd1\x8b\x07\x39\x00\x91\xe1\xa1\x96\x24\x02\xbd" + + "\x4b\x57\x04\x1e\xaf\xb7\x80\x3e\x0b\x2d\xa2\x97\x2d\x84\x02\x24" + + "\xd5\xab\x69\x49\x22\xd0\x5d\x77\x45\x25\xdb\x5f\x60\xd3\x31\x23" + + "\x32\xb2\xbd\x45\xf4\xb2\x03\x25\x03\xa0\x7c\x54\xb8\x25\x8a\x00" + + "\xc7\xcc\x9f\xfc\xb3\xa9\xd8\x6c\x6d\x96\x65\x64\x2c\xb0\x88\x5e" + + "\x86\xb0\xc7\x12\x41\xe0\x43\x4a\xc9\x8c\xef\x97\xe5\x7f\xd3\xf0" + + "\x78\xe6\x66\x5b\xae\x7b\x19\xd3\x01\x80\x4a\x15\xca\x5b\x92\x08" + + "\x64\xb7\x5d\x55\x0a\xcd\xa1\xcb\x23\x97\x53\xdb\x2d\xa2\x17\x0b" + + "\x7c\x09\xfc\xad\x2c\x33\x65\x20\x3e\xcf\x2d\x58\xe3\xae\x6e\xb7" + + "\x88\x5e\xd6\x68\x2e\x94\x63\x60\xad\x75\x2f\x0b\x8e\x9b\xf0\x55" + + "\xca\x96\xaa\x1a\xfe\x8d\x45\xf4\xb2\x17\xbb\xe5\x01\xa4\x67\x66" + + "\x5b\xc2\x08\x60\x78\xbc\x7a\xc1\x1a\x77\x61\x9a\xae\xed\x35\x6a" + + "\xbc\x69\x11\xbd\x8c\x39\x75\x87\x7c\xae\xbb\x61\x89\x22\x70\x3b" + + "\x73\x3e\x9a\x3a\xdf\xff\x5d\xd3\xe2\xff\x48\x4e\x7e\xda\x22\x7a" + + "\x19\x53\x83\x4a\x00\x79\x1e\xaf\x25\x8a\x00\x85\xa2\x08\xa6\x7d" + + "\xb7\x28\xff\x73\xe8\xef\x5e\xef\xa6\x63\x96\xeb\x5e\xe6\x60\xda" + + "\x7c\xbd\xbe\x25\x89\x80\xf5\xd9\x14\x15\x97\xdb\x97\x4c\xc6\x34" + + "\x8d\xff\xbb\xec\x3a\x2a\xeb\x15\x16\x1f\xc2\x42\x82\x2c\x21\x04" + + "\x28\x32\xb3\x73\x70\xe7\x79\x00\xa4\xa2\x38\x0d\x8b\xe8\x65\x10" + + "\x42\xd8\x36\x01\x6c\xdb\xb5\xcf\x12\x46\x80\x62\xef\x41\xff\x1e" + + "\x74\x21\xa5\xfc\x31\x28\xa8\xbd\x45\xf4\x32\x17\xa1\x4b\xaf\x0e" + + "\xe0\x74\x38\x2c\x61\x04\xe4\xfb\x95\xfc\xb8\x74\x0d\x80\x14\xc2" + + "\x81\x94\x19\xbb\x5c\xae\x05\x16\xd1\xcb\x12\x9c\xce\x26\x08\xa1" + + "\xd4\x05\xc8\xc9\x75\x59\x02\x09\x48\x8f\x4d\xf0\xd5\xec\x05\x00" + + "\x42\x55\xa3\xae\xbc\x1c\x9f\xc1\x22\xfa\x05\xc2\xed\x5e\x87\x94" + + "\x66\x1c\xc0\xf1\xac\x1c\x4b\x20\x01\x0a\xaf\xee\x5b\xf5\x68\x9a" + + "\xca\x2e\x8b\xe8\x65\x16\x46\x18\x80\x69\x0d\xbb\x07\xe6\xdb\x35" + + "\x0d\x4c\xd3\x47\xf4\xb0\xb0\x17\xb3\x2d\xa2\x97\x5d\x54\x01\xc8" + + "\xca\xce\xb5\x24\x11\x80\x18\x37\x69\xa6\x3f\x5c\x3f\x7e\xfc\x31" + + "\x1d\x5f\xa2\x91\x74\xb0\xfd\xa5\xaa\x71\x8f\x17\xfc\x31\x2c\xac" + + "\x93\x45\xf4\x00\x8f\xe2\x2a\xfa\xdc\x3b\xc3\xaa\xa8\x1a\x40\x90" + + "\x52\x62\x9a\x92\x2f\x66\xfe\x02\xc0\xb1\x63\xc7\x84\xcb\xe5\xe2" + + "\xd8\xb1\x63\xc1\x9b\x36\x6d\x8a\x7c\xfe\xf9\xa7\xea\x54\xae\x6c" + + "\x7b\x1f\x90\xa0\x1e\xc8\xc9\xd9\xdc\xcc\x22\x7a\x60\x3b\x77\x1a" + + "\xc0\xd1\xd4\x74\x14\x45\x58\xe2\x08\x10\x28\x42\x30\xe6\xe3\x6f" + + "\x01\xa8\x53\xa7\x0e\xe5\xca\x95\xc3\xe9\x74\x52\xae\x5c\x39\x1a" + + "\x34\x68\xc0\xab\xaf\xbe\xc6\x9e\x3d\x7b\x59\xb8\x70\x21\x51\x51" + + "\xe1\x15\x4d\x73\xcf\x6a\x21\x22\x3e\xb2\x88\x1e\x80\x08\x09\x79" + + "\xc1\xbf\x4a\x26\x33\x3b\x17\x45\x58\x22\x0d\x14\x6b\xee\xd1\x0d" + + "\xbe\x9a\xed\x4b\x0b\xb7\x68\xd1\xa2\xd3\x1e\xdb\xae\x5d\x3b\xd2" + + "\xd2\xd2\x44\xd7\xae\xb7\x49\x29\x8f\x3f\x2c\x44\xd4\xbd\x10\x61" + + "\x11\x3d\x90\x90\x93\x33\xca\x05\x3e\x2b\xde\xb4\x51\x2d\xff\xa0" + + "\x8d\x85\xcb\x3c\x18\x13\x82\x8d\xdb\xfe\x49\xf2\x1a\x1d\x1d\x7d" + + "\xd6\x8e\x61\xe6\xcc\x99\xa2\x53\xa7\x4e\x52\xca\xf4\xcf\x21\xf4" + + "\x0a\x80\x7a\xf5\xf6\x5b\x44\x0f\x34\xdc\xd9\xa9\x2d\x56\x84\x1e" + + "\x18\x30\x4d\x49\xdf\xe7\xdf\x06\x60\xf0\xe0\xc1\xd8\x6c\xb6\xb3" + + "\x76\x0c\x52\x4a\xe6\xce\x9d\x2b\x7a\xf7\xee\x2d\xe1\xc0\xdf\x76" + + "\x7b\xdd\x26\x5b\xb7\x56\x29\x1d\x1d\x97\xf5\x4a\x2f\x0c\xb1\xb1" + + "\x52\x39\x7a\x54\x18\x03\x1e\xe8\xca\x43\x3d\x6f\xb1\xa6\xd8\x02" + + "\x04\x1b\xff\xda\xcd\xbd\x83\x47\xfb\xad\xb5\x94\x12\x21\xc4\x59" + + "\xad\xba\x10\x02\xb7\xdb\x4d\x50\x50\x10\xe0\xdc\x07\xee\x6a\x96" + + "\x45\x0f\x00\x1c\x3d\x2a\x4c\x10\x39\x6e\x8f\xd7\xb2\xe6\x01\x12" + + "\x9b\x0b\x21\x18\xf4\xca\x87\x00\xf4\xe9\xd3\xc7\x6f\xb1\x8b\xe2" + + "\xee\x3f\xf5\xd4\x53\x48\x29\xb9\xe7\x9e\x7b\x24\xb8\xab\xda\x6c" + + "\x8d\x15\x8b\xe8\x81\x83\x90\x3f\xb7\xee\xc2\xa6\xa9\x96\x24\x2e" + + "\x73\x92\xab\xaa\xca\x07\xff\xf7\x1d\xa9\xe9\xbe\xb2\xc8\x93\x26" + + "\x4d\x2a\xf2\x94\x69\x66\x66\x26\x63\xc7\x8e\xe5\xa6\x9b\x6e\xa2" + + "\x79\xf3\xe6\x02\x40\xd7\x77\x97\x8a\x3a\x5d\x9a\xf5\x7a\x8b\x27" + + "\x02\xda\xb6\x6b\x3f\x9a\xaa\xe2\xf1\xed\x6f\xb1\x70\x19\x42\x51" + + "\x04\x3f\xff\xba\x96\x0f\xbf\x9c\xeb\xff\x6d\xf5\xea\xd5\x34\x6f" + + "\xde\xbc\x88\xe7\xfb\xec\xe6\xb2\x65\xcb\x58\xb1\x62\x45\x7e\xe7" + + "\x51\x3a\x92\x91\x58\x16\xbd\x78\x88\xbe\x49\xd3\x54\x2b\x3a\xbf" + + "\xbc\xed\x39\x19\x99\x39\x3c\x35\xf2\x43\xdf\x97\x7c\xb4\x68\xd1" + + "\x82\x15\x2b\x96\xf3\xfb\xef\xbf\xcb\x23\x47\x8e\xe0\x76\xbb\xfd" + + "\xd6\xff\xdf\x08\x0d\x0d\x65\xce\x9c\x39\xf8\x2c\xb9\x9e\x4f\xfe" + + "\x60\xaf\x45\xf4\x80\xb1\x04\xce\x55\x29\xc7\x32\x84\xaa\x58\xe2" + + "\xbc\x5c\x5d\x76\xbb\xcd\xc6\x7f\x9e\x19\x5b\x40\x72\x21\x44\xe4" + + "\x04\x40\xa8\x6a\x8b\x2b\xae\xb9\xe6\xe6\xd5\x2d\x5b\xb6\x14\x71" + + "\x71\x71\x04\x05\x05\xb1\x61\xc3\x86\xd3\xc6\xec\x2d\x5a\xb4\x38" + + "\xe1\xbb\xc3\x11\x67\x58\x44\x0f\x00\xf4\xe9\xe3\x01\xec\xc9\x00" + + "\x07\x8f\x1e\xb3\x04\x72\x19\x42\xd3\x54\x9e\x7b\xe3\x53\x76\xec" + + "\x39\x00\x20\xec\xf6\xaa\xe3\xa5\xcc\x78\x02\xc0\x30\x56\xed\x34" + + "\xcd\xac\x16\x36\x5b\xfd\x56\xf9\x9d\x80\x4c\x4a\x4a\x22\x39\x39" + + "\xf9\x24\xab\x2e\xa5\xc4\xe5\x72\x15\x74\x16\x7a\xb9\x72\x37\x56" + + "\x70\xb9\x36\x9b\x16\xd1\x03\x00\x9f\x7e\x6a\xc7\x34\xd3\x5f\x01" + + "\x58\xb5\xfe\x2f\x6b\xad\xfb\x65\x18\x97\xbf\xfe\xc1\xd7\xcc\xf9" + + "\x79\x39\x00\x36\x5b\xe5\x17\x3c\x9e\x7d\xfd\x63\x62\x5e\x3e\xe1" + + "\x38\xaf\x77\xcb\x4a\x45\xa9\x68\x53\xd5\xc8\xdf\x01\x5a\xb6\x6c" + + "\x79\x92\x55\x17\x42\xd0\xb6\x6d\x5b\x00\x61\xb3\x55\x79\x35\x2d" + + "\xed\xe7\xa3\xe5\xcb\x97\x8e\x8d\x2e\x16\xd1\x8b\xcf\x2e\xe4\x2e" + + "\x5f\xbb\x05\x55\xb5\x44\x7a\xb9\x40\x55\x15\xc6\x4e\xfa\x1f\x5f" + + "\xce\x5a\x90\x4f\xf2\x2a\xaf\x7a\xbd\x07\x5e\x85\x1a\xa4\xa4\xbc" + + "\x74\x0a\xcb\x5f\xdd\x30\x8c\x8c\x56\x42\x38\x57\x1c\x39\x72\x84" + + "\x97\x5e\x7a\xe9\x84\x78\x7d\xc5\x8a\x15\xec\xdd\xbb\x57\x82\x3d" + + "\xc3\xeb\xdd\xff\x12\xc0\xb1\x63\xf3\x2c\x78\x4e\x71\x41\x00\x00" + + "\x0a\xa5\x49\x44\x41\x54\xa2\x07\x12\x84\x50\x56\x6d\xdb\xb5\x4f" + + "\xda\x34\x6b\x22\xe3\x72\x80\xcd\xa6\x31\x6c\xec\x14\x3e\xfb\xf6" + + "\x47\x40\x20\x44\x85\x36\x5e\xef\xfe\x17\xaa\x55\x7b\x1a\x48\x3e" + + "\xe5\x39\x1e\xcf\x4a\x14\x25\x1e\x29\xdd\xad\x35\xad\xfc\xac\x11" + + "\x23\x46\x70\xfb\xed\xb7\xb3\x70\xe1\x42\x9a\x35\x6b\x26\x5b\xb7" + + "\x6e\x0d\x38\x76\x85\x85\x5d\x5d\xd1\xe9\xbc\xb6\x74\xe9\xa7\xf5" + + "\xca\x8b\x8b\xe8\xa1\x0f\x4a\x99\xfd\xe9\x86\xef\x27\x62\x9a\x66" + + "\x91\x16\x58\x58\xb8\x34\x08\x72\xda\xb9\x7b\xe0\x6b\xfc\xb1\xf1" + + "\x6f\x09\x8a\xb0\xd9\xaa\xb6\xf3\x7a\xf7\x2c\x2e\xea\xf9\x15\x2a" + + "\xdc\xcd\x91\x23\x5f\xa1\x69\x71\x8f\xeb\x7a\xea\xfb\xa0\x23\x44" + + "\xe8\xf7\x52\xea\xcf\x81\xfb\xcf\xa0\xa0\x06\xb8\x5c\x9b\x2d\xa2" + + "\x07\x22\xec\xf6\xf8\x06\x1e\xcf\xee\x4d\x75\x13\xaa\x32\xeb\xa3" + + "\xe1\xb8\xf3\xac\x62\x0e\xa5\x11\x4e\x87\x8d\xde\x4f\x8e\x66\xcd" + + "\xc6\xed\x12\x14\x21\x44\x74\x0b\x29\x8f\xae\x3e\xd7\xeb\xd4\xa8" + + "\x71\x9c\xe4\xe4\x13\x77\xa8\xc5\xc6\xd6\xe6\xe8\xd1\xd2\x59\x7b" + + "\xd1\x22\x7a\xf1\x3a\x84\x5b\xc0\x5b\x6f\xd7\xd2\xcf\xc9\xb4\xb2" + + "\xcd\x94\x2a\x48\x29\x09\x72\xda\xb9\x6b\xe0\x6b\xac\xdb\xb4\x43" + + "\x82\x9a\x17\x14\x94\x78\x85\xcb\xb5\xee\x40\x99\x18\x8f\xb0\x54" + + "\xa0\x38\xdd\x77\x47\x30\xe8\x37\xe6\xb8\xdc\x34\x4f\xac\x63\xb9" + + "\xef\xa5\x49\xd1\x55\x95\x6e\x8f\x8e\x60\xd3\xb6\x64\x84\x70\x6e" + + "\xa8\x52\x65\x7c\x42\x5a\xda\xc7\x19\x65\xe5\xf9\xad\xc1\xb8\xe2" + + "\x72\x09\x9d\x09\x48\x69\x6c\x00\x98\xf3\xf3\x0a\x82\x9d\x56\x8e" + + "\xf7\xd2\x60\xc5\x15\x21\x38\x78\xe4\x18\x57\xdf\xf1\xa4\xf4\x15" + + "\xd8\x70\xcc\x94\xd2\xdd\xe4\xc8\x91\x91\x9e\x32\xd5\xd1\x59\xea" + + "\x50\x3c\xd0\x75\xa2\x84\x30\x66\x82\x5e\x2e\xc7\xe5\x16\xa6\x34" + + "\xb9\xb2\x61\x2d\xcb\xaa\x5f\x52\x2b\xae\xf0\xed\xf7\xbf\xd2\x77" + + "\xc8\xdb\xd2\xe3\xf5\x0a\x87\xa3\xda\xab\x86\x91\xd6\x57\xd3\xae" + + "\x40\xd7\x93\xcb\x96\xb7\x69\xa9\x43\x71\xb9\xed\xf6\xcd\x52\x7a" + + "\xea\x55\xa9\x52\x45\x1c\x3a\x74\x88\xa8\x88\x50\x96\x7d\x33\xd6" + + "\xaa\xb0\x7a\x09\xac\xb8\xaa\x2a\xec\xdc\x73\x88\x5e\xfd\x5f\x95" + + "\x79\x1e\x8f\x00\x45\x57\xd5\x2b\x3a\x18\xc6\xf6\x45\xd1\xd1\x23" + + "\x48\x4d\x1d\x56\xe6\xe4\x62\xb9\xee\xc5\x62\x39\xe2\x6e\x94\xd2" + + "\x53\x7f\xc0\x80\x01\x62\xdf\xbe\x7d\x4c\x99\x32\x85\xd4\xb4\xe3" + + "\xfc\xb6\x66\xb3\xb5\x52\xee\xa2\x13\x1d\xbe\x98\xb9\x80\xdb\xfb" + + "\x0e\x97\x79\x1e\x8f\x50\x94\xa0\xdf\xc1\xb4\xd9\x6c\x21\x8b\x80" + + "\x32\x49\x72\xcb\xa2\x17\x9b\x35\xd7\xf6\x37\x69\x92\x58\xe9\x8f" + + "\x3f\xfe\xf0\xcb\xb3\x79\xf3\xe6\xec\xd9\xbd\x83\x05\x5f\xbe\x61" + + "\xe5\x91\xbb\x18\xb1\xb8\xa2\xb0\xff\x50\x0a\x5d\xfb\x0e\xc7\xeb" + + "\xd5\x01\x21\x55\xb5\x4a\x0f\xc3\xd8\x37\xc3\x92\x90\x45\xf4\x62" + + "\xd3\xb5\xd4\xd4\x54\xca\x95\x2b\xe7\xcf\x1d\xe6\x72\xb9\x08\x09" + + "\x09\x61\xec\xd0\x47\xb9\xbe\x75\x52\x91\x52\x11\x59\x38\x77\x82" + + "\x17\xc8\xfb\xd9\xd1\x93\xf8\x69\xe9\x1a\x09\xa0\x28\xc1\xdb\x4c" + + "\x33\xb7\x9e\xa2\x54\xc3\x34\xf7\x5a\x82\xb2\x88\x5e\x6c\xc8\x4b" + + "\x49\x49\xb1\x97\x2f\x5f\xfe\x04\x32\xdb\xed\x76\x4c\xd3\x60\xed" + + "\xbc\x0f\x90\xa6\xe5\xc2\x97\x44\x2c\x3e\x74\xcc\x67\xcc\xf9\x65" + + "\x05\x80\x14\x42\x15\x52\x46\xb4\x83\xb4\xc5\x96\x84\xac\x18\xbd" + + "\xf8\x85\xa8\x84\xfc\x18\x13\x13\xc3\x3b\xef\xbc\x43\x72\x72\x32" + + "\x00\xcf\x3c\xf3\x0c\x5e\xaf\x17\xc3\x30\x69\xde\xe5\x09\x2b\x56" + + "\x2f\x46\x82\x03\xfc\xfc\xeb\x5a\xae\xec\xf4\x78\x01\xc9\x51\x94" + + "\xf0\x85\x52\x36\x51\x7c\x24\xaf\x61\x09\xca\xb2\xe8\xc5\x0b\x4d" + + "\xab\x89\xae\xef\x42\xd3\x62\x5f\xd5\xf5\xa3\xcf\x9f\x4e\x3f\xc7" + + "\xbc\xd0\x57\xdc\x70\x75\x13\x4b\x60\x17\x08\xd3\x94\x74\x7d\x64" + + "\x18\x7b\x0f\xa6\x00\x48\xd0\xb2\x6c\xb6\x4a\x57\x7a\xbd\x7b\x77" + + "\xda\x6c\xe5\xf1\x7a\xad\x9c\x00\x16\xd1\x4b\x08\x4e\x67\x2b\xdc" + + "\xee\x15\x68\x5a\xd5\x70\x5d\x3f\x1a\xa7\x69\x91\x59\xa1\xa1\x6f" + + "\x1d\x76\xb9\xfe\x4f\x95\x72\xaf\x4d\xd7\x93\xd7\x99\x66\x6e\x9d" + + "\x99\x13\x87\x13\x5f\x35\xce\x12\xd8\x39\x5a\x70\x45\x11\x38\x1d" + + "\x0e\x5e\x79\xf7\x4b\xa6\x7e\xb7\x10\xc3\x30\x01\x05\x4d\x8b\xfe" + + "\xff\xf6\xce\x36\x28\xaa\xeb\x8c\xe3\xff\x73\xee\xcb\x5e\x70\x5f" + + "\x0c\x20\x0c\x08\x68\x42\x08\x45\xab\xc6\x3a\xd4\x69\x4a\x1b\x69" + + "\x52\xda\x66\xc6\xb6\x93\x92\xa4\x93\xb6\x32\xed\x87\xaa\xfd\x52" + + "\x26\x9d\x44\x74\x1c\x07\x94\xaa\x33\x4e\x1a\x13\x32\x99\x8e\x09" + + "\x1d\x51\x74\xb0\x11\xa3\x4e\x8d\x89\x14\x0d\x49\xad\x1a\x6b\x29" + + "\x83\x75\xc6\x26\xbe\x2c\x48\x95\x42\x79\x87\xbd\xbb\x77\xef\x7d" + + "\xfa\x81\xdd\x0d\x49\x51\x61\x9a\xa4\xab\x3c\xbf\x99\xfd\xc4\xee" + + "\xb2\xf3\x9c\xf3\xbb\xcf\x39\xe7\x9e\xf3\xdc\x92\x82\x82\xae\x86" + + "\x53\xa7\xb8\x1b\xb3\xe8\x71\x80\xa2\x2c\xd6\x6d\xbb\x2d\x08\x84" + + "\xe9\xcc\xc1\x6a\xa1\x6b\x2a\x2f\xcc\x4d\xf6\x22\xea\xd2\xf1\xec" + + "\xe6\x1d\x38\x72\xfc\x4c\xa4\x66\xbe\x84\x94\x33\x7e\xe5\x76\x97" + + "\x6d\x1f\x1c\xdc\xe4\x78\x3c\x4b\x31\x34\x74\x86\x03\xc5\xa2\xc7" + + "\x07\x29\x29\x8f\xa7\xf4\xf4\x1c\xe8\x9e\x33\x3b\x8d\x0e\xbd\x5a" + + "\x29\x78\x15\xfe\xe6\x19\xdc\xa5\x6b\x18\x1c\x0e\xe0\xf9\xd7\x5e" + + "\xc7\xa1\x63\x7f\x8e\x54\xd6\x15\x90\xd2\x57\x91\x9e\xfe\x46\x55" + + "\x67\x67\x91\xad\xaa\x59\x08\x87\x3b\x38\x60\x2c\x7a\x9c\x05\x5a" + + "\x24\x43\xd3\x7c\x45\xa1\xd0\xe5\xe3\x0f\x2d\x99\x4f\x3b\xb6\x94" + + "\x09\xcb\x0a\xb3\xec\x11\xa4\x14\x30\x74\x1d\x35\xaf\xbf\x85\x63" + + "\xef\x9e\x43\xcb\xdf\x3f\x8c\x75\x51\x21\x12\x36\xac\x58\x31\x52" + + "\x55\x5b\x2b\x78\x45\x93\x45\xbf\x53\x3a\xb4\x6f\x8d\xe3\x0c\x6c" + + "\x2d\xfa\xca\x83\x54\x5d\xf1\x0b\x61\x85\xed\x69\x2b\x3b\x11\x41" + + "\x91\x12\xc3\x01\x13\xaf\xec\x3e\x8c\xba\x37\x9a\x80\x48\x15\xd6" + + "\x88\xe0\x75\x86\x91\xb3\x36\x10\x68\xbb\xc6\x3d\x87\x45\xbf\xf3" + + "\xe6\x9d\x46\x5e\xb1\x69\x5e\x7c\xfb\x07\xdf\x29\xa4\xca\xb2\x52" + + "\x61\x85\xa7\x4f\x66\x8f\x9e\x28\xd3\x75\x0d\x97\xdb\xaf\xe3\xc8" + + "\x89\xf7\xf1\xca\xee\xc3\xe3\x04\xd7\xcf\x08\xa1\x35\x10\x8d\x6c" + + "\x1b\xfb\xc4\x5c\xdc\xac\xb4\x13\xc3\xa2\xc7\x3d\x8a\x92\xba\xc6" + + "\xb6\xff\xb5\x75\x7e\xee\x1c\xaa\x28\x5b\x21\xe6\xe5\x66\x23\x74" + + "\x97\x0f\xe5\x35\x55\xc1\x8d\x9e\x7e\x9c\x3c\x7b\x1e\x3b\xea\xdf" + + "\xc4\xb5\xeb\xdd\xe3\x04\x97\x03\xaa\x9a\x56\x1a\x0e\x5f\x3f\xc4" + + "\xbd\x83\x45\xbf\xab\x50\xd5\xac\x1f\x85\xc3\x1d\x75\x00\xe8\xa9" + + "\xe5\xcb\xc4\xda\xd5\x3f\xbc\xeb\xb6\xca\x2a\x52\x62\xd4\x0c\xe2" + + "\xc2\x07\x7e\xbc\xb4\xf3\x60\x74\xee\x4d\x1f\xf5\x3d\xd1\x21\x84" + + "\x77\x2f\xd1\x40\x39\xf7\x08\x16\xfd\xee\xcd\x72\x5a\xf6\x63\x96" + + "\xd5\x7e\x04\x00\xa5\xa7\x26\x89\xba\xed\xe5\x98\x95\xe4\x83\x6d" + + "\xff\x77\x81\xc9\xe8\x45\x40\x0a\x01\xcb\xb6\xe1\xd8\x0e\x74\x4d" + + "\x8d\xab\xa7\xb8\x46\x7f\xa3\x10\xc0\x95\x8e\x2e\xbc\x5c\x7b\x10" + + "\x7f\x3c\xd9\xf2\xc9\x6e\xd7\x27\xc4\x8c\xa6\xec\xec\xbf\x3c\xed" + + "\xf7\x7f\x81\xcf\xf1\xb2\xe8\xd3\x25\xb3\xdf\x77\x6f\x38\xec\xbf" + + "\x00\xd8\x06\x00\x14\x7f\x6d\x09\xb6\x3c\xf7\x33\x28\x8a\x12\xdb" + + "\xf2\x29\x84\x80\x19\x0c\xe1\xc2\x87\xed\xd8\xd5\xd0\x88\x77\x4e" + + "\xb7\x46\x86\xc3\x2a\xde\xdc\xf9\x6b\xcc\x4a\xf2\xfd\xdf\x47\x03" + + "\xb6\x6d\x23\x60\x86\xd0\x74\xb2\x05\x95\x2f\xd5\x4d\x70\x6a\x4f" + + "\xf5\x03\x99\x5f\x05\xae\x76\x6a\xda\x3c\x58\xd6\x05\x6e\x7c\x16" + + "\x7d\xfa\x60\x18\x5f\xc2\xf2\xe5\xe7\x94\xfd\xfb\x13\x87\x89\x02" + + "\x06\x00\x12\x02\xe2\xb7\x55\xbf\xc4\xd2\xc5\xf9\x20\x22\x1c\x3a" + + "\x76\x0a\x15\x2f\xee\x8a\x64\xcb\xb1\x97\xe3\x38\xb1\x0b\xc1\xb3" + + "\x3f\x7f\x02\x4f\x7f\xef\x1b\xb1\xd3\x5c\x9f\x97\xf0\x8e\x43\x08" + + "\xdb\x36\xb6\xd7\x34\x60\xef\xe1\x13\x13\xbe\x05\xd0\xbb\xa5\x9c" + + "\xfd\xb0\xe3\x5c\xb9\xc8\xad\xcd\xa2\x33\x00\x34\x2d\x73\xbd\x65" + + "\x5d\xdb\x14\xcd\xe2\x8d\xbb\xb7\xa2\xea\xe5\xbd\x78\xe7\x74\x2b" + + "\xf6\xed\xdb\x87\x92\x92\x92\x8f\x1a\x4f\x08\x34\x37\x37\xa3\xa8" + + "\xa8\x08\xc0\x58\xbd\xf2\x4d\xcf\xfc\x14\x8f\x16\x2e\xfe\x4c\x64" + + "\x1f\xff\x9d\x9d\x37\x7a\xf0\xbb\xdf\xbf\x85\x03\x6f\xff\x09\x13" + + "\x9f\xd7\xd1\x2f\x4a\x39\x73\xed\xfd\xf7\xff\xf8\x0f\x57\xaf\xb6" + + "\x5a\xa1\x50\x13\x16\x2d\x6a\x43\x6b\xeb\x02\x6e\x64\x16\x7d\xba" + + "\x67\xf6\x2f\xc2\x34\xcf\x43\x51\x52\x5d\x9a\x96\xf8\x75\xd3\xec" + + "\x3c\x06\x58\x40\x64\xf1\x2a\x18\x0c\x42\xd3\xb4\x98\x6c\x51\xf1" + + "\x4c\xd3\xc4\xaa\x55\xab\x50\x5b\x5b\x0b\x00\xf0\xb8\x13\x71\xfa" + + "\xc0\x8b\xb0\xc2\xf6\xa7\x22\xb7\x14\x02\x9a\xae\xe1\xaf\xe7\x3f" + + "\xc0\x73\x9b\x5f\x45\x57\x4f\x1f\xec\x09\x8b\x69\xb8\xb7\x09\x21" + + "\x37\x4b\xa9\x9a\xb6\xdd\x6b\x72\x8b\xb2\xe8\xcc\x2d\xf0\x78\x1e" + + "\xc4\xd0\xd0\xdf\xa2\xfa\xbf\x00\x98\x65\x95\x95\x95\xd8\xb0\xe1" + + "\xd6\x65\x90\x46\x46\x46\xb0\x72\xe5\x4a\xec\xd9\xb3\x07\x52\x4a" + + "\x6c\xdf\xb0\x1a\x0f\x2f\x5d\x38\xe9\x27\xc7\xc4\x16\xfb\xa4\x84" + + "\x4b\xd7\xf0\xcf\xae\x7f\x63\xff\xd1\x77\xd1\x70\xf4\x3d\xf4\x0d" + + "\x0c\x8f\xaf\x7f\x17\xdb\xd4\xa2\xaa\x69\x4f\x86\xc3\xbd\x4d\x2e" + + "\x57\x56\x20\x18\xbc\x14\xe0\xd6\x63\xd1\x99\x29\xe2\x72\x7d\x19" + + "\xc1\x60\x4b\x2f\x60\xdd\x33\x30\x30\x00\x8f\xc7\x73\x4b\x61\xa3" + + "\xa2\x36\x36\x36\xa2\xb8\xb8\x18\x00\xb0\x66\xf5\x53\x28\x7d\xfc" + + "\x9b\xb1\x79\xbd\x19\x0c\x61\xd4\x0c\xe2\xe8\x89\xb3\x48\x30\x74" + + "\xd8\xb6\x8d\xf3\xff\xb8\x8a\xb4\x59\x49\x38\xdb\x7a\x11\x7d\x03" + + "\x43\x18\x19\x35\x71\xa3\xbb\x17\x56\xd8\x1e\x77\x1b\xcc\xd5\x0e" + + "\xc8\x5d\x80\xad\x28\x8a\xe7\x12\x51\xe8\xf8\xdc\xb9\xdb\xfc\x97" + + "\x2f\xaf\xe2\x3a\x59\x0c\xf3\xbf\xe0\xf5\x3e\x9a\x05\xc0\x29\x2c" + + "\x2c\xa4\xc9\xe2\x38\x0e\x39\x8e\x43\x8a\xa2\x10\x00\xd2\x34\x95" + + "\xee\xcb\x4e\xa7\xf4\xd4\x24\x9a\x95\x3c\x93\x22\xd9\x78\x12\x2f" + + "\xd5\x2f\xa5\xfb\x79\x20\x71\xa9\xa6\x3d\xe0\xfb\xe4\x6f\x4b\x48" + + "\x78\x88\x1b\x88\x33\x3a\xf3\xe9\x90\x70\x04\x08\x3c\xd6\xdf\xdf" + + "\x0f\x9f\xcf\x37\xe9\x39\xb5\x10\x02\xaa\xaa\xc2\xb6\x6d\x02\x84" + + "\x00\xb4\xe3\x80\x33\x1f\x40\x1a\x20\xdb\x00\x9c\x02\x30\x04\x60" + + "\x09\x40\x06\xa0\x9c\x03\x9c\x4b\x42\x28\x3d\x00\x06\x14\x25\xa5" + + "\x2d\x1c\xee\xb8\xc2\xf1\x67\x98\xcf\xf2\xea\x2b\xd2\xe1\x76\x2f" + + "\x50\x00\x41\x25\x25\x25\x0e\x4d\x91\x8d\x1b\x37\xc6\x32\xb3\xa2" + + "\xcc\xc9\x9b\xca\xff\x5e\xb8\x70\x27\x37\x00\xc3\x7c\x5e\x48\x79" + + "\xcf\x0b\x00\x9c\xe8\x70\x7c\xb2\xf4\xf6\xf6\x12\x00\x07\x00\x49" + + "\xe9\xdd\xc5\x91\x64\x98\x38\x45\xd7\xf3\xb3\x00\x50\x45\x45\xc5" + + "\x94\x32\xf9\x78\xc9\x01\xa3\x05\x18\xdb\x8c\xc3\x30\x4c\x1c\x91" + + "\x9f\xdf\x11\x19\xba\x1b\xdd\xa9\xa9\xa9\x93\xce\xe6\xd1\xf7\xb8" + + "\xdd\x6e\x07\x10\x04\x18\x5b\xc6\x46\x05\x19\x1c\x54\x86\x89\x47" + + "\x14\x25\xe3\x11\x00\xd4\xdf\xdf\x3f\xa5\x21\xfb\xfa\xf5\xeb\x1d" + + "\x00\xa4\xeb\xf3\xbe\x3b\x63\xc6\x42\x0e\x24\xc3\xc4\x23\x1e\xcf" + + "\xb7\xe0\xf5\x6e\x13\x80\x4a\xe9\xe9\xe9\x53\x1a\xb2\x77\x77\x77" + + "\x13\x00\x12\x22\xf1\x3d\x8e\x24\xc3\xc4\x39\x52\x7a\x7f\x02\xc0" + + "\xa9\xae\xae\xfe\x58\x36\xbf\x5d\x66\x6f\x6e\x6e\x26\x00\x64\x18" + + "\x79\x3c\x21\x67\x98\xf8\x17\xdd\x30\x53\x52\x52\x62\x02\x97\x95" + + "\x95\x91\xdf\xef\xbf\xed\xdc\x3c\x39\x39\xd9\x01\x04\x3f\xb9\x80" + + "\x61\xe2\x5f\xf2\xa4\x05\x00\xa8\xa0\xa0\x80\x88\x88\x06\x07\x07" + + "\x63\xf7\xc2\x6f\x95\xd1\xeb\xeb\xeb\xa3\xc3\xf6\xdf\x70\x14\x19" + + "\x26\xee\x99\x79\xef\x78\xd1\x2d\xcb\x8a\x89\x3e\x3a\x3a\x7a\x53" + + "\xd9\xbd\x5e\x2f\x8d\xad\xb4\x33\x0c\x13\xf7\xcc\x9e\xbd\x57\x03" + + "\x40\x19\x19\x19\x31\x89\x73\x73\x73\x09\x00\x95\x96\x96\x4e\x28" + + "\x79\x6d\x6d\x6d\xe4\xbe\xb9\xf1\x3e\x47\x90\x61\xee\x98\xe1\xfb" + + "\xcc\x9d\x00\x9c\xfa\xfa\x7a\x72\x1c\x27\xba\x01\xe6\x63\x59\x3d" + + "\x9a\xd9\x6b\x6a\x6a\x62\x7f\x53\xd5\xfc\x45\x86\x31\x8f\x03\xc8" + + "\x30\xf1\x8e\xa6\x3d\x00\x4d\x5b\xec\x03\x5c\x7d\x8a\x22\xa9\xbc" + + "\xbc\x9c\x86\x87\x87\x29\x27\x27\x87\x00\xd0\xb2\x65\xcb\xa8\xa1" + + "\xa1\x81\xd6\xad\x5b\x47\x79\x79\x79\x51\xc9\x1d\x21\xb4\x61\x8e" + + "\x1e\xc3\xdc\x81\x08\xe1\x79\x06\x50\x26\x71\x94\x54\x92\x94\x99" + + "\x73\x39\x62\xcc\x6d\xfb\x14\x87\x20\xbe\x30\x8c\x02\x98\xe6\x59" + + "\x00\x80\xa2\x24\x7d\xdb\x71\x02\x39\x80\xe8\x02\x44\x3b\x40\xdf" + + "\x27\x0a\xe5\x02\xc8\x91\xd2\x75\x30\x33\xf3\x4a\x55\x7b\x7b\x2a" + + "\x17\x7e\x60\x6e\xcb\x7f\x00\x4a\x3f\xff\x3a\x92\xd3\x63\x31\x00" + + "\x00\x00\x00\x49\x45\x4e\x44\xae\x42\x60\x82" + +// gophercolor_png returns the raw file data data. +// +// WARNING: The returned byte slice is READ-ONLY. +// Attempting to alter the slice contents will yield a runtime panic. +func gophercolor_png() []byte { + var empty [0]byte + sx := (*reflect.StringHeader)(unsafe.Pointer(&_gophercolor_png)) + b := empty[:] + bx := (*reflect.SliceHeader)(unsafe.Pointer(&b)) + bx.Data = sx.Data + bx.Len = len(_gophercolor_png) + bx.Cap = bx.Len + return b +} diff --git a/third_party/github.com/jteeuwen/go-bindata/testdata/toc.go b/third_party/github.com/jteeuwen/go-bindata/testdata/toc.go new file mode 100644 index 00000000000..0efb33f5c4c --- /dev/null +++ b/third_party/github.com/jteeuwen/go-bindata/testdata/toc.go @@ -0,0 +1,1853 @@ +package main + +import ( + "bytes" + "compress/gzip" + "io" +) + +// gophercolor_png returns raw, uncompressed file data. +func gophercolor_png() []byte { + gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x00, 0xff, 0x34, 0x9b, + 0x65, 0x50, 0x1b, 0x5f, 0x17, 0xc6, 0x43, 0x69, 0x8b, 0xbb, 0xbb, 0xbb, + 0x16, 0x2d, 0x5e, 0xb4, 0xb8, 0xbb, 0x16, 0x77, 0x77, 0xa7, 0xb8, 0x14, + 0x77, 0x28, 0xee, 0xee, 0x4e, 0x70, 0x77, 0x08, 0x52, 0xdc, 0x09, 0xee, + 0xae, 0x6f, 0xfe, 0x1f, 0xde, 0xc9, 0x6c, 0x32, 0xb3, 0xb3, 0xb9, 0xbb, + 0xf7, 0xec, 0x39, 0xe7, 0x79, 0x7e, 0xd9, 0x9b, 0x08, 0x45, 0xf9, 0x9f, + 0x48, 0xf0, 0xf8, 0xf0, 0x00, 0x00, 0x00, 0x49, 0x4a, 0x52, 0x4c, 0x19, + 0xf2, 0xf9, 0xfc, 0xdf, 0x06, 0xfb, 0x15, 0xf2, 0x1e, 0x7e, 0xaa, 0x2d, + 0x00, 0x00, 0xc0, 0x2b, 0x58, 0x8a, 0x8a, 0x2a, 0x2a, 0x5a, 0xd8, 0xbb, + 0xd8, 0x3b, 0x5b, 0xd8, 0x3b, 0x90, 0x4a, 0x89, 0x8a, 0x92, 0x3a, 0x38, + 0xd9, 0x9b, 0x59, 0xda, 0x98, 0x02, 0x00, 0x1e, 0x2b, 0x59, 0x2a, 0xe6, + 0xaa, 0x2a, 0x47, 0xd8, 0x02, 0x0f, 0x1b, 0xb7, 0x22, 0x32, 0xe1, 0x01, + 0x49, 0x32, 0xf6, 0xca, 0x58, 0xb0, 0xa4, 0xca, 0x22, 0x51, 0x01, 0x98, + 0xf1, 0xd4, 0xf4, 0x64, 0x70, 0xa8, 0xd2, 0xe1, 0x64, 0x79, 0xcb, 0x58, + 0x4a, 0xbd, 0x68, 0xe2, 0xe2, 0x9f, 0x09, 0x46, 0x72, 0xc3, 0xa1, 0x63, + 0x62, 0x02, 0xfe, 0x60, 0x29, 0x31, 0x22, 0x46, 0xc2, 0xff, 0x83, 0xd9, + 0x23, 0xcb, 0x8f, 0x09, 0x2a, 0x08, 0x8f, 0x1c, 0x7b, 0xd9, 0xf6, 0x2e, + 0xb0, 0x5e, 0xec, 0x7c, 0x38, 0x98, 0x7c, 0x6f, 0x5a, 0x12, 0x04, 0x57, + 0xde, 0x64, 0x35, 0x4c, 0xc3, 0xf4, 0xc0, 0x22, 0xa6, 0x48, 0xb2, 0x2b, + 0x71, 0x06, 0x20, 0x96, 0x89, 0x10, 0xa3, 0x6d, 0x05, 0x0d, 0x0f, 0x0c, + 0x6d, 0xef, 0x31, 0xff, 0xf8, 0x0d, 0x4f, 0xe9, 0x00, 0x40, 0x85, 0x6d, + 0x30, 0x21, 0x73, 0x7e, 0xa3, 0x80, 0x02, 0x3c, 0xfa, 0xf1, 0xf3, 0x33, + 0x90, 0xf7, 0xc0, 0x74, 0x01, 0xa0, 0x3c, 0x40, 0x08, 0xb0, 0x80, 0x1e, + 0xb9, 0xcc, 0x1e, 0x56, 0xc2, 0xb0, 0x0f, 0x94, 0x63, 0x91, 0x74, 0x5d, + 0xa8, 0x80, 0x60, 0xa8, 0x1e, 0x97, 0xf8, 0xef, 0x32, 0xb0, 0x01, 0x98, + 0x80, 0x1f, 0x5e, 0x31, 0x22, 0xc5, 0x80, 0x1f, 0x12, 0x50, 0x01, 0x59, + 0x69, 0xd4, 0x2a, 0x80, 0xdc, 0xcf, 0x80, 0x5f, 0xe3, 0xc6, 0x46, 0xbb, + 0x00, 0x45, 0x26, 0xc0, 0x2f, 0x1a, 0xff, 0x03, 0x10, 0x20, 0x20, 0xeb, + 0x31, 0xdd, 0x1b, 0x0a, 0xa0, 0x93, 0x44, 0x86, 0x05, 0x95, 0x1b, 0x0f, + 0x20, 0xc5, 0x30, 0x0d, 0x17, 0x03, 0x58, 0xf0, 0x01, 0x2a, 0xa7, 0xd5, + 0x23, 0xc5, 0x01, 0x9a, 0xac, 0x00, 0x4c, 0x33, 0x99, 0x01, 0x1e, 0xc0, + 0x3f, 0x26, 0x00, 0xab, 0x94, 0x86, 0x99, 0x24, 0xa0, 0xb6, 0x05, 0xd0, + 0x33, 0x85, 0x8a, 0x50, 0x0f, 0x80, 0x45, 0x04, 0xb0, 0x2a, 0x85, 0x87, + 0xd0, 0x01, 0x3e, 0x7b, 0x03, 0x7e, 0x8d, 0x50, 0x50, 0x78, 0x00, 0x82, + 0xd3, 0x01, 0x98, 0x12, 0xd7, 0x1a, 0xfc, 0x57, 0x0c, 0xd5, 0xa8, 0x60, + 0x7a, 0x48, 0xa4, 0xd2, 0xeb, 0xf9, 0xdb, 0x28, 0x79, 0xc4, 0x7f, 0xeb, + 0xc0, 0x32, 0x39, 0xc2, 0x68, 0x68, 0x30, 0x13, 0xd3, 0x4e, 0x49, 0xe1, + 0x30, 0x60, 0x72, 0x19, 0x7e, 0x32, 0xcc, 0xf8, 0xc1, 0xdc, 0xe7, 0x99, + 0x8e, 0xc7, 0xf6, 0x9b, 0x03, 0x65, 0xeb, 0x66, 0x02, 0x00, 0xc8, 0x8d, + 0xc7, 0x82, 0xcc, 0xf6, 0xe6, 0xcd, 0x63, 0x0a, 0xb9, 0x7a, 0x6a, 0x8a, + 0x2b, 0xa6, 0x19, 0xd9, 0x80, 0xe9, 0xb8, 0xfb, 0xeb, 0x07, 0xb9, 0x91, + 0xd1, 0xee, 0xfb, 0xfe, 0x74, 0x85, 0xc3, 0x0f, 0x00, 0x60, 0xdb, 0xc5, + 0x6f, 0xf6, 0x9d, 0x91, 0xa5, 0x01, 0x3f, 0x80, 0xef, 0x6b, 0x80, 0xed, + 0x7b, 0x3e, 0xd5, 0xf9, 0x67, 0x0b, 0x7d, 0x84, 0x5c, 0xd7, 0x87, 0x28, + 0xb3, 0x7a, 0x94, 0x1f, 0x4d, 0x80, 0xdc, 0xa3, 0x15, 0x8d, 0x1b, 0x87, + 0xff, 0x62, 0x23, 0x9e, 0x17, 0xd7, 0xb6, 0xbc, 0xbc, 0xbf, 0xb7, 0xf7, + 0x4f, 0x7a, 0x40, 0x44, 0xc7, 0x70, 0x4c, 0xc3, 0xf7, 0xdd, 0xbc, 0xcf, + 0xa0, 0x47, 0xe3, 0xcd, 0xe6, 0xc9, 0x8f, 0xff, 0xf5, 0xe1, 0x6e, 0xab, + 0x6b, 0x87, 0xf2, 0x37, 0x9b, 0xde, 0xef, 0x9f, 0x9f, 0x1f, 0xb7, 0xfa, + 0x26, 0x6e, 0x65, 0x8b, 0x08, 0xa7, 0x13, 0xe0, 0x82, 0x8d, 0xd6, 0x0e, + 0x62, 0x7f, 0xbe, 0xb6, 0x20, 0x7c, 0xbc, 0x12, 0x81, 0xc8, 0x07, 0xa4, + 0x8c, 0xda, 0x34, 0xe9, 0x31, 0x77, 0x95, 0xd0, 0x1d, 0x63, 0xc4, 0x32, + 0xfe, 0xdc, 0xb0, 0x15, 0x91, 0x47, 0x88, 0x24, 0xd0, 0x0d, 0x52, 0xcd, + 0x7d, 0x98, 0xec, 0x6c, 0x32, 0xbe, 0x40, 0x0b, 0x6e, 0x72, 0x02, 0x6a, + 0xad, 0x04, 0xa1, 0xbc, 0xe3, 0x99, 0x4a, 0xf5, 0x8c, 0xa1, 0xef, 0x65, + 0x68, 0x50, 0x35, 0x5d, 0x7a, 0x76, 0x1e, 0x00, 0x80, 0xeb, 0x0e, 0xfb, + 0xde, 0x79, 0x5a, 0x58, 0xe8, 0x00, 0x8b, 0xa0, 0xed, 0x69, 0xf7, 0x8f, + 0x0b, 0xa1, 0xb7, 0x9f, 0xb9, 0x54, 0x80, 0x00, 0x33, 0xa9, 0x04, 0x47, + 0x00, 0x40, 0x5f, 0x8c, 0x92, 0x59, 0x75, 0xac, 0x41, 0x68, 0x18, 0x16, + 0x00, 0x10, 0xcb, 0xfd, 0x4d, 0x5f, 0x2b, 0x4c, 0x70, 0xdb, 0x8b, 0xcb, + 0xd8, 0xf3, 0x95, 0xb0, 0xf7, 0xcb, 0x5a, 0x2f, 0xc2, 0xeb, 0x2f, 0xae, + 0x60, 0x11, 0xca, 0x81, 0x3e, 0x11, 0x54, 0x11, 0x78, 0x93, 0x00, 0x42, + 0xa7, 0x5f, 0x74, 0x95, 0x81, 0x22, 0xb4, 0xa1, 0x93, 0xb5, 0x44, 0xf4, + 0xbf, 0x58, 0xe6, 0x7f, 0x10, 0x71, 0xf4, 0x28, 0x59, 0x84, 0x26, 0x3a, + 0x20, 0x33, 0xf7, 0xa9, 0xb5, 0x22, 0x0b, 0x38, 0xa0, 0x3c, 0x1b, 0xc2, + 0x66, 0xf7, 0xd2, 0x76, 0xfe, 0x86, 0xfb, 0x2c, 0x3c, 0x02, 0x8b, 0x61, + 0x48, 0xb6, 0x12, 0x0e, 0x65, 0x14, 0xa9, 0x49, 0x11, 0x03, 0x8b, 0x93, + 0x1e, 0xf2, 0x48, 0xd6, 0x2b, 0xf9, 0x19, 0x3d, 0x8a, 0x92, 0x74, 0x34, + 0x1c, 0x53, 0x89, 0x5c, 0x26, 0x9e, 0x53, 0xf2, 0x9b, 0x72, 0xa4, 0x2a, + 0xa9, 0x9a, 0x24, 0xd1, 0xb5, 0x80, 0xd3, 0x27, 0x9e, 0x30, 0x5d, 0x89, + 0xf6, 0x78, 0xbe, 0x11, 0x00, 0x5b, 0xe0, 0x6b, 0x68, 0xe7, 0xcf, 0x6f, + 0x49, 0xbf, 0xeb, 0x95, 0x04, 0xe6, 0x11, 0x9b, 0x44, 0xdb, 0x4a, 0xb9, + 0xf1, 0x83, 0x25, 0xf2, 0x11, 0x66, 0x4c, 0x5c, 0xbe, 0x65, 0xc4, 0x62, + 0xe7, 0x66, 0xce, 0x38, 0x35, 0xe2, 0x0b, 0xfc, 0xe1, 0xca, 0x03, 0xcf, + 0x54, 0x58, 0xa0, 0xac, 0xc4, 0x0a, 0x8a, 0x0e, 0xb3, 0xf6, 0x1c, 0xe2, + 0xc2, 0xb0, 0x0f, 0xd8, 0xb0, 0x32, 0x0f, 0xf5, 0x8b, 0xd4, 0x7d, 0x67, + 0x84, 0x4b, 0x34, 0x1e, 0xaf, 0x23, 0xaf, 0x44, 0xac, 0x18, 0xc2, 0xaf, + 0x55, 0xaf, 0x84, 0xee, 0x88, 0xb8, 0x33, 0x9e, 0xae, 0x73, 0xff, 0x8c, + 0xfe, 0x5b, 0xbc, 0x07, 0x8e, 0xeb, 0xb3, 0xbb, 0x08, 0xa9, 0x21, 0xb1, + 0xb0, 0xa4, 0xa6, 0xac, 0xa6, 0xfc, 0x3f, 0xc9, 0x52, 0x52, 0x42, 0x4a, + 0x0e, 0xb4, 0x15, 0x38, 0x6e, 0x38, 0xe8, 0x60, 0xa5, 0x3e, 0x1a, 0xf2, + 0xc4, 0x52, 0x99, 0x46, 0xea, 0x76, 0xb4, 0xa7, 0x01, 0x5c, 0x23, 0xb6, + 0x6f, 0x61, 0x9a, 0x92, 0x8c, 0x14, 0x8b, 0xe8, 0xb1, 0x18, 0x2c, 0xa8, + 0xde, 0xe1, 0xa2, 0x03, 0xdc, 0x94, 0xe8, 0x11, 0xa2, 0x6c, 0x34, 0x6d, + 0x71, 0x9f, 0xa4, 0xea, 0x0a, 0x55, 0xe7, 0xd0, 0xe7, 0x24, 0xe6, 0xec, + 0x94, 0x29, 0x8e, 0x18, 0xcb, 0x32, 0x39, 0x24, 0xf1, 0x29, 0x12, 0x47, + 0x57, 0x4c, 0xac, 0xeb, 0x61, 0x78, 0x92, 0x18, 0x49, 0x19, 0x46, 0x42, + 0xf6, 0xb2, 0xf6, 0xfa, 0xf7, 0xd8, 0xf7, 0x08, 0xf6, 0xc8, 0xae, 0x75, + 0xe0, 0xb3, 0x8c, 0x7e, 0x38, 0x16, 0x3e, 0xaa, 0xec, 0xd0, 0x2a, 0x8f, + 0x59, 0x49, 0xe3, 0xed, 0xa3, 0xee, 0x73, 0xec, 0x7f, 0x35, 0x4d, 0x63, + 0x13, 0x56, 0x2b, 0xc8, 0x50, 0x5e, 0x2f, 0xcd, 0x53, 0x45, 0xe3, 0x8c, + 0xd5, 0x16, 0xa9, 0xca, 0x6b, 0x56, 0xae, 0x51, 0x0a, 0x2b, 0xc5, 0xe0, + 0x70, 0xcd, 0xe0, 0x99, 0x0c, 0xc2, 0x96, 0x92, 0x29, 0xaa, 0xca, 0x4f, + 0x06, 0xe1, 0x5b, 0xe0, 0x58, 0x3c, 0x58, 0xd6, 0x1c, 0xba, 0xb4, 0xa3, + 0xad, 0x27, 0x13, 0xcb, 0xa7, 0xce, 0x68, 0xcc, 0x8d, 0x1f, 0xfd, 0x3c, + 0x4c, 0x3d, 0x84, 0xbe, 0x75, 0x47, 0x44, 0x0a, 0xc5, 0x0a, 0x1a, 0x0e, + 0x37, 0xa7, 0xc5, 0xcb, 0xc4, 0x85, 0xc1, 0x35, 0xc7, 0x73, 0xc7, 0xad, + 0x49, 0x93, 0x2d, 0xc6, 0x03, 0x45, 0xe1, 0x0d, 0xab, 0xb2, 0x72, 0x7f, + 0x3b, 0x49, 0x03, 0xa7, 0xa3, 0xa4, 0xdb, 0xab, 0x69, 0xd2, 0x37, 0xd3, + 0xfb, 0x60, 0xc5, 0x8f, 0xc1, 0x27, 0x4b, 0x27, 0x53, 0x27, 0x13, 0xd0, + 0xb3, 0xa8, 0x96, 0x95, 0x17, 0x97, 0x6f, 0x94, 0x23, 0xa8, 0xdd, 0xa8, + 0x8d, 0xab, 0xc6, 0x96, 0xe9, 0xab, 0xf8, 0x56, 0x4b, 0xa8, 0xb1, 0xab, + 0xec, 0x96, 0xc1, 0xcd, 0xa7, 0x54, 0xa8, 0x95, 0x67, 0x29, 0x9e, 0xa8, + 0x10, 0xa8, 0x98, 0x97, 0xf1, 0x95, 0x86, 0x95, 0x9b, 0x97, 0xda, 0xab, + 0x0a, 0x15, 0xfa, 0x69, 0xbd, 0x45, 0x7c, 0xd5, 0xea, 0x97, 0xed, 0x57, + 0x10, 0x2d, 0x54, 0xca, 0xad, 0x33, 0xd8, 0xed, 0x1c, 0x22, 0x45, 0x30, + 0xc6, 0x6b, 0xf0, 0x60, 0x24, 0xb3, 0x46, 0xaa, 0x08, 0x75, 0xfd, 0xcd, + 0x39, 0x40, 0x5d, 0x37, 0xb9, 0xec, 0xe3, 0x46, 0xdf, 0x91, 0xf6, 0x46, + 0xd4, 0x11, 0x25, 0x50, 0x5e, 0x96, 0xc7, 0x23, 0xca, 0x2e, 0xcd, 0xae, + 0xd1, 0xa0, 0x7c, 0x93, 0x64, 0x26, 0x04, 0xb3, 0x9b, 0xe6, 0xf8, 0xf8, + 0xd7, 0x45, 0x1e, 0x0e, 0x4c, 0x5b, 0x92, 0x7a, 0xe3, 0x17, 0xb9, 0x81, + 0x79, 0x41, 0xb7, 0x43, 0x47, 0x50, 0xcc, 0x21, 0xdb, 0xf6, 0xcd, 0x54, + 0xd7, 0xba, 0x3c, 0x25, 0x35, 0x45, 0xb3, 0x42, 0xb2, 0x42, 0xa9, 0xe2, + 0xe7, 0x09, 0x90, 0xab, 0xfa, 0xac, 0x24, 0xab, 0x18, 0x28, 0xde, 0xa1, + 0xf5, 0xf2, 0x1b, 0x59, 0x78, 0x58, 0x9a, 0x46, 0x97, 0xe6, 0xa7, 0x79, + 0xec, 0xd4, 0x97, 0x2c, 0xb0, 0xca, 0xb2, 0xca, 0x7a, 0x09, 0x7c, 0x09, + 0xb6, 0x9c, 0x00, 0xef, 0x5d, 0x35, 0x73, 0xb9, 0x75, 0x11, 0x41, 0x5e, + 0x87, 0x98, 0x7b, 0xb7, 0x5d, 0xc9, 0x79, 0x5a, 0x76, 0x97, 0x7e, 0x40, + 0xb6, 0xac, 0x7d, 0xc9, 0x86, 0x27, 0x10, 0xfc, 0x4c, 0xe8, 0xcb, 0xf2, + 0xa6, 0xfa, 0x66, 0xfb, 0x5c, 0x72, 0xf7, 0x13, 0x51, 0xf3, 0x6b, 0x03, + 0x22, 0xe5, 0xd7, 0x55, 0xc4, 0x29, 0x5c, 0xfe, 0x41, 0x4e, 0x47, 0x7b, + 0x7e, 0x22, 0x96, 0xe1, 0xd5, 0x2b, 0x25, 0x51, 0xbd, 0xfe, 0x1f, 0xa2, + 0x45, 0x86, 0xc9, 0x86, 0xa9, 0xdb, 0xc1, 0xf1, 0x6d, 0xb3, 0xfc, 0x05, + 0x0b, 0x12, 0xd1, 0x12, 0x28, 0x7f, 0x2c, 0x87, 0x74, 0x77, 0x29, 0x77, + 0x6d, 0x87, 0x6c, 0x87, 0x0a, 0xa8, 0xbf, 0x52, 0x93, 0x51, 0xcb, 0xc8, + 0x1d, 0xcb, 0x9d, 0x67, 0x28, 0xcb, 0xb5, 0x15, 0xd3, 0x15, 0xf3, 0xc9, + 0xf2, 0xc9, 0x0e, 0x4f, 0x4e, 0x4c, 0xe6, 0x4f, 0x2e, 0xa6, 0x73, 0x66, + 0x0a, 0x7c, 0x5b, 0x62, 0x03, 0x67, 0x82, 0x33, 0x97, 0x32, 0x37, 0x5b, + 0x7e, 0x69, 0x7b, 0x68, 0x33, 0x36, 0x97, 0x36, 0xb7, 0x9a, 0x4a, 0xd5, + 0xef, 0x69, 0x17, 0x6b, 0x9d, 0x37, 0x77, 0xda, 0x85, 0x68, 0xf3, 0x68, + 0x15, 0x69, 0xaa, 0x69, 0xeb, 0x35, 0x48, 0x54, 0x65, 0x55, 0x51, 0x2d, + 0x76, 0x54, 0x75, 0x94, 0xa0, 0x95, 0xb4, 0xc9, 0x27, 0xca, 0x57, 0x64, + 0x2f, 0x9a, 0xf7, 0xd7, 0x5e, 0x35, 0x8f, 0x36, 0x97, 0xb5, 0xe0, 0xd5, + 0xee, 0xff, 0xfb, 0xba, 0x5a, 0xdd, 0x6c, 0xdb, 0xec, 0x6b, 0x68, 0x6e, + 0x84, 0x63, 0xde, 0x32, 0x58, 0xdd, 0x7f, 0x9e, 0x08, 0x4c, 0xf4, 0x6d, + 0xf7, 0x8d, 0x7e, 0x13, 0x80, 0x41, 0x0a, 0x5b, 0x46, 0xae, 0x20, 0xd2, + 0x26, 0xf2, 0x73, 0x6e, 0x74, 0xc2, 0xe4, 0x55, 0xe7, 0xdd, 0xc8, 0x98, + 0xfa, 0x7b, 0x2e, 0xe4, 0x3b, 0x78, 0x9b, 0x72, 0xc4, 0x62, 0xae, 0x39, + 0x8d, 0x3a, 0xfd, 0x8f, 0x7d, 0xb7, 0x19, 0x63, 0x9c, 0x6e, 0xc0, 0x32, + 0x4b, 0x25, 0x13, 0xf4, 0xd3, 0x1c, 0xc7, 0xbc, 0xcd, 0x39, 0xe8, 0x26, + 0x3c, 0x2a, 0x42, 0x26, 0x70, 0x9c, 0x39, 0x55, 0x90, 0x39, 0x93, 0x60, + 0x68, 0x7d, 0x04, 0xb8, 0x27, 0xed, 0x72, 0xe7, 0xa8, 0xb7, 0xed, 0x35, + 0x77, 0x97, 0x95, 0xd9, 0x90, 0xd9, 0x77, 0x56, 0xba, 0x5a, 0x73, 0xce, + 0x75, 0x6e, 0x79, 0x1e, 0xb6, 0x96, 0x33, 0xc1, 0x91, 0x43, 0x97, 0xad, + 0xc5, 0xee, 0x3c, 0xd3, 0x3f, 0x22, 0xba, 0xa5, 0xb4, 0x3f, 0x2b, 0x84, + 0x90, 0x93, 0xcc, 0x6a, 0xbd, 0x59, 0xe9, 0xa7, 0x20, 0xaa, 0xf0, 0xdb, + 0xbc, 0x09, 0x4c, 0xc1, 0x62, 0xcc, 0x12, 0xaf, 0x51, 0xb5, 0x54, 0xdb, + 0x52, 0xe4, 0x5e, 0xfe, 0x60, 0x78, 0x81, 0x23, 0x78, 0x2f, 0xe8, 0x94, + 0xb3, 0x2b, 0xb8, 0xcb, 0xcf, 0xbd, 0xc1, 0xb6, 0xa1, 0x65, 0x30, 0xc1, + 0xdd, 0xd3, 0x32, 0xd2, 0x32, 0xae, 0xd0, 0x6f, 0x9f, 0x6d, 0x10, 0xb2, + 0x29, 0xea, 0x4f, 0xf1, 0x61, 0xf2, 0xe1, 0xf5, 0x31, 0x0b, 0x28, 0x09, + 0xa0, 0x82, 0x32, 0x87, 0x8e, 0xf8, 0x2d, 0xfc, 0x5b, 0xe7, 0xd3, 0xcb, + 0xa3, 0x97, 0x0f, 0x59, 0x77, 0x8c, 0x10, 0xef, 0xaa, 0xe9, 0x7d, 0xfd, + 0xf2, 0x99, 0xf0, 0x9f, 0xdc, 0x36, 0x61, 0x2c, 0xe1, 0xd8, 0xc0, 0xaa, + 0xc0, 0xfd, 0xde, 0x2a, 0x32, 0x8b, 0x91, 0xd3, 0xb8, 0x2a, 0xb2, 0x07, + 0x70, 0xda, 0x54, 0xfc, 0x94, 0x25, 0x72, 0x88, 0xa2, 0xdf, 0xe1, 0xe2, + 0x0c, 0x8c, 0xe1, 0x81, 0x61, 0x54, 0xbf, 0x1f, 0x22, 0x4d, 0x48, 0x58, + 0x88, 0x46, 0xa8, 0x50, 0x8c, 0x43, 0xb8, 0x26, 0xfe, 0xec, 0xb7, 0x54, + 0x4e, 0xf7, 0xd9, 0x35, 0x51, 0xe7, 0x4d, 0xb1, 0x67, 0xb1, 0x14, 0xb1, + 0x8d, 0x4c, 0xf3, 0x6f, 0x0a, 0x3c, 0x35, 0x4c, 0xd2, 0x9c, 0xf4, 0x82, + 0xe5, 0xcc, 0x56, 0xfc, 0x2b, 0xdc, 0xed, 0x1c, 0xed, 0x42, 0x43, 0xcc, + 0x66, 0x5a, 0x13, 0x0b, 0x9a, 0x59, 0x9a, 0x52, 0x36, 0x32, 0x84, 0x3c, + 0xcc, 0xf4, 0xd5, 0x5c, 0x76, 0x36, 0x5d, 0x9b, 0xaf, 0x67, 0x37, 0x61, + 0x3b, 0x59, 0x3b, 0x08, 0xbb, 0xde, 0x38, 0x69, 0x2c, 0x23, 0x7a, 0x0e, + 0x9e, 0x79, 0x53, 0x7d, 0xb7, 0x21, 0x25, 0xd8, 0x65, 0xcc, 0xe8, 0x8c, + 0xbc, 0x29, 0x3f, 0x64, 0xc3, 0xe5, 0xbf, 0x27, 0x5d, 0x0a, 0xa3, 0xd2, + 0x97, 0x62, 0xff, 0xa1, 0xba, 0xc6, 0x70, 0xa7, 0x8a, 0x81, 0xf7, 0xec, + 0x23, 0xea, 0x33, 0x27, 0x67, 0x99, 0xe5, 0x9a, 0x0d, 0xff, 0x27, 0xaa, + 0x4b, 0x4f, 0x2c, 0x7f, 0x2d, 0x49, 0x2f, 0xe7, 0x95, 0x70, 0x1a, 0xdf, + 0xc9, 0xe9, 0x49, 0x39, 0xc8, 0x5e, 0xc4, 0xb8, 0xdf, 0x16, 0x4c, 0x53, + 0x16, 0xd7, 0x29, 0x8b, 0x24, 0xbb, 0x9e, 0xc9, 0x9b, 0x8d, 0x9d, 0xe1, + 0x46, 0x6a, 0xcb, 0x26, 0xc0, 0x0b, 0xfc, 0x16, 0x94, 0x10, 0x1f, 0xe7, + 0x28, 0x52, 0x4e, 0x26, 0x97, 0xd8, 0x6c, 0x7e, 0x6c, 0x7e, 0x60, 0xe6, + 0x36, 0x5e, 0x69, 0x1a, 0x52, 0xff, 0x3e, 0x68, 0x17, 0xd5, 0xc2, 0x42, + 0x9c, 0x0c, 0x33, 0x6a, 0xdd, 0x10, 0x57, 0xf9, 0x45, 0x8b, 0x09, 0xbe, + 0x59, 0xa4, 0xf8, 0x50, 0x55, 0x9b, 0x76, 0x89, 0x1e, 0xa6, 0xde, 0xdc, + 0x54, 0xc3, 0xac, 0x7b, 0x32, 0x62, 0x8c, 0x27, 0xa5, 0x22, 0x9b, 0x61, + 0xf2, 0x6c, 0xa2, 0x61, 0x6c, 0x35, 0x8e, 0xfb, 0xef, 0x45, 0xce, 0xc7, + 0x19, 0x7a, 0x1f, 0xfa, 0x76, 0x42, 0x73, 0x51, 0xa8, 0x8c, 0x06, 0x13, + 0x91, 0xe6, 0x01, 0xb0, 0xd2, 0x8a, 0xa7, 0x9e, 0xdf, 0xd1, 0x73, 0x15, + 0x7e, 0x17, 0xeb, 0x0b, 0x43, 0xa8, 0xfa, 0xd7, 0x4a, 0xfe, 0xd6, 0xc8, + 0x66, 0x7a, 0xdb, 0x05, 0x85, 0x8a, 0x33, 0x8d, 0xd4, 0x6a, 0x3f, 0x20, + 0xb5, 0x97, 0x9c, 0xf5, 0x6f, 0xfd, 0xde, 0xb1, 0xc0, 0xde, 0x26, 0x28, + 0xeb, 0x13, 0x04, 0x35, 0xf8, 0xfd, 0x10, 0xdf, 0x93, 0xb5, 0xa5, 0x33, + 0x3d, 0x05, 0x4d, 0x16, 0xad, 0xcd, 0x26, 0xc3, 0xe7, 0xd0, 0x2c, 0x02, + 0x41, 0xac, 0x88, 0xc8, 0x6a, 0xcc, 0x55, 0x9c, 0x54, 0x2c, 0xff, 0x7f, + 0xb4, 0x6b, 0x1e, 0xfb, 0x04, 0x61, 0xf6, 0x63, 0xdd, 0xe9, 0x6b, 0x49, + 0x8d, 0x65, 0x15, 0x03, 0x6d, 0x26, 0xd3, 0x66, 0x73, 0x66, 0x47, 0x07, + 0x1b, 0x4c, 0xd9, 0x3a, 0xc8, 0x29, 0xe5, 0xa9, 0x07, 0xa9, 0xc8, 0x76, + 0x48, 0xcb, 0x2b, 0x8d, 0x48, 0x9b, 0xea, 0x8d, 0x67, 0x77, 0xf7, 0xe2, + 0xab, 0x2c, 0xa9, 0x93, 0xb4, 0xab, 0x1d, 0x41, 0xcd, 0xa2, 0x6d, 0x05, + 0xdd, 0xfc, 0xad, 0x9d, 0xa6, 0x25, 0xa3, 0x53, 0x93, 0x7c, 0x42, 0xaa, + 0x85, 0xaa, 0xb7, 0xaa, 0xcf, 0xaa, 0x5c, 0xe7, 0x73, 0xeb, 0x4d, 0x86, + 0x4b, 0x8f, 0x76, 0xb3, 0xe7, 0x04, 0xde, 0x9d, 0xf7, 0x1c, 0xa7, 0x0b, + 0xab, 0x3a, 0x9d, 0x0f, 0x6f, 0x82, 0xa3, 0x5d, 0xab, 0x6a, 0x50, 0x6a, + 0x72, 0x0b, 0x66, 0x0b, 0xa6, 0x2f, 0x52, 0x2f, 0x0d, 0x0f, 0x42, 0xd5, + 0x11, 0xe5, 0x47, 0x8f, 0x29, 0x2f, 0xb6, 0x7a, 0x55, 0xf2, 0xb6, 0x8e, + 0x67, 0xc3, 0xd0, 0x73, 0xd0, 0x6f, 0x30, 0x14, 0xc8, 0xcd, 0x4b, 0x6d, + 0xf3, 0x0b, 0x44, 0x73, 0x02, 0xaa, 0xca, 0xb1, 0x8b, 0x0c, 0x27, 0x3f, + 0x91, 0x87, 0x49, 0xba, 0xde, 0xb3, 0x2e, 0xdc, 0x99, 0x90, 0xb8, 0x90, + 0xd4, 0xa2, 0xff, 0x0e, 0xed, 0x50, 0x38, 0x88, 0x79, 0xee, 0x1d, 0x3d, + 0xc0, 0x6d, 0x3e, 0x10, 0x23, 0xf1, 0xae, 0xb8, 0xfd, 0xf1, 0xae, 0xdc, + 0x86, 0x01, 0x91, 0xb8, 0x11, 0x99, 0x13, 0xb1, 0x58, 0x89, 0x64, 0x5c, + 0x67, 0x48, 0x64, 0xaa, 0x64, 0xbc, 0xe8, 0x18, 0xe9, 0xb4, 0x2b, 0x4c, + 0x08, 0xce, 0x2e, 0x1e, 0x6f, 0x78, 0xbd, 0xfe, 0x5c, 0x25, 0x41, 0xf9, + 0xcb, 0xc1, 0xaf, 0xe5, 0x29, 0x7d, 0xa3, 0x3a, 0x6a, 0xb9, 0x72, 0x14, + 0x08, 0x4a, 0x34, 0xbf, 0x9e, 0xfe, 0x93, 0x95, 0x9c, 0xe5, 0xeb, 0xc7, + 0xfc, 0xb4, 0xf6, 0x6b, 0x35, 0xbf, 0xd9, 0x1b, 0x6c, 0x3c, 0xb5, 0x69, + 0x85, 0x62, 0x7f, 0xd1, 0x8e, 0xea, 0xb2, 0x3d, 0x27, 0xfe, 0x11, 0x05, + 0xe6, 0xeb, 0xe4, 0x9b, 0xd2, 0xbd, 0x6e, 0x75, 0xb9, 0xae, 0x5f, 0xdd, + 0xc7, 0xd0, 0x68, 0xd5, 0xc8, 0xa8, 0xe1, 0x35, 0xb0, 0x3d, 0x76, 0x39, + 0xe6, 0x7f, 0x4f, 0x04, 0x29, 0x0c, 0x77, 0xfc, 0xed, 0xc8, 0xa8, 0x6e, + 0xd3, 0xb5, 0x6e, 0x3b, 0xf7, 0x02, 0x36, 0x79, 0x9b, 0x3d, 0x1c, 0x11, + 0x64, 0x73, 0x4f, 0xad, 0xdf, 0x02, 0x3d, 0xaf, 0xb0, 0x3f, 0x16, 0x17, + 0xb2, 0x79, 0x04, 0xd6, 0x81, 0x37, 0x5e, 0xf6, 0x0f, 0x83, 0x0f, 0x77, + 0x9b, 0xd8, 0xeb, 0x7e, 0x4e, 0x34, 0x6f, 0x53, 0xe3, 0x1d, 0xcb, 0xee, + 0x34, 0xe7, 0x55, 0x9d, 0x0a, 0x9d, 0x06, 0xb7, 0x3f, 0xce, 0x84, 0x97, + 0x45, 0xd7, 0xc3, 0x16, 0x84, 0x74, 0xde, 0xd7, 0xfe, 0x5d, 0xac, 0xf9, + 0x5b, 0xf5, 0xb8, 0xe7, 0xde, 0xcc, 0xae, 0xfd, 0x7c, 0xf8, 0x1a, 0x12, + 0x34, 0xfd, 0x1e, 0x7f, 0x17, 0x8b, 0x22, 0xfa, 0x25, 0x36, 0x3d, 0x76, + 0x3c, 0x14, 0x29, 0xf4, 0x24, 0xfb, 0xbb, 0x20, 0x0f, 0xcf, 0x8e, 0x90, + 0xd3, 0xdb, 0xd1, 0x6b, 0x89, 0xe8, 0xb4, 0xc9, 0x34, 0x75, 0x36, 0xce, + 0x7b, 0xfe, 0xfb, 0x78, 0x35, 0x0e, 0x36, 0x8b, 0xdf, 0xe3, 0xc2, 0xc9, + 0xd2, 0xd4, 0x6c, 0xda, 0x6c, 0x5e, 0xea, 0x75, 0x6a, 0x62, 0xb7, 0xad, + 0x4f, 0xd1, 0xdb, 0x71, 0xcf, 0x09, 0x5e, 0xcd, 0xea, 0x50, 0xdf, 0x10, + 0x71, 0xd7, 0xa8, 0x07, 0xfb, 0x37, 0xfd, 0x5b, 0xf5, 0x97, 0xb3, 0x5e, + 0xf7, 0x35, 0x77, 0xa2, 0x8b, 0x82, 0x4d, 0x14, 0x85, 0x3d, 0x1f, 0x52, + 0x7f, 0xda, 0x0f, 0x8b, 0xa7, 0xba, 0x3b, 0x95, 0x99, 0x92, 0x97, 0x44, + 0xbc, 0xc4, 0xc4, 0x8f, 0xcf, 0xd0, 0x69, 0x37, 0xaf, 0xc6, 0xec, 0x4c, + 0xab, 0x10, 0x87, 0xf9, 0xd5, 0x48, 0xe6, 0xa7, 0x18, 0xe0, 0xe3, 0xbf, + 0x57, 0x2e, 0xb0, 0x24, 0x11, 0xb2, 0x07, 0xce, 0x41, 0x52, 0xcb, 0x19, + 0x00, 0xd0, 0x95, 0xf8, 0x6f, 0x83, 0xc2, 0x4c, 0x12, 0x15, 0x86, 0xec, + 0x84, 0x71, 0x91, 0x92, 0x13, 0x87, 0x59, 0x81, 0xc1, 0x81, 0xe2, 0x64, + 0x40, 0x99, 0x71, 0x27, 0x06, 0x00, 0x48, 0x01, 0x52, 0x62, 0xc2, 0xaa, + 0x1e, 0x2b, 0xa7, 0x59, 0xee, 0x1e, 0x98, 0x6a, 0x0f, 0xc3, 0x42, 0xee, + 0x66, 0xcd, 0x85, 0x34, 0x71, 0xc1, 0x33, 0x70, 0xf1, 0xc8, 0x64, 0x92, + 0x68, 0x8a, 0x62, 0x62, 0x4a, 0x74, 0x91, 0xb0, 0x81, 0x91, 0xd1, 0x1a, + 0x83, 0x22, 0x68, 0x3e, 0xb6, 0xe1, 0xe1, 0xb0, 0xf4, 0x91, 0xe1, 0x91, + 0xb4, 0xf4, 0xe4, 0xcf, 0x2a, 0x8a, 0xdb, 0x45, 0xb0, 0x9a, 0x47, 0xf0, + 0xb0, 0x92, 0xf1, 0xd4, 0x92, 0x70, 0x40, 0x4a, 0x4a, 0x32, 0x5b, 0xee, + 0x0e, 0x76, 0x67, 0x7f, 0x7f, 0x9b, 0xfa, 0xaf, 0x96, 0x50, 0xe8, 0xb0, + 0x99, 0x4f, 0x82, 0x28, 0x28, 0x1d, 0x99, 0x74, 0xee, 0x53, 0x7f, 0x2f, + 0x04, 0xc0, 0x6b, 0x4e, 0xc8, 0x9a, 0xb0, 0xdf, 0x4b, 0xad, 0x5e, 0x2d, + 0x3f, 0xd4, 0x2b, 0x52, 0x30, 0x23, 0xd1, 0x0d, 0x88, 0xfb, 0x30, 0x30, + 0xa9, 0x9a, 0x53, 0x50, 0x5d, 0x29, 0x18, 0x23, 0xbe, 0x08, 0x06, 0x53, + 0x05, 0x8e, 0x13, 0x70, 0xe9, 0x08, 0x74, 0x6e, 0x54, 0x85, 0xf8, 0x45, + 0x44, 0x63, 0x91, 0x9f, 0xd0, 0xd3, 0x99, 0xea, 0xb1, 0xe2, 0xc3, 0xe5, + 0x49, 0xc8, 0x1c, 0x95, 0x01, 0x38, 0x00, 0x20, 0xc7, 0x50, 0x86, 0xb7, + 0xe8, 0x19, 0x33, 0xf8, 0x09, 0x24, 0x91, 0xe0, 0x1a, 0xee, 0xc9, 0x24, + 0xa6, 0x2f, 0xfe, 0x11, 0x91, 0xed, 0xf1, 0xb7, 0xb7, 0x39, 0x51, 0x4d, + 0xeb, 0x91, 0xca, 0x48, 0xa0, 0x35, 0x6b, 0xe0, 0x8f, 0xa8, 0x43, 0xec, + 0xa9, 0x50, 0x6c, 0x26, 0xe4, 0xa8, 0x85, 0x8a, 0x13, 0x4e, 0xd4, 0xbe, + 0x56, 0x10, 0x87, 0x33, 0x6f, 0xa0, 0x0d, 0x3c, 0xf5, 0x02, 0xc3, 0xf0, + 0x8e, 0xcb, 0xbf, 0xba, 0x55, 0x9b, 0xe0, 0xcf, 0x7c, 0x11, 0x91, 0xb0, + 0x14, 0x75, 0x76, 0xfc, 0x48, 0xd6, 0x4b, 0xa4, 0x75, 0xfa, 0x72, 0x7d, + 0x22, 0x41, 0x8f, 0xd6, 0xc3, 0x82, 0x1c, 0x6a, 0x83, 0x86, 0x2d, 0x86, + 0x92, 0x89, 0x74, 0xec, 0x7e, 0x66, 0x4b, 0xd9, 0x41, 0x6c, 0x4b, 0xc9, + 0x34, 0x6b, 0xc4, 0x28, 0x99, 0x2d, 0x17, 0xd0, 0x36, 0x32, 0x21, 0x33, + 0x1b, 0x2a, 0x98, 0x2f, 0x12, 0xfd, 0x92, 0x9b, 0xea, 0xba, 0x67, 0xcd, + 0xab, 0x04, 0x61, 0x9e, 0xfc, 0xfe, 0xc3, 0x9f, 0xb8, 0x03, 0x5f, 0xa0, + 0x37, 0x6b, 0x56, 0x62, 0x45, 0xbe, 0x27, 0x60, 0x0d, 0x0a, 0x52, 0x50, + 0xee, 0xa5, 0x50, 0xa5, 0xa7, 0x7f, 0x80, 0xa1, 0x0e, 0x41, 0xc3, 0xe3, + 0x25, 0x39, 0x2b, 0x95, 0xa9, 0xb6, 0x68, 0xc0, 0x4a, 0x32, 0x6a, 0x11, + 0xa5, 0x3f, 0x62, 0x74, 0xeb, 0x2c, 0x52, 0xe9, 0xab, 0xc4, 0xb9, 0xba, + 0x26, 0x69, 0x95, 0xb8, 0x3a, 0xfa, 0x2e, 0x6d, 0x79, 0x54, 0xb1, 0xb7, + 0xc7, 0xe1, 0x0c, 0xd3, 0x41, 0x1c, 0x54, 0xc5, 0x49, 0x69, 0xc5, 0x11, + 0x49, 0xe8, 0x64, 0x62, 0x64, 0xd6, 0xf4, 0xf2, 0x1e, 0x6f, 0xca, 0xd2, + 0x85, 0x49, 0x5d, 0x06, 0xc1, 0x3f, 0x60, 0x85, 0xab, 0xf9, 0x62, 0x85, + 0x29, 0x04, 0x7f, 0x81, 0xb3, 0x83, 0x08, 0x05, 0x52, 0xe2, 0xfd, 0x6d, + 0x66, 0x74, 0x51, 0x29, 0x54, 0x39, 0xed, 0x2a, 0x0c, 0x4f, 0x27, 0x04, + 0xcd, 0x4d, 0x31, 0x0c, 0x27, 0x7b, 0xd3, 0xe8, 0x4c, 0xed, 0xf6, 0x5f, + 0x73, 0xf6, 0xba, 0x5d, 0x6a, 0x67, 0xdc, 0x5f, 0x90, 0x33, 0x99, 0x9e, + 0xa9, 0xd4, 0x6c, 0x6f, 0x9b, 0x56, 0x81, 0x6b, 0x01, 0x8c, 0x27, 0x0b, + 0x3a, 0x62, 0x28, 0x8c, 0x16, 0xda, 0x9f, 0xda, 0xbb, 0x1a, 0x02, 0x08, + 0xf7, 0x39, 0x98, 0xe8, 0xa5, 0x96, 0x40, 0xc4, 0x71, 0x73, 0xae, 0x2c, + 0xb8, 0xb4, 0x91, 0xa4, 0xc5, 0xba, 0x9f, 0xa7, 0xf9, 0x57, 0xa2, 0x32, + 0xab, 0x6d, 0x23, 0xc9, 0x3f, 0x13, 0xd7, 0xf8, 0xf5, 0x71, 0x16, 0x26, + 0x3d, 0x05, 0x63, 0xa7, 0x15, 0x4b, 0xa9, 0xc7, 0x44, 0xe9, 0x6a, 0x10, + 0x67, 0xb6, 0xd0, 0x34, 0x58, 0xec, 0x7f, 0x15, 0x09, 0x24, 0x24, 0x94, + 0x96, 0x12, 0x45, 0x0b, 0xe4, 0x85, 0x5b, 0x68, 0x05, 0x8a, 0x5a, 0xd3, + 0x8b, 0xfd, 0xcc, 0xa3, 0x87, 0x37, 0x13, 0xbc, 0x0a, 0x41, 0xfd, 0xc7, + 0x81, 0x55, 0x63, 0x58, 0x7b, 0xd9, 0x72, 0xe7, 0xf3, 0x7a, 0xe7, 0xf3, + 0xbe, 0x52, 0x87, 0x43, 0x27, 0x0c, 0x2a, 0xdf, 0xfc, 0x22, 0xcc, 0xee, + 0xfa, 0x56, 0xd3, 0xc9, 0xfe, 0x07, 0x1b, 0xf0, 0x95, 0x4b, 0x67, 0xf0, + 0x40, 0xdf, 0x7e, 0x8c, 0x4d, 0x8d, 0x93, 0x48, 0x09, 0x2d, 0x6f, 0x20, + 0x29, 0x76, 0x6f, 0xf1, 0xbe, 0xca, 0x72, 0xb9, 0x90, 0x19, 0xb9, 0x23, + 0x7a, 0xe6, 0x17, 0x4d, 0x77, 0xf5, 0xf2, 0x9f, 0x7e, 0x43, 0xf0, 0x99, + 0x14, 0x68, 0xd9, 0x9f, 0xb6, 0xc9, 0x41, 0x32, 0x9e, 0x5e, 0x4c, 0x55, + 0x8d, 0x2e, 0xb6, 0x9d, 0x69, 0xb2, 0x36, 0xdc, 0x6f, 0xb9, 0xff, 0xc8, + 0x26, 0x70, 0x15, 0xc8, 0x9c, 0xb3, 0x86, 0x26, 0x2f, 0x0e, 0xd7, 0x06, + 0x62, 0x8d, 0x91, 0x83, 0x22, 0x89, 0x3d, 0xe5, 0x8a, 0x26, 0xce, 0xaa, + 0xde, 0xef, 0x3d, 0xe7, 0x46, 0xc3, 0x51, 0x41, 0xee, 0xe7, 0x60, 0x9d, + 0xc4, 0x60, 0x72, 0x35, 0xdc, 0xdb, 0x6b, 0x7f, 0xba, 0x5d, 0x25, 0xd1, + 0x6a, 0x70, 0xe4, 0x23, 0x68, 0x3e, 0x65, 0x74, 0xe2, 0x2b, 0x57, 0x8f, + 0xdc, 0x2f, 0x82, 0x02, 0x5a, 0xea, 0x0a, 0x41, 0x65, 0x16, 0x67, 0xef, + 0x21, 0xa5, 0x15, 0x8f, 0xa2, 0xc8, 0x2c, 0xb2, 0x3a, 0xbd, 0x6a, 0xec, + 0xb5, 0x5f, 0x2b, 0x26, 0xa5, 0x21, 0x8b, 0xa4, 0x75, 0xe3, 0x4b, 0x94, + 0x9f, 0x6c, 0x2e, 0x59, 0x2d, 0x58, 0x61, 0x33, 0x68, 0x6c, 0x34, 0xc9, + 0x49, 0x6b, 0x9c, 0x45, 0xd6, 0x29, 0xf9, 0x08, 0xed, 0xbb, 0x8a, 0xbd, + 0x73, 0xca, 0x2f, 0xd9, 0x05, 0x24, 0x35, 0xee, 0x22, 0x12, 0xb1, 0xa7, + 0xab, 0x11, 0xbc, 0x8b, 0x48, 0x53, 0xc2, 0x1a, 0x36, 0x2c, 0x1c, 0xa2, + 0x13, 0x6f, 0xd8, 0x6d, 0xd4, 0x19, 0x3b, 0x53, 0x16, 0x6a, 0xc3, 0x1b, + 0xa3, 0xbd, 0x5d, 0x8b, 0x68, 0x6a, 0x5a, 0xd7, 0xd7, 0xa7, 0x14, 0x58, + 0x72, 0xa2, 0xf5, 0xad, 0xfe, 0x58, 0x52, 0x59, 0x94, 0x39, 0x5a, 0x04, + 0xe0, 0x61, 0xe3, 0x33, 0x0a, 0xfa, 0xd9, 0xa2, 0x36, 0xed, 0x30, 0x14, + 0x94, 0x1d, 0x5b, 0xc2, 0x29, 0xa7, 0xfd, 0x21, 0xae, 0x91, 0x1f, 0x86, + 0x09, 0x8d, 0xcb, 0xc8, 0x7e, 0xaa, 0xe9, 0x65, 0x09, 0x81, 0xd4, 0x8f, + 0x5d, 0xeb, 0xf6, 0xd7, 0x0c, 0x2e, 0xdb, 0xb1, 0x72, 0xb5, 0x8a, 0x7f, + 0xed, 0xae, 0x0c, 0xab, 0xcd, 0xb6, 0x7c, 0xbd, 0xae, 0x2c, 0x7f, 0x94, + 0x14, 0x1e, 0xf2, 0x54, 0x3a, 0xb0, 0x4b, 0x80, 0x06, 0x12, 0x66, 0x5c, + 0x4b, 0x77, 0xcf, 0xf6, 0xef, 0x35, 0xc2, 0x18, 0x67, 0xf7, 0x56, 0x6f, + 0x02, 0xd9, 0xb6, 0x1b, 0x22, 0x8b, 0x8f, 0x28, 0x97, 0x5c, 0x78, 0x97, + 0x5b, 0x67, 0x3b, 0xdb, 0xd9, 0x2d, 0xbf, 0xd2, 0x0e, 0x78, 0x46, 0x87, + 0xb2, 0xc0, 0x86, 0x5b, 0xe3, 0x18, 0x8f, 0xbe, 0x6f, 0x06, 0x4f, 0xdd, + 0x9f, 0x4b, 0x7e, 0xd8, 0x31, 0xfe, 0x90, 0x94, 0x24, 0x70, 0x72, 0x4a, + 0xd1, 0x82, 0x37, 0x41, 0x7b, 0x73, 0xaa, 0xdd, 0x41, 0x72, 0xe8, 0xe0, + 0x2e, 0xb0, 0x54, 0x6b, 0x3f, 0x7e, 0xfe, 0x5d, 0x23, 0xbc, 0x02, 0x95, + 0x7d, 0xee, 0x4d, 0x03, 0x6c, 0xd7, 0x0d, 0xcb, 0xb8, 0xf0, 0xfa, 0xc8, + 0x9c, 0xa0, 0x19, 0xee, 0xaa, 0x75, 0x9e, 0x26, 0xcb, 0xf6, 0x37, 0x46, + 0x5f, 0x75, 0x27, 0xf9, 0xfe, 0xbb, 0xb1, 0xfb, 0xf5, 0x1a, 0x61, 0x49, + 0xdf, 0x9b, 0x29, 0x32, 0x3f, 0xb7, 0xfc, 0x58, 0x0a, 0x9c, 0x51, 0x7c, + 0xf4, 0xf1, 0x30, 0xce, 0xd2, 0xcd, 0xea, 0xf7, 0xea, 0xd1, 0xf5, 0xf1, + 0xfe, 0x82, 0x11, 0xe3, 0x59, 0x3b, 0x3c, 0x89, 0x78, 0xea, 0xcc, 0x57, + 0xaf, 0xf8, 0x53, 0x65, 0x86, 0x18, 0xc4, 0x01, 0x89, 0x79, 0x0e, 0xae, + 0x90, 0x29, 0x76, 0x00, 0x26, 0xb9, 0x51, 0x40, 0xd2, 0xf0, 0x30, 0x83, + 0xf4, 0x1e, 0x4e, 0x71, 0xfa, 0xec, 0x0b, 0x98, 0x45, 0x02, 0x43, 0x45, + 0x70, 0xe7, 0xcf, 0x99, 0x10, 0x46, 0x61, 0x12, 0xe1, 0xf8, 0xc8, 0x5e, + 0xa8, 0x77, 0xf6, 0x64, 0xec, 0x26, 0x96, 0xbb, 0x89, 0x49, 0xec, 0xc8, + 0x9f, 0x3f, 0x35, 0x37, 0x8c, 0x48, 0xf5, 0xe1, 0xfa, 0xe5, 0x68, 0x38, + 0x0f, 0xfc, 0x61, 0x28, 0xdd, 0xa1, 0xc5, 0x0b, 0x9d, 0x4a, 0x4b, 0x89, + 0xdd, 0xd0, 0xa0, 0xed, 0xda, 0xd5, 0x47, 0x9f, 0xd3, 0x86, 0xf9, 0x4a, + 0xf5, 0xab, 0x24, 0xe6, 0x9b, 0x23, 0xc9, 0x5f, 0x2d, 0xab, 0x87, 0x7c, + 0x26, 0x84, 0xdd, 0xd5, 0x1b, 0x2e, 0x7d, 0x9b, 0xd9, 0xbe, 0xf7, 0x5a, + 0x12, 0x31, 0x84, 0x11, 0xc5, 0x96, 0x60, 0x59, 0xb6, 0xd8, 0xe1, 0xfe, + 0x8b, 0xee, 0x8f, 0xa9, 0x99, 0x3c, 0x51, 0xad, 0x76, 0x97, 0xb2, 0x8f, + 0x07, 0xcb, 0xdd, 0x30, 0x65, 0x96, 0x3c, 0xf8, 0x12, 0xd6, 0xf7, 0x3d, + 0xcf, 0xce, 0x5b, 0x8f, 0x57, 0xac, 0xb6, 0x87, 0x2d, 0xa5, 0x8f, 0xe8, + 0x0d, 0xfc, 0xe7, 0x52, 0x9b, 0x4d, 0xde, 0x32, 0x4d, 0x97, 0x09, 0x0e, + 0x39, 0x3f, 0xbd, 0x86, 0x74, 0xa8, 0xfe, 0x4d, 0x98, 0x60, 0x3c, 0xc2, + 0x35, 0x66, 0x41, 0x6e, 0xf7, 0x3c, 0x4e, 0x21, 0x73, 0x34, 0xe0, 0x2d, + 0x39, 0xbd, 0xd5, 0x0f, 0x4d, 0xaf, 0xe7, 0x17, 0x9d, 0x9e, 0x52, 0xe5, + 0x36, 0xa3, 0x46, 0x9f, 0xdd, 0xb6, 0x30, 0xf1, 0x37, 0x86, 0xf2, 0x2c, + 0x0c, 0x3c, 0x84, 0x91, 0xd5, 0xf7, 0x6c, 0xf3, 0xef, 0xea, 0x23, 0x72, + 0xfe, 0xa7, 0x75, 0xc3, 0xb8, 0x02, 0x5e, 0x1c, 0x86, 0x86, 0x86, 0x38, + 0x26, 0xe2, 0x08, 0x53, 0xe3, 0xe3, 0xe3, 0x05, 0xda, 0x4d, 0x56, 0x84, + 0x02, 0x5e, 0x98, 0x01, 0x99, 0xab, 0x5c, 0xbb, 0xeb, 0xa9, 0xd9, 0xa4, + 0xc3, 0xfb, 0x82, 0xdb, 0x2b, 0x5d, 0xbd, 0x42, 0x21, 0xdd, 0x6e, 0xe7, + 0x45, 0xe6, 0xdd, 0x4a, 0x00, 0x0e, 0x99, 0xca, 0xc9, 0x79, 0xf7, 0xb0, + 0xb8, 0xac, 0x5d, 0xde, 0x32, 0xa2, 0x2f, 0xfd, 0x27, 0x91, 0xe4, 0xac, + 0x72, 0x11, 0x49, 0x82, 0xd3, 0x7b, 0x36, 0xb6, 0xda, 0xed, 0x56, 0xc6, + 0x82, 0xcb, 0x58, 0x7a, 0xe1, 0xb5, 0x33, 0xe1, 0xe1, 0xcc, 0x9e, 0x78, + 0x78, 0x1f, 0xe6, 0x74, 0xc9, 0xe2, 0xb7, 0x32, 0xb1, 0x82, 0xbe, 0xfc, + 0x3e, 0xcf, 0x65, 0x44, 0x00, 0xb5, 0xe8, 0xd7, 0x19, 0x56, 0xb6, 0x67, + 0x64, 0x82, 0x99, 0xcc, 0x8a, 0x4c, 0xce, 0x42, 0xa7, 0x90, 0xeb, 0xc3, + 0x50, 0xc7, 0x29, 0x31, 0x59, 0xbe, 0x6c, 0xb9, 0xd4, 0x0f, 0x44, 0xaa, + 0xa5, 0xfd, 0x9b, 0x5f, 0x34, 0xf2, 0x7f, 0x25, 0x2f, 0x36, 0x80, 0x70, + 0xc8, 0xc8, 0x82, 0xf6, 0x81, 0x39, 0xe7, 0xfa, 0xca, 0xd2, 0xcd, 0x36, + 0xcb, 0x22, 0x1d, 0x6e, 0x67, 0x92, 0x72, 0x72, 0x54, 0x1c, 0xdf, 0xc3, + 0xa9, 0x0c, 0xba, 0x43, 0xf2, 0xc3, 0x55, 0x36, 0xc0, 0x8a, 0xd2, 0xbc, + 0x44, 0x24, 0xb8, 0x81, 0x49, 0x09, 0xc7, 0x6a, 0xf4, 0x97, 0x6f, 0x7b, + 0x4a, 0x90, 0x08, 0x3d, 0x4d, 0xd7, 0xc4, 0x25, 0x9e, 0x1c, 0xec, 0x5a, + 0x6e, 0x36, 0xb8, 0x0e, 0x13, 0x23, 0x97, 0x14, 0xaa, 0x30, 0xb3, 0xfd, + 0x25, 0x26, 0xaa, 0x29, 0x92, 0x79, 0xa9, 0xb8, 0xf4, 0xb8, 0x03, 0x35, + 0x03, 0xb7, 0xda, 0x42, 0x3b, 0x9f, 0xe8, 0x6e, 0x70, 0xaa, 0x12, 0x32, + 0xb0, 0x5f, 0x72, 0xab, 0x49, 0xb2, 0x8c, 0x2c, 0xd3, 0xc4, 0x6f, 0x95, + 0x83, 0xe3, 0x5a, 0xc8, 0xc0, 0x1b, 0xde, 0x1d, 0xeb, 0x9a, 0x7e, 0x8a, + 0x39, 0xab, 0x45, 0xe5, 0xe5, 0x3b, 0x36, 0x05, 0xd3, 0x25, 0x2a, 0xf7, + 0x58, 0x27, 0xef, 0x3b, 0x11, 0x38, 0x9b, 0xc2, 0xd2, 0x16, 0x73, 0x85, + 0x70, 0x37, 0xe0, 0x69, 0x1a, 0x66, 0xe6, 0x6a, 0x31, 0x1d, 0x83, 0x1f, + 0xf5, 0x53, 0x59, 0x7c, 0xec, 0xf6, 0x1b, 0xfc, 0x56, 0xaa, 0x65, 0x3d, + 0x28, 0x2a, 0xfa, 0xfd, 0x59, 0x60, 0x9e, 0xfd, 0xab, 0x30, 0xe5, 0xb9, + 0xb6, 0xfd, 0x77, 0x30, 0x54, 0xa9, 0x5a, 0x01, 0x9b, 0x90, 0xdf, 0x3b, + 0x7e, 0xbb, 0xe1, 0xe5, 0xbf, 0x19, 0x77, 0xad, 0xcb, 0xd5, 0x76, 0x91, + 0x5d, 0x4c, 0xf9, 0x35, 0xbb, 0xcd, 0x09, 0xad, 0xb3, 0x47, 0xfa, 0xd7, + 0xcd, 0x56, 0x70, 0xad, 0xe5, 0x31, 0xde, 0xfe, 0xce, 0xd5, 0x71, 0x63, + 0x6b, 0x15, 0xc9, 0xaf, 0x4c, 0xa4, 0x8d, 0xea, 0x14, 0x3b, 0x2d, 0xb1, + 0x6e, 0x13, 0x87, 0xef, 0x3b, 0xa3, 0xe3, 0xab, 0xd1, 0xe5, 0xea, 0x67, + 0x16, 0xb8, 0x2f, 0x5d, 0xb6, 0x0b, 0x85, 0xfa, 0x97, 0x5b, 0x61, 0xed, + 0xe3, 0xed, 0x70, 0x6a, 0x21, 0x76, 0xb8, 0xe0, 0x33, 0xe5, 0x8b, 0x35, + 0xef, 0xf0, 0xf3, 0xd8, 0xe8, 0xcb, 0xc1, 0x09, 0x50, 0xf1, 0x6c, 0x83, + 0xf9, 0xb4, 0x51, 0xd3, 0x96, 0x79, 0x87, 0xbf, 0xbe, 0x3e, 0x71, 0x32, + 0x83, 0x06, 0x35, 0x13, 0x53, 0xf4, 0xd4, 0x99, 0x0e, 0xdb, 0x71, 0x37, + 0x4a, 0xb6, 0xd3, 0x00, 0x31, 0x1d, 0x8f, 0xd3, 0xe9, 0x77, 0xee, 0xcd, + 0x0f, 0x7e, 0x65, 0x69, 0x91, 0xa0, 0x2c, 0x48, 0x68, 0x3c, 0x5c, 0x73, + 0x1e, 0xc8, 0x83, 0x7f, 0x48, 0x8f, 0x2e, 0x56, 0xe9, 0x40, 0x62, 0x4b, + 0x2d, 0x28, 0x08, 0x49, 0x74, 0xcb, 0x95, 0x52, 0x2e, 0x95, 0x21, 0x7e, + 0x5f, 0x09, 0x96, 0x9f, 0xd2, 0xa3, 0x0b, 0xeb, 0x30, 0xdb, 0x4e, 0x51, + 0x05, 0xca, 0xed, 0xf4, 0x9a, 0x2b, 0x6e, 0x6e, 0xb4, 0x29, 0xb0, 0xd2, + 0x5f, 0x91, 0x08, 0xfc, 0x0a, 0xe1, 0xb7, 0x1f, 0x67, 0x59, 0xef, 0x27, + 0xbd, 0x2f, 0xf6, 0xfd, 0x84, 0x13, 0x37, 0x3c, 0x22, 0xb5, 0xaf, 0x92, + 0x43, 0x1c, 0x89, 0xd5, 0x3b, 0x01, 0x32, 0x1a, 0xb6, 0xd7, 0x89, 0x3e, + 0x5b, 0x3d, 0xbf, 0x6f, 0x4f, 0x97, 0x41, 0x0a, 0x19, 0x51, 0x46, 0x58, + 0xc1, 0x3d, 0xdb, 0x0e, 0xb5, 0xee, 0x35, 0xf6, 0x88, 0x36, 0xe7, 0x81, + 0xc9, 0xaf, 0xad, 0x2e, 0x93, 0x65, 0x11, 0xcc, 0x9f, 0x1d, 0xbf, 0x29, + 0xf1, 0xcf, 0x00, 0xa1, 0x01, 0xcb, 0x6d, 0x76, 0x42, 0xa3, 0x50, 0xa7, + 0x6e, 0x7c, 0x3a, 0x2d, 0x76, 0x45, 0xfa, 0x40, 0xaf, 0x35, 0xc7, 0x61, + 0x22, 0xda, 0x48, 0x05, 0xac, 0x8f, 0x7f, 0xad, 0x59, 0x45, 0x15, 0xef, + 0xfc, 0x77, 0x9c, 0x08, 0x4d, 0x05, 0xc6, 0x1d, 0xd8, 0x92, 0x47, 0x91, + 0xae, 0xbc, 0x7c, 0x9a, 0x27, 0xc6, 0x4c, 0x51, 0x7e, 0x8e, 0x86, 0x0d, + 0x83, 0x5b, 0x77, 0xc5, 0xa4, 0x62, 0x16, 0x24, 0x74, 0x7a, 0x89, 0xe4, + 0x14, 0x81, 0x7b, 0x7f, 0x63, 0x7f, 0x86, 0x5f, 0xc6, 0xfd, 0xa2, 0x04, + 0xdb, 0xbb, 0x16, 0x25, 0xbd, 0x76, 0xf9, 0x11, 0xc4, 0x76, 0xb1, 0x4e, + 0x81, 0x77, 0x16, 0xab, 0xcf, 0x72, 0x3e, 0xba, 0x34, 0x29, 0x3f, 0xb8, + 0xdd, 0x2e, 0x36, 0xe4, 0x26, 0x33, 0x38, 0xa1, 0x1e, 0xef, 0x56, 0x5b, + 0xe0, 0x90, 0xe0, 0x78, 0xf7, 0x1d, 0x3d, 0x89, 0x59, 0xdc, 0xf9, 0x79, + 0x03, 0xdf, 0xc2, 0x54, 0xe4, 0x9f, 0x1e, 0x20, 0xd5, 0x95, 0xad, 0x63, + 0xd7, 0xb6, 0x1f, 0xbc, 0x69, 0xa5, 0x5e, 0x31, 0xf2, 0x3a, 0x5a, 0x20, + 0x4f, 0x80, 0x0d, 0x98, 0xd0, 0x79, 0xe4, 0x63, 0xf9, 0x93, 0x8e, 0xf8, + 0x79, 0x58, 0xf8, 0x1c, 0x4d, 0xe6, 0x85, 0x31, 0xca, 0x6f, 0x96, 0xb4, + 0xa8, 0x21, 0x03, 0x35, 0x86, 0x83, 0xae, 0x81, 0x46, 0x0f, 0x54, 0x99, + 0xad, 0x2b, 0xe8, 0xfb, 0x17, 0x16, 0x01, 0xee, 0x93, 0xa3, 0xa1, 0xfa, + 0xd2, 0x51, 0x2d, 0xfb, 0xaf, 0x47, 0x8f, 0x8f, 0x87, 0xa7, 0x1b, 0xb0, + 0x81, 0x72, 0x02, 0xf4, 0xc0, 0xf0, 0xf0, 0xcd, 0x2d, 0x48, 0x01, 0x7a, + 0xa6, 0xe9, 0xe2, 0x12, 0xa4, 0xfd, 0xab, 0x43, 0xce, 0x78, 0xf1, 0x67, + 0x3f, 0x1d, 0x0e, 0x96, 0xdf, 0x58, 0xf7, 0x47, 0xe6, 0x1e, 0xa0, 0xe3, + 0x9e, 0x78, 0xc7, 0x41, 0xfa, 0x19, 0xef, 0x22, 0xe7, 0x64, 0x8a, 0xc1, + 0x81, 0x30, 0x90, 0x71, 0xb2, 0x21, 0x5c, 0x7f, 0xa7, 0x76, 0xd5, 0xd7, + 0xbb, 0x5b, 0x8d, 0xf3, 0x59, 0x5e, 0x6f, 0xb3, 0x9b, 0x86, 0x7c, 0x02, + 0xfb, 0x69, 0x70, 0x75, 0xfa, 0x7a, 0xbb, 0xf4, 0xf7, 0x8e, 0x2c, 0xc5, + 0x08, 0xd8, 0x68, 0x92, 0xc6, 0xf7, 0xa6, 0xc0, 0x9f, 0xe9, 0xa5, 0x54, + 0xa9, 0x40, 0x52, 0x52, 0x92, 0xd6, 0x31, 0x01, 0x07, 0x55, 0x63, 0xc9, + 0xd5, 0x6c, 0x32, 0x63, 0xf0, 0x7a, 0x7f, 0x7c, 0xb9, 0xd1, 0x12, 0x63, + 0x74, 0x74, 0xf4, 0xdd, 0x38, 0xdb, 0xe7, 0x5a, 0x0a, 0xa0, 0x0c, 0x8a, + 0x49, 0x4c, 0xcf, 0xf2, 0xd6, 0xe6, 0xd6, 0x51, 0xc3, 0xf2, 0x7b, 0x7f, + 0x75, 0x3d, 0xa6, 0x56, 0x65, 0xba, 0x7e, 0xd3, 0x3b, 0x67, 0xaf, 0x09, + 0x73, 0xdc, 0x9f, 0xe6, 0x92, 0x3c, 0xe2, 0x64, 0x62, 0x6a, 0x84, 0x9c, + 0xf0, 0xd2, 0x63, 0x4e, 0x04, 0x8b, 0xca, 0xa5, 0xc4, 0xb0, 0xd0, 0x2f, + 0x4b, 0xb5, 0x9b, 0x9a, 0xe1, 0xe3, 0x21, 0xc6, 0x42, 0xb3, 0x63, 0x88, + 0x47, 0x21, 0x3d, 0x2b, 0xc6, 0x26, 0x32, 0xff, 0xf2, 0x9e, 0x06, 0x04, + 0xe5, 0xf0, 0x9d, 0x90, 0xff, 0x69, 0x2f, 0xd1, 0xfc, 0x8f, 0x41, 0x14, + 0x3b, 0x71, 0x0e, 0xab, 0xe1, 0x27, 0xae, 0xcc, 0xe8, 0x55, 0xfb, 0xee, + 0xac, 0x8a, 0x9a, 0xae, 0x0c, 0x89, 0x68, 0x7c, 0x34, 0x12, 0x12, 0x05, + 0x59, 0x59, 0xc4, 0xf1, 0x54, 0x96, 0x12, 0xe5, 0x22, 0xe9, 0xf7, 0x5d, + 0xbe, 0x37, 0xf3, 0x43, 0xbc, 0x9a, 0x22, 0xda, 0xc0, 0x23, 0x8c, 0x50, + 0xc7, 0x33, 0xb5, 0x08, 0x73, 0xa0, 0x7b, 0xc8, 0x21, 0x6d, 0xb9, 0x46, + 0x8c, 0x80, 0x4c, 0x5a, 0xe1, 0xbe, 0xc1, 0xaa, 0xd9, 0x51, 0xad, 0xf8, + 0x6d, 0x82, 0x08, 0x12, 0x40, 0x44, 0x54, 0x27, 0x29, 0xec, 0x54, 0xb6, + 0x63, 0x12, 0xe3, 0xe3, 0xe9, 0x9f, 0xf9, 0xe3, 0xce, 0x99, 0xb9, 0xe3, + 0xbe, 0x67, 0xe4, 0xe4, 0x0a, 0x3c, 0xd9, 0xa3, 0xe1, 0x95, 0xd5, 0x07, + 0xfe, 0x95, 0x80, 0xf7, 0x3d, 0x2a, 0x26, 0xa6, 0xaa, 0xd1, 0x62, 0x67, + 0x2e, 0xf4, 0x72, 0x87, 0x3b, 0x8b, 0xb0, 0xb0, 0x30, 0xff, 0xd5, 0x00, + 0x46, 0x09, 0x3f, 0x33, 0x27, 0x67, 0x32, 0xe4, 0x66, 0x85, 0x40, 0xb2, + 0xc5, 0xf1, 0xa4, 0xaa, 0xe5, 0x53, 0x5c, 0xd6, 0x19, 0xaf, 0xe1, 0xf6, + 0x54, 0x59, 0xf6, 0x60, 0xc5, 0x49, 0x27, 0xf7, 0x2b, 0x9c, 0x99, 0x1c, + 0xa1, 0xdf, 0x90, 0xdb, 0x2d, 0xb0, 0x03, 0x24, 0xd5, 0x71, 0x25, 0xbf, + 0xae, 0xfe, 0xdb, 0xf2, 0x80, 0x52, 0xe7, 0xd9, 0x02, 0x0d, 0x98, 0xb6, + 0x7e, 0xcb, 0x0b, 0xb0, 0xd0, 0xfe, 0x22, 0x1e, 0x9e, 0x96, 0xb7, 0xbc, + 0x5c, 0xa1, 0x7c, 0xd1, 0x5d, 0x20, 0xe0, 0x3f, 0x91, 0x45, 0x8a, 0xbf, + 0x10, 0x31, 0xcc, 0x3a, 0x61, 0x44, 0x95, 0x97, 0x94, 0x90, 0x67, 0x2c, + 0x04, 0x9d, 0xab, 0x51, 0xa3, 0xaf, 0x5a, 0xaa, 0x98, 0x27, 0xa6, 0xe7, + 0x46, 0xf6, 0x99, 0x41, 0xbd, 0x32, 0xd7, 0xeb, 0xee, 0xf8, 0xbd, 0xed, + 0x0c, 0x64, 0xd8, 0x13, 0x00, 0xe5, 0xf3, 0x76, 0xb7, 0x84, 0x48, 0xcc, + 0xff, 0xef, 0x63, 0xb5, 0xb6, 0xa5, 0x24, 0xf7, 0x6f, 0x9e, 0xef, 0x17, + 0x15, 0x2b, 0xce, 0xf2, 0xa1, 0x3a, 0x9f, 0x21, 0x3e, 0xd5, 0x3f, 0x4c, + 0xfe, 0x73, 0xdf, 0xd6, 0xed, 0xba, 0x42, 0xb3, 0xb3, 0x56, 0xb3, 0x9b, + 0xaa, 0x1a, 0xfc, 0xf5, 0x91, 0xbf, 0x9b, 0xeb, 0x8e, 0x39, 0x46, 0x8e, + 0xc3, 0xa5, 0x24, 0x45, 0x06, 0x0c, 0x0c, 0x44, 0xbf, 0x3e, 0x99, 0x17, + 0xf8, 0xa4, 0x7e, 0xcb, 0x4b, 0x58, 0xea, 0xf0, 0xd6, 0x70, 0x88, 0x9b, + 0xac, 0x3b, 0xac, 0xe0, 0x76, 0xfc, 0x7e, 0x4a, 0x65, 0xa6, 0x27, 0x07, + 0x2a, 0x5c, 0xae, 0x52, 0x20, 0xa5, 0xda, 0xdb, 0xdb, 0x43, 0x25, 0x24, + 0xd4, 0xcd, 0xff, 0xd6, 0x66, 0x7f, 0xd6, 0x49, 0xb4, 0x4b, 0xd8, 0x17, + 0x53, 0x5b, 0xcb, 0xba, 0xd2, 0x62, 0x3f, 0x5d, 0xa9, 0xd5, 0x00, 0xd0, + 0x14, 0xfa, 0xe8, 0x27, 0xdf, 0x54, 0x80, 0xed, 0x9d, 0xe7, 0x89, 0xc2, + 0x7d, 0xdb, 0x5e, 0x7a, 0xdf, 0x04, 0xfb, 0xb0, 0x05, 0x5a, 0x25, 0x1b, + 0x4d, 0x7b, 0xd9, 0x35, 0x30, 0x2b, 0x04, 0x94, 0xce, 0xac, 0xb2, 0x22, + 0xfb, 0x5a, 0x5a, 0x96, 0x9a, 0xce, 0x46, 0x7d, 0xf1, 0x7f, 0x5f, 0x0e, + 0x5f, 0x69, 0x4a, 0xb5, 0x40, 0x6d, 0x9f, 0x3f, 0x59, 0xf7, 0xfc, 0x65, + 0xf1, 0xd1, 0x41, 0xd8, 0x2d, 0x47, 0x42, 0x23, 0x13, 0xb9, 0xf7, 0x65, + 0x91, 0x85, 0x61, 0x29, 0x24, 0x69, 0xa6, 0xd6, 0xb0, 0x1f, 0x9d, 0xca, + 0xb3, 0xf7, 0x6d, 0x0e, 0x04, 0x9a, 0x8e, 0xd8, 0x03, 0x68, 0x69, 0x69, + 0x4d, 0xa4, 0x7d, 0xdb, 0x0d, 0xd6, 0xb3, 0x15, 0x07, 0x92, 0x48, 0x9e, + 0xb7, 0x37, 0x11, 0x21, 0x74, 0x7b, 0x81, 0x3e, 0x5e, 0x6c, 0x51, 0xba, + 0xc5, 0x62, 0xc6, 0x7d, 0x87, 0x4e, 0x3a, 0x6f, 0x96, 0x19, 0x11, 0x41, + 0x03, 0x1c, 0xeb, 0xce, 0x8c, 0x67, 0x8d, 0xe1, 0x7e, 0xbd, 0xb5, 0xcb, + 0xe5, 0xfb, 0x0c, 0xa6, 0x21, 0x2f, 0xcd, 0xf6, 0xf6, 0x27, 0x75, 0xcb, + 0x6b, 0xb6, 0xd8, 0xfe, 0x02, 0x12, 0x6e, 0xdd, 0x45, 0x93, 0xe2, 0x57, + 0xa1, 0x8b, 0x76, 0x2b, 0x50, 0x8f, 0x6e, 0xf9, 0x7a, 0x88, 0x7f, 0x88, + 0xf8, 0x9a, 0x25, 0x87, 0x5d, 0xcf, 0x56, 0x92, 0x51, 0x04, 0x1f, 0x7b, + 0x3e, 0x7c, 0xc7, 0x53, 0x98, 0xde, 0x99, 0x6a, 0x2d, 0x87, 0x1b, 0xa6, + 0x0c, 0x6b, 0x67, 0x68, 0xfa, 0x9c, 0xe2, 0x0d, 0x19, 0x9f, 0x9e, 0x99, + 0xfc, 0x7f, 0x3a, 0xde, 0x11, 0x29, 0xd4, 0x83, 0x7b, 0x66, 0x41, 0xfa, + 0xe6, 0xf7, 0x0a, 0xdd, 0x52, 0x06, 0xb0, 0x58, 0x95, 0x3a, 0xda, 0x56, + 0x68, 0x6f, 0x9d, 0x79, 0x96, 0x55, 0x00, 0xc5, 0x85, 0x56, 0xaf, 0x5c, + 0x53, 0x69, 0x4f, 0x13, 0xe9, 0xcc, 0x0a, 0x74, 0x2d, 0x01, 0x6f, 0xfe, + 0x78, 0xc3, 0xbc, 0x1d, 0xa3, 0x49, 0xc3, 0x8f, 0xba, 0x2b, 0x7b, 0x00, + 0x24, 0x38, 0xfe, 0x4a, 0xd9, 0x2f, 0x36, 0x8e, 0x3c, 0x3f, 0xa6, 0xb3, + 0x05, 0xd2, 0x79, 0x9c, 0xa9, 0x63, 0x05, 0x9f, 0x69, 0xb1, 0x18, 0xd4, + 0x27, 0x48, 0x3a, 0x6a, 0x41, 0x65, 0x2a, 0x34, 0x80, 0x7b, 0x12, 0x69, + 0x01, 0x96, 0x65, 0xb1, 0xe0, 0x90, 0xcb, 0x7e, 0x70, 0xcd, 0xd3, 0x34, + 0xd0, 0xba, 0x87, 0x96, 0xad, 0xc1, 0x59, 0x81, 0xbc, 0x63, 0xa9, 0x22, + 0xd1, 0x8f, 0x5f, 0xa7, 0xd4, 0xea, 0x56, 0xac, 0xe6, 0x21, 0x60, 0x7c, + 0x4e, 0x3d, 0xea, 0x10, 0xe7, 0xd7, 0x68, 0x03, 0x41, 0x90, 0x6d, 0xcb, + 0x2a, 0xd0, 0x0a, 0xe1, 0x30, 0xaf, 0x2a, 0x63, 0x1e, 0x1c, 0x84, 0xf6, + 0x26, 0xbe, 0xfd, 0x39, 0x92, 0x0a, 0x89, 0x8e, 0xca, 0x60, 0x4a, 0x1b, + 0xf0, 0xc3, 0xc3, 0xe3, 0x51, 0x3c, 0x5d, 0x19, 0x01, 0x9e, 0xf7, 0x65, + 0x25, 0xb1, 0x6f, 0xda, 0xef, 0x70, 0x86, 0x8c, 0x43, 0xae, 0xc0, 0xed, + 0x6c, 0x85, 0xff, 0x66, 0x82, 0xbd, 0x8b, 0x15, 0x5c, 0xa3, 0x07, 0x66, + 0xca, 0x69, 0x3d, 0xa4, 0x35, 0xf3, 0x3f, 0xd5, 0xd9, 0x39, 0x71, 0xf2, + 0xf6, 0xb0, 0x39, 0xec, 0x08, 0x0e, 0xf9, 0x9a, 0x73, 0x79, 0xea, 0x7f, + 0xc1, 0x03, 0x4b, 0x4a, 0x66, 0x15, 0x57, 0x54, 0x86, 0x3d, 0x3b, 0x8d, + 0xd7, 0xdb, 0x3b, 0x79, 0x8d, 0xc5, 0xe3, 0xf2, 0x77, 0xbc, 0xc9, 0x51, + 0xc0, 0xa3, 0x02, 0xba, 0x93, 0xe3, 0x52, 0xff, 0xdb, 0x51, 0x74, 0xaf, + 0x87, 0xeb, 0x5b, 0x7e, 0xb5, 0x18, 0xc3, 0x5f, 0x37, 0x29, 0xa5, 0xa4, + 0x28, 0xa7, 0x8b, 0x93, 0xea, 0x8e, 0xf3, 0xca, 0x61, 0xd9, 0x4c, 0xae, + 0xc9, 0x1c, 0xba, 0xe6, 0x5c, 0x5b, 0xc0, 0x42, 0x85, 0x06, 0xcc, 0xd6, + 0x30, 0x95, 0x35, 0xbc, 0x69, 0xba, 0x80, 0xbb, 0x6a, 0x05, 0x66, 0x7f, + 0x5f, 0xcd, 0x46, 0xa1, 0xa7, 0x11, 0x51, 0xf3, 0x2b, 0x30, 0xec, 0xc4, + 0xeb, 0x2d, 0x35, 0xbc, 0x36, 0xee, 0xd5, 0x1d, 0x5a, 0xc5, 0x9c, 0xe5, + 0xb3, 0x2a, 0xcb, 0x67, 0xad, 0xc5, 0x35, 0xf2, 0xe7, 0x36, 0x0a, 0xde, + 0xe7, 0x6a, 0x45, 0x87, 0x42, 0xc9, 0x74, 0x01, 0xd4, 0xb8, 0x26, 0x2b, + 0x55, 0xec, 0x1b, 0x41, 0xa2, 0x51, 0x22, 0x6e, 0x03, 0x51, 0xc7, 0x4a, + 0x07, 0x3e, 0x0d, 0x0f, 0x84, 0x49, 0x0a, 0xeb, 0x0c, 0x37, 0x25, 0xfe, + 0xf0, 0x93, 0x9a, 0xee, 0xf7, 0xce, 0xe9, 0x1a, 0xdd, 0xb6, 0x7a, 0x21, + 0x09, 0x1c, 0x9c, 0xef, 0x3e, 0x68, 0x58, 0x68, 0x63, 0x03, 0x1a, 0x39, + 0x26, 0x73, 0x87, 0xbd, 0x84, 0x52, 0x72, 0xe0, 0x3c, 0xf0, 0x10, 0x3a, + 0xfe, 0x8e, 0x8e, 0x10, 0xab, 0xff, 0x95, 0x15, 0x26, 0x95, 0x47, 0xb1, + 0xe1, 0xab, 0x25, 0x5d, 0x95, 0xf8, 0x6d, 0x7e, 0x7b, 0x39, 0xd2, 0xb5, + 0xf1, 0x15, 0xbc, 0xc2, 0xc5, 0xfa, 0xcb, 0x26, 0xe6, 0x08, 0x03, 0x8a, + 0x83, 0x6f, 0x05, 0xe5, 0x37, 0xfb, 0x6b, 0x49, 0x7a, 0x35, 0xda, 0x58, + 0x4f, 0x34, 0xdd, 0x83, 0x06, 0x2d, 0x17, 0x3d, 0x96, 0xa7, 0x6f, 0x34, + 0x9b, 0xc7, 0x9c, 0x21, 0x0f, 0x9b, 0xfe, 0x1f, 0x7f, 0xa7, 0x32, 0x39, + 0xad, 0x17, 0xe1, 0xaa, 0x5d, 0x25, 0x6b, 0x31, 0x22, 0x18, 0x43, 0x9b, + 0x61, 0x33, 0xb8, 0x2a, 0x07, 0x71, 0x8c, 0x73, 0x4c, 0xb7, 0xaf, 0xef, + 0x1a, 0x8f, 0x04, 0xc3, 0x35, 0x36, 0x73, 0x22, 0x0d, 0x71, 0x2d, 0xb4, + 0x77, 0x8c, 0xb1, 0x5d, 0x99, 0xf2, 0x71, 0xf0, 0x98, 0x97, 0xcc, 0x82, + 0x61, 0xdf, 0xc7, 0x59, 0x63, 0x3a, 0xa1, 0xe6, 0xd0, 0xd0, 0x4f, 0x31, + 0x29, 0x3e, 0x5e, 0x48, 0x31, 0x24, 0x90, 0x49, 0x69, 0x96, 0x7c, 0x9d, + 0x9e, 0x0b, 0x44, 0xe0, 0xf5, 0xe6, 0xa0, 0x9d, 0x84, 0x63, 0xfc, 0xcb, + 0x69, 0xf0, 0x06, 0x6b, 0xff, 0x36, 0x97, 0xaa, 0xbf, 0x5d, 0x0d, 0x93, + 0x94, 0x68, 0x35, 0x98, 0x99, 0x0b, 0x2c, 0xda, 0xb5, 0xbc, 0xb8, 0x67, + 0x63, 0x8d, 0x08, 0xc3, 0x0a, 0xef, 0xc3, 0xcf, 0x92, 0x46, 0x55, 0x3d, + 0xe4, 0x19, 0x9e, 0xe1, 0x23, 0x92, 0xb8, 0x90, 0x40, 0xf4, 0x38, 0xed, + 0x7e, 0xdd, 0xa8, 0x27, 0xfb, 0xf4, 0xea, 0x35, 0xc5, 0x4e, 0xe9, 0x61, + 0x2d, 0x37, 0x79, 0x3f, 0x0c, 0x87, 0xdc, 0x25, 0x2c, 0x69, 0x19, 0x0e, + 0xa0, 0x67, 0x67, 0x41, 0xb7, 0xe1, 0x71, 0x6e, 0x78, 0xcd, 0xaf, 0xff, + 0x71, 0x2f, 0x49, 0xf6, 0x43, 0x4d, 0xad, 0xde, 0x79, 0xed, 0x35, 0xab, + 0x33, 0xe7, 0xcf, 0xee, 0x32, 0xe1, 0xf4, 0x58, 0x2f, 0xab, 0x5c, 0x7e, + 0x01, 0xf9, 0xbf, 0xb6, 0x07, 0xd4, 0x6f, 0xfb, 0xe4, 0xa5, 0xe8, 0x38, + 0x72, 0x0b, 0xd2, 0xdf, 0x06, 0x06, 0xc5, 0xe2, 0xc1, 0x57, 0xca, 0x84, + 0xf2, 0x01, 0xe5, 0xf4, 0xc1, 0x5f, 0xf9, 0x46, 0xa7, 0x5c, 0x13, 0xb3, + 0xb3, 0xe5, 0x2d, 0x2c, 0x0e, 0xc5, 0xeb, 0x8e, 0x95, 0xa5, 0xd5, 0x6b, + 0xf4, 0x27, 0x3b, 0x3d, 0x6f, 0x3c, 0x9e, 0x8f, 0x4a, 0x96, 0xdb, 0x5d, + 0x4f, 0xa4, 0x86, 0x39, 0x6d, 0x57, 0x92, 0xf6, 0xc7, 0x53, 0x8d, 0x1b, + 0x96, 0xe5, 0x8f, 0x69, 0x6d, 0x06, 0x09, 0xcd, 0x0e, 0x43, 0x85, 0x1f, + 0x32, 0x57, 0xff, 0xf3, 0xa6, 0x5d, 0x8f, 0x9b, 0xd3, 0xe8, 0x9d, 0xc9, + 0x4c, 0x2b, 0xd9, 0xc3, 0x1d, 0x4c, 0x32, 0x28, 0x04, 0xf8, 0x5e, 0x4c, + 0xcb, 0xcb, 0x32, 0x75, 0xab, 0x89, 0x3b, 0xfc, 0xd9, 0x4c, 0x39, 0x41, + 0x60, 0x89, 0x92, 0xfe, 0x29, 0x6f, 0x16, 0x21, 0xe1, 0xf1, 0x42, 0xaf, + 0xf7, 0x68, 0xda, 0xf2, 0x7c, 0x87, 0x53, 0x0f, 0x6c, 0x19, 0xf5, 0xf7, + 0xd1, 0xc8, 0x26, 0x9d, 0x47, 0xc7, 0xa3, 0x0c, 0x72, 0x3d, 0xa1, 0x62, + 0x9f, 0xcf, 0xd0, 0xc4, 0xad, 0x59, 0xd6, 0x19, 0x9c, 0x94, 0xbc, 0xf4, + 0x7f, 0x6f, 0x26, 0x29, 0x55, 0x02, 0xc4, 0x19, 0xb8, 0x59, 0x2d, 0x06, + 0x60, 0x10, 0x89, 0x78, 0x93, 0x34, 0xf4, 0x3d, 0xf7, 0xf3, 0xc7, 0xca, + 0x6b, 0x87, 0xeb, 0x4d, 0xc7, 0xb1, 0x17, 0xab, 0xf5, 0x64, 0x5c, 0x5c, + 0x38, 0xa4, 0xa4, 0xa4, 0x06, 0x06, 0x07, 0xb3, 0xf6, 0x66, 0x5f, 0x5f, + 0x6f, 0xa6, 0x5d, 0x0e, 0x67, 0x7e, 0x2b, 0xe8, 0x93, 0xc8, 0x36, 0x0c, + 0xac, 0x0c, 0x6e, 0x60, 0xab, 0xa9, 0xe1, 0xf2, 0xbc, 0x33, 0x74, 0xa5, + 0x77, 0x7c, 0x16, 0x87, 0xfb, 0xe8, 0xe7, 0x60, 0xcf, 0xd4, 0xba, 0x6d, + 0x0a, 0xf7, 0x43, 0x17, 0x11, 0xa9, 0x26, 0xa3, 0x01, 0x6b, 0x70, 0x54, + 0xcd, 0xdd, 0xb5, 0xfb, 0x39, 0x40, 0xfd, 0x78, 0xd3, 0x5e, 0x78, 0x32, + 0x7b, 0x45, 0x78, 0x9f, 0xa0, 0x89, 0xcc, 0xdf, 0xba, 0xf2, 0x48, 0x6f, + 0xb1, 0xc0, 0xea, 0x1f, 0xf6, 0x7d, 0x8f, 0x5d, 0x2d, 0xb0, 0x54, 0xad, + 0xd4, 0x8c, 0x8f, 0x8b, 0x43, 0xed, 0xd4, 0x67, 0x71, 0xeb, 0x20, 0xa2, + 0xa3, 0x6a, 0xaf, 0x36, 0xce, 0xa0, 0xe6, 0xe1, 0xe1, 0x69, 0xd5, 0x65, + 0x66, 0x63, 0xff, 0x58, 0x9e, 0x77, 0x34, 0x07, 0x37, 0x23, 0x1e, 0x83, + 0xa2, 0x49, 0x4b, 0x76, 0x3b, 0x93, 0x57, 0x59, 0xe5, 0x32, 0x5c, 0x3d, + 0x03, 0x21, 0x5a, 0x96, 0x98, 0xa1, 0xdd, 0x5e, 0x91, 0xe8, 0x72, 0xb2, + 0x18, 0x9d, 0xce, 0x6e, 0x7e, 0xbd, 0xac, 0x33, 0x1d, 0xa7, 0x5a, 0x63, + 0x16, 0xe1, 0xc2, 0x1f, 0x6a, 0xe9, 0x74, 0x96, 0x15, 0x34, 0xea, 0xba, + 0x9f, 0x57, 0x1e, 0x9f, 0x88, 0x44, 0x51, 0xa0, 0x00, 0xf4, 0x7a, 0xf9, + 0xd7, 0xde, 0x54, 0x3e, 0x68, 0x12, 0x75, 0x88, 0x4a, 0xb0, 0x9a, 0x55, + 0xf0, 0xb3, 0xc4, 0xb8, 0x41, 0xcd, 0xb7, 0xef, 0x64, 0x9d, 0x5b, 0x90, + 0x91, 0x32, 0xfc, 0x88, 0x04, 0x91, 0x82, 0x99, 0x7a, 0xcc, 0x60, 0x91, + 0x96, 0x65, 0xba, 0x7f, 0xef, 0x48, 0xe8, 0x6a, 0x3b, 0xd5, 0xf9, 0x86, + 0x2f, 0x14, 0x55, 0x44, 0x54, 0xf4, 0x68, 0xa9, 0x86, 0x44, 0x34, 0x0c, + 0x99, 0xe9, 0x1a, 0x38, 0x14, 0x4b, 0x32, 0x0d, 0x51, 0xb2, 0xb1, 0x78, + 0x8a, 0x9f, 0xac, 0x72, 0x89, 0x0f, 0xa1, 0x0f, 0xbb, 0xc7, 0x66, 0x77, + 0x6f, 0x20, 0x45, 0xf0, 0xa7, 0xff, 0x6c, 0xd0, 0x3e, 0x1c, 0xc4, 0xd0, + 0x0b, 0xf9, 0xfd, 0x85, 0xf4, 0x60, 0xe9, 0x55, 0xea, 0x98, 0x40, 0x47, + 0x84, 0x12, 0x50, 0xc4, 0xf0, 0xee, 0x36, 0x98, 0x45, 0x9f, 0xa6, 0xd8, + 0xc7, 0x24, 0x62, 0x5d, 0x46, 0x09, 0x13, 0x7f, 0x73, 0x4e, 0xe3, 0x53, + 0xdf, 0x76, 0xa9, 0x0a, 0x0d, 0x8b, 0xfd, 0xaa, 0x5b, 0xd4, 0x10, 0xf6, + 0xec, 0x6c, 0xa3, 0xf6, 0xe1, 0xf3, 0xe6, 0xbf, 0x33, 0xb5, 0xf3, 0x4b, + 0x0f, 0x48, 0x33, 0xcb, 0xf7, 0x12, 0x37, 0xf3, 0x7f, 0x85, 0xd7, 0xd7, + 0x7b, 0x60, 0x38, 0x5a, 0x91, 0xe3, 0x7b, 0x9c, 0x6e, 0xb0, 0x7f, 0xaa, + 0xf2, 0xea, 0xf6, 0x7b, 0xf3, 0x00, 0x67, 0xb9, 0xcb, 0x3b, 0x19, 0x4e, + 0x0f, 0xee, 0xdf, 0x79, 0xf0, 0x73, 0x58, 0x2f, 0xfe, 0x89, 0x26, 0xe0, + 0x46, 0xdf, 0x6b, 0x6c, 0xb0, 0x38, 0xc6, 0x25, 0x74, 0x5b, 0xe5, 0xc1, + 0x68, 0x3a, 0xa1, 0xe8, 0x97, 0xda, 0xf0, 0x4f, 0x43, 0x22, 0xe0, 0x8a, + 0x4b, 0x4b, 0xd3, 0xa4, 0x37, 0xfd, 0x9b, 0x9c, 0x2c, 0x2e, 0x16, 0x5c, + 0x83, 0x3f, 0x3e, 0x96, 0x7e, 0xb1, 0xc3, 0xd8, 0x2b, 0x00, 0x00, 0xb0, + 0xfb, 0x3d, 0x26, 0x6b, 0x6c, 0xbc, 0x75, 0xbf, 0x3e, 0xe8, 0xf0, 0x89, + 0x2c, 0xb5, 0x98, 0x6b, 0xab, 0x32, 0x83, 0xc4, 0xdc, 0x01, 0x9e, 0xa5, + 0x56, 0x13, 0x89, 0xa1, 0xf1, 0xd7, 0x6e, 0xab, 0x15, 0xe8, 0x9e, 0xee, + 0x8f, 0x18, 0xef, 0xf8, 0x66, 0x22, 0x3d, 0x72, 0x0f, 0x02, 0x17, 0x0a, + 0x3a, 0x98, 0x38, 0x38, 0xc3, 0x79, 0xa2, 0x61, 0x91, 0x4d, 0x5f, 0x60, + 0x90, 0x09, 0x73, 0xa5, 0x93, 0xe9, 0xdf, 0x2f, 0xc6, 0xcd, 0x3b, 0x45, + 0x1d, 0x14, 0x67, 0xd7, 0x9f, 0x9a, 0x32, 0xdd, 0x66, 0xc0, 0xe9, 0xdf, + 0x5c, 0x8e, 0xe7, 0x23, 0xda, 0x9c, 0xc1, 0x29, 0xd2, 0x74, 0x42, 0x10, + 0xbd, 0x95, 0x4d, 0x63, 0xcd, 0x1b, 0x58, 0xbb, 0x57, 0xf1, 0x44, 0xcd, + 0xfa, 0xb8, 0x21, 0x87, 0x41, 0x2e, 0x91, 0x56, 0x8b, 0xe6, 0x37, 0xc1, + 0xc5, 0x4b, 0x0e, 0x0a, 0xb2, 0x27, 0xe5, 0x7e, 0xc0, 0x89, 0x9a, 0x2d, + 0x11, 0xee, 0x91, 0xaf, 0xb1, 0xd4, 0x16, 0x11, 0x1c, 0x2d, 0x58, 0x24, + 0x90, 0x18, 0x5b, 0x7d, 0x3b, 0xef, 0x90, 0xeb, 0xba, 0x99, 0xdb, 0xf5, + 0x4b, 0x90, 0xa1, 0x6b, 0x2b, 0xb7, 0xf6, 0x17, 0x1a, 0x85, 0x98, 0x77, + 0xcd, 0x53, 0xca, 0x54, 0x65, 0xe5, 0x36, 0x27, 0x53, 0xc4, 0xfe, 0x5f, + 0x10, 0x1d, 0x05, 0x34, 0x76, 0x19, 0x93, 0x12, 0xc7, 0xb6, 0x58, 0xc8, + 0x68, 0x53, 0x71, 0xd8, 0xaa, 0x22, 0xa8, 0xa9, 0xa5, 0x71, 0xdb, 0x13, + 0x5b, 0x5a, 0x5a, 0x16, 0xc9, 0x4f, 0xe9, 0xd7, 0xa4, 0xc2, 0x61, 0x50, + 0x03, 0x14, 0x85, 0xf2, 0xac, 0x22, 0x58, 0x16, 0x98, 0x34, 0x93, 0xb9, + 0xfd, 0xaf, 0x69, 0x6d, 0x32, 0xe8, 0x11, 0x29, 0x0a, 0x12, 0x8a, 0xd5, + 0xda, 0x97, 0xc9, 0x3d, 0xf1, 0x28, 0xbd, 0x24, 0x47, 0xce, 0x9e, 0xee, + 0x06, 0x17, 0x18, 0xfb, 0x25, 0xed, 0x5f, 0x4f, 0x0b, 0x80, 0x1e, 0x8f, + 0xd5, 0x31, 0xef, 0x4d, 0xc7, 0xed, 0x86, 0x73, 0xaf, 0xcd, 0x7e, 0xad, + 0x37, 0x3e, 0x0b, 0xa2, 0x10, 0x43, 0x3b, 0x4c, 0x11, 0xcb, 0xaf, 0x7c, + 0x2a, 0xad, 0xed, 0xe1, 0xf7, 0x94, 0xe5, 0xbe, 0x41, 0xc6, 0xc9, 0x89, + 0x25, 0xae, 0x74, 0x38, 0x6f, 0x4e, 0xdb, 0xfc, 0x9a, 0x61, 0x61, 0x7e, + 0xd3, 0x52, 0x98, 0xd4, 0x68, 0xa5, 0x4c, 0x12, 0x40, 0xc4, 0xe7, 0x96, + 0xc4, 0xab, 0x65, 0xbf, 0xda, 0x8c, 0x14, 0xe0, 0x56, 0x95, 0x58, 0x9b, + 0xf3, 0xd3, 0xc2, 0xa9, 0x0b, 0x92, 0x0b, 0x24, 0xc5, 0x9d, 0x5f, 0x49, + 0x5d, 0xdf, 0xb6, 0x1e, 0xd0, 0xd0, 0xb2, 0xc6, 0xd4, 0x4a, 0x67, 0x36, + 0x8d, 0x5c, 0xb4, 0x9c, 0xd7, 0x43, 0x1c, 0xaa, 0x13, 0xe7, 0xb2, 0x38, + 0xb6, 0x7e, 0xaf, 0x7a, 0x96, 0x2e, 0x31, 0xe1, 0x86, 0xc2, 0x49, 0xba, + 0xac, 0xe7, 0xaa, 0xb5, 0xfc, 0x7e, 0x2f, 0x6b, 0x74, 0x3d, 0x59, 0x8c, + 0x4f, 0x4a, 0x72, 0x99, 0xad, 0x2f, 0x37, 0x33, 0x35, 0x45, 0xc3, 0xc3, + 0x93, 0x1a, 0x76, 0x63, 0xf2, 0x80, 0xe0, 0xd5, 0x54, 0x06, 0x5b, 0x59, + 0x6e, 0x81, 0xc5, 0xca, 0xbe, 0x23, 0xbd, 0x6a, 0xb8, 0xf1, 0xc7, 0xfb, + 0x5b, 0x2a, 0x3f, 0xb4, 0xcd, 0xef, 0x96, 0x9d, 0xcc, 0xfb, 0x6b, 0x5b, + 0x85, 0x37, 0xd7, 0xe7, 0xef, 0x10, 0x27, 0x8e, 0x03, 0x24, 0x32, 0xa6, + 0x99, 0xdc, 0xba, 0xc1, 0x13, 0x82, 0x16, 0xde, 0xed, 0x31, 0xb5, 0x75, + 0xa1, 0x93, 0xc1, 0xd6, 0x0b, 0x1f, 0x99, 0x55, 0x6e, 0x4f, 0x3c, 0xbf, + 0xd8, 0xf8, 0xfd, 0xf3, 0xfc, 0x38, 0xad, 0xbb, 0x5e, 0x36, 0xbf, 0x8c, + 0x5e, 0x01, 0xf8, 0x1c, 0x9f, 0xb3, 0x27, 0x93, 0xca, 0x0c, 0x2c, 0xa5, + 0x84, 0x31, 0x90, 0x48, 0x89, 0xc0, 0xa0, 0x61, 0x73, 0xef, 0x95, 0xc6, + 0x66, 0x36, 0x30, 0x30, 0x51, 0x8e, 0xf6, 0x3b, 0x37, 0x44, 0x26, 0xfc, + 0xfe, 0x03, 0x72, 0x8e, 0xb7, 0xbc, 0xe1, 0xa2, 0xe4, 0xba, 0x5f, 0xaf, + 0x1e, 0x06, 0xf0, 0xc7, 0x29, 0x72, 0xc7, 0xe8, 0xe8, 0x7b, 0xad, 0x94, + 0x67, 0x1e, 0xe7, 0xb3, 0xd9, 0x4f, 0x78, 0x96, 0x6b, 0x0a, 0xe5, 0x56, + 0x0b, 0x69, 0xa5, 0xa5, 0x04, 0x8c, 0xea, 0x39, 0x9f, 0xac, 0x48, 0x05, + 0x64, 0x75, 0x09, 0xf7, 0xf6, 0xc4, 0xf8, 0x19, 0xf6, 0xaf, 0x52, 0xed, + 0xec, 0x56, 0x65, 0xbb, 0xd8, 0x1e, 0xfd, 0xfc, 0xc7, 0x4c, 0x33, 0x8b, + 0xd9, 0x7c, 0xd6, 0x70, 0x5e, 0x6b, 0x4d, 0x38, 0x01, 0xce, 0x07, 0x13, + 0xdb, 0x43, 0x04, 0xf6, 0x8c, 0xdf, 0x12, 0x72, 0x8e, 0xda, 0x21, 0x4d, + 0x84, 0x0a, 0x62, 0x97, 0x05, 0x41, 0x60, 0x96, 0x83, 0xf6, 0x94, 0x68, + 0x3a, 0x12, 0x2c, 0xbf, 0x45, 0xae, 0xf9, 0x72, 0xfd, 0x88, 0xa6, 0x4c, + 0x77, 0x56, 0xe7, 0x29, 0x9f, 0x66, 0xfb, 0x4d, 0xf9, 0x87, 0xf3, 0x75, + 0x96, 0xb2, 0x4a, 0x6e, 0xfb, 0x8d, 0x89, 0xaa, 0x96, 0x0d, 0xe9, 0xaa, + 0x47, 0x7a, 0x59, 0x5f, 0xb0, 0x27, 0x73, 0xe1, 0xa7, 0xc9, 0x90, 0x1f, + 0xdc, 0x88, 0x7d, 0x22, 0x41, 0x5a, 0x8d, 0x46, 0x02, 0xbf, 0x74, 0xb7, + 0x9d, 0x6c, 0x5f, 0x56, 0xf2, 0x5d, 0x01, 0x3a, 0x2e, 0xe2, 0x65, 0x81, + 0xab, 0xeb, 0x0d, 0x7f, 0x93, 0xd2, 0x0b, 0x8c, 0xc6, 0x7f, 0x8e, 0x2d, + 0xe5, 0xf8, 0x4a, 0x42, 0xd0, 0xc4, 0xef, 0xa4, 0xc5, 0x7a, 0xd3, 0xd0, + 0xec, 0xb5, 0x16, 0x31, 0x00, 0x8f, 0xdd, 0x6c, 0xa6, 0x48, 0x26, 0x15, + 0x9d, 0xcc, 0xf3, 0x77, 0xd1, 0xe4, 0xd4, 0xd4, 0xc1, 0x76, 0x18, 0x09, + 0x84, 0x54, 0xb2, 0x63, 0x88, 0xf8, 0xe6, 0x20, 0xbc, 0x83, 0x51, 0x5f, + 0xdf, 0xf9, 0xb0, 0xe1, 0x0d, 0x58, 0xe3, 0xb3, 0x70, 0x89, 0x97, 0xd8, + 0x76, 0xed, 0xff, 0x35, 0x96, 0x56, 0x52, 0x86, 0xab, 0x12, 0x53, 0xa6, + 0x52, 0x72, 0xbd, 0xf9, 0xf1, 0xfe, 0x90, 0x30, 0xfa, 0x0c, 0x7c, 0xd8, + 0x28, 0x30, 0xec, 0x0f, 0xd5, 0x55, 0x2a, 0x60, 0x63, 0xa7, 0xe9, 0x72, + 0x91, 0x72, 0x7b, 0xc0, 0xba, 0x67, 0x3c, 0x3f, 0x9d, 0x4f, 0x19, 0xb9, + 0x54, 0x2c, 0x9c, 0x00, 0x0a, 0x70, 0xdc, 0xb6, 0x5f, 0x3b, 0x3a, 0x8a, + 0x32, 0x9a, 0xc1, 0xb8, 0xfd, 0xa9, 0x2e, 0xfa, 0x4d, 0xdc, 0x76, 0x9a, + 0x59, 0x97, 0xff, 0x4b, 0x87, 0xe2, 0xb1, 0x3c, 0x84, 0xff, 0x72, 0xf3, + 0x5b, 0xe5, 0xca, 0x25, 0x0e, 0x4b, 0xf7, 0x2b, 0x5b, 0x7f, 0x28, 0x12, + 0x43, 0x5d, 0xab, 0x1a, 0x6c, 0xaf, 0x8a, 0x82, 0x80, 0xeb, 0x50, 0xa3, + 0xac, 0xf1, 0xd0, 0x9f, 0xed, 0xb2, 0x92, 0xec, 0x7c, 0xa3, 0xc1, 0x48, + 0x42, 0x41, 0x9f, 0x34, 0x43, 0x68, 0x2f, 0xd0, 0xd9, 0x81, 0x75, 0xa4, + 0xcb, 0x1b, 0xf4, 0xcf, 0x98, 0xd9, 0xb6, 0x58, 0x7b, 0xa8, 0x69, 0xf7, + 0x4d, 0xdf, 0x9f, 0xe0, 0xa9, 0x2c, 0x96, 0xf2, 0x60, 0xf0, 0x44, 0x7a, + 0xbf, 0x6d, 0xcb, 0x7a, 0x68, 0x3e, 0x6d, 0x65, 0xe6, 0x03, 0xd4, 0x01, + 0x55, 0xfd, 0xde, 0x47, 0xca, 0xf8, 0x05, 0x41, 0xf3, 0xd5, 0x61, 0x9c, + 0x63, 0xc0, 0xae, 0xc0, 0x08, 0xc1, 0x2f, 0x25, 0xbe, 0x9e, 0xf0, 0x73, + 0x97, 0xe6, 0x81, 0x33, 0xfd, 0x88, 0x72, 0x09, 0x5f, 0x6b, 0xdc, 0xa3, + 0x75, 0x73, 0x9f, 0x4b, 0x74, 0xb8, 0x05, 0x86, 0xec, 0xdc, 0xcf, 0xab, + 0xee, 0x24, 0xb9, 0xe6, 0xdd, 0xf1, 0x8d, 0xd6, 0x4b, 0x31, 0x21, 0x08, + 0x38, 0x29, 0x9a, 0x3a, 0x5c, 0xb6, 0x2b, 0x62, 0x0d, 0x66, 0x93, 0x23, + 0xe3, 0x7c, 0xbe, 0xc9, 0xf1, 0x19, 0x9c, 0xd6, 0x09, 0x19, 0x19, 0x63, + 0x7a, 0xee, 0xeb, 0xe2, 0x90, 0x9e, 0x5e, 0x20, 0x11, 0x2d, 0x02, 0x52, + 0x55, 0x9f, 0x26, 0x36, 0x30, 0xe9, 0xab, 0xdc, 0xac, 0x4f, 0xf4, 0x63, + 0x89, 0xa6, 0xe1, 0x1b, 0x1e, 0x7c, 0xbe, 0x3b, 0xb1, 0x5c, 0x6b, 0xbd, + 0x12, 0x8f, 0xbf, 0x83, 0x28, 0xb1, 0xdc, 0xd9, 0x4a, 0x97, 0xeb, 0xe9, + 0x05, 0xb2, 0xff, 0x5f, 0xae, 0x49, 0x82, 0xe4, 0x8c, 0x64, 0x2f, 0xb7, + 0xcc, 0x21, 0x68, 0x85, 0x58, 0xc3, 0x85, 0x67, 0x22, 0xbb, 0x99, 0x2a, + 0x36, 0xb9, 0xdf, 0x07, 0xed, 0x1d, 0xeb, 0x0c, 0x4a, 0x49, 0xc7, 0xdc, + 0xf8, 0x12, 0x5e, 0x0f, 0xc3, 0x5e, 0xee, 0x79, 0x6e, 0xaf, 0x11, 0x99, + 0x99, 0x84, 0x6e, 0x04, 0x17, 0x7e, 0x5c, 0x08, 0x99, 0x93, 0xea, 0x46, + 0x90, 0x7a, 0xde, 0xb9, 0xf3, 0xb2, 0x9c, 0x2f, 0x9d, 0xc5, 0x93, 0x42, + 0xae, 0xb5, 0x5e, 0x62, 0xd6, 0x6e, 0xb1, 0x9b, 0x1c, 0x88, 0xc0, 0x38, + 0x02, 0x95, 0xc4, 0x4e, 0x33, 0x2d, 0x21, 0x6d, 0x7c, 0xec, 0x74, 0x90, + 0x6f, 0xd6, 0x38, 0x02, 0x33, 0x5e, 0xaf, 0xfd, 0x0b, 0xca, 0xf4, 0xdb, + 0x3e, 0x41, 0xb4, 0xc6, 0x66, 0xb9, 0xbe, 0x68, 0x60, 0x9f, 0xdf, 0x72, + 0x8e, 0x52, 0x68, 0x94, 0x04, 0x58, 0xc2, 0xbe, 0xd4, 0x2c, 0xc9, 0xa8, + 0x36, 0xee, 0x78, 0xc3, 0xaf, 0xff, 0xbc, 0xe9, 0x4f, 0x90, 0xcb, 0xa7, + 0xc6, 0xd9, 0x1c, 0x2b, 0x7e, 0x06, 0xa0, 0xf0, 0x3a, 0x59, 0x14, 0x82, + 0x03, 0xad, 0xdd, 0x24, 0x82, 0x0c, 0x53, 0x51, 0x36, 0x17, 0xae, 0x82, + 0x95, 0x25, 0x95, 0xa5, 0x13, 0x50, 0x08, 0xb7, 0x5b, 0x38, 0x33, 0x5b, + 0xaf, 0x86, 0x08, 0x64, 0x38, 0xe4, 0x7d, 0x8e, 0xcb, 0x4a, 0x22, 0x7d, + 0x5e, 0x51, 0x84, 0x5e, 0x01, 0x3d, 0xb1, 0x42, 0xef, 0x23, 0x90, 0xd8, + 0x7e, 0xed, 0xc9, 0xd1, 0x71, 0xcf, 0x2b, 0xb3, 0x8a, 0xbf, 0x00, 0xd7, + 0xae, 0xa6, 0xbd, 0xf7, 0xb2, 0xf6, 0x89, 0x06, 0x59, 0xce, 0xe4, 0x42, + 0x61, 0x50, 0xcb, 0x6e, 0xfd, 0xab, 0x33, 0xa6, 0x89, 0xc6, 0xd2, 0xa5, + 0xed, 0x7a, 0xbd, 0x96, 0x09, 0x46, 0xe5, 0x5f, 0x79, 0x40, 0x6b, 0x5f, + 0x77, 0x42, 0xca, 0xf4, 0x7b, 0x4d, 0x83, 0x4c, 0x8e, 0xce, 0xb7, 0xba, + 0x0c, 0x55, 0x77, 0xbd, 0x11, 0x0a, 0xb9, 0x43, 0x0e, 0x5f, 0xe2, 0xb8, + 0x30, 0x2e, 0x0e, 0xa6, 0xef, 0xef, 0x14, 0x36, 0x76, 0x3f, 0x74, 0xc8, + 0x21, 0x83, 0xba, 0xcb, 0xcb, 0xd7, 0xad, 0x93, 0x2b, 0xd1, 0x10, 0x65, + 0xda, 0x54, 0xbc, 0xc7, 0x7b, 0x17, 0x7f, 0x76, 0x77, 0x38, 0xc0, 0xdb, + 0xf3, 0x1d, 0xc3, 0xac, 0x9b, 0xcf, 0xcb, 0x59, 0x0b, 0xd8, 0x67, 0x6a, + 0x6f, 0x34, 0xf1, 0x8d, 0x68, 0x20, 0xb4, 0x8a, 0xaf, 0xce, 0xf2, 0x98, + 0x87, 0xae, 0xb9, 0x98, 0x9e, 0xf1, 0xc9, 0x8c, 0x31, 0xaf, 0xb7, 0xeb, + 0x5a, 0x6a, 0x76, 0x76, 0x56, 0x4c, 0x42, 0xa2, 0xde, 0x71, 0x57, 0xa2, + 0xbe, 0xbe, 0x7e, 0xea, 0x84, 0xc7, 0x45, 0xcf, 0xfc, 0x64, 0xb1, 0x0a, + 0x42, 0x10, 0xb8, 0x99, 0x93, 0x12, 0x17, 0x1b, 0x40, 0x4b, 0xdd, 0x1a, + 0x6a, 0x02, 0xff, 0xd7, 0x34, 0xc3, 0x37, 0xef, 0x02, 0x82, 0x21, 0xaa, + 0x9c, 0x15, 0x3e, 0xa2, 0x7a, 0x27, 0xdd, 0xb5, 0xe5, 0xb6, 0x2a, 0x80, + 0xca, 0xc1, 0x9e, 0xcf, 0x53, 0xd0, 0x2a, 0x70, 0xcc, 0xc8, 0x83, 0x14, + 0x75, 0xdd, 0x2e, 0x36, 0x36, 0x8e, 0xad, 0x83, 0x8f, 0xb4, 0x42, 0xfd, + 0x8f, 0xe0, 0xd1, 0x45, 0x46, 0x79, 0xfd, 0xfe, 0x05, 0xf0, 0x89, 0xb9, + 0xab, 0x43, 0x6c, 0xd5, 0xe9, 0xf1, 0x72, 0xbb, 0x81, 0x98, 0xdd, 0xef, + 0xd5, 0xa3, 0x30, 0xc9, 0xe7, 0xf5, 0x6a, 0x18, 0x50, 0xa9, 0xd7, 0x91, + 0x02, 0xc9, 0x8e, 0x47, 0xbd, 0xe9, 0xd1, 0xcd, 0x77, 0x0f, 0x79, 0x4a, + 0x74, 0x91, 0x80, 0xa4, 0xca, 0x8c, 0x37, 0xe2, 0xdd, 0x50, 0x81, 0x50, + 0x90, 0x21, 0xf8, 0xf1, 0x6a, 0x97, 0x92, 0x9a, 0x1a, 0x1d, 0x0d, 0xed, + 0x37, 0x6a, 0x20, 0xe4, 0xb0, 0x5a, 0xdf, 0x17, 0x37, 0x08, 0x01, 0x28, + 0xe0, 0xa9, 0xa7, 0x15, 0x4b, 0x99, 0x99, 0x51, 0xbf, 0x3c, 0x5c, 0x98, + 0xdf, 0x1e, 0xce, 0xfe, 0xb3, 0xe1, 0x3a, 0x5e, 0xaa, 0x99, 0x86, 0x90, + 0x9e, 0x5c, 0x72, 0xab, 0xc5, 0xfb, 0xab, 0x21, 0xb9, 0xca, 0x86, 0x57, + 0x28, 0x7f, 0x6c, 0xcb, 0xfc, 0x49, 0x50, 0x52, 0x02, 0x0d, 0xc9, 0x69, + 0x28, 0x79, 0x30, 0xc0, 0xfe, 0xa4, 0xb6, 0x96, 0xc0, 0x4a, 0x1b, 0x13, + 0x05, 0x90, 0xef, 0xcd, 0x0f, 0xb3, 0x04, 0x05, 0x21, 0xfa, 0xa0, 0xef, + 0x0b, 0x5c, 0x8a, 0xe2, 0x4a, 0x92, 0xe3, 0x97, 0xb6, 0x36, 0x91, 0xb1, + 0xb1, 0x2c, 0x8e, 0x4b, 0x57, 0x36, 0xb4, 0xe1, 0xa3, 0x0c, 0xaa, 0xa5, + 0x80, 0x1f, 0xca, 0xca, 0xe8, 0x01, 0xdd, 0x69, 0xdb, 0xf5, 0x9c, 0x99, + 0xd3, 0x3c, 0xe7, 0x58, 0x22, 0x5b, 0x57, 0xc7, 0x8d, 0x40, 0xfa, 0x78, + 0xb5, 0x86, 0x61, 0x95, 0xba, 0xbe, 0x88, 0xbb, 0x09, 0xbf, 0xb7, 0x3b, + 0x8d, 0xe3, 0xf9, 0x32, 0xd1, 0x36, 0xa7, 0x7d, 0x07, 0xde, 0xac, 0xf7, + 0xa6, 0x9d, 0x2f, 0x1f, 0xaa, 0x6a, 0x54, 0xe4, 0xe4, 0x81, 0xe5, 0x6a, + 0x15, 0xd7, 0x6b, 0xce, 0xd3, 0x6f, 0xce, 0x1b, 0xb7, 0x2d, 0xe9, 0x77, + 0xb3, 0x97, 0xaf, 0x58, 0x21, 0xa9, 0xf1, 0x22, 0x0c, 0x81, 0xb3, 0xd3, + 0x42, 0xa7, 0x0d, 0x97, 0x5a, 0xe3, 0x95, 0x9e, 0xcd, 0x9f, 0x0a, 0xdc, + 0x97, 0xef, 0x15, 0xa5, 0x35, 0xbf, 0x80, 0x26, 0x22, 0x61, 0x3c, 0x2a, + 0x1f, 0xca, 0xc0, 0x03, 0x08, 0x6c, 0x89, 0x00, 0x9a, 0x53, 0xd2, 0x0e, + 0xd5, 0xc7, 0xbd, 0x44, 0x85, 0x91, 0xf1, 0xf1, 0xdd, 0x10, 0x1c, 0x7d, + 0xc0, 0xa5, 0xce, 0x8d, 0xf7, 0x79, 0x5b, 0x18, 0xc4, 0x8b, 0xe6, 0x20, + 0xe0, 0xad, 0xa2, 0x8b, 0xfc, 0xc2, 0xfd, 0xba, 0xa9, 0x40, 0xfc, 0xa5, + 0xe7, 0xdc, 0x37, 0xec, 0xf0, 0x6d, 0xef, 0x64, 0xaf, 0xf5, 0x3a, 0xd2, + 0xc5, 0xef, 0x66, 0x80, 0x57, 0xb5, 0x0a, 0x53, 0x74, 0x3d, 0xd8, 0xea, + 0xf6, 0xf0, 0xdf, 0xc1, 0x5f, 0xde, 0xda, 0x39, 0xec, 0x8a, 0x16, 0xd9, + 0xf2, 0xcc, 0x8f, 0xc9, 0x17, 0xc0, 0xa5, 0xa2, 0x48, 0xdf, 0xcd, 0xe9, + 0xda, 0x53, 0x86, 0xf8, 0xed, 0x38, 0x5a, 0xb1, 0x25, 0xf2, 0x70, 0x6a, + 0xa8, 0xe2, 0x3f, 0x8e, 0xbb, 0x24, 0x78, 0x4b, 0x5c, 0xea, 0x55, 0xb7, + 0x33, 0x18, 0x52, 0x5e, 0xde, 0x76, 0xb1, 0x55, 0x50, 0x91, 0x0c, 0xce, + 0xc1, 0x04, 0x8f, 0x33, 0x5d, 0x6d, 0x20, 0x91, 0xcd, 0x17, 0x5a, 0x52, + 0xaf, 0x0a, 0xf5, 0xae, 0x31, 0x56, 0x18, 0x14, 0x92, 0x7e, 0x33, 0x90, + 0xbc, 0xe4, 0x9d, 0x0f, 0x2f, 0x8e, 0x9a, 0x92, 0x8f, 0x96, 0x0e, 0x52, + 0xbf, 0x98, 0x1a, 0x96, 0xc1, 0x57, 0xec, 0x5d, 0x92, 0x16, 0xa1, 0x2e, + 0x2b, 0x71, 0xbf, 0x2c, 0x41, 0xe4, 0x75, 0x56, 0x56, 0xb9, 0xd1, 0xf8, + 0x79, 0x17, 0x0e, 0x35, 0x32, 0x36, 0x7e, 0x44, 0x35, 0x35, 0xd4, 0x4f, + 0x9f, 0xc9, 0xbb, 0x29, 0x30, 0x71, 0x92, 0xa2, 0x80, 0x8d, 0x64, 0x8c, + 0x8d, 0xb9, 0xf2, 0x59, 0x65, 0xf0, 0x68, 0xcf, 0x83, 0x53, 0x59, 0x35, + 0xeb, 0xd6, 0x5a, 0xb7, 0xe3, 0xa8, 0x79, 0x79, 0x1a, 0x3c, 0xf5, 0x0d, + 0xfe, 0xfe, 0x98, 0xb9, 0x1b, 0x58, 0x0b, 0xf2, 0x99, 0x40, 0xaa, 0xa7, + 0x9e, 0xf8, 0x28, 0xb3, 0xe4, 0xe7, 0x87, 0x0b, 0x7f, 0xf5, 0xfc, 0x0f, + 0x02, 0x4e, 0xeb, 0x6c, 0x5c, 0x91, 0xe2, 0x62, 0x92, 0xe1, 0xe2, 0xf2, + 0x12, 0xaa, 0xf6, 0x24, 0xf4, 0x8e, 0xc9, 0xa1, 0xb5, 0x90, 0x6e, 0x76, + 0xab, 0xd7, 0xd4, 0xd6, 0x1b, 0x7e, 0x8f, 0x4e, 0xde, 0x48, 0x72, 0x58, + 0x27, 0x5e, 0x0f, 0xd3, 0xcb, 0xa0, 0xc6, 0x6e, 0xde, 0x88, 0x3e, 0x2c, + 0x53, 0x78, 0xb7, 0xd1, 0xac, 0x2a, 0x11, 0xcb, 0x1f, 0x74, 0x4d, 0xe5, + 0x73, 0x4f, 0xcf, 0x29, 0x13, 0x7a, 0xad, 0xca, 0xd2, 0xe9, 0xbe, 0x15, + 0xfb, 0xad, 0x06, 0xa1, 0xdf, 0xda, 0x8b, 0x39, 0x49, 0xf9, 0xbc, 0x0b, + 0x36, 0xf9, 0xfd, 0xaf, 0xee, 0x8f, 0x10, 0x50, 0xb1, 0xf7, 0x24, 0x58, + 0x46, 0x2e, 0xd4, 0x13, 0x5a, 0xb7, 0x43, 0x09, 0x5e, 0xb1, 0xdb, 0x69, + 0x71, 0xd9, 0xcd, 0xfe, 0x08, 0x49, 0xc0, 0x18, 0x98, 0x2d, 0xfc, 0xd5, + 0xe5, 0x2e, 0x21, 0xb6, 0x46, 0x6d, 0x5d, 0xf3, 0xfc, 0xf3, 0xe2, 0x42, + 0xbd, 0x74, 0xeb, 0x91, 0x60, 0x6e, 0xf0, 0x91, 0x36, 0x71, 0xaa, 0xfb, + 0x31, 0xc5, 0x0d, 0x4a, 0xda, 0xdb, 0x6b, 0xc7, 0xe3, 0xbf, 0x86, 0x3e, + 0x27, 0xe0, 0xc0, 0xef, 0x8d, 0x00, 0x59, 0x9b, 0x56, 0x85, 0xb6, 0x6a, + 0x2c, 0x1d, 0x79, 0x2b, 0xd0, 0x5c, 0x32, 0xc1, 0x22, 0x68, 0x2c, 0x1c, + 0x0b, 0x30, 0x45, 0xf9, 0x69, 0x91, 0x71, 0xad, 0xb1, 0xde, 0xaf, 0x7d, + 0x31, 0x13, 0xfd, 0x01, 0x4f, 0x36, 0x90, 0x7e, 0x90, 0x0d, 0xcc, 0x8e, + 0xd2, 0x38, 0x13, 0x93, 0x1e, 0x85, 0x14, 0xfa, 0x7f, 0x3f, 0x03, 0x3e, + 0x41, 0xda, 0xba, 0xfb, 0x90, 0xf1, 0x12, 0xc4, 0xc3, 0x26, 0x52, 0xcb, + 0xc9, 0xe3, 0x6a, 0xd0, 0x6a, 0x54, 0xeb, 0xbe, 0x0f, 0xc5, 0xf9, 0x87, + 0xd1, 0x26, 0x74, 0xc0, 0x74, 0x0d, 0xa5, 0x74, 0x3f, 0xfc, 0xbd, 0xf7, + 0x3a, 0x5b, 0x6d, 0xa1, 0x61, 0x63, 0x4b, 0x80, 0xa8, 0x47, 0xed, 0x76, + 0x8d, 0x5e, 0xe5, 0x6a, 0x7d, 0x62, 0xe1, 0x33, 0x9c, 0xa9, 0x7a, 0x59, + 0x08, 0x2a, 0xa5, 0x6a, 0x85, 0x64, 0x74, 0x39, 0x96, 0xe4, 0x86, 0xfb, + 0x67, 0x10, 0xff, 0x2b, 0x9b, 0x62, 0x31, 0x77, 0x87, 0x2e, 0x36, 0xb6, + 0x00, 0xf5, 0x41, 0x1c, 0xb5, 0xd8, 0x23, 0x85, 0x48, 0x21, 0x51, 0x2a, + 0xdb, 0x8b, 0xa8, 0xc3, 0x03, 0x74, 0x70, 0xbf, 0x1d, 0xdf, 0xae, 0xe7, + 0x42, 0x85, 0xea, 0xa5, 0xff, 0xbd, 0x07, 0x3f, 0x55, 0xb5, 0xae, 0x7c, + 0x49, 0x22, 0x64, 0xb8, 0xb8, 0xbf, 0x7f, 0x27, 0xc2, 0xd1, 0x28, 0xe4, + 0xcb, 0x03, 0x79, 0x6d, 0xfe, 0xb1, 0x2a, 0x2b, 0x2b, 0x5b, 0xcc, 0x15, + 0x8e, 0x4a, 0xa5, 0x34, 0xf7, 0x24, 0xe0, 0x6c, 0x04, 0x3e, 0x3f, 0x9c, + 0x43, 0xeb, 0x2e, 0x3f, 0xd5, 0xb1, 0x27, 0xbc, 0xf6, 0xa3, 0x08, 0x99, + 0x9c, 0xad, 0x34, 0x4d, 0x64, 0x70, 0x9e, 0xe4, 0xd4, 0x05, 0x6c, 0x5f, + 0xb0, 0xef, 0x4d, 0x2a, 0xa4, 0x77, 0x5b, 0x60, 0xb5, 0x1b, 0xa2, 0xe2, + 0xa3, 0x0f, 0x6c, 0x76, 0xa0, 0xd4, 0x3a, 0xad, 0xab, 0x9b, 0xbd, 0x35, + 0x12, 0xa6, 0xf2, 0xa1, 0x64, 0xca, 0x31, 0x34, 0xdd, 0x6c, 0x0f, 0x42, + 0xc5, 0x6a, 0x98, 0xcf, 0xab, 0xa0, 0x1c, 0xb8, 0xf7, 0x65, 0x5b, 0x99, + 0xae, 0xe7, 0xa7, 0x4a, 0xe7, 0x4c, 0xd9, 0xd9, 0x47, 0x0c, 0x37, 0xa0, + 0xab, 0x31, 0x6d, 0x6e, 0xb2, 0xb2, 0x19, 0x7c, 0xb5, 0xc7, 0xec, 0x89, + 0xb6, 0xcd, 0x90, 0xee, 0x90, 0xc3, 0xb8, 0xc1, 0x7e, 0x47, 0x7e, 0x94, + 0x3e, 0x63, 0xce, 0xe6, 0x3a, 0x68, 0xac, 0x2e, 0x93, 0x03, 0x59, 0xcc, + 0x6c, 0x93, 0xf5, 0xa4, 0x66, 0x51, 0x91, 0x6d, 0xef, 0x68, 0xb6, 0xb1, + 0x9f, 0x45, 0xf2, 0xad, 0x6f, 0x8a, 0x88, 0x88, 0x8c, 0x4c, 0x4e, 0x9a, + 0xae, 0xf6, 0x8d, 0xee, 0x79, 0xf2, 0x77, 0x7e, 0x3c, 0x11, 0x52, 0xfb, + 0x71, 0x5e, 0x34, 0x3e, 0x58, 0xcc, 0x97, 0xa2, 0x11, 0x70, 0xdb, 0xf1, + 0x5f, 0x70, 0x71, 0xc9, 0x54, 0x4e, 0xce, 0xd0, 0x9c, 0x93, 0x75, 0x69, + 0x7f, 0xf9, 0x94, 0x7c, 0xa1, 0x2c, 0x9d, 0xe1, 0x72, 0xad, 0x66, 0x5f, + 0xbf, 0x6c, 0xd5, 0x7c, 0xe7, 0xe5, 0xce, 0xd4, 0xa4, 0xfb, 0xa0, 0xbc, + 0xef, 0x65, 0xca, 0xf9, 0xe9, 0x45, 0x23, 0xd4, 0xb1, 0xd2, 0xd0, 0x67, + 0xe1, 0xbb, 0xa7, 0xff, 0x03, 0xf2, 0x9a, 0xf3, 0xc9, 0x9d, 0x47, 0xa7, + 0x3e, 0x59, 0xf0, 0x56, 0x86, 0x19, 0x92, 0xe7, 0x7b, 0xb4, 0x94, 0x6a, + 0x19, 0xa6, 0xf3, 0xd0, 0x23, 0xc8, 0x4c, 0x05, 0x0e, 0xce, 0x29, 0x61, + 0xd7, 0x29, 0x31, 0x25, 0xc5, 0x6d, 0x36, 0xdc, 0xaf, 0x98, 0x2b, 0xeb, + 0xd8, 0x3e, 0x0c, 0x85, 0x84, 0x9a, 0x9b, 0x3b, 0x35, 0xee, 0x17, 0x4d, + 0x46, 0xf9, 0xe5, 0x62, 0xd2, 0xb2, 0x0f, 0x87, 0xeb, 0x3b, 0xde, 0x85, + 0x52, 0x97, 0xcf, 0x53, 0x43, 0xf3, 0x6e, 0xc6, 0x01, 0x52, 0xa9, 0x6e, + 0x1e, 0x3f, 0x96, 0xf1, 0x78, 0x7c, 0xf2, 0x99, 0xd1, 0x43, 0x57, 0x3f, + 0xa3, 0x4e, 0x03, 0x5e, 0x8d, 0x86, 0xf9, 0x64, 0xce, 0xea, 0xd1, 0xca, + 0x03, 0x55, 0xe8, 0x7d, 0x77, 0x18, 0x31, 0x51, 0x0d, 0x92, 0xe4, 0x49, + 0xb5, 0x25, 0xbd, 0xad, 0x0e, 0x75, 0xbd, 0xc9, 0x7e, 0x8e, 0x07, 0x10, + 0xc6, 0x1e, 0x8a, 0x31, 0x3f, 0x35, 0x43, 0x53, 0x24, 0xc2, 0x83, 0x8f, + 0xc5, 0xfe, 0x26, 0x31, 0xc0, 0xe7, 0xbc, 0x4e, 0xff, 0xf6, 0xc8, 0x31, + 0x71, 0xc4, 0x15, 0x1c, 0xbf, 0xeb, 0xf4, 0xe2, 0x7d, 0x34, 0xf0, 0x2a, + 0x5b, 0xe0, 0x7d, 0x4f, 0xc4, 0xb3, 0xdb, 0x84, 0xd8, 0x4f, 0xd9, 0xa5, + 0x4d, 0x98, 0xd9, 0x4c, 0x95, 0xfa, 0x38, 0xd6, 0x8a, 0xd0, 0x0b, 0x41, + 0x0b, 0x28, 0x07, 0xde, 0x53, 0x13, 0x93, 0x83, 0xbf, 0xc9, 0x7b, 0x80, + 0x28, 0x1c, 0x16, 0xc1, 0xb3, 0x4d, 0xbb, 0x8e, 0x3a, 0x2a, 0x94, 0x4d, + 0x57, 0x04, 0xfd, 0xd8, 0x18, 0xc3, 0x0c, 0x3f, 0xf4, 0x8a, 0xf2, 0xbe, + 0xc4, 0x8f, 0xa3, 0x31, 0x94, 0xa8, 0x25, 0x56, 0xdb, 0x73, 0xe5, 0x39, + 0xb6, 0x7a, 0xdc, 0xa7, 0x8d, 0x54, 0xd5, 0x2a, 0x5c, 0xd1, 0xa8, 0x21, + 0xd7, 0x4f, 0x76, 0x33, 0x53, 0xac, 0xb2, 0xd9, 0x7e, 0x7b, 0xf2, 0x29, + 0x95, 0xaf, 0x18, 0x97, 0xc8, 0x80, 0xec, 0x16, 0xa3, 0x5b, 0x99, 0x3f, + 0xd2, 0xc9, 0xc4, 0x70, 0xd0, 0x3a, 0xc5, 0x96, 0xc4, 0xfe, 0xca, 0x6a, + 0x58, 0x7e, 0xcf, 0xaf, 0xae, 0xb3, 0x15, 0xe3, 0xae, 0x7b, 0xed, 0x37, + 0xf2, 0xf5, 0xe1, 0x7e, 0x27, 0xce, 0xd8, 0xc0, 0x89, 0x1b, 0x8f, 0x23, + 0x37, 0x10, 0x27, 0xcd, 0x40, 0x4f, 0x11, 0xda, 0xf2, 0xf2, 0xf2, 0xbb, + 0xa2, 0xc9, 0x7e, 0x7b, 0x15, 0x8e, 0x4b, 0xd5, 0xcf, 0x8b, 0xa3, 0x01, + 0x60, 0x4b, 0xcb, 0xf0, 0xfe, 0xd8, 0xcd, 0x34, 0x2f, 0x66, 0x65, 0x20, + 0xbb, 0x5c, 0xc4, 0x70, 0x01, 0xa8, 0x5a, 0xcd, 0x58, 0x90, 0x80, 0x3d, + 0x4d, 0xa7, 0xda, 0x3a, 0x06, 0x75, 0xc6, 0x7d, 0xa4, 0x75, 0xdc, 0x28, + 0xe2, 0xe4, 0x29, 0x57, 0xc4, 0x45, 0xf8, 0x1b, 0x5c, 0xbf, 0x14, 0x9e, + 0x44, 0xd9, 0xcc, 0x7b, 0x6e, 0x2e, 0xf6, 0x3e, 0x0a, 0xf1, 0x2b, 0x25, + 0x46, 0xa7, 0x21, 0x67, 0x85, 0xda, 0x5e, 0x6c, 0x1b, 0x46, 0x78, 0xb4, + 0xa0, 0x6f, 0x53, 0x6c, 0xe7, 0x2b, 0x79, 0x78, 0xfc, 0x76, 0xde, 0xdf, + 0x2a, 0x17, 0x73, 0x90, 0x44, 0xd3, 0x2b, 0x76, 0x6d, 0x63, 0x7b, 0x09, + 0xa4, 0x36, 0x3e, 0x6d, 0x95, 0x17, 0xb6, 0xf3, 0x7f, 0xb3, 0x79, 0xf5, + 0x34, 0x28, 0xd1, 0x3b, 0xa1, 0xb4, 0x6b, 0xe5, 0x2d, 0x93, 0xdc, 0x0e, + 0x1a, 0x2e, 0xb8, 0x99, 0xc9, 0x22, 0xc2, 0x65, 0x1f, 0x68, 0xd1, 0xbd, + 0x8d, 0x27, 0xdb, 0xfd, 0xa4, 0xdc, 0x4b, 0xe4, 0xc6, 0x17, 0x7d, 0x3c, + 0x00, 0x4d, 0x9b, 0xa1, 0xfe, 0x44, 0x15, 0x75, 0x98, 0x0a, 0xbb, 0xbd, + 0xbd, 0x6e, 0xc3, 0xbc, 0x44, 0xf2, 0xb3, 0x27, 0x69, 0x12, 0x7f, 0xc1, + 0xec, 0xfe, 0x27, 0x0b, 0xc5, 0x24, 0x08, 0x43, 0x19, 0x8d, 0x5c, 0x48, + 0x08, 0x77, 0xbe, 0x4c, 0x6b, 0xbf, 0xd2, 0x0c, 0xe8, 0xe6, 0x83, 0x95, + 0x70, 0x4c, 0xc3, 0xea, 0x79, 0xbd, 0xa7, 0xac, 0x35, 0x35, 0x72, 0x72, + 0x57, 0xef, 0x72, 0xa4, 0x9c, 0x4f, 0x2e, 0x82, 0xb7, 0x72, 0x61, 0xc7, + 0x71, 0xf6, 0xaa, 0x2f, 0x55, 0xc1, 0x70, 0x32, 0xfb, 0xd1, 0xe5, 0x3c, + 0xce, 0x45, 0xf1, 0x3b, 0x43, 0x6d, 0x57, 0xb2, 0x3c, 0xa7, 0x67, 0x91, + 0xc8, 0x5b, 0x09, 0x13, 0xcd, 0xa1, 0xae, 0xdd, 0x32, 0xa3, 0x93, 0xff, + 0x50, 0x10, 0x50, 0x76, 0x02, 0x13, 0xca, 0x47, 0x18, 0x8d, 0x15, 0x75, + 0x88, 0x83, 0x26, 0x22, 0xe6, 0x7c, 0x52, 0x95, 0x70, 0x7a, 0xfa, 0x85, + 0xaf, 0x57, 0x51, 0x55, 0x0d, 0x8d, 0xa4, 0x5b, 0xb0, 0x54, 0x12, 0x4b, + 0x72, 0xe1, 0x95, 0xe7, 0xea, 0x6b, 0x53, 0x79, 0xce, 0x9d, 0xe0, 0x39, + 0x3f, 0x4a, 0xec, 0xc3, 0x79, 0xdf, 0x54, 0x16, 0x9f, 0x56, 0x87, 0x9b, + 0x6d, 0x99, 0xb2, 0x09, 0x59, 0xb0, 0x7f, 0xce, 0xeb, 0x9b, 0xbb, 0x4b, + 0x4f, 0x52, 0x02, 0x41, 0x8d, 0xa5, 0x57, 0x8c, 0x31, 0x7e, 0x60, 0x76, + 0x56, 0xcd, 0x7e, 0x37, 0x8a, 0x7b, 0x3f, 0xc7, 0xf4, 0xdb, 0x32, 0x22, + 0x68, 0xaf, 0x3d, 0x25, 0xc2, 0x03, 0xc8, 0xdb, 0xfd, 0xda, 0xcc, 0x4c, + 0x8a, 0xb3, 0x34, 0x5f, 0x8a, 0x0e, 0x46, 0xb9, 0xbd, 0xbe, 0xbe, 0xff, + 0xc7, 0x94, 0x6f, 0xa0, 0x6a, 0x9d, 0x32, 0xf6, 0x97, 0x27, 0x4f, 0x54, + 0xf6, 0xc6, 0x03, 0xeb, 0xc7, 0x9a, 0xf7, 0x0b, 0xd2, 0xbb, 0xe6, 0xf5, + 0x9f, 0x3d, 0xc5, 0xcf, 0x15, 0xdc, 0xaf, 0x5f, 0x7e, 0x34, 0x49, 0x63, + 0x45, 0x8a, 0x15, 0x1f, 0x3c, 0x65, 0x12, 0xf2, 0xba, 0x9f, 0xeb, 0x96, + 0x63, 0x3e, 0x45, 0x73, 0x6f, 0xfc, 0x1a, 0x4d, 0xa0, 0x7a, 0xaa, 0x73, + 0x35, 0x2f, 0x4c, 0x9a, 0x8f, 0x64, 0xf1, 0x7a, 0xba, 0x58, 0x94, 0x3b, + 0x61, 0x07, 0xef, 0x6e, 0xff, 0x27, 0x0f, 0x81, 0xc5, 0x23, 0xb2, 0x47, + 0x51, 0xfb, 0x39, 0xf8, 0x3c, 0xdf, 0x48, 0x7f, 0x6e, 0x62, 0x89, 0xb3, + 0x9e, 0x01, 0x71, 0xae, 0x85, 0xac, 0x73, 0xb4, 0xc4, 0xba, 0xc3, 0x15, + 0x17, 0xa4, 0xe5, 0x48, 0xcd, 0x93, 0x62, 0xa3, 0x02, 0x85, 0x3f, 0x8e, + 0xd4, 0x39, 0x18, 0xc2, 0x2f, 0x51, 0x08, 0x39, 0xe5, 0x2a, 0xd5, 0x0a, + 0x9d, 0x6d, 0x20, 0x15, 0xac, 0x1e, 0xcd, 0xe3, 0x72, 0x88, 0xda, 0xfd, + 0x76, 0xaf, 0x53, 0x86, 0xf6, 0x1e, 0xc5, 0xd2, 0x05, 0xdb, 0x73, 0x49, + 0x73, 0xd1, 0x9f, 0x05, 0xee, 0xe5, 0xa0, 0xaa, 0xaa, 0xa9, 0xf4, 0x5f, + 0xd9, 0x35, 0x1e, 0xff, 0x89, 0x4a, 0x40, 0x34, 0xba, 0x20, 0x47, 0x00, + 0x19, 0x14, 0x8c, 0xfb, 0x9a, 0xb3, 0xea, 0x08, 0xd6, 0xfb, 0x27, 0x42, + 0x4a, 0xf6, 0x59, 0xe7, 0xe6, 0xe1, 0x74, 0xa9, 0x54, 0x99, 0xec, 0x57, + 0xea, 0xf8, 0x62, 0xb9, 0x12, 0xdd, 0x66, 0xb2, 0xc2, 0x59, 0x19, 0x2d, + 0x1a, 0x99, 0x69, 0xf4, 0x61, 0xaa, 0x8d, 0xd7, 0x24, 0x07, 0xfb, 0xf9, + 0x34, 0x0a, 0x86, 0xb8, 0xec, 0xd3, 0x08, 0xb9, 0x0e, 0x25, 0xd9, 0x96, + 0x87, 0x3a, 0x33, 0x40, 0x58, 0xbc, 0x72, 0xa5, 0xec, 0xe4, 0xeb, 0xb4, + 0xe0, 0x4a, 0x94, 0x38, 0x03, 0x97, 0xc3, 0xcd, 0x24, 0x77, 0x8e, 0xfb, + 0x7d, 0xf2, 0x8e, 0x18, 0xa4, 0x83, 0x5e, 0x79, 0xf0, 0xbb, 0x74, 0xb4, + 0x21, 0xb9, 0xaf, 0xf9, 0x20, 0xb5, 0x1b, 0x3a, 0x6c, 0x39, 0xd0, 0x03, + 0x9f, 0x5f, 0x9a, 0xd3, 0xef, 0x4f, 0x5d, 0x98, 0x5a, 0x36, 0xe7, 0xf1, + 0xe1, 0x4a, 0x23, 0x73, 0x29, 0xdd, 0xd2, 0xb0, 0x34, 0x2f, 0x72, 0xfc, + 0xf5, 0x60, 0x7a, 0xc6, 0x83, 0x2e, 0x50, 0x34, 0xc5, 0x35, 0xd4, 0x92, + 0x0f, 0xa2, 0x4f, 0x6a, 0xbd, 0xfa, 0xb9, 0x64, 0xd9, 0xc7, 0x2d, 0x1b, + 0x16, 0xa6, 0xaf, 0xbc, 0x5a, 0x92, 0xf0, 0xf2, 0x0d, 0x44, 0x96, 0xd6, + 0x6d, 0xf4, 0x50, 0xb9, 0x39, 0xcb, 0x55, 0x76, 0xff, 0x30, 0x17, 0x24, + 0xa1, 0x43, 0x18, 0xc2, 0x76, 0xca, 0xc8, 0xce, 0xd7, 0x02, 0x97, 0x2f, + 0x01, 0x25, 0xb7, 0x8c, 0xde, 0x90, 0x4c, 0x5c, 0xe1, 0x5e, 0xeb, 0x1b, + 0x57, 0x64, 0x8b, 0xfe, 0xa1, 0x8c, 0x9b, 0x1b, 0x17, 0x84, 0x6c, 0xf9, + 0x85, 0x67, 0x59, 0xb8, 0xbc, 0x6e, 0x0f, 0x7b, 0x4f, 0x97, 0x1b, 0xda, + 0xe8, 0xcb, 0x88, 0x59, 0x50, 0xda, 0x71, 0xd6, 0x59, 0xbf, 0x67, 0x27, + 0xf7, 0x13, 0xc5, 0x02, 0x03, 0x51, 0x03, 0x15, 0xf4, 0x49, 0x24, 0x63, + 0xe5, 0x6d, 0x15, 0xd5, 0xbe, 0x64, 0x16, 0xe7, 0xd3, 0x4a, 0x47, 0xd8, + 0x50, 0x4c, 0x8e, 0xba, 0xde, 0xec, 0x85, 0x47, 0x04, 0xd8, 0x7f, 0xa5, + 0xa8, 0x93, 0xf7, 0xf9, 0xc1, 0x12, 0x95, 0xc5, 0x9a, 0x2f, 0x24, 0xb8, + 0xbb, 0xa3, 0xa9, 0x93, 0x22, 0xe9, 0x37, 0xca, 0xe8, 0x65, 0x73, 0x4a, + 0x7e, 0x77, 0x38, 0xeb, 0x36, 0x89, 0xd0, 0x15, 0xeb, 0x73, 0xdc, 0xdb, + 0xc0, 0x8c, 0x77, 0xb6, 0xe2, 0xfe, 0xdd, 0xff, 0x42, 0x50, 0xb1, 0x34, + 0x4c, 0x7d, 0x8a, 0x8e, 0x99, 0x53, 0x6e, 0xf4, 0xae, 0x2a, 0x71, 0x26, + 0x4b, 0x58, 0xed, 0xee, 0xff, 0x6b, 0x01, 0x38, 0x92, 0x27, 0x33, 0xb9, + 0x41, 0xcd, 0xb6, 0x3a, 0x29, 0x8c, 0x5a, 0x4e, 0xb3, 0x27, 0x69, 0x77, + 0xa7, 0xcb, 0x89, 0x10, 0x17, 0x0b, 0x7d, 0xd9, 0x3b, 0xf2, 0x7e, 0xd8, + 0xee, 0x28, 0x52, 0x7d, 0x6f, 0xfe, 0x7c, 0x3f, 0x92, 0xc3, 0x55, 0x38, + 0x40, 0x3f, 0x52, 0x1a, 0x19, 0x38, 0x59, 0x3c, 0xfb, 0xe6, 0x98, 0xf3, + 0x90, 0x4b, 0x0f, 0x7f, 0x43, 0x13, 0x4b, 0x8f, 0xc5, 0xb9, 0x2e, 0x8d, + 0xce, 0x95, 0x19, 0x9a, 0x0f, 0x6b, 0x98, 0x51, 0x6d, 0x95, 0xa1, 0x5e, + 0xd6, 0x5c, 0x4d, 0xb2, 0x66, 0xa7, 0x75, 0x8b, 0x2e, 0x66, 0x65, 0x75, + 0x91, 0x16, 0x38, 0x8d, 0xdc, 0x11, 0x7d, 0x39, 0x03, 0xd2, 0x31, 0xab, + 0xb1, 0x8b, 0x9b, 0xbf, 0x9a, 0xc5, 0x90, 0x44, 0xee, 0x52, 0x33, 0xf9, + 0x82, 0x20, 0x12, 0x89, 0x1f, 0x05, 0x4f, 0x96, 0xc6, 0x67, 0x49, 0xb9, + 0x55, 0xd1, 0x81, 0x8d, 0x85, 0x25, 0x6e, 0x65, 0x75, 0x2c, 0x7e, 0xa9, + 0xef, 0x23, 0xa0, 0x3c, 0x1c, 0x43, 0xf4, 0x74, 0x90, 0x69, 0x0f, 0x48, + 0x7b, 0xde, 0x40, 0x4d, 0xf2, 0xe9, 0x03, 0xde, 0xd5, 0xc5, 0x3e, 0x32, + 0xe1, 0x2f, 0xe5, 0x99, 0x26, 0xf2, 0x0e, 0x72, 0x00, 0x8a, 0x7e, 0xb5, + 0x58, 0x7e, 0x3f, 0xe5, 0x0a, 0x0c, 0x22, 0x2c, 0x34, 0x37, 0xbb, 0x59, + 0x0f, 0xdc, 0xd4, 0x6a, 0xf3, 0x23, 0x10, 0x1d, 0xf6, 0x93, 0x4f, 0x0d, + 0x0b, 0x85, 0xee, 0x3a, 0x1a, 0x7e, 0xca, 0x89, 0x26, 0x1f, 0xb1, 0x64, + 0xc1, 0x58, 0xd5, 0xea, 0xce, 0xba, 0x54, 0x7e, 0x67, 0xb8, 0xba, 0x6d, + 0x62, 0xf4, 0x8a, 0xe8, 0x77, 0x16, 0xef, 0xa5, 0xe9, 0x98, 0x0c, 0x75, + 0xcc, 0xb0, 0x24, 0x2b, 0xe9, 0x9f, 0xc2, 0xe8, 0xdf, 0xe0, 0x8b, 0x0e, + 0xb9, 0xb7, 0x3d, 0xe2, 0xd7, 0x50, 0x60, 0x67, 0xa6, 0x8e, 0x1c, 0x04, + 0x0d, 0x39, 0xbc, 0xad, 0x4f, 0xc7, 0xc5, 0xc7, 0x0b, 0xdb, 0xd8, 0x30, + 0x82, 0x8a, 0xe5, 0x0a, 0xcc, 0xa7, 0xb3, 0x07, 0xed, 0x66, 0xe1, 0xab, + 0x53, 0xb3, 0xb2, 0xe4, 0xa2, 0xb0, 0x99, 0x1e, 0x4f, 0x58, 0x36, 0x6d, + 0x7a, 0x11, 0xde, 0xce, 0x8b, 0x85, 0xab, 0xf0, 0xd3, 0x8a, 0x23, 0xe4, + 0xd4, 0x96, 0x81, 0xd3, 0x1a, 0xa9, 0xbe, 0x5b, 0x41, 0x2d, 0x86, 0xb8, + 0x60, 0x83, 0x27, 0x31, 0x90, 0x2a, 0x87, 0x9a, 0x61, 0xef, 0x35, 0xd2, + 0x6b, 0xed, 0x1e, 0x4a, 0x4c, 0x91, 0xb9, 0x51, 0x06, 0xb5, 0x16, 0x4f, + 0x75, 0xa2, 0x7a, 0xd7, 0x82, 0x4c, 0xa9, 0x44, 0x37, 0x8c, 0xdd, 0xee, + 0xb3, 0xba, 0x14, 0xc4, 0x6f, 0x98, 0x9a, 0x4d, 0x7b, 0x89, 0x34, 0x7d, + 0xaf, 0xb6, 0xcf, 0xa2, 0xc4, 0x66, 0xde, 0x64, 0xc5, 0xf5, 0xa0, 0x67, + 0x64, 0x0c, 0xad, 0xac, 0xc4, 0x8c, 0x3a, 0xa4, 0x82, 0x02, 0x4d, 0x94, + 0xcb, 0xbc, 0xff, 0x58, 0x0b, 0x72, 0xdf, 0x08, 0xf6, 0xc4, 0x1f, 0xef, + 0x7c, 0x9e, 0xb6, 0x7d, 0x6d, 0x36, 0xbb, 0xb1, 0x64, 0x5a, 0x7c, 0x0a, + 0x4d, 0x84, 0xb4, 0x35, 0x9b, 0xc7, 0x77, 0x51, 0x3f, 0xad, 0xe6, 0x14, + 0x61, 0x0a, 0xa1, 0x42, 0x8d, 0x1a, 0x7d, 0x9f, 0xbb, 0x05, 0x8d, 0x89, + 0x6c, 0x81, 0x0e, 0xcd, 0x1c, 0x15, 0xd8, 0xde, 0x9e, 0xdf, 0xd0, 0x9e, + 0x0f, 0xeb, 0xee, 0x6f, 0xdd, 0xbe, 0xb7, 0x2b, 0xa4, 0xda, 0x9c, 0x65, + 0x60, 0x03, 0x96, 0x5c, 0x82, 0xaf, 0xa3, 0x2f, 0xb1, 0xa8, 0xf8, 0xa7, + 0x94, 0x6c, 0xfc, 0x9f, 0x53, 0x5f, 0xd6, 0x24, 0x66, 0xbf, 0x74, 0xb5, + 0xdb, 0xa8, 0x74, 0xeb, 0xf4, 0x35, 0xeb, 0xbd, 0x48, 0x6d, 0xf8, 0xef, + 0x15, 0x5a, 0xaf, 0x79, 0xec, 0x46, 0xdb, 0xfc, 0xab, 0xfd, 0x5d, 0xf0, + 0x33, 0xf6, 0x93, 0x22, 0x77, 0x1e, 0xbd, 0x33, 0x75, 0x34, 0x29, 0xcd, + 0x92, 0x9c, 0xb0, 0x6f, 0x7b, 0x7d, 0x30, 0x98, 0xa4, 0x3b, 0x2a, 0x64, + 0x7c, 0x89, 0x22, 0xb8, 0x3f, 0x2b, 0x39, 0x79, 0x67, 0xbe, 0x91, 0xfe, + 0xf0, 0xd9, 0x6f, 0xdc, 0x95, 0xe5, 0x3e, 0x93, 0x1e, 0x59, 0xa1, 0xf7, + 0xc7, 0xda, 0x7a, 0x4b, 0x33, 0xdf, 0x99, 0x63, 0x58, 0x4c, 0x71, 0xd6, + 0xa0, 0x76, 0xb6, 0x3f, 0x01, 0x62, 0xff, 0x5e, 0xbb, 0x7a, 0xf4, 0x93, + 0xdd, 0x1c, 0x03, 0x87, 0x42, 0x65, 0xff, 0x5f, 0x1e, 0x67, 0x58, 0x78, + 0xf8, 0x9d, 0x8a, 0x9a, 0xae, 0xf6, 0x5f, 0x38, 0x44, 0x5b, 0x93, 0xdc, + 0x9b, 0x03, 0x76, 0x2d, 0x90, 0x73, 0x16, 0x80, 0x9a, 0x6c, 0x0c, 0xbe, + 0xa7, 0x63, 0x47, 0x9f, 0xfc, 0x55, 0xf4, 0x6c, 0xdc, 0x4a, 0x2a, 0x0d, + 0xaf, 0x75, 0x1e, 0x6b, 0xd5, 0x8e, 0x7b, 0x74, 0x1a, 0x42, 0xe9, 0x9e, + 0xb5, 0xee, 0xb1, 0x80, 0xf8, 0x80, 0xc0, 0x2a, 0xbe, 0xfb, 0x6d, 0x27, + 0xef, 0xf0, 0xc8, 0x48, 0xa3, 0x83, 0x89, 0x74, 0xc8, 0xd5, 0x41, 0xd1, + 0x51, 0x09, 0xbd, 0xdd, 0x62, 0xfc, 0x8c, 0x21, 0xcc, 0xb3, 0x5e, 0xaa, + 0x0e, 0xb0, 0x47, 0x8b, 0x1f, 0x2e, 0x50, 0x51, 0x47, 0xd3, 0x11, 0x4c, + 0x80, 0x38, 0x19, 0xc9, 0x74, 0xb0, 0x62, 0x52, 0x78, 0xd9, 0x6f, 0xf9, + 0x6b, 0x19, 0x09, 0x17, 0xba, 0x64, 0xac, 0x2f, 0x39, 0xec, 0x0d, 0xce, + 0x0a, 0xde, 0x6b, 0x55, 0xdc, 0x8d, 0x85, 0x76, 0x7d, 0x65, 0xce, 0x5f, + 0x4d, 0xf6, 0xf7, 0xb0, 0xdb, 0xea, 0x47, 0xd3, 0x19, 0xed, 0x20, 0x16, + 0xbe, 0xcb, 0xe3, 0x45, 0x4b, 0x42, 0x60, 0x1d, 0xa6, 0x70, 0x54, 0xe0, + 0x61, 0x2d, 0x08, 0x74, 0x64, 0xb2, 0xd6, 0xea, 0x48, 0xf8, 0xdd, 0x21, + 0x77, 0x64, 0x49, 0x1b, 0xe8, 0xc5, 0xa9, 0x52, 0x22, 0x6f, 0x3a, 0x10, + 0x9e, 0x8a, 0x26, 0x72, 0x39, 0xc1, 0x3e, 0xbd, 0x75, 0xe7, 0xd5, 0x76, + 0xb4, 0x62, 0xf6, 0x7a, 0xb5, 0xfe, 0x59, 0xff, 0x74, 0x73, 0xb5, 0x1b, + 0xfa, 0x62, 0xf3, 0x9c, 0x2b, 0x60, 0x55, 0xc5, 0xe5, 0x41, 0x4b, 0x3a, + 0xa2, 0xdf, 0xce, 0xf8, 0x35, 0xe9, 0xe2, 0xd1, 0x2a, 0xc6, 0x1e, 0xc9, + 0x1f, 0x02, 0xb6, 0x6f, 0x11, 0xcc, 0xe5, 0xea, 0x55, 0xd2, 0x90, 0x3e, + 0x22, 0x25, 0x2b, 0xcb, 0xbf, 0xfd, 0x37, 0x9d, 0xdd, 0x7c, 0xae, 0x5c, + 0x8d, 0x21, 0xe1, 0x81, 0xc4, 0xf0, 0x50, 0x27, 0x11, 0x9a, 0x9e, 0x6c, + 0xaf, 0xcd, 0x6d, 0x44, 0xf4, 0x65, 0x53, 0xd4, 0x21, 0x0e, 0x1a, 0xc4, + 0xbf, 0x31, 0x6e, 0x56, 0xe1, 0x62, 0x9a, 0x2b, 0x74, 0x43, 0xe5, 0x09, + 0x2a, 0xd7, 0x16, 0x18, 0x62, 0xbd, 0x27, 0xfc, 0x82, 0x0b, 0x11, 0x8c, + 0x76, 0xf7, 0xce, 0xfb, 0x46, 0x16, 0x4a, 0x86, 0x61, 0xf1, 0x30, 0xef, + 0x1c, 0x3e, 0x4b, 0xc3, 0x70, 0xd4, 0x40, 0x39, 0x21, 0x5f, 0x9f, 0xc5, + 0x3f, 0x65, 0xe0, 0xeb, 0xeb, 0xa0, 0x20, 0xc2, 0xce, 0xef, 0x6e, 0x91, + 0xdd, 0xed, 0x08, 0x3f, 0xf6, 0xc7, 0x92, 0x8f, 0x66, 0xf2, 0xa0, 0x89, + 0x63, 0xc3, 0x26, 0x26, 0x64, 0x9d, 0x9c, 0x9c, 0x4a, 0xd5, 0xab, 0xf0, + 0x17, 0xca, 0xd5, 0x76, 0x2f, 0xba, 0x3f, 0xcc, 0x7a, 0x03, 0xbf, 0xf8, + 0xa5, 0xe2, 0x92, 0x7e, 0x1f, 0x11, 0xce, 0xc4, 0x8e, 0x6e, 0xdd, 0x3e, + 0x66, 0x68, 0xdb, 0x4d, 0xb1, 0x1e, 0x99, 0x72, 0xcc, 0xe1, 0xe2, 0xb8, + 0xf2, 0xe4, 0x0e, 0x1c, 0xc8, 0xf4, 0xdd, 0x15, 0x71, 0x16, 0xc4, 0x8b, + 0x01, 0x33, 0x76, 0x8c, 0xc0, 0x4b, 0xbe, 0x0f, 0x0d, 0x1c, 0xbd, 0xb6, + 0x9b, 0xb1, 0xff, 0xb7, 0x02, 0xe0, 0xe6, 0x60, 0x92, 0x82, 0x63, 0x9e, + 0x5b, 0xd9, 0xa0, 0x48, 0x1a, 0x0b, 0x99, 0x98, 0x3f, 0x7c, 0x78, 0x97, + 0x87, 0xce, 0x37, 0x85, 0x45, 0xa5, 0x52, 0x18, 0x02, 0xd3, 0xc3, 0xb1, + 0x24, 0xe0, 0x30, 0xba, 0x1a, 0x25, 0x32, 0xde, 0xfa, 0x89, 0xb6, 0x73, + 0x41, 0x22, 0xd5, 0x56, 0xef, 0x1e, 0x6b, 0x9b, 0xb5, 0x66, 0x9d, 0x40, + 0x5f, 0x4e, 0x8f, 0x6a, 0xe5, 0x24, 0x5c, 0x59, 0xf6, 0x41, 0x72, 0xaf, + 0xab, 0x3e, 0xbb, 0xc0, 0x9f, 0x8d, 0x76, 0xda, 0x39, 0x0f, 0x3f, 0x6f, + 0x85, 0x76, 0x6f, 0xb5, 0xa5, 0x57, 0x1c, 0x79, 0xd7, 0x5e, 0xd6, 0xa4, + 0x52, 0x1c, 0xfa, 0x5f, 0x51, 0xae, 0x8e, 0x25, 0xea, 0x5a, 0x18, 0x19, + 0xf8, 0x52, 0x87, 0xc3, 0x80, 0xc3, 0x06, 0x90, 0x5b, 0xbc, 0x70, 0x87, + 0xd1, 0x32, 0x74, 0x40, 0x41, 0x39, 0x12, 0x31, 0x59, 0xac, 0x54, 0x20, + 0xfd, 0x05, 0xfd, 0x64, 0x08, 0x74, 0xf4, 0x9c, 0xd2, 0x68, 0xa1, 0x0b, + 0xa3, 0xc7, 0x0d, 0x31, 0x32, 0x58, 0x9e, 0x82, 0xbc, 0x23, 0x47, 0x5b, + 0xcf, 0x1d, 0x39, 0x07, 0x3d, 0xdb, 0xaa, 0xe7, 0x27, 0x8b, 0xd9, 0x3a, + 0xf6, 0x16, 0x9e, 0x12, 0x09, 0xab, 0x1c, 0xd1, 0xa3, 0x20, 0x7f, 0xa3, + 0xfb, 0x69, 0x06, 0x3c, 0x7e, 0x0f, 0x56, 0x7b, 0xe0, 0xdd, 0x22, 0x5c, + 0x50, 0xf8, 0x7f, 0x0f, 0x4e, 0x0d, 0xfb, 0x43, 0xdf, 0x4d, 0xd2, 0x4a, + 0xc6, 0x5c, 0xbb, 0xdd, 0xce, 0xb4, 0x33, 0x38, 0xad, 0x25, 0xce, 0x32, + 0xd4, 0x1e, 0x37, 0x61, 0x58, 0xb2, 0x14, 0x6b, 0xaa, 0x19, 0xb5, 0xee, + 0x4a, 0xc3, 0x59, 0x57, 0x6e, 0x71, 0x07, 0xba, 0x44, 0xe9, 0x60, 0xd1, + 0x3a, 0xa5, 0xc7, 0xd6, 0x2a, 0x18, 0xa3, 0xf4, 0xd7, 0xb8, 0xb1, 0xa7, + 0x0c, 0x8a, 0x5c, 0x0c, 0x88, 0x88, 0xb0, 0x7a, 0x59, 0x0c, 0x62, 0x04, + 0x4f, 0xae, 0xc6, 0xd5, 0xed, 0x77, 0x92, 0xdd, 0xbb, 0x7b, 0x92, 0xc1, + 0x54, 0x0c, 0x7c, 0x38, 0x61, 0xeb, 0x43, 0x24, 0x9b, 0x72, 0xb4, 0x68, + 0x54, 0x77, 0xfb, 0x72, 0x1a, 0xb0, 0x6e, 0xbc, 0x92, 0xd3, 0x13, 0xf9, + 0xba, 0x26, 0xdf, 0xf1, 0xd2, 0x79, 0x18, 0x75, 0xbb, 0x62, 0x46, 0x2e, + 0x15, 0x5d, 0xf4, 0x4a, 0x7c, 0x4f, 0xa8, 0xd3, 0x94, 0x77, 0xae, 0x8e, + 0x71, 0x54, 0x29, 0x74, 0xa5, 0x47, 0x87, 0x6c, 0x47, 0xd7, 0xb8, 0x3d, + 0x38, 0x32, 0xca, 0xaf, 0xcf, 0x82, 0xc9, 0x0e, 0x9f, 0xcb, 0xb8, 0xef, + 0xf7, 0xda, 0xd7, 0xfe, 0x78, 0x7d, 0x58, 0x37, 0x9f, 0xad, 0x4c, 0xe7, + 0x0a, 0x07, 0xc9, 0x6b, 0x05, 0xad, 0xb3, 0x22, 0x12, 0x22, 0xc3, 0xd8, + 0x77, 0x3d, 0x51, 0x07, 0x7d, 0x45, 0xe2, 0x3d, 0xf3, 0x35, 0xf0, 0xb9, + 0x12, 0x97, 0x88, 0x21, 0x1c, 0xfd, 0x83, 0x62, 0x90, 0x6f, 0xb6, 0xd0, + 0xca, 0x66, 0xb9, 0x00, 0xb5, 0xf1, 0xf7, 0x11, 0xc9, 0x60, 0x4a, 0x3f, + 0xad, 0x26, 0x71, 0x1b, 0x45, 0x32, 0x9d, 0x07, 0x12, 0xdb, 0x3a, 0x74, + 0xe0, 0x5a, 0x83, 0x72, 0x31, 0xb0, 0xba, 0x1c, 0xfe, 0x04, 0xec, 0xef, + 0xa1, 0xbd, 0xff, 0xfe, 0xb0, 0x05, 0x8a, 0x11, 0x08, 0x3b, 0x5c, 0x74, + 0xff, 0xce, 0x25, 0x67, 0xaa, 0xf7, 0xee, 0x99, 0xca, 0x67, 0x93, 0x31, + 0xe6, 0x6f, 0xf0, 0x16, 0x7e, 0x81, 0x53, 0x53, 0x6c, 0x24, 0x8b, 0x89, + 0xa1, 0x7f, 0x04, 0x3e, 0xe9, 0x09, 0xbe, 0xec, 0xce, 0x85, 0x40, 0xc7, + 0x33, 0xff, 0x4d, 0xf3, 0xc7, 0xdf, 0x2f, 0x08, 0xd8, 0x33, 0xd5, 0x7a, + 0x1d, 0x2a, 0x63, 0x36, 0xeb, 0xed, 0xae, 0xba, 0x9e, 0xfb, 0xc9, 0x26, + 0x26, 0x26, 0xa8, 0x68, 0x68, 0x8e, 0x56, 0x35, 0x54, 0x0c, 0x51, 0x97, + 0x97, 0x0e, 0xe5, 0x6a, 0x91, 0xa6, 0xe7, 0x25, 0x14, 0xd3, 0xb8, 0xfa, + 0x9f, 0xd9, 0xfa, 0x44, 0x02, 0x2d, 0x3c, 0x5f, 0x00, 0xca, 0x73, 0x31, + 0xd0, 0xf0, 0x64, 0x64, 0x02, 0x14, 0xd7, 0xd7, 0xea, 0xe6, 0xc7, 0xcf, + 0xc8, 0xd6, 0xac, 0xd5, 0x9a, 0x76, 0x95, 0xc1, 0xbe, 0x47, 0x0d, 0x3f, + 0xa8, 0xdf, 0x42, 0x81, 0xfe, 0xd2, 0x0f, 0x8f, 0x9b, 0xdf, 0xda, 0x2c, + 0xa5, 0xcb, 0xc1, 0x38, 0xe4, 0x71, 0x49, 0x73, 0x13, 0x29, 0xe0, 0xff, + 0xcc, 0xfb, 0xc8, 0xae, 0x3b, 0x1a, 0xfa, 0xe9, 0x96, 0xa2, 0x34, 0xef, + 0x74, 0xa7, 0x20, 0x5c, 0x8d, 0xdd, 0xd6, 0xdd, 0xf1, 0x7f, 0x79, 0xb6, + 0x4e, 0x42, 0xb2, 0x4f, 0xe3, 0xa9, 0x18, 0x07, 0x1d, 0x50, 0xa6, 0x52, + 0x92, 0xdd, 0x6c, 0x6c, 0xb3, 0xd6, 0xfa, 0x8d, 0x2b, 0x33, 0xfa, 0xa4, + 0xa6, 0x7b, 0x63, 0x53, 0x2a, 0xbb, 0x70, 0xcb, 0xd0, 0xf0, 0xf6, 0x62, + 0x33, 0x07, 0x08, 0x13, 0x5b, 0xff, 0xa1, 0x08, 0x6f, 0x8a, 0x95, 0xbe, + 0xdc, 0xc6, 0x98, 0xc1, 0x4b, 0xfd, 0x91, 0x49, 0xe7, 0xec, 0x54, 0xcc, + 0xb2, 0x70, 0x47, 0x24, 0x26, 0xd1, 0xdc, 0x12, 0xae, 0x4f, 0x48, 0xab, + 0x71, 0x5e, 0x11, 0xf4, 0x25, 0x27, 0xf5, 0xb5, 0x0b, 0xaa, 0x21, 0x3c, + 0x4a, 0xed, 0x94, 0x2b, 0x7a, 0x95, 0xb7, 0x51, 0x02, 0x2d, 0xf0, 0x87, + 0x56, 0x32, 0x53, 0x30, 0x29, 0x23, 0x47, 0x1a, 0xd6, 0x87, 0x77, 0xec, + 0xcb, 0xc5, 0x61, 0xcf, 0xe6, 0x28, 0x4c, 0x9b, 0xf7, 0x96, 0xc5, 0x39, + 0xf1, 0xf7, 0x0c, 0xc7, 0x9c, 0xe7, 0xb9, 0xef, 0x2d, 0x36, 0xca, 0xb1, + 0x78, 0x24, 0xc8, 0xc6, 0x67, 0xee, 0x18, 0x92, 0xa5, 0xfa, 0x11, 0x10, + 0x88, 0xf9, 0x9b, 0x87, 0xa1, 0xf6, 0xaf, 0xd5, 0x91, 0x02, 0xd2, 0x58, + 0xf6, 0x4b, 0x72, 0xfc, 0x78, 0xab, 0x74, 0x11, 0x22, 0x05, 0xbb, 0xbe, + 0xbb, 0x49, 0xff, 0xa8, 0x71, 0x14, 0xca, 0x3c, 0x45, 0x02, 0x15, 0x3e, + 0x9c, 0x9d, 0xdd, 0x08, 0xe8, 0xda, 0x1b, 0x8b, 0xd3, 0x3d, 0x48, 0x48, + 0x0c, 0x7d, 0x82, 0x74, 0x40, 0xc1, 0x0a, 0x7d, 0xbb, 0x46, 0xcb, 0x62, + 0x01, 0xa4, 0x70, 0x2f, 0x0d, 0x55, 0xe3, 0xe3, 0x0e, 0xd1, 0xc8, 0xf9, + 0x2d, 0x83, 0x94, 0x9a, 0xdd, 0xb2, 0x78, 0x8c, 0xe5, 0xeb, 0x1b, 0x4a, + 0xfe, 0x8d, 0x9e, 0x51, 0x57, 0x75, 0xd6, 0xd2, 0xe2, 0x93, 0xaf, 0x3f, + 0x63, 0x4c, 0x59, 0x8d, 0xdf, 0xd9, 0xe8, 0x35, 0x22, 0x0f, 0x9a, 0xef, + 0xab, 0x5e, 0x41, 0x0e, 0x80, 0x02, 0xa1, 0xd7, 0x8c, 0x32, 0x67, 0xec, + 0xf8, 0xc4, 0x44, 0x51, 0x07, 0x87, 0x4b, 0xcb, 0x1a, 0x0c, 0x7c, 0x7c, + 0x69, 0x79, 0xf9, 0x29, 0x3c, 0x29, 0x42, 0x7c, 0xfc, 0xc8, 0xfe, 0xe2, + 0x5a, 0x8f, 0x63, 0xdc, 0x09, 0x36, 0x69, 0x37, 0x3a, 0xaa, 0x8c, 0xf2, + 0xb9, 0xc3, 0x83, 0x9f, 0xaf, 0x53, 0x46, 0x8a, 0x33, 0xfd, 0xdb, 0xcc, + 0x4b, 0xc3, 0x5a, 0xbf, 0xaf, 0xff, 0xa4, 0x4e, 0xe4, 0x5b, 0x6b, 0xbb, + 0xca, 0xb2, 0x9f, 0x69, 0x89, 0x47, 0x3c, 0x22, 0xc5, 0xe9, 0xb8, 0x9f, + 0x9a, 0x6e, 0x47, 0xde, 0xd1, 0x24, 0x63, 0xf0, 0x1e, 0x15, 0x01, 0x34, + 0x54, 0x1f, 0xce, 0x98, 0xba, 0x66, 0x80, 0xff, 0x8a, 0x66, 0xdb, 0x03, + 0xec, 0x8f, 0x4b, 0x9c, 0x42, 0xae, 0x11, 0x3d, 0x6a, 0x19, 0xdc, 0xd7, + 0x5c, 0x26, 0xe1, 0x28, 0x53, 0x95, 0x9f, 0x4b, 0x28, 0xc0, 0x13, 0xe9, + 0xce, 0xd2, 0x5a, 0x58, 0xc1, 0xc2, 0x99, 0x5c, 0xb6, 0xe4, 0x5c, 0x5c, + 0x29, 0x7b, 0x7b, 0x7b, 0xa0, 0x31, 0x97, 0x9e, 0x00, 0x28, 0x13, 0xc2, + 0xe9, 0x21, 0x33, 0x10, 0x6d, 0x05, 0xaa, 0xc8, 0x2f, 0xe9, 0xc4, 0x5e, + 0x14, 0xd7, 0x07, 0x02, 0xa1, 0x54, 0xeb, 0x6b, 0x2f, 0xc8, 0x54, 0xbf, + 0x1e, 0x3e, 0x67, 0xfb, 0x14, 0x21, 0x5f, 0xa8, 0x93, 0xae, 0xf5, 0xd6, + 0xf8, 0x8c, 0xdc, 0xd8, 0xfb, 0x45, 0xae, 0xdb, 0xf1, 0x51, 0x2c, 0x57, + 0x68, 0xdc, 0x9b, 0xa6, 0x2c, 0xbb, 0x3f, 0x5a, 0xf4, 0xd0, 0x6f, 0xa4, + 0xbf, 0x2f, 0xb6, 0x56, 0xd7, 0x2c, 0xbe, 0xa4, 0xaf, 0x39, 0x91, 0x7d, + 0xe6, 0xb4, 0x9a, 0xdf, 0x9a, 0xc9, 0x13, 0xe5, 0x7f, 0x58, 0x73, 0x2e, + 0x11, 0x38, 0x5f, 0xd4, 0xb1, 0x67, 0x6a, 0x48, 0xe9, 0x64, 0x95, 0x55, + 0x71, 0x32, 0x68, 0x21, 0x80, 0xb7, 0xb6, 0x98, 0x8c, 0x8b, 0xfb, 0xaa, + 0xa1, 0x25, 0xc6, 0x42, 0xed, 0x01, 0x7c, 0x51, 0x0e, 0x86, 0x35, 0x88, + 0xb7, 0xaf, 0xfc, 0x0f, 0x05, 0xb9, 0xef, 0xfd, 0x42, 0x9b, 0x32, 0xdf, + 0xa9, 0x29, 0x0f, 0xe7, 0xf1, 0xbc, 0x10, 0xd4, 0x07, 0xa2, 0xdf, 0x1e, + 0xfd, 0x4d, 0xd9, 0x9a, 0x5d, 0xc0, 0x9e, 0xad, 0xd5, 0xa6, 0xc6, 0xc5, + 0xb2, 0x3c, 0x2f, 0xb2, 0x8a, 0x1d, 0x6e, 0x67, 0x8f, 0xcf, 0x27, 0x35, + 0xa3, 0x23, 0x23, 0xdf, 0x8b, 0x2a, 0x89, 0x7d, 0x2e, 0x85, 0xfb, 0x44, + 0x82, 0x2e, 0x8f, 0xbd, 0xd2, 0xa9, 0xdb, 0x51, 0x3f, 0x89, 0xc3, 0x66, + 0x7b, 0xb5, 0x76, 0x7c, 0xb4, 0xfc, 0xfd, 0x6b, 0x9b, 0xa3, 0x2d, 0xc6, + 0xd2, 0xfc, 0x5d, 0x00, 0x9b, 0x8e, 0x5d, 0x87, 0x41, 0x3a, 0x58, 0x68, + 0xfa, 0xaa, 0x97, 0xc2, 0xf2, 0x15, 0xbc, 0x5c, 0xd7, 0x78, 0xca, 0x15, + 0x30, 0xfd, 0xf1, 0x80, 0x40, 0xd6, 0x2b, 0x7f, 0xda, 0x05, 0x1c, 0xe8, + 0x9a, 0x28, 0x7b, 0x92, 0x6e, 0x33, 0x3a, 0x3b, 0x60, 0xe3, 0xf9, 0xa8, + 0x20, 0xa0, 0x46, 0x3a, 0x79, 0x74, 0xeb, 0x96, 0xcb, 0xe4, 0x3a, 0x6b, + 0xcc, 0xca, 0x4a, 0x4a, 0xb0, 0x5a, 0xf0, 0x91, 0x09, 0xb7, 0x5e, 0x28, + 0x1f, 0x44, 0x60, 0xe9, 0xec, 0x99, 0x48, 0x67, 0xc7, 0xe3, 0xb4, 0xc2, + 0x74, 0xdf, 0xf0, 0xa4, 0xd1, 0xd0, 0x75, 0xfc, 0xe2, 0xfd, 0x70, 0x7a, + 0x30, 0xee, 0x38, 0x9d, 0x03, 0xe2, 0x7d, 0xf9, 0x11, 0x62, 0x98, 0xe1, + 0xd2, 0x5d, 0x24, 0x97, 0x47, 0xab, 0x54, 0x43, 0xc0, 0x4a, 0xa4, 0x65, + 0x69, 0x11, 0x5c, 0xbc, 0xf0, 0x63, 0x29, 0xb1, 0x6e, 0xd5, 0x9a, 0xe6, + 0xe8, 0x39, 0x6f, 0xe3, 0x87, 0x1c, 0xc8, 0x45, 0x0f, 0xca, 0x8b, 0xe2, + 0x92, 0x0a, 0xb1, 0x54, 0xa7, 0x01, 0x1e, 0xcb, 0x68, 0xcc, 0xd1, 0xbf, + 0x85, 0xa0, 0x4c, 0xf9, 0x5b, 0x86, 0x9b, 0xbb, 0x40, 0xb4, 0x1c, 0x48, + 0x53, 0xc0, 0xdc, 0x27, 0x5d, 0x15, 0x02, 0xb1, 0x14, 0x65, 0xbb, 0x05, + 0x1c, 0xd2, 0x36, 0x10, 0x42, 0xb6, 0x9e, 0xf2, 0xa8, 0x2c, 0x2f, 0x2f, + 0xf9, 0x4b, 0x73, 0xe1, 0x0c, 0x9e, 0xe4, 0x5f, 0x2e, 0x21, 0x90, 0x40, + 0x63, 0xbb, 0x48, 0xc1, 0xb2, 0xc8, 0x90, 0xfe, 0x48, 0x1d, 0x6a, 0xb0, + 0x18, 0x0c, 0x7d, 0xbd, 0xa8, 0x9b, 0x3e, 0x7a, 0x9e, 0x9d, 0x4d, 0x7d, + 0x73, 0x4e, 0x86, 0x47, 0x28, 0xa2, 0x81, 0xde, 0xba, 0xb2, 0x31, 0x40, + 0xa9, 0x64, 0x99, 0x87, 0xa2, 0xc4, 0x1b, 0x8a, 0x7f, 0x01, 0xea, 0x03, + 0x27, 0x21, 0x1d, 0x48, 0x8e, 0xdb, 0x4e, 0xc6, 0x6c, 0x90, 0x44, 0x67, + 0xaf, 0xdf, 0xdd, 0x15, 0xf3, 0x47, 0xba, 0x1b, 0x77, 0x08, 0xea, 0xbc, + 0x5c, 0xe9, 0x10, 0x89, 0xff, 0x6b, 0x6f, 0xb1, 0x5c, 0xa6, 0xc5, 0x4c, + 0x6e, 0x2f, 0xdd, 0x68, 0xbb, 0xe3, 0xae, 0x44, 0xf7, 0xa0, 0xfb, 0x3f, + 0x5b, 0x97, 0x8e, 0x23, 0xfc, 0x72, 0xd5, 0xc8, 0xbc, 0x9b, 0x3b, 0x0d, + 0x25, 0x66, 0xf2, 0xd3, 0xe0, 0x2c, 0x17, 0x6c, 0x9f, 0xa7, 0x2f, 0x24, + 0x27, 0xeb, 0x83, 0x0a, 0x2e, 0x7a, 0xd3, 0xed, 0x6c, 0x94, 0x7b, 0x76, + 0x4e, 0x4e, 0x5d, 0xed, 0x9d, 0x77, 0xfb, 0x4e, 0x96, 0x16, 0x6d, 0x4e, + 0x02, 0x7d, 0x73, 0x73, 0x5d, 0x04, 0x88, 0xcf, 0x05, 0xb9, 0xc3, 0xfb, + 0xc3, 0xb7, 0x46, 0xfd, 0x0f, 0x18, 0x2c, 0x16, 0x70, 0xc7, 0xfb, 0x7f, + 0x36, 0x17, 0x6d, 0x0e, 0x22, 0x0c, 0x2f, 0x66, 0x6b, 0x0d, 0xfb, 0x55, + 0xd4, 0xd5, 0xf5, 0x1a, 0xcc, 0x90, 0x26, 0x33, 0x5f, 0xd8, 0xdc, 0x5c, + 0x3b, 0x9e, 0xf0, 0xf5, 0xfd, 0xef, 0x85, 0x5a, 0xd7, 0xb2, 0xd9, 0x18, + 0x6a, 0xf2, 0xdd, 0xf3, 0x67, 0x8a, 0x87, 0x7f, 0xb1, 0x35, 0x38, 0x53, + 0xfd, 0xfa, 0x94, 0x01, 0x5a, 0xfe, 0xda, 0xe2, 0x9d, 0x8e, 0x55, 0x90, + 0xca, 0x01, 0x19, 0x72, 0xf2, 0xd3, 0xc3, 0x78, 0xae, 0xc6, 0x7c, 0x63, + 0xda, 0x7b, 0xae, 0x0d, 0xcc, 0x3f, 0xe7, 0x4b, 0x8b, 0x74, 0xb5, 0x95, + 0x13, 0x1d, 0x1a, 0x47, 0x95, 0xd5, 0x03, 0x9b, 0x95, 0x46, 0x38, 0xb2, + 0xcf, 0x02, 0x5e, 0xb7, 0xbf, 0xce, 0xd7, 0xda, 0xfe, 0x10, 0x0b, 0x64, + 0xa4, 0xfd, 0xd3, 0xac, 0xee, 0x3e, 0xc7, 0x3f, 0x3f, 0x5a, 0xc9, 0xb6, + 0xdf, 0x20, 0x19, 0x1b, 0x0f, 0x3a, 0xa7, 0xa0, 0x59, 0x22, 0xba, 0xf4, + 0xdc, 0x8e, 0x84, 0x56, 0xee, 0x23, 0xe8, 0x52, 0x56, 0xfb, 0xa4, 0xc8, + 0xc9, 0x54, 0xf7, 0x9c, 0x2e, 0x49, 0x4c, 0xbc, 0x01, 0x17, 0x10, 0x17, + 0xe7, 0x96, 0x22, 0xce, 0x62, 0x37, 0x5f, 0xc3, 0xb2, 0x5c, 0x2f, 0x3e, + 0xc2, 0x67, 0x67, 0xd9, 0x1f, 0xf5, 0x76, 0x41, 0x57, 0x00, 0x4f, 0x37, + 0xcb, 0x91, 0xdc, 0x01, 0xd3, 0x35, 0xd5, 0x9c, 0xe5, 0x3d, 0x8b, 0x94, + 0x78, 0x39, 0x43, 0x4a, 0xde, 0x27, 0x82, 0xf7, 0x3a, 0x43, 0xb4, 0xe6, + 0x62, 0xdb, 0x10, 0xa0, 0x63, 0xed, 0x50, 0xf7, 0x2b, 0x83, 0x5a, 0xc3, + 0xcd, 0x8f, 0xff, 0x43, 0xe7, 0x50, 0x77, 0xe4, 0xfc, 0x3a, 0x76, 0x90, + 0x85, 0x84, 0x41, 0x4c, 0x32, 0xe6, 0x82, 0xa8, 0x70, 0x58, 0xfa, 0x28, + 0x42, 0x35, 0x29, 0x17, 0x17, 0x34, 0xc1, 0xf3, 0x89, 0xab, 0xa5, 0x38, + 0x59, 0xa8, 0xfc, 0xe4, 0x65, 0x8e, 0x29, 0x1f, 0x27, 0x33, 0x0a, 0xeb, + 0xb9, 0xbc, 0x7c, 0x4f, 0xb0, 0xc5, 0x6b, 0xea, 0x25, 0xbf, 0x52, 0xbb, + 0x2c, 0x30, 0xcb, 0x4d, 0x18, 0xb9, 0x43, 0x88, 0x33, 0xd9, 0x10, 0xf7, + 0x70, 0x91, 0xf6, 0xf9, 0x4b, 0xec, 0xef, 0x3f, 0x84, 0x1f, 0x58, 0x9b, + 0x53, 0x3a, 0x09, 0xdd, 0x55, 0x22, 0xe3, 0x5f, 0xd9, 0x39, 0xba, 0x09, + 0xbe, 0x2f, 0x5c, 0x29, 0x4b, 0xd3, 0xbd, 0x3f, 0x11, 0xf3, 0x50, 0x51, + 0x16, 0x2e, 0x53, 0x2c, 0x2f, 0x77, 0x7b, 0x18, 0x19, 0xe9, 0x93, 0x51, + 0xed, 0x8b, 0x47, 0x1d, 0x82, 0xa1, 0xd5, 0xaa, 0x42, 0x2d, 0x89, 0x91, + 0x13, 0xe6, 0x12, 0x3b, 0xac, 0xee, 0x3d, 0x6c, 0x32, 0xac, 0xda, 0x0a, + 0xb1, 0x23, 0x9b, 0xa0, 0x5d, 0xf7, 0x3d, 0x2d, 0xd6, 0x6d, 0x49, 0x01, + 0xc2, 0xc2, 0xc2, 0xf3, 0xa5, 0x4a, 0x41, 0x49, 0x49, 0x98, 0xe8, 0x18, + 0x18, 0x47, 0xff, 0xea, 0x28, 0xd4, 0xca, 0x55, 0xb7, 0xef, 0x57, 0xed, + 0x87, 0xed, 0x3e, 0x4a, 0xe2, 0x24, 0xa5, 0xa5, 0xe1, 0xff, 0x0b, 0xd9, + 0x72, 0xbd, 0x69, 0x5a, 0xf1, 0xc0, 0x91, 0x0d, 0xe1, 0x85, 0x5b, 0x8d, + 0x02, 0x32, 0x5c, 0x5b, 0xca, 0xdf, 0xbf, 0x45, 0xa9, 0xa7, 0x9e, 0xcc, + 0xab, 0xae, 0xb1, 0xa1, 0x75, 0x38, 0x6a, 0x55, 0xe7, 0xdf, 0xfe, 0x08, + 0x10, 0x0a, 0x7a, 0xfd, 0xb1, 0x4a, 0x66, 0xd0, 0x5c, 0xb1, 0xcc, 0xd3, + 0x64, 0xf2, 0xaa, 0x05, 0xfd, 0x9e, 0x2c, 0xde, 0xbe, 0x3b, 0x2e, 0x2c, + 0x2a, 0xda, 0x76, 0xc9, 0x06, 0x0e, 0x78, 0xc9, 0xca, 0x22, 0x46, 0xe1, + 0xb0, 0x7c, 0xdd, 0xca, 0x0b, 0x23, 0x36, 0x9a, 0x2b, 0x94, 0xea, 0xb0, + 0x6d, 0x3e, 0x76, 0x75, 0x76, 0x4a, 0xb1, 0xeb, 0x48, 0x9a, 0x76, 0xd6, + 0x4a, 0x8d, 0xe3, 0xfc, 0x23, 0x10, 0x8a, 0x4a, 0x36, 0xbe, 0x91, 0xc1, + 0x01, 0x3d, 0x9e, 0xa2, 0x43, 0x0b, 0x87, 0x78, 0x06, 0x26, 0xf8, 0x78, + 0x7a, 0xbd, 0x99, 0x9e, 0x7e, 0x5a, 0x53, 0x10, 0x7c, 0xfa, 0x5c, 0x02, + 0xa2, 0x03, 0x6e, 0xf4, 0xfb, 0xeb, 0x1d, 0x83, 0x5b, 0x4f, 0xad, 0xc5, + 0x11, 0xd0, 0x9e, 0xfa, 0x12, 0x92, 0xfd, 0x21, 0x82, 0x99, 0xa2, 0xae, + 0x4d, 0xeb, 0xb6, 0xe6, 0xa4, 0xc7, 0x0e, 0x78, 0xbe, 0x67, 0xd2, 0x06, + 0x86, 0x05, 0x76, 0xfa, 0x6c, 0xbc, 0xea, 0x25, 0xbb, 0xaf, 0x39, 0x86, + 0x05, 0xf0, 0x63, 0x16, 0xf9, 0xb8, 0x42, 0x90, 0x38, 0xc0, 0x69, 0x5f, + 0xfb, 0x1d, 0x3e, 0xad, 0xda, 0x79, 0x04, 0x64, 0xd6, 0x2b, 0xa2, 0x64, + 0x08, 0xe2, 0xe8, 0x13, 0xf5, 0xd5, 0xf9, 0xbd, 0x6e, 0xe7, 0x1e, 0xaf, + 0xfb, 0x1e, 0x2b, 0xf4, 0x8e, 0xdb, 0xb1, 0x2e, 0xaf, 0xac, 0x8c, 0x0e, + 0x99, 0xe8, 0xc8, 0x05, 0xc0, 0x25, 0x53, 0xfc, 0x16, 0xe1, 0x87, 0xd9, + 0x81, 0x04, 0x5c, 0x9a, 0x31, 0xcd, 0x64, 0x5c, 0x79, 0xbe, 0x66, 0x86, + 0x65, 0xde, 0x3f, 0xe3, 0x3b, 0x43, 0x68, 0x2f, 0xe8, 0x2e, 0xa8, 0xb6, + 0xb5, 0xb5, 0xb5, 0x06, 0xed, 0xdb, 0x9f, 0x24, 0xbe, 0x49, 0x11, 0x18, + 0x34, 0xf5, 0x86, 0xfd, 0x5f, 0xbd, 0xee, 0x8e, 0xbf, 0xd0, 0x4a, 0x7a, + 0xfa, 0x84, 0x41, 0xbe, 0x16, 0x30, 0xdc, 0x40, 0x44, 0x46, 0x3e, 0x34, + 0xf1, 0x6c, 0xd9, 0x36, 0x32, 0xe6, 0xaa, 0xa5, 0xbe, 0x2e, 0x52, 0x57, + 0xa0, 0x57, 0x41, 0xe9, 0x5f, 0x17, 0xb7, 0x3b, 0x1c, 0x9b, 0xd3, 0x64, + 0x1c, 0xa7, 0x7c, 0x7d, 0xd9, 0x8f, 0x82, 0xe3, 0x01, 0xa5, 0xac, 0x44, + 0x2e, 0xa1, 0xf5, 0xcd, 0xeb, 0x5b, 0xae, 0xdd, 0xfa, 0x1f, 0xb2, 0x80, + 0x6f, 0x36, 0xa7, 0x98, 0xa7, 0xa6, 0xdf, 0x7f, 0x07, 0x86, 0x24, 0xfc, + 0xae, 0xf1, 0x69, 0x3a, 0xa0, 0x79, 0x5e, 0xfe, 0x23, 0x25, 0x2c, 0x3c, + 0xcd, 0x77, 0xd1, 0x89, 0xf1, 0x23, 0x00, 0xb7, 0xac, 0x3c, 0x99, 0x31, + 0xdf, 0x80, 0xb6, 0xd9, 0x0e, 0x0b, 0x7d, 0xbe, 0x00, 0xc1, 0x4f, 0x67, + 0x38, 0xb4, 0xaa, 0x83, 0x69, 0xb2, 0x51, 0xb1, 0x32, 0x4e, 0x79, 0xae, + 0xd0, 0x90, 0x09, 0x90, 0xc2, 0x62, 0xa0, 0xbf, 0x77, 0x87, 0x7f, 0xb2, + 0x58, 0x05, 0x4d, 0x4a, 0x59, 0xd8, 0x30, 0xcc, 0x8a, 0xac, 0xbf, 0x82, + 0x4a, 0x89, 0xaa, 0x2f, 0xa6, 0xa8, 0xa8, 0x82, 0x48, 0x01, 0xac, 0x1a, + 0x6e, 0xb0, 0x76, 0x6c, 0x0b, 0x67, 0x4d, 0xf1, 0x63, 0x6f, 0x96, 0xc5, + 0xa6, 0xaa, 0x55, 0x74, 0xd2, 0xec, 0x74, 0x82, 0x56, 0x88, 0x79, 0x04, + 0x64, 0x4e, 0x4a, 0x40, 0x0c, 0x9b, 0x94, 0xa4, 0x64, 0xae, 0x7a, 0x95, + 0xf6, 0x7f, 0x4f, 0xac, 0x2b, 0xaa, 0xdb, 0xc7, 0xba, 0xc3, 0x72, 0xb7, + 0x3e, 0xfe, 0xfc, 0x41, 0xcc, 0xcf, 0xcf, 0x2f, 0x2c, 0x1e, 0x94, 0x9d, + 0xbf, 0x4b, 0x1a, 0x66, 0x7e, 0xd2, 0x14, 0xfd, 0x05, 0x44, 0x5e, 0xfd, + 0x7a, 0x49, 0x2d, 0xef, 0xeb, 0x0e, 0xeb, 0x8a, 0x51, 0xae, 0xcd, 0xb0, + 0xcf, 0x88, 0x99, 0x63, 0x66, 0x89, 0x8f, 0xc4, 0xcb, 0xef, 0x32, 0xd6, + 0xbd, 0x50, 0xad, 0xa7, 0x13, 0x43, 0xc8, 0xa3, 0xfc, 0x3d, 0x4b, 0x5e, + 0x44, 0x54, 0x34, 0x5d, 0xd0, 0xb7, 0xfd, 0x2f, 0xab, 0x61, 0x20, 0x60, + 0xb5, 0x39, 0x2e, 0xee, 0xf9, 0xce, 0xd9, 0x19, 0x25, 0x5d, 0x99, 0xb8, + 0x29, 0x5c, 0x5f, 0x77, 0xa6, 0x71, 0xcc, 0xd6, 0xf5, 0xad, 0x67, 0x27, + 0xd4, 0x75, 0xe7, 0xe9, 0x19, 0x88, 0x33, 0xe6, 0xe3, 0x45, 0x89, 0x30, + 0x28, 0xe5, 0x92, 0x3c, 0x44, 0x44, 0x22, 0xbe, 0x84, 0xa4, 0xca, 0xe9, + 0x12, 0x64, 0x18, 0xec, 0xa6, 0xb4, 0xd0, 0x39, 0x8c, 0xd4, 0x9f, 0x05, + 0xbf, 0x11, 0x26, 0x5c, 0xa3, 0x42, 0x5b, 0x3e, 0x8d, 0xdd, 0x77, 0x2f, + 0x44, 0xb7, 0xfe, 0x90, 0x9a, 0x72, 0x0f, 0x33, 0xef, 0x93, 0x54, 0x4d, + 0xa2, 0xf9, 0xa9, 0xb9, 0x9b, 0xdb, 0x76, 0x75, 0x72, 0x65, 0xd2, 0x1f, + 0x7a, 0x11, 0xb6, 0x33, 0x64, 0xe7, 0x77, 0x7a, 0x0e, 0x3a, 0x7e, 0x0f, + 0xeb, 0x35, 0x7e, 0xff, 0xb6, 0x6f, 0x68, 0xfc, 0x7b, 0x67, 0xbe, 0xaa, + 0x2f, 0xc2, 0x6d, 0x3b, 0x1c, 0x5c, 0x0d, 0x0f, 0x69, 0xbd, 0xcc, 0xfe, + 0x18, 0xca, 0x22, 0x14, 0x85, 0xac, 0xf4, 0xc3, 0x1b, 0xf2, 0x4f, 0x51, + 0xc6, 0xfc, 0x54, 0x58, 0x2b, 0x52, 0xb1, 0xbd, 0x44, 0x80, 0x8c, 0xdf, + 0x07, 0x86, 0xb8, 0xe6, 0xcd, 0x31, 0x76, 0x24, 0x8c, 0xf2, 0x98, 0x08, + 0x3b, 0xaf, 0x3e, 0x47, 0x95, 0x36, 0x52, 0x49, 0x03, 0x5e, 0x71, 0x1b, + 0x51, 0x40, 0x16, 0x5f, 0xce, 0xc3, 0x8f, 0xa1, 0x35, 0x68, 0x18, 0x6a, + 0x7b, 0x96, 0xef, 0x34, 0x2c, 0xf6, 0x89, 0xe2, 0xb7, 0x09, 0xc2, 0x6c, + 0xca, 0xc5, 0x13, 0x30, 0x4c, 0x9e, 0xcb, 0xc7, 0x72, 0x73, 0x66, 0x72, + 0x4d, 0x5a, 0xb4, 0x5c, 0xf9, 0x2f, 0xc5, 0xcb, 0xf0, 0x5d, 0xf8, 0x12, + 0xc7, 0x4c, 0x76, 0x0f, 0x19, 0xe3, 0xc5, 0x6f, 0x50, 0xe6, 0xd7, 0x98, + 0x45, 0xf0, 0x91, 0xe8, 0xfc, 0xae, 0xab, 0x53, 0xbe, 0x7d, 0x1a, 0xdf, + 0xa7, 0x03, 0x43, 0x8c, 0x1c, 0xd6, 0xe5, 0x12, 0x49, 0xc3, 0x0a, 0xb6, + 0x2a, 0x00, 0x9e, 0x37, 0xa6, 0x83, 0x51, 0x3d, 0xde, 0x64, 0x2f, 0xdf, + 0x26, 0x27, 0xc8, 0x72, 0x2f, 0x27, 0xd7, 0x06, 0x17, 0xa5, 0x79, 0x10, + 0x06, 0x3f, 0x41, 0x1a, 0x12, 0xaf, 0x81, 0xce, 0x63, 0x89, 0x0e, 0xe4, + 0xd2, 0xe0, 0xc7, 0x89, 0xd9, 0xf6, 0x32, 0x3d, 0x65, 0x60, 0x91, 0xe0, + 0xea, 0x1b, 0xe7, 0x45, 0xea, 0x72, 0x85, 0xab, 0x8e, 0x43, 0x74, 0x3d, + 0x5b, 0x26, 0xc9, 0xbd, 0xca, 0x6a, 0x4f, 0xb5, 0x6a, 0x88, 0x04, 0x52, + 0x46, 0x4b, 0x36, 0xdf, 0x2d, 0xb4, 0x5c, 0x11, 0x06, 0x03, 0xd5, 0x35, + 0x16, 0x59, 0x08, 0x9b, 0x0a, 0x68, 0x93, 0x2b, 0x74, 0xc4, 0x58, 0xb0, + 0xce, 0xd5, 0x67, 0x60, 0x54, 0x8a, 0x3f, 0x28, 0x99, 0x2a, 0xec, 0xf2, + 0xbe, 0xe7, 0x96, 0xd1, 0x93, 0xf5, 0x4e, 0x89, 0xa8, 0x0b, 0xbc, 0x59, + 0x85, 0x15, 0x14, 0x8b, 0xdf, 0xca, 0x44, 0xcc, 0xcc, 0xc8, 0xd3, 0x4a, + 0x47, 0x93, 0xc1, 0xc4, 0x12, 0x62, 0xa8, 0x98, 0xb3, 0xd4, 0x57, 0xb6, + 0x87, 0x18, 0x5d, 0xa2, 0x17, 0xf9, 0x3b, 0x23, 0x90, 0xf6, 0xb6, 0x42, + 0x26, 0x98, 0xfa, 0xdf, 0x31, 0xa7, 0xa7, 0x9e, 0xb8, 0x43, 0x29, 0x88, + 0x0e, 0x3e, 0x2f, 0xb3, 0x8d, 0x4f, 0xf5, 0x0e, 0x0a, 0x36, 0xbd, 0x7d, + 0x4f, 0xb9, 0xa4, 0x40, 0xae, 0x80, 0xf2, 0x72, 0x11, 0xe9, 0x37, 0x2b, + 0x88, 0x8c, 0x7f, 0x51, 0x5c, 0x18, 0x83, 0x35, 0xfc, 0x30, 0xfa, 0xc8, + 0xc3, 0xbc, 0x3a, 0xeb, 0x2c, 0x4a, 0x51, 0x6b, 0x71, 0xf8, 0x8d, 0xbd, + 0x3e, 0x60, 0xf9, 0x10, 0xdd, 0xfb, 0x05, 0x06, 0x06, 0x9a, 0xad, 0x8f, + 0xdc, 0x8b, 0x2e, 0xc0, 0x90, 0xb0, 0x67, 0xdb, 0xc3, 0x4a, 0x10, 0x76, + 0xe3, 0xd9, 0x84, 0xb7, 0x57, 0xf8, 0xbf, 0x63, 0xa0, 0x9e, 0x4d, 0x32, + 0x28, 0xe7, 0x4b, 0x11, 0x7f, 0xa8, 0xab, 0x63, 0xff, 0xca, 0x2f, 0x87, + 0xe3, 0x64, 0x46, 0x28, 0xc0, 0x39, 0x64, 0xf9, 0xb9, 0x84, 0x23, 0x23, + 0x0e, 0x27, 0x37, 0xe6, 0x6a, 0xba, 0x14, 0x41, 0x87, 0x43, 0x76, 0x3c, + 0x5f, 0x69, 0x74, 0x66, 0xd5, 0x21, 0x66, 0x57, 0x63, 0x87, 0x7b, 0x2a, + 0xe0, 0x90, 0x24, 0xfa, 0x33, 0x36, 0xde, 0x8d, 0x3f, 0x8a, 0x89, 0xef, + 0xbe, 0xfe, 0xc3, 0x81, 0x9e, 0xef, 0x81, 0xd0, 0xe4, 0x22, 0xaf, 0xc0, + 0x43, 0x80, 0x57, 0x3e, 0x2c, 0x2b, 0x8e, 0xd4, 0xbc, 0xac, 0xf2, 0x92, + 0x3e, 0xf8, 0xd7, 0xcb, 0xd4, 0x5b, 0xe5, 0x66, 0x7d, 0xc3, 0x8c, 0xf3, + 0xfe, 0xe4, 0xe7, 0x8b, 0xec, 0x39, 0xe6, 0x8f, 0x80, 0xa5, 0xc7, 0xdc, + 0x5c, 0x89, 0xbd, 0x7d, 0xa2, 0x08, 0xee, 0xc9, 0x82, 0x0e, 0x88, 0xa3, + 0x6b, 0x72, 0x16, 0xaa, 0xbd, 0x2f, 0x37, 0xf8, 0x64, 0x51, 0x3e, 0xa6, + 0xd6, 0x82, 0x02, 0x81, 0x52, 0xd7, 0x45, 0xe3, 0x1a, 0xfb, 0x2c, 0xb1, + 0x8b, 0xea, 0x33, 0x4b, 0xc5, 0x8e, 0xaf, 0xf2, 0x61, 0x40, 0x55, 0xac, + 0x10, 0x59, 0x29, 0xbe, 0xa9, 0x68, 0x60, 0x0e, 0x93, 0xd8, 0xfb, 0x96, + 0x77, 0xd1, 0xc4, 0x5b, 0xb7, 0x32, 0xcb, 0xc7, 0x0f, 0x0d, 0xd6, 0x54, + 0x6b, 0x8a, 0x1e, 0x3b, 0x32, 0x11, 0x6c, 0xef, 0x4e, 0x1e, 0xdb, 0x1c, + 0xcf, 0x37, 0xe3, 0xab, 0x63, 0x90, 0x2f, 0xe3, 0x81, 0x9e, 0x09, 0x04, + 0xbe, 0xf7, 0xf6, 0xda, 0x21, 0x5c, 0xf9, 0x9f, 0xf9, 0xf9, 0xef, 0x1f, + 0x0d, 0x1a, 0xd5, 0xb2, 0x0e, 0x0e, 0xac, 0x9e, 0x9e, 0x9e, 0x36, 0x9b, + 0x5d, 0x44, 0x74, 0x54, 0xc3, 0x31, 0x44, 0xc6, 0xb3, 0x91, 0x48, 0x8c, + 0xba, 0xe7, 0x15, 0xd9, 0x02, 0x85, 0x9b, 0xeb, 0x6b, 0x6e, 0xa2, 0x0c, + 0xa7, 0x5c, 0x55, 0xaa, 0x71, 0x40, 0xab, 0x41, 0xdc, 0xb1, 0x9f, 0xa1, + 0xce, 0x8c, 0x14, 0x20, 0x9c, 0x17, 0x7a, 0xb7, 0x12, 0xc3, 0xbd, 0x14, + 0xaf, 0xa7, 0x54, 0x38, 0x12, 0xae, 0xef, 0xad, 0x50, 0xcc, 0x4b, 0xbf, + 0xad, 0x94, 0xcd, 0x4e, 0xa1, 0x34, 0xbf, 0xdd, 0xe8, 0x05, 0x12, 0xba, + 0x45, 0xf9, 0xe2, 0x80, 0xf2, 0x2d, 0xe8, 0x91, 0xf8, 0xf8, 0x27, 0xe6, + 0xcb, 0x54, 0x8e, 0x16, 0x2a, 0x96, 0xe0, 0xd4, 0xf4, 0x14, 0xd4, 0xb6, + 0x5d, 0x21, 0x83, 0xeb, 0xfa, 0xbf, 0xdd, 0x9d, 0xae, 0xb6, 0xa0, 0xc8, + 0xa9, 0x68, 0x5d, 0xdf, 0x69, 0x06, 0x66, 0xdd, 0xcb, 0xc8, 0xbb, 0x91, + 0x6d, 0x75, 0xf5, 0xda, 0xd7, 0x1a, 0x5a, 0xf7, 0x36, 0x66, 0x9c, 0x7d, + 0xb9, 0xec, 0xd2, 0x5e, 0xd9, 0x37, 0xa0, 0xbe, 0x77, 0x3c, 0x38, 0xb0, + 0x93, 0x18, 0xfb, 0x03, 0xe4, 0xd2, 0xd9, 0x1f, 0x06, 0xf0, 0x0a, 0x3c, + 0x06, 0x1a, 0xf4, 0x32, 0xea, 0x12, 0xda, 0xac, 0xd7, 0xe1, 0x90, 0x0a, + 0xbd, 0x7d, 0xa3, 0x41, 0xbd, 0x57, 0xdd, 0x93, 0x93, 0x94, 0xfc, 0x75, + 0xb1, 0x01, 0xfc, 0x6f, 0x35, 0x72, 0xe6, 0xa4, 0x7c, 0xea, 0x07, 0xe2, + 0x9d, 0x82, 0xad, 0x9e, 0x74, 0x5a, 0x71, 0xa1, 0x91, 0x4d, 0x9b, 0x88, + 0xad, 0x06, 0x73, 0x14, 0xda, 0x06, 0x5b, 0x93, 0xe1, 0xea, 0x89, 0xc0, + 0x25, 0xd9, 0x3f, 0xc9, 0xf1, 0x91, 0x04, 0xd3, 0xb6, 0xea, 0xe7, 0xc1, + 0xd0, 0xa4, 0x84, 0xbd, 0xb7, 0xb0, 0xe9, 0x59, 0xa7, 0xe7, 0x0d, 0x39, + 0x08, 0xd4, 0x1d, 0x1d, 0x1d, 0x15, 0x9a, 0x9a, 0x5c, 0x1d, 0xbf, 0x80, + 0x98, 0x81, 0x08, 0x8c, 0x9f, 0xa5, 0xee, 0x6a, 0x68, 0x6f, 0x6d, 0xbc, + 0x3c, 0xa3, 0x0e, 0x8b, 0x50, 0xc9, 0xc8, 0xe4, 0x8c, 0x01, 0x8e, 0xab, + 0xa2, 0xfd, 0xac, 0x95, 0x73, 0x7d, 0x38, 0xd1, 0x84, 0x3c, 0x6d, 0x76, + 0xaa, 0x6a, 0x02, 0x35, 0xce, 0x44, 0xc8, 0x30, 0x89, 0xd4, 0x72, 0x46, + 0x2b, 0xee, 0x51, 0xd2, 0x83, 0x37, 0xf3, 0x1d, 0xf9, 0xda, 0x79, 0xf7, + 0x96, 0x4b, 0x9a, 0x62, 0x2c, 0xa6, 0x16, 0x73, 0x26, 0x64, 0x3c, 0x3c, + 0x0c, 0x2a, 0x6a, 0x4d, 0xe7, 0x72, 0x80, 0xcd, 0x09, 0x1d, 0xab, 0xba, + 0x4b, 0xf2, 0x41, 0x70, 0xc5, 0xc7, 0x5a, 0x8c, 0x21, 0xee, 0x44, 0x16, + 0xdf, 0x43, 0x27, 0x7f, 0x60, 0xa6, 0x5d, 0x3b, 0x1c, 0x1d, 0x15, 0xf4, + 0x4f, 0xd9, 0xd6, 0x3d, 0x32, 0x8d, 0xa0, 0x95, 0x10, 0xf7, 0x3a, 0xcf, + 0x20, 0x48, 0xed, 0xb0, 0x55, 0xea, 0xb8, 0x0b, 0x70, 0xe9, 0xc8, 0x20, + 0xee, 0xd6, 0xe4, 0x60, 0x52, 0xf9, 0xdc, 0x1c, 0xa2, 0x94, 0xc4, 0x83, + 0xaf, 0xb4, 0xeb, 0xbc, 0xe0, 0x44, 0xd5, 0x30, 0x21, 0x04, 0x4d, 0xc6, + 0xc6, 0x96, 0x30, 0x95, 0xc5, 0x87, 0xf3, 0xd4, 0x59, 0x59, 0xc9, 0x68, + 0xb5, 0x50, 0x1e, 0x82, 0x1a, 0x38, 0x96, 0xec, 0x8b, 0xa0, 0xa6, 0xfe, + 0x5a, 0xb7, 0x9f, 0x10, 0xa6, 0xab, 0x84, 0x66, 0x18, 0xc2, 0xe4, 0xf5, + 0x05, 0x72, 0xd1, 0x6c, 0xf2, 0x06, 0xe9, 0x17, 0x7d, 0xdd, 0x0b, 0xb0, + 0x5b, 0x43, 0x1b, 0xd3, 0x52, 0x20, 0x0e, 0xa6, 0xfa, 0x54, 0xd6, 0xeb, + 0x6b, 0x7e, 0x97, 0x93, 0xc5, 0xb3, 0xbf, 0x53, 0x70, 0x08, 0x08, 0xc6, + 0x03, 0xe1, 0x68, 0x03, 0xc3, 0xc3, 0xd0, 0x97, 0x10, 0x65, 0xda, 0x5d, + 0x40, 0x0a, 0x6e, 0x6d, 0x84, 0x2e, 0x2b, 0xb6, 0xdc, 0x99, 0xc8, 0x69, + 0x8a, 0x9f, 0xc3, 0x40, 0x86, 0x9b, 0xc5, 0x9e, 0x95, 0xa1, 0xab, 0xaf, + 0x27, 0x4e, 0xe9, 0xed, 0x8d, 0xb6, 0x36, 0x35, 0x03, 0xa7, 0xbd, 0xe9, + 0x7b, 0x6d, 0x8e, 0xb1, 0x68, 0x88, 0xc4, 0x29, 0xd7, 0x83, 0x7b, 0x94, + 0xdc, 0x4e, 0x79, 0x46, 0x1e, 0xaf, 0x76, 0x21, 0x92, 0xa6, 0x63, 0x0f, + 0x4c, 0xde, 0xdb, 0x33, 0xa1, 0x56, 0xc8, 0x96, 0x5d, 0x6d, 0xb6, 0x8d, + 0x57, 0x4e, 0xc8, 0xda, 0xed, 0x2a, 0x54, 0x5f, 0x67, 0x95, 0x25, 0xef, + 0x17, 0x45, 0x78, 0x4b, 0x50, 0x5e, 0x85, 0x14, 0x93, 0x84, 0xdd, 0x79, + 0x28, 0x51, 0xa4, 0xf2, 0xf9, 0x3a, 0x96, 0x75, 0x4b, 0xf8, 0xfa, 0x0b, + 0xb2, 0x8e, 0x3a, 0xf0, 0x33, 0xea, 0x9e, 0xcb, 0x7f, 0x6b, 0x9e, 0x7d, + 0x72, 0x82, 0x02, 0xa7, 0x24, 0x8f, 0xfd, 0x45, 0x61, 0x85, 0x25, 0x25, + 0x61, 0xff, 0x53, 0xe8, 0xad, 0x9e, 0xdf, 0x9d, 0x4f, 0x07, 0x99, 0xa5, + 0x91, 0x22, 0x16, 0xb5, 0x37, 0x09, 0x5d, 0x0c, 0x35, 0x1a, 0x25, 0x56, + 0x83, 0x7d, 0x12, 0x6b, 0xab, 0x53, 0x90, 0x91, 0xc2, 0x2c, 0x8b, 0x91, + 0xbe, 0xee, 0x4b, 0x3e, 0x0c, 0xeb, 0xc7, 0x87, 0xa9, 0xcc, 0x31, 0x99, + 0x61, 0x27, 0x62, 0xe1, 0x88, 0xe4, 0xc2, 0xf6, 0x9e, 0xe6, 0xa4, 0x34, + 0x78, 0xe7, 0xd8, 0x5b, 0x9d, 0x85, 0x12, 0xd8, 0xe5, 0xfe, 0x8c, 0x25, + 0xfe, 0x6f, 0xe9, 0x4c, 0x6b, 0xeb, 0x77, 0x7c, 0x2e, 0x1b, 0x49, 0xa0, + 0xd7, 0x9d, 0xc7, 0x65, 0x2f, 0x8c, 0x56, 0xbd, 0xc9, 0xa8, 0x25, 0xdc, + 0x66, 0x9b, 0xb3, 0x79, 0x7d, 0x7d, 0xbd, 0x66, 0xed, 0x2f, 0x28, 0xf4, + 0x88, 0x60, 0x19, 0xeb, 0x59, 0xb9, 0xc4, 0x01, 0x2f, 0xb8, 0x90, 0x9c, + 0x7f, 0xd9, 0x69, 0x8f, 0xe9, 0xe2, 0xb7, 0x73, 0x88, 0x99, 0x55, 0x82, + 0x91, 0xe8, 0x77, 0xc2, 0xd9, 0xc3, 0x14, 0xbc, 0x82, 0xac, 0xc1, 0x14, + 0x65, 0x47, 0x29, 0xa6, 0xdd, 0x00, 0x58, 0x32, 0x49, 0x48, 0x69, 0x1d, + 0xa2, 0x74, 0x9b, 0x8d, 0xc7, 0x00, 0xf9, 0xf9, 0x9e, 0x31, 0x82, 0x77, + 0x28, 0xf6, 0x03, 0xbf, 0x20, 0xa4, 0x09, 0x5d, 0xf0, 0xc9, 0x3e, 0x38, + 0x1e, 0x0f, 0x0f, 0x2b, 0x75, 0x48, 0x53, 0x0e, 0xc7, 0x12, 0xed, 0x37, + 0x4c, 0x0b, 0x16, 0xe8, 0x75, 0xb8, 0xad, 0xf9, 0x3e, 0xac, 0xbf, 0x3f, + 0x9b, 0x86, 0xd4, 0xd9, 0x2c, 0xb3, 0x7d, 0x80, 0x61, 0xce, 0x0f, 0x15, + 0xe9, 0x37, 0xa6, 0x89, 0x5a, 0x1e, 0xcd, 0x17, 0x28, 0xd3, 0xd4, 0xde, + 0x5f, 0x6f, 0x7e, 0xcf, 0xbd, 0xe9, 0x33, 0x2e, 0xdc, 0xaa, 0xf8, 0xd1, + 0x78, 0xec, 0x5a, 0xab, 0x57, 0xd0, 0x76, 0x7d, 0xd8, 0x60, 0xb6, 0x31, + 0xa3, 0x2c, 0x7a, 0x65, 0x38, 0xb7, 0xec, 0x55, 0x70, 0xd9, 0x14, 0x5c, + 0xc4, 0x39, 0xae, 0xad, 0x9e, 0x7d, 0xed, 0x7a, 0xdc, 0x64, 0x99, 0xaf, + 0x50, 0xcb, 0x7a, 0xca, 0xe9, 0x12, 0xb1, 0xf2, 0xc2, 0xe6, 0x92, 0xa3, + 0xe2, 0xa0, 0x84, 0x8d, 0x1d, 0x8e, 0xb2, 0x94, 0x46, 0x04, 0x51, 0x8e, + 0x58, 0x9a, 0x1f, 0xa6, 0x4d, 0x3f, 0x9a, 0x65, 0x4f, 0xe7, 0xd3, 0x50, + 0x53, 0x5f, 0x0f, 0xe2, 0x18, 0xc4, 0xa5, 0xa5, 0xc1, 0xf6, 0x6c, 0xf7, + 0x87, 0x95, 0x69, 0xd6, 0x91, 0x47, 0x92, 0xe3, 0x72, 0x9f, 0x3a, 0xae, + 0x34, 0x59, 0xe9, 0xd2, 0x2f, 0x0a, 0xfd, 0x6d, 0x6d, 0x1b, 0x6c, 0x46, + 0xec, 0x5c, 0x82, 0xd0, 0xc0, 0xfe, 0x4c, 0x63, 0xfb, 0x7c, 0xf9, 0x9d, + 0xf1, 0xcc, 0x62, 0x15, 0xa7, 0x93, 0xf0, 0xd8, 0xfb, 0x83, 0x3b, 0x96, + 0xef, 0xc5, 0x51, 0x70, 0xb5, 0x4e, 0xc7, 0xda, 0xd4, 0x42, 0x46, 0x4b, + 0x37, 0x18, 0x4f, 0xf9, 0xe6, 0xe6, 0x48, 0xa3, 0x0a, 0x77, 0xfc, 0x21, + 0x27, 0x7a, 0x82, 0x84, 0x0e, 0xc7, 0xfc, 0x3b, 0x8a, 0xba, 0xed, 0xbf, + 0xa9, 0x91, 0x87, 0x2e, 0x93, 0xe9, 0xdf, 0x14, 0x80, 0xef, 0x84, 0xc8, + 0xa7, 0x6e, 0x7c, 0x5a, 0x5a, 0x5a, 0xc1, 0x21, 0x5f, 0x6f, 0x06, 0x57, + 0xa7, 0x95, 0x81, 0xca, 0x36, 0x65, 0x5a, 0x93, 0x1e, 0x8c, 0x02, 0x89, + 0xf7, 0x0f, 0xfc, 0x2a, 0x5b, 0xb8, 0xf6, 0x5f, 0x1b, 0xf2, 0xc4, 0x6f, + 0xc7, 0x25, 0x94, 0x44, 0x05, 0x93, 0xd8, 0x4a, 0x58, 0x7a, 0xa2, 0xa0, + 0xbd, 0x40, 0x6f, 0x29, 0xd6, 0xb5, 0x9d, 0x8f, 0xf9, 0x3a, 0xce, 0x01, + 0xd4, 0x59, 0xc4, 0x14, 0xc5, 0xba, 0x9f, 0xf3, 0xc8, 0x7a, 0x27, 0x5e, + 0x6e, 0xb1, 0xae, 0xf9, 0x37, 0xc8, 0xe3, 0x55, 0xa4, 0xa5, 0xe2, 0x40, + 0xbd, 0x8d, 0x6d, 0x51, 0xe9, 0x82, 0xf6, 0x74, 0xc5, 0x49, 0x10, 0xfc, + 0x2a, 0x7e, 0x5b, 0x90, 0xc3, 0x7d, 0xdf, 0x03, 0x9a, 0x98, 0x81, 0x53, + 0xca, 0xa9, 0xc6, 0xb7, 0xf3, 0x7a, 0xe7, 0x3b, 0xef, 0xbb, 0x3e, 0xe5, + 0x5f, 0x7c, 0x69, 0x68, 0x87, 0x0c, 0x3d, 0x1e, 0xb2, 0x09, 0xb3, 0x31, + 0x16, 0x90, 0xf6, 0x72, 0x24, 0x49, 0x3f, 0x2c, 0xae, 0x23, 0x42, 0xff, + 0xc9, 0xa5, 0x3f, 0xf7, 0x2c, 0x2b, 0xe8, 0xe9, 0x72, 0x41, 0x98, 0xc3, + 0xd6, 0xe3, 0x4b, 0xa8, 0x65, 0xfe, 0x09, 0x4e, 0x0a, 0xbf, 0x88, 0x5a, + 0xb2, 0x72, 0x87, 0x2c, 0x0f, 0xc7, 0x3f, 0xa9, 0xa0, 0x4c, 0x62, 0x6a, + 0x96, 0x69, 0x04, 0x8b, 0x0a, 0x61, 0x7b, 0xa5, 0x8c, 0xcd, 0x58, 0xb5, + 0x90, 0x5a, 0xd4, 0xd3, 0xfb, 0xe7, 0x52, 0xab, 0xbd, 0xb8, 0x40, 0xe4, + 0xcb, 0x6d, 0xf0, 0xd0, 0x4e, 0xad, 0xf5, 0xe2, 0x96, 0x2e, 0x5a, 0x2d, + 0xfb, 0x13, 0xea, 0x6e, 0xa8, 0x80, 0x7c, 0x01, 0x2f, 0xe7, 0xa5, 0x87, + 0x2b, 0xe2, 0x6e, 0x5f, 0xa2, 0x78, 0xd2, 0x01, 0x10, 0xd6, 0x93, 0x98, + 0x25, 0xc2, 0x75, 0x07, 0x41, 0x5b, 0xfb, 0x58, 0x63, 0x18, 0x02, 0x6b, + 0x9f, 0x63, 0xcd, 0x11, 0x42, 0xe3, 0xaf, 0xbf, 0x94, 0xf2, 0x01, 0x90, + 0x89, 0xe4, 0x76, 0xe3, 0xc1, 0x43, 0x79, 0xb4, 0x29, 0x69, 0x53, 0xa1, + 0xbc, 0x70, 0x6a, 0x16, 0x1b, 0xad, 0xf2, 0x78, 0xd7, 0xbf, 0x64, 0x77, + 0x8c, 0xc6, 0xc1, 0x61, 0x3e, 0xf7, 0xfb, 0xd8, 0xb3, 0x2c, 0xa2, 0x16, + 0xda, 0x13, 0x14, 0x5f, 0x72, 0xfc, 0xf5, 0x1a, 0x22, 0xf4, 0x20, 0x89, + 0x2f, 0x52, 0xe7, 0x50, 0x1e, 0x1f, 0x12, 0x78, 0x15, 0xb2, 0x77, 0x79, + 0x59, 0xc1, 0x94, 0xff, 0x3f, 0x00, 0x5b, 0x14, 0xa4, 0xeb, 0x5f, 0x76, + 0xbd, 0xb5, 0x7e, 0x78, 0x11, 0x84, 0xff, 0x5f, 0xc1, 0x9b, 0x7c, 0xee, + 0xf5, 0x4f, 0x2c, 0xa1, 0x58, 0x38, 0x2d, 0x84, 0x50, 0xc3, 0x32, 0x33, + 0x7f, 0xb3, 0x88, 0x7e, 0xb9, 0x41, 0x55, 0xeb, 0x02, 0x99, 0xf7, 0x0a, + 0x61, 0x13, 0x00, 0xdb, 0x76, 0xed, 0xc3, 0xe5, 0xf6, 0x58, 0x82, 0xb1, + 0xf0, 0x8f, 0x2d, 0xcf, 0x9f, 0x25, 0xf0, 0x7d, 0xf4, 0x6a, 0x36, 0x5b, + 0x43, 0x8b, 0xe8, 0x97, 0x1b, 0x0c, 0xe3, 0xaf, 0xfc, 0x4f, 0x11, 0x37, + 0x14, 0xfc, 0x36, 0x7c, 0xdc, 0xe7, 0xd6, 0xa0, 0x9c, 0x85, 0x13, 0x88, + 0x6e, 0xf3, 0x25, 0xd2, 0x10, 0x40, 0x13, 0xaf, 0x77, 0x93, 0x45, 0xf4, + 0xcb, 0xf7, 0x65, 0xa6, 0x2e, 0x00, 0xc7, 0x34, 0x40, 0xfe, 0xf2, 0xdb, + 0x5a, 0x1c, 0x76, 0x9b, 0x45, 0x76, 0x0b, 0xa7, 0x70, 0xdd, 0x83, 0xbe, + 0x2d, 0xed, 0x6d, 0xb4, 0x88, 0x7e, 0xc6, 0x17, 0x18, 0x8d, 0xcd, 0x16, + 0x3b, 0x08, 0x10, 0x86, 0x61, 0xf2, 0xf3, 0xaf, 0xeb, 0xac, 0x3c, 0x73, + 0x16, 0xf2, 0xc3, 0x3b, 0x85, 0xb4, 0xe3, 0x59, 0xf9, 0x06, 0xc1, 0xdc, + 0x62, 0x11, 0xfd, 0xf2, 0xb6, 0xe8, 0x78, 0xbd, 0xfb, 0x0e, 0x0b, 0x11, + 0xf5, 0x05, 0x20, 0x9f, 0x79, 0xed, 0x23, 0xcb, 0xa2, 0x5b, 0x00, 0x7c, + 0x05, 0x29, 0x0e, 0x1e, 0x4d, 0xf3, 0x91, 0x48, 0x09, 0xca, 0xb0, 0x88, + 0x7e, 0xb9, 0xbf, 0x50, 0xad, 0x0e, 0x52, 0xf2, 0x28, 0x20, 0xf2, 0x3c, + 0x5e, 0x3c, 0x5e, 0xaf, 0x25, 0x14, 0x0b, 0x18, 0x86, 0x49, 0x79, 0x7f, + 0x4a, 0xa9, 0xb0, 0xdd, 0x16, 0xd1, 0x2f, 0x73, 0xe8, 0xfa, 0x36, 0x20, + 0x3d, 0x57, 0xd3, 0xe2, 0x46, 0x03, 0xf2, 0xf5, 0x0f, 0xbe, 0xb6, 0x84, + 0x62, 0x01, 0x21, 0x04, 0xde, 0xfc, 0x4d, 0x4f, 0xa6, 0x99, 0x61, 0xad, + 0x8c, 0x0b, 0x04, 0x44, 0x46, 0xb6, 0x41, 0xd7, 0x53, 0x5e, 0x06, 0x21, + 0x66, 0xfe, 0xf8, 0x9b, 0xb4, 0x6a, 0xb7, 0x59, 0xd0, 0x0d, 0xa3, 0x50, + 0x21, 0x87, 0x90, 0x83, 0x16, 0xd1, 0x03, 0x00, 0x19, 0x19, 0xcb, 0x00, + 0xc3, 0x25, 0x44, 0xf8, 0x1b, 0x80, 0x70, 0xb9, 0xf3, 0x2c, 0xa1, 0x58, + 0x16, 0x9d, 0x7f, 0xf4, 0xc0, 0x65, 0x59, 0xf4, 0x40, 0x81, 0xaa, 0x56, + 0x05, 0x8e, 0xbf, 0x00, 0xc8, 0x29, 0x33, 0x7e, 0xb1, 0x76, 0xb4, 0x95, + 0x75, 0x7d, 0x50, 0x04, 0x19, 0x59, 0x39, 0x00, 0x68, 0x9a, 0xd2, 0xc9, + 0x22, 0x7a, 0x80, 0xc0, 0x30, 0xf6, 0xa1, 0xaa, 0x09, 0x06, 0x08, 0xf9, + 0xc9, 0xb4, 0xf9, 0x64, 0xe5, 0xb8, 0x2c, 0xa1, 0x94, 0xe9, 0x8e, 0x5f, + 0xe5, 0x58, 0x7a, 0x26, 0x80, 0xd4, 0xf5, 0xf4, 0x04, 0x8b, 0xe8, 0x81, + 0x14, 0x97, 0xe9, 0x3b, 0x25, 0x04, 0x2d, 0x2e, 0x70, 0xdd, 0x2c, 0x94, + 0x5d, 0x98, 0xa6, 0x49, 0x48, 0x90, 0x03, 0x40, 0x80, 0x50, 0x2d, 0xa2, + 0x07, 0x18, 0x14, 0x25, 0xe4, 0x13, 0x00, 0xb7, 0xc7, 0x63, 0xcd, 0xa9, + 0x97, 0x61, 0x78, 0x75, 0x83, 0x6a, 0x95, 0x63, 0x0b, 0xfc, 0xbd, 0x3a, + 0x16, 0xd1, 0x03, 0x0c, 0x76, 0x7b, 0xad, 0x9f, 0x01, 0x16, 0xaf, 0xd8, + 0x80, 0xa2, 0x58, 0x56, 0xbd, 0xac, 0x42, 0x4a, 0x89, 0xd3, 0x6e, 0xcf, + 0xff, 0xa6, 0x25, 0x5b, 0x44, 0x0f, 0x20, 0x04, 0x05, 0xb5, 0xc6, 0xed, + 0x5e, 0x9e, 0x0a, 0x98, 0x6f, 0x7d, 0x32, 0xdd, 0x4a, 0x48, 0x51, 0xa6, + 0x3d, 0x3b, 0x85, 0xcc, 0xec, 0xdc, 0x02, 0xfb, 0x1e, 0x65, 0x11, 0x3d, + 0x80, 0xe0, 0x72, 0xf9, 0x13, 0x89, 0x18, 0xb9, 0xae, 0x3c, 0x0e, 0x1e, + 0x39, 0x66, 0xb9, 0xef, 0x65, 0xd8, 0xa2, 0x87, 0x85, 0x06, 0x03, 0x20, + 0x84, 0x23, 0xd3, 0x22, 0x7a, 0x60, 0x42, 0x07, 0xe4, 0xec, 0x9f, 0x57, + 0xa0, 0x5a, 0x56, 0xbd, 0xcc, 0xc2, 0x34, 0xcd, 0x7c, 0xd2, 0x7b, 0x6b, + 0x5a, 0x44, 0x0f, 0x30, 0xd8, 0x6c, 0xf5, 0x01, 0xc7, 0x6e, 0x40, 0x4c, + 0xf8, 0x62, 0x0e, 0x96, 0x3d, 0x2f, 0x9b, 0x10, 0x80, 0xc7, 0xab, 0x17, + 0x7c, 0xb5, 0x36, 0xb5, 0x04, 0x1a, 0xbc, 0xde, 0x2d, 0x80, 0x3a, 0xaf, + 0xe0, 0xfb, 0x5f, 0x3b, 0xf7, 0x59, 0xee, 0x7b, 0x59, 0x24, 0xba, 0x22, + 0xc8, 0x75, 0xb9, 0x01, 0x24, 0x08, 0xdd, 0x22, 0x7a, 0x00, 0x42, 0x51, + 0x42, 0x56, 0x17, 0x7c, 0x9e, 0xf5, 0xe3, 0x6f, 0x68, 0x9a, 0x6a, 0x09, + 0xa5, 0x2c, 0xc6, 0xe9, 0x7e, 0xe3, 0xae, 0xae, 0xb4, 0xd9, 0xea, 0x58, + 0x44, 0x0f, 0xbc, 0xd8, 0x2c, 0xc5, 0x9f, 0x51, 0x64, 0xea, 0x77, 0x8b, + 0x30, 0x4d, 0xcb, 0xa2, 0x97, 0xb9, 0x10, 0x4e, 0xd3, 0x48, 0xde, 0xe7, + 0x2b, 0x1f, 0x25, 0x84, 0xb6, 0xdf, 0xeb, 0xdd, 0x66, 0x11, 0x3d, 0x40, + 0x9d, 0x37, 0xff, 0xce, 0x96, 0x1d, 0x7b, 0x0e, 0x5a, 0xe2, 0x28, 0x6b, + 0xd6, 0x5c, 0x4a, 0xbf, 0x27, 0x27, 0xa5, 0x5e, 0xbd, 0xb4, 0xb7, 0xd7, + 0x22, 0xfa, 0xf9, 0x23, 0xa7, 0xe0, 0xc3, 0xae, 0xbd, 0x87, 0x2c, 0x69, + 0x94, 0x31, 0xa8, 0xaa, 0x42, 0x46, 0x66, 0x76, 0x81, 0x45, 0xaf, 0x66, + 0x11, 0x3d, 0x00, 0xe1, 0x74, 0x5e, 0x89, 0xa2, 0x84, 0x7d, 0x56, 0xf0, + 0xfd, 0xcf, 0xbf, 0x76, 0x59, 0xd3, 0x6c, 0x65, 0xd0, 0xa2, 0x3b, 0x1d, + 0xb6, 0x7c, 0xa2, 0xab, 0x5b, 0x2d, 0xa2, 0x07, 0x20, 0xdc, 0xee, 0xb5, + 0x98, 0x66, 0xe6, 0x53, 0x05, 0xdf, 0x97, 0xfe, 0xbe, 0xd1, 0x1a, 0x90, + 0x2b, 0x83, 0x31, 0x7a, 0x41, 0x05, 0x1f, 0x29, 0x75, 0x6b, 0x7a, 0x2d, + 0xc0, 0xe3, 0xf4, 0x14, 0x80, 0x3d, 0x07, 0x8e, 0x94, 0xea, 0x82, 0x80, + 0x16, 0x8a, 0x1f, 0x3b, 0xf6, 0x1c, 0x44, 0x9a, 0x12, 0x10, 0x48, 0x99, + 0xfb, 0x91, 0x45, 0xf4, 0x80, 0x86, 0x7d, 0x6a, 0xc1, 0xa7, 0xd5, 0x1b, + 0xb7, 0x5b, 0xf3, 0xe9, 0x65, 0x08, 0xbf, 0xaf, 0x2f, 0x28, 0xf2, 0xa1, + 0xae, 0xbd, 0x1c, 0xda, 0x6b, 0x11, 0xfd, 0x02, 0xa0, 0x69, 0x15, 0x37, + 0x14, 0x7c, 0xfe, 0xee, 0x97, 0x15, 0x96, 0xfb, 0x5e, 0x86, 0x90, 0x96, + 0xe1, 0xcb, 0xe9, 0xae, 0x28, 0x36, 0xc5, 0x22, 0x7a, 0x80, 0x43, 0xd7, + 0x93, 0x3f, 0x2d, 0xf8, 0xbc, 0x69, 0x7b, 0x32, 0x36, 0x8b, 0xe8, 0x65, + 0x02, 0x52, 0x4a, 0x92, 0xf7, 0x1f, 0x2e, 0xf8, 0xb8, 0xdd, 0x22, 0x7a, + 0xd9, 0x88, 0xd3, 0x0d, 0x80, 0xdd, 0x7b, 0x0f, 0xa3, 0x69, 0x9a, 0x25, + 0x8e, 0xb2, 0xe0, 0xc9, 0xa9, 0x2a, 0xfb, 0x0f, 0xa7, 0x02, 0x08, 0x90, + 0x3b, 0x2c, 0xa2, 0x97, 0x09, 0x28, 0x7b, 0x00, 0xbc, 0xba, 0x5e, 0x28, + 0x6e, 0xb3, 0x10, 0xd0, 0x5d, 0xbb, 0x10, 0xe8, 0xf9, 0x39, 0xdd, 0xa5, + 0x94, 0x1e, 0x8b, 0xe8, 0x65, 0x03, 0xfb, 0x0a, 0x3e, 0x04, 0x39, 0xec, + 0x96, 0x34, 0xca, 0x42, 0xd7, 0xae, 0x08, 0x72, 0x7c, 0x1b, 0x5a, 0x00, + 0x8e, 0x58, 0x44, 0x0f, 0x70, 0x44, 0x46, 0xde, 0x03, 0xd8, 0xfd, 0xd9, + 0x28, 0x72, 0x5d, 0x56, 0xbe, 0xf7, 0x32, 0xe1, 0xba, 0x6b, 0x2a, 0xa9, + 0x69, 0xc7, 0xf3, 0x49, 0x1f, 0xbe, 0xd3, 0x22, 0x7a, 0x80, 0x23, 0x23, + 0xe3, 0x4b, 0x84, 0x70, 0xfc, 0x56, 0xf0, 0x3d, 0x38, 0xd8, 0x69, 0x09, + 0xa5, 0x2c, 0xbc, 0xf7, 0xcc, 0x1c, 0xd2, 0x8f, 0xfb, 0x96, 0xbf, 0x9a, + 0xa6, 0xf8, 0xcb, 0x22, 0x7a, 0xd9, 0x40, 0x70, 0xc1, 0x87, 0xb4, 0xf4, + 0x4c, 0x4b, 0x1a, 0x65, 0x00, 0xfb, 0x0f, 0xa5, 0x14, 0x72, 0xe3, 0x8d, + 0x04, 0x8b, 0xe8, 0x65, 0x22, 0x5e, 0x8b, 0x5f, 0x5a, 0xf0, 0xf9, 0x70, + 0x6a, 0xba, 0x25, 0x90, 0x00, 0x87, 0x94, 0x92, 0x95, 0xeb, 0xfe, 0x59, + 0xda, 0xae, 0xaa, 0xc1, 0xe9, 0x16, 0xd1, 0xcb, 0x00, 0x0c, 0x63, 0x9d, + 0x7f, 0x30, 0x26, 0x24, 0xc8, 0x72, 0xdd, 0x03, 0x1d, 0xaa, 0xaa, 0x30, + 0x67, 0xc1, 0x4a, 0x3f, 0x7d, 0xbc, 0xde, 0x7d, 0x1b, 0x2c, 0xa2, 0x97, + 0x19, 0x88, 0xe3, 0x00, 0xbb, 0xf6, 0x59, 0xdb, 0x55, 0x03, 0x1d, 0x76, + 0x4d, 0xe3, 0x68, 0xaa, 0x6f, 0x0f, 0x8b, 0x10, 0xda, 0xee, 0xcb, 0xa5, + 0xdd, 0x16, 0xd1, 0x8b, 0x07, 0x39, 0x00, 0x91, 0xe1, 0xa1, 0x96, 0x24, + 0x02, 0xbd, 0x4b, 0x57, 0x04, 0x1e, 0xaf, 0xb7, 0x80, 0x3e, 0x0b, 0x2d, + 0xa2, 0x97, 0x2d, 0x84, 0x02, 0x24, 0xd5, 0xab, 0x69, 0x49, 0x22, 0xd0, + 0x5d, 0x77, 0x45, 0x25, 0xdb, 0x5f, 0x60, 0xd3, 0x31, 0x23, 0x32, 0xb2, + 0xbd, 0x45, 0xf4, 0xb2, 0x03, 0x25, 0x03, 0xa0, 0x7c, 0x54, 0xb8, 0x25, + 0x8a, 0x00, 0xc7, 0xcc, 0x9f, 0xfc, 0xb3, 0xa9, 0xd8, 0x6c, 0x6d, 0x96, + 0x65, 0x64, 0x2c, 0xb0, 0x88, 0x5e, 0x86, 0xb0, 0xc7, 0x12, 0x41, 0xe0, + 0x43, 0x4a, 0xc9, 0x8c, 0xef, 0x97, 0xe5, 0x7f, 0xd3, 0xf0, 0x78, 0xe6, + 0x66, 0x5b, 0xae, 0x7b, 0x19, 0xd3, 0x01, 0x80, 0x4a, 0x15, 0xca, 0x5b, + 0x92, 0x08, 0x64, 0xb7, 0x5d, 0x55, 0x0a, 0xcd, 0xa1, 0xcb, 0x23, 0x97, + 0x53, 0xdb, 0x2d, 0xa2, 0x17, 0x0b, 0x7c, 0x09, 0xfc, 0xad, 0x2c, 0x33, + 0x65, 0x20, 0x3e, 0xcf, 0x2d, 0x58, 0xe3, 0xae, 0x6e, 0xb7, 0x88, 0x5e, + 0xd6, 0x68, 0x2e, 0x94, 0x63, 0x60, 0xad, 0x75, 0x2f, 0x0b, 0x8e, 0x9b, + 0xf0, 0x55, 0xca, 0x96, 0xaa, 0x1a, 0xfe, 0x8d, 0x45, 0xf4, 0xb2, 0x17, + 0xbb, 0xe5, 0x01, 0xa4, 0x67, 0x66, 0x5b, 0xc2, 0x08, 0x60, 0x78, 0xbc, + 0x7a, 0xc1, 0x1a, 0x77, 0x61, 0x9a, 0xae, 0xed, 0x35, 0x6a, 0xbc, 0x69, + 0x11, 0xbd, 0x8c, 0x39, 0x75, 0x87, 0x7c, 0xae, 0xbb, 0x61, 0x89, 0x22, + 0x70, 0x3b, 0x73, 0x3e, 0x9a, 0x3a, 0xdf, 0xff, 0x5d, 0xd3, 0xe2, 0xff, + 0x48, 0x4e, 0x7e, 0xda, 0x22, 0x7a, 0x19, 0x53, 0x83, 0x4a, 0x00, 0x79, + 0x1e, 0xaf, 0x25, 0x8a, 0x00, 0x85, 0xa2, 0x08, 0xa6, 0x7d, 0xb7, 0x28, + 0xff, 0x73, 0xe8, 0xef, 0x5e, 0xef, 0xa6, 0x63, 0x96, 0xeb, 0x5e, 0xe6, + 0x60, 0xda, 0x7c, 0xbd, 0xbe, 0x25, 0x89, 0x80, 0xf5, 0xd9, 0x14, 0x15, + 0x97, 0xdb, 0x97, 0x4c, 0xc6, 0x34, 0x8d, 0xff, 0xbb, 0xec, 0x3a, 0x2a, + 0xeb, 0x15, 0x16, 0x1f, 0xc2, 0x42, 0x82, 0x2c, 0x21, 0x04, 0x28, 0x32, + 0xb3, 0x73, 0x70, 0xe7, 0x79, 0x00, 0xa4, 0xa2, 0x38, 0x0d, 0x8b, 0xe8, + 0x65, 0x10, 0x42, 0xd8, 0x36, 0x01, 0x6c, 0xdb, 0xb5, 0xcf, 0x12, 0x46, + 0x80, 0x62, 0xef, 0x41, 0xff, 0x1e, 0x74, 0x21, 0xa5, 0xfc, 0x31, 0x28, + 0xa8, 0xbd, 0x45, 0xf4, 0x32, 0x17, 0xa1, 0x4b, 0xaf, 0x0e, 0xe0, 0x74, + 0x38, 0x2c, 0x61, 0x04, 0xe4, 0xfb, 0x95, 0xfc, 0xb8, 0x74, 0x0d, 0x80, + 0x14, 0xc2, 0x81, 0x94, 0x19, 0xbb, 0x5c, 0xae, 0x05, 0x16, 0xd1, 0xcb, + 0x12, 0x9c, 0xce, 0x26, 0x08, 0xa1, 0xd4, 0x05, 0xc8, 0xc9, 0x75, 0x59, + 0x02, 0x09, 0x48, 0x8f, 0x4d, 0xf0, 0xd5, 0xec, 0x05, 0x00, 0x42, 0x55, + 0xa3, 0xae, 0xbc, 0x1c, 0x9f, 0xc1, 0x22, 0xfa, 0x05, 0xc2, 0xed, 0x5e, + 0x87, 0x94, 0x66, 0x1c, 0xc0, 0xf1, 0xac, 0x1c, 0x4b, 0x20, 0x01, 0x0a, + 0xaf, 0xee, 0x5b, 0xf5, 0x68, 0x9a, 0xca, 0x2e, 0x8b, 0xe8, 0x65, 0x16, + 0x46, 0x18, 0x80, 0x69, 0x0d, 0xbb, 0x07, 0xe6, 0xdb, 0x35, 0x0d, 0x4c, + 0xd3, 0x47, 0xf4, 0xb0, 0xb0, 0x17, 0xb3, 0x2d, 0xa2, 0x97, 0x5d, 0x54, + 0x01, 0xc8, 0xca, 0xce, 0xb5, 0x24, 0x11, 0x80, 0x18, 0x37, 0x69, 0xa6, + 0x3f, 0x5c, 0x3f, 0x7e, 0xfc, 0x31, 0x1d, 0x5f, 0xa2, 0x91, 0x74, 0xb0, + 0xfd, 0xa5, 0xaa, 0x71, 0x8f, 0x17, 0xfc, 0x31, 0x2c, 0xac, 0x93, 0x45, + 0xf4, 0x00, 0x8f, 0xe2, 0x2a, 0xfa, 0xdc, 0x3b, 0xc3, 0xaa, 0xa8, 0x1a, + 0x40, 0x90, 0x52, 0x62, 0x9a, 0x92, 0x2f, 0x66, 0xfe, 0x02, 0xc0, 0xb1, + 0x63, 0xc7, 0x84, 0xcb, 0xe5, 0xe2, 0xd8, 0xb1, 0x63, 0xc1, 0x9b, 0x36, + 0x6d, 0x8a, 0x7c, 0xfe, 0xf9, 0xa7, 0xea, 0x54, 0xae, 0x6c, 0x7b, 0x1f, + 0x90, 0xa0, 0x1e, 0xc8, 0xc9, 0xd9, 0xdc, 0xcc, 0x22, 0x7a, 0x60, 0x3b, + 0x77, 0x1a, 0xc0, 0xd1, 0xd4, 0x74, 0x14, 0x45, 0x58, 0xe2, 0x08, 0x10, + 0x28, 0x42, 0x30, 0xe6, 0xe3, 0x6f, 0x01, 0xa8, 0x53, 0xa7, 0x0e, 0xe5, + 0xca, 0x95, 0xc3, 0xe9, 0x74, 0x52, 0xae, 0x5c, 0x39, 0x1a, 0x34, 0x68, + 0xc0, 0xab, 0xaf, 0xbe, 0xc6, 0x9e, 0x3d, 0x7b, 0x59, 0xb8, 0x70, 0x21, + 0x51, 0x51, 0xe1, 0x15, 0x4d, 0x73, 0xcf, 0x6a, 0x21, 0x22, 0x3e, 0xb2, + 0x88, 0x1e, 0x80, 0x08, 0x09, 0x79, 0xc1, 0xbf, 0x4a, 0x26, 0x33, 0x3b, + 0x17, 0x45, 0x58, 0x22, 0x0d, 0x14, 0x6b, 0xee, 0xd1, 0x0d, 0xbe, 0x9a, + 0xed, 0x4b, 0x0b, 0xb7, 0x68, 0xd1, 0xa2, 0xd3, 0x1e, 0xdb, 0xae, 0x5d, + 0x3b, 0xd2, 0xd2, 0xd2, 0x44, 0xd7, 0xae, 0xb7, 0x49, 0x29, 0x8f, 0x3f, + 0x2c, 0x44, 0xd4, 0xbd, 0x10, 0x61, 0x11, 0x3d, 0x90, 0x90, 0x93, 0x33, + 0xca, 0x05, 0x3e, 0x2b, 0xde, 0xb4, 0x51, 0x2d, 0xff, 0xa0, 0x8d, 0x85, + 0xcb, 0x3c, 0x18, 0x13, 0x82, 0x8d, 0xdb, 0xfe, 0x49, 0xf2, 0x1a, 0x1d, + 0x1d, 0x7d, 0xd6, 0x8e, 0x61, 0xe6, 0xcc, 0x99, 0xa2, 0x53, 0xa7, 0x4e, + 0x52, 0xca, 0xf4, 0xcf, 0x21, 0xf4, 0x0a, 0x80, 0x7a, 0xf5, 0xf6, 0x5b, + 0x44, 0x0f, 0x34, 0xdc, 0xd9, 0xa9, 0x2d, 0x56, 0x84, 0x1e, 0x18, 0x30, + 0x4d, 0x49, 0xdf, 0xe7, 0xdf, 0x06, 0x60, 0xf0, 0xe0, 0xc1, 0xd8, 0x6c, + 0xb6, 0xb3, 0x76, 0x0c, 0x52, 0x4a, 0xe6, 0xce, 0x9d, 0x2b, 0x7a, 0xf7, + 0xee, 0x2d, 0xe1, 0xc0, 0xdf, 0x76, 0x7b, 0xdd, 0x26, 0x5b, 0xb7, 0x56, + 0x29, 0x1d, 0x1d, 0x97, 0xf5, 0x4a, 0x2f, 0x0c, 0xb1, 0xb1, 0x52, 0x39, + 0x7a, 0x54, 0x18, 0x03, 0x1e, 0xe8, 0xca, 0x43, 0x3d, 0x6f, 0xb1, 0xa6, + 0xd8, 0x02, 0x04, 0x1b, 0xff, 0xda, 0xcd, 0xbd, 0x83, 0x47, 0xfb, 0xad, + 0xb5, 0x94, 0x12, 0x21, 0xc4, 0x59, 0xad, 0xba, 0x10, 0x02, 0xb7, 0xdb, + 0x4d, 0x50, 0x50, 0x10, 0xe0, 0xdc, 0x07, 0xee, 0x6a, 0x96, 0x45, 0x0f, + 0x00, 0x1c, 0x3d, 0x2a, 0x4c, 0x10, 0x39, 0x6e, 0x8f, 0xd7, 0xb2, 0xe6, + 0x01, 0x12, 0x9b, 0x0b, 0x21, 0x18, 0xf4, 0xca, 0x87, 0x00, 0xf4, 0xe9, + 0xd3, 0xc7, 0x6f, 0xb1, 0x8b, 0xe2, 0xee, 0x3f, 0xf5, 0xd4, 0x53, 0x48, + 0x29, 0xb9, 0xe7, 0x9e, 0x7b, 0x24, 0xb8, 0xab, 0xda, 0x6c, 0x8d, 0x15, + 0x8b, 0xe8, 0x81, 0x83, 0x90, 0x3f, 0xb7, 0xee, 0xc2, 0xa6, 0xa9, 0x96, + 0x24, 0x2e, 0x73, 0x92, 0xab, 0xaa, 0xca, 0x07, 0xff, 0xf7, 0x1d, 0xa9, + 0xe9, 0xbe, 0xb2, 0xc8, 0x93, 0x26, 0x4d, 0x2a, 0xf2, 0x94, 0x69, 0x66, + 0x66, 0x26, 0x63, 0xc7, 0x8e, 0xe5, 0xa6, 0x9b, 0x6e, 0xa2, 0x79, 0xf3, + 0xe6, 0x02, 0x40, 0xd7, 0x77, 0x97, 0x8a, 0x3a, 0x5d, 0x9a, 0xf5, 0x7a, + 0x8b, 0x27, 0x02, 0xda, 0xb6, 0x6b, 0x3f, 0x9a, 0xaa, 0xe2, 0xf1, 0xed, + 0x6f, 0xb1, 0x70, 0x19, 0x42, 0x51, 0x04, 0x3f, 0xff, 0xba, 0x96, 0x0f, + 0xbf, 0x9c, 0xeb, 0xff, 0x6d, 0xf5, 0xea, 0xd5, 0x34, 0x6f, 0xde, 0xbc, + 0x88, 0xe7, 0xfb, 0xec, 0xe6, 0xb2, 0x65, 0xcb, 0x58, 0xb1, 0x62, 0x45, + 0x7e, 0xe7, 0x51, 0x3a, 0x92, 0x91, 0x58, 0x16, 0xbd, 0x78, 0x88, 0xbe, + 0x49, 0xd3, 0x54, 0x2b, 0x3a, 0xbf, 0xbc, 0xed, 0x39, 0x19, 0x99, 0x39, + 0x3c, 0x35, 0xf2, 0x43, 0xdf, 0x97, 0x7c, 0xb4, 0x68, 0xd1, 0x82, 0x15, + 0x2b, 0x96, 0xf3, 0xfb, 0xef, 0xbf, 0xcb, 0x23, 0x47, 0x8e, 0xe0, 0x76, + 0xbb, 0xfd, 0xd6, 0xff, 0xdf, 0x08, 0x0d, 0x0d, 0x65, 0xce, 0x9c, 0x39, + 0xf8, 0x2c, 0xb9, 0x9e, 0x4f, 0xfe, 0x60, 0xaf, 0x45, 0xf4, 0x80, 0xb1, + 0x04, 0xce, 0x55, 0x29, 0xc7, 0x32, 0x84, 0xaa, 0x58, 0xe2, 0xbc, 0x5c, + 0x5d, 0x76, 0xbb, 0xcd, 0xc6, 0x7f, 0x9e, 0x19, 0x5b, 0x40, 0x72, 0x21, + 0x44, 0xe4, 0x04, 0x40, 0xa8, 0x6a, 0x8b, 0x2b, 0xae, 0xb9, 0xe6, 0xe6, + 0xd5, 0x2d, 0x5b, 0xb6, 0x14, 0x71, 0x71, 0x71, 0x04, 0x05, 0x05, 0xb1, + 0x61, 0xc3, 0x86, 0xd3, 0xc6, 0xec, 0x2d, 0x5a, 0xb4, 0x38, 0xe1, 0xbb, + 0xc3, 0x11, 0x67, 0x58, 0x44, 0x0f, 0x00, 0xf4, 0xe9, 0xe3, 0x01, 0xec, + 0xc9, 0x00, 0x07, 0x8f, 0x1e, 0xb3, 0x04, 0x72, 0x19, 0x42, 0xd3, 0x54, + 0x9e, 0x7b, 0xe3, 0x53, 0x76, 0xec, 0x39, 0x00, 0x20, 0xec, 0xf6, 0xaa, + 0xe3, 0xa5, 0xcc, 0x78, 0x02, 0xc0, 0x30, 0x56, 0xed, 0x34, 0xcd, 0xac, + 0x16, 0x36, 0x5b, 0xfd, 0x56, 0xf9, 0x9d, 0x80, 0x4c, 0x4a, 0x4a, 0x22, + 0x39, 0x39, 0xf9, 0x24, 0xab, 0x2e, 0xa5, 0xc4, 0xe5, 0x72, 0x15, 0x74, + 0x16, 0x7a, 0xb9, 0x72, 0x37, 0x56, 0x70, 0xb9, 0x36, 0x9b, 0x16, 0xd1, + 0x03, 0x00, 0x9f, 0x7e, 0x6a, 0xc7, 0x34, 0xd3, 0x5f, 0x01, 0x58, 0xb5, + 0xfe, 0x2f, 0x6b, 0xad, 0xfb, 0x65, 0x18, 0x97, 0xbf, 0xfe, 0xc1, 0xd7, + 0xcc, 0xf9, 0x79, 0x39, 0x00, 0x36, 0x5b, 0xe5, 0x17, 0x3c, 0x9e, 0x7d, + 0xfd, 0x63, 0x62, 0x5e, 0x3e, 0xe1, 0x38, 0xaf, 0x77, 0xcb, 0x4a, 0x45, + 0xa9, 0x68, 0x53, 0xd5, 0xc8, 0xdf, 0x01, 0x5a, 0xb6, 0x6c, 0x79, 0x92, + 0x55, 0x17, 0x42, 0xd0, 0xb6, 0x6d, 0x5b, 0x00, 0x61, 0xb3, 0x55, 0x79, + 0x35, 0x2d, 0xed, 0xe7, 0xa3, 0xe5, 0xcb, 0x97, 0x8e, 0x8d, 0x2e, 0x16, + 0xd1, 0x8b, 0xcf, 0x2e, 0xe4, 0x2e, 0x5f, 0xbb, 0x05, 0x55, 0xb5, 0x44, + 0x7a, 0xb9, 0x40, 0x55, 0x15, 0xc6, 0x4e, 0xfa, 0x1f, 0x5f, 0xce, 0x5a, + 0x90, 0x4f, 0xf2, 0x2a, 0xaf, 0x7a, 0xbd, 0x07, 0x5e, 0x85, 0x1a, 0xa4, + 0xa4, 0xbc, 0x74, 0x0a, 0xcb, 0x5f, 0xdd, 0x30, 0x8c, 0x8c, 0x56, 0x42, + 0x38, 0x57, 0x1c, 0x39, 0x72, 0x84, 0x97, 0x5e, 0x7a, 0xe9, 0x84, 0x78, + 0x7d, 0xc5, 0x8a, 0x15, 0xec, 0xdd, 0xbb, 0x57, 0x82, 0x3d, 0xc3, 0xeb, + 0xdd, 0xff, 0x12, 0xc0, 0xb1, 0x63, 0xf3, 0x2c, 0x78, 0x4e, 0x71, 0x41, + 0x00, 0x00, 0x0a, 0xa5, 0x49, 0x44, 0x41, 0x54, 0xa2, 0x07, 0x12, 0x84, + 0x50, 0x56, 0x6d, 0xdb, 0xb5, 0x4f, 0xda, 0x34, 0x6b, 0x22, 0xe3, 0x72, + 0x80, 0xcd, 0xa6, 0x31, 0x6c, 0xec, 0x14, 0x3e, 0xfb, 0xf6, 0x47, 0x40, + 0x20, 0x44, 0x85, 0x36, 0x5e, 0xef, 0xfe, 0x17, 0xaa, 0x55, 0x7b, 0x1a, + 0x48, 0x3e, 0xe5, 0x39, 0x1e, 0xcf, 0x4a, 0x14, 0x25, 0x1e, 0x29, 0xdd, + 0xad, 0x35, 0xad, 0xfc, 0xac, 0x11, 0x23, 0x46, 0x70, 0xfb, 0xed, 0xb7, + 0xb3, 0x70, 0xe1, 0x42, 0x9a, 0x35, 0x6b, 0x26, 0x5b, 0xb7, 0x6e, 0x0d, + 0x38, 0x76, 0x85, 0x85, 0x5d, 0x5d, 0xd1, 0xe9, 0xbc, 0xb6, 0x74, 0xe9, + 0xa7, 0xf5, 0xca, 0x8b, 0x8b, 0xe8, 0xa1, 0x0f, 0x4a, 0x99, 0xfd, 0xe9, + 0x86, 0xef, 0x27, 0x62, 0x9a, 0x66, 0x91, 0x16, 0x58, 0x58, 0xb8, 0x34, + 0x08, 0x72, 0xda, 0xb9, 0x7b, 0xe0, 0x6b, 0xfc, 0xb1, 0xf1, 0x6f, 0x09, + 0x8a, 0xb0, 0xd9, 0xaa, 0xb6, 0xf3, 0x7a, 0xf7, 0x2c, 0x2e, 0xea, 0xf9, + 0x15, 0x2a, 0xdc, 0xcd, 0x91, 0x23, 0x5f, 0xa1, 0x69, 0x71, 0x8f, 0xeb, + 0x7a, 0xea, 0xfb, 0xa0, 0x23, 0x44, 0xe8, 0xf7, 0x52, 0xea, 0xcf, 0x81, + 0xfb, 0xcf, 0xa0, 0xa0, 0x06, 0xb8, 0x5c, 0x9b, 0x2d, 0xa2, 0x07, 0x22, + 0xec, 0xf6, 0xf8, 0x06, 0x1e, 0xcf, 0xee, 0x4d, 0x75, 0x13, 0xaa, 0x32, + 0xeb, 0xa3, 0xe1, 0xb8, 0xf3, 0xac, 0x62, 0x0e, 0xa5, 0x11, 0x4e, 0x87, + 0x8d, 0xde, 0x4f, 0x8e, 0x66, 0xcd, 0xc6, 0xed, 0x12, 0x14, 0x21, 0x44, + 0x74, 0x0b, 0x29, 0x8f, 0xae, 0x3e, 0xd7, 0xeb, 0xd4, 0xa8, 0x71, 0x9c, + 0xe4, 0xe4, 0x13, 0x77, 0xa8, 0xc5, 0xc6, 0xd6, 0xe6, 0xe8, 0xd1, 0xd2, + 0x59, 0x7b, 0xd1, 0x22, 0x7a, 0xf1, 0x3a, 0x84, 0x5b, 0xc0, 0x5b, 0x6f, + 0xd7, 0xd2, 0xcf, 0xc9, 0xb4, 0xb2, 0xcd, 0x94, 0x2a, 0x48, 0x29, 0x09, + 0x72, 0xda, 0xb9, 0x6b, 0xe0, 0x6b, 0xac, 0xdb, 0xb4, 0x43, 0x82, 0x9a, + 0x17, 0x14, 0x94, 0x78, 0x85, 0xcb, 0xb5, 0xee, 0x40, 0x99, 0x18, 0x8f, + 0xb0, 0x54, 0xa0, 0x38, 0xdd, 0x77, 0x47, 0x30, 0xe8, 0x37, 0xe6, 0xb8, + 0xdc, 0x34, 0x4f, 0xac, 0x63, 0xb9, 0xef, 0xa5, 0x49, 0xd1, 0x55, 0x95, + 0x6e, 0x8f, 0x8e, 0x60, 0xd3, 0xb6, 0x64, 0x84, 0x70, 0x6e, 0xa8, 0x52, + 0x65, 0x7c, 0x42, 0x5a, 0xda, 0xc7, 0x19, 0x65, 0xe5, 0xf9, 0xad, 0xc1, + 0xb8, 0xe2, 0x72, 0x09, 0x9d, 0x09, 0x48, 0x69, 0x6c, 0x00, 0x98, 0xf3, + 0xf3, 0x0a, 0x82, 0x9d, 0x56, 0x8e, 0xf7, 0xd2, 0x60, 0xc5, 0x15, 0x21, + 0x38, 0x78, 0xe4, 0x18, 0x57, 0xdf, 0xf1, 0xa4, 0xf4, 0x15, 0xd8, 0x70, + 0xcc, 0x94, 0xd2, 0xdd, 0xe4, 0xc8, 0x91, 0x91, 0x9e, 0x32, 0xd5, 0xd1, + 0x59, 0xea, 0x50, 0x3c, 0xd0, 0x75, 0xa2, 0x84, 0x30, 0x66, 0x82, 0x5e, + 0x2e, 0xc7, 0xe5, 0x16, 0xa6, 0x34, 0xb9, 0xb2, 0x61, 0x2d, 0xcb, 0xaa, + 0x5f, 0x52, 0x2b, 0xae, 0xf0, 0xed, 0xf7, 0xbf, 0xd2, 0x77, 0xc8, 0xdb, + 0xd2, 0xe3, 0xf5, 0x0a, 0x87, 0xa3, 0xda, 0xab, 0x86, 0x91, 0xd6, 0x57, + 0xd3, 0xae, 0x40, 0xd7, 0x93, 0xcb, 0x96, 0xb7, 0x69, 0xa9, 0x43, 0x71, + 0xb9, 0xed, 0xf6, 0xcd, 0x52, 0x7a, 0xea, 0x55, 0xa9, 0x52, 0x45, 0x1c, + 0x3a, 0x74, 0x88, 0xa8, 0x88, 0x50, 0x96, 0x7d, 0x33, 0xd6, 0xaa, 0xb0, + 0x7a, 0x09, 0xac, 0xb8, 0xaa, 0x2a, 0xec, 0xdc, 0x73, 0x88, 0x5e, 0xfd, + 0x5f, 0x95, 0x79, 0x1e, 0x8f, 0x00, 0x45, 0x57, 0xd5, 0x2b, 0x3a, 0x18, + 0xc6, 0xf6, 0x45, 0xd1, 0xd1, 0x23, 0x48, 0x4d, 0x1d, 0x56, 0xe6, 0xe4, + 0x62, 0xb9, 0xee, 0xc5, 0x62, 0x39, 0xe2, 0x6e, 0x94, 0xd2, 0x53, 0x7f, + 0xc0, 0x80, 0x01, 0x62, 0xdf, 0xbe, 0x7d, 0x4c, 0x99, 0x32, 0x85, 0xd4, + 0xb4, 0xe3, 0xfc, 0xb6, 0x66, 0xb3, 0xb5, 0x52, 0xee, 0xa2, 0x13, 0x1d, + 0xbe, 0x98, 0xb9, 0x80, 0xdb, 0xfb, 0x0e, 0x97, 0x79, 0x1e, 0x8f, 0x50, + 0x94, 0xa0, 0xdf, 0xc1, 0xb4, 0xd9, 0x6c, 0x21, 0x8b, 0x80, 0x32, 0x49, + 0x72, 0xcb, 0xa2, 0x17, 0x9b, 0x35, 0xd7, 0xf6, 0x37, 0x69, 0x92, 0x58, + 0xe9, 0x8f, 0x3f, 0xfe, 0xf0, 0xcb, 0xb3, 0x79, 0xf3, 0xe6, 0xec, 0xd9, + 0xbd, 0x83, 0x05, 0x5f, 0xbe, 0x61, 0xe5, 0x91, 0xbb, 0x18, 0xb1, 0xb8, + 0xa2, 0xb0, 0xff, 0x50, 0x0a, 0x5d, 0xfb, 0x0e, 0xc7, 0xeb, 0xd5, 0x01, + 0x21, 0x55, 0xb5, 0x4a, 0x0f, 0xc3, 0xd8, 0x37, 0xc3, 0x92, 0x90, 0x45, + 0xf4, 0x62, 0xd3, 0xb5, 0xd4, 0xd4, 0x54, 0xca, 0x95, 0x2b, 0xe7, 0xcf, + 0x1d, 0xe6, 0x72, 0xb9, 0x08, 0x09, 0x09, 0x61, 0xec, 0xd0, 0x47, 0xb9, + 0xbe, 0x75, 0x52, 0x91, 0x52, 0x11, 0x59, 0x38, 0x77, 0x82, 0x17, 0xc8, + 0xfb, 0xd9, 0xd1, 0x93, 0xf8, 0x69, 0xe9, 0x1a, 0x09, 0xa0, 0x28, 0xc1, + 0xdb, 0x4c, 0x33, 0xb7, 0x9e, 0xa2, 0x54, 0xc3, 0x34, 0xf7, 0x5a, 0x82, + 0xb2, 0x88, 0x5e, 0x6c, 0xc8, 0x4b, 0x49, 0x49, 0xb1, 0x97, 0x2f, 0x5f, + 0xfe, 0x04, 0x32, 0xdb, 0xed, 0x76, 0x4c, 0xd3, 0x60, 0xed, 0xbc, 0x0f, + 0x90, 0xa6, 0xe5, 0xc2, 0x97, 0x44, 0x2c, 0x3e, 0x74, 0xcc, 0x67, 0xcc, + 0xf9, 0x65, 0x05, 0x80, 0x14, 0x42, 0x15, 0x52, 0x46, 0xb4, 0x83, 0xb4, + 0xc5, 0x96, 0x84, 0xac, 0x18, 0xbd, 0xf8, 0x85, 0xa8, 0x84, 0xfc, 0x18, + 0x13, 0x13, 0xc3, 0x3b, 0xef, 0xbc, 0x43, 0x72, 0x72, 0x32, 0x00, 0xcf, + 0x3c, 0xf3, 0x0c, 0x5e, 0xaf, 0x17, 0xc3, 0x30, 0x69, 0xde, 0xe5, 0x09, + 0x2b, 0x56, 0x2f, 0x46, 0x82, 0x03, 0xfc, 0xfc, 0xeb, 0x5a, 0xae, 0xec, + 0xf4, 0x78, 0x01, 0xc9, 0x51, 0x94, 0xf0, 0x85, 0x52, 0x36, 0x51, 0x7c, + 0x24, 0xaf, 0x61, 0x09, 0xca, 0xb2, 0xe8, 0xc5, 0x0b, 0x4d, 0xab, 0x89, + 0xae, 0xef, 0x42, 0xd3, 0x62, 0x5f, 0xd5, 0xf5, 0xa3, 0xcf, 0x9f, 0x4e, + 0x3f, 0xc7, 0xbc, 0xd0, 0x57, 0xdc, 0x70, 0x75, 0x13, 0x4b, 0x60, 0x17, + 0x08, 0xd3, 0x94, 0x74, 0x7d, 0x64, 0x18, 0x7b, 0x0f, 0xa6, 0x00, 0x48, + 0xd0, 0xb2, 0x6c, 0xb6, 0x4a, 0x57, 0x7a, 0xbd, 0x7b, 0x77, 0xda, 0x6c, + 0xe5, 0xf1, 0x7a, 0xad, 0x9c, 0x00, 0x16, 0xd1, 0x4b, 0x08, 0x4e, 0x67, + 0x2b, 0xdc, 0xee, 0x15, 0x68, 0x5a, 0xd5, 0x70, 0x5d, 0x3f, 0x1a, 0xa7, + 0x69, 0x91, 0x59, 0xa1, 0xa1, 0x6f, 0x1d, 0x76, 0xb9, 0xfe, 0x4f, 0x95, + 0x72, 0xaf, 0x4d, 0xd7, 0x93, 0xd7, 0x99, 0x66, 0x6e, 0x9d, 0x99, 0x13, + 0x87, 0x13, 0x5f, 0x35, 0xce, 0x12, 0xd8, 0x39, 0x5a, 0x70, 0x45, 0x11, + 0x38, 0x1d, 0x0e, 0x5e, 0x79, 0xf7, 0x4b, 0xa6, 0x7e, 0xb7, 0x10, 0xc3, + 0x30, 0x01, 0x05, 0x4d, 0x8b, 0xfe, 0xff, 0xf6, 0xce, 0x36, 0x28, 0xaa, + 0xeb, 0x8c, 0xe3, 0xff, 0x73, 0xee, 0xcb, 0x5e, 0x70, 0x5f, 0x0c, 0x20, + 0x0c, 0x08, 0x68, 0x42, 0x08, 0x45, 0xab, 0xc6, 0x3a, 0xd4, 0x69, 0x4a, + 0x1b, 0x69, 0x52, 0xda, 0x66, 0xc6, 0xb6, 0x93, 0x92, 0xa4, 0x93, 0xb6, + 0x32, 0xed, 0x87, 0xaa, 0xfd, 0x52, 0x26, 0x9d, 0x44, 0x74, 0x1c, 0x07, + 0x94, 0xaa, 0x33, 0x4e, 0x1a, 0x13, 0x32, 0x99, 0x8e, 0x09, 0x1d, 0x51, + 0x74, 0xb0, 0x11, 0xa3, 0x4e, 0x8d, 0x89, 0x14, 0x0d, 0x49, 0xad, 0x1a, + 0x6b, 0x29, 0x83, 0x75, 0xc6, 0x26, 0xbe, 0x2c, 0x48, 0x95, 0x42, 0x79, + 0x87, 0xbd, 0xbb, 0x77, 0xef, 0x7d, 0xfa, 0x81, 0xdd, 0x0d, 0x49, 0x51, + 0x61, 0x9a, 0xa4, 0xab, 0x3c, 0xbf, 0x99, 0xfd, 0xc4, 0xee, 0xb2, 0xf3, + 0x9c, 0xf3, 0xbb, 0xcf, 0x39, 0xe7, 0x9e, 0xf3, 0xdc, 0x92, 0x82, 0x82, + 0xae, 0x86, 0x53, 0xa7, 0xb8, 0x1b, 0xb3, 0xe8, 0x71, 0x80, 0xa2, 0x2c, + 0xd6, 0x6d, 0xbb, 0x2d, 0x08, 0x84, 0xe9, 0xcc, 0xc1, 0x6a, 0xa1, 0x6b, + 0x2a, 0x2f, 0xcc, 0x4d, 0xf6, 0x22, 0xea, 0xd2, 0xf1, 0xec, 0xe6, 0x1d, + 0x38, 0x72, 0xfc, 0x4c, 0xa4, 0x66, 0xbe, 0x84, 0x94, 0x33, 0x7e, 0xe5, + 0x76, 0x97, 0x6d, 0x1f, 0x1c, 0xdc, 0xe4, 0x78, 0x3c, 0x4b, 0x31, 0x34, + 0x74, 0x86, 0x03, 0xc5, 0xa2, 0xc7, 0x07, 0x29, 0x29, 0x8f, 0xa7, 0xf4, + 0xf4, 0x1c, 0xe8, 0x9e, 0x33, 0x3b, 0x8d, 0x0e, 0xbd, 0x5a, 0x29, 0x78, + 0x15, 0xfe, 0xe6, 0x19, 0xdc, 0xa5, 0x6b, 0x18, 0x1c, 0x0e, 0xe0, 0xf9, + 0xd7, 0x5e, 0xc7, 0xa1, 0x63, 0x7f, 0x8e, 0x54, 0xd6, 0x15, 0x90, 0xd2, + 0x57, 0x91, 0x9e, 0xfe, 0x46, 0x55, 0x67, 0x67, 0x91, 0xad, 0xaa, 0x59, + 0x08, 0x87, 0x3b, 0x38, 0x60, 0x2c, 0x7a, 0x9c, 0x05, 0x5a, 0x24, 0x43, + 0xd3, 0x7c, 0x45, 0xa1, 0xd0, 0xe5, 0xe3, 0x0f, 0x2d, 0x99, 0x4f, 0x3b, + 0xb6, 0x94, 0x09, 0xcb, 0x0a, 0xb3, 0xec, 0x11, 0xa4, 0x14, 0x30, 0x74, + 0x1d, 0x35, 0xaf, 0xbf, 0x85, 0x63, 0xef, 0x9e, 0x43, 0xcb, 0xdf, 0x3f, + 0x8c, 0x75, 0x51, 0x21, 0x12, 0x36, 0xac, 0x58, 0x31, 0x52, 0x55, 0x5b, + 0x2b, 0x78, 0x45, 0x93, 0x45, 0xbf, 0x53, 0x3a, 0xb4, 0x6f, 0x8d, 0xe3, + 0x0c, 0x6c, 0x2d, 0xfa, 0xca, 0x83, 0x54, 0x5d, 0xf1, 0x0b, 0x61, 0x85, + 0xed, 0x69, 0x2b, 0x3b, 0x11, 0x41, 0x91, 0x12, 0xc3, 0x01, 0x13, 0xaf, + 0xec, 0x3e, 0x8c, 0xba, 0x37, 0x9a, 0x80, 0x48, 0x15, 0xd6, 0x88, 0xe0, + 0x75, 0x86, 0x91, 0xb3, 0x36, 0x10, 0x68, 0xbb, 0xc6, 0x3d, 0x87, 0x45, + 0xbf, 0xf3, 0xe6, 0x9d, 0x46, 0x5e, 0xb1, 0x69, 0x5e, 0x7c, 0xfb, 0x07, + 0xdf, 0x29, 0xa4, 0xca, 0xb2, 0x52, 0x61, 0x85, 0xa7, 0x4f, 0x66, 0x8f, + 0x9e, 0x28, 0xd3, 0x75, 0x0d, 0x97, 0xdb, 0xaf, 0xe3, 0xc8, 0x89, 0xf7, + 0xf1, 0xca, 0xee, 0xc3, 0xe3, 0x04, 0xd7, 0xcf, 0x08, 0xa1, 0x35, 0x10, + 0x8d, 0x6c, 0x1b, 0xfb, 0xc4, 0x5c, 0xdc, 0xac, 0xb4, 0x13, 0xc3, 0xa2, + 0xc7, 0x3d, 0x8a, 0x92, 0xba, 0xc6, 0xb6, 0xff, 0xb5, 0x75, 0x7e, 0xee, + 0x1c, 0xaa, 0x28, 0x5b, 0x21, 0xe6, 0xe5, 0x66, 0x23, 0x74, 0x97, 0x0f, + 0xe5, 0x35, 0x55, 0xc1, 0x8d, 0x9e, 0x7e, 0x9c, 0x3c, 0x7b, 0x1e, 0x3b, + 0xea, 0xdf, 0xc4, 0xb5, 0xeb, 0xdd, 0xe3, 0x04, 0x97, 0x03, 0xaa, 0x9a, + 0x56, 0x1a, 0x0e, 0x5f, 0x3f, 0xc4, 0xbd, 0x83, 0x45, 0xbf, 0xab, 0x50, + 0xd5, 0xac, 0x1f, 0x85, 0xc3, 0x1d, 0x75, 0x00, 0xe8, 0xa9, 0xe5, 0xcb, + 0xc4, 0xda, 0xd5, 0x3f, 0xbc, 0xeb, 0xb6, 0xca, 0x2a, 0x52, 0x62, 0xd4, + 0x0c, 0xe2, 0xc2, 0x07, 0x7e, 0xbc, 0xb4, 0xf3, 0x60, 0x74, 0xee, 0x4d, + 0x1f, 0xf5, 0x3d, 0xd1, 0x21, 0x84, 0x77, 0x2f, 0xd1, 0x40, 0x39, 0xf7, + 0x08, 0x16, 0xfd, 0xee, 0xcd, 0x72, 0x5a, 0xf6, 0x63, 0x96, 0xd5, 0x7e, + 0x04, 0x00, 0xa5, 0xa7, 0x26, 0x89, 0xba, 0xed, 0xe5, 0x98, 0x95, 0xe4, + 0x83, 0x6d, 0xff, 0x77, 0x81, 0xc9, 0xe8, 0x45, 0x40, 0x0a, 0x01, 0xcb, + 0xb6, 0xe1, 0xd8, 0x0e, 0x74, 0x4d, 0x8d, 0xab, 0xa7, 0xb8, 0x46, 0x7f, + 0xa3, 0x10, 0xc0, 0x95, 0x8e, 0x2e, 0xbc, 0x5c, 0x7b, 0x10, 0x7f, 0x3c, + 0xd9, 0xf2, 0xc9, 0x6e, 0xd7, 0x27, 0xc4, 0x8c, 0xa6, 0xec, 0xec, 0xbf, + 0x3c, 0xed, 0xf7, 0x7f, 0x81, 0xcf, 0xf1, 0xb2, 0xe8, 0xd3, 0x25, 0xb3, + 0xdf, 0x77, 0x6f, 0x38, 0xec, 0xbf, 0x00, 0xd8, 0x06, 0x00, 0x14, 0x7f, + 0x6d, 0x09, 0xb6, 0x3c, 0xf7, 0x33, 0x28, 0x8a, 0x12, 0xdb, 0xf2, 0x29, + 0x84, 0x80, 0x19, 0x0c, 0xe1, 0xc2, 0x87, 0xed, 0xd8, 0xd5, 0xd0, 0x88, + 0x77, 0x4e, 0xb7, 0x46, 0x86, 0xc3, 0x2a, 0xde, 0xdc, 0xf9, 0x6b, 0xcc, + 0x4a, 0xf2, 0xfd, 0xdf, 0x47, 0x03, 0xb6, 0x6d, 0x23, 0x60, 0x86, 0xd0, + 0x74, 0xb2, 0x05, 0x95, 0x2f, 0xd5, 0x4d, 0x70, 0x6a, 0x4f, 0xf5, 0x03, + 0x99, 0x5f, 0x05, 0xae, 0x76, 0x6a, 0xda, 0x3c, 0x58, 0xd6, 0x05, 0x6e, + 0x7c, 0x16, 0x7d, 0xfa, 0x60, 0x18, 0x5f, 0xc2, 0xf2, 0xe5, 0xe7, 0x94, + 0xfd, 0xfb, 0x13, 0x87, 0x89, 0x02, 0x06, 0x00, 0x12, 0x02, 0xe2, 0xb7, + 0x55, 0xbf, 0xc4, 0xd2, 0xc5, 0xf9, 0x20, 0x22, 0x1c, 0x3a, 0x76, 0x0a, + 0x15, 0x2f, 0xee, 0x8a, 0x64, 0xcb, 0xb1, 0x97, 0xe3, 0x38, 0xb1, 0x0b, + 0xc1, 0xb3, 0x3f, 0x7f, 0x02, 0x4f, 0x7f, 0xef, 0x1b, 0xb1, 0xd3, 0x5c, + 0x9f, 0x97, 0xf0, 0x8e, 0x43, 0x08, 0xdb, 0x36, 0xb6, 0xd7, 0x34, 0x60, + 0xef, 0xe1, 0x13, 0x13, 0xbe, 0x05, 0xd0, 0xbb, 0xa5, 0x9c, 0xfd, 0xb0, + 0xe3, 0x5c, 0xb9, 0xc8, 0xad, 0xcd, 0xa2, 0x33, 0x00, 0x34, 0x2d, 0x73, + 0xbd, 0x65, 0x5d, 0xdb, 0x14, 0xcd, 0xe2, 0x8d, 0xbb, 0xb7, 0xa2, 0xea, + 0xe5, 0xbd, 0x78, 0xe7, 0x74, 0x2b, 0xf6, 0xed, 0xdb, 0x87, 0x92, 0x92, + 0x92, 0x8f, 0x1a, 0x4f, 0x08, 0x34, 0x37, 0x37, 0xa3, 0xa8, 0xa8, 0x08, + 0xc0, 0x58, 0xbd, 0xf2, 0x4d, 0xcf, 0xfc, 0x14, 0x8f, 0x16, 0x2e, 0xfe, + 0x4c, 0x64, 0x1f, 0xff, 0x9d, 0x9d, 0x37, 0x7a, 0xf0, 0xbb, 0xdf, 0xbf, + 0x85, 0x03, 0x6f, 0xff, 0x09, 0x13, 0x9f, 0xd7, 0xd1, 0x2f, 0x4a, 0x39, + 0x73, 0xed, 0xfd, 0xf7, 0xff, 0xf8, 0x0f, 0x57, 0xaf, 0xb6, 0x5a, 0xa1, + 0x50, 0x13, 0x16, 0x2d, 0x6a, 0x43, 0x6b, 0xeb, 0x02, 0x6e, 0x64, 0x16, + 0x7d, 0xba, 0x67, 0xf6, 0x2f, 0xc2, 0x34, 0xcf, 0x43, 0x51, 0x52, 0x5d, + 0x9a, 0x96, 0xf8, 0x75, 0xd3, 0xec, 0x3c, 0x06, 0x58, 0x40, 0x64, 0xf1, + 0x2a, 0x18, 0x0c, 0x42, 0xd3, 0xb4, 0x98, 0x6c, 0x51, 0xf1, 0x4c, 0xd3, + 0xc4, 0xaa, 0x55, 0xab, 0x50, 0x5b, 0x5b, 0x0b, 0x00, 0xf0, 0xb8, 0x13, + 0x71, 0xfa, 0xc0, 0x8b, 0xb0, 0xc2, 0xf6, 0xa7, 0x22, 0xb7, 0x14, 0x02, + 0x9a, 0xae, 0xe1, 0xaf, 0xe7, 0x3f, 0xc0, 0x73, 0x9b, 0x5f, 0x45, 0x57, + 0x4f, 0x1f, 0xec, 0x09, 0x8b, 0x69, 0xb8, 0xb7, 0x09, 0x21, 0x37, 0x4b, + 0xa9, 0x9a, 0xb6, 0xdd, 0x6b, 0x72, 0x8b, 0xb2, 0xe8, 0xcc, 0x2d, 0xf0, + 0x78, 0x1e, 0xc4, 0xd0, 0xd0, 0xdf, 0xa2, 0xfa, 0xbf, 0x00, 0x98, 0x65, + 0x95, 0x95, 0x95, 0xd8, 0xb0, 0xe1, 0xd6, 0x65, 0x90, 0x46, 0x46, 0x46, + 0xb0, 0x72, 0xe5, 0x4a, 0xec, 0xd9, 0xb3, 0x07, 0x52, 0x4a, 0x6c, 0xdf, + 0xb0, 0x1a, 0x0f, 0x2f, 0x5d, 0x38, 0xe9, 0x27, 0xc7, 0xc4, 0x16, 0xfb, + 0xa4, 0x84, 0x4b, 0xd7, 0xf0, 0xcf, 0xae, 0x7f, 0x63, 0xff, 0xd1, 0x77, + 0xd1, 0x70, 0xf4, 0x3d, 0xf4, 0x0d, 0x0c, 0x8f, 0xaf, 0x7f, 0x17, 0xdb, + 0xd4, 0xa2, 0xaa, 0x69, 0x4f, 0x86, 0xc3, 0xbd, 0x4d, 0x2e, 0x57, 0x56, + 0x20, 0x18, 0xbc, 0x14, 0xe0, 0xd6, 0x63, 0xd1, 0x99, 0x29, 0xe2, 0x72, + 0x7d, 0x19, 0xc1, 0x60, 0x4b, 0x2f, 0x60, 0xdd, 0x33, 0x30, 0x30, 0x00, + 0x8f, 0xc7, 0x73, 0x4b, 0x61, 0xa3, 0xa2, 0x36, 0x36, 0x36, 0xa2, 0xb8, + 0xb8, 0x18, 0x00, 0xb0, 0x66, 0xf5, 0x53, 0x28, 0x7d, 0xfc, 0x9b, 0xb1, + 0x79, 0xbd, 0x19, 0x0c, 0x61, 0xd4, 0x0c, 0xe2, 0xe8, 0x89, 0xb3, 0x48, + 0x30, 0x74, 0xd8, 0xb6, 0x8d, 0xf3, 0xff, 0xb8, 0x8a, 0xb4, 0x59, 0x49, + 0x38, 0xdb, 0x7a, 0x11, 0x7d, 0x03, 0x43, 0x18, 0x19, 0x35, 0x71, 0xa3, + 0xbb, 0x17, 0x56, 0xd8, 0x1e, 0x77, 0x1b, 0xcc, 0xd5, 0x0e, 0xc8, 0x5d, + 0x80, 0xad, 0x28, 0x8a, 0xe7, 0x12, 0x51, 0xe8, 0xf8, 0xdc, 0xb9, 0xdb, + 0xfc, 0x97, 0x2f, 0xaf, 0xe2, 0x3a, 0x59, 0x0c, 0xf3, 0xbf, 0xe0, 0xf5, + 0x3e, 0x9a, 0x05, 0xc0, 0x29, 0x2c, 0x2c, 0xa4, 0xc9, 0xe2, 0x38, 0x0e, + 0x39, 0x8e, 0x43, 0x8a, 0xa2, 0x10, 0x00, 0xd2, 0x34, 0x95, 0xee, 0xcb, + 0x4e, 0xa7, 0xf4, 0xd4, 0x24, 0x9a, 0x95, 0x3c, 0x93, 0x22, 0xd9, 0x78, + 0x12, 0x2f, 0xd5, 0x2f, 0xa5, 0xfb, 0x79, 0x20, 0x71, 0xa9, 0xa6, 0x3d, + 0xe0, 0xfb, 0xe4, 0x6f, 0x4b, 0x48, 0x78, 0x88, 0x1b, 0x88, 0x33, 0x3a, + 0xf3, 0xe9, 0x90, 0x70, 0x04, 0x08, 0x3c, 0xd6, 0xdf, 0xdf, 0x0f, 0x9f, + 0xcf, 0x37, 0xe9, 0x39, 0xb5, 0x10, 0x02, 0xaa, 0xaa, 0xc2, 0xb6, 0x6d, + 0x02, 0x84, 0x00, 0xb4, 0xe3, 0x80, 0x33, 0x1f, 0x40, 0x1a, 0x20, 0xdb, + 0x00, 0x9c, 0x02, 0x30, 0x04, 0x60, 0x09, 0x40, 0x06, 0xa0, 0x9c, 0x03, + 0x9c, 0x4b, 0x42, 0x28, 0x3d, 0x00, 0x06, 0x14, 0x25, 0xa5, 0x2d, 0x1c, + 0xee, 0xb8, 0xc2, 0xf1, 0x67, 0x98, 0xcf, 0xf2, 0xea, 0x2b, 0xd2, 0xe1, + 0x76, 0x2f, 0x50, 0x00, 0x41, 0x25, 0x25, 0x25, 0x0e, 0x4d, 0x91, 0x8d, + 0x1b, 0x37, 0xc6, 0x32, 0xb3, 0xa2, 0xcc, 0xc9, 0x9b, 0xca, 0xff, 0x5e, + 0xb8, 0x70, 0x27, 0x37, 0x00, 0xc3, 0x7c, 0x5e, 0x48, 0x79, 0xcf, 0x0b, + 0x00, 0x9c, 0xe8, 0x70, 0x7c, 0xb2, 0xf4, 0xf6, 0xf6, 0x12, 0x00, 0x07, + 0x00, 0x49, 0xe9, 0xdd, 0xc5, 0x91, 0x64, 0x98, 0x38, 0x45, 0xd7, 0xf3, + 0xb3, 0x00, 0x50, 0x45, 0x45, 0xc5, 0x94, 0x32, 0xf9, 0x78, 0xc9, 0x01, + 0xa3, 0x05, 0x18, 0xdb, 0x8c, 0xc3, 0x30, 0x4c, 0x1c, 0x91, 0x9f, 0xdf, + 0x11, 0x19, 0xba, 0x1b, 0xdd, 0xa9, 0xa9, 0xa9, 0x93, 0xce, 0xe6, 0xd1, + 0xf7, 0xb8, 0xdd, 0x6e, 0x07, 0x10, 0x04, 0x18, 0x5b, 0xc6, 0x46, 0x05, + 0x19, 0x1c, 0x54, 0x86, 0x89, 0x47, 0x14, 0x25, 0xe3, 0x11, 0x00, 0xd4, + 0xdf, 0xdf, 0x3f, 0xa5, 0x21, 0xfb, 0xfa, 0xf5, 0xeb, 0x1d, 0x00, 0xa4, + 0xeb, 0xf3, 0xbe, 0x3b, 0x63, 0xc6, 0x42, 0x0e, 0x24, 0xc3, 0xc4, 0x23, + 0x1e, 0xcf, 0xb7, 0xe0, 0xf5, 0x6e, 0x13, 0x80, 0x4a, 0xe9, 0xe9, 0xe9, + 0x53, 0x1a, 0xb2, 0x77, 0x77, 0x77, 0x13, 0x00, 0x12, 0x22, 0xf1, 0x3d, + 0x8e, 0x24, 0xc3, 0xc4, 0x39, 0x52, 0x7a, 0x7f, 0x02, 0xc0, 0xa9, 0xae, + 0xae, 0xfe, 0x58, 0x36, 0xbf, 0x5d, 0x66, 0x6f, 0x6e, 0x6e, 0x26, 0x00, + 0x64, 0x18, 0x79, 0x3c, 0x21, 0x67, 0x98, 0xf8, 0x17, 0xdd, 0x30, 0x53, + 0x52, 0x52, 0x62, 0x02, 0x97, 0x95, 0x95, 0x91, 0xdf, 0xef, 0xbf, 0xed, + 0xdc, 0x3c, 0x39, 0x39, 0xd9, 0x01, 0x04, 0x3f, 0xb9, 0x80, 0x61, 0xe2, + 0x5f, 0xf2, 0xa4, 0x05, 0x00, 0xa8, 0xa0, 0xa0, 0x80, 0x88, 0x88, 0x06, + 0x07, 0x07, 0x63, 0xf7, 0xc2, 0x6f, 0x95, 0xd1, 0xeb, 0xeb, 0xeb, 0xa3, + 0xc3, 0xf6, 0xdf, 0x70, 0x14, 0x19, 0x26, 0xee, 0x99, 0x79, 0xef, 0x78, + 0xd1, 0x2d, 0xcb, 0x8a, 0x89, 0x3e, 0x3a, 0x3a, 0x7a, 0x53, 0xd9, 0xbd, + 0x5e, 0x2f, 0x8d, 0xad, 0xb4, 0x33, 0x0c, 0x13, 0xf7, 0xcc, 0x9e, 0xbd, + 0x57, 0x03, 0x40, 0x19, 0x19, 0x19, 0x31, 0x89, 0x73, 0x73, 0x73, 0x09, + 0x00, 0x95, 0x96, 0x96, 0x4e, 0x28, 0x79, 0x6d, 0x6d, 0x6d, 0xe4, 0xbe, + 0xb9, 0xf1, 0x3e, 0x47, 0x90, 0x61, 0xee, 0x98, 0xe1, 0xfb, 0xcc, 0x9d, + 0x00, 0x9c, 0xfa, 0xfa, 0x7a, 0x72, 0x1c, 0x27, 0xba, 0x01, 0xe6, 0x63, + 0x59, 0x3d, 0x9a, 0xd9, 0x6b, 0x6a, 0x6a, 0x62, 0x7f, 0x53, 0xd5, 0xfc, + 0x45, 0x86, 0x31, 0x8f, 0x03, 0xc8, 0x30, 0xf1, 0x8e, 0xa6, 0x3d, 0x00, + 0x4d, 0x5b, 0xec, 0x03, 0x5c, 0x7d, 0x8a, 0x22, 0xa9, 0xbc, 0xbc, 0x9c, + 0x86, 0x87, 0x87, 0x29, 0x27, 0x27, 0x87, 0x00, 0xd0, 0xb2, 0x65, 0xcb, + 0xa8, 0xa1, 0xa1, 0x81, 0xd6, 0xad, 0x5b, 0x47, 0x79, 0x79, 0x79, 0x51, + 0xc9, 0x1d, 0x21, 0xb4, 0x61, 0x8e, 0x1e, 0xc3, 0xdc, 0x81, 0x08, 0xe1, + 0x79, 0x06, 0x50, 0x26, 0x71, 0x94, 0x54, 0x92, 0x94, 0x99, 0x73, 0x39, + 0x62, 0xcc, 0x6d, 0xfb, 0x14, 0x87, 0x20, 0xbe, 0x30, 0x8c, 0x02, 0x98, + 0xe6, 0x59, 0x00, 0x80, 0xa2, 0x24, 0x7d, 0xdb, 0x71, 0x02, 0x39, 0x80, + 0xe8, 0x02, 0x44, 0x3b, 0x40, 0xdf, 0x27, 0x0a, 0xe5, 0x02, 0xc8, 0x91, + 0xd2, 0x75, 0x30, 0x33, 0xf3, 0x4a, 0x55, 0x7b, 0x7b, 0x2a, 0x17, 0x7e, + 0x60, 0x6e, 0xcb, 0x7f, 0x00, 0x4a, 0x3f, 0xff, 0x3a, 0x92, 0xd3, 0x63, + 0x31, 0x00, 0x00, 0x00, 0x00, 0x49, 0x45, 0x4e, 0x44, 0xae, 0x42, 0x60, + 0x82, 0x01, 0x00, 0x00, 0xff, 0xff, 0x25, 0x50, 0x56, 0x5e, 0x8b, 0x55, + 0x00, 0x00, + })) + + if err != nil { + panic("Decompression failed: " + err.Error()) + } + + var b bytes.Buffer + io.Copy(&b, gz) + gz.Close() + + return b.Bytes() +} + +func init() { + go_bindata["/gophercolor.png"] = gophercolor_png +} From af3804d7282d22c53b3b368198a131597b19836a Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Mon, 7 Oct 2013 21:53:36 -0700 Subject: [PATCH 195/247] feat(dashboard): inital commit of resources package --- dashboard/build | 3 +- dashboard/resources/bindata-file.go | 11 + dashboard/resources/bindata-toc.go | 7 + dashboard/resources/browser.html.go | 76 + dashboard/resources/index.html.go | 261 + .../resources/scripts-browser-modules.js.go | 9500 +++++++++++ .../resources/scripts-browser-scripts.js.go | 232 + .../resources/scripts-stats-modules.js.go | 13760 ++++++++++++++++ .../resources/scripts-stats-scripts.js.go | 2748 +++ dashboard/resources/stats.html.go | 76 + dashboard/resources/styles-main.css.go | 216 + dashboard/resources/views-browser.html.go | 169 + dashboard/resources/views-stats.html.go | 95 + 13 files changed, 27153 insertions(+), 1 deletion(-) create mode 100644 dashboard/resources/bindata-file.go create mode 100644 dashboard/resources/bindata-toc.go create mode 100644 dashboard/resources/browser.html.go create mode 100644 dashboard/resources/index.html.go create mode 100644 dashboard/resources/scripts-browser-modules.js.go create mode 100644 dashboard/resources/scripts-browser-scripts.js.go create mode 100644 dashboard/resources/scripts-stats-modules.js.go create mode 100644 dashboard/resources/scripts-stats-scripts.js.go create mode 100644 dashboard/resources/stats.html.go create mode 100644 dashboard/resources/styles-main.css.go create mode 100644 dashboard/resources/views-browser.html.go create mode 100644 dashboard/resources/views-stats.html.go diff --git a/dashboard/build b/dashboard/build index 60d6fcdcb2c..0ae99eb45db 100755 --- a/dashboard/build +++ b/dashboard/build @@ -5,6 +5,7 @@ grunt build git clean -x -f dashboard/dist for i in `find dashboard/dist -type f`; do + file=$(echo $i | sed 's#dashboard/dist/##g' | sed 's#/#-#g') go build github.com/jteeuwen/go-bindata - ./go-bindata -pkg "dist" -toc -prefix dashboard/dist $i + ./go-bindata -pkg "resources" -toc -out dashboard/resources/$file.go -prefix dashboard/dist $i done diff --git a/dashboard/resources/bindata-file.go b/dashboard/resources/bindata-file.go new file mode 100644 index 00000000000..82dd5aaa93f --- /dev/null +++ b/dashboard/resources/bindata-file.go @@ -0,0 +1,11 @@ +package resources + +func File(name string) ([]byte, bool) { + data, ok := go_bindata[name] + + if ok == false { + return nil, false + } + + return data(), true +} diff --git a/dashboard/resources/bindata-toc.go b/dashboard/resources/bindata-toc.go new file mode 100644 index 00000000000..261416f47c6 --- /dev/null +++ b/dashboard/resources/bindata-toc.go @@ -0,0 +1,7 @@ +package resources + +// Global Table of Contents map. Generated by go-bindata. +// After startup of the program, all generated data files will +// put themselves in this map. The key is the full filename, as +// supplied to go-bindata. +var go_bindata = make(map[string]func() []byte) \ No newline at end of file diff --git a/dashboard/resources/browser.html.go b/dashboard/resources/browser.html.go new file mode 100644 index 00000000000..9b1a8dacb5f --- /dev/null +++ b/dashboard/resources/browser.html.go @@ -0,0 +1,76 @@ +package resources + +import ( + "bytes" + "compress/gzip" + "io" +) + +// browser_html returns raw, uncompressed file data. +func browser_html() []byte { + gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ +0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0x8c,0x54, +0x4f,0x6f,0xdb,0x3e,0x0c,0xbd,0xf7,0x53,0xb0,0xbe,0xcb,0x3e, +0xfc,0xf0,0x43,0xdb,0x41,0x36,0xd0,0x0d,0x3d,0xf4,0xb6,0xcb, +0x80,0x0d,0x45,0x51,0x28,0x12,0x13,0xab,0xb3,0x45,0x4d,0xa2, +0x93,0xe5,0xdb,0x4f,0xf2,0x9f,0xd6,0x6d,0xb3,0x21,0x39,0xc4, +0xb4,0xc8,0xf7,0x44,0xf2,0xbd,0x44,0x5e,0x1a,0xd2,0x7c,0xf4, +0x08,0x2d,0xf7,0x5d,0x73,0x21,0x2f,0x85,0x78,0xb0,0x5b,0xe8, +0x18,0xee,0xef,0xe0,0xea,0xb1,0x81,0xf1,0x23,0x73,0x16,0x74, +0xa7,0x62,0xac,0x0b,0x47,0xe2,0x39,0xa6,0x0a,0x61,0xf1,0x66, +0x7a,0x5c,0x4f,0x8f,0xab,0xa2,0x01,0x79,0xf9,0x80,0xce,0xd8, +0xed,0xa3,0x10,0xaf,0x6c,0x6b,0xaa,0x33,0xd8,0xfe,0x41,0x73, +0x7d,0x0e,0xcd,0xdf,0xf0,0x3b,0x9e,0x29,0xf2,0x41,0x73,0x02, +0x3f,0x02,0x85,0x78,0x03,0xce,0xf7,0xa0,0x32,0x39,0x48,0x61, +0x8f,0xac,0x40,0xb7,0x2a,0x44,0xe4,0xba,0x18,0x78,0x2b,0x52, +0xb7,0xab,0x54,0xcb,0xec,0x05,0xfe,0x1a,0xec,0xbe,0x2e,0xbe, +0x8b,0x6f,0xb7,0xe2,0x0b,0xf5,0x5e,0xb1,0xdd,0x74,0x58,0x80, +0x26,0xc7,0xe8,0x12,0xee,0xfe,0xae,0x46,0xb3,0xc3,0x05,0xc9, +0x96,0x3b,0x6c,0x90,0xb5,0x81,0xcf,0x81,0x0e,0x11,0x83,0xac, +0xa6,0xb3,0x15,0xb3,0x53,0x3d,0xd6,0x85,0xc1,0xa8,0x83,0xf5, +0x6c,0xc9,0xad,0xf8,0x8a,0x8f,0x85,0x7b,0x8b,0x07,0x4f,0x81, +0x57,0x55,0x07,0x6b,0xb8,0xad,0x0d,0xee,0xad,0x46,0x31,0xbe, +0x2c,0xb8,0x34,0x33,0x7c,0xed,0x94,0x46,0xd8,0xaa,0x94,0x25, +0x57,0xa6,0x2f,0x50,0xce,0x80,0xf2,0xbe,0x43,0xc1,0x34,0xe8, +0x56,0x8c,0x09,0xef,0x76,0x60,0x1d,0x70,0x8b,0x10,0x88,0x18, +0x8c,0x0d,0xa8,0x99,0xc2,0x11,0xf2,0xb2,0x2e,0x5e,0xb4,0xe9, +0xac,0xfb,0x09,0x01,0xbb,0xba,0x88,0x7c,0xec,0x30,0xb6,0x88, +0xa9,0x97,0x36,0xe0,0x76,0x39,0xa9,0x7a,0x65,0x5d,0xa9,0x63, +0xda,0xfa,0x85,0xac,0x96,0x1d,0xcb,0x0d,0x99,0x23,0xb8,0x9d, +0x48,0x37,0xd7,0x45,0xde,0xc9,0xbc,0x92,0x55,0xaf,0x6f,0x0c, +0x3a,0x5f,0x29,0xfd,0xa2,0xe4,0x66,0xac,0x6f,0x13,0xfe,0x58, +0x34,0x3f,0x68,0x00,0x15,0x10,0x86,0x68,0x53,0xe3,0xca,0x81, +0x8c,0x1c,0xc8,0xed,0x1a,0x1a,0xd8,0x28,0x46,0x23,0xab,0xf9, +0x00,0x26,0x5c,0x28,0xd3,0x26,0x50,0x45,0x04,0xa9,0xe6,0x76, +0xb3,0xa6,0x9f,0xaa,0x6a,0xc5,0x5b,0x6a,0xea,0xab,0xa2,0x19, +0xfc,0x2e,0x28,0x83,0x70,0xa4,0x21,0x2c,0x70,0x59,0xa9,0x06, +0x98,0xc0,0xf6,0x3e,0xd0,0x7e,0xce,0xe1,0x6f,0x8f,0xc1,0xa2, +0xd3,0x58,0xca,0xca,0x2f,0x83,0xac,0x4c,0x76,0x62,0xb4,0x9b, +0xd7,0xd1,0x26,0xc9,0x21,0x06,0x9d,0xa6,0xa3,0x03,0x86,0xa7, +0x74,0xbf,0x27,0x97,0x54,0x8d,0x15,0xc6,0xff,0x45,0x6c,0x6d, +0xff,0x12,0x94,0xd9,0xc7,0x69,0xac,0x11,0x74,0x1e,0xc7,0x73, +0x24,0xf7,0x5f,0xd5,0xd9,0xcd,0x14,0x95,0x7d,0x12,0xe6,0x23, +0xcb,0xe9,0x8e,0xe1,0xd6,0x98,0x69,0xca,0x68,0x19,0x81,0xc2, +0xe8,0x19,0xab,0x55,0xf6,0xe8,0x62,0x3e,0x68,0x31,0x89,0x30, +0xfd,0x9c,0x12,0xcc,0xd8,0x3d,0x58,0x93,0xe5,0x35,0x4f,0xf3, +0xde,0x8a,0xac,0x79,0x36,0xed,0x24,0xfa,0xa2,0x76,0x95,0x4a, +0x9b,0x57,0x57,0xad,0xa7,0x98,0xe2,0x38,0xeb,0x12,0x44,0x4f, +0x66,0x48,0xb6,0x7a,0xd7,0xf8,0x79,0xd8,0xf9,0xfd,0x1d,0x56, +0x56,0xd9,0x8c,0xa3,0x3b,0xf3,0x9f,0xe3,0x9f,0x00,0x00,0x00, +0xff,0xff,0x4a,0x5c,0x90,0x9e,0x2c,0x05,0x00,0x00, + })) + + if err != nil { + panic("Decompression failed: " + err.Error()) + } + + var b bytes.Buffer + io.Copy(&b, gz) + gz.Close() + + return b.Bytes() +} + +func init() { + go_bindata["/browser.html"] = browser_html +} diff --git a/dashboard/resources/index.html.go b/dashboard/resources/index.html.go new file mode 100644 index 00000000000..aa411167fa7 --- /dev/null +++ b/dashboard/resources/index.html.go @@ -0,0 +1,261 @@ +package resources + +import ( + "bytes" + "compress/gzip" + "io" +) + +// index_html returns raw, uncompressed file data. +func index_html() []byte { + gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ +0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0xb4,0x59, +0x6d,0x6f,0xe3,0xc6,0x11,0xfe,0xde,0x5f,0xc1,0x30,0xe8,0xa7, +0x90,0xd4,0xbe,0xbf,0xf8,0x24,0x17,0x57,0x5f,0x82,0x04,0x48, +0x8a,0xa0,0x41,0x8b,0x16,0x41,0x50,0xd0,0x14,0x2d,0xb1,0x47, +0x8b,0xaa,0x48,0xdb,0x77,0x57,0xf4,0xbf,0xf7,0x99,0x5d,0x52, +0xb6,0x65,0x59,0x76,0x2e,0xad,0x00,0x9d,0x38,0xe4,0xee,0xec, +0xbc,0x3c,0xf3,0xcc,0xd0,0x37,0xff,0x62,0xd9,0x55,0xc3,0xc7, +0x6d,0x9d,0xac,0x87,0xeb,0xf6,0xfc,0x77,0xf3,0x2f,0xf2,0xfc, +0xe7,0xe6,0x2a,0x69,0x87,0xe4,0xbb,0xaf,0x13,0xfb,0xcb,0x79, +0x12,0x3e,0x73,0x7a,0x9a,0x54,0x6d,0xd9,0xf7,0x8b,0x74,0xd3, +0xe5,0xff,0xec,0xb1,0x22,0x6f,0x6a,0x1f,0x7f,0x5c,0xfc,0xb1, +0xe9,0x79,0x32,0xff,0xe2,0xe7,0x7a,0xb3,0x6c,0xae,0x7e,0xc9, +0xf3,0x7b,0x6d,0x0f,0x55,0xbd,0x42,0xdb,0x09,0x35,0xee,0x35, +0x6a,0x9e,0xdb,0xbf,0x1a,0x46,0x15,0x74,0xe3,0xfc,0xc8,0xfe, +0xb0,0x31,0xcf,0x1f,0x6d,0xa6,0x73,0xea,0x72,0x49,0x17,0xb8, +0xbc,0xae,0x87,0x32,0xa9,0xd6,0xe5,0xae,0xaf,0x87,0x45,0x7a, +0x33,0x5c,0xe5,0xb0,0xf6,0xc1,0xa3,0xf5,0x30,0x6c,0xf3,0xfa, +0x5f,0x37,0xcd,0xed,0x22,0xfd,0x5b,0xfe,0x97,0xb7,0xf9,0x45, +0x77,0xbd,0x2d,0x87,0xe6,0xb2,0xad,0xd3,0xa4,0xea,0x36,0x43, +0xbd,0xc1,0xbe,0xef,0xbe,0x5e,0xd4,0xcb,0x55,0x3d,0xed,0x1c, +0x9a,0xa1,0xad,0xcf,0xeb,0xa1,0x5a,0x26,0xcb,0xb2,0x5f,0x5f, +0x76,0xe5,0x6e,0x39,0x9f,0xc5,0xbb,0x0f,0x74,0x6f,0xca,0xeb, +0x7a,0x91,0x2e,0xeb,0xbe,0xda,0x35,0xdb,0xa1,0xe9,0x36,0x0f, +0x34,0xa6,0x4f,0x17,0xde,0x36,0xf5,0xdd,0xb6,0xdb,0x0d,0x0f, +0x56,0xdd,0x35,0xcb,0x61,0xbd,0x58,0xd6,0xb7,0x4d,0x55,0xe7, +0x41,0x98,0xf6,0xb5,0xcd,0xe6,0x7d,0xb2,0xde,0xd5,0x57,0x8b, +0x94,0x5c,0x38,0x9b,0xcd,0xae,0xb0,0xa9,0x2f,0x56,0x5d,0xb7, +0x6a,0xeb,0x72,0xdb,0xf4,0x45,0xd5,0x5d,0xcf,0xaa,0xbe,0xff, +0xc3,0x55,0x79,0xdd,0xb4,0x1f,0x17,0x3f,0x75,0x37,0xbb,0xaa, +0xfe,0xea,0xa7,0x72,0xd3,0x7f,0xf5,0xe3,0xae,0x3b,0x13,0x8c, +0x65,0x12,0x5f,0x15,0xbf,0xcd,0x50,0xb6,0x4d,0x95,0x19,0x48, +0x16,0x5f,0xcf,0x58,0x9a,0xec,0xea,0x76,0x91,0xf6,0xc3,0xc7, +0xb6,0xee,0xd7,0x75,0x0d,0xc3,0x08,0x78,0x8b,0x74,0xa8,0x3f, +0x0c,0xa4,0xf9,0x37,0xdb,0x72,0xd1,0x2d,0xeb,0x60,0x0b,0xd9, +0xa0,0xf1,0x1d,0x4f,0x7f,0xf5,0xc9,0x61,0x45,0xbc,0xa6,0xcf, +0x65,0xb7,0xfc,0x98,0xfc,0x7b,0x2f,0xd2,0x67,0x5b,0x2e,0x97, +0xcd,0x66,0x75,0x96,0x48,0xb6,0xfd,0xf0,0xe6,0xd1,0xa3,0xeb, +0x72,0xb7,0x6a,0x36,0x67,0xc9,0xa3,0x07,0xff,0xd9,0x5f,0xad, +0xf9,0x81,0x2a,0x72,0x2a,0x8f,0x0e,0x9c,0x25,0x69,0x74,0x21, +0xa1,0x70,0x26,0x70,0x21,0xcd,0x92,0xf4,0xdb,0xba,0xbd,0xad, +0x87,0xa6,0x2a,0x93,0x3f,0xd5,0x37,0x35,0xee,0xec,0x6f,0x64, +0xc9,0xdb,0x5d,0x53,0xb6,0x59,0xd2,0x63,0x79,0xde,0xd7,0xbb, +0xe6,0xea,0xcd,0x53,0xdd,0x77,0x75,0xb3,0x5a,0x0f,0x67,0x09, +0xa2,0xf1,0xac,0xa5,0xe1,0x2b,0xc6,0x8b,0x37,0xc7,0x5d,0x7d, +0xc6,0xa1,0xe6,0x6a,0x07,0x9c,0x1d,0x38,0x75,0xd9,0xed,0x96, +0xf5,0xee,0x2c,0xd9,0x74,0x9b,0xfa,0xe1,0xa6,0xfd,0x65,0x79, +0xb0,0xa1,0xea,0xda,0x0e,0xeb,0xbf,0xe4,0xb5,0xa9,0x2b,0xfe, +0xd8,0x02,0x4a,0x4f,0xbe,0xac,0xab,0x6e,0x57,0x12,0xdc,0x4f, +0x69,0x3d,0x5b,0x77,0xb7,0xf5,0xee,0x40,0xf7,0x93,0xfd,0x37, +0x1b,0x18,0x07,0x78,0x1d,0x57,0x72,0xd4,0xa1,0x18,0xab,0xfc, +0xb2,0x1b,0x86,0xee,0xfa,0x30,0xed,0x4f,0x37,0x7f,0xf5,0x2a, +0x2d,0xcf,0x29,0xf9,0xf2,0xaa,0xeb,0x86,0x27,0x6e,0x84,0x42, +0x3d,0x4b,0x38,0x63,0xbf,0x3f,0x92,0xe5,0xff,0x13,0x82,0x46, +0x8b,0x87,0x6e,0x7b,0x16,0xf0,0x71,0xd4,0xde,0x60,0x33,0xc2, +0x5b,0x77,0x7d,0xde,0x76,0xab,0xee,0xc0,0xf0,0x7b,0x3d,0x64, +0x3d,0x30,0x56,0xde,0x0c,0x5d,0xc2,0xc2,0xcf,0x9b,0x27,0x2b, +0xd7,0x23,0x5c,0x9f,0x56,0xd6,0x83,0x20,0xb8,0x27,0x0f,0x0f, +0xcc,0x79,0x62,0x52,0x7f,0xbb,0x3a,0x62,0x56,0x08,0x5f,0xd3, +0xb6,0x40,0x9e,0xf7,0xfe,0xe9,0x71,0xd1,0xf4,0x0f,0xf9,0x7d, +0xec,0x8f,0x19,0x45,0x9f,0x65,0xd3,0x6f,0xdb,0x12,0x09,0x68, +0x36,0x84,0xac,0xfc,0xb2,0xed,0xaa,0xf7,0xc7,0x97,0x02,0xa1, +0x14,0xfb,0x36,0x07,0x35,0xae,0x10,0x93,0xeb,0x66,0xb9,0x6c, +0xeb,0xa7,0x6b,0x0f,0x23,0xbc,0xed,0xee,0xea,0x5d,0xbd,0xcc, +0x2f,0x0f,0xd9,0x28,0x78,0x41,0x20,0xe8,0x9b,0x4f,0x35,0xac, +0x14,0xc7,0x8c,0x9c,0x2a,0x4c,0x4a,0xf9,0x6c,0x58,0x9f,0x62, +0xeb,0x57,0xb8,0xf6,0x5a,0xb7,0x82,0x8e,0x29,0xcb,0xdc,0x1f, +0x3b,0x31,0x54,0xec,0xa8,0xa6,0x42,0xcb,0xaa,0x77,0x87,0xc9, +0xa6,0x7f,0xe7,0xb3,0x91,0xa8,0xe7,0xb3,0xd8,0x9a,0xe7,0xc4, +0xd4,0x23,0x87,0xaf,0x79,0xec,0xa4,0xef,0xee,0x3b,0x29,0x6e, +0xc5,0x67,0x63,0x6d,0xf6,0xbb,0x8a,0xba,0x41,0x89,0x9e,0x42, +0x23,0x40,0x9a,0x04,0x75,0x63,0x7b,0x1c,0x83,0xb1,0x87,0xa3, +0x0a,0xa9,0x4f,0xcf,0xe7,0xb3,0xb8,0xfb,0x88,0xaa,0xcb,0x5d, +0x77,0x87,0x12,0xfa,0x7c,0x65,0xcb,0xe6,0x36,0x69,0x96,0x8b, +0x34,0x96,0x7f,0x7a,0xdf,0x81,0xf6,0x4f,0xee,0x21,0x90,0x9e, +0xff,0x18,0xaf,0x13,0xc0,0x61,0x5e,0x3e,0x68,0x94,0x3d,0x3a, +0xe5,0xaa,0x19,0xd6,0x37,0x97,0xb1,0x3f,0x86,0x22,0x98,0x51, +0x30,0xd2,0x10,0x92,0xf9,0xac,0xc4,0xc1,0xd0,0x78,0x44,0xff, +0x83,0x8a,0x79,0x70,0x7c,0x58,0x52,0x3e,0xee,0xc5,0x71,0x25, +0x9d,0x70,0xb0,0x30,0x2c,0xa6,0x6a,0x03,0x1e,0x7a,0xd0,0xed, +0x22,0xe5,0x05,0x4f,0x83,0xf6,0xef,0xcb,0x8f,0xf5,0xee,0x1f, +0x10,0x3e,0x5c,0xb7,0x9b,0x7e,0xaf,0xea,0xee,0xee,0xae,0xb8, +0x93,0x45,0xb7,0x5b,0xcd,0x30,0x3d,0xb0,0x19,0x36,0x8f,0x4b, +0xce,0x3e,0xd0,0x10,0x70,0x6c,0x21,0x47,0xb9,0xce,0xc2,0x53, +0x2c,0x5d,0xa4,0x08,0x67,0x9a,0x7c,0x8c,0xbf,0x47,0x6b,0x2e, +0xd9,0xee,0x6a,0xe4,0xe6,0xb6,0x7e,0xdb,0x6f,0xeb,0x6a,0xf8, +0x33,0x75,0x82,0x45,0xfa,0xe1,0x87,0x66,0xf3,0x77,0x7c,0xd3, +0x84,0xa6,0xa4,0x3f,0x76,0xa4,0x09,0xc4,0x64,0xbd,0x00,0xfd, +0x98,0x34,0xa9,0x37,0x25,0x26,0xb6,0xfc,0xb2,0xac,0xde,0xaf, +0x76,0x1d,0xda,0x06,0x06,0xc4,0xfa,0x2e,0x79,0xb4,0x06,0x86, +0x9e,0xf5,0xdb,0xb2,0x42,0xae,0xa7,0x33,0x8e,0x45,0x64,0xf5, +0xf4,0xde,0xa9,0xfb,0xe1,0x19,0x26,0xc6,0x75,0xa0,0xa7,0x45, +0xfa,0xa5,0x96,0x6f,0xe5,0xbb,0xb7,0x69,0x02,0x13,0x7e,0xe0, +0xd2,0x14,0xdc,0xb8,0x4c,0xe9,0x42,0x0b,0x7b,0x61,0x4d,0xe1, +0xfc,0x24,0x65,0xc2,0x15,0xc6,0xf9,0xcc,0xcb,0xc2,0x4a,0x3f, +0x49,0x5c,0xcb,0x0a,0xb3,0x90,0x2f,0x84,0xd1,0x99,0x72,0x85, +0x60,0xb8,0xc7,0x6c,0xa1,0xac,0x1a,0x7f,0xf7,0xf2,0xb3,0xe6, +0xd0,0xa7,0x22,0x15,0x5a,0x64,0x2c,0xae,0x36,0x3a,0x7f,0xa8, +0x0c,0xe2,0xa8,0xe4,0x42,0x28,0x59,0x18,0x29,0x27,0x33,0xb8, +0xd7,0x85,0x12,0x93,0x89,0x8f,0xed,0xff,0x94,0xce,0x5e,0x19, +0x83,0x6f,0xb8,0x61,0xe6,0xdd,0xe3,0x18,0x68,0x5d,0x48,0xe7, +0xab,0x9c,0xdb,0x42,0x38,0x99,0xb1,0x5c,0xf2,0xc2,0x2b,0x9e, +0x09,0x5b,0x18,0xa5,0x73,0x89,0xdb,0x52,0x67,0xc6,0x14,0xcc, +0x60,0x15,0xc3,0x1e,0x58,0x83,0x7b,0x06,0xd7,0xa0,0xc2,0x4c, +0x14,0xca,0x59,0x5c,0x2b,0xeb,0x32,0xd8,0xaa,0xcc,0x69,0xff, +0xb1,0xd2,0x0a,0x99,0x41,0x9d,0xb2,0x39,0xf4,0x70,0x99,0x71, +0x01,0x45,0xf0,0xbc,0x50,0xda,0x65,0xdc,0x15,0x8e,0xb9,0x70, +0x10,0x47,0x2e,0x0a,0x6f,0x04,0x5d,0x5b,0x9d,0xe9,0xc2,0x6b, +0x33,0x5e,0xbb,0xc2,0x3b,0x87,0x84,0xc8,0x82,0x09,0x9f,0xb1, +0x82,0x61,0xa7,0x19,0xaf,0xa7,0xe7,0xfa,0xb4,0x21,0xac,0x10, +0x0a,0x7e,0xc1,0x6e,0x64,0x03,0x31,0xd6,0x64,0x87,0x35,0x48, +0xe7,0xde,0x0e,0xce,0x2b,0x57,0x70,0x46,0xb7,0x98,0x82,0xd7, +0x06,0xcf,0xc9,0x79,0x6b,0x10,0x1e,0x0d,0x3b,0xc9,0x40,0xe6, +0x7d,0x25,0x0b,0x8f,0x6b,0x1c,0xad,0x4c,0x66,0x0b,0x17,0x34, +0x0a,0xc4,0x86,0xf3,0xf1,0x01,0x84,0xd3,0xd6,0x28,0x18,0x8f, +0xe0,0x67,0x0e,0x0e,0x32,0xe7,0x68,0xa7,0xf3,0x3a,0x0f,0x3b, +0x2b,0x4f,0xc7,0x87,0x70,0xe3,0x81,0x2d,0x2c,0x73,0x08,0x16, +0xd3,0xc1,0x06,0xee,0xf3,0x68,0x03,0xc3,0x86,0x10,0x2a,0x4e, +0xf6,0x5a,0x1b,0xf6,0x02,0xbf,0x85,0xd1,0x61,0xaf,0x71,0xa7, +0x2d,0x90,0x48,0x9e,0x56,0xb9,0x42,0x0a,0x33,0x1d,0x92,0x9e, +0x73,0x8d,0x1f,0x31,0x49,0x00,0x84,0x12,0xee,0x42,0xd0,0x42, +0x6b,0x32,0xef,0xa3,0xa1,0x1e,0x06,0x5b,0x3e,0xc2,0x28,0x7b, +0x8c,0xaa,0x5f,0x81,0xcc,0xf0,0x19,0x91,0x89,0x7a,0xd4,0x40, +0x20,0x87,0x7f,0x3a,0x82,0xce,0x5b,0x4f,0xf8,0x10,0x8e,0x9c, +0x15,0xe4,0xb2,0xf5,0x26,0x97,0x08,0xb8,0x83,0xc5,0xc8,0x53, +0xee,0x02,0x3a,0x38,0x2b,0xb4,0xb6,0xb9,0x10,0x85,0xf0,0x36, +0x40,0xda,0x44,0x08,0x0b,0x35,0x4a,0x2f,0xa0,0x53,0x15,0x0e, +0x41,0x63,0xb9,0x27,0xd5,0xc8,0xa5,0x62,0x50,0x13,0xb3,0x08, +0x44,0x78,0x46,0x85,0x82,0x70,0x9b,0x4c,0x15,0xda,0xe1,0x3c, +0x94,0x8a,0xcd,0xfc,0x98,0x06,0xe5,0x01,0x22,0xd4,0xa9,0x0f, +0xf8,0x55,0x08,0x1d,0x56,0x69,0x8e,0x6b,0x03,0xd6,0xf0,0x85, +0xf4,0xd3,0x35,0x57,0x10,0xfc,0x49,0x5b,0xfa,0x08,0x21,0xe8, +0x56,0x84,0xea,0xfd,0x2e,0x53,0x91,0x52,0x95,0xc5,0x03,0xa0, +0xd4,0xd9,0xf0,0x58,0xd1,0x63,0xc5,0x44,0x14,0x2a,0x5d,0x30, +0xb0,0x0a,0x0b,0xfb,0x45,0xc0,0x82,0x18,0xf7,0x07,0x13,0x5e, +0x60,0x29,0xc4,0x91,0x71,0x32,0x96,0x3c,0x02,0x03,0x6a,0x09, +0xf7,0x1c,0x0a,0x4b,0x00,0x1f,0x56,0xe7,0x70,0x12,0x41,0xa7, +0x22,0x36,0x78,0xa0,0x19,0xc7,0x39,0x06,0x31,0xd6,0x85,0xe1, +0x7a,0xbc,0x0e,0x35,0x78,0xc1,0x9d,0x42,0x0d,0xa3,0x14,0x14, +0x14,0x90,0x0d,0x0e,0xc8,0xa6,0xfa,0x02,0x8c,0x24,0xe1,0xe7, +0x71,0xb2,0x3f,0x3d,0xd3,0x7d,0xc6,0xcf,0x33,0x70,0x9a,0xcf, +0x7e,0x6b,0x77,0x10,0x92,0x7f,0x23,0x58,0xc4,0x9f,0x54,0x40, +0x81,0x87,0x49,0x8c,0x88,0x45,0x56,0xe0,0x05,0xed,0x34,0x62, +0x09,0x58,0x39,0x6e,0xc0,0x19,0x1c,0x85,0x28,0xa8,0x17,0xc0, +0x1f,0x50,0x90,0x11,0x2d,0x21,0x8f,0x51,0x0e,0x78,0x21,0xc5, +0x4b,0x14,0x88,0xda,0x40,0x40,0x75,0xc1,0x51,0xb0,0xd8,0x00, +0xc2,0x42,0xb0,0x00,0x9e,0x9c,0x53,0xa5,0xf3,0x28,0x00,0x68, +0x66,0x04,0xa3,0x20,0x70,0x53,0x08,0x51,0x5a,0x93,0x40,0xa0, +0xf6,0x1c,0x14,0x28,0x64,0xa1,0x54,0x20,0x0c,0x62,0x66,0xdc, +0x36,0xcc,0x51,0xd7,0x42,0xac,0xa3,0x70,0xda,0x18,0x00,0x5d, +0x38,0x6a,0x47,0x3a,0x10,0x0e,0xd5,0x36,0xb2,0x0c,0x02,0xe1, +0x96,0x7c,0x12,0xc2,0xb5,0xc1,0x35,0x3b,0x9e,0x40,0x95,0x06, +0xb6,0x46,0x8e,0xb5,0x77,0xc1,0x60,0x22,0x25,0xe4,0x18,0x7b, +0x03,0x11,0x45,0xa1,0xca,0x05,0xc2,0xc4,0xc9,0x7a,0x05,0xda, +0x94,0x3c,0x27,0x42,0x75,0x93,0xa0,0x89,0xb8,0x4f,0xd7,0xe2, +0x05,0x2a,0xb8,0x90,0x38,0x56,0xe0,0x78,0x69,0x32,0x09,0x05, +0x4a,0xab,0x29,0x2b,0xd9,0x41,0x96,0x5e,0x4f,0x35,0x0f,0x53, +0xad,0x10,0x63,0x6f,0x22,0xfa,0x2c,0xe3,0x15,0xfc,0x91,0xa1, +0x3b,0x83,0xc6,0x9c,0x08,0x15,0x63,0xbd,0x9c,0x24,0x05,0x76, +0xf1,0x96,0x62,0x0e,0x9c,0xab,0xc0,0x2f,0x0a,0xd4,0xa7,0xa8, +0x14,0x7d,0x7e,0xbf,0x08,0xd2,0x0b,0x08,0xc0,0x39,0xdc,0x5b, +0x6a,0xb6,0x06,0xc5,0x98,0x87,0x63,0xc4,0x28,0x44,0x05,0x17, +0xd2,0x81,0xb4,0x51,0x58,0x9c,0x64,0xaf,0x32,0x89,0x70,0x84, +0xf6,0x14,0x4d,0xcd,0x0e,0x4c,0xff,0x94,0xdc,0x3b,0xe3,0x89, +0x96,0x5e,0x60,0x3a,0x02,0x2a,0x35,0x1c,0x44,0x55,0xe8,0x31, +0xd3,0xa3,0x10,0xbd,0xab,0x18,0xf1,0xbf,0x46,0x23,0x81,0x56, +0x91,0x87,0x7e,0xa7,0xf2,0xfd,0x0a,0x08,0x3d,0x49,0xcc,0x53, +0x4a,0xb0,0xdb,0x4e,0x52,0x7c,0x78,0x3a,0xb7,0xd2,0x03,0x0d, +0x84,0x29,0x17,0x90,0x86,0xa0,0xe1,0x4c,0xb9,0x37,0x3d,0x3b, +0x70,0xe5,0x33,0x93,0x8b,0x59,0xcb,0x5a,0x8a,0x90,0x45,0x01, +0xfb,0x35,0x01,0x53,0xc9,0x16,0x43,0x87,0xa1,0xb2,0x35,0xce, +0xad,0x41,0x85,0xc2,0x83,0x2b,0x35,0x0b,0xbd,0x83,0x58,0x1e, +0x0f,0x40,0xd3,0xd1,0x75,0x2a,0x04,0x67,0x27,0xe9,0x85,0xfe, +0x59,0x78,0x49,0x85,0x84,0x89,0x42,0x6b,0x62,0x67,0x4c,0x01, +0x61,0x86,0x40,0x37,0xd6,0x56,0xb5,0xb1,0x5b,0x51,0x9d,0x59, +0xa9,0x50,0x1e,0x85,0xa1,0x16,0x6d,0x5d,0x68,0xb9,0x3c,0xcc, +0x42,0xe8,0x5c,0x08,0x21,0x77,0xf1,0xba,0xca,0x89,0x68,0xa8, +0x82,0x60,0x93,0xc6,0x85,0xa2,0x89,0x25,0x66,0xc0,0xc4,0x7a, +0x95,0xb7,0x0a,0x34,0xc0,0x4e,0xcf,0x16,0xeb,0x98,0x18,0xf1, +0xd7,0x31,0x0e,0x9f,0x17,0x4c,0x70,0x34,0x98,0xe2,0xbe,0x52, +0x50,0x95,0x3e,0x10,0xc7,0x38,0x2e,0xc2,0x1e,0x8c,0xae,0x93, +0x14,0x90,0x6c,0xc3,0x80,0x86,0x61,0x8a,0xc6,0x0f,0x4c,0x25, +0xb8,0x07,0x04,0x91,0xcf,0x36,0x0b,0x14,0xb7,0xce,0x15,0xf8, +0xe3,0x85,0x5a,0xa1,0xe8,0x51,0x1d,0x32,0x26,0x09,0x68,0x12, +0x83,0x36,0x65,0x05,0x5c,0x17,0xe8,0x98,0x8f,0x12,0x9a,0x91, +0xe1,0x01,0xd0,0xe8,0xfd,0x96,0xda,0x15,0x23,0xab,0x2c,0x99, +0x43,0x6d,0x09,0x1c,0x4d,0xe1,0xd4,0xaf,0x63,0x68,0x30,0x0f, +0x3a,0x1f,0x08,0xc6,0x3a,0xa2,0x61,0x83,0x79,0x07,0xb4,0x07, +0x4d,0x60,0x35,0x6a,0xbb,0x41,0x40,0x12,0x31,0x93,0x60,0x3c, +0x44,0x19,0x23,0xc2,0x2e,0x1f,0x69,0x30,0x5c,0x8f,0x55,0xac, +0x19,0xb5,0x35,0xaa,0x62,0x0a,0x5f,0xa6,0x85,0x0c,0x83,0xde, +0x54,0xc4,0x07,0x51,0xfd,0x74,0xd2,0xac,0xe4,0x07,0xcc,0x36, +0x64,0x3c,0x82,0x0d,0x9e,0xb2,0x86,0x4a,0x94,0xbc,0xb5,0xd4, +0x4b,0x9c,0x27,0x56,0x31,0x11,0xad,0x56,0x8a,0x51,0xaa,0x68, +0x94,0x11,0x9a,0x50,0x84,0x92,0xf2,0xd4,0xbd,0xac,0x71,0x04, +0x09,0x0a,0x6b,0x58,0xf2,0xed,0xa8,0xf6,0xf3,0xde,0x23,0x0c, +0xf8,0xd7,0x60,0xec,0xe0,0x1a,0x90,0x66,0x44,0x1b,0xe8,0x03, +0x16,0x93,0x38,0x4d,0xaa,0x4e,0x13,0xd5,0x83,0xaa,0x33,0x45, +0x23,0x9b,0x1f,0xa5,0x0a,0x63,0x84,0x33,0x94,0x2d,0xdc,0x36, +0xf4,0xb2,0xc1,0xc2,0xe0,0x30,0x4a,0x71,0xd1,0x0b,0xe3,0x7b, +0x86,0x63,0x1c,0x39,0x8d,0x63,0x50,0xa9,0x1a,0xcd,0x51,0xe6, +0x93,0x02,0x12,0x2e,0x0c,0x9c,0xa4,0xe9,0x48,0x30,0x6a,0x3a, +0x36,0xdb,0x5b,0xea,0x68,0x46,0x33,0xd9,0x81,0xe5,0x60,0x50, +0x63,0x09,0x97,0xd3,0x8d,0x17,0x0c,0xa0,0x4e,0x6c,0xc0,0xfb, +0xd4,0x96,0xf5,0xbe,0xb3,0x49,0x0c,0x5e,0x7e,0x14,0x02,0x40, +0x98,0x0b,0x00,0xc1,0x28,0xad,0x02,0xc1,0x32,0x2f,0x27,0x69, +0x5c,0xc5,0xa8,0x5d,0x63,0x9e,0x8d,0x0d,0x1e,0xa1,0x42,0x99, +0x02,0x13,0xd3,0xa2,0x20,0x9d,0xa6,0x53,0xcc,0x73,0x60,0x73, +0x01,0x4f,0x59,0x68,0xe7,0x93,0x1f,0x8e,0xf0,0x29,0xb3,0xc7, +0x6e,0xbd,0x3e,0xcd,0x78,0x65,0x56,0xef,0xfc,0x98,0x66,0xcc, +0xfb,0x96,0x5a,0xbd,0x33,0x84,0xa7,0xca,0xd2,0xa0,0x1d,0xcb, +0x98,0xbc,0x52,0x5c,0x4c,0x45,0x2a,0xd1,0x25,0x85,0x1f,0xa5, +0x2a,0x82,0x31,0x63,0xe3,0x0b,0x53,0x98,0x6d,0xfc,0x24,0x20, +0xed,0xd6,0xbc,0x18,0x67,0x60,0xdc,0x33,0x15,0x76,0x86,0x29, +0x9e,0xeb,0xd0,0x87,0xf0,0x1a,0x99,0xc7,0x41,0xa5,0x8d,0x90, +0x37,0x28,0x5b,0xe6,0x44,0x78,0x17,0x30,0xe8,0xc7,0x20,0x64, +0x43,0x47,0x78,0xa9,0x88,0xdc,0x25,0xc0,0x38,0x4a,0xe1,0xf8, +0x17,0x06,0x23,0x16,0xca,0x59,0xd1,0x00,0x12,0x8e,0x17,0x04, +0x1a,0x41,0xde,0x31,0xbc,0x50,0x45,0x29,0x8e,0x86,0x2e,0x8e, +0x86,0x06,0x53,0xad,0x0e,0x73,0x3a,0x8d,0x68,0x14,0x6f,0xbc, +0x11,0x48,0xdd,0xe6,0xc4,0xdb,0x2a,0x92,0x09,0xcd,0x7e,0xde, +0x89,0x9c,0x9a,0x84,0xa0,0x03,0xa8,0x4e,0x11,0x84,0x9c,0xe6, +0x6a,0x16,0xfa,0xf0,0x0b,0xe1,0xc8,0x63,0x83,0xa2,0xa9,0x30, +0x4e,0xa3,0xf4,0x72,0xe6,0x27,0x81,0xca,0xda,0x68,0x20,0x8a, +0xde,0x59,0x68,0x3a,0x0d,0x6e,0xd3,0x9b,0x2b,0x21,0x8a,0xd8, +0x10,0x93,0x14,0xaa,0x1e,0x93,0x79,0x1b,0x62,0x46,0xaf,0xcd, +0xfe,0x05,0x78,0xd1,0xac,0xa0,0x9c,0xc9,0x68,0x60,0x1d,0x67, +0xc3,0xf0,0x6a,0x8b,0x21,0x61,0x92,0xa8,0xb3,0x48,0xdf,0xe7, +0x61,0x1a,0x0e,0x7f,0x3b,0x70,0x78,0xef,0xc1,0x88,0x2f,0xd8, +0x24,0x55,0xf9,0xd8,0x22,0x18,0x91,0x28,0x46,0x5e,0x8a,0x01, +0xd6,0x20,0xc8,0x14,0x04,0xea,0xb0,0xe6,0xfb,0x03,0x94,0xfd, +0x0f,0xdf,0x06,0x9e,0xb9,0xd9,0xdf,0x1e,0xdc,0xa6,0x3f,0xee, +0xdd,0xff,0x59,0xef,0xfe,0xaf,0x7c,0xe3,0xe5,0x7c,0x16,0xff, +0x44,0x3a,0x9f,0xc5,0xff,0xe7,0xfd,0x6f,0x00,0x00,0x00,0xff, +0xff,0x81,0x9d,0x08,0x43,0xf8,0x1d,0x00,0x00, + })) + + if err != nil { + panic("Decompression failed: " + err.Error()) + } + + var b bytes.Buffer + io.Copy(&b, gz) + gz.Close() + + return b.Bytes() +} + +func init() { + go_bindata["/index.html"] = index_html +} diff --git a/dashboard/resources/scripts-browser-modules.js.go b/dashboard/resources/scripts-browser-modules.js.go new file mode 100644 index 00000000000..da47a3537f8 --- /dev/null +++ b/dashboard/resources/scripts-browser-modules.js.go @@ -0,0 +1,9500 @@ +package resources + +import ( + "bytes" + "compress/gzip" + "io" +) + +// scripts_browser_modules_js returns raw, uncompressed file data. +func scripts_browser_modules_js() []byte { + gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ +0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0xdc,0xbd, +0xeb,0x76,0x1b,0x47,0xd2,0x20,0xf8,0x7f,0x9e,0x82,0xa8,0xf6, +0xc0,0x55,0x42,0x12,0x22,0x64,0xbb,0xe7,0xeb,0x82,0x8b,0x18, +0x59,0x92,0x3f,0x6b,0xc7,0xb7,0xcf,0xd2,0xd7,0x97,0x05,0x21, +0x1d,0x90,0x28,0x8a,0x25,0x81,0x00,0x8c,0x02,0x45,0xc9,0x04, +0xfa,0x59,0xf6,0x3d,0xf6,0xc7,0x9e,0xb3,0x2f,0xb4,0xaf,0xb0, +0x71,0xcd,0x5b,0x15,0x20,0xc9,0xee,0x99,0xb3,0x67,0xbb,0x8f, +0x45,0x54,0x5e,0x23,0x23,0x23,0x23,0x23,0x23,0x23,0x22,0x3b, +0x97,0x37,0x8b,0x8b,0x4d,0xb5,0x5c,0xa4,0xb7,0xd5,0x62,0xb6, +0xbc,0x35,0x37,0x8b,0x59,0x79,0x59,0x2d,0xca,0x59,0x76,0xa7, +0x59,0x47,0x55,0xfd,0x70,0xbd,0x9e,0xbe,0x9f,0x57,0x6f,0xca, +0x74,0x79,0xfe,0x3a,0xbb,0x7b,0x3b,0x5d,0x1f,0xcd,0xcb,0xc5, +0xab,0xcd,0x55,0x01,0xdf,0x7d,0xfe,0x69,0x36,0xef,0x57,0x65, +0xf1,0xfa,0x3f,0x6e,0xca,0xf5,0xfb,0x3e,0xfe,0xa6,0xb2,0xc3, +0x75,0xb9,0xb9,0x59,0x2f,0x8e,0x24,0xbd,0xaa,0xff,0x46,0xfd, +0x50,0xde,0xa8,0x33,0xc8,0x07,0x45,0x41,0x6d,0x2c,0x96,0xb3, +0xf2,0x39,0x54,0xea,0x76,0xb9,0xb5,0x51,0xe7,0x24,0x4f,0xa6, +0xd8,0x6d,0x02,0x25,0xb0,0xb9,0xed,0x36,0x51,0x88,0x92,0x0e, +0x27,0x75,0xbb,0xe9,0x09,0xe4,0x72,0x0d,0xc8,0x5f,0xdc,0x5c, +0x9f,0x97,0xeb,0x84,0x33,0x97,0x97,0x02,0xa3,0x36,0x79,0x7a, +0xa2,0xbf,0x8e,0x07,0x47,0xd5,0xe2,0x08,0x41,0xd8,0xd9,0x41, +0x5e,0xac,0xcb,0xe9,0xa6,0xfc,0x69,0x85,0x1f,0x75,0xba,0xe4, +0xbf,0x3c,0x54,0x28,0x58,0x5e,0x6c,0x0a,0x49,0x7b,0x34,0xbd, +0xb8,0x2a,0xc7,0xf2,0x31,0x29,0xee,0x76,0xd1,0x10,0x4b,0xc8, +0xd7,0xfa,0xfd,0xeb,0xe9,0x06,0xbe,0x2e,0x96,0xeb,0xf2,0xe5, +0x7a,0xb1,0xdc,0xdc,0x5e,0x55,0x9b,0x32,0xdb,0x6e,0xc7,0x13, +0x63,0x11,0xff,0xd2,0x5c,0xce,0xa7,0xaf,0xb2,0x3b,0xee,0x65, +0x8c,0x1f,0x93,0xa2,0x73,0xb2,0xcb,0x0c,0xa7,0x38,0x10,0xab, +0xc5,0xa6,0x5c,0x2f,0xa6,0xf3,0xc7,0xd3,0xcd,0x34,0x2d,0xe7, +0xe5,0xb5,0x59,0x4c,0xaf,0x4b,0x33,0x83,0x4f,0xb3,0x7a,0xbb, +0xc9,0xee,0xaa,0xcb,0x54,0x80,0x98,0x5e,0x5c,0x94,0xab,0x8d, +0x2d,0x98,0xf1,0x40,0x36,0x57,0x15,0x83,0x6f,0x00,0x64,0xa3, +0xcd,0xfd,0x8f,0xf2,0xbd,0x4e,0x5b,0xf9,0x6e,0x35,0x85,0xe9, +0x31,0xaf,0xca,0xcd,0x37,0xef,0x7f,0x84,0xc6,0x8b,0xa4,0xde, +0xac,0xab,0xc5,0x2b,0x87,0x53,0xea,0xb2,0xaa,0x7f,0x84,0xf9, +0x2a,0xb0,0x69,0x3b,0x73,0xe6,0x02,0x5b,0x2e,0x38,0x6b,0x24, +0x0d,0x52,0x5a,0x4e,0xb0,0x56,0x33,0xcd,0xc3,0xcf,0xb1,0xd7, +0xfb,0x24,0x6f,0xa4,0x74,0xbb,0xde,0xd7,0x10,0xc6,0x55,0xcd, +0xba,0x5d,0x6a,0x6c,0x5c,0xcd,0x20,0x37,0x85,0xf1,0x6e,0xb7, +0x36,0xa1,0x8f,0x38,0x00,0xbc,0x76,0x2c,0xe0,0xdb,0x2d,0x26, +0x01,0x9d,0x38,0x82,0x96,0x69,0xaa,0x66,0xdb,0x6d,0xba,0x0f, +0x92,0x02,0xa0,0xa4,0xd9,0x9a,0x41,0xd6,0xa6,0x9c,0x3d,0x9d, +0xd5,0xfd,0xd5,0x72,0x95,0x42,0xdb,0x32,0xa2,0x57,0x37,0xd5, +0xac,0xd7,0xcb,0x71,0x34,0xae,0x5e,0x66,0x2c,0x28,0xd0,0xb8, +0xfd,0x0d,0xc4,0x21,0xa8,0xf2,0x53,0xfb,0x9b,0xe5,0xff,0xf6, +0xec,0xa7,0x1f,0x15,0xe7,0x8b,0xe5,0x72,0x95,0x65,0x26,0x4d, +0x78,0xba,0x43,0x4c,0xfb,0x44,0x1f,0x64,0x64,0x8c,0x83,0x91, +0xeb,0xcb,0x4e,0xe1,0xa6,0x5c,0xcc,0x5c,0x6f,0x44,0x23,0x59, +0x1e,0xa2,0x6a,0x5f,0x61,0xca,0xe4,0x1a,0x99,0xb1,0xd4,0x52, +0xb8,0xc6,0x08,0xed,0xa9,0xcd,0xa1,0xf2,0x8d,0x04,0x18,0xb6, +0x5f,0x3b,0xcc,0xcc,0x4c,0x3c,0x31,0x30,0x12,0x5b,0x64,0x6c, +0xe9,0xe6,0xba,0x9c,0x3f,0x9a,0xd6,0x65,0x4a,0xc0,0x4c,0x0a, +0xae,0x6a,0xa7,0x77,0x94,0xc2,0x6c,0xba,0x96,0xc7,0x58,0x0a, +0x86,0x7a,0x33,0x9f,0x17,0x05,0xe4,0x40,0x93,0x61,0xfe,0x9e, +0x66,0xb3,0x2c,0x0f,0xca,0xe1,0xba,0xd8,0x35,0x17,0xdc,0x2f, +0xe5,0xf5,0xf2,0x6d,0x19,0x2d,0xbb,0x8f,0x5a,0x71,0x95,0x99, +0x3b,0x4c,0xfc,0x2b,0x96,0x4d,0xb8,0x50,0x27,0x79,0xf8,0x8d, +0x2b,0xc5,0x4e,0x16,0x41,0x87,0xa0,0xfa,0x08,0x2e,0x02,0xa2, +0x89,0xc8,0x02,0x80,0xb6,0x7c,0x9a,0x58,0x3e,0xa3,0x69,0x84, +0xff,0x16,0xf8,0x4f,0xff,0x62,0xb9,0xb8,0x98,0x6e,0x74,0xcc, +0xd7,0xd3,0x15,0x95,0x30,0x31,0x7a,0x01,0xaf,0x98,0x8e,0x5c, +0xd6,0xf6,0xcc,0xad,0xf0,0x4c,0xe5,0x54,0xad,0x68,0x9f,0x15, +0x63,0xbb,0x0b,0xeb,0x4b,0x55,0x82,0xa3,0x5e,0xcd,0xab,0x4d, +0x9a,0x1c,0x25,0x59,0x36,0xbc,0x5c,0xae,0xd3,0xaa,0x38,0x31, +0x73,0x06,0x91,0x59,0xfc,0x70,0x7e,0x5a,0x0d,0xab,0x5e,0x2f, +0xe3,0x85,0x7c,0x14,0x92,0xca,0xb8,0x9a,0x4c,0x10,0x57,0x1d, +0x5a,0x43,0x55,0xfd,0xe4,0x7a,0xb5,0x79,0x8f,0x73,0xf7,0x13, +0xad,0xc1,0xdc,0x62,0x81,0x32,0x38,0x31,0x73,0x48,0xcc,0x84, +0x97,0xec,0x98,0x0d,0xa5,0xd2,0x47,0xb4,0x8c,0x1a,0xcd,0x7a, +0x53,0x93,0xe1,0xfa,0x8d,0xe6,0x7c,0x5e,0x4e,0x17,0x44,0x3f, +0x63,0x9c,0xe9,0x89,0xe9,0x9c,0x64,0x0a,0x48,0x7d,0xb3,0x5a, +0x2d,0xd7,0x9b,0x3e,0x77,0xf4,0x84,0x27,0x5b,0xf8,0x5f,0x87, +0x97,0x67,0x9f,0x37,0xf0,0x51,0x0c,0x8b,0x9b,0xe2,0x02,0x97, +0x47,0xb6,0xf3,0xe8,0x1b,0xc1,0x7c,0xb8,0xd9,0xac,0x99,0xaa, +0xdf,0x94,0xef,0x69,0x6d,0x12,0xdd,0xd0,0x3a,0x0e,0x16,0x29, +0xee,0xd4,0x01,0xe5,0x32,0x85,0xd3,0x64,0x25,0x58,0xfc,0x38, +0xe9,0x41,0x13,0xfd,0x75,0xb9,0x9a,0x4f,0x2f,0xca,0x74,0x7d, +0x7d,0x33,0xdf,0x54,0x8f,0xa7,0xf5,0x95,0x49,0x8e,0x3f,0x1b, +0x24,0x19,0xf0,0xbd,0xef,0x97,0xb7,0xe5,0x9a,0x66,0x3a,0x1b, +0x6a,0x27,0xd4,0x26,0x2c,0x6c,0x04,0xa4,0x3a,0xbf,0xd9,0x28, +0x15,0x34,0x76,0x1e,0x86,0x6d,0xb3,0x7e,0x7f,0x47,0xf5,0x92, +0xcd,0xfa,0xa6,0x44,0xe1,0x00,0xbf,0x48,0x5c,0xb8,0x9c,0xce, +0x6b,0x2f,0x65,0x90,0x27,0x38,0x62,0x9b,0x80,0x1f,0x79,0x0f, +0x7f,0xf6,0x12,0x9b,0x48,0xdf,0xf9,0xfa,0x7c,0x0d,0x20,0xf7, +0x37,0x65,0xbd,0x21,0xa0,0x32,0x9d,0x94,0xd5,0x74,0x5d,0x97, +0xc8,0xad,0x39,0x39,0xc7,0x7f,0x77,0x17,0xb4,0xa9,0xc3,0xf8, +0x77,0x52,0x6a,0x66,0x39,0x83,0xc5,0xe1,0xae,0x04,0x58,0x08, +0x64,0x87,0xc3,0x9d,0xec,0x40,0xd4,0x88,0x27,0x5d,0xc5,0x64, +0x62,0x65,0x2c,0x44,0x04,0x91,0xb7,0x2e,0x06,0xcc,0x01,0xbc, +0xa5,0x84,0x6f,0x94,0x82,0x78,0x8f,0xe8,0xb4,0xd1,0x2b,0x36, +0xc3,0x4b,0x06,0x89,0x2d,0xe1,0x4d,0x47,0xeb,0x08,0x01,0x77, +0x06,0x22,0xbc,0x80,0xbc,0x61,0x01,0xe2,0x94,0xe7,0x80,0xdd, +0x34,0xbb,0xdb,0x97,0xfd,0x2d,0xe2,0xda,0xe5,0x0f,0x5c,0x7e, +0x5d,0x9d,0xcf,0x61,0xde,0xd2,0x8b,0x9b,0xb5,0x99,0x55,0xeb, +0xec,0x6e,0xb6,0x3c,0x82,0xdf,0x05,0xfc,0x37,0x86,0xef,0xc9, +0x10,0x84,0xa0,0x79,0x89,0xd9,0x40,0x53,0x00,0x0e,0xfc,0x70, +0x24,0xa5,0xb2,0x14,0x24,0xba,0x16,0x81,0xb2,0x17,0x20,0x32, +0x22,0x82,0xcb,0xc5,0xa6,0x36,0xbf,0xde,0x4c,0xe7,0xd5,0x65, +0x55,0xae,0x01,0xdd,0xe5,0x8a,0xc8,0xd5,0x26,0x15,0xf6,0xd7, +0x76,0x7b,0x62,0x2c,0x5e,0xbe,0x55,0x69,0xcb,0x66,0xeb,0x1a, +0x56,0xb1,0xed,0x15,0xd0,0xad,0xeb,0xc2,0x4a,0x67,0xcc,0x83, +0x79,0x3a,0xa0,0xc2,0x5f,0xa7,0xf3,0xa2,0xd3,0xb1,0x8d,0x00, +0xeb,0x9a,0xcf,0xa5,0x8c,0x21,0xb6,0xaf,0x03,0x90,0xb2,0x45, +0x81,0x20,0xee,0x88,0xda,0x5d,0x2d,0x3b,0xdc,0x8f,0x06,0x41, +0x11,0x7d,0x84,0x1f,0xd0,0xaa,0x1b,0xaf,0xdf,0x43,0x63,0xc9, +0xb8,0xd1,0xd2,0x00,0x2e,0xab,0x39,0x6c,0x68,0xe5,0xac,0xf8, +0x84,0x0e,0x1b,0xcb,0x9e,0xbb,0xaa,0xea,0x67,0xd5,0xf5,0x6a, +0x2e,0x6b,0x66,0x2f,0x56,0xb9,0x47,0x97,0x6f,0x14,0x04,0xd3, +0xa1,0xb9,0x1b,0xba,0x81,0x7c,0xa8,0x46,0xb6,0xfb,0x74,0x6c, +0xe9,0xfc,0x2f,0x78,0x37,0xa3,0x89,0x72,0xb0,0x9e,0x16,0x27, +0x16,0x7d,0xf1,0x41,0xe0,0xd9,0xf4,0xb2,0xfc,0x76,0x3d,0x7d, +0x85,0xad,0xa7,0xb3,0xe5,0xc5,0x0d,0xfe,0x90,0x93,0x4f,0x55, +0x6f,0x0a,0xc4,0x06,0x0a,0x22,0xb5,0x6e,0x45,0xdb,0x24,0x33, +0xb5,0x54,0x2a,0xb4,0x42,0x9f,0x1b,0x7b,0x2c,0x9f,0xb6,0x41, +0x42,0xa1,0x96,0x96,0x42,0x4f,0x78,0x28,0x19,0xae,0xf7,0x21, +0xf6,0xa1,0x3b,0x59,0xd6,0x5e,0x30,0xa5,0x32,0x24,0x9c,0x5a, +0xaa,0xd3,0x92,0x6e,0x34,0xc0,0x78,0x66,0x3f,0xad,0x1f,0xae, +0x56,0x28,0xe7,0xd1,0xf8,0x37,0x78,0xd8,0xf0,0xa8,0x09,0x79, +0xaf,0x34,0x59,0x7f,0xf3,0xfe,0xf9,0xf4,0x15,0x0e,0x2b,0xc5, +0x52,0xe3,0x13,0x90,0x66,0xa9,0xc8,0x94,0xea,0x3f,0x82,0x85, +0xcb,0x8d,0xf4,0x97,0xb7,0x8b,0x72,0xfd,0x38,0x1c,0xa5,0xc2, +0x85,0x55,0x3d,0x7c,0xce,0xaa,0x7a,0x7a,0x3e,0x2f,0x9f,0x5d, +0xac,0xab,0xd5,0x46,0xa6,0x07,0xd1,0x38,0x05,0x7e,0xdf,0x64, +0xfe,0xb8,0x23,0xa6,0x09,0x92,0x6f,0x62,0x47,0x45,0x85,0xe8, +0x74,0x99,0x62,0xa5,0x6e,0x17,0xff,0x05,0xbc,0x97,0x17,0x38, +0x8f,0xb3,0xac,0x97,0xdc,0x4f,0x7a,0xb6,0x10,0xad,0x43,0x9f, +0x5d,0xd5,0x1b,0x10,0xe8,0x1b,0xdd,0xd3,0xc9,0xac,0x58,0xd7, +0x94,0x8e,0x74,0xfd,0xc3,0xb4,0x7e,0x53,0xce,0x40,0x92,0x2a, +0x2f,0x52,0xdb,0x98,0x85,0x81,0x8a,0x8f,0x1c,0x24,0xf4,0x3d, +0x1e,0xf0,0xd1,0x05,0xf6,0x3c,0x14,0x10,0xdd,0x0e,0x26,0x03, +0x88,0x40,0xa9,0xcb,0xcd,0xbf,0xcf,0x97,0xe7,0xd3,0xf9,0x93, +0xb7,0x53,0xe6,0x1a,0x35,0x08,0x9c,0x97,0x8a,0x7c,0x38,0x71, +0xc3,0xd4,0x23,0x6c,0xcc,0x50,0x8a,0x93,0x21,0x6e,0x59,0x9d, +0x82,0x8a,0x12,0xaa,0x6a,0x10,0x5c,0x32,0x12,0x6b,0x84,0xb2, +0x5f,0xba,0xad,0x27,0x79,0x65,0xdb,0x4e,0x4c,0xc7,0x6b,0xd7, +0x1e,0x5d,0xb8,0xb0,0x97,0x03,0xad,0x05,0xd5,0xfc,0x69,0xbb, +0x98,0x2f,0x17,0xe5,0xa3,0xe5,0xea,0xfd,0x93,0xb7,0x38,0xa9, +0xf5,0xfa,0xc2,0xcc,0x00,0x95,0xc4,0x6d,0x91,0x27,0xe0,0x87, +0x77,0x6a,0x97,0x2e,0xae,0xa6,0x35,0x89,0x30,0x50,0x5c,0xcf, +0x9c,0x38,0x25,0x28,0x06,0x2f,0xe7,0xb3,0xc7,0xde,0xd1,0x83, +0x81,0xc1,0x72,0x06,0x98,0x7d,0x33,0x07,0xdb,0xd7,0x3a,0x80, +0x48,0x04,0xa2,0x2e,0xe4,0xbb,0xcf,0x9f,0xb8,0x88,0xf8,0x17, +0x6c,0x32,0x22,0xf6,0x70,0x53,0x00,0xc7,0x62,0x36,0x2f,0xb5, +0x65,0x29,0x8f,0x67,0x75,0x44,0x31,0x82,0x84,0xbb,0xa9,0xd4, +0x75,0xf2,0x23,0x27,0x8c,0x31,0x7f,0x12,0xcb,0x91,0x2a,0x68, +0x63,0x91,0xfe,0x74,0x36,0x63,0x00,0x99,0xe0,0xbc,0x6a,0x38, +0x41,0x3b,0xed,0x16,0x07,0x02,0x72,0x9e,0xff,0x19,0x9d,0xbc, +0xe0,0x80,0xe8,0xe7,0xc2,0x04,0xf8,0x4b,0xf7,0xdd,0x23,0x9c, +0x04,0x5c,0x12,0x4f,0xeb,0xfa,0xa6,0xac,0xbd,0x59,0x20,0xf1, +0x40,0x78,0x90,0x61,0x1d,0xc0,0xb0,0x6d,0x66,0xf8,0x10,0x20, +0x05,0x5d,0x16,0x7e,0x85,0xf2,0x98,0xe9,0x44,0xb2,0xe6,0x62, +0x49,0xbd,0xd3,0xf4,0x77,0xbb,0x58,0x33,0x3e,0x7c,0x64,0x77, +0xb3,0xd6,0x69,0x63,0x91,0x9c,0x50,0x3c,0x73,0xd8,0x57,0x0c, +0xf2,0x5a,0x61,0xaa,0x22,0x1c,0x32,0xf4,0x32,0x65,0xd9,0x90, +0x60,0x8c,0x17,0x54,0xd8,0x73,0xb6,0x4b,0x78,0xdd,0xa2,0x18, +0xa7,0xc3,0x61,0x18,0x61,0x3f,0x7a,0xb7,0x01,0xa9,0x02,0x30, +0x45,0x3f,0x47,0x69,0xc8,0x7f,0x08,0x3e,0xca,0xb1,0x45,0x4c, +0xc8,0x22,0xa8,0x44,0x96,0xbb,0xe3,0xb8,0xed,0x61,0x44,0x79, +0x28,0x13,0x02,0xec,0x38,0x2b,0x30,0xb7,0x94,0xb2,0x04,0x10, +0xd7,0xdf,0x3d,0xff,0xe1,0x7b,0x6a,0xd3,0x7e,0x65,0x26,0x42, +0xe9,0xd5,0xe6,0x7a,0xfe,0x15,0x61,0xb5,0xdb,0xc5,0x92,0x20, +0xd7,0x70,0xc9,0x6e,0x57,0xb1,0x0f,0x03,0xbe,0xe6,0x46,0x6d, +0x66,0xa6,0xdd,0xd8,0x94,0x22,0xa8,0x8c,0xc0,0x56,0x8b,0xd5, +0x4d,0x84,0x8d,0xeb,0xe9,0xa2,0x5a,0xdd,0xcc,0xa7,0x48,0x4b, +0x2f,0xd7,0x70,0x06,0xb8,0x78,0x83,0x78,0x40,0xa2,0xe0,0x4d, +0x9b,0xc6,0x8f,0x24,0x22,0xe3,0x02,0x29,0x75,0x0a,0x22,0xfb, +0x23,0x2c,0x09,0x32,0x02,0xa5,0x5d,0xc8,0x07,0x96,0x95,0xdf, +0x44,0x80,0x7d,0x60,0x17,0x37,0xa5,0xe0,0x99,0x7e,0x2b,0x90, +0xf4,0xe1,0x92,0x09,0x93,0x2b,0xd1,0x5f,0x38,0x4c,0xfa,0x1d, +0x3e,0x83,0xa5,0x7b,0xb1,0xd1,0x1e,0x6b,0xfd,0xc2,0x26,0xa2, +0x12,0x79,0xda,0x1c,0xe8,0x76,0x9b,0xe0,0x24,0xc2,0x9c,0x4c, +0xfd,0x64,0x8b,0x34,0x69,0xe2,0xaf,0x16,0x2c,0x3f,0xc1,0x5f, +0x6e,0xb8,0x0b,0x81,0x30,0x07,0x27,0x5c,0x22,0x0a,0xda,0x26, +0x95,0x19,0xd7,0x46,0x59,0xb2,0xb9,0x5c,0x82,0x3c,0xaf,0xf2, +0x95,0x14,0x6e,0xdd,0x42,0x51,0xb4,0x45,0x4d,0x12,0xc8,0x64, +0xf6,0x04,0x30,0x3a,0x54,0x01,0x37,0x4e,0x18,0xcd,0xbd,0x24, +0xcb,0xa3,0xe6,0x7f,0x45,0xd2,0x60,0x24,0x2c,0xd7,0x0f,0x71, +0x4b,0x38,0xd0,0x74,0x5c,0xd8,0x6b,0xd6,0x16,0xa6,0xb3,0x2f, +0x8d,0x84,0xb8,0x1f,0x8f,0x69,0x3c,0xa1,0x51,0xd6,0x85,0xb6, +0x74,0x81,0xfb,0x3d,0xd2,0x39,0x6c,0x21,0x92,0x76,0x60,0x47, +0xea,0x50,0x3f,0x56,0xa9,0xc5,0xd3,0xe0,0x44,0x8e,0x11,0x75, +0xd2,0x5f,0xdd,0xd4,0x57,0xbc,0x0b,0xeb,0xd9,0xf6,0xba,0x5c, +0xbf,0x2a,0x19,0x04,0x23,0x73,0x60,0x2b,0xd9,0xed,0x17,0x3e, +0xfc,0xd3,0xe8,0x76,0x0b,0x09,0x76,0xd3,0xb1,0x9d,0xf9,0x73, +0x37,0x0a,0x9a,0x1f,0x4b,0xd6,0x84,0xe7,0x2f,0xcb,0xe9,0x4f, +0xc0,0x6b,0x1f,0x07,0xe4,0x2f,0x92,0xc2,0x07,0x17,0x91,0x93, +0x14,0x80,0xe0,0xe8,0x23,0x5a,0x46,0x94,0x26,0x4b,0xc7,0xdb, +0x5e,0x81,0xfd,0xcd,0x96,0xeb,0x9f,0xd7,0xcb,0x15,0x41,0x5e, +0x6f,0xde,0xc3,0x76,0x45,0x27,0x33,0xd5,0xda,0x20,0xfb,0xa4, +0x64,0x15,0xa9,0xed,0x69,0x10,0xa9,0xf2,0x62,0x4a,0x15,0x45, +0x2b,0x73,0x35,0x5d,0x3f,0xdc,0xa4,0x27,0x78,0xc4,0xfe,0x4f, +0x10,0xd5,0x84,0xa5,0xf7,0x58,0x55,0x32,0xaf,0xe0,0x34,0x3e, +0xc8,0xcc,0x72,0x5d,0xbd,0xb2,0x75,0x80,0x96,0x2f,0xea,0xfa, +0x67,0x10,0x06,0xaa,0x77,0x20,0xc4,0xca,0x66,0x57,0x1d,0x1f, +0x0f,0x33,0xe9,0xdf,0xcf,0x87,0x49,0xee,0x49,0x8f,0x66,0x3f, +0x6c,0xf2,0x5b,0xfb,0xf1,0x8f,0xb8,0xdf,0x55,0xb3,0x59,0x29, +0xa7,0xa9,0x72,0x1e,0x9e,0x66,0xca,0x39,0x4b,0x99,0x26,0x59, +0x00,0x67,0xc4,0x45,0xac,0x1a,0x91,0xba,0x16,0xd1,0x06,0x78, +0xf8,0x6a,0x3e,0x7d,0x9f,0x64,0xee,0xac,0x8b,0x13,0x3a,0xad, +0x16,0x75,0x8b,0x3c,0xca,0x27,0x31,0x4f,0xee,0xba,0x5a,0xde, +0x02,0x00,0xa5,0x3b,0x2d,0x60,0x8a,0x13,0xb7,0xa4,0x75,0x5e, +0xe3,0x57,0x04,0xa9,0x21,0xe6,0x55,0xe3,0x9a,0x00,0x01,0xba, +0x7c,0x87,0xe2,0x01,0x5f,0x78,0x68,0x1b,0x56,0x3e,0xe0,0xbb, +0x05,0x2a,0x35,0xa4,0x7f,0x61,0x2d,0xd8,0xe5,0xc1,0x52,0x16, +0xa6,0xf2,0xe2,0xea,0x13,0xd6,0x80,0x54,0xb8,0x7d,0xc9,0x2a, +0x5a,0xa4,0x39,0x10,0x75,0xec,0xa8,0x8d,0xfc,0x2a,0x5c,0x13, +0x7d,0x05,0x1a,0x87,0x32,0x0a,0x9b,0xc3,0x9b,0x10,0xc4,0x24, +0xb0,0x09,0x29,0xb5,0xdd,0xa6,0xcd,0xaa,0x45,0x02,0x2d,0x93, +0x26,0xa4,0x99,0xd7,0xed,0x06,0x73,0x96,0x7d,0x22,0xc4,0x06, +0xa6,0xee,0xa5,0x2c,0x84,0xc7,0x9c,0x96,0xda,0x43,0x24,0xb1, +0x67,0xf8,0x5f,0x1e,0x01,0x9d,0x32,0xea,0x8b,0xb0,0x67,0x93, +0x5a,0x90,0x9a,0xa3,0xea,0x70,0x95,0xcc,0xb2,0x82,0x7d,0xf0, +0x70,0xb9,0x91,0x7c,0xe7,0xfb,0x09,0x2c,0x53,0x85,0x22,0xcf, +0xfa,0xef,0x9d,0x5e,0x9c,0x15,0x0f,0xe0,0x36,0x04,0x27,0xed, +0x19,0xed,0x53,0x45,0xb3,0x1c,0x4f,0x72,0x92,0x73,0x07,0x59, +0x70,0x52,0x42,0x98,0x82,0x33,0xc7,0xcf,0xcb,0xba,0xda,0x54, +0x6f,0xcb,0x1f,0xe9,0x76,0x8c,0x07,0x4c,0x4d,0x99,0xfa,0xe6, +0x7c,0xb3,0x9e,0x5e,0x6c,0xbc,0x13,0x11,0xd0,0xfc,0x7a,0x71, +0x73,0x4d,0x87,0x5a,0x3e,0x0b,0xf1,0x36,0x1e,0x9c,0x83,0xca, +0x7a,0xf4,0xc3,0x74,0x73,0xd5,0xbf,0x9e,0xbe,0x4b,0x4f,0x8c, +0x24,0xc1,0x61,0xe8,0x38,0xd5,0x16,0xb7,0xdb,0x93,0x2c,0xeb, +0xa5,0x9a,0xf5,0x00,0xc1,0x5d,0xbd,0x4b,0x64,0xce,0x1d,0x78, +0xd3,0x1b,0x3a,0x10,0xff,0xad,0x9a,0x6d,0xae,0x7e,0x5a,0x7f, +0x57,0x56,0xaf,0xae,0x36,0x9e,0x8e,0x1d,0x38,0xf6,0x1a,0x15, +0xa9,0xdf,0x2c,0xd7,0xb3,0x72,0xfd,0xcd,0xf2,0x9d,0x21,0xb4, +0x78,0x47,0xa5,0xaa,0xa0,0x32,0x40,0xc4,0xa9,0x57,0x6c,0x94, +0x9c,0xd3,0x4f,0xc0,0x10,0x31,0xfe,0xc5,0x26,0xc9,0x46,0x5f, +0xe6,0xc9,0x2d,0xf6,0x43,0x62,0x02,0x0a,0x21,0x83,0xfc,0x04, +0x11,0x01,0xb3,0xfc,0x25,0x09,0xf9,0xc5,0x83,0x2c,0xb9,0x9e, +0xae,0x5f,0x55,0x24,0xab,0x50,0xbb,0x4c,0xf7,0xbd,0x06,0x47, +0xa2,0xcc,0x1e,0x7c,0xb2,0x2e,0x16,0x0f,0x52,0x9d,0x13,0x05, +0x2e,0xf3,0x21,0x1e,0xa5,0x16,0x84,0xb0,0xd1,0xe3,0x26,0x9b, +0x5b,0xc1,0xb9,0x02,0xf5,0x37,0xfb,0x1b,0x56,0xf8,0x3a,0x1f, +0x6a,0x4a,0xc6,0x1f,0xb4,0xd4,0x4b,0x08,0xcf,0x89,0xdf,0x62, +0x96,0xb7,0x0f,0xf0,0x83,0xb0,0xb8,0x12,0x9d,0x0f,0xe0,0xea, +0xe3,0x61,0x51,0x22,0x83,0x66,0x02,0xb9,0xec,0x20,0x79,0x30, +0xf1,0x12,0x5d,0x3d,0x75,0x68,0x2f,0x3a,0x3c,0xb7,0xd1,0x94, +0xf3,0x66,0x71,0x79,0x59,0x4b,0xa3,0xb9,0x97,0xc0,0xcd,0x0b, +0x34,0x05,0x74,0xfb,0x8c,0x7e,0x09,0x1b,0xf2,0xa6,0xb4,0x88, +0xc4,0xfa,0xf3,0xe5,0xbb,0x67,0xd5,0x6f,0x80,0x0a,0x58,0xd4, +0x3c,0xd6,0x63,0x48,0x6a,0xdd,0xc8,0x6c,0x51,0x18,0xf8,0x40, +0x07,0x8e,0xd2,0xd8,0xc9,0x69,0x01,0xe0,0x6e,0xb7,0x7c,0x85, +0x05,0x3f,0x49,0x0e,0xc0,0x11,0xc0,0xf9,0xf0,0xd1,0xb3,0x67, +0xde,0xa8,0x75,0x02,0xa0,0x4e,0x54,0x85,0x67,0xc0,0x63,0x2a, +0xa2,0xff,0x35,0xb8,0xa4,0x81,0x55,0xac,0xde,0xb1,0xd0,0x82, +0x65,0x33,0x87,0xeb,0x61,0x03,0x79,0xde,0x60,0xa1,0xcd,0x7d, +0xc3,0xfd,0xa5,0x9c,0x57,0x28,0x0d,0x6d,0xb7,0x6f,0x49,0xd5, +0xd9,0xec,0x17,0xd3,0x49,0x85,0xfe,0xed,0x7c,0x39,0xe5,0x7e, +0x81,0x33,0xec,0x5c,0xd7,0xbd,0x8f,0x61,0x00,0x74,0x5f,0x7b, +0x70,0x6d,0x9b,0x78,0x04,0x8a,0xa4,0x1e,0x72,0x1d,0x4f,0xa9, +0xd1,0xdc,0x96,0xec,0x8e,0x44,0x64,0x34,0x5b,0x5e,0x58,0xe5, +0x5d,0xb0,0xeb,0xca,0xef,0xb1,0x16,0x9f,0x28,0xad,0x3a,0xb6, +0xad,0xc5,0x81,0xfd,0xdd,0x4c,0xe7,0x71,0xfb,0x06,0x9a,0xcd, +0x4c,0xbc,0x85,0xc1,0xa1,0xd5,0xd6,0xaf,0x2e,0xd7,0x28,0x76, +0xc9,0x5f,0x15,0xa5,0xd3,0xe4,0x6b,0x4e,0x38,0xa2,0x7f,0x79, +0xec,0xc5,0xe7,0x27,0x9f,0x1f,0x11,0x65,0xd3,0xaf,0x2b,0x42, +0x1b,0xfe,0xbc,0x7f,0x9a,0x64,0x44,0x6e,0x09,0xfc,0xf3,0x1c, +0xb0,0x97,0xd8,0xbd,0x2d,0x3f,0x9f,0x2f,0x2f,0xde,0x1c,0x75, +0xaa,0x6b,0x9c,0xc5,0x29,0xa2,0x2d,0x13,0xc5,0xde,0xf3,0x25, +0xaa,0x38,0xfb,0x3a,0x70,0xd5,0x44,0x22,0xc8,0x0a,0xce,0xf8, +0x64,0xd2,0x17,0x74,0xb3,0x05,0xc7,0x76,0xdb,0xc8,0x50,0xf9, +0x2b,0xeb,0x3b,0x14,0x42,0xb3,0xb7,0xeb,0x0a,0x55,0x62,0x5f, +0x77,0xe0,0x83,0x54,0x2f,0x78,0x0c,0x3e,0xfd,0x9a,0xff,0x3d, +0x5f,0xce,0xde,0x9f,0x26,0xd4,0x55,0xff,0x62,0xbe,0x24,0x4d, +0xc4,0xc7,0x20,0x92,0x3b,0x07,0x61,0x7b,0x83,0xe6,0x16,0x19, +0xab,0xda,0x1a,0x93,0xa4,0x68,0xb6,0x6d,0x7a,0x3b,0x4f,0xd8, +0xb8,0x36,0x6c,0x8f,0x7c,0xb2,0x76,0x09,0x31,0xa1,0x7e,0x93, +0xaf,0xc7,0x43,0xd4,0xe1,0x30,0x1c,0xe0,0xd1,0xb2,0x07,0x14, +0x79,0x12,0x46,0xa0,0xd5,0x64,0x3d,0x47,0xda,0x02,0xdf,0xf9, +0x0d,0x1c,0xbe,0x7e,0x9e,0xc2,0x28,0xeb,0x74,0x45,0x62,0x38, +0xda,0x83,0x18,0x58,0x0e,0xb3,0x0a,0x0b,0x4c,0xe7,0x06,0xf8, +0xaf,0x77,0x4f,0xe4,0xae,0xa2,0xf5,0xf2,0x16,0x6f,0x8b,0xb2, +0xc0,0x2a,0x05,0x1a,0xb0,0x8a,0xf2,0xca,0xbc,0xc5,0xdb,0x34, +0xdb,0xdc,0x76,0x4b,0xb7,0x60,0x6f,0xca,0x0d,0x33,0x09,0xee, +0x34,0x1b,0xa1,0xae,0x4b,0x00,0x78,0x9b,0xe5,0x4d,0xb0,0x7a, +0xc9,0x38,0xe9,0x35,0xad,0x17,0xde,0x8e,0xaa,0x1c,0x04,0xcc, +0x5e,0x32,0x49,0xcc,0xdb,0x06,0xd8,0xbb,0x6c,0x48,0x17,0x64, +0x00,0x74,0x00,0x82,0x36,0xd3,0x29,0x1a,0x96,0x44,0x99,0x07, +0x08,0x59,0x16,0x51,0x03,0xf1,0xed,0xd8,0x1e,0xf8,0xb0,0x08, +0x81,0x62,0x2f,0xc5,0x9a,0x20,0x39,0xd2,0x98,0xc1,0xb4,0xd2, +0xd1,0x07,0x6f,0x1e,0xea,0x9f,0xd6,0xcf,0xd7,0xd3,0x45,0x8d, +0xab,0xa6,0x4e,0xf1,0xc8,0x0d,0x94,0xb3,0x2e,0xed,0x19,0xc6, +0x22,0x14,0x45,0x4f,0x3c,0x18,0xc2,0x1e,0xb7,0x2e,0xeb,0x1a, +0x92,0x08,0xd9,0xd9,0x9d,0x5e,0xcb,0x74,0xfc,0x9b,0xcc,0xb0, +0x24,0x70,0x59,0x2c,0x5b,0xb4,0xb4,0xd1,0x4c,0x2a,0xf0,0x30, +0x3f,0x24,0x76,0x25,0x79,0xa4,0x99,0xd0,0x8f,0xba,0xa5,0x95, +0x50,0xd7,0xb7,0xdf,0x2c,0x29,0xa0,0x22,0x7b,0x69,0x46,0xc3, +0xe0,0x8b,0x09,0x6d,0xda,0xf6,0x01,0xf2,0x68,0xaf,0x37,0x19, +0x66,0x49,0x4f,0xef,0x54,0x31,0x15,0x48,0x7e,0x94,0x36,0xca, +0xda,0xb3,0x28,0xe9,0x23,0x8c,0x43,0xe6,0x58,0x4b,0x4c,0x8a, +0x96,0x34,0x04,0x2c,0xeb,0xdf,0x2c,0xea,0xab,0xea,0x72,0x23, +0xc0,0xe4,0x9f,0x56,0x99,0xd4,0x0e,0x54,0x33,0x30,0xef,0xc0, +0x3b,0x84,0xcd,0x87,0x66,0xda,0x88,0x31,0x17,0x1d,0xa0,0x2b, +0x20,0x17,0x31,0x12,0x33,0xaf,0x7f,0xfd,0xfb,0x77,0xbf,0xf8, +0x76,0x72,0xdc,0x9e,0x1d,0x37,0x2f,0x4e,0x55,0x68,0xe9,0xba, +0x97,0x52,0xe5,0xcc,0x83,0xbb,0x63,0xaf,0x2a,0x69,0x9d,0xee, +0x19,0x86,0x6f,0x33,0xb6,0x52,0xa0,0x7f,0x5a,0x7f,0x3b,0x45, +0x4d,0xcf,0x7b,0xd9,0xc0,0xa4,0x82,0x37,0x94,0xa2,0x59,0x36, +0x3d,0x3c,0x24,0x01,0x75,0x2f,0xdd,0x7a,0x8d,0x6f,0xb7,0x75, +0x59,0xbe,0x81,0x52,0x5e,0x4a,0x73,0x84,0x5e,0x85,0xc9,0x28, +0xae,0x30,0xea,0xa4,0x56,0xe9,0xd7,0x52,0x01,0xce,0x0c,0xcb, +0x6a,0x76,0x74,0x92,0x5b,0xa3,0x3a,0x4b,0x7a,0x96,0x26,0xda, +0xaa,0x99,0x78,0x3a,0x82,0xcc,0xce,0x00,0x78,0x90,0xd1,0x7e, +0x77,0x74,0x94,0x50,0xb0,0xd1,0x6c,0x2b,0x06,0xd2,0x91,0x16, +0x9a,0x23,0x5a,0x2a,0x89,0x26,0xb5,0x09,0x22,0x2c,0x04,0x54, +0x5a,0x38,0x94,0x00,0xe1,0x93,0x75,0x1b,0x97,0xc7,0x95,0xec, +0x31,0x9e,0xd7,0xd3,0x77,0x4f,0xf8,0x62,0x60,0x03,0x72,0x7e, +0x09,0xc2,0xe8,0x5a,0xb6,0xa4,0x59,0x59,0xae,0xc8,0xba,0xe0, +0x72,0x3e,0xdd,0xc8,0x6c,0x29,0x8b,0xc4,0x6a,0xcf,0xca,0xcd, +0x06,0xe0,0xad,0xfb,0x5e,0xfe,0x76,0x2b,0x17,0x1e,0x50,0x8f, +0x34,0x36,0xd0,0x18,0xfc,0x37,0x86,0xcf,0x49,0x64,0x80,0x95, +0x7a,0xd5,0x28,0x7f,0xc4,0x00,0xe4,0xd8,0x2f,0x59,0xb8,0x94, +0x2b,0x34,0xeb,0xca,0x28,0xb3,0xd0,0x56,0xec,0x66,0x86,0xf9, +0xf6,0x10,0x2e,0x77,0x1b,0x40,0xd4,0x32,0x0a,0xcc,0xcd,0xe4, +0x23,0x1c,0xed,0x77,0xa4,0xf4,0xff,0xa5,0x04,0x6c,0x2e,0x6a, +0xbc,0xe0,0x60,0x0a,0x44,0x95,0x3c,0xa7,0xe8,0xc5,0xf5,0xba, +0x26,0xdb,0x2a,0xb6,0x98,0xda,0x98,0xcb,0x4a,0xec,0x20,0x29, +0x81,0x6e,0x61,0x44,0x08,0xa9,0x8b,0x5a,0xe5,0x91,0xda,0x63, +0x89,0xde,0x94,0xd8,0xc6,0xbf,0xad,0xca,0xf9,0x0c,0xb3,0xc2, +0x84,0xe0,0x8e,0x28,0xcc,0xca,0xe2,0xe4,0x1a,0xc5,0x63,0x84, +0x78,0x1c,0x16,0xe4,0xcb,0xa0,0x49,0x61,0xcb,0x71,0x02,0xeb, +0x16,0x86,0x30,0xeb,0x1e,0xaf,0x44,0x1a,0x19,0x66,0x8e,0xa8, +0x99,0xa4,0x33,0x18,0x66,0x68,0x80,0x93,0x42,0x42,0xdd,0xbf, +0xae,0xae,0x49,0xfd,0x08,0xf2,0x21,0xf6,0x8b,0x9a,0x64,0x45, +0xdf,0x77,0xe5,0x14,0x64,0xc3,0x34,0x79,0xc4,0xc3,0x3f,0x7e, +0x4e,0x97,0x90,0x74,0xbc,0xb8,0xe0,0x7b,0x65,0x05,0x5f,0x11, +0x84,0xba,0x3e,0xfd,0xcd,0x10,0x76,0xbb,0xe1,0x37,0xcb,0x02, +0x50,0x9d,0x2f,0x7b,0xc2,0x65,0xc7,0x37,0xa4,0xe7,0x20,0x1d, +0xbd,0xd9,0x89,0x39,0x8f,0x0e,0xc8,0xc7,0x51,0x16,0xcc,0x56, +0x38,0x6e,0xdc,0xc4,0xef,0xda,0x10,0x5e,0xd3,0x01,0xa8,0xe3, +0x17,0x06,0x5e,0x83,0x53,0xfb,0xb6,0x5c,0x23,0xb3,0x26,0xf8, +0x7a,0xc9,0x51,0xd2,0xf3,0xcb,0x4c,0x80,0x1f,0x07,0xbd,0x61, +0x29,0x01,0x31,0xa0,0x22,0x20,0xea,0xe0,0x9b,0x0a,0xc2,0x5a, +0x0c,0x2a,0x07,0x5f,0xdb,0x6d,0x50,0x41,0x4f,0x31,0x41,0x99, +0x51,0x1a,0x7c,0x76,0xc2,0x59,0x06,0x69,0xbf,0x81,0xc3,0xa0, +0x7c,0xe6,0x08,0x7f,0x1c,0x64,0x4c,0x94,0x0b,0x86,0xcb,0xe7, +0x11,0x63,0x23,0x75,0x34,0xcd,0xeb,0x05,0xb1,0xf4,0x00,0x6f, +0x16,0xf1,0xa6,0x0a,0x57,0x06,0x48,0x60,0xd7,0x2b,0xe3,0x90, +0x47,0x96,0xa9,0x81,0xcc,0xe0,0xb3,0x55,0xde,0xa6,0x33,0xdc, +0x64,0xde,0x86,0xd3,0x85,0x66,0x0b,0x54,0xf0,0x5b,0xda,0x50, +0xc8,0xce,0x92,0x3b,0x2e,0xfc,0x74,0x9b,0x6a,0x5c,0xb3,0x59, +0xe6,0x7a,0x1b,0x0f,0x26,0x44,0x8f,0x08,0x10,0x71,0x26,0x6f, +0x5e,0x33,0x6f,0x8a,0xf1,0x67,0x28,0xba,0x4c,0x8a,0x80,0x06, +0xf0,0xe7,0x84,0x17,0x95,0x8c,0xd6,0x83,0xb7,0xd7,0xab,0x26, +0xa4,0xce,0x86,0xf5,0xc6,0x86,0x46,0x6b,0x32,0xe2,0xb0,0x29, +0x38,0xbe,0x6e,0x17,0xff,0x8d,0xb2,0xb1,0xd9,0xc2,0xeb,0x06, +0x8b,0x10,0xa9,0x49,0xa1,0x09,0x5d,0x88,0x68,0x6e,0x72,0xcf, +0xcb,0x31,0x1d,0xcc,0xb1,0xa3,0x7b,0x20,0xcb,0x4d,0x07,0x87, +0x02,0xef,0xf5,0x8a,0xda,0x7e,0xe0,0x59,0x31,0xe2,0xf4,0x00, +0x52,0x0a,0x0b,0x06,0x2e,0xf7,0x7d,0x40,0x60,0x59,0xa0,0xf4, +0x26,0x0c,0x92,0x01,0x6b,0x95,0xea,0x16,0x20,0x57,0x8c,0xe2, +0x56,0xa8,0xe7,0x49,0xde,0x48,0xe9,0x60,0x69,0xbe,0xac,0x26, +0x34,0x72,0x5b,0xc6,0x23,0x8a,0x15,0x51,0x45,0x75,0x7c,0x6c, +0x4e,0x94,0xb4,0x32,0x6f,0xf5,0x63,0x33,0xd4,0x88,0x30,0x15, +0xc0,0x2c,0x80,0xb5,0xb9,0x5a,0x2f,0x6f,0xeb,0x64,0x92,0x59, +0x3a,0xc1,0x2c,0x4b,0x1f,0x22,0xc5,0xa3,0x51,0xdf,0x9e,0x02, +0xce,0xda,0x8e,0xd7,0xdb,0x5d,0xbd,0x81,0xb3,0x58,0x9e,0x90, +0x5a,0x61,0x5d,0xae,0xd7,0xcb,0x75,0x62,0xe8,0x0f,0x8d,0x69, +0x04,0x59,0x3f,0x2e,0x05,0xe3,0x35,0x5d,0xed,0xac,0x97,0xd7, +0x80,0x1b,0xc1,0xde,0x66,0xe9,0xe6,0x6a,0xb7,0xdb,0x11,0x85, +0xeb,0x67,0xd8,0x41,0x7d,0x73,0x71,0x01,0xc2,0x73,0x62,0xd8, +0x42,0x50,0xe0,0xd9,0x35,0x0c,0x85,0xe0,0x28,0x3d,0x9b,0xae, +0x67,0xc0,0x8b,0x53,0xb6,0x4e,0xd4,0x2b,0x91,0xf2,0xf6,0x88, +0x6d,0x32,0xfb,0x7f,0xff,0xe1,0xfb,0xef,0x36,0x9b,0xd5,0x2f, +0xe5,0xaf,0x37,0xc0,0x4d,0x3d,0xfb,0xc1,0xb8,0xb1,0x87,0x17, +0xa8,0xa8,0x3d,0xd4,0x94,0x94,0x10,0x13,0xbf,0xe4,0x87,0xea, +0x62,0xbd,0xac,0x97,0x97,0x1b,0xea,0xe3,0xf9,0xf3,0x9f,0x93, +0xec,0x40,0xf3,0xdf,0xbe,0xfb,0x71,0x79,0x6b,0x8d,0xf6,0x50, +0x37,0xfc,0x1c,0x76,0x93,0xe5,0x0d,0x0b,0xd5,0x24,0x59,0x02, +0xfb,0xc4,0x42,0x9e,0xd1,0x62,0x66,0x38,0xc5,0x5e,0xb6,0x41, +0x0b,0x71,0xc3,0xcf,0x6f,0xcb,0x72,0x51,0xa7,0xd3,0x45,0x75, +0x4d,0x57,0x65,0xc0,0x39,0x96,0xab,0xda,0x5a,0x13,0x93,0x50, +0x4b,0x49,0x4e,0x84,0xc5,0x4f,0x56,0xdb,0x38,0x65,0xee,0xc5, +0x72,0x8e,0xf2,0x18,0x1e,0x71,0xd2,0x0d,0x36,0xc9,0x64,0xbf, +0x5c,0x89,0x14,0x2f,0xd6,0xc7,0x36,0x0b,0x65,0xa9,0x2c,0xbe, +0xa0,0x71,0x8d,0x1c,0xbc,0xa2,0x21,0x22,0xd5,0x92,0xa2,0x53, +0x67,0x73,0xbe,0x70,0x14,0x02,0xa3,0x5a,0xfc,0x7a,0x43,0x7f, +0xa8,0xe5,0x58,0x5f,0x85,0x85,0x61,0x45,0x55,0x20,0x60,0x04, +0xde,0x23,0x40,0x39,0x37,0x73,0x54,0x2b,0x2e,0x57,0xab,0x72, +0x16,0x43,0x6b,0x3b,0x73,0xa7,0x10,0xf5,0xa7,0x81,0x09,0x00, +0xba,0x76,0xd6,0x7a,0x8f,0xe5,0x1b,0xce,0x6f,0xd3,0xf9,0xed, +0xf4,0x7d,0xed,0xcf,0x9a,0x9a,0x39,0x57,0x17,0x6f,0xfa,0x64, +0x75,0x04,0x2c,0x05,0x7e,0x17,0x5e,0x11,0xe4,0xdd,0x0c,0x84, +0xb3,0xfe,0xb4,0x98,0xe7,0x25,0x80,0xe4,0x50,0xd0,0x74,0x03, +0x73,0xf1,0x49,0x06,0xf6,0x97,0xeb,0x69,0xb5,0x00,0x09,0xb3, +0xf0,0x34,0xff,0x16,0xf8,0x3e,0xac,0x99,0x35,0xd5,0xee,0xb9, +0xb4,0xd9,0xcd,0x9a,0x7e,0x1c,0x7b,0x8d,0x03,0x54,0x25,0xb0, +0x3f,0xdb,0xda,0xfd,0x66,0x71,0xb4,0xdb,0x04,0x44,0x5e,0x20, +0x13,0x1a,0x1c,0x63,0xf1,0xbd,0x38,0xeb,0x13,0x25,0x1c,0xbe, +0x8a,0x8b,0x0b,0xeb,0x54,0xaf,0x6f,0x80,0x06,0xb9,0x1b,0x4f, +0x8a,0x65,0x0c,0x03,0x91,0x6f,0xaa,0xcb,0xf7,0x7f,0xab,0x36, +0x7c,0x57,0x6d,0xc6,0x1e,0x4d,0x70,0x1d,0x87,0x10,0xa0,0xc0, +0xc1,0xa9,0xa4,0x5a,0xef,0x25,0x9b,0x9b,0xa7,0xb6,0x51,0x20, +0x84,0xe5,0xfc,0x6d,0xd9,0xd6,0xea,0x84,0x0f,0x24,0x0e,0xa1, +0x85,0xad,0x05,0x64,0x75,0x5d,0xc1,0xce,0x77,0x87,0x55,0x72, +0x4b,0x69,0x75,0xde,0x30,0x24,0x72,0x04,0x98,0x21,0x05,0xc6, +0x25,0x40,0x1c,0xbf,0x23,0x9b,0xb9,0xe9,0xfc,0xc9,0xb4,0x46, +0xd0,0x80,0x41,0x58,0x52,0xb5,0x27,0xc1,0x9f,0x6d,0x2b,0xb9, +0x4f,0xd1,0xe1,0x39,0x31,0xd7,0xf3,0xa3,0x9d,0xf7,0xbc,0x95, +0x6a,0x74,0x4e,0x73,0x7b,0x2e,0x92,0x04,0xc3,0x93,0x91,0xc3, +0xa1,0xd6,0xe3,0x20,0x79,0xc8,0x1d,0x00,0x6a,0xb1,0x21,0xc3, +0x4c,0x5d,0x06,0x54,0x92,0x11,0xe8,0xe6,0x16,0xc7,0x6b,0xb4, +0x52,0x94,0xde,0x0f,0x46,0xad,0xfc,0x24,0x2a,0x53,0x52,0xa6, +0xa5,0x84,0x06,0x89,0x91,0x02,0x81,0x7e,0x67,0x0c,0xfb,0x8e, +0x56,0xb4,0x83,0xf8,0xd5,0x72,0xb3,0x7c,0xa2,0x00,0x47,0x04, +0x2b,0x79,0xa3,0x3d,0x84,0x9b,0x9f,0x0c,0x1b,0x6b,0x93,0x1c, +0x16,0x68,0x79,0x4a,0x3a,0x6c,0xac,0x9f,0x4e,0xe0,0x03,0x3b, +0x20,0x05,0xe1,0x63,0xa8,0xd1,0x48,0x61,0x10,0x3a,0xbd,0xe2, +0xb8,0xe1,0x1c,0x2e,0x4d,0x9e,0x2e,0xbb,0x5d,0xc6,0x04,0xea, +0x2d,0x53,0xfa,0xa6,0xc1,0xe0,0x2f,0x11,0x11,0x79,0x3f,0x38, +0x34,0x55,0xd9,0x5e,0xce,0xcd,0x3c,0xb5,0x8d,0x79,0xee,0xe1, +0xe4,0x6e,0xdd,0x44,0x3d,0xda,0x7b,0x10,0x6e,0xd2,0xda,0x9f, +0x1f,0xda,0xd6,0x5a,0xcc,0xca,0xe3,0x61,0xe0,0xb2,0xc8,0xba, +0xdd,0xb6,0x64,0xcf,0x66,0xdc,0x66,0xdb,0x26,0x2f,0xdf,0xf5, +0x37,0xb0,0x9a,0xd6,0x69,0xb8,0x76,0x91,0x95,0x1b,0x8f,0x03, +0x60,0xc5,0xdc,0x81,0x05,0x62,0xc5,0x4d,0x99,0x47,0x9d,0x51, +0x22,0x9c,0xd8,0x4d,0x30,0x0f,0xaf,0x50,0x21,0x18,0x83,0xab, +0xe9,0xa8,0xb8,0x5f,0x94,0x71,0x2e,0xa6,0xc5,0xf3,0x74,0xb1, +0x44,0xe3,0xef,0x4d,0x99,0xf5,0x2f,0xa7,0xd5,0x3c,0xae,0x81, +0x69,0x76,0x93,0x8a,0xf2,0x38,0xd5,0xdb,0x46,0x1b,0x24,0x11, +0xd2,0x80,0xbb,0xdd,0xe3,0x1b,0x21,0x9a,0x5f,0xc3,0xcb,0xd5, +0x5c,0x2d,0x97,0x6f,0x6a,0x77,0x5f,0x8f,0xb2,0x36,0x4f,0x91, +0xda,0xb0,0x34,0x3c,0x8c,0xa8,0x5c,0x26,0xf5,0x8b,0x90,0x33, +0xb0,0x5e,0x98,0x0d,0xd8,0xa8,0x19,0xbd,0xda,0x8f,0xf4,0xea, +0x2c,0x11,0xa0,0xbd,0x0f,0xb7,0x42,0xdf,0x20,0xbd,0xb7,0x54, +0x95,0xbc,0x13,0x15,0x53,0xc4,0xfd,0x02,0x9d,0xf7,0xa8,0x14, +0x75,0xc9,0x85,0x8c,0x6c,0xe1,0x7e,0xf5,0x8c,0x47,0xe8,0x5d, +0x28,0x7c,0x87,0xdf,0x02,0x29,0xe5,0x75,0xbb,0x09,0x5b,0x45, +0x26,0x30,0x78,0x4a,0x41,0x94,0x21,0x1c,0xf4,0x21,0x26,0x93, +0x02,0x73,0xd8,0x07,0xb5,0x12,0x62,0x8f,0x8b,0x85,0xc8,0x84, +0x93,0x72,0xcb,0x90,0x04,0x35,0x21,0x06,0xa5,0x84,0x30,0x53, +0xf6,0x81,0x69,0xc1,0xb1,0x14,0xf0,0x6c,0xc4,0xf9,0x4e,0xce, +0x2e,0xe3,0xd4,0x5b,0xaf,0xb4,0x4a,0x89,0x0a,0x88,0xbf,0xf3, +0xfc,0x8b,0x88,0x24,0x88,0x03,0x01,0xfd,0xd9,0xd5,0xf2,0xd6, +0x6c,0x96,0xaf,0x5e,0xcd,0x4b,0xe6,0xcf,0x8c,0x1d,0xb4,0x20, +0x86,0x53,0x3b,0xd3,0x2f,0xf9,0xf9,0xf1,0xcd,0xa0,0x77,0x57, +0x49,0x7b,0x1b,0x9e,0x87,0xd9,0x00,0x95,0xec,0xe0,0xc4,0x12, +0x25,0xf0,0x7b,0x88,0x4d,0x62,0x86,0x6e,0x95,0xa1,0xed,0x8a, +0x3f,0x4f,0x2f,0x29,0x95,0xa6,0x4a,0xee,0x7d,0x2f,0xdf,0xc1, +0x19,0x8f,0x2f,0x6a,0x79,0x5a,0x6e,0x16,0x54,0x06,0xd5,0x3a, +0x61,0x02,0xec,0x1a,0x02,0xb3,0x4e,0x20,0xba,0xf4,0xf4,0x69, +0x10,0x71,0x82,0x2f,0xdd,0x85,0xad,0x6c,0xb7,0xd2,0x48,0x8a, +0x6a,0xce,0x30,0xaf,0xd7,0x23,0x6c,0xb4,0x08,0x91,0x7b,0x92, +0xc3,0xea,0x70,0xfe,0x93,0x61,0xd2,0xb7,0x37,0xc0,0xbe,0x3a, +0x62,0xc7,0x70,0x22,0x10,0xc8,0x89,0x1a,0xbe,0x24,0x30,0xfa, +0x84,0x2f,0x2f,0x13,0x47,0x6d,0x72,0x65,0x6f,0xd7,0x32,0x14, +0x22,0x54,0x2f,0xe1,0x44,0x77,0x39,0x87,0x93,0xc8,0x98,0x6d, +0x63,0xf4,0xdb,0x84,0x9f,0x7f,0x8f,0xbe,0xff,0x31,0x31,0xd0, +0xd6,0xbc,0xfa,0x90,0x69,0x99,0x9a,0xeb,0xb4,0x15,0xba,0xc4, +0xeb,0x6b,0x2c,0x12,0xdf,0x85,0x73,0xc3,0xdf,0xe0,0xcd,0xea, +0x8f,0x65,0x39,0xab,0xbf,0x9f,0xbe,0x87,0xe3,0x14,0x34,0x25, +0x3d,0xa2,0x5e,0xe1,0x43,0xa6,0x50,0x23,0x86,0xf7,0xb7,0xe5, +0xf2,0xba,0x18,0xe4,0x91,0x89,0x16,0xb7,0x73,0x4c,0x77,0xb7, +0x49,0xc6,0x82,0x9d,0x1d,0x1a,0x1a,0x1a,0x05,0x63,0x2d,0x12, +0xa6,0xdc,0x24,0xb6,0x3c,0xae,0xaf,0xd6,0xd5,0xe2,0xcd,0xdf, +0xd6,0xd3,0x15,0xc1,0x5a,0xb3,0xfc,0xd3,0x32,0xd7,0x51,0x7b, +0x41,0x77,0xa8,0x0f,0x88,0x70,0x1d,0x15,0x18,0xc4,0x05,0xfe, +0x11,0x15,0x78,0x30,0xd9,0xf9,0x06,0x56,0x01,0xc3,0x6e,0x61, +0xbc,0xeb,0xcb,0x77,0x1b,0x52,0x3e,0x78,0x96,0x48,0xec,0x9e, +0xd8,0xe4,0x95,0xb2,0xfa,0x0b,0xfe,0x83,0x66,0xc1,0xf4,0x03, +0xe7,0x93,0xd9,0x04,0xb7,0x5f,0x14,0x62,0x67,0x36,0x42,0x5c, +0x95,0x09,0x1c,0xf4,0x81,0x75,0x00,0x6a,0x51,0xef,0x59,0x2d, +0x6e,0xca,0xa1,0xf0,0x01,0x16,0xf9,0x78,0xbf,0x40,0xfd,0x86, +0xc8,0x72,0x9a,0xcb,0x9f,0xac,0x15,0x45,0xe6,0xd3,0x66,0x11, +0x77,0xf9,0x8e,0xdb,0x8e,0x3c,0x30,0x82,0x4c,0x83,0xce,0xd2, +0x3a,0x6d,0x62,0x31,0xff,0x8c,0xac,0xc7,0xd4,0x1e,0x4e,0x53, +0xfa,0x62,0xec,0x26,0x03,0x45,0x03,0xe7,0x30,0xa7,0x50,0x73, +0x38,0x35,0x77,0x93,0x2b,0x6c,0x62,0x57,0x7d,0xec,0x2d,0xcd, +0x48,0x4a,0xe0,0x6d,0xde,0x9b,0xf5,0xa0,0x20,0xa2,0x85,0xd8, +0x46,0x6b,0x51,0x65,0xc4,0x43,0x1d,0xd2,0x3a,0x72,0x8e,0xb6, +0xa3,0xb6,0x02,0x1f,0xdd,0xcd,0x02,0x93,0xd5,0xbb,0x68,0x22, +0x11,0xc7,0xe2,0x89,0x01,0xb3,0x58,0x8e,0xf7,0xc2,0x1f,0xb4, +0xbe,0xc3,0x92,0x3a,0x0d,0x76,0xee,0xe9,0x84,0x40,0x00,0x7b, +0xa2,0x1b,0x1f,0x23,0xd4,0xf6,0x4f,0x90,0xc5,0x3d,0xe5,0x27, +0x99,0xd7,0x6f,0x11,0x66,0xda,0xf9,0x8a,0x20,0x65,0xc9,0xf6, +0xc8,0x9b,0x25,0xbc,0x1a,0x09,0xaa,0x16,0x04,0x09,0x4b,0x7a, +0xd2,0x33,0xba,0x5f,0x53,0x62,0x89,0x86,0xe3,0x5e,0xb6,0xf7, +0xdb,0xb3,0x53,0xc2,0x66,0x80,0x7a,0x85,0x2d,0x4a,0x02,0xda, +0xaa,0x05,0xce,0x21,0xde,0x29,0x48,0xcf,0x62,0xf6,0xf8,0x23, +0xdb,0xaf,0xaf,0x3d,0xa2,0xe2,0x28,0xe7,0x6d,0x96,0xb8,0xa0, +0x80,0x6f,0x55,0x9b,0xc3,0xb5,0x7d,0x03,0xac,0xc5,0xb7,0xef, +0x52,0xf6,0xe3,0x59,0x5c,0xcc,0x6f,0x66,0x25,0x59,0xeb,0x30, +0x29,0xdc,0x5e,0x55,0x17,0x57,0x06,0x1d,0xb2,0xea,0xe2,0x8e, +0x61,0x26,0xeb,0x75,0x52,0x2f,0xcb,0x5c,0xba,0x3a,0x85,0xff, +0x81,0x63,0xb2,0x96,0x77,0xc7,0x41,0xd3,0xd4,0x6a,0x11,0x58, +0x9e,0x51,0x17,0x63,0xb5,0x7f,0xeb,0x51,0x89,0x49,0x21,0xa9, +0xd6,0x58,0x4d,0x92,0x49,0xe9,0x6f,0xaf,0xe6,0x5c,0xc3,0x30, +0x15,0x54,0x03,0x44,0xd1,0xe9,0x45,0xb5,0x79,0xcf,0xf5,0xfb, +0x6c,0x49,0x43,0x17,0x00,0xdc,0x4f,0x64,0x7d,0x46,0x11,0x4b, +0xca,0x36,0xc7,0xc2,0xda,0xcf,0x24,0xeb,0xb2,0xfc,0x2f,0xf1, +0x26,0x37,0xf2,0xcd,0xc2,0xff,0x5a,0x95,0xb7,0xe2,0x54,0xc7, +0x7e,0x24,0xdc,0x40,0xde,0x19,0xec,0x58,0x59,0x34,0x9d,0xbd, +0xff,0xbe,0xaa,0x37,0x66,0xbd,0x5c,0x6e,0xb8,0x17,0xd3,0xb0, +0xf5,0xd7,0xfb,0x58,0x9b,0x60,0xd4,0xe0,0xa6,0x10,0x35,0xa1, +0x35,0xc0,0x01,0x6e,0xcf,0x0a,0x05,0xc9,0xd0,0x6f,0xf3,0x92, +0x1b,0xd7,0x74,0xe9,0xea,0xe5,0x67,0x9a,0xf0,0x99,0xb9,0x98, +0x4f,0xeb,0xfa,0x01,0x79,0xbb,0xa1,0x9b,0x52,0x18,0x15,0x03, +0xa5,0x24,0x4a,0x12,0x35,0x6b,0x91,0x0c,0xfa,0x7f,0xe9,0x0f, +0x12,0x4e,0x64,0x45,0x5d,0x23,0x92,0x06,0x27,0x73,0x11,0xe4, +0xae,0xcd,0x50,0x1b,0x90,0x28,0xc3,0x45,0x65,0x73,0x23,0x9f, +0x52,0xb9,0x00,0x2d,0xfc,0x9f,0x2e,0x1b,0x45,0x24,0x9d,0x0b, +0x6d,0x96,0xcf,0xe8,0x0a,0xbb,0x70,0x63,0xe9,0x6b,0x1a,0x97, +0xb8,0x9a,0xd6,0x3f,0xdd,0x2e,0xfc,0x7c,0x4e,0x11,0x35,0x88, +0x20,0x1f,0xbd,0x72,0x0a,0x7f,0xb8,0xe4,0xa7,0x23,0x9b,0xac, +0x93,0xc2,0x6a,0xf1,0xba,0x30,0xe2,0x6d,0x10,0x2c,0x41,0x3d, +0xe1,0x2d,0x78,0xf1,0xc5,0x65,0xbd,0x09,0xcf,0x04,0xdb,0xab, +0xc5,0xcd,0x75,0x71,0x7f,0xdc,0x3b,0x9e,0x8c,0xd2,0x51,0x7e, +0x36,0xbb,0x77,0xd6,0xdf,0x66,0x67,0xb3,0x1e,0x7c,0x8c,0xcb, +0x27,0x13,0xca,0x80,0xcf,0x6d,0x76,0xbf,0x5f,0x2f,0x6f,0xd6, +0x8a,0x19,0x6b,0xd1,0x51,0xdc,0x3f,0x7b,0xd6,0xbb,0xff,0xca, +0xac,0x09,0xfc,0xfb,0x2f,0xc6,0x67,0xf5,0xd9,0xcd,0xb7,0x4f, +0xbe,0xfd,0xf6,0xec,0xdd,0xc3,0x93,0x49,0x6f,0x1b,0x7d,0x7f, +0x86,0x45,0x7f,0xbd,0x81,0xa3,0x26,0xda,0x8d,0x40,0x79,0xe8, +0x27,0xfd,0x7a,0x7c,0x76,0x7b,0xf6,0xb7,0x49,0xef,0x34,0x1b, +0xbf,0x38,0x9d,0xdc,0xdb,0xfe,0x29,0x85,0x84,0xe3,0xc9,0xbd, +0x2c,0xfb,0xec,0xbe,0x59,0x23,0xa7,0x98,0x97,0xcf,0xa7,0xaf, +0xa0,0xf4,0xd7,0xe9,0xd9,0x6d,0x2f,0x3b,0xab,0xef,0x9d,0xdd, +0x1f,0x9d,0x42,0xd5,0xaf,0xcf,0xee,0x9f,0x0d,0x4e,0xb7,0x54, +0x0e,0xb6,0xde,0x6a,0x86,0x5e,0x0a,0x35,0x81,0x31,0x31,0xf9, +0xdd,0xee,0xac,0x9e,0xdc,0xb3,0x79,0xe4,0x7f,0x0f,0x99,0x50, +0xef,0xc5,0x36,0xdf,0x9a,0x0c,0x07,0x0c,0x4d,0x8d,0x33,0x1a, +0x00,0x95,0x29,0xeb,0x8b,0xe9,0x0a,0x07,0x75,0x86,0x08,0x48, +0xce,0xce,0xce,0xee,0x9f,0x5f,0x2e,0xd6,0x9b,0xc9,0xf6,0x66, +0x7c,0x36,0x9b,0x1e,0x5f,0x3e,0x3c,0xfe,0x76,0x72,0xf7,0xe5, +0x2e,0xb3,0x35,0x36,0xcb,0x37,0x70,0x7e,0x2f,0xee,0x27,0xe3, +0x17,0x58,0x7c,0x7d,0xb6,0x98,0xdc,0x4b,0xb6,0x18,0x37,0x60, +0x4b,0xb1,0x02,0xc8,0x74,0x72,0x7b,0xcc,0xd8,0xed,0xed,0xc5, +0x2e,0x34,0x77,0x2d,0xae,0x10,0x00,0xfe,0xf1,0x75,0x7d,0x0c, +0x60,0xcf,0xa6,0xf5,0xd5,0xc3,0xf9,0xea,0x6a,0x5a,0xdc,0x3f, +0x4e,0xa9,0xff,0xdf,0x26,0x50,0xb4,0x32,0x97,0xf6,0xb0,0xe9, +0x08,0x03,0x4e,0xfc,0x70,0x64,0xd9,0x6c,0xd0,0x11,0x5b,0x48, +0x82,0x3f,0x43,0x87,0x0d,0x9c,0x77,0x3e,0x57,0xcf,0x5c,0x5d, +0xf2,0x9e,0xcb,0xee,0xac,0x13,0x32,0x3a,0x1f,0x92,0xf7,0x1c, +0xb2,0x09,0xd4,0x83,0xc3,0x66,0x01,0xe2,0xe9,0x8c,0xcc,0x8b, +0xc9,0x3b,0x51,0xa2,0x25,0x69,0x5b,0x74,0xd7,0xac,0x95,0x89, +0xc1,0x3c,0xc3,0x1b,0x0e,0xf6,0x90,0x62,0xf3,0x39,0x63,0x7d, +0xf3,0x20,0x37,0x85,0x8d,0xc6,0x70,0x8e,0x7f,0xc0,0xd8,0xdb, +0xff,0x28,0xf5,0x5a,0xb7,0xbe,0x7d,0x9a,0x9b,0x26,0x8f,0x7f, +0xfa,0x41,0x6e,0xa4,0xbf,0x07,0x30,0xcb,0x59,0xe2,0x46,0x89, +0x5a,0x4f,0x23,0x0c,0xa7,0xb5,0x2e,0x0d,0x2c,0x2c,0x9f,0xe5, +0xae,0x3f,0x86,0x92,0x7d,0x09,0x93,0xe5,0x82,0xc0,0xa7,0xeb, +0x1b,0xa0,0xb5,0xc5,0xab,0xd2,0xab,0x69,0xbb,0x89,0xaa,0x44, +0x1d,0xc0,0xd0,0x87,0x76,0xa1,0xaa,0x48,0x67,0xb7,0xcd,0xe2, +0xee,0x35,0x79,0x57,0xe5,0x3e,0x2b,0xc0,0x05,0xcc,0xc6,0x21, +0xcb,0xb5,0xa8,0x5c,0x0d,0xae,0xf0,0x7c,0x2f,0x5b,0xf0,0x97, +0xba,0xb3,0xbc,0x27,0x1d,0x15,0x39,0x66,0x69,0xf9,0x40,0x15, +0xd8,0xe6,0xde,0x6f,0x0b,0xa2,0x48,0xcc,0x0e,0xcd,0xc9,0xd7, +0x38,0xdd,0x9a,0xe1,0x3c,0x83,0xe0,0x48,0x72,0xda,0x96,0x63, +0xbf,0x35,0x32,0x16,0x94,0x8c,0xd2,0x4e,0x8b,0x2f,0x46,0x63, +0x5c,0x29,0xc6,0x0e,0x04,0xbf,0x26,0xb9,0x63,0x16,0x2c,0x9f, +0x5b,0x70,0x4c,0x87,0x80,0xd9,0x6e,0x3b,0xea,0x26,0x2d,0x66, +0x05,0xc0,0x12,0xe5,0xea,0x41,0x3e,0xad,0x63,0x59,0x9f,0x31, +0x3b,0x4a,0x6d,0x86,0x87,0xa4,0x3e,0x7a,0xb1,0xbb,0xe6,0x73, +0x44,0x48,0xdf,0x43,0xbb,0x56,0x8a,0x0b,0x0e,0x15,0x2d,0x78, +0xdf,0x7c,0xa7,0xb6,0x0e,0xef,0x36,0xea,0xe1,0x86,0x66,0x3b, +0x9b,0xe9,0xe2,0x02,0x71,0xc9,0x5d,0xa9,0x17,0x1d,0x1c,0x75, +0x72,0x9d,0xad,0xc0,0x91,0x8c,0x54,0x08,0x7e,0x70,0x10,0xf4, +0xc0,0xb4,0x9d,0xe8,0x0c,0xdb,0xe1,0x3a,0x21,0x40,0x13,0x02, +0x4f,0x25,0x3b,0xfc,0xdc,0x6e,0xd8,0x1d,0x90,0xf3,0x3c,0xc6, +0xca,0xf6,0x17,0x76,0x10,0xd6,0xc6,0xa6,0xaa,0x7f,0x9e,0x4f, +0xab,0x85,0x06,0x96,0x91,0xf1,0xd3,0xb5,0x33,0x15,0xb6,0x56, +0x1e,0x90,0xda,0x54,0x57,0xe2,0x28,0xc6,0x54,0x6e,0x92,0x8d, +0xbc,0x0f,0x6d,0x48,0xf3,0x18,0xd3,0x28,0x12,0x71,0xab,0x26, +0xca,0x1f,0x7a,0x14,0x8a,0x67,0x24,0x32,0x9c,0xb5,0xab,0xd3, +0x39,0x39,0x7e,0xf3,0xfe,0xe9,0x4c,0x06,0xf1,0x60,0xc2,0xb6, +0xba,0xdd,0xae,0x27,0x04,0xa1,0x93,0x21,0x4d,0x10,0xa5,0x55, +0x33,0x38,0x36,0xdb,0xd2,0xaa,0x9f,0xb5,0xe4,0x10,0x4f,0x32, +0xc1,0x28,0xa7,0xb3,0x01,0xe9,0xa0,0x61,0xf6,0x48,0x04,0xdb, +0x79,0xe0,0xf5,0x75,0xee,0x2d,0xa6,0x29,0x55,0x9b,0xb1,0x0b, +0x83,0x95,0xd8,0xf6,0x86,0x54,0x56,0x82,0x9d,0xc6,0x34,0x68, +0x4b,0x3b,0x0b,0x2a,0x87,0xb0,0x64,0x79,0x13,0xfd,0x16,0xf6, +0x91,0x37,0x2a,0xe6,0xbd,0x8e,0xc8,0xdd,0xd2,0xd4,0x1f,0x2d, +0x41,0xb3,0x9a,0xf0,0xf7,0x43,0x58,0x14,0x52,0xb7,0xee,0x85, +0x28,0x2c,0x59,0x4f,0xdf,0x94,0xac,0xd6,0x0c,0x2a,0xe2,0x1e, +0xa0,0x09,0x79,0x92,0x18,0xbd,0xa9,0x30,0x75,0xf5,0x5b,0x99, +0x7b,0xdb,0x82,0x8f,0x62,0x2e,0xb4,0x83,0x03,0x2a,0xb5,0xd8, +0x52,0xcc,0xc9,0x76,0xac,0x06,0xa7,0xae,0x76,0xe8,0xf7,0xe9, +0x0a,0x83,0xd4,0xe3,0x04,0x27,0x52,0x99,0x41,0x0a,0x51,0x69, +0x5f,0x1a,0x86,0x63,0xec,0xc9,0xa9,0x26,0x8e,0xbd,0xae,0x7b, +0x90,0x38,0x21,0xa2,0x1d,0xe3,0xaf,0x9d,0x41,0x81,0x12,0x76, +0xbb,0x8b,0x37,0x79,0x10,0x20,0xc4,0x5e,0xcd,0x6e,0x8a,0xc6, +0xea,0x0e,0x18,0x0b,0xd3,0x6a,0xed,0x87,0x75,0xe9,0xa3,0xcd, +0x00,0xaf,0x3b,0x56,0x28,0x62,0x9a,0x4f,0x10,0x7d,0xcb,0xe6, +0xcb,0xcd,0xce,0xe0,0xdd,0xb7,0xeb,0x1c,0x07,0x7d,0x0e,0xe0, +0x18,0x38,0xc6,0xd4,0xf1,0x69,0x82,0xae,0xc9,0xa9,0xc9,0xb0, +0xd8,0xce,0x10,0x6d,0xb8,0x56,0x2e,0x17,0x71,0x55,0x2a,0x60, +0xef,0x0a,0x45,0x8f,0x7f,0xb9,0x90,0x2b,0x19,0x43,0x18,0xdf, +0x37,0x6b,0x16,0x47,0xa9,0x37,0x3b,0xd3,0xd5,0x6a,0xfe,0x9e, +0x81,0x01,0x18,0x6e,0x38,0x40,0x05,0x40,0x42,0x26,0x50,0xfb, +0x5a,0x2a,0x7f,0x85,0x9d,0x66,0x67,0x40,0x96,0x3e,0x54,0xe4, +0x18,0x6f,0x3a,0xcb,0x5f,0x5d,0x89,0xca,0x46,0x94,0x2c,0xbc, +0xb9,0x34,0xaf,0x8b,0x5e,0xd5,0x4b,0x4f,0x4e,0xab,0x11,0x24, +0xc0,0xf1,0x77,0xd8,0x0e,0xf4,0xeb,0xd3,0x82,0x03,0x3b,0x9e, +0xbe,0x1e,0x11,0x2d,0x8c,0x5f,0x4f,0x26,0xf9,0x78,0x02,0x9d, +0x5c,0x4f,0x57,0x4d,0xd4,0xef,0x1b,0xbc,0x17,0xd3,0x8c,0x86, +0xdd,0x08,0x01,0xa4,0x24,0x2c,0xed,0x34,0x63,0xff,0xec,0x10, +0x41,0x70,0x84,0xde,0x8b,0x68,0x4b,0x38,0xdb,0x6d,0x83,0xd2, +0x38,0x38,0x17,0x11,0x6c,0x6e,0x0f,0x48,0xa6,0x5e,0xae,0x37, +0x30,0x96,0x3e,0xfe,0x35,0x6c,0x81,0x43,0x9f,0xf4,0x6b,0x67, +0xc2,0x33,0x85,0x27,0xa8,0xd8,0x0c,0x13,0x5c,0x24,0xb9,0x74, +0x4d,0x88,0x34,0x3b,0x18,0x60,0xe2,0x62,0xb9,0x7a,0xff,0x94, +0xaf,0x3a,0xe8,0x37,0xdf,0xbb,0xa8,0x9e,0x80,0xe2,0x82,0x88, +0x2d,0x67,0x61,0x49,0x83,0xac,0xf3,0xc8,0xa6,0x6c,0x60,0x6f, +0xe9,0x35,0xcf,0xd9,0x33,0x94,0xab,0x42,0x6c,0x0e,0x92,0xf3, +0xe5,0x12,0xe3,0x9d,0x39,0x59,0x86,0x5b,0x24,0x99,0x14,0x8a, +0x89,0xe1,0x68,0xa3,0x9b,0x81,0x76,0xf3,0x20,0x33,0x0d,0x97, +0x03,0x2e,0x6d,0x95,0x37,0xfe,0x7e,0x47,0x39,0x19,0x46,0x2a, +0xe4,0x16,0x51,0xe9,0x26,0x70,0x16,0x45,0x85,0xcc,0x94,0xd3, +0x69,0xe6,0x8f,0x8f,0x2b,0x77,0xf5,0x48,0xbe,0xed,0x78,0x9d, +0xc4,0x8e,0xef,0x82,0x06,0x0f,0xa4,0x6a,0xc2,0xdb,0xae,0xf5, +0x42,0x90,0x1b,0x75,0x40,0xa5,0x0c,0x43,0xae,0x6b,0x10,0x97, +0x1a,0x44,0x54,0x7d,0x10,0x28,0x9f,0xdc,0xf9,0x57,0xef,0x65, +0xec,0x28,0x3b,0xd0,0xc7,0x9e,0x9d,0x7e,0xf5,0x1e,0xc7,0xe1, +0x4d,0x52,0x11,0x5d,0x4f,0x51,0x91,0x2c,0x1b,0xf9,0x65,0x82, +0x0f,0x98,0x04,0x9e,0x46,0xb4,0xe3,0xf5,0x24,0x0a,0xd9,0x06, +0xd6,0x17,0xd9,0x08,0xfe,0xc1,0x25,0x94,0xb7,0x15,0xf3,0xc1, +0xb1,0x85,0x61,0x52,0xfc,0xc1,0x46,0x61,0x4e,0xc8,0x7a,0x99, +0x29,0x87,0xa1,0xcb,0xf1,0x4f,0xbc,0x9f,0xf9,0xf5,0xb9,0x98, +0x73,0xff,0x27,0xd3,0x61,0x13,0xd9,0x3c,0x2c,0x96,0x70,0xb0, +0xb8,0x84,0xa5,0xe0,0xb1,0x1b,0xb2,0x35,0xd6,0x65,0xa7,0xba, +0x0c,0xab,0xd3,0x87,0x6e,0x6c,0xda,0xcb,0xcf,0x32,0xc3,0x08, +0x0f,0x74,0x20,0x2d,0x65,0x25,0x43,0x54,0x26,0xba,0x75,0x62, +0x6c,0xcf,0x5f,0x88,0x31,0x03,0x3e,0x89,0x01,0xff,0x6d,0x0a, +0x62,0xff,0xc0,0x5c,0x2d,0xe7,0xb3,0x5f,0x42,0x8e,0x8d,0x49, +0x78,0x91,0x32,0x9f,0x8d,0x7c,0x8e,0x8d,0x15,0x7a,0xbd,0x3c, +0x90,0x00,0x3a,0x27,0x4d,0x86,0x7f,0x0b,0xe5,0x48,0x3e,0xc2, +0x1f,0x6c,0xd7,0xd7,0x39,0x3e,0x8e,0x5b,0xca,0x5d,0x1c,0x38, +0xea,0x5e,0x2c,0x68,0x55,0x1c,0x23,0x1f,0xa5,0xa6,0xe9,0x97, +0xdf,0x4a,0x36,0x0c,0x5b,0x40,0xc7,0x04,0xec,0x52,0x8c,0x03, +0x9b,0x5d,0x9e,0x9e,0x00,0x2d,0x5a,0xa5,0x55,0x60,0x54,0x60, +0xe5,0x2c,0x09,0x1b,0x33,0xc9,0x3c,0x76,0x05,0x07,0x99,0x57, +0xaf,0xd0,0x7e,0xd4,0x79,0x59,0x89,0xeb,0x98,0xe4,0xa4,0x09, +0xb5,0x9a,0x64,0xe8,0x9c,0x69,0x3f,0xb2,0x1d,0x22,0x5d,0x97, +0xb5,0xc3,0x0f,0x45,0xcc,0x13,0xa7,0x05,0x2f,0x5e,0x6a,0xc3, +0x87,0x08,0x6b,0xb3,0x68,0x42,0xff,0x2a,0xcd,0x6f,0xb7,0xad, +0x2d,0xd9,0xf0,0xc3,0x6d,0xcd,0x88,0xde,0xae,0xad,0xe2,0x11, +0xf3,0x0a,0x48,0xe9,0x76,0xe1,0x1f,0x0e,0x70,0xcc,0xa4,0x44, +0xf1,0x60,0x61,0xa4,0xeb,0xea,0xa2,0xb5,0x6a,0x07,0xb2,0xa7, +0x3f,0xa6,0x9e,0xcf,0x24,0xb9,0x3e,0xe1,0xdd,0xe3,0xb7,0xc8, +0xe2,0xb5,0x7f,0x04,0x65,0x7f,0xdf,0xd4,0xe3,0x88,0xd5,0x5c, +0x94,0x99,0x37,0x78,0x25,0x7c,0xb7,0xc6,0x96,0xc5,0x8a,0x4e, +0x19,0x36,0x0e,0xb4,0x68,0xbc,0xe3,0x61,0x7b,0x13,0xe7,0xa9, +0x95,0xbb,0x8a,0x38,0x38,0x8f,0x39,0x44,0xf0,0x21,0x2d,0x72, +0xaf,0xfb,0x7d,0xbc,0xb6,0x5b,0x3f,0x16,0xb4,0xc7,0xc9,0xbd, +0xa8,0xd1,0xce,0x78,0x0d,0x2d,0x24,0xa1,0x55,0xac,0xe2,0xed, +0xa4,0xdd,0x6e,0xc7,0xd3,0xec,0x59,0x98,0x4d,0xe2,0x95,0xc1, +0xab,0xbc,0xd6,0x52,0x7e,0x43,0x6e,0x3b,0x35,0x09,0x0c,0x4c, +0x3f,0x7e,0xba,0x4c,0x1c,0x0c,0xb1,0x75,0xaa,0xa8,0x71,0xdf, +0x94,0xef,0x7d,0xc7,0x0b,0x3f,0x10,0x36,0x24,0x85,0x91,0x4c, +0x5a,0x81,0x85,0x52,0x44,0x65,0x5e,0x3c,0xc7,0x08,0x9d,0x7b, +0xe3,0x43,0x36,0x23,0x3b,0x8a,0x79,0xac,0xad,0x7f,0x5d,0xbf, +0xca,0xee,0xc8,0x30,0x97,0x14,0x93,0x4f,0x30,0x97,0x12,0x41, +0xfc,0xd0,0x23,0x6d,0x1e,0xb8,0xb5,0x59,0xcd,0x05,0x06,0xcb, +0xe3,0x38,0x49,0x9e,0x7d,0x3e,0x4c,0x69,0x9b,0xa7,0x50,0xe6, +0x51,0xe4,0xb0,0xb9,0xe1,0xdb,0xa3,0x72,0xea,0x35,0xaa,0x27, +0x73,0xed,0xb0,0x40,0x05,0x51,0x74,0x6a,0xdf,0x6e,0x95,0x5b, +0x90,0x0f,0x1c,0x81,0x3c,0x2b,0xbc,0x23,0x33,0x29,0x23,0x08, +0x02,0x53,0x4b,0xb3,0x1d,0xaf,0x8f,0x6e,0x77,0x6c,0xbd,0x77, +0xb9,0xf2,0x68,0x6c,0x43,0xde,0x04,0x9e,0x9e,0x9c,0x8b,0xe7, +0xee,0x49,0x2e,0x1f,0x4a,0xb2,0xe4,0x67,0x68,0xe3,0xf9,0x91, +0xa3,0x93,0x3d,0xfd,0x6b,0xaf,0xb6,0x7b,0xcb,0xe6,0x34,0xc3, +0xf9,0x7d,0x86,0x51,0x6a,0x26,0x3c,0x01,0x33,0x2f,0xf4,0x4e, +0xa6,0xb3,0x82,0xe1,0x3b,0xc3,0x59,0x89,0xb7,0x39,0x2c,0x61, +0x37,0x33,0xfc,0x60,0x0d,0xc5,0xa8,0x91,0x22,0xc1,0x4c,0x99, +0x55,0x70,0x1c,0x54,0x32,0x72,0x6e,0x0d,0xb9,0x2a,0xf7,0x8c, +0x45,0x10,0xa0,0x6a,0xaa,0x81,0x9c,0xbb,0x5d,0x4f,0xb9,0xeb, +0x02,0xa8,0xba,0x28,0xb0,0x9e,0xea,0xd6,0x24,0xff,0x3d,0xc9, +0xa2,0x1c,0x56,0xd1,0x9a,0x64,0xd2,0xc8,0x61,0x95,0xb0,0x49, +0x30,0x26,0xc7,0x08,0xe9,0xd4,0x4a,0x73,0x89,0x0c,0x9b,0x3d, +0x3f,0x32,0x38,0x0d,0x5a,0x73,0x29,0x22,0xe5,0xe4,0xe9,0x82, +0x5a,0x38,0x22,0x94,0x69,0x31,0xc3,0x0e,0x14,0x8a,0xce,0xbf, +0xc7,0x34,0xce,0x4b,0xea,0xdd,0xf5,0x1c,0x6d,0xf1,0x87,0x9f, +0x42,0xdb,0xc8,0x87,0x04,0xc7,0x8f,0x7f,0xfa,0xe1,0x67,0x32, +0x4c,0x1f,0x91,0x99,0x3f,0x82,0x6d,0x93,0x0c,0xb4,0x8d,0x66, +0xf5,0x3c,0x07,0xdf,0xc2,0x59,0x4d,0x38,0x34,0xad,0x30,0x8a, +0x62,0x75,0x1f,0x8a,0x24,0xa8,0xe1,0xc4,0xa2,0x58,0xf9,0x90, +0xb9,0x37,0x34,0x9c,0x64,0xd8,0x68,0x7f,0x5a,0xbf,0x5f,0x5c, +0x14,0x12,0xff,0x96,0x52,0x50,0xad,0x09,0x45,0x52,0x09,0x2a, +0x67,0xb9,0x14,0xb6,0xdb,0x88,0x47,0x0b,0x89,0xdd,0x2e,0xd6, +0x8a,0x1c,0xc0,0x81,0x47,0x62,0x6a,0x6b,0xac,0xaa,0xc0,0xfe, +0xde,0x59,0x7e,0xb4,0xcf,0x03,0xe2,0xda,0x4e,0x03,0x34,0xb9, +0x33,0x18,0xf4,0xdc,0x3f,0x26,0xed,0x8c,0x0b,0x43,0x18,0x4f, +0x0b,0x93,0x59,0x83,0xfc,0x9c,0x78,0x86,0xeb,0x9d,0xd7,0xb6, +0xb7,0x91,0x73,0x5d,0x2d,0x01,0xcd,0x32,0x73,0x95,0x67,0x0e, +0x28,0x77,0x97,0x71,0xa9,0x9d,0xb1,0x2a,0x7b,0x4f,0x69,0x4b, +0x73,0xe3,0x6c,0xe5,0x79,0x07,0x74,0xc1,0x8d,0xe5,0x42,0xc0, +0x24,0xd7,0xf5,0xb1,0x4f,0xbb,0xf6,0x5e,0xc0,0xbb,0x08,0xc8, +0x70,0xc0,0x6c,0xed,0x11,0xea,0x21,0x24,0xe2,0x93,0xef,0x11, +0xee,0x02,0xb8,0x05,0x9f,0xa1,0x17,0x8c,0x84,0xb3,0x08,0x13, +0x63,0x5d,0x03,0xee,0x23,0x91,0xbe,0xc1,0xd9,0xd2,0x55,0xce, +0x3a,0xd5,0x7b,0xc2,0x41,0xe4,0xa1,0x22,0x7e,0xf1,0x01,0x97, +0x03,0xb7,0x40,0xa1,0x5f,0x29,0x53,0x02,0x95,0xca,0x31,0x49, +0x03,0x04,0x15,0xf6,0x78,0xcc,0x1a,0x04,0xf4,0xb7,0xc6,0xdb, +0x5b,0xac,0x6c,0x34,0x68,0x1c,0x30,0x75,0x3a,0x56,0x39,0x17, +0xee,0xca,0x8b,0x6e,0xfc,0x91,0xed,0x14,0xd4,0x0e,0xbb,0x9f, +0xa8,0x2f,0xf9,0x47,0x82,0xa6,0xdb,0x2c,0xb6,0x58,0x19,0xfe, +0xf1,0x3b,0xa1,0x3b,0xd0,0x94,0x07,0xa0,0x0d,0x8d,0x85,0x42, +0x12,0x92,0x70,0x6e,0xaf,0x15,0x55,0x10,0xc1,0xdf,0xdc,0x5a, +0xf2,0xff,0xfc,0x9f,0xff,0xd7,0xff,0xfd,0x7f,0x24,0xd9,0xc8, +0xce,0x64,0x78,0x97,0x48,0x5c,0x9b,0x62,0x1b,0x26,0x49,0x1e, +0xd5,0xa5,0x92,0xbb,0xfc,0x63,0x6a,0x52,0x5e,0x2f,0xf1,0x69, +0x97,0x2e,0x35,0x21,0x05,0x35,0x27,0xa2,0x15,0x74,0x4d,0x81, +0x34,0x6c,0xd8,0x48,0xd6,0xd3,0x9e,0x49,0x02,0xb9,0x8e,0x07, +0xc2,0x2f,0x94,0xa6,0x08,0xe7,0x8e,0x8e,0x84,0x87,0x41,0x46, +0x16,0x05,0x67,0xc3,0xc7,0x28,0x1a,0x5b,0x0f,0x94,0x1b,0x8d, +0xe1,0x9f,0x49,0x8e,0x35,0x9c,0x4a,0x84,0xc7,0x89,0x55,0xa8, +0x25,0x56,0xb2,0x49,0x80,0xdf,0x68,0x71,0x21,0xc4,0x4e,0xb7, +0xc4,0x34,0xbc,0x96,0x7b,0x00,0x77,0x83,0x9c,0xf9,0xaa,0x49, +0x49,0x13,0xf3,0x61,0xa8,0x2f,0xca,0x1f,0x92,0xb2,0x50,0x41, +0x05,0x69,0x76,0xb1,0x14,0xd5,0x08,0xb5,0x53,0x9e,0x43,0x04, +0xe4,0xf4,0xaa,0x2c,0xaf,0x72,0x32,0x5d,0x71,0xca,0x03,0x22, +0x20,0x42,0x09,0xfc,0x03,0x14,0x22,0x26,0x02,0xf6,0x49,0x0a, +0xe1,0xc4,0xc7,0x03,0xc0,0x3c,0xa2,0xc4,0x53,0xf5,0xa1,0xbe, +0xcd,0xd4,0x25,0x88,0x16,0x62,0x5b,0x3e,0x2f,0xf8,0xcb,0x81, +0x41,0x65,0x9c,0xd6,0x8c,0xac,0xc9,0x9b,0x6f,0xa1,0xc8,0xa2, +0x38,0x7d,0x3d,0x7c,0x0d,0x40,0x51,0x1d,0x72,0xdd,0x97,0xe6, +0xc6,0xaf,0x27,0x8e,0xe0,0x87,0x36,0xcd,0x3f,0xa3,0x0f,0xdb, +0x6a,0xa1,0xf3,0xbf,0xf5,0x84,0x74,0x70,0x14,0x15,0xab,0x0a, +0x81,0xb1,0x03,0x79,0x45,0xda,0x57,0xc7,0x96,0xaa,0xc5,0x5b, +0x3f,0xb6,0x36,0xce,0x26,0xc5,0x5c,0x0b,0xe3,0xad,0x59,0x0f, +0x0f,0xb6,0x27,0x79,0x5b,0x74,0x3a,0xf0,0x6f,0xa8,0xa2,0xb1, +0xc1,0xb9,0xb5,0x6d,0xee,0x8a,0xd6,0x25,0x1a,0xdf,0xa2,0x6f, +0x1a,0x97,0x01,0x91,0x05,0xb5,0xb8,0x1a,0x88,0x90,0x23,0x18, +0x3a,0xfd,0x6e,0xa4,0x36,0x8c,0x00,0x06,0x06,0xb4,0x87,0x8d, +0xfa,0x80,0xb6,0x32,0x52,0xd6,0x24,0xcb,0x10,0x87,0xfb,0xb8, +0x15,0x0d,0x26,0x64,0x32,0xde,0x48,0xa8,0x7f,0xc3,0x4b,0x4c, +0xc3,0x6e,0x42,0x83,0x63,0x1c,0x11,0xb7,0x20,0xf6,0xb9,0x31, +0xf7,0xe2,0xce,0xff,0x15,0xed,0xfa,0xcb,0x85,0x0d,0x3a,0x84, +0x45,0xc3,0xbc,0x41,0x1e,0x6a,0xf3,0x6f,0xaa,0x59,0x3e,0x40, +0x8b,0xa2,0x77,0x81,0xd2,0xda,0x99,0x45,0x50,0xfc,0x67,0x60, +0xe5,0x5c,0x86,0x04,0xae,0x30,0xe4,0x40,0xcb,0x29,0x01,0xe5, +0xa9,0xcb,0x85,0x0b,0xe4,0xa8,0xa7,0x02,0x68,0xf7,0x72,0x81, +0x72,0x55,0x9b,0x5d,0xfe,0xe5,0x22,0x1b,0xd1,0xce,0x55,0xc4, +0x37,0x0f,0x56,0x81,0x67,0x1e,0x64,0x0c,0x46,0xd1,0x54,0xdb, +0x5e,0x2e,0x64,0x6c,0xf6,0xe8,0xa1,0x1a,0x71,0x35,0x66,0x49, +0xf7,0xb6,0x9b,0x91,0xe4,0x8e,0x0d,0xd3,0x9b,0x31,0x00,0x67, +0xf0,0x37,0x7a,0x50,0x86,0x4b,0x7a,0x21,0x3a,0x77,0x66,0x4a, +0xfe,0x87,0x31,0x25,0xc2,0x70,0x31,0x0c,0x00,0x93,0x1f,0x48, +0xdd,0xd5,0x02,0x43,0x30,0x19,0x32,0xb6,0xfd,0x77,0xe0,0x8a, +0xeb,0xe9,0xad,0xf8,0xa0,0xec,0x21,0xcd,0xf3,0x9b,0xf9,0x9b, +0x82,0xb7,0x82,0x37,0xfc,0xb6,0x8e,0x17,0xea,0xd6,0x3f,0x90, +0xe3,0x19,0x34,0xbb,0xb3,0x5d,0xa0,0x03,0x8a,0x25,0x27,0xcc, +0xd3,0xe8,0x03,0x04,0xa6,0x83,0xae,0x42,0xf8,0x24,0x56,0x58, +0x00,0x95,0xdd,0xa5,0x75,0xa3,0x0d,0x7c,0xdb,0xbd,0x6e,0x5a, +0xe6,0x91,0x69,0x0f,0x75,0x4d,0xd3,0x5b,0x74,0x31,0xa5,0x51, +0x20,0x8d,0x4e,0x6f,0x47,0x30,0xc9,0x4e,0x17,0x5f,0x8b,0xbf, +0x1e,0xd2,0x04,0xa9,0xd4,0xf3,0x94,0x06,0xcc,0x54,0x12,0xee, +0x0f,0x16,0x8f,0x76,0xbe,0xb1,0x28,0xb7,0xe5,0x5b,0x44,0x4a, +0x21,0xb4,0x60,0x06,0x7a,0x6a,0x59,0xae,0x97,0x0b,0xb7,0x94, +0xb0,0x51,0x84,0x8a,0xea,0x70,0xb8,0x3b,0x0f,0x3a,0x5e,0x6c, +0x51,0x79,0x2f,0xf2,0x99,0x45,0x03,0x19,0x8d,0xd5,0x39,0x02, +0x34,0x0a,0xc6,0x97,0xe5,0xe2,0x62,0x66,0x1b,0x39,0xe1,0x46, +0x72,0x45,0x36,0x0a,0x9b,0xb7,0xcd,0x3b,0x88,0x94,0x0e,0x23, +0x68,0xf8,0x81,0x12,0x3d,0x6a,0xff,0x40,0x64,0xdc,0x85,0xa6, +0x1f,0x7a,0x83,0x54,0x34,0x55,0x37,0x56,0xc9,0xc7,0xfe,0x37, +0xf2,0xd1,0xf4,0x4d,0x34,0x1f,0xb4,0x3b,0xd9,0xab,0x7b,0x54, +0xfa,0xd8,0x6b,0x69,0x92,0xed,0xcd,0xf9,0x68,0x2b,0x93,0x66, +0xc5,0x16,0x13,0x13,0x8e,0x4a,0xe0,0x3a,0xdb,0x7c,0xb2,0x95, +0x49,0x54,0x25,0xb6,0x32,0x21,0xc5,0xc5,0x66,0x49,0xd7,0x21, +0x78,0x7a,0xc4,0x9f,0xbc,0x24,0xa5,0x3e,0x45,0xb7,0xb2,0xc7, +0x30,0x67,0xef,0x12,0x1e,0xd0,0x3c,0x57,0x5f,0x68,0xa1,0xdb, +0x85,0x7f,0xa0,0x08,0x1c,0x85,0x96,0x73,0xd8,0xe5,0x3a,0xce, +0xbd,0x42,0xd2,0x28,0x26,0x2d,0x7b,0x82,0x36,0x54,0xc5,0x02, +0x86,0xad,0x0f,0x88,0x29,0x2f,0x37,0x49,0xc3,0xff,0xda,0x57, +0x1d,0x07,0xed,0x9a,0xaf,0xe0,0x6c,0xbd,0xc7,0xa0,0x68,0xb7, +0x4b,0xed,0xf3,0x0d,0x4e,0x5b,0xac,0xd7,0x95,0xac,0xd2,0xf4, +0x2f,0x41,0x93,0x6f,0x58,0x5b,0x74,0xc4,0x31,0x2b,0x8f,0xf8, +0xa0,0x6c,0x35,0x00,0x47,0xb4,0x6f,0x12,0x31,0x1f,0xfd,0x52, +0xbe,0x7a,0xf2,0x6e,0x75,0xc4,0xd2,0x25,0xeb,0xb2,0x12,0xdf, +0xb1,0xde,0x8b,0x77,0xc5,0x87,0x2d,0x4f,0xbb,0x99,0x8c,0x99, +0xf3,0x1d,0xb9,0x00,0x51,0x93,0xb6,0x83,0x55,0xe6,0x59,0xec, +0x14,0xb1,0xc6,0x9a,0xe6,0xd2,0x7f,0x1b,0x0e,0xc3,0xcc,0xcb, +0x60,0x1e,0xc9,0x4e,0x5b,0x7b,0x0b,0x4a,0x7d,0x83,0xf5,0x26, +0xa9,0xb1,0xe5,0x49,0xc6,0xa8,0xf5,0xbd,0x39,0xf5,0xb0,0x8c, +0x1f,0xaa,0x6b,0xba,0x83,0x6a,0xce,0x50,0x02,0xa8,0xa0,0xc9, +0xe3,0x75,0x79,0xbd,0x5c,0xbf,0x47,0xc9,0x0c,0x08,0x9d,0xd3, +0xbe,0xe7,0x5d,0x81,0x3f,0x9e,0x92,0x73,0x0d,0xff,0x7e,0x46, +0x36,0xca,0xf4,0xd8,0xc5,0x18,0xcd,0xfa,0x61,0x18,0x45,0x47, +0xfd,0x38,0x61,0xef,0x2b,0x51,0x8f,0x66,0x42,0x37,0x14,0x3e, +0x74,0x93,0x11,0x0a,0xf5,0x54,0xd8,0x77,0xf0,0xe8,0x93,0xe3, +0x5f,0x70,0xf7,0xc8,0xe7,0xbd,0x4e,0x0b,0xaf,0x53,0xf4,0x00, +0xf6,0x3e,0x8b,0x93,0x00,0xd4,0xc2,0x7b,0x1b,0x43,0x32,0xc8, +0x37,0x12,0x52,0xbb,0x5d,0xbf,0xe0,0xa9,0xd7,0xfc,0xd0,0xfb, +0xcd,0x22,0x39,0x96,0x1f,0x7b,0xa9,0x13,0xd9,0xe9,0x11,0x42, +0x0d,0x7c,0x80,0x9a,0x3f,0x3a,0xc5,0x75,0xbb,0x3a,0x10,0xf4, +0xc6,0xfc,0x89,0x5f,0xbf,0xc9,0xee,0x64,0x90,0xb0,0x88,0x6d, +0x80,0x11,0x82,0x66,0x60,0x18,0x9a,0x94,0xb0,0x36,0xa2,0x7f, +0xfb,0xfa,0x18,0x21,0x39,0xc4,0x70,0x92,0x44,0x7a,0xc9,0x72, +0x6e,0x68,0x24,0xc8,0xce,0xeb,0x72,0x7e,0xd9,0x97,0xb8,0xf5, +0xa9,0x58,0x62,0x5c,0x16,0x77,0xc0,0xbb,0xf2,0xd0,0x9f,0x1b, +0x2b,0xc8,0xc5,0x2d,0x61,0xca,0x7f,0x36,0xa4,0xe3,0xc7,0x33, +0x93,0xd3,0xbb,0xbf,0xc6,0x48,0xfe,0xf2,0x22,0x4a,0x59,0xc9, +0xd6,0xbf,0x40,0x26,0x41,0x00,0x73,0x86,0xc1,0xbd,0x0a,0x26, +0x2b,0x7d,0xf6,0x6f,0x16,0xd5,0xaf,0x28,0x30,0x12,0xd4,0x57, +0xd3,0x9a,0xca,0x6f,0xb7,0xfc,0x3c,0x09,0x4a,0xd9,0xf8,0x0d, +0xa7,0xb8,0x57,0x78,0xfa,0x79,0x65,0xf1,0xe0,0xf4,0x6a,0xf2, +0x96,0xa3,0x80,0x89,0xbe,0x42,0x9e,0x14,0x25,0x53,0x3c,0xda, +0x47,0x02,0xb9,0x92,0x56,0xea,0x93,0x0c,0x1b,0xd8,0x13,0xaa, +0x39,0x3f,0xcb,0x7c,0x63,0x20,0xbc,0x60,0x43,0x05,0x6c,0xcb, +0x4d,0x3d,0xcf,0x5c,0x84,0xa8,0x9b,0xe8,0x91,0x19,0xc1,0x96, +0x0d,0x42,0x4b,0x24,0x26,0x5e,0x0b,0xca,0x58,0xe5,0xcd,0x19, +0x28,0x48,0xd4,0xc0,0x8e,0x6b,0x59,0x76,0x7a,0x3c,0x18,0x66, +0x04,0xbf,0x06,0xd4,0xa0,0x35,0x37,0xd0,0x81,0xda,0x81,0x08, +0x11,0x17,0x94,0x1f,0x52,0xf6,0xf1,0xb1,0xbf,0x74,0xa2,0x22, +0x94,0x76,0x7c,0x9c,0xed,0xd4,0xf8,0x03,0xa6,0xa4,0xd5,0x74, +0xe4,0x52,0x1d,0x44,0x2c,0xac,0x20,0x24,0x11,0x45,0x01,0x8c, +0x79,0x27,0xed,0xe0,0xef,0xed,0xb6,0xe3,0x21,0x1b,0x75,0x4c, +0x28,0x65,0xec,0xc1,0x1b,0x32,0x0a,0xee,0x53,0x88,0x77,0x5f, +0x39,0xe6,0x26,0xb2,0x7c,0x9c,0x15,0x7b,0x50,0xb7,0xc5,0x8c, +0x82,0x40,0xd9,0xa1,0x4d,0xfb,0x9b,0x96,0x96,0xb9,0x51,0xd7, +0x1a,0x37,0x8f,0x01,0xca,0xfc,0xd5,0x24,0x9d,0x60,0x1b,0xad, +0x5d,0x50,0x2b,0x64,0xe7,0x42,0xd7,0x98,0x9e,0xfd,0x88,0x28, +0xf7,0x03,0xcb,0x1d,0x3a,0x5a,0xe0,0x3f,0x14,0x9f,0x8d,0xbe, +0xc6,0x7e,0x41,0xb6,0x58,0x1f,0xb9,0x9f,0x29,0xad,0x84,0x1a, +0xa4,0x63,0xc6,0x2e,0xf1,0x42,0xd8,0xab,0xa9,0x5b,0x8e,0x49, +0x84,0xd4,0xce,0xfc,0x41,0x57,0x0f,0xb0,0x77,0x22,0x65,0xfa, +0xa9,0x23,0xc0,0x94,0x36,0x2c,0xe0,0x68,0x15,0xfa,0xd8,0x80, +0xc7,0xab,0xda,0x36,0xf6,0x0e,0x65,0xec,0xec,0x8b,0xa6,0xd8, +0x56,0xe3,0xd2,0x5d,0x45,0x3d,0x8f,0xa6,0x28,0xca,0x21,0xb1, +0x8e,0x9b,0x15,0xc6,0xce,0x1d,0x8f,0x13,0xb9,0x09,0xc6,0xf0, +0xa3,0xe8,0x54,0x67,0xe2,0x0d,0x11,0xa5,0xa2,0x8b,0xf2,0x88, +0xe7,0x08,0xe3,0x90,0x4b,0x85,0x59,0x32,0x31,0x58,0x9b,0x0e, +0x24,0x26,0x41,0x4f,0xe2,0x8f,0xaa,0xfc,0x9a,0xa2,0xab,0x51, +0x65,0x0e,0xde,0x00,0x95,0xd5,0xad,0xb9,0xad,0x01,0xad,0x3b, +0xa1,0xad,0x6d,0x53,0x16,0x09,0x46,0xf3,0xa4,0xc0,0xbc,0x2a, +0x03,0x4b,0x20,0x98,0x56,0x42,0xdb,0x94,0x70,0x3e,0x23,0xe7, +0xb9,0x96,0x7c,0xeb,0x34,0xcf,0xde,0xd4,0x16,0xfb,0xe2,0x2b, +0x1d,0xcf,0xc6,0xe6,0xca,0x8f,0x7c,0x20,0x01,0xcf,0x7c,0x13, +0x93,0xe8,0x81,0x59,0x2b,0x6b,0x3b,0x8b,0xb9,0xf2,0x96,0x12, +0x43,0x06,0xcf,0x93,0xe1,0x0b,0x40,0x94,0x22,0xa7,0x6f,0x0e, +0xb3,0x42,0x29,0xb8,0xdd,0x39,0x6b,0xe7,0xe0,0xf4,0x4c,0xca, +0x11,0xe0,0x2c,0xf4,0x63,0xa8,0x03,0x1b,0x73,0xb5,0xc1,0x64, +0x12,0x7b,0x86,0x31,0xa4,0x68,0xbe,0xbe,0xc0,0x5a,0xed,0x66, +0x64,0x43,0x2d,0xe5,0x19,0x95,0xd8,0x3e,0x35,0x4f,0xa5,0xc3, +0x6c,0x14,0xa7,0xa8,0x79,0x9b,0x8e,0x5a,0xad,0x0e,0x04,0xbf, +0x5e,0x32,0x3d,0xdb,0xe8,0xbc,0xde,0x6d,0x0e,0x53,0x48,0x96, +0x6b,0xc2,0x98,0xb1,0x81,0x31,0xa9,0x37,0x57,0xc9,0x84,0xe0, +0x65,0xc7,0x2b,0xec,0x6f,0x64,0xeb,0x59,0x00,0x72,0xb6,0x10, +0x5b,0x8c,0xc6,0x0a,0x1c,0xaa,0x29,0x75,0x7c,0xe8,0xec,0x6a, +0x70,0x06,0x51,0xa4,0xdf,0x65,0xae,0xda,0x4e,0x69,0xeb,0x03, +0x66,0x03,0xa3,0x70,0xc9,0xa1,0x22,0x5f,0xb1,0x91,0xcb,0x8f, +0xdd,0xce,0xc5,0x87,0x71,0x4f,0x10,0x4b,0x66,0x7f,0x55,0xad, +0x4a,0x05,0xbf,0x8f,0xf4,0x65,0x3e,0x9a,0x30,0x88,0x3f,0xf3, +0xfc,0x3e,0x90,0xb5,0x21,0x3e,0x34,0x9c,0xf8,0xc5,0x64,0x28, +0x0d,0x3b,0x2a,0xe0,0xfd,0x18,0xf6,0x6f,0xbf,0x7c,0xb7,0xab, +0xa9,0xa1,0x7f,0x29,0x2e,0x36,0xaf,0xd8,0x8e,0x3b,0xaf,0xc7, +0x83,0x17,0xd5,0x04,0xba,0x54,0x4e,0xad,0xc9,0x0f,0x28,0x11, +0xd9,0x75,0x66,0x22,0xfa,0x3b,0x81,0x9e,0xf7,0xaf,0x3e,0x5b, +0x28,0x9e,0x56,0x2d,0x30,0xd2,0xc9,0x68,0x67,0x92,0x7b,0xdb, +0xe1,0xd1,0x2a,0x8b,0xe5,0x70,0x17,0x8c,0x75,0x99,0x67,0xad, +0xc9,0x87,0x0f,0x58,0x07,0xf0,0x2f,0x9f,0xe0,0x35,0xc7,0xb8, +0x22,0xfa,0x6b,0x67,0x6e,0x03,0x3e,0x50,0xdf,0x60,0xa8,0xe7, +0x6a,0x81,0x27,0x66,0x75,0xba,0x24,0x32,0xa6,0x87,0x6b,0x48, +0x73,0x46,0x9f,0x8f,0x78,0xb7,0xa1,0x08,0x70,0xb8,0x0a,0xec, +0x37,0xea,0x7f,0x24,0x8d,0xab,0xec,0x57,0x81,0x59,0x5b,0xb8, +0xa0,0xbc,0xca,0xd8,0x2e,0xf4,0x0f,0x3e,0xd7,0xa8,0x37,0x7d, +0x1e,0x7c,0x6d,0xcb,0xd8,0xcb,0x76,0x2b,0xd9,0x1a,0x11,0x5b, +0xd2,0x45,0x8f,0x70,0xdb,0xfe,0xc8,0xab,0x94,0x37,0x15,0x0b, +0x37,0xab,0x19,0x86,0x96,0x81,0x1e,0x0a,0x8f,0x74,0x2f,0x74, +0xbc,0xfc,0x3a,0x42,0x33,0x2c,0xaf,0x68,0x78,0xb4,0x1c,0x6a, +0xdf,0x69,0xc2,0xf5,0x35,0x85,0x49,0xc3,0x52,0xf1,0x74,0x30, +0xda,0x8f,0xaa,0xdc,0x73,0x26,0x16,0x26,0xe1,0x4d,0xcb,0xa8, +0x2d,0x84,0x50,0x0c,0x62,0x7e,0x7c,0x6c,0xc7,0xbc,0xdd,0xb6, +0x46,0x64,0x89,0xab,0xfc,0x17,0xd8,0x94,0xad,0x0f,0xf2,0xe9, +0x20,0x13,0xd7,0x5a,0xaf,0x67,0xbe,0x2e,0x26,0x31,0x4e,0x64, +0xb5,0x06,0x89,0xb4,0x14,0x89,0x88,0xa6,0x59,0x22,0x56,0xb2, +0x7b,0xf4,0x01,0xa8,0x6b,0x67,0xe0,0x61,0x19,0x9f,0x91,0xb7, +0xe7,0x28,0x43,0x77,0x13,0x0c,0xf3,0x1a,0x93,0x73,0x50,0x39, +0x13,0x5e,0x1f,0x85,0xa7,0xf1,0x78,0x7d,0xd0,0x56,0x63,0xad, +0x84,0xa8,0xcb,0x82,0x19,0x71,0x57,0x01,0x87,0xa7,0xe8,0x30, +0x80,0xa6,0x11,0xc0,0xc9,0x57,0xb6,0x89,0x5b,0x7e,0xc3,0xfa, +0x96,0x93,0x0d,0xba,0x8b,0x4d,0x0d,0x3d,0x75,0x25,0x46,0xf8, +0xe6,0x52,0x4c,0x4d,0xf0,0x58,0xcf,0x0f,0xce,0xfd,0xc8,0x0f, +0xb1,0x3f,0xe3,0x3a,0x18,0x59,0x0c,0x24,0xe7,0xb7,0xf1,0xd3, +0x93,0x6a,0xca,0x92,0x40,0x5e,0xc2,0x8f,0xee,0x56,0x6f,0xfb, +0xb5,0xff,0xdc,0x6e,0x42,0x1a,0x11,0x6c,0x0e,0x64,0xa6,0x0d, +0xbd,0x36,0xf3,0xd6,0x7b,0x6b,0x2c,0x39,0x3a,0xfa,0x7a,0x5e, +0x2d,0xde,0xdc,0x3f,0xfd,0x7a,0x83,0x8c,0xf9,0xf4,0xeb,0xfb, +0xf2,0x77,0x7a,0x74,0xb5,0x2e,0x2f,0x8b,0xcf,0xef,0x4f,0x3f, +0x3f,0x9d,0x7e,0x7d,0x7f,0x7a,0xfa,0x35,0x81,0xcc,0x07,0xc8, +0xcf,0xe9,0xcd,0xa3,0xf3,0xe5,0x3b,0x8c,0x49,0x8f,0x23,0x2a, +0xb0,0xd9,0x76,0x5b,0x82,0x7b,0xd0,0xe9,0xf4,0x40,0xfe,0x34, +0xc1,0xb7,0x28,0x4d,0x07,0x5a,0x81,0xf3,0xc8,0x14,0xff,0x9b, +0xcf,0xf5,0x48,0x22,0x51,0xfb,0x76,0x43,0xc6,0xd4,0x5e,0x04, +0x70,0x76,0x42,0x21,0x14,0xc4,0x1d,0x22,0x78,0xd7,0x72,0x5f, +0x3d,0x79,0xbd,0x0c,0x5f,0xf5,0xc0,0xd1,0x1d,0x00,0x93,0xdf, +0x26,0x23,0x50,0xa7,0xf2,0x7e,0x8b,0x44,0xe1,0x2f,0x12,0x8c, +0x15,0x35,0x58,0xbd,0x1b,0x52,0x24,0x89,0x1c,0x55,0x6b,0x43, +0xf1,0x41,0xce,0xfb,0x5f,0x25,0x46,0x09,0xe2,0x0e,0x9f,0x9d, +0xf0,0x26,0x27,0x4f,0x36,0xfc,0x5a,0xc0,0xdb,0xbe,0x9d,0x25, +0x60,0xd6,0x53,0x94,0x54,0xff,0x86,0x3e,0x9e,0x35,0x34,0x52, +0xe6,0x5f,0x14,0x5c,0x86,0xae,0xdc,0x68,0x38,0xee,0x7d,0xf1, +0x0d,0xda,0x7d,0xe6,0x9d,0xfd,0x60,0x53,0x01,0x6b,0xc3,0x61, +0x30,0x2e,0xff,0xb3,0x72,0x5d,0x4d,0xe7,0xe8,0xe9,0xd1,0x39, +0x50,0x11,0xa9,0xc2,0xd5,0xa3,0x01,0xe7,0xf7,0x61,0xa0,0xf7, +0xd9,0x0e,0x68,0x1a,0xbe,0xea,0x9c,0x50,0x01,0xc4,0x23,0x52, +0xcd,0x8f,0xcb,0xf5,0x35,0xf5,0x30,0xcb,0x93,0xfb,0xf4,0x72, +0x5b,0x5c,0x1c,0x4b,0xd1,0x6c,0x31,0x96,0xee,0xbf,0x38,0xe9, +0x7f,0x65,0x5b,0x96,0x28,0x13,0x9c,0x97,0xe1,0x73,0x43,0x64, +0x2d,0x09,0xe0,0x7a,0x98,0xa7,0x24,0x43,0x74,0xf8,0xd3,0x22, +0xc7,0x3b,0x43,0x98,0x1f,0x7e,0x96,0x0e,0x89,0xc0,0xbe,0x2a, +0x07,0xbf,0xed,0xb3,0x73,0xa6,0x5c,0xd0,0x23,0x05,0x38,0xf0, +0x3d,0x04,0x01,0xbc,0xf7,0x1a,0x86,0x2d,0x05,0x8d,0x7b,0xd4, +0x2f,0x4f,0xbe,0xce,0x17,0xd3,0xb7,0xb0,0x40,0xe8,0x0f,0x4d, +0xdc,0x9e,0x36,0x20,0x1f,0x1f,0x6a,0xd0,0xe7,0x1d,0xd1,0xe8, +0xd7,0xbd,0x1b,0x68,0x60,0xd5,0xfc,0x00,0xa9,0xf3,0x3c,0x79, +0xf4,0xec,0xd9,0xe0,0xd1,0xf2,0x7a,0x35,0xdd,0x04,0xda,0xf4, +0x0b,0x4a,0xc2,0x32,0x26,0x78,0x12,0x3c,0xef,0x9c,0x18,0xff, +0xdd,0x46,0xfc,0x6e,0x0f,0x51,0x82,0x36,0xcb,0x71,0x50,0x10, +0xb6,0x63,0xe6,0x37,0x3d,0x7e,0x20,0x3f,0xfb,0x5f,0xc8,0x9f, +0x1f,0x1a,0x69,0xbc,0xf9,0x81,0x89,0xab,0xea,0x5d,0x39,0xe7, +0xd7,0x85,0x96,0x0b,0xf4,0x60,0xe7,0x15,0x62,0x1f,0x0d,0xc4, +0xa7,0x5d,0xc2,0xc7,0x24,0xf5,0x51,0x34,0x29,0x17,0x0c,0x5f, +0x9f,0x17,0x94,0xa5,0xa9,0x5a,0x00,0xbf,0x15,0x98,0xa8,0xc7, +0x36,0x19,0x67,0x4d,0x0b,0x91,0x12,0x5d,0x62,0x82,0x20,0xbd, +0x6e,0x82,0x30,0x9b,0xad,0xaf,0xa7,0x17,0x00,0xaf,0x2c,0xe8, +0x3d,0x73,0x24,0xcb,0x59,0x06,0x15,0xb2,0x4c,0x22,0xa2,0x04, +0x6d,0x25,0x8c,0x0b,0x05,0x83,0x8d,0xd1,0x7b,0x5a,0x5c,0xe3, +0x55,0x4b,0x0d,0x6d,0x8d,0x2f,0x62,0x61,0x71,0xb7,0xb6,0x8e, +0x74,0x05,0x8d,0xe3,0x03,0x05,0x4b,0xaf,0x07,0xfa,0xfe,0xab, +0xd6,0xb4,0xfd,0xc8,0x4d,0x74,0x4b,0x3b,0x82,0x52,0x61,0xeb, +0x6d,0x25,0x16,0x8e,0xeb,0xeb,0x0e,0xf3,0xe1,0xc7,0x8b,0x6d, +0xd1,0x80,0x7f,0x52,0xf3,0x0e,0x58,0xcd,0x0a,0x26,0x5c,0xa7, +0x58,0xca,0xd0,0x37,0x11,0x46,0x61,0x9b,0x8c,0x88,0x22,0xf8, +0x42,0xc7,0x22,0xe6,0x6e,0xf6,0x2d,0x4a,0x98,0x6c,0xef,0xae, +0x05,0x6d,0x15,0xc3,0x14,0x52,0x15,0x80,0x1c,0xf7,0x26,0x31, +0xfe,0x29,0xa4,0xe5,0x8d,0x53,0x24,0x08,0xde,0xfb,0x62,0x10, +0x2a,0xbc,0x34,0xc9,0xec,0xe5,0x27,0xd4,0x3e,0xbf,0xae,0x68, +0x59,0xf0,0x05,0x10,0xfe,0xba,0x04,0x2c,0xd5,0x15,0xac,0x82, +0x93,0x5d,0xd6,0xd8,0x61,0xed,0x7e,0x5d,0x00,0x34,0x49,0xaf, +0x62,0x84,0x0b,0x10,0xe3,0xaa,0x97,0x7c,0x73,0x73,0x0e,0x64, +0x5c,0xc3,0xd9,0xc2,0x16,0xc5,0x5b,0xd6,0x5b,0x79,0x69,0x45, +0xc6,0xc4,0xad,0xd5,0x63,0x5b,0x66,0xa2,0x0f,0xa2,0x92,0x92, +0xda,0xbd,0x48,0xf3,0x56,0x78,0x20,0xea,0x3a,0x5e,0xad,0xf1, +0xa5,0xc3,0x47,0xf3,0x6a,0x55,0xe8,0x7b,0x39,0xf4,0x3a,0x51, +0xcb,0x48,0xdb,0x2b,0x25,0x6e,0xba,0x60,0xe3,0x59,0x13,0xc2, +0xe8,0x65,0xa4,0xb0,0x3d,0xd9,0x81,0xda,0xda,0x10,0xa9,0x27, +0xd6,0x15,0xc8,0x0b,0x7e,0xe5,0xda,0x70,0x50,0x8f,0xc7,0xd5, +0x5b,0xb3,0x99,0xd5,0x08,0xd8,0x2f,0x25,0xe0,0xaf,0xd0,0xb0, +0x1e,0xf9,0xc9,0x90,0x4b,0xc0,0x0f,0x7e,0xff,0x06,0x7e,0x04, +0xaf,0xda,0x40,0xf2,0xbb,0xe3,0x9a,0x18,0x54,0xee,0x01,0x35, +0x3c,0xbe,0x5e,0xfe,0x76,0xbc,0x2f,0xef,0xb6,0x3c,0x7f,0x53, +0x6d,0xf6,0x65,0x27,0x06,0xb7,0xc5,0x36,0x67,0x56,0x7f,0x0b, +0xe4,0xad,0x13,0x23,0x63,0xe3,0x2f,0x0e,0xda,0xcc,0x83,0x3a, +0x2c,0x88,0x19,0x5b,0x2e,0x16,0x14,0xec,0x00,0x29,0x28,0x09, +0xfc,0x95,0xa0,0x2a,0x27,0xc3,0x95,0x72,0xda,0xe9,0x39,0x48, +0x9a,0x40,0x09,0x43,0x14,0x2a,0xd0,0xd2,0xe8,0x72,0x93,0x1f, +0xff,0x05,0xfe,0x07,0xf2,0x05,0x23,0xea,0x58,0xc4,0x0d,0x1e, +0x44,0xb0,0x4c,0x6d,0xc7,0x59,0x28,0xfd,0x54,0x6f,0x1b,0x82, +0x9f,0xca,0x7b,0x9b,0x35,0xfc,0x37,0x43,0xb9,0x6f,0x46,0x3f, +0x36,0xfc,0xeb,0x3e,0xa6,0x8b,0x2c,0x98,0xe0,0xcc,0x1d,0x10, +0x8d,0x36,0x33,0x8c,0x71,0x3d,0xc3,0xcb,0xea,0x78,0xc0,0x1f, +0x33,0xc9,0x14,0x98,0xcb,0x17,0x76,0xe9,0xe5,0x76,0x69,0x2f, +0x78,0xbd,0x2b,0xe8,0xc3,0x3d,0x74,0x48,0xe9,0x83,0x46,0x3a, +0xb7,0x6b,0xf9,0xab,0xec,0x6d,0x1c,0xe5,0xed,0x27,0x6a,0xb6, +0x2e,0xbc,0x5e,0xbb,0xdd,0x7d,0xdd,0x46,0x98,0xe3,0xd5,0xd5, +0x98,0x59,0x4b,0x68,0xee,0x95,0xb0,0x06,0x89,0xfa,0x59,0x4d, +0x0a,0xf5,0x72,0x15,0x6f,0x28,0x55,0x0a,0xc2,0xf0,0x67,0xb8, +0x2e,0x98,0x88,0xbe,0x8c,0x08,0xe3,0xbf,0xee,0xa1,0x25,0xc8, +0x70,0xd8,0xb0,0x5b,0x7e,0xf1,0xa5,0xac,0x6d,0xef,0xdd,0x34, +0x5b,0x6a,0xb6,0x2c,0xeb,0x1f,0x97,0x9b,0xa7,0x1c,0x64,0x87, +0xa5,0x86,0xa7,0x8b,0x6f,0x80,0xea,0x18,0x7f,0xa4,0x33,0x20, +0x22,0xe4,0xda,0xcf,0x97,0x2b,0xbd,0x23,0x07,0x4a,0x41,0xd1, +0x06,0x3a,0x9f,0x3d,0xd3,0x67,0x13,0xa5,0xd5,0x40,0xb2,0x28, +0x92,0xc1,0x7f,0x45,0x59,0x2a,0xdd,0x53,0x0f,0x69,0x97,0x4c, +0x97,0x32,0xf4,0x0d,0xc4,0x37,0x50,0x57,0xcd,0x41,0xa8,0xdc, +0x52,0x24,0x80,0x0c,0x64,0x56,0x1f,0xd5,0x1a,0xa3,0x8f,0xaa, +0x40,0xc3,0xf4,0xe5,0x98,0x15,0xe1,0xe4,0x63,0x4e,0x10,0xb4, +0xe2,0x33,0x57,0x31,0x22,0x8c,0x26,0xa9,0x28,0x07,0x6c,0x54, +0xb9,0x76,0x52,0x59,0x11,0xe7,0x71,0x10,0xa3,0xe4,0xc4,0xa7, +0x3d,0x49,0x23,0x4e,0x10,0x53,0xb9,0x27,0xe1,0x15,0x1d,0xcf, +0xe1,0x68,0x2f,0x62,0x1c,0x97,0xf6,0x90,0xed,0x41,0x84,0x37, +0x13,0x62,0x21,0x6f,0x01,0xc0,0xa0,0x76,0x6d,0xaf,0x15,0xcb, +0x46,0xfd,0x81,0x25,0xa3,0x78,0xe8,0x71,0x74,0x2c,0xa2,0x6f, +0x9f,0xec,0x95,0xd6,0x59,0xc4,0x1d,0x62,0x67,0xf9,0x20,0x31, +0x87,0x83,0xf3,0x15,0x5f,0xb4,0x90,0xb3,0xeb,0xdb,0xf2,0x06, +0x0e,0xb9,0xd7,0xe0,0x89,0xf0,0x0d,0x5c,0x0f,0xff,0xe5,0x3c, +0xef,0xc8,0x15,0xe0,0xfc,0x2b,0x1f,0xe7,0xb1,0xa4,0x5d,0x7c, +0xd1,0xd9,0xbf,0xa4,0xf6,0xc5,0x14,0x4c,0x69,0x19,0xf9,0xc1, +0x02,0x01,0xe3,0x94,0xc6,0x97,0x99,0x31,0x7f,0x37,0xde,0x5e, +0x04,0xd4,0x8a,0x0c,0xda,0xd1,0x2e,0xbb,0x1b,0x67,0x74,0x30, +0x97,0x83,0xb3,0x15,0x00,0xf1,0x80,0x3c,0x65,0x91,0xad,0xe0, +0xb8,0x1f,0x0c,0xd9,0x2e,0x65,0x63,0x02,0x7a,0x27,0xac,0xa4, +0x60,0x3d,0x67,0x77,0x18,0x44,0xe8,0xd9,0xe4,0xde,0xd9,0x6e, +0x7b,0x36,0xd6,0xdf,0x13,0x0a,0xfa,0x73,0x7d,0x33,0xdf,0x54, +0x8f,0xa7,0xf5,0x15,0x94,0x1c,0x3f,0x3c,0xfe,0xdf,0x31,0x3c, +0xce,0x30,0xba,0x4c,0xba,0x40,0xe3,0x06,0x74,0x1e,0x15,0x41, +0x26,0x4f,0xb8,0x40,0xd2,0x4b,0xfd,0x18,0x2b,0x3d,0xb2,0xb1, +0x5d,0x63,0x89,0x6b,0x10,0xc5,0xac,0xc5,0xf2,0xfd,0xb3,0xc7, +0xf7,0x5f,0x91,0x14,0xbe,0x58,0x62,0x04,0xba,0xfc,0xae,0xbc, +0x3e,0x87,0x13,0x1d,0x08,0x64,0x6c,0xd9,0x91,0x27,0x17,0xf3, +0xba,0x9a,0xe5,0x8f,0x1f,0xfc,0xb7,0x47,0x8f,0xbf,0xf9,0xf3, +0x93,0xe3,0x87,0x4f,0xfe,0xfc,0xf8,0x78,0x30,0xb8,0xb8,0x3c, +0xfe,0xcb,0x9f,0xbf,0xf9,0xb7,0xe3,0x2f,0xbf,0xfc,0xf2,0xab, +0xaf,0xbe,0xf8,0xea,0xcb,0x13,0xf8,0x5f,0x62,0xf0,0xb6,0xa2, +0x44,0xc1,0x8e,0x2e,0x4d,0xa9,0xc1,0xc0,0x62,0x2c,0x7e,0xb7, +0xd8,0x0f,0xf2,0x65,0x23,0xcd,0xa2,0xb5,0x06,0x66,0xc5,0xaf, +0xe5,0x4f,0xf2,0xb6,0x54,0xd3,0xe9,0x70,0x34,0x8c,0x8e,0x38, +0x7b,0x61,0xaf,0x62,0x2a,0xcd,0x4e,0xe5,0xfc,0xfe,0x40,0xd3, +0x6d,0xc0,0x04,0x2e,0x40,0x15,0x48,0x2e,0x6b,0x79,0x30,0x24, +0x2e,0xa3,0xb7,0xdd,0xcd,0x01,0x85,0xfe,0x07,0xda,0xc6,0x2f, +0x51,0x44,0x3f,0x2a,0xb4,0x33,0x2f,0xff,0x28,0x20,0x86,0x5c, +0x5b,0x5f,0xfe,0x71,0x60,0xb8,0x21,0x34,0x3c,0x5c,0x6d,0xda, +0x66,0x49,0x03,0x89,0xb8,0x30,0xa3,0x83,0x4e,0x23,0xf2,0xe8, +0x5f,0xe2,0x24,0xe7,0x39,0x47,0xbe,0x75,0x44,0x50,0x45,0xe4, +0x93,0x21,0xb3,0xc7,0x99,0xe3,0x03,0x1e,0x1a,0x6a,0x14,0xdd, +0xe1,0xa2,0xdb,0x2d,0xff,0x15,0x47,0x5a,0xaa,0xf8,0xaa,0xa9, +0x84,0xab,0x40,0x4c,0xa2,0x47,0xec,0xb1,0xac,0xa7,0x2f,0xb4, +0xfe,0xfa,0xe9,0x5d,0x38,0x09,0x9e,0x09,0x23,0xdd,0xea,0x61, +0xf0,0x3a,0x79,0x39,0x13,0x49,0x54,0xe2,0x93,0xd8,0xa7,0x5e, +0x68,0x39,0xa3,0x12,0x30,0xf2,0x42,0x24,0x94,0x79,0xb1,0x17, +0x22,0x9f,0x2f,0x1b,0xb5,0xb2,0x35,0x84,0x6b,0xa7,0x25,0xb8, +0x25,0xbb,0xb1,0xe1,0xf0,0x6a,0xf4,0xdf,0x22,0x4b,0x09,0x0e, +0x12,0x48,0x95,0xdd,0x61,0x66,0x58,0x7d,0xcd,0x31,0xf8,0xf4, +0x05,0xf2,0x5e,0x2f,0xa3,0xc8,0xcd,0x1c,0xd9,0xaf,0x9a,0xf4, +0x17,0xfa,0xe8,0xb8,0x86,0x76,0x83,0x0d,0x15,0x3a,0x3a,0xc6, +0xb0,0x9a,0x7b,0x82,0x3c,0x7b,0xef,0x9e,0x7f,0x25,0xcf,0xce, +0x20,0x28,0x11,0x39,0xca,0x23,0xa5,0xd6,0xe7,0x79,0x1f,0xfc, +0x44,0x6d,0x7a,0xb6,0xc2,0x79,0x11,0x53,0xe5,0xd8,0xa5,0x16, +0x70,0xca,0x51,0x4a,0xe8,0x92,0xad,0x11,0x5d,0x93,0xb1,0x48, +0x57,0x0f,0xe4,0xdd,0x69,0x4d,0xb6,0xc4,0x80,0x36,0x0c,0x3a, +0x11,0xda,0xa5,0xaa,0x3f,0x89,0x7b,0xed,0x1f,0xc1,0x1c,0x85, +0x03,0x43,0x5a,0x88,0x27,0x8c,0xcd,0x4b,0xc9,0xd3,0x2f,0x4f, +0x3f,0x1a,0x38,0x6b,0xf0,0xea,0x7c,0xe6,0x68,0x1f,0x60,0x4d, +0x43,0xf3,0xba,0x84,0x73,0xc5,0x71,0xbd,0xb9,0xb8,0x11,0x86, +0x30,0x08,0x48,0x3b,0x0c,0x5e,0x24,0x51,0x0f,0x4d,0xde,0x32, +0xd0,0x35,0xf0,0x2b,0xc5,0x3b,0x0f,0x99,0x07,0xe9,0xe2,0x9c, +0x1f,0x1f,0x15,0xf1,0x9f,0xda,0x1c,0x51,0x04,0xcb,0x22,0x95, +0xe0,0x68,0x18,0xaf,0xb8,0x97,0x50,0xa9,0x84,0xe3,0xa7,0xb7, +0x04,0x70,0xe5,0x88,0x90,0xe2,0x04,0xd9,0xf9,0x95,0x43,0x3d, +0x47,0xe1,0x12,0xa8,0xcb,0xd1,0xa1,0x16,0x9a,0x91,0x76,0xd8, +0x2b,0x2f,0xa7,0x4a,0x6c,0xe4,0xc1,0x29,0x46,0xba,0x18,0xbb, +0xe7,0x9e,0xcc,0xac,0xdc,0x37,0xda,0xec,0x6e,0xa3,0xef,0x56, +0xf1,0x80,0x86,0x76,0xe0,0x0a,0xc9,0xaf,0x2e,0x42,0x33,0x8f, +0x85,0xec,0xa2,0xc4,0x88,0x8a,0xbb,0x57,0x43,0xba,0x85,0x7c, +0xeb,0xeb,0x63,0x87,0x03,0x5a,0x73,0x6b,0x0b,0xb2,0xbf,0x6f, +0xd2,0x52,0x19,0xf7,0xbb,0x1b,0xa2,0x36,0x4d,0xcd,0x31,0x80, +0x90,0x2f,0xe9,0xc5,0xcb,0xb8,0x4f,0x0f,0xba,0xe3,0x63,0x0d, +0x5b,0x7d,0x71,0xb3,0x66,0xeb,0x6d,0x0c,0x16,0x7d,0xf9,0x4e, +0xcd,0xcf,0xba,0x5d,0xae,0xab,0xef,0x68,0xf9,0x1d,0xd8,0x38, +0xe7,0xdc,0x02,0xda,0xec,0x19,0xdf,0x88,0x9a,0x00,0xe7,0xe6, +0x01,0xe9,0x1d,0xaf,0xdb,0x6e,0x57,0x82,0xaa,0xb7,0x84,0xb0, +0x36,0x1e,0x16,0x5a,0xa7,0x43,0x3c,0xbf,0x0b,0x7e,0x98,0xcc, +0x15,0x4e,0x22,0x03,0x8e,0x97,0xe1,0xf2,0x6c,0x8b,0x0e,0x8c, +0x8b,0xf0,0x8e,0x4d,0xad,0x3e,0x60,0xfc,0x12,0x5f,0xb0,0x6b, +0x5b,0x71,0x58,0x5e,0x0f,0xa8,0xc4,0x2e,0xa8,0x46,0x29,0x59, +0x72,0x59,0xfb,0xde,0x13,0x51,0x62,0xb4,0xe4,0x6a,0x8a,0x68, +0x58,0x3c,0xd8,0xf7,0x7a,0x24,0xcf,0x1a,0x6f,0x2c,0x1b,0x7d, +0xb6,0xaf,0xc0,0x29,0x35,0x5c,0x15,0xe7,0x3c,0x66,0x2e,0x5f, +0x73,0xd6,0x28,0x20,0x68,0xdd,0xd6,0x08,0xed,0x39,0x35,0xe8, +0xf3,0x46,0xcc,0xce,0x5b,0x19,0xcd,0x9e,0x05,0x42,0xdc,0xc6, +0x8d,0x66,0xd8,0x42,0xf5,0xb6,0x48,0x66,0x02,0x22,0xf4,0xc9, +0x0e,0x36,0x77,0xaa,0x41,0x2f,0xbd,0x45,0x6b,0xc1,0xd5,0xdf, +0x65,0x2d,0x8b,0x9a,0x09,0xe8,0x23,0x78,0xe4,0xbe,0xf6,0xe6, +0xbe,0xf7,0x1b,0x3e,0x21,0x61,0xc2,0x26,0x2b,0xb7,0x4f,0x5e, +0xbe,0x1b,0xd9,0x5f,0xf8,0xe0,0x46,0x89,0xcf,0x15,0x42,0xfe, +0x64,0xbb,0xc5,0x3f,0xb9,0xad,0xed,0xb1,0x16,0x0e,0x26,0x26, +0x5d,0x23,0xa2,0x3c,0xdb,0x23,0xbb,0x94,0xd8,0x04,0x8c,0x6d, +0xc3,0x0b,0xcf,0x4c,0x9c,0x8a,0x60,0x7a,0x36,0x74,0x0b,0xd2, +0xe7,0x1b,0xa4,0x58,0xd4,0xd2,0xd2,0x00,0x92,0xe0,0xce,0x50, +0xce,0x7f,0x7c,0x08,0x57,0x0e,0x2e,0x81,0x96,0xa2,0x3f,0x35, +0x6c,0x6d,0x08,0x70,0x1b,0x3c,0x81,0x1f,0xc4,0xbb,0xc1,0x57, +0x86,0xf8,0xb2,0xb6,0xc5,0x97,0xa1,0x14,0x4d,0x16,0x5b,0x0c, +0x54,0x41,0x78,0x2a,0xb9,0xef,0xf5,0xc7,0x71,0x7c,0x4c,0x0d, +0xca,0x85,0x71,0xe3,0x89,0x15,0xb2,0x21,0x1d,0xeb,0x2f,0x00, +0x91,0x43,0x21,0xed,0x59,0x27,0x18,0x39,0xc4,0x2d,0x13,0x27, +0xaa,0xc5,0x53,0x33,0xac,0x8e,0x8f,0x87,0x19,0x7a,0x47,0x35, +0xd8,0x88,0xc4,0x27,0x32,0x0d,0x86,0x44,0xef,0xcd,0x75,0xbb, +0xe8,0x8c,0x4e,0x4c,0x86,0x14,0x97,0x00,0x79,0xaf,0x67,0x6c, +0x1a,0x31,0x16,0x35,0xad,0xf2,0x1c,0xe4,0x28,0x21,0x95,0x0b, +0xee,0xd0,0x86,0x7f,0x97,0x89,0xe8,0x3c,0xa3,0x7e,0x0c,0xc6, +0x7c,0xa0,0x1f,0x6b,0x92,0x6e,0x8b,0xfb,0xe3,0xb3,0x0d,0x45, +0x61,0xc5,0x90,0xaa,0xdc,0x60,0x71,0xff,0x6c,0x8d,0x5f,0xa4, +0x3f,0x27,0x05,0x0d,0xc5,0x9d,0xa5,0xc3,0xe7,0x96,0x0f,0xa6, +0x5b,0xbc,0x5f,0x9f,0xae,0xcb,0xe9,0x16,0x24,0xc6,0xcd,0x72, +0xb1,0x65,0xc1,0x0b,0xce,0x98,0x15,0x36,0x5c,0x5d,0xbc,0x71, +0xf5,0xa6,0x5b,0x2c,0xc8,0x59,0x12,0x71,0x82,0x33,0xe4,0xc2, +0x60,0xab,0xf7,0x7c,0xdb,0xe9,0xcd,0x66,0x49,0x9d,0xd2,0x2f, +0x7a,0x82,0x9d,0xdc,0xe8,0x29,0x22,0xe3,0x7a,0x39,0xaf,0x79, +0x12,0xb7,0x7a,0xc5,0xb4,0xe5,0x60,0xde,0xdb,0xf9,0x72,0xb9, +0xda,0xd2,0xd1,0x16,0xce,0x8a,0xdb,0xe5,0x0a,0x92,0xd0,0xa3, +0x61,0xb9,0x98,0xbf,0x87,0x1f,0xbf,0xde,0xa0,0x05,0xe5,0xb6, +0xbe,0x58,0xe2,0x53,0x3e,0x04,0xc6,0x4d,0x5d,0x3e,0xe6,0xd8, +0xcf,0xed,0x90,0x50,0xa9,0xe8,0xda,0xb7,0x08,0xed,0x04,0xfa, +0x51,0xb6,0x14,0x7f,0x4a,0x27,0xf4,0xa8,0x28,0x61,0x6e,0xd8, +0xe4,0xdd,0x28,0x4f,0x7b,0x31,0xf6,0x50,0x08,0x0e,0x65,0xcc, +0x16,0x69,0x54,0x93,0xa0,0xaa,0x71,0x35,0x5a,0x04,0x40,0x2b, +0xf8,0x3d,0x6c,0xf4,0xf2,0x09,0x92,0x1f,0x89,0xb2,0xd4,0x31, +0x9f,0x38,0x79,0x0d,0xaf,0x7e,0x27,0xd4,0x14,0xd0,0xfc,0xe3, +0xa0,0xfe,0xb9,0xd1,0x8b,0xb3,0xc9,0xf3,0x0e,0x18,0xfc,0x52, +0xcd,0x3b,0x3e,0x37,0xc0,0x89,0x0e,0xdb,0x6e,0x1d,0x15,0xf9, +0xd2,0x50,0x10,0x42,0x8a,0x65,0xe5,0x85,0xe9,0x66,0xb1,0xc4, +0xe5,0xf9,0x8f,0xf5,0xe1,0x91,0x76,0x36,0x7b,0x84,0x2b,0x25, +0x8f,0xcf,0x01,0x74,0x5f,0x82,0x39,0x65,0xcd,0x2e,0xc9,0x20, +0x15,0x61,0x80,0xee,0xdf,0x7e,0x33,0xaf,0xd5,0x09,0x35,0xe0, +0x4e,0x00,0xeb,0x05,0x70,0xf6,0xa6,0x77,0x89,0x78,0x77,0xd2, +0x9f,0xf6,0xd7,0xbe,0x83,0x07,0xef,0xda,0xa6,0xed,0xb5,0x7d, +0x30,0x80,0xc2,0x37,0xf6,0x15,0xea,0xd4,0xf3,0x76,0xe3,0x99, +0x90,0x18,0x94,0x6a,0xb9,0x90,0x61,0x7c,0x7f,0xe8,0x53,0x80, +0xe3,0x47,0x32,0x79,0x54,0x05,0x57,0x06,0x96,0x96,0x1c,0x7a, +0x9e,0x3c,0xf0,0xaa,0x76,0x47,0x5c,0xe0,0x71,0x28,0x26,0xb6, +0xbd,0x2f,0x42,0xdf,0x16,0x82,0x11,0x3a,0x05,0xf5,0xc2,0x34, +0x7c,0x53,0xd3,0x73,0x82,0xa7,0x74,0x83,0x69,0x39,0xfe,0xc3, +0x67,0x57,0xf4,0xaa,0x26,0x74,0x17,0x02,0x2f,0x3b,0x3e,0x67, +0xd0,0xab,0x3b,0x96,0xe2,0xcb,0x92,0x58,0x86,0x1a,0xfc,0x5a, +0x1e,0xd3,0xec,0x15,0x2e,0x6d,0x18,0x76,0x1c,0x84,0x54,0x81, +0xa2,0xad,0x2e,0x0a,0xff,0x62,0x7a,0xc0,0x0b,0x8c,0x78,0x25, +0xb8,0x18,0x27,0xff,0x33,0xa9,0xc4,0x1b,0xce,0xff,0xff,0x08, +0xe5,0x43,0x74,0xa2,0x2f,0xd4,0xee,0x21,0x96,0x53,0xa8,0x86, +0xb4,0x84,0x4f,0x91,0xda,0x0e,0x82,0x22,0xa6,0x8d,0x7e,0x08, +0x13,0xa3,0x98,0x8a,0x00,0x9a,0x90,0x90,0xf8,0xbd,0x90,0x36, +0x42,0x62,0x7b,0xdd,0xbf,0x4e,0xe7,0x9e,0xe7,0x8f,0x4f,0x01, +0xa6,0xaa,0xd1,0x13,0xaf,0x68,0x46,0x6f,0xd2,0x8a,0xd1,0xd1, +0xa6,0x41,0x20,0x6d,0x1a,0x91,0x2a,0xa2,0x0c,0x0f,0xbe,0x06, +0x65,0x54,0x11,0x65,0x38,0x88,0xdd,0xaf,0x5d,0xd6,0x2e,0xed, +0x07,0x21,0xb3,0xf9,0x79,0x07,0xfb,0xde,0xa5,0x6d,0x0f,0x17, +0x0b,0xb9,0x57,0xf9,0x20,0x19,0xcf,0x4c,0x19,0xc3,0x12,0xd8, +0xe2,0x35,0xe3,0xfc,0x00,0xe1,0xb9,0xd9,0x71,0x95,0x28,0x62, +0xc2,0x30,0xe3,0x46,0x19,0xa5,0x23,0x76,0x2e,0xe8,0xa8,0xd3, +0x14,0x8f,0xde,0x2d,0x00,0x82,0x69,0x4c,0x85,0x46,0x89,0x32, +0xd8,0x24,0x4f,0xbc,0x55,0x94,0x4c,0xbc,0xf2,0xe4,0xb2,0xca, +0x2a,0x8e,0x96,0xeb,0x16,0x58,0x2d,0x6e,0x06,0x19,0x13,0x1a, +0x40,0xd8,0x36,0x61,0x8f,0x2c,0x2f,0x9d,0x2e,0x28,0x79,0xf9, +0xd2,0xe6,0xbf,0x7c,0x99,0xc4,0xab,0x34,0xfa,0x2e,0xc2,0xcf, +0xed,0xd6,0x85,0x41,0xc1,0x80,0x23,0x1f,0x6c,0x3e,0xa3,0x45, +0x8d,0xfb,0xa0,0x22,0xa4,0x19,0x43,0xdd,0x7b,0x2c,0xd6,0x76, +0x8b,0x0b,0x45,0xf3,0x69,0xad,0x10,0x03,0xf4,0xd9,0xdf,0x70, +0xee,0x98,0x01,0xae,0x7d,0xe1,0x05,0xc1,0x3b,0x54,0xf8,0x7e, +0xb2,0x24,0x7f,0x78,0xc5,0xdb,0x75,0xec,0x90,0x01,0x6b,0x58, +0x35,0xc8,0x27,0x43,0x1b,0x0d,0xce,0x04,0x01,0x8d,0x3c,0xee, +0x8d,0x11,0x4c,0xf8,0x89,0x32,0xb7,0x66,0x02,0xa5,0xed,0xf0, +0x8e,0x83,0xee,0x04,0xac,0xda,0x46,0x0c,0xb1,0x75,0x5a,0xdc, +0x36,0xc4,0x03,0xbe,0x7d,0xe5,0x49,0xec,0x8a,0x26,0xcd,0x0f, +0x15,0x31,0x3e,0x56,0xa0,0x64,0xe1,0x5a,0x1e,0x35,0x57,0x27, +0xd1,0x2f,0x24,0xa3,0x17,0xa3,0xbc,0xd2,0x47,0x7e,0xcc,0xf0, +0x1b,0x4b,0x17,0x30,0xef,0x8d,0x50,0x24,0x92,0xd7,0x2b,0x1c, +0x51,0x78,0x4f,0xeb,0x65,0xd2,0xad,0xd5,0x9a,0xad,0xf0,0x73, +0x9f,0x66,0xd4,0xf6,0x06,0x8c,0x10,0x9a,0xa3,0x1f,0x3d,0x60, +0x80,0x59,0xa4,0xc1,0x82,0x0c,0x7e,0x3a,0x8f,0xe3,0x48,0xe3, +0x93,0xea,0x56,0xef,0x12,0xe6,0xed,0xd1,0xe7,0xdb,0xd7,0xf6, +0xe0,0x60,0x6b,0x9f,0xda,0x53,0x3d,0x11,0xa4,0xa5,0x6a,0x54, +0x6e,0xd4,0xd4,0xab,0x13,0x84,0xf9,0xe3,0xd5,0xc6,0x2b,0x02, +0x47,0x29,0x3b,0x9c,0x1f,0x06,0xa6,0x1d,0x60,0xe2,0xf9,0x7b, +0x00,0x3e,0x74,0x01,0x61,0x01,0x7e,0x15,0x00,0x8c,0x0e,0x36, +0xf2,0xb2,0x1c,0x64,0x88,0xba,0x5b,0x20,0x0e,0x40,0x46,0xbf, +0x9b,0x9c,0x4a,0x53,0x2f,0x3c,0xb9,0x0d,0x41,0x01,0xf2,0xb1, +0xa0,0x5b,0x23,0x3c,0x16,0xbc,0x93,0x93,0xf8,0x72,0x58,0x02, +0x66,0x86,0x42,0x83,0xb4,0xe8,0x72,0x75,0x2c,0xb9,0x38,0x44, +0xe7,0x77,0x41,0x30,0x70,0xbe,0xd0,0x11,0xa2,0x8d,0x2f,0x0f, +0x18,0x28,0x5d,0x6c,0xf0,0x41,0x2c,0x87,0x1f,0x11,0xbd,0xac, +0x44,0x4b,0xce,0x85,0x72,0x46,0x23,0xf4,0xba,0xd3,0xc0,0xe6, +0xad,0x3d,0x29,0x77,0xb1,0x56,0xa2,0xb8,0x2c,0xd5,0x55,0x9b, +0xda,0xd0,0x40,0xc8,0xec,0xed,0x49,0x49,0x7a,0x94,0x63,0xf7, +0x69,0x34,0x67,0x13,0x6b,0xe3,0x63,0x79,0x50,0xce,0xce,0xe1, +0x76,0x7b,0xc2,0xaf,0x59,0xa9,0x03,0x01,0x14,0x18,0x91,0x76, +0x7e,0x3c,0x31,0xd7,0xd3,0x77,0xf4,0xcd,0x0f,0x5d,0x0d,0xec, +0xa3,0xb9,0x36,0xd0,0x8f,0xd4,0x1d,0x41,0xc1,0xdc,0x16,0x24, +0xc3,0x99,0x77,0x8e,0xbf,0x71,0x2d,0x1b,0x68,0x18,0x03,0x71, +0xa4,0xe2,0xb7,0x6d,0x01,0xed,0x76,0xab,0x4e,0xc1,0x6e,0xa3, +0x40,0x99,0xd1,0x09,0xd2,0xb3,0xb2,0x14,0x1f,0x5f,0x6b,0x67, +0x99,0xb3,0x8b,0x92,0x34,0x1e,0xdd,0x60,0x69,0x21,0xa0,0xa3, +0xed,0x56,0x4a,0xb8,0xf0,0xfe,0xb6,0x0d,0xef,0x12,0x8d,0xe9, +0x36,0x6d,0x94,0x35,0x68,0x74,0x8d,0xf6,0x63,0x2b,0xba,0x3c, +0xb2,0x51,0xb7,0x84,0x5d,0x71,0xf9,0x8c,0xd9,0x0e,0xa2,0x3b, +0xf3,0xaf,0x49,0x86,0x8c,0x59,0xd6,0xab,0xcb,0x45,0x86,0x9f, +0x8f,0xe1,0xd0,0xe3,0x79,0xf7,0xef,0xd0,0x64,0x62,0x1a,0x4a, +0xfb,0x30,0xfc,0x8d,0x1f,0x1f,0x84,0x1f,0x24,0xb0,0x86,0xe2, +0xcd,0xe3,0xa0,0x0b,0xd4,0xef,0x02,0x48,0xaa,0xcf,0x6d,0x20, +0x09,0xf1,0x88,0xc4,0xe7,0x03,0x36,0x94,0x9d,0x7e,0x58,0x49, +0x3d,0x6d,0x12,0x5c,0x71,0x3c,0xd0,0x62,0xb0,0xc0,0x4c,0x78, +0xd0,0x77,0x57,0x5e,0xde,0x10,0x79,0xdf,0x01,0xb9,0x05,0xa3, +0x1d,0xe2,0x46,0xb4,0x40,0x76,0x1f,0x4a,0xc4,0xca,0x9c,0xba, +0x5d,0xb4,0x55,0x58,0xf0,0x7e,0xf0,0x6f,0xee,0xe7,0x03,0xfd, +0x69,0x8f,0x00,0xcc,0x10,0x1a,0x57,0x9b,0x6d,0xf2,0xc8,0xc8, +0x3b,0x51,0x37,0x40,0xcc,0x53,0x86,0x8c,0x4c,0x87,0x16,0x1c, +0x08,0xd6,0x85,0xab,0xc0,0xc0,0x87,0xcb,0x0b,0xb9,0x8d,0xe4, +0x82,0xc0,0xd4,0xe8,0x98,0xde,0x8c,0xdc,0x10,0xee,0x01,0x88, +0x19,0xef,0xfd,0x54,0x8c,0x26,0x23,0x52,0x11,0x9b,0x95,0xd3, +0xc1,0x7f,0xa4,0x4a,0xab,0x5c,0xd5,0x58,0x99,0x8b,0xe7,0xe6, +0x06,0x20,0xdc,0x54,0x01,0x88,0x98,0xac,0x2c,0x93,0x36,0x5e, +0x4b,0x9d,0x64,0xcc,0x5c,0xf7,0xa1,0x6c,0x8f,0xc5,0x8c,0xe5, +0xc5,0xc1,0xba,0xe3,0x06,0x8d,0x63,0xb6,0xb6,0x0a,0xf1,0x5c, +0x5d,0xb2,0xbc,0x45,0xc6,0x60,0xd7,0xfb,0xf6,0x86,0x5a,0xe1, +0x7d,0x6b,0x9f,0xde,0x6d,0xdb,0x1e,0x84,0x1a,0x23,0x70,0x8c, +0x6e,0xc3,0x99,0x9d,0xd2,0xa6,0xa2,0xc7,0x61,0x43,0x6f,0x19, +0xf7,0x68,0x91,0xe2,0x05,0x4a,0x3d,0x20,0xe5,0x58,0xf9,0x1d, +0x27,0xd6,0x93,0xce,0xe5,0xdc,0xda,0x2e,0xa4,0x53,0x18,0x38, +0x2d,0xdf,0xed,0x36,0x4e,0x82,0x7c,0x54,0xb3,0x57,0xd0,0xbe, +0xfc,0xae,0x7d,0x1e,0xd2,0x07,0xb5,0x51,0x54,0x27,0x52,0xdf, +0x75,0xbb,0x9e,0x52,0xd0,0x2f,0xe8,0x1b,0x88,0xb8,0xdb,0xec, +0x44,0x9e,0x8e,0x3b,0xe6,0x28,0x25,0x19,0xbf,0x6d,0x32,0x56, +0x68,0x26,0x20,0x5e,0xe7,0xcd,0x14,0x8f,0xe6,0xbd,0x15,0x86, +0x53,0x42,0xc3,0x75,0x88,0xe6,0x69,0x8b,0x40,0x1c,0x61,0xe9, +0x5c,0x1b,0xcc,0x98,0xab,0xc8,0x3e,0x4d,0xee,0x11,0x77,0x7b, +0x79,0xa8,0x17,0x5c,0xa6,0x69,0xb8,0x0e,0x24,0xc7,0x56,0xed, +0xfa,0x2a,0x67,0x73,0x37,0x60,0x89,0x44,0x8c,0xef,0xa3,0xad, +0xdf,0xdf,0xef,0x8f,0x9a,0xb4,0x27,0x86,0xf3,0x22,0xff,0xbe, +0xc5,0x50,0x6f,0xa9,0xab,0x47,0x62,0x17,0x67,0xee,0x90,0x51, +0xca,0xf4,0xc1,0x80,0xa6,0xe7,0xbc,0x93,0x26,0xf0,0x8b,0x58, +0x6a,0x62,0x54,0xa7,0x9b,0x53,0xe8,0xf1,0x9f,0xe0,0x17,0x7a, +0xa3,0x2f,0xd7,0x20,0xce,0xa2,0xc3,0xc7,0xb7,0xf0,0xcb,0xb0, +0xcd,0x46,0x92,0xfb,0x0e,0x54,0xb0,0x15,0x8b,0x97,0x63,0x02, +0x3f,0xf9,0x52,0x31,0x31,0x17,0xe5,0x7c,0x8e,0x9e,0x3a,0x68, +0xb8,0x96,0xe0,0xc7,0x33,0xfe,0xe0,0x1c,0x35,0x69,0xa3,0x9c, +0x9f,0xe5,0x29,0x43,0xb3,0x5e,0xde,0x42,0x95,0x05,0x00,0xb0, +0xbc,0x85,0xe2,0x0b,0x8c,0x1f,0x34,0xe7,0x14,0xf8,0xc1,0x29, +0x40,0x46,0x18,0xac,0x2e,0x81,0xbf,0x3f,0x4c,0x57,0x89,0xa1, +0xb0,0x41,0x62,0x0c,0x9a,0xd0,0xc7,0x37,0xf4,0x91,0x18,0x31, +0x6d,0x2e,0x67,0x15,0x19,0xec,0xe6,0x6a,0xbb,0xfd,0x44,0x12, +0x92,0x58,0xd7,0xda,0xba,0x71,0xb8,0x43,0x8b,0x6c,0x1e,0xff, +0x8a,0x8d,0xe3,0x77,0x72,0xfb,0x03,0x8b,0x30,0xe0,0xfa,0x58, +0xc0,0xe3,0xfa,0x2e,0xce,0x67,0x83,0x99,0xff,0x51,0x6e,0x48, +0x8b,0x90,0x55,0xbe,0x2c,0x73,0xb6,0x4b,0xe0,0x1f,0xb7,0x39, +0xb8,0xc6,0x78,0x66,0x74,0xf1,0x09,0x85,0xee,0x15,0x93,0xad, +0x74,0x8c,0x82,0x54,0x73,0xbb,0x20,0x63,0xff,0x44,0x29,0x3e, +0xb1,0xf2,0x4c,0x50,0xad,0xdb,0x0d,0x3e,0x3d,0x81,0x9a,0xec, +0x62,0x9e,0x2e,0x36,0x69,0x58,0x80,0xd1,0x32,0x38,0xc9,0x72, +0x77,0x91,0xc3,0x8c,0x2d,0x38,0xa1,0x80,0x5c,0xe8,0x2e,0x6c, +0xda,0xf2,0xc5,0x32,0x0a,0x5d,0xbf,0x46,0x27,0x5e,0x2c,0x3c, +0x3a,0x38,0xe8,0xc6,0x5c,0x34,0x87,0x2e,0xa6,0x63,0xfa,0x36, +0x6d,0xd1,0x2a,0x5e,0xf0,0xdb,0x9e,0x2d,0xba,0x2c,0x2c,0xd6, +0x66,0x93,0xc5,0x95,0x30,0xa4,0x55,0xd5,0xa6,0x02,0xa3,0x77, +0x59,0xbd,0x9b,0x98,0x6e,0xb7,0xc1,0x44,0x39,0x14,0x2a,0xee, +0x66,0x7f,0x8c,0xe3,0xa3,0x5f,0x1b,0xb6,0xd2,0x3e,0x9d,0x54, +0xc6,0x7a,0x7f,0x10,0xb8,0xdd,0x2e,0xff,0xed,0xbf,0xb5,0x41, +0x6d,0x47,0x4d,0xf1,0xc8,0x8f,0x36,0xb8,0x87,0x9f,0x87,0x46, +0x79,0x9e,0x42,0xe7,0xd0,0x86,0x9e,0x1f,0xc2,0x0a,0x2c,0xef, +0x03,0xc8,0x08,0x19,0x7a,0x73,0xe7,0x3c,0xb0,0xea,0xb9,0xeb, +0x4f,0xda,0x43,0x79,0xb1,0xa2,0xd3,0x1c,0xfc,0xd8,0xed,0xcc, +0x61,0xb8,0xd3,0x58,0x8e,0x94,0x7d,0xe5,0x20,0x3d,0xb6,0x0a, +0x6d,0xcd,0x69,0x3b,0xb8,0x0b,0x86,0x2f,0xc8,0x12,0x6b,0x59, +0xe3,0x5b,0x39,0x78,0xf0,0x76,0x6b,0x13,0xbf,0x98,0xef,0x7c, +0xea,0xb4,0x1e,0xee,0x3c,0x6d,0xf4,0xae,0x0f,0x6a,0xb3,0xd8, +0x66,0x05,0x65,0x94,0x2b,0xf9,0x57,0x2b,0xc3,0xc4,0x35,0xdc, +0xc4,0xa9,0x56,0x89,0xf5,0x1e,0x7d,0xbe,0xec,0xfd,0x97,0x20, +0x97,0x90,0x95,0x26,0x15,0xbd,0x91,0xb9,0x20,0xd5,0x24,0x7b, +0xbe,0xb9,0xcf,0x8b,0x25,0x6c,0x94,0xb5,0x26,0x8c,0x92,0x84, +0xc3,0xc5,0x0a,0x42,0x03,0x44,0x67,0xbf,0x07,0xd3,0x9f,0x06, +0xad,0xbc,0x10,0x5c,0xc7,0xe5,0x6c,0x0b,0xc1,0x2b,0x82,0xe2, +0x6f,0xd0,0x38,0x12,0x58,0x30,0x79,0xbe,0x50,0xdf,0xa6,0xaa, +0x1e,0x3b,0x70,0x5d,0xd1,0x7b,0xb8,0xdf,0x28,0x1e,0xe6,0xce, +0x34,0x16,0x41,0x24,0x5a,0xf0,0x8c,0x59,0x4a,0xc0,0x67,0x9c, +0x0e,0xa3,0xa6,0x49,0x34,0x89,0x15,0x0f,0x47,0x20,0xc9,0x06, +0x24,0x14,0x04,0x32,0x19,0xcb,0xab,0xd9,0x46,0x5f,0xcb,0x9e, +0x34,0xc3,0xfd,0xb5,0x9f,0xfd,0xa2,0xf7,0x89,0xda,0x0b,0x99, +0xfd,0x42,0xae,0x98,0x38,0x39,0x38,0xf7,0x9d,0x84,0x12,0xb4, +0x2e,0x70,0xe7,0x20,0xb1,0xea,0xdb,0x51,0x30,0xd1,0x48,0x4a, +0x0e,0xdd,0xa2,0x1d,0xbb,0x91,0xa1,0x92,0x3f,0xb4,0x49,0xea, +0xf5,0x05,0xfc,0xfb,0x3f,0x79,0xdc,0x7b,0x64,0x8b,0xfd,0x07, +0x50,0xf3,0x20,0x1b,0x06,0x1a,0xd9,0xc6,0x41,0x94,0xc6,0x6c, +0xf6,0x0e,0x69,0xff,0x08,0x22,0xf9,0xad,0x85,0x21,0x84,0x01, +0xf4,0x9b,0xb0,0x7d,0x49,0x8a,0xc7,0x06,0xbe,0xc9,0x93,0xa2, +0x8d,0xab,0x53,0xc6,0x07,0xfb,0x09,0x5c,0x55,0xb6,0xdb,0x0f, +0x73,0x82,0x03,0xd5,0x0b,0xab,0xb7,0x6e,0xc4,0x88,0xe8,0x7b, +0x9e,0xea,0x0e,0x58,0x8b,0x94,0x86,0x5e,0x29,0x9c,0xd9,0x66, +0xb9,0xbd,0x93,0xcb,0xea,0x37,0x9e,0x5f,0xa7,0x8a,0xf3,0x1e, +0x4a,0x61,0xff,0x5a,0xfe,0x11,0xa9,0x3c,0x25,0xd1,0x55,0xeb, +0x76,0x1b,0x49,0x61,0x15,0x56,0x58,0xb4,0x4d,0x8b,0xf8,0xd5, +0x87,0x63,0x85,0xbd,0x5e,0x33,0x8a,0x04,0x7e,0x2c,0xe9,0x8c, +0xd4,0xa8,0x2b,0xbe,0xfe,0xee,0x1d,0x0a,0xa6,0x33,0x3e,0x72, +0x9a,0x44,0x43,0x52,0xf8,0xd4,0x66,0xe9,0x2c,0xb8,0x05,0x38, +0x48,0x66,0xf2,0x6e,0x4a,0x93,0xda,0x54,0x8f,0x3e,0x42,0x37, +0xdf,0xdc,0x9d,0x3c,0x03,0xbd,0xf7,0xef,0x07,0xa9,0x75,0x86, +0xc3,0x32,0x1f,0x64,0x59,0x47,0xcd,0x0b,0x97,0x1b,0x15,0xbd, +0xd4,0x6f,0xbf,0x5d,0x43,0xc9,0xaa,0x4e,0x4f,0x43,0x89,0x0a, +0x4a,0x9f,0xa1,0xb1,0x6f,0x10,0x46,0x47,0x78,0x02,0x45,0xeb, +0x03,0x86,0x5c,0x6c,0x18,0xf5,0xa6,0x7c,0xcf,0x8e,0xd7,0xf7, +0x5f,0xc0,0x4f,0x74,0x18,0x5a,0x82,0x5c,0xa8,0x49,0x50,0x97, +0xbe,0xf5,0x85,0x5c,0xd8,0xe8,0x6e,0xb2,0x2d,0x9d,0x1e,0xee, +0x8b,0xb9,0xd8,0x0f,0xcb,0xf5,0xea,0x8a,0x8b,0x8a,0xfb,0x35, +0x1b,0x74,0xd1,0xbf,0xcb,0x9b,0xcd,0xf9,0xfc,0x66,0x4d,0x9e, +0x48,0x48,0x36,0xc8,0x09,0x28,0x58,0x06,0x56,0x18,0xbf,0xe8, +0x4f,0xee,0xd1,0x1b,0xe3,0xfd,0xb4,0xdf,0xcb,0xf0,0x91,0x72, +0xeb,0x99,0x44,0xfd,0xdf,0xf1,0x73,0x24,0xe8,0x99,0x14,0x44, +0xdd,0x2c,0xd5,0x94,0xb6,0x36,0x57,0xd3,0xc5,0x6c,0x5e,0xae, +0xd9,0xb3,0xc5,0x5d,0x4f,0xaa,0x69,0x21,0xb5,0x53,0x9b,0x8d, +0x94,0xfb,0xe9,0xfc,0xf5,0xd3,0x85,0x21,0x19,0x62,0x3a,0xe7, +0xcc,0xef,0x28,0xc3,0xe5,0x6b,0x8b,0x62,0x8a,0x6a,0x01,0xae, +0xcd,0x72,0x5d,0xbd,0xa2,0x58,0x1e,0xd8,0xfd,0x63,0xcf,0x31, +0xc3,0x99,0xfa,0xd9,0xcb,0xa4,0xc7,0x36,0x04,0xab,0x34,0xd7, +0x97,0xbf,0xb0,0x76,0x3d,0x50,0x0a,0x05,0x5f,0xfe,0x16,0x5e, +0x9e,0xd6,0xb0,0xa3,0x0a,0x32,0xdd,0x2b,0xcf,0xda,0x01,0x87, +0x33,0x4f,0xfd,0x4f,0x05,0x90,0x83,0x9b,0x67,0x86,0x5d,0xe0, +0xf9,0x06,0x05,0x21,0x64,0x34,0xd7,0xe8,0xc2,0xd1,0x9e,0x43, +0x2f,0x34,0xa6,0x1e,0x9e,0x5c,0x01,0xee,0xc7,0x56,0x6d,0xcf, +0xf6,0x42,0x7d,0x3b,0x0b,0x34,0x3e,0xb7,0x31,0x68,0xed,0xd7, +0xe4,0xee,0x84,0x21,0x4f,0xa9,0xf3,0x6b,0x74,0xb0,0x28,0x60, +0xbd,0xd3,0x0d,0x8e,0xb7,0xab,0x05,0x25,0xd1,0x6f,0x10,0xf5, +0x8d,0x12,0xf2,0xce,0x03,0xad,0x4f,0x44,0xe3,0x85,0x87,0x33, +0x71,0x26,0x3f,0x06,0xc1,0x74,0xc5,0x1e,0x0a,0xf5,0x61,0xfb, +0x96,0x04,0xd8,0xc5,0x86,0xce,0xa1,0xf6,0x86,0x7b,0xa3,0x56, +0xa0,0x21,0xb5,0xf3,0x63,0x58,0x54,0x72,0xbc,0x99,0x90,0x89, +0x02,0x1b,0x90,0x2a,0x4d,0x61,0xe4,0x7c,0x7c,0xb9,0xda,0x91, +0x5b,0x81,0xa1,0xf6,0xc7,0x0f,0x26,0x0c,0x82,0x84,0x52,0xee, +0xe3,0x4f,0x60,0xb1,0x68,0xa5,0xcf,0x44,0x5c,0x04,0xc3,0x97, +0xc4,0xb1,0x5c,0x54,0xde,0xf1,0x13,0x75,0x85,0xbd,0xb7,0x1f, +0x49,0x01,0x0a,0xc2,0xf1,0x0a,0xa4,0x55,0xec,0x3b,0xd7,0x44, +0xd4,0x46,0x90,0x4a,0x68,0xbb,0x25,0xda,0xff,0xc8,0x2e,0x2c, +0x59,0x46,0xac,0x91,0x55,0x94,0x6c,0xee,0x2b,0xc3,0xcc,0xed, +0x1a,0x22,0x07,0x2a,0x5a,0xb4,0x42,0xb0,0xb9,0x92,0x3b,0xbd, +0x5b,0xe0,0x53,0xb1,0x7b,0x01,0xd9,0xbd,0x7c,0x8e,0xe6,0xd2, +0x12,0xf3,0xc9,0xa6,0x3a,0xa2,0xc1,0xb7,0xd0,0x69,0xd2,0xfa, +0x7e,0x41,0x3e,0xe4,0xba,0x85,0x63,0x71,0x9d,0x3b,0xac,0xf7, +0x5f,0x2f,0xab,0x05,0xe1,0x79,0xe7,0xb3,0x0c,0x58,0x04,0xca, +0x16,0x38,0x36,0x44,0xcd,0xe3,0xcf,0xdc,0x82,0x0b,0x33,0x30, +0x06,0xab,0x66,0x58,0x64,0x3f,0x22,0x3b,0xe7,0x13,0xc5,0x2b, +0xca,0xa9,0x37,0xab,0x6e,0x37,0xf8,0xf4,0xdc,0x23,0x08,0x3b, +0x1e,0xff,0xf1,0xe8,0x35,0x23,0x5d,0x82,0xde,0x43,0xc5,0xd1, +0xd4,0x47,0xad,0xa9,0x6c,0x78,0xed,0xf3,0xbc,0xce,0x80,0x0f, +0xea,0x61,0xec,0x8e,0x38,0x25,0xa5,0x68,0x19,0x71,0xe5,0x2c, +0xb3,0x24,0x88,0x3d,0xa1,0xd7,0xb8,0xfb,0xf2,0x06,0x61,0x91, +0x98,0xb9,0x9f,0xfd,0x56,0x96,0x15,0x67,0x14,0xfe,0x47,0x96, +0x59,0x2a,0x18,0x59,0xbc,0x4a,0xe8,0xde,0x76,0x3c,0xf7,0x7a, +0xe6,0xc4,0xeb,0x3d,0xb7,0xa5,0xe8,0x46,0xd2,0x03,0x2b,0xa0, +0x6e,0xde,0x70,0x64,0x0e,0x3b,0x27,0x64,0xea,0xc5,0x01,0x24, +0x9b,0xc1,0x8a,0x5b,0x76,0x21,0x4b,0xa0,0xd7,0xe8,0x91,0x4e, +0x8b,0x49,0x0c,0xf2,0x5f,0x7b,0x3b,0x0c,0x6e,0x4c,0xb8,0x10, +0x08,0x4e,0xb3,0xd1,0x4d,0x4a,0xb7,0xa5,0x4f,0xdf,0x80,0xc4, +0x4d,0x96,0xb7,0xa0,0xc8,0x82,0xa8,0xb1,0x2d,0xa1,0x36,0xbe, +0x7d,0x1b,0xe0,0x0d,0xeb,0x8f,0xf3,0x3f,0xf4,0x62,0xfc,0x5f, +0xc1,0x02,0xd9,0x05,0x01,0x61,0xfe,0x5f,0xc0,0x0c,0x5b,0x97, +0x39,0x8f,0x02,0xc6,0xca,0x80,0x76,0xbb,0x18,0xb9,0x8f,0x63, +0xe1,0xa7,0x49,0xfa,0x62,0x7b,0x76,0xd6,0xcf,0x58,0xd7,0x15, +0xb0,0x17,0x48,0x06,0xa9,0xa7,0x7f,0x0f,0xfe,0x6e,0x33,0xf4, +0x80,0x4b,0xf1,0xd7,0x67,0x19,0x06,0xda,0x52,0xba,0x28,0x5e, +0x17,0x96,0x64,0x05,0xb9,0xaf,0x11,0xb9,0x8e,0xdb,0x6a,0xf6, +0xf8,0xf5,0xc4,0x74,0x3c,0x82,0xeb,0x76,0x15,0xa5,0xc0,0x25, +0xdc,0xda,0xd2,0xc4,0xed,0xd6,0x8a,0x22,0xfe,0x0a,0x0b,0xca, +0xf2,0xc2,0x24,0xef,0x84,0x0e,0xba,0x22,0x10,0xe7,0x74,0xd9, +0x76,0x40,0xd9,0x76,0xeb,0xb8,0xaf,0xf7,0x40,0xbe,0x2b,0xea, +0xb2,0xd3,0xe4,0xde,0x3d,0xd4,0xe7,0x68,0xca,0x76,0xdb,0x69, +0x16,0xf3,0x19,0xaa,0xae,0xef,0xd7,0x18,0x96,0xbb,0xad,0xc5, +0xf6,0x85,0x7f,0x7c,0x6c,0x19,0x13,0xaf,0x58,0xc7,0x61,0xf9, +0xbb,0x95,0x3b,0x65,0x43,0x8b,0x79,0x18,0x73,0x84,0x79,0x8f, +0xbb,0x6d,0xca,0xe9,0x7a,0xb6,0xbc,0x5d,0xb8,0x46,0x35,0xc5, +0x77,0x6c,0xf3,0x98,0x76,0x24,0x2f,0x09,0xe3,0x0e,0xf4,0xb6, +0xcc,0x66,0x2d,0x3b,0x69,0xd4,0x51,0x83,0xf5,0x60,0xe7,0xd9, +0xd9,0x97,0x85,0xf0,0x9b,0x1e,0x17,0xe2,0x55,0x1c,0xac,0x01, +0x79,0xe1,0xd2,0xb6,0xdd,0xd3,0x05,0xd8,0x64,0x59,0x9d,0x13, +0xef,0xdd,0x61,0xef,0xc5,0x53,0xe1,0x16,0x19,0x3d,0x8c,0xcd, +0x70,0x84,0xf0,0xed,0x75,0x63,0x4b,0xb8,0x26,0x3f,0x18,0x2c, +0x32,0x9c,0xc7,0x3e,0x31,0x93,0x37,0x39,0x2a,0x8d,0xf7,0x7c, +0xdf,0x09,0xde,0xc5,0x40,0x82,0x9b,0x87,0xdd,0x1b,0xb1,0x82, +0xb6,0xd5,0xe7,0x14,0xa1,0xe9,0xb9,0x27,0x9c,0x10,0x33,0xad, +0x98,0x85,0xfe,0x3c,0xdd,0x5c,0x15,0x64,0x8d,0xe5,0x1e,0x27, +0x15,0x26,0xd3,0x78,0xde,0x95,0x7b,0xe7,0xab,0x4a,0x38,0x78, +0xb1,0x98,0x89,0xeb,0x9f,0x33,0x3c,0x26,0xb4,0xaf,0xaa,0x2d, +0x62,0xeb,0x3b,0x5e,0xe7,0x78,0x55,0xce,0x8f,0x4c,0xa1,0x55, +0x31,0xf2,0x09,0x2b,0x67,0x3a,0x08,0xcd,0x17,0x4d,0xaf,0xf8, +0x7f,0x6b,0x26,0x75,0xbc,0xd3,0x16,0xaf,0x47,0x9a,0xcf,0x76, +0x39,0x99,0xec,0x4a,0x21,0xdb,0x19,0x3a,0xf7,0xc9,0xbc,0x59, +0x6e,0xed,0x64,0x64,0x54,0xc2,0x81,0xca,0x98,0xf2,0x98,0x95, +0x3a,0x84,0xfa,0x49,0xc4,0x79,0x33,0x99,0x93,0x22,0xec,0x23, +0x67,0x7b,0xfb,0x48,0x5e,0x60,0x86,0x19,0x47,0x5c,0x60,0x94, +0x61,0xec,0x67,0x3d,0x0b,0xf3,0x2a,0xa0,0x7a,0x0d,0xa7,0xee, +0x52,0x44,0x13,0xfc,0x93,0x71,0xb3,0x40,0xa4,0xcf,0x79,0xb8, +0x78,0x73,0x10,0x4d,0x40,0xd1,0x26,0xd1,0xc5,0x85,0x80,0x5e, +0x8b,0x28,0x69,0xf4,0x47,0xf9,0x37,0xd9,0x79,0x18,0x5d,0x7b, +0xf8,0x2a,0x9f,0xe7,0x77,0x22,0x93,0x24,0xef,0xe3,0xa7,0xfe, +0xa7,0x9c,0x3d,0x6c,0x74,0x00,0x79,0x0f,0x96,0x83,0x8c,0x4d, +0xf2,0x56,0xdf,0x65,0x23,0xb9,0x1f,0x7d,0x16,0xf0,0x57,0x18, +0x70,0x5f,0xcb,0xc0,0x18,0x8d,0xc0,0xca,0xc3,0x04,0x3d,0x43, +0xa9,0x2c,0xca,0xbc,0x2b,0x93,0x37,0xa4,0xbd,0xb6,0x30,0x70, +0xbe,0x54,0x5d,0x2c,0x39,0x88,0x9a,0x8b,0x47,0x60,0x5f,0x8d, +0xa6,0x11,0xf2,0x76,0xed,0x96,0x71,0xd1,0xb6,0x07,0xcb,0x8e, +0xdb,0x20,0x78,0x57,0x8d,0xc8,0x8b,0x1e,0xe0,0x17,0x6b,0x7d, +0xa7,0xe0,0xca,0xd0,0xc0,0x7f,0xd8,0x92,0x6e,0x59,0x04,0x4b, +0x80,0x68,0xa3,0x4f,0x5b,0x37,0x16,0xc7,0xbf,0x45,0x91,0x36, +0x95,0xfa,0x6e,0x99,0x66,0x42,0x7e,0xae,0x05,0xdc,0x18,0xf5, +0x3e,0xa6,0x2a,0x6f,0x69,0xbf,0x94,0x0e,0xff,0x26,0x21,0xe3, +0x38,0x88,0x4f,0xb6,0xa3,0xa0,0x75,0xc5,0xc9,0x90,0xe0,0xb5, +0xcd,0x90,0x41,0x0b,0xbe,0x81,0xad,0x04,0x8d,0x1e,0x50,0x53, +0xc0,0x02,0xb0,0xc7,0x67,0x9b,0x25,0x6e,0xe8,0x69,0x36,0xcc, +0x1c,0x73,0x2a,0xaa,0xd3,0xc1,0xc8,0x61,0xa1,0x21,0xab,0x04, +0xa2,0x4a,0x91,0x06,0x72,0x20,0xb2,0x4f,0xcb,0x90,0x29,0x6a, +0xd0,0xd8,0x35,0x3c,0x89,0x84,0x46,0x2a,0xcc,0xcd,0x24,0xba, +0xf7,0xea,0x7e,0xab,0x6f,0xb1,0xdd,0xb0,0xd2,0x45,0xb3,0x0b, +0x66,0x09,0xdd,0x2e,0x64,0x8c,0xf9,0xf7,0xc4,0xd6,0xf4,0xfc, +0xc3,0x38,0x56,0x08,0xa1,0x3f,0x6c,0x71,0x5f,0xfb,0xf2,0xf4, +0x0b,0x03,0xbb,0x5a,0xd3,0x5f,0xb9,0xb6,0x4c,0x59,0xb0,0x75, +0xf8,0xa1,0xe1,0x77,0xd2,0x90,0xd0,0x15,0xbb,0x52,0xe9,0x67, +0x6e,0x02,0x71,0xeb,0x48,0xfe,0xa5,0x4c,0xa4,0xdb,0xd6,0x35, +0xc5,0x5b,0x05,0x21,0x69,0x78,0x6b,0x02,0x6f,0xae,0x28,0xba, +0xa1,0x75,0x48,0x6e,0xbf,0xcb,0x9b,0x22,0xe6,0x3b,0x4d,0x64, +0xd0,0xd2,0x80,0x1c,0x46,0x1b,0xfc,0xa0,0x6b,0xd1,0xd0,0xda, +0x38,0x5a,0x47,0xd9,0x9d,0x6e,0x27,0x16,0xd9,0x24,0xae,0xa5, +0x7e,0x12,0xc7,0x17,0x32,0x7b,0x74,0x29,0x58,0x84,0x42,0x76, +0xba,0xde,0x52,0xef,0xa1,0xa9,0xdd,0x9e,0x6a,0xde,0x43,0x21, +0xcd,0x0e,0xf1,0x01,0x40,0xb5,0xf3,0xf4,0xb9,0xe0,0x8e,0x1e, +0x14,0x21,0xd5,0x4c,0x24,0x01,0x64,0x77,0xbc,0x4b,0x04,0xbd, +0x5d,0x56,0xef,0x24,0x97,0xd4,0x99,0x15,0x99,0x47,0xba,0x03, +0x14,0x1d,0x48,0x00,0x00,0xab,0xac,0x23,0xdf,0xe4,0x42,0xdf, +0xfe,0x38,0x10,0x61,0xdd,0x4a,0xf1,0x69,0x8b,0x1f,0xc4,0xde, +0xd5,0x41,0x62,0xfe,0x41,0x26,0x1b,0x14,0xbe,0xdb,0xe9,0x4b, +0xbd,0xe3,0x13,0x89,0x25,0x29,0xbc,0xdf,0x72,0x39,0x66,0xfa, +0xd4,0xad,0xe5,0x9d,0x40,0xdb,0x8f,0x05,0x47,0x8e,0x30,0xbd, +0x44,0xcf,0xe2,0x9f,0x71,0xc3,0xfc,0xd8,0xd7,0x32,0xfe,0x87, +0xef,0x55,0xcf,0x5d,0x5a,0x51,0x36,0xaa,0x6d,0x51,0x91,0x19, +0xe2,0x4c,0x82,0xd3,0xc2,0x6f,0xe9,0x23,0x19,0x14,0x02,0xc0, +0x25,0x60,0xd1,0x22,0xfb,0x93,0xe1,0x49,0x93,0xac,0x79,0x43, +0x9f,0x2c,0x77,0x84,0xb0,0x79,0xee,0x10,0x13,0x75,0xf5,0xf4, +0xfa,0xba,0x9c,0x55,0x80,0xab,0xf6,0x3e,0xd3,0x4e,0x73,0x37, +0xd7,0xa5,0xee,0xa7,0xed,0x3f,0xba,0x64,0x7a,0x10,0xee,0xc7, +0x47,0x2a,0x24,0x31,0x99,0x2f,0xdc,0x8f,0x5d,0x65,0xda,0x75, +0xf1,0xfa,0x2d,0x4d,0x5b,0xa9,0xa0,0x79,0xd8,0x9a,0x70,0x7c, +0x36,0xce,0xd0,0x93,0x97,0xa7,0xe9,0xc8,0x84,0xbd,0x04,0xa8, +0xe2,0xe7,0x95,0xa1,0x9f,0xe8,0x75,0xc6,0x40,0xac,0x40,0xab, +0x52,0x61,0x8d,0x69,0x3b,0x6f,0x94,0x31,0xa0,0x9b,0xbf,0x87, +0xc3,0xd4,0x7b,0xe7,0xd0,0x12,0xd9,0xb2,0xde,0x28,0x95,0x39, +0x0e,0xe8,0xa7,0x36,0x68,0xcf,0x44,0xab,0x5b,0xe7,0x31,0x96, +0xef,0xaf,0x02,0x89,0x1e,0x0e,0x1b,0x8d,0x65,0x8c,0xfe,0x2b, +0xf1,0x32,0x0e,0x35,0x66,0xed,0xe7,0x3c,0x63,0xf7,0x52,0x91, +0xa1,0x28,0xac,0xba,0x5f,0x82,0xf6,0x22,0xdf,0x8f,0x46,0x68, +0x46,0x1c,0xd8,0x95,0x69,0xa3,0xac,0x6d,0x97,0x6f,0x66,0x9d, +0x04,0x3b,0xb4,0x3c,0x5b,0x24,0x09,0x7e,0x7e,0x54,0x9d,0x98, +0xa2,0x2e,0xf0,0x53,0x6d,0xec,0x29,0xcc,0xd4,0xbe,0x6e,0xf8, +0x7d,0x36,0xc6,0x80,0x3c,0xb6,0x3b,0x0c,0xa0,0x17,0x5f,0x82, +0x96,0xf3,0x7e,0x35,0xc1,0x53,0x5b,0xcb,0x21,0x9b,0xdc,0xae, +0xa4,0xcd,0x31,0x24,0x4e,0x8a,0x90,0x80,0x82,0x2c,0x6f,0x4d, +0x78,0x4a,0x52,0xb1,0x28,0x42,0x25,0x89,0x61,0x6b,0x78,0x12, +0xef,0x69,0xbf,0xc6,0x9b,0x26,0x75,0x69,0x47,0x5b,0x7b,0x2d, +0xc3,0x81,0x87,0xc6,0x50,0x64,0xa2,0x71,0xc8,0xb3,0x00,0x8c, +0x6e,0x57,0xbe,0x62,0xe5,0xdb,0x50,0xd3,0xf5,0x9c,0xed,0x13, +0x02,0x17,0xa6,0xdd,0x29,0x47,0x69,0xc0,0x92,0x98,0x54,0xda, +0xb9,0x78,0x50,0x3e,0xde,0xbe,0x6e,0x1c,0xde,0xf7,0x35,0x4a, +0xb0,0xdb,0x56,0x9d,0xda,0x81,0xb4,0x0e,0x41,0x9b,0x78,0x69, +0xee,0xb7,0x82,0x8f,0x37,0xbd,0x6b,0x6c,0x64,0x2a,0x8b,0xc4, +0x67,0x9d,0xcc,0xdf,0x10,0x65,0x43,0x23,0xbf,0xf8,0x8b,0xe5, +0xea,0x3d,0x9f,0xba,0x1c,0x65,0x90,0x06,0xa8,0x5a,0xe0,0x23, +0xfc,0xf6,0xf0,0x84,0xbd,0x91,0x75,0x0e,0xf9,0x30,0xc8,0x87, +0x28,0x02,0x28,0x7a,0x85,0x24,0xa9,0x7f,0x53,0x58,0xa2,0xd0, +0xda,0xde,0xd5,0x9e,0x3b,0x41,0x8a,0x6f,0x28,0x65,0xb1,0x75, +0xa3,0xbd,0x12,0x6c,0x14,0x82,0x0c,0x31,0x80,0xdc,0x65,0x04, +0xbc,0x36,0x4d,0xb7,0xc3,0x35,0x17,0xa2,0x9f,0xfa,0x2e,0x6f, +0x90,0x2f,0x0e,0xa2,0xf4,0x5b,0xce,0x86,0x8d,0xf3,0x5f,0x30, +0x7a,0xdc,0x9a,0xb0,0x1b,0x1b,0x96,0x0c,0x15,0x60,0x64,0x5a, +0x88,0xa9,0xb8,0x0e,0x18,0xdf,0x98,0x34,0x29,0x82,0xaa,0x9c, +0x36,0x0c,0x44,0x91,0xf6,0x93,0x57,0x50,0xad,0x5f,0xaf,0x2f, +0xf0,0xde,0x34,0x14,0xfd,0x0d,0x06,0xab,0xf4,0x2b,0x05,0x1e, +0xcb,0xc1,0x31,0xce,0x2f,0xe4,0x1d,0x3e,0x84,0x5d,0x5e,0x97, +0x9b,0xe9,0xff,0x28,0xdf,0x17,0x9d,0x4e,0xf0,0xad,0xf3,0x0b, +0x13,0x37,0xc7,0x30,0x3b,0xe1,0xa7,0x70,0xd2,0x10,0x33,0xac, +0xa7,0x60,0xa3,0xd4,0x3a,0x4f,0xa6,0xf3,0x0d,0xb4,0x73,0xc4, +0x67,0x83,0xfa,0xe8,0x62,0xba,0xb8,0x28,0xe7,0xc8,0x84,0x8e, +0x2e,0x36,0xeb,0x39,0x66,0x05,0x5b,0x33,0x23,0xe4,0xe7,0xab, +0x69,0x8d,0x21,0x8c,0x08,0x86,0xa3,0x35,0x54,0x00,0xa9,0x58, +0x0a,0xd0,0xd9,0x1f,0x93,0x79,0x2c,0x14,0x50,0xe6,0xd9,0x66, +0x7a,0xbd,0x3a,0x7a,0x0b,0x67,0x9d,0xa3,0xdb,0xab,0xea,0xe2, +0x2a,0x7c,0xe1,0x53,0x68,0x0e,0x2f,0x6b,0x1d,0x9d,0x08,0x78, +0x17,0x57,0xe8,0x2a,0x0a,0xff,0x3c,0x02,0x64,0x60,0x74,0x26, +0xfc,0x0f,0x7f,0x47,0x4d,0xe0,0x70,0xe3,0x1d,0x44,0xc7,0x1d, +0x5d,0xfc,0x33,0x02,0x09,0x0e,0x3b,0x09,0xf4,0x55,0xc8,0x03, +0x4e,0x52,0xad,0xaf,0xdd,0x8e,0x1a,0x29,0xb9,0x4d,0x11,0x68, +0x64,0x9a,0x60,0x3b,0xf3,0x96,0x83,0x8e,0x81,0x77,0x8d,0x23, +0xfe,0x03,0x28,0x9e,0x57,0x50,0xf4,0xef,0xf2,0xf7,0x1f,0x47, +0x97,0xeb,0xe5,0xb5,0x50,0xce,0x11,0xc7,0x13,0xfd,0xbb,0xfc, +0xfd,0xc7,0x11,0xec,0xbd,0xe5,0xdf,0xe9,0xdf,0x7f,0x1c,0xd5, +0x17,0xeb,0xb2,0x5c,0xfc,0x5d,0xfe,0xfe,0xe3,0x68,0xb3,0x94, +0x5a,0x9f,0x84,0x0a,0x64,0x22,0x18,0x74,0x94,0x01,0x86,0x23, +0x89,0x01,0x62,0x35,0x62,0xac,0x67,0x87,0xc5,0xdf,0xc6,0x03, +0xcd,0xe5,0x79,0x89,0xc3,0x16,0xc4,0x12,0xc8,0x6a,0x25,0xed, +0x30,0xc7,0x83,0x56,0x84,0x43,0xbf,0x21,0xc5,0xef,0x39,0x3b, +0x23,0x70,0x85,0xd6,0x88,0x9f,0xcf,0xe5,0x08,0xdf,0x36,0xd7, +0x8d,0x8a,0x61,0x68,0x74,0xde,0xc3,0x08,0xbf,0xf4,0x1a,0x6f, +0xbf,0xa6,0x77,0x6f,0xbf,0x2f,0x2f,0xa1,0x2f,0x8e,0xfc,0xcd, +0xc1,0x59,0xbd,0xe4,0x93,0xec,0xd8,0x95,0xe7,0x16,0x1a,0xe5, +0xfd,0xe4,0x93,0xcc,0xeb,0xfc,0x1f,0x71,0xe7,0xff,0x68,0x74, +0xfe,0x7c,0xb9,0x6a,0xe9,0x9b,0x52,0x5b,0xba,0x8e,0x4b,0x7b, +0xa9,0x27,0x18,0x03,0x4d,0x85,0x2a,0x6f,0x25,0x76,0xbb,0xde, +0x4c,0x79,0x42,0xa0,0x57,0xa4,0xf0,0x27,0x38,0x64,0x54,0x8e, +0xe8,0x2d,0x9d,0xe5,0x5e,0x69,0x1d,0x2c,0x2d,0x9c,0xad,0x44, +0xf6,0x29,0x42,0x8f,0x59,0x7f,0x69,0x0d,0xba,0x5c,0x64,0x34, +0xc8,0x1f,0xe8,0xcf,0x2f,0xf2,0x2f,0xf5,0xe7,0x83,0xfc,0xc4, +0x2d,0x21,0x11,0x23,0xf3,0x3b,0x7c,0x76,0x39,0xbf,0x53,0xcd, +0x10,0xc5,0x75,0x25,0x89,0x28,0xbf,0x6b,0xe8,0x84,0xf7,0xda, +0xc2,0xf2,0x01,0x4d,0x6c,0x61,0xbb,0x5d,0x67,0xb0,0xa3,0xde, +0xd2,0x7c,0xec,0x16,0xd7,0x77,0x68,0x7c,0x94,0xba,0xdf,0x20, +0x0a,0xe3,0x55,0xa7,0x1a,0xc9,0xf0,0x2b,0x01,0xad,0xbd,0x4b, +0xfc,0x4d,0xff,0xc9,0x10,0x7c,0x94,0xee,0xad,0x7b,0x04,0x9a, +0xf7,0x57,0x6c,0x20,0xc3,0xa3,0xb3,0x1f,0x78,0x83,0x52,0xa9, +0x2f,0x3f,0x9e,0x8c,0x09,0x6e,0x97,0x12,0x31,0x91,0x49,0x76, +0x06,0xed,0x62,0x0e,0xa1,0x40,0xdf,0x49,0x3b,0x04,0x08,0xb6, +0x21,0x23,0xc5,0x9f,0xe1,0x40,0xdb,0x7a,0x5e,0x02,0xfe,0xa0, +0xeb,0x12,0x84,0x86,0xf2,0x66,0xc1,0x33,0xe3,0xcb,0xf9,0xed, +0x87,0x73,0x11,0xf3,0x5b,0x8f,0x23,0xe1,0xf6,0xc9,0xb0,0xb3, +0x79,0xb2,0x5f,0x15,0x6d,0xff,0x4c,0x5d,0x5d,0xdf,0xcc,0x83, +0xc7,0x1c,0xed,0x75,0x87,0x9c,0x49,0x79,0xff,0x62,0xf6,0x56, +0x46,0xe6,0x04,0xb1,0xa8,0x20,0x55,0x3c,0x2b,0x83,0xaa,0x7e, +0x26,0x3d,0x70,0xec,0x61,0x1f,0xb2,0x1c,0x43,0xfb,0x0c,0xb9, +0xfd,0x51,0x9b,0x66,0x23,0x65,0x9f,0x79,0x82,0x26,0xdb,0x63, +0x54,0xe2,0x6e,0x78,0x70,0xab,0x68,0x55,0x29,0xed,0x53,0x54, +0x39,0x9b,0x59,0xef,0xce,0xc7,0x7f,0x98,0xdd,0x26,0xda,0x5b, +0xfc,0x96,0xc8,0x9a,0x72,0x21,0x44,0xe2,0x6b,0x5b,0x1d,0xb9, +0xbd,0x6f,0xc9,0x49,0xbd,0xfa,0x48,0x24,0xbb,0xb6,0xc0,0x9d, +0xda,0xbc,0xfa,0xf7,0x15,0x56,0x95,0xcf,0x31,0x4f,0xf8,0x5d, +0xf1,0x27,0xac,0x8c,0xf7,0xfd,0x26,0xc5,0x28,0xb4,0xdd,0x61, +0xd2,0x2b,0xc0,0xfa,0xa8,0xb8,0x2d,0xb6,0x0f,0x95,0xce,0xdd, +0x1b,0xe4,0x8c,0x22,0x17,0x7b,0x62,0x7d,0xc1,0xd2,0x4e,0xb0, +0x42,0x8e,0xaa,0x45,0xbd,0x41,0x69,0xc7,0x9a,0x14,0x71,0xbd, +0x11,0x96,0x87,0xb3,0xec,0xfa,0x82,0xed,0x85,0x78,0x91,0x84, +0xb2,0x35,0xb6,0x68,0x99,0x47,0xa1,0x45,0x39,0xa9,0x39,0xb7, +0x54,0x60,0x16,0x25,0x6e,0xb7,0x98,0xea,0x53,0x3d,0xeb,0x03, +0x31,0x15,0xd8,0xef,0xcf,0x01,0x15,0x30,0x38,0x8d,0xe4,0x54, +0x1f,0xd7,0x7c,0xbe,0x66,0x23,0x77,0xf8,0xc9,0x4f,0x5b,0xe7, +0x01,0x74,0x3c,0x7a,0xcf,0xca,0x85,0x96,0x05,0x71,0x46,0xc6, +0x8b,0x0c,0x46,0x25,0xb5,0xc2,0x61,0x40,0x93,0xac,0x36,0x71, +0xb1,0xbc,0x95,0x47,0x78,0xe3,0xc3,0x0a,0xde,0xa6,0x58,0x97, +0x82,0x58,0x3a,0x77,0x73,0x10,0xce,0x12,0xca,0xf4,0x9b,0x25, +0x01,0x7a,0xd7,0xc4,0x9c,0x3f,0x26,0xd3,0xa6,0x44,0x8a,0x0a, +0x1c,0x50,0xfd,0x04,0x25,0xc3,0x35,0x16,0x3f,0xde,0x2a,0x81, +0x50,0x82,0x29,0x1f,0xee,0x9b,0x5b,0x37,0x01,0x86,0x64,0xfa, +0x68,0xfd,0x8e,0xe2,0x84,0x14,0x04,0xf0,0x60,0xda,0x71,0x49, +0x99,0x48,0xd5,0xf2,0x29,0x10,0x35,0x87,0xda,0x84,0x29,0x6a, +0x1e,0x16,0x7b,0x53,0xb9,0x63,0x50,0xe7,0x89,0xc2,0x3f,0xef, +0xb6,0x05,0x85,0x03,0xc6,0x52,0x6d,0x48,0xcd,0x63,0xf7,0xf8, +0x83,0xb8,0xf7,0x01,0x62,0x67,0xfa,0xb8,0xf3,0xc8,0x31,0xe0, +0x8e,0xa4,0x68,0x44,0xf0,0x3a,0x4f,0xe8,0x37,0x70,0xa5,0x75, +0xc2,0xc2,0xf5,0xbc,0x9c,0xbe,0x2d,0x35,0x99,0x76,0x25,0xf7, +0x1a,0x2b,0x20,0x08,0xcf,0x13,0xee,0x21,0xdd,0x40,0xcb,0x86, +0xd9,0x93,0xe2,0x2e,0xd8,0xdd,0xa0,0xb4,0xd1,0x5b,0x10,0xfa, +0x60,0x6e,0xd2,0xd8,0xce,0xd4,0x7d,0x72,0xe3,0x29,0x60,0x45, +0x96,0x2a,0x5a,0x24,0x2b,0xcf,0xb6,0x2d,0x52,0x15,0x8a,0xcc, +0x9c,0x76,0xa4,0xfc,0x76,0x2b,0x3f,0xf0,0x21,0x75,0x11,0xdc, +0x54,0xd3,0x2f,0x2f,0x00,0xd4,0x29,0x67,0x68,0x87,0x9e,0x06, +0x92,0x56,0x4e,0x53,0x7b,0x48,0xba,0xc6,0xa6,0x75,0x54,0xeb, +0x73,0xbe,0xc6,0x6b,0x09,0x71,0x67,0xc8,0xda,0xbf,0x69,0xc8, +0xce,0xcf,0x26,0xc9,0x53,0x47,0x9e,0x7b,0x83,0x8f,0x62,0x29, +0x54,0xdc,0x91,0x49,0xda,0x47,0x8b,0x67,0xfc,0x24,0x1b,0x3a, +0x8b,0x84,0xad,0x62,0x08,0x4b,0x2e,0x41,0xe2,0x58,0xff,0x25, +0x37,0x8f,0x67,0xc0,0x15,0x86,0x6c,0xd5,0x04,0xef,0x71,0x28, +0x95,0x01,0xe8,0xba,0x5c,0x64,0x59,0x83,0xed,0x17,0x07,0xbd, +0xa4,0x3c,0xfe,0x16,0x64,0xb3,0x7c,0xaa,0x2e,0x5c,0xd8,0x8e, +0x73,0xa6,0x41,0x05,0xca,0x75,0x1c,0x3f,0x1d,0xd3,0x4c,0x12, +0xe0,0x0a,0xe5,0xce,0xe6,0xb8,0xfc,0x82,0x6d,0xe3,0xf0,0xa5, +0x28,0xc9,0x7e,0x79,0xae,0x4b,0xd3,0xcd,0xce,0xfe,0x5e,0xd1, +0x2e,0x23,0x88,0x01,0xfe,0x11,0xe2,0x5a,0xd8,0x91,0x67,0xb8, +0xd1,0x92,0xcb,0x4b,0xd9,0xf7,0x19,0xb0,0x2a,0x78,0xb9,0x5f, +0x8f,0x8c,0x71,0x55,0x8a,0x4b,0x13,0x1d,0x6d,0xd4,0x82,0x08, +0x66,0x04,0xb7,0x51,0xeb,0x98,0x3f,0x4c,0x44,0x62,0xc9,0xc2, +0x85,0x2c,0xa6,0x1d,0x5e,0x9a,0x94,0xce,0xcf,0x82,0x7d,0x80, +0xd2,0xb9,0xd0,0x7e,0x4a,0x77,0x96,0xf4,0xa2,0x0d,0xf3,0x83, +0x13,0x65,0xa3,0x34,0x6d,0x3f,0x8a,0x6c,0xb7,0xce,0xc3,0xde, +0x26,0xb6,0x12,0x10,0x0f,0x08,0x37,0xd5,0x72,0xbd,0x79,0xcf, +0xe0,0xf4,0x5f,0xf2,0xdf,0x26,0x21,0xd9,0xc7,0xe4,0xec,0x39, +0x2f,0x94,0xbf,0xb5,0x1d,0x84,0x4e,0x63,0x8b,0xbd,0x7c,0x7d, +0x53,0x6f,0xa4,0xc5,0x19,0x6d,0x08,0xd1,0x95,0x60,0x63,0x7d, +0xee,0xeb,0xbd,0xd9,0x5c,0x1b,0xb5,0xb4,0xf6,0x3a,0x88,0xfa, +0x74,0x54,0xa4,0x9d,0x79,0x57,0x53,0x04,0x23,0x9f,0x65,0xf6, +0x21,0x8c,0x8f,0x30,0x74,0x26,0xc2,0xc7,0x8d,0x5b,0x40,0x6e, +0xe1,0x20,0xc3,0x78,0x36,0xad,0x8d,0x8d,0xf8,0x3d,0xb7,0x3d, +0xa2,0x10,0x90,0x51,0x3b,0x13,0xf0,0x0b,0xee,0xc7,0x5e,0x27, +0x5a,0x27,0xee,0x66,0xda,0x1e,0x5b,0x5c,0xd2,0x73,0x35,0xca, +0xf8,0x08,0xac,0xb5,0xae,0xbd,0x98,0xb7,0xb4,0x0d,0xa6,0xc1, +0x5b,0x0e,0xec,0x9a,0x8c,0x48,0xff,0xba,0xc5,0x93,0xc2,0xc5, +0x5c,0xe9,0xe3,0x46,0x24,0x2b,0xa3,0xe3,0x05,0x64,0xf2,0x4f, +0xf4,0x7e,0xfa,0x28,0xda,0x75,0x0f,0x6f,0x81,0xf6,0xe0,0xfb, +0x61,0xa6,0xd3,0xca,0x53,0x04,0xa7,0x40,0x78,0x87,0x57,0x7d, +0x0b,0xaf,0x91,0x63,0xbd,0x65,0x36,0x81,0x18,0xc4,0xfa,0x06, +0x7b,0xf4,0xe7,0x93,0xbf,0x7f,0x1e,0x6f,0x91,0x7c,0x24,0x88, +0xc0,0x94,0xae,0x87,0x4e,0xac,0x7f,0x48,0x3c,0x33,0xed,0xd4, +0x81,0x02,0x90,0x3f,0x55,0x66,0xcf,0xe5,0x3a,0xd1,0xc9,0xb0, +0x55,0xc8,0x82,0x42,0x93,0x16,0xae,0x48,0x81,0x4b,0x05,0xac, +0x5e,0x8f,0x54,0x5b,0xa2,0x9d,0x88,0xad,0xcd,0x69,0x28,0x6a, +0x64,0x48,0x52,0x68,0xdb,0xac,0x60,0x7b,0xc7,0xc7,0xda,0xa2, +0xd7,0x5e,0xdb,0x11,0xb6,0xd1,0x64,0x7b,0x7c,0x7c,0x5f,0xae, +0x65,0xf3,0x6c,0x6b,0xe3,0x48,0x17,0xb8,0x97,0x0b,0x8a,0x56, +0x65,0x63,0x6d,0x92,0xa2,0xe0,0xdb,0x05,0x5e,0x1f,0x36,0x0c, +0xcf,0x36,0x6c,0xc1,0xdd,0x88,0x0e,0xee,0x99,0xb5,0xd2,0x1d, +0xf1,0x8c,0x5e,0x97,0xb1,0xfd,0x58,0x97,0x1d,0x17,0x30,0x7c, +0xe8,0x9b,0x6a,0x72,0xb3,0x7c,0x1a,0x10,0x2d,0x48,0x08,0xa3, +0x98,0x69,0x92,0x5d,0x07,0x02,0xeb,0x2f,0xb6,0x1d,0x00,0xea, +0x6c,0xc4,0x58,0x47,0x8b,0x6f,0x39,0x8c,0xf0,0x29,0x87,0xa0, +0x99,0xa2,0x05,0x8e,0x5c,0x8b,0xa3,0x45,0x6e,0x1c,0x47,0xce, +0x9a,0x46,0x63,0x53,0x04,0x08,0x35,0xe3,0xd5,0x0e,0x33,0x0e, +0x0d,0x38,0xc3,0xd7,0x2c,0xc8,0x42,0x01,0xfe,0x7a,0xc7,0xb6, +0xe1,0x7f,0x21,0xcb,0x55,0x34,0x22,0xbb,0x5c,0xf8,0x91,0x6b, +0x75,0x8c,0x78,0xab,0x0a,0x63,0xc6,0xe0,0xeb,0x34,0x31,0xfc, +0xee,0x44,0x83,0xf4,0xc3,0x10,0x60,0x19,0x3e,0xe2,0xa5,0x64, +0xcd,0xf5,0xda,0x99,0xc4,0x0e,0xdf,0xa0,0x20,0xff,0x02,0x29, +0x25,0x4e,0x08,0xde,0x57,0xe4,0x36,0xd5,0x1a,0xc3,0x31,0x5a, +0x7a,0x76,0x63,0x62,0x82,0x03,0x88,0x43,0x87,0x34,0x0c,0xa7, +0x89,0x6f,0x13,0x7f,0x80,0x30,0xc3,0x30,0xdd,0xfb,0xe9,0x17, +0x0f,0x9b,0x30,0xde,0xbd,0xcd,0x61,0x4b,0xce,0x82,0x96,0x7c, +0x0e,0xde,0x73,0x4c,0x98,0x0d,0x5b,0x88,0xb3,0xd1,0xfe,0x2a, +0x52,0x4f,0x70,0xaa,0xbb,0x9e,0xd5,0x88,0x88,0xf6,0x10,0x14, +0x15,0xd0,0x27,0x45,0x39,0x39,0x34,0x6e,0xe1,0x09,0x69,0x31, +0xbc,0x18,0x35,0xcf,0x39,0xbd,0xa4,0x9f,0xf4,0x5a,0x8a,0xe6, +0x2d,0x47,0xa2,0xe6,0xad,0x77,0xd3,0xb1,0x84,0x67,0xec,0xd0, +0x82,0xde,0xb7,0x18,0x2f,0x2f,0xa3,0xd5,0xe8,0x2d,0xc4,0x70, +0x11,0xca,0xf9,0xcf,0x12,0xbe,0xd8,0x81,0xe9,0x8c,0x34,0xd7, +0x54,0xc6,0x6f,0xad,0x1c,0x5a,0x32,0xba,0x62,0xb8,0xa4,0xaf, +0x09,0xfa,0x08,0x12,0xf4,0x37,0x33,0x4b,0x85,0x01,0x01,0xe2, +0x01,0x39,0x26,0x99,0xc3,0x84,0x47,0x1a,0x51,0x2d,0xb2,0x33, +0x37,0x8b,0xb6,0x26,0x1a,0xb5,0x05,0x89,0x52,0x9d,0x6a,0x2a, +0x6d,0x34,0x43,0xc9,0x9a,0x8f,0x81,0xa3,0xb1,0x4e,0x10,0x96, +0x0f,0xb6,0xe9,0x35,0x37,0x68,0x09,0xb8,0x3d,0xb2,0xd0,0xda, +0x7a,0xe8,0xa2,0x20,0x5a,0x37,0x37,0x0a,0xe7,0xb0,0x80,0xd9, +0xdc,0x7b,0x43,0x6d,0x4f,0x54,0x13,0x3c,0xc1,0xf6,0xe1,0x19, +0x53,0xcd,0xb3,0xad,0xcb,0x16,0x17,0x3b,0xd7,0xbe,0x18,0x31, +0xee,0x7d,0xec,0x85,0x84,0x32,0x8d,0x54,0xeb,0xf9,0xb5,0xb7, +0x6b,0xb8,0x5d,0x3f,0x24,0x0c,0xc2,0x0e,0xea,0x1c,0x87,0x9d, +0x0c,0xc2,0x36,0xab,0xc6,0x7b,0x95,0x4c,0xb3,0x8e,0xaa,0xfa, +0xc7,0x29,0x5e,0x42,0xa4,0x1e,0x66,0xd7,0x0b,0x4a,0x62,0x39, +0xe9,0x72,0x81,0xe1,0xe6,0x76,0xb6,0x02,0x87,0xc4,0x78,0x84, +0x9b,0xbb,0x3e,0x37,0x8c,0xbf,0xf1,0x82,0x18,0x8d,0x5e,0x14, +0x64,0x4a,0x2c,0xfc,0x97,0xaa,0x22,0x77,0x69,0x2c,0xcf,0x96, +0x1b,0xf0,0xab,0x57,0x50,0xb0,0xee,0x27,0xe8,0xe5,0x47,0x35, +0xf5,0xd1,0x20,0x39,0xe5,0xf2,0xe3,0x7f,0x54,0x45,0x2c,0xd8, +0x27,0xc6,0xa6,0x49,0x50,0xa8,0x9d,0x83,0xf1,0x7a,0xba,0x7e, +0x63,0x83,0xf6,0x7a,0x03,0xbb,0x5c,0x8c,0x7d,0x9d,0xe8,0xe5, +0xc2,0x55,0xc1,0x98,0xe2,0xeb,0x4d,0xaa,0x5c,0x16,0xdf,0x75, +0x3c,0xf8,0xea,0xe8,0xd0,0xbb,0x2a,0xba,0x5c,0xd0,0x2b,0xbf, +0xee,0x8a,0xc8,0x06,0x29,0xbe,0xc4,0x63,0xdc,0xfc,0xfd,0xdd, +0x4c,0x1e,0x86,0xf4,0x60,0x7c,0x56,0xfd,0xf6,0xdb,0xbc,0x74, +0x74,0x2a,0xde,0xd8,0x86,0x2f,0x57,0x90,0x44,0x71,0x9e,0x10, +0x16,0xb2,0x8d,0xe1,0xf9,0xbd,0x36,0x6a,0xa5,0x60,0x2a,0x43, +0x71,0x3c,0x6b,0xb3,0x9c,0xcf,0xcc,0xa2,0x82,0xff,0xca,0x5b, +0x31,0xf8,0xc1,0x9f,0xcf,0xa4,0x5d,0xe4,0x98,0xa9,0xb4,0x3d, +0x92,0xbf,0xf1,0x9d,0xae,0x24,0xe7,0xb0,0x77,0xf0,0x03,0x2d, +0x90,0x95,0x79,0x17,0x66,0xe8,0x07,0xb4,0xd1,0xe2,0xda,0x58, +0xa6,0x10,0x17,0xf2,0xd7,0xbb,0x1b,0x96,0x31,0x14,0xf2,0x97, +0x6c,0x2d,0x3b,0xde,0x9a,0xdb,0x27,0x7a,0x65,0x36,0x50,0x0b, +0xd5,0x43,0xd8,0x31,0x78,0x59,0xaa,0x83,0xd6,0xae,0x5c,0x4c, +0x41,0x7e,0x6d,0x30,0x7a,0x68,0x90,0xfd,0x33,0x3a,0x0a,0xce, +0x53,0x0c,0x76,0x86,0x06,0xed,0xa5,0xbc,0xc7,0x47,0x08,0x2d, +0xd6,0xbf,0xde,0xc0,0xc9,0x98,0x48,0x8e,0xfc,0xd8,0x2c,0x10, +0x68,0x06,0x76,0xcd,0xd6,0x8c,0xe3,0xc1,0x84,0x6a,0xfc,0xa5, +0xf0,0x3a,0xd1,0x57,0x10,0x2d,0x34,0xee,0x65,0xe6,0x6f,0xde, +0x3f,0x9d,0xa5,0xd7,0x78,0xeb,0x4b,0x47,0x27,0xfa,0xe3,0xdb, +0x8c,0x34,0xc7,0x47,0x25,0x2a,0xf4,0x82,0xbe,0x8e,0x72,0x79, +0x71,0xb0,0x13,0x81,0x24,0xed,0x54,0xcc,0x6a,0x9d,0x48,0xb9, +0x97,0x29,0x5a,0x33,0x9b,0x40,0x02,0xee,0xac,0x16,0x53,0x89, +0xaf,0x64,0x6f,0xc3,0x4f,0x07,0xca,0xa2,0x75,0xfc,0xc0,0xda, +0x46,0x61,0x51,0x11,0xd8,0x2c,0x51,0x3b,0x0b,0xde,0x26,0xf2, +0xbc,0x67,0xad,0x9d,0x47,0xef,0x49,0x66,0x3b,0x22,0x52,0xd6, +0x89,0xf9,0x02,0x2d,0x4b,0xbd,0x57,0x5c,0xbe,0x79,0xff,0xc8, +0xc5,0x5b,0x6f,0x6d,0xdb,0xe6,0xff,0x01,0xf8,0x6c,0x1b,0x38, +0xc9,0x1e,0x68,0x28,0xbf,0x2b,0x30,0xbf,0xd6,0x53,0x74,0xed, +0x39,0xbf,0x79,0xf5,0xea,0xfd,0x7f,0x3c,0x7b,0x18,0xb9,0x28, +0x13,0xf5,0xc0,0x92,0xa5,0xd8,0x58,0x80,0x62,0xe1,0x47,0xde, +0xea,0xd5,0xf9,0xf3,0x57,0x71,0xe1,0x13,0xa0,0x73,0xcb,0xa3, +0x47,0x23,0x5d,0xb2,0x4a,0x46,0x14,0xe5,0xd4,0x2d,0x94,0x1f, +0x1b,0xe1,0xc9,0x58,0x58,0x62,0xfe,0x51,0x6c,0x96,0x6f,0xca, +0x45,0xf5,0x9b,0x8f,0x74,0x02,0xd0,0x43,0x80,0x17,0x88,0x03, +0x1f,0xd2,0xcc,0x46,0x08,0x39,0x94,0x71,0xc1,0xa9,0xcb,0xfa, +0x62,0x8a,0x5e,0x3e,0x67,0x67,0x9f,0x75,0x61,0xbb,0xd5,0xba, +0x75,0x5c,0x17,0xc7,0x9c,0xd1,0xc0,0x93,0x31,0xfc,0xf3,0x79, +0xd2,0x83,0xdf,0xbd,0xe4,0xf3,0x09,0x05,0x97,0x67,0x88,0x02, +0xd3,0x30,0x4e,0x1a,0x57,0x93,0x02,0x0b,0x6e,0x96,0x8a,0x91, +0xd4,0x66,0x64,0x43,0x0f,0x77,0xeb,0xba,0x3a,0x9f,0x03,0x6f, +0x89,0xb0,0xee,0x48,0xc2,0xd7,0xd5,0xb4,0x21,0x5a,0x40,0x60, +0xa7,0x20,0x03,0x1b,0x1f,0x9e,0xcc,0x5c,0xbe,0x6f,0x21,0x70, +0x98,0x7e,0x1c,0x50,0xf8,0x8a,0xd6,0xfa,0xbd,0xb6,0xf0,0x90, +0xf3,0x9e,0xb5,0x51,0xb8,0xec,0x20,0x40,0x41,0x4f,0xd6,0x6b, +0x8c,0x79,0x61,0x37,0x10,0x40,0xb6,0x85,0xb7,0x11,0x62,0x94, +0x26,0x05,0x4e,0xce,0x6a,0x02,0x4c,0x4d,0xdb,0xc1,0xbb,0x49, +0xc2,0x87,0x2c,0x4c,0xf2,0xd9,0x20,0xc9,0xda,0x77,0x1b,0xb7, +0x39,0x09,0x16,0x1f,0xa1,0x02,0x27,0x9d,0x9a,0x73,0xd9,0xe8, +0x6f,0xd6,0xc5,0x79,0xb7,0x0b,0x07,0xc5,0xea,0xf2,0x12,0xad, +0x64,0x81,0xe3,0xfc,0xf3,0xbc,0x5f,0x2f,0x6f,0xd6,0x17,0xe5, +0x53,0x8e,0x79,0xfd,0xc3,0xc3,0xbf,0xbf,0xfc,0xf1,0xc9,0xbf, +0x3f,0x7c,0xfe,0xf4,0xaf,0x4f,0xb2,0xe3,0xf4,0x9f,0xd3,0x03, +0xd9,0x64,0xdd,0x0b,0x4d,0xe9,0x82,0xc4,0xdf,0xe2,0x59,0x67, +0x4d,0x76,0xd9,0x18,0x17,0x40,0x7d,0xc6,0x20,0x91,0x2b,0x32, +0xa6,0x17,0xc5,0xb9,0xd4,0x3b,0x1e,0xd8,0x58,0x8b,0xa3,0x41, +0x7e,0x3c,0x88,0x65,0x15,0x8a,0x3a,0xf7,0x73,0x5d,0xde,0xcc, +0x96,0xe1,0x13,0x67,0x2d,0xb1,0x7a,0xe8,0x8a,0x3c,0x50,0x21, +0x86,0x2b,0x47,0x9f,0xe0,0xe3,0xcb,0x09,0x09,0xf6,0x25,0x0c, +0x93,0xee,0x69,0xf8,0x8c,0xb0,0x8b,0x61,0xf8,0x86,0x6e,0x2b, +0xfe,0xb5,0x40,0xa4,0x21,0x14,0xf8,0x14,0x05,0xdf,0x89,0x48, +0x42,0xf6,0x11,0x70,0xe9,0x53,0xf0,0xd3,0xb9,0xc0,0xe6,0x49, +0x4b,0xa1,0x14,0xa5,0x3f,0x54,0xe4,0xb6,0xc5,0x34,0xa1,0xe8, +0xe9,0x2f,0xd3,0x5e,0x13,0x69,0x4c,0x6d,0x88,0x5d,0x3c,0x78, +0xb1,0x19,0x27,0xfa,0x28,0xeb,0x02,0x64,0x29,0x74,0xcc,0x80, +0xa2,0x1a,0x9d,0xdd,0x76,0x08,0x0c,0xc2,0x2f,0x1a,0xb0,0x09, +0xac,0x30,0x7e,0x1d,0xe4,0x03,0x6b,0x98,0xa0,0x97,0x2f,0xe5, +0x80,0xbc,0x67,0x4d,0xa5,0xe1,0x43,0x12,0x33,0x7e,0x92,0xd1, +0xe2,0xa4,0xc1,0x0a,0x0d,0x85,0xe5,0xc4,0x20,0xb5,0x9e,0x28, +0xa6,0xa3,0x30,0x54,0x5c,0x7c,0xfa,0xeb,0xe5,0xb7,0xd3,0xb5, +0x8a,0x64,0x20,0x48,0x7d,0x4b,0xd6,0x7e,0x35,0x4b,0xa9,0x33, +0x66,0xb2,0x24,0x34,0x8f,0x7d,0xcb,0x6e,0x76,0x24,0xa5,0x22, +0x99,0x8b,0x07,0xc5,0x3d,0x8e,0x4e,0x72,0xce,0x11,0xeb,0xe5, +0x13,0xd6,0x37,0x51,0x47,0xee,0xc8,0x29,0x4c,0x1c,0x70,0xe6, +0x3a,0x2d,0x48,0xa8,0xb1,0xdf,0x43,0xaa,0x32,0xcc,0xee,0xd2, +0x8e,0xc0,0xbf,0xdd,0xaa,0x0c,0x74,0xb1,0xbc,0xbe,0x9e,0x8a, +0xfc,0x83,0xa5,0x32,0xba,0xc1,0xbc,0x66,0xaf,0x01,0xed,0x0b, +0xff,0x15,0x20,0x78,0xf3,0x3d,0x99,0xa8,0x4d,0xf8,0x76,0xcb, +0xd5,0x04,0x0e,0xf1,0xa2,0x23,0xbc,0x00,0x4c,0x99,0xda,0x8c, +0xe3,0x6d,0x81,0xf1,0xfa,0x84,0x33,0xe7,0x14,0xa0,0xaf,0x83, +0x9e,0xad,0x35,0xfb,0x8c,0xe7,0xd1,0x3a,0xab,0x72,0x7b,0x62, +0xe6,0xcd,0x71,0xf3,0x74,0x22,0xc8,0x52,0xc8,0x02,0x15,0x71, +0xba,0x23,0x7c,0xf3,0xc4,0xec,0x19,0x84,0x25,0xb0,0x2c,0xd4, +0xe3,0x11,0xee,0xd8,0x56,0x33,0x13,0x9a,0x61,0x70,0x30,0x83, +0xb5,0x06,0x01,0xd4,0xdb,0xad,0x43,0xfc,0x58,0x1c,0xef,0xb4, +0x5e,0x9c,0xc3,0xc9,0x58,0xe7,0x77,0x8e,0x94,0x48,0x4d,0xe8, +0x98,0x73,0x3e,0x6e,0x80,0x28,0x0e,0x4b,0x62,0x76,0x0e,0x0b, +0xff,0xcd,0xae,0x41,0x6d,0x5c,0x5f,0x42,0x2c,0xd3,0xc7,0x88, +0x8f,0x28,0xfd,0x12,0xf7,0x22,0xb7,0x97,0xe6,0x8e,0x96,0xd3, +0x88,0x0c,0x33,0x4b,0xaa,0xfe,0x9a,0xb2,0x5b,0x36,0x8f,0xce, +0x2d,0x7d,0xfb,0x9c,0x16,0x8f,0x5a,0xd6,0xbb,0x55,0xa2,0x24, +0x89,0xf7,0xd8,0x94,0x5d,0x35,0x52,0x1a,0x5f,0x93,0x09,0x82, +0x56,0x6b,0x01,0xef,0x48,0x37,0x9b,0x3d,0xb2,0xa4,0x26,0x38, +0xc1,0x73,0x96,0x26,0x99,0x73,0xe0,0xa6,0x7a,0xdc,0x83,0x8d, +0xc6,0x66,0xf4,0xe1,0xd3,0xd0,0xe5,0xc5,0x8f,0xcb,0x85,0x0a, +0x7f,0x05,0x96,0x06,0x19,0xcb,0x09,0x10,0xc8,0x67,0xb1,0xe4, +0x6c,0xb9,0x20,0x26,0x5d,0xe0,0x8f,0x5e,0xcf,0x9e,0x80,0x5d, +0x7b,0x97,0xd5,0xba,0xde,0x44,0xc6,0x5f,0xba,0x03,0xbf,0xbb, +0x9e,0x33,0x46,0x38,0x30,0x09,0x59,0x56,0x41,0xab,0x93,0xa1, +0x7a,0xa3,0x94,0xbe,0xd3,0x38,0x08,0x01,0x11,0x5c,0x99,0xe5, +0xd6,0x34,0xbe,0x66,0xe3,0xb1,0x59,0x58,0xd0,0x31,0x0d,0x1e, +0xd5,0x07,0x7c,0x88,0x5f,0x82,0x38,0xb1,0xa6,0xa9,0x85,0x3d, +0x1e,0x9d,0x06,0x70,0x80,0xeb,0x9b,0x45,0x8d,0xcc,0xaa,0xa7, +0x03,0x45,0x72,0x3a,0x08,0xf6,0x47,0xc1,0xad,0x1e,0x24,0x2d, +0x20,0xdb,0xe7,0x7e,0x5c,0x44,0x84,0xdf,0x8b,0x1d,0x94,0xb6, +0xed,0xa0,0xb8,0xbe,0xaa,0x00,0x24,0xd6,0x8e,0x53,0x09,0x50, +0x64,0x2c,0x56,0x5c,0xb8,0x3a,0xd4,0x1d,0x0a,0x90,0xf4,0x71, +0x32,0xe1,0x49,0xa7,0x97,0xb7,0x71,0xa4,0xa4,0x33,0xe7,0x3c, +0x38,0x2b,0x16,0xec,0x1e,0x24,0x0f,0xe7,0x32,0xfb,0x46,0xf4, +0x59,0x09,0x87,0x33,0x74,0x60,0xca,0xfc,0xe3,0xee,0x8a,0x31, +0x77,0xa1,0x4a,0x8e,0xc1,0xa4,0xd8,0x8b,0x2b,0x18,0xb3,0xed, +0xc6,0x15,0xc7,0x3e,0x1c,0x16,0xdd,0x9a,0x28,0x19,0x2f,0x3f, +0x48,0x6b,0xd2,0x6a,0xed,0xed,0xf9,0x9c,0x60,0x9f,0x72,0xfc, +0x10,0xd5,0xf2,0x3a,0x8e,0xaa,0xf1,0x9e,0xec,0x98,0x0e,0x39, +0x38,0xed,0x9f,0x66,0x95,0xdf,0x00,0xd2,0xdc,0x56,0x38,0x99, +0x78,0xb2,0xca,0x12,0xb6,0xf4,0x05,0x88,0x3e,0x37,0x0b,0xb7, +0x03,0xaf,0xc4,0xa2,0xbe,0x1d,0x26,0xea,0x0c,0xe4,0xec,0xff, +0xd4,0x1a,0xe2,0x90,0x45,0x0c,0xc7,0x36,0xa3,0x3c,0x87,0x43, +0xad,0x88,0x9b,0x01,0x7c,0x78,0x8c,0x87,0xcf,0xd4,0xb6,0x06, +0x1e,0x38,0xd0,0xed,0x8c,0xfb,0xde,0x6e,0xd5,0x9f,0xa3,0x31, +0x36,0x8c,0xcb,0xe0,0x75,0xef,0x1f,0x9c,0xb9,0x37,0xa4,0xff, +0x15,0x27,0x57,0xce,0x7f,0xcf,0xaf,0xe3,0xc9,0xe3,0xa5,0x9d, +0x32,0xbb,0xa7,0xf8,0x91,0x89,0x98,0xb5,0xa1,0x19,0x89,0xe4, +0xf1,0x4f,0xc0,0x1a,0xff,0xb4,0xa7,0x0d,0x7b,0x76,0xb1,0x45, +0x61,0xb7,0x72,0x1f,0x76,0x2d,0x60,0x3c,0x4a,0x9b,0x5a,0xf8, +0xdd,0xdb,0x54,0xd8,0xdc,0x5d,0x2f,0xb6,0x15,0xfc,0x68,0xb6, +0x82,0xa9,0xcd,0x56,0x9a,0xe0,0x65,0x87,0x44,0x47,0x3d,0xab, +0x34,0x18,0xd8,0xa6,0xe4,0x80,0x21,0x38,0x07,0x80,0x9e,0x1f, +0xa6,0x2b,0x92,0x87,0xa0,0x61,0xfd,0xb9,0x2e,0xcb,0x77,0x55, +0xbd,0x81,0xf3,0x83,0xea,0xa4,0x74,0xe6,0xb1,0x52,0x4d,0xf2, +0xe0,0xd6,0x3e,0x3e,0x2b,0xe7,0xb6,0x3a,0xf5,0xd5,0xc4,0x89, +0x89,0x95,0x4f,0xa3,0xb1,0xa4,0x4c,0xf4,0xbc,0x8b,0xcf,0x23, +0xeb,0x6c,0x3c,0x5d,0x14,0x1d,0x3b,0x57,0x18,0x9a,0xa1,0xc4, +0x29,0xb7,0xb7,0x71,0xd4,0x6f,0x6e,0x09,0x9b,0x3e,0x05,0x78, +0x27,0xc9,0x05,0x43,0xd5,0x86,0x7f,0xba,0x51,0x37,0xdc,0xf5, +0xc8,0xe1,0x11,0x98,0x19,0xf6,0x30,0xb2,0x75,0x73,0x6f,0xd0, +0x20,0xa3,0xb8,0x69,0x1b,0x8d,0x27,0xb9,0x20,0x21,0xb7,0xb0, +0x0e,0x55,0x7f,0x83,0x53,0x79,0x1d,0x72,0x87,0xa7,0x0b,0xaf, +0xeb,0x10,0x24,0xaf,0x59,0x92,0xa1,0x4a,0x0c,0xbe,0xa0,0x63, +0xf2,0x2a,0xc9,0x5c,0xf8,0x15,0xa8,0x30,0x20,0x2c,0x6c,0xb0, +0x2a,0x30,0x39,0x60,0x23,0xf2,0xc8,0x24,0x24,0xcb,0xda,0x73, +0xed,0x8e,0xa5,0x5d,0x14,0xf5,0xad,0x80,0x0f,0xf0,0x06,0xe9, +0xb4,0xe6,0x48,0x04,0xb2,0x0a,0x40,0x1f,0x6d,0x16,0x61,0x51, +0x8e,0x5c,0x28,0xe1,0x88,0x88,0x73,0xb8,0x5e,0x5b,0xa0,0xf3, +0x40,0x22,0x18,0x69,0x10,0xb4,0xb4,0x1d,0x4c,0x95,0x80,0x32, +0x74,0x7d,0xd0,0x9d,0xaf,0x3f,0xaf,0x18,0x7a,0x0a,0xd1,0x42, +0xfb,0x36,0x47,0x98,0xf8,0xa4,0x7e,0x19,0x5e,0xd7,0xc1,0x48, +0x02,0xb9,0xb0,0x4a,0x82,0x56,0x11,0xc1,0x90,0x33,0xa5,0x61, +0xa5,0xd3,0xe3,0x81,0x9e,0x92,0xb0,0x32,0xe2,0x51,0xa8,0x43, +0xbe,0x19,0x7f,0x3b,0xde,0xb1,0x3c,0x58,0x5b,0xe6,0xb9,0x28, +0x74,0x79,0x8d,0x3c,0xb0,0x25,0xe6,0x93,0x47,0x8f,0xa6,0x31, +0xa8,0x2c,0x77,0x49,0x3e,0x5f,0x19,0xc5,0xc8,0x52,0x26,0xe0, +0x11,0x17,0x22,0x2b,0x6f,0x51,0xc6,0x78,0x2d,0xfa,0x87,0x3c, +0x49,0xfe,0x76,0xbd,0xbc,0x7e,0x4e,0xa2,0x64,0x43,0x2e,0x25, +0x21,0x42,0x75,0xe9,0xca,0x5c,0x5f,0xb7,0x88,0xaa,0xf3,0x72, +0x8a,0xe1,0x72,0x7f,0x41,0xe3,0xcf,0xea,0x6d,0xc9,0xe7,0xae, +0xb5,0x7c,0x8d,0x45,0x50,0x85,0x73,0x09,0x5f,0xfa,0x57,0xd7, +0x88,0x88,0x6a,0x63,0x8b,0x47,0xd5,0xb7,0xdb,0xb0,0x3e,0x9e, +0x0f,0x81,0xf0,0xa2,0x52,0xa3,0x41,0x7e,0xc2,0x40,0xa9,0xce, +0x2b,0x94,0x73,0xf7,0x07,0x56,0x46,0x89,0xc4,0x1b,0xd9,0xae, +0x01,0x10,0x5e,0x1f,0x71,0xd3,0x0f,0x17,0xef,0x3f,0xa5,0xf5, +0x80,0xca,0x02,0xec,0x51,0x31,0x20,0xb1,0x43,0x9d,0xe1,0x89, +0x75,0x7c,0x40,0xcc,0x10,0xe1,0x20,0xc2,0x03,0x10,0x2d,0xe4, +0x5a,0xa5,0x18,0xbe,0x9d,0x86,0x42,0xd4,0x35,0xd0,0x8b,0x74, +0x8e,0xc7,0xac,0x00,0x1a,0x55,0x69,0x66,0x8e,0x85,0xfb,0x88, +0x6c,0x76,0x9d,0x47,0xc8,0x68,0xd9,0xe6,0x77,0xd1,0x8b,0xb8, +0x32,0xa2,0x76,0x52,0xa8,0x84,0x14,0x32,0x37,0xee,0x10,0xbd, +0xfb,0xc4,0x33,0x45,0x54,0x36,0x19,0x06,0xda,0x76,0xed,0x88, +0xa5,0x90,0xb8,0x1b,0x59,0x0f,0xb4,0x6e,0x5c,0x96,0x6a,0x60, +0xb4,0x4d,0xbb,0x5b,0xeb,0x43,0xbb,0xbd,0x5e,0x45,0x63,0x7a, +0x0d,0x9b,0x7a,0xeb,0x30,0x5e,0x4b,0xfb,0xc3,0xd7,0x30,0x66, +0x77,0xfe,0xb2,0x60,0x57,0xa7,0x03,0x56,0x3d,0xb5,0x0e,0x85, +0x72,0x1b,0xa7,0x42,0x3d,0x39,0x9a,0xea,0x78,0x90,0x65,0xad, +0x3a,0x4b,0xbb,0x12,0x4f,0x2b,0xbb,0x47,0x35,0x56,0xb1,0xb4, +0x53,0x99,0xd7,0x20,0x4c,0xc8,0x28,0xf6,0x95,0x2d,0x82,0x2a, +0x5e,0x85,0xe6,0x99,0x35,0xdb,0x59,0x11,0xd7,0x67,0xeb,0x2e, +0x30,0xca,0x9e,0xd1,0xb6,0xb2,0x9d,0x7f,0xc7,0xc3,0xb2,0x14, +0xad,0xa3,0x39,0x47,0x55,0xac,0xfd,0xed,0x2b,0x9d,0xf8,0x78, +0x30,0xfb,0x05,0xc4,0x7c,0x90,0x62,0xcf,0xdf,0x3f,0x2b,0x37, +0x9e,0x44,0x65,0x05,0x76,0xcc,0x92,0xc3,0x4f,0x11,0xb5,0xec, +0x8a,0xd4,0x37,0xab,0x72,0x2d,0xc9,0x45,0x28,0x64,0x79,0xc4, +0x6d,0x79,0x2d,0x13,0x88,0xae,0x2a,0x7b,0x4b,0x6d,0x5e,0xdb, +0x29,0xb1,0x80,0x90,0x98,0x2d,0xe2,0xab,0xc6,0x46,0xad,0x8a, +0xe4,0x24,0x31,0x56,0x82,0x2e,0x58,0x12,0x82,0x72,0x76,0xc1, +0x8a,0xe0,0x1d,0xf4,0xa3,0x90,0x7c,0x33,0xbd,0x78,0x73,0xb3, +0x6a,0x2c,0xee,0x40,0x70,0xb3,0x43,0xee,0x76,0x65,0x39,0x2c, +0x66,0xfd,0xe7,0x0f,0xff,0x3d,0x45,0xb1,0x2d,0x68,0xf5,0xe0, +0x45,0x40,0x66,0xe4,0xb4,0xfb,0x9f,0x8b,0xea,0xd7,0x9b,0xd2, +0x9e,0x7d,0x25,0x7c,0x57,0x00,0x10,0xb0,0xe1,0x1f,0x30,0x6c, +0xd4,0x1a,0x97,0xce,0x35,0xc6,0x1d,0xea,0x0f,0x48,0x8f,0x64, +0x01,0x45,0x63,0xa7,0x08,0xe8,0xc2,0x31,0x2b,0x77,0xb7,0xaa, +0x18,0x77,0xc7,0xb8,0xa2,0x31,0xe3,0xd9,0x90,0x31,0x94,0xda, +0x53,0x30,0xdf,0x7c,0x20,0xd3,0x41,0x66,0xe0,0x21,0x80,0x19, +0xb3,0x3e,0x99,0xad,0x5c,0x22,0x22,0x05,0x79,0x3c,0xdb,0xb1, +0x91,0x16,0xce,0x76,0xd7,0xbc,0xf2,0x1b,0xb2,0xee,0xc8,0x1f, +0xa2,0xe0,0xa8,0x08,0x30,0xe7,0x8f,0xa5,0xd7,0x6b,0x8e,0x66, +0x47,0xe4,0x0b,0xb5,0x79,0x3c,0x1d,0x2b,0x79,0xca,0xfd,0xa3, +0x4f,0x40,0x18,0xfb,0x91,0x08,0xe6,0xa6,0xe5,0x28,0x45,0xd7, +0x33,0x7e,0xe9,0x5e,0x51,0x19,0x69,0x1c,0x1f,0xe2,0xf4,0xb3, +0x9a,0x48,0xf1,0x96,0x8f,0x20,0x44,0xb1,0xe1,0xce,0x99,0x8e, +0xb2,0x03,0xec,0x04,0xe2,0xa4,0xdf,0xcb,0xe9,0x09,0x5f,0x5d, +0x90,0x90,0xe6,0x9f,0x1a,0xd1,0x94,0x50,0x9b,0xa2,0xcf,0x34, +0xf8,0x06,0xa1,0x4d,0x02,0x05,0x0b,0xda,0x41,0x64,0xf5,0x56, +0x95,0x15,0xb6,0x5c,0x5a,0xb6,0x6b,0xbb,0x80,0x72,0xd9,0xc6, +0x9b,0x26,0x7b,0xfa,0xd0,0x5c,0xcb,0x0b,0x42,0x64,0xf7,0x5a, +0x38,0x0a,0x70,0x6b,0x51,0x04,0xde,0xd0,0xe4,0x3e,0xc3,0x78, +0x81,0x0a,0xa4,0xf2,0xbf,0x0f,0x93,0xc4,0xbe,0xb5,0xc0,0xcb, +0x29,0x73,0xfc,0x61,0xa7,0x3b,0x0a,0xcd,0xe3,0x28,0x38,0x10, +0xfa,0x8c,0x2b,0xcb,0xfd,0x2f,0x8f,0xd1,0xee,0x3b,0xc5,0xe9, +0x04,0xd6,0xca,0xd8,0x9a,0x9a,0x48,0x2d,0x61,0x5f,0xb7,0xb6, +0x5b,0xfb,0x1e,0x73,0x0d,0x8a,0xdc,0xa2,0xcd,0x0d,0xc3,0xbb, +0x72,0xff,0xf8,0x1e,0xdc,0xc8,0x1d,0xb0,0xf5,0xa8,0xec,0xa5, +0x02,0xfe,0xe1,0xab,0x05,0xe4,0x66,0xcc,0x53,0x5b,0xee,0x6a, +0x49,0xb5,0xcb,0xb3,0x8b,0xea,0x30,0x56,0x27,0x8b,0x4c,0x4d, +0xbe,0xcf,0xbc,0xd3,0xa9,0x8a,0xdc,0xfe,0xb0,0x9a,0x5a,0x13, +0x88,0xb4,0xa7,0x0f,0xba,0xdd,0xe4,0xe9,0x63,0x54,0x6d,0x72, +0xd5,0xc2,0x4a,0xb1,0x99,0x98,0x75,0xff,0xa5,0x28,0x1a,0x76, +0x18,0x40,0x61,0x91,0xb5,0x45,0xab,0xe0,0x30,0x50,0xf9,0xe7, +0xce,0xd9,0x30,0x14,0x8e,0x63,0x3f,0x7d,0xcc,0x7d,0xaa,0x88, +0x12,0xa8,0xf4,0x6f,0x16,0x72,0xc7,0x7c,0xa9,0xbf,0xec,0x2d, +0x66,0x06,0x05,0x4d,0x47,0x3f,0x22,0x1b,0x0b,0xab,0x4e,0xb6, +0xf7,0xa1,0x3c,0x72,0xdd,0xfb,0x59,0xf3,0xce,0xba,0x64,0x45, +0x9c,0x7f,0xf4,0x22,0xf0,0x0e,0xc4,0x1c,0x1f,0x9d,0xe4,0x01, +0x06,0x71,0xe5,0xe3,0x41,0xcc,0x47,0x1e,0x3e,0x15,0x1c,0x21, +0x84,0x42,0xdb,0xd1,0x60,0xc5,0x10,0x91,0xf4,0xa6,0x88,0x06, +0x87,0x10,0xb1,0x51,0x94,0xe3,0x59,0x41,0x01,0x7a,0x3e,0x0d, +0x41,0xe1,0x75,0x78,0x78,0x20,0x39,0x7c,0x29,0x2e,0xcf,0x03, +0x2b,0x96,0x56,0x22,0x55,0x0d,0x5c,0xd0,0xee,0xc2,0xbb,0xa4, +0x6b,0x13,0x99,0x9c,0xc5,0xcf,0xc7,0x19,0x5c,0x90,0xf8,0xe1, +0x9b,0x7b,0xf0,0x66,0xb3,0x73,0xea,0xf4,0x55,0xe5,0xaf,0x40, +0xbe,0x4b,0x09,0xa5,0x96,0x80,0x08,0xed,0xf2,0xda,0x63,0x14, +0xe0,0xae,0xdd,0x7d,0x3d,0x9b,0x5c,0xd8,0xa4,0xd9,0xdd,0x8e, +0x17,0xa4,0xa7,0x5e,0xc5,0x99,0xc1,0x97,0xb6,0xf0,0x65,0x19, +0x43,0x6f,0x28,0x1a,0x01,0x0b,0x1f,0xd6,0x7f,0x7c,0x83,0x58, +0x9a,0x6e,0x9a,0xbc,0xce,0x78,0xf6,0x53,0xc6,0x8f,0xa0,0x81, +0xbb,0x76,0x0c,0xb4,0x1a,0x8e,0xc8,0x2f,0xe6,0x6f,0xb5,0x8d, +0xac,0xa5,0x36,0x3b,0x06,0xe3,0xb6,0xfe,0x84,0x6f,0x50,0x8a, +0x84,0xb3,0x2c,0x92,0x9a,0x98,0x54,0xd2,0x3b,0x46,0x2f,0xdf, +0xc7,0x08,0x89,0x6f,0xd2,0x55,0xb0,0x19,0xa0,0x0d,0xd6,0x61, +0xc4,0x62,0xa5,0xb8,0xdb,0xa9,0xe0,0x03,0x6c,0x10,0x95,0xfc, +0xf0,0x87,0x5e,0xde,0x64,0xa5,0x79,0x60,0xf1,0x67,0xdc,0xed, +0x4f,0x94,0x21,0x98,0x58,0xb7,0xe5,0xf9,0xee,0xe3,0x6a,0xf4, +0xe5,0x82,0x0e,0xfa,0x46,0x01,0xc5,0xe0,0xeb,0xaf,0xbf,0x18, +0x98,0xe9,0x7a,0xcd,0x3a,0xbc,0x55,0x01,0x3f,0xfb,0xf0,0xd7, +0x20,0xf5,0xf0,0x07,0xfc,0x60,0xc2,0xa1,0x4f,0xfa,0x65,0xe4, +0x08,0x4a,0x29,0xf2,0x7b,0xbb,0xdd,0xf3,0x04,0xb9,0xbd,0x77, +0x42,0x73,0xce,0x06,0xa7,0x97,0x80,0x11,0xb8,0x27,0xf3,0xf5, +0x82,0x92,0x6f,0x35,0x54,0x7b,0x83,0x9d,0xa1,0x80,0xe9,0x1c, +0x97,0x36,0x19,0x9f,0x9d,0xbd,0x7b,0x70,0x72,0x76,0xb6,0x39, +0x3b,0x5b,0x9f,0x9d,0x2d,0xce,0xce,0x2e,0x27,0x89,0xc1,0x80, +0x34,0xd3,0x0b,0x20,0x83,0x27,0xf2,0xa6,0x4e,0x91,0xe0,0xab, +0x28,0x67,0x18,0x55,0x16,0x2a,0xdc,0x1e,0x4f,0xb6,0xe3,0x17, +0x50,0xf1,0xe4,0xe4,0x18,0xfe,0x9d,0x02,0x7b,0xed,0x25,0xa6, +0x82,0x3d,0x7e,0x83,0x4f,0xa1,0xad,0x8b,0x46,0x7d,0xbb,0xd0, +0x93,0x5b,0x7c,0xa6,0xea,0x4f,0x18,0x58,0x1c,0xb6,0x3f,0xba, +0x46,0x85,0xb6,0xc7,0xf7,0x5e,0x7c,0xb6,0xed,0xfc,0x73,0x32, +0x2a,0xb2,0xc4,0xb8,0x87,0xdc,0x8b,0xe4,0xec,0x6c,0x9c,0xf4, +0x1c,0xb8,0xbd,0xe4,0x5e,0x9a,0xf4,0x1a,0xad,0xf7,0x92,0x2c, +0x2e,0x35,0xca,0x93,0x9e,0xed,0xa1,0x91,0x97,0x8e,0x3f,0x3f, +0x4b,0x26,0x59,0xea,0x06,0xf5,0x02,0xff,0x4e,0xb2,0x7b,0xa3, +0xec,0xec,0xec,0x8b,0x2d,0x74,0xe2,0x46,0x03,0xad,0x6f,0xe1, +0xff,0x51,0x0f,0x50,0x3a,0x31,0x2b,0x32,0x3b,0x00,0x38,0xf3, +0x3d,0x60,0x51,0x07,0x69,0xfa,0xc1,0xfe,0xa2,0x8c,0x34,0x03, +0x2c,0x4f,0x26,0xdb,0xa4,0xe7,0xbd,0x6a,0xaf,0x28,0xfc,0xc2, +0xfc,0x1b,0xa0,0x3b,0xbb,0x97,0x6d,0xfb,0xf7,0xa0,0x36,0x82, +0x66,0xe8,0xa4,0x59,0xf8,0xc1,0x82,0x5f,0x84,0xf0,0xf6,0xa8, +0x8f,0x17,0xb6,0x63,0xed,0x10,0x9a,0x89,0x0a,0x7e,0x06,0x13, +0x84,0x6f,0x28,0xf1,0xed,0xfa,0xa1,0x36,0xef,0x99,0xe8,0x5b, +0x6a,0xe9,0xfd,0xf8,0xc1,0xba,0x69,0x93,0xf0,0x4e,0x7b,0xff, +0x9c,0xc4,0x68,0xc6,0x26,0x19,0xcb,0x7e,0x6b,0x82,0x77,0xc8, +0xf3,0x88,0x2e,0xea,0xcd,0x9f,0xc0,0xcf,0xf4,0x00,0x8e,0x3c, +0xb0,0xb8,0x7b,0xfa,0x38,0x0f,0x0a,0xff,0x69,0x1f,0x51,0x65, +0xe6,0xd1,0xf7,0x0f,0x9f,0x3d,0x0b,0x4b,0x63,0xb0,0xe5,0xbd, +0xe5,0x7f,0x7c,0xf8,0xc3,0x93,0xb8,0x38,0xc5,0x9a,0x28,0x88, +0x04,0x46,0x7b,0x6a,0x72,0x26,0xd2,0x54,0x66,0xe0,0xe8,0x17, +0xb6,0xd0,0x56,0x27,0x5a,0x50,0xf7,0x30,0xe8,0x33,0xf6,0xff, +0xf0,0xf9,0xf3,0x5f,0xf2,0x08,0x13,0x8e,0x86,0x32,0xf3,0xf3, +0xb3,0x27,0xff,0xf9,0xf8,0xa7,0xb8,0x84,0xc5,0xe7,0xa3,0xef, +0x9e,0x7e,0x1f,0x21,0x27,0xa7,0xe8,0xb5,0x5b,0xba,0x05,0xde, +0x02,0x5f,0xdd,0x6c,0x17,0x9b,0x2b,0xfc,0xef,0x18,0x3f,0xb2, +0xe3,0xf4,0xe2,0xaa,0x9a,0xcf,0xb6,0xcb,0xcb,0x63,0xda,0x97, +0x99,0xe2,0xe3,0xc9,0x46,0xbb,0xef,0xed,0x72,0x36,0x03,0x2a, +0x1c,0xf7,0x80,0x7b,0x64,0xe9,0xd9,0xd9,0xec,0x5e,0xb6,0x68, +0x2c,0x2b,0x5a,0x9c,0x54,0x22,0xce,0x80,0x0a,0xf8,0x24,0x53, +0x73,0x1d,0xd2,0x02,0x48,0x2a,0x18,0x7c,0xf0,0x0a,0xcb,0x21, +0xe2,0x1b,0x23,0xa9,0x6d,0x73,0x07,0x56,0xf9,0xeb,0xf6,0x15, +0x8c,0x8e,0xc7,0xe6,0x86,0xba,0x67,0x34,0x90,0x0a,0x7c,0x6f, +0x96,0x8d,0x68,0x10,0x6d,0x00,0xa5,0xa3,0x62,0xfc,0x02,0x46, +0xf1,0x99,0x80,0xb6,0xb3,0xdb,0x78,0x71,0x7f,0x4c,0x74,0xbf, +0x01,0xaa,0x5f,0x00,0xb3,0xbd,0x37,0x06,0x50,0xee,0x1b,0xb1, +0x5a,0x2f,0xee,0xbf,0x18,0xbf,0xb8,0x9b,0xf4,0xce,0xee,0xce, +0xea,0x7b,0x48,0x38,0x98,0x08,0xd2,0xc3,0xac,0x84,0x22,0xd6, +0xe4,0x97,0x1f,0xb7,0xfa,0x13,0x2c,0x21,0xe0,0xc4,0x80,0x94, +0xf4,0xec,0x16,0xfe,0x05,0xb2,0x94,0x04,0x7a,0xdd,0x8a,0xcc, +0xab,0x0e,0x3d,0xb7,0x25,0xd1,0x95,0xf8,0xd5,0xad,0xab,0x72, +0x8a,0xb7,0x5c,0xf7,0x5f,0x5c,0x9d,0xcd,0x28,0x81,0x85,0xb0, +0xe2,0xfe,0xe7,0xdb,0xb3,0xb3,0xfb,0xaf,0xcc,0xda,0x12,0xd1, +0x7f,0xdc,0x2c,0x91,0x35,0xdf,0x3f,0x2b,0xa2,0x81,0xa4,0xe3, +0x17,0x9f,0x27,0xc0,0xb5,0xee,0x65,0x51,0xc6,0xd9,0x04,0x1b, +0x50,0xb9,0x0e,0x6a,0x9e,0x01,0xa0,0xb3,0xe9,0xf1,0xe5,0xc3, +0xe3,0x6f,0x27,0x77,0x03,0xf3,0xe7,0x5d,0x58,0x61,0xb4,0xed, +0x67,0x50,0xc3,0x4a,0x82,0x4e,0xc9,0xf3,0xd2,0x70,0x8a,0x9c, +0x37,0xae,0xaa,0x57,0x57,0x45,0x72,0xf2,0x2e,0xe9,0x49,0xf2, +0xf1,0x9f,0xbf,0xfa,0xea,0x8b,0x3f,0xeb,0x49,0x06,0xb3,0xf1, +0xed,0x07,0xf8,0x33,0x92,0x02,0xf9,0xc9,0x29,0x7d,0x3e,0x23, +0x1b,0x6e,0x8a,0x39,0xf6,0x48,0x82,0xaf,0xa5,0x98,0xd1,0xa3, +0x06,0xb2,0xbc,0x2d,0xff,0xab,0xaf,0x1e,0xfc,0xe5,0xcf,0x5b, +0x2c,0x75,0x7a,0x3a,0x38,0x31,0x5f,0xfd,0xf9,0x8b,0x07,0x27, +0xdb,0xc1,0xc9,0x83,0x2f,0xba,0x98,0x96,0xed,0xc8,0xa8,0xde, +0x13,0x09,0x7d,0xe9,0x25,0x8e,0x31,0xd6,0xa7,0x45,0x83,0x32, +0x6b,0x0d,0x52,0x23,0x0a,0xb5,0x7a,0x18,0x71,0x76,0xf8,0x2c, +0x24,0xb8,0xf7,0x0f,0xa3,0xeb,0x61,0x35,0x53,0x1f,0x4f,0xd8, +0xba,0x80,0xb7,0x7d,0x52,0x04,0xb4,0xe8,0x3e,0xa2,0xc3,0xdd, +0x8e,0xe4,0xbf,0x42,0x8e,0xc6,0xfc,0xd1,0x62,0xcb,0x17,0x41, +0x4d,0x02,0x85,0x3e,0x9e,0x1e,0x99,0xe1,0x53,0xa5,0x78,0x98, +0xf6,0xf5,0xe1,0x30,0x79,0x94,0x7c,0xf7,0xfc,0x87,0xef,0x13, +0x4f,0x91,0xa4,0x68,0x51,0x93,0xc1,0xbc,0x33,0xd8,0xf9,0x72, +0xa7,0x42,0xea,0x27,0x59,0x78,0xb1,0x92,0x85,0x97,0x2c,0x8a, +0x47,0xf8,0x4f,0x0c,0x21,0xa6,0x05,0x5e,0x02,0x1e,0x74,0x81, +0x4e,0xeb,0x2f,0x1c,0xe5,0xca,0x3b,0x1f,0xce,0x9a,0x33,0x38, +0x4a,0x35,0x01,0xcb,0xaa,0x38,0x5c,0xb4,0x94,0x0c,0x45,0xe4, +0x82,0x90,0x8d,0x95,0x33,0x15,0x61,0xfb,0x1b,0x36,0x1c,0xff, +0x71,0xf9,0x08,0xb6,0x5c,0x32,0xd7,0x51,0x87,0x0e,0x1d,0x22, +0x7a,0x68,0xdc,0x59,0x43,0xd3,0xb7,0x78,0x0e,0x29,0x17,0xb3, +0x47,0x48,0x44,0x29,0x45,0x73,0x63,0x81,0x95,0xab,0xa7,0x49, +0x82,0xce,0xea,0x58,0xae,0xd5,0x3e,0x1d,0x77,0x55,0x91,0x1c, +0x77,0x0e,0x08,0x4f,0xec,0x6a,0xed,0x1d,0x9b,0xab,0x16,0x80, +0x51,0x9c,0xbb,0x22,0xf9,0x9a,0xb9,0xc9,0xe9,0xd7,0xf7,0xe5, +0x47,0x32,0x54,0xff,0x55,0x95,0x91,0xb1,0x06,0x72,0x51,0x02, +0x33,0xb2,0xc4,0x56,0x3d,0x87,0x7d,0x6a,0xdc,0x3e,0xa4,0xdd, +0xd1,0x90,0xe5,0xce,0xd1,0x82,0xad,0x3f,0x1d,0xa8,0xa1,0xc5, +0xfc,0x07,0x91,0xe5,0x43,0x0d,0xdf,0x47,0x74,0x3a,0x28,0x3e, +0xbf,0xaa,0x66,0x20,0x22,0x1c,0x95,0x9f,0xc3,0x18,0x20,0xf9, +0xb4,0x99,0xa7,0x39,0x89,0x69,0xe0,0xd2,0xb3,0xd7,0xdf,0x9f, +0x97,0x26,0xa5,0xc5,0xf4,0x28,0x0d,0xd1,0x71,0x61,0xc1,0x87, +0x42,0xe6,0x01,0x19,0xe6,0x7c,0x4c,0x3b,0x19,0x2e,0x8d,0x08, +0x15,0xfb,0xb1,0x40,0xc3,0xb7,0x76,0xfa,0xbd,0x13,0x13,0xe3, +0x63,0xca,0xf6,0xba,0x9f,0x03,0x0b,0x95,0x32,0x09,0x8e,0x7a, +0xca,0xd8,0x68,0xcd,0x13,0x8c,0x30,0xc1,0x43,0x63,0xd8,0xed, +0x37,0x14,0xfd,0x00,0xfb,0xb4,0x19,0xb4,0x8b,0xd2,0x58,0x39, +0x66,0xfa,0x0a,0x11,0x8b,0xb4,0x1a,0x8c,0x51,0x50,0xd8,0x96, +0x9c,0x4a,0xaf,0x3a,0x72,0x40,0xd1,0x83,0xde,0xc1,0x92,0xbd, +0x13,0x2d,0x6b,0xef,0x75,0x1c,0x9a,0x9e,0x02,0xbb,0xdd,0x10, +0xa6,0x3a,0x61,0x23,0xe4,0x51,0xa2,0x7d,0x59,0xf0,0xd9,0x4c, +0x5d,0x56,0x18,0xa0,0xd2,0x20,0xfc,0x80,0x78,0xd2,0x6f,0xd0, +0xa3,0xb0,0xfc,0xca,0xc0,0xa7,0x10,0xdf,0xf4,0x08,0x9f,0xb6, +0x2d,0x3e,0xff,0x13,0xa3,0x98,0xc9,0xca,0xe1,0x89,0xfd,0x5d, +0x65,0xe1,0xb8,0xe4,0x60,0xe5,0xe0,0x7b,0x45,0x41,0x34,0xb3, +0xe4,0x4f,0x89,0xd0,0xce,0x9e,0x1a,0x29,0xbf,0xa7,0x9b,0xed, +0xb2,0xd1,0xdd,0x2e,0xbf,0xc3,0x8f,0x8f,0x7e,0x27,0x57,0x9e, +0xe2,0x7d,0x90,0xf1,0x53,0x55,0x1f,0x5f,0x0f,0x4b,0xa3,0x55, +0xbf,0x69,0x9b,0x80,0x51,0xea,0x6b,0xcd,0xbc,0x1d,0xce,0xaa, +0x42,0x58,0x7b,0xc3,0xc8,0x68,0xf7,0x53,0x6a,0xe0,0x21,0x54, +0xe1,0xc9,0xdd,0xd3,0x3e,0x27,0xa7,0x6a,0x66,0x77,0x44,0xd8, +0xcf,0x7c,0x27,0xa7,0xd1,0xf8,0x7a,0x92,0x8f,0x27,0x00,0xb9, +0x77,0x35,0x19,0x41,0x69,0x83,0x21,0xac,0x9f,0xce,0x8a,0x6a, +0x76,0x50,0x89,0x35,0x6c,0x37,0x94,0xdf,0x8f,0x39,0xf4,0x86, +0xe0,0x98,0x06,0xd0,0xfa,0x6e,0x97,0xe5,0xff,0x9f,0x42,0xd6, +0x48,0x3c,0xab,0xe8,0x35,0x2e,0xee,0xb2,0xf9,0x6e,0x79,0xa3, +0xbb,0x96,0xa7,0xcd,0x69,0x9c,0x7d,0x7d,0x66,0xbc,0x9a,0x11, +0xde,0xdd,0x1b,0x9a,0xff,0xf3,0x67,0x80,0x5c,0x15,0x00,0x92, +0xc2,0x8b,0x1a,0xf8,0xe1,0x91,0xb4,0x96,0xe2,0xc1,0x58,0x1b, +0x38,0x8a,0x59,0x44,0x02,0x88,0x0e,0xcf,0x4e,0xa6,0x09,0x2e, +0xf8,0x8a,0xbd,0xfb,0xbf,0xb3,0x5b,0x84,0x2c,0x37,0xcf,0xe1, +0x83,0xa5,0x87,0x7c,0xd0,0x22,0xc0,0x47,0x07,0xfd,0xd5,0xa0, +0x0f,0xeb,0x78,0x9b,0xb7,0xf7,0x6c,0xa5,0xce,0x8d,0x5a,0x14, +0x9d,0x58,0x01,0xf4,0x83,0x6d,0x93,0xdf,0xfb,0x3d,0x8a,0x3c, +0x04,0x5f,0x9e,0xdd,0xad,0x9a,0xea,0xb0,0xd4,0xda,0x30,0x88, +0x05,0x7e,0xa8,0x86,0x48,0x81,0x10,0x0b,0xa9,0xbb,0x48,0x9e, +0xf5,0x10,0x8b,0x27,0xf0,0xa2,0xb1,0x3d,0x76,0xbb,0xbf,0x1b, +0xa5,0x1f,0x8d,0x4f,0x1a,0x30,0xf9,0xae,0xd8,0xc0,0x2f,0x0e, +0x2c,0x52,0x24,0x14,0xfb,0x7c,0xfe,0x2c,0x70,0x56,0x2c,0xf8, +0x24,0x10,0x9d,0x2c,0x14,0xc2,0xe9,0xbc,0x5a,0x69,0xa1,0x8f, +0x18,0xac,0xfc,0x03,0xde,0x81,0x16,0x06,0x3c,0xb6,0xfa,0x6a, +0x5c,0x9c,0x79,0xab,0xe1,0x2d,0xc6,0x09,0xbf,0xaf,0x9c,0x4c, +0x8c,0xef,0x3e,0x58,0x58,0xa7,0x6c,0xdc,0x66,0x63,0xef,0x31, +0x32,0x2d,0xfd,0x24,0xf9,0x72,0xb9,0xf2,0x6e,0xa1,0xca,0x59, +0xf1,0x39,0x6e,0x9e,0x9c,0xe8,0x09,0x9f,0xb4,0x95,0x36,0x5c, +0xd5,0x92,0xb1,0x56,0x9b,0x58,0x01,0x6a,0xbb,0x75,0xde,0x8d, +0x44,0x5b,0x6d,0x4a,0xc6,0x51,0x7e,0xc1,0x71,0xad,0xb6,0xfa, +0x76,0xc2,0xb6,0xaa,0xaf,0xa7,0x2b,0x6b,0x65,0xb9,0x05,0x81, +0x7b,0x46,0x8a,0x11,0xed,0x61,0xcb,0x1e,0xe2,0x49,0xb6,0x07, +0x14,0x6d,0xf1,0x00,0x20,0xae,0x08,0xb0,0x8a,0x8f,0x42,0x12, +0x1d,0xec,0x59,0xe8,0x56,0x99,0xf5,0xa8,0x02,0x14,0xdd,0xdf, +0x8f,0x91,0xea,0x05,0xe4,0x3b,0x74,0x74,0xbb,0x31,0x14,0xa8, +0x8c,0x9d,0x14,0x4d,0x8c,0x9c,0x25,0x67,0xc9,0xf6,0xf3,0xcf, +0x0f,0x8c,0xb0,0x5c,0x10,0xaa,0x0e,0x8d,0x50,0x8b,0x98,0x24, +0x57,0xcc,0xee,0x6d,0xef,0x9e,0xc9,0xdf,0xa1,0xfe,0x2f,0x6a, +0xc3,0xf4,0xef,0xe5,0x09,0xc5,0xe3,0xb2,0x64,0x27,0xd7,0x0b, +0xd6,0x0f,0xd2,0x92,0xa0,0x64,0x14,0x2a,0xd5,0x45,0x05,0x69, +0x7d,0x70,0xc6,0xf2,0xb7,0x1f,0xf6,0xe5,0xdd,0x96,0xe7,0x6f, +0xaa,0xcd,0xde,0xec,0xe5,0xde,0x9c,0xeb,0x3a,0xca,0x42,0xea, +0x6f,0x9d,0x57,0x1d,0x08,0xe0,0x04,0x16,0xe7,0x82,0x08,0x8a, +0xea,0x8a,0x11,0x80,0xbc,0x96,0x83,0xf2,0x35,0x7b,0xf0,0x9b, +0x66,0xfa,0xb8,0xee,0xe0,0xcc,0x7a,0x38,0x93,0xde,0x05,0x6f, +0x9d,0xc2,0x2a,0xac,0x91,0xbe,0xdc,0x42,0xf6,0x34,0x65,0x0e, +0xd7,0xec,0x43,0xba,0xc5,0x93,0x64,0xc8,0x03,0x1a,0xa5,0xb5, +0x17,0xaf,0x86,0x5e,0xf3,0x04,0xbc,0x80,0x50,0xa2,0x39,0x99, +0xc3,0x12,0x5e,0xbd,0x80,0xfc,0xa5,0x07,0x75,0x75,0xea,0x73, +0x7b,0xa0,0x75,0xe1,0x9c,0x62,0xb4,0x26,0xf2,0x26,0x9e,0x3a, +0x23,0xb8,0x69,0x7c,0xde,0xce,0xa1,0xfc,0xcd,0x0a,0x7d,0x3d, +0xcf,0x3d,0xc1,0xce,0x7a,0x59,0xa2,0xef,0xe5,0xcd,0x6a,0xbb, +0xed,0xa4,0x1d,0xfa,0x8b,0x3e,0xfa,0xf0,0xc3,0x73,0xc3,0xe8, +0xa4,0x53,0x7e,0x3f,0x53,0x60,0x1d,0x85,0x9f,0x29,0x94,0xce, +0xf2,0xe9,0x3e,0xb8,0xbb,0xdd,0xc1,0x9f,0xbb,0x7b,0x73,0xa9, +0x72,0xe6,0xbb,0xb5,0xd0,0xe8,0xd0,0x0a,0x86,0x4d,0x2f,0xce, +0x8b,0x00,0x6a,0xbc,0xd0,0x39,0xc7,0x01,0x5b,0x7f,0x88,0xa1, +0x8d,0xcf,0xe0,0x2e,0xd1,0x8a,0x4f,0xc7,0xa5,0x94,0x0c,0xd0, +0x32,0x4a,0xfd,0x8b,0x40,0x74,0xf2,0x3e,0x01,0xf9,0x53,0x4a, +0x02,0x5c,0x7b,0x47,0x7c,0x00,0x19,0x07,0x30,0x91,0x65,0xa3, +0x41,0x57,0x72,0xb7,0xdb,0x69,0x10,0xf3,0x71,0x40,0x01,0x52, +0xbc,0x24,0x6f,0xc2,0x59,0x3d,0xc3,0x97,0xbd,0x34,0x21,0xbe, +0x52,0xc7,0x4c,0xb3,0xd1,0xf1,0x20,0x3f,0x3f,0x5c,0xe8,0x3c, +0x43,0x4b,0xd4,0xfc,0x4b,0xed,0x1e,0xab,0x0c,0xf6,0xcf,0x29, +0x65,0xc7,0x73,0x26,0x4e,0xc5,0x24,0x0b,0x4d,0x81,0xe0,0x7c, +0x68,0x99,0x02,0xfd,0x04,0xb4,0xe5,0x9f,0x4e,0xcc,0x39,0xfc, +0x39,0x27,0xc7,0xc9,0xa9,0xe7,0x04,0x7c,0xd4,0xc0,0x3b,0x59, +0x46,0x4c,0x89,0x4e,0x91,0x62,0xbc,0x59,0x82,0x51,0x79,0x03, +0x84,0x61,0x40,0x29,0x4a,0x20,0xb3,0x2e,0xaa,0x88,0xd0,0x10, +0x51,0x6b,0xbd,0x86,0x47,0x34,0x19,0x7b,0xa1,0x23,0xf2,0xb4, +0xe5,0x3d,0xa1,0x61,0x36,0x5d,0xf5,0x6f,0x16,0x6c,0x5c,0x80, +0xee,0xcc,0xb6,0xf4,0x79,0x6b,0xe9,0xf3,0x96,0xd2,0x43,0xb2, +0xcf,0x46,0x28,0xf0,0x2f,0x90,0xb1,0x73,0x2d,0xab,0x46,0x21, +0x38,0x58,0xc0,0x50,0x31,0x58,0x55,0x52,0xcb,0x9f,0x2c,0x1a, +0x5c,0x5b,0x3a,0x8c,0x76,0x67,0x42,0xcc,0x0d,0xcc,0x18,0x90, +0x37,0xe1,0x67,0x41,0xed,0xf2,0x70,0xea,0x92,0x59,0xb9,0x01, +0xde,0x6a,0x2b,0xd4,0x45,0x70,0xf3,0x6d,0xdf,0x57,0xc9,0xf5, +0xd7,0xce,0x88,0xe2,0x51,0x37,0x12,0x77,0x8a,0xc0,0xdb,0xf4, +0x52,0x1d,0xb5,0x54,0x48,0x13,0x43,0x1b,0xca,0x24,0xcb,0x55, +0x1b,0xc2,0x5d,0x62,0x7d,0x85,0xcd,0xd9,0x0d,0x2b,0x34,0x20, +0xc6,0xea,0xec,0xa0,0xb5,0x57,0xd3,0xba,0x37,0xc0,0x09,0xe5, +0x52,0x0b,0x45,0xc9,0x46,0x1b,0x72,0x34,0x8a,0x14,0xf8,0x26, +0x29,0x3e,0xff,0x6c,0x80,0x82,0x80,0x01,0x4e,0xb8,0x67,0x23, +0x8d,0xe4,0x47,0xdd,0xcf,0x85,0xe3,0xab,0xe8,0xa0,0x1b,0x00, +0x47,0xb0,0x44,0xd8,0xfd,0x8d,0xdf,0xa5,0x66,0x14,0x7f,0x40, +0x22,0x1e,0x87,0x1b,0x9b,0x1b,0x36,0xd2,0xef,0x1a,0x1f,0xc5, +0xd9,0xbb,0x25,0x32,0x06,0xfa,0x6e,0xf8,0x83,0x81,0x86,0x68, +0xb4,0x41,0x6f,0xa2,0x98,0x2a,0xd8,0xa1,0xf7,0x06,0x42,0xcb, +0x64,0x59,0xb3,0x02,0x7e,0x3a,0x0a,0x5b,0xb3,0x6f,0x47,0x9d, +0x9e,0xd8,0x79,0xb3,0x1b,0x9b,0x13,0xd8,0x7d,0x5b,0x71,0x21, +0x83,0xf6,0x08,0x27,0xce,0x3a,0xe5,0xe3,0xa2,0xd3,0x34,0x63, +0x9b,0x58,0x30,0x70,0x2e,0x23,0x9a,0xa1,0x73,0x07,0x21,0x17, +0xe4,0x50,0xf5,0xb1,0xff,0xbd,0xe4,0x13,0xcd,0x3a,0x1d,0x6a, +0xc8,0x2b,0x3f,0x8a,0xc4,0x61,0x52,0xe8,0xac,0x88,0x54,0x62, +0x1c,0x5a,0x3f,0x1b,0x41,0x16,0x37,0x97,0x47,0xcd,0x35,0x35, +0xce,0xa3,0xa6,0x22,0x84,0xcf,0x51,0x29,0x75,0xd0,0x7e,0xea, +0xa6,0x12,0x99,0x50,0x43,0xb3,0xae,0x04,0x10,0xb0,0x2f,0x01, +0x74,0x4e,0x46,0xf8,0x33,0x87,0x06,0xbb,0xdd,0xb7,0x18,0x53, +0x1a,0x23,0x4b,0x5e,0x56,0x70,0x9a,0xc3,0x2f,0xf6,0x57,0xa6, +0x30,0x47,0xc6,0x77,0x1d,0x76,0x68,0xbe,0xae,0x5f,0x61,0xc4, +0xd9,0xf5,0xf2,0x16,0xfd,0xdc,0x8e,0x28,0xc8,0x45,0x9a,0x3c, +0x7b,0x0f,0x33,0xf5,0xee,0x88,0xca,0x9a,0xa3,0x9b,0xc5,0xba, +0xbc,0x58,0xbe,0x42,0x2b,0xb7,0xd9,0x11,0x12,0x56,0x59,0xd7, +0x18,0xd7,0xfc,0x28,0xe9,0x61,0x75,0xdb,0xb4,0x33,0x46,0x74, +0xed,0x5b,0xbb,0x3e,0x7b,0xd8,0x9e,0x39,0x3e,0x45,0x67,0xee, +0x01,0x3d,0x5a,0x58,0x5d,0x46,0xdb,0xf4,0x3e,0xe6,0xa6,0x27, +0xf4,0x06,0x37,0xf4,0xab,0xb7,0x9e,0xc7,0x27,0x64,0x4b,0x52, +0xb2,0xdb,0x85,0x4d,0x3d,0x1e,0xa0,0x4f,0xdc,0xeb,0xc2,0x81, +0xe5,0x1c,0x00,0xa9,0x11,0x7a,0x0f,0xde,0x76,0xca,0x56,0x57, +0xae,0x30,0x3e,0x0b,0x3f,0xc8,0x1a,0x27,0x77,0xb1,0x48,0xd2, +0x1b,0x1e,0xfd,0xdc,0xa3,0xae,0xa1,0x08,0xe4,0x49,0x42,0x9b, +0xae,0x0d,0xac,0x14,0x68,0x0d,0x10,0x3f,0x41,0xb4,0x23,0x3f, +0xcc,0xcc,0x76,0xfb,0x97,0xe0,0x6b,0x30,0x88,0x63,0x23,0x35, +0x42,0x50,0x52,0xe3,0xb8,0x06,0xe9,0x62,0x79,0x61,0x4d,0xf4, +0xe2,0x74,0x42,0x41,0xa9,0x16,0xce,0x9e,0x36,0x96,0x70,0xeb, +0x3c,0x80,0xfd,0x78,0x20,0xd8,0x56,0xaf,0x90,0x21,0xf3,0x48, +0xad,0x6f,0xed,0x17,0x01,0xa0,0x5f,0x16,0x8d,0x40,0x51,0x47, +0x76,0xd8,0x14,0xf1,0xdf,0xf3,0x36,0x26,0xbd,0x16,0x11,0xbf, +0xcc,0x64,0xd0,0x0d,0x5d,0x97,0x0d,0x3d,0xc6,0x48,0x5a,0x0a, +0x77,0xc3,0xc6,0xac,0xbf,0x2e,0xee,0xbc,0x58,0x66,0xf9,0x57, +0x27,0x46,0x62,0x6e,0xd0,0x09,0x22,0xf7,0x0d,0x5b,0xf9,0x2c, +0x92,0x5b,0x03,0x0b,0xb2,0xf9,0xc4,0x67,0xb7,0xd4,0x60,0x30, +0xbf,0x4b,0x4e,0x93,0x1c,0x0e,0xaf,0xeb,0xdc,0xf7,0x40,0x37, +0x84,0x23,0x7a,0x7f,0x27,0x39,0x6a,0xc9,0x87,0xe4,0x9e,0x4d, +0x5e,0x97,0x6f,0xab,0xe5,0x4d,0x2d,0x88,0x0b,0xea,0xfe,0x73, +0x5f,0xa1,0xdd,0xce,0x79,0x23,0xe6,0x77,0x64,0x1a,0xe1,0x56, +0x33,0x19,0xe0,0x05,0x9e,0xc3,0xd6,0x53,0x19,0x4d,0x3d,0x0f, +0x5a,0x26,0x72,0xa9,0x2f,0x26,0x85,0x84,0x94,0xf8,0x72,0xb2, +0xdd,0xf2,0xaf,0xaf,0xe0,0x57,0x92,0x64,0x87,0xab,0x27,0xff, +0x2c,0x12,0x35,0x77,0x1d,0x3f,0x98,0xa8,0x57,0x20,0x36,0x88, +0x9e,0xea,0xfa,0xd1,0xa3,0x97,0xb7,0xd8,0x28,0x56,0x1d,0x4c, +0xbe,0x04,0x1e,0xc2,0x46,0x1a,0x1f,0x3d,0x92,0x80,0x65,0x9b, +0x64,0xb1,0xb9,0x72,0x9d,0x0f,0xac,0x29,0xad,0xf9,0x22,0x1b, +0x59,0x30,0xb6,0xdb,0x20,0x7a,0x82,0x9a,0xdd,0xea,0xc0,0xbf, +0x9c,0x14,0x3d,0x3b,0xf2,0x91,0x0e,0x5c,0x93,0xfe,0x0c,0xd5, +0x07,0x59,0xfe,0xe0,0x5e,0x4a,0x0f,0xd8,0xba,0xce,0xb0,0xdd, +0x64,0x39,0x9b,0xf9,0x29,0x1a,0x64,0x03,0xea,0xdb,0x36,0xff, +0xdb,0x44,0x50,0xf0,0x6f,0xad,0x15,0x72,0xfd,0x69,0x4d,0xbb, +0x5b,0xa1,0xdc,0xa9,0xad,0x4b,0x8c,0x28,0x62,0xaf,0xef,0x2e, +0x80,0x33,0x9b,0x9b,0xc5,0xaf,0x28,0x0c,0xcd,0xc4,0x88,0x1f, +0x80,0x10,0x8b,0x72,0x98,0x95,0xa1,0x8f,0x50,0xda,0xe2,0x08, +0xef,0x2c,0xcf,0xd8,0xbe,0x46,0xb8,0x63,0xe4,0x31,0x32,0x1e, +0x28,0xfa,0xbf,0x44,0xa5,0x36,0xf7,0x01,0xd2,0x12,0x9f,0xbe, +0xb9,0x05,0x4d,0xa6,0x37,0x17,0x08,0x1a,0x67,0x1f,0xad,0x79, +0x14,0x0e,0xdb,0xe5,0x6b,0xb2,0x7b,0xe0,0x3e,0x4b,0xec,0x10, +0x44,0x56,0x39,0xe6,0xb2,0xd9,0x71,0x94,0x6e,0x5d,0x4f,0xdb, +0xcc,0xa8,0x8d,0x54,0x32,0x16,0x7a,0x5b,0x3b,0x2a,0x11,0x93, +0xe3,0x17,0xe8,0x5b,0xa9,0x8f,0xc3,0xdd,0xa1,0xe5,0x52,0x70, +0x2b,0x4f,0x4a,0x44,0xa1,0x4c,0xd6,0x00,0x6b,0xea,0xa8,0x11, +0x09,0x1b,0xfd,0xe7,0x6d,0x25,0x5b,0xee,0xe0,0x42,0x8a,0x48, +0xfb,0xc0,0x1d,0x8b,0x36,0x27,0x1a,0xfc,0xf6,0xc8,0x44,0x1e, +0x78,0x18,0xf2,0x92,0x4d,0xbf,0x9a,0x9a,0x59,0xa6,0xa0,0x15, +0x48,0x2f,0xe5,0x7a,0x51,0x38,0x53,0xd3,0xb1,0x2d,0xc1,0xc1, +0x71,0x6c,0x90,0x12,0x2a,0x08,0x52,0x94,0x56,0x89,0xde,0xf4, +0x0f,0x35,0x6d,0x59,0xd2,0xf3,0xda,0x89,0x4c,0x8f,0xe8,0x49, +0x7f,0x34,0x7d,0xb6,0x9d,0x7a,0x0a,0xe3,0xf6,0xe1,0x4b,0xa7, +0x5e,0x78,0x79,0x5b,0xc3,0x5e,0xe1,0x34,0x24,0xa8,0x0f,0xdf, +0x7b,0xa4,0x09,0x35,0x83,0x2f,0x5c,0x60,0x80,0x4d,0x40,0x57, +0xc8,0x5e,0xe9,0x05,0x25,0x35,0xc0,0xe4,0x70,0x24,0x87,0x82, +0x46,0xc9,0x1b,0xc2,0x9e,0x78,0xeb,0x49,0xb5,0xe1,0x73,0x80, +0x5c,0x74,0x84,0xea,0x2c,0x8c,0xcb,0x2c,0x5d,0xe4,0xfa,0x63, +0x24,0x72,0x54,0x0f,0xa5,0x84,0x24,0x28,0x33,0x92,0x5e,0xc4, +0x93,0x34,0x8f,0x9a,0x18,0xd9,0x47,0xc5,0x24,0xfb,0x45,0x98, +0x4d,0xa9,0xdd,0xee,0x89,0x15,0x89,0xec,0x3a,0xe4,0xe1,0xe5, +0xc9,0xbd,0xd6,0x0a,0xad,0x85,0x4f,0xe1,0x50,0x9b,0x7c,0x76, +0xa8,0x3c,0x2f,0xb1,0x63,0x4a,0xd4,0x45,0xec,0x40,0xff,0x67, +0x58,0x15,0x1f,0x69,0xec,0xc9,0xc0,0x71,0xd7,0x68,0xeb,0x6d, +0x7b,0x08,0x19,0xf8,0x5c,0x8c,0xd7,0xed,0x89,0xf1,0xfb,0xed, +0x0d,0x6c,0xcf,0xbd,0xe4,0x38,0xc9,0xf1,0x1d,0x04,0x0c,0xcc, +0x11,0xef,0x44,0xe4,0xf0,0x71,0x7b,0x35,0xdd,0xd8,0xb0,0x57, +0xbc,0x4d,0x1b,0xb2,0xa5,0xe3,0x07,0x97,0xd1,0x93,0xb6,0x2c, +0x68,0x17,0x12,0xc3,0x0e,0x8f,0x99,0xe0,0x8b,0x2a,0xb7,0xd3, +0xf5,0xac,0x48,0xb0,0x46,0x54,0xe0,0xf8,0xcb,0xcc,0x2c,0x2f, +0x49,0xe0,0x4b,0xc4,0xda,0x10,0x07,0x84,0xfd,0xf9,0x91,0xba, +0xa9,0x47,0x9e,0x26,0x6c,0x64,0xd4,0xba,0x2c,0x3a,0x71,0x70, +0xcb,0x0f,0xc6,0x99,0x69,0x84,0x98,0x21,0x69,0x14,0x83,0xbe, +0xd1,0x2f,0x0a,0xdb,0x65,0xea,0xcd,0x74,0xbd,0x41,0x73,0xf4, +0x82,0xc7,0x09,0x03,0x90,0x11,0x8d,0x12,0x4f,0xec,0x4b,0x5a, +0xe4,0x19,0x06,0xa5,0x88,0xc0,0xa2,0x05,0x50,0xf0,0xa8,0x0f, +0xf2,0x2e,0x73,0x53,0xb3,0xbd,0x7a,0xd1,0x01,0x90,0xbb,0xdd, +0x0e,0xd7,0x41,0x61,0x98,0x5b,0x23,0xe1,0x96,0xa1,0x12,0x91, +0x1f,0xc0,0x1c,0xf2,0x4f,0x2b,0x2f,0xb2,0xe4,0x88,0xff,0xb8, +0x38,0x35,0xdc,0x10,0x5b,0x54,0xed,0xe7,0x9b,0x78,0xb2,0x52, +0x59,0x3a,0x3e,0x7d,0x77,0x06,0x43,0x42,0x0c,0xfa,0x54,0xe1, +0x0b,0x6e,0xf3,0xf7,0x49,0xa1,0x46,0x3e,0x1d,0xca,0xe9,0x76, +0x03,0xf4,0xec,0xec,0x8e,0x80,0x30,0x53,0xd5,0xb1,0xe2,0x91, +0x47,0xe3,0x89,0xd7,0xb9,0xa4,0x58,0xd3,0x9a,0x89,0x52,0x51, +0xb7,0xab,0x48,0xe1,0x61,0x7a,0xa1,0x74,0xb8,0x8e,0x1f,0x4c, +0x27,0x4a,0xa1,0x70,0x3a,0x8d,0xf0,0x36,0xe4,0xd2,0x42,0x01, +0x61,0xed,0x9c,0x17,0x61,0x68,0x1d,0xf4,0x43,0xd0,0x70,0x3b, +0x83,0x89,0xc4,0x1a,0xdc,0x5b,0xe2,0x01,0x37,0x54,0xd8,0xd6, +0xba,0x5d,0x19,0x8d,0xb3,0x0f,0x1c,0xdb,0xcc,0x09,0x4f,0x4f, +0xaf,0xe7,0x15,0x77,0xb7,0xcd,0x34,0x63,0x30,0x10,0xea,0xd2, +0x81,0x77,0x82,0x51,0xce,0x10,0x85,0xe8,0x86,0x90,0x66,0x36, +0xf4,0x50,0x30,0x51,0xdd,0x6e,0xaf,0x87,0xf5,0xb8,0x25,0xf5, +0x1c,0xb8,0x8b,0x47,0x4e,0x91,0x7d,0xc8,0x97,0xc4,0x91,0x3c, +0x56,0x9b,0xa8,0xb7,0x8b,0x9e,0x5d,0x14,0xef,0xf8,0xfe,0x38, +0xe1,0x30,0x8c,0x55,0xd4,0x16,0xbb,0x28,0xb3,0xfe,0x42,0x4d, +0x74,0x65,0x1e,0x1a,0x07,0xec,0x44,0xce,0x07,0x9d,0xf4,0x8f, +0x23,0x04,0x05,0xa3,0x8f,0xa0,0xf1,0x8e,0xd2,0x78,0x27,0xa6, +0xf1,0xed,0xb6,0xc3,0xc8,0x83,0xae,0xbc,0x71,0x13,0x68,0xfe, +0x88,0x83,0x6f,0x6f,0xc4,0x0e,0xa9,0x84,0xca,0x8c,0x70,0x2b, +0x0a,0x2d,0x38,0x58,0xdb,0xdd,0x0f,0x73,0x8f,0x89,0xa5,0x31, +0x55,0x09,0xa7,0xdb,0x6e,0x4f,0x08,0x51,0x97,0x97,0xff,0x55, +0x38,0x1f,0xfe,0xbe,0x4f,0xbf,0x4f,0x8b,0x13,0x7c,0xe6,0x32, +0x96,0x83,0x59,0x0c,0x75,0xa1,0x09,0xf9,0x6e,0x65,0xfd,0x0a, +0x03,0x94,0x4b,0xf4,0x3d,0xbe,0x26,0x1a,0xf3,0xdf,0x89,0xc4, +0x76,0x70,0xfe,0x45,0x63,0x95,0x64,0x7d,0x14,0xc5,0xa7,0x86, +0x04,0xc6,0xc4,0x9a,0x89,0x72,0x76,0xc4,0x15,0x50,0x0f,0xc2, +0xbf,0x9c,0x59,0x86,0x8b,0x6a,0x3d,0xba,0xf4,0x02,0x34,0xe6, +0x97,0x0b,0x17,0xc6,0x09,0x93,0xeb,0x42,0x7a,0x95,0x4b,0x2c, +0x03,0xfb,0xbb,0x96,0x9e,0x98,0x08,0xc0,0xfe,0xd5,0xb4,0xfe, +0xe9,0x76,0xf1,0xb3,0xbc,0x3e,0x94,0xb6,0xc1,0x9b,0x8d,0x3e, +0x25,0xd2,0x63,0x35,0x7b,0x67,0xc3,0x01,0x5e,0x4a,0xa1,0x46, +0x70,0x47,0x2b,0x6e,0x4b,0x0c,0xa9,0xd9,0xbb,0xa2,0x19,0xc3, +0xc4,0x79,0xd5,0x66,0xe4,0x3b,0x39,0x86,0x62,0x7e,0x84,0x47, +0xfa,0xf4,0x0a,0xed,0xb2,0x3d,0x76,0x57,0x97,0xb2,0x57,0x9d, +0x20,0x20,0x35,0x95,0x5b,0xe0,0x91,0x97,0x27,0x0f,0x5f,0xb8, +0xdd,0xe4,0xfb,0x86,0xa8,0x11,0x95,0x68,0x68,0x78,0x89,0x4c, +0x77,0xfb,0xd6,0xa4,0x58,0x7d,0xe3,0x8b,0xd8,0x71,0xad,0x2d, +0xb6,0x41,0x36,0x0c,0xe3,0x6f,0xb9,0x09,0xfd,0x30,0x82,0x0f, +0xc4,0xbf,0x72,0x3e,0xf7,0xea,0xd8,0x4c,0x15,0x49,0x41,0x8b, +0x0e,0xfe,0x18,0xae,0xa8,0xf2,0xfd,0xf8,0xfc,0xd8,0x32,0x71, +0xc8,0x2b,0x46,0x74,0x80,0x66,0x1b,0x1c,0x26,0x3b,0xb4,0xf7, +0xdb,0x20,0x21,0x80,0x23,0x64,0x4a,0x54,0x42,0x01,0xa2,0x54, +0x07,0x91,0x6a,0xf0,0x4c,0xc7,0x5a,0x60,0x23,0x8b,0xd9,0xd1, +0x63,0xfd,0xf5,0x07,0xe7,0xe2,0xa0,0xa9,0x58,0xec,0xc2,0x4b, +0x99,0x4e,0x41,0xbd,0x73,0x3a,0xe3,0x3d,0xfd,0x04,0xc6,0x24, +0xad,0x7d,0xa4,0xb1,0x4a,0x4b,0x74,0xab,0x64,0x70,0xf0,0x9c, +0x82,0xa4,0x07,0xaa,0x2a,0x27,0x65,0x52,0xdb,0x18,0x28,0x05, +0xc0,0x98,0x4f,0x17,0xaf,0xf6,0x80,0x80,0x59,0x2e,0x4e,0xbf, +0x73,0xd0,0xe1,0xc3,0x09,0xe6,0xd2,0x69,0xe2,0x10,0x1f,0xa1, +0xd6,0x81,0x8b,0x50,0x53,0xd4,0x57,0x81,0xff,0xfc,0x81,0x63, +0xa2,0xd2,0xdb,0xf7,0xd0,0xcc,0x70,0xb6,0x3c,0x92,0xb0,0xe6, +0xf8,0x59,0x84,0xd6,0x33,0x2d,0x87,0x20,0x98,0xf5,0x1c,0xfb, +0x4f,0x5a,0xf5,0xd0,0x09,0x67,0xe5,0x94,0x43,0x10,0x7b,0x67, +0x52,0xea,0x40,0x7f,0x44,0x20,0xda,0xfc,0xa2,0x60,0xac,0x9c, +0xc8,0x9e,0x4c,0x65,0x15,0xeb,0x98,0x85,0xf2,0x78,0x36,0x84, +0x33,0x22,0xd0,0x86,0xa7,0x7d,0xf4,0xa2,0xb6,0xb3,0x6b,0x75, +0xa0,0x22,0xcd,0xdc,0x1d,0x32,0x4c,0x18,0x3f,0xc9,0x14,0xf3, +0x18,0x7e,0xad,0xa5,0xbe,0x52,0xbf,0xcb,0xf9,0xf2,0x42,0x1e, +0xf9,0x8c,0x12,0x90,0xd5,0x5a,0x7b,0x5b,0xfc,0xdd,0xed,0xe2, +0xbf,0x22,0xb4,0xd3,0xc9,0x81,0xa9,0x68,0xb6,0x33,0xeb,0xe5, +0xb2,0xd1,0x51,0x18,0xe3,0x47,0x2e,0xb2,0xf5,0x4d,0xec,0x0f, +0x95,0x6d,0x7d,0x89,0xda,0x05,0xcf,0x47,0xd8,0xbe,0xc5,0x86, +0xdc,0x45,0x96,0x4d,0xa2,0xad,0xbf,0xd3,0x11,0xa2,0x27,0x25, +0x2b,0xfd,0x44,0x93,0xd7,0xed,0xf6,0x9f,0x9c,0x3c,0x3d,0x27, +0xa1,0x01,0x8e,0xb7,0x62,0xb8,0x72,0xc8,0x04,0x56,0x0d,0x5a, +0xe8,0xc9,0x14,0xf4,0x4a,0xad,0x3f,0xa9,0x0a,0x3e,0x43,0xce, +0xf6,0x3f,0x6d,0x73,0x61,0x35,0x23,0x07,0x0e,0x01,0xcd,0xf8, +0xce,0x56,0xed,0x21,0x07,0x1e,0xb5,0x6a,0xda,0x26,0x6c,0x41, +0xd5,0x5a,0x48,0x2d,0x9a,0x76,0x46,0x7f,0x1d,0x1a,0x82,0x7f, +0x6b,0x1f,0x25,0xd8,0x96,0x58,0x54,0x0c,0x1a,0x97,0x11,0x97, +0xd7,0xab,0xcd,0xfb,0xb8,0xf9,0xdf,0xa3,0x49,0xd7,0xd7,0x08, +0x74,0x34,0xa7,0xc9,0x7f,0x4f,0xb6,0xdb,0x2f,0x9a,0x41,0x36, +0xbf,0x6c,0xac,0x86,0x66,0x20,0x47,0x39,0x8d,0xb5,0x0f,0xbb, +0xe3,0x8b,0x48,0x7d,0x1a,0x80,0xa8,0xee,0x0d,0x7b,0x51,0xed, +0xc1,0x96,0xf8,0x58,0xf5,0x5b,0x1e,0xfa,0xdb,0x19,0x9a,0xb2, +0x7d,0x15,0xd9,0x95,0xab,0xbd,0x22,0x3b,0x71,0xb5,0x52,0xcc, +0xef,0x8b,0x06,0xee,0xc5,0xe1,0xf6,0x96,0x46,0x14,0x9d,0x1b, +0x1f,0x4f,0x7b,0xd7,0xca,0x33,0x50,0x77,0xd3,0x68,0xf9,0x00, +0x14,0xd0,0x21,0x36,0x15,0x74,0x87,0xd1,0x27,0x49,0xd7,0x93, +0xd2,0x45,0x67,0x0b,0x63,0x65,0xfb,0x72,0x60,0xba,0x58,0xa0, +0x71,0x1a,0xb5,0x0d,0xe1,0xe3,0x5a,0x74,0x4f,0xb0,0x2f,0x5e, +0x78,0xac,0x84,0xc4,0xc0,0x9d,0x19,0xa9,0x2b,0x3e,0x58,0x85, +0x64,0x00,0x89,0xd6,0x6d,0x34,0x66,0x85,0xb4,0x22,0xba,0xd8, +0x01,0x36,0x56,0xfe,0xfa,0x7b,0x9a,0x32,0x71,0xa4,0xf2,0xf1, +0xc9,0xa9,0x26,0x8d,0xf4,0x47,0x4f,0x02,0x1e,0x5b,0x31,0x78, +0xc7,0xaf,0xee,0xfe,0x3e,0xd8,0x3d,0x07,0xf3,0xa1,0xc8,0x15, +0xe8,0x53,0x5e,0x3c,0xc8,0xfc,0xe2,0x7a,0xe5,0x17,0xc8,0x7c, +0x92,0x07,0xdd,0x2f,0x67,0xb3,0x3f,0xda,0xfb,0xe0,0xf7,0xf7, +0x3e,0xff,0x5d,0xd3,0xe6,0xe1,0xda,0xc3,0xc1,0x87,0xd1,0x3d, +0x3c,0x3e,0xae,0xe0,0x88,0x35,0xfc,0x14,0x08,0x5f,0xfd,0xaf, +0x85,0xb0,0xd7,0xab,0xbe,0x16,0xe1,0xf8,0x13,0xa0,0xdc,0xed, +0xe8,0x32,0xb3,0x02,0xa9,0xf7,0x8e,0xde,0xc6,0xcc,0x3b,0xa2, +0x10,0x3c,0x5f,0xbe,0xc3,0xdf,0x97,0x20,0x6e,0xe0,0x5f,0xf4, +0x89,0xb9,0x5d,0xae,0x67,0xf8,0xbb,0xba,0x9e,0xbe,0xc2,0xc4, +0x5d,0x16,0x1c,0x21,0x41,0xcc,0x6e,0x3e,0x64,0x50,0x65,0xae, +0x03,0x7e,0x38,0x17,0x5b,0x00,0xa1,0xb9,0xdc,0x1c,0x68,0x21, +0x78,0x86,0x00,0x9a,0x90,0xb3,0x49,0x61,0x6d,0x33,0xf8,0xb3, +0x21,0x59,0x73,0x7c,0x6f,0x0d,0x51,0xe3,0x05,0x09,0xc2,0x83, +0x4e,0x39,0x0f,0x62,0x4b,0x61,0x92,0x44,0xbd,0x0f,0x42,0x42, +0x34,0x03,0xdf,0x77,0x24,0xf2,0xbd,0x7b,0x8d,0x04,0x0e,0xf9, +0xf4,0xb7,0x25,0xd0,0x4d,0xa6,0xef,0x83,0x04,0x47,0x15,0xe9, +0x48,0x4e,0x13,0x5e,0x84,0x37,0x2a,0x4a,0x07,0x44,0x2e,0xe2, +0x4e,0x54,0x7e,0x88,0x23,0x9a,0x43,0x81,0x22,0x8f,0xc6,0x11, +0x64,0x0e,0xdb,0x46,0x94,0xc6,0x11,0x78,0x3f,0x2d,0xb8,0x9b, +0xbd,0xc8,0xe7,0xb6,0xc5,0xd4,0x5d,0x77,0xc5,0xc5,0xe6,0xaa, +0x08,0xb7,0xc9,0x5f,0x7d,0x77,0x0e,0x0c,0x7d,0x66,0x8f,0xea, +0xee,0xa9,0x7e,0xbf,0x46,0x7c,0xa2,0xa7,0x5b,0x13,0xf7,0xe9, +0x3b,0x87,0x82,0xc0,0xec,0xdb,0xc5,0xc8,0x53,0x63,0xf8,0xdb, +0x3e,0xc6,0x89,0x91,0x69,0xb8,0x8c,0x7d,0x7a,0xb4,0xed,0xce, +0xdb,0xcf,0x1c,0x27,0x79,0x32,0x29,0xbc,0x04,0x0b,0x99,0xa4, +0xb1,0x0d,0x47,0xd1,0xb0,0xe8,0xd0,0xfc,0x4d,0xd3,0xaa,0x41, +0xb3,0xc8,0x07,0x14,0x83,0x9a,0xf8,0x0e,0xb8,0x26,0x7a,0x9d, +0xbd,0x88,0x6c,0x8e,0x76,0xf2,0x12,0x1a,0xbb,0xd3,0xc1,0x41, +0x67,0x53,0xcd,0x8b,0xfb,0xff,0x89,0x7f,0xd0,0xdf,0x9b,0x45, +0x96,0x1a,0x15,0xcb,0xec,0xf4,0x2d,0x09,0x18,0x23,0xf6,0x2d, +0x7c,0x52,0xc1,0x2d,0x99,0xe0,0xdf,0x37,0xf4,0x20,0x2e,0x2a, +0xe2,0xef,0xbf,0xe8,0x8f,0x5f,0xe4,0x7f,0x3a,0x1b,0x9f,0xf5, +0xcd,0xe4,0x1e,0xb6,0xe3,0x3b,0xd1,0x07,0xc3,0xe7,0x9b,0x3f, +0x3f,0xdb,0xbc,0xba,0x99,0xae,0xa7,0xf0,0xb3,0x9c,0x49,0xd0, +0xb9,0x3b,0xd2,0x5b,0x42,0xbf,0xc4,0x31,0xf8,0x14,0x59,0xe3, +0x6f,0x14,0xd5,0x88,0x63,0x00,0x30,0xb8,0xc4,0x87,0xcd,0x67, +0x52,0x2f,0x83,0xa7,0xfc,0x22,0x15,0x05,0xda,0x7d,0xa0,0x38, +0x7a,0xd9,0x88,0x7d,0xe2,0xd9,0x6c,0xec,0x7d,0x92,0x0b,0x2b, +0xf2,0xdb,0xfa,0xfc,0x3c,0x31,0xac,0x8d,0x67,0x9b,0xe9,0xc5, +0x1b,0x79,0x39,0xd9,0x75,0x26,0xf4,0xe9,0x0b,0x06,0x1c,0x4d, +0xe9,0x24,0x8c,0xb1,0x12,0x3f,0xa4,0x8f,0x1d,0xa0,0x19,0x23, +0xbd,0x97,0xe7,0x42,0x97,0x8b,0x61,0x0c,0xda,0xac,0xb0,0xd3, +0x8c,0xd7,0x8a,0x47,0x9a,0xde,0x3b,0x81,0x1c,0xbb,0x05,0x47, +0xeb,0x9b,0x68,0x14,0x11,0xd8,0xd8,0xca,0x60,0x14,0xd0,0x21, +0x76,0x92,0xe5,0xf8,0x0f,0x56,0xb6,0x24,0x5d,0xf0,0x53,0xc1, +0x36,0x72,0x73,0xf0,0x85,0xbc,0x2c,0x87,0x03,0x77,0xcf,0xf6, +0x4f,0xe6,0x20,0xa8,0xab,0xf0,0xbc,0x65,0xe8,0xf1,0x4c,0x8d, +0xef,0x45,0x5f,0x75,0xa1,0xef,0x6c,0xf2,0x6b,0xbe,0x34,0x6a, +0x9e,0x18,0xce,0x8f,0xbc,0x30,0xa9,0xd3,0xdf,0x89,0x59,0x9e, +0x35,0x6e,0x16,0x39,0xa2,0x87,0xdc,0x9d,0x59,0xf8,0xc7,0xc9, +0x86,0x22,0x25,0x42,0x1a,0xac,0x9e,0xc5,0xf2,0x96,0x1b,0xb4, +0x03,0xee,0x0c,0xb2,0xcc,0x5e,0x60,0xff,0xc1,0xa6,0xe8,0x7d, +0xf9,0xaa,0xde,0xdf,0x4c,0xa7,0xe3,0x3d,0xd3,0xbb,0xff,0xad, +0xdb,0xf5,0xa1,0x80,0x5d,0x11,0xc5,0x32,0x41,0xdb,0xd0,0xb0, +0xa4,0x1a,0x48,0xe5,0x01,0xc6,0x0c,0x84,0x93,0xdc,0x92,0x19, +0x61,0x3f,0xa8,0xe6,0xd4,0x47,0xb9,0x3f,0x43,0x6e,0x31,0x68, +0x36,0x0c,0xeb,0x62,0xbe,0xac,0x01,0x90,0xe6,0xd8,0x6a,0xe7, +0xa8,0x64,0x23,0x10,0x8b,0x51,0xf4,0xdc,0x5f,0xa7,0x46,0x16, +0xc1,0x6a,0x59,0x17,0x07,0x06,0x88,0x8e,0x01,0x7b,0x97,0x73, +0x1d,0x8f,0xde,0x76,0xbe,0xdd,0x06,0x78,0xc8,0x81,0xa8,0x84, +0xa4,0xd4,0x74,0x59,0x56,0xd6,0x90,0x1e,0x7e,0x42,0x0b,0xe6, +0xe8,0xf5,0x39,0x48,0x72,0x2f,0x8f,0x89,0x4d,0x29,0xbd,0xde, +0xa4,0x66,0x65,0x1a,0x61,0x1b,0x83,0x3a,0x0b,0x9e,0xd1,0xda, +0x19,0xaf,0x56,0xbd,0xa5,0x1c,0x5b,0xd0,0x62,0x19,0xe3,0x06, +0x47,0x94,0x20,0x7b,0x33,0x9a,0x4a,0xf3,0xbd,0x49,0xd3,0xa8, +0x7a,0xd7,0x4e,0x77,0x58,0xd9,0x2a,0xc5,0xf7,0x2c,0xff,0x1d, +0x47,0x93,0x3a,0x70,0x64,0x1f,0xb5,0xda,0xb8,0xfd,0xbf,0xd4, +0xbd,0x79,0x7f,0x1b,0x47,0x92,0x28,0xf8,0xff,0xfb,0x14,0x60, +0x0d,0x07,0x5d,0x65,0x14,0x41,0x52,0x76,0xf7,0x74,0x17,0x5d, +0xc2,0xa3,0x29,0xa9,0xad,0x19,0x5d,0x23,0xd2,0xed,0x9e,0x05, +0x61,0x3d,0x90,0x00,0x49,0xc8,0x20,0x00,0xa3,0x40,0x1d,0x4d, +0xe0,0xbb,0x6f,0x9c,0x99,0x91,0x59,0x05,0x4a,0xee,0x37,0xfb, +0x76,0xf7,0x67,0x8b,0xc8,0xca,0x23,0xf2,0xce,0x8c,0x88,0x8c, +0x43,0xe1,0x4d,0x66,0xc7,0xcb,0xe5,0xf0,0xb3,0x2e,0x22,0x75, +0xa9,0xcb,0x1c,0xbf,0x22,0xca,0x44,0x94,0xd6,0x7b,0x52,0xdd, +0x21,0xf6,0x18,0x14,0x20,0x4c,0x84,0xd7,0x57,0x21,0x30,0xda, +0x6d,0x09,0x58,0x45,0x5b,0x59,0x72,0x40,0x9d,0xa5,0x19,0x39, +0xfe,0x45,0x9d,0x1f,0x5d,0x73,0xe8,0x08,0x2b,0x1f,0x8e,0x1a, +0xee,0x83,0x50,0x27,0xb1,0x42,0x69,0xc0,0xad,0x1b,0x29,0xde, +0x2b,0x6e,0x79,0x48,0x2f,0x6e,0x87,0xbf,0x8e,0xb9,0x1f,0x7e, +0x5f,0x3a,0x46,0xb7,0x37,0x94,0xaf,0x51,0x83,0xc2,0x3b,0x99, +0x1b,0x02,0x79,0xaa,0x50,0xc6,0x70,0x38,0xf1,0x31,0x0b,0x87, +0x14,0xda,0x0b,0x33,0x87,0x77,0xe3,0x7d,0xa3,0x73,0x36,0xc4, +0xcb,0x98,0xfa,0x89,0x76,0x24,0xbf,0x74,0xfc,0xa0,0x4b,0x67, +0x26,0x8b,0xc3,0xd3,0x1c,0xc7,0xee,0x35,0xf9,0xce,0x2b,0xa2, +0xef,0xda,0x8e,0xce,0x42,0x07,0xe5,0x80,0x52,0x9e,0xe2,0xcd, +0x68,0x62,0xb8,0x29,0x79,0xe0,0x2e,0xbe,0x99,0x19,0xc2,0x42, +0x30,0x4d,0x8f,0xd6,0x5e,0xf2,0x65,0xe9,0xe5,0xb3,0xe5,0x5d, +0xd3,0x8d,0xaa,0x00,0x65,0x81,0x5b,0x41,0x55,0xb6,0xac,0x59, +0x69,0xcc,0x68,0x22,0x42,0x21,0x56,0x46,0x30,0x73,0x85,0x09, +0xbf,0x89,0x38,0xf9,0x93,0x9c,0xb0,0xa4,0xaf,0x82,0x24,0x59, +0x37,0x8c,0xaa,0x34,0xb7,0x44,0x14,0x17,0xa4,0xb0,0x7d,0xb2, +0xce,0x36,0x8c,0xda,0x7c,0x4d,0xb9,0xf8,0xed,0x5f,0xea,0x84, +0xf5,0xff,0xb5,0x03,0xd0,0x50,0xf3,0xef,0x28,0xbd,0xa5,0xfe, +0x7f,0x6a,0x00,0x6d,0x4b,0x24,0xef,0xff,0xe0,0x06,0xfd,0x73, +0xf3,0x11,0x8b,0x45,0xe8,0xa4,0xc8,0x08,0x7e,0x61,0x89,0xe8, +0x38,0xa7,0xd1,0x8a,0x5c,0xaf,0xef,0x37,0x99,0x61,0x24,0xaa, +0x38,0xbd,0xc3,0x56,0xbf,0x0a,0x6a,0xc4,0x8d,0xc4,0xf2,0x8a, +0xe1,0x3e,0x58,0x5e,0x19,0x5f,0xd2,0xc7,0xc9,0xd5,0x12,0x3e, +0x92,0x8c,0xdf,0x13,0x04,0x44,0x28,0x9f,0xaf,0xb1,0x3f,0x87, +0x36,0x0f,0x8b,0xe0,0xc4,0x11,0xe2,0xd5,0x88,0x0a,0x90,0x2c, +0x5e,0x20,0x81,0x85,0x0e,0xf1,0xdc,0xee,0x16,0x51,0x78,0x97, +0x83,0xc6,0x36,0x0f,0x11,0x6d,0xbc,0xa5,0xdd,0xe9,0xb8,0x60, +0x0c,0xe7,0x6a,0x26,0xd3,0xe0,0x30,0x52,0xfa,0xe4,0x8b,0x9b, +0x24,0xb4,0xc8,0x1e,0xb0,0xf3,0x1e,0x8e,0x59,0x73,0x7f,0xa2, +0x6e,0x3d,0x9f,0x01,0x09,0x32,0xd5,0xc5,0x18,0x0a,0xde,0x65, +0x84,0xcb,0x5a,0x2c,0x02,0xed,0xf9,0xee,0xc4,0x54,0x07,0xf7, +0x6b,0xcb,0x7d,0x98,0x47,0xa5,0x2d,0x9d,0x64,0x7a,0x20,0x6d, +0xc1,0x0b,0x16,0x7d,0xd0,0x2f,0x59,0xbd,0xa1,0x7e,0xfd,0xda, +0x13,0xd4,0x13,0x2e,0x21,0xf6,0xe8,0x54,0x83,0x50,0x4e,0xc2, +0xbf,0xb9,0x41,0x98,0x64,0x39,0x81,0xee,0x4c,0x0a,0xf8,0x48, +0xc9,0xae,0xca,0x92,0xed,0xb6,0xe9,0x1b,0x8d,0xf3,0x05,0xfe, +0x10,0x4e,0x41,0x19,0xf1,0x6e,0x26,0x8d,0x99,0x5e,0x5f,0xbf, +0xd1,0x72,0x45,0x13,0x32,0x62,0x5a,0x94,0x91,0xb5,0xcc,0x68, +0x5b,0xa2,0x1f,0x30,0xd9,0x98,0x8a,0xc3,0xe9,0xd3,0x2b,0xb2, +0x43,0xee,0x96,0xc6,0x75,0x15,0xe1,0x4f,0x7f,0x89,0x90,0x23, +0xe8,0x17,0x93,0xa3,0x65,0x69,0x74,0xcf,0x63,0x14,0x6a,0xbd, +0xde,0x91,0x1b,0x19,0x71,0x1f,0x20,0x7a,0xb9,0x10,0x0a,0x39, +0x60,0xf7,0x43,0x80,0xfa,0x9e,0xee,0x70,0xa5,0x5c,0xb0,0x24, +0x6e,0x46,0xf0,0xe2,0x3c,0x72,0xe7,0x83,0x11,0x40,0xcc,0xfd, +0x5b,0x02,0x2d,0x6c,0x34,0x94,0x35,0x3b,0x9a,0x95,0xb3,0xe0, +0xd9,0x80,0x44,0x61,0x4c,0xb5,0x33,0x91,0xbc,0x80,0x85,0xc2, +0x55,0xcf,0xfc,0xaa,0x87,0xa9,0x3f,0xb2,0x4f,0x31,0x55,0x99, +0x0c,0x2f,0x2e,0x96,0xeb,0xe1,0x72,0x35,0xb9,0x9c,0x8e,0xd7, +0xc3,0x6a,0x02,0x47,0xcd,0xf0,0x6e,0x34,0x99,0xaf,0x2f,0x46, +0x93,0xf5,0xe5,0x70,0xf6,0x61,0x58,0x91,0x3f,0x2d,0xfa,0x33, +0x9d,0x54,0xab,0xf5,0x68,0x0c,0x74,0xcd,0xb4,0x5a,0x5f,0x4d, +0xae,0x2f,0x87,0xf4,0x06,0x83,0xc1,0xbb,0xe5,0x78,0x7d,0x35, +0x9f,0xa3,0xb3,0x1d,0x7e,0x25,0x58,0xdf,0x30,0x4b,0x09,0x1f, +0x52,0xd7,0xb7,0x63,0x4c,0x98,0x0d,0x3f,0xac,0xe7,0x77,0x2b, +0xb4,0xe4,0xb6,0x58,0xce,0xaf,0x51,0xb5,0x64,0x5d,0x8d,0xa9, +0xb7,0xeb,0xea,0xee,0x16,0x72,0x7e,0x5e,0xaf,0x26,0xb7,0xe3, +0xf5,0x07,0x68,0xc6,0x3c,0x41,0x0b,0x70,0xd0,0xc5,0x31,0x0f, +0x79,0xb9,0x2f,0xa7,0xd1,0xf9,0xa8,0x43,0xc6,0x3c,0xf1,0xe2, +0x5d,0xa3,0x49,0xbd,0x04,0x8d,0xb3,0xcd,0xe6,0xd5,0xcd,0xe4, +0x96,0x65,0x75,0xac,0x78,0xea,0xf7,0x64,0x48,0xd3,0x75,0x18, +0x2d,0x15,0x9e,0x9f,0x57,0xfb,0x8f,0x07,0x62,0x71,0x6f,0x29, +0x7e,0x2d,0x7e,0xf6,0x16,0x45,0xf7,0x7f,0x39,0xaf,0x3a,0xfb, +0xf9,0xf2,0xd3,0xcd,0xea,0x76,0x7a,0x36,0xbc,0x2e,0xf7,0x01, +0xc8,0x0e,0x1b,0x9c,0x5b,0xae,0x2f,0xe7,0xd3,0xf5,0xf8,0xf6, +0x02,0x96,0xc7,0xcd,0x72,0x3d,0xb9,0xbd,0x5e,0xb3,0x69,0x3a, +0x68,0x28,0x75,0x73,0xb8,0x86,0xed,0x39,0xbc,0xcd,0x52,0x34, +0x65,0x57,0x0c,0x3a,0x59,0xff,0x97,0xc7,0x83,0x6f,0xb2,0xf3, +0xfd,0xc7,0xfb,0xd7,0x93,0x7c,0x29,0x86,0x32,0x10,0xa4,0xa4, +0x43,0x45,0xab,0x8b,0xf9,0x08,0xfa,0xf7,0x3d,0xfd,0x92,0x2d, +0x3b,0xa8,0x18,0xbe,0xd7,0xed,0x7f,0xe9,0x9d,0x7f,0xec,0x1c, +0x21,0x4f,0x63,0xfe,0x1c,0xdf,0xb0,0x25,0x01,0xfa,0x54,0x5d, +0x2e,0x27,0x8b,0xd5,0xba,0x5a,0x7d,0x86,0x79,0xc3,0xca,0x33, +0x28,0x78,0x3b,0x9c,0x4d,0x16,0x77,0x53,0x7a,0xd5,0x7c,0xb7, +0x24,0x7e,0x27,0x3e,0xce,0x91,0xc4,0x21,0x71,0x54,0x94,0x05, +0xba,0x26,0xa6,0x28,0x1b,0xce,0x93,0x17,0xb5,0x72,0x5f,0x02, +0xe7,0x15,0xea,0xca,0xf7,0x7f,0x29,0x07,0xeb,0x12,0xc2,0xfa, +0xe2,0xd6,0xc5,0x0a,0x96,0x5c,0xad,0x00,0xdc,0x5d,0x9f,0xef, +0x43,0xd6,0xf7,0xc3,0x0f,0xc3,0xf5,0xf8,0xf2,0x76,0x98,0x71, +0x6a,0x98,0xef,0xe5,0xb0,0x22,0xe8,0xbf,0xac,0x96,0x77,0x63, +0xc8,0xdf,0xfd,0x06,0xbb,0x7c,0x89,0x96,0xb2,0x4e,0x29,0x13, +0x8d,0xf7,0x37,0xdf,0xef,0xa0,0x82,0x7e,0xff,0xe4,0xc9,0xf1, +0xd9,0xf1,0x79,0x7f,0xbd,0xb7,0x97,0xad,0x31,0x62,0x70,0x3e, +0xc0,0xf0,0x63,0xc8,0xb1,0x0b,0xd3,0xfc,0x71,0x39,0x5c,0xa0, +0x53,0xa9,0x7b,0x7e,0xfc,0x2b,0xfa,0x87,0xb9,0xda,0x59,0x70, +0x86,0xc7,0xcb,0x3f,0x68,0xe8,0x0f,0x8f,0x61,0x92,0xbd,0x85, +0x85,0x01,0x10,0xf8,0xd7,0x70,0xde,0x71,0xa9,0xab,0xc9,0x78, +0x3a,0x02,0x44,0x97,0xf3,0xf8,0xaf,0x41,0x8e,0x33,0xcd,0x79, +0xe0,0x02,0xe1,0x64,0x0a,0x0c,0x72,0x9a,0x5a,0x4e,0x62,0x5f, +0xcf,0x9c,0xaa,0x61,0xe4,0x9e,0xc0,0x6a,0xe2,0x0c,0x2b,0x1c, +0x79,0x4e,0x97,0x20,0x24,0x2f,0x8b,0xfe,0x23,0x97,0xc6,0xf3, +0x2d,0x59,0x28,0x68,0xb2,0xc2,0x32,0x6b,0xc8,0xeb,0x32,0x42, +0x32,0x6d,0x2f,0x2e,0xed,0xbe,0x6c,0x5d,0xd0,0x8e,0x6f,0x6b, +0xe5,0x57,0x4b,0xa9,0x6f,0xf9,0xb8,0xa1,0xd2,0x77,0x70,0xf2, +0x0d,0x61,0xf0,0xf4,0x08,0x56,0xbd,0x2c,0x5c,0x76,0xa7,0xe3, +0xe5,0x04,0x0e,0x80,0x7f,0x00,0x39,0x71,0x80,0x92,0x4b,0x49, +0x32,0xa0,0x8e,0xfe,0xfd,0x7b,0x36,0xa7,0x95,0x88,0x5d,0xad, +0x01,0x1c,0x65,0xc3,0xab,0xf1,0xb3,0xe5,0xf0,0x9a,0x6c,0xcb, +0x31,0xe7,0xfb,0xd4,0x44,0x39,0xbb,0x73,0x59,0x7e,0x25,0x51, +0x4f,0x26,0x1f,0x4a,0x5b,0x2a,0x36,0x0f,0xc7,0x0f,0xdf,0x0c, +0xea,0x29,0xf3,0x74,0x53,0x36,0x23,0x90,0x1d,0xc9,0x9a,0xe8, +0xc2,0x92,0x60,0xe6,0xb5,0x89,0x40,0xdd,0x1e,0xfd,0xe4,0x5d, +0xe6,0xbe,0xf0,0xb0,0x72,0x5f,0x3a,0x80,0x3e,0x82,0x4f,0x36, +0x9f,0x1d,0x27,0xd6,0x83,0xba,0xf1,0x09,0xa3,0xbc,0xce,0x08, +0x0c,0x9f,0x16,0xd9,0xac,0x46,0x84,0x52,0x0d,0x2f,0x51,0x1d, +0x41,0xd0,0x93,0xe6,0x9c,0x6a,0x7c,0xc7,0x1b,0x6d,0x31,0x7c, +0xd9,0xd4,0xd3,0xa7,0xf2,0x82,0x9b,0xc9,0xa0,0xa5,0x69,0x8d, +0x68,0x8d,0x14,0x28,0xdd,0xf8,0xcb,0x88,0x22,0x63,0x97,0x14, +0x12,0xb9,0x7e,0xc4,0xa6,0x51,0x20,0x88,0xbe,0xdc,0xc3,0x4d, +0xe5,0xac,0xb3,0xd3,0x30,0x04,0xd8,0x3a,0xae,0x0f,0xe2,0x2e, +0x38,0xee,0xf0,0xb3,0x20,0x29,0xb3,0x54,0x20,0x91,0x64,0xd6, +0x28,0xa4,0xb2,0xe2,0x88,0xa1,0x23,0xb0,0xa9,0x1c,0x8b,0x9c, +0xd1,0x10,0x4d,0x50,0xca,0xe9,0x48,0x6d,0x43,0x1f,0x0c,0x18, +0xd7,0xc3,0xcc,0xca,0xc9,0xc3,0x02,0x79,0x63,0x87,0xb3,0xee, +0xf8,0xb7,0xf4,0x00,0x3a,0x3b,0x9d,0xcf,0xc6,0xe9,0xce,0x41, +0x76,0x54,0x27,0xe6,0xdb,0x6d,0x84,0x15,0xda,0x82,0xd3,0xaa, +0xa8,0xbf,0x84,0x4c,0x46,0xac,0x3f,0x15,0xbc,0x21,0xfc,0xee, +0x28,0x42,0xae,0x8d,0xd4,0x8a,0xb1,0x6b,0xe6,0x99,0x31,0x8d, +0xe2,0x01,0x86,0xc7,0xb1,0x71,0xf3,0x49,0x03,0x63,0xf9,0x29, +0x3c,0x03,0x74,0x21,0xc4,0x73,0x10,0x2e,0xb2,0x78,0x1e,0x7a, +0x5f,0x39,0x01,0x04,0xba,0x71,0x0a,0x8a,0x06,0x08,0xca,0xc8, +0x70,0x74,0x38,0x43,0x72,0xb4,0x05,0x52,0xfb,0x57,0x4a,0x10, +0x54,0x29,0x3e,0x7b,0x71,0x50,0x31,0x45,0xf7,0x6d,0x27,0x3f, +0x2b,0xa8,0x98,0x8c,0x01,0xc5,0x6c,0x64,0xed,0xc5,0x9d,0x26, +0xce,0xae,0xeb,0x6a,0xb9,0xa5,0xf3,0x47,0xbf,0x7f,0x11,0x7a, +0x10,0xbd,0xda,0x60,0x14,0xae,0x49,0x77,0xb3,0xb0,0x51,0x11, +0xcf,0x95,0xd6,0x18,0xec,0xce,0x78,0xd0,0x62,0xda,0x8a,0xe0, +0x26,0x78,0x48,0xa1,0x2c,0x56,0xd0,0x1e,0x91,0x0d,0xfb,0x79, +0xb2,0xba,0x61,0x36,0x8d,0xa5,0x98,0x00,0x32,0x0c,0x10,0x72, +0x61,0x68,0xa8,0xb6,0xb5,0x63,0x34,0xbf,0x7d,0x89,0xe8,0x81, +0x13,0x50,0xad,0x72,0x7c,0x07,0x0d,0xc9,0x3e,0x12,0xac,0xa6, +0xec,0x91,0x86,0x69,0x14,0xf7,0x97,0x38,0x2a,0xe3,0xc3,0x26, +0x38,0xb7,0x99,0x44,0x65,0xea,0xfe,0xff,0xfd,0x96,0x05,0x5b, +0xdb,0x71,0xf8,0x02,0x6a,0x18,0x85,0x51,0x28,0xfd,0xf7,0xb4, +0xf4,0x30,0x6e,0xa9,0x99,0x75,0x3e,0x59,0xa2,0x88,0x2d,0x2d, +0xa1,0xea,0x87,0x57,0x01,0x1d,0xf6,0x7f,0xae,0xf6,0x80,0xa4, +0xc0,0x96,0xb0,0xf5,0xc8,0x06,0xee,0xe5,0xaf,0xe3,0xf1,0xe2, +0x09,0xd0,0x02,0x91,0xac,0x2b,0x3e,0x83,0x58,0x27,0x49,0xc2, +0xad,0x66,0x1f,0x49,0xe9,0x8e,0x77,0xe5,0xb9,0x85,0x58,0x8e, +0x0d,0x16,0x20,0x3d,0xab,0x55,0x31,0x21,0x36,0x0e,0xa5,0xa0, +0xf4,0x99,0x05,0x11,0x49,0xcc,0x94,0xa2,0x7c,0x8d,0x18,0x62, +0xc8,0x58,0x00,0x31,0x18,0x07,0x07,0xad,0xdd,0x8e,0x5f,0x68, +0x28,0x6f,0x70,0x65,0x30,0xe9,0x45,0x86,0x05,0xfe,0x3a,0x9d, +0x5f,0x0c,0xa7,0x4f,0xd1,0x16,0x80,0xa9,0x21,0x4f,0x18,0xc3, +0x4d,0xea,0x55,0x05,0x86,0x37,0xb9,0x35,0xc1,0xd9,0x53,0x13, +0x19,0xfb,0x3d,0x23,0x49,0x59,0x1b,0xec,0xcc,0x3d,0x34,0x1a, +0xf4,0x44,0x14,0x5f,0x4f,0x7c,0x01,0xd5,0xda,0x1a,0x18,0x3d, +0xe5,0x71,0x21,0x44,0xa8,0x72,0x55,0x44,0xbc,0x20,0x9e,0xc0, +0x24,0x73,0x46,0x95,0x39,0xbb,0x5a,0x3c,0x3d,0x88,0x2e,0x2e, +0xba,0x83,0x7d,0xd7,0x91,0xa8,0x3c,0x9e,0x8d,0x9e,0x7e,0xa0, +0xd5,0x3c,0x92,0x19,0x72,0x31,0xde,0x0e,0xa9,0x8d,0x15,0x7f, +0x60,0x41,0x5c,0x6f,0xe7,0xb0,0xf8,0x02,0x30,0x2d,0x16,0xc7, +0xf7,0x82,0x72,0x45,0x2d,0x9d,0xf7,0x47,0x74,0xfd,0x87,0xf7, +0x2c,0x63,0x16,0x74,0x80,0x7f,0xa9,0x47,0xb8,0xb7,0xf0,0xe6, +0xf8,0xdf,0x40,0x0f,0x03,0xb4,0x03,0x10,0x14,0xe4,0x0d,0xd6, +0x9f,0xab,0x10,0x59,0xaa,0x61,0x90,0x99,0x51,0x33,0x0b,0x56, +0x10,0xb3,0xf2,0x9c,0x61,0x36,0x2f,0x8d,0x6c,0x49,0x70,0xc0, +0xf9,0x33,0x6f,0xe2,0x92,0x44,0x48,0xea,0x2f,0xd7,0x54,0xe7, +0x7a,0x6d,0x29,0x56,0xe6,0x52,0x71,0xeb,0x1d,0x0b,0xa5,0x99, +0xb8,0x68,0xb7,0x2d,0x25,0xff,0x70,0xc1,0x1a,0xe5,0x0e,0x85, +0x6b,0x71,0x21,0x08,0xc1,0xda,0xfb,0xa9,0x12,0xe2,0x80,0xb2, +0x8f,0x2f,0x5d,0x72,0x9f,0xe9,0x9a,0xac,0xa6,0x74,0x3e,0xc8, +0x70,0xd8,0x71,0x30,0xd9,0x51,0x91,0x1b,0x1e,0xe5,0x11,0x00, +0xf9,0xb3,0x7b,0x08,0xa4,0xd4,0xee,0xa3,0xc7,0x49,0x46,0x76, +0xd4,0x49,0xbd,0x47,0xdf,0xf0,0xec,0x46,0xa6,0xe9,0x6a,0xd8, +0xc2,0xe9,0x17,0xf7,0x70,0x1e,0x4e,0x12,0xb7,0x85,0xb7,0x69, +0x79,0x60,0xac,0xc3,0x6c,0x98,0xff,0xd3,0x44,0x1b,0x70,0x91, +0x2f,0x21,0xf8,0x06,0xdf,0xa8,0xad,0x53,0x8f,0x6c,0x35,0x20, +0x5a,0xd2,0x22,0x55,0x39,0xa0,0x04,0xff,0x1c,0x5a,0x46,0x6b, +0x44,0xd6,0xa7,0xa0,0x3a,0x5c,0xb6,0x8b,0x7c,0x45,0xc6,0x7a, +0x90,0xd1,0x04,0x3d,0x52,0x2e,0xa6,0xbb,0xfe,0xfa,0x94,0x73, +0xd0,0x80,0x24,0x10,0x6f,0x0b,0xa5,0x3e,0xe2,0x2b,0x4d,0x95, +0x13,0xa3,0xcb,0xf0,0x48,0x1f,0x78,0xd2,0x08,0xdd,0xc2,0x43, +0x31,0xcd,0xa4,0x54,0xc3,0x95,0x39,0x23,0x17,0x4d,0xb8,0x97, +0xb9,0x91,0x5f,0x7a,0xf9,0x12,0x88,0xf6,0xa5,0x1d,0x8a,0x4a, +0x87,0x8c,0x55,0x2e,0x54,0x61,0x22,0x6a,0x3c,0x47,0x54,0xf3, +0x62,0x88,0x3a,0xc7,0xa4,0x39,0x74,0x09,0x55,0xbf,0x83,0x0b, +0x0b,0x26,0x59,0x5c,0x38,0xf5,0x07,0xac,0x34,0x43,0x0c,0x3d, +0x56,0x52,0x25,0x85,0xce,0x9b,0x61,0xc5,0x7c,0x95,0x2a,0xaf, +0xe4,0x17,0xcd,0xbb,0x2b,0xa5,0xdd,0xf0,0xae,0x5d,0x09,0x87, +0x3a,0x9f,0xbc,0x9a,0x9f,0xe0,0x49,0x56,0x4e,0xf7,0x0e,0x79, +0x71,0x94,0x58,0x05,0xf2,0x69,0x1f,0x44,0xaf,0x65,0xd6,0xe1, +0x3c,0xf0,0x91,0x68,0x84,0xee,0xf0,0x71,0x39,0x6d,0x78,0x0c, +0x97,0xd9,0x8f,0x76,0x33,0xb1,0x99,0xa8,0x7a,0xb8,0x4b,0x94, +0x29,0x65,0x77,0xef,0x43,0x44,0x24,0x89,0xb8,0x7b,0x32,0x04, +0x7a,0x84,0x34,0x1f,0x47,0x1f,0xf9,0x46,0xa1,0xe5,0x4e,0xee, +0x90,0xec,0x63,0x83,0xd0,0x93,0x80,0x37,0x8d,0x7d,0x8f,0x08, +0x0f,0xdc,0xd8,0xa9,0x39,0xee,0x32,0x7a,0x11,0xb8,0x0a,0xb0, +0xb0,0xda,0x64,0x31,0x8d,0x3a,0x85,0x7a,0x74,0xbc,0x75,0xb8, +0x2e,0xee,0xe0,0x4a,0x75,0x1c,0x10,0x2e,0xdb,0x44,0xac,0x22, +0x4a,0xc7,0xe4,0x13,0xcd,0x69,0xa9,0x70,0xec,0xdb,0x0f,0x29, +0x09,0x6b,0xbc,0x27,0x05,0x9c,0x3d,0x4c,0x5f,0x39,0x15,0x12, +0x50,0x99,0xb8,0xd4,0xc6,0x16,0x97,0xf4,0xb7,0x7e,0xa7,0xf3, +0x3a,0x4a,0x56,0xcb,0x24,0xd3,0xd5,0x63,0x1f,0x53,0xe4,0x40, +0x72,0x6b,0xc9,0x21,0x41,0xaa,0x2a,0xc0,0x2b,0x2f,0x33,0x8b, +0xb0,0x14,0x30,0xce,0xd1,0x93,0x9c,0x88,0xa4,0xf7,0xe8,0x17, +0x25,0xe0,0x77,0xba,0xfa,0x50,0x76,0x1a,0x13,0x83,0xbb,0x95, +0x96,0x36,0x6c,0x78,0xf4,0x5a,0xec,0x81,0xbb,0x0e,0xf0,0x7b, +0x92,0xae,0x77,0x69,0x26,0x95,0xf1,0x78,0x5a,0xe6,0xa6,0x49, +0xa6,0xbd,0x79,0x08,0x54,0x80,0x29,0xa1,0xf4,0x24,0xeb,0xe1, +0x3b,0xc4,0xeb,0xe5,0xb1,0xa7,0xc5,0x39,0x95,0x09,0xb5,0x42, +0x23,0xa8,0xb2,0x49,0x26,0xb6,0x8d,0xa4,0x7d,0x24,0xbb,0x81, +0x2e,0x1d,0xa4,0x65,0xfd,0x70,0x30,0xf6,0x0e,0xe3,0xc9,0x37, +0x63,0xad,0x9d,0x59,0xc2,0x0e,0x80,0xcd,0xaf,0x23,0x8b,0xd8, +0xa1,0xaf,0xc0,0x0e,0xa6,0xd6,0x81,0xc2,0x57,0x9e,0xfd,0x2a, +0x2f,0x44,0x88,0x97,0x8a,0x1c,0x3c,0xe2,0x6a,0x7a,0x95,0xbe, +0x43,0x74,0x45,0x06,0xea,0xda,0x21,0xba,0x98,0x23,0xc6,0x91, +0xf1,0x0c,0x99,0xb1,0x36,0x0e,0x43,0xab,0x96,0x97,0xca,0x8f, +0x1a,0xbe,0x1f,0x7e,0x4a,0xef,0xef,0x96,0xd3,0x42,0x53,0xd8, +0xd8,0x7a,0xf2,0xd7,0xa7,0x67,0x09,0x61,0x44,0x67,0xf4,0x29, +0x53,0x91,0x0f,0xab,0xcf,0xb3,0xcb,0x02,0x16,0x3a,0x57,0x89, +0xa1,0x84,0xcc,0x52,0x55,0x09,0x09,0xcf,0x2a,0xfb,0xd1,0xb7, +0x28,0x95,0x1e,0x90,0xa0,0x8c,0x0b,0x3a,0x25,0xaf,0x19,0x13, +0x36,0x72,0x2f,0x46,0xf6,0x6b,0x0c,0xab,0x19,0x11,0x18,0x94, +0x68,0x0b,0xb6,0x07,0xe1,0x85,0x01,0x96,0x6a,0x1e,0xc8,0x48, +0x7c,0x80,0xef,0xce,0xb3,0x79,0x91,0x70,0x28,0x51,0x8a,0x16, +0xa3,0x24,0x98,0xe4,0xf6,0x96,0x28,0x12,0x26,0x24,0x35,0xf6, +0x98,0xe8,0xba,0x84,0xc8,0xbb,0x44,0xaf,0x57,0x64,0xa1,0x25, +0xe6,0xaa,0x4d,0xe2,0xb7,0xcf,0xf9,0x72,0x72,0x3d,0x99,0x0d, +0xa7,0x0f,0xbc,0x80,0xfa,0xeb,0xc6,0xd2,0x10,0x95,0x98,0xb7, +0x66,0xa1,0x3d,0x6a,0x41,0x19,0x0b,0x0a,0x92,0x8e,0x40,0xc9, +0x89,0x6e,0x3d,0x1e,0x61,0xe4,0xe3,0xd2,0x23,0x2d,0x55,0x39, +0x51,0x33,0x04,0x38,0x34,0xcc,0xfe,0x71,0x8c,0x34,0x15,0xad, +0x61,0x28,0x48,0xa4,0xf4,0xb5,0xcd,0x03,0x7e,0xe3,0x43,0x2e, +0x10,0xdc,0x5c,0x81,0xeb,0x41,0x71,0x1e,0xcc,0xb2,0x26,0xdb, +0xe4,0x4c,0xb6,0xbc,0x54,0x46,0xf4,0x03,0x3f,0xfe,0x7d,0x89, +0x88,0x20,0xc7,0x2a,0xb0,0x0d,0x84,0x73,0xcc,0xca,0xdf,0x4c, +0x8a,0xe4,0x93,0x1c,0x96,0xab,0x4b,0x98,0xcc,0xde,0x0c,0xaf, +0xfd,0xd1,0xf3,0x25,0x02,0xf1,0xc8,0xb3,0x3c,0x2d,0x36,0xfb, +0x47,0x3a,0xc7,0xdc,0x0d,0xa7,0xb2,0xb2,0x8c,0xa4,0xc0,0xc5, +0x58,0x47,0x71,0x93,0xef,0x93,0xce,0xd8,0xaa,0x98,0x74,0x12, +0xc0,0x23,0x7b,0xd4,0x44,0xc6,0x13,0x29,0x48,0x4c,0xda,0x1d, +0x34,0x69,0x6a,0x18,0xe7,0x06,0x21,0xe4,0x76,0xa2,0xce,0x3b, +0x7e,0x5a,0xee,0x7a,0x40,0xe8,0x31,0x58,0x9b,0x6a,0xa8,0x3e, +0xb4,0xa6,0x18,0xf5,0x69,0xc6,0x07,0x33,0x0d,0xa7,0x3b,0x19, +0xa2,0xc4,0x13,0xd5,0xcb,0xaa,0x91,0xea,0xd6,0xbc,0x61,0x8d, +0x7c,0x0f,0xc7,0x26,0xe3,0x03,0xd3,0x4c,0x55,0x29,0x67,0x39, +0x35,0x39,0xb3,0x73,0x55,0x5a,0x7a,0xdf,0x12,0xcd,0x7c,0x18, +0xfa,0x8c,0x44,0x3b,0x77,0x3a,0x93,0xcc,0xc2,0x85,0xc8,0x76, +0xfb,0x6a,0xf2,0xe9,0x44,0x87,0xf5,0x79,0x55,0xdd,0x8d,0x2b, +0x3e,0x0a,0xa3,0x8c,0x34,0xcd,0xc1,0x22,0x43,0x31,0xcf,0xfa, +0x3a,0xc3,0xb6,0xdb,0x06,0x9a,0x30,0xe9,0x9d,0xfa,0xd6,0x06, +0x3d,0xb4,0x1f,0x2e,0x9f,0xf4,0xf7,0x4b,0xdd,0xc2,0x6d,0x4a, +0x59,0x4f,0xe6,0x8b,0xcf,0xd4,0x8c,0x2d,0x3d,0x20,0x8b,0x03, +0x51,0x4e,0xda,0x3c,0x5c,0x91,0x53,0xd1,0xdf,0x36,0xf4,0xf6, +0xaa,0x37,0x79,0x8c,0x1f,0xe3,0x90,0x55,0x12,0x6c,0xb7,0x1d, +0xde,0x56,0xed,0xf6,0x16,0x06,0x4a,0x50,0xab,0x1d,0x41,0x36, +0x2d,0x81,0xa4,0x0a,0x35,0x02,0x75,0xc5,0x0c,0x16,0x15,0x1e, +0x03,0x5e,0x11,0x5b,0x6f,0x4e,0x3e,0xea,0x20,0xdd,0x1f,0x8e, +0xef,0x73,0xee,0xb4,0x3a,0xf1,0x5c,0xdd,0x2e,0x72,0xb4,0xa6, +0x4f,0xb7,0x39,0xf1,0x96,0xf3,0x69,0x20,0xc5,0x40,0x6f,0x59, +0x4d,0x6f,0x58,0xce,0xd0,0x25,0xb6,0xb1,0x72,0xc2,0xd1,0x5e, +0x0c,0xd8,0xf1,0xf7,0xe9,0x46,0xc6,0x80,0x57,0x82,0xc5,0xf4, +0x84,0x5f,0x0d,0x51,0x6d,0x4c,0x9f,0x75,0x60,0x63,0xc8,0x3e, +0x08,0xd0,0x1a,0xaa,0x21,0x0f,0xc9,0x75,0x66,0x97,0x15,0x7c, +0x0a,0xa9,0xd1,0x0a,0x43,0x6c,0x33,0x18,0x46,0xf8,0x6e,0x17, +0x25,0xfc,0x5b,0xaf,0xb1,0x2b,0x01,0x83,0x56,0x8d,0x7c,0x36, +0xbe,0xab,0xe1,0xb8,0x94,0x11,0x9d,0x2c,0x07,0xd8,0x56,0x32, +0x99,0x46,0x50,0x9f,0xc6,0xfa,0x50,0x74,0xe0,0x48,0xee,0xae, +0xbe,0x2d,0xe2,0x98,0x9b,0x23,0x0b,0x93,0x01,0x4e,0x47,0xb8, +0x51,0x0f,0x91,0xd5,0x1d,0xca,0xfb,0x68,0x90,0xbf,0xe7,0x52, +0x07,0x03,0xb6,0xd7,0x28,0x1d,0xf4,0xf6,0x53,0x88,0x33,0xf1, +0xcf,0x33,0x0b,0x84,0xf7,0x47,0xc3,0x2e,0xd2,0x1b,0xc1,0x40, +0xb9,0xe7,0xb2,0x06,0x00,0x7e,0x98,0xf0,0xe5,0x08,0xce,0xd2, +0xa8,0x19,0xb4,0xd0,0x32,0xa7,0x1e,0x2a,0x48,0xe6,0x0e,0x39, +0x96,0x58,0xaf,0x59,0x12,0xc0,0xb4,0xa2,0xe7,0xde,0x91,0x21, +0x8b,0x0c,0x55,0x53,0xb6,0x83,0x02,0x06,0x00,0xff,0x59,0x32, +0xe1,0xbd,0xc8,0x81,0x8c,0x43,0x19,0x2b,0xc5,0xc4,0x69,0xec, +0x6a,0xd8,0x2f,0x3d,0x92,0x46,0x25,0xd0,0xee,0xa5,0x22,0xbc, +0x80,0x3b,0x72,0xed,0x0d,0x74,0x47,0x8d,0xa3,0xc8,0xbd,0x25, +0x0d,0x82,0x86,0x15,0x8d,0xcd,0x35,0x0f,0x19,0xf4,0x6d,0xd0, +0xba,0x32,0x49,0x8e,0xc2,0x1e,0xd1,0x54,0x87,0x15,0x04,0xe9, +0x19,0xe6,0xa7,0x77,0x64,0xbf,0x16,0xd8,0xa2,0xcb,0x97,0xe7, +0x92,0x37,0xcc,0x91,0x6c,0x18,0x38,0xce,0x10,0x4a,0x54,0x95, +0xc3,0x46,0x9c,0xfd,0x57,0xd9,0x4c,0x72,0xf5,0x29,0xda,0x0a, +0xcb,0x38,0x35,0x34,0x48,0x95,0x8b,0x1e,0x29,0x12,0x62,0x9f, +0x9e,0xf0,0x36,0x90,0x42,0x7c,0xc4,0xd3,0x5a,0xa0,0xac,0xe2, +0x49,0x04,0x2d,0x27,0xef,0xb8,0xf3,0x6b,0xbd,0xde,0x3b,0xf4, +0xc7,0x84,0x95,0x5c,0x36,0x67,0x1c,0x22,0xe6,0x4e,0xf1,0xe6, +0x6b,0x11,0x18,0xf2,0x86,0x22,0x4d,0xad,0x1d,0x0f,0x9c,0xc3, +0x1f,0xfc,0x0a,0x2d,0x3e,0xeb,0x69,0x64,0xe4,0xd0,0xe5,0x1b, +0xfc,0xbd,0xf6,0x09,0x92,0xc4,0xb5,0x7f,0x8d,0x24,0xa1,0x26, +0x19,0x92,0x44,0x49,0xa3,0x46,0x6f,0x29,0x7c,0x09,0x60,0x0b, +0x91,0x85,0x2c,0xfc,0xb1,0xf8,0xfc,0x47,0x9e,0xe9,0x62,0xd5, +0xf0,0x36,0x41,0x56,0xbd,0x26,0x23,0x42,0x14,0x09,0x33,0x9e, +0xcc,0xd0,0xa2,0xdc,0x70,0xfa,0x1f,0xe3,0xcf,0x46,0x41,0x08, +0xf5,0xd3,0xc4,0x62,0x92,0x8e,0x1f,0xd9,0xc9,0x1a,0x01,0x10, +0x38,0x1e,0xc5,0xcb,0x73,0xb4,0x04,0x82,0xc4,0x9c,0x2c,0xf2, +0x0e,0x9d,0x60,0x35,0xaa,0xb2,0xae,0xba,0x12,0x19,0x30,0xf5, +0xf5,0x62,0xc8,0xf4,0xba,0x48,0x7d,0xeb,0xdd,0x42,0xf2,0x51, +0xb2,0x3c,0x61,0x8a,0xd1,0x17,0x1a,0x8a,0xb2,0x99,0x2e,0x0c, +0xa8,0x63,0xe5,0x64,0xa4,0x96,0x87,0x26,0xa3,0x01,0xbb,0x0f, +0xc7,0x78,0x6e,0x04,0x63,0x27,0x38,0x10,0xad,0x09,0xb3,0xd3, +0x35,0x5e,0x5a,0xc7,0xa6,0x7c,0x7a,0x41,0xc3,0x85,0x6b,0xe5, +0xc6,0xd0,0x11,0x66,0x9c,0x60,0xb0,0x08,0x1a,0x61,0x82,0x7a, +0x43,0x16,0x94,0x45,0x61,0x0f,0x5b,0x02,0x6d,0xe6,0x31,0x6a, +0xb9,0xa8,0x70,0x44,0x7b,0x92,0x5c,0xeb,0x56,0x61,0x8d,0xfb, +0x71,0x95,0xd6,0xbe,0x1f,0x11,0x16,0x81,0x4f,0x9b,0xa6,0x8c, +0xa9,0x81,0xc8,0x3a,0x86,0x41,0x1d,0x82,0x5d,0x20,0x28,0x6e, +0xc6,0xe8,0xf9,0x48,0x75,0x4b,0x47,0x28,0x6d,0x2e,0x12,0x73, +0x2c,0xf2,0x8a,0x9c,0x85,0x53,0x94,0xb8,0xaa,0x50,0x96,0xef, +0xe4,0xf4,0x34,0x5f,0x0e,0xa7,0x8b,0x9b,0x61,0xb9,0x4f,0x3f, +0xe7,0x69,0xff,0x97,0x6c,0xf0,0xcd,0x39,0x49,0x4a,0xcd,0xe1, +0x3e,0x98,0xac,0x3e,0x97,0xfb,0x12,0x38,0xaf,0xbe,0x41,0x71, +0x2a,0xce,0x82,0x22,0x50,0x0b,0xd1,0x9f,0x45,0xc9,0xac,0xd5, +0x7c,0xb1,0x06,0xc2,0xe9,0x66,0xb5,0xbe,0x98,0xaf,0x56,0xf3, +0xdb,0xf5,0x74,0x7c,0xb5,0x22,0x27,0xa8,0xa3,0x49,0x05,0x77, +0xe2,0xe7,0xea,0x23,0x5c,0xac,0x90,0x71,0x86,0x84,0x05,0x5d, +0x09,0x69,0x6f,0x67,0xef,0xb2,0x3f,0x1e,0x0e,0xb2,0x2e,0x09, +0x91,0xdd,0x0e,0x97,0x40,0x78,0x41,0x1e,0x0e,0xa0,0xcc,0xd8, +0xdd,0x2d,0x9a,0x30,0x5e,0x95,0x35,0x5f,0xbc,0x44,0x92,0x41, +0x32,0xfa,0x77,0xee,0x7e,0x93,0xed,0xaa,0x2c,0x1c,0x44,0x41, +0x05,0x8b,0x4f,0x0f,0x97,0xe8,0xed,0x2c,0x3e,0x65,0xfd,0xe1, +0xde,0x3f,0xfe,0x75,0xd0,0x71,0x45,0x97,0xe3,0xe9,0xab,0xbb, +0xc8,0x81,0x33,0xf9,0xc1,0xcd,0xca,0xa8,0xbc,0x94,0xc0,0xa9, +0x90,0xce,0x95,0xf7,0x3f,0xbc,0x7e,0xf2,0x5f,0x40,0x33,0x4f, +0xe7,0x97,0xbf,0x02,0x15,0x7c,0x59,0x55,0xa7,0x37,0xf3,0x8f, +0xe5,0xbd,0x0e,0x11,0x50,0xce,0x17,0xd5,0x7c,0x0a,0xd3,0x99, +0xe4,0x1f,0x26,0xd5,0xe4,0x62,0x02,0xdd,0xfa,0x5c,0x24,0xec, +0x30,0x06,0x1d,0xc5,0x10,0x9c,0x00,0xc2,0xab,0x39,0x0c,0xc9, +0xf4,0x6c,0x39,0x9c,0x55,0xb0,0xf4,0x6f,0xcb,0x7b,0x98,0x5d, +0x98,0xf7,0x53,0x9c,0x8d,0xd9,0x75,0x71,0x90,0x5f,0xc1,0x71, +0xf6,0xf3,0x18,0x07,0xbd,0xf8,0xee,0xe0,0x80,0xca,0xf0,0x8a, +0x2c,0xfb,0xc9,0xd9,0x7c,0x01,0xcd,0x7c,0x8b,0x89,0xf0,0xfb, +0x03,0xcd,0x09,0x04,0x5e,0xc0,0xac,0xa0,0x6c,0x56,0x55,0xbd, +0x59,0xc2,0x9a,0xfb,0x84,0x88,0x5f,0xf2,0x33,0x39,0x57,0x81, +0xd4,0xd7,0xf0,0xef,0xe5,0xfc,0x1f,0xf0,0xf7,0xb6,0x4a,0x06, +0x0d,0xfa,0x81,0x50,0x2e,0xb2,0x2f,0xf9,0xf5,0x6f,0x3f,0xcc, +0x76,0x36,0x65,0x70,0x55,0xc2,0xc5,0x9b,0x57,0xbc,0x1e,0x6f, +0x51,0x2a,0x8e,0x9e,0x81,0x8e,0xac,0x1c,0x0e,0x5f,0x15,0x6c, +0x4e,0x9c,0x0e,0x45,0xce,0x5e,0xba,0x85,0x2c,0xa7,0x3c,0xaa, +0xbe,0x91,0x79,0xf4,0x9a,0x3f,0x76,0x00,0x4c,0x4c,0x07,0x38, +0xac,0x9c,0x32,0x29,0x74,0xc4,0x37,0x08,0xd1,0xdb,0x9d,0x43, +0x69,0x87,0xd1,0xc7,0x76,0x2e,0xae,0xa8,0xc1,0x3b,0x0d,0x72, +0x4c,0x54,0xa4,0xd6,0xb3,0xa2,0xa9,0x12,0x7c,0xb2,0x70,0x59, +0x6a,0x4f,0x16,0x8f,0x0f,0x51,0xc0,0x1e,0x96,0x4c,0xc3,0xd3, +0x39,0x46,0xff,0x38,0x19,0xc9,0xf3,0x1c,0x31,0xde,0x61,0xd9, +0x34,0x3d,0xf1,0x07,0x39,0xd1,0x61,0xe0,0xfc,0xfa,0x7a,0x6a, +0x9f,0xc0,0x57,0x64,0x6d,0x1c,0x07,0x1e,0x5d,0x7d,0x96,0xce, +0xe1,0xa7,0x97,0x0b,0xc7,0x1c,0x0f,0xc8,0x99,0x64,0xf7,0x29, +0x96,0xe9,0x51,0xbe,0x62,0x52,0xfd,0x48,0xeb,0x97,0xeb,0x73, +0x8a,0x71,0xfc,0x06,0x81,0x8d,0x49,0x75,0x2c,0x24,0x0e,0xdb, +0x9d,0xe2,0xab,0x43,0x03,0x5f,0xa4,0xaa,0x7e,0x9c,0xcf,0x7f, +0xad,0x8a,0x7b,0x39,0x6e,0x8a,0xfb,0x9a,0x49,0x9c,0x1c,0xb5, +0xa7,0xef,0xd0,0x62,0x2f,0xde,0x0d,0xfe,0x43,0xa5,0xe6,0xf9, +0x48,0x13,0xea,0x4c,0xc0,0x78,0xef,0xa6,0x09,0x59,0x0d,0x5d, +0xf5,0x92,0xc3,0x04,0xa5,0xd2,0x37,0x68,0x97,0x0d,0xb7,0xd9, +0xdd,0xed,0x05,0x5a,0xce,0xbd,0x84,0xed,0x79,0x3b,0x3b,0x99, +0xdf,0xcd,0x56,0xa2,0x5f,0x3f,0x7d,0x2d,0x2d,0xd9,0x09,0xf6, +0x1a,0x7c,0xe1,0x8b,0xe2,0x8f,0xee,0x6b,0xee,0xb3,0xcd,0x97, +0x70,0x8a,0xce,0x48,0xdd,0xf6,0xe3,0x64,0x34,0xff,0x48,0xa1, +0x7f,0x90,0x56,0x3f,0x85,0xe6,0xf3,0x5b,0x32,0x98,0x4d,0x1b, +0x70,0xbe,0x80,0xde,0x26,0x57,0xd3,0xf9,0x70,0x95,0xc4,0x02, +0x8b,0x90,0xe1,0x19,0x26,0xf4,0x12,0x0d,0x25,0x45,0x42,0xcb, +0x8d,0x3f,0x36,0xbc,0x5c,0x8b,0x6d,0xdb,0x2b,0x87,0x71,0x5d, +0x0e,0x69,0x9c,0x18,0x87,0xfe,0xb6,0xce,0xd5,0xf8,0x73,0x3d, +0x8a,0x3e,0x09,0xb2,0x1b,0x55,0xbe,0x1a,0x6f,0x70,0x6e,0x88, +0x9f,0x47,0xd2,0xbf,0x0e,0xbd,0xb8,0x1d,0x4f,0x89,0x72,0xa2, +0x25,0xce,0x6d,0x2a,0x3d,0x10,0x32,0x04,0x6f,0xf3,0x4b,0xb7, +0xfb,0x0a,0x08,0xad,0xf2,0x6d,0x4d,0x2b,0xe1,0x7a,0x1e,0xcd, +0x97,0x18,0x4b,0xa8,0x3d,0x81,0x74,0x6d,0x00,0xda,0x84,0x1a, +0x65,0x60,0xd3,0x02,0x62,0x06,0xa3,0x17,0xaf,0xd0,0x68,0x07, +0x36,0xdf,0xfa,0xa8,0x4c,0xf0,0xda,0xed,0x04,0x16,0x1e,0x20, +0xbc,0xfa,0x49,0x5a,0x09,0x14,0x26,0x86,0x9f,0x3c,0x66,0xca, +0x00,0x67,0xc1,0xb1,0x00,0x39,0x0b,0x6a,0x25,0x37,0xe2,0x48, +0xbc,0x42,0x06,0xef,0x46,0xb9,0x7f,0x4c,0x14,0x03,0x9c,0xac, +0xf6,0xc0,0xf7,0x8f,0x7d,0xce,0x45,0x6c,0x89,0xdb,0x8a,0x39, +0x90,0xc6,0x3c,0xcc,0xbe,0xc1,0xd0,0xa3,0x41,0x67,0x31,0x5c, +0x56,0xbc,0x14,0xd2,0xc6,0xd3,0x26,0xa3,0x69,0x2b,0x93,0x19, +0xad,0x6e,0xf2,0x30,0xc2,0xd2,0x03,0xf2,0x76,0xa5,0x09,0xae, +0x11,0xe8,0x26,0xea,0x95,0x7b,0x47,0x4e,0x35,0x5d,0xec,0xbf, +0xda,0xe1,0xe4,0x0d,0x13,0x4c,0x21,0x15,0xeb,0x94,0xc9,0x02, +0x1d,0x5f,0xc5,0x2b,0x19,0x4e,0x98,0x25,0x71,0xa9,0xe8,0xbc, +0x46,0x64,0x79,0xc7,0x35,0xe3,0x40,0x8c,0x3a,0x7a,0x7b,0xda, +0xf8,0xbc,0x81,0x92,0xac,0xb3,0x51,0x42,0x3a,0x2d,0x7e,0x34, +0x4b,0x98,0x93,0x9b,0xf1,0x72,0x82,0x98,0xbc,0xce,0x54,0x15, +0xce,0x14,0x8f,0x16,0xcf,0x55,0xa5,0x73,0x65,0xf7,0x43,0x16, +0x4c,0x3b,0x7b,0x43,0xb1,0x55,0x50,0x5e,0xfb,0x10,0x4d,0x5b, +0xb5,0x69,0x97,0x11,0x3c,0xbd,0x33,0xf8,0xdd,0xf6,0x8e,0xea, +0xfa,0xba,0xad,0xe2,0x0c,0x2a,0xff,0xb7,0xed,0x0e,0xbf,0xeb, +0xfe,0x1b,0xb6,0xc8,0x96,0x9d,0x80,0x9e,0x40,0xe2,0x9d,0x70, +0xa0,0x43,0x8b,0x9d,0xb7,0xc3,0x2b,0xf9,0xed,0xa1,0x4c,0x23, +0x27,0x63,0x86,0x76,0xf0,0x09,0xc3,0xc1,0x35,0x48,0x1e,0x41, +0x30,0x15,0x91,0xfb,0x3a,0xee,0xa3,0xa0,0x6a,0x09,0xe2,0xe8, +0x24,0xa7,0xc3,0x9d,0xda,0xb1,0x5e,0xd3,0x4f,0x0f,0x96,0xfb, +0x6d,0x69,0x76,0x09,0x94,0xcf,0xb8,0xa5,0x64,0xf6,0xca,0xf0, +0x6f,0x61,0x3d,0xc3,0xaa,0xba,0xc4,0x02,0x68,0x4b,0x1e,0xb9, +0x5c,0xe8,0xa1,0x24,0xc3,0x3f,0x70,0xcc,0x7e,0xb4,0x92,0x91, +0xd4,0x0b,0x91,0x10,0x72,0xaf,0x71,0xfc,0x8c,0xed,0xce,0x4b, +0x7e,0xf9,0x98,0x8e,0x00,0x9b,0x21,0x6a,0x5c,0xbb,0x25,0xc5, +0x32,0x48,0x92,0xd5,0xe6,0xa7,0x8c,0x23,0xf2,0x38,0xa2,0x94, +0x32,0x72,0x9c,0xd0,0x1d,0xa7,0x4f,0x80,0xfc,0x18,0x41,0x0d, +0xc2,0xfa,0xd1,0x24,0x6e,0xd6,0x58,0x5f,0x1d,0xa8,0x36,0xc0, +0x7a,0xa0,0x80,0x1b,0x59,0xac,0xca,0xc1,0xd4,0x9e,0xc8,0xd5, +0x4a,0x3b,0xb6,0x97,0x3a,0x5c,0x2b,0x76,0x09,0x22,0xe5,0xb7, +0x14,0x94,0x39,0x87,0x63,0x07,0xf5,0xf4,0x68,0x19,0x34,0x78, +0xcc,0xc9,0xdf,0x85,0x17,0x39,0xdc,0x9b,0xab,0x9b,0xfc,0x76, +0x32,0xfb,0x99,0x03,0xc3,0x4f,0x1c,0xd0,0x5c,0xa5,0xcb,0x4f, +0xbc,0xe6,0x00,0x09,0xa4,0x01,0x92,0xd4,0x9e,0x06,0xb0,0x5d, +0x6a,0x85,0x94,0xdc,0x74,0xa8,0xc2,0x9c,0x66,0xe0,0xc1,0xf0, +0x0f,0xe5,0xf5,0x6b,0x4c,0xed,0xac,0x48,0x01,0x34,0x1d,0x80, +0x87,0xd8,0x12,0x9d,0x24,0x7d,0x2d,0xa7,0x02,0xf6,0xb2,0x51, +0xb5,0x8b,0x30,0xc7,0xcc,0xd0,0x2e,0xcc,0x5d,0xc0,0xe7,0xa3, +0x76,0x5b,0x88,0xa2,0x50,0x4b,0x8e,0x46,0xa8,0x24,0x08,0xdd, +0x70,0xb4,0x24,0xb2,0x36,0x78,0x1a,0xaf,0x63,0x19,0x66,0x8b, +0x52,0x2d,0x64,0xec,0x62,0x30,0xaa,0x52,0xa1,0xcd,0xf2,0xb1, +0x09,0xa4,0x6b,0x42,0x04,0x5b,0x03,0xac,0x57,0xb8,0xf1,0x8e, +0x89,0x62,0x17,0x82,0x5d,0x58,0x30,0x28,0xbb,0x42,0xf3,0x0b, +0x9d,0xfe,0xd2,0x1a,0x64,0x8e,0x9f,0x29,0xf3,0x3b,0x96,0x1c, +0xd2,0xaa,0xf9,0xb2,0x82,0xff,0x91,0x3e,0xfa,0xe7,0x57,0xda, +0x57,0x2f,0x24,0x35,0xa3,0x8f,0x9e,0x9f,0xb8,0x7f,0x66,0x7f, +0x0a,0x42,0x60,0x62,0xb6,0x2c,0x8e,0x1d,0x47,0x81,0x87,0xeb, +0x03,0xbb,0x23,0x93,0x28,0x3d,0xe3,0x26,0x90,0xce,0xe8,0x2d, +0x5f,0xc3,0xd2,0xd7,0x72,0x89,0x2e,0xc4,0x2a,0xcd,0x87,0x51, +0x58,0x3d,0x47,0x94,0xb5,0x31,0xa5,0x68,0x41,0xf3,0x38,0x4b, +0x82,0x88,0xf1,0xe9,0xe4,0x1f,0x63,0x35,0x4b,0x07,0xe8,0xf5, +0xf8,0x96,0x10,0xec,0xdc,0xf5,0xa2,0xbb,0x00,0x4a,0x73,0x8a, +0xc0,0x3b,0x88,0x22,0xd8,0xf2,0xcd,0x15,0x73,0x44,0x26,0xe7, +0x3a,0x21,0xed,0xc3,0xbb,0xd5,0x9c,0xf1,0xf6,0xf0,0x6d,0xbb, +0x9f,0xdc,0x8c,0x85,0xca,0xa5,0x85,0x08,0xd4,0xad,0x97,0xaa, +0x11,0x97,0x5c,0x8d,0xd7,0x5f,0xf9,0x00,0x7d,0xa1,0xd8,0x73, +0xb4,0xef,0x7b,0xfa,0xfc,0xd1,0x9d,0x5f,0x5d,0x01,0x7e,0x41, +0x2b,0x19,0xc6,0xcf,0xb0,0x38,0x78,0x26,0x62,0x94,0x2c,0x91, +0x1c,0x89,0x23,0x92,0xba,0x98,0x59,0x2a,0x65,0x66,0x41,0x5e, +0xa7,0xe7,0xae,0xa5,0x8a,0xd7,0x4b,0x26,0x36,0x62,0xf4,0x03, +0xd5,0x19,0xbe,0x98,0xc7,0x39,0x42,0xae,0x6a,0xdd,0x0d,0x28, +0x05,0x92,0x3d,0xe2,0x1d,0x46,0x31,0xf4,0xd8,0x65,0xd7,0xbc, +0x73,0xe8,0x3f,0x16,0xa7,0x8c,0x1f,0xc6,0x8c,0x0a,0xd6,0x80, +0xf5,0x86,0x77,0xf4,0xc6,0xf4,0x60,0xc3,0x62,0x3c,0xf1,0x62, +0xfe,0x09,0xd6,0x11,0x20,0xc6,0x68,0xff,0x70,0x8e,0x5e,0xb2, +0xf6,0x20,0xca,0xbc,0x2e,0xf9,0xd1,0x74,0x59,0x13,0x43,0xb7, +0xeb,0x6f,0x71,0x40,0x4c,0xaf,0x18,0xbc,0x10,0x69,0x01,0x46, +0x45,0xab,0x41,0x53,0x1e,0x5a,0x0f,0xde,0x0e,0xa4,0x64,0xe6, +0x69,0x4e,0xfd,0x85,0x50,0xdb,0x2a,0xbd,0xfa,0xe6,0x11,0x25, +0x65,0x7f,0x1c,0x48,0x0c,0x3b,0xf0,0xe8,0x75,0x0f,0x0e,0xbf, +0x31,0x08,0x0b,0x33,0xa1,0xba,0xbb,0x87,0x59,0x27,0x49,0x0a, +0xb7,0x04,0x91,0x7a,0x4d,0x92,0xad,0xb3,0x69,0xe6,0xd1,0x9c, +0x3b,0xb9,0x6d,0x47,0x7d,0x5b,0x2b,0x05,0x5b,0xd6,0x70,0x22, +0x86,0x09,0xbb,0x0f,0xb9,0x83,0xa9,0x66,0x4b,0x3a,0x87,0x07, +0x07,0xdf,0x30,0xd2,0x9f,0x64,0xd8,0x1e,0x31,0xdf,0x53,0x86, +0x87,0x76,0x43,0xef,0xd7,0xeb,0x2a,0xf8,0x4a,0x92,0x23,0x8e, +0x40,0xd2,0xb8,0x3c,0xcc,0xb9,0xc6,0xc7,0xe5,0x21,0x26,0x95, +0x42,0x27,0xa0,0x85,0x4a,0xfb,0xcc,0xb8,0x9c,0xdc,0xa6,0xe2, +0x94,0xde,0xfb,0x56,0xc4,0x16,0x92,0x78,0x8c,0x1c,0xa6,0x31, +0x1f,0x15,0x2d,0x50,0x37,0xc5,0xa7,0x09,0x83,0x4a,0xe4,0xb4, +0x11,0xca,0x24,0xec,0xc9,0x4e,0x43,0x57,0x1c,0x79,0x22,0xdf, +0x25,0xb7,0x81,0x97,0x86,0x64,0xe9,0x35,0x37,0x53,0x06,0x32, +0x2b,0x38,0x19,0x8d,0x40,0x75,0x34,0xce,0xaf,0xdd,0x06,0x5d, +0x1b,0x5d,0xcd,0x40,0x2c,0x4e,0x90,0x87,0xfa,0x92,0xd0,0x03, +0x62,0xf3,0x35,0xac,0xec,0x5b,0x9f,0xfa,0x55,0xab,0xdb,0x2d, +0xb2,0xda,0x19,0x75,0xef,0x18,0x94,0x2c,0x4e,0xbd,0xe7,0xf8, +0x94,0xcc,0x45,0xee,0xf3,0xc6,0x34,0x15,0x26,0x03,0x77,0xf2, +0x6c,0xea,0xaf,0x88,0x74,0x21,0x78,0x0f,0xb5,0x9e,0xd3,0xb8, +0x88,0xe3,0xe4,0x90,0x5f,0x11,0x3f,0x73,0xca,0xec,0x4b,0x73, +0xc0,0x2f,0x60,0x47,0xd6,0x0f,0x78,0x8c,0x7d,0xf0,0x80,0xaf, +0x77,0xd9,0xed,0xe5,0x80,0x60,0x21,0xf0,0xf1,0xed,0xeb,0x80, +0x28,0xc3,0x8b,0x8e,0x47,0xd7,0xf4,0x34,0xe3,0xfa,0xe9,0xae, +0x73,0x1b,0xd7,0x8f,0xc6,0x26,0xb3,0xdc,0xaf,0xc5,0xd2,0x77, +0xd5,0x58,0xe7,0xf3,0xd2,0xd4,0x36,0xb6,0xcb,0xfc,0xe2,0x87, +0x50,0x20,0x73,0x33,0x7d,0x5f,0x1e,0xc8,0xc1,0xc4,0x91,0x7c, +0x0e,0x43,0x6c,0x5d,0x12,0x5d,0xd7,0x13,0xf3,0xf3,0x5e,0x53, +0x76,0xa4,0xff,0x90,0x5b,0x8f,0x9b,0xc2,0x50,0x97,0x96,0x49, +0xd4,0x95,0x65,0x61,0x49,0xc9,0xda,0x8d,0xb7,0xc9,0x9b,0x7a, +0x42,0x5c,0x70,0x6b,0x2a,0xd2,0x76,0x65,0x67,0x7b,0xdf,0x55, +0x29,0x2b,0x14,0x70,0xe3,0x85,0x87,0x07,0xd1,0x62,0x38,0xc2, +0x17,0x6d,0x0c,0xf2,0x2d,0x52,0x24,0x34,0x18,0x56,0x36,0x6d, +0x41,0xbc,0xef,0xbc,0xba,0xbb,0x82,0x9f,0xa6,0xd5,0x83,0xc9, +0x1d,0x4e,0x86,0x65,0xc4,0x2f,0x6b,0x35,0x39,0x72,0x63,0x06, +0x34,0xe7,0x2c,0x63,0xa4,0xf9,0x50,0xe6,0x7a,0x55,0xd5,0x4d, +0x22,0x51,0xa9,0x1e,0x4b,0xed,0xd2,0xab,0x06,0x3a,0x0c,0xca, +0x0a,0x91,0x05,0x3f,0xfa,0x4e,0x05,0xed,0x05,0x92,0xb6,0xc2, +0x31,0xf5,0xfb,0x93,0x81,0x6b,0x12,0x55,0x41,0x92,0xf8,0x12, +0xda,0x7b,0xe4,0xc2,0x07,0x8e,0xac,0x53,0x50,0x9b,0x4d,0x1e, +0xd0,0x11,0x0c,0x39,0xab,0x1f,0x18,0x51,0xcf,0x91,0x85,0x52, +0xd6,0x6e,0xfb,0x4c,0x1e,0x94,0x96,0x8f,0x0e,0xca,0xfd,0x7f, +0x7d,0x74,0x80,0xb6,0x0b,0x2e,0x96,0x40,0x91,0x42,0xe6,0xfd, +0xf3,0xfe,0xf9,0x00,0x1f,0x7d,0x4e,0xde,0xbe,0x78,0x06,0x5f, +0xcb,0xde,0xf9,0x0c,0xd3,0xd9,0xc0,0x28,0xcc,0x22,0x72,0x19, +0x2b,0xd6,0xe4,0xe7,0xb8,0x35,0x1b,0x35,0x5e,0x93,0xe9,0xd2, +0x35,0x99,0x1f,0x5d,0xa3,0x69,0x53,0xd6,0xec,0x97,0x72,0x24, +0xf0,0x4b,0x85,0xd8,0x54,0x01,0xbf,0x19,0xaf,0xf1,0x15,0x9c, +0x8c,0x1a,0xfc,0x3a,0xfe,0x7c,0x3d,0x9e,0x65,0xfb,0x93,0x86, +0x27,0x8b,0x4a,0x15,0x33,0x1a,0xf8,0xe5,0x92,0x9b,0x94,0xe2, +0xd5,0x48,0x9f,0xe4,0xe6,0x67,0x07,0x5c,0xbc,0x61,0xd4,0x36, +0xd5,0xb6,0x48,0xbb,0x46,0x5f,0x6e,0x49,0x08,0x48,0xab,0x81, +0x43,0x41,0x74,0x24,0x35,0xc9,0x31,0xa7,0x5d,0xe6,0x9e,0x13, +0xa0,0x55,0xbb,0x58,0x9a,0xc2,0x92,0xba,0x9b,0x26,0xfb,0x88, +0x58,0x1b,0xf3,0x19,0xb1,0x29,0x18,0x0a,0x58,0xf8,0x6c,0x54, +0x7a,0x27,0xe0,0xc4,0x4f,0xaa,0xd4,0xba,0xff,0x47,0x22,0xc0, +0x8f,0x35,0x2f,0x13,0xa7,0x84,0xf8,0x8a,0xe9,0x8a,0x9d,0x68, +0x1a,0x25,0x17,0x6b,0x28,0xaa,0x56,0xa7,0x08,0x12,0xec,0x3c, +0x60,0xc8,0xc1,0x94,0x43,0xdd,0xcf,0x60,0xe4,0x26,0x62,0x32, +0x44,0x9c,0xe8,0x06,0xea,0xb8,0xe8,0x1f,0x36,0x8d,0x3c,0x91, +0x41,0x14,0xbb,0xff,0x8b,0xde,0x8b,0x90,0x7f,0xd3,0x33,0xb2, +0xc8,0xc8,0x97,0xb3,0xfb,0x57,0x27,0xef,0x9e,0x3c,0xca,0x30, +0x13,0xdc,0xf1,0xce,0x91,0xf1,0xe3,0x6f,0x6c,0x5c,0xca,0x79, +0x72,0xbe,0x3c,0x9f,0x25,0x78,0x37,0x17,0xbf,0xbf,0x08,0xcb, +0x89,0x1a,0x9c,0x94,0x16,0x9c,0x3f,0xf9,0x86,0xf9,0x0a,0xad, +0x59,0x60,0x78,0x28,0xca,0xc0,0x7a,0x42,0xa1,0x58,0x16,0x1c, +0x68,0x3e,0x2f,0x2c,0x75,0x8f,0xe8,0x79,0xdd,0x93,0xba,0x32, +0x03,0x9f,0x34,0x69,0x56,0x18,0xb6,0x6f,0x0f,0x10,0x49,0x46, +0xd3,0xab,0xbe,0xbe,0x3c,0x0d,0xca,0xf1,0xec,0x12,0x26,0xf9, +0xa7,0xb7,0xcf,0x91,0x79,0x03,0xe7,0xfd,0x6c,0x85,0xb5,0x00, +0xd6,0x09,0x78,0x5e,0x43,0x9a,0x28,0xe0,0x10,0x6f,0xdb,0x37, +0x3b,0x62,0xfc,0xd9,0x14,0x23,0x63,0x7d,0x3a,0x5e,0xad,0xd0, +0x84,0x94,0xbb,0xf2,0x6c,0x64,0xd7,0x8e,0x42,0x1e,0x4d,0xe8, +0x10,0x0d,0x8d,0x8b,0x99,0x3b,0x2f,0xfd,0x3d,0xa9,0xde,0x4c, +0x87,0x93,0x19,0x9b,0x5a,0x83,0x3c,0x99,0xbd,0x14,0x86,0x96, +0xa2,0x42,0xcb,0x6d,0x6e,0x33,0xb0,0x3a,0x8e,0x74,0x24,0xf3, +0xce,0x8c,0x78,0xd4,0x91,0x77,0x36,0xcc,0x48,0x50,0xef,0x0d, +0x4e,0x54,0xa5,0xf7,0xc5,0x50,0xce,0xc7,0x81,0x9d,0x2f,0x9c, +0x1e,0x4f,0x1a,0x75,0xdf,0xcf,0x27,0xb3,0x34,0x69,0x5b,0xf1, +0xed,0x47,0x07,0xe8,0x44,0xd5,0x5c,0x83,0xd8,0xb8,0xe4,0x62, +0x7a,0xb7,0x6c,0x91,0x87,0x05,0xfe,0x0b,0x95,0xd2,0xef,0xfc, +0x6e,0xd5,0x02,0xfc,0x7f,0x84,0xfe,0xf8,0xe0,0xd0,0x69,0x55, +0x97,0xcb,0xf9,0x74,0xda,0xba,0x9b,0x51,0xe4,0xe5,0x74,0x72, +0xf9,0x6b,0x6b,0x74,0x31,0xe5,0xc0,0xed,0xfc,0xae,0x1a,0x8f, +0xe6,0x1f,0x67,0x1c,0xba,0x5b,0xf0,0x2f,0xe2,0xb7,0x1c,0x82, +0xc0,0x52,0x42,0x00,0x97,0x02,0x63,0x14,0x1e,0xe0,0xe0,0x74, +0x3c,0x84,0x8c,0x97,0x37,0xc3,0xd9,0xf5,0x58,0x4c,0x42,0xb5, +0x78,0x8f,0xb7,0x60,0x05,0x10,0x5c,0xf8,0x25,0x0f,0xca,0x18, +0x00,0xf0,0xe4,0x0e,0xa4,0x25,0x12,0x48,0x70,0x2c,0xdd,0x25, +0xe6,0x2e,0xdb,0x4a,0x78,0xd7,0xa5,0xbe,0x49,0x90,0x05,0x2d, +0x63,0xc9,0xc0,0xd5,0xde,0x43,0x0f,0x58,0xd3,0x5f,0x9f,0x99, +0x49,0xc8,0x41,0x0b,0xb1,0x0c,0x37,0x5c,0xaf,0xd7,0xd7,0x63, +0x66,0x77,0x46,0x46,0xf5,0x6e,0xb0,0xdb,0xbe,0xb2,0xab,0xd9, +0x6b,0xf8,0x86,0x92,0xaf,0xef,0x56,0xd1,0xa9,0xed,0x06,0x44, +0x32,0x65,0x5d,0x3f,0x30,0x29,0x15,0x58,0xaf,0x25,0x65,0x43, +0xb7,0x1e,0x2e,0xd8,0x17,0xf3,0xcb,0x37,0x78,0xd5,0xe6,0xf2, +0x41,0x87,0x1c,0x7d,0xbc,0x03,0x7c,0xe9,0xd2,0xed,0x49,0xb4, +0x4f,0x9a,0x71,0xfc,0x92,0x16,0x2e,0xdc,0x88,0x3d,0xb8,0x1c, +0xc9,0xa7,0xc7,0xfe,0xbf,0x74,0xc9,0xd6,0x2f,0xdc,0x0f,0xfb, +0x69,0xbf,0xd7,0x1e,0x64,0xef,0xca,0xfe,0x2f,0xed,0xc1,0x37, +0x98,0x81,0xec,0x0f,0xd1,0x3d,0xd9,0xfd,0xa6,0x07,0x38,0x42, +0xeb,0x7c,0x35,0x40,0xc1,0x0b,0x3c,0x52,0xd0,0x16,0xcf,0xb2, +0xb7,0xbb,0x7f,0x7d,0x9b,0x2f,0xd1,0x01,0xc8,0xf4,0x0d,0x1a, +0x6f,0xbe,0x9c,0x4f,0xf9,0x86,0x1c,0x5e,0xc0,0x4c,0xaf,0x87, +0x8b,0x05,0xfe,0xdb,0x43,0xcd,0x0b,0xbc,0x58,0xbb,0x9d,0x3d, +0xba,0x10,0x2b,0x36,0x76,0x34,0xa5,0xab,0x76,0xfd,0x71,0x32, +0x42,0xc3,0xb2,0x05,0xd9,0x1c,0x9e,0xab,0xd4,0x1a,0x81,0xf9, +0xeb,0xd3,0xb3,0xf5,0x8f,0x4f,0x8f,0x9f,0x90,0x08,0xc7,0xc2, +0xd7,0x70,0xbe,0x7f,0xbe,0x0f,0x31,0x77,0x4b,0xaa,0xae,0x7f, +0xfe,0x11,0x20,0x0f,0x3a,0x45,0x86,0xa6,0x6e,0x20,0x09,0xdb, +0xb8,0xdf,0xfb,0x97,0x02,0x1a,0x09,0x31,0x45,0x8a,0xd6,0x8d, +0xd6,0xf0,0xdf,0x7e,0xfe,0x0e,0xd7,0xaf,0x31,0x73,0x88,0x9f, +0x39,0xed,0x27,0xb6,0x2d,0x0d,0x88,0xd3,0x0a,0xb9,0xfa,0x88, +0x8e,0xd2,0xd7,0x70,0x3a,0x65,0x6c,0x21,0xf9,0x66,0x3f,0xe9, +0xb2,0xe6,0x58,0x9a,0x7c,0x23,0xca,0x89,0x76,0xe8,0x4b,0xef, +0x07,0x05,0xe0,0xf9,0xd7,0x9b,0x20,0xcf,0x36,0x1b,0x2c,0xc3, +0x24,0x0b,0xe6,0x91,0x60,0xa0,0xe3,0xca,0xa0,0x74,0x2d,0xc7, +0xc6,0x2e,0x84,0x12,0xc7,0x83,0x5f,0xee,0x82,0x8c,0xa1,0xaf, +0x2b,0x72,0x5a,0x17,0x0e,0x80,0x31,0x04,0xb7,0x9c,0xb2,0x71, +0x9e,0xca,0x28,0xce,0x35,0xd9,0x55,0x86,0x8c,0xed,0x36,0x0d, +0xa6,0xbe,0x5a,0xd2,0x87,0x3c,0x07,0x10,0x7a,0xe1,0x36,0x13, +0x23,0x6a,0xc6,0x9c,0x1b,0x8c,0xee,0xac,0x1a,0xf3,0x8b,0xae, +0x37,0xc8,0x0c,0xd4,0x41,0x89,0x1d,0x70,0x6f,0x70,0x2d,0x8f, +0x97,0x40,0xda,0x63,0x24,0x23,0x8c,0x71,0x39,0xc8,0xc9,0x8e, +0x65,0x20,0x2d,0xc7,0x2f,0xd1,0xba,0xc4,0xb0,0x49,0x3d,0x40, +0xb8,0x99,0x39,0xc8,0xdd,0x05,0xc5,0xfd,0xcc,0x80,0xe4,0x92, +0x9e,0x96,0xd2,0x73,0xfe,0x31,0x0f,0x74,0x05,0xc7,0x00,0x21, +0xe2,0x64,0x8d,0x65,0x18,0x34,0x81,0xdf,0x57,0x93,0x37,0xaf, +0x4f,0xcf,0x12,0xd1,0x6f,0xf3,0x12,0xdd,0x35,0xed,0x1e,0x1c, +0x65,0x52,0xec,0x71,0xe2,0x5e,0xac,0xd7,0x83,0xc2,0xba,0xac, +0xe7,0x23,0x35,0xc2,0xa5,0x3d,0x42,0x15,0x11,0x37,0x3f,0x3a, +0x76,0x67,0xe2,0xc2,0x89,0xbf,0x4a,0x6f,0xc8,0xc0,0xab,0xdc, +0xc5,0xb6,0x4f,0x13,0xb6,0x2f,0xe4,0x34,0x58,0x3d,0x12,0x50, +0x8d,0x51,0x82,0x38,0x84,0x9d,0x85,0xe6,0xa8,0xd1,0x80,0x9e, +0x49,0x85,0x76,0x21,0x41,0x89,0x72,0x5e,0x6e,0xf4,0xda,0x6d, +0xd7,0xca,0xf7,0xbf,0xfd,0xfd,0xc7,0xb7,0xe8,0x02,0x73,0x75, +0x57,0x65,0xf7,0xd4,0x24,0xba,0x6d,0xdc,0x4b,0x95,0x02,0x83, +0xb5,0x48,0x79,0xbb,0x16,0xba,0x14,0xcc,0x29,0x05,0x9d,0x9e, +0xe5,0xac,0xf2,0x1e,0x52,0xe2,0x74,0x65,0xa3,0xf3,0x3e,0xa0, +0xc7,0x39,0x4c,0xa4,0x39,0x06,0x4f,0xa4,0x69,0xf2,0x49,0xee, +0xf9,0x35,0xd7,0x1d,0xc9,0x04,0xe9,0x17,0xea,0x1b,0x05,0x74, +0x3c,0xa1,0x83,0xe6,0xbe,0x60,0xbf,0x7c,0xe6,0x08,0x0f,0x0f, +0xee,0xb9,0x78,0x3a,0x85,0xf8,0x58,0xd9,0xa9,0x4f,0x8f,0x96, +0x79,0x02,0xe4,0x78,0xc4,0x2b,0xb8,0x1d,0xaf,0x6e,0xe6,0x23, +0xad,0xa5,0xcf,0x9f,0x83,0x70,0x0f,0xd2,0x1d,0xe3,0x86,0x8b, +0x5b,0xb5,0xd5,0x24,0x0d,0x66,0xce,0x74,0x0d,0xf2,0x33,0xb9, +0x2b,0xea,0xd6,0x36,0x41,0x24,0xf9,0x46,0xbf,0xae,0xf3,0xed, +0x2b,0x93,0x5b,0xe5,0xd7,0xa6,0x5b,0xa9,0x64,0x1d,0x00,0x68, +0x56,0x1a,0xc7,0x42,0xc1,0x37,0xca,0xd2,0xb0,0x33,0xa6,0xe2, +0x80,0x34,0xa4,0x5e,0xce,0x47,0xe8,0xe1,0x8b,0xbc,0xc1,0x8f, +0x57,0xc3,0x6b,0xfc,0xb5,0x68,0x57,0x41,0xf5,0x07,0x57,0x99, +0xd1,0x7d,0x9b,0x54,0x18,0x3b,0x2d,0xc2,0xab,0x86,0x51,0x78, +0x7b,0x04,0xf6,0x0f,0x07,0x99,0xd3,0x86,0x43,0x1b,0xf3,0x73, +0x6c,0x26,0x09,0xb9,0xc2,0xa7,0xe8,0xcb,0x39,0x7b,0xf4,0xbc, +0xeb,0xf0,0xc8,0x9a,0x70,0x95,0xfb,0x9f,0xf6,0x3e,0x7e,0xfc, +0xb8,0x87,0x4f,0xba,0x7b,0xd0,0x1c,0x46,0x3b,0x47,0x47,0x88, +0x9b,0x2c,0x91,0x0a,0xfd,0xe9,0xec,0xd9,0xde,0x9f,0x13,0x11, +0x90,0x45,0xa9,0x99,0x6f,0x92,0x42,0x2f,0x08,0xf6,0x83,0x43, +0x2e,0x6c,0xf6,0x17,0x88,0x0e,0x26,0x6c,0x9f,0x80,0x63,0x78, +0x6b,0xa3,0x93,0xb0,0xb0,0xbe,0xdb,0x69,0xde,0xa2,0x0c,0x9f, +0x30,0xfd,0x7d,0x45,0x32,0x7a,0x26,0x03,0xc6,0x48,0x0e,0x34, +0x0f,0x27,0x32,0xc4,0xc6,0xde,0xe8,0x3d,0xc2,0xc4,0xd2,0xfb, +0x5c,0x1d,0xd5,0xb4,0xcf,0x90,0xa8,0xf4,0xfe,0xc6,0x6d,0xb7, +0x67,0x64,0x9c,0x8d,0x8b,0x24,0x1a,0xf9,0xf7,0x97,0x2f,0x12, +0x69,0xbb,0xdd,0x85,0x5c,0x07,0x60,0x1c,0x78,0x2d,0x62,0x4f, +0xa9,0x11,0x49,0x21,0x8f,0xac,0xa7,0x74,0x1f,0xe4,0xd4,0xb9, +0x16,0x75,0x0e,0xc7,0x95,0x3f,0xb1,0x56,0x27,0x49,0x44,0xe7, +0xcb,0xbf,0x9f,0xbe,0x7e,0x25,0x89,0xd8,0xcf,0x20,0x0d,0xea, +0xdf,0xe4,0x57,0x40,0xa2,0xbd,0xe6,0x87,0x62,0x5e,0x08,0x3a, +0x49,0xe4,0x2b,0x60,0xe3,0x96,0xca,0xdd,0x22,0xb6,0x47,0x8f, +0x1c,0x68,0x5a,0x40,0x5e,0xfc,0x4c,0x22,0x7a,0xb4,0xf9,0x79, +0x2d,0x9a,0xa0,0x14,0x6b,0xc0,0xfc,0x33,0x0f,0xab,0x30,0x05, +0x1a,0x72,0x8a,0x19,0xfa,0x8c,0xdb,0xf5,0x46,0xd1,0x87,0x02, +0x90,0xf0,0xb3,0xb9,0xfb,0xac,0x5e,0x2f,0xcf,0x1c,0x2e,0x91, +0x7a,0x24,0x83,0x6f,0x7a,0x97,0xf4,0x60,0x29,0x8f,0x8c,0x70, +0xa9,0x22,0x38,0x26,0xf4,0x6d,0xfd,0x5e,0x63,0x5b,0x74,0x5d, +0xc8,0x01,0x3a,0x1b,0xe2,0xfe,0x3b,0xa5,0x0f,0x3a,0x58,0x75, +0x7e,0x2b,0x71,0x4f,0x55,0xa9,0x15,0x02,0x39,0x16,0x75,0x5b, +0xe7,0x84,0x5f,0xfb,0x8b,0xfa,0x56,0xf6,0xae,0x9c,0xcc,0x08, +0xab,0x8c,0x81,0x1f,0x3d,0x22,0x57,0xe1,0x43,0xe1,0x70,0xc3, +0x6f,0xf9,0x28,0xc7,0x77,0x3e,0xc0,0x03,0xcf,0xe0,0x07,0x6d, +0xad,0xa3,0x08,0xce,0x19,0x47,0xa5,0x36,0x29,0xf3,0x38,0x97, +0x3f,0x9d,0x5c,0xf5,0x3f,0x72,0x5b,0x79,0xc5,0x95,0xd2,0x72, +0xe4,0xcb,0xe7,0x7a,0x77,0x0c,0x47,0x9f,0x4f,0xa9,0x46,0x6e, +0x1f,0x60,0xed,0xdf,0x15,0x07,0xbe,0xb7,0xf4,0x92,0xa9,0x37, +0x25,0x8c,0xe1,0x8f,0x24,0x5a,0xfd,0x56,0x93,0x53,0xb9,0x6a, +0x7c,0x01,0x54,0x51,0x67,0x48,0xe5,0xa3,0x03,0xb8,0xc0,0xbf, +0x3d,0x38,0x78,0xcc,0x11,0xeb,0xf5,0xb7,0x07,0xe8,0x36,0x8c, +0xbf,0x7a,0x69,0xd5,0x9d,0x5c,0xe9,0xc9,0x86,0x7e,0xf5,0x25, +0x58,0x72,0xcb,0x60,0x89,0xbc,0x0d,0x3a,0x91,0x26,0x2f,0xe0, +0x28,0xdc,0xd3,0x12,0x80,0x2a,0xdc,0xfa,0xc2,0xb2,0xce,0xec, +0x61,0xd9,0x27,0x51,0xef,0x9f,0xde,0xbe,0x18,0x94,0x9a,0xd1, +0x17,0xd9,0x5e,0x09,0x9e,0xaf,0x8d,0xb0,0x31,0xa1,0x09,0x66, +0x96,0x3f,0x0a,0xba,0xe5,0x56,0x44,0x09,0xfb,0xd0,0x4c,0x7a, +0x32,0x9b,0xcb,0xc1,0x93,0x64,0xc5,0xb7,0x5f,0x5b,0x64,0x75, +0xeb,0xfa,0x5b,0x98,0x7c,0x7c,0x55,0xd3,0x11,0x93,0x56,0x6e, +0xf0,0x33,0x5b,0xd8,0x65,0xee,0xd2,0x8a,0xd2,0xf5,0x69,0xe2, +0xe9,0x36,0xa2,0xe5,0x6a,0x22,0x79,0xf9,0x9a,0x16,0x51,0x44, +0x06,0xb5,0x73,0x4e,0x5f,0x43,0x9e,0xea,0xbc,0xee,0xf8,0xc8, +0x4c,0x56,0xb0,0xf6,0x60,0xcc,0x08,0x84,0xae,0x01,0x97,0x5a, +0x1e,0xa0,0x32,0x3c,0xcf,0x82,0xc4,0x58,0xd4,0xa5,0x6b,0x60, +0xa4,0xf1,0x76,0x21,0xcf,0xc3,0x5a,0x5f,0x27,0x49,0x7c,0x6b, +0x7b,0xb0,0xfe,0xa1,0xc6,0xf1,0x08,0x91,0xa2,0xf9,0xf4,0x03, +0x5b,0x2b,0xd3,0x8b,0x56,0xfd,0xad,0xf4,0x75,0xab,0x9a,0xbe, +0x08,0xc2,0x54,0x18,0x00,0x88,0xad,0x36,0x97,0xb7,0x38,0x1a, +0x95,0xa6,0x5e,0x0e,0xc2,0xfe,0x9c,0xa0,0x46,0x8f,0x0f,0xea, +0xdc,0x60,0xd8,0x6c,0xd4,0xab,0xc9,0x72,0xcc,0x6a,0x2c,0x30, +0x36,0xaa,0x6b,0x0e,0x6b,0xc4,0xb9,0x75,0x10,0x12,0xd9,0x77, +0x31,0xc0,0xc5,0x0a,0x8b,0xa5,0x69,0xbb,0xcc,0x78,0x28,0xae, +0xa1,0x0d,0x54,0xcc,0xf3,0x89,0x76,0x13,0xeb,0xff,0xba,0x4e, +0x0e,0xb2,0xb0,0xb1,0xe9,0x03,0xad,0x8d,0x50,0x49,0x85,0x05, +0x20,0xf6,0xf6,0x9c,0x80,0x39,0x4e,0xaa,0x7b,0x75,0x60,0x65, +0x8f,0xa0,0x3c,0x61,0xa5,0xb0,0x4a,0x36,0x35,0xca,0x81,0x08, +0xa8,0x54,0x0e,0x70,0xa4,0x57,0x98,0x6e,0xf1,0xb8,0x99,0x26, +0xc9,0x2f,0x5a,0xbb,0x39,0x12,0x4b,0xfb,0xa8,0x60,0x9d,0xeb, +0x36,0x6e,0x3e,0x27,0x83,0xa3,0xd7,0x76,0xda,0x1f,0xb5,0x71, +0xc1,0xbc,0x8a,0x78,0x64,0x77,0x8b,0x14,0x90,0x34,0xbd,0x63, +0xf2,0x68,0x74,0x4b,0xe7,0x64,0x03,0xd6,0x72,0x5e,0x1f,0x48, +0x9f,0x4e,0xbe,0xbf,0x83,0xb2,0x46,0x7f,0x39,0x4e,0x61,0xde, +0x9a,0x7b,0xd3,0x8a,0x92,0x9d,0x50,0x3b,0x8d,0x75,0xae,0x2b, +0x5d,0x1b,0xae,0x4b,0x22,0xad,0xaf,0x12,0xcd,0x72,0x22,0x00, +0xab,0x34,0x41,0x26,0x49,0xeb,0x76,0x7c,0x3b,0x5f,0x7e,0x4e, +0x82,0xa5,0x5d,0x99,0xe5,0x4f,0x56,0x86,0x96,0x63,0x68,0x55, +0xb5,0x92,0x81,0x2a,0x6b,0x31,0x6c,0x03,0xfa,0x7e,0x93,0xf3, +0xc5,0x87,0x07,0xe0,0xf2,0xf8,0x02,0xaf,0xb3,0xe4,0x72,0x08, +0xb5,0x20,0x73,0x9b,0x37,0x56,0x79,0xef,0xaf,0x2b,0xb8,0xa3, +0x6a,0x87,0x77,0x61,0x59,0xac,0x7c,0x3d,0x93,0x65,0x6b,0xe4, +0x73,0x3e,0x92,0xe3,0x76,0x4c,0x24,0xf9,0x4e,0x34,0x7d,0x19, +0xfb,0x17,0x0a,0xe2,0x50,0xf6,0x8f,0x8a,0x97,0xca,0xcc,0x61, +0xf6,0x40,0xe3,0x92,0xc9,0x8e,0xb2,0x28,0xbe,0x4f,0x65,0xeb, +0x46,0x9c,0xd8,0x4f,0x59,0xff,0xd1,0x40,0x81,0x47,0xc5,0xa0, +0xe9,0x51,0x89,0x4d,0xc0,0x2e,0xa7,0x52,0xcc,0x30,0xa7,0xe0, +0x46,0x2c,0x8d,0x84,0x43,0x51,0x35,0x3c,0x72,0xb8,0x21,0xe8, +0x35,0x76,0x41,0x9c,0x48,0x54,0x38,0xaa,0x66,0x7e,0x9a,0x95, +0x45,0x48,0xe4,0x8a,0x04,0x5f,0x67,0xdb,0x3c,0x62,0xb2,0x22, +0x02,0xba,0x6c,0xc7,0x6c,0x0d,0x73,0xde,0x9f,0x32,0x47,0x71, +0x7b,0xd2,0x7a,0x4d,0x75,0x86,0x19,0xac,0x98,0xaf,0x92,0xb7, +0xc8,0x2c,0x44,0x4f,0xc8,0x2f,0x61,0xc7,0x12,0x61,0xe2,0x11, +0x5d,0x4b,0xfe,0x69,0x8b,0xaa,0xee,0xad,0x64,0xa4,0x13,0x45, +0xa1,0xf8,0x85,0xeb,0xcb,0xdf,0x0e,0xc5,0xe5,0x1d,0x52,0x33, +0xb8,0x90,0x30,0x02,0xd7,0xd3,0x63,0x5e,0x4d,0xe4,0x18,0x07, +0x92,0x90,0xe3,0x8c,0x49,0x1e,0x46,0x1f,0xa3,0x07,0x65,0x3f, +0x8e,0x41,0x2d,0x19,0x0e,0x89,0x0f,0x7e,0xbe,0x34,0x86,0xd3, +0x8f,0xc3,0xcf,0x15,0x82,0xef,0xdb,0x5b,0x64,0x10,0x99,0xed, +0x1b,0xe2,0xce,0x08,0xd5,0x41,0xe4,0x22,0xbc,0x67,0x1b,0x4c, +0xb3,0xe1,0x94,0x6e,0xcd,0x2a,0xb8,0x2e,0x79,0x47,0x39,0x50, +0x7a,0x8c,0xb5,0xdb,0x2e,0xd8,0x25,0xc8,0xa9,0x03,0x90,0xe5, +0x84,0x0e,0xa3,0x16,0x85,0x8b,0x61,0xc3,0x24,0x64,0x89,0x40, +0xef,0x0f,0xa0,0x12,0x6f,0x27,0x30,0xeb,0xd4,0x66,0xcf,0xd9, +0x28,0x6b,0x17,0x0d,0x20,0xea,0x7a,0x3d,0x0a,0x66,0xc1,0x5f, +0x58,0x8b,0x24,0x30,0x82,0xc1,0xe1,0xab,0x21,0x3a,0x55,0xe8, +0xe2,0xb9,0x9e,0x22,0xba,0x0e,0x9d,0xe8,0xd2,0x8f,0xa5,0x72, +0xf1,0xf2,0x37,0x5c,0x7a,0x64,0xca,0xe6,0x41,0x8c,0x72,0x3d, +0xf3,0x88,0xce,0xed,0x24,0xfb,0xfb,0x78,0x6c,0xd1,0x43,0x9b, +0xde,0x12,0x5d,0xa6,0xdb,0xd7,0x6b,0xfd,0x66,0x76,0x80,0x8f, +0xe7,0x98,0x9c,0x11,0x27,0x66,0x6c,0x5a,0x71,0x16,0x1f,0x0f, +0x08,0xf6,0x37,0x49,0xe4,0x30,0x9b,0xbd,0x08,0xa4,0xa4,0x84, +0xb6,0x04,0xbc,0xee,0x23,0xea,0x6b,0xb3,0x1e,0xfb,0x20,0x17, +0x87,0x34,0xdd,0xcb,0xe5,0x1c,0x28,0xee,0xf9,0x2d,0x50,0xc0, +0x70,0xf0,0x2f,0x22,0xc6,0x24,0x8d,0x40,0xc4,0x91,0xcc,0x83, +0x42,0xe5,0x4e,0xba,0x43,0xa5,0xf4,0x25,0xf9,0x70,0x00,0x9b, +0x3e,0xea,0x7c,0xbb,0xcd,0x69,0x8f,0x6a,0x69,0x8f,0x06,0x5a, +0x6b,0xff,0x5b,0x14,0x5c,0x4f,0x6e,0x56,0xab,0x45,0x81,0xc2, +0x03,0x0a,0xad,0xf7,0xe7,0x83,0xe2,0xbb,0xef,0xbe,0x45,0x21, +0xfc,0x90,0x77,0x10,0xe5,0x8f,0xaa,0x74,0xc5,0xa8,0xc1,0x23, +0x32,0x80,0x49,0x2e,0x0b,0x95,0xc3,0xe0,0xfd,0x5b,0x78,0x77, +0x55,0x92,0x4f,0xc6,0xb5,0x0c,0x5e,0x7a,0x05,0x79,0x0d,0x1f, +0x9c,0x32,0x34,0x33,0xb3,0x00,0x4c,0xe1,0x8b,0x54,0x24,0xe0, +0x48,0x2a,0x01,0xce,0x0b,0x37,0xf7,0xf7,0x83,0xb2,0x8a,0x30, +0xfe,0xc8,0x5c,0xfe,0x30,0x3d,0x7c,0x51,0x87,0x68,0xd0,0x81, +0x17,0x6a,0x62,0x94,0xa6,0xd3,0xf1,0xd2,0x20,0x4d,0x48,0x0d, +0xb2,0xdd,0xdc,0xd2,0xe3,0x1f,0x98,0xd3,0x9f,0x16,0x0b,0x67, +0xc0,0xa0,0x42,0xef,0xda,0xca,0xca,0xdf,0xf1,0x6c,0x7d,0xf1, +0xf1,0xc5,0x3e,0x79,0x1d,0x0e,0x53,0xd2,0xb2,0x08,0x0a,0xd1, +0x11,0x27,0xa3,0x17,0x66,0xeb,0xf0,0xb4,0xc9,0x63,0x86,0xc8, +0xc2,0x48,0x8e,0xac,0x97,0xb4,0x01,0xa1,0xec,0xa1,0x27,0x3b, +0x1e,0x5e,0x51,0x8a,0xe5,0x2f,0x5a,0x6a,0xa4,0x98,0x8c,0x8e, +0xb9,0x69,0x5e,0x70,0x77,0x2e,0xd5,0xa5,0xb2,0x07,0xa3,0x21, +0xbf,0x0f,0x01,0xe9,0x4a,0x76,0x0f,0xdf,0x95,0x49,0xc7,0x3f, +0xb1,0x74,0x3a,0x59,0xa1,0x39,0x3b,0x5f,0xd5,0xaa,0xa4,0x06, +0x00,0xdb,0x14,0x10,0x91,0x0f,0xd2,0x81,0x30,0x2f,0x7c,0x00, +0x45,0x57,0x5c,0x9a,0x3c,0xbf,0x72,0x74,0xe5,0xde,0xe9,0x04, +0x60,0x27,0xf9,0x83,0x90,0x3c,0xbb,0x2e,0x20,0x0a,0x1f,0xac, +0xe0,0x15,0x1c,0x74,0x7b,0xe4,0xf8,0x33,0x69,0x2e,0x0d,0xbd, +0x49,0xfd,0xe6,0xf0,0xb3,0x89,0x5f,0x86,0xe9,0xb6,0x83,0xc3, +0xef,0x0f,0x29,0x93,0x92,0x6d,0xaf,0x5f,0x40,0xed,0x61,0xb6, +0x24,0x0f,0x4b,0xe5,0xdb,0x0a,0x1d,0x13,0xab,0x2e,0xb1,0x07, +0x1e,0x59,0x74,0xaf,0x44,0x29,0x1c,0x5f,0xa0,0x6d,0xca,0xa0, +0xb7,0x35,0xa5,0x83,0xcf,0x3e,0xc8,0xe4,0x08,0xa2,0x7b,0x49, +0xde,0x82,0x19,0x15,0x26,0x60,0x27,0x39,0x6a,0xfd,0x56,0x1e, +0x74,0x0f,0x48,0x3c,0x32,0x2b,0x3c,0x30,0x28,0x3b,0xf0,0xbe, +0x6e,0x61,0x3d,0x2a,0x0b,0x66,0x4b,0xcb,0x27,0xb9,0xcb,0xa2, +0xa6,0x73,0xaa,0x2e,0x1b,0xc3,0x42,0xb6,0x35,0xad,0x5d,0xff, +0xc9,0x96,0xd8,0x62,0xfa,0x47,0x48,0x96,0xac,0xe4,0xe1,0xf6, +0xe7,0x43,0x70,0x40,0xc8,0xad,0x99,0x1d,0x79,0xa4,0x95,0x62, +0x12,0xeb,0x99,0x97,0x49,0xb0,0x43,0xa6,0x12,0xe1,0x57,0xef, +0xc6,0xe2,0x70,0xc3,0x3d,0x80,0x46,0xa6,0xae,0xa5,0x9e,0x9d, +0xf3,0xf0,0x69,0xe6,0xb9,0x5b,0xb5,0xd3,0x2c,0xbb,0xaf,0x71, +0x77,0x0e,0xbf,0x9a,0xd2,0xf4,0xdc,0x7d,0x4b,0xb7,0xc1,0x64, +0x20,0x03,0x18,0x27,0x5f,0xa8,0x23,0x7c,0x9d,0x09,0x38,0x51, +0x28,0x67,0xa4,0x1c,0x2a,0x2b,0xf6,0x68,0xc6,0x29,0x91,0xfc, +0xf8,0x94,0xee,0x00,0x65,0xfc,0x1c,0x58,0x49,0x3b,0x3d,0x46, +0x52,0x21,0x07,0x31,0x44,0xfe,0x08,0x29,0xc9,0xfc,0xfb,0x20, +0x99,0x7f,0x55,0x5c,0x2c,0x23,0xfb,0x6d,0xad,0xf1,0x11,0x61, +0x2e,0x7b,0x30,0xe0,0xd9,0x86,0x4d,0x6f,0x68,0x44,0xf2,0x6a, +0xde,0x72,0x43,0xe8,0x5f,0xc8,0xa8,0x89,0x84,0x48,0xb3,0x89, +0xb6,0x90,0x51,0xe8,0x1f,0xf3,0xc2,0x57,0x04,0x14,0x09,0x21, +0xda,0xd3,0x11,0xf3,0xee,0xe5,0xc0,0xd9,0xab,0x20,0xa0,0xc8, +0xc1,0x2d,0xb6,0x3f,0x51,0x6c,0x83,0x1b,0xbe,0x63,0x24,0xc4, +0x19,0xb6,0x4f,0x05,0x86,0xd2,0x74,0x4c,0x75,0xae,0x57,0xf8, +0xe6,0x9e,0xe9,0x9d,0xb7,0x02,0xbe,0xf8,0x96,0x78,0x74,0x9f, +0xd2,0x14,0xff,0x69,0xcf,0xa7,0x04,0xec,0x73,0xa9,0xad,0xd1, +0xff,0x4a,0xc4,0x02,0x27,0xfe,0xb5,0x80,0x30,0xd8,0xb9,0x3c, +0xc5,0x05,0xdd,0x37,0x46,0x3d,0x18,0xeb,0x84,0xbf,0x9b,0xa8, +0xdf,0x6e,0x4f,0xa4,0xce,0xa0,0x9f,0xc7,0x89,0xb3,0x7b,0x7f, +0x53,0x59,0x89,0x16,0x8d,0xdd,0x39,0x8c,0xd0,0x26,0x4a,0xe3, +0x57,0x48,0x7a,0x31,0xd1,0x9b,0x1e,0x73,0x86,0xf5,0xba,0xc5, +0xd3,0x5c,0x2f,0x1d,0x34,0x06,0xb0,0x88,0x72,0xf3,0x98,0xe2, +0x89,0xe4,0xdf,0xb0,0xf1,0xcb,0x59,0xd8,0x4f,0xf0,0x2b,0xc9, +0xc8,0x8e,0xf2,0x36,0x6d,0x1a,0x59,0xac,0xf7,0x55,0x60,0xc5, +0xfe,0x9d,0x59,0x46,0x5c,0xcf,0xd6,0x67,0x72,0x67,0x43,0x85, +0x03,0xbc,0xa1,0x89,0xe3,0xd8,0xe5,0x98,0x13,0x7e,0xa9,0xc1, +0xd1,0xe0,0x1c,0xfa,0x74,0x13,0x65,0x70,0x10,0xaa,0xe5,0xa5, +0x62,0x1f,0x1c,0x31,0x27,0x11,0x97,0xd2,0x7d,0xd1,0x01,0x44, +0x9b,0x93,0x25,0x54,0x4a,0xd3,0xee,0x49,0x45,0x07,0x66,0x76, +0x9f,0x4a,0x08,0xf9,0x88,0x5c,0xd0,0x9f,0x5b,0xeb,0xf5,0x3e, +0x42,0x1c,0x8f,0xd6,0x7a,0x68,0xee,0x0b,0x16,0x14,0xe7,0x24, +0x55,0xdb,0xaf,0x6d,0x06,0x5b,0x72,0xe1,0x64,0x6b,0x9f,0xbd, +0x16,0x15,0x58,0xdf,0xa9,0xc4,0xfc,0xa5,0x8c,0x33,0x01,0x71, +0x6d,0xd7,0x69,0x48,0x1f,0x1d,0x1c,0xc0,0xfe,0x17,0xfe,0x1c, +0x4a,0x1f,0xe2,0xdc,0x86,0xe6,0x7b,0xcd,0x82,0x08,0x3d,0xe1, +0x45,0xd4,0x9d,0x4e,0xa9,0x6b,0x18,0x77,0x2c,0xf5,0x27,0xce, +0x0e,0x69,0x4a,0x88,0x38,0xe7,0x7c,0x3a,0x72,0x1c,0x1a,0x94, +0x3a,0x5b,0xe2,0x91,0xb1,0x28,0xf7,0xd3,0x32,0x3b,0xef,0xa5, +0xbd,0xb2,0xbd,0xde,0xcd,0xd6,0xe7,0xbd,0xf3,0xde,0xfe,0x51, +0xfd,0x08,0xa1,0xbc,0x45,0xa2,0xdd,0xe0,0xf7,0xb4,0x85,0xc2, +0x2b,0x22,0x51,0x45,0xf7,0x34,0x6a,0xeb,0xec,0x2e,0xe6,0x8b, +0x34,0xf3,0x2c,0x3d,0x36,0xad,0x02,0x68,0x5b,0x88,0xb5,0x59, +0x82,0xb6,0xaf,0x80,0x06,0xb8,0x10,0xdd,0x8b,0xe8,0xd6,0xcd, +0x8e,0x8d,0xa2,0x37,0xb2,0x85,0xdd,0x78,0xce,0x48,0xa5,0x7b, +0x63,0xe2,0x7b,0x30,0x68,0xea,0x2b,0xd2,0xe9,0x84,0x13,0xe9, +0xe3,0x12,0x05,0x20,0x67,0x8e,0x85,0x77,0xc2,0x6a,0x7f,0xe3, +0x25,0x75,0x19,0xd5,0x0c,0x61,0x51,0x53,0x1d,0x3b,0x82,0xe9, +0xf2,0x40,0x2a,0xfa,0x0d,0xeb,0x1d,0x90,0x52,0xf8,0x4b,0xea, +0xfc,0x91,0x53,0x3e,0xc1,0xdf,0x76,0xd2,0x00,0xc7,0x62,0x6b, +0xa0,0x4e,0x38,0xe3,0x8b,0xaf,0xa6,0x24,0xcd,0x19,0x54,0x2a, +0xef,0xd4,0x09,0xfe,0x9a,0x8b,0x4b,0x1a,0x0c,0xf0,0x79,0x50, +0xca,0x1a,0x8a,0x95,0xda,0xfe,0x6b,0xc7,0x74,0xca,0x1a,0x44, +0x0f,0xa3,0x1c,0x59,0x2f,0x8a,0x48,0x11,0x31,0x0b,0x62,0xdc, +0xb0,0xf5,0xaa,0xbe,0x06,0x07,0xa5,0x09,0x7b,0x3a,0x80,0xca, +0x21,0x29,0x90,0x74,0x6c,0xb3,0x1c,0xc8,0x1d,0x4b,0x5a,0x34, +0xd1,0x2a,0x3a,0xfa,0x9e,0x50,0xa1,0x72,0x24,0xe8,0x18,0x40, +0x64,0x1c,0x57,0xee,0x9f,0xbe,0x9c,0x7d,0xfc,0xba,0x6a,0xc4, +0x05,0xbc,0x52,0x50,0xbc,0x16,0xfc,0x2a,0x46,0xa4,0x2d,0x18, +0xc4,0x4e,0xd2,0xfa,0x38,0xac,0xd0,0x11,0x21,0xad,0x2d,0x7a, +0x8a,0xaa,0x95,0x87,0xa1,0xdf,0x44,0x88,0x73,0xc9,0x57,0xb8, +0x5d,0x83,0x25,0xbf,0x08,0xf7,0x2d,0xf8,0x41,0xde,0x14,0x19, +0xb6,0x39,0xaa,0xcc,0xcb,0xb0,0x6c,0xf2,0x80,0x9b,0x64,0x0a, +0x35,0x02,0xb5,0xdb,0xa1,0x0a,0xd3,0x68,0x16,0xc2,0xc5,0x12, +0xef,0xb2,0x68,0x19,0x84,0x47,0x01,0x99,0x39,0xb3,0x33,0xd2, +0x30,0x48,0x8e,0x50,0x36,0xeb,0xcf,0x34,0x09,0xd6,0xbb,0xf9, +0x4a,0x9b,0xc6,0xb8,0x01,0xa8,0xed,0x94,0xbf,0xfe,0x37,0xc6, +0x88,0x98,0xaa,0x71,0xf0,0xa9,0xf9,0xe9,0x66,0xe9,0x9a,0x9d, +0xc3,0xc7,0x29,0xeb,0x53,0xc0,0xe1,0x0a,0x1f,0xcf,0x47,0xe5, +0x01,0xfe,0xbe,0x9e,0xfd,0x44,0x47,0x2f,0xe3,0xf8,0xf2,0x8e, +0x7f,0x4c,0xf4,0xfe,0xdf,0x59,0x74,0xd5,0x88,0xf0,0xf0,0xb9, +0xf3,0xeb,0xf8,0x33,0x11,0x01,0xf0,0x8b,0x44,0x8b,0xad,0x26, +0xb3,0x1f,0xc8,0xf1,0x1d,0x44,0x47,0xfa,0x51,0x93,0x6c,0x2d, +0x14,0x6a,0xac,0xb9,0x57,0x5b,0xce,0x3b,0xec,0x6c,0x86,0xe5, +0x3e,0xda,0x6d,0xb1,0x1c,0xb9,0x82,0xd3,0x78,0xb8,0x1c,0xc1, +0xf2,0xc0,0x43,0x9a,0x23,0x05,0x10,0x46,0x6d,0x8a,0x5a,0xbe, +0x60,0x38,0x9a,0x84,0x80,0xb1,0x4d,0x69,0xdd,0x5c,0xc4,0x7c, +0x59,0x95,0x3b,0x3b,0xb6,0x30,0x9c,0x5c,0x1f,0x27,0xab,0x9b, +0x93,0xe5,0x78,0x04,0xcb,0x74,0x02,0x34,0x48,0xc2,0x63,0x12, +0x8c,0x76,0xad,0x2e,0x67,0xe3,0x0e,0xea,0x8c,0x20,0xe6,0x21, +0xf8,0x46,0x1c,0x2d,0x42,0xcd,0x76,0x02,0xdc,0xac,0x6e,0xfc, +0x7d,0xae,0x8f,0xfa,0xba,0x6e,0x9b,0x51,0x2e,0xa1,0x2c,0x1d, +0x19,0xc7,0x65,0xd8,0xe6,0x58,0x3e,0xc1,0x76,0x95,0x32,0x30, +0x4c,0x78,0xde,0x55,0x68,0xe9,0xeb,0x76,0xdc,0x83,0xb8,0xee, +0x7c,0x31,0x9e,0xa5,0x8e,0x91,0xc8,0x8c,0x1b,0x42,0xc4,0x72, +0x9f,0x31,0x47,0xf7,0x00,0x55,0xf5,0x71,0xbe,0x1c,0x65,0xc5, +0x83,0x85,0xf0,0x98,0x83,0x0c,0x2c,0x9b,0x92,0x19,0x02,0xd9, +0x47,0x42,0x08,0x28,0xcb,0xd2,0x44,0xa1,0xff,0x76,0xcf,0x01, +0x6f,0xb7,0xa9,0x86,0x88,0x81,0xde,0x1c,0x6b,0x18,0xe7,0x11, +0x06,0xbd,0x5e,0x2b,0xbd,0x9d,0xfc,0x7d,0x4f,0x88,0xf1,0xf1, +0x68,0x8f,0x2c,0x4a,0x23,0x67,0xf0,0x81,0xd4,0x32,0xf9,0xfb, +0xcb,0x17,0x3f,0xae,0x56,0x0b,0x49,0x30,0xae,0x2b,0xa8,0x33, +0x4a,0xeb,0x63,0x83,0x1a,0x28,0x7d,0x43,0xe7,0x2b,0x59,0xb8, +0x5c,0x66,0xf7,0x1b,0xce,0x8e,0x62,0x78,0x11,0x2b,0x85,0x1c, +0x89,0xae,0xc9,0xa4,0x81,0x97,0xf0,0x6a,0xc2,0x4b,0x59,0x0b, +0x93,0xde,0xa4,0x6b,0xaf,0x7d,0x0d,0x72,0x21,0xd4,0x6a,0xb4, +0x69,0xe4,0x64,0xfa,0x3c,0x62,0x8b,0x8f,0xfd,0xd8,0x20,0x8b, +0xae,0x9a,0xac,0xe6,0x3d,0x98,0x97,0x11,0x94,0xa5,0xf1,0xaf, +0x63,0xae,0x4e,0x2e,0x79,0xbe,0xa8,0x9d,0x4b,0xed,0xb6,0x30, +0xec,0x82,0xe3,0x85,0x21,0xc2,0x49,0xa9,0x1d,0xfb,0x6e,0x27, +0x6e,0x0c,0xcf,0xb6,0xb2,0x32,0x90,0x62,0x76,0x77,0x8c,0x7b, +0x99,0xbb,0xab,0xa8,0x50,0xf3,0x80,0x50,0x52,0xe3,0x3b,0x54, +0x9a,0xe5,0x35,0x24,0x89,0x2b,0xf7,0xe2,0x52,0x46,0xc6,0xa4, +0x22,0xe3,0x9b,0x65,0x9c,0xc3,0x73,0x08,0x44,0x26,0xc0,0x37, +0x05,0x3f,0x3d,0x3f,0xc0,0x8a,0x1e,0x24,0x1b,0x2f,0x9f,0xa0, +0x87,0x21,0x32,0xf1,0xcd,0xb2,0xed,0x1d,0x3e,0x7a,0xf4,0xad, +0x13,0xc4,0xf0,0xc2,0x09,0x8f,0x0e,0xbe,0x03,0x9c,0x84,0xc3, +0x61,0xd3,0x7a,0x80,0xde,0x17,0xdf,0x1d,0x7c,0xb7,0x91,0x3a, +0x91,0xab,0x72,0x35,0xff,0x74,0x4c,0xe8,0xfe,0xd3,0x4f,0x48, +0x86,0xe3,0xc3,0xc4,0xbd,0xa7,0x0b,0x54,0xd6,0x73,0x8f,0x59, +0x30,0x0d,0x99,0x37,0x46,0xda,0xc6,0x65,0x97,0x81,0x6e,0x5a, +0x69,0xf1,0xe8,0x13,0x33,0x85,0xce,0x83,0x5e,0x7d,0xa1,0xf5, +0x0c,0x57,0xc6,0x91,0x8a,0x45,0xca,0xab,0xa2,0xec,0x74,0xe8, +0x92,0x6b,0x58,0x4a,0xa9,0x5d,0x44,0xb0,0x83,0xed,0x27,0xae, +0x09,0xa1,0x5c,0xf9,0x42,0xca,0xba,0xac,0xdc,0x90,0xc6,0x70, +0xe0,0xd6,0x6f,0x5a,0x8c,0xce,0x04,0x0a,0x25,0x37,0xad,0x73, +0xdf,0x54,0x47,0x58,0xa1,0x4b,0xf4,0x1a,0x81,0xe4,0x77,0x9b, +0xcb,0xb7,0x85,0x3a,0xba,0xfa,0xf4,0x6a,0xfe,0x91,0x9e,0xf6, +0xb1,0xc3,0xcb,0xab,0x4f,0x2b,0xaf,0xc8,0xc6,0x56,0xd6,0xd6, +0x68,0xef,0x6c,0x8d,0x06,0xce,0x48,0x64,0xfe,0xea,0xd3,0xac, +0x66,0x4c,0xb0,0x57,0x88,0x3d,0xc1,0x75,0x16,0x5b,0x24,0x64, +0x63,0x84,0xc6,0x90,0xe1,0xf2,0x6e,0x56,0xee,0xc3,0x61,0x75, +0x37,0x26,0xcd,0x3c,0x00,0x39,0x9c,0x4d,0x6e,0x09,0xeb,0xf7, +0x2c,0xbd,0xb2,0x2f,0x26,0x89,0x5d,0xd4,0x20,0x5f,0x7d,0x1c, +0x8f,0x67,0xf4,0x02,0x8d,0x32,0x92,0x7d,0xa3,0xf3,0x08,0x9b, +0xde,0xfa,0x5f,0x9a,0x8d,0xf2,0xbb,0xd9,0x64,0xc5,0x05,0x58, +0x73,0x4c,0x2c,0xca,0x62,0x84,0xcd,0x2f,0x1a,0x8d,0xdc,0x27, +0x6b,0xa0,0x4a,0xa4,0xf1,0x4a,0x02,0x81,0x1a,0xe5,0x29,0xbd, +0xe6,0x03,0x92,0xd3,0xe1,0x94,0xf5,0xfa,0x00,0x08,0xde,0xe1, +0x14,0x59,0x73,0xb7,0xc3,0x4f,0xcf,0xa1,0x85,0x43,0x16,0xae, +0x78,0x44,0x56,0xfe,0x08,0x30,0x1b,0x29,0x9b,0x8d,0xca,0x8e, +0x3e,0x2d,0x51,0xc3,0x4a,0xf3,0x96,0x54,0x33,0x3e,0x45,0x0a, +0xb6,0xa8,0xce,0xc4,0x26,0xa7,0xf0,0x2f,0x19,0xe2,0x9a,0x90, +0x45,0x8d,0x21,0x9e,0xc0,0xdc,0x10,0xa3,0x97,0xca,0xad,0x74, +0x6a,0xbd,0x38,0xc1,0xeb,0x35,0xd4,0xbb,0x5e,0x1f,0x1e,0x8d, +0xe6,0x2d,0x6e,0x27,0xfd,0x05,0x8a,0xa8,0xfb,0xc7,0x84,0xbb, +0xb2,0xcf,0x51,0x79,0x60,0xb7,0x25,0x06,0x45,0x39,0x3b,0x58, +0x7d,0x76,0xf4,0x11,0xc8,0x71,0xa4,0xd2,0xa1,0x10,0x34,0x89, +0x03,0x76,0x84,0xf6,0x45,0x84,0xb1,0xdd,0x46,0xa3,0xf2,0x94, +0xdc,0x6e,0xef,0xed,0x05,0xc3,0x93,0x6d,0xb8,0x00,0x8d,0x83, +0x9f,0xa5,0x2e,0xf7,0x89,0xfe,0x4a,0x0c,0x8e,0x9b,0x7b,0x5f, +0xe3,0x66,0xa4,0xfa,0x8d,0xe6,0xc2,0x10,0xf9,0x80,0x7f,0xce, +0x3b,0x04,0x16,0xda,0x0c,0x1c,0x8e,0x78,0xac,0x8b,0xaa,0x0c, +0x45,0x88,0x5d,0x7c,0x7e,0x2f,0x0b,0xaa,0x08,0x56,0x92,0xd5, +0x52,0x68,0x10,0xf0,0xc7,0x1c,0x81,0x7c,0x3f,0x15,0xa1,0xbf, +0x25,0x33,0xe4,0x0b,0xfe,0xa0,0xbf,0x46,0x85,0xe8,0x48,0x95, +0x6a,0x69,0x58,0x89,0xe6,0x45,0x6f,0x3f,0xec,0x8a,0x8d,0x33, +0x7b,0xfb,0x8e,0x28,0xe5,0x4f,0x59,0x8e,0xe8,0x6f,0xa7,0x93, +0x61,0x0e,0xce,0xd6,0xa7,0x28,0xbf,0x1d,0x44,0x25,0x3c,0xfc, +0x24,0x8d,0x8c,0x30,0x0a,0x86,0xbc,0xba,0x99,0x5c,0x99,0xe3, +0x6e,0xe3,0xb5,0x54,0xfc,0x18,0x38,0xce,0xaa,0x38,0xcf,0xc8, +0xee,0x25,0xd0,0x6b,0xd8,0xa6,0x75,0x98,0x45,0x53,0xae,0x80, +0xea,0xb1,0xbc,0x5a,0xda,0x90,0x25,0xfd,0xcd,0xe9,0x6f,0x97, +0x9e,0xa6,0x89,0xe7,0x78,0x0f,0x84,0x2a,0x9a,0x90,0x45,0x5d, +0x81,0x82,0xb3,0x4c,0x60,0xbd,0x6c,0xb1,0x58,0x45,0xa3,0x8a, +0x3b,0x7f,0x3c,0xac,0x50,0x56,0x89,0x56,0x2c,0x7b,0x5c,0x74, +0x46,0x7d,0xc5,0xe3,0xbe,0x8e,0x64,0x2e,0x56,0x20,0x31,0x7f, +0xc9,0x3f,0xe8,0x11,0xe9,0x23,0x59,0xdf,0x60,0x59,0xfc,0x50, +0x5e,0x8a,0x23,0x79,0xa5,0x8a,0x0a,0xe9,0x47,0x39,0x5e,0xe8, +0x80,0x60,0x78,0xb0,0x6c,0xb1,0x1d,0xf4,0xe1,0x56,0xf9,0x97, +0x77,0x3a,0x19,0x1c,0x88,0xf9,0x49,0x6c,0xe3,0xcc,0x8d,0xcc, +0x82,0x75,0x99,0x5d,0x37,0x9c,0x3e,0xb4,0x58,0x31,0x73,0x06, +0xcc,0x7a,0xde,0x94,0x19,0xfb,0x22,0x8e,0x40,0x38,0x8b,0xef, +0x3e,0xcb,0x26,0x87,0x83,0xd9,0x6c,0x85,0xf1,0xf2,0x12,0x30, +0x43,0x39,0x51,0x87,0x15,0xe2,0x60,0x5f,0xd9,0x18,0x8e,0x99, +0x57,0x25,0x15,0x2b,0xed,0x48,0x76,0x47,0x77,0x7c,0x0e,0x38, +0x63,0xc6,0x34,0xec,0x7d,0x33,0x13,0x03,0xad,0x3a,0x6f,0x2c, +0xf8,0x8d,0xa6,0x1e,0xe4,0x87,0xcd,0x39,0x60,0x03,0x5a,0x00, +0x38,0x47,0xa9,0xce,0xcc,0x9e,0x9f,0x41,0x38,0x40,0xb0,0x79, +0x1d,0x1f,0x13,0x82,0x03,0x54,0x7c,0x21,0x7e,0xda,0x6c,0x94, +0x77,0x4b,0xd5,0xf5,0x4b,0x6a,0x86,0x37,0x28,0x39,0x86,0x0a, +0x26,0xa2,0x72,0x13,0x51,0x7d,0x79,0x22,0x5c,0x16,0x11,0x27, +0x89,0xf7,0x43,0x17,0x17,0xbf,0xd9,0x1e,0x51,0x72,0x1e,0x01, +0x2e,0xef,0x9d,0xc3,0xf0,0xd0,0x58,0x04,0x1d,0x0a,0x6a,0xd2, +0xad,0x82,0xf4,0x50,0xed,0xd8,0x9f,0xff,0xfd,0x95,0x83,0x08, +0xc7,0x89,0x8f,0x57,0x5b,0x09,0x6c,0x32,0x3b,0x8e,0xb7,0xa5, +0x7a,0x29,0xd7,0xb0,0xe5,0xaa,0xf2,0x39,0x51,0x3a,0x25,0xe7, +0xbc,0x40,0x11,0x93,0x3d,0x24,0xb2,0x41,0x86,0x11,0x3d,0xfe, +0x29,0x0e,0xb2,0xa2,0xb9,0x6d,0x91,0xc1,0x18,0xe9,0x9f,0xea, +0xdb,0x7c,0xa2,0x59,0x0b,0x5a,0xb5,0x3d,0x49,0x0a,0x17,0xf5, +0xce,0xa6,0x5b,0x7a,0x1b,0xdb,0x30,0x34,0xc0,0x1a,0xac,0x10, +0x9a,0xd4,0xac,0xb7,0xed,0xda,0x35,0xc3,0xc2,0x41,0x58,0x5d, +0x1d,0x7f,0x69,0x6e,0x1b,0x86,0xd2,0x65,0xde,0xd8,0xc5,0x23, +0xcb,0x8c,0x75,0x6d,0xcf,0xe0,0xe4,0x6b,0x4e,0x21,0x8b,0x59, +0xf7,0x4d,0x03,0x69,0xba,0xec,0x8d,0xa9,0x9a,0xc8,0xc0,0x8f, +0xeb,0x97,0x9a,0x06,0x67,0x7f,0x1e,0x5b,0x3f,0x41,0x14,0x13, +0x70,0x43,0x44,0x32,0xe1,0x07,0xd1,0xcc,0x26,0x43,0x57,0xc4, +0x7f,0xa8,0xaa,0x67,0xb3,0x32,0xd2,0xbc,0x3d,0xda,0xaa,0x89, +0x5b,0x2d,0xc6,0x63,0x77,0x1f,0xd4,0x5e,0x15,0x45,0xb6,0x08, +0xf3,0xc0,0xa9,0x5f,0x37,0x17,0x8c,0x09,0x3d,0xaa,0xb1,0x59, +0x01,0x91,0x55,0x75,0xf9,0xb6,0x1b,0xa7,0xd7,0xe3,0xd9,0xb3, +0x4f,0x2c,0x0a,0x88,0xbe,0x97,0x9a,0xab,0x0e,0x55,0x79,0x55, +0xa5,0xe9,0x0a,0x88,0x95,0xb3,0x79,0x11,0x35,0x7b,0x35,0xdf, +0xda,0x72,0xf1,0x8d,0x4c,0x9c,0x7f,0xb5,0x53,0x9c,0xd1,0xe6, +0x72,0x96,0x81,0xf3,0x03,0xb5,0x52,0xcc,0x0e,0xa8,0x5d,0x3b, +0x9d,0x09,0xe2,0xd5,0x7c,0xb3,0xad,0x95,0x82,0x8f,0x8f,0x43, +0xbc,0x68,0x4b,0x6e,0xbe,0x1f,0xd0,0x91,0xa5,0x67,0x9a,0x3f, +0xc5,0x4f,0xd1,0x5c,0x67,0x33,0x33,0x70,0x82,0x42,0x01,0xc7, +0xbe,0x42,0x48,0x5b,0xa6,0x27,0x1f,0xcd,0x3d,0xe6,0x16,0xdd, +0x86,0xd8,0xae,0xd2,0xa5,0xf2,0x84,0x44,0xfa,0x61,0x9b,0xdc, +0x56,0x98,0x1d,0x19,0x68,0xa8,0x7d,0x38,0xa9,0x6e,0x2c,0x50, +0x04,0x08,0x1b,0x70,0xbe,0x48,0xc9,0xf6,0x74,0x4a,0xfd,0x70, +0xdb,0x96,0x1d,0xb0,0xb1,0x59,0x0b,0x2e,0x4b,0x26,0x98,0xc2, +0x42,0xce,0x3b,0x4f,0xbd,0x22,0x13,0x95,0x0b,0x64,0x6e,0x56, +0x97,0xa8,0x1f,0x92,0xa0,0x30,0x5e,0xdf,0x4d,0x76,0x59,0x5c, +0x94,0x2d,0xb5,0x65,0xec,0xe8,0x20,0x79,0x0b,0xad,0x08,0x65, +0x3a,0xc9,0x03,0xc3,0xf2,0x3f,0x29,0xef,0x35,0x5c,0x0d,0x4f, +0x67,0x23,0xe5,0xd8,0xcc,0x17,0x14,0xed,0xfb,0x4f,0x57,0x93, +0x4f,0x55,0xe3,0xb2,0x10,0x3c,0x12,0x86,0x89,0x8f,0xa1,0xaa, +0x52,0x85,0xa8,0xbd,0xae,0x09,0xa7,0x89,0xd5,0x5f,0xc9,0x57, +0x9a,0xc6,0xf8,0x20,0xe5,0x65,0x1b,0xbe,0x46,0x34,0xdd,0xa7, +0xc3,0xe9,0xa2,0xe2,0x3c,0x72,0xff,0xf2,0x38,0x88,0x3f,0x8a, +0xab,0x4f,0x49,0xde,0x1f,0x28,0xb2,0xd5,0xe0,0xec,0x7e,0x34, +0xe6,0xe1,0xdd,0x39,0x10,0x2c,0x5b,0x8e,0x6e,0x3e,0xb8,0xe0, +0x6f,0x27,0xf1,0xd4,0x67,0xc2,0x94,0xaf,0x97,0x58,0xa4,0xaf, +0xdc,0x4a,0xd6,0xf9,0x65,0xc0,0x6e,0x34,0xc9,0x71,0x25,0xc6, +0x09,0x22,0xde,0x6e,0x9b,0x0f,0x1a,0x2c,0x24,0xd4,0x64,0xb4, +0x53,0x93,0x66,0xcc,0x31,0x50,0x84,0x7a,0x7b,0xf8,0x12,0x30, +0xa4,0x98,0xf9,0x99,0x87,0xeb,0xde,0x0a,0xde,0x41,0x2e,0xb9, +0x1b,0xce,0xdb,0x30,0xc6,0x91,0x37,0x1e,0x8a,0x56,0xe8,0x78, +0x4c,0xef,0xb0,0x47,0x76,0x66,0xfc,0xb9,0x41,0x0a,0xb2,0xd1, +0x60,0x39,0x63,0xc9,0x69,0x98,0xe8,0x37,0x83,0xae,0x8d,0xdc, +0x8d,0xff,0xa1,0x8c,0x2d,0x11,0x41,0x97,0x63,0x6e,0x5a,0x8e, +0xfe,0xae,0x53,0xc9,0xb3,0x5e,0xef,0x68,0x31,0xc7,0xaa,0x96, +0x24,0xde,0x77,0x24,0xd2,0x87,0xae,0x5c,0x79,0x4b,0x6d,0x91, +0x5f,0x36,0xeb,0xc5,0x2a,0xac,0xe2,0x52,0x79,0x60,0x9d,0x70, +0x73,0xb6,0xcc,0x74,0xce,0x7d,0xa0,0xe1,0x35,0x6b,0x26,0x11, +0xfb,0xc1,0xb5,0x04,0x5e,0x4c,0x83,0xe6,0xd5,0x24,0x44,0x1e, +0x65,0xec,0xd1,0x5f,0x99,0x99,0xe2,0x80,0xe6,0x8c,0x9c,0x73, +0xc8,0xa1,0x01,0xab,0x56,0x0a,0x47,0xc3,0x40,0x8b,0x3e,0x40, +0x2f,0x81,0x5e,0x30,0x41,0x01,0x50,0x8f,0xf1,0x18,0x6b,0x96, +0xff,0xce,0xd5,0x21,0xfe,0xfa,0x1b,0x57,0x84,0xb7,0xf1,0xbd, +0x6d,0x45,0xe0,0x45,0xb8,0x6d,0x05,0xf8,0xa5,0x7a,0xd0,0x4c, +0xef,0x52,0x1d,0x6e,0x53,0xd8,0x2f,0xd7,0xd1,0x86,0x48,0xd3, +0x57,0x3d,0xc3,0xcc,0xe0,0x6e,0x6a,0x1a,0xd5,0xf7,0xd5,0x14, +0xf0,0x8d,0x27,0xf3,0x8f,0xb3,0x82,0x6f,0x70,0x46,0x43,0xe0, +0x02,0xc7,0xf8,0x9f,0x16,0x1a,0x4b,0x58,0x89,0xc4,0x9e,0xb1, +0xeb,0x01,0x49,0x11,0xfc,0x25,0xcb,0xf1,0x1a,0x7f,0x3e,0xf3, +0x36,0xfe,0x19,0xd2,0x86,0xe2,0x5f,0xdf,0xad,0x4c,0x02,0x01, +0xe3,0x04,0x81,0xe5,0xd3,0x04,0xdc,0x26,0x76,0x37,0xc9,0x7c, +0x87,0x07,0x7c,0x4d,0x3e,0x88,0xeb,0x04,0xa8,0x0a,0x73,0x2c, +0xbe,0x88,0xa1,0x50,0x86,0x2d,0x15,0x5c,0x09,0x01,0x01,0x57, +0x13,0x63,0x50,0x0d,0x66,0x0c,0x18,0x81,0xaa,0x5d,0xcf,0x14, +0x9d,0xa1,0x4f,0x02,0x11,0xf6,0xbb,0x42,0x77,0xc4,0x57,0xb3, +0x76,0x5b,0x29,0xf0,0x86,0xb7,0x77,0x2a,0x03,0xc7,0x1e,0xb5, +0x41,0xe9,0xbc,0xc2,0x36,0xa9,0xb0,0x10,0xf8,0xd7,0xda,0xd2, +0x71,0xa0,0x38,0x29,0xd3,0xbc,0xee,0xfa,0x86,0x9e,0x38,0x02, +0xb2,0xf4,0x04,0xc2,0xfc,0xea,0xaa,0x77,0x50,0x78,0x8b,0xf2, +0xd2,0x37,0x9b,0xbb,0x67,0x3f,0x0a,0xfb,0x81,0xe7,0xbb,0xa1, +0x35,0xb0,0xb5,0x55,0x2f,0x8e,0xe8,0xdb,0x12,0x83,0x22,0x4e, +0xf6,0xae,0xd9,0xc4,0xc8,0x3d,0x66,0x97,0xb3,0xd3,0x05,0xc9, +0xec,0x77,0xc6,0xaa,0x5f,0x12,0xc3,0x67,0x1f,0x7e,0xa3,0xb9, +0x6e,0xfc,0xd5,0x01,0xcf,0xed,0x47,0x59,0xb7,0xc3,0x68,0xdf, +0x9c,0xb9,0x38,0xbe,0x37,0x73,0xc8,0x1e,0x25,0xae,0xae,0xe6, +0x93,0xdb,0x25,0x67,0xa4,0xee,0x65,0xd0,0x7e,0x62,0xb7,0xdc, +0xa3,0xa9,0xc5,0xa1,0x65,0xbd,0xb9,0xc5,0xba,0x40,0xcb,0xe4, +0x34,0xa5,0xb5,0xb4,0xee,0x1f,0x51,0x52,0x18,0x76,0xf9,0xbc, +0x4a,0x17,0xdf,0x50,0xf0,0xcd,0xf3,0x6c,0xff,0x91,0xa7,0x2a, +0xe4,0xfc,0x35,0xe6,0x48,0x3e,0xc5,0x94,0x32,0x11,0xd2,0x3e, +0x19,0x8a,0xd8,0x87,0x2e,0x31,0x39,0x46,0x2a,0x6e,0x8d,0x87, +0x39,0x7a,0x71,0xc1,0xd3,0x8b,0xf8,0xe6,0x81,0x3d,0x9c,0xa3, +0xc9,0xf7,0xd1,0xc9,0x0a,0xe7,0x18,0xc5,0x94,0x7a,0x44,0xca, +0x05,0x81,0x0f,0xcd,0x2e,0x0a,0xaf,0x56,0x0c,0x6b,0x94,0x3b, +0x2c,0xf7,0xf6,0xe0,0xa8,0x3c,0x0a,0x40,0xba,0xdd,0x41,0xc4, +0x2b,0x8a,0x15,0xe5,0xdc,0x0e,0xff,0xa0,0x1f,0xf4,0xec,0xd6, +0x1a,0x0b,0xa2,0x4f,0x20,0xe8,0xb8,0x05,0x6e,0xda,0xa4,0x02, +0xe2,0xd1,0x71,0x16,0x6f,0xb5,0xf2,0x13,0x73,0x46,0xd2,0xcc, +0x82,0x25,0x1f,0x4d,0x68,0xc2,0xec,0xf0,0xdb,0x3c,0xca,0x68, +0xc7,0x51,0x9e,0x14,0x14,0x65,0x78,0x3e,0x42,0x41,0xd8,0xe7, +0x52,0x36,0x0d,0xc7,0xbf,0x01,0x7a,0x16,0xd4,0x49,0x08,0xaa, +0x7d,0xdf,0x40,0x74,0xd1,0x01,0x93,0x0a,0x32,0x7d,0xc6,0x60, +0xb7,0xc1,0x79,0xbc,0x99,0x80,0xc0,0x9d,0xce,0x3f,0x16,0x7f, +0x3a,0x38,0x80,0xc3,0xb7,0x5a,0x15,0x28,0x92,0xe6,0xb8,0x24, +0xe4,0x67,0x28,0x64,0x0d,0xf8,0xf7,0x9c,0xdf,0x6d,0xcd,0x52, +0x0e,0xdb,0xd1,0x16,0x7b,0x96,0x2a,0xd3,0x89,0xde,0xe8,0xc2, +0xe5,0xd5,0x64,0x17,0x44,0xae,0xe2,0x2b,0xa6,0xac,0x37,0x99, +0x2c,0x86,0x80,0x9a,0x64,0x3b,0x98,0xbe,0x3a,0xa7,0xc6,0x0f, +0x28,0x6b,0x6c,0x82,0x2a,0xf3,0x87,0x1e,0xb1,0x37,0xad,0x9f, +0x0f,0xef,0x49,0x38,0x72,0xe3,0xee,0x0e,0x08,0xae,0x09,0xf9, +0x55,0x6c,0x4c,0xd3,0xed,0x77,0x62,0x91,0x4e,0xd4,0x88,0xe2, +0x68,0x4e,0xbe,0x42,0x51,0x6a,0x27,0xbf,0x98,0x7f,0x2a,0xef, +0x91,0x56,0x41,0x86,0xf7,0xd5,0xaa,0x80,0x91,0xa6,0x3e,0x89, +0x97,0x75,0x74,0x82,0x6f,0xfd,0x21,0x06,0x26,0xe6,0x49,0xad, +0x6a,0x7e,0x99,0x39,0x3a,0x8b,0xc0,0xa2,0x34,0x67,0x2c,0x09, +0xaa,0xa3,0x61,0xfd,0x62,0x53,0x13,0x68,0xe8,0x7b,0xa9,0x75, +0x59,0x76,0x3d,0x5e,0xfd,0x80,0x9e,0x38,0xe0,0x9c,0x39,0x99, +0x4e,0xa0,0xf0,0x5b,0xb8,0xbc,0x9a,0x1c,0x97,0xc1,0xfc,0x62, +0xeb,0xb7,0x17,0x42,0x65,0x23,0xe8,0x63,0x49,0xd6,0xaa,0xf1, +0xd1,0x8f,0x5a,0x9b,0x53,0x77,0xa1,0x64,0x17,0x7e,0x3b,0xf8, +0x1c,0xd8,0x5d,0x0c,0xaf,0xc7,0xff,0xc5,0x43,0x46,0xd2,0xac, +0x4f,0x89,0x8b,0xa4,0x7c,0x99,0x6c,0x4f,0x9b,0xdb,0xbd,0x24, +0xd8,0x67,0x28,0xbe,0x76,0x90,0xf1,0x80,0x21,0x20,0x0c,0x78, +0x48,0x7f,0x6f,0x86,0x44,0xc6,0xc5,0x63,0x50,0x18,0x89,0xb0, +0x36,0x19,0x02,0x72,0x6b,0x5a,0x96,0xcc,0xbd,0x9b,0xc9,0x2d, +0xac,0xf5,0x89,0x18,0xee,0x53,0x9f,0x68,0x86,0x9b,0xc7,0x06, +0x51,0x35,0x25,0xc9,0x8e,0x12,0x7c,0xac,0x9c,0x5c,0x92,0x6e, +0x94,0xb3,0x77,0x6b,0x4c,0xac,0x3a,0x53,0xb2,0x65,0xb2,0x1c, +0x4f,0x49,0x81,0x3d,0xe1,0xf5,0x02,0x08,0x2b,0x74,0x19,0x99, +0xe0,0x6c,0xa9,0xfe,0x6e,0x49,0xf3,0x2c,0xef,0xa9,0x6c,0x9a, +0x1e,0x22,0xb9,0xa5,0xa5,0x24,0x4b,0x1f,0xd2,0x4c,0xac,0xf5, +0x22,0x83,0xab,0xd6,0x3c,0xd2,0x97,0x96,0x0c,0xc4,0xe7,0xaa, +0xe5,0x20,0xe3,0xbb,0x24,0xf6,0x70,0x89,0x86,0x1d,0xc7,0x6a, +0xbe,0xb7,0x4c,0xbd,0x73,0x33,0xd3,0x23,0x24,0x2d,0x26,0x9f, +0xc6,0x23,0x1b,0xe7,0x0f,0x4d,0xf5,0x01,0xc9,0x7c,0xcc,0xbc, +0xef,0x5a,0x66,0x9a,0x30,0xc8,0x1e,0xef,0x1d,0xca,0xab,0xd1, +0x3d,0x71,0xfe,0x5d,0x95,0xf7,0x9b,0xa3,0x5a,0x3b,0x7a,0xa9, +0xcd,0xa1,0x7d,0xf7,0x56,0x79,0x73,0x1e,0xbc,0xd2,0xe4,0xc2, +0x75,0xa7,0x83,0x19,0xc4,0x93,0x9d,0xfb,0x22,0x95,0x12,0xc6, +0x2a,0xb7,0x6b,0x67,0x86,0x6f,0x9b,0x5a,0xb4,0x96,0x81,0x56, +0x18,0xad,0xcc,0x46,0x84,0x81,0x0e,0x1d,0xa3,0x85,0xee,0xf4, +0x71,0x86,0xea,0x5c,0x78,0xe2,0xa7,0x11,0xb6,0x0e,0xd3,0x9d, +0x4e,0xb5,0x10,0xe9,0x5d,0xc6,0x50,0x31,0x6c,0xe3,0xf7,0x5c, +0x29,0xda,0x52,0xdc,0xfe,0xb8,0xfc,0x94,0x8d,0xed,0xeb,0x73, +0x1a,0xb4,0xdf,0xa6,0x18,0x10,0xb4,0x99,0xa4,0x8f,0x59,0x9e, +0xa0,0x91,0xc2,0xeb,0xc4,0x7b,0x16,0xe9,0x69,0x31,0x4a,0x30, +0x8d,0x67,0x3c,0xbc,0xd0,0x29,0xc0,0x25,0xc4,0x51,0x1e,0x05, +0x31,0xac,0x3d,0xe7,0x23,0xcf,0xdc,0x5c,0xa8,0xca,0xc2,0xc7, +0x9e,0xe0,0xd0,0xd4,0xa0,0x37,0xc4,0x45,0xcd,0xf9,0x23,0x67, +0x9e,0xaa,0x2c,0xf5,0x07,0x0e,0x4e,0x65,0xc3,0xb8,0xd5,0xf8, +0xc0,0xd6,0xec,0xc9,0x76,0x7f,0xe8,0x24,0x2b,0x52,0xdb,0x1a, +0x79,0xb9,0x31,0x31,0xb0,0xd2,0x04,0x8a,0x49,0xf2,0x82,0x6e, +0xce,0x53,0xad,0x2d,0x83,0xe7,0x3b,0x5b,0x21,0x43,0xb3,0xbc, +0x41,0xcf,0x6c,0x36,0x07,0x2c,0x0b,0x7a,0x4f,0x73,0x6d,0xbb, +0x55,0x03,0xcd,0xa6,0x90,0x61,0x2d,0xb0,0x2d,0x64,0x62,0xbf, +0x06,0x10,0x68,0xaa,0xbf,0x02,0x04,0x2e,0x05,0x0f,0x43,0x4e, +0xf0,0xb9,0x6b,0xc5,0x5e,0xdc,0xac,0xbd,0xda,0x60,0xb3,0x25, +0x62,0xf2,0x4f,0xb8,0xa3,0x27,0xf7,0xdc,0x37,0x62,0xaf,0xd6, +0xac,0x6d,0x20,0xc8,0x97,0xa1,0x88,0x69,0xe4,0xb6,0xb9,0x5f, +0x69,0xa3,0x57,0x9f,0x9a,0x1f,0x9e,0xcd,0x07,0x74,0x2a,0x6c, +0x36,0x4f,0x47,0x35,0xce,0xaf,0x4e,0x2e,0xa9,0xaa,0xea,0xd9, +0xbf,0x65,0xb4,0x83,0xab,0x22,0x0b,0x5a,0xd7,0xb0,0x16,0xf8, +0xc3,0x98,0xeb,0xfb,0x8a,0x96,0x37,0x91,0xf8,0xee,0x62,0x2c, +0x12,0x73,0x73,0x26,0xb9,0xbb,0x7b,0x39,0x5e,0xee,0x66,0x6b, +0x4d,0x5b,0x2c,0x96,0xb1,0x25,0x76,0xa2,0x0b,0xe0,0x4c,0xda, +0xff,0xaf,0x7d,0x35,0x36,0x0d,0xd1,0xe6,0xf9,0xa1,0x66,0x75, +0xcd,0x58,0xe3,0xfd,0xb2,0xff,0x48,0xa9,0xeb,0x83,0xda,0xc9, +0x0d,0x71,0x8a,0xc0,0x4d,0x45,0xe4,0x8d,0xaa,0x07,0x59,0x7b, +0xd8,0x18,0x24,0x38,0x31,0x0c,0xff,0xf8,0x71,0x19,0x2d,0x6b, +0x75,0xb7,0x0d,0x95,0xb6,0x97,0x1d,0xaf,0xea,0x47,0x2a,0x00, +0x1c,0x66,0x82,0xfe,0x4f,0x7b,0x5e,0xaa,0x29,0x33,0x88,0x46, +0x4a,0x8e,0xa4,0x72,0xcc,0x00,0xbf,0x45,0x3d,0xd3,0x19,0x52, +0x28,0x59,0x50,0x01,0x1a,0x28,0xc8,0x59,0xd0,0x18,0x10,0x7c, +0xdf,0xe9,0x9a,0xdf,0x48,0xf1,0xae,0x14,0xcf,0xa6,0xf8,0x26, +0x74,0xae,0x51,0x68,0xc7,0x16,0xe2,0x20,0x25,0xe6,0x9a,0x04, +0xa6,0xf7,0xb8,0xbc,0x33,0xa2,0x4e,0x6e,0xce,0x93,0x0e,0xe5, +0x13,0x95,0x04,0xb6,0x48,0x87,0x0f,0xf2,0xf3,0xbb,0x95,0x26, +0x1a,0x98,0x42,0x2a,0x3c,0x25,0x1f,0x1f,0x18,0xfb,0x2a,0x32, +0x05,0xab,0x71,0x66,0x0d,0xf0,0x86,0xb6,0xe2,0x49,0x97,0x37, +0x80,0x9c,0x92,0xdd,0xef,0xb8,0xcb,0xe4,0x8e,0xd7,0x57,0x61, +0x9e,0xa5,0x1c,0x83,0x9d,0xc1,0xa9,0xbb,0xaf,0x30,0xb7,0xd4, +0x25,0x4e,0xc0,0xd4,0x15,0xe0,0xce,0x41,0x4f,0x4e,0x15,0xe8, +0x17,0x9f,0x74,0x46,0x51,0xe2,0x4b,0xab,0x92,0x46,0xc4,0x34, +0x1e,0xd6,0xd0,0x51,0x6c,0x34,0xd0,0x2e,0x51,0xf6,0x0d,0xb2, +0x75,0xc9,0x25,0x8c,0x90,0xf2,0xc8,0x0e,0x8a,0xbf,0x94,0x91, +0x9b,0xc6,0x5e,0xaa,0xa4,0x40,0x0d,0xb9,0x27,0x6a,0xff,0x76, +0xf8,0x89,0x91,0x49,0x74,0x5e,0x4e,0xea,0x0b,0xb0,0xca,0x04, +0x1a,0x52,0x11,0x71,0x94,0xc9,0xcb,0xe7,0x47,0x90,0xb7,0x21, +0x2a,0x68,0x5f,0x96,0x15,0x35,0x87,0x8a,0xbd,0xf8,0xc8,0xa6, +0x11,0x12,0x7f,0x34,0xc1,0x93,0x6e,0x34,0x7e,0xea,0xd7,0x86, +0xa3,0xdc,0x22,0xe8,0x89,0x7d,0x7f,0xa3,0x2d,0xa9,0x49,0x6e, +0x07,0x78,0x07,0x66,0x5c,0x81,0x8a,0xbb,0xef,0xca,0x39,0x9b, +0x27,0x3a,0x69,0x9e,0x45,0xc5,0xd0,0x50,0xf4,0x15,0x7f,0xbb, +0xc3,0xdb,0x91,0x0d,0x0b,0x20,0x8d,0x4a,0x13,0x36,0x91,0x83, +0x2f,0x2b,0x0d,0xce,0x7a,0x38,0x33,0xb4,0x43,0xc5,0x1a,0x7d, +0x1e,0x8e,0xc8,0x75,0xb2,0xbc,0xd2,0x67,0x76,0x0f,0x48,0xd5, +0xb8,0x85,0x0f,0x44,0x97,0xa8,0xf6,0xab,0xc6,0xe4,0x6e,0x27, +0xb3,0xa7,0xcb,0x25,0x5a,0x35,0xbb,0x9b,0x7a,0x0e,0x7e,0xc4, +0x7e,0xb9,0x85,0xd5,0x08,0x27,0x32,0xa2,0x8a,0x68,0x3b,0xc7, +0xed,0x13,0xbc,0xad,0xd9,0x50,0x75,0x99,0xf4,0x93,0x8e,0x40, +0xe9,0xf1,0x4f,0x07,0xd6,0x37,0x2a,0x48,0x77,0xb0,0x4c,0x27, +0x19,0xb4,0xd0,0x04,0xe1,0xed,0x02,0xf1,0x68,0x03,0xe1,0x70, +0xe0,0x62,0x8f,0x97,0xd7,0x95,0x35,0x73,0x4a,0x6f,0x59,0x93, +0xab,0xcf,0x86,0x88,0xbe,0x78,0xef,0x9a,0x68,0xd1,0x5c,0x88, +0xee,0xc1,0x1f,0xc0,0x02,0xd8,0x46,0x4c,0xea,0x6d,0x6b,0xec, +0xb7,0xce,0xef,0xfb,0xe7,0xd5,0xf9,0xe9,0x00,0x6d,0x20,0xa3, +0xbe,0xf6,0xa4,0xfa,0x49,0x07,0x85,0x4b,0x26,0x6e,0x90,0x12, +0x48,0x14,0x08,0x0a,0xb3,0x40,0xad,0xd8,0xae,0x6b,0x0b,0xc5, +0xb3,0xb3,0x3d,0x19,0x94,0x52,0x3c,0x19,0x68,0x2f,0x7c,0xcd, +0xe7,0xf7,0xe7,0xa3,0xce,0xf9,0x66,0xff,0xda,0x5c,0x60,0x28, +0x8c,0x2b,0x8f,0xa7,0xcb,0x6b,0x61,0xff,0x77,0x28,0x56,0x6c, +0xd9,0x1e,0xe6,0x7b,0x87,0xee,0x38,0x60,0xe6,0xfb,0xa3,0xef, +0xed,0x08,0xc9,0xd9,0xd4,0x43,0x2e,0x42,0x69,0x13,0xfa,0x92, +0x7b,0x90,0x9b,0xa1,0x81,0x4c,0x59,0x0f,0xfe,0x34,0x0f,0x4d, +0xef,0xa1,0xb1,0xa1,0xa2,0x8d,0x63,0xa3,0x40,0x8b,0xd5,0xfc, +0xdf,0x2b,0xa9,0x24,0x13,0x33,0x3f,0x59,0xae,0xe3,0x22,0xbf, +0x9d,0xe4,0x7c,0x46,0xf6,0x36,0xf6,0xf7,0x49,0x47,0x09,0x79, +0x31,0xd7,0x40,0x4d,0x2d,0xdf,0x03,0xf6,0xb3,0xbc,0xde,0x4f, +0x3a,0x1f,0xc6,0xcb,0x8a,0x9e,0x69,0x61,0x7b,0x75,0x92,0xfd, +0xfa,0x3a,0xda,0xf7,0xeb,0x28,0x9f,0x94,0x8f,0x8e,0x26,0xdf, +0xc7,0x27,0x35,0x7b,0x5f,0x8e,0x2a,0x46,0x63,0x4a,0x93,0x5e, +0xd2,0x83,0xd2,0x6d,0x34,0xcb,0xb0,0x00,0xc8,0x93,0xbd,0x47, +0x5b,0xcd,0xcd,0xfb,0x49,0xf6,0xcb,0x73,0x02,0x87,0x8e,0x13, +0xc8,0x19,0x7f,0x6c,0x91,0x09,0x33,0x9d,0x7a,0x38,0x0b,0xdc, +0x3e,0x12,0xeb,0xf1,0x2f,0x26,0xbf,0x8e,0x79,0x99,0xa2,0x3f, +0x5a,0xe6,0x10,0x5f,0xbc,0x5f,0xaf,0xdd,0xa9,0x8c,0x69,0xc2, +0x3b,0xd9,0x39,0x3c,0x12,0xc7,0xd4,0xf8,0x1a,0x85,0xeb,0x57, +0x3a,0x23,0xf5,0x1d,0x96,0x54,0xd8,0x88,0x79,0xc8,0xd4,0xef, +0x1c,0x14,0x6a,0xac,0x1e,0xc1,0xad,0xd7,0x3b,0xd1,0x66,0x80, +0x8b,0x0b,0xed,0x7d,0x28,0x8f,0xb2,0xc6,0x2f,0xd7,0xeb,0xcd, +0xdb,0x2f,0xe6,0xd0,0xde,0x21,0x79,0x73,0xc4,0x25,0xee,0x3a, +0x06,0x4b,0xfd,0x29,0xde,0xd4,0x10,0x9b,0x4f,0x48,0x6c,0x74, +0xbe,0x54,0xbb,0x9d,0x5e,0x83,0x08,0x7a,0x8b,0xc5,0xf0,0x3d, +0x36,0x6c,0x4a,0x66,0x74,0x8b,0xf0,0x3b,0x71,0x7c,0x5f,0xb8, +0x44,0x21,0x1e,0xd0,0x54,0xae,0xdb,0x7d,0xe2,0x51,0xaf,0x1f, +0xd8,0xfd,0x9b,0x61,0xf5,0xfa,0xe3,0x4c,0xdd,0x2a,0x92,0xcd, +0xac,0x76,0x5b,0x5b,0x22,0x46,0x11,0xc4,0x18,0x02,0x64,0x27, +0x35,0xa5,0x1c,0x33,0xf1,0xeb,0x2e,0x37,0xac,0x2b,0xbd,0x60, +0x88,0xf2,0xb1,0x53,0x96,0x12,0xca,0x4c,0x6c,0x5a,0xeb,0xa4, +0x03,0x14,0xcf,0xb1,0xf6,0xad,0x3c,0x38,0x82,0xbf,0xdf,0x9b, +0x19,0x84,0x4f,0x58,0x90,0x5f,0xdd,0xc8,0x68,0x8c,0xfe,0xb7, +0x7a,0xad,0xe8,0xf9,0xc5,0x7b,0x3f,0x87,0x15,0x69,0x1f,0xfd, +0xc7,0xf8,0x73,0xc5,0x6b,0x53,0x66,0x0d,0xf9,0xf2,0x4e,0x0c, +0xf6,0x6b,0xea,0xc7,0x32,0xcc,0x98,0xb6,0x55,0x51,0x6c,0x45, +0x4a,0x19,0xb5,0x65,0x73,0x4a,0x35,0x6f,0x59,0x3c,0xa6,0xea, +0xaa,0x8c,0xda,0xc8,0xde,0xd9,0xbf,0x27,0xd8,0x66,0x8f,0x3f, +0x38,0x02,0xb8,0x5d,0x69,0x14,0xd8,0x48,0x85,0x69,0x9e,0x6f, +0xd7,0x72,0x8c,0xc7,0xcd,0x58,0xdc,0x2d,0x28,0xb8,0x67,0xb3, +0xfa,0xd5,0xc7,0x58,0x02,0x99,0x68,0xf3,0xd9,0x8c,0x57,0x0c, +0xb3,0xf9,0x67,0xd0,0x88,0x9f,0x26,0x23,0x43,0xe9,0x8d,0x26, +0xd7,0x93,0x95,0x9c,0xef,0x77,0x93,0x51,0xf0,0xb6,0x7b,0x44, +0x87,0x83,0x3c,0xf3,0xe6,0x94,0x13,0xf3,0xe8,0xcb,0x29,0x6a, +0xb3,0xa3,0x19,0xae,0xe3,0x55,0x0a,0x44,0xeb,0x1f,0xff,0xad, +0x2c,0x29,0x8b,0x32,0x5c,0x7d,0xce,0x32,0x39,0x4e,0x72,0x04, +0xce,0x9e,0x20,0x12,0x12,0x87,0xf8,0xcb,0xc1,0xce,0xd6,0xfc, +0x7c,0x7c,0x77,0xaf,0x96,0xf3,0xdb,0x13,0xa9,0x24,0xa5,0xbc, +0x9d,0xc3,0x2c,0x04,0x64,0x2b,0x39,0x48,0x36,0x1e,0x94,0x13, +0x30,0x86,0xe8,0xb0,0x8c,0x59,0x6d,0xe3,0xd5,0x8f,0xc3,0xea, +0x06,0xa6,0x92,0xe6,0x1d,0xee,0xbb,0x1b,0xba,0x9d,0x77,0x77, +0x6f,0x38,0xba,0xbc,0x29,0xe4,0x31,0x38,0x88,0xf6,0x10,0x84, +0x5f,0x33,0xaa,0xe4,0x84,0xb9,0x29,0x21,0xe8,0x33,0xea,0xcc, +0xea,0x8e,0xf5,0x38,0x43,0x88,0x29,0xc0,0x1f,0xd8,0xe3,0x50, +0xb4,0xdd,0xb6,0xa7,0x58,0xd3,0x0c,0x43,0x26,0xda,0x43,0xe2, +0x96,0x19,0x51,0x3c,0xd3,0x0d,0x48,0x85,0x6e,0xe4,0xf0,0x63, +0x0e,0x7c,0xbe,0x31,0xfc,0x33,0x19,0x32,0xe6,0x9e,0x73,0x64, +0x7e,0x78,0x90,0xd9,0x9c,0xe4,0x4b,0x5a,0x78,0x2d,0x91,0xc3, +0x47,0xe9,0x2a,0xdc,0x2d,0xa9,0x04,0x0d,0xee,0xb5,0xc9,0xef, +0xdd,0x71,0x59,0x70,0xf1,0x4d,0x96,0x29,0xe6,0xea,0x97,0xdf, +0x1c,0xe9,0xba,0x7b,0x53,0x23,0x69,0x2f,0xc2,0xb6,0xdd,0x75, +0xf5,0xec,0xfa,0x54,0xea,0xe2,0x33,0xe7,0x5c,0xaa,0x8e,0xf2, +0x79,0xb2,0x16,0x86,0xc2,0x5e,0x71,0x1e,0x39,0x08,0xca,0x1a, +0x24,0x21,0xf4,0x41,0x65,0xcb,0x3e,0xf9,0x42,0xc9,0x9d,0xad, +0x25,0x45,0x72,0x2d,0x6c,0x2e,0xf3,0x19,0x29,0xae,0xe1,0xb1, +0xbb,0x06,0x43,0x30,0x97,0xb0,0xf2,0x46,0xbf,0x59,0xb6,0x94, +0x38,0x9d,0x0c,0x4b,0xc5,0x97,0x69,0xbd,0xa7,0xf8,0xaa,0x1f, +0x96,0xe9,0x73,0x03,0x5b,0x98,0x34,0xc0,0xa2,0x82,0x8d,0x89, +0x28,0xa3,0x1c,0x28,0x31,0x32,0xb1,0x05,0x08,0xa5,0x7d,0x15, +0x94,0x67,0xc1,0x4a,0x77,0x80,0xea,0xd4,0x49,0xad,0x0f,0xa2, +0x4a,0xd4,0xdc,0x00,0x4e,0xfc,0xaa,0x16,0x18,0xac,0xe7,0xde, +0xdf,0x4c,0x7c,0x0d,0x2b,0x91,0xc2,0x5f,0xea,0x59,0x83,0xbf, +0x86,0xd3,0xf1,0x52,0x12,0xcc,0x2b,0x65,0x30,0xa3,0x97,0x70, +0x35,0x35,0x03,0xde,0x1d,0x43,0xde,0x63,0xb6,0x03,0x44,0xdf, +0x1f,0x09,0x35,0xb5,0x03,0x83,0x8a,0x33,0xa6,0xac,0xeb,0x1a, +0x26,0x0c,0xc8,0x55,0x7d,0xd8,0xb3,0x10,0x27,0x9a,0x54,0x6a, +0xad,0x04,0x71,0x33,0xbf,0x26,0x59,0xe8,0x16,0x7f,0x1c,0x6f, +0x6e,0xbd,0xa6,0x4f,0xec,0x17,0x05,0xd0,0xe5,0x82,0x81,0x84, +0x8c,0xc2,0xed,0x98,0x15,0x8b,0x5a,0xd3,0x35,0x1d,0x1d,0x79, +0x0d,0xe7,0x18,0x0b,0xf3,0x4c,0x27,0x15,0x7b,0x8d,0xc0,0x82, +0x7c,0x57,0x37,0xdf,0x99,0xb5,0x42,0x78,0xe6,0x49,0x39,0x7b, +0x76,0x5d,0x4e,0xef,0x46,0xe3,0x0a,0xce,0x58,0x58,0x74,0xb9, +0x19,0xb2,0xbd,0xc3,0x9d,0x52,0x2d,0x45,0xf8,0x44,0x5b,0x32, +0x4e,0xe3,0xa7,0x4f,0xf8,0x52,0x0b,0x13,0x99,0x73,0xc9,0x63, +0x22,0x69,0xa8,0x8f,0x8c,0xc7,0x3b,0x42,0xf7,0x31,0x83,0x45, +0x03,0x28,0x1b,0x1a,0xf0,0xc3,0x14,0xbc,0xed,0x95,0x62,0x3a, +0xd2,0xd6,0xf9,0x96,0x50,0x9e,0xb7,0x64,0x20,0x45,0x5a,0x63, +0x18,0x28,0x7c,0x45,0x87,0x8d,0xe5,0xe4,0x80,0x08,0x23,0xcf, +0x24,0xdc,0x8c,0x48,0x76,0x2a,0x8f,0xf6,0x0e,0x2c,0xcb,0xcf, +0x69,0x35,0xbf,0x5b,0x5e,0x8e,0x73,0x18,0xb8,0xd5,0x64,0xc6, +0xd2,0xa1,0x74,0xe7,0xeb,0x7e,0xe0,0xf4,0x0c,0xe9,0x02,0x5e, +0xc7,0x12,0x21,0xa6,0xa1,0x66,0xd7,0x2f,0x99,0x2c,0x4f,0x2e, +0x17,0x1f,0xd1,0x93,0xc5,0xc9,0x70,0xf6,0x87,0x15,0x81,0xde, +0x69,0xbd,0x1c,0xfe,0x0a,0xeb,0x12,0x3f,0x26,0xe3,0xaa,0x05, +0x9b,0x97,0x61,0xb6,0xe6,0xcb,0x16,0xc1,0x82,0x06,0x57,0x2b, +0xb4,0x48,0x5b,0xc1,0x3a,0x25,0xf3,0x0f,0x95,0xea,0x9f,0x77, +0x19,0x47,0x88,0x9b,0xc5,0x95,0xc3,0x68,0xda,0x84,0x7a,0x4b, +0x26,0x51,0x43,0x4e,0xa9,0x58,0x6b,0x38,0x1b,0xb5,0x4c,0x41, +0x18,0xee,0xb1,0x5c,0x40,0xb0,0xde,0xa4,0x46,0x3d,0xd0,0xb4, +0x9b,0xf7,0xa6,0x80,0xcc,0xab,0x88,0x71,0xf8,0x49,0xe7,0xbc, +0x76,0xd6,0x6d,0x21,0x56,0x61,0xf2,0x63,0x4d,0x94,0x1a,0x59, +0xd1,0x52,0x8c,0xc1,0x64,0xf6,0x98,0x83,0xee,0x1f,0x93,0xda, +0x8c,0x0f,0x88,0xb0,0x9a,0xcf,0x46,0xe8,0xc1,0x26,0x8b,0xf1, +0x65,0xe9,0x51,0x9c,0xb1,0xb4,0x4d,0xc3,0x88,0xec,0xc8,0xe2, +0x13,0xa6,0xf6,0x1b,0x67,0xfc,0xcb,0x47,0x96,0xb2,0x7e,0xf8, +0x87,0x35,0xb4,0xed,0xf8,0xf5,0x6c,0x5e,0xbb,0xe0,0xfa,0x03, +0xa4,0xe1,0xe9,0x02,0x6a,0xca,0x8a,0x14,0xac,0x49,0xc4,0x67, +0x27,0xd4,0xb9,0x45,0x9e,0xb0,0x3b,0xf2,0xb7,0x95,0x0b,0x92, +0xbb,0x92,0xab,0x70,0xb7,0xb3,0x44,0x10,0xcb,0xb4,0xb9,0x6d, +0xf7,0x80,0xba,0xb8,0x4d,0x65,0x32,0x19,0xc4,0xf1,0x06,0x0e, +0xa8,0xf9,0xc7,0x13,0x2a,0xb4,0xbc,0xcc,0x09,0xff,0x83,0x3f, +0x88,0xc2,0x91,0x6d,0xec,0x78,0xec,0x97,0x97,0x19,0xfc,0x6b, +0xa6,0x55,0x92,0xdd,0x5d,0x54,0x5a,0x41,0x3b,0xc5,0xd5,0xdd, +0x05,0x5c,0xf5,0xe9,0x41,0xfe,0x88,0xda,0xa7,0xa8,0x1e,0x14, +0x95,0xa9,0xd1,0x46,0x59,0xfc,0x6e,0xfc,0xdb,0xdd,0x70,0x0a, +0xb4,0xc8,0x61,0x3e,0x7f,0x44,0xbb,0x64,0x4e,0xf4,0xf8,0x23, +0x25,0xde,0x49,0xab,0x94,0x29,0xfc,0x72,0x7e,0xc8,0x02,0xbb, +0x41,0x86,0x43,0x22,0x8a,0x51,0xf3,0x72,0x7e,0x08,0xf7,0x10, +0xfa,0x4e,0xb0,0xa5,0x3d,0xe9,0x8f,0x2b,0x0e,0xff,0x9d,0x8e, +0x57,0xf9,0xea,0xd0,0xc9,0xb4,0x1d,0xe6,0xab,0x47,0xee,0xe3, +0x11,0x99,0xdc,0x83,0x16,0xac,0x1e,0x05,0xe8,0xce,0x21,0x5b, +0x8c,0x70,0xec,0x80,0xc3,0x4c,0x0e,0x1a,0x9a,0x66,0xfc,0x74, +0xec,0x32,0x8e,0xc1,0x21,0x98,0x1f,0xfa,0x99,0xc7,0x46,0xf9, +0x2f,0xde,0xab,0x32,0xd5,0x50,0x1a,0x48,0x4e,0xf7,0xf5,0xc8, +0xc1,0x82,0xf2,0x9e,0x9d,0x44,0x00,0xfc,0x27,0x43,0x90,0xcb, +0xf9,0xd0,0x9c,0x70,0x50,0xde,0xb2,0x41,0x0e,0x83,0x2f,0x4e, +0x93,0x3e,0x3c,0x32,0x0c,0x12,0x1e,0x96,0x52,0xe6,0x5e,0x69, +0xd4,0x43,0xbc,0x00,0x12,0x37,0xbf,0x48,0x31,0x11,0xb5,0xd4, +0x6e,0x07,0x9c,0x90,0x43,0x9e,0x5e,0x1e,0x22,0x37,0x9f,0x4c, +0x2b,0xcf,0x1f,0x49,0x62,0x54,0x15,0x2f,0x8d,0x9d,0x83,0x8d, +0xad,0xef,0x11,0xd6,0xb7,0xc3,0x19,0xb6,0xad,0xb6,0x86,0xc6, +0x48,0x1d,0x3b,0x81,0x79,0xb7,0xa0,0x85,0xb5,0x46,0xe8,0xf2, +0xd8,0x04,0xb3,0x6a,0x47,0x04,0x12,0x52,0x65,0x19,0x1d,0xaa, +0x00,0x11,0x4d,0x82,0x84,0xef,0x3d,0x57,0x42,0xf8,0x3b,0xc8, +0xa0,0x11,0x8e,0xc4,0xd7,0x0c,0x84,0x6b,0xc3,0x46,0xa3,0xec, +0xed,0x46,0xce,0xd1,0xe8,0x26,0x3c,0xcc,0xe9,0xe7,0x11,0x23, +0x11,0xc6,0xc1,0x1e,0xa6,0xa9,0x1b,0x35,0xe2,0x6a,0x32,0xe6, +0x11,0xe4,0xb6,0xf4,0x22,0x66,0x41,0x0e,0x26,0x12,0x72,0x15, +0x6b,0x38,0x3f,0x0f,0x40,0x06,0x40,0x82,0x2c,0x24,0x46,0xe3, +0x20,0x5d,0x88,0x3f,0xab,0x2b,0x27,0x11,0x8b,0xce,0xb4,0x3f, +0x87,0x8c,0x65,0xf5,0xda,0xf5,0xa8,0x17,0x54,0x2c,0x14,0xe4, +0xa3,0xac,0x70,0xd8,0x96,0x9d,0x2a,0x00,0x88,0xfe,0xf9,0xdc, +0xe5,0x0a,0x9b,0x92,0x37,0x46,0xef,0x6a,0x56,0xb8,0x5a,0x94, +0x39,0x5b,0x27,0xa6,0xe2,0xea,0xa1,0x98,0xa0,0x96,0xd4,0x5e, +0x0f,0xa1,0x69,0xdc,0xb4,0x71,0xe8,0x8c,0xa2,0xd8,0x52,0x30, +0xdb,0x34,0x3c,0x46,0x7f,0xa1,0x56,0xa3,0x3d,0x05,0x09,0x54, +0x19,0xc6,0x5b,0xd6,0x06,0xb3,0x78,0xdf,0x32,0xcb,0x78,0x19, +0x3a,0x05,0x65,0xc7,0xa9,0xf4,0x29,0x23,0xb6,0xff,0xcb,0xf9, +0x6e,0x47,0x9e,0x63,0x71,0x63,0xe0,0x3b,0xa4,0x5f,0xfc,0x85, +0xdb,0xf0,0xde,0x71,0x68,0x99,0xec,0xfe,0xfc,0xfc,0xd5,0x93, +0xd7,0x3f,0x8b,0xbf,0xd0,0x76,0x5b,0xc9,0x02,0x75,0x90,0x2e, +0xb9,0x9e,0xbc,0x3e,0xf9,0xe9,0xe5,0xd3,0x57,0x67,0xc4,0x87, +0xa6,0x13,0x45,0x1d,0xb6,0xa7,0x9c,0xe1,0xf4,0xe4,0xf5,0x9b, +0xa7,0x09,0xe1,0x63,0x71,0xf3,0x09,0x5d,0x5e,0x40,0x13,0x57, +0x9f,0x1f,0xa2,0x59,0x21,0x57,0xcf,0x37,0xb6,0xce,0xf8,0xcf, +0xc3,0xc1,0x10,0x88,0xbd,0xa4,0xd5,0x4a,0x0a,0x7e,0x1b,0xf2, +0xec,0xaf,0xe5,0xfc,0x96,0x6a,0x46,0xdb,0x53,0xe6,0xcd,0x42, +0x8e,0x49,0x8a,0xed,0x51,0x0d,0xc4,0x37,0xe0,0x88,0x02,0xff, +0xda,0xb6,0xff,0xc0,0xcf,0x8d,0x4a,0x86,0xc1,0xee,0x95,0x31, +0x3a,0xd8,0x91,0xc1,0x71,0x9b,0x9e,0x66,0xa3,0x9c,0xa2,0xc9, +0xed,0x4b,0x34,0xcf,0x9c,0x24,0x1d,0x41,0x63,0xf9,0xc9,0x6c, +0x27,0x4d,0xae,0xa0,0xa7,0x1f,0xd6,0xeb,0xe4,0x40,0x7e,0xaf, +0xe0,0x18,0x18,0x4b,0x78,0x36,0xd7,0x80,0xfc,0xf6,0x91,0x0a, +0xfa,0xc0,0x18,0x55,0x4b,0x40,0x1c,0x1e,0x05,0x9c,0x01,0xbf, +0x81,0x71,0x3b,0x42,0xb7,0xce,0x86,0xd7,0xea,0x43,0x38,0xbb, +0x97,0x40,0xf9,0xfe,0xb7,0x17,0x40,0x80,0xb8,0xf8,0xee,0xe5, +0x14,0x8d,0x9c,0xb2,0xa5,0x14,0x89,0x64,0x67,0x76,0xc8,0x43, +0x72,0x06,0x52,0x36,0xd8,0x9f,0xb3,0xa7,0x7f,0x3f,0x7b,0xf7, +0xea,0xf5,0x93,0xa7,0xe5,0xb7,0xf4,0x72,0xf8,0x23,0x64,0x53, +0x78,0xb1,0xab,0x3b,0x07,0x9f,0x60,0x31,0xf8,0xd0,0xe1,0x71, +0xff,0x60,0xe0,0x58,0xe9,0xb0,0xb6,0x1c,0xf0,0x9e,0x1f,0x34, +0xad,0x84,0x9f,0xc8,0x31,0x24,0xe6,0xd0,0xf7,0x7f,0x49,0xbf, +0xef,0xff,0xf2,0x78,0xd0,0x79,0x9c,0xed,0x67,0xe8,0x2c,0xc1, +0xbd,0xa1,0xfc,0xf2,0x3d,0xba,0xa6,0x3c,0xdf,0x1b,0x74,0xb2, +0xfd,0xe8,0x9d,0x27,0x77,0x1e,0x8d,0x75,0xcd,0x7d,0x9f,0x74, +0x7c,0x65,0x2e,0x75,0x63,0xfa,0x2d,0x4d,0x6e,0x68,0x92,0xdd, +0x93,0xcb,0xcf,0x4f,0xc6,0x5b,0xbc,0xe5,0xde,0x9b,0x8e,0x8f, +0xb6,0x65,0x32,0xe3,0xec,0xa1,0xd2,0x42,0x04,0xec,0xf4,0x6f, +0xe4,0xcd,0xf7,0x57,0x09,0x38,0x66,0xf1,0x3b,0x87,0x1d,0x23, +0x31,0x87,0x37,0x72,0x44,0x92,0xba,0x22,0x6c,0x45,0x50,0x2c, +0x2d,0xb4,0xad,0xb3,0x56,0x0f,0x14,0x96,0xb2,0x7e,0xc0,0x06, +0x76,0xe0,0x4b,0x8d,0xd4,0xe2,0x25,0x14,0xc7,0xbb,0xac,0xb9, +0xcb,0xae,0x1c,0x99,0x58,0xf3,0x3c,0x26,0x3c,0x7a,0x32,0x7f, +0x42,0x05,0x09,0x52,0xe0,0x70,0x90,0xf5,0xbe,0x04,0x14,0xf2, +0x14,0x80,0xa9,0x29,0x7f,0xbd,0x67,0x1e,0x5d,0xf8,0xce,0xec, +0x69,0x88,0xc9,0x11,0x94,0x15,0x29,0x34,0xaa,0xec,0x3b,0xbe, +0x3c,0xc4,0x0f,0x7c,0x3c,0x9e,0x4f,0x40,0x6a,0x07,0x0c,0xfa, +0xd5,0xdc,0x0d,0xbc,0x63,0xd0,0xb3,0x29,0x94,0xaf,0x22,0xfd, +0x99,0x49,0x1d,0x30,0x8d,0x7a,0x9a,0x9d,0x73,0x5c,0xf9,0x07, +0x41,0xc8,0x23,0x93,0x40,0x35,0x70,0xd3,0xe5,0x39,0x6c,0x39, +0x61,0x99,0x11,0x9c,0xe5,0x9d,0x83,0xac,0x63,0xb2,0x8b,0xe8, +0x42,0x52,0x98,0xd7,0x33,0xcd,0xee,0x73,0x91,0xa8,0x16,0x0a, +0xb6,0x7e,0x05,0xec,0x0f,0x5f,0x02,0xfb,0xc1,0x42,0x64,0xd3, +0x30,0x7a,0x8f,0xf1,0x87,0x73,0x81,0x5c,0x24,0x89,0xc1,0xdc, +0x15,0xca,0xe9,0xf8,0xfa,0x56,0x16,0xbd,0xe7,0xb7,0xd6,0xaa, +0xc0,0x0a,0xfc,0x7e,0xfe,0xd7,0x47,0x7f,0xda,0xbf,0x9e,0xe4, +0x81,0x5f,0xe5,0xfd,0x7f,0xfd,0xf6,0x09,0x45,0x96,0x41,0xe4, +0xa3,0x1f,0x28,0xb2,0x63,0x79,0xdf,0x0d,0xe0,0x17,0x97,0xab, +0xa7,0x14,0x7b,0xba,0x80,0x72,0x55,0xdc,0x92,0x68,0x77,0xda, +0x0a,0xbe,0x3b,0xa0,0x0a,0xfe,0x67,0xd8,0x94,0x63,0x8a,0x2c, +0xc2,0xa6,0x7c,0xb7,0x7f,0x9d,0x03,0x0e,0x1a,0xc4,0x9d,0x50, +0xc6,0x3c,0x8c,0x44,0x27,0xfa,0x51,0x8b,0x7a,0x09,0x44,0x03, +0xc0,0xa0,0x23,0xf2,0x48,0xfb,0x7c,0x06,0xbb,0x50,0x0e,0xd0, +0xfc,0x62,0x3e,0x5f,0xc1,0x85,0x88,0xae,0x46,0x7c,0xbe,0xf0, +0xf8,0xd5,0x23,0x9d,0x25,0xc7,0x09,0xf3,0xe0,0x35,0x20,0xe9, +0x74,0xae,0x43,0x19,0x95,0x22,0xe1,0x27,0xde,0xdc,0x39,0xb0, +0xef,0x4b,0x68,0x40,0x6a,0xe6,0x68,0xc2,0x65,0x76,0x5d,0x40, +0x7e,0xe8,0xc6,0xec,0x7a,0x8f,0x03,0x9f,0xf6,0x5c,0x10,0x95, +0xc9,0xf4,0x6b,0x90,0xbf,0xfa,0xeb,0xbb,0xe3,0x37,0x6f,0xde, +0x9d,0xbc,0x38,0x3e,0x3d,0x7d,0xf7,0xf6,0xe9,0x5f,0x9f,0xfe, +0xfd,0x4d,0xb9,0x7f,0x5e,0xcd,0xae,0xfb,0x05,0x1c,0xd0,0x90, +0x27,0x2d,0xce,0xab,0x6f,0xe8,0xbc,0x1e,0xbd,0x83,0x03,0xfb, +0xa8,0x97,0xf5,0xce,0xab,0x7d,0xc7,0x14,0xa0,0x2a,0x43,0x91, +0xa5,0x8c,0x7c,0xb0,0xab,0x33,0x17,0xf4,0x2f,0xc9,0xfd,0x75, +0x12,0x35,0x40,0x3a,0x49,0x5f,0x7e,0xf8,0xfc,0x7c,0xc4,0x65, +0xb2,0xdc,0xbb,0x9c,0xd1,0xa1,0x87,0xf1,0xcd,0x93,0xf3,0x73, +0x98,0x36,0xed,0x6c,0x97,0x44,0x3c,0x4e,0xc5,0xfb,0xea,0xf1, +0x14,0x7d,0x45,0x69,0x4b,0xb6,0x65,0x49,0x93,0x2e,0xcb,0xc2, +0x64,0xd2,0x10,0x38,0x5c,0xbf,0xb6,0x48,0x47,0xaa,0xff,0xfa, +0x82,0x7d,0x2d,0x38,0xf0,0xc5,0x70,0x27,0x46,0x25,0x23,0x19, +0x25,0x5a,0x06,0x48,0x68,0xf8,0x69,0x16,0x5c,0x7c,0x3a,0xac, +0xc8,0x6f,0x4e,0x99,0xb4,0x60,0xaf,0x4b,0xb5,0x2e,0xb6,0x03, +0xb1,0x39,0x3b,0x1b,0x6a,0x98,0x48,0x36,0x06,0xe5,0x32,0x67, +0xec,0x96,0xa8,0x97,0xfa,0x4a,0xca,0x71,0xb0,0xa6,0xca,0x54, +0x3d,0x18,0xf1,0x7d,0xe4,0xc5,0x32,0xaa,0x0e,0x6e,0x16,0xd8, +0x17,0x80,0xf5,0x46,0x63,0x30,0x5c,0x01,0x7e,0x76,0x71,0xb7, +0xb2,0xcb,0x00,0xe3,0xb2,0x7b,0xd3,0x9b,0x76,0x9b,0xd7,0x04, +0x26,0x90,0x93,0x77,0x34,0x42,0xba,0xbd,0x1d,0x94,0x4d,0x3d, +0xc0,0xe3,0xc9,0x6f,0x21,0xb9,0x2d,0x95,0xd6,0x36,0x45,0xaf, +0xcf,0xbf,0x03,0x20,0x48,0x2c,0x8d,0xe3,0x4a,0x84,0xf5,0x54, +0x2a,0x10,0xf6,0x83,0x66,0x28,0x43,0x29,0xe7,0x2d,0x08,0x99, +0x5b,0x8e,0x93,0xd9,0x7b,0x9a,0xfa,0x54,0x6e,0xce,0xd5,0xf0, +0xba,0xf4,0xa8,0x13,0x72,0x07,0x65,0xcd,0xf7,0x12,0x0d,0x25, +0x45,0x13,0xea,0x77,0x14,0xb3,0x0e,0x2f,0xb0,0x3d,0x8b,0x11, +0x0c,0xf9,0xf1,0x62,0xd1,0x3a,0x9e,0x92,0x15,0xb5,0x96,0x6b, +0xe7,0x62,0x3c,0x6a,0xa1,0xd5,0x4d,0x12,0x9d,0x6d,0xc9,0x20, +0xb4,0xfe,0x70,0x7f,0xb0,0xf9,0x43,0x92,0x43,0x2b,0xb2,0x8d, +0xf4,0xb0,0x94,0x5f,0x32,0xa6,0x24,0x61,0xf7,0x20,0xd9,0x4f, +0x76,0x17,0xcb,0xf9,0x07,0xd4,0xe5,0xf4,0x73,0xa7,0x51,0xd9, +0xbd,0x86,0x78,0x26,0xd2,0x64,0x77,0x09,0xd5,0x4b,0x5d,0x49, +0xee,0x8e,0xa7,0x41,0x56,0x03,0x0c,0x07,0x8f,0xa8,0x44,0xe8, +0x08,0x95,0x6c,0xbd,0xec,0xb9,0x0e,0x98,0x4e,0x80,0xe7,0x18, +0x73,0x02,0x0c,0xe9,0x87,0xf9,0xaf,0x63,0x6c,0x1a,0xd6,0x46, +0x54,0x0b,0x0c,0x42,0x58,0x75,0xb2,0x8b,0xca,0x7f,0x13,0xb2, +0xb9,0xb1,0xab,0x25,0x31,0x2c,0x2a,0x4b,0xd6,0x80,0x34,0x02, +0xd0,0xb6,0xe6,0x52,0x2c,0xd7,0x32,0x6a,0x26,0x02,0xad,0x71, +0x43,0xbe,0xee,0x2e,0x13,0x7a,0x66,0x11,0xe8,0x4c,0x93,0x5a, +0xb5,0xad,0x4c,0x43,0x99,0x02,0x75,0x73,0xc9,0x75,0xe2,0xa6, +0x17,0xe8,0xdd,0x31,0xc9,0xc2,0x8d,0xc8,0xd8,0x02,0x22,0x5d, +0x52,0x74,0x83,0xa7,0xef,0x93,0xa7,0xcf,0x9e,0xbe,0x7d,0xf7, +0xc3,0xeb,0xd7,0x67,0xa7,0x67,0x6f,0x8f,0xe1,0xf0,0xfd,0xa5, +0x1e,0xb9,0xb3,0xaf,0xc3,0xc4,0xb2,0x6a,0xed,0xf6,0x4e,0x3d, +0x13,0x53,0x90,0x22,0x56,0x47,0xe7,0x5d,0xcf,0x2c,0x6c,0x14, +0x8c,0x37,0x69,0xa5,0x09,0xbb,0x6d,0x5e,0x87,0x49,0x46,0x6d, +0xe4,0x5a,0x43,0xab,0x8d,0xb0,0x82,0x1b,0xb6,0x0a,0xbd,0xad, +0xbe,0xd4,0x1d,0xe5,0xce,0x07,0x13,0x6b,0xb0,0x7c,0x11,0x9a, +0xd3,0x15,0x43,0x37,0x9d,0x44,0x6e,0xd0,0x72,0x86,0x69,0xf1, +0xc6,0x09,0xd9,0x7a,0xb2,0x69,0x36,0xfc,0x75,0xfc,0x8e,0x69, +0x01,0x32,0xa7,0x3a,0x46,0xa7,0x41,0x38,0x0d,0xde,0xcd,0xad, +0xc4,0x94,0x2e,0x04,0x87,0xd9,0xbb,0x24,0x0f,0xba,0x7a,0xfa, +0xea,0xf8,0x3f,0x9e,0xbe,0x3b,0x39,0x3e,0x7d,0x2a,0x67,0xa5, +0x6f,0xe1,0x14,0x08,0x54,0x24,0x54,0xe7,0x0e,0xed,0x48,0x21, +0xdc,0x73,0xd0,0x48,0xae,0x8a,0x33,0x85,0x0e,0x9b,0x36,0x11, +0x57,0xe5,0xdf,0x19,0x9f,0x51,0xe1,0xda,0x32,0x90,0x7f,0x14, +0x89,0xe0,0x5e,0xca,0x47,0x8b,0x4a,0x41,0x8e,0x03,0xdf,0xba, +0x57,0xb3,0x9c,0x17,0x66,0xf1,0xef,0xff,0x89,0xb9,0xde,0x38, +0xfd,0x4f,0x5e,0xd6,0xf8,0x2e,0x85,0xa2,0xa2,0xe3,0x65,0x2d, +0x83,0x4f,0x72,0xcb,0xad,0x96,0xc7,0xed,0x02,0x79,0x6d,0x1f, +0x8f,0xc8,0x23,0x74,0x3d,0x9b,0x49,0x85,0x39,0x92,0x74,0xbc, +0x29,0xb8,0x8b,0xf2,0x4e,0x94,0xb0,0x41,0x7d,0x14,0xfa,0xa7, +0xff,0x0f,0x1f,0xc8,0x4a,0xb6,0x44,0x20,0xe7,0x21,0xff,0xff, +0x40,0x4e,0xf6,0x10,0x2d,0x19,0x01,0xad,0x2d,0x64,0xc0,0xb8, +0x80,0x5b,0x9c,0xe1,0x49,0x6d,0x70,0xb2,0x0a,0x8d,0xf3,0x1f, +0x2f,0x49,0x02,0x2f,0x17,0x9f,0x73,0xc3,0x4a,0x5e,0x71,0x76, +0x50,0x0c,0x2f,0x3e,0x7a,0x87,0xcb,0xf1,0x6f,0x78,0xee,0x0a, +0xa3,0x87,0x4f,0x55,0x7c,0x17,0xba,0x3f,0xdc,0xf0,0x32,0x82, +0x05,0xd5,0x4b,0x04,0x0e,0x84,0xd1,0x8f,0xc9,0x64,0x89,0xc6, +0xc2,0x8f,0x3c,0x0f,0xa9,0xa1,0x09,0xcf,0x66,0xbe,0x11,0xec, +0xd6,0x83,0xa8,0x8f,0xe3,0xd9,0x6c,0xbe,0x92,0x97,0x25,0x2d, +0xdf,0x94,0x8a,0x8c,0x65,0x26,0x57,0xb0,0xd5,0x78,0x7f,0x2e, +0xaf,0x91,0x45,0xd7,0x47,0x59,0x45,0x15,0x46,0x83,0xe3,0xc5, +0x77,0x39,0x12,0x6a,0xe4,0x9a,0xd1,0xf3,0x6c,0x6b,0xe8,0x64, +0x17,0xf2,0xd6,0x35,0x7c,0x27,0x48,0xb6,0x5c,0x37,0x48,0x05, +0x40,0x2c,0xc9,0x42,0x1a,0x43,0x76,0x5d,0x19,0x02,0x7e,0xc5, +0x48,0x0a,0x9f,0x13,0xb0,0xb8,0xa0,0xe7,0xd7,0xb4,0x4b,0x98, +0x93,0x34,0x5c,0xdd,0xe4,0xb8,0x2b,0x9e,0xcd,0xce,0xe6,0x74, +0xaa,0xf3,0x04,0x60,0x7c,0xe6,0x1f,0xa3,0xed,0x73,0x05,0x49, +0x25,0x95,0x98,0x41,0x29,0xdb,0x2e,0x9c,0x46,0xe8,0xb5,0xe9, +0xb9,0x30,0x14,0x51,0xd4,0x0f,0x2d,0x53,0x94,0x46,0xe4,0x89, +0x84,0xa0,0x20,0xf8,0x78,0x42,0xef,0x5f,0x48,0x08,0x8b,0x74, +0x53,0x4e,0x8f,0xdd,0xf4,0x08,0x9a,0xc6,0x50,0xb2,0xe0,0x55, +0x63,0x27,0x68,0x28,0x8e,0x7b,0x28,0x37,0x4b,0x3c,0x53,0x0b, +0x82,0xde,0x6a,0x8b,0x50,0x7e,0x0c,0x7d,0x2f,0xf0,0xf1,0xf7, +0x62,0x4e,0x06,0x8d,0x45,0xf0,0xf8,0xde,0x10,0x3e,0xd5,0xdd, +0x92,0xa8,0x57,0x9e,0x99,0xab,0x21,0x0e,0xef,0x67,0xfb,0x38, +0xdf,0x17,0x67,0x89,0xa9,0x0b,0x97,0x92,0x0b,0x50,0x8e,0x8d, +0xa3,0x8a,0x08,0x8e,0xfc,0x88,0x38,0x73,0x22,0x3b,0x23,0xc9, +0x79,0x9e,0xb2,0x3c,0xe1,0x63,0x36,0xc9,0x63,0x49,0x65,0xc1, +0x14,0x0c,0x7b,0x22,0xd0,0x40,0x90,0xe5,0x8d,0x56,0xb1,0x67, +0x57,0x93,0x6b,0x23,0xf7,0xa5,0x29,0xed,0xb6,0x1e,0xe7,0x11, +0xa3,0x9f,0xae,0x21,0x76,0x00,0x8d,0xc9,0xd2,0x03,0x36,0xd0, +0x2c,0xcd,0x95,0x24,0x19,0x01,0xa3,0xff,0xe3,0x1f,0xc5,0x11, +0x21,0x78,0x31,0xc4,0x95,0x24,0x08,0xc9,0x52,0x95,0x2e,0xd8, +0xf5,0xc6,0x4b,0xfa,0x78,0x40,0x2a,0x87,0x41,0xfc,0xa7,0xd8, +0xda,0xf0,0x45,0x60,0x11,0xe3,0xfd,0x93,0x0c,0xd2,0x7e,0x0c, +0xd9,0xb1,0x79,0x1d,0x76,0xa3,0x73,0xbd,0xd9,0xb0,0x97,0x51, +0xee,0xba,0x9c,0x1e,0x22,0x12,0x6e,0x10,0x84,0x0c,0x90,0xa0, +0x39,0x94,0x84,0x93,0x84,0x57,0x81,0x3f,0x47,0x68,0x03,0x7e, +0x18,0x4e,0xa6,0x88,0x15,0xec,0xb4,0xfe,0x6b,0x7e,0xd7,0x1a, +0x03,0x2e,0x37,0x86,0xa9,0x98,0x54,0xd5,0x62,0x8c,0x8e,0x07, +0x00,0xb1,0x1b,0xcb,0xcc,0xb4,0x70,0x68,0xf0,0xa9,0x1a,0xf6, +0x06,0xee,0xd5,0xd5,0xbc,0x85,0x46,0x78,0x5b,0x93,0x55,0xb7, +0xf5,0xfc,0x0a,0x26,0xe1,0x7a,0x52,0xc1,0xe0,0xe0,0x03,0xf7, +0x50,0x8b,0xf0,0xe0,0x02,0x90,0xe1,0xaa,0xf5,0x19,0xe0,0xa3, +0xb7,0xa9,0xc9,0xd5,0x67,0x82,0x3a,0x22,0x53,0x98,0x40,0x4a, +0xe3,0x5b,0xf8,0xb0,0xa2,0xb8,0x6a,0x0c,0x93,0x3b,0x72,0xbc, +0xef,0x2e,0x9f,0x73,0x8a,0xc2,0xb9,0xd1,0x23,0xd7,0x23,0x77, +0xb3,0x1f,0xa6,0x73,0xf1,0x43,0xc2,0x4b,0xa2,0xb4,0x73,0x14, +0x60,0x64,0x9c,0x90,0xc4,0x63,0x58,0xde,0xbf,0x33,0x50,0x0b, +0x13,0xce,0xdf,0x39,0xf8,0x85,0x0b,0xb9,0x25,0x58,0xb8,0xb5, +0x88,0xcd,0x2b,0xd4,0xac,0x08,0xcd,0x5c,0x11,0x36,0xc2,0x21, +0xb6,0x89,0x66,0x20,0x03,0x27,0xb4,0x77,0xb6,0x66,0x95,0x74, +0x34,0x93,0x32,0x5e,0x7e,0x98,0x5c,0x8e,0xb7,0xe6,0x94,0xf4, +0x44,0x64,0x19,0xb6,0xe6,0xa3,0x54,0xd4,0xcb,0x9d,0x53,0xd7, +0x57,0x5b,0x33,0x6a,0x06,0x08,0x0a,0x16,0x9d,0x64,0xde,0xaa, +0x70,0x54,0x4c,0x10,0xcb,0x37,0xda,0xb3,0x3c,0xd1,0x55,0x80, +0x9d,0x64,0x6b,0xa8,0x61,0x09,0x8e,0x6c,0x2e,0x60,0xd0,0x88, +0xb0,0x90,0x4f,0x68,0x2e,0x38,0x82,0xa9,0xb8,0x44,0xfd,0xe7, +0x5a,0x39,0xc2,0x88,0x4d,0x21,0x97,0x93,0xab,0x83,0x35,0x53, +0xf0,0x4f,0x68,0xb1,0xf3,0x02,0xa7,0xdb,0x1f,0x2f,0x3a,0xff, +0x8c,0x25,0x72,0xa2,0x7a,0x2a,0x95,0x3c,0x7a,0x26,0xa1,0x79, +0x6d,0x0c,0xa5,0xee,0x90,0x8a,0xb7,0x2d,0x69,0xa1,0x78,0xce, +0xed,0xdd,0xc5,0x74,0x52,0xdd,0x3c,0x05,0x7c,0x6b,0x39,0x1b, +0x4e,0x8f,0xdf,0x3c,0x4f,0xe5,0xc0,0x04,0x9c,0x9f,0x91,0x30, +0xf9,0xce,0xef,0x1d,0xf9,0x58,0xb8,0x50,0x8e,0xaf,0xf4,0x05, +0xfe,0x11,0x94,0xad,0xe0,0x9f,0x9c,0x5f,0x0a,0x0b,0xfe,0x51, +0xba,0x43,0x70,0x16,0x65,0x02,0x28,0x09,0xed,0x31,0xb3,0x90, +0x38,0xca,0x51,0x38,0xb1,0x20,0x3b,0xf0,0x78,0xd3,0x14,0xf8, +0x47,0xde,0x51,0x44,0x7d,0x20,0xd7,0xb7,0x92,0x42,0x03,0xb9, +0xca,0x2f,0x16,0x1a,0xc8,0x8d,0xec,0xa1,0x55,0x52,0xf0,0x3c, +0xdf,0xc2,0x85,0x72,0x7d,0x65,0x71,0x0a,0x0b,0x46,0x21,0xa2, +0xf0,0xc1,0x5c,0x65,0x17,0x9c,0x10,0x43,0xae,0x12,0x7f,0x85, +0x06,0x72,0x27,0xe9,0x55,0xb8,0x50,0x2e,0x98,0x8b,0xca,0xe0, +0xe7,0xbb,0xbb,0x7c,0x62,0x16,0xfc,0x93,0x8b,0x3e,0x43,0x21, +0xbf,0x39,0x3f,0xbd,0x8b,0x5c,0x46,0xee,0x58,0xfb,0x85,0x0b, +0xe5,0x77,0xe8,0x55,0x93,0xe2,0x5c,0xc8,0xd9,0x00,0xaa,0xd0, +0x36,0xcf,0x1d,0x0a,0xc0,0x15,0x07,0xc4,0x3d,0xe0,0x99,0xe4, +0x73,0xb8,0xdc,0x7a,0x33,0xd3,0x03,0x48,0x90,0x17,0x09,0x59, +0x32,0x2e,0x3f,0xb6,0x0f,0x2d,0xdb,0xb2,0xa0,0x28,0x49,0x57, +0x4f,0x1b,0xd8,0x06,0x53,0x89,0xde,0xe5,0x74,0xdd,0x0e,0xd9, +0xa6,0x06,0x00,0x8a,0x7a,0x30,0x83,0xfc,0x6b,0xe9,0x72,0x53, +0x97,0x23,0x88,0x77,0x4f,0xc2,0xcd,0x97,0x75,0xdd,0xde,0x4b, +0xef,0x87,0x05,0xa2,0xd3,0xc7,0xb3,0xcb,0x9b,0xf9,0xf2,0x89, +0x46,0xc3,0x32,0x5c,0xdc,0xe1,0xc1,0x04,0x7f,0x7d,0x24,0x0a, +0xbc,0x01,0x26,0x3c,0x8c,0xe3,0xd1,0x9d,0x11,0x2e,0xe0,0x5b, +0x1f,0x25,0xbe,0xe2,0xf8,0xc7,0x44,0x13,0x5f,0xac,0xe0,0x1f, +0x13,0x8d,0x9a,0x65,0x05,0xfd,0xf5,0x91,0xac,0xab,0x5e,0xf0, +0x8f,0x8f,0x9e,0x5d,0xff,0x80,0x1b,0x80,0x7f,0xe2,0x68,0x7c, +0xe2,0x29,0x7c,0x30,0x4e,0x3e,0x13,0x3d,0x9f,0x22,0xfc,0xb4, +0xd9,0x4e,0x90,0x35,0x56,0xc8,0x6f,0x2d,0x01,0x5d,0x28,0x16, +0x26,0x5c,0xcb,0xf0,0x7a,0x34,0x2a,0x7c,0x30,0x48,0xae,0x16, +0x05,0xfd,0x0d,0xcb,0xcc,0x87,0xbf,0x16,0xf2,0x1b,0x24,0xf8, +0xf3,0xd7,0x7e,0xd8,0x2c,0xcf,0x70,0xd4,0xf9,0xc7,0x46,0xff, +0x08,0x33,0x5c,0xf0,0x8f,0x8d,0x7e,0x7e,0x55,0xe0,0x9f,0x20, +0x8a,0x85,0x12,0x0b,0x17,0x0a,0x13,0x27,0xab,0x82,0x7f,0x6c, +0xf4,0xab,0xf9,0x0c,0x47,0x0e,0xb1,0x94,0x22,0xf8,0xb2,0x99, +0xde,0x4c,0xef,0x96,0xc3,0xe9,0xe4,0x1f,0x98,0xc5,0x85,0x6d, +0x86,0xb7,0x80,0x6e,0x0c,0x11,0x3c,0x07,0x6c,0xd2,0xe9,0xcd, +0xfc,0x63,0xc1,0x3f,0x41,0x34,0xad,0x10,0xf9,0x0d,0x12,0x3e, +0x4e,0x60,0x0b,0x16,0x1a,0xa8,0x27,0xfd,0x7c,0x43,0x53,0xe6, +0x3f,0xea,0x59,0x9e,0x88,0xad,0x9c,0xe8,0xdb,0x66,0x7c,0xcd, +0x86,0x13,0x0a,0x17,0xb2,0x89,0xe4,0x82,0x46,0xc7,0xd2,0x7f, +0xd8,0x2c,0xb0,0xaf,0xc7,0xb8,0x32,0xe9,0xd7,0x26,0xbc,0x80, +0x8b,0xb3,0xe0,0x9f,0x60,0xfe,0xc9,0x2f,0x42,0xa1,0x01,0x9f, +0xa4,0x84,0xa5,0xa2,0x3d,0xa3,0x70,0x5c,0x1f,0x48,0xa4,0x87, +0xa5,0x42,0x7e,0x5d,0xc2,0xc6,0x1e,0x06,0xb3,0xeb,0x63,0xe5, +0xdc,0x1e,0x4f,0x27,0x43,0xdf,0xc9,0x2a,0xcc,0x45,0x9e,0x44, +0x4d,0x62,0x5e,0x3f,0x7e,0xee,0x01,0x21,0xc1,0x23,0xe5,0x94, +0x14,0x57,0x8b,0xdd,0x63,0xf3,0xa5,0xe7,0x50,0xae,0x48,0x0b, +0x26,0x07,0xd8,0x4b,0xbe,0x7b,0x01,0x28,0x34,0x20,0x54,0xc5, +0xee,0x0f,0x1c,0xf0,0x29,0xe4,0xd5,0xf1,0x99,0x60,0x6d,0xbb, +0x27,0xe6,0xcb,0xe4,0xf1,0xdb,0x67,0xf7,0xa4,0x86,0xb1,0xe4, +0xbb,0xca,0x79,0x2d,0x76,0xd5,0x1e,0x8f,0x4f,0x1b,0xab,0xa7, +0x8e,0x1f,0xc9,0x7b,0x05,0x40,0x78,0x1a,0xc5,0xf8,0xbc,0x82, +0x57,0xed,0x3e,0x0b,0x50,0xa9,0x7c,0x97,0xac,0x3d,0x2d,0xe6, +0x74,0xd6,0xec,0x3e,0xf7,0x1f,0x3e,0x07,0xe9,0xf2,0xed,0xa2, +0x23,0x9c,0x30,0xee,0x07,0xb8,0xab,0x10,0x77,0xa0,0x24,0xf9, +0xf0,0x39,0x54,0xba,0xbb,0xd8,0x55,0xa7,0xe8,0x36,0xed,0x1a, +0xa3,0xaf,0x7d,0x0c,0x3d,0x45,0x17,0xbb,0x6f,0xf0,0xc7,0xc7, +0x3a,0x1e,0x6a,0xb1,0xfb,0x56,0x83,0x3e,0xf5,0xb7,0x62,0xf7, +0x3f,0xfd,0x57,0x05,0xe8,0xee,0xee,0xe9,0xe5,0x38,0x88,0x79, +0x02,0xa7,0xf7,0x35,0xf5,0xec,0xd4,0x7f,0x98,0x1c,0xb3,0xc9, +0xd5,0x15,0x0e,0xca,0x29,0x07,0x7c,0x8a,0xaa,0x59,0xd2,0x9c, +0x15,0xbb,0x67,0xf6,0xd3,0xe4,0x62,0xf7,0x25,0x90,0xce,0x01, +0x9f,0xc2,0x77,0x72,0xb1,0xcb,0xd2,0x2f,0x3e,0x7e,0xf7,0x6e, +0x39,0xfd,0x69,0x35,0x01,0x1c,0x6b,0x77,0xf7,0x27,0x09,0x6a, +0xea,0x06,0xd9,0xab,0x1e,0xc7,0x7b,0xff,0xdb,0x2b,0xb8,0xc4, +0x9e,0x8f,0x1c,0x21,0xd8,0xe9,0xbc,0xff,0xed,0xf9,0xc8,0x48, +0x66,0x01,0xfd,0x30,0x3d,0x51,0x66,0xa2,0x97,0x49,0x0f,0x38, +0x85,0x6f,0x9e,0x9e,0x3c,0x3f,0x7e,0xf1,0xee,0xe4,0xc7,0xe3, +0xb7,0xa7,0x35,0x66,0xe1,0x3b,0xcf,0x81,0xcc,0x85,0x71,0xc8, +0x1a,0xd9,0x9e,0x88,0xa7,0xcf,0x9e,0x63,0x18,0x1a,0x6f,0xe0, +0x05,0x47,0x6e,0xfc,0x43,0xcb,0xcb,0xd7,0xff,0xd7,0xbb,0x1f, +0x8f,0x4f,0xfe,0x43,0xeb,0x01,0xd2,0xf1,0x1f,0xbb,0x87,0xf6, +0x8d,0x71,0x1b,0x9f,0x8c,0x48,0xa1,0xd1,0xa4,0x5a,0x60,0xca, +0x19,0x29,0xc1,0xd3,0x32,0x45,0x04,0xac,0xca,0x99,0x11,0xf3, +0xfc,0xea,0xd5,0xfc,0xd8,0x89,0x2f,0xdd,0x1b,0xfd,0x2e,0x04, +0x41,0x30,0x53,0xf2,0xc1,0x2e,0x06,0x75,0xc7,0xab,0x9c,0x14, +0x09,0x50,0x68,0x1b,0x02,0x2f,0x98,0xbf,0xe2,0x78,0xe9,0xe8, +0xd8,0x92,0x13,0x29,0x28,0xc9,0x14,0x5e,0x8e,0x67,0x24,0x20, +0x5f,0x9a,0x46,0x90,0x6f,0xfa,0xe1,0x6d,0xaf,0x6f,0x6d,0x3f, +0x73,0x75,0x83,0x82,0x22,0x07,0xe4,0x45,0x87,0xce,0x9b,0xd2, +0x76,0x05,0xe5,0xf7,0x76,0x1a,0xba,0xa0,0x16,0x5f,0x19,0x08, +0x72,0x8e,0x8e,0xb0,0x52,0x15,0x7f,0xa6,0x18,0xb4,0xd1,0x42, +0x91,0xfc,0x30,0x91,0xb9,0x1e,0x95,0x07,0xbe,0x4f,0x25,0x1b, +0x33,0xa1,0x52,0x2e,0xf2,0xb1,0xe6,0x3c,0xd2,0x40,0xa7,0x43, +0x20,0xa3,0x47,0x21,0x48,0xed,0x6b,0x8e,0x41,0xe6,0xbb,0xd0, +0xd3,0xa7,0x03,0xf1,0xbe,0x2c,0xe7,0x09,0xe0,0x6a,0x28,0xc8, +0xbb,0x9c,0x03,0x95,0x59,0xf8,0xfe,0xee,0xb8,0xa0,0x19,0x58, +0x68,0xa3,0x19,0xda,0x32,0xd5,0xb1,0x2d,0xdd,0x43,0xa0,0x44, +0xa4,0x99,0x1a,0x54,0x3b,0x32,0x05,0x1e,0x7b,0x48,0x47,0x3e, +0x08,0xbd,0xa0,0x01,0x21,0x9a,0x4a,0xec,0x5d,0x28,0x9c,0xbe, +0xcf,0x66,0x94,0x78,0xd5,0x39,0xe0,0xbf,0xff,0xb6,0xcd,0x90, +0x38,0x3d,0x51,0xdb,0x6c,0x35,0x0b,0xe7,0x41,0x62,0x00,0x70, +0x57,0xbf,0xd6,0x6b,0x1b,0x9f,0x9b,0x55,0xe9,0xf3,0x04,0x45, +0xf3,0xd8,0x94,0xa8,0x29,0x12,0xef,0x98,0xe0,0x99,0x55,0xc2, +0x56,0x30,0x91,0x73,0x65,0xa1,0xf8,0x13,0x2d,0x3c,0xea,0x6b, +0x43,0x56,0x11,0x1a,0x16,0xe9,0x34,0x05,0xdf,0x6e,0x27,0xdf, +0x27,0x3b,0x66,0x82,0x44,0xb2,0x55,0xf8,0x44,0xbc,0x66,0x94, +0xd3,0x3c,0x9b,0x03,0x72,0x0c,0xd4,0xf1,0x8b,0xf9,0x9c,0xf4, +0x13,0xee,0x16,0x5a,0x73,0xd5,0xfa,0x30,0x19,0xb6,0x2a,0x79, +0x59,0xae,0x6b,0x25,0xb4,0x2e,0x3e,0x0b,0xac,0x9d,0xd6,0xe9, +0x78,0x5c,0xb4,0x44,0x5d,0x1c,0xee,0xba,0x58,0x59,0x7c,0xb8, +0x98,0xec,0x47,0x5c,0xf2,0x24,0xd0,0xcf,0x8e,0x06,0x68,0xd3, +0xd4,0x2d,0x79,0x28,0x9d,0x7c,0xd8,0xea,0x82,0x18,0xd2,0x00, +0x2c,0xfc,0xed,0x92,0x55,0x92,0x1f,0xcf,0x5e,0xbe,0x28,0x59, +0xe0,0xac,0xfd,0x2f,0x87,0x7f,0x3a,0x38,0xfa,0x7e,0x9f,0x84, +0xcf,0xf4,0x0d,0x3b,0xc7,0x9c,0xd6,0x03,0x2f,0x7e,0x1b,0x87, +0xb9,0xf2,0x20,0x70,0x3c,0x1a,0xa1,0x59,0x7e,0xb1,0xeb,0x81, +0x79,0x68,0x85,0x52,0x9c,0xf8,0x7e,0x5a,0x0e,0xaf,0xed,0x76, +0x8c,0xda,0xa7,0x57,0xfe,0x33,0xc9,0x96,0xa2,0xe1,0x5e,0x09, +0xab,0x1c,0x1c,0xfb,0x0e,0x21,0xa9,0xbd,0xa6,0x5a,0xdd,0xc0, +0x44,0x4b,0xea,0x84,0xa4,0xf2,0xdc,0xba,0x0a,0x97,0x0e,0xcb, +0xec,0x21,0x14,0x7a,0xf5,0x8b,0x8a,0x3e,0x19,0x0f,0xe1,0x76, +0xf7,0x65,0x39,0x96,0x8f,0x72,0x7c,0x65,0xf1,0xef,0xc2,0x46, +0xbf,0x23,0x6f,0xde,0xfe,0xd4,0x52,0x7c,0xde,0x3d,0x9a,0x7c, +0xaf,0x39,0xac,0x06,0x48,0x50,0xa3,0xdb,0xe8,0x93,0x41,0xad, +0x51,0xaf,0xaf,0xdc,0xce,0x60,0x03,0x1f,0x57,0xb3,0xfc,0x6e, +0xe6,0x96,0x9c,0x0a,0xc9,0x8b,0x58,0x98,0x4d,0x69,0x5c,0xdc, +0x70,0xf1,0xa1,0xe0,0x31,0x2c,0x6f,0x8e,0xff,0x17,0x88,0x48, +0xb3,0xd6,0x68,0x3e,0x0e,0xd6,0x32,0xb1,0x1f,0xff,0x97,0xae, +0xf4,0xff,0xe5,0x58,0x90,0xf2,0x7e,0x3c,0xfe,0x40,0x32,0x2e, +0xdc,0xc2,0xa7,0xec,0x76,0xf8,0x74,0x85,0x9e,0x96,0xb5,0xa9, +0x09,0x67,0x49,0x32,0x71,0xb0,0xf7,0x60,0x5e,0xce,0x02,0xb0, +0x9d,0x33,0x3e,0xab,0xa1,0x49,0xb6,0x76,0x9c,0x14,0x18,0xc3, +0x35,0x82,0x1a,0xf8,0x2d,0xa7,0x78,0xae,0xc6,0xbb,0x71,0xc6, +0xe8,0xd8,0x46,0xa4,0x1e,0x3d,0x15,0x3d,0x9b,0x85,0x83,0x68, +0x4b,0xa1,0x61,0x71,0xd2,0x91,0x61,0xd0,0x64,0x7c,0x7b,0xb0, +0xf1,0x22,0x15,0xfc,0xb4,0xe7,0xdc,0x2d,0xf9,0xaa,0xb9,0x36, +0xdb,0xd4,0xab,0x59,0x86,0xae,0x49,0xbe,0xae,0x7a,0xa9,0xa9, +0xb1,0xfa,0xac,0xb0,0xba,0x5e,0x36,0x85,0x44,0x06,0xa0,0x9e, +0x4d,0x56,0x5b,0x2a,0xf5,0x95,0x6a,0x5c,0x5c,0x88,0x73,0xe8, +0xe7,0x23,0x27,0x0a,0x01,0x88,0x18,0x9b,0xb2,0x31,0x73,0x02, +0xfb,0x95,0x70,0xc1,0xbe,0xcb,0x3e,0xc0,0xe3,0xd6,0x66,0x61, +0x83,0x10,0x08,0xd7,0x09,0x67,0x52,0xf3,0xeb,0x25,0xe9,0x79, +0x9e,0x2f,0x01,0x79,0x33,0x3e,0xb2,0x80,0xba,0x6e,0xb6,0x83, +0x58,0xee,0x6c,0x57,0xef,0xe3,0x76,0xbb,0xa1,0x0c,0xda,0x7b, +0x36,0x37,0x76,0x5e,0xdb,0x2a,0x99,0x1b,0xd5,0x7a,0xb3,0xf2, +0x68,0x00,0x8c,0x89,0xdb,0x78,0x44,0x1b,0x97,0x6b,0x24,0x39, +0xfe,0xcf,0x0e,0xec,0x7a,0xbd,0x77,0xe8,0x44,0x22,0x63,0xad, +0xe2,0x5e,0x30,0x26,0xeb,0x75,0x1a,0xb7,0xd9,0xd7,0xea,0xf1, +0xe9,0x2f,0xcd,0x64,0x79,0xbf,0x09,0xf3,0x18,0x45,0x71,0x7d, +0xd4,0x2f,0x6c,0x7a,0x38,0xf6,0xac,0x37,0x16,0x9f,0x99,0x76, +0xb5,0x45,0x23,0x43,0x16,0xf0,0x1f,0xda,0xf7,0xec,0x5b,0x1b, +0xd9,0x9c,0x84,0x46,0x96,0xf1,0x28,0x20,0x40,0x89,0x29,0x77, +0x34,0x17,0x3e,0xfc,0x59,0xf1,0x58,0x2c,0x3e,0x41,0xa3,0xd3, +0x7f,0x65,0x20,0xbe,0x0c,0xa9,0xa8,0x88,0x56,0x17,0x99,0x79, +0x40,0x8d,0x41,0x72,0x46,0x1a,0x16,0x59,0xaf,0xbf,0xd8,0x48, +0xb6,0xe6,0x8f,0xe3,0xa7,0xcd,0x60,0xdf,0x0a,0x7e,0x00,0xd9, +0x8b,0xa7,0x68,0xd7,0x48,0x03,0xdc,0x0e,0x81,0xac,0xa2,0x4e, +0x64,0xaa,0xb5,0xa9,0xe2,0x9d,0x01,0xc1,0x1d,0xa9,0xcd,0x00, +0x88,0x20,0x79,0xd8,0xda,0xaa,0xfc,0x71,0x58,0x11,0x7b,0xcb, +0xb5,0x51,0xcf,0x69,0x27,0x1a,0xb1,0x55,0x2c,0xcd,0x48,0x8f, +0xf5,0xcf,0x67,0xe7,0xab,0x01,0x0a,0x90,0xb5,0xac,0xab,0x74, +0x28,0xa9,0xe0,0xa8,0xc0,0x63,0xab,0x80,0x6a,0xcf,0x99,0xb0, +0x09,0x97,0x15,0xb7,0x09,0x65,0x4d,0x7c,0xd8,0x9b,0x2b,0xf0, +0x71,0x8d,0x47,0xa9,0x26,0x7b,0xd1,0x1e,0x2f,0x63,0x07,0x38, +0xcf,0x6d,0xfa,0x4f,0x74,0xc9,0x09,0x2c,0x42,0x49,0x82,0xe1, +0x2a,0x21,0x01,0x3d,0xcc,0x13,0x48,0x87,0x38,0x2c,0xe3,0xff, +0xa1,0x9e,0x6d,0x99,0x3c,0x97,0xc1,0xef,0xf2,0xb8,0xf3,0x8d, +0xfd,0x8e,0xfa,0xb4,0xa5,0x2f,0x8c,0x31,0x21,0xc3,0xc1,0x89, +0xa6,0x5a,0x64,0xdb,0x0f,0x78,0x55,0x3a,0x39,0x57,0xaf,0xf1, +0xbd,0xe3,0xb7,0x9a,0x4b,0x15,0x3d,0x0d,0xa3,0xf4,0xe6,0x60, +0xf5,0x54,0xe8,0xb5,0x62,0xdb,0x7f,0x18,0x8a,0x74,0x61,0x23, +0x30,0x84,0x0b,0x61,0xeb,0x02,0xe1,0x5a,0x76,0x2e,0x5c,0xc3, +0xec,0x1c,0x17,0x29,0xba,0xde,0x64,0x1f,0x71,0xae,0xe7,0x56, +0x2e,0xc6,0xef,0xe2,0xdd,0xa4,0x93,0x8a,0x84,0x84,0xe5,0xe6, +0xa2,0x35,0x25,0xfb,0x15,0xd7,0xd9,0x0c,0x8d,0xc8,0x7a,0x39, +0xe8,0x1a,0x68,0x4f,0x27,0x90,0xf8,0x97,0xb2,0x6c,0x50,0xd9, +0x10,0xb3,0xc5,0x46,0xc0,0x92,0x74,0xe9,0x45,0xa8,0x26,0x43, +0x4b,0x3a,0x1c,0xab,0x14,0x33,0xce,0x97,0xc8,0x98,0x07,0x82, +0x6f,0x2c,0xa2,0x6b,0x95,0xf2,0x32,0xab,0xd6,0x72,0x14,0xd7, +0xb2,0x10,0x53,0xab,0x9b,0x40,0x16,0x44,0x54,0x73,0x90,0xed, +0x48,0x26,0x38,0xeb,0xa8,0xc3,0x85,0xcf,0x51,0xfe,0xf0,0xfa, +0xf5,0x8b,0xa7,0xc7,0xaf,0xde,0x1d,0x9f,0x9d,0xbd,0xa5,0xab, +0x3d,0x94,0xb6,0x72,0x17,0x9a,0x29,0xd3,0x6e,0x6b,0xa1,0xa7, +0x2f,0x9e,0xa2,0xe2,0xd3,0xa9,0x2e,0x13,0xb7,0xd2,0x06,0x24, +0x3f,0xaa,0x05,0x0c,0x2b,0x88,0xa9,0x19,0x83,0xac,0xb9,0xe6, +0x31,0x9a,0x20,0x57,0xb0,0xc9,0x50,0x86,0x98,0xa1,0xa0,0x84, +0xb8,0xe2,0x3f,0xd0,0x18,0x2c,0xe9,0x57,0xf8,0xcb,0xb8,0xe5, +0x1a,0xa2,0xad,0xbc,0x29,0xa7,0x73,0xaf,0xfe,0x26,0xda,0x43, +0x78,0x8d,0x52,0x34,0x1a,0x8f,0x47,0xb1,0x8a,0xe1,0x35,0x71, +0x02,0x1d,0xb8,0x28,0xbe,0x0e,0xef,0x12,0xa9,0xd6,0xe9,0x0f, +0x77,0x17,0x68,0xad,0x71,0xe7,0xc0,0x01,0x54,0x17,0xb0,0xa9, +0xfd,0x2c,0x05,0xe8,0xf2,0x52,0xe8,0x3a,0x6f,0xa5,0x34,0xb3, +0xef,0x89,0x52,0xc8,0xfb,0xb7,0xc5,0x4f,0x24,0x04,0x58,0x0b, +0x83,0xbf,0xcb,0xa6,0x0e,0x1f,0x7d,0xdd,0x28,0xc4,0x90,0x51, +0x8c,0x5c,0xca,0x88,0x25,0xe3,0x0f,0x44,0x92,0xe5,0xdb,0xb2, +0x1f,0x6e,0x38,0x85,0x8e,0x95,0x30,0xad,0x2e,0x1f,0xd2,0x0c, +0x64,0xbd,0x3e,0x28,0xcb,0xda,0xa4,0x6c,0xf2,0x90,0x3e,0xe8, +0xb3,0xa3,0x1e,0x19,0x45,0xc4,0x9b,0x43,0xfb,0xf7,0xaa,0x24, +0x18,0xac,0x26,0x14,0x75,0xfc,0xf3,0xe3,0xf2,0xb6,0x9a,0x8c, +0x7b,0xcd,0xeb,0x02,0x39,0x5b,0xcd,0x53,0x6f,0x93,0x1a,0x7a, +0x47,0xc2,0x35,0x45,0x6a,0x31,0xfd,0x08,0x76,0x40,0x05,0xc4, +0xd0,0xc3,0xc4,0x3a,0x7c,0xef,0x2d,0xcc,0x6e,0x05,0xef,0x9f, +0xd4,0x75,0x51,0x52,0xfc,0x0e,0xbb,0xf1,0xf6,0x8d,0x9c,0x82, +0x14,0xca,0x4d,0x9d,0xa9,0xb7,0x23,0xd6,0x26,0x54,0x7b,0xd0, +0x4e,0x10,0x4d,0xb2,0xa8,0xb3,0x46,0xfc,0xee,0xd5,0xcd,0xb1, +0x90,0xf6,0x6e,0x60,0x1a,0x29,0xeb,0xd5,0xa2,0xd2,0xac,0xc0, +0xb8,0x40,0xb7,0xb8,0x5e,0xb0,0x74,0xa6,0xa9,0x38,0x3b,0xca, +0x67,0x49,0x23,0xd0,0x3e,0x63,0xe7,0x57,0x6b,0x76,0x09,0xcd, +0x13,0xbc,0x44,0x51,0x74,0x24,0xa2,0xbc,0xf0,0x2b,0x1b,0xc5, +0x60,0x2f,0xa2,0x77,0xec,0x95,0x33,0x50,0x14,0x21,0x99,0xbe, +0x71,0xaa,0x2a,0xb7,0x22,0x32,0x03,0xd4,0x16,0xb9,0x03,0x87, +0xbd,0xf8,0x64,0x7c,0x39,0x25,0x59,0xb7,0xa3,0xad,0x06,0x68, +0xae,0x66,0xbd,0x54,0x0a,0xa2,0x5f,0x05,0x09,0xe2,0xe5,0xae, +0xb1,0x44,0xbf,0x79,0xc3,0xac,0x0c,0x1d,0xf3,0x36,0x58,0x36, +0x3c,0x3d,0x7b,0xfb,0xfc,0xcd,0xbb,0x93,0xd7,0x2f,0xe9,0xdc, +0x64,0xc1,0x5f,0x6e,0x46,0xc9,0xe5,0x44,0xad,0xef,0x19,0x9c, +0xc8,0x6f,0xff,0x7a,0xea,0x35,0x1a,0x24,0x17,0x6a,0xf8,0x31, +0x56,0xc2,0x39,0xde,0x9d,0xbe,0x79,0xf1,0xfc,0x2c,0xb3,0x5a, +0x54,0xd7,0xd9,0x3d,0x8a,0x12,0x6a,0x8d,0x9c,0xcf,0x64,0x80, +0x65,0x8d,0x13,0xb3,0xac,0xd0,0x83,0x81,0xdc,0x0a,0xd2,0x15, +0xbe,0xb4,0x67,0xa2,0xfa,0x07,0x34,0x97,0xef,0x70,0xa9,0x1d, +0x77,0x06,0xfe,0x88,0x36,0xc6,0xb1,0x2b,0x5d,0xef,0xf7,0x0e, +0x73,0x2b,0x86,0x79,0x35,0xeb,0x63,0xfa,0x20,0x4f,0xae,0x66, +0xd0,0x51,0x33,0x8a,0x6c,0x51,0xf2,0x80,0x86,0x1e,0xe6,0x3f, +0x2c,0x44,0xd9,0xc9,0x2e,0xb6,0x94,0x88,0xef,0x8f,0x58,0xbc, +0xfd,0x6c,0x8e,0x12,0x06,0x86,0xd5,0x2e,0x4c,0x0f,0x21,0x15, +0x46,0xf2,0x9e,0x52,0x17,0x5d,0x33,0xf4,0x8d,0xa3,0xdb,0x0c, +0x7d,0xd1,0x73,0x5a,0x2f,0x98,0x31,0xb4,0xcf,0xe6,0x80,0x66, +0x8e,0xd2,0xd2,0x28,0x03,0xf6,0x7f,0x58,0x2d,0x46,0x7d,0xc7, +0x0b,0xe4,0xa9,0xde,0x31,0x36,0x10,0x3a,0x8e,0x96,0x14,0x6f, +0x79,0xc0,0xc7,0x65,0x6c,0xb4,0x9e,0xbf,0x4a,0x0d,0x3d,0xf7, +0x72,0xfd,0x24,0xde,0x34,0x11,0xf7,0x4d,0x5a,0x28,0xdf,0x71, +0x1f,0xdd,0x5d,0x74,0xbc,0xcd,0xac,0x24,0x27,0x43,0xa6,0xdc, +0xa4,0xc5,0x35,0x9a,0xbb,0x4e,0xf4,0xa5,0x47,0x84,0xe9,0x6e, +0xef,0xaa,0x95,0x98,0x89,0x6d,0x61,0xe9,0x96,0x08,0x70,0xb5, +0x58,0x8a,0xcf,0x89,0xb1,0xa9,0xbd,0x31,0x29,0xcd,0xe4,0x2a, +0xe9,0xe9,0x68,0xd4,0xe9,0xdd,0xd5,0xd5,0xe4,0xd3,0xc0,0xb5, +0xfb,0x9d,0xd1,0x4b,0x16,0x81,0x4b,0x2b,0xa3,0x69,0x84,0x20, +0xc3,0xd1,0xbb,0xdf,0x25,0x4f,0xb5,0x9a,0x6b,0x13,0xd8,0x78, +0x23,0x91,0xb1,0x54,0xcd,0x35,0xab,0x3c,0xad,0x9f,0x7d,0x5b, +0x51,0xdf,0xca,0xd1,0x79,0x21,0x10,0xa7,0x76,0xe0,0x8c,0x94, +0x4d,0x9a,0x46,0xd8,0x82,0x0f,0xde,0xc4,0x58,0x95,0xc3,0xa2, +0x95,0x4d,0x95,0x87,0xe6,0xce,0xb2,0xc0,0x96,0x01,0x55,0x12, +0x40,0xa8,0x0f,0xab,0x92,0xfa,0xca,0x35,0xaf,0xa5,0x78,0x88, +0xa8,0x70,0x4b,0xef,0x67,0xa9,0x8c,0xcf,0x2b,0x7a,0xc9,0xc2, +0xd8,0x67,0xea,0xa2,0x6b,0x39,0x71,0x0f,0x9d,0xf5,0x85,0x85, +0xde,0x9f,0x4d,0xd1,0x68,0x46,0xb3,0x1c,0x4b,0xe3,0xa4,0x94, +0x16,0x0c,0xad,0xb5,0xa3,0x5a,0x4c,0xec,0x43,0x09,0x33,0x38, +0x99,0x45,0xed,0xcc,0xf3,0x48,0x55,0x45,0x2b,0xc8,0x2d,0x38, +0xa3,0xd8,0xd2,0x5c,0x4a,0x7a,0x48,0x06,0x92,0x61,0xd5,0xe8, +0x16,0x2d,0x6c,0x95,0x1b,0x8b,0x40,0xa3,0xcc,0xa7,0xa8,0x53, +0xc4,0xc7,0x0b,0xd9,0xbd,0x32,0x62,0x99,0xb1,0xfa,0x6b,0x90, +0xbd,0xae,0x8b,0x81,0xac,0x04,0x84,0x3e,0x56,0xf8,0x34,0xa6, +0x92,0x9a,0xdd,0x87,0x49,0x70,0x9d,0x49,0x12,0x1e,0x84,0x24, +0x32,0x65,0x1f,0x0c,0x5c,0x29,0x2f,0x59,0xfc,0x6c,0x56,0x86, +0x02,0x4f,0x92,0xe7,0xc8,0x37,0xd9,0x4b,0xfa,0x89,0x29,0x87, +0x7a,0x67,0x9f,0xcd,0xba,0x4e,0xe6,0x36,0xd3,0x6c,0x2e,0xc9, +0x4b,0x8d,0x1a,0x96,0xb9,0x91,0x5b,0xf5,0x19,0xad,0xac,0x29, +0x32,0xd5,0x27,0x93,0xd2,0x44,0x39,0x62,0x71,0x22,0xf2,0xe3, +0xf7,0x1e,0x12,0x99,0xc4,0x08,0x44,0x89,0x07,0xee,0xb0,0x6c, +0x5e,0x96,0xbe,0x1c,0x2a,0x5b,0x1f,0x69,0x9e,0xbe,0x89,0x3f, +0x1c,0x0c,0xe4,0x49,0xcd,0x49,0x20,0x9b,0xd4,0x47,0x03,0xb5, +0x74,0x64,0x8e,0x61,0x19,0xbe,0x5e,0x24,0x1e,0xd9,0x70,0xe2, +0xd2,0x42,0xd3,0x19,0x71,0xd7,0xe3,0x3f,0x5b,0xde,0xde,0x85, +0xb2,0x02,0x54,0xa4,0xdc,0x08,0xc7,0xf1,0xf9,0x1d,0xd5,0xe5, +0x44,0xc0,0x65,0x22,0x44,0x4d,0xcf,0x6a,0x2d,0x8c,0xbb,0x62, +0x21,0xb8,0xdd,0x1e,0xa3,0x23,0xae,0xcb,0x5f,0xdb,0xed,0xbd, +0x43,0x40,0xc0,0xf9,0xc3,0xb1,0x8d,0x5c,0x3e,0x04,0x0a,0x64, +0x6a,0xd7,0xd8,0x4c,0x4e,0x3a,0x92,0xdb,0x5d,0xd0,0xfe,0x02, +0xe1,0x1a,0x97,0x28,0x8c,0xfa,0x6c,0x38,0x21,0x09,0xeb,0x79, +0xcb,0x1c,0x99,0x2a,0x39,0x0d,0xf7,0x4a,0x6b,0x74,0x37,0x86, +0xd4,0xe2,0x7c,0x46,0x1a,0x1f,0xaa,0x6f,0xcb,0xa0,0x01,0xd5, +0xd7,0x2a,0x21,0x88,0xde,0x2d,0x32,0x2f,0x12,0x5d,0xc7,0x07, +0x58,0xbc,0xd4,0xe1,0x05,0x24,0xcf,0xe2,0x05,0xfd,0x2d,0x75, +0x7c,0x2a,0xd7,0x83,0x39,0xcb,0x68,0x6b,0x52,0x91,0x58,0xb0, +0xde,0x66,0xf2,0xb9,0xfa,0x26,0x1a,0xb5,0x0f,0x9f,0xbf,0x3a, +0x3d,0x3b,0x7e,0x75,0xf6,0xfc,0xf8,0xec,0xf9,0xab,0xbf,0x6e, +0xbb,0x59,0x2f,0x47,0x63,0x54,0x0b,0x3e,0x99,0x2c,0xd1,0x61, +0xcf,0xd2,0x8b,0x87,0x7f,0x86,0xf3,0x03,0xf0,0xb1,0x02,0x87, +0x24,0x21,0x05,0x0e,0x51,0x1e,0x6f,0x7d,0xbf,0x87,0x1c,0x2d, +0x27,0x82,0x5b,0xab,0x7a,0x63,0x0c,0x2a,0x50,0x39,0xd5,0x0b, +0xb4,0xcd,0xce,0x1b,0x9a,0x1c,0x34,0xb8,0x29,0x83,0xde,0x52, +0x16,0xd0,0xe6,0x0a,0x1f,0x84,0xa7,0x9f,0xef,0x59,0x55,0x84, +0xdf,0xf9,0xad,0x0d,0x4e,0x5e,0xc7,0x80,0xbe,0x91,0x01,0x17, +0x92,0xca,0x14,0x72,0x5e,0x55,0x46,0x88,0x9f,0x8c,0xaf,0x55, +0x88,0x32,0x2b,0x36,0x68,0xb1,0x74,0x76,0xf6,0x59,0x1e,0xa8, +0x1f,0x54,0x45,0x4b,0xe5,0xb8,0x50,0xf7,0x9f,0x7c,0x64,0xb0, +0xa1,0x06,0xcd,0x83,0xc7,0x44,0xcd,0xc9,0x30,0x22,0x71,0x5b, +0xe6,0x63,0xb2,0xfa,0x15,0x90,0xcc,0xe4,0xf9,0x0c,0xee,0x07, +0x14,0xba,0x12,0xd5,0x48,0x36,0x79,0xf0,0xeb,0x78,0xb6,0xd3, +0x7a,0xfa,0x69,0x01,0x11,0xb0,0x82,0x65,0x14,0x58,0x41,0x60, +0x58,0xb5,0xb8,0x16,0xd6,0xe8,0xa1,0x49,0x23,0x5e,0x34,0xf6, +0x8b,0xb7,0x38,0x77,0xbd,0xdd,0xe6,0xdf,0x26,0xab,0x4c,0x3d, +0x4e,0x22,0xf6,0x70,0x61,0x16,0x25,0x59,0x87,0xd8,0x54,0x24, +0x31,0x97,0x7a,0xdc,0x1b,0x68,0x8d,0xab,0x59,0x89,0xd8,0x34, +0x0d,0xc0,0x20,0xa3,0x21,0xee,0xed,0x1d,0x16,0x43,0x6f,0xd9, +0x27,0xbb,0x47,0xe1,0xdc,0xd6,0x41,0xa1,0xb7,0x12,0xdc,0xae, +0x47,0x14,0x75,0x68,0xa2,0x86,0x7a,0x4a,0x52,0xca,0xa3,0x7a, +0x0a,0xcd,0x0f,0xda,0x97,0xe0,0x1c,0xdf,0x6e,0xcf,0xc1,0xbf, +0x8f,0x34,0xe7,0x77,0x5f,0xce,0xc9,0xbf,0xdf,0x6a,0x89,0x3f, +0x7e,0x7d,0x09,0xfe,0xfd,0x4e,0x4b,0xfe,0xe9,0xf7,0x97,0xe4, +0xdf,0x3f,0x2a,0x84,0x7f,0xfb,0xe7,0x21,0xf0,0xef,0x9f,0x14, +0xd2,0x9f,0xff,0xf7,0x21,0xf1,0xef,0xbf,0x29,0xc4,0xbf,0xfc, +0xf7,0x41,0xe4,0xdf,0x3f,0x2b,0xe4,0xc3,0x83,0xff,0x7e,0xd0, +0xfc,0xfb,0x97,0x01,0xba,0xd1,0x65,0x99,0x4f,0x57,0x45,0x64, +0xd0,0xa9,0x0a,0x8f,0x0b,0x8f,0x46,0x23,0xcd,0x1f,0x1c,0x18, +0x8a,0xcd,0xe5,0x0c,0x6a,0x3c,0x62,0x0b,0x1e,0x27,0x1e,0xdd, +0xb6,0x28,0xa4,0xe3,0x95,0x98,0x74,0xef,0xc4,0xb3,0x74,0x26, +0x0a,0xcf,0xe8,0xdd,0x19,0xff,0xf6,0xf1,0x8f,0xbf,0x16,0x0b, +0x4a,0xf1,0x45,0x1c,0x42,0x4d,0xe6,0x05,0x0d,0xd4,0xb0,0x3d, +0x82,0xa5,0x70,0xf3,0x5d,0x93,0xa5,0x1f,0x4e,0x86,0x3f,0x0d, +0xca,0x00,0x3e,0x60,0x3f,0x0b,0x2d,0x26,0x4a,0x66,0xf7,0x0c, +0x52,0x14,0x3d,0x72,0x33,0x48,0x85,0x09,0xa3,0xa0,0x99,0x39, +0x36,0x72,0x3d,0x42,0x0b,0x0d,0xe4,0x70,0xe6,0x14,0x91,0xb5, +0x88,0x26,0xe2,0xac,0x49,0x91,0x2c,0xc6,0xec,0xd7,0xeb,0xc6, +0xab,0x91,0x99,0x04,0x1b,0x12,0x49,0x0a,0xee,0x93,0x92,0x3d, +0xeb,0x1b,0x10,0x65,0xe2,0x55,0x54,0xf0,0xf2,0xc0,0xd3,0x3f, +0xc0,0x76,0x69,0x98,0x85,0xc5,0x93,0x07,0x0d,0x2c,0x9d,0x14, +0x7e,0xe1,0xe8,0x9f,0x22,0xa4,0xee,0x35,0xda,0xab,0x1d,0x85, +0xe9,0x8a,0x05,0x38,0x5d,0xa3,0x30,0x59,0x62,0x55,0xc1,0x28, +0x4c,0x94,0x57,0x4b,0xa7,0x55,0x14,0xa6,0x6a,0x74,0x96,0x3b, +0xea,0xaa,0x70,0xa1,0x8d,0x1f,0x05,0x45,0x4b,0xca,0x70,0xec, +0x77,0x6b,0x4a,0xfb,0x11,0x16,0x13,0x64,0xb7,0x4a,0x7b,0x5b, +0xae,0xb4,0xbb,0xd9,0x02,0xf1,0xae,0x9f,0x66,0xbf,0xce,0xe6, +0x1f,0xfd,0x4c,0x6f,0xc5,0x2c,0xf0,0xf1,0xd3,0x52,0x8e,0x38, +0x73,0x31,0x1d,0x55,0x06,0x39,0xbe,0xdc,0xe6,0x20,0xbb,0xb1, +0x0f,0xc0,0xa3,0xec,0x9f,0x20,0x1e,0xc6,0xe8,0x4d,0xf6,0x78, +0x39,0x7e,0x89,0xe0,0x5b,0x58,0x42,0xd3,0x4d,0x01,0xba,0x1a, +0x8d,0xa8,0xb5,0xed,0x84,0x5e,0xc8,0x59,0xde,0x56,0x11,0x7a, +0xa3,0x46,0xe5,0x21,0x3b,0x8c,0x9a,0xc7,0x1f,0x73,0x8d,0x42, +0xda,0x42,0xf6,0xa2,0xd7,0x47,0x4e,0x00,0x74,0xe2,0x29,0x1b, +0x2f,0x28,0x77,0x0e,0x8e,0x88,0x9f,0x39,0x9a,0x54,0x18,0x71, +0x6c,0xf3,0xd8,0x43,0xaf,0xb9,0xf0,0xe1,0x86,0xb9,0xa1,0x44, +0x65,0xf7,0x13,0x11,0xef,0x45,0x9b,0x0d,0x2a,0xe6,0xac,0x56, +0x1e,0xc4,0xe4,0x83,0x67,0x79,0x88,0x42,0xab,0xcb,0x68,0xa4, +0x9a,0x43,0x0c,0xfa,0x19,0x8a,0x8f,0x71,0xb7,0x52,0xb6,0xe9, +0xec,0x0d,0x42,0x13,0x5f,0xbc,0x36,0xd2,0x90,0xa9,0xc1,0x10, +0x0b,0x97,0x58,0xaf,0x93,0x21,0x1a,0xa8,0x0c,0xed,0x8a,0xd9, +0xb7,0x24,0x64,0xb5,0x0a,0x74,0x27,0x23,0xa6,0xf6,0xa0,0x0d, +0xdb,0x87,0xc6,0x42,0x46,0x76,0x3c,0xbd,0xc5,0x23,0x10,0x10, +0x48,0xed,0x0d,0x1e,0x5f,0x37,0x29,0x0a,0x24,0x55,0x37,0x3d, +0xa8,0xe2,0xb6,0xdc,0x66,0x27,0x07,0x73,0x64,0xe8,0x9d,0x4a, +0x1d,0xa5,0xc2,0x1a,0x9f,0xff,0x6d,0x32,0xfe,0x88,0xb6,0x1d, +0xb0,0x60,0x34,0x04,0x0d,0x70,0xaa,0x1f,0x3e,0xd3,0xcb,0x1b, +0x83,0xda,0x02,0x8b,0x9c,0x8d,0x02,0xf5,0x80,0x99,0xda,0x6d, +0x19,0x7e,0xef,0x4b,0xed,0x20,0x3f,0xc8,0x8a,0xc6,0xd8,0x8d, +0x78,0xd7,0x62,0x53,0x84,0x9a,0x45,0x23,0x9c,0x6e,0x7c,0xc3, +0xfa,0x80,0x6a,0xdc,0x9c,0x8a,0xe9,0xf1,0xb4,0xfe,0x4c,0x13, +0x8f,0xd9,0xc6,0x1e,0x3d,0x16,0x80,0xb3,0x65,0x9e,0x72,0xfb, +0xc8,0x35,0x01,0x85,0x36,0x46,0xac,0x44,0x94,0x0d,0x6a,0xfe, +0x9f,0x50,0xb8,0xde,0x49,0xb6,0x9b,0x15,0xa6,0x0e,0xc7,0x5f, +0xdf,0xad,0x70,0x53,0xa1,0x0b,0x36,0xd4,0xc1,0x40,0xab,0x1b, +0xb8,0x1d,0x91,0xe6,0x71,0x98,0x05,0x71,0x75,0x9a,0x6c,0x65, +0x1e,0x66,0x9e,0x5e,0x41,0x5b,0xb7,0x35,0x58,0x27,0xa8,0xb8, +0xb6,0xb7,0x97,0xa3,0x33,0x9a,0x2d,0xa9,0x2c,0xea,0xdc,0x90, +0xa8,0xea,0x6f,0xee,0x3d,0x17,0xdb,0xf4,0x50,0xbe,0x05,0x3a, +0x41,0x48,0x0d,0xf5,0x8e,0x7d,0xef,0x92,0xcf,0xa1,0x94,0x2e, +0xd1,0xd0,0x5e,0xe1,0x1b,0x7e,0x13,0x57,0xf7,0xd6,0x28,0x40, +0x2d,0x72,0xfb,0xd9,0xfd,0x8e,0x1f,0xa7,0x9b,0xf1,0xe5,0xaf, +0xa9,0x7f,0x0b,0x59,0x40,0xa9,0x67,0x33,0x23,0x48,0xc7,0x11, +0xd9,0x3d,0xff,0xa6,0x64,0xa8,0x0c,0x57,0x11,0x83,0x2a,0x3d, +0xd4,0x94,0x40,0xe5,0xce,0x9d,0xf6,0x26,0xf0,0x96,0x32,0x59, +0x8e,0x7f,0x5a,0x4e,0x59,0x67,0x06,0xaa,0x43,0x8e,0xbd,0x4c, +0x29,0x44,0xef,0x94,0x88,0xdc,0x75,0xef,0x96,0x53,0x74,0x13, +0x9e,0x86,0x89,0x26,0xcd,0x3d,0x63,0xdc,0x29,0x28,0x95,0xa4, +0x33,0x0d,0x9e,0x4a,0x14,0x54,0x22,0xa1,0xd4,0x43,0x20,0xf9, +0x38,0x96,0x97,0x9f,0x5e,0x91,0x87,0xca,0x7c,0x39,0xfc,0xa8, +0x72,0xa7,0x6e,0x37,0x23,0x42,0xab,0xeb,0x57,0x4d,0x81,0xb8, +0x13,0x0d,0x0a,0x21,0x2a,0xa0,0xf1,0xf2,0x69,0xc6,0x57,0x53, +0x7c,0x4c,0x4e,0xee,0xc3,0xa3,0x64,0x1b,0x97,0x23,0x19,0x0f, +0x93,0xfe,0x64,0x7c,0x85,0xfe,0xc4,0x49,0xbf,0x9f,0x5a,0x3d, +0xa9,0x5e,0xce,0x2f,0x7f,0x2d,0xc5,0x95,0xd2,0x96,0x35,0x06, +0xa4,0xee,0x03,0x0b,0x07,0x79,0x8c,0x04,0x6b,0x77,0x77,0xfb, +0xae,0x28,0xb7,0x27,0xe5,0x52,0x78,0x32,0xbb,0xac,0x27,0x72, +0xfd,0x66,0x6b,0x6f,0x69,0x62,0xa7,0xb3,0x61,0x38,0x80,0x5a, +0x4e,0xae,0x3e,0xa3,0xba,0xd7,0xab,0x79,0x1d,0x5c,0xe5,0x41, +0xa9,0x7e,0xe8,0xef,0x5b,0x9b,0x0f,0xec,0xc4,0x9e,0x42,0x84, +0x63,0xf3,0xc1,0x7d,0x46,0x96,0xd7,0xb5,0xf6,0xcd,0x11,0xfb, +0xa2,0x76,0x4b,0x3e,0x97,0x66,0xb8,0x61,0x1d,0x8e,0x46,0x6f, +0x28,0xaa,0x0c,0x6e,0x7c,0xf7,0x14,0xe4,0x1f,0xe2,0x0d,0x14, +0x58,0xe7,0x76,0x97,0x1e,0x1e,0x1c,0xd8,0x0d,0xaa,0x75,0x70, +0x5b,0x00,0x5a,0x7e,0x35,0xe3,0x86,0x44,0x7b,0xc3,0x1f,0xb2, +0xcb,0xf1,0x55,0x7e,0x01,0x17,0x9f,0x1a,0xe9,0x72,0x17,0x0a, +0x0b,0x91,0x60,0x52,0x82,0xf7,0x1d,0x3d,0xe7,0x8d,0xb0,0x2c, +0x5d,0xb2,0xba,0x35,0x7c,0xdb,0xe1,0x43,0xb3,0x11,0x77,0x04, +0xbe,0xe9,0x37,0xaa,0xb9,0xc4,0x78,0xee,0xa3,0x5e,0x16,0x51, +0x0e,0x04,0xa4,0x27,0xb3,0xee,0x92,0x9e,0x40,0xee,0xc9,0xb7, +0xbe,0x2f,0x9e,0x12,0xef,0x86,0x0e,0xe2,0x24,0xc9,0x11,0x74, +0x91,0x6a,0x16,0x1c,0x83,0x7a,0xba,0xed,0x2c,0x99,0x35,0x4b, +0x13,0x1c,0x83,0x64,0x5b,0x3c,0x5a,0x16,0xd6,0xca,0x53,0x37, +0x6c,0xfa,0xbc,0x49,0x10,0xed,0xe0,0x70,0x13,0x82,0xe1,0x2d, +0xcd,0xc8,0xb8,0x01,0x64,0x26,0xca,0xc6,0xd9,0xd8,0x70,0xa9, +0xeb,0x75,0x50,0xd8,0x9a,0x28,0xfc,0x37,0x14,0x45,0xfb,0x43, +0x22,0x4b,0xab,0x7e,0x96,0x21,0x65,0xe3,0x62,0x51,0xcf,0x13, +0xb7,0x3f,0x4d,0xd4,0x7c,0xe6,0x0e,0xd1,0xa6,0x8d,0x22,0xad, +0x08,0xca,0xe2,0x0b,0x73,0x34,0x09,0xed,0xb6,0x48,0x24,0x89, +0x2a,0x35,0x80,0x4d,0x13,0xb8,0x5d,0xd0,0x9d,0x2e,0x22,0x73, +0xf6,0xac,0xce,0xcc,0x14,0xc2,0x3d,0x7e,0x49,0x91,0xbd,0x86, +0xf2,0x3e,0x35,0x86,0x50,0x84,0xbb,0x24,0x8d,0xe0,0x47,0x3d, +0x3d,0x30,0x31,0x6e,0x44,0xa2,0x3d,0xe9,0x74,0xc7,0xe5,0x4c, +0xc1,0x29,0xff,0x11,0x67,0x28,0x7a,0x11,0xa2,0x59,0xdb,0xb6, +0x1e,0x74,0xd5,0xe2,0x57,0x2f,0x9c,0xa2,0x5f,0x50,0x41,0xa3, +0xea,0x9d,0x17,0xe7,0xfb,0xe7,0xfb,0xfd,0x5f,0xce,0xf7,0x07, +0xdf,0xb0,0xfb,0xc0,0x24,0xf1,0xbb,0xf0,0x04,0xf5,0x40,0xc8, +0x12,0x4b,0xee,0x3f,0xf9,0x61,0xa5,0x84,0x35,0x7a,0x49,0xdf, +0x6f,0x86,0xa4,0xb0,0x64,0xda,0x98,0x66,0x3c,0x95,0x97,0x52, +0x3c,0x34,0xdf,0x62,0xdd,0xb6,0x52,0x06,0x55,0xd4,0xa2,0x0f, +0x56,0xc7,0xe7,0x70,0x3e,0x61,0x43,0xe3,0xa4,0xf6,0x32,0x53, +0xfe,0xb3,0xb9,0xcc,0xa4,0x06,0x44,0x8a,0xa3,0xe6,0x11,0x4a, +0x52,0x6b,0x73,0xbd,0xa8,0xad,0xb5,0x06,0x44,0xa5,0x1a,0x8f, +0x5a,0x62,0xe7,0xc7,0x8c,0x07,0xcb,0xf4,0x99,0xd2,0x91,0x6b, +0x13,0x4e,0x29,0x4d,0x06,0xe4,0xbb,0xb2,0xa7,0x12,0x8e,0xf4, +0x82,0xa7,0x68,0x20,0x97,0x3d,0x94,0x1c,0xf0,0xb8,0x90,0x45, +0xb4,0xbb,0xd9,0xb8,0xba,0x1c,0x2e,0xc6,0x02,0x4a,0x3c,0x20, +0xe0,0x9b,0xd6,0x81,0xda,0x5f,0x3f,0x32,0x8d,0x92,0xd7,0xcc, +0x50,0x96,0xa4,0x9e,0xbe,0x1d,0x2a,0xbb,0x9e,0x3c,0xcc,0xbc, +0x1d,0x1f,0x53,0x7a,0x53,0x77,0xda,0x5a,0x1f,0xcc,0x52,0x40, +0xd3,0x54,0x75,0x92,0xf2,0x88,0x78,0x18,0x49,0xc7,0xaf,0x93, +0x4e,0x82,0x72,0xf0,0xf8,0x66,0x56,0x9e,0xdd,0xdc,0xe5,0xad, +0x83,0xc3,0xd6,0xbf,0x0f,0x67,0xad,0xc3,0xbf,0xfc,0xdb,0x41, +0xeb,0xe0,0xa0,0xa0,0xff,0x5b,0x7f,0x7d,0x79,0x66,0x1c,0x55, +0x3a,0x0b,0xe1,0x76,0xb1,0x94,0x0d,0xab,0x20,0xae,0x3d,0xe9, +0x48,0x04,0x43,0x80,0xaa,0x6b,0xcd,0x51,0x25,0xb5,0xce,0x61, +0x6e,0xa1,0x3f,0xfe,0xee,0xe0,0x2f,0x7f,0x02,0x12,0x00,0x71, +0xcf,0x8f,0x43,0x14,0x2a,0xe6,0x51,0x68,0xfd,0x41,0x8d,0x69, +0xfe,0x01,0x6e,0xcb,0xaa,0x9a,0x5c,0x4c,0x3f,0xb3,0xea,0xc8, +0x78,0x85,0xf6,0x70,0xe6,0x1f,0xc6,0xcb,0x2b,0x24,0xcf,0x46, +0xad,0x8b,0xf1,0xe5,0x10,0x7d,0xb5,0x4e,0x56,0xad,0x8f,0x68, +0xd2,0x66,0x3e,0x87,0xd1,0x5c,0x5e,0x8f,0x5b,0xa9,0xd6,0xcf, +0x55,0x75,0x92,0xd6,0xe3,0x16,0x56,0xd7,0xba,0xf8,0xbc,0x1a, +0x57,0xd9,0x0e,0x72,0x1b,0x78,0xc7,0x8f,0x10,0x4f,0xb2,0xf7, +0x2d,0x8a,0x42,0xa1,0x4c,0x0f,0x19,0x76,0xe4,0x5b,0xf4,0xf9, +0xc8,0xe9,0xc4,0x6d,0xc3,0x49,0x72,0x97,0xd5,0x22,0xb2,0xe6, +0x14,0x11,0x01,0xab,0x08,0x3b,0xeb,0xbb,0x62,0x83,0xfc,0x61, +0x42,0x63,0xc3,0xed,0x42,0x2b,0xfe,0x31,0x8a,0x67,0x80,0xa0, +0xc0,0x9c,0xfb,0xb2,0x3d,0x14,0x81,0x40,0xdf,0xd1,0x11,0x17, +0xf6,0x7c,0xb8,0x08,0xa6,0xa4,0x0f,0x7a,0xe9,0x96,0x96,0x6b, +0x86,0x00,0x23,0x75,0x60,0x1f,0xea,0x0d,0x71,0x29,0xf0,0x19, +0xb9,0xd8,0x39,0x34,0xb4,0x46,0xac,0x06,0x4e,0x8c,0xa5,0x2d, +0xbc,0x83,0x6b,0xfc,0x91,0xeb,0x04,0x83,0xce,0xcc,0x66,0x23, +0x03,0xc1,0xd3,0x75,0x5e,0x2d,0xdc,0xab,0xfd,0x8e,0x3f,0x3a, +0x9a,0xd0,0x95,0xd9,0x42,0x15,0x5a,0x3a,0xb2,0x51,0x2d,0x3d, +0x68,0x73,0x93,0x59,0x2b,0xab,0xd9,0xce,0xaf,0x77,0xcf,0x47, +0x62,0x06,0x23,0xd4,0xcd,0xbd,0x82,0xdd,0x8b,0x06,0xa1,0x57, +0xf8,0x60,0x48,0x3f,0x3b,0x25,0xc5,0x3d,0x9d,0xe1,0x91,0x03, +0x63,0x3a,0x1d,0x43,0xb0,0xa7,0x81,0xb2,0xa4,0x4c,0x26,0x89, +0x23,0xba,0xb3,0xac,0x08,0x63,0x72,0xa0,0xc0,0x7f,0x4d,0x25, +0x31,0xe7,0x5f,0x98,0x0f,0x1f,0x9b,0x6b,0x3d,0x99,0x0b,0x95, +0x61,0x42,0x97,0xc5,0x11,0x0d,0x59,0x46,0xa5,0x51,0x82,0xee, +0x29,0x65,0x44,0xd1,0xc3,0xa7,0xdc,0x76,0x17,0xb9,0x53,0xba, +0x58,0x74,0xfd,0xa5,0xd1,0x36,0xdc,0x5d,0xf8,0x3c,0x59,0x6e, +0xb3,0xbb,0x30,0x56,0xad,0xd9,0x33,0xd2,0x5a,0x94,0x51,0x44, +0xf7,0x23,0x14,0xac,0x19,0xe7,0xb2,0x63,0x8e,0x06,0xba,0x26, +0x93,0x11,0xf9,0x67,0xe2,0x62,0xce,0x3e,0xd7,0x50,0xac,0xad, +0xae,0x86,0xf8,0xdc,0x96,0xe4,0x02,0x97,0x15,0xd2,0xaa,0xc9, +0x3f,0xc6,0xa8,0x3b,0x0c,0x88,0x4d,0x55,0x8a,0x02,0x05,0xdc, +0x4f,0x32,0x73,0xf9,0xfd,0x64,0x54,0x48,0x7e,0x34,0x67,0xc9, +0xba,0x1c,0x00,0x61,0x31,0xbc,0x9c,0xac,0x3e,0x97,0x92,0xad, +0xdd,0x96,0x40,0x57,0x53,0xd6,0x6b,0xb6,0x9e,0xd3,0x7d,0x79, +0xfc,0xf7,0x77,0x7f,0x3b,0x7e,0xf1,0xd3,0xd3,0x7c,0xba,0xbc, +0x43,0xd6,0x33,0x96,0x77,0xa3,0xcf,0x7c,0x05,0x9d,0x45,0xcb, +0xd3,0xe2,0x1e,0xf7,0xa5,0xee,0x41,0x79,0x8f,0x36,0x5d,0x9a, +0xe4,0xc9,0x08,0xd9,0x58,0xde,0xd1,0xb8,0x95,0x52,0x07,0xbd, +0xf7,0x01,0x5a,0x67,0x3f,0xcb,0x7b,0xf8,0x8b,0x02,0x90,0x9e, +0x43,0xa9,0x8b,0x51,0x8b,0x87,0x32,0xc1,0xea,0xa0,0x82,0x84, +0xce,0x0a,0x75,0x04,0xc3,0x6a,0x30,0x38,0x6a,0x70,0x36,0x46, +0x9a,0x2c,0x39,0x46,0x3f,0xd6,0x21,0x68,0xb7,0x69,0xc7,0xb0, +0x52,0x9b,0x5b,0xbc,0x5d,0xd2,0xbc,0x11,0x5b,0xc0,0xf4,0xbe, +0x60,0x3b,0xf5,0x40,0x77,0x10,0x8b,0x71,0x0d,0xdd,0xda,0x01, +0xd7,0xa2,0x8d,0x28,0x1c,0x7f,0x35,0x78,0x8d,0xc5,0x6b,0x5f, +0x33,0xd8,0x8d,0xe9,0xe6,0x4c,0x53,0x69,0x6f,0xb9,0x9c,0xda, +0x3f,0xbb,0x4f,0x5d,0xce,0x99,0xec,0x42,0x1f,0x91,0x5b,0x28, +0x72,0x10,0xdb,0xe6,0x68,0x9c,0xeb,0x0f,0x8d,0xed,0xde,0x5e, +0xa6,0xfd,0x3a,0x9e,0x4e,0xad,0xb3,0x13,0x5d,0x99,0xb2,0x9c, +0x9b,0xd6,0x5a,0xb0,0xcc,0x36,0xb9,0xe8,0xc0,0xd5,0x80,0xe8, +0x92,0x84,0xed,0x40,0x41,0x05,0x45,0x1f,0xd2,0xaa,0x68,0x71, +0x22,0x37,0xff,0x6a,0xde,0xe0,0x7b,0xc5,0xef,0x27,0x02,0x98, +0xdf,0x63,0xf3,0x0a,0xfc,0xb3,0xd1,0x27,0x1d,0x06,0x65,0x8c, +0x15,0xda,0x4d,0xdd,0x45,0xb8,0x31,0xca,0x4e,0x71,0x75,0xdf, +0x0b,0x0c,0xc8,0xa8,0xcf,0xd0,0xc3,0x80,0xee,0xf5,0x7b,0x2c, +0xe5,0x77,0x13,0xbf,0x34,0x61,0x5c,0xca,0x6c,0xf5,0xab,0xf9, +0x26,0x0f,0x6a,0x0e,0xce,0x79,0x07,0xa5,0x79,0x73,0x86,0x45, +0xed,0xcd,0xd7,0x68,0x26,0x23,0xbe,0xff,0x82,0x63,0xcc,0xdc, +0x73,0x36,0xde,0x33,0x4d,0x83,0x8b,0x26,0x51,0xb3,0x1c,0x55, +0x12,0xdd,0x61,0xa1,0xd5,0x28,0x63,0x6d,0x8a,0xa8,0x1e,0x6b, +0x1e,0x06,0x17,0x89,0xbe,0x9f,0xb9,0xd8,0x24,0x17,0xd1,0xe1, +0x77,0x4f,0x9e,0xbf,0x7d,0x7a,0x72,0xf6,0xfc,0x6f,0x4f,0x9d, +0x39,0xf9,0x5f,0xce,0xab,0x6f,0x9c,0x45,0x99,0x73,0xb1,0x27, +0x3f,0x42,0x0f,0x20,0x68,0x51,0xfe,0xbc,0xea,0xa4,0xdd,0x6f, +0xb2,0xdd,0xfd,0x9c,0x6d,0x97,0xd7,0xcb,0xa7,0x36,0x7b,0xda, +0x2b,0xce,0x8b,0xb4,0xff,0xcb,0x11,0x84,0xb3,0xde,0x51,0x2f, +0xdb,0xcf,0x87,0x48,0x04,0x9d,0x0e,0x81,0xd6,0x9b,0xfc,0x83, +0xc8,0xe3,0x9f,0x6f,0x80,0x94,0x24,0xfb,0x13,0x54,0x77,0xca, +0xb4,0xd7,0xfa,0x6a,0xb5,0x58,0xdf,0x0e,0x27,0xd3,0xd5,0x7c, +0x7d,0x05,0x7d,0xcd,0x8a,0xfd,0x7c,0x72,0x7b,0x7d,0xba,0xbc, +0xfc,0xca,0xb2,0x5c,0x68,0x8d,0xab,0xbe,0x98,0xdc,0x0e,0xaf, +0xc7,0xe7,0xfb,0xfb,0xf9,0xd3,0xbf,0x61,0xa7,0x7f,0x3c,0x7e, +0xf5,0xe4,0xc5,0xd3,0xb7,0xa4,0xac,0xe2,0xfb,0x9d,0xce,0x67, +0xfd,0xe1,0xde,0x3f,0x06,0xdf,0xac,0xd1,0x60,0xd6,0x90,0x86, +0x1a,0x3a,0xaa,0xef,0x28,0x32,0x22,0xa5,0xb9,0xec,0xd9,0xb4, +0xde,0x13,0x6f,0x7e,0x87,0xcd,0x7a,0xc8,0x67,0x3c,0xb7,0x0e, +0x69,0x67,0x73,0xcf,0xa9,0xb7,0xfc,0x1a,0x17,0x31,0x16,0xf8, +0xdc,0xfd,0x97,0x07,0xb3,0xda,0xf8,0x8c,0x0a,0x57,0x42,0x90, +0x49,0x88,0x1a,0x14,0x92,0x51,0x23,0x40,0x56,0x6e,0xb4,0xc3, +0xcb,0x22,0x94,0x5d,0x4d,0x6a,0x46,0x76,0x9a,0xe4,0x59,0xeb, +0xa6,0x78,0xd4,0xe4,0x80,0x5b,0x76,0x75,0xb9,0xc6,0x86,0xb6, +0x79,0xd0,0xf5,0x51,0x43,0x6e,0x78,0x00,0xb3,0xdc,0x8d,0xcd, +0x8e,0xd7,0x0a,0x1d,0x19,0xc1,0x3b,0x97,0x98,0xf5,0x3c,0x88, +0x7b,0xb1,0xbc,0x56,0xa8,0xcc,0xac,0xcf,0xb5,0x29,0x76,0xdc, +0x47,0x57,0xb2,0xb5,0xdb,0x3e,0x0a,0x8f,0x78,0x74,0xae,0x17, +0xe7,0x29,0x6b,0xa0,0x28,0x6b,0x66,0x0c,0x2e,0x76,0x17,0xcb, +0xc9,0x7c,0x89,0x18,0x45,0x3d,0x0a,0x28,0x03,0x93,0x91,0x68, +0xdb,0xf0,0x73,0xbd,0x0e,0x97,0x95,0x0a,0x56,0x96,0xb5,0x98, +0xf5,0xda,0x36,0x4e,0x95,0xd7,0x6c,0x1f,0x6a,0x90,0x90,0xba, +0xbd,0x5c,0x95,0xf5,0xa8,0xf5,0x1a,0x7d,0x6f,0xfb,0xf9,0x64, +0x06,0x8c,0x19,0x2d,0xff,0x08,0x11,0x2f,0x05,0x7a,0x8b,0x30, +0xbd,0xaf,0x36,0x83,0x2c,0x5a,0xbe,0x3c,0xfb,0x11,0x4c,0x9d, +0x44,0xaf,0x61,0x2f,0x06,0x6d,0xad,0xbc,0x7c,0x6d,0xd3,0x65, +0x62,0x63,0x92,0x1f,0x2c,0x1f,0x38,0x5c,0xdc,0x52,0x03,0x10, +0x40,0x66,0x9b,0x6d,0xa9,0xaa,0x8c,0x92,0x00,0x3b,0x73,0x3b, +0x14,0xce,0xc3,0x2a,0x22,0xc5,0xf6,0x7c,0xd2,0x9c,0x87,0x0e, +0xac,0xdf,0xd1,0x9e,0x87,0xc0,0x04,0x0d,0x7a,0x20,0x63,0xf4, +0xa2,0x6b,0x77,0xbc,0x31,0x89,0xd5,0x7c,0x00,0x24,0x64,0xff, +0x0a,0x7f,0x03,0x3b,0x51,0x18,0x41,0x86,0xac,0xd8,0xc6,0xbf, +0xd3,0x96,0xcc,0x23,0x6f,0x00,0x9e,0xca,0x4b,0xd0,0x48,0x15, +0xfe,0x38,0xbb,0x50,0xcd,0x7e,0x00,0xcc,0x41,0x63,0x1a,0x57, +0x3f,0x75,0xd8,0x30,0x57,0x64,0xbe,0x4a,0xac,0x6b,0x59,0x53, +0x63,0xe6,0x11,0xda,0x12,0x89,0xd0,0x18,0x63,0xa2,0xca,0xd9, +0x3c,0x8b,0x5e,0x11,0xd1,0x5b,0x80,0x5a,0x6c,0x24,0xb5,0xd9, +0x7c,0xe5,0x0c,0xc9,0x3d,0x9b,0xe5,0xb7,0xc3,0x4f,0x6f,0x64, +0x2f,0xe7,0x93,0xeb,0xd9,0x7c,0xe9,0x2d,0xb8,0xc1,0xe6,0xb0, +0xe5,0xac,0x59,0x1a,0x66,0xa0,0x22,0x6f,0xd6,0xe6,0x50,0x4d, +0xd1,0x20,0x32,0xf3,0xcf,0x5e,0x61,0x33,0x3c,0xe7,0x10,0x3e, +0xd5,0x47,0xe1,0xb7,0x65,0xe9,0xdc,0x33,0x8b,0x26,0x96,0x7e, +0xb2,0xdf,0x29,0xf1,0xf7,0x75,0x7e,0xda,0xd9,0x47,0xc6,0x51, +0x00,0x52,0x5d,0xe2,0x63,0x6e,0x6d,0x0b,0x39,0x80,0xee,0x7e, +0x44,0x1b,0xff,0xc9,0xf7,0xd5,0x62,0x38,0x7b,0xfc,0xfd,0x3e, +0xfd,0x24,0x99,0xd3,0x1c,0x45,0x51,0xc0,0x0d,0x53,0x5c,0x08, +0x6e,0x5e,0x41,0xc1,0x17,0x70,0x08,0x3e,0x9b,0x95,0x16,0xfc, +0x43,0xc3,0x18,0x26,0x3d,0x34,0xa8,0x35,0xbb,0xd7,0x62,0x5e, +0x1f,0x4d,0xb8,0x9c,0xcc,0x67,0x33,0xc8,0x86,0x6f,0x42,0xfe, +0x7a,0xe5,0xf4,0x84,0x7e,0x12,0x2f,0x71,0xbe,0x8b,0xc7,0x34, +0x56,0x57,0x86,0x45,0x7b,0x35,0xeb,0xfc,0x98,0xcc,0x7a,0x81, +0xe1,0xc4,0x14,0x61,0xa3,0x45,0x40,0xdd,0xc1,0x6d,0x16,0x4f, +0xa7,0xb1,0x75,0x79,0xfa,0x93,0xc1,0x51,0x7a,0x18,0xcd,0xd9, +0x7a,0xfd,0x97,0x28,0x26,0x43,0x86,0x9b,0x82,0x1d,0xff,0x96, +0x4e,0x32,0xf5,0x71,0xc1,0xbd,0xca,0xc5,0x8b,0x85,0xba,0x55, +0x18,0x5e,0x79,0x85,0x75,0x57,0x90,0x3c,0x0c,0xc9,0x28,0x44, +0xc3,0xd5,0x6e,0x87,0xdf,0xa6,0x10,0x43,0xce,0xa3,0x79,0x45, +0x0b,0xbe,0x41,0x84,0x8c,0xb2,0x2f,0xe7,0x42,0x99,0x0f,0xda, +0x27,0xea,0xa0,0x89,0x4e,0xe9,0xdd,0xb9,0xbf,0xa1,0xfb,0x5f, +0xe3,0xf1,0xf1,0x80,0x73,0xfa,0x0c,0x8d,0x2e,0xde,0x82,0xd5, +0x86,0x83,0x87,0x2f,0x08,0xd1,0x42,0x33,0xce,0x49,0x1e,0xde, +0xbc,0x01,0xd4,0x7a,0x57,0x1d,0xf8,0x00,0xe2,0x05,0x4a,0x61, +0x9f,0x99,0x0a,0xfd,0xac,0x73,0x69,0x31,0x12,0xc6,0x61,0xda, +0xb9,0x14,0xc1,0x87,0x13,0x05,0x6d,0x69,0xe4,0xf2,0x4f,0xf2, +0x19,0x52,0x58,0x17,0xdc,0x2f,0xac,0x12,0xb1,0x2b,0x95,0x71, +0x86,0x15,0xa7,0x2d,0xa9,0x2d,0xb8,0xb0,0x94,0xa8,0xe5,0xc9, +0x17,0xea,0xd2,0x3b,0x20,0x33,0x06,0x34,0xa5,0x56,0x55,0x01, +0x9c,0x19,0xc0,0xa1,0x35,0x1b,0x02,0xeb,0xcf,0x06,0xb9,0xef, +0x94,0x96,0xec,0x43,0xad,0x03,0xdb,0xc3,0x30,0xc1,0x17,0xe8, +0xa5,0x3e,0xcc,0xfe,0x31,0x7a,0xa9,0x1f,0x87,0x52,0x1c,0xbc, +0xcc,0xc6,0x1f,0x53,0x27,0x00,0x19,0x17,0x80,0x73,0x31,0x38, +0xa2,0xc2,0x0d,0xe1,0x81,0x01,0x42,0x11,0x43,0xae,0x8f,0x73, +0x69,0xa0,0xfb,0xe5,0x52,0xcf,0xb7,0x5e,0xef,0xd4,0x66,0xb8, +0xdd,0xb6,0x0b,0xac,0xe7,0x21,0xa5,0x76,0xaa,0xcd,0x2c,0xd3, +0xac,0x07,0xab,0xc6,0xdb,0x0e,0x0a,0x56,0x4e,0x7c,0xd2,0xd1, +0x26,0xd5,0x25,0xe5,0xb3,0xd6,0xc6,0xcc,0x9d,0x91,0x51,0x9e, +0xee,0xee,0xae,0x8f,0x21,0x35,0x6d,0x5b,0x5f,0x1a,0xe5,0xce, +0xb5,0x3a,0x7a,0xdf,0x73,0x66,0x75,0xc8,0xcc,0x75,0x2d,0x6f, +0xad,0x26,0xc9,0x9e,0x01,0x32,0x98,0x36,0x0c,0x63,0xd0,0xd1, +0xac,0xf8,0xca,0x41,0x73,0x0f,0x2c,0x0d,0xfb,0x4c,0xa6,0xd9, +0x9d,0x4f,0xfe,0xc3,0x6c,0x58,0x63,0x85,0xeb,0x41,0x60,0x1b, +0xbd,0x25,0xb6,0x6c,0x5c,0x8f,0xdd,0xe6,0xf8,0xa6,0x58,0xe5, +0xbc,0xce,0x9f,0x21,0x20,0x09,0xd3,0x33,0x2e,0xbf,0x84,0xd5, +0x36,0x28,0xec,0x29,0x2a,0x46,0x22,0xab,0xc7,0xde,0x27,0x97, +0x21,0xa0,0x2e,0x11,0x73,0x31,0xa6,0x98,0x2b,0xbb,0x73,0x73, +0x00,0xcd,0xf5,0x1e,0x94,0xe5,0xa4,0x67,0x8e,0xb1,0xc2,0xf7, +0x2a,0x3e,0xd0,0xec,0x7e,0x35,0x98,0xbd,0x38,0x1c,0x24,0x51, +0x28,0x5f,0xdb,0xd9,0x9c,0xcc,0xa1,0x99,0x7e,0xda,0xea,0xb9, +0xee,0xad,0xe7,0x6a,0x46,0xde,0x62,0x83,0x73,0xc0,0xd7,0xcd, +0x18,0x89,0xee,0xb5,0xf1,0xf2,0x96,0xcd,0x09,0xee,0x18,0xf8, +0x81,0xad,0xb4,0x2d,0x09,0xda,0x6e,0xac,0xa9,0x68,0x3c,0xf7, +0xc3,0xfc,0xf6,0xf0,0x69,0xdc,0xec,0x45,0xb0,0x24,0x75,0x12, +0xed,0xa1,0x89,0x11,0x51,0x82,0xe9,0x62,0x66,0xd7,0x40,0x69, +0xc2,0x28,0xe0,0xa9,0xc5,0xd7,0x6b,0x53,0xc2,0xc9,0x4d,0xf8, +0xbc,0xbd,0xe8,0xae,0xa1,0x81,0xb4,0xf7,0x5b,0xd3,0xa2,0xa8, +0xaf,0xc6,0x07,0xef,0x35,0x52,0xf6,0x23,0xff,0xad,0xee,0x36, +0xcd,0x9d,0xdf,0xd8,0x00,0xed,0x60,0x68,0x2f,0x87,0x0b,0xf2, +0xf1,0x06,0xd4,0x03,0xfe,0x1c,0x89,0x62,0x89,0xc3,0x4d,0xee, +0x45,0x3d,0x64,0x68,0xcc,0x6a,0xdb,0x85,0xe3,0x82,0xaf,0x90, +0xaf,0x82,0x16,0xa7,0x9d,0x6b,0xd8,0x77,0x72,0x76,0x07,0xc6, +0x42,0xb2,0x3c,0x79,0x9a,0x3c,0x8c,0x02,0xea,0xee,0xc4,0xf6, +0xb0,0xd9,0x95,0x19,0x77,0xe3,0x5a,0xed,0x95,0x30,0x13,0x3a, +0x27,0xc3,0x21,0x15,0xf7,0xca,0xf8,0xbe,0x7b,0x0f,0xf7,0xdd, +0xfb,0xf7,0x25,0xa7,0xc2,0x82,0xa4,0x5f,0xdd,0x9d,0xef,0xdf, +0x3f,0x7e,0x7f,0xf4,0x5e,0xf1,0x35,0x2c,0x75,0x8a,0x92,0x3a, +0x04,0x15,0xbf,0x9e,0xce,0x46,0x14,0xe6,0xe7,0x77,0x8c,0x11, +0x40,0xfd,0xf7,0x83,0x3c,0xdd,0x41,0x3b,0x11,0xeb,0x35,0xfe, +0x7d,0x5c,0xfe,0x79,0xbd,0x26,0xef,0x78,0xec,0xd1,0x63,0x32, +0x1e,0x21,0xca,0x4d,0x44,0xbe,0xf3,0xad,0x67,0xda,0x5c,0x36, +0x8d,0x14,0x69,0x5f,0xa1,0xbf,0x40,0x64,0x56,0xfd,0xf0,0xfc, +0xd5,0x93,0xe7,0xaf,0xfe,0xca,0xde,0xc3,0x7c,0x41,0x07,0xd5, +0x47,0xc9,0xdb,0x75,0xfa,0xa7,0xda,0xe0,0xee,0x1d,0xee,0x94, +0xfc,0x9e,0x6d,0xb3,0xb3,0x83,0x1c,0x79,0x73,0xa7,0xfe,0x26, +0xa8,0x24,0xce,0xf9,0x82,0x8c,0xac,0xee,0xf0,0x47,0x74,0x69, +0x64,0x87,0xa6,0x9c,0x45,0xe3,0xc3,0xee,0x21,0xa5,0x19,0x07, +0xec,0xd1,0x4b,0x0b,0x67,0x9d,0x64,0x3c,0x1b,0x25,0xc6,0x8d, +0x64,0x63,0xbe,0x3f,0xa1,0xa7,0xc9,0x87,0x46,0x26,0xee,0x9b, +0xae,0xd7,0xfe,0x8c,0x35,0xd0,0x5c,0x9b,0x2a,0x8d,0xe1,0xa7, +0x7a,0xb2,0x91,0x84,0x53,0xd4,0x6e,0xb3,0x34,0x48,0x49,0x59, +0x7b,0x0d,0x0e,0x85,0x69,0xe9,0x00,0xd5,0xec,0x4e,0x69,0x66, +0x4b,0x3c,0x42,0x1d,0x4b,0xef,0xf8,0x30,0x6f,0xb0,0x98,0x43, +0x5b,0x73,0xa6,0xf3,0x13,0xb4,0x02,0xa5,0x5c,0x60,0xbb,0x60, +0x56,0x63,0x83,0xda,0x70,0x12,0xa3,0x5d,0x2d,0x8b,0x99,0x55, +0xf1,0xb6,0xed,0x33,0xde,0x02,0xc8,0xb7,0x79,0x60,0xef,0xe4, +0x5b,0xd7,0xb3,0x3c,0xcb,0x39,0x13,0x53,0x7c,0x5d,0xba,0x13, +0xc2,0xb1,0x2f,0x3d,0x0a,0x0e,0x83,0x87,0x32,0xd7,0x3e,0x82, +0x44,0x5d,0xd9,0xc3,0x65,0x33,0x7b,0xb8,0xe6,0xe4,0x32,0xdb, +0x3a,0xbd,0xea,0xdc,0xf2,0x8b,0xfd,0x3d,0x79,0xf8,0xac,0x88, +0x87,0x9e,0xa7,0x9e,0x80,0x7f,0x8b,0x8c,0x29,0xdf,0x61,0x17, +0xd2,0xc5,0x48,0xb9,0x58,0x12,0xa5,0xc3,0x25,0x0e,0x06,0xaa, +0x26,0x77,0x74,0xb1,0x1c,0x0f,0x7f,0x55,0xed,0x36,0x68,0x22, +0x5a,0xdd,0x68,0x9c,0xcb,0xe8,0x0a,0xf5,0xa4,0x78,0x00,0xe3, +0xcf,0x05,0x12,0x3c,0x32,0x76,0x5b,0x98,0xf3,0x3c,0x7a,0x11, +0x10,0xf6,0x28,0x8a,0x07,0xc0,0xc3,0x23,0x79,0xf8,0xe5,0x91, +0x7c,0xf9,0x4f,0x8f,0x24,0x4c,0x53,0x16,0x50,0x64,0x6a,0xc3, +0xce,0xa3,0x19,0xd5,0x7c,0xb9,0x4a,0x2f,0x3e,0x2b,0xf4,0x80, +0x41,0xe8,0xf5,0x02,0x96,0xf3,0xbb,0xc5,0xe9,0xe5,0x50,0x78, +0x1a,0x6e,0xad,0xea,0x3a,0xf5,0xc4,0x14,0x61,0x57,0xa3,0xf1, +0x62,0x75,0x83,0x18,0xd6,0x95,0x3f,0x8b,0x84,0xe1,0x71,0x33, +0xac,0xdc,0x8e,0x6d,0x88,0xf2,0xf9,0xb3,0xec,0x7e,0x34,0x27, +0xad,0x7a,0xba,0x8a,0x44,0x29,0x46,0x10,0x0a,0xa7,0x13,0x03, +0xf3,0x0a,0xed,0x25,0xb5,0x18,0x41,0x59,0x50,0xa5,0xd3,0xdd, +0x28,0x39,0x6b,0xdf,0xca,0xab,0x34,0xc4,0x40,0x1b,0xf9,0x8e, +0x45,0x4b,0xc1,0x7f,0xb8,0x3f,0x84,0x58,0xca,0xd1,0x4d,0x1a, +0x3a,0x75,0x14,0x33,0x00,0x70,0x36,0x1f,0x68,0x71,0xbb,0x4d, +0x1d,0xef,0x74,0xf2,0xe6,0x5c,0x08,0x53,0xf2,0xec,0xed,0x31, +0xda,0x67,0xf0,0x18,0x8e,0x90,0x0a,0x61,0xd1,0x9e,0x4e,0x2e, +0x50,0xb8,0x7f,0xf3,0xf1,0x06,0x39,0x5d,0x54,0xea,0xf1,0x81, +0x58,0xd8,0x8d,0x8b,0x2a,0xc6,0x62,0x08,0x2f,0x6b,0x2a,0x87, +0xa6,0x4f,0xb5,0x17,0x18,0x81,0xf9,0x99,0xfc,0x9f,0xa2,0x8a, +0x07,0xe1,0xcd,0x0d,0x33,0xda,0xcc,0xce,0x51,0x8e,0x00,0xe3, +0x33,0x9e,0x95,0x57,0xc5,0xb6,0x7b,0x4b,0xbf,0x66,0xbc,0x51, +0xb5,0x86,0x7a,0x04,0x3b,0xfb,0x22,0x78,0xc3,0x4d,0xf8,0x22, +0x4a,0x6c,0xd0,0xce,0x5c,0x79,0x90,0xc7,0x75,0xd4,0xf8,0xfd, +0x6f,0x27,0x8c,0xb7,0xa1,0xf4,0xb8,0xda,0xc4,0x7e,0xcb,0xc2, +0x86,0x4d,0xfc,0x06,0xd8,0xa6,0x3c,0x78,0x15,0x0a,0x50,0xa0, +0x73,0xcb,0x55,0xd3,0xc0,0x2d,0xd0,0x12,0xa7,0x5d,0xf8,0x98, +0xbb,0x7c,0x60,0x0e,0x10,0x58,0x1d,0x0e,0x49,0x6c,0x6c,0x7f, +0x55,0xc0,0xd4,0x17,0x16,0xe5,0x85,0x6f,0x2c,0x03,0x8d,0x8a, +0x6b,0x87,0xa8,0x07,0xab,0x6f,0xee,0x07,0xc3,0x7a,0xa8,0x01, +0x90,0x1c,0xb6,0x60,0x8e,0xe6,0x7e,0x02,0x6b,0x75,0xde,0x58, +0x1f,0xbe,0x11,0x70,0xb9,0xdd,0xb1,0x75,0xce,0xcc,0x57,0x28, +0xac,0x9d,0xe5,0x04,0x35,0x4e,0xd8,0xa7,0x5c,0x29,0x66,0x3c, +0x59,0xc0,0x63,0x38,0x45,0x41,0x5b,0x6b,0x20,0x43,0x40,0x65, +0x6c,0xcf,0xef,0x28,0xf9,0x05,0x10,0x05,0x31,0xb9,0x27,0x49, +0xc6,0x68,0x38,0x39,0x9a,0x2c,0xc5,0xec,0x67,0xa6,0xfd,0xd1, +0x7c,0x72,0xa7,0x1c,0x02,0x76,0xfb,0x8b,0xe6,0x82,0x41,0xab, +0x35,0x28,0x70,0xe0,0x99,0x64,0xbe,0x69,0x1a,0x08,0xaa,0x81, +0xc6,0x72,0x73,0xb4,0xb3,0xfd,0x08,0xe0,0x20,0x45,0x1b,0x87, +0xd2,0x88,0xd0,0xa8,0x61,0xbe,0x23,0x8d,0xd8,0x51,0xd0,0x5b, +0x4e,0xbf,0xcb,0x15,0x3b,0xd9,0xf4,0x85,0xf9,0xa0,0xcb,0xd5, +0xa7,0x20,0x59,0x38,0x77,0x13,0xc7,0xe7,0x5d,0xde,0x82,0x2d, +0xf9,0x87,0x55,0xeb,0x62,0xcc,0x27,0xdf,0x4e,0xa2,0x1e,0x52, +0x0c,0x41,0x60,0xed,0xfd,0xb0,0xa9,0x19,0xf7,0x0a,0xc2,0xca, +0xb6,0x3a,0x01,0x30,0x52,0xdc,0x4f,0xb4,0x92,0x25,0xdc,0x70, +0x05,0x67,0x5e,0x52,0x38,0xf3,0x3d,0x65,0xe5,0xb5,0xf2,0xa5, +0xb5,0x41,0xf6,0xa9,0x22,0x2b,0x37,0x5b,0x98,0x11,0x7c,0x72, +0x78,0x06,0xe7,0x57,0xb0,0xfd,0xf8,0x78,0x71,0xec,0x4d,0x62, +0xe6,0xc9,0x49,0xe8,0xcf,0x1c,0xbd,0xbe,0x2a,0xcb,0x2c,0x2f, +0xcb,0x52,0x6b,0xea,0x05,0x47,0x4b,0x51,0xdd,0x0c,0xa7,0xd3, +0xf9,0xc7,0x93,0xf9,0xe2,0x73,0x1a,0x1e,0x3a,0x21,0x33,0x41, +0x9c,0xd5,0xa6,0x9e,0x0f,0x1b,0xe4,0x66,0x62,0x0e,0x3a,0xaf, +0xad,0x53,0x1a,0xcf,0xb5,0x16,0xc0,0x3d,0xaf,0x08,0x99,0x21, +0x0e,0x4c,0x44,0x41,0xbe,0x78,0x7d,0x72,0xfc,0x22,0x90,0x1a, +0x48,0xfb,0xff,0xb3,0x6c,0x0f,0xb2,0xf4,0xbc,0xd7,0xcb,0xf0, +0xf3,0xfc,0xe3,0x37,0xf8,0xbb,0xbb,0x9f,0xf3,0x3b,0x41,0xc0, +0xa6,0xe2,0xa8,0xf5,0x9a,0x3e,0xbd,0x83,0xfa,0x2d,0x55,0x8a, +0x0b,0x5d,0x2b,0x06,0x39,0xc1,0x47,0xae,0xb9,0xcc,0xca,0x2b, +0xa7,0x11,0x8a,0x74,0x0c,0xab,0x79,0x73,0x0d,0x7f,0x45,0xf5, +0x4d,0xae,0x7e,0xbc,0x12,0xef,0xeb,0xbe,0xb4,0xbc,0x86,0xd8, +0xbe,0x64,0x64,0x49,0x7b,0xa8,0x24,0x99,0x62,0x8a,0xd2,0x52, +0x42,0x94,0xdc,0x96,0xa4,0x8d,0xa8,0x18,0x10,0x9a,0x1e,0xd1, +0xfc,0x87,0x03,0xa5,0x92,0xa5,0xbb,0xbb,0x13,0xee,0x16,0x3a, +0x65,0x82,0x83,0xa5,0xea,0x3b,0x68,0x03,0xb4,0xb2,0x32,0xee, +0x68,0x85,0x04,0x85,0x49,0xea,0xe4,0x7f,0x26,0x85,0xcc,0xc9, +0xfc,0x02,0x75,0x59,0xf9,0x76,0xa7,0x5c,0x6e,0x24,0x44,0x4e, +0x8c,0xc0,0x59,0xa0,0xbc,0xa4,0x85,0x22,0x82,0xfa,0x05,0xc2, +0x92,0xbd,0xba,0x53,0x1e,0x88,0xa5,0x02,0xa5,0x99,0x1e,0x21, +0x96,0x5c,0x1e,0x14,0x78,0x8a,0x41,0xdb,0x37,0xb9,0x34,0xca, +0x9f,0xa5,0x06,0x58,0x66,0x31,0xdc,0xa4,0x4c,0x0a,0xd4,0xa1, +0x93,0xb1,0x83,0x53,0x27,0x2e,0x2a,0xca,0x24,0x6e,0xde,0x4a, +0x7e,0xc8,0xab,0x55,0xe1,0xe7,0xb3,0x74,0x79,0xbb,0x80,0xba, +0x03,0xbe,0xba,0x5e,0xd7,0xf4,0x9b,0xdd,0x82,0x28,0x6b,0x1d, +0x71,0xa5,0x83,0x46,0xe7,0xf1,0xf1,0x37,0x83,0xf6,0x12,0x74, +0x38,0x02,0x9f,0x7e,0x82,0xeb,0xaf,0x42,0xaf,0x77,0x82,0xeb, +0xdd,0x55,0xea,0xdc,0x3d,0x3a,0xff,0xd8,0x0b,0xc4,0x6c,0x8f, +0x8b,0x92,0xf3,0xd0,0x24,0x1e,0xdd,0xad,0x5b,0x8c,0x5d,0x82, +0xb3,0x62,0xc1,0xef,0x6b,0x7c,0xb5,0x45,0x2b,0x93,0x94,0x98, +0x28,0x23,0xc3,0x6b,0x2e,0xee,0x6c,0xaf,0xf9,0x9c,0x40,0xe0, +0xc5,0x75,0xe3,0x5d,0x1f,0x64,0x70,0xcd,0xec,0x7d,0xb1,0xc1, +0x6c,0xaf,0xc0,0xcd,0xa0,0xad,0x3d,0xb7,0xed,0xdb,0x0e,0x28, +0xcb,0x6c,0xc6,0x4d,0xb0,0xc6,0xda,0x49,0xf1,0xe5,0xe5,0x53, +0x6b,0x99,0x57,0x1d,0x14,0xe3,0x11,0xc1,0x28,0x44,0x63,0xa4, +0x96,0x19,0x36,0x52,0xaf,0x9a,0xab,0x68,0xbe,0x3a,0x27,0xd5, +0xe5,0x82,0xec,0xc3,0xc0,0x7e,0x9c,0x8c,0x5a,0x72,0x0a,0xb4, +0xa8,0xf2,0x96,0x3f,0x87,0x50,0x48,0x26,0x58,0x40,0xb0,0xb4, +0xba,0xad,0x27,0x2e,0xbd,0x68,0xdd,0x77,0xbb,0x5d,0xf4,0x69, +0x5d,0x40,0xe2,0x23,0x58,0x5d,0xf0,0x89,0xfe,0xad,0x1f,0x5a, +0x3f,0xfe,0x68,0xcc,0x7d,0x4d,0xec,0x58,0x13,0x8d,0x1b,0xd5, +0x5d,0xd7,0x59,0xeb,0xd5,0x0d,0xa9,0x0d,0xd2,0x3a,0xaa,0xa9, +0xa2,0x99,0x9f,0x87,0xe6,0x2b,0xca,0x7b,0x3e,0x61,0x0a,0x79, +0x1c,0x94,0x9b,0xa5,0x70,0x57,0x0c,0xdd,0x41,0x55,0x21,0xd7, +0xa4,0xe1,0xc5,0xd6,0xee,0xd3,0xcd,0x91,0xaf,0xa5,0x6c,0x92, +0x73,0xc9,0xe1,0xb8,0x2c,0x4b,0x2b,0xf7,0x62,0x3a,0xc1,0x77, +0x5b,0x3f,0x94,0x84,0x19,0x58,0x67,0xaa,0xce,0x70,0x9c,0x11, +0x1b,0x30,0x00,0x9c,0x3d,0x8e,0xdd,0xc0,0xd4,0x32,0xe2,0x56, +0x21,0xd0,0x00,0xc5,0x6a,0x80,0x6f,0xc5,0x82,0x7c,0xea,0x71, +0x85,0x82,0xa9,0x6c,0xe8,0x87,0x87,0xac,0xdf,0x9c,0x6d,0x50, +0x36,0x80,0x44,0x19,0x47,0x7e,0xc0,0x33,0xb8,0x7a,0xfc,0x16, +0x88,0xec,0x05,0x46,0x3d,0x4c,0x2e,0xe4,0xe5,0x07,0x74,0xd1, +0x6e,0x48,0x18,0x71,0x9a,0x62,0xe2,0xed,0x76,0x84,0x4d,0x85, +0xc9,0x16,0xa9,0x7a,0x48,0x3c,0x28,0x27,0xc5,0x47,0xb8,0x05, +0xcf,0x86,0xd7,0xa9,0x29,0x43,0xab,0xf2,0xe1,0x27,0x1c,0xf7, +0x06,0xfe,0x95,0xcf,0x38,0x6e,0x5c,0x0c,0x09,0xf1,0xd0,0xc0, +0xf8,0x6c,0xff,0x9f,0x1a,0x19,0xf7,0x1a,0xe5,0x16,0x45,0x88, +0x43,0x7b,0x09,0x18,0x43,0x78,0x7a,0x96,0x5f,0xe3,0x66,0x9e, +0x46,0x0f,0x59,0x8c,0x34,0xe9,0x7b,0x8c,0xb2,0x69,0xca,0xbd, +0x9a,0xa4,0xbb,0x5f,0x3d,0x88,0x8c,0x9b,0x21,0xc3,0x4f,0x38, +0x90,0xc2,0x93,0x88,0x25,0x8b,0xb7,0x9d,0x53,0x9c,0xba,0x8a, +0xbd,0x7d,0x72,0xb4,0x95,0xae,0x28,0x23,0xdc,0x75,0x37,0xb2, +0xad,0x6e,0xb2,0x3a,0x3d,0x4e,0x0f,0x6e,0x1b,0xcd,0xdd,0xf0, +0x0e,0x1c,0x10,0xee,0xb2,0x7a,0x6a,0x8f,0x65,0x46,0xa4,0xc3, +0xcb,0x16,0xfa,0x5c,0x28,0xd4,0x11,0x3c,0x1d,0x98,0x03,0x0b, +0x70,0x2e,0x4b,0xfd,0x06,0x29,0xe3,0xd9,0x28,0x62,0x65,0x05, +0x02,0x27,0x86,0xd3,0x61,0x19,0x0f,0x4d,0x14,0xb5,0x5b,0x0f, +0x5e,0x25,0xad,0x36,0xb5,0x8f,0xeb,0xc2,0x88,0x19,0x5f,0x69, +0x68,0x49,0x36,0x5c,0x17,0xa6,0x99,0xfc,0xf6,0x8e,0xa4,0x3f, +0x09,0xd8,0xbc,0x9a,0x3f,0xb9,0x5b,0x4c,0x27,0x97,0x88,0x0b, +0x26,0x72,0xbd,0x8d,0x5a,0xf2,0x08,0xbf,0x6d,0xda,0xfd,0xaa, +0x0b,0x26,0xd9,0x58,0x3b,0x0a,0xab,0xc7,0xea,0x42,0x81,0x11, +0x3b,0x00,0x28,0xd6,0x22,0x35,0x3b,0xf1,0x96,0xad,0x0b,0xce, +0x5f,0x5d,0xf9,0xc3,0x10,0x0d,0xa4,0x78,0xcd,0xc6,0x31,0x46, +0xf0,0x32,0x0b,0x37,0x66,0x24,0xcb,0x99,0x6f,0x1f,0x56,0xbf, +0x47,0xb3,0xe0,0xe6,0x7a,0x62,0x5f,0x84,0xeb,0x91,0xeb,0xf5, +0xfd,0x26,0x6f,0x98,0x89,0x3f,0x98,0x7b,0xe9,0x95,0x68,0xe5, +0x5d,0x36,0x5e,0x4c,0x46,0x2a,0x33,0x28,0x31,0xd8,0x3a,0x49, +0x5f,0x51,0xd6,0x0e,0xf3,0xf6,0x3e,0xfb,0xbd,0xb6,0x65,0x3d, +0x69,0x86,0x8a,0x2c,0xce,0x34,0x1d,0x6d,0xdb,0xda,0xd8,0x90, +0xd7,0x57,0x5c,0x3f,0xe5,0xea,0x5b,0x21,0x4f,0xd4,0xfd,0x5a, +0x59,0x86,0x1d,0xe8,0xa5,0x7e,0x77,0x7d,0xed,0x8e,0xfc,0x5d, +0x67,0x59,0xe4,0x14,0xed,0x64,0x7e,0xcb,0x4e,0xdb,0x5a,0xb5, +0x39,0x2d,0xd0,0xdd,0x87,0x05,0x16,0x4d,0x03,0xf9,0x41,0xc9, +0x2c,0x9b,0xb2,0xb4,0x0d,0x41,0x46,0xa9,0x9c,0x95,0x3f,0x03, +0x05,0x93,0x06,0x7c,0x4a,0x75,0x98,0xe8,0xcc,0xa2,0xb8,0x5e, +0x87,0x10,0xb3,0x86,0x43,0xd4,0x89,0x4d,0x36,0x5c,0x4b,0x70, +0xac,0xc6,0xc3,0x5f,0x3b,0xb0,0xdb,0xed,0x38,0x86,0x09,0xa2, +0xac,0x30,0x63,0x2f,0x0d,0xb4,0xee,0xdd,0x6c,0xb3,0xc8,0xec, +0xec,0x0a,0xb9,0x90,0x69,0x38,0xfc,0x5d,0xf4,0xa7,0x91,0xa2, +0xc9,0xf4,0xdf,0xdb,0xf0,0x2c,0x10,0xea,0x76,0xc3,0x81,0x47, +0x76,0xc3,0xda,0x95,0xe4,0xa4,0x7e,0xbd,0x6d,0x5f,0xb5,0xb5, +0x8b,0xb0,0xe1,0xc2,0x17,0xa3,0x71,0x0d,0x92,0xee,0xbe,0x49, +0xbd,0x7a,0x5c,0x78,0xc2,0x05,0xab,0x26,0x2b,0xea,0xd9,0xe3, +0xfa,0x46,0xe3,0x99,0x3e,0x28,0xa9,0xda,0x4b,0x7c,0x48,0x07, +0x92,0xe5,0xde,0x7e,0x44,0xed,0x36,0x36,0xbd,0x8f,0x27,0x93, +0x7d,0x10,0x8a,0x0f,0x9b,0x08,0x7c,0x27,0x11,0xaf,0x84,0xc1, +0xcc,0x06,0x6b,0x5b,0xc1,0xe1,0xc2,0x3e,0xdc,0xf1,0xdf,0x72, +0x6d,0xaf,0xd7,0x87,0xf8,0x5c,0x69,0x96,0x82,0x93,0x61,0x68, +0xa6,0xde,0x56,0x8b,0xe9,0x12,0xc5,0x96,0xb5,0xc7,0x4d,0x64, +0x1a,0x9b,0x35,0xbf,0x19,0xc2,0xf7,0xf8,0xd3,0xf0,0x72,0x35, +0xfd,0xdc,0x82,0xa5,0xd8,0x42,0xe6,0xa0,0x73,0x2f,0xd8,0x22, +0xd3,0xb4,0x21,0xca,0x96,0x90,0x05,0x82,0x2d,0x9b,0x2f,0x98, +0x2b,0xbb,0x44,0x08,0xaf,0x80,0x0b,0xe8,0xcc,0xce,0x1f,0x90, +0x59,0x78,0xd0,0x14,0xf7,0x9b,0xcd,0x91,0x91,0x20,0x32,0x68, +0x8b,0x58,0x60,0xae,0x8b,0x8f,0xd8,0x7a,0xec,0x63,0xdd,0x82, +0x2c,0xeb,0x4f,0x3a,0x87,0x79,0x0d,0xf9,0xd9,0xc3,0xe8,0x8c, +0x2e,0xc6,0xa0,0x15,0x10,0x75,0x3b,0x5e,0x5e,0x8f,0x6d,0xa4, +0x70,0x22,0x6b,0x6c,0xca,0xb0,0x60,0x23,0x8e,0xc5,0x0f,0x50, +0xf5,0x8d,0x1b,0xad,0x0a,0x72,0xde,0x55,0x5b,0xbd,0x3f,0x2d, +0xa7,0xd9,0xff,0xc9,0x3d,0xa9,0x0b,0x9e,0x18,0xfa,0xdb,0x96, +0x7b,0x20,0x80,0x25,0x75,0x9c,0xf9,0x06,0xa7,0x0d,0xe3,0xdf, +0x30,0xfa,0x13,0x0b,0x26,0xdf,0xbe,0xa9,0xc3,0x57,0xa7,0xda, +0x51,0xd7,0x3c,0xe8,0x47,0x6c,0xbe,0xfa,0xaa,0xae,0xe6,0x62, +0xc9,0xa4,0x5a,0xe2,0x03,0x67,0x4b,0x63,0xcd,0xfe,0xec,0x62, +0x88,0x59,0xcf,0x3c,0x79,0xb1,0x5e,0xe2,0xb6,0xe7,0xc2,0x62, +0x2a,0xc4,0xa1,0x29,0x21,0xc4,0x16,0x3e,0x6f,0x69,0xb0,0xf9, +0xa9,0xe9,0x77,0x10,0x5f,0xc1,0x55,0xb2,0xb1,0x2b,0x8c,0xef, +0x2e,0x79,0xa2,0x8d,0xc4,0xd8,0x48,0xa4,0x32,0x46,0x2e,0x5e, +0xa2,0xc9,0xc4,0xdb,0xe1,0xa7,0xb4,0x76,0xef,0x35,0x60,0xe0, +0x4e,0x86,0x3b,0x16,0x77,0xad,0xa3,0x9d,0xed,0x76,0x2d,0xaa, +0x6b,0x45,0x93,0x23,0xf9,0xb6,0xb2,0x01,0x27,0x12,0x1a,0x3b, +0x90,0x38,0xf6,0x85,0x37,0xf6,0x45,0xd2,0x0b,0x0e,0x98,0xf3, +0x83,0x65,0xaf,0x9c,0x65,0xac,0x87,0xe4,0x4e,0x68,0x74,0x9d, +0x74,0x16,0x90,0x3a,0x4e,0x78,0x49,0x1d,0x47,0x96,0x65,0x19, +0x8b,0x1c,0xe8,0x58,0xa0,0xea,0xb4,0x13,0x5b,0x63,0x4d,0x6a, +0x28,0xf4,0x65,0x95,0xb8,0xec,0x01,0xc2,0xb9,0x32,0x0a,0x65, +0x68,0x2d,0xd2,0xe8,0xc4,0x65,0x5f,0x41,0xf9,0xe1,0x7e,0xd8, +0x42,0xfc,0xe5,0xa9,0x19,0x08,0x6b,0x16,0x64,0xbd,0x36,0x09, +0x4d,0xf4,0x17,0x5a,0x51,0xdf,0x69,0x50,0xc7,0x72,0xe6,0x50, +0x74,0xa8,0x33,0xd6,0x44,0xf6,0x23,0x6a,0x55,0xd3,0x4a,0x79, +0x43,0xf4,0x31,0xf9,0xbd,0xd0,0x9d,0x45,0x38,0x0d,0x44,0x73, +0x16,0x66,0x32,0xf0,0x35,0xcc,0xcc,0x6f,0xac,0xf6,0xa5,0x0f, +0x28,0x2e,0xe2,0x0b,0x7a,0x60,0x32,0x7f,0xb7,0xa1,0x93,0xe7, +0x6d,0xd7,0xc4,0x08,0x6d,0x8d,0x2d,0x2f,0xc5,0xa1,0xf9,0xf2, +0x12,0x93,0x4a,0xf8,0xe5,0xa7,0xaa,0x1c,0x92,0x29,0x06,0x7e, +0x25,0xc6,0x61,0xce,0x14,0xa5,0x5f,0xee,0x35,0x69,0x64,0x0d, +0x46,0xf2,0xfb,0x2f,0xa9,0x89,0x27,0xbb,0xc9,0x0e,0xfa,0x83, +0xf4,0x6f,0xb7,0x38,0x9a,0xcb,0x4b,0x52,0xc5,0xd6,0x97,0xc6, +0x4e,0x99,0x26,0xd5,0xea,0xf3,0x74,0x8c,0xd6,0x15,0x21,0xa1, +0x97,0x1c,0x25,0x05,0xe2,0xd3,0x1d,0xcd,0x99,0xe5,0x54,0x6d, +0x35,0x5e,0x79,0x8d,0xfd,0x1c,0x0e,0x00,0x69,0x39,0xe7,0x41, +0xe6,0x9c,0xb6,0x07,0x12,0x9a,0xdb,0x43,0x92,0x47,0x09,0x57, +0x93,0x36,0x6b,0x46,0x88,0x98,0x19,0xd4,0xd8,0x97,0xec,0x83, +0x32,0xb5,0x5f,0x3d,0xfb,0x81,0x88,0x7f,0x01,0x18,0x46,0x87, +0x8b,0x15,0xae,0x27,0x58,0x83,0xd7,0xaa,0x58,0xa1,0x35,0x24, +0x4e,0xca,0x1b,0x63,0xd1,0x30,0x4c,0xe2,0x80,0xec,0x32,0x00, +0x3f,0x6c,0x40,0xf8,0xc2,0x08,0x34,0x58,0x26,0x5f,0xaf,0xa9, +0x6d,0xc6,0x78,0x80,0x4c,0x1e,0x47,0x05,0x23,0xb4,0xc9,0x6a, +0xfa,0x1b,0xcd,0xb7,0x62,0x7e,0x31,0x86,0x91,0x74,0x89,0xaf, +0xb6,0xdd,0x82,0x7c,0xef,0x04,0xcf,0xb3,0xf5,0x4b,0x88,0x9f, +0x67,0xaf,0xe0,0x48,0x6e,0x80,0x17,0xc4,0x9f,0x98,0x27,0x60, +0xbc,0x5e,0xd8,0x47,0x45,0x7f,0x10,0x35,0xe7,0xe4,0x01,0x22, +0x0b,0xf9,0x4f,0x64,0xe2,0xb2,0x01,0x2b,0xa8,0x9c,0xb3,0xfb, +0x11,0xec,0xd9,0x0f,0xe3,0xd1,0x69,0x90,0xcf,0x18,0xad,0xa8, +0x01,0xc9,0xef,0x3d,0x29,0x5e,0x04,0x9c,0x34,0x18,0x38,0x89, +0xf0,0x2c,0x75,0x56,0xfd,0x27,0x76,0x3c,0x05,0x05,0x51,0x61, +0x09,0xe3,0xcc,0x16,0xb5,0xc4,0x45,0xbd,0xd6,0x00,0xc9,0xea, +0x3d,0x9c,0x9e,0x36,0xcc,0x4c,0x56,0x3c,0x5c,0xe6,0xc8,0xe9, +0xc2,0x37,0xd1,0x6c,0xa4,0xf1,0x47,0xe7,0x35,0xaa,0xef,0x61, +0xe0,0x6c,0x09,0x08,0xf8,0x78,0xf4,0x76,0x5c,0xcd,0xef,0x96, +0x97,0x54,0xab,0x6d,0x21,0x0c,0x12,0x6a,0x05,0x16,0xa1,0x92, +0xe0,0x26,0xeb,0x56,0x77,0x97,0x97,0x63,0xd8,0x62,0x5e,0xbb, +0x82,0xa9,0x0a,0x7d,0xd3,0x08,0x11,0x9a,0x00,0x63,0xf5,0x84, +0xcb,0x11,0x59,0xb3,0xa2,0x72,0x8d,0x24,0x92,0xc2,0x6c,0x98, +0xbd,0x80,0x48,0x7a,0x98,0x12,0x52,0x28,0xff,0xbf,0x27,0x81, +0x1a,0x46,0x81,0x30,0x08,0x3b,0x61,0x47,0xb5,0xe1,0x36,0xf4, +0x4d,0xc0,0xaf,0x08,0xb6,0xf8,0x56,0x92,0x29,0xff,0x5a,0xaa, +0xa7,0x3e,0xcf,0x0f,0x50,0x34,0x22,0xe6,0x55,0x2b,0xc1,0x64, +0x8b,0x9d,0x92,0xad,0xc7,0x44,0x03,0x57,0x42,0x27,0x9a,0x04, +0xdc,0xcd,0x11,0xa1,0x2e,0x39,0x9a,0x8e,0x88,0x2c,0xdf,0x72, +0x8c,0x95,0xbf,0x4f,0x7e,0x6d,0x0b,0xba,0x1e,0x0e,0x6c,0x7d, +0x02,0x8d,0x76,0x6a,0xa3,0x46,0x13,0x09,0x72,0x4e,0xb2,0x7b, +0x12,0x3a,0xb4,0x2b,0x0f,0xb9,0xdb,0xa6,0x04,0xe0,0x4d,0xf1, +0xb9,0x89,0x77,0xe7,0xb6,0xa3,0x78,0xbb,0x62,0x29,0x0a,0xe9, +0x9a,0xe7,0xa1,0xfa,0xd1,0x7f,0xe4,0x4e,0x71,0xef,0x4a,0x95, +0x30,0x0e,0x42,0xb3,0x7d,0xa2,0x9e,0xcb,0xe1,0x04,0xbe,0x50, +0xad,0xd1,0x7a,0x46,0x8c,0x79,0xeb,0x7b,0xd4,0x90,0xc3,0xbc, +0x48,0x36,0x17,0x6f,0xba,0x3e,0x8e,0x9a,0x1b,0x00,0xfb,0x78, +0xeb,0xd2,0xc2,0xd7,0x44,0x85,0xb6,0x8d,0x41,0x16,0x6c,0xa5, +0xa8,0xe9,0xca,0xfd,0x6b,0xae,0xd9,0x37,0x35,0xdb,0xba,0xf8, +0xd2,0x6d,0x97,0x75,0xba,0xf5,0x72,0x7d,0x48,0xbe,0xca,0xb0, +0xc7,0xbf,0x36,0xdf,0xc6,0x5f,0xd6,0x7c,0xc5,0x89,0xa9,0x64, +0x23,0x2e,0x56,0x2d,0xe6,0xb3,0x0a,0x4f,0x0b,0x00,0x73,0x33, +0x1e,0xa2,0x37,0x43,0x84,0x70,0x35,0xb9,0x76,0xd6,0xf1,0xeb, +0xc7,0xe1,0x7c,0x38,0x0a,0xdc,0x12,0x61,0x44,0x4b,0x0f,0x30, +0xb1,0x92,0xcf,0x40,0xd0,0xae,0x2a,0xe1,0x7f,0x5a,0x23,0x93, +0x37,0xf5,0x3e,0xd3,0x1e,0xd9,0xd2,0x8d,0x7b,0xd7,0x8d,0x5e, +0xea,0xd7,0x0c,0xe1,0xe5,0xa2,0x9a,0x1b,0xc5,0xce,0xdc,0x0c, +0x99,0x48,0xab,0x9c,0x15,0xa7,0x99,0xca,0xb2,0x62,0xdb,0x74, +0x1a,0x51,0x92,0x7f,0x76,0x66,0x1f,0xea,0xe5,0xe6,0x6b,0xf2, +0x78,0x3c,0xd1,0xcb,0x91,0xa7,0xc3,0xfc,0xc2,0x09,0x6c,0x5c, +0x38,0x4a,0x6a,0x6f,0xe8,0x82,0x86,0x8a,0xad,0x71,0x84,0x3e, +0xde,0x0c,0x57,0x64,0xb0,0x6c,0x32,0xbf,0xab,0x9a,0x78,0x41, +0x4e,0x84,0x14,0xee,0xe8,0x5a,0xbe,0x2d,0x57,0xe6,0xed,0xdd, +0x74,0x35,0x19,0x91,0xb4,0xf8,0x4b,0x0c,0x2e,0xa6,0x63,0x23, +0x08,0xdf,0xea,0xc3,0x0a,0xc9,0xf1,0x3a,0x1c,0x40,0x83,0x7e, +0x45,0xf9,0x70,0xbc,0x52,0xef,0x1f,0x6d,0x5a,0x24,0xe7,0xf1, +0x2d,0x3a,0x59,0x88,0x6b,0x8a,0x2d,0x77,0xd0,0x27,0x35,0xde, +0x32,0x2d,0xcc,0x53,0xba,0x21,0xdc,0xbf,0x46,0x31,0x61,0x05, +0x79,0xd4,0xfe,0x92,0xcb,0x09,0x87,0x6c,0x20,0xea,0x85,0x99, +0xc8,0x8d,0x5c,0x90,0xc7,0xd8,0x17,0x11,0x72,0xf1,0x5e,0x47, +0xbe,0x38,0xc8,0x63,0x7b,0x2b,0x91,0x94,0x37,0xad,0x55,0x23, +0x9b,0xc4,0x52,0xe9,0x6a,0x56,0x80,0x34,0x4d,0x51,0x5a,0x4e, +0x64,0x87,0x54,0xe6,0x42,0xa2,0x13,0x12,0xd5,0x3b,0xd2,0x4c, +0x5c,0x79,0xd0,0xb6,0xe8,0x8d,0xaf,0x19,0x8a,0xab,0x25,0xa3, +0x77,0x3f,0x07,0x3c,0x14,0xa5,0x0a,0xe0,0xd6,0xc4,0xef,0x66, +0x72,0xef,0x38,0xb5,0x0c,0x95,0xbf,0xdb,0x04,0xe4,0x8d,0xc7, +0x56,0x51,0x56,0x00,0x86,0xd3,0xab,0x39,0x38,0x75,0x0d,0xd6, +0xc5,0x91,0x15,0x9d,0x7c,0xc2,0xbd,0xfa,0x23,0x6b,0x28,0xd5, +0xb3,0xad,0xd7,0xc9,0xf3,0x97,0x7f,0x05,0x62,0x36,0xd2,0xa7, +0x83,0xe3,0x3f,0x01,0x1a,0x6b,0x5b,0x99,0x19,0x5a,0x09,0x69, +0x4c,0xcc,0x7a,0x84,0x55,0xbf,0x7d,0x7a,0xfa,0xfa,0xa7,0xb7, +0x27,0x4f,0xdf,0xfd,0xf4,0xf6,0x45,0xc1,0x26,0xe2,0x82,0x25, +0xf5,0x7b,0xf5,0x96,0x9c,0x74,0xe6,0x03,0xab,0x4b,0xc9,0x67, +0xe2,0xe8,0x86,0xb3,0x88,0x9b,0x8f,0x77,0xd5,0x82,0xc9,0xf1, +0x19,0x31,0x3c,0xfe,0x6f,0xee,0xde,0x7e,0xbf,0x8d,0xdc,0x58, +0x14,0xfc,0xfb,0xde,0xfb,0x12,0x54,0x1f,0x85,0xee,0x1e,0x42, +0x94,0x28,0x4f,0x72,0x92,0xe6,0xb4,0x79,0x34,0xb2,0x1c,0x3b, +0xb1,0x2c,0x1d,0x49,0x93,0x93,0x44,0xe2,0x38,0x94,0xd8,0x92, +0x68,0x53,0xa4,0x86,0x1f,0xb2,0x1d,0x91,0xef,0xb2,0x7f,0xec, +0x93,0xdc,0x27,0xdb,0xfa,0x02,0x50,0xe8,0x6e,0x4a,0xb2,0x93, +0xbd,0x39,0xbb,0x33,0x3f,0x8b,0x8d,0xaf,0x42,0x01,0x28,0x14, +0x0a,0x40,0xa1,0x2a,0x3a,0xde,0x7b,0xbb,0xb7,0x7b,0x42,0x11, +0x61,0xfb,0x57,0x4c,0xcc,0x69,0x3e,0x24,0x28,0x30,0x31,0x45, +0x0f,0x13,0xf9,0xf7,0xec,0x1a,0xc4,0x58,0x0b,0xfd,0x99,0x7f, +0xd3,0xc1,0x0a,0x7c,0x33,0xeb,0x05,0x36,0xef,0x37,0x6b,0xc2, +0x94,0x84,0xc9,0xeb,0xa9,0xc7,0xd5,0xb6,0x57,0x4f,0x05,0x34, +0xf0,0xfd,0xc8,0x64,0xd0,0x0a,0x26,0xe2,0x69,0xd8,0xab,0x6d, +0xf2,0x83,0x41,0x15,0x01,0x5b,0xec,0x62,0x54,0x76,0xbf,0xa4, +0xde,0x5b,0x69,0xe5,0x4a,0xde,0x0d,0xd2,0xc9,0x58,0x75,0x17, +0x41,0x3b,0xc6,0x37,0xec,0x2b,0x9b,0xd4,0xd6,0xec,0x30,0xa0, +0xf9,0x43,0xe2,0x53,0xaf,0x4f,0xf6,0xdf,0xd6,0x5e,0x1e,0xec, +0xb3,0x47,0x69,0xdf,0x5b,0xd3,0x1a,0x4c,0xae,0x1a,0xba,0x1b, +0x19,0x92,0x91,0xd7,0x66,0xad,0x76,0x38,0xcc,0xf1,0x65,0x13, +0x5a,0x7a,0xc5,0x3e,0x86,0xb9,0x55,0x43,0x2c,0x09,0x54,0x0c, +0xbb,0xaf,0x6b,0xf4,0x82,0x06,0xb1,0x17,0xc0,0x8c,0x3f,0x92, +0x71,0x14,0x58,0x82,0x6b,0xe8,0x58,0x7a,0x44,0x51,0x89,0x8d, +0x6b,0x46,0x05,0x9e,0x53,0x56,0x41,0x15,0x7b,0x52,0x6b,0x5b, +0x66,0xc5,0x1c,0xe3,0x46,0x9b,0x02,0xeb,0x52,0x65,0xb3,0x20, +0xc9,0xae,0xad,0xb1,0xd6,0x9c,0xa5,0x7c,0x68,0xb3,0xa5,0x18, +0x97,0x9d,0x76,0x93,0x84,0x2c,0xe4,0x03,0x08,0x3c,0x03,0x2e, +0x0d,0x4d,0xbd,0x5e,0x8c,0x11,0x1b,0x48,0xa2,0x83,0x2b,0xea, +0xc5,0xc9,0xd3,0xd8,0x0d,0xc3,0x9a,0xe6,0x81,0x67,0x57,0xf4, +0x6c,0x90,0x04,0x4e,0x48,0x57,0xee,0x95,0x84,0xd2,0x60,0x47, +0x70,0x44,0xe6,0x06,0xf1,0x4a,0xe6,0x9d,0x63,0xc5,0xa4,0xa5, +0x7e,0x89,0xbe,0x49,0x24,0xbf,0xcd,0x96,0x15,0xcb,0xf1,0x05, +0x32,0x7e,0xb1,0x1b,0x80,0x62,0xba,0xc8,0xd7,0xa2,0xaf,0x99, +0x55,0xc1,0x14,0x66,0x8f,0xb5,0x23,0xe9,0x06,0xaf,0xc8,0x95, +0x89,0x0b,0x1d,0x5f,0x3a,0x8a,0x2d,0x94,0xc3,0x9d,0x44,0x65, +0x5d,0xe2,0x77,0xc4,0x65,0x6b,0x34,0xba,0x99,0xb4,0xdc,0xbd, +0x28,0xfe,0x90,0x0d,0xcc,0x87,0xed,0xec,0x43,0x43,0xb5,0x6b, +0xa3,0x85,0x8f,0x85,0xab,0x50,0xb0,0x4f,0x85,0xa1,0x08,0xe0, +0x81,0xa1,0xed,0x4e,0x50,0xc7,0x87,0x6e,0x16,0x86,0xb7,0xbb, +0xa9,0x98,0x59,0x2c,0xe4,0x6b,0x57,0xc0,0xdf,0xc8,0x02,0x2c, +0x58,0xe3,0x74,0xc9,0x3d,0x56,0xaf,0xcb,0x72,0x26,0xa3,0x4c, +0xf2,0x56,0x2c,0xcd,0xa9,0x1c,0x3e,0xbe,0x3c,0xbc,0x9c,0xf4, +0xae,0x42,0x07,0x01,0xac,0x66,0x60,0x0d,0x46,0xbf,0x92,0xf4, +0x18,0x36,0xa1,0xf2,0x89,0x6e,0x52,0xf2,0x51,0x9f,0x6b,0xa8, +0x84,0x6c,0x09,0xe8,0x94,0xf7,0x0d,0xcd,0xfc,0xf3,0x6d,0x6f, +0xd4,0x1f,0x77,0x2b,0x87,0xa1,0x98,0xc9,0xf5,0xfd,0xc7,0xac, +0x65,0x3e,0x7e,0x5c,0x45,0x45,0xed,0x8f,0x1f,0x5f,0x7c,0x6c, +0x7f,0xb4,0xef,0xb2,0xed,0xd1,0x6f,0x89,0x26,0x3f,0x76,0xdb, +0xb2,0x7b,0xb1,0xf2,0x90,0xb5,0x52,0x8a,0x86,0x71,0x2b,0x5a, +0x64,0xb3,0x39,0x6f,0xfa,0x65,0x88,0xcb,0x0a,0xca,0xb7,0xa4, +0x53,0x9a,0x4d,0x82,0x6f,0xd6,0x22,0x13,0x98,0xfe,0xd0,0x20, +0x2b,0x3a,0x2f,0x12,0x4e,0xcf,0x86,0xb9,0xd6,0x0b,0x0d,0x12, +0x7b,0x5d,0x98,0x85,0x78,0x3f,0x6a,0xfa,0x2c,0xdb,0x1e,0x9a, +0x72,0x4d,0x77,0xbf,0xee,0x0e,0x9e,0xfc,0x4d,0xbe,0x5b,0xd8, +0xcd,0xba,0xb5,0x9e,0x93,0x2a,0x43,0x22,0x10,0x04,0x41,0x25, +0xb9,0xb7,0x5f,0x68,0x01,0x88,0xbf,0x04,0xfd,0x17,0x5b,0xf5, +0xba,0xb5,0x88,0xe5,0xed,0xef,0x84,0xb8,0x1a,0x07,0x67,0x09, +0xdb,0x30,0x26,0xd5,0x7f,0xac,0x22,0x05,0x64,0x65,0x5d,0xff, +0x13,0x2a,0x9b,0xe6,0x55,0xf6,0x72,0xcd,0x27,0x7c,0x39,0x85, +0x7d,0xe4,0x1e,0x70,0xa8,0x67,0x75,0xe4,0x90,0xf3,0x25,0x99, +0x84,0xce,0x47,0xe8,0xbb,0x75,0x36,0x69,0xc1,0x22,0x3e,0xd9, +0x56,0x0f,0xc4,0x48,0x89,0x91,0x32,0x4e,0x5b,0x19,0x66,0x10, +0x33,0xf7,0x9b,0x67,0xd3,0xc6,0x66,0x22,0x29,0xdb,0x98,0xb2, +0x1d,0xa4,0xa0,0x83,0x9f,0x7c,0x92,0x3a,0x6f,0xc6,0x64,0xf6, +0x43,0xe0,0x68,0xab,0x1f,0xf7,0x36,0x07,0xa5,0x65,0x92,0x03, +0xef,0x8c,0x3e,0x40,0x91,0x0f,0x52,0x64,0xdb,0xf1,0x17,0xe6, +0x6e,0x9c,0x59,0x72,0x6f,0x03,0xb3,0x48,0x70,0x5b,0x34,0x18, +0xcd,0xf3,0x1a,0xd5,0xdb,0x66,0xdc,0x59,0xe6,0xa0,0x5c,0xee, +0xea,0x85,0x53,0xf0,0x65,0xb7,0xbe,0x65,0x90,0x27,0x52,0x2c, +0xa5,0x82,0x44,0x18,0xb1,0xed,0xae,0xf9,0x84,0xf8,0x74,0xd8, +0xf1,0x72,0x21,0xc0,0xa5,0x13,0xca,0xcc,0x19,0x82,0xb1,0x2a, +0xf4,0xad,0x80,0x92,0xa5,0xc9,0x3a,0x8c,0x8b,0xc4,0x5a,0x5f, +0x73,0xdd,0x13,0x54,0xa1,0x20,0x8f,0xa3,0x14,0xd7,0x05,0xe9, +0x60,0x4d,0x9e,0xff,0x5a,0x91,0x15,0xa8,0xc1,0x58,0xd1,0xcf, +0x9c,0xf3,0xf3,0xfb,0x3f,0xe6,0x5f,0xb2,0x8a,0xc7,0xf8,0x61, +0x9b,0x70,0xe1,0x22,0x77,0xab,0xbe,0x10,0x08,0x03,0x85,0x76, +0xc3,0x04,0xbb,0x55,0xb6,0x98,0xfd,0x9b,0x20,0x5f,0x88,0x9b, +0xa3,0xef,0x39,0x6c,0xa6,0x8e,0x9f,0xbe,0x9c,0x6c,0x13,0x52, +0xf7,0x88,0x27,0x2b,0x64,0x71,0x65,0x41,0xc4,0x58,0x55,0x3a, +0x9b,0x8e,0x7a,0x1f,0xf3,0xf7,0xe4,0xe1,0x0c,0x51,0x8b,0x36, +0xd0,0xb1,0x89,0xeb,0x04,0x25,0x08,0x87,0x8d,0x01,0x49,0x26, +0xda,0xd1,0x92,0xb2,0x37,0x7c,0x80,0xf4,0x20,0x1b,0x89,0x30, +0x9d,0x77,0x10,0x44,0x2e,0xd0,0x39,0xa1,0x5d,0x0b,0xb2,0x38, +0xa1,0xc7,0x21,0xf3,0x86,0xf4,0xf0,0xd6,0x72,0x3c,0xbc,0xb3, +0xbe,0x05,0x0c,0x19,0x09,0x08,0x32,0xe3,0xfe,0x44,0xd5,0x5e, +0xaf,0xaf,0x05,0xc9,0xf2,0x2a,0x6b,0xb5,0x79,0x45,0xa4,0x42, +0x8f,0xdd,0x8a,0xe2,0x0f,0x18,0x43,0x44,0xcb,0x16,0x71,0x61, +0xe4,0xb2,0x68,0x3e,0xc2,0xdd,0x62,0x1a,0x35,0x02,0x68,0xd0, +0xbb,0x8e,0xab,0x40,0x3b,0xd6,0x5a,0xd8,0xf2,0xf9,0x70,0x98, +0x65,0x76,0xf6,0x94,0xbd,0x3f,0x14,0x08,0x49,0x6c,0x3a,0xe3, +0x34,0x72,0x6c,0x29,0xad,0x9a,0x64,0xee,0x79,0x97,0x75,0xe5, +0x5f,0xdc,0x04,0x48,0x21,0x17,0xd1,0x0e,0x44,0x4c,0x77,0x10, +0xab,0xa4,0x4c,0x22,0xac,0xc0,0x2f,0x11,0xbb,0x3e,0x93,0xc1, +0x79,0xd4,0x8e,0x27,0xf0,0x5a,0x01,0x16,0xf2,0xdb,0x4b,0xfd, +0xc2,0x91,0x1d,0x69,0x15,0x77,0x2b,0xd3,0x8a,0xed,0xca,0xb4, +0xb8,0x5f,0x31,0x43,0xe7,0xf5,0xa6,0x88,0x76,0x41,0xd2,0xa6, +0xa1,0x02,0x41,0xdb,0x5b,0xcd,0x09,0x9c,0xc3,0xa0,0x93,0xa4, +0x6a,0xb7,0x72,0xea,0xe8,0xca,0x97,0x11,0x61,0x7d,0xb1,0x40, +0x17,0xb6,0xa4,0x3a,0x29,0x97,0x91,0xe8,0x69,0x89,0x9d,0xbc, +0xd0,0xfe,0xee,0xf8,0xcb,0xcd,0xf9,0x78,0x98,0xc5,0xeb,0xca, +0x35,0x98,0x48,0x4c,0x22,0xd6,0xc4,0x11,0x3e,0xd3,0xd5,0x3b, +0x92,0xa6,0x2a,0x89,0x56,0x49,0x40,0xd2,0x10,0x30,0x41,0x2e, +0x17,0x4d,0x77,0x7f,0xa5,0xbb,0xa3,0x2c,0xba,0xbf,0x8f,0xc8, +0xf4,0xb8,0x85,0x05,0x54,0xbf,0x5c,0x46,0xe8,0x32,0x40,0x0a, +0x76,0x06,0x50,0x6e,0x86,0xdb,0x4b,0x6f,0x40,0xcc,0xaa,0x00, +0xda,0x33,0x31,0x77,0x01,0xe4,0xbc,0xdb,0x9c,0xdd,0x9f,0xdd, +0x6f,0x5e,0x19,0x05,0x38,0xf1,0x89,0xcb,0x25,0x24,0xb9,0x0a, +0x60,0xf8,0x0b,0xa6,0x67,0xb2,0xcd,0x9f,0xd9,0x18,0xcc,0xe9, +0xce,0xc6,0x5f,0xbb,0x9b,0xce,0xae,0x37,0x6f,0x23,0xb5,0x95, +0xea,0xb2,0x24,0x12,0xba,0x90,0xbd,0x80,0xc0,0x90,0x0c,0xb7, +0x90,0x25,0x17,0x8b,0xc1,0xe1,0xd1,0xde,0xab,0x37,0x7f,0x96, +0x0d,0x2b,0x6a,0xc9,0x25,0x81,0xe1,0x6b,0x7b,0x1a,0x58,0x70, +0x82,0xa9,0x5e,0xf0,0xa3,0xc5,0xeb,0xdd,0x77,0x27,0x47,0xf4, +0xa0,0x13,0xed,0x3a,0x9f,0x1d,0x37,0x92,0x18,0x56,0xe9,0xde, +0x14,0xad,0x57,0x9f,0x7d,0x42,0x53,0xd4,0xd6,0xae,0xb3,0xb5, +0x27,0x5b,0x70,0xc3,0x73,0xe1,0x9d,0xf5,0x26,0xf7,0xde,0x02, +0x1d,0x9d,0x90,0xc8,0x8d,0xac,0xaa,0x91,0x37,0x97,0xa9,0x8a, +0x91,0x6d,0xa1,0x02,0xf3,0x80,0xfd,0x55,0xeb,0x21,0xa3,0xca, +0xfc,0xa9,0x78,0x58,0x2a,0x19,0x49,0xc8,0xdd,0x3b,0xc4,0x6a, +0x8f,0xc7,0x62,0xdf,0x49,0xf9,0x1c,0x66,0x52,0xb9,0x1c,0xf0, +0x7b,0x63,0xf7,0xc2,0xdd,0x03,0x22,0xbf,0xff,0xa4,0xde,0xe1, +0xe3,0x84,0x95,0xba,0xee,0x4c,0x34,0x48,0xf7,0xd6,0x55,0xc1, +0x76,0xef,0x65,0x8d,0x07,0xa2,0x14,0xd7,0x4b,0x5a,0x3a,0xba, +0xa3,0x3b,0xba,0x03,0x55,0x02,0xf9,0x36,0x9f,0xa1,0x0d,0x09, +0xfd,0x40,0x2a,0x68,0xdc,0x1a,0xea,0x22,0x48,0x2e,0xeb,0x0d, +0xa4,0x90,0x6e,0x9c,0x41,0xd0,0x57,0x41,0xf7,0x15,0xb2,0x79, +0x2f,0xac,0x81,0xbd,0x69,0xef,0x65,0xba,0xdc,0xf3,0xaa,0xf9, +0x74,0x54,0xb5,0xc6,0xf1,0x30,0x5b,0xc7,0x44,0x37,0xde,0x91, +0x7c,0xd0,0x80,0x92,0xbf,0x0b,0xf5,0x34,0x1f,0x0f,0x62,0xf8, +0xe5,0xe0,0x2e,0x39,0x63,0xae,0x41,0xad,0xe3,0xc9,0x4c,0x91, +0xb9,0xdc,0xbc,0xf6,0xa6,0xfc,0xda,0x74,0xad,0xf6,0x6e,0x5c, +0x63,0xb8,0x35,0xae,0xd5,0xfa,0xc6,0xed,0xd7,0xee,0x06,0xbd, +0xda,0xdf,0xb8,0xea,0xbf,0x35,0x23,0xdd,0xe0,0xc5,0x42,0x0d, +0x35,0x11,0xbe,0x6a,0x4a,0x3b,0x7c,0x8f,0xe6,0x53,0xba,0x59, +0xc1,0xdd,0xb4,0x23,0xbb,0x65,0x60,0xa5,0xde,0xee,0x28,0x1f, +0xf3,0x0c,0xe3,0x88,0xba,0x40,0xec,0x81,0x93,0x31,0xe7,0x99, +0x34,0xb4,0x84,0xbf,0x57,0x58,0xb4,0x56,0xd5,0x45,0xbe,0x67, +0xfc,0x04,0x83,0x60,0x79,0x4e,0x69,0x87,0x9a,0xe2,0x19,0x94, +0xfc,0xc8,0x38,0x7f,0xa0,0x49,0xd0,0x3e,0x7a,0x5d,0xfa,0x9a, +0x6f,0x8d,0x62,0xb9,0x3d,0xe2,0x59,0x28,0x32,0xa3,0x19,0x18, +0xca,0xd3,0x57,0xfe,0x0c,0x24,0x5f,0x27,0x76,0x56,0xd1,0x39, +0xc2,0x7a,0xd3,0x3a,0x1b,0x45,0x89,0x76,0x65,0x39,0x42,0xf5, +0x00,0xbc,0x2b,0x54,0x1e,0xb1,0x52,0xc8,0x02,0x55,0x28,0x5f, +0xb7,0xa4,0x24,0x40,0x99,0x9c,0x19,0xad,0x41,0xc2,0x76,0x0d, +0xb2,0x52,0x1a,0x6b,0xef,0x92,0xdc,0x14,0x33,0x7e,0xb4,0xf4, +0x75,0xd4,0x77,0x23,0x8b,0x4c,0x8d,0xf4,0x7e,0x52,0x15,0x9b, +0x91,0x57,0xcf,0x44,0x1a,0x95,0x48,0x92,0xef,0x10,0x69,0xca, +0xef,0x79,0x1a,0x06,0x3d,0x22,0x01,0xe0,0xa3,0x99,0xe3,0xa6, +0x36,0x43,0x47,0x3e,0xbc,0xe5,0xc2,0x92,0x85,0xdf,0x60,0xdd, +0xf0,0xb0,0xd0,0xe0,0xbd,0x07,0x5c,0x39,0x1e,0x09,0xf1,0xe6, +0x8e,0xcf,0x76,0xea,0x7b,0x8d,0xa0,0x82,0x94,0x41,0x16,0x04, +0x7d,0x0e,0x75,0x80,0x46,0xda,0x33,0xe8,0x0d,0x00,0x6d,0x71, +0xc4,0x78,0x6f,0xe0,0xae,0x09,0x2f,0x47,0x53,0x65,0xb2,0xfb, +0x95,0x17,0xb0,0xa0,0x41,0xf0,0x27,0xc8,0x9c,0xa4,0x6e,0xb8, +0x2f,0xb5,0xfb,0x4a,0x94,0xa1,0xc8,0x2b,0x07,0xc8,0x20,0x41, +0x7e,0x71,0x46,0xa3,0x56,0x3e,0xe0,0xd7,0xa2,0xae,0x82,0xbe, +0x36,0xe6,0xbe,0x6e,0x0e,0xbe,0xc8,0xb6,0xb7,0x60,0x1b,0xfd, +0x7c,0x6b,0xeb,0x05,0x47,0xa8,0x29,0xf2,0x7a,0x36,0xbb,0x2d, +0xac,0x96,0x7f,0x38,0x3e,0x78,0xf7,0xfe,0xf8,0x64,0xe7,0xe8, +0x44,0x2c,0x37,0x9c,0x9d,0x2e,0xce,0xee,0x4f,0x7f,0x3e,0xbb, +0xef,0x26,0x9b,0x86,0x52,0xf7,0xde,0xbd,0xcc,0x36,0x4f,0xcf, +0x96,0x67,0xdd,0x2e,0xdb,0x70,0x38,0x3c,0x3a,0x38,0x41,0x6b, +0x54,0x90,0xc6,0x4b,0x34,0x16,0x4d,0xce,0xba,0x67,0xcb,0x67, +0xa6,0x73,0x36,0xda,0x34,0xbb,0x07,0xef,0x4e,0xf0,0xdc,0xf9, +0xe4,0x2f,0x87,0x7b,0xef,0x77,0x0e,0x0f,0xdf,0xbe,0xd9,0xdd, +0xa1,0xec,0x08,0x2f,0xbb,0xa7,0x77,0xb1,0x30,0x8b,0x36,0x50, +0xd9,0x04,0x68,0x18,0x27,0xd8,0x80,0x95,0x19,0x37,0x3f,0x4c, +0xc7,0xa3,0x36,0x6a,0x95,0xc1,0xb6,0x3e,0x9b,0xcf,0x2e,0x37, +0x7e,0x1b,0xa1,0x0b,0x13,0x7a,0x58,0x2d,0x42,0xaf,0x0b,0xdd, +0xbb,0x41,0x39,0x92,0xfb,0xdb,0xf4,0xd4,0x3f,0x4c,0xc6,0x5e, +0x2b,0x39,0x5c,0xa0,0x58,0x54,0x90,0xc4,0xbe,0xc6,0x3f,0x4a, +0xd8,0x28,0x34,0x0a,0x05,0x0e,0xe3,0xbb,0x87,0x0f,0xcc,0xa5, +0xbc,0xed,0x97,0x20,0x92,0x81,0x5e,0x4e,0xc6,0x37,0x7f,0x98, +0x5a,0x04,0x12,0x1e,0xbe,0x65,0xd7,0x28,0x5c,0xc9,0x89,0x96, +0x46,0x55,0xe1,0x69,0x1f,0xbd,0x01,0xbc,0x35,0x20,0x26,0xb2, +0xc3,0x94,0x74,0x66,0x63,0x06,0x99,0xa4,0x7d,0x00,0x65,0x67, +0x09,0xba,0x30,0xb8,0x19,0x8f,0xd2,0xfb,0x9d,0x0b,0x64,0x7d, +0xe5,0x8e,0x34,0x35,0x3c,0xf5,0xde,0x84,0x06,0x0e,0xe0,0xfb, +0xbb,0xcd,0xef,0xa0,0x2f,0x51,0x1f,0x3a,0x7d,0x70,0x80,0x0c, +0x7a,0x20,0x7a,0x24,0x07,0x2e,0xed,0x0f,0xe7,0x59,0x9a,0xcf, +0xd3,0xc9,0x25,0x3b,0x84,0xa3,0xcd,0x6e,0xf4,0xe7,0xe3,0xa3, +0x57,0x1b,0x27,0x07,0x7f,0xdc,0x7b,0x17,0x51,0x1a,0x4f,0x50, +0x49,0xdb,0x50,0xa9,0x4b,0x3e,0x9b,0xc7,0x36,0x8d,0x27,0xec, +0x0b,0x00,0xbd,0x09,0xb2,0x1d,0x7d,0x9f,0x42,0x07,0x34,0xf6, +0xe6,0xfe,0xcd,0xca,0x12,0x15,0x39,0xc8,0x27,0x45,0xb0,0x28, +0xa0,0x2a,0xd9,0x8f,0xbd,0x8b,0x8f,0x64,0x8a,0x30,0x5a,0x3f, +0x67,0x77,0x62,0x64,0xcd,0x3e,0x70,0xda,0x52,0xb0,0x67,0xff, +0x0b,0x1b,0xcd,0xf7,0x02,0x9c,0xb2,0x66,0xef,0x97,0x18,0x05, +0xdc,0x58,0xd0,0x26,0x00,0x1c,0x58,0xa5,0xff,0xc5,0x28,0xa9, +0xcf,0x01,0x54,0xa7,0x57,0x04,0x90,0xec,0xd7,0x90,0xfb,0x3a, +0x56,0x52,0x28,0x73,0x2a,0x3b,0x29,0x9c,0x76,0x83,0xf3,0xab, +0x7e,0xab,0x74,0x09,0x9d,0xea,0x03,0xf1,0x9f,0x34,0xe4,0x73, +0x36,0x91,0x2e,0x4a,0x5d,0xd6,0x66,0xa8,0x20,0xd1,0x2c,0xd5, +0xa8,0x9c,0xe3,0x7b,0x6e,0xe5,0x4a,0x0b,0xdb,0xea,0x60,0x44, +0xba,0xfe,0x0b,0x0c,0x10,0x91,0x3c,0x06,0x93,0x82,0x66,0xb0, +0xe5,0xe1,0x17,0xc5,0x46,0xa2,0xf9,0xbb,0xca,0x05,0x97,0x03, +0xc2,0x60,0xda,0x85,0xa5,0xd5,0x8f,0x09,0x47,0xbc,0x1a,0xc9, +0x5c,0x42,0xc9,0xfe,0x55,0x31,0x0d,0xa7,0x74,0x00,0x2e,0xb3, +0x29,0xb0,0x4b,0xc3,0x95,0x62,0x2d,0x0b,0x92,0xed,0xd2,0x72, +0xca,0xbf,0xdd,0x30,0xd5,0x9e,0xf5,0x17,0x32,0xe1,0x35,0x0d, +0x69,0xa5,0xe7,0x6a,0x3e,0x18,0xb7,0x32,0xbd,0x0c,0xa2,0x61, +0xc8,0x55,0xc8,0x15,0x99,0x66,0x96,0x25,0xba,0x91,0x71,0x39, +0xb5,0xaf,0x33,0x19,0x2e,0xdb,0x5f,0x6d,0x05,0xc0,0x67,0xf2, +0x91,0x4d,0x66,0x2f,0x2a,0x46,0xad,0x98,0x02,0xeb,0x86,0x2c, +0x4b,0x25,0x5d,0xd8,0xd9,0xaa,0x11,0xf1,0x25,0xc2,0x78,0x8f, +0x55,0x62,0x8d,0x65,0x48,0xf8,0x0d,0xcc,0x4d,0xe2,0x5b,0x74, +0x6e,0x1a,0x74,0x06,0x39,0x23,0xf3,0x00,0xef,0xab,0xfb,0x46, +0x89,0x40,0x41,0x69,0xd6,0x9f,0x0b,0x3a,0x0e,0x01,0x2a,0x4c, +0xd0,0xdd,0x98,0x2b,0x1b,0x64,0x4c,0xd0,0xf2,0x52,0x65,0x6d, +0xfe,0xd0,0x75,0x45,0x33,0xda,0xbe,0x82,0xd3,0x00,0x9f,0x6e, +0xa6,0xba,0x33,0x4c,0xf1,0x9e,0x83,0x6d,0xd1,0xa5,0xec,0x67, +0xa1,0xa7,0x83,0xc5,0x8e,0x17,0x10,0x37,0xe8,0xc5,0x14,0x53, +0x5e,0x17,0xab,0xf2,0x72,0xd2,0xd2,0x2e,0x27,0x59,0x30,0xe3, +0x42,0xee,0xd2,0xf6,0x3b,0x5d,0x08,0x9a,0x30,0xb1,0x40,0x57, +0x59,0x81,0x3b,0x30,0x89,0x64,0x73,0x34,0xfc,0x56,0x41,0x3a, +0x74,0xbe,0x82,0xcb,0x01,0x2b,0x4b,0xa8,0xb3,0x43,0x60,0x1d, +0xd0,0x2d,0x07,0xf4,0x60,0x3f,0x56,0x6a,0x54,0x1d,0xcb,0x43, +0xad,0xe3,0xdc,0x38,0x39,0x95,0xe4,0x70,0xc9,0x59,0x2c,0x5c, +0xb3,0xc3,0x84,0xae,0x12,0x30,0x5d,0xd5,0x6e,0xb6,0x4f,0x35, +0x34,0x3f,0x3e,0x05,0x68,0x7a,0x48,0x1d,0x0c,0x71,0x35,0x48, +0xc7,0x54,0xd6,0x9b,0xba,0x56,0x2a,0x26,0x26,0x66,0xfb,0x29, +0xec,0xb6,0x36,0xf3,0xe6,0x5f,0x90,0xef,0x66,0x21,0x17,0x96, +0x8c,0x5a,0x2e,0x2c,0x48,0xd6,0x15,0x9c,0x98,0x2a,0x57,0x6c, +0xd8,0xfb,0xfb,0x53,0xe0,0x12,0x7f,0x52,0x58,0x62,0x91,0x7c, +0x88,0x6d,0xf9,0x63,0x24,0xfa,0xa8,0x1b,0xb8,0x7f,0x8d,0xf4, +0xc4,0x90,0x92,0x64,0x66,0xb2,0x8a,0xc3,0x2d,0x43,0x6f,0x83, +0x52,0x3b,0xda,0x49,0xda,0x9d,0xe4,0xb4,0x93,0xc4,0x9d,0x33, +0x89,0x3b,0x3e,0x97,0xeb,0xea,0x72,0xbe,0x15,0x10,0xb2,0x95, +0x45,0xcc,0x14,0x48,0x17,0x3a,0x44,0x51,0xef,0x4b,0x2d,0x62, +0x37,0x67,0xd7,0xb9,0x58,0xbc,0xd7,0x13,0xa3,0x3c,0x8b,0x12, +0x60,0xa1,0xd7,0x20,0x4a,0x65,0xa7,0xc1,0x08,0x7b,0xb3,0x23, +0x5d,0x03,0x3b,0xed,0x9b,0xc1,0x14,0xc8,0xf8,0x97,0xe6,0x27, +0x84,0x2a,0x63,0x4e,0x4c,0xc8,0x1b,0xab,0x23,0x37,0x46,0x7d, +0x2d,0x90,0x28,0xc5,0x40,0x1f,0x9b,0xdc,0xeb,0x50,0x53,0x66, +0xdd,0x62,0x51,0x11,0xb9,0x87,0x3b,0xd5,0x04,0xdf,0x70,0x01, +0x7e,0x4e,0x47,0xb8,0x22,0xa3,0x59,0x59,0xd8,0x14,0xb2,0x73, +0x9b,0x8b,0xb5,0x71,0x6c,0x58,0x9d,0xd7,0xb6,0x2a,0x64,0x33, +0xab,0xcb,0x82,0x84,0xc0,0x85,0x03,0xbd,0x5b,0x1c,0x09,0x54, +0xe8,0xa5,0x14,0xab,0x0c,0xcb,0xf2,0x41,0x31,0xba,0x6d,0xbb, +0x5a,0x7e,0x65,0x14,0x09,0x80,0x2b,0xe2,0xee,0xbe,0x6c,0x26, +0xd1,0xf4,0xaf,0x74,0xcf,0x1f,0x00,0x2a,0xa9,0x86,0x26,0x78, +0x52,0xbe,0x42,0x1e,0x62,0x89,0x66,0x95,0x7c,0x44,0x1b,0x65, +0x86,0xbd,0xb4,0x1f,0x7c,0xb4,0xf0,0x38,0x1a,0xf4,0x3c,0xe3, +0xff,0x65,0x5c,0xd4,0x9b,0x1b,0x3a,0xcc,0x3e,0xbe,0x1e,0x4f, +0x66,0x6c,0x32,0x72,0x1a,0xd3,0x35,0x25,0x91,0xad,0x3b,0xfb, +0x30,0x85,0xcd,0x38,0x89,0xa3,0x72,0xcc,0xe9,0x92,0xd0,0x41, +0xbe,0x65,0x78,0xf6,0x15,0x07,0x89,0xad,0xc1,0x4a,0x42,0x36, +0x2b,0xee,0x79,0x25,0x48,0xe9,0xd4,0x09,0xca,0xa5,0xf0,0x0f, +0x35,0x58,0x82,0xd7,0x40,0x25,0xcc,0x50,0x95,0x85,0xb8,0xe3, +0x3f,0x82,0x21,0x75,0xdd,0x3f,0x80,0x26,0x01,0x48,0x69,0x87, +0x57,0x40,0x78,0x05,0xcf,0x51,0x72,0x87,0x17,0x66,0xfb,0xa8, +0x8f,0x5d,0x18,0x37,0xcb,0x9d,0xc4,0x4d,0x3b,0xbf,0x5c,0x01, +0xe6,0x57,0xda,0xfd,0x77,0xd8,0xc5,0x25,0xec,0xd9,0xa8,0x45, +0xa7,0x45,0x38,0x55,0x67,0x21,0x02,0xb4,0x9b,0xb0,0xcb,0x5b, +0xab,0x14,0x81,0x8b,0x2b,0x4e,0x37,0xba,0xb2,0x3b,0x64,0xe2, +0xf0,0xaa,0xd1,0x02,0x38,0x04,0x11,0xdc,0xb3,0xac,0xdf,0x5e, +0xf7,0x90,0x63,0xe8,0x38,0x3e,0x3c,0x4b,0xb4,0x26,0xd2,0x53, +0xc0,0x27,0xf7,0x1c,0xf6,0x2f,0x66,0x25,0x7d,0x0b,0xf9,0x54, +0xa9,0x0f,0xc8,0x69,0x35,0xb0,0x7c,0x7b,0xdf,0x98,0xaa,0x08, +0xe4,0x04,0x49,0xcc,0xbb,0x9b,0x42,0x6d,0x69,0x58,0x69,0xfa, +0xe0,0xe2,0x9a,0xf2,0xcf,0x32,0x68,0x0b,0xf6,0xdb,0x61,0x6e, +0x9d,0x63,0x5b,0xc7,0xa5,0xfd,0xcf,0x99,0x3d,0xa2,0xe3,0x47, +0x4a,0xb7,0xb9,0xf6,0x9f,0xed,0x26,0x63,0x1b,0x9f,0x78,0x66, +0x90,0xbd,0x5e,0xaf,0xcc,0xe7,0x1e,0x7f,0xf7,0x3f,0x9b,0x56, +0xe2,0xdd,0xa8,0xb2,0x1b,0x52,0x7c,0xe7,0x74,0x6b,0x6c,0x4b, +0x71,0xc9,0xa1,0xef,0xd8,0xcd,0xf0,0xcc,0xf5,0x82,0x44,0x20, +0xd9,0x66,0xe7,0xf3,0xc1,0xb0,0x8f,0x4f,0xa3,0xbc,0x48,0x65, +0xc5,0x87,0x5b,0xf2,0xac,0x47,0x8a,0x83,0xd5,0xf8,0x58,0x25, +0x71,0x12,0xfb,0x02,0x46,0x55,0xec,0x0a,0x53,0x8c,0x80,0x71, +0x93,0x5a,0x08,0x79,0x25,0x4d,0x51,0x18,0x17,0x13,0x95,0x2c, +0xd7,0xab,0xd1,0xef,0xf7,0x4e,0xd8,0x94,0x9b,0x97,0x18,0x51, +0x02,0xc0,0x2c,0xfe,0xf8,0x50,0x17,0xa4,0xab,0x03,0x17,0x4a, +0xfd,0x81,0x4a,0x58,0x5b,0x27,0x0c,0x5b,0xf9,0x98,0x1e,0x84, +0x25,0xdc,0xbd,0x89,0xf5,0x4c,0x4d,0xfd,0x2c,0x9e,0x64,0xf1, +0xb5,0x19,0x4e,0x13,0xe3,0x7d,0xf3,0xf9,0x3c,0x09,0x9d,0xf9, +0xfb,0x30,0xf5,0x4c,0xa2,0x5d,0xc9,0xfa,0xe8,0xa7,0x74,0x98, +0x2f,0xd2,0xb6,0x46,0x70,0x55,0x65,0x9d,0xc2,0x4c,0xf2,0x49, +0x78,0x0f,0xa3,0x42,0x5b,0xe8,0x99,0xfe,0x56,0x97,0x25,0x23, +0xed,0xe9,0xca,0xf2,0x66,0x7b,0x6b,0xcb,0xdc,0x2f,0xed,0x03, +0xa6,0x80,0xbf,0xc8,0xa0,0x57,0x0b,0x96,0x1e,0x39,0x21,0x68, +0x39,0xf5,0x08,0x45,0x7e,0xa4,0x42,0xc7,0x10,0x91,0xf7,0x29, +0xae,0xe8,0x04,0x59,0xf6,0x46,0x6f,0xaa,0x65,0x3d,0x1b,0x6d, +0xa7,0x33,0x3d,0x5a,0x2b,0x2f,0x67,0x8e,0xd2,0x09,0x71,0xa6, +0x6d,0xba,0x95,0x91,0x6f,0x69,0x02,0xa4,0xb6,0x45,0xef,0x7c, +0x56,0xe1,0xad,0xf4,0x98,0x34,0x7e,0x63,0x2e,0x53,0xf9,0x68, +0x97,0xcf,0x02,0xc4,0x82,0xf2,0x1d,0x1b,0xc2,0x74,0x32,0x21, +0xf1,0x6c,0x1e,0x3c,0xbe,0x67,0x5f,0x2c,0xac,0xfd,0x62,0xfa, +0xe9,0xfa,0x57,0x53,0x0c,0xd2,0xd7,0xa0,0xee,0x1b,0xef,0xc8, +0xec,0x71,0x26,0xe7,0x80,0x77,0x6c,0xf8,0xd2,0x4e,0xc8,0x7c, +0x44,0xae,0x2a,0x26,0x83,0xff,0x9c,0xe7,0x13,0x7e,0x75,0xdb, +0x88,0xb2,0xa8,0x51,0x88,0xbf,0x4b,0x58,0x31,0x14,0xfb,0xbf, +0x11,0x6f,0xb4,0xb2,0x0c,0x3e,0xfc,0x5d,0x42,0x27,0x4a,0x3a, +0xf0,0x27,0x8d,0xea,0x51,0xd2,0x60,0xe8,0xac,0x72,0x03,0x61, +0x7b,0x44,0xe1,0xe6,0x48,0x56,0x70,0x4f,0xcc,0x6e,0x21,0x13, +0x53,0x25,0xdb,0x56,0x38,0x80,0xad,0x3a,0xdf,0xab,0x14,0x81, +0x95,0xe7,0xdc,0x32,0x60,0x2f,0xe7,0xda,0xe3,0xdd,0x8a,0x92, +0x9d,0xf0,0x69,0x7f,0x45,0x8e,0xb4,0xe4,0x4d,0xb6,0x22,0x93, +0x7e,0xbc,0xfd,0xd0,0x99,0xe3,0x43,0xad,0x30,0xe2,0x98,0xd1, +0x9e,0xc2,0x21,0x0c,0x10,0x6b,0xff,0x0f,0x60,0xdf,0xae,0xec, +0x3e,0xbb,0xba,0x20,0x56,0x06,0x66,0xbc,0xc5,0x29,0xad,0x10, +0x38,0xdd,0xd9,0x84,0x45,0x3b,0xb6,0x5b,0x1c,0x97,0x27,0xf1, +0x87,0x89,0x24,0xe3,0x3f,0x19,0x8c,0x3a,0x00,0x14,0x40,0xa8, +0xbd,0x6c,0x2a,0x97,0x20,0x3c,0xf3,0xad,0x10,0x53,0x23,0xe8, +0x9d,0xc8,0x44,0xbc,0x03,0x85,0x0f,0x5c,0xbb,0xe1,0x07,0x0f, +0xc1,0x91,0x2c,0x1f,0x10,0x1f,0x23,0x3c,0x10,0x87,0xac,0xc0, +0xe0,0xdc,0x9b,0x62,0x77,0xc3,0x60,0x3f,0x38,0x3e,0xb8,0x6e, +0x7c,0xed,0xd9,0xdb,0xaa,0x9b,0x46,0x75,0x8a,0x6c,0x2f,0x38, +0xb5,0x0b,0x54,0x7f,0x40,0xec,0xce,0x84,0xe5,0xbe,0xda,0x65, +0xf2,0x1a,0x12,0xd4,0x00,0x55,0xa7,0x2f,0xf4,0xe7,0xd7,0x47, +0xee,0x54,0x99,0x25,0x00,0x0b,0xa7,0xd9,0x1b,0x5d,0xcd,0x87, +0xbd,0x09,0x39,0xac,0x3c,0x87,0x72,0x53,0xa3,0x75,0x57,0x5c, +0x36,0x6b,0x49,0x82,0xf5,0x48,0x2f,0xc6,0x43,0x77,0x29,0x02, +0xdc,0x80,0xf5,0x2e,0xba,0x45,0x51,0xfc,0x51,0x54,0x5e,0x12, +0x26,0xbe,0xe6,0x49,0xef,0x93,0xbd,0x05,0x76,0x56,0x42,0x0e, +0xa5,0x3e,0x25,0x0b,0xd3,0x88,0xa1,0x28,0x45,0xd2,0xf5,0x78, +0x94,0xdb,0x07,0x99,0x93,0xc1,0xed,0x2c,0x53,0x40,0x8a,0x9a, +0x37,0x9c,0x03,0x86,0x10,0x0b,0x89,0x9f,0x01,0xed,0x51,0x5e, +0x17,0x3d,0x1f,0xf7,0xbf,0x58,0xfd,0x52,0xd2,0xfb,0xe5,0xc2, +0x5c,0xb6,0x5e,0x27,0x79,0x3c,0x71,0x17,0xb5,0x9c,0xd8,0x24, +0xfd,0xda,0x88,0x2e,0x54,0x3e,0xf4,0xee,0x7a,0x52,0x9f,0x91, +0xd4,0xe9,0xe4,0x02,0xd9,0xa9,0x41,0xa5,0xb8,0x8e,0xc4,0x8d, +0x47,0x80,0x62,0xff,0x0b,0x0a,0x99,0x39,0xec,0x5d,0x47,0x57, +0xb9,0x46,0x68,0x13,0x5f,0x0b,0xe6,0xfd,0x05,0xea,0xcb,0x20, +0xd5,0x6e,0xf2,0x65,0x92,0x94,0xa5,0x92,0xc7,0x58,0x32,0x61, +0x84,0xac,0xeb,0x84,0x64,0x99,0x3a,0xf0,0x08,0x21,0x73,0x21, +0xde,0x52,0xaa,0xbc,0xa6,0xd4,0x66,0xad,0xeb,0xac,0xdb,0x2c, +0x05,0x96,0xc5,0xdb,0x56,0xb5,0x60,0x93,0x1d,0x1d,0x3b,0x9c, +0xee,0xee,0xd3,0xae,0xd3,0xc5,0x05,0x3a,0x58,0x99,0xd5,0xdd, +0x04,0x67,0x97,0xc9,0x1c,0x3b,0x29,0x1f,0xd5,0xea,0x71,0xe0, +0x5f,0x52,0xff,0xbb,0x4f,0x10,0x65,0x3f,0x5f,0x4f,0xea,0x75, +0xf8,0xd3,0xec,0x9d,0xa3,0xd7,0x98,0x82,0x81,0x08,0xec,0x37, +0x0b,0xcc,0xe1,0xf6,0xc8,0x4e,0x8a,0x96,0x7a,0x21,0xbc,0x0c, +0xc9,0x4c,0x34,0x5b,0x8e,0xf7,0x8e,0xfe,0xb4,0x77,0xf4,0x7e, +0x7f,0xe7,0x64,0xf7,0x35,0x3e,0x27,0x8b,0xa2,0x12,0x9d,0x76, +0x13,0x34,0xe7,0x2e,0x8d,0x78,0x03,0x4b,0x7b,0x40,0xec,0x30, +0xcd,0x46,0x17,0xf9,0x30,0x76,0xe9,0x89,0x6f,0x54,0x06,0x4d, +0x60,0x53,0xa6,0xd2,0xe2,0x08,0x9d,0xd2,0x83,0x68,0x6b,0x31, +0xe9,0x58,0x74,0x3b,0x20,0x7c,0xa5,0xdf,0x6f,0x7d,0x6f,0xf7, +0x26,0x92,0xbf,0xb5,0xbd,0xfd,0x9c,0x34,0xc0,0x20,0x00,0x59, +0x5c,0xb2,0x6d,0xf5,0x23,0xfb,0x47,0xcf,0x21,0xd6,0xd7,0x6d, +0xc7,0x1d,0xcc,0x67,0xa8,0xab,0xa1,0xb8,0x6b,0x3c,0x1a,0x8f, +0x6f,0x79,0xb1,0x67,0x70,0xb4,0x13,0xf0,0x25,0x07,0xa3,0x8b, +0x72,0x21,0x7a,0x91,0x10,0x93,0x54,0x81,0x33,0x00,0xb6,0x7f, +0xb6,0x00,0x84,0x20,0x5e,0x98,0xb0,0x3e,0x31,0x94,0xa3,0x5f, +0xd1,0xd7,0x92,0x16,0xbc,0xe9,0x67,0xd1,0xfb,0xa8,0xe1,0xc6, +0x11,0xef,0x1f,0xe6,0xb8,0x5c,0x35,0x1a,0xe8,0xb8,0x4c,0xd6, +0xc8,0xe7,0xbf,0x49,0xda,0x2e,0xc7,0xa9,0x2f,0xab,0x36,0xf6, +0x7c,0xf3,0x5b,0x99,0xa9,0xe9,0xee,0x7d,0x59,0xaf,0xcf,0x8f, +0x8f,0xe6,0x3b,0x9e,0xfd,0xd1,0x3d,0xef,0xee,0xce,0xdb,0xb7, +0x3f,0xee,0xec,0xfe,0x11,0x58,0x61,0x89,0xa1,0x36,0xa3,0x86, +0x07,0xaf,0xd4,0x35,0x1e,0xaa,0xbf,0xb3,0x92,0x72,0x51,0xee, +0x5e,0x5d,0x2e,0x49,0x1f,0x21,0xf9,0xc5,0x62,0x63,0xdb,0x3d, +0x5d,0xa8,0x84,0xb3,0x54,0xba,0xd3,0x44,0x91,0xf9,0xa7,0x1a, +0x30,0xed,0x36,0x4e,0x30,0xd8,0xae,0x07,0x13,0x1e,0x15,0xa5, +0x1e,0x39,0x1c,0xf6,0xea,0xe3,0x3c,0x47,0xa7,0xb9,0x9d,0xda, +0x2c,0xce,0x2b,0x95,0xe9,0x25,0x4d,0xe5,0x47,0x58,0x22,0x50, +0xdb,0xf7,0x19,0x4e,0x15,0xcd,0xfe,0x02,0x71,0xc9,0xde,0x57, +0x61,0x1e,0xf4,0xc4,0x36,0x1c,0x1e,0x85,0x29,0x40,0x70,0xd3, +0x01,0xf6,0x92,0xcd,0x79,0x1a,0x91,0xc0,0xba,0x21,0xfa,0x82, +0xe2,0xbb,0x04,0x4f,0xb2,0xdf,0xe2,0x68,0xf6,0xae,0x72,0x15, +0x45,0x0a,0x09,0x64,0xda,0x7f,0x00,0x35,0xc2,0xd7,0xdb,0xde, +0x74,0xb6,0xb1,0x3f,0xee,0x93,0xc7,0x3e,0x08,0x1f,0xe2,0xab, +0x91,0x5e,0x84,0x02,0x6d,0x50,0x2d,0x08,0xf5,0x45,0x14,0x81, +0x7f,0x38,0x6b,0x3f,0x1a,0xa3,0xe2,0x25,0xa4,0x7a,0x72,0x60, +0x9b,0x15,0xb6,0xc9,0xe6,0x6b,0x7b,0x47,0x31,0x41,0x4d,0x0d, +0xb9,0x71,0x61,0x23,0xa0,0x6c,0xb1,0x88,0xb4,0x87,0x96,0x62, +0x9e,0xf3,0x41,0x92,0xa1,0x71,0x63,0xde,0xc1,0x1d,0xef,0x19, +0x77,0x47,0x47,0xa4,0x41,0x2a,0xbe,0xff,0x2d,0xa0,0x01,0x52, +0x5b,0x71,0x19,0x00,0x5c,0xb1,0x54,0xf1,0xa4,0x1e,0x29,0x4b, +0xd7,0x23,0xd9,0x74,0x54,0x16,0x6e,0xed,0x98,0xb8,0x40,0xd4, +0xc0,0x55,0x08,0xdf,0x1d,0x90,0x83,0x3a,0x61,0xb4,0x2f,0xb6, +0x12,0x3a,0x3b,0xb6,0x6c,0x37,0x0b,0xb8,0x72,0x1c,0xae,0x39, +0x76,0xc5,0x4a,0xd8,0xbe,0x9d,0x84,0xea,0x75,0xf9,0xa0,0xbd, +0x79,0x18,0x2a,0x00,0xd0,0xaf,0xea,0xd6,0xd5,0xcb,0xd7,0x82, +0x5e,0x8d,0xd6,0x18,0x46,0xed,0x5d,0xa5,0xf9,0x8b,0x9a,0xbb, +0xac,0x7d,0xa0,0x33,0x15,0x1f,0xf6,0xe9,0x47,0x1b,0x9d,0x58, +0xe7,0xe4,0xb9,0xc7,0x6e,0xfe,0x55,0xbc,0xa8,0x95,0xfa,0x7a, +0x1e,0x86,0xe8,0xf3,0x69,0x78,0x2e,0xb6,0xa0,0xa4,0xea,0x9c, +0xfb,0x17,0x55,0xc3,0x9d,0x27,0x7f,0x2f,0xcd,0x8a,0xcf,0xfd, +0xb2,0x87,0xfe,0xe9,0x85,0x96,0x05,0xca,0x0f,0xca,0xd1,0x76, +0xcc,0xeb,0xde,0x5d,0xee,0x3d,0x6b,0x98,0x59,0xf0,0x7e,0xd3, +0xbf,0x99,0xa1,0x76,0x93,0x93,0x4a,0xec,0x58,0xfe,0xb8,0x1c, +0xa1,0xa6,0x29,0x6f,0xb2,0xb2,0x2d,0x63,0xb7,0xf3,0x46,0xde, +0x60,0x21,0x00,0xfb,0x0a,0xf1,0xba,0x37,0x0d,0x1e,0xb4,0x66, +0x6b,0x2d,0xc3,0x86,0x31,0x71,0xb3,0x2a,0x8f,0x91,0xd8,0xa9, +0x67,0x42,0x4e,0x32,0x7d,0x7d,0x0c,0xc7,0xee,0x9a,0x55,0xff, +0xcb,0xee,0x4e,0x0c,0xb7,0xc5,0x16,0xad,0x30,0xbf,0xeb,0x5f, +0xe3,0x21,0x36,0x14,0x90,0xb7,0xec,0x52,0x30,0xe9,0xf0,0xae, +0x6c,0x2d,0xf3,0xd9,0xe8,0x95,0x9f,0xdd,0xf2,0x13,0x50,0xd6, +0xe2,0xe3,0x9d,0x23,0x76,0x80,0xcf,0x1b,0x9e,0x0f,0x5c,0x8e, +0xac,0x03,0x0c,0xe8,0x9f,0xac,0x50,0xf4,0x21,0x3c,0x5c,0xdf, +0xa2,0x1e,0xd3,0xe5,0x08,0x9f,0xe9,0x65,0xbe,0x8b,0x6d,0x62, +0xc3,0x35,0xea,0xed,0xaa,0xde,0xdd,0x4a,0x52,0xdb,0x20,0xee, +0xdc,0x47,0x1b,0x43,0xda,0xb7,0x58,0x8b,0x75,0xb2,0x88,0x96, +0xc8,0x65,0x24,0xb9,0xa8,0x24,0x00,0xe7,0x55,0xa0,0x50,0x67, +0xcb,0xbe,0xb9,0x4b,0x0a,0xd4,0x63,0xeb,0x94,0xe1,0x6d,0xd9, +0x77,0xcf,0x8a,0x0e,0xfd,0xdb,0x67,0xa6,0x06,0x5c,0x0b,0x50, +0xaa,0xae,0x91,0x47,0x3c,0xf5,0x66,0x1d,0xf0,0xa4,0xa7,0xdf, +0x67,0xa3,0x63,0xb2,0xca,0x57,0x93,0x3a,0xe6,0xbd,0x61,0x6d, +0x6f,0x7a,0xd1,0xbb,0xc5,0x17,0xe5,0xf6,0x09,0xf4,0x54,0x17, +0xc4,0x27,0xcf,0xb3,0xeb,0x1e,0x29,0xf0,0xa2,0xe1,0x09,0x74, +0x1f,0x58,0xb3,0x6f,0xce,0x6b,0x5e,0x09,0x77,0x5a,0xc3,0xdd, +0x75,0xad,0x57,0x93,0x46,0xf0,0xbc,0xc5,0x97,0xe8,0xd6,0xd3, +0x56,0xb3,0x56,0x3b,0xce,0xf3,0x1a,0xee,0x4c,0xd3,0xcd,0x4d, +0xd8,0xd8,0x4d,0xed,0x86,0xef,0xc3,0xb4,0x39,0x9e,0x5c,0x6d, +0x02,0x16,0x9b,0xa3,0xab,0x26,0xcf,0x4f,0x9a,0x3f,0xb2,0xa5, +0x59,0x2b,0xcf,0xb4,0xc5,0xa2,0x38,0x6a,0x9d,0x98,0x31,0xb4, +0x6f,0x18,0x65,0xea,0x00,0x31,0x85,0x66,0xb2,0x70,0x5a,0xd2, +0x43,0x12,0x99,0x9a,0xd8,0xc9,0xd6,0x68,0x62,0xf1,0x79,0x6e, +0x64,0x8b,0x82,0x38,0xc8,0x8a,0xd0,0x34,0x78,0x3c,0xa2,0xe8, +0x86,0x9e,0x15,0x51,0x39,0xc2,0x81,0x27,0x5a,0xce,0xc2,0xc1, +0xec,0x14,0x6c,0x7e,0xc5,0x61,0x32,0x15,0x49,0x52,0xca,0x44, +0x1d,0x77,0xc0,0x35,0xb1,0xe2,0x4e,0x46,0xf0,0x17,0x0b,0x82, +0xab,0x1e,0xeb,0x50,0x38,0x8a,0xd0,0x46,0x1e,0x52,0xa2,0xd7, +0xd6,0xbe,0x15,0xb7,0x76,0x84,0x07,0x9f,0xc1,0x11,0x34,0x5a, +0x60,0xb1,0x93,0xf0,0xa1,0x31,0xc6,0xf8,0x47,0x10,0xd4,0x75, +0x7c,0x80,0x16,0xb9,0xb7,0x35,0x13,0x59,0xed,0x41,0xee,0x02, +0xba,0xca,0xaa,0x48,0x8f,0xa2,0x26,0xac,0xee,0xfd,0x6c,0xa6, +0xcd,0x24,0x08,0xc1,0x91,0xf5,0x2c,0x6a,0x23,0xe4,0xf3,0x82, +0x71,0x92,0xb4,0xcb,0xef,0x76,0xb8,0x1e,0x5c,0x99,0x65,0xf6, +0x52,0x39,0xf8,0x66,0xe6,0x48,0x7f,0xf1,0xfd,0x8e,0xb5,0xec, +0x50,0x58,0xb9,0x78,0x3e,0xeb,0x77,0x27,0x96,0x81,0x16,0xa6, +0xbc,0x7f,0x87,0x62,0xef,0x73,0xed,0x65,0xda,0x8a,0xb7,0x30, +0xc1,0xce,0xdc,0xa9,0xaf,0xfa,0xa5,0xac,0xfa,0x75,0x4c,0x45, +0x29,0xb5,0x60,0xe9,0x32,0xc1,0x51,0xcd,0x5b,0xd4,0x5e,0xcf, +0x2b,0x4f,0x69,0x4a,0x00,0xef,0x07,0xfd,0x34,0xca,0x47,0x1b, +0x73,0x90,0xfe,0xde,0xfd,0xb4,0xff,0x23,0xec,0x08,0x5f,0x1d, +0x1c,0xc1,0xa6,0xf0,0x38,0xbd,0x7f,0xb9,0xb7,0xfb,0x66,0x7f, +0xe7,0xed,0xfb,0xe3,0xbd,0xc3,0x34,0x6a,0x46,0xe6,0xf7,0x47, +0x07,0x3f,0x1d,0x72,0xc8,0x44,0xe6,0x70,0xe7,0xe4,0x64,0xef, +0xe8,0xdd,0x71,0x7a,0x7a,0x7f,0x33,0x18,0xc1,0x54,0x4a,0x5b, +0x06,0x3e,0x5e,0x4d,0x7a,0x17,0xe9,0x16,0xda,0x43,0xa5,0xaf, +0xe7,0xb8,0x91,0x3e,0x9c,0xe4,0x29,0x88,0x87,0xf0,0x75,0x3c, +0xbf,0xc4,0xaf,0x51,0x7e,0x45,0x71,0x1b,0xf4,0x29,0x91,0x57, +0xc7,0xf8,0x8a,0xf7,0xb9,0x19,0xca,0x07,0x5e,0x41,0x16,0x01, +0x6f,0x3b,0xc0,0xdb,0x0e,0xf0,0xff,0xfe,0xbf,0xab,0x40,0xc7, +0x18,0x6d,0x81,0x27,0x15,0xd0,0xbb,0x66,0xf7,0xa7,0xa3,0xa3, +0xbd,0x77,0xbb,0x7f,0x79,0x7f,0xfc,0x97,0x7d,0x34,0xf1,0xb8, +0x34,0x2f,0x77,0x4e,0xf6,0x4e,0xde,0xec,0xef,0xf9,0x3e,0xd8, +0x3f,0x78,0x77,0xf2,0x3a,0x8d,0xfe,0xd0,0x1b,0xcd,0x7b,0x93, +0x2f,0xe6,0x55,0x7e,0x3e,0xa1,0x8f,0xfd,0xde,0xe4,0xe2,0xda, +0xec,0xdc,0x4e,0x06,0x43,0xf8,0xfe,0x62,0xfe,0x30,0x1f,0xe5, +0xf0,0x67,0xf8,0xc5,0xec,0xcc,0xaf,0x60,0x66,0x9a,0x63,0xa0, +0xcc,0x1c,0xfd,0x87,0x98,0x83,0x8b,0xd9,0x18,0x7f,0xdf,0x8d, +0xef,0x38,0xe2,0x65,0x7e,0x41,0x1f,0x91,0x55,0x7d,0x37,0xc0, +0xc5,0x8f,0x5f,0x1f,0x1c,0x9d,0xf8,0xda,0xb0,0x26,0xac,0x04, +0xab,0xb0,0x15,0x20,0x7c,0x04,0x8f,0xb0,0x11,0x2a,0x42,0x44, +0x60,0x01,0x9c,0x97,0x3b,0x7f,0x49,0xa3,0x63,0x98,0xe3,0x50, +0x66,0x7f,0x4c,0x3f,0x27,0x20,0xc7,0xe1,0xef,0x7f,0xe5,0xfd, +0x11,0x7f,0x9d,0x5c,0xcf,0x27,0xf4,0xf1,0x6a,0x32,0xc0,0x9f, +0x63,0x90,0x87,0x27,0xf0,0x51,0xc6,0xc8,0x82,0x43,0x58,0x08, +0x08,0x81,0x60,0x71,0x2c,0x89,0xc5,0x82,0x12,0x3b,0xfb,0x87, +0xfb,0x40,0x0f,0xd1,0xce,0x3e,0x6e,0x1d,0xf6,0xa3,0xae,0xb9, +0xc9,0xfb,0x83,0xf9,0x4d,0x1a,0xed,0xef,0xef,0xd7,0xfa,0xa6, +0xf6,0xa5,0x76,0x9d,0xde,0xdc,0xa4,0xd3,0x69,0xad,0x07,0x39, +0xa6,0x78,0x3e,0x09,0xdc,0x67,0x7f,0xb3,0xbf,0xf9,0x85,0x93, +0x30,0xfe,0x12,0x38,0xd6,0x4b,0x9c,0xff,0xd1,0x1e,0xfc,0x67, +0x6a,0xfb,0xb6,0x30,0x1e,0x50,0x8c,0xae,0x38,0x49,0x45,0x72, +0x1d,0x2e,0x5a,0x62,0x09,0xb8,0x44,0x12,0x7c,0x9b,0xf1,0x64, +0x80,0x0a,0xb8,0x0a,0x0f,0xca,0xe9,0x63,0x21,0x6a,0x69,0x6e, +0x87,0xf3,0x49,0x6f,0xb8,0xdb,0x53,0x4f,0xad,0x47,0xf3,0x1b, +0x37,0x0b,0x5b,0xf8,0x5c,0x74,0x7e,0xd3,0x89,0x60,0x2b,0x0d, +0x0d,0x18,0x83,0xdc,0x3c,0x01,0x31,0x57,0x89,0xca,0x7c,0x17, +0x71,0xd8,0x9b,0x5d,0x03,0xe7,0x84,0x65,0xdb,0x4b,0x72,0x39, +0x3d,0xc3,0x47,0x66,0x34,0xbb,0xb6,0x9d,0xb7,0x19,0xa1,0xdd, +0x5d,0x9b,0xe4,0x5e,0x4d,0x6f,0x6c,0xb4,0x13,0x1b,0x89,0x1c, +0xd7,0x5d,0x70,0x1c,0x73,0x64,0xac,0x12,0xdd,0x0d,0x95,0x83, +0xc2,0xcc,0x18,0x40,0x2b,0x25,0x53,0x64,0xcb,0xf6,0x86,0x68, +0x7c,0xfe,0x41,0x79,0xc4,0xcf,0xf4,0x79,0x10,0xbb,0x5b,0xc6, +0x8b,0xbf,0x36,0xe4,0xc2,0x9b,0x6e,0x7b,0x80,0xe4,0x1e,0x3e, +0x71,0xfc,0x35,0x3a,0x3e,0x75,0x0f,0x9f,0x24,0x2f,0x74,0x27, +0x9a,0x14,0x11,0x07,0xc9,0xbf,0xee,0x82,0xcc,0xf2,0x72,0xef, +0xd5,0xce,0x4f,0x6f,0x4f,0xde,0x1f,0x02,0x49,0x1d,0x9f,0x5a, +0x20,0xf2,0x02,0xa1,0x80,0xdf,0xce,0xed,0xed,0x0a,0x14,0x81, +0xe5,0xbc,0x5e,0x85,0x20,0xf4,0x66,0x56,0xe1,0xc3,0xdc,0x7b, +0x80,0xe6,0x7c,0xd3,0x1c,0x27,0x2e,0xbf,0x98,0xf8,0x63,0xfe, +0x85,0xb4,0xd8,0xbc,0x4f,0x6c,0xdb,0xaa,0xde,0xf4,0x01,0x60, +0xbf,0xee,0xd2,0x8e,0xcc,0xf8,0x8a,0xeb,0x75,0xe8,0xe6,0xb5, +0xcc,0x47,0x04,0xe6,0x75,0x15,0x82,0x90,0xad,0xe1,0x83,0x6a, +0x60,0xce,0xf3,0xab,0xc1,0x88,0x0e,0xe9,0x63,0xfa,0x34,0x9f, +0xae,0xc7,0xc3,0xdc,0x10,0x61,0x7d,0x1a,0xa8,0x9b,0x84,0xad, +0x2c,0xa3,0x24,0x27,0x61,0x53,0xf6,0xa4,0xc3,0x91,0xf2,0xce, +0x85,0xe2,0xac,0xc0,0x98,0x3a,0x20,0x4a,0x5b,0x03,0xd6,0xd0, +0xdb,0xd7,0xd0,0x4a,0xea,0x42,0x79,0x3f,0x87,0xd2,0x67,0x70, +0x43,0xf6,0x6f,0x91,0x25,0x2a,0xbc,0x3c,0xa3,0xe8,0x8e,0x28, +0x82,0xa8,0xd7,0x36,0x24,0xbd,0x16,0x40,0x93,0x1e,0x3f,0x81, +0xf6,0x77,0x8e,0xbe,0x48,0x58,0x7b,0xe8,0x7f,0x1f,0xc8,0xb5, +0xd1,0x0a,0xf4,0x4a,0x50,0xf3,0xec,0x47,0x3c,0x45,0xab,0x86, +0x87,0xb2,0xc0,0x96,0x09,0xf0,0xde,0x8c,0x0a,0x61,0x04,0xba, +0xad,0x5f,0x53,0xbe,0x95,0x83,0xce,0xd7,0xb3,0x9b,0xe1,0xaf, +0x91,0xd8,0x7a,0xb7,0xb7,0x58,0x87,0x39,0x87,0x3f,0xb0,0x7c, +0xa0,0x21,0x6c,0x6b,0x9c,0x02,0xad,0x53,0xfe,0x1a,0x2d,0xdb, +0xf8,0xb4,0xcc,0x7f,0x3a,0x0b,0x02,0x02,0xe1,0xdd,0x18,0xdb, +0x9e,0xf9,0x5e,0x90,0xf8,0x44,0xf4,0xef,0x79,0x63,0x12,0x68, +0xe5,0x38,0x2b,0x60,0xf2,0x5e,0x2a,0x98,0xa4,0xf2,0xe2,0x48, +0x2e,0x6c,0x67,0x18,0x9f,0x29,0x62,0x09,0x6a,0x35,0x34,0x1f, +0xf0,0xc2,0xd7,0x5d,0xb1,0x49,0x11,0x67,0xf3,0xc8,0x9e,0xf0, +0x3a,0xe9,0xeb,0x76,0x76,0x7d,0x3b,0xb9,0xfc,0x1c,0x99,0x67, +0xd6,0x51,0x1f,0x00,0xa9,0x45,0x64,0xe5,0xa9,0x76,0x33,0x00, +0x59,0x19,0x04,0x7b,0x84,0x52,0xbb,0xa5,0xf6,0x42,0x12,0xc8, +0x63,0xcd,0x67,0x74,0x64,0x16,0xd4,0x9e,0xb4,0xf5,0xec,0x95, +0x8a,0x2d,0xfe,0x46,0xf4,0x8b,0xe8,0xb5,0xb4,0x8d,0x73,0x3d, +0x82,0xc6,0x59,0x63,0x15,0xca,0x88,0x21,0x4a,0x04,0x9e,0xe8, +0x8c,0xa7,0x78,0x41,0x51,0x88,0xc9,0x0a,0x3e,0x1e,0x65,0x6a, +0xcf,0xc6,0x6e,0x5e,0x4b,0x7e,0x4e,0x48,0x0c,0x4d,0x6b,0x3b, +0xaa,0xf0,0xdd,0x01,0x0a,0x6f,0x94,0x78,0xaa,0xca,0x90,0xa4, +0xce,0xe2,0x03,0xaa,0xe9,0x66,0x8a,0xa9,0x2b,0x6c,0x93,0x46, +0xcc,0x35,0xe0,0x75,0x72,0x83,0x3f,0xc9,0x02,0x35,0x82,0xb0, +0x38,0xf7,0xce,0xa7,0x38,0x74,0x41,0x8f,0x35,0x3c,0x68,0xef, +0x23,0xd9,0xb5,0x72,0x92,0xd3,0xe3,0xfb,0x0a,0x52,0xe9,0x51, +0x17,0x93,0xa9,0x3c,0xee,0x6d,0x99,0xa5,0x31,0x27,0x54,0xd0, +0x07,0x51,0x46,0xb2,0xa6,0x65,0xff,0xd8,0x17,0xcf,0x04,0x60, +0x45,0x79,0x4f,0xe8,0x86,0x13,0x0b,0x50,0xc2,0xf6,0xc4,0xaa, +0x24,0xce,0x42,0x29,0xb2,0x58,0xc8,0x47,0x2a,0xd9,0x1b,0xbe, +0x6e,0xd8,0x28,0xaf,0xc4,0x5a,0x51,0xf5,0x43,0xb5,0x72,0xf9, +0x34,0x9c,0x80,0xc8,0xc9,0x1a,0x80,0x43,0x98,0xd7,0xca,0xfd, +0x15,0x9c,0x00,0xc6,0xea,0x1c,0xb6,0x93,0x9a,0x19,0xe0,0xf8, +0x59,0x66,0xf0,0xa4,0x29,0xee,0x66,0xae,0x85,0x40,0x87,0x4c, +0xe6,0x91,0x99,0x8f,0xc7,0x85,0xe3,0xf9,0x0c,0x0b,0x3c,0x30, +0x7a,0x8b,0xc5,0x83,0x1d,0x64,0x04,0x0a,0xb6,0x03,0xa1,0x00, +0x69,0xc3,0x4a,0x11,0x40,0xf6,0x8b,0x52,0x47,0x81,0xf2,0x8d, +0x34,0x61,0x76,0x67,0xa7,0x81,0x18,0x60,0x27,0x4c,0xc4,0x89, +0x11,0x30,0x9a,0xb0,0xfa,0xd5,0xfc,0xe6,0x7a,0xfa,0x04,0x7e, +0x83,0x38,0x55,0xf1,0x1b,0x35,0x20,0x01,0xb3,0x09,0x2b,0x0f, +0x7b,0xfd,0xff,0x8f,0xec,0xa3,0x11,0x7b,0xa8,0x1d,0xdf,0x29, +0x8a,0x9f,0x20,0x80,0x47,0x38,0x89,0xdb,0x86,0xda,0xc5,0xd8, +0x92,0x71,0x96,0x85,0x0b,0x34,0xad,0xf9,0x8f,0x4e,0x9c,0x37, +0xe5,0xc5,0x54,0xcf,0x9f,0xe2,0x62,0x5a,0x31,0xef,0xe4,0xe1, +0x31,0x2d,0x11,0xfe,0xe1,0xf1,0x57,0x2d,0xae,0x8f,0xf1,0x4c, +0x2b,0x23,0x4b,0xb9,0xca,0x96,0x3e,0x8d,0x1f,0x59,0xbe,0xd2, +0x50,0xdd,0x5f,0xcd,0x88,0x9e,0xca,0x89,0xec,0x54,0x11,0x6d, +0xcc,0x5b,0xb1,0x13,0x50,0xf5,0x58,0xdb,0xda,0xb7,0x40,0x33, +0x2e,0x36,0x5f,0x77,0x25,0xa8,0xe3,0x10,0x20,0xae,0x1b,0xf0, +0x8d,0xfa,0xa4,0x65,0xd8,0xe1,0xd1,0xb9,0x56,0x76,0xe3,0x94, +0x4e,0x58,0x67,0x1a,0x87,0xe1,0xcc,0xc3,0xb6,0x26,0x70,0x8a, +0xd3,0x90,0x67,0xe7,0xb2,0x70,0x5c,0x21,0xb7,0xce,0xfa,0x46, +0xc1,0x77,0x2c,0x5e,0x2d,0x11,0xe1,0xec,0xa3,0x51,0xe7,0xb5, +0x16,0x8f,0xb6,0x4a,0x77,0xe8,0xdf,0x0a,0xb5,0x39,0xfc,0xad, +0xd2,0xa2,0x24,0x74,0x14,0xbb,0xcb,0x38,0x4e,0x2e,0x02,0x7c, +0xbc,0x4c,0x1b,0x5f,0xa1,0xd7,0x0e,0x20,0x3b,0x81,0x25,0xd8, +0x14,0x0d,0x90,0x5d,0x81,0x1b,0x32,0x2a,0xce,0x60,0x6d,0x64, +0xe1,0x7e,0x21,0x78,0x55,0xa9,0xf4,0x66,0xa6,0x23,0xb2,0x07, +0x65,0x1f,0x5e,0x8a,0xb6,0x87,0xbe,0x6d,0x50,0x0f,0x27,0x9d, +0x1a,0x8d,0x94,0x0a,0xec,0x41,0xab,0x9b,0x07,0x32,0x4e,0x6c, +0x7b,0x79,0x97,0x6e,0x21,0xe3,0x31,0x69,0x0a,0x8a,0x59,0x40, +0xd1,0x5d,0x06,0x78,0xbd,0xfe,0x05,0x48,0xe3,0x71,0xe4,0x18, +0x37,0x67,0x17,0xfd,0xe3,0xc8,0xb8,0xf8,0x26,0x73,0x24,0x18, +0x4f,0x81,0xc4,0x06,0x7b,0x9c,0x3f,0x1c,0x5b,0xdd,0x3e,0x59, +0x79,0x55,0xd2,0x35,0x5a,0x36,0x72,0x37,0x56,0x4d,0x1b,0x13, +0xe3,0xf1,0xf7,0x00,0xaf,0xcb,0x70,0xea,0x85,0x37,0xea,0x6d, +0xd7,0x89,0x1d,0x3b,0x13,0x33,0xb5,0x2d,0xf0,0xe5,0x80,0x93, +0x5a,0x78,0x20,0x96,0xa3,0x08,0xa9,0xb1,0xc8,0x6c,0x37,0x35, +0x61,0x20,0x50,0x4b,0xac,0x53,0xdc,0x05,0xa4,0xab,0x79,0x1a, +0x8b,0x29,0x5c,0xb5,0x63,0x19,0xaa,0xe6,0xb0,0xa6,0x0a,0xe6, +0x96,0xf8,0xae,0xa3,0x5b,0x68,0x9d,0xdf,0x31,0x4c,0x5c,0x50, +0x14,0xd3,0x54,0xbd,0x2d,0xd2,0x43,0xac,0x63,0x84,0xd9,0x69, +0x34,0x92,0x80,0x04,0x9a,0x40,0x2f,0x11,0xd9,0x13,0x55,0x04, +0x44,0xc6,0x4b,0x59,0x2d,0x94,0x3e,0x9b,0x17,0xb3,0xc9,0x90, +0x8c,0x80,0x49,0xf8,0x26,0x9f,0xf5,0x28,0xbc,0xbd,0x96,0x71, +0xcc,0xa7,0xeb,0xc1,0x85,0x3a,0xc1,0xc8,0x87,0x37,0xd6,0x55, +0x03,0xa7,0xcf,0x80,0x5b,0xe7,0xb3,0xa4,0x1d,0xf5,0xd0,0xdc, +0x95,0x57,0x7b,0x80,0x8c,0xd6,0x32,0x30,0x3f,0x4d,0x44,0x15, +0x63,0x8e,0x04,0xb6,0x18,0x18,0x81,0xdc,0x82,0x3d,0xf5,0x1a, +0xa6,0x65,0xf0,0xcf,0x59,0x41,0x4e,0xd0,0x12,0xbe,0xf8,0xc6, +0x27,0x26,0x7e,0x3e,0x25,0xf2,0xa1,0x3c,0x68,0xa2,0x8c,0x4d, +0x69,0xe1,0xed,0x2a,0x76,0x05,0xec,0x30,0x88,0x7a,0x2a,0xfa, +0x48,0x4a,0x26,0x6d,0xf9,0xc0,0xd6,0x02,0x10,0xb6,0xef,0xc6, +0xf8,0x47,0x09,0xfa,0x5e,0xf4,0x70,0x5c,0x87,0xd0,0x4c,0x47, +0xb5,0xb8,0xc3,0x09,0x45,0xc0,0x94,0xc7,0x6d,0x3d,0x27,0xde, +0x72,0x9c,0xe4,0x88,0x43,0x54,0xd6,0x0a,0x94,0x8c,0x76,0xff, +0x4b,0x43,0xaa,0x0b,0x84,0xef,0x0b,0xe4,0x2d,0x81,0x09,0xb5, +0xdb,0x4e,0xa3,0xcb,0xcb,0x8d,0xdf,0xfc,0xf6,0xfb,0xed,0xad, +0xdf,0x6e,0x84,0xb5,0x47,0x5d,0xbc,0x79,0x42,0x6d,0xc2,0x8a, +0x79,0xba,0x96,0x79,0x3a,0xf1,0x8a,0x3c,0x84,0x57,0xc5,0xa4, +0xc6,0x2b,0x6b,0x97,0x67,0x8c,0xa8,0x09,0xdf,0xf0,0xc7,0x61, +0xf9,0x27,0x62,0x20,0xa8,0x31,0x53,0x51,0x99,0x4e,0x7e,0x1a, +0x8b,0xc1,0xe3,0x70,0xf2,0x9b,0x8b,0x22,0x5c,0x19,0x64,0x62, +0xb5,0x14,0xdd,0x28,0x58,0x7d,0xe6,0x47,0xda,0x92,0x18,0x5e, +0x6a,0xdb,0x8f,0x9a,0xc8,0x42,0x12,0x63,0x6e,0x96,0x95,0xc1, +0xb4,0xcb,0x23,0x27,0x6d,0x34,0x0f,0xf0,0xd7,0xe5,0xe3,0x2f, +0x46,0xfa,0x83,0x2b,0xd2,0x0d,0x83,0x61,0x63,0x53,0x84,0x04, +0x60,0x97,0x75,0x81,0xb2,0x2d,0x77,0xb3,0xa0,0x8a,0xb0,0xa1, +0xdc,0x15,0x98,0x87,0xda,0x48,0x62,0x4e,0x50,0xdc,0x3f,0x17, +0xe6,0x06,0xc5,0xb9,0x2b,0x1c,0x5d,0x6f,0xbd,0x2e,0xe0,0x2a, +0x7a,0x02,0xf6,0xea,0x41,0xde,0x46,0xe3,0x71,0xeb,0x63,0x5f, +0x45,0x03,0xab,0x17,0x99,0x12,0x0d,0x74,0xca,0xa3,0x22,0x59, +0xd3,0xf8,0x31,0x1a,0x0f,0xbb,0xe6,0xc1,0x61,0x24,0xd5,0x92, +0x8a,0xbe,0xa3,0x7b,0x74,0xdd,0x17,0x7a,0xea,0x15,0xee,0x64, +0xae,0x0a,0xf2,0x4d,0x3f,0x3f,0x9f,0x5f,0xa1,0x28,0x3c,0xcd, +0x87,0x97,0xb4,0xaf,0x68,0x8b,0xb9,0x0f,0x88,0xdf,0x1b,0xf5, +0xce,0x87,0x79,0x5f,0x3d,0x98,0x1b,0xf6,0xae,0x2a,0x64,0x0f, +0x8a,0xee,0xc4,0x0c,0x0b,0x03,0x22,0x77,0x50,0x44,0x41,0xe6, +0x28,0x1b,0xdb,0xb2,0x56,0x87,0x1c,0x9a,0xf8,0xfa,0xb3,0xc7, +0xef,0x23,0xf1,0x75,0x99,0xab,0x10,0xbe,0x9d,0xa5,0xa3,0xf1, +0x65,0x8d,0x32,0xa0,0x09,0xe9,0xc9,0x15,0x5e,0x72,0x5d,0x7c, +0xec,0xc0,0x57,0x86,0xa1,0x1b,0x90,0x12,0x7a,0x57,0x39,0xea, +0x04,0x00,0x8b,0x77,0xe9,0xee,0x2c,0x4e,0xe5,0x49,0x3a,0x7c, +0xd7,0x8c,0x4a,0x3c,0x2a,0x9a,0x54,0x79,0x1a,0xae,0x64,0x5a, +0xf8,0x62,0x47,0x47,0x47,0x6f,0xb9,0xf6,0xac,0xba,0xa0,0xcd, +0xd4,0x88,0x52,0x8e,0x21,0x6b,0x42,0x09,0xee,0x2a,0xb4,0x3e, +0xe5,0x68,0x3a,0x1e,0xe6,0x30,0x30,0xf1,0x8c,0xb4,0x36,0xe5, +0xf1,0x39,0x46,0x66,0x56,0x55,0x58,0xc2,0xf4,0x20,0x6e,0x38, +0xbe,0x22,0xe7,0x2b,0x14,0x73,0x8a,0x65,0x60,0xd1,0x92,0x60, +0x13,0x12,0x17,0x0b,0x54,0x27,0x74,0x36,0xfa,0x30,0x37,0x6f, +0x69,0x3a,0x85,0x99,0x0a,0x68,0x54,0x3d,0x12,0xa8,0x78,0xce, +0x47,0x63,0x80,0xd9,0x45,0xb1,0xa1,0x30,0x3c,0xc8,0x5e,0x54, +0x3d,0xb1,0x20,0x83,0xed,0x04,0x11,0x3b,0xd5,0x60,0x5a,0x18, +0xb9,0x8d,0xd6,0x04,0xd0,0x08,0x98,0x8f,0x58,0x8a,0x0e,0x2c, +0x26,0xa4,0xaa,0x4b,0x22,0x34,0x1b,0x85,0x42,0xd9,0xe5,0x38, +0x88,0xc6,0x08,0x88,0xff,0xd4,0x9b,0x8c,0x82,0x78,0x8c,0x80, +0x78,0xd2,0xc9,0x0d,0x12,0x28,0x06,0xd5,0x94,0x91,0x22,0xd3, +0x42,0x4f,0x5c,0xba,0xfe,0xa4,0xbc,0x94,0xc7,0x1d,0x3a,0xeb, +0x1d,0x0f,0xa5,0xd4,0xeb,0x97,0xb6,0xa5,0x38,0x67,0x02,0xeb, +0x54,0x71,0x68,0xa1,0x2a,0x1f,0x4d,0xe7,0x93,0xfc,0xb8,0x77, +0x99,0xef,0xd3,0x55,0x1b,0x19,0x42,0xa5,0x97,0x8c,0x78,0xbb, +0xe4,0xd5,0x00,0xd8,0xc0,0xbe,0xb2,0x0b,0x66,0x6d,0xec,0xdb, +0x03,0x0c,0xe2,0x29,0xee,0xf4,0x62,0x9a,0x5f,0x5c,0x0e,0xfb, +0x91,0x79,0x76,0x94,0xb3,0xd5,0x56,0x3c,0xad,0x08,0x8a,0xd7, +0x2e,0x07,0xf9,0xb0,0x8f,0x26,0x17,0x76,0x78,0xb9,0x0e,0x54, +0x1c,0x06,0x53,0x65,0x27,0x7e,0xad,0xe6,0xf1,0xa0,0xbb,0xee, +0x67,0x45,0xe4,0x6c,0x47,0x20,0x42,0x55,0x6d,0x93,0xc7,0x32, +0xe3,0xf3,0x0f,0x55,0xcd,0x82,0x68,0x60,0xe3,0xe7,0x1f,0x9a, +0xda,0x70,0x5d,0x86,0xb7,0x14,0xab,0x1b,0x37,0x82,0x9d,0x87, +0x6e,0xdb,0x2b,0x67,0x85,0xe9,0x5b,0x1a,0x14,0xad,0x6a,0x10, +0xe0,0xa0,0x76,0xab,0xf9,0x67,0xd6,0x87,0xba,0x98,0xde,0x2a, +0x5e,0x34,0x98,0xc6,0x64,0x17,0xc9,0x32,0x21,0xd4,0x32,0xa2, +0x08,0xc7,0x4c,0x2e,0xf4,0x0d,0xca,0xa7,0xde,0xa3,0xf9,0xf1, +0xa2,0x61,0x57,0x97,0xb9,0xcd,0xf3,0x8f,0xf1,0x40,0x54,0x14, +0xe6,0x37,0xd9,0x60,0xb1,0x68,0xb9,0x87,0x5e,0xa4,0xea,0x03, +0xb1,0x3f,0x28,0x45,0xaa,0x0e,0x7d,0xcb,0x39,0x9a,0xcb,0x91, +0xa4,0x6b,0x2d,0x6d,0xaf,0xea,0xdd,0x1c,0x49,0x0e,0xd1,0x73, +0xcf,0x19,0xae,0x5f,0x64,0xd1,0x56,0x54,0xaf,0x47,0xbf,0x8b, +0x5e,0x64,0xda,0x21,0xe4,0x60,0x4a,0xa6,0x59,0xa7,0xb7,0xa8, +0x7f,0xeb,0x4b,0x44,0x35,0x7c,0x06,0x78,0x0d,0xdb,0x98,0xb3, +0x89,0xfd,0xfa,0x1f,0x2e,0x6a,0x64,0xbf,0xfe,0x97,0xfd,0xf8, +0xdf,0xff,0x17,0x7d,0x69,0xb8,0x6f,0x50,0xff,0xb0,0x84,0x44, +0x0f,0x91,0xf8,0x3b,0x21,0x01,0xcc,0x0b,0x63,0x76,0x30,0xe6, +0xaf,0x12,0x13,0xbd,0xb7,0x10,0xd7,0x4b,0x00,0x61,0x1c,0x0f, +0x6e,0xd1,0xde,0xc7,0x58,0xb7,0x2d,0xda,0xb0,0x25,0x1a,0xf2, +0xa1,0x3b,0x40,0x19,0x14,0x43,0x92,0x63,0xd6,0x45,0x2c,0x81, +0xb5,0xb7,0x50,0xb1,0x22,0xb9,0x87,0x3f,0xa8,0x51,0x81,0x6f, +0xef,0x51,0x05,0x8d,0x19,0xf1,0xf0,0x78,0x36,0xc9,0xfc,0x4a, +0x47,0xd9,0x61,0xcd,0x98,0xc2,0x72,0x41,0xdf,0x0d,0xa8,0xb9, +0xc1,0x63,0x10,0xd5,0x4e,0xd1,0x1b,0x7c,0x59,0xcf,0x8b,0xc0, +0x37,0xa2,0x2e,0xb9,0xaf,0x44,0xbd,0xad,0x76,0x15,0xe5,0x03, +0x01,0xb2,0x0a,0xca,0x5b,0xfc,0xa8,0xc9,0xaa,0x04,0xf4,0x5b, +0x23,0xb5,0xa5,0xe1,0xfc,0x06,0x95,0x50,0x70,0x0a,0x78,0xd2, +0xaf,0x9d,0xde,0x6f,0x2f,0xbb,0xcd,0x88,0x39,0x9e,0x61,0x74, +0x59,0xe9,0x48,0xbf,0x97,0xed,0xf5,0xa5,0x2f,0xfc,0xf6,0x69, +0x44,0x11,0x78,0xa6,0x41,0x28,0xf2,0xbd,0x5a,0x9b,0xfe,0x6a, +0x52,0x13,0x0b,0x01,0x17,0xd7,0xda,0xd8,0x5d,0x91,0xfa,0x12, +0xba,0xee,0x89,0x9a,0x15,0x3d,0x9f,0x70,0x3d,0x0d,0x48,0x68, +0x3b,0x1d,0x68,0x24,0xf8,0xdd,0xeb,0x8c,0xe8,0x9e,0x8b,0xe6, +0x54,0xb4,0x5e,0x0f,0x87,0x97,0xf3,0x95,0x60,0xd4,0xc8,0x7c, +0x66,0x81,0x0e,0xea,0x75,0xce,0x8d,0x30,0xa4,0x7a,0x29,0x0e, +0x74,0x85,0xe0,0x19,0x88,0xc5,0x5b,0x42,0x62,0xe5,0xbf,0x55, +0xc6,0x93,0x8f,0x95,0x0b,0x95,0x2c,0x16,0xab,0x2a,0x01,0xc2, +0xcb,0xd1,0x0d,0xcc,0xc3,0x95,0x90,0xfb,0x80,0xb6,0xa2,0xc1, +0xc8,0x1e,0x3f,0xa3,0x79,0x49,0xbc,0xdc,0x8d,0x60,0x01,0x61, +0x6a,0x6a,0x2c,0x65,0x88,0x5a,0xdf,0xf1,0x87,0xd8,0x3e,0x17, +0xbf,0x26,0x94,0x89,0x15,0x51,0x69,0xbc,0x53,0xc9,0x84,0x1a, +0xf5,0x29,0x88,0x73,0x97,0xa3,0xb4,0x7c,0x54,0xc7,0x79,0x96, +0xcb,0x02,0x71,0xf0,0x1c,0xf5,0xb4,0x81,0x1c,0xea,0xe5,0x78, +0x66,0xb0,0x6d,0xac,0xe4,0xc9,0x0a,0xea,0x64,0xff,0xe8,0xe2, +0x9a,0x6d,0x54,0x3e,0x4a,0x3a,0xf8,0x62,0x58,0xf4,0x3f,0x03, +0x62,0x31,0x8a,0x50,0x1c,0x77,0x28,0x50,0x4d,0x3b,0x91,0x4c, +0xe8,0x47,0x8d,0xb1,0xc9,0xa4,0x34,0x55,0x8e,0xe3,0x64,0xa4, +0x9f,0x90,0x80,0x24,0x0f,0x39,0xac,0x70,0x58,0x0b,0x62,0x2e, +0x5c,0xa0,0x6b,0x7c,0x34,0x1c,0xe2,0xe7,0x72,0x02,0x8e,0x31, +0x55,0x9f,0xdc,0xfb,0x96,0x67,0x54,0xb3,0xbd,0x22,0x93,0x1a, +0x37,0x98,0x09,0xb4,0x04,0xaf,0x30,0xcf,0x96,0x09,0x72,0x59, +0xdd,0x4a,0x57,0x8d,0xb8,0x93,0x60,0x52,0x0b,0x19,0xb0,0x90, +0x8a,0xcb,0x0a,0xe4,0xe0,0x8d,0xd6,0x97,0x07,0x9f,0xaa,0x5d, +0x62,0x4f,0x1c,0x1c,0xee,0x1d,0xed,0x9c,0x1c,0x1c,0x1d,0x17, +0x0d,0xbd,0x52,0x96,0x24,0x21,0x08,0xcd,0x4b,0xb1,0x7c,0xdf, +0x44,0x72,0xc9,0x5c,0x19,0xb6,0x31,0xda,0xf5,0x33,0x95,0x2d, +0xba,0x66,0xfc,0x03,0x12,0x1b,0xa5,0xe3,0x0a,0x29,0x8a,0x8d, +0x0e,0x9a,0x5c,0xac,0x3a,0x92,0x23,0xc9,0xc8,0x1a,0xc6,0x15, +0xea,0x13,0xf3,0xb0,0x3a,0x69,0x69,0xee,0x7b,0xc0,0xc8,0xae, +0x14,0xb1,0x52,0x72,0x78,0x00,0x3c,0x55,0x05,0x19,0x03,0x51, +0x9e,0x26,0x3e,0x07,0xd4,0xac,0x67,0x06,0x5b,0xdf,0x57,0x14, +0x8b,0x96,0xc6,0xcb,0x33,0xc7,0xd2,0x38,0x75,0x1f,0xaa,0x95, +0xf1,0xc4,0x69,0x2d,0x93,0xaa,0x79,0x26,0xb9,0x1b,0x2d,0xce, +0xaf,0xa6,0x83,0x2b,0x56,0x98,0x53,0x72,0x21,0xf5,0xcb,0x7c, +0x6c,0x1f,0x53,0x94,0x26,0x0b,0x90,0xae,0x57,0xac,0xc6,0xdc, +0x38,0xa5,0x26,0xbd,0x4f,0x5c,0x34,0xa3,0xa2,0x26,0x47,0xb5, +0x56,0x3a,0x6e,0x7e,0x80,0x37,0x97,0x27,0x19,0x92,0x82,0x03, +0x45,0xb3,0x85,0x01,0xb1,0xa4,0x39,0x17,0xe2,0x66,0xd3,0x6d, +0x9f,0x8b,0xca,0xc9,0x8c,0x5d,0x4b,0x26,0xd8,0xaf,0x93,0x36, +0xe4,0x91,0xe7,0x5a,0x9b,0xa7,0x67,0xfd,0xde,0xc6,0x65,0xf7, +0xfe,0xfb,0xe5,0xe6,0x00,0x66,0x6d,0x15,0x2f,0x9b,0x8f,0x06, +0x78,0xc3,0x54,0xe3,0x1a,0x6b,0xa7,0x67,0x67,0xf3,0xa8,0x71, +0x8d,0x0b,0x64,0x37,0x92,0x39,0xd0,0xc8,0xbe,0x37,0x53,0xc1, +0x8d,0x71,0x6c,0xa2,0x05,0x47,0xd8,0xee,0x4e,0x76,0xf1,0x28, +0x92,0x16,0xc6,0x37,0x23,0xb4,0x51,0xfa,0xd9,0xb4,0x7e,0xa3, +0x9d,0x0f,0xc0,0x9e,0x37,0xdb,0x3b,0xde,0xdd,0x39,0xdc,0x3b, +0xbd,0xb8,0xee,0xb6,0x2d,0x14,0x88,0xee,0xc0,0xbf,0x14,0x84, +0x05,0xd7,0x65,0x4b,0xbb,0x5a,0x44,0x67,0x67,0xdc,0x62,0x9b, +0xb4,0xe5,0x98,0x3c,0xf4,0x1e,0xf7,0x74,0x12,0x48,0x5c,0x8d, +0x47,0xb8,0xad,0xeb,0x5b,0x69,0x46,0xca,0x3f,0x0f,0xb3,0x5f, +0xce,0x83,0x07,0x63,0x72,0x16,0x34,0x75,0xc3,0xe3,0x98,0xbe, +0xee,0xd0,0x9f,0x46,0xe2,0xb0,0x1e,0x15,0x91,0x09,0x49,0x61, +0xb9,0x40,0x6b,0xda,0x8b,0x05,0xf2,0x64,0x46,0x17,0x15,0xef, +0xad,0x26,0x3e,0x4d,0x6d,0x54,0xc4,0x27,0x79,0x33,0x03,0xe1, +0xa3,0x92,0x84,0xca,0x6c,0xd0,0x32,0xda,0x29,0xf4,0x5b,0xf4, +0x2c,0x4a,0x12,0x45,0xcf,0xd0,0x87,0x6a,0x0d,0xf6,0x1c,0x1b, +0xf9,0x37,0x4a,0x00,0x49,0x61,0x69,0x8c,0x13,0x2e,0x6e,0xe5, +0x0f,0x55,0xd8,0xf1,0xfe,0x44,0xad,0x41,0x06,0x05,0xe8,0xe8, +0xde,0x20,0xa0,0x08,0xad,0xaf,0x63,0x2b,0x4e,0xb7,0xba,0x76, +0x12,0x8b,0x0b,0x8e,0x53,0x19,0x1d,0xbb,0xb4,0x92,0x0a,0xb3, +0xe2,0x68,0xb8,0xef,0xe7,0x60,0xf0,0x0e,0x00,0x11,0xd4,0x28, +0xc4,0x11,0x0c,0xce,0xf2,0xb4,0xdb,0x34,0xed,0xb4,0x13,0x09, +0x73,0x0c,0xc7,0x9c,0x75,0xfc,0x69,0xcc,0xa1,0x97,0x69,0x78, +0x09,0xc5,0xf4,0xd4,0x70,0x63,0x01,0xdb,0xd3,0x48,0xda,0xbf, +0xec,0xa6,0x06,0x1f,0x00,0x19,0x1b,0xcd,0xaf,0x2a,0xdd,0x8b, +0x79,0xbc,0xbf,0xe5,0x7c,0x2e,0xc9,0x9a,0x5b,0xb2,0x1c,0xc1, +0xd2,0x65,0x69,0x41,0xb8,0xb7,0x39,0xac,0x11,0x3c,0x36,0x4c, +0x72,0xbd,0x0d,0xd4,0xd3,0xe0,0xae,0x06,0x3a,0x78,0x0e,0xa1, +0x6d,0x0e,0x6e,0xe3,0xe3,0x01,0xc5,0xdb,0x61,0xb6,0x40,0xc4, +0x76,0x10,0xb3,0x8d,0x51,0xcf,0x83,0xa8,0xe7,0xdd,0x36,0x44, +0x75,0xaa,0x78,0x66,0xd0,0x17,0xcf,0x89,0xc6,0x47,0xcf,0x97, +0x6e,0x4a,0x3f,0x4f,0x20,0xbc,0xfd,0x78,0xc9,0x6d,0x2e,0xb9, +0xed,0x4b,0x6e,0x63,0xc9,0xc7,0x0b,0x72,0x39,0x35,0x08,0xa7, +0x26,0xb5,0x83,0xd0,0xd8,0xe0,0x8e,0x67,0x80,0x2d,0x54,0x1b, +0xd0,0x13,0x09,0xe4,0x2b,0xd8,0xa3,0xc2,0x34,0x82,0xaf,0x19, +0x1e,0x32,0x4e,0x7a,0x10,0x9c,0xd4,0x22,0x2e,0x21,0xe5,0x5a, +0xc9,0x52,0x66,0x0b,0xcc,0x48,0x7b,0xbd,0x49,0x48,0x15,0xac, +0x4e,0x4f,0x78,0xab,0x48,0xd6,0x5e,0xd7,0x2f,0x07,0xc3,0x19, +0x3e,0xba,0x0e,0xf6,0x8d,0xaa,0xfa,0x9b,0xe9,0x15,0x4f,0x4f, +0xe7,0xd3,0x34,0x90,0xfb,0xa7,0x5f,0x46,0xb3,0xde,0x67,0x90, +0xfb,0x8f,0xe9,0xc3,0x0a,0xfe,0x27,0x58,0x42,0x8c,0x8c,0xa3, +0xdc,0xef,0x36,0x01,0xec,0xa9,0xf2,0x92,0x5c,0xba,0x05,0x3b, +0x81,0xe7,0xcb,0x6e,0xcd,0xba,0xc4,0xc3,0xec,0xa7,0xc0,0xa4, +0x71,0x6f,0xe0,0x27,0x82,0x71,0xa8,0x34,0x2d,0x8f,0xa7,0xe8, +0x02,0xff,0x57,0x39,0x92,0xc2,0x3e,0x95,0xb0,0xe2,0x37,0x83, +0x5b,0x99,0x75,0x88,0x63,0x75,0xfc,0xaa,0x5a,0x37,0xcf,0xc7, +0x78,0xbb,0xa8,0x46,0x00,0xe4,0x04,0x44,0x3f,0x2f,0x6e,0xd2, +0xf5,0x53,0x09,0xe9,0x75,0x98,0xf8,0x85,0x6d,0x72,0xde,0x32, +0xf9,0xb6,0xc9,0x9f,0x9b,0xfc,0x7b,0x42,0x22,0x40,0xe0,0xc5, +0x96,0xd8,0x33,0xd3,0x9c,0x62,0xab,0x6b,0x66,0x8a,0x17,0xe0, +0xc2,0x38,0xcb,0xb2,0xbc,0x05,0x6b,0x17,0xfc,0x6c,0xf3,0xcf, +0x73,0xfe,0xf9,0x7e,0xb1,0x58,0xcb,0x5b,0x78,0xe5,0xb2,0x8d, +0x7f,0x9e,0xe3,0x9f,0xef,0x13,0x8d,0x92,0x50,0x85,0xde,0x69, +0x73,0xc3,0x42,0xcc,0x3c,0x16,0x25,0xac,0x83,0x16,0x76,0x62, +0x24,0x21,0xa8,0xc6,0xf3,0xae,0x7a,0x5d,0x53,0xae,0xb8,0x40, +0xe4,0x85,0x15,0x93,0x65,0x38,0x9d,0x94,0x62,0xf9,0x07,0xc7, +0x06,0x47,0x00,0x78,0xe0,0x32,0xbf,0xc9,0xa1,0x76,0xd8,0xdb, +0x5a,0x24,0x0b,0x6b,0x36,0xc0,0x9f,0xbb,0xa1,0x31,0xd2,0x16, +0xa4,0x1f,0xd8,0xca,0xe6,0x2d,0x5c,0xb1,0x8d,0xb0,0x71,0x0f, +0x77,0x3e,0xea,0x4d,0xbe,0xa0,0xf7,0xc4,0x91,0x99,0x0c,0xae, +0xae,0xbd,0xb9,0x84,0xa7,0x08,0x81,0x97,0x41,0xb4,0x00,0x58, +0x92,0x23,0x7d,0x32,0xf1,0x9f,0x52,0x4c,0xd3,0x06,0xf5,0x56, +0x05,0x26,0x99,0xd4,0x3c,0xcc,0x2f,0x81,0x96,0x07,0xfd,0xfe, +0x30,0xff,0x06,0x14,0xb0,0x74,0x90,0xd0,0x61,0x50,0x41,0x1c, +0xe3,0x51,0x14,0x56,0x1d,0x96,0x08,0xc3,0x21,0x59,0xaf,0x33, +0x00,0x15,0xb1,0xba,0x15,0xe7,0x03,0xdd,0x88,0x7f,0x46,0x1f, +0x12,0xa0,0x72,0x47,0x16,0x50,0x5c,0x8d,0x11,0x3d,0x0b,0xa6, +0x13,0xcc,0x38,0x78,0xea,0x27,0x91,0x78,0x46,0xdc,0x4e,0xca, +0x73,0x0d,0xa8,0x96,0x48,0x23,0x5a,0xa2,0xd6,0x3d,0xfc,0x6b, +0xc3,0x3f,0x5a,0xd8,0x7c,0x59,0x39,0x30,0x26,0x06,0xb9,0x8b, +0x76,0x04,0xf1,0x22,0x6c,0x4d,0x88,0x11,0x0a,0x24,0x89,0x57, +0x65,0x57,0xa5,0xe4,0x98,0xcb,0xc7,0xc0,0x2c,0x4e,0xab,0xbb, +0xc4,0xe2,0xcb,0x3b,0x02,0x76,0xfb,0x55,0x82,0xc4,0x9e,0xbf, +0x82,0x66,0xa9,0xea,0x4e,0x07,0x28,0x46,0x4a,0x08,0x0d,0xfe, +0xd0,0xa3,0x60,0x17,0x13,0xd4,0x17,0xba,0xf5,0x52,0xfa,0x23, +0xef,0x83,0x46,0x86,0x5e,0xc6,0x68,0x84,0x94,0x53,0x0f,0x10, +0x83,0x58,0xf0,0x5a,0x13,0xa1,0xc6,0x76,0xc8,0x22,0x4a,0x5c, +0x8f,0x60,0x99,0x36,0x15,0x0c,0x29,0xc6,0x6e,0xb9,0x0c,0x57, +0x88,0x33,0xd3,0x63,0x61,0xe3,0x0a,0x5e,0xce,0x04,0x3e,0x09, +0x04,0xb2,0x5c,0xc5,0x9e,0x25,0xd2,0x8d,0xc4,0xf4,0xd5,0x88, +0x07,0xba,0x0a,0xaf,0x14,0xf1,0x92,0xa3,0xf2,0x37,0x64,0xee, +0x26,0xab,0x1a,0x0d,0x58,0x45,0x6f,0xe7,0xea,0xb9,0x28,0xdf, +0x30,0x50,0x64,0x57,0x86,0x86,0x6b,0xd2,0xc3,0xe2,0xef,0x15, +0x38,0x0d,0x46,0x23,0xec,0x71,0x77,0x0e,0x5f,0x3c,0x75,0x9f, +0x7a,0x63,0x21,0x65,0x59,0xdb,0x22,0xba,0x5c,0x4a,0x95,0x6c, +0xb3,0x49,0x8d,0x81,0xee,0x37,0x1d,0xef,0x6e,0x9b,0x68,0x63, +0xca,0x9e,0x18,0xd5,0x38,0xeb,0x68,0xde,0x90,0xe0,0xc4,0x32, +0x6a,0xa0,0x85,0x4f,0xc5,0x16,0xf3,0x42,0x67,0x66,0x11,0x3e, +0x34,0xa5,0xd9,0xc9,0xb0,0x0a,0x1c,0xf9,0xe6,0x76,0x38,0x40, +0x6f,0xae,0xae,0x9e,0xda,0xf9,0x7c,0x56,0x71,0xb4,0xb8,0xa5, +0x57,0x72,0x3c,0x5e,0xac,0x5d,0xf4,0x46,0xb4,0x5a,0x9c,0xe7, +0x52,0x9a,0xfc,0xa3,0xbb,0x15,0x83,0x10,0xf5,0xd8,0xf9,0xfb, +0x1d,0xf1,0xe7,0x5e,0xe6,0x92,0x82,0xa0,0x64,0x10,0x8e,0xa8, +0x33,0x3b,0xd6,0x98,0x10,0xbb,0x29,0xf1,0x6a,0xe9,0x22,0x61, +0xd5,0xaa,0x8f,0x86,0xe3,0xab,0x01,0x94,0x3c,0x38,0x5a,0xd5, +0x4b,0x1d,0xea,0x25,0x2e,0xa8,0x70,0xae,0x20,0xcc,0x4e,0xf5, +0xba,0xe0,0xca,0x24,0xa8,0x1e,0xe6,0xfb,0xd7,0xc9,0x21,0xa9, +0xeb,0x18,0xde,0x8d,0x25,0xc5,0x36,0x28,0x1c,0xab,0xe6,0xb3, +0x24,0xef,0xbc,0x7b,0xb9,0x7a,0x3e,0x7f,0xd5,0x84,0xd6,0x00, +0x93,0x65,0x09,0x0f,0x8a,0xbf,0x2f,0x32,0x95,0x5f,0xe6,0x20, +0x1a,0xcc,0x56,0x12,0x1b,0xec,0xa2,0xc8,0xd7,0xda,0x93,0xeb, +0x4d,0x4c,0xd8,0x09,0xbe,0x82,0x62,0xd5,0x93,0x9c,0x5f,0xac, +0xf6,0x86,0x2b,0x29,0x3d,0x83,0x15,0x61,0x0d,0xff,0x64,0xf2, +0x99,0x3d,0x8e,0x8d,0xaf,0xb0,0x84,0x8b,0xae,0xb1,0x88,0x4d, +0xaf,0xdf,0x1f,0x90,0x07,0xec,0x15,0xb8,0xfc,0x00,0xf5,0xbf, +0x80,0x7f,0x3f,0x20,0x22,0x2f,0x9e,0x80,0x87,0xae,0xac,0x84, +0x89,0xaf,0xad,0x8a,0x32,0xe4,0x5d,0x31,0xf9,0x84,0x20,0x8c, +0x42,0x54,0x1a,0x11,0x79,0x0d,0x6c,0x27,0x0f,0x55,0x5f,0x84, +0xe1,0xbd,0xae,0x05,0x88,0x14,0xb3,0x55,0xa1,0x33,0x17,0x6e, +0x14,0x62,0xf1,0x1d,0x60,0xb1,0x09,0xff,0x7e,0xf5,0x18,0x26, +0x52,0x7c,0x05,0x02,0x73,0x35,0xcd,0xa9,0x48,0xdb,0x09,0x32, +0xb6,0xb5,0x49,0xe7,0x76,0x32,0xb8,0xa1,0x6c,0x69,0x61,0x54, +0xd0,0x79,0x62,0xc7,0xd5,0xfb,0xd7,0xbd,0xa3,0x83,0x72,0xbd, +0xc5,0x32,0x6b,0x58,0xc6,0xca,0xa0,0xe5,0xdc,0xae,0x2e,0xb5, +0x73,0xb0,0x51,0x62,0x85,0x88,0x42,0xb8,0x11,0xb0,0x20,0x63, +0x00,0x29,0xd1,0x59,0xb0,0x8e,0x1b,0x2b,0x43,0x83,0x80,0xe3, +0x0f,0x0e,0x6c,0xb1,0x53,0x55,0xac,0x87,0xb6,0x06,0x5f,0xe6, +0x17,0xc3,0x1e,0x9b,0xa2,0x8f,0xcb,0xd9,0xef,0x55,0x76,0x76, +0x11,0x25,0x99,0xee,0xcb,0x2b,0x75,0xdb,0x66,0x74,0x0d,0x94, +0x88,0x70,0xb1,0x40,0x6e,0xdf,0xb3,0x2d,0x52,0xab,0x58,0xb8, +0x4b,0x90,0x7d,0x45,0x2c,0xf9,0x9c,0x10,0x98,0xd9,0x08,0x98, +0x6f,0xf9,0xa4,0x37,0x44,0xbd,0x2d,0x77,0xb2,0x34,0xa2,0xbb, +0x50,0x7e,0xd7,0xdd,0xc6,0x40,0xe6,0x7b,0xcb,0x40,0xd3,0xf1, +0x18,0x1f,0xcf,0xe9,0x63,0xba,0xab,0xc6,0xa5,0x89,0x1e,0x88, +0xc7,0xae,0x1f,0xa5,0xf7,0x77,0x7b,0xc3,0xa1,0x8d,0x34,0xee, +0x6d,0xb9,0x7c,0x90,0x71,0xa6,0x24,0x05,0x70,0x21,0x10,0x9b, +0x6c,0xcb,0x85,0xdd,0x46,0x27,0xe2,0x16,0x26,0x0c,0x39,0xdd, +0x15,0x3c,0xa5,0x38,0xdd,0x8a,0xef,0xb0,0x11,0x5b,0x5f,0x3c, +0x38,0xc3,0xdc,0x3f,0x3c,0x38,0x3e,0x7e,0xf3,0xe3,0xdb,0x3d, +0xaf,0x00,0x20,0x39,0x03,0x59,0xcf,0xc3,0x61,0x94,0x44,0x3a, +0xc2,0x78,0x37,0x86,0xbc,0xe5,0x2e,0x1e,0xa1,0x53,0x1e,0x75, +0x84,0xbe,0x4a,0xee,0x57,0xcb,0x2c,0xe9,0xe6,0x54,0x1d,0xa3, +0x2f,0x16,0x42,0x47,0xd5,0x8b,0x72,0xc5,0xe1,0x3a,0xe5,0x63, +0x59,0xb9,0xb0,0xde,0xcb,0x21,0x7b,0x25,0x40,0xc6,0xb9,0x70, +0xe8,0xee,0xbb,0x43,0x8f,0x8a,0x7b,0x2b,0x49,0x62,0xc9,0xab, +0x51,0x28,0xf6,0xfa,0x77,0xfa,0x3c,0xab,0xf0,0x40,0xf8,0xc1, +0x1d,0x0f,0x89,0xf6,0xe6,0xd6,0x8c,0x71,0xe8,0x83,0x24,0x10, +0x28,0xa5,0x8a,0x20,0xda,0xdd,0xed,0x77,0xe2,0xbb,0xac,0xac, +0xa3,0x80,0x8e,0x7e,0x99,0x02,0xef,0xea,0xf5,0x3b,0xb1,0x3b, +0x13,0xdf,0x66,0x77,0xe8,0x45,0xe6,0x2e,0x1a,0x80,0x60,0x8f, +0xa6,0x2d,0x9a,0x10,0xf0,0x2f,0x7d,0xcc,0x6d,0xc1,0x3a,0x39, +0xfa,0xe1,0xba,0xe7,0x4c,0xf0,0x89,0x5a,0x59,0xf0,0x81,0x41, +0xf8,0x48,0xbc,0x97,0x83,0x07,0x6f,0x37,0x82,0x26,0xa2,0x37, +0xb1,0x87,0x5a,0x53,0xa5,0x6c,0x11,0xf6,0x06,0x35,0x4a,0x79, +0x53,0x0d,0x47,0x28,0x98,0x8c,0x97,0x23,0x3b,0xfd,0x58,0x33, +0xdf,0xab,0x03,0xf1,0x1e,0x00,0x0f,0xcc,0x13,0x58,0xac,0xd5, +0x79,0x0f,0xef,0x13,0xfa,0xe3,0xda,0x4a,0x61,0xba,0x4d,0xb6, +0x3a,0x1c,0xaf,0x33,0x51,0x52,0x1e,0xed,0x24,0x5a,0x29,0x74, +0x86,0x5b,0x86,0xae,0xe3,0x0f,0x01,0xa2,0x9d,0x20,0x14,0x42, +0x48,0x39,0xf4,0x55,0xbb,0x0c,0x5d,0x5e,0x7c,0xb3,0x8f,0x0e, +0x67,0x13,0xf4,0x06,0x16,0x4c,0x3f,0xcb,0xb5,0x58,0xb5,0x0a, +0xc6,0x9a,0xb2,0x59,0xad,0x2a,0xff,0x6d,0xf9,0x0e,0x6f,0x4e, +0x52,0x4a,0xa1,0xfa,0xf0,0x10,0x8a,0x7e,0x5b,0xf2,0xbb,0x2d, +0xbf,0xcf,0xe5,0xf7,0xfb,0x2e,0xdd,0xd8,0x38,0x9a,0x94,0x95, +0x2a,0xbb,0x6b,0xff,0x53,0xc8,0xd2,0x6d,0x5a,0x95,0x54,0x59, +0x5e,0xb0,0xb4,0x57,0xf7,0x57,0x7c,0x97,0x00,0xe4,0xb2,0x6b, +0x17,0x8b,0xb5,0x2d,0xa2,0x8c,0x6e,0x35,0x65,0x84,0x85,0xc3, +0x59,0xef,0x61,0x0a,0xe1,0xd8,0x70,0x62,0xdc,0xa7,0x5b,0x94, +0xd0,0x4f,0xad,0xae,0xb5,0x95,0x2c,0x9f,0x40,0x5a,0x8f,0x32, +0x12,0x4f,0x60,0xd0,0x6c,0xba,0x26,0x21,0x4a,0x51,0xa8,0x05, +0xd4,0x02,0x99,0x0a,0xb8,0x4e,0xcb,0x1b,0xd3,0xff,0xe9,0xf4, +0x12,0x21,0x3b,0x4c,0x76,0x59,0x46,0xf1,0xfe,0xc7,0x9d,0xc2, +0xa8,0xb6,0xe8,0x19,0x69,0xd7,0x7f,0x3b,0xfb,0xff,0xe4,0x7c, +0x9f,0x57,0x74,0xf9,0xf2,0x81,0x2e,0x2f,0x6c,0xf0,0x91,0x91, +0xf0,0x9a,0xcf,0xbb,0x44,0x10,0x17,0xfc,0xd1,0xa7,0xeb,0x2d, +0xd8,0x36,0xb5,0xbd,0xf9,0xb3,0x60,0xac,0x1c,0x2e,0x72,0x0c, +0x0f,0xe1,0xd4,0xd9,0x93,0x4b,0x99,0xb5,0x24,0x1c,0xfa,0x07, +0x87,0x6c,0xf9,0xd4,0x21,0xe3,0xbe,0x42,0xaf,0xb2,0x3c,0x66, +0x1e,0xc5,0xe2,0xe9,0x8e,0x4d,0xc9,0x5c,0x16,0x3c,0xdb,0xe1, +0xf2,0xa7,0x36,0xae,0x49,0x4c,0xd2,0x85,0xa8,0x29,0xe1,0x39, +0x9f,0x57,0x0f,0x83,0x72,0x4f,0x1a,0x57,0x7f,0x02,0x85,0x32, +0x2c,0x73,0x60,0x60,0xe5,0x5b,0xf6,0x98,0x36,0x0b,0xb4,0xcb, +0x8c,0xdf,0xdb,0x67,0xea,0x3c,0xc1,0x68,0x2e,0x9d,0x05,0x3c, +0xdb,0x28,0x89,0x23,0xd3,0xe2,0x87,0x51,0x6b,0x6f,0xa6,0x17, +0x62,0xa3,0xe4,0xd8,0x4c,0x1f,0x4e,0xd9,0x71,0x40,0x99,0xb0, +0x13,0x2b,0x4c,0xdc,0x76,0x37,0xc4,0x43,0xd7,0xac,0x2b,0xd3, +0xf0,0xd5,0xf1,0xcb,0x23,0x87,0xd7,0xf7,0x74,0x95,0x43,0x3d, +0xc1,0x17,0x3c,0x5b,0xe8,0xe1,0x9a,0xe9,0xd0,0x89,0xe9,0x49, +0x5a,0x38,0x82,0x43,0xfb,0x82,0x5b,0x6b,0x85,0xeb,0x86,0xd2, +0x41,0x79,0x6f,0xa4,0xce,0xb2,0x79,0x62,0x88,0x1c,0x8c,0x0c, +0xd8,0x12,0xad,0x93,0x75,0xd7,0x82,0x70,0x81,0xa4,0x5d,0xb2, +0x8d,0xe0,0x74,0x6d,0x11,0xc0,0x8a,0x4b,0xe8,0x96,0xef,0x1a, +0x44,0xb4,0xd9,0x9f,0xc4,0x90,0x38,0xe9,0x1a,0x7a,0xf2,0xc5, +0xd9,0x23,0x5c,0x24,0xb0,0xb6,0xd1,0x24,0x6b,0x1b,0x5b,0x96, +0x3b,0x3a,0x03,0x59,0x4c,0xce,0x38,0x91,0x2b,0xd5,0x47,0x6d, +0x76,0x7b,0xf0,0x6f,0xeb,0x6b,0x8b,0x85,0x55,0xd2,0xf6,0x40, +0x77,0x9c,0x80,0x1a,0xc9,0x03,0x6d,0x15,0x89,0x8b,0x88,0xca, +0x02,0x53,0xca,0xe6,0xca,0x54,0x3c,0x19,0x95,0xd0,0x11,0x64, +0x64,0x42,0x64,0x25,0xbb,0x20,0x8d,0x11,0xda,0x9a,0x7a,0xf3, +0xc6,0x16,0xea,0xef,0xab,0xfd,0xb9,0xf0,0xd2,0x24,0x49,0x6e, +0x81,0x5a,0x92,0xb8,0x28,0x26,0x2c,0xee,0xb4,0x07,0x77,0x67, +0xb0,0xe2,0x8e,0x7c,0x94,0x8f,0xb9,0x3d,0x24,0x62,0xd9,0xe9, +0xf9,0xd5,0x3d,0xe4,0xdb,0xea,0x06,0xcb,0x7e,0xa8,0x4b,0x93, +0xe9,0x2d,0x82,0xfb,0xbd,0x15,0xd7,0x21,0xff,0x16,0x72,0xd5, +0x16,0xfe,0xd9,0xc6,0x3f,0xcf,0xf1,0xcf,0xf7,0x7e,0x9c,0x4b, +0x22,0x9a,0x42,0x85,0x4a,0xbb,0xfa,0x57,0x65,0x69,0x3d,0x9e, +0x65,0xfb,0xf1,0x2c,0xcf,0x1f,0xcf,0xe2,0xb1,0x5e,0x25,0x87, +0x09,0x11,0x91,0xdb,0x0a,0x24,0x56,0xe8,0x9d,0x8c,0xd3,0xea, +0x75,0x71,0x47,0x5c,0xd0,0x2b,0xc2,0x26,0x26,0x1d,0x4e,0x63, +0x21,0xcc,0xe9,0x0b,0xb3,0x93,0x7e,0x01,0x83,0x76,0xbf,0x18, +0x5e,0x68,0xfb,0x8b,0xe2,0xd2,0xd8,0x26,0xca,0x2f,0x8e,0x13, +0x08,0x4d,0x12,0x42,0x1b,0x72,0xf4,0x51,0x24,0x42,0x07,0x3b, +0x76,0x7e,0x89,0x38,0xc6,0x68,0x5a,0x53,0x12,0xd3,0x57,0x10, +0x67,0x62,0x0a,0x38,0xb1,0x80,0x8f,0xe3,0x55,0xaf,0xb3,0xf3, +0x81,0xac,0x88,0x5f,0xc1,0xb6,0x41,0xb9,0x4d,0xad,0xff,0xae, +0x6d,0xda,0xfe,0xf6,0x36,0x6d,0xff,0x77,0x6d,0xd3,0xf3,0x6f, +0x6f,0xd3,0xf3,0xff,0xae,0x6d,0xfa,0xfe,0xdb,0xdb,0xf4,0xfd, +0x7f,0xcb,0x36,0x49,0x00,0xbd,0x59,0x3f,0xfa,0xa1,0xb6,0x2f, +0xee,0x3c,0x85,0x56,0x5f,0x3c,0x4e,0x71,0x0c,0x19,0x44,0x66, +0x9b,0x4a,0x46,0x88,0x8b,0x1c,0x8b,0xcc,0x1d,0x24,0xc1,0x69, +0x8a,0xe4,0x3c,0xc5,0xa4,0xae,0xec,0x04,0x09,0xb3,0x3f,0xe6, +0x5f,0xa6,0xa5,0x55,0xdb,0x26,0xbc,0xb5,0x86,0x31,0x39,0xe8, +0xc4,0xd1,0xcb,0x18,0xa5,0xbc,0xcb,0x51,0xf6,0x9b,0x17,0x61, +0xd6,0x4e,0x71,0x75,0xb1,0xc9,0xa7,0xc2,0xeb,0xe8,0xbb,0xa5, +0xbe,0xb7,0xd5,0xf7,0x73,0xf5,0x0d,0x63,0x69,0xdb,0x5b,0x3c, +0xdd,0x09,0x8e,0x4e,0xd0,0x09,0x3a,0xc8,0x18,0xb0,0x73,0x87, +0xcf,0x6c,0x65,0xf5,0x20,0x72,0x28,0xe0,0x5f,0x19,0xb2,0x88, +0x54,0x9e,0x42,0x29,0x62,0xa1,0x64,0xa4,0x05,0x39,0x2b,0x08, +0xfb,0xe6,0xc5,0xc0,0x6d,0x11,0x90,0x5c,0xfc,0xc1,0x28,0xea, +0x12,0x66,0x11,0xe9,0x25,0x1b,0x1c,0x96,0xda,0x6d,0xfb,0x6c, +0x14,0x39,0x27,0xb3,0x16,0x8a,0x5f,0xd1,0x50,0xe0,0x12,0x9f, +0x1f,0xab,0x16,0x42,0xbf,0x0e,0x22,0xf4,0x46,0x16,0xc1,0x98, +0x4d,0x6b,0xb0,0x32,0xd1,0xa2,0x55,0x5b,0x2c,0x6a,0x1c,0x72, +0xc8,0x27,0x35,0x7b,0x4c,0x06,0x95,0x0f,0x33,0xfc,0x3b,0xcd, +0xa2,0x06,0xeb,0xca,0x75,0xa2,0x69,0x94,0x3e,0x8b,0xe3,0x8f, +0xf5,0xfa,0xc7,0x22,0xb1,0x45,0xcf,0x1a,0x50,0x5f,0x03,0xd5, +0xe8,0x3a,0x1f,0xd3,0x69,0xf2,0x2c,0x69,0x3c,0x3b,0x75,0x91, +0xdd,0x67,0x8d,0x08,0x5b,0xd3,0x00,0x04,0x6a,0x80,0x41,0xbd, +0x5e,0x9b,0xf2,0x01,0x40,0xed,0x1e,0xa3,0x9f,0xd5,0x30,0x7e, +0x8d,0xe7,0x27,0xaa,0xfe,0xc3,0xb6,0x13,0x53,0xa0,0x58,0xed, +0x96,0xb0,0x88,0xf0,0x0b,0x67,0x52,0x4d,0x61,0xeb,0xe2,0x0b, +0xf3,0x12,0xca,0xca,0xf1,0x40,0x7b,0x99,0x70,0xa6,0x25,0x67, +0x9d,0x66,0x68,0x1e,0xe1,0xce,0x45,0x2d,0x5d,0xcf,0xb8,0x76, +0x47,0x78,0x7b,0x6c,0x5f,0xe2,0xc4,0x11,0xda,0xe9,0xfe,0x18, +0x51,0x2e,0xb2,0x52,0x6b,0x4d,0x62,0x56,0x58,0x8b,0xc4,0x3c, +0xf6,0x85,0x57,0xd5,0x84,0xcb,0x2e,0x47,0xea,0x8d,0xe2,0x21, +0xaa,0x0f,0x15,0x5e,0x29,0xb2,0xeb,0xa9,0xfb,0x65,0xe8,0x3e, +0x9a,0xb7,0x20,0x81,0xc1,0x02,0xff,0xaa,0x50,0xb4,0xb2,0x6c, +0x52,0xd9,0xc8,0x44,0x8e,0xbc,0x62,0xfa,0x69,0x80,0x6a,0xad, +0x62,0x70,0x94,0xa2,0xf0,0x8d,0x83,0x35,0x45,0x9a,0x6a,0x37, +0x52,0xc5,0xa1,0xc5,0xdc,0xec,0x07,0xee,0x14,0x3e,0xbb,0xa9, +0xff,0xcc,0x44,0x49,0x0c,0xad,0xf6,0xae,0xb5,0x4c,0x11,0x97, +0x26,0xb2,0x87,0x36,0xd5,0x63,0xb1,0x71,0x35,0x41,0x11,0xeb, +0xa4,0xd7,0x46,0xe1,0xa1,0xd3,0x32,0x78,0x57,0xb6,0xfe,0x9f, +0xab,0x9c,0x9f,0x04,0x16,0x1d,0xca,0x56,0xa3,0x2b,0x0d,0x38, +0x14,0xb3,0xb9,0xae,0xfa,0xc5,0x3a,0x1a,0x72,0xc5,0xac,0x1d, +0xf3,0xf0,0x29,0xad,0x7f,0x6b,0xeb,0xd2,0x97,0x15,0x60,0x75, +0x0b,0x1c,0x68,0x3c,0xb5,0x3f,0x19,0x5c,0x7c,0x34,0x65,0x2c, +0x5c,0x66,0xe7,0x02,0x49,0x9c,0x2e,0xdc,0x95,0x2d,0x6b,0x2f, +0x8b,0x99,0x61,0x5b,0x48,0x79,0x27,0x79,0x6f,0x3a,0x1e,0x29, +0x17,0x38,0xe2,0xf4,0x86,0x62,0xd5,0x69,0x18,0x5d,0x52,0xd0, +0x8a,0x35,0xb5,0x0f,0x63,0xc5,0xc3,0x9b,0x75,0xef,0x76,0x61, +0xdf,0x42,0xa3,0x15,0x75,0xf2,0x53,0x63,0x3d,0x4c,0xb9,0x82, +0x9d,0xd3,0x6e,0x7a,0xbf,0x2c,0xbe,0xa8,0xb4,0xc9,0xa6,0xb8, +0x59,0x62,0x4b,0xfe,0xce,0xdf,0x82,0x41,0x0b,0x14,0x76,0x1f, +0x55,0x5e,0x52,0xb9,0xc9,0x53,0x76,0x38,0x5d,0x92,0xc1,0x13, +0xb6,0x85,0x8f,0xa9,0xea,0x14,0xd8,0x6c,0x6c,0x08,0x78,0xf2, +0xfc,0x16,0x78,0xeb,0xb3,0xd9,0xd1,0x83,0x90,0xab,0xc7,0x77, +0xd7,0x03,0x15,0x15,0xdc,0xfc,0xb9,0xde,0x44,0x05,0x4c,0xd4, +0x0a,0xbc,0xb0,0xaf,0xb7,0x57,0x56,0x69,0x8a,0x3e,0xf2,0x96, +0xae,0xcb,0x8b,0x06,0x91,0xb8,0x1d,0x36,0xbf,0x11,0xc7,0x44, +0xea,0xe1,0xaa,0x1b,0xa8,0x7b,0xeb,0x86,0x30,0x14,0x44,0x80, +0xb5,0x4b,0xa1,0xd0,0xf3,0x05,0xac,0xeb,0x1c,0xdd,0xb6,0x30, +0xfd,0x52,0xc5,0xe7,0x0f,0x38,0x1e,0x08,0xc2,0x7b,0x87,0x70, +0x47,0x0f,0x96,0x6c,0xf5,0xeb,0x72,0xbb,0xe1,0x77,0xc6,0xfe, +0xc5,0x1c,0x73,0xb1,0xb4,0x37,0xcc,0x6c,0x53,0x7c,0x16,0xbc, +0x9f,0xe0,0x83,0x07,0x1a,0x7f,0x1b,0x4f,0xde,0xe4,0xec,0x77, +0x4b,0x7d,0x6f,0x93,0x2b,0xf3,0xa5,0xb8,0x5e,0x4d,0x4b,0xe3, +0x58,0x31,0x02,0x7a,0xc8,0x60,0xe8,0x47,0xe3,0xd9,0xe0,0xf2, +0x4b,0xaa,0x69,0xf3,0x6a,0x42,0x56,0x79,0x9e,0xdc,0x71,0xff, +0x47,0x7a,0x45,0x35,0xd9,0xe3,0x48,0x6d,0x17,0x12,0x4a,0xef, +0xb1,0xcb,0xd2,0x12,0xaf,0xc2,0xb7,0x71,0xf4,0x6b,0x4b,0x31, +0x03,0x13,0xff,0x16,0x40,0x8e,0x6e,0x86,0x7f,0x22,0x7f,0x3c, +0x7d,0xcb,0x67,0x4a,0xa6,0xfc,0xd1,0xde,0x36,0x17,0x71,0xdd, +0x19,0x2b,0x0f,0xf6,0x8e,0xf9,0x75,0xec,0x57,0x5a,0xe0,0x5c, +0x89,0x40,0x72,0x96,0xa9,0x13,0x0f,0x8f,0x86,0x25,0x4f,0x4a, +0x7c,0x10,0xe2,0xd0,0xc5,0x03,0xa3,0x26,0x5c,0x2d,0x2b,0x8d, +0xf3,0x23,0xa8,0x49,0x17,0x24,0x1d,0xf9,0x48,0x43,0x2e,0x99, +0x78,0x7a,0xf8,0x46,0xc4,0x0e,0x55,0xdf,0x66,0x15,0xb4,0xa4, +0xf0,0x63,0x82,0x0b,0xd0,0x0b,0x46,0xa6,0xa3,0x43,0xe5,0x1e, +0x74,0x30,0x15,0xae,0x95,0xa8,0xb9,0xeb,0x5b,0x26,0xd2,0x8e, +0x25,0x56,0x3a,0xd7,0x3e,0x2d,0x8c,0x75,0xa1,0x83,0xab,0x9a, +0xd5,0x95,0x63,0x49,0x9e,0x99,0x5f,0x5f,0x3e,0x91,0x75,0xc3, +0xb1,0x3c,0x13,0x51,0x03,0xa2,0x32,0xc9,0x06,0xb6,0xb5,0x94, +0x4b,0x62,0xb5,0xbc,0x46,0xc0,0xa8,0x20,0xf4,0xa5,0xb2,0xb4, +0x57,0x0f,0xe9,0x7d,0x74,0x9e,0x26,0x99,0x91,0x0a,0x75,0xf4, +0xab,0x26,0x40,0xdb,0xfb,0x89,0xa3,0x3c,0x9d,0x02,0x45,0x31, +0xf1,0xa6,0x21,0x65,0x70,0x64,0xb1,0x6d,0x0e,0x83,0x6b,0x1a, +0x91,0x70,0xe9,0x36,0x83,0xe9,0x51,0x80,0x86,0xc5,0xfc,0x60, +0x3e,0xbb,0x9d,0xb3,0x52,0x40,0x1b,0x29,0xa6,0x10,0xef,0x5a, +0xe8,0xdc,0x98,0x7a,0xb2,0x08,0x28,0x97,0x5a,0xa1,0xdb,0x9e, +0x1b,0xbc,0x94,0x70,0xc2,0x9c,0x86,0x8a,0xcf,0x07,0x4a,0x53, +0x98,0x93,0x58,0x12,0xef,0x54,0x44,0xc6,0x65,0x21,0xb7,0xdc, +0xd5,0xaa,0x95,0x6a,0x85,0xa5,0x97,0xba,0xd5,0x48,0xd2,0x1b, +0x5e,0x44,0x34,0x49,0x1f,0x86,0x56,0xa2,0x8e,0xb2,0x9c,0x40, +0x19,0x2a,0x3b,0x1f,0x95,0x3c,0x56,0xa1,0x53,0x28,0xa0,0x30, +0x42,0xb3,0xd2,0x28,0xa0,0x3c,0xe8,0xdf,0x24,0xe8,0x4b,0x3f, +0x59,0x92,0x0e,0xdf,0x1e,0xad,0x60,0xd0,0x0f,0x91,0x62,0xd5, +0x8a,0x52,0xa0,0xca,0x8b,0x50,0x2c,0x44,0x21,0xa4,0x40,0x8c, +0x6e,0x7d,0x2c,0xf3,0x4d,0x31,0x3b,0xf1,0xf0,0xca,0xf1,0xb5, +0x08,0xfe,0xab,0x98,0x71,0x55,0xc3,0xd1,0x9d,0x45,0x61,0xcc, +0xcc,0x13,0x56,0x46,0x71,0x10,0xfb,0x0d,0xcb,0x23,0xe9,0xde, +0x7d,0xd3,0xaa,0x58,0x9a,0xc1,0x15,0x8d,0x34,0xae,0x0b,0x9e, +0xbc,0x28,0x16,0x11,0x7a,0x6a,0xf7,0x7f,0x2b,0x3a,0x4f,0x5b, +0x0a,0x8b,0x58,0x7d,0xd3,0x12,0xf8,0xc4,0x15,0xb0,0x7a,0x12, +0x5d,0x4a,0xb7,0x57,0x73,0x10,0x24,0x00,0xd8,0x52,0xe0,0x0f, +0x5a,0x07,0x2a,0x90,0x73,0xb1,0xf4,0x37,0xac,0x86,0x95,0xdb, +0x8e,0x87,0x6b,0x0d,0xa1,0xba,0x69,0xa2,0x01,0xf9,0x6e,0x66, +0x50,0xa1,0xcc,0x51,0x81,0x87,0x96,0x27,0x2a,0x58,0x87,0x74, +0x21,0xcb,0xd1,0xec,0xdf,0xdc,0xca,0xda,0xfc,0x43,0x13,0x2c, +0xc5,0x3f,0x78,0x73,0x8f,0xd7,0xc2,0xda,0xa8,0xe4,0x91,0xdd, +0x24,0x17,0xce,0x33,0x4e,0x4e,0xde,0x66,0xad,0x2d,0x65,0xb0, +0x8a,0x9f,0x4b,0x65,0x37,0xf2,0x6a,0x4a,0xed,0xe3,0xc5,0xaa, +0x28,0xdb,0xe7,0x3a,0x99,0xad,0x74,0x30,0xe5,0xcc,0xd0,0xf8, +0xe7,0xf7,0x58,0x8b,0x2c,0xcf,0xf0,0x59,0xb0,0xc3,0x64,0xfd, +0xd7,0xae,0x72,0x2f,0xe5,0x7c,0x4f,0x39,0xbb,0x90,0xfe,0xf0, +0xc0,0x96,0xad,0x72,0x37,0x25,0x7e,0xa8,0xa4,0x94,0x92,0x45, +0xa8,0x39,0xee,0xbc,0x62,0xd0,0x27,0x85,0xbd,0x9f,0x06,0xfd, +0xd8,0x9b,0x26,0x46,0xb3,0x64,0x62,0xdb,0x96,0x8d,0xef,0x59, +0x43,0xb7,0x64,0x6a,0x0c,0xdd,0xbb,0x49,0x18,0x8b,0x1e,0x0f, +0xce,0x87,0xb8,0x6b,0xb3,0x85,0x27,0xf9,0x5d,0x21,0xea,0x02, +0x3d,0x6e,0xa2,0x9f,0xb4,0x20,0xe2,0xa4,0x37,0x18,0xb2,0x53, +0x48,0xb2,0x1a,0x1a,0xe1,0xdf,0xa8,0x2b,0x59,0xb0,0xdf,0xe9, +0xd3,0xe2,0xd4,0x87,0x5e,0x9f,0x8c,0xbf,0xc0,0xf6,0x72,0xad, +0xe5,0xcc,0xe0,0xe2,0x59,0xc7,0x7f,0xce,0x73,0xf4,0x26,0xdd, +0x75,0xc8,0x8f,0xa7,0xb3,0x97,0x34,0x48,0xc5,0x94,0xe1,0x60, +0x3a,0xcb,0x47,0x88,0xfd,0xbd,0x33,0x86,0x3b,0x98,0x92,0xc3, +0x94,0x1f,0x07,0x24,0x93,0x62,0x4a,0xc1,0x0a,0xff,0x21,0xf6, +0x45,0x4c,0x3d,0x52,0x32,0x5f,0xc7,0x1d,0x65,0x9f,0xdb,0x15, +0x68,0x08,0x8d,0x23,0x21,0x4d,0xc3,0xd0,0x91,0xed,0x90,0x21, +0x39,0xf5,0xc3,0xb3,0x43,0x4b,0xe9,0x51,0x85,0x31,0xb8,0xa4, +0x5d,0x8e,0xcb,0xe8,0xaf,0xba,0x1a,0x1d,0xe6,0xbd,0x09,0xe3, +0x15,0x9e,0xff,0x48,0xf6,0xd0,0x89,0x02,0x9a,0x5a,0x1d,0x0c, +0xf3,0x93,0xf1,0x2b,0x3a,0x6a,0x33,0x64,0xce,0xc8,0x9a,0x5b, +0xf2,0x8e,0xb1,0xdc,0x42,0xda,0x9b,0x02,0xb9,0xcc,0x76,0x26, +0x57,0xfc,0xb2,0x8c,0xb2,0x1b,0x7d,0x30,0x88,0x86,0x03,0xff, +0x0b,0xe9,0xe0,0x4f,0xa4,0xff,0xbd,0xe4,0xca,0xc9,0x25,0x04, +0x39,0x89,0xbd,0x57,0x06,0x86,0x52,0x39,0xda,0x1a,0xe5,0x9f, +0xfc,0xb2,0x2e,0x7d,0xce,0x48,0x90,0x37,0x56,0x43,0x04,0xe1, +0x6c,0xed,0x70,0x7a,0x27,0xa6,0x58,0xb2,0x8f,0xc9,0x60,0x28, +0xac,0x68,0x83,0x3f,0x6d,0xb4,0x26,0x87,0x12,0x81,0xb8,0x4c, +0x45,0xf2,0xa8,0xa6,0x9a,0x24,0x8d,0x09,0x31,0x7d,0x08,0xb2, +0x34,0x14,0xa5,0x5a,0x4a,0xf4,0xe9,0x91,0x54,0x4d,0x09,0xe7, +0x55,0xc2,0x91,0x8e,0xc4,0x2f,0x74,0xc6,0x90,0x2c,0x25,0x52, +0xcd,0x3a,0x97,0xcf,0xcd,0x3d,0x1b,0xa1,0x27,0x9f,0x8d,0xf3, +0x53,0x2d,0x88,0xf1,0x73,0xcd,0xf5,0xc3,0x8a,0x69,0x8a,0x39, +0x4d,0x71,0xde,0x76,0xe2,0x62,0x96,0xaa,0xea,0x4d,0x69,0x76, +0xd3,0x97,0xb3,0x1c,0xfe,0x00,0x1b,0x50,0x3d,0xb2,0x34,0xdc, +0x52,0x4f,0x2f,0x14,0xdc,0x03,0xd2,0xb5,0x3d,0x25,0xca,0x36, +0x7b,0xf2,0x42,0xc1,0xfa,0x48,0x1e,0xdb,0x21,0x41,0xce,0xaa, +0xc9,0xde,0x01,0x88,0xe8,0x2b,0xc2,0x87,0x56,0xa8,0x7d,0x36, +0x95,0x69,0x63,0x7b,0xd6,0xc8,0x47,0x76,0x7f,0x39,0x4a,0x5d, +0x65,0xf8,0x12,0x39,0xd5,0x44,0x8f,0xf0,0x53,0xf8,0x87,0xfe, +0xf6,0x52,0x07,0x3b,0xff,0x25,0x5d,0x5b,0x0b,0x11,0x5b,0x8a, +0x51,0x74,0x27,0x5c,0x58,0x98,0xf2,0x88,0x8b,0x83,0x64,0x45, +0xce,0x23,0x6b,0xf3,0x88,0x92,0x63,0x64,0xc3,0xb0,0x00,0x09, +0x7a,0x4d,0xed,0x84,0x0c,0x08,0x0f,0x51,0x1a,0x0f,0xfb,0xf8, +0x43,0x2d,0x4a,0xee,0x2d,0x60,0xbe,0x09,0x42,0x53,0x36,0x97, +0xb1,0x3d,0x4a,0xb7,0xae,0xc7,0x6a,0x16,0xf3,0x7a,0x1d,0xda, +0xe2,0xf4,0x6b,0xc4,0x7a,0xe4,0x64,0x70,0x85,0xfb,0x5a,0xc0, +0xcd,0x57,0xfa,0xf4,0xfa,0x7d,0x71,0xf2,0x15,0xcb,0x46,0xbc, +0xab,0x72,0xf2,0x40,0x1c,0x91,0x2f,0xea,0x98,0xbe,0xed,0x18, +0x38,0x53,0x74,0xac,0xd5,0x87,0xaa,0x6d,0x95,0x63,0x06,0xfc, +0x5d,0xa0,0xb8,0xd7,0xfd,0x16,0x82,0xf6,0x47,0xfb,0x50,0x3d, +0x42,0x70,0xbb,0xe3,0xe1,0x30,0xa7,0xfc,0x9e,0xf4,0x50,0xad, +0xc8,0x0d,0x9a,0x72,0xde,0x58,0x28,0x40,0x94,0x01,0xb5,0x70, +0x13,0x81,0xab,0x40,0xb9,0xdf,0x7b,0xf5,0x70,0xd6,0x09,0x82, +0x44,0xf1,0x10,0xf8,0x31,0xa7,0x57,0x0f,0x83,0xa9,0x28,0x11, +0xdb,0x62,0x49,0x42,0xb1,0x74,0x98,0xfd,0x76,0xf0,0x31,0x57, +0x09,0xf7,0xdc,0x6d,0xf3,0x7c,0x0d,0x5d,0xaf,0xd0,0x9b,0xaa, +0x21,0xe5,0x43,0x05,0x1d,0x49,0x0a,0x13,0xb0,0xa3,0xe5,0x62, +0xd4,0x66,0xb0,0xbe,0xec,0xb6,0xc4,0x4a,0xe5,0xcb,0x7c,0x46, +0x3a,0x5a,0x8d,0x46,0x62,0x1c,0x76,0x99,0xad,0xd4,0x3a,0x3b, +0x73,0x70,0xd6,0x50,0xb7,0x5f,0x72,0x91,0x51,0xa1,0x10,0x86, +0x29,0x56,0xe3,0x11,0x70,0xc5,0x12,0x67,0x69,0x05,0x6f,0x43, +0x5d,0xb4,0x1c,0x38,0x5a,0x00,0xa7,0x83,0x2e,0xd7,0x65,0x43, +0x0f,0xd5,0x86,0x9e,0x90,0x54,0x56,0xb1,0x50,0x52,0xd1,0x5d, +0xdc,0xd9,0x55,0xfd,0x75,0xe0,0x14,0x1b,0x3d,0xca,0x0f,0xf7, +0xd1,0x16,0xb5,0x03,0xc6,0x11,0xd7,0x73,0x37,0x4a,0xae,0xe7, +0x2a,0x4e,0xeb,0xa1,0x5e,0x57,0x5c,0xf7,0x55,0x45,0xd6,0x8e, +0x6b,0x1a,0x5e,0x20,0xe8,0x9e,0xc0,0xf0,0x83,0x7d,0x41,0x37, +0x0e,0x41,0x76,0x58,0xc9,0x5c,0xa3,0x1e,0xce,0x59,0x6e,0x30, +0xe9,0x42,0xbb,0xd2,0x2f,0xfc,0x28,0xde,0x17,0xb3,0xea,0xee, +0xb0,0x55,0x24,0x0f,0xb5,0xb1,0x5e,0x5f,0x7b,0xa4,0xb7,0x5c, +0xbd,0x1b,0x1b,0xd6,0xa9,0x73,0x80,0x3c,0x4c,0x5c,0x7a,0xd8, +0xa3,0x46,0xda,0x42,0xd4,0x63,0x6c,0xe3,0xca,0xad,0x0b,0x2d, +0xdd,0xda,0x84,0xe5,0xca,0x49,0xbe,0x63,0x79,0x89,0x65,0x08, +0x6e,0x7a,0xba,0x5e,0xe5,0x07,0x22,0x4b,0xb1,0xbf,0xcb,0x51, +0x2e,0x93,0x33,0xec,0x5a,0x40,0x05,0x48,0xcd,0xb1,0x0b,0x2b, +0x90,0x91,0x3a,0x5e,0x30,0x9d,0xd9,0xa2,0xcc,0x03,0xf4,0xda, +0xd6,0x87,0x51,0x62,0x13,0xb8,0x92,0x4f,0x95,0xd8,0x1d,0x37, +0x0c,0xaf,0x0a,0x79,0x73,0x53,0x34,0xcb,0x49,0xd9,0xed,0x63, +0x09,0x58,0x0c,0x8d,0x5b,0x2f,0x85,0x43,0xf4,0x07,0xe8,0x48, +0x80,0x1f,0x2a,0xb1,0x15,0x5d,0x7c,0xcb,0xf8,0xa6,0xff,0x19, +0x7f,0xf6,0xa7,0x57,0xe6,0x41,0xc9,0xec,0x69,0x12,0x99,0x99, +0xc1,0x96,0x0b,0x76,0x4f,0x86,0xcd,0x73,0x73,0x47,0x12,0x22, +0x6f,0xc7,0x74,0x0b,0xa5,0xa4,0xf5,0x48,0x1a,0x02,0x4b,0x66, +0x7f,0x4c,0x17,0x1f,0x84,0x21,0x99,0xe7,0x65,0xfc,0x32,0x86, +0xd2,0xf6,0x68,0x38,0x43,0x3c,0x74,0xde,0xc9,0xb9,0xf8,0x46, +0x35,0x56,0x99,0x44,0x95,0x51,0x6d,0xf8,0xcb,0x5e,0x27,0x61, +0xbd,0x85,0x5a,0x61,0xe6,0x78,0x81,0xcd,0x82,0x73,0xab,0x16, +0x19,0x48,0x13,0x36,0x69,0xe3,0x2c,0x06,0x62,0x4b,0x67,0x83, +0x51,0xa1,0x54,0x97,0xe7,0x94,0x13,0xbb,0xdc,0x72,0xf7,0x46, +0x9e,0x42,0xe8,0x0e,0x34,0x96,0xaa,0xc8,0x43,0x0e,0xd9,0x46, +0x93,0x34,0xfc,0xc4,0x49,0xc7,0x48,0x35,0xf3,0x5f,0x3a,0xf4, +0xbe,0x53,0xfc,0x32,0xd0,0xb0,0x26,0x69,0xc4,0x76,0xe9,0xbc, +0x88,0x20,0x87,0x89,0xa5,0x78,0xcc,0x4e,0x06,0x86,0x7a,0xef, +0x64,0xf7,0x6b,0x43,0x04,0x08,0xa7,0xaf,0xf4,0xf8,0x96,0xf1, +0xf5,0x67,0xae,0x6e,0x58,0xc3,0xbf,0xd8,0x53,0x6c,0x46,0x80, +0x93,0x2e,0x47,0x0a,0x9f,0x2c,0xcb,0xb4,0xd4,0x25,0x67,0x97, +0x44,0x80,0xb6,0x95,0xe6,0xd7,0x2f,0x80,0x2c,0xf0,0x01,0x29, +0x91,0x5b,0xf6,0xfd,0x06,0x04,0x1d,0x55,0x9c,0x72,0x6c,0x17, +0x64,0x87,0x62,0x14,0x49,0x0d,0x4c,0x9c,0x99,0x92,0xcf,0x04, +0x41,0xd4,0x27,0x88,0x40,0x04,0xac,0x45,0x0d,0x1f,0xd5,0xc4, +0x9d,0xd0,0x62,0xe1,0xc3,0xca,0xf7,0x68,0xea,0x62,0x05,0x68, +0x23,0x8b,0xda,0xb2,0x2e,0x20,0x14,0x71,0x9a,0xca,0x2d,0x6e, +0x40,0x12,0x73,0x07,0x95,0x44,0xfd,0x56,0xc2,0x9c,0xef,0x48, +0x18,0x64,0xf2,0x28,0xd5,0xd1,0xc3,0x67,0x7a,0x21,0xe8,0x09, +0xce,0xc9,0xdc,0x8b,0x85,0x44,0xa2,0xfa,0x36,0x11,0x7f,0xbd, +0xee,0xb3,0x29,0x59,0x3e,0x21,0xda,0x6c,0x97,0x73,0x97,0x60, +0xeb,0x42,0xed,0xc4,0xce,0x2c,0x97,0xce,0x7b,0x18,0x79,0x81, +0x60,0x53,0x47,0xf4,0xd6,0x0e,0x30,0x25,0xfa,0x40,0x83,0x2b, +0xb3,0xe1,0xc6,0x86,0x6c,0xa1,0xf5,0xc6,0xb6,0x74,0x26,0x43, +0xc6,0x86,0x61,0x5e,0xcb,0x7e,0xda,0x19,0x4a,0xaf,0x91,0xde, +0x38,0x76,0x05,0xfa,0x06,0x46,0xd5,0x8f,0x7e,0xb3,0xb6,0x73, +0x3e,0x26,0xf3,0x3b,0x6b,0x67,0xcc,0xf3,0x60,0xea,0xd4,0x2e, +0xd1,0x6b,0x30,0x2e,0x50,0x68,0xb0,0x07,0xfb,0xbb,0xf6,0x6b, +0x55,0x36,0xad,0x91,0x67,0x59,0xe2,0x30,0x3c,0x24,0x76,0x30, +0x12,0xfb,0x8a,0x82,0x50,0x5e,0x2c,0x4a,0x3c,0x83,0x85,0x1a, +0x8d,0x7c,0xbb,0xc0,0xbc,0x02,0xe6,0x52,0x4c,0x13,0x9e,0x12, +0x3f,0x36,0xbc,0xc8,0x9f,0xf9,0x10,0x44,0x33,0xe8,0xe0,0x48, +0x62,0x8d,0xd8,0x22,0xf6,0x6a,0xe1,0xd0,0xc4,0x39,0x95,0xf3, +0x67,0x39,0x1c,0x10,0x45,0x1e,0x6d,0x84,0x5d,0x0a,0x79,0x97, +0x6b,0xea,0xe8,0x85,0xfc,0x8a,0x17,0x28,0x2b,0x93,0x4a,0xe3, +0x8a,0xa4,0xf2,0x31,0x51,0x52,0x80,0x40,0xdb,0xbf,0x6a,0x08, +0x94,0x54,0x3e,0x55,0xf2,0xc7,0x54,0x3e,0x0e,0xad,0x8e,0x95, +0x22,0xab,0xcf,0xa7,0x02,0x5c,0xca,0x71,0x1e,0x92,0x8a,0xac, +0xde,0x2f,0x57,0xe0,0x14,0x9e,0x96,0xfd,0x53,0x4e,0xc7,0x70, +0xdc,0x71,0x19,0x4a,0xb5,0xc2,0xd4,0xa4,0xf8,0x6a,0xd4,0x1f, +0xe4,0x4c,0x12,0xde,0x6e,0xb9,0xc7,0xa7,0x5e,0x2d,0x28,0x04, +0x51,0x75,0x76,0x14,0xfa,0x22,0x58,0x2f,0x91,0x3a,0xa4,0x5b, +0x33,0xfa,0x7c,0x1b,0xa1,0x28,0xf1,0xe1,0x92,0xf5,0x7a,0xa5, +0x97,0x83,0xa4,0x7c,0x96,0xe7,0x1f,0x35,0x22,0xf2,0x4a,0x0e, +0x28,0xa2,0x5f,0x2d,0x28,0x84,0xc5,0xe9,0x3d,0x60,0xa9,0xa0, +0x3b,0xf9,0xaf,0x05,0x52,0x03,0x65,0x76,0x74,0x4f,0x6b,0x3f, +0xc3,0x79,0x78,0x62,0xca,0x0d,0xf0,0x7d,0xc0,0x01,0xb0,0x8e, +0xca,0x16,0x3b,0x58,0x72,0x6c,0x58,0x75,0x9b,0x41,0x37,0x46, +0xeb,0x7a,0x23,0x4a,0xc6,0xca,0xfd,0x4e,0x94,0xf6,0x93,0x10, +0xd5,0x7f,0xeb,0x0e,0x8f,0x8a,0x87,0x9c,0xa7,0x98,0xde,0xd5, +0x36,0xc3,0x7d,0x66,0xef,0x5f,0xb1,0x90,0x3d,0x2b,0x00,0xc5, +0x65,0x32,0x8c,0x92,0x25,0xc9,0x62,0xa2,0xf7,0xd8,0x61,0xc6, +0x53,0x6b,0xec,0x2f,0x8c,0xf6,0x8d,0xe8,0x7a,0xf2,0xbe,0x19, +0xcc,0xc2,0xa6,0x56,0xb5,0xd0,0x0c,0xac,0xbc,0x99,0xdf,0xdc, +0xce,0x48,0x20,0x56,0x67,0x3e,0xd3,0xd9,0xf8,0x16,0xb7,0x10, +0xbd,0x2b,0x71,0xba,0xdf,0x32,0xe4,0x14,0x22,0x23,0xb4,0x52, +0xea,0x3f,0x5e,0xc9,0x68,0x40,0xe4,0x99,0x6a,0xa1,0x94,0x66, +0xac,0x25,0x80,0x5b,0x4b,0x13,0x3a,0x58,0xd1,0xb9,0xd9,0xfd, +0x4b,0xd1,0x1f,0x05,0x15,0x2a,0x46,0xa2,0x6d,0x30,0xd7,0x0b, +0x3b,0xf8,0xce,0x96,0xdd,0xa5,0xc7,0xa7,0x94,0xa1,0xeb,0x0d, +0xd9,0x9b,0x96,0x13,0x60,0x0b,0xe3,0x62,0x4f,0x41,0x0a,0xc3, +0xb7,0x58,0x50,0xd7,0x18,0x71,0xe5,0xc3,0x6b,0x2e,0xb5,0x37, +0x73,0xcf,0x72,0xa5,0x13,0x0b,0x23,0x1d,0x0a,0x9f,0xb2,0x1f, +0x1f,0x14,0x07,0x0f,0xf7,0xd7,0x48,0xd8,0x55,0x09,0xf2,0xf4, +0x96,0x0e,0x19,0x75,0xeb,0x92,0x62,0x27,0x5b,0x8d,0x6f,0x42, +0xf2,0x91,0xa9,0x45,0x3b,0xbc,0x02,0xa6,0xa8,0x00,0x7e,0x91, +0xc7,0x03,0x83,0xc6,0x7e,0x61,0x73,0xe8,0xe4,0xe5,0xa9,0x6f, +0x68,0x41,0xfa,0xe0,0xc3,0xa6,0x76,0x50,0xb1,0xf1,0x8b,0x5e, +0x15,0xed,0x0d,0xcb,0x64,0xa7,0x77,0x1e,0xe1,0x3e,0x82,0xf6, +0x3e,0xf6,0xfb,0x41,0xb2,0x93,0x3c,0xff,0x3a,0x42,0xd2,0x82, +0x58,0x15,0x9d,0xb8,0x0d,0x9c,0xa3,0x35,0x2f,0xf0,0x95,0xa8, +0x8d,0x9f,0xef,0x5a,0x8a,0x1a,0x3e,0x46,0x4c,0xc3,0x22,0x1d, +0x0d,0x9f,0x42,0x3f,0x4f,0x21,0x91,0xe1,0xe3,0xd4,0xf1,0xff, +0x1d,0xe9,0x38,0x20,0xd3,0x25,0x9d,0x1d,0xfa,0x95,0xc4,0x5f, +0x87,0x94,0x5d,0x01,0x05,0xaa,0xc9,0xc7,0x17,0xf9,0xcb,0x7c, +0x98,0xc3,0x9c,0x2b,0xf9,0xfe,0x3f,0xde,0xdd,0x7b,0xbf,0x7b, +0xf0,0xee,0x64,0xef,0xcf,0x27,0xc7,0x99,0x0e,0xb4,0x45,0xbf, +0x82,0x3d,0xa6,0x4c,0x86,0x64,0xfa,0x14,0xfb,0x36,0x3b,0x8d, +0xf0,0xf8,0x22,0xea,0x1a,0x95,0xfa,0xe3,0xb0,0x77,0xf1,0x91, +0x53,0xbb,0x2c,0x47,0x56,0x16,0x7d,0xfa,0x3d,0x69,0x65,0x71, +0xaf,0xd7,0x54,0x4a,0x93,0x5b,0xbc,0x4a,0x94,0xbe,0xa9,0x56, +0x5f,0xbc,0x5c,0xab,0x4b,0x2b,0xdc,0xdf,0x0e,0xe9,0x8a,0x6f, +0xbd,0x3f,0xbe,0x98,0xb3,0x73,0xbe,0xf0,0x4a,0x17,0xdd,0x6f, +0xfe,0x34,0x1b,0x0c,0xa7,0xfa,0xde,0x16,0xca,0x18,0x57,0xc2, +0xa8,0x6b,0x5c,0x97,0x3b,0xd0,0x1d,0x13,0x37,0xb2,0x37,0xbc, +0x93,0x11,0x87,0xc9,0x3f,0x79,0xaf,0x9d,0x3c,0x36,0xb0,0x5f, +0x96,0x1c,0x1d,0x0f,0xa7,0x39,0x98,0x1e,0xc3,0x5c,0x3d,0xa0, +0x03,0xf9,0xd8,0x97,0x4c,0xd7,0xd6,0x5c,0xa0,0x89,0x6e,0xd3, +0xc4,0xae,0xf4,0x8d,0x3d,0x15,0xf7,0xd7,0x7d,0xa4,0xef,0x24, +0x9d,0xb0,0xc3,0xae,0x3d,0x7e,0xfc,0x72,0x38,0x86,0x69,0xf6, +0x45,0xf9,0x0b,0x37,0x23,0x8f,0x57,0xa6,0xea,0xb7,0x3a,0x03, +0xe8,0xd5,0xd8,0xef,0x98,0xc9,0xa3,0x98,0xf8,0x09,0x41,0x9b, +0xda,0x38,0xa9,0x90,0x93,0x8c,0xb2,0xaa,0x71,0xb6,0xfc,0x64, +0xe4,0x59,0x89,0xeb,0x94,0xaa,0xfc,0xa8,0xa6,0xea,0x9b,0x9a, +0xdc,0xbb,0x8a,0xb6,0xbc,0xa5,0x77,0x89,0x4b,0x2a,0x6b,0x76, +0x63,0xfd,0xc4,0x9a,0x5d,0xfe,0x95,0x35,0xb7,0xa4,0x66,0x4b, +0x86,0x1c,0xaf,0x9f,0x51,0x8d,0x70,0x27,0x9a,0xbf,0x1e,0x0f, +0x61,0xa2,0x9e,0x7c,0xb9,0xcd,0xc9,0x79,0xa1,0x98,0x06,0x77, +0x91,0x9e,0xac,0x67,0x93,0x39,0xb0,0x3c,0x39,0x69,0xb5,0xa2, +0xf0,0x7c,0x84,0x9a,0x15,0x27,0x2a,0xa9,0xe2,0x15,0x88,0x2e, +0xe9,0x75,0x53,0xb0,0x36,0x98,0x0c,0xbe,0x2a,0x75,0xcf,0x89, +0x2c,0x87,0xb0,0x31,0x55,0xc9,0xfc,0x72,0xff,0xe0,0xb2,0xaa, +0xaa,0x55,0x68,0xa1,0xff,0xdd,0x4a,0x58,0x0f,0xbc,0x5e,0x79, +0x00,0x98,0xa2,0x2c,0x0d,0x57,0x99,0xc1,0xc3,0xec,0x3b,0x53, +0x7a,0x5f,0x62,0xc2,0xae,0xb3,0x7e,0x9e,0xac,0x4b,0x9c,0xf3, +0x2f,0x84,0x52,0xe1,0x50,0x9a,0x7c,0x42,0x75,0x38,0x8d,0x9d, +0x3d,0xa5,0xa4,0x1d,0x89,0xab,0x8a,0x2a,0x6d,0x75,0x01,0xa6, +0x17,0xfe,0xd4,0x42,0x8c,0x91,0x00,0x33,0xd8,0x99,0xcd,0x50, +0x3e,0xa3,0x47,0xf1,0x8c,0x52,0xad,0xc7,0x67,0x6c,0x78,0x2c, +0x31,0x10,0xb3,0xed,0x92,0xbf,0x59,0xdb,0xe5,0x0f,0x32,0xcb, +0xdb,0xae,0x11,0xba,0x72,0x4a,0x51,0x6e,0x06,0x22,0x22,0xef, +0x7c,0x75,0xfc,0x62,0xa1,0x43,0xfa,0xc5,0x2f,0x7a,0xaa,0x2f, +0x64,0x4e,0x2a,0x08,0xa4,0xad,0x6e,0x0e,0xd7,0xec,0xf1,0x5f, +0x50,0xa8,0xa2,0xc5,0x98,0x6d,0x55,0x73,0x47,0xe3,0xd1,0x06, +0xc3,0xf3,0x2d,0xef,0x71,0x9b,0x47,0x33,0x58,0x7b,0x7e,0x99, +0x0f,0x28,0xb1,0x27,0x56,0xda,0xd3,0xa0,0x1b,0xb8,0xe9,0x4a, +0xe9,0xef,0x93,0x1e,0xbb,0x70,0x52,0xf8,0xd1,0x17,0x02,0x85, +0xa9,0xfb,0xe5,0x3c,0x17,0xca,0x51,0x9a,0x9f,0x3e,0x52,0x7b, +0x35,0xd3,0xb0,0x78,0x56,0xa2,0x53,0xce,0x8e,0xce,0xbe,0x82, +0x1a,0x53,0x9d,0x27,0x78,0x29,0x29,0x71,0x4c,0x86,0x21,0x36, +0x7e,0xfc,0x74,0xfc,0x62,0xa1,0x43,0x15,0xe3,0x17,0x00,0xa9, +0x68,0x51,0xfb,0x1f,0x26,0x6f,0x55,0xb8,0x5e,0x5f,0xd5,0x59, +0x7a,0x0a,0x54,0x60,0xb1,0xa2,0xa3,0xc8,0x06,0x34,0x32,0x98, +0x2c,0x90,0x42,0x9a,0x47,0x7b,0xc7,0x07,0x3f,0x1d,0x41,0xcc, +0x4f,0x47,0x6f,0x13,0x36,0xc2,0xfe,0xc0,0x1a,0x14,0x74,0x41, +0x65,0x1f,0x54,0x90,0xe8,0x68,0x9a,0x83,0xd0,0x35,0x04,0x2a, +0xfd,0x71,0x38,0xbe,0xf8,0x08,0xad,0x19,0xc2,0x66,0x00,0x09, +0xcf,0xb2,0xf5,0x1a,0x3a,0x42,0x20,0x77,0xe4,0x64,0xdc,0x8d, +0x2b,0xad,0x9d,0x7f,0x21,0x38,0x56,0xb2,0xaa,0xdd,0x12,0x0e, +0xcd,0x5a,0x0d,0x50,0x15,0x0a,0x0d,0x1a,0xae,0x0e,0x89,0x97, +0x2b,0x5a,0xfb,0xfa,0x64,0xff,0xad,0x45,0x1b,0x7d,0xdb,0x1e, +0xf7,0xd0,0x33,0xe7,0xdf,0x41,0x5c,0x0b,0x5a,0x56,0xd1,0x8a, +0xf9,0x68,0xda,0xbb,0x54,0x33,0x0d,0xd1,0x87,0xa9,0x36,0x07, +0x81,0x98,0x6c,0x70,0x60,0xaa,0x9e,0x65,0x14,0xb6,0xec,0x25, +0xe2,0x7b,0xaa,0xa0,0xc6,0x92,0x01,0x91,0x7f,0x6a,0x75,0x6d, +0x27,0xe8,0x20,0xf9,0xc5,0xd1,0xfa,0x54,0xea,0x45,0x5b,0xc5, +0x71,0x88,0x88,0xcf,0x8a,0xb7,0x1a,0x3a,0x2b,0x5f,0xa6,0x57, +0xcf,0xce,0xac,0x62,0x19,0x4d,0x0c,0x93,0x75,0xe6,0x1f,0x84, +0x09,0x9d,0x97,0x46,0xa1,0x5b,0x55,0xbe,0xba,0x26,0x0b,0x35, +0x04,0xb2,0x7b,0x7c,0xfc,0x0f,0xc3,0x00,0x42,0xfa,0x87,0x61, +0xfc,0xe1,0x1f,0x47,0x43,0xcf,0xc1,0x4a,0x60,0xab,0x70,0x4f, +0xcc,0xbd,0xac,0xb6,0xa9,0xfc,0x1a,0xcf,0xf9,0x52,0xff,0x69, +0x84,0x2b,0xa7,0xf2,0xbb,0x2c,0xee,0x5e,0x0a,0xaa,0x9a,0xb9, +0xf8,0xc0,0x04,0x21,0x8e,0x24,0x81,0xbc,0xe8,0x13,0xf3,0x51, +0x41,0xdf,0x41,0x58,0x13,0xe9,0x5e,0x22,0x0a,0x02,0xbd,0x53, +0xb8,0xd4,0x22,0xbd,0x9a,0xf3,0x5a,0x90,0x17,0x25,0x4b,0x2f, +0xca,0xab,0x7c,0xc4,0xb9,0xa4,0x0a,0xe0,0x9c,0xd3,0x41,0x6e, +0x55,0xba,0x39,0x33,0x7b,0xa1,0xb6,0x21,0x74,0x8d,0xac,0x53, +0xe8,0x8e,0x44,0x85,0xd7,0x02,0xa3,0x2c,0xbf,0x7d,0xa1,0xd3, +0xaa,0xd6,0x60,0x5a,0x48,0x3f,0xe2,0xe3,0x60,0x64,0x40,0x17, +0x33,0xbb,0x8c,0xce,0x7b,0xc3,0xda,0x1e,0x7a,0x5f,0xc1,0xd9, +0xdb,0x1f,0xe7,0x6c,0x1a,0x68,0x3a,0xbf,0xbd,0x1d,0x4f,0x66, +0xb5,0x37,0x74,0xc3,0x9c,0xcf,0xd0,0x75,0xdf,0x70,0x3c,0xc9, +0x27,0xb5,0x3b,0xd8,0x54,0xe3,0x88,0xfc,0x50,0xfb,0x1d,0xce, +0x6a,0x86,0x5a,0x43,0x9f,0xe6,0xc0,0xf4,0xfe,0x32,0x9e,0x93, +0xb1,0xe3,0xcb,0xc1,0x67,0x12,0xcf,0x90,0x3d,0xa2,0x49,0x56, +0x64,0x0c,0xd7,0x79,0x0d,0xeb,0xab,0xfd,0xb0,0x06,0xa8,0x22, +0xdf,0x23,0x56,0xf3,0x82,0x64,0x01,0x4c,0x1b,0xdf,0xa2,0xf7, +0x81,0x2f,0xc0,0x6b,0x6b,0x38,0xf7,0x5c,0xb7,0x00,0xd8,0xe3, +0x1c,0x33,0xcf,0x6e,0xd3,0xcd,0x4d,0x88,0x9d,0x5a,0x57,0xc7, +0x1f,0xa6,0xcd,0xf1,0xe4,0x6a,0x13,0x50,0xdf,0xc4,0x23,0xf9, +0x29,0xf2,0xe8,0xf1,0x04,0x70,0x99,0x20,0xc3,0x61,0x6f,0x98, +0x80,0xaa,0xe5,0x6c,0x90,0x9e,0xd1,0x45,0xa3,0x26,0xd1,0xa4, +0x0d,0xd1,0xb0,0x29,0x2a,0x79,0x54,0x55,0xf6,0x63,0x84,0x28, +0x30,0xa3,0x50,0x70,0xa6,0xc7,0xd5,0x46,0x52,0x06,0x4f,0xcf, +0x61,0x1e,0x45,0xe7,0x98,0xcd,0x8a,0xc8,0x41,0x1e,0x89,0xb4, +0x64,0xb8,0x58,0xc4,0xba,0xca,0x02,0x74,0x2f,0xf8,0x7f,0xb1, +0x96,0x23,0x0b,0x2f,0x6f,0x83,0x8a,0xc8,0x3d,0x13,0xea,0xc6, +0x51,0x2c,0x91,0x29,0xc0,0x0c,0x81,0xf0,0x61,0xb8,0x5c,0x0c, +0x4d,0xb1,0x05,0xea,0x12,0xc1,0xbd,0xda,0xa2,0x24,0x6b,0x9d, +0x09,0x0d,0x56,0x50,0xd8,0xaa,0x8b,0x75,0x38,0xbc,0xc2,0x28, +0xbc,0x7d,0xb3,0x1e,0x34,0x85,0x2b,0xe7,0x72,0x05,0x9b,0xee, +0x7c,0xd8,0x41,0x49,0x99,0xc2,0x5b,0x31,0x90,0x42,0xb7,0x18, +0xdd,0x5b,0xf2,0xed,0x14,0x59,0x99,0x6a,0x9a,0xf6,0xf9,0xaf, +0x26,0x03,0xf5,0xee,0x66,0x34,0xbf,0x11,0x45,0x0d,0x7f,0xf6, +0x47,0x6e,0xc6,0xbc,0xa3,0x3d,0x4a,0x41,0xc2,0x39,0xbd,0x80, +0xaf,0xe1,0x2e,0xdd,0x16,0x10,0x6e,0xef,0x7b,0xd3,0xf7,0x51, +0x83,0xf2,0x27,0xdd,0xac,0x70,0xcd,0xa0,0x7b,0x50,0xd5,0x23, +0x57,0x13,0x05,0x70,0xd0,0xa6,0xf7,0xc2,0x9f,0xab,0x20,0x86, +0x03,0xae,0x3a,0xd3,0xc3,0x95,0xc7,0x1b,0x45,0xc0,0x04,0x74, +0x05,0x9e,0x21,0x54,0xbb,0x49,0x2a,0x81,0x5c,0x12,0x15,0x85, +0xfc,0x99,0x5f,0xd0,0xaf,0x7a,0xfe,0x6e,0x9d,0x0b,0x6b,0x26, +0xea,0xb9,0x26,0xa7,0x7a,0xb6,0x29,0x0f,0x8c,0x73,0x88,0x9d, +0xb0,0xb3,0x7b,0x43,0xbb,0x6c,0x3e,0x9f,0x3c,0x66,0xde,0x84, +0xea,0x2e,0xbd,0x51,0x7f,0x32,0x1e,0xf4,0x51,0x71,0x2b,0x8e, +0x37,0x25,0x54,0x8b,0xcf,0xfa,0x8d,0x64,0xb3,0x99,0x7f,0xce, +0x2f,0x62,0x3f,0x68,0xb6,0x9a,0xe6,0xa8,0x77,0x37,0xb8,0x42, +0xcf,0x81,0xe8,0xb6,0x37,0x69,0x82,0xb8,0x32,0xd9,0xb9,0x22, +0x1d,0x09,0x3c,0xc0,0x4c,0x4e,0x5b,0xb0,0x64,0x9d,0x8f,0x3f, +0xe7,0x79,0xb6,0xf9,0x23,0xfe,0x6c,0x0e,0x9a,0x33,0xbc,0xaa, +0x79,0x1c,0x80,0xb1,0x0d,0x08,0xd8,0x38,0x79,0x07,0xe6,0xc6, +0x1c,0xe5,0x57,0xf9,0xe7,0x6c,0xf3,0xe7,0x78,0x7f,0xfc,0xf7, +0xc5,0xa7,0xfc,0xfc,0xe3,0x60,0xb6,0x38,0x58,0xdc,0x4c,0x93, +0xb8,0x93,0x9d,0xee,0x6c,0xfc,0xb5,0x9b,0x6c,0x42,0xd5,0xfd, +0x2f,0xc7,0xb3,0x2f,0xc3,0x3c,0x73,0x1c,0x10,0xa3,0xea,0xf5, +0x20,0xd8,0x9c,0x62,0x16,0x20,0xf7,0xde,0x68,0x3a,0xa0,0x0b, +0x6b,0xbc,0xe1,0x00,0xa1,0x88,0x79,0xde,0x94,0x1c,0x93,0x5d, +0xc6,0x0e,0x98,0x7f,0xef,0x8b,0xe6,0xc3,0x90,0x73,0xfb,0x24, +0x7b,0x88,0x91,0x29,0x24,0xb9,0xf7,0x30,0x2f,0xea,0xa6,0xaa, +0x91,0xe0,0x33,0x25,0x7c,0xf7,0x1c,0xc4,0xea,0x80,0x77,0xb3, +0xd7,0xc2,0x9d,0xf8,0x4f,0xb7,0xb0,0xa7,0x20,0xda,0x4b,0x1a, +0x55,0xd9,0x5a,0x89,0x9c,0x85,0xe8,0x44,0x60,0x7e,0x01,0xfc, +0xe8,0xbf,0xa8,0xb7,0x0e,0x6e,0x7b,0x17,0xc0,0xc5,0x22,0x8d, +0x7f,0xbd,0x1e,0x71,0x57,0xe2,0x4d,0x9d,0xee,0x8f,0x35,0x24, +0x76,0x1b,0x0e,0x8a,0x2c,0x16,0x1a,0x78,0x23,0x3a,0xa9,0xce, +0x96,0x04,0x1d,0x0a,0xe0,0x5c,0xf0,0x41,0x68,0x3b,0x95,0xb9, +0x12,0xb3,0x26,0x04,0x8a,0x9b,0x70,0x87,0x66,0xbd,0xee,0xeb, +0xc0,0x6b,0x38,0xd5,0x80,0xc1,0x54,0xf6,0x0a,0x15,0x23,0xdf, +0xe4,0x26,0x7b,0xc4,0x03,0x5c,0x9f,0x50,0xd2,0x21,0xe9,0x5c, +0x67,0xdc,0xc3,0x84,0x45,0xd3,0x0f,0xe9,0x5a,0xbc,0x66,0x49, +0x5d,0xa2,0x16,0x8b,0x62,0x0c,0x5d,0xfb,0x1d,0xa3,0x1d,0xc0, +0xc5,0xe2,0xfb,0x17,0xae,0x61,0x34,0x6f,0x12,0x03,0xe2,0xfc, +0x35,0x2b,0xb5,0xa5,0xd1,0x78,0xe4,0x43,0xd8,0x21,0x02,0x08, +0xe4,0xaf,0x35,0x87,0x9d,0x16,0x5e,0x16,0x8b,0xca,0xe8,0x17, +0xff,0x4e,0x60,0xf7,0x90,0x03,0xa8,0xfb,0xdb,0x3b,0x62,0x17, +0x78,0x28,0x41,0xee,0x2c,0x60,0x0b,0x4c,0x51,0xf5,0xfa,0xef, +0x60,0x2f,0x8c,0x22,0x96,0xf5,0x0b,0xc4,0x0a,0xb6,0x3f,0x59, +0x91,0x29,0xd6,0xac,0x44,0xee,0x47,0x44,0x0b,0xbb,0x3f,0xb8, +0xdb,0x1b,0xde,0xf8,0xd9,0x77,0x01,0xc4,0x39,0xcb,0xf7,0xd8, +0x78,0x5d,0x1c,0x41,0x32,0x9a,0x1c,0x2f,0x17,0xcf,0xa0,0xa9, +0x51,0x83,0xbe,0x71,0x7e,0x31,0x98,0xa5,0x3e,0xbc,0x0f,0xf3, +0x2f,0xd1,0x3a,0x52,0xea,0xaa,0xa1,0x0d,0x28,0xd0,0x36,0xef, +0x62,0x3b,0x2b,0xe2,0x41,0x54,0x41,0xf5,0xbc,0xbb,0x3c,0x85, +0xe9,0xae,0xc9,0x2e,0x0d,0xd8,0xa5,0x22,0xa3,0x54,0x7d,0x2b, +0x1a,0x49,0xfd,0x67,0x28,0x69,0x9f,0x0c,0x6e,0xf2,0xf1,0x7c, +0xf6,0x24,0x43,0x26,0xee,0x09,0x52,0xb4,0xfe,0xcb,0x57,0x18, +0x36,0x91,0x62,0x66,0xfd,0x97,0x2a,0x23,0x27,0xfe,0x90,0x8e, +0x31,0xc1,0x57,0x27,0xfd,0x7c,0xd8,0x43,0xe3,0x41,0xe8,0x39, +0x64,0x07,0xef,0x85,0xc4,0xb0,0x3d,0xe7,0x78,0xd3,0x77,0x76, +0x28,0xb2,0xf5,0x5f,0x9a,0xf6,0x5d,0xa4,0x35,0x95,0x55,0xb4, +0x69,0x61,0xa6,0x1f,0x07,0xb7,0x04,0x45,0x39,0x10,0xd6,0xb0, +0xeb,0xf5,0x35,0x15,0x74,0x8a,0x94,0xb6,0xb2,0x6c,0xb5,0xce, +0x03,0x5e,0x62,0x95,0x4c,0x3b,0x5c,0x8e,0x02,0x0d,0xc1,0xa2, +0x95,0x0e,0x98,0x2f,0x0f,0xaa,0x11,0x88,0xb2,0xab,0x2d,0x36, +0x3d,0xf5,0x16,0xbc,0x1c,0x4a,0xdd,0xa5,0x6b,0x53,0xa8,0xb2, +0xc1,0x97,0x68,0xc9,0x92,0x7b,0x30,0x31,0x15,0x65,0xb3,0x72, +0x2f,0x4e,0x4f,0x3d,0xe4,0xcc,0x9b,0xf8,0x28,0x9a,0x04,0xc1, +0x9c,0x6a,0x1b,0x2d,0x65,0x9a,0x17,0x78,0x08,0x34,0xcc,0x8a, +0x56,0x55,0xbc,0x00,0xc4,0x61,0x90,0x1d,0xcb,0xc8,0xd0,0xcc, +0xb1,0xb0,0x3b,0xf1,0xc3,0x8d,0xb6,0x3d,0x18,0x71,0x8d,0x79, +0x9f,0x1c,0xc9,0x3f,0xa1,0xb7,0x4c,0x38,0x82,0x82,0x71,0x5c, +0x91,0x35,0x21,0xd7,0x5d,0x46,0xc2,0xc1,0x44,0x59,0xff,0x49, +0xee,0x34,0xaa,0xa7,0x8a,0x36,0xb5,0x61,0xcb,0xa8,0xcb,0x0f, +0x16,0x78,0xe5,0x3c,0x1f,0xdf,0x6f,0x43,0x9c,0xed,0xc8,0x56, +0xeb,0x05,0x31,0x2f,0xe0,0x91,0x10,0x8b,0x76,0x97,0x80,0x97, +0xbf,0xc3,0x2d,0xd6,0x34,0x9f,0xed,0xcc,0x80,0xb5,0x9f,0xcf, +0x67,0x20,0xca,0x61,0xb9,0xc8,0xe0,0xdf,0xc4,0x58,0x18,0x3a, +0x37,0xa7,0x3c,0x19,0x04,0x61,0xd4,0xb9,0xc7,0x40,0x5a,0x01, +0xc9,0xd0,0xa9,0xfc,0xc5,0x78,0x58,0x4c,0xb4,0xf1,0xe6,0x7a, +0x3c,0x9d,0x95,0x4a,0x42,0xdc,0xb2,0x0a,0x1c,0x91,0x51,0x18, +0xbf,0x92,0xe7,0xf6,0x60,0x60,0xf9,0x0d,0x09,0xde,0x26,0xb9, +0x67,0xa7,0xbc,0x22,0xe1,0x6e,0x81,0xf6,0x79,0x84,0xe4,0xda, +0x96,0xdd,0xaa,0x38,0xb3,0x33,0xf2,0x6b,0xf4,0xd5,0x97,0xb6, +0xc8,0xf2,0xcb,0x1c,0xc4,0xbb,0x9f,0xec,0xcd,0x95,0xec,0x7d, +0x4a,0xcf,0x63,0x7c,0xbe,0x8e,0x7f,0x6d,0x6b,0xa3,0xb0,0xda, +0xd4,0x07,0x0b,0x9b,0x25,0xdb,0x41,0xe8,0x5e,0xdf,0xb6,0xc2, +0x45,0xba,0x2d,0x14,0xf6,0x54,0x90,0x83,0xba,0x2e,0x64,0xce, +0xff,0x45,0x4d,0xae,0x24,0x38,0x92,0xcd,0xf1,0x71,0x13,0x65, +0x51,0xa7,0xdd,0xeb,0xaf,0xc8,0xca,0x95,0x2b,0xb4,0x7e,0xcb, +0x5f,0x01,0x5d,0x5e,0xe1,0x85,0x39,0xab,0x95,0x98,0x4b,0x36, +0x01,0xa5,0x74,0xc9,0xb8,0x40,0xf3,0xd2,0xda,0x86,0x82,0x5c, +0x8d,0xe9,0xfc,0x12,0x57,0x1a,0x9b,0x99,0xb7,0xd8,0x14,0x97, +0x45,0x5c,0x63,0x64,0x2f,0x81,0x19,0x78,0x66,0x3f,0x56,0xbd, +0x77,0x2d,0xbf,0x63,0x2d,0x1b,0x07,0xe3,0x4d,0x99,0x45,0x2c, +0x38,0x14,0x54,0x58,0xc1,0x16,0x05,0x6f,0xa6,0xa5,0x51,0x11, +0x5f,0xa7,0x5f,0x7c,0x89,0x8c,0xfd,0x62,0x04,0x13,0x95,0xa7, +0x4f,0x27,0x39,0xf8,0xb7,0x9c,0x66,0xed,0x99,0xf1,0x6f,0x39, +0x9d,0x4d,0x28,0xe3,0xdf,0x72,0xda,0x70,0x70,0x03,0x62,0xda, +0x38,0x32,0xf2,0x51,0x91,0xc3,0x6e,0x51,0x20,0x8f,0xfd,0x2c, +0xe7,0x12,0x95,0x6b,0xc3,0xbf,0xe5,0xf4,0xf1,0x04,0x8f,0xef, +0xa0,0x89,0xf2,0x51,0xce,0x31,0x47,0x39,0x9c,0xeb,0x71,0x9f, +0x92,0xab,0xe8,0xa8,0xec,0x95,0x75,0x57,0x56,0xec,0x7e,0x7e, +0x2a,0xe5,0x0d,0x93,0x1b,0x7c,0xa9,0xc6,0x7e,0xd8,0x13,0x71, +0xcd,0xce,0xa6,0xb7,0x28,0xa3,0x3b,0x7d,0xa7,0x90,0x18,0x42, +0xce,0xfb,0xe8,0x14,0x87,0x6c,0xaa,0xb7,0xc5,0xca,0x9b,0x8f, +0x6c,0x5e,0x5c,0xe7,0x15,0x16,0x0c,0xec,0xde,0xe5,0x43,0xb6, +0xd5,0xfe,0xf0,0x83,0xca,0x2e,0xd7,0xb1,0x1f,0xf8,0x2a,0x76, +0xcd,0xa7,0x9c,0x7e,0xe8,0x5a,0xbb,0x13,0x4e,0x02,0x94,0x8f, +0xad,0xa5,0x91,0x29,0xad,0x91,0x2f,0x58,0x7d,0x63,0x37,0xdf, +0x14,0x79,0x3e,0x1e,0x0f,0xf3,0x1e,0xc4,0x41,0x0d,0xad,0x2c, +0x0b,0x0a,0xb9,0xef,0x2c,0x78,0x22,0x46,0x86,0xe0,0xef,0x0b, +0xe7,0x0e,0xa2,0x60,0xef,0xd2,0x97,0xb2,0xed,0xb1,0x56,0xe5, +0x9e,0x04,0x8c,0x9c,0x2f,0xc4,0x11,0x3b,0x1d,0xc3,0xdd,0xd5, +0x5b,0x24,0x19,0xde,0x5d,0x19,0x8c,0xc7,0x97,0x22,0x61,0xb4, +0xf3,0xb2,0x4b,0x45,0x5e,0x6c,0xb4,0x96,0x3c,0x57,0xf3,0xde, +0x04,0xf6,0x7c,0x15,0x55,0x55,0x3d,0x0e,0xc4,0x14,0xd8,0x68, +0xad,0xd1,0x1d,0xa1,0xf2,0x49,0xbc,0xe5,0xfa,0x97,0xe1,0x39, +0x30,0x7e,0x77,0x97,0xb4,0x43,0x63,0x7e,0xe4,0x89,0x24,0xec, +0x59,0x0a,0x09,0x85,0xa7,0x95,0x76,0xfe,0x5c,0xdf,0x78,0x3c, +0x79,0x74,0xf8,0x85,0x65,0x94,0x86,0x75,0x70,0x43,0x82,0x0c, +0x0f,0x41,0xb2,0x43,0xa0,0x2c,0xdb,0xff,0x51,0x1e,0x2f,0x01, +0xb2,0xd8,0x1f,0xeb,0xe8,0x24,0x8b,0xa3,0x7d,0xd3,0xeb,0x75, +0xdf,0xe8,0x53,0x4e,0x14,0x2f,0x26,0x89,0xa3,0x35,0x47,0x7d, +0x84,0x0c,0x4d,0x84,0x28,0xd5,0x8f,0xde,0x06,0x3f,0xa0,0x01, +0x6d,0x65,0x31,0x1f,0x4d,0x5e,0x7a,0xb0,0x83,0x22,0xc4,0xb6, +0x83,0x18,0x9a,0x23,0x04,0x31,0x65,0xd9,0x2e,0x99,0x4d,0x94, +0x79,0xfa,0xf4,0x0e,0xf7,0x65,0xb2,0xfb,0x75,0x15,0x5a,0x86, +0xdd,0xad,0x8c,0xa8,0x63,0x37,0xa9,0x9a,0xa0,0xab,0x32,0xb4, +0xe2,0xdf,0x59,0x0b,0xb5,0xd2,0x7d,0x16,0x7e,0xae,0x25,0x8b, +0xed,0xec,0x1a,0x33,0xb7,0xd5,0x9c,0x66,0xd7,0x94,0xd5,0x47, +0x52,0xd2,0x31,0x77,0xee,0xe0,0x4c,0x20,0x92,0xb9,0x4a,0xb2, +0x86,0x13,0x83,0xc0,0x16,0x56,0x1c,0xb9,0x73,0x72,0x7f,0x63, +0xfd,0xcf,0x44,0x46,0xfc,0xfe,0x30,0x4e,0x64,0xc3,0x76,0x15, +0x66,0x6d,0xc5,0x55,0x3c,0xab,0x29,0xd6,0xa6,0xfa,0x52,0x0a, +0x14,0xec,0x4e,0xb2,0x5b,0x0b,0x3b,0x02,0xcc,0xb1,0x41,0x62, +0x39,0xed,0x1a,0xe6,0x8f,0xfc,0xfa,0x55,0xb1,0x46,0x6f,0xaf, +0x8f,0xdd,0x61,0x01,0x73,0x6c,0x17,0x79,0xae,0x7b,0xb7,0x63, +0xe1,0x31,0x32,0x72,0xe2,0x67,0xd7,0x01,0x49,0x53,0x46,0x11, +0xc2,0x15,0x15,0x55,0x99,0x2e,0x7a,0x43,0x6b,0x50,0x80,0x8e, +0xe0,0xa7,0x99,0x44,0x36,0xdf,0xfd,0xb4,0xff,0xe3,0xde,0xd1, +0xfb,0x57,0x07,0x47,0xfb,0x3b,0x27,0xc7,0x25,0x1f,0x91,0xbd, +0x1b,0xb4,0x4f,0xe8,0xd6,0xe8,0xe3,0x2f,0x37,0xe7,0x63,0xa7, +0xd3,0x54,0xd3,0x9b,0xf7,0x42,0x16,0x7c,0x02,0x19,0xc4,0x64, +0x52,0x73,0x73,0xf7,0xa7,0xa3,0xa3,0xbd,0x77,0xbb,0x7f,0x79, +0x7f,0xfc,0x97,0xfd,0xc4,0x70,0xac,0xb8,0x3c,0x97,0xda,0x6c, +0xce,0xc3,0x9d,0x93,0x93,0xbd,0xa3,0x77,0xc7,0x68,0xc5,0xcf, +0xc6,0xfd,0xfe,0xe8,0xe0,0xa7,0xc3,0xf7,0xc7,0x7b,0x87,0x2e, +0xe6,0xe5,0xde,0xee,0x9b,0xfd,0x9d,0xb7,0x14,0xb7,0x9d,0x80, +0x68,0x73,0x3b,0x44,0x3f,0xe0,0x9b,0x67,0xf3,0xad,0xad,0x9d, +0xef,0x37,0xaf,0x8a,0xc8,0xab,0x8e,0xd2,0x0b,0xf7,0x3f,0xd4, +0x4d,0x0c,0xc8,0x5c,0xa2,0xa7,0x6c,0x08,0x1f,0x0f,0xfe,0xee, +0x49,0x32,0x68,0xa1,0xcd,0x58,0x6c,0xe1,0xd6,0x53,0x5b,0x18, +0x54,0xa1,0xbd,0x99,0x56,0xd4,0x02,0x74,0x8e,0x57,0x45,0xe6, +0x6a,0x32,0x9e,0xdf,0x1e,0xe7,0xb7,0xb0,0x11,0xbb,0x18,0xdc, +0xf4,0x86,0xf8,0x19,0xa2,0xca,0x1e,0xea,0x7b,0xef,0xa4,0x64, +0xb2,0x58,0xe0,0x93,0x78,0x7c,0xda,0x95,0xdb,0x28,0x61,0x76, +0x51,0x44,0x32,0x03,0xe4,0xc6,0x0b,0x92,0xc1,0x5d,0x9e,0x6d, +0xbd,0xe0,0x1c,0x6d,0xfe,0xc9,0xf6,0xd1,0x02,0x74,0xef,0x7c, +0x6a,0x0b,0xf2,0xc3,0xea,0xf9,0xcd,0xf1,0x6c,0x92,0x71,0x54, +0x23,0x8a,0xa4,0x6d,0x79,0xff,0x04,0x17,0x4f,0x08,0x83,0xc0, +0x4d,0xee,0x73,0xe9,0x40,0xe9,0xf3,0xed,0x78,0x94,0x93,0x77, +0x14,0x3c,0x22,0xda,0x68,0xe1,0xbb,0x51,0x02,0xe0,0xbd,0xd3, +0xe3,0x2d,0x33,0xfb,0xcb,0xa4,0x63,0x52,0x49,0x66,0x05,0xb9, +0xcd,0xf8,0xf4,0xac,0x7f,0xd6,0xec,0x36,0x92,0x3c,0xde,0xe8, +0x24,0x7c,0xf6,0x9c,0xb4,0x6f,0xf8,0x3d,0x5e,0xb4,0x11,0x89, +0x16,0xde,0xe9,0x76,0x17,0xd5,0x25,0xf0,0xeb,0x79,0xf7,0x85, +0xee,0x91,0x46,0xab,0x23,0x18,0x47,0x5b,0x51,0x1a,0x07,0xc8, +0x72,0x42,0x88,0xe7,0x16,0x69,0x0e,0xa8,0xa8,0x44,0x43,0x43, +0xff,0xbf,0xdc,0x72,0x58,0xe9,0xeb,0xf5,0x96,0x74,0x18,0x4c, +0x91,0x22,0x60,0x88,0x05,0x41,0xe1,0xd5,0xe0,0x33,0x4c,0xa8, +0x60,0x84,0x94,0xf3,0x3b,0x1b,0xff,0x36,0x1f,0x65,0xb1,0xb4, +0x9b,0x0d,0x6e,0x2b,0x3a,0xc1,0x23,0x73,0x54,0x3f,0x49,0xdc, +0x2a,0xa6,0xa6,0x6a,0x00,0x19,0xb1,0x50,0x61,0x1e,0xbe,0x9b, +0xc1,0x28,0xe6,0x8f,0xde,0xe7,0x58,0xc8,0x08,0x23,0x5f,0x41, +0x4e,0xa3,0x10,0x48,0x8c,0x4b,0xec,0x7d,0xc6,0x44,0xb9,0xfa, +0xbf,0x1d,0x7f,0x62,0x40,0xf0,0x11,0xb7,0xb6,0x42,0x6a,0x0b, +0x08,0x05,0x48,0x73,0xd4,0x17,0x52,0xf9,0x0e,0x72,0x27,0x9b, +0xf0,0xa7,0xad,0xdb,0x49,0xa2,0x95,0xd0,0x52,0x45,0x43,0xcd, +0xa7,0x6b,0x58,0x40,0x33,0x9b,0x1b,0x66,0x52,0xdb,0x95,0x74, +0x91,0xdc,0x19,0x82,0xda,0x14,0xd5,0xa1,0x69,0x4e,0x64,0x16, +0xfb,0xe1,0x15,0x62,0x66,0xc2,0x48,0x8a,0x43,0x02,0xa4,0x1a, +0xac,0xcf,0x91,0x8c,0x8b,0x36,0xe8,0x6f,0x82,0x2f,0xc5,0x32, +0x9d,0xbe,0xc1,0xc9,0xc1,0x1b,0x7a,0xc8,0x23,0xba,0x89,0x68, +0xf1,0x35,0x86,0xe0,0xc6,0x20,0xf9,0x15,0xe5,0xab,0xd7,0xd1, +0x57,0xcb,0xa0,0x40,0x0a,0x8d,0xcc,0x4e,0xd9,0xc4,0x84,0xf1, +0x5c,0x95,0x48,0x3b,0x03,0xf6,0x40,0x38,0xc8,0x00,0x22,0xc8, +0x2c,0x1a,0x0d,0x5f,0x5b,0x80,0x1c,0x54,0x3b,0xfc,0x67,0xd4, +0x4b,0xcd,0x73,0xfd,0x2c,0xd0,0x7f,0xd0,0xa3,0xdc,0x76,0x33, +0xa0,0x81,0x73,0xa8,0xad,0xd3,0x60,0x0e,0x6e,0xa1,0xec,0x16, +0xc6,0x15,0x30,0xf1,0xbc,0xaa,0xe1,0xea,0x71,0x97,0x18,0xe1, +0xdc,0x58,0xfa,0xad,0xbd,0x75,0xa3,0xed,0xf9,0x53,0xc7,0x0e, +0xe7,0x28,0xbf,0x3a,0x9c,0xe4,0xa9,0x0d,0x42,0x9f,0x41,0x30, +0x31,0xaa,0x90,0x46,0x20,0x48,0xa8,0x86,0x76,0x3c,0xbf,0xd4, +0xd0,0x20,0x68,0x0b,0x7d,0x18,0xc3,0xfc,0x81,0xc9,0xa7,0xdc, +0x6a,0xf6,0xfa,0x9e,0x37,0x9b,0xfe,0xe0,0x6a,0x30,0x9b,0x1a, +0x10,0xef,0x6e,0xe4,0x15,0x4d,0x7e,0x05,0x4c,0x90,0x7a,0x95, +0xd8,0x29,0x59,0x22,0x80,0xa8,0x0d,0xda,0x5c,0x66,0x1b,0xf0, +0x27,0xa1,0x2f,0x9e,0x0a,0x38,0x83,0x6c,0xa7,0x33,0xac,0x76, +0x42,0xa9,0x5b,0x9c,0xec,0x2e,0x0b,0x07,0x04,0x09,0x52,0xb0, +0x80,0xf4,0x9e,0x2f,0xbb,0xc1,0x65,0x13,0xb4,0x9a,0x70,0x85, +0x05,0x95,0x65,0x68,0xe8,0x07,0xb1,0x8e,0x41,0xe7,0x0f,0x53, +0x9c,0x1e,0xe3,0xcb,0xcb,0x69,0x3e,0x13,0xb4,0xad,0xe3,0x27, +0x8a,0xcb,0xf8,0x67,0xb1,0xd8,0xf2,0x27,0x06,0x7d,0x67,0x3f, +0x88,0xc5,0x1c,0x0c,0x9f,0xe2,0x1d,0x2a,0xe0,0x88,0x4f,0x13, +0xbc,0xf7,0x59,0x2e,0xfc,0x62,0x6b,0xb1,0xa0,0x9c,0x2f,0x36, +0x38,0x22,0xb1,0x6f,0xa7,0x1b,0x02,0x9e,0x2d,0x26,0xcb,0x83, +0xe7,0x8d,0xd6,0x76,0x26,0xf1,0xee,0x8d,0x75,0x6b,0x1b,0x47, +0xc0,0xf6,0x34,0x4b,0x7f,0x84,0x39,0xa1,0xbc,0x0c,0x9b,0x07, +0x7c,0x33,0x68,0xe1,0xf5,0x78,0x32,0x7b,0x05,0x14,0x50,0xde, +0x61,0x63,0x6e,0xbb,0x18,0x3f,0xdc,0x22,0x32,0x6b,0xe3,0x76, +0xf2,0xb1,0x83,0xd9,0x89,0x8e,0x5f,0x1f,0x1c,0x9d,0x70,0xbe, +0x94,0xef,0xaa,0x03,0xf1,0x60,0x7a,0x0a,0x25,0xbb,0xa7,0x04, +0xb7,0xab,0x10,0xc5,0x43,0xce,0xbf,0xc2,0x8a,0x22,0x98,0xfa, +0x3e,0xfd,0x3b,0x9a,0x7d,0xdb,0x68,0x7d,0xd7,0xb7,0x9a,0x0c, +0x90,0x11,0xe3,0x0e,0xa8,0x47,0x62,0xea,0x87,0x7e,0xde,0xc7, +0xb2,0x19,0xc6,0xbf,0xc8,0xb6,0x3a,0x51,0x23,0x4a,0x81,0xca, +0xdc,0x6c,0xb1,0x19,0x1a,0x99,0xef,0x34,0x64,0xca,0xa7,0x54, +0x00,0xf2,0x5f,0x0e,0xc7,0x63,0xd8,0x8c,0x44,0x17,0xf9,0x60, +0x18,0x75,0x63,0x8c,0xde,0xfc,0xcd,0x56,0x02,0x62,0x56,0x23, +0x2c,0x41,0xeb,0x3d,0x26,0xff,0x8a,0x93,0x95,0xb1,0xf0,0x9b, +0xdb,0x1b,0x85,0xbc,0xef,0x46,0x6b,0x91,0x5a,0xf0,0x7f,0x3d, +0x9e,0x4f,0xa6,0x71,0xf2,0x43,0x6b,0xbb,0x63,0xc5,0x9e,0x9d, +0xfd,0xc3,0xfd,0x63,0xf2,0x66,0x1f,0x44,0xb4,0xba,0xe1,0x38, +0x16,0xc5,0x37,0x97,0x88,0x87,0x3f,0x7c,0xe7,0x76,0x32,0x7e, +0x09,0x19,0x63,0xde,0x4f,0x29,0xb1,0xa1,0xed,0xae,0x59,0x39, +0x49,0xe4,0x87,0xa3,0xf7,0x6f,0x8e,0x0f,0x7e,0xfb,0x9b,0xad, +0xd6,0xfb,0xe3,0x93,0x23,0x7b,0x03,0x05,0x00,0x48,0xc9,0x9a, +0x20,0xa1,0xb3,0xb2,0xbf,0x23,0xca,0xb0,0x94,0xcc,0xfe,0xbe, +0x3f,0x18,0xc1,0x2f,0xd1,0x14,0x5b,0x88,0x60,0x99,0xe2,0xb7, +0xdd,0x0e,0xb5,0x0e,0xc6,0xe3,0xa7,0x93,0xdd,0x57,0xf3,0xe1, +0xf0,0x2f,0xb0,0x3b,0x49,0x6d,0x9c,0x8d,0xa0,0x83,0xec,0x07, +0x0a,0x52,0xcf,0xb8,0x52,0x14,0x62,0x51,0xe6,0xf4,0x77,0x68, +0x68,0x44,0xf0,0xc0,0xeb,0x76,0x1b,0xdb,0xe0,0x8f,0x16,0x3a, +0xf2,0x62,0xec,0xaa,0x52,0x5b,0x5d,0x98,0xf8,0x1e,0x69,0xb6, +0x02,0x44,0x63,0xe4,0x73,0xe3,0xbd,0xbb,0x0f,0x6d,0x77,0x93, +0x8d,0x96,0x0a,0x3f,0xef,0xca,0x72,0x7f,0xad,0x2a,0xf8,0x1e, +0x96,0xda,0xad,0x64,0x83,0xd1,0x32,0x37,0x2a,0xe5,0xd7,0x36, +0x05,0x30,0x32,0x53,0x95,0xf0,0x1b,0x4a,0x30,0x37,0x53,0x2d, +0x12,0xb4,0xf2,0xe7,0xdf,0xd1,0x29,0xec,0xab,0xe1,0xb8,0x37, +0x8b,0xa3,0xad,0x66,0xd4,0x90,0xfc,0xff,0x4e,0xf9,0xbd,0x9b, +0x3c,0xdf,0x81,0xaa,0x11,0xd7,0xe6,0xc6,0x4c,0x01,0x26,0xb7, +0xd1,0x2e,0x11,0x3c,0xce,0x74,0xd8,0x12,0x8c,0x32,0x2a,0x00, +0x9c,0xf5,0xef,0xbf,0x5f,0x26,0x1b,0x1d,0xf8,0x38,0xeb,0xbb, +0xdf,0xb8,0x93,0x9e,0xb8,0xaf,0xb4,0x53,0xf5,0x79,0xd6,0x24, +0x11,0x33,0xe9,0xe0,0xff,0xf1,0x5f,0x17,0xf1,0x69,0x63,0xa3, +0x9b,0x70,0xb2,0xcd,0x86,0x49,0xeb,0x9b,0xa5,0x8d,0x83,0x9a, +0x13,0xd6,0xe0,0x9b,0xe8,0x55,0xcc,0x0a,0x82,0x31,0xd2,0x2a, +0x67,0x94,0x9d,0x14,0x48,0x34,0x37,0xb0,0x5f,0x9c,0xdf,0xbc, +0x64,0x85,0x37,0x4e,0xb3,0x1b,0x96,0x97,0x3b,0x27,0x7b,0x27, +0x6f,0xf6,0xf7,0xec,0x96,0xe5,0x94,0xd3,0xa1,0xe7,0xf8,0xc3, +0xf8,0xfb,0x68,0xe4,0x29,0x68,0x0d,0x02,0x29,0x5c,0xf6,0x39, +0xd0,0x23,0x6f,0xde,0xfd,0x9e,0x75,0x2c,0x28,0xbd,0x83,0xa3, +0x45,0x5f,0x69,0x69,0x5a,0x51,0x34,0xd0,0xc9,0x54,0x18,0x42, +0x00,0xd0,0x4d,0x19,0xc9,0x05,0xfb,0x0a,0x15,0x54,0x4c,0x40, +0x44,0x0a,0x42,0xae,0x9d,0xf0,0xac,0xc4,0x46,0xd8,0x06,0xbc, +0x3f,0x3e,0x7c,0xfb,0xe6,0x84,0x75,0x20,0xa4,0xc3,0xb8,0xa7, +0xd0,0x7f,0x0f,0x76,0x91,0x3c,0xac,0xa3,0x80,0xf4,0x61,0xcb, +0x8a,0x31,0x99,0x2c,0xe6,0xe3,0x5b,0x72,0x92,0x5d,0x5a,0xf3, +0x5d,0x3e,0xf2,0xb1,0x5c,0x72,0x4a,0x40,0x20,0x4b,0x07,0xa0, +0xa3,0x00,0x3f,0xe1,0xdd,0x34,0x6e,0x8d,0xec,0x72,0xd4,0xb9, +0x94,0xd1,0x5d,0x35,0x22,0xd6,0x44,0xb6,0xdb,0xb0,0xc6,0x3f, +0x3f,0x5b,0x3c,0x5b,0x4f,0x60,0xc7,0x8a,0x72,0xbb,0x8b,0x7e, +0xf6,0x0c,0x63,0x9e,0x45,0xf4,0x10,0x1b,0x80,0xab,0xd5,0xc1, +0x1f,0x6f,0x57,0x9c,0x0b,0x5b,0x67,0xcb,0x76,0x92,0xb0,0xc5, +0x04,0x8e,0x25,0xc3,0xc6,0xca,0x6b,0xbe,0x3e,0x07,0xaf,0x00, +0x45,0xf7,0xf8,0x7c,0x5a,0x1e,0x1e,0x29,0x53,0x02,0xdd,0xcd, +0x5a,0x6a,0xe2,0x18,0x3b,0xb0,0x14,0x6a,0xd3,0xb3,0x42,0x28, +0x4b,0x53,0x9e,0xa1,0x98,0x15,0x05,0x28,0xb5,0x43,0x7f,0x71, +0xc5,0xa2,0xc4,0xe6,0x94,0x9e,0x0b,0x6e,0x09,0x02,0xa9,0x8e, +0xa4,0x28,0xc3,0x31,0x62,0xe9,0x21,0xb5,0x5b,0x54,0x33,0x32, +0xe3,0x39,0x3d,0x7c,0x23,0x4b,0x2e,0x04,0x53,0xe7,0xec,0x58, +0xa4,0x7c,0x54,0x4a,0x51,0x3f,0x6c,0xe8,0x38,0x34,0x5d,0x42, +0x19,0x83,0xd8,0x84,0xb1,0x81,0x55,0x52,0x9e,0x26,0x09,0x72, +0xf1,0x20,0x00,0xd8,0x60,0x04,0x47,0x41,0x64,0xe2,0xde,0x2a, +0xe1,0xd5,0x2c,0x4b,0x98,0x98,0x8c,0x4f,0x2e,0x9d,0x4f,0xe6, +0xb9,0x1e,0xe8,0xe0,0x02,0x41,0x54,0x5a,0x57,0xdd,0x04,0x4c, +0x41,0xe6,0x38,0xb4,0x27,0x49,0x06,0x1f,0xa2,0x42,0xe6,0x03, +0x04,0xa0,0xd6,0x46,0x3c,0x72,0xed,0xc9,0x91,0x6b,0xcb,0x8c, +0xb7,0xfd,0x79,0x3e,0x9f,0x7f,0x06,0x40,0x4a,0xbe,0x43,0xb1, +0x74,0x16,0x64,0x41,0x87,0xaf,0x0c,0x08,0x47,0x1b,0xf7,0x16, +0x98,0x27,0x51,0x07,0xbc,0x96,0x03,0x6f,0x2d,0xd5,0x15,0x17, +0xe1,0xb6,0xeb,0x71,0xc1,0x8c,0xa6,0x9f,0x4f,0x2f,0xac,0x2f, +0x05,0x47,0xbc,0x3f,0xf2,0x31,0x69,0xac,0x12,0x3b,0xbe,0xe1, +0xe6,0x5c,0x39,0xd0,0xb9,0xb9,0x8d,0xcf,0x4d,0x2f,0x59,0xd2, +0xe9,0xfd,0xb2,0xd0,0xe4,0x3c,0xbe,0x6b,0x99,0xbb,0x6d,0xd1, +0x55,0x68,0x39,0x2b,0x3a,0x2d,0x33,0xdb,0x76,0x81,0x6d,0xb7, +0xb2,0xb4,0xb2,0x6c,0xb6,0xdd,0xd1,0x67,0xef,0x2d,0x94,0x36, +0x5b,0xd9,0x5d,0xab,0x70,0xd0,0x7f,0xb7,0x9d,0xdd,0x6d,0x87, +0x71,0x10,0xd9,0x42,0x81,0x75,0xbb,0xb3,0x95,0xde,0x6d,0xbf, +0xb8,0x6b,0x75,0x36,0x5a,0x69,0x2b,0x49,0x67,0xdb,0x2f,0x66, +0xfc,0xbd,0x7c,0xec,0x7e,0x06,0xd3,0x83,0x8e,0x0e,0x93,0x83, +0x24,0xe7,0x63,0x25,0x2c,0xd0,0x09,0x82,0xe9,0x69,0x10,0xec, +0x86,0xe4,0x02,0x02,0xc8,0x6d,0x58,0x5c,0x9b,0x0b,0xb6,0x10, +0xc5,0xe7,0x8b,0x1d,0x08,0xd4,0x72,0x43,0xb9,0xd7,0x65,0x58, +0x2c,0xac,0x6a,0xab,0x37,0x17,0x2a,0xb3,0xdd,0x03,0x81,0x6e, +0x8c,0x41,0x20,0xcd,0x7c,0x31,0x7f,0x88,0x0f,0x4b,0xdb,0x46, +0x75,0x0a,0xad,0x2b,0xbe,0xe6,0x55,0xd9,0x8c,0x8b,0x53,0xa9, +0xbc,0xff,0x41,0x2c,0x5a,0x09,0x4b,0xea,0xa2,0x4c,0xeb,0x91, +0x4a,0x4c,0x99,0x26,0x57,0x52,0x19,0x12,0x13,0x5e,0x72,0xf6, +0x08,0x58,0x7c,0x9e,0x90,0x4a,0x87,0x23,0xcf,0xa5,0x37,0xa4, +0x47,0x83,0xb5,0x3b,0xbe,0x55,0xce,0x94,0x83,0x03,0x61,0xe7, +0x47,0x19,0xf3,0x58,0xd7,0xdb,0x78,0x28,0xac,0x18,0x82,0x4f, +0xc7,0xf1,0x89,0xab,0xe7,0x0e,0x7f,0x86,0x93,0x3e,0x38,0xf0, +0xbc,0x7a,0x39,0x98,0xe4,0xa4,0xac,0x84,0xa6,0x72,0xf8,0x4b, +0x1d,0xe7,0x3a,0x2b,0x4b,0x3e,0x91,0xed,0x44,0x71,0x20,0xbb, +0x1f,0x0e,0x46,0x1f,0x53,0x17,0x86,0x15,0xc9,0x7d,0xa3,0x6a, +0x0d,0x69,0xb1,0x67,0xe5,0x28,0x18,0xcf,0x9d,0xdd,0xc8,0xd8, +0xeb,0x70,0x0f,0xdc,0x63,0x86,0xbb,0x24,0x54,0x7f,0x9f,0xa0, +0xc9,0xb3,0x89,0xf5,0x4b,0x6a,0x60,0x7b,0x3d,0xd1,0x0f,0x6d, +0x67,0xe3,0xab,0xab,0x61,0xfe,0x27,0x7c,0x84,0xb7,0x3b,0x9d, +0xc2,0xc6,0x9c,0x3e,0x0d,0xbd,0xca,0x23,0x8d,0x03,0x72,0x71, +0xfb,0xc7,0x9c,0x94,0x91,0x8a,0x71,0x59,0x39,0xaa,0x03,0x14, +0xd4,0x98,0x8e,0x7a,0x1f,0xf3,0xf7,0xb4,0x63,0x2b,0xe7,0x30, +0x90,0x03,0xd7,0x14,0xeb,0x97,0x16,0x1a,0x85,0x16,0x2a,0x77, +0x87,0x3d,0xa8,0xdf,0x22,0xd0,0x79,0xf3,0xee,0x4f,0x3b,0x6f, +0xdf,0xbc,0x7c,0xbf,0xfb,0x76,0xe7,0xf8,0x38,0x55,0xdf,0x49, +0xa3,0x02,0xb7,0x26,0x6c,0xbd,0x0a,0x00,0x74,0xf1,0x00,0x58, +0x25,0x80,0xa5,0x3d,0xa3,0x66,0x4b,0x07,0xfc,0x50,0x1d,0xbb, +0x30,0xb3,0x58,0x72,0x54,0x9c,0xa0,0x16,0xb8,0xed,0xd4,0x08, +0x4b,0x44,0xe8,0x9a,0x1d,0xa4,0x1c,0xea,0xf0,0xd9,0x64,0x68, +0xe4,0x4d,0xe3,0x2e,0x1e,0xbd,0xc3,0x4a,0x46,0x7e,0x0c,0xa6, +0x24,0x63,0x36,0xd7,0x29,0x40,0x36,0x6f,0x19,0xca,0x54,0x56, +0x53,0x48,0xc2,0x4d,0x6c,0x46,0xc3,0x23,0x56,0xb7,0xe4,0xfb, +0x0a,0x01,0x1b,0xce,0xe3,0x4c,0xba,0x71,0xf0,0x76,0x32,0x98, +0xce,0x06,0x6c,0xae,0x9c,0x63,0xa8,0x66,0x1f,0x14,0x54,0xb0, +0x84,0x6f,0x52,0x73,0x1d,0x7b,0x8b,0xeb,0x27,0x71,0xcd,0x79, +0x42,0xf7,0xdd,0x78,0x78,0xf4,0xe6,0xf8,0xe4,0xcd,0xbb,0x3d, +0xe9,0x32,0x53,0x20,0x13,0x7c,0xc9,0x7c,0x59,0x00,0xe5,0x6f, +0x59,0xa5,0x6d,0x78,0x75,0xcc,0x8d,0xe4,0x19,0x68,0xa3,0x6d, +0xdb,0xb9,0xc9,0x20,0xeb,0x10,0xa4,0xc2,0xe3,0xbc,0x20,0x4f, +0x22,0x87,0x59,0xa7,0x41,0x6c,0x37,0xb3,0x10,0xff,0xe7,0x52, +0xb0,0x11,0x52,0x7a,0x0c,0x21,0x5b,0x73,0x15,0xcc,0xcc,0x42, +0x45,0x0f,0x52,0xa4,0x66,0x55,0x91,0xcb,0x58,0x49,0x96,0x07, +0xd7,0xf3,0xf4,0x5f,0xc8,0x6c,0x9f,0x27,0x30,0xf2,0x9a,0x4e, +0xb2,0x00,0x60,0xc7,0x0e,0x7f,0x07,0x7d,0xe0,0xe3,0x71,0x21, +0x8b,0x61,0xf7,0xde,0x84,0xe5,0x32,0xb4,0x10,0x6b,0x3b,0xd1, +0xa7,0x9b,0x12,0xb8,0x40,0x99,0x20,0x80,0x6b,0x27,0xb4,0xeb, +0x02,0x24,0x74,0xc2,0x32,0x63,0xdc,0x4f,0x0b,0x25,0xba,0xac, +0x44,0x4a,0xa5,0x12,0xca,0x08,0x9d,0xaf,0xf1,0xe1,0x26,0xba, +0xc1,0xfc,0x25,0xb0,0x94,0x14,0x6b,0xe2,0xdf,0xd8,0x08,0xe6, +0x02,0x6a,0xfe,0x56,0x72,0x9b,0xe4,0x31,0xe2,0x4d,0xcc,0x0a, +0x5c,0xc9,0x2e,0x7a,0x91,0x34,0x4b,0xdd,0x1f,0xd0,0xfe,0x23, +0x83,0x40,0xf3,0xc1,0xde,0x0e,0x60,0x4f,0x04,0x0d,0x58,0x85, +0x3f,0x75,0x02,0xdf,0xfa,0x8c,0x2e,0x86,0x73,0x58,0xb4,0x0a, +0xdd,0x24,0x52,0x06,0x1b,0xee,0x58,0xd5,0x98,0x5f,0xac,0xed, +0x76,0x5d,0x69,0xa3,0x51,0x6a,0x61,0xeb,0x1f,0x68,0x21,0xb3, +0x8e,0xa4,0xfd,0x8b,0x37,0x1b,0xe5,0x86,0x32,0x18,0x85,0x56, +0x71,0x14,0xb6,0x96,0x8a,0xf0,0x5e,0x12,0x23,0xd2,0x26,0x5c, +0x2a,0x98,0x79,0x81,0x89,0x78,0xee,0xf2,0xf2,0xcd,0xd1,0xc9, +0x5f,0x2c,0x6b,0x09,0x38,0xdb,0x56,0x89,0xb3,0xb5,0x8a,0x6d, +0xa3,0xaa,0x63,0x3d,0x0b,0x0e,0x6d,0xe6,0x47,0xf0,0xd1,0xd5, +0xae,0x66,0x75,0x4f,0xe1,0xb4,0x34,0xf7,0xdd,0xc4,0x7c,0x80, +0xcf,0x28,0xec,0x62,0xba,0x20,0x57,0x26,0x26,0xf6,0xd0,0x6c, +0x51,0xe1,0xba,0x5d,0xdf,0x2a,0x71,0x8a,0xbc,0x79,0xbe,0xe3, +0x97,0xed,0xf2,0x5c,0x5a,0x42,0x77,0x62,0x03,0x96,0xdf,0x42, +0xf9,0x35,0x1d,0x36,0xb5,0x6f,0x70,0xff,0x43,0x6f,0x16,0xd9, +0x08,0x92,0x5e,0xff,0xcd,0x05,0x2e,0x53,0xd6,0x21,0xa4,0x72, +0x97,0xa0,0x6d,0x00,0x55,0xba,0xc2,0x73,0xab,0x21,0x9a,0x09, +0x4b,0xda,0x7e,0x17,0x81,0x60,0x61,0xa9,0x3a,0x99,0x0c,0x6e, +0x00,0xe1,0x93,0xc8,0x1d,0x35,0x67,0x78,0x52,0x6c,0xb5,0x94, +0xa8,0x62,0xa0,0xb0,0x81,0x98,0x87,0x5d,0x73,0x87,0xcf,0x53, +0xad,0xc9,0xab,0x6a,0xe6,0x02,0x48,0xd1,0xb6,0x8c,0xbd,0xad, +0x4f,0xc8,0xd0,0xb8,0xf3,0x6a,0x69,0xd5,0xe6,0xad,0x8e,0x7c, +0x92,0x58,0x54,0x01,0x8e,0xc4,0x79,0x33,0x5c,0xfe,0xf2,0x4f, +0x94,0x5e,0x59,0x2b,0xf8,0x6d,0x45,0xdb,0x25,0x03,0x32,0x2f, +0xfe,0x7a,0x40,0x33,0xda,0x19,0xc9,0x4d,0xac,0x32,0x2d,0x9b, +0xfb,0x02,0x91,0xb1,0xad,0xd1,0xf9,0x98,0x7f,0xe9,0x8f,0x3f, +0x8d,0x94,0x36,0xa0,0xe8,0xf8,0x8b,0xb6,0x09,0xeb,0xf7,0x37, +0xe1,0x6b,0x17,0x5f,0x45,0xfe,0x0e,0x37,0x3d,0x10,0x58,0x2c, +0xe0,0xcf,0x8b,0xd6,0xaf,0xeb,0xf5,0xd6,0xef,0x5e,0xb8,0x70, +0xf6,0xfc,0xdf,0xeb,0xf5,0xef,0xb7,0x5e,0x70,0x8e,0xa0,0x1d, +0x64,0x6b,0x4e,0x57,0x2c,0x2f,0x14,0x7c,0x47,0x98,0x8a,0x0e, +0xbc,0xed,0x41,0x22,0x8e,0xa0,0x2e,0x49,0x91,0xb5,0x0b,0xec, +0xc5,0xa0,0x8a,0x64,0xc9,0x43,0x04,0x53,0xb4,0x1f,0x76,0x9c, +0x26,0x15,0x4b,0xec,0x85,0xf1,0x4f,0x3a,0x51,0x94,0x16,0xe3, +0xec,0x7b,0x38,0xba,0x07,0xfa,0x13,0xf3,0x2f,0x10,0xc1,0xf9, +0x00,0x4a,0xa2,0x33,0xa1,0xb7,0x43,0xb9,0x90,0x17,0x36,0x97, +0x6b,0x37,0x3c,0x57,0x68,0x99,0xb4,0x38,0xbb,0xf4,0x9c,0x43, +0x07,0x2d,0x57,0x64,0xd4,0x14,0x4f,0xe5,0x38,0xae,0x13,0x7b, +0x8a,0xb3,0x3c,0x34,0x92,0x4a,0x23,0x32,0xe0,0xc2,0xf9,0xd2, +0x87,0xf3,0xc1,0x8a,0xe5,0xbd,0xf7,0x12,0xa1,0x4a,0x1a,0x4c, +0x0b,0x3e,0x89,0xf3,0xd7,0xbc,0x74,0xc1,0xfe,0xf3,0xd9,0x66, +0xdc,0xfc,0x2e,0x81,0xbf,0xa7,0x57,0x83,0x9b,0xee,0x77,0xc9, +0xfa,0xa6,0x3a,0x8d,0xe3,0x36,0xe3,0xc9,0xdf,0x51,0x7e,0xb5, +0xf7,0xf9,0xd6,0x1d,0x26,0x1b,0x77,0x8e,0x6c,0x8a,0x1d,0xf6, +0x80,0xaf,0x2c,0xea,0x2a,0x0b,0xd6,0xbe,0x77,0x4b,0xd2,0x47, +0x21,0xa8,0x71,0x39,0x38,0xff,0x60,0x8d,0x8e,0x91,0xc9,0x40, +0x89,0xa6,0xc3,0x8a,0x35,0x9f,0x67,0xb1,0x50,0x01,0xea,0x67, +0x79,0xb6,0x6b,0x7d,0xcd,0xb8,0x21,0x8c,0x12,0x08,0x8c,0x79, +0x40,0x22,0x13,0x41,0x23,0xc9,0x4e,0x34,0x5a,0x17,0xc0,0xd7, +0xb3,0xe7,0x79,0xad,0x27,0x8d,0xaf,0x9d,0xcf,0x67,0xb5,0x4f, +0xbd,0x29,0x5a,0x05,0x69,0xd6,0x44,0x7d,0x3a,0xad,0xdd,0x6f, +0x2f,0x23,0xa7,0xa0,0xe1,0x2b,0x35,0xd3,0x59,0x8f,0xac,0xa2, +0x9e,0xf4,0xae,0xec,0x26,0x28,0xd1,0xce,0x9e,0x83,0xce,0xc0, +0x02,0xf6,0x49,0x21,0x0f,0x30,0x1f,0x63,0xce,0x9c,0x11,0xc0, +0x62,0x27,0x59,0x9e,0xc6,0x4e,0x77,0x57,0x65,0x4a,0x8c,0x10, +0xec,0xfe,0x60,0x24,0x47,0x56,0x7c,0x69,0x62,0x83,0x74,0x92, +0x57,0xca,0x63,0x20,0x9d,0x4d,0x5d,0x3f,0x36,0xac,0x6b,0x21, +0x69,0xd7,0xeb,0x7c,0x14,0x2a,0x57,0x9a,0xae,0x9a,0x4a,0xf2, +0x76,0xa9,0x45,0xc2,0xad,0x24,0x72,0x9d,0xdb,0x4d,0x87,0x65, +0xbb,0xa2,0x17,0xca,0xc8,0x27,0x2b,0x3a,0xb5,0x22,0x27,0x99, +0x2b,0x92,0xfe,0xe8,0x7d,0x0e,0xfa,0xcc,0x06,0x83,0x3e,0x73, +0x79,0x60,0x4a,0x7c,0xfe,0x27,0xf4,0xd9,0x0b,0x57,0x4d,0x75, +0x9f,0xd9,0xd4,0xa7,0xf5,0x99,0xca,0xfd,0x48,0x9f,0x95,0x90, +0x5f,0xd9,0x67,0xe5,0x9c,0x25,0x45,0xac,0x6f,0x12,0x02,0x06, +0xac,0x44,0xfb,0x2d,0x65,0xab,0xe6,0x42,0x15,0x13,0x61,0x5b, +0x96,0x61,0xf7,0x3b,0x0b,0x70,0x18,0xb7,0x58,0xc8,0xbd,0xc9, +0xd1,0xde,0xef,0xf7,0xfe,0x7c,0xf8,0x28,0x87,0xb6,0x8a,0xe3, +0xd8,0xbb,0x5e,0x5c,0xea,0xe0,0xe2,0x9b,0x12,0x40,0x31,0xa4, +0xad,0xee,0xbf,0x44,0x24,0xa9,0x1c,0x2f,0x07,0x2e,0xe4,0xe3, +0xab,0x86,0x62,0x05,0xa7,0x0d,0xdb,0x87,0x4b,0x5d,0x14,0x35, +0x58,0x54,0x13,0x7e,0x00,0x74,0xef,0xf8,0x40,0xa6,0x70,0x73, +0x89,0xc8,0x00,0xbe,0x9a,0x8c,0xa1,0xcc,0x0b,0x6e,0xff,0x8a, +0xf9,0xfb,0xd4,0x79,0xfe,0x94,0x19,0xfe,0x94,0xb9,0x5d,0x31, +0xab,0x81,0x7a,0xdd,0x6c,0x2e,0xb7,0x1c,0x12,0x71,0x1a,0x7f, +0xdb,0x04,0xc6,0x99,0xbb,0x6a,0xce,0x3e,0x75,0xb6,0x3e,0x65, +0x9e,0x3e,0x65,0x86,0xaa,0x96,0xff,0x03,0xa4,0xb3,0x58,0xb8, +0xab,0xc0,0x27,0xce,0x81,0x07,0x84,0x94,0x55,0xb4,0xed,0x79, +0xc7,0x7c,0x32,0xfc,0x26,0xc6,0xf1,0xad,0x5c,0xa3,0x2d,0x8f, +0xa4,0x1e,0x15,0x5d,0x8a,0xbd,0xf2,0xd3,0xd1,0xdb,0xa7,0x32, +0x08,0x32,0x7b,0xf4,0x48,0xcf,0x70,0x9e,0x82,0xe8,0x56,0x3d, +0x6c,0x1a,0xdb,0x4a,0xae,0x17,0x64,0xf0,0x5d,0x9b,0xdf,0xf4, +0x06,0xff,0x82,0xce,0xa5,0x6a,0xbf,0xba,0x7b,0xf7,0xf6,0x77, +0xde,0x3c,0xb9,0x83,0xa9,0x8a,0x47,0xbb,0xd8,0xe6,0x7a,0x52, +0x27,0x87,0x58,0x57,0x76,0x73,0x21,0x8b,0xba,0x09,0xeb,0xf5, +0x07,0xe3,0x47,0xfb,0x0b,0x16,0x3b,0xb5,0xdf,0x66,0x51,0x82, +0x8f,0x1b,0xdd,0x81,0x28,0xc4,0xc1,0x9c,0x81,0xc8,0xc8,0x28, +0xd7,0x66,0xc1,0x9e,0x6a,0x38,0xb8,0xf8,0x18,0x99,0xf2,0xce, +0x07,0xad,0xdb,0x90,0xda,0x39,0x1a,0xaf,0xf9,0x8a,0x0d,0x2e, +0xe1,0xe1,0x76,0xb9,0xb6,0xdd,0xe5,0x0d,0x96,0x52,0x78,0x77, +0x25,0xda,0xe5,0xba,0x79,0x2d,0xcc,0xb2,0xc2,0x3e,0x6b,0xc9, +0x4b,0xd0,0xfa,0xf8,0x1c,0xba,0xf3,0x2e,0x8f,0x23,0xca,0x16, +0x05,0xb5,0xa9,0x0e,0x25,0x60,0xe7,0xe3,0xcf,0x4f,0xe8,0x53, +0x31,0x55,0x95,0xff,0xc9,0xe3,0x86,0x27,0x03,0x12,0x61,0x2e, +0x7b,0xb0,0xf7,0x0e,0xd2,0x5e,0xb9,0x98,0xb6,0xbb,0xd7,0x72, +0x00,0x12,0x32,0x18,0x60,0xa1,0x21,0x7d,0xb9,0x3c,0x1e,0x12, +0x66,0x52,0x70,0xe9,0xc4,0xf0,0xc1,0x21,0xfa,0x8a,0xe1,0x28, +0xf7,0xe8,0xc3,0xc3,0x52,0x31,0x02,0xa5,0xae,0xff,0xaa,0xc5, +0x40,0xc6,0x2f,0x73,0xbd,0xb0,0x7c,0x92,0xa0,0xa5,0x4b,0x77, +0x5c,0xd9,0xd4,0x77,0x93,0xe6,0xf9,0x17,0x78,0x14,0xe6,0xaf, +0xb2,0x58,0x81,0x2f,0x1f,0x86,0x0f,0x14,0xe9,0x86,0x02,0xb6, +0x6c,0x74,0x6e,0xc6,0xaa,0x77,0xa6,0x64,0x4b,0xe8,0xde,0xde, +0x51,0xa5,0x74,0x45,0x45,0xf7,0x5b,0xde,0x50,0x4e,0x89,0x66, +0xd4,0x55,0x94,0x00,0x26,0xdf,0x15,0xe2,0x77,0x88,0xbd,0x89, +0x24,0xf7,0xb1,0xc5,0x05,0xba,0x61,0x6d,0x6b,0xb1,0x90,0xe1, +0x23,0xe5,0xf5,0x5f,0x6d,0x43,0xa4,0xc3,0xd5,0xb9,0x41,0xaa, +0xd7,0xd7,0xe4,0x31,0x5a,0xe0,0x83,0x0d,0x32,0xe8,0x23,0x41, +0x89,0x34,0xee,0x34,0x50,0x6a,0x4c,0x24,0x3b,0x1b,0x53,0x92, +0x48,0x7d,0xd1,0xee,0x41,0x50,0xd7,0x11,0x96,0x3c,0xa1,0x74, +0xda,0xe5,0x10,0xc7,0x77,0x44,0x81,0x5c,0x65,0xd5,0xfa,0x7c, +0xb6,0xe6,0x22,0x1c,0x97,0xf0,0x04,0x20,0xab,0xb2,0xdc,0x7b, +0xdf,0x6a,0xaa,0x9c,0xbd,0x72,0x95,0x08,0xd1,0xae,0xad,0x45, +0x49,0xe0,0xa0,0xcd,0xe7,0x67,0x15,0x05,0x86,0x8d,0xf7,0x55, +0x05,0x6d,0x1e,0x9b,0x51,0x29,0xf4,0x18,0xf4,0xe4,0x5c,0xaf, +0x4b,0x19,0xa6,0xcf,0x8f,0x34,0x69,0x24,0xc6,0x55,0xb9,0x2c, +0x20,0xa3,0xfc,0x48,0x79,0xdb,0x5f,0x6d,0x19,0x6f,0x76,0xf0, +0x84,0xdd,0xc3,0xc6,0xb8,0x4d,0x99,0x64,0xd8,0xc8,0x6e,0xc8, +0xd9,0x08,0x76,0x30,0xfd,0x2b,0x48,0x4d,0x9f,0x66,0xf8,0x1a, +0xc8,0xa3,0xb7,0x23,0x79,0x7c,0x36,0x41,0x17,0x47,0x01,0x3e, +0x11,0xd3,0x61,0xf8,0xb6,0x17,0x22,0x90,0x84,0xf8,0x4b,0x44, +0xdd,0x71,0x3f,0x6b,0xd5,0x39,0xa6,0xdd,0xaa,0x43,0x10,0x9f, +0xdb,0xd9,0x3c,0x78,0x32,0x34,0xee,0x2b,0x52,0xee,0x38,0x02, +0x58,0x85,0x5a,0xaa,0x49,0x6d,0x55,0x26,0x71,0x9f,0xce,0x83, +0xaa,0x8c,0xce,0x86,0xf7,0x6d,0x24,0x26,0xb8,0x97,0xb9,0x9e, +0x9d,0x59,0x0d,0xcf,0xa2,0xda,0x81,0xc4,0x77,0x44,0xcd,0x33, +0xd0,0xd2,0x48,0x45,0x27,0xd0,0xbf,0xc0,0xfd,0x06,0x78,0xca, +0xa0,0x8e,0x83,0x77,0xd3,0x1b,0xc1,0x84,0x7e,0x5b,0x81,0x65, +0x05,0x40,0x80,0xe5,0x55,0xbe,0xc8,0xee,0xd0,0xe6,0x95,0x3a, +0xa9,0xbf,0x76,0x45,0xb8,0x40,0x13,0x2d,0x5f,0xee,0x5e,0xf7, +0x26,0x78,0xe2,0x1a,0x3f,0xdf,0x5e,0x5c,0x5c,0x93,0xf6,0x03, +0x06,0x59,0x51,0x62,0x09,0x88,0x58,0x1c,0x7e,0xaa,0x68,0xd9, +0xa3,0x38,0xf4,0x36,0xfe,0xfe,0x35,0x38,0x6c,0x3c,0x7f,0x5e, +0xaf,0x46,0xa2,0x1d,0x0d,0x90,0x14,0xa3,0x37,0x51,0xd8,0xf1, +0xe4,0x13,0xca,0x76,0x4e,0xa1,0xb3,0xd4,0x60,0x14,0x9a,0xc0, +0x22,0x22,0x9a,0x62,0x30,0x1f,0x7e,0x79,0x3b,0x98,0xc1,0xcf, +0x7f,0xce,0xf3,0xc9,0x17,0x23,0x6f,0x7a,0xf7,0xc7,0xfd,0xf9, +0x30,0x37,0x23,0xc0,0x01,0x4d,0x69,0xbd,0x37,0xa4,0x28,0x06, +0x7c,0x80,0x35,0xc6,0x0c,0x4e,0x6e,0x0c,0xe1,0xaf,0x71,0xa6, +0x8c,0x4b,0xf4,0x66,0x53,0x0c,0x1d,0x70,0x69,0x7f,0xd3,0xa3, +0x2b,0x74,0x4e,0xca,0x95,0x65,0xd6,0xe0,0x81,0x84,0x4d,0x18, +0x44,0xaf,0x56,0x41,0x44,0x76,0xbf,0x04,0x5e,0x3d,0x1f,0xf4, +0xb3,0xd3,0x68,0x2b,0x32,0xf2,0xaf,0xdb,0xc6,0xe6,0x88,0xc5, +0x2c,0xfc,0x5c,0x61,0x2e,0xcb,0x19,0xb9,0xaa,0x36,0x90,0xc5, +0x2f,0xb8,0xc8,0xc2,0x0e,0xce,0x4e,0x0f,0x12,0x1a,0x82,0xfa, +0x38,0x67,0x9b,0xcd,0xef,0xda,0xb5,0xc9,0x5d,0xfa,0x2d,0xd0, +0x13,0x83,0xae,0x4e,0x9b,0xf2,0xbc,0x9f,0x6e,0xfd,0x44,0xc7, +0x47,0xc5,0x89,0xd9,0xcf,0xc1,0x4d,0x85,0xcd,0x3e,0xa1,0x19, +0xd5,0xc3,0x90,0xaf,0xb3,0x52,0xb6,0x17,0x9a,0x14,0x11,0x9e, +0x8f,0xba,0xe8,0x6e,0x46,0xd4,0x26,0x97,0xe9,0x57,0x15,0x75, +0x64,0xfd,0xf3,0xd9,0xf4,0xbb,0xcd,0x50,0xc1,0x12,0x62,0xd6, +0x29,0x4a,0x20,0xe3,0xab,0x50,0x47,0x3e,0xd9,0xef,0x5e,0x60, +0x4f,0x7a,0x44,0xed,0xa1,0xac,0xb3,0x45,0xc8,0x61,0x77,0xd5, +0x64,0x4b,0x76,0x24,0x22,0xf5,0x02,0x96,0x13,0xf2,0x88,0xf9, +0xbd,0x23,0xd6,0x1c,0xa1,0x61,0x45,0x98,0x1e,0xa5,0xa4,0x8e, +0x7f,0x46,0x50,0x4a,0x43,0x3d,0xfe,0x46,0xb1,0xc2,0x24,0x2d, +0xc6,0xa8,0x3e,0x5a,0x81,0x75,0x09,0x5b,0x17,0xa1,0xd0,0xf6, +0xf0,0x68,0x7c,0x8f,0xdf,0xed,0xfc,0x71,0xef,0xfd,0xee,0xce, +0xf1,0x9e,0x6c,0xb6,0x32,0xc7,0xaf,0xc4,0x10,0x65,0x06,0x42, +0xd2,0x70,0x98,0x46,0xad,0xe6,0x76,0x73,0x6b,0x63,0x72,0xd1, +0xdc,0x46,0x8b,0xbb,0x1f,0xc6,0x93,0xb4,0x85,0x27,0x44,0xf0, +0xbb,0x6d,0xfa,0xe3,0x59,0x8a,0x3a,0x04,0x52,0x5b,0x74,0xde, +0x9b,0xe4,0xd7,0xbd,0x51,0x7f,0x03,0x28,0xf0,0x86,0x5e,0x79, +0xe1,0x29,0x78,0xb4,0x84,0x49,0xbe,0x8b,0x0e,0xc3,0xb2,0x3f, +0xfc,0x27,0x4e,0xf6,0xe6,0x05,0x05,0xee,0x31,0x9e,0x2c,0x00, +0x4a,0x74,0xfe,0xf9,0x16,0x0a,0x8f,0x51,0xdc,0xdb,0x88,0x1a, +0xb1,0xd5,0x3a,0x4e,0xec,0xd3,0x88,0x38,0x81,0x02,0x6f,0x60, +0x41,0x43,0xd1,0x89,0x2e,0x8e,0xec,0x8d,0x10,0xba,0xcf,0xe5, +0x39,0xea,0x2c,0x9d,0x14,0xb3,0x94,0x06,0x9f,0x8d,0x7b,0x5f, +0x8e,0xfc,0xad,0x51,0xb1,0x48,0x2c,0x39,0x70,0xef,0x58,0x1e, +0x86,0x8a,0xf2,0xb3,0x19,0x34,0x4c,0x6e,0xb4,0xd0,0x66,0x95, +0xcd,0xb1,0x34,0xbc,0x6a,0x3e,0x86,0x73,0x45,0xae,0x27,0xa0, +0x5d,0x51,0xea,0x6b,0x31,0xef,0xe7,0x2b,0x31,0x3f,0x3e,0xdc, +0xdb,0x7d,0xb3,0xf3,0xf6,0xfd,0xee,0xeb,0x9d,0xa3,0x63,0x47, +0x2c,0xf1,0xe9,0x59,0x7a,0xb6,0x71,0xf6,0xbe,0xdb,0x88,0x9b, +0x09,0x6a,0x3b,0xef,0x1f,0xfc,0xf5,0xfd,0xeb,0x9d,0xdd,0x3f, +0xba,0x1c,0x3f,0xdf,0x8c,0xff,0x1e,0x5b,0xdb,0x7b,0xcc,0xe4, +0x0b,0x3c,0x98,0x23,0x81,0x0f,0xf3,0xf0,0x1f,0x3a,0x33,0xf8, +0x42,0x0e,0xca,0xd3,0x39,0xb9,0x74,0xf7,0xed,0x40,0xd4,0xfd, +0xcd,0xf3,0x64,0x70,0x75,0x45,0x1a,0xcf,0xe4,0x78,0x0e,0x77, +0x65,0xf8,0x4b,0xb7,0xe6,0x64,0x20,0x8a,0x1f,0x73,0x4f,0xd8, +0x3b,0x40,0x84,0x1a,0x76,0xa8,0x62,0x83,0x27,0xb6,0xaa,0xe3, +0x7b,0x68,0x4d,0x0e,0x9d,0xa1,0x4f,0x99,0xd6,0xd0,0x3e,0x96, +0x40,0x4e,0x52,0x76,0xe2,0x84,0xbb,0xba,0x97,0x07,0xfb,0xbb, +0x6c,0x42,0xfd,0xed,0xb8,0xd7,0xcf,0xfb,0x91,0xb1,0x99,0xa4, +0x15,0xd6,0x4c,0x0c,0xe5,0x46,0x53,0xd7,0x3e,0x07,0x74,0xa6, +0x5d,0x90,0x8a,0x9e,0x3f,0x79,0xbb,0x55,0x16,0x73,0x49,0x31, +0xcc,0x0f,0x1f,0xe9,0xc3,0x59,0x15,0x8a,0x08,0xb8,0x07,0x49, +0x8a,0xa7,0x72,0xc2,0x2b,0x02,0xae,0x01,0x09,0xb7,0x11,0x75, +0x61,0xd6,0xe5,0xbf,0x28,0x5f,0xf0,0x2c,0x13,0x3a,0x95,0x6e, +0x08,0xa1,0x6e,0x36,0x8f,0x01,0xd5,0xc3,0x7e,0xa4,0xba,0x49, +0xaa,0xe3,0xa8,0xe1,0xa2,0x05,0x2d,0xe9,0x4b,0x71,0xf5,0x02, +0x33,0x1f,0xf1,0x48,0x69,0x0d,0x46,0x75,0xc6,0x14,0x97,0x67, +0xf8,0x35,0xec,0x1a,0x86,0x82,0xf4,0xb5,0x34,0x3f,0x1e,0x1c, +0xbc,0xdd,0xdb,0x79,0xf7,0x7e,0xe7,0xe4,0xe4,0x08,0x0d,0x5b, +0xd9,0xf6,0x45,0x37,0xf3,0xe1,0x6c,0x00,0xe3,0x21,0x1b,0xbf, +0x1c,0x7d,0x93,0xd3,0xfe,0xd5,0xf4,0x07,0x53,0xb2,0xd6,0x6a, +0x70,0x6c,0x0e,0x46,0xc3,0x2f,0x86,0x6d,0xd6,0x43,0x0c,0x30, +0xd0,0x51,0x24,0xaf,0x48,0x61,0xed,0x4d,0x4a,0x5a,0xfd,0xba, +0xba,0x53,0xbf,0x38,0x72,0x6a,0x57,0x54,0x17,0x58,0x06,0xb1, +0x59,0xf7,0xde,0xee,0xed,0xef,0xbd,0x3b,0x39,0x0e,0xb0,0x63, +0x95,0x79,0x46,0x0d,0x6a,0x25,0x59,0x1f,0x0f,0xc5,0x80,0xcd, +0xf5,0xcc,0xf9,0x7c,0x36,0x83,0x30,0x6e,0xa8,0x0d,0xce,0x1f, +0x74,0x3d,0xf2,0x14,0xa4,0x6c,0x4d,0xa7,0x7e,0x6d,0xb0,0x88, +0xad,0x6d,0x2d,0x13,0xa7,0xea,0x71,0xdf,0xef,0xcd,0x7a,0x29, +0x53,0x15,0xb0,0xc1,0x9e,0x19,0x8c,0xae,0xf3,0x09,0x04,0xfa, +0x2f,0x7d,0xc2,0x1b,0x1d,0xc7,0x6e,0xba,0x56,0xaf,0x17,0x15, +0x45,0x1c,0x4b,0x40,0xcb,0xcb,0x68,0x7c,0x2e,0x71,0x6a,0x82, +0xc3,0x7c,0x22,0x95,0x78,0x95,0x4e,0x63,0x0d,0x03,0x7d,0x6b, +0x25,0xce,0x28,0x91,0xe3,0x89,0x68,0x33,0xab,0xcc,0xa1,0xd8, +0x1c,0x51,0xc8,0xe1,0xbc,0x75,0x2d,0x4a,0x5d,0xe2,0xeb,0x6d, +0xda,0x87,0x08,0x9a,0xaf,0x25,0x68,0x2e,0x20,0xaa,0x12,0xa2, +0xbd,0x9a,0x17,0xb7,0x5a,0x99,0xb7,0xde,0x4a,0x10,0xe9,0x5d, +0x4b,0xa0,0x08,0xe3,0x66,0xa7,0x9d,0x98,0xbf,0x75,0x06,0xc4, +0x20,0xd2,0xad,0xf7,0xd6,0xb1,0x13,0x5b,0xcd,0xac,0x8a,0x95, +0xbd,0xa3,0xbd,0x27,0x92,0xf2,0x51,0x6f,0x3e,0x1b,0x47,0x09, +0x1d,0x5f,0x62,0xf4,0x62,0xe1,0xa4,0x04,0x55,0x28,0xac,0xd3, +0x82,0xe8,0xb8,0x9d,0x2a,0x4a,0x3c,0x04,0x62,0x59,0x51,0x5a, +0x88,0x9d,0xb6,0xa7,0x0f,0x77,0x4a,0xb0,0x1b,0xeb,0x57,0x5a, +0xe6,0x85,0x7e,0xab,0x9e,0x5a,0x94,0xbd,0x6b,0x9f,0x95,0x14, +0xfa,0x30,0x94,0x56,0xac,0x4b,0xab,0x58,0x1f,0x77,0xd2,0xb0, +0x4e,0x71,0x9d,0x47,0x40,0xfd,0x37,0xb3,0xfc,0x86,0xab,0x64, +0xe7,0xfc,0xf8,0x64,0x3c,0xbf,0x18,0x5c,0x0e,0xf2,0x7e,0x27, +0xac,0x33,0xf5,0xfb,0x75,0xb9,0x7e,0x0a,0xaa,0xc1,0x45,0xc0, +0x75,0x8a,0xb6,0xcf,0xc6,0x6e,0xfe,0x02,0x50,0x78,0x2f,0x57, +0x28,0xdc,0x32,0xab,0x28,0xb0,0x58,0x94,0x95,0xec,0xe8,0x28, +0xa3,0xd0,0xfa,0xd5,0xd5,0x73,0x86,0xb6,0x2d,0x69,0x33,0x5e, +0xa9,0x8c,0x3c,0x2a,0xd0,0x81,0x59,0x55,0x2a,0x83,0xd9,0xf6, +0x5e,0x3a,0x58,0x51,0x0b,0x42,0x8a,0x38,0x20,0xb4,0x44,0x87, +0x7a,0xe3,0xdb,0x87,0x87,0xdf,0x49,0xde,0x21,0xfe,0xc5,0x0e, +0xa5,0x58,0x73,0x87,0xe6,0x80,0xb7,0x9c,0xb0,0xca,0x69,0x4b, +0xe2,0x8d,0x7a,0x69,0x73,0x6b,0x34,0x0a,0x70,0x90,0xe6,0x2a, +0x56,0x24,0x87,0x65,0xf0,0x44,0x20,0x7b,0x77,0xf0,0x72,0xef, +0xfd,0xc9,0x5f,0x0e,0xe1,0xcf,0xde,0x9f,0x4f,0xde,0x1f,0x1e, +0x1d,0x1c,0xee,0x1d,0x9d,0xfc,0xe5,0x54,0x0b,0xb5,0x78,0x08, +0xdc,0x6d,0xaf,0xd4,0x5b,0xeb,0x58,0x60,0x56,0x12,0x3e,0xb5, +0x11,0xdd,0x34,0x8a,0xfc,0xf0,0xba,0xd8,0xb0,0x39,0x24,0x29, +0xac,0x40,0x43,0x2d,0xce,0xb2,0x99,0x88,0x57,0x21,0xdc,0xea, +0x66,0xb0,0x72,0x80,0x14,0x76,0x42,0xde,0x68,0x56,0x65,0x7b, +0x0e,0xd9,0xb0,0x4d,0x74,0x2e,0x0a,0xfb,0x96,0x07,0xc0,0x3d, +0x04,0x02,0xdb,0x22,0x22,0x49,0x64,0xa4,0x9f,0x9b,0xeb,0xfd, +0x3b,0x7c,0x76,0x28,0xc1,0x65,0x4c,0xec,0xa1,0x3c,0xfe,0x9e, +0x1d,0x96,0xfb,0x92,0x4d,0xfc,0x1c,0xc3,0x6a,0xb5,0x7b,0x82, +0x2c,0xc7,0x6d,0xa6,0x1c,0xc7,0xf7,0x9c,0xce,0xae,0xe2,0x96, +0x5c,0xa7,0x10,0xae,0x90,0x66,0xdc,0xf1,0x38,0xad,0xa4,0x4a, +0xb0,0xe1,0x88,0xe4,0x9e,0x7f,0x9b,0x56,0x16,0xc0,0x63,0x53, +0x04,0xc5,0xf2,0x8e,0x24,0xde,0xb1,0x1a,0xa2,0x84,0xd8,0xe0, +0x17,0xbf,0x29,0x97,0xcc,0xa2,0x41,0x41,0xf7,0xf0,0x1c,0xb5, +0x2c,0xec,0x97,0x98,0x25,0x06,0x21,0xcb,0x26,0xd1,0x74,0xfe, +0x57,0xf6,0x53,0x01,0x38,0x8d,0x3c,0xee,0x07,0xb5,0xf5,0x08, +0x43,0x0e,0xf0,0xd0,0x14,0xe3,0xd4,0x2f,0x1a,0x2e,0xaa,0x3d, +0xf8,0xc1,0x07,0xf4,0x73,0x12,0x59,0xfc,0xf3,0xde,0x70,0x7c, +0x11,0xfb,0x2c,0xf4,0xa2,0xa4,0x54,0x9f,0x34,0x61,0x69,0x94, +0x98,0x2c,0xeb,0x68,0x51,0xa0,0x96,0xb9,0xac,0xde,0x9c,0x5d, +0xb5,0xd0,0x89,0xe1,0xb6,0xf5,0x31,0x86,0xc6,0x9a,0xa0,0xb5, +0xf1,0x76,0x96,0x5d,0x8e,0xdc,0x13,0xba,0xcb,0xd1,0x5a,0x96, +0x85,0x6b,0x6d,0x10,0xe9,0xe5,0x84,0x0e,0x42,0x4c,0x09,0xa2, +0xf6,0xa5,0x23,0x5d,0x28,0x87,0xbd,0x98,0x27,0xe1,0x37,0x6b, +0xfc,0x9c,0x46,0x09,0x9b,0xd6,0x60,0xd7,0xe5,0x28,0xb3,0xd0, +0x51,0x92,0x48,0x40,0xa2,0x67,0x59,0x95,0x9c,0x2e,0xb6,0x84, +0x7b,0x22,0x08,0x31,0x98,0x45,0xb1,0x2a,0x17,0x44,0x53,0x4e, +0x36,0x47,0xa5,0x7d,0xf7,0x3b,0x43,0x4f,0x4c,0x00,0xd0,0x4e, +0x98,0x35,0xe6,0xc3,0x07,0x7b,0x77,0xe5,0xb0,0xee,0x38,0x6b, +0x2a,0x0a,0x41,0x83,0x4f,0xc0,0x54,0x90,0x2c,0x44,0x7d,0x78, +0xf1,0xc1,0xdb,0x85,0x72,0x73,0x3b,0xb3,0xe8,0x7c,0x60,0xa4, +0xb9,0xa7,0xdb,0x8a,0xee,0xf8,0x7c,0xa3,0xe1,0x4a,0xa4,0xee, +0x6b,0x19,0x78,0x1e,0x58,0xdd,0x55,0x85,0x7e,0x91,0x2a,0x74, +0x6b,0xb5,0x64,0xc9,0xab,0x99,0x12,0x23,0x8f,0x5c,0x04,0x88, +0xb2,0x48,0x6d,0xa9,0x26,0x3d,0xa3,0x1c,0xb6,0xd6,0xc6,0xa3, +0x57,0xa5,0x8d,0xa4,0x99,0x8f,0xc4,0xb1,0x85,0x1b,0x63,0xbb, +0x8a,0xe8,0x14,0xd1,0xc3,0xd3,0x9b,0x41,0xdc,0x6d,0x02,0xae, +0xe8,0x3e,0x83,0xa3,0xff,0x0d,0x17,0x8e,0xb2,0xbf,0x0c,0xf4, +0x64,0xf1,0x37,0x7b,0x4a,0xfd,0xb7,0xda,0x78,0x52,0xfb,0x1b, +0x3b,0xf6,0x04,0x94,0xff,0x86,0x36,0x43,0x80,0xa0,0xf1,0x52, +0x2b,0x92,0xcb,0x67,0x4c,0x9b,0x0a,0xe1,0xec,0xf1,0xb1,0xc2, +0x31,0x94,0xcc,0xbd,0x28,0xca,0x59,0x22,0x34,0x7e,0x8d,0x96, +0x81,0x1f,0xcc,0xcb,0x59,0xac,0x75,0xea,0xe9,0x62,0xf1,0x04, +0xc0,0x46,0x70,0xb8,0x5f,0xda,0x2a,0x1e,0x2e,0x26,0x75,0x58, +0x74,0xc4,0x50,0x2b,0xc2,0x70,0xa6,0x8b,0x25,0x2b,0x03,0x4e, +0xfc,0x78,0xd2,0x49,0x9c,0xec,0x3c,0x6a,0x7a,0xe7,0x41,0xde, +0xa6,0xee,0x5d,0x87,0xbc,0x1a,0x4d,0x59,0xcb,0x77,0xca,0xae, +0xa7,0x48,0x71,0xd2,0xa6,0x30,0xbf,0xbf,0x19,0xcf,0xa7,0x79, +0x8e,0xce,0x49,0xc4,0x44,0x21,0x3e,0x50,0xc7,0xb8,0x61,0xde, +0xbb,0xcb,0x25,0xce,0xf9,0x72,0x83,0x4d,0x0f,0x80,0x0c,0x6d, +0xa1,0xdb,0x68,0x65,0x6e,0x5c,0xe2,0xe9,0xb1,0xdd,0x4b,0x89, +0x3c,0x1c,0xb3,0xe9,0xec,0xc2,0xfb,0x4f,0x7a,0x68,0x87,0x7a, +0xca,0xd9,0xef,0x60,0xe2,0xf7,0xdc,0xa2,0xdf,0xe9,0xb9,0x53, +0x12,0xab,0x7d,0x89,0x3b,0xb0,0xdb,0xec,0xbc,0x5e,0x3f,0x97, +0x77,0x52,0xc8,0x1b,0xdd,0x43,0x3b,0x28,0x0d,0xc9,0x8b,0xc5, +0x5a,0xbc,0x46,0xbf,0x68,0x7e,0x0a,0x3e,0x1c,0x40,0x4c,0xa1, +0x8a,0x1c,0xc2,0x9d,0x30,0x18,0x43,0xee,0x24,0xed,0xad,0xc2, +0xbb,0x5e,0x6f,0xfd,0xa6,0xbe,0x32,0x95,0x0a,0x27,0xfa,0xe8, +0x85,0x5a,0x87,0x8e,0x03,0xd8,0xaf,0xe9,0x79,0x16,0x60,0x8d, +0xac,0xee,0x1c,0x1b,0x5c,0xb6,0x46,0xb8,0x6c,0xeb,0x21,0xb3, +0xe7,0xb3,0x14,0x77,0xd3,0xbb,0xcd,0xee,0xfd,0xe8,0xa4,0x3c, +0x52,0x63,0xd4,0x97,0xf6,0xe3,0x68,0x63,0xef,0x60,0x48,0x97, +0xed,0x60,0xf6,0x5a,0x20,0x0c,0xba,0x52,0x31,0x5c,0xbb,0xe2, +0x9d,0xe4,0x43,0x34,0xac,0x23,0x8a,0xe2,0x12,0x3a,0xa1,0x0c, +0xed,0x78,0x4d,0xc2,0xa8,0xe0,0x4c,0x1f,0xda,0x65,0xab,0xeb, +0x53,0xf1,0xca,0x2b,0x59,0xf0,0x21,0x28,0x93,0x3b,0x57,0xc9, +0x3e,0xe7,0x96,0x22,0x4c,0x97,0xcf,0xf8,0x42,0xb6,0xc3,0x25, +0x13,0x53,0xec,0x9e,0x4a,0x52,0x5f,0xda,0x58,0xb9,0xb7,0x1e, +0x61,0x35,0x68,0x0b,0x47,0x18,0xdd,0xc1,0xe5,0xa5,0x91,0xb3, +0xe4,0xff,0x1a,0xcc,0xae,0xcb,0x22,0x80,0x24,0xe2,0x68,0xc9, +0x22,0x49,0xd7,0x6d,0xe2,0x87,0x3e,0x7c,0xb0,0x47,0x43,0x1a, +0xac,0xdd,0x56,0x64,0x72,0x53,0x16,0xcf,0x36,0xe5,0xc0,0x48, +0x43,0xf6,0x63,0x30,0xa2,0x8a,0xa8,0x92,0x8e,0x78,0xeb,0x47, +0x5f,0x6a,0x93,0xd9,0x8f,0xf9,0x25,0xf2,0x0d,0x4c,0x37,0x94, +0xdc,0xd4,0x0e,0x70,0x53,0xc9,0x2b,0x40,0x77,0x51,0x66,0xe0, +0xbc,0x0e,0x05,0x2a,0x44,0xe2,0x1c,0x76,0x01,0x49,0x15,0x50, +0xa4,0x62,0x43,0x4f,0x93,0x5c,0x92,0x1f,0x90,0xe6,0xbc,0x5c, +0x62,0xca,0x30,0xf0,0x15,0x41,0x51,0x80,0xaf,0xd7,0x2d,0x54, +0x51,0xa8,0x91,0xcc,0x78,0x55,0x2b,0x09,0x7c,0x08,0x81,0xc3, +0xf7,0xf8,0xe9,0xb7,0xaf,0x1f,0xef,0x39,0x60,0xa7,0x7b,0x7b, +0x9b,0x8f,0xfa,0x15,0x9b,0x1d,0xea,0xd1,0x8a,0x01,0x18,0x85, +0x3d,0x4f,0xf0,0x92,0xfb,0xb8,0x0a,0x75,0xe0,0x23,0x55,0xd1, +0x5a,0x79,0x87,0x6a,0xe7,0x7e,0x67,0x48,0xd8,0xcb,0xb7,0x30, +0x1e,0x0f,0x20,0xc5,0x06,0x70,0xcb,0x60,0x3d,0xa1,0xb9,0xb4, +0xcb,0xc1,0x64,0x3a,0x23,0xf0,0xed,0x27,0x37,0xc5,0x8b,0x8a, +0x8a,0x80,0x28,0x8d,0x89,0x81,0xae,0x6b,0x0d,0xba,0x0d,0x2c, +0x23,0x88,0xb1,0x4c,0xf4,0xf6,0x2b,0x93,0xf3,0x41,0x97,0x82, +0xd6,0xda,0xc4,0xc0,0xf7,0x8a,0xb9,0xc0,0x9f,0x64,0x90,0xbb, +0x44,0x9c,0x16,0x8c,0x27,0x50,0x1b,0x13,0xf4,0xa5,0x23,0x12, +0x39,0x36,0xaa,0xa0,0x8b,0xca,0x19,0xf7,0xb5,0x98,0xd1,0xe5, +0x76,0xa1,0xc6,0xde,0x25,0xf2,0xd1,0xf2,0xd8,0xe5,0x9f,0xf6, +0xf4,0x5c,0x09,0x06,0xea,0x01,0xce,0x50,0x35,0x6e,0x1e,0x54, +0x91,0x05,0x7c,0xc5,0xe4,0x2f,0x4c,0x6d,0x7b,0x97,0x2f,0x2c, +0x6e,0x47,0x82,0x46,0xdd,0xe0,0x07,0xe2,0x1f,0xa7,0xf2,0xc3, +0x3f,0x4e,0x2d,0x35,0xd9,0x4a,0x60,0x38,0x3d,0xfb,0xec,0x1e, +0x25,0x50,0x67,0xf3,0xd1,0x68,0x3a,0xd4,0x06,0xb2,0xb5,0x70, +0xd3,0x50,0x02,0x07,0xb2,0x8c,0xcf,0xdd,0x09,0xd1,0x2d,0xa3, +0x98,0x94,0xcb,0x2f,0xa5,0xbb,0x57,0x70,0xb1,0xd5,0x63,0xe1, +0x8d,0xce,0x11,0x19,0xb4,0x50,0x4c,0x90,0x1e,0x77,0xa2,0x87, +0x80,0xa6,0x57,0x55,0xa4,0x90,0x57,0x51,0x8b,0x3a,0xbf,0xc1, +0x1c,0x32,0x98,0x76,0x60,0x8a,0xf7,0x75,0xa5,0x1c,0x6e,0xaf, +0x98,0x0f,0xfd,0xeb,0x6f,0x35,0xb4,0x6d,0xac,0x1c,0xef,0x17, +0x6f,0x00,0xc7,0x35,0xe4,0x14,0x37,0x0e,0xbf,0x76,0xc2,0x85, +0x6e,0x82,0x02,0xae,0xca,0x1b,0xd8,0x10,0x0e,0xaa,0x58,0x4f, +0x49,0xd5,0x4a,0x1d,0x31,0x09,0x76,0xd3,0x1f,0xbf,0x9c,0xf4, +0xae,0x70,0xd7,0x1f,0xab,0xae,0xbe,0x18,0x8e,0x47,0xb9,0x3d, +0x1a,0xc6,0x6f,0x7b,0xc3,0x21,0x82,0x6a,0xb9,0x2a,0x5a,0x7c, +0x11,0x8c,0x71,0x42,0x7b,0x41,0x2c,0x8d,0x9f,0x22,0xaa,0x93, +0x5b,0xa8,0x53,0x07,0x4c,0x96,0x7a,0x84,0x96,0xb9,0x2f,0xc8, +0x53,0xf0,0xf9,0x8f,0x07,0x88,0x66,0x3a,0x1b,0xdf,0xe2,0x89, +0x4f,0xef,0x8a,0x1e,0xa6,0x52,0xe4,0xd2,0xbf,0xa8,0x16,0x34, +0x4c,0x78,0xc3,0x34,0x62,0x4b,0x54,0x41,0x2b,0x08,0x77,0xe2, +0x97,0xff,0xc0,0x36,0x1b,0xff,0x3c,0xf7,0xd6,0x5a,0xf8,0x10, +0xaa,0x7a,0x63,0x57,0xda,0x8c,0xc6,0x77,0xc1,0x9e,0x52,0x6f, +0xf8,0x18,0x2c,0x9f,0x56,0x84,0x6e,0x0c,0xa5,0x90,0x70,0x6e, +0x39,0xb8,0x48,0x3c,0x63,0xa0,0xe5,0x53,0x6c,0xed,0xad,0x86, +0xac,0x9f,0x30,0x85,0x48,0x61,0x01,0xb9,0x78,0x37,0xc5,0x5e, +0x68,0x9e,0x03,0xf9,0x95,0xee,0xf4,0x9a,0xe3,0x51,0x39,0xe7, +0x7c,0xb4,0x22,0xef,0xe5,0x25,0x08,0x07,0xc0,0x3e,0xae,0xf7, +0x7b,0xb7,0xfa,0x5e,0xf0,0x76,0xae,0x26,0x23,0x6e,0xf5,0xe5, +0xa4,0x86,0x1a,0x80,0xee,0x91,0xfe,0x98,0x7f,0xc1,0x78,0x77, +0xe9,0x83,0x47,0x62,0x41,0x89,0xc0,0xeb,0x75,0x58,0xa4,0xbc, +0xd4,0x7c,0xcc,0xbf,0xe8,0xfb,0x3a,0x2a,0x82,0x4f,0x27,0x83, +0x62,0xb6,0x93,0xe4,0x25,0xbf,0xcd,0xd4,0x35,0x72,0x3e,0x43, +0x2b,0xd2,0xab,0x77,0xef,0x77,0x8e,0x7e,0x7f,0x9c,0x6d,0xfe, +0x6c,0x81,0x9f,0x4d,0xbf,0x3b,0xfd,0xf9,0x2c,0xee,0x7e,0x77, +0x16,0xc3,0x67,0x0c,0xdf,0x49,0xf7,0xbb,0xe4,0x2c,0xd9,0xbc, +0x31,0x9c,0x99,0x6d,0x67,0x65,0x9b,0x66,0x53,0x22,0x32,0xd2, +0x83,0x88,0xdf,0xa3,0x71,0xdc,0xe3,0x46,0x27,0x39,0x6b,0xb1, +0x12,0x04,0xda,0xfd,0x3a,0x7c,0xbf,0x7b,0xb0,0xcf,0x17,0x5a, +0x9b,0x71,0x7c,0xb6,0x89,0x6a,0x23,0xeb,0xc9,0x02,0xbf,0xbe, +0x3b,0x3d,0x9b,0x9e,0x1d,0x77,0xbf,0xeb,0x9c,0x7d,0x77,0xb6, +0x89,0x77,0xb7,0x37,0xde,0x31,0x7e,0xe1,0x86,0x56,0x5d,0xd7, +0x98,0x75,0xf6,0x6b,0x54,0xbc,0xc4,0xb5,0xd1,0x98,0x83,0x5d, +0x5f,0x39,0x7f,0xa2,0xe4,0xe3,0x93,0xbf,0x03,0x57,0x9e,0xd6, +0x4b,0x86,0xf8,0x19,0xb7,0x9c,0x05,0xb7,0xca,0x26,0x74,0x6c, +0xe1,0x97,0xc2,0xc0,0x7d,0x86,0x7d,0xb1,0x4a,0x8e,0x29,0xa2, +0x0d,0xef,0x3c,0xac,0x2d,0x17,0x3b,0xf5,0x7a,0xd4,0x8c,0xd6, +0x28,0x5d,0x59,0x9d,0x11,0xe7,0x9d,0x41,0x3b,0xf0,0x01,0xe0, +0xec,0x02,0x50,0x70,0xef,0xff,0xd0,0xa7,0x26,0x29,0xef,0xd5, +0x2c,0x62,0x35,0xfb,0x9c,0xaf,0xf6,0x09,0xb6,0x06,0xb5,0x67, +0xcd,0x67,0xb5,0xab,0xf1,0xac,0xf6,0xec,0x7e,0x6b,0xf9,0xac, +0x19,0xf1,0xdc,0x6f,0x17,0x1b,0x43,0x73,0xdf,0xdb,0xea,0xef, +0xe2,0x4b,0x59,0x53,0x72,0xf8,0x81,0x91,0xce,0xd1,0x47,0xe8, +0xbe,0x43,0x1e,0xf3,0xea,0x9e,0x93,0x28,0xa7,0xf5,0xca,0xfb, +0xbb,0x12,0xb7,0xe5,0xf5,0x8a,0xa5,0x16,0xd3,0x07,0xfe,0x2c, +0xdb,0x6a,0x0c,0x93,0x00,0x47,0x5f,0xf5,0x3a,0xfd,0x9c,0xd2, +0x5f,0x6b,0x4b,0xb4,0xd5,0x35,0x7e,0x71,0xcc,0x42,0xe1,0x88, +0x3c,0xee,0x39,0x20,0x52,0x9c,0xfd,0xd4,0xb8,0x22,0x5c,0xe9, +0x3b,0xbf,0x0e,0x65,0x95,0x05,0xd4,0x42,0xc5,0xcf,0xcb,0xdb, +0x85,0x4d,0x45,0xb5,0x10,0x44,0x65,0xcb,0x82,0x50,0xb1,0x4e, +0xdc,0x48,0x60,0xb3,0xeb,0x75,0xdb,0x63,0x31,0x06,0xcd,0x16, +0x69,0x43,0x18,0xde,0x2b,0x97,0x7a,0x8d,0x3b,0x2a,0xbc,0xef, +0x89,0x1f,0x04,0x54,0x29,0x86,0x56,0xf5,0xbe,0xb8,0xd1,0x9d, +0xa9,0x33,0x9c,0x72,0x2e,0x25,0xaf,0x95,0x60,0x12,0x3d,0xd2, +0xfa,0xc9,0x00,0x5d,0xd8,0x3b,0xa4,0x73,0x51,0x49,0xc7,0x7d, +0xa6,0x81,0x1a,0x6e,0x21,0xd1,0xab,0xc3,0x6a,0xbb,0x38,0xa1, +0x66,0x30,0x15,0x79,0xa8,0x0b,0xb4,0x28,0xf9,0x2f,0xc4,0xba, +0xa4,0x1a,0xfd,0x28,0xe2,0xe2,0x0a,0x96,0xe5,0x80,0x65,0x77, +0xd9,0x35,0xeb,0x78,0x94,0x33,0x18,0x96,0xf8,0x9b,0x44,0x47, +0x49,0x7b,0x7d,0x97,0x3f,0x2d,0x7f,0xf3,0x3a,0x73,0x9e,0xcf, +0xf1,0x4e,0xe8,0xf0,0x68,0xef,0xd5,0x9b,0x3f,0x7b,0x55,0x98, +0xf8,0x33,0xa9,0xcb,0xbc,0xef,0x2e,0xf0,0x4e,0x5f,0xbe,0x93, +0xcd,0x81,0xf9,0xf3,0xeb,0x23,0xab,0x04,0xf4,0xe7,0xfd,0xb7, +0xaf,0x67,0xb3,0xdb,0x23,0xf6,0x75,0xb4,0x58,0x68,0x73,0x00, +0x93,0x2f,0x4e,0x1d,0x1e,0xf6,0x0c,0xec,0xa7,0xee,0xcf,0x72, +0x8a,0x1e,0xed,0x4f,0x3f,0xdf,0x0c,0xb7,0xa9,0xfc,0xc9,0xc9, +0x61,0xf3,0x37,0xcd,0xad,0xc8,0x39,0x45,0x6b,0x25,0xf7,0xcb, +0xaf,0x2a,0xfd,0x5c,0x97,0xde,0xfe,0xca,0xd2,0xbe,0xe4,0x73, +0x2c,0x19,0x3c,0xba,0x5e,0x47,0x3f,0xc3,0x3f,0xf6,0x2e,0x3e, +0xc2,0x8e,0x8e,0xdf,0x5d,0x7f,0xbe,0x46,0xff,0x76,0x27,0xe4, +0xcd,0x98,0x1f,0x0e,0x95,0x8f,0x7d,0xc3,0x4e,0x41,0x87,0xc3, +0xb8,0x56,0xc1,0x9c,0xb9,0x1d,0x0f,0x2b,0xd6,0x22,0x95,0x04, +0xeb,0xd1,0xf1,0xde,0xd1,0x9f,0xf6,0x8e,0xde,0xef,0xef,0x9c, +0xec,0xbe,0xc6,0x41,0x38,0xfd,0x39,0xed,0x36,0x92,0x14,0x17, +0xc3,0xf8,0xec,0x53,0x23,0xbd,0xdf,0x32,0xad,0xe5,0xd9,0xa7, +0xef,0xfe,0x23,0xe9,0xc4,0x67,0xf7,0x9d,0xd3,0xb3,0x4f,0x67, +0xcd,0x0d,0x58,0x7f,0x97,0xb0,0xa0,0xa6,0xf1,0xe9,0xd6,0xc6, +0xef,0xba,0x68,0x0f,0x14,0xd6,0x4c,0x58,0x8c,0x3b,0xff,0x06, +0xab,0x31,0x7c,0x77,0x00,0x0c,0x7e,0xc2,0xf7,0xbf,0xe1,0xeb, +0x79,0x34,0x08,0x6a,0x0e,0x77,0x4e,0x5e,0xeb,0x7a,0x38,0xf7, +0x8a,0xcc,0x2f,0xf7,0x5e,0xed,0xfc,0xf4,0xf6,0xe4,0xfd,0xe1, +0xc1,0x11,0xaa,0x9b,0x90,0xff,0xe5,0xdf,0x6e,0x19,0xfc,0x9d, +0xa6,0xdf,0x7f,0xff,0xdc,0x5c,0x42,0xc4,0x76,0x6b,0xc9,0x36, +0x27,0x91,0x04,0x8a,0xcd,0xb4,0xf1,0x40,0x93,0x6f,0xe5,0x13, +0x65,0xa3,0xf3,0xde,0xe8,0xea,0xcd,0xe8,0xf5,0xec,0x66,0xf8, +0x6b,0xe7,0x07,0x8b,0xc4,0xa4,0x62,0xa6,0x15,0xa9,0xe5,0x82, +0xf7,0xeb,0xeb,0x78,0x21,0xf6,0x6b,0xf4,0x89,0xb8,0xbe,0x2e, +0xdb,0x76,0x0c,0xf4,0xce,0xa7,0x90,0x33,0xb5,0x88,0x88,0x6d, +0xe0,0x68,0x7d,0x9d,0x13,0x22,0x72,0x8e,0xe6,0x39,0x02,0xba, +0x65,0x93,0xd2,0xa5,0x5b,0x34,0x48,0x73,0x77,0x68,0xb2,0x88, +0xa2,0xc7,0x36,0xef,0x15,0xc0,0x77,0x2e,0x2b,0xbb,0x62,0x01, +0x2b,0x63,0x59,0x9b,0x04,0xf5,0x3a,0x15,0x45,0x9f,0x20,0x71, +0x3f,0x47,0x85,0xc4,0x9f,0x8e,0xde,0xe0,0x74,0x25,0x3b,0xfe, +0xde,0x0e,0x2e,0xec,0x33,0xad,0xf5,0x82,0xc5,0xc2,0xc5,0x4a, +0x69,0x71,0x3b,0x62,0xcd,0xe2,0x92,0xe5,0x7d,0x5e,0x93,0x51, +0xbe,0x53,0xf6,0x6f,0x81,0xf3,0xd8,0xd6,0x50,0xfa,0xd2,0x7b, +0x70,0x2b,0xf7,0x88,0x4d,0xc2,0x9b,0x06,0x74,0xe3,0x56,0xce, +0x81,0xd1,0x90,0x8a,0x04,0x5f,0x55,0x1e,0xa2,0x23,0x32,0xb5, +0x70,0x5d,0x48,0x3d,0xf6,0x79,0x7a,0xf8,0xcc,0xdb,0x75,0x37, +0xf9,0x4a,0x91,0x59,0x1b,0x6d,0xa2,0x71,0x3d,0xbc,0xc0,0x72, +0x82,0x50,0x87,0x60,0x41,0x42,0x03,0x3f,0xd0,0xe7,0x30,0x35, +0x5c,0xbb,0x96,0xc6,0xb0,0xa1,0x5b,0x15,0x7e,0xea,0x74,0x2f, +0xfe,0x6f,0x8a,0xce,0xde,0xd9,0xfd,0x4d,0x6d,0x2b,0x0d,0x07, +0x90,0x01,0x90,0x3f,0x96,0x5a,0x2b,0xa5,0x01,0xb7,0xea,0xe9, +0x94,0x82,0xb2,0x98,0xca,0xc8,0x8f,0x71,0x41,0x80,0xe6,0xc7, +0x4f,0x92,0xc7,0x99,0x43,0x5a,0x73,0x37,0x86,0xbe,0x34,0x49, +0x72,0xe1,0xfc,0x88,0xa3,0xc1,0x74,0x82,0x8d,0xbc,0x22,0xb6, +0x92,0xd7,0xe8,0xa8,0xcc,0xf9,0xa7,0x47,0x27,0xe8,0x74,0x8b, +0xe4,0x8a,0xfd,0x9b,0x0c,0x78,0xf2,0xb7,0x1a,0xee,0xf3,0x6a, +0x37,0x73,0xc8,0x4e,0x46,0x1f,0x58,0xf9,0x1f,0x6f,0x99,0x7a, +0xe4,0xa9,0x08,0xb5,0xca,0x23,0x25,0xe3,0x11,0xd2,0xfc,0xb3, +0x0c,0x3d,0xc9,0xf8,0x3e,0x53,0xdb,0x23,0x6b,0x42,0xc7,0x27, +0x76,0xd4,0xe6,0xc0,0x41,0x3c,0xe5,0x9f,0x6e,0x5a,0x19,0xab, +0x4a,0x2f,0xc3,0xde,0xc6,0x15,0x6a,0x4c,0xe6,0x2b,0x99,0x18, +0x91,0x5a,0x57,0x92,0x0a,0x26,0x46,0xc6,0x7b,0x2d,0xb7,0x93, +0xba,0xac,0xeb,0x2d,0xd0,0x25,0x03,0x6a,0xd2,0xf0,0x3d,0x22, +0x4d,0x4d,0x7e,0x06,0x56,0x64,0x4c,0xec,0xd1,0x3f,0x31,0xa8, +0x9e,0xb0,0x73,0x72,0x70,0x04,0xec,0x2d,0xc2,0xa6,0x47,0x15, +0xe0,0xf9,0x38,0x05,0xbd,0x66,0xe7,0x15,0xc9,0xe8,0x4c,0x2c, +0xa2,0x67,0x63,0x55,0x89,0xc0,0x1c,0xbd,0x82,0x0b,0xed,0xef, +0xd1,0x2a,0xba,0xa2,0x5f,0xe7,0xeb,0xdc,0x68,0x7b,0x8f,0xbd, +0xac,0x17,0xf8,0x41,0x37,0xe7,0xd9,0x79,0x18,0xe1,0xef,0x2d, +0x7b,0x49,0xc7,0x07,0xce,0x93,0x4e,0xaf,0x71,0x9e,0xf6,0xd2, +0x20,0xea,0xdc,0x2b,0x1f,0x2d,0xd1,0xc0,0xe0,0x3f,0x03,0x81, +0x38,0xc0,0xa0,0x97,0x6e,0x25,0x1b,0x71,0xa1,0xd2,0x2d,0x58, +0x04,0xa3,0xef,0x9e,0x52,0x5b,0x00,0xfa,0xbb,0xb0,0x26,0x80, +0xb1,0xf9,0xd5,0x30,0x36,0x4b,0x30,0x7e,0xf5,0xd5,0x30,0x7e, +0x55,0x82,0xf1,0xf3,0x57,0xc3,0xf8,0xb9,0x04,0x23,0x8b,0x84, +0x0e,0xb2,0x2c,0xfb,0x6a,0x70,0x78,0x57,0x58,0x04,0xb8,0xf6, +0x0d,0x70,0xd6,0x2a,0xe0,0x7c,0x13,0x3a,0x15,0xd8,0x7c,0x03, +0x32,0x25,0x28,0x3f,0x7c,0x35,0x90,0x1f,0x4a,0x30,0x5e,0x7c, +0x35,0x8c,0x17,0x65,0x3c,0xbe,0xbe,0x35,0x3f,0x94,0x5b,0xf3, +0xe2,0xeb,0xa1,0xbc,0x28,0x43,0xa9,0xd7,0xbf,0x1a,0x4a,0xbd, +0x5e,0x82,0x02,0x32,0xc1,0xd7,0x42,0x59,0x2c,0xca,0xb8,0x7c, +0x3d,0x2a,0x65,0x4c,0x9e,0x00,0x23,0x2c,0x14,0x66,0x0b,0x92, +0x88,0xf4,0x56,0x01,0x74,0xdc,0x38,0x2c,0xb3,0x5c,0x9a,0xbd, +0xe3,0xdd,0x9d,0xc3,0xbd,0xec,0x7e,0x94,0x46,0x67,0x68,0x70, +0x0c,0x7e,0x2e,0x41,0x64,0x82,0x1f,0x90,0xf8,0x67,0x69,0xf4, +0x3f,0x22,0x73,0x97,0x46,0xff,0x2b,0x42,0x83,0xe8,0x29,0xfc, +0x33,0xcf,0xa2,0x67,0x29,0xfc,0xa3,0xd3,0x39,0xf4,0x92,0x39, +0xda,0xb5,0x0f,0x44,0xd6,0xa7,0x17,0xa5,0x55,0x06,0xa2,0x50, +0xba,0xdf,0xdd,0x7b,0xbf,0x7b,0xf0,0x0e,0x15,0xe2,0x60,0x99, +0x41,0x05,0xa8,0x34,0x42,0x61,0x35,0x32,0xbb,0xc7,0xc7,0x69, +0x74,0x81,0xaf,0x2f,0x7f,0x3a,0x82,0x48,0x32,0xb1,0x70,0xb4, +0x77,0x7c,0xf0,0xd3,0x11,0x14,0xa1,0x28,0xf4,0x33,0x3b,0x9f, +0x5c,0xe4,0x28,0xaa,0x9a,0x3f,0x40,0xee,0x0f,0xd3,0x68,0xd9, +0x2e,0xf8,0x73,0xad,0xdc,0xde,0x15,0x1c,0x9d,0xea,0x3c,0x6c, +0x21,0x1e,0xb2,0x68,0x87,0x64,0x55,0x19,0x68,0x01,0x55,0xde, +0x8f,0xb2,0xa8,0x19,0x19,0x6d,0x85,0x3e,0xbb,0xff,0x02,0xff, +0xa5,0xca,0x7d,0x4b,0x64,0xfd,0x4c,0x44,0xe6,0xfb,0xc4,0xac, +0x4c,0xdb,0xc6,0xed,0xed,0x16,0x64,0x58,0x91,0xde,0x4a,0xcc, +0x3e,0xfc,0x97,0x86,0x9e,0x53,0xa2,0xfd,0xf1,0x08,0xe4,0x47, +0x4a,0xab,0x4e,0x22,0xa0,0x92,0x58,0x48,0xd9,0x26,0xa0,0x55, +0x29,0x2d,0x4c,0xe9,0xf7,0x83,0x24,0xf6,0x72,0xb0,0x0d,0xf1, +0x15,0xd1,0x90,0xfd,0xf5,0xeb,0x20,0x9e,0xfc,0x64,0x50,0xfe, +0xca,0x78,0x28,0x70,0x7d,0x5d,0x59,0xc0,0x6c,0xa0,0x17,0x99, +0xca,0xb4,0x16,0xa7,0xdd,0xdc,0x84,0x38,0x0f,0x46,0xa8,0x81, +0x4c,0x75,0xad,0x48,0x81,0xda,0xa6,0xd3,0x20,0xe9,0x38,0xc7, +0x4b,0x2b,0x2e,0xb4,0x22,0x85,0x0a,0x4d,0x0b,0x00,0x87,0xc3, +0xc1,0xd4,0x66,0x78,0x9e,0x98,0x3d,0xf8,0xaf,0xd8,0xef,0x2f, +0x7b,0x5f,0x22,0x4a,0xa9,0x4a,0xe0,0xf7,0xc6,0xa9,0xf7,0xca, +0x62,0xfe,0x9a,0x86,0x3e,0x66,0x96,0xa6,0xec,0x75,0x01,0x0f, +0x82,0x3b,0xe9,0xe9,0xcf,0x5f,0xf6,0xfb,0xaf,0xaf,0x6f,0xa6, +0xbd,0xbf,0xee,0x3d,0x83,0xfd,0xed,0x02,0xe2,0x9e,0x51,0xfc, +0xb3,0xee,0xe2,0xd9,0xb3,0xe4,0xbb,0x67,0x14,0xb5,0xd7,0x58, +0x7c,0x69,0x2c,0xf6,0x1b,0x8b,0x7e,0x63,0xf1,0xba,0xb1,0xb8, +0x6e,0x2c,0x6e,0x1a,0x8b,0x69,0x63,0xd1,0x5b,0xfc,0x35,0x49, +0x70,0x2b,0xbb,0x69,0x02,0x2f,0x13,0x78,0x16,0xdd,0x6f,0xac, +0x6f,0xb6,0xbd,0x33,0x97,0x95,0xa4,0x5f,0x70,0xc6,0xeb,0x1c, +0x2c,0xbb,0xf8,0xa4,0xe8,0x47,0xd7,0x65,0x99,0xfb,0x07,0xa4, +0x81,0x35,0xfd,0x60,0xae,0x92,0x10,0xca,0x55,0x21,0x43,0xd8, +0x19,0x5d,0x5c,0x8f,0x27,0xce,0xce,0x80,0x83,0xa5,0x8c,0x06, +0xec,0x45,0x46,0x8e,0x79,0xca,0xa7,0x58,0x6c,0x30,0xa0,0xa4, +0xfe,0x4f,0xcf,0xbd,0xd1,0x31,0x36,0xdb,0x4c,0x56,0xe6,0x93, +0xc9,0x94,0x83,0x75,0x00,0x8e,0x5b,0xc8,0x50,0x35,0x20,0x2e, +0xf8,0xe2,0x86,0x5d,0x2a,0x86,0xe2,0xe8,0xcd,0x1e,0xec,0x58, +0x3e,0x47,0x49,0xa2,0xae,0x80,0x03,0xdb,0x05,0xfe,0x74,0xb2, +0xca,0xbe,0x84,0x28,0xed,0x68,0xad,0x7a,0x41,0x02,0x96,0x1a, +0x56,0xd9,0x09,0x2f,0xc6,0xd8,0x36,0x28,0x6c,0xff,0x46,0x57, +0x4e,0xa3,0x7c,0x67,0x38,0xe8,0x79,0x93,0x0c,0x53,0xfd,0x18, +0x46,0x2b,0xfe,0x6b,0x7b,0xf2,0xe3,0x5b,0x3a,0xe2,0xc3,0xfa, +0xe8,0xd4,0x8d,0xd5,0xc8,0x44,0x21,0x18,0x04,0x17,0x9b,0xc3, +0x2a,0x45,0x4e,0x6e,0x7a,0xc3,0xc1,0xdf,0xf3,0xbe,0xb7,0x30, +0xfe,0xce,0xc6,0xc5,0xfc,0xf0,0xcf,0x81,0x6a,0xaf,0xc6,0xec, +0xd4,0x03,0xea,0x96,0xdf,0xa9,0xde,0xdf,0x4e,0x06,0xe3,0x09, +0xec,0x6f,0xd2,0xd6,0xd6,0x56,0x79,0x68,0xcb,0x4e,0x17,0xaa, +0x6c,0x44,0x54,0xbc,0xff,0xf7,0x75,0x96,0x5e,0xf7,0xf8,0x91, +0xb7,0xe8,0x9b,0xb5,0x35,0x67,0x50,0x85,0xfe,0xf3,0xda,0x3f, +0xa7,0x11,0xec,0x54,0x81,0x38,0xe0,0x2f,0x94,0x80,0x0f,0x1a, +0x27,0x05,0xd4,0xf7,0xe6,0xbf,0xa2,0xd3,0x7e,0xf7,0xbb,0x27, +0x98,0xd0,0x08,0xed,0x1d,0x78,0xc8,0xa5,0x9e,0xa1,0x1f,0x3b, +0x63,0xc2,0x1e,0xe2,0x1c,0x86,0xa7,0x94,0xbb,0x9d,0x07,0x82, +0xf1,0x59,0xa8,0xe3,0x6d,0xa8,0x6b,0x2d,0x0b,0x58,0x7f,0x94, +0xde,0x02,0x7a,0x76,0xaf,0xcc,0x82,0xf3,0x5e,0x20,0xb4,0xcc, +0x2d,0x71,0xca,0x34,0x90,0x8f,0x21,0xf3,0xbe,0x3e,0x68,0xad, +0xe7,0xf2,0xe9,0x70,0x3b,0xb4,0x6a,0xaf,0x59,0x8d,0xa0,0x0c, +0x03,0xb8,0x4e,0x86,0xd3,0x23,0xf7,0x8c,0x8a,0xd9,0x0f,0xbe, +0x0d,0x73,0xdd,0xff,0x8a,0xef,0x79,0x7c,0xc7,0x0f,0xa6,0xef, +0xae,0xb4,0xaf,0xb2,0x47,0x2e,0x7d,0x4a,0x00,0x41,0xc0,0xa2, +0x57,0xb7,0x64,0x10,0xde,0x14,0xb8,0x99,0x7b,0xc2,0x15,0x62, +0xbf,0x7a,0x32,0xe0,0xb5,0x79,0x71,0xc4,0x11,0xf2,0x9e,0x1a, +0x75,0x05,0x97,0xdf,0xfa,0xd0,0x98,0xb2,0x17,0x3f,0xd1,0xb4, +0x08,0x58,0x4c,0xd9,0xb6,0xad,0xe5,0x52,0x15,0xfc,0xa8,0x53, +0xcd,0xa4,0x52,0xab,0x6e,0x88,0x48,0x5a,0x53,0x3b,0xcb,0x76, +0x85,0x76,0xa0,0xc2,0x16,0xdf,0x69,0x47,0xd3,0xf9,0xf9,0xcd, +0x00,0xfa,0xb2,0x1a,0xa7,0x44,0xb7,0x8e,0x38,0xea,0x7a,0x1f, +0xbb,0x70,0xfc,0x25,0xb0,0xda,0xe1,0x6e,0x0f,0x82,0x0e,0xab, +0x78,0xcc,0xfb,0xb5,0xd5,0x2f,0xe5,0x2a,0x82,0x5f,0x9d,0x78, +0x7b,0xd3,0x44,0xcc,0x1a,0xb5,0x07,0x9c,0x00,0x98,0x1e,0x4e, +0xf0,0xac,0xb8,0x00,0xb1,0xf9,0xfe,0x36,0x25,0xa2,0x2b,0x6d, +0x92,0x1c,0x78,0x44,0x29,0x4e,0x0d,0x23,0x47,0x68,0x5b,0xde, +0x58,0x3f,0x5b,0xa9,0x7f,0xb4,0x6f,0xc2,0x42,0x05,0x4b,0xf8, +0xb1,0xa2,0x15,0xb3,0x1a,0x15,0x77,0x84,0x62,0x31,0xc9,0x3f, +0xcf,0x70,0x95,0x54,0x28,0xea,0x69,0x2e,0xf3,0xdf,0x3f,0x01, +0xe2,0x29,0xd4,0x71,0xa5,0x6e,0xbf,0xc4,0xc1,0x24,0x49,0x8c, +0x5e,0xe7,0x77,0x76,0xa3,0x65,0x92,0x06,0x19,0x96,0x5d,0xb6, +0xea,0xed,0xa7,0x55,0xd5,0xac,0x8d,0x71,0x99,0x7c,0xf5,0x78, +0x36,0x94,0xcc,0xbc,0x89,0x38,0xbc,0x0d,0xb8,0x9c,0xdd,0x2e, +0xf0,0x6c,0x9f,0xfe,0x4c,0x57,0x5d,0x3f,0x1c,0x37,0x92,0x38, +0x95,0x2b,0x07,0xbc,0x71,0x58,0xa0,0x55,0xde,0xb3,0x4f,0xff, +0xb6,0x96,0x36,0x3b,0x8d,0xac,0xfe,0xab,0xff,0x58,0x3b,0xdb, +0x38,0xdb,0xec,0xf2,0x0d,0x82,0x36,0x92,0x06,0x55,0xe0,0x23, +0xeb,0xde,0xc6,0xdf,0xa1,0x70,0xf3,0xfd,0xaf,0x1a,0x1b,0xdd, +0xc6,0x7f,0xa8,0x18,0x08,0x9e,0x35,0x25,0xdc,0xbd,0xdf,0x36, +0xbf,0x59,0xae,0x3b,0xd1,0xcd,0x41,0x40,0x35,0x82,0xb3,0x8d, +0xc5,0x19,0xd5,0x0d,0x12,0x1f,0xfc,0x81,0x88,0x26,0xfc,0x05, +0x69,0x84,0x95,0x0a,0x06,0xd6,0x06,0x57,0x46,0x66,0xe1,0xd2, +0xc0,0x36,0x9c,0x6c,0x79,0xd2,0x82,0x05,0x50,0xba,0x04,0xd0, +0x96,0xfd,0x0c,0xd9,0x4e,0x4b,0x43,0x8b,0x74,0x86,0xec,0xa6, +0xa5,0xa1,0xf5,0x34,0x63,0x6d,0x7f,0xa5,0x25,0x23,0x60,0xe6, +0x7a,0xd0,0xef,0xe7,0xac,0xc2,0x23,0x8f,0x70,0x45,0xc7,0x87, +0x26,0x1c,0x7f,0xc3,0x98,0xe7,0x33,0xd1,0xf2,0x21,0xd4,0xfd, +0xb8,0x01,0x9b,0x95,0xfb,0x26,0x62,0xd7,0x6c,0xbc,0x4e,0x73, +0x5c,0x49,0x75,0x86,0xed,0x2a,0xac,0x4c,0xed,0x45,0xf6,0x35, +0x72,0x1a,0x75,0x46,0x57,0xfb,0xe3,0x3e,0x6a,0x10,0x3c,0xb6, +0x68,0x8a,0xad,0xb2,0x0b,0x9a,0x61,0xb1,0xeb,0x51,0xf5,0x42, +0x99,0xa6,0x2f,0x69,0x41,0x77,0x17,0x0b,0x97,0x81,0x5f,0x39, +0x7d,0x85,0x01,0x3e,0x98,0x21,0x5d,0xa3,0xdc,0x8f,0x90,0x45, +0x85,0x3b,0x5c,0xf6,0x22,0x13,0xf8,0x25,0xa1,0x04,0xb1,0xb6, +0x1f,0x99,0xd0,0x2a,0x3d,0xa5,0x59,0x1b,0xf4,0xb0,0x41,0xf5, +0x86,0xec,0x29,0x85,0x8c,0xd5,0x47,0xe6,0x1d,0x37,0xde,0xaf, +0x30,0xd8,0xbf,0xbc,0x06,0x42,0xef,0xe6,0x9f,0x2f,0xf2,0x5b, +0xbe,0x7a,0x22,0xd5,0xb0,0x60,0xa5,0x54,0xab,0x27,0xcb,0xec, +0x6a,0x0c,0xb8,0xad,0xa5,0xf2,0x86,0x4a,0x1b,0x5b,0xd4,0x58, +0x27,0x5a,0xde,0x18,0xc0,0xbf,0xce,0xb5,0xcc,0x7a,0xd5,0x6d, +0xf4,0xbf,0xcc,0xb7,0x0c,0x9f,0xde,0x3b,0xf3,0x6f,0x19,0xdb, +0xf2,0x6c,0xbe,0xeb,0xbd,0x13,0x35,0x93,0x1b,0x1c,0xb6,0x15, +0x69,0x62,0xe3,0x0d,0x0d,0xc4,0x70,0x84,0xb7,0x1a,0xe7,0xe3, +0x10,0xf6,0x2e,0xd9,0x76,0xb7,0x6b,0x9a,0x4a,0xd4,0xbe,0x0b, +0x38,0xc6,0x39,0x37,0x90,0xd2,0xd6,0xef,0x06,0x07,0x95,0xd3, +0x18,0x8e,0xa0,0xf7,0xd9,0xeb,0x6e,0x65,0x63,0x79,0x8f,0x69, +0xed,0xf7,0xde,0xe9,0xd3,0xba,0x35,0xc5,0x8c,0xf1,0xc8,0xa2, +0xe9,0xe3,0x18,0x32,0xf8,0xbc,0x4d,0xe8,0xc6,0xc1,0xd5,0x88, +0x1e,0xbb,0xf8,0x0c,0x25,0x9b,0xe0,0x3c,0x87,0xe9,0x66,0x7a, +0xc4,0x25,0x58,0x25,0x08,0x26,0x3d,0x9a,0x43,0x61,0x75,0x1f, +0x58,0x70,0x40,0x1a,0x1f,0x6d,0x70,0x06,0x54,0x21,0xd0,0xd6, +0xc0,0x5b,0xcb,0xc8,0x04,0x18,0x05,0x86,0xc0,0x2d,0x7d,0x24, +0xf6,0x22,0x49,0xcc,0xf8,0x21,0x93,0x6a,0x87,0x32,0x40,0xb6, +0xee,0x55,0xb6,0xf5,0x3b,0xfa,0x88,0x06,0xc2,0xcf,0xb5,0xc7, +0xdc,0x00,0x89,0xeb,0x1f,0xae,0xce,0xba,0x01,0x6a,0xaf,0x97, +0x94,0x3d,0x9e,0xe0,0x79,0x87,0x61,0x3c,0xe2,0xfe,0xc5,0xcd, +0x09,0xeb,0xa4,0xe4,0x9e,0x6b,0x3d,0x2d,0x67,0xe9,0xae,0x65, +0xd9,0x9a,0x64,0x43,0x66,0x28,0x54,0x1e,0xaf,0x2e,0x50,0xaf, +0x7f,0x95,0x9f,0x17,0x8f,0xf4,0x4a,0x52,0xc3,0x17,0xdf,0x25, +0x3f,0x27,0x49,0x31,0xdb,0x56,0x08,0xa6,0x55,0xf0,0x94,0x92, +0x98,0xd5,0x28,0xbb,0x16,0x16,0x7b,0xf4,0x01,0xbe,0xf4,0x04, +0xb7,0x2a,0xc5,0x6e,0x26,0x04,0x13,0xa7,0x40,0xb6,0xc2,0x3b, +0x49,0xe5,0x34,0xd4,0xf3,0xb4,0x92,0x81,0x3d,0xc9,0x7f,0x89, +0xaa,0xd9,0xd9,0xb0,0x2c,0x59,0x7c,0x2d,0x72,0x24,0xd6,0x36, +0x0d,0xd1,0x40,0x57,0xb8,0x1a,0xcd,0xad,0x12,0x9a,0xad,0x6a, +0x34,0x9f,0xe6,0xf6,0xa5,0xda,0xab,0x8b,0x7a,0x48,0xa7,0xb9, +0x5f,0xa8,0x0c,0xec,0xb4,0x6e,0xed,0x9e,0xbe,0xc4,0x46,0xbd, +0x63,0x91,0xb8,0xc4,0x61,0xb9,0xad,0x9e,0xfb,0xd8,0x05,0x4e, +0x36,0xc0,0x61,0xfd,0x15,0x8c,0xd5,0xe3,0xe2,0xbc,0x68,0x90, +0x56,0x90,0x77,0xfe,0x61,0x35,0x6f,0x70,0xca,0x15,0x16,0x4d, +0x88,0x5c,0x92,0x3b,0x10,0x7a,0xd0,0x83,0xdb,0x0b,0xac,0xa7, +0xbd,0x1e,0x1c,0x70,0x54,0x1a,0xa4,0xf1,0x4c,0x54,0x10,0x26, +0x13,0x10,0x6c,0x07,0xb4,0xdc,0x70,0xbf,0x45,0x95,0xa5,0xa2, +0x68,0x71,0xd4,0x0c,0xfa,0x9f,0x33,0x65,0x80,0x54,0xd4,0x9c, +0x51,0x0d,0xba,0x08,0x93,0x21,0xb6,0xa1,0xc0,0xc6,0x46,0x5b, +0x94,0xa0,0x7d,0xc9,0x53,0x88,0xef,0x5a,0x1b,0xee,0x2b,0xfd, +0xbb,0x14,0x3d,0x7f,0xc8,0x28,0x68,0x1b,0xaa,0x68,0x1e,0x08, +0x76,0x14,0x5d,0x3b,0x34,0x4a,0xba,0x2f,0xed,0x8f,0xad,0xc0, +0x77,0xea,0x16,0x0b,0x13,0xfd,0x4c,0x7e,0xae,0xf1,0x5a,0xc0, +0x6f,0xb8,0x4b,0x12,0xd1,0xd3,0xe4,0xc2,0xa9,0x33,0xd8,0x08, +0x65,0x71,0x90,0x28,0x12,0x77,0x92,0x97,0x76,0x57,0xc8,0x31, +0xad,0x6e,0xc8,0xf4,0xdb,0x97,0x6e,0x03,0xa6,0x7c,0xac,0x39, +0x38,0xa1,0x55,0xda,0xca,0x7d,0xdc,0xe5,0x8a,0x1d,0x9c,0x87, +0x41,0xbb,0x2e,0xb4,0x7c,0x49,0x64,0x59,0x79,0x96,0x2a,0xd2, +0xf0,0xb7,0x08,0xc3,0x95,0x24,0x5f,0x30,0x30,0xeb,0xce,0xe3, +0x9c,0xb1,0xc9,0xa6,0x45,0x87,0xb0,0x4b,0x9c,0x79,0xa0,0xa7, +0x8c,0xe1,0xa3,0x42,0xfb,0x4d,0x60,0xb0,0x99,0x69,0x5e,0xce, +0xbc,0x6c,0x3d,0xc0,0x9d,0xda,0x32,0x55,0x1e,0xb6,0x71,0x1d, +0x94,0xa2,0xb5,0x2e,0xb4,0x79,0x2d,0xda,0xf4,0xb8,0x20,0x55, +0x9a,0xb9,0xb6,0x45,0xd9,0x86,0xb5,0x35,0xbd,0xf1,0x48,0xce, +0x92,0x29,0xf9,0xa2,0xe1,0xdf,0xbb,0x55,0x46,0xae,0xe7,0xa3, +0xe9,0xf5,0xe0,0x72,0xa6,0x33,0x14,0x6c,0x9b,0xfa,0x6a,0x02, +0xae,0x21,0xd9,0x4b,0x0e,0x77,0x78,0xcb,0x0e,0xd4,0x83,0x63, +0xfb,0xa4,0xd1,0xf9,0x5a,0x22,0xf2,0xca,0x62,0x9b,0xd6,0xa7, +0x8d,0x58,0x46,0x14,0x42,0xc1,0x9a,0x51,0xcf,0x49,0x7c,0x6b, +0xb2,0x8b,0xf6,0x7a,0xbd,0xc2,0xbb,0x4d,0xe2,0xce,0x50,0xb0, +0xcc,0x62,0x01,0x93,0xdc,0x50,0xcf,0xa8,0xd1,0x75,0x2d,0x73, +0xce,0xab,0xd4,0xf3,0x48,0x97,0x4a,0x87,0x28,0xc4,0xd6,0x5d, +0x94,0x3c,0xd2,0x76,0x78,0x94,0x8d,0x44,0x09,0xeb,0x42,0x98, +0x3c,0x4c,0xda,0x95,0xd5,0x32,0x21,0x27,0x4a,0x95,0xce,0x01, +0x78,0x23,0xf4,0xb5,0xce,0x22,0x58,0xd1,0xf6,0x4e,0x1b,0x59, +0xf4,0x46,0xc4,0x94,0x7a,0x0b,0x4d,0xff,0xdd,0x83,0x77,0xc7, +0x27,0x3b,0xef,0x4e,0xde,0xc3,0xe6,0xe3,0xa7,0x3d,0x75,0xc4, +0x81,0x3a,0x3c,0x0b,0x52,0xd6,0x59,0xa0,0x51,0xca,0xf5,0x4d, +0x18,0x6b,0x6a,0xef,0x43,0x83,0xfd,0xf0,0x11,0xfd,0xec,0x76, +0x68,0xe0,0xdf,0x8e,0xbe,0x7c,0xa9,0xac,0x9f,0x8d,0xc4,0x4b, +0xd6,0xa6,0xd4,0xab,0xbc,0x0a,0x87,0x13,0x5a,0x9f,0xd3,0x5b, +0xe3,0xe3,0x15,0x7c,0xe5,0x4f,0xd2,0xdf,0xab,0xf8,0x42,0xc5, +0xe5,0x80,0x2d,0xf5,0xc0,0xc5,0x80,0xad,0xf0,0x2e,0x9c,0x15, +0x3f,0x0e,0x46,0x8a,0x67,0x69,0xe7,0xae,0x0f,0x9e,0xbf,0x97, +0x84,0x78,0xdc,0x94,0xe3,0xf3,0x19,0xb4,0xb8,0x98,0x34,0xfb, +0xbc,0x5f,0xb0,0x11,0xd6,0x17,0x10,0x56,0x96,0x98,0x2a,0xec, +0x31,0xa5,0x84,0xbc,0xad,0x04,0x0f,0x20,0xe2,0xd2,0xab,0x9f, +0x28,0x4a,0x5d,0x5b,0x12,0x69,0xc9,0x49,0x7e,0x73,0x8b,0xfa, +0xb7,0xc1,0x79,0x8b,0xd6,0xcb,0x0d,0x2c,0x16,0xbb,0xe8,0xa7, +0x5d,0xc9,0xf0,0x2b,0x4b,0x57,0xe8,0xd5,0x28,0xd3,0x30,0x02, +0x8b,0x52,0x72,0xdd,0xa0,0x1a,0x67,0x31,0x4b,0xbc,0xed,0x94, +0x27,0xf6,0x5d,0x50,0x65,0x89,0x1b,0x86,0xd0,0xa3,0x27,0xf4, +0x21,0x8d,0x7d,0x57,0x3a,0x0c,0xd5,0x6e,0x83,0xce,0x42,0xf5, +0x86,0xe0,0x24,0xe4,0x89,0x9d,0xf3,0xed,0x04,0x81,0x28,0xac, +0x26,0x0a,0x4c,0x5d,0xd9,0x28,0xbc,0x60,0x25,0x1c,0xc9,0x46, +0xe7,0x64,0x0e,0x8b,0x37,0x15,0x50,0x8e,0x04,0x5d,0x6b,0x77, +0x03,0x7b,0xef,0x59,0xc1,0xfc,0x7b,0xc4,0x8b,0x96,0x64,0x3b, +0xe8,0xf7,0x57,0xe7,0x84,0xc4,0xc8,0xf8,0xbc,0x78,0x10,0xbf, +0x3a,0x33,0xa6,0xd2,0x3d,0x3c,0xe6,0x1e,0xf7,0x3e,0x56,0x4f, +0xb5,0xfb,0x47,0xee,0x7f,0xd5,0x2c,0x16,0x38,0x91,0xf2,0x26, +0x51,0xf9,0x10,0x01,0xbb,0xff,0x82,0x72,0x26,0x7c,0xcb,0xea, +0x25,0x42,0x35,0xde,0x65,0xc6,0xc8,0x46,0xfc,0xbc,0x6f,0x56, +0x94,0x29,0xa3,0xff,0x88,0xa4,0x0f,0xa7,0xb7,0x21,0xb1,0x94, +0x8f,0x2f,0x8b,0xa7,0x96,0x9e,0xd1,0xe6,0xcf,0xab,0xae,0x7f, +0x9c,0x37,0xbe,0x8b,0xe9,0x2d,0x79,0xdc,0xa4,0x8a,0xe8,0x76, +0xa3,0xfa,0x26,0x98,0xaf,0x9e,0x6b,0xfd,0xf3,0x21,0x7f,0x90, +0x7d,0x09,0xb4,0x9d,0xc1,0x5f,0xf3,0xdb,0x9a,0xb3,0x38,0x51, +0xb3,0x16,0x29,0xf8,0x03,0x3b,0xa7,0xe6,0x4d,0x53,0xd4,0xbc, +0xed,0x8a,0x9a,0xb8,0x27,0xc4,0x5f,0x00,0x00,0x7f,0xe9,0xe0, +0xa5,0xc6,0x67,0xba,0xb0,0xa9,0xb8,0x98,0x4f,0x6b,0xe7,0xc3, +0xf9,0x24,0xaa,0x34,0x74,0x32,0x72,0x77,0xa5,0xfe,0x82,0x14, +0x0f,0x90,0x56,0x5e,0x97,0x8e,0xe4,0xaa,0xb4,0xd0,0xcc,0xd3, +0xa0,0x78,0xd7,0xab,0x13,0xe8,0xf7,0x6b,0xd5,0x7e,0xfb,0x57, +0xf1,0xab,0xcb,0x91,0x3d,0xac,0xa2,0xab,0xcc,0xb0,0x82,0x44, +0xbb,0x68,0x2c,0x58,0xdb,0x2b,0x5d,0xed,0xaf,0xf2,0x1f,0x71, +0x69,0x2b,0x87,0xad,0x1f,0xe6,0xe4,0x1b,0xb3,0x25,0x71,0x65, +0x7c,0xdb,0x22,0x97,0xa5,0x57,0x6f,0x2e,0x03,0xd2,0xb1,0x0f, +0xf7,0x54,0xd3,0x24,0xca,0x91,0xce,0x6c,0xd2,0x1b,0x4d,0xc9, +0xb7,0x6c,0x1a,0xb9,0xd3,0xda,0x80,0x9e,0x60,0x14,0x6f,0x06, +0xa3,0xde,0x10,0xe9,0x55,0x39,0x61,0x78,0x44,0x9f,0xc2,0x78, +0xc0,0xe5,0x8e,0x74,0x67,0xbf,0xf6,0x8c,0xd7,0xf7,0x25,0xd9, +0x2c,0xb0,0x17,0x92,0x14,0x38,0xc6,0xbc,0x85,0xad,0xac,0x65, +0xfa,0x6f,0x2e,0x4b,0x6c,0x4b,0x03,0x00,0x49,0xdc,0x36,0xb8, +0x49,0x34,0x18,0xeb,0x54,0xb1,0x45,0x21,0x21,0x65,0xc2,0x4a, +0xd5,0x8b,0x5b,0x4c,0x17,0x68,0xda,0xad,0x55,0xac,0xb3,0xe8, +0x82,0xde,0x73,0x29,0x63,0x13,0x14,0xcf,0x6c,0x1b,0x40,0x26, +0x45,0x4d,0x6e,0xd7,0x41,0x2a,0x8f,0x32,0xec,0x30,0xe4,0x17, +0x55,0x1a,0x45,0x8a,0x73,0xcf,0x34,0xe5,0xbd,0x99,0x44,0xe6, +0x85,0xbb,0x44,0x17,0x93,0x2c,0xed,0x95,0x3a,0x4e,0xfa,0x37, +0xec,0x48,0x38,0x20,0x13,0xbc,0xb6,0xc2,0x23,0xfa,0x99,0x2c, +0x70,0xa4,0x9a,0x47,0xe7,0xf8,0xa4,0x57,0x73,0xfc,0xff,0x50, +0xf7,0xfe,0xff,0x6d,0xdb,0xc8,0xa2,0xe8,0xcf,0xef,0xfc,0x15, +0x36,0x4f,0xd6,0x21,0x23,0x58,0xb6,0x9c,0xb4,0xdd,0x52,0xa1, +0x75,0xd3,0x38,0x69,0x73,0x6e,0x9c,0xe4,0xc6,0xce,0x66,0xb7, +0xb2,0xea,0x8f,0x6c,0xc9,0x36,0x1b,0x59,0x72,0x45,0xa9,0x49, +0x6a,0xe9,0xfd,0xed,0x0f,0xf3,0x05,0xc0,0x00,0x24,0x65,0x25, +0xed,0x9e,0xfb,0xce,0x6e,0x63,0x11,0xc0,0x60,0x30,0xf8,0x3e, +0x18,0x0c,0x66,0xce,0x61,0x89,0x82,0xb0,0x79,0x28,0xa5,0xc4, +0xd0,0x2a,0xed,0x66,0x80,0x4d,0xf9,0xb8,0x94,0x87,0xc9,0xbe, +0xc3,0xb2,0x35,0x51,0x72,0x0b,0x94,0x77,0x30,0x4f,0xf5,0x30, +0x93,0xe3,0xaf,0x72,0xb4,0xae,0x37,0x0e,0x0b,0x7b,0x81,0x5d, +0x4c,0xcf,0xf5,0x71,0xc0,0x78,0x89,0xe1,0x06,0xe1,0x33,0x81, +0x4e,0x53,0x93,0x31,0x18,0xf5,0xb5,0x20,0x14,0xc4,0x27,0x21, +0x60,0xcb,0x93,0xea,0x60,0x53,0x21,0xaa,0xc0,0xa8,0x76,0xf5, +0x72,0x71,0xcf,0x37,0xb6,0xc2,0x06,0x43,0x31,0x89,0x03,0x6e, +0xb4,0xc3,0x51,0x17,0x65,0x8e,0xc3,0x29,0x18,0x94,0x83,0x81, +0x34,0xbf,0x79,0xd9,0x2f,0x66,0x4c,0x24,0x1b,0x01,0x94,0xac, +0xb6,0x44,0x08,0xa3,0x4d,0x04,0xbd,0xe1,0x2a,0xe2,0xd1,0x79, +0x6c,0x12,0x94,0x5e,0x31,0x55,0xbc,0xf4,0x10,0x9e,0x90,0xfc, +0xc7,0xd2,0xf7,0xa1,0x81,0x6c,0x02,0x2e,0x85,0x4f,0x8a,0xb7, +0x4e,0x03,0x33,0xa6,0x26,0x97,0x5a,0x4f,0xd3,0x73,0xb6,0x3e, +0x74,0x95,0x17,0x74,0xc6,0x7f,0x31,0xc8,0x1a,0x0d,0xaf,0x0d, +0xda,0x1a,0xaa,0x43,0x03,0x0a,0x78,0x0f,0xc8,0xa4,0x6e,0xd1, +0xb0,0x78,0xea,0x8f,0xaf,0x65,0xd2,0x2c,0xe6,0xe7,0xe7,0x43, +0xf0,0x6a,0x62,0x4a,0xd0,0xa3,0xe8,0x66,0x32,0x2e,0x48,0x5b, +0xc9,0x2b,0x25,0xcb,0xbc,0x52,0x58,0x01,0x67,0xf8,0x91,0x1a, +0x47,0xce,0xd1,0xb6,0x1c,0x3c,0xb1,0x01,0x29,0xcf,0xd1,0xba, +0x8e,0x2a,0x35,0xbc,0xc1,0x10,0xb4,0x25,0x4d,0x61,0x3f,0x92, +0xf8,0x2e,0x5b,0x8b,0xd2,0xac,0xf7,0xc7,0x0e,0xf4,0x86,0x1b, +0x6a,0x76,0x86,0x05,0x60,0x4d,0x63,0x72,0x27,0x4e,0x12,0x6f, +0xa8,0x78,0x76,0x71,0xbd,0x51,0x0e,0x87,0x65,0x19,0xde,0xda, +0xda,0xf4,0x4e,0x53,0x01,0xac,0x37,0xd1,0x83,0xea,0xeb,0x2c, +0xd7,0xb0,0xad,0xdf,0xcb,0xbd,0x46,0x62,0x2b,0xdb,0x89,0x77, +0x4a,0xb3,0x73,0x90,0x24,0x3f,0x4d,0x94,0xbf,0xc7,0x81,0xa8, +0xbb,0xae,0x47,0xc1,0x43,0x4c,0x6d,0x8f,0x2c,0x5d,0x41,0x55, +0xd4,0xf0,0x3b,0x44,0x20,0x28,0x49,0x57,0xa2,0x71,0x2b,0x6a, +0x3e,0x5b,0x97,0xe3,0x5c,0xa5,0x89,0x23,0x57,0xad,0xa2,0x2c, +0x0b,0x2b,0x9a,0x54,0x14,0x29,0x47,0xe9,0x72,0x5f,0x4d,0xc6, +0xc0,0xbb,0xc3,0x0d,0x55,0x4d,0xf1,0x72,0xe5,0x94,0x5b,0x3a, +0x66,0x7f,0xa3,0x39,0x2d,0x64,0x97,0xbc,0xad,0x80,0xb5,0x38, +0x55,0xed,0x81,0x8e,0x00,0x94,0x7f,0xb0,0x43,0x2b,0xdb,0x6f, +0x9f,0x3c,0x7d,0x96,0xed,0xdc,0x2e,0x77,0x8c,0x65,0x12,0x4f, +0x2b,0x63,0x1d,0x97,0x4d,0xa4,0xf0,0x05,0x17,0x95,0x76,0x79, +0x3d,0x87,0x1e,0x55,0x1f,0xaf,0x86,0x63,0x1b,0x45,0x7b,0x3f, +0x44,0x05,0xfe,0xd4,0x82,0xd4,0x04,0x6c,0x7f,0x69,0x76,0x9e, +0x57,0x71,0xfc,0x5e,0x2c,0x76,0x11,0x59,0xe1,0x79,0x19,0x66, +0xf4,0x68,0x7c,0x84,0x92,0x75,0x08,0x4c,0x96,0xe8,0x20,0x5e, +0xef,0x1d,0x7d,0xbe,0x3e,0x9b,0x8c,0xbc,0x93,0x68,0x53,0x24, +0xe8,0x71,0x3e,0x1c,0x0f,0xaa,0x80,0x6c,0x74,0x0c,0x0f,0x8f, +0xde,0x6b,0xd4,0xd9,0xce,0x2f,0x50,0x42,0x0c,0x3a,0xc7,0x45, +0xd2,0x89,0x9b,0x20,0x4c,0xb1,0x7c,0x37,0xee,0x58,0x6e,0x1f, +0xb3,0x57,0x93,0xca,0x5a,0x50,0x66,0x05,0x4c,0xc4,0x45,0x82, +0x11,0x3f,0x49,0xaf,0xe3,0x58,0x83,0x40,0xe3,0xc0,0x02,0x58, +0x77,0x1a,0x11,0x80,0x45,0x9e,0x87,0x0d,0x54,0x84,0x86,0x7b, +0xfb,0x6d,0x3d,0xf8,0x7b,0x59,0x4d,0xe3,0x76,0x3d,0x7c,0xe4, +0xca,0xc8,0xd0,0x8f,0x45,0x57,0x56,0x00,0xed,0x6a,0x88,0xb6, +0x45,0x7b,0x19,0xc1,0xd9,0xde,0x42,0x5b,0x8a,0x70,0x50,0xc9, +0x3e,0x68,0xd8,0xf1,0xd1,0x80,0x6b,0x7b,0xea,0xd5,0x86,0x6d, +0xe6,0x44,0x4c,0xef,0x15,0x97,0x1e,0xc2,0x33,0xa8,0x1c,0x07, +0x16,0xb7,0x33,0x83,0x42,0xbe,0x5a,0x58,0x7a,0x06,0xf6,0x85, +0xf1,0x13,0xac,0x90,0x62,0x5d,0x16,0x0b,0x16,0x99,0xf0,0xbc, +0x68,0xde,0xe0,0xac,0x7a,0x6a,0x1c,0xc1,0x6e,0x13,0x81,0x49, +0x22,0x87,0x55,0x17,0x93,0x7a,0xc1,0x14,0xd8,0x04,0x37,0x3c, +0xe2,0xb8,0xc3,0x2e,0xca,0x3c,0xd1,0x82,0x71,0x18,0x96,0xf0, +0xd2,0xf3,0x76,0x78,0x33,0xec,0xfb,0x7a,0x2f,0x7c,0x96,0xa9, +0xe4,0xfd,0x31,0x4d,0xb9,0x23,0x00,0xda,0x42,0xfe,0xf1,0xf4, +0xed,0xb3,0xc3,0xd7,0xff,0x78,0x76,0x90,0x45,0xf7,0xee,0xb9, +0x50,0x64,0xd1,0x97,0xdc,0xea,0x50,0x74,0x64,0xda,0xe8,0x0b, +0x8f,0x10,0xab,0xb9,0x35,0x58,0x24,0xdc,0xc1,0x76,0xad,0x13, +0x83,0x2e,0xfe,0xfc,0xc3,0x0f,0x9f,0x75,0xdb,0x2a,0xf7,0xc9, +0x4a,0xf3,0x1c,0xf1,0x62,0xf0,0x7c,0xec,0xbe,0x51,0x22,0x2a, +0x23,0x5e,0x9f,0xfd,0xaa,0x83,0xa3,0xab,0x42,0x4d,0xf5,0x3f, +0xec,0x9d,0x17,0xf8,0x7a,0xf2,0x22,0xd7,0x38,0xf4,0x40,0x15, +0x21,0x37,0x48,0x8d,0xea,0x02,0xb7,0x07,0xf9,0x4c,0xcf,0xc4, +0x20,0xb6,0x3e,0xd8,0x8b,0x07,0x30,0xcd,0x4f,0x8a,0x46,0x3e, +0xd6,0x7f,0xe2,0xe6,0x83,0x4e,0x82,0xea,0x58,0x45,0x03,0x49, +0xd0,0xbf,0x67,0x9f,0x31,0x01,0x80,0x1e,0x80,0xfe,0x57,0x82, +0x2f,0x3c,0x9f,0x8f,0x5f,0xe2,0x83,0x9b,0xec,0xf6,0x5e,0x3e, +0x48,0xd9,0x02,0x0d,0xba,0x7d,0xdf,0x44,0xdc,0xac,0xe0,0xe0, +0x77,0x54,0x1c,0xe5,0x81,0xaf,0x73,0x47,0x12,0x8c,0x5b,0x10, +0x1c,0xc3,0x43,0xd9,0xfb,0xa7,0xb9,0xe6,0xa1,0x4e,0x21,0xea, +0xf4,0x1c,0x44,0x10,0xd8,0xd0,0xa7,0x5d,0x6a,0xcf,0x8d,0xb3, +0xcf,0x1b,0xa7,0xf9,0xe0,0xb4,0x77,0x1f,0xfd,0xa2,0x4b,0x63, +0x28,0x0e,0x1d,0x5e,0x15,0xea,0x66,0xcb,0xac,0xdf,0xf8,0xa9, +0x0d,0xec,0xf5,0x44,0x6f,0x70,0xdc,0x23,0x19,0xd7,0x89,0xc3, +0xde,0x32,0x87,0x68,0x17,0x9f,0xc8,0x0e,0xcc,0xa4,0x8d,0x1e, +0xea,0x25,0xe5,0xbb,0xbc,0xf0,0x7a,0x4a,0x2f,0x84,0xb2,0x0d, +0xbb,0x5e,0x22,0xda,0x6a,0xf1,0x1b,0xb9,0x1b,0xf4,0xbb,0x31, +0x1c,0x2e,0x61,0xd8,0x35,0x60,0x46,0x86,0x21,0x43,0xfa,0xcd, +0x20,0x95,0x39,0xc0,0xb3,0x56,0x1c,0x8e,0xbc,0x8a,0x9a,0xd8, +0x3a,0x18,0x3b,0x43,0x2c,0x45,0x0c,0x06,0xa9,0x97,0x53,0xd6, +0x7b,0xc9,0x6e,0xfd,0x33,0xdd,0x1d,0x76,0xe0,0xc5,0x9d,0x34, +0xee,0x9e,0xdc,0x3b,0xf9,0x08,0x8f,0x40,0x4e,0x62,0xfb,0xad, +0x07,0x99,0x42,0x03,0x44,0x26,0x9c,0x24,0x30,0xe4,0x56,0x0f, +0xaa,0x7c,0x40,0xc3,0x8a,0xc7,0xcd,0x7d,0x18,0x38,0x35,0x63, +0xe8,0xfe,0x46,0x71,0x35,0x99,0x8f,0x06,0xf8,0xd2,0x5a,0x0f, +0x3a,0xdb,0xa6,0xf0,0xdc,0xfa,0x7e,0x7c,0xaa,0xe9,0x3d,0x55, +0x1b,0xa7,0x58,0xc5,0xd3,0xe4,0xbe,0x18,0xa0,0xaa,0x34,0xda, +0x74,0x7d,0xd8,0x8e,0xb2,0xeb,0x9a,0xcc,0xbd,0xdf,0xb7,0x23, +0xcf,0xeb,0x5e,0x3b,0x06,0xe9,0x31,0x8a,0x66,0xdf,0x7e,0xd0, +0xeb,0xf3,0x87,0x43,0x30,0xa6,0xba,0xf4,0x45,0x09,0x4f,0x2d, +0xd9,0x31,0x4c,0x7e,0xc7,0xda,0xdb,0x78,0x69,0x0b,0x94,0x2d, +0x41,0x83,0x71,0x1b,0xb2,0x85,0x03,0x1d,0x4a,0xfe,0xd2,0x85, +0x44,0x40,0xb9,0x01,0x6a,0xbb,0x4f,0x39,0x84,0xba,0x7b,0x0b, +0x75,0x06,0x04,0xa1,0xe2,0x70,0x3e,0x99,0x17,0x68,0x8b,0xc7, +0x2c,0x6e,0x70,0x21,0x0c,0x25,0x08,0x9a,0x5d,0xf8,0xf5,0x14, +0xcd,0x2e,0xf5,0xda,0xce,0x05,0xe3,0xcb,0xfc,0xc3,0x50,0xd2, +0x9b,0xf8,0x45,0x65,0x2e,0xe8,0x4d,0x27,0xf1,0xbd,0x58,0x84, +0x43,0x94,0x5e,0xe6,0xdf,0x0d,0x4e,0x4b,0x67,0x50,0xa0,0xa6, +0x4e,0x98,0xeb,0x16,0x94,0xb9,0xcf,0xc0,0x4f,0x20,0x0e,0xe7, +0xad,0xad,0xe8,0x5e,0xb4,0x09,0x33,0xd3,0x19,0x32,0xd0,0xac, +0xbc,0x87,0x9b,0x7d,0x3e,0x6a,0xe8,0x76,0x90,0x00,0x6e,0x69, +0x34,0x5b,0x0e,0x05,0x8b,0x5e,0xc9,0x02,0x28,0xee,0x40,0xf6, +0xf4,0xef,0xb7,0x2a,0x27,0xd6,0x64,0xa1,0x99,0xbf,0xdb,0x66, +0x3f,0xfe,0xe4,0x6e,0x11,0xff,0x92,0x19,0x73,0xb0,0x56,0xe5, +0x72,0xc2,0x91,0xc4,0x43,0xd3,0x41,0xd0,0xd4,0x8f,0x64,0x7f, +0x3c,0x34,0x56,0x44,0x06,0x32,0x26,0x66,0x1b,0x59,0xb6,0x7d, +0x69,0xf1,0x53,0x72,0x78,0x87,0xcd,0x6a,0x33,0x26,0x09,0x0e, +0xb8,0x4c,0x02,0x77,0x6d,0x6a,0x4f,0xb1,0xb1,0x82,0xba,0x64, +0x39,0x1c,0x45,0x7c,0x46,0xa3,0xd8,0x6f,0x46,0xae,0x14,0xa5, +0x59,0x0b,0x0f,0x12,0xc3,0x0a,0x2a,0x69,0xf1,0x71,0xe6,0x33, +0x25,0x62,0x37,0x39,0x11,0x75,0x72,0x8b,0x3f,0x5b,0x5b,0xf8, +0x43,0xfc,0x38,0x59,0xa1,0x8a,0xbd,0x6a,0x50,0x72,0x6e,0xa8, +0xa5,0x7b,0x2c,0x7f,0x71,0x1b,0xcc,0x6f,0xe0,0xf9,0x5f,0x74, +0x30,0x07,0x67,0x46,0x9a,0x37,0x2a,0xd0,0xce,0xfc,0xc6,0x14, +0xa1,0xf4,0xba,0xd5,0x9f,0x0e,0xd1,0x68,0x4d,0x7f,0x04,0x6c, +0xf5,0xa0,0xb9,0xf1,0xae,0x18,0x6e,0xdc,0x37,0x3b,0xa5,0x5c, +0xc1,0x36,0x66,0x93,0x0d,0xf2,0x64,0xf2,0x79,0x63,0x3e,0xce, +0xf5,0x61,0x12,0x96,0xe7,0xa2,0xb9,0xf1,0x96,0x71,0xa5,0x1b, +0x7a,0x65,0x53,0x1b,0xb6,0x28,0x48,0x66,0xc5,0x42,0xb1,0x0e, +0xba,0x26,0x69,0x57,0xb7,0xee,0xad,0xe6,0x08,0x2c,0xd0,0xb2, +0xb6,0x83,0x36,0x5b,0x4b,0x31,0x15,0x65,0xbb,0x24,0xab,0xc6, +0x0d,0x4d,0xc7,0xb8,0x62,0xcc,0xe0,0xc8,0x0c,0x24,0x45,0xd4, +0xc2,0xbc,0x7a,0x15,0xee,0x34,0xe0,0xc7,0x57,0x98,0x0e,0x36, +0x0b,0x9e,0x63,0x3b,0xd9,0x41,0x12,0x77,0x69,0x20,0xc5,0x4a, +0x70,0x59,0xe1,0x89,0xa8,0x56,0xcd,0xd6,0xea,0x59,0x7a,0xfb, +0x6f,0x99,0xa6,0xd4,0x48,0x95,0xbd,0xa4,0x82,0xa1,0xc9,0xe2, +0x5c,0x12,0x06,0x89,0x3a,0xda,0x0d,0x25,0x93,0x1b,0x42,0x7b, +0x30,0xd9,0xb0,0x09,0xe6,0xc3,0xb3,0x00,0xfa,0xf1,0x0a,0x64, +0x3c,0x26,0x09,0x94,0x1d,0xe8,0x4b,0x36,0x68,0xd2,0x0e,0x88, +0xc8,0x2c,0x2e,0x14,0xd7,0x50,0x47,0xa2,0x29,0xb4,0xa0,0xbf, +0x50,0xa6,0xc4,0xf6,0x24,0x25,0x5d,0xfa,0x34,0xe3,0xed,0x5b, +0x9c,0x6d,0x8c,0x26,0x26,0xc9,0xb0,0x77,0x9d,0xd0,0xbb,0xed, +0x12,0xea,0xf8,0xad,0x90,0x89,0x13,0x39,0xaa,0x58,0x38,0x29, +0x9d,0x97,0xcc,0x99,0x8c,0x47,0x2b,0x31,0x19,0xf8,0x0f,0x29, +0xa7,0xc1,0xe0,0xa6,0x68,0xb0,0xcd,0xee,0x36,0x8e,0xed,0x96, +0x07,0x76,0x9d,0x0f,0xc0,0x64,0xff,0x66,0x5c,0xc2,0xbb,0x58, +0x84,0xe8,0x7c,0xa2,0x26,0x83,0x41,0x90,0x0d,0xee,0x6e,0x80, +0x1a,0x76,0x33,0x9d,0x84,0xa3,0x64,0xb1,0xa0,0x53,0xd0,0xaa, +0x6b,0x81,0xca,0x3b,0x80,0x75,0x3b,0x8c,0xa0,0xc5,0xf8,0xcb, +0x44,0x49,0xe1,0x68,0x41,0x60,0x60,0x49,0xfc,0xd1,0xe1,0x21, +0xe1,0xbe,0x67,0x58,0xfc,0x2b,0xcc,0x0d,0x7a,0x6b,0x53,0xb0, +0x1a,0xeb,0xa3,0xac,0xc7,0x9e,0x49,0x58,0x27,0x62,0x3b,0xba, +0x9a,0x7c,0xfc,0xc2,0x8b,0xad,0x2f,0x7c,0xf8,0xd9,0xa4,0x42, +0x4a,0xb7,0x49,0x06,0x6d,0x37,0xbc,0xd9,0xe9,0x44,0xe2,0x22, +0x38,0x4a,0x23,0x73,0x25,0x1f,0xf5,0x9c,0x51,0xdb,0xf1,0xe5, +0xf6,0x15,0x98,0x14,0xb0,0xd7,0xe3,0x3f,0xe5,0xc1,0xcd,0xcb, +0xbf,0xa3,0x1e,0x50,0xc8,0x97,0xd4,0xc3,0x52,0x9e,0x7a,0x55, +0x5a,0x55,0x0f,0xf4,0x77,0xf6,0x15,0x0a,0x2d,0x95,0xcd,0x0e, +0xb8,0x3c,0xb9,0x07,0xc6,0x14,0xe0,0xe6,0x9a,0xbe,0x92,0x5b, +0xfb,0x89,0xfa,0x5c,0xf4,0x49,0x4e,0xae,0x4d,0xb4,0xd9,0x72, +0x6c,0x94,0xd7,0x02,0x0a,0xbd,0xa5,0x39,0x61,0xca,0x39,0x38, +0xb6,0xc6,0x72,0x51,0x77,0x41,0x59,0xa4,0xc2,0xb3,0x1b,0x39, +0x6d,0x67,0x0a,0x96,0x20,0x9e,0x41,0xc6,0xe1,0x08,0xad,0x64, +0x7d,0xe9,0x2d,0xab,0x2f,0x09,0x15,0x4a,0x70,0x84,0xce,0x7b, +0xcd,0xe9,0x1e,0xd7,0x84,0x9a,0xdd,0x20,0xcd,0x03,0xa1,0xa4, +0xee,0x83,0x3b,0xb5,0xe6,0x0c,0xea,0xa7,0xe2,0x39,0x27,0x5e, +0x7e,0x0d,0xc9,0xf1,0xd2,0xb1,0x15,0xda,0x14,0xd6,0x31,0xa3, +0x31,0x09,0xad,0xb0,0x83,0xe0,0x61,0x84,0xb9,0x24,0x23,0x5c, +0x7c,0x47,0xa6,0xd9,0x13,0x93,0x03,0x97,0x0c,0x64,0xf8,0xbd, +0xae,0xb5,0xf9,0x4b,0xe3,0x50,0x3a,0x4c,0xca,0xf3,0xcc,0xc7, +0x63,0x8d,0x22,0xe7,0xfb,0x39,0x5a,0x46,0xf6,0x93,0xbb,0x79, +0x4f,0x5e,0x69,0x05,0x4c,0x48,0x58,0x09,0xf4,0xa0,0x14,0x46, +0xc2,0xeb,0x95,0x12,0xed,0x2a,0xae,0x68,0x94,0xac,0xdc,0x80, +0xd4,0xfe,0xdd,0x68,0x93,0x9d,0x71,0x82,0x92,0x6e,0x2d,0x50, +0x27,0x02,0xa3,0x73,0x25,0xf7,0xe9,0x4d,0xba,0xa8,0x70,0x3c, +0x52,0x55,0x7f,0x48,0x63,0x2c,0x41,0xa2,0xdf,0x89,0x15,0x37, +0x56,0x41,0x8b,0xe2,0x71,0xc9,0x8b,0x4b,0x54,0x19,0x6b,0x53, +0xdc,0x45,0x7b,0xc0,0x62,0xdf,0xd1,0xd5,0x7a,0x26,0x6f,0x32, +0xe9,0xa2,0x27,0xab,0x40,0xc6,0xe3,0xb0,0xd4,0xf8,0x44,0x8c, +0x44,0x54,0xba,0xd4,0x72,0x69,0x8a,0xf0,0xbb,0xeb,0x6c,0x0a, +0xa3,0xc6,0x83,0x13,0x80,0x52,0xeb,0x83,0x2c,0xbc,0xee,0x0a, +0x64,0xa5,0x6c,0xf2,0x9b,0xdd,0x5d,0x37,0x19,0x7f,0x91,0xb3, +0x71,0x85,0x8c,0xb2,0x58,0xa9,0xda,0xb0,0x5a,0x0b,0xda,0x8d, +0x20,0x73,0xa5,0xe3,0x6a,0xd0,0xcb,0xee,0x06,0x01,0xf7,0x17, +0xea,0x6e,0x30,0x6a,0x69,0x59,0x79,0xf7,0x69,0x68,0x33,0x8e, +0xfd,0xac,0x85,0x08,0x5e,0xd9,0xe8,0x0d,0xf3,0xff,0xa8,0xe6, +0xd4,0x73,0x2d,0x0b,0xc2,0xa5,0x86,0xd2,0x71,0x5f,0xd5,0x2a, +0x6e,0x5c,0xd7,0xde,0xf1,0xc9,0x95,0xdb,0x61,0xf4,0x1e,0xf8, +0x8b,0x1a,0xd2,0x03,0x10,0x17,0x93,0x89,0xef,0xd2,0xc2,0x5e, +0x29,0xf6,0x2e,0xbc,0x37,0xfa,0xe7,0xc2,0x6c,0x81,0x98,0xc6, +0x25,0x86,0xd1,0x53,0xe7,0x03,0x9b,0x25,0xee,0x8d,0x18,0x19, +0x2d,0x21,0x40,0x52,0xb4,0x2f,0xce,0xa7,0xf9,0x8d,0x7f,0xb1, +0x10,0x28,0x87,0x88,0xca,0xc9,0x84,0xea,0x87,0xb4,0x6b,0xcb, +0xff,0xc9,0xb4,0x08,0xdc,0x75,0xec,0x68,0xa6,0xc3,0x60,0x8e, +0xb2,0xcc,0xbd,0x9a,0x65,0x8f,0x92,0x94,0xf2,0x6e,0x3a,0xa2, +0x24,0x78,0x21,0xa5,0x73,0x65,0x4e,0x84,0x86,0x37,0x26,0x6d, +0x9f,0x3a,0x3d,0x02,0x66,0xb1,0xc8,0xab,0xc8,0xb7,0x20,0x2f, +0x26,0xaf,0xc9,0x67,0x61,0xc5,0xd3,0x07,0x41,0xff,0xd2,0xac, +0x9f,0x5e,0xe3,0x08,0xc5,0x98,0x92,0x32,0x99,0x55,0x76,0x61, +0xad,0x32,0xbe,0x75,0x79,0xfd,0xe6,0xf8,0xc5,0xeb,0x57,0x47, +0xfe,0x0b,0x6d,0xb8,0x1a,0x88,0x3b,0xe9,0x49,0xd1,0xe8,0x17, +0xe6,0xaa,0x20,0xe9,0x50,0xcc,0xe5,0x74,0x32,0xbf,0x71,0x77, +0x05,0x60,0x6a,0x56,0x7f,0xe8,0x7d,0x04,0x82,0x56,0xd2,0x8b, +0x7f,0x4f,0x06,0xbd,0x07,0x68,0xe3,0x87,0x0d,0xcd,0x07,0x29, +0xbe,0xfc,0xd7,0x8b,0x3f,0xc1,0xd7,0xe0,0xe2,0xa2,0x82,0xca, +0x0e,0xee,0x29,0x90,0x2a,0xd0,0x01,0xd7,0x07,0x8e,0x43,0xfb, +0x9a,0xe5,0xd6,0x7b,0x1c,0xc6,0x96,0x35,0x56,0xbd,0xab,0xee, +0x46,0xd4,0x94,0xba,0xd9,0xec,0x63,0x8d,0x5e,0xc0,0x0a,0x89, +0x17,0xc4,0xf6,0xc9,0x31,0x3f,0x2f,0x76,0x4d,0x6c,0xa7,0x07, +0x4f,0x17,0xbe,0x42,0x37,0x06,0x44,0xa8,0x63,0xd5,0x7c,0xfc, +0x61,0x3c,0xf9,0x38,0xe6,0x10,0x18,0x34,0x23,0xa7,0x57,0xec, +0xab,0xd2,0x48,0x5a,0x2f,0x5d,0x8d,0xbc,0xfa,0xc1,0x7e,0x76, +0x81,0x3a,0xb6,0x67,0x93,0xf9,0x78,0x40,0xd7,0x3f,0x85,0x7b, +0x07,0x0a,0xa9,0xf9,0x38,0x17,0xda,0x39,0x02,0xd5,0xa9,0x72, +0x84,0x9c,0xfa,0x94,0x9c,0x26,0xb7,0x5e,0x99,0xd5,0x99,0xb2, +0xda,0xfc,0x99,0x8f,0x6d,0x49,0x94,0x68,0xae,0x9e,0x93,0x43, +0x16,0xcc,0x55,0x97,0x2f,0x22,0xe1,0x0d,0x9e,0x28,0x55,0x3e, +0xa7,0xb2,0xef,0xac,0xec,0x62,0x01,0x8c,0x0c,0x3f,0x69,0xf3, +0xca,0xb5,0x1b,0xf5,0xd6,0x96,0x1f,0x6f,0xcc,0xac,0x27,0x4c, +0x18,0x85,0x6b,0x68,0x33,0xa6,0x78,0x29,0xd9,0x29,0xd1,0xb1, +0x78,0xb2,0x44,0xfa,0x5d,0x74,0xb3,0xaf,0x01,0x78,0x07,0xf6, +0x4e,0x52,0x65,0x9f,0x5b,0x18,0x9a,0x4a,0x10,0x1e,0x69,0x34, +0x94,0xb8,0x5e,0xff,0x00,0x0f,0xcd,0x9d,0x8d,0xa8,0x21,0xae, +0x6a,0x92,0x46,0xb4,0xd1,0x89,0xda,0x7e,0xcd,0xa1,0xa9,0x5c, +0x1e,0xb1,0xe0,0xb2,0xab,0xa8,0xd8,0x03,0x17,0xe9,0x61,0xc6, +0xa0,0xa1,0xc1,0x78,0x4e,0x64,0x98,0x2b,0x54,0xa9,0xe6,0x6a, +0xd8,0x86,0xab,0x7b,0xa6,0xe4,0xda,0x2f,0x14,0xf8,0x99,0xeb, +0x26,0x23,0xad,0xa9,0x7d,0x48,0x56,0xdb,0x5e,0x38,0xdd,0x93, +0x75,0x8e,0x26,0xfc,0x14,0xce,0x3e,0xe8,0x3f,0xca,0xc7,0x97, +0xa3,0x21,0x43,0x52,0xbd,0xac,0x5a,0x93,0xeb,0x5e,0x4e,0xc1, +0xb7,0x6a,0xb7,0x5e,0xb7,0xf3,0x23,0xeb,0xf0,0xa6,0xdf,0x0e, +0x86,0xea,0x31,0xd2,0x76,0xf8,0xe4,0x88,0xb3,0x0f,0x7f,0x3a, +0xf1,0x97,0x8d,0x6f,0x9f,0x72,0x9a,0x27,0x16,0x19,0x7b,0x05, +0x77,0xef,0x86,0x86,0xf0,0x3c,0xac,0xbe,0x43,0x93,0xd4,0xf3, +0x42,0x6b,0xf1,0x78,0x19,0x3b,0xe5,0x12,0xf5,0xbe,0x9e,0x8a, +0x7a,0x55,0x8e,0x7c,0xf7,0x68,0x2b,0x20,0x19,0xfa,0x9c,0x8e, +0x27,0x7e,0x8f,0xd7,0xe8,0x24,0x7f,0x61,0xf3,0x78,0xbd,0x20, +0x37,0x8a,0xb8,0x5c,0x8d,0x84,0x14,0x9b,0xed,0x08,0x39,0x64, +0x5b,0x63,0x95,0x63,0xc4,0xbd,0x0f,0x03,0x49,0x12,0xa0,0x6d, +0x9f,0xaf,0x18,0x17,0x70,0x63,0x59,0x80,0x26,0xde,0x06,0x3b, +0x9b,0x29,0x3d,0x66,0x6b,0xfb,0x67,0x33,0x43,0x18,0xb8,0x58, +0x8a,0x23,0x9a,0x42,0x52,0x31,0xbd,0xc6,0xb3,0x71,0xe6,0x14, +0xea,0xb0,0x48,0xd4,0x60,0x94,0x0e,0x8e,0xb1,0x92,0xb5,0xca, +0x2a,0x7a,0x8b,0xec,0x8f,0x8a,0xd8,0x54,0x49,0x85,0x44,0x2e, +0x16,0x36,0x2d,0x43,0x93,0x38,0x21,0x40,0xe9,0x11,0x6c,0x38, +0x42,0xbf,0xb0,0xbb,0xf1,0xcc,0x07,0xf2,0x51,0xbe,0xde,0xfb, +0xf3,0x2d,0xb4,0xb5,0x85,0xf8,0xca,0xae,0x9f,0xd1,0x4d,0x60, +0x79,0x9c,0x20,0x74,0x30,0x32,0x98,0x67,0x5b,0x31,0x30,0x2c, +0xa8,0x69,0x09,0xac,0x09,0x15,0xf7,0x23,0xb0,0x54,0x68,0x83, +0x4c,0x84,0xf9,0x5b,0x0d,0x3f,0xe1,0x83,0xf4,0xcb,0x37,0xe4, +0xc7,0xc3,0x04,0xb9,0xc4,0x20,0xac,0x90,0x3b,0x7b,0x29,0x6f, +0x17,0x29,0xea,0x05,0x4a,0x9c,0x49,0xee,0x6c,0x2a,0x8e,0x57, +0x76,0x86,0x4a,0xb3,0x34,0x8e,0xfa,0x67,0x9a,0x8b,0x10,0x74, +0x14,0xd9,0x6d,0x14,0xa5,0xe0,0xee,0x30,0x20,0xb6,0xd0,0x1c, +0xa7,0xe6,0x93,0xc4,0x4b,0xea,0xf0,0x69,0x35,0xdd,0x54,0x14, +0xb4,0x0b,0x16,0xcf,0x79,0x15,0x4e,0xf0,0x50,0x04,0xb7,0x51, +0x20,0x3a,0x07,0x4c,0x9d,0x02,0xbd,0xf6,0xc2,0x1d,0x07,0xed, +0x00,0x45,0x42,0xcf,0xbd,0x0a,0x36,0xf5,0x8a,0x1a,0x70,0x46, +0x20,0x30,0x9c,0x65,0x9b,0x2d,0xb8,0x77,0xb6,0xbe,0xc0,0x41, +0xb3,0x16,0x38,0xc3,0xe7,0xe3,0xad,0x2d,0xf3,0xd2,0xd0,0x51, +0x91,0xe0,0x6e,0x61,0xf3,0xca,0x29,0xd7,0xed,0x25,0xd6,0x1f, +0x1a,0x62,0x78,0xc1,0xb7,0xab,0x2e,0xf0,0xd8,0x21,0x32,0x72, +0x21,0x97,0xd8,0x68,0x24,0x23,0xa1,0xb5,0x41,0x2f,0x39,0x5c, +0x86,0xae,0x83,0xec,0x49,0xfa,0x89,0xff,0x27,0x8a,0x79,0xc8, +0x18,0xab,0xdc,0xd5,0x99,0xd9,0x4f,0x69,0x5d,0x35,0x44,0x5d, +0xe5,0xd5,0x14,0xdf,0x3e,0x65,0x1f,0x4a,0x37,0xce,0x15,0x17, +0x52,0xe5,0x7a,0x50,0x0f,0x74,0x4d,0x1f,0x8d,0xac,0xfe,0x0a, +0x25,0x7f,0x70,0x37,0x52,0x29,0x5f,0x33,0x05,0x03,0x24,0xc3, +0x91,0xf7,0xc3,0xe7,0xb0,0x92,0xa8,0xc3,0x1e,0x0b,0xe0,0x4c, +0x0e,0xb7,0x6e,0x80,0x05,0xde,0xc9,0xae,0x0f,0x0c,0x72,0xb4, +0x70,0x9c,0xca,0x99,0x6d,0x63,0x13,0xdd,0xd6,0x66,0xfc,0xd8, +0xb5,0x52,0xf6,0x11,0xef,0x18,0xdc,0x4d,0x9d,0xca,0xee,0x4a, +0xcd,0xd1,0xcc,0x8b,0x4d,0x36,0xdd,0x5b,0x0c,0x7b,0xf1,0xcc, +0xf9,0xe5,0xd3,0x7b,0xb8,0xf2,0xb9,0x5d,0xb6,0x6d,0xa0,0x7a, +0x10,0xd9,0x81,0x93,0xf9,0x24,0xd8,0x6c,0x60,0x54,0xbc,0x92, +0x3a,0x7f,0xd0,0x08,0x94,0x59,0x96,0x55,0x12,0xde,0x96,0x03, +0x4c,0x7c,0x2f,0x16,0x26,0xb0,0xc4,0xe5,0x21,0x1b,0xe4,0x85, +0x3e,0xbb,0x96,0x3a,0x96,0x16,0x8f,0x8c,0xfe,0x06,0xcf,0x37, +0x4b,0xeb,0x0a,0x8b,0x41,0xcc,0xfd,0x71,0x6d,0x13,0x9b,0xf1, +0x57,0x1a,0x6f,0x54,0x1a,0x23,0x36,0x04,0xa6,0x96,0x52,0xd2, +0xc1,0x30,0x7d,0xac,0xc7,0x90,0x3b,0xb9,0x18,0x2f,0x09,0xa2, +0x49,0x3a,0xde,0xa8,0xd2,0xeb,0x9a,0x7d,0x9c,0x0e,0x14,0xea, +0xf1,0x4a,0x85,0xe9,0x0f,0x5b,0xd2,0xa6,0x68,0xa1,0x65,0x92, +0x7a,0xed,0xb5,0x1a,0x5b,0xa7,0x12,0x9d,0x3e,0xd3,0x27,0x62, +0xb1,0xce,0x76,0xe5,0x62,0x9e,0x95,0x06,0x35,0x2f,0x46,0x02, +0x66,0xdf,0x65,0x6e,0xbb,0x4f,0x98,0xdf,0xd0,0x12,0xe1,0xe4, +0x0c,0x11,0x76,0x5d,0x16,0x6f,0x0a,0xad,0x9e,0x6f,0xde,0x4e, +0x41,0xc2,0x0d,0x22,0xec,0x71,0xe6,0xf0,0x75,0x62,0x7f,0x13, +0xcb,0xcc,0x0d,0x48,0xaa,0x73,0x63,0x56,0xf3,0xa6,0xb5,0x89, +0xd2,0x9f,0x38,0x61,0xa3,0xb2,0xd8,0x4a,0x51,0x38,0xa1,0x79, +0x9c,0xa5,0x72,0x30,0x61,0xcc,0x32,0xdc,0x1d,0xb3,0xae,0x5f, +0x70,0x15,0xb9,0xe4,0x18,0xd8,0xcf,0x17,0xf2,0x28,0x2c,0x9a, +0xf2,0x91,0x19,0x01,0x33,0xd8,0xf6,0x09,0xcb,0x2d,0x15,0xe3, +0xb5,0x6e,0xd0,0x1a,0x41,0x66,0xb8,0xe4,0x0c,0x4a,0xc2,0xda, +0x6d,0x86,0x5d,0xa6,0xb9,0xf0,0x4a,0x82,0xfc,0xc6,0xab,0x42, +0x15,0x62,0x4a,0x12,0xc9,0x0b,0xe0,0x19,0x5f,0x05,0xfa,0x0d, +0x5e,0x63,0xaf,0x50,0x6e,0x20,0x38,0x09,0x6e,0x94,0x10,0x82, +0x46,0x2a,0x55,0x9b,0x30,0xb4,0x7a,0xfa,0xc8,0x23,0x89,0xf1, +0xc1,0x4c,0x1d,0x83,0x9e,0x36,0x2d,0xc4,0xe5,0x52,0x70,0x6b, +0x4b,0xe0,0x21,0xcd,0xe5,0xaa,0x5c,0x5e,0x9e,0x24,0x44,0x9c, +0x0f,0x1c,0x56,0xb0,0x1f,0x25,0x51,0xc2,0x19,0xa1,0x04,0xee, +0x80,0xbd,0x46,0x05,0x51,0xa0,0x99,0xeb,0x0e,0xa3,0xe3,0x41, +0x25,0xde,0xf0,0x24,0x16,0x94,0x61,0x57,0xf4,0x00,0x09,0x0c, +0x44,0x3c,0xe0,0x09,0x72,0xdd,0xa2,0xd7,0x19,0x8a,0xde,0xa5, +0xa8,0xd4,0x08,0x3d,0x39,0x47,0x38,0x0b,0x13,0xac,0xa1,0xab, +0x0f,0x0f,0x2c,0x47,0x58,0x48,0x00,0x35,0xf2,0xaa,0xf6,0x2c, +0xaa,0xe6,0x9b,0x5b,0x0f,0x3c,0x2e,0x34,0x95,0x88,0x94,0x5e, +0x38,0x2d,0x25,0x6e,0xd5,0x0c,0x28,0x58,0x7a,0x6d,0xde,0x91, +0x8d,0x8a,0x0e,0xf0,0xac,0xce,0x4f,0x5a,0x37,0x75,0x78,0xa6, +0x9b,0x9b,0x21,0x6f,0x24,0x72,0xa4,0xe3,0xb1,0x1a,0x8d,0x76, +0x58,0x3d,0x6f,0x52,0x24,0xa5,0xca,0xeb,0x8e,0x4d,0x6c,0x61, +0xe6,0x50,0x8a,0x3b,0x55,0xbb,0x6e,0x25,0x95,0xcb,0x7a,0x52, +0xb1,0x7e,0x01,0x4a,0x18,0x5d,0x25,0xac,0xd6,0x56,0x09,0xaa, +0x6a,0x93,0xdd,0x11,0xee,0x69,0x78,0x02,0xc0,0xfa,0xb9,0x25, +0x19,0x70,0x92,0x94,0x2c,0xd6,0x31,0xf9,0x60,0xb3,0x6e,0x5d, +0xad,0x6e,0xea,0x91,0xd3,0x8d,0xb8,0x5f,0x6c,0x9c,0x62,0x0f, +0x9e,0x26,0x1d,0x48,0xdf,0x60,0xc5,0xdb,0xa4,0xc3,0x9a,0xb7, +0x65,0xb5,0x5d,0x5f,0xf9,0x36,0x30,0x7b,0xe7,0x2a,0xe0,0xd9, +0xbc,0xf3,0x16,0xec,0x84,0x1e,0x22,0x5b,0x36,0xc5,0xa8,0x74, +0x57,0x78,0xe4,0x52,0x96,0xe9,0xb2,0xda,0xe1,0x26,0xf5,0x5b, +0x3c,0xa5,0x88,0xa4,0x6f,0x7a,0xca,0xb2,0xb4,0x3e,0x4a,0xe3, +0xc3,0x8b,0xd9,0xaa,0xb0,0xbc,0x8e,0x29,0x2e,0xb5,0xa5,0x31, +0x6c,0x11,0x02,0x7f,0xd7,0x63,0x6d,0x73,0x2e,0xf3,0xef,0xac, +0x80,0x69,0x54,0x5f,0x3b,0x1e,0xf4,0xdf,0x7b,0x09,0x7a,0x8b, +0x2e,0xef,0x6b,0x59,0xb7,0x6b,0x67,0x95,0x7f,0x1e,0x35,0xac, +0xc7,0xb2,0xd7,0x6b,0xbb,0xb5,0x00,0x84,0xab,0xe6,0xb5,0x9e, +0x8b,0x65,0x53,0xa0,0x89,0x90,0x00,0x97,0x2c,0x0f,0x90,0x3c, +0xbc,0x02,0x04,0xec,0x9d,0xf9,0xfb,0xa8,0xbd,0xf4,0xf9,0xd3, +0x22,0x00,0x79,0x50,0x76,0xaa,0xa8,0x14,0x6b,0x2a,0x4a,0x5c, +0xa2,0x38,0xfe,0xca,0x03,0xb1,0xd1,0x96,0x76,0x3a,0x78,0x55, +0xa7,0x54,0x7b,0xfc,0xf4,0xce,0x9b,0xac,0x3e,0x00,0x9c,0x74, +0xb7,0xb7,0x16,0xcb,0xe6,0x4d,0xe8,0xb5,0x78,0xb6,0x80,0x65, +0xbb,0x83,0xa3,0xa0,0xcd,0xba,0xf5,0xa5,0x9b,0xb5,0xae,0x54, +0xec,0xb5,0x59,0xc5,0xbe,0x6d,0xd9,0x1c,0xb9,0x83,0x59,0x1d, +0x46,0x2f,0x37,0x09,0xcd,0xcc,0xa4,0x01,0x05,0xd8,0xf2,0x99, +0x31,0x31,0xc3,0x39,0xb9,0xeb,0xe0,0x2d,0xb4,0xb4,0x89,0x78, +0x87,0x50,0x1c,0x94,0x84,0x1a,0xa4,0x3c,0x6d,0x57,0x9e,0x25, +0x50,0xa9,0x3b,0xf1,0xcf,0xef,0x78,0x40,0xdb,0x58,0x89,0x17, +0xd4,0x2b,0x49,0xf7,0xdf,0xda,0xb3,0xaa,0x38,0xf4,0x2d,0xe9, +0xb8,0xc5,0xed,0x52,0x21,0x4c,0x54,0x9a,0xef,0xcf,0x90,0x02, +0x1a,0x3a,0xfe,0x19,0x11,0x32,0x46,0x5e,0x3a,0x7a,0xab,0x35, +0x49,0xf6,0xf8,0xf8,0xc5,0xad,0xe6,0x57,0xb7,0xf2,0xa4,0xff, +0x25,0x6d,0x48,0x14,0xb2,0x91,0xc2,0xea,0x53,0x24,0xfa,0xa9, +0xe3,0xe6,0xb8,0xb3,0x61,0xef,0x1a,0x2d,0xab,0x0a,0xaa,0x10, +0xce,0x09,0x2b,0x41,0x9e,0xf4,0x95,0x7e,0x7c,0x31,0x27,0xc5, +0x25,0x4b,0x36,0xba,0x06,0xf6,0xee,0x5c,0xfb,0x0a,0x01,0xb7, +0x10,0xf9,0x3b,0x53,0x79,0xf2,0xae,0xcc,0xe4,0xb6,0x12,0x05, +0xba,0x0c,0x36,0x21,0xb1,0x65,0x19,0x45,0x26,0x23,0xbd,0x13, +0x17,0x6b,0x9b,0x2d,0xe5,0x73,0x61,0xc6,0x87,0x7a,0xe0,0xc5, +0x83,0xc7,0x94,0x13,0x70,0x26,0x2a,0x3c,0x45,0xad,0x91,0x13, +0x17,0x0f,0xc8,0xeb,0xdf,0xe1,0x55,0xf3,0x81,0xe0,0x30,0x9e, +0x94,0x49,0x75,0x93,0xd9,0xf7,0x98,0x26,0x02,0xd2,0xf3,0xcc, +0x84,0x4a,0xca,0x53,0x3c,0xae,0x4d,0x3a,0x68,0x4f,0x19,0xcb, +0x3e,0xae,0x89,0x05,0x53,0xea,0x30,0x0d,0x7f,0x8b,0x73,0x33, +0x9a,0x34,0x16,0x71,0xa1,0x00,0xf7,0x9b,0xf2,0x5a,0x53,0xd5, +0x5d,0xaf,0x3a,0x21,0x8f,0x71,0x0b,0x61,0x2c,0xca,0x59,0xe3, +0x6b,0x6f,0x39,0x22,0xa1,0x8d,0xc5,0xa4,0xff,0xe3,0x2e,0x5b, +0x7b,0xa5,0x8b,0x84,0x2a,0xdb,0x78,0x41,0x91,0x7c,0x1d,0xf8, +0xbb,0x90,0x26,0xf2,0x08,0x5f,0xb6,0x3d,0x7c,0x9e,0xc1,0xb5, +0x12,0x49,0xc1,0x2d,0x86,0xb0,0xc1,0x66,0xc4,0x0c,0x15,0x59, +0xd6,0xb1,0xac,0x57,0xca,0x16,0x57,0x5f,0x5a,0xc1,0x04,0x73, +0x83,0xba,0xe3,0x4b,0xbf,0x87,0xe5,0x4b,0xb3,0x24,0x35,0xdd, +0xd0,0x09,0xee,0x50,0x2a,0x81,0xbd,0x9b,0xb8,0x0a,0x08,0x79, +0x07,0x47,0x5a,0x13,0x44,0xcc,0x17,0x5b,0x02,0x33,0x37,0xf3, +0x47,0x6e,0x82,0xdf,0xda,0x1b,0x6b,0x63,0xde,0xdd,0xdd,0x13, +0xaf,0xd0,0x25,0x58,0x6d,0x63,0xae,0xa4,0x5b,0x22,0xaf,0xd5, +0xb0,0x63,0xf8,0x2a,0x66,0x5d,0xbb,0x63,0x78,0xce,0x4a,0xe0, +0x8a,0xae,0xed,0x41,0x7b,0x7e,0x81,0x7e,0x27,0x23,0x70,0x7e, +0x9e,0x64,0xb9,0xae,0x39,0x21,0xd7,0xc8,0xc8,0x02,0x47,0xec, +0x71,0x5e,0xd8,0x84,0x66,0x2b,0xb7,0x76,0x45,0xb0,0x4a,0x72, +0x62,0xc1,0xa4,0x38,0xb2,0x09,0xe6,0x63,0xd4,0x1c,0x16,0x27, +0x5a,0xbf,0x13,0x55,0x50,0xe2,0x82,0x14,0xdc,0x3a,0xd8,0x35, +0xc0,0x6a,0x40,0x74,0x86,0x35,0xc7,0xe6,0xcd,0x96,0xbc,0x85, +0xcc,0xfc,0x8e,0xf6,0x4d,0xad,0x75,0xbc,0x8d,0xc1,0x4b,0x0a, +0xdf,0x42,0x83,0x42,0x30,0x3e,0x89,0x2e,0xb7,0x34,0xbf,0x8b, +0xe6,0x5f,0xd2,0x0c,0xd6,0x1f,0x1e,0xd9,0x72,0x3c,0xc5,0x8c, +0x4a,0xb4,0x97,0x53,0x98,0x70,0xaf,0xac,0xd3,0xca,0x64,0x31, +0x6c,0xee,0xb6,0xbb,0x5a,0x47,0x80,0x40,0x62,0xd4,0xdc,0x0b, +0x5f,0xa9,0xda,0x2a,0x1f,0xd5,0x69,0x52,0x2d,0xf5,0x12,0xad, +0x39,0xc5,0xff,0xfa,0x3f,0xf3,0x21,0x3a,0xaf,0xb8,0x99,0x9f, +0x8d,0xf2,0xe2,0xea,0xd9,0x27,0x0d,0xa3,0x21,0x9e,0xbc,0x79, +0x11,0x6b,0xce,0x7e,0x3e,0xea,0xeb,0x35,0x28,0xd8,0x97,0xe0, +0xc9,0x7f,0x7f,0xe0,0xf1,0xf6,0x0c,0x0a,0x16,0x26,0x2c,0x98, +0x3a,0x9b,0x4c,0x66,0xba,0xf0,0x3e,0x5a,0xe1,0x88,0xc9,0x8b, +0xb9,0xb2,0x48,0x14,0xe7,0x31,0x8c,0xaa,0x40,0x4f,0x37,0x81, +0x57,0xba,0x90,0x28,0xb1,0xea,0x0e,0xf7,0x1f,0x63,0x15,0x37, +0xd0,0xef,0x34,0x29,0x79,0x81,0x0b,0xbf,0xfd,0xff,0x05,0xaf, +0xfd,0x74,0x77,0x6e,0x44,0xef,0x8e,0x9f,0x6f,0xff,0x3d,0x6a, +0x77,0xc7,0x97,0x27,0x27,0x29,0x1a,0x56,0xeb,0xa9,0xae,0xb1, +0xb1,0xa6,0x3f,0x61,0xe8,0x6d,0x8b,0xf0,0x27,0x11,0x68,0x9a, +0x4f,0xd5,0x74,0xd1,0x18,0x0b,0xba,0xeb,0xb7,0x7c,0x24,0xd5, +0x2b,0xc9,0x78,0xb8,0xb1,0x99,0x5f,0x83,0xcf,0xe5,0xfe,0x78, +0xd6,0x5e,0x62,0x59,0xb0,0x92,0x5b,0x10,0x7a,0xb8,0xb6,0x7c, +0xbc,0x83,0xe4,0xee,0xdf,0x17,0x37,0x99,0xdc,0x04,0x5c,0x71, +0x61,0x1e,0xee,0x36,0x9a,0x83,0xc8,0x1f,0x3b,0x2a,0x22,0xf7, +0xb9,0xc6,0x03,0x21,0xbf,0x9f,0x37,0x8d,0x75,0xef,0x9e,0xf5, +0x72,0x68,0x20,0xa2,0xa4,0x6d,0x52,0xaf,0x27,0x83,0xf9,0x68, +0x48,0xaf,0xec,0x39,0x51,0x81,0x35,0xe2,0xa8,0xa7,0x1b,0x95, +0x1d,0x90,0x88,0x8c,0x4a,0x58,0x6f,0xb2,0x2f,0xff,0x7f,0x2b, +0x5b,0x63,0xe2,0x97,0xff,0xbf,0x89,0xeb,0xd0,0xe1,0x18,0x7d, +0x68,0x4f,0xf3,0xa3,0xe1,0x25,0xf6,0x1e,0xea,0xb9,0xf0,0x1a, +0x65,0x13,0x69,0x70,0x81,0x0a,0xbd,0x5e,0xf2,0xac,0x59,0x86, +0x9d,0xbf,0xed,0x7d,0xbb,0x73,0x99,0x83,0x3b,0x4b,0x19,0xf9, +0xf0,0x00,0x23,0x33,0x2f,0x72,0xef,0x07,0x8c,0x6c,0x44,0xe2, +0xd6,0xb6,0x02,0xfd,0x8d,0x66,0xda,0x31,0xf6,0xe8,0x46,0xe7, +0x2b,0x42,0x4a,0xa4,0xab,0x6f,0x20,0x54,0x14,0xf0,0x68,0x17, +0x0b,0xf8,0x5f,0x3e,0x29,0x4f,0x30,0x32,0xf5,0x49,0x79,0xb4, +0x73,0xa9,0xdb,0xc7,0x8f,0x7b,0x8a,0x80,0xca,0x8f,0xd4,0x28, +0x43,0x8a,0x3a,0x91,0x8e,0xd6,0x08,0xbd,0x8a,0xbc,0x9d,0xcc, +0xf5,0xb4,0x32,0x1a,0x85,0x8a,0x5d,0x44,0x17,0xac,0xc8,0x64, +0xe2,0x33,0x0b,0x80,0xd1,0x06,0x2a,0x33,0x1f,0x68,0x77,0x04, +0x93,0xe6,0xd3,0xd1,0x1b,0xf0,0x01,0x8d,0x5a,0xfe,0xae,0x14, +0xee,0x6e,0xe3,0x81,0x06,0x7c,0xad,0xa3,0xab,0x68,0x56,0xd6, +0x2d,0x14,0xf9,0x67,0x92,0x4a,0x35,0x7a,0x76,0xe9,0x53,0xc4, +0x8c,0xb0,0xc5,0x30,0x71,0x18,0x88,0x62,0x78,0xc3,0x1b,0xe0, +0x11,0xdb,0x18,0x31,0x16,0xe9,0x19,0x7b,0xdb,0xd1,0x84,0x55, +0x95,0xc4,0x48,0xac,0x16,0xbb,0x8c,0x0d,0x74,0xfe,0xc9,0xf2, +0x47,0x5e,0x3c,0xf7,0xb9,0x39,0xcd,0x16,0x8a,0x13,0x06,0x48, +0x2c,0x34,0x31,0x64,0x14,0xc4,0x63,0xd6,0xe8,0x25,0xf0,0xd6, +0x96,0xee,0x5f,0xbe,0x3c,0x13,0x5e,0xce,0xc9,0x97,0x28,0xd5, +0x8e,0xd2,0x8a,0xf9,0x99,0x9e,0x84,0x31,0xd8,0xfd,0x27,0xf6, +0x0e,0xf1,0xba,0xa6,0xe4,0x26,0x7f,0xcb,0x66,0x95,0x5e,0xc0, +0x86,0x03,0xa6,0xd4,0x35,0xc7,0xe5,0x0c,0x46,0x71,0x7b,0x98, +0x88,0xa6,0x99,0x6e,0xe5,0x1e,0xb1,0x46,0xe7,0x40,0x03,0x03, +0xbf,0x6d,0x5f,0x92,0x18,0x70,0x0a,0x03,0x04,0xef,0x8c,0x69, +0xa8,0x48,0xcf,0xf6,0xdc,0x69,0xa2,0xa9,0x0f,0x9e,0x3d,0x7f, +0xf2,0xee,0xe5,0xf1,0xe9,0x93,0xa7,0x28,0x17,0xb4,0xdd,0x1a, +0xb4,0xb3,0x68,0x62,0x8a,0x50,0xce,0x38,0xe2,0x55,0xbf,0xf8, +0x61,0x32,0xf8,0x0c,0x36,0x7b,0xdf,0xbc,0x3e,0x3a,0x5e,0xbc, +0x79,0xa7,0xff,0x81,0x67,0xfd,0xe4,0xde,0x4e,0xce,0x06,0x63, +0xe8,0x94,0x7a,0x3d,0x9c,0x5d,0x4d,0x06,0x49,0xdb,0xd4,0xa5, +0x3b,0xc6,0xc3,0x9f,0x43,0xdd,0x52,0xfd,0x3d,0xd5,0x7f,0xa8, +0xfa,0x8f,0xd8,0xee,0x22,0x34,0x33,0xdb,0xd9,0x52,0x68,0x90, +0x89,0xc6,0x06,0x8e,0xa1,0xd5,0xde,0xe3,0x1f,0xa5,0xe4,0x9a, +0xa2,0xff,0xc8,0x20,0xc8,0xfa,0x0f,0xc9,0x79,0xfc,0xc3,0x14, +0x7f,0xf6,0x52,0x72,0x03,0x6f,0x87,0x49,0x7f,0x2f,0x41,0xf7, +0x53,0x80,0x5f,0x93,0x02,0x85,0x67,0x9a,0x1e,0x91,0xdd,0x1e, +0x50,0x64,0xae,0x16,0x68,0x1d,0x18,0x98,0x96,0xe2,0x62,0xf7, +0x18,0xd8,0xa6,0xec,0x99,0x94,0x87,0xd6,0x85,0xbd,0x87,0xa4, +0xe3,0x70,0xa4,0xdc,0xa4,0x1d,0x22,0xa1,0x95,0x5a,0xa2,0x08, +0x69,0x9b,0xfd,0xe3,0xfb,0x5e,0xe2,0xd9,0x7f,0xbd,0xbf,0x09, +0xc4,0xd1,0x59,0x7f,0xa0,0x9b,0xa8,0x90,0x82,0xdc,0xf9,0x0d, +0xbc,0x19,0x7e,0x64,0xbd,0xd8,0x17,0x1b,0x5d,0x2a,0x41,0x51, +0x8b,0x6f,0x98,0x26,0xdf,0x40,0x92,0x7b,0x0a,0x45,0xb4,0xb7, +0xbb,0x4b,0x97,0x23,0x52,0xa5,0x76,0xc7,0xd1,0x97,0x17,0x2f, +0xc6,0x85,0xde,0xe6,0xce,0x87,0x4f,0xfb,0xa3,0x51,0x06,0xe8, +0x34,0x93,0x4b,0x31,0x93,0x0b,0x3b,0x8a,0xf9,0xac,0xef,0x43, +0x63,0x75,0x53,0x1e,0x2b,0xac,0xdc,0xd1,0xe9,0xf6,0x52,0x32, +0xa8,0xcd,0xc3,0x1f,0x60,0x12,0x05,0x7b,0x8c,0xe6,0x4e,0x2f, +0xf2,0x4b,0xd0,0x17,0x99,0x96,0xe7,0x56,0x66,0xd0,0xb8,0xa8, +0xad,0xad,0x72,0x5c,0xd3,0x64,0x5d,0x2c,0xea,0xe7,0xa9,0xc5, +0x8f,0x2e,0x32,0xfe,0x44,0x21,0x98,0x7f,0xb1,0x70,0xd2,0x20, +0x7f,0x92,0x55,0x2e,0x63,0x11,0xf5,0x0c,0x59,0x29,0xd0,0x8b, +0x12,0xb7,0x8b,0x0b,0xbb,0x72,0x4c,0x5c,0xec,0x5a,0x87,0x96, +0x37,0xb7,0x56,0xa0,0x52,0x96,0x4b,0x46,0xce,0x1a,0x3b,0x49, +0xe1,0xb2,0xd1,0xd4,0x3c,0xd1,0x3b,0xb3,0x21,0x08,0x34,0xca, +0x2d,0x18,0xb5,0xab,0x7c,0x93,0x08,0x85,0xd5,0x28,0xe1,0x59, +0x9a,0x98,0x24,0x5c,0x83,0xc8,0x93,0xdf,0xe4,0x3a,0x2f,0x86, +0x19,0x32,0x09,0xa2,0x80,0xa4,0x39,0x03,0x13,0x55,0x15,0xa6, +0xf4,0xcc,0x32,0x90,0xd9,0xb5,0x11,0xcb,0x34,0x88,0x68,0x0d, +0xbe,0xc7,0x41,0x10,0xdd,0xe2,0x00,0x81,0x83,0x97,0x61,0x70, +0x8c,0x9a,0x10,0x26,0x6c,0x66,0x9b,0x9b,0xfe,0x08,0x4b,0xea, +0xe7,0xcd,0xf9,0xc5,0x25,0x4c,0x1b,0xe8,0x35,0xb8,0xb9,0x30, +0x10,0x1b,0xe7,0x48,0xf3,0x7c,0x8a,0x7e,0x53,0x9a,0x1b,0x76, +0x5e,0x19,0x12,0x61,0x76,0x81,0x4e,0x79,0x1f,0x9e,0xf4,0x8f, +0x71,0xe2,0x98,0x7b,0x0e,0x08,0xc2,0xd5,0x46,0x30,0xca,0x23, +0xd4,0x34,0xd3,0x7b,0xfd,0xe4,0xec,0x57,0x54,0x4f,0xaf,0x24, +0xbe,0x04,0xa6,0xb9,0x38,0x1f,0x4f,0x2c,0xe5,0x0d,0xd9,0xae, +0x5d,0xc5,0xb1,0xcd,0x6c,0xeb,0x82,0x76,0x20,0xcb,0xf5,0x48, +0xec,0xe0,0xcd,0x30,0x4c,0x85,0x37,0x4f,0x29,0x79,0xbe,0x73, +0x7b,0x1d,0x0b,0x30,0x6c,0x73,0x67,0xfc,0x6b,0x0f,0x9a,0x9c, +0x0c,0x0d,0x35,0xfa,0x1d,0x9d,0x19,0xa8,0x98,0xd7,0x92,0xc5, +0x02,0x4e,0xd4,0x09,0x0f,0x6d,0xdb,0x99,0xc0,0xb0,0x0f,0xa7, +0x45,0xa2,0x4a,0x5b,0x5f,0xe6,0x43,0x0a,0x43,0x55,0xa5,0x6d, +0xb3,0xaa,0xd8,0x21,0xcd,0x35,0x2a,0x54,0xd8,0x34,0xfc,0x4d, +0x17,0x00,0x6d,0xe7,0xe2,0x96,0x3c,0xf8,0x2a,0x56,0x91,0xda, +0x99,0x2f,0xac,0x75,0x79,0x2b,0x18,0x37,0x08,0x1b,0xed,0x2a, +0x35,0x94,0x2a,0x51,0xda,0x32,0x86,0xdb,0x95,0x69,0x7f,0x38, +0xdb,0xce,0x26,0x70,0x62,0xd1,0x9c,0x36,0xd9,0x14,0x16,0x5b, +0x24,0xaf,0xd3,0xde,0x9e,0xe8,0xf1,0x3a,0x3c,0xf3,0xf4,0x2a, +0x40,0xdb,0x8d,0x81,0x34,0xbb,0x0b,0x23,0xb0,0xdb,0x28,0xcd, +0x4b,0x4d,0x90,0x5e,0xfb,0x32,0x7f,0x63,0x36,0x85,0xe1,0x93, +0x05,0xbf,0xc4,0xb6,0x63,0x58,0x74,0x3e,0x5b,0xcf,0xc5,0x82, +0x22,0xe0,0x41,0x8f,0xad,0x0f,0x1c,0x19,0xc5,0x1e,0x3f,0x18, +0xe4,0xf0,0xd1,0xa7,0x55,0xe6,0xc0,0xf2,0xaf,0x8c,0xb1,0x8a, +0xf3,0xac,0xe5,0x0c,0x6b,0x70,0x59,0x2e,0xc6,0xb5,0xea,0x92, +0xdc,0xa3,0x7b,0xcc,0x4e,0x76,0xab,0xb9,0xb9,0xf4,0x96,0xb8, +0x92,0x34,0xfa,0xf1,0xd9,0x71,0xb4,0x54,0x45,0xff,0xf7,0xa1, +0x8b,0x03,0x86,0x46,0x47,0xfe,0x06,0x87,0x06,0x1f,0x52,0xf1, +0x94,0x83,0x63,0x31,0x4b,0x8e,0x1c,0xc0,0xc1,0xb3,0x97,0xcf, +0x8e,0x9f,0xe9,0x8c,0x11,0x3d,0x23,0x88,0xca,0x49,0x4b,0x05, +0x43,0xd3,0x9e,0xd1,0x50,0x00,0xc5,0xb3,0xd5,0x46,0x72,0x98, +0xeb,0x6f,0xa3,0x29,0xa8,0x60,0x72,0xda,0x38,0x08,0x28,0x37, +0x0c,0x32,0xb7,0x7e,0x98,0x28,0x76,0x83,0xef,0x7a,0x42,0xaf, +0x21,0xba,0x3d,0x81,0x61,0xe2,0xa6,0xe7,0xeb,0x46,0x8c,0x83, +0xd4,0xc4,0xf2,0xe9,0xc8,0x49,0xba,0x81,0x99,0xdd,0xca,0x7d, +0x22,0x15,0x16,0x8c,0x70,0xb3,0xe0,0x61,0x43,0x9d,0xf0,0xce, +0xa8,0x4a,0xc3,0x91,0x8b,0x8e,0x56,0x20,0x08,0x11,0x8f,0x61, +0xe6,0xf0,0xa2,0xca,0xc0,0xa2,0xfe,0xd9,0x85,0x3d,0xc7,0x28, +0x77,0x3c,0xc1,0x78,0x79,0x5a,0xb1,0xbb,0xa7,0x8e,0x64,0xfb, +0xdd,0x3b,0x27,0xef,0x77,0xc4,0xd9,0x19,0x09,0x49,0x6e,0x37, +0x85,0x47,0x8e,0xe8,0x97,0x13,0x70,0x29,0x1e,0x25,0xc4,0x98, +0x12,0xc4,0xd6,0x16,0xfe,0x7a,0xae,0x3b,0xa2,0xf8,0x97,0x45, +0x57,0x03,0x9f,0x9c,0xf4,0x12,0x7d,0x04,0x43,0x80,0x46,0x14, +0x9f,0x9c,0xbc,0x5f,0xdc,0x4b,0x4c,0x76,0xd8,0xd7,0xf4,0x4c, +0xb3,0x54,0x11,0x1b,0x05,0xcf,0x5b,0x60,0x9f,0x85,0x8a,0xcd, +0x51,0x0c,0xc3,0xa7,0x3c,0x7d,0xec,0x87,0xe3,0xa0,0x3e,0x22, +0x9a,0xf9,0xe7,0xf6,0x4c,0xf9,0x3c,0x57,0xd4,0xd3,0x55,0xe6, +0xd4,0xb6,0x05,0x2e,0xdd,0x9c,0x35,0x7c,0x55,0x61,0x61,0x3a, +0x94,0xde,0x35,0x11,0xbd,0x94,0x1e,0x13,0xf1,0x1c,0x71,0xf1, +0x62,0xab,0x31,0xda,0xe3,0x70,0xdc,0x25,0x7d,0x12,0xf2,0x7b, +0xd4,0x89,0x5d,0xc7,0x65,0x95,0xa7,0xf8,0x52,0x5d,0x65,0x4b, +0xea,0xe6,0x33,0xa5,0xb9,0x16,0x54,0xd1,0x25,0x78,0x31,0xb7, +0x78,0x1b,0xd1,0xbd,0x16,0x18,0x40,0x5d,0x85,0x28,0xde,0xe9, +0x24,0x2b,0x90,0xd9,0xa6,0xc2,0x0b,0x73,0x35,0xd2,0x5b,0x4b, +0x3e,0xbe,0x3c,0x1a,0xf5,0x8b,0xab,0xa1,0x5e,0xc5,0xfa,0xb9, +0x95,0x36,0x44,0x3b,0xfa,0x68,0x07,0x11,0xe2,0x64,0x07,0xc1, +0xd4,0xcf,0xd3,0x80,0xb8,0x65,0x75,0x57,0xee,0x34,0xee,0xed, +0xa0,0x23,0xf6,0x8a,0xa4,0x93,0x66,0xdc,0xc9,0x4e,0x3e,0x36, +0xe2,0x7b,0x8b,0x93,0x4e,0x92,0x68,0xb8,0xa6,0x06,0xa4,0xd9, +0xd1,0xac,0x82,0x3f,0x39,0x69,0x6a,0xa0,0x1d,0x80,0x32,0xc3, +0xe0,0xa6,0xfe,0x64,0xeb,0x8f,0x10,0x64,0xf1,0x16,0x0b,0x9e, +0x7c,0xcc,0x85,0x65,0x5e,0x08,0xc7,0x97,0x17,0x23,0x8e,0xbd, +0x28,0x0d,0x54,0xc1,0xb2,0xbb,0xec,0x39,0xe9,0x1b,0xfd,0x34, +0xad,0x64,0xaf,0x46,0x34,0x15,0x08,0xa4,0xec,0xc1,0x95,0x9d, +0x0d,0xc6,0x37,0x46,0xa9,0x5d,0xf3,0x8f,0x4e,0xd6,0x42,0x2b, +0x3b,0x3c,0x24,0xe7,0x4f,0x21,0x20,0x5c,0xaa,0x5b,0xbb,0xe4, +0xa4,0x94,0x1d,0xf4,0x26,0x09,0x83,0x3b,0x19,0xdf,0xc3,0xd5, +0x49,0xcf,0x80,0xdf,0x73,0xd2,0xb6,0xb7,0x29,0xb0,0x92,0xf1, +0xd8,0x81,0x4f,0xb8,0x3a,0x30,0x62,0x08,0xbd,0xa7,0x8f,0x8b, +0x1c,0xc5,0x9e,0x10,0x8b,0x6f,0x76,0x5f,0xb8,0xc8,0x43,0x1c, +0x41,0x9a,0xc8,0xec,0x76,0x32,0xcd,0x2f,0x73,0xdc,0x62,0x66, +0x57,0x29,0xa2,0x99,0x0e,0x2f,0x87,0x9f,0x6e,0xf0,0x7b,0x49, +0xca,0xed,0x1a,0xb0,0xf9,0x81,0xed,0x90,0x71,0xcd,0x20,0x35, +0x83,0x3f,0xae,0xa3,0xe3,0x6e,0x9c,0x34,0x7b,0x09,0x2c,0x01, +0x27,0x27,0x30,0xda,0x45,0xd2,0xc9,0x4e,0xd2,0x49,0xc1,0x83, +0x6f,0x12,0x77,0x4f,0x3a,0x8b,0x93,0x07,0xbd,0xa4,0xa3,0x01, +0xc5,0xec,0x2f,0x60,0x4c,0xa2,0x02,0x84,0x79,0xdc,0xe0,0x94, +0x23,0xf0,0x7d,0x98,0x53,0x07,0x63,0x0d,0x57,0x52,0x17,0x01, +0xcd,0x99,0x2c,0x7a,0x50,0x9d,0xda,0x76,0xf6,0xfc,0xf8,0x06, +0x8a,0xdc,0x7f,0xbb,0x62,0x40,0xf0,0xbb,0x69,0x3e,0xe1,0x45, +0x07,0x90,0x91,0xe1,0x5f,0x54,0xee,0x8e,0xa2,0x46,0x6c,0x92, +0x41,0xf9,0x17,0x53,0x12,0x3d,0x35,0x3b,0xa9,0x4c,0xc1,0xe8, +0x54,0xcf,0x97,0x46,0x0c,0x04,0xe9,0x83,0x0b,0xd8,0x9e,0xec, +0x44,0x1a,0x47,0xdc,0xfd,0x65,0x07,0x1c,0x14,0xeb,0xb4,0x28, +0x11,0x79,0xd8,0xcd,0x87,0x6c,0xbf,0x93,0x9d,0x7b,0xd0,0x32, +0xae,0x05,0xa1,0x8b,0x9a,0xd4,0x1f,0x99,0xb7,0xca,0xc3,0x8a, +0x3d,0xbb,0xd2,0x8b,0x0a,0x38,0x5a,0xb1,0xdd,0xda,0x89,0xf2, +0x08,0x88,0x80,0x6c,0x4e,0x58,0x82,0x9b,0x09,0x0a,0xc3,0xc0, +0x64,0xaa,0xe4,0xb5,0xa0,0xb3,0x01,0x02,0xcf,0x13,0x04,0xdb, +0x85,0xd8,0x9e,0x95,0xa0,0x4c,0x87,0x60,0x35,0xfa,0xf5,0xf8, +0x68,0xd8,0x9f,0x9e,0x5f,0x11,0x33,0x00,0x70,0xb8,0xb3,0xc2, +0xbe,0xe2,0x8f,0x40,0xc2,0x96,0xf0,0xbe,0x4b,0x9c,0x17,0x39, +0x6f,0x80,0xe1,0x95,0xe1,0xba,0x04,0x69,0x58,0x8a,0x33,0x68, +0xd3,0xc1,0x20,0x4b,0x9a,0x76,0x95,0x97,0x98,0xa4,0x54,0xd3, +0x9d,0xa8,0xcd,0x14,0x4a,0x94,0x92,0x52,0x8a,0x3d,0x9e,0xf0, +0xc8,0x15,0xb4,0xc9,0x1c,0x86,0x46,0xc3,0xd3,0x43,0xc3,0xb0, +0xac,0x70,0xa2,0xf9,0xe4,0xe9,0xc7,0x5c,0xba,0xad,0x62,0x76, +0xf3,0x56,0x00,0x63,0x2b,0xa2,0x62,0x91,0x3d,0x06,0x0a,0x1c, +0xf7,0x34,0x1b,0x02,0x77,0x76,0xd3,0xc9,0x84,0x6c,0x74,0x83, +0xfc,0x18,0x6e,0xf9,0xf1,0x86,0x5b,0x41,0x02,0xcc,0x66,0x3a, +0xf4,0x92,0x68,0x39,0x62,0xe7,0xf8,0x13,0xf4,0xf4,0x5b,0xeb, +0x3a,0x20,0x70,0x0a,0x60,0x0b,0x50,0x16,0xbd,0x92,0xc8,0xf5, +0x51,0x40,0x59,0xc4,0xaa,0xda,0x87,0x00,0xfa,0x08,0xb0,0xcb, +0x09,0x09,0x9c,0x70,0xb5,0xc1,0xf5,0x41,0x2f,0x36,0x1a,0x27, +0x0f,0x10,0xe8,0x4b,0x5a,0x0b,0x90,0x57,0x82,0x4f,0x21,0xab, +0x02,0x49,0x13,0x25,0xd0,0x60,0x4d,0xcc,0xcd,0x32,0xcc,0x41, +0x54,0xde,0xcb,0x64,0x32,0xb9,0x19,0x63,0x73,0xa7,0x9b,0xd7, +0x1e,0xb4,0xb3,0x67,0x82,0x3a,0x40,0xd9,0xb5,0x50,0xfb,0xd9, +0xcf,0xdb,0x8d,0x46,0x6e,0x89,0xe1,0x87,0x1b,0x60,0x11,0x09, +0x38,0x86,0x08,0xd6,0xe6,0xf1,0x25,0x6c,0x7d,0x7a,0x45,0x9d, +0x5c,0x6c,0x5c,0x77,0xf3,0x5e,0x67,0x30,0x2c,0x89,0xbb,0x21, +0x3e,0x49,0xe1,0x6f,0x1b,0x05,0x0a,0xbf,0xc3,0xc5,0x56,0xec, +0x76,0x8e,0xe6,0xd8,0xbc,0x1a,0xb1,0xa3,0x84,0x12,0xdd,0xa2, +0x3c,0xbf,0xd1,0xc7,0xc6,0x21,0x49,0x20,0x8d,0xb9,0xfb,0x4f, +0x33,0x32,0x83,0xcc,0xb1,0xa8,0x70,0x99,0x51,0x97,0x34,0xd9, +0x6a,0x3b,0x9a,0x9e,0x23,0x0d,0x59,0xb2,0x2d,0xd6,0xbc,0x47, +0x00,0x7a,0xf5,0x82,0x48,0x13,0xdc,0xda,0xe2,0xd7,0x6a,0x08, +0x03,0x23,0x99,0x3b,0x15,0x81,0x5c,0x58,0xb3,0x31,0x9b,0x08, +0xe2,0xcf,0x53,0x1d,0xab,0x9b,0xf1,0x7c,0xf8,0x16,0x63,0x49, +0x09,0xd9,0xec,0x9c,0x8c,0x11,0xb1,0xe1,0xf9,0x57,0x24,0x7a, +0x03,0x28,0x51,0x6e,0x8c,0x35,0xc1,0x4b,0x76,0x7f,0xa0,0x77, +0x11,0xb0,0xef,0x8e,0x40,0xef,0xb0,0x05,0x22,0xa4,0x08,0xce, +0xd2,0x80,0x77,0xb1,0xc0,0x90,0x6e,0x4c,0x51,0x3c,0xba,0x1f, +0x5d,0x85,0x89,0x8c,0xce,0x1f,0x81,0x06,0x64,0x84,0xa6,0xad, +0x08,0xa7,0xf2,0x9b,0x0e,0x09,0x57,0xd4,0x7e,0x5c,0x65,0x33, +0xe1,0xd1,0x3d,0xe0,0x11,0x76,0x7e,0x1c,0x24,0x25,0x1d,0x3b, +0x3b,0xb0,0xd9,0xe4,0x85,0x68,0x08,0xab,0x44,0xd3,0x24,0x49, +0xb3,0xc0,0xa6,0x8c,0x65,0xa4,0x5d,0xab,0x93,0xd4,0xa1,0xd5, +0x9c,0x4a,0x88,0xa9,0xd4,0x6d,0x01,0x11,0x89,0x88,0xe0,0x62, +0x92,0x44,0x60,0xc7,0x73,0x3c,0x2d,0x33,0x60,0x4e,0x21,0x10, +0x1e,0x25,0xc6,0xfa,0xa3,0x6f,0xb9,0x41,0x49,0x33,0x0c,0xac, +0xed,0xe7,0x4e,0x96,0x4c,0x21,0x9e,0xcb,0xed,0xf1,0xd6,0x30, +0x63,0x04,0x5d,0xc9,0x8c,0x39,0x4d,0xa6,0x5e,0x66,0x1b,0x99, +0x18,0xab,0x8e,0x5d,0x5e,0xf0,0x25,0x26,0x45,0xa6,0x2e,0x32, +0x1f,0xff,0x3e,0xf9,0xe0,0xf4,0x98,0x94,0x63,0xc0,0xed,0x35, +0x0e,0x52,0x65,0x42,0x49,0x47,0x9c,0xf3,0x6d,0x24,0x78,0x86, +0x0d,0x6f,0x7d,0xbc,0x4e,0x81,0x97,0xbd,0x21,0x62,0xb0,0x62, +0xe1,0xe1,0x7e,0x47,0x27,0x99,0x8a,0x02,0x38,0x45,0xe6,0x14, +0xdf,0x7e,0x49,0xb2,0x89,0xb3,0xc0,0x81,0x97,0x74,0xce,0x21, +0x71,0x57,0x54,0x9b,0x8b,0x44,0xd4,0x23,0xf4,0xd5,0x70,0x5c, +0x5d,0xba,0x2d,0x2f,0x73,0x2e,0x3b,0x64,0x72,0x9d,0xeb,0x8e, +0x3a,0x61,0x63,0x78,0x0d,0x03,0xa2,0x2f,0xcd,0x72,0x56,0xd1, +0x98,0x58,0x2d,0xb6,0xe6,0xbd,0x52,0xfb,0xe3,0xf8,0xec,0x8f, +0x46,0xb1,0x79,0x9e,0x55,0x2a,0x93,0x13,0x6e,0x71,0x55,0x0c, +0x16,0x41,0xae,0xba,0x6b,0x01,0x1c,0xaa,0x3c,0x06,0x71,0x49, +0x92,0x6b,0x94,0xb7,0x24,0xdd,0xb1,0x26,0xf1,0x4a,0x42,0xa2, +0x1c,0xb9,0x96,0x48,0x3b,0xf0,0x2c,0x53,0xaa,0xa6,0x6c,0x0d, +0xf4,0x28,0x2b,0x13,0xc8,0x59,0x64,0x04,0x46,0x94,0x04,0x67, +0xee,0xf6,0x81,0x5b,0x76,0x53,0x0e,0x95,0x21,0x1d,0xf6,0x60, +0xf6,0x11,0x5b,0x23,0xe4,0x7f,0xb4,0xe3,0x6e,0xb2,0xbf,0x4d, +0xde,0x98,0xb2,0x8a,0xed,0xb9,0xb4,0xac,0x30,0x6f,0xa3,0x33, +0x91,0x8a,0xbc,0x39,0x9c,0x10,0xcf,0xc6,0x97,0x3f,0xa9,0x5b, +0x15,0xca,0xcb,0x90,0xe5,0x69,0xdc,0xea,0xc5,0xd7,0x33,0xc6, +0x20,0xb6,0xdd,0xb2,0xa8,0x30,0x13,0xbd,0x58,0x30,0x7b,0x06, +0xbb,0x38,0xfa,0x1d,0x17,0x45,0x73,0xac,0xa5,0xe0,0x76,0x29, +0xf1,0xdf,0x2e,0xe5,0x03,0x64,0xb9,0x42,0xd3,0x8e,0x6e,0x68, +0xba,0x15,0xa2,0x3c,0x77,0x12,0x31,0xcd,0xc8,0xc0,0xc8,0x56, +0x1b,0x7f,0x67,0xa9,0x3c,0x36,0x17,0x74,0xa0,0x57,0x39,0x2e, +0xa0,0x68,0xbc,0x32,0x61,0x01,0x1f,0x1b,0xf5,0xba,0xa4,0x47, +0x11,0xf8,0xd0,0x91,0xf4,0x86,0x30,0x06,0x9b,0x3b,0xe3,0x80, +0xb1,0x05,0x4e,0x87,0x99,0xe6,0x83,0x64,0x07,0xd5,0x84,0x33, +0x09,0xdb,0x6d,0x01,0x75,0x0e,0xb3,0x38,0x94,0xa2,0x24,0x38, +0x2c,0xf2,0xd0,0x2a,0xf3,0x03,0xf3,0xce,0xa6,0x39,0x44,0x26, +0xf2,0xd4,0x8b,0xd9,0xc8,0xd3,0x68,0x44,0xb7,0x51,0xe5,0x5d, +0x16,0xfa,0xe5,0x96,0x9a,0x3c,0xe5,0x71,0x45,0xac,0x81,0xf4, +0x98,0xe2,0x65,0xdb,0xf5,0xa6,0x14,0x98,0x55,0x7b,0x52,0x7c, +0x1e,0x9f,0xc7,0x82,0xc3,0xd1,0xf3,0xdb,0x34,0xb6,0x84,0x45, +0x15,0x1d,0x33,0x80,0x82,0xa9,0x27,0x73,0x33,0x5d,0xcb,0x5e, +0xe9,0x4c,0x8b,0x35,0x14,0x27,0x5b,0xc7,0x41,0x7b,0x9a,0x7c, +0xa8,0xa3,0xb6,0x14,0xd7,0xf7,0xe3,0x4b,0x50,0x93,0x35,0x22, +0x54,0x42,0x5f,0xeb,0x11,0xcb,0x59,0xbf,0x51,0xab,0xac,0x08, +0x86,0x8e,0xb1,0x7c,0x77,0x0b,0x5f,0xee,0x25,0xab,0xce,0xef, +0x42,0x60,0x7f,0x8a,0x94,0xd4,0x6c,0xcd,0x84,0x7f,0x1c,0xa8, +0xe2,0xff,0x75,0x77,0x54,0x21,0xc7,0xcb,0xcb,0x19,0xaf,0xd9, +0x15,0xab,0xa7,0x08,0xf3,0xca,0xee,0x36,0x31,0x0a,0x6b,0xf6, +0x37,0xd8,0x57,0xe0,0x14,0x60,0xb7,0x96,0x15,0xbe,0xa3,0xd8, +0x88,0xeb,0x3a,0x5e,0xa3,0xa8,0xf1,0x14,0x19,0x4e,0xc5,0xb7, +0x1a,0x62,0xef,0xaa,0xb5,0xf4,0x6a,0xdd,0x3d,0x91,0x95,0x7b, +0x5d,0x5c,0xe6,0xfc,0x3e,0x21,0x2a,0xe1,0xee,0xc9,0xb4,0x57, +0xc8,0xe2,0x83,0x0e,0xb6,0xec,0x0c,0xd3,0x1a,0xc5,0xfa,0x0e, +0xab,0x9a,0xd2,0x1c,0x99,0x69,0xad,0x42,0x62,0x43,0x1c,0xe4, +0x9b,0xde,0x82,0x66,0x62,0xa8,0xc7,0x65,0x4c,0x4e,0xd7,0xbc, +0x94,0xf4,0xa4,0x08,0x46,0x56,0xb7,0x0a,0x06,0x2e,0x5a,0x2d, +0x59,0xdc,0xb4,0xec,0x20,0x55,0xfa,0xe9,0x94,0x5a,0x95,0xe5, +0x0c,0xae,0x01,0xbf,0x20,0xef,0x12,0xba,0x22,0x70,0xaf,0x55, +0xe9,0x07,0x0b,0x5d,0xa2,0x07,0x4e,0xb0,0x02,0xc0,0xdf,0x3d, +0x5f,0x58,0x2a,0x70,0xaf,0xb5,0xe4,0xa7,0xe6,0xa5,0xa1,0xb4, +0xbc,0xcb,0xd7,0x5c,0xad,0x8f,0xbb,0xb6,0x34,0x00,0x54,0xc5, +0xa6,0xd0,0xcc,0x4a,0x94,0x99,0x61,0xa0,0xbc,0x48,0x3d,0x1b, +0x5e,0x80,0xd0,0xa1,0xd0,0x5d,0x94,0x60,0xb0,0xe6,0xfa,0xa4, +0xee,0xb2,0xc5,0xb2,0x7a,0x59,0x49,0x30,0x7e,0xc7,0x05,0x8b, +0x61,0xff,0x45,0x1a,0x45,0xb4,0x03,0xb5,0x45,0xe5,0xdd,0xfa, +0x98,0xc0,0x6c,0xf2,0x5f,0xc5,0x64,0xcc,0x6e,0x35,0x89,0x8d, +0x41,0xd5,0xbc,0xac,0x42,0x53,0x0f,0x52,0x9d,0x9a,0xde,0x8d, +0xd9,0x21,0xb8,0xfd,0x22,0xe5,0xcb,0x44,0xc1,0x27,0xa9,0x40, +0x58,0x82,0x37,0xd2,0x96,0xaa,0x5d,0x07,0x4d,0xd7,0x8b,0xbc, +0x03,0xe7,0x04,0x97,0x76,0x19,0x70,0xdb,0x23,0x76,0x9b,0xc4, +0x0f,0x36,0xf9,0xb8,0x43,0x22,0x9f,0xf9,0xec,0x2e,0x27,0x8f, +0xe7,0x62,0x80,0x3b,0x3b,0xb7,0xbd,0x2f,0x15,0x46,0xd7,0xea, +0x49,0x96,0xdb,0xf2,0xe9,0x64,0xf2,0x21,0x07,0x5b,0xfc,0x65, +0xa5,0xc7,0x73,0x97,0xe4,0xcb,0xab,0x34,0xe7,0xfb,0xb1,0xf0, +0x1d,0xe4,0x0a,0x71,0x13,0xa7,0x4a,0x41,0x34,0xf0,0x32,0xbc, +0x72,0x83,0xed,0x17,0x3a,0x3f,0x32,0x1c,0x13,0xc0,0xc3,0x1b, +0x95,0x42,0xd0,0x57,0xab,0x31,0x9f,0xcf,0xe9,0x89,0x54,0x24, +0x67,0xca,0xe8,0x9e,0x56,0xb3,0xb3,0xa6,0xcc,0x26,0x27,0x20, +0x02,0xd1,0x08,0x1e,0x52,0x86,0xe1,0xb7,0x47,0x1e,0x2a,0x15, +0x0e,0x5d,0xcf,0x8d,0xfb,0x26,0x09,0x61,0x9e,0xca,0x1c,0xb0, +0x6d,0x56,0x16,0xce,0x1a,0x03,0x5c,0x2b,0xb8,0x12,0x4b,0xcb, +0x37,0x4d,0x25,0x74,0x49,0xc7,0xa3,0xa7,0x5c,0x5e,0xca,0xdc, +0x9f,0x07,0x06,0x7b,0x0a,0x17,0x94,0xdc,0xda,0x12,0x5b,0x41, +0x0b,0x67,0x25,0x42,0xab,0xdb,0xc5,0x43,0xad,0xeb,0xec,0x63, +0xb1,0xd5,0x96,0xfd,0x51,0x05,0x92,0x74,0xaa,0x48,0x4d,0xfd, +0xfa,0x55,0x65,0x94,0x6d,0x46,0x2b,0x1d,0x1a,0x2c,0xf7,0x47, +0x0b,0xa3,0x01,0xf5,0x27,0xd1,0x46,0xa8,0x0d,0x35,0x1f,0x3f, +0x03,0x71,0x9f,0xae,0x7f,0xd5,0x15,0xb1,0x25,0x5a,0x2c,0x53, +0x36,0xce,0xb2,0xb1,0xa6,0xa5,0xfa,0x83,0xc1,0x1b,0x3d,0x1d, +0x9f,0x8f,0xc3,0x57,0x99,0xbc,0xba,0xaf,0x68,0xda,0x32,0xd1, +0x9b,0x99,0x9f,0xcb,0xf8,0xcd,0xf0,0x7b,0xc9,0x87,0xa1,0x03, +0x6f,0x10,0x27,0xa7,0x45,0x25,0x80,0xe9,0x4a,0xd3,0x18,0xfe, +0xc1,0x95,0x9e,0x99,0x82,0x8e,0x4b,0x9c,0xb8,0xe6,0xf2,0x19, +0x7d,0x7a,0x1e,0x00,0x13,0x37,0x31,0xd8,0x96,0x15,0x8b,0xc3, +0x91,0x0e,0x91,0x52,0xb4,0x5d,0x2c,0xa4,0x1d,0x4e,0xa2,0xc2, +0x70,0xd3,0xa0,0x6e,0x60,0x53,0xf9,0x6d,0x9d,0x71,0x10,0x67, +0x80,0xe9,0x09,0xa2,0x54,0x6c,0xe9,0xd8,0x1d,0x6a,0x3a,0xb9, +0x86,0x2d,0xc2,0x48,0x92,0x48,0xdd,0x54,0xdd,0xcc,0x7d,0xac, +0xc6,0xbd,0x93,0x87,0x31,0xf3,0x77,0x19,0x6b,0x7d,0x8f,0x35, +0x17,0x7c,0xaa,0x78,0xdc,0x7a,0x08,0xe0,0xc1,0xc0,0x5f,0x76, +0x27,0x78,0xdd,0xff,0x30,0x04,0x6b,0x46,0x85,0xf5,0xbf,0xaf, +0x26,0x67,0xbf,0xc2,0xe0,0x25,0x5b,0x69,0x05,0xf8,0xc5,0xa5, +0x13,0xaa,0x8a,0xf8,0x11,0x7e,0xb6,0xdb,0xce,0x1f,0x93,0x5d, +0x33,0xf3,0xee,0x0c,0x6c,0x50,0x9c,0xfd,0xda,0xc5,0xc8,0x6e, +0xde,0x83,0x7b,0x77,0xd3,0x72,0x3a,0xde,0xb1,0xe6,0xc0,0xdf, +0xbe,0xc1,0x67,0x56,0x31,0x7c,0x2a,0xcd,0x5e,0x0c,0x46,0xfe, +0x02,0x0d,0xa9,0x28,0x61,0x85,0x07,0xe6,0xb3,0xfe,0xa5,0xd2, +0xff,0xd0,0x4c,0x17,0x9c,0x80,0xf4,0x42,0xda,0x9f,0x7e,0x26, +0xb7,0xae,0x14,0x6d,0x9b,0xd3,0xf9,0x3c,0xe4,0x14,0xf6,0xad, +0xf0,0xcc,0xd8,0x04,0xe7,0xe8,0x1e,0xbe,0x81,0x6d,0x17,0xb3, +0xfe,0xf9,0x87,0x26,0x0c,0x5f,0xb0,0x5b,0x97,0x8f,0x35,0xdf, +0x36,0xb4,0xa0,0x32,0xb1,0xd7,0x4e,0x90,0xa6,0x67,0xe3,0x01, +0x50,0x04,0xd6,0x63,0x44,0x6a,0xd2,0x36,0x97,0x64,0x94,0x5e, +0x2a,0x6d,0x6b,0x4b,0x82,0xc3,0x5d,0xf7,0x25,0xbd,0xbc,0x0c, +0x90,0x5a,0xa2,0xb1,0x86,0xd9,0xef,0x93,0x7c,0x50,0xc2,0xb5, +0x58,0x6c,0x6e,0x62,0x32,0x01,0x2d,0x16,0x84,0x1a,0xf7,0x34, +0x93,0x1f,0xb9,0x16,0xb4,0x4e,0x4a,0xda,0xdd,0xc5,0xcc,0x0a, +0x6b,0x9f,0x1c,0x1f,0xbf,0x65,0x73,0x03,0xe1,0x0d,0x3d,0xee, +0x13,0x83,0xc9,0xfc,0x6c,0x34,0xfc,0x3f,0xf3,0xc9,0x0c,0xf5, +0x00,0xc0,0x0a,0x12,0xbe,0x09,0x93,0x31,0xf3,0xf1,0x6f,0x2e, +0x24,0x67,0x4d,0x29,0xb3,0xa6,0x2e,0xcc,0x0d,0x0a,0x97,0xbf, +0xc9,0xa0,0x66,0x39,0x91,0x54,0x5e,0x82,0xe9,0x02,0xe4,0xd9, +0x78,0x96,0xcf,0x60,0xe1,0xb2,0xe2,0x58,0x1e,0x25,0xe4,0x98, +0x62,0x6b,0xcb,0x0b,0x9a,0x8a,0xb3,0x3d,0x69,0x1a,0x1f,0x81, +0x78,0x8b,0x5b,0x59,0x0c,0x26,0x33,0xd6,0x6f,0x26,0x05,0x8c, +0xe6,0x75,0x46,0x13,0x0e,0x1b,0x80,0xe7,0xfe,0xe4,0x4b,0xbf, +0xb6,0x8e,0xda,0xcf,0x76,0xb9,0x9b,0xbb,0x3a,0xa4,0x37,0x2a, +0xce,0x03,0x69,0xdb,0xdb,0x78,0x75,0x44,0x50,0xf4,0xd2,0x35, +0x0f,0x51,0xe4,0xfb,0x99,0x4e,0x6f,0xe7,0x1a,0xd6,0xd4,0x4d, +0xf3,0xc5,0xae,0xa2,0x20,0x02,0x23,0xf4,0x68,0xd7,0x5e,0x64, +0x86,0x7c,0xb4,0x31,0x19,0xef,0x26,0xba,0xbe,0x24,0xe5,0xa2, +0x61,0x0a,0x6f,0xe5,0xf1,0x76,0x07,0xa6,0x1b,0x4e,0x5f,0x37, +0x1c,0xcb,0xa2,0x8b,0x0d,0x2a,0xc5,0x27,0xaf,0xb7,0x6c,0x63, +0x66,0x9c,0x75,0x58,0x00,0xac,0xd0,0xfe,0xfc,0x41,0xaf,0x47, +0x7a,0x0e,0x54,0x4e,0xa0,0x04,0xb2,0x23,0x01,0xd5,0xba,0x24, +0xcd,0x07,0xc9,0xe3,0x13,0xb0,0x3f,0x0c,0xfa,0x17,0xc5,0x83, +0xa8,0x21,0x73,0x37,0xa2,0xee,0x2f,0xfb,0xbd,0x07,0xfb,0x7a, +0xf9,0xc9,0xa5,0x88,0xac,0x3f,0x62,0x0b,0xce,0xf6,0xc2,0x13, +0xa4,0xa4,0x33,0xba,0x3b,0xa0,0x42,0x9e,0xbe,0x3e,0x3c,0x7c, +0xf6,0xea,0xd8,0x0c,0xf9,0xc8,0xbb,0xe8,0x7f,0x7a,0xf0,0xe4, +0xf8,0x89,0x97,0x64,0x87,0x19,0xd6,0xd1,0xb5,0x3e,0x06,0xe3, +0x60,0x6c,0x62,0xd1,0x60,0xda,0x93,0x54,0x6c,0xeb,0x97,0x07, +0xf3,0x96,0x1c,0x84,0x78,0xd8,0x04,0xd8,0x51,0xaf,0x2f,0xe2, +0xe8,0xf1,0xe6,0xf6,0x76,0x94,0x74,0xd8,0x70,0x9c,0x9f,0xb6, +0xbd,0xbd,0xaf,0x09,0xc2,0x20,0x0e,0xae,0xd8,0x12,0x33,0xb9, +0x26,0x11,0x49,0x10,0x81,0xcb,0x29,0xdf,0x4b,0x03,0xb3,0xf8, +0x88,0x3d,0x84,0x81,0x8e,0xb1,0x69,0x7c,0x97,0x4a,0xa6,0x08, +0x1e,0x26,0x8a,0xbb,0x13,0x9e,0x4c,0xfc,0xf0,0xec,0xc7,0x17, +0xaf,0x7e,0x3c,0x7d,0xf6,0xea,0xe0,0xf4,0xf8,0xc9,0x8f,0xcf, +0xb8,0x69,0x48,0xbb,0x0a,0xb2,0x6b,0x4a,0x49,0x52,0x8b,0xb8, +0x48,0xb4,0xc8,0xc0,0xc6,0x86,0x89,0x32,0x72,0xe0,0xaa,0x32, +0xc9,0x76,0xc6,0x6e,0xcf,0x3e,0xc1,0xb5,0x11,0xa6,0x4b,0x7c, +0x6c,0xb2,0x59,0xcb,0x84,0x0a,0x40,0x41,0xa2,0x95,0x26,0x0b, +0x1a,0x8f,0x8e,0x9f,0xbc,0x3d,0xfe,0x0b,0xa9,0x0c,0xf1,0x29, +0x6f,0xc7,0x92,0x94,0xd2,0xa7,0x9e,0x36,0x15,0x1d,0xfc,0x58, +0x6f,0xa6,0xff,0x41,0x77,0x62,0x7a,0xd8,0xee,0x92,0x89,0x88, +0x0e,0xc0,0xa4,0x01,0x45,0xbb,0xdc,0x93,0x38,0x09,0x2d,0x64, +0x14,0x85,0x70,0xec,0x11,0xee,0xcb,0x07,0x31,0xbc,0xa2,0x40, +0xdc,0xc8,0xe2,0x1b,0x25,0xec,0x42,0x9f,0xf8,0x66,0xf9,0x1f, +0x52,0x09,0xdb,0x3c,0x32,0x3b,0xbe,0xd2,0x8c,0x04,0x27,0x4f, +0x37,0x3e,0xf6,0x8b,0x0d,0xbd,0xea,0x9e,0x8d,0x50,0xd9,0x1a, +0x61,0x36,0x66,0x1a,0xe2,0x62,0x02,0xfe,0xd2,0x34,0x65,0x1b, +0xb8,0x0d,0x83,0x81,0x1a,0xac,0x1e,0x28,0x61,0x47,0x38,0x2a, +0x89,0x31,0xc5,0x86,0x59,0xca,0x39,0x94,0xc8,0x27,0x3b,0x15, +0x3b,0x82,0x75,0x97,0x99,0x0f,0x06,0xc3,0xf1,0x9b,0xe9,0x50, +0x37,0xeb,0x78,0x38,0xfd,0xe9,0xf8,0xf0,0x25,0x6b,0xa1,0x5b, +0x45,0x95,0xc7,0xa0,0xa0,0xb2,0x35,0x9a,0xb5,0x61,0x7a,0xfb, +0xe0,0xc7,0x78,0x0b,0xec,0x22,0xa1,0x31,0x58,0x20,0x03,0x1b, +0x53,0xf8,0x60,0xad,0x86,0x86,0xa0,0xbc,0x2d,0x2c,0xaf,0x7f, +0x7d,0xd3,0x16,0x4b,0xcd,0xab,0xd7,0xaf,0x4e,0x9f,0xbc,0x7c, +0xf3,0xd3,0x93,0x57,0xef,0x0e,0x9f,0xbd,0x7d,0xf1,0xb4,0xb4, +0x07,0x7b,0x28,0xa3,0xad,0xff,0x64,0x1f,0x23,0xd8,0x69,0x4f, +0x75,0xd9,0xa8,0x16,0xd7,0x88,0xda,0x91,0xd4,0xc1,0x11,0x55, +0x73,0x91,0xfb,0x18,0x79,0x09,0x91,0x3e,0xd3,0x75,0xc4,0xfd, +0xf5,0x7e,0x9a,0x83,0x24,0xf1,0x6c,0x7e,0xc1,0x5b,0xe0,0xe5, +0x58,0xb3,0xcd,0x68,0x07,0x61,0x3e,0xb3,0xbb,0x1f,0x28,0x0a, +0x03,0x8c,0xd2,0xff,0x90,0xbd,0xb0,0xfe,0x7e,0x71,0xcf,0x75, +0x4c,0x2a,0x6c,0xa9,0x72,0xe3,0xbd,0xd5,0x11,0xd5,0x7b,0x28, +0x78,0x99,0xc7,0xb2,0xca,0x9b,0x85,0x4e,0xc5,0x53,0x1c,0x91, +0x82,0xb0,0xf4,0xbd,0x58,0xb4,0x36,0xa1,0x47,0x7d,0x56,0x08, +0x54,0xec,0x34,0xb1,0x38,0x87,0x80,0x6a,0xc2,0x1e,0x08,0x8f, +0xc8,0x6d,0x7a,0xe5,0x3d,0x31,0x1e,0xe3,0xe0,0x1e,0xa4,0x4c, +0x27,0x5a,0x2a,0x31,0x65,0x3e,0x41,0xce,0x64,0x44,0x4a,0x7d, +0xf3,0x69,0x2e,0xc2,0x7a,0x2d,0xdf,0xd4,0xab,0xf2,0x26,0x75, +0x13,0xad,0x33,0xef,0xde,0xbe,0x30,0x2b,0x8c,0x21,0x70,0x83, +0x09,0x44,0x9b,0x1a,0xf0,0x71,0x3f,0x8b,0xee,0xd3,0x57,0xe5, +0xa0,0x62,0x20,0x0d,0x03,0x5c,0x0f,0x7c,0x63,0xab,0x76,0xa2, +0x9d,0xfd,0x28,0x8d,0xf4,0x8e,0xa0,0x8f,0x09,0x9a,0x17,0xf0, +0x9a,0x7f,0x65,0x93,0xaf,0xd7,0x8c,0x3b,0xb2,0x1d,0x31,0x0a, +0x8a,0x02,0x56,0x29,0xcb,0x4c,0x9f,0xc5,0x76,0xa0,0x68,0x1a, +0x70,0x31,0x11,0xca,0xc6,0xbc,0xcc,0x71,0x61,0x15,0xd5,0x23, +0x88,0x84,0xa5,0x85,0xc1,0xda,0x52,0xf5,0x3a,0xd6,0x40,0x80, +0xab,0x0a,0xf3,0xed,0xd8,0x15,0x5c,0x3c,0xb0,0x1f,0xf5,0xf8, +0x14,0xf7,0x6b,0xa5,0xd3,0x45,0xf5,0xc0,0x4f,0x70,0x58,0xdb, +0xeb,0x29,0x15,0xae,0xe8,0xd9,0xce,0x2f,0x8f,0xc9,0x45,0xc2, +0xc7,0x74,0xbb,0xd7,0x48,0x62,0xf2,0x83,0xc0,0x41,0x0c,0x3c, +0xc8,0x00,0xa0,0x93,0x82,0x6e,0x5d,0xf7,0x97,0xa8,0xf7,0x20, +0x42,0x97,0x0b,0xf7,0xbb,0xbf,0xdc,0xef,0x3d,0xb8,0x9f,0x2c, +0x34,0xcf,0x72,0x52,0xe8,0xac,0x49,0x27,0x79,0x40,0xfe,0x9e, +0x77,0xd0,0xef,0xf3,0xfe,0x8e,0xf2,0x77,0x39,0x2e,0x4b,0x73, +0x3e,0xa2,0x40,0xe2,0x78,0x76,0x94,0x60,0xd8,0xb3,0x1d,0x47, +0x4e,0x48,0x00,0xd0,0x07,0x44,0xe8,0xa2,0x98,0x0c,0x8a,0xb9, +0x0f,0x31,0x9a,0x98,0xd8,0x52,0x83,0x9a,0x8c,0xe1,0xfe,0x09, +0x24,0xec,0xa8,0xea,0xed,0xdf,0x92,0xb7,0xa3,0x7c,0x6e,0x2a, +0xdb,0x01,0xf6,0x05,0xbd,0x41,0x68,0x5e,0x45,0x63,0x95,0xfc, +0x14,0x24,0x9e,0x74,0x31,0xe6,0xa4,0x8b,0x30,0xbd,0x1e,0xc0, +0xb8,0x29,0x02,0x4f,0x14,0xe3,0x8b,0xd9,0xcd,0x02,0x94,0x05, +0x8a,0x4e,0x92,0xea,0x06,0xd8,0x59,0x5c,0xf7,0xf3,0xd1,0x6c, +0x92,0x2e,0x66,0xc3,0x51,0xba,0xf8,0xcf,0x64,0x27,0x57,0x35, +0x4b,0x27,0xb4,0xc6,0x2f,0x27,0xff,0xb9,0xfd,0xff,0x2e,0x36, +0x16,0x9b,0xa8,0x85,0x28,0xcf,0x4b,0x99,0x39,0xd4,0x46,0xfd, +0xe9,0xb0,0xaf,0xce,0xa6,0x60,0x70,0x4a,0x5d,0x4d,0x55,0x7e, +0x7d,0xa9,0x3e,0x9e,0x4d,0xa3,0x44,0xf9,0x87,0xb6,0x1f,0xe4, +0x39,0xd1,0xe5,0xd6,0xb9,0xd0,0x62,0x8b,0x1a,0x0c,0xd4,0x60, +0xa6,0x46,0xfa,0xb8,0xa0,0x66,0x67,0x93,0xc1,0x67,0x35,0x1b, +0xa8,0xd9,0xc5,0x64,0x32,0x53,0xb3,0x2b,0xfd,0xdf,0xb0,0xaf, +0x83,0x65,0xac,0x2f,0xbc,0x33,0xa5,0x43,0x3b,0xbd,0x51,0xd3, +0x59,0x09,0xda,0xc2,0xf9,0xc2,0x71,0xb8,0xfa,0x5e,0x85,0x76, +0x55,0x4d,0x82,0x03,0xf0,0x9d,0x98,0xbd,0xcc,0xca,0x35,0xe2, +0x60,0x00,0xc6,0xe6,0x14,0x98,0x7d,0x3b,0x1f,0xe9,0x83,0x56, +0x01,0x4e,0xe6,0x10,0x35,0x1e,0xe6,0xd4,0x79,0x9f,0x8c,0xc0, +0x9c,0xe3,0x2d,0x15,0x5c,0x0a,0xab,0x41,0xae,0x7f,0xf3,0xdf, +0xd5,0x60,0xa4,0xf0,0x7d,0xd6,0x10,0x7e,0x0c,0x1c,0xb4,0x9c, +0x86,0xbb,0x6a,0xa9,0xab,0x3d,0x75,0xf5,0x50,0x5d,0x3d,0x52, +0x57,0xdf,0xa8,0xab,0x6f,0x15,0x3d,0x41,0x52,0x57,0xd4,0xea, +0xd0,0x61,0x63,0xa0,0xe3,0x46,0x69,0x8a,0xe6,0xfa,0x8c,0xfa, +0xbb,0xd2,0xfd,0x78,0xa3,0x91,0x91,0x8b,0x1a,0x55,0x90,0x49, +0x23,0xbd,0x52,0x69,0xee,0x43,0xcd,0x47,0xb0,0x6a,0xf9,0x27, +0xf9,0x2f,0x6d,0x4d,0x57,0x69,0xd5,0x3f,0xd3,0x03,0xa7,0x7f, +0x3e,0x9d,0x8c,0x3f,0x5f,0xab,0x33,0x75,0x36,0xc8,0xf5,0xbf, +0x89,0x3a,0xcb,0x2f,0x71,0x44,0xe5,0x50,0x71,0x70,0x03,0x8d, +0xd5,0xbd,0x18,0xab,0xe1,0xb5,0xae,0x18,0xdc,0xd9,0xe3,0x30, +0x03,0xca,0x3f,0x9c,0x0d,0xd8,0xb2,0x37,0xd6,0xa1,0x3f,0xfd, +0xa0,0x7e,0x53,0xd3,0xf9,0xd9,0x67,0x85,0x43,0x40,0x15,0xaa, +0xd0,0x9c,0x82,0x2a,0xae,0xe1,0x2c,0x53,0xdc,0xf4,0xc7,0x0a, +0x58,0xb9,0x0f,0x43,0xf8,0x99,0x8c,0x2f,0x95,0xe6,0xed,0xf4, +0x3f,0x3d,0xe6,0x72,0x7d,0xbc,0x9d,0xcd,0xd4,0x5c,0xe9,0x25, +0x0f,0xea,0x18,0x6c,0xa0,0x6e,0x68,0x99,0x76,0x01,0xdb,0x05, +0x64,0x18,0x4f,0xcc,0x8a,0x72,0x53,0xc8,0x49,0xe3,0x8f,0x95, +0xa0,0x19,0x6b,0xc6,0x2a,0xa8,0xdc,0xd3,0xb6,0xe8,0x48,0x38, +0xd3,0x87,0x1f,0xe8,0x3f,0x78,0x16,0x03,0x6d,0x74,0x35,0x1d, +0x5e,0xa8,0x91,0xae,0xce,0x60,0x58,0x9c,0xab,0x62,0x7a,0xae, +0xe6,0xc5,0x50,0xb7,0x87,0x21,0x8f,0xb2,0x97,0x69,0x33,0x98, +0x45,0x97,0x60,0x87,0x8c,0xf4,0x36,0xa3,0xff,0xce,0x54,0xff, +0x53,0xae,0x89,0xbe,0xd4,0x93,0x74,0x32,0x55,0x67,0x13,0x74, +0x12,0x7c,0x3e,0x1c,0x8d,0x6e,0xe0,0x2d,0x92,0x6e,0x3d,0xf8, +0xd6,0x8d,0x7a,0x8e,0xdf,0x60,0x37,0x4f,0xc1,0x4d,0x19,0xae, +0x05,0x13,0xfc,0x5b,0xe0,0x1f,0x68,0x76,0xb8,0xf5,0xe8,0x9f, +0xcf,0x40,0xba,0x38,0x1d,0x14,0x38,0x78,0x2f,0x34,0x3b,0xc5, +0xe3,0xb1,0xd0,0xbf,0xf9,0xe5,0xd5,0x0c,0xab,0x32,0xd2,0x94, +0xaa,0x2b,0xc0,0x3b,0x54,0x79,0x01,0xfd,0x8a,0x31,0xf0,0x67, +0xde,0xbf,0x1c,0xaa,0xf1,0x04,0x2b,0x3c,0x9e,0x7c,0x9c,0xf6, +0xc1,0x8e,0xcd,0x48,0xff,0xfb,0x5d,0x81,0x58,0x15,0xff,0x40, +0x69,0xd3,0x39,0xb8,0x28,0x64,0xa3,0xf5,0x78,0xf3,0x02,0x24, +0x16,0x57,0x7d,0x08,0xd3,0x28,0xe8,0xc3,0xe8,0x98,0x5f,0x5f, +0x83,0xa8,0x47,0x07,0x2e,0x87,0x7a,0xa5,0xc9,0x67,0x7a,0x8c, +0x83,0xba,0x28,0xb6,0x9b,0x6e,0x04,0x62,0x69,0x7e,0x27,0x5a, +0x3e,0xe6,0x83,0xd9,0x15,0x8c,0x0d,0xcb,0xd0,0x66,0x75,0x86, +0xa8,0xf4,0x04,0xaa,0x34,0x3a,0x61,0x76,0xca,0x48,0x75,0x7b, +0x09,0xd9,0x27,0x91,0x3b,0xb1,0xdb,0x88,0x1d,0xb3,0x55,0x91, +0x19,0xec,0x7f,0x8c,0x60,0xa7,0x8d,0xe0,0x26,0xd4,0x37,0x88, +0x02,0x3b,0xf6,0xcb,0x17,0xaf,0xfe,0xf7,0xbf,0x4e,0xdf,0xbd, +0x7d,0xe9,0x96,0xf2,0xf2,0x46,0x10,0xf3,0x4e,0x90,0x74,0xba, +0x4f,0xb6,0x7f,0xee,0x6f,0xff,0xb1,0xbb,0xfd,0x7d,0xf3,0xf4, +0x6f,0x0d,0xbd,0xfd,0xfd,0xaf,0xe4,0xe4,0xe8,0x81,0x5e,0xfc, +0x8b,0x93,0xe6,0x49,0xfb,0x44,0x9d,0xc4,0x27,0xc9,0xc9,0xed, +0xc9,0xf2,0xe4,0xf1,0xc9,0x7e,0x6f,0x47,0x1d,0x3e,0x79,0xf1, +0xf2,0xf8,0xb5,0xdb,0x64,0x18,0xcf,0x4e,0x3b,0xd4,0x6d,0x00, +0x6e,0x9f,0x9b,0x16,0xe5,0x1a,0x9b,0x78,0x16,0x12,0xa2,0x84, +0xb6,0x35,0xe4,0x09,0xef,0x4a,0xf4,0xac,0x9e,0xf6,0x3f,0xa2, +0x78,0x81,0x0e,0xd4,0xdd,0x9e,0xfa,0x88,0x1c,0x45,0x56,0xc1, +0x64,0x20,0x7f,0x02,0x0f,0x63,0xe1,0x25,0x10,0xdd,0x05,0xa0, +0xe8,0xa5,0x7c,0xeb,0xc2,0x14,0x80,0xea,0x94,0x85,0x6e,0x52, +0x64,0xc6,0x69,0x6d,0x3a,0xd1,0xea,0xe2,0x99,0xd1,0x0c,0x5b, +0x30,0x69,0x27,0xf0,0x92,0xc5,0x1c,0x54,0x95,0xb1,0x7f,0x99, +0x59,0x57,0xf9,0xf4,0x30,0x2a,0x8b,0xb8,0x35,0xf0,0xdd,0x0e, +0xd8,0x17,0x23,0x25,0x29,0x92,0x1c,0x51,0x6d,0xf8,0x9c,0x08, +0x85,0x59,0x1d,0xf7,0x3c,0x91,0x75,0x69,0xc2,0xf8,0x86,0x67, +0x33,0x26,0x07,0x49,0xe1,0xa2,0x7e,0x24,0x80,0x12,0x1f,0x5d, +0xe9,0x0c,0xed,0x75,0x13,0xbc,0xdd,0xb1,0x19,0x60,0xf6,0x6b, +0x5c,0x09,0x36,0xb7,0x23,0x03,0x0f,0xb8,0x8d,0xf0,0x70,0x6e, +0x7a,0x35,0xa4,0x9d,0x84,0x1e,0x8e,0xc3,0x5b,0xae,0x21,0x2e, +0xb7,0x36,0xe2,0xab,0xee,0x79,0x41,0x8c,0xca,0x71,0x38,0x3b, +0xda,0xd7,0xe1,0xf5,0xed,0x5b,0x09,0x11,0x20,0x7d,0xca,0x8f, +0x94,0xf5,0x60,0xd1,0x03,0xc1,0x85,0x02,0x8f,0x51,0xf4,0x92, +0x98,0x5f,0x07,0x09,0x89,0xb8,0x40,0x6d,0x5f,0x06,0xf3,0xfb, +0x3e,0xd2,0x95,0x87,0xce,0x38,0xb7,0x58,0xad,0xee,0x19,0xbd, +0xdd,0x75,0x26,0x3e,0x4e,0x9b,0x43,0x38,0x18,0x9d,0xa2,0x46, +0x7c,0x2c,0x32,0xa8,0xba,0x9b,0x10,0x07,0xe3,0x99,0x63,0x24, +0x25,0xbf,0x53,0xb9,0x91,0x88,0x04,0xf3,0xbc,0xc9,0xbe,0xe2, +0xff,0x6d,0xae,0x2b,0xc0,0x2f,0xa4,0xf8,0xc0,0x84,0x2f,0x30, +0x9b,0xb3,0xc9,0x4b,0x38,0xae,0x3c,0x85,0xe3,0x4a,0xd2,0x4b, +0xd4,0xa9,0x9e,0x18,0xcf,0xc0,0x3c,0x5d,0x2c,0xf1,0xe9,0xc9, +0xc1,0xb7,0x20,0x5e,0x29,0xa6,0x82,0xe6,0x5d,0x3d,0x96,0x99, +0x17,0x47,0xfd,0x0b,0xd6,0xe3,0x35,0x36,0x2e,0x3a,0x65,0x29, +0x27,0xbd,0x79,0xb7,0x15,0xa0,0x85,0xf4,0x56,0x37,0x23,0xbc, +0x79,0x03,0xed,0x48,0x77,0xb6,0xa1,0x37,0xec,0xeb,0xe4,0x43, +0x2b,0x15,0x29,0xeb,0xa9,0xb2,0x1a,0x9c,0x7b,0xf0,0x5a,0x68, +0xba,0x0e,0x91,0x20,0xf0,0xe0,0xa1,0xe7,0x75,0xa4,0xc8,0x44, +0x92,0x62,0x93,0x82,0xf0,0xc4,0x02,0xde,0xf3,0x0f,0xa3,0x5e, +0xdb,0xab,0x8d,0x18,0x20,0x7a,0x76,0xf5,0xe9,0xc1,0x91,0xed, +0x52,0x7e,0xdc,0x5e,0xc4,0xa2,0x00,0x65,0x01,0xfd,0x26,0x4e, +0x96,0x74,0x39,0x70,0x56,0x4c,0x46,0xa8,0x5d,0x00,0xaf,0x50, +0xc7,0x7a,0x95,0xa4,0xb5,0x17,0x97,0xde,0x9d,0xdc,0x95,0xfe, +0x84,0x01,0x41,0xd5,0xd7,0x29,0x85,0xe1,0x64,0x14,0x02,0x63, +0x08,0x6e,0x6d,0x05,0x48,0x49,0x26,0xc7,0xb0,0xf6,0xc5,0xdb, +0x99,0x26,0x03,0x90,0x9d,0x36,0xfd,0x1b,0x74,0x99,0x98,0x80, +0x68,0xcb,0x8f,0x52,0x34,0x33,0xc0,0x24,0xc2,0x0f,0x8c,0x41, +0x9a,0x07,0xe3,0x38,0x4b,0x52,0x50,0x98,0xae,0xd4,0xbd,0x1d, +0xa2,0x47,0x00,0x77,0xdc,0xb7,0x27,0x6b,0x13,0xd1,0xee,0x59, +0x8f,0x8b,0xe4,0xf7,0x33,0x5c,0x04,0xbe,0x7f,0x7b,0x9e,0x0f, +0x47,0x03,0xfb,0xcc,0x4f,0x44,0xa1,0x2d,0x5b,0x47,0xfa,0x33, +0x01,0x2c,0xc9,0x17,0xf1,0x61,0x15,0x24,0x7e,0x1f,0xd2,0xa7, +0x83,0xa7,0xda,0x4f,0xba,0x1b,0x7d,0x6a,0x4a,0x09,0xf8,0xfe, +0xd0,0xd1,0x74,0x50,0xca,0xe8,0x0b,0x4c,0x4a,0x14,0x95,0x4b, +0x22,0x38,0x9f,0x9e,0x8f,0xf9,0xec,0x0a,0x40,0xcc,0x33,0x70, +0x6f,0x8d,0x13,0x83,0xd7,0x3c,0x81,0xc5,0x07,0xd0,0x75,0x25, +0xb8,0xe1,0x43,0x42,0x85,0x17,0x83,0xa2,0x7a,0x00,0xd9,0xe4, +0xa4,0xb3,0xb9,0x9b,0x86,0x91,0xb2,0x27,0x2c,0x1e,0xa7,0x0f, +0x89,0x51,0xe8,0x50,0xd3,0x2f,0x89,0xbe,0xc2,0x96,0xf6,0x16, +0xb5,0x6c,0x45,0x9a,0x6e,0x6f,0x7c,0xd9,0xbe,0x84,0x9b,0x26, +0xfa,0x9d,0xe3,0x8f,0x79,0xa7,0x0e,0x98,0xaf,0xaf,0x27,0x63, +0xd0,0x2c,0x2e,0x77,0x8c,0x5f,0x8e,0xff,0xd4,0xab,0x45,0xba, +0xc6,0x7b,0xbc,0x71,0x99,0xb5,0xa5,0xe7,0x3d,0x68,0xde,0x43, +0x73,0x7f,0x1a,0xd8,0x6d,0x02,0xb2,0xe5,0x18,0x43,0x87,0x73, +0xd3,0x4d,0x62,0x44,0x14,0x45,0x49,0x7a,0x6a,0x8d,0x5e,0x10, +0x12,0x0b,0x48,0xc8,0x5b,0xa9,0x97,0x8f,0x61,0x14,0xef,0x33, +0x9c,0x26,0x6e,0x1a,0x69,0x3d,0xbe,0x5d,0xb5,0x3f,0x10,0x4c, +0x2f,0xb3,0x7a,0xdc,0x34,0xae,0xb8,0x61,0xa6,0xeb,0x36,0x7b, +0x38,0x98,0x88,0x8f,0x0f,0x67,0x05,0xc5,0xd6,0x4c,0x09,0xce, +0xe2,0x64,0x4e,0x6c,0x1e,0xa3,0x66,0x42,0x30,0xb8,0x39,0x31, +0x30,0xbe,0x75,0xca,0xf7,0xab,0xb8,0x56,0x0e,0x8e,0xa5,0xc6, +0x7a,0xfd,0xfb,0x70,0x3a,0xcd,0x25,0x74,0x18,0x1f,0x2c,0x44, +0x87,0x61,0xb6,0xd2,0x9c,0xc7,0xe7,0xac,0x2e,0xdd,0x6e,0x78, +0xdd,0x1e,0xdb,0x4f,0xb7,0x9c,0x97,0xdd,0x2b,0x0c,0xba,0x31, +0x61,0x8f,0x8d,0xb9,0x05,0xe5,0xf0,0xe8,0x8d,0xdc,0x05,0x78, +0xa8,0xd1,0x34,0x70,0x4f,0xb2,0x4b,0x55,0x72,0x39,0xfc,0x25, +0xa6,0x54,0x62,0x16,0x8c,0x33,0x25,0x2b,0x43,0x4f,0x06,0x78, +0xad,0x5a,0x2c,0x6a,0x0a,0xe3,0x4a,0x6d,0xfa,0x73,0xe4,0x94, +0x0c,0x1a,0x12,0x0a,0xe1,0xd7,0x6c,0x2c,0x5c,0x2b,0x8e,0x87, +0xfe,0x4e,0x0b,0x5e,0x6e,0x2a,0x78,0x1c,0x60,0x0c,0xc4,0xe3, +0xf3,0xa7,0x70,0x2e,0x9b,0x4c,0xb3,0x52,0xcc,0x62,0x11,0xc1, +0x9b,0x83,0x48,0x74,0xda,0x3b,0x07,0xee,0x36,0x0f,0xbc,0xc8, +0x86,0xc3,0xcb,0x29,0x18,0x1f,0x88,0x4b,0x78,0x58,0xf3,0x8f, +0x2c,0x84,0xf1,0xc5,0x0e,0xdc,0xbc,0xe2,0xbb,0x90,0x38,0xd2, +0x47,0x88,0x0d,0x78,0x67,0x6a,0xbd,0x15,0x6d,0xe4,0xc5,0xf8, +0xfe,0x6c,0x03,0xcf,0xe5,0x51,0xd8,0xc7,0x82,0x60,0xd4,0x16, +0xf0,0xba,0x43,0x30,0xc8,0xfe,0x0e,0x54,0x4a,0xd0,0xd3,0x0d, +0xbc,0xf1,0xe8,0x02,0xe8,0x09,0x48,0x1a,0xb1,0x32,0x22,0x3d, +0x2b,0x37,0x1c,0x6e,0x1a,0xf9,0xe1,0x48,0x09,0x4c,0x4f,0xad, +0x35,0xeb,0x34,0xaa,0x8c,0x8e,0xd4,0x79,0x7f,0x3c,0x9e,0x8c, +0xf3,0xf3,0xfe,0xe8,0x85,0x2e,0xed,0xf4,0x54,0x86,0x23,0x35, +0x9c,0xf5,0x2f,0xbd,0xac,0xcf,0x74,0x04,0xba,0x05,0xba,0x78, +0x09,0x1e,0x2c,0x23,0x38,0xe4,0x44,0x60,0xa7,0x23,0x25,0x5e, +0x4d,0xff,0x79,0x99,0x17,0x14,0x82,0x8f,0x08,0xd7,0xf1,0x48, +0xff,0x89,0x68,0x65,0x8f,0xe0,0x6f,0x64,0x96,0x75,0x33,0xae, +0x51,0x82,0x90,0x32,0xa3,0x87,0xdc,0x5d,0xca,0x4c,0x9e,0x31, +0x4f,0x9d,0x3a,0xf6,0xef,0x06,0x4e,0x37,0x50,0x6f,0xf4,0x9b, +0xad,0x0b,0x12,0x8c,0xbf,0xee,0x7e,0x2c,0xdb,0x8f,0x42,0x2a, +0x8c,0xa3,0x86,0xc8,0x7d,0x47,0x60,0x8f,0x45,0x80,0x1e,0xb2, +0xc5,0x93,0xaa,0x58,0x2c,0xe9,0x0d,0xb6,0xb5,0xad,0xa1,0x0b, +0x46,0xa4,0x0b,0x9d,0x46,0xf8,0x13,0xc9,0x85,0xf2,0x6d,0xa9, +0xcb,0xe5,0x33,0xae,0x6a,0x86,0xa6,0x3c,0x4c,0xec,0xda,0x52, +0x07,0xa1,0x1c,0x32,0x7f,0xc8,0x69,0x12,0x30,0xfe,0x78,0x02, +0xb5,0x76,0x85,0x5f,0x40,0x24,0x1a,0x67,0x35,0xea,0x57,0x6c, +0xee,0xca,0x1c,0xc6,0x11,0xc0,0x28,0x35,0x81,0xa1,0x87,0x9c, +0xd4,0x51,0xd0,0xa2,0x6e,0xe9,0xb0,0x04,0x67,0xb4,0xbc,0x3f, +0x8a,0xe5,0xe1,0xd6,0x6d,0xc4,0x3a,0x52,0xcf,0x40,0xca,0xdf, +0x85,0x50,0x0f,0x55,0xa7,0x18,0xa1,0x97,0xb0,0xb4,0x05,0x75, +0x4f,0x49,0x3b,0x40,0xa0,0x64,0x0b,0x10,0x7e,0x0d,0x2f,0xb9, +0x86,0xcf,0xa7,0x93,0xeb,0xba,0x3a,0xae,0x5f,0x3b,0xa9,0x7c, +0x88,0xef,0x3f,0x92,0xa0,0xae,0x0e,0x49,0x4d,0x05,0xe1,0x16, +0xa8,0xaa,0x66,0x89,0xab,0x9a,0xec,0x9d,0x17,0xa5,0xae,0xc1, +0x4e,0xc9,0x07,0xe1,0xa8,0xf0,0x7b,0xb2,0x76,0x24,0x80,0x0b, +0x1a,0x83,0xa1,0xd4,0x50,0x2f,0x2a,0x5a,0x89,0xda,0xc7,0x2f, +0x2b,0x6c,0xd3,0x3b,0x4b,0x4b,0xc4,0x96,0xf3,0x0f,0x58,0x18, +0x5f,0x0c,0xfc,0x12,0x5e,0xd8,0xfa,0x44,0xd1,0x66,0x96,0x51, +0xd4,0xd6,0x56,0xb0,0x8b,0x30,0x24,0xc7,0xbf,0x9a,0x8f,0x46, +0x26,0x4a,0x36,0x99,0x9e,0xd3,0x95,0x6d,0x36,0x2f,0x1f,0x6f, +0xd6,0x6d,0x34,0xb3,0xa6,0x29,0x8b,0xa8,0xd4,0x76,0xba,0xd4, +0xbf,0xba,0xf1,0xbc,0x52,0xc5,0xa6,0x57,0x0c,0x9f,0x8a,0x85, +0xb8,0x9a,0x8f,0x0f,0x80,0x34,0x37,0xdf,0x4a,0xab,0x93,0xe4, +0xfe,0x18,0x60,0xae,0xbe,0x5b,0xdf,0xa8,0xa1,0xa3,0x7a,0xf2, +0x49,0x90,0xba,0x06,0x42,0x05,0x5c,0x89,0x0a,0xa2,0xbb,0xb5, +0xed,0x22,0x41,0x7b,0x60,0x97,0x69,0x8e,0x99,0xc2,0x11,0x16, +0x9f,0x7b,0x0d,0xe0,0x6d,0x66,0x55,0x63,0xde,0x9f,0xcf,0x06, +0xad,0xdc,0x9d,0xc9,0x76,0x1a,0x59,0x0c,0x74,0x2c,0x47,0x29, +0x61,0xb1,0xa8,0x96,0x7c,0xa0,0x58,0xc3,0x5f,0xfc,0x03,0x8c, +0xae,0x5d,0x4c,0x54,0xc5,0xe2,0x1f,0xe4,0xb1,0xa0,0x3e,0x0f, +0x2c,0xf0,0x4b,0x13,0x8f,0x2b,0x4a,0x37,0x15,0xbd,0xd0,0xf3, +0x8a,0x0f,0x02,0x32,0xe7,0xca,0x54,0x51,0x65,0xf3,0x28,0xc4, +0xca,0x4f,0xf0,0xe9,0x27,0xca,0xd7,0x0c,0x63,0xef,0x5b,0xd9, +0x28,0xb9,0xd0,0x62,0xb0,0xd4,0x07,0x37,0xcf,0x52,0xfd,0x1a, +0x96,0xc8,0x74,0x56,0x01,0x85,0x79,0xbb,0xa0,0x11,0x6b,0x6a, +0xe8,0x2f,0x17,0xeb,0xd3,0xbf,0x92,0x5e,0x63,0xe1,0x3c,0x15, +0xf4,0xd4,0x15,0x00,0x96,0x8b,0xc2,0x4e,0x7c,0x7e,0x07,0xad, +0x5f,0x5e,0x4d,0x91,0x23,0x90,0xc4,0x84,0xf6,0x40,0x6b,0xe2, +0x45,0x67,0x27,0xde,0xb1,0xaf,0x64,0x50,0x74,0x1d,0x2a,0x4b, +0xa5,0xd6,0xd2,0x37,0x19,0xff,0x30,0xbc,0x98,0x4c,0xf1,0x12, +0x46,0x30,0x4e,0xf9,0x1f,0x43,0x3b,0xf9,0x57,0x80,0x04,0x43, +0xd4,0xd9,0x6c,0xd2,0x01,0x59,0x89,0xd7,0x2b,0x4a,0x71,0x5b, +0xba,0x66,0x54,0xc3,0x8a,0xac,0x22,0x0f,0xe0,0xc3,0xca,0xac, +0xac,0xc6,0x9f,0xaa,0xc0,0x57,0x90,0x7e,0x07,0xd1,0x0e,0x3b, +0x30,0xb4,0xc7,0x9f,0x6f,0x86,0x2f,0x48,0xa8,0x91,0x79,0x43, +0xc1,0x6c,0xab,0x57,0x93,0xf9,0x68,0x70,0xd4,0xff,0x7d,0x48, +0x4c,0x70,0x56,0x13,0xef,0x0d,0x24,0x3e,0x37,0xee,0xca,0xc2, +0x08,0x6c,0x04,0x86,0x1e,0xeb,0x64,0x6a,0x4e,0xbc,0xc2,0x09, +0x9d,0x3a,0x22,0xc2,0x07,0xf7,0xf6,0xa0,0x6a,0xc5,0xbf,0x2c, +0x86,0xe3,0x87,0xee,0x28,0xb9,0xf9,0x61,0x32,0x81,0xa7,0x81, +0x06,0x39,0xbc,0x9b,0xbc,0x0b,0xbd,0xc5,0x4c,0x79,0x96,0x21, +0xcb,0x3d,0xbf,0xb8,0xc8,0x3f,0x55,0xef,0xda,0x94,0x96,0x74, +0xe0,0x75,0x6a,0xea,0xc5,0x95,0x17,0xbd,0x23,0xc2,0x23,0xa5, +0xa0,0x14,0x55,0x62,0x72,0x08,0xd0,0xa6,0xfb,0xe4,0xe0,0x53, +0x67,0xb0,0xde,0x2f,0x64,0x1f,0x32,0x4e,0x4a,0x76,0xf4,0xd1, +0x87,0x0f,0x47,0xc7,0x0e,0xc2,0x51,0x80,0xf7,0xaa,0xc5,0x50, +0x23,0x19,0x3c,0x99,0x5e,0x42,0x31,0x53,0xf8,0x60,0x1d,0xb9, +0xc2,0x9d,0x2c,0xc9,0x4d,0xa7,0x28,0x85,0x3c,0x46,0xed,0xe9, +0x13,0x7f,0x68,0x86,0xb9,0x23,0xa1,0x2c,0xee,0x34,0x96,0xd1, +0xa6,0x20,0xe5,0x95,0x61,0x81,0x49,0x9b,0x1f,0x88,0x3b,0xbe, +0xa3,0xae,0x5d,0x00,0xea,0xb5,0x43,0xd0,0xc5,0x22,0x5e,0x3f, +0x77,0xd6,0x05,0x37,0x78,0x01,0x38,0x09,0x6a,0x84,0x61,0x49, +0x5d,0x7d,0x6f,0x0e,0x5b,0x7e,0x56,0x56,0x21,0x01,0x21,0x0b, +0x78,0xfb,0x95,0x91,0xb2,0x41,0x68,0x1d,0xc0,0x7d,0x13,0x3c, +0xe1,0x73,0x2f,0xd1,0x59,0x50,0xb4,0x84,0x3f,0xf0,0xd5,0x85, +0xbb,0xfa,0x58,0xd5,0xaf,0x0c,0xbd,0xb9,0x0b,0x19,0x02,0xe4, +0xe8,0xcf,0xe6,0x4f,0xe2,0x6d,0x11,0xde,0xb0,0x21,0xab,0x0e, +0x3b,0xa2,0xfa,0x6c,0x8e,0x4c,0x2c,0x55,0x6c,0x15,0x67,0x9d, +0x0e,0xc2,0xbc,0x3d,0x45,0x0e,0xfb,0x90,0x34,0xef,0xb8,0x1a, +0xe2,0xd8,0xda,0xe2,0x43,0x5d,0x98,0xe0,0x8e,0x76,0x02,0x7d, +0x72,0x2b,0xf1,0xca,0x5e,0xf2,0xe8,0x17,0x40,0xe8,0xea,0x6b, +0xc5,0x02,0x1c,0x0b,0xd8,0xbb,0x1a,0x61,0xe9,0x73,0x6f,0xc4, +0xda,0x55,0x2f,0x32,0x12,0x42,0x9e,0x0b,0x64,0x7c,0x89,0x01, +0x61,0x84,0xee,0xf0,0x0c,0xd6,0x02,0x2a,0xf9,0x0d,0x0b,0x39, +0x0a,0x2e,0x7a,0x4a,0x22,0x35,0xb8,0x56,0x85,0xbe,0x83,0x6b, +0xa1,0x92,0x5c,0x4e,0x6f,0x27,0x6d,0x91,0xe0,0xac,0xb5,0x02, +0x4d,0x74,0xfd,0x9b,0x05,0x96,0x5a,0x7d,0xb3,0x6d,0x14,0xc7, +0xe3,0x80,0x29,0xa9,0x46,0x48,0x42,0xb2,0x70,0x3f,0xe3,0x27, +0x78,0xce,0xaf,0x1a,0x83,0x81,0xaa,0xa4,0x79,0xfb,0x9f,0x70, +0x1c,0xcd,0x6f,0x93,0xc3,0x5a,0x0a,0xe0,0xdf,0xae,0xa0,0xa7, +0xe2,0x38,0xe3,0xcb,0xe8,0x84,0xc5,0x45,0x42,0x3d,0x1d,0xfe, +0x3e,0x9c,0xa2,0xe0,0xb3,0x86,0x7c,0x7b,0x4b,0x1d,0x92,0xce, +0x37,0xdb,0xfa,0xc4,0x33,0xb2,0x42,0x6f,0xfd,0xcd,0xd2,0xfd, +0x8f,0x57,0xfd,0x19,0xca,0xf0,0x94,0xb8,0x12,0x75,0xb6,0x6b, +0x32,0x71,0xb3,0xe4,0x72,0x39,0x47,0x1b,0x2b,0x2e,0x0e,0x9a, +0x74,0xf7,0x91,0x18,0x7e,0x39,0x44,0x25,0xef,0x0c,0x2a,0x70, +0x71,0x72,0xd2,0x06,0xe2,0xdc,0x86,0xcb,0xd7,0xd2,0x8e,0xd8, +0x0e,0xa3,0xef,0x46,0x2f,0x2e,0xb6,0x5f,0x4d,0xc6,0xc3,0x6d, +0xb4,0xab,0x12,0xf5,0x32,0x94,0x4c,0xca,0x54,0x99,0x40,0xdb, +0x01,0xa8,0x72,0x60,0xd9,0x70,0xc3,0x69,0x7b,0xce,0xb4,0xbd, +0x4e,0x6d,0x64,0xd0,0x40,0x9d,0x68,0x27,0x6a,0xc0,0x07,0x38, +0x3c,0xc7,0x58,0x49,0x30,0xed,0xac,0x68,0x37,0xb2,0x4a,0xbf, +0x40,0x80,0x0a,0x25,0x83,0x5b,0x23,0x06,0x95,0x98,0xc2,0xcb, +0xbe,0xd8,0x37,0xce,0xec,0x9f,0x28,0x82,0xf3,0x86,0x5e,0x45, +0x40,0xc6,0xfa,0x17,0xa2,0x03,0xb1,0xec,0x7a,0xe8,0xde,0xbc, +0x5b,0x07,0xdd,0x64,0xed,0xda,0xa2,0x81,0xea,0x3b,0x11,0xb2, +0x80,0x78,0x3d,0x94,0x6c,0xa1,0xfa,0x4e,0xa4,0x28,0x65,0x5e, +0x0f,0xe5,0x4f,0xcf,0x9e,0x1c,0xdc,0x8d,0x90,0x44,0xd5,0xeb, +0x61,0x3c,0x7e,0xfb,0xe4,0xe9,0x1a,0x34,0x1a,0x79,0xf7,0x7a, +0x48,0xd9,0x39,0xf2,0x1a,0x1d,0x84,0x42,0xf3,0x35,0x7b,0x08, +0x5c,0xa1,0xdc,0x89,0x72,0xc9,0x6a,0x12,0x68,0x06,0xd4,0x5b, +0xd3,0x21,0x46,0x98,0xde,0x86,0x9b,0x14,0xb1,0xac,0x29,0x3f, +0x19,0x67,0x67,0xc5,0x92,0x0c,0xa8,0x4f,0x4f,0x81,0x60,0x9c, +0x8f,0x4e,0x06,0x3b,0x1d,0x0e,0xe6,0x66,0xe6,0xc9,0x65,0xdd, +0xad,0xcd,0xee,0xed,0xdb,0xf9,0xfc,0x5a,0x48,0x7f,0xe1,0x0b, +0x74,0x13,0xe0,0xf7,0x88,0x05,0x6f,0x19,0x95,0xd1,0xac,0x94, +0xf2,0xb1,0xb8,0x28,0xbf,0x88,0x65,0x16,0xbc,0x4a,0xf2,0xb3, +0x79,0x3a,0x20,0x3e,0x70,0x22,0x8e,0x70,0x26,0xb2,0xcd,0x94, +0x64,0x32,0xd2,0x7a,0x68,0x95,0x89,0x5d,0xbf,0x9c,0xf2,0xce, +0xc2,0xfc,0xce,0xe6,0x7a,0xc0,0x55,0x77,0x40,0x3d,0xd7,0x38, +0x2f,0x06,0x6d,0xfa,0x09,0x5a,0x25,0x10,0x03,0x76,0x4a,0x6d, +0x56,0x25,0x01,0x64,0xce,0xb5,0x04,0x1b,0x42,0x94,0xc4,0xc6, +0x4e,0x06,0x6c,0x9a,0xa2,0x01,0x66,0x66,0x1b,0x41,0x9b,0x5b, +0xdd,0x83,0x4e,0x85,0xfb,0x2b,0x46,0x91,0xf2,0xaf,0x35,0xfa, +0x09,0x23,0x02,0x0c,0xcf,0x36,0x18,0xb3,0xbf,0x3b,0x19,0xd5, +0x9a,0x65,0x38,0x46,0x2f,0x86,0x7a,0xf6,0x78,0xfa,0x34,0x66, +0xff,0x85,0x8d,0x83,0x9f,0x58,0xb0,0x1e,0x4d,0xfd,0x96,0x03, +0xb0,0xba,0x52,0x0c,0x48,0x95,0xc2,0xfc,0x8a,0xa3,0xea,0xf9, +0x28,0xb4,0x06,0x97,0x01,0x55,0x34,0xe9,0x2e,0x47,0x93,0xb3, +0xfe,0x88,0x78,0x24,0xf6,0xea,0x51,0xa1,0x33,0x87,0x93,0x44, +0x55,0xc0,0x26,0xbe,0x65,0x5b,0x63,0xa0,0xb6,0xc2,0x29,0x9a, +0xf4,0x86,0x46,0xfa,0xb1,0x47,0xc3,0xe9,0xef,0xf9,0xf9,0xf0, +0xf9,0x64,0xea,0xa1,0xb4,0x0c,0x9a,0x05,0x9f,0x7a,0xac,0x2e, +0xac,0x00,0xd6,0xa2,0x36,0x70,0xbb,0xce,0x4c,0xf1,0x6a,0x51, +0x30,0x8d,0x70,0x32,0x4d,0xa7,0x56,0x83,0x96,0x2e,0xfe,0x7a, +0x9a,0x2f,0xc1,0x37,0x5a,0xba,0x41,0x7f,0xe2,0x17,0x7e,0xa6, +0x2f,0x95,0x8b,0xa3,0x35,0xe2,0x0e,0xe4,0x55,0x97,0x82,0x16, +0x7f,0x55,0xa2,0xb5,0xb2,0xb3,0x0e,0x76,0xbc,0x37,0xb4,0xe8, +0xe0,0xee,0x29,0x38,0x29,0xf0,0x29,0x8b,0xda,0xce,0x61,0xd4, +0xc7,0x8a,0xa1,0xc9,0x05,0xe6,0xa6,0xc2,0x64,0x70,0x69,0x64, +0x68,0xe4,0x13,0x69,0x90,0x9b,0x14,0xcd,0x0c,0x02,0xb3,0x3e, +0x06,0x38,0x04,0x10,0x85,0x24,0x10,0xf5,0xea,0xd6,0x56,0x8d, +0xd8,0x84,0x85,0x31,0x96,0xf3,0xd4,0x51,0x4e,0x94,0xef,0x2f, +0x08,0x94,0x6a,0x30,0x42,0xa1,0xd5,0xeb,0xb2,0x81,0x2b,0xb5, +0xa3,0x1e,0x95,0x2c,0xab,0x39,0x6d,0xce,0xc7,0x30,0x2c,0x4f, +0x49,0x43,0xbb,0xd0,0x1f,0x37,0xf9,0xf9,0x87,0xfa,0xfb,0xd3, +0xae,0xb9,0x60,0x0f,0x6f,0xd4,0x7b,0x49,0x52,0xa1,0xc8,0x96, +0x04,0x17,0xf1,0x19,0xe3,0xe7,0x21,0x5e,0x4f,0x59,0xd2,0x2e, +0x2d,0x78,0xa6,0x51,0x12,0xd7,0x86,0xee,0x5a,0x30,0xf6,0xcb, +0x51,0x16,0xb8,0xbc,0x70,0xda,0x66,0xf3,0x10,0xd9,0xcb,0xb2, +0x6a,0x4c,0x68,0xd1,0x74,0xf5,0xe0,0x0c,0x8e,0x2f,0x99,0x1f, +0xa6,0x8d,0xeb,0xcb,0x30,0x48,0x4b,0xee,0x28,0xdb,0xb0,0xab, +0x06,0xf8,0x74,0x36,0x6d,0x88,0x53,0x1e,0xee,0x41,0xcd,0x06, +0x25,0xb4,0x74,0x2b,0x1a,0xca,0x5e,0x7a,0xfa,0x2b,0x8f,0xa8, +0xba,0x5c,0x79,0x5c,0x91,0x60,0xf6,0x54,0x16,0x29,0xad,0xaa, +0x0a,0x3c,0x6e,0xcb,0x34,0xd0,0xb7,0xc6,0x74,0x3a,0x98,0xbd, +0x91,0x35,0x00,0x2e,0xc0,0xab,0xc4,0xdc,0xf8,0x14,0xa9,0xad, +0x85,0xeb,0x25,0x77,0x01,0xf9,0xe5,0xf5,0xf8,0xb7,0x14,0x5c, +0xae,0xb8,0x2b,0xde,0xaf,0xba,0x9f,0xef,0x0d,0xb9,0xd8,0x89, +0x8d,0x2b,0x21,0x4f,0xf0,0x65,0xda,0x98,0x13,0x9b,0xe7,0x62, +0xa5,0xe2,0x38,0xf0,0x4f,0x64,0xdc,0x66,0x25,0xe6,0x03,0x96, +0x82,0x00,0xee,0xc7,0xe1,0xac,0x04,0xb6,0x62,0x3f,0xa9,0x64, +0x82,0x3c,0x01,0x9c,0x47,0x29,0xbd,0x39,0xc0,0xa2,0x41,0x04, +0xb0,0x82,0x46,0xbd,0x72,0x68,0x80,0x28,0xb1,0x44,0x08,0xf3, +0x1a,0x0e,0xda,0xea,0xf7,0xa1,0x9f,0xb0,0xe1,0xc5,0x50,0x33, +0x09,0x83,0xec,0xde,0x6f,0x4d,0xfc,0x06,0x0b,0x87,0x1a,0xe8, +0xc9,0xf4,0xb2,0x70,0x82,0xd2,0xb6,0x14,0x7a,0xf8,0x56,0x79, +0xc1,0xa2,0xb9,0x3c,0xd2,0x23,0x2b,0x2c,0x25,0x29,0x23,0xbd, +0x57,0x63,0xe3,0xc6,0x06,0xaf,0x6a,0x11,0x8b,0x0c,0xb7,0xb9, +0x46,0x91,0x10,0x0d,0xdd,0x34,0xc9,0xca,0x10,0xb8,0xbd,0x31, +0x26,0x63,0x0d,0x7d,0xc6,0xce,0x34,0x96,0xb7,0x0c,0x07,0x88, +0xe9,0x68,0x0b,0x6d,0x1a,0x04,0x08,0xfe,0xd2,0x9e,0x48,0x4a, +0xad,0xa6,0xfb,0x37,0x76,0x7c,0x56,0x45,0x93,0xdd,0xd5,0x3e, +0x55,0xb5,0xe8,0x02,0xc6,0xde,0x7f,0x5f,0x5d,0xb8,0xe8,0x10, +0xbd,0xf5,0xd7,0xa5,0xbc,0x7b,0xe4,0x0a,0x71,0x5b,0xa7,0x54, +0x0b,0xab,0x13,0x64,0x71,0xdc,0x4a,0xdd,0xfa,0x34,0x2e,0xe5, +0x20,0x57,0x84,0xf0,0x9c,0x6d,0x43,0x4e,0x5b,0xd0,0xea,0xbe, +0x11,0xfc,0x46,0x1c,0x08,0xae,0x27,0x60,0x8a,0x11,0xe7,0xbc, +0xd8,0x4d,0x31,0xb2,0x76,0x37,0x05,0xc5,0xb5,0x44,0xda,0x4f, +0xd6,0x1c,0xd2,0xd3,0x79,0x31,0x9b,0x5c,0xbf,0x36,0xd2,0x1d, +0x2e,0x05,0x59,0x8c,0x73,0x3f,0xc9,0xf2,0x41,0x18,0x1d,0x30, +0x52,0x2c,0xb1,0xed,0x92,0x56,0x19,0x2b,0x94,0xf1,0x53,0x01, +0xd6,0xa6,0xec,0x29,0xef,0x31,0x40,0x72,0x6b,0xf3,0x0c,0x26, +0x1a,0x8a,0xd0,0x4a,0x28,0xbd,0x81,0xf6,0x0b,0xa3,0xa5,0x30, +0x1a,0xd9,0x12,0x21,0xe0,0xa8,0x32,0xe8,0xd1,0x31,0xc9,0x70, +0xda,0x31,0xea,0x6b,0x29,0x84,0x50,0x7b,0x30,0x43,0x44,0x0d, +0x08,0x37,0x67,0x93,0x77,0x37,0x37,0x46,0xa5,0xb1,0x2d,0xd1, +0x66,0x48,0xfa,0x66,0x96,0x79,0xe8,0xb7,0xb6,0xa8,0x2e,0x61, +0x7c,0xc7,0x6f,0x85,0xb4,0xfc,0xce,0x81,0xd6,0x63,0xbc,0x62, +0xf6,0x4f,0xeb,0xa2,0x0f,0xab,0xda,0x13,0x5d,0xb0,0xc6,0xc1, +0x25,0xb5,0x8f,0xc1,0x28,0xa6,0xe0,0xf6,0x4e,0xd6,0x87,0x0c, +0x2e,0xd9,0x4e,0x48,0x80,0x47,0x75,0x82,0x5e,0x88,0x44,0xe7, +0xfe,0xf8,0xec,0xf8,0xe5,0x8b,0xa3,0x63,0x93,0x1d,0xd9,0xf0, +0x0a,0x16,0xb9,0x39,0x98,0x18,0xc8,0x72,0x66,0x37,0x9c,0xea, +0x39,0xe4,0xf8,0x82,0xb9,0x44,0x2b,0x46,0xd0,0xb0,0xb9,0x4d, +0xf6,0xf5,0xbb,0x24,0x6c,0xbb,0x72,0xd7,0x67,0xfd,0x1d,0x81, +0x61,0x6d,0x2e,0x47,0x79,0xd9,0xd4,0x9a,0x48,0xe8,0xbc,0x53, +0xbb,0xa9,0x86,0x5c,0x80,0xdb,0xea,0xcf,0xc5,0xe6,0x6a,0x77, +0xfc,0xbb,0x2f,0xae,0xad,0x36,0xc7,0x66,0x8b,0x39,0x0a,0x32, +0x2b,0x8b,0x77,0x1c,0x6b,0x9d,0xe1,0x42,0x05,0x58,0x5f,0x5a, +0x60,0xac,0xce,0x3f,0x5b,0x7d,0xfe,0x91,0x0a,0x3f,0xd5,0x87, +0x03,0x8b,0x05,0xf9,0x71,0xb1,0x3d,0xaf,0x81,0xde,0x53,0xd6, +0xec,0x55,0x18,0x74,0x72,0x08,0x97,0xa2,0xfe,0x5f,0xcc,0x45, +0xe8,0x36,0x5c,0x8f,0x1a,0x3b,0x8d,0x40,0x57,0xcc,0xcc,0x02, +0x57,0xc3,0x35,0xb1,0x50,0x6d,0x2a,0x67,0xd4,0x97,0xe1,0xd2, +0xeb,0x91,0xc5,0xa3,0xbf,0xbf,0x16,0xcb,0x44,0x90,0x03,0x81, +0xaf,0xc4,0x43,0xcb,0xaa,0xc5,0x44,0xab,0xee,0x57,0xe2,0x82, +0x75,0xcc,0x62,0x82,0xc0,0x57,0xe2,0x41,0x51,0xaf,0x45,0x84, +0xa1,0xaf,0xc4,0xc4,0x12,0x5e,0x8b,0x8b,0xc3,0x5f,0xdb,0xe6, +0x20,0xd8,0x75,0x8d,0xde,0xaf,0x19,0x03,0x15,0xdb,0xaf,0x48, +0xad,0xba,0xa1,0x75,0xe9,0x76,0x65,0x50,0x05,0xc9,0x7e,0x6a, +0x17,0xa7,0xea,0x93,0x82,0x38,0x58,0x7d,0xd5,0xba,0xb4,0xfb, +0x27,0xd7,0xa5,0x3f,0x31,0x9d,0x77,0xff,0xa2,0x61,0xaf,0xc8, +0x7e,0xfb,0xff,0xc8,0xf1,0xea,0xf4,0xe5,0xe5,0x12,0xc1,0x51, +0xff,0x93,0xe6,0xc0,0x97,0xaf,0xcc,0x3f,0x7c,0x7e,0x31,0xf8, +0xf7,0xac,0xca,0x3c,0x20,0xfe,0xdc,0xb4,0xdc,0xad,0x9d,0x96, +0x4c,0x7b,0x9c,0xc3,0x01,0xe3,0xb7,0x37,0xd5,0x4c,0x21,0x89, +0xbe,0x0d,0x7f,0xa5,0x61,0x35,0xcb,0xca,0x36,0x94,0x93,0x8a, +0x6c,0xe2,0x5c,0x56,0xea,0x7e,0x9d,0xf9,0x53,0x89,0xf7,0x0c, +0xae,0x70,0x70,0x6a,0x1e,0x4f,0xde,0xcc,0x67,0x18,0xec,0xea, +0x2c,0x3d,0xb5,0xe2,0x34,0x67,0xc1,0x61,0x04,0xc6,0x01,0xf2, +0xe0,0x98,0x07,0x2d,0x30,0x9c,0x62,0xab,0x19,0x6f,0x03,0x74, +0xd7,0xbf,0x82,0x4f,0x24,0xd2,0x92,0xb6,0x81,0x45,0x82,0x32, +0x87,0xa9,0x7c,0xf8,0x35,0x90,0x49,0xa5,0x6b,0x67,0x01,0x1e, +0xba,0x6b,0x5e,0xf7,0x8c,0xe9,0x09,0x53,0xd0,0xd4,0x9b,0x39, +0xfd,0x01,0xb6,0x03,0x70,0x6b,0xed,0xb8,0x75,0x56,0xc9,0x31, +0xe2,0x6c,0x77,0x90,0x64,0xac,0xc2,0x13,0x79,0x9d,0x66,0xf1, +0xd7,0x60,0x45,0x85,0x02,0xe7,0xdb,0x9c,0x3b,0x24,0x8e,0x9e, +0x1d,0xf7,0x2f,0xdd,0x63,0x28,0x28,0x76,0x6b,0x8b,0xaf,0xf7, +0x21,0x50,0x3f,0x63,0x00,0x88,0xef,0xed,0xf1,0x10,0xec,0x5a, +0xc0,0x9b,0x3b,0x28,0x02,0xa8,0x1a,0xcd,0xe1,0x30,0xab,0x92, +0xa9,0x4c,0xdc,0x19,0xce,0xbe,0x4c,0x62,0xb7,0xa6,0xe1,0xbd, +0x00,0x5d,0x9a,0xd0,0xbd,0x0c,0xfc,0x7d,0x0e,0xf1,0x1a,0x1b, +0x7c,0x2f,0x16,0x54,0xd0,0x5d,0xdc,0xbb,0xe2,0x17,0x98,0xab, +0x35,0xaa,0xc9,0xe3,0xa0,0x6b,0x7a,0x51,0x9a,0xd4,0x45,0x46, +0x25,0x0a,0x5b,0x6f,0xf4,0x56,0x2f,0x94,0x18,0x0c,0xf9,0x53, +0xa9,0x92,0xc0,0xba,0x08,0x4c,0x85,0xe9,0x23,0x1b,0xbe,0x11, +0xea,0x21,0xab,0xab,0x83,0x7d,0x23,0xb4,0x47,0xcc,0xf2,0x16, +0xaf,0x76,0x7f,0xcf,0xc3,0x2a,0xf0,0x80,0x8f,0x43,0xf1,0xae, +0x51,0x1d,0x36,0x42,0xc5,0x00,0xd4,0xb3,0x05,0xdc,0x2c,0x0c, +0x07,0x58,0xc6,0x69,0xf3,0xba,0x7f,0x13,0xb8,0x8b,0xf7,0xe5, +0x17,0x77,0x75,0x59,0xe5,0xde,0xdf,0xa9,0x38,0x76,0xdd,0x85, +0x29,0x3c,0xff,0x91,0xc8,0x64,0xbd,0x11,0x83,0xa2,0x99,0xb4, +0xb6,0x50,0xc2,0x85,0x3d,0x86,0x90,0x7a,0x0c,0x84,0xcd,0xc0, +0xc2,0x20,0x6c,0x09,0x2f,0x2d,0xb9,0x9b,0x86,0xda,0x26,0x58, +0x2d,0xb3,0xaa,0xe5,0xfe,0xc8,0x97,0xa6,0x24,0x62,0xcd,0x76, +0x48,0xb0,0x0d,0xbe,0xb2,0x54,0x6e,0x2a,0xbf,0x5c,0x9c,0xc9, +0xd5,0x2b,0x76,0x8d,0x7a,0xba,0x83,0xd8,0xcc,0xf4,0x89,0x0e, +0xec,0x6a,0xfc,0xb5,0x2b,0x3b,0xf4,0xe5,0xf3,0xb2,0x14,0x07, +0x7b,0x97,0xe7,0x26,0xbc,0xf4,0xff,0xa2,0x15,0x4e,0xd3,0xf1, +0xa6,0xe4,0xa1,0x9b,0xbc,0x47,0xd1,0xca,0xb5,0xd6,0x28,0xb4, +0x17,0xdd,0x77,0x2c,0x8c,0x28,0xea,0x39,0xfb,0x15,0xde,0x9a, +0x10,0x66,0xda,0x15,0x38,0xfa,0x8e,0xf5,0xa4,0x6d,0x72,0x97, +0xe4,0x8e,0x9c,0x60,0x3c,0xfc,0xaf,0xb3,0x86,0x72,0x96,0xfa, +0x1d,0xcc,0x5f,0x45,0x0b,0xb1,0x8a,0xaa,0xc9,0x07,0x10,0xc8, +0x83,0x25,0xae,0xec,0x8b,0x16,0x32,0x3c,0xc1,0xfc,0xe9,0xed, +0x19,0x75,0x2d,0x3a,0x56,0xe6,0x67,0x73,0xae,0xb1,0xc9,0xfc, +0x25,0xd3,0xf5,0xdf,0xbd,0xa4,0x7d,0xf9,0x54,0xae,0x5c,0xef, +0xd6,0x41,0x63,0x24,0xda,0xe4,0xb8,0x6f,0x55,0xa7,0xfe,0xf9, +0x29,0x1f,0xc8,0x84,0x5d,0x87,0x0b,0xbd,0x47,0xcf,0xe8,0x4f, +0xb0,0x0b,0x83,0x6b,0x46,0xf3,0x96,0xde,0xa1,0xb1,0x77,0xb9, +0xa1,0x81,0x00,0xb7,0xf7,0x9a,0x4d,0xbf,0x22,0x3b,0xa8,0x50, +0xfa,0x92,0x6a,0x92,0x8a,0x57,0x92,0x24,0x95,0x45,0x6f,0xa3, +0x7f,0x6e,0xff,0x74,0x7c,0xfc,0x66,0x9b,0x4a,0xdb,0x36,0x98, +0x23,0xfb,0x34,0x3b,0x13,0x23,0xb3,0x63,0x34,0xed,0x52,0x1b, +0xb5,0x4c,0xc4,0x3d,0x74,0xa8,0xbd,0x59,0x41,0x69,0xe7,0x0e, +0x96,0x45,0x12,0x57,0xc5,0xae,0xa0,0x36,0xab,0x2f,0x68,0xee, +0x7a,0xc1,0x9e,0x6e,0x72,0x66,0x4e,0xdc,0xf4,0xf6,0xc7,0x05, +0xba,0xec,0xff,0x37,0x13,0xf1,0xff,0x03,0x12,0x4c,0x36,0x96, +0xa3,0xdc,0x45,0xd1,0x9a,0x3b,0x99,0x7f,0xc8,0x7c,0xee,0x99, +0x5a,0xa9,0xbd,0x77,0x90,0x1b,0x1e,0xdf,0x3a,0xd0,0x85,0x0d, +0x4d,0x5c,0xb3,0xe9,0x71,0xa8,0x7c,0xe0,0xf4,0x25,0x80,0x5f, +0x51,0x98,0xb1,0x33,0xb0,0x66,0x79,0x42,0x04,0xfa,0x15,0x85, +0xe1,0xbd,0xd4,0xba,0x25,0x09,0x71,0x11,0x9d,0x70,0xf8,0x2e, +0xe7,0x8b,0x0b,0xc5,0x09,0x2f,0x59,0x08,0x44,0x54,0x2e,0x51, +0xca,0x94,0xbe,0xa2,0x72,0x64,0xa3,0x61,0xcd,0xda,0x79,0x62, +0xa7,0xaf,0x28,0x8c,0x6d,0x40,0xac,0x59,0x5a,0x20,0x44,0xfa, +0x8a,0xf2,0xac,0x8d,0x89,0x75,0x7b,0x4f,0x8a,0x9a,0xe2,0xaf, +0xed,0x39,0x32,0x64,0xe1,0x17,0x59,0xd3,0x79,0xfe,0x05,0xde, +0x3a,0x57,0x77,0x77,0x13,0x50,0x8d,0xa4,0x86,0x80,0x55,0x8a, +0x6e,0xe4,0x45,0x2b,0x40,0xc7,0x1a,0xff,0x7c,0x68,0x0f,0x0c, +0xf4,0x70,0xd0,0xc9,0x6d,0x80,0xc6,0xa1,0x45,0xd8,0xf6,0x83, +0xee,0xec,0xee,0x6d,0x4a,0x95,0x82,0x36,0x3a,0x08,0x68,0x02, +0xc0,0x3a,0x53,0xdd,0xdd,0x67,0x40,0x2b,0xb1,0x9d,0xa4,0xe5, +0xe8,0x0a,0xbd,0xa8,0x1e,0x4e,0x42,0xaf,0xd8,0xbd,0x9d,0x90, +0xcf,0x20,0x6e,0x57,0xea,0x4f,0x63,0x76,0x7a,0xd6,0x65,0x5d, +0xe8,0xfa,0x4d,0x65,0x32,0x04,0x4d,0x86,0xf9,0x44,0xbb,0x39, +0x61,0x80,0xdf,0x58,0xe2,0x45,0x87,0x35,0x38,0xe8,0x62,0xae, +0x4a,0x8f,0x45,0x9a,0x74,0xd1,0xdb,0x46,0xce,0x8e,0x2e,0x7a, +0x6b,0xf7,0xf5,0xa0,0x89,0x7c,0x77,0xa9,0x75,0x33,0x20,0xc8, +0x54,0xd9,0x9c,0xc2,0x21,0x2c,0xa8,0xa7,0x7b,0x0a,0xa6,0xcf, +0xfd,0x97,0x40,0xce,0x84,0xe3,0x18,0x1d,0x44,0xc2,0x43,0x20, +0xef,0x6a,0xb7,0x4a,0xed,0xd5,0x34,0x55,0xa8,0x27,0x6b,0x51, +0x28,0xfb,0x95,0x08,0x53,0x91,0xb1,0x8c,0x5d,0xa9,0x04,0xeb, +0x00,0xc9,0x9a,0x21,0x81,0x65,0x68,0xa7,0xd7,0xec,0xf7,0xa8, +0x10,0x5f,0xa7,0xe8,0xdb,0x2d,0x25,0xf4,0x2a,0x84,0x3d,0xf6, +0x7d,0x94,0xd1,0xba,0x55,0x61,0x8d,0xb8,0x64,0x63,0x25,0xd3, +0x88,0x88,0xb1,0x69,0xd6,0xd0,0x32,0x35,0x12,0x65,0x9b,0xcf, +0x75,0x87,0xc9,0x5d,0xdb,0x41,0xe5,0xcc,0x81,0x8e,0xaa,0xa1, +0x8d,0x44,0xe0,0x06,0x28,0xd0,0x54,0xad,0x06,0xaa,0xd4,0x57, +0xad,0xc5,0x57,0xd6,0x5a,0xad,0x06,0x2d,0x9f,0x42,0x48,0x83, +0x14,0x73,0x96,0x13,0xcb,0x15,0xac,0x93,0x48,0x54,0xe3,0x10, +0x8a,0x66,0x21,0xa6,0xa5,0x37,0x59,0xea,0x46,0x59,0xd5,0xd0, +0x5e,0xf6,0xc0,0x7a,0x6b,0xc9,0x50,0x2b,0xf8,0x87,0xcb,0x58, +0x2a,0x32,0xfc,0x3d,0x9f,0xcc,0xf1,0xb1,0xe0,0xb4,0x38,0x07, +0x27,0x05,0x90,0xd8,0x3c,0x55,0x67,0xba,0xb0,0x0f,0x68,0x74, +0x55,0xa1,0x78,0xfb,0x0d,0x68,0x8f,0x85,0x9a,0x64,0x4a,0x1f, +0xb2,0x29,0xe1,0x35,0xbd,0x1b,0x74,0x29,0xd0,0xf1,0x94,0x64, +0x86,0x80,0x48,0x44,0x9d,0x39,0x87,0x16,0x75,0xe8,0x14,0x6a, +0xa5,0xc9,0x58,0x8c,0x80,0xc1,0x7a,0xde,0x9f,0xc9,0x78,0x8a, +0x51,0xe6,0x5e,0x22,0x33,0x44,0xd8,0x9b,0x0a,0x75,0xd5,0x2f, +0x5e,0x7f,0x1c,0xbf,0x21,0x9b,0x3c,0x9f,0x1d,0x80,0x1f,0xaf, +0xc6,0x7d,0xf0,0x00,0xfa,0x9c,0x3d,0xa8,0x8a,0x12,0x8c,0x13, +0x55,0x02,0x38,0xec,0xdf,0xc8,0x44,0xb0,0x5b,0x4d,0x09,0x6f, +0xf1,0xd5,0x89,0x4c,0xa3,0x77,0x28,0x5e,0xf2,0x5b,0x30,0x7f, +0x5d,0x86,0xc1,0x68,0x43,0x01,0x1a,0x7d,0xf6,0x08,0xc0,0x18, +0x4e,0x7e,0xa6,0xcf,0x4b,0x9f,0x65,0x2a,0xbc,0x0c,0x34,0xd4, +0x1f,0x4d,0xae,0xfd,0x46,0xd3,0x61,0x4e,0x7a,0x41,0xfe,0x72, +0x64,0x2a,0xbb,0xd0,0x61,0x00,0x70,0x7b,0x5b,0x01,0x34,0x72, +0xd1,0x06,0x13,0x3f,0x92,0xa4,0xee,0xe7,0x87,0xf7,0x9c,0xf6, +0xbf,0x87,0x9f,0x0b,0xd3,0xfd,0x60,0x1c,0x97,0xa3,0x7f,0xd0, +0x45,0x65,0x76,0x10,0xe0,0x88,0x57,0xa7,0xd5,0xb6,0x2c,0xf5, +0xf7,0x46,0x3e,0x86,0xc9,0x70,0x3e,0x9c,0x5c,0x6c,0x9c,0x76, +0x74,0x04,0xbe,0x4a,0xf2,0x63,0xe9,0x75,0xcf,0x29,0xd8,0x06, +0xbf,0x19,0x0e,0x40,0xca,0x64,0x8e,0xf8,0x60,0x63,0x74,0xe3, +0x14,0x71,0x2e,0xdb,0x91,0x75,0x17,0x1a,0x6d,0x66,0x30,0xda, +0x74,0xe6,0xe1,0xa7,0x9b,0xc9,0x74,0x56,0x74,0xe2,0xaa,0x44, +0x32,0x47,0xbc,0xb5,0xc5,0x66,0x89,0x19,0x16,0x1e,0x9a,0xd0, +0x57,0xe6,0x27,0x64,0xa7,0x89,0xe2,0xcf,0xe6,0xa9,0x0e,0xa4, +0x34,0x61,0xb2,0x53,0x75,0xda,0xfc,0xc7,0xb3,0xb7,0x47,0x2f, +0x5e,0xbf,0xca,0xa2,0x56,0xf3,0x9b,0xe6,0x5e,0x84,0xac,0x03, +0xe8,0x98,0x65,0xa4,0x6a,0xa6,0x7f,0x8c,0xbb,0x5e,0xd9,0x10, +0xe0,0x2d,0x70,0xda,0x67,0x4b,0x2c,0xe4,0x96,0x2b,0xbf,0x40, +0x69,0xe9,0x26,0xd4,0x32,0x81,0x80,0x1c,0xa9,0x5b,0x5b,0x3a, +0xd6,0x62,0xca,0x32,0x2f,0x31,0x11,0x69,0x71,0x09,0xb1,0xf5, +0xa8,0x05,0x50,0xec,0x05,0x2d,0xcb,0x1a,0x2e,0xe4,0x9e,0xbd, +0xe6,0xd9,0xae,0x62,0x08,0x97,0xdc,0xa6,0x9f,0xfd,0x1c,0xbd, +0x16,0x6a,0x3c,0xa6,0x04,0xd6,0x26,0xa5,0x62,0x14,0xba,0x33, +0xec,0x29,0x74,0x87,0x08,0xf6,0xfc,0x78,0x31,0xe1,0x67,0x4f, +0xa4,0x1b,0x6e,0xca,0x81,0x31,0x93,0xb1,0x5d,0x65,0x00,0x57, +0xa2,0x60,0x88,0xfc,0xf2,0x92,0x21,0x17,0xf8,0x51,0x54,0xfc, +0x51,0x43,0x45,0x1b,0x2f,0x09,0x32,0xb0,0x03,0x81,0x4b,0xef, +0x5d,0x7d,0xc2,0xa2,0x3f,0xb4,0xc9,0xea,0xdc,0xa6,0x40,0x37, +0x65,0xd0,0x42,0x1d,0x4e,0x4b,0xed,0xa2,0x41,0xfd,0x04,0x65, +0x98,0x3e,0xd2,0x91,0x1d,0x8e,0x2b,0xf7,0x4d,0x1a,0xa3,0x3a, +0x22,0x14,0x2e,0x35,0x55,0xe7,0x43,0xf2,0x61,0xa5,0x46,0x39, +0x19,0xf0,0xc0,0x52,0xe8,0xe5,0x71,0x75,0x1b,0x94,0x32,0xb1, +0x48,0x1a,0xf2,0xf1,0xfb,0x3c,0x5a,0x83,0xd0,0x6e,0x4b,0x16, +0xd1,0x3a,0x05,0x3e,0xa6,0x86,0x60,0x40,0x7a,0xa3,0x0f,0xb3, +0x1b,0x79,0xad,0x8d,0xf1,0x64,0x83,0x6d,0xb8,0x91,0x01,0xe9, +0xa8,0x6d,0xde,0xda,0xe1,0x58,0x1e,0x0d,0xd0,0x1e,0x31,0xb9, +0x46,0xae,0x6e,0xbe,0x6b,0x7d,0xc8,0xf6,0xdb,0x90,0x11,0x96, +0x4c,0x3e,0xec,0xef,0xb5,0x79,0xd8,0x63,0x7b,0xea,0x09,0x08, +0xce,0x34,0xc1,0xac,0x82,0x5c,0x4e,0xa9,0x51,0x99,0x06,0xd3, +0xae,0x94,0x94,0x38,0x95,0x33,0x28,0x0d,0x2c,0xae,0x31,0x15, +0x66,0xcb,0x2d,0xb5,0x39,0xb8,0xbc,0x40,0x6a,0x3a,0x0e,0x6b, +0xec,0xd1,0x9e,0xa4,0x15,0x29,0xf4,0xfc,0x6f,0x8d,0xde,0x32, +0xd8,0x01,0x53,0x56,0xdd,0x5b,0xd8,0x40,0xa5,0x9c,0x69,0x8c, +0x59,0x4c,0x3c,0xb5,0x18,0xdc,0x17,0x80,0x2b,0x28,0x0a,0x0a, +0x9b,0x94,0x60,0x98,0x85,0xec,0x52,0x8a,0x8e,0xb5,0x8c,0x2d, +0x60,0x5a,0xaa,0x53,0xb9,0xf1,0x70,0xef,0x4d,0xff,0x9b,0x3a, +0x0d,0xcb,0x94,0x3d,0x47,0x44,0xf8,0xdd,0x87,0x71,0x7f,0x51, +0x1f,0x22,0xae,0xfa,0x8e,0xf4,0x93,0xe9,0x88,0x57,0x5e,0xeb, +0x74,0xc5,0xe8,0x6b,0x53,0x2f,0x91,0x66,0x79,0xac,0x5a,0xb2, +0xda,0xe5,0xe5,0x6a,0xb9,0xf6,0x00,0x01,0xf7,0x76,0x90,0xb1, +0x83,0xab,0xd5,0xf6,0x36,0xe5,0xef,0xa5,0xe6,0x4b,0xad,0x3b, +0x86,0x70,0xd1,0x05,0x74,0xbd,0x8a,0x81,0xe4,0x25,0xfe,0x25, +0xa3,0x09,0x0c,0xbb,0xe2,0xd1,0x76,0xf6,0x45,0x6b,0xa7,0xb5, +0xb3,0x36,0xfe,0x7c,0xf7,0x52,0x47,0xa2,0xed,0xf5,0x16,0xb9, +0x4e,0x4c,0x05,0xf0,0x9c,0x01,0x47,0xdd,0xc4,0x1e,0xd8,0xc5, +0x8f,0xa8,0x46,0x56,0xeb,0xb4,0x49,0x66,0x5b,0xff,0xda,0x35, +0x9f,0xf8,0x38,0xde,0x9e,0xa9,0x20,0xbb,0x3b,0x63,0xb0,0xe3, +0x52,0xbe,0x76,0xfd,0x5f,0xb3,0x35,0xb6,0xb6,0xbc,0x8d,0xc2, +0x7a,0x83,0xb5,0xdb,0x00,0xae,0x07,0x2b,0xd6,0x6d,0xdb,0x04, +0x56,0x54,0xc4,0x74,0xaf,0xd5,0x69,0x9b,0x6b,0x92,0xb9,0xb4, +0xe5,0x00,0x41,0xc8,0xd7,0xea,0xbe,0x81,0x53,0xdf,0x5d,0x0c, +0x12,0xc7,0x2c,0x16,0x72,0x81,0x80,0x2b,0x92,0x59,0x3e,0xfb, +0x6c,0xee,0x0a,0x71,0x3c,0x38,0x6f,0xca,0x61,0xbf,0xa5,0x82, +0xbf,0xa6,0x5e,0x23,0x0a,0x4c,0xa7,0x61,0x42,0xc7,0xc6,0x7f, +0xfd,0x96,0x0d,0xa5,0x9b,0xe1,0x49,0x3f,0x5b,0x5b,0xeb,0xee, +0xdd,0x1d,0x1a,0xc5,0x29,0x73,0x2f,0x30,0x65,0x37,0x09,0x87, +0x71,0x49,0x30,0x86,0x36,0x03,0xbe,0x1f,0x9a,0x6e,0xfc,0xf9, +0xaf,0x6c,0xba,0xd6,0x1d,0x4d,0x07,0xa7,0x0f,0x6a,0x39,0x2c, +0xdf,0x34,0x1c,0x44,0x77,0x4c,0xec,0x9f,0x6b,0x36,0x26,0x45, +0x13,0xcb,0x34,0xad,0xdd,0x6e,0xdc,0x60,0x6e,0x15,0x10,0xed, +0xe6,0x8c,0x7f,0x21,0xff,0x72,0x3e,0x9a,0x0f,0x86,0x7e,0xbb, +0x19,0xf7,0x36,0xa5,0xfa,0x6f,0xb6,0x52,0xef,0x78,0x45,0xd5, +0xe7,0x83,0x95,0x6d,0x01,0x4e,0xec,0x6c,0xb7,0x90,0x87,0xb7, +0xbe,0x4b,0x19,0x6d,0x5a,0xbd,0xfc,0xf9,0x9e,0x31,0xb3,0x8c, +0xdd,0xd8,0x2c,0x71,0x76,0xe4,0xe3,0xdf,0x27,0x1f,0x02,0x2a, +0xe5,0xb3,0xb1,0x3e,0x3c,0x0f,0x13,0x4f,0xbb,0xec,0x56,0xad, +0xf6,0xe0,0x7a,0x11,0xce,0x62,0x68,0x18,0xc8,0xca,0xcc,0x38, +0xb3,0xb3,0xa2,0x00,0x6c,0x69,0x3d,0x51,0x31,0x65,0x65,0x33, +0xf6,0xe4,0xb6,0xdd,0x3c,0x15,0x4b,0xdc,0x33,0x31,0xdd,0x0d, +0x40,0x49,0x42,0x44,0xdf,0x8c,0xe6,0xf2,0xce,0x15,0xb0,0xa3, +0x9f,0x94,0x35,0xcb,0xa4,0x86,0x20,0xcf,0xed,0x88,0xef,0xe3, +0xd5,0x70,0x1a,0xb4,0x01,0x3b,0xab,0xcc,0xa7,0xc5,0xcc,0xb3, +0xab,0x45,0xde,0x50,0x30,0x35,0xe9,0x60,0xb2,0x99,0x4a,0xdd, +0x5e,0x7a,0xda,0xa5,0x98,0x08,0xb6,0xb1,0x28,0x8d,0x68,0x71, +0x8b,0x7a,0x95,0xb4,0x88,0xa3,0x8a,0x66,0x84,0xc8,0x41,0x38, +0x1c,0x41,0xc8,0xfd,0x36,0x10,0xa7,0x39,0x04,0x47,0x29,0xf3, +0x30,0x76,0xf2,0x6c,0xee,0x12,0xed,0x50,0xd4,0xfb,0x1a,0xfa, +0x05,0xe5,0x58,0x45,0x51,0x33,0xd8,0xa7,0x15,0x34,0xd4,0xa7, +0x35,0x4e,0x8d,0x76,0xe1,0x05,0x23,0x56,0xc6,0x42,0x1e,0xf0, +0x28,0x38,0x4c,0xbb,0xbb,0x3d,0x3e,0xee,0xe9,0x2f,0x1a,0xb8, +0xc4,0x64,0x3c,0xfe,0xf6,0x9b,0x6f,0x1e,0x7e,0x63,0x98,0xaf, +0x43,0x30,0x74,0xa0,0xcb,0xe3,0x4e,0x85,0x20,0x9e,0xa1,0xda, +0xe5,0x02,0xa8,0x91,0x21,0x91,0x33,0x6f,0xb7,0x76,0x76,0xe5, +0x12,0x72,0x0b,0xfe,0xc1,0xe6,0xb3,0xe1,0x20,0x85,0x14,0x9a, +0xa7,0xf8,0x69,0x5f,0x9d,0xae,0xb3,0x16,0xd0,0x8b,0x1d,0x42, +0x64,0x57,0x80,0xce,0x9a,0x4b,0x01,0x0d,0xd5,0xb6,0xc9,0xbf, +0x4f,0x94,0x35,0x4d,0x58,0xf3,0x98,0x86,0x56,0xa2,0x8e,0x10, +0x58,0xba,0xcd,0xc7,0xd2,0xed,0x9f,0xf4,0xe0,0x0d,0x7b,0x25, +0x1f,0xff,0xf7,0xf6,0x4a,0x3e,0xfe,0x9a,0x5e,0xd9,0xa8,0xed, +0x15,0xd7,0x29,0xff,0x37,0xfb,0xe4,0xf1,0x5f,0xd9,0x27,0xc5, +0xd5,0xfc,0xe2,0x62,0x34,0x0c,0x84,0x4d,0x58,0xfb,0xfe,0x78, +0x40,0x54,0x64,0xbb,0x8a,0xc1,0x06,0x82,0xb1,0xab,0xa9,0xb6, +0x9e,0x9c,0x7d,0x64,0x76,0xe1,0x67,0x72,0xcd,0x1e,0xc0,0x1b, +0x89,0x45,0x41,0xac,0xf5,0x76,0xab,0x97,0xd9,0x18,0x00,0xed, +0x29,0x3f,0x48,0x6b,0xc4,0xd2,0xe5,0x43,0x6a,0xfb,0xd7,0x37, +0x01,0xb1,0x6a,0xac,0x2e,0xe7,0xfd,0xa9,0xb3,0xf5,0x1d,0x9e, +0xba,0x1e,0xef,0x2d,0x16,0x08,0x01,0x9b,0x6c,0xd7,0xd2,0xe5, +0x46,0xce,0x76,0x2b,0xd1,0x6b,0x9c,0x69,0x09,0x6c,0x00,0x12, +0xa7,0xc6,0xbb,0xca,0xcc,0x6e,0x70,0xd8,0x63,0xdc,0x1a,0x8d, +0x26,0x93,0x0f,0xf3,0x9b,0x17,0x86,0x1d,0xa8,0x59,0x8c,0xbd, +0xdd,0x83,0x92,0x3a,0xd4,0x3d,0x75,0x72,0x3d,0xf2,0x59,0xd5, +0x5b,0xc2,0x8e,0x5b,0x4c,0xa6,0xb3,0x1f,0x02,0xee,0xc4,0x74, +0xab,0x3c,0x69,0x1a,0x22,0x7c,0x9a,0xb8,0x3c,0xb7,0x59,0xe1, +0xce,0x12,0xd7,0x6d,0x20,0x15,0x9c,0x84,0x37,0x92,0x30,0x39, +0x65,0xbf,0xff,0xe8,0x01,0x2d,0xef,0xa7,0xeb,0x32,0xad,0xcb, +0x04,0x2b,0xe3,0x14,0x5d,0x47,0xc3,0x8b,0x99,0x9a,0xe2,0xf9, +0x95,0xb6,0xe2,0x0c,0x62,0x9a,0x06,0xb1,0x3a,0xcb,0x30,0xd1, +0x46,0xc0,0x94,0xed,0xeb,0x3d,0xe3,0x0c,0x57,0x87,0xfe,0xfe, +0xd9,0x62,0x41,0x5b,0x13,0x58,0xc2,0xb4,0x53,0x16,0xa0,0xce, +0xf6,0xfb,0x22,0xed,0xcc,0x2e,0xb2,0xe6,0x4e,0x00,0xcb,0xa1, +0xd1,0x47,0x45,0xe0,0xf7,0x7f,0xe8,0x31,0x16,0x91,0xc0,0x86, +0x3b,0x18,0xbd,0x82,0xba,0xb6,0x3f,0x1b,0x5e,0xf5,0x7f,0xcf, +0x85,0x51,0xde,0xbb,0x7a,0xc5,0xcc,0xc6,0xa5,0x5d,0xe0,0x32, +0x62,0x89,0x10,0xb6,0xe3,0xf8,0xc7,0x74,0x65,0xb7,0xad,0x5c, +0x57,0xec,0xe1,0x7a,0x0d,0x1e,0x0f,0x17,0x3e,0x53,0x0b,0x5e, +0x2b,0x80,0xb3,0x50,0xc1,0x49,0x07,0x47,0x1e,0x56,0x5e,0x0f, +0x3d,0xfc,0xf5,0xd4,0x93,0xfd,0x5c,0xb7,0x31,0xf9,0x02,0x71, +0x09,0x09,0xb3,0xbb,0xb8,0xaf,0xa7,0xe2,0x1b,0x44,0x1c,0xc1, +0xd1,0xea,0x94,0xda,0x7e,0x9d,0x72,0x24,0x22,0xb3,0x26,0x00, +0x4b,0x3a,0x1f,0xcf,0x56,0x65,0x87,0x27,0xcb,0x2b,0xe8,0x6b, +0x34,0x3c,0x0a,0x5b,0x88,0x14,0x06,0xea,0x70,0x80,0xec,0xa8, +0xeb,0x7e,0x94,0xf0,0xa9,0xd5,0xc7,0x03,0xee,0x5f,0xbb,0xb0, +0xd7,0x77,0xb1,0x93,0xa3,0x18,0x46,0x89,0xd8,0xd7,0x5a,0x09, +0x2d,0xbc,0x99,0xf9,0xa8,0x57,0xe0,0x2b,0x3d,0x62,0x33,0xa4, +0xc5,0x88,0x5c,0x20,0x66,0x5f,0x27,0xb6,0xd9,0x31,0x53,0x3e, +0xd0,0xcb,0xc0,0xc7,0x06,0x46,0xef,0xef,0xeb,0x19,0x51,0x89, +0x12,0x51,0x74,0x35,0x70,0x2f,0x79,0x4c,0xe3,0x11,0xf0,0xeb, +0x70,0xa3,0x95,0x62,0x19,0xfa,0xd3,0x4e,0x98,0xc9,0x47,0x58, +0x75,0x67,0x93,0xc0,0xec,0x63,0xb0,0x70,0x75,0xfc,0x7d,0xba, +0x23,0xf8,0x6a,0x08,0xa7,0x75,0xf2,0xf3,0x8e,0x5b,0x90,0xc4, +0x99,0x2a,0xb5,0x4f,0xe4,0x31,0x73,0xb7,0x87,0x0b,0xbf,0xe7, +0x96,0x5b,0x96,0xef,0x0e,0x1b,0xbb,0xf5,0x05,0xb9,0xcf,0x54, +0x88,0xa2,0x8c,0xf8,0x09,0xd9,0x4d,0xcd,0xe0,0xea,0x4d,0x0b, +0x2e,0xd3,0xf5,0xcf,0xac,0x2f,0x0f,0x0f,0x34,0x02,0xc2,0x7d, +0x86,0x0a,0xc6,0x34,0xc3,0x2b,0x53,0xd4,0xd8,0xec,0x37,0xd4, +0xd4,0xbb,0xbd,0xd4,0x3b,0x68,0x00,0x2e,0xd8,0x4d,0xe8,0x98, +0x42,0xe2,0xa5,0x3b,0x8a,0xaa,0xc8,0x2f,0x47,0xc2,0x76,0x1c, +0x14,0xac,0x4f,0x5d,0x09,0xe2,0x87,0xdb,0xa9,0xbf,0xaa,0x1e, +0x5e,0x89,0xad,0x8a,0x4a,0xd9,0xbd,0xd2,0x83,0x1c,0xab,0xdd, +0x84,0x25,0x28,0xd8,0xc2,0xfa,0x20,0x09,0x72,0xf0,0xc1,0x74, +0x72,0xf3,0xc5,0xd5,0xae,0xa8,0xe6,0x12,0xd7,0x02,0xf4,0xaa, +0x1b,0xa0,0x2b,0x4b,0x63,0x08,0x89,0x18,0x6a,0xb4,0xdc,0x5f, +0x8c,0xc0,0x95,0xe0,0x58,0x1a,0x62,0xd7,0xfc,0x12,0xf8,0xc9, +0x1d,0xe9,0x09,0x00,0x7e,0xea,0x75,0xd0,0x11,0x45,0xd1,0x68, +0xf7,0x96,0x04,0x1d,0x08,0x6d,0xa7,0x00,0xda,0xd6,0x3e,0xef, +0xcf,0x98,0xf3,0xa4,0xdc,0x0a,0x81,0xcc,0x59,0x9e,0x72,0x84, +0x7c,0x43,0x60,0xa4,0x3b,0x59,0x2c,0x28,0x86,0xb9,0x19,0xc3, +0x43,0x70,0xf9,0x1d,0x58,0x4f,0xfd,0x32,0x08,0x20,0xe5,0xea, +0xf0,0x66,0x11,0xd4,0x22,0xa5,0xdf,0x60,0x35,0xe6,0x44,0x58, +0xfe,0x4b,0x8d,0x41,0x8d,0xc6,0x68,0xdc,0x16,0xc8,0x85,0x78, +0xa9,0xe0,0xb0,0x14,0xcf,0x9d,0xb9,0x3e,0xe7,0xce,0x57,0xf4, +0xc7,0x20,0xbf,0xb8,0xd0,0xe7,0xb6,0xb1,0x66,0xb1,0x18,0x41, +0xd5,0x49,0xbc,0x45,0xe3,0x66,0x3e,0xce,0x7f,0x23,0x63,0x54, +0xbf,0xcd,0x4b,0x93,0x32,0x2f,0x8e,0x70,0xcd,0xae,0x58,0x9b, +0x3d,0x06,0xcc,0xc0,0xb1,0x71,0x72,0x80,0xb0,0x4b,0xae,0xdb, +0xa5,0x1d,0x36,0xfe,0xc8,0x36,0x5b,0x24,0xe3,0x31,0x13,0x55, +0xac,0xf0,0xb0,0x74,0x31,0x11,0x25,0x99,0x0d,0xc5,0x3b,0x61, +0xa8,0x2a,0x86,0xba,0x4d,0x03,0xde,0x99,0x71,0xd6,0x6c,0xef, +0x96,0xe4,0x0e,0x46,0x6c,0x6d,0x01,0x8a,0x2e,0xfc,0x71,0x73, +0x30,0x63,0x96,0x22,0x95,0xee,0x39,0x35,0x04,0x0f,0x05,0x3d, +0x84,0x30,0x28,0x7b,0x5b,0x79,0xa2,0x4e,0x9a,0xd9,0x24,0xeb, +0x96,0x37,0x5f,0xdc,0xee,0x52,0x79,0x4b,0x74,0x1f,0x74,0x45, +0x6c,0x87,0x8a,0xe8,0xb1,0x4d,0x9e,0xea,0xe8,0x9d,0xa0,0x08, +0x2d,0x6b,0xf3,0x28,0x60,0x0e,0x69,0x56,0x2d,0x7e,0x69,0x09, +0xae,0x95,0xa7,0x2d,0x17,0x48,0xd9,0x5d,0x6b,0xe9,0x56,0x97, +0xca,0x79,0x34,0x19,0x01,0xb1,0x30,0x5d,0x32,0xbb,0x1a,0x4e, +0x25,0x47,0xce,0x82,0x25,0x8c,0x57,0x88,0x60,0x3f,0xdb,0x5d, +0xb2,0xf8,0xc5,0x8d,0xca,0x7a,0x9a,0xbd,0xb9,0xed,0xd4,0x00, +0x6a,0x07,0x70,0xa9,0x32,0x34,0x32,0xaa,0xcf,0x0c,0xd2,0xcc, +0x3e,0x56,0xc4,0x4c,0x50,0xa0,0xee,0x8f,0xfc,0x46,0x76,0x86, +0xe1,0x1e,0x78,0xbb,0x43,0x91,0x47,0x6c,0x18,0x7d,0x47,0x42, +0x44,0xe9,0x51,0xc2,0x8a,0x20,0xb1,0xee,0x21,0x3b,0x2e,0xe1, +0xfe,0x82,0x56,0x1c,0xbe,0xb0,0x81,0xeb,0x64,0xff,0xee,0x98, +0x41,0xbb,0x39,0x3c,0xc7,0x2d,0x21,0x8f,0x1a,0xb9,0x34,0xfa, +0x61,0x46,0xce,0xe4,0xcc,0x97,0x95,0xc3,0x41,0xc0,0x7a,0xc9, +0xb3,0x37,0x60,0x78,0x3c,0xe0,0xd3,0xc6,0xd2,0x72,0x43,0x82, +0x77,0x76,0x57,0xdb,0x00,0x5a,0x79,0xb5,0x4d,0x48,0x0d,0x5f, +0x07,0x60,0x70,0x97,0xcd,0xee,0xf7,0xf4,0x67,0xea,0xa7,0xe8, +0xbd,0xb8,0x97,0x99,0xef,0x56,0xcf,0x27,0x7d,0xa9,0xec,0x00, +0x29,0xad,0x32,0x33,0x34,0x2e,0xce,0x4b,0x88,0xab,0x02,0x8d, +0x0e,0x73,0xcc,0x68,0x07,0x9a,0x00,0x1e,0xb3,0x06,0x97,0xf0, +0x12,0x41,0x34,0x9e,0x5f,0x9f,0x0d,0xa7,0x4e,0xb3,0xc2,0xa6, +0x9a,0x9b,0x9c,0xcc,0xe3,0x47,0x05,0x25,0x09,0xf3,0x70,0x39, +0xcc,0x7f,0x88,0xe8,0xe4,0xa9,0x2e,0x5d,0x97,0xbc,0x6f,0xd7, +0x0c,0x71,0x6e,0x25,0x02,0x1a,0x16,0x7f,0x6a,0xbe,0x96,0x56, +0x57,0xc2,0x4a,0x67,0x89,0xe4,0x1a,0xf9,0x6c,0x62,0x0f,0xd8, +0x02,0x28,0xf6,0xdb,0x06,0xfb,0xb1,0xa4,0x7e,0x10,0xd0,0x6b, +0xab,0xd8,0xb6,0x27,0x34,0xe6,0x55,0x5e,0xac,0x68,0x7f,0xb0, +0x34,0xb3,0xba,0xed,0x35,0xbb,0x4f,0xbc,0x3b,0xa9,0x83,0x40, +0x86,0xb6,0xad,0xa3,0xd0,0xdf,0x31,0xf5,0x94,0x25,0x9a,0xba, +0x0a,0x30,0x43,0xa6,0x41,0xdb,0x29,0xe5,0x8a,0x1d,0x61,0x69, +0x75,0xa2,0x63,0xf3,0xf3,0xcc,0xe2,0x81,0x0c,0xa9,0x3f,0x3a, +0xb6,0xb7,0xdb,0x6b,0x37,0xd4,0x14,0x0c,0xde,0x4b,0x3f,0xc9, +0xe8,0xc4,0x7f,0x36,0xb9,0xd1,0x7f,0x86,0x37,0xc9,0x6d,0x49, +0x00,0x92,0xb5,0xf4,0xae,0x07,0x00,0x19,0xc2,0x2e,0x16,0xbb, +0xe4,0xf9,0x3f,0xdb,0x4d,0x30,0x8b,0xbb,0xa8,0xee,0xee,0xf5, +0x16,0x8b,0x56,0x3b,0x58,0x5c,0xec,0x60,0xc2,0x8f,0xf3,0x61, +0x3e,0x8a,0x11,0xdd,0x36,0x62,0x49,0x76,0xb0,0x58,0xcd,0xe6, +0xa9,0x7c,0x00,0xb2,0x22,0xa2,0xaf,0xb4,0xb8,0xd8,0x51,0x31, +0xf8,0xd4,0x4e,0x10,0x06,0xde,0x8b,0x37,0x1a,0xbd,0xcc,0xd4, +0x40,0xff,0x6d,0x64,0x80,0xcc,0x4e,0x4c,0x80,0x22,0x26,0xed, +0xbc,0x64,0xf8,0x9e,0xae,0xb8,0xa5,0xb5,0xfd,0xf1,0xb9,0x7f, +0x0e,0x07,0xd1,0xba,0x3a,0xd3,0x07,0xc5,0x81,0x1b,0x06,0xa0, +0x6a,0xb5,0xb5,0x05,0xb0,0x94,0xdd,0xf4,0x3b,0xc4,0x9b,0xe6, +0x76,0x31,0xbc,0xd0,0x23,0xea,0xda,0x25,0x9e,0xfc,0x62,0x0a, +0x66,0x03,0xc0,0x93,0xaa,0x3b,0xe2,0xb6,0x13,0x51,0xd5,0xdf, +0x3e,0x20,0xc1,0xb2,0xaa,0x80,0x3f,0x0e,0x55,0xbc,0x10,0xca, +0x4a,0x2c,0xb1,0x3e,0x44,0xab,0x3b,0xe7,0x5d,0x16,0x66,0xd1, +0xaf,0x2a,0x2a,0xd1,0x94,0x9f,0x7b,0x46,0xf2,0xb1,0x4c,0x17, +0x6c,0x93,0xfe,0xed,0xe8,0x82,0xd4,0x6d,0x35,0x68,0x08,0x8f, +0x36,0x28,0xc5,0xa2,0x2d,0xa8,0x80,0x7c,0x6b,0x91,0xc0,0x15, +0x20,0x9d,0x38,0x3e,0xac,0x83,0xfe,0x11,0x7d,0x99,0x0b,0x34, +0x40,0xb7,0xc4,0x2b,0x12,0x3d,0x46,0xbc,0x03,0x13,0xb6,0xf5, +0x1d,0x57,0x3a,0x8e,0xa7,0x28,0x73,0x33,0x82,0x66,0x54,0xf2, +0x5c,0x83,0x66,0x24,0x04,0x06,0xcf,0x93,0xe0,0xea,0x95,0xe8, +0x80,0x98,0x7a,0x42,0x74,0x67,0x82,0x70,0x0a,0x81,0x8c,0x1a, +0x59,0xc9,0x5f,0x2a,0x23,0xdf,0xb8,0x9e,0x17,0xb3,0x8d,0xb3, +0xe1,0xc6,0x4d,0x1f,0x1e,0xe3,0x5a,0xea,0x37,0x40,0xbb,0xbc, +0x88,0x7c,0x41,0x11,0x62,0x74,0x8c,0xc5,0x45,0x72,0x0b,0x22, +0xc5,0x0b,0x67,0x20,0x04,0x43,0x28,0x51,0x00,0x8e,0xff,0xec, +0x57,0x94,0xc8,0x0f,0xaf,0x27,0xde,0xc9,0x1a,0x47,0xba,0x5e, +0xa8,0xae,0x8c,0x6e,0x38,0x2a,0x44,0x38,0xab,0x97,0x94,0xa4, +0xf9,0x4a,0xfa,0x90,0x77,0xa1,0xa1,0xe2,0x2c,0xc8,0xa6,0x08, +0x2a,0x68,0x5f,0x6e,0x48,0xc7,0x1a,0x81,0x98,0x06,0xb5,0x32, +0x50,0x48,0x03,0x5f,0x24,0x42,0xb2,0x5f,0x59,0x45,0x2f,0x31, +0x16,0xec,0x8c,0xc1,0x70,0x24,0xc5,0x13,0x58,0x89,0x8f,0xfd, +0x7c,0x76,0xf7,0x5d,0x9f,0x21,0xa2,0x18,0xce,0x8e,0xf3,0xeb, +0xa1,0x3e,0xb5,0xc4,0x2b,0xc7,0x08,0x3e,0xcb,0xa6,0x2b,0x3b, +0x2a,0x81,0x8a,0xbf,0x90,0x2e,0x7e,0x68,0x48,0xba,0xc3,0x0e, +0x10,0xc7,0xd9,0x4f,0x55,0x17,0x89,0x6b,0xf5,0x56,0x8d,0x31, +0x58,0x55,0x10,0x31,0x0c,0x8c,0xd9,0x6c,0x14,0xf6,0x0f,0x14, +0x6c,0xcc,0xdc,0x9b,0xab,0x05,0x37,0xe9,0x99,0xbd,0x53,0x33, +0xaa,0x4f,0xc6,0x2f,0xc9,0x49,0x77,0x59,0xf3,0x77,0x9c,0x51, +0xf7,0x21,0x7f,0x65,0x60,0x0f,0x01,0x97,0xfb,0xfe,0x6c,0xe8, +0x2d,0xb2,0x36,0x17,0x43,0xea,0x01,0xdb,0x1f,0x80,0x16,0x31, +0x3c,0xf1,0xec,0xec,0xa2,0x2a,0xe9,0x81,0xce,0xe4,0x97,0x55, +0x5e,0x0d,0x24,0x79,0xc9,0xb2,0x62,0x32,0xe2,0x2b,0x84,0xc9, +0xc7,0xcc,0x20,0x6c,0x9b,0x92,0x17,0x8b,0xa0,0x68,0x7c,0x5d, +0xaa,0x69,0xb7,0xa4,0xe9,0x6c,0xe6,0x1a,0xfe,0x5a,0x73,0xce, +0x40,0x1d,0xb4,0xcf,0x76,0xac,0x13,0xb6,0x0d,0x94,0xb4,0xb0, +0x86,0x67,0x3f,0x3b,0xd7,0xdd,0xde,0xa7,0x76,0xf7,0x33,0x8b, +0xa3,0x13,0x9f,0xeb,0x02,0xa7,0x66,0x4c,0x70,0xfd,0x92,0x9a, +0x46,0xd5,0x45,0xdd,0x55,0xeb,0x24,0xe5,0xac,0xae,0x46,0xb3, +0x69,0x3f,0x1f,0x99,0xd6,0x04,0xff,0x49,0x8c,0x5b,0x8c,0x45, +0xec,0x11,0x65,0xa9,0x4a,0x9c,0xa4,0x16,0x87,0x1d,0x6c,0x02, +0xe7,0x95,0xa3,0x23,0xbf,0xbe,0x1e,0x0e,0xf2,0xbe,0xb1,0x18, +0xc5,0xa8,0x69,0x7c,0x18,0xba,0x20,0x52,0xef,0x27,0xd7,0x37, +0xca,0xd7,0x36,0x12,0xfd,0xb2,0xaa,0xb9,0x6c,0x76,0xd7,0x6b, +0x95,0x83,0x88,0x22,0xf5,0xd9,0xc9,0x80,0x6d,0xdb,0x9c,0x6d, +0xa0,0x75,0x1f,0x12,0x3b,0xb5,0x95,0xc7,0xce,0x04,0x90,0x24, +0x8d,0xbd,0xd6,0xb7,0x55,0x74,0xea,0x0e,0xb5,0x8d,0xcf,0x8f, +0x8e,0x5f,0xe9,0x21,0x66,0xb3,0x6d,0x6d,0x6d,0x32,0x3e,0x6b, +0xfd,0xd4,0xf4,0x50,0x7d,0x57,0xe0,0x9c,0x4f,0x0c,0x32,0x77, +0xf3,0x56,0x5b,0xb2,0xec,0xb0,0x49,0xa9,0xb3,0xdc,0x1a,0x0b, +0x5c,0x93,0x50,0x21,0x29,0x2f,0x42,0x3a,0x1d,0x97,0xc6,0x34, +0x46,0xc8,0x5d,0xd4,0x63,0x5b,0xb1,0x30,0x92,0x85,0x5a,0x6c, +0x29,0xd4,0xe8,0x43,0x0a,0x40,0x1f,0x3c,0x1c,0x2e,0xa8,0x22, +0x5e,0x71,0xb9,0x21,0x16,0x4f,0x5c,0xb4,0x9c,0xd3,0x1d,0x27, +0x79,0xc2,0xf1,0x24,0xca,0x64,0x64,0xe1,0x86,0x4a,0x85,0x83, +0x98,0x6e,0x22,0x5d,0x55,0xc8,0xfd,0xb2,0x64,0xaa,0xb7,0xe2, +0xac,0x1b,0x0c,0xc0,0xdc,0xdb,0x45,0xe1,0x04,0xa4,0x0f,0xf3, +0xc0,0x4a,0x27,0x8e,0x68,0x38,0x04,0x96,0xa8,0xe9,0x49,0x3e, +0x4c,0x9f,0x09,0x91,0xba,0xfe,0x85,0x37,0x6e,0x71,0x8c,0x2a, +0x6f,0x29,0x2f,0xf5,0xc9,0x36,0x8d,0xe4,0xc7,0xad,0x4e,0x7d, +0x37,0x18,0x8d,0x1a,0x28,0x02,0x55,0x21,0xdd,0x43,0x00,0xe1, +0xd9,0x0f,0x99,0x07,0xd2,0x34,0xd7,0x0b,0x1c,0x33,0x43,0x78, +0x25,0x5d,0xa5,0x69,0x18,0xbd,0x18,0xa3,0xe7,0x74,0x76,0x21, +0x16,0xd1,0xfa,0x87,0xd8,0xbb,0xbd,0x76,0xa0,0x80,0x01,0x58, +0x68,0x8b,0x35,0xfa,0x24,0x5b,0x5b,0xa8,0x7b,0x89,0x02,0x20, +0x08,0x9b,0xd6,0x80,0xd8,0xa5,0x35,0x50,0x1b,0x70,0x36,0xb5, +0x1a,0xe8,0x65,0x75,0x4e,0x3e,0xea,0xaf,0x23,0x5e,0xb0,0x07, +0xf5,0x4c,0xea,0xa2,0xb7,0xa5,0x3e,0x4b,0x41,0x6c,0x5f,0x3e, +0xfd,0x33,0x14,0x51,0xfe,0x35,0x08,0x42,0x40,0xa0,0xa7,0x2b, +0xf4,0xe1,0x2d,0x61,0xc2,0xef,0x94,0x06,0x63,0x3d,0xa3,0xe1, +0x74,0x56,0x43,0x99,0x93,0x65,0x7c,0x95,0xde,0x3e,0xcb,0x2e, +0x3c,0x02,0x32,0xfe,0x2a,0x8b,0x2e,0x0c,0x09,0x50,0x0e,0x29, +0x1a,0x85,0x0d,0x86,0x9b,0x2c,0x70,0x8e,0xb5,0xa3,0x44,0x9c, +0x61,0xb8,0xd4,0x9e,0x1e,0x2d,0x98,0xa7,0x3c,0x5c,0x28,0x1a, +0xef,0x76,0x49,0x21,0x11,0x8d,0x23,0x54,0x5f,0xd2,0x20,0x87, +0x5a,0xc3,0xea,0x38,0xbe,0x91,0x1e,0xf1,0xe3,0x44,0xe0,0x4f, +0xeb,0x56,0x6c,0x3a,0xb9,0x01,0x32,0x39,0x1a,0x88,0x23,0x1f, +0xe8,0x14,0x61,0xfd,0x9e,0x33,0x5b,0x0b,0x7e,0x04,0x2a,0x2a, +0x8f,0x4f,0xf7,0x4c,0x6f,0x7c,0xad,0x24,0x0f,0xab,0x82,0x6f, +0x77,0x6c,0x01,0x78,0xdb,0xe8,0x1a,0x12,0x45,0xcc,0x37,0x9f, +0x89,0x73,0xb5,0xed,0x88,0x7e,0xec,0x6e,0x3e,0xe3,0x3e,0x70, +0x9d,0x87,0x43,0xe6,0xcf,0x93,0x57,0xd9,0xa1,0x56,0x9c,0x88, +0x04,0x03,0x9d,0x7a,0x73,0xab,0x20,0xce,0xb1,0x47,0x44,0xa1, +0x79,0x83,0xfb,0xdf,0xd2,0x9b,0xf6,0xc6,0xde,0x76,0x2b,0x69, +0xc6,0x57,0x74,0xb1,0x38,0xba,0xa0,0xbf,0x8f,0x6a,0xfa,0x60, +0x20,0x8b,0x35,0x34,0xbc,0x9e,0x44,0x5d,0x4b,0xd4,0xef,0x80, +0xbb,0x46,0x67,0x66,0xc4,0xdc,0x55,0x22,0xe7,0x2d,0xf7,0x49, +0xbc,0xfb,0xad,0x70,0xce,0x2b,0xbd,0x15,0xe3,0xac,0x86,0xcc, +0xf4,0x8e,0xe9,0x37,0x21,0xc2,0x52,0x67,0xaa,0x7f,0x34,0x03, +0x7b,0x11,0x67,0xf8,0x43,0x7a,0x0c,0x42,0x43,0x61,0x63,0x57, +0x2f,0xfa,0xfd,0xc5,0xa2,0xb5,0xa3,0x63,0x5b,0x3b,0x67,0xee, +0x95,0x80,0x8e,0xa4,0x0f,0x0b,0x8a,0xf9,0xda,0x7d,0xef,0xb5, +0x97,0x6e,0xae,0x7e,0xd6,0xb7,0xaf,0xbd,0x12,0x75,0x16,0x26, +0x9f,0x65,0x67,0x2e,0x99,0xe4,0x29,0x9a,0xa5,0x2a,0x5e,0x81, +0xc1,0x6d,0xf3,0x10,0x90,0xfb,0x12,0x8f,0xa8,0x36,0x75,0x33, +0x48,0x3e,0x4b,0x9c,0xc2,0x5e,0xf1,0x31,0x9f,0xe9,0x81,0x60, +0x61,0x35,0xbb,0xd8,0x2f,0x86,0x51,0x97,0x76,0xa5,0x0d,0xca, +0xd6,0x8b,0x52,0x47,0x39,0xdb,0x46,0x3c,0x03,0x63,0xde,0x02, +0xf2,0x15,0x8a,0x42,0x05,0xe4,0x66,0xd6,0xe8,0x77,0xce,0xf4, +0xdf,0xb3,0x54,0x8f,0x8b,0x7e,0xc7,0x34,0x4c,0xaa,0x7f,0x1a, +0x67,0x7e,0x66,0xe0,0x28,0x75,0x56,0x2f,0x8e,0x1d,0xd2,0x5a, +0x8c,0x8d,0x8a,0x7c,0x6f,0x87,0x97,0xcf,0x3e,0xdd,0x88,0x42, +0x9b,0xec,0x86,0x44,0x37,0x15,0x7d,0x6d,0x6d,0xf5,0x9b,0xf4, +0x50,0x14,0xe2,0xe8,0x0b,0xe2,0xae,0xf5,0xcc,0x00,0xc6,0x1d, +0x41,0x6d,0x00,0x52,0xf2,0xcb,0xf1,0x64,0x3a,0x04,0x7b,0xe5, +0x90,0xe4,0x42,0x20,0x69,0x8d,0x78,0xb3,0xb6,0x02,0x5f,0xdd, +0xb7,0xa5,0xb8,0x33,0xd7,0xba,0x81,0x0c,0x8e,0x46,0x90,0xbf, +0x4f,0x18,0xa1,0x21,0x26,0x75,0xf9,0x2d,0x84,0x50,0x8a,0x39, +0x0b,0x12,0xce,0x48,0x05,0xfa,0x29,0x88,0xd2,0xfa,0xb0,0x36, +0x14,0xb3,0xe9,0x1c,0xbd,0x9a,0x9f,0x61,0xdc,0x99,0x8c,0x43, +0xb5,0x1b,0x88,0x06,0xd5,0x9b,0xa7,0xa8,0x2d,0xb7,0x19,0x7b, +0x5b,0x04,0xa6,0xea,0xfd,0x01,0x7f,0xe5,0x88,0xeb,0x3f,0xb5, +0xca,0x75,0x16,0xf8,0x8c,0x81,0xcf,0x42,0x60,0x4a,0x70,0xf5, +0xe6,0x8a,0xd2,0xf5,0x54,0xc2,0x53,0x86,0x82,0x67,0x34,0x74, +0x51,0x17,0x60,0x57,0x39,0x8d,0x75,0x68,0x5d,0xd3,0xad,0x38, +0xcf,0x7b,0x51,0x96,0x89,0x61,0x09,0x2b,0x10,0xe4,0xe9,0x1b, +0x76,0x80,0xb3,0x62,0xa4,0xae,0x35,0x73,0x03,0x28,0xba,0x86, +0xb8,0xed,0x6d,0xc7,0xd8,0x0f,0x7f,0x8b,0xfb,0x5d,0x88,0xec, +0xa9,0x33,0xfe,0xf5,0x27,0x73,0xd2,0x4e,0xda,0xf8,0x48,0xaf, +0xa4,0xf9,0x0a,0x7d,0x43,0x9c,0x57,0x9f,0xf9,0x2e,0x24,0xa3, +0xd1,0x50,0x9b,0x06,0x3b,0x25,0x9f,0x71,0x32,0x96,0x05,0xeb, +0xb1,0x2e,0x0a,0x7f,0x82,0x92,0x92,0x04,0xd5,0xb4,0xa1,0xbe, +0x2c,0x28,0xc3,0x32,0xb9,0xbc,0x33,0x57,0x9e,0x41,0xb8,0x49, +0xb5,0xe1,0x6c,0xa6,0xc1,0x20,0x72,0x69,0xfd,0x79,0x71,0xfb, +0x4e,0x6e,0x62,0xd7,0xd8,0x18,0x60,0xd6,0xa2,0x8d,0x3a,0x92, +0xbf,0xcd,0x3d,0x2d,0x05,0x75,0xe6,0x36,0x83,0xdf,0x70,0x85, +0xeb,0xf6,0xcc,0xed,0x2f,0x6b,0x54,0x66,0x25,0xe6,0xd6,0x6a, +0x6a,0x98,0xce,0xc6,0x9e,0xf3,0x17,0x67,0xba,0xf4,0xe6,0x75, +0x42,0xea,0x64,0xf2,0xfe,0x60,0x26,0x40,0xc5,0x6e,0x67,0x6b, +0x6f,0xf8,0xdc,0x0a,0xd5,0x62,0x22,0x8f,0x9f,0xa7,0x57,0xed, +0x1d,0x9b,0xf1,0xe6,0x04,0x2c,0xb2,0xb5,0x36,0xa9,0xb8,0xf1, +0x64,0x30,0x04,0xc6,0x9b,0x6b,0x46,0x7a,0x30,0xde,0x3b,0xdf, +0x90,0x8b,0x27,0x3c,0xe5,0xf1,0xe8,0xaf,0xa3,0x28,0x86,0x53, +0x6e,0xa3,0xaa,0xd5,0xad,0xc9,0xbc,0xe3,0xc0,0x52,0xb1,0x17, +0x06,0xab,0x16,0x10,0xa9,0xc8,0xcc,0x31,0xfd,0x49,0x45,0xe8, +0x0f,0x5a,0x52,0xf5,0x07,0x2c,0x8f,0xfa,0x87,0x56,0x3b,0xe9, +0xb0,0x61,0x8c,0x53,0xe3,0xb4,0x1b,0xe5,0x45,0xd4,0x20,0x73, +0x15,0x95,0x34,0x94,0xe9,0xce,0x32,0x5b,0x3b,0xca,0xd9,0x88, +0x7a,0xd1,0x92,0x94,0xba,0x84,0xbe,0x82,0x3b,0x03,0x69,0x9e, +0xc3,0x4b,0x5a,0xd9,0xf2,0x9b,0xae,0x13,0x23,0x28,0x72,0x38, +0x8c,0xf0,0x66,0x3a,0x32,0x99,0xec,0x6a,0xb9,0xd3,0xdc,0xd1, +0x53,0x4a,0xae,0x32,0x95,0x88,0x5d,0xc6,0xcc,0x2c,0xb3,0xb0, +0x4f,0x13,0xb9,0xcf,0xe1,0x12,0xbe,0x86,0x8d,0x30,0xa9,0xac, +0x7c,0xbc,0x99,0xeb,0xe5,0xe4,0x55,0x8c,0x56,0xeb,0x9e,0x8f, +0x26,0x7d,0x3e,0xa0,0x51,0x2f,0xea,0x94,0x7a,0x5e,0x84,0x3a, +0xc3,0xe9,0x30,0x6f,0xa2,0x6e,0x12,0x65,0xe4,0xad,0x6a,0x55, +0xff,0x6f,0xee,0x2e,0x16,0xfc,0xd5,0xd2,0x7b,0x46,0x69,0x93, +0xab,0x1f,0x5b,0xe0,0x2d,0x7b,0x95,0xd6,0x54,0x66,0xc9,0xb0, +0x0e,0x90,0xab,0xc1,0x25,0x8b,0x06,0x19,0x74,0x0f,0xd5,0xbf, +0x53,0xf0,0x0d,0x07,0x58,0x9a,0x10,0x06,0x32,0x8f,0x27,0x60, +0x78,0x41,0x33,0x60,0x15,0xfe,0xe0,0x37,0xf8,0x81,0x78,0xd9, +0xd6,0x02,0x7b,0x27,0x76,0x02,0xe7,0x3a,0xc5,0x5b,0xab,0xda, +0x8c,0x87,0x73,0x07,0x35,0xae,0xd0,0x11,0xb1,0x92,0x85,0xf3, +0xf3,0xf9,0x35,0xbd,0xd8,0x8f,0x7d,0x95,0x5f,0x3c,0x22,0x8e, +0xf9,0x44,0x86,0x60,0x70,0x3a,0xac,0x56,0xad,0x73,0x97,0xe1, +0x08,0xc8,0x77,0x75,0x83,0x89,0xf0,0xd5,0x7d,0x9d,0x8f,0x95, +0xc6,0x1d,0xf4,0x83,0x8e,0xd1,0x23,0x19,0x1e,0x2d,0x60,0x7a, +0x3e,0x86,0x9b,0x39,0xfd,0xd3,0x40,0x5a,0x2e,0x46,0x13,0x4d, +0x26,0x7e,0xb2,0xee,0x72,0xf2,0x00,0x80,0xb7,0x01,0xa2,0x65, +0xb4,0x92,0xa9,0x51,0xc0,0xfc,0xc2,0xed,0xb0,0x38,0xef,0xdf, +0x0c,0xd3,0xdb,0x68,0x2b,0x4a,0xa3,0x2d,0x90,0xb3,0xe9,0x15, +0xe0,0x31,0x7c,0x8f,0x66,0xf0,0xb9,0x0f,0x9f,0x97,0xf0,0x79, +0x3f,0xba,0xaf,0x3f,0x7f,0x9b,0x4f,0x30,0xfe,0x3e,0xc4,0xff, +0xe7,0xa7,0xbd,0xef,0xda,0x7a,0x3a,0xb7,0x2d,0xc2,0xe6,0x7c, +0x4c,0x18,0x33,0x73,0xde,0x8d,0x5d,0x1a,0xa5,0x24,0x82,0x04, +0xbd,0xd4,0x0c,0x3f,0xe9,0x86,0x37,0x64,0xc0,0xa9,0x9b,0x96, +0x1f,0xbd,0x4b,0x47,0x0d,0x3e,0x05,0x97,0x30,0x34,0x7f,0x9d, +0xe4,0xe3,0x38,0x8a,0x12,0x58,0x4d,0x54,0x74,0x19,0x25,0xca, +0x94,0xeb,0xa1,0x88,0x2b,0x50,0x18,0x40,0x83,0x64,0x01,0x58, +0x12,0xc2,0x02,0x1b,0x18,0x2f,0x9a,0x04,0xa4,0xa3,0x0d,0xbc, +0x5c,0x0f,0xcd,0xeb,0xa0,0x53,0xf3,0x4c,0x47,0xde,0xb0,0xc2, +0xe4,0x0a,0xfa,0x8c,0x22,0x3b,0x51,0x94,0x6a,0xa2,0x1b,0x0c, +0xd2,0x9c,0x0e,0x6f,0x46,0x7d,0x7d,0xb2,0xf0,0x9a,0xc2,0x60, +0x14,0x85,0x81,0xc5,0x2a,0xb7,0x85,0x9a,0x8a,0x18,0xc0,0x2e, +0xa6,0xc3,0x2b,0x1e,0x5c,0xa6,0x84,0x8c,0xd1,0x4c,0x38,0xbd, +0x00,0x80,0x95,0x56,0x9c,0x61,0xfe,0xe6,0xaa,0x53,0x12,0x6f, +0xd6,0xb6,0x9d,0x92,0x29,0x25,0x77,0x4d,0xc6,0x5e,0xfb,0x2e, +0x5d,0x75,0x3b,0x79,0x01,0x6b,0x2a,0x1e,0x72,0x7c,0xca,0xc3, +0x25,0x0b,0xdb,0x58,0x88,0x21,0xe8,0x44,0x14,0x6c,0x37,0x46, +0xde,0x97,0x9d,0xb2,0x85,0x24,0x38,0xe6,0xe1,0x97,0xee,0x25, +0x7b,0xb1,0xd8,0x0d,0xb6,0x23,0x29,0x89,0xf4,0xec,0x5a,0xac, +0x23,0x92,0x34,0x8f,0x25,0xa0,0x22,0x28,0x9b,0x13,0xb2,0xba, +0x53,0x16,0xd2,0x2e,0x8d,0x2b,0xde,0x7c,0xf0,0x14,0xf4,0x8a, +0x87,0x53,0x3d,0xe9,0x8d,0xca,0xd9,0x0b,0xb1,0x26,0xea,0x65, +0xe9,0x22,0x67,0xdd,0xeb,0x7c,0x90,0x35,0x1a,0x36,0x43,0x23, +0x8a,0x2c,0x31,0x08,0xd4,0xa1,0x1f,0x0d,0x91,0xe6,0xf8,0x76, +0x61,0x36,0xbc,0xbe,0x19,0xa1,0x4d,0x9a,0xd9,0x4c,0x8f,0x15, +0x98,0x20,0xd0,0xa0,0x3a,0x26,0xdd,0x79,0xfc,0xb7,0xb8,0x7b, +0x52,0x9c,0x1c,0xf5,0x1a,0x9d,0xe4,0x6f,0xfb,0x3b,0x97,0x74, +0xdc,0xbc,0x99,0x8c,0x38,0x35,0x0b,0x92,0x79,0x5e,0xe8,0x94, +0x6d,0x3f,0x85,0xaa,0x31,0x9e,0xa0,0x33,0xef,0x6c,0x27,0x6e, +0x26,0xbf,0xec,0x30,0xb4,0x2e,0x10,0x67,0xf8,0x7d,0x3d,0x01, +0x4e,0x4e,0xf4,0x87,0xfe,0xa3,0xbf,0xa6,0x60,0xa9,0x12,0x3e, +0xc6,0xfa,0x03,0x78,0x88,0xff,0x47,0xff,0x82,0x27,0xa8,0x93, +0xf9,0xde,0xee,0xde,0xdf,0x75,0x80,0x7e,0x39,0xe2,0x7b,0x8e, +0xf8,0x3e,0x5a,0x32,0xde,0x69,0xb6,0x73,0x72,0xb2,0xb8,0xbf, +0x38,0x99,0x2e,0x4e,0xc6,0x8b,0x93,0xd9,0x82,0x32,0xd2,0xcf, +0xf7,0x3b,0x97,0x6d,0x57,0x75,0x21,0x5e,0x85,0xf5,0x12,0x4d, +0xc1,0x16,0xdc,0x1c,0x46,0xc9,0x0a,0x16,0xfc,0xb6,0x89,0xf4, +0xcc,0x81,0x2d,0x2d,0x6c,0x45,0x6b,0xd2,0x0a,0x84,0xf3,0x86, +0x0d,0x44,0xf1,0x82,0xd1,0x8d,0x4d,0x2e,0x5e,0x69,0xf4,0x91, +0x9a,0x9a,0x27,0xe1,0x73,0x9e,0x72,0x10,0xa2,0xd9,0x57,0x81, +0x99,0x8e,0x2b,0xc1,0xf4,0xe4,0x02,0xb4,0xb8,0xc7,0xeb,0x98, +0x7d,0x44,0x43,0x07,0xcc,0xe8,0xf4,0xf4,0xa6,0x91,0xdd,0x8f, +0xda,0xd0,0x08,0x76,0xc5,0x60,0xd2,0x83,0x35,0x82,0xdb,0x58, +0x8e,0x07,0x65,0x4a,0x57,0x93,0x8b,0x8b,0x42,0x3c,0xbb,0x24, +0xf4,0x8d,0x0c,0xd1,0x92,0x80,0x83,0xf5,0xff,0x09,0xce,0x2d, +0x4e,0xd4,0x6f,0x35,0xcb,0x11,0x8c,0x8b,0x06,0x0f,0x19,0xbb, +0x10,0x31,0x1d,0x70,0x84,0xe1,0x52,0xa2,0xfb,0x8d,0x93,0x71, +0x0c,0xc6,0x66,0xb3,0xd8,0xc0,0xeb,0x45,0x57,0x73,0x87,0xb0, +0x14,0x75,0xee,0xdf,0x07,0xd9,0x0a,0xc6,0x02,0x4c,0x92,0x68, +0xe8,0xfb,0xd8,0x16,0xb6,0x22,0xb5,0xc8,0x04,0x4c,0x80,0x51, +0x23,0x62,0x3c,0xa6,0x11,0x3c,0x24,0x6d,0x3d,0x88,0x1b,0x26, +0xa5,0xa1,0x87,0x34,0xb7,0xb4,0xe9,0x02,0x6a,0x88,0x06,0x56, +0xca,0x9c,0x05,0x31,0x00,0xcf,0x8c,0x3c,0x2c,0x76,0xa8,0x35, +0xf5,0xb0,0xca,0xfb,0x67,0x23,0xb8,0x65,0x32,0xfd,0x07,0xfa, +0xad,0x31,0x32,0xab,0xb7,0xcb,0xe4,0x16,0xca,0xe4,0xcc,0xd1, +0x52,0x07,0x12,0xdb,0xcf,0x6c,0x75,0x5a,0x69,0x2a,0xb2,0xfb, +0xf7,0xf5,0xcf,0xaf,0x25,0xef,0x82,0x30,0x5c,0xf4,0x42,0x9e, +0x8f,0x3d,0x5e,0x08,0xc9,0xd6,0xe0,0xa1,0x78,0xed,0xfe,0x7d, +0x7d,0xd6,0x6c,0xcb,0x02,0xcd,0x02,0x7e,0x7a,0x03,0xd1,0xed, +0xd9,0xf4,0xf3,0x2d,0x4d,0x21,0x9c,0x02,0x76,0x4d,0xaf,0xa8, +0x0d,0xc8,0x1e,0xf4,0xf8,0x3c,0x8d,0x98,0xdc,0x64,0x79,0x0e, +0x2d,0x11,0xeb,0x55,0x9a,0xee,0x1c,0x86,0x46,0x20,0xc2,0xc3, +0x1f,0xa5,0x18,0x68,0x3a,0xdd,0xca,0x9e,0xa1,0x20,0x32,0x97, +0x7e,0x4a,0xf3,0xaf,0x3c,0xd1,0x3d,0xa7,0x7c,0x94,0x43,0xac, +0xc5,0x9c,0xd7,0x5e,0x09,0x9b,0xfc,0xa6,0x68,0xcb,0xbb,0xeb, +0x51,0x51,0x5b,0x09,0xd8,0xeb,0xfd,0x5e,0x88,0x94,0x41,0x84, +0x92,0xc2,0xab,0x7e,0x69,0xb7,0x32,0xed,0x46,0xef,0x00,0x10, +0x22,0xb6,0xe6,0x5e,0x4a,0xbb,0xad,0xef,0xf0,0xe3,0x14,0xc1, +0x3b,0x7e,0x5e,0x14,0x1c,0xb6,0x79,0x6b,0x8c,0xc1,0xec,0x11, +0xbb,0xd8,0x9b,0x80,0x63,0x68,0x74,0x6b,0xa9,0x74,0x6f,0xfd, +0x0e,0x7a,0xb1,0xfa,0xab,0xb8,0xca,0x2f,0x60,0x91,0x05,0xf1, +0x39,0xfc,0xdc,0xc0,0x74,0x45,0xce,0x84,0x52,0x4a,0x07,0x35, +0xba,0x27,0x04,0xee,0x40,0x58,0x9f,0x5a,0x77,0xeb,0x84,0xd3, +0x83,0xb7,0x71,0x3a,0xfb,0x14,0x80,0xd1,0xa8,0x91,0xc3,0x2b, +0x5a,0xb7,0x71,0x32,0x8d,0x9b,0x19,0xa0,0xdc,0xda,0x32,0x34, +0x52,0x78,0xb1,0xd8,0xdd,0x94,0x07,0xf2,0xc5,0x82,0x2c,0xcc, +0xe2,0x3b,0xb6,0xdd,0x5e,0x79,0xd7,0xc5,0x43,0xc9,0xd2,0x36, +0x0b,0x89,0xb7,0x75,0x85,0x61,0x02,0x40,0x03,0x20,0xee,0xbf, +0xae,0xd6,0xde,0xf5,0x88,0x20,0xc3,0xab,0xb0,0xd7,0x24,0xa2, +0xea,0xcc,0x76,0xb1,0x64,0x58,0x94,0xa3,0x6e,0xb1,0xb7,0xd3, +0x72,0x41,0x62,0x60,0xc0,0xdd,0x2c,0x9d,0x55,0x82,0xe7,0x7e, +0x01,0x30,0x97,0x0b,0xbc,0x87,0x23,0x50,0x70,0x4c,0xd6,0xfa, +0x96,0xf0,0x08,0x7e,0xd3,0x1f,0x1c,0x4f,0x3e,0x0c,0xad,0xb2, +0xdb,0x7c,0x3c,0x2b,0x5f,0x4f,0xba,0xf9,0x06,0x0f,0xde,0x7e, +0x1e,0x4e,0x27,0xcf,0x73,0x8d,0x1e,0xf9,0x1e,0xd7,0x12,0xfd, +0x84,0xf3,0x0b,0x93,0x91,0x93,0xe9,0x20,0x1f,0xf7,0x47,0xf9, +0x1f,0x43,0x51,0x8a,0xde,0x29,0xf2,0xc9,0x60,0x55,0x31,0x58, +0x9f,0x51,0x1f,0xbc,0xde,0x34,0x19,0x45,0x45,0x69,0x8c,0x47, +0x14,0xf7,0x12,0x2c,0xfa,0xf5,0x2f,0x87,0xa0,0xce,0x67,0x23, +0x0f,0x27,0xe8,0x5b,0x86,0xcd,0x2e,0xea,0x06,0x1f,0x9e,0x7f, +0x00,0x23,0xd8,0xfa,0x14,0xf4,0xd1,0x1a,0x6b,0xe4,0xae,0x41, +0xd4,0x1c,0xe7,0x30,0x1c,0x18,0x2b,0x7f,0x03,0x63,0xda,0x8f, +0xf5,0x4b,0xa6,0xd7,0x58,0xb7,0xc1,0x0b,0x78,0x7f,0x91,0xd9, +0x30,0xc9,0x57,0xde,0xe9,0xa3,0x7e,0xe1,0xb2,0xa8,0xcf,0xc3, +0x3e,0x5c,0x19,0xfa,0x99,0x9a,0x10,0x0b,0xfa,0x93,0xd7,0xfa, +0xc8,0x77,0x55,0x4e,0xc6,0x68,0x48,0xff,0x38,0x1c,0x7e,0x28, +0x27,0x43,0x2c,0xa4,0x0e,0xfa,0x9f,0xcb,0x89,0x03,0x90,0x25, +0xed,0xaa,0x2b,0xbd,0x88,0x95,0x13,0x21,0x16,0xcb,0xcd,0xc7, +0xf3,0xd9,0xb0,0xa2,0x60,0x8c,0x47,0xcd,0xce,0xa1,0x6e,0x90, +0x41,0x19,0x82,0xe2,0x09,0xc7,0x68,0x94,0xd7,0x81,0x89,0x44, +0x0d,0xdb,0xa6,0xa1,0x8a,0x0f,0x56,0x32,0xd3,0x36,0xe4,0xfa, +0xfe,0xd4,0x43,0xd3,0x90,0xa1,0x46,0x6b,0xf8,0xf0,0x81,0xf9, +0xfe,0x76,0xf8,0xe8,0x01,0x53,0xdd,0x78,0xf8,0xed,0xf0,0x9b, +0x07,0x58,0x41,0xc6,0x81,0x0d,0xd1,0x80,0xbf,0x8d,0xef,0x1e, +0x60,0x9b,0x19,0xe4,0xd4,0xbe,0x0d,0xfa,0x6d,0xb4,0xf6,0x1e, +0x60,0x7f,0xd8,0x6c,0xb3,0x3e,0xdc,0x94,0x51,0xe8,0x6c,0x7e, +0xa6,0x37,0x83,0x58,0xba,0x87,0xa0,0xe1,0x81,0x62,0x4a,0xab, +0x7f,0x4b,0x32,0xd2,0xb3,0xc0,0xae,0x61,0x9c,0x83,0x5c,0xb6, +0x0f,0x87,0xfa,0x33,0xfd,0xc7,0xd9,0x82,0x0d,0xe1,0x22,0x23, +0x64,0x89,0x30,0x83,0xb5,0x9f,0x98,0x9d,0xd9,0xcf,0x44,0x95, +0x73,0xe1,0x1a,0xf0,0xfa,0x82,0x33,0x71,0x48,0xe7,0xe1,0xaf, +0x44,0x09,0x6f,0x3d,0xfd,0xb3,0xe2,0x2d,0xa8,0x75,0xc6,0xa4, +0x1b,0x6e,0xe7,0xd7,0xee,0x3e,0x45,0x74,0x9c,0xfe,0x2d,0x43, +0xa4,0x42,0x40,0xc0,0x51,0x0e,0x9d,0x37,0xfb,0x29,0x95,0x2d, +0x75,0xbc,0x0c,0xcc,0xd7,0xd1,0xfb,0x9f,0x8c,0x60,0xe0,0x24, +0xc3,0xaf,0x84,0x58,0x7f,0x58,0x66,0x6a,0x27,0x0c,0x1c,0xed, +0x46,0x0d,0xfa,0x34,0x4d,0x46,0x21,0xcf,0xd2,0xf1,0xeb,0xe9, +0xd1,0xfc,0x0c,0x5d,0x25,0x99,0x49,0x09,0xce,0x28,0x79,0x8a, +0x5f,0x4f,0xae,0x95,0x1d,0x53,0x79,0xf1,0x64,0x00,0xfa,0x5d, +0x8a,0x6e,0x4d,0xde,0xdd,0xe8,0x3e,0xd6,0xed,0x43,0x0c,0x2e, +0x3d,0x6f,0xc4,0x31,0x44,0xf3,0xc3,0x1f,0xc3,0x06,0x89,0x3f, +0x26,0x69,0x96,0xb9,0x34,0x08,0x9a,0x79,0x2b,0x72,0x60,0x44, +0x5b,0x66,0x04,0xeb,0x86,0xd7,0x1a,0xbe,0xc9,0xda,0x3f,0x71, +0x83,0xc2,0xde,0x18,0x7f,0x60,0x08,0x4e,0x54,0x0c,0x98,0x17, +0x0b,0xc2,0x04,0xfd,0x6c,0x26,0x29,0x64,0xa3,0xef,0x38,0xe1, +0x79,0x0d,0x51,0xf0,0x15,0x27,0x09,0xd2,0x47,0x65,0x41,0x5d, +0x63,0xfb,0x91,0xe0,0x84,0x10,0x05,0x10,0x66,0x02,0xc5,0xef, +0xd8,0x7d,0x25,0x3c,0x45,0x24,0xb8,0x57,0x97,0xcd,0x72,0x83, +0x22,0x26,0xdd,0x03,0xcd,0xb9,0x88,0x04,0x9c,0x55,0x75,0x71, +0x75,0xe0,0x6a,0x01,0x3d,0x5c,0x09,0xac,0x93,0xf4,0x51,0x6c, +0x44,0xf9,0xf4,0xdc,0xad,0x56,0x16,0x5e,0xb2,0x80,0xda,0xf4, +0x25,0x98,0x94,0x5d,0xa2,0x05,0x31,0x76,0x0d,0x56,0xba,0x00, +0xfc,0x72,0xa4,0xf8,0x6a,0x70,0x3a,0x44,0xa2,0x0a,0xd2,0xbc, +0x6f,0xd1,0xcb,0x8a,0x3d,0x35,0xd0,0xf5,0x7f,0x3a,0x41,0x59, +0x17,0x1f,0xf6,0x41,0x5b,0x23,0x33,0x56,0x25,0x18,0xda,0x1c, +0x21,0x28,0x93,0xb9,0x46,0x62,0xbd,0x8e,0x83,0xfc,0xe2,0x82, +0x32,0xe8,0x69,0xed,0x67,0xd8,0x0e,0x32,0xc0,0xc3,0x23,0xd0, +0xcc,0x84,0xe9,0xc8,0x1a,0x29,0x2c,0x70,0x8c,0x05,0x21,0xfc, +0x6a,0xa1,0xa5,0x57,0xa9,0x4d,0x7e,0xfe,0xb0,0xa7,0xbf,0x17, +0x8b,0x4d,0x0f,0x68,0x36,0x79,0xa1,0xe7,0x96,0x05,0x05,0x77, +0x1c,0x22,0x0a,0x72,0x24,0xba,0x5f,0xb1,0xc8,0x46,0xc3,0x4c, +0x5d,0x0a,0x3a,0xc2,0x5d,0x2b,0xd9,0xfd,0x81,0xf6,0xc7,0x39, +0xfc,0x45,0x81,0x13,0x7f,0x91,0x41,0x84,0x8f,0x43,0x70,0xa6, +0x83,0x51,0xba,0xd5,0x5f,0x42,0x98,0xbc,0x36,0xdb,0x33,0x27, +0x88,0x22,0x8a,0x7b,0x3b,0x2a,0xba,0xd7,0x8a,0x92,0x36,0x42, +0x22,0xfc,0x13,0x70,0xf7,0xac,0x0f,0x9a,0x18,0xa3,0x2b,0x73, +0xae,0x39,0x39,0xeb,0x15,0xb9,0xe8,0x32,0x6a,0x9d,0xc0,0x5f, +0xe6,0xde,0x0a,0xe1,0x2b,0xc8,0x94,0x9b,0x39,0xf6,0x38,0x45, +0x84,0x2c,0x00,0x2c,0xd0,0x28,0x40,0x53,0x21,0x5b,0xc0,0x6f, +0x91,0x8c,0x8e,0x82,0xc4,0x21,0xbe,0xc3,0x75,0x1e,0xc0,0x61, +0xba,0xf8,0x25,0x64,0x41,0xeb,0x21,0x94,0xf2,0x61,0xbc,0x4c, +0x48,0x42,0xd7,0x07,0xe8,0x65,0xa2,0x5c,0x56,0x83,0x70,0x4a, +0x38,0x7e,0x56,0xd7,0x20,0xd7,0xfd,0x0f,0x43,0xf4,0x33,0x76, +0x01,0x1e,0x65,0x8c,0x82,0xc9,0x1c,0x2d,0x53,0xcf,0xf4,0xf1, +0xda,0x2a,0xa2,0x43,0xb2,0x7d,0xcc,0x13,0xc1,0x1e,0xac,0x8f, +0xdc,0x08,0x99,0x7d,0xc7,0xb0,0x59,0xa4,0x57,0x85,0x08,0xcd, +0xa1,0x42,0xcf,0x03,0xe7,0x1f,0x64,0xc3,0xf5,0x22,0x32,0x37, +0x6e,0x6d,0xca,0xde,0xda,0xb3,0xf9,0x29,0x7d,0x49,0xcb,0x4e, +0x17,0x33,0x0b,0x76,0x1d,0x3c,0x33,0xf6,0x67,0xd2,0xae,0x41, +0xae,0x2e,0x31,0x27,0xf3,0xeb,0x19,0xaf,0x57,0x17,0x7a,0xc1, +0x06,0x3e,0x93,0x31,0x88,0x07,0x96,0x6d,0xf1,0xa4,0xca,0x5e, +0xf3,0x10,0x5e,0x30,0x8b,0x48,0x2f,0xfa,0xa9,0x18,0xfa,0xc9, +0x1c,0x7b,0xcd,0x65,0x89,0xf7,0xbf,0xbc,0xe3,0x70,0xb1,0x7a, +0x14,0xcf,0x67,0xe7,0xfa,0x2f,0xac,0x92,0x54,0x23,0x21,0xea, +0xe7,0x13,0x05,0x2e,0x31,0x01,0x99,0xea,0x9a,0x0b,0xd3,0x07, +0xd5,0x28,0x59,0x72,0x05,0x39,0x1f,0x15,0x4a,0xa4,0x25,0x76, +0xea,0x63,0xcb,0xf9,0x6f,0xef,0xe8,0xee,0xdb,0x80,0x27,0xa5, +0x07,0x77,0xae,0xcf,0xcd,0x3c,0xa7,0x73,0x0c,0x1a,0xbd,0x1e, +0x4e,0xcf,0x2d,0x03,0x7c,0x0e,0xa1,0xe1,0x80,0x6e,0xa3,0xb2, +0x46,0x05,0x1c,0x9d,0x58,0x32,0x6b,0x03,0x0e,0x7a,0xda,0xcb, +0xb5,0xb5,0x65,0x2f,0xc5,0xbc,0x78,0x18,0xfa,0x94,0xd7,0x8b, +0xde,0xcf,0x76,0x3b,0x82,0x43,0xf1,0xb3,0xa4,0x8e,0x9b,0xf1, +0x13,0x12,0x22,0xc3,0xd5,0x0b,0xf6,0xa4,0x17,0xe3,0x43,0xdc, +0xef,0x80,0x0d,0xa4,0x0d,0xd1,0xc9,0xe3,0x59,0xaf,0x38,0x86, +0x3f,0xcd,0x77,0xc7,0x4f,0x05,0x50,0xa3,0x05,0xcf,0xcf,0xc1, +0xe9,0x9e,0x8e,0x47,0xa0,0x24,0xc4,0xfb,0x2f,0x0d,0x8c,0x39, +0xc4,0xc5,0xdf,0xcb,0x61,0xff,0xc6,0xc5,0x77,0x1e,0x7e,0xfb, +0x6d,0xfa,0xf0,0xdb,0x6f,0xe4,0x86,0xe4,0x43,0xdc,0x8a,0x5b, +0x6a,0x88,0xf8,0xdb,0xa3,0xad,0x2d,0x68,0x3c,0xfc,0x6e,0xed, +0xee,0x6a,0x56,0xda,0xa6,0xec,0xee,0x8a,0x3d,0xc8,0x3b,0xdd, +0xb0,0x37,0x84,0x09,0x87,0xdb,0x9a,0xe9,0xe8,0x6f,0x6d,0x6d, +0xef,0xe9,0xac,0xfa,0xf3,0xe6,0xa2,0x69,0x52,0x40,0x3f,0x8a, +0x3f,0x21,0xa5,0xdf,0x3d,0x7c,0xfd,0xea,0xf8,0xa7,0xde,0x63, +0x5d,0x8e,0x08,0xee,0xb7,0x5a,0x1d,0xfc,0x4a,0x31,0xf2,0xe0, +0xc9,0xf1,0xb3,0xde,0xe3,0x16,0x83,0x60,0x68,0x5f,0xb6,0x2c, +0x46,0xff,0xeb,0xd9,0x93,0xb7,0x3d,0x25,0x90,0x24,0x1d,0x80, +0x24,0x0c,0x3f,0xbd,0x7e,0xf7,0xd6,0x15,0x82,0xa1,0xfd,0xbd, +0x87,0x1d,0xf8,0x20,0x80,0xc3,0x17,0xaf,0xde,0x41,0x21,0x96, +0x0e,0x0a,0xef,0x7f,0xf3,0x7d,0x87,0x3e,0x09,0xec,0xe8,0xd9, +0xd3,0xd7,0xaf,0x0e,0x1c,0x18,0x87,0x01,0x8c,0x3e,0x0d,0xb6, +0x97,0x2f,0x5f,0x84,0xb0,0x32,0x72,0xff,0xfb,0xef,0x01,0xb1, +0x8d,0x48,0xb7,0x5b,0x8a,0x1a,0xea,0xd4,0x34,0xcf,0x41,0xff, +0xf3,0xeb,0x0b,0xe8,0x27,0xdd,0x64,0x50,0xb7,0x7d,0x93,0xb0, +0x58,0x98,0xaf,0x7d,0xa8,0x60,0x22,0x9b,0x14,0x23,0x94,0xdf, +0xe4,0x99,0xf9,0x90,0x6c,0x09,0x3d,0xe0,0x06,0x5f,0x3f,0xfa, +0x6c,0xa2,0x19,0x8d,0xe7,0xa3,0xfe,0x65,0xe1,0x8e,0xae,0xe4, +0x88,0x40,0x63,0xc9,0x6e,0xd1,0x58,0x70,0xba,0xd9,0x52,0xf3, +0xf1,0xbc,0x18,0xd2,0x21,0xbe,0x48,0xbb,0x3d,0x0e,0xe3,0xe2, +0x0d,0x41,0x53,0x4a,0xba,0xbd,0xa7,0xce,0xaf,0x34,0xd6,0x97, +0x9a,0x89,0x87,0x01,0x92,0xee,0xa2,0xf5,0x03,0x02,0xd4,0x78, +0x72,0x52,0x01,0xc6,0xae,0x4b,0x49,0x1f,0x9e,0x62,0x9e,0xe3, +0x8a,0x83,0x45,0x15,0xc3,0x29,0x6b,0x0a,0x83,0xdf,0x08,0x1d, +0xb5,0x94,0x23,0xf8,0x1f,0x90,0x10,0x5f,0x87,0x57,0x91,0xfa, +0xb4,0x47,0x49,0xc0,0xfc,0xd9,0x40,0xc6,0xb7,0xdf,0xc8,0x17, +0x5f,0x32,0x5f,0x0c,0x8c,0x84,0xdf,0x4a,0x8f,0x77,0x35,0xc3, +0x49,0x51,0x58,0x63,0x1b,0x92,0xe4,0xda,0x48,0x5b,0xa1,0x10, +0xec,0x39,0x2f,0xdc,0x1c,0x1b,0x54,0x04,0x7a,0x06,0xee,0xdb, +0xce,0x67,0x3e,0x89,0x92,0xf4,0x5d,0x3b,0x67,0xbc,0x56,0x94, +0x09,0xb2,0x27,0x0c,0x2b,0x86,0xbd,0xce,0x48,0x2a,0x98,0x0b, +0x2b,0xb0,0x90,0xd7,0xdd,0xfa,0x1b,0xcc,0xd1,0xd6,0xf0,0x3d, +0x20,0x60,0x8d,0xb6,0xa3,0x24,0xd5,0x20,0xe2,0x68,0x36,0xe9, +0x0f,0x00,0x59,0x6c,0xed,0xdc,0x14,0xfe,0xfd,0x75,0xa1,0xb9, +0xc6,0xb3,0x29,0x68,0xfb,0xaa,0x11,0x17,0x4a,0xc6,0xf3,0x16, +0x8b,0xd8,0x8f,0x40,0x41,0xaf,0x21,0x2c,0x09,0xa0,0x71,0xcb, +0x62,0xfc,0x41,0x92,0x23,0x66,0x3e,0x96,0xe4,0x80,0x1b,0x5e, +0x94,0xe4,0xd5,0x81,0xc3,0x51,0x50,0x27,0x1d,0xc0,0xee,0x99, +0x3b,0xfd,0x57,0x58,0xb9,0x7e,0xc5,0x32,0xd4,0x18,0x2e,0x77, +0x40,0x5c,0x38,0x43,0x3d,0x67,0x9d,0xc1,0xed,0xad,0xa4,0xfd, +0xb8,0x29,0x90,0xf7,0xb6,0xb6,0x34,0x33,0x75,0x88,0x76,0xe3, +0x13,0x12,0x60,0xff,0x36,0xcf,0xa7,0xba,0xe9,0x9a,0x3b,0x00, +0xb6,0x13,0x35,0x3e,0x08,0xe1,0xb4,0xb5,0x4d,0x23,0x30,0x2c, +0xf1,0x0d,0x26,0x90,0x61,0x36,0x65,0x7f,0x23,0x26,0xab,0x6f, +0x7c,0x36,0x41,0xd5,0x20,0x20,0x02,0x52,0x32,0x4d,0x5c,0x2c, +0x94,0x85,0x10,0x6d,0x1b,0x9e,0xae,0x71,0xad,0x41,0x27,0x2c, +0x7f,0x0c,0xfd,0xcb,0x9a,0x47,0x74,0x6a,0xc6,0xda,0x65,0x95, +0x03,0x03,0x38,0xec,0x26,0xa6,0xc7,0xd0,0xf1,0xea,0xd7,0x0c, +0x03,0xe6,0xb0,0x00,0x8d,0x53,0x97,0xb1,0xd1,0x02,0x5b,0xd0, +0x08,0xa0,0xff,0x74,0xc6,0x78,0x91,0x63,0x31,0xe1,0x34,0x6f, +0xff,0xba,0xbf,0xdb,0xf6,0xc9,0x27,0xf4,0xc6,0x62,0xd9,0xaf, +0xe6,0x12,0x5c,0xe7,0xf0,0x6b,0x05,0xb7,0xc4,0x68,0x26,0x1a, +0xf1,0xb2,0xaa,0x79,0xf6,0xeb,0xd6,0x96,0x7f,0x12,0xa2,0x8e, +0xc3,0x3e,0xdc,0xdc,0x4d,0x34,0xc0,0x76,0x8b,0x15,0xc8,0x7e, +0xdd,0xde,0x5e,0x6a,0xc6,0x64,0x59,0xdd,0xc8,0x6e,0x84,0x90, +0xaf,0x30,0x9a,0xc4,0x20,0xb1,0xa7,0x39,0xe6,0x1f,0xdf,0x88, +0xb1,0x6e,0xe2,0xed,0x4b,0xbc,0x73,0xd2,0xa5,0x5b,0xcc,0x9d, +0xa4,0x43,0xf1,0xf6,0xd8,0xf0,0xcb,0x49,0x77,0x71,0xd2,0xbb, +0xb7,0x73,0xa9,0x34,0x0b,0x95,0x06,0x89,0x27,0x27,0x14,0xef, +0xb3,0xbd,0x54,0xf0,0x73,0x9f,0xbb,0x14,0xc7,0x38,0x7b,0x64, +0x63,0x8e,0x90,0x89,0xb8,0x08,0xe8,0xb5,0x4c,0x59,0xe5,0x7b, +0x7e,0x4f,0x53,0xdf,0x3e,0xdb,0x26,0x1c,0x98,0xdf,0x9d,0x61, +0x4c,0x6a,0xaf,0xb3,0x3a,0x39,0xad,0x69,0x37,0x03,0x50,0x7e, +0xc5,0x0a,0x47,0xf6,0x5b,0x21,0xc0,0x89,0x22,0x79,0x86,0x74, +0xf4,0x51,0x72,0x23,0x33,0x98,0x84,0xfa,0xa5,0xa1,0xa3,0x63, +0xd2,0x2c,0x27,0xcb,0xbc,0x2b,0xbf,0x6c,0x17,0x6f,0x0e,0x58, +0xce,0xe3,0x1a,0x9d,0x00,0x8d,0x54,0xc7,0xe4,0x33,0x1d,0x7d, +0xdd,0x34,0x7b,0x4d,0xd2,0xe1,0x46,0xce,0x86,0x9f,0x6e,0xfa, +0x63,0x5e,0xe6,0x0d,0xfb,0x7f,0xcd,0x22,0xe4,0x84,0x11,0xb8, +0x16,0xa2,0x30,0x2c,0x7d,0x35,0x29,0x59,0x7d,0xb7,0xd7,0x62, +0xd3,0x3b,0x5f,0x92,0x9a,0x32,0xcd,0xc6,0x13,0x72,0x88,0x55, +0x74,0x42,0x0e,0x21,0x92,0xe7,0xc1,0xf8,0x72,0xa2,0x97,0x44, +0x9d,0xfb,0xb9,0xeb,0xe2,0x70,0xc4,0x43,0xc6,0xe6,0xc8,0x83, +0x63,0x88,0xc5,0x02,0x7f,0x97,0x64,0x27,0xe0,0x1b,0xec,0x44, +0xf4,0x52,0x1f,0x8e,0x05,0x67,0x5f,0x00,0x5e,0x3a,0x69,0xf6, +0x7a,0x6b,0xab,0x1a,0x6e,0x36,0x2c,0x0c,0xc1,0x49,0x3b,0xe1, +0x56,0xe7,0xe1,0x6e,0xa6,0x4f,0x65,0x4e,0xb5,0xa2,0x3e,0xec, +0xbe,0x7f,0x15,0x51,0x2a,0xdf,0xce,0xdc,0x23,0x36,0x04,0xf4, +0x76,0x8e,0x37,0xe4,0xf3,0xf5,0x72,0xf8,0x49,0x63,0xa1,0x4b, +0x86,0x19,0xfc,0x35,0x52,0x7c,0xd2,0xf4,0x30,0xaa,0xe5,0x98, +0xc4,0x6a,0xe5,0x07,0xfa,0x7f,0x56,0x59,0x1b,0x95,0xf0,0x30, +0xfb,0xf1,0xd5,0x74,0x38,0x3c,0xc8,0x2f,0xf5,0xf1,0x9a,0xf4, +0xbb,0xff,0xa5,0xff,0xc7,0xea,0xe0,0x3f,0xea,0xff,0xf1,0xe7, +0xa5,0xfe,0x5f,0x45,0xee,0xe7,0x93,0xf9,0x34,0xcc,0x2c,0x73, +0xcb,0xec,0x55,0xf9,0x8f,0xf2,0x4f,0x32,0xfb,0x11,0x83,0x1f, +0xb9,0x0f,0xf3,0x55,0x4d,0xfd,0xeb,0xb1,0xfe,0x2d,0x55,0xe1, +0xf0,0xf0,0x90,0x73,0x1d,0xba,0xcf,0xc1,0xc0,0x7e,0x88,0xaf, +0x41,0x05,0xd2,0xf7,0x93,0xe9,0x80,0x10,0xf5,0x19,0xf0,0x89, +0x85,0x2a,0x6f,0xde,0x86,0x4b,0x1d,0x25,0xcd,0xd3,0xeb,0xe1, +0x34,0x1f,0xe4,0xc3,0x6b,0xec,0x25,0x42,0xf1,0xcf,0xaa,0x36, +0x37,0xef,0x37,0x0f,0x99,0xe0,0x9f,0xb9,0x9c,0x9f,0x7f,0xae, +0x81,0xfe,0x63,0x32,0x66,0x7c,0xc7,0x55,0x10,0xa6,0xda,0xb6, +0xad,0xf8,0x43,0xf4,0x85,0xed,0x08,0xfe,0xf8,0xe9,0x27,0xfe, +0xb8,0xba,0xe2,0x8f,0xeb,0x6b,0xfe,0x28,0x0a,0xd3,0x7a,0x06, +0x9f,0x69,0x2f,0x93,0xd7,0x64,0x35,0x39,0x4d,0x46,0xfe,0xfd, +0x68,0x7e,0xcd,0xc7,0x7b,0xf3,0x6b,0x3e,0x86,0xfc,0xfb,0xac, +0xba,0x4f,0x5f,0x4f,0x8f,0x3f,0x4e,0xb8,0x47,0x59,0x7f,0xc5, +0xbd,0x82,0x10,0xba,0x29,0x53,0x98,0x08,0x37,0xcf,0x48,0x4b, +0xc2,0x28,0xa5,0xf1,0xca,0x80,0x63,0xdf,0xb1,0x91,0xa8,0xac, +0xa3,0x77,0x74,0x15,0xe5,0xa0,0x0f,0x2b,0x0e,0xfd,0xdc,0xbc, +0x87,0x24,0x9d,0x05,0xf9,0x3a,0xab,0x4f,0x1b,0xb5,0x35,0xbc, +0x98,0xff,0xe3,0xfc,0x6a,0x3e,0xfe,0x90,0xc5,0xe5,0x7e,0x69, +0x0e,0x3f,0x0d,0xcf,0x0d,0xf0,0x62,0xd1,0xed,0x25,0x70,0x81, +0x0b,0x86,0x16,0x8a,0x2c,0xe6,0x7c,0x0d,0x5d,0x34,0x6f,0x93, +0x84,0x80,0xf3,0x3e,0x85,0x44,0x7d,0x64,0xd7,0xd9,0x34,0xbf, +0xa1,0x76,0x95,0xce,0x69,0xa4,0xdf,0x8d,0xf8,0xdb,0xdd,0x07, +0x88,0xa6,0xab,0xd9,0x9a,0x06,0x09,0x27,0x28,0xbc,0x67,0xf7, +0xb2,0xa8,0x01,0x92,0x5b,0x8a,0xdd,0xed,0x75,0xb6,0x39,0x73, +0xca,0xbf,0xde,0x65,0x02,0x14,0x7a,0x4c,0xa6,0x08,0xa1,0x96, +0x72,0xf5,0x20,0x8b,0x6a,0xde,0x1a,0x02,0xba,0x05,0x70,0x2e, +0x9b,0x91,0xce,0xb6,0x19,0xe6,0x95,0x6b,0x8b,0x9b,0x6e,0x51, +0x4a,0xb6,0x63,0x72,0x3a,0x96,0xc4,0x1e,0x12,0x3e,0x12,0xb3, +0x3c,0x95,0x96,0x6e,0xcd,0x1c,0xb5,0x89,0x3b,0xaa,0x9c,0xba, +0xfd,0x6c,0xe5,0x8c,0x23,0x39,0x3b,0x4e,0x37,0xc6,0xa7,0xa8, +0xfc,0x7e,0xa7,0xb2,0xe4,0x7e,0xea,0x0e,0x95,0xde,0x99,0x8a, +0x08,0x96,0x94,0x1c,0x88,0xc9,0xb4,0xaa,0x4e,0x28,0x08,0xf0, +0xaa,0xe4,0x55,0xe8,0xe0,0xc0,0x21,0x2a,0xa3,0x32,0xd4,0x0c, +0xcc,0x31,0xbb,0x1e,0x11,0xcc,0x65,0xbf,0x64,0x94,0x35,0x78, +0x19,0x1a,0xb1,0x0c,0xed,0x7f,0xfb,0xf7,0x4e,0xeb,0xfb,0xdd, +0xdd,0x74,0x6f,0xf8,0x30,0x40,0x65,0x17,0x06,0x5e,0xb0,0xef, +0x42,0x2c,0x73,0x8b,0x45,0xd1,0x90,0x9f,0x17,0x6f,0xae,0x57, +0x77,0x94,0x86,0x38,0xac,0xc0,0xf5,0x53,0x69,0x35,0x12,0xab, +0x92,0x4f,0x14,0x8a,0x4b,0x6a,0x89,0xba,0x16,0x8b,0x58,0xd0, +0xf3,0x24,0x43,0xa9,0xcd,0x59,0x88,0x55,0xcf,0xcf,0xc9,0x52, +0x92,0xda,0x9c,0xd5,0xbb,0x55,0x58,0xba,0x13,0xb7,0x30,0x22, +0xb8,0xdc,0x8d,0xa3,0xdd,0x26,0xe8,0x69,0x95,0x7a,0xf9,0x9f, +0xae,0x51,0x07,0xf6,0xc1,0x3f,0x66,0x11,0x7a,0xf3,0x15,0xf9, +0xe4,0x0e,0x62,0x10,0xe8,0xe3,0xf9,0xbb,0xe3,0xa7,0xa0,0x4e, +0x61,0x62,0x66,0x7f,0x5c,0x67,0xf5,0xeb,0x5d,0xb9,0x86,0x77, +0x2f,0xe4,0x83,0xbb,0x76,0xd7,0x70,0xb9,0xc7,0x65,0x23,0xa3, +0xc5,0xb9,0x98,0x9f,0xe9,0x65,0x53,0x9f,0xb9,0x5a,0xfc,0xa0, +0xcd,0xee,0x4e,0xc4,0x2e,0xf8,0x5b,0x97,0x60,0x27,0x98,0xb5, +0xa8,0xc4,0x05,0xfe,0x07,0x82,0xd9,0xf5,0xd1,0xae,0x5e,0x1f, +0xd1,0x09,0xbf,0x0d,0x75,0x31,0x2f,0x5f,0x10,0xc8,0x4d,0x01, +0x3a,0x11,0x5a,0xc6,0xf7,0x71,0xc9,0x47,0x20,0x48,0x54,0xe7, +0xf3,0x29,0xf8,0xa5,0x47,0xeb,0x1d,0x20,0x97,0x3c,0x9e,0xbc, +0x2b,0x86,0xea,0x22,0xff,0x04,0xb3,0x58,0x7d,0x44,0x05,0x2a, +0x3a,0xcd,0xc3,0x95,0x80,0x9e,0xdf,0xf8,0x4b,0xa4,0xb1,0xbc, +0x7d,0xd3,0xf6,0x34,0x9d,0x89,0x05,0xca,0x4c,0x7c,0xd3,0xa9, +0xfb,0xdc,0xf3,0x09,0xaa,0x69,0xd7,0xc7,0x50,0x14,0x38,0xd9, +0x85,0x99,0x56,0xa2,0x72,0x34,0x2d,0x7d,0xba,0x39,0x98,0x38, +0x4f,0xef,0x5f,0x4a,0x4d,0x40,0x65,0xd9,0x5c,0x59,0x3f,0xec, +0xe0,0x78,0x83,0xf1,0xaa,0x63,0x55,0x6b,0x17,0x57,0x93,0xa8, +0xf5,0x7d,0xd4,0xd0,0xe1,0x34,0xda,0xdb,0xa5,0x0f,0xf8,0x17, +0x16,0x88,0xcb,0x47,0xc7,0xca,0xfe,0xa1,0xde,0x28,0xe9,0x4d, +0xd2,0x00,0x66,0xa9,0x44,0xcf,0xf0,0xc2,0xfd,0xb1,0xf9,0xe3, +0x8f,0xf4,0x68,0x13,0xbe,0xdf,0xbb,0xcf,0x67,0x1d,0x68,0xd2, +0xcc,0xae,0x94,0xd0,0x3d,0xef,0x41,0xe3,0xc1,0xd4,0x2b,0x86, +0xac,0x89,0xc2,0x4c,0x2d,0xfd,0xf3,0x4c,0x3d,0xd2,0x03,0x2b, +0xb5,0xe7,0xfc,0xda,0xc5,0xc9,0xf4,0x50,0x66,0x8a,0x1a,0x50, +0xe5,0xdf,0x53,0xb4,0xc6,0x3b,0xa0,0x43,0x4b,0x6a,0x00,0x86, +0xae,0x75,0x74,0x00,0x5a,0xa7,0x81,0x87,0x93,0x53,0xc0,0xd4, +0x1c,0x4c,0x3e,0xa6,0xa4,0xc1,0x92,0x09,0xb0,0x8f,0x00,0x06, +0x94,0xd9,0x52,0xb6,0xb6,0xb8,0xe0,0xc7,0x7e,0x66,0x8a,0x6f, +0x34,0xd4,0xdd,0x15,0xbe,0xbc,0x24,0xea,0xed,0x20,0xf3,0x30, +0x05,0xc1,0x8f,0xce,0x53,0xbe,0xe9,0x81,0x0c,0x8a,0x40,0xa5, +0x1c,0x55,0xb1,0x15,0x41,0x9a,0x0d,0x26,0x65,0x08,0x3d,0xa6, +0xec,0xd0,0xcf,0xaa,0x47,0x81,0x18,0xc7,0x14,0x13,0x8e,0x81, +0x32,0xd6,0xfd,0xe0,0x5a,0x01,0xd1,0x27,0x62,0x3a,0x57,0xca, +0xa7,0xc1,0xc3,0x14,0xce,0x4c,0x3c,0xcb,0x9a,0xcb,0x0a,0x37, +0x33,0x77,0xcb,0x05,0xc9,0xd6,0x60,0xfe,0x00,0x10,0xf0,0x65, +0x07,0x09,0xf5,0x25,0x0c,0xed,0xf4,0x02,0x84,0x4e,0xbb,0xf4, +0xc4,0xe5,0xe1,0x7e,0x5e,0x9e,0x7a,0xfa,0xcc,0xdf,0x68,0xe4, +0x89,0x8c,0xa0,0xb5,0x06,0x3e,0x64,0xd3,0xe4,0x64,0x50,0xa0, +0xfd,0x1d,0x0b,0x1c,0xaa,0x73,0x94,0xd1,0x77,0xe0,0x6e,0x23, +0xef,0xb4,0xd2,0xdd,0xd4,0x2b,0x95,0xb2,0xe0,0x96,0xd9,0xe0, +0x6d,0x27,0x16,0x9b,0xc0,0x62,0xb1,0x9b,0xec,0x7c,0xbb,0xcb, +0x2b,0xa5,0xd9,0x22,0xeb,0x20,0xff,0x06,0x90,0x6e,0x57,0x8a, +0xfd,0xfd,0xa5,0x23,0xda,0x3b,0x85,0x6f,0xf8,0x48,0xa4,0xb9, +0xa7,0xda,0xd5,0x95,0x1f,0xe6,0xc9,0xd5,0x35,0xb8,0x8f,0x6d, +0xdb,0x62,0x17,0x8b,0xf0,0x9a,0xb7,0x5a,0x01,0xcd,0xb2,0x24, +0xa2,0xe7,0xb2,0x6e,0x95,0x1a,0x5a,0x78,0x73,0x4d,0xcc,0x64, +0x29,0x16,0x66,0x55,0x95,0x36,0x59,0x39,0x3b,0x6e,0xa8,0xa5, +0x68,0x52,0x24,0xa9,0x80,0xb6,0x4a,0x26,0x3d,0x55,0xbd,0xdd, +0x78,0x5e,0xd7,0xab,0xb7,0x83,0xb2,0xe1,0x27,0x67,0xa7,0x49, +0xf6,0x91,0x6e,0x80,0x8f,0x3c,0x68,0x9f,0xeb,0x2e,0xa1,0xe5, +0x58,0xb9,0x48,0x33,0xd8,0x5d,0x0c,0x8d,0xed,0x5e,0x6a,0x72, +0x96,0xb3,0x05,0x79,0x38,0x83,0x2f,0x4a,0x3c,0xe0,0x8a,0x11, +0x87,0xf1,0xc4,0x8a,0x82,0xc2,0x2b,0x9e,0x3e,0xd8,0xf8,0x15, +0xf3,0x1b,0x2f,0x3f,0xe0,0x69,0x32,0xed,0xb8,0xb8,0x8c,0x52, +0xc3,0xa9,0x19,0x89,0x59,0xe8,0xdc,0x52,0x7c,0xc8,0x51,0xf7, +0xf4,0xee,0x35,0x9e,0xce,0x67,0x59,0x14,0x35,0xec,0x08,0xe1, +0x38,0xd2,0xcb,0xe2,0xf7,0x48,0x46,0xca,0x3c,0x9b,0xcc,0xfa, +0xa3,0x37,0xae,0x58,0x06,0x22,0x1d,0x13,0x22,0xc1,0x17,0xc2, +0x19,0xac,0x17,0xb4,0x55,0xd4,0xc9,0x44,0xe1,0x7c,0x88,0xeb, +0x45,0xfe,0x78,0x26,0x6f,0x4e,0x70,0xd6,0x0b,0x7e,0x06,0x4d, +0xa7,0x88,0x6a,0x67,0xf1,0x1a,0xc2,0x9f,0xda,0x93,0xa8,0xc1, +0x02,0xca,0xfe,0xd4,0x62,0xa6,0xba,0x96,0x6d,0xe2,0xb0,0xd1, +0x48,0x10,0x99,0xf4,0x02,0xc7,0x99,0x8c,0xcc,0x7b,0x17,0x44, +0xde,0xb6,0xaf,0xc4,0x0d,0x1c,0x5d,0xb5,0x33,0xb4,0x6d,0x73, +0x53,0x14,0x0a,0xd8,0x57,0x94,0xd3,0x10,0x01,0xab,0xdb,0x53, +0xdd,0x11,0x8d,0xac,0x0a,0xb6,0x52,0x3c,0x4c,0x2c,0x5e,0x47, +0x96,0xd4,0x29,0x8f,0xb4,0x56,0x5a,0xaa,0x11,0x0b,0xe7,0xb0, +0x4a,0x74,0xd8,0x55,0xab,0x0f,0xd1,0x72,0x94,0x9a,0x29,0x6c, +0xd1,0x9a,0x0b,0xb7,0x4d,0xaf,0x3b,0xd6,0x28,0xb4,0x2d,0x60, +0xbc,0xbb,0xb8,0x4c,0x0e,0xdf,0xed,0xea,0x76,0x52,0xde,0xa8, +0xbe,0xb3,0xe3,0x58,0x0b,0x53,0x1e,0xeb,0x5c,0x0e,0x73,0x7f, +0xdd,0xda,0x13,0x7b,0x71,0xdf,0x6c,0x32,0xad,0x3d,0x3f,0x1f, +0xbe,0xb6,0xdd,0xda,0x6a,0xc1,0x06,0x15,0x00,0x97,0xb3,0x67, +0xbc,0x77,0x97,0xd7,0x40,0x55,0xa9,0xbf,0x2c,0xef,0xde,0x3c, +0xa1,0x8f,0xbb,0x05,0x2c,0xe4,0x2d,0x47,0x7c,0xd2,0x4d,0x16, +0xf0,0xd3,0xd3,0x3f,0xdd,0xb8,0xfb,0xcb,0x49,0xef,0xa4,0xdb, +0x7b,0x90,0x9c,0xf4,0x20,0xb6,0x99,0xec,0x5c,0x06,0x6f,0x79, +0xf4,0x92,0x72,0xd3,0x52,0x37,0x7b,0xea,0xe6,0xa1,0xba,0x79, +0x64,0x91,0xde,0xb4,0x16,0x8b,0x9b,0x3d,0xfd,0xef,0xa1,0xfe, +0xf7,0x68,0x99,0xc8,0x2b,0x1b,0x21,0x89,0xaa,0xa4,0xa2,0xbb, +0x7d,0xa2,0x29,0xf9,0xe5,0xde,0x83,0x46,0xa7,0x19,0x27,0x8b, +0xee,0x49,0xef,0x76,0xd9,0x83,0xeb,0x97,0x93,0x93,0x7b,0x5b, +0xe1,0x15,0x4c,0x69,0xdd,0x2c,0xaf,0xfa,0xc0,0x9d,0x51,0x5b, +0xa9,0xb3,0x61,0xc1,0xb7,0x07,0x0a,0x9f,0xfa,0x1e,0x4f,0x7e, +0x18,0x82,0x3e,0x90,0x39,0xa4,0x1c,0x41,0xa4,0x51,0x58,0xb2, +0x0b,0x95,0x99,0x36,0xc1,0x8e,0x11,0xde,0x22,0xcb,0xf3,0xa3, +0x38,0x92,0xee,0xee,0xe8,0x3e,0xe3,0x07,0x92,0xe6,0xce,0x24, +0x7f,0x1c,0x22,0x27,0x66,0x46,0x50,0x91,0xed,0x2a,0x47,0x78, +0xe6,0x6c,0xc7,0x98,0xee,0xae,0x51,0x09,0x70,0x79,0x12,0x91, +0x5f,0x97,0xe3,0xaa,0x03,0x0b,0xe6,0xaa,0x2d,0x47,0xa2,0x30, +0x17,0x2a,0x22,0x0e,0x06,0xa5,0xa0,0xb3,0x91,0xc9,0x62,0xc2, +0xa9,0xa7,0x7c,0xd0,0xd6,0xee,0x83,0x00,0xba,0xe2,0x6e,0x5c, +0x05,0x20,0xe4,0x00,0x5f,0x22,0x52,0xfc,0x08,0x55,0xf4,0xe1, +0x62,0x21,0x02,0xfb,0x12,0x18,0xad,0x61,0xb8,0x34,0x1f,0x91, +0x1b,0x10,0xa2,0x1a,0x49,0xd2,0xe6,0xf6,0x3e,0x0f,0x87,0xcd, +0x62,0x21,0xc0,0x56,0x8d,0xc4,0xe0,0x24,0xcc,0x0b,0xbc,0xdb, +0x51,0x71,0xf2,0x64,0x79,0x31,0xc1,0x5d,0xca,0xdb,0x8f,0x60, +0xfc,0xf1,0x3b,0x39,0x1a,0x2f,0x8f,0xda,0xf9,0x3e,0x99,0x9b, +0x33,0x29,0x70,0x47,0x67,0x59,0x82,0x0b,0x40,0x03,0xa5,0xeb, +0xad,0x70,0xbb,0xd5,0x6b,0x30,0xc8,0xb7,0xbd,0xc5,0x22,0xda, +0x88,0x58,0x4c,0xb1,0x34,0x43,0xef,0x91,0xf0,0xd5,0x5d,0x4c, +0x50,0xc0,0x4e,0x36,0xca,0x3d,0x1a,0x04,0xf6,0x46,0x26,0xe1, +0x76,0x7b,0x8c,0xef,0x0e,0xf1,0xae,0x58,0xba,0x34,0x86,0x68, +0xe3,0xe7,0x28,0x59,0x39,0xe8,0xcc,0x7a,0x85,0x4e,0xc8,0x2b, +0x66,0x11,0xa3,0xad,0x6e,0x72,0x5c,0x99,0xfd,0x16,0x47,0x56, +0x20,0x68,0x6e,0xbd,0x99,0xf7,0x8b,0x9b,0x57,0xc3,0xd9,0x7f, +0x15,0x93,0xb1,0x68,0x77,0x16,0xe7,0x50,0x9e,0xcc,0xe9,0xeb, +0x75,0xca,0x74,0xa4,0x8c,0xa8,0x22,0x29,0x6e,0x70,0x1a,0x08, +0xa1,0xd3,0x88,0x08,0x76,0x2a,0x82,0x88,0xbd,0x53,0x3f,0x50, +0x52,0x5f,0xa3,0xb9,0xe3,0x56,0x7e,0x3a,0xd6,0x98,0x6b,0xf7, +0xba,0xf5,0x1f,0x10,0x08,0xdd,0xe5,0x4a,0x02,0xd9,0x37,0x88, +0x31,0x06,0x14,0xd0,0x56,0x7d,0xd8,0xa8,0x92,0xb2,0x85,0x0a, +0xce,0xa6,0x56,0xf1,0x67,0x75,0xad,0x06,0xea,0x4a,0x1d,0xaa, +0x42,0x5d,0xb3,0xce,0x2e,0x1e,0x35,0x6d,0xd6,0x00,0xc2,0x70, +0xe1,0xad,0xef,0xbf,0xdb,0xdd,0xff,0xbc,0xb5,0x85,0x67,0xc6, +0x42,0x30,0xd1,0x9f,0xa9,0xbe,0x7e,0x59,0xf6,0xd0,0x5a,0x55, +0x82,0xd1,0xcd,0x0b,0xec,0xe8,0x9a,0x17,0x57,0x75,0x25,0x4a, +0x86,0xbf,0x54,0xa8,0x27,0xe5,0xa0,0x2b,0x00,0xa3,0x15,0x42, +0x36,0xf6,0x2b,0x7b,0x9b,0xe6,0x18,0x28,0x33,0xb1,0x1c,0x12, +0x40,0x69,0x94,0x99,0xdc,0x4d,0x16,0x47,0xf8,0x92,0xf9,0xb2, +0xc9,0x7e,0x8c,0x17,0xea,0x53,0x34,0x4f,0x08,0x97,0x15,0x9d, +0x10,0x61,0xad,0x5d,0x5b,0xc7,0xdc,0xd7,0xd4,0x45,0x16,0x36, +0x9f,0xe9,0xb3,0x17,0x4c,0xd9,0x27,0x97,0x13,0x9e,0x55,0x8a, +0x9f,0x6c,0xb0,0x37,0x95,0xa3,0xf9,0xc5,0x45,0xfe,0x09,0x7d, +0x93,0x6a,0x2c,0x43,0xbe,0x86,0x96,0x57,0xcb,0xd3,0xe1,0x08, +0x0d,0xd3,0xa0,0x6e,0x16,0x65,0x06,0x79,0xcd,0xe6,0xa6,0x8f, +0x81,0xb1,0x1b,0x44,0x1e,0x13,0x20,0x10,0x78,0x4f,0x29,0x7c, +0x0c,0x54,0x34,0x99,0x07,0xa7,0x17,0x18,0x53,0x7c,0xb6,0x62, +0xb5,0xdd,0x65,0xde,0x64,0xa7,0x35,0x7c,0x98,0xd8,0x5b,0x21, +0x82,0xe4,0x34,0x3c,0xc3,0xd3,0xc3,0x08,0x8a,0x67,0x28,0x8c, +0xc7,0xe7,0x1b,0x14,0x8d,0x20,0x3b,0x7b,0x8f,0xcc,0xab,0x2c, +0x8a,0x05,0x80,0x9d,0x87,0xdf,0x7e,0x93,0x90,0x65,0xda,0x47, +0xdf,0xec,0xdb,0x47,0x0f,0xdd,0xa8,0x88,0xcc,0x4b,0x28,0xb0, +0x68,0x0f,0x4a,0x5f,0x84,0x1a,0xd2,0xae,0x23,0x1d,0xa7,0xc1, +0x65,0xd4,0x75,0x64,0x48,0x64,0x78,0x2c,0x13,0x92,0xae,0x00, +0x7a,0x6f,0x6f,0xdf,0x45,0x5c,0x45,0x44,0x34,0x43,0xd2,0x3b, +0x8e,0x6e,0x34,0x40,0xc0,0x6f,0xf6,0x5d,0xc4,0x20,0xc2,0x5a, +0x50,0x69,0x36,0xfa,0x10,0xe0,0x1e,0xea,0x18,0x1b,0x71,0x18, +0x29,0x59,0xa5,0xdd,0x84,0x31,0x63,0x6d,0x01,0xe2,0x33,0x64, +0xd1,0x3f,0x9f,0x23,0x6a,0x01,0xdf,0x82,0xe9,0x5e,0x2f,0xf3, +0x3b,0x08,0x63,0x1f,0xf6,0x32,0xd9,0x0f,0xfb,0xbb,0x14,0xfd, +0xa8,0x87,0xc3,0x5c,0x95,0x86,0x1d,0xcf,0x4c,0xcd,0xcc,0x90, +0xc9,0x56,0x3b,0x2c,0x60,0x32,0x90,0xc4,0x89,0xd4,0x42,0xc0, +0x55,0x16,0xca,0xad,0x60,0xea,0x05,0x41,0x96,0x4c,0xd1,0x15, +0xdc,0xe0,0xd7,0x79,0x31,0x1b,0x0e,0x98,0xb5,0x43,0xe3,0x90, +0x15,0xc0,0xdb,0x01,0x42,0x68,0x83,0xe3,0x89,0x0d,0x57,0xe7, +0xa1,0xc7,0x32,0x9f,0x63,0x3b,0xa9,0x82,0x5c,0xfb,0x43,0x30, +0xf3,0x1f,0x07,0xb1,0xdb,0xd9,0x77,0x09,0xd0,0xb1,0xfd,0xdd, +0x7e,0x90,0x52,0x86,0x6d,0x00,0xac,0x5f,0x05,0xa3,0x0f,0x0e, +0x6a,0x37,0x4d,0x7d,0x96,0x8a,0x23,0xee,0x60,0x91,0x2d,0x51, +0xb7,0xd0,0x5e,0x42,0xb9,0xd9,0xc7,0xe1,0x04,0x92,0x71,0xb2, +0xf3,0x1d,0x0d,0xe8,0x34,0x00,0xf9,0x8c,0xa9,0x9e,0xc0,0xa9, +0x24,0x43,0x45,0x09,0x90,0x27,0x3a,0xad,0x6a,0xa7,0x20,0xd2, +0x2c,0xcc,0x40,0xf1,0x93,0xc1,0x40,0x59,0xbc,0x6a,0x50,0x5e, +0xa9,0xa9,0x08,0xa9,0x3d,0x2d,0x9a,0x3b,0x10,0x39,0x53,0xa8, +0xc3,0xbf,0x69,0x65,0x8f,0xea,0x02,0x83,0xbe,0xdc,0x1e,0x34, +0xe2,0xc1,0x7e,0x15,0xdd,0x9d,0xef,0x52,0x5a,0x01,0x58,0x38, +0xfa,0xdd,0x83,0x18,0x70,0x6f,0xb7,0x92,0x46,0xcc,0x85,0x04, +0xc3,0x86,0x1e,0x4d,0x61,0x29,0x8d,0x96,0xba,0xc5,0x66,0x75, +0x22,0xd9,0xdd,0x0e,0x46,0xc0,0x9f,0xed,0x96,0xc3,0xeb,0x41, +0x78,0xb1,0x52,0x80,0x0b,0xa5,0xda,0xc4,0xa5,0xbf,0xe7,0x05, +0xaf,0x55,0xab,0x98,0x1c,0x56,0xbe,0xb1,0x7c,0x97,0xb9,0xf9, +0xb6,0x0c,0x8d,0xdb,0x9f,0xdc,0x19,0x66,0x6b,0x6b,0xb5,0x72, +0x31,0xbb,0x42,0xcb,0x98,0x7f,0xa1,0x71,0xc3,0xe7,0x9e,0xf8, +0x56,0x68,0x0a,0xef,0x2e,0x93,0xb4,0x66,0x23,0x14,0xcc,0x60, +0x4e,0x88,0x2a,0x44,0x50,0x49,0xf3,0x46,0x1f,0x01,0xc5,0x36, +0x88,0xcf,0xbf,0xb0,0xb4,0x82,0x2b,0xef,0xb3,0x46,0xe2,0x50, +0xc4,0xfb,0x66,0x2d,0xcf,0x03,0x37,0x47,0x49,0x4a,0xed,0xd3, +0x31,0x8c,0x16,0x2b,0x2d,0x55,0xf0,0x64,0xe1,0x31,0x32,0x5d, +0x83,0x73,0x4d,0x6b,0x19,0x52,0x05,0xc4,0xf8,0xdd,0x17,0xf0, +0x4e,0x3f,0xe2,0x93,0x0b,0x8d,0xf2,0x88,0xde,0x5e,0xc0,0x23, +0x73,0x32,0x74,0x65,0x35,0x1f,0xf9,0xe1,0xb9,0x1f,0x6e,0xe8, +0xcd,0x47,0xbc,0x6e,0x61,0x25,0x30,0x18,0x1a,0xf3,0xd9,0x39, +0xbf,0xc3,0xcf,0x0b,0x10,0x6b,0x46,0xfa,0x4f,0x94,0x3a,0xcb, +0x31,0xe2,0x72,0xbd,0xc3,0xaf,0xd3,0x07,0x7a,0x2b,0x1b,0xce, +0xa2,0x86,0xce,0xda,0x00,0x8d,0x54,0xc3,0x8c,0x54,0xbd,0xe0, +0xa3,0x07,0xe4,0xf8,0x37,0xb5,0xb9,0x2f,0xbd,0xdc,0x49,0x30, +0x7e,0xcd,0xb3,0xcc,0x1f,0x5d,0x15,0x6d,0xf5,0xec,0x23,0x49, +0x5b,0xcf,0xba,0xa7,0xec,0xf0,0x1e,0x97,0x40,0x6a,0xd0,0x3f, +0x29,0x44,0x01,0xea,0xa2,0x7f,0x8e,0x66,0x54,0x2b,0xca,0x89, +0xfa,0x65,0x6b,0x74,0xa6,0xb0,0x06,0x14,0xb6,0x43,0x99,0x83, +0x72,0x7e,0x44,0xe3,0x9c,0x1a,0xb2,0x6a,0x56,0xa1,0xd1,0x09, +0x3d,0xd8,0x91,0xd6,0x6b,0xb9,0x92,0x27,0x4b,0xf3,0x02,0x96, +0xc2,0x2a,0x57,0xff,0x78,0xf6,0xf6,0xe8,0xc5,0xeb,0x57,0x59, +0xb4,0xd7,0x7c,0xd8,0xdc,0xe5,0x7d,0x99,0xde,0xf1,0xe1,0xa7, +0x82,0x5b,0x9f,0x6c,0x57,0xe1,0x85,0x4b,0xd6,0x52,0x70,0xa9, +0x92,0xed,0x29,0x10,0x03,0x65,0x0f,0x15,0xdd,0x44,0x64,0x8f, +0x14,0x5d,0x99,0x67,0xdf,0x28,0x71,0x81,0x9e,0x7d,0xeb,0x54, +0xb0,0xe1,0xf5,0xb2,0xd5,0x7a,0xce,0x04,0xd5,0x96,0xb9,0xbc, +0xc6,0x24,0x78,0xaa,0x09,0xbf,0xfa,0x48,0x74,0x33,0x99,0xce, +0x0a,0x15,0x1c,0x95,0xb2,0x9d,0x5f,0x4e,0x76,0x3a,0x30,0xc0, +0x4f,0xe2,0xf8,0x64,0xbb,0x73,0x32,0x68,0x24,0x3b,0x39,0x43, +0xc1,0xc6,0x7e,0x74,0xd3,0x1f,0x0b,0x68,0x0d,0xa3,0x67,0x6a, +0x27,0x8d,0x4f,0x06,0x0f,0x92,0x93,0xa6,0xfe,0x86,0x1c,0x27, +0x29,0xfe,0xe8,0x78,0xfe,0x3a,0x69,0x42,0xc2,0xed,0xc3,0x65, +0xd2,0x49,0x3a,0x3b,0x0a,0x4e,0xb2,0xdc,0x49,0xa6,0xd4,0x58, +0xe3,0x79,0x03,0x88,0xf4,0x7f,0xdd,0xdd,0xed,0xef,0x55,0xb3, +0xf7,0x20,0xf9,0x17,0xe1,0xb6,0xe1,0xc3,0x20,0x7c,0x80,0xe1, +0x63,0x2f,0xee,0xa7,0x3b,0xf2,0x1c,0x01,0x09,0x0b,0x17,0x7e, +0x9f,0xdc,0xdb,0x51,0xa1,0x5c,0x1a,0x2a,0xd6,0xed,0xfe,0x02, +0x42,0x31,0x90,0x91,0xc5,0x27,0x27,0x1a,0xcb,0xe1,0x64,0x71, +0x78,0xd8,0x81,0xff,0x2f,0x0e,0x26,0x8b,0x83,0x03,0xfc,0xd3, +0x81,0xff,0x2f,0x06,0x83,0x41,0x67,0xa0,0x7f,0x26,0x9d,0xc5, +0xc7,0xee,0x64,0xf1,0xb1,0xd7,0x59,0xbc,0xd7,0xbf,0xef,0xf5, +0x2f,0x6a,0x94,0x2c,0xf8,0xcf,0xe2,0xf2,0x32,0xbe,0xbc,0xbc, +0x04,0x0a,0x7e,0xfc,0x31,0xfe,0xf1,0xc7,0x1f,0xe1,0x6b,0xb8, +0x78,0xb6,0xe8,0x2f,0x9e,0x2c,0xae,0xae,0x3a,0x8b,0x9f,0x7e, +0xea,0x2c,0xae,0xaf,0x3b,0x8b,0xa2,0xe8,0x2c,0x8e,0x8e,0x3a, +0x47,0x9d,0xc5,0x3f,0x17,0x7f,0xfc,0xd1,0x59,0xfc,0xfc,0x73, +0x67,0x81,0x72,0xb9,0x4a,0x8d,0xc9,0x4a,0x72,0x5f,0x1e,0x2f, +0x5e,0xbe,0xec,0xc0,0xff,0x17,0xa3,0xdb,0x96,0x7a,0xb4,0x84, +0xec,0x75,0x6a,0x65,0xd9,0xce,0xc9,0xe0,0x64,0xa0,0xbb,0xa6, +0x5e,0x97,0x10,0x40,0x34,0x9e,0x87,0x4b,0x09,0x14,0xa6,0xfb, +0xa9,0x4e,0x1d,0x92,0x33,0x3f,0xf2,0x92,0xad,0xb6,0x63,0xb6, +0xd3,0x6d,0x9c,0x6c,0xf7,0x3a,0x08,0xf3,0xad,0x07,0x03,0xca, +0x87,0x3a,0x59,0xf7,0x56,0xef,0x41,0xf7,0x7e,0x7f,0xfb,0x8f, +0x93,0xf9,0xee,0xee,0x93,0xdd,0x6d,0xfd,0xf3,0xcd,0xf3,0xe7, +0xfa,0xef,0x77,0xbb,0x10,0x38,0xf8,0x0e,0x02,0xcf,0xbf,0xc7, +0xc0,0xf3,0x83,0xa7,0x10,0x38,0x78,0x8e,0x81,0xe7,0xcf,0x9e, +0xf7,0x1a,0x8b,0xae,0x06,0xfd,0x16,0x53,0x77,0xbf,0xd5,0xa0, +0x3b,0xbd,0x46,0x7c,0x52,0x3c,0xe8,0xf8,0xd1,0xbd,0x46,0xa2, +0x09,0xd8,0x5b,0xee,0xe4,0xaa,0x2c,0xf8,0xc8,0x76,0x7e,0xd6, +0x58,0x80,0x4e,0x68,0xaa,0xb4,0x03,0x7f,0x7d,0xc0,0x6c,0xe7, +0xb8,0x94,0x93,0xf5,0x1a,0x75,0x15,0x4e,0xb8,0x8a,0xba,0xe0, +0x26,0xb7,0x24,0xcf,0x05,0x3b,0xf3,0x8a,0x07,0x3a,0xe1,0xd1, +0x72,0x9b,0x26,0xd4,0xc9,0x60,0x1b,0xfe,0xe8,0xfe,0x7c,0x0f, +0xbf,0xf7,0xcc,0xc7,0x36,0xc6,0xc1,0x97,0xfe,0x48,0xe2,0xf8, +0x78,0xb1,0x91,0x60,0x30,0x4e,0xc5,0x5f,0x28,0x44,0xf7,0xa8, +0xfe,0x2f,0xe1,0xff,0xc7,0x21,0xf5,0x10,0x79,0x0f,0x29,0x60, +0xd1,0x26,0xaa,0x3f,0x6d,0x1f,0x1e,0x6e,0x1f,0x1c,0x1c,0xff, +0xf4,0x53,0x7a,0x7d,0x9d,0x16,0xc5,0xcf,0x91,0x32,0x82,0xa7, +0xac,0x2b,0x00,0x22,0x85,0x6a,0x27,0xdb,0xdd,0xf7,0xbd,0xf7, +0xef,0xbd,0xc0,0xf6,0x33,0x1d,0x44,0x40,0xd0,0xf0,0xea,0x29, +0x23,0x58,0xca,0xba,0xdd,0xc8,0x60,0x6d,0x1e,0x45,0x6a,0x07, +0x29,0x47,0x72,0xec,0x1f,0xdb,0x34,0x3b,0x3d,0xe5,0xa0,0xab, +0x61,0x1d,0x48,0x29,0x9d,0x93,0x64,0xfc,0x4e,0x8f,0xaf,0x7e, +0x02,0x5d,0x43,0x3d,0x77,0xa8,0x5d,0x16,0xd8,0x24,0x3b,0x97, +0x39,0xbc,0xe3,0xfd,0xf4,0x99,0x36,0x98,0xc2,0x6e,0xd7,0x45, +0x86,0xb6,0x4f,0x17,0x3f,0xc1,0x49,0x6d,0xc1,0xda,0x42,0x8b, +0x23,0x3a,0x0c,0xe9,0xb0,0x3b,0x19,0x45,0xe6,0x8d,0xc5,0x02, +0x6d,0x0e,0xe6,0x33,0x91,0xf8,0x1c,0x77,0x1b,0xbd,0x5c,0xcb, +0x0c,0x69,0x4b,0x1d,0x99,0xaf,0xa1,0x5e,0xf4,0x59,0x67,0xf1, +0xdb,0xe1,0x23,0x85,0xa5,0xa5,0x60,0xe7,0x42,0x69,0xd6,0xb4, +0x48,0xff,0xfe,0xed,0x23,0xfd,0x89,0x57,0x8d,0x45,0xba,0xf7, +0xcd,0xf7,0x7b,0xc3,0x6f,0x15,0xb0,0x91,0x1a,0xa6,0xf5,0x8d, +0x06,0xfb,0x76,0xa9,0xc4,0x2b,0xe8,0xec,0xf6,0xba,0x48,0x23, +0x71,0x6a,0xd3,0x87,0xd8,0x34,0x32,0x9f,0xd7,0x90,0x04,0x45, +0xe9,0xe3,0x67,0x1a,0xc1,0x09,0x54,0x9f,0x3e,0x52,0x7c,0x9b, +0xab,0x0e,0xe0,0x17,0x52,0x3e,0xa6,0xf4,0x86,0x57,0xbd,0x4f, +0x23,0xdd,0x91,0xef,0xf1,0xfb,0x30,0xe5,0x17,0xb8,0xea,0x73, +0x1a,0x01,0x4b,0xab,0x33,0x1c,0x60,0x16,0xe6,0x6a,0x23,0x35, +0xa4,0x7c,0x88,0xec,0x99,0xcd,0x8a,0xc1,0xcb,0x4b,0x4a,0x23, +0xc0,0x1f,0x7f,0xb4,0xa9,0x18,0x01,0x9e,0x55,0xe4,0x93,0xed, +0xec,0x56,0x67,0x9a,0x5c,0x20,0xbf,0x2d,0x0b,0xd0,0x79,0xcc, +0xf1,0xc0,0xc3,0xce,0xf1,0x82,0x5a,0x08,0x52,0x76,0x57,0x2a, +0x43,0x51,0xb4,0x5f,0x7c,0xa0,0xdf,0x0f,0x3b,0x6b,0x60,0xc7, +0x06,0x86,0xc2,0xc1,0xc1,0xc6,0xc7,0x8d,0xf7,0x1b,0x87,0x1b, +0x07,0x1b,0x03,0xdb,0xe1,0x1b,0xba,0xc3,0x6f,0xf4,0x41,0xce, +0x88,0xba,0xb3,0x08,0xd2,0x7f,0xda,0xb8,0xda,0xb8,0xde,0x28, +0x00,0xde,0x83,0xac,0xba,0x9e,0xd3,0x03,0xa3,0xce,0xc0,0x8f, +0xb1,0xcf,0xd0,0xfa,0x8f,0xa5,0x3a,0x3c,0x14,0x60,0xc1,0xfb, +0x08,0x69,0x3d,0x87,0xf4,0x41,0x8f,0xae,0xc0,0xa9,0x02,0x19, +0x2b,0x24,0x58,0xc4,0xf0,0x25,0x28,0x82,0xdc,0x07,0x75,0x44, +0x92,0xc1,0x89,0x25,0x0e,0x87,0x5a,0x10,0x7b,0x60,0x5d,0xea, +0xf1,0x56,0x0f,0x85,0xe9,0x83,0xb5,0x88,0x34,0xf2,0x35,0x3d, +0x75,0x02,0x4a,0x07,0x5f,0x88,0xa1,0xaa,0xad,0x06,0x5f,0x8a, +0x24,0xc8,0xff,0xb1,0xae,0x92,0x00,0x0f,0xb5,0x7c,0x5f,0x07, +0xc0,0x23,0x13,0x60,0xfe,0xf5,0xaf,0x0a,0x20,0xcf,0x24,0x0b, +0xe6,0xa0,0xb3,0x3e,0xbc,0x11,0x56,0x7b,0x98,0xeb,0x8b,0xf2, +0xa9,0x47,0x9c,0xe7,0x8b,0x32,0x7d,0xa3,0x33,0xe9,0x49,0xbd, +0x4e,0x0e,0xa7,0xfc,0x66,0x49,0x04,0xdd,0xc6,0x55,0x0d,0x64, +0x86,0xca,0x65,0x0d,0xe0,0xaa,0x42,0x90,0x34,0xbd,0xc2,0xac, +0x93,0x4b,0xac,0x02,0x82,0x3a,0xd8,0xd6,0xee,0xe8,0x1d,0x43, +0xe0,0x8f,0x35,0xb0,0x77,0x14,0x85,0x34,0xd6,0x9a,0xf5,0xe2, +0x21,0x05,0xf8,0x9f,0xdd,0x41,0x07,0x83,0xf5,0xeb,0xc0,0xcc, +0x84,0xe6,0x47,0x19,0x44,0x0a,0x0a,0x1e,0x63,0x3a,0xe5,0xb1, +0xca,0x0e,0x04,0x37,0x77,0x35,0xa6,0x27,0x7f,0x09,0xa6,0x96, +0xc6,0xf4,0x53,0x1d,0x26,0xce,0xa4,0x0f,0x2f,0x77,0x40,0xfc, +0xad,0xb5,0xb7,0x58,0xb4,0xf6,0x96,0x7a,0xd7,0xaa,0x5b,0x1f, +0x4d,0x91,0x4b,0xbd,0xc9,0xd5,0xc0,0xf0,0x86,0x0b,0x30,0x47, +0x55,0x30,0xa8,0xef,0xc5,0xd8,0xdc,0xfe,0x1c,0x27,0x3b,0x7a, +0x38,0x40,0x9e,0xaa,0x4c,0x7e,0xf7,0xae,0xc0,0x90,0xe0,0x80, +0x3a,0x5a,0x03,0x49,0x39,0xbb,0x7a,0xa8,0xb3,0xfe,0x9c,0x06, +0x76,0xf8,0xfa,0xd9,0x36,0xc2,0x02,0x3b,0x03,0xe6,0xf6,0xb3, +0xa8,0x61,0xcf,0xfe,0xbb,0xfb,0x7d,0xf4,0xe3,0xb1,0xdd,0x87, +0x78,0x78,0x38,0x7a,0xd6,0xa8,0x20,0xb5,0x8f,0xb2,0xf1,0xbd, +0xa4,0x11,0xa5,0x51,0x65,0x3a,0xa8,0xbb,0x21,0xe1,0x3f,0xff, +0x3b,0x8a,0x6f,0xed,0x3e,0xd0,0x14,0xd0,0xba,0xf3,0x47,0x5d, +0xaf,0x01,0xfe,0x27,0x67,0x67,0x38,0xc9,0xfe,0x58,0x09,0x05, +0xfe,0x19,0x00,0xea,0x9f,0x75,0x40,0x9a,0x39,0xfa,0x04,0x42, +0x0b,0x05,0x5e,0x9b,0x81,0xa1,0xa5,0xed,0x4d,0x33,0xac,0x62, +0xab,0xd4,0x21,0xb3,0x92,0x8b,0xcf,0x30,0x49,0xef,0x35,0x51, +0xaf,0x1d,0xb2,0x06,0xf6,0x59,0x6f,0x9e,0x95,0x92,0xc8,0x0f, +0x42,0xa5,0x56,0x4e,0xde,0x88,0x26,0x51,0x2f,0xcc,0x12,0x57, +0xc3,0xf6,0xc0,0x5c,0x08,0xea,0x47,0x4a,0x66,0x43,0x94,0xec, +0x45,0xaf,0x2c,0x35,0xef,0x65,0xce,0x06,0x60,0x4d,0x61,0x7b, +0x54,0x58,0x55,0x72,0x13,0x1e,0x71,0xac,0xc6,0x00,0x20,0x7a, +0xf4,0x1a,0x13,0x7b,0xe6,0x81,0xb2,0x34,0x82,0x58,0x0c,0x67, +0xae,0xbf,0xa4,0xe8,0x13,0x0d,0xea,0xe4,0xa4,0x4f,0x01,0xc6, +0x73,0x38,0x0d,0xa2,0x59,0xfe,0x09,0xf4,0x55,0x18,0xe4,0x07, +0x88,0x0e,0x74,0x38,0xa8,0x89,0x42,0x00,0xe5,0x57,0xdd,0xe8, +0x34,0x6a,0x70,0xc4,0x52,0xb1,0xa9,0xae,0x34,0xfa,0xaf,0xfe, +0x78,0xde,0x9f,0x7e,0x3e,0x7d,0x3e,0x3c,0x9b,0xe2,0xc7,0x61, +0x7f,0x7a,0x7e,0x75,0xfa,0xe4,0x66,0x9a,0x8f,0xf4,0xf7,0xe7, +0xd3,0xff,0x9a,0x8f,0x87,0xfa,0xcf,0xe8,0xf3,0xe9,0x93,0xf9, +0xe5,0xbc,0x98,0x9d,0x1e,0x0d,0x6f,0x66,0x43,0xb8,0x0f,0x3b, +0x7d,0xad,0xd9,0x79,0xf8,0x7d,0x35,0xf9,0x9d,0x22,0x0e,0x86, +0xe7,0xf8,0x61,0x39,0xbd,0xd3,0xc8,0x18,0xdc,0x72,0x75,0xbc, +0x0e,0xe4,0x5d,0x94,0xde,0xb5,0x36,0xb8,0x7a,0x96,0x3a,0x1c, +0x74,0x48,0x22,0x90,0x07,0x94,0x01,0x5d,0x86,0x2a,0x20,0x0a, +0x68,0x02,0x82,0x80,0x14,0x20,0x03,0x28,0xa8,0x28,0x9c,0x10, +0xdd,0x41,0x01,0x02,0x79,0x64,0x88,0x07,0x45,0x22,0x33,0x44, +0xbe,0xb2,0x06,0x35,0x73,0x10,0x1e,0x2a,0xd4,0xf8,0x21,0x75, +0x40,0x81,0x11,0x73,0x82,0x03,0xc4,0x30,0x2e,0xeb,0xf6,0x48, +0x51,0xb8,0xb5,0xe7,0x54,0x15,0x4a,0x50,0x68,0x8f,0x0a,0xee, +0x47,0x8c,0x48,0x14,0x0c,0xe7,0x74,0xf7,0xf4,0x21,0x29,0xef, +0x25,0x54,0x64,0x16,0xfd,0x12,0x35,0x1c,0x83,0x5c,0xe0,0x9d, +0x12,0x9a,0x7d,0x5f,0xf8,0x09,0xc4,0xd9,0x71,0xaa,0xaa,0x2a, +0xaa,0xf4,0x94,0xce,0x3d,0x97,0x6b,0xe2,0x6b,0x39,0x7a,0x2c, +0x57,0x99,0x97,0x5e,0xc8,0xba,0x96,0xb1,0xae,0xb6,0x75,0x57, +0x9a,0x15,0x23,0x8d,0x8e,0xe6,0x63,0xfd,0x71,0xaa,0xcf,0x6f, +0xf0,0x73,0x3c,0x1f,0x16,0xf0,0xfb,0x7e,0x38,0x18,0xd3,0xd7, +0xf1,0x95,0xde,0xec,0xe0,0xe3,0xb9,0xde,0x59,0xf5,0xcf,0x51, +0x5f,0x23,0x81,0x63,0x8d,0xec,0x50,0x8b,0xae,0xb6,0x37,0x0d, +0x44,0x97,0x2f,0xb5,0x7a,0x82,0x08,0x1e,0x51,0x9a,0x12,0x20, +0x03,0x68,0x80,0xf2,0xa1,0x64,0x28,0x14,0x4a,0xac,0x2c,0xec, +0x8e,0xf1,0xe3,0x81,0x55,0x15,0xab,0x57,0x4b,0x28,0x54,0x97, +0xa9,0x8b,0xd4,0x25,0xea,0x02,0x75,0x79,0xba,0xb8,0xca,0xd2, +0x0e,0xa5,0x51,0xd4,0xba,0xb2,0x34,0x90,0x28,0xc9,0xbb,0x62, +0x77,0x99,0x39,0xfa,0xae,0xd1,0xea,0xe5,0xb6,0xe3,0xd5,0x8b, +0xb5,0x23,0xf6,0xbb,0x70,0xc0,0x7a,0x60,0xe1,0x90,0xa5,0xe1, +0xda,0xea,0x25,0x48,0x69,0x5e,0x1a,0xb5,0xf6,0xcc,0x50,0x1e, +0xb7,0xfe,0x99,0xa4,0x3e,0x1d,0x4e,0x3d,0xfe,0xb8,0x0e,0x29, +0xfa,0xb2,0x91,0x1d,0xe6,0xa6,0xb1,0x2d,0x5b,0x52,0x8e,0x6e, +0xff,0x05,0x7a,0x7a,0xfb,0xf2,0x38,0x8d,0xae,0xd2,0xeb,0xeb, +0x8d,0x27,0x91,0x7a,0x99,0x46,0x87,0x87,0x3b,0x07,0x07,0x3b, +0xf8,0xac,0x4e,0xbd,0xc4,0xf0,0x21,0x1c,0x8a,0x4d,0x84,0x1f, +0xb3,0xf1,0xf2,0x18,0x23,0x75,0x2c,0x1c,0xba,0xd4,0x46,0x90, +0xa6,0xf7,0x6d,0xbf,0x34,0x67,0x07,0xc4,0xd8,0x0c,0x61,0xd3, +0x05,0x54,0x13,0x1f,0x1a,0x4d,0x70,0x18,0xe7,0x3d,0x04,0xb7, +0xb5,0x55,0x07,0xd8,0x9c,0x4d,0xde,0x81,0xef,0x51,0xb2,0xbf, +0x82,0x2e,0xe4,0x56,0x63,0x0e,0x32,0x38,0x4d,0x47,0xa8,0xc3, +0x42,0xff,0x77,0x70,0x00,0x72,0xe2,0x81,0x54,0xae,0x0c,0x5e, +0x1c,0xb1,0xf6,0x4f,0x0b,0xfc,0xd3,0xd4,0x56,0x20,0x23,0x3a, +0x12,0xc5,0x76,0x13,0x14,0xbc,0x25,0x4c,0xc3,0xeb,0x21,0xbe, +0x14,0xbc,0x81,0xe7,0xb0,0x14,0x87,0x6f,0x6e,0x7d,0xa3,0x32, +0xa0,0xc4,0xf7,0x64,0x16,0x03,0x6b,0xeb,0xbf,0xd5,0x4e,0x77, +0xba,0xfd,0x9b,0xde,0x49,0xb3,0x73,0xdd,0xd1,0x7f,0x76,0xf4, +0x9c,0xe1,0x54,0x57,0x8e,0xb1,0x9b,0x49,0x56,0x34,0xf3,0x02, +0xf1,0x3a,0x9f,0x2d,0x90,0x0a,0x06,0xa3,0x38,0xa1,0x13,0xdd, +0x5c,0x6b,0x0e,0xf3,0xcd,0x61,0x94,0xda,0x98,0x3e,0xc4,0x3c, +0x39,0xd4,0x9d,0x7a,0x7a,0xde,0xd7,0xcc,0xcb,0xa0,0x3f,0x4d, +0x6f,0x0b,0x3d,0xbc,0x0e,0x40,0xa6,0xd3,0x3d,0x9e,0xe8,0xd1, +0xb6,0xd1,0x9f,0xf5,0x70,0x50,0x80,0xfd,0x10,0x13,0x7f,0x3d, +0x99,0x82,0x41,0x71,0x99,0xf4,0x1e,0x45,0x3e,0xd0,0xbc,0x1b, +0x5d,0x13,0x0f,0x66,0x04,0x28,0xcb,0xbf,0xf4,0xf8,0x1d,0x4e, +0x25,0x3a,0x48,0xa3,0x3c,0xdd,0x97,0xfa,0xb3,0xb7,0xe1,0x67, +0x05,0x2a,0x9e,0x8d,0x74,0x3b,0x44,0x2f,0x51,0x20,0xc5,0xd4, +0xc9,0xd1,0xa6,0x42,0x63,0x19,0x6c,0xb2,0x99,0x61,0xe5,0x58, +0xab,0xf2,0x18,0x84,0x79,0x3a,0xff,0x1f,0x73,0x57,0xbf,0xd3, +0x48,0x8e,0xc4,0x5f,0x05,0x5a,0x0a,0x8a,0x49,0x37,0x24,0x0c, +0x03,0xab,0x66,0x9a,0xd6,0xdc,0x8d,0x74,0xda,0x93,0x98,0x59, +0x89,0xf9,0xe7,0x06,0x21,0x94,0xd9,0xc0,0x80,0xb4,0x81,0x15, +0x61,0x75,0xcb,0x91,0xbc,0xfb,0xb9,0xbe,0xec,0x2a,0xb7,0x3b, +0x30,0x8c,0xb4,0x5a,0x21,0x91,0xc4,0x2e,0xdb,0xd5,0x6e,0x7f, +0x94,0xcb,0x55,0xbf,0x62,0x5c,0x54,0xb2,0x94,0x80,0x0a,0x6b, +0x79,0xa9,0x17,0xda,0x78,0xa6,0x7e,0xba,0x42,0xdb,0x9a,0xba, +0xf0,0x42,0xd0,0xc0,0x0b,0xa5,0xbf,0x7b,0x9e,0xeb,0x62,0xb0, +0xd8,0x98,0x7e,0xbb,0x43,0x9d,0xde,0x74,0xe3,0xca,0xcf,0x70, +0x51,0x40,0x82,0x6a,0x6f,0xba,0x21,0xca,0xbd,0xb9,0xff,0x35, +0x98,0x09,0xe0,0x29,0x2a,0xfb,0xa6,0xf4,0x86,0xfc,0xf7,0x6b, +0xcc,0xc3,0xd7,0x85,0xda,0xbf,0xe9,0x06,0x6a,0xd1,0x66,0x33, +0x4c,0x27,0x11,0xf8,0x04,0x6b,0x23,0x85,0xdf,0xc9,0x09,0x55, +0xc6,0x62,0xf3,0x23,0x64,0x91,0x06,0xf0,0xf1,0x11,0x73,0xd0, +0x9e,0xc4,0x77,0x9a,0xe1,0x3f,0x74,0x5c,0xd6,0x06,0x29,0xb5, +0x20,0xea,0xf6,0xaa,0xae,0xec,0x8c,0xc8,0x5f,0xda,0xb9,0x2f, +0x6b,0x92,0xfb,0x3d,0x4e,0xda,0x01,0xe8,0xf6,0x05,0x02,0x17, +0xfb,0x9b,0x28,0xe3,0xa3,0x00,0xc6,0x25,0xcf,0x41,0x0e,0x44, +0x42,0x4a,0xf4,0x0c,0xc7,0x40,0x7a,0x3c,0x6e,0x0b,0x7a,0x8b, +0x7e,0xd4,0x43,0x7d,0xc5,0x9a,0x27,0xe0,0x3b,0x6c,0xfa,0xe0, +0x85,0x47,0x6e,0xb6,0x15,0x8f,0x0b,0xcf,0x23,0x67,0x06,0x25, +0x65,0xda,0xd7,0xc9,0xe6,0xc9,0x54,0x71,0xfd,0x1f,0xcc,0x8a, +0xf8,0x9c,0x92,0x5d,0x63,0xb2,0xdc,0xd4,0xc7,0x2a,0x93,0xc8, +0x35,0xf4,0xd3,0x77,0xcf,0xdd,0xe2,0xe1,0x2a,0x59,0x90,0x7b, +0x48,0x51,0x37,0x6b,0x01,0x67,0x94,0x09,0x88,0x32,0x0b,0x8a, +0x5b,0x11,0xb8,0xea,0xd9,0x9f,0x8f,0xa4,0x6a,0x63,0xf1,0xa2, +0x7e,0x22,0x0f,0x43,0x9f,0x5e,0x1f,0xf8,0x34,0x85,0xbd,0x52, +0x87,0xd0,0xc6,0xa4,0xcd,0xd6,0x59,0x7d,0x97,0xcf,0x8a,0x06, +0x70,0xdb,0xf9,0x82,0xd7,0xae,0xaf,0xa5,0xc2,0x6e,0x29,0xc9, +0x95,0x21,0x2c,0xb9,0x5f,0x29,0x8e,0x56,0x7c,0x99,0x40,0x04, +0xd6,0xc9,0x48,0x46,0xc6,0x52,0xe8,0x9f,0xa3,0x10,0x23,0x95, +0xe9,0xc7,0xd3,0xc5,0x0d,0xc1,0x12,0x95,0x17,0x57,0xfc,0xc6, +0xfd,0x1e,0x5b,0x63,0x31,0xf6,0x9a,0xa8,0xe9,0xa3,0xa4,0x5b, +0x7f,0xc0,0x61,0x73,0xab,0x32,0x0a,0xc8,0x2f,0xe1,0x15,0xef, +0xa8,0x8f,0x7e,0x84,0xe3,0xc6,0xf0,0x4c,0x7e,0x55,0xf5,0xe6, +0x38,0x30,0x35,0x8e,0x5c,0x67,0x1e,0xc8,0x3c,0xc8,0x8a,0x01, +0x31,0xe3,0x43,0xf8,0x53,0x7a,0xc7,0xe0,0xc1,0xa0,0x44,0xa1, +0x0b,0x38,0x1b,0xa8,0x96,0xc9,0xbd,0x7f,0xfa,0xfc,0x41,0x34, +0x58,0xdc,0x7c,0xbb,0xf5,0xeb,0xd2,0x03,0xdd,0x03,0xfd,0xbc, +0xb8,0xf3,0xf3,0x5f,0xee,0xa1,0x9b,0x60,0x85,0x12,0xe0,0xde, +0xd9,0x32,0x42,0xe2,0xad,0x35,0x1d,0x3c,0x50,0xaa,0x3e,0x34, +0x1b,0x8b,0x32,0xe8,0x14,0xe1,0x9c,0xd7,0x52,0x41,0xfb,0xb4, +0xe2,0x8e,0x20,0xb3,0x70,0x84,0xdf,0x0a,0x10,0x91,0x4c,0x03, +0xe8,0x6f,0x52,0x23,0x6d,0xf7,0x54,0x49,0x30,0x6a,0x30,0x48, +0xd5,0x94,0x47,0xa6,0xe0,0x4d,0xcf,0x25,0xbd,0xb6,0x7e,0x76, +0xed,0x10,0x3a,0x01,0x14,0x32,0x60,0xc2,0x88,0x16,0xe4,0x93, +0xf3,0xb6,0x9a,0xd4,0x93,0xf8,0x1c,0x4f,0x8f,0x30,0x99,0x6a, +0x52,0xd0,0x10,0x0d,0xba,0x7a,0xba,0x6d,0xec,0xc0,0x6b,0x93, +0x83,0xfe,0x2a,0x9c,0x33,0x37,0x39,0xec,0x45,0xc9,0x79,0x0b, +0x93,0xc7,0x08,0x00,0x52,0x6e,0x91,0x14,0x8c,0x18,0x01,0x44, +0x01,0x36,0x47,0xc1,0x94,0xde,0x58,0x0e,0x98,0x87,0xc3,0x80, +0x98,0x3d,0x4f,0x27,0xaf,0xdc,0x0c,0x0e,0x89,0x11,0x84,0x1d, +0xb9,0xb5,0x65,0x21,0x05,0xe2,0x12,0x59,0xfa,0xbf,0x9d,0x22, +0x18,0x15,0xb3,0xb5,0xaf,0x2f,0xe6,0xda,0x71,0x0d,0x1f,0xc4, +0xa4,0xe9,0x41,0x69,0x8f,0x9f,0x69,0xcf,0x9f,0x1d,0x4e,0xd2, +0xc4,0x37,0x3e,0x71,0x96,0x26,0xee,0xfb,0xc4,0xeb,0x34,0xf1, +0xad,0x4f,0x9c,0xa7,0x89,0x07,0x3e,0x71,0x91,0x26,0x1e,0xfa, +0xc4,0xff,0xa6,0x89,0x3f,0x61,0xa0,0x63,0xcf,0x3d,0x59,0x4c, +0x75,0xe2,0x19,0xa8,0x79,0x00,0x86,0x62,0xe8,0xd2,0x99,0xe0, +0xce,0x23,0x24,0x1b,0xa2,0xce,0xfb,0x6a,0x08,0xa0,0x8d,0x2d, +0xd6,0xf1,0x3b,0xd6,0x1e,0x66,0x22,0x44,0x25,0x81,0x7e,0x60, +0xe3,0x97,0x30,0x41,0x09,0xba,0x86,0x2f,0x9b,0xc3,0xb5,0x73, +0xce,0xf6,0x48,0x5b,0xeb,0x84,0x6a,0xb1,0x51,0x23,0x83,0x09, +0xd4,0x21,0xbe,0xc6,0x23,0x05,0xa0,0xc8,0x20,0x85,0x6d,0x40, +0x22,0xcc,0x63,0x2e,0x4a,0x05,0xec,0xb3,0xdf,0x70,0xa9,0x61, +0x02,0x61,0x08,0x4b,0x48,0x53,0x5c,0xde,0x16,0xae,0x4e,0xd1, +0x13,0xf3,0xe8,0x85,0xe5,0x7d,0xd3,0x35,0x46,0xe2,0x5e,0x4b, +0x60,0xee,0x32,0x36,0x72,0x54,0xc3,0xce,0x05,0x40,0x36,0xba, +0x3a,0xa1,0xa7,0x64,0xd3,0x27,0x1f,0x20,0x7c,0x81,0x3d,0x09, +0xc5,0xae,0xc0,0xe8,0xfc,0x54,0x54,0x7d,0xa5,0x5a,0xfc,0xdb, +0x84,0x27,0x4b,0x52,0x11,0xc0,0x38,0xc3,0xd2,0x2a,0xb5,0xd4, +0xeb,0x0d,0x32,0xa9,0xe3,0xf1,0x12,0xa9,0x2a,0xfb,0xa1,0xb3, +0x4c,0xaf,0x29,0x2d,0xc4,0xfe,0x94,0xd3,0xa0,0xca,0x56,0x20, +0xcf,0x27,0x08,0x57,0x76,0x54,0x55,0x37,0x2e,0x40,0x53,0x23, +0x01,0x02,0xdb,0xc1,0xc1,0x9e,0x1b,0xb4,0x90,0xd9,0xb1,0x51, +0xc6,0x1b,0x4f,0x71,0xaf,0x35,0x2c,0x79,0x64,0x9a,0x44,0x02, +0x85,0x30,0x0d,0xb6,0x93,0x16,0xd5,0x19,0xb7,0x30,0xf0,0x02, +0xb3,0xc6,0x77,0x48,0xd9,0xb2,0xf2,0x13,0x81,0x42,0x4b,0x2a, +0x5c,0x67,0xb1,0x48,0xc1,0xbb,0x6c,0x1e,0xda,0xc5,0x69,0xfc, +0xc5,0xc4,0x25,0xcf,0x6e,0x86,0x94,0x18,0xc9,0x61,0x2b,0x95, +0x26,0x65,0xf4,0x34,0x6c,0x0c,0xac,0x83,0xcf,0x40,0xc8,0xf3, +0x8c,0x1c,0xc4,0xb5,0xa2,0xe1,0x1f,0x47,0x9d,0xf9,0x74,0xd5, +0xa5,0x1b,0xb1,0xb1,0x1e,0x86,0xe4,0x60,0x55,0x08,0x45,0xc4, +0x02,0x2f,0x79,0x34,0x44,0xf8,0x33,0x53,0xbb,0x02,0x70,0x26, +0x0b,0x3c,0xb0,0xe7,0x5f,0x95,0x02,0xae,0xdf,0x27,0x97,0x21, +0xb3,0xfe,0xcc,0x8a,0x77,0x4f,0x38,0x17,0x77,0x58,0x3c,0x86, +0x63,0xdf,0x06,0xea,0x09,0x58,0x51,0x20,0x66,0x23,0x1b,0x67, +0xff,0x3a,0xf9,0x7c,0xfe,0xe5,0x4b,0x81,0xd5,0xaf,0x15,0xfa, +0x88,0xef,0x36,0x1a,0x94,0x1a,0xab,0x47,0x28,0xfe,0xf3,0xe9, +0xa7,0x5e,0x06,0x2d,0x60,0xa1,0xea,0x3d,0x92,0x6b,0x4a,0x65, +0x36,0x73,0xf6,0xf9,0x3c,0xda,0xc0,0x9c,0x9e,0x9e,0x7d,0x39, +0x27,0xee,0xd0,0x12,0x35,0xbd,0x60,0x99,0xe3,0x61,0x82,0x07, +0xd4,0xd9,0x5c,0xee,0x5b,0x83,0x82,0xb6,0x94,0x08,0x0f,0xe5, +0x3c,0x5c,0xbf,0xcd,0xd5,0xdd,0xdb,0x3c,0x5e,0x78,0x61,0xba, +0xbe,0x51,0x3a,0x5f,0x89,0xdb,0x5f,0xe6,0x81,0x82,0x43,0x20, +0x0d,0x01,0x3f,0x69,0x4f,0x3f,0x9f,0x42,0xf8,0xa6,0xcb,0xde, +0x0b,0xfb,0x8b,0x69,0xcb,0x97,0x91,0x0c,0xce,0x98,0xe2,0x80, +0x32,0x55,0x39,0xd4,0x16,0xab,0x6a,0xee,0x70,0xbe,0x2c,0x78, +0xe1,0x37,0x28,0x2d,0xc8,0x4e,0xd7,0xb9,0xe3,0x31,0x48,0xbb, +0xb8,0x99,0x8b,0x05,0x73,0x86,0x9f,0x68,0x31,0x4c,0x75,0xfc, +0x7e,0x85,0x60,0xe9,0x38,0xc9,0xde,0x3f,0xf4,0x3e,0x80,0xc2, +0x1d,0xf6,0x83,0xf7,0xe1,0xd7,0x75,0x17,0x4d,0xa8,0x33,0x41, +0x5b,0xbc,0x67,0xa8,0x4a,0xf5,0xb4,0xcd,0xe6,0x84,0x23,0x39, +0xa5,0x87,0x25,0x9c,0xc0,0xa7,0x0a,0x5f,0x8d,0x4f,0xbf,0x66, +0x60,0xa1,0x51,0x81,0xa2,0x84,0xb0,0x1a,0xdd,0x7d,0x35,0xac, +0x3f,0xfa,0xa6,0x36,0x1e,0xd0,0x86,0xe1,0xbc,0x38,0xd5,0x46, +0x0d,0x24,0x97,0xa2,0x3a,0x0a,0x4d,0xfa,0xff,0x08,0xbb,0xa9, +0xff,0xda,0xf4,0xb9,0xb9,0xd9,0x2d,0x6e,0x08,0x08,0x38,0x6c, +0xa4,0x5d,0xa7,0x79,0xb1,0x81,0xf2,0xb9,0x38,0x2e,0xf8,0x9c, +0xbe,0x60,0x39,0x71,0xdc,0x5d,0x0b,0x26,0xfe,0x9b,0xf3,0x5b, +0x05,0x86,0xe1,0xc0,0x9f,0x32,0x8b,0x7b,0x4a,0x39,0x5d,0xa0, +0x80,0xc9,0x6c,0x47,0x15,0x82,0x2f,0x18,0x74,0x07,0x7a,0x6a, +0xc8,0xda,0x8e,0xc3,0x29,0x5d,0x63,0x6b,0x4b,0x83,0xe3,0xd1, +0x4f,0x75,0xa0,0xc5,0x70,0x21,0x71,0x61,0xa6,0x2b,0xda,0x0a, +0x1a,0xe1,0xef,0x61,0x9c,0x50,0xec,0x8c,0xec,0x06,0x58,0x92, +0x71,0x17,0x7a,0x4b,0xfa,0xdf,0xcb,0x25,0x9b,0x7d,0x49,0x42, +0x8b,0x4a,0x90,0x66,0xff,0xcd,0xde,0xe5,0x5b,0x6e,0x48,0x23, +0xc0,0xbb,0x11,0xb6,0x67,0x92,0x44,0x6d,0xd9,0x4c,0xf6,0xb6, +0xb5,0x11,0x09,0xb1,0x46,0xdf,0x01,0xb7,0x4d,0x59,0x3c,0x51, +0xd6,0xdc,0x96,0x1f,0x35,0x48,0x53,0x99,0xd5,0xd6,0x8b,0x0d, +0xf7,0x0f,0x2a,0x7e,0x44,0x35,0x84,0xa2,0x91,0xc6,0xf7,0x7b, +0x87,0xc6,0xb9,0x5d,0xf5,0x1a,0xaa,0x6e,0x9f,0xad,0x6d,0x41, +0x88,0x86,0xaa,0x67,0xd7,0xb7,0x27,0xdd,0xcf,0xcd,0xda,0x1e, +0x0e,0x5a,0xe6,0x5d,0x00,0x10,0xf0,0xa7,0x1e,0xec,0x5f,0x7c, +0x50,0xa8,0x4d,0xfa,0x4e,0x4c,0xf8,0xc2,0x7b,0x00,0x32,0xd8, +0x47,0x83,0x45,0x9f,0xcd,0xf1,0xcf,0xc4,0x06,0x7e,0x36,0x1d, +0xec,0x0a,0xc9,0xe0,0xcf,0xbe,0xd1,0x4a,0x46,0x90,0xdb,0x45, +0x83,0x43,0xb6,0x01,0xec,0x25,0x3a,0x18,0xef,0xff,0xe4,0xa9, +0x20,0xd5,0xc9,0x08,0x67,0xf5,0x5b,0x1d,0x82,0x50,0x85,0x85, +0xe7,0xca,0x4f,0x9d,0x38,0x39,0x00,0x7f,0xcd,0x2a,0xe7,0x12, +0x39,0x24,0xce,0x49,0x1a,0x60,0xbe,0x11,0x2c,0xe4,0x58,0x16, +0xd0,0xeb,0x1c,0xc9,0xaf,0x3b,0xd7,0x7f,0xcc,0xa7,0xb7,0x7e, +0x2c,0x0f,0xad,0xab,0x22,0xb7,0xfd,0xf1,0x4e,0x19,0x64,0xe5, +0x5b,0xc6,0x3a,0x81,0x56,0x76,0x73,0x57,0xa6,0x35,0x75,0x95, +0xc2,0x71,0x4e,0x37,0x91,0xd3,0x00,0xfb,0x15,0x27,0x30,0x0f, +0x80,0x38,0x32,0xe0,0x05,0xf8,0xb9,0x46,0xca,0x56,0xc0,0x6d, +0x62,0x4d,0x62,0x75,0x70,0x0c,0x75,0xb4,0x85,0xa8,0xa3,0x8b, +0xba,0x9a,0x70,0x92,0x28,0xb0,0x8b,0x7a,0xac,0x52,0x3e,0xf8, +0x9a,0xea,0x89,0x2a,0x85,0x09,0x7b,0x9c,0xc0,0x3a,0xf4,0xa2, +0x3e,0x54,0x09,0x54,0x49,0x6c,0xc2,0x6c,0x1f,0xbc,0x67,0xe8, +0x2e,0x96,0xe7,0x16,0xf8,0x62,0x9c,0x17,0x28,0x23,0x48,0x08, +0x8b,0xac,0x3c,0x11,0xe2,0x5b,0xa8,0x39,0x2f,0x92,0xc5,0xda, +0x2d,0xd4,0xbd,0x33,0xe2,0x1f,0x2d,0x03,0x63,0x99,0x46,0xcb, +0xe5,0x33,0x94,0x6f,0x85,0x72,0x55,0xce,0xb4,0x80,0xa5,0xbc, +0x68,0xc0,0xef,0x4c,0x2f,0xbc,0x2c,0xf4,0x69,0x37,0xb5,0x5a, +0xa5,0x19,0xbf,0x35,0xe3,0x61,0xa3,0x1c,0x75,0xad,0x17,0xb1, +0xea,0x3f,0x16,0x08,0xc0,0xdb,0xef,0x69,0x46,0xea,0xa0,0xca, +0x13,0xae,0xfc,0x64,0x87,0x0f,0xba,0x70,0xef,0x63,0xf3,0xd3, +0x15,0xae,0xa2,0xe5,0xf7,0xf8,0xfd,0xf4,0xfa,0x6a,0x11,0xb7, +0xc6,0x9e,0xab,0x8b,0x1e,0xda,0xe3,0xa3,0x8c,0x7d,0x54,0x0f, +0x23,0x4b,0x3c,0xe2,0x49,0x18,0x8d,0xdf,0x27,0x22,0xff,0x68, +0xb7,0xa3,0x02,0x4b,0x14,0xc1,0xf5,0x28,0x92,0x9f,0x48,0x3c, +0x2c,0xf5,0xac,0xdd,0x5d,0xc5,0xbd,0xc6,0x5b,0x29,0x34,0x0a, +0xe6,0x61,0x34,0xf1,0xea,0xf4,0x2c,0xc8,0x48,0xae,0x6b,0x36, +0x44,0xc6,0x77,0xc5,0x35,0xbb,0x56,0x7b,0xd4,0x98,0x01,0x1b, +0x69,0x8d,0xaf,0x75,0x07,0x30,0x72,0x24,0xce,0x32,0xfc,0x2a, +0x76,0xc9,0x8c,0x0a,0x09,0xf3,0x51,0x19,0xba,0x49,0x4d,0xb8, +0x5a,0xd7,0xc6,0xb0,0x2d,0x34,0x42,0x4b,0x7c,0x6d,0x2c,0xda, +0x24,0x93,0x77,0x86,0xba,0x6b,0x43,0xe6,0x25,0x56,0xd6,0xf7, +0xda,0xc5,0xdc,0x98,0x19,0x7a,0x99,0x22,0x30,0x18,0x37,0xa5, +0xd4,0xca,0x30,0xc8,0x39,0x7e,0x25,0xc8,0xf4,0xe3,0x0b,0x04, +0x0b,0xe2,0x9d,0x17,0x40,0x4a,0x23,0x0f,0xd8,0xb4,0xf1,0x96, +0xbb,0x8e,0xa4,0xa7,0x89,0xdb,0x11,0x59,0xd0,0x6f,0xa8,0x7e, +0xb5,0x9c,0xe0,0x1a,0xf2,0xde,0x1f,0x4c,0xee,0xb3,0xf2,0x56, +0xc2,0x4c,0xce,0x31,0x8a,0x9a,0xc1,0xff,0x89,0xad,0xfc,0xc8, +0xac,0x27,0x96,0xdb,0xe3,0x91,0x95,0xbd,0x6c,0x2e,0x30,0xf5, +0x8f,0x4b,0xbf,0x42,0x5e,0xfe,0xb5,0x5c,0xbd,0x7b,0x96,0xab, +0xd3,0xe9,0xfc,0x2f,0xe6,0xc9,0xbf,0xc9,0x67,0xb8,0x9a,0x6b, +0x6b,0x8f,0xbb,0x87,0x6b,0x75,0x69,0x85,0xbf,0x44,0xc3,0x92, +0x85,0x7a,0xc0,0xa1,0x74,0x8c,0x74,0xb4,0x2a,0xe1,0x57,0x5f, +0xe9,0xf4,0xcf,0x1f,0xa8,0x14,0x49,0x8e,0xa1,0x3e,0x53,0xe9, +0xff,0x8c,0xc2,0x44,0x2d,0xce,0x24,0x93,0x37,0x89,0x80,0xae, +0xd7,0x63,0xf1,0xad,0xd5,0xeb,0x36,0x11,0xea,0xcd,0x45,0x9c, +0x53,0x78,0x4d,0x5b,0xe7,0x6b,0xcb,0xeb,0xf7,0x33,0xf0,0xb5, +0xae,0x0c,0x28,0x0a,0x94,0xf0,0x6e,0x72,0x10,0x0a,0x1f,0x8c, +0xb7,0xf5,0x1a,0xcc,0x8c,0x37,0x6a,0xd3,0x92,0x83,0xeb,0x98, +0x83,0xbe,0x6f,0x36,0x82,0xd5,0xfc,0xa2,0x53,0x51,0x2a,0xc1, +0x51,0x25,0x15,0x35,0x50,0xcc,0xbd,0xd4,0x33,0x41,0x81,0x87, +0xf5,0x0c,0x62,0xe8,0xd9,0x7f,0x37,0x67,0xb6,0x3b,0x2a,0xf0, +0xd1,0x8c,0xe8,0x7c,0x81,0x7f,0xde,0xd1,0xc5,0xe6,0xc3,0xe5, +0x6c,0xc3,0xaf,0x43,0xa0,0x9f,0x9e,0xfe,0xb6,0x01,0x9d,0x4d, +0xf5,0x04,0xcd,0x59,0xb7,0xa2,0x4e,0xff,0x73,0xc5,0xbc,0x2a, +0xaa,0x73,0xd9,0x8d,0xac,0x8a,0xd7,0xd3,0xc5,0xfb,0xdf,0x6e, +0xbe,0xf9,0xb9,0x03,0xee,0x3b,0xf4,0x2a,0x7b,0x0c,0x34,0xd8, +0xbb,0x5b,0x9f,0x4f,0xcd,0x89,0xcf,0xd5,0xe3,0x12,0x50,0xaa, +0xcc,0x89,0x84,0x08,0x06,0x07,0x63,0x94,0x69,0x64,0x63,0xcc, +0xf4,0x81,0xde,0x36,0x95,0xdc,0x55,0xea,0xd3,0x15,0x09,0x46, +0xec,0xa9,0xde,0x2b,0x77,0xa0,0xd7,0x3c,0x61,0x4a,0x0c,0xf3, +0x07,0x22,0x14,0x62,0x7b,0x0e,0x4b,0xb8,0x5f,0x3a,0x3e,0x4c, +0xb8,0xd1,0x24,0x37,0x2f,0x22,0x13,0x41,0x40,0x02,0x38,0x84, +0x20,0x21,0x31,0x12,0x04,0xdd,0x20,0xf7,0x32,0x0b,0x2d,0x35, +0xea,0x26,0x19,0x47,0xa1,0x39,0x21,0xd8,0x1b,0xe5,0x24,0xf5, +0xd1,0x61,0x17,0xe5,0xf8,0x7b,0xb4,0xac,0x3d,0x0a,0x6b,0x8f, +0xc4,0x95,0x32,0xe9,0x7f,0x39,0x63,0x93,0x72,0xff,0xd5,0x0d, +0xda,0x8b,0x74,0xd5,0x12,0x02,0xed,0xea,0x87,0x43,0x2f,0x13, +0x7c,0x23,0xb9,0x66,0xb0,0x1e,0xd3,0xe5,0x87,0xdb,0x54,0x5d, +0x05,0x59,0x2e,0x3e,0x5b,0x7f,0x6b,0xd9,0xe7,0x82,0xc4,0x57, +0x36,0x18,0x70,0x1f,0x7a,0x1a,0x04,0x89,0x3d,0xa8,0x1b,0x86, +0x6e,0x74,0x58,0x65,0x5f,0xb1,0x1b,0x1c,0xf6,0x31,0x00,0xb5, +0x67,0xc6,0x19,0x67,0xc5,0x67,0xce,0x71,0xd1,0xb7,0xa8,0x23, +0x2f,0xcb,0xe5,0x61,0x1d,0x7e,0xc5,0xe4,0xc1,0x61,0xcb,0x97, +0xc1,0xd8,0xce,0x21,0xf8,0xb3,0xe8,0x45,0xe1,0xfb,0x24,0x27, +0x8e,0x6e,0x8a,0xb2,0x6c,0xa7,0x16,0x52,0xd8,0xbf,0xa8,0xae, +0x8c,0xe5,0x89,0xaa,0x9e,0x56,0x38,0x69,0x8b,0x6a,0xe5,0x15, +0x0e,0x7a,0xba,0xee,0xbb,0x71,0x32,0xe0,0x5a,0xb4,0x34,0x22, +0xfd,0x30,0x7e,0xef,0xbb,0xf4,0xa2,0x7d,0x60,0xe5,0x18,0x12, +0x34,0xef,0x88,0xa9,0xb1,0xf4,0x72,0xb8,0x0a,0xf9,0x52,0x68, +0xc7,0x98,0x0f,0x25,0x8b,0x71,0x64,0xc1,0x75,0xaf,0xaf,0xa0, +0x3b,0xca,0xb5,0xc3,0x9e,0x8f,0x85,0x00,0x4a,0x15,0xe1,0x5c, +0x72,0x75,0x8b,0x27,0x96,0xc6,0xfc,0x54,0x99,0x1c,0x35,0x3a, +0x49,0x50,0x04,0x30,0x0e,0x75,0x3e,0x02,0xb3,0xc4,0x9f,0x3c, +0x38,0x35,0x05,0x27,0x29,0xa2,0x87,0xbb,0x7f,0x9f,0x7e,0xfa, +0xd8,0xe8,0x84,0x70,0x87,0x91,0xdc,0x0e,0xa9,0x4b,0xca,0x46, +0xb6,0x72,0x7d,0x51,0xc4,0x21,0xd1,0x53,0x55,0x87,0xa0,0x39, +0xd9,0x50,0xda,0x14,0x58,0xdd,0x98,0x29,0xd0,0x8b,0xef,0x06, +0xd3,0x66,0xc1,0x47,0x05,0xd2,0x66,0x4a,0xfc,0x51,0x62,0x58, +0x76,0xa1,0x79,0x98,0x1e,0xc1,0x3f,0x6b,0xff,0xa0,0x7f,0x0c, +0xc0,0x77,0x56,0x32,0x82,0xd6,0x49,0x53,0x10,0x72,0x14,0xd6, +0x22,0x74,0x52,0xf6,0x20,0x86,0xc2,0x0f,0x45,0x35,0xa8,0x14, +0x37,0xcd,0x71,0xb8,0xe9,0x13,0x0a,0x11,0xd8,0x94,0x6a,0x4d, +0xe3,0x4d,0xf9,0x22,0x94,0x8f,0xff,0x07,0x7b,0xfb,0xf8,0xd4, +0xa3,0x48,0x1e,0x71,0xa8,0x90,0x18,0xfb,0x04,0xfe,0x0d,0xde, +0x8c,0xb9,0x43,0x14,0xf1,0x8c,0xb1,0x9c,0x98,0x17,0x19,0x42, +0xf0,0x31,0x98,0xec,0x31,0x92,0x55,0xe4,0x04,0x33,0x76,0x01, +0x86,0x14,0xe9,0x29,0x1b,0xff,0xd3,0xfa,0x9a,0xbb,0x38,0x09, +0xa5,0xc3,0xa1,0x1b,0x41,0x85,0xd6,0xdc,0x01,0x6e,0x74,0x5f, +0xed,0x08,0x77,0xf9,0xed,0xf8,0x6e,0x47,0xe4,0xa7,0xbc,0x6d, +0xcc,0xfb,0x3d,0xcb,0x6e,0xc4,0x3e,0xcb,0xdb,0xca,0x6b,0xea, +0x22,0x32,0xee,0x05,0x29,0xd6,0xe3,0x59,0x75,0x9d,0xe8,0xea, +0x44,0xdb,0x76,0x79,0x7f,0x79,0xfb,0xeb,0x65,0x83,0x67,0x10, +0xd1,0x90,0x1a,0xe8,0xb1,0x48,0x54,0x6e,0xc6,0x0a,0x8c,0x56, +0x26,0xe0,0x10,0x85,0xec,0xc4,0xe0,0x57,0x2e,0x52,0x82,0x21, +0xa0,0xae,0x95,0x55,0x9b,0xae,0xfc,0xc1,0x3b,0x97,0xd4,0x68, +0x40,0x91,0x1c,0xf5,0xf7,0x37,0x44,0x9f,0x4f,0x66,0x97,0xea, +0x7c,0xca,0xc5,0x39,0xa6,0x3b,0x58,0x4a,0xd1,0x4c,0xa3,0x1c, +0x9a,0xe5,0xc3,0xef,0xb9,0x75,0x79,0x15,0xc7,0xd5,0x5a,0x8e, +0xab,0x1e,0x8e,0xab,0xe7,0x39,0xfe,0xe1,0x2d,0xd5,0xee,0x10, +0x08,0xbc,0x83,0x2e,0x95,0x8b,0x57,0x57,0x8b,0x00,0x34,0x54, +0x77,0xb0,0xb6,0xb6,0x96,0xe2,0x9c,0x2b,0xa6,0xdf,0xa1,0x51, +0xdc,0x33,0xe3,0xea,0x8d,0xa6,0x74,0x7e,0xa2,0x2c,0x32,0xf7, +0xe4,0x22,0x60,0x2e,0x62,0x80,0xf8,0x20,0xee,0x2f,0x86,0x4e, +0x7c,0x80,0x92,0x5c,0xf6,0x97,0x71,0x0c,0x8d,0x67,0x33,0x69, +0xf2,0x0b,0x9a,0x9e,0xcd,0xe3,0xab,0xe6,0x08,0xc2,0x97,0xd4, +0x2b,0xf7,0xcf,0x2e,0x2c,0xc7,0x96,0x20,0xdc,0x41,0x8f,0xba, +0x5a,0x2a,0x82,0xf7,0x33,0x63,0x67,0xba,0x38,0x95,0x4c,0x46, +0x31,0x52,0x29,0xef,0xc6,0x6d,0x51,0xc1,0x21,0xce,0x77,0xdc, +0x2f,0xc5,0x08,0x01,0xb6,0x16,0x28,0x3b,0x2f,0x46,0xc5,0x7f, +0x28,0x83,0x17,0xc3,0x96,0x47,0x3e,0x84,0x67,0xc2,0x64,0x78, +0xc6,0x16,0x27,0x09,0x44,0x37,0xc2,0x24,0x7c,0xb4,0xe5,0x92, +0x1f,0x61,0xb9,0x64,0xb6,0x5a,0x88,0x33,0x16,0x09,0x5a,0xfc, +0x3f,0x82,0x70,0x3d,0x54,0x3f,0x91,0xb7,0xfc,0x19,0x5b,0x90, +0xe2,0x32,0x55,0x21,0x46,0x0e,0xc4,0x9d,0x2c,0x7e,0x19,0x7f, +0x28,0xbc,0x9c,0x13,0x5d,0xdb,0xf2,0x38,0x11,0x2e,0x9f,0x9c, +0x5a,0x67,0xdd,0x80,0x5d,0x56,0x16,0x1f,0xe9,0xa6,0x07,0x80, +0x02,0x84,0x9a,0x32,0x03,0xd8,0x74,0x63,0x67,0x80,0x23,0xc9, +0xa7,0x53,0x6d,0x81,0xe2,0x47,0x51,0xd2,0xc5,0x8f,0x4b,0xcf, +0xf9,0x30,0x5e,0x01,0x8c,0x08,0x07,0x5d,0x67,0xcf,0x20,0xe3, +0x8a,0x2a,0x2c,0xfe,0x7a,0xa4,0xee,0xd2,0x5e,0x31,0x9a,0xec, +0x99,0x74,0x63,0xee,0x84,0xd6,0x1f,0xe5,0x53,0xaf,0xbd,0x33, +0xcc,0x86,0xaf,0x0d,0xfd,0x1a,0x4c,0xc6,0xe1,0xc2,0xd1,0xcb, +0xa5,0xb4,0xd1,0x84,0xac,0x31,0x78,0xe5,0xb6,0x05,0xa8,0x6c, +0x21,0xf7,0x2b,0x28,0xcb,0x8b,0x7a,0x8f,0xbe,0x82,0x16,0xf5, +0x0d,0x7d,0xbd,0xf7,0x5f,0x81,0x2a,0x9e,0x26,0xa0,0x82,0x91, +0x04,0xae,0x74,0x11,0x7a,0xa9,0x1d,0x5a,0x7c,0x25,0x5e,0x1f, +0x4b,0x0d,0x2a,0xe5,0x5f,0x7f,0x57,0xee,0x26,0x79,0x79,0x6b, +0x8b,0x3e,0x77,0xa6,0xf3,0x59,0x4b,0x5f,0xe1,0xe2,0x10,0xaa, +0x28,0xa2,0xd7,0x08,0x07,0xb4,0x2d,0x05,0xc3,0x89,0x9a,0x54, +0xd7,0x66,0xc8,0x01,0x39,0x48,0xfa,0x8d,0xe8,0xf6,0x8e,0x5a, +0xde,0x6c,0x9a,0xcd,0xf1,0xd6,0x96,0x66,0x85,0xbb,0x75,0x45, +0xb8,0x66,0x92,0xba,0xa2,0x98,0x9d,0x74,0x64,0xfc,0x7f,0x00, +0x00,0x00,0xff,0xff,0xc9,0xc8,0xe4,0xfd,0x32,0x22,0x06,0x00, + })) + + if err != nil { + panic("Decompression failed: " + err.Error()) + } + + var b bytes.Buffer + io.Copy(&b, gz) + gz.Close() + + return b.Bytes() +} + +func init() { + go_bindata["/scripts/browser-modules.js"] = scripts_browser_modules_js +} diff --git a/dashboard/resources/scripts-browser-scripts.js.go b/dashboard/resources/scripts-browser-scripts.js.go new file mode 100644 index 00000000000..ab389cd7d00 --- /dev/null +++ b/dashboard/resources/scripts-browser-scripts.js.go @@ -0,0 +1,232 @@ +package resources + +import ( + "bytes" + "compress/gzip" + "io" +) + +// scripts_browser_scripts_js returns raw, uncompressed file data. +func scripts_browser_scripts_js() []byte { + gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ +0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0xac,0x58, +0x0b,0x73,0xdb,0xc6,0xf1,0xff,0x2a,0xd0,0x45,0x56,0x80,0x11, +0x08,0x49,0x7f,0x3f,0xfe,0x53,0x30,0xa8,0xdb,0xd8,0x9a,0x69, +0xa6,0xb6,0xe3,0x89,0x94,0xa4,0x1d,0x8d,0xa6,0x73,0x22,0x8e, +0x24,0x22,0xf0,0x8e,0x05,0x8e,0xa2,0x58,0x1a,0xdf,0xbd,0xbf, +0xbd,0x07,0x08,0x90,0x90,0x6b,0xcf,0xd8,0x6d,0x44,0xdc,0xde, +0xee,0xde,0xbe,0x77,0xef,0x8e,0xa6,0x2b,0x39,0xd1,0x85,0x92, +0xa1,0x88,0xb6,0xc5,0x34,0x64,0x7e,0xcd,0xb2,0x4c,0x6f,0x96, +0x42,0x4d,0x83,0x3b,0xa5,0x74,0xad,0x2b,0xbe,0x8c,0xda,0xaf, +0x90,0xc9,0xd9,0x48,0x17,0x0b,0x31,0xaa,0x44,0xc9,0x75,0xf1, +0x20,0x58,0x2c,0xa2,0xb1,0x28,0x6b,0x11,0x10,0x13,0x75,0xf7, +0x87,0x98,0xe8,0x1d,0x0b,0xf1,0xb8,0x54,0x95,0xae,0xa3,0x85, +0xca,0x57,0xa5,0x48,0xdc,0x32,0x13,0x61,0x87,0xe6,0xf0,0xe0, +0x5c,0x4c,0x0b,0x29,0x4e,0x4e,0xec,0x6f,0xc2,0x17,0x79,0x64, +0x3f,0xc3,0xee,0x59,0x2b,0x69,0x81,0x39,0x3b,0xf2,0x84,0xb5, +0xa8,0x8d,0x32,0x47,0xf8,0x48,0xd4,0x7d,0x18,0x45,0x95,0xd0, +0xab,0x4a,0x8e,0x69,0xbd,0xe0,0xf7,0xe2,0xc3,0xec,0x1a,0xc2, +0xff,0xe2,0x64,0xcf,0x44,0x43,0xdc,0x86,0x38,0xad,0x0b,0x99, +0xab,0xf5,0x6b,0xfb,0x93,0xc8,0x3d,0xb2,0x30,0x4a,0x67,0xa5, +0xba,0xe3,0xe5,0xc0,0x4e,0x13,0xb6,0x96,0x8d,0xb6,0xf6,0xf8, +0x60,0x67,0xeb,0x58,0xc7,0x32,0xda,0xfa,0x75,0x50,0x84,0x32, +0x76,0x22,0xeb,0x1b,0x79,0x6b,0xbf,0x84,0xf9,0x7a,0xe0,0x55, +0xa0,0xb2,0x01,0xeb,0x54,0xe2,0xdf,0xab,0xa2,0x82,0x79,0xdc, +0xc7,0xd8,0x28,0x7c,0x72,0xa2,0x9c,0xb2,0x81,0x02,0xd3,0xa3, +0xf3,0x88,0xe0,0x95,0x87,0x55,0x0e,0xa6,0xe7,0x95,0x5a,0x07, +0x52,0xac,0x83,0xcb,0xaa,0x52,0x55,0xc8,0xde,0x70,0x29,0x95, +0x0e,0xa0,0x7e,0x1e,0x58,0x2f,0x05,0xdf,0xb3,0x53,0x79,0xca, +0xbe,0x67,0x51,0x43,0x32,0xac,0x32,0x92,0x2c,0xdb,0x3a,0xdf, +0xa5,0xdb,0xa6,0x19,0x93,0x84,0x37,0xe7,0xb7,0x3b,0x4d,0xb5, +0x95,0xb7,0xca,0xcc,0xce,0xc5,0xed,0x8d,0xbe,0x1d,0xbb,0x93, +0x8b,0xb0,0x7a,0x5d,0xa5,0x3a,0x6a,0xe2,0x55,0xbc,0xf2,0x11, +0x10,0x35,0x6e,0x97,0x78,0x7b,0x60,0x33,0x85,0x40,0x96,0xcd, +0x17,0xa8,0x1d,0xd7,0xd9,0xf9,0xb8,0xfe,0x41,0x26,0xa5,0x90, +0x33,0x3d,0x1f,0xd7,0xa7,0xa7,0x11,0xcc,0x79,0x53,0xdf,0x46, +0xed,0xd1,0x4d,0xb8,0xbd,0x48,0x6f,0x5a,0x29,0x3d,0xa5,0xd5, +0x33,0xf6,0xb2,0x6c,0xd9,0x0a,0x21,0x85,0xf8,0x2e,0x10,0xba, +0xe3,0xd6,0x37,0x39,0x50,0x27,0xe4,0xd5,0xf0,0x98,0x22,0x5e, +0xad,0x34,0x08,0x17,0x42,0x6a,0xef,0x56,0xfc,0x58,0xa2,0x94, +0xfd,0xf5,0x0d,0x8b,0xeb,0x89,0x5a,0x8a,0x74,0x9b,0x73,0x2d, +0x08,0x3f,0x65,0x7f,0x61,0x4d,0x5c,0x16,0xf2,0x3e,0x6d,0x05, +0x30,0x28,0xb1,0x28,0x05,0xb1,0x89,0xb9,0xd6,0x55,0x6d,0x0d, +0xe7,0x0e,0x18,0x1b,0x84,0xe4,0x78,0xcd,0xf5,0x64,0x1e,0x32, +0xcf,0x8a,0xc5,0x2d,0x07,0x02,0x5d,0xe1,0x50,0x39,0xeb,0x44, +0xd1,0x6a,0x49,0x60,0x0a,0x44,0x84,0x9c,0xe3,0x9e,0x68,0xf1, +0xa8,0xc3,0xbc,0x98,0x4e,0x2d,0xba,0xa1,0x8c,0xb5,0x42,0x4e, +0x44,0x4d,0x47,0xc7,0x76,0x9f,0xc7,0x77,0x6d,0xbc,0xbe,0xe7, +0x7a,0x9e,0xf0,0xbb,0x3a,0xe4,0xc9,0xa4,0x54,0xc8,0xbb,0x28, +0xa9,0x35,0xaf,0xf4,0xcf,0x53,0x12,0x6a,0xc3,0xa2,0x84,0x08, +0xc3,0xbb,0x98,0x56,0x35,0xa3,0xc8,0x8a,0x7e,0xb8,0x78,0xcd, +0x93,0x69,0xa5,0x16,0x00,0xaf,0x0b,0x3d,0x87,0x3a,0x57,0xab, +0xe9,0xb4,0x78,0x8c,0x52,0x70,0xe1,0xf0,0x52,0xce,0xab,0xf0, +0xae,0x73,0xb8,0x15,0xfb,0x1d,0xfe,0xab,0x20,0x77,0x57,0x89, +0xd8,0xd9,0x23,0xf3,0x96,0xef,0xe6,0x54,0x8f,0xac,0x89,0x25, +0xf4,0xfc,0xd5,0x80,0x7e,0xc2,0x6e,0x87,0x7d,0x7f,0xc3,0x98, +0x39,0x17,0xa5,0xe6,0x59,0xab,0x9d,0x75,0x67,0xe8,0xb4,0x21, +0xcc,0xa8,0x8d,0x9d,0x17,0x2f,0xc5,0xf3,0x3f,0x1b,0xfc,0xd7, +0xf4,0x39,0x32,0x9f,0xe9,0x9f,0xc4,0x0b,0x07,0xc4,0x97,0x83, +0xfd,0xdf,0xff,0x8b,0x97,0x0e,0xf8,0x0a,0xc0,0xd0,0x7c,0x9e, +0x3e,0x17,0x2f,0xa2,0x67,0x58,0xa7,0xcf,0x5f,0xbd,0xf2,0xa8, +0xcf,0x9e,0xbf,0x12,0x2f,0x1b,0xaf,0x13,0x8c,0x22,0x27,0xa2, +0x0c,0xdd,0x32,0x1a,0x1b,0x09,0x21,0x44,0xe6,0xe4,0xea,0xf8, +0x9a,0x72,0xd9,0xc8,0x67,0xa3,0x45,0x65,0x87,0x35,0xc6,0x11, +0x99,0xa0,0x4a,0xb4,0x82,0x65,0x7a,0x5e,0xc8,0x58,0x6f,0xc9, +0x0a,0x19,0x18,0xd4,0xb1,0x23,0x28,0x74,0x29,0x3e,0x7d,0xf2, +0xc1,0x43,0xc0,0x90,0x19,0x20,0x8b,0xe9,0xe0,0x04,0xb9,0xb9, +0xe0,0x3a,0x64,0xef,0xf0,0x8f,0x45,0x91,0x0f,0xe2,0xe4,0x0e, +0x85,0x23,0x64,0xc7,0x39,0xa5,0x82,0xda,0x74,0x22,0x35,0xda, +0x3e,0xa5,0x68,0x13,0xc5,0x7d,0x1f,0x36,0xf8,0x7f,0x03,0x0d, +0x7d,0x7b,0xd8,0xeb,0x16,0x2d,0x4b,0x0b,0x8f,0xb6,0x6e,0x7f, +0xa2,0x24,0xa2,0x12,0x3a,0x33,0xdd,0xa9,0xc1,0x6f,0x94,0x9c, +0x16,0x33,0x16,0x6f,0x7d,0xcc,0xa5,0x5b,0x21,0xd3,0x6d,0xc9, +0x6b,0xfd,0x96,0x6f,0x52,0x76,0xf3,0x4f,0x88,0x2a,0x2a,0x04, +0xee,0x6d,0x1c,0xbc,0xbb,0x46,0xde,0xf2,0x85,0xb0,0x3b,0xd7, +0x6a,0x07,0xa5,0xe8,0xf1,0xd0,0x85,0x42,0xad,0x5c,0xbb,0x0d, +0x62,0xf4,0xbb,0x10,0xf7,0x29,0xcb,0xf1,0x6f,0x87,0x6c,0x61, +0x1f,0xf0,0x15,0xec,0x36,0x88,0xf7,0x25,0x5a,0x4c,0x0a,0xbb, +0x31,0x28,0x49,0xa1,0xe6,0xab,0x0a,0xdb,0xb5,0xd0,0x1b,0xe6, +0x6d,0xc5,0x62,0x66,0x1d,0x09,0xb3,0x7b,0xcc,0xdb,0x28,0xa9, +0x56,0x32,0xec,0x98,0xc1,0x94,0x8f,0x43,0xa5,0xa3,0x2d,0x97, +0xb3,0x55,0xc9,0x2b,0xf2,0xd6,0x25,0x47,0x09,0x39,0xc4,0x69, +0x53,0x71,0xe7,0x29,0x34,0x76,0x59,0x13,0x8e,0x92,0xd0,0x8e, +0x2a,0x8b,0x3d,0x21,0xa1,0x45,0x48,0x7f,0x3a,0xc6,0xec,0x20, +0xc3,0x6d,0xf4,0x3f,0x5f,0x47,0x77,0xaa,0x65,0xed,0x57,0xec, +0x04,0xb2,0x71,0x0b,0xbf,0x65,0x5e,0x42,0xeb,0xc3,0xbe,0xeb, +0x60,0x08,0x14,0x6f,0x7b,0xf8,0xc9,0x09,0xb9,0xbb,0xe3,0x63, +0x6f,0x16,0x57,0x83,0xe3,0xae,0x8c,0x4c,0x48,0x38,0xbc,0xa1, +0xb0,0xb4,0xb2,0x10,0x2d,0x82,0x0a,0xb0,0x5b,0xfa,0x13,0xa3, +0x25,0x51,0xd8,0xed,0x1f,0x2e,0xf4,0x24,0x37,0x87,0x26,0x53, +0x3e,0xd1,0xaa,0xda,0x84,0xec,0x12,0xa0,0xdf,0x2e,0x8c,0x4b, +0xe6,0x5a,0x2f,0x3b,0x01,0x6d,0xd6,0x9d,0xaa,0x3b,0x29,0x05, +0x97,0xab,0xe5,0x47,0x54,0x93,0x70,0x89,0x3f,0x56,0xc7,0x25, +0xa7,0x90,0xa5,0x75,0x52,0x2f,0xcb,0x02,0x82,0x9f,0xb1,0xb6, +0xa8,0x9c,0x67,0x59,0x66,0x10,0x5c,0xd3,0x7a,0xcd,0x58,0x1a, +0x7a,0x0a,0x02,0x4f,0x8b,0x92,0x52,0xa2,0x3d,0xf3,0xc1,0xa7, +0x37,0xc3,0x68,0x92,0x3d,0x40,0x05,0x8b,0xf7,0x87,0x2a,0xa4, +0xe1,0xdc,0xab,0x77,0xeb,0xbf,0x8b,0x4d,0x78,0x2f,0x36,0x1f, +0x10,0x75,0x56,0x9a,0x5a,0x94,0xd3,0x6c,0xdb,0xf8,0xf3,0x69, +0x99,0x48,0xec,0x66,0x5d,0xe1,0x3d,0x45,0x6c,0xb6,0x67,0x42, +0x7f,0xe4,0x15,0x4c,0xdb,0xad,0x31,0x3b,0xcd,0x5a,0x16,0x5f, +0xa2,0x9e,0x13,0x89,0xb1,0xc8,0xa9,0x99,0x2c,0xd5,0x12,0xe5, +0xdd,0xc1,0xf7,0x75,0x41,0x2c,0x19,0xf6,0x64,0xbd,0xc3,0x0a, +0x07,0x94,0xd3,0x3d,0xb1,0x3f,0x56,0x98,0xdb,0x1e,0x4f,0x5b, +0x99,0x3c,0x03,0xe8,0x30,0x50,0x21,0x8d,0x03,0x69,0x2f,0x6c, +0x4f,0x09,0xdb,0x33,0xeb,0x2e,0x09,0x58,0xff,0xc6,0xcb,0x95, +0xe8,0x93,0x86,0xdb,0x55,0x55,0xa6,0x1d,0x5a,0xaa,0x8c,0x3c, +0x3d,0xc6,0xaa,0xe2,0x8b,0x10,0x46,0x02,0x49,0xea,0x69,0xe1, +0xab,0x85,0x40,0xc5,0xcd,0x53,0xf6,0xf1,0xe7,0x2b,0xd4,0x82, +0xb9,0xe0,0xb9,0xa8,0x30,0x40,0x31,0x64,0xa2,0x86,0x81,0x47, +0xd7,0x98,0x6c,0x58,0xca,0xf8,0x12,0x96,0x9c,0x98,0x94,0x3a, +0x7b,0x1c,0xad,0xd7,0xeb,0x11,0x15,0xdb,0x11,0x0e,0x13,0x72, +0xa2,0x72,0x8c,0xa5,0x54,0x21,0xad,0x98,0xe8,0x23,0x42,0x0b, +0x18,0xef,0x29,0xfd,0x0e,0x85,0xf4,0x52,0xbc,0xbd,0x7c,0x77, +0x79,0x7d,0xf9,0x6d,0xe4,0xe8,0x45,0xdd,0x95,0x46,0x63,0x40, +0x96,0xea,0xaf,0x8c,0xbb,0x96,0xe4,0x6b,0xbc,0x4e,0x44,0xf5, +0xb7,0xf3,0xbb,0x19,0x71,0xdb,0x48,0xca,0xd8,0xd9,0xc3,0xc5, +0x19,0x96,0xf5,0x19,0x6a,0xf7,0xee,0x24,0x0b,0x37,0x80,0x33, +0xe6,0x95,0xb2,0xae,0x70,0xdc,0x71,0x81,0xe0,0xab,0x12,0xd1, +0xec,0xac,0x8b,0xc2,0xb5,0x58,0x28,0x79,0xc3,0xfe,0x31,0xfa, +0x05,0x53,0x27,0xb5,0x9c,0x7c,0xf4,0x3b,0x3a,0x30,0xbb,0x8d, +0xb7,0x10,0x85,0x6c,0x96,0x3a,0xdb,0xc5,0x58,0xc3,0xa3,0xa9, +0xcd,0x8a,0xa6,0xb9,0x1d,0x2e,0x54,0x3f,0xa2,0x07,0xd5,0xa2, +0xa2,0xd2,0x24,0x67,0xbf,0xa0,0x57,0xa0,0x5e,0xba,0x0a,0xd6, +0xaf,0xa2,0x28,0x67,0xbb,0xb2,0xd9,0xea,0x06,0x2c,0xaf,0x1c, +0x33,0x08,0x68,0x06,0x21,0xaa,0x5c,0x45,0x9c,0x3e,0x56,0xea, +0xa1,0xc8,0x89,0x79,0x97,0x60,0x57,0xfa,0x7a,0x48,0x71,0x8b, +0x12,0x6d,0x4b,0x85,0xce,0x70,0x85,0xd2,0xc9,0x67,0x82,0x4c, +0xfc,0x93,0x16,0x0b,0x2b,0xee,0xbf,0xc8,0xce,0x2c,0x8a,0xfb, +0xb4,0xc9,0x7a,0x2e,0x4c,0xb6,0xc7,0xf0,0x8e,0x6d,0x13,0xd7, +0x2a,0x6d,0x19,0xa2,0x41,0x2a,0x3d,0x17,0xd5,0xba,0xa8,0x45, +0xb8,0x05,0xaf,0x25,0x74,0x12,0xbf,0x22,0xa2,0xd9,0x43,0x21, +0xd6,0xf5,0xd9,0x9d,0x35,0x42,0x32,0xd7,0x8b,0x92,0xc5,0xd0, +0x02,0x53,0x47,0x59,0x8a,0x2a,0x65,0xef,0x79,0x21,0xdf,0xe8, +0xaa,0x64,0x88,0x51,0x6b,0x01,0xb7,0x15,0xee,0xb6,0xa8,0xaa, +0x9b,0x11,0x1b,0x7a,0x1e,0x93,0xe8,0xe6,0x8a,0x11,0xb7,0x45, +0x7f,0x58,0x79,0x3b,0xb5,0xb7,0xf8,0xb1,0xc5,0xee,0x5a,0xc1, +0xa2,0x24,0x35,0x47,0xef,0x33,0xca,0x8f,0xe8,0x73,0x34,0x87, +0xc6,0x2c,0x76,0x9b,0xcb,0x4a,0x90,0x0a,0x6e,0xdf,0xad,0xfa, +0x28,0x42,0xf2,0xbb,0x52,0xfc,0xc8,0x27,0xf7,0xd9,0xd1,0xb9, +0x07,0xae,0xab,0x42,0x63,0x04,0xfc,0x00,0xd2,0xa3,0x8b,0x16, +0x13,0x3c,0x28,0x1d,0xb2,0x56,0x2a,0x9f,0xed,0xc7,0xfd,0x2b, +0x84,0x47,0xec,0x0d,0x66,0x6d,0xf2,0xfa,0x5d,0x2a,0xc6,0x6d, +0xba,0x2c,0x1d,0x60,0xef,0xa8,0x4e,0xa7,0xf1,0x18,0x9f,0xe9, +0x7a,0x3b,0xe3,0xf8,0x74,0xb9,0x18,0xe8,0x7d,0x66,0x8d,0xde, +0xdc,0xf4,0xc2,0xa8,0xee,0x86,0x91,0x15,0x7e,0x5f,0x96,0x61, +0x8b,0x31,0xdc,0x16,0xb3,0x3d,0x54,0xcc,0x11,0x03,0xb8,0x17, +0x2d,0x07,0x08,0x9a,0x59,0x87,0x26,0x36,0x0d,0xc3,0xae,0x51, +0xf6,0xcf,0xa5,0x21,0x62,0xcf,0xc2,0x60,0xd0,0x9f,0x7a,0xf7, +0xdd,0x86,0xc6,0x7d,0x74,0x7e,0x72,0xb2,0x3b,0xcf,0x94,0x22, +0xdc,0xa8,0x56,0x93,0x89,0xa8,0xeb,0xb0,0x7b,0xb7,0xe3,0xa0, +0x0f,0xd9,0x77,0x26,0x44,0x6c,0xa8,0x8f,0x04,0xdd,0xd3,0x91, +0xb1,0x14,0x29,0xae,0xe5,0x78,0x0b,0x7a,0xf1,0xca,0xa2,0xd6, +0x19,0x6d,0x7c,0x41,0xac,0xa1,0x13,0xfb,0x68,0x85,0x78,0xa5, +0x30,0xbd,0xca,0x10,0x27,0xa6,0x7f,0x7d,0x9e,0x05,0x7e,0x04, +0x2f,0x59,0xdc,0x57,0xc6,0xce,0x0b,0x50,0xe9,0xf3,0x8a,0xed, +0x09,0x4b,0xe3,0x5a,0xff,0xb0,0xf7,0xa0,0x42,0x00,0x64,0xec, +0x83,0xa2,0x9a,0x4c,0x6d,0x03,0xf5,0x41,0xe4,0x09,0x72,0x3a, +0x31,0x76,0x78,0x82,0xe7,0x3e,0x3d,0x5c,0x17,0xe4,0x4a,0xd4, +0x01,0x3d,0x6d,0x88,0x47,0x9c,0x98,0xb4,0x32,0xd7,0x73,0xb5, +0xb6,0xa5,0xd4,0xbe,0x80,0x18,0xd5,0x17,0x96,0xd2,0x0c,0xb4, +0x1e,0xf1,0x8e,0x22,0xe5,0xd0,0xb1,0xbb,0xd4,0x1b,0xb6,0x41, +0x3f,0x0f,0xeb,0x8d,0x9c,0xbc,0x73,0x29,0x1a,0xee,0xeb,0xfb, +0xb9,0x5a,0xd0,0x4b,0xfb,0x66,0x88,0x5d,0x4f,0xb6,0x7e,0x19, +0x38,0x48,0xdd,0xae,0xf2,0x57,0x54,0xa5,0x0e,0xf5,0xda,0x2f, +0x5e,0xce,0xd5,0x3b,0x52,0x00,0xdf,0xc2,0x56,0x03,0xa4,0x64, +0x02,0x24,0xed,0x40,0x60,0x7d,0xfa,0xc4,0x02,0x36,0x10,0x11, +0xdf,0xac,0x64,0x92,0x93,0x76,0x76,0xed,0x1b,0xed,0xf3,0x31, +0xe3,0x4d,0x31,0x1c,0x05,0x9e,0xe3,0xe0,0xbc,0xd5,0x51,0xbb, +0xdd,0x1f,0x8a,0xfb,0x6f,0xac,0xe5,0xa1,0x42,0x3d,0x65,0xba, +0x41,0xcd,0xcc,0x4f,0x1a,0xbc,0x51,0xab,0x32,0x37,0x59,0xe0, +0xa6,0x15,0x34,0x57,0x4a,0x2d,0xd6,0xd5,0x91,0xe7,0xf9,0x97, +0xc7,0xc3,0x57,0xd5,0x87,0x6e,0x89,0x61,0x80,0x86,0x2c,0xe9, +0x94,0xb6,0x6a,0x64,0x47,0x84,0x64,0x6a,0xde,0x10,0x0a,0xb9, +0xc4,0xfd,0x17,0x2b,0x35,0x59,0xd5,0xc3,0x3e,0x3d,0x6f,0x9e, +0xc8,0xe2,0xce,0x1b,0x81,0xf3,0xe1,0x93,0x75,0xd4,0x1e,0x66, +0xe5,0xc0,0x6d,0x44,0x3d,0x40,0x8e,0x89,0x1d,0x84,0xb1,0x6b, +0x5e,0xcd,0xbc,0xf5,0xd8,0xa9,0xe7,0x16,0x3f,0xc5,0x0d,0xb6, +0x7b,0x53,0x72,0x38,0x9d,0xf5,0x18,0x56,0xc5,0x6c,0x4e,0xec, +0x48,0xcc,0xf0,0x30,0xfd,0xfe,0xa7,0xcc,0xc6,0xe2,0xdf,0x48, +0xe2,0x1e,0xaf,0x27,0xe4,0x2d,0xc5,0x74,0x40,0x5c,0x94,0xb5, +0xbf,0x09,0xd2,0x64,0x68,0xa8,0x0e,0xed,0xdb,0x38,0x1a,0x93, +0x41,0xe9,0x90,0xb9,0xce,0xb8,0xcf,0xa4,0xd7,0x24,0xdb,0x48, +0x50,0x39,0x3d,0x27,0x4e,0x48,0x22,0xcb,0x88,0x1d,0x9c,0x1e, +0x46,0xa3,0x17,0x2f,0xa9,0x30,0xbb,0xd7,0x78,0x25,0x2b,0x51, +0x17,0xff,0x19,0x2a,0x62,0xc7,0x74,0x87,0xd9,0xd0,0x93,0xd2, +0x6d,0xef,0xa9,0x45,0xce,0x2e,0x61,0xaf,0xaa,0xd7,0xa8,0xf7, +0xdf,0xe6,0x07,0x1f,0x67,0xfb,0xcf,0x5c,0xc8,0x1c,0x08,0x20, +0x29,0x83,0x10,0xf1,0x75,0xdd,0xe1,0x87,0xc0,0xa7,0x97,0xe1, +0x8b,0xe7,0x18,0x41,0xcc,0x37,0x26,0xdd,0x62,0x42,0xe3,0x47, +0x4f,0xb2,0xce,0xf9,0x0e,0x2e,0xd0,0x76,0xdd,0xab,0x9d,0x93, +0x92,0x54,0xb5,0x2c,0x28,0xad,0xf0,0xfb,0xd6,0x5e,0x2e,0xe8, +0xd2,0x12,0xed,0xbd,0x21,0xcd,0x61,0xa0,0xd2,0x9a,0xed,0x40, +0xb5,0xee,0xfb,0x34,0xfb,0x92,0x97,0x68,0xf6,0x1d,0x3b,0xdd, +0x6b,0x72,0x59,0x66,0x45,0x9b,0x63,0x98,0x3b,0x39,0xf1,0xd6, +0x58,0xee,0x9a,0x9d,0xfb,0xd8,0x8b,0x2b,0xdf,0xb9,0x19,0xbd, +0xec,0x0d,0xbd,0xd5,0xf8,0xb7,0x2f,0x7a,0xef,0x4d,0x31,0x90, +0x42,0x5e,0x91,0xb2,0xcb,0xc7,0x25,0x54,0xab,0x83,0x42,0x06, +0xcf,0x60,0xdc,0x25,0xaf,0xb5,0x07,0xe6,0x80,0x04,0x7c,0xa6, +0x70,0x2f,0x4b,0x59,0x2d,0x10,0xff,0x39,0x30,0x16,0xb8,0xb5, +0x06,0x8b,0x42,0x9a,0xcb,0xd0,0x02,0xab,0x67,0xb9,0x5b,0x62, +0x73,0x8e,0x4d,0x19,0xcc,0xd5,0x0a,0x7e,0x9f,0xcf,0xcd,0x1e, +0x2d,0xb0,0x93,0x13,0x19,0x3d,0x64,0xc7,0x79,0x6e,0xe0,0xf6, +0x19,0xfb,0xbd,0xe1,0x86,0xcc,0xc2,0xc4,0xf9,0xfe,0xbd,0x65, +0x46,0x2b,0x6c,0x6d,0x68,0x6b,0x23,0x38,0x58,0x6d,0x36,0x66, +0x87,0x16,0x35,0xdd,0x88,0xc7,0xff,0x0d,0x00,0x00,0xff,0xff, +0xb0,0x30,0xf2,0xd3,0x4b,0x1b,0x00,0x00, + })) + + if err != nil { + panic("Decompression failed: " + err.Error()) + } + + var b bytes.Buffer + io.Copy(&b, gz) + gz.Close() + + return b.Bytes() +} + +func init() { + go_bindata["/scripts/browser-scripts.js"] = scripts_browser_scripts_js +} diff --git a/dashboard/resources/scripts-stats-modules.js.go b/dashboard/resources/scripts-stats-modules.js.go new file mode 100644 index 00000000000..d544eff6c5f --- /dev/null +++ b/dashboard/resources/scripts-stats-modules.js.go @@ -0,0 +1,13760 @@ +package resources + +import ( + "bytes" + "compress/gzip" + "io" +) + +// scripts_stats_modules_js returns raw, uncompressed file data. +func scripts_stats_modules_js() []byte { + gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ +0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0xdc,0xbd, +0xeb,0x76,0x1b,0x47,0xd2,0x20,0xf8,0x7f,0x9e,0x82,0xa8,0xf6, +0xc0,0x55,0x42,0x12,0x22,0x64,0xbb,0xe7,0xeb,0x82,0x8b,0x18, +0x59,0x92,0x3f,0x6b,0xc7,0xb7,0xcf,0xd2,0xd7,0x97,0x05,0x21, +0x1d,0x90,0x28,0x8a,0x25,0x81,0x00,0x8c,0x02,0x45,0xc9,0x04, +0xfa,0x59,0xf6,0x3d,0xf6,0xc7,0x9e,0xb3,0x2f,0xb4,0xaf,0xb0, +0x71,0xcd,0x5b,0x15,0x20,0xc9,0xee,0x99,0xb3,0x67,0xbb,0x8f, +0x45,0x54,0x5e,0x23,0x23,0x23,0x23,0x23,0x23,0x23,0x22,0x3b, +0x97,0x37,0x8b,0x8b,0x4d,0xb5,0x5c,0xa4,0xb7,0xd5,0x62,0xb6, +0xbc,0x35,0x37,0x8b,0x59,0x79,0x59,0x2d,0xca,0x59,0x76,0xa7, +0x59,0x47,0x55,0xfd,0x70,0xbd,0x9e,0xbe,0x9f,0x57,0x6f,0xca, +0x74,0x79,0xfe,0x3a,0xbb,0x7b,0x3b,0x5d,0x1f,0xcd,0xcb,0xc5, +0xab,0xcd,0x55,0x01,0xdf,0x7d,0xfe,0x69,0x36,0xef,0x57,0x65, +0xf1,0xfa,0x3f,0x6e,0xca,0xf5,0xfb,0x3e,0xfe,0xa6,0xb2,0xc3, +0x75,0xb9,0xb9,0x59,0x2f,0x8e,0x24,0xbd,0xaa,0xff,0x46,0xfd, +0x50,0xde,0xa8,0x33,0xc8,0x07,0x45,0x41,0x6d,0x2c,0x96,0xb3, +0xf2,0x39,0x54,0xea,0x76,0xb9,0xb5,0x51,0xe7,0x24,0x4f,0xa6, +0xd8,0x6d,0x02,0x25,0xb0,0xb9,0xed,0x36,0x51,0x88,0x92,0x0e, +0x27,0x75,0xbb,0xe9,0x09,0xe4,0x72,0x0d,0xc8,0x5f,0xdc,0x5c, +0x9f,0x97,0xeb,0x84,0x33,0x97,0x97,0x02,0xa3,0x36,0x79,0x7a, +0xa2,0xbf,0x8e,0x07,0x47,0xd5,0xe2,0x08,0x41,0xd8,0xd9,0x41, +0x5e,0xac,0xcb,0xe9,0xa6,0xfc,0x69,0x85,0x1f,0x75,0xba,0xe4, +0xbf,0x3c,0x54,0x28,0x58,0x5e,0x6c,0x0a,0x49,0x7b,0x34,0xbd, +0xb8,0x2a,0xc7,0xf2,0x31,0x29,0xee,0x76,0xd1,0x10,0x4b,0xc8, +0xd7,0xfa,0xfd,0xeb,0xe9,0x06,0xbe,0x2e,0x96,0xeb,0xf2,0xe5, +0x7a,0xb1,0xdc,0xdc,0x5e,0x55,0x9b,0x32,0xdb,0x6e,0xc7,0x13, +0x63,0x11,0xff,0xd2,0x5c,0xce,0xa7,0xaf,0xb2,0x3b,0xee,0x65, +0x8c,0x1f,0x93,0xa2,0x73,0xb2,0xcb,0x0c,0xa7,0x38,0x10,0xab, +0xc5,0xa6,0x5c,0x2f,0xa6,0xf3,0xc7,0xd3,0xcd,0x34,0x2d,0xe7, +0xe5,0xb5,0x59,0x4c,0xaf,0x4b,0x33,0x83,0x4f,0xb3,0x7a,0xbb, +0xc9,0xee,0xaa,0xcb,0x54,0x80,0x98,0x5e,0x5c,0x94,0xab,0x8d, +0x2d,0x98,0xf1,0x40,0x36,0x57,0x15,0x83,0x6f,0x00,0x64,0xa3, +0xcd,0xfd,0x8f,0xf2,0xbd,0x4e,0x5b,0xf9,0x6e,0x35,0x85,0xe9, +0x31,0xaf,0xca,0xcd,0x37,0xef,0x7f,0x84,0xc6,0x8b,0xa4,0xde, +0xac,0xab,0xc5,0x2b,0x87,0x53,0xea,0xb2,0xaa,0x7f,0x84,0xf9, +0x2a,0xb0,0x69,0x3b,0x73,0xe6,0x02,0x5b,0x2e,0x38,0x6b,0x24, +0x0d,0x52,0x5a,0x4e,0xb0,0x56,0x33,0xcd,0xc3,0xcf,0xb1,0xd7, +0xfb,0x24,0x6f,0xa4,0x74,0xbb,0xde,0xd7,0x10,0xc6,0x55,0xcd, +0xba,0x5d,0x6a,0x6c,0x5c,0xcd,0x20,0x37,0x85,0xf1,0x6e,0xb7, +0x36,0xa1,0x8f,0x38,0x00,0xbc,0x76,0x2c,0xe0,0xdb,0x2d,0x26, +0x01,0x9d,0x38,0x82,0x96,0x69,0xaa,0x66,0xdb,0x6d,0xba,0x0f, +0x92,0x02,0xa0,0xa4,0xd9,0x9a,0x41,0xd6,0xa6,0x9c,0x3d,0x9d, +0xd5,0xfd,0xd5,0x72,0x95,0x42,0xdb,0x32,0xa2,0x57,0x37,0xd5, +0xac,0xd7,0xcb,0x71,0x34,0xae,0x5e,0x66,0x2c,0x28,0xd0,0xb8, +0xfd,0x0d,0xc4,0x21,0xa8,0xf2,0x53,0xfb,0x9b,0xe5,0xff,0xf6, +0xec,0xa7,0x1f,0x15,0xe7,0x8b,0xe5,0x72,0x95,0x65,0x26,0x4d, +0x78,0xba,0x43,0x4c,0xfb,0x44,0x1f,0x64,0x64,0x8c,0x83,0x91, +0xeb,0xcb,0x4e,0xe1,0xa6,0x5c,0xcc,0x5c,0x6f,0x44,0x23,0x59, +0x1e,0xa2,0x6a,0x5f,0x61,0xca,0xe4,0x1a,0x99,0xb1,0xd4,0x52, +0xb8,0xc6,0x08,0xed,0xa9,0xcd,0xa1,0xf2,0x8d,0x04,0x18,0xb6, +0x5f,0x3b,0xcc,0xcc,0x4c,0x3c,0x31,0x30,0x12,0x5b,0x64,0x6c, +0xe9,0xe6,0xba,0x9c,0x3f,0x9a,0xd6,0x65,0x4a,0xc0,0x4c,0x0a, +0xae,0x6a,0xa7,0x77,0x94,0xc2,0x6c,0xba,0x96,0xc7,0x58,0x0a, +0x86,0x7a,0x33,0x9f,0x17,0x05,0xe4,0x40,0x93,0x61,0xfe,0x9e, +0x66,0xb3,0x2c,0x0f,0xca,0xe1,0xba,0xd8,0x35,0x17,0xdc,0x2f, +0xe5,0xf5,0xf2,0x6d,0x19,0x2d,0xbb,0x8f,0x5a,0x71,0x95,0x99, +0x3b,0x4c,0xfc,0x2b,0x96,0x4d,0xb8,0x50,0x27,0x79,0xf8,0x8d, +0x2b,0xc5,0x4e,0x16,0x41,0x87,0xa0,0xfa,0x08,0x2e,0x02,0xa2, +0x89,0xc8,0x02,0x80,0xb6,0x7c,0x9a,0x58,0x3e,0xa3,0x69,0x84, +0xff,0x16,0xf8,0x4f,0xff,0x62,0xb9,0xb8,0x98,0x6e,0x74,0xcc, +0xd7,0xd3,0x15,0x95,0x30,0x31,0x7a,0x01,0xaf,0x98,0x8e,0x5c, +0xd6,0xf6,0xcc,0xad,0xf0,0x4c,0xe5,0x54,0xad,0x68,0x9f,0x15, +0x63,0xbb,0x0b,0xeb,0x4b,0x55,0x82,0xa3,0x5e,0xcd,0xab,0x4d, +0x9a,0x1c,0x25,0x59,0x36,0xbc,0x5c,0xae,0xd3,0xaa,0x38,0x31, +0x73,0x06,0x91,0x59,0xfc,0x70,0x7e,0x5a,0x0d,0xab,0x5e,0x2f, +0xe3,0x85,0x7c,0x14,0x92,0xca,0xb8,0x9a,0x4c,0x10,0x57,0x1d, +0x5a,0x43,0x55,0xfd,0xe4,0x7a,0xb5,0x79,0x8f,0x73,0xf7,0x13, +0xad,0xc1,0xdc,0x62,0x81,0x32,0x38,0x31,0x73,0x48,0xcc,0x84, +0x97,0xec,0x98,0x0d,0xa5,0xd2,0x47,0xb4,0x8c,0x1a,0xcd,0x7a, +0x53,0x93,0xe1,0xfa,0x8d,0xe6,0x7c,0x5e,0x4e,0x17,0x44,0x3f, +0x63,0x9c,0xe9,0x89,0xe9,0x9c,0x64,0x0a,0x48,0x7d,0xb3,0x5a, +0x2d,0xd7,0x9b,0x3e,0x77,0xf4,0x84,0x27,0x5b,0xf8,0x5f,0x87, +0x97,0x67,0x9f,0x37,0xf0,0x51,0x0c,0x8b,0x9b,0xe2,0x02,0x97, +0x47,0xb6,0xf3,0xe8,0x1b,0xc1,0x7c,0xb8,0xd9,0xac,0x99,0xaa, +0xdf,0x94,0xef,0x69,0x6d,0x12,0xdd,0xd0,0x3a,0x0e,0x16,0x29, +0xee,0xd4,0x01,0xe5,0x32,0x85,0xd3,0x64,0x25,0x58,0xfc,0x38, +0xe9,0x41,0x13,0xfd,0x75,0xb9,0x9a,0x4f,0x2f,0xca,0x74,0x7d, +0x7d,0x33,0xdf,0x54,0x8f,0xa7,0xf5,0x95,0x49,0x8e,0x3f,0x1b, +0x24,0x19,0xf0,0xbd,0xef,0x97,0xb7,0xe5,0x9a,0x66,0x3a,0x1b, +0x6a,0x27,0xd4,0x26,0x2c,0x6c,0x04,0xa4,0x3a,0xbf,0xd9,0x28, +0x15,0x34,0x76,0x1e,0x86,0x6d,0xb3,0x7e,0x7f,0x47,0xf5,0x92, +0xcd,0xfa,0xa6,0x44,0xe1,0x00,0xbf,0x48,0x5c,0xb8,0x9c,0xce, +0x6b,0x2f,0x65,0x90,0x27,0x38,0x62,0x9b,0x80,0x1f,0x79,0x0f, +0x7f,0xf6,0x12,0x9b,0x48,0xdf,0xf9,0xfa,0x7c,0x0d,0x20,0xf7, +0x37,0x65,0xbd,0x21,0xa0,0x32,0x9d,0x94,0xd5,0x74,0x5d,0x97, +0xc8,0xad,0x39,0x39,0xc7,0x7f,0x77,0x17,0xb4,0xa9,0xc3,0xf8, +0x77,0x52,0x6a,0x66,0x39,0x83,0xc5,0xe1,0xae,0x04,0x58,0x08, +0x64,0x87,0xc3,0x9d,0xec,0x40,0xd4,0x88,0x27,0x5d,0xc5,0x64, +0x62,0x65,0x2c,0x44,0x04,0x91,0xb7,0x2e,0x06,0xcc,0x01,0xbc, +0xa5,0x84,0x6f,0x94,0x82,0x78,0x8f,0xe8,0xb4,0xd1,0x2b,0x36, +0xc3,0x4b,0x06,0x89,0x2d,0xe1,0x4d,0x47,0xeb,0x08,0x01,0x77, +0x06,0x22,0xbc,0x80,0xbc,0x61,0x01,0xe2,0x94,0xe7,0x80,0xdd, +0x34,0xbb,0xdb,0x97,0xfd,0x2d,0xe2,0xda,0xe5,0x0f,0x5c,0x7e, +0x5d,0x9d,0xcf,0x61,0xde,0xd2,0x8b,0x9b,0xb5,0x99,0x55,0xeb, +0xec,0x6e,0xb6,0x3c,0x82,0xdf,0x05,0xfc,0x37,0x86,0xef,0xc9, +0x10,0x84,0xa0,0x79,0x89,0xd9,0x40,0x53,0x00,0x0e,0xfc,0x70, +0x24,0xa5,0xb2,0x14,0x24,0xba,0x16,0x81,0xb2,0x17,0x20,0x32, +0x22,0x82,0xcb,0xc5,0xa6,0x36,0xbf,0xde,0x4c,0xe7,0xd5,0x65, +0x55,0xae,0x01,0xdd,0xe5,0x8a,0xc8,0xd5,0x26,0x15,0xf6,0xd7, +0x76,0x7b,0x62,0x2c,0x5e,0xbe,0x55,0x69,0xcb,0x66,0xeb,0x1a, +0x56,0xb1,0xed,0x15,0xd0,0xad,0xeb,0xc2,0x4a,0x67,0xcc,0x83, +0x79,0x3a,0xa0,0xc2,0x5f,0xa7,0xf3,0xa2,0xd3,0xb1,0x8d,0x00, +0xeb,0x9a,0xcf,0xa5,0x8c,0x21,0xb6,0xaf,0x03,0x90,0xb2,0x45, +0x81,0x20,0xee,0x88,0xda,0x5d,0x2d,0x3b,0xdc,0x8f,0x06,0x41, +0x11,0x7d,0x84,0x1f,0xd0,0xaa,0x1b,0xaf,0xdf,0x43,0x63,0xc9, +0xb8,0xd1,0xd2,0x00,0x2e,0xab,0x39,0x6c,0x68,0xe5,0xac,0xf8, +0x84,0x0e,0x1b,0xcb,0x9e,0xbb,0xaa,0xea,0x67,0xd5,0xf5,0x6a, +0x2e,0x6b,0x66,0x2f,0x56,0xb9,0x47,0x97,0x6f,0x14,0x04,0xd3, +0xa1,0xb9,0x1b,0xba,0x81,0x7c,0xa8,0x46,0xb6,0xfb,0x74,0x6c, +0xe9,0xfc,0x2f,0x78,0x37,0xa3,0x89,0x72,0xb0,0x9e,0x16,0x27, +0x16,0x7d,0xf1,0x41,0xe0,0xd9,0xf4,0xb2,0xfc,0x76,0x3d,0x7d, +0x85,0xad,0xa7,0xb3,0xe5,0xc5,0x0d,0xfe,0x90,0x93,0x4f,0x55, +0x6f,0x0a,0xc4,0x06,0x0a,0x22,0xb5,0x6e,0x45,0xdb,0x24,0x33, +0xb5,0x54,0x2a,0xb4,0x42,0x9f,0x1b,0x7b,0x2c,0x9f,0xb6,0x41, +0x42,0xa1,0x96,0x96,0x42,0x4f,0x78,0x28,0x19,0xae,0xf7,0x21, +0xf6,0xa1,0x3b,0x59,0xd6,0x5e,0x30,0xa5,0x32,0x24,0x9c,0x5a, +0xaa,0xd3,0x92,0x6e,0x34,0xc0,0x78,0x66,0x3f,0xad,0x1f,0xae, +0x56,0x28,0xe7,0xd1,0xf8,0x37,0x78,0xd8,0xf0,0xa8,0x09,0x79, +0xaf,0x34,0x59,0x7f,0xf3,0xfe,0xf9,0xf4,0x15,0x0e,0x2b,0xc5, +0x52,0xe3,0x13,0x90,0x66,0xa9,0xc8,0x94,0xea,0x3f,0x82,0x85, +0xcb,0x8d,0xf4,0x97,0xb7,0x8b,0x72,0xfd,0x38,0x1c,0xa5,0xc2, +0x85,0x55,0x3d,0x7c,0xce,0xaa,0x7a,0x7a,0x3e,0x2f,0x9f,0x5d, +0xac,0xab,0xd5,0x46,0xa6,0x07,0xd1,0x38,0x05,0x7e,0xdf,0x64, +0xfe,0xb8,0x23,0xa6,0x09,0x92,0x6f,0x62,0x47,0x45,0x85,0xe8, +0x74,0x99,0x62,0xa5,0x6e,0x17,0xff,0x05,0xbc,0x97,0x17,0x38, +0x8f,0xb3,0xac,0x97,0xdc,0x4f,0x7a,0xb6,0x10,0xad,0x43,0x9f, +0x5d,0xd5,0x1b,0x10,0xe8,0x1b,0xdd,0xd3,0xc9,0xac,0x58,0xd7, +0x94,0x8e,0x74,0xfd,0xc3,0xb4,0x7e,0x53,0xce,0x40,0x92,0x2a, +0x2f,0x52,0xdb,0x98,0x85,0x81,0x8a,0x8f,0x1c,0x24,0xf4,0x3d, +0x1e,0xf0,0xd1,0x05,0xf6,0x3c,0x14,0x10,0xdd,0x0e,0x26,0x03, +0x88,0x40,0xa9,0xcb,0xcd,0xbf,0xcf,0x97,0xe7,0xd3,0xf9,0x93, +0xb7,0x53,0xe6,0x1a,0x35,0x08,0x9c,0x97,0x8a,0x7c,0x38,0x71, +0xc3,0xd4,0x23,0x6c,0xcc,0x50,0x8a,0x93,0x21,0x6e,0x59,0x9d, +0x82,0x8a,0x12,0xaa,0x6a,0x10,0x5c,0x32,0x12,0x6b,0x84,0xb2, +0x5f,0xba,0xad,0x27,0x79,0x65,0xdb,0x4e,0x4c,0xc7,0x6b,0xd7, +0x1e,0x5d,0xb8,0xb0,0x97,0x03,0xad,0x05,0xd5,0xfc,0x69,0xbb, +0x98,0x2f,0x17,0xe5,0xa3,0xe5,0xea,0xfd,0x93,0xb7,0x38,0xa9, +0xf5,0xfa,0xc2,0xcc,0x00,0x95,0xc4,0x6d,0x91,0x27,0xe0,0x87, +0x77,0x6a,0x97,0x2e,0xae,0xa6,0x35,0x89,0x30,0x50,0x5c,0xcf, +0x9c,0x38,0x25,0x28,0x06,0x2f,0xe7,0xb3,0xc7,0xde,0xd1,0x83, +0x81,0xc1,0x72,0x06,0x98,0x7d,0x33,0x07,0xdb,0xd7,0x3a,0x80, +0x48,0x04,0xa2,0x2e,0xe4,0xbb,0xcf,0x9f,0xb8,0x88,0xf8,0x17, +0x6c,0x32,0x22,0xf6,0x70,0x53,0x00,0xc7,0x62,0x36,0x2f,0xb5, +0x65,0x29,0x8f,0x67,0x75,0x44,0x31,0x82,0x84,0xbb,0xa9,0xd4, +0x75,0xf2,0x23,0x27,0x8c,0x31,0x7f,0x12,0xcb,0x91,0x2a,0x68, +0x63,0x91,0xfe,0x74,0x36,0x63,0x00,0x99,0xe0,0xbc,0x6a,0x38, +0x41,0x3b,0xed,0x16,0x07,0x02,0x72,0x9e,0xff,0x19,0x9d,0xbc, +0xe0,0x80,0xe8,0xe7,0xc2,0x04,0xf8,0x4b,0xf7,0xdd,0x23,0x9c, +0x04,0x5c,0x12,0x4f,0xeb,0xfa,0xa6,0xac,0xbd,0x59,0x20,0xf1, +0x40,0x78,0x90,0x61,0x1d,0xc0,0xb0,0x6d,0x66,0xf8,0x10,0x20, +0x05,0x5d,0x16,0x7e,0x85,0xf2,0x98,0xe9,0x44,0xb2,0xe6,0x62, +0x49,0xbd,0xd3,0xf4,0x77,0xbb,0x58,0x33,0x3e,0x7c,0x64,0x77, +0xb3,0xd6,0x69,0x63,0x91,0x9c,0x50,0x3c,0x73,0xd8,0x57,0x0c, +0xf2,0x5a,0x61,0xaa,0x22,0x1c,0x32,0xf4,0x32,0x65,0xd9,0x90, +0x60,0x8c,0x17,0x54,0xd8,0x73,0xb6,0x4b,0x78,0xdd,0xa2,0x18, +0xa7,0xc3,0x61,0x18,0x61,0x3f,0x7a,0xb7,0x01,0xa9,0x02,0x30, +0x45,0x3f,0x47,0x69,0xc8,0x7f,0x08,0x3e,0xca,0xb1,0x45,0x4c, +0xc8,0x22,0xa8,0x44,0x96,0xbb,0xe3,0xb8,0xed,0x61,0x44,0x79, +0x28,0x13,0x02,0xec,0x38,0x2b,0x30,0xb7,0x94,0xb2,0x04,0x10, +0xd7,0xdf,0x3d,0xff,0xe1,0x7b,0x6a,0xd3,0x7e,0x65,0x26,0x42, +0xe9,0xd5,0xe6,0x7a,0xfe,0x15,0x61,0xb5,0xdb,0xc5,0x92,0x20, +0xd7,0x70,0xc9,0x6e,0x57,0xb1,0x0f,0x03,0xbe,0xe6,0x46,0x6d, +0x66,0xa6,0xdd,0xd8,0x94,0x22,0xa8,0x8c,0xc0,0x56,0x8b,0xd5, +0x4d,0x84,0x8d,0xeb,0xe9,0xa2,0x5a,0xdd,0xcc,0xa7,0x48,0x4b, +0x2f,0xd7,0x70,0x06,0xb8,0x78,0x83,0x78,0x40,0xa2,0xe0,0x4d, +0x9b,0xc6,0x8f,0x24,0x22,0xe3,0x02,0x29,0x75,0x0a,0x22,0xfb, +0x23,0x2c,0x09,0x32,0x02,0xa5,0x5d,0xc8,0x07,0x96,0x95,0xdf, +0x44,0x80,0x7d,0x60,0x17,0x37,0xa5,0xe0,0x99,0x7e,0x2b,0x90, +0xf4,0xe1,0x92,0x09,0x93,0x2b,0xd1,0x5f,0x38,0x4c,0xfa,0x1d, +0x3e,0x83,0xa5,0x7b,0xb1,0xd1,0x1e,0x6b,0xfd,0xc2,0x26,0xa2, +0x12,0x79,0xda,0x1c,0xe8,0x76,0x9b,0xe0,0x24,0xc2,0x9c,0x4c, +0xfd,0x64,0x8b,0x34,0x69,0xe2,0xaf,0x16,0x2c,0x3f,0xc1,0x5f, +0x6e,0xb8,0x0b,0x81,0x30,0x07,0x27,0x5c,0x22,0x0a,0xda,0x26, +0x95,0x19,0xd7,0x46,0x59,0xb2,0xb9,0x5c,0x82,0x3c,0xaf,0xf2, +0x95,0x14,0x6e,0xdd,0x42,0x51,0xb4,0x45,0x4d,0x12,0xc8,0x64, +0xf6,0x04,0x30,0x3a,0x54,0x01,0x37,0x4e,0x18,0xcd,0xbd,0x24, +0xcb,0xa3,0xe6,0x7f,0x45,0xd2,0x60,0x24,0x2c,0xd7,0x0f,0x71, +0x4b,0x38,0xd0,0x74,0x5c,0xd8,0x6b,0xd6,0x16,0xa6,0xb3,0x2f, +0x8d,0x84,0xb8,0x1f,0x8f,0x69,0x3c,0xa1,0x51,0xd6,0x85,0xb6, +0x74,0x81,0xfb,0x3d,0xd2,0x39,0x6c,0x21,0x92,0x76,0x60,0x47, +0xea,0x50,0x3f,0x56,0xa9,0xc5,0xd3,0xe0,0x44,0x8e,0x11,0x75, +0xd2,0x5f,0xdd,0xd4,0x57,0xbc,0x0b,0xeb,0xd9,0xf6,0xba,0x5c, +0xbf,0x2a,0x19,0x04,0x23,0x73,0x60,0x2b,0xd9,0xed,0x17,0x3e, +0xfc,0xd3,0xe8,0x76,0x0b,0x09,0x76,0xd3,0xb1,0x9d,0xf9,0x73, +0x37,0x0a,0x9a,0x1f,0x4b,0xd6,0x84,0xe7,0x2f,0xcb,0xe9,0x4f, +0xc0,0x6b,0x1f,0x07,0xe4,0x2f,0x92,0xc2,0x07,0x17,0x91,0x93, +0x14,0x80,0xe0,0xe8,0x23,0x5a,0x46,0x94,0x26,0x4b,0xc7,0xdb, +0x5e,0x81,0xfd,0xcd,0x96,0xeb,0x9f,0xd7,0xcb,0x15,0x41,0x5e, +0x6f,0xde,0xc3,0x76,0x45,0x27,0x33,0xd5,0xda,0x20,0xfb,0xa4, +0x64,0x15,0xa9,0xed,0x69,0x10,0xa9,0xf2,0x62,0x4a,0x15,0x45, +0x2b,0x73,0x35,0x5d,0x3f,0xdc,0xa4,0x27,0x78,0xc4,0xfe,0x4f, +0x10,0xd5,0x84,0xa5,0xf7,0x58,0x55,0x32,0xaf,0xe0,0x34,0x3e, +0xc8,0xcc,0x72,0x5d,0xbd,0xb2,0x75,0x80,0x96,0x2f,0xea,0xfa, +0x67,0x10,0x06,0xaa,0x77,0x20,0xc4,0xca,0x66,0x57,0x1d,0x1f, +0x0f,0x33,0xe9,0xdf,0xcf,0x87,0x49,0xee,0x49,0x8f,0x66,0x3f, +0x6c,0xf2,0x5b,0xfb,0xf1,0x8f,0xb8,0xdf,0x55,0xb3,0x59,0x29, +0xa7,0xa9,0x72,0x1e,0x9e,0x66,0xca,0x39,0x4b,0x99,0x26,0x59, +0x00,0x67,0xc4,0x45,0xac,0x1a,0x91,0xba,0x16,0xd1,0x06,0x78, +0xf8,0x6a,0x3e,0x7d,0x9f,0x64,0xee,0xac,0x8b,0x13,0x3a,0xad, +0x16,0x75,0x8b,0x3c,0xca,0x27,0x31,0x4f,0xee,0xba,0x5a,0xde, +0x02,0x00,0xa5,0x3b,0x2d,0x60,0x8a,0x13,0xb7,0xa4,0x75,0x5e, +0xe3,0x57,0x04,0xa9,0x21,0xe6,0x55,0xe3,0x9a,0x00,0x01,0xba, +0x7c,0x87,0xe2,0x01,0x5f,0x78,0x68,0x1b,0x56,0x3e,0xe0,0xbb, +0x05,0x2a,0x35,0xa4,0x7f,0x61,0x2d,0xd8,0xe5,0xc1,0x52,0x16, +0xa6,0xf2,0xe2,0xea,0x13,0xd6,0x80,0x54,0xb8,0x7d,0xc9,0x2a, +0x5a,0xa4,0x39,0x10,0x75,0xec,0xa8,0x8d,0xfc,0x2a,0x5c,0x13, +0x7d,0x05,0x1a,0x87,0x32,0x0a,0x9b,0xc3,0x9b,0x10,0xc4,0x24, +0xb0,0x09,0x29,0xb5,0xdd,0xa6,0xcd,0xaa,0x45,0x02,0x2d,0x93, +0x26,0xa4,0x99,0xd7,0xed,0x06,0x73,0x96,0x7d,0x22,0xc4,0x06, +0xa6,0xee,0xa5,0x2c,0x84,0xc7,0x9c,0x96,0xda,0x43,0x24,0xb1, +0x67,0xf8,0x5f,0x1e,0x01,0x9d,0x32,0xea,0x8b,0xb0,0x67,0x93, +0x5a,0x90,0x9a,0xa3,0xea,0x70,0x95,0xcc,0xb2,0x82,0x7d,0xf0, +0x70,0xb9,0x91,0x7c,0xe7,0xfb,0x09,0x2c,0x53,0x85,0x22,0xcf, +0xfa,0xef,0x9d,0x5e,0x9c,0x15,0x0f,0xe0,0x36,0x04,0x27,0xed, +0x19,0xed,0x53,0x45,0xb3,0x1c,0x4f,0x72,0x92,0x73,0x07,0x59, +0x70,0x52,0x42,0x98,0x82,0x33,0xc7,0xcf,0xcb,0xba,0xda,0x54, +0x6f,0xcb,0x1f,0xe9,0x76,0x8c,0x07,0x4c,0x4d,0x99,0xfa,0xe6, +0x7c,0xb3,0x9e,0x5e,0x6c,0xbc,0x13,0x11,0xd0,0xfc,0x7a,0x71, +0x73,0x4d,0x87,0x5a,0x3e,0x0b,0xf1,0x36,0x1e,0x9c,0x83,0xca, +0x7a,0xf4,0xc3,0x74,0x73,0xd5,0xbf,0x9e,0xbe,0x4b,0x4f,0x8c, +0x24,0xc1,0x61,0xe8,0x38,0xd5,0x16,0xb7,0xdb,0x93,0x2c,0xeb, +0xa5,0x9a,0xf5,0x00,0xc1,0x5d,0xbd,0x4b,0x64,0xce,0x1d,0x78, +0xd3,0x1b,0x3a,0x10,0xff,0xad,0x9a,0x6d,0xae,0x7e,0x5a,0x7f, +0x57,0x56,0xaf,0xae,0x36,0x9e,0x8e,0x1d,0x38,0xf6,0x1a,0x15, +0xa9,0xdf,0x2c,0xd7,0xb3,0x72,0xfd,0xcd,0xf2,0x9d,0x21,0xb4, +0x78,0x47,0xa5,0xaa,0xa0,0x32,0x40,0xc4,0xa9,0x57,0x6c,0x94, +0x9c,0xd3,0x4f,0xc0,0x10,0x31,0xfe,0xc5,0x26,0xc9,0x46,0x5f, +0xe6,0xc9,0x2d,0xf6,0x43,0x62,0x02,0x0a,0x21,0x83,0xfc,0x04, +0x11,0x01,0xb3,0xfc,0x25,0x09,0xf9,0xc5,0x83,0x2c,0xb9,0x9e, +0xae,0x5f,0x55,0x24,0xab,0x50,0xbb,0x4c,0xf7,0xbd,0x06,0x47, +0xa2,0xcc,0x1e,0x7c,0xb2,0x2e,0x16,0x0f,0x52,0x9d,0x13,0x05, +0x2e,0xf3,0x21,0x1e,0xa5,0x16,0x84,0xb0,0xd1,0xe3,0x26,0x9b, +0x5b,0xc1,0xb9,0x02,0xf5,0x37,0xfb,0x1b,0x56,0xf8,0x3a,0x1f, +0x6a,0x4a,0xc6,0x1f,0xb4,0xd4,0x4b,0x08,0xcf,0x89,0xdf,0x62, +0x96,0xb7,0x0f,0xf0,0x83,0xb0,0xb8,0x12,0x9d,0x0f,0xe0,0xea, +0xe3,0x61,0x51,0x22,0x83,0x66,0x02,0xb9,0xec,0x20,0x79,0x30, +0xf1,0x12,0x5d,0x3d,0x75,0x68,0x2f,0x3a,0x3c,0xb7,0xd1,0x94, +0xf3,0x66,0x71,0x79,0x59,0x4b,0xa3,0xb9,0x97,0xc0,0xcd,0x0b, +0x34,0x05,0x74,0xfb,0x8c,0x7e,0x09,0x1b,0xf2,0xa6,0xb4,0x88, +0xc4,0xfa,0xf3,0xe5,0xbb,0x67,0xd5,0x6f,0x80,0x0a,0x58,0xd4, +0x3c,0xd6,0x63,0x48,0x6a,0xdd,0xc8,0x6c,0x51,0x18,0xf8,0x40, +0x07,0x8e,0xd2,0xd8,0xc9,0x69,0x01,0xe0,0x6e,0xb7,0x7c,0x85, +0x05,0x3f,0x49,0x0e,0xc0,0x11,0xc0,0xf9,0xf0,0xd1,0xb3,0x67, +0xde,0xa8,0x75,0x02,0xa0,0x4e,0x54,0x85,0x67,0xc0,0x63,0x2a, +0xa2,0xff,0x35,0xb8,0xa4,0x81,0x55,0xac,0xde,0xb1,0xd0,0x82, +0x65,0x33,0x87,0xeb,0x61,0x03,0x79,0xde,0x60,0xa1,0xcd,0x7d, +0xc3,0xfd,0xa5,0x9c,0x57,0x28,0x0d,0x6d,0xb7,0x6f,0x49,0xd5, +0xd9,0xec,0x17,0xd3,0x49,0x85,0xfe,0xed,0x7c,0x39,0xe5,0x7e, +0x81,0x33,0xec,0x5c,0xd7,0xbd,0x8f,0x61,0x00,0x74,0x5f,0x7b, +0x70,0x6d,0x9b,0x78,0x04,0x8a,0xa4,0x1e,0x72,0x1d,0x4f,0xa9, +0xd1,0xdc,0x96,0xec,0x8e,0x44,0x64,0x34,0x5b,0x5e,0x58,0xe5, +0x5d,0xb0,0xeb,0xca,0xef,0xb1,0x16,0x9f,0x28,0xad,0x3a,0xb6, +0xad,0xc5,0x81,0xfd,0xdd,0x4c,0xe7,0x71,0xfb,0x06,0x9a,0xcd, +0x4c,0xbc,0x85,0xc1,0xa1,0xd5,0xd6,0xaf,0x2e,0xd7,0x28,0x76, +0xc9,0x5f,0x15,0xa5,0xd3,0xe4,0x6b,0x4e,0x38,0xa2,0x7f,0x79, +0xec,0xc5,0xe7,0x27,0x9f,0x1f,0x11,0x65,0xd3,0xaf,0x2b,0x42, +0x1b,0xfe,0xbc,0x7f,0x9a,0x64,0x44,0x6e,0x09,0xfc,0xf3,0x1c, +0xb0,0x97,0xd8,0xbd,0x2d,0x3f,0x9f,0x2f,0x2f,0xde,0x1c,0x75, +0xaa,0x6b,0x9c,0xc5,0x29,0xa2,0x2d,0x13,0xc5,0xde,0xf3,0x25, +0xaa,0x38,0xfb,0x3a,0x70,0xd5,0x44,0x22,0xc8,0x0a,0xce,0xf8, +0x64,0xd2,0x17,0x74,0xb3,0x05,0xc7,0x76,0xdb,0xc8,0x50,0xf9, +0x2b,0xeb,0x3b,0x14,0x42,0xb3,0xb7,0xeb,0x0a,0x55,0x62,0x5f, +0x77,0xe0,0x83,0x54,0x2f,0x78,0x0c,0x3e,0xfd,0x9a,0xff,0x3d, +0x5f,0xce,0xde,0x9f,0x26,0xd4,0x55,0xff,0x62,0xbe,0x24,0x4d, +0xc4,0xc7,0x20,0x92,0x3b,0x07,0x61,0x7b,0x83,0xe6,0x16,0x19, +0xab,0xda,0x1a,0x93,0xa4,0x68,0xb6,0x6d,0x7a,0x3b,0x4f,0xd8, +0xb8,0x36,0x6c,0x8f,0x7c,0xb2,0x76,0x09,0x31,0xa1,0x7e,0x93, +0xaf,0xc7,0x43,0xd4,0xe1,0x30,0x1c,0xe0,0xd1,0xb2,0x07,0x14, +0x79,0x12,0x46,0xa0,0xd5,0x64,0x3d,0x47,0xda,0x02,0xdf,0xf9, +0x0d,0x1c,0xbe,0x7e,0x9e,0xc2,0x28,0xeb,0x74,0x45,0x62,0x38, +0xda,0x83,0x18,0x58,0x0e,0xb3,0x0a,0x0b,0x4c,0xe7,0x06,0xf8, +0xaf,0x77,0x4f,0xe4,0xae,0xa2,0xf5,0xf2,0x16,0x6f,0x8b,0xb2, +0xc0,0x2a,0x05,0x1a,0xb0,0x8a,0xf2,0xca,0xbc,0xc5,0xdb,0x34, +0xdb,0xdc,0x76,0x4b,0xb7,0x60,0x6f,0xca,0x0d,0x33,0x09,0xee, +0x34,0x1b,0xa1,0xae,0x4b,0x00,0x78,0x9b,0xe5,0x4d,0xb0,0x7a, +0xc9,0x38,0xe9,0x35,0xad,0x17,0xde,0x8e,0xaa,0x1c,0x04,0xcc, +0x5e,0x32,0x49,0xcc,0xdb,0x06,0xd8,0xbb,0x6c,0x48,0x17,0x64, +0x00,0x74,0x00,0x82,0x36,0xd3,0x29,0x1a,0x96,0x44,0x99,0x07, +0x08,0x59,0x16,0x51,0x03,0xf1,0xed,0xd8,0x1e,0xf8,0xb0,0x08, +0x81,0x62,0x2f,0xc5,0x9a,0x20,0x39,0xd2,0x98,0xc1,0xb4,0xd2, +0xd1,0x07,0x6f,0x1e,0xea,0x9f,0xd6,0xcf,0xd7,0xd3,0x45,0x8d, +0xab,0xa6,0x4e,0xf1,0xc8,0x0d,0x94,0xb3,0x2e,0xed,0x19,0xc6, +0x22,0x14,0x45,0x4f,0x3c,0x18,0xc2,0x1e,0xb7,0x2e,0xeb,0x1a, +0x92,0x08,0xd9,0xd9,0x9d,0x5e,0xcb,0x74,0xfc,0x9b,0xcc,0xb0, +0x24,0x70,0x59,0x2c,0x5b,0xb4,0xb4,0xd1,0x4c,0x2a,0xf0,0x30, +0x3f,0x24,0x76,0x25,0x79,0xa4,0x99,0xd0,0x8f,0xba,0xa5,0x95, +0x50,0xd7,0xb7,0xdf,0x2c,0x29,0xa0,0x22,0x7b,0x69,0x46,0xc3, +0xe0,0x8b,0x09,0x6d,0xda,0xf6,0x01,0xf2,0x68,0xaf,0x37,0x19, +0x66,0x49,0x4f,0xef,0x54,0x31,0x15,0x48,0x7e,0x94,0x36,0xca, +0xda,0xb3,0x28,0xe9,0x23,0x8c,0x43,0xe6,0x58,0x4b,0x4c,0x8a, +0x96,0x34,0x04,0x2c,0xeb,0xdf,0x2c,0xea,0xab,0xea,0x72,0x23, +0xc0,0xe4,0x9f,0x56,0x99,0xd4,0x0e,0x54,0x33,0x30,0xef,0xc0, +0x3b,0x84,0xcd,0x87,0x66,0xda,0x88,0x31,0x17,0x1d,0xa0,0x2b, +0x20,0x17,0x31,0x12,0x33,0xaf,0x7f,0xfd,0xfb,0x77,0xbf,0xf8, +0x76,0x72,0xdc,0x9e,0x1d,0x37,0x2f,0x4e,0x55,0x68,0xe9,0xba, +0x97,0x52,0xe5,0xcc,0x83,0xbb,0x63,0xaf,0x2a,0x69,0x9d,0xee, +0x19,0x86,0x6f,0x33,0xb6,0x52,0xa0,0x7f,0x5a,0x7f,0x3b,0x45, +0x4d,0xcf,0x7b,0xd9,0xc0,0xa4,0x82,0x37,0x94,0xa2,0x59,0x36, +0x3d,0x3c,0x24,0x01,0x75,0x2f,0xdd,0x7a,0x8d,0x6f,0xb7,0x75, +0x59,0xbe,0x81,0x52,0x5e,0x4a,0x73,0x84,0x5e,0x85,0xc9,0x28, +0xae,0x30,0xea,0xa4,0x56,0xe9,0xd7,0x52,0x01,0xce,0x0c,0xcb, +0x6a,0x76,0x74,0x92,0x5b,0xa3,0x3a,0x4b,0x7a,0x96,0x26,0xda, +0xaa,0x99,0x78,0x3a,0x82,0xcc,0xce,0x00,0x78,0x90,0xd1,0x7e, +0x77,0x74,0x94,0x50,0xb0,0xd1,0x6c,0x2b,0x06,0xd2,0x91,0x16, +0x9a,0x23,0x5a,0x2a,0x89,0x26,0xb5,0x09,0x22,0x2c,0x04,0x54, +0x5a,0x38,0x94,0x00,0xe1,0x93,0x75,0x1b,0x97,0xc7,0x95,0xec, +0x31,0x9e,0xd7,0xd3,0x77,0x4f,0xf8,0x62,0x60,0x03,0x72,0x7e, +0x09,0xc2,0xe8,0x5a,0xb6,0xa4,0x59,0x59,0xae,0xc8,0xba,0xe0, +0x72,0x3e,0xdd,0xc8,0x6c,0x29,0x8b,0xc4,0x6a,0xcf,0xca,0xcd, +0x06,0xe0,0xad,0xfb,0x5e,0xfe,0x76,0x2b,0x17,0x1e,0x50,0x8f, +0x34,0x36,0xd0,0x18,0xfc,0x37,0x86,0xcf,0x49,0x64,0x80,0x95, +0x7a,0xd5,0x28,0x7f,0xc4,0x00,0xe4,0xd8,0x2f,0x59,0xb8,0x94, +0x2b,0x34,0xeb,0xca,0x28,0xb3,0xd0,0x56,0xec,0x66,0x86,0xf9, +0xf6,0x10,0x2e,0x77,0x1b,0x40,0xd4,0x32,0x0a,0xcc,0xcd,0xe4, +0x23,0x1c,0xed,0x77,0xa4,0xf4,0xff,0xa5,0x04,0x6c,0x2e,0x6a, +0xbc,0xe0,0x60,0x0a,0x44,0x95,0x3c,0xa7,0xe8,0xc5,0xf5,0xba, +0x26,0xdb,0x2a,0xb6,0x98,0xda,0x98,0xcb,0x4a,0xec,0x20,0x29, +0x81,0x6e,0x61,0x44,0x08,0xa9,0x8b,0x5a,0xe5,0x91,0xda,0x63, +0x89,0xde,0x94,0xd8,0xc6,0xbf,0xad,0xca,0xf9,0x0c,0xb3,0xc2, +0x84,0xe0,0x8e,0x28,0xcc,0xca,0xe2,0xe4,0x1a,0xc5,0x63,0x84, +0x78,0x1c,0x16,0xe4,0xcb,0xa0,0x49,0x61,0xcb,0x71,0x02,0xeb, +0x16,0x86,0x30,0xeb,0x1e,0xaf,0x44,0x1a,0x19,0x66,0x8e,0xa8, +0x99,0xa4,0x33,0x18,0x66,0x68,0x80,0x93,0x42,0x42,0xdd,0xbf, +0xae,0xae,0x49,0xfd,0x08,0xf2,0x21,0xf6,0x8b,0x9a,0x64,0x45, +0xdf,0x77,0xe5,0x14,0x64,0xc3,0x34,0x79,0xc4,0xc3,0x3f,0x7e, +0x4e,0x97,0x90,0x74,0xbc,0xb8,0xe0,0x7b,0x65,0x05,0x5f,0x11, +0x84,0xba,0x3e,0xfd,0xcd,0x10,0x76,0xbb,0xe1,0x37,0xcb,0x02, +0x50,0x9d,0x2f,0x7b,0xc2,0x65,0xc7,0x37,0xa4,0xe7,0x20,0x1d, +0xbd,0xd9,0x89,0x39,0x8f,0x0e,0xc8,0xc7,0x51,0x16,0xcc,0x56, +0x38,0x6e,0xdc,0xc4,0xef,0xda,0x10,0x5e,0xd3,0x01,0xa8,0xe3, +0x17,0x06,0x5e,0x83,0x53,0xfb,0xb6,0x5c,0x23,0xb3,0x26,0xf8, +0x7a,0xc9,0x51,0xd2,0xf3,0xcb,0x4c,0x80,0x1f,0x07,0xbd,0x61, +0x29,0x01,0x31,0xa0,0x22,0x20,0xea,0xe0,0x9b,0x0a,0xc2,0x5a, +0x0c,0x2a,0x07,0x5f,0xdb,0x6d,0x50,0x41,0x4f,0x31,0x41,0x99, +0x51,0x1a,0x7c,0x76,0xc2,0x59,0x06,0x69,0xbf,0x81,0xc3,0xa0, +0x7c,0xe6,0x08,0x7f,0x1c,0x64,0x4c,0x94,0x0b,0x86,0xcb,0xe7, +0x11,0x63,0x23,0x75,0x34,0xcd,0xeb,0x05,0xb1,0xf4,0x00,0x6f, +0x16,0xf1,0xa6,0x0a,0x57,0x06,0x48,0x60,0xd7,0x2b,0xe3,0x90, +0x47,0x96,0xa9,0x81,0xcc,0xe0,0xb3,0x55,0xde,0xa6,0x33,0xdc, +0x64,0xde,0x86,0xd3,0x85,0x66,0x0b,0x54,0xf0,0x5b,0xda,0x50, +0xc8,0xce,0x92,0x3b,0x2e,0xfc,0x74,0x9b,0x6a,0x5c,0xb3,0x59, +0xe6,0x7a,0x1b,0x0f,0x26,0x44,0x8f,0x08,0x10,0x71,0x26,0x6f, +0x5e,0x33,0x6f,0x8a,0xf1,0x67,0x28,0xba,0x4c,0x8a,0x80,0x06, +0xf0,0xe7,0x84,0x17,0x95,0x8c,0xd6,0x83,0xb7,0xd7,0xab,0x26, +0xa4,0xce,0x86,0xf5,0xc6,0x86,0x46,0x6b,0x32,0xe2,0xb0,0x29, +0x38,0xbe,0x6e,0x17,0xff,0x8d,0xb2,0xb1,0xd9,0xc2,0xeb,0x06, +0x8b,0x10,0xa9,0x49,0xa1,0x09,0x5d,0x88,0x68,0x6e,0x72,0xcf, +0xcb,0x31,0x1d,0xcc,0xb1,0xa3,0x7b,0x20,0xcb,0x4d,0x07,0x87, +0x02,0xef,0xf5,0x8a,0xda,0x7e,0xe0,0x59,0x31,0xe2,0xf4,0x00, +0x52,0x0a,0x0b,0x06,0x2e,0xf7,0x7d,0x40,0x60,0x59,0xa0,0xf4, +0x26,0x0c,0x92,0x01,0x6b,0x95,0xea,0x16,0x20,0x57,0x8c,0xe2, +0x56,0xa8,0xe7,0x49,0xde,0x48,0xe9,0x60,0x69,0xbe,0xac,0x26, +0x34,0x72,0x5b,0xc6,0x23,0x8a,0x15,0x51,0x45,0x75,0x7c,0x6c, +0x4e,0x94,0xb4,0x32,0x6f,0xf5,0x63,0x33,0xd4,0x88,0x30,0x15, +0xc0,0x2c,0x80,0xb5,0xb9,0x5a,0x2f,0x6f,0xeb,0x64,0x92,0x59, +0x3a,0xc1,0x2c,0x4b,0x1f,0x22,0xc5,0xa3,0x51,0xdf,0x9e,0x02, +0xce,0xda,0x8e,0xd7,0xdb,0x5d,0xbd,0x81,0xb3,0x58,0x9e,0x90, +0x5a,0x61,0x5d,0xae,0xd7,0xcb,0x75,0x62,0xe8,0x0f,0x8d,0x69, +0x04,0x59,0x3f,0x2e,0x05,0xe3,0x35,0x5d,0xed,0xac,0x97,0xd7, +0x80,0x1b,0xc1,0xde,0x66,0xe9,0xe6,0x6a,0xb7,0xdb,0x11,0x85, +0xeb,0x67,0xd8,0x41,0x7d,0x73,0x71,0x01,0xc2,0x73,0x62,0xd8, +0x42,0x50,0xe0,0xd9,0x35,0x0c,0x85,0xe0,0x28,0x3d,0x9b,0xae, +0x67,0xc0,0x8b,0x53,0xb6,0x4e,0xd4,0x2b,0x91,0xf2,0xf6,0x88, +0x6d,0x32,0xfb,0x7f,0xff,0xe1,0xfb,0xef,0x36,0x9b,0xd5,0x2f, +0xe5,0xaf,0x37,0xc0,0x4d,0x3d,0xfb,0xc1,0xb8,0xb1,0x87,0x17, +0xa8,0xa8,0x3d,0xd4,0x94,0x94,0x10,0x13,0xbf,0xe4,0x87,0xea, +0x62,0xbd,0xac,0x97,0x97,0x1b,0xea,0xe3,0xf9,0xf3,0x9f,0x93, +0xec,0x40,0xf3,0xdf,0xbe,0xfb,0x71,0x79,0x6b,0x8d,0xf6,0x50, +0x37,0xfc,0x1c,0x76,0x93,0xe5,0x0d,0x0b,0xd5,0x24,0x59,0x02, +0xfb,0xc4,0x42,0x9e,0xd1,0x62,0x66,0x38,0xc5,0x5e,0xb6,0x41, +0x0b,0x71,0xc3,0xcf,0x6f,0xcb,0x72,0x51,0xa7,0xd3,0x45,0x75, +0x4d,0x57,0x65,0xc0,0x39,0x96,0xab,0xda,0x5a,0x13,0x93,0x50, +0x4b,0x49,0x4e,0x84,0xc5,0x4f,0x56,0xdb,0x38,0x65,0xee,0xc5, +0x72,0x8e,0xf2,0x18,0x1e,0x71,0xd2,0x0d,0x36,0xc9,0x64,0xbf, +0x5c,0x89,0x14,0x2f,0xd6,0xc7,0x36,0x0b,0x65,0xa9,0x2c,0xbe, +0xa0,0x71,0x8d,0x1c,0xbc,0xa2,0x21,0x22,0xd5,0x92,0xa2,0x53, +0x67,0x73,0xbe,0x70,0x14,0x02,0xa3,0x5a,0xfc,0x7a,0x43,0x7f, +0xa8,0xe5,0x58,0x5f,0x85,0x85,0x61,0x45,0x55,0x20,0x60,0x04, +0xde,0x23,0x40,0x39,0x37,0x73,0x54,0x2b,0x2e,0x57,0xab,0x72, +0x16,0x43,0x6b,0x3b,0x73,0xa7,0x10,0xf5,0xa7,0x81,0x09,0x00, +0xba,0x76,0xd6,0x7a,0x8f,0xe5,0x1b,0xce,0x6f,0xd3,0xf9,0xed, +0xf4,0x7d,0xed,0xcf,0x9a,0x9a,0x39,0x57,0x17,0x6f,0xfa,0x64, +0x75,0x04,0x2c,0x05,0x7e,0x17,0x5e,0x11,0xe4,0xdd,0x0c,0x84, +0xb3,0xfe,0xb4,0x98,0xe7,0x25,0x80,0xe4,0x50,0xd0,0x74,0x03, +0x73,0xf1,0x49,0x06,0xf6,0x97,0xeb,0x69,0xb5,0x00,0x09,0xb3, +0xf0,0x34,0xff,0x16,0xf8,0x3e,0xac,0x99,0x35,0xd5,0xee,0xb9, +0xb4,0xd9,0xcd,0x9a,0x7e,0x1c,0x7b,0x8d,0x03,0x54,0x25,0xb0, +0x3f,0xdb,0xda,0xfd,0x66,0x71,0xb4,0xdb,0x04,0x44,0x5e,0x20, +0x13,0x1a,0x1c,0x63,0xf1,0xbd,0x38,0xeb,0x13,0x25,0x1c,0xbe, +0x8a,0x8b,0x0b,0xeb,0x54,0xaf,0x6f,0x80,0x06,0xb9,0x1b,0x4f, +0x8a,0x65,0x0c,0x03,0x91,0x6f,0xaa,0xcb,0xf7,0x7f,0xab,0x36, +0x7c,0x57,0x6d,0xc6,0x1e,0x4d,0x70,0x1d,0x87,0x10,0xa0,0xc0, +0xc1,0xa9,0xa4,0x5a,0xef,0x25,0x9b,0x9b,0xa7,0xb6,0x51,0x20, +0x84,0xe5,0xfc,0x6d,0xd9,0xd6,0xea,0x84,0x0f,0x24,0x0e,0xa1, +0x85,0xad,0x05,0x64,0x75,0x5d,0xc1,0xce,0x77,0x87,0x55,0x72, +0x4b,0x69,0x75,0xde,0x30,0x24,0x72,0x04,0x98,0x21,0x05,0xc6, +0x25,0x40,0x1c,0xbf,0x23,0x9b,0xb9,0xe9,0xfc,0xc9,0xb4,0x46, +0xd0,0x80,0x41,0x58,0x52,0xb5,0x27,0xc1,0x9f,0x6d,0x2b,0xb9, +0x4f,0xd1,0xe1,0x39,0x31,0xd7,0xf3,0xa3,0x9d,0xf7,0xbc,0x95, +0x6a,0x74,0x4e,0x73,0x7b,0x2e,0x92,0x04,0xc3,0x93,0x91,0xc3, +0xa1,0xd6,0xe3,0x20,0x79,0xc8,0x1d,0x00,0x6a,0xb1,0x21,0xc3, +0x4c,0x5d,0x06,0x54,0x92,0x11,0xe8,0xe6,0x16,0xc7,0x6b,0xb4, +0x52,0x94,0xde,0x0f,0x46,0xad,0xfc,0x24,0x2a,0x53,0x52,0xa6, +0xa5,0x84,0x06,0x89,0x91,0x02,0x81,0x7e,0x67,0x0c,0xfb,0x8e, +0x56,0xb4,0x83,0xf8,0xd5,0x72,0xb3,0x7c,0xa2,0x00,0x47,0x04, +0x2b,0x79,0xa3,0x3d,0x84,0x9b,0x9f,0x0c,0x1b,0x6b,0x93,0x1c, +0x16,0x68,0x79,0x4a,0x3a,0x6c,0xac,0x9f,0x4e,0xe0,0x03,0x3b, +0x20,0x05,0xe1,0x63,0xa8,0xd1,0x48,0x61,0x10,0x3a,0xbd,0xe2, +0xb8,0xe1,0x1c,0x2e,0x4d,0x9e,0x2e,0xbb,0x5d,0xc6,0x04,0xea, +0x2d,0x53,0xfa,0xa6,0xc1,0xe0,0x2f,0x11,0x11,0x79,0x3f,0x38, +0x34,0x55,0xd9,0x5e,0xce,0xcd,0x3c,0xb5,0x8d,0x79,0xee,0xe1, +0xe4,0x6e,0xdd,0x44,0x3d,0xda,0x7b,0x10,0x6e,0xd2,0xda,0x9f, +0x1f,0xda,0xd6,0x5a,0xcc,0xca,0xe3,0x61,0xe0,0xb2,0xc8,0xba, +0xdd,0xb6,0x64,0xcf,0x66,0xdc,0x66,0xdb,0x26,0x2f,0xdf,0xf5, +0x37,0xb0,0x9a,0xd6,0x69,0xb8,0x76,0x91,0x95,0x1b,0x8f,0x03, +0x60,0xc5,0xdc,0x81,0x05,0x62,0xc5,0x4d,0x99,0x47,0x9d,0x51, +0x22,0x9c,0xd8,0x4d,0x30,0x0f,0xaf,0x50,0x21,0x18,0x83,0xab, +0xe9,0xa8,0xb8,0x5f,0x94,0x71,0x2e,0xa6,0xc5,0xf3,0x74,0xb1, +0x44,0xe3,0xef,0x4d,0x99,0xf5,0x2f,0xa7,0xd5,0x3c,0xae,0x81, +0x69,0x76,0x93,0x8a,0xf2,0x38,0xd5,0xdb,0x46,0x1b,0x24,0x11, +0xd2,0x80,0xbb,0xdd,0xe3,0x1b,0x21,0x9a,0x5f,0xc3,0xcb,0xd5, +0x5c,0x2d,0x97,0x6f,0x6a,0x77,0x5f,0x8f,0xb2,0x36,0x4f,0x91, +0xda,0xb0,0x34,0x3c,0x8c,0xa8,0x5c,0x26,0xf5,0x8b,0x90,0x33, +0xb0,0x5e,0x98,0x0d,0xd8,0xa8,0x19,0xbd,0xda,0x8f,0xf4,0xea, +0x2c,0x11,0xa0,0xbd,0x0f,0xb7,0x42,0xdf,0x20,0xbd,0xb7,0x54, +0x95,0xbc,0x13,0x15,0x53,0xc4,0xfd,0x02,0x9d,0xf7,0xa8,0x14, +0x75,0xc9,0x85,0x8c,0x6c,0xe1,0x7e,0xf5,0x8c,0x47,0xe8,0x5d, +0x28,0x7c,0x87,0xdf,0x02,0x29,0xe5,0x75,0xbb,0x09,0x5b,0x45, +0x26,0x30,0x78,0x4a,0x41,0x94,0x21,0x1c,0xf4,0x21,0x26,0x93, +0x02,0x73,0xd8,0x07,0xb5,0x12,0x62,0x8f,0x8b,0x85,0xc8,0x84, +0x93,0x72,0xcb,0x90,0x04,0x35,0x21,0x06,0xa5,0x84,0x30,0x53, +0xf6,0x81,0x69,0xc1,0xb1,0x14,0xf0,0x6c,0xc4,0xf9,0x4e,0xce, +0x2e,0xe3,0xd4,0x5b,0xaf,0xb4,0x4a,0x89,0x0a,0x88,0xbf,0xf3, +0xfc,0x8b,0x88,0x24,0x88,0x03,0x01,0xfd,0xd9,0xd5,0xf2,0xd6, +0x6c,0x96,0xaf,0x5e,0xcd,0x4b,0xe6,0xcf,0x8c,0x1d,0xb4,0x20, +0x86,0x53,0x3b,0xd3,0x2f,0xf9,0xf9,0xf1,0xcd,0xa0,0x77,0x57, +0x49,0x7b,0x1b,0x9e,0x87,0xd9,0x00,0x95,0xec,0xe0,0xc4,0x12, +0x25,0xf0,0x7b,0x88,0x4d,0x62,0x86,0x6e,0x95,0xa1,0xed,0x8a, +0x3f,0x4f,0x2f,0x29,0x95,0xa6,0x4a,0xee,0x7d,0x2f,0xdf,0xc1, +0x19,0x8f,0x2f,0x6a,0x79,0x5a,0x6e,0x16,0x54,0x06,0xd5,0x3a, +0x61,0x02,0xec,0x1a,0x02,0xb3,0x4e,0x20,0xba,0xf4,0xf4,0x69, +0x10,0x71,0x82,0x2f,0xdd,0x85,0xad,0x6c,0xb7,0xd2,0x48,0x8a, +0x6a,0xce,0x30,0xaf,0xd7,0x23,0x6c,0xb4,0x08,0x91,0x7b,0x92, +0xc3,0xea,0x70,0xfe,0x93,0x61,0xd2,0xb7,0x37,0xc0,0xbe,0x3a, +0x62,0xc7,0x70,0x22,0x10,0xc8,0x89,0x1a,0xbe,0x24,0x30,0xfa, +0x84,0x2f,0x2f,0x13,0x47,0x6d,0x72,0x65,0x6f,0xd7,0x32,0x14, +0x22,0x54,0x2f,0xe1,0x44,0x77,0x39,0x87,0x93,0xc8,0x98,0x6d, +0x63,0xf4,0xdb,0x84,0x9f,0x7f,0x8f,0xbe,0xff,0x31,0x31,0xd0, +0xd6,0xbc,0xfa,0x90,0x69,0x99,0x9a,0xeb,0xb4,0x15,0xba,0xc4, +0xeb,0x6b,0x2c,0x12,0xdf,0x85,0x73,0xc3,0xdf,0xe0,0xcd,0xea, +0x8f,0x65,0x39,0xab,0xbf,0x9f,0xbe,0x87,0xe3,0x14,0x34,0x25, +0x3d,0xa2,0x5e,0xe1,0x43,0xa6,0x50,0x23,0x86,0xf7,0xb7,0xe5, +0xf2,0xba,0x18,0xe4,0x91,0x89,0x16,0xb7,0x73,0x4c,0x77,0xb7, +0x49,0xc6,0x82,0x9d,0x1d,0x1a,0x1a,0x1a,0x05,0x63,0x2d,0x12, +0xa6,0xdc,0x24,0xb6,0x3c,0xae,0xaf,0xd6,0xd5,0xe2,0xcd,0xdf, +0xd6,0xd3,0x15,0xc1,0x5a,0xb3,0xfc,0xd3,0x32,0xd7,0x51,0x7b, +0x41,0x77,0xa8,0x0f,0x88,0x70,0x1d,0x15,0x18,0xc4,0x05,0xfe, +0x11,0x15,0x78,0x30,0xd9,0xf9,0x06,0x56,0x01,0xc3,0x6e,0x61, +0xbc,0xeb,0xcb,0x77,0x1b,0x52,0x3e,0x78,0x96,0x48,0xec,0x9e, +0xd8,0xe4,0x95,0xb2,0xfa,0x0b,0xfe,0x83,0x66,0xc1,0xf4,0x03, +0xe7,0x93,0xd9,0x04,0xb7,0x5f,0x14,0x62,0x67,0x36,0x42,0x5c, +0x95,0x09,0x1c,0xf4,0x81,0x75,0x00,0x6a,0x51,0xef,0x59,0x2d, +0x6e,0xca,0xa1,0xf0,0x01,0x16,0xf9,0x78,0xbf,0x40,0xfd,0x86, +0xc8,0x72,0x9a,0xcb,0x9f,0xac,0x15,0x45,0xe6,0xd3,0x66,0x11, +0x77,0xf9,0x8e,0xdb,0x8e,0x3c,0x30,0x82,0x4c,0x83,0xce,0xd2, +0x3a,0x6d,0x62,0x31,0xff,0x8c,0xac,0xc7,0xd4,0x1e,0x4e,0x53, +0xfa,0x62,0xec,0x26,0x03,0x45,0x03,0xe7,0x30,0xa7,0x50,0x73, +0x38,0x35,0x77,0x93,0x2b,0x6c,0x62,0x57,0x7d,0xec,0x2d,0xcd, +0x48,0x4a,0xe0,0x6d,0xde,0x9b,0xf5,0xa0,0x20,0xa2,0x85,0xd8, +0x46,0x6b,0x51,0x65,0xc4,0x43,0x1d,0xd2,0x3a,0x72,0x8e,0xb6, +0xa3,0xb6,0x02,0x1f,0xdd,0xcd,0x02,0x93,0xd5,0xbb,0x68,0x22, +0x11,0xc7,0xe2,0x89,0x01,0xb3,0x58,0x8e,0xf7,0xc2,0x1f,0xb4, +0xbe,0xc3,0x92,0x3a,0x0d,0x76,0xee,0xe9,0x84,0x40,0x00,0x7b, +0xa2,0x1b,0x1f,0x23,0xd4,0xf6,0x4f,0x90,0xc5,0x3d,0xe5,0x27, +0x99,0xd7,0x6f,0x11,0x66,0xda,0xf9,0x8a,0x20,0x65,0xc9,0xf6, +0xc8,0x9b,0x25,0xbc,0x1a,0x09,0xaa,0x16,0x04,0x09,0x4b,0x7a, +0xd2,0x33,0xba,0x5f,0x53,0x62,0x89,0x86,0xe3,0x5e,0xb6,0xf7, +0xdb,0xb3,0x53,0xc2,0x66,0x80,0x7a,0x85,0x2d,0x4a,0x02,0xda, +0xaa,0x05,0xce,0x21,0xde,0x29,0x48,0xcf,0x62,0xf6,0xf8,0x23, +0xdb,0xaf,0xaf,0x3d,0xa2,0xe2,0x28,0xe7,0x6d,0x96,0xb8,0xa0, +0x80,0x6f,0x55,0x9b,0xc3,0xb5,0x7d,0x03,0xac,0xc5,0xb7,0xef, +0x52,0xf6,0xe3,0x59,0x5c,0xcc,0x6f,0x66,0x25,0x59,0xeb,0x30, +0x29,0xdc,0x5e,0x55,0x17,0x57,0x06,0x1d,0xb2,0xea,0xe2,0x8e, +0x61,0x26,0xeb,0x75,0x52,0x2f,0xcb,0x5c,0xba,0x3a,0x85,0xff, +0x81,0x63,0xb2,0x96,0x77,0xc7,0x41,0xd3,0xd4,0x6a,0x11,0x58, +0x9e,0x51,0x17,0x63,0xb5,0x7f,0xeb,0x51,0x89,0x49,0x21,0xa9, +0xd6,0x58,0x4d,0x92,0x49,0xe9,0x6f,0xaf,0xe6,0x5c,0xc3,0x30, +0x15,0x54,0x03,0x44,0xd1,0xe9,0x45,0xb5,0x79,0xcf,0xf5,0xfb, +0x6c,0x49,0x43,0x17,0x00,0xdc,0x4f,0x64,0x7d,0x46,0x11,0x4b, +0xca,0x36,0xc7,0xc2,0xda,0xcf,0x24,0xeb,0xb2,0xfc,0x2f,0xf1, +0x26,0x37,0xf2,0xcd,0xc2,0xff,0x5a,0x95,0xb7,0xe2,0x54,0xc7, +0x7e,0x24,0xdc,0x40,0xde,0x19,0xec,0x58,0x59,0x34,0x9d,0xbd, +0xff,0xbe,0xaa,0x37,0x66,0xbd,0x5c,0x6e,0xb8,0x17,0xd3,0xb0, +0xf5,0xd7,0xfb,0x58,0x9b,0x60,0xd4,0xe0,0xa6,0x10,0x35,0xa1, +0x35,0xc0,0x01,0x6e,0xcf,0x0a,0x05,0xc9,0xd0,0x6f,0xf3,0x92, +0x1b,0xd7,0x74,0xe9,0xea,0xe5,0x67,0x9a,0xf0,0x99,0xb9,0x98, +0x4f,0xeb,0xfa,0x01,0x79,0xbb,0xa1,0x9b,0x52,0x18,0x15,0x03, +0xa5,0x24,0x4a,0x12,0x35,0x6b,0x91,0x0c,0xfa,0x7f,0xe9,0x0f, +0x12,0x4e,0x64,0x45,0x5d,0x23,0x92,0x06,0x27,0x73,0x11,0xe4, +0xae,0xcd,0x50,0x1b,0x90,0x28,0xc3,0x45,0x65,0x73,0x23,0x9f, +0x52,0xb9,0x00,0x2d,0xfc,0x9f,0x2e,0x1b,0x45,0x24,0x9d,0x0b, +0x6d,0x96,0xcf,0xe8,0x0a,0xbb,0x70,0x63,0xe9,0x6b,0x1a,0x97, +0xb8,0x9a,0xd6,0x3f,0xdd,0x2e,0xfc,0x7c,0x4e,0x11,0x35,0x88, +0x20,0x1f,0xbd,0x72,0x0a,0x7f,0xb8,0xe4,0xa7,0x23,0x9b,0xac, +0x93,0xc2,0x6a,0xf1,0xba,0x30,0xe2,0x6d,0x10,0x2c,0x41,0x3d, +0xe1,0x2d,0x78,0xf1,0xc5,0x65,0xbd,0x09,0xcf,0x04,0xdb,0xab, +0xc5,0xcd,0x75,0x71,0x7f,0xdc,0x3b,0x9e,0x8c,0xd2,0x51,0x7e, +0x36,0xbb,0x77,0xd6,0xdf,0x66,0x67,0xb3,0x1e,0x7c,0x8c,0xcb, +0x27,0x13,0xca,0x80,0xcf,0x6d,0x76,0xbf,0x5f,0x2f,0x6f,0xd6, +0x8a,0x19,0x6b,0xd1,0x51,0xdc,0x3f,0x7b,0xd6,0xbb,0xff,0xca, +0xac,0x09,0xfc,0xfb,0x2f,0xc6,0x67,0xf5,0xd9,0xcd,0xb7,0x4f, +0xbe,0xfd,0xf6,0xec,0xdd,0xc3,0x93,0x49,0x6f,0x1b,0x7d,0x7f, +0x86,0x45,0x7f,0xbd,0x81,0xa3,0x26,0xda,0x8d,0x40,0x79,0xe8, +0x27,0xfd,0x7a,0x7c,0x76,0x7b,0xf6,0xb7,0x49,0xef,0x34,0x1b, +0xbf,0x38,0x9d,0xdc,0xdb,0xfe,0x29,0x85,0x84,0xe3,0xc9,0xbd, +0x2c,0xfb,0xec,0xbe,0x59,0x23,0xa7,0x98,0x97,0xcf,0xa7,0xaf, +0xa0,0xf4,0xd7,0xe9,0xd9,0x6d,0x2f,0x3b,0xab,0xef,0x9d,0xdd, +0x1f,0x9d,0x42,0xd5,0xaf,0xcf,0xee,0x9f,0x0d,0x4e,0xb7,0x54, +0x0e,0xb6,0xde,0x6a,0x86,0x5e,0x0a,0x35,0x81,0x31,0x31,0xf9, +0xdd,0xee,0xac,0x9e,0xdc,0xb3,0x79,0xe4,0x7f,0x0f,0x99,0x50, +0xef,0xc5,0x36,0xdf,0x9a,0x0c,0x07,0x0c,0x4d,0x8d,0x33,0x1a, +0x00,0x95,0x29,0xeb,0x8b,0xe9,0x0a,0x07,0x75,0x86,0x08,0x48, +0xce,0xce,0xce,0xee,0x9f,0x5f,0x2e,0xd6,0x9b,0xc9,0xf6,0x66, +0x7c,0x36,0x9b,0x1e,0x5f,0x3e,0x3c,0xfe,0x76,0x72,0xf7,0xe5, +0x2e,0xb3,0x35,0x36,0xcb,0x37,0x70,0x7e,0x2f,0xee,0x27,0xe3, +0x17,0x58,0x7c,0x7d,0xb6,0x98,0xdc,0x4b,0xb6,0x18,0x37,0x60, +0x4b,0xb1,0x02,0xc8,0x74,0x72,0x7b,0xcc,0xd8,0xed,0xed,0xc5, +0x2e,0x34,0x77,0x2d,0xae,0x10,0x00,0xfe,0xf1,0x75,0x7d,0x0c, +0x60,0xcf,0xa6,0xf5,0xd5,0xc3,0xf9,0xea,0x6a,0x5a,0xdc,0x3f, +0x4e,0xa9,0xff,0xdf,0x26,0x50,0xb4,0x32,0x97,0xf6,0xb0,0xe9, +0x08,0x03,0x4e,0xfc,0x70,0x64,0xd9,0x6c,0xd0,0x11,0x5b,0x48, +0x82,0x3f,0x43,0x87,0x0d,0x9c,0x77,0x3e,0x57,0xcf,0x5c,0x5d, +0xf2,0x9e,0xcb,0xee,0xac,0x13,0x32,0x3a,0x1f,0x92,0xf7,0x1c, +0xb2,0x09,0xd4,0x83,0xc3,0x66,0x01,0xe2,0xe9,0x8c,0xcc,0x8b, +0xc9,0x3b,0x51,0xa2,0x25,0x69,0x5b,0x74,0xd7,0xac,0x95,0x89, +0xc1,0x3c,0xc3,0x1b,0x0e,0xf6,0x90,0x62,0xf3,0x39,0x63,0x7d, +0xf3,0x20,0x37,0x85,0x8d,0xc6,0x70,0x8e,0x7f,0xc0,0xd8,0xdb, +0xff,0x28,0xf5,0x5a,0xb7,0xbe,0x7d,0x9a,0x9b,0x26,0x8f,0x7f, +0xfa,0x41,0x6e,0xa4,0xbf,0x07,0x30,0xcb,0x59,0xe2,0x46,0x89, +0x5a,0x4f,0x23,0x0c,0xa7,0xb5,0x2e,0x0d,0x2c,0x2c,0x9f,0xe5, +0xae,0x3f,0x86,0x92,0x7d,0x09,0x93,0xe5,0x82,0xc0,0xa7,0xeb, +0x1b,0xa0,0xb5,0xc5,0xab,0xd2,0xab,0x69,0xbb,0x89,0xaa,0x44, +0x1d,0xc0,0xd0,0x87,0x76,0xa1,0xaa,0x48,0x67,0xb7,0xcd,0xe2, +0xee,0x35,0x79,0x57,0xe5,0x3e,0x2b,0xc0,0x05,0xcc,0xc6,0x21, +0xcb,0xb5,0xa8,0x5c,0x0d,0xae,0xf0,0x7c,0x2f,0x5b,0xf0,0x97, +0xba,0xb3,0xbc,0x27,0x1d,0x15,0x39,0x66,0x69,0xf9,0x40,0x15, +0xd8,0xe6,0xde,0x6f,0x0b,0xa2,0x48,0xcc,0x0e,0xcd,0xc9,0xd7, +0x38,0xdd,0x9a,0xe1,0x3c,0x83,0xe0,0x48,0x72,0xda,0x96,0x63, +0xbf,0x35,0x32,0x16,0x94,0x8c,0xd2,0x4e,0x8b,0x2f,0x46,0x63, +0x5c,0x29,0xc6,0x0e,0x04,0xbf,0x26,0xb9,0x63,0x16,0x2c,0x9f, +0x5b,0x70,0x4c,0x87,0x80,0xd9,0x6e,0x3b,0xea,0x26,0x2d,0x66, +0x05,0xc0,0x12,0xe5,0xea,0x41,0x3e,0xad,0x63,0x59,0x9f,0x31, +0x3b,0x4a,0x6d,0x86,0x87,0xa4,0x3e,0x7a,0xb1,0xbb,0xe6,0x73, +0x44,0x48,0xdf,0x43,0xbb,0x56,0x8a,0x0b,0x0e,0x15,0x2d,0x78, +0xdf,0x7c,0xa7,0xb6,0x0e,0xef,0x36,0xea,0xe1,0x86,0x66,0x3b, +0x9b,0xe9,0xe2,0x02,0x71,0xc9,0x5d,0xa9,0x17,0x1d,0x1c,0x75, +0x72,0x9d,0xad,0xc0,0x91,0x8c,0x54,0x08,0x7e,0x70,0x10,0xf4, +0xc0,0xb4,0x9d,0xe8,0x0c,0xdb,0xe1,0x3a,0x21,0x40,0x13,0x02, +0x4f,0x25,0x3b,0xfc,0xdc,0x6e,0xd8,0x1d,0x90,0xf3,0x3c,0xc6, +0xca,0xf6,0x17,0x76,0x10,0xd6,0xc6,0xa6,0xaa,0x7f,0x9e,0x4f, +0xab,0x85,0x06,0x96,0x91,0xf1,0xd3,0xb5,0x33,0x15,0xb6,0x56, +0x1e,0x90,0xda,0x54,0x57,0xe2,0x28,0xc6,0x54,0x6e,0x92,0x8d, +0xbc,0x0f,0x6d,0x48,0xf3,0x18,0xd3,0x28,0x12,0x71,0xab,0x26, +0xca,0x1f,0x7a,0x14,0x8a,0x67,0x24,0x32,0x9c,0xb5,0xab,0xd3, +0x39,0x39,0x7e,0xf3,0xfe,0xe9,0x4c,0x06,0xf1,0x60,0xc2,0xb6, +0xba,0xdd,0xae,0x27,0x04,0xa1,0x93,0x21,0x4d,0x10,0xa5,0x55, +0x33,0x38,0x36,0xdb,0xd2,0xaa,0x9f,0xb5,0xe4,0x10,0x4f,0x32, +0xc1,0x28,0xa7,0xb3,0x01,0xe9,0xa0,0x61,0xf6,0x48,0x04,0xdb, +0x79,0xe0,0xf5,0x75,0xee,0x2d,0xa6,0x29,0x55,0x9b,0xb1,0x0b, +0x83,0x95,0xd8,0xf6,0x86,0x54,0x56,0x82,0x9d,0xc6,0x34,0x68, +0x4b,0x3b,0x0b,0x2a,0x87,0xb0,0x64,0x79,0x13,0xfd,0x16,0xf6, +0x91,0x37,0x2a,0xe6,0xbd,0x8e,0xc8,0xdd,0xd2,0xd4,0x1f,0x2d, +0x41,0xb3,0x9a,0xf0,0xf7,0x43,0x58,0x14,0x52,0xb7,0xee,0x85, +0x28,0x2c,0x59,0x4f,0xdf,0x94,0xac,0xd6,0x0c,0x2a,0xe2,0x1e, +0xa0,0x09,0x79,0x92,0x18,0xbd,0xa9,0x30,0x75,0xf5,0x5b,0x99, +0x7b,0xdb,0x82,0x8f,0x62,0x2e,0xb4,0x83,0x03,0x2a,0xb5,0xd8, +0x52,0xcc,0xc9,0x76,0xac,0x06,0xa7,0xae,0x76,0xe8,0xf7,0xe9, +0x0a,0x83,0xd4,0xe3,0x04,0x27,0x52,0x99,0x41,0x0a,0x51,0x69, +0x5f,0x1a,0x86,0x63,0xec,0xc9,0xa9,0x26,0x8e,0xbd,0xae,0x7b, +0x90,0x38,0x21,0xa2,0x1d,0xe3,0xaf,0x9d,0x41,0x81,0x12,0x76, +0xbb,0x8b,0x37,0x79,0x10,0x20,0xc4,0x5e,0xcd,0x6e,0x8a,0xc6, +0xea,0x0e,0x18,0x0b,0xd3,0x6a,0xed,0x87,0x75,0xe9,0xa3,0xcd, +0x00,0xaf,0x3b,0x56,0x28,0x62,0x9a,0x4f,0x10,0x7d,0xcb,0xe6, +0xcb,0xcd,0xce,0xe0,0xdd,0xb7,0xeb,0x1c,0x07,0x7d,0x0e,0xe0, +0x18,0x38,0xc6,0xd4,0xf1,0x69,0x82,0xae,0xc9,0xa9,0xc9,0xb0, +0xd8,0xce,0x10,0x6d,0xb8,0x56,0x2e,0x17,0x71,0x55,0x2a,0x60, +0xef,0x0a,0x45,0x8f,0x7f,0xb9,0x90,0x2b,0x19,0x43,0x18,0xdf, +0x37,0x6b,0x16,0x47,0xa9,0x37,0x3b,0xd3,0xd5,0x6a,0xfe,0x9e, +0x81,0x01,0x18,0x6e,0x38,0x40,0x05,0x40,0x42,0x26,0x50,0xfb, +0x5a,0x2a,0x7f,0x85,0x9d,0x66,0x67,0x40,0x96,0x3e,0x54,0xe4, +0x18,0x6f,0x3a,0xcb,0x5f,0x5d,0x89,0xca,0x46,0x94,0x2c,0xbc, +0xb9,0x34,0xaf,0x8b,0x5e,0xd5,0x4b,0x4f,0x4e,0xab,0x11,0x24, +0xc0,0xf1,0x77,0xd8,0x0e,0xf4,0xeb,0xd3,0x82,0x03,0x3b,0x9e, +0xbe,0x1e,0x11,0x2d,0x8c,0x5f,0x4f,0x26,0xf9,0x78,0x02,0x9d, +0x5c,0x4f,0x57,0x4d,0xd4,0xef,0x1b,0xbc,0x17,0xd3,0x8c,0x86, +0xdd,0x08,0x01,0xa4,0x24,0x2c,0xed,0x34,0x63,0xff,0xec,0x10, +0x41,0x70,0x84,0xde,0x8b,0x68,0x4b,0x38,0xdb,0x6d,0x83,0xd2, +0x38,0x38,0x17,0x11,0x6c,0x6e,0x0f,0x48,0xa6,0x5e,0xae,0x37, +0x30,0x96,0x3e,0xfe,0x35,0x6c,0x81,0x43,0x9f,0xf4,0x6b,0x67, +0xc2,0x33,0x85,0x27,0xa8,0xd8,0x0c,0x13,0x5c,0x24,0xb9,0x74, +0x4d,0x88,0x34,0x3b,0x18,0x60,0xe2,0x62,0xb9,0x7a,0xff,0x94, +0xaf,0x3a,0xe8,0x37,0xdf,0xbb,0xa8,0x9e,0x80,0xe2,0x82,0x88, +0x2d,0x67,0x61,0x49,0x83,0xac,0xf3,0xc8,0xa6,0x6c,0x60,0x6f, +0xe9,0x35,0xcf,0xd9,0x33,0x94,0xab,0x42,0x6c,0x0e,0x92,0xf3, +0xe5,0x12,0xe3,0x9d,0x39,0x59,0x86,0x5b,0x24,0x99,0x14,0x8a, +0x89,0xe1,0x68,0xa3,0x9b,0x81,0x76,0xf3,0x20,0x33,0x0d,0x97, +0x03,0x2e,0x6d,0x95,0x37,0xfe,0x7e,0x47,0x39,0x19,0x46,0x2a, +0xe4,0x16,0x51,0xe9,0x26,0x70,0x16,0x45,0x85,0xcc,0x94,0xd3, +0x69,0xe6,0x8f,0x8f,0x2b,0x77,0xf5,0x48,0xbe,0xed,0x78,0x9d, +0xc4,0x8e,0xef,0x82,0x06,0x0f,0xa4,0x6a,0xc2,0xdb,0xae,0xf5, +0x42,0x90,0x1b,0x75,0x40,0xa5,0x0c,0x43,0xae,0x6b,0x10,0x97, +0x1a,0x44,0x54,0x7d,0x10,0x28,0x9f,0xdc,0xf9,0x57,0xef,0x65, +0xec,0x28,0x3b,0xd0,0xc7,0x9e,0x9d,0x7e,0xf5,0x1e,0xc7,0xe1, +0x4d,0x52,0x11,0x5d,0x4f,0x51,0x91,0x2c,0x1b,0xf9,0x65,0x82, +0x0f,0x98,0x04,0x9e,0x46,0xb4,0xe3,0xf5,0x24,0x0a,0xd9,0x06, +0xd6,0x17,0xd9,0x08,0xfe,0xc1,0x25,0x94,0xb7,0x15,0xf3,0xc1, +0xb1,0x85,0x61,0x52,0xfc,0xc1,0x46,0x61,0x4e,0xc8,0x7a,0x99, +0x29,0x87,0xa1,0xcb,0xf1,0x4f,0xbc,0x9f,0xf9,0xf5,0xb9,0x98, +0x73,0xff,0x27,0xd3,0x61,0x13,0xd9,0x3c,0x2c,0x96,0x70,0xb0, +0xb8,0x84,0xa5,0xe0,0xb1,0x1b,0xb2,0x35,0xd6,0x65,0xa7,0xba, +0x0c,0xab,0xd3,0x87,0x6e,0x6c,0xda,0xcb,0xcf,0x32,0xc3,0x08, +0x0f,0x74,0x20,0x2d,0x65,0x25,0x43,0x54,0x26,0xba,0x75,0x62, +0x6c,0xcf,0x5f,0x88,0x31,0x03,0x3e,0x89,0x01,0xff,0x6d,0x0a, +0x62,0xff,0xc0,0x5c,0x2d,0xe7,0xb3,0x5f,0x42,0x8e,0x8d,0x49, +0x78,0x91,0x32,0x9f,0x8d,0x7c,0x8e,0x8d,0x15,0x7a,0xbd,0x3c, +0x90,0x00,0x3a,0x27,0x4d,0x86,0x7f,0x0b,0xe5,0x48,0x3e,0xc2, +0x1f,0x6c,0xd7,0xd7,0x39,0x3e,0x8e,0x5b,0xca,0x5d,0x1c,0x38, +0xea,0x5e,0x2c,0x68,0x55,0x1c,0x23,0x1f,0xa5,0xa6,0xe9,0x97, +0xdf,0x4a,0x36,0x0c,0x5b,0x40,0xc7,0x04,0xec,0x52,0x8c,0x03, +0x9b,0x5d,0x9e,0x9e,0x00,0x2d,0x5a,0xa5,0x55,0x60,0x54,0x60, +0xe5,0x2c,0x09,0x1b,0x33,0xc9,0x3c,0x76,0x05,0x07,0x99,0x57, +0xaf,0xd0,0x7e,0xd4,0x79,0x59,0x89,0xeb,0x98,0xe4,0xa4,0x09, +0xb5,0x9a,0x64,0xe8,0x9c,0x69,0x3f,0xb2,0x1d,0x22,0x5d,0x97, +0xb5,0xc3,0x0f,0x45,0xcc,0x13,0xa7,0x05,0x2f,0x5e,0x6a,0xc3, +0x87,0x08,0x6b,0xb3,0x68,0x42,0xff,0x2a,0xcd,0x6f,0xb7,0xad, +0x2d,0xd9,0xf0,0xc3,0x6d,0xcd,0x88,0xde,0xae,0xad,0xe2,0x11, +0xf3,0x0a,0x48,0xe9,0x76,0xe1,0x1f,0x0e,0x70,0xcc,0xa4,0x44, +0xf1,0x60,0x61,0xa4,0xeb,0xea,0xa2,0xb5,0x6a,0x07,0xb2,0xa7, +0x3f,0xa6,0x9e,0xcf,0x24,0xb9,0x3e,0xe1,0xdd,0xe3,0xb7,0xc8, +0xe2,0xb5,0x7f,0x04,0x65,0x7f,0xdf,0xd4,0xe3,0x88,0xd5,0x5c, +0x94,0x99,0x37,0x78,0x25,0x7c,0xb7,0xc6,0x96,0xc5,0x8a,0x4e, +0x19,0x36,0x0e,0xb4,0x68,0xbc,0xe3,0x61,0x7b,0x13,0xe7,0xa9, +0x95,0xbb,0x8a,0x38,0x38,0x8f,0x39,0x44,0xf0,0x21,0x2d,0x72, +0xaf,0xfb,0x7d,0xbc,0xb6,0x5b,0x3f,0x16,0xb4,0xc7,0xc9,0xbd, +0xa8,0xd1,0xce,0x78,0x0d,0x2d,0x24,0xa1,0x55,0xac,0xe2,0xed, +0xa4,0xdd,0x6e,0xc7,0xd3,0xec,0x59,0x98,0x4d,0xe2,0x95,0xc1, +0xab,0xbc,0xd6,0x52,0x7e,0x43,0x6e,0x3b,0x35,0x09,0x0c,0x4c, +0x3f,0x7e,0xba,0x4c,0x1c,0x0c,0xb1,0x75,0xaa,0xa8,0x71,0xdf, +0x94,0xef,0x7d,0xc7,0x0b,0x3f,0x10,0x36,0x24,0x85,0x91,0x4c, +0x5a,0x81,0x85,0x52,0x44,0x65,0x5e,0x3c,0xc7,0x08,0x9d,0x7b, +0xe3,0x43,0x36,0x23,0x3b,0x8a,0x79,0xac,0xad,0x7f,0x5d,0xbf, +0xca,0xee,0xc8,0x30,0x97,0x14,0x93,0x4f,0x30,0x97,0x12,0x41, +0xfc,0xd0,0x23,0x6d,0x1e,0xb8,0xb5,0x59,0xcd,0x05,0x06,0xcb, +0xe3,0x38,0x49,0x9e,0x7d,0x3e,0x4c,0x69,0x9b,0xa7,0x50,0xe6, +0x51,0xe4,0xb0,0xb9,0xe1,0xdb,0xa3,0x72,0xea,0x35,0xaa,0x27, +0x73,0xed,0xb0,0x40,0x05,0x51,0x74,0x6a,0xdf,0x6e,0x95,0x5b, +0x90,0x0f,0x1c,0x81,0x3c,0x2b,0xbc,0x23,0x33,0x29,0x23,0x08, +0x02,0x53,0x4b,0xb3,0x1d,0xaf,0x8f,0x6e,0x77,0x6c,0xbd,0x77, +0xb9,0xf2,0x68,0x6c,0x43,0xde,0x04,0x9e,0x9e,0x9c,0x8b,0xe7, +0xee,0x49,0x2e,0x1f,0x4a,0xb2,0xe4,0x67,0x68,0xe3,0xf9,0x91, +0xa3,0x93,0x3d,0xfd,0x6b,0xaf,0xb6,0x7b,0xcb,0xe6,0x34,0xc3, +0xf9,0x7d,0x86,0x51,0x6a,0x26,0x3c,0x01,0x33,0x2f,0xf4,0x4e, +0xa6,0xb3,0x82,0xe1,0x3b,0xc3,0x59,0x89,0xb7,0x39,0x2c,0x61, +0x37,0x33,0xfc,0x60,0x0d,0xc5,0xa8,0x91,0x22,0xc1,0x4c,0x99, +0x55,0x70,0x1c,0x54,0x32,0x72,0x6e,0x0d,0xb9,0x2a,0xf7,0x8c, +0x45,0x10,0xa0,0x6a,0xaa,0x81,0x9c,0xbb,0x5d,0x4f,0xb9,0xeb, +0x02,0xa8,0xba,0x28,0xb0,0x9e,0xea,0xd6,0x24,0xff,0x3d,0xc9, +0xa2,0x1c,0x56,0xd1,0x9a,0x64,0xd2,0xc8,0x61,0x95,0xb0,0x49, +0x30,0x26,0xc7,0x08,0xe9,0xd4,0x4a,0x73,0x89,0x0c,0x9b,0x3d, +0x3f,0x32,0x38,0x0d,0x5a,0x73,0x29,0x22,0xe5,0xe4,0xe9,0x82, +0x5a,0x38,0x22,0x94,0x69,0x31,0xc3,0x0e,0x14,0x8a,0xce,0xbf, +0xc7,0x34,0xce,0x4b,0xea,0xdd,0xf5,0x1c,0x6d,0xf1,0x87,0x9f, +0x42,0xdb,0xc8,0x87,0x04,0xc7,0x8f,0x7f,0xfa,0xe1,0x67,0x32, +0x4c,0x1f,0x91,0x99,0x3f,0x82,0x6d,0x93,0x0c,0xb4,0x8d,0x66, +0xf5,0x3c,0x07,0xdf,0xc2,0x59,0x4d,0x38,0x34,0xad,0x30,0x8a, +0x62,0x75,0x1f,0x8a,0x24,0xa8,0xe1,0xc4,0xa2,0x58,0xf9,0x90, +0xb9,0x37,0x34,0x9c,0x64,0xd8,0x68,0x7f,0x5a,0xbf,0x5f,0x5c, +0x14,0x12,0xff,0x96,0x52,0x50,0xad,0x09,0x45,0x52,0x09,0x2a, +0x67,0xb9,0x14,0xb6,0xdb,0x88,0x47,0x0b,0x89,0xdd,0x2e,0xd6, +0x8a,0x1c,0xc0,0x81,0x47,0x62,0x6a,0x6b,0xac,0xaa,0xc0,0xfe, +0xde,0x59,0x7e,0xb4,0xcf,0x03,0xe2,0xda,0x4e,0x03,0x34,0xb9, +0x33,0x18,0xf4,0xdc,0x3f,0x26,0xed,0x8c,0x0b,0x43,0x18,0x4f, +0x0b,0x93,0x59,0x83,0xfc,0x9c,0x78,0x86,0xeb,0x9d,0xd7,0xb6, +0xb7,0x91,0x73,0x5d,0x2d,0x01,0xcd,0x32,0x73,0x95,0x67,0x0e, +0x28,0x77,0x97,0x71,0xa9,0x9d,0xb1,0x2a,0x7b,0x4f,0x69,0x4b, +0x73,0xe3,0x6c,0xe5,0x79,0x07,0x74,0xc1,0x8d,0xe5,0x42,0xc0, +0x24,0xd7,0xf5,0xb1,0x4f,0xbb,0xf6,0x5e,0xc0,0xbb,0x08,0xc8, +0x70,0xc0,0x6c,0xed,0x11,0xea,0x21,0x24,0xe2,0x93,0xef,0x11, +0xee,0x02,0xb8,0x05,0x9f,0xa1,0x17,0x8c,0x84,0xb3,0x08,0x13, +0x63,0x5d,0x03,0xee,0x23,0x91,0xbe,0xc1,0xd9,0xd2,0x55,0xce, +0x3a,0xd5,0x7b,0xc2,0x41,0xe4,0xa1,0x22,0x7e,0xf1,0x01,0x97, +0x03,0xb7,0x40,0xa1,0x5f,0x29,0x53,0x02,0x95,0xca,0x31,0x49, +0x03,0x04,0x15,0xf6,0x78,0xcc,0x1a,0x04,0xf4,0xb7,0xc6,0xdb, +0x5b,0xac,0x6c,0x34,0x68,0x1c,0x30,0x75,0x3a,0x56,0x39,0x17, +0xee,0xca,0x8b,0x6e,0xfc,0x91,0xed,0x14,0xd4,0x0e,0xbb,0x9f, +0xa8,0x2f,0xf9,0x47,0x82,0xa6,0xdb,0x2c,0xb6,0x58,0x19,0xfe, +0xf1,0x3b,0xa1,0x3b,0xd0,0x94,0x07,0xa0,0x0d,0x8d,0x85,0x42, +0x12,0x92,0x70,0x6e,0xaf,0x15,0x55,0x10,0xc1,0xdf,0xdc,0x5a, +0xf2,0xff,0xfc,0x9f,0xff,0xd7,0xff,0xfd,0x7f,0x24,0xd9,0xc8, +0xce,0x64,0x78,0x97,0x48,0x5c,0x9b,0x62,0x1b,0x26,0x49,0x1e, +0xd5,0xa5,0x92,0xbb,0xfc,0x63,0x6a,0x52,0x5e,0x2f,0xf1,0x69, +0x97,0x2e,0x35,0x21,0x05,0x35,0x27,0xa2,0x15,0x74,0x4d,0x81, +0x34,0x6c,0xd8,0x48,0xd6,0xd3,0x9e,0x49,0x02,0xb9,0x8e,0x07, +0xc2,0x2f,0x94,0xa6,0x08,0xe7,0x8e,0x8e,0x84,0x87,0x41,0x46, +0x16,0x05,0x67,0xc3,0xc7,0x28,0x1a,0x5b,0x0f,0x94,0x1b,0x8d, +0xe1,0x9f,0x49,0x8e,0x35,0x9c,0x4a,0x84,0xc7,0x89,0x55,0xa8, +0x25,0x56,0xb2,0x49,0x80,0xdf,0x68,0x71,0x21,0xc4,0x4e,0xb7, +0xc4,0x34,0xbc,0x96,0x7b,0x00,0x77,0x83,0x9c,0xf9,0xaa,0x49, +0x49,0x13,0xf3,0x61,0xa8,0x2f,0xca,0x1f,0x92,0xb2,0x50,0x41, +0x05,0x69,0x76,0xb1,0x14,0xd5,0x08,0xb5,0x53,0x9e,0x43,0x04, +0xe4,0xf4,0xaa,0x2c,0xaf,0x72,0x32,0x5d,0x71,0xca,0x03,0x22, +0x20,0x42,0x09,0xfc,0x03,0x14,0x22,0x26,0x02,0xf6,0x49,0x0a, +0xe1,0xc4,0xc7,0x03,0xc0,0x3c,0xa2,0xc4,0x53,0xf5,0xa1,0xbe, +0xcd,0xd4,0x25,0x88,0x16,0x62,0x5b,0x3e,0x2f,0xf8,0xcb,0x81, +0x41,0x65,0x9c,0xd6,0x8c,0xac,0xc9,0x9b,0x6f,0xa1,0xc8,0xa2, +0x38,0x7d,0x3d,0x7c,0x0d,0x40,0x51,0x1d,0x72,0xdd,0x97,0xe6, +0xc6,0xaf,0x27,0x8e,0xe0,0x87,0x36,0xcd,0x3f,0xa3,0x0f,0xdb, +0x6a,0xa1,0xf3,0xbf,0xf5,0x84,0x74,0x70,0x14,0x15,0xab,0x0a, +0x81,0xb1,0x03,0x79,0x45,0xda,0x57,0xc7,0x96,0xaa,0xc5,0x5b, +0x3f,0xb6,0x36,0xce,0x26,0xc5,0x5c,0x0b,0xe3,0xad,0x59,0x0f, +0x0f,0xb6,0x27,0x79,0x5b,0x74,0x3a,0xf0,0x6f,0xa8,0xa2,0xb1, +0xc1,0xb9,0xb5,0x6d,0xee,0x8a,0xd6,0x25,0x1a,0xdf,0xa2,0x6f, +0x1a,0x97,0x01,0x91,0x05,0xb5,0xb8,0x1a,0x88,0x90,0x23,0x18, +0x3a,0xfd,0x6e,0xa4,0x36,0x8c,0x00,0x06,0x06,0xb4,0x87,0x8d, +0xfa,0x80,0xb6,0x32,0x52,0xd6,0x24,0xcb,0x10,0x87,0xfb,0xb8, +0x15,0x0d,0x26,0x64,0x32,0xde,0x48,0xa8,0x7f,0xc3,0x4b,0x4c, +0xc3,0x6e,0x42,0x83,0x63,0x1c,0x11,0xb7,0x20,0xf6,0xb9,0x31, +0xf7,0xe2,0xce,0xff,0x15,0xed,0xfa,0xcb,0x85,0x0d,0x3a,0x84, +0x45,0xc3,0xbc,0x41,0x1e,0x6a,0xf3,0x6f,0xaa,0x59,0x3e,0x40, +0x8b,0xa2,0x77,0x81,0xd2,0xda,0x99,0x45,0x50,0xfc,0x67,0x60, +0xe5,0x5c,0x86,0x04,0xae,0x30,0xe4,0x40,0xcb,0x29,0x01,0xe5, +0xa9,0xcb,0x85,0x0b,0xe4,0xa8,0xa7,0x02,0x68,0xf7,0x72,0x81, +0x72,0x55,0x9b,0x5d,0xfe,0xe5,0x22,0x1b,0xd1,0xce,0x55,0xc4, +0x37,0x0f,0x56,0x81,0x67,0x1e,0x64,0x0c,0x46,0xd1,0x54,0xdb, +0x5e,0x2e,0x64,0x6c,0xf6,0xe8,0xa1,0x1a,0x71,0x35,0x66,0x49, +0xf7,0xb6,0x9b,0x91,0xe4,0x8e,0x0d,0xd3,0x9b,0x31,0x00,0x67, +0xf0,0x37,0x7a,0x50,0x86,0x4b,0x7a,0x21,0x3a,0x77,0x66,0x4a, +0xfe,0x87,0x31,0x25,0xc2,0x70,0x31,0x0c,0x00,0x93,0x1f,0x48, +0xdd,0xd5,0x02,0x43,0x30,0x19,0x32,0xb6,0xfd,0x77,0xe0,0x8a, +0xeb,0xe9,0xad,0xf8,0xa0,0xec,0x21,0xcd,0xf3,0x9b,0xf9,0x9b, +0x82,0xb7,0x82,0x37,0xfc,0xb6,0x8e,0x17,0xea,0xd6,0x3f,0x90, +0xe3,0x19,0x34,0xbb,0xb3,0x5d,0xa0,0x03,0x8a,0x25,0x27,0xcc, +0xd3,0xe8,0x03,0x04,0xa6,0x83,0xae,0x42,0xf8,0x24,0x56,0x58, +0x00,0x95,0xdd,0xa5,0x75,0xa3,0x0d,0x7c,0xdb,0xbd,0x6e,0x5a, +0xe6,0x91,0x69,0x0f,0x75,0x4d,0xd3,0x5b,0x74,0x31,0xa5,0x51, +0x20,0x8d,0x4e,0x6f,0x47,0x30,0xc9,0x4e,0x17,0x5f,0x8b,0xbf, +0x1e,0xd2,0x04,0xa9,0xd4,0xf3,0x94,0x06,0xcc,0x54,0x12,0xee, +0x0f,0x16,0x8f,0x76,0xbe,0xb1,0x28,0xb7,0xe5,0x5b,0x44,0x4a, +0x21,0xb4,0x60,0x06,0x7a,0x6a,0x59,0xae,0x97,0x0b,0xb7,0x94, +0xb0,0x51,0x84,0x8a,0xea,0x70,0xb8,0x3b,0x0f,0x3a,0x5e,0x6c, +0x51,0x79,0x2f,0xf2,0x99,0x45,0x03,0x19,0x8d,0xd5,0x39,0x02, +0x34,0x0a,0xc6,0x97,0xe5,0xe2,0x62,0x66,0x1b,0x39,0xe1,0x46, +0x72,0x45,0x36,0x0a,0x9b,0xb7,0xcd,0x3b,0x88,0x94,0x0e,0x23, +0x68,0xf8,0x81,0x12,0x3d,0x6a,0xff,0x40,0x64,0xdc,0x85,0xa6, +0x1f,0x7a,0x83,0x54,0x34,0x55,0x37,0x56,0xc9,0xc7,0xfe,0x37, +0xf2,0xd1,0xf4,0x4d,0x34,0x1f,0xb4,0x3b,0xd9,0xab,0x7b,0x54, +0xfa,0xd8,0x6b,0x69,0x92,0xed,0xcd,0xf9,0x68,0x2b,0x93,0x66, +0xc5,0x16,0x13,0x13,0x8e,0x4a,0xe0,0x3a,0xdb,0x7c,0xb2,0x95, +0x49,0x54,0x25,0xb6,0x32,0x21,0xc5,0xc5,0x66,0x49,0xd7,0x21, +0x78,0x7a,0xc4,0x9f,0xbc,0x24,0xa5,0x3e,0x45,0xb7,0xb2,0xc7, +0x30,0x67,0xef,0x12,0x1e,0xd0,0x3c,0x57,0x5f,0x68,0xa1,0xdb, +0x85,0x7f,0xa0,0x08,0x1c,0x85,0x96,0x73,0xd8,0xe5,0x3a,0xce, +0xbd,0x42,0xd2,0x28,0x26,0x2d,0x7b,0x82,0x36,0x54,0xc5,0x02, +0x86,0xad,0x0f,0x88,0x29,0x2f,0x37,0x49,0xc3,0xff,0xda,0x57, +0x1d,0x07,0xed,0x9a,0xaf,0xe0,0x6c,0xbd,0xc7,0xa0,0x68,0xb7, +0x4b,0xed,0xf3,0x0d,0x4e,0x5b,0xac,0xd7,0x95,0xac,0xd2,0xf4, +0x2f,0x41,0x93,0x6f,0x58,0x5b,0x74,0xc4,0x31,0x2b,0x8f,0xf8, +0xa0,0x6c,0x35,0x00,0x47,0xb4,0x6f,0x12,0x31,0x1f,0xfd,0x52, +0xbe,0x7a,0xf2,0x6e,0x75,0xc4,0xd2,0x25,0xeb,0xb2,0x12,0xdf, +0xb1,0xde,0x8b,0x77,0xc5,0x87,0x2d,0x4f,0xbb,0x99,0x8c,0x99, +0xf3,0x1d,0xb9,0x00,0x51,0x93,0xb6,0x83,0x55,0xe6,0x59,0xec, +0x14,0xb1,0xc6,0x9a,0xe6,0xd2,0x7f,0x1b,0x0e,0xc3,0xcc,0xcb, +0x60,0x1e,0xc9,0x4e,0x5b,0x7b,0x0b,0x4a,0x7d,0x83,0xf5,0x26, +0xa9,0xb1,0xe5,0x49,0xc6,0xa8,0xf5,0xbd,0x39,0xf5,0xb0,0x8c, +0x1f,0xaa,0x6b,0xba,0x83,0x6a,0xce,0x50,0x02,0xa8,0xa0,0xc9, +0xe3,0x75,0x79,0xbd,0x5c,0xbf,0x47,0xc9,0x0c,0x08,0x9d,0xd3, +0xbe,0xe7,0x5d,0x81,0x3f,0x9e,0x92,0x73,0x0d,0xff,0x7e,0x46, +0x36,0xca,0xf4,0xd8,0xc5,0x18,0xcd,0xfa,0x61,0x18,0x45,0x47, +0xfd,0x38,0x61,0xef,0x2b,0x51,0x8f,0x66,0x42,0x37,0x14,0x3e, +0x74,0x93,0x11,0x0a,0xf5,0x54,0xd8,0x77,0xf0,0xe8,0x93,0xe3, +0x5f,0x70,0xf7,0xc8,0xe7,0xbd,0x4e,0x0b,0xaf,0x53,0xf4,0x00, +0xf6,0x3e,0x8b,0x93,0x00,0xd4,0xc2,0x7b,0x1b,0x43,0x32,0xc8, +0x37,0x12,0x52,0xbb,0x5d,0xbf,0xe0,0xa9,0xd7,0xfc,0xd0,0xfb, +0xcd,0x22,0x39,0x96,0x1f,0x7b,0xa9,0x13,0xd9,0xe9,0x11,0x42, +0x0d,0x7c,0x80,0x9a,0x3f,0x3a,0xc5,0x75,0xbb,0x3a,0x10,0xf4, +0xc6,0xfc,0x89,0x5f,0xbf,0xc9,0xee,0x64,0x90,0xb0,0x88,0x6d, +0x80,0x11,0x82,0x66,0x60,0x18,0x9a,0x94,0xb0,0x36,0xa2,0x7f, +0xfb,0xfa,0x18,0x21,0x39,0xc4,0x70,0x92,0x44,0x7a,0xc9,0x72, +0x6e,0x68,0x24,0xc8,0xce,0xeb,0x72,0x7e,0xd9,0x97,0xb8,0xf5, +0xa9,0x58,0x62,0x5c,0x16,0x77,0xc0,0xbb,0xf2,0xd0,0x9f,0x1b, +0x2b,0xc8,0xc5,0x2d,0x61,0xca,0x7f,0x36,0xa4,0xe3,0xc7,0x33, +0x93,0xd3,0xbb,0xbf,0xc6,0x48,0xfe,0xf2,0x22,0x4a,0x59,0xc9, +0xd6,0xbf,0x40,0x26,0x41,0x00,0x73,0x86,0xc1,0xbd,0x0a,0x26, +0x2b,0x7d,0xf6,0x6f,0x16,0xd5,0xaf,0x28,0x30,0x12,0xd4,0x57, +0xd3,0x9a,0xca,0x6f,0xb7,0xfc,0x3c,0x09,0x4a,0xd9,0xf8,0x0d, +0xa7,0xb8,0x57,0x78,0xfa,0x79,0x65,0xf1,0xe0,0xf4,0x6a,0xf2, +0x96,0xa3,0x80,0x89,0xbe,0x42,0x9e,0x14,0x25,0x53,0x3c,0xda, +0x47,0x02,0xb9,0x92,0x56,0xea,0x93,0x0c,0x1b,0xd8,0x13,0xaa, +0x39,0x3f,0xcb,0x7c,0x63,0x20,0xbc,0x60,0x43,0x05,0x6c,0xcb, +0x4d,0x3d,0xcf,0x5c,0x84,0xa8,0x9b,0xe8,0x91,0x19,0xc1,0x96, +0x0d,0x42,0x4b,0x24,0x26,0x5e,0x0b,0xca,0x58,0xe5,0xcd,0x19, +0x28,0x48,0xd4,0xc0,0x8e,0x6b,0x59,0x76,0x7a,0x3c,0x18,0x66, +0x04,0xbf,0x06,0xd4,0xa0,0x35,0x37,0xd0,0x81,0xda,0x81,0x08, +0x11,0x17,0x94,0x1f,0x52,0xf6,0xf1,0xb1,0xbf,0x74,0xa2,0x22, +0x94,0x76,0x7c,0x9c,0xed,0xd4,0xf8,0x03,0xa6,0xa4,0xd5,0x74, +0xe4,0x52,0x1d,0x44,0x2c,0xac,0x20,0x24,0x11,0x45,0x01,0x8c, +0x79,0x27,0xed,0xe0,0xef,0xed,0xb6,0xe3,0x21,0x1b,0x75,0x4c, +0x28,0x65,0xec,0xc1,0x1b,0x32,0x0a,0xee,0x53,0x88,0x77,0x5f, +0x39,0xe6,0x26,0xb2,0x7c,0x9c,0x15,0x7b,0x50,0xb7,0xc5,0x8c, +0x82,0x40,0xd9,0xa1,0x4d,0xfb,0x9b,0x96,0x96,0xb9,0x51,0xd7, +0x1a,0x37,0x8f,0x01,0xca,0xfc,0xd5,0x24,0x9d,0x60,0x1b,0xad, +0x5d,0x50,0x2b,0x64,0xe7,0x42,0xd7,0x98,0x9e,0xfd,0x88,0x28, +0xf7,0x03,0xcb,0x1d,0x3a,0x5a,0xe0,0x3f,0x14,0x9f,0x8d,0xbe, +0xc6,0x7e,0x41,0xb6,0x58,0x1f,0xb9,0x9f,0x29,0xad,0x84,0x1a, +0xa4,0x63,0xc6,0x2e,0xf1,0x42,0xd8,0xab,0xa9,0x5b,0x8e,0x49, +0x84,0xd4,0xce,0xfc,0x41,0x57,0x0f,0xb0,0x77,0x22,0x65,0xfa, +0xa9,0x23,0xc0,0x94,0x36,0x2c,0xe0,0x68,0x15,0xfa,0xd8,0x80, +0xc7,0xab,0xda,0x36,0xf6,0x0e,0x65,0xec,0xec,0x8b,0xa6,0xd8, +0x56,0xe3,0xd2,0x5d,0x45,0x3d,0x8f,0xa6,0x28,0xca,0x21,0xb1, +0x8e,0x9b,0x15,0xc6,0xce,0x1d,0x8f,0x13,0xb9,0x09,0xc6,0xf0, +0xa3,0xe8,0x54,0x67,0xe2,0x0d,0x11,0xa5,0xa2,0x8b,0xf2,0x88, +0xe7,0x08,0xe3,0x90,0x4b,0x85,0x59,0x32,0x31,0x58,0x9b,0x0e, +0x24,0x26,0x41,0x4f,0xe2,0x8f,0xaa,0xfc,0x9a,0xa2,0xab,0x51, +0x65,0x0e,0xde,0x00,0x95,0xd5,0xad,0xb9,0xad,0x01,0xad,0x3b, +0xa1,0xad,0x6d,0x53,0x16,0x09,0x46,0xf3,0xa4,0xc0,0xbc,0x2a, +0x03,0x4b,0x20,0x98,0x56,0x42,0xdb,0x94,0x70,0x3e,0x23,0xe7, +0xb9,0x96,0x7c,0xeb,0x34,0xcf,0xde,0xd4,0x16,0xfb,0xe2,0x2b, +0x1d,0xcf,0xc6,0xe6,0xca,0x8f,0x7c,0x20,0x01,0xcf,0x7c,0x13, +0x93,0xe8,0x81,0x59,0x2b,0x6b,0x3b,0x8b,0xb9,0xf2,0x96,0x12, +0x43,0x06,0xcf,0x93,0xe1,0x0b,0x40,0x94,0x22,0xa7,0x6f,0x0e, +0xb3,0x42,0x29,0xb8,0xdd,0x39,0x6b,0xe7,0xe0,0xf4,0x4c,0xca, +0x11,0xe0,0x2c,0xf4,0x63,0xa8,0x03,0x1b,0x73,0xb5,0xc1,0x64, +0x12,0x7b,0x86,0x31,0xa4,0x68,0xbe,0xbe,0xc0,0x5a,0xed,0x66, +0x64,0x43,0x2d,0xe5,0x19,0x95,0xd8,0x3e,0x35,0x4f,0xa5,0xc3, +0x6c,0x14,0xa7,0xa8,0x79,0x9b,0x8e,0x5a,0xad,0x0e,0x04,0xbf, +0x5e,0x32,0x3d,0xdb,0xe8,0xbc,0xde,0x6d,0x0e,0x53,0x48,0x96, +0x6b,0xc2,0x98,0xb1,0x81,0x31,0xa9,0x37,0x57,0xc9,0x84,0xe0, +0x65,0xc7,0x2b,0xec,0x6f,0x64,0xeb,0x59,0x00,0x72,0xb6,0x10, +0x5b,0x8c,0xc6,0x0a,0x1c,0xaa,0x29,0x75,0x7c,0xe8,0xec,0x6a, +0x70,0x06,0x51,0xa4,0xdf,0x65,0xae,0xda,0x4e,0x69,0xeb,0x03, +0x66,0x03,0xa3,0x70,0xc9,0xa1,0x22,0x5f,0xb1,0x91,0xcb,0x8f, +0xdd,0xce,0xc5,0x87,0x71,0x4f,0x10,0x4b,0x66,0x7f,0x55,0xad, +0x4a,0x05,0xbf,0x8f,0xf4,0x65,0x3e,0x9a,0x30,0x88,0x3f,0xf3, +0xfc,0x3e,0x90,0xb5,0x21,0x3e,0x34,0x9c,0xf8,0xc5,0x64,0x28, +0x0d,0x3b,0x2a,0xe0,0xfd,0x18,0xf6,0x6f,0xbf,0x7c,0xb7,0xab, +0xa9,0xa1,0x7f,0x29,0x2e,0x36,0xaf,0xd8,0x8e,0x3b,0xaf,0xc7, +0x83,0x17,0xd5,0x04,0xba,0x54,0x4e,0xad,0xc9,0x0f,0x28,0x11, +0xd9,0x75,0x66,0x22,0xfa,0x3b,0x81,0x9e,0xf7,0xaf,0x3e,0x5b, +0x28,0x9e,0x56,0x2d,0x30,0xd2,0xc9,0x68,0x67,0x92,0x7b,0xdb, +0xe1,0xd1,0x2a,0x8b,0xe5,0x70,0x17,0x8c,0x75,0x99,0x67,0xad, +0xc9,0x87,0x0f,0x58,0x07,0xf0,0x2f,0x9f,0xe0,0x35,0xc7,0xb8, +0x22,0xfa,0x6b,0x67,0x6e,0x03,0x3e,0x50,0xdf,0x60,0xa8,0xe7, +0x6a,0x81,0x27,0x66,0x75,0xba,0x24,0x32,0xa6,0x87,0x6b,0x48, +0x73,0x46,0x9f,0x8f,0x78,0xb7,0xa1,0x08,0x70,0xb8,0x0a,0xec, +0x37,0xea,0x7f,0x24,0x8d,0xab,0xec,0x57,0x81,0x59,0x5b,0xb8, +0xa0,0xbc,0xca,0xd8,0x2e,0xf4,0x0f,0x3e,0xd7,0xa8,0x37,0x7d, +0x1e,0x7c,0x6d,0xcb,0xd8,0xcb,0x76,0x2b,0xd9,0x1a,0x11,0x5b, +0xd2,0x45,0x8f,0x70,0xdb,0xfe,0xc8,0xab,0x94,0x37,0x15,0x0b, +0x37,0xab,0x19,0x86,0x96,0x81,0x1e,0x0a,0x8f,0x74,0x2f,0x74, +0xbc,0xfc,0x3a,0x42,0x33,0x2c,0xaf,0x68,0x78,0xb4,0x1c,0x6a, +0xdf,0x69,0xc2,0xf5,0x35,0x85,0x49,0xc3,0x52,0xf1,0x74,0x30, +0xda,0x8f,0xaa,0xdc,0x73,0x26,0x16,0x26,0xe1,0x4d,0xcb,0xa8, +0x2d,0x84,0x50,0x0c,0x62,0x7e,0x7c,0x6c,0xc7,0xbc,0xdd,0xb6, +0x46,0x64,0x89,0xab,0xfc,0x17,0xd8,0x94,0xad,0x0f,0xf2,0xe9, +0x20,0x13,0xd7,0x5a,0xaf,0x67,0xbe,0x2e,0x26,0x31,0x4e,0x64, +0xb5,0x06,0x89,0xb4,0x14,0x89,0x88,0xa6,0x59,0x22,0x56,0xb2, +0x7b,0xf4,0x01,0xa8,0x6b,0x67,0xe0,0x61,0x19,0x9f,0x91,0xb7, +0xe7,0x28,0x43,0x77,0x13,0x0c,0xf3,0x1a,0x93,0x73,0x50,0x39, +0x13,0x5e,0x1f,0x85,0xa7,0xf1,0x78,0x7d,0xd0,0x56,0x63,0xad, +0x84,0xa8,0xcb,0x82,0x19,0x71,0x57,0x01,0x87,0xa7,0xe8,0x30, +0x80,0xa6,0x11,0xc0,0xc9,0x57,0xb6,0x89,0x5b,0x7e,0xc3,0xfa, +0x96,0x93,0x0d,0xba,0x8b,0x4d,0x0d,0x3d,0x75,0x25,0x46,0xf8, +0xe6,0x52,0x4c,0x4d,0xf0,0x58,0xcf,0x0f,0xce,0xfd,0xc8,0x0f, +0xb1,0x3f,0xe3,0x3a,0x18,0x59,0x0c,0x24,0xe7,0xb7,0xf1,0xd3, +0x93,0x6a,0xca,0x92,0x40,0x5e,0xc2,0x8f,0xee,0x56,0x6f,0xfb, +0xb5,0xff,0xdc,0x6e,0x42,0x1a,0x11,0x6c,0x0e,0x64,0xa6,0x0d, +0xbd,0x36,0xf3,0xd6,0x7b,0x6b,0x2c,0x39,0x3a,0xfa,0x7a,0x5e, +0x2d,0xde,0xdc,0x3f,0xfd,0x7a,0x83,0x8c,0xf9,0xf4,0xeb,0xfb, +0xf2,0x77,0x7a,0x74,0xb5,0x2e,0x2f,0x8b,0xcf,0xef,0x4f,0x3f, +0x3f,0x9d,0x7e,0x7d,0x7f,0x7a,0xfa,0x35,0x81,0xcc,0x07,0xc8, +0xcf,0xe9,0xcd,0xa3,0xf3,0xe5,0x3b,0x8c,0x49,0x8f,0x23,0x2a, +0xb0,0xd9,0x76,0x5b,0x82,0x7b,0xd0,0xe9,0xf4,0x40,0xfe,0x34, +0xc1,0xb7,0x28,0x4d,0x07,0x5a,0x81,0xf3,0xc8,0x14,0xff,0x9b, +0xcf,0xf5,0x48,0x22,0x51,0xfb,0x76,0x43,0xc6,0xd4,0x5e,0x04, +0x70,0x76,0x42,0x21,0x14,0xc4,0x1d,0x22,0x78,0xd7,0x72,0x5f, +0x3d,0x79,0xbd,0x0c,0x5f,0xf5,0xc0,0xd1,0x1d,0x00,0x93,0xdf, +0x26,0x23,0x50,0xa7,0xf2,0x7e,0x8b,0x44,0xe1,0x2f,0x12,0x8c, +0x15,0x35,0x58,0xbd,0x1b,0x52,0x24,0x89,0x1c,0x55,0x6b,0x43, +0xf1,0x41,0xce,0xfb,0x5f,0x25,0x46,0x09,0xe2,0x0e,0x9f,0x9d, +0xf0,0x26,0x27,0x4f,0x36,0xfc,0x5a,0xc0,0xdb,0xbe,0x9d,0x25, +0x60,0xd6,0x53,0x94,0x54,0xff,0x86,0x3e,0x9e,0x35,0x34,0x52, +0xe6,0x5f,0x14,0x5c,0x86,0xae,0xdc,0x68,0x38,0xee,0x7d,0xf1, +0x0d,0xda,0x7d,0xe6,0x9d,0xfd,0x60,0x53,0x01,0x6b,0xc3,0x61, +0x30,0x2e,0xff,0xb3,0x72,0x5d,0x4d,0xe7,0xe8,0xe9,0xd1,0x39, +0x50,0x11,0xa9,0xc2,0xd5,0xa3,0x01,0xe7,0xf7,0x61,0xa0,0xf7, +0xd9,0x0e,0x68,0x1a,0xbe,0xea,0x9c,0x50,0x01,0xc4,0x23,0x52, +0xcd,0x8f,0xcb,0xf5,0x35,0xf5,0x30,0xcb,0x93,0xfb,0xf4,0x72, +0x5b,0x5c,0x1c,0x4b,0xd1,0x6c,0x31,0x96,0xee,0xbf,0x38,0xe9, +0x7f,0x65,0x5b,0x96,0x28,0x13,0x9c,0x97,0xe1,0x73,0x43,0x64, +0x2d,0x09,0xe0,0x7a,0x98,0xa7,0x24,0x43,0x74,0xf8,0xd3,0x22, +0xc7,0x3b,0x43,0x98,0x1f,0x7e,0x96,0x0e,0x89,0xc0,0xbe,0x2a, +0x07,0xbf,0xed,0xb3,0x73,0xa6,0x5c,0xd0,0x23,0x05,0x38,0xf0, +0x3d,0x04,0x01,0xbc,0xf7,0x1a,0x86,0x2d,0x05,0x8d,0x7b,0xd4, +0x2f,0x4f,0xbe,0xce,0x17,0xd3,0xb7,0xb0,0x40,0xe8,0x0f,0x4d, +0xdc,0x9e,0x36,0x20,0x1f,0x1f,0x6a,0xd0,0xe7,0x1d,0xd1,0xe8, +0xd7,0xbd,0x1b,0x68,0x60,0xd5,0xfc,0x00,0xa9,0xf3,0x3c,0x79, +0xf4,0xec,0xd9,0xe0,0xd1,0xf2,0x7a,0x35,0xdd,0x04,0xda,0xf4, +0x0b,0x4a,0xc2,0x32,0x26,0x78,0x12,0x3c,0xef,0x9c,0x18,0xff, +0xdd,0x46,0xfc,0x6e,0x0f,0x51,0x82,0x36,0xcb,0x71,0x50,0x10, +0xb6,0x63,0xe6,0x37,0x3d,0x7e,0x20,0x3f,0xfb,0x5f,0xc8,0x9f, +0x1f,0x1a,0x69,0xbc,0xf9,0x81,0x89,0xab,0xea,0x5d,0x39,0xe7, +0xd7,0x85,0x96,0x0b,0xf4,0x60,0xe7,0x15,0x62,0x1f,0x0d,0xc4, +0xa7,0x5d,0xc2,0xc7,0x24,0xf5,0x51,0x34,0x29,0x17,0x0c,0x5f, +0x9f,0x17,0x94,0xa5,0xa9,0x5a,0x00,0xbf,0x15,0x98,0xa8,0xc7, +0x36,0x19,0x67,0x4d,0x0b,0x91,0x12,0x5d,0x62,0x82,0x20,0xbd, +0x6e,0x82,0x30,0x9b,0xad,0xaf,0xa7,0x17,0x00,0xaf,0x2c,0xe8, +0x3d,0x73,0x24,0xcb,0x59,0x06,0x15,0xb2,0x4c,0x22,0xa2,0x04, +0x6d,0x25,0x8c,0x0b,0x05,0x83,0x8d,0xd1,0x7b,0x5a,0x5c,0xe3, +0x55,0x4b,0x0d,0x6d,0x8d,0x2f,0x62,0x61,0x71,0xb7,0xb6,0x8e, +0x74,0x05,0x8d,0xe3,0x03,0x05,0x4b,0xaf,0x07,0xfa,0xfe,0xab, +0xd6,0xb4,0xfd,0xc8,0x4d,0x74,0x4b,0x3b,0x82,0x52,0x61,0xeb, +0x6d,0x25,0x16,0x8e,0xeb,0xeb,0x0e,0xf3,0xe1,0xc7,0x8b,0x6d, +0xd1,0x80,0x7f,0x52,0xf3,0x0e,0x58,0xcd,0x0a,0x26,0x5c,0xa7, +0x58,0xca,0xd0,0x37,0x11,0x46,0x61,0x9b,0x8c,0x88,0x22,0xf8, +0x42,0xc7,0x22,0xe6,0x6e,0xf6,0x2d,0x4a,0x98,0x6c,0xef,0xae, +0x05,0x6d,0x15,0xc3,0x14,0x52,0x15,0x80,0x1c,0xf7,0x26,0x31, +0xfe,0x29,0xa4,0xe5,0x8d,0x53,0x24,0x08,0xde,0xfb,0x62,0x10, +0x2a,0xbc,0x34,0xc9,0xec,0xe5,0x27,0xd4,0x3e,0xbf,0xae,0x68, +0x59,0xf0,0x05,0x10,0xfe,0xba,0x04,0x2c,0xd5,0x15,0xac,0x82, +0x93,0x5d,0xd6,0xd8,0x61,0xed,0x7e,0x5d,0x00,0x34,0x49,0xaf, +0x62,0x84,0x0b,0x10,0xe3,0xaa,0x97,0x7c,0x73,0x73,0x0e,0x64, +0x5c,0xc3,0xd9,0xc2,0x16,0xc5,0x5b,0xd6,0x5b,0x79,0x69,0x45, +0xc6,0xc4,0xad,0xd5,0x63,0x5b,0x66,0xa2,0x0f,0xa2,0x92,0x92, +0xda,0xbd,0x48,0xf3,0x56,0x78,0x20,0xea,0x3a,0x5e,0xad,0xf1, +0xa5,0xc3,0x47,0xf3,0x6a,0x55,0xe8,0x7b,0x39,0xf4,0x3a,0x51, +0xcb,0x48,0xdb,0x2b,0x25,0x6e,0xba,0x60,0xe3,0x59,0x13,0xc2, +0xe8,0x65,0xa4,0xb0,0x3d,0xd9,0x81,0xda,0xda,0x10,0xa9,0x27, +0xd6,0x15,0xc8,0x0b,0x7e,0xe5,0xda,0x70,0x50,0x8f,0xc7,0xd5, +0x5b,0xb3,0x99,0xd5,0x08,0xd8,0x2f,0x25,0xe0,0xaf,0xd0,0xb0, +0x1e,0xf9,0xc9,0x90,0x4b,0xc0,0x0f,0x7e,0xff,0x06,0x7e,0x04, +0xaf,0xda,0x40,0xf2,0xbb,0xe3,0x9a,0x18,0x54,0xee,0x01,0x35, +0x3c,0xbe,0x5e,0xfe,0x76,0xbc,0x2f,0xef,0xb6,0x3c,0x7f,0x53, +0x6d,0xf6,0x65,0x27,0x06,0xb7,0xc5,0x36,0x67,0x56,0x7f,0x0b, +0xe4,0xad,0x13,0x23,0x63,0xe3,0x2f,0x0e,0xda,0xcc,0x83,0x3a, +0x2c,0x88,0x19,0x5b,0x2e,0x16,0x14,0xec,0x00,0x29,0x28,0x09, +0xfc,0x95,0xa0,0x2a,0x27,0xc3,0x95,0x72,0xda,0xe9,0x39,0x48, +0x9a,0x40,0x09,0x43,0x14,0x2a,0xd0,0xd2,0xe8,0x72,0x93,0x1f, +0xff,0x05,0xfe,0x07,0xf2,0x05,0x23,0xea,0x58,0xc4,0x0d,0x1e, +0x44,0xb0,0x4c,0x6d,0xc7,0x59,0x28,0xfd,0x54,0x6f,0x1b,0x82, +0x9f,0xca,0x7b,0x9b,0x35,0xfc,0x37,0x43,0xb9,0x6f,0x46,0x3f, +0x36,0xfc,0xeb,0x3e,0xa6,0x8b,0x2c,0x98,0xe0,0xcc,0x1d,0x10, +0x8d,0x36,0x33,0x8c,0x71,0x3d,0xc3,0xcb,0xea,0x78,0xc0,0x1f, +0x33,0xc9,0x14,0x98,0xcb,0x17,0x76,0xe9,0xe5,0x76,0x69,0x2f, +0x78,0xbd,0x2b,0xe8,0xc3,0x3d,0x74,0x48,0xe9,0x83,0x46,0x3a, +0xb7,0x6b,0xf9,0xab,0xec,0x6d,0x1c,0xe5,0xed,0x27,0x6a,0xb6, +0x2e,0xbc,0x5e,0xbb,0xdd,0x7d,0xdd,0x46,0x98,0xe3,0xd5,0xd5, +0x98,0x59,0x4b,0x68,0xee,0x95,0xb0,0x06,0x89,0xfa,0x59,0x4d, +0x0a,0xf5,0x72,0x15,0x6f,0x28,0x55,0x0a,0xc2,0xf0,0x67,0xb8, +0x2e,0x98,0x88,0xbe,0x8c,0x08,0xe3,0xbf,0xee,0xa1,0x25,0xc8, +0x70,0xd8,0xb0,0x5b,0x7e,0xf1,0xa5,0xac,0x6d,0xef,0xdd,0x34, +0x5b,0x6a,0xb6,0x2c,0xeb,0x1f,0x97,0x9b,0xa7,0x1c,0x64,0x87, +0xa5,0x86,0xa7,0x8b,0x6f,0x80,0xea,0x18,0x7f,0xa4,0x33,0x20, +0x22,0xe4,0xda,0xcf,0x97,0x2b,0xbd,0x23,0x07,0x4a,0x41,0xd1, +0x06,0x3a,0x9f,0x3d,0xd3,0x67,0x13,0xa5,0xd5,0x40,0xb2,0x28, +0x92,0xc1,0x7f,0x45,0x59,0x2a,0xdd,0x53,0x0f,0x69,0x97,0x4c, +0x97,0x32,0xf4,0x0d,0xc4,0x37,0x50,0x57,0xcd,0x41,0xa8,0xdc, +0x52,0x24,0x80,0x0c,0x64,0x56,0x1f,0xd5,0x1a,0xa3,0x8f,0xaa, +0x40,0xc3,0xf4,0xe5,0x98,0x15,0xe1,0xe4,0x63,0x4e,0x10,0xb4, +0xe2,0x33,0x57,0x31,0x22,0x8c,0x26,0xa9,0x28,0x07,0x6c,0x54, +0xb9,0x76,0x52,0x59,0x11,0xe7,0x71,0x10,0xa3,0xe4,0xc4,0xa7, +0x3d,0x49,0x23,0x4e,0x10,0x53,0xb9,0x27,0xe1,0x15,0x1d,0xcf, +0xe1,0x68,0x2f,0x62,0x1c,0x97,0xf6,0x90,0xed,0x41,0x84,0x37, +0x13,0x62,0x21,0x6f,0x01,0xc0,0xa0,0x76,0x6d,0xaf,0x15,0xcb, +0x46,0xfd,0x81,0x25,0xa3,0x78,0xe8,0x71,0x74,0x2c,0xa2,0x6f, +0x9f,0xec,0x95,0xd6,0x59,0xc4,0x1d,0x62,0x67,0xf9,0x20,0x31, +0x87,0x83,0xf3,0x15,0x5f,0xb4,0x90,0xb3,0xeb,0xdb,0xf2,0x06, +0x0e,0xb9,0xd7,0xe0,0x89,0xf0,0x0d,0x5c,0x0f,0xff,0xe5,0x3c, +0xef,0xc8,0x15,0xe0,0xfc,0x2b,0x1f,0xe7,0xb1,0xa4,0x5d,0x7c, +0xd1,0xd9,0xbf,0xa4,0xf6,0xc5,0x14,0x4c,0x69,0x19,0xf9,0xc1, +0x02,0x01,0xe3,0x94,0xc6,0x97,0x99,0x31,0x7f,0x37,0xde,0x5e, +0x04,0xd4,0x8a,0x0c,0xda,0xd1,0x2e,0xbb,0x1b,0x67,0x74,0x30, +0x97,0x83,0xb3,0x15,0x00,0xf1,0x80,0x3c,0x65,0x91,0xad,0xe0, +0xb8,0x1f,0x0c,0xd9,0x2e,0x65,0x63,0x02,0x7a,0x27,0xac,0xa4, +0x60,0x3d,0x67,0x77,0x18,0x44,0xe8,0xd9,0xe4,0xde,0xd9,0x6e, +0x7b,0x36,0xd6,0xdf,0x13,0x0a,0xfa,0x73,0x7d,0x33,0xdf,0x54, +0x8f,0xa7,0xf5,0x15,0x94,0x1c,0x3f,0x3c,0xfe,0xdf,0x31,0x3c, +0xce,0x30,0xba,0x4c,0xba,0x40,0xe3,0x06,0x74,0x1e,0x15,0x41, +0x26,0x4f,0xb8,0x40,0xd2,0x4b,0xfd,0x18,0x2b,0x3d,0xb2,0xb1, +0x5d,0x63,0x89,0x6b,0x10,0xc5,0xac,0xc5,0xf2,0xfd,0xb3,0xc7, +0xf7,0x5f,0x91,0x14,0xbe,0x58,0x62,0x04,0xba,0xfc,0xae,0xbc, +0x3e,0x87,0x13,0x1d,0x08,0x64,0x6c,0xd9,0x91,0x27,0x17,0xf3, +0xba,0x9a,0xe5,0x8f,0x1f,0xfc,0xb7,0x47,0x8f,0xbf,0xf9,0xf3, +0x93,0xe3,0x87,0x4f,0xfe,0xfc,0xf8,0x78,0x30,0xb8,0xb8,0x3c, +0xfe,0xcb,0x9f,0xbf,0xf9,0xb7,0xe3,0x2f,0xbf,0xfc,0xf2,0xab, +0xaf,0xbe,0xf8,0xea,0xcb,0x13,0xf8,0x5f,0x62,0xf0,0xb6,0xa2, +0x44,0xc1,0x8e,0x2e,0x4d,0xa9,0xc1,0xc0,0x62,0x2c,0x7e,0xb7, +0xd8,0x0f,0xf2,0x65,0x23,0xcd,0xa2,0xb5,0x06,0x66,0xc5,0xaf, +0xe5,0x4f,0xf2,0xb6,0x54,0xd3,0xe9,0x70,0x34,0x8c,0x8e,0x38, +0x7b,0x61,0xaf,0x62,0x2a,0xcd,0x4e,0xe5,0xfc,0xfe,0x40,0xd3, +0x6d,0xc0,0x04,0x2e,0x40,0x15,0x48,0x2e,0x6b,0x79,0x30,0x24, +0x2e,0xa3,0xb7,0xdd,0xcd,0x01,0x85,0xfe,0x07,0xda,0xc6,0x2f, +0x51,0x44,0x3f,0x2a,0xb4,0x33,0x2f,0xff,0x28,0x20,0x86,0x5c, +0x5b,0x5f,0xfe,0x71,0x60,0xb8,0x21,0x34,0x3c,0x5c,0x6d,0xda, +0x66,0x49,0x03,0x89,0xb8,0x30,0xa3,0x83,0x4e,0x23,0xf2,0xe8, +0x5f,0xe2,0x24,0xe7,0x39,0x47,0xbe,0x75,0x44,0x50,0x45,0xe4, +0x93,0x21,0xb3,0xc7,0x99,0xe3,0x03,0x1e,0x1a,0x6a,0x14,0xdd, +0xe1,0xa2,0xdb,0x2d,0xff,0x15,0x47,0x5a,0xaa,0xf8,0xaa,0xa9, +0x84,0xab,0x40,0x4c,0xa2,0x47,0xec,0xb1,0xac,0xa7,0x2f,0xb4, +0xfe,0xfa,0xe9,0x5d,0x38,0x09,0x9e,0x09,0x23,0xdd,0xea,0x61, +0xf0,0x3a,0x79,0x39,0x13,0x49,0x54,0xe2,0x93,0xd8,0xa7,0x5e, +0x68,0x39,0xa3,0x12,0x30,0xf2,0x42,0x24,0x94,0x79,0xb1,0x17, +0x22,0x9f,0x2f,0x1b,0xb5,0xb2,0x35,0x84,0x6b,0xa7,0x25,0xb8, +0x25,0xbb,0xb1,0xe1,0xf0,0x6a,0xf4,0xdf,0x22,0x4b,0x09,0x0e, +0x12,0x48,0x95,0xdd,0x61,0x66,0x58,0x7d,0xcd,0x31,0xf8,0xf4, +0x05,0xf2,0x5e,0x2f,0xa3,0xc8,0xcd,0x1c,0xd9,0xaf,0x9a,0xf4, +0x17,0xfa,0xe8,0xb8,0x86,0x76,0x83,0x0d,0x15,0x3a,0x3a,0xc6, +0xb0,0x9a,0x7b,0x82,0x3c,0x7b,0xef,0x9e,0x7f,0x25,0xcf,0xce, +0x20,0x28,0x11,0x39,0xca,0x23,0xa5,0xd6,0xe7,0x79,0x1f,0xfc, +0x44,0x6d,0x7a,0xb6,0xc2,0x79,0x11,0x53,0xe5,0xd8,0xa5,0x16, +0x70,0xca,0x51,0x4a,0xe8,0x92,0xad,0x11,0x5d,0x93,0xb1,0x48, +0x57,0x0f,0xe4,0xdd,0x69,0x4d,0xb6,0xc4,0x80,0x36,0x0c,0x3a, +0x11,0xda,0xa5,0xaa,0x3f,0x89,0x7b,0xed,0x1f,0xc1,0x1c,0x85, +0x03,0x43,0x5a,0x88,0x27,0x8c,0xcd,0x4b,0xc9,0xd3,0x2f,0x4f, +0x3f,0x1a,0x38,0x6b,0xf0,0xea,0x7c,0xe6,0x68,0x1f,0x60,0x4d, +0x43,0xf3,0xba,0x84,0x73,0xc5,0x71,0xbd,0xb9,0xb8,0x11,0x86, +0x30,0x08,0x48,0x3b,0x0c,0x5e,0x24,0x51,0x0f,0x4d,0xde,0x32, +0xd0,0x35,0xf0,0x2b,0xc5,0x3b,0x0f,0x99,0x07,0xe9,0xe2,0x9c, +0x1f,0x1f,0x15,0xf1,0x9f,0xda,0x1c,0x51,0x04,0xcb,0x22,0x95, +0xe0,0x68,0x18,0xaf,0xb8,0x97,0x50,0xa9,0x84,0xe3,0xa7,0xb7, +0x04,0x70,0xe5,0x88,0x90,0xe2,0x04,0xd9,0xf9,0x95,0x43,0x3d, +0x47,0xe1,0x12,0xa8,0xcb,0xd1,0xa1,0x16,0x9a,0x91,0x76,0xd8, +0x2b,0x2f,0xa7,0x4a,0x6c,0xe4,0xc1,0x29,0x46,0xba,0x18,0xbb, +0xe7,0x9e,0xcc,0xac,0xdc,0x37,0xda,0xec,0x6e,0xa3,0xef,0x56, +0xf1,0x80,0x86,0x76,0xe0,0x0a,0xc9,0xaf,0x2e,0x42,0x33,0x8f, +0x85,0xec,0xa2,0xc4,0x88,0x8a,0xbb,0x57,0x43,0xba,0x85,0x7c, +0xeb,0xeb,0x63,0x87,0x03,0x5a,0x73,0x6b,0x0b,0xb2,0xbf,0x6f, +0xd2,0x52,0x19,0xf7,0xbb,0x1b,0xa2,0x36,0x4d,0xcd,0x31,0x80, +0x90,0x2f,0xe9,0xc5,0xcb,0xb8,0x4f,0x0f,0xba,0xe3,0x63,0x0d, +0x5b,0x7d,0x71,0xb3,0x66,0xeb,0x6d,0x0c,0x16,0x7d,0xf9,0x4e, +0xcd,0xcf,0xba,0x5d,0xae,0xab,0xef,0x68,0xf9,0x1d,0xd8,0x38, +0xe7,0xdc,0x02,0xda,0xec,0x19,0xdf,0x88,0x9a,0x00,0xe7,0xe6, +0x01,0xe9,0x1d,0xaf,0xdb,0x6e,0x57,0x82,0xaa,0xb7,0x84,0xb0, +0x36,0x1e,0x16,0x5a,0xa7,0x43,0x3c,0xbf,0x0b,0x7e,0x98,0xcc, +0x15,0x4e,0x22,0x03,0x8e,0x97,0xe1,0xf2,0x6c,0x8b,0x0e,0x8c, +0x8b,0xf0,0x8e,0x4d,0xad,0x3e,0x60,0xfc,0x12,0x5f,0xb0,0x6b, +0x5b,0x71,0x58,0x5e,0x0f,0xa8,0xc4,0x2e,0xa8,0x46,0x29,0x59, +0x72,0x59,0xfb,0xde,0x13,0x51,0x62,0xb4,0xe4,0x6a,0x8a,0x68, +0x58,0x3c,0xd8,0xf7,0x7a,0x24,0xcf,0x1a,0x6f,0x2c,0x1b,0x7d, +0xb6,0xaf,0xc0,0x29,0x35,0x5c,0x15,0xe7,0x3c,0x66,0x2e,0x5f, +0x73,0xd6,0x28,0x20,0x68,0xdd,0xd6,0x08,0xed,0x39,0x35,0xe8, +0xf3,0x46,0xcc,0xce,0x5b,0x19,0xcd,0x9e,0x05,0x42,0xdc,0xc6, +0x8d,0x66,0xd8,0x42,0xf5,0xb6,0x48,0x66,0x02,0x22,0xf4,0xc9, +0x0e,0x36,0x77,0xaa,0x41,0x2f,0xbd,0x45,0x6b,0xc1,0xd5,0xdf, +0x65,0x2d,0x8b,0x9a,0x09,0xe8,0x23,0x78,0xe4,0xbe,0xf6,0xe6, +0xbe,0xf7,0x1b,0x3e,0x21,0x61,0xc2,0x26,0x2b,0xb7,0x4f,0x5e, +0xbe,0x1b,0xd9,0x5f,0xf8,0xe0,0x46,0x89,0xcf,0x15,0x42,0xfe, +0x64,0xbb,0xc5,0x3f,0xb9,0xad,0xed,0xb1,0x16,0x0e,0x26,0x26, +0x5d,0x23,0xa2,0x3c,0xdb,0x23,0xbb,0x94,0xd8,0x04,0x8c,0x6d, +0xc3,0x0b,0xcf,0x4c,0x9c,0x8a,0x60,0x7a,0x36,0x74,0x0b,0xd2, +0xe7,0x1b,0xa4,0x58,0xd4,0xd2,0xd2,0x00,0x92,0xe0,0xce,0x50, +0xce,0x7f,0x7c,0x08,0x57,0x0e,0x2e,0x81,0x96,0xa2,0x3f,0x35, +0x6c,0x6d,0x08,0x70,0x1b,0x3c,0x81,0x1f,0xc4,0xbb,0xc1,0x57, +0x86,0xf8,0xb2,0xb6,0xc5,0x97,0xa1,0x14,0x4d,0x16,0x5b,0x0c, +0x54,0x41,0x78,0x2a,0xb9,0xef,0xf5,0xc7,0x71,0x7c,0x4c,0x0d, +0xca,0x85,0x71,0xe3,0x89,0x15,0xb2,0x21,0x1d,0xeb,0x2f,0x00, +0x91,0x43,0x21,0xed,0x59,0x27,0x18,0x39,0xc4,0x2d,0x13,0x27, +0xaa,0xc5,0x53,0x33,0xac,0x8e,0x8f,0x87,0x19,0x7a,0x47,0x35, +0xd8,0x88,0xc4,0x27,0x32,0x0d,0x86,0x44,0xef,0xcd,0x75,0xbb, +0xe8,0x8c,0x4e,0x4c,0x86,0x14,0x97,0x00,0x79,0xaf,0x67,0x6c, +0x1a,0x31,0x16,0x35,0xad,0xf2,0x1c,0xe4,0x28,0x21,0x95,0x0b, +0xee,0xd0,0x86,0x7f,0x97,0x89,0xe8,0x3c,0xa3,0x7e,0x0c,0xc6, +0x7c,0xa0,0x1f,0x6b,0x92,0x6e,0x8b,0xfb,0xe3,0xb3,0x0d,0x45, +0x61,0xc5,0x90,0xaa,0xdc,0x60,0x71,0xff,0x6c,0x8d,0x5f,0xa4, +0x3f,0x27,0x05,0x0d,0xc5,0x9d,0xa5,0xc3,0xe7,0x96,0x0f,0xa6, +0x5b,0xbc,0x5f,0x9f,0xae,0xcb,0xe9,0x16,0x24,0xc6,0xcd,0x72, +0xb1,0x65,0xc1,0x0b,0xce,0x98,0x15,0x36,0x5c,0x5d,0xbc,0x71, +0xf5,0xa6,0x5b,0x2c,0xc8,0x59,0x12,0x71,0x82,0x33,0xe4,0xc2, +0x60,0xab,0xf7,0x7c,0xdb,0xe9,0xcd,0x66,0x49,0x9d,0xd2,0x2f, +0x7a,0x82,0x9d,0xdc,0xe8,0x29,0x22,0xe3,0x7a,0x39,0xaf,0x79, +0x12,0xb7,0x7a,0xc5,0xb4,0xe5,0x60,0xde,0xdb,0xf9,0x72,0xb9, +0xda,0xd2,0xd1,0x16,0xce,0x8a,0xdb,0xe5,0x0a,0x92,0xd0,0xa3, +0x61,0xb9,0x98,0xbf,0x87,0x1f,0xbf,0xde,0xa0,0x05,0xe5,0xb6, +0xbe,0x58,0xe2,0x53,0x3e,0x04,0xc6,0x4d,0x5d,0x3e,0xe6,0xd8, +0xcf,0xed,0x90,0x50,0xa9,0xe8,0xda,0xb7,0x08,0xed,0x04,0xfa, +0x51,0xb6,0x14,0x7f,0x4a,0x27,0xf4,0xa8,0x28,0x61,0x6e,0xd8, +0xe4,0xdd,0x28,0x4f,0x7b,0x31,0xf6,0x50,0x08,0x0e,0x65,0xcc, +0x16,0x69,0x54,0x93,0xa0,0xaa,0x71,0x35,0x5a,0x04,0x40,0x2b, +0xf8,0x3d,0x6c,0xf4,0xf2,0x09,0x92,0x1f,0x89,0xb2,0xd4,0x31, +0x9f,0x38,0x79,0x0d,0xaf,0x7e,0x27,0xd4,0x14,0xd0,0xfc,0xe3, +0xa0,0xfe,0xb9,0xd1,0x8b,0xb3,0xc9,0xf3,0x0e,0x18,0xfc,0x52, +0xcd,0x3b,0x3e,0x37,0xc0,0x89,0x0e,0xdb,0x6e,0x1d,0x15,0xf9, +0xd2,0x50,0x10,0x42,0x8a,0x65,0xe5,0x85,0xe9,0x66,0xb1,0xc4, +0xe5,0xf9,0x8f,0xf5,0xe1,0x91,0x76,0x36,0x7b,0x84,0x2b,0x25, +0x8f,0xcf,0x01,0x74,0x5f,0x82,0x39,0x65,0xcd,0x2e,0xc9,0x20, +0x15,0x61,0x80,0xee,0xdf,0x7e,0x33,0xaf,0xd5,0x09,0x35,0xe0, +0x4e,0x00,0xeb,0x05,0x70,0xf6,0xa6,0x77,0x89,0x78,0x77,0xd2, +0x9f,0xf6,0xd7,0xbe,0x83,0x07,0xef,0xda,0xa6,0xed,0xb5,0x7d, +0x30,0x80,0xc2,0x37,0xf6,0x15,0xea,0xd4,0xf3,0x76,0xe3,0x99, +0x90,0x18,0x94,0x6a,0xb9,0x90,0x61,0x7c,0x7f,0xe8,0x53,0x80, +0xe3,0x47,0x32,0x79,0x54,0x05,0x57,0x06,0x96,0x96,0x1c,0x7a, +0x9e,0x3c,0xf0,0xaa,0x76,0x47,0x5c,0xe0,0x71,0x28,0x26,0xb6, +0xbd,0x2f,0x42,0xdf,0x16,0x82,0x11,0x3a,0x05,0xf5,0xc2,0x34, +0x7c,0x53,0xd3,0x73,0x82,0xa7,0x74,0x83,0x69,0x39,0xfe,0xc3, +0x67,0x57,0xf4,0xaa,0x26,0x74,0x17,0x02,0x2f,0x3b,0x3e,0x67, +0xd0,0xab,0x3b,0x96,0xe2,0xcb,0x92,0x58,0x86,0x1a,0xfc,0x5a, +0x1e,0xd3,0xec,0x15,0x2e,0x6d,0x18,0x76,0x1c,0x84,0x54,0x81, +0xa2,0xad,0x2e,0x0a,0xff,0x62,0x7a,0xc0,0x0b,0x8c,0x78,0x25, +0xb8,0x18,0x27,0xff,0x33,0xa9,0xc4,0x1b,0xce,0xff,0xff,0x08, +0xe5,0x43,0x74,0xa2,0x2f,0xd4,0xee,0x21,0x96,0x53,0xa8,0x86, +0xb4,0x84,0x4f,0x91,0xda,0x0e,0x82,0x22,0xa6,0x8d,0x7e,0x08, +0x13,0xa3,0x98,0x8a,0x00,0x9a,0x90,0x90,0xf8,0xbd,0x90,0x36, +0x42,0x62,0x7b,0xdd,0xbf,0x4e,0xe7,0x9e,0xe7,0x8f,0x4f,0x01, +0xa6,0xaa,0xd1,0x13,0xaf,0x68,0x46,0x6f,0xd2,0x8a,0xd1,0xd1, +0xa6,0x41,0x20,0x6d,0x1a,0x91,0x2a,0xa2,0x0c,0x0f,0xbe,0x06, +0x65,0x54,0x11,0x65,0x38,0x88,0xdd,0xaf,0x5d,0xd6,0x2e,0xed, +0x07,0x21,0xb3,0xf9,0x79,0x07,0xfb,0xde,0xa5,0x6d,0x0f,0x17, +0x0b,0xb9,0x57,0xf9,0x20,0x19,0xcf,0x4c,0x19,0xc3,0x12,0xd8, +0xe2,0x35,0xe3,0xfc,0x00,0xe1,0xb9,0xd9,0x71,0x95,0x28,0x62, +0xc2,0x30,0xe3,0x46,0x19,0xa5,0x23,0x76,0x2e,0xe8,0xa8,0xd3, +0x14,0x8f,0xde,0x2d,0x00,0x82,0x69,0x4c,0x85,0x46,0x89,0x32, +0xd8,0x24,0x4f,0xbc,0x55,0x94,0x4c,0xbc,0xf2,0xe4,0xb2,0xca, +0x2a,0x8e,0x96,0xeb,0x16,0x58,0x2d,0x6e,0x06,0x19,0x13,0x1a, +0x40,0xd8,0x36,0x61,0x8f,0x2c,0x2f,0x9d,0x2e,0x28,0x79,0xf9, +0xd2,0xe6,0xbf,0x7c,0x99,0xc4,0xab,0x34,0xfa,0x2e,0xc2,0xcf, +0xed,0xd6,0x85,0x41,0xc1,0x80,0x23,0x1f,0x6c,0x3e,0xa3,0x45, +0x8d,0xfb,0xa0,0x22,0xa4,0x19,0x43,0xdd,0x7b,0x2c,0xd6,0x76, +0x8b,0x0b,0x45,0xf3,0x69,0xad,0x10,0x03,0xf4,0xd9,0xdf,0x70, +0xee,0x98,0x01,0xae,0x7d,0xe1,0x05,0xc1,0x3b,0x54,0xf8,0x7e, +0xb2,0x24,0x7f,0x78,0xc5,0xdb,0x75,0xec,0x90,0x01,0x6b,0x58, +0x35,0xc8,0x27,0x43,0x1b,0x0d,0xce,0x04,0x01,0x8d,0x3c,0xee, +0x8d,0x11,0x4c,0xf8,0x89,0x32,0xb7,0x66,0x02,0xa5,0xed,0xf0, +0x8e,0x83,0xee,0x04,0xac,0xda,0x46,0x0c,0xb1,0x75,0x5a,0xdc, +0x36,0xc4,0x03,0xbe,0x7d,0xe5,0x49,0xec,0x8a,0x26,0xcd,0x0f, +0x15,0x31,0x3e,0x56,0xa0,0x64,0xe1,0x5a,0x1e,0x35,0x57,0x27, +0xd1,0x2f,0x24,0xa3,0x17,0xa3,0xbc,0xd2,0x47,0x7e,0xcc,0xf0, +0x1b,0x4b,0x17,0x30,0xef,0x8d,0x50,0x24,0x92,0xd7,0x2b,0x1c, +0x51,0x78,0x4f,0xeb,0x65,0xd2,0xad,0xd5,0x9a,0xad,0xf0,0x73, +0x9f,0x66,0xd4,0xf6,0x06,0x8c,0x10,0x9a,0xa3,0x1f,0x3d,0x60, +0x80,0x59,0xa4,0xc1,0x82,0x0c,0x7e,0x3a,0x8f,0xe3,0x48,0xe3, +0x93,0xea,0x56,0xef,0x12,0xe6,0xed,0xd1,0xe7,0xdb,0xd7,0xf6, +0xe0,0x60,0x6b,0x9f,0xda,0x53,0x3d,0x11,0xa4,0xa5,0x6a,0x54, +0x6e,0xd4,0xd4,0xab,0x13,0x84,0xf9,0xe3,0xd5,0xc6,0x2b,0x02, +0x47,0x29,0x3b,0x9c,0x1f,0x06,0xa6,0x1d,0x60,0xe2,0xf9,0x7b, +0x00,0x3e,0x74,0x01,0x61,0x01,0x7e,0x15,0x00,0x8c,0x0e,0x36, +0xf2,0xb2,0x1c,0x64,0x88,0xba,0x5b,0x20,0x0e,0x40,0x46,0xbf, +0x9b,0x9c,0x4a,0x53,0x2f,0x3c,0xb9,0x0d,0x41,0x01,0xf2,0xb1, +0xa0,0x5b,0x23,0x3c,0x16,0xbc,0x93,0x93,0xf8,0x72,0x58,0x02, +0x66,0x86,0x42,0x83,0xb4,0xe8,0x72,0x75,0x2c,0xb9,0x38,0x44, +0xe7,0x77,0x41,0x30,0x70,0xbe,0xd0,0x11,0xa2,0x8d,0x2f,0x0f, +0x18,0x28,0x5d,0x6c,0xf0,0x41,0x2c,0x87,0x1f,0x11,0xbd,0xac, +0x44,0x4b,0xce,0x85,0x72,0x46,0x23,0xf4,0xba,0xd3,0xc0,0xe6, +0xad,0x3d,0x29,0x77,0xb1,0x56,0xa2,0xb8,0x2c,0xd5,0x55,0x9b, +0xda,0xd0,0x40,0xc8,0xec,0xed,0x49,0x49,0x7a,0x94,0x63,0xf7, +0x69,0x34,0x67,0x13,0x6b,0xe3,0x63,0x79,0x50,0xce,0xce,0xe1, +0x76,0x7b,0xc2,0xaf,0x59,0xa9,0x03,0x01,0x14,0x18,0x91,0x76, +0x7e,0x3c,0x31,0xd7,0xd3,0x77,0xf4,0xcd,0x0f,0x5d,0x0d,0xec, +0xa3,0xb9,0x36,0xd0,0x8f,0xd4,0x1d,0x41,0xc1,0xdc,0x16,0x24, +0xc3,0x99,0x77,0x8e,0xbf,0x71,0x2d,0x1b,0x68,0x18,0x03,0x71, +0xa4,0xe2,0xb7,0x6d,0x01,0xed,0x76,0xab,0x4e,0xc1,0x6e,0xa3, +0x40,0x99,0xd1,0x09,0xd2,0xb3,0xb2,0x14,0x1f,0x5f,0x6b,0x67, +0x99,0xb3,0x8b,0x92,0x34,0x1e,0xdd,0x60,0x69,0x21,0xa0,0xa3, +0xed,0x56,0x4a,0xb8,0xf0,0xfe,0xb6,0x0d,0xef,0x12,0x8d,0xe9, +0x36,0x6d,0x94,0x35,0x68,0x74,0x8d,0xf6,0x63,0x2b,0xba,0x3c, +0xb2,0x51,0xb7,0x84,0x5d,0x71,0xf9,0x8c,0xd9,0x0e,0xa2,0x3b, +0xf3,0xaf,0x49,0x86,0x8c,0x59,0xd6,0xab,0xcb,0x45,0x86,0x9f, +0x8f,0xe1,0xd0,0xe3,0x79,0xf7,0xef,0xd0,0x64,0x62,0x1a,0x4a, +0xfb,0x30,0xfc,0x8d,0x1f,0x1f,0x84,0x1f,0x24,0xb0,0x86,0xe2, +0xcd,0xe3,0xa0,0x0b,0xd4,0xef,0x02,0x48,0xaa,0xcf,0x6d,0x20, +0x09,0xf1,0x88,0xc4,0xe7,0x03,0x36,0x94,0x9d,0x7e,0x58,0x49, +0x3d,0x6d,0x12,0x5c,0x71,0x3c,0xd0,0x62,0xb0,0xc0,0x4c,0x78, +0xd0,0x77,0x57,0x5e,0xde,0x10,0x79,0xdf,0x01,0xb9,0x05,0xa3, +0x1d,0xe2,0x46,0xb4,0x40,0x76,0x1f,0x4a,0xc4,0xca,0x9c,0xba, +0x5d,0xb4,0x55,0x58,0xf0,0x7e,0xf0,0x6f,0xee,0xe7,0x03,0xfd, +0x69,0x8f,0x00,0xcc,0x10,0x1a,0x57,0x9b,0x6d,0xf2,0xc8,0xc8, +0x3b,0x51,0x37,0x40,0xcc,0x53,0x86,0x8c,0x4c,0x87,0x16,0x1c, +0x08,0xd6,0x85,0xab,0xc0,0xc0,0x87,0xcb,0x0b,0xb9,0x8d,0xe4, +0x82,0xc0,0xd4,0xe8,0x98,0xde,0x8c,0xdc,0x10,0xee,0x01,0x88, +0x19,0xef,0xfd,0x54,0x8c,0x26,0x23,0x52,0x11,0x9b,0x95,0xd3, +0xc1,0x7f,0xa4,0x4a,0xab,0x5c,0xd5,0x58,0x99,0x8b,0xe7,0xe6, +0x06,0x20,0xdc,0x54,0x01,0x88,0x98,0xac,0x2c,0x93,0x36,0x5e, +0x4b,0x9d,0x64,0xcc,0x5c,0xf7,0xa1,0x6c,0x8f,0xc5,0x8c,0xe5, +0xc5,0xc1,0xba,0xe3,0x06,0x8d,0x63,0xb6,0xb6,0x0a,0xf1,0x5c, +0x5d,0xb2,0xbc,0x45,0xc6,0x60,0xd7,0xfb,0xf6,0x86,0x5a,0xe1, +0x7d,0x6b,0x9f,0xde,0x6d,0xdb,0x1e,0x84,0x1a,0x23,0x70,0x8c, +0x6e,0xc3,0x99,0x9d,0xd2,0xa6,0xa2,0xc7,0x61,0x43,0x6f,0x19, +0xf7,0x68,0x91,0xe2,0x05,0x4a,0x3d,0x20,0xe5,0x58,0xf9,0x1d, +0x27,0xd6,0x93,0xce,0xe5,0xdc,0xda,0x2e,0xa4,0x53,0x18,0x38, +0x2d,0xdf,0xed,0x36,0x4e,0x82,0x7c,0x54,0xb3,0x57,0xd0,0xbe, +0xfc,0xae,0x7d,0x1e,0xd2,0x07,0xb5,0x51,0x54,0x27,0x52,0xdf, +0x75,0xbb,0x9e,0x52,0xd0,0x2f,0xe8,0x1b,0x88,0xb8,0xdb,0xec, +0x44,0x9e,0x8e,0x3b,0xe6,0x28,0x25,0x19,0xbf,0x6d,0x32,0x56, +0x68,0x26,0x20,0x5e,0xe7,0xcd,0x14,0x8f,0xe6,0xbd,0x15,0x86, +0x53,0x42,0xc3,0x75,0x88,0xe6,0x69,0x8b,0x40,0x1c,0x61,0xe9, +0x5c,0x1b,0xcc,0x98,0xab,0xc8,0x3e,0x4d,0xee,0x11,0x77,0x7b, +0x79,0xa8,0x17,0x5c,0xa6,0x69,0xb8,0x0e,0x24,0xc7,0x56,0xed, +0xfa,0x2a,0x67,0x73,0x37,0x60,0x89,0x44,0x8c,0xef,0xa3,0xad, +0xdf,0xdf,0xef,0x8f,0x9a,0xb4,0x27,0x86,0xf3,0x22,0xff,0xbe, +0xc5,0x50,0x6f,0xa9,0xab,0x47,0x62,0x17,0x67,0xee,0x90,0x51, +0xca,0xf4,0xc1,0x80,0xa6,0xe7,0xbc,0x93,0x26,0xf0,0x8b,0x58, +0x6a,0x62,0x54,0xa7,0x9b,0x53,0xe8,0xf1,0x9f,0xe0,0x17,0x7a, +0xa3,0x2f,0xd7,0x20,0xce,0xa2,0xc3,0xc7,0xb7,0xf0,0xcb,0xb0, +0xcd,0x46,0x92,0xfb,0x0e,0x54,0xb0,0x15,0x8b,0x97,0x63,0x02, +0x3f,0xf9,0x52,0x31,0x31,0x17,0xe5,0x7c,0x8e,0x9e,0x3a,0x68, +0xb8,0x96,0xe0,0xc7,0x33,0xfe,0xe0,0x1c,0x35,0x69,0xa3,0x9c, +0x9f,0xe5,0x29,0x43,0xb3,0x5e,0xde,0x42,0x95,0x05,0x00,0xb0, +0xbc,0x85,0xe2,0x0b,0x8c,0x1f,0x34,0xe7,0x14,0xf8,0xc1,0x29, +0x40,0x46,0x18,0xac,0x2e,0x81,0xbf,0x3f,0x4c,0x57,0x89,0xa1, +0xb0,0x41,0x62,0x0c,0x9a,0xd0,0xc7,0x37,0xf4,0x91,0x18,0x31, +0x6d,0x2e,0x67,0x15,0x19,0xec,0xe6,0x6a,0xbb,0xfd,0x44,0x12, +0x92,0x58,0xd7,0xda,0xba,0x71,0xb8,0x43,0x8b,0x6c,0x1e,0xff, +0x8a,0x8d,0xe3,0x77,0x72,0xfb,0x03,0x8b,0x30,0xe0,0xfa,0x58, +0xc0,0xe3,0xfa,0x2e,0xce,0x67,0x83,0x99,0xff,0x51,0x6e,0x48, +0x8b,0x90,0x55,0xbe,0x2c,0x73,0xb6,0x4b,0xe0,0x1f,0xb7,0x39, +0xb8,0xc6,0x78,0x66,0x74,0xf1,0x09,0x85,0xee,0x15,0x93,0xad, +0x74,0x8c,0x82,0x54,0x73,0xbb,0x20,0x63,0xff,0x44,0x29,0x3e, +0xb1,0xf2,0x4c,0x50,0xad,0xdb,0x0d,0x3e,0x3d,0x81,0x9a,0xec, +0x62,0x9e,0x2e,0x36,0x69,0x58,0x80,0xd1,0x32,0x38,0xc9,0x72, +0x77,0x91,0xc3,0x8c,0x2d,0x38,0xa1,0x80,0x5c,0xe8,0x2e,0x6c, +0xda,0xf2,0xc5,0x32,0x0a,0x5d,0xbf,0x46,0x27,0x5e,0x2c,0x3c, +0x3a,0x38,0xe8,0xc6,0x5c,0x34,0x87,0x2e,0xa6,0x63,0xfa,0x36, +0x6d,0xd1,0x2a,0x5e,0xf0,0xdb,0x9e,0x2d,0xba,0x2c,0x2c,0xd6, +0x66,0x93,0xc5,0x95,0x30,0xa4,0x55,0xd5,0xa6,0x02,0xa3,0x77, +0x59,0xbd,0x9b,0x98,0x6e,0xb7,0xc1,0x44,0x39,0x14,0x2a,0xee, +0x66,0x7f,0x8c,0xe3,0xa3,0x5f,0x1b,0xb6,0xd2,0x3e,0x9d,0x54, +0xc6,0x7a,0x7f,0x10,0xb8,0xdd,0x2e,0xff,0xed,0xbf,0xb5,0x41, +0x6d,0x47,0x4d,0xf1,0xc8,0x8f,0x36,0xb8,0x87,0x9f,0x87,0x46, +0x79,0x9e,0x42,0xe7,0xd0,0x86,0x9e,0x1f,0xc2,0x0a,0x2c,0xef, +0x03,0xc8,0x08,0x19,0x7a,0x73,0xe7,0x3c,0xb0,0xea,0xb9,0xeb, +0x4f,0xda,0x43,0x79,0xb1,0xa2,0xd3,0x1c,0xfc,0xd8,0xed,0xcc, +0x61,0xb8,0xd3,0x58,0x8e,0x94,0x7d,0xe5,0x20,0x3d,0xb6,0x0a, +0x6d,0xcd,0x69,0x3b,0xb8,0x0b,0x86,0x2f,0xc8,0x12,0x6b,0x59, +0xe3,0x5b,0x39,0x78,0xf0,0x76,0x6b,0x13,0xbf,0x98,0xef,0x7c, +0xea,0xb4,0x1e,0xee,0x3c,0x6d,0xf4,0xae,0x0f,0x6a,0xb3,0xd8, +0x66,0x05,0x65,0x94,0x2b,0xf9,0x57,0x2b,0xc3,0xc4,0x35,0xdc, +0xc4,0xa9,0x56,0x89,0xf5,0x1e,0x7d,0xbe,0xec,0xfd,0x97,0x20, +0x97,0x90,0x95,0x26,0x15,0xbd,0x91,0xb9,0x20,0xd5,0x24,0x7b, +0xbe,0xb9,0xcf,0x8b,0x25,0x6c,0x94,0xb5,0x26,0x8c,0x92,0x84, +0xc3,0xc5,0x0a,0x42,0x03,0x44,0x67,0xbf,0x07,0xd3,0x9f,0x06, +0xad,0xbc,0x10,0x5c,0xc7,0xe5,0x6c,0x0b,0xc1,0x2b,0x82,0xe2, +0x6f,0xd0,0x38,0x12,0x58,0x30,0x79,0xbe,0x50,0xdf,0xa6,0xaa, +0x1e,0x3b,0x70,0x5d,0xd1,0x7b,0xb8,0xdf,0x28,0x1e,0xe6,0xce, +0x34,0x16,0x41,0x24,0x5a,0xf0,0x8c,0x59,0x4a,0xc0,0x67,0x9c, +0x0e,0xa3,0xa6,0x49,0x34,0x89,0x15,0x0f,0x47,0x20,0xc9,0x06, +0x24,0x14,0x04,0x32,0x19,0xcb,0xab,0xd9,0x46,0x5f,0xcb,0x9e, +0x34,0xc3,0xfd,0xb5,0x9f,0xfd,0xa2,0xf7,0x89,0xda,0x0b,0x99, +0xfd,0x42,0xae,0x98,0x38,0x39,0x38,0xf7,0x9d,0x84,0x12,0xb4, +0x2e,0x70,0xe7,0x20,0xb1,0xea,0xdb,0x51,0x30,0xd1,0x48,0x4a, +0x0e,0xdd,0xa2,0x1d,0xbb,0x91,0xa1,0x92,0x3f,0xb4,0x49,0xea, +0xf5,0x05,0xfc,0xfb,0x3f,0x79,0xdc,0x7b,0x64,0x8b,0xfd,0x07, +0x50,0xf3,0x20,0x1b,0x06,0x1a,0xd9,0xc6,0x41,0x94,0xc6,0x6c, +0xf6,0x0e,0x69,0xff,0x08,0x22,0xf9,0xad,0x85,0x21,0x84,0x01, +0xf4,0x9b,0xb0,0x7d,0x49,0x8a,0xc7,0x06,0xbe,0xc9,0x93,0xa2, +0x8d,0xab,0x53,0xc6,0x07,0xfb,0x09,0x5c,0x55,0xb6,0xdb,0x0f, +0x73,0x82,0x03,0xd5,0x0b,0xab,0xb7,0x6e,0xc4,0x88,0xe8,0x7b, +0x9e,0xea,0x0e,0x58,0x8b,0x94,0x86,0x5e,0x29,0x9c,0xd9,0x66, +0xb9,0xbd,0x93,0xcb,0xea,0x37,0x9e,0x5f,0xa7,0x8a,0xf3,0x1e, +0x4a,0x61,0xff,0x5a,0xfe,0x11,0xa9,0x3c,0x25,0xd1,0x55,0xeb, +0x76,0x1b,0x49,0x61,0x15,0x56,0x58,0xb4,0x4d,0x8b,0xf8,0xd5, +0x87,0x63,0x85,0xbd,0x5e,0x33,0x8a,0x04,0x7e,0x2c,0xe9,0x8c, +0xd4,0xa8,0x2b,0xbe,0xfe,0xee,0x1d,0x0a,0xa6,0x33,0x3e,0x72, +0x9a,0x44,0x43,0x52,0xf8,0xd4,0x66,0xe9,0x2c,0xb8,0x05,0x38, +0x48,0x66,0xf2,0x6e,0x4a,0x93,0xda,0x54,0x8f,0x3e,0x42,0x37, +0xdf,0xdc,0x9d,0x3c,0x03,0xbd,0xf7,0xef,0x07,0xa9,0x75,0x86, +0xc3,0x32,0x1f,0x64,0x59,0x47,0xcd,0x0b,0x97,0x1b,0x15,0xbd, +0xd4,0x6f,0xbf,0x5d,0x43,0xc9,0xaa,0x4e,0x4f,0x43,0x89,0x0a, +0x4a,0x9f,0xa1,0xb1,0x6f,0x10,0x46,0x47,0x78,0x02,0x45,0xeb, +0x03,0x86,0x5c,0x6c,0x18,0xf5,0xa6,0x7c,0xcf,0x8e,0xd7,0xf7, +0x5f,0xc0,0x4f,0x74,0x18,0x5a,0x82,0x5c,0xa8,0x49,0x50,0x97, +0xbe,0xf5,0x85,0x5c,0xd8,0xe8,0x6e,0xb2,0x2d,0x9d,0x1e,0xee, +0x8b,0xb9,0xd8,0x0f,0xcb,0xf5,0xea,0x8a,0x8b,0x8a,0xfb,0x35, +0x1b,0x74,0xd1,0xbf,0xcb,0x9b,0xcd,0xf9,0xfc,0x66,0x4d,0x9e, +0x48,0x48,0x36,0xc8,0x09,0x28,0x58,0x06,0x56,0x18,0xbf,0xe8, +0x4f,0xee,0xd1,0x1b,0xe3,0xfd,0xb4,0xdf,0xcb,0xf0,0x91,0x72, +0xeb,0x99,0x44,0xfd,0xdf,0xf1,0x73,0x24,0xe8,0x99,0x14,0x44, +0xdd,0x2c,0xd5,0x94,0xb6,0x36,0x57,0xd3,0xc5,0x6c,0x5e,0xae, +0xd9,0xb3,0xc5,0x5d,0x4f,0xaa,0x69,0x21,0xb5,0x53,0x9b,0x8d, +0x94,0xfb,0xe9,0xfc,0xf5,0xd3,0x85,0x21,0x19,0x62,0x3a,0xe7, +0xcc,0xef,0x28,0xc3,0xe5,0x6b,0x8b,0x62,0x8a,0x6a,0x01,0xae, +0xcd,0x72,0x5d,0xbd,0xa2,0x58,0x1e,0xd8,0xfd,0x63,0xcf,0x31, +0xc3,0x99,0xfa,0xd9,0xcb,0xa4,0xc7,0x36,0x04,0xab,0x34,0xd7, +0x97,0xbf,0xb0,0x76,0x3d,0x50,0x0a,0x05,0x5f,0xfe,0x16,0x5e, +0x9e,0xd6,0xb0,0xa3,0x0a,0x32,0xdd,0x2b,0xcf,0xda,0x01,0x87, +0x33,0x4f,0xfd,0x4f,0x05,0x90,0x83,0x9b,0x67,0x86,0x5d,0xe0, +0xf9,0x06,0x05,0x21,0x64,0x34,0xd7,0xe8,0xc2,0xd1,0x9e,0x43, +0x2f,0x34,0xa6,0x1e,0x9e,0x5c,0x01,0xee,0xc7,0x56,0x6d,0xcf, +0xf6,0x42,0x7d,0x3b,0x0b,0x34,0x3e,0xb7,0x31,0x68,0xed,0xd7, +0xe4,0xee,0x84,0x21,0x4f,0xa9,0xf3,0x6b,0x74,0xb0,0x28,0x60, +0xbd,0xd3,0x0d,0x8e,0xb7,0xab,0x05,0x25,0xd1,0x6f,0x10,0xf5, +0x8d,0x12,0xf2,0xce,0x03,0xad,0x4f,0x44,0xe3,0x85,0x87,0x33, +0x71,0x26,0x3f,0x06,0xc1,0x74,0xc5,0x1e,0x0a,0xf5,0x61,0xfb, +0x96,0x04,0xd8,0xc5,0x86,0xce,0xa1,0xf6,0x86,0x7b,0xa3,0x56, +0xa0,0x21,0xb5,0xf3,0x63,0x58,0x54,0x72,0xbc,0x99,0x90,0x89, +0x02,0x1b,0x90,0x2a,0x4d,0x61,0xe4,0x7c,0x7c,0xb9,0xda,0x91, +0x5b,0x81,0xa1,0xf6,0xc7,0x0f,0x26,0x0c,0x82,0x84,0x52,0xee, +0xe3,0x4f,0x60,0xb1,0x68,0xa5,0xcf,0x44,0x5c,0x04,0xc3,0x97, +0xc4,0xb1,0x5c,0x54,0xde,0xf1,0x13,0x75,0x85,0xbd,0xb7,0x1f, +0x49,0x01,0x0a,0xc2,0xf1,0x0a,0xa4,0x55,0xec,0x3b,0xd7,0x44, +0xd4,0x46,0x90,0x4a,0x68,0xbb,0x25,0xda,0xff,0xc8,0x2e,0x2c, +0x59,0x46,0xac,0x91,0x55,0x94,0x6c,0xee,0x2b,0xc3,0xcc,0xed, +0x1a,0x22,0x07,0x2a,0x5a,0xb4,0x42,0xb0,0xb9,0x92,0x3b,0xbd, +0x5b,0xe0,0x53,0xb1,0x7b,0x01,0xd9,0xbd,0x7c,0x8e,0xe6,0xd2, +0x12,0xf3,0xc9,0xa6,0x3a,0xa2,0xc1,0xb7,0xd0,0x69,0xd2,0xfa, +0x7e,0x41,0x3e,0xe4,0xba,0x85,0x63,0x71,0x9d,0x3b,0xac,0xf7, +0x5f,0x2f,0xab,0x05,0xe1,0x79,0xe7,0xb3,0x0c,0x58,0x04,0xca, +0x16,0x38,0x36,0x44,0xcd,0xe3,0xcf,0xdc,0x82,0x0b,0x33,0x30, +0x06,0xab,0x66,0x58,0x64,0x3f,0x22,0x3b,0xe7,0x13,0xc5,0x2b, +0xca,0xa9,0x37,0xab,0x6e,0x37,0xf8,0xf4,0xdc,0x23,0x08,0x3b, +0x1e,0xff,0xf1,0xe8,0x35,0x23,0x5d,0x82,0xde,0x43,0xc5,0xd1, +0xd4,0x47,0xad,0xa9,0x6c,0x78,0xed,0xf3,0xbc,0xce,0x80,0x0f, +0xea,0x61,0xec,0x8e,0x38,0x25,0xa5,0x68,0x19,0x71,0xe5,0x2c, +0xb3,0x24,0x88,0x3d,0xa1,0xd7,0xb8,0xfb,0xf2,0x06,0x61,0x91, +0x98,0xb9,0x9f,0xfd,0x56,0x96,0x15,0x67,0x14,0xfe,0x47,0x96, +0x59,0x2a,0x18,0x59,0xbc,0x4a,0xe8,0xde,0x76,0x3c,0xf7,0x7a, +0xe6,0xc4,0xeb,0x3d,0xb7,0xa5,0xe8,0x46,0xd2,0x03,0x2b,0xa0, +0x6e,0xde,0x70,0x64,0x0e,0x3b,0x27,0x64,0xea,0xc5,0x01,0x24, +0x9b,0xc1,0x8a,0x5b,0x76,0x21,0x4b,0xa0,0xd7,0xe8,0x91,0x4e, +0x8b,0x49,0x0c,0xf2,0x5f,0x7b,0x3b,0x0c,0x6e,0x4c,0xb8,0x10, +0x08,0x4e,0xb3,0xd1,0x4d,0x4a,0xb7,0xa5,0x4f,0xdf,0x80,0xc4, +0x4d,0x96,0xb7,0xa0,0xc8,0x82,0xa8,0xb1,0x2d,0xa1,0x36,0xbe, +0x7d,0x1b,0xe0,0x0d,0xeb,0x8f,0xf3,0x3f,0xf4,0x62,0xfc,0x5f, +0xc1,0x02,0xd9,0x05,0x01,0x61,0xfe,0x5f,0xc0,0x0c,0x5b,0x97, +0x39,0x8f,0x02,0xc6,0xca,0x80,0x76,0xbb,0x18,0xb9,0x8f,0x63, +0xe1,0xa7,0x49,0xfa,0x62,0x7b,0x76,0xd6,0xcf,0x58,0xd7,0x15, +0xb0,0x17,0x48,0x06,0xa9,0xa7,0x7f,0x0f,0xfe,0x6e,0x33,0xf4, +0x80,0x4b,0xf1,0xd7,0x67,0x19,0x06,0xda,0x52,0xba,0x28,0x5e, +0x17,0x96,0x64,0x05,0xb9,0xaf,0x11,0xb9,0x8e,0xdb,0x6a,0xf6, +0xf8,0xf5,0xc4,0x74,0x3c,0x82,0xeb,0x76,0x15,0xa5,0xc0,0x25, +0xdc,0xda,0xd2,0xc4,0xed,0xd6,0x8a,0x22,0xfe,0x0a,0x0b,0xca, +0xf2,0xc2,0x24,0xef,0x84,0x0e,0xba,0x22,0x10,0xe7,0x74,0xd9, +0x76,0x40,0xd9,0x76,0xeb,0xb8,0xaf,0xf7,0x40,0xbe,0x2b,0xea, +0xb2,0xd3,0xe4,0xde,0x3d,0xd4,0xe7,0x68,0xca,0x76,0xdb,0x69, +0x16,0xf3,0x19,0xaa,0xae,0xef,0xd7,0x18,0x96,0xbb,0xad,0xc5, +0xf6,0x85,0x7f,0x7c,0x6c,0x19,0x13,0xaf,0x58,0xc7,0x61,0xf9, +0xbb,0x95,0x3b,0x65,0x43,0x8b,0x79,0x18,0x73,0x84,0x79,0x8f, +0xbb,0x6d,0xca,0xe9,0x7a,0xb6,0xbc,0x5d,0xb8,0x46,0x35,0xc5, +0x77,0x6c,0xf3,0x98,0x76,0x24,0x2f,0x09,0xe3,0x0e,0xf4,0xb6, +0xcc,0x66,0x2d,0x3b,0x69,0xd4,0x51,0x83,0xf5,0x60,0xe7,0xd9, +0xd9,0x97,0x85,0xf0,0x9b,0x1e,0x17,0xe2,0x55,0x1c,0xac,0x01, +0x79,0xe1,0xd2,0xb6,0xdd,0xd3,0x05,0xd8,0x64,0x59,0x9d,0x13, +0xef,0xdd,0x61,0xef,0xc5,0x53,0xe1,0x16,0x19,0x3d,0x8c,0xcd, +0x70,0x84,0xf0,0xed,0x75,0x63,0x4b,0xb8,0x26,0x3f,0x18,0x2c, +0x32,0x9c,0xc7,0x3e,0x31,0x93,0x37,0x39,0x2a,0x8d,0xf7,0x7c, +0xdf,0x09,0xde,0xc5,0x40,0x82,0x9b,0x87,0xdd,0x1b,0xb1,0x82, +0xb6,0xd5,0xe7,0x14,0xa1,0xe9,0xb9,0x27,0x9c,0x10,0x33,0xad, +0x98,0x85,0xfe,0x3c,0xdd,0x5c,0x15,0x64,0x8d,0xe5,0x1e,0x27, +0x15,0x26,0xd3,0x78,0xde,0x95,0x7b,0xe7,0xab,0x4a,0x38,0x78, +0xb1,0x98,0x89,0xeb,0x9f,0x33,0x3c,0x26,0xb4,0xaf,0xaa,0x2d, +0x62,0xeb,0x3b,0x5e,0xe7,0x78,0x55,0xce,0x8f,0x4c,0xa1,0x55, +0x31,0xf2,0x09,0x2b,0x67,0x3a,0x08,0xcd,0x17,0x4d,0xaf,0xf8, +0x7f,0x6b,0x26,0x75,0xbc,0xd3,0x16,0xaf,0x47,0x9a,0xcf,0x76, +0x39,0x99,0xec,0x4a,0x21,0xdb,0x19,0x3a,0xf7,0xc9,0xbc,0x59, +0x6e,0xed,0x64,0x64,0x54,0xc2,0x81,0xca,0x98,0xf2,0x98,0x95, +0x3a,0x84,0xfa,0x49,0xc4,0x79,0x33,0x99,0x93,0x22,0xec,0x23, +0x67,0x7b,0xfb,0x48,0x5e,0x60,0x86,0x19,0x47,0x5c,0x60,0x94, +0x61,0xec,0x67,0x3d,0x0b,0xf3,0x2a,0xa0,0x7a,0x0d,0xa7,0xee, +0x52,0x44,0x13,0xfc,0x93,0x71,0xb3,0x40,0xa4,0xcf,0x79,0xb8, +0x78,0x73,0x10,0x4d,0x40,0xd1,0x26,0xd1,0xc5,0x85,0x80,0x5e, +0x8b,0x28,0x69,0xf4,0x47,0xf9,0x37,0xd9,0x79,0x18,0x5d,0x7b, +0xf8,0x2a,0x9f,0xe7,0x77,0x22,0x93,0x24,0xef,0xe3,0xa7,0xfe, +0xa7,0x9c,0x3d,0x6c,0x74,0x00,0x79,0x0f,0x96,0x83,0x8c,0x4d, +0xf2,0x56,0xdf,0x65,0x23,0xb9,0x1f,0x7d,0x16,0xf0,0x57,0x18, +0x70,0x5f,0xcb,0xc0,0x18,0x8d,0xc0,0xca,0xc3,0x04,0x3d,0x43, +0xa9,0x2c,0xca,0xbc,0x2b,0x93,0x37,0xa4,0xbd,0xb6,0x30,0x70, +0xbe,0x54,0x5d,0x2c,0x39,0x88,0x9a,0x8b,0x47,0x60,0x5f,0x8d, +0xa6,0x11,0xf2,0x76,0xed,0x96,0x71,0xd1,0xb6,0x07,0xcb,0x8e, +0xdb,0x20,0x78,0x57,0x8d,0xc8,0x8b,0x1e,0xe0,0x17,0x6b,0x7d, +0xa7,0xe0,0xca,0xd0,0xc0,0x7f,0xd8,0x92,0x6e,0x59,0x04,0x4b, +0x80,0x68,0xa3,0x4f,0x5b,0x37,0x16,0xc7,0xbf,0x45,0x91,0x36, +0x95,0xfa,0x6e,0x99,0x66,0x42,0x7e,0xae,0x05,0xdc,0x18,0xf5, +0x3e,0xa6,0x2a,0x6f,0x69,0xbf,0x94,0x0e,0xff,0x26,0x21,0xe3, +0x38,0x88,0x4f,0xb6,0xa3,0xa0,0x75,0xc5,0xc9,0x90,0xe0,0xb5, +0xcd,0x90,0x41,0x0b,0xbe,0x81,0xad,0x04,0x8d,0x1e,0x50,0x53, +0xc0,0x02,0xb0,0xc7,0x67,0x9b,0x25,0x6e,0xe8,0x69,0x36,0xcc, +0x1c,0x73,0x2a,0xaa,0xd3,0xc1,0xc8,0x61,0xa1,0x21,0xab,0x04, +0xa2,0x4a,0x91,0x06,0x72,0x20,0xb2,0x4f,0xcb,0x90,0x29,0x6a, +0xd0,0xd8,0x35,0x3c,0x89,0x84,0x46,0x2a,0xcc,0xcd,0x24,0xba, +0xf7,0xea,0x7e,0xab,0x6f,0xb1,0xdd,0xb0,0xd2,0x45,0xb3,0x0b, +0x66,0x09,0xdd,0x2e,0x64,0x8c,0xf9,0xf7,0xc4,0xd6,0xf4,0xfc, +0xc3,0x38,0x56,0x08,0xa1,0x3f,0x6c,0x71,0x5f,0xfb,0xf2,0xf4, +0x0b,0x03,0xbb,0x5a,0xd3,0x5f,0xb9,0xb6,0x4c,0x59,0xb0,0x75, +0xf8,0xa1,0xe1,0x77,0xd2,0x90,0xd0,0x15,0xbb,0x52,0xe9,0x67, +0x6e,0x02,0x71,0xeb,0x48,0xfe,0xa5,0x4c,0xa4,0xdb,0xd6,0x35, +0xc5,0x5b,0x05,0x21,0x69,0x78,0x6b,0x02,0x6f,0xae,0x28,0xba, +0xa1,0x75,0x48,0x6e,0xbf,0xcb,0x9b,0x22,0xe6,0x3b,0x4d,0x64, +0xd0,0xd2,0x80,0x1c,0x46,0x1b,0xfc,0xa0,0x6b,0xd1,0xd0,0xda, +0x38,0x5a,0x47,0xd9,0x9d,0x6e,0x27,0x16,0xd9,0x24,0xae,0xa5, +0x7e,0x12,0xc7,0x17,0x32,0x7b,0x74,0x29,0x58,0x84,0x42,0x76, +0xba,0xde,0x52,0xef,0xa1,0xa9,0xdd,0x9e,0x6a,0xde,0x43,0x21, +0xcd,0x0e,0xf1,0x01,0x40,0xb5,0xf3,0xf4,0xb9,0xe0,0x8e,0x1e, +0x14,0x21,0xd5,0x4c,0x24,0x01,0x64,0x77,0xbc,0x4b,0x04,0xbd, +0x5d,0x56,0xef,0x24,0x97,0xd4,0x99,0x15,0x99,0x47,0xba,0x03, +0x14,0x1d,0x48,0x00,0x00,0xab,0xac,0x23,0xdf,0xe4,0x42,0xdf, +0xfe,0x38,0x10,0x61,0xdd,0x4a,0xf1,0x69,0x8b,0x1f,0xc4,0xde, +0xd5,0x41,0x62,0xfe,0x41,0x26,0x1b,0x14,0xbe,0xdb,0xe9,0x4b, +0xbd,0xe3,0x13,0x89,0x25,0x29,0xbc,0xdf,0x72,0x39,0x66,0xfa, +0xd4,0xad,0xe5,0x9d,0x40,0xdb,0x8f,0x05,0x47,0x8e,0x30,0xbd, +0x44,0xcf,0xe2,0x9f,0x71,0xc3,0xfc,0xd8,0xd7,0x32,0xfe,0x87, +0xef,0x55,0xcf,0x5d,0x5a,0x51,0x36,0xaa,0x6d,0x51,0x91,0x19, +0xe2,0x4c,0x82,0xd3,0xc2,0x6f,0xe9,0x23,0x19,0x14,0x02,0xc0, +0x25,0x60,0xd1,0x22,0xfb,0x93,0xe1,0x49,0x93,0xac,0x79,0x43, +0x9f,0x2c,0x77,0x84,0xb0,0x79,0xee,0x10,0x13,0x75,0xf5,0xf4, +0xfa,0xba,0x9c,0x55,0x80,0xab,0xf6,0x3e,0xd3,0x4e,0x73,0x37, +0xd7,0xa5,0xee,0xa7,0xed,0x3f,0xba,0x64,0x7a,0x10,0xee,0xc7, +0x47,0x2a,0x24,0x31,0x99,0x2f,0xdc,0x8f,0x5d,0x65,0xda,0x75, +0xf1,0xfa,0x2d,0x4d,0x5b,0xa9,0xa0,0x79,0xd8,0x9a,0x70,0x7c, +0x36,0xce,0xd0,0x93,0x97,0xa7,0xe9,0xc8,0x84,0xbd,0x04,0xa8, +0xe2,0xe7,0x95,0xa1,0x9f,0xe8,0x75,0xc6,0x40,0xac,0x40,0xab, +0x52,0x61,0x8d,0x69,0x3b,0x6f,0x94,0x31,0xa0,0x9b,0xbf,0x87, +0xc3,0xd4,0x7b,0xe7,0xd0,0x12,0xd9,0xb2,0xde,0x28,0x95,0x39, +0x0e,0xe8,0xa7,0x36,0x68,0xcf,0x44,0xab,0x5b,0xe7,0x31,0x96, +0xef,0xaf,0x02,0x89,0x1e,0x0e,0x1b,0x8d,0x65,0x8c,0xfe,0x2b, +0xf1,0x32,0x0e,0x35,0x66,0xed,0xe7,0x3c,0x63,0xf7,0x52,0x91, +0xa1,0x28,0xac,0xba,0x5f,0x82,0xf6,0x22,0xdf,0x8f,0x46,0x68, +0x46,0x1c,0xd8,0x95,0x69,0xa3,0xac,0x6d,0x97,0x6f,0x66,0x9d, +0x04,0x3b,0xb4,0x3c,0x5b,0x24,0x09,0x7e,0x7e,0x54,0x9d,0x98, +0xa2,0x2e,0xf0,0x53,0x6d,0xec,0x29,0xcc,0xd4,0xbe,0x6e,0xf8, +0x7d,0x36,0xc6,0x80,0x3c,0xb6,0x3b,0x0c,0xa0,0x17,0x5f,0x82, +0x96,0xf3,0x7e,0x35,0xc1,0x53,0x5b,0xcb,0x21,0x9b,0xdc,0xae, +0xa4,0xcd,0x31,0x24,0x4e,0x8a,0x90,0x80,0x82,0x2c,0x6f,0x4d, +0x78,0x4a,0x52,0xb1,0x28,0x42,0x25,0x89,0x61,0x6b,0x78,0x12, +0xef,0x69,0xbf,0xc6,0x9b,0x26,0x75,0x69,0x47,0x5b,0x7b,0x2d, +0xc3,0x81,0x87,0xc6,0x50,0x64,0xa2,0x71,0xc8,0xb3,0x00,0x8c, +0x6e,0x57,0xbe,0x62,0xe5,0xdb,0x50,0xd3,0xf5,0x9c,0xed,0x13, +0x02,0x17,0xa6,0xdd,0x29,0x47,0x69,0xc0,0x92,0x98,0x54,0xda, +0xb9,0x78,0x50,0x3e,0xde,0xbe,0x6e,0x1c,0xde,0xf7,0x35,0x4a, +0xb0,0xdb,0x56,0x9d,0xda,0x81,0xb4,0x0e,0x41,0x9b,0x78,0x69, +0xee,0xb7,0x82,0x8f,0x37,0xbd,0x6b,0x6c,0x64,0x2a,0x8b,0xc4, +0x67,0x9d,0xcc,0xdf,0x10,0x65,0x43,0x23,0xbf,0xf8,0x8b,0xe5, +0xea,0x3d,0x9f,0xba,0x1c,0x65,0x90,0x06,0xa8,0x5a,0xe0,0x23, +0xfc,0xf6,0xf0,0x84,0xbd,0x91,0x75,0x0e,0xf9,0x30,0xc8,0x87, +0x28,0x02,0x28,0x7a,0x85,0x24,0xa9,0x7f,0x53,0x58,0xa2,0xd0, +0xda,0xde,0xd5,0x9e,0x3b,0x41,0x8a,0x6f,0x28,0x65,0xb1,0x75, +0xa3,0xbd,0x12,0x6c,0x14,0x82,0x0c,0x31,0x80,0xdc,0x65,0x04, +0xbc,0x36,0x4d,0xb7,0xc3,0x35,0x17,0xa2,0x9f,0xfa,0x2e,0x6f, +0x90,0x2f,0x0e,0xa2,0xf4,0x5b,0xce,0x86,0x8d,0xf3,0x5f,0x30, +0x7a,0xdc,0x9a,0xb0,0x1b,0x1b,0x96,0x0c,0x15,0x60,0x64,0x5a, +0x88,0xa9,0xb8,0x0e,0x18,0xdf,0x98,0x34,0x29,0x82,0xaa,0x9c, +0x36,0x0c,0x44,0x91,0xf6,0x93,0x57,0x50,0xad,0x5f,0xaf,0x2f, +0xf0,0xde,0x34,0x14,0xfd,0x0d,0x06,0xab,0xf4,0x2b,0x05,0x1e, +0xcb,0xc1,0x31,0xce,0x2f,0xe4,0x1d,0x3e,0x84,0x5d,0x5e,0x97, +0x9b,0xe9,0xff,0x28,0xdf,0x17,0x9d,0x4e,0xf0,0xad,0xf3,0x0b, +0x13,0x37,0xc7,0x30,0x3b,0xe1,0xa7,0x70,0xd2,0x10,0x33,0xac, +0xa7,0x60,0xa3,0xd4,0x3a,0x4f,0xa6,0xf3,0x0d,0xb4,0x73,0xc4, +0x67,0x83,0xfa,0xe8,0x62,0xba,0xb8,0x28,0xe7,0xc8,0x84,0x8e, +0x2e,0x36,0xeb,0x39,0x66,0x05,0x5b,0x33,0x23,0xe4,0xe7,0xab, +0x69,0x8d,0x21,0x8c,0x08,0x86,0xa3,0x35,0x54,0x00,0xa9,0x58, +0x0a,0xd0,0xd9,0x1f,0x93,0x79,0x2c,0x14,0x50,0xe6,0xd9,0x66, +0x7a,0xbd,0x3a,0x7a,0x0b,0x67,0x9d,0xa3,0xdb,0xab,0xea,0xe2, +0x2a,0x7c,0xe1,0x53,0x68,0x0e,0x2f,0x6b,0x1d,0x9d,0x08,0x78, +0x17,0x57,0xe8,0x2a,0x0a,0xff,0x3c,0x02,0x64,0x60,0x74,0x26, +0xfc,0x0f,0x7f,0x47,0x4d,0xe0,0x70,0xe3,0x1d,0x44,0xc7,0x1d, +0x5d,0xfc,0x33,0x02,0x09,0x0e,0x3b,0x09,0xf4,0x55,0xc8,0x03, +0x4e,0x52,0xad,0xaf,0xdd,0x8e,0x1a,0x29,0xb9,0x4d,0x11,0x68, +0x64,0x9a,0x60,0x3b,0xf3,0x96,0x83,0x8e,0x81,0x77,0x8d,0x23, +0xfe,0x03,0x28,0x9e,0x57,0x50,0xf4,0xef,0xf2,0xf7,0x1f,0x47, +0x97,0xeb,0xe5,0xb5,0x50,0xce,0x11,0xc7,0x13,0xfd,0xbb,0xfc, +0xfd,0xc7,0x11,0xec,0xbd,0xe5,0xdf,0xe9,0xdf,0x7f,0x1c,0xd5, +0x17,0xeb,0xb2,0x5c,0xfc,0x5d,0xfe,0xfe,0xe3,0x68,0xb3,0x94, +0x5a,0x9f,0x84,0x0a,0x64,0x22,0x18,0x74,0x94,0x01,0x86,0x23, +0x89,0x01,0x62,0x35,0x62,0xac,0x67,0x87,0xc5,0xdf,0xc6,0x03, +0xcd,0xe5,0x79,0x89,0xc3,0x16,0xc4,0x12,0xc8,0x6a,0x25,0xed, +0x30,0xc7,0x83,0x56,0x84,0x43,0xbf,0x21,0xc5,0xef,0x39,0x3b, +0x23,0x70,0x85,0xd6,0x88,0x9f,0xcf,0xe5,0x08,0xdf,0x36,0xd7, +0x8d,0x8a,0x61,0x68,0x74,0xde,0xc3,0x08,0xbf,0xf4,0x1a,0x6f, +0xbf,0xa6,0x77,0x6f,0xbf,0x2f,0x2f,0xa1,0x2f,0x8e,0xfc,0xcd, +0xc1,0x59,0xbd,0xe4,0x93,0xec,0xd8,0x95,0xe7,0x16,0x1a,0xe5, +0xfd,0xe4,0x93,0xcc,0xeb,0xfc,0x1f,0x71,0xe7,0xff,0x68,0x74, +0xfe,0x7c,0xb9,0x6a,0xe9,0x9b,0x52,0x5b,0xba,0x8e,0x4b,0x7b, +0xa9,0x27,0x18,0x03,0x4d,0x85,0x2a,0x6f,0x25,0x76,0xbb,0xde, +0x4c,0x79,0x42,0xa0,0x57,0xa4,0xf0,0x27,0x38,0x64,0x54,0x8e, +0xe8,0x2d,0x9d,0xe5,0x5e,0x69,0x1d,0x2c,0x2d,0x9c,0xad,0x44, +0xf6,0x29,0x42,0x8f,0x59,0x7f,0x69,0x0d,0xba,0x5c,0x64,0x34, +0xc8,0x1f,0xe8,0xcf,0x2f,0xf2,0x2f,0xf5,0xe7,0x83,0xfc,0xc4, +0x2d,0x21,0x11,0x23,0xf3,0x3b,0x7c,0x76,0x39,0xbf,0x53,0xcd, +0x10,0xc5,0x75,0x25,0x89,0x28,0xbf,0x6b,0xe8,0x84,0xf7,0xda, +0xc2,0xf2,0x01,0x4d,0x6c,0x61,0xbb,0x5d,0x67,0xb0,0xa3,0xde, +0xd2,0x7c,0xec,0x16,0xd7,0x77,0x68,0x7c,0x94,0xba,0xdf,0x20, +0x0a,0xe3,0x55,0xa7,0x1a,0xc9,0xf0,0x2b,0x01,0xad,0xbd,0x4b, +0xfc,0x4d,0xff,0xc9,0x10,0x7c,0x94,0xee,0xad,0x7b,0x04,0x9a, +0xf7,0x57,0x6c,0x20,0xc3,0xa3,0xb3,0x1f,0x78,0x83,0x52,0xa9, +0x2f,0x3f,0x9e,0x8c,0x09,0x6e,0x97,0x12,0x31,0x91,0x49,0x76, +0x06,0xed,0x62,0x0e,0xa1,0x40,0xdf,0x49,0x3b,0x04,0x08,0xb6, +0x21,0x23,0xc5,0x9f,0xe1,0x40,0xdb,0x7a,0x5e,0x02,0xfe,0xa0, +0xeb,0x12,0x84,0x86,0xf2,0x66,0xc1,0x33,0xe3,0xcb,0xf9,0xed, +0x87,0x73,0x11,0xf3,0x5b,0x8f,0x23,0xe1,0xf6,0xc9,0xb0,0xb3, +0x79,0xb2,0x5f,0x15,0x6d,0xff,0x4c,0x5d,0x5d,0xdf,0xcc,0x83, +0xc7,0x1c,0xed,0x75,0x87,0x9c,0x49,0x79,0xff,0x62,0xf6,0x56, +0x46,0xe6,0x04,0xb1,0xa8,0x20,0x55,0x3c,0x2b,0x83,0xaa,0x7e, +0x26,0x3d,0x70,0xec,0x61,0x1f,0xb2,0x1c,0x43,0xfb,0x0c,0xb9, +0xfd,0x51,0x9b,0x66,0x23,0x65,0x9f,0x79,0x82,0x26,0xdb,0x63, +0x54,0xe2,0x6e,0x78,0x70,0xab,0x68,0x55,0x29,0xed,0x53,0x54, +0x39,0x9b,0x59,0xef,0xce,0xc7,0x7f,0x98,0xdd,0x26,0xda,0x5b, +0xfc,0x96,0xc8,0x9a,0x72,0x21,0x44,0xe2,0x6b,0x5b,0x1d,0xb9, +0xbd,0x6f,0xc9,0x49,0xbd,0xfa,0x48,0x24,0xbb,0xb6,0xc0,0x9d, +0xda,0xbc,0xfa,0xf7,0x15,0x56,0x95,0xcf,0x31,0x4f,0xf8,0x5d, +0xf1,0x27,0xac,0x8c,0xf7,0xfd,0x26,0xc5,0x28,0xb4,0xdd,0x61, +0xd2,0x2b,0xc0,0xfa,0xa8,0xb8,0x2d,0xb6,0x0f,0x95,0xce,0xdd, +0x1b,0xe4,0x8c,0x22,0x17,0x7b,0x62,0x7d,0xc1,0xd2,0x4e,0xb0, +0x42,0x8e,0xaa,0x45,0xbd,0x41,0x69,0xc7,0x9a,0x14,0x71,0xbd, +0x11,0x96,0x87,0xb3,0xec,0xfa,0x82,0xed,0x85,0x78,0x91,0x84, +0xb2,0x35,0xb6,0x68,0x99,0x47,0xa1,0x45,0x39,0xa9,0x39,0xb7, +0x54,0x60,0x16,0x25,0x6e,0xb7,0x98,0xea,0x53,0x3d,0xeb,0x03, +0x31,0x15,0xd8,0xef,0xcf,0x01,0x15,0x30,0x38,0x8d,0xe4,0x54, +0x1f,0xd7,0x7c,0xbe,0x66,0x23,0x77,0xf8,0xc9,0x4f,0x5b,0xe7, +0x01,0x74,0x3c,0x7a,0xcf,0xca,0x85,0x96,0x05,0x71,0x46,0xc6, +0x8b,0x0c,0x46,0x25,0xb5,0xc2,0x61,0x40,0x93,0xac,0x36,0x71, +0xb1,0xbc,0x95,0x47,0x78,0xe3,0xc3,0x0a,0xde,0xa6,0x58,0x97, +0x82,0x58,0x3a,0x77,0x73,0x10,0xce,0x12,0xca,0xf4,0x9b,0x25, +0x01,0x7a,0xd7,0xc4,0x9c,0x3f,0x26,0xd3,0xa6,0x44,0x8a,0x0a, +0x1c,0x50,0xfd,0x04,0x25,0xc3,0x35,0x16,0x3f,0xde,0x2a,0x81, +0x50,0x82,0x29,0x1f,0xee,0x9b,0x5b,0x37,0x01,0x86,0x64,0xfa, +0x68,0xfd,0x8e,0xe2,0x84,0x14,0x04,0xf0,0x60,0xda,0x71,0x49, +0x99,0x48,0xd5,0xf2,0x29,0x10,0x35,0x87,0xda,0x84,0x29,0x6a, +0x1e,0x16,0x7b,0x53,0xb9,0x63,0x50,0xe7,0x89,0xc2,0x3f,0xef, +0xb6,0x05,0x85,0x03,0xc6,0x52,0x6d,0x48,0xcd,0x63,0xf7,0xf8, +0x83,0xb8,0xf7,0x01,0x62,0x67,0xfa,0xb8,0xf3,0xc8,0x31,0xe0, +0x8e,0xa4,0x68,0x44,0xf0,0x3a,0x4f,0xe8,0x37,0x70,0xa5,0x75, +0xc2,0xc2,0xf5,0xbc,0x9c,0xbe,0x2d,0x35,0x99,0x76,0x25,0xf7, +0x1a,0x2b,0x20,0x08,0xcf,0x13,0xee,0x21,0xdd,0x40,0xcb,0x86, +0xd9,0x93,0xe2,0x2e,0xd8,0xdd,0xa0,0xb4,0xd1,0x5b,0x10,0xfa, +0x60,0x6e,0xd2,0xd8,0xce,0xd4,0x7d,0x72,0xe3,0x29,0x60,0x45, +0x96,0x2a,0x5a,0x24,0x2b,0xcf,0xb6,0x2d,0x52,0x15,0x8a,0xcc, +0x9c,0x76,0xa4,0xfc,0x76,0x2b,0x3f,0xf0,0x21,0x75,0x11,0xdc, +0x54,0xd3,0x2f,0x2f,0x00,0xd4,0x29,0x67,0x68,0x87,0x9e,0x06, +0x92,0x56,0x4e,0x53,0x7b,0x48,0xba,0xc6,0xa6,0x75,0x54,0xeb, +0x73,0xbe,0xc6,0x6b,0x09,0x71,0x67,0xc8,0xda,0xbf,0x69,0xc8, +0xce,0xcf,0x26,0xc9,0x53,0x47,0x9e,0x7b,0x83,0x8f,0x62,0x29, +0x54,0xdc,0x91,0x49,0xda,0x47,0x8b,0x67,0xfc,0x24,0x1b,0x3a, +0x8b,0x84,0xad,0x62,0x08,0x4b,0x2e,0x41,0xe2,0x58,0xff,0x25, +0x37,0x8f,0x67,0xc0,0x15,0x86,0x6c,0xd5,0x04,0xef,0x71,0x28, +0x95,0x01,0xe8,0xba,0x5c,0x64,0x59,0x83,0xed,0x17,0x07,0xbd, +0xa4,0x3c,0xfe,0x16,0x64,0xb3,0x7c,0xaa,0x2e,0x5c,0xd8,0x8e, +0x73,0xa6,0x41,0x05,0xca,0x75,0x1c,0x3f,0x1d,0xd3,0x4c,0x12, +0xe0,0x0a,0xe5,0xce,0xe6,0xb8,0xfc,0x82,0x6d,0xe3,0xf0,0xa5, +0x28,0xc9,0x7e,0x79,0xae,0x4b,0xd3,0xcd,0xce,0xfe,0x5e,0xd1, +0x2e,0x23,0x88,0x01,0xfe,0x11,0xe2,0x5a,0xd8,0x91,0x67,0xb8, +0xd1,0x92,0xcb,0x4b,0xd9,0xf7,0x19,0xb0,0x2a,0x78,0xb9,0x5f, +0x8f,0x8c,0x71,0x55,0x8a,0x4b,0x13,0x1d,0x6d,0xd4,0x82,0x08, +0x66,0x04,0xb7,0x51,0xeb,0x98,0x3f,0x4c,0x44,0x62,0xc9,0xc2, +0x85,0x2c,0xa6,0x1d,0x5e,0x9a,0x94,0xce,0xcf,0x82,0x7d,0x80, +0xd2,0xb9,0xd0,0x7e,0x4a,0x77,0x96,0xf4,0xa2,0x0d,0xf3,0x83, +0x13,0x65,0xa3,0x34,0x6d,0x3f,0x8a,0x6c,0xb7,0xce,0xc3,0xde, +0x26,0xb6,0x12,0x10,0x0f,0x08,0x37,0xd5,0x72,0xbd,0x79,0xcf, +0xe0,0xf4,0x5f,0xf2,0xdf,0x26,0x21,0xd9,0xc7,0xe4,0xec,0x39, +0x2f,0x94,0xbf,0xb5,0x1d,0x84,0x4e,0x63,0x8b,0xbd,0x7c,0x7d, +0x53,0x6f,0xa4,0xc5,0x19,0x6d,0x08,0xd1,0x95,0x60,0x63,0x7d, +0xee,0xeb,0xbd,0xd9,0x5c,0x1b,0xb5,0xb4,0xf6,0x3a,0x88,0xfa, +0x74,0x54,0xa4,0x9d,0x79,0x57,0x53,0x04,0x23,0x9f,0x65,0xf6, +0x21,0x8c,0x8f,0x30,0x74,0x26,0xc2,0xc7,0x8d,0x5b,0x40,0x6e, +0xe1,0x20,0xc3,0x78,0x36,0xad,0x8d,0x8d,0xf8,0x3d,0xb7,0x3d, +0xa2,0x10,0x90,0x51,0x3b,0x13,0xf0,0x0b,0xee,0xc7,0x5e,0x27, +0x5a,0x27,0xee,0x66,0xda,0x1e,0x5b,0x5c,0xd2,0x73,0x35,0xca, +0xf8,0x08,0xac,0xb5,0xae,0xbd,0x98,0xb7,0xb4,0x0d,0xa6,0xc1, +0x5b,0x0e,0xec,0x9a,0x8c,0x48,0xff,0xba,0xc5,0x93,0xc2,0xc5, +0x5c,0xe9,0xe3,0x46,0x24,0x2b,0xa3,0xe3,0x05,0x64,0xf2,0x4f, +0xf4,0x7e,0xfa,0x28,0xda,0x75,0x0f,0x6f,0x81,0xf6,0xe0,0xfb, +0x61,0xa6,0xd3,0xca,0x53,0x04,0xa7,0x40,0x78,0x87,0x57,0x7d, +0x0b,0xaf,0x91,0x63,0xbd,0x65,0x36,0x81,0x18,0xc4,0xfa,0x06, +0x7b,0xf4,0xe7,0x93,0xbf,0x7f,0x1e,0x6f,0x91,0x7c,0x24,0x88, +0xc0,0x94,0xae,0x87,0x4e,0xac,0x7f,0x48,0x3c,0x33,0xed,0xd4, +0x81,0x02,0x90,0x3f,0x55,0x66,0xcf,0xe5,0x3a,0xd1,0xc9,0xb0, +0x55,0xc8,0x82,0x42,0x93,0x16,0xae,0x48,0x81,0x4b,0x05,0xac, +0x5e,0x8f,0x54,0x5b,0xa2,0x9d,0x88,0xad,0xcd,0x69,0x28,0x6a, +0x64,0x48,0x52,0x68,0xdb,0xac,0x60,0x7b,0xc7,0xc7,0xda,0xa2, +0xd7,0x5e,0xdb,0x11,0xb6,0xd1,0x64,0x7b,0x7c,0x7c,0x5f,0xae, +0x65,0xf3,0x6c,0x6b,0xe3,0x48,0x17,0xb8,0x97,0x0b,0x8a,0x56, +0x65,0x63,0x6d,0x92,0xa2,0xe0,0xdb,0x05,0x5e,0x1f,0x36,0x0c, +0xcf,0x36,0x6c,0xc1,0xdd,0x88,0x0e,0xee,0x99,0xb5,0xd2,0x1d, +0xf1,0x8c,0x5e,0x97,0xb1,0xfd,0x58,0x97,0x1d,0x17,0x30,0x7c, +0xe8,0x9b,0x6a,0x72,0xb3,0x7c,0x1a,0x10,0x2d,0x48,0x08,0xa3, +0x98,0x69,0x92,0x5d,0x07,0x02,0xeb,0x2f,0xb6,0x1d,0x00,0xea, +0x6c,0xc4,0x58,0x47,0x8b,0x6f,0x39,0x8c,0xf0,0x29,0x87,0xa0, +0x99,0xa2,0x05,0x8e,0x5c,0x8b,0xa3,0x45,0x6e,0x1c,0x47,0xce, +0x9a,0x46,0x63,0x53,0x04,0x08,0x35,0xe3,0xd5,0x0e,0x33,0x0e, +0x0d,0x38,0xc3,0xd7,0x2c,0xc8,0x42,0x01,0xfe,0x7a,0xc7,0xb6, +0xe1,0x7f,0x21,0xcb,0x55,0x34,0x22,0xbb,0x5c,0xf8,0x91,0x6b, +0x75,0x8c,0x78,0xab,0x0a,0x63,0xc6,0xe0,0xeb,0x34,0x31,0xfc, +0xee,0x44,0x83,0xf4,0xc3,0x10,0x60,0x19,0x3e,0xe2,0xa5,0x64, +0xcd,0xf5,0xda,0x99,0xc4,0x0e,0xdf,0xa0,0x20,0xff,0x02,0x29, +0x25,0x4e,0x08,0xde,0x57,0xe4,0x36,0xd5,0x1a,0xc3,0x31,0x5a, +0x7a,0x76,0x63,0x62,0x82,0x03,0x88,0x43,0x87,0x34,0x0c,0xa7, +0x89,0x6f,0x13,0x7f,0x80,0x30,0xc3,0x30,0xdd,0xfb,0xe9,0x17, +0x0f,0x9b,0x30,0xde,0xbd,0xcd,0x61,0x4b,0xce,0x82,0x96,0x7c, +0x0e,0xde,0x73,0x4c,0x98,0x0d,0x5b,0x88,0xb3,0xd1,0xfe,0x2a, +0x52,0x4f,0x70,0xaa,0xbb,0x9e,0xd5,0x88,0x88,0xf6,0x10,0x14, +0x15,0xd0,0x27,0x45,0x39,0x39,0x34,0x6e,0xe1,0x09,0x69,0x31, +0xbc,0x18,0x35,0xcf,0x39,0xbd,0xa4,0x9f,0xf4,0x5a,0x8a,0xe6, +0x2d,0x47,0xa2,0xe6,0xad,0x77,0xd3,0xb1,0x84,0x67,0xec,0xd0, +0x82,0xde,0xb7,0x18,0x2f,0x2f,0xa3,0xd5,0xe8,0x2d,0xc4,0x70, +0x11,0xca,0xf9,0xcf,0x12,0xbe,0xd8,0x81,0xe9,0x8c,0x34,0xd7, +0x54,0xc6,0x6f,0xad,0x1c,0x5a,0x32,0xba,0x62,0xb8,0xa4,0xaf, +0x09,0xfa,0x08,0x12,0xf4,0x37,0x33,0x4b,0x85,0x01,0x01,0xe2, +0x01,0x39,0x26,0x99,0xc3,0x84,0x47,0x1a,0x51,0x2d,0xb2,0x33, +0x37,0x8b,0xb6,0x26,0x1a,0xb5,0x05,0x89,0x52,0x9d,0x6a,0x2a, +0x6d,0x34,0x43,0xc9,0x9a,0x8f,0x81,0xa3,0xb1,0x4e,0x10,0x96, +0x0f,0xb6,0xe9,0x35,0x37,0x68,0x09,0xb8,0x3d,0xb2,0xd0,0xda, +0x7a,0xe8,0xa2,0x20,0x5a,0x37,0x37,0x0a,0xe7,0xb0,0x80,0xd9, +0xdc,0x7b,0x43,0x6d,0x4f,0x54,0x13,0x3c,0xc1,0xf6,0xe1,0x19, +0x53,0xcd,0xb3,0xad,0xcb,0x16,0x17,0x3b,0xd7,0xbe,0x18,0x31, +0xee,0x7d,0xec,0x85,0x84,0x32,0x8d,0x54,0xeb,0xf9,0xb5,0xb7, +0x6b,0xb8,0x5d,0x3f,0x24,0x0c,0xc2,0x0e,0xea,0x1c,0x87,0x9d, +0x0c,0xc2,0x36,0xab,0xc6,0x7b,0x95,0x4c,0xb3,0x8e,0xaa,0xfa, +0xc7,0x29,0x5e,0x42,0xa4,0x1e,0x66,0xd7,0x0b,0x4a,0x62,0x39, +0xe9,0x72,0x81,0xe1,0xe6,0x76,0xb6,0x02,0x87,0xc4,0x78,0x84, +0x9b,0xbb,0x3e,0x37,0x8c,0xbf,0xf1,0x82,0x18,0x8d,0x5e,0x14, +0x64,0x4a,0x2c,0xfc,0x97,0xaa,0x22,0x77,0x69,0x2c,0xcf,0x96, +0x1b,0xf0,0xab,0x57,0x50,0xb0,0xee,0x27,0xe8,0xe5,0x47,0x35, +0xf5,0xd1,0x20,0x39,0xe5,0xf2,0xe3,0x7f,0x54,0x45,0x2c,0xd8, +0x27,0xc6,0xa6,0x49,0x50,0xa8,0x9d,0x83,0xf1,0x7a,0xba,0x7e, +0x63,0x83,0xf6,0x7a,0x03,0xbb,0x5c,0x8c,0x7d,0x9d,0xe8,0xe5, +0xc2,0x55,0xc1,0x98,0xe2,0xeb,0x4d,0xaa,0x5c,0x16,0xdf,0x75, +0x3c,0xf8,0xea,0xe8,0xd0,0xbb,0x2a,0xba,0x5c,0xd0,0x2b,0xbf, +0xee,0x8a,0xc8,0x06,0x29,0xbe,0xc4,0x63,0xdc,0xfc,0xfd,0xdd, +0x4c,0x1e,0x86,0xf4,0x60,0x7c,0x56,0xfd,0xf6,0xdb,0xbc,0x74, +0x74,0x2a,0xde,0xd8,0x86,0x2f,0x57,0x90,0x44,0x71,0x9e,0x10, +0x16,0xb2,0x8d,0xe1,0xf9,0xbd,0x36,0x6a,0xa5,0x60,0x2a,0x43, +0x71,0x3c,0x6b,0xb3,0x9c,0xcf,0xcc,0xa2,0x82,0xff,0xca,0x5b, +0x31,0xf8,0xc1,0x9f,0xcf,0xa4,0x5d,0xe4,0x98,0xa9,0xb4,0x3d, +0x92,0xbf,0xf1,0x9d,0xae,0x24,0xe7,0xb0,0x77,0xf0,0x03,0x2d, +0x90,0x95,0x79,0x17,0x66,0xe8,0x07,0xb4,0xd1,0xe2,0xda,0x58, +0xa6,0x10,0x17,0xf2,0xd7,0xbb,0x1b,0x96,0x31,0x14,0xf2,0x97, +0x6c,0x2d,0x3b,0xde,0x9a,0xdb,0x27,0x7a,0x65,0x36,0x50,0x0b, +0xd5,0x43,0xd8,0x31,0x78,0x59,0xaa,0x83,0xd6,0xae,0x5c,0x4c, +0x41,0x7e,0x6d,0x30,0x7a,0x68,0x90,0xfd,0x33,0x3a,0x0a,0xce, +0x53,0x0c,0x76,0x86,0x06,0xed,0xa5,0xbc,0xc7,0x47,0x08,0x2d, +0xd6,0xbf,0xde,0xc0,0xc9,0x98,0x48,0x8e,0xfc,0xd8,0x2c,0x10, +0x68,0x06,0x76,0xcd,0xd6,0x8c,0xe3,0xc1,0x84,0x6a,0xfc,0xa5, +0xf0,0x3a,0xd1,0x57,0x10,0x2d,0x34,0xee,0x65,0xe6,0x6f,0xde, +0x3f,0x9d,0xa5,0xd7,0x78,0xeb,0x4b,0x47,0x27,0xfa,0xe3,0xdb, +0x8c,0x34,0xc7,0x47,0x25,0x2a,0xf4,0x82,0xbe,0x8e,0x72,0x79, +0x71,0xb0,0x13,0x81,0x24,0xed,0x54,0xcc,0x6a,0x9d,0x48,0xb9, +0x97,0x29,0x5a,0x33,0x9b,0x40,0x02,0xee,0xac,0x16,0x53,0x89, +0xaf,0x64,0x6f,0xc3,0x4f,0x07,0xca,0xa2,0x75,0xfc,0xc0,0xda, +0x46,0x61,0x51,0x11,0xd8,0x2c,0x51,0x3b,0x0b,0xde,0x26,0xf2, +0xbc,0x67,0xad,0x9d,0x47,0xef,0x49,0x66,0x3b,0x22,0x52,0xd6, +0x89,0xf9,0x02,0x2d,0x4b,0xbd,0x57,0x5c,0xbe,0x79,0xff,0xc8, +0xc5,0x5b,0x6f,0x6d,0xdb,0xe6,0xff,0x01,0xf8,0x6c,0x1b,0x38, +0xc9,0x1e,0x68,0x28,0xbf,0x2b,0x30,0xbf,0xd6,0x53,0x74,0xed, +0x39,0xbf,0x79,0xf5,0xea,0xfd,0x7f,0x3c,0x7b,0x18,0xb9,0x28, +0x13,0xf5,0xc0,0x92,0xa5,0xd8,0x58,0x80,0x62,0xe1,0x47,0xde, +0xea,0xd5,0xf9,0xf3,0x57,0x71,0xe1,0x13,0xa0,0x73,0xcb,0xa3, +0x47,0x23,0x5d,0xb2,0x4a,0x46,0x14,0xe5,0xd4,0x2d,0x94,0x1f, +0x1b,0xe1,0xc9,0x58,0x58,0x62,0xfe,0x51,0x6c,0x96,0x6f,0xca, +0x45,0xf5,0x9b,0x8f,0x74,0x02,0xd0,0x43,0x80,0x17,0x88,0x03, +0x1f,0xd2,0xcc,0x46,0x08,0x39,0x94,0x71,0xc1,0xa9,0xcb,0xfa, +0x62,0x8a,0x5e,0x3e,0x67,0x67,0x9f,0x75,0x61,0xbb,0xd5,0xba, +0x75,0x5c,0x17,0xc7,0x9c,0xd1,0xc0,0x93,0x31,0xfc,0xf3,0x79, +0xd2,0x83,0xdf,0xbd,0xe4,0xf3,0x09,0x05,0x97,0x67,0x88,0x02, +0xd3,0x30,0x4e,0x1a,0x57,0x93,0x02,0x0b,0x6e,0x96,0x8a,0x91, +0xd4,0x66,0x64,0x43,0x0f,0x77,0xeb,0xba,0x3a,0x9f,0x03,0x6f, +0x89,0xb0,0xee,0x48,0xc2,0xd7,0xd5,0xb4,0x21,0x5a,0x40,0x60, +0xa7,0x20,0x03,0x1b,0x1f,0x9e,0xcc,0x5c,0xbe,0x6f,0x21,0x70, +0x98,0x7e,0x1c,0x50,0xf8,0x8a,0xd6,0xfa,0xbd,0xb6,0xf0,0x90, +0xf3,0x9e,0xb5,0x51,0xb8,0xec,0x20,0x40,0x41,0x4f,0xd6,0x6b, +0x8c,0x79,0x61,0x37,0x10,0x40,0xb6,0x85,0xb7,0x11,0x62,0x94, +0x26,0x05,0x4e,0xce,0x6a,0x02,0x4c,0x4d,0xdb,0xc1,0xbb,0x49, +0xc2,0x87,0x2c,0x4c,0xf2,0xd9,0x20,0xc9,0xda,0x77,0x1b,0xb7, +0x39,0x09,0x16,0x1f,0xa1,0x02,0x27,0x9d,0x9a,0x73,0xd9,0xe8, +0x6f,0xd6,0xc5,0x79,0xb7,0x0b,0x07,0xc5,0xea,0xf2,0x12,0xad, +0x64,0x81,0xe3,0xfc,0xf3,0xbc,0x5f,0x2f,0x6f,0xd6,0x17,0xe5, +0x53,0x8e,0x79,0xfd,0xc3,0xc3,0xbf,0xbf,0xfc,0xf1,0xc9,0xbf, +0x3f,0x7c,0xfe,0xf4,0xaf,0x4f,0xb2,0xe3,0xf4,0x9f,0xd3,0x03, +0xd9,0x64,0xdd,0x0b,0x4d,0xe9,0x82,0xc4,0xdf,0xe2,0x59,0x67, +0x4d,0x76,0xd9,0x18,0x17,0x40,0x7d,0xc6,0x20,0x91,0x2b,0x32, +0xa6,0x17,0xc5,0xb9,0xd4,0x3b,0x1e,0xd8,0x58,0x8b,0xa3,0x41, +0x7e,0x3c,0x88,0x65,0x15,0x8a,0x3a,0xf7,0x73,0x5d,0xde,0xcc, +0x96,0xe1,0x13,0x67,0x2d,0xb1,0x7a,0xe8,0x8a,0x3c,0x50,0x21, +0x86,0x2b,0x47,0x9f,0xe0,0xe3,0xcb,0x09,0x09,0xf6,0x25,0x0c, +0x93,0xee,0x69,0xf8,0x8c,0xb0,0x8b,0x61,0xf8,0x86,0x6e,0x2b, +0xfe,0xb5,0x40,0xa4,0x21,0x14,0xf8,0x14,0x05,0xdf,0x89,0x48, +0x42,0xf6,0x11,0x70,0xe9,0x53,0xf0,0xd3,0xb9,0xc0,0xe6,0x49, +0x4b,0xa1,0x14,0xa5,0x3f,0x54,0xe4,0xb6,0xc5,0x34,0xa1,0xe8, +0xe9,0x2f,0xd3,0x5e,0x13,0x69,0x4c,0x6d,0x88,0x5d,0x3c,0x78, +0xb1,0x19,0x27,0xfa,0x28,0xeb,0x02,0x64,0x29,0x74,0xcc,0x80, +0xa2,0x1a,0x9d,0xdd,0x76,0x08,0x0c,0xc2,0x2f,0x1a,0xb0,0x09, +0xac,0x30,0x7e,0x1d,0xe4,0x03,0x6b,0x98,0xa0,0x97,0x2f,0xe5, +0x80,0xbc,0x67,0x4d,0xa5,0xe1,0x43,0x12,0x33,0x7e,0x92,0xd1, +0xe2,0xa4,0xc1,0x0a,0x0d,0x85,0xe5,0xc4,0x20,0xb5,0x9e,0x28, +0xa6,0xa3,0x30,0x54,0x5c,0x7c,0xfa,0xeb,0xe5,0xb7,0xd3,0xb5, +0x8a,0x64,0x20,0x48,0x7d,0x4b,0xd6,0x7e,0x35,0x4b,0xa9,0x33, +0x66,0xb2,0x24,0x34,0x8f,0x7d,0xcb,0x6e,0x76,0x24,0xa5,0x22, +0x99,0x8b,0x07,0xc5,0x3d,0x8e,0x4e,0x72,0xce,0x11,0xeb,0xe5, +0x13,0xd6,0x37,0x51,0x47,0xee,0xc8,0x29,0x4c,0x1c,0x70,0xe6, +0x3a,0x2d,0x48,0xa8,0xb1,0xdf,0x43,0xaa,0x32,0xcc,0xee,0xd2, +0x8e,0xc0,0xbf,0xdd,0xaa,0x0c,0x74,0xb1,0xbc,0xbe,0x9e,0x8a, +0xfc,0x83,0xa5,0x32,0xba,0xc1,0xbc,0x66,0xaf,0x01,0xed,0x0b, +0xff,0x15,0x20,0x78,0xf3,0x3d,0x99,0xa8,0x4d,0xf8,0x76,0xcb, +0xd5,0x04,0x0e,0xf1,0xa2,0x23,0xbc,0x00,0x4c,0x99,0xda,0x8c, +0xe3,0x6d,0x81,0xf1,0xfa,0x84,0x33,0xe7,0x14,0xa0,0xaf,0x83, +0x9e,0xad,0x35,0xfb,0x8c,0xe7,0xd1,0x3a,0xab,0x72,0x7b,0x62, +0xe6,0xcd,0x71,0xf3,0x74,0x22,0xc8,0x52,0xc8,0x02,0x15,0x71, +0xba,0x23,0x7c,0xf3,0xc4,0xec,0x19,0x84,0x25,0xb0,0x2c,0xd4, +0xe3,0x11,0xee,0xd8,0x56,0x33,0x13,0x9a,0x61,0x70,0x30,0x83, +0xb5,0x06,0x01,0xd4,0xdb,0xad,0x43,0xfc,0x58,0x1c,0xef,0xb4, +0x5e,0x9c,0xc3,0xc9,0x58,0xe7,0x77,0x8e,0x94,0x48,0x4d,0xe8, +0x98,0x73,0x3e,0x6e,0x80,0x28,0x0e,0x4b,0x62,0x76,0x0e,0x0b, +0xff,0xcd,0xae,0x41,0x6d,0x5c,0x5f,0x42,0x2c,0xd3,0xc7,0x88, +0x8f,0x28,0xfd,0x12,0xf7,0x22,0xb7,0x97,0xe6,0x8e,0x96,0xd3, +0x88,0x0c,0x33,0x4b,0xaa,0xfe,0x9a,0xb2,0x5b,0x36,0x8f,0xce, +0x2d,0x7d,0xfb,0x9c,0x16,0x8f,0x5a,0xd6,0xbb,0x55,0xa2,0x24, +0x89,0xf7,0xd8,0x94,0x5d,0x35,0x52,0x1a,0x5f,0x93,0x09,0x82, +0x56,0x6b,0x01,0xef,0x48,0x37,0x9b,0x3d,0xb2,0xa4,0x26,0x38, +0xc1,0x73,0x96,0x26,0x99,0x73,0xe0,0xa6,0x7a,0xdc,0x83,0x8d, +0xc6,0x66,0xf4,0xe1,0xd3,0xd0,0xe5,0xc5,0x8f,0xcb,0x85,0x0a, +0x7f,0x05,0x96,0x06,0x19,0xcb,0x09,0x10,0xc8,0x67,0xb1,0xe4, +0x6c,0xb9,0x20,0x26,0x5d,0xe0,0x8f,0x5e,0xcf,0x9e,0x80,0x5d, +0x7b,0x97,0xd5,0xba,0xde,0x44,0xc6,0x5f,0xba,0x03,0xbf,0xbb, +0x9e,0x33,0x46,0x38,0x30,0x09,0x59,0x56,0x41,0xab,0x93,0xa1, +0x7a,0xa3,0x94,0xbe,0xd3,0x38,0x08,0x01,0x11,0x5c,0x99,0xe5, +0xd6,0x34,0xbe,0x66,0xe3,0xb1,0x59,0x58,0xd0,0x31,0x0d,0x1e, +0xd5,0x07,0x7c,0x88,0x5f,0x82,0x38,0xb1,0xa6,0xa9,0x85,0x3d, +0x1e,0x9d,0x06,0x70,0x80,0xeb,0x9b,0x45,0x8d,0xcc,0xaa,0xa7, +0x03,0x45,0x72,0x3a,0x08,0xf6,0x47,0xc1,0xad,0x1e,0x24,0x2d, +0x20,0xdb,0xe7,0x7e,0x5c,0x44,0x84,0xdf,0x8b,0x1d,0x94,0xb6, +0xed,0xa0,0xb8,0xbe,0xaa,0x00,0x24,0xd6,0x8e,0x53,0x09,0x50, +0x64,0x2c,0x56,0x5c,0xb8,0x3a,0xd4,0x1d,0x0a,0x90,0xf4,0x71, +0x32,0xe1,0x49,0xa7,0x97,0xb7,0x71,0xa4,0xa4,0x33,0xe7,0x3c, +0x38,0x2b,0x16,0xec,0x1e,0x24,0x0f,0xe7,0x32,0xfb,0x46,0xf4, +0x59,0x09,0x87,0x33,0x74,0x60,0xca,0xfc,0xe3,0xee,0x8a,0x31, +0x77,0xa1,0x4a,0x8e,0xc1,0xa4,0xd8,0x8b,0x2b,0x18,0xb3,0xed, +0xc6,0x15,0xc7,0x3e,0x1c,0x16,0xdd,0x9a,0x28,0x19,0x2f,0x3f, +0x48,0x6b,0xd2,0x6a,0xed,0xed,0xf9,0x9c,0x60,0x9f,0x72,0xfc, +0x10,0xd5,0xf2,0x3a,0x8e,0xaa,0xf1,0x9e,0xec,0x98,0x0e,0x39, +0x38,0xed,0x9f,0x66,0x95,0xdf,0x00,0xd2,0xdc,0x56,0x38,0x99, +0x78,0xb2,0xca,0x12,0xb6,0xf4,0x05,0x88,0x3e,0x37,0x0b,0xb7, +0x03,0xaf,0xc4,0xa2,0xbe,0x1d,0x26,0xea,0x0c,0xe4,0xec,0xff, +0xd4,0x1a,0xe2,0x90,0x45,0x0c,0xc7,0x36,0xa3,0x3c,0x87,0x43, +0xad,0x88,0x9b,0x01,0x7c,0x78,0x8c,0x87,0xcf,0xd4,0xb6,0x06, +0x1e,0x38,0xd0,0xed,0x8c,0xfb,0xde,0x6e,0xd5,0x9f,0xa3,0x31, +0x36,0x8c,0xcb,0xe0,0x75,0xef,0x1f,0x9c,0xb9,0x37,0xa4,0xff, +0x15,0x27,0x57,0xce,0x7f,0xcf,0xaf,0xe3,0xc9,0xe3,0xa5,0x9d, +0x32,0xbb,0xa7,0xf8,0x91,0x89,0x98,0xb5,0xa1,0x19,0x89,0xe4, +0xf1,0x4f,0xc0,0x1a,0xff,0xb4,0xa7,0x0d,0x7b,0x76,0xb1,0x45, +0x61,0xb7,0x72,0x1f,0x76,0x2d,0x60,0x3c,0x4a,0x9b,0x5a,0xf8, +0xdd,0xdb,0x54,0xd8,0xdc,0x5d,0x2f,0xb6,0x15,0xfc,0x68,0xb6, +0x82,0xa9,0xcd,0x56,0x9a,0xe0,0x65,0x87,0x44,0x47,0x3d,0xab, +0x34,0x18,0xd8,0xa6,0xe4,0x80,0x21,0x38,0x07,0x80,0x9e,0x1f, +0xa6,0x2b,0x92,0x87,0xa0,0x61,0xfd,0xb9,0x2e,0xcb,0x77,0x55, +0xbd,0x81,0xf3,0x83,0xea,0xa4,0x74,0xe6,0xb1,0x52,0x4d,0xf2, +0xe0,0xd6,0x3e,0x3e,0x2b,0xe7,0xb6,0x3a,0xf5,0xd5,0xc4,0x89, +0x89,0x95,0x4f,0xa3,0xb1,0xa4,0x4c,0xf4,0xbc,0x8b,0xcf,0x23, +0xeb,0x6c,0x3c,0x5d,0x14,0x1d,0x3b,0x57,0x18,0x9a,0xa1,0xc4, +0x29,0xb7,0xb7,0x71,0xd4,0x6f,0x6e,0x09,0x9b,0x3e,0x05,0x78, +0x27,0xc9,0x05,0x43,0xd5,0x86,0x7f,0xba,0x51,0x37,0xdc,0xf5, +0xc8,0xe1,0x11,0x98,0x19,0xf6,0x30,0xb2,0x75,0x73,0x6f,0xd0, +0x20,0xa3,0xb8,0x69,0x1b,0x8d,0x27,0xb9,0x20,0x21,0xb7,0xb0, +0x0e,0x55,0x7f,0x83,0x53,0x79,0x1d,0x72,0x87,0xa7,0x0b,0xaf, +0xeb,0x10,0x24,0xaf,0x59,0x92,0xa1,0x4a,0x0c,0xbe,0xa0,0x63, +0xf2,0x2a,0xc9,0x5c,0xf8,0x15,0xa8,0x30,0x20,0x2c,0x6c,0xb0, +0x2a,0x30,0x39,0x60,0x23,0xf2,0xc8,0x24,0x24,0xcb,0xda,0x73, +0xed,0x8e,0xa5,0x5d,0x14,0xf5,0xad,0x80,0x0f,0xf0,0x06,0xe9, +0xb4,0xe6,0x48,0x04,0xb2,0x0a,0x40,0x1f,0x6d,0x16,0x61,0x51, +0x8e,0x5c,0x28,0xe1,0x88,0x88,0x73,0xb8,0x5e,0x5b,0xa0,0xf3, +0x40,0x22,0x18,0x69,0x10,0xb4,0xb4,0x1d,0x4c,0x95,0x80,0x32, +0x74,0x7d,0xd0,0x9d,0xaf,0x3f,0xaf,0x18,0x7a,0x0a,0xd1,0x42, +0xfb,0x36,0x47,0x98,0xf8,0xa4,0x7e,0x19,0x5e,0xd7,0xc1,0x48, +0x02,0xb9,0xb0,0x4a,0x82,0x56,0x11,0xc1,0x90,0x33,0xa5,0x61, +0xa5,0xd3,0xe3,0x81,0x9e,0x92,0xb0,0x32,0xe2,0x51,0xa8,0x43, +0xbe,0x19,0x7f,0x3b,0xde,0xb1,0x3c,0x58,0x5b,0xe6,0xb9,0x28, +0x74,0x79,0x8d,0x3c,0xb0,0x25,0xe6,0x93,0x47,0x8f,0xa6,0x31, +0xa8,0x2c,0x77,0x49,0x3e,0x5f,0x19,0xc5,0xc8,0x52,0x26,0xe0, +0x11,0x17,0x22,0x2b,0x6f,0x51,0xc6,0x78,0x2d,0xfa,0x87,0x3c, +0x49,0xfe,0x76,0xbd,0xbc,0x7e,0x4e,0xa2,0x64,0x43,0x2e,0x25, +0x21,0x42,0x75,0xe9,0xca,0x5c,0x5f,0xb7,0x88,0xaa,0xf3,0x72, +0x8a,0xe1,0x72,0x7f,0x41,0xe3,0xcf,0xea,0x6d,0xc9,0xe7,0xae, +0xb5,0x7c,0x8d,0x45,0x50,0x85,0x73,0x09,0x5f,0xfa,0x57,0xd7, +0x88,0x88,0x6a,0x63,0x8b,0x47,0xd5,0xb7,0xdb,0xb0,0x3e,0x9e, +0x0f,0x81,0xf0,0xa2,0x52,0xa3,0x41,0x7e,0xc2,0x40,0xa9,0xce, +0x2b,0x94,0x73,0xf7,0x07,0x56,0x46,0x89,0xc4,0x1b,0xd9,0xae, +0x01,0x10,0x5e,0x1f,0x71,0xd3,0x0f,0x17,0xef,0x3f,0xa5,0xf5, +0x80,0xca,0x02,0xec,0x51,0x31,0x20,0xb1,0x43,0x9d,0xe1,0x89, +0x75,0x7c,0x40,0xcc,0x10,0xe1,0x20,0xc2,0x03,0x10,0x2d,0xe4, +0x5a,0xa5,0x18,0xbe,0x9d,0x86,0x42,0xd4,0x35,0xd0,0x8b,0x74, +0x8e,0xc7,0xac,0x00,0x1a,0x55,0x69,0x66,0x8e,0x85,0xfb,0x88, +0x6c,0x76,0x9d,0x47,0xc8,0x68,0xd9,0xe6,0x77,0xd1,0x8b,0xb8, +0x32,0xa2,0x76,0x52,0xa8,0x84,0x14,0x32,0x37,0xee,0x10,0xbd, +0xfb,0xc4,0x33,0x45,0x54,0x36,0x19,0x06,0xda,0x76,0xed,0x88, +0xa5,0x90,0xb8,0x1b,0x59,0x0f,0xb4,0x6e,0x5c,0x96,0x6a,0x60, +0xb4,0x4d,0xbb,0x5b,0xeb,0x43,0xbb,0xbd,0x5e,0x45,0x63,0x7a, +0x0d,0x9b,0x7a,0xeb,0x30,0x5e,0x4b,0xfb,0xc3,0xd7,0x30,0x66, +0x77,0xfe,0xb2,0x60,0x57,0xa7,0x03,0x56,0x3d,0xb5,0x0e,0x85, +0x72,0x1b,0xa7,0x42,0x3d,0x39,0x9a,0xea,0x78,0x90,0x65,0xad, +0x3a,0x4b,0xbb,0x12,0x4f,0x2b,0xbb,0x47,0x35,0x56,0xb1,0xb4, +0x53,0x99,0xd7,0x20,0x4c,0xc8,0x28,0xf6,0x95,0x2d,0x82,0x2a, +0x5e,0x85,0xe6,0x99,0x35,0xdb,0x59,0x11,0xd7,0x67,0xeb,0x2e, +0x30,0xca,0x9e,0xd1,0xb6,0xb2,0x9d,0x7f,0xc7,0xc3,0xb2,0x14, +0xad,0xa3,0x39,0x47,0x55,0xac,0xfd,0xed,0x2b,0x9d,0xf8,0x78, +0x30,0xfb,0x05,0xc4,0x7c,0x90,0x62,0xcf,0xdf,0x3f,0x2b,0x37, +0x9e,0x44,0x65,0x05,0x76,0xcc,0x92,0xc3,0x4f,0x11,0xb5,0xec, +0x8a,0xd4,0x37,0xab,0x72,0x2d,0xc9,0x45,0x28,0x64,0x79,0xc4, +0x6d,0x79,0x2d,0x13,0x88,0xae,0x2a,0x7b,0x4b,0x6d,0x5e,0xdb, +0x29,0xb1,0x80,0x90,0x98,0x2d,0xe2,0xab,0xc6,0x46,0xad,0x8a, +0xe4,0x24,0x31,0x56,0x82,0x2e,0x58,0x12,0x82,0x72,0x76,0xc1, +0x8a,0xe0,0x1d,0xf4,0xa3,0x90,0x7c,0x33,0xbd,0x78,0x73,0xb3, +0x6a,0x2c,0xee,0x40,0x70,0xb3,0x43,0xee,0x76,0x65,0x39,0x2c, +0x66,0xfd,0xe7,0x0f,0xff,0x3d,0x45,0xb1,0x2d,0x68,0xf5,0xe0, +0x45,0x40,0x66,0xe4,0xb4,0xfb,0x9f,0x8b,0xea,0xd7,0x9b,0xd2, +0x9e,0x7d,0x25,0x7c,0x57,0x00,0x10,0xb0,0xe1,0x1f,0x30,0x6c, +0xd4,0x1a,0x97,0xce,0x35,0xc6,0x1d,0xea,0x0f,0x48,0x8f,0x64, +0x01,0x45,0x63,0xa7,0x08,0xe8,0xc2,0x31,0x2b,0x77,0xb7,0xaa, +0x18,0x77,0xc7,0xb8,0xa2,0x31,0xe3,0xd9,0x90,0x31,0x94,0xda, +0x53,0x30,0xdf,0x7c,0x20,0xd3,0x41,0x66,0xe0,0x21,0x80,0x19, +0xb3,0x3e,0x99,0xad,0x5c,0x22,0x22,0x05,0x79,0x3c,0xdb,0xb1, +0x91,0x16,0xce,0x76,0xd7,0xbc,0xf2,0x1b,0xb2,0xee,0xc8,0x1f, +0xa2,0xe0,0xa8,0x08,0x30,0xe7,0x8f,0xa5,0xd7,0x6b,0x8e,0x66, +0x47,0xe4,0x0b,0xb5,0x79,0x3c,0x1d,0x2b,0x79,0xca,0xfd,0xa3, +0x4f,0x40,0x18,0xfb,0x91,0x08,0xe6,0xa6,0xe5,0x28,0x45,0xd7, +0x33,0x7e,0xe9,0x5e,0x51,0x19,0x69,0x1c,0x1f,0xe2,0xf4,0xb3, +0x9a,0x48,0xf1,0x96,0x8f,0x20,0x44,0xb1,0xe1,0xce,0x99,0x8e, +0xb2,0x03,0xec,0x04,0xe2,0xa4,0xdf,0xcb,0xe9,0x09,0x5f,0x5d, +0x90,0x90,0xe6,0x9f,0x1a,0xd1,0x94,0x50,0x9b,0xa2,0xcf,0x34, +0xf8,0x06,0xa1,0x4d,0x02,0x05,0x0b,0xda,0x41,0x64,0xf5,0x56, +0x95,0x15,0xb6,0x5c,0x5a,0xb6,0x6b,0xbb,0x80,0x72,0xd9,0xc6, +0x9b,0x26,0x7b,0xfa,0xd0,0x5c,0xcb,0x0b,0x42,0x64,0xf7,0x5a, +0x38,0x0a,0x70,0x6b,0x51,0x04,0xde,0xd0,0xe4,0x3e,0xc3,0x78, +0x81,0x0a,0xa4,0xf2,0xbf,0x0f,0x93,0xc4,0xbe,0xb5,0xc0,0xcb, +0x29,0x73,0xfc,0x61,0xa7,0x3b,0x0a,0xcd,0xe3,0x28,0x38,0x10, +0xfa,0x8c,0x2b,0xcb,0xfd,0x2f,0x8f,0xd1,0xee,0x3b,0xc5,0xe9, +0x04,0xd6,0xca,0xd8,0x9a,0x9a,0x48,0x2d,0x61,0x5f,0xb7,0xb6, +0x5b,0xfb,0x1e,0x73,0x0d,0x8a,0xdc,0xa2,0xcd,0x0d,0xc3,0xbb, +0x72,0xff,0xf8,0x1e,0xdc,0xc8,0x1d,0xb0,0xf5,0xa8,0xec,0xa5, +0x02,0xfe,0xe1,0xab,0x05,0xe4,0x66,0xcc,0x53,0x5b,0xee,0x6a, +0x49,0xb5,0xcb,0xb3,0x8b,0xea,0x30,0x56,0x27,0x8b,0x4c,0x4d, +0xbe,0xcf,0xbc,0xd3,0xa9,0x8a,0xdc,0xfe,0xb0,0x9a,0x5a,0x13, +0x88,0xb4,0xa7,0x0f,0xba,0xdd,0xe4,0xe9,0x63,0x54,0x6d,0x72, +0xd5,0xc2,0x4a,0xb1,0x99,0x98,0x75,0xff,0xa5,0x28,0x1a,0x76, +0x18,0x40,0x61,0x91,0xb5,0x45,0xab,0xe0,0x30,0x50,0xf9,0xe7, +0xce,0xd9,0x30,0x14,0x8e,0x63,0x3f,0x7d,0xcc,0x7d,0xaa,0x88, +0x12,0xa8,0xf4,0x6f,0x16,0x72,0xc7,0x7c,0xa9,0xbf,0xec,0x2d, +0x66,0x06,0x05,0x4d,0x47,0x3f,0x22,0x1b,0x0b,0xab,0x4e,0xb6, +0xf7,0xa1,0x3c,0x72,0xdd,0xfb,0x59,0xf3,0xce,0xba,0x64,0x45, +0x9c,0x7f,0xf4,0x22,0xf0,0x0e,0xc4,0x1c,0x1f,0x9d,0xe4,0x01, +0x06,0x71,0xe5,0xe3,0x41,0xcc,0x47,0x1e,0x3e,0x15,0x1c,0x21, +0x84,0x42,0xdb,0xd1,0x60,0xc5,0x10,0x91,0xf4,0xa6,0x88,0x06, +0x87,0x10,0xb1,0x51,0x94,0xe3,0x59,0x41,0x01,0x7a,0x3e,0x0d, +0x41,0xe1,0x75,0x78,0x78,0x20,0x39,0x7c,0x29,0x2e,0xcf,0x03, +0x2b,0x96,0x56,0x22,0x55,0x0d,0x5c,0xd0,0xee,0xc2,0xbb,0xa4, +0x6b,0x13,0x99,0x9c,0xc5,0xcf,0xc7,0x19,0x5c,0x90,0xf8,0xe1, +0x9b,0x7b,0xf0,0x66,0xb3,0x73,0xea,0xf4,0x55,0xe5,0xaf,0x40, +0xbe,0x4b,0x09,0xa5,0x96,0x80,0x08,0xed,0xf2,0xda,0x63,0x14, +0xe0,0xae,0xdd,0x7d,0x3d,0x9b,0x5c,0xd8,0xa4,0xd9,0xdd,0x8e, +0x17,0xa4,0xa7,0x5e,0xc5,0x99,0xc1,0x97,0xb6,0xf0,0x65,0x19, +0x43,0x6f,0x28,0x1a,0x01,0x0b,0x1f,0xd6,0x7f,0x7c,0x83,0x58, +0x9a,0x6e,0x9a,0xbc,0xce,0x78,0xf6,0x53,0xc6,0x8f,0xa0,0x81, +0xbb,0x76,0x0c,0xb4,0x1a,0x8e,0xc8,0x2f,0xe6,0x6f,0xb5,0x8d, +0xac,0xa5,0x36,0x3b,0x06,0xe3,0xb6,0xfe,0x84,0x6f,0x50,0x8a, +0x84,0xb3,0x2c,0x92,0x9a,0x98,0x54,0xd2,0x3b,0x46,0x2f,0xdf, +0xc7,0x08,0x89,0x6f,0xd2,0x55,0xb0,0x19,0xa0,0x0d,0xd6,0x61, +0xc4,0x62,0xa5,0xb8,0xdb,0xa9,0xe0,0x03,0x6c,0x10,0x95,0xfc, +0xf0,0x87,0x5e,0xde,0x64,0xa5,0x79,0x60,0xf1,0x67,0xdc,0xed, +0x4f,0x94,0x21,0x98,0x58,0xb7,0xe5,0xf9,0xee,0xe3,0x6a,0xf4, +0xe5,0x82,0x0e,0xfa,0x46,0x01,0xc5,0xe0,0xeb,0xaf,0xbf,0x18, +0x98,0xe9,0x7a,0xcd,0x3a,0xbc,0x55,0x01,0x3f,0xfb,0xf0,0xd7, +0x20,0xf5,0xf0,0x07,0xfc,0x60,0xc2,0xa1,0x4f,0xfa,0x65,0xe4, +0x08,0x4a,0x29,0xf2,0x7b,0xbb,0xdd,0xf3,0x04,0xb9,0xbd,0x77, +0x42,0x73,0xce,0x06,0xa7,0x97,0x80,0x11,0xb8,0x27,0xf3,0xf5, +0x82,0x92,0x6f,0x35,0x54,0x7b,0x83,0x9d,0xa1,0x80,0xe9,0x1c, +0x97,0x36,0x19,0x9f,0x9d,0xbd,0x7b,0x70,0x72,0x76,0xb6,0x39, +0x3b,0x5b,0x9f,0x9d,0x2d,0xce,0xce,0x2e,0x27,0x89,0xc1,0x80, +0x34,0xd3,0x0b,0x20,0x83,0x27,0xf2,0xa6,0x4e,0x91,0xe0,0xab, +0x28,0x67,0x18,0x55,0x16,0x2a,0xdc,0x1e,0x4f,0xb6,0xe3,0x17, +0x50,0xf1,0xe4,0xe4,0x18,0xfe,0x9d,0x02,0x7b,0xed,0x25,0xa6, +0x82,0x3d,0x7e,0x83,0x4f,0xa1,0xad,0x8b,0x46,0x7d,0xbb,0xd0, +0x93,0x5b,0x7c,0xa6,0xea,0x4f,0x18,0x58,0x1c,0xb6,0x3f,0xba, +0x46,0x85,0xb6,0xc7,0xf7,0x5e,0x7c,0xb6,0xed,0xfc,0x73,0x32, +0x2a,0xb2,0xc4,0xb8,0x87,0xdc,0x8b,0xe4,0xec,0x6c,0x9c,0xf4, +0x1c,0xb8,0xbd,0xe4,0x5e,0x9a,0xf4,0x1a,0xad,0xf7,0x92,0x2c, +0x2e,0x35,0xca,0x93,0x9e,0xed,0xa1,0x91,0x97,0x8e,0x3f,0x3f, +0x4b,0x26,0x59,0xea,0x06,0xf5,0x02,0xff,0x4e,0xb2,0x7b,0xa3, +0xec,0xec,0xec,0x8b,0x2d,0x74,0xe2,0x46,0x03,0xad,0x6f,0xe1, +0xff,0x51,0x0f,0x50,0x3a,0x31,0x2b,0x32,0x3b,0x00,0x38,0xf3, +0x3d,0x60,0x51,0x07,0x69,0xfa,0xc1,0xfe,0xa2,0x8c,0x34,0x03, +0x2c,0x4f,0x26,0xdb,0xa4,0xe7,0xbd,0x6a,0xaf,0x28,0xfc,0xc2, +0xfc,0x1b,0xa0,0x3b,0xbb,0x97,0x6d,0xfb,0xf7,0xa0,0x36,0x82, +0x66,0xe8,0xa4,0x59,0xf8,0xc1,0x82,0x5f,0x84,0xf0,0xf6,0xa8, +0x8f,0x17,0xb6,0x63,0xed,0x10,0x9a,0x89,0x0a,0x7e,0x06,0x13, +0x84,0x6f,0x28,0xf1,0xed,0xfa,0xa1,0x36,0xef,0x99,0xe8,0x5b, +0x6a,0xe9,0xfd,0xf8,0xc1,0xba,0x69,0x93,0xf0,0x4e,0x7b,0xff, +0x9c,0xc4,0x68,0xc6,0x26,0x19,0xcb,0x7e,0x6b,0x82,0x77,0xc8, +0xf3,0x88,0x2e,0xea,0xcd,0x9f,0xc0,0xcf,0xf4,0x00,0x8e,0x3c, +0xb0,0xb8,0x7b,0xfa,0x38,0x0f,0x0a,0xff,0x69,0x1f,0x51,0x65, +0xe6,0xd1,0xf7,0x0f,0x9f,0x3d,0x0b,0x4b,0x63,0xb0,0xe5,0xbd, +0xe5,0x7f,0x7c,0xf8,0xc3,0x93,0xb8,0x38,0xc5,0x9a,0x28,0x88, +0x04,0x46,0x7b,0x6a,0x72,0x26,0xd2,0x54,0x66,0xe0,0xe8,0x17, +0xb6,0xd0,0x56,0x27,0x5a,0x50,0xf7,0x30,0xe8,0x33,0xf6,0xff, +0xf0,0xf9,0xf3,0x5f,0xf2,0x08,0x13,0x8e,0x86,0x32,0xf3,0xf3, +0xb3,0x27,0xff,0xf9,0xf8,0xa7,0xb8,0x84,0xc5,0xe7,0xa3,0xef, +0x9e,0x7e,0x1f,0x21,0x27,0xa7,0xe8,0xb5,0x5b,0xba,0x05,0xde, +0x02,0x5f,0xdd,0x6c,0x17,0x9b,0x2b,0xfc,0xef,0x18,0x3f,0xb2, +0xe3,0xf4,0xe2,0xaa,0x9a,0xcf,0xb6,0xcb,0xcb,0x63,0xda,0x97, +0x99,0xe2,0xe3,0xc9,0x46,0xbb,0xef,0xed,0x72,0x36,0x03,0x2a, +0x1c,0xf7,0x80,0x7b,0x64,0xe9,0xd9,0xd9,0xec,0x5e,0xb6,0x68, +0x2c,0x2b,0x5a,0x9c,0x54,0x22,0xce,0x80,0x0a,0xf8,0x24,0x53, +0x73,0x1d,0xd2,0x02,0x48,0x2a,0x18,0x7c,0xf0,0x0a,0xcb,0x21, +0xe2,0x1b,0x23,0xa9,0x6d,0x73,0x07,0x56,0xf9,0xeb,0xf6,0x15, +0x8c,0x8e,0xc7,0xe6,0x86,0xba,0x67,0x34,0x90,0x0a,0x7c,0x6f, +0x96,0x8d,0x68,0x10,0x6d,0x00,0xa5,0xa3,0x62,0xfc,0x02,0x46, +0xf1,0x99,0x80,0xb6,0xb3,0xdb,0x78,0x71,0x7f,0x4c,0x74,0xbf, +0x01,0xaa,0x5f,0x00,0xb3,0xbd,0x37,0x06,0x50,0xee,0x1b,0xb1, +0x5a,0x2f,0xee,0xbf,0x18,0xbf,0xb8,0x9b,0xf4,0xce,0xee,0xce, +0xea,0x7b,0x48,0x38,0x98,0x08,0xd2,0xc3,0xac,0x84,0x22,0xd6, +0xe4,0x97,0x1f,0xb7,0xfa,0x13,0x2c,0x21,0xe0,0xc4,0x80,0x94, +0xf4,0xec,0x16,0xfe,0x05,0xb2,0x94,0x04,0x7a,0xdd,0x8a,0xcc, +0xab,0x0e,0x3d,0xb7,0x25,0xd1,0x95,0xf8,0xd5,0xad,0xab,0x72, +0x8a,0xb7,0x5c,0xf7,0x5f,0x5c,0x9d,0xcd,0x28,0x81,0x85,0xb0, +0xe2,0xfe,0xe7,0xdb,0xb3,0xb3,0xfb,0xaf,0xcc,0xda,0x12,0xd1, +0x7f,0xdc,0x2c,0x91,0x35,0xdf,0x3f,0x2b,0xa2,0x81,0xa4,0xe3, +0x17,0x9f,0x27,0xc0,0xb5,0xee,0x65,0x51,0xc6,0xd9,0x04,0x1b, +0x50,0xb9,0x0e,0x6a,0x9e,0x01,0xa0,0xb3,0xe9,0xf1,0xe5,0xc3, +0xe3,0x6f,0x27,0x77,0x03,0xf3,0xe7,0x5d,0x58,0x61,0xb4,0xed, +0x67,0x50,0xc3,0x4a,0x82,0x4e,0xc9,0xf3,0xd2,0x70,0x8a,0x9c, +0x37,0xae,0xaa,0x57,0x57,0x45,0x72,0xf2,0x2e,0xe9,0x49,0xf2, +0xf1,0x9f,0xbf,0xfa,0xea,0x8b,0x3f,0xeb,0x49,0x06,0xb3,0xf1, +0xed,0x07,0xf8,0x33,0x92,0x02,0xf9,0xc9,0x29,0x7d,0x3e,0x23, +0x1b,0x6e,0x8a,0x39,0xf6,0x48,0x82,0xaf,0xa5,0x98,0xd1,0xa3, +0x06,0xb2,0xbc,0x2d,0xff,0xab,0xaf,0x1e,0xfc,0xe5,0xcf,0x5b, +0x2c,0x75,0x7a,0x3a,0x38,0x31,0x5f,0xfd,0xf9,0x8b,0x07,0x27, +0xdb,0xc1,0xc9,0x83,0x2f,0xba,0x98,0x96,0xed,0xc8,0xa8,0xde, +0x13,0x09,0x7d,0xe9,0x25,0x8e,0x31,0xd6,0xa7,0x45,0x83,0x32, +0x6b,0x0d,0x52,0x23,0x0a,0xb5,0x7a,0x18,0x71,0x76,0xf8,0x2c, +0x24,0xb8,0xf7,0x0f,0xa3,0xeb,0x61,0x35,0x53,0x1f,0x4f,0xd8, +0xba,0x80,0xb7,0x7d,0x52,0x04,0xb4,0xe8,0x3e,0xa2,0xc3,0xdd, +0x8e,0xe4,0xbf,0x42,0x8e,0xc6,0xfc,0xd1,0x62,0xcb,0x17,0x41, +0x4d,0x02,0x85,0x3e,0x9e,0x1e,0x99,0xe1,0x53,0xa5,0x78,0x98, +0xf6,0xf5,0xe1,0x30,0x79,0x94,0x7c,0xf7,0xfc,0x87,0xef,0x13, +0x4f,0x91,0xa4,0x68,0x51,0x93,0xc1,0xbc,0x33,0xd8,0xf9,0x72, +0xa7,0x42,0xea,0x27,0x59,0x78,0xb1,0x92,0x85,0x97,0x2c,0x8a, +0x47,0xf8,0x4f,0x0c,0x21,0xa6,0x05,0x5e,0x02,0x1e,0x74,0x81, +0x4e,0xeb,0x2f,0x1c,0xe5,0xca,0x3b,0x1f,0xce,0x9a,0x33,0x38, +0x4a,0x35,0x01,0xcb,0xaa,0x38,0x5c,0xb4,0x94,0x0c,0x45,0xe4, +0x82,0x90,0x8d,0x95,0x33,0x15,0x61,0xfb,0x1b,0x36,0x1c,0xff, +0x71,0xf9,0x08,0xb6,0x5c,0x32,0xd7,0x51,0x87,0x0e,0x1d,0x22, +0x7a,0x68,0xdc,0x59,0x43,0xd3,0xb7,0x78,0x0e,0x29,0x17,0xb3, +0x47,0x48,0x44,0x29,0x45,0x73,0x63,0x81,0x95,0xab,0xa7,0x49, +0x82,0xce,0xea,0x58,0xae,0xd5,0x3e,0x1d,0x77,0x55,0x91,0x1c, +0x77,0x0e,0x08,0x4f,0xec,0x6a,0xed,0x1d,0x9b,0xab,0x16,0x80, +0x51,0x9c,0xbb,0x22,0xf9,0x9a,0xb9,0xc9,0xe9,0xd7,0xf7,0xe5, +0x47,0x32,0x54,0xff,0x55,0x95,0x91,0xb1,0x06,0x72,0x51,0x02, +0x33,0xb2,0xc4,0x56,0x3d,0x87,0x7d,0x6a,0xdc,0x3e,0xa4,0xdd, +0xd1,0x90,0xe5,0xce,0xd1,0x82,0xad,0x3f,0x1d,0xa8,0xa1,0xc5, +0xfc,0x07,0x91,0xe5,0x43,0x0d,0xdf,0x47,0x74,0x3a,0x28,0x3e, +0xbf,0xaa,0x66,0x20,0x22,0x1c,0x95,0x9f,0xc3,0x18,0x20,0xf9, +0xb4,0x99,0xa7,0x39,0x89,0x69,0xe0,0xd2,0xb3,0xd7,0xdf,0x9f, +0x97,0x26,0xa5,0xc5,0xf4,0x28,0x0d,0xd1,0x71,0x61,0xc1,0x87, +0x42,0xe6,0x01,0x19,0xe6,0x7c,0x4c,0x3b,0x19,0x2e,0x8d,0x08, +0x15,0xfb,0xb1,0x40,0xc3,0xb7,0x76,0xfa,0xbd,0x13,0x13,0xe3, +0x63,0xca,0xf6,0xba,0x9f,0x03,0x0b,0x95,0x32,0x09,0x8e,0x7a, +0xca,0xd8,0x68,0xcd,0x13,0x8c,0x30,0xc1,0x43,0x63,0xd8,0xed, +0x37,0x14,0xfd,0x00,0xfb,0xb4,0x19,0xb4,0x8b,0xd2,0x58,0x39, +0x66,0xfa,0x0a,0x11,0x8b,0xb4,0x1a,0x8c,0x51,0x50,0xd8,0x96, +0x9c,0x4a,0xaf,0x3a,0x72,0x40,0xd1,0x83,0xde,0xc1,0x92,0xbd, +0x13,0x2d,0x6b,0xef,0x75,0x1c,0x9a,0x9e,0x02,0xbb,0xdd,0x10, +0xa6,0x3a,0x61,0x23,0xe4,0x51,0xa2,0x7d,0x59,0xf0,0xd9,0x4c, +0x5d,0x56,0x18,0xa0,0xd2,0x20,0xfc,0x80,0x78,0xd2,0x6f,0xd0, +0xa3,0xb0,0xfc,0xca,0xc0,0xa7,0x10,0xdf,0xf4,0x08,0x9f,0xb6, +0x2d,0x3e,0xff,0x13,0xa3,0x98,0xc9,0xca,0xe1,0x89,0xfd,0x5d, +0x65,0xe1,0xb8,0xe4,0x60,0xe5,0xe0,0x7b,0x45,0x41,0x34,0xb3, +0xe4,0x4f,0x89,0xd0,0xce,0x9e,0x1a,0x29,0xbf,0xa7,0x9b,0xed, +0xb2,0xd1,0xdd,0x2e,0xbf,0xc3,0x8f,0x8f,0x7e,0x27,0x57,0x9e, +0xe2,0x7d,0x90,0xf1,0x53,0x55,0x1f,0x5f,0x0f,0x4b,0xa3,0x55, +0xbf,0x69,0x9b,0x80,0x51,0xea,0x6b,0xcd,0xbc,0x1d,0xce,0xaa, +0x42,0x58,0x7b,0xc3,0xc8,0x68,0xf7,0x53,0x6a,0xe0,0x21,0x54, +0xe1,0xc9,0xdd,0xd3,0x3e,0x27,0xa7,0x6a,0x66,0x77,0x44,0xd8, +0xcf,0x7c,0x27,0xa7,0xd1,0xf8,0x7a,0x92,0x8f,0x27,0x00,0xb9, +0x77,0x35,0x19,0x41,0x69,0x83,0x21,0xac,0x9f,0xce,0x8a,0x6a, +0x76,0x50,0x89,0x35,0x6c,0x37,0x94,0xdf,0x8f,0x39,0xf4,0x86, +0xe0,0x98,0x06,0xd0,0xfa,0x6e,0x97,0xe5,0xff,0x9f,0x42,0xd6, +0x48,0x3c,0xab,0xe8,0x35,0x2e,0xee,0xb2,0xf9,0x6e,0x79,0xa3, +0xbb,0x96,0xa7,0xcd,0x69,0x9c,0x7d,0x7d,0x66,0xbc,0x9a,0x11, +0xde,0xdd,0x1b,0x9a,0xff,0xf3,0x67,0x80,0x5c,0x15,0x00,0x92, +0xc2,0x8b,0x1a,0xf8,0xe1,0x91,0xb4,0x96,0xe2,0xc1,0x58,0x1b, +0x38,0x8a,0x59,0x44,0x02,0x88,0x0e,0xcf,0x4e,0xa6,0x09,0x2e, +0xf8,0x8a,0xbd,0xfb,0xbf,0xb3,0x5b,0x84,0x2c,0x37,0xcf,0xe1, +0x83,0xa5,0x87,0x7c,0xd0,0x22,0xc0,0x47,0x07,0xfd,0xd5,0xa0, +0x0f,0xeb,0x78,0x9b,0xb7,0xf7,0x6c,0xa5,0xce,0x8d,0x5a,0x14, +0x9d,0x58,0x01,0xf4,0x83,0x6d,0x93,0xdf,0xfb,0x3d,0x8a,0x3c, +0x04,0x5f,0x9e,0xdd,0xad,0x9a,0xea,0xb0,0xd4,0xda,0x30,0x88, +0x05,0x7e,0xa8,0x86,0x48,0x81,0x10,0x0b,0xa9,0xbb,0x48,0x9e, +0xf5,0x10,0x8b,0x27,0xf0,0xa2,0xb1,0x3d,0x76,0xbb,0xbf,0x1b, +0xa5,0x1f,0x8d,0x4f,0x1a,0x30,0xf9,0xae,0xd8,0xc0,0x2f,0x0e, +0x2c,0x52,0x24,0x14,0xfb,0x7c,0xfe,0x2c,0x70,0x56,0x2c,0xf8, +0x24,0x10,0x9d,0x2c,0x14,0xc2,0xe9,0xbc,0x5a,0x69,0xa1,0x8f, +0x18,0xac,0xfc,0x03,0xde,0x81,0x16,0x06,0x3c,0xb6,0xfa,0x6a, +0x5c,0x9c,0x79,0xab,0xe1,0x2d,0xc6,0x09,0xbf,0xaf,0x9c,0x4c, +0x8c,0xef,0x3e,0x58,0x58,0xa7,0x6c,0xdc,0x66,0x63,0xef,0x31, +0x32,0x2d,0xfd,0x24,0xf9,0x72,0xb9,0xf2,0x6e,0xa1,0xca,0x59, +0xf1,0x39,0x6e,0x9e,0x9c,0xe8,0x09,0x9f,0xb4,0x95,0x36,0x5c, +0xd5,0x92,0xb1,0x56,0x9b,0x58,0x01,0x6a,0xbb,0x75,0xde,0x8d, +0x44,0x5b,0x6d,0x4a,0xc6,0x51,0x7e,0xc1,0x71,0xad,0xb6,0xfa, +0x76,0xc2,0xb6,0xaa,0xaf,0xa7,0x2b,0x6b,0x65,0xb9,0x05,0x81, +0x7b,0x46,0x8a,0x11,0xed,0x61,0xcb,0x1e,0xe2,0x49,0xb6,0x07, +0x14,0x6d,0xf1,0x00,0x20,0xae,0x08,0xb0,0x8a,0x8f,0x42,0x12, +0x1d,0xec,0x59,0xe8,0x56,0x99,0xf5,0xa8,0x02,0x14,0xdd,0xdf, +0x8f,0x91,0xea,0x05,0xe4,0x3b,0x74,0x74,0xbb,0x31,0x14,0xa8, +0x8c,0x9d,0x14,0x4d,0x8c,0x9c,0x25,0x67,0xc9,0xf6,0xf3,0xcf, +0x0f,0x8c,0xb0,0x5c,0x10,0xaa,0x0e,0x8d,0x50,0x8b,0x98,0x24, +0x57,0xcc,0xee,0x6d,0xef,0x9e,0xc9,0xdf,0xa1,0xfe,0x2f,0x6a, +0xc3,0xf4,0xef,0xe5,0x09,0xc5,0xe3,0xb2,0x64,0x27,0xd7,0x0b, +0xd6,0x0f,0xd2,0x92,0xa0,0x64,0x14,0x2a,0xd5,0x45,0x05,0x69, +0x7d,0x70,0xc6,0xf2,0xb7,0x1f,0xf6,0xe5,0xdd,0x96,0xe7,0x6f, +0xaa,0xcd,0xde,0xec,0xe5,0xde,0x9c,0xeb,0x3a,0xca,0x42,0xea, +0x6f,0x9d,0x57,0x1d,0x08,0xe0,0x04,0x16,0xe7,0x82,0x08,0x8a, +0xea,0x8a,0x11,0x80,0xbc,0x96,0x83,0xf2,0x35,0x7b,0xf0,0x9b, +0x66,0xfa,0xb8,0xee,0xe0,0xcc,0x7a,0x38,0x93,0xde,0x05,0x6f, +0x9d,0xc2,0x2a,0xac,0x91,0xbe,0xdc,0x42,0xf6,0x34,0x65,0x0e, +0xd7,0xec,0x43,0xba,0xc5,0x93,0x64,0xc8,0x03,0x1a,0xa5,0xb5, +0x17,0xaf,0x86,0x5e,0xf3,0x04,0xbc,0x80,0x50,0xa2,0x39,0x99, +0xc3,0x12,0x5e,0xbd,0x80,0xfc,0xa5,0x07,0x75,0x75,0xea,0x73, +0x7b,0xa0,0x75,0xe1,0x9c,0x62,0xb4,0x26,0xf2,0x26,0x9e,0x3a, +0x23,0xb8,0x69,0x7c,0xde,0xce,0xa1,0xfc,0xcd,0x0a,0x7d,0x3d, +0xcf,0x3d,0xc1,0xce,0x7a,0x59,0xa2,0xef,0xe5,0xcd,0x6a,0xbb, +0xed,0xa4,0x1d,0xfa,0x8b,0x3e,0xfa,0xf0,0xc3,0x73,0xc3,0xe8, +0xa4,0x53,0x7e,0x3f,0x53,0x60,0x1d,0x85,0x9f,0x29,0x94,0xce, +0xf2,0xe9,0x3e,0xb8,0xbb,0xdd,0xc1,0x9f,0xbb,0x7b,0x73,0xa9, +0x72,0xe6,0xbb,0xb5,0xd0,0xe8,0xd0,0x0a,0x86,0x4d,0x2f,0xce, +0x8b,0x00,0x6a,0xbc,0xd0,0x39,0xc7,0x01,0x5b,0x7f,0x88,0xa1, +0x8d,0xcf,0xe0,0x2e,0xd1,0x8a,0x4f,0xc7,0xa5,0x94,0x0c,0xd0, +0x32,0x4a,0xfd,0x8b,0x40,0x74,0xf2,0x3e,0x01,0xf9,0x53,0x4a, +0x02,0x5c,0x7b,0x47,0x7c,0x00,0x19,0x07,0x30,0x91,0x65,0xa3, +0x41,0x57,0x72,0xb7,0xdb,0x69,0x10,0xf3,0x71,0x40,0x01,0x52, +0xbc,0x24,0x6f,0xc2,0x59,0x3d,0xc3,0x97,0xbd,0x34,0x21,0xbe, +0x52,0xc7,0x4c,0xb3,0xd1,0xf1,0x20,0x3f,0x3f,0x5c,0xe8,0x3c, +0x43,0x4b,0xd4,0xfc,0x4b,0xed,0x1e,0xab,0x0c,0xf6,0xcf,0x29, +0x65,0xc7,0x73,0x26,0x4e,0xc5,0x24,0x0b,0x4d,0x81,0xe0,0x7c, +0x68,0x99,0x02,0xfd,0x04,0xb4,0xe5,0x9f,0x4e,0xcc,0x39,0xfc, +0x39,0x27,0xc7,0xc9,0xa9,0xe7,0x04,0x7c,0xd4,0xc0,0x3b,0x59, +0x46,0x4c,0x89,0x4e,0x91,0x62,0xbc,0x59,0x82,0x51,0x79,0x03, +0x84,0x61,0x40,0x29,0x4a,0x20,0xb3,0x2e,0xaa,0x88,0xd0,0x10, +0x51,0x6b,0xbd,0x86,0x47,0x34,0x19,0x7b,0xa1,0x23,0xf2,0xb4, +0xe5,0x3d,0xa1,0x61,0x36,0x5d,0xf5,0x6f,0x16,0x6c,0x5c,0x80, +0xee,0xcc,0xb6,0xf4,0x79,0x6b,0xe9,0xf3,0x96,0xd2,0x43,0xb2, +0xcf,0x46,0x28,0xf0,0x2f,0x90,0xb1,0x73,0x2d,0xab,0x46,0x21, +0x38,0x58,0xc0,0x50,0x31,0x58,0x55,0x52,0xcb,0x9f,0x2c,0x1a, +0x5c,0x5b,0x3a,0x8c,0x76,0x67,0x42,0xcc,0x0d,0xcc,0x18,0x90, +0x37,0xe1,0x67,0x41,0xed,0xf2,0x70,0xea,0x92,0x59,0xb9,0x01, +0xde,0x6a,0x2b,0xd4,0x45,0x70,0xf3,0x6d,0xdf,0x57,0xc9,0xf5, +0xd7,0xce,0x88,0xe2,0x51,0x37,0x12,0x77,0x8a,0xc0,0xdb,0xf4, +0x52,0x1d,0xb5,0x54,0x48,0x13,0x43,0x1b,0xca,0x24,0xcb,0x55, +0x1b,0xc2,0x5d,0x62,0x7d,0x85,0xcd,0xd9,0x0d,0x2b,0x34,0x20, +0xc6,0xea,0xec,0xa0,0xb5,0x57,0xd3,0xba,0x37,0xc0,0x09,0xe5, +0x52,0x0b,0x45,0xc9,0x46,0x1b,0x72,0x34,0x8a,0x14,0xf8,0x26, +0x29,0x3e,0xff,0x6c,0x80,0x82,0x80,0x01,0x4e,0xb8,0x67,0x23, +0x8d,0xe4,0x47,0xdd,0xcf,0x85,0xe3,0xab,0xe8,0xa0,0x1b,0x00, +0x47,0xb0,0x44,0xd8,0xfd,0x8d,0xdf,0xa5,0x66,0x14,0x7f,0x40, +0x22,0x1e,0x87,0x1b,0x9b,0x1b,0x36,0xd2,0xef,0x1a,0x1f,0xc5, +0xd9,0xbb,0x25,0x32,0x06,0xfa,0x6e,0xf8,0x83,0x81,0x86,0x68, +0xb4,0x41,0x6f,0xa2,0x98,0x2a,0xd8,0xa1,0xf7,0x06,0x42,0xcb, +0x64,0x59,0xb3,0x02,0x7e,0x3a,0x0a,0x5b,0xb3,0x6f,0x47,0x9d, +0x9e,0xd8,0x79,0xb3,0x1b,0x9b,0x13,0xd8,0x7d,0x5b,0x71,0x21, +0x83,0xf6,0x08,0x27,0xce,0x3a,0xe5,0xe3,0xa2,0xd3,0x34,0x63, +0x9b,0x58,0x30,0x70,0x2e,0x23,0x9a,0xa1,0x73,0x07,0x21,0x17, +0xe4,0x50,0xf5,0xb1,0xff,0xbd,0xe4,0x13,0xcd,0x3a,0x1d,0x6a, +0xc8,0x2b,0x3f,0x8a,0xc4,0x61,0x52,0xe8,0xac,0x88,0x54,0x62, +0x1c,0x5a,0x3f,0x1b,0x41,0x16,0x37,0x97,0x47,0xcd,0x35,0x35, +0xce,0xa3,0xa6,0x22,0x84,0xcf,0x51,0x29,0x75,0xd0,0x7e,0xea, +0xa6,0x12,0x99,0x50,0x43,0xb3,0xae,0x04,0x10,0xb0,0x2f,0x01, +0x74,0x4e,0x46,0xf8,0x33,0x87,0x06,0xbb,0xdd,0xb7,0x18,0x53, +0x1a,0x23,0x4b,0x5e,0x56,0x70,0x9a,0xc3,0x2f,0xf6,0x57,0xa6, +0x30,0x47,0xc6,0x77,0x1d,0x76,0x68,0xbe,0xae,0x5f,0x61,0xc4, +0xd9,0xf5,0xf2,0x16,0xfd,0xdc,0x8e,0x28,0xc8,0x45,0x9a,0x3c, +0x7b,0x0f,0x33,0xf5,0xee,0x88,0xca,0x9a,0xa3,0x9b,0xc5,0xba, +0xbc,0x58,0xbe,0x42,0x2b,0xb7,0xd9,0x11,0x12,0x56,0x59,0xd7, +0x18,0xd7,0xfc,0x28,0xe9,0x61,0x75,0xdb,0xb4,0x33,0x46,0x74, +0xed,0x5b,0xbb,0x3e,0x7b,0xd8,0x9e,0x39,0x3e,0x45,0x67,0xee, +0x01,0x3d,0x5a,0x58,0x5d,0x46,0xdb,0xf4,0x3e,0xe6,0xa6,0x27, +0xf4,0x06,0x37,0xf4,0xab,0xb7,0x9e,0xc7,0x27,0x64,0x4b,0x52, +0xb2,0xdb,0x85,0x4d,0x3d,0x1e,0xa0,0x4f,0xdc,0xeb,0xc2,0x81, +0xe5,0x1c,0x00,0xa9,0x11,0x7a,0x0f,0xde,0x76,0xca,0x56,0x57, +0xae,0x30,0x3e,0x0b,0x3f,0xc8,0x1a,0x27,0x77,0xb1,0x48,0xd2, +0x1b,0x1e,0xfd,0xdc,0xa3,0xae,0xa1,0x08,0xe4,0x49,0x42,0x9b, +0xae,0x0d,0xac,0x14,0x68,0x0d,0x10,0x3f,0x41,0xb4,0x23,0x3f, +0xcc,0xcc,0x76,0xfb,0x97,0xe0,0x6b,0x30,0x88,0x63,0x23,0x35, +0x42,0x50,0x52,0xe3,0xb8,0x06,0xe9,0x62,0x79,0x61,0x4d,0xf4, +0xe2,0x74,0x42,0x41,0xa9,0x16,0xce,0x9e,0x36,0x96,0x70,0xeb, +0x3c,0x80,0xfd,0x78,0x20,0xd8,0x56,0xaf,0x90,0x21,0xf3,0x48, +0xad,0x6f,0xed,0x17,0x01,0xa0,0x5f,0x16,0x8d,0x40,0x51,0x47, +0x76,0xd8,0x14,0xf1,0xdf,0xf3,0x36,0x26,0xbd,0x16,0x11,0xbf, +0xcc,0x64,0xd0,0x0d,0x5d,0x97,0x0d,0x3d,0xc6,0x48,0x5a,0x0a, +0x77,0xc3,0xc6,0xac,0xbf,0x2e,0xee,0xbc,0x58,0x66,0xf9,0x57, +0x27,0x46,0x62,0x6e,0xd0,0x09,0x22,0xf7,0x0d,0x5b,0xf9,0x2c, +0x92,0x5b,0x03,0x0b,0xb2,0xf9,0xc4,0x67,0xb7,0xd4,0x60,0x30, +0xbf,0x4b,0x4e,0x93,0x1c,0x0e,0xaf,0xeb,0xdc,0xf7,0x40,0x37, +0x84,0x23,0x7a,0x7f,0x27,0x39,0x6a,0xc9,0x87,0xe4,0x9e,0x4d, +0x5e,0x97,0x6f,0xab,0xe5,0x4d,0x2d,0x88,0x0b,0xea,0xfe,0x73, +0x5f,0xa1,0xdd,0xce,0x79,0x23,0xe6,0x77,0x64,0x1a,0xe1,0x56, +0x33,0x19,0xe0,0x05,0x9e,0xc3,0xd6,0x53,0x19,0x4d,0x3d,0x0f, +0x5a,0x26,0x72,0xa9,0x2f,0x26,0x85,0x84,0x94,0xf8,0x72,0xb2, +0xdd,0xf2,0xaf,0xaf,0xe0,0x57,0x92,0x64,0x87,0xab,0x27,0xff, +0x2c,0x12,0x35,0x77,0x1d,0x3f,0x98,0xa8,0x57,0x20,0x36,0x88, +0x9e,0xea,0xfa,0xd1,0xa3,0x97,0xb7,0xd8,0x28,0x56,0x1d,0x4c, +0xbe,0x04,0x1e,0xc2,0x46,0x1a,0x1f,0x3d,0x92,0x80,0x65,0x9b, +0x64,0xb1,0xb9,0x72,0x9d,0x0f,0xac,0x29,0xad,0xf9,0x22,0x1b, +0x59,0x30,0xb6,0xdb,0x20,0x7a,0x82,0x9a,0xdd,0xea,0xc0,0xbf, +0x9c,0x14,0x3d,0x3b,0xf2,0x91,0x0e,0x5c,0x93,0xfe,0x0c,0xd5, +0x07,0x59,0xfe,0xe0,0x5e,0x4a,0x0f,0xd8,0xba,0xce,0xb0,0xdd, +0x64,0x39,0x9b,0xf9,0x29,0x1a,0x64,0x03,0xea,0xdb,0x36,0xff, +0xdb,0x44,0x50,0xf0,0x6f,0xad,0x15,0x72,0xfd,0x69,0x4d,0xbb, +0x5b,0xa1,0xdc,0xa9,0xad,0x4b,0x8c,0x28,0x62,0xaf,0xef,0x2e, +0x80,0x33,0x9b,0x9b,0xc5,0xaf,0x28,0x0c,0xcd,0xc4,0x88,0x1f, +0x80,0x10,0x8b,0x72,0x98,0x95,0xa1,0x8f,0x50,0xda,0xe2,0x08, +0xef,0x2c,0xcf,0xd8,0xbe,0x46,0xb8,0x63,0xe4,0x31,0x32,0x1e, +0x28,0xfa,0xbf,0x44,0xa5,0x36,0xf7,0x01,0xd2,0x12,0x9f,0xbe, +0xb9,0x05,0x4d,0xa6,0x37,0x17,0x08,0x1a,0x67,0x1f,0xad,0x79, +0x14,0x0e,0xdb,0xe5,0x6b,0xb2,0x7b,0xe0,0x3e,0x4b,0xec,0x10, +0x44,0x56,0x39,0xe6,0xb2,0xd9,0x71,0x94,0x6e,0x5d,0x4f,0xdb, +0xcc,0xa8,0x8d,0x54,0x32,0x16,0x7a,0x5b,0x3b,0x2a,0x11,0x93, +0xe3,0x17,0xe8,0x5b,0xa9,0x8f,0xc3,0xdd,0xa1,0xe5,0x52,0x70, +0x2b,0x4f,0x4a,0x44,0xa1,0x4c,0xd6,0x00,0x6b,0xea,0xa8,0x11, +0x09,0x1b,0xfd,0xe7,0x6d,0x25,0x5b,0xee,0xe0,0x42,0x8a,0x48, +0xfb,0xc0,0x1d,0x8b,0x36,0x27,0x1a,0xfc,0xf6,0xc8,0x44,0x1e, +0x78,0x18,0xf2,0x92,0x4d,0xbf,0x9a,0x9a,0x59,0xa6,0xa0,0x15, +0x48,0x2f,0xe5,0x7a,0x51,0x38,0x53,0xd3,0xb1,0x2d,0xc1,0xc1, +0x71,0x6c,0x90,0x12,0x2a,0x08,0x52,0x94,0x56,0x89,0xde,0xf4, +0x0f,0x35,0x6d,0x59,0xd2,0xf3,0xda,0x89,0x4c,0x8f,0xe8,0x49, +0x7f,0x34,0x7d,0xb6,0x9d,0x7a,0x0a,0xe3,0xf6,0xe1,0x4b,0xa7, +0x5e,0x78,0x79,0x5b,0xc3,0x5e,0xe1,0x34,0x24,0xa8,0x0f,0xdf, +0x7b,0xa4,0x09,0x35,0x83,0x2f,0x5c,0x60,0x80,0x4d,0x40,0x57, +0xc8,0x5e,0xe9,0x05,0x25,0x35,0xc0,0xe4,0x70,0x24,0x87,0x82, +0x46,0xc9,0x1b,0xc2,0x9e,0x78,0xeb,0x49,0xb5,0xe1,0x73,0x80, +0x5c,0x74,0x84,0xea,0x2c,0x8c,0xcb,0x2c,0x5d,0xe4,0xfa,0x63, +0x24,0x72,0x54,0x0f,0xa5,0x84,0x24,0x28,0x33,0x92,0x5e,0xc4, +0x93,0x34,0x8f,0x9a,0x18,0xd9,0x47,0xc5,0x24,0xfb,0x45,0x98, +0x4d,0xa9,0xdd,0xee,0x89,0x15,0x89,0xec,0x3a,0xe4,0xe1,0xe5, +0xc9,0xbd,0xd6,0x0a,0xad,0x85,0x4f,0xe1,0x50,0x9b,0x7c,0x76, +0xa8,0x3c,0x2f,0xb1,0x63,0x4a,0xd4,0x45,0xec,0x40,0xff,0x67, +0x58,0x15,0x1f,0x69,0xec,0xc9,0xc0,0x71,0xd7,0x68,0xeb,0x6d, +0x7b,0x08,0x19,0xf8,0x5c,0x8c,0xd7,0xed,0x89,0xf1,0xfb,0xed, +0x0d,0x6c,0xcf,0xbd,0xe4,0x38,0xc9,0xf1,0x1d,0x04,0x0c,0xcc, +0x11,0xef,0x44,0xe4,0xf0,0x71,0x7b,0x35,0xdd,0xd8,0xb0,0x57, +0xbc,0x4d,0x1b,0xb2,0xa5,0xe3,0x07,0x97,0xd1,0x93,0xb6,0x2c, +0x68,0x17,0x12,0xc3,0x0e,0x8f,0x99,0xe0,0x8b,0x2a,0xb7,0xd3, +0xf5,0xac,0x48,0xb0,0x46,0x54,0xe0,0xf8,0xcb,0xcc,0x2c,0x2f, +0x49,0xe0,0x4b,0xc4,0xda,0x10,0x07,0x84,0xfd,0xf9,0x91,0xba, +0xa9,0x47,0x9e,0x26,0x6c,0x64,0xd4,0xba,0x2c,0x3a,0x71,0x70, +0xcb,0x0f,0xc6,0x99,0x69,0x84,0x98,0x21,0x69,0x14,0x83,0xbe, +0xd1,0x2f,0x0a,0xdb,0x65,0xea,0xcd,0x74,0xbd,0x41,0x73,0xf4, +0x82,0xc7,0x09,0x03,0x90,0x11,0x8d,0x12,0x4f,0xec,0x4b,0x5a, +0xe4,0x19,0x06,0xa5,0x88,0xc0,0xa2,0x05,0x50,0xf0,0xa8,0x0f, +0xf2,0x2e,0x73,0x53,0xb3,0xbd,0x7a,0xd1,0x01,0x90,0xbb,0xdd, +0x0e,0xd7,0x41,0x61,0x98,0x5b,0x23,0xe1,0x96,0xa1,0x12,0x91, +0x1f,0xc0,0x1c,0xf2,0x4f,0x2b,0x2f,0xb2,0xe4,0x88,0xff,0xb8, +0x38,0x35,0xdc,0x10,0x5b,0x54,0xed,0xe7,0x9b,0x78,0xb2,0x52, +0x59,0x3a,0x3e,0x7d,0x77,0x06,0x43,0x42,0x0c,0xfa,0x54,0xe1, +0x0b,0x6e,0xf3,0xf7,0x49,0xa1,0x46,0x3e,0x1d,0xca,0xe9,0x76, +0x03,0xf4,0xec,0xec,0x8e,0x80,0x30,0x53,0xd5,0xb1,0xe2,0x91, +0x47,0xe3,0x89,0xd7,0xb9,0xa4,0x58,0xd3,0x9a,0x89,0x52,0x51, +0xb7,0xab,0x48,0xe1,0x61,0x7a,0xa1,0x74,0xb8,0x8e,0x1f,0x4c, +0x27,0x4a,0xa1,0x70,0x3a,0x8d,0xf0,0x36,0xe4,0xd2,0x42,0x01, +0x61,0xed,0x9c,0x17,0x61,0x68,0x1d,0xf4,0x43,0xd0,0x70,0x3b, +0x83,0x89,0xc4,0x1a,0xdc,0x5b,0xe2,0x01,0x37,0x54,0xd8,0xd6, +0xba,0x5d,0x19,0x8d,0xb3,0x0f,0x1c,0xdb,0xcc,0x09,0x4f,0x4f, +0xaf,0xe7,0x15,0x77,0xb7,0xcd,0x34,0x63,0x30,0x10,0xea,0xd2, +0x81,0x77,0x82,0x51,0xce,0x10,0x85,0xe8,0x86,0x90,0x66,0x36, +0xf4,0x50,0x30,0x51,0xdd,0x6e,0xaf,0x87,0xf5,0xb8,0x25,0xf5, +0x1c,0xb8,0x8b,0x47,0x4e,0x91,0x7d,0xc8,0x97,0xc4,0x91,0x3c, +0x56,0x9b,0xa8,0xb7,0x8b,0x9e,0x5d,0x14,0xef,0xf8,0xfe,0x38, +0xe1,0x30,0x8c,0x55,0xd4,0x16,0xbb,0x28,0xb3,0xfe,0x42,0x4d, +0x74,0x65,0x1e,0x1a,0x07,0xec,0x44,0xce,0x07,0x9d,0xf4,0x8f, +0x23,0x04,0x05,0xa3,0x8f,0xa0,0xf1,0x8e,0xd2,0x78,0x27,0xa6, +0xf1,0xed,0xb6,0xc3,0xc8,0x83,0xae,0xbc,0x71,0x13,0x68,0xfe, +0x88,0x83,0x6f,0x6f,0xc4,0x0e,0xa9,0x84,0xca,0x8c,0x70,0x2b, +0x0a,0x2d,0x38,0x58,0xdb,0xdd,0x0f,0x73,0x8f,0x89,0xa5,0x31, +0x55,0x09,0xa7,0xdb,0x6e,0x4f,0x08,0x51,0x97,0x97,0xff,0x55, +0x38,0x1f,0xfe,0xbe,0x4f,0xbf,0x4f,0x8b,0x13,0x7c,0xe6,0x32, +0x96,0x83,0x59,0x0c,0x75,0xa1,0x09,0xf9,0x6e,0x65,0xfd,0x0a, +0x03,0x94,0x4b,0xf4,0x3d,0xbe,0x26,0x1a,0xf3,0xdf,0x89,0xc4, +0x76,0x70,0xfe,0x45,0x63,0x95,0x64,0x7d,0x14,0xc5,0xa7,0x86, +0x04,0xc6,0xc4,0x9a,0x89,0x72,0x76,0xc4,0x15,0x50,0x0f,0xc2, +0xbf,0x9c,0x59,0x86,0x8b,0x6a,0x3d,0xba,0xf4,0x02,0x34,0xe6, +0x97,0x0b,0x17,0xc6,0x09,0x93,0xeb,0x42,0x7a,0x95,0x4b,0x2c, +0x03,0xfb,0xbb,0x96,0x9e,0x98,0x08,0xc0,0xfe,0xd5,0xb4,0xfe, +0xe9,0x76,0xf1,0xb3,0xbc,0x3e,0x94,0xb6,0xc1,0x9b,0x8d,0x3e, +0x25,0xd2,0x63,0x35,0x7b,0x67,0xc3,0x01,0x5e,0x4a,0xa1,0x46, +0x70,0x47,0x2b,0x6e,0x4b,0x0c,0xa9,0xd9,0xbb,0xa2,0x19,0xc3, +0xc4,0x79,0xd5,0x66,0xe4,0x3b,0x39,0x86,0x62,0x7e,0x84,0x47, +0xfa,0xf4,0x0a,0xed,0xb2,0x3d,0x76,0x57,0x97,0xb2,0x57,0x9d, +0x20,0x20,0x35,0x95,0x5b,0xe0,0x91,0x97,0x27,0x0f,0x5f,0xb8, +0xdd,0xe4,0xfb,0x86,0xa8,0x11,0x95,0x68,0x68,0x78,0x89,0x4c, +0x77,0xfb,0xd6,0xa4,0x58,0x7d,0xe3,0x8b,0xd8,0x71,0xad,0x2d, +0xb6,0x41,0x36,0x0c,0xe3,0x6f,0xb9,0x09,0xfd,0x30,0x82,0x0f, +0xc4,0xbf,0x72,0x3e,0xf7,0xea,0xd8,0x4c,0x15,0x49,0x41,0x8b, +0x0e,0xfe,0x18,0xae,0xa8,0xf2,0xfd,0xf8,0xfc,0xd8,0x32,0x71, +0xc8,0x2b,0x46,0x74,0x80,0x66,0x1b,0x1c,0x26,0x3b,0xb4,0xf7, +0xdb,0x20,0x21,0x80,0x23,0x64,0x4a,0x54,0x42,0x01,0xa2,0x54, +0x07,0x91,0x6a,0xf0,0x4c,0xc7,0x5a,0x60,0x23,0x8b,0xd9,0xd1, +0x63,0xfd,0xf5,0x07,0xe7,0xe2,0xa0,0xa9,0x58,0xec,0xc2,0x4b, +0x99,0x4e,0x41,0xbd,0x73,0x3a,0xe3,0x3d,0xfd,0x04,0xc6,0x24, +0xad,0x7d,0xa4,0xb1,0x4a,0x4b,0x74,0xab,0x64,0x70,0xf0,0x9c, +0x82,0xa4,0x07,0xaa,0x2a,0x27,0x65,0x52,0xdb,0x18,0x28,0x05, +0xc0,0x98,0x4f,0x17,0xaf,0xf6,0x80,0x80,0x59,0x2e,0x4e,0xbf, +0x73,0xd0,0xe1,0xc3,0x09,0xe6,0xd2,0x69,0xe2,0x10,0x1f,0xa1, +0xd6,0x81,0x8b,0x50,0x53,0xd4,0x57,0x81,0xff,0xfc,0x81,0x63, +0xa2,0xd2,0xdb,0xf7,0xd0,0xcc,0x70,0xb6,0x3c,0x92,0xb0,0xe6, +0xf8,0x59,0x84,0xd6,0x33,0x2d,0x87,0x20,0x98,0xf5,0x1c,0xfb, +0x4f,0x5a,0xf5,0xd0,0x09,0x67,0xe5,0x94,0x43,0x10,0x7b,0x67, +0x52,0xea,0x40,0x7f,0x44,0x20,0xda,0xfc,0xa2,0x60,0xac,0x9c, +0xc8,0x9e,0x4c,0x65,0x15,0xeb,0x98,0x85,0xf2,0x78,0x36,0x84, +0x33,0x22,0xd0,0x86,0xa7,0x7d,0xf4,0xa2,0xb6,0xb3,0x6b,0x75, +0xa0,0x22,0xcd,0xdc,0x1d,0x32,0x4c,0x18,0x3f,0xc9,0x14,0xf3, +0x18,0x7e,0xad,0xa5,0xbe,0x52,0xbf,0xcb,0xf9,0xf2,0x42,0x1e, +0xf9,0x8c,0x12,0x90,0xd5,0x5a,0x7b,0x5b,0xfc,0xdd,0xed,0xe2, +0xbf,0x22,0xb4,0xd3,0xc9,0x81,0xa9,0x68,0xb6,0x33,0xeb,0xe5, +0xb2,0xd1,0x51,0x18,0xe3,0x47,0x2e,0xb2,0xf5,0x4d,0xec,0x0f, +0x95,0x6d,0x7d,0x89,0xda,0x05,0xcf,0x47,0xd8,0xbe,0xc5,0x86, +0xdc,0x45,0x96,0x4d,0xa2,0xad,0xbf,0xd3,0x11,0xa2,0x27,0x25, +0x2b,0xfd,0x44,0x93,0xd7,0xed,0xf6,0x9f,0x9c,0x3c,0x3d,0x27, +0xa1,0x01,0x8e,0xb7,0x62,0xb8,0x72,0xc8,0x04,0x56,0x0d,0x5a, +0xe8,0xc9,0x14,0xf4,0x4a,0xad,0x3f,0xa9,0x0a,0x3e,0x43,0xce, +0xf6,0x3f,0x6d,0x73,0x61,0x35,0x23,0x07,0x0e,0x01,0xcd,0xf8, +0xce,0x56,0xed,0x21,0x07,0x1e,0xb5,0x6a,0xda,0x26,0x6c,0x41, +0xd5,0x5a,0x48,0x2d,0x9a,0x76,0x46,0x7f,0x1d,0x1a,0x82,0x7f, +0x6b,0x1f,0x25,0xd8,0x96,0x58,0x54,0x0c,0x1a,0x97,0x11,0x97, +0xd7,0xab,0xcd,0xfb,0xb8,0xf9,0xdf,0xa3,0x49,0xd7,0xd7,0x08, +0x74,0x34,0xa7,0xc9,0x7f,0x4f,0xb6,0xdb,0x2f,0x9a,0x41,0x36, +0xbf,0x6c,0xac,0x86,0x66,0x20,0x47,0x39,0x8d,0xb5,0x0f,0xbb, +0xe3,0x8b,0x48,0x7d,0x1a,0x80,0xa8,0xee,0x0d,0x7b,0x51,0xed, +0xc1,0x96,0xf8,0x58,0xf5,0x5b,0x1e,0xfa,0xdb,0x19,0x9a,0xb2, +0x7d,0x15,0xd9,0x95,0xab,0xbd,0x22,0x3b,0x71,0xb5,0x52,0xcc, +0xef,0x8b,0x06,0xee,0xc5,0xe1,0xf6,0x96,0x46,0x14,0x9d,0x1b, +0x1f,0x4f,0x7b,0xd7,0xca,0x33,0x50,0x77,0xd3,0x68,0xf9,0x00, +0x14,0xd0,0x21,0x36,0x15,0x74,0x87,0xd1,0x27,0x49,0xd7,0x93, +0xd2,0x45,0x67,0x0b,0x63,0x65,0xfb,0x72,0x60,0xba,0x58,0xa0, +0x71,0x1a,0xb5,0x0d,0xe1,0xe3,0x5a,0x74,0x4f,0xb0,0x2f,0x5e, +0x78,0xac,0x84,0xc4,0xc0,0x9d,0x19,0xa9,0x2b,0x3e,0x58,0x85, +0x64,0x00,0x89,0xd6,0x6d,0x34,0x66,0x85,0xb4,0x22,0xba,0xd8, +0x01,0x36,0x56,0xfe,0xfa,0x7b,0x9a,0x32,0x71,0xa4,0xf2,0xf1, +0xc9,0xa9,0x26,0x8d,0xf4,0x47,0x4f,0x02,0x1e,0x5b,0x31,0x78, +0xc7,0xaf,0xee,0xfe,0x3e,0xd8,0x3d,0x07,0xf3,0xa1,0xc8,0x15, +0xe8,0x53,0x5e,0x3c,0xc8,0xfc,0xe2,0x7a,0xe5,0x17,0xc8,0x7c, +0x92,0x07,0xdd,0x2f,0x67,0xb3,0x3f,0xda,0xfb,0xe0,0xf7,0xf7, +0x3e,0xff,0x5d,0xd3,0xe6,0xe1,0xda,0xc3,0xc1,0x87,0xd1,0x3d, +0x3c,0x3e,0xae,0xe0,0x88,0x35,0xfc,0x14,0x08,0x5f,0xfd,0xaf, +0x85,0xb0,0xd7,0xab,0xbe,0x16,0xe1,0xf8,0x13,0xa0,0xdc,0xed, +0xe8,0x32,0xb3,0x02,0xa9,0xf7,0x8e,0xde,0xc6,0xcc,0x3b,0xa2, +0x10,0x3c,0x5f,0xbe,0xc3,0xdf,0x97,0x20,0x6e,0xe0,0x5f,0xf4, +0x89,0xb9,0x5d,0xae,0x67,0xf8,0xbb,0xba,0x9e,0xbe,0xc2,0xc4, +0x5d,0x16,0x1c,0x21,0x41,0xcc,0x6e,0x3e,0x64,0x50,0x65,0xae, +0x03,0x7e,0x38,0x17,0x5b,0x00,0xa1,0xb9,0xdc,0x1c,0x68,0x21, +0x78,0x86,0x00,0x9a,0x90,0xb3,0x49,0x61,0x6d,0x33,0xf8,0xb3, +0x21,0x59,0x73,0x7c,0x6f,0x0d,0x51,0xe3,0x05,0x09,0xc2,0x83, +0x4e,0x39,0x0f,0x62,0x4b,0x61,0x92,0x44,0xbd,0x0f,0x42,0x42, +0x34,0x03,0xdf,0x77,0x24,0xf2,0xbd,0x7b,0x8d,0x04,0x0e,0xf9, +0xf4,0xb7,0x25,0xd0,0x4d,0xa6,0xef,0x83,0x04,0x47,0x15,0xe9, +0x48,0x4e,0x13,0x5e,0x84,0x37,0x2a,0x4a,0x07,0x44,0x2e,0xe2, +0x4e,0x54,0x7e,0x88,0x23,0x9a,0x43,0x81,0x22,0x8f,0xc6,0x11, +0x64,0x0e,0xdb,0x46,0x94,0xc6,0x11,0x78,0x3f,0x2d,0xb8,0x9b, +0xbd,0xc8,0xe7,0xb6,0xc5,0xd4,0x5d,0x77,0xc5,0xc5,0xe6,0xaa, +0x08,0xb7,0xc9,0x5f,0x7d,0x77,0x0e,0x0c,0x7d,0x66,0x8f,0xea, +0xee,0xa9,0x7e,0xbf,0x46,0x7c,0xa2,0xa7,0x5b,0x13,0xf7,0xe9, +0x3b,0x87,0x82,0xc0,0xec,0xdb,0xc5,0xc8,0x53,0x63,0xf8,0xdb, +0x3e,0xc6,0x89,0x91,0x69,0xb8,0x8c,0x7d,0x7a,0xb4,0xed,0xce, +0xdb,0xcf,0x1c,0x27,0x79,0x32,0x29,0xbc,0x04,0x0b,0x99,0xa4, +0xb1,0x0d,0x47,0xd1,0xb0,0xe8,0xd0,0xfc,0x4d,0xd3,0xaa,0x41, +0xb3,0xc8,0x07,0x14,0x83,0x9a,0xf8,0x0e,0xb8,0x26,0x7a,0x9d, +0xbd,0x88,0x6c,0x8e,0x76,0xf2,0x12,0x1a,0xbb,0xd3,0xc1,0x41, +0x67,0x53,0xcd,0x8b,0xfb,0xff,0x89,0x7f,0xd0,0xdf,0x9b,0x45, +0x96,0x1a,0x15,0xcb,0xec,0xf4,0x2d,0x09,0x18,0x23,0xf6,0x2d, +0x7c,0x52,0xc1,0x2d,0x99,0xe0,0xdf,0x37,0xf4,0x20,0x2e,0x2a, +0xe2,0xef,0xbf,0xe8,0x8f,0x5f,0xe4,0x7f,0x3a,0x1b,0x9f,0xf5, +0xcd,0xe4,0x1e,0xb6,0xe3,0x3b,0xd1,0x07,0xc3,0xe7,0x9b,0x3f, +0x3f,0xdb,0xbc,0xba,0x99,0xae,0xa7,0xf0,0xb3,0x9c,0x49,0xd0, +0xb9,0x3b,0xd2,0x5b,0x42,0xbf,0xc4,0x31,0xf8,0x14,0x59,0xe3, +0x6f,0x14,0xd5,0x88,0x63,0x00,0x30,0xb8,0xc4,0x87,0xcd,0x67, +0x52,0x2f,0x83,0xa7,0xfc,0x22,0x15,0x05,0xda,0x7d,0xa0,0x38, +0x7a,0xd9,0x88,0x7d,0xe2,0xd9,0x6c,0xec,0x7d,0x92,0x0b,0x2b, +0xf2,0xdb,0xfa,0xfc,0x3c,0x31,0xac,0x8d,0x67,0x9b,0xe9,0xc5, +0x1b,0x79,0x39,0xd9,0x75,0x26,0xf4,0xe9,0x0b,0x06,0x1c,0x4d, +0xe9,0x24,0x8c,0xb1,0x12,0x3f,0xa4,0x8f,0x1d,0xa0,0x19,0x23, +0xbd,0x97,0xe7,0x42,0x97,0x8b,0x61,0x0c,0xda,0xac,0xb0,0xd3, +0x8c,0xd7,0x8a,0x47,0x9a,0xde,0x3b,0x81,0x1c,0xbb,0x05,0x47, +0xeb,0x9b,0x68,0x14,0x11,0xd8,0xd8,0xca,0x60,0x14,0xd0,0x21, +0x76,0x92,0xe5,0xf8,0x0f,0x56,0xb6,0x24,0x5d,0xf0,0x53,0xc1, +0x36,0x72,0x73,0xf0,0x85,0xbc,0x2c,0x87,0x03,0x77,0xcf,0xf6, +0x4f,0xe6,0x20,0xa8,0xab,0xf0,0xbc,0x65,0xe8,0xf1,0x4c,0x8d, +0xef,0x45,0x5f,0x75,0xa1,0xef,0x6c,0xf2,0x6b,0xbe,0x34,0x6a, +0x9e,0x18,0xce,0x8f,0xbc,0x30,0xa9,0xd3,0xdf,0x89,0x59,0x9e, +0x35,0x6e,0x16,0x39,0xa2,0x87,0xdc,0x9d,0x59,0xf8,0xc7,0xc9, +0x86,0x22,0x25,0x42,0x1a,0xac,0x9e,0xc5,0xf2,0x96,0x1b,0xb4, +0x03,0xee,0x0c,0xb2,0xcc,0x5e,0x60,0xff,0xc1,0xa6,0xe8,0x7d, +0xf9,0xaa,0xde,0xdf,0x4c,0xa7,0xe3,0x3d,0xd3,0xbb,0xff,0xad, +0xdb,0xf5,0xa1,0x80,0x5d,0x11,0xc5,0x32,0x41,0xdb,0xd0,0xb0, +0xa4,0x1a,0x48,0xe5,0x01,0xc6,0x0c,0x84,0x93,0xdc,0x92,0x19, +0x61,0x3f,0xa8,0xe6,0xd4,0x47,0xb9,0x3f,0x43,0x6e,0x31,0x68, +0x36,0x0c,0xeb,0x62,0xbe,0xac,0x01,0x90,0xe6,0xd8,0x6a,0xe7, +0xa8,0x64,0x23,0x10,0x8b,0x51,0xf4,0xdc,0x5f,0xa7,0x46,0x16, +0xc1,0x6a,0x59,0x17,0x07,0x06,0x88,0x8e,0x01,0x7b,0x97,0x73, +0x1d,0x8f,0xde,0x76,0xbe,0xdd,0x06,0x78,0xc8,0x81,0xa8,0x84, +0xa4,0xd4,0x74,0x59,0x56,0xd6,0x90,0x1e,0x7e,0x42,0x0b,0xe6, +0xe8,0xf5,0x39,0x48,0x72,0x2f,0x8f,0x89,0x4d,0x29,0xbd,0xde, +0xa4,0x66,0x65,0x1a,0x61,0x1b,0x83,0x3a,0x0b,0x9e,0xd1,0xda, +0x19,0xaf,0x56,0xbd,0xa5,0x1c,0x5b,0xd0,0x62,0x19,0xe3,0x06, +0x47,0x94,0x20,0x7b,0x33,0x9a,0x4a,0xf3,0xbd,0x49,0xd3,0xa8, +0x7a,0xd7,0x4e,0x77,0x58,0xd9,0x2a,0xc5,0xf7,0x2c,0xff,0x1d, +0x47,0x93,0x3a,0x70,0x64,0x1f,0xb5,0xda,0xb8,0xfd,0xbf,0xd4, +0xbd,0x79,0x7f,0x1b,0x47,0x92,0x28,0xf8,0xff,0xfb,0x14,0x60, +0x0d,0x07,0x5d,0x65,0x14,0x41,0x52,0x76,0xf7,0x74,0x17,0x5d, +0xc2,0xa3,0x29,0xa9,0xad,0x19,0x5d,0x23,0xd2,0xed,0x9e,0x05, +0x61,0x3d,0x90,0x00,0x49,0xc8,0x20,0x00,0xa3,0x40,0x1d,0x4d, +0xe0,0xbb,0x6f,0x9c,0x99,0x91,0x59,0x05,0x4a,0xee,0x37,0xfb, +0x76,0xf7,0x67,0x8b,0xc8,0xca,0x23,0xf2,0xce,0x8c,0x88,0x8c, +0x43,0xe1,0x4d,0x66,0xc7,0xcb,0xe5,0xf0,0xb3,0x2e,0x22,0x75, +0xa9,0xcb,0x1c,0xbf,0x22,0xca,0x44,0x94,0xd6,0x7b,0x52,0xdd, +0x21,0xf6,0x18,0x14,0x20,0x4c,0x84,0xd7,0x57,0x21,0x30,0xda, +0x6d,0x09,0x58,0x45,0x5b,0x59,0x72,0x40,0x9d,0xa5,0x19,0x39, +0xfe,0x45,0x9d,0x1f,0x5d,0x73,0xe8,0x08,0x2b,0x1f,0x8e,0x1a, +0xee,0x83,0x50,0x27,0xb1,0x42,0x69,0xc0,0xad,0x1b,0x29,0xde, +0x2b,0x6e,0x79,0x48,0x2f,0x6e,0x87,0xbf,0x8e,0xb9,0x1f,0x7e, +0x5f,0x3a,0x46,0xb7,0x37,0x94,0xaf,0x51,0x83,0xc2,0x3b,0x99, +0x1b,0x02,0x79,0xaa,0x50,0xc6,0x70,0x38,0xf1,0x31,0x0b,0x87, +0x14,0xda,0x0b,0x33,0x87,0x77,0xe3,0x7d,0xa3,0x73,0x36,0xc4, +0xcb,0x98,0xfa,0x89,0x76,0x24,0xbf,0x74,0xfc,0xa0,0x4b,0x67, +0x26,0x8b,0xc3,0xd3,0x1c,0xc7,0xee,0x35,0xf9,0xce,0x2b,0xa2, +0xef,0xda,0x8e,0xce,0x42,0x07,0xe5,0x80,0x52,0x9e,0xe2,0xcd, +0x68,0x62,0xb8,0x29,0x79,0xe0,0x2e,0xbe,0x99,0x19,0xc2,0x42, +0x30,0x4d,0x8f,0xd6,0x5e,0xf2,0x65,0xe9,0xe5,0xb3,0xe5,0x5d, +0xd3,0x8d,0xaa,0x00,0x65,0x81,0x5b,0x41,0x55,0xb6,0xac,0x59, +0x69,0xcc,0x68,0x22,0x42,0x21,0x56,0x46,0x30,0x73,0x85,0x09, +0xbf,0x89,0x38,0xf9,0x93,0x9c,0xb0,0xa4,0xaf,0x82,0x24,0x59, +0x37,0x8c,0xaa,0x34,0xb7,0x44,0x14,0x17,0xa4,0xb0,0x7d,0xb2, +0xce,0x36,0x8c,0xda,0x7c,0x4d,0xb9,0xf8,0xed,0x5f,0xea,0x84, +0xf5,0xff,0xb5,0x03,0xd0,0x50,0xf3,0xef,0x28,0xbd,0xa5,0xfe, +0x7f,0x6a,0x00,0x6d,0x4b,0x24,0xef,0xff,0xe0,0x06,0xfd,0x73, +0xf3,0x11,0x8b,0x45,0xe8,0xa4,0xc8,0x08,0x7e,0x61,0x89,0xe8, +0x38,0xa7,0xd1,0x8a,0x5c,0xaf,0xef,0x37,0x99,0x61,0x24,0xaa, +0x38,0xbd,0xc3,0x56,0xbf,0x0a,0x6a,0xc4,0x8d,0xc4,0xf2,0x8a, +0xe1,0x3e,0x58,0x5e,0x19,0x5f,0xd2,0xc7,0xc9,0xd5,0x12,0x3e, +0x92,0x8c,0xdf,0x13,0x04,0x44,0x28,0x9f,0xaf,0xb1,0x3f,0x87, +0x36,0x0f,0x8b,0xe0,0xc4,0x11,0xe2,0xd5,0x88,0x0a,0x90,0x2c, +0x5e,0x20,0x81,0x85,0x0e,0xf1,0xdc,0xee,0x16,0x51,0x78,0x97, +0x83,0xc6,0x36,0x0f,0x11,0x6d,0xbc,0xa5,0xdd,0xe9,0xb8,0x60, +0x0c,0xe7,0x6a,0x26,0xd3,0xe0,0x30,0x52,0xfa,0xe4,0x8b,0x9b, +0x24,0xb4,0xc8,0x1e,0xb0,0xf3,0x1e,0x8e,0x59,0x73,0x7f,0xa2, +0x6e,0x3d,0x9f,0x01,0x09,0x32,0xd5,0xc5,0x18,0x0a,0xde,0x65, +0x84,0xcb,0x5a,0x2c,0x02,0xed,0xf9,0xee,0xc4,0x54,0x07,0xf7, +0x6b,0xcb,0x7d,0x98,0x47,0xa5,0x2d,0x9d,0x64,0x7a,0x20,0x6d, +0xc1,0x0b,0x16,0x7d,0xd0,0x2f,0x59,0xbd,0xa1,0x7e,0xfd,0xda, +0x13,0xd4,0x13,0x2e,0x21,0xf6,0xe8,0x54,0x83,0x50,0x4e,0xc2, +0xbf,0xb9,0x41,0x98,0x64,0x39,0x81,0xee,0x4c,0x0a,0xf8,0x48, +0xc9,0xae,0xca,0x92,0xed,0xb6,0xe9,0x1b,0x8d,0xf3,0x05,0xfe, +0x10,0x4e,0x41,0x19,0xf1,0x6e,0x26,0x8d,0x99,0x5e,0x5f,0xbf, +0xd1,0x72,0x45,0x13,0x32,0x62,0x5a,0x94,0x91,0xb5,0xcc,0x68, +0x5b,0xa2,0x1f,0x30,0xd9,0x98,0x8a,0xc3,0xe9,0xd3,0x2b,0xb2, +0x43,0xee,0x96,0xc6,0x75,0x15,0xe1,0x4f,0x7f,0x89,0x90,0x23, +0xe8,0x17,0x93,0xa3,0x65,0x69,0x74,0xcf,0x63,0x14,0x6a,0xbd, +0xde,0x91,0x1b,0x19,0x71,0x1f,0x20,0x7a,0xb9,0x10,0x0a,0x39, +0x60,0xf7,0x43,0x80,0xfa,0x9e,0xee,0x70,0xa5,0x5c,0xb0,0x24, +0x6e,0x46,0xf0,0xe2,0x3c,0x72,0xe7,0x83,0x11,0x40,0xcc,0xfd, +0x5b,0x02,0x2d,0x6c,0x34,0x94,0x35,0x3b,0x9a,0x95,0xb3,0xe0, +0xd9,0x80,0x44,0x61,0x4c,0xb5,0x33,0x91,0xbc,0x80,0x85,0xc2, +0x55,0xcf,0xfc,0xaa,0x87,0xa9,0x3f,0xb2,0x4f,0x31,0x55,0x99, +0x0c,0x2f,0x2e,0x96,0xeb,0xe1,0x72,0x35,0xb9,0x9c,0x8e,0xd7, +0xc3,0x6a,0x02,0x47,0xcd,0xf0,0x6e,0x34,0x99,0xaf,0x2f,0x46, +0x93,0xf5,0xe5,0x70,0xf6,0x61,0x58,0x91,0x3f,0x2d,0xfa,0x33, +0x9d,0x54,0xab,0xf5,0x68,0x0c,0x74,0xcd,0xb4,0x5a,0x5f,0x4d, +0xae,0x2f,0x87,0xf4,0x06,0x83,0xc1,0xbb,0xe5,0x78,0x7d,0x35, +0x9f,0xa3,0xb3,0x1d,0x7e,0x25,0x58,0xdf,0x30,0x4b,0x09,0x1f, +0x52,0xd7,0xb7,0x63,0x4c,0x98,0x0d,0x3f,0xac,0xe7,0x77,0x2b, +0xb4,0xe4,0xb6,0x58,0xce,0xaf,0x51,0xb5,0x64,0x5d,0x8d,0xa9, +0xb7,0xeb,0xea,0xee,0x16,0x72,0x7e,0x5e,0xaf,0x26,0xb7,0xe3, +0xf5,0x07,0x68,0xc6,0x3c,0x41,0x0b,0x70,0xd0,0xc5,0x31,0x0f, +0x79,0xb9,0x2f,0xa7,0xd1,0xf9,0xa8,0x43,0xc6,0x3c,0xf1,0xe2, +0x5d,0xa3,0x49,0xbd,0x04,0x8d,0xb3,0xcd,0xe6,0xd5,0xcd,0xe4, +0x96,0x65,0x75,0xac,0x78,0xea,0xf7,0x64,0x48,0xd3,0x75,0x18, +0x2d,0x15,0x9e,0x9f,0x57,0xfb,0x8f,0x07,0x62,0x71,0x6f,0x29, +0x7e,0x2d,0x7e,0xf6,0x16,0x45,0xf7,0x7f,0x39,0xaf,0x3a,0xfb, +0xf9,0xf2,0xd3,0xcd,0xea,0x76,0x7a,0x36,0xbc,0x2e,0xf7,0x01, +0xc8,0x0e,0x1b,0x9c,0x5b,0xae,0x2f,0xe7,0xd3,0xf5,0xf8,0xf6, +0x02,0x96,0xc7,0xcd,0x72,0x3d,0xb9,0xbd,0x5e,0xb3,0x69,0x3a, +0x68,0x28,0x75,0x73,0xb8,0x86,0xed,0x39,0xbc,0xcd,0x52,0x34, +0x65,0x57,0x0c,0x3a,0x59,0xff,0x97,0xc7,0x83,0x6f,0xb2,0xf3, +0xfd,0xc7,0xfb,0xd7,0x93,0x7c,0x29,0x86,0x32,0x10,0xa4,0xa4, +0x43,0x45,0xab,0x8b,0xf9,0x08,0xfa,0xf7,0x3d,0xfd,0x92,0x2d, +0x3b,0xa8,0x18,0xbe,0xd7,0xed,0x7f,0xe9,0x9d,0x7f,0xec,0x1c, +0x21,0x4f,0x63,0xfe,0x1c,0xdf,0xb0,0x25,0x01,0xfa,0x54,0x5d, +0x2e,0x27,0x8b,0xd5,0xba,0x5a,0x7d,0x86,0x79,0xc3,0xca,0x33, +0x28,0x78,0x3b,0x9c,0x4d,0x16,0x77,0x53,0x7a,0xd5,0x7c,0xb7, +0x24,0x7e,0x27,0x3e,0xce,0x91,0xc4,0x21,0x71,0x54,0x94,0x05, +0xba,0x26,0xa6,0x28,0x1b,0xce,0x93,0x17,0xb5,0x72,0x5f,0x02, +0xe7,0x15,0xea,0xca,0xf7,0x7f,0x29,0x07,0xeb,0x12,0xc2,0xfa, +0xe2,0xd6,0xc5,0x0a,0x96,0x5c,0xad,0x00,0xdc,0x5d,0x9f,0xef, +0x43,0xd6,0xf7,0xc3,0x0f,0xc3,0xf5,0xf8,0xf2,0x76,0x98,0x71, +0x6a,0x98,0xef,0xe5,0xb0,0x22,0xe8,0xbf,0xac,0x96,0x77,0x63, +0xc8,0xdf,0xfd,0x06,0xbb,0x7c,0x89,0x96,0xb2,0x4e,0x29,0x13, +0x8d,0xf7,0x37,0xdf,0xef,0xa0,0x82,0x7e,0xff,0xe4,0xc9,0xf1, +0xd9,0xf1,0x79,0x7f,0xbd,0xb7,0x97,0xad,0x31,0x62,0x70,0x3e, +0xc0,0xf0,0x63,0xc8,0xb1,0x0b,0xd3,0xfc,0x71,0x39,0x5c,0xa0, +0x53,0xa9,0x7b,0x7e,0xfc,0x2b,0xfa,0x87,0xb9,0xda,0x59,0x70, +0x86,0xc7,0xcb,0x3f,0x68,0xe8,0x0f,0x8f,0x61,0x92,0xbd,0x85, +0x85,0x01,0x10,0xf8,0xd7,0x70,0xde,0x71,0xa9,0xab,0xc9,0x78, +0x3a,0x02,0x44,0x97,0xf3,0xf8,0xaf,0x41,0x8e,0x33,0xcd,0x79, +0xe0,0x02,0xe1,0x64,0x0a,0x0c,0x72,0x9a,0x5a,0x4e,0x62,0x5f, +0xcf,0x9c,0xaa,0x61,0xe4,0x9e,0xc0,0x6a,0xe2,0x0c,0x2b,0x1c, +0x79,0x4e,0x97,0x20,0x24,0x2f,0x8b,0xfe,0x23,0x97,0xc6,0xf3, +0x2d,0x59,0x28,0x68,0xb2,0xc2,0x32,0x6b,0xc8,0xeb,0x32,0x42, +0x32,0x6d,0x2f,0x2e,0xed,0xbe,0x6c,0x5d,0xd0,0x8e,0x6f,0x6b, +0xe5,0x57,0x4b,0xa9,0x6f,0xf9,0xb8,0xa1,0xd2,0x77,0x70,0xf2, +0x0d,0x61,0xf0,0xf4,0x08,0x56,0xbd,0x2c,0x5c,0x76,0xa7,0xe3, +0xe5,0x04,0x0e,0x80,0x7f,0x00,0x39,0x71,0x80,0x92,0x4b,0x49, +0x32,0xa0,0x8e,0xfe,0xfd,0x7b,0x36,0xa7,0x95,0x88,0x5d,0xad, +0x01,0x1c,0x65,0xc3,0xab,0xf1,0xb3,0xe5,0xf0,0x9a,0x6c,0xcb, +0x31,0xe7,0xfb,0xd4,0x44,0x39,0xbb,0x73,0x59,0x7e,0x25,0x51, +0x4f,0x26,0x1f,0x4a,0x5b,0x2a,0x36,0x0f,0xc7,0x0f,0xdf,0x0c, +0xea,0x29,0xf3,0x74,0x53,0x36,0x23,0x90,0x1d,0xc9,0x9a,0xe8, +0xc2,0x92,0x60,0xe6,0xb5,0x89,0x40,0xdd,0x1e,0xfd,0xe4,0x5d, +0xe6,0xbe,0xf0,0xb0,0x72,0x5f,0x3a,0x80,0x3e,0x82,0x4f,0x36, +0x9f,0x1d,0x27,0xd6,0x83,0xba,0xf1,0x09,0xa3,0xbc,0xce,0x08, +0x0c,0x9f,0x16,0xd9,0xac,0x46,0x84,0x52,0x0d,0x2f,0x51,0x1d, +0x41,0xd0,0x93,0xe6,0x9c,0x6a,0x7c,0xc7,0x1b,0x6d,0x31,0x7c, +0xd9,0xd4,0xd3,0xa7,0xf2,0x82,0x9b,0xc9,0xa0,0xa5,0x69,0x8d, +0x68,0x8d,0x14,0x28,0xdd,0xf8,0xcb,0x88,0x22,0x63,0x97,0x14, +0x12,0xb9,0x7e,0xc4,0xa6,0x51,0x20,0x88,0xbe,0xdc,0xc3,0x4d, +0xe5,0xac,0xb3,0xd3,0x30,0x04,0xd8,0x3a,0xae,0x0f,0xe2,0x2e, +0x38,0xee,0xf0,0xb3,0x20,0x29,0xb3,0x54,0x20,0x91,0x64,0xd6, +0x28,0xa4,0xb2,0xe2,0x88,0xa1,0x23,0xb0,0xa9,0x1c,0x8b,0x9c, +0xd1,0x10,0x4d,0x50,0xca,0xe9,0x48,0x6d,0x43,0x1f,0x0c,0x18, +0xd7,0xc3,0xcc,0xca,0xc9,0xc3,0x02,0x79,0x63,0x87,0xb3,0xee, +0xf8,0xb7,0xf4,0x00,0x3a,0x3b,0x9d,0xcf,0xc6,0xe9,0xce,0x41, +0x76,0x54,0x27,0xe6,0xdb,0x6d,0x84,0x15,0xda,0x82,0xd3,0xaa, +0xa8,0xbf,0x84,0x4c,0x46,0xac,0x3f,0x15,0xbc,0x21,0xfc,0xee, +0x28,0x42,0xae,0x8d,0xd4,0x8a,0xb1,0x6b,0xe6,0x99,0x31,0x8d, +0xe2,0x01,0x86,0xc7,0xb1,0x71,0xf3,0x49,0x03,0x63,0xf9,0x29, +0x3c,0x03,0x74,0x21,0xc4,0x73,0x10,0x2e,0xb2,0x78,0x1e,0x7a, +0x5f,0x39,0x01,0x04,0xba,0x71,0x0a,0x8a,0x06,0x08,0xca,0xc8, +0x70,0x74,0x38,0x43,0x72,0xb4,0x05,0x52,0xfb,0x57,0x4a,0x10, +0x54,0x29,0x3e,0x7b,0x71,0x50,0x31,0x45,0xf7,0x6d,0x27,0x3f, +0x2b,0xa8,0x98,0x8c,0x01,0xc5,0x6c,0x64,0xed,0xc5,0x9d,0x26, +0xce,0xae,0xeb,0x6a,0xb9,0xa5,0xf3,0x47,0xbf,0x7f,0x11,0x7a, +0x10,0xbd,0xda,0x60,0x14,0xae,0x49,0x77,0xb3,0xb0,0x51,0x11, +0xcf,0x95,0xd6,0x18,0xec,0xce,0x78,0xd0,0x62,0xda,0x8a,0xe0, +0x26,0x78,0x48,0xa1,0x2c,0x56,0xd0,0x1e,0x91,0x0d,0xfb,0x79, +0xb2,0xba,0x61,0x36,0x8d,0xa5,0x98,0x00,0x32,0x0c,0x10,0x72, +0x61,0x68,0xa8,0xb6,0xb5,0x63,0x34,0xbf,0x7d,0x89,0xe8,0x81, +0x13,0x50,0xad,0x72,0x7c,0x07,0x0d,0xc9,0x3e,0x12,0xac,0xa6, +0xec,0x91,0x86,0x69,0x14,0xf7,0x97,0x38,0x2a,0xe3,0xc3,0x26, +0x38,0xb7,0x99,0x44,0x65,0xea,0xfe,0xff,0xfd,0x96,0x05,0x5b, +0xdb,0x71,0xf8,0x02,0x6a,0x18,0x85,0x51,0x28,0xfd,0xf7,0xb4, +0xf4,0x30,0x6e,0xa9,0x99,0x75,0x3e,0x59,0xa2,0x88,0x2d,0x2d, +0xa1,0xea,0x87,0x57,0x01,0x1d,0xf6,0x7f,0xae,0xf6,0x80,0xa4, +0xc0,0x96,0xb0,0xf5,0xc8,0x06,0xee,0xe5,0xaf,0xe3,0xf1,0xe2, +0x09,0xd0,0x02,0x91,0xac,0x2b,0x3e,0x83,0x58,0x27,0x49,0xc2, +0xad,0x66,0x1f,0x49,0xe9,0x8e,0x77,0xe5,0xb9,0x85,0x58,0x8e, +0x0d,0x16,0x20,0x3d,0xab,0x55,0x31,0x21,0x36,0x0e,0xa5,0xa0, +0xf4,0x99,0x05,0x11,0x49,0xcc,0x94,0xa2,0x7c,0x8d,0x18,0x62, +0xc8,0x58,0x00,0x31,0x18,0x07,0x07,0xad,0xdd,0x8e,0x5f,0x68, +0x28,0x6f,0x70,0x65,0x30,0xe9,0x45,0x86,0x05,0xfe,0x3a,0x9d, +0x5f,0x0c,0xa7,0x4f,0xd1,0x16,0x80,0xa9,0x21,0x4f,0x18,0xc3, +0x4d,0xea,0x55,0x05,0x86,0x37,0xb9,0x35,0xc1,0xd9,0x53,0x13, +0x19,0xfb,0x3d,0x23,0x49,0x59,0x1b,0xec,0xcc,0x3d,0x34,0x1a, +0xf4,0x44,0x14,0x5f,0x4f,0x7c,0x01,0xd5,0xda,0x1a,0x18,0x3d, +0xe5,0x71,0x21,0x44,0xa8,0x72,0x55,0x44,0xbc,0x20,0x9e,0xc0, +0x24,0x73,0x46,0x95,0x39,0xbb,0x5a,0x3c,0x3d,0x88,0x2e,0x2e, +0xba,0x83,0x7d,0xd7,0x91,0xa8,0x3c,0x9e,0x8d,0x9e,0x7e,0xa0, +0xd5,0x3c,0x92,0x19,0x72,0x31,0xde,0x0e,0xa9,0x8d,0x15,0x7f, +0x60,0x41,0x5c,0x6f,0xe7,0xb0,0xf8,0x02,0x30,0x2d,0x16,0xc7, +0xf7,0x82,0x72,0x45,0x2d,0x9d,0xf7,0x47,0x74,0xfd,0x87,0xf7, +0x2c,0x63,0x16,0x74,0x80,0x7f,0xa9,0x47,0xb8,0xb7,0xf0,0xe6, +0xf8,0xdf,0x40,0x0f,0x03,0xb4,0x03,0x10,0x14,0xe4,0x0d,0xd6, +0x9f,0xab,0x10,0x59,0xaa,0x61,0x90,0x99,0x51,0x33,0x0b,0x56, +0x10,0xb3,0xf2,0x9c,0x61,0x36,0x2f,0x8d,0x6c,0x49,0x70,0xc0, +0xf9,0x33,0x6f,0xe2,0x92,0x44,0x48,0xea,0x2f,0xd7,0x54,0xe7, +0x7a,0x6d,0x29,0x56,0xe6,0x52,0x71,0xeb,0x1d,0x0b,0xa5,0x99, +0xb8,0x68,0xb7,0x2d,0x25,0xff,0x70,0xc1,0x1a,0xe5,0x0e,0x85, +0x6b,0x71,0x21,0x08,0xc1,0xda,0xfb,0xa9,0x12,0xe2,0x80,0xb2, +0x8f,0x2f,0x5d,0x72,0x9f,0xe9,0x9a,0xac,0xa6,0x74,0x3e,0xc8, +0x70,0xd8,0x71,0x30,0xd9,0x51,0x91,0x1b,0x1e,0xe5,0x11,0x00, +0xf9,0xb3,0x7b,0x08,0xa4,0xd4,0xee,0xa3,0xc7,0x49,0x46,0x76, +0xd4,0x49,0xbd,0x47,0xdf,0xf0,0xec,0x46,0xa6,0xe9,0x6a,0xd8, +0xc2,0xe9,0x17,0xf7,0x70,0x1e,0x4e,0x12,0xb7,0x85,0xb7,0x69, +0x79,0x60,0xac,0xc3,0x6c,0x98,0xff,0xd3,0x44,0x1b,0x70,0x91, +0x2f,0x21,0xf8,0x06,0xdf,0xa8,0xad,0x53,0x8f,0x6c,0x35,0x20, +0x5a,0xd2,0x22,0x55,0x39,0xa0,0x04,0xff,0x1c,0x5a,0x46,0x6b, +0x44,0xd6,0xa7,0xa0,0x3a,0x5c,0xb6,0x8b,0x7c,0x45,0xc6,0x7a, +0x90,0xd1,0x04,0x3d,0x52,0x2e,0xa6,0xbb,0xfe,0xfa,0x94,0x73, +0xd0,0x80,0x24,0x10,0x6f,0x0b,0xa5,0x3e,0xe2,0x2b,0x4d,0x95, +0x13,0xa3,0xcb,0xf0,0x48,0x1f,0x78,0xd2,0x08,0xdd,0xc2,0x43, +0x31,0xcd,0xa4,0x54,0xc3,0x95,0x39,0x23,0x17,0x4d,0xb8,0x97, +0xb9,0x91,0x5f,0x7a,0xf9,0x12,0x88,0xf6,0xa5,0x1d,0x8a,0x4a, +0x87,0x8c,0x55,0x2e,0x54,0x61,0x22,0x6a,0x3c,0x47,0x54,0xf3, +0x62,0x88,0x3a,0xc7,0xa4,0x39,0x74,0x09,0x55,0xbf,0x83,0x0b, +0x0b,0x26,0x59,0x5c,0x38,0xf5,0x07,0xac,0x34,0x43,0x0c,0x3d, +0x56,0x52,0x25,0x85,0xce,0x9b,0x61,0xc5,0x7c,0x95,0x2a,0xaf, +0xe4,0x17,0xcd,0xbb,0x2b,0xa5,0xdd,0xf0,0xae,0x5d,0x09,0x87, +0x3a,0x9f,0xbc,0x9a,0x9f,0xe0,0x49,0x56,0x4e,0xf7,0x0e,0x79, +0x71,0x94,0x58,0x05,0xf2,0x69,0x1f,0x44,0xaf,0x65,0xd6,0xe1, +0x3c,0xf0,0x91,0x68,0x84,0xee,0xf0,0x71,0x39,0x6d,0x78,0x0c, +0x97,0xd9,0x8f,0x76,0x33,0xb1,0x99,0xa8,0x7a,0xb8,0x4b,0x94, +0x29,0x65,0x77,0xef,0x43,0x44,0x24,0x89,0xb8,0x7b,0x32,0x04, +0x7a,0x84,0x34,0x1f,0x47,0x1f,0xf9,0x46,0xa1,0xe5,0x4e,0xee, +0x90,0xec,0x63,0x83,0xd0,0x93,0x80,0x37,0x8d,0x7d,0x8f,0x08, +0x0f,0xdc,0xd8,0xa9,0x39,0xee,0x32,0x7a,0x11,0xb8,0x0a,0xb0, +0xb0,0xda,0x64,0x31,0x8d,0x3a,0x85,0x7a,0x74,0xbc,0x75,0xb8, +0x2e,0xee,0xe0,0x4a,0x75,0x1c,0x10,0x2e,0xdb,0x44,0xac,0x22, +0x4a,0xc7,0xe4,0x13,0xcd,0x69,0xa9,0x70,0xec,0xdb,0x0f,0x29, +0x09,0x6b,0xbc,0x27,0x05,0x9c,0x3d,0x4c,0x5f,0x39,0x15,0x12, +0x50,0x99,0xb8,0xd4,0xc6,0x16,0x97,0xf4,0xb7,0x7e,0xa7,0xf3, +0x3a,0x4a,0x56,0xcb,0x24,0xd3,0xd5,0x63,0x1f,0x53,0xe4,0x40, +0x72,0x6b,0xc9,0x21,0x41,0xaa,0x2a,0xc0,0x2b,0x2f,0x33,0x8b, +0xb0,0x14,0x30,0xce,0xd1,0x93,0x9c,0x88,0xa4,0xf7,0xe8,0x17, +0x25,0xe0,0x77,0xba,0xfa,0x50,0x76,0x1a,0x13,0x83,0xbb,0x95, +0x96,0x36,0x6c,0x78,0xf4,0x5a,0xec,0x81,0xbb,0x0e,0xf0,0x7b, +0x92,0xae,0x77,0x69,0x26,0x95,0xf1,0x78,0x5a,0xe6,0xa6,0x49, +0xa6,0xbd,0x79,0x08,0x54,0x80,0x29,0xa1,0xf4,0x24,0xeb,0xe1, +0x3b,0xc4,0xeb,0xe5,0xb1,0xa7,0xc5,0x39,0x95,0x09,0xb5,0x42, +0x23,0xa8,0xb2,0x49,0x26,0xb6,0x8d,0xa4,0x7d,0x24,0xbb,0x81, +0x2e,0x1d,0xa4,0x65,0xfd,0x70,0x30,0xf6,0x0e,0xe3,0xc9,0x37, +0x63,0xad,0x9d,0x59,0xc2,0x0e,0x80,0xcd,0xaf,0x23,0x8b,0xd8, +0xa1,0xaf,0xc0,0x0e,0xa6,0xd6,0x81,0xc2,0x57,0x9e,0xfd,0x2a, +0x2f,0x44,0x88,0x97,0x8a,0x1c,0x3c,0xe2,0x6a,0x7a,0x95,0xbe, +0x43,0x74,0x45,0x06,0xea,0xda,0x21,0xba,0x98,0x23,0xc6,0x91, +0xf1,0x0c,0x99,0xb1,0x36,0x0e,0x43,0xab,0x96,0x97,0xca,0x8f, +0x1a,0xbe,0x1f,0x7e,0x4a,0xef,0xef,0x96,0xd3,0x42,0x53,0xd8, +0xd8,0x7a,0xf2,0xd7,0xa7,0x67,0x09,0x61,0x44,0x67,0xf4,0x29, +0x53,0x91,0x0f,0xab,0xcf,0xb3,0xcb,0x02,0x16,0x3a,0x57,0x89, +0xa1,0x84,0xcc,0x52,0x55,0x09,0x09,0xcf,0x2a,0xfb,0xd1,0xb7, +0x28,0x95,0x1e,0x90,0xa0,0x8c,0x0b,0x3a,0x25,0xaf,0x19,0x13, +0x36,0x72,0x2f,0x46,0xf6,0x6b,0x0c,0xab,0x19,0x11,0x18,0x94, +0x68,0x0b,0xb6,0x07,0xe1,0x85,0x01,0x96,0x6a,0x1e,0xc8,0x48, +0x7c,0x80,0xef,0xce,0xb3,0x79,0x91,0x70,0x28,0x51,0x8a,0x16, +0xa3,0x24,0x98,0xe4,0xf6,0x96,0x28,0x12,0x26,0x24,0x35,0xf6, +0x98,0xe8,0xba,0x84,0xc8,0xbb,0x44,0xaf,0x57,0x64,0xa1,0x25, +0xe6,0xaa,0x4d,0xe2,0xb7,0xcf,0xf9,0x72,0x72,0x3d,0x99,0x0d, +0xa7,0x0f,0xbc,0x80,0xfa,0xeb,0xc6,0xd2,0x10,0x95,0x98,0xb7, +0x66,0xa1,0x3d,0x6a,0x41,0x19,0x0b,0x0a,0x92,0x8e,0x40,0xc9, +0x89,0x6e,0x3d,0x1e,0x61,0xe4,0xe3,0xd2,0x23,0x2d,0x55,0x39, +0x51,0x33,0x04,0x38,0x34,0xcc,0xfe,0x71,0x8c,0x34,0x15,0xad, +0x61,0x28,0x48,0xa4,0xf4,0xb5,0xcd,0x03,0x7e,0xe3,0x43,0x2e, +0x10,0xdc,0x5c,0x81,0xeb,0x41,0x71,0x1e,0xcc,0xb2,0x26,0xdb, +0xe4,0x4c,0xb6,0xbc,0x54,0x46,0xf4,0x03,0x3f,0xfe,0x7d,0x89, +0x88,0x20,0xc7,0x2a,0xb0,0x0d,0x84,0x73,0xcc,0xca,0xdf,0x4c, +0x8a,0xe4,0x93,0x1c,0x96,0xab,0x4b,0x98,0xcc,0xde,0x0c,0xaf, +0xfd,0xd1,0xf3,0x25,0x02,0xf1,0xc8,0xb3,0x3c,0x2d,0x36,0xfb, +0x47,0x3a,0xc7,0xdc,0x0d,0xa7,0xb2,0xb2,0x8c,0xa4,0xc0,0xc5, +0x58,0x47,0x71,0x93,0xef,0x93,0xce,0xd8,0xaa,0x98,0x74,0x12, +0xc0,0x23,0x7b,0xd4,0x44,0xc6,0x13,0x29,0x48,0x4c,0xda,0x1d, +0x34,0x69,0x6a,0x18,0xe7,0x06,0x21,0xe4,0x76,0xa2,0xce,0x3b, +0x7e,0x5a,0xee,0x7a,0x40,0xe8,0x31,0x58,0x9b,0x6a,0xa8,0x3e, +0xb4,0xa6,0x18,0xf5,0x69,0xc6,0x07,0x33,0x0d,0xa7,0x3b,0x19, +0xa2,0xc4,0x13,0xd5,0xcb,0xaa,0x91,0xea,0xd6,0xbc,0x61,0x8d, +0x7c,0x0f,0xc7,0x26,0xe3,0x03,0xd3,0x4c,0x55,0x29,0x67,0x39, +0x35,0x39,0xb3,0x73,0x55,0x5a,0x7a,0xdf,0x12,0xcd,0x7c,0x18, +0xfa,0x8c,0x44,0x3b,0x77,0x3a,0x93,0xcc,0xc2,0x85,0xc8,0x76, +0xfb,0x6a,0xf2,0xe9,0x44,0x87,0xf5,0x79,0x55,0xdd,0x8d,0x2b, +0x3e,0x0a,0xa3,0x8c,0x34,0xcd,0xc1,0x22,0x43,0x31,0xcf,0xfa, +0x3a,0xc3,0xb6,0xdb,0x06,0x9a,0x30,0xe9,0x9d,0xfa,0xd6,0x06, +0x3d,0xb4,0x1f,0x2e,0x9f,0xf4,0xf7,0x4b,0xdd,0xc2,0x6d,0x4a, +0x59,0x4f,0xe6,0x8b,0xcf,0xd4,0x8c,0x2d,0x3d,0x20,0x8b,0x03, +0x51,0x4e,0xda,0x3c,0x5c,0x91,0x53,0xd1,0xdf,0x36,0xf4,0xf6, +0xaa,0x37,0x79,0x8c,0x1f,0xe3,0x90,0x55,0x12,0x6c,0xb7,0x1d, +0xde,0x56,0xed,0xf6,0x16,0x06,0x4a,0x50,0xab,0x1d,0x41,0x36, +0x2d,0x81,0xa4,0x0a,0x35,0x02,0x75,0xc5,0x0c,0x16,0x15,0x1e, +0x03,0x5e,0x11,0x5b,0x6f,0x4e,0x3e,0xea,0x20,0xdd,0x1f,0x8e, +0xef,0x73,0xee,0xb4,0x3a,0xf1,0x5c,0xdd,0x2e,0x72,0xb4,0xa6, +0x4f,0xb7,0x39,0xf1,0x96,0xf3,0x69,0x20,0xc5,0x40,0x6f,0x59, +0x4d,0x6f,0x58,0xce,0xd0,0x25,0xb6,0xb1,0x72,0xc2,0xd1,0x5e, +0x0c,0xd8,0xf1,0xf7,0xe9,0x46,0xc6,0x80,0x57,0x82,0xc5,0xf4, +0x84,0x5f,0x0d,0x51,0x6d,0x4c,0x9f,0x75,0x60,0x63,0xc8,0x3e, +0x08,0xd0,0x1a,0xaa,0x21,0x0f,0xc9,0x75,0x66,0x97,0x15,0x7c, +0x0a,0xa9,0xd1,0x0a,0x43,0x6c,0x33,0x18,0x46,0xf8,0x6e,0x17, +0x25,0xfc,0x5b,0xaf,0xb1,0x2b,0x01,0x83,0x56,0x8d,0x7c,0x36, +0xbe,0xab,0xe1,0xb8,0x94,0x11,0x9d,0x2c,0x07,0xd8,0x56,0x32, +0x99,0x46,0x50,0x9f,0xc6,0xfa,0x50,0x74,0xe0,0x48,0xee,0xae, +0xbe,0x2d,0xe2,0x98,0x9b,0x23,0x0b,0x93,0x01,0x4e,0x47,0xb8, +0x51,0x0f,0x91,0xd5,0x1d,0xca,0xfb,0x68,0x90,0xbf,0xe7,0x52, +0x07,0x03,0xb6,0xd7,0x28,0x1d,0xf4,0xf6,0x53,0x88,0x33,0xf1, +0xcf,0x33,0x0b,0x84,0xf7,0x47,0xc3,0x2e,0xd2,0x1b,0xc1,0x40, +0xb9,0xe7,0xb2,0x06,0x00,0x7e,0x98,0xf0,0xe5,0x08,0xce,0xd2, +0xa8,0x19,0xb4,0xd0,0x32,0xa7,0x1e,0x2a,0x48,0xe6,0x0e,0x39, +0x96,0x58,0xaf,0x59,0x12,0xc0,0xb4,0xa2,0xe7,0xde,0x91,0x21, +0x8b,0x0c,0x55,0x53,0xb6,0x83,0x02,0x06,0x00,0xff,0x59,0x32, +0xe1,0xbd,0xc8,0x81,0x8c,0x43,0x19,0x2b,0xc5,0xc4,0x69,0xec, +0x6a,0xd8,0x2f,0x3d,0x92,0x46,0x25,0xd0,0xee,0xa5,0x22,0xbc, +0x80,0x3b,0x72,0xed,0x0d,0x74,0x47,0x8d,0xa3,0xc8,0xbd,0x25, +0x0d,0x82,0x86,0x15,0x8d,0xcd,0x35,0x0f,0x19,0xf4,0x6d,0xd0, +0xba,0x32,0x49,0x8e,0xc2,0x1e,0xd1,0x54,0x87,0x15,0x04,0xe9, +0x19,0xe6,0xa7,0x77,0x64,0xbf,0x16,0xd8,0xa2,0xcb,0x97,0xe7, +0x92,0x37,0xcc,0x91,0x6c,0x18,0x38,0xce,0x10,0x4a,0x54,0x95, +0xc3,0x46,0x9c,0xfd,0x57,0xd9,0x4c,0x72,0xf5,0x29,0xda,0x0a, +0xcb,0x38,0x35,0x34,0x48,0x95,0x8b,0x1e,0x29,0x12,0x62,0x9f, +0x9e,0xf0,0x36,0x90,0x42,0x7c,0xc4,0xd3,0x5a,0xa0,0xac,0xe2, +0x49,0x04,0x2d,0x27,0xef,0xb8,0xf3,0x6b,0xbd,0xde,0x3b,0xf4, +0xc7,0x84,0x95,0x5c,0x36,0x67,0x1c,0x22,0xe6,0x4e,0xf1,0xe6, +0x6b,0x11,0x18,0xf2,0x86,0x22,0x4d,0xad,0x1d,0x0f,0x9c,0xc3, +0x1f,0xfc,0x0a,0x2d,0x3e,0xeb,0x69,0x64,0xe4,0xd0,0xe5,0x1b, +0xfc,0xbd,0xf6,0x09,0x92,0xc4,0xb5,0x7f,0x8d,0x24,0xa1,0x26, +0x19,0x92,0x44,0x49,0xa3,0x46,0x6f,0x29,0x7c,0x09,0x60,0x0b, +0x91,0x85,0x2c,0xfc,0xb1,0xf8,0xfc,0x47,0x9e,0xe9,0x62,0xd5, +0xf0,0x36,0x41,0x56,0xbd,0x26,0x23,0x42,0x14,0x09,0x33,0x9e, +0xcc,0xd0,0xa2,0xdc,0x70,0xfa,0x1f,0xe3,0xcf,0x46,0x41,0x08, +0xf5,0xd3,0xc4,0x62,0x92,0x8e,0x1f,0xd9,0xc9,0x1a,0x01,0x10, +0x38,0x1e,0xc5,0xcb,0x73,0xb4,0x04,0x82,0xc4,0x9c,0x2c,0xf2, +0x0e,0x9d,0x60,0x35,0xaa,0xb2,0xae,0xba,0x12,0x19,0x30,0xf5, +0xf5,0x62,0xc8,0xf4,0xba,0x48,0x7d,0xeb,0xdd,0x42,0xf2,0x51, +0xb2,0x3c,0x61,0x8a,0xd1,0x17,0x1a,0x8a,0xb2,0x99,0x2e,0x0c, +0xa8,0x63,0xe5,0x64,0xa4,0x96,0x87,0x26,0xa3,0x01,0xbb,0x0f, +0xc7,0x78,0x6e,0x04,0x63,0x27,0x38,0x10,0xad,0x09,0xb3,0xd3, +0x35,0x5e,0x5a,0xc7,0xa6,0x7c,0x7a,0x41,0xc3,0x85,0x6b,0xe5, +0xc6,0xd0,0x11,0x66,0x9c,0x60,0xb0,0x08,0x1a,0x61,0x82,0x7a, +0x43,0x16,0x94,0x45,0x61,0x0f,0x5b,0x02,0x6d,0xe6,0x31,0x6a, +0xb9,0xa8,0x70,0x44,0x7b,0x92,0x5c,0xeb,0x56,0x61,0x8d,0xfb, +0x71,0x95,0xd6,0xbe,0x1f,0x11,0x16,0x81,0x4f,0x9b,0xa6,0x8c, +0xa9,0x81,0xc8,0x3a,0x86,0x41,0x1d,0x82,0x5d,0x20,0x28,0x6e, +0xc6,0xe8,0xf9,0x48,0x75,0x4b,0x47,0x28,0x6d,0x2e,0x12,0x73, +0x2c,0xf2,0x8a,0x9c,0x85,0x53,0x94,0xb8,0xaa,0x50,0x96,0xef, +0xe4,0xf4,0x34,0x5f,0x0e,0xa7,0x8b,0x9b,0x61,0xb9,0x4f,0x3f, +0xe7,0x69,0xff,0x97,0x6c,0xf0,0xcd,0x39,0x49,0x4a,0xcd,0xe1, +0x3e,0x98,0xac,0x3e,0x97,0xfb,0x12,0x38,0xaf,0xbe,0x41,0x71, +0x2a,0xce,0x82,0x22,0x50,0x0b,0xd1,0x9f,0x45,0xc9,0xac,0xd5, +0x7c,0xb1,0x06,0xc2,0xe9,0x66,0xb5,0xbe,0x98,0xaf,0x56,0xf3, +0xdb,0xf5,0x74,0x7c,0xb5,0x22,0x27,0xa8,0xa3,0x49,0x05,0x77, +0xe2,0xe7,0xea,0x23,0x5c,0xac,0x90,0x71,0x86,0x84,0x05,0x5d, +0x09,0x69,0x6f,0x67,0xef,0xb2,0x3f,0x1e,0x0e,0xb2,0x2e,0x09, +0x91,0xdd,0x0e,0x97,0x40,0x78,0x41,0x1e,0x0e,0xa0,0xcc,0xd8, +0xdd,0x2d,0x9a,0x30,0x5e,0x95,0x35,0x5f,0xbc,0x44,0x92,0x41, +0x32,0xfa,0x77,0xee,0x7e,0x93,0xed,0xaa,0x2c,0x1c,0x44,0x41, +0x05,0x8b,0x4f,0x0f,0x97,0xe8,0xed,0x2c,0x3e,0x65,0xfd,0xe1, +0xde,0x3f,0xfe,0x75,0xd0,0x71,0x45,0x97,0xe3,0xe9,0xab,0xbb, +0xc8,0x81,0x33,0xf9,0xc1,0xcd,0xca,0xa8,0xbc,0x94,0xc0,0xa9, +0x90,0xce,0x95,0xf7,0x3f,0xbc,0x7e,0xf2,0x5f,0x40,0x33,0x4f, +0xe7,0x97,0xbf,0x02,0x15,0x7c,0x59,0x55,0xa7,0x37,0xf3,0x8f, +0xe5,0xbd,0x0e,0x11,0x50,0xce,0x17,0xd5,0x7c,0x0a,0xd3,0x99, +0xe4,0x1f,0x26,0xd5,0xe4,0x62,0x02,0xdd,0xfa,0x5c,0x24,0xec, +0x30,0x06,0x1d,0xc5,0x10,0x9c,0x00,0xc2,0xab,0x39,0x0c,0xc9, +0xf4,0x6c,0x39,0x9c,0x55,0xb0,0xf4,0x6f,0xcb,0x7b,0x98,0x5d, +0x98,0xf7,0x53,0x9c,0x8d,0xd9,0x75,0x71,0x90,0x5f,0xc1,0x71, +0xf6,0xf3,0x18,0x07,0xbd,0xf8,0xee,0xe0,0x80,0xca,0xf0,0x8a, +0x2c,0xfb,0xc9,0xd9,0x7c,0x01,0xcd,0x7c,0x8b,0x89,0xf0,0xfb, +0x03,0xcd,0x09,0x04,0x5e,0xc0,0xac,0xa0,0x6c,0x56,0x55,0xbd, +0x59,0xc2,0x9a,0xfb,0x84,0x88,0x5f,0xf2,0x33,0x39,0x57,0x81, +0xd4,0xd7,0xf0,0xef,0xe5,0xfc,0x1f,0xf0,0xf7,0xb6,0x4a,0x06, +0x0d,0xfa,0x81,0x50,0x2e,0xb2,0x2f,0xf9,0xf5,0x6f,0x3f,0xcc, +0x76,0x36,0x65,0x70,0x55,0xc2,0xc5,0x9b,0x57,0xbc,0x1e,0x6f, +0x51,0x2a,0x8e,0x9e,0x81,0x8e,0xac,0x1c,0x0e,0x5f,0x15,0x6c, +0x4e,0x9c,0x0e,0x45,0xce,0x5e,0xba,0x85,0x2c,0xa7,0x3c,0xaa, +0xbe,0x91,0x79,0xf4,0x9a,0x3f,0x76,0x00,0x4c,0x4c,0x07,0x38, +0xac,0x9c,0x32,0x29,0x74,0xc4,0x37,0x08,0xd1,0xdb,0x9d,0x43, +0x69,0x87,0xd1,0xc7,0x76,0x2e,0xae,0xa8,0xc1,0x3b,0x0d,0x72, +0x4c,0x54,0xa4,0xd6,0xb3,0xa2,0xa9,0x12,0x7c,0xb2,0x70,0x59, +0x6a,0x4f,0x16,0x8f,0x0f,0x51,0xc0,0x1e,0x96,0x4c,0xc3,0xd3, +0x39,0x46,0xff,0x38,0x19,0xc9,0xf3,0x1c,0x31,0xde,0x61,0xd9, +0x34,0x3d,0xf1,0x07,0x39,0xd1,0x61,0xe0,0xfc,0xfa,0x7a,0x6a, +0x9f,0xc0,0x57,0x64,0x6d,0x1c,0x07,0x1e,0x5d,0x7d,0x96,0xce, +0xe1,0xa7,0x97,0x0b,0xc7,0x1c,0x0f,0xc8,0x99,0x64,0xf7,0x29, +0x96,0xe9,0x51,0xbe,0x62,0x52,0xfd,0x48,0xeb,0x97,0xeb,0x73, +0x8a,0x71,0xfc,0x06,0x81,0x8d,0x49,0x75,0x2c,0x24,0x0e,0xdb, +0x9d,0xe2,0xab,0x43,0x03,0x5f,0xa4,0xaa,0x7e,0x9c,0xcf,0x7f, +0xad,0x8a,0x7b,0x39,0x6e,0x8a,0xfb,0x9a,0x49,0x9c,0x1c,0xb5, +0xa7,0xef,0xd0,0x62,0x2f,0xde,0x0d,0xfe,0x43,0xa5,0xe6,0xf9, +0x48,0x13,0xea,0x4c,0xc0,0x78,0xef,0xa6,0x09,0x59,0x0d,0x5d, +0xf5,0x92,0xc3,0x04,0xa5,0xd2,0x37,0x68,0x97,0x0d,0xb7,0xd9, +0xdd,0xed,0x05,0x5a,0xce,0xbd,0x84,0xed,0x79,0x3b,0x3b,0x99, +0xdf,0xcd,0x56,0xa2,0x5f,0x3f,0x7d,0x2d,0x2d,0xd9,0x09,0xf6, +0x1a,0x7c,0xe1,0x8b,0xe2,0x8f,0xee,0x6b,0xee,0xb3,0xcd,0x97, +0x70,0x8a,0xce,0x48,0xdd,0xf6,0xe3,0x64,0x34,0xff,0x48,0xa1, +0x7f,0x90,0x56,0x3f,0x85,0xe6,0xf3,0x5b,0x32,0x98,0x4d,0x1b, +0x70,0xbe,0x80,0xde,0x26,0x57,0xd3,0xf9,0x70,0x95,0xc4,0x02, +0x8b,0x90,0xe1,0x19,0x26,0xf4,0x12,0x0d,0x25,0x45,0x42,0xcb, +0x8d,0x3f,0x36,0xbc,0x5c,0x8b,0x6d,0xdb,0x2b,0x87,0x71,0x5d, +0x0e,0x69,0x9c,0x18,0x87,0xfe,0xb6,0xce,0xd5,0xf8,0x73,0x3d, +0x8a,0x3e,0x09,0xb2,0x1b,0x55,0xbe,0x1a,0x6f,0x70,0x6e,0x88, +0x9f,0x47,0xd2,0xbf,0x0e,0xbd,0xb8,0x1d,0x4f,0x89,0x72,0xa2, +0x25,0xce,0x6d,0x2a,0x3d,0x10,0x32,0x04,0x6f,0xf3,0x4b,0xb7, +0xfb,0x0a,0x08,0xad,0xf2,0x6d,0x4d,0x2b,0xe1,0x7a,0x1e,0xcd, +0x97,0x18,0x4b,0xa8,0x3d,0x81,0x74,0x6d,0x00,0xda,0x84,0x1a, +0x65,0x60,0xd3,0x02,0x62,0x06,0xa3,0x17,0xaf,0xd0,0x68,0x07, +0x36,0xdf,0xfa,0xa8,0x4c,0xf0,0xda,0xed,0x04,0x16,0x1e,0x20, +0xbc,0xfa,0x49,0x5a,0x09,0x14,0x26,0x86,0x9f,0x3c,0x66,0xca, +0x00,0x67,0xc1,0xb1,0x00,0x39,0x0b,0x6a,0x25,0x37,0xe2,0x48, +0xbc,0x42,0x06,0xef,0x46,0xb9,0x7f,0x4c,0x14,0x03,0x9c,0xac, +0xf6,0xc0,0xf7,0x8f,0x7d,0xce,0x45,0x6c,0x89,0xdb,0x8a,0x39, +0x90,0xc6,0x3c,0xcc,0xbe,0xc1,0xd0,0xa3,0x41,0x67,0x31,0x5c, +0x56,0xbc,0x14,0xd2,0xc6,0xd3,0x26,0xa3,0x69,0x2b,0x93,0x19, +0xad,0x6e,0xf2,0x30,0xc2,0xd2,0x03,0xf2,0x76,0xa5,0x09,0xae, +0x11,0xe8,0x26,0xea,0x95,0x7b,0x47,0x4e,0x35,0x5d,0xec,0xbf, +0xda,0xe1,0xe4,0x0d,0x13,0x4c,0x21,0x15,0xeb,0x94,0xc9,0x02, +0x1d,0x5f,0xc5,0x2b,0x19,0x4e,0x98,0x25,0x71,0xa9,0xe8,0xbc, +0x46,0x64,0x79,0xc7,0x35,0xe3,0x40,0x8c,0x3a,0x7a,0x7b,0xda, +0xf8,0xbc,0x81,0x92,0xac,0xb3,0x51,0x42,0x3a,0x2d,0x7e,0x34, +0x4b,0x98,0x93,0x9b,0xf1,0x72,0x82,0x98,0xbc,0xce,0x54,0x15, +0xce,0x14,0x8f,0x16,0xcf,0x55,0xa5,0x73,0x65,0xf7,0x43,0x16, +0x4c,0x3b,0x7b,0x43,0xb1,0x55,0x50,0x5e,0xfb,0x10,0x4d,0x5b, +0xb5,0x69,0x97,0x11,0x3c,0xbd,0x33,0xf8,0xdd,0xf6,0x8e,0xea, +0xfa,0xba,0xad,0xe2,0x0c,0x2a,0xff,0xb7,0xed,0x0e,0xbf,0xeb, +0xfe,0x1b,0xb6,0xc8,0x96,0x9d,0x80,0x9e,0x40,0xe2,0x9d,0x70, +0xa0,0x43,0x8b,0x9d,0xb7,0xc3,0x2b,0xf9,0xed,0xa1,0x4c,0x23, +0x27,0x63,0x86,0x76,0xf0,0x09,0xc3,0xc1,0x35,0x48,0x1e,0x41, +0x30,0x15,0x91,0xfb,0x3a,0xee,0xa3,0xa0,0x6a,0x09,0xe2,0xe8, +0x24,0xa7,0xc3,0x9d,0xda,0xb1,0x5e,0xd3,0x4f,0x0f,0x96,0xfb, +0x6d,0x69,0x76,0x09,0x94,0xcf,0xb8,0xa5,0x64,0xf6,0xca,0xf0, +0x6f,0x61,0x3d,0xc3,0xaa,0xba,0xc4,0x02,0x68,0x4b,0x1e,0xb9, +0x5c,0xe8,0xa1,0x24,0xc3,0x3f,0x70,0xcc,0x7e,0xb4,0x92,0x91, +0xd4,0x0b,0x91,0x10,0x72,0xaf,0x71,0xfc,0x8c,0xed,0xce,0x4b, +0x7e,0xf9,0x98,0x8e,0x00,0x9b,0x21,0x6a,0x5c,0xbb,0x25,0xc5, +0x32,0x48,0x92,0xd5,0xe6,0xa7,0x8c,0x23,0xf2,0x38,0xa2,0x94, +0x32,0x72,0x9c,0xd0,0x1d,0xa7,0x4f,0x80,0xfc,0x18,0x41,0x0d, +0xc2,0xfa,0xd1,0x24,0x6e,0xd6,0x58,0x5f,0x1d,0xa8,0x36,0xc0, +0x7a,0xa0,0x80,0x1b,0x59,0xac,0xca,0xc1,0xd4,0x9e,0xc8,0xd5, +0x4a,0x3b,0xb6,0x97,0x3a,0x5c,0x2b,0x76,0x09,0x22,0xe5,0xb7, +0x14,0x94,0x39,0x87,0x63,0x07,0xf5,0xf4,0x68,0x19,0x34,0x78, +0xcc,0xc9,0xdf,0x85,0x17,0x39,0xdc,0x9b,0xab,0x9b,0xfc,0x76, +0x32,0xfb,0x99,0x03,0xc3,0x4f,0x1c,0xd0,0x5c,0xa5,0xcb,0x4f, +0xbc,0xe6,0x00,0x09,0xa4,0x01,0x92,0xd4,0x9e,0x06,0xb0,0x5d, +0x6a,0x85,0x94,0xdc,0x74,0xa8,0xc2,0x9c,0x66,0xe0,0xc1,0xf0, +0x0f,0xe5,0xf5,0x6b,0x4c,0xed,0xac,0x48,0x01,0x34,0x1d,0x80, +0x87,0xd8,0x12,0x9d,0x24,0x7d,0x2d,0xa7,0x02,0xf6,0xb2,0x51, +0xb5,0x8b,0x30,0xc7,0xcc,0xd0,0x2e,0xcc,0x5d,0xc0,0xe7,0xa3, +0x76,0x5b,0x88,0xa2,0x50,0x4b,0x8e,0x46,0xa8,0x24,0x08,0xdd, +0x70,0xb4,0x24,0xb2,0x36,0x78,0x1a,0xaf,0x63,0x19,0x66,0x8b, +0x52,0x2d,0x64,0xec,0x62,0x30,0xaa,0x52,0xa1,0xcd,0xf2,0xb1, +0x09,0xa4,0x6b,0x42,0x04,0x5b,0x03,0xac,0x57,0xb8,0xf1,0x8e, +0x89,0x62,0x17,0x82,0x5d,0x58,0x30,0x28,0xbb,0x42,0xf3,0x0b, +0x9d,0xfe,0xd2,0x1a,0x64,0x8e,0x9f,0x29,0xf3,0x3b,0x96,0x1c, +0xd2,0xaa,0xf9,0xb2,0x82,0xff,0x91,0x3e,0xfa,0xe7,0x57,0xda, +0x57,0x2f,0x24,0x35,0xa3,0x8f,0x9e,0x9f,0xb8,0x7f,0x66,0x7f, +0x0a,0x42,0x60,0x62,0xb6,0x2c,0x8e,0x1d,0x47,0x81,0x87,0xeb, +0x03,0xbb,0x23,0x93,0x28,0x3d,0xe3,0x26,0x90,0xce,0xe8,0x2d, +0x5f,0xc3,0xd2,0xd7,0x72,0x89,0x2e,0xc4,0x2a,0xcd,0x87,0x51, +0x58,0x3d,0x47,0x94,0xb5,0x31,0xa5,0x68,0x41,0xf3,0x38,0x4b, +0x82,0x88,0xf1,0xe9,0xe4,0x1f,0x63,0x35,0x4b,0x07,0xe8,0xf5, +0xf8,0x96,0x10,0xec,0xdc,0xf5,0xa2,0xbb,0x00,0x4a,0x73,0x8a, +0xc0,0x3b,0x88,0x22,0xd8,0xf2,0xcd,0x15,0x73,0x44,0x26,0xe7, +0x3a,0x21,0xed,0xc3,0xbb,0xd5,0x9c,0xf1,0xf6,0xf0,0x6d,0xbb, +0x9f,0xdc,0x8c,0x85,0xca,0xa5,0x85,0x08,0xd4,0xad,0x97,0xaa, +0x11,0x97,0x5c,0x8d,0xd7,0x5f,0xf9,0x00,0x7d,0xa1,0xd8,0x73, +0xb4,0xef,0x7b,0xfa,0xfc,0xd1,0x9d,0x5f,0x5d,0x01,0x7e,0x41, +0x2b,0x19,0xc6,0xcf,0xb0,0x38,0x78,0x26,0x62,0x94,0x2c,0x91, +0x1c,0x89,0x23,0x92,0xba,0x98,0x59,0x2a,0x65,0x66,0x41,0x5e, +0xa7,0xe7,0xae,0xa5,0x8a,0xd7,0x4b,0x26,0x36,0x62,0xf4,0x03, +0xd5,0x19,0xbe,0x98,0xc7,0x39,0x42,0xae,0x6a,0xdd,0x0d,0x28, +0x05,0x92,0x3d,0xe2,0x1d,0x46,0x31,0xf4,0xd8,0x65,0xd7,0xbc, +0x73,0xe8,0x3f,0x16,0xa7,0x8c,0x1f,0xc6,0x8c,0x0a,0xd6,0x80, +0xf5,0x86,0x77,0xf4,0xc6,0xf4,0x60,0xc3,0x62,0x3c,0xf1,0x62, +0xfe,0x09,0xd6,0x11,0x20,0xc6,0x68,0xff,0x70,0x8e,0x5e,0xb2, +0xf6,0x20,0xca,0xbc,0x2e,0xf9,0xd1,0x74,0x59,0x13,0x43,0xb7, +0xeb,0x6f,0x71,0x40,0x4c,0xaf,0x18,0xbc,0x10,0x69,0x01,0x46, +0x45,0xab,0x41,0x53,0x1e,0x5a,0x0f,0xde,0x0e,0xa4,0x64,0xe6, +0x69,0x4e,0xfd,0x85,0x50,0xdb,0x2a,0xbd,0xfa,0xe6,0x11,0x25, +0x65,0x7f,0x1c,0x48,0x0c,0x3b,0xf0,0xe8,0x75,0x0f,0x0e,0xbf, +0x31,0x08,0x0b,0x33,0xa1,0xba,0xbb,0x87,0x59,0x27,0x49,0x0a, +0xb7,0x04,0x91,0x7a,0x4d,0x92,0xad,0xb3,0x69,0xe6,0xd1,0x9c, +0x3b,0xb9,0x6d,0x47,0x7d,0x5b,0x2b,0x05,0x5b,0xd6,0x70,0x22, +0x86,0x09,0xbb,0x0f,0xb9,0x83,0xa9,0x66,0x4b,0x3a,0x87,0x07, +0x07,0xdf,0x30,0xd2,0x9f,0x64,0xd8,0x1e,0x31,0xdf,0x53,0x86, +0x87,0x76,0x43,0xef,0xd7,0xeb,0x2a,0xf8,0x4a,0x92,0x23,0x8e, +0x40,0xd2,0xb8,0x3c,0xcc,0xb9,0xc6,0xc7,0xe5,0x21,0x26,0x95, +0x42,0x27,0xa0,0x85,0x4a,0xfb,0xcc,0xb8,0x9c,0xdc,0xa6,0xe2, +0x94,0xde,0xfb,0x56,0xc4,0x16,0x92,0x78,0x8c,0x1c,0xa6,0x31, +0x1f,0x15,0x2d,0x50,0x37,0xc5,0xa7,0x09,0x83,0x4a,0xe4,0xb4, +0x11,0xca,0x24,0xec,0xc9,0x4e,0x43,0x57,0x1c,0x79,0x22,0xdf, +0x25,0xb7,0x81,0x97,0x86,0x64,0xe9,0x35,0x37,0x53,0x06,0x32, +0x2b,0x38,0x19,0x8d,0x40,0x75,0x34,0xce,0xaf,0xdd,0x06,0x5d, +0x1b,0x5d,0xcd,0x40,0x2c,0x4e,0x90,0x87,0xfa,0x92,0xd0,0x03, +0x62,0xf3,0x35,0xac,0xec,0x5b,0x9f,0xfa,0x55,0xab,0xdb,0x2d, +0xb2,0xda,0x19,0x75,0xef,0x18,0x94,0x2c,0x4e,0xbd,0xe7,0xf8, +0x94,0xcc,0x45,0xee,0xf3,0xc6,0x34,0x15,0x26,0x03,0x77,0xf2, +0x6c,0xea,0xaf,0x88,0x74,0x21,0x78,0x0f,0xb5,0x9e,0xd3,0xb8, +0x88,0xe3,0xe4,0x90,0x5f,0x11,0x3f,0x73,0xca,0xec,0x4b,0x73, +0xc0,0x2f,0x60,0x47,0xd6,0x0f,0x78,0x8c,0x7d,0xf0,0x80,0xaf, +0x77,0xd9,0xed,0xe5,0x80,0x60,0x21,0xf0,0xf1,0xed,0xeb,0x80, +0x28,0xc3,0x8b,0x8e,0x47,0xd7,0xf4,0x34,0xe3,0xfa,0xe9,0xae, +0x73,0x1b,0xd7,0x8f,0xc6,0x26,0xb3,0xdc,0xaf,0xc5,0xd2,0x77, +0xd5,0x58,0xe7,0xf3,0xd2,0xd4,0x36,0xb6,0xcb,0xfc,0xe2,0x87, +0x50,0x20,0x73,0x33,0x7d,0x5f,0x1e,0xc8,0xc1,0xc4,0x91,0x7c, +0x0e,0x43,0x6c,0x5d,0x12,0x5d,0xd7,0x13,0xf3,0xf3,0x5e,0x53, +0x76,0xa4,0xff,0x90,0x5b,0x8f,0x9b,0xc2,0x50,0x97,0x96,0x49, +0xd4,0x95,0x65,0x61,0x49,0xc9,0xda,0x8d,0xb7,0xc9,0x9b,0x7a, +0x42,0x5c,0x70,0x6b,0x2a,0xd2,0x76,0x65,0x67,0x7b,0xdf,0x55, +0x29,0x2b,0x14,0x70,0xe3,0x85,0x87,0x07,0xd1,0x62,0x38,0xc2, +0x17,0x6d,0x0c,0xf2,0x2d,0x52,0x24,0x34,0x18,0x56,0x36,0x6d, +0x41,0xbc,0xef,0xbc,0xba,0xbb,0x82,0x9f,0xa6,0xd5,0x83,0xc9, +0x1d,0x4e,0x86,0x65,0xc4,0x2f,0x6b,0x35,0x39,0x72,0x63,0x06, +0x34,0xe7,0x2c,0x63,0xa4,0xf9,0x50,0xe6,0x7a,0x55,0xd5,0x4d, +0x22,0x51,0xa9,0x1e,0x4b,0xed,0xd2,0xab,0x06,0x3a,0x0c,0xca, +0x0a,0x91,0x05,0x3f,0xfa,0x4e,0x05,0xed,0x05,0x92,0xb6,0xc2, +0x31,0xf5,0xfb,0x93,0x81,0x6b,0x12,0x55,0x41,0x92,0xf8,0x12, +0xda,0x7b,0xe4,0xc2,0x07,0x8e,0xac,0x53,0x50,0x9b,0x4d,0x1e, +0xd0,0x11,0x0c,0x39,0xab,0x1f,0x18,0x51,0xcf,0x91,0x85,0x52, +0xd6,0x6e,0xfb,0x4c,0x1e,0x94,0x96,0x8f,0x0e,0xca,0xfd,0x7f, +0x7d,0x74,0x80,0xb6,0x0b,0x2e,0x96,0x40,0x91,0x42,0xe6,0xfd, +0xf3,0xfe,0xf9,0x00,0x1f,0x7d,0x4e,0xde,0xbe,0x78,0x06,0x5f, +0xcb,0xde,0xf9,0x0c,0xd3,0xd9,0xc0,0x28,0xcc,0x22,0x72,0x19, +0x2b,0xd6,0xe4,0xe7,0xb8,0x35,0x1b,0x35,0x5e,0x93,0xe9,0xd2, +0x35,0x99,0x1f,0x5d,0xa3,0x69,0x53,0xd6,0xec,0x97,0x72,0x24, +0xf0,0x4b,0x85,0xd8,0x54,0x01,0xbf,0x19,0xaf,0xf1,0x15,0x9c, +0x8c,0x1a,0xfc,0x3a,0xfe,0x7c,0x3d,0x9e,0x65,0xfb,0x93,0x86, +0x27,0x8b,0x4a,0x15,0x33,0x1a,0xf8,0xe5,0x92,0x9b,0x94,0xe2, +0xd5,0x48,0x9f,0xe4,0xe6,0x67,0x07,0x5c,0xbc,0x61,0xd4,0x36, +0xd5,0xb6,0x48,0xbb,0x46,0x5f,0x6e,0x49,0x08,0x48,0xab,0x81, +0x43,0x41,0x74,0x24,0x35,0xc9,0x31,0xa7,0x5d,0xe6,0x9e,0x13, +0xa0,0x55,0xbb,0x58,0x9a,0xc2,0x92,0xba,0x9b,0x26,0xfb,0x88, +0x58,0x1b,0xf3,0x19,0xb1,0x29,0x18,0x0a,0x58,0xf8,0x6c,0x54, +0x7a,0x27,0xe0,0xc4,0x4f,0xaa,0xd4,0xba,0xff,0x47,0x22,0xc0, +0x8f,0x35,0x2f,0x13,0xa7,0x84,0xf8,0x8a,0xe9,0x8a,0x9d,0x68, +0x1a,0x25,0x17,0x6b,0x28,0xaa,0x56,0xa7,0x08,0x12,0xec,0x3c, +0x60,0xc8,0xc1,0x94,0x43,0xdd,0xcf,0x60,0xe4,0x26,0x62,0x32, +0x44,0x9c,0xe8,0x06,0xea,0xb8,0xe8,0x1f,0x36,0x8d,0x3c,0x91, +0x41,0x14,0xbb,0xff,0x8b,0xde,0x8b,0x90,0x7f,0xd3,0x33,0xb2, +0xc8,0xc8,0x97,0xb3,0xfb,0x57,0x27,0xef,0x9e,0x3c,0xca,0x30, +0x13,0xdc,0xf1,0xce,0x91,0xf1,0xe3,0x6f,0x6c,0x5c,0xca,0x79, +0x72,0xbe,0x3c,0x9f,0x25,0x78,0x37,0x17,0xbf,0xbf,0x08,0xcb, +0x89,0x1a,0x9c,0x94,0x16,0x9c,0x3f,0xf9,0x86,0xf9,0x0a,0xad, +0x59,0x60,0x78,0x28,0xca,0xc0,0x7a,0x42,0xa1,0x58,0x16,0x1c, +0x68,0x3e,0x2f,0x2c,0x75,0x8f,0xe8,0x79,0xdd,0x93,0xba,0x32, +0x03,0x9f,0x34,0x69,0x56,0x18,0xb6,0x6f,0x0f,0x10,0x49,0x46, +0xd3,0xab,0xbe,0xbe,0x3c,0x0d,0xca,0xf1,0xec,0x12,0x26,0xf9, +0xa7,0xb7,0xcf,0x91,0x79,0x03,0xe7,0xfd,0x6c,0x85,0xb5,0x00, +0xd6,0x09,0x78,0x5e,0x43,0x9a,0x28,0xe0,0x10,0x6f,0xdb,0x37, +0x3b,0x62,0xfc,0xd9,0x14,0x23,0x63,0x7d,0x3a,0x5e,0xad,0xd0, +0x84,0x94,0xbb,0xf2,0x6c,0x64,0xd7,0x8e,0x42,0x1e,0x4d,0xe8, +0x10,0x0d,0x8d,0x8b,0x99,0x3b,0x2f,0xfd,0x3d,0xa9,0xde,0x4c, +0x87,0x93,0x19,0x9b,0x5a,0x83,0x3c,0x99,0xbd,0x14,0x86,0x96, +0xa2,0x42,0xcb,0x6d,0x6e,0x33,0xb0,0x3a,0x8e,0x74,0x24,0xf3, +0xce,0x8c,0x78,0xd4,0x91,0x77,0x36,0xcc,0x48,0x50,0xef,0x0d, +0x4e,0x54,0xa5,0xf7,0xc5,0x50,0xce,0xc7,0x81,0x9d,0x2f,0x9c, +0x1e,0x4f,0x1a,0x75,0xdf,0xcf,0x27,0xb3,0x34,0x69,0x5b,0xf1, +0xed,0x47,0x07,0xe8,0x44,0xd5,0x5c,0x83,0xd8,0xb8,0xe4,0x62, +0x7a,0xb7,0x6c,0x91,0x87,0x05,0xfe,0x0b,0x95,0xd2,0xef,0xfc, +0x6e,0xd5,0x02,0xfc,0x7f,0x84,0xfe,0xf8,0xe0,0xd0,0x69,0x55, +0x97,0xcb,0xf9,0x74,0xda,0xba,0x9b,0x51,0xe4,0xe5,0x74,0x72, +0xf9,0x6b,0x6b,0x74,0x31,0xe5,0xc0,0xed,0xfc,0xae,0x1a,0x8f, +0xe6,0x1f,0x67,0x1c,0xba,0x5b,0xf0,0x2f,0xe2,0xb7,0x1c,0x82, +0xc0,0x52,0x42,0x00,0x97,0x02,0x63,0x14,0x1e,0xe0,0xe0,0x74, +0x3c,0x84,0x8c,0x97,0x37,0xc3,0xd9,0xf5,0x58,0x4c,0x42,0xb5, +0x78,0x8f,0xb7,0x60,0x05,0x10,0x5c,0xf8,0x25,0x0f,0xca,0x18, +0x00,0xf0,0xe4,0x0e,0xa4,0x25,0x12,0x48,0x70,0x2c,0xdd,0x25, +0xe6,0x2e,0xdb,0x4a,0x78,0xd7,0xa5,0xbe,0x49,0x90,0x05,0x2d, +0x63,0xc9,0xc0,0xd5,0xde,0x43,0x0f,0x58,0xd3,0x5f,0x9f,0x99, +0x49,0xc8,0x41,0x0b,0xb1,0x0c,0x37,0x5c,0xaf,0xd7,0xd7,0x63, +0x66,0x77,0x46,0x46,0xf5,0x6e,0xb0,0xdb,0xbe,0xb2,0xab,0xd9, +0x6b,0xf8,0x86,0x92,0xaf,0xef,0x56,0xd1,0xa9,0xed,0x06,0x44, +0x32,0x65,0x5d,0x3f,0x30,0x29,0x15,0x58,0xaf,0x25,0x65,0x43, +0xb7,0x1e,0x2e,0xd8,0x17,0xf3,0xcb,0x37,0x78,0xd5,0xe6,0xf2, +0x41,0x87,0x1c,0x7d,0xbc,0x03,0x7c,0xe9,0xd2,0xed,0x49,0xb4, +0x4f,0x9a,0x71,0xfc,0x92,0x16,0x2e,0xdc,0x88,0x3d,0xb8,0x1c, +0xc9,0xa7,0xc7,0xfe,0xbf,0x74,0xc9,0xd6,0x2f,0xdc,0x0f,0xfb, +0x69,0xbf,0xd7,0x1e,0x64,0xef,0xca,0xfe,0x2f,0xed,0xc1,0x37, +0x98,0x81,0xec,0x0f,0xd1,0x3d,0xd9,0xfd,0xa6,0x07,0x38,0x42, +0xeb,0x7c,0x35,0x40,0xc1,0x0b,0x3c,0x52,0xd0,0x16,0xcf,0xb2, +0xb7,0xbb,0x7f,0x7d,0x9b,0x2f,0xd1,0x01,0xc8,0xf4,0x0d,0x1a, +0x6f,0xbe,0x9c,0x4f,0xf9,0x86,0x1c,0x5e,0xc0,0x4c,0xaf,0x87, +0x8b,0x05,0xfe,0xdb,0x43,0xcd,0x0b,0xbc,0x58,0xbb,0x9d,0x3d, +0xba,0x10,0x2b,0x36,0x76,0x34,0xa5,0xab,0x76,0xfd,0x71,0x32, +0x42,0xc3,0xb2,0x05,0xd9,0x1c,0x9e,0xab,0xd4,0x1a,0x81,0xf9, +0xeb,0xd3,0xb3,0xf5,0x8f,0x4f,0x8f,0x9f,0x90,0x08,0xc7,0xc2, +0xd7,0x70,0xbe,0x7f,0xbe,0x0f,0x31,0x77,0x4b,0xaa,0xae,0x7f, +0xfe,0x11,0x20,0x0f,0x3a,0x45,0x86,0xa6,0x6e,0x20,0x09,0xdb, +0xb8,0xdf,0xfb,0x97,0x02,0x1a,0x09,0x31,0x45,0x8a,0xd6,0x8d, +0xd6,0xf0,0xdf,0x7e,0xfe,0x0e,0xd7,0xaf,0x31,0x73,0x88,0x9f, +0x39,0xed,0x27,0xb6,0x2d,0x0d,0x88,0xd3,0x0a,0xb9,0xfa,0x88, +0x8e,0xd2,0xd7,0x70,0x3a,0x65,0x6c,0x21,0xf9,0x66,0x3f,0xe9, +0xb2,0xe6,0x58,0x9a,0x7c,0x23,0xca,0x89,0x76,0xe8,0x4b,0xef, +0x07,0x05,0xe0,0xf9,0xd7,0x9b,0x20,0xcf,0x36,0x1b,0x2c,0xc3, +0x24,0x0b,0xe6,0x91,0x60,0xa0,0xe3,0xca,0xa0,0x74,0x2d,0xc7, +0xc6,0x2e,0x84,0x12,0xc7,0x83,0x5f,0xee,0x82,0x8c,0xa1,0xaf, +0x2b,0x72,0x5a,0x17,0x0e,0x80,0x31,0x04,0xb7,0x9c,0xb2,0x71, +0x9e,0xca,0x28,0xce,0x35,0xd9,0x55,0x86,0x8c,0xed,0x36,0x0d, +0xa6,0xbe,0x5a,0xd2,0x87,0x3c,0x07,0x10,0x7a,0xe1,0x36,0x13, +0x23,0x6a,0xc6,0x9c,0x1b,0x8c,0xee,0xac,0x1a,0xf3,0x8b,0xae, +0x37,0xc8,0x0c,0xd4,0x41,0x89,0x1d,0x70,0x6f,0x70,0x2d,0x8f, +0x97,0x40,0xda,0x63,0x24,0x23,0x8c,0x71,0x39,0xc8,0xc9,0x8e, +0x65,0x20,0x2d,0xc7,0x2f,0xd1,0xba,0xc4,0xb0,0x49,0x3d,0x40, +0xb8,0x99,0x39,0xc8,0xdd,0x05,0xc5,0xfd,0xcc,0x80,0xe4,0x92, +0x9e,0x96,0xd2,0x73,0xfe,0x31,0x0f,0x74,0x05,0xc7,0x00,0x21, +0xe2,0x64,0x8d,0x65,0x18,0x34,0x81,0xdf,0x57,0x93,0x37,0xaf, +0x4f,0xcf,0x12,0xd1,0x6f,0xf3,0x12,0xdd,0x35,0xed,0x1e,0x1c, +0x65,0x52,0xec,0x71,0xe2,0x5e,0xac,0xd7,0x83,0xc2,0xba,0xac, +0xe7,0x23,0x35,0xc2,0xa5,0x3d,0x42,0x15,0x11,0x37,0x3f,0x3a, +0x76,0x67,0xe2,0xc2,0x89,0xbf,0x4a,0x6f,0xc8,0xc0,0xab,0xdc, +0xc5,0xb6,0x4f,0x13,0xb6,0x2f,0xe4,0x34,0x58,0x3d,0x12,0x50, +0x8d,0x51,0x82,0x38,0x84,0x9d,0x85,0xe6,0xa8,0xd1,0x80,0x9e, +0x49,0x85,0x76,0x21,0x41,0x89,0x72,0x5e,0x6e,0xf4,0xda,0x6d, +0xd7,0xca,0xf7,0xbf,0xfd,0xfd,0xc7,0xb7,0xe8,0x02,0x73,0x75, +0x57,0x65,0xf7,0xd4,0x24,0xba,0x6d,0xdc,0x4b,0x95,0x02,0x83, +0xb5,0x48,0x79,0xbb,0x16,0xba,0x14,0xcc,0x29,0x05,0x9d,0x9e, +0xe5,0xac,0xf2,0x1e,0x52,0xe2,0x74,0x65,0xa3,0xf3,0x3e,0xa0, +0xc7,0x39,0x4c,0xa4,0x39,0x06,0x4f,0xa4,0x69,0xf2,0x49,0xee, +0xf9,0x35,0xd7,0x1d,0xc9,0x04,0xe9,0x17,0xea,0x1b,0x05,0x74, +0x3c,0xa1,0x83,0xe6,0xbe,0x60,0xbf,0x7c,0xe6,0x08,0x0f,0x0f, +0xee,0xb9,0x78,0x3a,0x85,0xf8,0x58,0xd9,0xa9,0x4f,0x8f,0x96, +0x79,0x02,0xe4,0x78,0xc4,0x2b,0xb8,0x1d,0xaf,0x6e,0xe6,0x23, +0xad,0xa5,0xcf,0x9f,0x83,0x70,0x0f,0xd2,0x1d,0xe3,0x86,0x8b, +0x5b,0xb5,0xd5,0x24,0x0d,0x66,0xce,0x74,0x0d,0xf2,0x33,0xb9, +0x2b,0xea,0xd6,0x36,0x41,0x24,0xf9,0x46,0xbf,0xae,0xf3,0xed, +0x2b,0x93,0x5b,0xe5,0xd7,0xa6,0x5b,0xa9,0x64,0x1d,0x00,0x68, +0x56,0x1a,0xc7,0x42,0xc1,0x37,0xca,0xd2,0xb0,0x33,0xa6,0xe2, +0x80,0x34,0xa4,0x5e,0xce,0x47,0xe8,0xe1,0x8b,0xbc,0xc1,0x8f, +0x57,0xc3,0x6b,0xfc,0xb5,0x68,0x57,0x41,0xf5,0x07,0x57,0x99, +0xd1,0x7d,0x9b,0x54,0x18,0x3b,0x2d,0xc2,0xab,0x86,0x51,0x78, +0x7b,0x04,0xf6,0x0f,0x07,0x99,0xd3,0x86,0x43,0x1b,0xf3,0x73, +0x6c,0x26,0x09,0xb9,0xc2,0xa7,0xe8,0xcb,0x39,0x7b,0xf4,0xbc, +0xeb,0xf0,0xc8,0x9a,0x70,0x95,0xfb,0x9f,0xf6,0x3e,0x7e,0xfc, +0xb8,0x87,0x4f,0xba,0x7b,0xd0,0x1c,0x46,0x3b,0x47,0x47,0x88, +0x9b,0x2c,0x91,0x0a,0xfd,0xe9,0xec,0xd9,0xde,0x9f,0x13,0x11, +0x90,0x45,0xa9,0x99,0x6f,0x92,0x42,0x2f,0x08,0xf6,0x83,0x43, +0x2e,0x6c,0xf6,0x17,0x88,0x0e,0x26,0x6c,0x9f,0x80,0x63,0x78, +0x6b,0xa3,0x93,0xb0,0xb0,0xbe,0xdb,0x69,0xde,0xa2,0x0c,0x9f, +0x30,0xfd,0x7d,0x45,0x32,0x7a,0x26,0x03,0xc6,0x48,0x0e,0x34, +0x0f,0x27,0x32,0xc4,0xc6,0xde,0xe8,0x3d,0xc2,0xc4,0xd2,0xfb, +0x5c,0x1d,0xd5,0xb4,0xcf,0x90,0xa8,0xf4,0xfe,0xc6,0x6d,0xb7, +0x67,0x64,0x9c,0x8d,0x8b,0x24,0x1a,0xf9,0xf7,0x97,0x2f,0x12, +0x69,0xbb,0xdd,0x85,0x5c,0x07,0x60,0x1c,0x78,0x2d,0x62,0x4f, +0xa9,0x11,0x49,0x21,0x8f,0xac,0xa7,0x74,0x1f,0xe4,0xd4,0xb9, +0x16,0x75,0x0e,0xc7,0x95,0x3f,0xb1,0x56,0x27,0x49,0x44,0xe7, +0xcb,0xbf,0x9f,0xbe,0x7e,0x25,0x89,0xd8,0xcf,0x20,0x0d,0xea, +0xdf,0xe4,0x57,0x40,0xa2,0xbd,0xe6,0x87,0x62,0x5e,0x08,0x3a, +0x49,0xe4,0x2b,0x60,0xe3,0x96,0xca,0xdd,0x22,0xb6,0x47,0x8f, +0x1c,0x68,0x5a,0x40,0x5e,0xfc,0x4c,0x22,0x7a,0xb4,0xf9,0x79, +0x2d,0x9a,0xa0,0x14,0x6b,0xc0,0xfc,0x33,0x0f,0xab,0x30,0x05, +0x1a,0x72,0x8a,0x19,0xfa,0x8c,0xdb,0xf5,0x46,0xd1,0x87,0x02, +0x90,0xf0,0xb3,0xb9,0xfb,0xac,0x5e,0x2f,0xcf,0x1c,0x2e,0x91, +0x7a,0x24,0x83,0x6f,0x7a,0x97,0xf4,0x60,0x29,0x8f,0x8c,0x70, +0xa9,0x22,0x38,0x26,0xf4,0x6d,0xfd,0x5e,0x63,0x5b,0x74,0x5d, +0xc8,0x01,0x3a,0x1b,0xe2,0xfe,0x3b,0xa5,0x0f,0x3a,0x58,0x75, +0x7e,0x2b,0x71,0x4f,0x55,0xa9,0x15,0x02,0x39,0x16,0x75,0x5b, +0xe7,0x84,0x5f,0xfb,0x8b,0xfa,0x56,0xf6,0xae,0x9c,0xcc,0x08, +0xab,0x8c,0x81,0x1f,0x3d,0x22,0x57,0xe1,0x43,0xe1,0x70,0xc3, +0x6f,0xf9,0x28,0xc7,0x77,0x3e,0xc0,0x03,0xcf,0xe0,0x07,0x6d, +0xad,0xa3,0x08,0xce,0x19,0x47,0xa5,0x36,0x29,0xf3,0x38,0x97, +0x3f,0x9d,0x5c,0xf5,0x3f,0x72,0x5b,0x79,0xc5,0x95,0xd2,0x72, +0xe4,0xcb,0xe7,0x7a,0x77,0x0c,0x47,0x9f,0x4f,0xa9,0x46,0x6e, +0x1f,0x60,0xed,0xdf,0x15,0x07,0xbe,0xb7,0xf4,0x92,0xa9,0x37, +0x25,0x8c,0xe1,0x8f,0x24,0x5a,0xfd,0x56,0x93,0x53,0xb9,0x6a, +0x7c,0x01,0x54,0x51,0x67,0x48,0xe5,0xa3,0x03,0xb8,0xc0,0xbf, +0x3d,0x38,0x78,0xcc,0x11,0xeb,0xf5,0xb7,0x07,0xe8,0x36,0x8c, +0xbf,0x7a,0x69,0xd5,0x9d,0x5c,0xe9,0xc9,0x86,0x7e,0xf5,0x25, +0x58,0x72,0xcb,0x60,0x89,0xbc,0x0d,0x3a,0x91,0x26,0x2f,0xe0, +0x28,0xdc,0xd3,0x12,0x80,0x2a,0xdc,0xfa,0xc2,0xb2,0xce,0xec, +0x61,0xd9,0x27,0x51,0xef,0x9f,0xde,0xbe,0x18,0x94,0x9a,0xd1, +0x17,0xd9,0x5e,0x09,0x9e,0xaf,0x8d,0xb0,0x31,0xa1,0x09,0x66, +0x96,0x3f,0x0a,0xba,0xe5,0x56,0x44,0x09,0xfb,0xd0,0x4c,0x7a, +0x32,0x9b,0xcb,0xc1,0x93,0x64,0xc5,0xb7,0x5f,0x5b,0x64,0x75, +0xeb,0xfa,0x5b,0x98,0x7c,0x7c,0x55,0xd3,0x11,0x93,0x56,0x6e, +0xf0,0x33,0x5b,0xd8,0x65,0xee,0xd2,0x8a,0xd2,0xf5,0x69,0xe2, +0xe9,0x36,0xa2,0xe5,0x6a,0x22,0x79,0xf9,0x9a,0x16,0x51,0x44, +0x06,0xb5,0x73,0x4e,0x5f,0x43,0x9e,0xea,0xbc,0xee,0xf8,0xc8, +0x4c,0x56,0xb0,0xf6,0x60,0xcc,0x08,0x84,0xae,0x01,0x97,0x5a, +0x1e,0xa0,0x32,0x3c,0xcf,0x82,0xc4,0x58,0xd4,0xa5,0x6b,0x60, +0xa4,0xf1,0x76,0x21,0xcf,0xc3,0x5a,0x5f,0x27,0x49,0x7c,0x6b, +0x7b,0xb0,0xfe,0xa1,0xc6,0xf1,0x08,0x91,0xa2,0xf9,0xf4,0x03, +0x5b,0x2b,0xd3,0x8b,0x56,0xfd,0xad,0xf4,0x75,0xab,0x9a,0xbe, +0x08,0xc2,0x54,0x18,0x00,0x88,0xad,0x36,0x97,0xb7,0x38,0x1a, +0x95,0xa6,0x5e,0x0e,0xc2,0xfe,0x9c,0xa0,0x46,0x8f,0x0f,0xea, +0xdc,0x60,0xd8,0x6c,0xd4,0xab,0xc9,0x72,0xcc,0x6a,0x2c,0x30, +0x36,0xaa,0x6b,0x0e,0x6b,0xc4,0xb9,0x75,0x10,0x12,0xd9,0x77, +0x31,0xc0,0xc5,0x0a,0x8b,0xa5,0x69,0xbb,0xcc,0x78,0x28,0xae, +0xa1,0x0d,0x54,0xcc,0xf3,0x89,0x76,0x13,0xeb,0xff,0xba,0x4e, +0x0e,0xb2,0xb0,0xb1,0xe9,0x03,0xad,0x8d,0x50,0x49,0x85,0x05, +0x20,0xf6,0xf6,0x9c,0x80,0x39,0x4e,0xaa,0x7b,0x75,0x60,0x65, +0x8f,0xa0,0x3c,0x61,0xa5,0xb0,0x4a,0x36,0x35,0xca,0x81,0x08, +0xa8,0x54,0x0e,0x70,0xa4,0x57,0x98,0x6e,0xf1,0xb8,0x99,0x26, +0xc9,0x2f,0x5a,0xbb,0x39,0x12,0x4b,0xfb,0xa8,0x60,0x9d,0xeb, +0x36,0x6e,0x3e,0x27,0x83,0xa3,0xd7,0x76,0xda,0x1f,0xb5,0x71, +0xc1,0xbc,0x8a,0x78,0x64,0x77,0x8b,0x14,0x90,0x34,0xbd,0x63, +0xf2,0x68,0x74,0x4b,0xe7,0x64,0x03,0xd6,0x72,0x5e,0x1f,0x48, +0x9f,0x4e,0xbe,0xbf,0x83,0xb2,0x46,0x7f,0x39,0x4e,0x61,0xde, +0x9a,0x7b,0xd3,0x8a,0x92,0x9d,0x50,0x3b,0x8d,0x75,0xae,0x2b, +0x5d,0x1b,0xae,0x4b,0x22,0xad,0xaf,0x12,0xcd,0x72,0x22,0x00, +0xab,0x34,0x41,0x26,0x49,0xeb,0x76,0x7c,0x3b,0x5f,0x7e,0x4e, +0x82,0xa5,0x5d,0x99,0xe5,0x4f,0x56,0x86,0x96,0x63,0x68,0x55, +0xb5,0x92,0x81,0x2a,0x6b,0x31,0x6c,0x03,0xfa,0x7e,0x93,0xf3, +0xc5,0x87,0x07,0xe0,0xf2,0xf8,0x02,0xaf,0xb3,0xe4,0x72,0x08, +0xb5,0x20,0x73,0x9b,0x37,0x56,0x79,0xef,0xaf,0x2b,0xb8,0xa3, +0x6a,0x87,0x77,0x61,0x59,0xac,0x7c,0x3d,0x93,0x65,0x6b,0xe4, +0x73,0x3e,0x92,0xe3,0x76,0x4c,0x24,0xf9,0x4e,0x34,0x7d,0x19, +0xfb,0x17,0x0a,0xe2,0x50,0xf6,0x8f,0x8a,0x97,0xca,0xcc,0x61, +0xf6,0x40,0xe3,0x92,0xc9,0x8e,0xb2,0x28,0xbe,0x4f,0x65,0xeb, +0x46,0x9c,0xd8,0x4f,0x59,0xff,0xd1,0x40,0x81,0x47,0xc5,0xa0, +0xe9,0x51,0x89,0x4d,0xc0,0x2e,0xa7,0x52,0xcc,0x30,0xa7,0xe0, +0x46,0x2c,0x8d,0x84,0x43,0x51,0x35,0x3c,0x72,0xb8,0x21,0xe8, +0x35,0x76,0x41,0x9c,0x48,0x54,0x38,0xaa,0x66,0x7e,0x9a,0x95, +0x45,0x48,0xe4,0x8a,0x04,0x5f,0x67,0xdb,0x3c,0x62,0xb2,0x22, +0x02,0xba,0x6c,0xc7,0x6c,0x0d,0x73,0xde,0x9f,0x32,0x47,0x71, +0x7b,0xd2,0x7a,0x4d,0x75,0x86,0x19,0xac,0x98,0xaf,0x92,0xb7, +0xc8,0x2c,0x44,0x4f,0xc8,0x2f,0x61,0xc7,0x12,0x61,0xe2,0x11, +0x5d,0x4b,0xfe,0x69,0x8b,0xaa,0xee,0xad,0x64,0xa4,0x13,0x45, +0xa1,0xf8,0x85,0xeb,0xcb,0xdf,0x0e,0xc5,0xe5,0x1d,0x52,0x33, +0xb8,0x90,0x30,0x02,0xd7,0xd3,0x63,0x5e,0x4d,0xe4,0x18,0x07, +0x92,0x90,0xe3,0x8c,0x49,0x1e,0x46,0x1f,0xa3,0x07,0x65,0x3f, +0x8e,0x41,0x2d,0x19,0x0e,0x89,0x0f,0x7e,0xbe,0x34,0x86,0xd3, +0x8f,0xc3,0xcf,0x15,0x82,0xef,0xdb,0x5b,0x64,0x10,0x99,0xed, +0x1b,0xe2,0xce,0x08,0xd5,0x41,0xe4,0x22,0xbc,0x67,0x1b,0x4c, +0xb3,0xe1,0x94,0x6e,0xcd,0x2a,0xb8,0x2e,0x79,0x47,0x39,0x50, +0x7a,0x8c,0xb5,0xdb,0x2e,0xd8,0x25,0xc8,0xa9,0x03,0x90,0xe5, +0x84,0x0e,0xa3,0x16,0x85,0x8b,0x61,0xc3,0x24,0x64,0x89,0x40, +0xef,0x0f,0xa0,0x12,0x6f,0x27,0x30,0xeb,0xd4,0x66,0xcf,0xd9, +0x28,0x6b,0x17,0x0d,0x20,0xea,0x7a,0x3d,0x0a,0x66,0xc1,0x5f, +0x58,0x8b,0x24,0x30,0x82,0xc1,0xe1,0xab,0x21,0x3a,0x55,0xe8, +0xe2,0xb9,0x9e,0x22,0xba,0x0e,0x9d,0xe8,0xd2,0x8f,0xa5,0x72, +0xf1,0xf2,0x37,0x5c,0x7a,0x64,0xca,0xe6,0x41,0x8c,0x72,0x3d, +0xf3,0x88,0xce,0xed,0x24,0xfb,0xfb,0x78,0x6c,0xd1,0x43,0x9b, +0xde,0x12,0x5d,0xa6,0xdb,0xd7,0x6b,0xfd,0x66,0x76,0x80,0x8f, +0xe7,0x98,0x9c,0x11,0x27,0x66,0x6c,0x5a,0x71,0x16,0x1f,0x0f, +0x08,0xf6,0x37,0x49,0xe4,0x30,0x9b,0xbd,0x08,0xa4,0xa4,0x84, +0xb6,0x04,0xbc,0xee,0x23,0xea,0x6b,0xb3,0x1e,0xfb,0x20,0x17, +0x87,0x34,0xdd,0xcb,0xe5,0x1c,0x28,0xee,0xf9,0x2d,0x50,0xc0, +0x70,0xf0,0x2f,0x22,0xc6,0x24,0x8d,0x40,0xc4,0x91,0xcc,0x83, +0x42,0xe5,0x4e,0xba,0x43,0xa5,0xf4,0x25,0xf9,0x70,0x00,0x9b, +0x3e,0xea,0x7c,0xbb,0xcd,0x69,0x8f,0x6a,0x69,0x8f,0x06,0x5a, +0x6b,0xff,0x5b,0x14,0x5c,0x4f,0x6e,0x56,0xab,0x45,0x81,0xc2, +0x03,0x0a,0xad,0xf7,0xe7,0x83,0xe2,0xbb,0xef,0xbe,0x45,0x21, +0xfc,0x90,0x77,0x10,0xe5,0x8f,0xaa,0x74,0xc5,0xa8,0xc1,0x23, +0x32,0x80,0x49,0x2e,0x0b,0x95,0xc3,0xe0,0xfd,0x5b,0x78,0x77, +0x55,0x92,0x4f,0xc6,0xb5,0x0c,0x5e,0x7a,0x05,0x79,0x0d,0x1f, +0x9c,0x32,0x34,0x33,0xb3,0x00,0x4c,0xe1,0x8b,0x54,0x24,0xe0, +0x48,0x2a,0x01,0xce,0x0b,0x37,0xf7,0xf7,0x83,0xb2,0x8a,0x30, +0xfe,0xc8,0x5c,0xfe,0x30,0x3d,0x7c,0x51,0x87,0x68,0xd0,0x81, +0x17,0x6a,0x62,0x94,0xa6,0xd3,0xf1,0xd2,0x20,0x4d,0x48,0x0d, +0xb2,0xdd,0xdc,0xd2,0xe3,0x1f,0x98,0xd3,0x9f,0x16,0x0b,0x67, +0xc0,0xa0,0x42,0xef,0xda,0xca,0xca,0xdf,0xf1,0x6c,0x7d,0xf1, +0xf1,0xc5,0x3e,0x79,0x1d,0x0e,0x53,0xd2,0xb2,0x08,0x0a,0xd1, +0x11,0x27,0xa3,0x17,0x66,0xeb,0xf0,0xb4,0xc9,0x63,0x86,0xc8, +0xc2,0x48,0x8e,0xac,0x97,0xb4,0x01,0xa1,0xec,0xa1,0x27,0x3b, +0x1e,0x5e,0x51,0x8a,0xe5,0x2f,0x5a,0x6a,0xa4,0x98,0x8c,0x8e, +0xb9,0x69,0x5e,0x70,0x77,0x2e,0xd5,0xa5,0xb2,0x07,0xa3,0x21, +0xbf,0x0f,0x01,0xe9,0x4a,0x76,0x0f,0xdf,0x95,0x49,0xc7,0x3f, +0xb1,0x74,0x3a,0x59,0xa1,0x39,0x3b,0x5f,0xd5,0xaa,0xa4,0x06, +0x00,0xdb,0x14,0x10,0x91,0x0f,0xd2,0x81,0x30,0x2f,0x7c,0x00, +0x45,0x57,0x5c,0x9a,0x3c,0xbf,0x72,0x74,0xe5,0xde,0xe9,0x04, +0x60,0x27,0xf9,0x83,0x90,0x3c,0xbb,0x2e,0x20,0x0a,0x1f,0xac, +0xe0,0x15,0x1c,0x74,0x7b,0xe4,0xf8,0x33,0x69,0x2e,0x0d,0xbd, +0x49,0xfd,0xe6,0xf0,0xb3,0x89,0x5f,0x86,0xe9,0xb6,0x83,0xc3, +0xef,0x0f,0x29,0x93,0x92,0x6d,0xaf,0x5f,0x40,0xed,0x61,0xb6, +0x24,0x0f,0x4b,0xe5,0xdb,0x0a,0x1d,0x13,0xab,0x2e,0xb1,0x07, +0x1e,0x59,0x74,0xaf,0x44,0x29,0x1c,0x5f,0xa0,0x6d,0xca,0xa0, +0xb7,0x35,0xa5,0x83,0xcf,0x3e,0xc8,0xe4,0x08,0xa2,0x7b,0x49, +0xde,0x82,0x19,0x15,0x26,0x60,0x27,0x39,0x6a,0xfd,0x56,0x1e, +0x74,0x0f,0x48,0x3c,0x32,0x2b,0x3c,0x30,0x28,0x3b,0xf0,0xbe, +0x6e,0x61,0x3d,0x2a,0x0b,0x66,0x4b,0xcb,0x27,0xb9,0xcb,0xa2, +0xa6,0x73,0xaa,0x2e,0x1b,0xc3,0x42,0xb6,0x35,0xad,0x5d,0xff, +0xc9,0x96,0xd8,0x62,0xfa,0x47,0x48,0x96,0xac,0xe4,0xe1,0xf6, +0xe7,0x43,0x70,0x40,0xc8,0xad,0x99,0x1d,0x79,0xa4,0x95,0x62, +0x12,0xeb,0x99,0x97,0x49,0xb0,0x43,0xa6,0x12,0xe1,0x57,0xef, +0xc6,0xe2,0x70,0xc3,0x3d,0x80,0x46,0xa6,0xae,0xa5,0x9e,0x9d, +0xf3,0xf0,0x69,0xe6,0xb9,0x5b,0xb5,0xd3,0x2c,0xbb,0xaf,0x71, +0x77,0x0e,0xbf,0x9a,0xd2,0xf4,0xdc,0x7d,0x4b,0xb7,0xc1,0x64, +0x20,0x03,0x18,0x27,0x5f,0xa8,0x23,0x7c,0x9d,0x09,0x38,0x51, +0x28,0x67,0xa4,0x1c,0x2a,0x2b,0xf6,0x68,0xc6,0x29,0x91,0xfc, +0xf8,0x94,0xee,0x00,0x65,0xfc,0x1c,0x58,0x49,0x3b,0x3d,0x46, +0x52,0x21,0x07,0x31,0x44,0xfe,0x08,0x29,0xc9,0xfc,0xfb,0x20, +0x99,0x7f,0x55,0x5c,0x2c,0x23,0xfb,0x6d,0xad,0xf1,0x11,0x61, +0x2e,0x7b,0x30,0xe0,0xd9,0x86,0x4d,0x6f,0x68,0x44,0xf2,0x6a, +0xde,0x72,0x43,0xe8,0x5f,0xc8,0xa8,0x89,0x84,0x48,0xb3,0x89, +0xb6,0x90,0x51,0xe8,0x1f,0xf3,0xc2,0x57,0x04,0x14,0x09,0x21, +0xda,0xd3,0x11,0xf3,0xee,0xe5,0xc0,0xd9,0xab,0x20,0xa0,0xc8, +0xc1,0x2d,0xb6,0x3f,0x51,0x6c,0x83,0x1b,0xbe,0x63,0x24,0xc4, +0x19,0xb6,0x4f,0x05,0x86,0xd2,0x74,0x4c,0x75,0xae,0x57,0xf8, +0xe6,0x9e,0xe9,0x9d,0xb7,0x02,0xbe,0xf8,0x96,0x78,0x74,0x9f, +0xd2,0x14,0xff,0x69,0xcf,0xa7,0x04,0xec,0x73,0xa9,0xad,0xd1, +0xff,0x4a,0xc4,0x02,0x27,0xfe,0xb5,0x80,0x30,0xd8,0xb9,0x3c, +0xc5,0x05,0xdd,0x37,0x46,0x3d,0x18,0xeb,0x84,0xbf,0x9b,0xa8, +0xdf,0x6e,0x4f,0xa4,0xce,0xa0,0x9f,0xc7,0x89,0xb3,0x7b,0x7f, +0x53,0x59,0x89,0x16,0x8d,0xdd,0x39,0x8c,0xd0,0x26,0x4a,0xe3, +0x57,0x48,0x7a,0x31,0xd1,0x9b,0x1e,0x73,0x86,0xf5,0xba,0xc5, +0xd3,0x5c,0x2f,0x1d,0x34,0x06,0xb0,0x88,0x72,0xf3,0x98,0xe2, +0x89,0xe4,0xdf,0xb0,0xf1,0xcb,0x59,0xd8,0x4f,0xf0,0x2b,0xc9, +0xc8,0x8e,0xf2,0x36,0x6d,0x1a,0x59,0xac,0xf7,0x55,0x60,0xc5, +0xfe,0x9d,0x59,0x46,0x5c,0xcf,0xd6,0x67,0x72,0x67,0x43,0x85, +0x03,0xbc,0xa1,0x89,0xe3,0xd8,0xe5,0x98,0x13,0x7e,0xa9,0xc1, +0xd1,0xe0,0x1c,0xfa,0x74,0x13,0x65,0x70,0x10,0xaa,0xe5,0xa5, +0x62,0x1f,0x1c,0x31,0x27,0x11,0x97,0xd2,0x7d,0xd1,0x01,0x44, +0x9b,0x93,0x25,0x54,0x4a,0xd3,0xee,0x49,0x45,0x07,0x66,0x76, +0x9f,0x4a,0x08,0xf9,0x88,0x5c,0xd0,0x9f,0x5b,0xeb,0xf5,0x3e, +0x42,0x1c,0x8f,0xd6,0x7a,0x68,0xee,0x0b,0x16,0x14,0xe7,0x24, +0x55,0xdb,0xaf,0x6d,0x06,0x5b,0x72,0xe1,0x64,0x6b,0x9f,0xbd, +0x16,0x15,0x58,0xdf,0xa9,0xc4,0xfc,0xa5,0x8c,0x33,0x01,0x71, +0x6d,0xd7,0x69,0x48,0x1f,0x1d,0x1c,0xc0,0xfe,0x17,0xfe,0x1c, +0x4a,0x1f,0xe2,0xdc,0x86,0xe6,0x7b,0xcd,0x82,0x08,0x3d,0xe1, +0x45,0xd4,0x9d,0x4e,0xa9,0x6b,0x18,0x77,0x2c,0xf5,0x27,0xce, +0x0e,0x69,0x4a,0x88,0x38,0xe7,0x7c,0x3a,0x72,0x1c,0x1a,0x94, +0x3a,0x5b,0xe2,0x91,0xb1,0x28,0xf7,0xd3,0x32,0x3b,0xef,0xa5, +0xbd,0xb2,0xbd,0xde,0xcd,0xd6,0xe7,0xbd,0xf3,0xde,0xfe,0x51, +0xfd,0x08,0xa1,0xbc,0x45,0xa2,0xdd,0xe0,0xf7,0xb4,0x85,0xc2, +0x2b,0x22,0x51,0x45,0xf7,0x34,0x6a,0xeb,0xec,0x2e,0xe6,0x8b, +0x34,0xf3,0x2c,0x3d,0x36,0xad,0x02,0x68,0x5b,0x88,0xb5,0x59, +0x82,0xb6,0xaf,0x80,0x06,0xb8,0x10,0xdd,0x8b,0xe8,0xd6,0xcd, +0x8e,0x8d,0xa2,0x37,0xb2,0x85,0xdd,0x78,0xce,0x48,0xa5,0x7b, +0x63,0xe2,0x7b,0x30,0x68,0xea,0x2b,0xd2,0xe9,0x84,0x13,0xe9, +0xe3,0x12,0x05,0x20,0x67,0x8e,0x85,0x77,0xc2,0x6a,0x7f,0xe3, +0x25,0x75,0x19,0xd5,0x0c,0x61,0x51,0x53,0x1d,0x3b,0x82,0xe9, +0xf2,0x40,0x2a,0xfa,0x0d,0xeb,0x1d,0x90,0x52,0xf8,0x4b,0xea, +0xfc,0x91,0x53,0x3e,0xc1,0xdf,0x76,0xd2,0x00,0xc7,0x62,0x6b, +0xa0,0x4e,0x38,0xe3,0x8b,0xaf,0xa6,0x24,0xcd,0x19,0x54,0x2a, +0xef,0xd4,0x09,0xfe,0x9a,0x8b,0x4b,0x1a,0x0c,0xf0,0x79,0x50, +0xca,0x1a,0x8a,0x95,0xda,0xfe,0x6b,0xc7,0x74,0xca,0x1a,0x44, +0x0f,0xa3,0x1c,0x59,0x2f,0x8a,0x48,0x11,0x31,0x0b,0x62,0xdc, +0xb0,0xf5,0xaa,0xbe,0x06,0x07,0xa5,0x09,0x7b,0x3a,0x80,0xca, +0x21,0x29,0x90,0x74,0x6c,0xb3,0x1c,0xc8,0x1d,0x4b,0x5a,0x34, +0xd1,0x2a,0x3a,0xfa,0x9e,0x50,0xa1,0x72,0x24,0xe8,0x18,0x40, +0x64,0x1c,0x57,0xee,0x9f,0xbe,0x9c,0x7d,0xfc,0xba,0x6a,0xc4, +0x05,0xbc,0x52,0x50,0xbc,0x16,0xfc,0x2a,0x46,0xa4,0x2d,0x18, +0xc4,0x4e,0xd2,0xfa,0x38,0xac,0xd0,0x11,0x21,0xad,0x2d,0x7a, +0x8a,0xaa,0x95,0x87,0xa1,0xdf,0x44,0x88,0x73,0xc9,0x57,0xb8, +0x5d,0x83,0x25,0xbf,0x08,0xf7,0x2d,0xf8,0x41,0xde,0x14,0x19, +0xb6,0x39,0xaa,0xcc,0xcb,0xb0,0x6c,0xf2,0x80,0x9b,0x64,0x0a, +0x35,0x02,0xb5,0xdb,0xa1,0x0a,0xd3,0x68,0x16,0xc2,0xc5,0x12, +0xef,0xb2,0x68,0x19,0x84,0x47,0x01,0x99,0x39,0xb3,0x33,0xd2, +0x30,0x48,0x8e,0x50,0x36,0xeb,0xcf,0x34,0x09,0xd6,0xbb,0xf9, +0x4a,0x9b,0xc6,0xb8,0x01,0xa8,0xed,0x94,0xbf,0xfe,0x37,0xc6, +0x88,0x98,0xaa,0x71,0xf0,0xa9,0xf9,0xe9,0x66,0xe9,0x9a,0x9d, +0xc3,0xc7,0x29,0xeb,0x53,0xc0,0xe1,0x0a,0x1f,0xcf,0x47,0xe5, +0x01,0xfe,0xbe,0x9e,0xfd,0x44,0x47,0x2f,0xe3,0xf8,0xf2,0x8e, +0x7f,0x4c,0xf4,0xfe,0xdf,0x59,0x74,0xd5,0x88,0xf0,0xf0,0xb9, +0xf3,0xeb,0xf8,0x33,0x11,0x01,0xf0,0x8b,0x44,0x8b,0xad,0x26, +0xb3,0x1f,0xc8,0xf1,0x1d,0x44,0x47,0xfa,0x51,0x93,0x6c,0x2d, +0x14,0x6a,0xac,0xb9,0x57,0x5b,0xce,0x3b,0xec,0x6c,0x86,0xe5, +0x3e,0xda,0x6d,0xb1,0x1c,0xb9,0x82,0xd3,0x78,0xb8,0x1c,0xc1, +0xf2,0xc0,0x43,0x9a,0x23,0x05,0x10,0x46,0x6d,0x8a,0x5a,0xbe, +0x60,0x38,0x9a,0x84,0x80,0xb1,0x4d,0x69,0xdd,0x5c,0xc4,0x7c, +0x59,0x95,0x3b,0x3b,0xb6,0x30,0x9c,0x5c,0x1f,0x27,0xab,0x9b, +0x93,0xe5,0x78,0x04,0xcb,0x74,0x02,0x34,0x48,0xc2,0x63,0x12, +0x8c,0x76,0xad,0x2e,0x67,0xe3,0x0e,0xea,0x8c,0x20,0xe6,0x21, +0xf8,0x46,0x1c,0x2d,0x42,0xcd,0x76,0x02,0xdc,0xac,0x6e,0xfc, +0x7d,0xae,0x8f,0xfa,0xba,0x6e,0x9b,0x51,0x2e,0xa1,0x2c,0x1d, +0x19,0xc7,0x65,0xd8,0xe6,0x58,0x3e,0xc1,0x76,0x95,0x32,0x30, +0x4c,0x78,0xde,0x55,0x68,0xe9,0xeb,0x76,0xdc,0x83,0xb8,0xee, +0x7c,0x31,0x9e,0xa5,0x8e,0x91,0xc8,0x8c,0x1b,0x42,0xc4,0x72, +0x9f,0x31,0x47,0xf7,0x00,0x55,0xf5,0x71,0xbe,0x1c,0x65,0xc5, +0x83,0x85,0xf0,0x98,0x83,0x0c,0x2c,0x9b,0x92,0x19,0x02,0xd9, +0x47,0x42,0x08,0x28,0xcb,0xd2,0x44,0xa1,0xff,0x76,0xcf,0x01, +0x6f,0xb7,0xa9,0x86,0x88,0x81,0xde,0x1c,0x6b,0x18,0xe7,0x11, +0x06,0xbd,0x5e,0x2b,0xbd,0x9d,0xfc,0x7d,0x4f,0x88,0xf1,0xf1, +0x68,0x8f,0x2c,0x4a,0x23,0x67,0xf0,0x81,0xd4,0x32,0xf9,0xfb, +0xcb,0x17,0x3f,0xae,0x56,0x0b,0x49,0x30,0xae,0x2b,0xa8,0x33, +0x4a,0xeb,0x63,0x83,0x1a,0x28,0x7d,0x43,0xe7,0x2b,0x59,0xb8, +0x5c,0x66,0xf7,0x1b,0xce,0x8e,0x62,0x78,0x11,0x2b,0x85,0x1c, +0x89,0xae,0xc9,0xa4,0x81,0x97,0xf0,0x6a,0xc2,0x4b,0x59,0x0b, +0x93,0xde,0xa4,0x6b,0xaf,0x7d,0x0d,0x72,0x21,0xd4,0x6a,0xb4, +0x69,0xe4,0x64,0xfa,0x3c,0x62,0x8b,0x8f,0xfd,0xd8,0x20,0x8b, +0xae,0x9a,0xac,0xe6,0x3d,0x98,0x97,0x11,0x94,0xa5,0xf1,0xaf, +0x63,0xae,0x4e,0x2e,0x79,0xbe,0xa8,0x9d,0x4b,0xed,0xb6,0x30, +0xec,0x82,0xe3,0x85,0x21,0xc2,0x49,0xa9,0x1d,0xfb,0x6e,0x27, +0x6e,0x0c,0xcf,0xb6,0xb2,0x32,0x90,0x62,0x76,0x77,0x8c,0x7b, +0x99,0xbb,0xab,0xa8,0x50,0xf3,0x80,0x50,0x52,0xe3,0x3b,0x54, +0x9a,0xe5,0x35,0x24,0x89,0x2b,0xf7,0xe2,0x52,0x46,0xc6,0xa4, +0x22,0xe3,0x9b,0x65,0x9c,0xc3,0x73,0x08,0x44,0x26,0xc0,0x37, +0x05,0x3f,0x3d,0x3f,0xc0,0x8a,0x1e,0x24,0x1b,0x2f,0x9f,0xa0, +0x87,0x21,0x32,0xf1,0xcd,0xb2,0xed,0x1d,0x3e,0x7a,0xf4,0xad, +0x13,0xc4,0xf0,0xc2,0x09,0x8f,0x0e,0xbe,0x03,0x9c,0x84,0xc3, +0x61,0xd3,0x7a,0x80,0xde,0x17,0xdf,0x1d,0x7c,0xb7,0x91,0x3a, +0x91,0xab,0x72,0x35,0xff,0x74,0x4c,0xe8,0xfe,0xd3,0x4f,0x48, +0x86,0xe3,0xc3,0xc4,0xbd,0xa7,0x0b,0x54,0xd6,0x73,0x8f,0x59, +0x30,0x0d,0x99,0x37,0x46,0xda,0xc6,0x65,0x97,0x81,0x6e,0x5a, +0x69,0xf1,0xe8,0x13,0x33,0x85,0xce,0x83,0x5e,0x7d,0xa1,0xf5, +0x0c,0x57,0xc6,0x91,0x8a,0x45,0xca,0xab,0xa2,0xec,0x74,0xe8, +0x92,0x6b,0x58,0x4a,0xa9,0x5d,0x44,0xb0,0x83,0xed,0x27,0xae, +0x09,0xa1,0x5c,0xf9,0x42,0xca,0xba,0xac,0xdc,0x90,0xc6,0x70, +0xe0,0xd6,0x6f,0x5a,0x8c,0xce,0x04,0x0a,0x25,0x37,0xad,0x73, +0xdf,0x54,0x47,0x58,0xa1,0x4b,0xf4,0x1a,0x81,0xe4,0x77,0x9b, +0xcb,0xb7,0x85,0x3a,0xba,0xfa,0xf4,0x6a,0xfe,0x91,0x9e,0xf6, +0xb1,0xc3,0xcb,0xab,0x4f,0x2b,0xaf,0xc8,0xc6,0x56,0xd6,0xd6, +0x68,0xef,0x6c,0x8d,0x06,0xce,0x48,0x64,0xfe,0xea,0xd3,0xac, +0x66,0x4c,0xb0,0x57,0x88,0x3d,0xc1,0x75,0x16,0x5b,0x24,0x64, +0x63,0x84,0xc6,0x90,0xe1,0xf2,0x6e,0x56,0xee,0xc3,0x61,0x75, +0x37,0x26,0xcd,0x3c,0x00,0x39,0x9c,0x4d,0x6e,0x09,0xeb,0xf7, +0x2c,0xbd,0xb2,0x2f,0x26,0x89,0x5d,0xd4,0x20,0x5f,0x7d,0x1c, +0x8f,0x67,0xf4,0x02,0x8d,0x32,0x92,0x7d,0xa3,0xf3,0x08,0x9b, +0xde,0xfa,0x5f,0x9a,0x8d,0xf2,0xbb,0xd9,0x64,0xc5,0x05,0x58, +0x73,0x4c,0x2c,0xca,0x62,0x84,0xcd,0x2f,0x1a,0x8d,0xdc,0x27, +0x6b,0xa0,0x4a,0xa4,0xf1,0x4a,0x02,0x81,0x1a,0xe5,0x29,0xbd, +0xe6,0x03,0x92,0xd3,0xe1,0x94,0xf5,0xfa,0x00,0x08,0xde,0xe1, +0x14,0x59,0x73,0xb7,0xc3,0x4f,0xcf,0xa1,0x85,0x43,0x16,0xae, +0x78,0x44,0x56,0xfe,0x08,0x30,0x1b,0x29,0x9b,0x8d,0xca,0x8e, +0x3e,0x2d,0x51,0xc3,0x4a,0xf3,0x96,0x54,0x33,0x3e,0x45,0x0a, +0xb6,0xa8,0xce,0xc4,0x26,0xa7,0xf0,0x2f,0x19,0xe2,0x9a,0x90, +0x45,0x8d,0x21,0x9e,0xc0,0xdc,0x10,0xa3,0x97,0xca,0xad,0x74, +0x6a,0xbd,0x38,0xc1,0xeb,0x35,0xd4,0xbb,0x5e,0x1f,0x1e,0x8d, +0xe6,0x2d,0x6e,0x27,0xfd,0x05,0x8a,0xa8,0xfb,0xc7,0x84,0xbb, +0xb2,0xcf,0x51,0x79,0x60,0xb7,0x25,0x06,0x45,0x39,0x3b,0x58, +0x7d,0x76,0xf4,0x11,0xc8,0x71,0xa4,0xd2,0xa1,0x10,0x34,0x89, +0x03,0x76,0x84,0xf6,0x45,0x84,0xb1,0xdd,0x46,0xa3,0xf2,0x94, +0xdc,0x6e,0xef,0xed,0x05,0xc3,0x93,0x6d,0xb8,0x00,0x8d,0x83, +0x9f,0xa5,0x2e,0xf7,0x89,0xfe,0x4a,0x0c,0x8e,0x9b,0x7b,0x5f, +0xe3,0x66,0xa4,0xfa,0x8d,0xe6,0xc2,0x10,0xf9,0x80,0x7f,0xce, +0x3b,0x04,0x16,0xda,0x0c,0x1c,0x8e,0x78,0xac,0x8b,0xaa,0x0c, +0x45,0x88,0x5d,0x7c,0x7e,0x2f,0x0b,0xaa,0x08,0x56,0x92,0xd5, +0x52,0x68,0x10,0xf0,0xc7,0x1c,0x81,0x7c,0x3f,0x15,0xa1,0xbf, +0x25,0x33,0xe4,0x0b,0xfe,0xa0,0xbf,0x46,0x85,0xe8,0x48,0x95, +0x6a,0x69,0x58,0x89,0xe6,0x45,0x6f,0x3f,0xec,0x8a,0x8d,0x33, +0x7b,0xfb,0x8e,0x28,0xe5,0x4f,0x59,0x8e,0xe8,0x6f,0xa7,0x93, +0x61,0x0e,0xce,0xd6,0xa7,0x28,0xbf,0x1d,0x44,0x25,0x3c,0xfc, +0x24,0x8d,0x8c,0x30,0x0a,0x86,0xbc,0xba,0x99,0x5c,0x99,0xe3, +0x6e,0xe3,0xb5,0x54,0xfc,0x18,0x38,0xce,0xaa,0x38,0xcf,0xc8, +0xee,0x25,0xd0,0x6b,0xd8,0xa6,0x75,0x98,0x45,0x53,0xae,0x80, +0xea,0xb1,0xbc,0x5a,0xda,0x90,0x25,0xfd,0xcd,0xe9,0x6f,0x97, +0x9e,0xa6,0x89,0xe7,0x78,0x0f,0x84,0x2a,0x9a,0x90,0x45,0x5d, +0x81,0x82,0xb3,0x4c,0x60,0xbd,0x6c,0xb1,0x58,0x45,0xa3,0x8a, +0x3b,0x7f,0x3c,0xac,0x50,0x56,0x89,0x56,0x2c,0x7b,0x5c,0x74, +0x46,0x7d,0xc5,0xe3,0xbe,0x8e,0x64,0x2e,0x56,0x20,0x31,0x7f, +0xc9,0x3f,0xe8,0x11,0xe9,0x23,0x59,0xdf,0x60,0x59,0xfc,0x50, +0x5e,0x8a,0x23,0x79,0xa5,0x8a,0x0a,0xe9,0x47,0x39,0x5e,0xe8, +0x80,0x60,0x78,0xb0,0x6c,0xb1,0x1d,0xf4,0xe1,0x56,0xf9,0x97, +0x77,0x3a,0x19,0x1c,0x88,0xf9,0x49,0x6c,0xe3,0xcc,0x8d,0xcc, +0x82,0x75,0x99,0x5d,0x37,0x9c,0x3e,0xb4,0x58,0x31,0x73,0x06, +0xcc,0x7a,0xde,0x94,0x19,0xfb,0x22,0x8e,0x40,0x38,0x8b,0xef, +0x3e,0xcb,0x26,0x87,0x83,0xd9,0x6c,0x85,0xf1,0xf2,0x12,0x30, +0x43,0x39,0x51,0x87,0x15,0xe2,0x60,0x5f,0xd9,0x18,0x8e,0x99, +0x57,0x25,0x15,0x2b,0xed,0x48,0x76,0x47,0x77,0x7c,0x0e,0x38, +0x63,0xc6,0x34,0xec,0x7d,0x33,0x13,0x03,0xad,0x3a,0x6f,0x2c, +0xf8,0x8d,0xa6,0x1e,0xe4,0x87,0xcd,0x39,0x60,0x03,0x5a,0x00, +0x38,0x47,0xa9,0xce,0xcc,0x9e,0x9f,0x41,0x38,0x40,0xb0,0x79, +0x1d,0x1f,0x13,0x82,0x03,0x54,0x7c,0x21,0x7e,0xda,0x6c,0x94, +0x77,0x4b,0xd5,0xf5,0x4b,0x6a,0x86,0x37,0x28,0x39,0x86,0x0a, +0x26,0xa2,0x72,0x13,0x51,0x7d,0x79,0x22,0x5c,0x16,0x11,0x27, +0x89,0xf7,0x43,0x17,0x17,0xbf,0xd9,0x1e,0x51,0x72,0x1e,0x01, +0x2e,0xef,0x9d,0xc3,0xf0,0xd0,0x58,0x04,0x1d,0x0a,0x6a,0xd2, +0xad,0x82,0xf4,0x50,0xed,0xd8,0x9f,0xff,0xfd,0x95,0x83,0x08, +0xc7,0x89,0x8f,0x57,0x5b,0x09,0x6c,0x32,0x3b,0x8e,0xb7,0xa5, +0x7a,0x29,0xd7,0xb0,0xe5,0xaa,0xf2,0x39,0x51,0x3a,0x25,0xe7, +0xbc,0x40,0x11,0x93,0x3d,0x24,0xb2,0x41,0x86,0x11,0x3d,0xfe, +0x29,0x0e,0xb2,0xa2,0xb9,0x6d,0x91,0xc1,0x18,0xe9,0x9f,0xea, +0xdb,0x7c,0xa2,0x59,0x0b,0x5a,0xb5,0x3d,0x49,0x0a,0x17,0xf5, +0xce,0xa6,0x5b,0x7a,0x1b,0xdb,0x30,0x34,0xc0,0x1a,0xac,0x10, +0x9a,0xd4,0xac,0xb7,0xed,0xda,0x35,0xc3,0xc2,0x41,0x58,0x5d, +0x1d,0x7f,0x69,0x6e,0x1b,0x86,0xd2,0x65,0xde,0xd8,0xc5,0x23, +0xcb,0x8c,0x75,0x6d,0xcf,0xe0,0xe4,0x6b,0x4e,0x21,0x8b,0x59, +0xf7,0x4d,0x03,0x69,0xba,0xec,0x8d,0xa9,0x9a,0xc8,0xc0,0x8f, +0xeb,0x97,0x9a,0x06,0x67,0x7f,0x1e,0x5b,0x3f,0x41,0x14,0x13, +0x70,0x43,0x44,0x32,0xe1,0x07,0xd1,0xcc,0x26,0x43,0x57,0xc4, +0x7f,0xa8,0xaa,0x67,0xb3,0x32,0xd2,0xbc,0x3d,0xda,0xaa,0x89, +0x5b,0x2d,0xc6,0x63,0x77,0x1f,0xd4,0x5e,0x15,0x45,0xb6,0x08, +0xf3,0xc0,0xa9,0x5f,0x37,0x17,0x8c,0x09,0x3d,0xaa,0xb1,0x59, +0x01,0x91,0x55,0x75,0xf9,0xb6,0x1b,0xa7,0xd7,0xe3,0xd9,0xb3, +0x4f,0x2c,0x0a,0x88,0xbe,0x97,0x9a,0xab,0x0e,0x55,0x79,0x55, +0xa5,0xe9,0x0a,0x88,0x95,0xb3,0x79,0x11,0x35,0x7b,0x35,0xdf, +0xda,0x72,0xf1,0x8d,0x4c,0x9c,0x7f,0xb5,0x53,0x9c,0xd1,0xe6, +0x72,0x96,0x81,0xf3,0x03,0xb5,0x52,0xcc,0x0e,0xa8,0x5d,0x3b, +0x9d,0x09,0xe2,0xd5,0x7c,0xb3,0xad,0x95,0x82,0x8f,0x8f,0x43, +0xbc,0x68,0x4b,0x6e,0xbe,0x1f,0xd0,0x91,0xa5,0x67,0x9a,0x3f, +0xc5,0x4f,0xd1,0x5c,0x67,0x33,0x33,0x70,0x82,0x42,0x01,0xc7, +0xbe,0x42,0x48,0x5b,0xa6,0x27,0x1f,0xcd,0x3d,0xe6,0x16,0xdd, +0x86,0xd8,0xae,0xd2,0xa5,0xf2,0x84,0x44,0xfa,0x61,0x9b,0xdc, +0x56,0x98,0x1d,0x19,0x68,0xa8,0x7d,0x38,0xa9,0x6e,0x2c,0x50, +0x04,0x08,0x1b,0x70,0xbe,0x48,0xc9,0xf6,0x74,0x4a,0xfd,0x70, +0xdb,0x96,0x1d,0xb0,0xb1,0x59,0x0b,0x2e,0x4b,0x26,0x98,0xc2, +0x42,0xce,0x3b,0x4f,0xbd,0x22,0x13,0x95,0x0b,0x64,0x6e,0x56, +0x97,0xa8,0x1f,0x92,0xa0,0x30,0x5e,0xdf,0x4d,0x76,0x59,0x5c, +0x94,0x2d,0xb5,0x65,0xec,0xe8,0x20,0x79,0x0b,0xad,0x08,0x65, +0x3a,0xc9,0x03,0xc3,0xf2,0x3f,0x29,0xef,0x35,0x5c,0x0d,0x4f, +0x67,0x23,0xe5,0xd8,0xcc,0x17,0x14,0xed,0xfb,0x4f,0x57,0x93, +0x4f,0x55,0xe3,0xb2,0x10,0x3c,0x12,0x86,0x89,0x8f,0xa1,0xaa, +0x52,0x85,0xa8,0xbd,0xae,0x09,0xa7,0x89,0xd5,0x5f,0xc9,0x57, +0x9a,0xc6,0xf8,0x20,0xe5,0x65,0x1b,0xbe,0x46,0x34,0xdd,0xa7, +0xc3,0xe9,0xa2,0xe2,0x3c,0x72,0xff,0xf2,0x38,0x88,0x3f,0x8a, +0xab,0x4f,0x49,0xde,0x1f,0x28,0xb2,0xd5,0xe0,0xec,0x7e,0x34, +0xe6,0xe1,0xdd,0x39,0x10,0x2c,0x5b,0x8e,0x6e,0x3e,0xb8,0xe0, +0x6f,0x27,0xf1,0xd4,0x67,0xc2,0x94,0xaf,0x97,0x58,0xa4,0xaf, +0xdc,0x4a,0xd6,0xf9,0x65,0xc0,0x6e,0x34,0xc9,0x71,0x25,0xc6, +0x09,0x22,0xde,0x6e,0x9b,0x0f,0x1a,0x2c,0x24,0xd4,0x64,0xb4, +0x53,0x93,0x66,0xcc,0x31,0x50,0x84,0x7a,0x7b,0xf8,0x12,0x30, +0xa4,0x98,0xf9,0x99,0x87,0xeb,0xde,0x0a,0xde,0x41,0x2e,0xb9, +0x1b,0xce,0xdb,0x30,0xc6,0x91,0x37,0x1e,0x8a,0x56,0xe8,0x78, +0x4c,0xef,0xb0,0x47,0x76,0x66,0xfc,0xb9,0x41,0x0a,0xb2,0xd1, +0x60,0x39,0x63,0xc9,0x69,0x98,0xe8,0x37,0x83,0xae,0x8d,0xdc, +0x8d,0xff,0xa1,0x8c,0x2d,0x11,0x41,0x97,0x63,0x6e,0x5a,0x8e, +0xfe,0xae,0x53,0xc9,0xb3,0x5e,0xef,0x68,0x31,0xc7,0xaa,0x96, +0x24,0xde,0x77,0x24,0xd2,0x87,0xae,0x5c,0x79,0x4b,0x6d,0x91, +0x5f,0x36,0xeb,0xc5,0x2a,0xac,0xe2,0x52,0x79,0x60,0x9d,0x70, +0x73,0xb6,0xcc,0x74,0xce,0x7d,0xa0,0xe1,0x35,0x6b,0x26,0x11, +0xfb,0xc1,0xb5,0x04,0x5e,0x4c,0x83,0xe6,0xd5,0x24,0x44,0x1e, +0x65,0xec,0xd1,0x5f,0x99,0x99,0xe2,0x80,0xe6,0x8c,0x9c,0x73, +0xc8,0xa1,0x01,0xab,0x56,0x0a,0x47,0xc3,0x40,0x8b,0x3e,0x40, +0x2f,0x81,0x5e,0x30,0x41,0x01,0x50,0x8f,0xf1,0x18,0x6b,0x96, +0xff,0xce,0xd5,0x21,0xfe,0xfa,0x1b,0x57,0x84,0xb7,0xf1,0xbd, +0x6d,0x45,0xe0,0x45,0xb8,0x6d,0x05,0xf8,0xa5,0x7a,0xd0,0x4c, +0xef,0x52,0x1d,0x6e,0x53,0xd8,0x2f,0xd7,0xd1,0x86,0x48,0xd3, +0x57,0x3d,0xc3,0xcc,0xe0,0x6e,0x6a,0x1a,0xd5,0xf7,0xd5,0x14, +0xf0,0x8d,0x27,0xf3,0x8f,0xb3,0x82,0x6f,0x70,0x46,0x43,0xe0, +0x02,0xc7,0xf8,0x9f,0x16,0x1a,0x4b,0x58,0x89,0xc4,0x9e,0xb1, +0xeb,0x01,0x49,0x11,0xfc,0x25,0xcb,0xf1,0x1a,0x7f,0x3e,0xf3, +0x36,0xfe,0x19,0xd2,0x86,0xe2,0x5f,0xdf,0xad,0x4c,0x02,0x01, +0xe3,0x04,0x81,0xe5,0xd3,0x04,0xdc,0x26,0x76,0x37,0xc9,0x7c, +0x87,0x07,0x7c,0x4d,0x3e,0x88,0xeb,0x04,0xa8,0x0a,0x73,0x2c, +0xbe,0x88,0xa1,0x50,0x86,0x2d,0x15,0x5c,0x09,0x01,0x01,0x57, +0x13,0x63,0x50,0x0d,0x66,0x0c,0x18,0x81,0xaa,0x5d,0xcf,0x14, +0x9d,0xa1,0x4f,0x02,0x11,0xf6,0xbb,0x42,0x77,0xc4,0x57,0xb3, +0x76,0x5b,0x29,0xf0,0x86,0xb7,0x77,0x2a,0x03,0xc7,0x1e,0xb5, +0x41,0xe9,0xbc,0xc2,0x36,0xa9,0xb0,0x10,0xf8,0xd7,0xda,0xd2, +0x71,0xa0,0x38,0x29,0xd3,0xbc,0xee,0xfa,0x86,0x9e,0x38,0x02, +0xb2,0xf4,0x04,0xc2,0xfc,0xea,0xaa,0x77,0x50,0x78,0x8b,0xf2, +0xd2,0x37,0x9b,0xbb,0x67,0x3f,0x0a,0xfb,0x81,0xe7,0xbb,0xa1, +0x35,0xb0,0xb5,0x55,0x2f,0x8e,0xe8,0xdb,0x12,0x83,0x22,0x4e, +0xf6,0xae,0xd9,0xc4,0xc8,0x3d,0x66,0x97,0xb3,0xd3,0x05,0xc9, +0xec,0x77,0xc6,0xaa,0x5f,0x12,0xc3,0x67,0x1f,0x7e,0xa3,0xb9, +0x6e,0xfc,0xd5,0x01,0xcf,0xed,0x47,0x59,0xb7,0xc3,0x68,0xdf, +0x9c,0xb9,0x38,0xbe,0x37,0x73,0xc8,0x1e,0x25,0xae,0xae,0xe6, +0x93,0xdb,0x25,0x67,0xa4,0xee,0x65,0xd0,0x7e,0x62,0xb7,0xdc, +0xa3,0xa9,0xc5,0xa1,0x65,0xbd,0xb9,0xc5,0xba,0x40,0xcb,0xe4, +0x34,0xa5,0xb5,0xb4,0xee,0x1f,0x51,0x52,0x18,0x76,0xf9,0xbc, +0x4a,0x17,0xdf,0x50,0xf0,0xcd,0xf3,0x6c,0xff,0x91,0xa7,0x2a, +0xe4,0xfc,0x35,0xe6,0x48,0x3e,0xc5,0x94,0x32,0x11,0xd2,0x3e, +0x19,0x8a,0xd8,0x87,0x2e,0x31,0x39,0x46,0x2a,0x6e,0x8d,0x87, +0x39,0x7a,0x71,0xc1,0xd3,0x8b,0xf8,0xe6,0x81,0x3d,0x9c,0xa3, +0xc9,0xf7,0xd1,0xc9,0x0a,0xe7,0x18,0xc5,0x94,0x7a,0x44,0xca, +0x05,0x81,0x0f,0xcd,0x2e,0x0a,0xaf,0x56,0x0c,0x6b,0x94,0x3b, +0x2c,0xf7,0xf6,0xe0,0xa8,0x3c,0x0a,0x40,0xba,0xdd,0x41,0xc4, +0x2b,0x8a,0x15,0xe5,0xdc,0x0e,0xff,0xa0,0x1f,0xf4,0xec,0xd6, +0x1a,0x0b,0xa2,0x4f,0x20,0xe8,0xb8,0x05,0x6e,0xda,0xa4,0x02, +0xe2,0xd1,0x71,0x16,0x6f,0xb5,0xf2,0x13,0x73,0x46,0xd2,0xcc, +0x82,0x25,0x1f,0x4d,0x68,0xc2,0xec,0xf0,0xdb,0x3c,0xca,0x68, +0xc7,0x51,0x9e,0x14,0x14,0x65,0x78,0x3e,0x42,0x41,0xd8,0xe7, +0x52,0x36,0x0d,0xc7,0xbf,0x01,0x7a,0x16,0xd4,0x49,0x08,0xaa, +0x7d,0xdf,0x40,0x74,0xd1,0x01,0x93,0x0a,0x32,0x7d,0xc6,0x60, +0xb7,0xc1,0x79,0xbc,0x99,0x80,0xc0,0x9d,0xce,0x3f,0x16,0x7f, +0x3a,0x38,0x80,0xc3,0xb7,0x5a,0x15,0x28,0x92,0xe6,0xb8,0x24, +0xe4,0x67,0x28,0x64,0x0d,0xf8,0xf7,0x9c,0xdf,0x6d,0xcd,0x52, +0x0e,0xdb,0xd1,0x16,0x7b,0x96,0x2a,0xd3,0x89,0xde,0xe8,0xc2, +0xe5,0xd5,0x64,0x17,0x44,0xae,0xe2,0x2b,0xa6,0xac,0x37,0x99, +0x2c,0x86,0x80,0x9a,0x64,0x3b,0x98,0xbe,0x3a,0xa7,0xc6,0x0f, +0x28,0x6b,0x6c,0x82,0x2a,0xf3,0x87,0x1e,0xb1,0x37,0xad,0x9f, +0x0f,0xef,0x49,0x38,0x72,0xe3,0xee,0x0e,0x08,0xae,0x09,0xf9, +0x55,0x6c,0x4c,0xd3,0xed,0x77,0x62,0x91,0x4e,0xd4,0x88,0xe2, +0x68,0x4e,0xbe,0x42,0x51,0x6a,0x27,0xbf,0x98,0x7f,0x2a,0xef, +0x91,0x56,0x41,0x86,0xf7,0xd5,0xaa,0x80,0x91,0xa6,0x3e,0x89, +0x97,0x75,0x74,0x82,0x6f,0xfd,0x21,0x06,0x26,0xe6,0x49,0xad, +0x6a,0x7e,0x99,0x39,0x3a,0x8b,0xc0,0xa2,0x34,0x67,0x2c,0x09, +0xaa,0xa3,0x61,0xfd,0x62,0x53,0x13,0x68,0xe8,0x7b,0xa9,0x75, +0x59,0x76,0x3d,0x5e,0xfd,0x80,0x9e,0x38,0xe0,0x9c,0x39,0x99, +0x4e,0xa0,0xf0,0x5b,0xb8,0xbc,0x9a,0x1c,0x97,0xc1,0xfc,0x62, +0xeb,0xb7,0x17,0x42,0x65,0x23,0xe8,0x63,0x49,0xd6,0xaa,0xf1, +0xd1,0x8f,0x5a,0x9b,0x53,0x77,0xa1,0x64,0x17,0x7e,0x3b,0xf8, +0x1c,0xd8,0x5d,0x0c,0xaf,0xc7,0xff,0xc5,0x43,0x46,0xd2,0xac, +0x4f,0x89,0x8b,0xa4,0x7c,0x99,0x6c,0x4f,0x9b,0xdb,0xbd,0x24, +0xd8,0x67,0x28,0xbe,0x76,0x90,0xf1,0x80,0x21,0x20,0x0c,0x78, +0x48,0x7f,0x6f,0x86,0x44,0xc6,0xc5,0x63,0x50,0x18,0x89,0xb0, +0x36,0x19,0x02,0x72,0x6b,0x5a,0x96,0xcc,0xbd,0x9b,0xc9,0x2d, +0xac,0xf5,0x89,0x18,0xee,0x53,0x9f,0x68,0x86,0x9b,0xc7,0x06, +0x51,0x35,0x25,0xc9,0x8e,0x12,0x7c,0xac,0x9c,0x5c,0x92,0x6e, +0x94,0xb3,0x77,0x6b,0x4c,0xac,0x3a,0x53,0xb2,0x65,0xb2,0x1c, +0x4f,0x49,0x81,0x3d,0xe1,0xf5,0x02,0x08,0x2b,0x74,0x19,0x99, +0xe0,0x6c,0xa9,0xfe,0x6e,0x49,0xf3,0x2c,0xef,0xa9,0x6c,0x9a, +0x1e,0x22,0xb9,0xa5,0xa5,0x24,0x4b,0x1f,0xd2,0x4c,0xac,0xf5, +0x22,0x83,0xab,0xd6,0x3c,0xd2,0x97,0x96,0x0c,0xc4,0xe7,0xaa, +0xe5,0x20,0xe3,0xbb,0x24,0xf6,0x70,0x89,0x86,0x1d,0xc7,0x6a, +0xbe,0xb7,0x4c,0xbd,0x73,0x33,0xd3,0x23,0x24,0x2d,0x26,0x9f, +0xc6,0x23,0x1b,0xe7,0x0f,0x4d,0xf5,0x01,0xc9,0x7c,0xcc,0xbc, +0xef,0x5a,0x66,0x9a,0x30,0xc8,0x1e,0xef,0x1d,0xca,0xab,0xd1, +0x3d,0x71,0xfe,0x5d,0x95,0xf7,0x9b,0xa3,0x5a,0x3b,0x7a,0xa9, +0xcd,0xa1,0x7d,0xf7,0x56,0x79,0x73,0x1e,0xbc,0xd2,0xe4,0xc2, +0x75,0xa7,0x83,0x19,0xc4,0x93,0x9d,0xfb,0x22,0x95,0x12,0xc6, +0x2a,0xb7,0x6b,0x67,0x86,0x6f,0x9b,0x5a,0xb4,0x96,0x81,0x56, +0x18,0xad,0xcc,0x46,0x84,0x81,0x0e,0x1d,0xa3,0x85,0xee,0xf4, +0x71,0x86,0xea,0x5c,0x78,0xe2,0xa7,0x11,0xb6,0x0e,0xd3,0x9d, +0x4e,0xb5,0x10,0xe9,0x5d,0xc6,0x50,0x31,0x6c,0xe3,0xf7,0x5c, +0x29,0xda,0x52,0xdc,0xfe,0xb8,0xfc,0x94,0x8d,0xed,0xeb,0x73, +0x1a,0xb4,0xdf,0xa6,0x18,0x10,0xb4,0x99,0xa4,0x8f,0x59,0x9e, +0xa0,0x91,0xc2,0xeb,0xc4,0x7b,0x16,0xe9,0x69,0x31,0x4a,0x30, +0x8d,0x67,0x3c,0xbc,0xd0,0x29,0xc0,0x25,0xc4,0x51,0x1e,0x05, +0x31,0xac,0x3d,0xe7,0x23,0xcf,0xdc,0x5c,0xa8,0xca,0xc2,0xc7, +0x9e,0xe0,0xd0,0xd4,0xa0,0x37,0xc4,0x45,0xcd,0xf9,0x23,0x67, +0x9e,0xaa,0x2c,0xf5,0x07,0x0e,0x4e,0x65,0xc3,0xb8,0xd5,0xf8, +0xc0,0xd6,0xec,0xc9,0x76,0x7f,0xe8,0x24,0x2b,0x52,0xdb,0x1a, +0x79,0xb9,0x31,0x31,0xb0,0xd2,0x04,0x8a,0x49,0xf2,0x82,0x6e, +0xce,0x53,0xad,0x2d,0x83,0xe7,0x3b,0x5b,0x21,0x43,0xb3,0xbc, +0x41,0xcf,0x6c,0x36,0x07,0x2c,0x0b,0x7a,0x4f,0x73,0x6d,0xbb, +0x55,0x03,0xcd,0xa6,0x90,0x61,0x2d,0xb0,0x2d,0x64,0x62,0xbf, +0x06,0x10,0x68,0xaa,0xbf,0x02,0x04,0x2e,0x05,0x0f,0x43,0x4e, +0xf0,0xb9,0x6b,0xc5,0x5e,0xdc,0xac,0xbd,0xda,0x60,0xb3,0x25, +0x62,0xf2,0x4f,0xb8,0xa3,0x27,0xf7,0xdc,0x37,0x62,0xaf,0xd6, +0xac,0x6d,0x20,0xc8,0x97,0xa1,0x88,0x69,0xe4,0xb6,0xb9,0x5f, +0x69,0xa3,0x57,0x9f,0x9a,0x1f,0x9e,0xcd,0x07,0x74,0x2a,0x6c, +0x36,0x4f,0x47,0x35,0xce,0xaf,0x4e,0x2e,0xa9,0xaa,0xea,0xd9, +0xbf,0x65,0xb4,0x83,0xab,0x22,0x0b,0x5a,0xd7,0xb0,0x16,0xf8, +0xc3,0x98,0xeb,0xfb,0x8a,0x96,0x37,0x91,0xf8,0xee,0x62,0x2c, +0x12,0x73,0x73,0x26,0xb9,0xbb,0x7b,0x39,0x5e,0xee,0x66,0x6b, +0x4d,0x5b,0x2c,0x96,0xb1,0x25,0x76,0xa2,0x0b,0xe0,0x4c,0xda, +0xff,0xaf,0x7d,0x35,0x36,0x0d,0xd1,0xe6,0xf9,0xa1,0x66,0x75, +0xcd,0x58,0xe3,0xfd,0xb2,0xff,0x48,0xa9,0xeb,0x83,0xda,0xc9, +0x0d,0x71,0x8a,0xc0,0x4d,0x45,0xe4,0x8d,0xaa,0x07,0x59,0x7b, +0xd8,0x18,0x24,0x38,0x31,0x0c,0xff,0xf8,0x71,0x19,0x2d,0x6b, +0x75,0xb7,0x0d,0x95,0xb6,0x97,0x1d,0xaf,0xea,0x47,0x2a,0x00, +0x1c,0x66,0x82,0xfe,0x4f,0x7b,0x5e,0xaa,0x29,0x33,0x88,0x46, +0x4a,0x8e,0xa4,0x72,0xcc,0x00,0xbf,0x45,0x3d,0xd3,0x19,0x52, +0x28,0x59,0x50,0x01,0x1a,0x28,0xc8,0x59,0xd0,0x18,0x10,0x7c, +0xdf,0xe9,0x9a,0xdf,0x48,0xf1,0xae,0x14,0xcf,0xa6,0xf8,0x26, +0x74,0xae,0x51,0x68,0xc7,0x16,0xe2,0x20,0x25,0xe6,0x9a,0x04, +0xa6,0xf7,0xb8,0xbc,0x33,0xa2,0x4e,0x6e,0xce,0x93,0x0e,0xe5, +0x13,0x95,0x04,0xb6,0x48,0x87,0x0f,0xf2,0xf3,0xbb,0x95,0x26, +0x1a,0x98,0x42,0x2a,0x3c,0x25,0x1f,0x1f,0x18,0xfb,0x2a,0x32, +0x05,0xab,0x71,0x66,0x0d,0xf0,0x86,0xb6,0xe2,0x49,0x97,0x37, +0x80,0x9c,0x92,0xdd,0xef,0xb8,0xcb,0xe4,0x8e,0xd7,0x57,0x61, +0x9e,0xa5,0x1c,0x83,0x9d,0xc1,0xa9,0xbb,0xaf,0x30,0xb7,0xd4, +0x25,0x4e,0xc0,0xd4,0x15,0xe0,0xce,0x41,0x4f,0x4e,0x15,0xe8, +0x17,0x9f,0x74,0x46,0x51,0xe2,0x4b,0xab,0x92,0x46,0xc4,0x34, +0x1e,0xd6,0xd0,0x51,0x6c,0x34,0xd0,0x2e,0x51,0xf6,0x0d,0xb2, +0x75,0xc9,0x25,0x8c,0x90,0xf2,0xc8,0x0e,0x8a,0xbf,0x94,0x91, +0x9b,0xc6,0x5e,0xaa,0xa4,0x40,0x0d,0xb9,0x27,0x6a,0xff,0x76, +0xf8,0x89,0x91,0x49,0x74,0x5e,0x4e,0xea,0x0b,0xb0,0xca,0x04, +0x1a,0x52,0x11,0x71,0x94,0xc9,0xcb,0xe7,0x47,0x90,0xb7,0x21, +0x2a,0x68,0x5f,0x96,0x15,0x35,0x87,0x8a,0xbd,0xf8,0xc8,0xa6, +0x11,0x12,0x7f,0x34,0xc1,0x93,0x6e,0x34,0x7e,0xea,0xd7,0x86, +0xa3,0xdc,0x22,0xe8,0x89,0x7d,0x7f,0xa3,0x2d,0xa9,0x49,0x6e, +0x07,0x78,0x07,0x66,0x5c,0x81,0x8a,0xbb,0xef,0xca,0x39,0x9b, +0x27,0x3a,0x69,0x9e,0x45,0xc5,0xd0,0x50,0xf4,0x15,0x7f,0xbb, +0xc3,0xdb,0x91,0x0d,0x0b,0x20,0x8d,0x4a,0x13,0x36,0x91,0x83, +0x2f,0x2b,0x0d,0xce,0x7a,0x38,0x33,0xb4,0x43,0xc5,0x1a,0x7d, +0x1e,0x8e,0xc8,0x75,0xb2,0xbc,0xd2,0x67,0x76,0x0f,0x48,0xd5, +0xb8,0x85,0x0f,0x44,0x97,0xa8,0xf6,0xab,0xc6,0xe4,0x6e,0x27, +0xb3,0xa7,0xcb,0x25,0x5a,0x35,0xbb,0x9b,0x7a,0x0e,0x7e,0xc4, +0x7e,0xb9,0x85,0xd5,0x08,0x27,0x32,0xa2,0x8a,0x68,0x3b,0xc7, +0xed,0x13,0xbc,0xad,0xd9,0x50,0x75,0x99,0xf4,0x93,0x8e,0x40, +0xe9,0xf1,0x4f,0x07,0xd6,0x37,0x2a,0x48,0x77,0xb0,0x4c,0x27, +0x19,0xb4,0xd0,0x04,0xe1,0xed,0x02,0xf1,0x68,0x03,0xe1,0x70, +0xe0,0x62,0x8f,0x97,0xd7,0x95,0x35,0x73,0x4a,0x6f,0x59,0x93, +0xab,0xcf,0x86,0x88,0xbe,0x78,0xef,0x9a,0x68,0xd1,0x5c,0x88, +0xee,0xc1,0x1f,0xc0,0x02,0xd8,0x46,0x4c,0xea,0x6d,0x6b,0xec, +0xb7,0xce,0xef,0xfb,0xe7,0xd5,0xf9,0xe9,0x00,0x6d,0x20,0xa3, +0xbe,0xf6,0xa4,0xfa,0x49,0x07,0x85,0x4b,0x26,0x6e,0x90,0x12, +0x48,0x14,0x08,0x0a,0xb3,0x40,0xad,0xd8,0xae,0x6b,0x0b,0xc5, +0xb3,0xb3,0x3d,0x19,0x94,0x52,0x3c,0x19,0x68,0x2f,0x7c,0xcd, +0xe7,0xf7,0xe7,0xa3,0xce,0xf9,0x66,0xff,0xda,0x5c,0x60,0x28, +0x8c,0x2b,0x8f,0xa7,0xcb,0x6b,0x61,0xff,0x77,0x28,0x56,0x6c, +0xd9,0x1e,0xe6,0x7b,0x87,0xee,0x38,0x60,0xe6,0xfb,0xa3,0xef, +0xed,0x08,0xc9,0xd9,0xd4,0x43,0x2e,0x42,0x69,0x13,0xfa,0x92, +0x7b,0x90,0x9b,0xa1,0x81,0x4c,0x59,0x0f,0xfe,0x34,0x0f,0x4d, +0xef,0xa1,0xb1,0xa1,0xa2,0x8d,0x63,0xa3,0x40,0x8b,0xd5,0xfc, +0xdf,0x2b,0xa9,0x24,0x13,0x33,0x3f,0x59,0xae,0xe3,0x22,0xbf, +0x9d,0xe4,0x7c,0x46,0xf6,0x36,0xf6,0xf7,0x49,0x47,0x09,0x79, +0x31,0xd7,0x40,0x4d,0x2d,0xdf,0x03,0xf6,0xb3,0xbc,0xde,0x4f, +0x3a,0x1f,0xc6,0xcb,0x8a,0x9e,0x69,0x61,0x7b,0x75,0x92,0xfd, +0xfa,0x3a,0xda,0xf7,0xeb,0x28,0x9f,0x94,0x8f,0x8e,0x26,0xdf, +0xc7,0x27,0x35,0x7b,0x5f,0x8e,0x2a,0x46,0x63,0x4a,0x93,0x5e, +0xd2,0x83,0xd2,0x6d,0x34,0xcb,0xb0,0x00,0xc8,0x93,0xbd,0x47, +0x5b,0xcd,0xcd,0xfb,0x49,0xf6,0xcb,0x73,0x02,0x87,0x8e,0x13, +0xc8,0x19,0x7f,0x6c,0x91,0x09,0x33,0x9d,0x7a,0x38,0x0b,0xdc, +0x3e,0x12,0xeb,0xf1,0x2f,0x26,0xbf,0x8e,0x79,0x99,0xa2,0x3f, +0x5a,0xe6,0x10,0x5f,0xbc,0x5f,0xaf,0xdd,0xa9,0x8c,0x69,0xc2, +0x3b,0xd9,0x39,0x3c,0x12,0xc7,0xd4,0xf8,0x1a,0x85,0xeb,0x57, +0x3a,0x23,0xf5,0x1d,0x96,0x54,0xd8,0x88,0x79,0xc8,0xd4,0xef, +0x1c,0x14,0x6a,0xac,0x1e,0xc1,0xad,0xd7,0x3b,0xd1,0x66,0x80, +0x8b,0x0b,0xed,0x7d,0x28,0x8f,0xb2,0xc6,0x2f,0xd7,0xeb,0xcd, +0xdb,0x2f,0xe6,0xd0,0xde,0x21,0x79,0x73,0xc4,0x25,0xee,0x3a, +0x06,0x4b,0xfd,0x29,0xde,0xd4,0x10,0x9b,0x4f,0x48,0x6c,0x74, +0xbe,0x54,0xbb,0x9d,0x5e,0x83,0x08,0x7a,0x8b,0xc5,0xf0,0x3d, +0x36,0x6c,0x4a,0x66,0x74,0x8b,0xf0,0x3b,0x71,0x7c,0x5f,0xb8, +0x44,0x21,0x1e,0xd0,0x54,0xae,0xdb,0x7d,0xe2,0x51,0xaf,0x1f, +0xd8,0xfd,0x9b,0x61,0xf5,0xfa,0xe3,0x4c,0xdd,0x2a,0x92,0xcd, +0xac,0x76,0x5b,0x5b,0x22,0x46,0x11,0xc4,0x18,0x02,0x64,0x27, +0x35,0xa5,0x1c,0x33,0xf1,0xeb,0x2e,0x37,0xac,0x2b,0xbd,0x60, +0x88,0xf2,0xb1,0x53,0x96,0x12,0xca,0x4c,0x6c,0x5a,0xeb,0xa4, +0x03,0x14,0xcf,0xb1,0xf6,0xad,0x3c,0x38,0x82,0xbf,0xdf,0x9b, +0x19,0x84,0x4f,0x58,0x90,0x5f,0xdd,0xc8,0x68,0x8c,0xfe,0xb7, +0x7a,0xad,0xe8,0xf9,0xc5,0x7b,0x3f,0x87,0x15,0x69,0x1f,0xfd, +0xc7,0xf8,0x73,0xc5,0x6b,0x53,0x66,0x0d,0xf9,0xf2,0x4e,0x0c, +0xf6,0x6b,0xea,0xc7,0x32,0xcc,0x98,0xb6,0x55,0x51,0x6c,0x45, +0x4a,0x19,0xb5,0x65,0x73,0x4a,0x35,0x6f,0x59,0x3c,0xa6,0xea, +0xaa,0x8c,0xda,0xc8,0xde,0xd9,0xbf,0x27,0xd8,0x66,0x8f,0x3f, +0x38,0x02,0xb8,0x5d,0x69,0x14,0xd8,0x48,0x85,0x69,0x9e,0x6f, +0xd7,0x72,0x8c,0xc7,0xcd,0x58,0xdc,0x2d,0x28,0xb8,0x67,0xb3, +0xfa,0xd5,0xc7,0x58,0x02,0x99,0x68,0xf3,0xd9,0x8c,0x57,0x0c, +0xb3,0xf9,0x67,0xd0,0x88,0x9f,0x26,0x23,0x43,0xe9,0x8d,0x26, +0xd7,0x93,0x95,0x9c,0xef,0x77,0x93,0x51,0xf0,0xb6,0x7b,0x44, +0x87,0x83,0x3c,0xf3,0xe6,0x94,0x13,0xf3,0xe8,0xcb,0x29,0x6a, +0xb3,0xa3,0x19,0xae,0xe3,0x55,0x0a,0x44,0xeb,0x1f,0xff,0xad, +0x2c,0x29,0x8b,0x32,0x5c,0x7d,0xce,0x32,0x39,0x4e,0x72,0x04, +0xce,0x9e,0x20,0x12,0x12,0x87,0xf8,0xcb,0xc1,0xce,0xd6,0xfc, +0x7c,0x7c,0x77,0xaf,0x96,0xf3,0xdb,0x13,0xa9,0x24,0xa5,0xbc, +0x9d,0xc3,0x2c,0x04,0x64,0x2b,0x39,0x48,0x36,0x1e,0x94,0x13, +0x30,0x86,0xe8,0xb0,0x8c,0x59,0x6d,0xe3,0xd5,0x8f,0xc3,0xea, +0x06,0xa6,0x92,0xe6,0x1d,0xee,0xbb,0x1b,0xba,0x9d,0x77,0x77, +0x6f,0x38,0xba,0xbc,0x29,0xe4,0x31,0x38,0x88,0xf6,0x10,0x84, +0x5f,0x33,0xaa,0xe4,0x84,0xb9,0x29,0x21,0xe8,0x33,0xea,0xcc, +0xea,0x8e,0xf5,0x38,0x43,0x88,0x29,0xc0,0x1f,0xd8,0xe3,0x50, +0xb4,0xdd,0xb6,0xa7,0x58,0xd3,0x0c,0x43,0x26,0xda,0x43,0xe2, +0x96,0x19,0x51,0x3c,0xd3,0x0d,0x48,0x85,0x6e,0xe4,0xf0,0x63, +0x0e,0x7c,0xbe,0x31,0xfc,0x33,0x19,0x32,0xe6,0x9e,0x73,0x64, +0x7e,0x78,0x90,0xd9,0x9c,0xe4,0x4b,0x5a,0x78,0x2d,0x91,0xc3, +0x47,0xe9,0x2a,0xdc,0x2d,0xa9,0x04,0x0d,0xee,0xb5,0xc9,0xef, +0xdd,0x71,0x59,0x70,0xf1,0x4d,0x96,0x29,0xe6,0xea,0x97,0xdf, +0x1c,0xe9,0xba,0x7b,0x53,0x23,0x69,0x2f,0xc2,0xb6,0xdd,0x75, +0xf5,0xec,0xfa,0x54,0xea,0xe2,0x33,0xe7,0x5c,0xaa,0x8e,0xf2, +0x79,0xb2,0x16,0x86,0xc2,0x5e,0x71,0x1e,0x39,0x08,0xca,0x1a, +0x24,0x21,0xf4,0x41,0x65,0xcb,0x3e,0xf9,0x42,0xc9,0x9d,0xad, +0x25,0x45,0x72,0x2d,0x6c,0x2e,0xf3,0x19,0x29,0xae,0xe1,0xb1, +0xbb,0x06,0x43,0x30,0x97,0xb0,0xf2,0x46,0xbf,0x59,0xb6,0x94, +0x38,0x9d,0x0c,0x4b,0xc5,0x97,0x69,0xbd,0xa7,0xf8,0xaa,0x1f, +0x96,0xe9,0x73,0x03,0x5b,0x98,0x34,0xc0,0xa2,0x82,0x8d,0x89, +0x28,0xa3,0x1c,0x28,0x31,0x32,0xb1,0x05,0x08,0xa5,0x7d,0x15, +0x94,0x67,0xc1,0x4a,0x77,0x80,0xea,0xd4,0x49,0xad,0x0f,0xa2, +0x4a,0xd4,0xdc,0x00,0x4e,0xfc,0xaa,0x16,0x18,0xac,0xe7,0xde, +0xdf,0x4c,0x7c,0x0d,0x2b,0x91,0xc2,0x5f,0xea,0x59,0x83,0xbf, +0x86,0xd3,0xf1,0x52,0x12,0xcc,0x2b,0x65,0x30,0xa3,0x97,0x70, +0x35,0x35,0x03,0xde,0x1d,0x43,0xde,0x63,0xb6,0x03,0x44,0xdf, +0x1f,0x09,0x35,0xb5,0x03,0x83,0x8a,0x33,0xa6,0xac,0xeb,0x1a, +0x26,0x0c,0xc8,0x55,0x7d,0xd8,0xb3,0x10,0x27,0x9a,0x54,0x6a, +0xad,0x04,0x71,0x33,0xbf,0x26,0x59,0xe8,0x16,0x7f,0x1c,0x6f, +0x6e,0xbd,0xa6,0x4f,0xec,0x17,0x05,0xd0,0xe5,0x82,0x81,0x84, +0x8c,0xc2,0xed,0x98,0x15,0x8b,0x5a,0xd3,0x35,0x1d,0x1d,0x79, +0x0d,0xe7,0x18,0x0b,0xf3,0x4c,0x27,0x15,0x7b,0x8d,0xc0,0x82, +0x7c,0x57,0x37,0xdf,0x99,0xb5,0x42,0x78,0xe6,0x49,0x39,0x7b, +0x76,0x5d,0x4e,0xef,0x46,0xe3,0x0a,0xce,0x58,0x58,0x74,0xb9, +0x19,0xb2,0xbd,0xc3,0x9d,0x52,0x2d,0x45,0xf8,0x44,0x5b,0x32, +0x4e,0xe3,0xa7,0x4f,0xf8,0x52,0x0b,0x13,0x99,0x73,0xc9,0x63, +0x22,0x69,0xa8,0x8f,0x8c,0xc7,0x3b,0x42,0xf7,0x31,0x83,0x45, +0x03,0x28,0x1b,0x1a,0xf0,0xc3,0x14,0xbc,0xed,0x95,0x62,0x3a, +0xd2,0xd6,0xf9,0x96,0x50,0x9e,0xb7,0x64,0x20,0x45,0x5a,0x63, +0x18,0x28,0x7c,0x45,0x87,0x8d,0xe5,0xe4,0x80,0x08,0x23,0xcf, +0x24,0xdc,0x8c,0x48,0x76,0x2a,0x8f,0xf6,0x0e,0x2c,0xcb,0xcf, +0x69,0x35,0xbf,0x5b,0x5e,0x8e,0x73,0x18,0xb8,0xd5,0x64,0xc6, +0xd2,0xa1,0x74,0xe7,0xeb,0x7e,0xe0,0xf4,0x0c,0xe9,0x02,0x5e, +0xc7,0x12,0x21,0xa6,0xa1,0x66,0xd7,0x2f,0x99,0x2c,0x4f,0x2e, +0x17,0x1f,0xd1,0x93,0xc5,0xc9,0x70,0xf6,0x87,0x15,0x81,0xde, +0x69,0xbd,0x1c,0xfe,0x0a,0xeb,0x12,0x3f,0x26,0xe3,0xaa,0x05, +0x9b,0x97,0x61,0xb6,0xe6,0xcb,0x16,0xc1,0x82,0x06,0x57,0x2b, +0xb4,0x48,0x5b,0xc1,0x3a,0x25,0xf3,0x0f,0x95,0xea,0x9f,0x77, +0x19,0x47,0x88,0x9b,0xc5,0x95,0xc3,0x68,0xda,0x84,0x7a,0x4b, +0x26,0x51,0x43,0x4e,0xa9,0x58,0x6b,0x38,0x1b,0xb5,0x4c,0x41, +0x18,0xee,0xb1,0x5c,0x40,0xb0,0xde,0xa4,0x46,0x3d,0xd0,0xb4, +0x9b,0xf7,0xa6,0x80,0xcc,0xab,0x88,0x71,0xf8,0x49,0xe7,0xbc, +0x76,0xd6,0x6d,0x21,0x56,0x61,0xf2,0x63,0x4d,0x94,0x1a,0x59, +0xd1,0x52,0x8c,0xc1,0x64,0xf6,0x98,0x83,0xee,0x1f,0x93,0xda, +0x8c,0x0f,0x88,0xb0,0x9a,0xcf,0x46,0xe8,0xc1,0x26,0x8b,0xf1, +0x65,0xe9,0x51,0x9c,0xb1,0xb4,0x4d,0xc3,0x88,0xec,0xc8,0xe2, +0x13,0xa6,0xf6,0x1b,0x67,0xfc,0xcb,0x47,0x96,0xb2,0x7e,0xf8, +0x87,0x35,0xb4,0xed,0xf8,0xf5,0x6c,0x5e,0xbb,0xe0,0xfa,0x03, +0xa4,0xe1,0xe9,0x02,0x6a,0xca,0x8a,0x14,0xac,0x49,0xc4,0x67, +0x27,0xd4,0xb9,0x45,0x9e,0xb0,0x3b,0xf2,0xb7,0x95,0x0b,0x92, +0xbb,0x92,0xab,0x70,0xb7,0xb3,0x44,0x10,0xcb,0xb4,0xb9,0x6d, +0xf7,0x80,0xba,0xb8,0x4d,0x65,0x32,0x19,0xc4,0xf1,0x06,0x0e, +0xa8,0xf9,0xc7,0x13,0x2a,0xb4,0xbc,0xcc,0x09,0xff,0x83,0x3f, +0x88,0xc2,0x91,0x6d,0xec,0x78,0xec,0x97,0x97,0x19,0xfc,0x6b, +0xa6,0x55,0x92,0xdd,0x5d,0x54,0x5a,0x41,0x3b,0xc5,0xd5,0xdd, +0x05,0x5c,0xf5,0xe9,0x41,0xfe,0x88,0xda,0xa7,0xa8,0x1e,0x14, +0x95,0xa9,0xd1,0x46,0x59,0xfc,0x6e,0xfc,0xdb,0xdd,0x70,0x0a, +0xb4,0xc8,0x61,0x3e,0x7f,0x44,0xbb,0x64,0x4e,0xf4,0xf8,0x23, +0x25,0xde,0x49,0xab,0x94,0x29,0xfc,0x72,0x7e,0xc8,0x02,0xbb, +0x41,0x86,0x43,0x22,0x8a,0x51,0xf3,0x72,0x7e,0x08,0xf7,0x10, +0xfa,0x4e,0xb0,0xa5,0x3d,0xe9,0x8f,0x2b,0x0e,0xff,0x9d,0x8e, +0x57,0xf9,0xea,0xd0,0xc9,0xb4,0x1d,0xe6,0xab,0x47,0xee,0xe3, +0x11,0x99,0xdc,0x83,0x16,0xac,0x1e,0x05,0xe8,0xce,0x21,0x5b, +0x8c,0x70,0xec,0x80,0xc3,0x4c,0x0e,0x1a,0x9a,0x66,0xfc,0x74, +0xec,0x32,0x8e,0xc1,0x21,0x98,0x1f,0xfa,0x99,0xc7,0x46,0xf9, +0x2f,0xde,0xab,0x32,0xd5,0x50,0x1a,0x48,0x4e,0xf7,0xf5,0xc8, +0xc1,0x82,0xf2,0x9e,0x9d,0x44,0x00,0xfc,0x27,0x43,0x90,0xcb, +0xf9,0xd0,0x9c,0x70,0x50,0xde,0xb2,0x41,0x0e,0x83,0x2f,0x4e, +0x93,0x3e,0x3c,0x32,0x0c,0x12,0x1e,0x96,0x52,0xe6,0x5e,0x69, +0xd4,0x43,0xbc,0x00,0x12,0x37,0xbf,0x48,0x31,0x11,0xb5,0xd4, +0x6e,0x07,0x9c,0x90,0x43,0x9e,0x5e,0x1e,0x22,0x37,0x9f,0x4c, +0x2b,0xcf,0x1f,0x49,0x62,0x54,0x15,0x2f,0x8d,0x9d,0x83,0x8d, +0xad,0xef,0x11,0xd6,0xb7,0xc3,0x19,0xb6,0xad,0xb6,0x86,0xc6, +0x48,0x1d,0x3b,0x81,0x79,0xb7,0xa0,0x85,0xb5,0x46,0xe8,0xf2, +0xd8,0x04,0xb3,0x6a,0x47,0x04,0x12,0x52,0x65,0x19,0x1d,0xaa, +0x00,0x11,0x4d,0x82,0x84,0xef,0x3d,0x57,0x42,0xf8,0x3b,0xc8, +0xa0,0x11,0x8e,0xc4,0xd7,0x0c,0x84,0x6b,0xc3,0x46,0xa3,0xec, +0xed,0x46,0xce,0xd1,0xe8,0x26,0x3c,0xcc,0xe9,0xe7,0x11,0x23, +0x11,0xc6,0xc1,0x1e,0xa6,0xa9,0x1b,0x35,0xe2,0x6a,0x32,0xe6, +0x11,0xe4,0xb6,0xf4,0x22,0x66,0x41,0x0e,0x26,0x12,0x72,0x15, +0x6b,0x38,0x3f,0x0f,0x40,0x06,0x40,0x82,0x2c,0x24,0x46,0xe3, +0x20,0x5d,0x88,0x3f,0xab,0x2b,0x27,0x11,0x8b,0xce,0xb4,0x3f, +0x87,0x8c,0x65,0xf5,0xda,0xf5,0xa8,0x17,0x54,0x2c,0x14,0xe4, +0xa3,0xac,0x70,0xd8,0x96,0x9d,0x2a,0x00,0x88,0xfe,0xf9,0xdc, +0xe5,0x0a,0x9b,0x92,0x37,0x46,0xef,0x6a,0x56,0xb8,0x5a,0x94, +0x39,0x5b,0x27,0xa6,0xe2,0xea,0xa1,0x98,0xa0,0x96,0xd4,0x5e, +0x0f,0xa1,0x69,0xdc,0xb4,0x71,0xe8,0x8c,0xa2,0xd8,0x52,0x30, +0xdb,0x34,0x3c,0x46,0x7f,0xa1,0x56,0xa3,0x3d,0x05,0x09,0x54, +0x19,0xc6,0x5b,0xd6,0x06,0xb3,0x78,0xdf,0x32,0xcb,0x78,0x19, +0x3a,0x05,0x65,0xc7,0xa9,0xf4,0x29,0x23,0xb6,0xff,0xcb,0xf9, +0x6e,0x47,0x9e,0x63,0x71,0x63,0xe0,0x3b,0xa4,0x5f,0xfc,0x85, +0xdb,0xf0,0xde,0x71,0x68,0x99,0xec,0xfe,0xfc,0xfc,0xd5,0x93, +0xd7,0x3f,0x8b,0xbf,0xd0,0x76,0x5b,0xc9,0x02,0x75,0x90,0x2e, +0xb9,0x9e,0xbc,0x3e,0xf9,0xe9,0xe5,0xd3,0x57,0x67,0xc4,0x87, +0xa6,0x13,0x45,0x1d,0xb6,0xa7,0x9c,0xe1,0xf4,0xe4,0xf5,0x9b, +0xa7,0x09,0xe1,0x63,0x71,0xf3,0x09,0x5d,0x5e,0x40,0x13,0x57, +0x9f,0x1f,0xa2,0x59,0x21,0x57,0xcf,0x37,0xb6,0xce,0xf8,0xcf, +0xc3,0xc1,0x10,0x88,0xbd,0xa4,0xd5,0x4a,0x0a,0x7e,0x1b,0xf2, +0xec,0xaf,0xe5,0xfc,0x96,0x6a,0x46,0xdb,0x53,0xe6,0xcd,0x42, +0x8e,0x49,0x8a,0xed,0x51,0x0d,0xc4,0x37,0xe0,0x88,0x02,0xff, +0xda,0xb6,0xff,0xc0,0xcf,0x8d,0x4a,0x86,0xc1,0xee,0x95,0x31, +0x3a,0xd8,0x91,0xc1,0x71,0x9b,0x9e,0x66,0xa3,0x9c,0xa2,0xc9, +0xed,0x4b,0x34,0xcf,0x9c,0x24,0x1d,0x41,0x63,0xf9,0xc9,0x6c, +0x27,0x4d,0xae,0xa0,0xa7,0x1f,0xd6,0xeb,0xe4,0x40,0x7e,0xaf, +0xe0,0x18,0x18,0x4b,0x78,0x36,0xd7,0x80,0xfc,0xf6,0x91,0x0a, +0xfa,0xc0,0x18,0x55,0x4b,0x40,0x1c,0x1e,0x05,0x9c,0x01,0xbf, +0x81,0x71,0x3b,0x42,0xb7,0xce,0x86,0xd7,0xea,0x43,0x38,0xbb, +0x97,0x40,0xf9,0xfe,0xb7,0x17,0x40,0x80,0xb8,0xf8,0xee,0xe5, +0x14,0x8d,0x9c,0xb2,0xa5,0x14,0x89,0x64,0x67,0x76,0xc8,0x43, +0x72,0x06,0x52,0x36,0xd8,0x9f,0xb3,0xa7,0x7f,0x3f,0x7b,0xf7, +0xea,0xf5,0x93,0xa7,0xe5,0xb7,0xf4,0x72,0xf8,0x23,0x64,0x53, +0x78,0xb1,0xab,0x3b,0x07,0x9f,0x60,0x31,0xf8,0xd0,0xe1,0x71, +0xff,0x60,0xe0,0x58,0xe9,0xb0,0xb6,0x1c,0xf0,0x9e,0x1f,0x34, +0xad,0x84,0x9f,0xc8,0x31,0x24,0xe6,0xd0,0xf7,0x7f,0x49,0xbf, +0xef,0xff,0xf2,0x78,0xd0,0x79,0x9c,0xed,0x67,0xe8,0x2c,0xc1, +0xbd,0xa1,0xfc,0xf2,0x3d,0xba,0xa6,0x3c,0xdf,0x1b,0x74,0xb2, +0xfd,0xe8,0x9d,0x27,0x77,0x1e,0x8d,0x75,0xcd,0x7d,0x9f,0x74, +0x7c,0x65,0x2e,0x75,0x63,0xfa,0x2d,0x4d,0x6e,0x68,0x92,0xdd, +0x93,0xcb,0xcf,0x4f,0xc6,0x5b,0xbc,0xe5,0xde,0x9b,0x8e,0x8f, +0xb6,0x65,0x32,0xe3,0xec,0xa1,0xd2,0x42,0x04,0xec,0xf4,0x6f, +0xe4,0xcd,0xf7,0x57,0x09,0x38,0x66,0xf1,0x3b,0x87,0x1d,0x23, +0x31,0x87,0x37,0x72,0x44,0x92,0xba,0x22,0x6c,0x45,0x50,0x2c, +0x2d,0xb4,0xad,0xb3,0x56,0x0f,0x14,0x96,0xb2,0x7e,0xc0,0x06, +0x76,0xe0,0x4b,0x8d,0xd4,0xe2,0x25,0x14,0xc7,0xbb,0xac,0xb9, +0xcb,0xae,0x1c,0x99,0x58,0xf3,0x3c,0x26,0x3c,0x7a,0x32,0x7f, +0x42,0x05,0x09,0x52,0xe0,0x70,0x90,0xf5,0xbe,0x04,0x14,0xf2, +0x14,0x80,0xa9,0x29,0x7f,0xbd,0x67,0x1e,0x5d,0xf8,0xce,0xec, +0x69,0x88,0xc9,0x11,0x94,0x15,0x29,0x34,0xaa,0xec,0x3b,0xbe, +0x3c,0xc4,0x0f,0x7c,0x3c,0x9e,0x4f,0x40,0x6a,0x07,0x0c,0xfa, +0xd5,0xdc,0x0d,0xbc,0x63,0xd0,0xb3,0x29,0x94,0xaf,0x22,0xfd, +0x99,0x49,0x1d,0x30,0x8d,0x7a,0x9a,0x9d,0x73,0x5c,0xf9,0x07, +0x41,0xc8,0x23,0x93,0x40,0x35,0x70,0xd3,0xe5,0x39,0x6c,0x39, +0x61,0x99,0x11,0x9c,0xe5,0x9d,0x83,0xac,0x63,0xb2,0x8b,0xe8, +0x42,0x52,0x98,0xd7,0x33,0xcd,0xee,0x73,0x91,0xa8,0x16,0x0a, +0xb6,0x7e,0x05,0xec,0x0f,0x5f,0x02,0xfb,0xc1,0x42,0x64,0xd3, +0x30,0x7a,0x8f,0xf1,0x87,0x73,0x81,0x5c,0x24,0x89,0xc1,0xdc, +0x15,0xca,0xe9,0xf8,0xfa,0x56,0x16,0xbd,0xe7,0xb7,0xd6,0xaa, +0xc0,0x0a,0xfc,0x7e,0xfe,0xd7,0x47,0x7f,0xda,0xbf,0x9e,0xe4, +0x81,0x5f,0xe5,0xfd,0x7f,0xfd,0xf6,0x09,0x45,0x96,0x41,0xe4, +0xa3,0x1f,0x28,0xb2,0x63,0x79,0xdf,0x0d,0xe0,0x17,0x97,0xab, +0xa7,0x14,0x7b,0xba,0x80,0x72,0x55,0xdc,0x92,0x68,0x77,0xda, +0x0a,0xbe,0x3b,0xa0,0x0a,0xfe,0x67,0xd8,0x94,0x63,0x8a,0x2c, +0xc2,0xa6,0x7c,0xb7,0x7f,0x9d,0x03,0x0e,0x1a,0xc4,0x9d,0x50, +0xc6,0x3c,0x8c,0x44,0x27,0xfa,0x51,0x8b,0x7a,0x09,0x44,0x03, +0xc0,0xa0,0x23,0xf2,0x48,0xfb,0x7c,0x06,0xbb,0x50,0x0e,0xd0, +0xfc,0x62,0x3e,0x5f,0xc1,0x85,0x88,0xae,0x46,0x7c,0xbe,0xf0, +0xf8,0xd5,0x23,0x9d,0x25,0xc7,0x09,0xf3,0xe0,0x35,0x20,0xe9, +0x74,0xae,0x43,0x19,0x95,0x22,0xe1,0x27,0xde,0xdc,0x39,0xb0, +0xef,0x4b,0x68,0x40,0x6a,0xe6,0x68,0xc2,0x65,0x76,0x5d,0x40, +0x7e,0xe8,0xc6,0xec,0x7a,0x8f,0x03,0x9f,0xf6,0x5c,0x10,0x95, +0xc9,0xf4,0x6b,0x90,0xbf,0xfa,0xeb,0xbb,0xe3,0x37,0x6f,0xde, +0x9d,0xbc,0x38,0x3e,0x3d,0x7d,0xf7,0xf6,0xe9,0x5f,0x9f,0xfe, +0xfd,0x4d,0xb9,0x7f,0x5e,0xcd,0xae,0xfb,0x05,0x1c,0xd0,0x90, +0x27,0x2d,0xce,0xab,0x6f,0xe8,0xbc,0x1e,0xbd,0x83,0x03,0xfb, +0xa8,0x97,0xf5,0xce,0xab,0x7d,0xc7,0x14,0xa0,0x2a,0x43,0x91, +0xa5,0x8c,0x7c,0xb0,0xab,0x33,0x17,0xf4,0x2f,0xc9,0xfd,0x75, +0x12,0x35,0x40,0x3a,0x49,0x5f,0x7e,0xf8,0xfc,0x7c,0xc4,0x65, +0xb2,0xdc,0xbb,0x9c,0xd1,0xa1,0x87,0xf1,0xcd,0x93,0xf3,0x73, +0x98,0x36,0xed,0x6c,0x97,0x44,0x3c,0x4e,0xc5,0xfb,0xea,0xf1, +0x14,0x7d,0x45,0x69,0x4b,0xb6,0x65,0x49,0x93,0x2e,0xcb,0xc2, +0x64,0xd2,0x10,0x38,0x5c,0xbf,0xb6,0x48,0x47,0xaa,0xff,0xfa, +0x82,0x7d,0x2d,0x38,0xf0,0xc5,0x70,0x27,0x46,0x25,0x23,0x19, +0x25,0x5a,0x06,0x48,0x68,0xf8,0x69,0x16,0x5c,0x7c,0x3a,0xac, +0xc8,0x6f,0x4e,0x99,0xb4,0x60,0xaf,0x4b,0xb5,0x2e,0xb6,0x03, +0xb1,0x39,0x3b,0x1b,0x6a,0x98,0x48,0x36,0x06,0xe5,0x32,0x67, +0xec,0x96,0xa8,0x97,0xfa,0x4a,0xca,0x71,0xb0,0xa6,0xca,0x54, +0x3d,0x18,0xf1,0x7d,0xe4,0xc5,0x32,0xaa,0x0e,0x6e,0x16,0xd8, +0x17,0x80,0xf5,0x46,0x63,0x30,0x5c,0x01,0x7e,0x76,0x71,0xb7, +0xb2,0xcb,0x00,0xe3,0xb2,0x7b,0xd3,0x9b,0x76,0x9b,0xd7,0x04, +0x26,0x90,0x93,0x77,0x34,0x42,0xba,0xbd,0x1d,0x94,0x4d,0x3d, +0xc0,0xe3,0xc9,0x6f,0x21,0xb9,0x2d,0x95,0xd6,0x36,0x45,0xaf, +0xcf,0xbf,0x03,0x20,0x48,0x2c,0x8d,0xe3,0x4a,0x84,0xf5,0x54, +0x2a,0x10,0xf6,0x83,0x66,0x28,0x43,0x29,0xe7,0x2d,0x08,0x99, +0x5b,0x8e,0x93,0xd9,0x7b,0x9a,0xfa,0x54,0x6e,0xce,0xd5,0xf0, +0xba,0xf4,0xa8,0x13,0x72,0x07,0x65,0xcd,0xf7,0x12,0x0d,0x25, +0x45,0x13,0xea,0x77,0x14,0xb3,0x0e,0x2f,0xb0,0x3d,0x8b,0x11, +0x0c,0xf9,0xf1,0x62,0xd1,0x3a,0x9e,0x92,0x15,0xb5,0x96,0x6b, +0xe7,0x62,0x3c,0x6a,0xa1,0xd5,0x4d,0x12,0x9d,0x6d,0xc9,0x20, +0xb4,0xfe,0x70,0x7f,0xb0,0xf9,0x43,0x92,0x43,0x2b,0xb2,0x8d, +0xf4,0xb0,0x94,0x5f,0x32,0xa6,0x24,0x61,0xf7,0x20,0xd9,0x4f, +0x76,0x17,0xcb,0xf9,0x07,0xd4,0xe5,0xf4,0x73,0xa7,0x51,0xd9, +0xbd,0x86,0x78,0x26,0xd2,0x64,0x77,0x09,0xd5,0x4b,0x5d,0x49, +0xee,0x8e,0xa7,0x41,0x56,0x03,0x0c,0x07,0x8f,0xa8,0x44,0xe8, +0x08,0x95,0x6c,0xbd,0xec,0xb9,0x0e,0x98,0x4e,0x80,0xe7,0x18, +0x73,0x02,0x0c,0xe9,0x87,0xf9,0xaf,0x63,0x6c,0x1a,0xd6,0x46, +0x54,0x0b,0x0c,0x42,0x58,0x75,0xb2,0x8b,0xca,0x7f,0x13,0xb2, +0xb9,0xb1,0xab,0x25,0x31,0x2c,0x2a,0x4b,0xd6,0x80,0x34,0x02, +0xd0,0xb6,0xe6,0x52,0x2c,0xd7,0x32,0x6a,0x26,0x02,0xad,0x71, +0x43,0xbe,0xee,0x2e,0x13,0x7a,0x66,0x11,0xe8,0x4c,0x93,0x5a, +0xb5,0xad,0x4c,0x43,0x99,0x02,0x75,0x73,0xc9,0x75,0xe2,0xa6, +0x17,0xe8,0xdd,0x31,0xc9,0xc2,0x8d,0xc8,0xd8,0x02,0x22,0x5d, +0x52,0x74,0x83,0xa7,0xef,0x93,0xa7,0xcf,0x9e,0xbe,0x7d,0xf7, +0xc3,0xeb,0xd7,0x67,0xa7,0x67,0x6f,0x8f,0xe1,0xf0,0xfd,0xa5, +0x1e,0xb9,0xb3,0xaf,0xc3,0xc4,0xb2,0x6a,0xed,0xf6,0x4e,0x3d, +0x13,0x53,0x90,0x22,0x56,0x47,0xe7,0x5d,0xcf,0x2c,0x6c,0x14, +0x8c,0x37,0x69,0xa5,0x09,0xbb,0x6d,0x5e,0x87,0x49,0x46,0x6d, +0xe4,0x5a,0x43,0xab,0x8d,0xb0,0x82,0x1b,0xb6,0x0a,0xbd,0xad, +0xbe,0xd4,0x1d,0xe5,0xce,0x07,0x13,0x6b,0xb0,0x7c,0x11,0x9a, +0xd3,0x15,0x43,0x37,0x9d,0x44,0x6e,0xd0,0x72,0x86,0x69,0xf1, +0xc6,0x09,0xd9,0x7a,0xb2,0x69,0x36,0xfc,0x75,0xfc,0x8e,0x69, +0x01,0x32,0xa7,0x3a,0x46,0xa7,0x41,0x38,0x0d,0xde,0xcd,0xad, +0xc4,0x94,0x2e,0x04,0x87,0xd9,0xbb,0x24,0x0f,0xba,0x7a,0xfa, +0xea,0xf8,0x3f,0x9e,0xbe,0x3b,0x39,0x3e,0x7d,0x2a,0x67,0xa5, +0x6f,0xe1,0x14,0x08,0x54,0x24,0x54,0xe7,0x0e,0xed,0x48,0x21, +0xdc,0x73,0xd0,0x48,0xae,0x8a,0x33,0x85,0x0e,0x9b,0x36,0x11, +0x57,0xe5,0xdf,0x19,0x9f,0x51,0xe1,0xda,0x32,0x90,0x7f,0x14, +0x89,0xe0,0x5e,0xca,0x47,0x8b,0x4a,0x41,0x8e,0x03,0xdf,0xba, +0x57,0xb3,0x9c,0x17,0x66,0xf1,0xef,0xff,0x89,0xb9,0xde,0x38, +0xfd,0x4f,0x5e,0xd6,0xf8,0x2e,0x85,0xa2,0xa2,0xe3,0x65,0x2d, +0x83,0x4f,0x72,0xcb,0xad,0x96,0xc7,0xed,0x02,0x79,0x6d,0x1f, +0x8f,0xc8,0x23,0x74,0x3d,0x9b,0x49,0x85,0x39,0x92,0x74,0xbc, +0x29,0xb8,0x8b,0xf2,0x4e,0x94,0xb0,0x41,0x7d,0x14,0xfa,0xa7, +0xff,0x0f,0x1f,0xc8,0x4a,0xb6,0x44,0x20,0xe7,0x21,0xff,0xff, +0x40,0x4e,0xf6,0x10,0x2d,0x19,0x01,0xad,0x2d,0x64,0xc0,0xb8, +0x80,0x5b,0x9c,0xe1,0x49,0x6d,0x70,0xb2,0x0a,0x8d,0xf3,0x1f, +0x2f,0x49,0x02,0x2f,0x17,0x9f,0x73,0xc3,0x4a,0x5e,0x71,0x76, +0x50,0x0c,0x2f,0x3e,0x7a,0x87,0xcb,0xf1,0x6f,0x78,0xee,0x0a, +0xa3,0x87,0x4f,0x55,0x7c,0x17,0xba,0x3f,0xdc,0xf0,0x32,0x82, +0x05,0xd5,0x4b,0x04,0x0e,0x84,0xd1,0x8f,0xc9,0x64,0x89,0xc6, +0xc2,0x8f,0x3c,0x0f,0xa9,0xa1,0x09,0xcf,0x66,0xbe,0x11,0xec, +0xd6,0x83,0xa8,0x8f,0xe3,0xd9,0x6c,0xbe,0x92,0x97,0x25,0x2d, +0xdf,0x94,0x8a,0x8c,0x65,0x26,0x57,0xb0,0xd5,0x78,0x7f,0x2e, +0xaf,0x91,0x45,0xd7,0x47,0x59,0x45,0x15,0x46,0x83,0xe3,0xc5, +0x77,0x39,0x12,0x6a,0xe4,0x9a,0xd1,0xf3,0x6c,0x6b,0xe8,0x64, +0x17,0xf2,0xd6,0x35,0x7c,0x27,0x48,0xb6,0x5c,0x37,0x48,0x05, +0x40,0x2c,0xc9,0x42,0x1a,0x43,0x76,0x5d,0x19,0x02,0x7e,0xc5, +0x48,0x0a,0x9f,0x13,0xb0,0xb8,0xa0,0xe7,0xd7,0xb4,0x4b,0x98, +0x93,0x34,0x5c,0xdd,0xe4,0xb8,0x2b,0x9e,0xcd,0xce,0xe6,0x74, +0xaa,0xf3,0x04,0x60,0x7c,0xe6,0x1f,0xa3,0xed,0x73,0x05,0x49, +0x25,0x95,0x98,0x41,0x29,0xdb,0x2e,0x9c,0x46,0xe8,0xb5,0xe9, +0xb9,0x30,0x14,0x51,0xd4,0x0f,0x2d,0x53,0x94,0x46,0xe4,0x89, +0x84,0xa0,0x20,0xf8,0x78,0x42,0xef,0x5f,0x48,0x08,0x8b,0x74, +0x53,0x4e,0x8f,0xdd,0xf4,0x08,0x9a,0xc6,0x50,0xb2,0xe0,0x55, +0x63,0x27,0x68,0x28,0x8e,0x7b,0x28,0x37,0x4b,0x3c,0x53,0x0b, +0x82,0xde,0x6a,0x8b,0x50,0x7e,0x0c,0x7d,0x2f,0xf0,0xf1,0xf7, +0x62,0x4e,0x06,0x8d,0x45,0xf0,0xf8,0xde,0x10,0x3e,0xd5,0xdd, +0x92,0xa8,0x57,0x9e,0x99,0xab,0x21,0x0e,0xef,0x67,0xfb,0x38, +0xdf,0x17,0x67,0x89,0xa9,0x0b,0x97,0x92,0x0b,0x50,0x8e,0x8d, +0xa3,0x8a,0x08,0x8e,0xfc,0x88,0x38,0x73,0x22,0x3b,0x23,0xc9, +0x79,0x9e,0xb2,0x3c,0xe1,0x63,0x36,0xc9,0x63,0x49,0x65,0xc1, +0x14,0x0c,0x7b,0x22,0xd0,0x40,0x90,0xe5,0x8d,0x56,0xb1,0x67, +0x57,0x93,0x6b,0x23,0xf7,0xa5,0x29,0xed,0xb6,0x1e,0xe7,0x11, +0xa3,0x9f,0xae,0x21,0x76,0x00,0x8d,0xc9,0xd2,0x03,0x36,0xd0, +0x2c,0xcd,0x95,0x24,0x19,0x01,0xa3,0xff,0xe3,0x1f,0xc5,0x11, +0x21,0x78,0x31,0xc4,0x95,0x24,0x08,0xc9,0x52,0x95,0x2e,0xd8, +0xf5,0xc6,0x4b,0xfa,0x78,0x40,0x2a,0x87,0x41,0xfc,0xa7,0xd8, +0xda,0xf0,0x45,0x60,0x11,0xe3,0xfd,0x93,0x0c,0xd2,0x7e,0x0c, +0xd9,0xb1,0x79,0x1d,0x76,0xa3,0x73,0xbd,0xd9,0xb0,0x97,0x51, +0xee,0xba,0x9c,0x1e,0x22,0x12,0x6e,0x10,0x84,0x0c,0x90,0xa0, +0x39,0x94,0x84,0x93,0x84,0x57,0x81,0x3f,0x47,0x68,0x03,0x7e, +0x18,0x4e,0xa6,0x88,0x15,0xec,0xb4,0xfe,0x6b,0x7e,0xd7,0x1a, +0x03,0x2e,0x37,0x86,0xa9,0x98,0x54,0xd5,0x62,0x8c,0x8e,0x07, +0x00,0xb1,0x1b,0xcb,0xcc,0xb4,0x70,0x68,0xf0,0xa9,0x1a,0xf6, +0x06,0xee,0xd5,0xd5,0xbc,0x85,0x46,0x78,0x5b,0x93,0x55,0xb7, +0xf5,0xfc,0x0a,0x26,0xe1,0x7a,0x52,0xc1,0xe0,0xe0,0x03,0xf7, +0x50,0x8b,0xf0,0xe0,0x02,0x90,0xe1,0xaa,0xf5,0x19,0xe0,0xa3, +0xb7,0xa9,0xc9,0xd5,0x67,0x82,0x3a,0x22,0x53,0x98,0x40,0x4a, +0xe3,0x5b,0xf8,0xb0,0xa2,0xb8,0x6a,0x0c,0x93,0x3b,0x72,0xbc, +0xef,0x2e,0x9f,0x73,0x8a,0xc2,0xb9,0xd1,0x23,0xd7,0x23,0x77, +0xb3,0x1f,0xa6,0x73,0xf1,0x43,0xc2,0x4b,0xa2,0xb4,0x73,0x14, +0x60,0x64,0x9c,0x90,0xc4,0x63,0x58,0xde,0xbf,0x33,0x50,0x0b, +0x13,0xce,0xdf,0x39,0xf8,0x85,0x0b,0xb9,0x25,0x58,0xb8,0xb5, +0x88,0xcd,0x2b,0xd4,0xac,0x08,0xcd,0x5c,0x11,0x36,0xc2,0x21, +0xb6,0x89,0x66,0x20,0x03,0x27,0xb4,0x77,0xb6,0x66,0x95,0x74, +0x34,0x93,0x32,0x5e,0x7e,0x98,0x5c,0x8e,0xb7,0xe6,0x94,0xf4, +0x44,0x64,0x19,0xb6,0xe6,0xa3,0x54,0xd4,0xcb,0x9d,0x53,0xd7, +0x57,0x5b,0x33,0x6a,0x06,0x08,0x0a,0x16,0x9d,0x64,0xde,0xaa, +0x70,0x54,0x4c,0x10,0xcb,0x37,0xda,0xb3,0x3c,0xd1,0x55,0x80, +0x9d,0x64,0x6b,0xa8,0x61,0x09,0x8e,0x6c,0x2e,0x60,0xd0,0x88, +0xb0,0x90,0x4f,0x68,0x2e,0x38,0x82,0xa9,0xb8,0x44,0xfd,0xe7, +0x5a,0x39,0xc2,0x88,0x4d,0x21,0x97,0x93,0xab,0x83,0x35,0x53, +0xf0,0x4f,0x68,0xb1,0xf3,0x02,0xa7,0xdb,0x1f,0x2f,0x3a,0xff, +0x8c,0x25,0x72,0xa2,0x7a,0x2a,0x95,0x3c,0x7a,0x26,0xa1,0x79, +0x6d,0x0c,0xa5,0xee,0x90,0x8a,0xb7,0x2d,0x69,0xa1,0x78,0xce, +0xed,0xdd,0xc5,0x74,0x52,0xdd,0x3c,0x05,0x7c,0x6b,0x39,0x1b, +0x4e,0x8f,0xdf,0x3c,0x4f,0xe5,0xc0,0x04,0x9c,0x9f,0x91,0x30, +0xf9,0xce,0xef,0x1d,0xf9,0x58,0xb8,0x50,0x8e,0xaf,0xf4,0x05, +0xfe,0x11,0x94,0xad,0xe0,0x9f,0x9c,0x5f,0x0a,0x0b,0xfe,0x51, +0xba,0x43,0x70,0x16,0x65,0x02,0x28,0x09,0xed,0x31,0xb3,0x90, +0x38,0xca,0x51,0x38,0xb1,0x20,0x3b,0xf0,0x78,0xd3,0x14,0xf8, +0x47,0xde,0x51,0x44,0x7d,0x20,0xd7,0xb7,0x92,0x42,0x03,0xb9, +0xca,0x2f,0x16,0x1a,0xc8,0x8d,0xec,0xa1,0x55,0x52,0xf0,0x3c, +0xdf,0xc2,0x85,0x72,0x7d,0x65,0x71,0x0a,0x0b,0x46,0x21,0xa2, +0xf0,0xc1,0x5c,0x65,0x17,0x9c,0x10,0x43,0xae,0x12,0x7f,0x85, +0x06,0x72,0x27,0xe9,0x55,0xb8,0x50,0x2e,0x98,0x8b,0xca,0xe0, +0xe7,0xbb,0xbb,0x7c,0x62,0x16,0xfc,0x93,0x8b,0x3e,0x43,0x21, +0xbf,0x39,0x3f,0xbd,0x8b,0x5c,0x46,0xee,0x58,0xfb,0x85,0x0b, +0xe5,0x77,0xe8,0x55,0x93,0xe2,0x5c,0xc8,0xd9,0x00,0xaa,0xd0, +0x36,0xcf,0x1d,0x0a,0xc0,0x15,0x07,0xc4,0x3d,0xe0,0x99,0xe4, +0x73,0xb8,0xdc,0x7a,0x33,0xd3,0x03,0x48,0x90,0x17,0x09,0x59, +0x32,0x2e,0x3f,0xb6,0x0f,0x2d,0xdb,0xb2,0xa0,0x28,0x49,0x57, +0x4f,0x1b,0xd8,0x06,0x53,0x89,0xde,0xe5,0x74,0xdd,0x0e,0xd9, +0xa6,0x06,0x00,0x8a,0x7a,0x30,0x83,0xfc,0x6b,0xe9,0x72,0x53, +0x97,0x23,0x88,0x77,0x4f,0xc2,0xcd,0x97,0x75,0xdd,0xde,0x4b, +0xef,0x87,0x05,0xa2,0xd3,0xc7,0xb3,0xcb,0x9b,0xf9,0xf2,0x89, +0x46,0xc3,0x32,0x5c,0xdc,0xe1,0xc1,0x04,0x7f,0x7d,0x24,0x0a, +0xbc,0x01,0x26,0x3c,0x8c,0xe3,0xd1,0x9d,0x11,0x2e,0xe0,0x5b, +0x1f,0x25,0xbe,0xe2,0xf8,0xc7,0x44,0x13,0x5f,0xac,0xe0,0x1f, +0x13,0x8d,0x9a,0x65,0x05,0xfd,0xf5,0x91,0xac,0xab,0x5e,0xf0, +0x8f,0x8f,0x9e,0x5d,0xff,0x80,0x1b,0x80,0x7f,0xe2,0x68,0x7c, +0xe2,0x29,0x7c,0x30,0x4e,0x3e,0x13,0x3d,0x9f,0x22,0xfc,0xb4, +0xd9,0x4e,0x90,0x35,0x56,0xc8,0x6f,0x2d,0x01,0x5d,0x28,0x16, +0x26,0x5c,0xcb,0xf0,0x7a,0x34,0x2a,0x7c,0x30,0x48,0xae,0x16, +0x05,0xfd,0x0d,0xcb,0xcc,0x87,0xbf,0x16,0xf2,0x1b,0x24,0xf8, +0xf3,0xd7,0x7e,0xd8,0x2c,0xcf,0x70,0xd4,0xf9,0xc7,0x46,0xff, +0x08,0x33,0x5c,0xf0,0x8f,0x8d,0x7e,0x7e,0x55,0xe0,0x9f,0x20, +0x8a,0x85,0x12,0x0b,0x17,0x0a,0x13,0x27,0xab,0x82,0x7f,0x6c, +0xf4,0xab,0xf9,0x0c,0x47,0x0e,0xb1,0x94,0x22,0xf8,0xb2,0x99, +0xde,0x4c,0xef,0x96,0xc3,0xe9,0xe4,0x1f,0x98,0xc5,0x85,0x6d, +0x86,0xb7,0x80,0x6e,0x0c,0x11,0x3c,0x07,0x6c,0xd2,0xe9,0xcd, +0xfc,0x63,0xc1,0x3f,0x41,0x34,0xad,0x10,0xf9,0x0d,0x12,0x3e, +0x4e,0x60,0x0b,0x16,0x1a,0xa8,0x27,0xfd,0x7c,0x43,0x53,0xe6, +0x3f,0xea,0x59,0x9e,0x88,0xad,0x9c,0xe8,0xdb,0x66,0x7c,0xcd, +0x86,0x13,0x0a,0x17,0xb2,0x89,0xe4,0x82,0x46,0xc7,0xd2,0x7f, +0xd8,0x2c,0xb0,0xaf,0xc7,0xb8,0x32,0xe9,0xd7,0x26,0xbc,0x80, +0x8b,0xb3,0xe0,0x9f,0x60,0xfe,0xc9,0x2f,0x42,0xa1,0x01,0x9f, +0xa4,0x84,0xa5,0xa2,0x3d,0xa3,0x70,0x5c,0x1f,0x48,0xa4,0x87, +0xa5,0x42,0x7e,0x5d,0xc2,0xc6,0x1e,0x06,0xb3,0xeb,0x63,0xe5, +0xdc,0x1e,0x4f,0x27,0x43,0xdf,0xc9,0x2a,0xcc,0x45,0x9e,0x44, +0x4d,0x62,0x5e,0x3f,0x7e,0xee,0x01,0x21,0xc1,0x23,0xe5,0x94, +0x14,0x57,0x8b,0xdd,0x63,0xf3,0xa5,0xe7,0x50,0xae,0x48,0x0b, +0x26,0x07,0xd8,0x4b,0xbe,0x7b,0x01,0x28,0x34,0x20,0x54,0xc5, +0xee,0x0f,0x1c,0xf0,0x29,0xe4,0xd5,0xf1,0x99,0x60,0x6d,0xbb, +0x27,0xe6,0xcb,0xe4,0xf1,0xdb,0x67,0xf7,0xa4,0x86,0xb1,0xe4, +0xbb,0xca,0x79,0x2d,0x76,0xd5,0x1e,0x8f,0x4f,0x1b,0xab,0xa7, +0x8e,0x1f,0xc9,0x7b,0x05,0x40,0x78,0x1a,0xc5,0xf8,0xbc,0x82, +0x57,0xed,0x3e,0x0b,0x50,0xa9,0x7c,0x97,0xac,0x3d,0x2d,0xe6, +0x74,0xd6,0xec,0x3e,0xf7,0x1f,0x3e,0x07,0xe9,0xf2,0xed,0xa2, +0x23,0x9c,0x30,0xee,0x07,0xb8,0xab,0x10,0x77,0xa0,0x24,0xf9, +0xf0,0x39,0x54,0xba,0xbb,0xd8,0x55,0xa7,0xe8,0x36,0xed,0x1a, +0xa3,0xaf,0x7d,0x0c,0x3d,0x45,0x17,0xbb,0x6f,0xf0,0xc7,0xc7, +0x3a,0x1e,0x6a,0xb1,0xfb,0x56,0x83,0x3e,0xf5,0xb7,0x62,0xf7, +0x3f,0xfd,0x57,0x05,0xe8,0xee,0xee,0xe9,0xe5,0x38,0x88,0x79, +0x02,0xa7,0xf7,0x35,0xf5,0xec,0xd4,0x7f,0x98,0x1c,0xb3,0xc9, +0xd5,0x15,0x0e,0xca,0x29,0x07,0x7c,0x8a,0xaa,0x59,0xd2,0x9c, +0x15,0xbb,0x67,0xf6,0xd3,0xe4,0x62,0xf7,0x25,0x90,0xce,0x01, +0x9f,0xc2,0x77,0x72,0xb1,0xcb,0xd2,0x2f,0x3e,0x7e,0xf7,0x6e, +0x39,0xfd,0x69,0x35,0x01,0x1c,0x6b,0x77,0xf7,0x27,0x09,0x6a, +0xea,0x06,0xd9,0xab,0x1e,0xc7,0x7b,0xff,0xdb,0x2b,0xb8,0xc4, +0x9e,0x8f,0x1c,0x21,0xd8,0xe9,0xbc,0xff,0xed,0xf9,0xc8,0x48, +0x66,0x01,0xfd,0x30,0x3d,0x51,0x66,0xa2,0x97,0x49,0x0f,0x38, +0x85,0x6f,0x9e,0x9e,0x3c,0x3f,0x7e,0xf1,0xee,0xe4,0xc7,0xe3, +0xb7,0xa7,0x35,0x66,0xe1,0x3b,0xcf,0x81,0xcc,0x85,0x71,0xc8, +0x1a,0xd9,0x9e,0x88,0xa7,0xcf,0x9e,0x63,0x18,0x1a,0x6f,0xe0, +0x05,0x47,0x6e,0xfc,0x43,0xcb,0xcb,0xd7,0xff,0xd7,0xbb,0x1f, +0x8f,0x4f,0xfe,0x43,0xeb,0x01,0xd2,0xf1,0x1f,0xbb,0x87,0xf6, +0x8d,0x71,0x1b,0x9f,0x8c,0x48,0xa1,0xd1,0xa4,0x5a,0x60,0xca, +0x19,0x29,0xc1,0xd3,0x32,0x45,0x04,0xac,0xca,0x99,0x11,0xf3, +0xfc,0xea,0xd5,0xfc,0xd8,0x89,0x2f,0xdd,0x1b,0xfd,0x2e,0x04, +0x41,0x30,0x53,0xf2,0xc1,0x2e,0x06,0x75,0xc7,0xab,0x9c,0x14, +0x09,0x50,0x68,0x1b,0x02,0x2f,0x98,0xbf,0xe2,0x78,0xe9,0xe8, +0xd8,0x92,0x13,0x29,0x28,0xc9,0x14,0x5e,0x8e,0x67,0x24,0x20, +0x5f,0x9a,0x46,0x90,0x6f,0xfa,0xe1,0x6d,0xaf,0x6f,0x6d,0x3f, +0x73,0x75,0x83,0x82,0x22,0x07,0xe4,0x45,0x87,0xce,0x9b,0xd2, +0x76,0x05,0xe5,0xf7,0x76,0x1a,0xba,0xa0,0x16,0x5f,0x19,0x08, +0x72,0x8e,0x8e,0xb0,0x52,0x15,0x7f,0xa6,0x18,0xb4,0xd1,0x42, +0x91,0xfc,0x30,0x91,0xb9,0x1e,0x95,0x07,0xbe,0x4f,0x25,0x1b, +0x33,0xa1,0x52,0x2e,0xf2,0xb1,0xe6,0x3c,0xd2,0x40,0xa7,0x43, +0x20,0xa3,0x47,0x21,0x48,0xed,0x6b,0x8e,0x41,0xe6,0xbb,0xd0, +0xd3,0xa7,0x03,0xf1,0xbe,0x2c,0xe7,0x09,0xe0,0x6a,0x28,0xc8, +0xbb,0x9c,0x03,0x95,0x59,0xf8,0xfe,0xee,0xb8,0xa0,0x19,0x58, +0x68,0xa3,0x19,0xda,0x32,0xd5,0xb1,0x2d,0xdd,0x43,0xa0,0x44, +0xa4,0x99,0x1a,0x54,0x3b,0x32,0x05,0x1e,0x7b,0x48,0x47,0x3e, +0x08,0xbd,0xa0,0x01,0x21,0x9a,0x4a,0xec,0x5d,0x28,0x9c,0xbe, +0xcf,0x66,0x94,0x78,0xd5,0x39,0xe0,0xbf,0xff,0xb6,0xcd,0x90, +0x38,0x3d,0x51,0xdb,0x6c,0x35,0x0b,0xe7,0x41,0x62,0x00,0x70, +0x57,0xbf,0xd6,0x6b,0x1b,0x9f,0x9b,0x55,0xe9,0xf3,0x04,0x45, +0xf3,0xd8,0x94,0xa8,0x29,0x12,0xef,0x98,0xe0,0x99,0x55,0xc2, +0x56,0x30,0x91,0x73,0x65,0xa1,0xf8,0x13,0x2d,0x3c,0xea,0x6b, +0x43,0x56,0x11,0x1a,0x16,0xe9,0x34,0x05,0xdf,0x6e,0x27,0xdf, +0x27,0x3b,0x66,0x82,0x44,0xb2,0x55,0xf8,0x44,0xbc,0x66,0x94, +0xd3,0x3c,0x9b,0x03,0x72,0x0c,0xd4,0xf1,0x8b,0xf9,0x9c,0xf4, +0x13,0xee,0x16,0x5a,0x73,0xd5,0xfa,0x30,0x19,0xb6,0x2a,0x79, +0x59,0xae,0x6b,0x25,0xb4,0x2e,0x3e,0x0b,0xac,0x9d,0xd6,0xe9, +0x78,0x5c,0xb4,0x44,0x5d,0x1c,0xee,0xba,0x58,0x59,0x7c,0xb8, +0x98,0xec,0x47,0x5c,0xf2,0x24,0xd0,0xcf,0x8e,0x06,0x68,0xd3, +0xd4,0x2d,0x79,0x28,0x9d,0x7c,0xd8,0xea,0x82,0x18,0xd2,0x00, +0x2c,0xfc,0xed,0x92,0x55,0x92,0x1f,0xcf,0x5e,0xbe,0x28,0x59, +0xe0,0xac,0xfd,0x2f,0x87,0x7f,0x3a,0x38,0xfa,0x7e,0x9f,0x84, +0xcf,0xf4,0x0d,0x3b,0xc7,0x9c,0xd6,0x03,0x2f,0x7e,0x1b,0x87, +0xb9,0xf2,0x20,0x70,0x3c,0x1a,0xa1,0x59,0x7e,0xb1,0xeb,0x81, +0x79,0x68,0x85,0x52,0x9c,0xf8,0x7e,0x5a,0x0e,0xaf,0xed,0x76, +0x8c,0xda,0xa7,0x57,0xfe,0x33,0xc9,0x96,0xa2,0xe1,0x5e,0x09, +0xab,0x1c,0x1c,0xfb,0x0e,0x21,0xa9,0xbd,0xa6,0x5a,0xdd,0xc0, +0x44,0x4b,0xea,0x84,0xa4,0xf2,0xdc,0xba,0x0a,0x97,0x0e,0xcb, +0xec,0x21,0x14,0x7a,0xf5,0x8b,0x8a,0x3e,0x19,0x0f,0xe1,0x76, +0xf7,0x65,0x39,0x96,0x8f,0x72,0x7c,0x65,0xf1,0xef,0xc2,0x46, +0xbf,0x23,0x6f,0xde,0xfe,0xd4,0x52,0x7c,0xde,0x3d,0x9a,0x7c, +0xaf,0x39,0xac,0x06,0x48,0x50,0xa3,0xdb,0xe8,0x93,0x41,0xad, +0x51,0xaf,0xaf,0xdc,0xce,0x60,0x03,0x1f,0x57,0xb3,0xfc,0x6e, +0xe6,0x96,0x9c,0x0a,0xc9,0x8b,0x58,0x98,0x4d,0x69,0x5c,0xdc, +0x70,0xf1,0xa1,0xe0,0x31,0x2c,0x6f,0x8e,0xff,0x17,0x88,0x48, +0xb3,0xd6,0x68,0x3e,0x0e,0xd6,0x32,0xb1,0x1f,0xff,0x97,0xae, +0xf4,0xff,0xe5,0x58,0x90,0xf2,0x7e,0x3c,0xfe,0x40,0x32,0x2e, +0xdc,0xc2,0xa7,0xec,0x76,0xf8,0x74,0x85,0x9e,0x96,0xb5,0xa9, +0x09,0x67,0x49,0x32,0x71,0xb0,0xf7,0x60,0x5e,0xce,0x02,0xb0, +0x9d,0x33,0x3e,0xab,0xa1,0x49,0xb6,0x76,0x9c,0x14,0x18,0xc3, +0x35,0x82,0x1a,0xf8,0x2d,0xa7,0x78,0xae,0xc6,0xbb,0x71,0xc6, +0xe8,0xd8,0x46,0xa4,0x1e,0x3d,0x15,0x3d,0x9b,0x85,0x83,0x68, +0x4b,0xa1,0x61,0x71,0xd2,0x91,0x61,0xd0,0x64,0x7c,0x7b,0xb0, +0xf1,0x22,0x15,0xfc,0xb4,0xe7,0xdc,0x2d,0xf9,0xaa,0xb9,0x36, +0xdb,0xd4,0xab,0x59,0x86,0xae,0x49,0xbe,0xae,0x7a,0xa9,0xa9, +0xb1,0xfa,0xac,0xb0,0xba,0x5e,0x36,0x85,0x44,0x06,0xa0,0x9e, +0x4d,0x56,0x5b,0x2a,0xf5,0x95,0x6a,0x5c,0x5c,0x88,0x73,0xe8, +0xe7,0x23,0x27,0x0a,0x01,0x88,0x18,0x9b,0xb2,0x31,0x73,0x02, +0xfb,0x95,0x70,0xc1,0xbe,0xcb,0x3e,0xc0,0xe3,0xd6,0x66,0x61, +0x83,0x10,0x08,0xd7,0x09,0x67,0x52,0xf3,0xeb,0x25,0xe9,0x79, +0x9e,0x2f,0x01,0x79,0x33,0x3e,0xb2,0x80,0xba,0x6e,0xb6,0x83, +0x58,0xee,0x6c,0x57,0xef,0xe3,0x76,0xbb,0xa1,0x0c,0xda,0x7b, +0x36,0x37,0x76,0x5e,0xdb,0x2a,0x99,0x1b,0xd5,0x7a,0xb3,0xf2, +0x68,0x00,0x8c,0x89,0xdb,0x78,0x44,0x1b,0x97,0x6b,0x24,0x39, +0xfe,0xcf,0x0e,0xec,0x7a,0xbd,0x77,0xe8,0x44,0x22,0x63,0xad, +0xe2,0x5e,0x30,0x26,0xeb,0x75,0x1a,0xb7,0xd9,0xd7,0xea,0xf1, +0xe9,0x2f,0xcd,0x64,0x79,0xbf,0x09,0xf3,0x18,0x45,0x71,0x7d, +0xd4,0x2f,0x6c,0x7a,0x38,0xf6,0xac,0x37,0x16,0x9f,0x99,0x76, +0xb5,0x45,0x23,0x43,0x16,0xf0,0x1f,0xda,0xf7,0xec,0x5b,0x1b, +0xd9,0x9c,0x84,0x46,0x96,0xf1,0x28,0x20,0x40,0x89,0x29,0x77, +0x34,0x17,0x3e,0xfc,0x59,0xf1,0x58,0x2c,0x3e,0x41,0xa3,0xd3, +0x7f,0x65,0x20,0xbe,0x0c,0xa9,0xa8,0x88,0x56,0x17,0x99,0x79, +0x40,0x8d,0x41,0x72,0x46,0x1a,0x16,0x59,0xaf,0xbf,0xd8,0x48, +0xb6,0xe6,0x8f,0xe3,0xa7,0xcd,0x60,0xdf,0x0a,0x7e,0x00,0xd9, +0x8b,0xa7,0x68,0xd7,0x48,0x03,0xdc,0x0e,0x81,0xac,0xa2,0x4e, +0x64,0xaa,0xb5,0xa9,0xe2,0x9d,0x01,0xc1,0x1d,0xa9,0xcd,0x00, +0x88,0x20,0x79,0xd8,0xda,0xaa,0xfc,0x71,0x58,0x11,0x7b,0xcb, +0xb5,0x51,0xcf,0x69,0x27,0x1a,0xb1,0x55,0x2c,0xcd,0x48,0x8f, +0xf5,0xcf,0x67,0xe7,0xab,0x01,0x0a,0x90,0xb5,0xac,0xab,0x74, +0x28,0xa9,0xe0,0xa8,0xc0,0x63,0xab,0x80,0x6a,0xcf,0x99,0xb0, +0x09,0x97,0x15,0xb7,0x09,0x65,0x4d,0x7c,0xd8,0x9b,0x2b,0xf0, +0x71,0x8d,0x47,0xa9,0x26,0x7b,0xd1,0x1e,0x2f,0x63,0x07,0x38, +0xcf,0x6d,0xfa,0x4f,0x74,0xc9,0x09,0x2c,0x42,0x49,0x82,0xe1, +0x2a,0x21,0x01,0x3d,0xcc,0x13,0x48,0x87,0x38,0x2c,0xe3,0xff, +0xa1,0x9e,0x6d,0x99,0x3c,0x97,0xc1,0xef,0xf2,0xb8,0xf3,0x8d, +0xfd,0x8e,0xfa,0xb4,0xa5,0x2f,0x8c,0x31,0x21,0xc3,0xc1,0x89, +0xa6,0x5a,0x64,0xdb,0x0f,0x78,0x55,0x3a,0x39,0x57,0xaf,0xf1, +0xbd,0xe3,0xb7,0x9a,0x4b,0x15,0x3d,0x0d,0xa3,0xf4,0xe6,0x60, +0xf5,0x54,0xe8,0xb5,0x62,0xdb,0x7f,0x18,0x8a,0x74,0x61,0x23, +0x30,0x84,0x0b,0x61,0xeb,0x02,0xe1,0x5a,0x76,0x2e,0x5c,0xc3, +0xec,0x1c,0x17,0x29,0xba,0xde,0x64,0x1f,0x71,0xae,0xe7,0x56, +0x2e,0xc6,0xef,0xe2,0xdd,0xa4,0x93,0x8a,0x84,0x84,0xe5,0xe6, +0xa2,0x35,0x25,0xfb,0x15,0xd7,0xd9,0x0c,0x8d,0xc8,0x7a,0x39, +0xe8,0x1a,0x68,0x4f,0x27,0x90,0xf8,0x97,0xb2,0x6c,0x50,0xd9, +0x10,0xb3,0xc5,0x46,0xc0,0x92,0x74,0xe9,0x45,0xa8,0x26,0x43, +0x4b,0x3a,0x1c,0xab,0x14,0x33,0xce,0x97,0xc8,0x98,0x07,0x82, +0x6f,0x2c,0xa2,0x6b,0x95,0xf2,0x32,0xab,0xd6,0x72,0x14,0xd7, +0xb2,0x10,0x53,0xab,0x9b,0x40,0x16,0x44,0x54,0x73,0x90,0xed, +0x48,0x26,0x38,0xeb,0xa8,0xc3,0x85,0xcf,0x51,0xfe,0xf0,0xfa, +0xf5,0x8b,0xa7,0xc7,0xaf,0xde,0x1d,0x9f,0x9d,0xbd,0xa5,0xab, +0x3d,0x94,0xb6,0x72,0x17,0x9a,0x29,0xd3,0x6e,0x6b,0xa1,0xa7, +0x2f,0x9e,0xa2,0xe2,0xd3,0xa9,0x2e,0x13,0xb7,0xd2,0x06,0x24, +0x3f,0xaa,0x05,0x0c,0x2b,0x88,0xa9,0x19,0x83,0xac,0xb9,0xe6, +0x31,0x9a,0x20,0x57,0xb0,0xc9,0x50,0x86,0x98,0xa1,0xa0,0x84, +0xb8,0xe2,0x3f,0xd0,0x18,0x2c,0xe9,0x57,0xf8,0xcb,0xb8,0xe5, +0x1a,0xa2,0xad,0xbc,0x29,0xa7,0x73,0xaf,0xfe,0x26,0xda,0x43, +0x78,0x8d,0x52,0x34,0x1a,0x8f,0x47,0xb1,0x8a,0xe1,0x35,0x71, +0x02,0x1d,0xb8,0x28,0xbe,0x0e,0xef,0x12,0xa9,0xd6,0xe9,0x0f, +0x77,0x17,0x68,0xad,0x71,0xe7,0xc0,0x01,0x54,0x17,0xb0,0xa9, +0xfd,0x2c,0x05,0xe8,0xf2,0x52,0xe8,0x3a,0x6f,0xa5,0x34,0xb3, +0xef,0x89,0x52,0xc8,0xfb,0xb7,0xc5,0x4f,0x24,0x04,0x58,0x0b, +0x83,0xbf,0xcb,0xa6,0x0e,0x1f,0x7d,0xdd,0x28,0xc4,0x90,0x51, +0x8c,0x5c,0xca,0x88,0x25,0xe3,0x0f,0x44,0x92,0xe5,0xdb,0xb2, +0x1f,0x6e,0x38,0x85,0x8e,0x95,0x30,0xad,0x2e,0x1f,0xd2,0x0c, +0x64,0xbd,0x3e,0x28,0xcb,0xda,0xa4,0x6c,0xf2,0x90,0x3e,0xe8, +0xb3,0xa3,0x1e,0x19,0x45,0xc4,0x9b,0x43,0xfb,0xf7,0xaa,0x24, +0x18,0xac,0x26,0x14,0x75,0xfc,0xf3,0xe3,0xf2,0xb6,0x9a,0x8c, +0x7b,0xcd,0xeb,0x02,0x39,0x5b,0xcd,0x53,0x6f,0x93,0x1a,0x7a, +0x47,0xc2,0x35,0x45,0x6a,0x31,0xfd,0x08,0x76,0x40,0x05,0xc4, +0xd0,0xc3,0xc4,0x3a,0x7c,0xef,0x2d,0xcc,0x6e,0x05,0xef,0x9f, +0xd4,0x75,0x51,0x52,0xfc,0x0e,0xbb,0xf1,0xf6,0x8d,0x9c,0x82, +0x14,0xca,0x4d,0x9d,0xa9,0xb7,0x23,0xd6,0x26,0x54,0x7b,0xd0, +0x4e,0x10,0x4d,0xb2,0xa8,0xb3,0x46,0xfc,0xee,0xd5,0xcd,0xb1, +0x90,0xf6,0x6e,0x60,0x1a,0x29,0xeb,0xd5,0xa2,0xd2,0xac,0xc0, +0xb8,0x40,0xb7,0xb8,0x5e,0xb0,0x74,0xa6,0xa9,0x38,0x3b,0xca, +0x67,0x49,0x23,0xd0,0x3e,0x63,0xe7,0x57,0x6b,0x76,0x09,0xcd, +0x13,0xbc,0x44,0x51,0x74,0x24,0xa2,0xbc,0xf0,0x2b,0x1b,0xc5, +0x60,0x2f,0xa2,0x77,0xec,0x95,0x33,0x50,0x14,0x21,0x99,0xbe, +0x71,0xaa,0x2a,0xb7,0x22,0x32,0x03,0xd4,0x16,0xb9,0x03,0x87, +0xbd,0xf8,0x64,0x7c,0x39,0x25,0x59,0xb7,0xa3,0xad,0x06,0x68, +0xae,0x66,0xbd,0x54,0x0a,0xa2,0x5f,0x05,0x09,0xe2,0xe5,0xae, +0xb1,0x44,0xbf,0x79,0xc3,0xac,0x0c,0x1d,0xf3,0x36,0x58,0x36, +0x3c,0x3d,0x7b,0xfb,0xfc,0xcd,0xbb,0x93,0xd7,0x2f,0xe9,0xdc, +0x64,0xc1,0x5f,0x6e,0x46,0xc9,0xe5,0x44,0xad,0xef,0x19,0x9c, +0xc8,0x6f,0xff,0x7a,0xea,0x35,0x1a,0x24,0x17,0x6a,0xf8,0x31, +0x56,0xc2,0x39,0xde,0x9d,0xbe,0x79,0xf1,0xfc,0x2c,0xb3,0x5a, +0x54,0xd7,0xd9,0x3d,0x8a,0x12,0x6a,0x8d,0x9c,0xcf,0x64,0x80, +0x65,0x8d,0x13,0xb3,0xac,0xd0,0x83,0x81,0xdc,0x0a,0xd2,0x15, +0xbe,0xb4,0x67,0xa2,0xfa,0x07,0x34,0x97,0xef,0x70,0xa9,0x1d, +0x77,0x06,0xfe,0x88,0x36,0xc6,0xb1,0x2b,0x5d,0xef,0xf7,0x0e, +0x73,0x2b,0x86,0x79,0x35,0xeb,0x63,0xfa,0x20,0x4f,0xae,0x66, +0xd0,0x51,0x33,0x8a,0x6c,0x51,0xf2,0x80,0x86,0x1e,0xe6,0x3f, +0x2c,0x44,0xd9,0xc9,0x2e,0xb6,0x94,0x88,0xef,0x8f,0x58,0xbc, +0xfd,0x6c,0x8e,0x12,0x06,0x86,0xd5,0x2e,0x4c,0x0f,0x21,0x15, +0x46,0xf2,0x9e,0x52,0x17,0x5d,0x33,0xf4,0x8d,0xa3,0xdb,0x0c, +0x7d,0xd1,0x73,0x5a,0x2f,0x98,0x31,0xb4,0xcf,0xe6,0x80,0x66, +0x8e,0xd2,0xd2,0x28,0x03,0xf6,0x7f,0x58,0x2d,0x46,0x7d,0xc7, +0x0b,0xe4,0xa9,0xde,0x31,0x36,0x10,0x3a,0x8e,0x96,0x14,0x6f, +0x79,0xc0,0xc7,0x65,0x6c,0xb4,0x9e,0xbf,0x4a,0x0d,0x3d,0xf7, +0x72,0xfd,0x24,0xde,0x34,0x11,0xf7,0x4d,0x5a,0x28,0xdf,0x71, +0x1f,0xdd,0x5d,0x74,0xbc,0xcd,0xac,0x24,0x27,0x43,0xa6,0xdc, +0xa4,0xc5,0x35,0x9a,0xbb,0x4e,0xf4,0xa5,0x47,0x84,0xe9,0x6e, +0xef,0xaa,0x95,0x98,0x89,0x6d,0x61,0xe9,0x96,0x08,0x70,0xb5, +0x58,0x8a,0xcf,0x89,0xb1,0xa9,0xbd,0x31,0x29,0xcd,0xe4,0x2a, +0xe9,0xe9,0x68,0xd4,0xe9,0xdd,0xd5,0xd5,0xe4,0xd3,0xc0,0xb5, +0xfb,0x9d,0xd1,0x4b,0x16,0x81,0x4b,0x2b,0xa3,0x69,0x84,0x20, +0xc3,0xd1,0xbb,0xdf,0x25,0x4f,0xb5,0x9a,0x6b,0x13,0xd8,0x78, +0x23,0x91,0xb1,0x54,0xcd,0x35,0xab,0x3c,0xad,0x9f,0x7d,0x5b, +0x51,0xdf,0xca,0xd1,0x79,0x21,0x10,0xa7,0x76,0xe0,0x8c,0x94, +0x4d,0x9a,0x46,0xd8,0x82,0x0f,0xde,0xc4,0x58,0x95,0xc3,0xa2, +0x95,0x4d,0x95,0x87,0xe6,0xce,0xb2,0xc0,0x96,0x01,0x55,0x12, +0x40,0xa8,0x0f,0xab,0x92,0xfa,0xca,0x35,0xaf,0xa5,0x78,0x88, +0xa8,0x70,0x4b,0xef,0x67,0xa9,0x8c,0xcf,0x2b,0x7a,0xc9,0xc2, +0xd8,0x67,0xea,0xa2,0x6b,0x39,0x71,0x0f,0x9d,0xf5,0x85,0x85, +0xde,0x9f,0x4d,0xd1,0x68,0x46,0xb3,0x1c,0x4b,0xe3,0xa4,0x94, +0x16,0x0c,0xad,0xb5,0xa3,0x5a,0x4c,0xec,0x43,0x09,0x33,0x38, +0x99,0x45,0xed,0xcc,0xf3,0x48,0x55,0x45,0x2b,0xc8,0x2d,0x38, +0xa3,0xd8,0xd2,0x5c,0x4a,0x7a,0x48,0x06,0x92,0x61,0xd5,0xe8, +0x16,0x2d,0x6c,0x95,0x1b,0x8b,0x40,0xa3,0xcc,0xa7,0xa8,0x53, +0xc4,0xc7,0x0b,0xd9,0xbd,0x32,0x62,0x99,0xb1,0xfa,0x6b,0x90, +0xbd,0xae,0x8b,0x81,0xac,0x04,0x84,0x3e,0x56,0xf8,0x34,0xa6, +0x92,0x9a,0xdd,0x87,0x49,0x70,0x9d,0x49,0x12,0x1e,0x84,0x24, +0x32,0x65,0x1f,0x0c,0x5c,0x29,0x2f,0x59,0xfc,0x6c,0x56,0x86, +0x02,0x4f,0x92,0xe7,0xc8,0x37,0xd9,0x4b,0xfa,0x89,0x29,0x87, +0x7a,0x67,0x9f,0xcd,0xba,0x4e,0xe6,0x36,0xd3,0x6c,0x2e,0xc9, +0x4b,0x8d,0x1a,0x96,0xb9,0x91,0x5b,0xf5,0x19,0xad,0xac,0x29, +0x32,0xd5,0x27,0x93,0xd2,0x44,0x39,0x62,0x71,0x22,0xf2,0xe3, +0xf7,0x1e,0x12,0x99,0xc4,0x08,0x44,0x89,0x07,0xee,0xb0,0x6c, +0x5e,0x96,0xbe,0x1c,0x2a,0x5b,0x1f,0x69,0x9e,0xbe,0x89,0x3f, +0x1c,0x0c,0xe4,0x49,0xcd,0x49,0x20,0x9b,0xd4,0x47,0x03,0xb5, +0x74,0x64,0x8e,0x61,0x19,0xbe,0x5e,0x24,0x1e,0xd9,0x70,0xe2, +0xd2,0x42,0xd3,0x19,0x71,0xd7,0xe3,0x3f,0x5b,0xde,0xde,0x85, +0xb2,0x02,0x54,0xa4,0xdc,0x08,0xc7,0xf1,0xf9,0x1d,0xd5,0xe5, +0x44,0xc0,0x65,0x22,0x44,0x4d,0xcf,0x6a,0x2d,0x8c,0xbb,0x62, +0x21,0xb8,0xdd,0x1e,0xa3,0x23,0xae,0xcb,0x5f,0xdb,0xed,0xbd, +0x43,0x40,0xc0,0xf9,0xc3,0xb1,0x8d,0x5c,0x3e,0x04,0x0a,0x64, +0x6a,0xd7,0xd8,0x4c,0x4e,0x3a,0x92,0xdb,0x5d,0xd0,0xfe,0x02, +0xe1,0x1a,0x97,0x28,0x8c,0xfa,0x6c,0x38,0x21,0x09,0xeb,0x79, +0xcb,0x1c,0x99,0x2a,0x39,0x0d,0xf7,0x4a,0x6b,0x74,0x37,0x86, +0xd4,0xe2,0x7c,0x46,0x1a,0x1f,0xaa,0x6f,0xcb,0xa0,0x01,0xd5, +0xd7,0x2a,0x21,0x88,0xde,0x2d,0x32,0x2f,0x12,0x5d,0xc7,0x07, +0x58,0xbc,0xd4,0xe1,0x05,0x24,0xcf,0xe2,0x05,0xfd,0x2d,0x75, +0x7c,0x2a,0xd7,0x83,0x39,0xcb,0x68,0x6b,0x52,0x91,0x58,0xb0, +0xde,0x66,0xf2,0xb9,0xfa,0x26,0x1a,0xb5,0x0f,0x9f,0xbf,0x3a, +0x3d,0x3b,0x7e,0x75,0xf6,0xfc,0xf8,0xec,0xf9,0xab,0xbf,0x6e, +0xbb,0x59,0x2f,0x47,0x63,0x54,0x0b,0x3e,0x99,0x2c,0xd1,0x61, +0xcf,0xd2,0x8b,0x87,0x7f,0x86,0xf3,0x03,0xf0,0xb1,0x02,0x87, +0x24,0x21,0x05,0x0e,0x51,0x1e,0x6f,0x7d,0xbf,0x87,0x1c,0x2d, +0x27,0x82,0x5b,0xab,0x7a,0x63,0x0c,0x2a,0x50,0x39,0xd5,0x0b, +0xb4,0xcd,0xce,0x1b,0x9a,0x1c,0x34,0xb8,0x29,0x83,0xde,0x52, +0x16,0xd0,0xe6,0x0a,0x1f,0x84,0xa7,0x9f,0xef,0x59,0x55,0x84, +0xdf,0xf9,0xad,0x0d,0x4e,0x5e,0xc7,0x80,0xbe,0x91,0x01,0x17, +0x92,0xca,0x14,0x72,0x5e,0x55,0x46,0x88,0x9f,0x8c,0xaf,0x55, +0x88,0x32,0x2b,0x36,0x68,0xb1,0x74,0x76,0xf6,0x59,0x1e,0xa8, +0x1f,0x54,0x45,0x4b,0xe5,0xb8,0x50,0xf7,0x9f,0x7c,0x64,0xb0, +0xa1,0x06,0xcd,0x83,0xc7,0x44,0xcd,0xc9,0x30,0x22,0x71,0x5b, +0xe6,0x63,0xb2,0xfa,0x15,0x90,0xcc,0xe4,0xf9,0x0c,0xee,0x07, +0x14,0xba,0x12,0xd5,0x48,0x36,0x79,0xf0,0xeb,0x78,0xb6,0xd3, +0x7a,0xfa,0x69,0x01,0x11,0xb0,0x82,0x65,0x14,0x58,0x41,0x60, +0x58,0xb5,0xb8,0x16,0xd6,0xe8,0xa1,0x49,0x23,0x5e,0x34,0xf6, +0x8b,0xb7,0x38,0x77,0xbd,0xdd,0xe6,0xdf,0x26,0xab,0x4c,0x3d, +0x4e,0x22,0xf6,0x70,0x61,0x16,0x25,0x59,0x87,0xd8,0x54,0x24, +0x31,0x97,0x7a,0xdc,0x1b,0x68,0x8d,0xab,0x59,0x89,0xd8,0x34, +0x0d,0xc0,0x20,0xa3,0x21,0xee,0xed,0x1d,0x16,0x43,0x6f,0xd9, +0x27,0xbb,0x47,0xe1,0xdc,0xd6,0x41,0xa1,0xb7,0x12,0xdc,0xae, +0x47,0x14,0x75,0x68,0xa2,0x86,0x7a,0x4a,0x52,0xca,0xa3,0x7a, +0x0a,0xcd,0x0f,0xda,0x97,0xe0,0x1c,0xdf,0x6e,0xcf,0xc1,0xbf, +0x8f,0x34,0xe7,0x77,0x5f,0xce,0xc9,0xbf,0xdf,0x6a,0x89,0x3f, +0x7e,0x7d,0x09,0xfe,0xfd,0x4e,0x4b,0xfe,0xe9,0xf7,0x97,0xe4, +0xdf,0x3f,0x2a,0x84,0x7f,0xfb,0xe7,0x21,0xf0,0xef,0x9f,0x14, +0xd2,0x9f,0xff,0xf7,0x21,0xf1,0xef,0xbf,0x29,0xc4,0xbf,0xfc, +0xf7,0x41,0xe4,0xdf,0x3f,0x2b,0xe4,0xc3,0x83,0xff,0x7e,0xd0, +0xfc,0xfb,0x97,0x01,0xba,0xd1,0x65,0x99,0x4f,0x57,0x45,0x64, +0xd0,0xa9,0x0a,0x8f,0x0b,0x8f,0x46,0x23,0xcd,0x1f,0x1c,0x18, +0x8a,0xcd,0xe5,0x0c,0x6a,0x3c,0x62,0x0b,0x1e,0x27,0x1e,0xdd, +0xb6,0x28,0xa4,0xe3,0x95,0x98,0x74,0xef,0xc4,0xb3,0x74,0x26, +0x0a,0xcf,0xe8,0xdd,0x19,0xff,0xf6,0xf1,0x8f,0xbf,0x16,0x0b, +0x4a,0xf1,0x45,0x1c,0x42,0x4d,0xe6,0x05,0x0d,0xd4,0xb0,0x3d, +0x82,0xa5,0x70,0xf3,0x5d,0x93,0xa5,0x1f,0x4e,0x86,0x3f,0x0d, +0xca,0x00,0x3e,0x60,0x3f,0x0b,0x2d,0x26,0x4a,0x66,0xf7,0x0c, +0x52,0x14,0x3d,0x72,0x33,0x48,0x85,0x09,0xa3,0xa0,0x99,0x39, +0x36,0x72,0x3d,0x42,0x0b,0x0d,0xe4,0x70,0xe6,0x14,0x91,0xb5, +0x88,0x26,0xe2,0xac,0x49,0x91,0x2c,0xc6,0xec,0xd7,0xeb,0xc6, +0xab,0x91,0x99,0x04,0x1b,0x12,0x49,0x0a,0xee,0x93,0x92,0x3d, +0xeb,0x1b,0x10,0x65,0xe2,0x55,0x54,0xf0,0xf2,0xc0,0xd3,0x3f, +0xc0,0x76,0x69,0x98,0x85,0xc5,0x93,0x07,0x0d,0x2c,0x9d,0x14, +0x7e,0xe1,0xe8,0x9f,0x22,0xa4,0xee,0x35,0xda,0xab,0x1d,0x85, +0xe9,0x8a,0x05,0x38,0x5d,0xa3,0x30,0x59,0x62,0x55,0xc1,0x28, +0x4c,0x94,0x57,0x4b,0xa7,0x55,0x14,0xa6,0x6a,0x74,0x96,0x3b, +0xea,0xaa,0x70,0xa1,0x8d,0x1f,0x05,0x45,0x4b,0xca,0x70,0xec, +0x77,0x6b,0x4a,0xfb,0x11,0x16,0x13,0x64,0xb7,0x4a,0x7b,0x5b, +0xae,0xb4,0xbb,0xd9,0x02,0xf1,0xae,0x9f,0x66,0xbf,0xce,0xe6, +0x1f,0xfd,0x4c,0x6f,0xc5,0x2c,0xf0,0xf1,0xd3,0x52,0x8e,0x38, +0x73,0x31,0x1d,0x55,0x06,0x39,0xbe,0xdc,0xe6,0x20,0xbb,0xb1, +0x0f,0xc0,0xa3,0xec,0x9f,0x20,0x1e,0xc6,0xe8,0x4d,0xf6,0x78, +0x39,0x7e,0x89,0xe0,0x5b,0x58,0x42,0xd3,0x4d,0x01,0xba,0x1a, +0x8d,0xa8,0xb5,0xed,0x84,0x5e,0xc8,0x59,0xde,0x56,0x11,0x7a, +0xa3,0x46,0xe5,0x21,0x3b,0x8c,0x9a,0xc7,0x1f,0x73,0x8d,0x42, +0xda,0x42,0xf6,0xa2,0xd7,0x47,0x4e,0x00,0x74,0xe2,0x29,0x1b, +0x2f,0x28,0x77,0x0e,0x8e,0x88,0x9f,0x39,0x9a,0x54,0x18,0x71, +0x6c,0xf3,0xd8,0x43,0xaf,0xb9,0xf0,0xe1,0x86,0xb9,0xa1,0x44, +0x65,0xf7,0x13,0x11,0xef,0x45,0x9b,0x0d,0x2a,0xe6,0xac,0x56, +0x1e,0xc4,0xe4,0x83,0x67,0x79,0x88,0x42,0xab,0xcb,0x68,0xa4, +0x9a,0x43,0x0c,0xfa,0x19,0x8a,0x8f,0x71,0xb7,0x52,0xb6,0xe9, +0xec,0x0d,0x42,0x13,0x5f,0xbc,0x36,0xd2,0x90,0xa9,0xc1,0x10, +0x0b,0x97,0x58,0xaf,0x93,0x21,0x1a,0xa8,0x0c,0xed,0x8a,0xd9, +0xb7,0x24,0x64,0xb5,0x0a,0x74,0x27,0x23,0xa6,0xf6,0xa0,0x0d, +0xdb,0x87,0xc6,0x42,0x46,0x76,0x3c,0xbd,0xc5,0x23,0x10,0x10, +0x48,0xed,0x0d,0x1e,0x5f,0x37,0x29,0x0a,0x24,0x55,0x37,0x3d, +0xa8,0xe2,0xb6,0xdc,0x66,0x27,0x07,0x73,0x64,0xe8,0x9d,0x4a, +0x1d,0xa5,0xc2,0x1a,0x9f,0xff,0x6d,0x32,0xfe,0x88,0xb6,0x1d, +0xb0,0x60,0x34,0x04,0x0d,0x70,0xaa,0x1f,0x3e,0xd3,0xcb,0x1b, +0x83,0xda,0x02,0x8b,0x9c,0x8d,0x02,0xf5,0x80,0x99,0xda,0x6d, +0x19,0x7e,0xef,0x4b,0xed,0x20,0x3f,0xc8,0x8a,0xc6,0xd8,0x8d, +0x78,0xd7,0x62,0x53,0x84,0x9a,0x45,0x23,0x9c,0x6e,0x7c,0xc3, +0xfa,0x80,0x6a,0xdc,0x9c,0x8a,0xe9,0xf1,0xb4,0xfe,0x4c,0x13, +0x8f,0xd9,0xc6,0x1e,0x3d,0x16,0x80,0xb3,0x65,0x9e,0x72,0xfb, +0xc8,0x35,0x01,0x85,0x36,0x46,0xac,0x44,0x94,0x0d,0x6a,0xfe, +0x9f,0x50,0xb8,0xde,0x49,0xb6,0x9b,0x15,0xa6,0x0e,0xc7,0x5f, +0xdf,0xad,0x70,0x53,0xa1,0x0b,0x36,0xd4,0xc1,0x40,0xab,0x1b, +0xb8,0x1d,0x91,0xe6,0x71,0x98,0x05,0x71,0x75,0x9a,0x6c,0x65, +0x1e,0x66,0x9e,0x5e,0x41,0x5b,0xb7,0x35,0x58,0x27,0xa8,0xb8, +0xb6,0xb7,0x97,0xa3,0x33,0x9a,0x2d,0xa9,0x2c,0xea,0xdc,0x90, +0xa8,0xea,0x6f,0xee,0x3d,0x17,0xdb,0xf4,0x50,0xbe,0x05,0x3a, +0x41,0x48,0x0d,0xf5,0x8e,0x7d,0xef,0x92,0xcf,0xa1,0x94,0x2e, +0xd1,0xd0,0x5e,0xe1,0x1b,0x7e,0x13,0x57,0xf7,0xd6,0x28,0x40, +0x2d,0x72,0xfb,0xd9,0xfd,0x8e,0x1f,0xa7,0x9b,0xf1,0xe5,0xaf, +0xa9,0x7f,0x0b,0x59,0x40,0xa9,0x67,0x33,0x23,0x48,0xc7,0x11, +0xd9,0x3d,0xff,0xa6,0x64,0xa8,0x0c,0x57,0x11,0x83,0x2a,0x3d, +0xd4,0x94,0x40,0xe5,0xce,0x9d,0xf6,0x26,0xf0,0x96,0x32,0x59, +0x8e,0x7f,0x5a,0x4e,0x59,0x67,0x06,0xaa,0x43,0x8e,0xbd,0x4c, +0x29,0x44,0xef,0x94,0x88,0xdc,0x75,0xef,0x96,0x53,0x74,0x13, +0x9e,0x86,0x89,0x26,0xcd,0x3d,0x63,0xdc,0x29,0x28,0x95,0xa4, +0x33,0x0d,0x9e,0x4a,0x14,0x54,0x22,0xa1,0xd4,0x43,0x20,0xf9, +0x38,0x96,0x97,0x9f,0x5e,0x91,0x87,0xca,0x7c,0x39,0xfc,0xa8, +0x72,0xa7,0x6e,0x37,0x23,0x42,0xab,0xeb,0x57,0x4d,0x81,0xb8, +0x13,0x0d,0x0a,0x21,0x2a,0xa0,0xf1,0xf2,0x69,0xc6,0x57,0x53, +0x7c,0x4c,0x4e,0xee,0xc3,0xa3,0x64,0x1b,0x97,0x23,0x19,0x0f, +0x93,0xfe,0x64,0x7c,0x85,0xfe,0xc4,0x49,0xbf,0x9f,0x5a,0x3d, +0xa9,0x5e,0xce,0x2f,0x7f,0x2d,0xc5,0x95,0xd2,0x96,0x35,0x06, +0xa4,0xee,0x03,0x0b,0x07,0x79,0x8c,0x04,0x6b,0x77,0x77,0xfb, +0xae,0x28,0xb7,0x27,0xe5,0x52,0x78,0x32,0xbb,0xac,0x27,0x72, +0xfd,0x66,0x6b,0x6f,0x69,0x62,0xa7,0xb3,0x61,0x38,0x80,0x5a, +0x4e,0xae,0x3e,0xa3,0xba,0xd7,0xab,0x79,0x1d,0x5c,0xe5,0x41, +0xa9,0x7e,0xe8,0xef,0x5b,0x9b,0x0f,0xec,0xc4,0x9e,0x42,0x84, +0x63,0xf3,0xc1,0x7d,0x46,0x96,0xd7,0xb5,0xf6,0xcd,0x11,0xfb, +0xa2,0x76,0x4b,0x3e,0x97,0x66,0xb8,0x61,0x1d,0x8e,0x46,0x6f, +0x28,0xaa,0x0c,0x6e,0x7c,0xf7,0x14,0xe4,0x1f,0xe2,0x0d,0x14, +0x58,0xe7,0x76,0x97,0x1e,0x1e,0x1c,0xd8,0x0d,0xaa,0x75,0x70, +0x5b,0x00,0x5a,0x7e,0x35,0xe3,0x86,0x44,0x7b,0xc3,0x1f,0xb2, +0xcb,0xf1,0x55,0x7e,0x01,0x17,0x9f,0x1a,0xe9,0x72,0x17,0x0a, +0x0b,0x91,0x60,0x52,0x82,0xf7,0x1d,0x3d,0xe7,0x8d,0xb0,0x2c, +0x5d,0xb2,0xba,0x35,0x7c,0xdb,0xe1,0x43,0xb3,0x11,0x77,0x04, +0xbe,0xe9,0x37,0xaa,0xb9,0xc4,0x78,0xee,0xa3,0x5e,0x16,0x51, +0x0e,0x04,0xa4,0x27,0xb3,0xee,0x92,0x9e,0x40,0xee,0xc9,0xb7, +0xbe,0x2f,0x9e,0x12,0xef,0x86,0x0e,0xe2,0x24,0xc9,0x11,0x74, +0x91,0x6a,0x16,0x1c,0x83,0x7a,0xba,0xed,0x2c,0x99,0x35,0x4b, +0x13,0x1c,0x83,0x64,0x5b,0x3c,0x5a,0x16,0xd6,0xca,0x53,0x37, +0x6c,0xfa,0xbc,0x49,0x10,0xed,0xe0,0x70,0x13,0x82,0xe1,0x2d, +0xcd,0xc8,0xb8,0x01,0x64,0x26,0xca,0xc6,0xd9,0xd8,0x70,0xa9, +0xeb,0x75,0x50,0xd8,0x9a,0x28,0xfc,0x37,0x14,0x45,0xfb,0x43, +0x22,0x4b,0xab,0x7e,0x96,0x21,0x65,0xe3,0x62,0x51,0xcf,0x13, +0xb7,0x3f,0x4d,0xd4,0x7c,0xe6,0x0e,0xd1,0xa6,0x8d,0x22,0xad, +0x08,0xca,0xe2,0x0b,0x73,0x34,0x09,0xed,0xb6,0x48,0x24,0x89, +0x2a,0x35,0x80,0x4d,0x13,0xb8,0x5d,0xd0,0x9d,0x2e,0x22,0x73, +0xf6,0xac,0xce,0xcc,0x14,0xc2,0x3d,0x7e,0x49,0x91,0xbd,0x86, +0xf2,0x3e,0x35,0x86,0x50,0x84,0xbb,0x24,0x8d,0xe0,0x47,0x3d, +0x3d,0x30,0x31,0x6e,0x44,0xa2,0x3d,0xe9,0x74,0xc7,0xe5,0x4c, +0xc1,0x29,0xff,0x11,0x67,0x28,0x7a,0x11,0xa2,0x59,0xdb,0xb6, +0x1e,0x74,0xd5,0xe2,0x57,0x2f,0x9c,0xa2,0x5f,0x50,0x41,0xa3, +0xea,0x9d,0x17,0xe7,0xfb,0xe7,0xfb,0xfd,0x5f,0xce,0xf7,0x07, +0xdf,0xb0,0xfb,0xc0,0x24,0xf1,0xbb,0xf0,0x04,0xf5,0x40,0xc8, +0x12,0x4b,0xee,0x3f,0xf9,0x61,0xa5,0x84,0x35,0x7a,0x49,0xdf, +0x6f,0x86,0xa4,0xb0,0x64,0xda,0x98,0x66,0x3c,0x95,0x97,0x52, +0x3c,0x34,0xdf,0x62,0xdd,0xb6,0x52,0x06,0x55,0xd4,0xa2,0x0f, +0x56,0xc7,0xe7,0x70,0x3e,0x61,0x43,0xe3,0xa4,0xf6,0x32,0x53, +0xfe,0xb3,0xb9,0xcc,0xa4,0x06,0x44,0x8a,0xa3,0xe6,0x11,0x4a, +0x52,0x6b,0x73,0xbd,0xa8,0xad,0xb5,0x06,0x44,0xa5,0x1a,0x8f, +0x5a,0x62,0xe7,0xc7,0x8c,0x07,0xcb,0xf4,0x99,0xd2,0x91,0x6b, +0x13,0x4e,0x29,0x4d,0x06,0xe4,0xbb,0xb2,0xa7,0x12,0x8e,0xf4, +0x82,0xa7,0x68,0x20,0x97,0x3d,0x94,0x1c,0xf0,0xb8,0x90,0x45, +0xb4,0xbb,0xd9,0xb8,0xba,0x1c,0x2e,0xc6,0x02,0x4a,0x3c,0x20, +0xe0,0x9b,0xd6,0x81,0xda,0x5f,0x3f,0x32,0x8d,0x92,0xd7,0xcc, +0x50,0x96,0xa4,0x9e,0xbe,0x1d,0x2a,0xbb,0x9e,0x3c,0xcc,0xbc, +0x1d,0x1f,0x53,0x7a,0x53,0x77,0xda,0x5a,0x1f,0xcc,0x52,0x40, +0xd3,0x54,0x75,0x92,0xf2,0x88,0x78,0x18,0x49,0xc7,0xaf,0x93, +0x4e,0x82,0x72,0xf0,0xf8,0x66,0x56,0x9e,0xdd,0xdc,0xe5,0xad, +0x83,0xc3,0xd6,0xbf,0x0f,0x67,0xad,0xc3,0xbf,0xfc,0xdb,0x41, +0xeb,0xe0,0xa0,0xa0,0xff,0x5b,0x7f,0x7d,0x79,0x66,0x1c,0x55, +0x3a,0x0b,0xe1,0x76,0xb1,0x94,0x0d,0xab,0x20,0xae,0x3d,0xe9, +0x48,0x04,0x43,0x80,0xaa,0x6b,0xcd,0x51,0x25,0xb5,0xce,0x61, +0x6e,0xa1,0x3f,0xfe,0xee,0xe0,0x2f,0x7f,0x02,0x12,0x00,0x71, +0xcf,0x8f,0x43,0x14,0x2a,0xe6,0x51,0x68,0xfd,0x41,0x8d,0x69, +0xfe,0x01,0x6e,0xcb,0xaa,0x9a,0x5c,0x4c,0x3f,0xb3,0xea,0xc8, +0x78,0x85,0xf6,0x70,0xe6,0x1f,0xc6,0xcb,0x2b,0x24,0xcf,0x46, +0xad,0x8b,0xf1,0xe5,0x10,0x7d,0xb5,0x4e,0x56,0xad,0x8f,0x68, +0xd2,0x66,0x3e,0x87,0xd1,0x5c,0x5e,0x8f,0x5b,0xa9,0xd6,0xcf, +0x55,0x75,0x92,0xd6,0xe3,0x16,0x56,0xd7,0xba,0xf8,0xbc,0x1a, +0x57,0xd9,0x0e,0x72,0x1b,0x78,0xc7,0x8f,0x10,0x4f,0xb2,0xf7, +0x2d,0x8a,0x42,0xa1,0x4c,0x0f,0x19,0x76,0xe4,0x5b,0xf4,0xf9, +0xc8,0xe9,0xc4,0x6d,0xc3,0x49,0x72,0x97,0xd5,0x22,0xb2,0xe6, +0x14,0x11,0x01,0xab,0x08,0x3b,0xeb,0xbb,0x62,0x83,0xfc,0x61, +0x42,0x63,0xc3,0xed,0x42,0x2b,0xfe,0x31,0x8a,0x67,0x80,0xa0, +0xc0,0x9c,0xfb,0xb2,0x3d,0x14,0x81,0x40,0xdf,0xd1,0x11,0x17, +0xf6,0x7c,0xb8,0x08,0xa6,0xa4,0x0f,0x7a,0xe9,0x96,0x96,0x6b, +0x86,0x00,0x23,0x75,0x60,0x1f,0xea,0x0d,0x71,0x29,0xf0,0x19, +0xb9,0xd8,0x39,0x34,0xb4,0x46,0xac,0x06,0x4e,0x8c,0xa5,0x2d, +0xbc,0x83,0x6b,0xfc,0x91,0xeb,0x04,0x83,0xce,0xcc,0x66,0x23, +0x03,0xc1,0xd3,0x75,0x5e,0x2d,0xdc,0xab,0xfd,0x8e,0x3f,0x3a, +0x9a,0xd0,0x95,0xd9,0x42,0x15,0x5a,0x3a,0xb2,0x51,0x2d,0x3d, +0x68,0x73,0x93,0x59,0x2b,0xab,0xd9,0xce,0xaf,0x77,0xcf,0x47, +0x62,0x06,0x23,0xd4,0xcd,0xbd,0x82,0xdd,0x8b,0x06,0xa1,0x57, +0xf8,0x60,0x48,0x3f,0x3b,0x25,0xc5,0x3d,0x9d,0xe1,0x91,0x03, +0x63,0x3a,0x1d,0x43,0xb0,0xa7,0x81,0xb2,0xa4,0x4c,0x26,0x89, +0x23,0xba,0xb3,0xac,0x08,0x63,0x72,0xa0,0xc0,0x7f,0x4d,0x25, +0x31,0xe7,0x5f,0x98,0x0f,0x1f,0x9b,0x6b,0x3d,0x99,0x0b,0x95, +0x61,0x42,0x97,0xc5,0x11,0x0d,0x59,0x46,0xa5,0x51,0x82,0xee, +0x29,0x65,0x44,0xd1,0xc3,0xa7,0xdc,0x76,0x17,0xb9,0x53,0xba, +0x58,0x74,0xfd,0xa5,0xd1,0x36,0xdc,0x5d,0xf8,0x3c,0x59,0x6e, +0xb3,0xbb,0x30,0x56,0xad,0xd9,0x33,0xd2,0x5a,0x94,0x51,0x44, +0xf7,0x23,0x14,0xac,0x19,0xe7,0xb2,0x63,0x8e,0x06,0xba,0x26, +0x93,0x11,0xf9,0x67,0xe2,0x62,0xce,0x3e,0xd7,0x50,0xac,0xad, +0xae,0x86,0xf8,0xdc,0x96,0xe4,0x02,0x97,0x15,0xd2,0xaa,0xc9, +0x3f,0xc6,0xa8,0x3b,0x0c,0x88,0x4d,0x55,0x8a,0x02,0x05,0xdc, +0x4f,0x32,0x73,0xf9,0xfd,0x64,0x54,0x48,0x7e,0x34,0x67,0xc9, +0xba,0x1c,0x00,0x61,0x31,0xbc,0x9c,0xac,0x3e,0x97,0x92,0xad, +0xdd,0x96,0x40,0x57,0x53,0xd6,0x6b,0xb6,0x9e,0xd3,0x7d,0x79, +0xfc,0xf7,0x77,0x7f,0x3b,0x7e,0xf1,0xd3,0xd3,0x7c,0xba,0xbc, +0x43,0xd6,0x33,0x96,0x77,0xa3,0xcf,0x7c,0x05,0x9d,0x45,0xcb, +0xd3,0xe2,0x1e,0xf7,0xa5,0xee,0x41,0x79,0x8f,0x36,0x5d,0x9a, +0xe4,0xc9,0x08,0xd9,0x58,0xde,0xd1,0xb8,0x95,0x52,0x07,0xbd, +0xf7,0x01,0x5a,0x67,0x3f,0xcb,0x7b,0xf8,0x8b,0x02,0x90,0x9e, +0x43,0xa9,0x8b,0x51,0x8b,0x87,0x32,0xc1,0xea,0xa0,0x82,0x84, +0xce,0x0a,0x75,0x04,0xc3,0x6a,0x30,0x38,0x6a,0x70,0x36,0x46, +0x9a,0x2c,0x39,0x46,0x3f,0xd6,0x21,0x68,0xb7,0x69,0xc7,0xb0, +0x52,0x9b,0x5b,0xbc,0x5d,0xd2,0xbc,0x11,0x5b,0xc0,0xf4,0xbe, +0x60,0x3b,0xf5,0x40,0x77,0x10,0x8b,0x71,0x0d,0xdd,0xda,0x01, +0xd7,0xa2,0x8d,0x28,0x1c,0x7f,0x35,0x78,0x8d,0xc5,0x6b,0x5f, +0x33,0xd8,0x8d,0xe9,0xe6,0x4c,0x53,0x69,0x6f,0xb9,0x9c,0xda, +0x3f,0xbb,0x4f,0x5d,0xce,0x99,0xec,0x42,0x1f,0x91,0x5b,0x28, +0x72,0x10,0xdb,0xe6,0x68,0x9c,0xeb,0x0f,0x8d,0xed,0xde,0x5e, +0xa6,0xfd,0x3a,0x9e,0x4e,0xad,0xb3,0x13,0x5d,0x99,0xb2,0x9c, +0x9b,0xd6,0x5a,0xb0,0xcc,0x36,0xb9,0xe8,0xc0,0xd5,0x80,0xe8, +0x92,0x84,0xed,0x40,0x41,0x05,0x45,0x1f,0xd2,0xaa,0x68,0x71, +0x22,0x37,0xff,0x6a,0xde,0xe0,0x7b,0xc5,0xef,0x27,0x02,0x98, +0xdf,0x63,0xf3,0x0a,0xfc,0xb3,0xd1,0x27,0x1d,0x06,0x65,0x8c, +0x15,0xda,0x4d,0xdd,0x45,0xb8,0x31,0xca,0x4e,0x71,0x75,0xdf, +0x0b,0x0c,0xc8,0xa8,0xcf,0xd0,0xc3,0x80,0xee,0xf5,0x7b,0x2c, +0xe5,0x77,0x13,0xbf,0x34,0x61,0x5c,0xca,0x6c,0xf5,0xab,0xf9, +0x26,0x0f,0x6a,0x0e,0xce,0x79,0x07,0xa5,0x79,0x73,0x86,0x45, +0xed,0xcd,0xd7,0x68,0x26,0x23,0xbe,0xff,0x82,0x63,0xcc,0xdc, +0x73,0x36,0xde,0x33,0x4d,0x83,0x8b,0x26,0x51,0xb3,0x1c,0x55, +0x12,0xdd,0x61,0xa1,0xd5,0x28,0x63,0x6d,0x8a,0xa8,0x1e,0x6b, +0x1e,0x06,0x17,0x89,0xbe,0x9f,0xb9,0xd8,0x24,0x17,0xd1,0xe1, +0x77,0x4f,0x9e,0xbf,0x7d,0x7a,0x72,0xf6,0xfc,0x6f,0x4f,0x9d, +0x39,0xf9,0x5f,0xce,0xab,0x6f,0x9c,0x45,0x99,0x73,0xb1,0x27, +0x3f,0x42,0x0f,0x20,0x68,0x51,0xfe,0xbc,0xea,0xa4,0xdd,0x6f, +0xb2,0xdd,0xfd,0x9c,0x6d,0x97,0xd7,0xcb,0xa7,0x36,0x7b,0xda, +0x2b,0xce,0x8b,0xb4,0xff,0xcb,0x11,0x84,0xb3,0xde,0x51,0x2f, +0xdb,0xcf,0x87,0x48,0x04,0x9d,0x0e,0x81,0xd6,0x9b,0xfc,0x83, +0xc8,0xe3,0x9f,0x6f,0x80,0x94,0x24,0xfb,0x13,0x54,0x77,0xca, +0xb4,0xd7,0xfa,0x6a,0xb5,0x58,0xdf,0x0e,0x27,0xd3,0xd5,0x7c, +0x7d,0x05,0x7d,0xcd,0x8a,0xfd,0x7c,0x72,0x7b,0x7d,0xba,0xbc, +0xfc,0xca,0xb2,0x5c,0x68,0x8d,0xab,0xbe,0x98,0xdc,0x0e,0xaf, +0xc7,0xe7,0xfb,0xfb,0xf9,0xd3,0xbf,0x61,0xa7,0x7f,0x3c,0x7e, +0xf5,0xe4,0xc5,0xd3,0xb7,0xa4,0xac,0xe2,0xfb,0x9d,0xce,0x67, +0xfd,0xe1,0xde,0x3f,0x06,0xdf,0xac,0xd1,0x60,0xd6,0x90,0x86, +0x1a,0x3a,0xaa,0xef,0x28,0x32,0x22,0xa5,0xb9,0xec,0xd9,0xb4, +0xde,0x13,0x6f,0x7e,0x87,0xcd,0x7a,0xc8,0x67,0x3c,0xb7,0x0e, +0x69,0x67,0x73,0xcf,0xa9,0xb7,0xfc,0x1a,0x17,0x31,0x16,0xf8, +0xdc,0xfd,0x97,0x07,0xb3,0xda,0xf8,0x8c,0x0a,0x57,0x42,0x90, +0x49,0x88,0x1a,0x14,0x92,0x51,0x23,0x40,0x56,0x6e,0xb4,0xc3, +0xcb,0x22,0x94,0x5d,0x4d,0x6a,0x46,0x76,0x9a,0xe4,0x59,0xeb, +0xa6,0x78,0xd4,0xe4,0x80,0x5b,0x76,0x75,0xb9,0xc6,0x86,0xb6, +0x79,0xd0,0xf5,0x51,0x43,0x6e,0x78,0x00,0xb3,0xdc,0x8d,0xcd, +0x8e,0xd7,0x0a,0x1d,0x19,0xc1,0x3b,0x97,0x98,0xf5,0x3c,0x88, +0x7b,0xb1,0xbc,0x56,0xa8,0xcc,0xac,0xcf,0xb5,0x29,0x76,0xdc, +0x47,0x57,0xb2,0xb5,0xdb,0x3e,0x0a,0x8f,0x78,0x74,0xae,0x17, +0xe7,0x29,0x6b,0xa0,0x28,0x6b,0x66,0x0c,0x2e,0x76,0x17,0xcb, +0xc9,0x7c,0x89,0x18,0x45,0x3d,0x0a,0x28,0x03,0x93,0x91,0x68, +0xdb,0xf0,0x73,0xbd,0x0e,0x97,0x95,0x0a,0x56,0x96,0xb5,0x98, +0xf5,0xda,0x36,0x4e,0x95,0xd7,0x6c,0x1f,0x6a,0x90,0x90,0xba, +0xbd,0x5c,0x95,0xf5,0xa8,0xf5,0x1a,0x7d,0x6f,0xfb,0xf9,0x64, +0x06,0x8c,0x19,0x2d,0xff,0x08,0x11,0x2f,0x05,0x7a,0x8b,0x30, +0xbd,0xaf,0x36,0x83,0x2c,0x5a,0xbe,0x3c,0xfb,0x11,0x4c,0x9d, +0x44,0xaf,0x61,0x2f,0x06,0x6d,0xad,0xbc,0x7c,0x6d,0xd3,0x65, +0x62,0x63,0x92,0x1f,0x2c,0x1f,0x38,0x5c,0xdc,0x52,0x03,0x10, +0x40,0x66,0x9b,0x6d,0xa9,0xaa,0x8c,0x92,0x00,0x3b,0x73,0x3b, +0x14,0xce,0xc3,0x2a,0x22,0xc5,0xf6,0x7c,0xd2,0x9c,0x87,0x0e, +0xac,0xdf,0xd1,0x9e,0x87,0xc0,0x04,0x0d,0x7a,0x20,0x63,0xf4, +0xa2,0x6b,0x77,0xbc,0x31,0x89,0xd5,0x7c,0x00,0x24,0x64,0xff, +0x0a,0x7f,0x03,0x3b,0x51,0x18,0x41,0x86,0xac,0xd8,0xc6,0xbf, +0xd3,0x96,0xcc,0x23,0x6f,0x00,0x9e,0xca,0x4b,0xd0,0x48,0x15, +0xfe,0x38,0xbb,0x50,0xcd,0x7e,0x00,0xcc,0x41,0x63,0x1a,0x57, +0x3f,0x75,0xd8,0x30,0x57,0x64,0xbe,0x4a,0xac,0x6b,0x59,0x53, +0x63,0xe6,0x11,0xda,0x12,0x89,0xd0,0x18,0x63,0xa2,0xca,0xd9, +0x3c,0x8b,0x5e,0x11,0xd1,0x5b,0x80,0x5a,0x6c,0x24,0xb5,0xd9, +0x7c,0xe5,0x0c,0xc9,0x3d,0x9b,0xe5,0xb7,0xc3,0x4f,0x6f,0x64, +0x2f,0xe7,0x93,0xeb,0xd9,0x7c,0xe9,0x2d,0xb8,0xc1,0xe6,0xb0, +0xe5,0xac,0x59,0x1a,0x66,0xa0,0x22,0x6f,0xd6,0xe6,0x50,0x4d, +0xd1,0x20,0x32,0xf3,0xcf,0x5e,0x61,0x33,0x3c,0xe7,0x10,0x3e, +0xd5,0x47,0xe1,0xb7,0x65,0xe9,0xdc,0x33,0x8b,0x26,0x96,0x7e, +0xb2,0xdf,0x29,0xf1,0xf7,0x75,0x7e,0xda,0xd9,0x47,0xc6,0x51, +0x00,0x52,0x5d,0xe2,0x63,0x6e,0x6d,0x0b,0x39,0x80,0xee,0x7e, +0x44,0x1b,0xff,0xc9,0xf7,0xd5,0x62,0x38,0x7b,0xfc,0xfd,0x3e, +0xfd,0x24,0x99,0xd3,0x1c,0x45,0x51,0xc0,0x0d,0x53,0x5c,0x08, +0x6e,0x5e,0x41,0xc1,0x17,0x70,0x08,0x3e,0x9b,0x95,0x16,0xfc, +0x43,0xc3,0x18,0x26,0x3d,0x34,0xa8,0x35,0xbb,0xd7,0x62,0x5e, +0x1f,0x4d,0xb8,0x9c,0xcc,0x67,0x33,0xc8,0x86,0x6f,0x42,0xfe, +0x7a,0xe5,0xf4,0x84,0x7e,0x12,0x2f,0x71,0xbe,0x8b,0xc7,0x34, +0x56,0x57,0x86,0x45,0x7b,0x35,0xeb,0xfc,0x98,0xcc,0x7a,0x81, +0xe1,0xc4,0x14,0x61,0xa3,0x45,0x40,0xdd,0xc1,0x6d,0x16,0x4f, +0xa7,0xb1,0x75,0x79,0xfa,0x93,0xc1,0x51,0x7a,0x18,0xcd,0xd9, +0x7a,0xfd,0x97,0x28,0x26,0x43,0x86,0x9b,0x82,0x1d,0xff,0x96, +0x4e,0x32,0xf5,0x71,0xc1,0xbd,0xca,0xc5,0x8b,0x85,0xba,0x55, +0x18,0x5e,0x79,0x85,0x75,0x57,0x90,0x3c,0x0c,0xc9,0x28,0x44, +0xc3,0xd5,0x6e,0x87,0xdf,0xa6,0x10,0x43,0xce,0xa3,0x79,0x45, +0x0b,0xbe,0x41,0x84,0x8c,0xb2,0x2f,0xe7,0x42,0x99,0x0f,0xda, +0x27,0xea,0xa0,0x89,0x4e,0xe9,0xdd,0xb9,0xbf,0xa1,0xfb,0x5f, +0xe3,0xf1,0xf1,0x80,0x73,0xfa,0x0c,0x8d,0x2e,0xde,0x82,0xd5, +0x86,0x83,0x87,0x2f,0x08,0xd1,0x42,0x33,0xce,0x49,0x1e,0xde, +0xbc,0x01,0xd4,0x7a,0x57,0x1d,0xf8,0x00,0xe2,0x05,0x4a,0x61, +0x9f,0x99,0x0a,0xfd,0xac,0x73,0x69,0x31,0x12,0xc6,0x61,0xda, +0xb9,0x14,0xc1,0x87,0x13,0x05,0x6d,0x69,0xe4,0xf2,0x4f,0xf2, +0x19,0x52,0x58,0x17,0xdc,0x2f,0xac,0x12,0xb1,0x2b,0x95,0x71, +0x86,0x15,0xa7,0x2d,0xa9,0x2d,0xb8,0xb0,0x94,0xa8,0xe5,0xc9, +0x17,0xea,0xd2,0x3b,0x20,0x33,0x06,0x34,0xa5,0x56,0x55,0x01, +0x9c,0x19,0xc0,0xa1,0x35,0x1b,0x02,0xeb,0xcf,0x06,0xb9,0xef, +0x94,0x96,0xec,0x43,0xad,0x03,0xdb,0xc3,0x30,0xc1,0x17,0xe8, +0xa5,0x3e,0xcc,0xfe,0x31,0x7a,0xa9,0x1f,0x87,0x52,0x1c,0xbc, +0xcc,0xc6,0x1f,0x53,0x27,0x00,0x19,0x17,0x80,0x73,0x31,0x38, +0xa2,0xc2,0x0d,0xe1,0x81,0x01,0x42,0x11,0x43,0xae,0x8f,0x73, +0x69,0xa0,0xfb,0xe5,0x52,0xcf,0xb7,0x5e,0xef,0xd4,0x66,0xb8, +0xdd,0xb6,0x0b,0xac,0xe7,0x21,0xa5,0x76,0xaa,0xcd,0x2c,0xd3, +0xac,0x07,0xab,0xc6,0xdb,0x0e,0x0a,0x56,0x4e,0x7c,0xd2,0xd1, +0x26,0xd5,0x25,0xe5,0xb3,0xd6,0xc6,0xcc,0x9d,0x91,0x51,0x9e, +0xee,0xee,0xae,0x8f,0x21,0x35,0x6d,0x5b,0x5f,0x1a,0xe5,0xce, +0xb5,0x3a,0x7a,0xdf,0x73,0x66,0x75,0xc8,0xcc,0x75,0x2d,0x6f, +0xad,0x26,0xc9,0x9e,0x01,0x32,0x98,0x36,0x0c,0x63,0xd0,0xd1, +0xac,0xf8,0xca,0x41,0x73,0x0f,0x2c,0x0d,0xfb,0x4c,0xa6,0xd9, +0x9d,0x4f,0xfe,0xc3,0x6c,0x58,0x63,0x85,0xeb,0x41,0x60,0x1b, +0xbd,0x25,0xb6,0x6c,0x5c,0x8f,0xdd,0xe6,0xf8,0xa6,0x58,0xe5, +0xbc,0xce,0x9f,0x21,0x20,0x09,0xd3,0x33,0x2e,0xbf,0x84,0xd5, +0x36,0x28,0xec,0x29,0x2a,0x46,0x22,0xab,0xc7,0xde,0x27,0x97, +0x21,0xa0,0x2e,0x11,0x73,0x31,0xa6,0x98,0x2b,0xbb,0x73,0x73, +0x00,0xcd,0xf5,0x1e,0x94,0xe5,0xa4,0x67,0x8e,0xb1,0xc2,0xf7, +0x2a,0x3e,0xd0,0xec,0x7e,0x35,0x98,0xbd,0x38,0x1c,0x24,0x51, +0x28,0x5f,0xdb,0xd9,0x9c,0xcc,0xa1,0x99,0x7e,0xda,0xea,0xb9, +0xee,0xad,0xe7,0x6a,0x46,0xde,0x62,0x83,0x73,0xc0,0xd7,0xcd, +0x18,0x89,0xee,0xb5,0xf1,0xf2,0x96,0xcd,0x09,0xee,0x18,0xf8, +0x81,0xad,0xb4,0x2d,0x09,0xda,0x6e,0xac,0xa9,0x68,0x3c,0xf7, +0xc3,0xfc,0xf6,0xf0,0x69,0xdc,0xec,0x45,0xb0,0x24,0x75,0x12, +0xed,0xa1,0x89,0x11,0x51,0x82,0xe9,0x62,0x66,0xd7,0x40,0x69, +0xc2,0x28,0xe0,0xa9,0xc5,0xd7,0x6b,0x53,0xc2,0xc9,0x4d,0xf8, +0xbc,0xbd,0xe8,0xae,0xa1,0x81,0xb4,0xf7,0x5b,0xd3,0xa2,0xa8, +0xaf,0xc6,0x07,0xef,0x35,0x52,0xf6,0x23,0xff,0xad,0xee,0x36, +0xcd,0x9d,0xdf,0xd8,0x00,0xed,0x60,0x68,0x2f,0x87,0x0b,0xf2, +0xf1,0x06,0xd4,0x03,0xfe,0x1c,0x89,0x62,0x89,0xc3,0x4d,0xee, +0x45,0x3d,0x64,0x68,0xcc,0x6a,0xdb,0x85,0xe3,0x82,0xaf,0x90, +0xaf,0x82,0x16,0xa7,0x9d,0x6b,0xd8,0x77,0x72,0x76,0x07,0xc6, +0x42,0xb2,0x3c,0x79,0x9a,0x3c,0x8c,0x02,0xea,0xee,0xc4,0xf6, +0xb0,0xd9,0x95,0x19,0x77,0xe3,0x5a,0xed,0x95,0x30,0x13,0x3a, +0x27,0xc3,0x21,0x15,0xf7,0xca,0xf8,0xbe,0x7b,0x0f,0xf7,0xdd, +0xfb,0xf7,0x25,0xa7,0xc2,0x82,0xa4,0x5f,0xdd,0x9d,0xef,0xdf, +0x3f,0x7e,0x7f,0xf4,0x5e,0xf1,0x35,0x2c,0x75,0x8a,0x92,0x3a, +0x04,0x15,0xbf,0x9e,0xce,0x46,0x14,0xe6,0xe7,0x77,0x8c,0x11, +0x40,0xfd,0xf7,0x83,0x3c,0xdd,0x41,0x3b,0x11,0xeb,0x35,0xfe, +0x7d,0x5c,0xfe,0x79,0xbd,0x26,0xef,0x78,0xec,0xd1,0x63,0x32, +0x1e,0x21,0xca,0x4d,0x44,0xbe,0xf3,0xad,0x67,0xda,0x5c,0x36, +0x8d,0x14,0x69,0x5f,0xa1,0xbf,0x40,0x64,0x56,0xfd,0xf0,0xfc, +0xd5,0x93,0xe7,0xaf,0xfe,0xca,0xde,0xc3,0x7c,0x41,0x07,0xd5, +0x47,0xc9,0xdb,0x75,0xfa,0xa7,0xda,0xe0,0xee,0x1d,0xee,0x94, +0xfc,0x9e,0x6d,0xb3,0xb3,0x83,0x1c,0x79,0x73,0xa7,0xfe,0x26, +0xa8,0x24,0xce,0xf9,0x82,0x8c,0xac,0xee,0xf0,0x47,0x74,0x69, +0x64,0x87,0xa6,0x9c,0x45,0xe3,0xc3,0xee,0x21,0xa5,0x19,0x07, +0xec,0xd1,0x4b,0x0b,0x67,0x9d,0x64,0x3c,0x1b,0x25,0xc6,0x8d, +0x64,0x63,0xbe,0x3f,0xa1,0xa7,0xc9,0x87,0x46,0x26,0xee,0x9b, +0xae,0xd7,0xfe,0x8c,0x35,0xd0,0x5c,0x9b,0x2a,0x8d,0xe1,0xa7, +0x7a,0xb2,0x91,0x84,0x53,0xd4,0x6e,0xb3,0x34,0x48,0x49,0x59, +0x7b,0x0d,0x0e,0x85,0x69,0xe9,0x00,0xd5,0xec,0x4e,0x69,0x66, +0x4b,0x3c,0x42,0x1d,0x4b,0xef,0xf8,0x30,0x6f,0xb0,0x98,0x43, +0x5b,0x73,0xa6,0xf3,0x13,0xb4,0x02,0xa5,0x5c,0x60,0xbb,0x60, +0x56,0x63,0x83,0xda,0x70,0x12,0xa3,0x5d,0x2d,0x8b,0x99,0x55, +0xf1,0xb6,0xed,0x33,0xde,0x02,0xc8,0xb7,0x79,0x60,0xef,0xe4, +0x5b,0xd7,0xb3,0x3c,0xcb,0x39,0x13,0x53,0x7c,0x5d,0xba,0x13, +0xc2,0xb1,0x2f,0x3d,0x0a,0x0e,0x83,0x87,0x32,0xd7,0x3e,0x82, +0x44,0x5d,0xd9,0xc3,0x65,0x33,0x7b,0xb8,0xe6,0xe4,0x32,0xdb, +0x3a,0xbd,0xea,0xdc,0xf2,0x8b,0xfd,0x3d,0x79,0xf8,0xac,0x88, +0x87,0x9e,0xa7,0x9e,0x80,0x7f,0x8b,0x8c,0x29,0xdf,0x61,0x17, +0xd2,0xc5,0x48,0xb9,0x58,0x12,0xa5,0xc3,0x25,0x0e,0x06,0xaa, +0x26,0x77,0x74,0xb1,0x1c,0x0f,0x7f,0x55,0xed,0x36,0x68,0x22, +0x5a,0xdd,0x68,0x9c,0xcb,0xe8,0x0a,0xf5,0xa4,0x78,0x00,0xe3, +0xcf,0x05,0x12,0x3c,0x32,0x76,0x5b,0x98,0xf3,0x3c,0x7a,0x11, +0x10,0xf6,0x28,0x8a,0x07,0xc0,0xc3,0x23,0x79,0xf8,0xe5,0x91, +0x7c,0xf9,0x4f,0x8f,0x24,0x4c,0x53,0x16,0x50,0x64,0x6a,0xc3, +0xce,0xa3,0x19,0xd5,0x7c,0xb9,0x4a,0x2f,0x3e,0x2b,0xf4,0x80, +0x41,0xe8,0xf5,0x02,0x96,0xf3,0xbb,0xc5,0xe9,0xe5,0x50,0x78, +0x1a,0x6e,0xad,0xea,0x3a,0xf5,0xc4,0x14,0x61,0x57,0xa3,0xf1, +0x62,0x75,0x83,0x18,0xd6,0x95,0x3f,0x8b,0x84,0xe1,0x71,0x33, +0xac,0xdc,0x8e,0x6d,0x88,0xf2,0xf9,0xb3,0xec,0x7e,0x34,0x27, +0xad,0x7a,0xba,0x8a,0x44,0x29,0x46,0x10,0x0a,0xa7,0x13,0x03, +0xf3,0x0a,0xed,0x25,0xb5,0x18,0x41,0x59,0x50,0xa5,0xd3,0xdd, +0x28,0x39,0x6b,0xdf,0xca,0xab,0x34,0xc4,0x40,0x1b,0xf9,0x8e, +0x45,0x4b,0xc1,0x7f,0xb8,0x3f,0x84,0x58,0xca,0xd1,0x4d,0x1a, +0x3a,0x75,0x14,0x33,0x00,0x70,0x36,0x1f,0x68,0x71,0xbb,0x4d, +0x1d,0xef,0x74,0xf2,0xe6,0x5c,0x08,0x53,0xf2,0xec,0xed,0x31, +0xda,0x67,0xf0,0x18,0x8e,0x90,0x0a,0x61,0xd1,0x9e,0x4e,0x2e, +0x50,0xb8,0x7f,0xf3,0xf1,0x06,0x39,0x5d,0x54,0xea,0xf1,0x81, +0x58,0xd8,0x8d,0x8b,0x2a,0xc6,0x62,0x08,0x2f,0x6b,0x2a,0x87, +0xa6,0x4f,0xb5,0x17,0x18,0x81,0xf9,0x99,0xfc,0x9f,0xa2,0x8a, +0x07,0xe1,0xcd,0x0d,0x33,0xda,0xcc,0xce,0x51,0x8e,0x00,0xe3, +0x33,0x9e,0x95,0x57,0xc5,0xb6,0x7b,0x4b,0xbf,0x66,0xbc,0x51, +0xb5,0x86,0x7a,0x04,0x3b,0xfb,0x22,0x78,0xc3,0x4d,0xf8,0x22, +0x4a,0x6c,0xd0,0xce,0x5c,0x79,0x90,0xc7,0x75,0xd4,0xf8,0xfd, +0x6f,0x27,0x8c,0xb7,0xa1,0xf4,0xb8,0xda,0xc4,0x7e,0xcb,0xc2, +0x86,0x4d,0xfc,0x06,0xd8,0xa6,0x3c,0x78,0x15,0x0a,0x50,0xa0, +0x73,0xcb,0x55,0xd3,0xc0,0x2d,0xd0,0x12,0xa7,0x5d,0xf8,0x98, +0xbb,0x7c,0x60,0x0e,0x10,0x58,0x1d,0x0e,0x49,0x6c,0x6c,0x7f, +0x55,0xc0,0xd4,0x17,0x16,0xe5,0x85,0x6f,0x2c,0x03,0x8d,0x8a, +0x6b,0x87,0xa8,0x07,0xab,0x6f,0xee,0x07,0xc3,0x7a,0xa8,0x01, +0x90,0x1c,0xb6,0x60,0x8e,0xe6,0x7e,0x02,0x6b,0x75,0xde,0x58, +0x1f,0xbe,0x11,0x70,0xb9,0xdd,0xb1,0x75,0xce,0xcc,0x57,0x28, +0xac,0x9d,0xe5,0x04,0x35,0x4e,0xd8,0xa7,0x5c,0x29,0x66,0x3c, +0x59,0xc0,0x63,0x38,0x45,0x41,0x5b,0x6b,0x20,0x43,0x40,0x65, +0x6c,0xcf,0xef,0x28,0xf9,0x05,0x10,0x05,0x31,0xb9,0x27,0x49, +0xc6,0x68,0x38,0x39,0x9a,0x2c,0xc5,0xec,0x67,0xa6,0xfd,0xd1, +0x7c,0x72,0xa7,0x1c,0x02,0x76,0xfb,0x8b,0xe6,0x82,0x41,0xab, +0x35,0x28,0x70,0xe0,0x99,0x64,0xbe,0x69,0x1a,0x08,0xaa,0x81, +0xc6,0x72,0x73,0xb4,0xb3,0xfd,0x08,0xe0,0x20,0x45,0x1b,0x87, +0xd2,0x88,0xd0,0xa8,0x61,0xbe,0x23,0x8d,0xd8,0x51,0xd0,0x5b, +0x4e,0xbf,0xcb,0x15,0x3b,0xd9,0xf4,0x85,0xf9,0xa0,0xcb,0xd5, +0xa7,0x20,0x59,0x38,0x77,0x13,0xc7,0xe7,0x5d,0xde,0x82,0x2d, +0xf9,0x87,0x55,0xeb,0x62,0xcc,0x27,0xdf,0x4e,0xa2,0x1e,0x52, +0x0c,0x41,0x60,0xed,0xfd,0xb0,0xa9,0x19,0xf7,0x0a,0xc2,0xca, +0xb6,0x3a,0x01,0x30,0x52,0xdc,0x4f,0xb4,0x92,0x25,0xdc,0x70, +0x05,0x67,0x5e,0x52,0x38,0xf3,0x3d,0x65,0xe5,0xb5,0xf2,0xa5, +0xb5,0x41,0xf6,0xa9,0x22,0x2b,0x37,0x5b,0x98,0x11,0x7c,0x72, +0x78,0x06,0xe7,0x57,0xb0,0xfd,0xf8,0x78,0x71,0xec,0x4d,0x62, +0xe6,0xc9,0x49,0xe8,0xcf,0x1c,0xbd,0xbe,0x2a,0xcb,0x2c,0x2f, +0xcb,0x52,0x6b,0xea,0x05,0x47,0x4b,0x51,0xdd,0x0c,0xa7,0xd3, +0xf9,0xc7,0x93,0xf9,0xe2,0x73,0x1a,0x1e,0x3a,0x21,0x33,0x41, +0x9c,0xd5,0xa6,0x9e,0x0f,0x1b,0xe4,0x66,0x62,0x0e,0x3a,0xaf, +0xad,0x53,0x1a,0xcf,0xb5,0x16,0xc0,0x3d,0xaf,0x08,0x99,0x21, +0x0e,0x4c,0x44,0x41,0xbe,0x78,0x7d,0x72,0xfc,0x22,0x90,0x1a, +0x48,0xfb,0xff,0xb3,0x6c,0x0f,0xb2,0xf4,0xbc,0xd7,0xcb,0xf0, +0xf3,0xfc,0xe3,0x37,0xf8,0xbb,0xbb,0x9f,0xf3,0x3b,0x41,0xc0, +0xa6,0xe2,0xa8,0xf5,0x9a,0x3e,0xbd,0x83,0xfa,0x2d,0x55,0x8a, +0x0b,0x5d,0x2b,0x06,0x39,0xc1,0x47,0xae,0xb9,0xcc,0xca,0x2b, +0xa7,0x11,0x8a,0x74,0x0c,0xab,0x79,0x73,0x0d,0x7f,0x45,0xf5, +0x4d,0xae,0x7e,0xbc,0x12,0xef,0xeb,0xbe,0xb4,0xbc,0x86,0xd8, +0xbe,0x64,0x64,0x49,0x7b,0xa8,0x24,0x99,0x62,0x8a,0xd2,0x52, +0x42,0x94,0xdc,0x96,0xa4,0x8d,0xa8,0x18,0x10,0x9a,0x1e,0xd1, +0xfc,0x87,0x03,0xa5,0x92,0xa5,0xbb,0xbb,0x13,0xee,0x16,0x3a, +0x65,0x82,0x83,0xa5,0xea,0x3b,0x68,0x03,0xb4,0xb2,0x32,0xee, +0x68,0x85,0x04,0x85,0x49,0xea,0xe4,0x7f,0x26,0x85,0xcc,0xc9, +0xfc,0x02,0x75,0x59,0xf9,0x76,0xa7,0x5c,0x6e,0x24,0x44,0x4e, +0x8c,0xc0,0x59,0xa0,0xbc,0xa4,0x85,0x22,0x82,0xfa,0x05,0xc2, +0x92,0xbd,0xba,0x53,0x1e,0x88,0xa5,0x02,0xa5,0x99,0x1e,0x21, +0x96,0x5c,0x1e,0x14,0x78,0x8a,0x41,0xdb,0x37,0xb9,0x34,0xca, +0x9f,0xa5,0x06,0x58,0x66,0x31,0xdc,0xa4,0x4c,0x0a,0xd4,0xa1, +0x93,0xb1,0x83,0x53,0x27,0x2e,0x2a,0xca,0x24,0x6e,0xde,0x4a, +0x7e,0xc8,0xab,0x55,0xe1,0xe7,0xb3,0x74,0x79,0xbb,0x80,0xba, +0x03,0xbe,0xba,0x5e,0xd7,0xf4,0x9b,0xdd,0x82,0x28,0x6b,0x1d, +0x71,0xa5,0x83,0x46,0xe7,0xf1,0xf1,0x37,0x83,0xf6,0x12,0x74, +0x38,0x02,0x9f,0x7e,0x82,0xeb,0xaf,0x42,0xaf,0x77,0x82,0xeb, +0xdd,0x55,0xea,0xdc,0x3d,0x3a,0xff,0xd8,0x0b,0xc4,0x6c,0x8f, +0x8b,0x92,0xf3,0xd0,0x24,0x1e,0xdd,0xad,0x5b,0x8c,0x5d,0x82, +0xb3,0x62,0xc1,0xef,0x6b,0x7c,0xb5,0x45,0x2b,0x93,0x94,0x98, +0x28,0x23,0xc3,0x6b,0x2e,0xee,0x6c,0xaf,0xf9,0x9c,0x40,0xe0, +0xc5,0x75,0xe3,0x5d,0x1f,0x64,0x70,0xcd,0xec,0x7d,0xb1,0xc1, +0x6c,0xaf,0xc0,0xcd,0xa0,0xad,0x3d,0xb7,0xed,0xdb,0x0e,0x28, +0xcb,0x6c,0xc6,0x4d,0xb0,0xc6,0xda,0x49,0xf1,0xe5,0xe5,0x53, +0x6b,0x99,0x57,0x1d,0x14,0xe3,0x11,0xc1,0x28,0x44,0x63,0xa4, +0x96,0x19,0x36,0x52,0xaf,0x9a,0xab,0x68,0xbe,0x3a,0x27,0xd5, +0xe5,0x82,0xec,0xc3,0xc0,0x7e,0x9c,0x8c,0x5a,0x72,0x0a,0xb4, +0xa8,0xf2,0x96,0x3f,0x87,0x50,0x48,0x26,0x58,0x40,0xb0,0xb4, +0xba,0xad,0x27,0x2e,0xbd,0x68,0xdd,0x77,0xbb,0x5d,0xf4,0x69, +0x5d,0x40,0xe2,0x23,0x58,0x5d,0xf0,0x89,0xfe,0xad,0x1f,0x5a, +0x3f,0xfe,0x68,0xcc,0x7d,0x4d,0xec,0x58,0x13,0x8d,0x1b,0xd5, +0x5d,0xd7,0x59,0xeb,0xd5,0x0d,0xa9,0x0d,0xd2,0x3a,0xaa,0xa9, +0xa2,0x99,0x9f,0x87,0xe6,0x2b,0xca,0x7b,0x3e,0x61,0x0a,0x79, +0x1c,0x94,0x9b,0xa5,0x70,0x57,0x0c,0xdd,0x41,0x55,0x21,0xd7, +0xa4,0xe1,0xc5,0xd6,0xee,0xd3,0xcd,0x91,0xaf,0xa5,0x6c,0x92, +0x73,0xc9,0xe1,0xb8,0x2c,0x4b,0x2b,0xf7,0x62,0x3a,0xc1,0x77, +0x5b,0x3f,0x94,0x84,0x19,0x58,0x67,0xaa,0xce,0x70,0x9c,0x11, +0x1b,0x30,0x00,0x9c,0x3d,0x8e,0xdd,0xc0,0xd4,0x32,0xe2,0x56, +0x21,0xd0,0x00,0xc5,0x6a,0x80,0x6f,0xc5,0x82,0x7c,0xea,0x71, +0x85,0x82,0xa9,0x6c,0xe8,0x87,0x87,0xac,0xdf,0x9c,0x6d,0x50, +0x36,0x80,0x44,0x19,0x47,0x7e,0xc0,0x33,0xb8,0x7a,0xfc,0x16, +0x88,0xec,0x05,0x46,0x3d,0x4c,0x2e,0xe4,0xe5,0x07,0x74,0xd1, +0x6e,0x48,0x18,0x71,0x9a,0x62,0xe2,0xed,0x76,0x84,0x4d,0x85, +0xc9,0x16,0xa9,0x7a,0x48,0x3c,0x28,0x27,0xc5,0x47,0xb8,0x05, +0xcf,0x86,0xd7,0xa9,0x29,0x43,0xab,0xf2,0xe1,0x27,0x1c,0xf7, +0x06,0xfe,0x95,0xcf,0x38,0x6e,0x5c,0x0c,0x09,0xf1,0xd0,0xc0, +0xf8,0x6c,0xff,0x9f,0x1a,0x19,0xf7,0x1a,0xe5,0x16,0x45,0x88, +0x43,0x7b,0x09,0x18,0x43,0x78,0x7a,0x96,0x5f,0xe3,0x66,0x9e, +0x46,0x0f,0x59,0x8c,0x34,0xe9,0x7b,0x8c,0xb2,0x69,0xca,0xbd, +0x9a,0xa4,0xbb,0x5f,0x3d,0x88,0x8c,0x9b,0x21,0xc3,0x4f,0x38, +0x90,0xc2,0x93,0x88,0x25,0x8b,0xb7,0x9d,0x53,0x9c,0xba,0x8a, +0xbd,0x7d,0x72,0xb4,0x95,0xae,0x28,0x23,0xdc,0x75,0x37,0xb2, +0xad,0x6e,0xb2,0x3a,0x3d,0x4e,0x0f,0x6e,0x1b,0xcd,0xdd,0xf0, +0x0e,0x1c,0x10,0xee,0xb2,0x7a,0x6a,0x8f,0x65,0x46,0xa4,0xc3, +0xcb,0x16,0xfa,0x5c,0x28,0xd4,0x11,0x3c,0x1d,0x98,0x03,0x0b, +0x70,0x2e,0x4b,0xfd,0x06,0x29,0xe3,0xd9,0x28,0x62,0x65,0x05, +0x02,0x27,0x86,0xd3,0x61,0x19,0x0f,0x4d,0x14,0xb5,0x5b,0x0f, +0x5e,0x25,0xad,0x36,0xb5,0x8f,0xeb,0xc2,0x88,0x19,0x5f,0x69, +0x68,0x49,0x36,0x5c,0x17,0xa6,0x99,0xfc,0xf6,0x8e,0xa4,0x3f, +0x09,0xd8,0xbc,0x9a,0x3f,0xb9,0x5b,0x4c,0x27,0x97,0x88,0x0b, +0x26,0x72,0xbd,0x8d,0x5a,0xf2,0x08,0xbf,0x6d,0xda,0xfd,0xaa, +0x0b,0x26,0xd9,0x58,0x3b,0x0a,0xab,0xc7,0xea,0x42,0x81,0x11, +0x3b,0x00,0x28,0xd6,0x22,0x35,0x3b,0xf1,0x96,0xad,0x0b,0xce, +0x5f,0x5d,0xf9,0xc3,0x10,0x0d,0xa4,0x78,0xcd,0xc6,0x31,0x46, +0xf0,0x32,0x0b,0x37,0x66,0x24,0xcb,0x99,0x6f,0x1f,0x56,0xbf, +0x47,0xb3,0xe0,0xe6,0x7a,0x62,0x5f,0x84,0xeb,0x91,0xeb,0xf5, +0xfd,0x26,0x6f,0x98,0x89,0x3f,0x98,0x7b,0xe9,0x95,0x68,0xe5, +0x5d,0x36,0x5e,0x4c,0x46,0x2a,0x33,0x28,0x31,0xd8,0x3a,0x49, +0x5f,0x51,0xd6,0x0e,0xf3,0xf6,0x3e,0xfb,0xbd,0xb6,0x65,0x3d, +0x69,0x86,0x8a,0x2c,0xce,0x34,0x1d,0x6d,0xdb,0xda,0xd8,0x90, +0xd7,0x57,0x5c,0x3f,0xe5,0xea,0x5b,0x21,0x4f,0xd4,0xfd,0x5a, +0x59,0x86,0x1d,0xe8,0xa5,0x7e,0x77,0x7d,0xed,0x8e,0xfc,0x5d, +0x67,0x59,0xe4,0x14,0xed,0x64,0x7e,0xcb,0x4e,0xdb,0x5a,0xb5, +0x39,0x2d,0xd0,0xdd,0x87,0x05,0x16,0x4d,0x03,0xf9,0x41,0xc9, +0x2c,0x9b,0xb2,0xb4,0x0d,0x41,0x46,0xa9,0x9c,0x95,0x3f,0x03, +0x05,0x93,0x06,0x7c,0x4a,0x75,0x98,0xe8,0xcc,0xa2,0xb8,0x5e, +0x87,0x10,0xb3,0x86,0x43,0xd4,0x89,0x4d,0x36,0x5c,0x4b,0x70, +0xac,0xc6,0xc3,0x5f,0x3b,0xb0,0xdb,0xed,0x38,0x86,0x09,0xa2, +0xac,0x30,0x63,0x2f,0x0d,0xb4,0xee,0xdd,0x6c,0xb3,0xc8,0xec, +0xec,0x0a,0xb9,0x90,0x69,0x38,0xfc,0x5d,0xf4,0xa7,0x91,0xa2, +0xc9,0xf4,0xdf,0xdb,0xf0,0x2c,0x10,0xea,0x76,0xc3,0x81,0x47, +0x76,0xc3,0xda,0x95,0xe4,0xa4,0x7e,0xbd,0x6d,0x5f,0xb5,0xb5, +0x8b,0xb0,0xe1,0xc2,0x17,0xa3,0x71,0x0d,0x92,0xee,0xbe,0x49, +0xbd,0x7a,0x5c,0x78,0xc2,0x05,0xab,0x26,0x2b,0xea,0xd9,0xe3, +0xfa,0x46,0xe3,0x99,0x3e,0x28,0xa9,0xda,0x4b,0x7c,0x48,0x07, +0x92,0xe5,0xde,0x7e,0x44,0xed,0x36,0x36,0xbd,0x8f,0x27,0x93, +0x7d,0x10,0x8a,0x0f,0x9b,0x08,0x7c,0x27,0x11,0xaf,0x84,0xc1, +0xcc,0x06,0x6b,0x5b,0xc1,0xe1,0xc2,0x3e,0xdc,0xf1,0xdf,0x72, +0x6d,0xaf,0xd7,0x87,0xf8,0x5c,0x69,0x96,0x82,0x93,0x61,0x68, +0xa6,0xde,0x56,0x8b,0xe9,0x12,0xc5,0x96,0xb5,0xc7,0x4d,0x64, +0x1a,0x9b,0x35,0xbf,0x19,0xc2,0xf7,0xf8,0xd3,0xf0,0x72,0x35, +0xfd,0xdc,0x82,0xa5,0xd8,0x42,0xe6,0xa0,0x73,0x2f,0xd8,0x22, +0xd3,0xb4,0x21,0xca,0x96,0x90,0x05,0x82,0x2d,0x9b,0x2f,0x98, +0x2b,0xbb,0x44,0x08,0xaf,0x80,0x0b,0xe8,0xcc,0xce,0x1f,0x90, +0x59,0x78,0xd0,0x14,0xf7,0x9b,0xcd,0x91,0x91,0x20,0x32,0x68, +0x8b,0x58,0x60,0xae,0x8b,0x8f,0xd8,0x7a,0xec,0x63,0xdd,0x82, +0x2c,0xeb,0x4f,0x3a,0x87,0x79,0x0d,0xf9,0xd9,0xc3,0xe8,0x8c, +0x2e,0xc6,0xa0,0x15,0x10,0x75,0x3b,0x5e,0x5e,0x8f,0x6d,0xa4, +0x70,0x22,0x6b,0x6c,0xca,0xb0,0x60,0x23,0x8e,0xc5,0x0f,0x50, +0xf5,0x8d,0x1b,0xad,0x0a,0x72,0xde,0x55,0x5b,0xbd,0x3f,0x2d, +0xa7,0xd9,0xff,0xc9,0x3d,0xa9,0x0b,0x9e,0x18,0xfa,0xdb,0x96, +0x7b,0x20,0x80,0x25,0x75,0x9c,0xf9,0x06,0xa7,0x0d,0xe3,0xdf, +0x30,0xfa,0x13,0x0b,0x26,0xdf,0xbe,0xa9,0xc3,0x57,0xa7,0xda, +0x51,0xd7,0x3c,0xe8,0x47,0x6c,0xbe,0xfa,0xaa,0xae,0xe6,0x62, +0xc9,0xa4,0x5a,0xe2,0x03,0x67,0x4b,0x63,0xcd,0xfe,0xec,0x62, +0x88,0x59,0xcf,0x3c,0x79,0xb1,0x5e,0xe2,0xb6,0xe7,0xc2,0x62, +0x2a,0xc4,0xa1,0x29,0x21,0xc4,0x16,0x3e,0x6f,0x69,0xb0,0xf9, +0xa9,0xe9,0x77,0x10,0x5f,0xc1,0x55,0xb2,0xb1,0x2b,0x8c,0xef, +0x2e,0x79,0xa2,0x8d,0xc4,0xd8,0x48,0xa4,0x32,0x46,0x2e,0x5e, +0xa2,0xc9,0xc4,0xdb,0xe1,0xa7,0xb4,0x76,0xef,0x35,0x60,0xe0, +0x4e,0x86,0x3b,0x16,0x77,0xad,0xa3,0x9d,0xed,0x76,0x2d,0xaa, +0x6b,0x45,0x93,0x23,0xf9,0xb6,0xb2,0x01,0x27,0x12,0x1a,0x3b, +0x90,0x38,0xf6,0x85,0x37,0xf6,0x45,0xd2,0x0b,0x0e,0x98,0xf3, +0x83,0x65,0xaf,0x9c,0x65,0xac,0x87,0xe4,0x4e,0x68,0x74,0x9d, +0x74,0x16,0x90,0x3a,0x4e,0x78,0x49,0x1d,0x47,0x96,0x65,0x19, +0x8b,0x1c,0xe8,0x58,0xa0,0xea,0xb4,0x13,0x5b,0x63,0x4d,0x6a, +0x28,0xf4,0x65,0x95,0xb8,0xec,0x01,0xc2,0xb9,0x32,0x0a,0x65, +0x68,0x2d,0xd2,0xe8,0xc4,0x65,0x5f,0x41,0xf9,0xe1,0x7e,0xd8, +0x42,0xfc,0xe5,0xa9,0x19,0x08,0x6b,0x16,0x64,0xbd,0x36,0x09, +0x4d,0xf4,0x17,0x5a,0x51,0xdf,0x69,0x50,0xc7,0x72,0xe6,0x50, +0x74,0xa8,0x33,0xd6,0x44,0xf6,0x23,0x6a,0x55,0xd3,0x4a,0x79, +0x43,0xf4,0x31,0xf9,0xbd,0xd0,0x9d,0x45,0x38,0x0d,0x44,0x73, +0x16,0x66,0x32,0xf0,0x35,0xcc,0xcc,0x6f,0xac,0xf6,0xa5,0x0f, +0x28,0x2e,0xe2,0x0b,0x7a,0x60,0x32,0x7f,0xb7,0xa1,0x93,0xe7, +0x6d,0xd7,0xc4,0x08,0x6d,0x8d,0x2d,0x2f,0xc5,0xa1,0xf9,0xf2, +0x12,0x93,0x4a,0xf8,0xe5,0xa7,0xaa,0x1c,0x92,0x29,0x06,0x7e, +0x25,0xc6,0x61,0xce,0x14,0xa5,0x5f,0xee,0x35,0x69,0x64,0x0d, +0x46,0xf2,0xfb,0x2f,0xa9,0x89,0x27,0xbb,0xc9,0x0e,0xfa,0x83, +0xf4,0x6f,0xb7,0x38,0x9a,0xcb,0x4b,0x52,0xc5,0xd6,0x97,0xc6, +0x4e,0x99,0x26,0xd5,0xea,0xf3,0x74,0x8c,0xd6,0x15,0x21,0xa1, +0x97,0x1c,0x25,0x05,0xe2,0xd3,0x1d,0xcd,0x99,0xe5,0x54,0x6d, +0x35,0x5e,0x79,0x8d,0xfd,0x1c,0x0e,0x00,0x69,0x39,0xe7,0x41, +0xe6,0x9c,0xb6,0x07,0x12,0x9a,0xdb,0x43,0x92,0x47,0x09,0x57, +0x93,0x36,0x6b,0x46,0x88,0x98,0x19,0xd4,0xd8,0x97,0xec,0x83, +0x32,0xb5,0x5f,0x3d,0xfb,0x81,0x88,0x7f,0x01,0x18,0x46,0x87, +0x8b,0x15,0xae,0x27,0x58,0x83,0xd7,0xaa,0x58,0xa1,0x35,0x24, +0x4e,0xca,0x1b,0x63,0xd1,0x30,0x4c,0xe2,0x80,0xec,0x32,0x00, +0x3f,0x6c,0x40,0xf8,0xc2,0x08,0x34,0x58,0x26,0x5f,0xaf,0xa9, +0x6d,0xc6,0x78,0x80,0x4c,0x1e,0x47,0x05,0x23,0xb4,0xc9,0x6a, +0xfa,0x1b,0xcd,0xb7,0x62,0x7e,0x31,0x86,0x91,0x74,0x89,0xaf, +0xb6,0xdd,0x82,0x7c,0xef,0x04,0xcf,0xb3,0xf5,0x4b,0x88,0x9f, +0x67,0xaf,0xe0,0x48,0x6e,0x80,0x17,0xc4,0x9f,0x98,0x27,0x60, +0xbc,0x5e,0xd8,0x47,0x45,0x7f,0x10,0x35,0xe7,0xe4,0x01,0x22, +0x0b,0xf9,0x4f,0x64,0xe2,0xb2,0x01,0x2b,0xa8,0x9c,0xb3,0xfb, +0x11,0xec,0xd9,0x0f,0xe3,0xd1,0x69,0x90,0xcf,0x18,0xad,0xa8, +0x01,0xc9,0xef,0x3d,0x29,0x5e,0x04,0x9c,0x34,0x18,0x38,0x89, +0xf0,0x2c,0x75,0x56,0xfd,0x27,0x76,0x3c,0x05,0x05,0x51,0x61, +0x09,0xe3,0xcc,0x16,0xb5,0xc4,0x45,0xbd,0xd6,0x00,0xc9,0xea, +0x3d,0x9c,0x9e,0x36,0xcc,0x4c,0x56,0x3c,0x5c,0xe6,0xc8,0xe9, +0xc2,0x37,0xd1,0x6c,0xa4,0xf1,0x47,0xe7,0x35,0xaa,0xef,0x61, +0xe0,0x6c,0x09,0x08,0xf8,0x78,0xf4,0x76,0x5c,0xcd,0xef,0x96, +0x97,0x54,0xab,0x6d,0x21,0x0c,0x12,0x6a,0x05,0x16,0xa1,0x92, +0xe0,0x26,0xeb,0x56,0x77,0x97,0x97,0x63,0xd8,0x62,0x5e,0xbb, +0x82,0xa9,0x0a,0x7d,0xd3,0x08,0x11,0x9a,0x00,0x63,0xf5,0x84, +0xcb,0x11,0x59,0xb3,0xa2,0x72,0x8d,0x24,0x92,0xc2,0x6c,0x98, +0xbd,0x80,0x48,0x7a,0x98,0x12,0x52,0x28,0xff,0xbf,0x27,0x81, +0x1a,0x46,0x81,0x30,0x08,0x3b,0x61,0x47,0xb5,0xe1,0x36,0xf4, +0x4d,0xc0,0xaf,0x08,0xb6,0xf8,0x56,0x92,0x29,0xff,0x5a,0xaa, +0xa7,0x3e,0xcf,0x0f,0x50,0x34,0x22,0xe6,0x55,0x2b,0xc1,0x64, +0x8b,0x9d,0x92,0xad,0xc7,0x44,0x03,0x57,0x42,0x27,0x9a,0x04, +0xdc,0xcd,0x11,0xa1,0x2e,0x39,0x9a,0x8e,0x88,0x2c,0xdf,0x72, +0x8c,0x95,0xbf,0x4f,0x7e,0x6d,0x0b,0xba,0x1e,0x0e,0x6c,0x7d, +0x02,0x8d,0x76,0x6a,0xa3,0x46,0x13,0x09,0x72,0x4e,0xb2,0x7b, +0x12,0x3a,0xb4,0x2b,0x0f,0xb9,0xdb,0xa6,0x04,0xe0,0x4d,0xf1, +0xb9,0x89,0x77,0xe7,0xb6,0xa3,0x78,0xbb,0x62,0x29,0x0a,0xe9, +0x9a,0xe7,0xa1,0xfa,0xd1,0x7f,0xe4,0x4e,0x71,0xef,0x4a,0x95, +0x30,0x0e,0x42,0xb3,0x7d,0xa2,0x9e,0xcb,0xe1,0x04,0xbe,0x50, +0xad,0xd1,0x7a,0x46,0x8c,0x79,0xeb,0x7b,0xd4,0x90,0xc3,0xbc, +0x48,0x36,0x17,0x6f,0xba,0x3e,0x8e,0x9a,0x1b,0x00,0xfb,0x78, +0xeb,0xd2,0xc2,0xd7,0x44,0x85,0xb6,0x8d,0x41,0x16,0x6c,0xa5, +0xa8,0xe9,0xca,0xfd,0x6b,0xae,0xd9,0x37,0x35,0xdb,0xba,0xf8, +0xd2,0x6d,0x97,0x75,0xba,0xf5,0x72,0x7d,0x48,0xbe,0xca,0xb0, +0xc7,0xbf,0x36,0xdf,0xc6,0x5f,0xd6,0x7c,0xc5,0x89,0xa9,0x64, +0x23,0x2e,0x56,0x2d,0xe6,0xb3,0x0a,0x4f,0x0b,0x00,0x73,0x33, +0x1e,0xa2,0x37,0x43,0x84,0x70,0x35,0xb9,0x76,0xd6,0xf1,0xeb, +0xc7,0xe1,0x7c,0x38,0x0a,0xdc,0x12,0x61,0x44,0x4b,0x0f,0x30, +0xb1,0x92,0xcf,0x40,0xd0,0xae,0x2a,0xe1,0x7f,0x5a,0x23,0x93, +0x37,0xf5,0x3e,0xd3,0x1e,0xd9,0xd2,0x8d,0x7b,0xd7,0x8d,0x5e, +0xea,0xd7,0x0c,0xe1,0xe5,0xa2,0x9a,0x1b,0xc5,0xce,0xdc,0x0c, +0x99,0x48,0xab,0x9c,0x15,0xa7,0x99,0xca,0xb2,0x62,0xdb,0x74, +0x1a,0x51,0x92,0x7f,0x76,0x66,0x1f,0xea,0xe5,0xe6,0x6b,0xf2, +0x78,0x3c,0xd1,0xcb,0x91,0xa7,0xc3,0xfc,0xc2,0x09,0x6c,0x5c, +0x38,0x4a,0x6a,0x6f,0xe8,0x82,0x86,0x8a,0xad,0x71,0x84,0x3e, +0xde,0x0c,0x57,0x64,0xb0,0x6c,0x32,0xbf,0xab,0x9a,0x78,0x41, +0x4e,0x84,0x14,0xee,0xe8,0x5a,0xbe,0x2d,0x57,0xe6,0xed,0xdd, +0x74,0x35,0x19,0x91,0xb4,0xf8,0x4b,0x0c,0x2e,0xa6,0x63,0x23, +0x08,0xdf,0xea,0xc3,0x0a,0xc9,0xf1,0x3a,0x1c,0x40,0x83,0x7e, +0x45,0xf9,0x70,0xbc,0x52,0xef,0x1f,0x6d,0x5a,0x24,0xe7,0xf1, +0x2d,0x3a,0x59,0x88,0x6b,0x8a,0x2d,0x77,0xd0,0x27,0x35,0xde, +0x32,0x2d,0xcc,0x53,0xba,0x21,0xdc,0xbf,0x46,0x31,0x61,0x05, +0x79,0xd4,0xfe,0x92,0xcb,0x09,0x87,0x6c,0x20,0xea,0x85,0x99, +0xc8,0x8d,0x5c,0x90,0xc7,0xd8,0x17,0x11,0x72,0xf1,0x5e,0x47, +0xbe,0x38,0xc8,0x63,0x7b,0x2b,0x91,0x94,0x37,0xad,0x55,0x23, +0x9b,0xc4,0x52,0xe9,0x6a,0x56,0x80,0x34,0x4d,0x51,0x5a,0x4e, +0x64,0x87,0x54,0xe6,0x42,0xa2,0x13,0x12,0xd5,0x3b,0xd2,0x4c, +0x5c,0x79,0xd0,0xb6,0xe8,0x8d,0xaf,0x19,0x8a,0xab,0x25,0xa3, +0x77,0x3f,0x07,0x3c,0x14,0xa5,0x0a,0xe0,0xd6,0xc4,0xef,0x66, +0x72,0xef,0x38,0xb5,0x0c,0x95,0xbf,0xdb,0x04,0xe4,0x8d,0xc7, +0x56,0x51,0x56,0x00,0x86,0xd3,0xab,0x39,0x38,0x75,0x0d,0xd6, +0xc5,0x91,0x15,0x9d,0x7c,0xc2,0xbd,0xfa,0x23,0x6b,0x28,0xd5, +0xb3,0xad,0xd7,0xc9,0xf3,0x97,0x7f,0x05,0x62,0x36,0xd2,0xa7, +0x83,0xe3,0x3f,0x01,0x1a,0x6b,0x5b,0x99,0x19,0x5a,0x09,0x69, +0x4c,0xcc,0x7a,0x84,0x55,0xbf,0x7d,0x7a,0xfa,0xfa,0xa7,0xb7, +0x27,0x4f,0xdf,0xfd,0xf4,0xf6,0x45,0xc1,0x26,0xe2,0x82,0x25, +0xf5,0x7b,0xf5,0x96,0x9c,0x74,0xe6,0x03,0xab,0x4b,0xc9,0x67, +0xe2,0xe8,0x86,0xb3,0x88,0x9b,0x8f,0x77,0xd5,0x82,0xc9,0xf1, +0x19,0x31,0x3c,0xfe,0x6f,0xee,0xde,0x7e,0xbf,0x8d,0xdc,0x58, +0x14,0xfc,0xfb,0xde,0xfb,0x12,0x54,0x1f,0x85,0xee,0x1e,0x42, +0x94,0x28,0x4f,0x72,0x92,0xe6,0xb4,0x79,0x34,0xb2,0x1c,0x3b, +0xb1,0x2c,0x1d,0x49,0x93,0x93,0x44,0xe2,0x38,0x94,0xd8,0x92, +0x68,0x53,0xa4,0x86,0x1f,0xb2,0x1d,0x91,0xef,0xb2,0x7f,0xec, +0x93,0xdc,0x27,0xdb,0xfa,0x02,0x50,0xe8,0x6e,0x4a,0xb2,0x93, +0xbd,0x39,0xbb,0x33,0x3f,0x8b,0x8d,0xaf,0x42,0x01,0x28,0x14, +0x0a,0x40,0xa1,0x2a,0x3a,0xde,0x7b,0xbb,0xb7,0x7b,0x42,0x11, +0x61,0xfb,0x57,0x4c,0xcc,0x69,0x3e,0x24,0x28,0x30,0x31,0x45, +0x0f,0x13,0xf9,0xf7,0xec,0x1a,0xc4,0x58,0x0b,0xfd,0x99,0x7f, +0xd3,0xc1,0x0a,0x7c,0x33,0xeb,0x05,0x36,0xef,0x37,0x6b,0xc2, +0x94,0x84,0xc9,0xeb,0xa9,0xc7,0xd5,0xb6,0x57,0x4f,0x05,0x34, +0xf0,0xfd,0xc8,0x64,0xd0,0x0a,0x26,0xe2,0x69,0xd8,0xab,0x6d, +0xf2,0x83,0x41,0x15,0x01,0x5b,0xec,0x62,0x54,0x76,0xbf,0xa4, +0xde,0x5b,0x69,0xe5,0x4a,0xde,0x0d,0xd2,0xc9,0x58,0x75,0x17, +0x41,0x3b,0xc6,0x37,0xec,0x2b,0x9b,0xd4,0xd6,0xec,0x30,0xa0, +0xf9,0x43,0xe2,0x53,0xaf,0x4f,0xf6,0xdf,0xd6,0x5e,0x1e,0xec, +0xb3,0x47,0x69,0xdf,0x5b,0xd3,0x1a,0x4c,0xae,0x1a,0xba,0x1b, +0x19,0x92,0x91,0xd7,0x66,0xad,0x76,0x38,0xcc,0xf1,0x65,0x13, +0x5a,0x7a,0xc5,0x3e,0x86,0xb9,0x55,0x43,0x2c,0x09,0x54,0x0c, +0xbb,0xaf,0x6b,0xf4,0x82,0x06,0xb1,0x17,0xc0,0x8c,0x3f,0x92, +0x71,0x14,0x58,0x82,0x6b,0xe8,0x58,0x7a,0x44,0x51,0x89,0x8d, +0x6b,0x46,0x05,0x9e,0x53,0x56,0x41,0x15,0x7b,0x52,0x6b,0x5b, +0x66,0xc5,0x1c,0xe3,0x46,0x9b,0x02,0xeb,0x52,0x65,0xb3,0x20, +0xc9,0xae,0xad,0xb1,0xd6,0x9c,0xa5,0x7c,0x68,0xb3,0xa5,0x18, +0x97,0x9d,0x76,0x93,0x84,0x2c,0xe4,0x03,0x08,0x3c,0x03,0x2e, +0x0d,0x4d,0xbd,0x5e,0x8c,0x11,0x1b,0x48,0xa2,0x83,0x2b,0xea, +0xc5,0xc9,0xd3,0xd8,0x0d,0xc3,0x9a,0xe6,0x81,0x67,0x57,0xf4, +0x6c,0x90,0x04,0x4e,0x48,0x57,0xee,0x95,0x84,0xd2,0x60,0x47, +0x70,0x44,0xe6,0x06,0xf1,0x4a,0xe6,0x9d,0x63,0xc5,0xa4,0xa5, +0x7e,0x89,0xbe,0x49,0x24,0xbf,0xcd,0x96,0x15,0xcb,0xf1,0x05, +0x32,0x7e,0xb1,0x1b,0x80,0x62,0xba,0xc8,0xd7,0xa2,0xaf,0x99, +0x55,0xc1,0x14,0x66,0x8f,0xb5,0x23,0xe9,0x06,0xaf,0xc8,0x95, +0x89,0x0b,0x1d,0x5f,0x3a,0x8a,0x2d,0x94,0xc3,0x9d,0x44,0x65, +0x5d,0xe2,0x77,0xc4,0x65,0x6b,0x34,0xba,0x99,0xb4,0xdc,0xbd, +0x28,0xfe,0x90,0x0d,0xcc,0x87,0xed,0xec,0x43,0x43,0xb5,0x6b, +0xa3,0x85,0x8f,0x85,0xab,0x50,0xb0,0x4f,0x85,0xa1,0x08,0xe0, +0x81,0xa1,0xed,0x4e,0x50,0xc7,0x87,0x6e,0x16,0x86,0xb7,0xbb, +0xa9,0x98,0x59,0x2c,0xe4,0x6b,0x57,0xc0,0xdf,0xc8,0x02,0x2c, +0x58,0xe3,0x74,0xc9,0x3d,0x56,0xaf,0xcb,0x72,0x26,0xa3,0x4c, +0xf2,0x56,0x2c,0xcd,0xa9,0x1c,0x3e,0xbe,0x3c,0xbc,0x9c,0xf4, +0xae,0x42,0x07,0x01,0xac,0x66,0x60,0x0d,0x46,0xbf,0x92,0xf4, +0x18,0x36,0xa1,0xf2,0x89,0x6e,0x52,0xf2,0x51,0x9f,0x6b,0xa8, +0x84,0x6c,0x09,0xe8,0x94,0xf7,0x0d,0xcd,0xfc,0xf3,0x6d,0x6f, +0xd4,0x1f,0x77,0x2b,0x87,0xa1,0x98,0xc9,0xf5,0xfd,0xc7,0xac, +0x65,0x3e,0x7e,0x5c,0x45,0x45,0xed,0x8f,0x1f,0x5f,0x7c,0x6c, +0x7f,0xb4,0xef,0xb2,0xed,0xd1,0x6f,0x89,0x26,0x3f,0x76,0xdb, +0xb2,0x7b,0xb1,0xf2,0x90,0xb5,0x52,0x8a,0x86,0x71,0x2b,0x5a, +0x64,0xb3,0x39,0x6f,0xfa,0x65,0x88,0xcb,0x0a,0xca,0xb7,0xa4, +0x53,0x9a,0x4d,0x82,0x6f,0xd6,0x22,0x13,0x98,0xfe,0xd0,0x20, +0x2b,0x3a,0x2f,0x12,0x4e,0xcf,0x86,0xb9,0xd6,0x0b,0x0d,0x12, +0x7b,0x5d,0x98,0x85,0x78,0x3f,0x6a,0xfa,0x2c,0xdb,0x1e,0x9a, +0x72,0x4d,0x77,0xbf,0xee,0x0e,0x9e,0xfc,0x4d,0xbe,0x5b,0xd8, +0xcd,0xba,0xb5,0x9e,0x93,0x2a,0x43,0x22,0x10,0x04,0x41,0x25, +0xb9,0xb7,0x5f,0x68,0x01,0x88,0xbf,0x04,0xfd,0x17,0x5b,0xf5, +0xba,0xb5,0x88,0xe5,0xed,0xef,0x84,0xb8,0x1a,0x07,0x67,0x09, +0xdb,0x30,0x26,0xd5,0x7f,0xac,0x22,0x05,0x64,0x65,0x5d,0xff, +0x13,0x2a,0x9b,0xe6,0x55,0xf6,0x72,0xcd,0x27,0x7c,0x39,0x85, +0x7d,0xe4,0x1e,0x70,0xa8,0x67,0x75,0xe4,0x90,0xf3,0x25,0x99, +0x84,0xce,0x47,0xe8,0xbb,0x75,0x36,0x69,0xc1,0x22,0x3e,0xd9, +0x56,0x0f,0xc4,0x48,0x89,0x91,0x32,0x4e,0x5b,0x19,0x66,0x10, +0x33,0xf7,0x9b,0x67,0xd3,0xc6,0x66,0x22,0x29,0xdb,0x98,0xb2, +0x1d,0xa4,0xa0,0x83,0x9f,0x7c,0x92,0x3a,0x6f,0xc6,0x64,0xf6, +0x43,0xe0,0x68,0xab,0x1f,0xf7,0x36,0x07,0xa5,0x65,0x92,0x03, +0xef,0x8c,0x3e,0x40,0x91,0x0f,0x52,0x64,0xdb,0xf1,0x17,0xe6, +0x6e,0x9c,0x59,0x72,0x6f,0x03,0xb3,0x48,0x70,0x5b,0x34,0x18, +0xcd,0xf3,0x1a,0xd5,0xdb,0x66,0xdc,0x59,0xe6,0xa0,0x5c,0xee, +0xea,0x85,0x53,0xf0,0x65,0xb7,0xbe,0x65,0x90,0x27,0x52,0x2c, +0xa5,0x82,0x44,0x18,0xb1,0xed,0xae,0xf9,0x84,0xf8,0x74,0xd8, +0xf1,0x72,0x21,0xc0,0xa5,0x13,0xca,0xcc,0x19,0x82,0xb1,0x2a, +0xf4,0xad,0x80,0x92,0xa5,0xc9,0x3a,0x8c,0x8b,0xc4,0x5a,0x5f, +0x73,0xdd,0x13,0x54,0xa1,0x20,0x8f,0xa3,0x14,0xd7,0x05,0xe9, +0x60,0x4d,0x9e,0xff,0x5a,0x91,0x15,0xa8,0xc1,0x58,0xd1,0xcf, +0x9c,0xf3,0xf3,0xfb,0x3f,0xe6,0x5f,0xb2,0x8a,0xc7,0xf8,0x61, +0x9b,0x70,0xe1,0x22,0x77,0xab,0xbe,0x10,0x08,0x03,0x85,0x76, +0xc3,0x04,0xbb,0x55,0xb6,0x98,0xfd,0x9b,0x20,0x5f,0x88,0x9b, +0xa3,0xef,0x39,0x6c,0xa6,0x8e,0x9f,0xbe,0x9c,0x6c,0x13,0x52, +0xf7,0x88,0x27,0x2b,0x64,0x71,0x65,0x41,0xc4,0x58,0x55,0x3a, +0x9b,0x8e,0x7a,0x1f,0xf3,0xf7,0xe4,0xe1,0x0c,0x51,0x8b,0x36, +0xd0,0xb1,0x89,0xeb,0x04,0x25,0x08,0x87,0x8d,0x01,0x49,0x26, +0xda,0xd1,0x92,0xb2,0x37,0x7c,0x80,0xf4,0x20,0x1b,0x89,0x30, +0x9d,0x77,0x10,0x44,0x2e,0xd0,0x39,0xa1,0x5d,0x0b,0xb2,0x38, +0xa1,0xc7,0x21,0xf3,0x86,0xf4,0xf0,0xd6,0x72,0x3c,0xbc,0xb3, +0xbe,0x05,0x0c,0x19,0x09,0x08,0x32,0xe3,0xfe,0x44,0xd5,0x5e, +0xaf,0xaf,0x05,0xc9,0xf2,0x2a,0x6b,0xb5,0x79,0x45,0xa4,0x42, +0x8f,0xdd,0x8a,0xe2,0x0f,0x18,0x43,0x44,0xcb,0x16,0x71,0x61, +0xe4,0xb2,0x68,0x3e,0xc2,0xdd,0x62,0x1a,0x35,0x02,0x68,0xd0, +0xbb,0x8e,0xab,0x40,0x3b,0xd6,0x5a,0xd8,0xf2,0xf9,0x70,0x98, +0x65,0x76,0xf6,0x94,0xbd,0x3f,0x14,0x08,0x49,0x6c,0x3a,0xe3, +0x34,0x72,0x6c,0x29,0xad,0x9a,0x64,0xee,0x79,0x97,0x75,0xe5, +0x5f,0xdc,0x04,0x48,0x21,0x17,0xd1,0x0e,0x44,0x4c,0x77,0x10, +0xab,0xa4,0x4c,0x22,0xac,0xc0,0x2f,0x11,0xbb,0x3e,0x93,0xc1, +0x79,0xd4,0x8e,0x27,0xf0,0x5a,0x01,0x16,0xf2,0xdb,0x4b,0xfd, +0xc2,0x91,0x1d,0x69,0x15,0x77,0x2b,0xd3,0x8a,0xed,0xca,0xb4, +0xb8,0x5f,0x31,0x43,0xe7,0xf5,0xa6,0x88,0x76,0x41,0xd2,0xa6, +0xa1,0x02,0x41,0xdb,0x5b,0xcd,0x09,0x9c,0xc3,0xa0,0x93,0xa4, +0x6a,0xb7,0x72,0xea,0xe8,0xca,0x97,0x11,0x61,0x7d,0xb1,0x40, +0x17,0xb6,0xa4,0x3a,0x29,0x97,0x91,0xe8,0x69,0x89,0x9d,0xbc, +0xd0,0xfe,0xee,0xf8,0xcb,0xcd,0xf9,0x78,0x98,0xc5,0xeb,0xca, +0x35,0x98,0x48,0x4c,0x22,0xd6,0xc4,0x11,0x3e,0xd3,0xd5,0x3b, +0x92,0xa6,0x2a,0x89,0x56,0x49,0x40,0xd2,0x10,0x30,0x41,0x2e, +0x17,0x4d,0x77,0x7f,0xa5,0xbb,0xa3,0x2c,0xba,0xbf,0x8f,0xc8, +0xf4,0xb8,0x85,0x05,0x54,0xbf,0x5c,0x46,0xe8,0x32,0x40,0x0a, +0x76,0x06,0x50,0x6e,0x86,0xdb,0x4b,0x6f,0x40,0xcc,0xaa,0x00, +0xda,0x33,0x31,0x77,0x01,0xe4,0xbc,0xdb,0x9c,0xdd,0x9f,0xdd, +0x6f,0x5e,0x19,0x05,0x38,0xf1,0x89,0xcb,0x25,0x24,0xb9,0x0a, +0x60,0xf8,0x0b,0xa6,0x67,0xb2,0xcd,0x9f,0xd9,0x18,0xcc,0xe9, +0xce,0xc6,0x5f,0xbb,0x9b,0xce,0xae,0x37,0x6f,0x23,0xb5,0x95, +0xea,0xb2,0x24,0x12,0xba,0x90,0xbd,0x80,0xc0,0x90,0x0c,0xb7, +0x90,0x25,0x17,0x8b,0xc1,0xe1,0xd1,0xde,0xab,0x37,0x7f,0x96, +0x0d,0x2b,0x6a,0xc9,0x25,0x81,0xe1,0x6b,0x7b,0x1a,0x58,0x70, +0x82,0xa9,0x5e,0xf0,0xa3,0xc5,0xeb,0xdd,0x77,0x27,0x47,0xf4, +0xa0,0x13,0xed,0x3a,0x9f,0x1d,0x37,0x92,0x18,0x56,0xe9,0xde, +0x14,0xad,0x57,0x9f,0x7d,0x42,0x53,0xd4,0xd6,0xae,0xb3,0xb5, +0x27,0x5b,0x70,0xc3,0x73,0xe1,0x9d,0xf5,0x26,0xf7,0xde,0x02, +0x1d,0x9d,0x90,0xc8,0x8d,0xac,0xaa,0x91,0x37,0x97,0xa9,0x8a, +0x91,0x6d,0xa1,0x02,0xf3,0x80,0xfd,0x55,0xeb,0x21,0xa3,0xca, +0xfc,0xa9,0x78,0x58,0x2a,0x19,0x49,0xc8,0xdd,0x3b,0xc4,0x6a, +0x8f,0xc7,0x62,0xdf,0x49,0xf9,0x1c,0x66,0x52,0xb9,0x1c,0xf0, +0x7b,0x63,0xf7,0xc2,0xdd,0x03,0x22,0xbf,0xff,0xa4,0xde,0xe1, +0xe3,0x84,0x95,0xba,0xee,0x4c,0x34,0x48,0xf7,0xd6,0x55,0xc1, +0x76,0xef,0x65,0x8d,0x07,0xa2,0x14,0xd7,0x4b,0x5a,0x3a,0xba, +0xa3,0x3b,0xba,0x03,0x55,0x02,0xf9,0x36,0x9f,0xa1,0x0d,0x09, +0xfd,0x40,0x2a,0x68,0xdc,0x1a,0xea,0x22,0x48,0x2e,0xeb,0x0d, +0xa4,0x90,0x6e,0x9c,0x41,0xd0,0x57,0x41,0xf7,0x15,0xb2,0x79, +0x2f,0xac,0x81,0xbd,0x69,0xef,0x65,0xba,0xdc,0xf3,0xaa,0xf9, +0x74,0x54,0xb5,0xc6,0xf1,0x30,0x5b,0xc7,0x44,0x37,0xde,0x91, +0x7c,0xd0,0x80,0x92,0xbf,0x0b,0xf5,0x34,0x1f,0x0f,0x62,0xf8, +0xe5,0xe0,0x2e,0x39,0x63,0xae,0x41,0xad,0xe3,0xc9,0x4c,0x91, +0xb9,0xdc,0xbc,0xf6,0xa6,0xfc,0xda,0x74,0xad,0xf6,0x6e,0x5c, +0x63,0xb8,0x35,0xae,0xd5,0xfa,0xc6,0xed,0xd7,0xee,0x06,0xbd, +0xda,0xdf,0xb8,0xea,0xbf,0x35,0x23,0xdd,0xe0,0xc5,0x42,0x0d, +0x35,0x11,0xbe,0x6a,0x4a,0x3b,0x7c,0x8f,0xe6,0x53,0xba,0x59, +0xc1,0xdd,0xb4,0x23,0xbb,0x65,0x60,0xa5,0xde,0xee,0x28,0x1f, +0xf3,0x0c,0xe3,0x88,0xba,0x40,0xec,0x81,0x93,0x31,0xe7,0x99, +0x34,0xb4,0x84,0xbf,0x57,0x58,0xb4,0x56,0xd5,0x45,0xbe,0x67, +0xfc,0x04,0x83,0x60,0x79,0x4e,0x69,0x87,0x9a,0xe2,0x19,0x94, +0xfc,0xc8,0x38,0x7f,0xa0,0x49,0xd0,0x3e,0x7a,0x5d,0xfa,0x9a, +0x6f,0x8d,0x62,0xb9,0x3d,0xe2,0x59,0x28,0x32,0xa3,0x19,0x18, +0xca,0xd3,0x57,0xfe,0x0c,0x24,0x5f,0x27,0x76,0x56,0xd1,0x39, +0xc2,0x7a,0xd3,0x3a,0x1b,0x45,0x89,0x76,0x65,0x39,0x42,0xf5, +0x00,0xbc,0x2b,0x54,0x1e,0xb1,0x52,0xc8,0x02,0x55,0x28,0x5f, +0xb7,0xa4,0x24,0x40,0x99,0x9c,0x19,0xad,0x41,0xc2,0x76,0x0d, +0xb2,0x52,0x1a,0x6b,0xef,0x92,0xdc,0x14,0x33,0x7e,0xb4,0xf4, +0x75,0xd4,0x77,0x23,0x8b,0x4c,0x8d,0xf4,0x7e,0x52,0x15,0x9b, +0x91,0x57,0xcf,0x44,0x1a,0x95,0x48,0x92,0xef,0x10,0x69,0xca, +0xef,0x79,0x1a,0x06,0x3d,0x22,0x01,0xe0,0xa3,0x99,0xe3,0xa6, +0x36,0x43,0x47,0x3e,0xbc,0xe5,0xc2,0x92,0x85,0xdf,0x60,0xdd, +0xf0,0xb0,0xd0,0xe0,0xbd,0x07,0x5c,0x39,0x1e,0x09,0xf1,0xe6, +0x8e,0xcf,0x76,0xea,0x7b,0x8d,0xa0,0x82,0x94,0x41,0x16,0x04, +0x7d,0x0e,0x75,0x80,0x46,0xda,0x33,0xe8,0x0d,0x00,0x6d,0x71, +0xc4,0x78,0x6f,0xe0,0xae,0x09,0x2f,0x47,0x53,0x65,0xb2,0xfb, +0x95,0x17,0xb0,0xa0,0x41,0xf0,0x27,0xc8,0x9c,0xa4,0x6e,0xb8, +0x2f,0xb5,0xfb,0x4a,0x94,0xa1,0xc8,0x2b,0x07,0xc8,0x20,0x41, +0x7e,0x71,0x46,0xa3,0x56,0x3e,0xe0,0xd7,0xa2,0xae,0x82,0xbe, +0x36,0xe6,0xbe,0x6e,0x0e,0xbe,0xc8,0xb6,0xb7,0x60,0x1b,0xfd, +0x7c,0x6b,0xeb,0x05,0x47,0xa8,0x29,0xf2,0x7a,0x36,0xbb,0x2d, +0xac,0x96,0x7f,0x38,0x3e,0x78,0xf7,0xfe,0xf8,0x64,0xe7,0xe8, +0x44,0x2c,0x37,0x9c,0x9d,0x2e,0xce,0xee,0x4f,0x7f,0x3e,0xbb, +0xef,0x26,0x9b,0x86,0x52,0xf7,0xde,0xbd,0xcc,0x36,0x4f,0xcf, +0x96,0x67,0xdd,0x2e,0xdb,0x70,0x38,0x3c,0x3a,0x38,0x41,0x6b, +0x54,0x90,0xc6,0x4b,0x34,0x16,0x4d,0xce,0xba,0x67,0xcb,0x67, +0xa6,0x73,0x36,0xda,0x34,0xbb,0x07,0xef,0x4e,0xf0,0xdc,0xf9, +0xe4,0x2f,0x87,0x7b,0xef,0x77,0x0e,0x0f,0xdf,0xbe,0xd9,0xdd, +0xa1,0xec,0x08,0x2f,0xbb,0xa7,0x77,0xb1,0x30,0x8b,0x36,0x50, +0xd9,0x04,0x68,0x18,0x27,0xd8,0x80,0x95,0x19,0x37,0x3f,0x4c, +0xc7,0xa3,0x36,0x6a,0x95,0xc1,0xb6,0x3e,0x9b,0xcf,0x2e,0x37, +0x7e,0x1b,0xa1,0x0b,0x13,0x7a,0x58,0x2d,0x42,0xaf,0x0b,0xdd, +0xbb,0x41,0x39,0x92,0xfb,0xdb,0xf4,0xd4,0x3f,0x4c,0xc6,0x5e, +0x2b,0x39,0x5c,0xa0,0x58,0x54,0x90,0xc4,0xbe,0xc6,0x3f,0x4a, +0xd8,0x28,0x34,0x0a,0x05,0x0e,0xe3,0xbb,0x87,0x0f,0xcc,0xa5, +0xbc,0xed,0x97,0x20,0x92,0x81,0x5e,0x4e,0xc6,0x37,0x7f,0x98, +0x5a,0x04,0x12,0x1e,0xbe,0x65,0xd7,0x28,0x5c,0xc9,0x89,0x96, +0x46,0x55,0xe1,0x69,0x1f,0xbd,0x01,0xbc,0x35,0x20,0x26,0xb2, +0xc3,0x94,0x74,0x66,0x63,0x06,0x99,0xa4,0x7d,0x00,0x65,0x67, +0x09,0xba,0x30,0xb8,0x19,0x8f,0xd2,0xfb,0x9d,0x0b,0x64,0x7d, +0xe5,0x8e,0x34,0x35,0x3c,0xf5,0xde,0x84,0x06,0x0e,0xe0,0xfb, +0xbb,0xcd,0xef,0xa0,0x2f,0x51,0x1f,0x3a,0x7d,0x70,0x80,0x0c, +0x7a,0x20,0x7a,0x24,0x07,0x2e,0xed,0x0f,0xe7,0x59,0x9a,0xcf, +0xd3,0xc9,0x25,0x3b,0x84,0xa3,0xcd,0x6e,0xf4,0xe7,0xe3,0xa3, +0x57,0x1b,0x27,0x07,0x7f,0xdc,0x7b,0x17,0x51,0x1a,0x4f,0x50, +0x49,0xdb,0x50,0xa9,0x4b,0x3e,0x9b,0xc7,0x36,0x8d,0x27,0xec, +0x0b,0x00,0xbd,0x09,0xb2,0x1d,0x7d,0x9f,0x42,0x07,0x34,0xf6, +0xe6,0xfe,0xcd,0xca,0x12,0x15,0x39,0xc8,0x27,0x45,0xb0,0x28, +0xa0,0x2a,0xd9,0x8f,0xbd,0x8b,0x8f,0x64,0x8a,0x30,0x5a,0x3f, +0x67,0x77,0x62,0x64,0xcd,0x3e,0x70,0xda,0x52,0xb0,0x67,0xff, +0x0b,0x1b,0xcd,0xf7,0x02,0x9c,0xb2,0x66,0xef,0x97,0x18,0x05, +0xdc,0x58,0xd0,0x26,0x00,0x1c,0x58,0xa5,0xff,0xc5,0x28,0xa9, +0xcf,0x01,0x54,0xa7,0x57,0x04,0x90,0xec,0xd7,0x90,0xfb,0x3a, +0x56,0x52,0x28,0x73,0x2a,0x3b,0x29,0x9c,0x76,0x83,0xf3,0xab, +0x7e,0xab,0x74,0x09,0x9d,0xea,0x03,0xf1,0x9f,0x34,0xe4,0x73, +0x36,0x91,0x2e,0x4a,0x5d,0xd6,0x66,0xa8,0x20,0xd1,0x2c,0xd5, +0xa8,0x9c,0xe3,0x7b,0x6e,0xe5,0x4a,0x0b,0xdb,0xea,0x60,0x44, +0xba,0xfe,0x0b,0x0c,0x10,0x91,0x3c,0x06,0x93,0x82,0x66,0xb0, +0xe5,0xe1,0x17,0xc5,0x46,0xa2,0xf9,0xbb,0xca,0x05,0x97,0x03, +0xc2,0x60,0xda,0x85,0xa5,0xd5,0x8f,0x09,0x47,0xbc,0x1a,0xc9, +0x5c,0x42,0xc9,0xfe,0x55,0x31,0x0d,0xa7,0x74,0x00,0x2e,0xb3, +0x29,0xb0,0x4b,0xc3,0x95,0x62,0x2d,0x0b,0x92,0xed,0xd2,0x72, +0xca,0xbf,0xdd,0x30,0xd5,0x9e,0xf5,0x17,0x32,0xe1,0x35,0x0d, +0x69,0xa5,0xe7,0x6a,0x3e,0x18,0xb7,0x32,0xbd,0x0c,0xa2,0x61, +0xc8,0x55,0xc8,0x15,0x99,0x66,0x96,0x25,0xba,0x91,0x71,0x39, +0xb5,0xaf,0x33,0x19,0x2e,0xdb,0x5f,0x6d,0x05,0xc0,0x67,0xf2, +0x91,0x4d,0x66,0x2f,0x2a,0x46,0xad,0x98,0x02,0xeb,0x86,0x2c, +0x4b,0x25,0x5d,0xd8,0xd9,0xaa,0x11,0xf1,0x25,0xc2,0x78,0x8f, +0x55,0x62,0x8d,0x65,0x48,0xf8,0x0d,0xcc,0x4d,0xe2,0x5b,0x74, +0x6e,0x1a,0x74,0x06,0x39,0x23,0xf3,0x00,0xef,0xab,0xfb,0x46, +0x89,0x40,0x41,0x69,0xd6,0x9f,0x0b,0x3a,0x0e,0x01,0x2a,0x4c, +0xd0,0xdd,0x98,0x2b,0x1b,0x64,0x4c,0xd0,0xf2,0x52,0x65,0x6d, +0xfe,0xd0,0x75,0x45,0x33,0xda,0xbe,0x82,0xd3,0x00,0x9f,0x6e, +0xa6,0xba,0x33,0x4c,0xf1,0x9e,0x83,0x6d,0xd1,0xa5,0xec,0x67, +0xa1,0xa7,0x83,0xc5,0x8e,0x17,0x10,0x37,0xe8,0xc5,0x14,0x53, +0x5e,0x17,0xab,0xf2,0x72,0xd2,0xd2,0x2e,0x27,0x59,0x30,0xe3, +0x42,0xee,0xd2,0xf6,0x3b,0x5d,0x08,0x9a,0x30,0xb1,0x40,0x57, +0x59,0x81,0x3b,0x30,0x89,0x64,0x73,0x34,0xfc,0x56,0x41,0x3a, +0x74,0xbe,0x82,0xcb,0x01,0x2b,0x4b,0xa8,0xb3,0x43,0x60,0x1d, +0xd0,0x2d,0x07,0xf4,0x60,0x3f,0x56,0x6a,0x54,0x1d,0xcb,0x43, +0xad,0xe3,0xdc,0x38,0x39,0x95,0xe4,0x70,0xc9,0x59,0x2c,0x5c, +0xb3,0xc3,0x84,0xae,0x12,0x30,0x5d,0xd5,0x6e,0xb6,0x4f,0x35, +0x34,0x3f,0x3e,0x05,0x68,0x7a,0x48,0x1d,0x0c,0x71,0x35,0x48, +0xc7,0x54,0xd6,0x9b,0xba,0x56,0x2a,0x26,0x26,0x66,0xfb,0x29, +0xec,0xb6,0x36,0xf3,0xe6,0x5f,0x90,0xef,0x66,0x21,0x17,0x96, +0x8c,0x5a,0x2e,0x2c,0x48,0xd6,0x15,0x9c,0x98,0x2a,0x57,0x6c, +0xd8,0xfb,0xfb,0x53,0xe0,0x12,0x7f,0x52,0x58,0x62,0x91,0x7c, +0x88,0x6d,0xf9,0x63,0x24,0xfa,0xa8,0x1b,0xb8,0x7f,0x8d,0xf4, +0xc4,0x90,0x92,0x64,0x66,0xb2,0x8a,0xc3,0x2d,0x43,0x6f,0x83, +0x52,0x3b,0xda,0x49,0xda,0x9d,0xe4,0xb4,0x93,0xc4,0x9d,0x33, +0x89,0x3b,0x3e,0x97,0xeb,0xea,0x72,0xbe,0x15,0x10,0xb2,0x95, +0x45,0xcc,0x14,0x48,0x17,0x3a,0x44,0x51,0xef,0x4b,0x2d,0x62, +0x37,0x67,0xd7,0xb9,0x58,0xbc,0xd7,0x13,0xa3,0x3c,0x8b,0x12, +0x60,0xa1,0xd7,0x20,0x4a,0x65,0xa7,0xc1,0x08,0x7b,0xb3,0x23, +0x5d,0x03,0x3b,0xed,0x9b,0xc1,0x14,0xc8,0xf8,0x97,0xe6,0x27, +0x84,0x2a,0x63,0x4e,0x4c,0xc8,0x1b,0xab,0x23,0x37,0x46,0x7d, +0x2d,0x90,0x28,0xc5,0x40,0x1f,0x9b,0xdc,0xeb,0x50,0x53,0x66, +0xdd,0x62,0x51,0x11,0xb9,0x87,0x3b,0xd5,0x04,0xdf,0x70,0x01, +0x7e,0x4e,0x47,0xb8,0x22,0xa3,0x59,0x59,0xd8,0x14,0xb2,0x73, +0x9b,0x8b,0xb5,0x71,0x6c,0x58,0x9d,0xd7,0xb6,0x2a,0x64,0x33, +0xab,0xcb,0x82,0x84,0xc0,0x85,0x03,0xbd,0x5b,0x1c,0x09,0x54, +0xe8,0xa5,0x14,0xab,0x0c,0xcb,0xf2,0x41,0x31,0xba,0x6d,0xbb, +0x5a,0x7e,0x65,0x14,0x09,0x80,0x2b,0xe2,0xee,0xbe,0x6c,0x26, +0xd1,0xf4,0xaf,0x74,0xcf,0x1f,0x00,0x2a,0xa9,0x86,0x26,0x78, +0x52,0xbe,0x42,0x1e,0x62,0x89,0x66,0x95,0x7c,0x44,0x1b,0x65, +0x86,0xbd,0xb4,0x1f,0x7c,0xb4,0xf0,0x38,0x1a,0xf4,0x3c,0xe3, +0xff,0x65,0x5c,0xd4,0x9b,0x1b,0x3a,0xcc,0x3e,0xbe,0x1e,0x4f, +0x66,0x6c,0x32,0x72,0x1a,0xd3,0x35,0x25,0x91,0xad,0x3b,0xfb, +0x30,0x85,0xcd,0x38,0x89,0xa3,0x72,0xcc,0xe9,0x92,0xd0,0x41, +0xbe,0x65,0x78,0xf6,0x15,0x07,0x89,0xad,0xc1,0x4a,0x42,0x36, +0x2b,0xee,0x79,0x25,0x48,0xe9,0xd4,0x09,0xca,0xa5,0xf0,0x0f, +0x35,0x58,0x82,0xd7,0x40,0x25,0xcc,0x50,0x95,0x85,0xb8,0xe3, +0x3f,0x82,0x21,0x75,0xdd,0x3f,0x80,0x26,0x01,0x48,0x69,0x87, +0x57,0x40,0x78,0x05,0xcf,0x51,0x72,0x87,0x17,0x66,0xfb,0xa8, +0x8f,0x5d,0x18,0x37,0xcb,0x9d,0xc4,0x4d,0x3b,0xbf,0x5c,0x01, +0xe6,0x57,0xda,0xfd,0x77,0xd8,0xc5,0x25,0xec,0xd9,0xa8,0x45, +0xa7,0x45,0x38,0x55,0x67,0x21,0x02,0xb4,0x9b,0xb0,0xcb,0x5b, +0xab,0x14,0x81,0x8b,0x2b,0x4e,0x37,0xba,0xb2,0x3b,0x64,0xe2, +0xf0,0xaa,0xd1,0x02,0x38,0x04,0x11,0xdc,0xb3,0xac,0xdf,0x5e, +0xf7,0x90,0x63,0xe8,0x38,0x3e,0x3c,0x4b,0xb4,0x26,0xd2,0x53, +0xc0,0x27,0xf7,0x1c,0xf6,0x2f,0x66,0x25,0x7d,0x0b,0xf9,0x54, +0xa9,0x0f,0xc8,0x69,0x35,0xb0,0x7c,0x7b,0xdf,0x98,0xaa,0x08, +0xe4,0x04,0x49,0xcc,0xbb,0x9b,0x42,0x6d,0x69,0x58,0x69,0xfa, +0xe0,0xe2,0x9a,0xf2,0xcf,0x32,0x68,0x0b,0xf6,0xdb,0x61,0x6e, +0x9d,0x63,0x5b,0xc7,0xa5,0xfd,0xcf,0x99,0x3d,0xa2,0xe3,0x47, +0x4a,0xb7,0xb9,0xf6,0x9f,0xed,0x26,0x63,0x1b,0x9f,0x78,0x66, +0x90,0xbd,0x5e,0xaf,0xcc,0xe7,0x1e,0x7f,0xf7,0x3f,0x9b,0x56, +0xe2,0xdd,0xa8,0xb2,0x1b,0x52,0x7c,0xe7,0x74,0x6b,0x6c,0x4b, +0x71,0xc9,0xa1,0xef,0xd8,0xcd,0xf0,0xcc,0xf5,0x82,0x44,0x20, +0xd9,0x66,0xe7,0xf3,0xc1,0xb0,0x8f,0x4f,0xa3,0xbc,0x48,0x65, +0xc5,0x87,0x5b,0xf2,0xac,0x47,0x8a,0x83,0xd5,0xf8,0x58,0x25, +0x71,0x12,0xfb,0x02,0x46,0x55,0xec,0x0a,0x53,0x8c,0x80,0x71, +0x93,0x5a,0x08,0x79,0x25,0x4d,0x51,0x18,0x17,0x13,0x95,0x2c, +0xd7,0xab,0xd1,0xef,0xf7,0x4e,0xd8,0x94,0x9b,0x97,0x18,0x51, +0x02,0xc0,0x2c,0xfe,0xf8,0x50,0x17,0xa4,0xab,0x03,0x17,0x4a, +0xfd,0x81,0x4a,0x58,0x5b,0x27,0x0c,0x5b,0xf9,0x98,0x1e,0x84, +0x25,0xdc,0xbd,0x89,0xf5,0x4c,0x4d,0xfd,0x2c,0x9e,0x64,0xf1, +0xb5,0x19,0x4e,0x13,0xe3,0x7d,0xf3,0xf9,0x3c,0x09,0x9d,0xf9, +0xfb,0x30,0xf5,0x4c,0xa2,0x5d,0xc9,0xfa,0xe8,0xa7,0x74,0x98, +0x2f,0xd2,0xb6,0x46,0x70,0x55,0x65,0x9d,0xc2,0x4c,0xf2,0x49, +0x78,0x0f,0xa3,0x42,0x5b,0xe8,0x99,0xfe,0x56,0x97,0x25,0x23, +0xed,0xe9,0xca,0xf2,0x66,0x7b,0x6b,0xcb,0xdc,0x2f,0xed,0x03, +0xa6,0x80,0xbf,0xc8,0xa0,0x57,0x0b,0x96,0x1e,0x39,0x21,0x68, +0x39,0xf5,0x08,0x45,0x7e,0xa4,0x42,0xc7,0x10,0x91,0xf7,0x29, +0xae,0xe8,0x04,0x59,0xf6,0x46,0x6f,0xaa,0x65,0x3d,0x1b,0x6d, +0xa7,0x33,0x3d,0x5a,0x2b,0x2f,0x67,0x8e,0xd2,0x09,0x71,0xa6, +0x6d,0xba,0x95,0x91,0x6f,0x69,0x02,0xa4,0xb6,0x45,0xef,0x7c, +0x56,0xe1,0xad,0xf4,0x98,0x34,0x7e,0x63,0x2e,0x53,0xf9,0x68, +0x97,0xcf,0x02,0xc4,0x82,0xf2,0x1d,0x1b,0xc2,0x74,0x32,0x21, +0xf1,0x6c,0x1e,0x3c,0xbe,0x67,0x5f,0x2c,0xac,0xfd,0x62,0xfa, +0xe9,0xfa,0x57,0x53,0x0c,0xd2,0xd7,0xa0,0xee,0x1b,0xef,0xc8, +0xec,0x71,0x26,0xe7,0x80,0x77,0x6c,0xf8,0xd2,0x4e,0xc8,0x7c, +0x44,0xae,0x2a,0x26,0x83,0xff,0x9c,0xe7,0x13,0x7e,0x75,0xdb, +0x88,0xb2,0xa8,0x51,0x88,0xbf,0x4b,0x58,0x31,0x14,0xfb,0xbf, +0x11,0x6f,0xb4,0xb2,0x0c,0x3e,0xfc,0x5d,0x42,0x27,0x4a,0x3a, +0xf0,0x27,0x8d,0xea,0x51,0xd2,0x60,0xe8,0xac,0x72,0x03,0x61, +0x7b,0x44,0xe1,0xe6,0x48,0x56,0x70,0x4f,0xcc,0x6e,0x21,0x13, +0x53,0x25,0xdb,0x56,0x38,0x80,0xad,0x3a,0xdf,0xab,0x14,0x81, +0x95,0xe7,0xdc,0x32,0x60,0x2f,0xe7,0xda,0xe3,0xdd,0x8a,0x92, +0x9d,0xf0,0x69,0x7f,0x45,0x8e,0xb4,0xe4,0x4d,0xb6,0x22,0x93, +0x7e,0xbc,0xfd,0xd0,0x99,0xe3,0x43,0xad,0x30,0xe2,0x98,0xd1, +0x9e,0xc2,0x21,0x0c,0x10,0x6b,0xff,0x0f,0x60,0xdf,0xae,0xec, +0x3e,0xbb,0xba,0x20,0x56,0x06,0x66,0xbc,0xc5,0x29,0xad,0x10, +0x38,0xdd,0xd9,0x84,0x45,0x3b,0xb6,0x5b,0x1c,0x97,0x27,0xf1, +0x87,0x89,0x24,0xe3,0x3f,0x19,0x8c,0x3a,0x00,0x14,0x40,0xa8, +0xbd,0x6c,0x2a,0x97,0x20,0x3c,0xf3,0xad,0x10,0x53,0x23,0xe8, +0x9d,0xc8,0x44,0xbc,0x03,0x85,0x0f,0x5c,0xbb,0xe1,0x07,0x0f, +0xc1,0x91,0x2c,0x1f,0x10,0x1f,0x23,0x3c,0x10,0x87,0xac,0xc0, +0xe0,0xdc,0x9b,0x62,0x77,0xc3,0x60,0x3f,0x38,0x3e,0xb8,0x6e, +0x7c,0xed,0xd9,0xdb,0xaa,0x9b,0x46,0x75,0x8a,0x6c,0x2f,0x38, +0xb5,0x0b,0x54,0x7f,0x40,0xec,0xce,0x84,0xe5,0xbe,0xda,0x65, +0xf2,0x1a,0x12,0xd4,0x00,0x55,0xa7,0x2f,0xf4,0xe7,0xd7,0x47, +0xee,0x54,0x99,0x25,0x00,0x0b,0xa7,0xd9,0x1b,0x5d,0xcd,0x87, +0xbd,0x09,0x39,0xac,0x3c,0x87,0x72,0x53,0xa3,0x75,0x57,0x5c, +0x36,0x6b,0x49,0x82,0xf5,0x48,0x2f,0xc6,0x43,0x77,0x29,0x02, +0xdc,0x80,0xf5,0x2e,0xba,0x45,0x51,0xfc,0x51,0x54,0x5e,0x12, +0x26,0xbe,0xe6,0x49,0xef,0x93,0xbd,0x05,0x76,0x56,0x42,0x0e, +0xa5,0x3e,0x25,0x0b,0xd3,0x88,0xa1,0x28,0x45,0xd2,0xf5,0x78, +0x94,0xdb,0x07,0x99,0x93,0xc1,0xed,0x2c,0x53,0x40,0x8a,0x9a, +0x37,0x9c,0x03,0x86,0x10,0x0b,0x89,0x9f,0x01,0xed,0x51,0x5e, +0x17,0x3d,0x1f,0xf7,0xbf,0x58,0xfd,0x52,0xd2,0xfb,0xe5,0xc2, +0x5c,0xb6,0x5e,0x27,0x79,0x3c,0x71,0x17,0xb5,0x9c,0xd8,0x24, +0xfd,0xda,0x88,0x2e,0x54,0x3e,0xf4,0xee,0x7a,0x52,0x9f,0x91, +0xd4,0xe9,0xe4,0x02,0xd9,0xa9,0x41,0xa5,0xb8,0x8e,0xc4,0x8d, +0x47,0x80,0x62,0xff,0x0b,0x0a,0x99,0x39,0xec,0x5d,0x47,0x57, +0xb9,0x46,0x68,0x13,0x5f,0x0b,0xe6,0xfd,0x05,0xea,0xcb,0x20, +0xd5,0x6e,0xf2,0x65,0x92,0x94,0xa5,0x92,0xc7,0x58,0x32,0x61, +0x84,0xac,0xeb,0x84,0x64,0x99,0x3a,0xf0,0x08,0x21,0x73,0x21, +0xde,0x52,0xaa,0xbc,0xa6,0xd4,0x66,0xad,0xeb,0xac,0xdb,0x2c, +0x05,0x96,0xc5,0xdb,0x56,0xb5,0x60,0x93,0x1d,0x1d,0x3b,0x9c, +0xee,0xee,0xd3,0xae,0xd3,0xc5,0x05,0x3a,0x58,0x99,0xd5,0xdd, +0x04,0x67,0x97,0xc9,0x1c,0x3b,0x29,0x1f,0xd5,0xea,0x71,0xe0, +0x5f,0x52,0xff,0xbb,0x4f,0x10,0x65,0x3f,0x5f,0x4f,0xea,0x75, +0xf8,0xd3,0xec,0x9d,0xa3,0xd7,0x98,0x82,0x81,0x08,0xec,0x37, +0x0b,0xcc,0xe1,0xf6,0xc8,0x4e,0x8a,0x96,0x7a,0x21,0xbc,0x0c, +0xc9,0x4c,0x34,0x5b,0x8e,0xf7,0x8e,0xfe,0xb4,0x77,0xf4,0x7e, +0x7f,0xe7,0x64,0xf7,0x35,0x3e,0x27,0x8b,0xa2,0x12,0x9d,0x76, +0x13,0x34,0xe7,0x2e,0x8d,0x78,0x03,0x4b,0x7b,0x40,0xec,0x30, +0xcd,0x46,0x17,0xf9,0x30,0x76,0xe9,0x89,0x6f,0x54,0x06,0x4d, +0x60,0x53,0xa6,0xd2,0xe2,0x08,0x9d,0xd2,0x83,0x68,0x6b,0x31, +0xe9,0x58,0x74,0x3b,0x20,0x7c,0xa5,0xdf,0x6f,0x7d,0x6f,0xf7, +0x26,0x92,0xbf,0xb5,0xbd,0xfd,0x9c,0x34,0xc0,0x20,0x00,0x59, +0x5c,0xb2,0x6d,0xf5,0x23,0xfb,0x47,0xcf,0x21,0xd6,0xd7,0x6d, +0xc7,0x1d,0xcc,0x67,0xa8,0xab,0xa1,0xb8,0x6b,0x3c,0x1a,0x8f, +0x6f,0x79,0xb1,0x67,0x70,0xb4,0x13,0xf0,0x25,0x07,0xa3,0x8b, +0x72,0x21,0x7a,0x91,0x10,0x93,0x54,0x81,0x33,0x00,0xb6,0x7f, +0xb6,0x00,0x84,0x20,0x5e,0x98,0xb0,0x3e,0x31,0x94,0xa3,0x5f, +0xd1,0xd7,0x92,0x16,0xbc,0xe9,0x67,0xd1,0xfb,0xa8,0xe1,0xc6, +0x11,0xef,0x1f,0xe6,0xb8,0x5c,0x35,0x1a,0xe8,0xb8,0x4c,0xd6, +0xc8,0xe7,0xbf,0x49,0xda,0x2e,0xc7,0xa9,0x2f,0xab,0x36,0xf6, +0x7c,0xf3,0x5b,0x99,0xa9,0xe9,0xee,0x7d,0x59,0xaf,0xcf,0x8f, +0x8f,0xe6,0x3b,0x9e,0xfd,0xd1,0x3d,0xef,0xee,0xce,0xdb,0xb7, +0x3f,0xee,0xec,0xfe,0x11,0x58,0x61,0x89,0xa1,0x36,0xa3,0x86, +0x07,0xaf,0xd4,0x35,0x1e,0xaa,0xbf,0xb3,0x92,0x72,0x51,0xee, +0x5e,0x5d,0x2e,0x49,0x1f,0x21,0xf9,0xc5,0x62,0x63,0xdb,0x3d, +0x5d,0xa8,0x84,0xb3,0x54,0xba,0xd3,0x44,0x91,0xf9,0xa7,0x1a, +0x30,0xed,0x36,0x4e,0x30,0xd8,0xae,0x07,0x13,0x1e,0x15,0xa5, +0x1e,0x39,0x1c,0xf6,0xea,0xe3,0x3c,0x47,0xa7,0xb9,0x9d,0xda, +0x2c,0xce,0x2b,0x95,0xe9,0x25,0x4d,0xe5,0x47,0x58,0x22,0x50, +0xdb,0xf7,0x19,0x4e,0x15,0xcd,0xfe,0x02,0x71,0xc9,0xde,0x57, +0x61,0x1e,0xf4,0xc4,0x36,0x1c,0x1e,0x85,0x29,0x40,0x70,0xd3, +0x01,0xf6,0x92,0xcd,0x79,0x1a,0x91,0xc0,0xba,0x21,0xfa,0x82, +0xe2,0xbb,0x04,0x4f,0xb2,0xdf,0xe2,0x68,0xf6,0xae,0x72,0x15, +0x45,0x0a,0x09,0x64,0xda,0x7f,0x00,0x35,0xc2,0xd7,0xdb,0xde, +0x74,0xb6,0xb1,0x3f,0xee,0x93,0xc7,0x3e,0x08,0x1f,0xe2,0xab, +0x91,0x5e,0x84,0x02,0x6d,0x50,0x2d,0x08,0xf5,0x45,0x14,0x81, +0x7f,0x38,0x6b,0x3f,0x1a,0xa3,0xe2,0x25,0xa4,0x7a,0x72,0x60, +0x9b,0x15,0xb6,0xc9,0xe6,0x6b,0x7b,0x47,0x31,0x41,0x4d,0x0d, +0xb9,0x71,0x61,0x23,0xa0,0x6c,0xb1,0x88,0xb4,0x87,0x96,0x62, +0x9e,0xf3,0x41,0x92,0xa1,0x71,0x63,0xde,0xc1,0x1d,0xef,0x19, +0x77,0x47,0x47,0xa4,0x41,0x2a,0xbe,0xff,0x2d,0xa0,0x01,0x52, +0x5b,0x71,0x19,0x00,0x5c,0xb1,0x54,0xf1,0xa4,0x1e,0x29,0x4b, +0xd7,0x23,0xd9,0x74,0x54,0x16,0x6e,0xed,0x98,0xb8,0x40,0xd4, +0xc0,0x55,0x08,0xdf,0x1d,0x90,0x83,0x3a,0x61,0xb4,0x2f,0xb6, +0x12,0x3a,0x3b,0xb6,0x6c,0x37,0x0b,0xb8,0x72,0x1c,0xae,0x39, +0x76,0xc5,0x4a,0xd8,0xbe,0x9d,0x84,0xea,0x75,0xf9,0xa0,0xbd, +0x79,0x18,0x2a,0x00,0xd0,0xaf,0xea,0xd6,0xd5,0xcb,0xd7,0x82, +0x5e,0x8d,0xd6,0x18,0x46,0xed,0x5d,0xa5,0xf9,0x8b,0x9a,0xbb, +0xac,0x7d,0xa0,0x33,0x15,0x1f,0xf6,0xe9,0x47,0x1b,0x9d,0x58, +0xe7,0xe4,0xb9,0xc7,0x6e,0xfe,0x55,0xbc,0xa8,0x95,0xfa,0x7a, +0x1e,0x86,0xe8,0xf3,0x69,0x78,0x2e,0xb6,0xa0,0xa4,0xea,0x9c, +0xfb,0x17,0x55,0xc3,0x9d,0x27,0x7f,0x2f,0xcd,0x8a,0xcf,0xfd, +0xb2,0x87,0xfe,0xe9,0x85,0x96,0x05,0xca,0x0f,0xca,0xd1,0x76, +0xcc,0xeb,0xde,0x5d,0xee,0x3d,0x6b,0x98,0x59,0xf0,0x7e,0xd3, +0xbf,0x99,0xa1,0x76,0x93,0x93,0x4a,0xec,0x58,0xfe,0xb8,0x1c, +0xa1,0xa6,0x29,0x6f,0xb2,0xb2,0x2d,0x63,0xb7,0xf3,0x46,0xde, +0x60,0x21,0x00,0xfb,0x0a,0xf1,0xba,0x37,0x0d,0x1e,0xb4,0x66, +0x6b,0x2d,0xc3,0x86,0x31,0x71,0xb3,0x2a,0x8f,0x91,0xd8,0xa9, +0x67,0x42,0x4e,0x32,0x7d,0x7d,0x0c,0xc7,0xee,0x9a,0x55,0xff, +0xcb,0xee,0x4e,0x0c,0xb7,0xc5,0x16,0xad,0x30,0xbf,0xeb,0x5f, +0xe3,0x21,0x36,0x14,0x90,0xb7,0xec,0x52,0x30,0xe9,0xf0,0xae, +0x6c,0x2d,0xf3,0xd9,0xe8,0x95,0x9f,0xdd,0xf2,0x13,0x50,0xd6, +0xe2,0xe3,0x9d,0x23,0x76,0x80,0xcf,0x1b,0x9e,0x0f,0x5c,0x8e, +0xac,0x03,0x0c,0xe8,0x9f,0xac,0x50,0xf4,0x21,0x3c,0x5c,0xdf, +0xa2,0x1e,0xd3,0xe5,0x08,0x9f,0xe9,0x65,0xbe,0x8b,0x6d,0x62, +0xc3,0x35,0xea,0xed,0xaa,0xde,0xdd,0x4a,0x52,0xdb,0x20,0xee, +0xdc,0x47,0x1b,0x43,0xda,0xb7,0x58,0x8b,0x75,0xb2,0x88,0x96, +0xc8,0x65,0x24,0xb9,0xa8,0x24,0x00,0xe7,0x55,0xa0,0x50,0x67, +0xcb,0xbe,0xb9,0x4b,0x0a,0xd4,0x63,0xeb,0x94,0xe1,0x6d,0xd9, +0x77,0xcf,0x8a,0x0e,0xfd,0xdb,0x67,0xa6,0x06,0x5c,0x0b,0x50, +0xaa,0xae,0x91,0x47,0x3c,0xf5,0x66,0x1d,0xf0,0xa4,0xa7,0xdf, +0x67,0xa3,0x63,0xb2,0xca,0x57,0x93,0x3a,0xe6,0xbd,0x61,0x6d, +0x6f,0x7a,0xd1,0xbb,0xc5,0x17,0xe5,0xf6,0x09,0xf4,0x54,0x17, +0xc4,0x27,0xcf,0xb3,0xeb,0x1e,0x29,0xf0,0xa2,0xe1,0x09,0x74, +0x1f,0x58,0xb3,0x6f,0xce,0x6b,0x5e,0x09,0x77,0x5a,0xc3,0xdd, +0x75,0xad,0x57,0x93,0x46,0xf0,0xbc,0xc5,0x97,0xe8,0xd6,0xd3, +0x56,0xb3,0x56,0x3b,0xce,0xf3,0x1a,0xee,0x4c,0xd3,0xcd,0x4d, +0xd8,0xd8,0x4d,0xed,0x86,0xef,0xc3,0xb4,0x39,0x9e,0x5c,0x6d, +0x02,0x16,0x9b,0xa3,0xab,0x26,0xcf,0x4f,0x9a,0x3f,0xb2,0xa5, +0x59,0x2b,0xcf,0xb4,0xc5,0xa2,0x38,0x6a,0x9d,0x98,0x31,0xb4, +0x6f,0x18,0x65,0xea,0x00,0x31,0x85,0x66,0xb2,0x70,0x5a,0xd2, +0x43,0x12,0x99,0x9a,0xd8,0xc9,0xd6,0x68,0x62,0xf1,0x79,0x6e, +0x64,0x8b,0x82,0x38,0xc8,0x8a,0xd0,0x34,0x78,0x3c,0xa2,0xe8, +0x86,0x9e,0x15,0x51,0x39,0xc2,0x81,0x27,0x5a,0xce,0xc2,0xc1, +0xec,0x14,0x6c,0x7e,0xc5,0x61,0x32,0x15,0x49,0x52,0xca,0x44, +0x1d,0x77,0xc0,0x35,0xb1,0xe2,0x4e,0x46,0xf0,0x17,0x0b,0x82, +0xab,0x1e,0xeb,0x50,0x38,0x8a,0xd0,0x46,0x1e,0x52,0xa2,0xd7, +0xd6,0xbe,0x15,0xb7,0x76,0x84,0x07,0x9f,0xc1,0x11,0x34,0x5a, +0x60,0xb1,0x93,0xf0,0xa1,0x31,0xc6,0xf8,0x47,0x10,0xd4,0x75, +0x7c,0x80,0x16,0xb9,0xb7,0x35,0x13,0x59,0xed,0x41,0xee,0x02, +0xba,0xca,0xaa,0x48,0x8f,0xa2,0x26,0xac,0xee,0xfd,0x6c,0xa6, +0xcd,0x24,0x08,0xc1,0x91,0xf5,0x2c,0x6a,0x23,0xe4,0xf3,0x82, +0x71,0x92,0xb4,0xcb,0xef,0x76,0xb8,0x1e,0x5c,0x99,0x65,0xf6, +0x52,0x39,0xf8,0x66,0xe6,0x48,0x7f,0xf1,0xfd,0x8e,0xb5,0xec, +0x50,0x58,0xb9,0x78,0x3e,0xeb,0x77,0x27,0x96,0x81,0x16,0xa6, +0xbc,0x7f,0x87,0x62,0xef,0x73,0xed,0x65,0xda,0x8a,0xb7,0x30, +0xc1,0xce,0xdc,0xa9,0xaf,0xfa,0xa5,0xac,0xfa,0x75,0x4c,0x45, +0x29,0xb5,0x60,0xe9,0x32,0xc1,0x51,0xcd,0x5b,0xd4,0x5e,0xcf, +0x2b,0x4f,0x69,0x4a,0x00,0xef,0x07,0xfd,0x34,0xca,0x47,0x1b, +0x73,0x90,0xfe,0xde,0xfd,0xb4,0xff,0x23,0xec,0x08,0x5f,0x1d, +0x1c,0xc1,0xa6,0xf0,0x38,0xbd,0x7f,0xb9,0xb7,0xfb,0x66,0x7f, +0xe7,0xed,0xfb,0xe3,0xbd,0xc3,0x34,0x6a,0x46,0xe6,0xf7,0x47, +0x07,0x3f,0x1d,0x72,0xc8,0x44,0xe6,0x70,0xe7,0xe4,0x64,0xef, +0xe8,0xdd,0x71,0x7a,0x7a,0x7f,0x33,0x18,0xc1,0x54,0x4a,0x5b, +0x06,0x3e,0x5e,0x4d,0x7a,0x17,0xe9,0x16,0xda,0x43,0xa5,0xaf, +0xe7,0xb8,0x91,0x3e,0x9c,0xe4,0x29,0x88,0x87,0xf0,0x75,0x3c, +0xbf,0xc4,0xaf,0x51,0x7e,0x45,0x71,0x1b,0xf4,0x29,0x91,0x57, +0xc7,0xf8,0x8a,0xf7,0xb9,0x19,0xca,0x07,0x5e,0x41,0x16,0x01, +0x6f,0x3b,0xc0,0xdb,0x0e,0xf0,0xff,0xfe,0xbf,0xab,0x40,0xc7, +0x18,0x6d,0x81,0x27,0x15,0xd0,0xbb,0x66,0xf7,0xa7,0xa3,0xa3, +0xbd,0x77,0xbb,0x7f,0x79,0x7f,0xfc,0x97,0x7d,0x34,0xf1,0xb8, +0x34,0x2f,0x77,0x4e,0xf6,0x4e,0xde,0xec,0xef,0xf9,0x3e,0xd8, +0x3f,0x78,0x77,0xf2,0x3a,0x8d,0xfe,0xd0,0x1b,0xcd,0x7b,0x93, +0x2f,0xe6,0x55,0x7e,0x3e,0xa1,0x8f,0xfd,0xde,0xe4,0xe2,0xda, +0xec,0xdc,0x4e,0x06,0x43,0xf8,0xfe,0x62,0xfe,0x30,0x1f,0xe5, +0xf0,0x67,0xf8,0xc5,0xec,0xcc,0xaf,0x60,0x66,0x9a,0x63,0xa0, +0xcc,0x1c,0xfd,0x87,0x98,0x83,0x8b,0xd9,0x18,0x7f,0xdf,0x8d, +0xef,0x38,0xe2,0x65,0x7e,0x41,0x1f,0x91,0x55,0x7d,0x37,0xc0, +0xc5,0x8f,0x5f,0x1f,0x1c,0x9d,0xf8,0xda,0xb0,0x26,0xac,0x04, +0xab,0xb0,0x15,0x20,0x7c,0x04,0x8f,0xb0,0x11,0x2a,0x42,0x44, +0x60,0x01,0x9c,0x97,0x3b,0x7f,0x49,0xa3,0x63,0x98,0xe3,0x50, +0x66,0x7f,0x4c,0x3f,0x27,0x20,0xc7,0xe1,0xef,0x7f,0xe5,0xfd, +0x11,0x7f,0x9d,0x5c,0xcf,0x27,0xf4,0xf1,0x6a,0x32,0xc0,0x9f, +0x63,0x90,0x87,0x27,0xf0,0x51,0xc6,0xc8,0x82,0x43,0x58,0x08, +0x08,0x81,0x60,0x71,0x2c,0x89,0xc5,0x82,0x12,0x3b,0xfb,0x87, +0xfb,0x40,0x0f,0xd1,0xce,0x3e,0x6e,0x1d,0xf6,0xa3,0xae,0xb9, +0xc9,0xfb,0x83,0xf9,0x4d,0x1a,0xed,0xef,0xef,0xd7,0xfa,0xa6, +0xf6,0xa5,0x76,0x9d,0xde,0xdc,0xa4,0xd3,0x69,0xad,0x07,0x39, +0xa6,0x78,0x3e,0x09,0xdc,0x67,0x7f,0xb3,0xbf,0xf9,0x85,0x93, +0x30,0xfe,0x12,0x38,0xd6,0x4b,0x9c,0xff,0xd1,0x1e,0xfc,0x67, +0x6a,0xfb,0xb6,0x30,0x1e,0x50,0x8c,0xae,0x38,0x49,0x45,0x72, +0x1d,0x2e,0x5a,0x62,0x09,0xb8,0x44,0x12,0x7c,0x9b,0xf1,0x64, +0x80,0x0a,0xb8,0x0a,0x0f,0xca,0xe9,0x63,0x21,0x6a,0x69,0x6e, +0x87,0xf3,0x49,0x6f,0xb8,0xdb,0x53,0x4f,0xad,0x47,0xf3,0x1b, +0x37,0x0b,0x5b,0xf8,0x5c,0x74,0x7e,0xd3,0x89,0x60,0x2b,0x0d, +0x0d,0x18,0x83,0xdc,0x3c,0x01,0x31,0x57,0x89,0xca,0x7c,0x17, +0x71,0xd8,0x9b,0x5d,0x03,0xe7,0x84,0x65,0xdb,0x4b,0x72,0x39, +0x3d,0xc3,0x47,0x66,0x34,0xbb,0xb6,0x9d,0xb7,0x19,0xa1,0xdd, +0x5d,0x9b,0xe4,0x5e,0x4d,0x6f,0x6c,0xb4,0x13,0x1b,0x89,0x1c, +0xd7,0x5d,0x70,0x1c,0x73,0x64,0xac,0x12,0xdd,0x0d,0x95,0x83, +0xc2,0xcc,0x18,0x40,0x2b,0x25,0x53,0x64,0xcb,0xf6,0x86,0x68, +0x7c,0xfe,0x41,0x79,0xc4,0xcf,0xf4,0x79,0x10,0xbb,0x5b,0xc6, +0x8b,0xbf,0x36,0xe4,0xc2,0x9b,0x6e,0x7b,0x80,0xe4,0x1e,0x3e, +0x71,0xfc,0x35,0x3a,0x3e,0x75,0x0f,0x9f,0x24,0x2f,0x74,0x27, +0x9a,0x14,0x11,0x07,0xc9,0xbf,0xee,0x82,0xcc,0xf2,0x72,0xef, +0xd5,0xce,0x4f,0x6f,0x4f,0xde,0x1f,0x02,0x49,0x1d,0x9f,0x5a, +0x20,0xf2,0x02,0xa1,0x80,0xdf,0xce,0xed,0xed,0x0a,0x14,0x81, +0xe5,0xbc,0x5e,0x85,0x20,0xf4,0x66,0x56,0xe1,0xc3,0xdc,0x7b, +0x80,0xe6,0x7c,0xd3,0x1c,0x27,0x2e,0xbf,0x98,0xf8,0x63,0xfe, +0x85,0xb4,0xd8,0xbc,0x4f,0x6c,0xdb,0xaa,0xde,0xf4,0x01,0x60, +0xbf,0xee,0xd2,0x8e,0xcc,0xf8,0x8a,0xeb,0x75,0xe8,0xe6,0xb5, +0xcc,0x47,0x04,0xe6,0x75,0x15,0x82,0x90,0xad,0xe1,0x83,0x6a, +0x60,0xce,0xf3,0xab,0xc1,0x88,0x0e,0xe9,0x63,0xfa,0x34,0x9f, +0xae,0xc7,0xc3,0xdc,0x10,0x61,0x7d,0x1a,0xa8,0x9b,0x84,0xad, +0x2c,0xa3,0x24,0x27,0x61,0x53,0xf6,0xa4,0xc3,0x91,0xf2,0xce, +0x85,0xe2,0xac,0xc0,0x98,0x3a,0x20,0x4a,0x5b,0x03,0xd6,0xd0, +0xdb,0xd7,0xd0,0x4a,0xea,0x42,0x79,0x3f,0x87,0xd2,0x67,0x70, +0x43,0xf6,0x6f,0x91,0x25,0x2a,0xbc,0x3c,0xa3,0xe8,0x8e,0x28, +0x82,0xa8,0xd7,0x36,0x24,0xbd,0x16,0x40,0x93,0x1e,0x3f,0x81, +0xf6,0x77,0x8e,0xbe,0x48,0x58,0x7b,0xe8,0x7f,0x1f,0xc8,0xb5, +0xd1,0x0a,0xf4,0x4a,0x50,0xf3,0xec,0x47,0x3c,0x45,0xab,0x86, +0x87,0xb2,0xc0,0x96,0x09,0xf0,0xde,0x8c,0x0a,0x61,0x04,0xba, +0xad,0x5f,0x53,0xbe,0x95,0x83,0xce,0xd7,0xb3,0x9b,0xe1,0xaf, +0x91,0xd8,0x7a,0xb7,0xb7,0x58,0x87,0x39,0x87,0x3f,0xb0,0x7c, +0xa0,0x21,0x6c,0x6b,0x9c,0x02,0xad,0x53,0xfe,0x1a,0x2d,0xdb, +0xf8,0xb4,0xcc,0x7f,0x3a,0x0b,0x02,0x02,0xe1,0xdd,0x18,0xdb, +0x9e,0xf9,0x5e,0x90,0xf8,0x44,0xf4,0xef,0x79,0x63,0x12,0x68, +0xe5,0x38,0x2b,0x60,0xf2,0x5e,0x2a,0x98,0xa4,0xf2,0xe2,0x48, +0x2e,0x6c,0x67,0x18,0x9f,0x29,0x62,0x09,0x6a,0x35,0x34,0x1f, +0xf0,0xc2,0xd7,0x5d,0xb1,0x49,0x11,0x67,0xf3,0xc8,0x9e,0xf0, +0x3a,0xe9,0xeb,0x76,0x76,0x7d,0x3b,0xb9,0xfc,0x1c,0x99,0x67, +0xd6,0x51,0x1f,0x00,0xa9,0x45,0x64,0xe5,0xa9,0x76,0x33,0x00, +0x59,0x19,0x04,0x7b,0x84,0x52,0xbb,0xa5,0xf6,0x42,0x12,0xc8, +0x63,0xcd,0x67,0x74,0x64,0x16,0xd4,0x9e,0xb4,0xf5,0xec,0x95, +0x8a,0x2d,0xfe,0x46,0xf4,0x8b,0xe8,0xb5,0xb4,0x8d,0x73,0x3d, +0x82,0xc6,0x59,0x63,0x15,0xca,0x88,0x21,0x4a,0x04,0x9e,0xe8, +0x8c,0xa7,0x78,0x41,0x51,0x88,0xc9,0x0a,0x3e,0x1e,0x65,0x6a, +0xcf,0xc6,0x6e,0x5e,0x4b,0x7e,0x4e,0x48,0x0c,0x4d,0x6b,0x3b, +0xaa,0xf0,0xdd,0x01,0x0a,0x6f,0x94,0x78,0xaa,0xca,0x90,0xa4, +0xce,0xe2,0x03,0xaa,0xe9,0x66,0x8a,0xa9,0x2b,0x6c,0x93,0x46, +0xcc,0x35,0xe0,0x75,0x72,0x83,0x3f,0xc9,0x02,0x35,0x82,0xb0, +0x38,0xf7,0xce,0xa7,0x38,0x74,0x41,0x8f,0x35,0x3c,0x68,0xef, +0x23,0xd9,0xb5,0x72,0x92,0xd3,0xe3,0xfb,0x0a,0x52,0xe9,0x51, +0x17,0x93,0xa9,0x3c,0xee,0x6d,0x99,0xa5,0x31,0x27,0x54,0xd0, +0x07,0x51,0x46,0xb2,0xa6,0x65,0xff,0xd8,0x17,0xcf,0x04,0x60, +0x45,0x79,0x4f,0xe8,0x86,0x13,0x0b,0x50,0xc2,0xf6,0xc4,0xaa, +0x24,0xce,0x42,0x29,0xb2,0x58,0xc8,0x47,0x2a,0xd9,0x1b,0xbe, +0x6e,0xd8,0x28,0xaf,0xc4,0x5a,0x51,0xf5,0x43,0xb5,0x72,0xf9, +0x34,0x9c,0x80,0xc8,0xc9,0x1a,0x80,0x43,0x98,0xd7,0xca,0xfd, +0x15,0x9c,0x00,0xc6,0xea,0x1c,0xb6,0x93,0x9a,0x19,0xe0,0xf8, +0x59,0x66,0xf0,0xa4,0x29,0xee,0x66,0xae,0x85,0x40,0x87,0x4c, +0xe6,0x91,0x99,0x8f,0xc7,0x85,0xe3,0xf9,0x0c,0x0b,0x3c,0x30, +0x7a,0x8b,0xc5,0x83,0x1d,0x64,0x04,0x0a,0xb6,0x03,0xa1,0x00, +0x69,0xc3,0x4a,0x11,0x40,0xf6,0x8b,0x52,0x47,0x81,0xf2,0x8d, +0x34,0x61,0x76,0x67,0xa7,0x81,0x18,0x60,0x27,0x4c,0xc4,0x89, +0x11,0x30,0x9a,0xb0,0xfa,0xd5,0xfc,0xe6,0x7a,0xfa,0x04,0x7e, +0x83,0x38,0x55,0xf1,0x1b,0x35,0x20,0x01,0xb3,0x09,0x2b,0x0f, +0x7b,0xfd,0xff,0x8f,0xec,0xa3,0x11,0x7b,0xa8,0x1d,0xdf,0x29, +0x8a,0x9f,0x20,0x80,0x47,0x38,0x89,0xdb,0x86,0xda,0xc5,0xd8, +0x92,0x71,0x96,0x85,0x0b,0x34,0xad,0xf9,0x8f,0x4e,0x9c,0x37, +0xe5,0xc5,0x54,0xcf,0x9f,0xe2,0x62,0x5a,0x31,0xef,0xe4,0xe1, +0x31,0x2d,0x11,0xfe,0xe1,0xf1,0x57,0x2d,0xae,0x8f,0xf1,0x4c, +0x2b,0x23,0x4b,0xb9,0xca,0x96,0x3e,0x8d,0x1f,0x59,0xbe,0xd2, +0x50,0xdd,0x5f,0xcd,0x88,0x9e,0xca,0x89,0xec,0x54,0x11,0x6d, +0xcc,0x5b,0xb1,0x13,0x50,0xf5,0x58,0xdb,0xda,0xb7,0x40,0x33, +0x2e,0x36,0x5f,0x77,0x25,0xa8,0xe3,0x10,0x20,0xae,0x1b,0xf0, +0x8d,0xfa,0xa4,0x65,0xd8,0xe1,0xd1,0xb9,0x56,0x76,0xe3,0x94, +0x4e,0x58,0x67,0x1a,0x87,0xe1,0xcc,0xc3,0xb6,0x26,0x70,0x8a, +0xd3,0x90,0x67,0xe7,0xb2,0x70,0x5c,0x21,0xb7,0xce,0xfa,0x46, +0xc1,0x77,0x2c,0x5e,0x2d,0x11,0xe1,0xec,0xa3,0x51,0xe7,0xb5, +0x16,0x8f,0xb6,0x4a,0x77,0xe8,0xdf,0x0a,0xb5,0x39,0xfc,0xad, +0xd2,0xa2,0x24,0x74,0x14,0xbb,0xcb,0x38,0x4e,0x2e,0x02,0x7c, +0xbc,0x4c,0x1b,0x5f,0xa1,0xd7,0x0e,0x20,0x3b,0x81,0x25,0xd8, +0x14,0x0d,0x90,0x5d,0x81,0x1b,0x32,0x2a,0xce,0x60,0x6d,0x64, +0xe1,0x7e,0x21,0x78,0x55,0xa9,0xf4,0x66,0xa6,0x23,0xb2,0x07, +0x65,0x1f,0x5e,0x8a,0xb6,0x87,0xbe,0x6d,0x50,0x0f,0x27,0x9d, +0x1a,0x8d,0x94,0x0a,0xec,0x41,0xab,0x9b,0x07,0x32,0x4e,0x6c, +0x7b,0x79,0x97,0x6e,0x21,0xe3,0x31,0x69,0x0a,0x8a,0x59,0x40, +0xd1,0x5d,0x06,0x78,0xbd,0xfe,0x05,0x48,0xe3,0x71,0xe4,0x18, +0x37,0x67,0x17,0xfd,0xe3,0xc8,0xb8,0xf8,0x26,0x73,0x24,0x18, +0x4f,0x81,0xc4,0x06,0x7b,0x9c,0x3f,0x1c,0x5b,0xdd,0x3e,0x59, +0x79,0x55,0xd2,0x35,0x5a,0x36,0x72,0x37,0x56,0x4d,0x1b,0x13, +0xe3,0xf1,0xf7,0x00,0xaf,0xcb,0x70,0xea,0x85,0x37,0xea,0x6d, +0xd7,0x89,0x1d,0x3b,0x13,0x33,0xb5,0x2d,0xf0,0xe5,0x80,0x93, +0x5a,0x78,0x20,0x96,0xa3,0x08,0xa9,0xb1,0xc8,0x6c,0x37,0x35, +0x61,0x20,0x50,0x4b,0xac,0x53,0xdc,0x05,0xa4,0xab,0x79,0x1a, +0x8b,0x29,0x5c,0xb5,0x63,0x19,0xaa,0xe6,0xb0,0xa6,0x0a,0xe6, +0x96,0xf8,0xae,0xa3,0x5b,0x68,0x9d,0xdf,0x31,0x4c,0x5c,0x50, +0x14,0xd3,0x54,0xbd,0x2d,0xd2,0x43,0xac,0x63,0x84,0xd9,0x69, +0x34,0x92,0x80,0x04,0x9a,0x40,0x2f,0x11,0xd9,0x13,0x55,0x04, +0x44,0xc6,0x4b,0x59,0x2d,0x94,0x3e,0x9b,0x17,0xb3,0xc9,0x90, +0x8c,0x80,0x49,0xf8,0x26,0x9f,0xf5,0x28,0xbc,0xbd,0x96,0x71, +0xcc,0xa7,0xeb,0xc1,0x85,0x3a,0xc1,0xc8,0x87,0x37,0xd6,0x55, +0x03,0xa7,0xcf,0x80,0x5b,0xe7,0xb3,0xa4,0x1d,0xf5,0xd0,0xdc, +0x95,0x57,0x7b,0x80,0x8c,0xd6,0x32,0x30,0x3f,0x4d,0x44,0x15, +0x63,0x8e,0x04,0xb6,0x18,0x18,0x81,0xdc,0x82,0x3d,0xf5,0x1a, +0xa6,0x65,0xf0,0xcf,0x59,0x41,0x4e,0xd0,0x12,0xbe,0xf8,0xc6, +0x27,0x26,0x7e,0x3e,0x25,0xf2,0xa1,0x3c,0x68,0xa2,0x8c,0x4d, +0x69,0xe1,0xed,0x2a,0x76,0x05,0xec,0x30,0x88,0x7a,0x2a,0xfa, +0x48,0x4a,0x26,0x6d,0xf9,0xc0,0xd6,0x02,0x10,0xb6,0xef,0xc6, +0xf8,0x47,0x09,0xfa,0x5e,0xf4,0x70,0x5c,0x87,0xd0,0x4c,0x47, +0xb5,0xb8,0xc3,0x09,0x45,0xc0,0x94,0xc7,0x6d,0x3d,0x27,0xde, +0x72,0x9c,0xe4,0x88,0x43,0x54,0xd6,0x0a,0x94,0x8c,0x76,0xff, +0x4b,0x43,0xaa,0x0b,0x84,0xef,0x0b,0xe4,0x2d,0x81,0x09,0xb5, +0xdb,0x4e,0xa3,0xcb,0xcb,0x8d,0xdf,0xfc,0xf6,0xfb,0xed,0xad, +0xdf,0x6e,0x84,0xb5,0x47,0x5d,0xbc,0x79,0x42,0x6d,0xc2,0x8a, +0x79,0xba,0x96,0x79,0x3a,0xf1,0x8a,0x3c,0x84,0x57,0xc5,0xa4, +0xc6,0x2b,0x6b,0x97,0x67,0x8c,0xa8,0x09,0xdf,0xf0,0xc7,0x61, +0xf9,0x27,0x62,0x20,0xa8,0x31,0x53,0x51,0x99,0x4e,0x7e,0x1a, +0x8b,0xc1,0xe3,0x70,0xf2,0x9b,0x8b,0x22,0x5c,0x19,0x64,0x62, +0xb5,0x14,0xdd,0x28,0x58,0x7d,0xe6,0x47,0xda,0x92,0x18,0x5e, +0x6a,0xdb,0x8f,0x9a,0xc8,0x42,0x12,0x63,0x6e,0x96,0x95,0xc1, +0xb4,0xcb,0x23,0x27,0x6d,0x34,0x0f,0xf0,0xd7,0xe5,0xe3,0x2f, +0x46,0xfa,0x83,0x2b,0xd2,0x0d,0x83,0x61,0x63,0x53,0x84,0x04, +0x60,0x97,0x75,0x81,0xb2,0x2d,0x77,0xb3,0xa0,0x8a,0xb0,0xa1, +0xdc,0x15,0x98,0x87,0xda,0x48,0x62,0x4e,0x50,0xdc,0x3f,0x17, +0xe6,0x06,0xc5,0xb9,0x2b,0x1c,0x5d,0x6f,0xbd,0x2e,0xe0,0x2a, +0x7a,0x02,0xf6,0xea,0x41,0xde,0x46,0xe3,0x71,0xeb,0x63,0x5f, +0x45,0x03,0xab,0x17,0x99,0x12,0x0d,0x74,0xca,0xa3,0x22,0x59, +0xd3,0xf8,0x31,0x1a,0x0f,0xbb,0xe6,0xc1,0x61,0x24,0xd5,0x92, +0x8a,0xbe,0xa3,0x7b,0x74,0xdd,0x17,0x7a,0xea,0x15,0xee,0x64, +0xae,0x0a,0xf2,0x4d,0x3f,0x3f,0x9f,0x5f,0xa1,0x28,0x3c,0xcd, +0x87,0x97,0xb4,0xaf,0x68,0x8b,0xb9,0x0f,0x88,0xdf,0x1b,0xf5, +0xce,0x87,0x79,0x5f,0x3d,0x98,0x1b,0xf6,0xae,0x2a,0x64,0x0f, +0x8a,0xee,0xc4,0x0c,0x0b,0x03,0x22,0x77,0x50,0x44,0x41,0xe6, +0x28,0x1b,0xdb,0xb2,0x56,0x87,0x1c,0x9a,0xf8,0xfa,0xb3,0xc7, +0xef,0x23,0xf1,0x75,0x99,0xab,0x10,0xbe,0x9d,0xa5,0xa3,0xf1, +0x65,0x8d,0x32,0xa0,0x09,0xe9,0xc9,0x15,0x5e,0x72,0x5d,0x7c, +0xec,0xc0,0x57,0x86,0xa1,0x1b,0x90,0x12,0x7a,0x57,0x39,0xea, +0x04,0x00,0x8b,0x77,0xe9,0xee,0x2c,0x4e,0xe5,0x49,0x3a,0x7c, +0xd7,0x8c,0x4a,0x3c,0x2a,0x9a,0x54,0x79,0x1a,0xae,0x64,0x5a, +0xf8,0x62,0x47,0x47,0x47,0x6f,0xb9,0xf6,0xac,0xba,0xa0,0xcd, +0xd4,0x88,0x52,0x8e,0x21,0x6b,0x42,0x09,0xee,0x2a,0xb4,0x3e, +0xe5,0x68,0x3a,0x1e,0xe6,0x30,0x30,0xf1,0x8c,0xb4,0x36,0xe5, +0xf1,0x39,0x46,0x66,0x56,0x55,0x58,0xc2,0xf4,0x20,0x6e,0x38, +0xbe,0x22,0xe7,0x2b,0x14,0x73,0x8a,0x65,0x60,0xd1,0x92,0x60, +0x13,0x12,0x17,0x0b,0x54,0x27,0x74,0x36,0xfa,0x30,0x37,0x6f, +0x69,0x3a,0x85,0x99,0x0a,0x68,0x54,0x3d,0x12,0xa8,0x78,0xce, +0x47,0x63,0x80,0xd9,0x45,0xb1,0xa1,0x30,0x3c,0xc8,0x5e,0x54, +0x3d,0xb1,0x20,0x83,0xed,0x04,0x11,0x3b,0xd5,0x60,0x5a,0x18, +0xb9,0x8d,0xd6,0x04,0xd0,0x08,0x98,0x8f,0x58,0x8a,0x0e,0x2c, +0x26,0xa4,0xaa,0x4b,0x22,0x34,0x1b,0x85,0x42,0xd9,0xe5,0x38, +0x88,0xc6,0x08,0x88,0xff,0xd4,0x9b,0x8c,0x82,0x78,0x8c,0x80, +0x78,0xd2,0xc9,0x0d,0x12,0x28,0x06,0xd5,0x94,0x91,0x22,0xd3, +0x42,0x4f,0x5c,0xba,0xfe,0xa4,0xbc,0x94,0xc7,0x1d,0x3a,0xeb, +0x1d,0x0f,0xa5,0xd4,0xeb,0x97,0xb6,0xa5,0x38,0x67,0x02,0xeb, +0x54,0x71,0x68,0xa1,0x2a,0x1f,0x4d,0xe7,0x93,0xfc,0xb8,0x77, +0x99,0xef,0xd3,0x55,0x1b,0x19,0x42,0xa5,0x97,0x8c,0x78,0xbb, +0xe4,0xd5,0x00,0xd8,0xc0,0xbe,0xb2,0x0b,0x66,0x6d,0xec,0xdb, +0x03,0x0c,0xe2,0x29,0xee,0xf4,0x62,0x9a,0x5f,0x5c,0x0e,0xfb, +0x91,0x79,0x76,0x94,0xb3,0xd5,0x56,0x3c,0xad,0x08,0x8a,0xd7, +0x2e,0x07,0xf9,0xb0,0x8f,0x26,0x17,0x76,0x78,0xb9,0x0e,0x54, +0x1c,0x06,0x53,0x65,0x27,0x7e,0xad,0xe6,0xf1,0xa0,0xbb,0xee, +0x67,0x45,0xe4,0x6c,0x47,0x20,0x42,0x55,0x6d,0x93,0xc7,0x32, +0xe3,0xf3,0x0f,0x55,0xcd,0x82,0x68,0x60,0xe3,0xe7,0x1f,0x9a, +0xda,0x70,0x5d,0x86,0xb7,0x14,0xab,0x1b,0x37,0x82,0x9d,0x87, +0x6e,0xdb,0x2b,0x67,0x85,0xe9,0x5b,0x1a,0x14,0xad,0x6a,0x10, +0xe0,0xa0,0x76,0xab,0xf9,0x67,0xd6,0x87,0xba,0x98,0xde,0x2a, +0x5e,0x34,0x98,0xc6,0x64,0x17,0xc9,0x32,0x21,0xd4,0x32,0xa2, +0x08,0xc7,0x4c,0x2e,0xf4,0x0d,0xca,0xa7,0xde,0xa3,0xf9,0xf1, +0xa2,0x61,0x57,0x97,0xb9,0xcd,0xf3,0x8f,0xf1,0x40,0x54,0x14, +0xe6,0x37,0xd9,0x60,0xb1,0x68,0xb9,0x87,0x5e,0xa4,0xea,0x03, +0xb1,0x3f,0x28,0x45,0xaa,0x0e,0x7d,0xcb,0x39,0x9a,0xcb,0x91, +0xa4,0x6b,0x2d,0x6d,0xaf,0xea,0xdd,0x1c,0x49,0x0e,0xd1,0x73, +0xcf,0x19,0xae,0x5f,0x64,0xd1,0x56,0x54,0xaf,0x47,0xbf,0x8b, +0x5e,0x64,0xda,0x21,0xe4,0x60,0x4a,0xa6,0x59,0xa7,0xb7,0xa8, +0x7f,0xeb,0x4b,0x44,0x35,0x7c,0x06,0x78,0x0d,0xdb,0x98,0xb3, +0x89,0xfd,0xfa,0x1f,0x2e,0x6a,0x64,0xbf,0xfe,0x97,0xfd,0xf8, +0xdf,0xff,0x17,0x7d,0x69,0xb8,0x6f,0x50,0xff,0xb0,0x84,0x44, +0x0f,0x91,0xf8,0x3b,0x21,0x01,0xcc,0x0b,0x63,0x76,0x30,0xe6, +0xaf,0x12,0x13,0xbd,0xb7,0x10,0xd7,0x4b,0x00,0x61,0x1c,0x0f, +0x6e,0xd1,0xde,0xc7,0x58,0xb7,0x2d,0xda,0xb0,0x25,0x1a,0xf2, +0xa1,0x3b,0x40,0x19,0x14,0x43,0x92,0x63,0xd6,0x45,0x2c,0x81, +0xb5,0xb7,0x50,0xb1,0x22,0xb9,0x87,0x3f,0xa8,0x51,0x81,0x6f, +0xef,0x51,0x05,0x8d,0x19,0xf1,0xf0,0x78,0x36,0xc9,0xfc,0x4a, +0x47,0xd9,0x61,0xcd,0x98,0xc2,0x72,0x41,0xdf,0x0d,0xa8,0xb9, +0xc1,0x63,0x10,0xd5,0x4e,0xd1,0x1b,0x7c,0x59,0xcf,0x8b,0xc0, +0x37,0xa2,0x2e,0xb9,0xaf,0x44,0xbd,0xad,0x76,0x15,0xe5,0x03, +0x01,0xb2,0x0a,0xca,0x5b,0xfc,0xa8,0xc9,0xaa,0x04,0xf4,0x5b, +0x23,0xb5,0xa5,0xe1,0xfc,0x06,0x95,0x50,0x70,0x0a,0x78,0xd2, +0xaf,0x9d,0xde,0x6f,0x2f,0xbb,0xcd,0x88,0x39,0x9e,0x61,0x74, +0x59,0xe9,0x48,0xbf,0x97,0xed,0xf5,0xa5,0x2f,0xfc,0xf6,0x69, +0x44,0x11,0x78,0xa6,0x41,0x28,0xf2,0xbd,0x5a,0x9b,0xfe,0x6a, +0x52,0x13,0x0b,0x01,0x17,0xd7,0xda,0xd8,0x5d,0x91,0xfa,0x12, +0xba,0xee,0x89,0x9a,0x15,0x3d,0x9f,0x70,0x3d,0x0d,0x48,0x68, +0x3b,0x1d,0x68,0x24,0xf8,0xdd,0xeb,0x8c,0xe8,0x9e,0x8b,0xe6, +0x54,0xb4,0x5e,0x0f,0x87,0x97,0xf3,0x95,0x60,0xd4,0xc8,0x7c, +0x66,0x81,0x0e,0xea,0x75,0xce,0x8d,0x30,0xa4,0x7a,0x29,0x0e, +0x74,0x85,0xe0,0x19,0x88,0xc5,0x5b,0x42,0x62,0xe5,0xbf,0x55, +0xc6,0x93,0x8f,0x95,0x0b,0x95,0x2c,0x16,0xab,0x2a,0x01,0xc2, +0xcb,0xd1,0x0d,0xcc,0xc3,0x95,0x90,0xfb,0x80,0xb6,0xa2,0xc1, +0xc8,0x1e,0x3f,0xa3,0x79,0x49,0xbc,0xdc,0x8d,0x60,0x01,0x61, +0x6a,0x6a,0x2c,0x65,0x88,0x5a,0xdf,0xf1,0x87,0xd8,0x3e,0x17, +0xbf,0x26,0x94,0x89,0x15,0x51,0x69,0xbc,0x53,0xc9,0x84,0x1a, +0xf5,0x29,0x88,0x73,0x97,0xa3,0xb4,0x7c,0x54,0xc7,0x79,0x96, +0xcb,0x02,0x71,0xf0,0x1c,0xf5,0xb4,0x81,0x1c,0xea,0xe5,0x78, +0x66,0xb0,0x6d,0xac,0xe4,0xc9,0x0a,0xea,0x64,0xff,0xe8,0xe2, +0x9a,0x6d,0x54,0x3e,0x4a,0x3a,0xf8,0x62,0x58,0xf4,0x3f,0x03, +0x62,0x31,0x8a,0x50,0x1c,0x77,0x28,0x50,0x4d,0x3b,0x91,0x4c, +0xe8,0x47,0x8d,0xb1,0xc9,0xa4,0x34,0x55,0x8e,0xe3,0x64,0xa4, +0x9f,0x90,0x80,0x24,0x0f,0x39,0xac,0x70,0x58,0x0b,0x62,0x2e, +0x5c,0xa0,0x6b,0x7c,0x34,0x1c,0xe2,0xe7,0x72,0x02,0x8e,0x31, +0x55,0x9f,0xdc,0xfb,0x96,0x67,0x54,0xb3,0xbd,0x22,0x93,0x1a, +0x37,0x98,0x09,0xb4,0x04,0xaf,0x30,0xcf,0x96,0x09,0x72,0x59, +0xdd,0x4a,0x57,0x8d,0xb8,0x93,0x60,0x52,0x0b,0x19,0xb0,0x90, +0x8a,0xcb,0x0a,0xe4,0xe0,0x8d,0xd6,0x97,0x07,0x9f,0xaa,0x5d, +0x62,0x4f,0x1c,0x1c,0xee,0x1d,0xed,0x9c,0x1c,0x1c,0x1d,0x17, +0x0d,0xbd,0x52,0x96,0x24,0x21,0x08,0xcd,0x4b,0xb1,0x7c,0xdf, +0x44,0x72,0xc9,0x5c,0x19,0xb6,0x31,0xda,0xf5,0x33,0x95,0x2d, +0xba,0x66,0xfc,0x03,0x12,0x1b,0xa5,0xe3,0x0a,0x29,0x8a,0x8d, +0x0e,0x9a,0x5c,0xac,0x3a,0x92,0x23,0xc9,0xc8,0x1a,0xc6,0x15, +0xea,0x13,0xf3,0xb0,0x3a,0x69,0x69,0xee,0x7b,0xc0,0xc8,0xae, +0x14,0xb1,0x52,0x72,0x78,0x00,0x3c,0x55,0x05,0x19,0x03,0x51, +0x9e,0x26,0x3e,0x07,0xd4,0xac,0x67,0x06,0x5b,0xdf,0x57,0x14, +0x8b,0x96,0xc6,0xcb,0x33,0xc7,0xd2,0x38,0x75,0x1f,0xaa,0x95, +0xf1,0xc4,0x69,0x2d,0x93,0xaa,0x79,0x26,0xb9,0x1b,0x2d,0xce, +0xaf,0xa6,0x83,0x2b,0x56,0x98,0x53,0x72,0x21,0xf5,0xcb,0x7c, +0x6c,0x1f,0x53,0x94,0x26,0x0b,0x90,0xae,0x57,0xac,0xc6,0xdc, +0x38,0xa5,0x26,0xbd,0x4f,0x5c,0x34,0xa3,0xa2,0x26,0x47,0xb5, +0x56,0x3a,0x6e,0x7e,0x80,0x37,0x97,0x27,0x19,0x92,0x82,0x03, +0x45,0xb3,0x85,0x01,0xb1,0xa4,0x39,0x17,0xe2,0x66,0xd3,0x6d, +0x9f,0x8b,0xca,0xc9,0x8c,0x5d,0x4b,0x26,0xd8,0xaf,0x93,0x36, +0xe4,0x91,0xe7,0x5a,0x9b,0xa7,0x67,0xfd,0xde,0xc6,0x65,0xf7, +0xfe,0xfb,0xe5,0xe6,0x00,0x66,0x6d,0x15,0x2f,0x9b,0x8f,0x06, +0x78,0xc3,0x54,0xe3,0x1a,0x6b,0xa7,0x67,0x67,0xf3,0xa8,0x71, +0x8d,0x0b,0x64,0x37,0x92,0x39,0xd0,0xc8,0xbe,0x37,0x53,0xc1, +0x8d,0x71,0x6c,0xa2,0x05,0x47,0xd8,0xee,0x4e,0x76,0xf1,0x28, +0x92,0x16,0xc6,0x37,0x23,0xb4,0x51,0xfa,0xd9,0xb4,0x7e,0xa3, +0x9d,0x0f,0xc0,0x9e,0x37,0xdb,0x3b,0xde,0xdd,0x39,0xdc,0x3b, +0xbd,0xb8,0xee,0xb6,0x2d,0x14,0x88,0xee,0xc0,0xbf,0x14,0x84, +0x05,0xd7,0x65,0x4b,0xbb,0x5a,0x44,0x67,0x67,0xdc,0x62,0x9b, +0xb4,0xe5,0x98,0x3c,0xf4,0x1e,0xf7,0x74,0x12,0x48,0x5c,0x8d, +0x47,0xb8,0xad,0xeb,0x5b,0x69,0x46,0xca,0x3f,0x0f,0xb3,0x5f, +0xce,0x83,0x07,0x63,0x72,0x16,0x34,0x75,0xc3,0xe3,0x98,0xbe, +0xee,0xd0,0x9f,0x46,0xe2,0xb0,0x1e,0x15,0x91,0x09,0x49,0x61, +0xb9,0x40,0x6b,0xda,0x8b,0x05,0xf2,0x64,0x46,0x17,0x15,0xef, +0xad,0x26,0x3e,0x4d,0x6d,0x54,0xc4,0x27,0x79,0x33,0x03,0xe1, +0xa3,0x92,0x84,0xca,0x6c,0xd0,0x32,0xda,0x29,0xf4,0x5b,0xf4, +0x2c,0x4a,0x12,0x45,0xcf,0xd0,0x87,0x6a,0x0d,0xf6,0x1c,0x1b, +0xf9,0x37,0x4a,0x00,0x49,0x61,0x69,0x8c,0x13,0x2e,0x6e,0xe5, +0x0f,0x55,0xd8,0xf1,0xfe,0x44,0xad,0x41,0x06,0x05,0xe8,0xe8, +0xde,0x20,0xa0,0x08,0xad,0xaf,0x63,0x2b,0x4e,0xb7,0xba,0x76, +0x12,0x8b,0x0b,0x8e,0x53,0x19,0x1d,0xbb,0xb4,0x92,0x0a,0xb3, +0xe2,0x68,0xb8,0xef,0xe7,0x60,0xf0,0x0e,0x00,0x11,0xd4,0x28, +0xc4,0x11,0x0c,0xce,0xf2,0xb4,0xdb,0x34,0xed,0xb4,0x13,0x09, +0x73,0x0c,0xc7,0x9c,0x75,0xfc,0x69,0xcc,0xa1,0x97,0x69,0x78, +0x09,0xc5,0xf4,0xd4,0x70,0x63,0x01,0xdb,0xd3,0x48,0xda,0xbf, +0xec,0xa6,0x06,0x1f,0x00,0x19,0x1b,0xcd,0xaf,0x2a,0xdd,0x8b, +0x79,0xbc,0xbf,0xe5,0x7c,0x2e,0xc9,0x9a,0x5b,0xb2,0x1c,0xc1, +0xd2,0x65,0x69,0x41,0xb8,0xb7,0x39,0xac,0x11,0x3c,0x36,0x4c, +0x72,0xbd,0x0d,0xd4,0xd3,0xe0,0xae,0x06,0x3a,0x78,0x0e,0xa1, +0x6d,0x0e,0x6e,0xe3,0xe3,0x01,0xc5,0xdb,0x61,0xb6,0x40,0xc4, +0x76,0x10,0xb3,0x8d,0x51,0xcf,0x83,0xa8,0xe7,0xdd,0x36,0x44, +0x75,0xaa,0x78,0x66,0xd0,0x17,0xcf,0x89,0xc6,0x47,0xcf,0x97, +0x6e,0x4a,0x3f,0x4f,0x20,0xbc,0xfd,0x78,0xc9,0x6d,0x2e,0xb9, +0xed,0x4b,0x6e,0x63,0xc9,0xc7,0x0b,0x72,0x39,0x35,0x08,0xa7, +0x26,0xb5,0x83,0xd0,0xd8,0xe0,0x8e,0x67,0x80,0x2d,0x54,0x1b, +0xd0,0x13,0x09,0xe4,0x2b,0xd8,0xa3,0xc2,0x34,0x82,0xaf,0x19, +0x1e,0x32,0x4e,0x7a,0x10,0x9c,0xd4,0x22,0x2e,0x21,0xe5,0x5a, +0xc9,0x52,0x66,0x0b,0xcc,0x48,0x7b,0xbd,0x49,0x48,0x15,0xac, +0x4e,0x4f,0x78,0xab,0x48,0xd6,0x5e,0xd7,0x2f,0x07,0xc3,0x19, +0x3e,0xba,0x0e,0xf6,0x8d,0xaa,0xfa,0x9b,0xe9,0x15,0x4f,0x4f, +0xe7,0xd3,0x34,0x90,0xfb,0xa7,0x5f,0x46,0xb3,0xde,0x67,0x90, +0xfb,0x8f,0xe9,0xc3,0x0a,0xfe,0x27,0x58,0x42,0x8c,0x8c,0xa3, +0xdc,0xef,0x36,0x01,0xec,0xa9,0xf2,0x92,0x5c,0xba,0x05,0x3b, +0x81,0xe7,0xcb,0x6e,0xcd,0xba,0xc4,0xc3,0xec,0xa7,0xc0,0xa4, +0x71,0x6f,0xe0,0x27,0x82,0x71,0xa8,0x34,0x2d,0x8f,0xa7,0xe8, +0x02,0xff,0x57,0x39,0x92,0xc2,0x3e,0x95,0xb0,0xe2,0x37,0x83, +0x5b,0x99,0x75,0x88,0x63,0x75,0xfc,0xaa,0x5a,0x37,0xcf,0xc7, +0x78,0xbb,0xa8,0x46,0x00,0xe4,0x04,0x44,0x3f,0x2f,0x6e,0xd2, +0xf5,0x53,0x09,0xe9,0x75,0x98,0xf8,0x85,0x6d,0x72,0xde,0x32, +0xf9,0xb6,0xc9,0x9f,0x9b,0xfc,0x7b,0x42,0x22,0x40,0xe0,0xc5, +0x96,0xd8,0x33,0xd3,0x9c,0x62,0xab,0x6b,0x66,0x8a,0x17,0xe0, +0xc2,0x38,0xcb,0xb2,0xbc,0x05,0x6b,0x17,0xfc,0x6c,0xf3,0xcf, +0x73,0xfe,0xf9,0x7e,0xb1,0x58,0xcb,0x5b,0x78,0xe5,0xb2,0x8d, +0x7f,0x9e,0xe3,0x9f,0xef,0x13,0x8d,0x92,0x50,0x85,0xde,0x69, +0x73,0xc3,0x42,0xcc,0x3c,0x16,0x25,0xac,0x83,0x16,0x76,0x62, +0x24,0x21,0xa8,0xc6,0xf3,0xae,0x7a,0x5d,0x53,0xae,0xb8,0x40, +0xe4,0x85,0x15,0x93,0x65,0x38,0x9d,0x94,0x62,0xf9,0x07,0xc7, +0x06,0x47,0x00,0x78,0xe0,0x32,0xbf,0xc9,0xa1,0x76,0xd8,0xdb, +0x5a,0x24,0x0b,0x6b,0x36,0xc0,0x9f,0xbb,0xa1,0x31,0xd2,0x16, +0xa4,0x1f,0xd8,0xca,0xe6,0x2d,0x5c,0xb1,0x8d,0xb0,0x71,0x0f, +0x77,0x3e,0xea,0x4d,0xbe,0xa0,0xf7,0xc4,0x91,0x99,0x0c,0xae, +0xae,0xbd,0xb9,0x84,0xa7,0x08,0x81,0x97,0x41,0xb4,0x00,0x58, +0x92,0x23,0x7d,0x32,0xf1,0x9f,0x52,0x4c,0xd3,0x06,0xf5,0x56, +0x05,0x26,0x99,0xd4,0x3c,0xcc,0x2f,0x81,0x96,0x07,0xfd,0xfe, +0x30,0xff,0x06,0x14,0xb0,0x74,0x90,0xd0,0x61,0x50,0x41,0x1c, +0xe3,0x51,0x14,0x56,0x1d,0x96,0x08,0xc3,0x21,0x59,0xaf,0x33, +0x00,0x15,0xb1,0xba,0x15,0xe7,0x03,0xdd,0x88,0x7f,0x46,0x1f, +0x12,0xa0,0x72,0x47,0x16,0x50,0x5c,0x8d,0x11,0x3d,0x0b,0xa6, +0x13,0xcc,0x38,0x78,0xea,0x27,0x91,0x78,0x46,0xdc,0x4e,0xca, +0x73,0x0d,0xa8,0x96,0x48,0x23,0x5a,0xa2,0xd6,0x3d,0xfc,0x6b, +0xc3,0x3f,0x5a,0xd8,0x7c,0x59,0x39,0x30,0x26,0x06,0xb9,0x8b, +0x76,0x04,0xf1,0x22,0x6c,0x4d,0x88,0x11,0x0a,0x24,0x89,0x57, +0x65,0x57,0xa5,0xe4,0x98,0xcb,0xc7,0xc0,0x2c,0x4e,0xab,0xbb, +0xc4,0xe2,0xcb,0x3b,0x02,0x76,0xfb,0x55,0x82,0xc4,0x9e,0xbf, +0x82,0x66,0xa9,0xea,0x4e,0x07,0x28,0x46,0x4a,0x08,0x0d,0xfe, +0xd0,0xa3,0x60,0x17,0x13,0xd4,0x17,0xba,0xf5,0x52,0xfa,0x23, +0xef,0x83,0x46,0x86,0x5e,0xc6,0x68,0x84,0x94,0x53,0x0f,0x10, +0x83,0x58,0xf0,0x5a,0x13,0xa1,0xc6,0x76,0xc8,0x22,0x4a,0x5c, +0x8f,0x60,0x99,0x36,0x15,0x0c,0x29,0xc6,0x6e,0xb9,0x0c,0x57, +0x88,0x33,0xd3,0x63,0x61,0xe3,0x0a,0x5e,0xce,0x04,0x3e,0x09, +0x04,0xb2,0x5c,0xc5,0x9e,0x25,0xd2,0x8d,0xc4,0xf4,0xd5,0x88, +0x07,0xba,0x0a,0xaf,0x14,0xf1,0x92,0xa3,0xf2,0x37,0x64,0xee, +0x26,0xab,0x1a,0x0d,0x58,0x45,0x6f,0xe7,0xea,0xb9,0x28,0xdf, +0x30,0x50,0x64,0x57,0x86,0x86,0x6b,0xd2,0xc3,0xe2,0xef,0x15, +0x38,0x0d,0x46,0x23,0xec,0x71,0x77,0x0e,0x5f,0x3c,0x75,0x9f, +0x7a,0x63,0x21,0x65,0x59,0xdb,0x22,0xba,0x5c,0x4a,0x95,0x6c, +0xb3,0x49,0x8d,0x81,0xee,0x37,0x1d,0xef,0x6e,0x9b,0x68,0x63, +0xca,0x9e,0x18,0xd5,0x38,0xeb,0x68,0xde,0x90,0xe0,0xc4,0x32, +0x6a,0xa0,0x85,0x4f,0xc5,0x16,0xf3,0x42,0x67,0x66,0x11,0x3e, +0x34,0xa5,0xd9,0xc9,0xb0,0x0a,0x1c,0xf9,0xe6,0x76,0x38,0x40, +0x6f,0xae,0xae,0x9e,0xda,0xf9,0x7c,0x56,0x71,0xb4,0xb8,0xa5, +0x57,0x72,0x3c,0x5e,0xac,0x5d,0xf4,0x46,0xb4,0x5a,0x9c,0xe7, +0x52,0x9a,0xfc,0xa3,0xbb,0x15,0x83,0x10,0xf5,0xd8,0xf9,0xfb, +0x1d,0xf1,0xe7,0x5e,0xe6,0x92,0x82,0xa0,0x64,0x10,0x8e,0xa8, +0x33,0x3b,0xd6,0x98,0x10,0xbb,0x29,0xf1,0x6a,0xe9,0x22,0x61, +0xd5,0xaa,0x8f,0x86,0xe3,0xab,0x01,0x94,0x3c,0x38,0x5a,0xd5, +0x4b,0x1d,0xea,0x25,0x2e,0xa8,0x70,0xae,0x20,0xcc,0x4e,0xf5, +0xba,0xe0,0xca,0x24,0xa8,0x1e,0xe6,0xfb,0xd7,0xc9,0x21,0xa9, +0xeb,0x18,0xde,0x8d,0x25,0xc5,0x36,0x28,0x1c,0xab,0xe6,0xb3, +0x24,0xef,0xbc,0x7b,0xb9,0x7a,0x3e,0x7f,0xd5,0x84,0xd6,0x00, +0x93,0x65,0x09,0x0f,0x8a,0xbf,0x2f,0x32,0x95,0x5f,0xe6,0x20, +0x1a,0xcc,0x56,0x12,0x1b,0xec,0xa2,0xc8,0xd7,0xda,0x93,0xeb, +0x4d,0x4c,0xd8,0x09,0xbe,0x82,0x62,0xd5,0x93,0x9c,0x5f,0xac, +0xf6,0x86,0x2b,0x29,0x3d,0x83,0x15,0x61,0x0d,0xff,0x64,0xf2, +0x99,0x3d,0x8e,0x8d,0xaf,0xb0,0x84,0x8b,0xae,0xb1,0x88,0x4d, +0xaf,0xdf,0x1f,0x90,0x07,0xec,0x15,0xb8,0xfc,0x00,0xf5,0xbf, +0x80,0x7f,0x3f,0x20,0x22,0x2f,0x9e,0x80,0x87,0xae,0xac,0x84, +0x89,0xaf,0xad,0x8a,0x32,0xe4,0x5d,0x31,0xf9,0x84,0x20,0x8c, +0x42,0x54,0x1a,0x11,0x79,0x0d,0x6c,0x27,0x0f,0x55,0x5f,0x84, +0xe1,0xbd,0xae,0x05,0x88,0x14,0xb3,0x55,0xa1,0x33,0x17,0x6e, +0x14,0x62,0xf1,0x1d,0x60,0xb1,0x09,0xff,0x7e,0xf5,0x18,0x26, +0x52,0x7c,0x05,0x02,0x73,0x35,0xcd,0xa9,0x48,0xdb,0x09,0x32, +0xb6,0xb5,0x49,0xe7,0x76,0x32,0xb8,0xa1,0x6c,0x69,0x61,0x54, +0xd0,0x79,0x62,0xc7,0xd5,0xfb,0xd7,0xbd,0xa3,0x83,0x72,0xbd, +0xc5,0x32,0x6b,0x58,0xc6,0xca,0xa0,0xe5,0xdc,0xae,0x2e,0xb5, +0x73,0xb0,0x51,0x62,0x85,0x88,0x42,0xb8,0x11,0xb0,0x20,0x63, +0x00,0x29,0xd1,0x59,0xb0,0x8e,0x1b,0x2b,0x43,0x83,0x80,0xe3, +0x0f,0x0e,0x6c,0xb1,0x53,0x55,0xac,0x87,0xb6,0x06,0x5f,0xe6, +0x17,0xc3,0x1e,0x9b,0xa2,0x8f,0xcb,0xd9,0xef,0x55,0x76,0x76, +0x11,0x25,0x99,0xee,0xcb,0x2b,0x75,0xdb,0x66,0x74,0x0d,0x94, +0x88,0x70,0xb1,0x40,0x6e,0xdf,0xb3,0x2d,0x52,0xab,0x58,0xb8, +0x4b,0x90,0x7d,0x45,0x2c,0xf9,0x9c,0x10,0x98,0xd9,0x08,0x98, +0x6f,0xf9,0xa4,0x37,0x44,0xbd,0x2d,0x77,0xb2,0x34,0xa2,0xbb, +0x50,0x7e,0xd7,0xdd,0xc6,0x40,0xe6,0x7b,0xcb,0x40,0xd3,0xf1, +0x18,0x1f,0xcf,0xe9,0x63,0xba,0xab,0xc6,0xa5,0x89,0x1e,0x88, +0xc7,0xae,0x1f,0xa5,0xf7,0x77,0x7b,0xc3,0xa1,0x8d,0x34,0xee, +0x6d,0xb9,0x7c,0x90,0x71,0xa6,0x24,0x05,0x70,0x21,0x10,0x9b, +0x6c,0xcb,0x85,0xdd,0x46,0x27,0xe2,0x16,0x26,0x0c,0x39,0xdd, +0x15,0x3c,0xa5,0x38,0xdd,0x8a,0xef,0xb0,0x11,0x5b,0x5f,0x3c, +0x38,0xc3,0xdc,0x3f,0x3c,0x38,0x3e,0x7e,0xf3,0xe3,0xdb,0x3d, +0xaf,0x00,0x20,0x39,0x03,0x59,0xcf,0xc3,0x61,0x94,0x44,0x3a, +0xc2,0x78,0x37,0x86,0xbc,0xe5,0x2e,0x1e,0xa1,0x53,0x1e,0x75, +0x84,0xbe,0x4a,0xee,0x57,0xcb,0x2c,0xe9,0xe6,0x54,0x1d,0xa3, +0x2f,0x16,0x42,0x47,0xd5,0x8b,0x72,0xc5,0xe1,0x3a,0xe5,0x63, +0x59,0xb9,0xb0,0xde,0xcb,0x21,0x7b,0x25,0x40,0xc6,0xb9,0x70, +0xe8,0xee,0xbb,0x43,0x8f,0x8a,0x7b,0x2b,0x49,0x62,0xc9,0xab, +0x51,0x28,0xf6,0xfa,0x77,0xfa,0x3c,0xab,0xf0,0x40,0xf8,0xc1, +0x1d,0x0f,0x89,0xf6,0xe6,0xd6,0x8c,0x71,0xe8,0x83,0x24,0x10, +0x28,0xa5,0x8a,0x20,0xda,0xdd,0xed,0x77,0xe2,0xbb,0xac,0xac, +0xa3,0x80,0x8e,0x7e,0x99,0x02,0xef,0xea,0xf5,0x3b,0xb1,0x3b, +0x13,0xdf,0x66,0x77,0xe8,0x45,0xe6,0x2e,0x1a,0x80,0x60,0x8f, +0xa6,0x2d,0x9a,0x10,0xf0,0x2f,0x7d,0xcc,0x6d,0xc1,0x3a,0x39, +0xfa,0xe1,0xba,0xe7,0x4c,0xf0,0x89,0x5a,0x59,0xf0,0x81,0x41, +0xf8,0x48,0xbc,0x97,0x83,0x07,0x6f,0x37,0x82,0x26,0xa2,0x37, +0xb1,0x87,0x5a,0x53,0xa5,0x6c,0x11,0xf6,0x06,0x35,0x4a,0x79, +0x53,0x0d,0x47,0x28,0x98,0x8c,0x97,0x23,0x3b,0xfd,0x58,0x33, +0xdf,0xab,0x03,0xf1,0x1e,0x00,0x0f,0xcc,0x13,0x58,0xac,0xd5, +0x79,0x0f,0xef,0x13,0xfa,0xe3,0xda,0x4a,0x61,0xba,0x4d,0xb6, +0x3a,0x1c,0xaf,0x33,0x51,0x52,0x1e,0xed,0x24,0x5a,0x29,0x74, +0x86,0x5b,0x86,0xae,0xe3,0x0f,0x01,0xa2,0x9d,0x20,0x14,0x42, +0x48,0x39,0xf4,0x55,0xbb,0x0c,0x5d,0x5e,0x7c,0xb3,0x8f,0x0e, +0x67,0x13,0xf4,0x06,0x16,0x4c,0x3f,0xcb,0xb5,0x58,0xb5,0x0a, +0xc6,0x9a,0xb2,0x59,0xad,0x2a,0xff,0x6d,0xf9,0x0e,0x6f,0x4e, +0x52,0x4a,0xa1,0xfa,0xf0,0x10,0x8a,0x7e,0x5b,0xf2,0xbb,0x2d, +0xbf,0xcf,0xe5,0xf7,0xfb,0x2e,0xdd,0xd8,0x38,0x9a,0x94,0x95, +0x2a,0xbb,0x6b,0xff,0x53,0xc8,0xd2,0x6d,0x5a,0x95,0x54,0x59, +0x5e,0xb0,0xb4,0x57,0xf7,0x57,0x7c,0x97,0x00,0xe4,0xb2,0x6b, +0x17,0x8b,0xb5,0x2d,0xa2,0x8c,0x6e,0x35,0x65,0x84,0x85,0xc3, +0x59,0xef,0x61,0x0a,0xe1,0xd8,0x70,0x62,0xdc,0xa7,0x5b,0x94, +0xd0,0x4f,0xad,0xae,0xb5,0x95,0x2c,0x9f,0x40,0x5a,0x8f,0x32, +0x12,0x4f,0x60,0xd0,0x6c,0xba,0x26,0x21,0x4a,0x51,0xa8,0x05, +0xd4,0x02,0x99,0x0a,0xb8,0x4e,0xcb,0x1b,0xd3,0xff,0xe9,0xf4, +0x12,0x21,0x3b,0x4c,0x76,0x59,0x46,0xf1,0xfe,0xc7,0x9d,0xc2, +0xa8,0xb6,0xe8,0x19,0x69,0xd7,0x7f,0x3b,0xfb,0xff,0xe4,0x7c, +0x9f,0x57,0x74,0xf9,0xf2,0x81,0x2e,0x2f,0x6c,0xf0,0x91,0x91, +0xf0,0x9a,0xcf,0xbb,0x44,0x10,0x17,0xfc,0xd1,0xa7,0xeb,0x2d, +0xd8,0x36,0xb5,0xbd,0xf9,0xb3,0x60,0xac,0x1c,0x2e,0x72,0x0c, +0x0f,0xe1,0xd4,0xd9,0x93,0x4b,0x99,0xb5,0x24,0x1c,0xfa,0x07, +0x87,0x6c,0xf9,0xd4,0x21,0xe3,0xbe,0x42,0xaf,0xb2,0x3c,0x66, +0x1e,0xc5,0xe2,0xe9,0x8e,0x4d,0xc9,0x5c,0x16,0x3c,0xdb,0xe1, +0xf2,0xa7,0x36,0xae,0x49,0x4c,0xd2,0x85,0xa8,0x29,0xe1,0x39, +0x9f,0x57,0x0f,0x83,0x72,0x4f,0x1a,0x57,0x7f,0x02,0x85,0x32, +0x2c,0x73,0x60,0x60,0xe5,0x5b,0xf6,0x98,0x36,0x0b,0xb4,0xcb, +0x8c,0xdf,0xdb,0x67,0xea,0x3c,0xc1,0x68,0x2e,0x9d,0x05,0x3c, +0xdb,0x28,0x89,0x23,0xd3,0xe2,0x87,0x51,0x6b,0x6f,0xa6,0x17, +0x62,0xa3,0xe4,0xd8,0x4c,0x1f,0x4e,0xd9,0x71,0x40,0x99,0xb0, +0x13,0x2b,0x4c,0xdc,0x76,0x37,0xc4,0x43,0xd7,0xac,0x2b,0xd3, +0xf0,0xd5,0xf1,0xcb,0x23,0x87,0xd7,0xf7,0x74,0x95,0x43,0x3d, +0xc1,0x17,0x3c,0x5b,0xe8,0xe1,0x9a,0xe9,0xd0,0x89,0xe9,0x49, +0x5a,0x38,0x82,0x43,0xfb,0x82,0x5b,0x6b,0x85,0xeb,0x86,0xd2, +0x41,0x79,0x6f,0xa4,0xce,0xb2,0x79,0x62,0x88,0x1c,0x8c,0x0c, +0xd8,0x12,0xad,0x93,0x75,0xd7,0x82,0x70,0x81,0xa4,0x5d,0xb2, +0x8d,0xe0,0x74,0x6d,0x11,0xc0,0x8a,0x4b,0xe8,0x96,0xef,0x1a, +0x44,0xb4,0xd9,0x9f,0xc4,0x90,0x38,0xe9,0x1a,0x7a,0xf2,0xc5, +0xd9,0x23,0x5c,0x24,0xb0,0xb6,0xd1,0x24,0x6b,0x1b,0x5b,0x96, +0x3b,0x3a,0x03,0x59,0x4c,0xce,0x38,0x91,0x2b,0xd5,0x47,0x6d, +0x76,0x7b,0xf0,0x6f,0xeb,0x6b,0x8b,0x85,0x55,0xd2,0xf6,0x40, +0x77,0x9c,0x80,0x1a,0xc9,0x03,0x6d,0x15,0x89,0x8b,0x88,0xca, +0x02,0x53,0xca,0xe6,0xca,0x54,0x3c,0x19,0x95,0xd0,0x11,0x64, +0x64,0x42,0x64,0x25,0xbb,0x20,0x8d,0x11,0xda,0x9a,0x7a,0xf3, +0xc6,0x16,0xea,0xef,0xab,0xfd,0xb9,0xf0,0xd2,0x24,0x49,0x6e, +0x81,0x5a,0x92,0xb8,0x28,0x26,0x2c,0xee,0xb4,0x07,0x77,0x67, +0xb0,0xe2,0x8e,0x7c,0x94,0x8f,0xb9,0x3d,0x24,0x62,0xd9,0xe9, +0xf9,0xd5,0x3d,0xe4,0xdb,0xea,0x06,0xcb,0x7e,0xa8,0x4b,0x93, +0xe9,0x2d,0x82,0xfb,0xbd,0x15,0xd7,0x21,0xff,0x16,0x72,0xd5, +0x16,0xfe,0xd9,0xc6,0x3f,0xcf,0xf1,0xcf,0xf7,0x7e,0x9c,0x4b, +0x22,0x9a,0x42,0x85,0x4a,0xbb,0xfa,0x57,0x65,0x69,0x3d,0x9e, +0x65,0xfb,0xf1,0x2c,0xcf,0x1f,0xcf,0xe2,0xb1,0x5e,0x25,0x87, +0x09,0x11,0x91,0xdb,0x0a,0x24,0x56,0xe8,0x9d,0x8c,0xd3,0xea, +0x75,0x71,0x47,0x5c,0xd0,0x2b,0xc2,0x26,0x26,0x1d,0x4e,0x63, +0x21,0xcc,0xe9,0x0b,0xb3,0x93,0x7e,0x01,0x83,0x76,0xbf,0x18, +0x5e,0x68,0xfb,0x8b,0xe2,0xd2,0xd8,0x26,0xca,0x2f,0x8e,0x13, +0x08,0x4d,0x12,0x42,0x1b,0x72,0xf4,0x51,0x24,0x42,0x07,0x3b, +0x76,0x7e,0x89,0x38,0xc6,0x68,0x5a,0x53,0x12,0xd3,0x57,0x10, +0x67,0x62,0x0a,0x38,0xb1,0x80,0x8f,0xe3,0x55,0xaf,0xb3,0xf3, +0x81,0xac,0x88,0x5f,0xc1,0xb6,0x41,0xb9,0x4d,0xad,0xff,0xae, +0x6d,0xda,0xfe,0xf6,0x36,0x6d,0xff,0x77,0x6d,0xd3,0xf3,0x6f, +0x6f,0xd3,0xf3,0xff,0xae,0x6d,0xfa,0xfe,0xdb,0xdb,0xf4,0xfd, +0x7f,0xcb,0x36,0x49,0x00,0xbd,0x59,0x3f,0xfa,0xa1,0xb6,0x2f, +0xee,0x3c,0x85,0x56,0x5f,0x3c,0x4e,0x71,0x0c,0x19,0x44,0x66, +0x9b,0x4a,0x46,0x88,0x8b,0x1c,0x8b,0xcc,0x1d,0x24,0xc1,0x69, +0x8a,0xe4,0x3c,0xc5,0xa4,0xae,0xec,0x04,0x09,0xb3,0x3f,0xe6, +0x5f,0xa6,0xa5,0x55,0xdb,0x26,0xbc,0xb5,0x86,0x31,0x39,0xe8, +0xc4,0xd1,0xcb,0x18,0xa5,0xbc,0xcb,0x51,0xf6,0x9b,0x17,0x61, +0xd6,0x4e,0x71,0x75,0xb1,0xc9,0xa7,0xc2,0xeb,0xe8,0xbb,0xa5, +0xbe,0xb7,0xd5,0xf7,0x73,0xf5,0x0d,0x63,0x69,0xdb,0x5b,0x3c, +0xdd,0x09,0x8e,0x4e,0xd0,0x09,0x3a,0xc8,0x18,0xb0,0x73,0x87, +0xcf,0x6c,0x65,0xf5,0x20,0x72,0x28,0xe0,0x5f,0x19,0xb2,0x88, +0x54,0x9e,0x42,0x29,0x62,0xa1,0x64,0xa4,0x05,0x39,0x2b,0x08, +0xfb,0xe6,0xc5,0xc0,0x6d,0x11,0x90,0x5c,0xfc,0xc1,0x28,0xea, +0x12,0x66,0x11,0xe9,0x25,0x1b,0x1c,0x96,0xda,0x6d,0xfb,0x6c, +0x14,0x39,0x27,0xb3,0x16,0x8a,0x5f,0xd1,0x50,0xe0,0x12,0x9f, +0x1f,0xab,0x16,0x42,0xbf,0x0e,0x22,0xf4,0x46,0x16,0xc1,0x98, +0x4d,0x6b,0xb0,0x32,0xd1,0xa2,0x55,0x5b,0x2c,0x6a,0x1c,0x72, +0xc8,0x27,0x35,0x7b,0x4c,0x06,0x95,0x0f,0x33,0xfc,0x3b,0xcd, +0xa2,0x06,0xeb,0xca,0x75,0xa2,0x69,0x94,0x3e,0x8b,0xe3,0x8f, +0xf5,0xfa,0xc7,0x22,0xb1,0x45,0xcf,0x1a,0x50,0x5f,0x03,0xd5, +0xe8,0x3a,0x1f,0xd3,0x69,0xf2,0x2c,0x69,0x3c,0x3b,0x75,0x91, +0xdd,0x67,0x8d,0x08,0x5b,0xd3,0x00,0x04,0x6a,0x80,0x41,0xbd, +0x5e,0x9b,0xf2,0x01,0x40,0xed,0x1e,0xa3,0x9f,0xd5,0x30,0x7e, +0x8d,0xe7,0x27,0xaa,0xfe,0xc3,0xb6,0x13,0x53,0xa0,0x58,0xed, +0x96,0xb0,0x88,0xf0,0x0b,0x67,0x52,0x4d,0x61,0xeb,0xe2,0x0b, +0xf3,0x12,0xca,0xca,0xf1,0x40,0x7b,0x99,0x70,0xa6,0x25,0x67, +0x9d,0x66,0x68,0x1e,0xe1,0xce,0x45,0x2d,0x5d,0xcf,0xb8,0x76, +0x47,0x78,0x7b,0x6c,0x5f,0xe2,0xc4,0x11,0xda,0xe9,0xfe,0x18, +0x51,0x2e,0xb2,0x52,0x6b,0x4d,0x62,0x56,0x58,0x8b,0xc4,0x3c, +0xf6,0x85,0x57,0xd5,0x84,0xcb,0x2e,0x47,0xea,0x8d,0xe2,0x21, +0xaa,0x0f,0x15,0x5e,0x29,0xb2,0xeb,0xa9,0xfb,0x65,0xe8,0x3e, +0x9a,0xb7,0x20,0x81,0xc1,0x02,0xff,0xaa,0x50,0xb4,0xb2,0x6c, +0x52,0xd9,0xc8,0x44,0x8e,0xbc,0x62,0xfa,0x69,0x80,0x6a,0xad, +0x62,0x70,0x94,0xa2,0xf0,0x8d,0x83,0x35,0x45,0x9a,0x6a,0x37, +0x52,0xc5,0xa1,0xc5,0xdc,0xec,0x07,0xee,0x14,0x3e,0xbb,0xa9, +0xff,0xcc,0x44,0x49,0x0c,0xad,0xf6,0xae,0xb5,0x4c,0x11,0x97, +0x26,0xb2,0x87,0x36,0xd5,0x63,0xb1,0x71,0x35,0x41,0x11,0xeb, +0xa4,0xd7,0x46,0xe1,0xa1,0xd3,0x32,0x78,0x57,0xb6,0xfe,0x9f, +0xab,0x9c,0x9f,0x04,0x16,0x1d,0xca,0x56,0xa3,0x2b,0x0d,0x38, +0x14,0xb3,0xb9,0xae,0xfa,0xc5,0x3a,0x1a,0x72,0xc5,0xac,0x1d, +0xf3,0xf0,0x29,0xad,0x7f,0x6b,0xeb,0xd2,0x97,0x15,0x60,0x75, +0x0b,0x1c,0x68,0x3c,0xb5,0x3f,0x19,0x5c,0x7c,0x34,0x65,0x2c, +0x5c,0x66,0xe7,0x02,0x49,0x9c,0x2e,0xdc,0x95,0x2d,0x6b,0x2f, +0x8b,0x99,0x61,0x5b,0x48,0x79,0x27,0x79,0x6f,0x3a,0x1e,0x29, +0x17,0x38,0xe2,0xf4,0x86,0x62,0xd5,0x69,0x18,0x5d,0x52,0xd0, +0x8a,0x35,0xb5,0x0f,0x63,0xc5,0xc3,0x9b,0x75,0xef,0x76,0x61, +0xdf,0x42,0xa3,0x15,0x75,0xf2,0x53,0x63,0x3d,0x4c,0xb9,0x82, +0x9d,0xd3,0x6e,0x7a,0xbf,0x2c,0xbe,0xa8,0xb4,0xc9,0xa6,0xb8, +0x59,0x62,0x4b,0xfe,0xce,0xdf,0x82,0x41,0x0b,0x14,0x76,0x1f, +0x55,0x5e,0x52,0xb9,0xc9,0x53,0x76,0x38,0x5d,0x92,0xc1,0x13, +0xb6,0x85,0x8f,0xa9,0xea,0x14,0xd8,0x6c,0x6c,0x08,0x78,0xf2, +0xfc,0x16,0x78,0xeb,0xb3,0xd9,0xd1,0x83,0x90,0xab,0xc7,0x77, +0xd7,0x03,0x15,0x15,0xdc,0xfc,0xb9,0xde,0x44,0x05,0x4c,0xd4, +0x0a,0xbc,0xb0,0xaf,0xb7,0x57,0x56,0x69,0x8a,0x3e,0xf2,0x96, +0xae,0xcb,0x8b,0x06,0x91,0xb8,0x1d,0x36,0xbf,0x11,0xc7,0x44, +0xea,0xe1,0xaa,0x1b,0xa8,0x7b,0xeb,0x86,0x30,0x14,0x44,0x80, +0xb5,0x4b,0xa1,0xd0,0xf3,0x05,0xac,0xeb,0x1c,0xdd,0xb6,0x30, +0xfd,0x52,0xc5,0xe7,0x0f,0x38,0x1e,0x08,0xc2,0x7b,0x87,0x70, +0x47,0x0f,0x96,0x6c,0xf5,0xeb,0x72,0xbb,0xe1,0x77,0xc6,0xfe, +0xc5,0x1c,0x73,0xb1,0xb4,0x37,0xcc,0x6c,0x53,0x7c,0x16,0xbc, +0x9f,0xe0,0x83,0x07,0x1a,0x7f,0x1b,0x4f,0xde,0xe4,0xec,0x77, +0x4b,0x7d,0x6f,0x93,0x2b,0xf3,0xa5,0xb8,0x5e,0x4d,0x4b,0xe3, +0x58,0x31,0x02,0x7a,0xc8,0x60,0xe8,0x47,0xe3,0xd9,0xe0,0xf2, +0x4b,0xaa,0x69,0xf3,0x6a,0x42,0x56,0x79,0x9e,0xdc,0x71,0xff, +0x47,0x7a,0x45,0x35,0xd9,0xe3,0x48,0x6d,0x17,0x12,0x4a,0xef, +0xb1,0xcb,0xd2,0x12,0xaf,0xc2,0xb7,0x71,0xf4,0x6b,0x4b,0x31, +0x03,0x13,0xff,0x16,0x40,0x8e,0x6e,0x86,0x7f,0x22,0x7f,0x3c, +0x7d,0xcb,0x67,0x4a,0xa6,0xfc,0xd1,0xde,0x36,0x17,0x71,0xdd, +0x19,0x2b,0x0f,0xf6,0x8e,0xf9,0x75,0xec,0x57,0x5a,0xe0,0x5c, +0x89,0x40,0x72,0x96,0xa9,0x13,0x0f,0x8f,0x86,0x25,0x4f,0x4a, +0x7c,0x10,0xe2,0xd0,0xc5,0x03,0xa3,0x26,0x5c,0x2d,0x2b,0x8d, +0xf3,0x23,0xa8,0x49,0x17,0x24,0x1d,0xf9,0x48,0x43,0x2e,0x99, +0x78,0x7a,0xf8,0x46,0xc4,0x0e,0x55,0xdf,0x66,0x15,0xb4,0xa4, +0xf0,0x63,0x82,0x0b,0xd0,0x0b,0x46,0xa6,0xa3,0x43,0xe5,0x1e, +0x74,0x30,0x15,0xae,0x95,0xa8,0xb9,0xeb,0x5b,0x26,0xd2,0x8e, +0x25,0x56,0x3a,0xd7,0x3e,0x2d,0x8c,0x75,0xa1,0x83,0xab,0x9a, +0xd5,0x95,0x63,0x49,0x9e,0x99,0x5f,0x5f,0x3e,0x91,0x75,0xc3, +0xb1,0x3c,0x13,0x51,0x03,0xa2,0x32,0xc9,0x06,0xb6,0xb5,0x94, +0x4b,0x62,0xb5,0xbc,0x46,0xc0,0xa8,0x20,0xf4,0xa5,0xb2,0xb4, +0x57,0x0f,0xe9,0x7d,0x74,0x9e,0x26,0x99,0x91,0x0a,0x75,0xf4, +0xab,0x26,0x40,0xdb,0xfb,0x89,0xa3,0x3c,0x9d,0x02,0x45,0x31, +0xf1,0xa6,0x21,0x65,0x70,0x64,0xb1,0x6d,0x0e,0x83,0x6b,0x1a, +0x91,0x70,0xe9,0x36,0x83,0xe9,0x51,0x80,0x86,0xc5,0xfc,0x60, +0x3e,0xbb,0x9d,0xb3,0x52,0x40,0x1b,0x29,0xa6,0x10,0xef,0x5a, +0xe8,0xdc,0x98,0x7a,0xb2,0x08,0x28,0x97,0x5a,0xa1,0xdb,0x9e, +0x1b,0xbc,0x94,0x70,0xc2,0x9c,0x86,0x8a,0xcf,0x07,0x4a,0x53, +0x98,0x93,0x58,0x12,0xef,0x54,0x44,0xc6,0x65,0x21,0xb7,0xdc, +0xd5,0xaa,0x95,0x6a,0x85,0xa5,0x97,0xba,0xd5,0x48,0xd2,0x1b, +0x5e,0x44,0x34,0x49,0x1f,0x86,0x56,0xa2,0x8e,0xb2,0x9c,0x40, +0x19,0x2a,0x3b,0x1f,0x95,0x3c,0x56,0xa1,0x53,0x28,0xa0,0x30, +0x42,0xb3,0xd2,0x28,0xa0,0x3c,0xe8,0xdf,0x24,0xe8,0x4b,0x3f, +0x59,0x92,0x0e,0xdf,0x1e,0xad,0x60,0xd0,0x0f,0x91,0x62,0xd5, +0x8a,0x52,0xa0,0xca,0x8b,0x50,0x2c,0x44,0x21,0xa4,0x40,0x8c, +0x6e,0x7d,0x2c,0xf3,0x4d,0x31,0x3b,0xf1,0xf0,0xca,0xf1,0xb5, +0x08,0xfe,0xab,0x98,0x71,0x55,0xc3,0xd1,0x9d,0x45,0x61,0xcc, +0xcc,0x13,0x56,0x46,0x71,0x10,0xfb,0x0d,0xcb,0x23,0xe9,0xde, +0x7d,0xd3,0xaa,0x58,0x9a,0xc1,0x15,0x8d,0x34,0xae,0x0b,0x9e, +0xbc,0x28,0x16,0x11,0x7a,0x6a,0xf7,0x7f,0x2b,0x3a,0x4f,0x5b, +0x0a,0x8b,0x58,0x7d,0xd3,0x12,0xf8,0xc4,0x15,0xb0,0x7a,0x12, +0x5d,0x4a,0xb7,0x57,0x73,0x10,0x24,0x00,0xd8,0x52,0xe0,0x0f, +0x5a,0x07,0x2a,0x90,0x73,0xb1,0xf4,0x37,0xac,0x86,0x95,0xdb, +0x8e,0x87,0x6b,0x0d,0xa1,0xba,0x69,0xa2,0x01,0xf9,0x6e,0x66, +0x50,0xa1,0xcc,0x51,0x81,0x87,0x96,0x27,0x2a,0x58,0x87,0x74, +0x21,0xcb,0xd1,0xec,0xdf,0xdc,0xca,0xda,0xfc,0x43,0x13,0x2c, +0xc5,0x3f,0x78,0x73,0x8f,0xd7,0xc2,0xda,0xa8,0xe4,0x91,0xdd, +0x24,0x17,0xce,0x33,0x4e,0x4e,0xde,0x66,0xad,0x2d,0x65,0xb0, +0x8a,0x9f,0x4b,0x65,0x37,0xf2,0x6a,0x4a,0xed,0xe3,0xc5,0xaa, +0x28,0xdb,0xe7,0x3a,0x99,0xad,0x74,0x30,0xe5,0xcc,0xd0,0xf8, +0xe7,0xf7,0x58,0x8b,0x2c,0xcf,0xf0,0x59,0xb0,0xc3,0x64,0xfd, +0xd7,0xae,0x72,0x2f,0xe5,0x7c,0x4f,0x39,0xbb,0x90,0xfe,0xf0, +0xc0,0x96,0xad,0x72,0x37,0x25,0x7e,0xa8,0xa4,0x94,0x92,0x45, +0xa8,0x39,0xee,0xbc,0x62,0xd0,0x27,0x85,0xbd,0x9f,0x06,0xfd, +0xd8,0x9b,0x26,0x46,0xb3,0x64,0x62,0xdb,0x96,0x8d,0xef,0x59, +0x43,0xb7,0x64,0x6a,0x0c,0xdd,0xbb,0x49,0x18,0x8b,0x1e,0x0f, +0xce,0x87,0xb8,0x6b,0xb3,0x85,0x27,0xf9,0x5d,0x21,0xea,0x02, +0x3d,0x6e,0xa2,0x9f,0xb4,0x20,0xe2,0xa4,0x37,0x18,0xb2,0x53, +0x48,0xb2,0x1a,0x1a,0xe1,0xdf,0xa8,0x2b,0x59,0xb0,0xdf,0xe9, +0xd3,0xe2,0xd4,0x87,0x5e,0x9f,0x8c,0xbf,0xc0,0xf6,0x72,0xad, +0xe5,0xcc,0xe0,0xe2,0x59,0xc7,0x7f,0xce,0x73,0xf4,0x26,0xdd, +0x75,0xc8,0x8f,0xa7,0xb3,0x97,0x34,0x48,0xc5,0x94,0xe1,0x60, +0x3a,0xcb,0x47,0x88,0xfd,0xbd,0x33,0x86,0x3b,0x98,0x92,0xc3, +0x94,0x1f,0x07,0x24,0x93,0x62,0x4a,0xc1,0x0a,0xff,0x21,0xf6, +0x45,0x4c,0x3d,0x52,0x32,0x5f,0xc7,0x1d,0x65,0x9f,0xdb,0x15, +0x68,0x08,0x8d,0x23,0x21,0x4d,0xc3,0xd0,0x91,0xed,0x90,0x21, +0x39,0xf5,0xc3,0xb3,0x43,0x4b,0xe9,0x51,0x85,0x31,0xb8,0xa4, +0x5d,0x8e,0xcb,0xe8,0xaf,0xba,0x1a,0x1d,0xe6,0xbd,0x09,0xe3, +0x15,0x9e,0xff,0x48,0xf6,0xd0,0x89,0x02,0x9a,0x5a,0x1d,0x0c, +0xf3,0x93,0xf1,0x2b,0x3a,0x6a,0x33,0x64,0xce,0xc8,0x9a,0x5b, +0xf2,0x8e,0xb1,0xdc,0x42,0xda,0x9b,0x02,0xb9,0xcc,0x76,0x26, +0x57,0xfc,0xb2,0x8c,0xb2,0x1b,0x7d,0x30,0x88,0x86,0x03,0xff, +0x0b,0xe9,0xe0,0x4f,0xa4,0xff,0xbd,0xe4,0xca,0xc9,0x25,0x04, +0x39,0x89,0xbd,0x57,0x06,0x86,0x52,0x39,0xda,0x1a,0xe5,0x9f, +0xfc,0xb2,0x2e,0x7d,0xce,0x48,0x90,0x37,0x56,0x43,0x04,0xe1, +0x6c,0xed,0x70,0x7a,0x27,0xa6,0x58,0xb2,0x8f,0xc9,0x60,0x28, +0xac,0x68,0x83,0x3f,0x6d,0xb4,0x26,0x87,0x12,0x81,0xb8,0x4c, +0x45,0xf2,0xa8,0xa6,0x9a,0x24,0x8d,0x09,0x31,0x7d,0x08,0xb2, +0x34,0x14,0xa5,0x5a,0x4a,0xf4,0xe9,0x91,0x54,0x4d,0x09,0xe7, +0x55,0xc2,0x91,0x8e,0xc4,0x2f,0x74,0xc6,0x90,0x2c,0x25,0x52, +0xcd,0x3a,0x97,0xcf,0xcd,0x3d,0x1b,0xa1,0x27,0x9f,0x8d,0xf3, +0x53,0x2d,0x88,0xf1,0x73,0xcd,0xf5,0xc3,0x8a,0x69,0x8a,0x39, +0x4d,0x71,0xde,0x76,0xe2,0x62,0x96,0xaa,0xea,0x4d,0x69,0x76, +0xd3,0x97,0xb3,0x1c,0xfe,0x00,0x1b,0x50,0x3d,0xb2,0x34,0xdc, +0x52,0x4f,0x2f,0x14,0xdc,0x03,0xd2,0xb5,0x3d,0x25,0xca,0x36, +0x7b,0xf2,0x42,0xc1,0xfa,0x48,0x1e,0xdb,0x21,0x41,0xce,0xaa, +0xc9,0xde,0x01,0x88,0xe8,0x2b,0xc2,0x87,0x56,0xa8,0x7d,0x36, +0x95,0x69,0x63,0x7b,0xd6,0xc8,0x47,0x76,0x7f,0x39,0x4a,0x5d, +0x65,0xf8,0x12,0x39,0xd5,0x44,0x8f,0xf0,0x53,0xf8,0x87,0xfe, +0xf6,0x52,0x07,0x3b,0xff,0x25,0x5d,0x5b,0x0b,0x11,0x5b,0x8a, +0x51,0x74,0x27,0x5c,0x58,0x98,0xf2,0x88,0x8b,0x83,0x64,0x45, +0xce,0x23,0x6b,0xf3,0x88,0x92,0x63,0x64,0xc3,0xb0,0x00,0x09, +0x7a,0x4d,0xed,0x84,0x0c,0x08,0x0f,0x51,0x1a,0x0f,0xfb,0xf8, +0x43,0x2d,0x4a,0xee,0x2d,0x60,0xbe,0x09,0x42,0x53,0x36,0x97, +0xb1,0x3d,0x4a,0xb7,0xae,0xc7,0x6a,0x16,0xf3,0x7a,0x1d,0xda, +0xe2,0xf4,0x6b,0xc4,0x7a,0xe4,0x64,0x70,0x85,0xfb,0x5a,0xc0, +0xcd,0x57,0xfa,0xf4,0xfa,0x7d,0x71,0xf2,0x15,0xcb,0x46,0xbc, +0xab,0x72,0xf2,0x40,0x1c,0x91,0x2f,0xea,0x98,0xbe,0xed,0x18, +0x38,0x53,0x74,0xac,0xd5,0x87,0xaa,0x6d,0x95,0x63,0x06,0xfc, +0x5d,0xa0,0xb8,0xd7,0xfd,0x16,0x82,0xf6,0x47,0xfb,0x50,0x3d, +0x42,0x70,0xbb,0xe3,0xe1,0x30,0xa7,0xfc,0x9e,0xf4,0x50,0xad, +0xc8,0x0d,0x9a,0x72,0xde,0x58,0x28,0x40,0x94,0x01,0xb5,0x70, +0x13,0x81,0xab,0x40,0xb9,0xdf,0x7b,0xf5,0x70,0xd6,0x09,0x82, +0x44,0xf1,0x10,0xf8,0x31,0xa7,0x57,0x0f,0x83,0xa9,0x28,0x11, +0xdb,0x62,0x49,0x42,0xb1,0x74,0x98,0xfd,0x76,0xf0,0x31,0x57, +0x09,0xf7,0xdc,0x6d,0xf3,0x7c,0x0d,0x5d,0xaf,0xd0,0x9b,0xaa, +0x21,0xe5,0x43,0x05,0x1d,0x49,0x0a,0x13,0xb0,0xa3,0xe5,0x62, +0xd4,0x66,0xb0,0xbe,0xec,0xb6,0xc4,0x4a,0xe5,0xcb,0x7c,0x46, +0x3a,0x5a,0x8d,0x46,0x62,0x1c,0x76,0x99,0xad,0xd4,0x3a,0x3b, +0x73,0x70,0xd6,0x50,0xb7,0x5f,0x72,0x91,0x51,0xa1,0x10,0x86, +0x29,0x56,0xe3,0x11,0x70,0xc5,0x12,0x67,0x69,0x05,0x6f,0x43, +0x5d,0xb4,0x1c,0x38,0x5a,0x00,0xa7,0x83,0x2e,0xd7,0x65,0x43, +0x0f,0xd5,0x86,0x9e,0x90,0x54,0x56,0xb1,0x50,0x52,0xd1,0x5d, +0xdc,0xd9,0x55,0xfd,0x75,0xe0,0x14,0x1b,0x3d,0xca,0x0f,0xf7, +0xd1,0x16,0xb5,0x03,0xc6,0x11,0xd7,0x73,0x37,0x4a,0xae,0xe7, +0x2a,0x4e,0xeb,0xa1,0x5e,0x57,0x5c,0xf7,0x55,0x45,0xd6,0x8e, +0x6b,0x1a,0x5e,0x20,0xe8,0x9e,0xc0,0xf0,0x83,0x7d,0x41,0x37, +0x0e,0x41,0x76,0x58,0xc9,0x5c,0xa3,0x1e,0xce,0x59,0x6e,0x30, +0xe9,0x42,0xbb,0xd2,0x2f,0xfc,0x28,0xde,0x17,0xb3,0xea,0xee, +0xb0,0x55,0x24,0x0f,0xb5,0xb1,0x5e,0x5f,0x7b,0xa4,0xb7,0x5c, +0xbd,0x1b,0x1b,0xd6,0xa9,0x73,0x80,0x3c,0x4c,0x5c,0x7a,0xd8, +0xa3,0x46,0xda,0x42,0xd4,0x63,0x6c,0xe3,0xca,0xad,0x0b,0x2d, +0xdd,0xda,0x84,0xe5,0xca,0x49,0xbe,0x63,0x79,0x89,0x65,0x08, +0x6e,0x7a,0xba,0x5e,0xe5,0x07,0x22,0x4b,0xb1,0xbf,0xcb,0x51, +0x2e,0x93,0x33,0xec,0x5a,0x40,0x05,0x48,0xcd,0xb1,0x0b,0x2b, +0x90,0x91,0x3a,0x5e,0x30,0x9d,0xd9,0xa2,0xcc,0x03,0xf4,0xda, +0xd6,0x87,0x51,0x62,0x13,0xb8,0x92,0x4f,0x95,0xd8,0x1d,0x37, +0x0c,0xaf,0x0a,0x79,0x73,0x53,0x34,0xcb,0x49,0xd9,0xed,0x63, +0x09,0x58,0x0c,0x8d,0x5b,0x2f,0x85,0x43,0xf4,0x07,0xe8,0x48, +0x80,0x1f,0x2a,0xb1,0x15,0x5d,0x7c,0xcb,0xf8,0xa6,0xff,0x19, +0x7f,0xf6,0xa7,0x57,0xe6,0x41,0xc9,0xec,0x69,0x12,0x99,0x99, +0xc1,0x96,0x0b,0x76,0x4f,0x86,0xcd,0x73,0x73,0x47,0x12,0x22, +0x6f,0xc7,0x74,0x0b,0xa5,0xa4,0xf5,0x48,0x1a,0x02,0x4b,0x66, +0x7f,0x4c,0x17,0x1f,0x84,0x21,0x99,0xe7,0x65,0xfc,0x32,0x86, +0xd2,0xf6,0x68,0x38,0x43,0x3c,0x74,0xde,0xc9,0xb9,0xf8,0x46, +0x35,0x56,0x99,0x44,0x95,0x51,0x6d,0xf8,0xcb,0x5e,0x27,0x61, +0xbd,0x85,0x5a,0x61,0xe6,0x78,0x81,0xcd,0x82,0x73,0xab,0x16, +0x19,0x48,0x13,0x36,0x69,0xe3,0x2c,0x06,0x62,0x4b,0x67,0x83, +0x51,0xa1,0x54,0x97,0xe7,0x94,0x13,0xbb,0xdc,0x72,0xf7,0x46, +0x9e,0x42,0xe8,0x0e,0x34,0x96,0xaa,0xc8,0x43,0x0e,0xd9,0x46, +0x93,0x34,0xfc,0xc4,0x49,0xc7,0x48,0x35,0xf3,0x5f,0x3a,0xf4, +0xbe,0x53,0xfc,0x32,0xd0,0xb0,0x26,0x69,0xc4,0x76,0xe9,0xbc, +0x88,0x20,0x87,0x89,0xa5,0x78,0xcc,0x4e,0x06,0x86,0x7a,0xef, +0x64,0xf7,0x6b,0x43,0x04,0x08,0xa7,0xaf,0xf4,0xf8,0x96,0xf1, +0xf5,0x67,0xae,0x6e,0x58,0xc3,0xbf,0xd8,0x53,0x6c,0x46,0x80, +0x93,0x2e,0x47,0x0a,0x9f,0x2c,0xcb,0xb4,0xd4,0x25,0x67,0x97, +0x44,0x80,0xb6,0x95,0xe6,0xd7,0x2f,0x80,0x2c,0xf0,0x01,0x29, +0x91,0x5b,0xf6,0xfd,0x06,0x04,0x1d,0x55,0x9c,0x72,0x6c,0x17, +0x64,0x87,0x62,0x14,0x49,0x0d,0x4c,0x9c,0x99,0x92,0xcf,0x04, +0x41,0xd4,0x27,0x88,0x40,0x04,0xac,0x45,0x0d,0x1f,0xd5,0xc4, +0x9d,0xd0,0x62,0xe1,0xc3,0xca,0xf7,0x68,0xea,0x62,0x05,0x68, +0x23,0x8b,0xda,0xb2,0x2e,0x20,0x14,0x71,0x9a,0xca,0x2d,0x6e, +0x40,0x12,0x73,0x07,0x95,0x44,0xfd,0x56,0xc2,0x9c,0xef,0x48, +0x18,0x64,0xf2,0x28,0xd5,0xd1,0xc3,0x67,0x7a,0x21,0xe8,0x09, +0xce,0xc9,0xdc,0x8b,0x85,0x44,0xa2,0xfa,0x36,0x11,0x7f,0xbd, +0xee,0xb3,0x29,0x59,0x3e,0x21,0xda,0x6c,0x97,0x73,0x97,0x60, +0xeb,0x42,0xed,0xc4,0xce,0x2c,0x97,0xce,0x7b,0x18,0x79,0x81, +0x60,0x53,0x47,0xf4,0xd6,0x0e,0x30,0x25,0xfa,0x40,0x83,0x2b, +0xb3,0xe1,0xc6,0x86,0x6c,0xa1,0xf5,0xc6,0xb6,0x74,0x26,0x43, +0xc6,0x86,0x61,0x5e,0xcb,0x7e,0xda,0x19,0x4a,0xaf,0x91,0xde, +0x38,0x76,0x05,0xfa,0x06,0x46,0xd5,0x8f,0x7e,0xb3,0xb6,0x73, +0x3e,0x26,0xf3,0x3b,0x6b,0x67,0xcc,0xf3,0x60,0xea,0xd4,0x2e, +0xd1,0x6b,0x30,0x2e,0x50,0x68,0xb0,0x07,0xfb,0xbb,0xf6,0x6b, +0x55,0x36,0xad,0x91,0x67,0x59,0xe2,0x30,0x3c,0x24,0x76,0x30, +0x12,0xfb,0x8a,0x82,0x50,0x5e,0x2c,0x4a,0x3c,0x83,0x85,0x1a, +0x8d,0x7c,0xbb,0xc0,0xbc,0x02,0xe6,0x52,0x4c,0x13,0x9e,0x12, +0x3f,0x36,0xbc,0xc8,0x9f,0xf9,0x10,0x44,0x33,0xe8,0xe0,0x48, +0x62,0x8d,0xd8,0x22,0xf6,0x6a,0xe1,0xd0,0xc4,0x39,0x95,0xf3, +0x67,0x39,0x1c,0x10,0x45,0x1e,0x6d,0x84,0x5d,0x0a,0x79,0x97, +0x6b,0xea,0xe8,0x85,0xfc,0x8a,0x17,0x28,0x2b,0x93,0x4a,0xe3, +0x8a,0xa4,0xf2,0x31,0x51,0x52,0x80,0x40,0xdb,0xbf,0x6a,0x08, +0x94,0x54,0x3e,0x55,0xf2,0xc7,0x54,0x3e,0x0e,0xad,0x8e,0x95, +0x22,0xab,0xcf,0xa7,0x02,0x5c,0xca,0x71,0x1e,0x92,0x8a,0xac, +0xde,0x2f,0x57,0xe0,0x14,0x9e,0x96,0xfd,0x53,0x4e,0xc7,0x70, +0xdc,0x71,0x19,0x4a,0xb5,0xc2,0xd4,0xa4,0xf8,0x6a,0xd4,0x1f, +0xe4,0x4c,0x12,0xde,0x6e,0xb9,0xc7,0xa7,0x5e,0x2d,0x28,0x04, +0x51,0x75,0x76,0x14,0xfa,0x22,0x58,0x2f,0x91,0x3a,0xa4,0x5b, +0x33,0xfa,0x7c,0x1b,0xa1,0x28,0xf1,0xe1,0x92,0xf5,0x7a,0xa5, +0x97,0x83,0xa4,0x7c,0x96,0xe7,0x1f,0x35,0x22,0xf2,0x4a,0x0e, +0x28,0xa2,0x5f,0x2d,0x28,0x84,0xc5,0xe9,0x3d,0x60,0xa9,0xa0, +0x3b,0xf9,0xaf,0x05,0x52,0x03,0x65,0x76,0x74,0x4f,0x6b,0x3f, +0xc3,0x79,0x78,0x62,0xca,0x0d,0xf0,0x7d,0xc0,0x01,0xb0,0x8e, +0xca,0x16,0x3b,0x58,0x72,0x6c,0x58,0x75,0x9b,0x41,0x37,0x46, +0xeb,0x7a,0x23,0x4a,0xc6,0xca,0xfd,0x4e,0x94,0xf6,0x93,0x10, +0xd5,0x7f,0xeb,0x0e,0x8f,0x8a,0x87,0x9c,0xa7,0x98,0xde,0xd5, +0x36,0xc3,0x7d,0x66,0xef,0x5f,0xb1,0x90,0x3d,0x2b,0x00,0xc5, +0x65,0x32,0x8c,0x92,0x25,0xc9,0x62,0xa2,0xf7,0xd8,0x61,0xc6, +0x53,0x6b,0xec,0x2f,0x8c,0xf6,0x8d,0xe8,0x7a,0xf2,0xbe,0x19, +0xcc,0xc2,0xa6,0x56,0xb5,0xd0,0x0c,0xac,0xbc,0x99,0xdf,0xdc, +0xce,0x48,0x20,0x56,0x67,0x3e,0xd3,0xd9,0xf8,0x16,0xb7,0x10, +0xbd,0x2b,0x71,0xba,0xdf,0x32,0xe4,0x14,0x22,0x23,0xb4,0x52, +0xea,0x3f,0x5e,0xc9,0x68,0x40,0xe4,0x99,0x6a,0xa1,0x94,0x66, +0xac,0x25,0x80,0x5b,0x4b,0x13,0x3a,0x58,0xd1,0xb9,0xd9,0xfd, +0x4b,0xd1,0x1f,0x05,0x15,0x2a,0x46,0xa2,0x6d,0x30,0xd7,0x0b, +0x3b,0xf8,0xce,0x96,0xdd,0xa5,0xc7,0xa7,0x94,0xa1,0xeb,0x0d, +0xd9,0x9b,0x96,0x13,0x60,0x0b,0xe3,0x62,0x4f,0x41,0x0a,0xc3, +0xb7,0x58,0x50,0xd7,0x18,0x71,0xe5,0xc3,0x6b,0x2e,0xb5,0x37, +0x73,0xcf,0x72,0xa5,0x13,0x0b,0x23,0x1d,0x0a,0x9f,0xb2,0x1f, +0x1f,0x14,0x07,0x0f,0xf7,0xd7,0x48,0xd8,0x55,0x09,0xf2,0xf4, +0x96,0x0e,0x19,0x75,0xeb,0x92,0x62,0x27,0x5b,0x8d,0x6f,0x42, +0xf2,0x91,0xa9,0x45,0x3b,0xbc,0x02,0xa6,0xa8,0x00,0x7e,0x91, +0xc7,0x03,0x83,0xc6,0x7e,0x61,0x73,0xe8,0xe4,0xe5,0xa9,0x6f, +0x68,0x41,0xfa,0xe0,0xc3,0xa6,0x76,0x50,0xb1,0xf1,0x8b,0x5e, +0x15,0xed,0x0d,0xcb,0x64,0xa7,0x77,0x1e,0xe1,0x3e,0x82,0xf6, +0x3e,0xf6,0xfb,0x41,0xb2,0x93,0x3c,0xff,0x3a,0x42,0xd2,0x82, +0x58,0x15,0x9d,0xb8,0x0d,0x9c,0xa3,0x35,0x2f,0xf0,0x95,0xa8, +0x8d,0x9f,0xef,0x5a,0x8a,0x1a,0x3e,0x46,0x4c,0xc3,0x22,0x1d, +0x0d,0x9f,0x42,0x3f,0x4f,0x21,0x91,0xe1,0xe3,0xd4,0xf1,0xff, +0x1d,0xe9,0x38,0x20,0xd3,0x25,0x9d,0x1d,0xfa,0x95,0xc4,0x5f, +0x87,0x94,0x5d,0x01,0x05,0xaa,0xc9,0xc7,0x17,0xf9,0xcb,0x7c, +0x98,0xc3,0x9c,0x2b,0xf9,0xfe,0x3f,0xde,0xdd,0x7b,0xbf,0x7b, +0xf0,0xee,0x64,0xef,0xcf,0x27,0xc7,0x99,0x0e,0xb4,0x45,0xbf, +0x82,0x3d,0xa6,0x4c,0x86,0x64,0xfa,0x14,0xfb,0x36,0x3b,0x8d, +0xf0,0xf8,0x22,0xea,0x1a,0x95,0xfa,0xe3,0xb0,0x77,0xf1,0x91, +0x53,0xbb,0x2c,0x47,0x56,0x16,0x7d,0xfa,0x3d,0x69,0x65,0x71, +0xaf,0xd7,0x54,0x4a,0x93,0x5b,0xbc,0x4a,0x94,0xbe,0xa9,0x56, +0x5f,0xbc,0x5c,0xab,0x4b,0x2b,0xdc,0xdf,0x0e,0xe9,0x8a,0x6f, +0xbd,0x3f,0xbe,0x98,0xb3,0x73,0xbe,0xf0,0x4a,0x17,0xdd,0x6f, +0xfe,0x34,0x1b,0x0c,0xa7,0xfa,0xde,0x16,0xca,0x18,0x57,0xc2, +0xa8,0x6b,0x5c,0x97,0x3b,0xd0,0x1d,0x13,0x37,0xb2,0x37,0xbc, +0x93,0x11,0x87,0xc9,0x3f,0x79,0xaf,0x9d,0x3c,0x36,0xb0,0x5f, +0x96,0x1c,0x1d,0x0f,0xa7,0x39,0x98,0x1e,0xc3,0x5c,0x3d,0xa0, +0x03,0xf9,0xd8,0x97,0x4c,0xd7,0xd6,0x5c,0xa0,0x89,0x6e,0xd3, +0xc4,0xae,0xf4,0x8d,0x3d,0x15,0xf7,0xd7,0x7d,0xa4,0xef,0x24, +0x9d,0xb0,0xc3,0xae,0x3d,0x7e,0xfc,0x72,0x38,0x86,0x69,0xf6, +0x45,0xf9,0x0b,0x37,0x23,0x8f,0x57,0xa6,0xea,0xb7,0x3a,0x03, +0xe8,0xd5,0xd8,0xef,0x98,0xc9,0xa3,0x98,0xf8,0x09,0x41,0x9b, +0xda,0x38,0xa9,0x90,0x93,0x8c,0xb2,0xaa,0x71,0xb6,0xfc,0x64, +0xe4,0x59,0x89,0xeb,0x94,0xaa,0xfc,0xa8,0xa6,0xea,0x9b,0x9a, +0xdc,0xbb,0x8a,0xb6,0xbc,0xa5,0x77,0x89,0x4b,0x2a,0x6b,0x76, +0x63,0xfd,0xc4,0x9a,0x5d,0xfe,0x95,0x35,0xb7,0xa4,0x66,0x4b, +0x86,0x1c,0xaf,0x9f,0x51,0x8d,0x70,0x27,0x9a,0xbf,0x1e,0x0f, +0x61,0xa2,0x9e,0x7c,0xb9,0xcd,0xc9,0x79,0xa1,0x98,0x06,0x77, +0x91,0x9e,0xac,0x67,0x93,0x39,0xb0,0x3c,0x39,0x69,0xb5,0xa2, +0xf0,0x7c,0x84,0x9a,0x15,0x27,0x2a,0xa9,0xe2,0x15,0x88,0x2e, +0xe9,0x75,0x53,0xb0,0x36,0x98,0x0c,0xbe,0x2a,0x75,0xcf,0x89, +0x2c,0x87,0xb0,0x31,0x55,0xc9,0xfc,0x72,0xff,0xe0,0xb2,0xaa, +0xaa,0x55,0x68,0xa1,0xff,0xdd,0x4a,0x58,0x0f,0xbc,0x5e,0x79, +0x00,0x98,0xa2,0x2c,0x0d,0x57,0x99,0xc1,0xc3,0xec,0x3b,0x53, +0x7a,0x5f,0x62,0xc2,0xae,0xb3,0x7e,0x9e,0xac,0x4b,0x9c,0xf3, +0x2f,0x84,0x52,0xe1,0x50,0x9a,0x7c,0x42,0x75,0x38,0x8d,0x9d, +0x3d,0xa5,0xa4,0x1d,0x89,0xab,0x8a,0x2a,0x6d,0x75,0x01,0xa6, +0x17,0xfe,0xd4,0x42,0x8c,0x91,0x00,0x33,0xd8,0x99,0xcd,0x50, +0x3e,0xa3,0x47,0xf1,0x8c,0x52,0xad,0xc7,0x67,0x6c,0x78,0x2c, +0x31,0x10,0xb3,0xed,0x92,0xbf,0x59,0xdb,0xe5,0x0f,0x32,0xcb, +0xdb,0xae,0x11,0xba,0x72,0x4a,0x51,0x6e,0x06,0x22,0x22,0xef, +0x7c,0x75,0xfc,0x62,0xa1,0x43,0xfa,0xc5,0x2f,0x7a,0xaa,0x2f, +0x64,0x4e,0x2a,0x08,0xa4,0xad,0x6e,0x0e,0xd7,0xec,0xf1,0x5f, +0x50,0xa8,0xa2,0xc5,0x98,0x6d,0x55,0x73,0x47,0xe3,0xd1,0x06, +0xc3,0xf3,0x2d,0xef,0x71,0x9b,0x47,0x33,0x58,0x7b,0x7e,0x99, +0x0f,0x28,0xb1,0x27,0x56,0xda,0xd3,0xa0,0x1b,0xb8,0xe9,0x4a, +0xe9,0xef,0x93,0x1e,0xbb,0x70,0x52,0xf8,0xd1,0x17,0x02,0x85, +0xa9,0xfb,0xe5,0x3c,0x17,0xca,0x51,0x9a,0x9f,0x3e,0x52,0x7b, +0x35,0xd3,0xb0,0x78,0x56,0xa2,0x53,0xce,0x8e,0xce,0xbe,0x82, +0x1a,0x53,0x9d,0x27,0x78,0x29,0x29,0x71,0x4c,0x86,0x21,0x36, +0x7e,0xfc,0x74,0xfc,0x62,0xa1,0x43,0x15,0xe3,0x17,0x00,0xa9, +0x68,0x51,0xfb,0x1f,0x26,0x6f,0x55,0xb8,0x5e,0x5f,0xd5,0x59, +0x7a,0x0a,0x54,0x60,0xb1,0xa2,0xa3,0xc8,0x06,0x34,0x32,0x98, +0x2c,0x90,0x42,0x9a,0x47,0x7b,0xc7,0x07,0x3f,0x1d,0x41,0xcc, +0x4f,0x47,0x6f,0x13,0x36,0xc2,0xfe,0xc0,0x1a,0x14,0x74,0x41, +0x65,0x1f,0x54,0x90,0xe8,0x68,0x9a,0x83,0xd0,0x35,0x04,0x2a, +0xfd,0x71,0x38,0xbe,0xf8,0x08,0xad,0x19,0xc2,0x66,0x00,0x09, +0xcf,0xb2,0xf5,0x1a,0x3a,0x42,0x20,0x77,0xe4,0x64,0xdc,0x8d, +0x2b,0xad,0x9d,0x7f,0x21,0x38,0x56,0xb2,0xaa,0xdd,0x12,0x0e, +0xcd,0x5a,0x0d,0x50,0x15,0x0a,0x0d,0x1a,0xae,0x0e,0x89,0x97, +0x2b,0x5a,0xfb,0xfa,0x64,0xff,0xad,0x45,0x1b,0x7d,0xdb,0x1e, +0xf7,0xd0,0x33,0xe7,0xdf,0x41,0x5c,0x0b,0x5a,0x56,0xd1,0x8a, +0xf9,0x68,0xda,0xbb,0x54,0x33,0x0d,0xd1,0x87,0xa9,0x36,0x07, +0x81,0x98,0x6c,0x70,0x60,0xaa,0x9e,0x65,0x14,0xb6,0xec,0x25, +0xe2,0x7b,0xaa,0xa0,0xc6,0x92,0x01,0x91,0x7f,0x6a,0x75,0x6d, +0x27,0xe8,0x20,0xf9,0xc5,0xd1,0xfa,0x54,0xea,0x45,0x5b,0xc5, +0x71,0x88,0x88,0xcf,0x8a,0xb7,0x1a,0x3a,0x2b,0x5f,0xa6,0x57, +0xcf,0xce,0xac,0x62,0x19,0x4d,0x0c,0x93,0x75,0xe6,0x1f,0x84, +0x09,0x9d,0x97,0x46,0xa1,0x5b,0x55,0xbe,0xba,0x26,0x0b,0x35, +0x04,0xb2,0x7b,0x7c,0xfc,0x0f,0xc3,0x00,0x42,0xfa,0x87,0x61, +0xfc,0xe1,0x1f,0x47,0x43,0xcf,0xc1,0x4a,0x60,0xab,0x70,0x4f, +0xcc,0xbd,0xac,0xb6,0xa9,0xfc,0x1a,0xcf,0xf9,0x52,0xff,0x69, +0x84,0x2b,0xa7,0xf2,0xbb,0x2c,0xee,0x5e,0x0a,0xaa,0x9a,0xb9, +0xf8,0xc0,0x04,0x21,0x8e,0x24,0x81,0xbc,0xe8,0x13,0xf3,0x51, +0x41,0xdf,0x41,0x58,0x13,0xe9,0x5e,0x22,0x0a,0x02,0xbd,0x53, +0xb8,0xd4,0x22,0xbd,0x9a,0xf3,0x5a,0x90,0x17,0x25,0x4b,0x2f, +0xca,0xab,0x7c,0xc4,0xb9,0xa4,0x0a,0xe0,0x9c,0xd3,0x41,0x6e, +0x55,0xba,0x39,0x33,0x7b,0xa1,0xb6,0x21,0x74,0x8d,0xac,0x53, +0xe8,0x8e,0x44,0x85,0xd7,0x02,0xa3,0x2c,0xbf,0x7d,0xa1,0xd3, +0xaa,0xd6,0x60,0x5a,0x48,0x3f,0xe2,0xe3,0x60,0x64,0x40,0x17, +0x33,0xbb,0x8c,0xce,0x7b,0xc3,0xda,0x1e,0x7a,0x5f,0xc1,0xd9, +0xdb,0x1f,0xe7,0x6c,0x1a,0x68,0x3a,0xbf,0xbd,0x1d,0x4f,0x66, +0xb5,0x37,0x74,0xc3,0x9c,0xcf,0xd0,0x75,0xdf,0x70,0x3c,0xc9, +0x27,0xb5,0x3b,0xd8,0x54,0xe3,0x88,0xfc,0x50,0xfb,0x1d,0xce, +0x6a,0x86,0x5a,0x43,0x9f,0xe6,0xc0,0xf4,0xfe,0x32,0x9e,0x93, +0xb1,0xe3,0xcb,0xc1,0x67,0x12,0xcf,0x90,0x3d,0xa2,0x49,0x56, +0x64,0x0c,0xd7,0x79,0x0d,0xeb,0xab,0xfd,0xb0,0x06,0xa8,0x22, +0xdf,0x23,0x56,0xf3,0x82,0x64,0x01,0x4c,0x1b,0xdf,0xa2,0xf7, +0x81,0x2f,0xc0,0x6b,0x6b,0x38,0xf7,0x5c,0xb7,0x00,0xd8,0xe3, +0x1c,0x33,0xcf,0x6e,0xd3,0xcd,0x4d,0x88,0x9d,0x5a,0x57,0xc7, +0x1f,0xa6,0xcd,0xf1,0xe4,0x6a,0x13,0x50,0xdf,0xc4,0x23,0xf9, +0x29,0xf2,0xe8,0xf1,0x04,0x70,0x99,0x20,0xc3,0x61,0x6f,0x98, +0x80,0xaa,0xe5,0x6c,0x90,0x9e,0xd1,0x45,0xa3,0x26,0xd1,0xa4, +0x0d,0xd1,0xb0,0x29,0x2a,0x79,0x54,0x55,0xf6,0x63,0x84,0x28, +0x30,0xa3,0x50,0x70,0xa6,0xc7,0xd5,0x46,0x52,0x06,0x4f,0xcf, +0x61,0x1e,0x45,0xe7,0x98,0xcd,0x8a,0xc8,0x41,0x1e,0x89,0xb4, +0x64,0xb8,0x58,0xc4,0xba,0xca,0x02,0x74,0x2f,0xf8,0x7f,0xb1, +0x96,0x23,0x0b,0x2f,0x6f,0x83,0x8a,0xc8,0x3d,0x13,0xea,0xc6, +0x51,0x2c,0x91,0x29,0xc0,0x0c,0x81,0xf0,0x61,0xb8,0x5c,0x0c, +0x4d,0xb1,0x05,0xea,0x12,0xc1,0xbd,0xda,0xa2,0x24,0x6b,0x9d, +0x09,0x0d,0x56,0x50,0xd8,0xaa,0x8b,0x75,0x38,0xbc,0xc2,0x28, +0xbc,0x7d,0xb3,0x1e,0x34,0x85,0x2b,0xe7,0x72,0x05,0x9b,0xee, +0x7c,0xd8,0x41,0x49,0x99,0xc2,0x5b,0x31,0x90,0x42,0xb7,0x18, +0xdd,0x5b,0xf2,0xed,0x14,0x59,0x99,0x6a,0x9a,0xf6,0xf9,0xaf, +0x26,0x03,0xf5,0xee,0x66,0x34,0xbf,0x11,0x45,0x0d,0x7f,0xf6, +0x47,0x6e,0xc6,0xbc,0xa3,0x3d,0x4a,0x41,0xc2,0x39,0xbd,0x80, +0xaf,0xe1,0x2e,0xdd,0x16,0x10,0x6e,0xef,0x7b,0xd3,0xf7,0x51, +0x83,0xf2,0x27,0xdd,0xac,0x70,0xcd,0xa0,0x7b,0x50,0xd5,0x23, +0x57,0x13,0x05,0x70,0xd0,0xa6,0xf7,0xc2,0x9f,0xab,0x20,0x86, +0x03,0xae,0x3a,0xd3,0xc3,0x95,0xc7,0x1b,0x45,0xc0,0x04,0x74, +0x05,0x9e,0x21,0x54,0xbb,0x49,0x2a,0x81,0x5c,0x12,0x15,0x85, +0xfc,0x99,0x5f,0xd0,0xaf,0x7a,0xfe,0x6e,0x9d,0x0b,0x6b,0x26, +0xea,0xb9,0x26,0xa7,0x7a,0xb6,0x29,0x0f,0x8c,0x73,0x88,0x9d, +0xb0,0xb3,0x7b,0x43,0xbb,0x6c,0x3e,0x9f,0x3c,0x66,0xde,0x84, +0xea,0x2e,0xbd,0x51,0x7f,0x32,0x1e,0xf4,0x51,0x71,0x2b,0x8e, +0x37,0x25,0x54,0x8b,0xcf,0xfa,0x8d,0x64,0xb3,0x99,0x7f,0xce, +0x2f,0x62,0x3f,0x68,0xb6,0x9a,0xe6,0xa8,0x77,0x37,0xb8,0x42, +0xcf,0x81,0xe8,0xb6,0x37,0x69,0x82,0xb8,0x32,0xd9,0xb9,0x22, +0x1d,0x09,0x3c,0xc0,0x4c,0x4e,0x5b,0xb0,0x64,0x9d,0x8f,0x3f, +0xe7,0x79,0xb6,0xf9,0x23,0xfe,0x6c,0x0e,0x9a,0x33,0xbc,0xaa, +0x79,0x1c,0x80,0xb1,0x0d,0x08,0xd8,0x38,0x79,0x07,0xe6,0xc6, +0x1c,0xe5,0x57,0xf9,0xe7,0x6c,0xf3,0xe7,0x78,0x7f,0xfc,0xf7, +0xc5,0xa7,0xfc,0xfc,0xe3,0x60,0xb6,0x38,0x58,0xdc,0x4c,0x93, +0xb8,0x93,0x9d,0xee,0x6c,0xfc,0xb5,0x9b,0x6c,0x42,0xd5,0xfd, +0x2f,0xc7,0xb3,0x2f,0xc3,0x3c,0x73,0x1c,0x10,0xa3,0xea,0xf5, +0x20,0xd8,0x9c,0x62,0x16,0x20,0xf7,0xde,0x68,0x3a,0xa0,0x0b, +0x6b,0xbc,0xe1,0x00,0xa1,0x88,0x79,0xde,0x94,0x1c,0x93,0x5d, +0xc6,0x0e,0x98,0x7f,0xef,0x8b,0xe6,0xc3,0x90,0x73,0xfb,0x24, +0x7b,0x88,0x91,0x29,0x24,0xb9,0xf7,0x30,0x2f,0xea,0xa6,0xaa, +0x91,0xe0,0x33,0x25,0x7c,0xf7,0x1c,0xc4,0xea,0x80,0x77,0xb3, +0xd7,0xc2,0x9d,0xf8,0x4f,0xb7,0xb0,0xa7,0x20,0xda,0x4b,0x1a, +0x55,0xd9,0x5a,0x89,0x9c,0x85,0xe8,0x44,0x60,0x7e,0x01,0xfc, +0xe8,0xbf,0xa8,0xb7,0x0e,0x6e,0x7b,0x17,0xc0,0xc5,0x22,0x8d, +0x7f,0xbd,0x1e,0x71,0x57,0xe2,0x4d,0x9d,0xee,0x8f,0x35,0x24, +0x76,0x1b,0x0e,0x8a,0x2c,0x16,0x1a,0x78,0x23,0x3a,0xa9,0xce, +0x96,0x04,0x1d,0x0a,0xe0,0x5c,0xf0,0x41,0x68,0x3b,0x95,0xb9, +0x12,0xb3,0x26,0x04,0x8a,0x9b,0x70,0x87,0x66,0xbd,0xee,0xeb, +0xc0,0x6b,0x38,0xd5,0x80,0xc1,0x54,0xf6,0x0a,0x15,0x23,0xdf, +0xe4,0x26,0x7b,0xc4,0x03,0x5c,0x9f,0x50,0xd2,0x21,0xe9,0x5c, +0x67,0xdc,0xc3,0x84,0x45,0xd3,0x0f,0xe9,0x5a,0xbc,0x66,0x49, +0x5d,0xa2,0x16,0x8b,0x62,0x0c,0x5d,0xfb,0x1d,0xa3,0x1d,0xc0, +0xc5,0xe2,0xfb,0x17,0xae,0x61,0x34,0x6f,0x12,0x03,0xe2,0xfc, +0x35,0x2b,0xb5,0xa5,0xd1,0x78,0xe4,0x43,0xd8,0x21,0x02,0x08, +0xe4,0xaf,0x35,0x87,0x9d,0x16,0x5e,0x16,0x8b,0xca,0xe8,0x17, +0xff,0x4e,0x60,0xf7,0x90,0x03,0xa8,0xfb,0xdb,0x3b,0x62,0x17, +0x78,0x28,0x41,0xee,0x2c,0x60,0x0b,0x4c,0x51,0xf5,0xfa,0xef, +0x60,0x2f,0x8c,0x22,0x96,0xf5,0x0b,0xc4,0x0a,0xb6,0x3f,0x59, +0x91,0x29,0xd6,0xac,0x44,0xee,0x47,0x44,0x0b,0xbb,0x3f,0xb8, +0xdb,0x1b,0xde,0xf8,0xd9,0x77,0x01,0xc4,0x39,0xcb,0xf7,0xd8, +0x78,0x5d,0x1c,0x41,0x32,0x9a,0x1c,0x2f,0x17,0xcf,0xa0,0xa9, +0x51,0x83,0xbe,0x71,0x7e,0x31,0x98,0xa5,0x3e,0xbc,0x0f,0xf3, +0x2f,0xd1,0x3a,0x52,0xea,0xaa,0xa1,0x0d,0x28,0xd0,0x36,0xef, +0x62,0x3b,0x2b,0xe2,0x41,0x54,0x41,0xf5,0xbc,0xbb,0x3c,0x85, +0xe9,0xae,0xc9,0x2e,0x0d,0xd8,0xa5,0x22,0xa3,0x54,0x7d,0x2b, +0x1a,0x49,0xfd,0x67,0x28,0x69,0x9f,0x0c,0x6e,0xf2,0xf1,0x7c, +0xf6,0x24,0x43,0x26,0xee,0x09,0x52,0xb4,0xfe,0xcb,0x57,0x18, +0x36,0x91,0x62,0x66,0xfd,0x97,0x2a,0x23,0x27,0xfe,0x90,0x8e, +0x31,0xc1,0x57,0x27,0xfd,0x7c,0xd8,0x43,0xe3,0x41,0xe8,0x39, +0x64,0x07,0xef,0x85,0xc4,0xb0,0x3d,0xe7,0x78,0xd3,0x77,0x76, +0x28,0xb2,0xf5,0x5f,0x9a,0xf6,0x5d,0xa4,0x35,0x95,0x55,0xb4, +0x69,0x61,0xa6,0x1f,0x07,0xb7,0x04,0x45,0x39,0x10,0xd6,0xb0, +0xeb,0xf5,0x35,0x15,0x74,0x8a,0x94,0xb6,0xb2,0x6c,0xb5,0xce, +0x03,0x5e,0x62,0x95,0x4c,0x3b,0x5c,0x8e,0x02,0x0d,0xc1,0xa2, +0x95,0x0e,0x98,0x2f,0x0f,0xaa,0x11,0x88,0xb2,0xab,0x2d,0x36, +0x3d,0xf5,0x16,0xbc,0x1c,0x4a,0xdd,0xa5,0x6b,0x53,0xa8,0xb2, +0xc1,0x97,0x68,0xc9,0x92,0x7b,0x30,0x31,0x15,0x65,0xb3,0x72, +0x2f,0x4e,0x4f,0x3d,0xe4,0xcc,0x9b,0xf8,0x28,0x9a,0x04,0xc1, +0x9c,0x6a,0x1b,0x2d,0x65,0x9a,0x17,0x78,0x08,0x34,0xcc,0x8a, +0x56,0x55,0xbc,0x00,0xc4,0x61,0x90,0x1d,0xcb,0xc8,0xd0,0xcc, +0xb1,0xb0,0x3b,0xf1,0xc3,0x8d,0xb6,0x3d,0x18,0x71,0x8d,0x79, +0x9f,0x1c,0xc9,0x3f,0xa1,0xb7,0x4c,0x38,0x82,0x82,0x71,0x5c, +0x91,0x35,0x21,0xd7,0x5d,0x46,0xc2,0xc1,0x44,0x59,0xff,0x49, +0xee,0x34,0xaa,0xa7,0x8a,0x36,0xb5,0x61,0xcb,0xa8,0xcb,0x0f, +0x16,0x78,0xe5,0x3c,0x1f,0xdf,0x6f,0x43,0x9c,0xed,0xc8,0x56, +0xeb,0x05,0x31,0x2f,0xe0,0x91,0x10,0x8b,0x76,0x97,0x80,0x97, +0xbf,0xc3,0x2d,0xd6,0x34,0x9f,0xed,0xcc,0x80,0xb5,0x9f,0xcf, +0x67,0x20,0xca,0x61,0xb9,0xc8,0xe0,0xdf,0xc4,0x58,0x18,0x3a, +0x37,0xa7,0x3c,0x19,0x04,0x61,0xd4,0xb9,0xc7,0x40,0x5a,0x01, +0xc9,0xd0,0xa9,0xfc,0xc5,0x78,0x58,0x4c,0xb4,0xf1,0xe6,0x7a, +0x3c,0x9d,0x95,0x4a,0x42,0xdc,0xb2,0x0a,0x1c,0x91,0x51,0x18, +0xbf,0x92,0xe7,0xf6,0x60,0x60,0xf9,0x0d,0x09,0xde,0x26,0xb9, +0x67,0xa7,0xbc,0x22,0xe1,0x6e,0x81,0xf6,0x79,0x84,0xe4,0xda, +0x96,0xdd,0xaa,0x38,0xb3,0x33,0xf2,0x6b,0xf4,0xd5,0x97,0xb6, +0xc8,0xf2,0xcb,0x1c,0xc4,0xbb,0x9f,0xec,0xcd,0x95,0xec,0x7d, +0x4a,0xcf,0x63,0x7c,0xbe,0x8e,0x7f,0x6d,0x6b,0xa3,0xb0,0xda, +0xd4,0x07,0x0b,0x9b,0x25,0xdb,0x41,0xe8,0x5e,0xdf,0xb6,0xc2, +0x45,0xba,0x2d,0x14,0xf6,0x54,0x90,0x83,0xba,0x2e,0x64,0xce, +0xff,0x45,0x4d,0xae,0x24,0x38,0x92,0xcd,0xf1,0x71,0x13,0x65, +0x51,0xa7,0xdd,0xeb,0xaf,0xc8,0xca,0x95,0x2b,0xb4,0x7e,0xcb, +0x5f,0x01,0x5d,0x5e,0xe1,0x85,0x39,0xab,0x95,0x98,0x4b,0x36, +0x01,0xa5,0x74,0xc9,0xb8,0x40,0xf3,0xd2,0xda,0x86,0x82,0x5c, +0x8d,0xe9,0xfc,0x12,0x57,0x1a,0x9b,0x99,0xb7,0xd8,0x14,0x97, +0x45,0x5c,0x63,0x64,0x2f,0x81,0x19,0x78,0x66,0x3f,0x56,0xbd, +0x77,0x2d,0xbf,0x63,0x2d,0x1b,0x07,0xe3,0x4d,0x99,0x45,0x2c, +0x38,0x14,0x54,0x58,0xc1,0x16,0x05,0x6f,0xa6,0xa5,0x51,0x11, +0x5f,0xa7,0x5f,0x7c,0x89,0x8c,0xfd,0x62,0x04,0x13,0x95,0xa7, +0x4f,0x27,0x39,0xf8,0xb7,0x9c,0x66,0xed,0x99,0xf1,0x6f,0x39, +0x9d,0x4d,0x28,0xe3,0xdf,0x72,0xda,0x70,0x70,0x03,0x62,0xda, +0x38,0x32,0xf2,0x51,0x91,0xc3,0x6e,0x51,0x20,0x8f,0xfd,0x2c, +0xe7,0x12,0x95,0x6b,0xc3,0xbf,0xe5,0xf4,0xf1,0x04,0x8f,0xef, +0xa0,0x89,0xf2,0x51,0xce,0x31,0x47,0x39,0x9c,0xeb,0x71,0x9f, +0x92,0xab,0xe8,0xa8,0xec,0x95,0x75,0x57,0x56,0xec,0x7e,0x7e, +0x2a,0xe5,0x0d,0x93,0x1b,0x7c,0xa9,0xc6,0x7e,0xd8,0x13,0x71, +0xcd,0xce,0xa6,0xb7,0x28,0xa3,0x3b,0x7d,0xa7,0x90,0x18,0x42, +0xce,0xfb,0xe8,0x14,0x87,0x6c,0xaa,0xb7,0xc5,0xca,0x9b,0x8f, +0x6c,0x5e,0x5c,0xe7,0x15,0x16,0x0c,0xec,0xde,0xe5,0x43,0xb6, +0xd5,0xfe,0xf0,0x83,0xca,0x2e,0xd7,0xb1,0x1f,0xf8,0x2a,0x76, +0xcd,0xa7,0x9c,0x7e,0xe8,0x5a,0xbb,0x13,0x4e,0x02,0x94,0x8f, +0xad,0xa5,0x91,0x29,0xad,0x91,0x2f,0x58,0x7d,0x63,0x37,0xdf, +0x14,0x79,0x3e,0x1e,0x0f,0xf3,0x1e,0xc4,0x41,0x0d,0xad,0x2c, +0x0b,0x0a,0xb9,0xef,0x2c,0x78,0x22,0x46,0x86,0xe0,0xef,0x0b, +0xe7,0x0e,0xa2,0x60,0xef,0xd2,0x97,0xb2,0xed,0xb1,0x56,0xe5, +0x9e,0x04,0x8c,0x9c,0x2f,0xc4,0x11,0x3b,0x1d,0xc3,0xdd,0xd5, +0x5b,0x24,0x19,0xde,0x5d,0x19,0x8c,0xc7,0x97,0x22,0x61,0xb4, +0xf3,0xb2,0x4b,0x45,0x5e,0x6c,0xb4,0x96,0x3c,0x57,0xf3,0xde, +0x04,0xf6,0x7c,0x15,0x55,0x55,0x3d,0x0e,0xc4,0x14,0xd8,0x68, +0xad,0xd1,0x1d,0xa1,0xf2,0x49,0xbc,0xe5,0xfa,0x97,0xe1,0x39, +0x30,0x7e,0x77,0x97,0xb4,0x43,0x63,0x7e,0xe4,0x89,0x24,0xec, +0x59,0x0a,0x09,0x85,0xa7,0x95,0x76,0xfe,0x5c,0xdf,0x78,0x3c, +0x79,0x74,0xf8,0x85,0x65,0x94,0x86,0x75,0x70,0x43,0x82,0x0c, +0x0f,0x41,0xb2,0x43,0xa0,0x2c,0xdb,0xff,0x51,0x1e,0x2f,0x01, +0xb2,0xd8,0x1f,0xeb,0xe8,0x24,0x8b,0xa3,0x7d,0xd3,0xeb,0x75, +0xdf,0xe8,0x53,0x4e,0x14,0x2f,0x26,0x89,0xa3,0x35,0x47,0x7d, +0x84,0x0c,0x4d,0x84,0x28,0xd5,0x8f,0xde,0x06,0x3f,0xa0,0x01, +0x6d,0x65,0x31,0x1f,0x4d,0x5e,0x7a,0xb0,0x83,0x22,0xc4,0xb6, +0x83,0x18,0x9a,0x23,0x04,0x31,0x65,0xd9,0x2e,0x99,0x4d,0x94, +0x79,0xfa,0xf4,0x0e,0xf7,0x65,0xb2,0xfb,0x75,0x15,0x5a,0x86, +0xdd,0xad,0x8c,0xa8,0x63,0x37,0xa9,0x9a,0xa0,0xab,0x32,0xb4, +0xe2,0xdf,0x59,0x0b,0xb5,0xd2,0x7d,0x16,0x7e,0xae,0x25,0x8b, +0xed,0xec,0x1a,0x33,0xb7,0xd5,0x9c,0x66,0xd7,0x94,0xd5,0x47, +0x52,0xd2,0x31,0x77,0xee,0xe0,0x4c,0x20,0x92,0xb9,0x4a,0xb2, +0x86,0x13,0x83,0xc0,0x16,0x56,0x1c,0xb9,0x73,0x72,0x7f,0x63, +0xfd,0xcf,0x44,0x46,0xfc,0xfe,0x30,0x4e,0x64,0xc3,0x76,0x15, +0x66,0x6d,0xc5,0x55,0x3c,0xab,0x29,0xd6,0xa6,0xfa,0x52,0x0a, +0x14,0xec,0x4e,0xb2,0x5b,0x0b,0x3b,0x02,0xcc,0xb1,0x41,0x62, +0x39,0xed,0x1a,0xe6,0x8f,0xfc,0xfa,0x55,0xb1,0x46,0x6f,0xaf, +0x8f,0xdd,0x61,0x01,0x73,0x6c,0x17,0x79,0xae,0x7b,0xb7,0x63, +0xe1,0x31,0x32,0x72,0xe2,0x67,0xd7,0x01,0x49,0x53,0x46,0x11, +0xc2,0x15,0x15,0x55,0x99,0x2e,0x7a,0x43,0x6b,0x50,0x80,0x8e, +0xe0,0xa7,0x99,0x44,0x36,0xdf,0xfd,0xb4,0xff,0xe3,0xde,0xd1, +0xfb,0x57,0x07,0x47,0xfb,0x3b,0x27,0xc7,0x25,0x1f,0x91,0xbd, +0x1b,0xb4,0x4f,0xe8,0xd6,0xe8,0xe3,0x2f,0x37,0xe7,0x63,0xa7, +0xd3,0x54,0xd3,0x9b,0xf7,0x42,0x16,0x7c,0x02,0x19,0xc4,0x64, +0x52,0x73,0x73,0xf7,0xa7,0xa3,0xa3,0xbd,0x77,0xbb,0x7f,0x79, +0x7f,0xfc,0x97,0xfd,0xc4,0x70,0xac,0xb8,0x3c,0x97,0xda,0x6c, +0xce,0xc3,0x9d,0x93,0x93,0xbd,0xa3,0x77,0xc7,0x68,0xc5,0xcf, +0xc6,0xfd,0xfe,0xe8,0xe0,0xa7,0xc3,0xf7,0xc7,0x7b,0x87,0x2e, +0xe6,0xe5,0xde,0xee,0x9b,0xfd,0x9d,0xb7,0x14,0xb7,0x9d,0x80, +0x68,0x73,0x3b,0x44,0x3f,0xe0,0x9b,0x67,0xf3,0xad,0xad,0x9d, +0xef,0x37,0xaf,0x8a,0xc8,0xab,0x8e,0xd2,0x0b,0xf7,0x3f,0xd4, +0x4d,0x0c,0xc8,0x5c,0xa2,0xa7,0x6c,0x08,0x1f,0x0f,0xfe,0xee, +0x49,0x32,0x68,0xa1,0xcd,0x58,0x6c,0xe1,0xd6,0x53,0x5b,0x18, +0x54,0xa1,0xbd,0x99,0x56,0xd4,0x02,0x74,0x8e,0x57,0x45,0xe6, +0x6a,0x32,0x9e,0xdf,0x1e,0xe7,0xb7,0xb0,0x11,0xbb,0x18,0xdc, +0xf4,0x86,0xf8,0x19,0xa2,0xca,0x1e,0xea,0x7b,0xef,0xa4,0x64, +0xb2,0x58,0xe0,0x93,0x78,0x7c,0xda,0x95,0xdb,0x28,0x61,0x76, +0x51,0x44,0x32,0x03,0xe4,0xc6,0x0b,0x92,0xc1,0x5d,0x9e,0x6d, +0xbd,0xe0,0x1c,0x6d,0xfe,0xc9,0xf6,0xd1,0x02,0x74,0xef,0x7c, +0x6a,0x0b,0xf2,0xc3,0xea,0xf9,0xcd,0xf1,0x6c,0x92,0x71,0x54, +0x23,0x8a,0xa4,0x6d,0x79,0xff,0x04,0x17,0x4f,0x08,0x83,0xc0, +0x4d,0xee,0x73,0xe9,0x40,0xe9,0xf3,0xed,0x78,0x94,0x93,0x77, +0x14,0x3c,0x22,0xda,0x68,0xe1,0xbb,0x51,0x02,0xe0,0xbd,0xd3, +0xe3,0x2d,0x33,0xfb,0xcb,0xa4,0x63,0x52,0x49,0x66,0x05,0xb9, +0xcd,0xf8,0xf4,0xac,0x7f,0xd6,0xec,0x36,0x92,0x3c,0xde,0xe8, +0x24,0x7c,0xf6,0x9c,0xb4,0x6f,0xf8,0x3d,0x5e,0xb4,0x11,0x89, +0x16,0xde,0xe9,0x76,0x17,0xd5,0x25,0xf0,0xeb,0x79,0xf7,0x85, +0xee,0x91,0x46,0xab,0x23,0x18,0x47,0x5b,0x51,0x1a,0x07,0xc8, +0x72,0x42,0x88,0xe7,0x16,0x69,0x0e,0xa8,0xa8,0x44,0x43,0x43, +0xff,0xbf,0xdc,0x72,0x58,0xe9,0xeb,0xf5,0x96,0x74,0x18,0x4c, +0x91,0x22,0x60,0x88,0x05,0x41,0xe1,0xd5,0xe0,0x33,0x4c,0xa8, +0x60,0x84,0x94,0xf3,0x3b,0x1b,0xff,0x36,0x1f,0x65,0xb1,0xb4, +0x9b,0x0d,0x6e,0x2b,0x3a,0xc1,0x23,0x73,0x54,0x3f,0x49,0xdc, +0x2a,0xa6,0xa6,0x6a,0x00,0x19,0xb1,0x50,0x61,0x1e,0xbe,0x9b, +0xc1,0x28,0xe6,0x8f,0xde,0xe7,0x58,0xc8,0x08,0x23,0x5f,0x41, +0x4e,0xa3,0x10,0x48,0x8c,0x4b,0xec,0x7d,0xc6,0x44,0xb9,0xfa, +0xbf,0x1d,0x7f,0x62,0x40,0xf0,0x11,0xb7,0xb6,0x42,0x6a,0x0b, +0x08,0x05,0x48,0x73,0xd4,0x17,0x52,0xf9,0x0e,0x72,0x27,0x9b, +0xf0,0xa7,0xad,0xdb,0x49,0xa2,0x95,0xd0,0x52,0x45,0x43,0xcd, +0xa7,0x6b,0x58,0x40,0x33,0x9b,0x1b,0x66,0x52,0xdb,0x95,0x74, +0x91,0xdc,0x19,0x82,0xda,0x14,0xd5,0xa1,0x69,0x4e,0x64,0x16, +0xfb,0xe1,0x15,0x62,0x66,0xc2,0x48,0x8a,0x43,0x02,0xa4,0x1a, +0xac,0xcf,0x91,0x8c,0x8b,0x36,0xe8,0x6f,0x82,0x2f,0xc5,0x32, +0x9d,0xbe,0xc1,0xc9,0xc1,0x1b,0x7a,0xc8,0x23,0xba,0x89,0x68, +0xf1,0x35,0x86,0xe0,0xc6,0x20,0xf9,0x15,0xe5,0xab,0xd7,0xd1, +0x57,0xcb,0xa0,0x40,0x0a,0x8d,0xcc,0x4e,0xd9,0xc4,0x84,0xf1, +0x5c,0x95,0x48,0x3b,0x03,0xf6,0x40,0x38,0xc8,0x00,0x22,0xc8, +0x2c,0x1a,0x0d,0x5f,0x5b,0x80,0x1c,0x54,0x3b,0xfc,0x67,0xd4, +0x4b,0xcd,0x73,0xfd,0x2c,0xd0,0x7f,0xd0,0xa3,0xdc,0x76,0x33, +0xa0,0x81,0x73,0xa8,0xad,0xd3,0x60,0x0e,0x6e,0xa1,0xec,0x16, +0xc6,0x15,0x30,0xf1,0xbc,0xaa,0xe1,0xea,0x71,0x97,0x18,0xe1, +0xdc,0x58,0xfa,0xad,0xbd,0x75,0xa3,0xed,0xf9,0x53,0xc7,0x0e, +0xe7,0x28,0xbf,0x3a,0x9c,0xe4,0xa9,0x0d,0x42,0x9f,0x41,0x30, +0x31,0xaa,0x90,0x46,0x20,0x48,0xa8,0x86,0x76,0x3c,0xbf,0xd4, +0xd0,0x20,0x68,0x0b,0x7d,0x18,0xc3,0xfc,0x81,0xc9,0xa7,0xdc, +0x6a,0xf6,0xfa,0x9e,0x37,0x9b,0xfe,0xe0,0x6a,0x30,0x9b,0x1a, +0x10,0xef,0x6e,0xe4,0x15,0x4d,0x7e,0x05,0x4c,0x90,0x7a,0x95, +0xd8,0x29,0x59,0x22,0x80,0xa8,0x0d,0xda,0x5c,0x66,0x1b,0xf0, +0x27,0xa1,0x2f,0x9e,0x0a,0x38,0x83,0x6c,0xa7,0x33,0xac,0x76, +0x42,0xa9,0x5b,0x9c,0xec,0x2e,0x0b,0x07,0x04,0x09,0x52,0xb0, +0x80,0xf4,0x9e,0x2f,0xbb,0xc1,0x65,0x13,0xb4,0x9a,0x70,0x85, +0x05,0x95,0x65,0x68,0xe8,0x07,0xb1,0x8e,0x41,0xe7,0x0f,0x53, +0x9c,0x1e,0xe3,0xcb,0xcb,0x69,0x3e,0x13,0xb4,0xad,0xe3,0x27, +0x8a,0xcb,0xf8,0x67,0xb1,0xd8,0xf2,0x27,0x06,0x7d,0x67,0x3f, +0x88,0xc5,0x1c,0x0c,0x9f,0xe2,0x1d,0x2a,0xe0,0x88,0x4f,0x13, +0xbc,0xf7,0x59,0x2e,0xfc,0x62,0x6b,0xb1,0xa0,0x9c,0x2f,0x36, +0x38,0x22,0xb1,0x6f,0xa7,0x1b,0x02,0x9e,0x2d,0x26,0xcb,0x83, +0xe7,0x8d,0xd6,0x76,0x26,0xf1,0xee,0x8d,0x75,0x6b,0x1b,0x47, +0xc0,0xf6,0x34,0x4b,0x7f,0x84,0x39,0xa1,0xbc,0x0c,0x9b,0x07, +0x7c,0x33,0x68,0xe1,0xf5,0x78,0x32,0x7b,0x05,0x14,0x50,0xde, +0x61,0x63,0x6e,0xbb,0x18,0x3f,0xdc,0x22,0x32,0x6b,0xe3,0x76, +0xf2,0xb1,0x83,0xd9,0x89,0x8e,0x5f,0x1f,0x1c,0x9d,0x70,0xbe, +0x94,0xef,0xaa,0x03,0xf1,0x60,0x7a,0x0a,0x25,0xbb,0xa7,0x04, +0xb7,0xab,0x10,0xc5,0x43,0xce,0xbf,0xc2,0x8a,0x22,0x98,0xfa, +0x3e,0xfd,0x3b,0x9a,0x7d,0xdb,0x68,0x7d,0xd7,0xb7,0x9a,0x0c, +0x90,0x11,0xe3,0x0e,0xa8,0x47,0x62,0xea,0x87,0x7e,0xde,0xc7, +0xb2,0x19,0xc6,0xbf,0xc8,0xb6,0x3a,0x51,0x23,0x4a,0x81,0xca, +0xdc,0x6c,0xb1,0x19,0x1a,0x99,0xef,0x34,0x64,0xca,0xa7,0x54, +0x00,0xf2,0x5f,0x0e,0xc7,0x63,0xd8,0x8c,0x44,0x17,0xf9,0x60, +0x18,0x75,0x63,0x8c,0xde,0xfc,0xcd,0x56,0x02,0x62,0x56,0x23, +0x2c,0x41,0xeb,0x3d,0x26,0xff,0x8a,0x93,0x95,0xb1,0xf0,0x9b, +0xdb,0x1b,0x85,0xbc,0xef,0x46,0x6b,0x91,0x5a,0xf0,0x7f,0x3d, +0x9e,0x4f,0xa6,0x71,0xf2,0x43,0x6b,0xbb,0x63,0xc5,0x9e,0x9d, +0xfd,0xc3,0xfd,0x63,0xf2,0x66,0x1f,0x44,0xb4,0xba,0xe1,0x38, +0x16,0xc5,0x37,0x97,0x88,0x87,0x3f,0x7c,0xe7,0x76,0x32,0x7e, +0x09,0x19,0x63,0xde,0x4f,0x29,0xb1,0xa1,0xed,0xae,0x59,0x39, +0x49,0xe4,0x87,0xa3,0xf7,0x6f,0x8e,0x0f,0x7e,0xfb,0x9b,0xad, +0xd6,0xfb,0xe3,0x93,0x23,0x7b,0x03,0x05,0x00,0x48,0xc9,0x9a, +0x20,0xa1,0xb3,0xb2,0xbf,0x23,0xca,0xb0,0x94,0xcc,0xfe,0xbe, +0x3f,0x18,0xc1,0x2f,0xd1,0x14,0x5b,0x88,0x60,0x99,0xe2,0xb7, +0xdd,0x0e,0xb5,0x0e,0xc6,0xe3,0xa7,0x93,0xdd,0x57,0xf3,0xe1, +0xf0,0x2f,0xb0,0x3b,0x49,0x6d,0x9c,0x8d,0xa0,0x83,0xec,0x07, +0x0a,0x52,0xcf,0xb8,0x52,0x14,0x62,0x51,0xe6,0xf4,0x77,0x68, +0x68,0x44,0xf0,0xc0,0xeb,0x76,0x1b,0xdb,0xe0,0x8f,0x16,0x3a, +0xf2,0x62,0xec,0xaa,0x52,0x5b,0x5d,0x98,0xf8,0x1e,0x69,0xb6, +0x02,0x44,0x63,0xe4,0x73,0xe3,0xbd,0xbb,0x0f,0x6d,0x77,0x93, +0x8d,0x96,0x0a,0x3f,0xef,0xca,0x72,0x7f,0xad,0x2a,0xf8,0x1e, +0x96,0xda,0xad,0x64,0x83,0xd1,0x32,0x37,0x2a,0xe5,0xd7,0x36, +0x05,0x30,0x32,0x53,0x95,0xf0,0x1b,0x4a,0x30,0x37,0x53,0x2d, +0x12,0xb4,0xf2,0xe7,0xdf,0xd1,0x29,0xec,0xab,0xe1,0xb8,0x37, +0x8b,0xa3,0xad,0x66,0xd4,0x90,0xfc,0xff,0x4e,0xf9,0xbd,0x9b, +0x3c,0xdf,0x81,0xaa,0x11,0xd7,0xe6,0xc6,0x4c,0x01,0x26,0xb7, +0xd1,0x2e,0x11,0x3c,0xce,0x74,0xd8,0x12,0x8c,0x32,0x2a,0x00, +0x9c,0xf5,0xef,0xbf,0x5f,0x26,0x1b,0x1d,0xf8,0x38,0xeb,0xbb, +0xdf,0xb8,0x93,0x9e,0xb8,0xaf,0xb4,0x53,0xf5,0x79,0xd6,0x24, +0x11,0x33,0xe9,0xe0,0xff,0xf1,0x5f,0x17,0xf1,0x69,0x63,0xa3, +0x9b,0x70,0xb2,0xcd,0x86,0x49,0xeb,0x9b,0xa5,0x8d,0x83,0x9a, +0x13,0xd6,0xe0,0x9b,0xe8,0x55,0xcc,0x0a,0x82,0x31,0xd2,0x2a, +0x67,0x94,0x9d,0x14,0x48,0x34,0x37,0xb0,0x5f,0x9c,0xdf,0xbc, +0x64,0x85,0x37,0x4e,0xb3,0x1b,0x96,0x97,0x3b,0x27,0x7b,0x27, +0x6f,0xf6,0xf7,0xec,0x96,0xe5,0x94,0xd3,0xa1,0xe7,0xf8,0xc3, +0xf8,0xfb,0x68,0xe4,0x29,0x68,0x0d,0x02,0x29,0x5c,0xf6,0x39, +0xd0,0x23,0x6f,0xde,0xfd,0x9e,0x75,0x2c,0x28,0xbd,0x83,0xa3, +0x45,0x5f,0x69,0x69,0x5a,0x51,0x34,0xd0,0xc9,0x54,0x18,0x42, +0x00,0xd0,0x4d,0x19,0xc9,0x05,0xfb,0x0a,0x15,0x54,0x4c,0x40, +0x44,0x0a,0x42,0xae,0x9d,0xf0,0xac,0xc4,0x46,0xd8,0x06,0xbc, +0x3f,0x3e,0x7c,0xfb,0xe6,0x84,0x75,0x20,0xa4,0xc3,0xb8,0xa7, +0xd0,0x7f,0x0f,0x76,0x91,0x3c,0xac,0xa3,0x80,0xf4,0x61,0xcb, +0x8a,0x31,0x99,0x2c,0xe6,0xe3,0x5b,0x72,0x92,0x5d,0x5a,0xf3, +0x5d,0x3e,0xf2,0xb1,0x5c,0x72,0x4a,0x40,0x20,0x4b,0x07,0xa0, +0xa3,0x00,0x3f,0xe1,0xdd,0x34,0x6e,0x8d,0xec,0x72,0xd4,0xb9, +0x94,0xd1,0x5d,0x35,0x22,0xd6,0x44,0xb6,0xdb,0xb0,0xc6,0x3f, +0x3f,0x5b,0x3c,0x5b,0x4f,0x60,0xc7,0x8a,0x72,0xbb,0x8b,0x7e, +0xf6,0x0c,0x63,0x9e,0x45,0xf4,0x10,0x1b,0x80,0xab,0xd5,0xc1, +0x1f,0x6f,0x57,0x9c,0x0b,0x5b,0x67,0xcb,0x76,0x92,0xb0,0xc5, +0x04,0x8e,0x25,0xc3,0xc6,0xca,0x6b,0xbe,0x3e,0x07,0xaf,0x00, +0x45,0xf7,0xf8,0x7c,0x5a,0x1e,0x1e,0x29,0x53,0x02,0xdd,0xcd, +0x5a,0x6a,0xe2,0x18,0x3b,0xb0,0x14,0x6a,0xd3,0xb3,0x42,0x28, +0x4b,0x53,0x9e,0xa1,0x98,0x15,0x05,0x28,0xb5,0x43,0x7f,0x71, +0xc5,0xa2,0xc4,0xe6,0x94,0x9e,0x0b,0x6e,0x09,0x02,0xa9,0x8e, +0xa4,0x28,0xc3,0x31,0x62,0xe9,0x21,0xb5,0x5b,0x54,0x33,0x32, +0xe3,0x39,0x3d,0x7c,0x23,0x4b,0x2e,0x04,0x53,0xe7,0xec,0x58, +0xa4,0x7c,0x54,0x4a,0x51,0x3f,0x6c,0xe8,0x38,0x34,0x5d,0x42, +0x19,0x83,0xd8,0x84,0xb1,0x81,0x55,0x52,0x9e,0x26,0x09,0x72, +0xf1,0x20,0x00,0xd8,0x60,0x04,0x47,0x41,0x64,0xe2,0xde,0x2a, +0xe1,0xd5,0x2c,0x4b,0x98,0x98,0x8c,0x4f,0x2e,0x9d,0x4f,0xe6, +0xb9,0x1e,0xe8,0xe0,0x02,0x41,0x54,0x5a,0x57,0xdd,0x04,0x4c, +0x41,0xe6,0x38,0xb4,0x27,0x49,0x06,0x1f,0xa2,0x42,0xe6,0x03, +0x04,0xa0,0xd6,0x46,0x3c,0x72,0xed,0xc9,0x91,0x6b,0xcb,0x8c, +0xb7,0xfd,0x79,0x3e,0x9f,0x7f,0x06,0x40,0x4a,0xbe,0x43,0xb1, +0x74,0x16,0x64,0x41,0x87,0xaf,0x0c,0x08,0x47,0x1b,0xf7,0x16, +0x98,0x27,0x51,0x07,0xbc,0x96,0x03,0x6f,0x2d,0xd5,0x15,0x17, +0xe1,0xb6,0xeb,0x71,0xc1,0x8c,0xa6,0x9f,0x4f,0x2f,0xac,0x2f, +0x05,0x47,0xbc,0x3f,0xf2,0x31,0x69,0xac,0x12,0x3b,0xbe,0xe1, +0xe6,0x5c,0x39,0xd0,0xb9,0xb9,0x8d,0xcf,0x4d,0x2f,0x59,0xd2, +0xe9,0xfd,0xb2,0xd0,0xe4,0x3c,0xbe,0x6b,0x99,0xbb,0x6d,0xd1, +0x55,0x68,0x39,0x2b,0x3a,0x2d,0x33,0xdb,0x76,0x81,0x6d,0xb7, +0xb2,0xb4,0xb2,0x6c,0xb6,0xdd,0xd1,0x67,0xef,0x2d,0x94,0x36, +0x5b,0xd9,0x5d,0xab,0x70,0xd0,0x7f,0xb7,0x9d,0xdd,0x6d,0x87, +0x71,0x10,0xd9,0x42,0x81,0x75,0xbb,0xb3,0x95,0xde,0x6d,0xbf, +0xb8,0x6b,0x75,0x36,0x5a,0x69,0x2b,0x49,0x67,0xdb,0x2f,0x66, +0xfc,0xbd,0x7c,0xec,0x7e,0x06,0xd3,0x83,0x8e,0x0e,0x93,0x83, +0x24,0xe7,0x63,0x25,0x2c,0xd0,0x09,0x82,0xe9,0x69,0x10,0xec, +0x86,0xe4,0x02,0x02,0xc8,0x6d,0x58,0x5c,0x9b,0x0b,0xb6,0x10, +0xc5,0xe7,0x8b,0x1d,0x08,0xd4,0x72,0x43,0xb9,0xd7,0x65,0x58, +0x2c,0xac,0x6a,0xab,0x37,0x17,0x2a,0xb3,0xdd,0x03,0x81,0x6e, +0x8c,0x41,0x20,0xcd,0x7c,0x31,0x7f,0x88,0x0f,0x4b,0xdb,0x46, +0x75,0x0a,0xad,0x2b,0xbe,0xe6,0x55,0xd9,0x8c,0x8b,0x53,0xa9, +0xbc,0xff,0x41,0x2c,0x5a,0x09,0x4b,0xea,0xa2,0x4c,0xeb,0x91, +0x4a,0x4c,0x99,0x26,0x57,0x52,0x19,0x12,0x13,0x5e,0x72,0xf6, +0x08,0x58,0x7c,0x9e,0x90,0x4a,0x87,0x23,0xcf,0xa5,0x37,0xa4, +0x47,0x83,0xb5,0x3b,0xbe,0x55,0xce,0x94,0x83,0x03,0x61,0xe7, +0x47,0x19,0xf3,0x58,0xd7,0xdb,0x78,0x28,0xac,0x18,0x82,0x4f, +0xc7,0xf1,0x89,0xab,0xe7,0x0e,0x7f,0x86,0x93,0x3e,0x38,0xf0, +0xbc,0x7a,0x39,0x98,0xe4,0xa4,0xac,0x84,0xa6,0x72,0xf8,0x4b, +0x1d,0xe7,0x3a,0x2b,0x4b,0x3e,0x91,0xed,0x44,0x71,0x20,0xbb, +0x1f,0x0e,0x46,0x1f,0x53,0x17,0x86,0x15,0xc9,0x7d,0xa3,0x6a, +0x0d,0x69,0xb1,0x67,0xe5,0x28,0x18,0xcf,0x9d,0xdd,0xc8,0xd8, +0xeb,0x70,0x0f,0xdc,0x63,0x86,0xbb,0x24,0x54,0x7f,0x9f,0xa0, +0xc9,0xb3,0x89,0xf5,0x4b,0x6a,0x60,0x7b,0x3d,0xd1,0x0f,0x6d, +0x67,0xe3,0xab,0xab,0x61,0xfe,0x27,0x7c,0x84,0xb7,0x3b,0x9d, +0xc2,0xc6,0x9c,0x3e,0x0d,0xbd,0xca,0x23,0x8d,0x03,0x72,0x71, +0xfb,0xc7,0x9c,0x94,0x91,0x8a,0x71,0x59,0x39,0xaa,0x03,0x14, +0xd4,0x98,0x8e,0x7a,0x1f,0xf3,0xf7,0xb4,0x63,0x2b,0xe7,0x30, +0x90,0x03,0xd7,0x14,0xeb,0x97,0x16,0x1a,0x85,0x16,0x2a,0x77, +0x87,0x3d,0xa8,0xdf,0x22,0xd0,0x79,0xf3,0xee,0x4f,0x3b,0x6f, +0xdf,0xbc,0x7c,0xbf,0xfb,0x76,0xe7,0xf8,0x38,0x55,0xdf,0x49, +0xa3,0x02,0xb7,0x26,0x6c,0xbd,0x0a,0x00,0x74,0xf1,0x00,0x58, +0x25,0x80,0xa5,0x3d,0xa3,0x66,0x4b,0x07,0xfc,0x50,0x1d,0xbb, +0x30,0xb3,0x58,0x72,0x54,0x9c,0xa0,0x16,0xb8,0xed,0xd4,0x08, +0x4b,0x44,0xe8,0x9a,0x1d,0xa4,0x1c,0xea,0xf0,0xd9,0x64,0x68, +0xe4,0x4d,0xe3,0x2e,0x1e,0xbd,0xc3,0x4a,0x46,0x7e,0x0c,0xa6, +0x24,0x63,0x36,0xd7,0x29,0x40,0x36,0x6f,0x19,0xca,0x54,0x56, +0x53,0x48,0xc2,0x4d,0x6c,0x46,0xc3,0x23,0x56,0xb7,0xe4,0xfb, +0x0a,0x01,0x1b,0xce,0xe3,0x4c,0xba,0x71,0xf0,0x76,0x32,0x98, +0xce,0x06,0x6c,0xae,0x9c,0x63,0xa8,0x66,0x1f,0x14,0x54,0xb0, +0x84,0x6f,0x52,0x73,0x1d,0x7b,0x8b,0xeb,0x27,0x71,0xcd,0x79, +0x42,0xf7,0xdd,0x78,0x78,0xf4,0xe6,0xf8,0xe4,0xcd,0xbb,0x3d, +0xe9,0x32,0x53,0x20,0x13,0x7c,0xc9,0x7c,0x59,0x00,0xe5,0x6f, +0x59,0xa5,0x6d,0x78,0x75,0xcc,0x8d,0xe4,0x19,0x68,0xa3,0x6d, +0xdb,0xb9,0xc9,0x20,0xeb,0x10,0xa4,0xc2,0xe3,0xbc,0x20,0x4f, +0x22,0x87,0x59,0xa7,0x41,0x6c,0x37,0xb3,0x10,0xff,0xe7,0x52, +0xb0,0x11,0x52,0x7a,0x0c,0x21,0x5b,0x73,0x15,0xcc,0xcc,0x42, +0x45,0x0f,0x52,0xa4,0x66,0x55,0x91,0xcb,0x58,0x49,0x96,0x07, +0xd7,0xf3,0xf4,0x5f,0xc8,0x6c,0x9f,0x27,0x30,0xf2,0x9a,0x4e, +0xb2,0x00,0x60,0xc7,0x0e,0x7f,0x07,0x7d,0xe0,0xe3,0x71,0x21, +0x8b,0x61,0xf7,0xde,0x84,0xe5,0x32,0xb4,0x10,0x6b,0x3b,0xd1, +0xa7,0x9b,0x12,0xb8,0x40,0x99,0x20,0x80,0x6b,0x27,0xb4,0xeb, +0x02,0x24,0x74,0xc2,0x32,0x63,0xdc,0x4f,0x0b,0x25,0xba,0xac, +0x44,0x4a,0xa5,0x12,0xca,0x08,0x9d,0xaf,0xf1,0xe1,0x26,0xba, +0xc1,0xfc,0x25,0xb0,0x94,0x14,0x6b,0xe2,0xdf,0xd8,0x08,0xe6, +0x02,0x6a,0xfe,0x56,0x72,0x9b,0xe4,0x31,0xe2,0x4d,0xcc,0x0a, +0x5c,0xc9,0x2e,0x7a,0x91,0x34,0x4b,0xdd,0x1f,0xd0,0xfe,0x23, +0x83,0x40,0xf3,0xc1,0xde,0x0e,0x60,0x4f,0x04,0x0d,0x58,0x85, +0x3f,0x75,0x02,0xdf,0xfa,0x8c,0x2e,0x86,0x73,0x58,0xb4,0x0a, +0xdd,0x24,0x52,0x06,0x1b,0xee,0x58,0xd5,0x98,0x5f,0xac,0xed, +0x76,0x5d,0x69,0xa3,0x51,0x6a,0x61,0xeb,0x1f,0x68,0x21,0xb3, +0x8e,0xa4,0xfd,0x8b,0x37,0x1b,0xe5,0x86,0x32,0x18,0x85,0x56, +0x71,0x14,0xb6,0x96,0x8a,0xf0,0x5e,0x12,0x23,0xd2,0x26,0x5c, +0x2a,0x98,0x79,0x81,0x89,0x78,0xee,0xf2,0xf2,0xcd,0xd1,0xc9, +0x5f,0x2c,0x6b,0x09,0x38,0xdb,0x56,0x89,0xb3,0xb5,0x8a,0x6d, +0xa3,0xaa,0x63,0x3d,0x0b,0x0e,0x6d,0xe6,0x47,0xf0,0xd1,0xd5, +0xae,0x66,0x75,0x4f,0xe1,0xb4,0x34,0xf7,0xdd,0xc4,0x7c,0x80, +0xcf,0x28,0xec,0x62,0xba,0x20,0x57,0x26,0x26,0xf6,0xd0,0x6c, +0x51,0xe1,0xba,0x5d,0xdf,0x2a,0x71,0x8a,0xbc,0x79,0xbe,0xe3, +0x97,0xed,0xf2,0x5c,0x5a,0x42,0x77,0x62,0x03,0x96,0xdf,0x42, +0xf9,0x35,0x1d,0x36,0xb5,0x6f,0x70,0xff,0x43,0x6f,0x16,0xd9, +0x08,0x92,0x5e,0xff,0xcd,0x05,0x2e,0x53,0xd6,0x21,0xa4,0x72, +0x97,0xa0,0x6d,0x00,0x55,0xba,0xc2,0x73,0xab,0x21,0x9a,0x09, +0x4b,0xda,0x7e,0x17,0x81,0x60,0x61,0xa9,0x3a,0x99,0x0c,0x6e, +0x00,0xe1,0x93,0xc8,0x1d,0x35,0x67,0x78,0x52,0x6c,0xb5,0x94, +0xa8,0x62,0xa0,0xb0,0x81,0x98,0x87,0x5d,0x73,0x87,0xcf,0x53, +0xad,0xc9,0xab,0x6a,0xe6,0x02,0x48,0xd1,0xb6,0x8c,0xbd,0xad, +0x4f,0xc8,0xd0,0xb8,0xf3,0x6a,0x69,0xd5,0xe6,0xad,0x8e,0x7c, +0x92,0x58,0x54,0x01,0x8e,0xc4,0x79,0x33,0x5c,0xfe,0xf2,0x4f, +0x94,0x5e,0x59,0x2b,0xf8,0x6d,0x45,0xdb,0x25,0x03,0x32,0x2f, +0xfe,0x7a,0x40,0x33,0xda,0x19,0xc9,0x4d,0xac,0x32,0x2d,0x9b, +0xfb,0x02,0x91,0xb1,0xad,0xd1,0xf9,0x98,0x7f,0xe9,0x8f,0x3f, +0x8d,0x94,0x36,0xa0,0xe8,0xf8,0x8b,0xb6,0x09,0xeb,0xf7,0x37, +0xe1,0x6b,0x17,0x5f,0x45,0xfe,0x0e,0x37,0x3d,0x10,0x58,0x2c, +0xe0,0xcf,0x8b,0xd6,0xaf,0xeb,0xf5,0xd6,0xef,0x5e,0xb8,0x70, +0xf6,0xfc,0xdf,0xeb,0xf5,0xef,0xb7,0x5e,0x70,0x8e,0xa0,0x1d, +0x64,0x6b,0x4e,0x57,0x2c,0x2f,0x14,0x7c,0x47,0x98,0x8a,0x0e, +0xbc,0xed,0x41,0x22,0x8e,0xa0,0x2e,0x49,0x91,0xb5,0x0b,0xec, +0xc5,0xa0,0x8a,0x64,0xc9,0x43,0x04,0x53,0xb4,0x1f,0x76,0x9c, +0x26,0x15,0x4b,0xec,0x85,0xf1,0x4f,0x3a,0x51,0x94,0x16,0xe3, +0xec,0x7b,0x38,0xba,0x07,0xfa,0x13,0xf3,0x2f,0x10,0xc1,0xf9, +0x00,0x4a,0xa2,0x33,0xa1,0xb7,0x43,0xb9,0x90,0x17,0x36,0x97, +0x6b,0x37,0x3c,0x57,0x68,0x99,0xb4,0x38,0xbb,0xf4,0x9c,0x43, +0x07,0x2d,0x57,0x64,0xd4,0x14,0x4f,0xe5,0x38,0xae,0x13,0x7b, +0x8a,0xb3,0x3c,0x34,0x92,0x4a,0x23,0x32,0xe0,0xc2,0xf9,0xd2, +0x87,0xf3,0xc1,0x8a,0xe5,0xbd,0xf7,0x12,0xa1,0x4a,0x1a,0x4c, +0x0b,0x3e,0x89,0xf3,0xd7,0xbc,0x74,0xc1,0xfe,0xf3,0xd9,0x66, +0xdc,0xfc,0x2e,0x81,0xbf,0xa7,0x57,0x83,0x9b,0xee,0x77,0xc9, +0xfa,0xa6,0x3a,0x8d,0xe3,0x36,0xe3,0xc9,0xdf,0x51,0x7e,0xb5, +0xf7,0xf9,0xd6,0x1d,0x26,0x1b,0x77,0x8e,0x6c,0x8a,0x1d,0xf6, +0x80,0xaf,0x2c,0xea,0x2a,0x0b,0xd6,0xbe,0x77,0x4b,0xd2,0x47, +0x21,0xa8,0x71,0x39,0x38,0xff,0x60,0x8d,0x8e,0x91,0xc9,0x40, +0x89,0xa6,0xc3,0x8a,0x35,0x9f,0x67,0xb1,0x50,0x01,0xea,0x67, +0x79,0xb6,0x6b,0x7d,0xcd,0xb8,0x21,0x8c,0x12,0x08,0x8c,0x79, +0x40,0x22,0x13,0x41,0x23,0xc9,0x4e,0x34,0x5a,0x17,0xc0,0xd7, +0xb3,0xe7,0x79,0xad,0x27,0x8d,0xaf,0x9d,0xcf,0x67,0xb5,0x4f, +0xbd,0x29,0x5a,0x05,0x69,0xd6,0x44,0x7d,0x3a,0xad,0xdd,0x6f, +0x2f,0x23,0xa7,0xa0,0xe1,0x2b,0x35,0xd3,0x59,0x8f,0xac,0xa2, +0x9e,0xf4,0xae,0xec,0x26,0x28,0xd1,0xce,0x9e,0x83,0xce,0xc0, +0x02,0xf6,0x49,0x21,0x0f,0x30,0x1f,0x63,0xce,0x9c,0x11,0xc0, +0x62,0x27,0x59,0x9e,0xc6,0x4e,0x77,0x57,0x65,0x4a,0x8c,0x10, +0xec,0xfe,0x60,0x24,0x47,0x56,0x7c,0x69,0x62,0x83,0x74,0x92, +0x57,0xca,0x63,0x20,0x9d,0x4d,0x5d,0x3f,0x36,0xac,0x6b,0x21, +0x69,0xd7,0xeb,0x7c,0x14,0x2a,0x57,0x9a,0xae,0x9a,0x4a,0xf2, +0x76,0xa9,0x45,0xc2,0xad,0x24,0x72,0x9d,0xdb,0x4d,0x87,0x65, +0xbb,0xa2,0x17,0xca,0xc8,0x27,0x2b,0x3a,0xb5,0x22,0x27,0x99, +0x2b,0x92,0xfe,0xe8,0x7d,0x0e,0xfa,0xcc,0x06,0x83,0x3e,0x73, +0x79,0x60,0x4a,0x7c,0xfe,0x27,0xf4,0xd9,0x0b,0x57,0x4d,0x75, +0x9f,0xd9,0xd4,0xa7,0xf5,0x99,0xca,0xfd,0x48,0x9f,0x95,0x90, +0x5f,0xd9,0x67,0xe5,0x9c,0x25,0x45,0xac,0x6f,0x12,0x02,0x06, +0xac,0x44,0xfb,0x2d,0x65,0xab,0xe6,0x42,0x15,0x13,0x61,0x5b, +0x96,0x61,0xf7,0x3b,0x0b,0x70,0x18,0xb7,0x58,0xc8,0xbd,0xc9, +0xd1,0xde,0xef,0xf7,0xfe,0x7c,0xf8,0x28,0x87,0xb6,0x8a,0xe3, +0xd8,0xbb,0x5e,0x5c,0xea,0xe0,0xe2,0x9b,0x12,0x40,0x31,0xa4, +0xad,0xee,0xbf,0x44,0x24,0xa9,0x1c,0x2f,0x07,0x2e,0xe4,0xe3, +0xab,0x86,0x62,0x05,0xa7,0x0d,0xdb,0x87,0x4b,0x5d,0x14,0x35, +0x58,0x54,0x13,0x7e,0x00,0x74,0xef,0xf8,0x40,0xa6,0x70,0x73, +0x89,0xc8,0x00,0xbe,0x9a,0x8c,0xa1,0xcc,0x0b,0x6e,0xff,0x8a, +0xf9,0xfb,0xd4,0x79,0xfe,0x94,0x19,0xfe,0x94,0xb9,0x5d,0x31, +0xab,0x81,0x7a,0xdd,0x6c,0x2e,0xb7,0x1c,0x12,0x71,0x1a,0x7f, +0xdb,0x04,0xc6,0x99,0xbb,0x6a,0xce,0x3e,0x75,0xb6,0x3e,0x65, +0x9e,0x3e,0x65,0x86,0xaa,0x96,0xff,0x03,0xa4,0xb3,0x58,0xb8, +0xab,0xc0,0x27,0xce,0x81,0x07,0x84,0x94,0x55,0xb4,0xed,0x79, +0xc7,0x7c,0x32,0xfc,0x26,0xc6,0xf1,0xad,0x5c,0xa3,0x2d,0x8f, +0xa4,0x1e,0x15,0x5d,0x8a,0xbd,0xf2,0xd3,0xd1,0xdb,0xa7,0x32, +0x08,0x32,0x7b,0xf4,0x48,0xcf,0x70,0x9e,0x82,0xe8,0x56,0x3d, +0x6c,0x1a,0xdb,0x4a,0xae,0x17,0x64,0xf0,0x5d,0x9b,0xdf,0xf4, +0x06,0xff,0x82,0xce,0xa5,0x6a,0xbf,0xba,0x7b,0xf7,0xf6,0x77, +0xde,0x3c,0xb9,0x83,0xa9,0x8a,0x47,0xbb,0xd8,0xe6,0x7a,0x52, +0x27,0x87,0x58,0x57,0x76,0x73,0x21,0x8b,0xba,0x09,0xeb,0xf5, +0x07,0xe3,0x47,0xfb,0x0b,0x16,0x3b,0xb5,0xdf,0x66,0x51,0x82, +0x8f,0x1b,0xdd,0x81,0x28,0xc4,0xc1,0x9c,0x81,0xc8,0xc8,0x28, +0xd7,0x66,0xc1,0x9e,0x6a,0x38,0xb8,0xf8,0x18,0x99,0xf2,0xce, +0x07,0xad,0xdb,0x90,0xda,0x39,0x1a,0xaf,0xf9,0x8a,0x0d,0x2e, +0xe1,0xe1,0x76,0xb9,0xb6,0xdd,0xe5,0x0d,0x96,0x52,0x78,0x77, +0x25,0xda,0xe5,0xba,0x79,0x2d,0xcc,0xb2,0xc2,0x3e,0x6b,0xc9, +0x4b,0xd0,0xfa,0xf8,0x1c,0xba,0xf3,0x2e,0x8f,0x23,0xca,0x16, +0x05,0xb5,0xa9,0x0e,0x25,0x60,0xe7,0xe3,0xcf,0x4f,0xe8,0x53, +0x31,0x55,0x95,0xff,0xc9,0xe3,0x86,0x27,0x03,0x12,0x61,0x2e, +0x7b,0xb0,0xf7,0x0e,0xd2,0x5e,0xb9,0x98,0xb6,0xbb,0xd7,0x72, +0x00,0x12,0x32,0x18,0x60,0xa1,0x21,0x7d,0xb9,0x3c,0x1e,0x12, +0x66,0x52,0x70,0xe9,0xc4,0xf0,0xc1,0x21,0xfa,0x8a,0xe1,0x28, +0xf7,0xe8,0xc3,0xc3,0x52,0x31,0x02,0xa5,0xae,0xff,0xaa,0xc5, +0x40,0xc6,0x2f,0x73,0xbd,0xb0,0x7c,0x92,0xa0,0xa5,0x4b,0x77, +0x5c,0xd9,0xd4,0x77,0x93,0xe6,0xf9,0x17,0x78,0x14,0xe6,0xaf, +0xb2,0x58,0x81,0x2f,0x1f,0x86,0x0f,0x14,0xe9,0x86,0x02,0xb6, +0x6c,0x74,0x6e,0xc6,0xaa,0x77,0xa6,0x64,0x4b,0xe8,0xde,0xde, +0x51,0xa5,0x74,0x45,0x45,0xf7,0x5b,0xde,0x50,0x4e,0x89,0x66, +0xd4,0x55,0x94,0x00,0x26,0xdf,0x15,0xe2,0x77,0x88,0xbd,0x89, +0x24,0xf7,0xb1,0xc5,0x05,0xba,0x61,0x6d,0x6b,0xb1,0x90,0xe1, +0x23,0xe5,0xf5,0x5f,0x6d,0x43,0xa4,0xc3,0xd5,0xb9,0x41,0xaa, +0xd7,0xd7,0xe4,0x31,0x5a,0xe0,0x83,0x0d,0x32,0xe8,0x23,0x41, +0x89,0x34,0xee,0x34,0x50,0x6a,0x4c,0x24,0x3b,0x1b,0x53,0x92, +0x48,0x7d,0xd1,0xee,0x41,0x50,0xd7,0x11,0x96,0x3c,0xa1,0x74, +0xda,0xe5,0x10,0xc7,0x77,0x44,0x81,0x5c,0x65,0xd5,0xfa,0x7c, +0xb6,0xe6,0x22,0x1c,0x97,0xf0,0x04,0x20,0xab,0xb2,0xdc,0x7b, +0xdf,0x6a,0xaa,0x9c,0xbd,0x72,0x95,0x08,0xd1,0xae,0xad,0x45, +0x49,0xe0,0xa0,0xcd,0xe7,0x67,0x15,0x05,0x86,0x8d,0xf7,0x55, +0x05,0x6d,0x1e,0x9b,0x51,0x29,0xf4,0x18,0xf4,0xe4,0x5c,0xaf, +0x4b,0x19,0xa6,0xcf,0x8f,0x34,0x69,0x24,0xc6,0x55,0xb9,0x2c, +0x20,0xa3,0xfc,0x48,0x79,0xdb,0x5f,0x6d,0x19,0x6f,0x76,0xf0, +0x84,0xdd,0xc3,0xc6,0xb8,0x4d,0x99,0x64,0xd8,0xc8,0x6e,0xc8, +0xd9,0x08,0x76,0x30,0xfd,0x2b,0x48,0x4d,0x9f,0x66,0xf8,0x1a, +0xc8,0xa3,0xb7,0x23,0x79,0x7c,0x36,0x41,0x17,0x47,0x01,0x3e, +0x11,0xd3,0x61,0xf8,0xb6,0x17,0x22,0x90,0x84,0xf8,0x4b,0x44, +0xdd,0x71,0x3f,0x6b,0xd5,0x39,0xa6,0xdd,0xaa,0x43,0x10,0x9f, +0xdb,0xd9,0x3c,0x78,0x32,0x34,0xee,0x2b,0x52,0xee,0x38,0x02, +0x58,0x85,0x5a,0xaa,0x49,0x6d,0x55,0x26,0x71,0x9f,0xce,0x83, +0xaa,0x8c,0xce,0x86,0xf7,0x6d,0x24,0x26,0xb8,0x97,0xb9,0x9e, +0x9d,0x59,0x0d,0xcf,0xa2,0xda,0x81,0xc4,0x77,0x44,0xcd,0x33, +0xd0,0xd2,0x48,0x45,0x27,0xd0,0xbf,0xc0,0xfd,0x06,0x78,0xca, +0xa0,0x8e,0x83,0x77,0xd3,0x1b,0xc1,0x84,0x7e,0x5b,0x81,0x65, +0x05,0x40,0x80,0xe5,0x55,0xbe,0xc8,0xee,0xd0,0xe6,0x95,0x3a, +0xa9,0xbf,0x76,0x45,0xb8,0x40,0x13,0x2d,0x5f,0xee,0x5e,0xf7, +0x26,0x78,0xe2,0x1a,0x3f,0xdf,0x5e,0x5c,0x5c,0x93,0xf6,0x03, +0x06,0x59,0x51,0x62,0x09,0x88,0x58,0x1c,0x7e,0xaa,0x68,0xd9, +0xa3,0x38,0xf4,0x36,0xfe,0xfe,0x35,0x38,0x6c,0x3c,0x7f,0x5e, +0xaf,0x46,0xa2,0x1d,0x0d,0x90,0x14,0xa3,0x37,0x51,0xd8,0xf1, +0xe4,0x13,0xca,0x76,0x4e,0xa1,0xb3,0xd4,0x60,0x14,0x9a,0xc0, +0x22,0x22,0x9a,0x62,0x30,0x1f,0x7e,0x79,0x3b,0x98,0xc1,0xcf, +0x7f,0xce,0xf3,0xc9,0x17,0x23,0x6f,0x7a,0xf7,0xc7,0xfd,0xf9, +0x30,0x37,0x23,0xc0,0x01,0x4d,0x69,0xbd,0x37,0xa4,0x28,0x06, +0x7c,0x80,0x35,0xc6,0x0c,0x4e,0x6e,0x0c,0xe1,0xaf,0x71,0xa6, +0x8c,0x4b,0xf4,0x66,0x53,0x0c,0x1d,0x70,0x69,0x7f,0xd3,0xa3, +0x2b,0x74,0x4e,0xca,0x95,0x65,0xd6,0xe0,0x81,0x84,0x4d,0x18, +0x44,0xaf,0x56,0x41,0x44,0x76,0xbf,0x04,0x5e,0x3d,0x1f,0xf4, +0xb3,0xd3,0x68,0x2b,0x32,0xf2,0xaf,0xdb,0xc6,0xe6,0x88,0xc5, +0x2c,0xfc,0x5c,0x61,0x2e,0xcb,0x19,0xb9,0xaa,0x36,0x90,0xc5, +0x2f,0xb8,0xc8,0xc2,0x0e,0xce,0x4e,0x0f,0x12,0x1a,0x82,0xfa, +0x38,0x67,0x9b,0xcd,0xef,0xda,0xb5,0xc9,0x5d,0xfa,0x2d,0xd0, +0x13,0x83,0xae,0x4e,0x9b,0xf2,0xbc,0x9f,0x6e,0xfd,0x44,0xc7, +0x47,0xc5,0x89,0xd9,0xcf,0xc1,0x4d,0x85,0xcd,0x3e,0xa1,0x19, +0xd5,0xc3,0x90,0xaf,0xb3,0x52,0xb6,0x17,0x9a,0x14,0x11,0x9e, +0x8f,0xba,0xe8,0x6e,0x46,0xd4,0x26,0x97,0xe9,0x57,0x15,0x75, +0x64,0xfd,0xf3,0xd9,0xf4,0xbb,0xcd,0x50,0xc1,0x12,0x62,0xd6, +0x29,0x4a,0x20,0xe3,0xab,0x50,0x47,0x3e,0xd9,0xef,0x5e,0x60, +0x4f,0x7a,0x44,0xed,0xa1,0xac,0xb3,0x45,0xc8,0x61,0x77,0xd5, +0x64,0x4b,0x76,0x24,0x22,0xf5,0x02,0x96,0x13,0xf2,0x88,0xf9, +0xbd,0x23,0xd6,0x1c,0xa1,0x61,0x45,0x98,0x1e,0xa5,0xa4,0x8e, +0x7f,0x46,0x50,0x4a,0x43,0x3d,0xfe,0x46,0xb1,0xc2,0x24,0x2d, +0xc6,0xa8,0x3e,0x5a,0x81,0x75,0x09,0x5b,0x17,0xa1,0xd0,0xf6, +0xf0,0x68,0x7c,0x8f,0xdf,0xed,0xfc,0x71,0xef,0xfd,0xee,0xce, +0xf1,0x9e,0x6c,0xb6,0x32,0xc7,0xaf,0xc4,0x10,0x65,0x06,0x42, +0xd2,0x70,0x98,0x46,0xad,0xe6,0x76,0x73,0x6b,0x63,0x72,0xd1, +0xdc,0x46,0x8b,0xbb,0x1f,0xc6,0x93,0xb4,0x85,0x27,0x44,0xf0, +0xbb,0x6d,0xfa,0xe3,0x59,0x8a,0x3a,0x04,0x52,0x5b,0x74,0xde, +0x9b,0xe4,0xd7,0xbd,0x51,0x7f,0x03,0x28,0xf0,0x86,0x5e,0x79, +0xe1,0x29,0x78,0xb4,0x84,0x49,0xbe,0x8b,0x0e,0xc3,0xb2,0x3f, +0xfc,0x27,0x4e,0xf6,0xe6,0x05,0x05,0xee,0x31,0x9e,0x2c,0x00, +0x4a,0x74,0xfe,0xf9,0x16,0x0a,0x8f,0x51,0xdc,0xdb,0x88,0x1a, +0xb1,0xd5,0x3a,0x4e,0xec,0xd3,0x88,0x38,0x81,0x02,0x6f,0x60, +0x41,0x43,0xd1,0x89,0x2e,0x8e,0xec,0x8d,0x10,0xba,0xcf,0xe5, +0x39,0xea,0x2c,0x9d,0x14,0xb3,0x94,0x06,0x9f,0x8d,0x7b,0x5f, +0x8e,0xfc,0xad,0x51,0xb1,0x48,0x2c,0x39,0x70,0xef,0x58,0x1e, +0x86,0x8a,0xf2,0xb3,0x19,0x34,0x4c,0x6e,0xb4,0xd0,0x66,0x95, +0xcd,0xb1,0x34,0xbc,0x6a,0x3e,0x86,0x73,0x45,0xae,0x27,0xa0, +0x5d,0x51,0xea,0x6b,0x31,0xef,0xe7,0x2b,0x31,0x3f,0x3e,0xdc, +0xdb,0x7d,0xb3,0xf3,0xf6,0xfd,0xee,0xeb,0x9d,0xa3,0x63,0x47, +0x2c,0xf1,0xe9,0x59,0x7a,0xb6,0x71,0xf6,0xbe,0xdb,0x88,0x9b, +0x09,0x6a,0x3b,0xef,0x1f,0xfc,0xf5,0xfd,0xeb,0x9d,0xdd,0x3f, +0xba,0x1c,0x3f,0xdf,0x8c,0xff,0x1e,0x5b,0xdb,0x7b,0xcc,0xe4, +0x0b,0x3c,0x98,0x23,0x81,0x0f,0xf3,0xf0,0x1f,0x3a,0x33,0xf8, +0x42,0x0e,0xca,0xd3,0x39,0xb9,0x74,0xf7,0xed,0x40,0xd4,0xfd, +0xcd,0xf3,0x64,0x70,0x75,0x45,0x1a,0xcf,0xe4,0x78,0x0e,0x77, +0x65,0xf8,0x4b,0xb7,0xe6,0x64,0x20,0x8a,0x1f,0x73,0x4f,0xd8, +0x3b,0x40,0x84,0x1a,0x76,0xa8,0x62,0x83,0x27,0xb6,0xaa,0xe3, +0x7b,0x68,0x4d,0x0e,0x9d,0xa1,0x4f,0x99,0xd6,0xd0,0x3e,0x96, +0x40,0x4e,0x52,0x76,0xe2,0x84,0xbb,0xba,0x97,0x07,0xfb,0xbb, +0x6c,0x42,0xfd,0xed,0xb8,0xd7,0xcf,0xfb,0x91,0xb1,0x99,0xa4, +0x15,0xd6,0x4c,0x0c,0xe5,0x46,0x53,0xd7,0x3e,0x07,0x74,0xa6, +0x5d,0x90,0x8a,0x9e,0x3f,0x79,0xbb,0x55,0x16,0x73,0x49,0x31, +0xcc,0x0f,0x1f,0xe9,0xc3,0x59,0x15,0x8a,0x08,0xb8,0x07,0x49, +0x8a,0xa7,0x72,0xc2,0x2b,0x02,0xae,0x01,0x09,0xb7,0x11,0x75, +0x61,0xd6,0xe5,0xbf,0x28,0x5f,0xf0,0x2c,0x13,0x3a,0x95,0x6e, +0x08,0xa1,0x6e,0x36,0x8f,0x01,0xd5,0xc3,0x7e,0xa4,0xba,0x49, +0xaa,0xe3,0xa8,0xe1,0xa2,0x05,0x2d,0xe9,0x4b,0x71,0xf5,0x02, +0x33,0x1f,0xf1,0x48,0x69,0x0d,0x46,0x75,0xc6,0x14,0x97,0x67, +0xf8,0x35,0xec,0x1a,0x86,0x82,0xf4,0xb5,0x34,0x3f,0x1e,0x1c, +0xbc,0xdd,0xdb,0x79,0xf7,0x7e,0xe7,0xe4,0xe4,0x08,0x0d,0x5b, +0xd9,0xf6,0x45,0x37,0xf3,0xe1,0x6c,0x00,0xe3,0x21,0x1b,0xbf, +0x1c,0x7d,0x93,0xd3,0xfe,0xd5,0xf4,0x07,0x53,0xb2,0xd6,0x6a, +0x70,0x6c,0x0e,0x46,0xc3,0x2f,0x86,0x6d,0xd6,0x43,0x0c,0x30, +0xd0,0x51,0x24,0xaf,0x48,0x61,0xed,0x4d,0x4a,0x5a,0xfd,0xba, +0xba,0x53,0xbf,0x38,0x72,0x6a,0x57,0x54,0x17,0x58,0x06,0xb1, +0x59,0xf7,0xde,0xee,0xed,0xef,0xbd,0x3b,0x39,0x0e,0xb0,0x63, +0x95,0x79,0x46,0x0d,0x6a,0x25,0x59,0x1f,0x0f,0xc5,0x80,0xcd, +0xf5,0xcc,0xf9,0x7c,0x36,0x83,0x30,0x6e,0xa8,0x0d,0xce,0x1f, +0x74,0x3d,0xf2,0x14,0xa4,0x6c,0x4d,0xa7,0x7e,0x6d,0xb0,0x88, +0xad,0x6d,0x2d,0x13,0xa7,0xea,0x71,0xdf,0xef,0xcd,0x7a,0x29, +0x53,0x15,0xb0,0xc1,0x9e,0x19,0x8c,0xae,0xf3,0x09,0x04,0xfa, +0x2f,0x7d,0xc2,0x1b,0x1d,0xc7,0x6e,0xba,0x56,0xaf,0x17,0x15, +0x45,0x1c,0x4b,0x40,0xcb,0xcb,0x68,0x7c,0x2e,0x71,0x6a,0x82, +0xc3,0x7c,0x22,0x95,0x78,0x95,0x4e,0x63,0x0d,0x03,0x7d,0x6b, +0x25,0xce,0x28,0x91,0xe3,0x89,0x68,0x33,0xab,0xcc,0xa1,0xd8, +0x1c,0x51,0xc8,0xe1,0xbc,0x75,0x2d,0x4a,0x5d,0xe2,0xeb,0x6d, +0xda,0x87,0x08,0x9a,0xaf,0x25,0x68,0x2e,0x20,0xaa,0x12,0xa2, +0xbd,0x9a,0x17,0xb7,0x5a,0x99,0xb7,0xde,0x4a,0x10,0xe9,0x5d, +0x4b,0xa0,0x08,0xe3,0x66,0xa7,0x9d,0x98,0xbf,0x75,0x06,0xc4, +0x20,0xd2,0xad,0xf7,0xd6,0xb1,0x13,0x5b,0xcd,0xac,0x8a,0x95, +0xbd,0xa3,0xbd,0x27,0x92,0xf2,0x51,0x6f,0x3e,0x1b,0x47,0x09, +0x1d,0x5f,0x62,0xf4,0x62,0xe1,0xa4,0x04,0x55,0x28,0xac,0xd3, +0x82,0xe8,0xb8,0x9d,0x2a,0x4a,0x3c,0x04,0x62,0x59,0x51,0x5a, +0x88,0x9d,0xb6,0xa7,0x0f,0x77,0x4a,0xb0,0x1b,0xeb,0x57,0x5a, +0xe6,0x85,0x7e,0xab,0x9e,0x5a,0x94,0xbd,0x6b,0x9f,0x95,0x14, +0xfa,0x30,0x94,0x56,0xac,0x4b,0xab,0x58,0x1f,0x77,0xd2,0xb0, +0x4e,0x71,0x9d,0x47,0x40,0xfd,0x37,0xb3,0xfc,0x86,0xab,0x64, +0xe7,0xfc,0xf8,0x64,0x3c,0xbf,0x18,0x5c,0x0e,0xf2,0x7e,0x27, +0xac,0x33,0xf5,0xfb,0x75,0xb9,0x7e,0x0a,0xaa,0xc1,0x45,0xc0, +0x75,0x8a,0xb6,0xcf,0xc6,0x6e,0xfe,0x02,0x50,0x78,0x2f,0x57, +0x28,0xdc,0x32,0xab,0x28,0xb0,0x58,0x94,0x95,0xec,0xe8,0x28, +0xa3,0xd0,0xfa,0xd5,0xd5,0x73,0x86,0xb6,0x2d,0x69,0x33,0x5e, +0xa9,0x8c,0x3c,0x2a,0xd0,0x81,0x59,0x55,0x2a,0x83,0xd9,0xf6, +0x5e,0x3a,0x58,0x51,0x0b,0x42,0x8a,0x38,0x20,0xb4,0x44,0x87, +0x7a,0xe3,0xdb,0x87,0x87,0xdf,0x49,0xde,0x21,0xfe,0xc5,0x0e, +0xa5,0x58,0x73,0x87,0xe6,0x80,0xb7,0x9c,0xb0,0xca,0x69,0x4b, +0xe2,0x8d,0x7a,0x69,0x73,0x6b,0x34,0x0a,0x70,0x90,0xe6,0x2a, +0x56,0x24,0x87,0x65,0xf0,0x44,0x20,0x7b,0x77,0xf0,0x72,0xef, +0xfd,0xc9,0x5f,0x0e,0xe1,0xcf,0xde,0x9f,0x4f,0xde,0x1f,0x1e, +0x1d,0x1c,0xee,0x1d,0x9d,0xfc,0xe5,0x54,0x0b,0xb5,0x78,0x08, +0xdc,0x6d,0xaf,0xd4,0x5b,0xeb,0x58,0x60,0x56,0x12,0x3e,0xb5, +0x11,0xdd,0x34,0x8a,0xfc,0xf0,0xba,0xd8,0xb0,0x39,0x24,0x29, +0xac,0x40,0x43,0x2d,0xce,0xb2,0x99,0x88,0x57,0x21,0xdc,0xea, +0x66,0xb0,0x72,0x80,0x14,0x76,0x42,0xde,0x68,0x56,0x65,0x7b, +0x0e,0xd9,0xb0,0x4d,0x74,0x2e,0x0a,0xfb,0x96,0x07,0xc0,0x3d, +0x04,0x02,0xdb,0x22,0x22,0x49,0x64,0xa4,0x9f,0x9b,0xeb,0xfd, +0x3b,0x7c,0x76,0x28,0xc1,0x65,0x4c,0xec,0xa1,0x3c,0xfe,0x9e, +0x1d,0x96,0xfb,0x92,0x4d,0xfc,0x1c,0xc3,0x6a,0xb5,0x7b,0x82, +0x2c,0xc7,0x6d,0xa6,0x1c,0xc7,0xf7,0x9c,0xce,0xae,0xe2,0x96, +0x5c,0xa7,0x10,0xae,0x90,0x66,0xdc,0xf1,0x38,0xad,0xa4,0x4a, +0xb0,0xe1,0x88,0xe4,0x9e,0x7f,0x9b,0x56,0x16,0xc0,0x63,0x53, +0x04,0xc5,0xf2,0x8e,0x24,0xde,0xb1,0x1a,0xa2,0x84,0xd8,0xe0, +0x17,0xbf,0x29,0x97,0xcc,0xa2,0x41,0x41,0xf7,0xf0,0x1c,0xb5, +0x2c,0xec,0x97,0x98,0x25,0x06,0x21,0xcb,0x26,0xd1,0x74,0xfe, +0x57,0xf6,0x53,0x01,0x38,0x8d,0x3c,0xee,0x07,0xb5,0xf5,0x08, +0x43,0x0e,0xf0,0xd0,0x14,0xe3,0xd4,0x2f,0x1a,0x2e,0xaa,0x3d, +0xf8,0xc1,0x07,0xf4,0x73,0x12,0x59,0xfc,0xf3,0xde,0x70,0x7c, +0x11,0xfb,0x2c,0xf4,0xa2,0xa4,0x54,0x9f,0x34,0x61,0x69,0x94, +0x98,0x2c,0xeb,0x68,0x51,0xa0,0x96,0xb9,0xac,0xde,0x9c,0x5d, +0xb5,0xd0,0x89,0xe1,0xb6,0xf5,0x31,0x86,0xc6,0x9a,0xa0,0xb5, +0xf1,0x76,0x96,0x5d,0x8e,0xdc,0x13,0xba,0xcb,0xd1,0x5a,0x96, +0x85,0x6b,0x6d,0x10,0xe9,0xe5,0x84,0x0e,0x42,0x4c,0x09,0xa2, +0xf6,0xa5,0x23,0x5d,0x28,0x87,0xbd,0x98,0x27,0xe1,0x37,0x6b, +0xfc,0x9c,0x46,0x09,0x9b,0xd6,0x60,0xd7,0xe5,0x28,0xb3,0xd0, +0x51,0x92,0x48,0x40,0xa2,0x67,0x59,0x95,0x9c,0x2e,0xb6,0x84, +0x7b,0x22,0x08,0x31,0x98,0x45,0xb1,0x2a,0x17,0x44,0x53,0x4e, +0x36,0x47,0xa5,0x7d,0xf7,0x3b,0x43,0x4f,0x4c,0x00,0xd0,0x4e, +0x98,0x35,0xe6,0xc3,0x07,0x7b,0x77,0xe5,0xb0,0xee,0x38,0x6b, +0x2a,0x0a,0x41,0x83,0x4f,0xc0,0x54,0x90,0x2c,0x44,0x7d,0x78, +0xf1,0xc1,0xdb,0x85,0x72,0x73,0x3b,0xb3,0xe8,0x7c,0x60,0xa4, +0xb9,0xa7,0xdb,0x8a,0xee,0xf8,0x7c,0xa3,0xe1,0x4a,0xa4,0xee, +0x6b,0x19,0x78,0x1e,0x58,0xdd,0x55,0x85,0x7e,0x91,0x2a,0x74, +0x6b,0xb5,0x64,0xc9,0xab,0x99,0x12,0x23,0x8f,0x5c,0x04,0x88, +0xb2,0x48,0x6d,0xa9,0x26,0x3d,0xa3,0x1c,0xb6,0xd6,0xc6,0xa3, +0x57,0xa5,0x8d,0xa4,0x99,0x8f,0xc4,0xb1,0x85,0x1b,0x63,0xbb, +0x8a,0xe8,0x14,0xd1,0xc3,0xd3,0x9b,0x41,0xdc,0x6d,0x02,0xae, +0xe8,0x3e,0x83,0xa3,0xff,0x0d,0x17,0x8e,0xb2,0xbf,0x0c,0xf4, +0x64,0xf1,0x37,0x7b,0x4a,0xfd,0xb7,0xda,0x78,0x52,0xfb,0x1b, +0x3b,0xf6,0x04,0x94,0xff,0x86,0x36,0x43,0x80,0xa0,0xf1,0x52, +0x2b,0x92,0xcb,0x67,0x4c,0x9b,0x0a,0xe1,0xec,0xf1,0xb1,0xc2, +0x31,0x94,0xcc,0xbd,0x28,0xca,0x59,0x22,0x34,0x7e,0x8d,0x96, +0x81,0x1f,0xcc,0xcb,0x59,0xac,0x75,0xea,0xe9,0x62,0xf1,0x04, +0xc0,0x46,0x70,0xb8,0x5f,0xda,0x2a,0x1e,0x2e,0x26,0x75,0x58, +0x74,0xc4,0x50,0x2b,0xc2,0x70,0xa6,0x8b,0x25,0x2b,0x03,0x4e, +0xfc,0x78,0xd2,0x49,0x9c,0xec,0x3c,0x6a,0x7a,0xe7,0x41,0xde, +0xa6,0xee,0x5d,0x87,0xbc,0x1a,0x4d,0x59,0xcb,0x77,0xca,0xae, +0xa7,0x48,0x71,0xd2,0xa6,0x30,0xbf,0xbf,0x19,0xcf,0xa7,0x79, +0x8e,0xce,0x49,0xc4,0x44,0x21,0x3e,0x50,0xc7,0xb8,0x61,0xde, +0xbb,0xcb,0x25,0xce,0xf9,0x72,0x83,0x4d,0x0f,0x80,0x0c,0x6d, +0xa1,0xdb,0x68,0x65,0x6e,0x5c,0xe2,0xe9,0xb1,0xdd,0x4b,0x89, +0x3c,0x1c,0xb3,0xe9,0xec,0xc2,0xfb,0x4f,0x7a,0x68,0x87,0x7a, +0xca,0xd9,0xef,0x60,0xe2,0xf7,0xdc,0xa2,0xdf,0xe9,0xb9,0x53, +0x12,0xab,0x7d,0x89,0x3b,0xb0,0xdb,0xec,0xbc,0x5e,0x3f,0x97, +0x77,0x52,0xc8,0x1b,0xdd,0x43,0x3b,0x28,0x0d,0xc9,0x8b,0xc5, +0x5a,0xbc,0x46,0xbf,0x68,0x7e,0x0a,0x3e,0x1c,0x40,0x4c,0xa1, +0x8a,0x1c,0xc2,0x9d,0x30,0x18,0x43,0xee,0x24,0xed,0xad,0xc2, +0xbb,0x5e,0x6f,0xfd,0xa6,0xbe,0x32,0x95,0x0a,0x27,0xfa,0xe8, +0x85,0x5a,0x87,0x8e,0x03,0xd8,0xaf,0xe9,0x79,0x16,0x60,0x8d, +0xac,0xee,0x1c,0x1b,0x5c,0xb6,0x46,0xb8,0x6c,0xeb,0x21,0xb3, +0xe7,0xb3,0x14,0x77,0xd3,0xbb,0xcd,0xee,0xfd,0xe8,0xa4,0x3c, +0x52,0x63,0xd4,0x97,0xf6,0xe3,0x68,0x63,0xef,0x60,0x48,0x97, +0xed,0x60,0xf6,0x5a,0x20,0x0c,0xba,0x52,0x31,0x5c,0xbb,0xe2, +0x9d,0xe4,0x43,0x34,0xac,0x23,0x8a,0xe2,0x12,0x3a,0xa1,0x0c, +0xed,0x78,0x4d,0xc2,0xa8,0xe0,0x4c,0x1f,0xda,0x65,0xab,0xeb, +0x53,0xf1,0xca,0x2b,0x59,0xf0,0x21,0x28,0x93,0x3b,0x57,0xc9, +0x3e,0xe7,0x96,0x22,0x4c,0x97,0xcf,0xf8,0x42,0xb6,0xc3,0x25, +0x13,0x53,0xec,0x9e,0x4a,0x52,0x5f,0xda,0x58,0xb9,0xb7,0x1e, +0x61,0x35,0x68,0x0b,0x47,0x18,0xdd,0xc1,0xe5,0xa5,0x91,0xb3, +0xe4,0xff,0x1a,0xcc,0xae,0xcb,0x22,0x80,0x24,0xe2,0x68,0xc9, +0x22,0x49,0xd7,0x6d,0xe2,0x87,0x3e,0x7c,0xb0,0x47,0x43,0x1a, +0xac,0xdd,0x56,0x64,0x72,0x53,0x16,0xcf,0x36,0xe5,0xc0,0x48, +0x43,0xf6,0x63,0x30,0xa2,0x8a,0xa8,0x92,0x8e,0x78,0xeb,0x47, +0x5f,0x6a,0x93,0xd9,0x8f,0xf9,0x25,0xf2,0x0d,0x4c,0x37,0x94, +0xdc,0xd4,0x0e,0x70,0x53,0xc9,0x2b,0x40,0x77,0x51,0x66,0xe0, +0xbc,0x0e,0x05,0x2a,0x44,0xe2,0x1c,0x76,0x01,0x49,0x15,0x50, +0xa4,0x62,0x43,0x4f,0x93,0x5c,0x92,0x1f,0x90,0xe6,0xbc,0x5c, +0x62,0xca,0x30,0xf0,0x15,0x41,0x51,0x80,0xaf,0xd7,0x2d,0x54, +0x51,0xa8,0x91,0xcc,0x78,0x55,0x2b,0x09,0x7c,0x08,0x81,0xc3, +0xf7,0xf8,0xe9,0xb7,0xaf,0x1f,0xef,0x39,0x60,0xa7,0x7b,0x7b, +0x9b,0x8f,0xfa,0x15,0x9b,0x1d,0xea,0xd1,0x8a,0x01,0x18,0x85, +0x3d,0x4f,0xf0,0x92,0xfb,0xb8,0x0a,0x75,0xe0,0x23,0x55,0xd1, +0x5a,0x79,0x87,0x6a,0xe7,0x7e,0x67,0x48,0xd8,0xcb,0xb7,0x30, +0x1e,0x0f,0x20,0xc5,0x06,0x70,0xcb,0x60,0x3d,0xa1,0xb9,0xb4, +0xcb,0xc1,0x64,0x3a,0x23,0xf0,0xed,0x27,0x37,0xc5,0x8b,0x8a, +0x8a,0x80,0x28,0x8d,0x89,0x81,0xae,0x6b,0x0d,0xba,0x0d,0x2c, +0x23,0x88,0xb1,0x4c,0xf4,0xf6,0x2b,0x93,0xf3,0x41,0x97,0x82, +0xd6,0xda,0xc4,0xc0,0xf7,0x8a,0xb9,0xc0,0x9f,0x64,0x90,0xbb, +0x44,0x9c,0x16,0x8c,0x27,0x50,0x1b,0x13,0xf4,0xa5,0x23,0x12, +0x39,0x36,0xaa,0xa0,0x8b,0xca,0x19,0xf7,0xb5,0x98,0xd1,0xe5, +0x76,0xa1,0xc6,0xde,0x25,0xf2,0xd1,0xf2,0xd8,0xe5,0x9f,0xf6, +0xf4,0x5c,0x09,0x06,0xea,0x01,0xce,0x50,0x35,0x6e,0x1e,0x54, +0x91,0x05,0x7c,0xc5,0xe4,0x2f,0x4c,0x6d,0x7b,0x97,0x2f,0x2c, +0x6e,0x47,0x82,0x46,0xdd,0xe0,0x07,0xe2,0x1f,0xa7,0xf2,0xc3, +0x3f,0x4e,0x2d,0x35,0xd9,0x4a,0x60,0x38,0x3d,0xfb,0xec,0x1e, +0x25,0x50,0x67,0xf3,0xd1,0x68,0x3a,0xd4,0x06,0xb2,0xb5,0x70, +0xd3,0x50,0x02,0x07,0xb2,0x8c,0xcf,0xdd,0x09,0xd1,0x2d,0xa3, +0x98,0x94,0xcb,0x2f,0xa5,0xbb,0x57,0x70,0xb1,0xd5,0x63,0xe1, +0x8d,0xce,0x11,0x19,0xb4,0x50,0x4c,0x90,0x1e,0x77,0xa2,0x87, +0x80,0xa6,0x57,0x55,0xa4,0x90,0x57,0x51,0x8b,0x3a,0xbf,0xc1, +0x1c,0x32,0x98,0x76,0x60,0x8a,0xf7,0x75,0xa5,0x1c,0x6e,0xaf, +0x98,0x0f,0xfd,0xeb,0x6f,0x35,0xb4,0x6d,0xac,0x1c,0xef,0x17, +0x6f,0x00,0xc7,0x35,0xe4,0x14,0x37,0x0e,0xbf,0x76,0xc2,0x85, +0x6e,0x82,0x02,0xae,0xca,0x1b,0xd8,0x10,0x0e,0xaa,0x58,0x4f, +0x49,0xd5,0x4a,0x1d,0x31,0x09,0x76,0xd3,0x1f,0xbf,0x9c,0xf4, +0xae,0x70,0xd7,0x1f,0xab,0xae,0xbe,0x18,0x8e,0x47,0xb9,0x3d, +0x1a,0xc6,0x6f,0x7b,0xc3,0x21,0x82,0x6a,0xb9,0x2a,0x5a,0x7c, +0x11,0x8c,0x71,0x42,0x7b,0x41,0x2c,0x8d,0x9f,0x22,0xaa,0x93, +0x5b,0xa8,0x53,0x07,0x4c,0x96,0x7a,0x84,0x96,0xb9,0x2f,0xc8, +0x53,0xf0,0xf9,0x8f,0x07,0x88,0x66,0x3a,0x1b,0xdf,0xe2,0x89, +0x4f,0xef,0x8a,0x1e,0xa6,0x52,0xe4,0xd2,0xbf,0xa8,0x16,0x34, +0x4c,0x78,0xc3,0x34,0x62,0x4b,0x54,0x41,0x2b,0x08,0x77,0xe2, +0x97,0xff,0xc0,0x36,0x1b,0xff,0x3c,0xf7,0xd6,0x5a,0xf8,0x10, +0xaa,0x7a,0x63,0x57,0xda,0x8c,0xc6,0x77,0xc1,0x9e,0x52,0x6f, +0xf8,0x18,0x2c,0x9f,0x56,0x84,0x6e,0x0c,0xa5,0x90,0x70,0x6e, +0x39,0xb8,0x48,0x3c,0x63,0xa0,0xe5,0x53,0x6c,0xed,0xad,0x86, +0xac,0x9f,0x30,0x85,0x48,0x61,0x01,0xb9,0x78,0x37,0xc5,0x5e, +0x68,0x9e,0x03,0xf9,0x95,0xee,0xf4,0x9a,0xe3,0x51,0x39,0xe7, +0x7c,0xb4,0x22,0xef,0xe5,0x25,0x08,0x07,0xc0,0x3e,0xae,0xf7, +0x7b,0xb7,0xfa,0x5e,0xf0,0x76,0xae,0x26,0x23,0x6e,0xf5,0xe5, +0xa4,0x86,0x1a,0x80,0xee,0x91,0xfe,0x98,0x7f,0xc1,0x78,0x77, +0xe9,0x83,0x47,0x62,0x41,0x89,0xc0,0xeb,0x75,0x58,0xa4,0xbc, +0xd4,0x7c,0xcc,0xbf,0xe8,0xfb,0x3a,0x2a,0x82,0x4f,0x27,0x83, +0x62,0xb6,0x93,0xe4,0x25,0xbf,0xcd,0xd4,0x35,0x72,0x3e,0x43, +0x2b,0xd2,0xab,0x77,0xef,0x77,0x8e,0x7e,0x7f,0x9c,0x6d,0xfe, +0x6c,0x81,0x9f,0x4d,0xbf,0x3b,0xfd,0xf9,0x2c,0xee,0x7e,0x77, +0x16,0xc3,0x67,0x0c,0xdf,0x49,0xf7,0xbb,0xe4,0x2c,0xd9,0xbc, +0x31,0x9c,0x99,0x6d,0x67,0x65,0x9b,0x66,0x53,0x22,0x32,0xd2, +0x83,0x88,0xdf,0xa3,0x71,0xdc,0xe3,0x46,0x27,0x39,0x6b,0xb1, +0x12,0x04,0xda,0xfd,0x3a,0x7c,0xbf,0x7b,0xb0,0xcf,0x17,0x5a, +0x9b,0x71,0x7c,0xb6,0x89,0x6a,0x23,0xeb,0xc9,0x02,0xbf,0xbe, +0x3b,0x3d,0x9b,0x9e,0x1d,0x77,0xbf,0xeb,0x9c,0x7d,0x77,0xb6, +0x89,0x77,0xb7,0x37,0xde,0x31,0x7e,0xe1,0x86,0x56,0x5d,0xd7, +0x98,0x75,0xf6,0x6b,0x54,0xbc,0xc4,0xb5,0xd1,0x98,0x83,0x5d, +0x5f,0x39,0x7f,0xa2,0xe4,0xe3,0x93,0xbf,0x03,0x57,0x9e,0xd6, +0x4b,0x86,0xf8,0x19,0xb7,0x9c,0x05,0xb7,0xca,0x26,0x74,0x6c, +0xe1,0x97,0xc2,0xc0,0x7d,0x86,0x7d,0xb1,0x4a,0x8e,0x29,0xa2, +0x0d,0xef,0x3c,0xac,0x2d,0x17,0x3b,0xf5,0x7a,0xd4,0x8c,0xd6, +0x28,0x5d,0x59,0x9d,0x11,0xe7,0x9d,0x41,0x3b,0xf0,0x01,0xe0, +0xec,0x02,0x50,0x70,0xef,0xff,0xd0,0xa7,0x26,0x29,0xef,0xd5, +0x2c,0x62,0x35,0xfb,0x9c,0xaf,0xf6,0x09,0xb6,0x06,0xb5,0x67, +0xcd,0x67,0xb5,0xab,0xf1,0xac,0xf6,0xec,0x7e,0x6b,0xf9,0xac, +0x19,0xf1,0xdc,0x6f,0x17,0x1b,0x43,0x73,0xdf,0xdb,0xea,0xef, +0xe2,0x4b,0x59,0x53,0x72,0xf8,0x81,0x91,0xce,0xd1,0x47,0xe8, +0xbe,0x43,0x1e,0xf3,0xea,0x9e,0x93,0x28,0xa7,0xf5,0xca,0xfb, +0xbb,0x12,0xb7,0xe5,0xf5,0x8a,0xa5,0x16,0xd3,0x07,0xfe,0x2c, +0xdb,0x6a,0x0c,0x93,0x00,0x47,0x5f,0xf5,0x3a,0xfd,0x9c,0xd2, +0x5f,0x6b,0x4b,0xb4,0xd5,0x35,0x7e,0x71,0xcc,0x42,0xe1,0x88, +0x3c,0xee,0x39,0x20,0x52,0x9c,0xfd,0xd4,0xb8,0x22,0x5c,0xe9, +0x3b,0xbf,0x0e,0x65,0x95,0x05,0xd4,0x42,0xc5,0xcf,0xcb,0xdb, +0x85,0x4d,0x45,0xb5,0x10,0x44,0x65,0xcb,0x82,0x50,0xb1,0x4e, +0xdc,0x48,0x60,0xb3,0xeb,0x75,0xdb,0x63,0x31,0x06,0xcd,0x16, +0x69,0x43,0x18,0xde,0x2b,0x97,0x7a,0x8d,0x3b,0x2a,0xbc,0xef, +0x89,0x1f,0x04,0x54,0x29,0x86,0x56,0xf5,0xbe,0xb8,0xd1,0x9d, +0xa9,0x33,0x9c,0x72,0x2e,0x25,0xaf,0x95,0x60,0x12,0x3d,0xd2, +0xfa,0xc9,0x00,0x5d,0xd8,0x3b,0xa4,0x73,0x51,0x49,0xc7,0x7d, +0xa6,0x81,0x1a,0x6e,0x21,0xd1,0xab,0xc3,0x6a,0xbb,0x38,0xa1, +0x66,0x30,0x15,0x79,0xa8,0x0b,0xb4,0x28,0xf9,0x2f,0xc4,0xba, +0xa4,0x1a,0xfd,0x28,0xe2,0xe2,0x0a,0x96,0xe5,0x80,0x65,0x77, +0xd9,0x35,0xeb,0x78,0x94,0x33,0x18,0x96,0xf8,0x9b,0x44,0x47, +0x49,0x7b,0x7d,0x97,0x3f,0x2d,0x7f,0xf3,0x3a,0x73,0x9e,0xcf, +0xf1,0x4e,0xe8,0xf0,0x68,0xef,0xd5,0x9b,0x3f,0x7b,0x55,0x98, +0xf8,0x33,0xa9,0xcb,0xbc,0xef,0x2e,0xf0,0x4e,0x5f,0xbe,0x93, +0xcd,0x81,0xf9,0xf3,0xeb,0x23,0xab,0x04,0xf4,0xe7,0xfd,0xb7, +0xaf,0x67,0xb3,0xdb,0x23,0xf6,0x75,0xb4,0x58,0x68,0x73,0x00, +0x93,0x2f,0x4e,0x1d,0x1e,0xf6,0x0c,0xec,0xa7,0xee,0xcf,0x72, +0x8a,0x1e,0xed,0x4f,0x3f,0xdf,0x0c,0xb7,0xa9,0xfc,0xc9,0xc9, +0x61,0xf3,0x37,0xcd,0xad,0xc8,0x39,0x45,0x6b,0x25,0xf7,0xcb, +0xaf,0x2a,0xfd,0x5c,0x97,0xde,0xfe,0xca,0xd2,0xbe,0xe4,0x73, +0x2c,0x19,0x3c,0xba,0x5e,0x47,0x3f,0xc3,0x3f,0xf6,0x2e,0x3e, +0xc2,0x8e,0x8e,0xdf,0x5d,0x7f,0xbe,0x46,0xff,0x76,0x27,0xe4, +0xcd,0x98,0x1f,0x0e,0x95,0x8f,0x7d,0xc3,0x4e,0x41,0x87,0xc3, +0xb8,0x56,0xc1,0x9c,0xb9,0x1d,0x0f,0x2b,0xd6,0x22,0x95,0x04, +0xeb,0xd1,0xf1,0xde,0xd1,0x9f,0xf6,0x8e,0xde,0xef,0xef,0x9c, +0xec,0xbe,0xc6,0x41,0x38,0xfd,0x39,0xed,0x36,0x92,0x14,0x17, +0xc3,0xf8,0xec,0x53,0x23,0xbd,0xdf,0x32,0xad,0xe5,0xd9,0xa7, +0xef,0xfe,0x23,0xe9,0xc4,0x67,0xf7,0x9d,0xd3,0xb3,0x4f,0x67, +0xcd,0x0d,0x58,0x7f,0x97,0xb0,0xa0,0xa6,0xf1,0xe9,0xd6,0xc6, +0xef,0xba,0x68,0x0f,0x14,0xd6,0x4c,0x58,0x8c,0x3b,0xff,0x06, +0xab,0x31,0x7c,0x77,0x00,0x0c,0x7e,0xc2,0xf7,0xbf,0xe1,0xeb, +0x79,0x34,0x08,0x6a,0x0e,0x77,0x4e,0x5e,0xeb,0x7a,0x38,0xf7, +0x8a,0xcc,0x2f,0xf7,0x5e,0xed,0xfc,0xf4,0xf6,0xe4,0xfd,0xe1, +0xc1,0x11,0xaa,0x9b,0x90,0xff,0xe5,0xdf,0x6e,0x19,0xfc,0x9d, +0xa6,0xdf,0x7f,0xff,0xdc,0x5c,0x42,0xc4,0x76,0x6b,0xc9,0x36, +0x27,0x91,0x04,0x8a,0xcd,0xb4,0xf1,0x40,0x93,0x6f,0xe5,0x13, +0x65,0xa3,0xf3,0xde,0xe8,0xea,0xcd,0xe8,0xf5,0xec,0x66,0xf8, +0x6b,0xe7,0x07,0x8b,0xc4,0xa4,0x62,0xa6,0x15,0xa9,0xe5,0x82, +0xf7,0xeb,0xeb,0x78,0x21,0xf6,0x6b,0xf4,0x89,0xb8,0xbe,0x2e, +0xdb,0x76,0x0c,0xf4,0xce,0xa7,0x90,0x33,0xb5,0x88,0x88,0x6d, +0xe0,0x68,0x7d,0x9d,0x13,0x22,0x72,0x8e,0xe6,0x39,0x02,0xba, +0x65,0x93,0xd2,0xa5,0x5b,0x34,0x48,0x73,0x77,0x68,0xb2,0x88, +0xa2,0xc7,0x36,0xef,0x15,0xc0,0x77,0x2e,0x2b,0xbb,0x62,0x01, +0x2b,0x63,0x59,0x9b,0x04,0xf5,0x3a,0x15,0x45,0x9f,0x20,0x71, +0x3f,0x47,0x85,0xc4,0x9f,0x8e,0xde,0xe0,0x74,0x25,0x3b,0xfe, +0xde,0x0e,0x2e,0xec,0x33,0xad,0xf5,0x82,0xc5,0xc2,0xc5,0x4a, +0x69,0x71,0x3b,0x62,0xcd,0xe2,0x92,0xe5,0x7d,0x5e,0x93,0x51, +0xbe,0x53,0xf6,0x6f,0x81,0xf3,0xd8,0xd6,0x50,0xfa,0xd2,0x7b, +0x70,0x2b,0xf7,0x88,0x4d,0xc2,0x9b,0x06,0x74,0xe3,0x56,0xce, +0x81,0xd1,0x90,0x8a,0x04,0x5f,0x55,0x1e,0xa2,0x23,0x32,0xb5, +0x70,0x5d,0x48,0x3d,0xf6,0x79,0x7a,0xf8,0xcc,0xdb,0x75,0x37, +0xf9,0x4a,0x91,0x59,0x1b,0x6d,0xa2,0x71,0x3d,0xbc,0xc0,0x72, +0x82,0x50,0x87,0x60,0x41,0x42,0x03,0x3f,0xd0,0xe7,0x30,0x35, +0x5c,0xbb,0x96,0xc6,0xb0,0xa1,0x5b,0x15,0x7e,0xea,0x74,0x2f, +0xfe,0x6f,0x8a,0xce,0xde,0xd9,0xfd,0x4d,0x6d,0x2b,0x0d,0x07, +0x90,0x01,0x90,0x3f,0x96,0x5a,0x2b,0xa5,0x01,0xb7,0xea,0xe9, +0x94,0x82,0xb2,0x98,0xca,0xc8,0x8f,0x71,0x41,0x80,0xe6,0xc7, +0x4f,0x92,0xc7,0x99,0x43,0x5a,0x73,0x37,0x86,0xbe,0x34,0x49, +0x72,0xe1,0xfc,0x88,0xa3,0xc1,0x74,0x82,0x8d,0xbc,0x22,0xb6, +0x92,0xd7,0xe8,0xa8,0xcc,0xf9,0xa7,0x47,0x27,0xe8,0x74,0x8b, +0xe4,0x8a,0xfd,0x9b,0x0c,0x78,0xf2,0xb7,0x1a,0xee,0xf3,0x6a, +0x37,0x73,0xc8,0x4e,0x46,0x1f,0x58,0xf9,0x1f,0x6f,0x99,0x7a, +0xe4,0xa9,0x08,0xb5,0xca,0x23,0x25,0xe3,0x11,0xd2,0xfc,0xb3, +0x0c,0x3d,0xc9,0xf8,0x3e,0x53,0xdb,0x23,0x6b,0x42,0xc7,0x27, +0x76,0xd4,0xe6,0xc0,0x41,0x3c,0xe5,0x9f,0x6e,0x5a,0x19,0xab, +0x4a,0x2f,0xc3,0xde,0xc6,0x15,0x6a,0x4c,0xe6,0x2b,0x99,0x18, +0x91,0x5a,0x57,0x92,0x0a,0x26,0x46,0xc6,0x7b,0x2d,0xb7,0x93, +0xba,0xac,0xeb,0x2d,0xd0,0x25,0x03,0x6a,0xd2,0xf0,0x3d,0x22, +0x4d,0x4d,0x7e,0x06,0x56,0x64,0x4c,0xec,0xd1,0x3f,0x31,0xa8, +0x9e,0xb0,0x73,0x72,0x70,0x04,0xec,0x2d,0xc2,0xa6,0x47,0x15, +0xe0,0xf9,0x38,0x05,0xbd,0x66,0xe7,0x15,0xc9,0xe8,0x4c,0x2c, +0xa2,0x67,0x63,0x55,0x89,0xc0,0x1c,0xbd,0x82,0x0b,0xed,0xef, +0xd1,0x2a,0xba,0xa2,0x5f,0xe7,0xeb,0xdc,0x68,0x7b,0x8f,0xbd, +0xac,0x17,0xf8,0x41,0x37,0xe7,0xd9,0x79,0x18,0xe1,0xef,0x2d, +0x7b,0x49,0xc7,0x07,0xce,0x93,0x4e,0xaf,0x71,0x9e,0xf6,0xd2, +0x20,0xea,0xdc,0x2b,0x1f,0x2d,0xd1,0xc0,0xe0,0x3f,0x03,0x81, +0x38,0xc0,0xa0,0x97,0x6e,0x25,0x1b,0x71,0xa1,0xd2,0x2d,0x58, +0x04,0xa3,0xef,0x9e,0x52,0x5b,0x00,0xfa,0xbb,0xb0,0x26,0x80, +0xb1,0xf9,0xd5,0x30,0x36,0x4b,0x30,0x7e,0xf5,0xd5,0x30,0x7e, +0x55,0x82,0xf1,0xf3,0x57,0xc3,0xf8,0xb9,0x04,0x23,0x8b,0x84, +0x0e,0xb2,0x2c,0xfb,0x6a,0x70,0x78,0x57,0x58,0x04,0xb8,0xf6, +0x0d,0x70,0xd6,0x2a,0xe0,0x7c,0x13,0x3a,0x15,0xd8,0x7c,0x03, +0x32,0x25,0x28,0x3f,0x7c,0x35,0x90,0x1f,0x4a,0x30,0x5e,0x7c, +0x35,0x8c,0x17,0x65,0x3c,0xbe,0xbe,0x35,0x3f,0x94,0x5b,0xf3, +0xe2,0xeb,0xa1,0xbc,0x28,0x43,0xa9,0xd7,0xbf,0x1a,0x4a,0xbd, +0x5e,0x82,0x02,0x32,0xc1,0xd7,0x42,0x59,0x2c,0xca,0xb8,0x7c, +0x3d,0x2a,0x65,0x4c,0x9e,0x00,0x23,0x2c,0x14,0x66,0x0b,0x92, +0x88,0xf4,0x56,0x01,0x74,0xdc,0x38,0x2c,0xb3,0x5c,0x9a,0xbd, +0xe3,0xdd,0x9d,0xc3,0xbd,0xec,0x7e,0x94,0x46,0x67,0x68,0x70, +0x0c,0x7e,0x2e,0x41,0x64,0x82,0x1f,0x90,0xf8,0x67,0x69,0xf4, +0x3f,0x22,0x73,0x97,0x46,0xff,0x2b,0x42,0x83,0xe8,0x29,0xfc, +0x33,0xcf,0xa2,0x67,0x29,0xfc,0xa3,0xd3,0x39,0xf4,0x92,0x39, +0xda,0xb5,0x0f,0x44,0xd6,0xa7,0x17,0xa5,0x55,0x06,0xa2,0x50, +0xba,0xdf,0xdd,0x7b,0xbf,0x7b,0xf0,0x0e,0x15,0xe2,0x60,0x99, +0x41,0x05,0xa8,0x34,0x42,0x61,0x35,0x32,0xbb,0xc7,0xc7,0x69, +0x74,0x81,0xaf,0x2f,0x7f,0x3a,0x82,0x48,0x32,0xb1,0x70,0xb4, +0x77,0x7c,0xf0,0xd3,0x11,0x14,0xa1,0x28,0xf4,0x33,0x3b,0x9f, +0x5c,0xe4,0x28,0xaa,0x9a,0x3f,0x40,0xee,0x0f,0xd3,0x68,0xd9, +0x2e,0xf8,0x73,0xad,0xdc,0xde,0x15,0x1c,0x9d,0xea,0x3c,0x6c, +0x21,0x1e,0xb2,0x68,0x87,0x64,0x55,0x19,0x68,0x01,0x55,0xde, +0x8f,0xb2,0xa8,0x19,0x19,0x6d,0x85,0x3e,0xbb,0xff,0x02,0xff, +0xa5,0xca,0x7d,0x4b,0x64,0xfd,0x4c,0x44,0xe6,0xfb,0xc4,0xac, +0x4c,0xdb,0xc6,0xed,0xed,0x16,0x64,0x58,0x91,0xde,0x4a,0xcc, +0x3e,0xfc,0x97,0x86,0x9e,0x53,0xa2,0xfd,0xf1,0x08,0xe4,0x47, +0x4a,0xab,0x4e,0x22,0xa0,0x92,0x58,0x48,0xd9,0x26,0xa0,0x55, +0x29,0x2d,0x4c,0xe9,0xf7,0x83,0x24,0xf6,0x72,0xb0,0x0d,0xf1, +0x15,0xd1,0x90,0xfd,0xf5,0xeb,0x20,0x9e,0xfc,0x64,0x50,0xfe, +0xca,0x78,0x28,0x70,0x7d,0x5d,0x59,0xc0,0x6c,0xa0,0x17,0x99, +0xca,0xb4,0x16,0xa7,0xdd,0xdc,0x84,0x38,0x0f,0x46,0xa8,0x81, +0x4c,0x75,0xad,0x48,0x81,0xda,0xa6,0xd3,0x20,0xe9,0x38,0xc7, +0x4b,0x2b,0x2e,0xb4,0x22,0x85,0x0a,0x4d,0x0b,0x00,0x87,0xc3, +0xc1,0xd4,0x66,0x78,0x9e,0x98,0x3d,0xf8,0xaf,0xd8,0xef,0x2f, +0x7b,0x5f,0x22,0x4a,0xa9,0x4a,0xe0,0xf7,0xc6,0xa9,0xf7,0xca, +0x62,0xfe,0x9a,0x86,0x3e,0x66,0x96,0xa6,0xec,0x75,0x01,0x0f, +0x82,0x3b,0xe9,0xe9,0xcf,0x5f,0xf6,0xfb,0xaf,0xaf,0x6f,0xa6, +0xbd,0xbf,0xee,0x3d,0x83,0xfd,0xed,0x02,0xe2,0x9e,0x51,0xfc, +0xb3,0xee,0xe2,0xd9,0xb3,0xe4,0xbb,0x67,0x14,0xb5,0xd7,0x58, +0x7c,0x69,0x2c,0xf6,0x1b,0x8b,0x7e,0x63,0xf1,0xba,0xb1,0xb8, +0x6e,0x2c,0x6e,0x1a,0x8b,0x69,0x63,0xd1,0x5b,0xfc,0x35,0x49, +0x70,0x2b,0xbb,0x69,0x02,0x2f,0x13,0x78,0x16,0xdd,0x6f,0xac, +0x6f,0xb6,0xbd,0x33,0x97,0x95,0xa4,0x5f,0x70,0xc6,0xeb,0x1c, +0x2c,0xbb,0xf8,0xa4,0xe8,0x47,0xd7,0x65,0x99,0xfb,0x07,0xa4, +0x81,0x35,0xfd,0x60,0xae,0x92,0x10,0xca,0x55,0x21,0x43,0xd8, +0x19,0x5d,0x5c,0x8f,0x27,0xce,0xce,0x80,0x83,0xa5,0x8c,0x06, +0xec,0x45,0x46,0x8e,0x79,0xca,0xa7,0x58,0x6c,0x30,0xa0,0xa4, +0xfe,0x4f,0xcf,0xbd,0xd1,0x31,0x36,0xdb,0x4c,0x56,0xe6,0x93, +0xc9,0x94,0x83,0x75,0x00,0x8e,0x5b,0xc8,0x50,0x35,0x20,0x2e, +0xf8,0xe2,0x86,0x5d,0x2a,0x86,0xe2,0xe8,0xcd,0x1e,0xec,0x58, +0x3e,0x47,0x49,0xa2,0xae,0x80,0x03,0xdb,0x05,0xfe,0x74,0xb2, +0xca,0xbe,0x84,0x28,0xed,0x68,0xad,0x7a,0x41,0x02,0x96,0x1a, +0x56,0xd9,0x09,0x2f,0xc6,0xd8,0x36,0x28,0x6c,0xff,0x46,0x57, +0x4e,0xa3,0x7c,0x67,0x38,0xe8,0x79,0x93,0x0c,0x53,0xfd,0x18, +0x46,0x2b,0xfe,0x6b,0x7b,0xf2,0xe3,0x5b,0x3a,0xe2,0xc3,0xfa, +0xe8,0xd4,0x8d,0xd5,0xc8,0x44,0x21,0x18,0x04,0x17,0x9b,0xc3, +0x2a,0x45,0x4e,0x6e,0x7a,0xc3,0xc1,0xdf,0xf3,0xbe,0xb7,0x30, +0xfe,0xce,0xc6,0xc5,0xfc,0xf0,0xcf,0x81,0x6a,0xaf,0xc6,0xec, +0xd4,0x03,0xea,0x96,0xdf,0xa9,0xde,0xdf,0x4e,0x06,0xe3,0x09, +0xec,0x6f,0xd2,0xd6,0xd6,0x56,0x79,0x68,0xcb,0x4e,0x17,0xaa, +0x6c,0x44,0x54,0xbc,0xff,0xf7,0x75,0x96,0x5e,0xf7,0xf8,0x91, +0xb7,0xe8,0x9b,0xb5,0x35,0x67,0x50,0x85,0xfe,0xf3,0xda,0x3f, +0xa7,0x11,0xec,0x54,0x81,0x38,0xe0,0x2f,0x94,0x80,0x0f,0x1a, +0x27,0x05,0xd4,0xf7,0xe6,0xbf,0xa2,0xd3,0x7e,0xf7,0xbb,0x27, +0x98,0xd0,0x08,0xed,0x1d,0x78,0xc8,0xa5,0x9e,0xa1,0x1f,0x3b, +0x63,0xc2,0x1e,0xe2,0x1c,0x86,0xa7,0x94,0xbb,0x9d,0x07,0x82, +0xf1,0x59,0xa8,0xe3,0x6d,0xa8,0x6b,0x2d,0x0b,0x58,0x7f,0x94, +0xde,0x02,0x7a,0x76,0xaf,0xcc,0x82,0xf3,0x5e,0x20,0xb4,0xcc, +0x2d,0x71,0xca,0x34,0x90,0x8f,0x21,0xf3,0xbe,0x3e,0x68,0xad, +0xe7,0xf2,0xe9,0x70,0x3b,0xb4,0x6a,0xaf,0x59,0x8d,0xa0,0x0c, +0x03,0xb8,0x4e,0x86,0xd3,0x23,0xf7,0x8c,0x8a,0xd9,0x0f,0xbe, +0x0d,0x73,0xdd,0xff,0x8a,0xef,0x79,0x7c,0xc7,0x0f,0xa6,0xef, +0xae,0xb4,0xaf,0xb2,0x47,0x2e,0x7d,0x4a,0x00,0x41,0xc0,0xa2, +0x57,0xb7,0x64,0x10,0xde,0x14,0xb8,0x99,0x7b,0xc2,0x15,0x62, +0xbf,0x7a,0x32,0xe0,0xb5,0x79,0x71,0xc4,0x11,0xf2,0x9e,0x1a, +0x75,0x05,0x97,0xdf,0xfa,0xd0,0x98,0xb2,0x17,0x3f,0xd1,0xb4, +0x08,0x58,0x4c,0xd9,0xb6,0xad,0xe5,0x52,0x15,0xfc,0xa8,0x53, +0xcd,0xa4,0x52,0xab,0x6e,0x88,0x48,0x5a,0x53,0x3b,0xcb,0x76, +0x85,0x76,0xa0,0xc2,0x16,0xdf,0x69,0x47,0xd3,0xf9,0xf9,0xcd, +0x00,0xfa,0xb2,0x1a,0xa7,0x44,0xb7,0x8e,0x38,0xea,0x7a,0x1f, +0xbb,0x70,0xfc,0x25,0xb0,0xda,0xe1,0x6e,0x0f,0x82,0x0e,0xab, +0x78,0xcc,0xfb,0xb5,0xd5,0x2f,0xe5,0x2a,0x82,0x5f,0x9d,0x78, +0x7b,0xd3,0x44,0xcc,0x1a,0xb5,0x07,0x9c,0x00,0x98,0x1e,0x4e, +0xf0,0xac,0xb8,0x00,0xb1,0xf9,0xfe,0x36,0x25,0xa2,0x2b,0x6d, +0x92,0x1c,0x78,0x44,0x29,0x4e,0x0d,0x23,0x47,0x68,0x5b,0xde, +0x58,0x3f,0x5b,0xa9,0x7f,0xb4,0x6f,0xc2,0x42,0x05,0x4b,0xf8, +0xb1,0xa2,0x15,0xb3,0x1a,0x15,0x77,0x84,0x62,0x31,0xc9,0x3f, +0xcf,0x70,0x95,0x54,0x28,0xea,0x69,0x2e,0xf3,0xdf,0x3f,0x01, +0xe2,0x29,0xd4,0x71,0xa5,0x6e,0xbf,0xc4,0xc1,0x24,0x49,0x8c, +0x5e,0xe7,0x77,0x76,0xa3,0x65,0x92,0x06,0x19,0x96,0x5d,0xb6, +0xea,0xed,0xa7,0x55,0xd5,0xac,0x8d,0x71,0x99,0x7c,0xf5,0x78, +0x36,0x94,0xcc,0xbc,0x89,0x38,0xbc,0x0d,0xb8,0x9c,0xdd,0x2e, +0xf0,0x6c,0x9f,0xfe,0x4c,0x57,0x5d,0x3f,0x1c,0x37,0x92,0x38, +0x95,0x2b,0x07,0xbc,0x71,0x58,0xa0,0x55,0xde,0xb3,0x4f,0xff, +0xb6,0x96,0x36,0x3b,0x8d,0xac,0xfe,0xab,0xff,0x58,0x3b,0xdb, +0x38,0xdb,0xec,0xf2,0x0d,0x82,0x36,0x92,0x06,0x55,0xe0,0x23, +0xeb,0xde,0xc6,0xdf,0xa1,0x70,0xf3,0xfd,0xaf,0x1a,0x1b,0xdd, +0xc6,0x7f,0xa8,0x18,0x08,0x9e,0x35,0x25,0xdc,0xbd,0xdf,0x36, +0xbf,0x59,0xae,0x3b,0xd1,0xcd,0x41,0x40,0x35,0x82,0xb3,0x8d, +0xc5,0x19,0xd5,0x0d,0x12,0x1f,0xfc,0x81,0x88,0x26,0xfc,0x05, +0x69,0x84,0x95,0x0a,0x06,0xd6,0x06,0x57,0x46,0x66,0xe1,0xd2, +0xc0,0x36,0x9c,0x6c,0x79,0xd2,0x82,0x05,0x50,0xba,0x04,0xd0, +0x96,0xfd,0x0c,0xd9,0x4e,0x4b,0x43,0x8b,0x74,0x86,0xec,0xa6, +0xa5,0xa1,0xf5,0x34,0x63,0x6d,0x7f,0xa5,0x25,0x23,0x60,0xe6, +0x7a,0xd0,0xef,0xe7,0xac,0xc2,0x23,0x8f,0x70,0x45,0xc7,0x87, +0x26,0x1c,0x7f,0xc3,0x98,0xe7,0x33,0xd1,0xf2,0x21,0xd4,0xfd, +0xb8,0x01,0x9b,0x95,0xfb,0x26,0x62,0xd7,0x6c,0xbc,0x4e,0x73, +0x5c,0x49,0x75,0x86,0xed,0x2a,0xac,0x4c,0xed,0x45,0xf6,0x35, +0x72,0x1a,0x75,0x46,0x57,0xfb,0xe3,0x3e,0x6a,0x10,0x3c,0xb6, +0x68,0x8a,0xad,0xb2,0x0b,0x9a,0x61,0xb1,0xeb,0x51,0xf5,0x42, +0x99,0xa6,0x2f,0x69,0x41,0x77,0x17,0x0b,0x97,0x81,0x5f,0x39, +0x7d,0x85,0x01,0x3e,0x98,0x21,0x5d,0xa3,0xdc,0x8f,0x90,0x45, +0x85,0x3b,0x5c,0xf6,0x22,0x13,0xf8,0x25,0xa1,0x04,0xb1,0xb6, +0x1f,0x99,0xd0,0x2a,0x3d,0xa5,0x59,0x1b,0xf4,0xb0,0x41,0xf5, +0x86,0xec,0x29,0x85,0x8c,0xd5,0x47,0xe6,0x1d,0x37,0xde,0xaf, +0x30,0xd8,0xbf,0xbc,0x06,0x42,0xef,0xe6,0x9f,0x2f,0xf2,0x5b, +0xbe,0x7a,0x22,0xd5,0xb0,0x60,0xa5,0x54,0xab,0x27,0xcb,0xec, +0x6a,0x0c,0xb8,0xad,0xa5,0xf2,0x86,0x4a,0x1b,0x5b,0xd4,0x58, +0x27,0x5a,0xde,0x18,0xc0,0xbf,0xce,0xb5,0xcc,0x7a,0xd5,0x6d, +0xf4,0xbf,0xcc,0xb7,0x0c,0x9f,0xde,0x3b,0xf3,0x6f,0x19,0xdb, +0xf2,0x6c,0xbe,0xeb,0xbd,0x13,0x35,0x93,0x1b,0x1c,0xb6,0x15, +0x69,0x62,0xe3,0x0d,0x0d,0xc4,0x70,0x84,0xb7,0x1a,0xe7,0xe3, +0x10,0xf6,0x2e,0xd9,0x76,0xb7,0x6b,0x9a,0x4a,0xd4,0xbe,0x0b, +0x38,0xc6,0x39,0x37,0x90,0xd2,0xd6,0xef,0x06,0x07,0x95,0xd3, +0x18,0x8e,0xa0,0xf7,0xd9,0xeb,0x6e,0x65,0x63,0x79,0x8f,0x69, +0xed,0xf7,0xde,0xe9,0xd3,0xba,0x35,0xc5,0x8c,0xf1,0xc8,0xa2, +0xe9,0xe3,0x18,0x32,0xf8,0xbc,0x4d,0xe8,0xc6,0xc1,0xd5,0x88, +0x1e,0xbb,0xf8,0x0c,0x25,0x9b,0xe0,0x3c,0x87,0xe9,0x66,0x7a, +0xc4,0x25,0x58,0x25,0x08,0x26,0x3d,0x9a,0x43,0x61,0x75,0x1f, +0x58,0x70,0x40,0x1a,0x1f,0x6d,0x70,0x06,0x54,0x21,0xd0,0xd6, +0xc0,0x5b,0xcb,0xc8,0x04,0x18,0x05,0x86,0xc0,0x2d,0x7d,0x24, +0xf6,0x22,0x49,0xcc,0xf8,0x21,0x93,0x6a,0x87,0x32,0x40,0xb6, +0xee,0x55,0xb6,0xf5,0x3b,0xfa,0x88,0x06,0xc2,0xcf,0xb5,0xc7, +0xdc,0x00,0x89,0xeb,0x1f,0xae,0xce,0xba,0x01,0x6a,0xaf,0x97, +0x94,0x3d,0x9e,0xe0,0x79,0x87,0x61,0x3c,0xe2,0xfe,0xc5,0xcd, +0x09,0xeb,0xa4,0xe4,0x9e,0x6b,0x3d,0x2d,0x67,0xe9,0xae,0x65, +0xd9,0x9a,0x64,0x43,0x66,0x28,0x54,0x1e,0xaf,0x2e,0x50,0xaf, +0x7f,0x95,0x9f,0x17,0x8f,0xf4,0x4a,0x52,0xc3,0x17,0xdf,0x25, +0x3f,0x27,0x49,0x31,0xdb,0x56,0x08,0xa6,0x55,0xf0,0x94,0x92, +0x98,0xd5,0x28,0xbb,0x16,0x16,0x7b,0xf4,0x01,0xbe,0xf4,0x04, +0xb7,0x2a,0xc5,0x6e,0x26,0x04,0x13,0xa7,0x40,0xb6,0xc2,0x3b, +0x49,0xe5,0x34,0xd4,0xf3,0xb4,0x92,0x81,0x3d,0xc9,0x7f,0x89, +0xaa,0xd9,0xd9,0xb0,0x2c,0x59,0x7c,0x2d,0x72,0x24,0xd6,0x36, +0x0d,0xd1,0x40,0x57,0xb8,0x1a,0xcd,0xad,0x12,0x9a,0xad,0x6a, +0x34,0x9f,0xe6,0xf6,0xa5,0xda,0xab,0x8b,0x7a,0x48,0xa7,0xb9, +0x5f,0xa8,0x0c,0xec,0xb4,0x6e,0xed,0x9e,0xbe,0xc4,0x46,0xbd, +0x63,0x91,0xb8,0xc4,0x61,0xb9,0xad,0x9e,0xfb,0xd8,0x05,0x4e, +0x36,0xc0,0x61,0xfd,0x15,0x8c,0xd5,0xe3,0xe2,0xbc,0x68,0x90, +0x56,0x90,0x77,0xfe,0x61,0x35,0x6f,0x70,0xca,0x15,0x16,0x4d, +0x88,0x5c,0x92,0x3b,0x10,0x7a,0xd0,0x83,0xdb,0x0b,0xac,0xa7, +0xbd,0x1e,0x1c,0x70,0x54,0x1a,0xa4,0xf1,0x4c,0x54,0x10,0x26, +0x13,0x10,0x6c,0x07,0xb4,0xdc,0x70,0xbf,0x45,0x95,0xa5,0xa2, +0x68,0x71,0xd4,0x0c,0xfa,0x9f,0x33,0x65,0x80,0x54,0xd4,0x9c, +0x51,0x0d,0xba,0x08,0x93,0x21,0xb6,0xa1,0xc0,0xc6,0x46,0x5b, +0x94,0xa0,0x7d,0xc9,0x53,0x88,0xef,0x5a,0x1b,0xee,0x2b,0xfd, +0xbb,0x14,0x3d,0x7f,0xc8,0x28,0x68,0x1b,0xaa,0x68,0x1e,0x08, +0x76,0x14,0x5d,0x3b,0x34,0x4a,0xba,0x2f,0xed,0x8f,0xad,0xc0, +0x77,0xea,0x16,0x0b,0x13,0xfd,0x4c,0x7e,0xae,0xf1,0x5a,0xc0, +0x6f,0xb8,0x4b,0x12,0xd1,0xd3,0xe4,0xc2,0xa9,0x33,0xd8,0x08, +0x65,0x71,0x90,0x28,0x12,0x77,0x92,0x97,0x76,0x57,0xc8,0x31, +0xad,0x6e,0xc8,0xf4,0xdb,0x97,0x6e,0x03,0xa6,0x7c,0xac,0x39, +0x38,0xa1,0x55,0xda,0xca,0x7d,0xdc,0xe5,0x8a,0x1d,0x9c,0x87, +0x41,0xbb,0x2e,0xb4,0x7c,0x49,0x64,0x59,0x79,0x96,0x2a,0xd2, +0xf0,0xb7,0x08,0xc3,0x95,0x24,0x5f,0x30,0x30,0xeb,0xce,0xe3, +0x9c,0xb1,0xc9,0xa6,0x45,0x87,0xb0,0x4b,0x9c,0x79,0xa0,0xa7, +0x8c,0xe1,0xa3,0x42,0xfb,0x4d,0x60,0xb0,0x99,0x69,0x5e,0xce, +0xbc,0x6c,0x3d,0xc0,0x9d,0xda,0x32,0x55,0x1e,0xb6,0x71,0x1d, +0x94,0xa2,0xb5,0x2e,0xb4,0x79,0x2d,0xda,0xf4,0xb8,0x20,0x55, +0x9a,0xb9,0xb6,0x45,0xd9,0x86,0xb5,0x35,0xbd,0xf1,0x48,0xce, +0x92,0x29,0xf9,0xa2,0xe1,0xdf,0xbb,0x55,0x46,0xae,0xe7,0xa3, +0xe9,0xf5,0xe0,0x72,0xa6,0x33,0x14,0x6c,0x9b,0xfa,0x6a,0x02, +0xae,0x21,0xd9,0x4b,0x0e,0x77,0x78,0xcb,0x0e,0xd4,0x83,0x63, +0xfb,0xa4,0xd1,0xf9,0x5a,0x22,0xf2,0xca,0x62,0x9b,0xd6,0xa7, +0x8d,0x58,0x46,0x14,0x42,0xc1,0x9a,0x51,0xcf,0x49,0x7c,0x6b, +0xb2,0x8b,0xf6,0x7a,0xbd,0xc2,0xbb,0x4d,0xe2,0xce,0x50,0xb0, +0xcc,0x62,0x01,0x93,0xdc,0x50,0xcf,0xa8,0xd1,0x75,0x2d,0x73, +0xce,0xab,0xd4,0xf3,0x48,0x97,0x4a,0x87,0x28,0xc4,0xd6,0x5d, +0x94,0x3c,0xd2,0x76,0x78,0x94,0x8d,0x44,0x09,0xeb,0x42,0x98, +0x3c,0x4c,0xda,0x95,0xd5,0x32,0x21,0x27,0x4a,0x95,0xce,0x01, +0x78,0x23,0xf4,0xb5,0xce,0x22,0x58,0xd1,0xf6,0x4e,0x1b,0x59, +0xf4,0x46,0xc4,0x94,0x7a,0x0b,0x4d,0xff,0xdd,0x83,0x77,0xc7, +0x27,0x3b,0xef,0x4e,0xde,0xc3,0xe6,0xe3,0xa7,0x3d,0x75,0xc4, +0x81,0x3a,0x3c,0x0b,0x52,0xd6,0x59,0xa0,0x51,0xca,0xf5,0x4d, +0x18,0x6b,0x6a,0xef,0x43,0x83,0xfd,0xf0,0x11,0xfd,0xec,0x76, +0x68,0xe0,0xdf,0x8e,0xbe,0x7c,0xa9,0xac,0x9f,0x8d,0xc4,0x4b, +0xd6,0xa6,0xd4,0xab,0xbc,0x0a,0x87,0x13,0x5a,0x9f,0xd3,0x5b, +0xe3,0xe3,0x15,0x7c,0xe5,0x4f,0xd2,0xdf,0xab,0xf8,0x42,0xc5, +0xe5,0x80,0x2d,0xf5,0xc0,0xc5,0x80,0xad,0xf0,0x2e,0x9c,0x15, +0x3f,0x0e,0x46,0x8a,0x67,0x69,0xe7,0xae,0x0f,0x9e,0xbf,0x97, +0x84,0x78,0xdc,0x94,0xe3,0xf3,0x19,0xb4,0xb8,0x98,0x34,0xfb, +0xbc,0x5f,0xb0,0x11,0xd6,0x17,0x10,0x56,0x96,0x98,0x2a,0xec, +0x31,0xa5,0x84,0xbc,0xad,0x04,0x0f,0x20,0xe2,0xd2,0xab,0x9f, +0x28,0x4a,0x5d,0x5b,0x12,0x69,0xc9,0x49,0x7e,0x73,0x8b,0xfa, +0xb7,0xc1,0x79,0x8b,0xd6,0xcb,0x0d,0x2c,0x16,0xbb,0xe8,0xa7, +0x5d,0xc9,0xf0,0x2b,0x4b,0x57,0xe8,0xd5,0x28,0xd3,0x30,0x02, +0x8b,0x52,0x72,0xdd,0xa0,0x1a,0x67,0x31,0x4b,0xbc,0xed,0x94, +0x27,0xf6,0x5d,0x50,0x65,0x89,0x1b,0x86,0xd0,0xa3,0x27,0xf4, +0x21,0x8d,0x7d,0x57,0x3a,0x0c,0xd5,0x6e,0x83,0xce,0x42,0xf5, +0x86,0xe0,0x24,0xe4,0x89,0x9d,0xf3,0xed,0x04,0x81,0x28,0xac, +0x26,0x0a,0x4c,0x5d,0xd9,0x28,0xbc,0x60,0x25,0x1c,0xc9,0x46, +0xe7,0x64,0x0e,0x8b,0x37,0x15,0x50,0x8e,0x04,0x5d,0x6b,0x77, +0x03,0x7b,0xef,0x59,0xc1,0xfc,0x7b,0xc4,0x8b,0x96,0x64,0x3b, +0xe8,0xf7,0x57,0xe7,0x84,0xc4,0xc8,0xf8,0xbc,0x78,0x10,0xbf, +0x3a,0x33,0xa6,0xd2,0x3d,0x3c,0xe6,0x1e,0xf7,0x3e,0x56,0x4f, +0xb5,0xfb,0x47,0xee,0x7f,0xd5,0x2c,0x16,0x38,0x91,0xf2,0x26, +0x51,0xf9,0x10,0x01,0xbb,0xff,0x82,0x72,0x26,0x7c,0xcb,0xea, +0x25,0x42,0x35,0xde,0x65,0xc6,0xc8,0x46,0xfc,0xbc,0x6f,0x56, +0x94,0x29,0xa3,0xff,0x88,0xa4,0x0f,0xa7,0xb7,0x21,0xb1,0x94, +0x8f,0x2f,0x8b,0xa7,0x96,0x9e,0xd1,0xe6,0xcf,0xab,0xae,0x7f, +0x9c,0x37,0xbe,0x8b,0xe9,0x2d,0x79,0xdc,0xa4,0x8a,0xe8,0x76, +0xa3,0xfa,0x26,0x98,0xaf,0x9e,0x6b,0xfd,0xf3,0x21,0x7f,0x90, +0x7d,0x09,0xb4,0x9d,0xc1,0x5f,0xf3,0xdb,0x9a,0xb3,0x38,0x51, +0xb3,0x16,0x29,0xf8,0x03,0x3b,0xa7,0xe6,0x4d,0x53,0xd4,0xbc, +0xed,0x8a,0x9a,0xb8,0x27,0xc4,0x5f,0x00,0x00,0x7f,0xe9,0xe0, +0xa5,0xc6,0x67,0xba,0xb0,0xa9,0xb8,0x98,0x4f,0x6b,0xe7,0xc3, +0xf9,0x24,0xaa,0x34,0x74,0x32,0x72,0x77,0xa5,0xfe,0x82,0x14, +0x0f,0x90,0x56,0x5e,0x97,0x8e,0xe4,0xaa,0xb4,0xd0,0xcc,0xd3, +0xa0,0x78,0xd7,0xab,0x13,0xe8,0xf7,0x6b,0xd5,0x7e,0xfb,0x57, +0xf1,0xab,0xcb,0x91,0x3d,0xac,0xa2,0xab,0xcc,0xb0,0x82,0x44, +0xbb,0x68,0x2c,0x58,0xdb,0x2b,0x5d,0xed,0xaf,0xf2,0x1f,0x71, +0x69,0x2b,0x87,0xad,0x1f,0xe6,0xe4,0x1b,0xb3,0x25,0x71,0x65, +0x7c,0xdb,0x22,0x97,0xa5,0x57,0x6f,0x2e,0x03,0xd2,0xb1,0x0f, +0xf7,0x54,0xd3,0x24,0xca,0x91,0xce,0x6c,0xd2,0x1b,0x4d,0xc9, +0xb7,0x6c,0x1a,0xb9,0xd3,0xda,0x80,0x9e,0x60,0x14,0x6f,0x06, +0xa3,0xde,0x10,0xe9,0x55,0x39,0x61,0x78,0x44,0x9f,0xc2,0x78, +0xc0,0xe5,0x8e,0x74,0x67,0xbf,0xf6,0x8c,0xd7,0xf7,0x25,0xd9, +0x2c,0xb0,0x17,0x92,0x14,0x38,0xc6,0xbc,0x85,0xad,0xac,0x65, +0xfa,0x6f,0x2e,0x4b,0x6c,0x4b,0x03,0x00,0x49,0xdc,0x36,0xb8, +0x49,0x34,0x18,0xeb,0x54,0xb1,0x45,0x21,0x21,0x65,0xc2,0x4a, +0xd5,0x8b,0x5b,0x4c,0x17,0x68,0xda,0xad,0x55,0xac,0xb3,0xe8, +0x82,0xde,0x73,0x29,0x63,0x13,0x14,0xcf,0x6c,0x1b,0x40,0x26, +0x45,0x4d,0x6e,0xd7,0x41,0x2a,0x8f,0x32,0xec,0x30,0xe4,0x17, +0x55,0x1a,0x45,0x8a,0x73,0xcf,0x34,0xe5,0xbd,0x99,0x44,0xe6, +0x85,0xbb,0x44,0x17,0x93,0x2c,0xed,0x95,0x3a,0x4e,0xfa,0x37, +0xec,0x48,0x38,0x20,0x13,0xbc,0xb6,0xc2,0x23,0xfa,0x99,0x2c, +0x70,0xa4,0x9a,0x47,0xe7,0xf8,0xa4,0x57,0x73,0xfc,0xff,0x90, +0xf7,0xa7,0xdb,0x6d,0x1c,0xc9,0xa2,0x28,0xfc,0xff,0x3c,0x85, +0x88,0xed,0xa6,0xab,0x80,0xc4,0x50,0xa0,0x24,0xdb,0x80,0x8a, +0x38,0xb2,0x2c,0xbb,0x7d,0x8e,0x3c,0x6c,0x49,0xee,0xb6,0x9b, +0x82,0xb9,0x30,0x91,0x84,0x05,0x02,0x30,0x0a,0xa0,0x00,0x92, +0xf8,0x56,0xef,0xb5,0xce,0xea,0x27,0xf8,0xde,0x64,0xf7,0xfe, +0xd5,0xef,0x20,0xbf,0xd2,0x8d,0x29,0xc7,0x2a,0x80,0x54,0x0f, +0x77,0xdf,0xbb,0xee,0xb2,0x45,0x54,0x4e,0x91,0x91,0x91,0x91, +0x99,0x91,0x99,0x91,0x11,0x03,0x9c,0xa2,0x30,0xac,0x1f,0x4a, +0x29,0x87,0xb5,0x72,0xab,0x19,0x42,0x53,0x3e,0x2c,0xe5,0x41, +0x32,0xef,0xb0,0x4c,0x4b,0x94,0xbb,0x04,0xba,0x77,0x30,0xcf, +0x80,0xcd,0x5c,0xfe,0x2b,0xe4,0xd6,0xfb,0xf1,0x61,0x66,0x2e, +0xb0,0xb3,0xc5,0x00,0xb6,0x03,0xda,0x4b,0x8c,0x10,0x44,0xf6, +0x04,0x90,0xa6,0x66,0x53,0x34,0xea,0x6b,0xb2,0x70,0x90,0x9e, +0x84,0xa0,0x2d,0x4f,0x6e,0x83,0x49,0xc5,0xa8,0x8c,0xa2,0xda, +0xc5,0xd3,0xc5,0x47,0xbe,0xb1,0x15,0x31,0x18,0x4a,0x49,0x12, +0xb0,0xdc,0x8e,0x5b,0x5d,0x3a,0x73,0x1c,0x2d,0xd0,0xa0,0x1c, +0x32,0xd2,0x6a,0xfe,0xa2,0x97,0x2d,0x05,0x49,0x31,0x02,0xe8, +0x8a,0xda,0x2e,0x40,0xe4,0x36,0x27,0xe8,0xb1,0xab,0x13,0x4f, +0xce,0x63,0xe3,0xa0,0xf6,0x82,0xa1,0xe2,0xa5,0x87,0xf9,0x19, +0xc8,0xff,0xd8,0xfa,0x3e,0x34,0x48,0x4c,0xa0,0xa9,0xf0,0x69, +0xf6,0xd2,0x6a,0x60,0x46,0x4c,0x72,0x57,0xeb,0x69,0x31,0x10, +0xeb,0x43,0x17,0xe3,0x8c,0xf7,0xf8,0x5f,0x0f,0xd3,0x4a,0xc5, +0xa3,0x41,0x1b,0x72,0x75,0x98,0xa1,0x50,0xf6,0xc0,0x42,0xea, +0x86,0x0c,0x8b,0xb7,0x7c,0xfe,0xda,0xc6,0xb5,0x6c,0x35,0x18, +0x8c,0xd0,0xab,0x89,0xae,0x01,0xb8,0x68,0x3e,0x9b,0x66,0xac, +0xad,0xe4,0xd5,0x92,0xa6,0x5e,0x2d,0xa2,0x80,0x33,0x7a,0xc7, +0xc4,0x71,0xc7,0x68,0xdb,0x65,0x9e,0x48,0x67,0xc9,0x8f,0xd1, +0x5d,0x1d,0x95,0x23,0xbc,0x86,0x10,0xd0,0x92,0x87,0xb0,0x1f, +0xc9,0x72,0x97,0x69,0x45,0x6e,0xd4,0xfb,0xbc,0x83,0xbd,0x61, +0x59,0xcd,0x8c,0xb0,0x20,0x5b,0x4d,0x9b,0xdc,0x89,0xe2,0xd8, +0x63,0x15,0xcf,0x2e,0xae,0xc7,0xe5,0xb8,0x59,0x76,0xc3,0x87, +0x87,0x07,0xde,0x6e,0x2a,0xc8,0xeb,0x0d,0xf4,0xa0,0xf9,0x50, +0xe4,0x12,0x97,0xf5,0x8f,0xc6,0x1e,0x91,0xc4,0xca,0x76,0xec, +0xed,0xd2,0xcc,0x18,0xe4,0x93,0x9f,0x1a,0x9d,0xbf,0x47,0xc1, +0x51,0xf7,0xae,0x1e,0x45,0x0f,0x31,0x3b,0x7b,0x64,0x6b,0x2b, +0x2a,0xc2,0x46,0xde,0x21,0x22,0x42,0x71,0x6b,0x2f,0x18,0x3b, +0xa3,0x8e,0x97,0xf7,0x95,0x38,0xf7,0x69,0xe2,0xb8,0xb3,0x56, +0x96,0x3f,0x0b,0xcb,0x6a,0x5c,0x15,0x2b,0x47,0x41,0xbd,0xdf, +0xce,0xa6,0x28,0xbb,0xe3,0x0d,0xd5,0x8e,0xea,0xdd,0x99,0xd3, +0x5d,0xd2,0xa9,0xf8,0xf7,0x20,0x69,0x91,0xb8,0xe4,0x2d,0x05, +0xa2,0xc5,0xa9,0x76,0x6e,0xe8,0x38,0x83,0xf2,0x37,0x76,0x64, +0x65,0xfb,0xe5,0xd3,0x67,0xcf,0xd3,0xfa,0xcd,0xb6,0xae,0x2d, +0x93,0x78,0x5a,0x19,0xf7,0x71,0xd9,0xc4,0x0a,0x5f,0x78,0x51, +0x69,0xa6,0xd7,0x01,0xf6,0xa8,0x7a,0x77,0x31,0x9a,0x9a,0x28, +0x5e,0xfb,0x31,0x2a,0xf0,0xa7,0x16,0xa4,0xc6,0x68,0xfb,0x0b, +0xc4,0x79,0x99,0xc5,0xe9,0xfb,0xf6,0xb6,0x41,0xc0,0x32,0xcf, +0xcb,0xb0,0x80,0x27,0xe3,0x23,0x9c,0x0c,0x21,0x34,0x59,0x02, +0x41,0xba,0xde,0x7b,0xb5,0xb9,0xec,0xcf,0x26,0xde,0x4e,0xb4, +0xe6,0x24,0x00,0x9f,0x8f,0xa6,0xc3,0xa2,0x4c,0x26,0x3a,0xc2, +0x87,0x47,0x7f,0x04,0xd0,0x69,0xfd,0x67,0xac,0x21,0x42,0x9d, +0xe3,0x2c,0xee,0x44,0x35,0x3c,0x4c,0x31,0x72,0x37,0xad,0x58, +0x76,0x1d,0x33,0x57,0x93,0xca,0x58,0x50,0x16,0x05,0x4c,0x82, +0xc5,0x07,0x23,0x7e,0x12,0xcc,0xe3,0xd4,0x82,0x40,0xe3,0xc0, +0x64,0x30,0xee,0x34,0x4a,0x98,0xad,0xe4,0x79,0xd8,0x20,0x45, +0x68,0xbc,0xb7,0xaf,0x02,0xf3,0x77,0xd3,0x1d,0xc4,0x3d,0xf1, +0xe0,0xb1,0x2b,0x23,0x8d,0x3f,0x55,0x5d,0xd8,0x00,0xb2,0xab, +0xe1,0xd0,0x96,0xec,0x65,0x04,0x7b,0x7b,0x93,0xdb,0x60,0x44, +0x4c,0xe5,0xf6,0x41,0xc5,0xf0,0x47,0x05,0xaf,0xed,0xb9,0x57, +0x2b,0x86,0xcc,0xb1,0x33,0xbc,0xf7,0x5c,0x7a,0x38,0x9e,0x41, +0x5d,0x3e,0x30,0xb0,0xad,0x19,0x14,0xf6,0xd5,0x22,0xa7,0x67, +0x68,0x5f,0x98,0x3e,0xd1,0x0a,0x29,0xb5,0xe5,0xf6,0x56,0x8e, +0x4c,0x64,0x5c,0xd4,0xe6,0x34,0xaa,0x9e,0x69,0x47,0xb0,0x55, +0x46,0x30,0x8e,0x5d,0xb6,0x3a,0xa1,0xa4,0x6e,0x30,0x04,0x0e, +0xd0,0x0d,0x8f,0xb3,0xdd,0x11,0x17,0x65,0xde,0xd1,0x82,0x76, +0x18,0x16,0xcb,0xd4,0xf3,0x72,0x34,0x1f,0xf5,0x7c,0xbd,0x17, +0xd9,0xcb,0x14,0xca,0xfe,0x94,0xa6,0xec,0x16,0x80,0x6c,0x21, +0x7f,0x75,0xfa,0xf2,0xf9,0x37,0xdf,0xfd,0xe1,0xf9,0x17,0x69, +0xe9,0xa3,0x8f,0x6c,0xa8,0x64,0xc0,0xe7,0xdc,0xea,0x70,0x74, +0x49,0xd3,0xe8,0x03,0xb7,0x10,0xfb,0xa5,0x35,0x9c,0x24,0xec, +0xc6,0xf6,0x5e,0x3b,0x06,0xa8,0x7e,0xf0,0xf6,0xf3,0x0d,0xd0, +0x56,0xd9,0x4f,0x51,0x9a,0x97,0x88,0xaf,0x87,0x5f,0x4e,0xed, +0x37,0x9d,0x88,0xba,0x11,0xdf,0xf5,0x7f,0x81,0xe0,0xe4,0x22, +0x53,0x0b,0xf8,0x47,0xbd,0xf3,0x35,0xbd,0x9e,0x3c,0x1b,0x03, +0x0c,0x60,0x54,0x27,0x64,0x99,0x54,0xab,0x2e,0x08,0x3d,0xd8, +0x67,0x7a,0xea,0x30,0xb1,0xf1,0xc1,0x9e,0x95,0x71,0x98,0xbf, +0xc9,0x2a,0xe3,0x29,0xfc,0x89,0x6a,0xe5,0x4e,0x4c,0xea,0x58, +0x59,0x85,0x50,0x80,0xdf,0xfe,0x86,0x12,0x30,0x53,0x19,0xf5, +0xbf,0x62,0x7a,0xe1,0xf9,0xe5,0xf4,0x05,0x3d,0xb8,0x49,0x6f, +0x3e,0x1a,0x0f,0x5b,0x62,0x81,0x86,0xdc,0xbe,0x1f,0x10,0x6c, +0x51,0x70,0xf0,0x3b,0x2a,0x2a,0x8d,0x03,0x5f,0xe7,0x16,0x25, +0xe4,0x5b,0x3c,0x38,0xc6,0x87,0xb2,0x1f,0x9f,0x8e,0x41,0x86, +0x3a,0xc5,0xa8,0xd3,0x01,0x1e,0x41,0x10,0xa1,0x4f,0x4f,0x98, +0x9e,0x0f,0xfa,0x9b,0x07,0xa7,0xe3,0xe1,0x69,0xf7,0x63,0xf2, +0x8b,0xee,0x1a,0x43,0xb1,0xe0,0xe8,0xaa,0x10,0xc8,0x96,0x1a, +0xbf,0xf1,0x0b,0x13,0x68,0x76,0x9d,0xde,0x90,0xb8,0x87,0x6e, +0x5c,0x27,0x0a,0x7b,0x4b,0x6f,0xa2,0x6d,0x7c,0xec,0x76,0x60, +0xea,0xda,0xe8,0xe1,0x5e,0x52,0xbe,0xcb,0x0b,0xaf,0xa7,0x60, +0x22,0x74,0x69,0x78,0xe2,0x25,0x92,0xad,0x16,0x9f,0xc8,0x27, +0x41,0xbf,0x6b,0xc3,0xe1,0x6e,0x1e,0x71,0x0d,0x98,0xb2,0x61, +0xc8,0x10,0x7f,0xcd,0xa4,0x6e,0x09,0xf4,0xac,0x15,0x85,0x9c, +0x57,0xd0,0x12,0xd3,0x06,0x6d,0x67,0x48,0x4e,0x11,0x03,0x26, +0xf5,0x4a,0xba,0xed,0xde,0x8a,0x5b,0xff,0x14,0xba,0xc3,0x30, +0x5e,0xd4,0x69,0x45,0x27,0x6f,0x3e,0x7a,0xf3,0x0e,0x1f,0x81, +0xbc,0x89,0xcc,0x37,0x30,0x99,0x22,0x03,0x44,0x3a,0x1c,0xc7, +0xc8,0x72,0xfb,0x99,0x6a,0x3c,0x64,0xb6,0x12,0xbe,0xf9,0x18, +0x19,0x67,0x07,0x0f,0x7d,0xfc,0x20,0xbb,0x98,0xad,0x26,0x43, +0x7a,0x69,0x0d,0x4c,0x67,0x68,0x8a,0xcf,0xad,0x3f,0x8e,0x4e, +0x01,0xdf,0x53,0xf5,0xe0,0x94,0x9a,0x78,0x1a,0x7f,0xec,0x30, +0xa8,0xca,0x71,0x1b,0xb4,0x47,0xec,0x28,0xdb,0xae,0x49,0xed, +0xfb,0x7d,0xc3,0x79,0x5e,0xf7,0x1a,0x1e,0xe4,0xc7,0x28,0x20, +0xbe,0x7d,0x0e,0xf3,0xf3,0xdb,0x6f,0xd0,0x98,0xea,0xd6,0x3f, +0x4a,0x78,0x66,0xd0,0x8e,0x70,0xf0,0x5b,0xd1,0xde,0xc4,0xbb, +0xb6,0x40,0xc5,0x12,0x34,0x1a,0xb7,0x61,0x5b,0x38,0xd8,0xa1, +0xec,0x2f,0xdd,0x39,0x11,0x50,0x96,0x41,0x4d,0xf7,0x29,0x0b, +0x10,0xba,0x37,0x53,0x7d,0x44,0x88,0x14,0x87,0xc7,0xb3,0x55, +0x46,0xb6,0x78,0xf4,0xe4,0x86,0x17,0xc2,0x58,0x83,0x83,0xb3, +0x0d,0x7f,0xb7,0x20,0xb3,0x4b,0xdd,0xb6,0x75,0xc1,0xf8,0x62, +0xfc,0x76,0xe4,0xe2,0x1b,0xfb,0x55,0xa5,0x36,0xe8,0x0d,0x27, +0xe7,0xfb,0xf6,0x36,0x64,0x51,0x7e,0x99,0x7f,0x77,0x76,0x9e, +0x3a,0x83,0x0a,0x01,0x3b,0xc7,0x5c,0xb7,0x83,0x99,0xfd,0x0c, +0xfc,0x04,0x12,0x3b,0x1f,0x1e,0x96,0x3e,0x2a,0x1d,0xe0,0xc8, +0xb4,0x86,0x0c,0x40,0x94,0xf7,0x60,0x8b,0xcf,0x47,0xc8,0xdd, +0x0e,0x12,0xd0,0x2d,0x0d,0x88,0xe5,0x58,0xb1,0xd3,0x2b,0x69, +0x90,0x4b,0x3a,0x50,0x3c,0xfd,0xfb,0x54,0x95,0xc4,0x1d,0x45, +0x78,0xe4,0x37,0xda,0xe2,0xc7,0x9f,0xdd,0x2d,0xd2,0x5f,0x36, +0x63,0x8e,0xd6,0xaa,0x6c,0x49,0xdc,0x92,0x78,0x60,0x3a,0x94, +0xb5,0xe5,0x47,0x8a,0x3f,0x1e,0xe6,0x15,0xa7,0x00,0x1b,0x13, +0x33,0x44,0x76,0x69,0x9f,0x9b,0xfc,0x94,0xcb,0xde,0x21,0x59, +0x4d,0xc1,0x38,0x26,0x86,0x4b,0xdd,0xcc,0x27,0x26,0xb5,0xab, +0xc4,0x58,0xc1,0xae,0x64,0x97,0x1d,0x9d,0xf8,0x94,0xb9,0xd8, +0x27,0xa3,0x34,0x8a,0xd3,0x8c,0x85,0x07,0x17,0xc2,0x1e,0x2c, +0x79,0xf2,0xb1,0xe6,0x33,0x5d,0xc0,0x76,0x70,0x12,0xe8,0xf8, +0x86,0x7e,0x0e,0x0f,0xe9,0x87,0xe5,0x71,0xb6,0x42,0x15,0x79, +0xcd,0xe0,0xe4,0xb1,0xc6,0x96,0xef,0xb1,0xfc,0xc9,0x6d,0xb8, +0x9a,0xe3,0xf3,0xbf,0xd2,0x17,0x2b,0x74,0x66,0x04,0xb2,0x51, +0x46,0x76,0xe6,0x1f,0x2c,0x28,0x17,0xcc,0x5b,0xbd,0xc5,0x88, +0x8c,0xd6,0xf4,0x26,0x28,0x56,0x0f,0x6b,0x0f,0x7e,0xc8,0x46, +0x0f,0x3e,0xd6,0x2b,0xa5,0x3b,0x83,0x3d,0x58,0xce,0x1e,0xb0, +0x27,0x93,0xcd,0x83,0xd5,0x74,0x0c,0x9b,0x49,0x9c,0x9e,0xb3, +0xda,0x83,0x97,0x02,0xab,0xf5,0x00,0x66,0x36,0xf5,0xc0,0x54, +0x85,0xc9,0xa2,0x58,0xe8,0xcc,0x83,0x96,0x24,0xed,0x62,0xea, +0xde,0x80,0x44,0x60,0x32,0x6d,0x77,0x76,0xd0,0x41,0xb2,0x75, +0x86,0xa2,0x4b,0x97,0x78,0x1f,0xdf,0xf0,0x70,0x8c,0x0a,0x78, +0x86,0x38,0x33,0x38,0x29,0x62,0x0a,0xcb,0xec,0x95,0xd9,0xdd, +0x80,0x1f,0x5f,0x60,0x3a,0x58,0x4f,0x78,0x56,0xec,0x14,0x07, +0x49,0xd2,0xa5,0xc1,0x29,0x56,0x4c,0xd3,0x8a,0x0c,0x44,0xb5, +0x6f,0xb4,0x16,0x8f,0xd2,0x9b,0x7f,0xc9,0x30,0x65,0x22,0x15, +0xf6,0x92,0x0a,0x58,0x53,0x8e,0x73,0xf9,0x30,0xc8,0x69,0xa3, +0x59,0x50,0x52,0x77,0x41,0x68,0x0f,0x67,0x0f,0x4c,0x82,0xfe, +0xf0,0x2c,0x80,0xbe,0xbb,0xc0,0x33,0x1e,0x9d,0x84,0xca,0x0e, +0xfc,0xe5,0x12,0x34,0x6e,0x07,0x48,0xa4,0x06,0x16,0x1d,0xd7, +0x70,0x47,0x92,0x29,0xb4,0xa0,0xbf,0xe8,0x4c,0x49,0xec,0x49, +0xba,0x78,0xc1,0x6e,0xc6,0x5b,0xb7,0xa4,0xd8,0x94,0x4c,0x4c, +0xb2,0x61,0xef,0x5d,0x87,0xde,0x6d,0x9b,0xb0,0x4b,0xde,0x0a, +0x85,0x38,0xa7,0x44,0x91,0x08,0xe7,0x9e,0xce,0xbb,0xc2,0x99, +0x1b,0x4f,0x56,0x62,0x52,0xf4,0x1f,0x92,0x4f,0x43,0xe6,0xe6, +0x68,0xb4,0xcd,0x6e,0x17,0x8e,0x6a,0xe2,0x65,0xbb,0x1c,0x0f, +0xd1,0x64,0xff,0x41,0x94,0x83,0x7b,0x7b,0x1b,0x82,0xf3,0x91, +0x9a,0x0d,0x87,0x41,0x31,0xbc,0xbb,0x41,0x6c,0xc4,0xcd,0x74, +0x1c,0x72,0xc9,0xed,0x2d,0xef,0x82,0xf6,0x5d,0x0b,0x14,0xde, +0x01,0xdc,0xb7,0xc3,0x38,0xb7,0xc3,0x7f,0xa9,0x53,0x53,0xc8, +0x2d,0x94,0x19,0x45,0x12,0x9f,0x3b,0x3c,0x20,0xd2,0xf7,0x92, +0x97,0xfe,0x3a,0xe6,0x06,0xbd,0xb9,0x29,0x98,0x8d,0x61,0x2b, +0xeb,0x89,0x67,0x6e,0x5e,0x7b,0xc4,0xf6,0xea,0x62,0xf6,0xee, +0x03,0x2f,0xb6,0x3e,0xf0,0xe1,0x67,0x8d,0x2b,0xc9,0xdd,0x26, +0x69,0xb0,0x27,0xe1,0xcd,0x4e,0xa7,0xe4,0x5c,0x04,0x97,0x5a, +0x25,0x7d,0x25,0x5f,0xea,0x5a,0xa3,0xb6,0xd3,0xf3,0xea,0x05, +0x9a,0x14,0x30,0xd7,0xe3,0xbf,0x1f,0x07,0x37,0x2f,0xff,0x8a, +0x76,0x60,0x25,0x1f,0xd2,0x0e,0x83,0x79,0xcb,0x6b,0xd2,0xbe, +0x76,0x90,0xbf,0xb3,0xbf,0x43,0xa1,0xa5,0x90,0xec,0x08,0xcb, +0x3b,0xf7,0xa0,0x98,0x0c,0xdd,0x5c,0xf3,0x57,0x7c,0x63,0x3e, +0x49,0x9f,0x8b,0x3f,0xd9,0xc9,0xb5,0x8e,0xd6,0x4b,0x8e,0x89, +0xf2,0x28,0xa0,0xc8,0x5b,0x9a,0x3d,0x4c,0x19,0xa0,0x63,0x6b, +0xaa,0x97,0x74,0x17,0x94,0x01,0xea,0x78,0x76,0x63,0xa7,0xed, +0x82,0xc1,0x16,0x8f,0x67,0x48,0x70,0x78,0x45,0x56,0xb2,0x3e, +0xf4,0x96,0xd5,0x3f,0x09,0x75,0x94,0xe0,0x18,0x9c,0xf7,0x9a, +0xd3,0x3e,0xae,0x09,0x35,0xbb,0xf1,0x34,0x0f,0x0f,0x25,0xa1, +0x0f,0xee,0xd4,0x9a,0xd3,0xa0,0x9f,0x39,0xcf,0x39,0xe9,0xf2, +0x6b,0xc4,0x8e,0x97,0x5e,0x9b,0x43,0x9b,0xcc,0x38,0x66,0xd4, +0x26,0xa1,0x15,0x75,0x10,0x3e,0x8c,0xd0,0x97,0x64,0x0c,0x4b, +0xee,0xc8,0x40,0x3c,0xd1,0x25,0x68,0xca,0x20,0x81,0xdf,0xeb, +0x5a,0x53,0x3e,0xc7,0x87,0xae,0xc3,0xa4,0xf1,0x38,0xf5,0xe1, +0x18,0xa3,0xc8,0xe3,0xe3,0x31,0x59,0x46,0xf6,0x93,0x4f,0xc6, +0x5d,0xf7,0x4a,0x2b,0x10,0x42,0xc2,0x46,0x90,0x07,0xa5,0x30, +0x12,0x5f,0xaf,0xe4,0x70,0x57,0x51,0x01,0x51,0xd2,0x3c,0x01, +0x99,0xfe,0x27,0xa5,0x03,0x71,0xc6,0x89,0x4a,0xba,0x3b,0x33, +0x75,0x4a,0x68,0x74,0x2e,0xe7,0x3e,0xbd,0xc6,0x17,0x15,0x56, +0x46,0x2a,0xea,0x0f,0xd7,0x18,0x4b,0x90,0xe8,0x77,0x62,0xc1, +0x8d,0x55,0x40,0x51,0xda,0x2e,0x79,0x71,0xb1,0xca,0x43,0xad, +0x39,0x77,0xd1,0x5e,0x66,0x67,0xdd,0x81,0x66,0x3d,0x77,0x6f, +0x32,0xf9,0xa2,0x27,0x2d,0x00,0x26,0x7c,0x98,0x23,0x3e,0x23, +0xe3,0x02,0xca,0x5d,0x6a,0xd9,0x34,0xc5,0xf0,0xed,0x75,0x36, +0x87,0x49,0xe3,0xc1,0x1e,0x80,0x32,0xf5,0xf1,0x2c,0x7c,0xd7, +0x15,0xc8,0xde,0xb3,0xc9,0x47,0x8d,0x86,0x1d,0x8c,0x3f,0xbb, +0xa3,0x71,0xcf,0x19,0x65,0xb6,0x57,0xb5,0x61,0xbf,0x16,0xb4, +0xe5,0x20,0x7d,0xa5,0x63,0x5b,0xd0,0x4d,0xef,0xce,0x82,0xee, +0x2f,0xd4,0xdd,0xd9,0x98,0xd2,0x6e,0xe3,0xed,0xa7,0xc6,0x4d, +0x3b,0xf6,0x33,0x16,0x22,0x64,0x66,0xe3,0x37,0xcc,0xff,0xaf, +0x22,0x27,0x8c,0xb5,0x34,0x08,0xe7,0x08,0x05,0x71,0x7f,0x17, +0x55,0x2c,0x5f,0xef,0xbc,0xe3,0x73,0x67,0x6e,0x0b,0xd1,0x7b, +0xe0,0xef,0xb4,0x90,0x1f,0x80,0xd8,0x98,0xd4,0xf9,0xce,0x4d, +0xec,0x85,0xc7,0xde,0x99,0xf7,0x46,0x7f,0xe0,0x98,0x2d,0x70, +0x86,0x71,0x4e,0x60,0xf4,0xd4,0xf9,0xd0,0x66,0x89,0x7d,0x23, +0xc6,0x46,0x4b,0x38,0x23,0x2b,0xda,0x67,0x83,0xc5,0x78,0xee, +0x5f,0x2c,0x04,0xca,0x21,0x4e,0xe3,0xdc,0x84,0xe2,0x87,0xb4, +0xf7,0x3e,0xff,0x67,0xd3,0x22,0x78,0xd7,0x51,0x07,0xa1,0x43, +0x43,0x2e,0xa5,0xa9,0x7d,0x35,0x2b,0x1e,0x25,0x39,0xe5,0x87, +0xc5,0x84,0x93,0xf0,0x85,0x14,0x94,0x4a,0xed,0x11,0x1a,0xdd, +0x98,0xb4,0x7d,0xec,0x80,0x03,0x96,0x91,0x53,0x56,0xb1,0x6f, +0x41,0x99,0x4c,0xbe,0x63,0x9f,0x85,0x05,0x4f,0x1f,0x1c,0xfc, +0xb7,0x7a,0xfe,0xf4,0x88,0xe3,0x28,0xc6,0xe4,0x94,0xc9,0x8c, +0xb2,0x8b,0x68,0x95,0xc9,0xad,0xcb,0x77,0xdf,0xbf,0xfe,0xfa, +0xbb,0x6f,0x5f,0xf9,0x2f,0xb4,0xf1,0x6a,0x20,0xea,0xb4,0xde, +0x64,0x95,0x5e,0xa6,0xaf,0x0a,0xe2,0x0e,0xc7,0x9c,0x2f,0x66, +0xab,0xb9,0xbd,0x2b,0x40,0x53,0xb3,0xf0,0x01,0xeb,0x08,0x06, +0xcd,0x49,0x2f,0xfd,0x7d,0x33,0xec,0x96,0xc9,0xc6,0x8f,0x18, +0x9a,0x0f,0x52,0xfc,0xf3,0x5f,0x2f,0xfe,0x0d,0xbd,0x06,0x77, +0x2e,0x2a,0xb8,0xee,0xe0,0x9e,0x82,0xb0,0x42,0x1d,0x70,0xd8, +0x70,0x7c,0x63,0x5e,0xb3,0xdc,0x78,0x8f,0xc3,0xc4,0xb2,0xc6, +0xbe,0x77,0xd5,0x27,0x25,0x26,0x25,0x90,0xcd,0x3c,0xd6,0xe8, +0x06,0xa2,0x90,0xf3,0x82,0xd8,0x3c,0x39,0x96,0xe7,0xc5,0x96, +0xc4,0x66,0x78,0xc8,0x70,0x91,0x2b,0x74,0x6d,0x40,0x84,0x3b, +0x56,0xad,0xa6,0x6f,0xa7,0xb3,0x77,0x53,0x09,0xa1,0x41,0x33, +0x76,0x7a,0x25,0xbe,0x2a,0xf5,0x49,0xeb,0xb9,0x6d,0x91,0xd7, +0x3e,0x5c,0xcf,0xce,0x48,0xc7,0xb6,0x3f,0x5b,0x4d,0x87,0x7c, +0xfd,0x93,0xd9,0x77,0xa0,0x98,0x3a,0x9e,0x8e,0x1d,0xed,0x1c, +0x07,0xd4,0xa9,0xb2,0x88,0x9c,0xfa,0x98,0x9c,0xc6,0x37,0x5e, +0x9d,0xc5,0x85,0xd2,0x9d,0xe5,0x53,0x1f,0xda,0x96,0x31,0x01, +0xa9,0x5e,0x92,0x43,0x11,0xcc,0x36,0x57,0x2e,0x22,0xf1,0x0d, +0x9e,0x53,0xab,0xfb,0x9c,0xca,0xbc,0xb3,0x32,0x93,0x05,0x0a, +0x32,0xf2,0xa4,0xcd,0xab,0xd7,0x2c,0xd4,0x87,0x87,0x7e,0xbc, +0x36,0xb3,0x1e,0x0b,0x62,0x1c,0xde,0x81,0x9b,0x36,0xc5,0xcb, +0xc9,0x56,0x89,0x4e,0x8e,0x27,0x73,0xa8,0xdf,0x85,0xb7,0xf8, +0x1a,0xc0,0x77,0x60,0x3f,0xb8,0x58,0x99,0xe7,0x16,0x1a,0xa7, +0x5c,0x0e,0x0f,0x35,0x66,0x25,0x69,0xd7,0x1f,0xd0,0x43,0x73, +0xe7,0x41,0xa9,0xe2,0x5c,0xd5,0xc4,0x95,0xd2,0x83,0x4e,0xa9, +0xed,0xb7,0x1c,0x49,0x65,0xcb,0x38,0x13,0xae,0xb8,0x8a,0x8a, +0xbc,0xec,0x4e,0x7a,0x58,0x30,0x20,0x34,0x1a,0xcf,0x29,0x69, +0xe1,0x8a,0x54,0xaa,0xa5,0x19,0x86,0x70,0xbb,0x9e,0x29,0x59, +0xfa,0x85,0x07,0x7e,0xfa,0xba,0x49,0x9f,0xd6,0xec,0x7c,0x48, +0xb6,0x93,0x5e,0x34,0xdc,0xe3,0xfb,0x6c,0x4d,0xe4,0x29,0x9c, +0x79,0xd0,0xff,0x6a,0x3c,0x3d,0x9f,0x8c,0x24,0x27,0xb7,0xcb, +0xa8,0x35,0xd9,0xee,0x95,0x14,0x7a,0xab,0x76,0xe3,0x75,0xbb, +0x3c,0xb2,0x0e,0x6f,0xfa,0x0d,0x33,0x14,0xf3,0x48,0xdb,0xc2, +0x73,0x39,0xce,0x3c,0xfc,0xe9,0x44,0x1f,0xc6,0xdf,0x3e,0xe6, +0x3c,0x4e,0x0c,0x30,0xf1,0x0a,0x6e,0xdf,0x0d,0x8d,0xf0,0x79, +0xd8,0xee,0x0e,0x8d,0x5b,0x9e,0x17,0x5a,0x03,0xc7,0x2b,0xd8, +0xc9,0xd7,0x08,0xeb,0x7a,0xcb,0x69,0x57,0x21,0xe7,0xdb,0x47, +0x5b,0x01,0xca,0xd8,0xe7,0xbc,0x3d,0xf1,0x7b,0x7c,0x87,0x4e, +0xf2,0x07,0x92,0xc7,0xeb,0x05,0x77,0xa1,0x88,0xf2,0xcd,0x88, +0x59,0xb1,0xd9,0x70,0xc8,0x37,0x62,0x6b,0xac,0x90,0x47,0xec, +0xfb,0x30,0x3c,0x49,0x42,0xb0,0xed,0xc1,0x1e,0xbe,0xc0,0x1b, +0xcb,0x0c,0x35,0xf1,0x1e,0x88,0xb3,0x99,0xdc,0x63,0xb6,0xb6, +0xbf,0x37,0xd3,0x88,0xa1,0x8b,0xa5,0xa8,0xc4,0x43,0xc8,0x55, +0x4c,0xdf,0xe1,0xd9,0x38,0xb5,0x0a,0x75,0x54,0x25,0x69,0x30, +0xba,0x0e,0x8e,0xa9,0x91,0x3b,0x95,0x55,0x60,0x89,0xec,0x4d, +0xb2,0x48,0x37,0x49,0x85,0x48,0xde,0xde,0x9a,0xb4,0x94,0x4c, +0xe2,0x84,0x19,0x72,0x8f,0x60,0x43,0x0e,0xfd,0xc0,0xee,0xa6, +0x3d,0x1f,0x9e,0x8f,0xca,0xf5,0xde,0x3f,0x4e,0xa1,0xc3,0x43, +0x82,0x97,0x77,0xfd,0x4c,0x6e,0x02,0xf3,0x7c,0x42,0xb9,0x03, +0xce,0x10,0x99,0x6d,0x0f,0x63,0x98,0xac,0x9a,0x12,0xd4,0x12, +0xae,0xee,0x2b,0x14,0xa9,0xc8,0x06,0x99,0x13,0x96,0x6f,0x35, +0x5a,0xd3,0x83,0xf4,0xf3,0xef,0xd9,0x8f,0x87,0x0e,0x4a,0x8d, +0x41,0x58,0x91,0x74,0xf6,0xc2,0xbd,0x5d,0xe4,0xa8,0xaf,0xe9, +0xc4,0x99,0xcf,0x9d,0x75,0xc3,0xe9,0xca,0x4e,0x63,0xa9,0xa7, +0xc6,0x49,0xaf,0x0f,0x52,0x84,0x83,0x47,0x96,0xde,0x94,0x4a, +0x2d,0x74,0x77,0x18,0x20,0x9b,0x81,0xc4,0x09,0x72,0x92,0xf3, +0x92,0x3a,0x7c,0x5a,0xcd,0x37,0x15,0x19,0xaf,0x82,0xd9,0x97, +0x32,0x0b,0xc7,0xb4,0x29,0xc2,0xdb,0x28,0x3c,0x3a,0x47,0x48, +0x9d,0x8c,0xbc,0xf6,0xe2,0x1d,0x07,0xaf,0x00,0x59,0xcc,0xcf, +0xbd,0x32,0x31,0xf5,0x4a,0x1a,0x70,0xfa,0x40,0x60,0xb4,0x4c, +0x0f,0x12,0xbc,0x77,0x36,0xbe,0xc0,0x51,0xb3,0x16,0x25,0xc3, +0x2f,0xa7,0x87,0x87,0xfa,0xa5,0xa1,0xc5,0x22,0xa6,0xd5,0xc2, +0x94,0x75,0x87,0xdc,0x49,0x37,0x36,0xfe,0xd0,0x08,0xc2,0xd7, +0x72,0xbb,0x6a,0x03,0x4f,0x2c,0x20,0x7d,0x2e,0x64,0x13,0x2b, +0x95,0x78,0xe2,0x68,0x6d,0xf0,0x4b,0x0e,0x5b,0xe0,0xc4,0xe6, +0xec,0xba,0xf8,0xb3,0xfc,0xcf,0x18,0x0b,0xcb,0x68,0xab,0xdc, +0xc5,0x85,0xc5,0x4f,0xe9,0xae,0x66,0x38,0x6d,0x75,0xaf,0xa6, +0xe4,0xf6,0x29,0x7d,0x9b,0xbb,0x71,0x2e,0xb8,0x90,0xca,0xb7, +0x83,0x7b,0xe0,0x44,0xf7,0xd1,0xc4,0xe8,0xaf,0x70,0xf2,0x5b, +0x7b,0x23,0xd5,0x92,0x6b,0xa6,0x80,0x41,0x52,0xe2,0xbc,0xcf, +0x37,0x61,0x23,0x49,0x87,0x3d,0x72,0x32,0xa7,0x2e,0xbb,0x9d, +0x04,0x50,0xf0,0x9d,0xec,0xfd,0x33,0xe3,0x39,0x5a,0xc8,0xa7, +0xee,0xc8,0x36,0xb1,0x31,0xd0,0x5a,0xf3,0x8f,0x99,0x2b,0xdd, +0x3e,0x92,0x15,0x43,0xba,0xa9,0x53,0xd8,0x5d,0x2d,0xbd,0x35, +0xf3,0x62,0xe3,0x03,0xfb,0x16,0xc3,0x5c,0x3c,0x4b,0x79,0xf7, +0xe9,0x3d,0x5e,0xf9,0xdc,0x6c,0xdb,0x26,0x50,0xcc,0x44,0x86, +0x71,0x52,0x1f,0x05,0x53,0x0c,0x8d,0x8a,0x17,0x62,0xe7,0x33, +0x8d,0x03,0x32,0x4d,0xd3,0x42,0xc4,0xdb,0x2e,0x83,0x39,0xdf, +0xb7,0xb7,0x3a,0xb0,0xa5,0xe9,0x21,0x1d,0x8e,0x33,0xd8,0xbb, +0xe6,0x3a,0x96,0x27,0x8f,0x94,0xff,0x06,0xcf,0x37,0x73,0xf3, +0x8a,0x1c,0x83,0xe8,0xfb,0xe3,0x9d,0x24,0xd6,0xfc,0x97,0xe3, +0x37,0xae,0x4d,0x00,0x6b,0x04,0x5b,0x06,0x53,0xd6,0xc1,0xd0, +0x7d,0x0c,0x3c,0x64,0x77,0x2e,0xda,0x4b,0x82,0x43,0x92,0x8e, +0xc7,0x55,0x30,0xaf,0x99,0xc7,0xe9,0x88,0x21,0xf0,0x2b,0x57, +0x06,0x1f,0xa6,0xa6,0x03,0x87,0x42,0xdb,0xb8,0xe5,0xd1,0x6b, +0x3f,0xb4,0x4e,0x21,0x38,0xd8,0xd3,0xc7,0xce,0x64,0x9d,0x36, +0xdc,0xc9,0x3c,0xcd,0x31,0xb5,0x4c,0x46,0x4e,0x9e,0x63,0x5b, +0xb8,0x6d,0x3f,0x71,0x7c,0x23,0x25,0xc2,0xc1,0x19,0x02,0x3c, +0xb1,0x45,0xbc,0x21,0xb4,0x7f,0xbc,0x79,0x2b,0x05,0x1f,0x6e, +0x30,0x62,0x4f,0x52,0x0b,0xaf,0x13,0xf9,0x8b,0x58,0xaa,0x6f, +0x40,0x5a,0x50,0x9a,0x8a,0xea,0x37,0xad,0x35,0x3a,0xfd,0x89, +0x62,0x31,0x2a,0x4b,0x54,0x2a,0x85,0x03,0x5a,0xf8,0xac,0xe5, +0x32,0x13,0xc5,0x6c,0xc3,0xd5,0x31,0x3d,0xf1,0x2b,0x2e,0x42, +0x97,0x1d,0x03,0xfb,0xe5,0x42,0x19,0x45,0x8e,0xa6,0x7c,0x60, +0xfa,0x80,0x19,0x6d,0xfb,0x84,0xf5,0xe6,0xaa,0xf1,0xa8,0x1b, +0x50,0x23,0x28,0x8c,0x97,0x9c,0x41,0x4d,0xd4,0xba,0x83,0xb0, +0xcb,0x40,0x0a,0x2f,0x44,0xc8,0x27,0x5e,0x11,0xa8,0x10,0x52, +0x1c,0xbb,0xb2,0x00,0xed,0xf1,0x55,0xa0,0xdf,0xe0,0x11,0x7b, +0x8f,0x72,0x03,0xe7,0x73,0xb3,0x6b,0x25,0x84,0x80,0x48,0xb9, +0x66,0x33,0x84,0xa4,0x0b,0x5b,0x1e,0x17,0x19,0x3f,0x9b,0x6e, +0x63,0xd0,0xd3,0x9a,0x42,0x52,0x2f,0x07,0x0f,0x0f,0x1d,0x38, +0xac,0xb9,0x5c,0x54,0xca,0x2b,0x13,0x87,0x80,0xc7,0x43,0x0b, +0x15,0xed,0x47,0xb9,0x20,0x71,0x8f,0x90,0xcb,0x6e,0x33,0x7b, +0x44,0xc5,0xa3,0x40,0x3d,0xd6,0x2d,0x44,0x2b,0x83,0xba,0x70, +0xc3,0x9d,0x58,0x50,0x87,0x99,0xd1,0x03,0x20,0xc8,0x88,0xb4, +0xc1,0x73,0xd0,0xb5,0x93,0x5e,0x67,0xe4,0xf4,0x2e,0x47,0xb5, +0xf4,0xa1,0xa7,0x94,0x08,0x47,0x61,0x4c,0x2d,0xb4,0xed,0x11, +0xc6,0xb2,0x88,0x85,0x08,0x30,0x91,0xf7,0xd1,0x33,0x2b,0x1a, +0x6f,0x76,0x3e,0xf0,0xa4,0xd0,0x96,0x0b,0x48,0xc1,0xc4,0x69, +0x30,0xb1,0xb3,0x66,0x80,0xc1,0xd6,0xa3,0x79,0xc7,0x25,0x2a, +0x39,0xc0,0x33,0x3a,0x3f,0xad,0x5d,0x43,0x47,0x46,0xba,0xbe, +0x19,0xf2,0x38,0x51,0x22,0xad,0x8c,0x55,0xa9,0xb4,0xc3,0xe6, +0x79,0x83,0x22,0xce,0x35,0x1e,0x3a,0x36,0x36,0x95,0xe9,0x4d, +0x29,0xad,0x54,0xed,0x5d,0x33,0xa9,0x3b,0xad,0xc7,0x05,0xf3, +0x17,0x82,0x44,0xee,0xca,0x41,0x35,0xb6,0x4a,0x48,0x55,0x9b, +0xed,0x8e,0x48,0x4f,0xe3,0x13,0x00,0xd1,0xcf,0xcd,0x9d,0x01, +0xc7,0x71,0xce,0x62,0x9d,0xa0,0x8f,0x36,0xeb,0xee,0xab,0xd5, +0xcd,0x3d,0x72,0xfa,0x20,0xea,0x65,0x0f,0x4e,0xa9,0x07,0x4f, +0xe3,0x0e,0xa6,0x3f,0x10,0xc5,0xdb,0xb8,0x23,0x9a,0xb7,0x79, +0xb5,0x5d,0x5f,0xf9,0x36,0x30,0x7b,0x67,0x1b,0xe0,0xd9,0xbc, +0xf3,0x26,0xec,0x98,0x1f,0x22,0x1b,0x31,0x45,0xab,0x74,0x17, +0x78,0xe4,0x52,0x46,0xe8,0x32,0xda,0xe1,0x3a,0xf5,0x31,0xed, +0x52,0x9c,0xa4,0x47,0x5d,0x65,0x44,0x5a,0x1f,0xa4,0xf6,0xe1, +0x25,0x62,0x55,0x58,0x5f,0x47,0x57,0xd7,0x32,0xb5,0x49,0xde, +0x2c,0xcc,0xfc,0x49,0x57,0xb4,0xcd,0xa5,0xce,0x4f,0x45,0x01, +0x53,0xab,0xbe,0x76,0xbc,0xdc,0x9f,0x76,0x63,0xf2,0x16,0x9d, +0x5f,0xd7,0xd2,0x93,0x13,0x33,0xaa,0xfc,0xfd,0xa8,0x16,0x3d, +0xb6,0xdd,0x6e,0xdb,0xce,0x05,0x78,0xb8,0xaa,0x5f,0xeb,0xd9, +0x58,0x31,0x05,0x1a,0x3b,0x27,0xc0,0x39,0xcb,0x03,0x7c,0x1e, +0x5e,0x90,0x05,0xed,0x9d,0xf9,0xeb,0xa8,0xb9,0xf4,0xf9,0x87, +0x8f,0x00,0xdc,0x8d,0xb2,0x55,0x45,0xe5,0x58,0xdd,0x50,0x96, +0x12,0x9d,0xed,0xaf,0xbb,0x21,0xd6,0xda,0xd2,0x56,0x07,0xaf, +0x68,0x97,0x6a,0xb6,0x9f,0xde,0x7e,0x53,0xd4,0x07,0x50,0x92, +0x3e,0xe9,0xde,0x4b,0x64,0xf3,0x06,0xf4,0xbd,0x64,0xb6,0x40, +0x64,0xbb,0x43,0xa2,0xe0,0xc5,0x3a,0xf9,0xd0,0xc5,0x1a,0x1a, +0x15,0x79,0x34,0x2b,0x58,0xb7,0x8d,0x98,0xe3,0xae,0x60,0x46, +0x87,0xd1,0x2b,0xcd,0x87,0x66,0x7a,0xd0,0xa0,0x02,0x6c,0x7e, +0xcf,0x18,0x6b,0x76,0x8e,0xef,0xda,0x78,0x3b,0x5a,0xda,0x8c, +0xbc,0x05,0xe8,0x6c,0x94,0x1c,0x35,0x48,0x77,0xb7,0x5d,0xb8, +0x97,0x20,0xa5,0xee,0xd8,0xdf,0xbf,0xd3,0x06,0xed,0xc1,0x5e, +0xb8,0xa8,0x5e,0xc9,0xba,0xff,0xc6,0x9e,0x55,0xc1,0xa6,0x6f, +0xcb,0xdb,0x2d,0xa1,0x4b,0xc1,0x61,0xa2,0x02,0xb9,0x3f,0x25, +0x0c,0x98,0x75,0xfc,0x3d,0x22,0x16,0x2c,0x79,0xe9,0xe4,0xad, +0x56,0x27,0x99,0xed,0xe3,0x07,0x53,0xcd,0x6f,0x6e,0xe1,0x4e, +0xff,0x43,0x68,0xc8,0x18,0x8a,0x91,0xc2,0xe2,0x5d,0x24,0xf9, +0xa9,0x13,0x72,0xdc,0x49,0xd8,0xbb,0xb8,0x65,0x5f,0x45,0x05, +0x87,0x73,0x8e,0x95,0x20,0xef,0xf4,0x95,0x7f,0xfc,0x63,0x4e, +0x8e,0x8b,0xb7,0x62,0x74,0x0d,0xed,0xdd,0x59,0xfa,0x3a,0x07, +0xdc,0xce,0x91,0xbf,0x35,0x95,0xe7,0xde,0x95,0xe9,0xd2,0xe6, +0x44,0x81,0x2f,0x83,0x75,0xc8,0x59,0xb2,0xb4,0x22,0x93,0x3e, +0xbd,0x73,0x2e,0xd6,0x0e,0x12,0xe5,0x4b,0x61,0xda,0x87,0x7a, +0xe0,0xc5,0x43,0x78,0xca,0x1e,0x70,0xc6,0x2a,0xdc,0x45,0xdd, +0xa3,0x24,0x4d,0x1e,0x58,0xd6,0xbf,0xc3,0x2b,0x96,0x03,0xd1, +0x61,0x3c,0x2b,0x93,0x02,0xc9,0xcc,0x7b,0x4c,0x1d,0x81,0xe9, +0xe3,0x54,0x87,0x72,0xca,0x53,0xc2,0xd7,0x3a,0x1d,0xb5,0xa7, +0xb4,0x65,0x1f,0x4b,0x62,0x47,0x28,0xb5,0x90,0x46,0xbf,0x46, +0x63,0xcd,0x4d,0x00,0xc5,0xb9,0x50,0xc0,0xfb,0x4d,0xf7,0x5a, +0x53,0xed,0xba,0x5e,0xb5,0x87,0x3c,0xda,0x2d,0x84,0xb6,0x28, +0x67,0x8c,0xaf,0xbd,0x94,0x88,0x98,0x17,0x16,0x9d,0xfe,0x87, +0xbb,0x6c,0xed,0xe5,0x2e,0x12,0x8a,0x6c,0xe3,0x05,0x55,0xca, +0x75,0xe0,0x95,0x73,0x9a,0x28,0x1c,0xbe,0x6d,0x7b,0xf0,0x3c, +0x83,0x6b,0x39,0x94,0x82,0x5b,0x0c,0xc7,0x06,0x9b,0x3e,0x66, +0x28,0x28,0x72,0x1f,0xcb,0x7a,0xb9,0x62,0x51,0xf1,0xa5,0x15, +0x0e,0x30,0xcb,0xd4,0x1d,0xff,0xf4,0x7b,0x94,0xbf,0x34,0x8b, +0x5b,0xba,0x1b,0x3a,0xc1,0x1d,0x4a,0x61,0x66,0xef,0x26,0xae, +0x20,0x87,0x7b,0x07,0xc7,0x5a,0x13,0x8c,0xcc,0x07,0x5b,0x02, +0xd3,0x37,0xf3,0xaf,0xec,0x00,0xbf,0x31,0x37,0xd6,0xda,0xbc, +0xbb,0xbd,0x27,0xde,0xa3,0x4b,0xb0,0xdf,0xc6,0x5c,0x4e,0xb7, +0xc4,0xbd,0x56,0xa3,0x8e,0x91,0xab,0x98,0xfb,0xda,0x1d,0xa3, +0x7d,0x56,0x8c,0x57,0x74,0x6d,0x2f,0xb7,0xe7,0x17,0xe8,0x8a, +0x8d,0xc0,0xf9,0x65,0xe2,0xed,0x7d,0xcd,0x09,0x59,0x22,0x93, +0x08,0x5c,0x12,0x8f,0xf3,0x8e,0x4d,0x68,0xb1,0x72,0x6b,0x66, +0x04,0xa3,0x24,0xe7,0x4c,0x98,0x1c,0xc7,0x36,0xc1,0x7c,0x88, +0x20,0x61,0x49,0xa2,0xf1,0x3b,0x51,0x94,0xcb,0xb9,0x20,0x45, +0xb7,0x0e,0x66,0x0e,0x30,0x1a,0x10,0x9d,0xd1,0x8e,0x6d,0xf3, +0x41,0xe2,0xde,0x42,0xa6,0x7e,0x47,0xfb,0xa6,0xd6,0x3a,0xde, +0xc2,0xe0,0x25,0x85,0x6f,0xa1,0x51,0x21,0x98,0x9e,0x44,0xe7, +0x29,0x2d,0xef,0xa2,0xe5,0x97,0x35,0x83,0xe1,0xc3,0x43,0xdb, +0xe5,0xa7,0x48,0x40,0x39,0xf4,0xb2,0x0a,0x13,0xf6,0x95,0x75, +0xab,0x30,0xd9,0x61,0x9b,0xbb,0xed,0xae,0xee,0x42,0xc0,0x01, +0xa2,0xd5,0xdc,0x33,0x5f,0xa9,0xda,0x28,0x1f,0xed,0xd2,0xa4, +0xda,0xc2,0x14,0x0d,0x92,0xe2,0xff,0xfa,0xf7,0xd5,0x88,0x9c, +0x57,0xcc,0x57,0xfd,0xc9,0x38,0xbb,0x78,0xbe,0x86,0x3c,0x90, +0xe3,0xe9,0xf7,0x5f,0x47,0x20,0xd9,0xaf,0x26,0x3d,0x98,0x83, +0x82,0x75,0x09,0x9f,0xfc,0xf7,0x86,0x9e,0x6c,0x2f,0x59,0xd1, +0xc2,0x84,0xc9,0xa6,0xfa,0xb3,0xd9,0x12,0x2a,0xef,0x91,0x15, +0x8e,0x88,0xbd,0x98,0x2b,0x03,0x44,0x49,0x19,0x2d,0xa8,0x3a, +0xe0,0xf9,0x26,0xf0,0x02,0x2a,0x29,0xc5,0x46,0xdd,0xe1,0xe3, +0x27,0xd4,0xc4,0x07,0xe4,0x77,0x9a,0x95,0xbc,0xd0,0x85,0xdf, +0xf1,0xff,0xc4,0xd7,0x7e,0xd0,0x9d,0x0f,0x4a,0x3f,0xbc,0xfe, +0xb2,0xfa,0x69,0xa9,0x7d,0x32,0x3d,0x7f,0xf3,0xa6,0x45,0x86, +0xd5,0xba,0xea,0x44,0xdb,0x58,0x83,0x4f,0x64,0xbd,0xaa,0x13, +0x5e,0x3b,0x81,0x9a,0xfe,0x54,0x35,0x1b,0x4d,0xb1,0xa8,0xbb, +0x7e,0x23,0x5b,0x52,0x98,0x49,0xa6,0xa3,0x07,0x07,0xe3,0x4b, +0xf4,0xb9,0xdc,0x9b,0x2e,0xdb,0x5b,0xaa,0x0b,0x67,0x72,0x93, +0x85,0x1f,0xae,0x6d,0x9f,0xd4,0x09,0xdd,0xe3,0x8f,0x9d,0x9b, +0x4c,0x21,0x81,0x34,0xdc,0x31,0x0f,0x77,0x53,0x5a,0xe1,0x91, +0x3f,0x75,0x54,0x89,0xdd,0xe7,0x6a,0x0f,0x84,0xf2,0x7e,0x5e, +0x13,0xeb,0xa3,0x8f,0x8c,0x97,0x43,0x9d,0xa3,0x14,0xb7,0x75, +0xea,0xe5,0x6c,0xb8,0x9a,0x8c,0xf8,0x95,0xbd,0x24,0x2a,0xb4, +0x46,0x5c,0xea,0x02,0x51,0xc5,0x01,0x89,0x53,0x50,0x39,0xd6, +0x9b,0xcc,0xcb,0xff,0x5f,0xf3,0xd6,0x98,0xe4,0xe5,0xff,0xaf, +0xce,0x75,0xe8,0x68,0x4a,0x3e,0xb4,0x17,0xe3,0x57,0xa3,0x73, +0xea,0x3d,0xd2,0x73,0x91,0x39,0xca,0x24,0x32,0x73,0xa1,0x0a, +0x3d,0x4c,0x79,0xc6,0x2c,0x43,0xfd,0x77,0xcd,0xc7,0xf5,0xf3, +0x31,0xba,0xb3,0x74,0x23,0x8f,0xbe,0xa0,0xc8,0xd4,0x8b,0x6c, +0x7e,0x4e,0x91,0x95,0x92,0x73,0x6b,0x5b,0x00,0x7e,0x0e,0x42, +0x3b,0xc5,0xbe,0x9a,0x43,0xb9,0x2c,0xc4,0xc4,0x75,0xf5,0x8d, +0x88,0x3a,0x15,0x3c,0x6c,0x50,0x05,0xff,0xd3,0x47,0xe5,0x29, +0x45,0xb6,0x7c,0x54,0x1e,0xd6,0xcf,0x81,0x3e,0x7e,0xdc,0x33, +0xca,0xa8,0xfc,0x48,0x00,0x19,0x62,0xd4,0x29,0x41,0x34,0x00, +0xf4,0x1a,0xf2,0x72,0xb6,0x82,0x61,0xa5,0x35,0x0a,0x95,0xb8, +0x88,0xce,0x44,0x91,0x49,0xc7,0xa7,0x26,0x03,0x45,0xeb,0x5c, +0xa9,0xfe,0x20,0xbb,0x23,0x94,0xb4,0x5a,0x4c,0xbe,0x47,0x1f, +0xd0,0xa4,0xe5,0x6f,0x6b,0x91,0xee,0xd6,0x1e,0x68,0xd0,0xd7, +0x3a,0xb9,0x8a,0x16,0x65,0xdd,0x4c,0xb1,0x7f,0x26,0x57,0xa9, +0x06,0x46,0x17,0xec,0x22,0x96,0x0c,0x2d,0xc2,0x81,0x23,0x99, +0x38,0x46,0x16,0xbc,0x21,0x6d,0xb1,0xb5,0x11,0x63,0x27,0x3d, +0x15,0x6f,0x3b,0x80,0x58,0x51,0x4d,0x02,0xc4,0x68,0xb1,0xbb, +0xb1,0x81,0xce,0x3f,0x5b,0xfe,0x18,0x67,0x5f,0xfa,0xd2,0x1c, +0x88,0x85,0xce,0x0e,0x03,0x4f,0x2c,0x00,0x19,0x36,0x0a,0xe2, +0x09,0x6b,0xfc,0x12,0xf8,0xf0,0x10,0xfa,0x57,0x2e,0xcf,0x1c, +0x2f,0xe7,0xec,0x4b,0x94,0x5b,0xc7,0x69,0xd9,0xaa,0x0f,0x83, +0x30,0x42,0xbb,0xff,0x2c,0xde,0x11,0x5c,0x4b,0x4a,0x21,0xf9, +0x4b,0x31,0xab,0xf4,0x35,0x2e,0x38,0x68,0x4a,0x1d,0x24,0x2e, +0x6b,0x30,0x4a,0xe8,0xa1,0x23,0x6a,0x7a,0xb8,0xe5,0x7b,0xc4, +0x18,0x9d,0x43,0x0d,0x0c,0xfa,0x36,0x7d,0xc9,0xc7,0x80,0x0b, +0x64,0x10,0xba,0x33,0x66,0x56,0x71,0x3d,0xdb,0x4b,0xa7,0x39, +0xa4,0xfe,0xe2,0xf9,0x97,0x4f,0x7f,0x78,0xf1,0xfa,0xf4,0xe9, +0x33,0x3a,0x17,0x34,0xdd,0x1a,0xd0,0xd9,0x21,0x31,0x47,0x28, +0x6b,0x1c,0xf1,0xa2,0x97,0x7d,0x3e,0x1b,0x6e,0xd0,0x66,0xef, +0xf7,0xdf,0xbd,0x7a,0x7d,0xfb,0xfd,0x0f,0xf0,0x0f,0x3d,0xeb, +0xc7,0x1f,0xd5,0xc7,0x62,0x30,0x86,0x77,0xa9,0x97,0xa3,0xe5, +0xc5,0x6c,0x18,0xb7,0x75,0x5b,0x4e,0xa6,0xb4,0xf9,0xb3,0xa0, +0x13,0xd5,0x6b,0xaa,0xde,0x91,0xea,0x3d,0x14,0xbb,0x8b,0x48, +0x66,0xb1,0xb3,0xa5,0xc8,0x20,0x13,0xf3,0x06,0xf1,0xd0,0x7e, +0xef,0xf1,0x0f,0x5b,0xec,0x9a,0xa2,0xf7,0x50,0x03,0x48,0x7b, +0x47,0xec,0x3c,0xfe,0xa8,0x45,0x3f,0xcd,0x16,0xbb,0x81,0x37, +0x6c,0xd2,0x6b,0xc6,0xe4,0x7e,0x0a,0xe1,0x03,0x2a,0x58,0x79, +0x0a,0xf8,0x38,0xc5,0xcd,0x06,0xc5,0x2d,0x95,0xa0,0xd6,0x81, +0xce,0x93,0x28,0xa9,0xb6,0x29,0x99,0x4d,0x4a,0x53,0xa7,0x1c, +0x19,0x17,0xf6,0x1e,0x90,0x8e,0x85,0xd1,0x12,0x92,0x76,0x18, +0x85,0xa4,0x65,0x90,0x62,0xa0,0x6d,0xf1,0x8f,0xef,0x7b,0x89, +0x17,0xff,0xf5,0xfe,0x22,0x10,0x95,0xfa,0xbd,0x21,0x90,0x28, +0x73,0x0f,0x72,0x57,0x73,0x7c,0x33,0xfc,0xd0,0x78,0xb1,0xcf, +0x1e,0x9c,0x70,0x0d,0x8a,0x29,0xfe,0x40,0x93,0xfc,0x01,0xa1, +0xdc,0x55,0x74,0x44,0x7b,0xd3,0xd8,0xda,0x12,0x25,0x95,0xa3, +0x3b,0x71,0xdf,0x38,0xfb,0x7a,0x9a,0xc1,0x32,0x37,0x18,0x3d, +0xeb,0x4d,0x26,0x29,0x82,0x03,0x21,0x97,0x63,0x66,0x67,0x86, +0x8b,0x65,0xaf,0xef,0xe7,0xa6,0xe6,0xb6,0x84,0x57,0x44,0xb9, +0xa3,0x73,0xd2,0x6d,0xb1,0x41,0x6d,0x61,0x7f,0xcc,0x13,0x2b, +0x5c,0x63,0x40,0x3a,0x3d,0x1b,0x9f,0xa3,0xbe,0xc8,0x22,0x3f, +0xb6,0x52,0x0d,0xc6,0x46,0x1d,0x1e,0xe6,0xe3,0x6a,0xba,0xe8, +0xed,0xed,0xee,0x71,0x6a,0xe0,0x93,0x8b,0x8c,0x7f,0xa0,0x12, +0x2a,0x7f,0x7b,0x6b,0x4f,0x83,0xfc,0x41,0x56,0x38,0x8d,0x95, +0xb8,0x67,0xd8,0x4a,0x01,0x4c,0x4a,0x42,0x17,0x1b,0xb6,0xf5, +0xe8,0xb8,0xc8,0x52,0x87,0xa7,0x37,0x3b,0x57,0x90,0x52,0x96, +0x4d,0x26,0xc9,0x9a,0x3a,0x49,0xd1,0xb4,0x51,0x03,0x99,0xe8, +0x07,0xbd,0x20,0x38,0x60,0x94,0x9d,0x30,0x76,0xce,0xf2,0x35, +0x46,0x14,0x67,0xa3,0x58,0x46,0x69,0xac,0x93,0x68,0x0e,0x62, +0x4f,0x7e,0xb3,0xcb,0x71,0x36,0x4a,0x49,0x48,0x70,0x2a,0x88, +0x6b,0x4b,0x34,0x51,0x55,0x60,0x4a,0x4f,0x4f,0x03,0xa9,0x99, +0x1b,0xa9,0x4e,0x0d,0x88,0xe7,0xe0,0x8f,0x24,0x88,0x47,0xb7, +0xc4,0x20,0xb8,0xf1,0xd2,0x02,0x8e,0x56,0x13,0xa2,0x84,0x83, +0xf4,0xe0,0xc0,0xe7,0xb0,0x78,0xf7,0xb8,0x19,0x9c,0x9d,0xe3, +0xb0,0xc1,0x5e,0xc3,0x9b,0x0b,0x9d,0xe3,0xc1,0x80,0x70,0x5e, +0x2d,0xc8,0x6f,0x4a,0xed,0x81,0x19,0x57,0x1a,0x45,0x1c,0x5d, +0xa8,0x53,0xde,0xc3,0x27,0xfd,0x53,0x1a,0x38,0xfa,0x9e,0x03, +0x83,0x78,0xb5,0x11,0x70,0x79,0x89,0x34,0xcd,0x60,0xad,0x9f, +0xf5,0x7f,0x21,0xf5,0xf4,0x42,0xe4,0x73,0xd9,0x40,0x8a,0xf3, +0xe1,0x44,0xee,0x79,0x43,0xda,0x30,0xb3,0x38,0xd1,0xcc,0x50, +0x17,0xb5,0x03,0xe5,0x5c,0x8f,0x8f,0x1d,0xbc,0x11,0x46,0xa9, +0xf8,0xe6,0xa9,0xc5,0x9e,0xef,0xec,0x5a,0x27,0x07,0x18,0x86, +0xdc,0xa9,0xfc,0x9a,0x8d,0xa6,0x24,0x23,0xa1,0x26,0x57,0xe4, +0xcc,0x40,0x45,0x32,0x97,0xdc,0xde,0xe2,0x8e,0x3a,0x16,0xd6, +0x36,0x9d,0x89,0x02,0xfb,0x68,0x91,0xc5,0x2a,0xb7,0xf4,0xa5, +0x7e,0x4e,0xc7,0x50,0x55,0x6e,0xd9,0x2c,0xaa,0x76,0xc4,0x63, +0x8d,0x2b,0x75,0x6c,0x1a,0xfe,0x0a,0x15,0x20,0xed,0x6c,0xdc, +0x56,0x98,0xaf,0x60,0x16,0xd9,0x39,0xf2,0x1d,0x6b,0x5d,0xde, +0x0c,0x26,0x04,0x11,0xa3,0x5d,0x39,0x42,0xa9,0x1c,0xa6,0x89, +0x36,0xdc,0xae,0x34,0xfd,0x71,0x6f,0xbb,0x9c,0xe1,0x8e,0x05, +0x24,0x6d,0xb6,0x29,0xec,0x2c,0x91,0x32,0x4f,0x7b,0x6b,0xa2, +0x27,0xeb,0xc8,0xc8,0x83,0x59,0x80,0x97,0x1b,0x9d,0x53,0xaf, +0x2e,0x02,0xc0,0x2c,0xa3,0x3c,0x2e,0x01,0x21,0x98,0xfb,0x52, +0x7f,0x61,0xd6,0x95,0xd1,0x93,0x05,0xbf,0xc6,0xb6,0x15,0x58, +0xa0,0x9c,0x69,0xe7,0xed,0x2d,0x47,0xe0,0x83,0x1e,0xd3,0x1e, +0xdc,0x32,0x3a,0x6b,0xfc,0x70,0x38,0xc6,0x8f,0x1e,0xcf,0x32, +0x5f,0x18,0xf9,0x55,0x20,0x16,0x49,0x9e,0x3b,0x25,0xc3,0x1d, +0xb0,0x8c,0x14,0x63,0xa9,0xba,0x65,0xf7,0xe8,0x9e,0xb0,0x93, +0xde,0x80,0x34,0xd7,0xba,0x61,0xa9,0xa4,0x55,0xfa,0xea,0xf9, +0xeb,0xd2,0x56,0x65,0xbd,0xab,0x91,0x8d,0x43,0x81,0x06,0x22, +0x7f,0xc5,0x4d,0x83,0x9f,0x53,0xc9,0x90,0xc3,0x6d,0xb1,0x9c, +0x1c,0xd9,0x0c,0x5f,0x3c,0x7f,0xf1,0xfc,0xf5,0x73,0x28,0x58, +0xe2,0x67,0x04,0xa5,0x7c,0xd2,0x56,0x21,0x6b,0x9a,0x3d,0x1a, +0x1d,0x40,0xc9,0x68,0x35,0x91,0x12,0x96,0xf6,0x9b,0x68,0x0e, +0x2a,0x1c,0x9c,0x26,0x0e,0x03,0xca,0xb2,0x41,0x6a,0xe7,0x0f, +0x1d,0x25,0x6e,0xf0,0x6d,0x4f,0xc0,0x1c,0x02,0xf4,0x44,0x81, +0x49,0x48,0x2f,0xd7,0x8d,0x14,0x87,0xa9,0xb1,0x91,0xd3,0x49, +0x92,0xb4,0x8c,0x99,0xde,0xb8,0xeb,0x44,0xcb,0xb1,0x60,0x44, +0x8b,0x85,0xb0,0x0d,0x77,0xc2,0x0f,0x5a,0x55,0x1a,0xb7,0x5c, +0xbc,0xb5,0xc2,0x83,0x10,0xe7,0x31,0xcc,0x0a,0x5f,0x54,0xe9, +0xbc,0xa4,0x7f,0x76,0x66,0xf6,0x31,0xca,0x6e,0x4f,0x28,0xde, +0xdd,0xad,0x98,0xd5,0x13,0x22,0xc5,0x7e,0x77,0xfd,0xcd,0x1f, +0xeb,0xce,0xde,0x99,0x10,0x89,0x6f,0x0e,0x1c,0x8f,0x1c,0xa5, +0x9f,0xdf,0xa0,0x4b,0xf1,0x52,0xcc,0x82,0x29,0xe7,0x38,0x3c, +0xa4,0x5f,0xcf,0x75,0x47,0x29,0xfa,0xf9,0xf6,0x04,0x32,0xbf, +0x79,0xd3,0x8d,0x61,0x0b,0x46,0x19,0x2a,0xa5,0xe8,0xcd,0x9b, +0x3f,0xde,0x7e,0x14,0xeb,0xe2,0xb8,0xae,0xc1,0x48,0x33,0x58, +0xb1,0x18,0x85,0xcf,0x5b,0x70,0x9d,0xc5,0x86,0xad,0xe8,0x18, +0x46,0x76,0x79,0xb0,0xed,0xc7,0xed,0x20,0x6c,0x11,0xf5,0xf8, +0xb3,0x6b,0xa6,0xfb,0x3c,0xd7,0x69,0xa7,0x6d,0xcc,0xa9,0xa1, +0x05,0x4d,0xdd,0x52,0x34,0x7c,0x55,0x61,0xf2,0x74,0x38,0xfd, +0x44,0x47,0x74,0x5b,0xfc,0x98,0x48,0xc6,0x88,0x8d,0x77,0x96, +0x1a,0xad,0x3d,0x8e,0xdb,0x5d,0xd6,0x27,0x61,0xbf,0x47,0x9d, +0xc8,0x76,0x5c,0x5a,0xb8,0x8b,0xcf,0xb5,0xd5,0xa5,0x24,0x90, +0x4f,0xd7,0x66,0x29,0xa8,0x4a,0xe7,0xe8,0xc5,0xdc,0xc0,0xad, +0x94,0x3e,0x4a,0xd0,0x00,0xea,0x3e,0x40,0x51,0xbd,0x13,0xef, +0x01,0x66,0x48,0x45,0x17,0xe6,0x6a,0x02,0x4b,0xcb,0x78,0x7a, +0xfe,0x6a,0xd2,0xcb,0x2e,0x46,0x30,0x8b,0xf5,0xc6,0xe6,0xb4, +0xa1,0x54,0x87,0xad,0x1d,0x46,0x38,0x3b,0x3b,0x0c,0xb6,0xfc, +0x32,0x15,0x8c,0xdb,0x16,0x77,0x65,0xbd,0xf2,0x51,0x9d,0x1c, +0xb1,0x17,0x24,0xbd,0xa9,0x45,0x9d,0xf4,0xcd,0xbb,0x4a,0xf4, +0xd1,0xed,0x9b,0x4e,0x1c,0x43,0xbe,0x1a,0x64,0xe4,0xd1,0x51, +0x2b,0xca,0xff,0xe6,0x4d,0x0d,0x32,0xd5,0x31,0x97,0x66,0x83, +0xf9,0xee,0x9d,0xad,0xcf,0x21,0x24,0xe2,0xdd,0xde,0xca,0xe0, +0x13,0x29,0x2c,0xf5,0x42,0xc4,0x5f,0x5e,0x8c,0xb3,0xed,0xa5, +0xd3,0x40,0x15,0x4c,0xbb,0xdb,0xae,0x3d,0x7d,0xe3,0x9f,0x9a, +0x39,0xd9,0xdb,0x71,0x34,0x15,0x1c,0x48,0x99,0x8d,0xab,0x38, +0x1b,0x8c,0xe6,0x5a,0xa9,0x1d,0xe4,0x47,0x7b,0xd6,0xc2,0x33, +0x3b,0x3e,0x24,0x97,0x4f,0xe7,0x80,0x70,0xab,0x6e,0xcc,0x94, +0xd3,0xe2,0xe2,0xa8,0x37,0xc9,0x10,0xec,0xce,0xf8,0x23,0x9a, +0x9d,0x60,0x04,0x5c,0x8d,0x59,0xdb,0xde,0xa4,0xe0,0x4c,0x26, +0xbc,0x83,0x9f,0x78,0x75,0xa0,0x8f,0x21,0x60,0x4d,0x9f,0x66, +0x63,0x3a,0xf6,0xc4,0x58,0x7a,0xb3,0xfb,0xb5,0x8d,0xfc,0x86, +0x38,0x08,0x90,0x4c,0x6f,0x66,0x8b,0xf1,0xf9,0x98,0x96,0x98, +0xe5,0x45,0x8b,0xc0,0x2c,0x46,0xe7,0xa3,0xf5,0x9c,0xbe,0xb7, +0xac,0xdc,0x0e,0x19,0x6b,0x6f,0xc5,0x0e,0x99,0xb4,0x0c,0x53, +0x53,0xfc,0x63,0x3b,0x3a,0x3a,0x89,0xe2,0x5a,0x37,0xc6,0x29, +0xe0,0xcd,0x1b,0xe4,0x76,0x27,0xe9,0x4d,0x3d,0xee,0xb4,0xd0, +0x83,0x6f,0x1c,0x9d,0xbc,0xe9,0xdc,0xbe,0x29,0x77,0xe3,0x0e, +0x64,0x74,0x46,0x7f,0x86,0x3c,0x49,0x0a,0x10,0xfa,0x71,0x83, +0x55,0x8e,0xa0,0xf7,0x61,0x56,0x1d,0x4c,0x34,0x5c,0x59,0x5d, +0x04,0x35,0x67,0xd2,0x52,0xb9,0x38,0xb5,0x6d,0xed,0xf9,0xc9, +0x0d,0x14,0xbb,0xff,0xb6,0xd5,0xe0,0xc1,0xef,0x81,0xfe,0xc4, +0x17,0x1d,0x88,0x46,0x4a,0x7f,0x49,0xb9,0xbb,0x54,0xaa,0x44, +0x3a,0x19,0x95,0x7f,0x29,0x25,0x86,0xa1,0xd9,0x69,0xb9,0x29, +0x14,0xdd,0x82,0xf1,0x52,0x89,0x10,0x21,0xd8,0xb8,0xa0,0xed, +0xc9,0x4e,0x09,0x60,0x44,0x27,0x3f,0xd7,0xd1,0x41,0x31,0xa4, +0x95,0x62,0xa7,0x8c,0xb8,0xf9,0x70,0xe9,0xf7,0xa6,0xfe,0x11, +0x52,0xc6,0x52,0x10,0xbb,0xa8,0xc6,0xfd,0x91,0x7a,0xb3,0x3c, +0xce,0xd8,0xcb,0x0b,0x98,0x54,0xd0,0xd1,0x8a,0xe9,0xd6,0x4e, +0x69,0x5c,0x42,0x24,0xb0,0x98,0x3d,0x2c,0xa1,0xc5,0x84,0x0e, +0xc3,0xd0,0x64,0xaa,0x2b,0x6b,0x61,0x67,0x63,0x0e,0xda,0x4f, +0x70,0xde,0x13,0x8c,0xed,0x9a,0x13,0x94,0xc5,0x08,0xad,0x46, +0x7f,0x37,0x7d,0x35,0xea,0x2d,0x06,0x17,0x2c,0x0c,0x60,0x3e, +0x5a,0x59,0x71,0x5d,0xf1,0x39,0x90,0xa1,0xc5,0xb2,0xee,0xb2, +0xe4,0xc5,0xce,0x1b,0x90,0xbd,0x52,0x9a,0x97,0x30,0x8d,0x6a, +0xb1,0x06,0x6d,0x3a,0x14,0x94,0x93,0xa6,0x86,0xf2,0x12,0xe3, +0x16,0xb7,0xb4,0x5e,0x6a,0x0b,0x86,0x2e,0x48,0x17,0x53,0x8e, +0x7d,0x3d,0x13,0xce,0x75,0x70,0x73,0x4b,0x68,0x1c,0xb5,0x4c, +0x8f,0x84,0x91,0xb3,0xc2,0x19,0xc8,0xc9,0x8b,0x77,0x63,0xd7, +0x6d,0x95,0x88,0x9b,0x37,0x4e,0x66,0xa2,0x22,0x29,0x16,0x99, +0x6d,0xa0,0x03,0xe3,0x23,0x10,0x43,0xf0,0xce,0x6e,0x31,0x9b, +0xb1,0x8d,0x6e,0x3c,0x3f,0xc6,0x5b,0x7e,0xba,0xe1,0x56,0x98, +0x80,0xa3,0x99,0x37,0xbd,0x7c,0xb4,0x5c,0x12,0xe7,0xf8,0x33, +0xf2,0xf4,0xbb,0xd3,0x75,0x40,0xe0,0x14,0xc0,0x54,0xa0,0x0c, +0x78,0xe5,0x02,0x87,0xad,0x80,0x32,0x80,0x55,0xb1,0x0f,0x01, +0xf2,0x11,0x60,0xa6,0x13,0x3e,0x70,0xa2,0xd9,0x86,0xe6,0x07, +0x98,0x6c,0x00,0xa6,0x30,0x08,0xf6,0x25,0xcf,0x05,0x24,0x2b, +0xe1,0xa7,0x73,0x56,0x85,0x27,0x4d,0x9c,0xc0,0xcc,0x1a,0xeb, +0x9b,0x65,0x1c,0x83,0xa4,0xbc,0x97,0xba,0xc9,0xec,0x66,0x4c, +0xcc,0x9d,0x1e,0x5c,0x7a,0xb9,0xad,0x3d,0x13,0xd2,0x01,0x4a, +0x2f,0x1d,0xb5,0x9f,0xe3,0x71,0xbb,0x52,0x19,0x1b,0x64,0xe4, +0xe1,0x06,0x5a,0x44,0x42,0x89,0xa1,0x84,0x73,0xf3,0xf4,0x1c, +0x97,0x3e,0x98,0x51,0x67,0x67,0x0f,0x2e,0x4f,0xc6,0xdd,0xce, +0x70,0x94,0x3b,0xee,0xc6,0xf8,0xb8,0x85,0x7f,0xdb,0x74,0xa0, +0x70,0x85,0x17,0x5b,0x91,0x5d,0x39,0x6a,0x53,0xfd,0x6a,0xc4, +0x70,0x09,0x27,0xda,0x49,0x79,0x35,0x87,0x6d,0xe3,0x88,0x4f, +0x20,0xb5,0xb9,0xfb,0xf5,0x92,0xcd,0x20,0x4b,0x2c,0x29,0x5c, +0xa6,0xdc,0x25,0x35,0xb1,0xda,0x4e,0xa6,0xe7,0x58,0x43,0x96, +0x6d,0x8b,0xd5,0x3e,0xe2,0x0c,0x30,0x7b,0x61,0xa4,0x0e,0x1e, +0x1e,0xca,0x6b,0x35,0xca,0x83,0x9c,0x2c,0x9d,0x4a,0x99,0x6c, +0x18,0xc4,0x98,0x03,0xca,0xe2,0x8f,0x53,0x88,0x05,0x32,0x0e, +0x46,0x2f,0x29,0x96,0x95,0x90,0xf5,0xca,0x29,0x10,0x09,0x1a, +0xed,0x7f,0x9d,0x44,0x8f,0x81,0x62,0x65,0x79,0xac,0x86,0x5e, +0xb2,0x7b,0x43,0x58,0x45,0xd0,0xbe,0x3b,0x65,0xfa,0x81,0x28, +0x50,0x22,0x8c,0x70,0x2f,0x8d,0x70,0x6f,0x6f,0x29,0x04,0xc4, +0x74,0xaa,0x27,0xf7,0xa3,0xfb,0x20,0xb1,0xd1,0xf9,0x57,0xa8, +0x01,0x59,0x22,0xd3,0x56,0x0c,0x53,0xf9,0xa4,0x23,0xc4,0x15, +0xd3,0x4f,0x9a,0xac,0x07,0x3c,0xb9,0x07,0x7c,0x45,0x9d,0x1f, +0x05,0x49,0x71,0xc7,0x8c,0x0e,0x22,0x9b,0x7b,0x21,0x1a,0xe6, +0x55,0x0e,0x69,0xe2,0xb8,0x96,0x11,0x29,0x23,0x37,0xd2,0xcc, +0xd5,0x71,0xcb,0x82,0x05,0x49,0x25,0x84,0x94,0xeb,0xb6,0x00, +0x89,0xd8,0x89,0x90,0x6a,0xe2,0xd8,0x81,0x4e,0xfb,0x78,0x9e, +0x66,0xd0,0x9c,0x42,0x70,0x78,0x14,0x6b,0xeb,0x8f,0xbe,0xe5, +0x06,0xe5,0x9a,0x61,0x10,0x6d,0x3f,0xbb,0xb3,0x14,0x0c,0x69, +0x5f,0x6e,0xb6,0xb7,0x5a,0x18,0xe3,0xdc,0x85,0xc2,0x98,0xd5, +0x64,0xea,0xa6,0x86,0xc8,0x2c,0x58,0x75,0xcc,0xf4,0x42,0x2f, +0x31,0x39,0xb2,0x65,0x23,0xc7,0xd3,0xab,0xd9,0x5b,0xab,0xc7, +0xa4,0xac,0x00,0x6e,0xae,0x71,0x08,0x2b,0x1d,0x8a,0x3b,0xce, +0x3e,0xdf,0x44,0xa2,0x67,0xd8,0xf0,0xd6,0xc7,0xeb,0x14,0x7c, +0xd9,0x1b,0x02,0x46,0x2b,0x16,0x1e,0xec,0x1f,0x78,0x27,0x53, +0x50,0x81,0xa4,0xb8,0x25,0x9d,0x6f,0xbf,0x26,0x97,0xc4,0x69, +0xe0,0xc0,0xcb,0x75,0xce,0xe1,0xc2,0x2e,0x68,0xb6,0x54,0x49, +0xa0,0x27,0xe4,0xab,0xe1,0x75,0x71,0xed,0xa6,0xbe,0xd4,0xba, +0xec,0x70,0x93,0x77,0xb9,0xee,0xd8,0x75,0xd8,0x18,0x5e,0xc3, +0xe0,0xd1,0x17,0x88,0x9c,0x45,0x38,0xc6,0x46,0x8b,0xad,0xf6, +0x51,0x8e,0xfe,0xc4,0x9f,0xbd,0xc9,0x24,0xd2,0xcf,0xb3,0x72, +0x75,0x4a,0xc2,0x0d,0xcd,0x8a,0xc1,0x24,0x28,0x4d,0xb7,0x14, +0x20,0x56,0x15,0x1e,0xa4,0x29,0xc9,0x9d,0xa3,0xbc,0x29,0xe9, +0x8e,0x39,0x49,0x66,0x12,0x3e,0xca,0x71,0xe7,0x12,0xd7,0x0e, +0xbc,0x9c,0x29,0x15,0x63,0x76,0x0f,0xf0,0x74,0x56,0xe6,0x00, +0x97,0x23,0x23,0x34,0xa2,0xe4,0x48,0xe6,0x76,0x1d,0xb8,0x11, +0x37,0xe5,0xd8,0x18,0xd6,0x61,0x0f,0x46,0x1f,0x8b,0x35,0xce, +0xf9,0x1f,0xaf,0xb8,0x07,0xe2,0x6f,0x53,0x16,0xa6,0xb4,0x60, +0x79,0xce,0x4d,0x2b,0x22,0xdb,0x40,0x21,0x56,0x91,0xd7,0x9b, +0x13,0x96,0xd9,0xe4,0xf2,0xa7,0x65,0x67,0x85,0xfc,0x34,0x64, +0x64,0x1a,0x3b,0x7b,0xc9,0xf5,0x8c,0x36,0x88,0x6d,0x96,0x2c, +0xae,0x4c,0x47,0xdf,0xde,0x8a,0x78,0x86,0xab,0x38,0xf9,0x1d, +0x77,0xaa,0x96,0x58,0x83,0xc1,0xcd,0xd6,0x85,0x7f,0xb3,0x75, +0x1f,0x20,0xbb,0x33,0x34,0xaf,0xe8,0x1a,0xa7,0x1b,0xe7,0x28, +0xcf,0xee,0x44,0x34,0x19,0x25,0x33,0x89,0xd5,0xda,0xdf,0x59, +0xcb,0xdd,0x36,0x67,0xbc,0xa1,0x57,0x63,0x9a,0x40,0xc9,0x78, +0x65,0x2c,0x07,0x7c,0x62,0xd4,0xeb,0x9c,0x1f,0x45,0xd0,0x43, +0x47,0xd6,0x1b,0xa2,0x18,0x22,0x77,0x2a,0x01,0x6d,0x0b,0x9c, +0x37,0x33,0xb5,0x72,0x5c,0x27,0x35,0xe1,0xd4,0xcd,0x7b,0x92, +0x20,0x76,0x16,0xb2,0xb3,0x29,0xa5,0x93,0xe0,0xb0,0xca,0x6f, +0x8c,0x32,0x3f,0x0a,0xef,0x62,0x9a,0xc3,0x29,0xc4,0x9e,0x7a, +0xa9,0x18,0x7b,0x1a,0x2d,0xf1,0x6d,0x54,0x7e,0x95,0xc5,0x7e, +0xb9,0x61,0x92,0xb7,0x84,0xaf,0x58,0x34,0x70,0x3d,0xa6,0x78, +0xc5,0x1a,0xde,0x90,0x42,0xb3,0x6a,0x4f,0xb3,0xcd,0x74,0x10, +0x39,0x12,0x0e,0x8c,0x6f,0x4d,0x6c,0x37,0x2f,0xa9,0xe8,0x68, +0x06,0x0a,0x86,0x9e,0x5b,0x5a,0xf0,0xda,0x76,0x73,0x7b,0x5a, +0x6a,0xa1,0xb3,0xb3,0xb5,0x12,0xb4,0xa7,0xc9,0x47,0x3a,0x6a, +0x5b,0xe7,0xfa,0x7e,0x7a,0x8e,0x6a,0xb2,0xfa,0x08,0x95,0xc1, +0xef,0xf4,0x88,0x65,0xad,0xdf,0xa8,0x7d,0x56,0x04,0x43,0xc7, +0x58,0xbe,0xbb,0x85,0x0f,0xf7,0x92,0xb5,0xcb,0xef,0x42,0x60, +0x7f,0x8a,0x95,0xd4,0x4c,0xcb,0x1c,0xff,0x38,0xd8,0xc4,0xff, +0x76,0x77,0x54,0xa1,0xc4,0x2b,0xd3,0x99,0xcc,0xd9,0x05,0xb3, +0xa7,0x13,0x96,0x99,0xdd,0x2e,0x62,0x1c,0x06,0xf1,0x37,0x58, +0x57,0x70,0x17,0x60,0x96,0x96,0x3d,0xbe,0xa3,0xc4,0x88,0xeb, +0x7d,0xbc,0x46,0x31,0xf1,0x14,0x1b,0x4e,0xa5,0xb7,0x1a,0xce, +0xda,0xb5,0xd3,0xd2,0xab,0x71,0xf7,0xc4,0x56,0xee,0xa1,0xba, +0xd4,0xfa,0x7d,0x22,0x50,0x8e,0xbb,0x27,0x4d,0xaf,0x50,0xc4, +0x47,0x1d,0x6c,0xb7,0x33,0x34,0x35,0xb2,0xfb,0x3b,0xac,0xaa, +0xb9,0xe6,0xc8,0x34,0xb5,0x32,0x17,0x1a,0xc1,0x60,0xdf,0xf4, +0x26,0x6b,0xea,0xb0,0x7a,0x94,0x87,0x64,0x75,0xcd,0x73,0x49, +0x4f,0xb3,0x80,0xb3,0x4e,0x8a,0xf2,0xe0,0x45,0xab,0x41,0x4b, +0x48,0x2b,0x0e,0x52,0x5d,0x3f,0x9d,0xae,0x56,0x65,0xbe,0x80, +0x25,0xe0,0x07,0x94,0xdd,0x62,0x57,0x04,0xee,0xb5,0x0a,0xfd, +0x60,0x91,0x4b,0xf4,0xc0,0x09,0x56,0x90,0xf1,0xca,0xf3,0x85, +0xa5,0x02,0xf7,0x5a,0x5b,0x79,0x6a,0x9e,0x63,0xa5,0xed,0x5d, +0xbe,0xe6,0x76,0xfa,0xb8,0x6b,0xbb,0x06,0x80,0x8a,0xc4,0x14, +0x1e,0x59,0xb1,0xd2,0x23,0x0c,0x95,0x17,0xb9,0x67,0xc3,0x0b, +0x10,0xde,0x14,0xda,0x8b,0x12,0x0a,0xee,0xb8,0x3e,0xd9,0x75, +0xd9,0x62,0x44,0xbd,0x34,0x77,0x30,0x7e,0xc7,0x05,0x8b,0x16, +0xff,0x9d,0x34,0x8e,0x68,0x07,0x6a,0x8b,0xca,0xbb,0xf5,0xd1, +0x81,0xe5,0xec,0x7f,0x65,0xb3,0xa9,0xb8,0xd5,0x64,0x31,0x86, +0x54,0xf3,0xd2,0x02,0x4d,0x3d,0x4c,0xb5,0x6a,0x7a,0x73,0xbd, +0x42,0x08,0xfd,0x4a,0xca,0x3f,0x13,0x45,0x9f,0xa4,0x0e,0xc0, +0x5c,0x7e,0x7d,0xda,0x52,0xb4,0xea,0x90,0xe9,0x7a,0xa7,0xec, +0xd0,0x3a,0xc1,0xe5,0x55,0x06,0xdd,0xf6,0x38,0xab,0x4d,0xec, +0x07,0x6b,0xb2,0xdd,0xe1,0x23,0x9f,0xd5,0xf2,0x2e,0x27,0x8f, +0x03,0x87,0xc1,0xad,0x9d,0xdb,0xee,0x87,0x1e,0x46,0xef,0xd4, +0x93,0xcc,0xd3,0xf2,0xd9,0x6c,0xf6,0x76,0x8c,0xb6,0xf8,0xf3, +0x4a,0x8f,0x03,0x9b,0xe4,0x9f,0x57,0x81,0xe4,0xfb,0x2e,0xf3, +0x1d,0xe4,0x3a,0xc7,0x4d,0x92,0xea,0x1e,0x44,0xa3,0x2c,0x23, +0x33,0x37,0xda,0x7e,0xe1,0xfd,0xa3,0xe4,0x13,0x04,0x84,0xbd, +0x49,0x29,0x84,0x7c,0xb5,0x6a,0xf3,0xf9,0x92,0x1e,0xbb,0x8a, +0xe4,0x82,0x19,0xdf,0xd3,0x82,0x38,0xab,0xeb,0xac,0x49,0x02, +0x01,0x70,0x88,0xe0,0x01,0x95,0x3c,0xf2,0xf6,0xc8,0x03,0xa5, +0x42,0xd6,0xf5,0xdc,0xb8,0x1f,0xf0,0x21,0xcc,0x33,0xb7,0x04, +0x2e,0x9b,0x85,0x95,0x8b,0xc6,0x80,0xb4,0x0a,0xaf,0xc4,0x5a, +0xf9,0x9b,0xa6,0x1c,0xb8,0xb8,0xe3,0xe1,0x93,0xaf,0xaf,0x25, +0xd2,0x9f,0x97,0x0d,0xd7,0x14,0xa9,0x28,0xbe,0x31,0x35,0x26, +0x01,0x85,0xd3,0x1c,0xa2,0xc5,0x74,0xf1,0x40,0x43,0x9b,0x7d, +0x28,0xa6,0xd9,0x6e,0x7f,0x14,0x65,0x89,0x3b,0x45,0xa8,0xb6, +0xfc,0xf6,0x15,0x15,0x74,0x69,0xc6,0x33,0x1d,0x19,0x2c,0xf7, +0xb9,0x45,0xc0,0xa0,0xfa,0x93,0x43,0x23,0xd2,0x86,0x5a,0x4d, +0x9f,0xe3,0x71,0x1f,0xb4,0xbf,0xe8,0x8a,0xd8,0x20,0xed,0x4c, +0x53,0x26,0xce,0x88,0xb1,0x9a,0x52,0xbd,0xe1,0xf0,0x7b,0x18, +0x8e,0x5f,0x4e,0xc3,0x57,0x99,0x32,0xbb,0xef,0x21,0x6d,0x1e, +0xe9,0x83,0xd4,0x2f,0xa5,0xfd,0x66,0xf8,0xbd,0xe4,0xe7,0xe1, +0x0d,0x6f,0x10,0xe7,0x0e,0x8b,0xc2,0x0c,0xba,0x2b,0x35,0x31, +0xfc,0x8d,0x2b,0x3f,0x33,0x45,0x1d,0x97,0x28,0xb6,0xe4,0xf2, +0x05,0x7d,0x7e,0x1e,0x80,0x03,0x37,0xd6,0xd0,0xb6,0x05,0x93, +0xc3,0x2b,0x08,0xb1,0x52,0xb4,0x99,0x2c,0x5c,0x3b,0x9c,0x8c, +0x85,0x96,0xa6,0x51,0xdd,0xc0,0xa4,0xca,0xdb,0x3a,0xed,0x20, +0x4e,0x67,0xe6,0x27,0x88,0xae,0x62,0x4b,0xc7,0xac,0x50,0x8b, +0xd9,0x25,0x2e,0x11,0xfa,0x24,0x89,0xd5,0x4d,0xd5,0x7c,0xe5, +0x43,0xd5,0xee,0x9d,0x3c,0x88,0xa9,0xbf,0xca,0x18,0xeb,0x7b, +0xa2,0xb9,0xe0,0x63,0x25,0x7c,0xeb,0x01,0xc0,0x07,0x03,0xff, +0xb4,0x3b,0xc1,0xcb,0xde,0xdb,0x11,0x5a,0x33,0xca,0x8c,0xff, +0x7d,0x35,0xeb,0xff,0x82,0xcc,0xcb,0xb6,0xd2,0x32,0xf4,0x8b, +0xcb,0x3b,0x54,0x55,0x92,0x47,0xf8,0x69,0xa3,0x3d,0x7e,0xc2, +0x76,0xcd,0xf4,0xbb,0x33,0xb4,0x41,0xd1,0xff,0xe5,0x84,0x22, +0x4f,0xc6,0x5d,0xbc,0x77,0xd7,0x94,0x83,0x78,0x2b,0x9a,0xa3, +0x7c,0xfb,0x3d,0x3d,0xb3,0x8a,0xf0,0x53,0x81,0x78,0x31,0x9c, +0xf8,0x13,0x34,0xa6,0xd2,0x09,0x2b,0x3e,0x30,0x5f,0xf6,0xce, +0x15,0xfc,0x23,0x33,0x5d,0xb8,0x03,0x82,0x89,0xb4,0xb7,0xd8, +0xb0,0x5b,0x57,0x8e,0x36,0xe4,0xb4,0x3e,0x0f,0x25,0x45,0x7c, +0x2b,0x3c,0xd7,0x36,0xc1,0x25,0xba,0x4b,0x6f,0x60,0xdb,0xd9, +0xb2,0x37,0x78,0x5b,0x43,0xf6,0x45,0xbb,0x75,0xe3,0x29,0xc8, +0x6d,0x23,0x93,0xd5,0x4d,0xec,0xb6,0x63,0xc2,0xe9,0xf9,0x74, +0x88,0x18,0xa1,0xf5,0x18,0x27,0x35,0x6e,0xeb,0x4b,0x32,0x4e, +0xcf,0xd5,0x76,0x78,0xe8,0x66,0xc7,0xbb,0xee,0x73,0x7e,0x79, +0x19,0x00,0x35,0x48,0x53,0x0b,0xd3,0xab,0xd9,0x78,0x98,0x83, +0x75,0x7b,0x7b,0x70,0x40,0xc9,0x9c,0xe9,0xf6,0x96,0x41,0xd3, +0x9a,0xa6,0xcb,0x93,0xd4,0x42,0xd6,0x49,0x59,0xbb,0x3b,0x5b, +0x9a,0xc3,0xda,0xa7,0xaf,0x5f,0xbf,0x14,0x73,0x03,0xe1,0x0d, +0x3d,0xad,0x13,0xc3,0xd9,0xaa,0x3f,0x19,0xfd,0xfb,0x6a,0xb6, +0x24,0x3d,0x00,0xb4,0x82,0x44,0x6f,0xc2,0xdc,0x98,0xd5,0xf4, +0x57,0x1b,0x72,0x47,0x4d,0xae,0x30,0x60,0x17,0x96,0x46,0x85, +0xcb,0x5f,0xdd,0x20,0x88,0x9c,0x84,0xaa,0x4c,0xc1,0x7c,0x01, +0xf2,0x7c,0xba,0x1c,0x2f,0x71,0xe2,0x32,0xc7,0xb1,0xc2,0x25, +0xec,0x98,0xe2,0xf0,0xd0,0x0b,0xea,0x86,0x8b,0x3d,0x69,0xe6, +0x8f,0xe0,0x78,0x4b,0xa8,0xec,0x30,0x93,0xe6,0xf5,0xf9,0x2c, +0x43,0x6e,0xbe,0x0f,0x37,0x11,0xdb,0x60,0x7e,0xe9,0x4f,0xb9, +0xf4,0x6b,0x43,0xd4,0x71,0xda,0x90,0x6e,0x3e,0x81,0x10,0x2c, +0x54,0x52,0x06,0xd3,0xaa,0x55,0xba,0x3a,0xe2,0x5c,0xfc,0xd2, +0x75,0x1c,0x82,0x18,0x1f,0xa7,0x90,0xde,0x1e,0x43,0x5e,0xdd, +0x36,0x90,0x8b,0x6d,0x43,0xf1,0x08,0x8c,0xc1,0x93,0x5d,0x7b, +0xa7,0x30,0x96,0xe3,0x85,0x49,0x7b,0x37,0x81,0xf6,0xf2,0x29, +0x17,0xb3,0x29,0xbe,0x95,0xa7,0xdb,0x1d,0x1c,0x6e,0x34,0x7c, +0x2d,0x3b,0xe6,0x8f,0x2e,0x1e,0x70,0x2d,0x3e,0x7a,0xdd,0x6d, +0x9b,0x0a,0xd3,0xa8,0xa3,0x0a,0x70,0x86,0xf6,0xc7,0x0f,0x79, +0x3d,0x82,0x31,0x50,0x38,0x80,0x62,0x2c,0x4e,0x08,0x14,0xeb, +0x92,0xd4,0xca,0xf1,0x93,0x37,0x68,0x7f,0x18,0xf5,0x2f,0xb2, +0x72,0xa9,0xe2,0x96,0xae,0x94,0x4e,0x7e,0x3e,0xee,0x96,0x8f, +0x61,0xfa,0x19,0xbb,0x47,0x64,0xbd,0x89,0x58,0x70,0x36,0x17, +0x9e,0x78,0x4a,0xba,0xe4,0xbb,0x03,0xae,0xe4,0xd9,0x77,0xdf, +0x7c,0xf3,0xfc,0xdb,0xd7,0x9a,0xe5,0x4b,0xde,0x45,0xff,0xb3, +0x2f,0x9e,0xbe,0x7e,0xea,0x25,0x19,0x36,0xa3,0x36,0x5a,0xea, +0x53,0x30,0x0a,0x78,0x93,0xaa,0x46,0xd3,0x9e,0xac,0x62,0xbb, +0x7b,0x7a,0xd0,0x6f,0xc9,0xf1,0x10,0x8f,0x48,0x40,0x1d,0xf5, +0xdd,0x59,0x54,0x7a,0x72,0x50,0xad,0x96,0xe2,0x8e,0x18,0x8e, +0xf3,0xd3,0xaa,0xd5,0x63,0x40,0x88,0x82,0xc4,0x5c,0x91,0x41, +0x66,0x76,0xc9,0x47,0x24,0x41,0x04,0x4d,0xa7,0x72,0x2f,0x8d, +0xc2,0xe2,0x43,0xf1,0x10,0x86,0x3a,0xc6,0x9a,0xf8,0x36,0x95, +0x4d,0x11,0x1c,0xc5,0x4a,0xba,0x13,0x9f,0x4c,0x7c,0xfe,0xfc, +0xab,0xaf,0xbf,0xfd,0xea,0xf4,0xf9,0xb7,0x5f,0x9c,0xbe,0x7e, +0xfa,0xd5,0x73,0x21,0x0d,0x6b,0x57,0x61,0x71,0xc0,0x94,0x4f, +0x6a,0x09,0x16,0x1f,0x2d,0x4a,0x66,0x6d,0xc3,0x44,0xe9,0x73, +0xe0,0xa2,0x3a,0xd9,0x76,0x46,0xa3,0x6b,0x9e,0xe0,0x9a,0x08, +0xdd,0x25,0x3e,0x34,0x97,0xac,0x79,0x44,0x9d,0x8c,0x0e,0x8a, +0xe6,0x34,0xd9,0xc1,0xf1,0xd5,0xeb,0xa7,0x2f,0x5f,0xff,0x13, +0xb1,0x0c,0xe1,0x29,0x6f,0xc5,0x72,0x31,0xe5,0x4f,0x18,0x36, +0x05,0x1d,0xfc,0x04,0x16,0xd3,0xff,0xc1,0x77,0x62,0xc0,0xb6, +0x0d,0x36,0x11,0xd1,0xc1,0x3c,0xad,0x00,0xa3,0x86,0xf4,0x24, +0x0d,0x42,0x93,0xb3,0x54,0x0a,0xf3,0x89,0x47,0xb8,0x0f,0x67, +0x62,0x7c,0x45,0x41,0xb0,0x49,0xc4,0xd7,0x4a,0xd8,0x19,0xec, +0xf8,0x96,0xe3,0x6b,0x57,0x09,0x5b,0x3f,0x32,0x7b,0x7d,0x01, +0x82,0x84,0x24,0x2f,0x1e,0xbc,0xeb,0x65,0x0f,0x60,0xd6,0xed, +0x4f,0x48,0xd9,0x9a,0xf2,0x3c,0x58,0x42,0x8e,0xb3,0x19,0xfa, +0x4b,0x03,0xcc,0x1e,0xd0,0x32,0x8c,0x06,0x6a,0xa8,0x79,0xa8, +0x84,0x5d,0x22,0xae,0x64,0xc1,0x94,0x08,0xb3,0x75,0xc7,0x50, +0xec,0x3e,0xd9,0x29,0x58,0x11,0x8c,0xbb,0xcc,0xf1,0x70,0x38, +0x9a,0x7e,0xbf,0x18,0x01,0x59,0xa7,0xa3,0xc5,0xef,0x5f,0x7f, +0xf3,0x42,0xb4,0xd0,0x8d,0xa2,0xca,0x13,0x54,0x50,0x39,0x9c, +0x2c,0xdb,0x38,0xbc,0xfd,0xec,0xaf,0xe9,0x16,0xd8,0x46,0x22, +0x31,0xe4,0x40,0x06,0x17,0xa6,0xf0,0xc1,0xda,0x0e,0x1c,0x82, +0xfa,0x0e,0xa9,0xbe,0xde,0xe5,0xbc,0xed,0x4c,0x35,0xdf,0x7e, +0xf7,0xed,0xe9,0xd3,0x17,0xdf,0xff,0xfe,0xe9,0xb7,0x3f,0x7c, +0xf3,0xfc,0xe5,0xd7,0xcf,0x72,0x6b,0xb0,0x07,0xb2,0x74,0xf8, +0x6f,0xe2,0x63,0x84,0x3a,0xed,0x19,0xd4,0x4d,0x6a,0x71,0x95, +0x52,0xbb,0xe4,0xea,0xe0,0x38,0x4d,0xb3,0x91,0xc7,0x14,0x79, +0x8e,0x91,0xbe,0xd0,0xf5,0x4a,0xfa,0xeb,0x8f,0x8b,0x31,0x9e, +0x24,0xf6,0x57,0x67,0xb2,0x04,0x9e,0x4f,0x41,0x6c,0x26,0x3b, +0x08,0xab,0xa5,0x59,0xfd,0x50,0x51,0x18,0xf3,0x28,0xf8,0x47, +0xe2,0x85,0xf1,0xf7,0x4b,0x6b,0xae,0x15,0x52,0x71,0x49,0x75, +0x17,0xde,0x1b,0x88,0x28,0x5e,0x43,0xd1,0xcb,0x3c,0xd5,0x95, +0x5f,0x2c,0x20,0x95,0x76,0x71,0x8c,0x0a,0xe5,0xe5,0xef,0xdb, +0xdb,0xe4,0x00,0x7b,0xd4,0x17,0x85,0x50,0xc5,0x0e,0x90,0xa5, +0x31,0x84,0x58,0x33,0xf4,0xe0,0xf0,0x88,0xdd,0xa6,0x17,0xde, +0x13,0xd3,0x36,0x0e,0xef,0x41,0xf2,0x78,0x92,0xa5,0x12,0x5d, +0xe7,0x53,0x92,0x4c,0x26,0xac,0xd4,0xb7,0x5a,0x8c,0x9d,0x30, +0xcc,0xe5,0x07,0x30,0x2b,0x1f,0x70,0x37,0xf1,0x3c,0xf3,0xc3, +0xcb,0xaf,0xf5,0x0c,0xa3,0x11,0x7c,0x20,0x08,0x92,0x4d,0x0d, +0xfc,0xf8,0x38,0x2d,0x7d,0xcc,0x5f,0x85,0x4c,0x25,0x99,0x20, +0x0f,0x4a,0x3d,0xf8,0x4d,0x54,0xed,0x94,0xea,0xc7,0xa5,0x56, +0x09,0x56,0x04,0xd8,0x26,0x80,0x2c,0xe0,0x91,0x7f,0x2f,0xc9, +0xef,0x47,0xc6,0xba,0x4b,0x47,0x8a,0xc2,0xaa,0x50,0x54,0x4a, +0x53,0xdd,0x67,0x91,0x61,0x14,0xc0,0x81,0x26,0x13,0x47,0xd9, +0x58,0xa6,0x39,0xa9,0xac,0xa0,0x79,0x9c,0x23,0x96,0xd3,0xc2, +0x60,0x6e,0x29,0x7a,0x1d,0xab,0x73,0xa0,0xab,0x0a,0xfd,0x6d, +0xc5,0x15,0x9a,0x3c,0xa8,0x1f,0x81,0x3f,0x9d,0xfb,0xb5,0xdc, +0xee,0xa2,0x98,0xf1,0x63,0x62,0x6b,0x73,0x3d,0xa5,0xc2,0x19, +0x3d,0xad,0xff,0xfc,0x84,0x5d,0x24,0xbc,0x6b,0x55,0xbb,0x95, +0x38,0x62,0x3f,0x08,0x12,0xa4,0x40,0x39,0xc5,0x0c,0x9d,0x16, +0xea,0xd6,0x9d,0xfc,0x5c,0xea,0x96,0x4b,0xe4,0x72,0xe1,0xe3, +0x93,0x9f,0x3f,0xee,0x96,0x3f,0x8e,0x6f,0x41,0x66,0x79,0x93, +0x41,0xd1,0xb8,0x13,0x97,0xd9,0xdf,0x73,0x9d,0xfc,0x3e,0x1f, +0xd7,0x95,0xbf,0xca,0x49,0x5d,0x20,0xf9,0x38,0x15,0xb2,0xc4, +0x53,0x57,0x8e,0xc0,0x9e,0xd6,0x2d,0x3a,0x21,0x02,0x88,0x1f, +0x22,0x01,0x55,0x09,0x1a,0x1c,0xf3,0x31,0xc6,0x00,0x32,0x91, +0xc1,0x86,0x34,0x19,0xc3,0xf5,0x13,0x51,0xa8,0xab,0xe2,0xe5, +0xdf,0xa0,0x57,0x57,0xbe,0x34,0x95,0xd6,0x51,0x7c,0x21,0x6f, +0x10,0x20,0xab,0x00,0x54,0x57,0x9e,0xc2,0xc4,0x37,0x27,0x14, +0xf3,0xe6,0x84,0xf2,0x74,0xbb,0x98,0xc7,0x0e,0x11,0x7c,0xa2, +0x18,0x9d,0x2d,0xe7,0xb7,0xa8,0x2c,0x90,0x75,0xe2,0x16,0x10, +0xa0,0x7e,0x7b,0xd9,0x1b,0x4f,0x96,0xb3,0xd6,0xed,0x72,0x34, +0x69,0xdd,0xfe,0x5b,0x5c,0x1f,0xab,0x1d,0x53,0x27,0x52,0xe3, +0xe7,0x37,0xff,0x56,0xfd,0xff,0xdd,0x3e,0xb8,0x3d,0x20,0x2d, +0x44,0x77,0xbf,0x94,0xea,0x4d,0x6d,0xa9,0xb7,0x18,0xf5,0x54, +0x7f,0x81,0x06,0xa7,0xd4,0xc5,0x42,0x8d,0x2f,0xcf,0xd5,0xbb, +0xfe,0xa2,0x14,0x2b,0x7f,0xd3,0xf6,0xb9,0xbb,0x4f,0xb4,0xa5, +0xa1,0x14,0x59,0x6c,0x51,0xc3,0xa1,0x1a,0x2e,0xd5,0x04,0xb6, +0x0b,0x6a,0xd9,0x9f,0x0d,0x37,0x6a,0x39,0x54,0xcb,0xb3,0xd9, +0x6c,0xa9,0x96,0x17,0xf0,0xff,0xa8,0x07,0xc1,0x3c,0xd4,0xaf, +0xbd,0x3d,0xa5,0x05,0xbb,0x98,0xab,0xc5,0x32,0x97,0xdb,0xe4, +0xf3,0x0f,0xc7,0xf1,0xea,0x7b,0x1f,0xd8,0x7d,0x2d,0x09,0x36, +0xc0,0x77,0x42,0xf6,0x0a,0x2b,0x4b,0xc4,0xe1,0x10,0x8d,0xcd, +0x29,0x34,0xfb,0x36,0x98,0xc0,0x46,0x2b,0x43,0x27,0x73,0x04, +0x9a,0x36,0x73,0x6a,0xd0,0x63,0x23,0x30,0x03,0xba,0xa5,0xc2, +0x4b,0x61,0x35,0x1c,0xc3,0xef,0xf8,0x4a,0x0d,0x27,0x8a,0xde, +0x67,0x8d,0xf0,0x47,0xe7,0x43,0xca,0x41,0xbe,0x8b,0x44,0x5d, +0x34,0xd5,0xc5,0x91,0xba,0x78,0xa8,0x2e,0x1e,0xa9,0x8b,0xc7, +0x8a,0x9f,0x20,0xa9,0x0b,0xa6,0x3a,0x76,0xd8,0x14,0xf1,0x98, +0x2b,0xc0,0x68,0x05,0x7b,0xd4,0x2b,0x05,0xfd,0x38,0x07,0x60, +0xec,0xa2,0x46,0x65,0x6c,0xd2,0x08,0x66,0x2a,0x90,0x3e,0xd4, +0x6a,0x82,0xb3,0x96,0xbf,0x93,0xff,0x50,0x6a,0xda,0x46,0xab, +0x5e,0x1f,0x18,0xa7,0x37,0x58,0xcc,0xa6,0x9b,0x4b,0xd5,0x57, +0xfd,0xe1,0x18,0xfe,0xcd,0x54,0x7f,0x7c,0x4e,0x1c,0x35,0xc6, +0x86,0xa3,0x1b,0x68,0x6a,0xee,0xd9,0x54,0x8d,0x2e,0xa1,0x61, +0x78,0x67,0x4f,0x6c,0x86,0x98,0xbf,0xed,0x0f,0xc5,0xb2,0x37, +0xb5,0xa1,0xb7,0x78,0xab,0x7e,0x55,0x8b,0x55,0x7f,0xa3,0x88, +0x05,0x54,0xa6,0x32,0x90,0x14,0x54,0x76,0x89,0x7b,0x99,0x6c, +0xde,0x9b,0x2a,0x14,0xe5,0xde,0x8e,0xf0,0x67,0x36,0x3d,0x57, +0x20,0xdb,0xc1,0x3f,0xe0,0xb9,0x31,0x6c,0x6f,0x97,0x4b,0xb5, +0x52,0x30,0xe5,0x61,0x1b,0x83,0x05,0xd4,0xb2,0x96,0xa6,0x0b, +0xda,0x2e,0x60,0xc3,0x78,0xce,0xa8,0xc8,0x93,0xc2,0x1d,0x34, +0x3e,0xaf,0x04,0x64,0xdc,0xc1,0xab,0xa8,0x72,0xcf,0xcb,0xa2, +0x45,0xa1,0x0f,0x9b,0x1f,0xec,0x3f,0x7c,0x16,0x83,0x34,0xba, +0x58,0x8c,0xce,0xd4,0x04,0x9a,0x33,0x1c,0x65,0x03,0x95,0x2d, +0x06,0x6a,0x95,0x8d,0x80,0x1e,0x1a,0x3d,0x2e,0x9e,0xc7,0x4d, +0x43,0x76,0xba,0x84,0x3a,0x64,0x02,0xcb,0x0c,0xfc,0x5d,0xaa, +0xde,0x7a,0x0c,0x48,0x9f,0xc3,0x20,0x9d,0x2d,0x54,0x7f,0x46, +0x4e,0x82,0x07,0xa3,0xc9,0x64,0x8e,0x6f,0x91,0x80,0x7a,0xf8, +0x0d,0x44,0x1d,0xd0,0x37,0xda,0xcd,0x53,0x78,0x53,0x46,0x73, +0xc1,0x8c,0xfe,0x66,0xf4,0x07,0xc9,0x8e,0xb7,0x1e,0xbd,0xc1, +0x12,0x4f,0x17,0x17,0xc3,0x8c,0x98,0xf7,0x0c,0xc4,0x29,0xe1, +0xc7,0x0c,0x7e,0xc7,0xe7,0x17,0x4b,0x6a,0xca,0x04,0x30,0x55, +0x17,0x08,0x77,0xa4,0xc6,0x19,0xf6,0x2b,0xc5,0xe0,0x9f,0x55, +0xef,0x7c,0xa4,0xa6,0x33,0x6a,0xf0,0x74,0xf6,0x6e,0xd1,0x43, +0x3b,0x36,0x13,0xf8,0x77,0xa5,0xf0,0x58,0x95,0xfe,0x60,0x6d, +0x8b,0x15,0xba,0x28,0x14,0xa3,0xf5,0x74,0xf3,0x82,0x28,0x66, +0x17,0x3d,0x0c,0x33,0x17,0xf4,0x90,0x3b,0x56,0x97,0x97,0x78, +0xd4,0x03,0x81,0xf3,0x11,0xcc,0x34,0xe3,0x25,0xf0,0x38,0xaa, +0x8b,0x12,0xdd,0x80,0x08,0x2c,0xd2,0x5c,0x31,0x2e,0xef,0xc6, +0xc3,0xe5,0x05,0xf2,0x86,0x11,0x68,0xd3,0x5d,0x86,0xa8,0x60, +0x00,0x15,0x1a,0x9d,0xd0,0x2b,0x65,0x49,0x9d,0x74,0x63,0xb6, +0x4f,0xe2,0xae,0xc4,0x76,0x21,0xb6,0xc2,0x56,0x41,0x61,0xb4, +0xff,0x31,0xc1,0x95,0xb6,0x84,0x37,0xa1,0xbe,0x41,0x14,0x5c, +0xb1,0x5f,0x7c,0xfd,0xed,0xff,0xfe,0xe9,0xf4,0x87,0x97,0x2f, +0xec,0x54,0x9e,0x5f,0x08,0x22,0x59,0x09,0xe2,0xce,0xc9,0xd3, +0xea,0x9f,0x7a,0xd5,0xeb,0x46,0xf5,0xb3,0xda,0xe9,0xef,0x2a, +0xb0,0xfc,0xfd,0xcf,0xf8,0xcd,0xab,0x32,0x4c,0xfe,0xd9,0x9b, +0xda,0x9b,0xf6,0x1b,0xf5,0x26,0x7a,0x13,0xbf,0xb9,0x79,0xb3, +0x7d,0xf3,0xe4,0xcd,0x71,0xb7,0xae,0xbe,0x79,0xfa,0xf5,0x8b, +0xd7,0xdf,0xd9,0x45,0x46,0xe0,0xd4,0xdb,0xa1,0x6e,0x03,0x4a, +0xfb,0x42,0x5a,0x3a,0xd7,0x38,0xa0,0xbd,0x90,0x73,0x94,0xd0, +0x36,0x86,0x3c,0xf1,0x5d,0x09,0x8c,0xea,0x45,0xef,0x1d,0x1d, +0x2f,0xf0,0x86,0xfa,0xa4,0xab,0xde,0x91,0x44,0x91,0x16,0x08, +0x19,0x24,0x9f,0xe0,0xc3,0x58,0x7c,0x09,0xc4,0x77,0x01,0x74, +0xf4,0x92,0xbf,0x75,0x11,0x0c,0x50,0x75,0xca,0xe4,0xae,0x71, +0x64,0x2a,0x69,0x6d,0xde,0xd1,0x42,0xf5,0x22,0x68,0x86,0x14, +0x8c,0xdb,0x31,0xbe,0x64,0xd1,0x1b,0x55,0xa5,0xed,0x5f,0xa6, +0xc6,0x55,0x3e,0x3f,0x8c,0x4a,0x4b,0x42,0x0d,0x7a,0xb7,0x83, +0xf6,0xc5,0x58,0x49,0x8a,0x4f,0x8e,0xb8,0x35,0xb2,0x4f,0xc4, +0xca,0x8c,0x8e,0xfb,0x38,0x76,0xdb,0x52,0x43,0xfe,0xc6,0x67, +0x33,0xba,0x04,0x9f,0xc2,0x95,0x7a,0x25,0x27,0x53,0xec,0x83, +0xcb,0xed,0xa1,0xbd,0x6e,0xc2,0xb7,0x3b,0xa6,0x00,0x8e,0x7e, +0x80,0x15,0x13,0xb9,0x2d,0x1a,0xb4,0xc1,0xad,0x84,0x9b,0x73, +0xdd,0xab,0x21,0xee,0x7c,0xe8,0x61,0x25,0xbc,0xed,0xee,0xe3, +0xf2,0xe1,0x91,0x7b,0x0a,0x66,0xf7,0x9d,0x47,0xa7,0xd3,0xd5, +0x65,0x1f,0xfa,0x72,0x6d,0x6d,0x8f,0xd1,0x2b,0xac,0x35,0x88, +0xfd,0xe3,0xec,0xdb,0xde,0xb7,0x90,0xb2,0x75,0xf3,0x5f,0x8f, +0xc5,0xba,0x64,0x34,0x34,0x45,0x86,0x82,0xa9,0x97,0x71,0x81, +0x57,0xea,0xa7,0xa8,0x3b,0x76,0x3e,0x5a,0xbc,0x1a,0xf4,0x60, +0x00,0xad,0xad,0x09,0xbc,0xb7,0x69,0xd2,0x5e,0x97,0xdf,0xfe, +0x2e,0x69,0xc7,0x6f,0xcb,0x69,0x62,0x8e,0xd1,0xdf,0x7a,0x30, +0x68,0x42,0x8b,0x48,0x5f,0xde,0xa1,0xfa,0xcd,0x72,0xb1,0xb1, +0x80,0xd8,0xed,0xb7,0x93,0xfc,0x1d,0x3d,0x55,0xae,0xf1,0x05, +0x93,0x79,0xa7,0x46,0x0a,0xb0,0xe6,0x75,0x0f,0x3d,0x6b,0xe1, +0x07,0xc9,0x2d,0x5b,0x94,0x8d,0x06,0xc2,0x22,0x7c,0x39,0x5a, +0xe0,0x5a,0xdb,0x3a,0x48,0x80,0xa6,0x03,0x62,0x47,0xd4,0x66, +0xf1,0x40,0xa4,0xb6,0xdc,0xd6,0x43,0x1a,0x67,0xf1,0xf8,0xc6, +0x8b,0x7a,0x35,0x5a,0x86,0x51,0x8b,0x11,0xed,0x3e,0x65,0xda, +0x13,0x23,0x05,0x62,0xaf,0x22,0xa7,0xa2,0xe4,0x9e,0x40,0x73, +0x9e,0x1a,0x5f,0x29,0x49,0x39,0x63,0x1a,0x21,0xf6,0x2e,0x72, +0x60,0xff,0xc6,0x19,0x3a,0x5c,0x8d,0x5c,0xe0,0x78,0x78,0x5c, +0x01,0x2b,0xce,0x16,0xaf,0x36,0x97,0xfd,0xd9,0x24,0xe2,0x57, +0xde,0x62,0x6a,0x03,0x75,0x9a,0xe5,0xa2,0x92,0xe3,0xcd,0xab, +0x00,0x3c,0xf3,0xc5,0x3f,0x29,0xfe,0xb1,0x2f,0xda,0x6a,0xcb, +0xd9,0x0f,0x73,0x20,0xc9,0x33,0xdc,0x98,0xc5,0xf4,0x90,0xd8, +0xe1,0xeb,0x24,0x6e,0xbb,0x9e,0x51,0xa7,0xa9,0xa9,0x1c,0x66, +0xf1,0xb3,0xf1,0xda,0x9a,0x9d,0xf7,0x5e,0x15,0xcc,0x29,0x91, +0x0e,0xad,0x73,0x05,0x4e,0xc6,0x5d,0xaa,0x85,0x4e,0x9e,0x4d, +0xbe,0x3c,0xc2,0x36,0xcd,0x6f,0x3b,0x6a,0x43,0x84,0xfd,0x82, +0x16,0x92,0xa8,0xc3,0x77,0xc4,0x9f,0xa2,0x6b,0xeb,0x65,0xa4, +0x83,0xae,0x39,0x1a,0x4a,0xb0,0x2c,0x3e,0x51,0xd7,0xe9,0x64, +0x9c,0xc1,0x4a,0x8f,0x8b,0xeb,0x38,0xad,0x26,0xd0,0xe8,0x6b, +0xdd,0x48,0x68,0xe0,0x93,0x69,0x3b,0x8e,0x26,0xe9,0x35,0x9a, +0x2d,0x9c,0x4d,0x61,0x76,0x9c,0x48,0xaf,0xd2,0x2b,0xd5,0x7c, +0x9f,0xea,0x3a,0xf9,0x86,0x56,0x43,0xa6,0xa3,0x6f,0x09,0x7c, +0xce,0x16,0x85,0xf1,0xd8,0x99,0x19,0x51,0x17,0x25,0xdc,0x6a, +0xae,0xd7,0x2c,0xba,0x0a,0xd1,0xe5,0xf4,0x3d,0xc1,0x24,0xf5, +0x21,0x91,0x7e,0x33,0x71,0x83,0x31,0xe8,0x12,0x18,0xe1,0x78, +0xd2,0xec,0xa0,0x2d,0xf4,0x1a,0x1a,0xfc,0xc6,0x37,0x1b,0xf8, +0xc5,0x36,0xde,0x2d,0x82,0xe6,0xab,0x96,0x4d,0xc6,0x30,0x2d, +0xc2,0x5c,0xeb,0xc4,0xe9,0xb3,0xc2,0x49,0x1c,0xa3,0x2e,0x12, +0x8c,0xb6,0x28,0x2c,0x30,0xae,0xe0,0x31,0x63,0x80,0x9a,0xd8, +0x01,0x9e,0x8a,0x89,0x79,0x49,0x04,0x5c,0x4c,0x61,0xba,0x30, +0x0a,0x4a,0x65,0xd2,0x20,0x75,0x03,0x18,0xeb,0x34,0x7c,0xe4, +0x67,0x3a,0x74,0xab,0x88,0x58,0x5e,0xe7,0x53,0x0c,0x70,0x1d, +0xfe,0xc8,0xa3,0x6f,0xe8,0xe8,0xe1,0x51,0x8d,0xe9,0x3a,0x0f, +0x52,0xf2,0x65,0x5f,0xb1,0xa5,0x03,0xcb,0x1c,0x99,0x42,0x7e, +0xe6,0xf2,0xed,0x2c,0x05,0xc4,0x28,0xc7,0x73,0x0a,0xc7,0xa3, +0x34,0x33,0x3d,0x97,0x07,0xf6,0x85,0x66,0x51,0xbd,0xb0,0x6b, +0xa0,0xba,0x0d,0x9a,0x01,0x74,0x58,0xf1,0x80,0x0b,0xfb,0x4e, +0xf3,0xa0,0xce,0x76,0x62,0x32,0xd0,0xdd,0xe6,0x1e,0xa6,0x0f, +0x39,0xb2,0x36,0x3b,0xb3,0x9c,0x05,0xdc,0x7b,0x6d,0xb8,0xf7, +0x3a,0x3f,0x99,0x8d,0x12,0x9e,0xc1,0xc9,0x12,0x6a,0x23,0x1d, +0x25,0x6e,0xdb,0x2d,0x51,0x20,0xde,0x93,0x10,0x94,0x4e,0x81, +0x12,0xa6,0xb7,0x4e,0x30,0x17,0x4c,0xc6,0x5d,0x3b,0x72,0xdc, +0xba,0xb7,0x67,0xa8,0x3c,0x3a,0xd9,0xdc,0xd8,0xb2,0x8d,0xed, +0x76,0x6b,0x8a,0x7b,0xb4,0x65,0x3b,0x76,0x88,0x21,0x6d,0xcc, +0xec,0x1b,0x33,0x4c,0x5b,0xf5,0x79,0x2d,0xe2,0x24,0xe5,0x66, +0xff,0x5e,0xaf,0x08,0xe2,0x61,0x23,0x2b,0x86,0x2a,0x5f,0x78, +0x35,0x25,0x1f,0xe6,0x28,0x54,0xe7,0xb7,0x0f,0xa5,0x74,0x96, +0x8e,0xfe,0x68,0xe5,0x2f,0xb1,0x0c,0x70,0x03,0x50,0x8c,0x32, +0xdd,0x51,0xff,0xd3,0xc9,0xe4,0x9f,0x8e,0x82,0x0b,0x73,0x3f, +0x16,0x64,0xe6,0xdf,0xd1,0xd7,0xb1,0xf3,0x27,0xa6,0x7c,0xbb, +0x42,0x9d,0xbf,0x1b,0xf1,0x69,0x88,0x03,0x1c,0x77,0x47,0xe3, +0x3e,0xea,0xed,0xd3,0x48,0xdf,0xe6,0xb2,0x7f,0xfb,0x6a,0x47, +0x01,0x48,0xe0,0x25,0x88,0x76,0x0e,0xf4,0x49,0x0a,0x97,0x01, +0x8c,0x67,0x33,0xb2,0xa7,0xb1,0xd4,0x50,0x60,0x86,0xf0,0xeb, +0xd4,0x06,0x33,0x0a,0x4b,0xd9,0xda,0xdd,0x72,0xbb,0xaa,0x2e, +0x04,0xf5,0xa5,0xbf,0xca,0xaf,0xe5,0x70,0xbf,0x78,0x29,0x60, +0xe7,0x32,0xeb,0xce,0x6e,0x0a,0xb5,0x76,0xb4,0x62,0xbd,0xa3, +0x5a,0x6a,0xc1,0x3f,0x56,0xf1,0x4e,0x4a,0xe7,0x71,0xd9,0x49, +0x99,0xb5,0x79,0xd9,0x37,0xe5,0xb5,0xbe,0x36,0xcd,0x6a,0xa8, +0x4e,0x39,0x3e,0xdb,0x70,0xc3,0x14,0xa3,0xc0,0x2a,0x2a,0xb6, +0x64,0xc7,0x32,0x42,0x4b,0x7f,0xb6,0x0a,0xf8,0xb9,0xb0,0x9c, +0x25,0x42,0xcb,0x0d,0xb6,0x82,0x6c,0xb6,0xb7,0x5b,0x6e,0xd0, +0x97,0x55,0x61,0x87,0xdb,0x9b,0x83,0xe4,0x63,0x27,0x0e,0xd8, +0xe3,0x2c,0xc6,0x97,0x91,0x73,0x71,0xf1,0x26,0xab,0xe0,0xd5, +0xc5,0x83,0x52,0xbc,0x63,0x78,0xd0,0x2c,0x83,0x6f,0x97,0xb8, +0xcd,0x46,0x28,0x77,0x2f,0x92,0x3b,0xad,0x9f,0x6f,0xdf,0x00, +0xa4,0xb8,0x54,0x01,0x32,0xa1,0x5d,0xdd,0x99,0xee,0x7c,0x7a, +0x06,0x8d,0x69,0xc6,0x48,0xc1,0xfe,0x7a,0x8a,0x47,0xa2,0x24, +0x3a,0x23,0xc3,0x4a,0x6e,0xd5,0x44,0x2f,0x1b,0x58,0x14,0x16, +0x0a,0xe6,0x94,0x90,0xb1,0x05,0xc4,0x97,0xde,0x1b,0x07,0x03, +0x42,0xed,0xe7,0xb5,0xa2,0x0a,0x80,0x3f,0xac,0xc8,0x29,0x64, +0xd5,0x36,0x38,0x80,0xa8,0x31,0xec,0x1a,0xe7,0x51,0x51,0x13, +0xad,0xba,0xc8,0x94,0x0b,0xcb,0xe2,0xb7,0x73,0xe2,0x63,0x46, +0x09,0x1a,0xd0,0x0a,0x68,0xb2,0x9f,0xac,0x58,0x67,0x64,0x0d, +0xd6,0x2d,0x48,0x76,0xdd,0xd9,0xc9,0xb9,0x5d,0xfb,0x14,0x0f, +0xdc,0xa4,0x53,0x50,0x19,0x21,0xc5,0x88,0x1a,0x95,0x7a,0x01, +0x22,0x4b,0xec,0x29,0x6b,0x0d,0x50,0x65,0x4e,0x86,0xfe,0xc0, +0x93,0x8a,0x58,0x1b,0x9d,0x0b,0x9f,0xbb,0xf3,0x41,0x89,0x40, +0x95,0xc8,0x59,0x1a,0x1b,0x9f,0xef,0x44,0x8b,0x11,0xdd,0xe5, +0x6b,0x5f,0x03,0x74,0x53,0x98,0x2d,0xa3,0x41,0x8c,0x26,0x8d, +0x86,0x23,0x7f,0x42,0x11,0x00,0xca,0x65,0xfb,0x41,0x05,0xd8, +0xba,0xc2,0xe2,0x58,0xdc,0xba,0x67,0x19,0x33,0x32,0x16,0x23, +0x1a,0x15,0xf1,0xee,0xd5,0x0b,0x0f,0xfb,0x1c,0x76,0x35,0xcf, +0x41,0xdc,0xf7,0xd2,0x98,0xc7,0x5d,0x42,0x28,0x42,0x68,0x62, +0x76,0x82,0xc1,0x3a,0x42,0x79,0x72,0x8b,0x00,0x15,0x04,0xfc, +0xbd,0x52,0x61,0xcd,0x01,0x90,0xbf,0x7f,0x22,0xdf,0x8d,0x67, +0x6b,0x1f,0x3a,0x6b,0x07,0x15,0x67,0xf7,0xae,0xe7,0x48,0x43, +0x8e,0xdd,0xd3,0xa1,0x87,0x77,0xcb,0x23,0xc5,0x8e,0x7e,0x98, +0xe7,0x09,0xe2,0xaa,0xb5,0x49,0xaa,0x74,0x82,0xe8,0xf5,0x61, +0x1b,0x58,0xf7,0x69,0x9b,0xcb,0x19,0x50,0xfe,0xc4,0xbc,0xf4, +0x5e,0x8d,0xf2,0x99,0xff,0x2e,0x0a,0xe7,0x90,0x68,0x39,0x15, +0xad,0x8b,0x08,0xe7,0xb6,0x62,0x37,0xed,0x42,0xa4,0x5a,0x61, +0x93,0x76,0xcd,0x11,0x78,0x4e,0x29,0x5a,0xc0,0x7b,0xe4,0x2f, +0x4c,0xa6,0xe5,0xaa,0x15,0xed,0x5a,0x11,0x63,0x59,0xa2,0x0a, +0x05,0xb3,0xe2,0x73,0xd1,0xdd,0x42,0x51,0xb1,0x78,0xb7,0x13, +0x0a,0xf1,0x25,0x96,0x27,0x48,0x3f,0xbc,0xfc,0x9a,0x4f,0x0d, +0x76,0x0d,0x5f,0x7c,0x5a,0xf2,0x2d,0xcc,0x09,0x62,0x2d,0x4f, +0xae,0xd9,0x4f,0x29,0xfe,0xf4,0xb4,0x45,0xef,0x5e,0x77,0x14, +0x95,0x83,0xd6,0x50,0x64,0x7d,0xb0,0x47,0x1c,0xe5,0xe7,0x98, +0x19,0xb3,0x84,0x29,0xb8,0x73,0x6a,0x99,0x2d,0x96,0x68,0x8b, +0xa0,0xb7,0xa0,0x7e,0x19,0x98,0x4f,0x03,0x38,0xdc,0x39,0x91, +0xf5,0x1b,0x9d,0x0b,0x7b,0xa6,0x97,0x0d,0x46,0x53,0x3c,0x8f, +0x77,0x55,0xb7,0x54,0xdf,0x02,0x38,0x3c,0xec,0x77,0x6c,0x91, +0xa8,0x57,0xd3,0x4d,0x57,0x7d,0xf3,0x19,0xb7,0x0e,0x7a,0xd5, +0x83,0xfe,0x2e,0x34,0x47,0x78,0xdf,0x2f,0xfb,0x0f,0xe8,0xb1, +0x09,0x5e,0x41,0xd8,0x65,0xf5,0x17,0x98,0xb4,0x2f,0xd9,0x2f, +0x9e,0xd9,0xdf,0x5d,0x1e,0xff,0xd2,0xfe,0x45,0xfc,0xbd,0xd0, +0x02,0x88,0xeb,0x0a,0x65,0x91,0x8c,0x27,0xbf,0x74,0x65,0x6f, +0x78,0xee,0xfa,0x73,0x99,0x8a,0x7b,0x05,0x5a,0x88,0x38,0x09, +0x55,0xf2,0x0e,0x0f,0x75,0xb5,0xbc,0x42,0x8d,0xd5,0x2f,0x66, +0xed,0xda,0xbb,0xf3,0xe1,0x67,0x5d,0x26,0x5c,0xb8,0xb1,0x32, +0xa9,0x2a,0x5f,0xd4,0xd9,0x61,0x99,0x94,0x7d,0x55,0xa1,0x5d, +0x9d,0xc5,0xf2,0xf3,0x11,0xb4,0x7b,0x14,0x51,0x8c,0x1c,0x70, +0x34,0xd4,0x2f,0x8d,0xdc,0x6a,0x3b,0xa4,0x86,0xdc,0x04,0x04, +0xa2,0x62,0x40,0x9f,0x1a,0xeb,0xc6,0x87,0x24,0x42,0x92,0xfe, +0x72,0x90,0xfe,0x82,0x4a,0x6b,0xbf,0x34,0xd2,0x5f,0xd4,0xb8, +0x91,0x36,0x62,0x35,0x3e,0x4e,0xc7,0x18,0x05,0x21,0x3c,0xb4, +0x68,0x1f,0x78,0x24,0x6c,0x20,0x0d,0x41,0xb8,0x69,0xa0,0x74, +0xa3,0xf1,0xc0,0x0c,0xbb,0xba,0x9c,0x5e,0xde,0x2e,0x56,0xf3, +0xa5,0x7d,0xe5,0xf7,0x96,0x6c,0x98,0x01,0xcf,0xd0,0x0b,0x48, +0xb2,0x46,0x77,0x7a,0xda,0xc6,0x04,0x04,0x8d,0xbf,0x35,0xb4, +0x6d,0x76,0x35,0xda,0x01,0x12,0x37,0xe8,0x78,0x00,0xaa,0xcf, +0x3e,0xe8,0x5a,0x73,0xb5,0x70,0x27,0xf1,0xd9,0xf4,0xa5,0xf8, +0x58,0xe2,0x4a,0x53,0x3b,0x5f,0xb6,0xf1,0x8c,0xc7,0xd9,0x02, +0xd0,0x9e,0xfd,0x85,0x80,0x12,0xc0,0x6a,0x52,0xfb,0xc8,0xbc, +0x99,0xb5,0x45,0x9d,0xd5,0x72,0x36,0x7d,0x0a,0x22,0x8b,0x86, +0x8e,0x57,0x48,0xe6,0x98,0x06,0x7b,0x9f,0x6c,0x4e,0x5a,0x33, +0xbb,0xa8,0x7c,0x2c,0x18,0xd5,0x90,0x01,0xa9,0x7e,0x36,0xf1, +0x82,0xa2,0x4f,0x01,0x0a,0xce,0xfc,0x4e,0xd8,0xa4,0xba,0x8d, +0x10,0x38,0x35,0x47,0x4f,0x16,0x1f,0xd9,0xce,0xd0,0x92,0x65, +0xef,0x43,0x16,0x23,0xdf,0xc2,0xcf,0x29,0xd2,0xee,0xe4,0xe7, +0x5a,0x37,0x10,0xbc,0x89,0x31,0xd1,0xf0,0x8f,0x3d,0xe0,0xd4, +0xe7,0xa6,0x84,0x29,0x7a,0x77,0xe2,0x53,0x19,0x9c,0x6d,0x2f, +0xc5,0x3d,0x4c,0x5c,0x40,0xdc,0x9d,0x84,0xd5,0x2e,0xbe,0x76, +0x12,0x97,0x35,0x4e,0x68,0x9d,0x28,0x21,0x9e,0xa5,0x0a,0x11, +0x62,0x4c,0x8b,0x89,0x55,0xc7,0x43,0x7b,0x5f,0x48,0xee,0x34, +0x60,0x09,0x5d,0x51,0x7b,0x7c,0x8c,0xdc,0x4b,0xc7,0xdb,0x54, +0xd2,0xd3,0xcf,0x13,0xcf,0x67,0x3c,0x1f,0x87,0x20,0xbe,0xa4, +0x58,0x76,0x01,0x4e,0x14,0x31,0xe3,0x8c,0x12,0x34,0x54,0x0e, +0x15,0x22,0xc1,0x10,0x60,0x0c,0x75,0x74,0x0f,0x75,0x88,0x51, +0x5a,0xba,0xf7,0xcd,0x79,0x5d,0x47,0xce,0x6e,0x5b,0xa6,0xe3, +0x76,0x32,0xbb,0x21,0xa1,0x61,0xb0,0x7d,0x27,0x53,0x62,0xe2, +0xca,0x1e,0x42,0xd9,0x73,0x23,0x5b,0xee,0xa4,0xd1,0xd5,0x63, +0x90,0xe7,0xed,0x36,0x9e,0x66,0x69,0xf8,0x45,0xc2,0x48,0xd1, +0x41,0xd4,0x6c,0xbb,0x6b,0xd0,0x6b,0x4a,0x14,0xa2,0xcc,0x3c, +0xf3,0x01,0x8d,0xcc,0x4d,0x77,0xd2,0x48,0x7d,0xb6,0x86,0x78, +0x2e,0x46,0x13,0x7a,0xfe,0x83,0x22,0x28,0x7d,0xbd,0xa6,0xc4, +0xb6,0x84,0xa0,0xef,0x74,0x8e,0x54,0x0e,0xe3,0x6e,0x6f,0x3f, +0x3d,0x94,0xb8,0x1a,0x2f,0x6b,0xa3,0x2f,0x44,0x52,0xf8,0x7e, +0xc6,0xb3,0x92,0x3e,0x9d,0x84,0x0d,0xc4,0x44,0xc6,0x2d,0x9f, +0xe3,0x85,0x42,0x02,0xd1,0xe3,0x74,0xb8,0xe8,0x9d,0xbf,0x5a, +0xcd,0xc9,0xe1,0x9f,0xf3,0x72,0x2d,0x2d,0xd5,0x30,0x25,0x93, +0x94,0x6a,0xa9,0xf2,0xa0,0x52,0xf1,0x0a,0x7d,0x3d,0x54,0xcb, +0xd9,0x6a,0x70,0x81,0x9c,0x90,0x96,0xcc,0x27,0x6f,0x45,0x64, +0xbd,0xa0,0xcb,0xbb,0xb4,0xe4,0x04,0x24,0x99,0x60,0x73,0xa2, +0xf9,0x94,0xa4,0xc1,0x64,0x0c,0x93,0x6d,0x89,0x7e,0x24,0xea, +0x1d,0x32,0x86,0x9c,0xb9,0x01,0x0e,0x7c,0xc9,0x16,0xa3,0x67, +0x0c,0x53,0xad,0x2a,0x3e,0x2d,0xa6,0x4c,0x4e,0xf5,0xfb,0xb2, +0x19,0x44,0x76,0x65,0x62,0xc5,0x87,0xd4,0x91,0xce,0x44,0x2c, +0xe3,0x7d,0x82,0xb4,0x39,0xa5,0xc0,0x89,0x4f,0x60,0x4a,0x31, +0x8a,0x64,0xbb,0x73,0xa4,0x25,0x74,0x22,0xe1,0xdc,0x6a,0xeb, +0x0e,0x78,0x86,0xe4,0x70,0xd7,0x8a,0xb3,0x33,0xe8,0xad,0x77, +0x35,0x7a,0xf3,0x0d,0x49,0x8a,0x1f,0xa9,0x53,0x04,0xd1,0x8c, +0x5f,0xbe,0xef,0xa9,0x89,0xb1,0x55,0x5e,0x05,0xc0,0x71,0x0e, +0x48,0x47,0xd0,0xdb,0x75,0x16,0xaf,0x08,0x8f,0x2d,0x7a,0x70, +0x80,0xe6,0x2f,0x5f,0x8f,0x2f,0x47,0xa8,0x9f,0x07,0xb1,0xaa, +0x11,0xee,0x29,0x2f,0x67,0xab,0x6c,0xf4,0xfd,0x0c,0xd6,0xd6, +0x48,0x6c,0x26,0xc3,0x80,0x41,0x97,0x51,0x35,0xf6,0x13,0x38, +0x7c,0x8d,0x9d,0x49,0x8f,0xc5,0x46,0x69,0x18,0x09,0x83,0x9f, +0x67,0xc1,0xec,0xea,0x3c,0x35,0xc5,0x6b,0xb3,0x77,0xf0,0xf7, +0xd5,0x1f,0xbe,0x92,0x8e,0xb8,0xbd,0x35,0x49,0x78,0x37,0x05, +0x79,0x45,0x7e,0x86,0x2c,0x54,0xb3,0xdc,0x6d,0xe1,0x67,0x9a, +0x4f,0x8d,0xe8,0x2d,0x45,0xe3,0x58,0x23,0x7b,0xda,0x5f,0x9d, +0x3f,0x7c,0xd8,0xf8,0xf4,0x08,0x50,0x32,0x7c,0x57,0x63,0xa5, +0x8a,0x1f,0x6f,0x6f,0xc3,0xa8,0x9f,0xd0,0x3e,0x3c,0xa0,0x67, +0x99,0xb5,0x84,0xea,0x5c,0xa5,0x58,0x7b,0x20,0x2d,0x41,0x2a, +0x9a,0xe4,0xa0,0x2d,0xf5,0xcd,0x5c,0x06,0x6c,0xab,0xd4,0xeb, +0x67,0xb3,0x09,0x3d,0x99,0x5d,0xc2,0xe4,0x8a,0xee,0x79,0xcf, +0x96,0xf0,0x73,0x09,0xe3,0x76,0x3c,0x6d,0xa1,0x19,0x38,0xd2, +0x39,0x81,0x2f,0x56,0x44,0x69,0x31,0x97,0x6c,0x55,0xc9,0x38, +0x1b,0x29,0xc9,0xa9,0xc3,0xf2,0x12,0xdb,0x05,0xd4,0xc2,0x7b, +0x6c,0x18,0xf5,0xaf,0xa0,0x85,0xa3,0xe9,0xb3,0xd7,0xdf,0x40, +0xdb,0x72,0xcd,0x4a,0x0f,0x22,0x28,0x50,0x3b,0x03,0xb2,0xc1, +0x0f,0x4a,0x79,0x40,0x12,0xe3,0x6a,0xd4,0x0a,0x8b,0x7e,0xa9, +0x4e,0x44,0xf4,0xab,0xad,0xa1,0x93,0xe6,0xbd,0xf3,0xd1,0x8f, +0x8a,0xc3,0x1b,0x09,0xff,0x14,0xb7,0x9c,0x1c,0xc0,0x4a,0xd0, +0x31,0x6e,0x1e,0x8e,0xf9,0x29,0xe6,0xa8,0x94,0x13,0x60,0xa5, +0x5d,0x8c,0xd7,0xaf,0x51,0xb6,0x42,0x87,0x29,0x96,0x43,0x82, +0x46,0xa0,0x2d,0xa5,0x11,0x1a,0xaf,0x8d,0x63,0x75,0x22,0xb5, +0x68,0xd8,0x5d,0xb6,0xf4,0x87,0x63,0xd0,0x2b,0xfe,0x39,0xaa, +0x0b,0x01,0xfd,0x9e,0x51,0xc5,0x2f,0xb1,0x63,0xf4,0xf4,0x7c, +0x62,0x10,0xac,0x62,0xb9,0x1a,0x52,0xbe,0x6a,0x0b,0x73,0xda, +0x0b,0x88,0x54,0x06,0x6f,0xce,0x08,0x1d,0x95,0xcb,0xf7,0x7a, +0x36,0xef,0xfa,0x6b,0xcb,0xf9,0xd4,0xb9,0xe0,0x5f,0x1f,0x37, +0x3a,0x49,0xab,0x71,0xbc,0xee,0x54,0xe1,0xc7,0xcb,0xd8,0x1b, +0xcc,0x32,0x2f,0x67,0xd2,0x69,0xb4,0xaa,0x09,0x64,0xfd,0xed, +0xcf,0xad,0x6f,0xd0,0x08,0xa0,0xe4,0xf0,0x4b,0x65,0xe3,0x69, +0x50,0xea,0xa2,0x37,0x39,0x83,0x22,0x54,0xb4,0x2a,0x01,0x2e, +0xcf,0x79,0x7d,0xf4,0xc6,0xd3,0x0b,0x5b,0x3e,0x8a,0xd6,0x29, +0x65,0x1d,0x81,0xd4,0xb5,0x8e,0xe3,0x6a,0x52,0x5f,0xc7,0xf5, +0x66,0x70,0xd6,0x9a,0xed,0x29,0x51,0x29,0x28,0x01,0xbc,0xb9, +0xab,0x44,0xb3,0x4c,0xb5,0xc4,0xf5,0x68,0x0d,0x22,0xbb,0x57, +0xea,0xa2,0x87,0x9d,0xec,0xb6,0x4e,0x17,0xa4,0xc8,0x7a,0x33, +0x8e,0xcb,0x6b,0xaf,0xc4,0x33,0x54,0xbb,0x0a,0xef,0x8e,0x2f, +0xb2,0x49,0x74,0xa1,0x32,0x35,0xf1,0xce,0x73,0x21,0xe1,0xf7, +0x26,0xc1,0xcb,0x6f,0xa3,0xf9,0x34,0xea,0x22,0xbd,0x60,0x71, +0x37,0x4b,0xd9,0x38,0x76,0x6d,0x92,0x4e,0xc2,0x1a,0x4e,0x17, +0xe7,0x7d,0x5d,0xca,0x24,0x5d,0x45,0xd6,0xda,0xf2,0xc5,0xf1, +0xd1,0xe3,0x46,0xe7,0xa2,0x9a,0xc2,0x0f,0x74,0x3f,0x3d,0x0d, +0xa9,0x60,0x20,0x56,0x8f,0x21,0xd8,0xb9,0x4c,0x2a,0xd1,0x65, +0xb3,0x7a,0x99,0xc4,0xe5,0x8b,0x3a,0x64,0x49,0x3e,0xa5,0xd8, +0x66,0xab,0xf9,0x30,0x48,0x8e,0x20,0xa6,0x7a,0x11,0x63,0xa6, +0xcb,0xc4,0xe2,0x71,0xe5,0xd6,0x46,0x54,0x22,0x05,0xb9,0xa8, +0xf9,0xe8,0x51,0x19,0x93,0xc4,0x6b,0x70,0xa2,0x2e,0x9b,0x46, +0x95,0x39,0x65,0x25,0x93,0x8b,0x18,0x18,0x2d,0xba,0xf8,0x1d, +0xa1,0xf3,0x04,0xb0,0xac,0x20,0x92,0xd0,0x1a,0x49,0xcf,0x30, +0xbd,0x71,0x9c,0xc1,0xdf,0x0c,0xd8,0x2b,0x69,0x41,0x33,0xd3, +0xc6,0xf1,0x04,0xc2,0x13,0x0a,0x4f,0x00,0x68,0x5a,0x7b,0x74, +0x9c,0x4e,0x3a,0x93,0x72,0x94,0x54,0xb2,0xb8,0x35,0xa9,0x64, +0xd5,0x49,0x39,0x53,0x97,0x49,0xda,0x2c,0x4f,0xaa,0x97,0x4d, +0x5c,0x4f,0x91,0x48,0x88,0x67,0x25,0x69,0x42,0xbb,0x09,0x63, +0xfa,0x5b,0xc5,0x70,0xd0,0xf9,0x03,0xec,0x85,0x41,0x41,0xaf, +0x99,0x04,0xbf,0xd7,0x6c,0x7e,0xbf,0xd7,0x06,0xe9,0x60,0x47, +0xaf,0x0d,0x26,0xe8,0xbe,0x38,0xa8,0x45,0x53,0x04,0xfb,0x87, +0xf6,0x94,0x14,0x1e,0x60,0x78,0x80,0x61,0x28,0x88,0x85,0x26, +0x8a,0x48,0x8c,0xe3,0xf1,0xa2,0x9c,0x92,0xc2,0xcd,0x70,0x0c, +0xd3,0x56,0x5c,0x1e,0x28,0xc3,0xa2,0x17,0x10,0xf2,0xb1,0xe4, +0xa2,0xee,0xd9,0x84,0xb4,0xea,0x85,0x49,0xf0,0xf2,0xdb,0xe8, +0x1b,0xdd,0x06,0xd9,0x7a,0xa5,0x3d,0xfe,0xe8,0xa7,0xfd,0xb0, +0x06,0x22,0xb3,0x94,0xc2,0x3e,0xdf,0xa4,0xd1,0xa4,0x92,0x3c, +0x8e,0xeb,0x49,0xf2,0x58,0xad,0xd3,0x4d,0xa5,0x57,0x7f,0xd4, +0x68,0xa8,0xeb,0x74,0x53,0xed,0xd7,0x9b,0x0d,0xb3,0x27,0x5f, +0xa7,0x52,0x7e,0xbd,0xb9,0x86,0x31,0x57,0x96,0xd0,0x8f,0x6a, +0xe3,0x26,0x6c,0x4c,0xc2,0x4f,0x00,0xc2,0x49,0xb8,0x36,0x09, +0x7f,0xd2,0x7d,0x0d,0x3f,0x90,0x42,0x9f,0x47,0x35,0x60,0xdb, +0x87,0x8f,0x1e,0xc2,0x78,0xaf,0x26,0xb5,0x47,0x47,0x9f,0x24, +0x47,0x9f,0x3e,0x2a,0x6f,0xaa,0xb5,0x87,0x9f,0x7d,0xfa,0xe8, +0x28,0x79,0x58,0xbe,0x26,0xda,0xea,0xec,0xd5,0xda,0x67,0x8f, +0x3f,0x6b,0x3e,0x7e,0x5c,0x86,0x59,0xa1,0xf6,0xe9,0x27,0x8f, +0x1b,0x49,0xe3,0xd3,0xf2,0xa6,0x52,0x6b,0x3c,0x4c,0x1e,0x3d, +0x7a,0x1c,0x64,0xae,0x35,0x20,0xee,0xe1,0xd1,0x43,0x00,0x5d, +0x6b,0x36,0x1e,0x36,0x9a,0x8f,0x3e,0x83,0xbc,0x09,0x44,0x7f, +0xd2,0x6c,0x3e,0x6a,0x42,0xee,0x5c,0x2f,0x10,0x7f,0xf9,0x3d, +0x31,0x81,0x99,0x59,0x18,0x8f,0x27,0x4b,0x98,0xb5,0x9a,0x51, +0x5f,0xf5,0xa8,0x5d,0xc3,0xd1,0x39,0x2c,0x3f,0x99,0x74,0xee, +0xaf,0x8b,0x65,0xd4,0x2b,0xf7,0x2a,0xfd,0x72,0x1f,0xb6,0xb5, +0x71,0x4b,0xca,0x35,0xea,0x0d,0x85,0xff,0x26,0xf9,0xfa,0x98, +0xa6,0x76,0x96,0x06,0x44,0x1f,0x7f,0xfa,0xd9,0x51,0xe3,0xe8, +0x61,0x67,0x5d,0x86,0xff,0x5a,0xd1,0xba,0xfa,0xb0,0xde,0xfc, +0x2c,0xae,0x7f,0x52,0xfb,0xe4,0xd3,0x4f,0x1a,0x47,0x9f,0x78, +0x20,0xb0,0xa9,0xc8,0x3c,0x1e,0x88,0x46,0xe3,0xd3,0x4f,0x1f, +0x3d,0xee,0x10,0x4a,0xf3,0xd9,0xbb,0x68,0xad,0x92,0xfa,0x51, +0xdc,0xd2,0x00,0x80,0x76,0x08,0x31,0x07,0x07,0x49,0xb6,0xd8, +0x35,0x55,0x00,0x31,0x1b,0x47,0x8d,0x87,0xc7,0xe9,0xa2,0x93, +0x34,0x6b,0x9f,0x35,0xcb,0x8b,0x16,0x12,0xf2,0x51,0xd9,0xd4, +0xb2,0x80,0x5a,0x9a,0xb5,0x87,0x71,0x15,0xa3,0x03,0xca,0x02, +0xe8,0x6f,0x59,0xa5,0xcd,0x7f,0xc1,0xa4,0xc7,0x3e,0x46,0x1e, +0x1f,0x03,0x1b,0x42,0x55,0x87,0x12,0xfa,0xd4,0x06,0x72,0xc0, +0x3c,0xab,0x14,0x3e,0x30,0xaf,0x9e,0x8a,0xfb,0x96,0x4a,0x2a, +0x5b,0xa8,0xf3,0xfc,0x38,0x7b,0x69,0x12,0xbc,0xfc,0x36,0x5a, +0x2e,0xb3,0x53,0xbe,0x48,0xaf,0x9d,0xa7,0xe7,0xc5,0xe3,0x0c, +0x6a,0x38,0xbd,0x18,0xad,0xa3,0x2b,0x53,0x43,0xf2,0xf8,0xf8, +0xaa,0x53,0x6a,0x94,0x2a,0x44,0xa9,0xcb,0xde,0x1a,0x76,0xed, +0x57,0xa8,0x96,0x25,0x8d,0x80,0x11,0xc8,0xab,0xf0,0x25,0x4c, +0x0d,0xd0,0xe4,0x30,0x31,0x07,0x9e,0xed,0xf3,0xb3,0x0f,0x50, +0x05,0x11,0x0a,0xd6,0x19,0x39,0x22,0xc1,0x59,0x9c,0xdf,0x53, +0x2f,0xd0,0x35,0x35,0xfc,0xeb,0xf3,0x13,0x63,0x98,0x6e,0xeb, +0xd1,0x49,0xaf,0x7a,0xdd,0xad,0xc4,0x6f,0x22,0x7c,0xf5,0xfa, +0x26,0xae,0x8f,0xd9,0xc0,0x2c,0x43,0x8a,0x63,0xf1,0xa4,0x04, +0x53,0xf6,0x65,0x72,0xd2,0xec,0x3a,0x4f,0xef,0x61,0xb6,0x26, +0x57,0xb9,0xf8,0xb8,0xa7,0x04,0xb5,0x95,0x5a,0x7a,0xa5,0x80, +0x35,0x91,0xd0,0xf9,0x72,0x32,0xeb,0x2d,0xa1,0x28,0x4a,0xe2, +0xca,0x8f,0x81,0x82,0x75,0xf4,0x91,0xe9,0xc7,0x36,0x39,0x36, +0x26,0x27,0x46,0x25,0x68,0x85,0x81,0x29,0x13,0x84,0x69,0xa9, +0x74,0xa8,0xc0,0x2e,0x4e,0x49,0x76,0xa6,0x40,0x3d,0x5a,0x58, +0xd5,0x27,0xf6,0x94,0x8d,0x0e,0xc9,0xe9,0x5c,0x44,0x37,0xbf, +0x83,0x15,0xd3,0x69,0xd0,0x82,0x8f,0xe0,0xcf,0xf9,0xa7,0x8f, +0x56,0x5b,0x49,0xfd,0x92,0x73,0x1e,0x1e,0x96,0xfe,0x0d,0xcd, +0x68,0x73,0xc8,0x2a,0xda,0xc1,0x4a,0xf0,0xd0,0x46,0xf3,0x89, +0x64,0x27,0x5a,0x04,0xf9,0x92,0x58,0x2d,0x2a,0xc0,0x45,0xe7, +0x41,0x7c,0x33,0x56,0xe7,0x15,0x60,0xaa,0x7e,0x10,0x7f,0x14, +0xab,0x7e,0x25,0x05,0x1c,0x3e,0x09,0x61,0xe3,0x01,0x80,0x8e, +0x71,0x14,0xfa,0x14,0x94,0x38,0xcf,0xc7,0x1f,0xa9,0x47,0xb1, +0x05,0x6e,0xe3,0x1f,0xa9,0x4f,0x40,0x58,0x5e,0xb0,0x49,0xde, +0xaf,0x61,0x97,0x03,0xc3,0xf8,0x31,0x42,0x30,0x11,0xe7,0x14, +0xd1,0xb7,0x11,0x7d,0x8c,0x88,0x95,0x1d,0x4b,0x79,0x16,0x45, +0xbe,0x90,0x71,0x43,0xbe,0xc1,0x40,0x00,0x02,0xee,0x4e,0x0d, +0x9b,0x2f,0xea,0x29,0x72,0xfa,0x39,0xff,0xf4,0xe9,0x07,0x9f, +0xbb,0x8a,0x1c,0x87,0x83,0x84,0x8b,0xab,0x61,0x0a,0x81,0x2a, +0x94,0x01,0xd1,0x22,0x82,0xcf,0x0a,0x7c,0x82,0x18,0x69,0x74, +0x9d,0x3a,0x51,0x86,0x12,0xc6,0xa4,0x33,0xac,0x9b,0xe4,0x16, +0x7c,0x83,0x4c,0xc4,0xe5,0x62,0x75,0x91,0x2e,0x50,0x1d,0x79, +0xdd,0x89,0xce,0xab,0xfd,0xb8,0x3e,0xac,0x44,0xfd,0xe3,0xf3, +0xce,0xe3,0x56,0x23,0x6e,0x9d,0x4b,0x42,0xbf,0xba,0xc0,0x84, +0x66,0x2b,0x5a,0x54,0xcf,0xf1,0xeb,0xa1,0x82,0xe5,0x1b,0xa4, +0x9e,0x16,0xae,0xf6,0x30,0x73,0x67,0xe9,0x04,0x0f,0xe1,0x12, +0x12,0x6e,0x2e,0x88,0xd9,0x2e,0x8a,0xc5,0x44,0x6c,0x3e,0xce, +0xc8,0x7a,0x9a,0xd1,0x0c,0x87,0x13,0xfd,0x02,0x09,0xeb,0x84, +0xcf,0x91,0xae,0x4e,0xb8,0xcf,0xfb,0x34,0x5a,0x70,0xf5,0xcc, +0x1e,0xd5,0x1e,0x26,0xcd,0x87,0xb0,0x8e,0x95,0x17,0x95,0xda, +0xd1,0xa3,0x4f,0x1e,0x7d,0xf2,0x38,0x29,0x9f,0x57,0x6a,0x20, +0x08,0x3e,0x3c,0xfa,0xe4,0x51,0x19,0x5b,0x24,0x0b,0x72,0xcc, +0x2b,0xb2,0x2d,0xd9,0x4c,0x9a,0x8f,0x3f,0x69,0x7e,0x86,0x25, +0x3f,0x49,0x1e,0x25,0x8f,0x9a,0x4d,0x2c,0xd9,0xf8,0xa4,0x99, +0x78,0x05,0x61,0xb7,0x75,0xed,0x17,0x6c,0x24,0x9f,0x1d,0x1d, +0x1d,0x51,0xc1,0x24,0xf9,0x2c,0xf9,0x8c,0xca,0x7d,0xf6,0x08, +0x57,0x80,0xc4,0x29,0xf8,0x27,0xab,0x73,0xc6,0x32,0x0c,0x48, +0x12,0xb0,0x76,0xc3,0x3c,0x0e,0x82,0x44,0x19,0x56,0xae,0x4d, +0xac,0x9a,0xf8,0xb5,0xa9,0x86,0x6b,0xad,0x25,0x89,0x96,0xe1, +0x99,0x27,0xe2,0x27,0x29,0x2c,0xe6,0x20,0x11,0x74,0x16,0x75, +0x5a,0x68,0x5a,0x66,0x81,0x89,0x00,0x1b,0x5c,0x5a,0xea,0xb4, +0xf0,0x28,0x5c,0x6b,0x8a,0xe7,0x46,0x19,0xfd,0x03,0x66,0xbe, +0xb3,0xd4,0x99,0x79,0x06,0x1a,0xe3,0xd2,0xef,0x70,0x0c,0x0f, +0xf4,0x30,0x1b,0x58,0x43,0xea,0x1d,0x77,0xe5,0xab,0xc1,0x0a, +0x77,0x16,0xb7,0xce,0xbc,0x8a,0xe8,0x1b,0x0f,0x95,0xf7,0x5c, +0x14,0x5e,0x75,0xae,0x0a,0xee,0xee,0xae,0xfc,0x43,0x8f,0xf1, +0x70,0x84,0xaf,0x14,0x37,0xae,0xd6,0xb6,0xbf,0x2c,0x5f,0x2c, +0x5e,0x03,0xbc,0xbc,0x6d,0x5a,0x03,0x1a,0x95,0xe1,0x2f,0xc7, +0x97,0x23,0xcc,0xe6,0x5c,0x40,0x49,0xb6,0x26,0xb4,0x31,0xbc, +0x26,0x83,0xf9,0x2b,0x8f,0xae,0x06,0x81,0xe2,0xac,0x00,0x49, +0x0d,0x58,0xfd,0x21,0x06,0x13,0x19,0x2f,0xbf,0x66,0x8d,0xa0, +0x45,0x61,0x1b,0x36,0xe4,0xae,0x02,0x37,0xce,0xc9,0x3f,0xa6, +0xe9,0xbb,0x08,0xb6,0xe8,0x89,0x0f,0x53,0x96,0x2b,0xf4,0xb1, +0xf0,0xeb,0x0a,0xed,0x8a,0x70,0x90,0x0c,0xa3,0xf3,0xe7,0xe1, +0xa1,0x4e,0xd2,0xa0,0xf9,0x91,0x35,0xa7,0x1e,0xa7,0xc0,0x87, +0x87,0x87,0x47,0x0d,0xd8,0xae,0x50,0xc4,0xed,0x2d,0xf0,0x32, +0xaa,0x62,0x53,0x88,0x75,0x1f,0xc5,0x66,0xaa,0xb1,0xfb,0x4b, +0xe7,0xa3,0x80,0xb9,0x12,0xc8,0x8e,0xd6,0x79,0xa8,0x6a,0x49, +0xc6,0x6d,0x6d,0x01,0xf4,0x91,0x35,0x1b,0x0f,0x1f,0x34,0xb6, +0x26,0x07,0xda,0xb9,0x73,0x21,0x62,0x55,0x62,0x45,0x6f,0x17, +0x14,0x53,0x2d,0x4d,0x0a,0x17,0x0b,0xb4,0x8c,0x63,0xb4,0x49, +0x87,0x47,0x35,0xa3,0x15,0x5d,0xea,0xd3,0xad,0x5a,0x36,0x9a, +0x0e,0x61,0xb9,0x9e,0x2f,0x66,0xe7,0x0b,0x32,0x99,0x57,0xc2, +0x4a,0xe1,0x87,0xe0,0xe2,0x2b,0x74,0x7e,0x53,0xc4,0xce,0xf2, +0x08,0x36,0x5d,0xa6,0xfc,0xf8,0xcd,0x8b,0xdf,0x2f,0x97,0xf3, +0x97,0x1c,0x65,0xba,0xc6,0x74,0xb9,0x0c,0x99,0x03,0x7b,0xa4, +0xf5,0xe3,0x17,0xb3,0xcb,0xde,0x78,0x2a,0x25,0x6e,0x6f,0x4b, +0x20,0x3a,0x5c,0x3c,0x5b,0x8c,0x88,0x9f,0x7b,0x93,0xac,0x44, +0x7e,0xd2,0x24,0xf1,0xa0,0xfe,0x33,0xf9,0x78,0xc3,0xfd,0x62, +0x2b,0xee,0xe0,0x23,0x9b,0xba,0x75,0x9c,0x73,0x7b,0x1b,0x79, +0xb8,0x78,0x90,0x63,0x55,0x62,0x0b,0x81,0x0e,0xc0,0x8e,0xee, +0x67,0x4e,0x49,0x6d,0x90,0x5d,0x5d,0x09,0xf7,0xb4,0x6c,0x3c, +0xf9,0x1c,0xc6,0x8e,0x1e,0xf1,0xb1,0xa1,0x6f,0x7b,0x44,0x33, +0x0d,0xe4,0x79,0x85,0x79,0x8e,0x8f,0x90,0x95,0x84,0x03,0x0d, +0xa1,0x00,0x8c,0xa6,0xab,0x2d,0x4e,0xc7,0x9d,0x7b,0xae,0x27, +0x28,0x88,0x9c,0x65,0x7a,0x58,0xc3,0x28,0xe8,0xe4,0xa2,0xcb, +0x08,0x05,0x39,0xc4,0x39,0x5a,0xa0,0x24,0xee,0x8b,0xbe,0x24, +0xd9,0x50,0x42,0x05,0xcd,0xfa,0x2e,0x67,0x2f,0xf0,0x2d,0x36, +0xab,0xfc,0xe7,0x5c,0x34,0x3e,0x69,0x76,0x84,0x11,0x44,0xf3, +0x22,0x72,0xd5,0x2c,0xe4,0x16,0xcb,0xcf,0xe1,0x85,0x38,0x23, +0x54,0x84,0xd8,0xc5,0x8c,0xa3,0x99,0x22,0x0c,0x96,0x3e,0x82, +0x21,0x0e,0x9d,0xc8,0x9b,0x54,0x8c,0x02,0x21,0xea,0x76,0x78, +0xe0,0x5b,0x3a,0x1f,0xd7,0xe3,0xf1,0xe6,0xbd,0xeb,0xf2,0x4a, +0xb1,0xba,0x10,0xc1,0x76,0xe3,0x7d,0xf8,0xbb,0x60,0x9b,0x74, +0x03,0x65,0xab,0x4e,0x4a,0x20,0x46,0xe2,0xb8,0x9b,0x65,0xcb, +0x52,0xd7,0x18,0x01,0x30,0x10,0xf4,0xfb,0x11,0xc8,0x7c,0xc2, +0xdf,0xdd,0x02,0xeb,0x37,0x58,0x3b,0x0e,0x60,0xb9,0xaa,0x42, +0xd6,0xd0,0xb9,0xb5,0x06,0x7e,0xd1,0x2d,0x2c,0x3e,0x30,0x52, +0xba,0x6c,0x1a,0xd4,0x49,0xce,0x4b,0x9d,0x19,0x16,0x66,0xcb, +0x7b,0xaf,0x08,0x58,0xd4,0x5d,0x0d,0x08,0x14,0xb9,0x60,0xe4, +0x55,0xc0,0x8c,0x8b,0xf9,0xc8,0x54,0x87,0x33,0x3c,0x1e,0xf6, +0x73,0x97,0xea,0x9e,0x83,0xe9,0xa1,0x37,0x40,0x87,0x79,0x38, +0x88,0x85,0x93,0x60,0xd4,0xcb,0x57,0x8d,0xd3,0x4c,0xee,0x4a, +0x49,0x95,0xeb,0xe5,0x92,0xad,0x20,0x1b,0x2d,0x65,0x36,0xf8, +0x3d,0x95,0x88,0xc3,0xfb,0x5c,0xed,0x3c,0x70,0x57,0x01,0x1e, +0x30,0x1e,0x0b,0xdb,0x7b,0x7e,0x12,0xec,0xed,0xea,0x67,0x5a, +0x75,0x05,0x93,0x09,0x2c,0xcf,0xdf,0xdc,0x99,0x62,0x18,0x99, +0x9b,0x7d,0x90,0xba,0x4c,0x45,0x97,0x74,0xc1,0xaa,0x84,0x3c, +0xe8,0x06,0x74,0x39,0x4d,0xe9,0xc3,0x43,0xec,0x4e,0x34,0x83, +0xca,0xd3,0xb6,0xed,0x3f,0x8a,0xe4,0x29,0xdd,0x74,0xb4,0x9e, +0x3e,0x6e,0xac,0x16,0x08,0xbe,0xf3,0x30,0xb3,0x8a,0x7d,0x43, +0x51,0xb3,0x8b,0x44,0x7e,0xf6,0x71,0xa8,0x8d,0x8e,0x98,0xa8, +0xff,0xb0,0xb3,0x79,0x44,0xb2,0xb7,0x55,0x62,0x76,0xc4,0xad, +0xd7,0x9f,0x2d,0x8a,0x0c,0x38,0x69,0x18,0x94,0x1e,0x49,0x01, +0xba,0x93,0xa7,0x57,0x55,0x58,0x9f,0x79,0xfc,0x00,0xf3,0x7a, +0x49,0x33,0x8a,0x46,0xbd,0x03,0x39,0x5a,0x58,0x01,0x6e,0xcb, +0x58,0x68,0x38,0x3d,0x1b,0xaf,0x9f,0xe9,0x96,0x19,0x42,0xc4, +0xa1,0x6c,0x51,0x9c,0xcd,0xec,0xc0,0x53,0x5b,0x87,0x9e,0x13, +0xec,0x1c,0x4e,0xfe,0x84,0x8b,0xc9,0x98,0xa6,0x94,0xaa,0x57, +0x9c,0x16,0x5f,0xa3,0xb5,0x74,0x1e,0xff,0x2c,0x1b,0xd8,0x60, +0x71,0x9a,0x2d,0x47,0x73,0x7d,0x51,0x3a,0xc3,0x9b,0x49,0x89, +0xc7,0x07,0xd8,0x11,0xa9,0x08,0xf4,0x36,0x36,0x36,0x7b,0x37, +0xc2,0xec,0x55,0xc8,0xda,0xa6,0xa4,0xe3,0xe6,0xc3,0x0e,0x7a, +0x5f,0x18,0x4f,0xc9,0x6b,0x3e,0x46,0xd1,0x09,0x23,0x3e,0x1e, +0xd6,0xd7,0x67,0xa6,0xf4,0x92,0x23,0x48,0x20,0xf3,0x62,0x52, +0xe7,0xb2,0xcd,0xc3,0x8c,0xeb,0x8f,0x9d,0x12,0xa4,0xcb,0x82, +0x76,0x0f,0x71,0xab,0x93,0x8f,0x4d,0x6c,0xce,0xb3,0x05,0xaa, +0xcf,0x7a,0xe0,0x82,0x7e,0xe0,0x04,0xad,0x3b,0xaa,0x89,0xc4, +0x75,0xa2,0xa5,0x02,0xf1,0x55,0x55,0xf0,0xe4,0xa5,0x79,0x3c, +0xc5,0x4b,0x34,0x22,0x0e,0x4c,0x1f,0x47,0x14,0xc4,0x12,0xe9, +0x17,0x68,0x80,0x1a,0xc8,0x13,0xb9,0x48,0xb3,0x2a,0x4d,0xcd, +0x4c,0x4d,0xb6,0xaa,0x20,0x07,0x06,0x52,0x84,0x53,0x21,0xd8, +0x05,0xd8,0x72,0xbf,0x98,0xfe,0xb2,0xd5,0x91,0xd6,0x48,0x00, +0xcf,0x76,0x1c,0xb0,0xc3,0x6a,0x84,0xb3,0x4b,0x3b,0xc8,0xd2, +0x8e,0xa1,0xf4,0x71,0x5a,0x84,0x08,0x5f,0x13,0x7a,0xd1,0x67, +0x93,0x55,0x76,0x91,0xcb,0xec,0xe8,0x73,0xbd,0xab,0x16,0x41, +0xca,0xd3,0x22,0x07,0x03,0xdd,0x2e,0x58,0x5d,0xa6,0x77,0x45, +0x9c,0xca,0xbc,0x67,0x54,0xd5,0x96,0x0d,0xb5,0x4c,0x0a,0x5a, +0x48,0x16,0x00,0xd2,0xa4,0xde,0x68,0x2f,0x93,0x76,0xbc,0x4c, +0x18,0xe9,0x0e,0xe4,0x5d,0x36,0x3a,0xcb,0x06,0xd5,0x94,0x42, +0x34,0xfe,0xb6,0xf2,0xc5,0x4d,0x52,0x04,0x1f,0x98,0xf6,0x44, +0x68,0xc1,0x7d,0xc3,0x71,0x31,0x56,0x1d,0x2d,0x1b,0x10,0x8e, +0x29,0xb7,0xbb,0xaf,0x74,0x00,0xbe,0xee,0x8d,0x27,0x29,0x22, +0x0a,0x51,0xfe,0x8e,0x8c,0x8e,0x35,0xf8,0x05,0x22,0xaa,0x8e, +0x89,0xef,0xa3,0xd4,0xec,0x1d,0x93,0x86,0x3a,0xe2,0xa3,0xca, +0x5e,0x3f,0x8b,0x3e,0xad,0xa2,0xda,0x8c,0xb6,0x2a,0x84,0xaf, +0x60,0x5b,0xe3,0xe3,0x4f,0xed,0x9c,0xe0,0x6c,0xc7,0xea,0x6f, +0x1d,0x1d,0x4c,0x27,0xbe,0xfc,0x76,0xab,0x32,0x7a,0x9f,0xd9, +0x1a,0x6e,0x0b,0x70,0x39,0x9d,0x2f,0x46,0x83,0x71,0x86,0xa5, +0xd6,0x6a,0x6e,0xca,0xcd,0xab,0xd1,0x9a,0x37,0x98,0x83,0xd1, +0x58,0x0e,0x94,0x27,0xb3,0xf3,0x68,0x1d,0xd7,0xe9,0xfb,0xc5, +0xb7,0x09,0x8c,0xc4,0xe0,0xc2,0x4b,0x20,0xe2,0xc2,0xac,0xef, +0xd6,0x9d,0xf3,0xde,0xf0,0x8c,0xb3,0x37,0x64,0x2f,0x01,0xf9, +0xc8,0x57,0xab,0x4b,0xd4,0x7f,0x54,0x33,0xad,0xa8,0x3b,0xab, +0x65,0x69,0xaf,0xd2,0x57,0xfd,0xab,0x74,0x5d,0xed,0xa9,0x1e, +0xfe,0xf4,0xaf,0xda,0xb3,0xda,0x32,0xed,0x55,0x7b,0x57,0x95, +0xa8,0x0f,0x41,0x1f,0x97,0xf3,0xd1,0x0c,0xc6,0x3e,0x48,0xcb, +0x97,0x5f,0x8d,0x66,0xb0,0xec,0x2f,0x36,0xd1,0xb9,0x7c,0x38, +0x0f,0x17,0x75,0xd4,0xe1,0x61,0x61,0x11,0x5c,0xf6,0x42,0x97, +0x95,0xba,0x08,0xbd,0x1c,0x8b,0xf7,0x14,0x3c,0xf1,0x72,0x76, +0x0b,0xaa,0xdf,0x81,0xf0,0x8b,0xf1,0x74,0x14,0x91,0x99,0x05, +0x90,0xb4,0xd1,0xe5,0x83,0xd5,0xb6,0x9b,0xcc,0x32,0xb4,0x19, +0xcc,0xe6,0xc4,0x75,0x06,0xfd,0x3a,0xd4,0x29,0xa2,0x4f,0x09, +0xb8,0x00,0xcd,0x11,0x46,0xc5,0x08,0x6d,0x56,0x90,0x25,0xb6, +0xc8,0xbc,0x95,0xb0,0x25,0x5d,0x20,0x27,0x63,0xfb,0x36,0xb4, +0x36,0x17,0x95,0x07,0x9d,0x8a,0x8f,0xe8,0x9d,0x50,0xe2,0x85, +0x9a,0x5d,0xb4,0xee,0xef,0xd4,0xf7,0x9c,0xf6,0x28,0xc5,0xcd, +0xfd,0x7e,0x36,0xd9,0x9c,0x93,0x3b,0xd6,0x7c,0x8b,0x45,0xf3, +0x72,0x57,0xfb,0xfc,0x86,0xcd,0x19,0x50,0xd0,0xb6,0xbd,0xa4, +0x75,0x9b,0xa8,0x12,0x07,0x69,0x81,0x55,0x8c,0x37,0x5a,0xcc, +0x79,0x89,0x7e,0x37,0xb9,0x26,0x7b,0x20,0x80,0x53,0x02,0xeb, +0x60,0xbc,0xff,0x9b,0xfa,0xed,0x2f,0xf1,0xcd,0xfb,0xbf,0xb9, +0x97,0x6b,0x10,0x95,0xfe,0xf6,0x97,0xb2,0x8d,0xa8,0x37,0x2b, +0xbf,0xfd,0xb9,0xfe,0x90,0x0e,0xd3,0x86,0xef,0xff,0x96,0xbe, +0xff,0x5b,0xf5,0xfd,0xdf,0x1a,0x40,0xc9,0x0c,0x72,0x9a,0x0b, +0x3a,0x00,0x84,0x46,0x46,0x75,0x14,0xde,0xcc,0x61,0xd4,0xdb, +0x94,0x22,0x1b,0x65,0xfa,0x51,0xab,0x94,0x8a,0x35,0xca,0xf4, +0x53,0x79,0x5b,0x36,0xe5,0x01,0x32,0x6c,0xf3,0x53,0x89,0xc1, +0xe2,0x18,0xd3,0x0e,0x1b,0xb3,0xba,0xa4,0xb7,0x20,0xce,0xdd, +0xd1,0x95,0x5a,0xc1,0x14,0x0e,0x18,0x01,0x62,0x8c,0x54,0x83, +0x2b,0x61,0x74,0x1a,0x8c,0x00,0xed,0xfa,0x21,0x53,0x03,0x9a, +0x07,0x7f,0x4c,0x03,0x1a,0x92,0xcb,0xad,0x88,0xd9,0xc8,0x8a, +0x64,0x42,0xa6,0x7c,0x0e,0x43,0x49,0xaa,0x3e,0x42,0xf0,0x80, +0x44,0xec,0xd0,0x4e,0x23,0xe4,0x90,0x29,0x8d,0x10,0x03,0x20, +0x71,0x9c,0xa7,0x71,0xac,0x71,0x76,0x69,0x88,0x52,0x9f,0xad, +0x59,0x18,0xd5,0x15,0x18,0xdd,0x0e,0x95,0x06,0x06,0x07,0x46, +0x58,0x7a,0x00,0x6c,0x30,0xca,0xa0,0xb2,0x28,0x9b,0xa3,0x97, +0x1c,0xd4,0x88,0xbf,0x61,0xaa,0xa4,0x26,0x06,0x07,0x0c,0xa0, +0x68,0xc3,0x34,0x64,0xc2,0x8e,0xd6,0x1a,0x1d,0x94,0x62,0xbb, +0x10,0x7b,0xd0,0x89,0x42,0xf4,0x31,0xca,0x6d,0x4b,0x77,0x37, +0x5a,0x5f,0xcc,0x96,0xbe,0x3e,0x39,0xe0,0x52,0xee,0xc3,0x9f, +0x4a,0x0f,0xb0,0x80,0xaf,0x04,0xbf,0x9a,0xf8,0xd5,0xdc,0x03, +0xe6,0xd9,0x62,0x96,0x65,0x2e,0xa0,0x13,0x29,0xde,0xec,0x56, +0xa5,0x38,0xb4,0x49,0xbe,0x1a,0x18,0xd7,0xe0,0x54,0x25,0x5f, +0x09,0xc6,0x25,0x9c,0xba,0xa7,0x1e,0x54,0xef,0xa5,0x5a,0xb0, +0x58,0x25,0xc5,0xdc,0x0a,0xcb,0xe1,0xa7,0x54,0x80,0x9f,0xfb, +0x50,0x65,0x8b,0x11,0x57,0xfc,0xa2,0xd4,0x08,0xd8,0x27,0x1c, +0x81,0xc8,0xbc,0x55,0xf2,0x9d,0x38,0xdf,0x80,0xf8,0xdb,0x3d, +0x40,0xbf,0xc5,0xe5,0x6d,0x32,0xbe,0x1e,0x45,0x43,0xad,0x4a, +0x69,0x6f,0x53,0x87,0x08,0x15,0xff,0x54,0x86,0x08,0x13,0xff, +0xc0,0x57,0x13,0xbf,0x70,0x36,0xc4,0x94,0x7a,0x3a,0x51,0x18, +0xcf,0xbf,0x4d,0xfc,0xcd,0x4f,0x89,0x9a,0x3d,0x22,0x53,0xad, +0xc1,0xde,0x19,0x99,0x26,0x91,0xb8,0xc8,0x04,0xe4,0xe2,0x89, +0xd4,0x64,0x6c,0x2c,0xd4,0x9f,0x6f,0x95,0xa9,0xe8,0x39,0x3e, +0x0a,0xf1,0xd8,0xc3,0x88,0x1f,0x48,0xff,0x2a,0x92,0x3f,0x7e, +0xf2,0xfe,0xbf,0x0e,0x0f,0x9d,0xf8,0x04,0xe3,0x13,0x8a,0xcf, +0xd3,0x6b,0x84,0x3e,0x11,0xc6,0xc3,0x3d,0x13,0xa1,0x78,0x3a, +0x09,0x98,0xdf,0xd3,0x43,0x68,0x17,0x41,0xfb,0xf1,0xa7,0x3f, +0x45,0x1f,0x3e,0x32,0xf2,0x13,0x78,0x0e,0xe8,0x5a,0x6d,0xd4, +0x75,0x7c,0x43,0xca,0xa1,0x6e,0x86,0x3f,0x36,0x54,0x10,0xf3, +0x63,0xa3,0x92,0x46,0xeb,0x6a,0x2e,0x96,0xee,0x1a,0xf6,0x17, +0xfd,0x09,0x8b,0x6e,0xc2,0xa2,0x3f,0xdd,0xa7,0xe8,0x9f,0xb0, +0xe8,0x75,0x58,0xf4,0x4f,0x45,0x45,0x77,0x36,0xf6,0x85,0x5d, +0xf7,0xef,0xbd,0x62,0xdd,0xa3,0xa3,0xd4,0x3a,0x2d,0xe8,0x92, +0x4d,0x5a,0xd0,0x29,0xd7,0xfe,0x02,0xf6,0x2e,0x75,0x38,0xda, +0x8e,0xa4,0xe8,0x5d,0xba,0x69,0x94,0xaf,0xab,0xd7,0x8d,0xf2, +0x26,0x2e,0xbf,0xab,0x40,0x18,0x3e,0xd7,0xd5,0x35,0x44,0x4a, +0x18,0x3e,0x37,0x55,0xc8,0xb4,0x86,0x30,0x20,0x00,0x1f,0x15, +0x08,0x6d,0x2a,0xd7,0x98,0x25,0xe4,0x9c,0x3f,0x26,0x95,0xf4, +0x5d,0xae,0x1b,0x31,0xb2,0x1c,0xad,0x1b,0x15,0xf8,0x97,0xae, +0x79,0xa7,0xe2,0x75,0x0b,0x67,0xd8,0x40,0x86,0x4d,0x23,0xdd, +0xe4,0x33,0xfc,0x89,0x33,0x5c,0x43,0x86,0xeb,0x46,0x7a,0x9d, +0xcf,0x60,0x99,0xab,0xa1,0x36,0x0d,0x75,0xdd,0x90,0xf3,0x71, +0x09,0x85,0x68,0xee,0x58,0x1a,0x3f,0x7c,0xe0,0xac,0x1b,0x85, +0x1d,0xd2,0x28,0xec,0x11,0x7f,0x3d,0x0c,0x9b,0x90,0x5b,0x8c, +0xef,0x6e,0xe2,0x5e,0xf6,0x23,0x71,0xea,0xa6,0xb8,0x92,0x22, +0xd0,0x3b,0x81,0xfd,0x3d,0xd2,0xd7,0xbf,0x94,0x97,0x07,0x6b, +0xcb,0xb4,0x6a,0xb0,0xb1,0x1c,0xab,0x06,0xd7,0x96,0x5d,0xd5, +0xa5,0xb3,0x62,0x0c,0xd6,0xe5,0xc1,0xba,0x32,0xd8,0x94,0x07, +0x9b,0xca,0xe0,0xba,0x3c,0xb8,0x8e,0x41,0x86,0xf3,0x79,0x19, +0xa4,0xb6,0xcb,0xc3,0x43,0x1c,0xf6,0xa4,0x39,0xb9,0x8a,0xeb, +0x97,0xfe,0xa8,0xb9,0x04,0x09,0x2d,0x64,0xa4,0x1f,0x9b,0x95, +0xf4,0x0a,0x40,0xe7,0x38,0x9a,0xe3,0x37,0x39,0x46,0xe6,0xf8, +0xeb,0x30,0xfe,0xff,0x61,0xe3,0xc6,0x4a,0x98,0x1f,0x3c,0x82, +0x48,0x76,0x54,0x22,0x1b,0xde,0xc9,0xe3,0xff,0xad,0x23,0xae, +0xa0,0xf1,0x39,0x84,0xef,0x29,0xa7,0xee,0x2c,0x57,0x3c,0x30, +0x77,0xd0,0xa5,0x70,0x58,0xfa,0xc7,0x34,0x8b,0xd5,0xc8,0x9c, +0xad,0x1e,0x14,0xac,0x3f,0x93,0xf1,0x5c,0xef,0xf1,0xc4,0x41, +0x62,0xa6,0xe4,0xd9,0x07,0xbe,0x93,0x98,0xd3,0x40,0xfe,0x7a, +0x4a,0x16,0x33,0x1d,0x6f,0x91,0xc1,0x26,0x30,0x5b,0xf5,0xd9, +0x33,0x53,0x97,0x0a,0xa1,0x45,0x5b,0x54,0x8e,0x17,0x80,0xf9, +0xdb,0x0c,0xed,0xfe,0x91,0x0c,0xd7,0x45,0xd1,0xd4,0x38,0x7c, +0x34,0x17,0xe8,0x4f,0xd2,0x86,0xbc,0xa3,0x9a,0xaa,0x79,0x43, +0xa7,0xa3,0xb4,0x39,0x4f,0x4c,0x68,0x4a,0xd5,0x14,0x0b,0x4d, +0xf3,0x06,0xe4,0x04,0x10,0x85,0x5b,0x6b,0xc7,0x5c,0x95,0xb1, +0x49,0x15,0xec,0xa6,0x23,0xa7,0xd6,0x71,0x37,0xa6,0x9a,0x1b, +0x28,0x56,0xe9,0x53,0xa5,0xfc,0x1e,0x5a,0x5f,0xd7,0x92,0x03, +0x8b,0xf4,0x86,0xe0,0xb4,0x10,0x0f,0xfc,0xc8,0x5a,0xda,0x27, +0x26,0xb9,0x3e,0x67,0x87,0xfa,0x57,0xe3,0x6c,0xbc,0x1c,0x0d, +0x5b,0x07,0x89,0xc2,0x3e,0xdc,0xa0,0x33,0x44,0xa1,0x25,0xf9, +0x7f,0xef,0xe7,0xa1,0x9c,0xcc,0x01,0x13,0x06,0xd1,0x2b,0x28, +0x9f,0xd8,0xf2,0xc9,0xb6,0xdd,0x63,0x3f,0xeb,0x69,0x5f,0xc7, +0xb2,0xe5,0xa4,0x5e,0x4c,0xdd,0xc4,0x81,0x7e,0xac,0x2c,0xb2, +0x89,0xad,0x26,0xe9,0xee,0xc7,0x34,0xd9,0x81,0x69,0x11,0x88, +0x22,0x4c,0x1b,0x2e,0xa6,0xea,0x7e,0x98,0xe2,0x3d,0x15,0x85, +0xf1,0x61,0xaf,0x3c,0xd4,0x1d,0xd9,0xc1,0x64,0x99,0x19,0xc6, +0xcd,0xdb,0x67,0xe3,0xc5,0x00,0x6d,0xe4,0x45,0x02,0xf1,0xce, +0x7c,0x18,0x1f,0x9b,0xfa,0xc5,0x44,0xdf,0x8d,0x6b,0xda,0x8c, +0x50,0x4f,0xc3,0x71,0x31,0xa5,0x98,0xd0,0xc6,0x19,0xc6,0xa1, +0xfd,0x2f,0x2e,0xc3,0x7f,0x0f,0xe8,0xc7,0xb0,0x9f,0x78,0x1a, +0x12,0x82,0xf1,0x0f,0x1b,0xcf,0x4c,0x05,0x0b,0x60,0xbb,0x76, +0x9b,0x71,0xd0,0x2e,0x10,0x29,0xdd,0x38,0x16,0x14,0xba,0xb6, +0xf1,0x2d,0xa2,0xc9,0xa2,0x13,0xe9,0x14,0x94,0xf5,0x13,0x16, +0xda,0xb6,0x44,0x9b,0x2b,0x33,0x79,0xa4,0xee,0xc2,0x61,0x32, +0x9c,0xdd,0x58,0x0f,0x8b,0xba,0x2a,0x53,0x92,0x3a,0xcc,0xc4, +0x42,0x7f,0xea,0x04,0x6a,0x64,0xec,0x16,0xd5,0x3d,0xe0,0x8e, +0xbb,0xf1,0x13,0xae,0xda,0x75,0xc1,0x93,0x1b,0x83,0xf6,0xfd, +0x46,0x66,0x46,0x21,0x86,0x68,0x20,0xb2,0x67,0x0a,0xc7,0x85, +0xad,0xd7,0x26,0xe5,0x52,0x41,0xa2,0x12,0x3b,0x6f,0xb5,0x8b, +0x88,0x45,0xaa,0x14,0x45,0x88,0xdf,0x84,0x54,0x5b,0x8c,0xae, +0x84,0x74,0xce,0x64,0xe2,0x37,0xa8,0x5a,0x1d,0x1f,0x43,0x3b, +0x3f,0xac,0x51,0xdb,0x7b,0x36,0xca,0x22,0xa0,0xaa,0xfb,0x5a, +0x85,0xf9,0xb6,0x61,0x24,0xf5,0x9c,0x2a,0x64,0x84,0xed,0xbb, +0x0b,0xf4,0xc0,0x79,0x10,0x74,0x7a,0xe1,0x91,0x61,0x81,0x3c, +0xb9,0x63,0x64,0xd1,0x8d,0x33,0xdb,0x1d,0x4c,0xe9,0x3b,0x37, +0xba,0xa6,0xaa,0x4f,0xef,0xd5,0x7b,0x9c,0x8e,0x7c,0x2f,0x67, +0x84,0x3d,0xbe,0x1d,0xe8,0x4b,0xc2,0xb8,0xab,0xfa,0xd4,0xaa, +0xb4,0x07,0x99,0xfb,0xed,0x20,0xb9,0x61,0x93,0x8b,0xd1,0x63, +0xf2,0xff,0x01,0x9a,0x85,0x26,0x97,0x31,0x06,0xd7,0x58,0x6f, +0x6d,0x33,0xe3,0x3b,0xaf,0x29,0xb5,0x98,0x2d,0xfd,0xd5,0xcf, +0x54,0x31,0x77,0x85,0x5b,0xfb,0x04,0x8c,0x0b,0x48,0x7c,0xdb, +0xad,0x1b,0xe2,0x98,0x09,0xe4,0xf0,0xc9,0xb0,0x80,0x35,0x74, +0x57,0x73,0x81,0x6e,0xfd,0xaa,0xe8,0xb4,0xf4,0x8e,0xea,0xb0, +0xab,0x04,0x86,0xa9,0xc9,0x72,0x9a,0x81,0xe7,0x0c,0xf9,0x1b, +0x9e,0x6d,0x2d,0x87,0x12,0x71,0x08,0x8e,0x93,0xcb,0x2f,0xca, +0x1b,0x87,0xb0,0xa0,0x2d,0x14,0x9c,0xd3,0x52,0x2a,0xee,0x11, +0x34,0xf6,0xa8,0x42,0xca,0x13,0xfc,0x09,0xc5,0xc8,0x33,0xbc, +0xa2,0x06,0x61,0xd6,0x17,0x3e,0x71,0xf6,0x35,0x6c,0xe1,0xec, +0x44,0xbc,0xa2,0x4e,0x5b,0x14,0xb9,0xd9,0x3c,0xe9,0xfa,0xa5, +0xb8,0x4d,0x16,0x55,0x8c,0xe3,0x27,0x6f,0x4a,0x7f,0xa2,0xba, +0x70,0x0e,0x28,0x95,0x6b,0x3b,0xfe,0xac,0xc9,0x66,0xf3,0x34, +0xf5,0x32,0x52,0x94,0xd4,0xfc,0x4a,0x64,0xa4,0xb4,0xbf,0x3a, +0x3b,0x83,0x34,0xfe,0x81,0x44,0x2e,0xa3,0x93,0xcd,0x24,0x79, +0x16,0x31,0xb9,0xd0,0xbc,0xa5,0x92,0x83,0x70,0x26,0xde,0x82, +0x6c,0x54,0x50,0x6b,0x68,0xc5,0x9e,0xd2,0x78,0x4b,0x0e,0xa9, +0xb6,0xf8,0x46,0xf0,0xf1,0xa0,0xe2,0x28,0x33,0xa4,0x56,0x79, +0x29,0x8c,0x2e,0x2f,0xee,0xbc,0xa0,0x28,0x9e,0xdf,0x42,0xd9, +0xc9,0xcc,0xda,0x77,0x8a,0x4f,0xd3,0xe3,0xe4,0xf0,0xb0,0xc9, +0x78,0x1f,0x1e,0x7a,0x54,0x30,0x0d,0xb5,0x31,0x48,0x08,0xad, +0xdb,0xe2,0xa5,0x64,0x17,0xe3,0x33,0x74,0x60,0x84,0xcf,0x46, +0x77,0x16,0x17,0x0b,0x24,0xce,0xfc,0x20,0x49,0x6c,0x09,0x37, +0x89,0xc5,0x6b,0xa2,0x91,0x8d,0x85,0xe4,0x44,0x67,0x62,0xf1, +0x54,0x4f,0x21,0x86,0x46,0xe8,0x1a,0x1e,0x99,0x76,0xf8,0x4c, +0xcf,0x23,0xc2,0xc5,0xfc,0xb2,0x70,0x19,0x99,0xf9,0x85,0xee, +0x69,0x4c,0x00,0x19,0x8a,0xa4,0x67,0x2d,0x49,0x99,0x61,0x44, +0xe9,0x2d,0xf3,0xa5,0x84,0x64,0x2d,0xf9,0x55,0xee,0xe5,0x8a, +0xab,0x30,0x1a,0x8e,0xc9,0x97,0x6c,0x4a,0x1c,0x65,0x15,0x0d, +0x2b,0x35,0x63,0xc4,0x26,0xe0,0x3e,0x44,0x06,0x81,0xa1,0x1d, +0x8a,0xf6,0x52,0x8d,0x6b,0x96,0x34,0xb8,0xd7,0xd9,0x2a,0x7b, +0x37,0xb3,0x0f,0x91,0x10,0x09,0xdb,0x34,0x0f,0x09,0xdd,0x3e, +0x83,0xc4,0xf0,0xa8,0x76,0x39,0x5a,0x9c,0x8f,0xcc,0xd6,0x42, +0xde,0xa4,0xfa,0x02,0x99,0xde,0x17,0x51,0x55,0x5f,0x4f,0xf5, +0xfe,0x26,0xec,0x15,0x8d,0x6b,0xdc,0xce,0xfc,0x51,0xd6,0xd9, +0xb7,0x39,0x72,0x59,0x65,0x26,0x08,0xdf,0xb9,0x49,0x6a,0x05, +0xb9,0xd0,0x9c,0x6a,0xd1,0x5c,0xe4,0xae,0xfb,0x3c,0x88,0xf1, +0x8f,0xb3,0xbe,0xab,0xfc,0xb0,0x89,0x73,0x9d,0xc1,0xa3,0xc9, +0x10,0x4d,0x77,0x1b,0xc2,0xda,0x2a,0xda,0x26,0x8d,0xdc,0xce, +0xd9,0xd1,0x97,0x85,0x22,0xe1,0xdf,0x85,0x61,0x31,0x82,0xdb, +0xad,0xe2,0xc9,0x2e,0x75,0x48,0xfa,0x39,0xc5,0x18,0xab,0x07, +0xfe,0x0c,0x6b,0x87,0x1a,0x17,0x34,0x73,0x09,0xc6,0x17,0xaf, +0xf6,0xfe,0x68,0xb6,0xfb,0x4e,0xfd,0x4c,0xde,0x9b,0xf0,0x8e, +0x93,0x42,0x18,0x21,0x4e,0x37,0xe2,0xb0,0x9c,0xd7,0xc5,0xcc, +0xba,0x6f,0xb9,0xb1,0x63,0xd5,0x23,0x2f,0x64,0xd2,0x16,0x6b, +0x61,0xbe,0x38,0xe9,0xd2,0x30,0xc1,0x11,0x1e,0x9e,0x86,0xf0, +0x7a,0xed,0xae,0x84,0x5b,0x33,0xd6,0x87,0x6c,0x18,0x43,0x68, +0xe6,0x56,0x20,0x6e,0x64,0x90,0x92,0x54,0x97,0x9d,0x61,0x19, +0x3b,0x9e,0xa6,0xa8,0x9f,0x38,0x1b,0xea,0xa3,0xa2,0x69,0xf1, +0x3c,0x96,0x9a,0x0e,0x28,0x82,0x78,0x58,0x07,0x13,0x2d,0x47, +0x99,0x19,0x76,0x87,0x24,0x88,0xe3,0xc3,0xbd,0xbc,0x88,0x22, +0x90,0xee,0x78,0x1a,0xc0,0x35,0xe1,0x49,0xa3,0x43,0x97,0x15, +0xfc,0x88,0xb8,0xfa,0xfe,0xbf,0x5a,0xf2,0x85,0xb1,0x71,0x35, +0x8a,0xfa,0x69,0xdf,0xcb,0xdd,0x2f,0xcc,0xdd,0xf7,0xd7,0xfc, +0xe2,0xb1,0xcf,0xb3,0x8e,0x1e,0xf0,0xfc,0xce,0x69,0xb4,0x18, +0xe3,0x19,0x91,0x15,0xc2,0xd0,0x84,0xd3,0x64,0x32,0x9a,0x18, +0x51,0x4c,0xe9,0x3c,0x7c,0xbb,0x94,0x9e,0x98,0xd3,0x21,0x9d, +0x10,0xab,0xaa,0x39,0x57,0xb2,0x71,0xb4,0xde,0x81,0xdc,0xfb, +0x14,0xdd,0x47,0x82,0x50,0x8b,0x0a,0xcf,0xb8,0x2e,0x37,0x8a, +0xee,0x75,0x51,0x9b,0x2e,0x38,0x93,0x50,0x53,0x3d,0x68,0x0b, +0x0d,0x67,0xd3,0x1a,0x2f,0x19,0x50,0x1e,0xbe,0xa4,0xd9,0xda, +0x11,0x11,0x2e,0xcd,0x46,0x8b,0x5a,0xd2,0x48,0x45,0x66,0xa1, +0xfb,0x5a,0x8e,0x62,0xa1,0xd3,0x84,0x92,0xae,0x5c,0xc5,0x16, +0xdc,0xc4,0x36,0xe2,0x82,0xeb,0x5c,0x88,0xfc,0x25,0x4d,0x70, +0x6f,0xfa,0x0b,0x6c,0x30,0x2f,0xd1,0x56,0x12,0x6a,0x5d,0x89, +0xd0,0x86,0xd5,0xfd,0xc2,0xb2,0xc5,0x0e,0x29,0xd7,0xab,0x2e, +0x38,0xae,0xcd,0xdf,0xb0,0x3b,0xf7,0xef,0xbd,0x29,0xe9,0xf2, +0x70,0xd7,0x99,0x5b,0x2f,0xbc,0x34,0x3f,0xfe,0xed,0xcf,0xc1, +0xbd,0xbb,0x73,0x3a,0xb4,0xe7,0x26,0x3d,0xb8,0x7b,0x57,0xfb, +0xae,0xea,0xe3,0xd8,0xe9,0xd9,0x4a,0xea,0x22,0xd3,0x81,0xf4, +0x0a,0x66,0x82,0x4d,0x5f,0xa7,0xd9,0xaa,0x36,0xe3,0xf2,0x6f, +0x7f,0x6e,0x41,0xd8,0x43,0xf9,0x67,0x68,0xc3,0x71,0xea,0x84, +0x6c,0x80,0x3b,0xb7,0xb7,0x18,0xa4,0xc5,0x97,0xbb,0xb9,0x0b, +0x6d,0xee,0x3b,0x7b,0xac,0xe1,0x27,0xc4,0xf6,0xf0,0x39,0x7f, +0x49,0x0a,0xb5,0xf0,0xd2,0x49,0xf3,0xb9,0xb8,0xa1,0xd9,0x51, +0xaf,0x3f,0x08,0x14,0x15,0xdd,0x0d,0xd9,0x05,0xc8,0x55,0xfc, +0xf6,0x17,0x6c,0x53,0xe4,0x51,0x41,0xc8,0x54,0x4d,0x5a,0x09, +0xe9,0x03,0xd0,0xad,0xa8,0x5b,0x94,0x2e,0x66,0x23,0x3d,0x22, +0x8f,0x09,0xc6,0xed,0xad,0x19,0xa1,0x69,0x4a,0x31,0xc0,0x76, +0xf0,0x17,0x58,0xeb,0xf6,0x16,0x7f,0x61,0x22,0x40,0x4d,0x47, +0x19,0x6e,0x7e,0xf7,0xe8,0x2a,0x93,0x56,0x35,0x21,0x8f,0x81, +0x07,0x68,0x3f,0xad,0x0f,0x6c,0xf1,0xb6,0x2d,0x5a,0x14,0xae, +0xe2,0x84,0xaf,0x52,0x21,0xc3,0x68,0xce,0x07,0xac,0x32,0x9b, +0xc1,0x1c,0x74,0x6c,0x99,0xe1,0xf6,0xd6,0x0b,0x1e,0x1e,0x92, +0x91,0x8f,0x80,0xf5,0xe2,0x9f,0x93,0x43,0xc1,0xae,0x70,0xc6, +0x7c,0xea,0x20,0xec,0xcb,0x94,0xac,0x56,0x84,0x22,0x31,0x0d, +0x62,0x7c,0x4e,0x86,0xe8,0xd1,0xb3,0x32,0x89,0xb8,0x63,0x15, +0x2a,0x58,0xcc,0x79,0x6f,0x92,0x14,0x2c,0x47,0x09,0x40,0x4f, +0xe4,0x6c,0x17,0x42,0x40,0x9d,0xe4,0xb8,0x81,0xa6,0x22,0xaa, +0x30,0xc8,0x70,0x34,0x9a,0x91,0x07,0x29,0x38,0x2e,0xe3,0xb6, +0x3b,0x16,0x21,0x17,0x5e,0x3f,0x77,0xa2,0xdc,0x4e,0x96,0xd1, +0xc6,0x09,0xa4,0x82,0x35,0xd4,0x9b,0x00,0x56,0x9b,0x94,0xe0, +0xdf,0x38,0xd4,0x6b,0xca,0xa4,0xd4,0x7e,0xd1,0xc2,0x6d,0x57, +0x1e,0x40,0x12,0x00,0xd0,0xe8,0x48,0x3c,0x13,0xa2,0x11,0xb7, +0xb0,0xae,0x83,0x34,0xc5,0x12,0x87,0x87,0xc4,0x31,0xbf,0xfd, +0x19,0x38,0xca,0x69,0x6d,0xa3,0x8a,0x79,0xf8,0xd2,0x9d,0x82, +0x98,0xb9,0x51,0x7e,0xff,0x5f,0x72,0xaf,0xad,0x69,0x82,0x20, +0x4c,0xae,0x84,0x72,0x25,0x94,0x0b,0xdb,0xbf,0xa3,0xcb,0xbf, +0xd6,0x03,0x40,0x13,0x4a,0xe9,0x9e,0xf8,0x17,0xd2,0x44,0xb7, +0x3d,0x4f,0x1c,0x1c,0x14,0x94,0x29,0x25,0x14,0x88,0xcf,0xb0, +0x9c,0x15,0x4a,0x76,0x31,0x18,0xe3,0x81,0xf9,0x85,0x49,0xb7, +0x5c,0x4d,0xc1,0x6b,0xb6,0x66,0x95,0x52,0x76,0x88,0x10,0xf7, +0xa3,0xce,0x8d,0xb9,0x07,0x12,0xf5,0xa6,0x04,0xc7,0x32,0xe4, +0x3a,0x45,0xde,0x75,0xae,0x78,0x1a,0x55,0xec,0x95,0x76,0xa8, +0x4b,0x61,0x33,0xc7,0xc7,0xc0,0xb5,0x66,0x6d,0x88,0x22,0x6f, +0x19,0x2c,0xb3,0x7e,0x43,0xe2,0xae,0x4d,0xb0,0x43,0x74,0xef, +0x90,0x92,0xb8,0xea,0x14,0x49,0x74,0x91,0x60,0xe9,0xf4,0x8a, +0x40,0xb0,0x1e,0xb9,0x6b,0x4e,0x52,0x76,0xf0,0x89,0x5b,0xce, +0x58,0xb9,0x1f,0x89,0x44,0x3a,0x47,0x17,0xef,0x6a,0x39,0x43, +0x67,0x50,0x62,0x7e,0xd1,0x9f,0x4c,0x78,0x79,0x64,0x0d,0x75, +0xcc,0x1b,0xc3,0xa2,0x6b,0xf2,0x96,0x79,0x2c,0x86,0x3c,0x81, +0xa3,0x1f,0xd7,0xe3,0x20,0xba,0x51,0x14,0x59,0x9c,0x15,0x62, +0xf3,0xac,0x06,0x91,0xd5,0x42,0xb0,0x45,0xb1,0xd5,0x1d,0xb9, +0xab,0x85,0xa0,0x05,0x63,0xe3,0xc5,0xd8,0xf4,0x39,0x36,0x19, +0x55,0x68,0x96,0x33,0xd4,0xa1,0x81,0x6e,0x97,0xd9,0x2e,0xd5, +0x29,0x4f,0x28,0x85,0xd7,0x0b,0x58,0xc9,0xdb,0x1e,0x79,0xb2, +0x7a,0x33,0x57,0x53,0x76,0x6f,0xd2,0x50,0x4e,0x3e,0xe9,0x0d, +0x52,0xa8,0x4e,0xe8,0xb5,0x42,0x99,0x16,0x7b,0x1b,0x8f,0x54, +0x27,0xa3,0x08,0xaf,0x37,0xf1,0xc5,0xa1,0xb5,0xd0,0x62,0x0e, +0x15,0xf9,0x4c,0xcd,0xe1,0x70,0xb9,0xf3,0x2c,0xbb,0x12,0xd5, +0xf1,0x60,0xe1,0x5a,0xf4,0x0e,0x4f,0x30,0xec,0x89,0x22,0x0c, +0xa8,0x86,0xba,0xc2,0xff,0x1b,0x3c,0x84,0xf7,0xae,0x32,0x90, +0x2b,0xbd,0x6a,0xa0,0x33,0xd6,0xdd,0x4b,0x8b,0x7f,0x62,0xd9, +0xe4,0x0c,0x49,0x2a,0x3b,0x1e,0x75,0x95,0xfa,0x4d,0x51,0x83, +0x94,0xfc,0xcb,0xbd,0xa4,0x16,0x77,0xae,0x3a,0x8d,0x16,0x7a, +0xad,0x93,0xd4,0xd6,0x55,0x47,0x42,0x95,0xa8,0x71,0xfc,0xfe, +0x6f,0xb2,0x44,0xc5,0x94,0x46,0x46,0x0b,0x0e,0xb8,0x19,0x30, +0x0d,0x0b,0x72,0x57,0xee,0x01,0xab,0x3b,0x43,0x5e,0xc1,0xc4, +0x7f,0x85,0x19,0x19,0xb1,0xd4,0x88,0x23,0x22,0x68,0x09,0xa6, +0xb1,0xda,0x75,0xe3,0xe8,0xe4,0x6a,0xc6,0x64,0xbc,0x6b,0x57, +0xae,0x44,0xe7,0x8a,0x75,0x75,0x09,0xa9,0xd3,0xbd,0xff,0x2f, +0xa9,0x84,0x34,0xea,0x20,0x64,0xa9,0x61,0x72,0xd9,0x1c,0x28, +0x8d,0x32,0xd2,0xb1,0x4c,0xe2,0xea,0xaa,0x53,0x7c,0xb0,0x50, +0xdc,0x24,0x41,0x23,0x3f,0x6a,0x38,0xbb,0xa9,0x8c,0x6c,0x23, +0x68,0x9b,0x5b,0x7b,0x08,0x73,0x0f,0x28,0x85,0x07,0x18,0xae, +0x78,0xd5,0x34,0x43,0x75,0x3a,0x5b,0xfe,0x7e,0x74,0x39,0xe6, +0x03,0x8b,0xc3,0x43,0xdd,0x8f,0x0e,0x37,0xfc,0x7c,0x25,0x16, +0x0f,0xdb,0x83,0xc3,0x41,0xe3,0xf6,0xf6,0x20,0x5a,0xee,0x6a, +0x22,0x3e,0x29,0xc3,0x67,0xa2,0x9a,0x50,0x2e,0x4f,0x15,0x93, +0x2c,0x1c,0x99,0x72,0xfc,0xbb,0xd4,0x67,0xbf,0xb9,0xf4,0x44, +0xd2,0x93,0xee,0xae,0x86,0xb6,0x42,0x79,0xe8,0x1e,0x85,0xee, +0xb5,0x9a,0x87,0xc8,0xc1,0x96,0xfc,0xe0,0x0a,0xc4,0x65,0x21, +0xd9,0xdd,0xfc,0x9a,0xa0,0x2d,0xc7,0x82,0xde,0x0b,0x18,0xce, +0xeb,0xa9,0x44,0xe1,0x68,0x82,0xf9,0x21,0x1d,0x14,0x4a,0x04, +0x38,0x96,0x0a,0xda,0x23,0x20,0xf8,0x34,0x6a,0x97,0x54,0x40, +0xf1,0xb7,0x38,0x64,0x0f,0x0f,0x81,0xbd,0x9f,0x3c,0x49,0x5c, +0x01,0xc1,0xf6,0x31,0xbe,0x44,0x58,0xbe,0x93,0xb7,0x08,0xf3, +0x5e,0x6e,0x07,0x83,0x17,0xc8,0xf3,0x7e,0x3e,0xba,0x1f,0xab, +0x29,0xcc,0x39,0x89,0x6a,0xe0,0x66,0x7d,0xda,0xdc,0xb1,0xf5, +0x99,0xf7,0xa0,0x34,0x64,0x6d,0x16,0xe4,0x40,0xc5,0xdd,0x69, +0x13,0xd2,0x10,0x14,0xa4,0x4f,0x89,0xd3,0x87,0x23,0x40,0xed, +0x72,0x3c,0xed,0xa1,0xda,0x0a,0x14,0xab,0x62,0x5a,0x19,0xff, +0xd0,0x64,0xe4,0x24,0xcb,0x7d,0xec,0x01,0xa0,0x7f,0x78,0xd8, +0xe3,0x73,0xcc,0x24,0x1d,0x2c,0xca,0x58,0xac,0xee,0x64,0x54, +0x83,0x66,0x5a,0xc5,0xf8,0x24,0x88,0x9f,0x26,0xeb,0x9d,0x98, +0x4f,0x13,0xc2,0xec,0x69,0x2e,0x99,0xb5,0x6f,0x21,0x79,0x00, +0x03,0xf6,0xf3,0x5d,0xc9,0x4d,0xa8,0x34,0xbf,0xcb,0x43,0xe5, +0xdf,0xa7,0xea,0x73,0xde,0xd6,0xad,0x52,0xaa,0x9f,0x4c,0x67, +0xe6,0x29,0xf3,0x54,0xad,0x62,0xb5,0x5a,0x15,0xa6,0xad,0x30, +0x0d,0xe6,0x91,0x77,0xe5,0x77,0xd5,0xd5,0xaa,0x9c,0xdb,0xd8, +0x72,0xf9,0xa7,0x68,0xb9,0xad,0xfd,0x3f,0x48,0x43,0xa4,0x71, +0x8c,0x73,0x25,0x8f,0x75,0x47,0x4b,0x0b,0x22,0xd5,0xaf,0x3b, +0x9a,0xb0,0x52,0x51,0xf5,0x5d,0x75,0x19,0xd7,0x57,0xab,0xd8, +0x39,0x07,0xf0,0xda,0xf2,0x2b,0x54,0x62,0x01,0x58,0xf5,0xde, +0x5f,0x63,0x85,0xdd,0xa2,0xbd,0x31,0xfc,0x4a,0xed,0xbd,0x26, +0x79,0xb6,0xc7,0xc7,0x28,0x09,0x6b,0x3e,0xa3,0x64,0x87,0xa7, +0x55,0x28,0x85,0x92,0x02,0x34,0x6e,0x25,0x8f,0x69,0xf3,0x10, +0x5d,0xa7,0x28,0xa6,0x6a,0x29,0x1a,0x8b,0x5c,0x33,0xe1,0xde, +0xff,0x95,0x4e,0x33,0x68,0xdb,0xc4,0xa7,0x09,0xce,0x6e,0xea, +0xaf,0x66,0xf3,0xa4,0x9c,0x03,0x2a,0xc8,0x43,0xfb,0x4b,0x4c, +0x96,0x45,0x0d,0xa2,0x0e,0x0f,0xa1,0x7e,0xd8,0x1a,0x73,0x6d, +0x28,0xfc,0x8a,0xac,0x4e,0xe8,0x5c,0xc7,0x06,0x42,0x87,0xb2, +0x77,0x44,0x90,0x3c,0x6e,0xfc,0xfc,0x2b,0xa0,0xfa,0xc4,0xca, +0x42,0xbf,0xa2,0x20,0x24,0xbb,0x1a,0xcc,0xd5,0x42,0x71,0x13, +0xfe,0x34,0x9e,0xa4,0x98,0xf3,0xf0,0x90,0xf2,0x23,0xe4,0x16, +0x62,0x70,0xfc,0xdb,0x9f,0x7f,0x46,0xc1,0x15,0x53,0x1b,0x94, +0x0a,0xd2,0x52,0x4a,0xd2,0x2a,0x75,0xd1,0xaf,0xc9,0xbe,0x3e, +0xa9,0x48,0x9f,0xd8,0xc7,0x59,0xf9,0x7e,0x49,0xb0,0x63,0x4e, +0x7e,0x55,0xf9,0x9e,0x49,0xe2,0xae,0x3b,0x13,0x38,0xa2,0x00, +0x1f,0x8d,0x79,0xc2,0x02,0x4b,0x49,0xad,0xdf,0xfe,0x5c,0xe5, +0x2f,0xf2,0x77,0x9b,0xea,0x8d,0x72,0x75,0x81,0x02,0x03,0x46, +0xdd,0xa6,0xd0,0xb0,0xbf,0x1d,0xa3,0x29,0x63,0x0e,0x02,0x5f, +0x41,0xea,0x6f,0x7f,0x91,0xd4,0x87,0x40,0x0b,0x27,0xf5,0xd3, +0x98,0x00,0xd1,0xc5,0xd0,0x60,0x61,0x45,0x7b,0x11,0xca,0xdc, +0xb5,0x05,0xc6,0xf3,0x71,0x43,0x79,0x2b,0x99,0xed,0xec,0xc1, +0x02,0xe5,0x4e,0xf7,0x48,0xdd,0xd0,0x8d,0xa4,0x3c,0x57,0x9a, +0x97,0x06,0x3c,0x2e,0xbb,0xba,0x6d,0x01,0x09,0xf1,0x76,0xfb, +0x6a,0xdf,0xc5,0xb6,0x4b,0x9b,0x13,0x90,0xae,0x19,0x68,0xb7, +0x75,0x82,0x52,0x32,0x07,0xe0,0x6b,0x87,0xe4,0xf9,0x1c,0x3d, +0xdb,0x2e,0x45,0xf7,0x6d,0x9d,0xa8,0x4d,0xe2,0xfa,0x95,0x99, +0x2d,0xf0,0x6c,0x7c,0x6e,0x77,0x1b,0x79,0x8d,0xf5,0x39,0x72, +0xd9,0x9a,0x99,0xcc,0x64,0x83,0xed,0x7f,0xa3,0x75,0xd4,0x0a, +0x32,0x25,0xb9,0x4c,0xcd,0x56,0xe2,0x66,0x4a,0xba,0xd5,0x4d, +0x1e,0x52,0xd2,0x6a,0xb4,0xdc,0xf0,0x51,0xab,0xe9,0xb2,0x0a, +0x69,0x0b,0x79,0xda,0xf4,0x12,0xf7,0x3d,0xa9,0x3e,0x44,0x72, +0x36,0xad,0xf4,0xa9,0x73,0xae,0xac,0xce,0xa7,0x4d,0x12,0x0d, +0x7a,0xa9,0x34,0xbc,0xa7,0x60,0x36,0x1d,0xf4,0x75,0xb0,0xaf, +0xec,0x2e,0x73,0xd0,0x03,0x11,0x6d,0xd0,0xef,0x0c,0x7a,0xd5, +0x41,0xbf,0xd5,0xa0,0x67,0xc4,0x1d,0xf3,0xde,0xa2,0xc5,0xef, +0x8a,0x3b,0x46,0x9d,0xbf,0xd5,0xd4,0x11,0xa2,0xc7,0xdf,0x32, +0xef,0x35,0x0a,0x24,0x76,0x83,0xca,0x70,0x9d,0x9a,0x7c,0x6a, +0xb8,0x49,0x4d,0x05,0x6a,0x99,0x42,0x57,0x27,0xdd,0xdc,0xa6, +0x77,0xb8,0x0e,0x5e,0x0e,0x0c,0x37,0x44,0xd0,0x35,0x8c,0xeb, +0x1e,0x8d,0xeb,0x1e,0x8d,0xeb,0x35,0x4c,0x2f,0x1b,0x8a,0x4b, +0x28,0x0e,0x67,0x82,0x4d,0xd2,0xca,0x71,0xc6,0x6b,0x60,0x0d, +0xa9,0x7e,0xad,0x96,0xf6,0xf1,0x9d,0x9b,0xa3,0xc7,0xbd,0xab, +0xaa,0xbb,0xb3,0x6c,0x1a,0x8c,0x36,0x3a,0x29,0xdf,0x05,0x04, +0x7b,0x1f,0x81,0x60,0x96,0x0e,0x09,0x51,0x4f,0x70,0x0e,0x44, +0x02,0x10,0xea,0x95,0x25,0x3d,0xf3,0x58,0x2b,0xa4,0x02,0x21, +0x2e,0x31,0x9b,0x98,0x44,0x24,0x32,0x85,0xce,0x0f,0x58,0x96, +0xf4,0x34,0x64,0x2d,0x6f,0x58,0x38,0x04,0xb9,0x40,0x5a,0x44, +0xb5,0xb4,0x50,0xfd,0xa3,0x40,0xef,0x63,0x4c,0x17,0x7a,0xe6, +0x9e,0xc1,0x6a,0xb4,0xbc,0x9b,0x16,0x1c,0xe4,0xab,0x4d,0x8a, +0xcc,0xab,0x5c,0xc5,0x43,0x5d,0xa0,0x8f,0x27,0xea,0x20,0xf1, +0x7b,0x27,0xfb,0x57,0xba,0x60,0x2f,0xbd,0xc2,0xeb,0x7a,0x74, +0xfc,0x50,0xa9,0xfc,0x12,0xf7,0x21,0xf8,0x0b,0x3f,0xbd,0x81, +0xee,0x20,0x7e,0x3a,0xde,0x1c,0x1e,0x8e,0x33,0x34,0x25,0x4b, +0x62,0xd1,0x3c,0xc6,0x66,0x56,0x2a,0xef,0xa6,0xad,0x3e,0xe7, +0x0a,0x92,0x9f,0x40,0x72,0xb5,0xfa,0x6e,0x4a,0x6a,0x34,0xd2, +0x52,0x3c,0xfc,0x7a,0xe7,0x78,0x60,0x70,0x0a,0x0c,0xcc,0x35, +0x8e,0x61,0x34,0x3c,0xde,0xd0,0x7c,0x16,0x57,0xa3,0x81,0x13, +0x6f,0xf8,0x2f,0x0e,0xc4,0xb6,0x7b,0x9e,0x4e,0xf4,0x50,0x17, +0x28,0x61,0xbb,0x73,0xf6,0x8c,0x02,0x04,0x78,0x33,0xe4,0x08, +0x82,0x9d,0x6e,0x62,0xc0,0x3c,0x82,0x12,0x92,0xea,0x82,0x46, +0xc1,0xdf,0x1f,0xc1,0x52,0x3b,0x90,0xe0,0x67,0x67,0xbe,0x88, +0x6f,0x86,0xb3,0x70,0x43,0x8e,0xe3,0xb5,0x77,0x7b,0x7b,0x84, +0x3f,0x30,0x34,0x5a,0xc0,0xbc,0xbd,0xe3,0xa4,0x03,0x03,0x00, +0xe6,0x9f,0x36,0x2b,0x4b,0x01,0x4e,0x51,0xaf,0x62,0x00,0x55, +0x1e,0xc6,0xbf,0x7b,0x88,0xe8,0xf4,0x92,0x0f,0xd9,0xe4,0x7b, +0xba,0x41,0x6b,0xb5,0x71,0x8c,0x31,0xa6,0x6b,0xe8,0xaa,0x75, +0x72,0x8c,0xbe,0x7b,0x37,0xc7,0xe9,0x06,0x42,0x1b,0x08,0x6d, +0x82,0xc2,0x5c,0x2a,0x07,0x27,0xa7,0x4f,0x84,0x91,0x77,0x6a, +0xff,0x60,0xf4,0xf7,0xee,0xad,0x19,0xee,0xc0,0x02,0x75,0x13, +0xbc,0xcb,0x54,0x67,0xe3,0x45,0xb6,0x44,0xf5,0xbf,0xab,0x53, +0xdc,0xf8,0xaf,0x4f,0xd3,0xcd,0x29,0x1d,0xf9,0x15,0x28,0x09, +0xe9,0x3b,0x69,0xba,0xff,0x16,0xf8,0xd1,0xfa,0xf4,0x54,0x6d, +0x4e,0x4f,0x61,0x2f,0x7b,0x7a,0x0a,0xd2,0x3e,0xfc,0xeb,0x7b, +0x37,0xae,0x35,0xbe,0xad,0x8c,0x42,0x45,0x8e,0x20,0x97,0xd6, +0x9c,0x89,0xb5,0xca,0xa8,0xa3,0x6e,0x80,0x40,0xf7,0xbd,0x79, +0x75,0x90,0x41,0x12,0x3a,0x16,0xe4,0xaa,0xb9,0x09,0xe8,0x9b, +0xa7,0x3f,0x2a,0x63,0x83,0xae,0x30,0x15,0x35,0xef,0x37,0xff, +0x18,0x88,0x8d,0x78,0x64,0xb8,0x4a,0x73,0xdd,0x49,0x2e,0x71, +0x75,0x97,0x38,0xba,0x53,0x90,0xa6,0x7b,0x23,0x06,0x92,0xa6, +0x6b,0x24,0x6a,0xba,0x41,0xaa,0xc2,0x2e,0x4d,0x7a,0x09,0xe6, +0x97,0x5d,0xba,0x07,0x05,0x4c,0x62,0x8f,0xcd,0xae,0xb0,0x63, +0x42,0x4d,0x1f,0xc2,0x86,0x34,0x26,0x79,0xc4,0x9e,0xac,0xb1, +0x23,0xbb,0xaa,0x9f,0x12,0x32,0x6d,0x6f,0x95,0xea,0x44,0x57, +0xa7,0x30,0x7c,0xef,0x53,0x75,0x4f,0x3f,0x2a,0x8c,0x73,0x49, +0x24,0x75,0xd3,0x8d,0xaf,0xba,0x72,0x76,0xac,0x76,0x77,0xc9, +0xfb,0xfb,0x03,0x10,0x5f,0x3f,0xa4,0xa1,0xb6,0xd8,0x76,0xcd, +0x94,0x23,0xc2,0xa5,0x57,0x7b,0x94,0x7f,0x84,0x6d,0x91,0xbe, +0x8a,0x1a,0x0e,0x5f,0x4c,0x66,0x86,0x66,0xea,0xb1,0x7e,0x49, +0x94,0xcf,0xb2,0x7b,0x15,0x1f,0xfe,0xb9,0x9a,0x40,0x1a,0x83, +0xd4,0xc7,0x60,0x97,0x7a,0x8f,0x90,0xa3,0xb1,0x4b,0x9f,0xc7, +0x80,0x33,0x6d,0xfc,0x60,0x1d,0x1d,0x7f,0xdd,0x3c,0x41,0xb1, +0x12,0x7b,0x95,0xa3,0x53,0x51,0xfe,0x0a,0x95,0x6b,0x44,0xba, +0x4d,0x03,0x85,0x9d,0x76,0xc4,0xc5,0x6e,0x6f,0x25,0x43,0xec, +0xf6,0x7d,0xa0,0xd6,0x32,0xfe,0x17,0x68,0xe0,0x48,0xb5,0x9e, +0xb8,0xf2,0x8f,0x3d,0xab,0x28,0xd6,0x99,0x29,0xd0,0xea,0x31, +0x0a,0x7f,0xdb,0xed,0xdd,0xea,0x30,0x5a,0x84,0x9a,0xae,0x60, +0xe9,0x1c,0x4d,0x67,0x78,0x90,0x40,0x7e,0x5d,0x49,0x57,0xd0, +0x4a,0x82,0x36,0x09,0x45,0x42,0xbd,0x0f,0x6e,0x1c,0x43,0x45, +0x97,0x7a,0xf3,0xbf,0xba,0xac,0x3b,0xf9,0x68,0x8b,0x6d,0x83, +0xb8,0x98,0x42,0xcc,0xea,0x98,0xd4,0xda,0xe4,0xa4,0x23,0x69, +0x63,0x18,0x25,0x4b,0x3a,0xb0,0x4a,0x57,0xb1,0xd1,0xb7,0x5e, +0x3d,0xc1,0x18,0x27,0xe3,0x93,0x25,0x89,0x9b,0x24,0xdc,0x61, +0xc6,0xdd,0x2f,0x58,0x80,0xaa,0xb3,0x4c,0x24,0x61,0x4f,0x60, +0xc7,0x58,0x6f,0x1d,0x4d,0x7b,0x3c,0xd8,0xfb,0xd1,0x1a,0xa7, +0x91,0x35,0xad,0xbf,0xfa,0xd9,0xb5,0x28,0xe7,0xc1,0xe2,0x63, +0xbe,0x22,0x81,0x22,0x11,0xf6,0x4d,0x8f,0x0f,0x55,0x17,0x60, +0xe0,0xb0,0x44,0xf7,0x4c,0x84,0xad,0x26,0x56,0x02,0xac,0xa0, +0x01,0xd3,0xf1,0x20,0x9a,0x2f,0x66,0xa8,0x68,0xfe,0x74,0xa9, +0x2f,0x67,0x1a,0x69,0x83,0xb6,0xf1,0xbf,0xfd,0xb9,0x7e,0x04, +0x72,0xa0,0x56,0x6b,0xe1,0x6c,0x50,0xfc,0x9b,0xd5,0x92,0xfc, +0x8d,0xd9,0x82,0x6a,0x9e,0x5e,0x46,0x7c,0x0a,0x60,0x77,0x1f, +0xb0,0x0e,0xca,0x9d,0xb9,0x63,0x72,0xee,0x74,0xb7,0x99,0x33, +0x82,0x90,0x9e,0xa2,0x1c,0x0c,0x35,0x27,0x9f,0x32,0x12,0xa7, +0x28,0x3d,0x73,0x38,0x6e,0x9d,0xc0,0xdf,0x32,0x66,0xab,0xe3, +0x41,0xbb,0x04,0x12,0x0c,0x74,0x61,0xbe,0x28,0x6e,0x1f,0x9d, +0x38,0x3e,0x5d,0x8c,0x7a,0x06,0x41,0xdb,0x57,0x20,0xff,0xbe, +0xeb,0x2d,0x86,0xee,0x36,0xfe,0xb7,0xff,0x70,0x4e,0x79,0x9e, +0x55,0x9b,0xe5,0x69,0xd9,0x7b,0x34,0x5c,0xd7,0xd7,0x10,0x27, +0xbf,0xfd,0x87,0x7b,0x97,0x56,0x4e,0xa7,0xb1,0xfa,0xed,0x3f, +0x1a,0x55,0x1d,0x2d,0x77,0x1f,0xec,0xb7,0xa1,0x50,0xb5,0x65, +0x9a,0x46,0x1c,0x5f,0xf1,0x2e,0xee,0xe2,0x7a,0x53,0x3d,0x4b, +0x93,0x8a,0xa8,0x93,0x44,0x80,0x42,0x95,0xbf,0xa9,0x02,0x17, +0x3d,0x8b,0x8d,0x6e,0x4a,0x31,0xc3,0x70,0xc3,0x1a,0xa7,0x9b, +0x14,0x31,0xdc,0xb4,0xf3,0x2f,0xc7,0xd7,0x8a,0xd2,0x01,0xa0, +0x79,0x29,0x0e,0xad,0x8f,0xd6,0xe5,0x75,0x85,0x12,0xca,0x9c, +0x0c,0xd4,0x98,0xc6,0x75,0x44,0x29,0x46,0x92,0x4b,0xa5,0x79, +0x15,0x28,0x00,0xfc,0xf4,0x6e,0x6b,0x18,0x84,0x5b,0x50,0x44, +0xe6,0xae,0x0a,0xbe,0x9b,0xa4,0xa7,0x92,0x1b,0x85,0x2f,0x0a, +0xf1,0xd9,0xdc,0x46,0x1e,0xca,0xe2,0x31,0x80,0x3c,0xf7,0x6b, +0x07,0xa5,0xc3,0xb7,0x7e,0x45,0x35,0xe4,0x5e,0xf6,0x01,0xc4, +0x54,0xea,0x80,0x4a,0xb0,0x1e,0x15,0x16,0xd9,0xfb,0xb6,0x4e, +0x30,0x2a,0x78,0xe4,0x8a,0xe5,0xc9,0x41,0x47,0x56,0xdc,0x60, +0x4e,0xfb,0xb1,0x71,0xbc,0x66,0xeb,0x4a,0x61,0x7c,0xba,0x86, +0x51,0x7d,0x9c,0x4f,0x48,0x0a,0xb3,0x27,0x98,0x3d,0x17,0xfd, +0x53,0x03,0x77,0x67,0xf9,0xec,0x3f,0xe1,0x33,0x4c,0xb5,0xc9, +0x43,0xff,0xa9,0x10,0xfa,0x4f,0x49,0xba,0x29,0xd0,0x75,0xc3, +0x74,0x91,0x7e,0xc3,0x07,0x09,0xd4,0x58,0xd1,0x2d,0x27,0x39, +0xb1,0xf4,0x4d,0x49,0xad,0x55,0x09,0xfe,0xdb,0xf0,0x51,0x3c, +0xdf,0x28,0x16,0x3d,0x2f,0x60,0xa6,0xd9,0x07,0x01,0xdd,0xff, +0xa0,0xe9,0x96,0xf0,0xc5,0x40,0xd1,0x5b,0x85,0x3c,0x9c,0x17, +0x16,0x4e,0xe1,0x66,0x21,0x07,0x3a,0x97,0x4b,0xaf,0xb1,0x01, +0xe0,0x3f,0x95,0xf8,0x59,0xaa,0xd3,0xbe,0x34,0x47,0x2b,0xb9, +0x49,0x7d,0x58,0x43,0x9b,0xd6,0xac,0x3e,0x09,0x92,0x0f,0x57, +0xba,0x4b,0xf0,0x72,0x38,0xaf,0xb8,0xe1,0x94,0x6d,0x7b,0x6f, +0xb1,0x4c,0x80,0x04,0xfa,0xce,0xd2,0xaa,0x5d,0xc2,0x57,0x71, +0xa1,0x82,0x9e,0x90,0xcb,0x57,0x3e,0xf0,0x6b,0x15,0xcc,0xfe, +0xf7,0xa1,0xcf,0xa9,0xee,0xe3,0xad,0xd8,0x9b,0x75,0x51,0x81, +0x95,0x5b,0x08,0xaf,0xdf,0xf3,0x58,0x63,0xbb,0xfa,0x45,0x03, +0xed,0xdd,0x4a,0x25,0xb3,0x22,0x59,0x5a,0x73,0xbe,0xad,0x15, +0x5c,0xa4,0xa2,0xdd,0xfc,0x1d,0xdc,0x7e,0x8b,0xe1,0xe4,0xcb, +0x86,0x2a,0x55,0x38,0xae,0x52,0xea,0xd9,0x4f,0x27,0xf6,0x41, +0xe3,0x41,0xa2,0x92,0x07,0x98,0x11,0x56,0x93,0x7b,0xe7,0xb5, +0x59,0xaf,0x4b,0x85,0x68,0x3d,0xf3,0xec,0x69,0xb8,0x33,0x8b, +0x6b,0x93,0x22,0xff,0xdc,0x1b,0x22,0x37,0x2a,0x67,0xd1,0xe2, +0x4f,0x79,0xf1,0xc6,0xad,0xe4,0x0e,0x23,0x11,0x66,0x91,0x19, +0xae,0x53,0x9c,0xb4,0xf1,0x18,0x10,0x9f,0xb6,0x9b,0xe7,0xf0, +0x64,0x0a,0x65,0x5d,0x1e,0xae,0x2b,0xc3,0x0d,0x9e,0x72,0xe5, +0x1e,0xa7,0x27,0x95,0xf4,0x9a,0xde,0x8f,0xa3,0x93,0x9d,0xa2, +0xc7,0xe3,0xd7,0xf4,0x78,0x7c,0x53,0x90,0x8a,0x2f,0xc7,0xaf, +0xd5,0x6e,0xca,0xe8,0xd5,0xc3,0xb1,0xb3,0xd0,0x2e,0xc8,0x7d, +0xd7,0xd2,0xf1,0xec,0x0e,0xe3,0x07,0x7b,0x6b,0xbe,0x93,0xb8, +0x9e,0x09,0x84,0x82,0xea,0x76,0x55,0xb2,0x17,0xf0,0x7d,0x96, +0xdf,0xff,0xde,0x5e,0xbb,0x76,0x96,0xf9,0x02,0x7b,0x05,0x1a, +0x78,0x91,0xcd,0x02,0x0d,0xba,0xc8,0x6e,0xc1,0x51,0xf9,0x43, +0x18,0x62,0x87,0x3c,0xf1,0xaf,0x64,0x8c,0x40,0xe0,0x88,0xb7, +0x45,0x99,0xff,0x11,0xa9,0xe3,0xd9,0x0c,0xb6,0x76,0x6b,0x76, +0x46,0x87,0x4f,0x71,0x0b,0x17,0x67,0x49,0xac,0xa1,0x4f,0xb4, +0xd7,0x33,0x39,0x8e,0x90,0xb8,0xde,0x62,0x40,0xe6,0x6f,0x9c, +0x79,0x5c,0x81,0xf8,0xfc,0x7f,0xee,0x58,0xad,0x8b,0x40,0x7e, +0xe0,0x42,0xad,0x41,0x60,0xeb,0x05,0xc4,0x3f,0xbc,0x46,0x6b, +0x98,0x64,0xf8,0xee,0x7b,0xa0,0x4f,0xe4,0x2c,0xd3,0x72,0x5f, +0x06,0xcb,0xf1,0xff,0x07,0xd7,0x60,0x69,0xfc,0x69,0xb8,0xd4, +0xca,0xa3,0x8f,0xbb,0x16,0x48,0xc8,0xdd,0xbb,0x9c,0xa3,0x92, +0x10,0x6f,0x06,0x7d,0x33,0xf8,0x9c,0xc4,0x45,0x8b,0x59,0x70, +0x9d,0x4a,0xc1,0x3c,0xab,0xb8,0x7b,0xd8,0xc2,0x23,0xe3,0x35, +0x2b,0x21,0xeb,0x7a,0x72,0xa7,0xc7,0x0e,0xa5,0x0a,0x5f,0x8f, +0xf2,0x38,0x72,0xf6,0x7e,0x79,0xf5,0x77,0xf3,0x1a,0x06,0x76, +0xb8,0x1a,0x51,0xfd,0x20,0x54,0xaa,0x7d,0x21,0x6c,0x4a,0x97, +0x93,0xf4,0x4c,0xa0,0xa1,0xfa,0xa4,0xbb,0x06,0xf8,0xe1,0x9d, +0x22,0xce,0x35,0x74,0xf3,0x22,0xba,0xde,0xbd,0x46,0x3a,0xa0, +0xb3,0x44,0xfc,0x45,0x3b,0x5d,0xf8,0xdb,0xec,0xa2,0xfb,0x90, +0x2f,0x46,0xf3,0xe5,0x85,0xe0,0x1d,0x12,0x03,0xe1,0x17,0x0e, +0x84,0xa0,0xfd,0xf3,0xb0,0xed,0xc1,0x81,0xb3,0xfb,0x32,0xd5, +0x7b,0x8c,0xea,0x83,0xc1,0x6c,0x4c,0x46,0x93,0x10,0x3c,0xd0, +0xf3,0x41,0x7a,0xc4,0x74,0xa9,0xeb,0x59,0x59,0x89,0x65,0xb3, +0x25,0x33,0x1f,0xd0,0xa2,0x91,0x12,0xbd,0x1a,0x29,0xd1,0x0c, +0x48,0xb1,0xbb,0x43,0x8b,0xde,0xc9,0xde,0xa7,0x17,0x78,0x7e, +0x27,0x0b,0x28,0x50,0x1f,0x56,0x86,0x35,0xe5,0xe9,0x9d,0x6b, +0xa8,0x1e,0x5b,0x96,0x8a,0xa1,0xc5,0x19,0x06,0xed,0x42,0xc5, +0xba,0x19,0x0b,0x83,0x81,0x86,0xfb,0xcf,0x3b,0x53,0x35,0x9c, +0xe1,0x1d,0x2f,0x7a,0xf8,0x07,0x4f,0x2c,0xef,0x98,0x4d,0xbc, +0xe7,0x7b,0x05,0x70,0xcc,0x97,0x95,0x98,0x75,0xae,0x6d,0x6e, +0xc0,0xef,0xe9,0x0b,0xbc,0xbb,0x27,0xdd,0xe9,0x5e,0xa2,0xfa, +0xa8,0x08,0xa4,0x86,0x6e,0x17,0x18,0x69,0x24,0xd1,0xe2,0x48, +0x82,0xf2,0xc8,0xb0,0x99,0x3a,0x62,0x08,0x9d,0xf7,0x35,0x8f, +0x1f,0x96,0xdf,0xff,0xb5,0x79,0x78,0x48,0xe5,0xab,0x55,0x7d, +0x75,0xd7,0x6b,0x54,0x10,0x34,0x70,0x53,0x05,0xe1,0x03,0x3b, +0x55,0xa0,0x12,0xd7,0xbe,0x92,0xf8,0x37,0xab,0x0c,0xca,0x03, +0xd4,0xd5,0x6c,0x8a,0x52,0x04,0xd9,0xc6,0xab,0xa7,0x97,0xa8, +0x3b,0xde,0xb4,0x8a,0x12,0x56,0x63,0x02,0x75,0x82,0xe8,0x61, +0x01,0xb4,0xa8,0x82,0x1a,0x27,0x75,0xf1,0xed,0x62,0x1d,0xaa, +0x79,0x93,0x44,0x13,0x81,0x03,0xbf,0x37,0x65,0x12,0x68,0xf2, +0x24,0x30,0x84,0x88,0x75,0x93,0x9b,0xd7,0x4c,0x37,0x4d,0x6a, +0xdf,0x75,0x8a,0x2d,0x83,0x68,0x6c,0xe6,0xa6,0xd9,0x8e,0x86, +0xd7,0xe5,0xe1,0x75,0x1d,0x5a,0x09,0x6d,0xbc,0xbd,0x35,0x48, +0xb0,0x34,0xd6,0x64,0x3a,0x34,0x63,0xc8,0x50,0x85,0x3d,0xe4, +0x71,0xed,0x08,0x6f,0x19,0xb3,0x6f,0xc6,0xd3,0x2f,0xc6,0x19, +0x60,0x33,0x18,0x1d,0xf7,0x1a,0xe5,0x5e,0x52,0xe9,0x37,0xca, +0xfd,0xa4,0x32,0x68,0x94,0x07,0x49,0x4c,0xb6,0xdc,0xef,0xec, +0xa0,0x26,0x20,0x8a,0x14,0x50,0x3d,0xa0,0x05,0x7a,0x36,0xba, +0x54,0x03,0xbf,0x8f,0x82,0x69,0x09,0x0b,0x58,0x96,0xd1,0x90, +0x2d,0x18,0xbc,0xbe,0xbd,0xb3,0xdf,0xe5,0xa9,0x33,0x34,0x36, +0x85,0x45,0xd8,0x6f,0x8a,0x55,0x72,0x39,0x6a,0xb8,0x2a,0x28, +0x66,0xfc,0xa6,0xc9,0xe3,0x90,0x27,0x6b,0xc6,0x7a,0xf1,0xbd, +0x4e,0x1e,0x23,0x03,0x2a,0x42,0x14,0x4e,0xcb,0xa7,0x74,0x81, +0x9d,0x3c,0x36,0xed,0x12,0x17,0x67,0xc4,0x40,0x90,0x85,0xfc, +0x6a,0x84,0xec,0x2f,0x6b,0xc7,0xd2,0xb8,0x79,0xd5,0x4c,0xcd, +0x5e,0x36,0x59,0xb2,0xd8,0xb3,0xcf,0xfc,0x9e,0x59,0xe7,0x15, +0xe5,0xb0,0xeb,0xa9,0x6c,0x6b,0x79,0x06,0x09,0x57,0xdd,0xc2, +0x73,0x61,0xcd,0x83,0x27,0x6b,0xd7,0xc3,0xdf,0xc6,0x09,0xe0, +0xa9,0x70,0xce,0x87,0xb7,0x3b,0x06,0x4d,0x23,0x52,0x71,0x30, +0x57,0xd4,0xb2,0x34,0x5c,0xe1,0x0d,0x4c,0x93,0xb3,0x50,0x76, +0xf6,0x19,0xc8,0xb5,0xdd,0xba,0x71,0x3b,0x78,0xab,0x0c,0x98, +0x02,0x19,0xd7,0x9c,0x46,0x5b,0x4a,0xf9,0x3a,0x49,0xf9,0xf3, +0xea,0xbc,0xae,0xa7,0xe4,0xd9,0xc6,0x05,0xd6,0x7c,0xf7,0x9d, +0x77,0x3b,0xa2,0x8c,0x83,0x07,0x69,0xee,0x78,0xc2,0x95,0x9e, +0x0e,0x5e,0xb2,0x29,0x09,0x8a,0x23,0x3d,0x0f,0xdb,0x64,0x8e, +0x4b,0xba,0x1e,0x6b,0x9f,0x98,0x9c,0x6f,0x2b,0xef,0xff,0xba, +0x56,0xef,0xff,0xba,0xa9,0x9a,0x8c,0xae,0x2d,0x52,0x39,0xed, +0xbf,0xb3,0x6a,0x7d,0x2d,0x60,0x50,0xa8,0x02,0xd8,0xb8,0xfe, +0x56,0x45,0x2e,0x68,0x88,0x10,0x25,0xd9,0xc3,0xc3,0x13,0x17, +0x59,0xcd,0x43,0x2e,0xb2,0x9a,0x95,0xbc,0xf9,0x1f,0x9f,0x93, +0xde,0x78,0x35,0xa7,0xc1,0x65,0xc9,0xc2,0x8b,0xa5,0x10,0x29, +0xe7,0xff,0x15,0x84,0x84,0xf7,0x7f,0xfd,0xed,0x2f,0xf0,0xe7, +0xfd,0x7f,0x02,0x16,0xd2,0xa9,0x7c,0x81,0x37,0xc2,0x7b,0xaa, +0xbc,0x00,0x46,0xad,0x85,0x86,0xc0,0xee,0x95,0xb3,0xb0,0x09, +0x57,0x68,0x52,0xba,0xa9,0x48,0x0c,0x19,0x72,0x85,0xd6,0xf7, +0x26,0xe3,0x21,0xf6,0x81,0xa7,0x14,0x62,0x63,0x6f,0x3c,0x31, +0x17,0xe6,0x4a,0x61,0x52,0xca,0x81,0x97,0xd7,0xc2,0xf2,0xec, +0x3c,0xc4,0xf6,0x3a,0xef,0x22,0x38,0x28,0xc6,0x17,0x94,0xd7, +0x7e,0x25,0xfd,0x27,0x00,0x4c,0xe2,0x07,0x8d,0xe8,0x40,0x4a, +0x3e,0x59,0xfb,0xac,0xb1,0x16,0xb6,0x40,0xe3,0xcf,0xc9,0x23, +0x98,0xc5,0xd3,0x87,0x9f,0xc2,0xdc,0x9e,0x36,0x1f,0xe1,0xf4, +0x4e,0xb7,0x39,0xf0,0x87,0x94,0x3f,0x1b,0x44,0x64,0x0e,0xfd, +0x27,0xfc,0xe0,0x44,0x89,0xd7,0xbd,0x3b,0x5e,0xf5,0x28,0x74, +0x0a,0xa3,0x33,0x68,0x77,0x5f,0x8a,0x73,0xe1,0x03,0x63,0xba, +0xa2,0xb4,0xd7,0x7b,0xae,0xbb,0x25,0x67,0x68,0xe8,0xc9,0xcf, +0xb4,0x6d,0xb6,0x5a,0xce,0x57,0xcb,0x3b,0x89,0xae,0xd7,0x9a, +0xfc,0x15,0xd4,0x4b,0x1e,0x29,0x91,0xa0,0x1f,0x05,0x94,0x37, +0x9b,0x12,0xc1,0x5e,0x57,0x48,0x26,0x39,0xbc,0x3a,0x1a,0x66, +0xdf,0xe3,0xa0,0x6b,0xdb,0x77,0xaf,0xd5,0x43,0xd3,0x90,0x95, +0x85,0x4e,0x3b,0x91,0x2d,0x7f,0xaa,0x76,0x10,0x36,0x6e,0xe5, +0x5f,0xd6,0x38,0xe5,0x2a,0xa7,0xb1,0x37,0x21,0xb8,0xac,0xca, +0x26,0x1d,0x28,0x5f,0x0e,0x6b,0xe9,0x86,0x7b,0xa1,0xed,0xe4, +0x3f,0xb5,0xfd,0x7c,0xda,0xc9,0x2b,0x5e,0x9e,0x8a,0x8e,0xff, +0x29,0xeb,0xf8,0xab,0x53,0x79,0x3b,0x70,0x2a,0x6f,0x07,0x5a, +0x2e,0x73,0xe4,0x51,0x65,0x28,0x1e,0xae,0xe4,0xb4,0xe0,0x7e, +0x68,0xbe,0x05,0x62,0x28,0x99,0x5a,0xe2,0xd6,0x5b,0x0f,0x0c, +0x2d,0x10,0xa4,0xbd,0x7a,0x2f,0x50,0x6b,0x00,0x45,0xb2,0x18, +0xfe,0x26,0x5d,0x0b,0x95,0x94,0x46,0x7c,0x62,0xf2,0x94,0x73, +0x2f,0xb0,0x30,0xac,0x10,0xec,0xef,0x8e,0x1e,0xbb,0x02,0x0a, +0x0e,0x3a,0xac,0x25,0x8c,0xb6,0x95,0xbe,0xff,0x9b,0x3b,0xb7, +0xb2,0x05,0x78,0x33,0xab,0xba,0xc8,0xc8,0x9c,0x79,0x3f,0x64, +0xfe,0xba,0x03,0x1d,0x1a,0xf7,0x45,0x08,0xd1,0x4c,0x70,0xaa, +0x0d,0x30,0x34,0x3b,0xa7,0xb0,0x19,0x0d,0xf2,0xb4,0x1a,0x2e, +0xda,0x7f,0xf5,0x11,0x47,0xc0,0x7e,0xf8,0xfd,0x7f,0xfa,0x4d, +0xb1,0x3e,0x74,0x6c,0xa3,0xc2,0xb1,0x8a,0x7e,0xdf,0x44,0x66, +0x2b,0xc5,0x6a,0xe7,0x52,0x9d,0x9a,0x45,0x58,0x1c,0x4d,0xa1, +0x70,0x65,0x5d,0x84,0xb9,0x53,0xb3,0xbe,0x0a,0x95,0x18,0x73, +0xb7,0xce,0xbf,0xba,0x41,0xfb,0x44,0x0b,0x3d,0xcb,0x7c,0xb0, +0x64,0x74,0xb7,0x20,0x24,0x1b,0x65,0x4f,0x14,0x62,0xb2,0x5a, +0xf6,0xd9,0x25,0x0c,0x85,0x08,0x8f,0x7e,0x5d,0x91,0x02,0x60, +0x6f,0x7a,0x4e,0x26,0xcf,0xbc,0x87,0x78,0x72,0x80,0x91,0x2b, +0xa4,0xc7,0xeb,0x4b,0xb3,0x02,0x07,0xa5,0x8e,0x7f,0xfb,0x73, +0x07,0x5f,0x04,0xfc,0x1f,0x7c,0xd5,0x86,0xea,0xea,0xef,0xff, +0x56,0x81,0xc0,0x0e,0x70,0xbb,0x16,0xf2,0x1b,0xb3,0x4a,0x23, +0x04,0x88,0xbe,0xbd,0xc5,0xf8,0x4e,0x20,0x15,0x04,0x50,0xde, +0xff,0x8d,0xe0,0x98,0xb3,0x5d,0x1d,0xff,0xdb,0x5f,0xde,0xff, +0x67,0x64,0x81,0x9b,0x49,0x34,0x2c,0xd7,0xca,0xd7,0xb4,0x03, +0x44,0x6b,0x07,0x39,0x72,0x0d,0x94,0xab,0xee,0x97,0x41,0x4d, +0x39,0x33,0x6d,0xfe,0x2b,0x48,0xa0,0x59,0x4a,0x14,0xb9,0x93, +0xa0,0x3b,0x29,0x6a,0x6a,0xa2,0x8d,0x80,0x44,0xa6,0xfb,0xb1, +0x32,0xfb,0x21,0x89,0xd7,0x43,0x61,0x67,0xa9,0x2a,0x93,0x7b, +0xb1,0xab,0xf5,0x3b,0xa8,0xe7,0x1c,0xf6,0x05,0x6c,0x54,0x68, +0x1e,0x17,0x0d,0x29,0x7b,0xef,0x42,0xd9,0xc4,0xc1,0xc6,0x7d, +0x27,0xad,0x23,0xaf,0x43,0xaf,0x16,0xd7,0x98,0x80,0x55,0x57, +0xd6,0xf4,0x4a,0x19,0xbe,0x0a,0xf4,0x18,0x36,0x9c,0xeb,0xfd, +0x7f,0x56,0xdf,0x72,0xae,0xf7,0xff,0xa9,0xd6,0xba,0x64,0xf5, +0x5a,0x97,0xb4,0xf6,0xf0,0xdf,0xea,0x02,0x95,0x8d,0x2e,0x20, +0x3a,0x1b,0x52,0xc8,0xc1,0xf8,0xaf,0xe2,0x72,0xc3,0x46,0x13, +0xce,0x7f,0x15,0xd3,0x21,0x34,0x91,0x3a,0xb9,0x51,0x8e,0x15, +0x90,0x6e,0x6e,0x88,0xde,0xd5,0x3f,0x45,0x6f,0x55,0xff,0x25, +0x54,0x04,0xfa,0x98,0xe6,0xee,0xa3,0x62,0xe5,0x3a,0x4f,0xc5, +0xca,0xdb,0x9d,0x54,0x04,0x12,0x9b,0xce,0x41,0xfd,0x90,0x9d, +0xfc,0xb4,0xf3,0x95,0x89,0x59,0x04,0xa4,0xf1,0x45,0x6f,0x5c, +0x16,0xb6,0x51,0x12,0x97,0xdb,0xcb,0xde,0xa5,0xe3,0x8d,0xee, +0xbf,0x9c,0x37,0xd5,0xa6,0xd6,0xb6,0x78,0xca,0x86,0xe2,0x1d, +0x02,0xe2,0x3f,0x8b,0x21,0xa1,0x2b,0x1a,0x2c,0x14,0xbd,0x5a, +0x87,0x0a,0x76,0x24,0x2f,0x67,0xf8,0x54,0xd7,0x79,0x0e,0xb2, +0x9c,0x1d,0x63,0x91,0x16,0xfe,0x39,0x86,0x54,0x90,0x75,0xf1, +0xb3,0xe2,0xe0,0xf0,0xdb,0xff,0xc1,0x57,0x9a,0x54,0xa7,0xdc, +0xfc,0xba,0x69,0x58,0x97,0x3c,0x93,0xa9,0x3d,0x2a,0x93,0xf7, +0xb2,0xb6,0x67,0x69,0x47,0x2d,0x09,0xeb,0xb6,0x57,0x2b,0x54, +0xd5,0x82,0xaa,0x97,0xed,0x65,0x35,0xa5,0x32,0xc5,0x06,0xf3, +0x86,0xe1,0x5b,0xa7,0x13,0x68,0x43,0x35,0x5b,0x58,0x45,0xa6, +0x65,0x6c,0xc3,0x48,0xf7,0x65,0xdc,0x8d,0x03,0xeb,0xa1,0x3b, +0xfa,0xd8,0xd0,0x44,0xf6,0xa8,0x7c,0x4c,0x97,0x3b,0x67,0xe7, +0xd4,0x36,0xbd,0xd2,0x48,0x21,0x77,0x98,0xc1,0xb1,0x1f,0xc3, +0x7b,0xc2,0x1e,0xc9,0xc9,0x43,0x31,0x81,0xce,0xfa,0xfe,0x6d, +0x6d,0xec,0x89,0xfc,0x8e,0x3c,0x69,0x74,0xaa,0x94,0xab,0x45, +0x7f,0xe3,0x4a,0x93,0xf1,0xff,0xfe,0xeb,0xea,0xfb,0xff,0x8a, +0x7f,0x17,0x99,0x60,0x7e,0xef,0x7f,0xbe,0x00,0xb6,0x1d,0xac, +0x26,0xa3,0x1f,0xa3,0x0d,0x6a,0x87,0xaa,0xa1,0xdc,0x94,0xa2, +0xf3,0xbb,0xda,0x02,0x9d,0xb0,0x72,0x02,0x40,0xc2,0x34,0x6d, +0x9c,0x6a,0x93,0xe4,0x59,0xd1,0xba,0xba,0xda,0xd4,0x2e,0x7b, +0x73,0xbb,0xc1,0x33,0xdb,0x3b,0x96,0x3e,0x0b,0x28,0x68,0xd0, +0xf8,0x09,0xcf,0xea,0xd6,0x80,0xc6,0x5a,0xbb,0xbe,0x32,0x68, +0x50,0x02,0xa1,0xb1,0x36,0x68,0xac,0x0b,0xd0,0x70,0x36,0x93, +0x3e,0x1a,0xeb,0xbd,0x68,0x64,0xb3,0xd5,0x62,0x30,0x72,0x3d, +0x87,0xd5,0x38,0xca,0xb7,0xf5,0x0d,0x02,0x18,0xfa,0x3c,0x74, +0x72,0x71,0x54,0x5e,0xec,0x70,0x86,0xbc,0xf7,0x9e,0x8b,0x46, +0xfb,0xc6,0xb1,0x56,0xb1,0x69,0xc0,0x50,0xdf,0x38,0x3a,0x73, +0x18,0x31,0xd8,0x38,0x26,0x30,0x36,0xb8,0x41,0xdc,0x38,0x56, +0x36,0x30,0xe2,0x2d,0x5a,0x5d,0xdf,0x34,0x2c,0xeb,0xae,0xa1, +0xd8,0xdb,0x8d,0x13,0x89,0x39,0x29,0x72,0x9d,0x40,0x64,0xe2, +0xe4,0xc4,0xe2,0x1b,0x27,0x92,0x72,0x26,0xe8,0xc4,0x5d,0x78, +0x85,0xe6,0x3a,0xe7,0x6e,0xfc,0xe8,0xf4,0xa2,0x07,0x13,0x37, +0xd7,0x8d,0xcf,0xc2,0x2a,0x58,0x0b,0x96,0x77,0x93,0xe8,0x34, +0x1b,0xb7,0x9e,0xb0,0x3b,0xaf,0x5b,0x83,0x4f,0xb1,0xff,0xfc, +0xce,0xba,0x6f,0x93,0x51,0xf2,0xb9,0x6d,0xd8,0xb2,0x9c,0x0e, +0xe3,0xf2,0x5b,0xf5,0xd4,0x46,0x0d,0xab,0x4b,0x8c,0x59,0xa7, +0x4f,0xcb,0xd0,0xe4,0xca,0xe7,0xf0,0x17,0x08,0x89,0xa1,0x0d, +0x85,0x80,0x69,0xaf,0x21,0x94,0x51,0x08,0xa8,0x54,0x34,0xcb, +0xab,0x75,0xec,0x0a,0xee,0x4e,0xd2,0xb5,0xb2,0xad,0x44,0x1d, +0xc0,0x4d,0x79,0x13,0xc7,0x9e,0x50,0x9f,0x7f,0x39,0x7d,0xb2, +0x6e,0x78,0xe7,0x8d,0x0d,0x2f,0xbf,0x66,0x48,0xa7,0xcd,0xe8, +0x00,0x9a,0x8f,0x78,0x87,0x2e,0x29,0x72,0x3c,0xc3,0x3b,0x93, +0x3b,0x54,0x50,0x9c,0x15,0x33,0x67,0xd8,0xcb,0x77,0xe6,0x50, +0xb0,0xa0,0x2e,0xed,0x81,0x7f,0xe4,0xdb,0xf9,0x8f,0xe9,0x71, +0x2a,0xad,0xee,0xff,0x7f,0x6d,0x7b,0x88,0x27,0xc4,0xb6,0x87, +0xdd,0xab,0x15,0x4c,0xe6,0xc5,0x7e,0x4b,0x96,0xa1,0xbd,0xff, +0x25,0x90,0x72,0x59,0xd1,0xf1,0x62,0x30,0x4c,0x54,0x3c,0x79, +0xdd,0x2e,0x4b,0x7d,0x90,0x4f,0x3b,0xa8,0xe2,0x5c,0x15,0xd7, +0x5a,0x8b,0xc9,0xa5,0xee,0xb2,0x22,0xa5,0xef,0xad,0x24,0x87, +0xa4,0xf9,0x2d,0xd8,0xed,0x26,0x21,0xf5,0x37,0x26,0x79,0x5d, +0xd6,0x02,0xfa,0x36,0x02,0x2b,0x6a,0x05,0x35,0x99,0xbe,0xdb, +0x06,0xc9,0x05,0x6a,0x0f,0x45,0xe5,0x8b,0x0b,0xe9,0x7b,0xeb, +0xbc,0x77,0xb6,0xeb,0xf1,0xe5,0x6a,0x79,0x01,0xab,0x1b,0x1d, +0x3e,0x28,0x5e,0x14,0x2c,0x2b,0xd9,0x74,0x5f,0xf8,0x72,0x7b, +0xdd,0xd8,0x68,0xf3,0x5b,0xf7,0x36,0x25,0x90,0x11,0xe5,0xe6, +0xbe,0x31,0x1e,0xbb,0xde,0x96,0x0b,0xbc,0x3d,0xbc,0xf5,0x74, +0x8a,0xbb,0x46,0x13,0x5c,0xa3,0xb0,0x4f,0x8b,0x37,0xcd,0x8f, +0x4c,0x35,0x48,0xa9,0x35,0xd1,0x6f,0xff,0x41,0xec,0x32,0xb0, +0x9d,0x33,0x20,0x84,0x07,0x16,0xdf,0x41,0x5c,0xa4,0xfd,0x8b, +0xdc,0x35,0x50,0xbf,0xfd,0x07,0x22,0x3b,0xd0,0xe6,0xa4,0x08, +0xbd,0xff,0x38,0x3c,0x24,0x49,0x71,0x50,0x07,0xe0,0x20,0xd2, +0x19,0x1c,0x8b,0xb5,0xad,0x9f,0xcd,0xa6,0xe4,0xee,0x71,0xf2, +0x61,0xda,0xd6,0xde,0x55,0x1b,0x24,0x6a,0x83,0x5c,0x78,0xe1, +0xd6,0x68,0x7d,0xc9,0xb3,0x26,0x3a,0xc2,0x5c,0x12,0xc1,0xa7, +0x71,0x91,0x12,0xf6,0xb4,0x8c,0xb4,0xfd,0xd2,0xd3,0xc0,0xa6, +0x38,0x23,0xcf,0xe7,0x2d,0x09,0x3a,0x34,0x0e,0x8d,0xd7,0xa0, +0x15,0x26,0x34,0x14,0x58,0xf9,0xed,0x2f,0x75,0xbc,0xf1,0x99, +0x92,0x45,0x29,0x34,0xff,0x96,0x74,0x3c,0x45,0x6e,0xbe,0x1b, +0x42,0x37,0x98,0x5c,0x45,0xdd,0x37,0xd2,0x54,0x37,0xc9,0x4b, +0x8a,0xa8,0x2f,0xd9,0x12,0x93,0xfa,0x52,0x4f,0x00,0x5e,0xeb, +0x50,0x31,0xdc,0xd1,0xeb,0x9e,0x82,0x1c,0x7a,0x3f,0xd5,0xee, +0x2f,0xab,0x1b,0xe8,0x41,0x1c,0x00,0xd9,0x39,0x10,0x43,0x5b, +0x7a,0xd2,0x8c,0xe2,0xaa,0x71,0xef,0x57,0x00,0xd7,0x8b,0x1d, +0x12,0xc0,0xa0,0xf6,0x25,0xf4,0xbf,0x4a,0xea,0xd3,0x58,0xf7, +0x8d,0x55,0xff,0x36,0x9b,0xe8,0xcb,0xd1,0x62,0x80,0x37,0x25, +0x3b,0x15,0xf1,0xc7,0x3c,0xe3,0x2f,0xef,0xcd,0x1c,0x5f,0x55, +0xed,0x2e,0x2f,0xdf,0xd5,0x5f,0x7d,0x50,0x57,0xef,0xee,0x3f, +0xe9,0xb7,0x6a,0xd0,0x6f,0xf8,0x53,0xa5,0xa2,0x5f,0x49,0x4f, +0xd5,0xa7,0x15,0x9c,0x37,0xc3,0xf7,0xcc,0x53,0x7e,0x0e,0x5e, +0x78,0x1a,0xd3,0xba,0x6f,0xf7,0x7d,0x75,0x97,0x5e,0xfe,0x57, +0xd5,0xfb,0x74,0xad,0xd3,0x2d,0xb9,0x6e,0xd0,0xfd,0x93,0x3f, +0x21,0x32,0x2c,0x5a,0xc0,0xf9,0x05,0x5e,0xe2,0x34,0xa0,0xef, +0x0b,0x6e,0xe1,0xf4,0x8b,0xab,0xa7,0x2b,0xd8,0x7f,0x15,0x1c, +0xdd,0x9b,0x9c,0x8a,0x0f,0x81,0x2f,0xf9,0x30,0x58,0xd9,0xb3, +0xdc,0x4b,0x7b,0xae,0xeb,0xde,0x2d,0x5c,0x3a,0x27,0xdc,0xba, +0x0b,0x2e,0xc3,0x93,0x64,0xae,0xff,0x8a,0x27,0x64,0x39,0x22, +0xbc,0x74,0xce,0x07,0x75,0xc1,0x2b,0x34,0x1d,0xda,0xd1,0x78, +0x76,0x5c,0xd8,0xf4,0xf8,0x2a,0x6e,0x5d,0xb6,0xae,0xb6,0xea, +0xb2,0xe8,0x88,0x59,0xd7,0x61,0x92,0xfe,0x19,0xf5,0xec,0x38, +0xbd,0xe7,0x9a,0x6c,0x62,0x51,0x55,0xf8,0x48,0x12,0xab,0x61, +0xbf,0x17,0x52,0x95,0xbe,0x8d,0xd0,0xae,0x84,0x7f,0xfb,0x73, +0x99,0x17,0x29,0x9c,0xf3,0x0c,0xe6,0x51,0xdc,0x76,0x10,0x3a, +0x39,0xa1,0xab,0xc1,0xb7,0x64,0x14,0xa8,0xfa,0xb6,0xab,0x28, +0x5c,0xe1,0x70,0xe5,0x6d,0x17,0x77,0x79,0xf4,0x2c,0x53,0x57, +0x82,0xa6,0xad,0xf8,0x4a,0xcc,0xb4,0xd7,0x6f,0x0b,0xb7,0xb1, +0xf8,0x8e,0x1c,0xa5,0xe3,0xd1,0x37,0x01,0x4b,0x3a,0x82,0xaf, +0xac,0xae,0xde,0xca,0xe9,0x8d,0x10,0xe4,0xd7,0x28,0xa9,0x7c, +0x0e,0x23,0x35,0xa9,0x7e,0x4e,0xef,0x66,0x42,0x09,0x8c,0x38, +0xf9,0x2f,0xb2,0xa0,0xc9,0x0a,0x1e,0xf0,0x73,0x76,0x75,0x7e, +0x8a,0xc2,0x83,0x73,0x36,0xed,0x4c,0x4b,0x94,0x42,0xfe,0xe3, +0x6d,0x9c,0x3c,0x84,0x73,0x1e,0x15,0xdb,0xd7,0x09,0xee,0xfe, +0x26,0xbc,0x1b,0x06,0xd9,0x08,0x68,0x42,0x87,0x1b,0x40,0x12, +0xd9,0xbc,0x87,0x76,0xce,0xc7,0xf2,0x25,0xce,0x6d,0xb1,0x6a, +0xfd,0x0e,0xff,0x0c,0xb7,0x7d,0xa7,0x84,0x06,0x14,0x5e,0xc7, +0xea,0x6c,0xe3,0x46,0x6c,0xac,0x6f,0xdb,0xd1,0x19,0xa0,0x2d, +0x6e,0xf2,0xe9,0x80,0x7c,0x48,0x80,0xf0,0x5d,0xff,0x38,0xee, +0x70,0x25,0xf2,0x76,0xb7,0x72,0xb6,0x76,0xf3,0x41,0xba,0xaa, +0x9c,0x6d,0x82,0xa8,0x6e,0xdc,0x92,0x42,0x8c,0x0a,0x5e,0xd5, +0x69,0x22,0x58,0x9c,0xed,0xcb,0x2f,0x3f,0xaf,0xcd,0x1a,0x5a, +0x3b,0x37,0x61,0xad,0xa9,0x4f,0x1e,0x7f,0xb6,0x7a,0x8b,0x6b, +0xfa,0xe6,0x47,0xb5,0x71,0x83,0x3f,0x29,0x69,0x21,0xf9,0xfb, +0x5e,0xac,0x46,0xa1,0x21,0x13,0x9d,0x11,0xf7,0x0c,0xbd,0x85, +0x9b,0xfa,0xbf,0x47,0x9b,0xd4,0xdd,0x82,0xbc,0x1d,0x6d,0x74, +0xa7,0xa4,0xb5,0x4f,0x5c,0xfb,0xd5,0xb5,0xf5,0x7d,0x2f,0x95, +0x4e,0x49,0x6f,0x2b,0x6e,0xad,0x59,0xf3,0xb3,0xb6,0xb9,0x5f, +0xc1,0x8d,0x29,0xb8,0x91,0x82,0xba,0x55,0xf7,0x2a,0xae,0x33, +0x6b,0x20,0x12,0x16,0x50,0x2e,0x41,0xee,0x05,0x2e,0xa0,0x51, +0x49,0x17,0x2a,0xa5,0x29,0xfa,0xa9,0x9e,0x9d,0x3d,0x38,0xed, +0xb8,0x40,0x4f,0x5b,0xd1,0x0e,0xa2,0xdb,0x53,0xbd,0xd9,0x22, +0xab,0xe1,0xb9,0xc0,0x29,0x9b,0xa4,0xf3,0x7b,0x25,0x26,0xda, +0x33,0xee,0x7e,0xe5,0xd2,0x04,0xdd,0x2d,0xf7,0x42,0x5f,0x67, +0xd6,0xd4,0x90,0x30,0x83,0x2a,0x1c,0xef,0x3f,0xba,0xc7,0x15, +0x9e,0x89,0x13,0x97,0xd3,0xbc,0x4c,0x49,0x71,0x26,0x6e,0x8f, +0x1e,0xe3,0x37,0xfe,0x20,0x60,0xce,0x7e,0x51,0x8a,0xf7,0x14, +0x7d,0x46,0xee,0xb8,0x43,0x00,0x3b,0x6b,0xa8,0x94,0xfe,0x54, +0x2a,0x84,0xf6,0x6a,0x39,0x9a,0x1b,0x28,0xa1,0x51,0x6f,0x67, +0x48,0xa2,0x9e,0x1a,0x85,0xd9,0xf4,0xf8,0xf2,0x22,0x3d,0x21, +0x1d,0x35,0x7c,0x10,0x85,0x4a,0x6a,0xc6,0x01,0xce,0x9c,0xa4, +0x3f,0x9a,0xdb,0x7e,0x5f,0x52,0x64,0x21,0xa7,0x12,0xcd,0x7d, +0x07,0x42,0x38,0xed,0x96,0xfe,0xc0,0x05,0xcd,0x0c,0x40,0x8e, +0x34,0xfc,0xd2,0x73,0x72,0x43,0x4b,0x71,0x7a,0xb4,0xef,0x6c, +0xc4,0xe7,0x23,0xc0,0x7e,0xf4,0xaf,0x6a,0xca,0x1f,0xb0,0x29, +0x6e,0x23,0x12,0xc8,0xaf,0x31,0x34,0x73,0xd8,0xbd,0x10,0x7d, +0x7a,0x86,0x3e,0x3c,0xfe,0x95,0x24,0x0f,0xbc,0x35,0xe5,0x28, +0x7d,0x37,0x9e,0xcf,0x7a,0xe4,0xd6,0x7c,0xf2,0xdd,0x7c,0xa4, +0x17,0x22,0xb3,0x0e,0xdd,0x14,0xcd,0xd8,0x4f,0x1e,0x76,0x76, +0xf2,0x9e,0xac,0x03,0xe4,0xc6,0xd8,0xe6,0xf9,0x3d,0x1a,0x86, +0xd3,0x0a,0x4e,0x59,0x2d,0x9b,0x8c,0x07,0xa3,0x48,0x7b,0x63, +0xb3,0x9e,0xf5,0x54,0x01,0x56,0xaf,0xf1,0x14,0x13,0x0d,0xa3, +0x04,0x98,0xed,0x6f,0x8b,0x37,0x62,0xee,0x68,0xcd,0xd1,0xdd, +0xad,0x69,0x14,0xb6,0x26,0x72,0x17,0x4a,0x93,0x53,0x2f,0x78, +0xfb,0x5b,0x73,0x22,0xf9,0x7d,0x12,0x34,0xbb,0x5d,0x7d,0x46, +0x2b,0xa8,0x9f,0x18,0x7a,0x76,0xe3,0x7b,0xb6,0xfd,0x5f,0xdc, +0x6a,0x81,0xfe,0x0f,0x77,0x55,0x00,0x6f,0x29,0x85,0x49,0x48, +0xd5,0x01,0x8d,0x6c,0x22,0x06,0x23,0x75,0xc4,0x41,0x1a,0xe4, +0x10,0x1b,0x9c,0x3b,0xd3,0x2b,0xcd,0xf8,0xae,0x59,0x93,0x6e, +0x0d,0x7e,0x5d,0xf5,0x86,0xe9,0x7e,0x50,0x3c,0x3a,0x4b,0x25, +0xf4,0xfd,0xe8,0x0c,0xd9,0xd4,0x74,0x94,0x5a,0x36,0x4c,0x19, +0xb2,0xa6,0x93,0xa2,0x29,0xcb,0x71,0x9a,0xa0,0x20,0x8e,0x15, +0xa0,0x61,0x55,0x80,0x51,0x49,0x4b,0xff,0x5e,0xaa,0xb0,0x59, +0xb1,0x66,0x99,0xfc,0x1e,0xd4,0x8f,0x62,0x7a,0xef,0xc7,0x66, +0xc4,0x28,0x32,0x31,0x91,0x34,0xbb,0xf2,0x47,0xd2,0x75,0x6a, +0xc7,0xc0,0x18,0xed,0xc2,0x05,0x88,0x1e,0xc3,0xf6,0x79,0x69, +0x31,0x49,0x1c,0x24,0xe7,0x63,0x2c,0x53,0xa9,0x28,0xc1,0xe3, +0x19,0x56,0xd9,0x20,0xf3,0x54,0xf8,0x57,0x23,0xd1,0x20,0xf3, +0x54,0xe4,0x5a,0x52,0xa3,0x85,0x66,0x89,0x9d,0x1c,0x88,0xe6, +0xd2,0xa4,0x7b,0x18,0x3a,0x5e,0x23,0x9a,0xed,0xf1,0x93,0x00, +0xb9,0x36,0xd5,0x8e,0x8e,0xf5,0x3c,0xa4,0x1c,0x7c,0xc7,0x5d, +0x8d,0xdd,0xab,0x0f,0xac,0x79,0x2b,0x74,0x16,0x77,0x28,0x6e, +0x05,0xed,0x80,0xf0,0x15,0xa0,0x71,0x48,0x77,0x8a,0xb3,0x64, +0x9f,0x18,0xd8,0x13,0x02,0xee,0xcc,0xab,0x7b,0x87,0xe1,0xae, +0xd1,0x60,0x56,0x81,0x79,0xc3,0xf4,0x18,0x0a,0xf4,0xbd,0x14, +0x76,0x29,0x3a,0x13,0xac,0x99,0xf3,0xc4,0x65,0xaf,0xa6,0xd3, +0xdb,0xc0,0x4a,0xe1,0xf2,0x61,0x56,0x07,0xe0,0x8a,0x84,0x8a, +0x36,0x9d,0x32,0x48,0x58,0x4d,0x7e,0x96,0xea,0x7b,0xe5,0x68, +0x8e,0x46,0x41,0xab,0xd4,0xeb,0xb1,0xe2,0x70,0x42,0x61,0xa0, +0xa9,0x5d,0x3e,0x74,0xb9,0xc2,0xa6,0x7e,0xde,0xcb,0xc6,0x99, +0x59,0xdd,0xc8,0xf8,0x8f,0x37,0xc5,0xdc,0x6f,0xcc,0x15,0x34, +0x07,0x19,0xda,0x36,0x1e,0xdf,0xc9,0x8c,0xf5,0x43,0x99,0x31, +0x31,0xf2,0x3a,0x45,0xb3,0x2c,0xfc,0x7f,0x64,0x33,0x27,0xb4, +0x08,0x42,0xfa,0x26,0x3d,0xd9,0xd0,0xed,0xd1,0x06,0x87,0x1e, +0xce,0x9e,0xb2,0xa8,0x42,0x7e,0x7a,0xd6,0x0d,0x3f,0x2f,0x4a, +0xee,0x24,0xf6,0xc5,0x6c,0xf9,0x30,0x0a,0xdb,0xf6,0xf9,0xe8, +0x7a,0x3c,0x5a,0x1c,0x41,0x7d,0x31,0x15,0xbb,0x77,0xfe,0x4d, +0xcc,0xfc,0x5f,0xb0,0x42,0x4c,0xab,0xb8,0x3a,0x63,0x7f,0xa5, +0xd8,0x61,0x63,0xa7,0x93,0xe6,0x6b,0xed,0xb8,0x06,0x3f,0xb9, +0xd0,0x98,0x97,0x94,0x8d,0x4d,0xd9,0x98,0x94,0xa4,0xeb,0x2d, +0x32,0x0e,0x0a,0x34,0xc1,0x00,0x10,0xc4,0x24,0xd8,0x75,0x89, +0x7f,0x36,0x2b,0x46,0x00,0x1d,0xe6,0xe3,0xfb,0x48,0x5e,0x04, +0xdf,0x91,0x12,0x0a,0xba,0xfc,0xe1,0xdd,0x5d,0x6e,0xd8,0x7f, +0x2c,0x5d,0x62,0xf6,0xb2,0x01,0x0b,0x40,0x1f,0x37,0xb8,0x2b, +0xc5,0xe3,0xe2,0x51,0x9e,0x5c,0x01,0x8d,0x1c,0xc2,0x30,0xf9, +0x4d,0x2b,0x3f,0xa0,0xa7,0x69,0xac,0x7f,0x40,0x4f,0xc7,0xaa, +0x5a,0x1d,0x9b,0x01,0xf8,0x7f,0x63,0x7f,0xde,0xaf,0xc3,0x82, +0xcd,0x83,0x21,0x3f,0x01,0x1c,0x17,0xd3,0xfe,0x32,0x9d,0x56, +0x1e,0x52,0x0f,0x70,0x07,0x30,0xfd,0x1f,0x7a,0xcd,0xfb,0xdd, +0xf4,0xbe,0x3d,0x90,0x9e,0xfc,0x2b,0xc7,0x99,0xa1,0xfe,0x65, +0x11,0x7a,0xff,0xbd,0xf4,0x5f,0x4d,0x87,0x93,0x51,0x6e,0x01, +0x20,0xa7,0xa2,0x69,0x20,0xfc,0x92,0x35,0x41,0xeb,0x1c,0x49, +0x2d,0x69,0xda,0xd3,0x73,0xa0,0x9e,0xfd,0x4c,0x98,0x5e,0x0a, +0xe9,0xf0,0xb4,0xcb,0xf6,0x4b,0xf1,0x45,0x94,0x8d,0x22,0x43, +0xa4,0xec,0x27,0xd1,0xcc,0x36,0xee,0x8a,0x90,0x8e,0xeb,0x53, +0xda,0xd3,0xa4,0x82,0x59,0x99,0xf7,0x6f,0x76,0x15,0xa2,0xd7, +0xd5,0xcb,0xf2,0x10,0x7a,0x05,0x97,0x3e,0x27,0x5f,0x12,0xe4, +0xdb,0x50,0xbe,0x8d,0x6b,0xf7,0xb5,0x70,0x91,0x28,0x24,0x13, +0x75,0xb1,0x6b,0xfb,0x14,0xb5,0x3c,0xca,0x68,0x4f,0xad,0x82, +0x6a,0x1b,0x65,0x34,0xaa,0x56,0x41,0x8d,0x0d,0xf8,0x6a,0xe2, +0xd7,0x11,0x7e,0x1d,0x15,0x6f,0xb6,0x73,0x9d,0xc6,0x96,0x07, +0xed,0x74,0xf7,0xec,0xbe,0xec,0x95,0xa8,0x0f,0xe2,0xc6,0x04, +0x35,0xef,0xef,0x9f,0xbd,0xf9,0x61,0xd0,0x9b,0x1f,0x06,0xfd, +0xe8,0xc3,0xa0,0x1f,0xa1,0x45,0xbd,0xe2,0x4d,0xec,0x64,0x36, +0x1f,0x89,0xc7,0x72,0x6d,0x64,0x73,0x9e,0x58,0x49,0xa1,0x8e, +0x21,0x23,0x47,0x14,0x82,0xf8,0x72,0x3c,0x05,0x39,0xff,0x8b, +0x31,0x5a,0xc9,0x18,0x4d,0x07,0xa3,0xac,0x70,0x3f,0xfc,0x4b, +0x38,0x20,0x60,0x12,0x3a,0xe9,0x06,0x42,0x76,0xe2,0x48,0x41, +0xc3,0x14,0x5d,0x61,0xa4,0x3b,0x70,0x25,0x96,0xff,0xa5,0x1d, +0x5f,0x02,0xab,0xa7,0xd1,0xb0,0x12,0x0d,0x8b,0x72,0x6a,0x61, +0x49,0x06,0x45,0x85,0xec,0xd3,0xd7,0x9b,0xe6,0xf6,0x00,0x0b, +0x0f,0xd5,0x65,0x61,0xb3,0xbe,0x99,0x4d,0x67,0xcb,0xd9,0x74, +0x14,0xc8,0x79,0xb6,0x55,0x43,0x7a,0xdc,0x96,0x79,0x42,0xde, +0x65,0x7a,0x1f,0xc2,0xf0,0xa4,0x9c,0xa3,0x88,0x6e,0x53,0x51, +0x53,0xec,0x9a,0xe3,0xb4,0xc5,0xba,0xfd,0x19,0xd2,0x0d,0x14, +0xb5,0xe7,0x92,0xd2,0xd2,0x46,0x2b,0xea,0xe1,0x77,0xb7,0x3e, +0x54,0x7d,0x89,0x84,0xcf,0x2c,0x95,0xf7,0x8f,0x2a,0x3b,0xfe, +0x0c,0x8f,0x71,0xd3,0xa3,0xf2,0xb0,0x6e,0xaf,0x93,0xf0,0x5d, +0x1d,0x42,0xc9,0xca,0x3d,0x25,0x90,0xb2,0x72,0x3f,0xe6,0xc9, +0xde,0xce,0x35,0x80,0x65,0x96,0x6a,0xac,0x8c,0x3d,0xc9,0x5f, +0x14,0x14,0x88,0x69,0xb2,0xf2,0x92,0x7a,0xeb,0x08,0x26,0xaa, +0x2a,0x27,0x01,0x4b,0x3d,0x2e,0x47,0x49,0x05,0xab,0x29,0xe3, +0x9f,0x38,0x0e,0xa5,0xd7,0xec,0xf6,0xb6,0x41,0x68,0x94,0xf1, +0xeb,0x9e,0xc2,0xaa,0xee,0xaf,0x1d,0xe7,0x70,0xff,0xfc,0x6d, +0xf1,0x2e,0x06,0xc9,0x5d,0x77,0x5c,0x9e,0x5e,0xac,0x26,0x93, +0x67,0xcf,0xfe,0x18,0x8d,0x13,0x35,0x6e,0xaa,0xf1,0x91,0xd2, +0xde,0x23,0xe4,0x7d,0xe4,0x50,0x8d,0xd4,0x99,0x69,0x65,0x7a, +0x75,0x32,0x4e,0x70,0xc3,0x40,0x9e,0xeb,0xfa,0xe9,0x92,0x6d, +0x29,0x43,0x6c,0xb3,0xab,0x06,0x1c,0x3b,0x74,0x62,0x8f,0xba, +0x6a,0xc4,0xb1,0x67,0x1c,0x1b,0x9d,0x55,0xfb,0x68,0x20,0xb7, +0xda,0x8b,0xab,0xd1,0x90,0xbe,0x47,0xf0,0x7d,0x9c,0xb3,0xf1, +0x72,0x79,0x2a,0x0f,0x7f,0xd9,0x1e,0x60,0x34,0x57,0xae,0xcf, +0x44,0xd7,0xd8,0xee,0xdc,0x18,0xd5,0x7d,0xe2,0x18,0xd8,0x2d, +0xf3,0xbe,0xad,0x97,0x9b,0x1b,0x7c,0xd8,0xda,0x65,0x02,0x36, +0xd4,0x98,0x8d,0x46,0xed,0x2b,0x92,0xfe,0x8f,0xd8,0xb2,0xfd, +0xba,0x99,0xa4,0x43,0x31,0xd2,0xbc,0x7e,0x78,0xc4,0x36,0xa5, +0xd7,0x30,0x73,0x25,0xfc,0x4c,0x7e,0x73,0xc4,0xf6,0xee,0x37, +0x94,0x8f,0xed,0x30,0x6f,0x28,0x1f,0x7e,0x1f,0xa9,0x15,0x6c, +0xb0,0xa0,0x5c,0x99,0x34,0xb1,0x8e,0xe2,0xea,0x06,0xbf,0x51, +0xf5,0xea,0x08,0xaf,0x72,0x31,0x04,0x35,0x54,0x31,0x07,0x40, +0x30,0x17,0x49,0xeb,0xa4,0xb2,0xea,0x61,0x0a,0xd4,0x83,0x5f, +0x90,0x96,0xbb,0xee,0x34,0x2d,0x11,0x51,0x6b,0x30,0x9b,0xd1, +0x16,0x70,0x39,0xca,0xb4,0x3e,0xa3,0x13,0xc5,0x7d,0xe6,0x46, +0x38,0xdf,0x66,0xac,0x6b,0xa3,0xd8,0x07,0x91,0x31,0xbf,0x7d, +0x7b,0x6b,0x0c,0x6f,0x17,0x10,0xf3,0x6a,0xb6,0x00,0x7e,0x1b, +0xbf,0x1e,0x65,0xd9,0x68,0x32,0xe9,0x59,0x8e,0xc4,0x9b,0x9a, +0x7e,0x6f,0xf0,0x96,0x51,0x79,0x05,0xbc,0x9a,0xa5,0x64,0x25, +0x53,0x5f,0xd9,0x78,0xda,0x7e,0x57,0x6a,0xac,0xfb,0xf7,0x66, +0x3c,0x1d,0x8e,0xd6,0xad,0xb1,0x5a,0xb7,0xae,0x48,0xfe,0x80, +0x1f,0x68,0xfc,0x36,0xae,0x65,0xe8,0x3d,0xd3,0x14,0xf1,0x56, +0xee,0xda,0xe6,0x49,0xbf,0xb6,0x41,0xef,0x79,0xf0,0x79,0x8c, +0x9f,0xf8,0xb5,0x86,0xc8,0x35,0x47,0xae,0x8f,0xf1,0x33,0x69, +0x35,0xb6,0xb1,0xea,0xcf,0x96,0xcb,0xd9,0x25,0xa2,0xc4,0xf7, +0x3a,0xea,0xf9,0xf0,0x7c,0x84,0x06,0x40,0x05,0x41,0x74,0x10, +0x3a,0x3a,0x5b,0xe2,0x93,0x74,0x7a,0x48,0xb6,0x18,0x9f,0x5f, +0xd8,0x10,0xce,0x9e,0xae,0x96,0x99,0x2e,0x5c,0x93,0x32,0xa9, +0x89,0x18,0x2c,0x46,0x40,0x90,0xdf,0xf7,0x26,0x67,0x18,0xc5, +0xc6,0x33,0x4b,0x93,0x52,0x6c,0xea,0xab,0x69,0xc8,0x1f,0x52, +0x46,0xaa,0xa9,0x2d,0xd2,0x1c,0x98,0x3c,0xe0,0xda,0x24,0x0d, +0x0b,0x3a,0x90,0xf0,0xcf,0x6a,0xca,0x12,0xe9,0xee,0x6c,0x1a, +0x56,0xbc,0x55,0x3e,0x76,0x96,0x0a,0x23,0x08,0x29,0x1c,0xaa, +0xa6,0x13,0x31,0xa6,0x65,0xa2,0x5b,0x6c,0xab,0x74,0xb4,0x58, +0x42,0xcf,0x52,0x9b,0x26,0x42,0x5a,0xee,0x81,0x2d,0x50,0x35, +0x1b,0xb9,0xc6,0x04,0x26,0x7d,0x75,0x01,0xd0,0x2e,0x46,0xd0, +0x02,0xfa,0x86,0xf6,0x4e,0xfa,0xb5,0x85,0xc2,0x3f,0x10,0x77, +0x31,0xa2,0x2f,0xf8,0xdd,0x52,0x67,0x91,0x35,0x38,0x5b,0x7c, +0xce,0x6c,0x77,0x31,0xca,0x35,0xbf,0x3d,0x9c,0x61,0x34,0x02, +0x14,0x63,0xd5,0x17,0xa3,0x83,0x3c,0x29,0x0f,0x0f,0xbf,0x02, +0xf6,0xe6,0xe0,0x77,0x67,0x90,0x47,0xcd,0xed,0x3b,0x62,0x2e, +0x0f,0x8c,0x33,0x1c,0x4d,0x6c,0x9d,0x1a,0x5f,0xc2,0x0a,0x30, +0xc5,0x3f,0x84,0x69,0x6d,0x82,0xdf,0x48,0x0c,0xf1,0xf1,0x42, +0x50,0xfd,0x82,0x06,0x72,0x6d,0xc1,0x0d,0xda,0x95,0x3c,0xe1, +0xe4,0x97,0xa3,0x73,0x48,0x2b,0xcc,0xc4,0x37,0xec,0x52,0x63, +0x87,0x46,0x5e,0xcd,0x61,0x79,0x49,0xa8,0x2d,0x08,0xc2,0x09, +0x04,0xb1,0x73,0xba,0x82,0xd5,0x3f,0x0d,0x6e,0x30,0x41,0x7c, +0x37,0x9f,0xcf,0x32,0xc8,0x66,0xea,0x83,0x11,0xad,0x90,0xc4, +0xe9,0x4d,0x7f,0x8c,0xb3,0xb0,0xad,0x32,0x4b,0x54,0xd6,0x94, +0xed,0xca,0xe8,0x1d,0xf6,0x4c,0x7a,0xc3,0x40,0x5b,0x37,0x93, +0x16,0xa4,0x2e,0x5a,0x59,0x73,0xab,0x46,0x73,0x0c,0xfa,0x5c, +0x04,0x7b,0x93,0xac,0x59,0x5b,0x57,0xb3,0xa4,0xb6,0xc6,0x5d, +0x09,0x04,0x36,0x18,0xd8,0xa8,0x1e,0x24,0x0d,0xd7,0xc7,0x8d, +0xce,0x70,0xdd,0x42,0x5b,0xfb,0x3d,0x48,0x1d,0x6e,0x30,0xbc, +0x81,0xf0,0xc6,0x5c,0x10,0x71,0x85,0xb5,0x41,0x8a,0x20,0xd0, +0xb6,0x03,0x96,0x86,0x2d,0x46,0xa5,0xf6,0xa8,0xec,0x19,0x9d, +0x42,0x88,0xc7,0x00,0xa5,0x13,0xe9,0x32,0x3d,0x3c,0x50,0x92, +0xef,0x3e,0x00,0xaf,0x43,0x35,0x06,0x5e,0x1d,0x6a,0x8f,0x5b, +0x91,0x4d,0xb7,0x79,0x7b,0x90,0x54,0x1f,0x6e,0xbc,0xbc,0x00, +0x5f,0x82,0x5b,0xbe,0x26,0xf6,0x49,0x34,0x9a,0x24,0x6a,0x34, +0x11,0x22,0x8d,0x92,0x14,0xc2,0x44,0x7c,0x35,0x6a,0xc2,0x77, +0x93,0xbe,0xc9,0x2b,0xca,0x28,0xb9,0xbd,0x3d,0x18,0x35,0x6f, +0x6f,0x47,0x89,0x74,0x0c,0x70,0x67,0x3a,0x6a,0x9a,0x40,0xec, +0x74,0x2e,0x1d,0x8b,0x0d,0x53,0xc8,0xda,0x2b,0x43,0x96,0x7e, +0x15,0xbe,0xfa,0xf8,0xd5,0x6b,0x7b,0xb6,0x70,0xe3,0x27,0xc9, +0xa8,0x9a,0x34,0x72,0x45,0x47,0x13,0x90,0x17,0xd6,0xa8,0x03, +0x19,0x41,0xc9,0x81,0xc0,0x68,0xe2,0x17,0x00,0x8a,0x41,0xcc, +0xdb,0x70,0xa2,0x44,0xf5,0xaa,0x3a,0x5b,0x0f,0x13,0x47,0xc9, +0x22,0x75,0xd0,0x84,0xb6,0x2c,0x5c,0x4c,0xdb,0x90,0x0e,0xd3, +0x3c,0xc4,0xd6,0x36,0xd8,0x1e,0xf8,0xc1,0x96,0xc0,0xcf,0xe1, +0x21,0x86,0xd6,0x94,0xb4,0xee,0x00,0x71,0x52,0xa2,0x0f,0x00, +0x03,0x8a,0x53,0xa8,0x89,0xa1,0x26,0x9f,0xfb,0xc9,0x90,0x46, +0x9e,0x4d,0x11,0xd9,0xe3,0x74,0x64,0xea,0xa8,0xad,0xcd,0x13, +0x18,0x9b,0xeb,0xf0,0x10,0x26,0xe0,0x14,0xea,0x9a,0xd4,0xd8, +0xa4,0xf2,0x81,0x9f,0xb8,0x70,0x12,0x3b,0x48,0x8c,0xd6,0xcd, +0xba,0x85,0x90,0x61,0xe9,0xc2,0x06,0x6f,0x65,0x68,0x7d,0x77, +0xe6,0x0e,0x2b,0x25,0x53,0xd4,0x48,0x8f,0x28,0xb5,0x9c,0xcd, +0x71,0x84,0x38,0xe8,0x28,0x17,0xd7,0x39,0xac,0x61,0x92,0x05, +0xd0,0x84,0x0e,0x29,0x40,0x51,0x06,0x97,0xee,0x19,0xda,0xdd, +0x17,0x61,0x1b,0xe4,0x23,0x9f,0x02,0xe8,0x76,0x03,0x98,0x51, +0x8c,0x9d,0x6c,0x60,0xcf,0x0e,0x63,0x47,0x57,0xb8,0x81,0xa1, +0x85,0x31,0x6b,0x13,0xb3,0x56,0x67,0x3d,0x58,0x31,0x1b,0xaa, +0xd7,0x9f,0x5d,0x8d,0xd8,0x2d,0x81,0x5f,0x15,0x70,0xf9,0x93, +0xc6,0xed,0x6d,0x18,0x87,0x2e,0x67,0xb9,0x0c,0x01,0x80,0x9a, +0xb0,0x07,0xfa,0x30,0xb2,0xe6,0xb0,0x37,0xa0,0x04,0xa8,0xa7, +0x02,0xb5,0x97,0x31,0x37,0x8c,0x07,0x45,0x90,0xd0,0x41,0x05, +0xa5,0x1e,0xd0,0x4f,0xcc,0x15,0xdf,0xde,0x46,0x04,0x26,0x01, +0x71,0xfd,0x00,0xbf,0xb4,0xb1,0x96,0x2c,0x35,0x98,0x56,0x0d, +0x45,0x27,0x40,0x39,0x06,0x82,0x35,0xc2,0x78,0x9e,0x63,0xb5, +0x30,0xfe,0xe1,0x77,0x33,0x8f,0x9f,0x40,0x31,0xfc,0xc0,0x7d, +0x40,0x13,0x53,0x60,0x08,0x67,0x15,0xcc,0x0a,0xff,0xe2,0x62, +0x34,0x58,0xf5,0x88,0x5f,0x61,0x00,0xab,0xd5,0x06,0x50,0x5b, +0xaf,0x0c,0x2d,0x50,0xcb,0x84,0x48,0xb8,0x99,0xa0,0x67,0x27, +0x9f,0x74,0xcb,0xa3,0x74,0x33,0xb1,0xc4,0x15,0x9c,0x96,0x49, +0x79,0x99,0x1c,0x2f,0x9b,0xe5,0x65,0xb3,0xb2,0x3c,0x2a,0x2f, +0x8f,0xe4,0xd0,0xdf,0xeb,0x5b,0xa6,0x5d,0x8b,0x6b,0x87,0x59, +0x70,0xca,0xc6,0xdb,0x0a,0xd6,0x62,0x85,0xc0,0x63,0x5a,0x89, +0x6b,0xa3,0xf9,0x09,0xcd,0xba,0x29,0xc6,0x29,0x1d,0xb5,0x6b, +0x8e,0xe6,0x09,0xfa,0xf0,0x50,0xcb,0x70,0x04,0x13,0x8d,0xc1, +0x89,0x62,0xbc,0x33,0x57,0x2b,0x43,0xf0,0x34,0xc3,0x26,0xca, +0xbc,0x8b,0xac,0x53,0xdb,0x78,0x1a,0x89,0x39,0xb3,0x7d,0x30, +0x2e,0x9e,0x5f,0xc1,0x0e,0xe5,0xdf,0x57,0xa3,0xd5,0xc8,0x8a, +0x5e,0xa1,0x24,0x70,0xc1,0x0d,0x51,0xb3,0xb3,0xb3,0x6c,0xb4, +0xa4,0x25,0x96,0x05,0x09,0x6e,0x0b,0x04,0x37,0x80,0xd5,0x22, +0x65,0x52,0x56,0x38,0x9b,0xe7,0x3b,0x1c,0x21,0xa7,0xb6,0x2a, +0x92,0x7b,0xd4,0x84,0x2c,0xb5,0xeb,0x7b,0x8c,0xc9,0xf1,0x18, +0xaf,0x86,0xf4,0xaa,0xb3,0x5e,0x52,0x2a,0x6c,0xf5,0x88,0xad, +0x23,0x5d,0xc9,0x31,0x26,0xf1,0xe7,0xed,0xad,0xa9,0x39,0xb5, +0xb1,0x87,0xff,0x17,0x75,0xef,0xb6,0xdd,0xb8,0x91,0x2c,0x88, +0x3e,0x9f,0xf9,0x0a,0x17,0xf7,0xb6,0x36,0x20,0x26,0x29,0x80, +0xba,0x93,0x86,0xb8,0xdc,0x6e,0x7b,0xba,0x66,0x7c,0x3b,0x2e, +0xf7,0xb4,0x6d,0x16,0x5b,0x0b,0x24,0x21,0x09,0xe2,0xd5,0x04, +0x55,0x22,0xab,0xc4,0xbd,0xf6,0xac,0x35,0x6b,0x7f,0xc1,0xf9, +0x95,0x7e,0xea,0xb7,0xf3,0x01,0xf6,0x2f,0x9d,0xb8,0xe4,0x15, +0x17,0x8a,0x55,0x76,0xf7,0x9c,0xe9,0x76,0x89,0x40,0x66,0x64, +0x64,0x64,0x64,0x64,0xe4,0x2d,0x10,0x71,0xc0,0x3d,0xcc,0x80, +0x4c,0x67,0x73,0xed,0x73,0x54,0xe5,0x2d,0xd5,0x98,0x2d,0xe8, +0x1a,0x3a,0x15,0x01,0xae,0x70,0x4a,0xd6,0x10,0x0e,0xe9,0x59, +0x19,0xe1,0x99,0x45,0xf6,0xc1,0xc1,0x04,0x37,0xe0,0x2f,0x40, +0x3e,0x28,0x76,0x4a,0x07,0x72,0x75,0x15,0x21,0xe0,0x4f,0xa6, +0x8b,0xd5,0xa6,0x24,0xa0,0x1c,0xda,0x9c,0xe6,0x70,0x4b,0xb4, +0x5b,0x81,0xd4,0x53,0xde,0x6f,0xa1,0x8c,0xc8,0x01,0xf6,0x11, +0x7d,0x28,0xbf,0x4a,0xdd,0x4c,0xf8,0x9c,0xa0,0x63,0xcd,0x1f, +0x5b,0x31,0x75,0xa3,0xef,0xf3,0x7c,0x92,0x4c,0xf3,0x95,0xe0, +0xa1,0xbc,0x5c,0x62,0xae,0xdb,0x08,0xa0,0x99,0x0c,0xda,0x96, +0xde,0xa9,0x1f,0x40,0xb6,0xa0,0x09,0xcb,0x78,0xb8,0xfa,0x2a, +0x2d,0x8b,0xa6,0x97,0x6f,0xb8,0x3c,0x95,0xdd,0x6e,0x3b,0x7a, +0x25,0x88,0xeb,0x7c,0xcf,0x17,0xf9,0x75,0x4e,0xc4,0x09,0x76, +0x29,0x2d,0x6c,0x30,0x65,0xc3,0xae,0x12,0xbf,0x06,0x81,0xc5, +0x29,0x2c,0xa1,0x97,0xf8,0x67,0xc2,0x8f,0xf4,0xcc,0x4b,0x9d, +0xf9,0x12,0xb7,0x20,0xa8,0xef,0xc5,0x0a,0x7a,0x47,0x2c,0xc4, +0x1b,0x98,0x38,0x17,0x53,0x9c,0xf2,0xab,0xaa,0xe8,0x20,0x27, +0x2d,0xaa,0xa9,0x57,0x3d,0x8c,0xdf,0x68,0x2a,0xb5,0x79,0x85, +0xe7,0x1e,0xbe,0xaf,0x30,0x82,0xd2,0x2a,0x2b,0x2b,0x73,0x61, +0x56,0x35,0x48,0x70,0x72,0xd5,0xe9,0x28,0xda,0x26,0xe3,0xe0, +0x40,0x65,0xac,0xed,0x02,0x20,0xde,0xd8,0x44,0x77,0xa5,0x4d, +0xeb,0x71,0x4f,0xc2,0xfb,0xc4,0x89,0xdc,0x22,0xdb,0xc3,0x42, +0xb4,0x19,0xcb,0x65,0xf0,0x12,0x54,0x66,0x27,0x11,0xad,0xc3, +0x99,0x71,0x1e,0xb2,0x4d,0xe3,0x54,0xcc,0xac,0xdc,0x3c,0x25, +0xb9,0x9d,0x13,0x6b,0x16,0x42,0xac,0x0b,0xa3,0x73,0x2c,0xaa, +0x41,0xaf,0xb2,0x0a,0xf9,0x2e,0xf3,0x60,0xb8,0x4a,0xd2,0xac, +0x44,0x1b,0xf3,0x82,0x96,0xb5,0xfa,0x23,0x22,0x6f,0xa1,0x29, +0xc6,0xe0,0x1d,0xc8,0x07,0x23,0x05,0x7b,0xb4,0x60,0xf9,0x01, +0x2d,0xd0,0x15,0x2c,0x89,0x50,0x68,0x41,0x91,0x56,0x0d,0xb3, +0x83,0xde,0x5d,0x02,0xa9,0xe2,0x14,0x14,0xe5,0x4a,0x86,0x90, +0x67,0x99,0xb0,0x72,0xf5,0x78,0xc4,0x88,0x23,0x45,0x89,0x91, +0x5c,0xad,0x96,0x94,0x65,0x59,0xd6,0xb2,0x28,0x44,0x66,0x7b, +0x24,0x0b,0xc2,0x40,0x2b,0x17,0x31,0x2e,0xfc,0x26,0x42,0x30, +0xa9,0x47,0x98,0x19,0x6a,0x56,0x25,0x04,0xbc,0x3a,0xa3,0x27, +0xde,0xc6,0xfa,0x39,0xa0,0xa5,0x06,0x5a,0xda,0x40,0xba,0xca, +0x52,0x91,0xc1,0x44,0xae,0xdf,0x81,0x93,0x3d,0x36,0x8d,0x40, +0x74,0xb1,0x59,0xcd,0x0d,0x2e,0xfa,0x70,0xec,0x79,0xa8,0x2b, +0x22,0x1c,0x01,0xd8,0x5a,0x52,0x1e,0xd0,0x2e,0x56,0x20,0x00, +0x8e,0x72,0x52,0x1c,0x2c,0x00,0xb2,0xd7,0x40,0x59,0x4c,0x4b, +0xa4,0x2c,0x27,0x66,0x6e,0xa3,0x13,0x51,0xb5,0x7a,0x58,0x4c, +0xfb,0xd8,0xfc,0xe2,0xb8,0xda,0x63,0x60,0x55,0x8e,0xac,0x8a, +0xa1,0x05,0x6d,0xc4,0x61,0xb5,0x63,0x04,0x7c,0xd8,0x10,0x20, +0xbc,0x64,0x0c,0x3d,0x29,0x11,0xba,0xfc,0x99,0x82,0x4f,0xc2, +0x5e,0x72,0x8a,0xd0,0x29,0x2b,0x4d,0x6d,0xf4,0xf5,0xda,0x4a, +0x8b,0x58,0xc9,0xe9,0x1a,0x9a,0x93,0xac,0x96,0x49,0xf2,0x19, +0x06,0x4a,0x59,0x39,0xa6,0xa1,0xcd,0xf5,0x73,0xe0,0x8e,0x91, +0x68,0x73,0x53,0x0d,0xfe,0x35,0x86,0x8c,0xd4,0x27,0x37,0x93, +0x24,0xbe,0x69,0xbf,0xc0,0x10,0x8d,0xa3,0x24,0x6b,0x2b,0xe3, +0x6f,0xde,0x68,0xcb,0xc3,0x9b,0x8d,0xdc,0x6f,0x57,0x62,0xc4, +0xe8,0x4c,0x2b,0xef,0x86,0x70,0x48,0x2f,0x7b,0xec,0x91,0x0f, +0x35,0xc6,0x0b,0x0c,0x64,0x9c,0x4b,0x97,0x5f,0x48,0xae,0x23, +0xdc,0x66,0xaf,0xc3,0xfa,0xba,0x85,0xdf,0xd0,0xd2,0xdb,0x26, +0xac,0xa3,0x2b,0xbf,0xe1,0x5d,0x3a,0x19,0x2d,0x93,0x59,0x84, +0x65,0x9b,0x44,0x5c,0x47,0xa5,0x51,0x04,0x95,0x2a,0x1a,0x2c, +0x20,0x59,0x65,0xb6,0x06,0xdc,0x06,0x23,0x85,0x55,0x79,0xb6, +0x74,0xd8,0xc7,0x72,0x4c,0xb0,0x53,0xba,0xb5,0x4f,0xe9,0x16, +0xd5,0x9d,0x6d,0x08,0x87,0xd5,0x9a,0xde,0xf1,0x3e,0xa5,0x8f, +0xfb,0x4c,0xb3,0x64,0x96,0xcb,0x78,0xcb,0x48,0xf9,0xbb,0xdb, +0x01,0x1f,0x8b,0xc6,0xf4,0x21,0x36,0xbe,0xc1,0xe8,0x57,0xcf, +0x03,0xf9,0x85,0xfa,0x32,0x8a,0x61,0x6b,0x19,0xdf,0xc2,0xcf, +0x2d,0x6c,0xa2,0xe0,0x67,0x20,0x06,0xcb,0x68,0xd0,0x5c,0x36, +0x60,0xe1,0x32,0xb8,0x85,0xa7,0xdb,0x46,0x7c,0x2b,0x06,0x03, +0x78,0x1a,0x34,0xe2,0x41,0xe1,0xa3,0x6d,0xed,0x22,0xab,0xf6, +0x2f,0x64,0xb2,0x00,0xd8,0xaf,0xef,0x92,0x35,0x1f,0x11,0x2c, +0x69,0xe6,0x8f,0x97,0xf5,0xc1,0xf2,0x10,0x06,0x51,0x55,0xfe, +0x6d,0x7d,0x70,0xbb,0x2b,0x7f,0x50,0x1f,0x0c,0x30,0xbf,0xb2, +0xb1,0xdf,0x0c,0xee,0x65,0xbc,0x63,0xf9,0x2d,0x89,0x48,0xa3, +0x77,0x5b,0x31,0x84,0x3f,0xb4,0x20,0x1b,0x7f,0x94,0xce,0x3e, +0x8a,0x7d,0xfa,0x19,0x74,0xd3,0xde,0x98,0x2e,0xed,0x2c,0x04, +0x5e,0x0c,0x69,0x62,0x00,0x7f,0xfc,0xf6,0x10,0xb3,0xf1,0xdd, +0x14,0x1d,0x70,0xd1,0x18,0x43,0x64,0x63,0x2e,0x01,0x96,0xb1, +0x42,0x97,0x48,0x7d,0x02,0xc4,0xaa,0xf0,0x43,0x5c,0x09,0x3a, +0xac,0x6c,0xc1,0xd7,0x0f,0xd3,0x01,0x45,0xca,0x34,0x07,0xd9, +0x83,0x46,0x14,0x47,0xf5,0x58,0x14,0x59,0xfd,0x11,0x5e,0x8f, +0xad,0x2a,0x71,0xbd,0x5a,0xa1,0x4f,0x55,0xc3,0x8d,0xa9,0x48, +0xc5,0xbd,0x98,0x89,0xb9,0xc8,0x30,0x6e,0x4e,0x16,0xe2,0x1f, +0xbc,0x0a,0xfc,0x19,0xcd,0x2a,0x90,0xe4,0xb8,0x8e,0xd6,0x84, +0x03,0xfa,0xeb,0xe2,0xba,0x9e,0x11,0x61,0xcd,0x09,0x6c,0xa7, +0x5f,0xe2,0x61,0x7c,0x14,0x50,0x78,0xc6,0x69,0x54,0x0e,0x97, +0xac,0x93,0x21,0x4a,0x17,0xae,0xf7,0x51,0x09,0x43,0x09,0xd8, +0x07,0xc9,0x90,0x74,0xcd,0xec,0x61,0x90,0x31,0x71,0x19,0xd1, +0x21,0x21,0x40,0x6b,0xff,0xcc,0x20,0xef,0xd2,0xb6,0xb6,0x00, +0x59,0xb7,0xf1,0x76,0x75,0x0b,0x83,0x9f,0xf3,0xd8,0xaf,0x1d, +0x34,0xe1,0x19,0x0a,0xa9,0x45,0x59,0xf0,0xc9,0xc0,0x7c,0xdd, +0x51,0x56,0xbf,0x8f,0x77,0x9c,0x68,0x04,0xfd,0xb3,0x8e,0x93, +0xb5,0xb3,0x51,0x31,0x86,0xa3,0x9f,0x99,0xad,0xcc,0x3c,0xfa, +0x19,0x2f,0x3b,0xe7,0xcd,0x75,0x44,0xd7,0xc0,0xa4,0xcf,0xe6, +0x4d,0xca,0xe3,0xe3,0xd1,0xac,0x07,0xaf,0x78,0x09,0x4a,0x04, +0xe1,0x4b,0x03,0xc3,0x68,0x42,0x09,0xa1,0xf7,0x65,0x90,0x88, +0xc1,0x51,0xef,0x23,0x00,0xec,0xcc,0x64,0xf8,0xca,0x9f,0x7b, +0xf7,0x7d,0x00,0x6e,0x70,0x80,0xb9,0x42,0xe9,0xba,0x42,0xec, +0xa2,0x69,0x55,0xa0,0x89,0xac,0xa0,0xf1,0x2e,0x5d,0xf4,0xd0, +0x47,0x94,0xb9,0x9a,0xfa,0xd5,0xd5,0xd4,0xc3,0x1d,0xf4,0xfe, +0x6c,0x6f,0x37,0xc5,0xac,0xd1,0x10,0x90,0xca,0xdf,0x63,0x21, +0x9f,0x4a,0x45,0x7e,0x11,0xc3,0x34,0xfd,0xc5,0x64,0x1e,0xaf, +0xbc,0xa9,0xb4,0xbf,0xd7,0x09,0x73,0xdc,0x50,0x50,0x7f,0x52, +0x3c,0x50,0x1f,0x78,0x2e,0x8d,0xc7,0xdc,0x96,0x74,0x0d,0xd5, +0xed,0xfd,0x5a,0x40,0xd4,0xa9,0x61,0x89,0xc7,0x5d,0xfa,0x3b, +0x0d,0x85,0x39,0xe8,0x77,0xa9,0x8f,0x83,0x3e,0x9e,0x6f,0x15, +0xc7,0xe0,0x1c,0x9e,0xeb,0xb5,0xda,0xd6,0x2f,0xd9,0x52,0x0e, +0x2c,0xdf,0x03,0x52,0x2b,0xd8,0x31,0x4d,0xb3,0x9e,0x94,0x1e, +0x5f,0x12,0x6a,0xa9,0x08,0xf3,0xa1,0x51,0xe5,0xf8,0x96,0xd1, +0xbc,0xe4,0x46,0xf3,0x06,0xc4,0x18,0xb4,0x7a,0xea,0x7c,0xd2, +0x22,0x25,0xba,0xd1,0x48,0xaf,0xa2,0xe0,0xe0,0xe0,0x85,0x77, +0x53,0x80,0x81,0xea,0x09,0x91,0xdf,0x31,0x9a,0xac,0xaa,0xc6, +0x4f,0x97,0xcb,0x78,0x63,0x14,0x4a,0x2a,0xc8,0x2c,0x6b,0x88, +0x7f,0x66,0x71,0xa4,0xbf,0xfe,0x9a,0xe1,0x2c,0xa1,0x9e,0xe5, +0xb7,0xad,0xb8,0xf9,0xd4,0x00,0x2a,0x57,0xdd,0xd5,0x03,0x4b, +0x02,0xc9,0x93,0xb5,0x36,0x92,0x73,0xd5,0x72,0x8a,0x6a,0x19, +0x6f,0xe0,0xa5,0x18,0xc4,0x12,0x7e,0x88,0x07,0x09,0x98,0x2d, +0xd3,0x07,0x76,0x3a,0x96,0xa8,0x52,0xcf,0x4e,0xad,0x04,0xbd, +0x46,0x56,0x54,0x2a,0xe9,0x24,0xce,0x92,0xeb,0xe1,0x24,0x9e, +0x2e,0xbc,0x9b,0xa2,0x2f,0x2d,0x23,0x0f,0xc1,0x55,0xb4,0xea, +0x06,0xed,0xd5,0x55,0x14,0x76,0xc3,0xf6,0x0d,0xe4,0x94,0xe0, +0x59,0x26,0xf4,0x49,0xe1,0x6e,0x4c,0x61,0xe3,0x06,0x2d,0x9b, +0xca,0xcb,0xdf,0x4c,0x70,0xa6,0xdb,0x55,0x1e,0x17,0x4a,0xcd, +0xd3,0xab,0x55,0xf7,0xc6,0x6b,0xc1,0xac,0xd9,0x6e,0x01,0xba, +0x16,0x5a,0x72,0xe7,0x27,0x50,0xc2,0x88,0x0b,0x0d,0xab,0xee, +0xd5,0xe1,0xaa,0xa4,0xf5,0x0f,0x83,0x74,0xe8,0x02,0x55,0x82, +0xbd,0x9c,0x7d,0xf3,0xb0,0xf2,0x38,0x64,0x1e,0xb2,0xc4,0x39, +0x57,0x46,0xe6,0x98,0x03,0x69,0x32,0x2f,0x68,0x45,0x80,0x0b, +0x0f,0x41,0xf1,0xac,0xb3,0xf3,0x5f,0x64,0xe6,0x89,0x6c,0xc2, +0xea,0xb8,0x7d,0x7c,0xe8,0xad,0x1a,0xab,0x96,0x5f,0x5f,0x1d, +0x37,0x9a,0xe7,0xa7,0x7e,0xb1,0x5e,0xbc,0xfa,0xf6,0x92,0x5d, +0x1c,0x35,0x5f,0xaf,0x8b,0xa4,0x8c,0x09,0xec,0xf2,0xc2,0xf0, +0xdf,0xf8,0xd0,0x38,0x94,0x5f,0xf9,0x17,0xcb,0x24,0xeb,0x45, +0x59,0x0d,0x2d,0x11,0x06,0x48,0x71,0xe8,0x97,0x94,0x61,0x7f, +0x43,0xc5,0xaa,0xe8,0x3c,0x14,0xba,0xfc,0x70,0x55,0x56,0x11, +0xce,0x6c,0xc0,0xff,0x58,0x5d,0x13,0x64,0x9d,0x8a,0x6f,0xcc, +0x3e,0x69,0xa1,0x5d,0x7f,0xd4,0x3c,0x39,0xf5,0x45,0xe1,0xfb, +0xb3,0x2c,0x5a,0x1c,0xfd,0xfa,0xbf,0x2c,0x27,0x31,0xe1,0x51, +0xec,0xa3,0x05,0x0e,0xac,0x45,0x21,0xeb,0xc4,0x2f,0x53,0x72, +0x61,0x3d,0x3e,0xb4,0x9a,0xd6,0x80,0xb6,0xad,0xac,0xcf,0x5d, +0xa1,0x9d,0x99,0x7f,0xf8,0xeb,0xff,0x3a,0x5a,0x94,0x71,0x95, +0xb6,0x34,0xc6,0xca,0x25,0x83,0xd5,0x53,0x16,0x85,0xcd,0xf3, +0x20,0x3c,0xbd,0x28,0xad,0x0d,0xe5,0xca,0xf3,0xb2,0x7a,0xe8, +0x1f,0x22,0xe2,0x32,0x94,0xb0,0x20,0x1c,0x3a,0x0c,0x3c,0x6a, +0x81,0x50,0x80,0xa4,0x9c,0x37,0x4f,0xcf,0x5a,0xa7,0x28,0x98, +0xed,0x56,0x2e,0x0d,0xa8,0x84,0x6a,0x4f,0x29,0x15,0xdd,0x9a, +0xc0,0x4f,0xbb,0x25,0xdf,0x5d,0xa8,0x56,0xb3,0x65,0x81,0x5d, +0x1e,0x03,0xa0,0x93,0x7b,0xe6,0x64,0x5f,0x9c,0x00,0x40,0x95, +0xca,0xfc,0xd3,0x70,0x62,0xad,0xbf,0xef,0xf0,0x4d,0xae,0xbf, +0xf1,0x59,0xad,0xbf,0xef,0x40,0x79,0xde,0x89,0x78,0x08,0x3f, +0x43,0x11,0x4f,0x50,0x95,0x8a,0xc1,0x1d,0x68,0xd1,0xbb,0x46, +0x0c,0xea,0x72,0x08,0x4f,0xc3,0x46,0x3c,0x14,0x83,0x09,0x6a, +0xd6,0x46,0xac,0x3d,0xd7,0xa6,0xd9,0xd7,0xf1,0xd7,0xde,0x60, +0x88,0x6e,0xba,0x00,0x2c,0x40,0x1c,0x9c,0x16,0x0f,0xfd,0x2e, +0x94,0x6a,0xc3,0xaf,0x90,0x50,0x77,0x7e,0x17,0xfe,0x20,0xd0, +0x9d,0x02,0xba,0x43,0xa0,0xbb,0x36,0xfc,0xb6,0x07,0x77,0x57, +0xe1,0x45,0xd0,0x1d,0xdc,0x35,0xa2,0xe3,0xb3,0xa0,0xdd,0x80, +0x97,0xab,0x01,0x7e,0x3f,0x3b,0xb8,0xab,0x63,0x4a,0x69,0x57, +0x41,0x63,0xa1,0x1d,0xd7,0x93,0x18,0xf6,0x15,0x77,0xf5,0xc1, +0x1d,0x8c,0xe1,0x78,0x58,0x1f,0x0c,0xf1,0x77,0x52,0x1f,0x4c, +0x0e,0x79,0x72,0xac,0xe4,0x4e,0xe6,0x70,0x27,0xb3,0xb8,0x93, +0x15,0xb8,0x93,0xc1,0x4f,0x56,0xc2,0x9d,0x0c,0x9e,0xb2,0x46, +0x9c,0x55,0x72,0x27,0x23,0xee,0x64,0xd8,0xf0,0x4c,0x35,0x3c, +0xc3,0x86,0x67,0xed,0x38,0xfb,0x47,0x73,0x27,0x9b,0x5c,0xd3, +0xae,0x4b,0x71,0x27,0xab,0x0f,0xb2,0x3d,0xb9,0xf3,0x65,0x6c, +0xef,0xdd,0x88,0xc7,0x92,0x3b,0xf8,0xac,0xb8,0xc3,0xfc,0x88, +0x71,0xfe,0x8d,0xf5,0xde,0x4d,0x71,0x42,0x0c,0x62,0x78,0x8a, +0x1b,0x71,0xbc,0xcf,0xde,0x0d,0xeb,0x07,0xd4,0x4c,0x30,0x13, +0x08,0x88,0xeb,0x83,0x18,0x7f,0xe5,0xee,0x6b,0x17,0xc1,0xdf, +0xf1,0x3e,0x2d,0xb7,0x79,0x29,0xe3,0x8c,0xbd,0xad,0xc3,0x4d, +0x4c,0x6e,0x90,0x1b,0x2f,0xa5,0xd2,0x97,0xe9,0x32,0x88,0x7a, +0x53,0x68,0xdf,0xb4,0x39,0xe8,0x8b,0x65,0x88,0x2f,0x43,0x78, +0x19,0xf5,0xc5,0x78,0x1d,0xd9,0x05,0x8c,0xf7,0xb5,0x25,0x3a, +0xce,0x7a,0xeb,0x64,0xfe,0x71,0xbe,0x82,0x64,0x28,0x8f,0xde, +0xb3,0x2a,0x8a,0xd9,0xa9,0x9f,0xcd,0xa7,0x03,0xfc,0xaa,0x7e, +0x19,0x0a,0x28,0xd6,0x18,0xbf,0xc5,0x20,0xe9,0x41,0x67,0x89, +0x16,0xa6,0x87,0x4b,0xb4,0x3d,0xfd,0x64,0x19,0xd2,0x73,0xc0, +0x81,0x58,0x39,0x07,0x4d,0x26,0x29,0x85,0x9e,0xc6,0x6b,0xfe, +0x79,0x8b,0x3f,0xbe,0x20,0x17,0xfb,0xd2,0x4d,0xae,0x37,0x5e, +0x77,0x2d,0x3f,0x00,0x54,0x46,0x10,0x0e,0xdf,0x0e,0xd8,0xd0, +0xa0,0x4a,0x04,0x55,0xe8,0xf8,0xb6,0x62,0x64,0xc6,0xef,0x03, +0x70,0x25,0x01,0xae,0xdc,0xf4,0x39,0x83,0x1d,0x4e,0xf4,0xc6, +0x6b,0x68,0xad,0x4a,0x1a,0x27,0x8f,0xd1,0x78,0x63,0xd7,0x3a, +0x7e,0x0b,0xd9,0x36,0xd6,0x76,0x50,0xde,0x17,0xc8,0xbd,0x67, +0x6c,0xa3,0xcb,0x0b,0xda,0x1e,0xf1,0xa4,0x8f,0x07,0xc7,0x33, +0x59,0xae,0x86,0xd8,0x58,0xd5,0x8c,0xf1,0x6a,0x15,0xbf,0x8a, +0x8a,0xc6,0x14,0xeb,0x1a,0x7e,0xa1,0xeb,0xca,0x6b,0x51,0x9d, +0x85,0x76,0x8a,0x63,0x87,0xc8,0x7a,0x34,0x26,0x3a,0x09,0x05, +0xbf,0xa0,0xd1,0x62,0x95,0x14,0x7f,0xaf,0x85,0x4f,0xaf,0x74, +0x67,0x66,0xa3,0x2c,0x3e,0xc5,0x01,0x68,0x04,0x14,0x86,0xe4, +0x1f,0xdc,0x94,0x01,0x5a,0x89,0x46,0x9f,0x5a,0xae,0x41,0x56, +0x83,0xe8,0x0f,0xd6,0xeb,0x12,0x73,0xa5,0x1f,0xf0,0x25,0x66, +0xc9,0xe7,0x47,0x4c,0xc7,0x1e,0x12,0x8f,0x98,0x4a,0x4f,0x63, +0x4a,0x23,0xb7,0x23,0x63,0x4a,0xc4,0x47,0x63,0x65,0x0a,0xad, +0x7a,0x11,0xad,0xd8,0x26,0x6f,0x85,0xcd,0xa3,0x37,0xd8,0x09, +0x79,0xca,0xd3,0x83,0x71,0xa0,0x51,0xe3,0x30,0xcd,0x68,0x8b, +0xcd,0x0f,0x7e,0xcd,0xde,0x70,0x87,0xb0,0xd3,0x2e,0xdd,0x99, +0x51,0x2d,0x82,0x2a,0xf1,0xb7,0x02,0x20,0x8f,0x77,0x40,0x86, +0x04,0x49,0xc1,0x7c,0xfd,0xb6,0x22,0x8c,0x28,0x2a,0x21,0xa8, +0xbe,0x1a,0xd4,0x91,0x8a,0xb6,0xca,0x03,0x82,0x96,0xf1,0x8b, +0x68,0x39,0xe8,0x7a,0xcb,0xb8,0xb1,0x1c,0x90,0xce,0x5d,0x0e, +0x48,0xc3,0xb6,0x97,0x83,0xc6,0x32,0xc6,0x14,0x1c,0x68,0xb1, +0xd4,0xba,0xf6,0x81,0x01,0x3d,0xc9,0xcf,0x8a,0xea,0x35,0x66, +0xaa,0x67,0x35,0xb6,0xd1,0xaa,0x22,0x7c,0x19,0x43,0x47,0x10, +0xc9,0xcb,0x81,0x3e,0x27,0x28,0x60,0xaa,0x2f,0x99,0x5c,0xe8, +0xa8,0x17,0xd1,0xe3,0xa0,0x5b,0x5d,0x37,0x76,0xdd,0x0f,0xfb, +0x55,0xfd,0x18,0x43,0x6f,0x43,0xd5,0xed,0xc7,0x92,0x9a,0x25, +0x9e,0xfa,0xa3,0xac,0x78,0xcc,0x1d,0x3e,0x66,0xbe,0x8e,0xb9, +0xc3,0xc7,0xdc,0xe1,0xb3,0xa8,0x50,0x9a,0xbc,0xa8,0x3c,0xd3, +0xed,0xb3,0xc6,0x49,0x15,0x69,0x54,0x9b,0x18,0x9b,0x8e,0x9f, +0x55,0x37,0x83,0x68,0x11,0x63,0xdd,0xf5,0x5e,0xa8,0xc9,0x0c, +0x25,0x89,0x7e,0x49,0xfb,0x98,0xc2,0xfa,0x58,0xb6,0xcf,0x9c, +0xc2,0x88,0xfc,0x7e,0x0d,0x87,0xe1,0xdc,0x7c,0x43,0x32,0xeb, +0x7c,0xd0,0xf6,0xf9,0x61,0xf6,0xcc,0x61,0x5b,0x34,0x68,0x78, +0x78,0xdc,0xe6,0x77,0xc3,0x23,0x0f,0x66,0x4b,0xbf,0x1d,0x18, +0x52,0xb4,0x03,0x4a,0x6f,0x0d,0x39,0x87,0x83,0x1d,0xb8,0x3f, +0xa3,0x9d,0xe2,0x07,0xa0,0xfe,0xc8,0x32,0x44,0xd7,0x3b,0xe7, +0x50,0xc8,0x1a,0xf3,0x7b,0xb7,0x49,0xbc,0x99,0x3f,0xac,0xae, +0x07,0xf4,0x79,0x0f,0x45,0x8b,0x9b,0xa4,0xb3,0xb1,0xe1,0x58, +0xa6,0x62,0x22,0x8d,0xa5,0x33,0x4c,0xb4,0x0e,0xe1,0x77,0x76, +0x7b,0x29,0x26,0x43,0xf2,0x71,0xea,0x20,0xfa,0x12,0x16,0xd9, +0x2b,0xd0,0xa7,0xd3,0xf9,0xec,0x53,0xfc,0x1a,0x00,0x7d,0xb6, +0x10,0x26,0x2c,0x6d,0xdc,0xa8,0x50,0x52,0xbf,0x43,0x3f,0x2f, +0xa2,0x08,0x30,0x41,0xa7,0x50,0x85,0xf4,0x17,0x83,0x6a,0xc3, +0xce,0x43,0xd8,0x5f,0x1e,0x52,0x86,0xb9,0x1d,0x1f,0xe7,0xbe, +0x1d,0x4d,0xf0,0xca,0x84,0x11,0xa9,0x0f,0xf4,0xf9,0xbc,0x66, +0x2c,0x02,0xae,0x1b,0xc9,0x87,0x7f,0x12,0xb7,0xfb,0x41,0x61, +0x35,0x67,0x54,0x2b,0x32,0xba,0x66,0x30,0xec,0x89,0x55,0x3a, +0x7d,0xe0,0x41,0x28,0xf9,0x32,0x41,0xa2,0x67,0x1f,0xbb,0xf2, +0xc5,0xd7,0xd0,0xf2,0x20,0x12,0x51,0xd1,0x7d,0x46,0xa4,0x5a, +0xca,0x18,0xf8,0x27,0x47,0x62,0x69,0x61,0x9d,0x58,0x4d,0x7a, +0x59,0x5f,0x90,0x58,0xc1,0x3e,0x3a,0x8e,0xa2,0x68,0xa0,0x36, +0xd2,0xb1,0xe6,0x6a,0x8c,0xb7,0x37,0x59,0xa1,0x57,0x0d,0x13, +0x70,0x19,0xf9,0x1c,0x0c,0xcc,0x62,0x84,0x27,0x62,0x6c,0xf2, +0xbc,0x8d,0x8a,0x45,0x03,0x3b,0x29,0xbb,0x83,0x76,0x8e,0x28, +0x9d,0xcc,0xfe,0xb8,0x54,0xc4,0x40,0x20,0x10,0x26,0x9b,0x72, +0xf6,0xc3,0xaa,0xc7,0xb1,0x2e,0x5d,0xc6,0x21,0x68,0xf0,0x30, +0xf9,0xe3,0x32,0xbe,0x25,0xb1,0xc2,0xbb,0xad,0x51,0xf3,0x26, +0x5d,0x27,0xa3,0xa7,0xa8,0xb5,0x13,0x1e,0x24,0xc8,0x82,0x3e, +0x88,0x1a,0xe7,0x95,0xe0,0x5f,0xcd,0x1f,0xb2,0x64,0xfe,0x06, +0xf4,0x84,0x8d,0xfe,0x44,0x80,0x08,0xc2,0xca,0x13,0x6d,0x9b, +0x9a,0x0b,0x58,0x4b,0xe6,0x2e,0xd3,0x8a,0x18,0x1e,0x56,0x6e, +0x8d,0xa7,0x95,0xf0,0x9f,0x0e,0x87,0x0f,0xd3,0x07,0x9a,0x21, +0xf1,0x54,0x06,0xb6,0x0a,0x8b,0xbb,0x58,0x0c,0xef,0x70,0xbc, +0x4a,0x33,0xfe,0x21,0x9e,0xbf,0x0f,0x37,0x6c,0x62,0x87,0x40, +0x4d,0xce,0x86,0xd4,0x17,0xf4,0x8a,0x37,0x76,0xfa,0x13,0xbe, +0x21,0x5f,0xdc,0x45,0x94,0x43,0x8f,0x82,0xef,0xcc,0xf4,0x19, +0xbb,0xad,0x56,0x87,0x9c,0x85,0x27,0x6e,0x2c,0xfe,0x43,0x0e, +0xb2,0x5d,0x41,0xe2,0x30,0x47,0x9f,0xb0,0xc8,0xa9,0x47,0x43, +0xf9,0x24,0x86,0x6b,0xf3,0x72,0x08,0x0f,0x6b,0x20,0xdf,0x4d, +0xd9,0xf8,0xba,0x31,0xd2,0x01,0xb3,0x6e,0xc9,0xd3,0x93,0x95, +0xde,0x5c,0x4b,0xf7,0xa5,0xb0,0x8e,0x18,0xcd,0xa7,0x9e,0xdf, +0x68,0x9e,0x0a,0x2b,0x7b,0x53,0xcc,0xe6,0x9d,0xd2,0x38,0x22, +0x4a,0x0f,0x25,0xa5,0x3d,0xab,0x0c,0x5d,0x3f,0xf4,0x3b,0x0e, +0xe9,0x26,0xfb,0x33,0xe6,0xed,0x98,0x1a,0x31,0x3e,0xb4,0x49, +0xa1,0x56,0x38,0x49,0x9b,0x2d,0x63,0x59,0x47,0xc3,0xf5,0x91, +0x85,0x50,0xf2,0x65,0x13,0x0d,0x37,0x76,0x72,0x99,0x14,0xdc, +0xa5,0xc9,0x32,0x5e,0x0e,0xef,0x36,0xdf,0x71,0x28,0x8a,0x39, +0x5d,0x6b,0x09,0x9d,0x6c,0x6d,0xd7,0x54,0xb4,0x8a,0x3c,0x88, +0xa8,0xe1,0x97,0x11,0x30,0xe7,0xab,0x0b,0x43,0x78,0x7c,0x13, +0x4f,0x1e,0x12,0x98,0x64,0x19,0x96,0x05,0x21,0x92,0x05,0x65, +0x1a,0x4e,0x0c,0xb6,0x52,0xd0,0xf8,0xbe,0xc4,0x0c,0x09,0xb5, +0x93,0xe4,0xcf,0x64,0x7d,0xce,0x45,0xb3,0x22,0x62,0x67,0xc9, +0xff,0x81,0xe4,0x39,0xc5,0x88,0xe0,0x9d,0x65,0x5e,0xe1,0xe7, +0x1f,0xce,0x24,0xcb,0x85,0x1a,0xf1,0x1e,0x85,0xa9,0x4d,0xa4, +0x88,0x6d,0x4f,0x46,0xcd,0x69,0xb2,0xc4,0xaf,0x2c,0x68,0x78, +0x38,0x1f,0xa4,0xb0,0x3e,0x37,0xdf,0x1e,0xb2,0x96,0x57,0x6d, +0x7b,0x7a,0xea,0xc1,0x42,0xc4,0x29,0x40,0x59,0xfa,0x12,0x9d, +0xe7,0xdf,0xb6,0x9c,0x2a,0x78,0xf6,0x6d,0x13,0xc8,0x76,0x8b, +0x4b,0xa6,0x32,0x5a,0x41,0xaf,0x0d,0xc7,0xbb,0xee,0xf8,0x6d, +0xb8,0x5d,0x97,0xfb,0x36,0x1c,0x1e,0xcc,0x8e,0xc8,0x23,0x01, +0xea,0xa3,0x0d,0x45,0xb7,0x84,0xdf,0x68,0x47,0x89,0xe5,0x28, +0x59,0xfe,0x31,0xb9,0x89,0x1f,0x26,0x2b,0xe9,0x62,0xce,0x92, +0x40,0x72,0xee,0x6d,0x79,0x7f,0xab,0x6e,0xc9,0x37,0x64,0x88, +0xf9,0x53,0xb2,0x9c,0x2b,0x47,0x75,0x52,0x43,0xdd,0x47,0xf4, +0x95,0x25,0x79,0x7e,0x0b,0xfa,0x4a,0x2b,0x01,0x69,0xf4,0xbd, +0xf7,0x3d,0x7e,0x50,0xbd,0x09,0x7a,0xf7,0xfd,0x48,0x7b,0xb4, +0xdc,0x04,0x95,0xb5,0x7c,0x15,0xaf,0xe9,0x02,0xd0,0x8b,0xf1, +0xba,0xc2,0x54,0x32,0x26,0x77,0x15,0xf7,0xa0,0x23,0xdf,0x44, +0x94,0x25,0xbf,0x5e,0x9e,0xf1,0x9b,0x5a,0x83,0xa8,0xdb,0x19, +0x6f,0x2c,0xa1,0x52,0x8a,0x1c,0x74,0xf5,0x06,0xd4,0xe0,0x7d, +0x94,0x42,0xe1,0xb1,0x9e,0x9e,0xee,0x2b,0x89,0xf8,0x2e,0x19, +0x3d,0x0c,0x93,0x57,0x0f,0x53,0xa7,0x53,0x96,0x94,0xea,0xe5, +0x60,0x01,0x4a,0x04,0xd5,0x5c,0x43,0x24,0x0b,0x61,0xd0,0x2c, +0xea,0x05,0xff,0x5e,0x5a,0xb4,0x61,0xee,0xbe,0x5d,0xc6,0xd3, +0x3f,0xa4,0xb3,0x57,0x00,0x0b,0x5a,0xce,0xa3,0xee,0x11,0x34, +0x1a,0x32,0xf7,0x9c,0xa7,0x50,0xe4,0x0b,0x9c,0x9a,0x64,0x01, +0x3e,0xfa,0x4e,0xd2,0x89,0xa7,0x1d,0x18,0x32,0x0e,0xd5,0xc9, +0xfc,0x25,0xe6,0x97,0x5f,0xb7,0xea,0xf9,0x23,0xee,0x67,0x30, +0x5b,0x7e,0x47,0xd6,0xd8,0xed,0x83,0xa8,0x4e,0x19,0xb8,0xd7, +0x98,0x46,0x0c,0x44,0x5f,0x97,0xa1,0x27,0xd1,0x1b,0x96,0x80, +0x35,0x7d,0x33,0x7e,0xd3,0x5b,0xf7,0xa3,0xe9,0xe1,0xba,0x3e, +0x28,0xbf,0xa9,0xca,0x57,0xfc,0x1d,0x49,0xa6,0xdb,0xf4,0x1e, +0x8e,0x70,0x58,0x51,0xcb,0x54,0x0c,0xf8,0x83,0x6b,0x6d,0xf9, +0x5a,0xca,0x55,0xb4,0xbf,0x78,0x95,0xc0,0xb8,0x8d,0x4b,0x3e, +0x32,0x93,0x4b,0xbe,0x68,0x20,0x9f,0xba,0x61,0xbb,0x74,0xe9, +0x81,0x48,0xbe,0x44,0x93,0x36,0x5e,0x80,0xd2,0xb4,0xec,0x98, +0xaf,0xa8,0x37,0x7d,0xfd,0x24,0xdf,0x0f,0x0e,0xd4,0x93,0x3a, +0xc1,0xb7,0xac,0x57,0xda,0x54,0x94,0x90,0x37,0x57,0x77,0xcb, +0x24,0x1e,0x55,0xd5,0xfd,0x1d,0xd9,0x19,0x99,0xca,0x67,0x62, +0xdf,0xea,0x61,0x63,0x99,0xa3,0xc0,0x37,0x24,0xa0,0x0b,0x91, +0xfd,0x89,0x78,0x95,0xa0,0xe2,0x65,0x53,0x1f,0x8c,0xa7,0x03, +0x1c,0xdb,0xc9,0x0a,0x74,0x12,0xba,0x8b,0x0e,0xb3,0xc2,0xc1, +0x0c,0x31,0x73,0xd7,0x31,0x5c,0x01,0x63,0x88,0x4a,0x09,0xd1, +0xcd,0x80,0xb5,0x8e,0xa2,0x87,0xd6,0x4a,0x14,0x54,0x93,0xe8, +0xe4,0x3a,0x8d,0xb7,0xb8,0x8a,0x95,0xa8,0x66,0xf1,0x74,0x9e, +0xe5,0x4e,0xc9,0x9a,0xeb,0xc6,0xa0,0x5c,0x6b,0x2b,0x99,0x28, +0x94,0x01,0x78,0x98,0xc1,0x2a,0xcb,0xfc,0x31,0x49,0x16,0x49, +0xa1,0x1a,0x0e,0x2f,0x3b,0xe0,0xdf,0xaa,0xa2,0x64,0x47,0xc4, +0xae,0xe0,0xb8,0x17,0xf4,0x27,0x99,0xba,0xc0,0x1b,0x32,0x35, +0xa2,0xdc,0xc5,0x32,0x79,0x93,0xc2,0xea,0xf5,0x55,0x3a,0x80, +0x95,0xc1,0xed,0xef,0xdb,0x57,0x0a,0x39,0xad,0x18,0x38,0x3c, +0x9f,0xd3,0x7d,0x94,0x6c,0xf7,0x10,0x13,0xc6,0xc5,0x9d,0xc2, +0x7e,0x0e,0x17,0x81,0x74,0xb4,0x35,0x5d,0x69,0x53,0xb6,0x56, +0x33,0x8b,0x9e,0x5f,0x6d,0x51,0x21,0xe3,0x56,0x77,0xdb,0xc8, +0x34,0x90,0xd9,0x2b,0xae,0xc3,0xef,0x50,0xcf,0xd0,0x43,0x19, +0x67,0xa0,0x55,0x39,0x5e,0xc8,0xfb,0xf8,0x92,0x36,0xf2,0x28, +0x62,0x44,0x18,0x19,0x76,0x92,0x4e,0xeb,0x11,0xd5,0x23,0xd3, +0xa6,0xf3,0x91,0x4a,0xa0,0xbf,0x75,0x2e,0xcd,0x26,0xb8,0x75, +0x8f,0x49,0x51,0x89,0xfc,0x56,0xd9,0xb2,0xaf,0x60,0x57,0xe3, +0xa9,0x1d,0x28,0xdb,0xe5,0x11,0x1a,0xff,0x9d,0x4a,0x8c,0xd4, +0x83,0xa4,0x8b,0x06,0x84,0x19,0xed,0x1c,0xf1,0x92,0x5b,0x4f, +0x25,0x8f,0x3c,0xb6,0xc4,0xa3,0x73,0x95,0x86,0x2e,0xcc,0xef, +0x7e,0x47,0x27,0x18,0x32,0x79,0x46,0x60,0x76,0xe1,0x73,0xc3, +0x49,0x93,0x4d,0xe4,0xf6,0xf2,0xbe,0xdc,0xe1,0x09,0x25,0x19, +0x96,0x54,0x35,0x54,0xef,0x9b,0xdf,0xa4,0x53,0x6e,0xa7,0xa2, +0x44,0x0f,0x1f,0xc8,0x91,0x1a,0x4c,0x13,0xa9,0x34,0xbb,0x75, +0x1e,0xd0,0x2d,0x82,0xb5,0xd5,0x43,0x59,0xe5,0x0b,0x9c,0xbd, +0xf3,0x4b,0x54,0xb9,0x34,0x6d,0x0c,0xaa,0x97,0xa8,0x58,0xee, +0x25,0x1b,0xa4,0xea,0x23,0x68,0xbc,0x1b,0xa4,0x8c,0x6b,0xfc, +0x12,0xa3,0x63,0xbf,0x44,0x03,0x31,0x90,0xaf,0x28,0xe6,0x51, +0xac,0x5f,0x29,0x77,0x28,0x86,0x76,0xee,0xa0,0x92,0x52,0x3e, +0x6a,0xe1,0x1b,0xa7,0x5d,0xe8,0xab,0x49,0x96,0xb6,0xb6,0x99, +0x21,0x7b,0xb4,0x8e,0xa4,0x36,0xc3,0x8f,0x80,0x06,0xcd,0x0d, +0x3c,0x6e,0xc4,0x88,0x0c,0x0e,0xeb,0x83,0xe6,0x52,0x2a,0xd6, +0xe6,0xe5,0xe5,0xe5,0xe1,0x68,0x09,0xff,0x5d,0x59,0xdf,0x03, +0x55,0x52,0xca,0x83,0x58,0x9f,0xe6,0x28,0xa8,0x01,0x7f,0x61, +0x40,0x89,0xeb,0xaf,0xd2,0x99,0x31,0x3d,0xa1,0x4e,0x5c,0x37, +0xe8,0x67,0x29,0x30,0xcf,0x87,0xbf,0xb1,0x0c,0x4f,0x85,0x2b, +0x00,0x86,0xa8,0x6b,0x88,0x78,0xed,0x8b,0x4d,0x11,0xc7,0x46, +0xe1,0xd8,0x10,0x8e,0x4d,0x11,0xc7,0xa6,0xae,0x21,0x00,0x07, +0x7a,0x80,0xe3,0xcd,0x84,0xab,0x15,0x7c,0x52,0x92,0xf6,0x26, +0x5c,0x1a,0xbc,0xf2,0xfe,0x9c,0x9d,0x2d,0xa0,0xbd,0xde,0x18, +0x54,0x25,0xb5,0x25,0x3c,0x0a,0x98,0xe4,0x06,0x3e,0x6d,0x54, +0xd2,0x46,0x25,0x91,0xdb,0x1e,0xc2,0x07,0x6a,0xea,0xf3,0x78, +0x78,0xe7,0xb9,0x4c,0xc3,0xed,0x8e,0x2f,0x62,0xb9,0xbb,0xc7, +0xdb,0x90,0x26,0x14,0x24,0xbb,0x4f,0x58,0xfc,0x07,0x82,0x99, +0x17,0xc3,0x04,0x88,0x1e,0x51,0xbd,0x81,0x04,0x84,0x05,0x32, +0x74,0x20,0x9a,0x82,0xc2,0xaf,0x81,0x1b,0x20,0x5c,0x8b,0xb5, +0xba,0x3a,0x31,0x68,0xf5,0x85,0x5b,0xe5,0xb7,0x93,0x98,0x05, +0x4a,0x0c,0x7d,0x59,0x6e,0xe8,0x8b,0x0a,0x39,0x87,0x9c,0xd8, +0x16,0xb3,0x61,0x39,0xe0,0x10,0x24,0x4b,0x0c,0x9c,0xc1,0x00, +0x1a,0xf6,0x98,0x96,0xed,0xf4,0xb9,0x56,0x25,0x09,0xfa,0x5c, +0x43,0x3a,0x7b,0x43,0x49,0x65,0x3b,0x48,0xfc,0xba,0x38,0x0a, +0xe9,0xfc,0xec,0x3e,0xb2,0xc7,0x4e,0xe7,0xfe,0x05,0x2c,0xed, +0x3a,0xf7,0xd1,0xbd,0x5d,0x5d,0x16,0x42,0x3d,0x18,0xda,0xbd, +0x4a,0xfa,0xef,0xa1,0x31,0xfe,0x3b,0xae,0x20,0xec,0xf0,0x87, +0x5e,0xfc,0x39,0x25,0xa5,0x11,0xd3,0xc6,0x91,0xdd,0xda,0xce, +0x18,0x2a,0xba,0xb7,0x12,0x0e,0x0e,0x5e,0x54,0xa2,0x1f,0x03, +0xfa,0xce,0x38,0x1a,0x5b,0xe0,0xd0,0x00,0xa0,0xa9,0x43,0xe8, +0xbb,0x5e,0xd6,0xba,0xca,0xc2,0xa7,0x27,0x68,0x59,0x94,0xb5, +0x0e,0x0e,0xa0,0xef,0x3e,0x81,0x7e,0xee,0x56,0x0d,0xf7,0xe8, +0x9e,0xe2,0x22,0x94,0xe4,0x45,0x63,0xe4,0x76,0xda,0x68,0xf8, +0xed,0x42,0x6b,0x4d,0xb7,0x0d,0xa0,0xaf,0x54,0xef,0xfa,0x5b, +0x79,0x3e,0xe5,0xa1,0xd0,0xd6,0x69,0x1c,0x1d,0xb5,0xf0,0xac, +0xca,0x43,0x91,0xad,0x6f,0x54,0xc2,0x12,0x26,0x40,0xdb,0x20, +0x0e,0xfb,0xce,0x3a,0x7a,0x1a,0xc2,0x68,0x8d,0xf0,0x94,0x08, +0x46,0x5c,0x34,0xdc,0x08,0x15,0x5e,0x0d,0x87,0xd9,0x70,0x09, +0xc9,0xcb,0xba,0xb9,0x43,0x04,0xe0,0x43,0xf8,0x57,0x07,0x60, +0xf8,0xdd,0xf8,0x7e,0x87,0x87,0x20,0x86,0xe7,0xda,0x35,0x24, +0xfe,0x3c,0xa3,0xd3,0xf2,0xd2,0x23,0x75,0x35,0x64,0xa4,0x3a, +0xe1,0x59,0xcf,0xe8,0x44,0xeb,0x9d,0xa4,0xb5,0x6a,0x61,0x68, +0xaa,0x91,0x88,0x46,0xc9,0x24,0x59,0x25,0x1f,0xe5,0xf0,0x89, +0x62,0x32,0xa2,0xad,0xc2,0x68,0xae,0x28,0xd9,0x6c,0x5e,0x6c, +0xc4,0xf8,0xb9,0xd5,0x19,0x2b,0xb8,0x88,0x8e,0xb0,0xf8,0x99, +0xe7,0x4f,0x8c,0x40,0xad,0x92,0x36,0x9c,0xb4,0x3c,0xc4,0xc3, +0x2e,0xa5,0xa9,0xcc,0x27,0x7e,0xe4,0x9f,0x2d,0xbf,0x98,0x51, +0x9e,0xdc,0x2b,0x08,0xb4,0x57,0x70,0x4c,0x67,0x55,0xa3,0x6c, +0x6d,0xc1,0x73,0xc7,0x80,0x66,0x09,0xe8,0x6b,0x51,0x3a,0x8d, +0x60,0xab,0x46,0x03,0x3c,0x9d,0x5c,0x3f,0x3d,0x8d,0x36,0x52, +0x81,0x8e,0xd0,0xe2,0x41,0x96,0x1a,0x46,0xd6,0x6c,0xd2,0x19, +0xc5,0x87,0xd1,0x28,0x16,0xa3,0x01,0xfc,0x0c,0x3a,0xbc,0x01, +0x6d,0x9e,0xd6,0x01,0x47,0x63,0x14,0xfb,0x47,0x5e,0xeb,0x70, +0x04,0xd2,0xbc,0xb1,0x2e,0xa7,0xad,0xbb,0x18,0xc8,0x8c,0x0f, +0x01,0xb4,0x0e,0x30,0x0d,0x2c,0x12,0xc1,0xc3,0x21,0x15,0x85, +0x1c,0x5f,0x15,0xef,0x80,0x1c,0x02,0xd5,0xeb,0x3a,0xd5,0x8b, +0xd5,0xa2,0x08,0x43,0xca,0x06,0x53,0x36,0x0d,0x48,0x59,0xcb, +0xc8,0x04,0x12,0x70,0x34,0x50,0x10,0x65,0x8c,0x19,0x4e,0x1e, +0x32,0x18,0xf8,0x3f,0x6a,0x3e,0x5a,0x36,0x59,0x72,0xef,0xaa, +0x17,0x9a,0xe5,0x07,0x4f,0x2c,0x12,0x80,0xbd,0x94,0xf1,0x12, +0xff,0x0f,0x45,0xfc,0xba,0x9f,0xe5,0xa1,0x85,0xb9,0xaa,0x12, +0x2e,0x7e,0x18,0x77,0x54,0xc3,0x7a,0x2b,0x02,0xff,0x28,0x27, +0x1e,0x3b,0xea,0xfc,0x9d,0xb6,0xc3,0xe5,0x78,0xad,0x4d,0xb2, +0xdf,0xae,0x1a,0x9c,0xb2,0xc0,0xef,0xbb,0x37,0xae,0x40,0xef, +0x6c,0x99,0xab,0x49,0xc2,0xc5,0xe5,0x34,0x5e,0x7c,0x1b,0x8f, +0xbe,0x86,0x5d,0x89,0x24,0x4a,0x7f,0xdd,0x2a,0xc7,0xed,0xa6, +0x2d,0x47,0xeb,0x48,0x26,0x8d,0x70,0x58,0xc8,0xc7,0x4d,0xa9, +0x3a,0x33,0x78,0xe5,0x7e,0x28,0x1e,0x8d,0xf4,0x96,0x6e,0x1d, +0xc9,0xd5,0x8f,0x4c,0xc5,0xaf,0x4b,0x36,0x91,0x5c,0xcd,0xa8, +0xb4,0x80,0xfc,0xfc,0xc9,0xea,0x1a,0x2a,0x15,0xdd,0xaf,0x99, +0x42,0x23,0x59,0x0a,0x04,0xdd,0x14,0xd3,0x8f,0x2d,0x6d,0x8c, +0x1b,0xc0,0x2a,0x0f,0xb8,0x07,0xba,0x68,0xb4,0x86,0xc9,0x00, +0xf0,0x06,0xbe,0x80,0x44,0xfc,0xa6,0x0d,0xb4,0xd1,0x68,0x83, +0x89,0x1b,0x4c,0x44,0xbf,0x09,0xe8,0x34,0x01,0x9b,0xca,0xad, +0xcc,0x37,0x90,0xee,0x90,0x65,0x0c,0x8f,0xd1,0x7c,0x1a,0xa7, +0x33,0x15,0x38,0x13,0x6f,0x22,0x39,0x05,0x89,0xcf,0xf0,0xdb, +0x38,0xf9,0xca,0x3f,0x05,0x17,0x47,0x1f,0x21,0xcc,0x15,0x15, +0xec,0xf2,0xed,0x26,0x95,0xea,0xb7,0x7b,0xf8,0x23,0xf8,0xbe, +0xb3,0x58,0x39,0x9f,0x33,0xd1,0xa3,0xb1,0x72,0xa4,0xa8,0x2e, +0x74,0x98,0xc5,0xb4,0x75,0x0b,0x29,0x1e,0xcd,0xba,0x36,0xf9, +0x16,0x88,0x97,0xf7,0x82,0x87,0x59,0xd7,0x83,0x74,0xc2,0x8e, +0xc0,0xb8,0x01,0x82,0x4f,0xd4,0x9c,0x8b,0x67,0x3b,0x84,0x04, +0x33,0xe2,0x21,0x72,0x00,0x3c,0xc3,0x12,0xf9,0x84,0xfe,0xd9, +0x52,0x3b,0xd6,0x84,0xa7,0x0f,0xe4,0xd4,0x69,0xdc,0xae,0x60, +0x91,0xa9,0xf7,0x00,0x6f,0xf9,0x68,0x8c,0x44,0xef,0x0c,0x57, +0x11,0x92,0x56,0x7c,0x56,0x7b,0x02,0x91,0xe2,0x17,0x28,0x69, +0x18,0xe5,0x3a,0x02,0x3d,0x4d,0x4a,0xa2,0x52,0xfa,0x36,0x4b, +0xbf,0x99,0x3e,0x5a,0x07,0x57,0xeb,0x90,0x94,0x7e,0x94,0x06, +0x88,0x08,0x5d,0x95,0x01,0xe0,0x1a,0xc5,0x88,0xfc,0xf0,0x46, +0xe8,0x97,0x0b,0x53,0x7c,0x61,0x90,0x45,0x58,0x7f,0xf3,0x66, +0x32,0xc7,0x80,0x24,0x81,0xc9,0x09,0x65,0x0e,0x9d,0x75,0x52, +0x08,0x45,0xca,0xa8,0x68,0x0c,0xc5,0x2d,0xa0,0xf8,0xa4,0xba, +0xa3,0xe1,0xa5,0xfb,0x8e,0xf0,0xb6,0xab,0x6e,0xf3,0x65,0xad, +0x47,0x54,0x90,0x42,0xa2,0x6e,0x05,0xd6,0x57,0x59,0x80,0x89, +0xb1,0xe1,0xb7,0x6d,0x87,0x8e,0x97,0x32,0x08,0x74,0x09,0x99, +0x68,0x06,0xfd,0x41,0x52,0x42,0xee,0x6f,0xe1,0x0f,0x9e,0x90, +0x8f,0xcd,0xb6,0xc7,0xe9,0x22,0xc6,0xa5,0x74,0x5d,0x83,0xd7, +0xcc,0x92,0x95,0xe3,0xfe,0x27,0x5a,0xb4,0xb0,0x7f,0xe8,0x59, +0xf5,0x30,0x3b,0xda,0xf7,0x9b,0xca,0xf2,0xdd,0x67,0x4c,0x11, +0xe3,0x2b,0xe4,0xfa,0x74,0xec,0x1f,0x8d,0x3b,0xfe,0x03,0xdf, +0x97,0x97,0x8a,0xf0,0x7d,0x23,0xd4,0x42,0x7c,0x8f,0xde,0xf9, +0x52,0x06,0x2e,0x8a,0x32,0x41,0xca,0xc7,0xbe,0x5f,0x2a,0xcd, +0x7c,0x0f,0x01,0x73,0xaa,0xfc,0x56,0x56,0x32,0x6f,0x2d,0x42, +0x58,0xb3,0x34,0x42,0x6d,0x53,0x0a,0x08,0xbc,0x07,0xfc,0x53, +0x21,0xf3,0x65,0xbc,0xb7,0x39,0x4f,0xdf,0x10,0xd8,0xf1,0xa8, +0x13,0x19,0xed,0x88,0x7d,0x84,0x53,0xe9,0xbd,0xb8,0x7f,0xd5, +0xea,0x96,0x74,0x7a,0xbb,0xa0,0x2c,0xdc,0x9e,0x8f,0xa8,0xfe, +0x6e,0xa9,0xbd,0x4a,0xbb,0xdc,0x42,0x46,0x35,0x1d,0xe6,0x91, +0xc5,0x03,0x59,0x93,0xbc,0x87,0x70,0xc1,0x8b,0x55,0x88,0xa1, +0x65,0x51,0xb7,0x90,0x6b,0x56,0x24,0x63,0x6e,0x19,0xfe,0x32, +0x93,0xcc,0x38,0x61,0x62,0x20,0x81,0x36,0x1c,0xfc,0xc6,0x75, +0x75,0x1c,0x15,0x9c,0x8f,0x65,0x66,0x6e,0x4f,0x09,0x18,0xde, +0xb7,0x5c,0x55,0x53,0x0a,0x6c,0xc9,0x98,0x2c,0x09,0x69,0xc3, +0xb0,0x1c,0xa0,0x96,0xb9,0xe4,0x0b,0xdd,0x93,0xa0,0xd8,0x59, +0x89,0x08,0x4b,0xa3,0xef,0x87,0x98,0x41,0xd7,0x36,0x2e,0x4a, +0x72,0x50,0x91,0x9d,0x6d,0x19,0x3e,0x7b,0x02,0x59,0xfb,0x4d, +0x67,0xc8,0x14,0x2d,0x75,0x75,0xcb,0x49,0x24,0xf6,0xa3,0x8f, +0x41,0x1d,0xfa,0x28,0x49,0xa1,0x4a,0xcb,0xa2,0xf9,0xec,0x42, +0x68,0x17,0x70,0xd0,0x5a,0x19,0x0a,0xf9,0x2a,0x1d,0x8e,0x33, +0x83,0x76,0xea,0x04,0xa0,0xb1,0x06,0xdf,0xf7,0x08,0xa7,0x04, +0x74,0xea,0xdb,0xc5,0xbf,0x40,0x5b,0x54,0x4b,0x1c,0xa6,0x82, +0x82,0x40,0xae,0x76,0xa1,0xe2,0x32,0x1a,0x9f,0x2a,0xa1,0xd0, +0xa2,0x3e,0xde,0x83,0xa8,0xaf,0xad,0xb9,0x70,0x6a,0x09,0x8b, +0xee,0x84,0xf9,0xc2,0x0a,0xc7,0x54,0x85,0xe5,0x39,0xbd,0xb2, +0xb5,0x10,0x57,0xe9,0x26,0x69,0x56,0xc0,0x56,0xa5,0x9c,0x54, +0x62,0x52,0x60,0x67,0x8b,0x1a,0xd5,0x57,0x93,0xbf,0x24,0x3d, +0xf0,0x62,0x55,0x8f,0xa6,0x06,0x48,0x40,0xde,0xff,0x74,0x25, +0x03,0x0a,0xed,0xb3,0xd7,0x0a,0xc5,0x39,0xb7,0xa4,0x5b,0x78, +0xd9,0xa5,0x11,0xc2,0xf2,0xb2,0x74,0xcd,0x54,0x59,0xe0,0x1d, +0x7f,0xe2,0x37,0x85,0xb9,0x6a,0x1a,0x85,0x01,0x9f,0x0a,0x25, +0x1c,0xd8,0xae,0x7c,0x5d,0x29,0xb2,0x45,0x3c,0x8b,0x18,0x04, +0x57,0xbe,0xf2,0x89,0x16,0x97,0xc9,0x22,0xd2,0x5f,0xc8,0x84, +0xd2,0x74,0x8f,0xe7,0x7d,0x7d,0x81,0x8a,0xa5,0x8f,0xa6,0xfa, +0xe6,0x34,0xc4,0x6f,0x4f,0x93,0xe5,0x32,0x9a,0x1e,0x61,0x0e, +0xcd,0xf3,0xea,0x38,0x34,0x3c,0xbd,0x8a,0x20,0xab,0x8b,0x69, +0x87,0x40,0x5c,0xbb,0x79,0xec,0xa4,0x9c,0xb6,0xf1,0xa3,0x15, +0x4c,0x40,0xcf,0xb8,0x94,0xd4,0xf2,0x85,0xa6,0x27,0x32,0xab, +0x08,0x9d,0x66,0xad,0x26,0x84,0x6e,0x42,0x64,0xd1,0xa9,0x13, +0x2d,0xc8,0xba,0x0c,0xe1,0xae,0x4a,0xb4,0xfa,0x91,0xf5,0xba, +0x93,0xdb,0x59,0x59,0x5f,0xb3,0x6e,0x92,0xa1,0x01,0x47,0xc7, +0xe2,0xf9,0x5e,0xda,0xdd,0xa5,0x15,0x43,0x93,0xa6,0x52,0x1d, +0x2a,0x3f,0x6a,0x94,0xf5,0xc6,0x7e,0x12,0x65,0x3a,0xab,0xde, +0x0c,0x42,0xcb,0xab,0x78,0x93,0xab,0xf2,0xf8,0xa7,0xcb,0x3f, +0x30,0x76,0x16,0x74,0x8a,0x01,0xc8,0x39,0xe5,0x7a,0x99,0x98, +0x7d,0xb9,0xe5,0x42,0x57,0xdc,0x8a,0x3b,0x3c,0xdf,0xd5,0xf7, +0xd2,0xb9,0x9c,0xa7,0xa7,0x5a,0x13,0xa3,0x80,0xa8,0x36,0x34, +0x5a,0x87,0x5e,0xed,0x63,0x74,0xfa,0x75,0x0f,0x52,0x73,0xdf, +0x37,0xf6,0xad,0x7e,0xbb,0x26,0x00,0x54,0x43,0xd6,0x6b,0x37, +0xe5,0x83,0x10,0xda,0x2c,0x47,0xf3,0x20,0xce,0x60,0x07,0x88, +0xae,0x39,0xd2,0x37,0x6a,0x36,0xb6,0x23,0x09,0x02,0xa4,0x31, +0x75,0x55,0x70,0x5d,0xcd,0x39,0x58,0x80,0x77,0x83,0xf6,0xda, +0x6f,0x37,0x74,0x12,0xba,0x51,0x0c,0xda,0x8d,0xb5,0x1d,0x43, +0x16,0x6b,0xb1,0x08,0xc1,0x81,0xb1,0xb6,0x7c,0x1d,0xdb,0x58, +0x31,0x8f,0x88,0xd2,0x48,0x75,0x0a,0xe0,0xac,0x5e,0x04,0x48, +0x9d,0xc8,0x04,0xfb,0xdb,0x9d,0x13,0xbf,0x5d,0xf7,0xa3,0xe4, +0x84,0x04,0xa2,0xc2,0x1f,0xb0,0x0a,0x50,0x8d,0x88,0xd6,0x30, +0xbe,0xae,0xc8,0xad,0x18,0x61,0x65,0x1c,0x5e,0xe9,0x2a,0x81, +0x6d,0x84,0x80,0x62,0x5f,0x2e,0x71,0xf2,0x8b,0x05,0x6c,0xb5, +0xa9,0x7e,0x57,0x24,0x39,0x82,0xac,0x5f,0xe7,0x6a,0x95,0x2b, +0xc6,0x42,0x2d,0x08,0x5d,0x3e,0x65,0xc9,0x53,0x0f,0x48,0x1b, +0x45,0x65,0xea,0x58,0xe3,0x12,0x4e,0xaf,0xb5,0x6d,0xd1,0x42, +0xe5,0xfe,0x75,0x72,0x1b,0x63,0xae,0x1e,0x25,0x2e,0x61,0x54, +0x81,0xda,0x5f,0xd1,0xa6,0x6b,0x44,0x98,0xa1,0x3b,0xd4,0x6a, +0xaf,0x7c,0x9e,0x97,0xae,0xbf,0x76,0x6a,0x66,0x2e,0x02,0xbb, +0x97,0x87,0xc8,0x68,0xe5,0x37,0x46,0x57,0xc3,0xd6,0xc6,0x52, +0x02,0x28,0x30,0x0f,0x38,0x8e,0x2c,0x45,0x49,0x26,0x2e,0xf8, +0x41,0x78,0x84,0x9c,0xfa,0x38,0xec,0xb6,0x88,0x65,0x78,0xc8, +0x98,0x66,0xec,0x20,0xdd,0xbb,0x6f,0xa4,0xbe,0x8c,0x35,0xc2, +0x8c,0xe0,0x3b,0xe1,0xce,0xbd,0x3c,0xc0,0x36,0x76,0xcd,0xf8, +0x55,0xfc,0xb8,0x33,0x86,0x34,0xa2,0x4c,0xc5,0x5d,0x79,0xf4, +0x52,0xff,0x70,0xec,0x77,0x0a,0x89,0xfe,0x56,0x7f,0x7a,0x5f, +0xcc,0x43,0xdc,0xe8,0x6e,0xdd,0xa0,0x9f,0xc1,0x86,0x64,0x7c, +0x15,0x74,0xc6,0x8d,0x46,0x79,0x05,0xea,0x68,0x9d,0x32,0x7b, +0x69,0xff,0x93,0x07,0x22,0x50,0xde,0x71,0x70,0x11,0x79,0x80, +0xcb,0x20,0xb8,0x43,0xba,0x7a,0xd3,0xb9,0x07,0x84,0x9c,0x22, +0x81,0x32,0xf9,0x21,0xff,0xbd,0x1e,0x5e,0x94,0xbe,0x6b,0x4d, +0x35,0xd3,0x6a,0x18,0x5d,0xbb,0xe4,0xe5,0xd6,0x2f,0x2c,0x6c, +0xe6,0xb7,0x8c,0xa1,0x53,0xfc,0x90,0x55,0xea,0xd5,0xa8,0x08, +0xdc,0x36,0x71,0x1f,0x5f,0xa8,0xb8,0x8f,0x0c,0x0c,0x33,0xa2, +0x2e,0xe5,0x6a,0x69,0x5f,0xce,0xf1,0x7a,0x7b,0x1b,0xaf,0xbd, +0x66,0x28,0x66,0x47,0x96,0xd8,0xc1,0x2e,0x54,0xd2,0x29,0x6e, +0x22,0x2d,0xef,0x5e,0x12,0xa1,0x83,0xd3,0x96,0xb1,0x8b,0x42, +0x3f,0xa2,0x51,0xc3,0x4a,0x24,0xc9,0x2a,0xee,0x2c,0x2d,0xf3, +0xaf,0x23,0xec,0x9e,0x1b,0x12,0xb4,0x91,0x5f,0x4f,0x7c,0x1f, +0x36,0xb8,0xb2,0x81,0x90,0xd0,0xc6,0x0f,0xeb,0xf6,0x5d,0x0b, +0x6a,0x5d,0x4e,0xa0,0x68,0x42,0x5d,0xa6,0xd2,0xb7,0xf9,0xf9, +0xb5,0x6c,0xe5,0x57,0x7a,0x84,0xa0,0x14,0x24,0x4c,0xf4,0x8b, +0xf9,0x0c,0x0d,0x16,0x0b,0xb3,0x44,0x85,0x2e,0x86,0xb2,0x0b, +0xd2,0xa7,0x34,0x05,0xc3,0x4b,0x64,0xa3,0xfd,0x16,0x30,0x2b, +0x94,0xa8,0x4e,0x1e,0x07,0x85,0xec,0xf0,0x48,0x03,0xec,0xde, +0xc9,0x39,0x0a,0x7d,0xf0,0xfb,0x68,0xf4,0xbd,0x15,0x38,0xb6, +0xac,0x4a,0x83,0xff,0x9f,0xb5,0x4d,0xb1,0x99,0x94,0x5f,0x14, +0xb9,0x4b,0x76,0x8d,0x4a,0x75,0xd0,0xbe,0xb3,0xe4,0x0e,0x21, +0x88,0xd6,0xcf,0x8b,0x41,0xf5,0x04,0xe7,0xf6,0x82,0x2a,0xb0, +0xef,0x28,0xb2,0xd6,0x01,0x72,0x14,0xe5,0xa5,0xfd,0x37,0x0c, +0x20,0x6a,0x64,0xd1,0x19,0xc1,0xda,0x72,0x14,0xb1,0xee,0x9a, +0xb5,0x4e,0x63,0x2d,0x12,0xf9,0xd1,0x27,0x2d,0x94,0x0a,0x0e, +0x0a,0x18,0xf1,0x5c,0x06,0x56,0xb4,0x77,0x81,0xcb,0x1d,0x83, +0x92,0x8f,0xbc,0x3c,0x8f,0x0c,0xdc,0x29,0xc2,0xed,0xda,0x87, +0xa5,0x25,0xef,0xe6,0x60,0x31,0xc9,0x08,0x9a,0xa0,0x33,0x19, +0x22,0x43,0x08,0xd9,0x7a,0x9e,0x52,0x60,0x20,0xf9,0x8d,0xd0, +0xff,0xd8,0x3e,0x7a,0xb2,0xce,0xc0,0x71,0x37,0x90,0x79,0xea, +0xac,0xdc,0x3a,0x17,0x35,0x96,0xc1,0xf6,0x09,0x56,0xce,0x42, +0x3a,0xb5,0x8e,0x51,0x01,0x43,0x9d,0x76,0x32,0xe9,0x96,0x95, +0x07,0x51,0x24,0x37,0xba,0xf4,0xf7,0x0f,0xf1,0x6c,0xd4,0x29, +0x91,0x5a,0x47,0x0c,0x77,0x4e,0x39,0x04,0xde,0x91,0xa5,0xc8, +0xb1,0x2c,0xba,0x32,0x9a,0x25,0x8f,0xc8,0xe1,0xaf,0xe2,0x85, +0xfe,0x84,0x66,0xad,0xc3,0x51,0xad,0x73,0xf7,0x9c,0xcc,0xa7, +0xbb,0x38,0xf3,0xd6,0x29,0xb9,0x0c,0x01,0x7e,0x5a,0xbc,0x4b, +0x5d,0xe6,0xa5,0xbe,0xab,0xc4,0x7a,0x8a,0xe1,0x7d,0xb9,0x07, +0x62,0x49,0x92,0xa9,0xb1,0xff,0x5b,0x4e,0x8c,0x14,0x8b,0x60, +0x1d,0xca,0xf8,0xa2,0x77,0x30,0x4b,0xca,0x8d,0x7b,0xdc,0xd6, +0x25,0xb7,0x6a,0xbc,0x14,0x0f,0x96,0xbe,0xe5,0x2f,0xbb,0xac, +0xab,0x3e,0x7d,0x4f,0x54,0xee,0x64,0x82,0x73,0x23,0xb9,0x79, +0xe6,0x2b,0x97,0xb5,0xbe,0x6d,0x59,0x93,0x8b,0x34,0xdc,0x1a, +0x7b,0xf8,0xde,0xe0,0xaf,0xbf,0x8e,0xcc,0xb5,0x6a,0x28,0x72, +0x17,0x00,0xbe,0xba,0x6e,0xd2,0x6c,0xe3,0xe6,0xb1,0x98,0x39, +0xc0,0xb0,0x38,0xf0,0x94,0xd4,0xcc,0x17,0xfe,0x51,0xab,0x6d, +0xc9,0x90,0xc4,0x72,0xd4,0x62,0xa1,0xdc,0xc9,0x1c,0x6e,0x74, +0x19,0x8b,0x1c,0xde,0x60,0xe9,0x32,0xd6,0x88,0xf9,0xc3,0x2a, +0x59,0x7e,0xbb,0x27,0x9f,0x0a,0x0e,0x39,0x3e,0x39,0x86,0x7c, +0x1b,0x45,0xa4,0x39,0x40,0x1f,0xbe,0xf3,0x89,0x38,0xb1,0xf2, +0x13,0xc9,0x59,0x66,0xb2,0xcc,0x69,0x58,0xcc,0x6e,0xc8,0xb4, +0x52,0xa6,0xbb,0x9c,0x96,0x95,0xd4,0x5b,0x87,0x0e,0xf9,0x65, +0x5c,0xb7,0xd8,0x6a,0xc3,0x2a,0xce,0x72,0x95,0x07,0x07,0xac, +0x20,0x72,0xe7,0xfb,0xc4,0x72,0x2a,0xea,0x85,0xaa,0x4e,0xbf, +0xd0,0x03,0xc4,0xda,0x67,0x3b,0x80,0x4e,0x9d,0xfe,0xcf,0xeb, +0x05,0x6b,0xdb,0xf1,0x01,0x1d,0x42,0x27,0x44,0xf3,0x65,0x64, +0x4a,0x36,0xca,0x0b,0xfa,0xf6,0xf9,0x51,0x49,0xff,0x59,0x1e, +0x17,0x08,0xe3,0x11,0xba,0x53,0xdc,0xb7,0x07,0xad,0xc2,0xf9, +0xce,0x2c,0xf6,0xa6,0xe9,0xa7,0xbd,0xc6,0x54,0xc9,0x0c,0xad, +0xf3,0x1c,0x60,0xde,0xf1,0xed,0x98,0xd0,0xe5,0x96,0x50,0x29, +0x53,0xfe,0x6c,0x79,0xbf,0xa5,0x40,0xf9,0xbc,0xea,0xae,0x20, +0xd5,0x5e,0xb3,0x64,0x5a,0xfe,0xf9,0x21,0x9e,0xad,0x30,0x70, +0x88,0x5d,0xbe,0xf2,0x8e,0x67,0x0c,0x4a,0xe8,0xe7,0xc8,0x9e, +0x50,0x69,0xde,0x41,0xdb,0xf9,0xec,0x6e,0x3e,0x19,0x65,0xfc, +0xd1,0xc3,0xf8,0x93,0x9f,0x3b,0xbe,0x49,0xec,0x8d,0x61,0x87, +0x86,0x5b,0x9a,0x7c,0x65,0xe3,0xa3,0x9f,0xdd,0x49,0xa6,0xfa, +0xbc,0x84,0xfd,0x9a,0xac,0xa3,0xfa,0xda,0xef,0xbe,0x99,0xa7, +0xa3,0x8f,0x02,0x9e,0x04,0x7a,0xe6,0xda,0xcb,0x54,0x28,0xd6, +0x7e,0x9f,0x26,0x62,0x93,0xf4,0xec,0xec,0xfb,0xfc,0x85,0xc9, +0x4d,0x3a,0x41,0xeb,0xf3,0xe2,0xfe,0xe8,0x05,0x13,0x37,0xf2, +0x55,0x58,0x1f,0xa0,0x29,0xce,0x86,0xc9,0x4c,0xea,0x8c,0x7f, +0xc2,0xe5,0x8a,0xe2,0x6c,0x56,0x22,0x2a,0x86,0x09,0xe6,0xaa, +0x03,0x77,0x1a,0x79,0xa9,0x34,0xf7,0x4b,0x1b,0xd9,0xc5,0xb4, +0x3a,0xf8,0xe6,0x06,0x72,0x44,0x70,0xb5,0xe9,0xf6,0x82,0xa3, +0x40,0xc0,0xbf,0x7e,0xbb,0x87,0xc1,0x46,0xac,0x0e,0xde,0xe0, +0xc7,0x14,0xe6,0x76,0x7e,0xf3,0x89,0xc9,0x53,0x4d,0xb0,0xa1, +0x35,0x6c,0xde,0x9a,0xa1,0xbf,0xaf,0xd4,0x97,0x8b,0xed,0xee, +0x2b,0x04,0x2e,0xf3,0x16,0xa4,0x2a,0x40,0xff,0xa5,0x79,0x49, +0x2f,0x5f,0x80,0x96,0x7d,0xd3,0x9e,0xda,0x67,0xe4,0xe3,0xf5, +0xa1,0xb7,0x6e,0xac,0x03,0x58,0x6c,0xda,0x1f,0xe1,0x98,0x71, +0xa3,0x1c,0x82,0xac,0x9d,0x71,0x73,0x44,0x61,0xb3,0x02,0x34, +0x5f,0xb0,0x93,0x61,0xe1,0xc6,0x03,0x81,0x46,0xdb,0x5a,0xa4, +0x1f,0x2e,0xb7,0xeb,0x00,0x06,0x0b,0xfb,0x89,0xc5,0x87,0xb5, +0xe1,0xb2,0x2d,0x45,0x14,0x02,0x38,0xec,0xff,0x03,0x44,0xf2, +0x03,0x84,0x6c,0x13,0xa1,0x98,0x81,0x84,0xb5,0x37,0x47,0xe3, +0x75,0x1d,0x48,0xeb,0x6d,0xc4,0xa6,0x1e,0xc2,0xcb,0x7b,0x4a, +0x46,0xae,0x97,0x77,0x0b,0x86,0x16,0xcd,0x2a,0x25,0x98,0x17, +0x8d,0xf5,0x55,0xb4,0xee,0xe6,0xd5,0x8f,0xba,0x75,0xf7,0xfb, +0x6d,0x56,0x50,0xdb,0x9d,0x5d,0xb7,0xeb,0x78,0x54,0xc2,0x5e, +0x97,0x6f,0xd2,0x73,0x7d,0xb4,0x0b,0x11,0x83,0x5e,0x97,0x2e, +0x9b,0x3f,0xa0,0x7f,0xe4,0x78,0xa5,0xe1,0xae,0xac,0x17,0x36, +0xfb,0x0f,0xda,0x0a,0x3e,0x6f,0xf3,0x17,0xe6,0xba,0x40,0x2a, +0x6d,0x45,0xbc,0xc2,0x11,0x8e,0xce,0xd1,0x9d,0x52,0x5f,0x2b, +0x7e,0xab,0x3c,0x75,0xec,0xa2,0xde,0x85,0xce,0x90,0xfc,0xd5, +0xef,0xef,0x21,0xf5,0xce,0xb9,0x8a,0x42,0xe0,0x6b,0xd4,0xa6, +0xaf,0x34,0xf2,0x0f,0x3e,0x53,0x71,0x30,0xfc,0xae,0xc7,0x2a, +0x1a,0xf3,0x73,0x3d,0x96,0xef,0x00,0x53,0xd4,0xed,0xad,0x37, +0xb7,0xd7,0xf1,0x72,0xf8,0x72,0x36,0x4b,0x96,0xdf,0xc5,0xa3, +0xf4,0x21,0x73,0xbe,0x1d,0x4d,0x4d,0x7a,0x59,0xb1,0x6f,0x70, +0xd5,0x58,0x52,0x6c,0x6e,0xd2,0xcb,0x8a,0x61,0xb8,0x86,0xd5, +0xa7,0xb3,0xdb,0x89,0xfb,0xe9,0x75,0xa6,0x93,0xcb,0x0a,0x7d, +0x3e,0x1b,0x15,0x8b,0x24,0x32,0xb1,0x50,0x00,0x59,0x89,0x04, +0xc0,0xf2,0xaa,0x10,0xfa,0x19,0xdf,0xc5,0x52,0xc4,0xa5,0xc1, +0x9f,0x75,0x24,0x6b,0x4c,0xb4,0xe2,0x05,0x2f,0xf9,0x99,0xbe, +0x61,0x90,0x8f,0xa1,0x8e,0x84,0x89,0xac,0xa0,0xcf,0x8c,0x85, +0x02,0x8a,0x96,0x87,0xda,0x91,0x63,0x2c,0xbd,0x97,0xe0,0xd5, +0xe7,0xd2,0x38,0x30,0x8c,0x73,0xa1,0xc9,0x4b,0x1a,0x9d,0xc4, +0xde,0x62,0x39,0xc7,0xef,0xdf,0xd3,0xb9,0x3d,0x86,0x28,0x47, +0x7e,0xcd,0xac,0x35,0x5d,0x72,0x3b,0x25,0x03,0xbf,0x77,0xf2, +0x49,0xf9,0x37,0xfa,0xaa,0x66,0x5f,0xd9,0x5b,0x08,0x25,0x67, +0xd0,0x3f,0x98,0x8c,0x64,0x2c,0xbe,0xb4,0x41,0xbf,0x93,0x8b, +0xf2,0x42,0x89,0xc0,0xb2,0x99,0x32,0x65,0x6b,0x3f,0xd5,0xd8, +0x55,0x3d,0x47,0x7f,0x45,0xbb,0x6b,0x45,0x49,0xa4,0xdc,0xb7, +0x67,0x81,0x79,0x0c,0xad,0xd8,0xe7,0xd6,0x97,0xdc,0xe2,0x66, +0x4d,0x6e,0x96,0xa9,0x61,0xd2,0x6e,0xee,0x66,0xe3,0x24,0x6d, +0x30,0x09,0xe6,0xc6,0x35,0xc6,0x8e,0x59,0x87,0xdd,0xe2,0x30, +0x58,0x93,0xe9,0x9a,0x46,0x11,0x22,0x8a,0x30,0xda,0x20,0xfc, +0xa6,0x0c,0x7e,0xe3,0xc0,0x6f,0x64,0x00,0x5d,0xb4,0x30,0x4f, +0x6e,0x40,0x92,0x46,0x4d,0x18,0x53,0x13,0x0f,0xbd,0x99,0x89, +0x11,0x7f,0x31,0x0e,0x22,0x91,0xfa,0x5d,0xcd,0x10,0x8e,0xcc, +0x0a,0x8b,0x5c,0xa0,0xde,0x06,0x06,0x20,0x98,0x1b,0xeb,0xd0, +0x80,0x5c,0x6a,0x5f,0xb9,0xb4,0x09,0x65,0x59,0x28,0x19,0xe6, +0x4b,0x42,0xb9,0x7c,0x5a,0xdf,0x57,0x91,0x58,0x03,0xed,0x63, +0xda,0xd3,0x7d,0x5f,0xce,0xe5,0x9c,0xa0,0x99,0x82,0xa6,0x9c, +0x96,0x19,0xe5,0xb1,0x53,0xbd,0xab,0x2b,0x5f,0x76,0xb5,0x4f, +0xc7,0x54,0x81,0x1d,0x80,0xf7,0x07,0xb2,0x7a,0xb4,0xdf,0x37, +0x68,0x66,0xb9,0x71,0x12,0x7f,0x14,0x92,0x8f,0xec,0xff,0xee, +0xc1,0x31,0x22,0xb1,0x01,0x39,0xe4,0xac,0x9d,0xfb,0xdf,0x93, +0x8d,0x6d,0x25,0xda,0x1c,0x27,0x9b,0x12,0x19,0xb5,0x41,0xc4, +0x97,0x11,0x06,0xf5,0x97,0x72,0x19,0x35,0xcf,0x3b,0x7a,0x4a, +0x48,0xf0,0xf3,0xa3,0xbd,0xa6,0x5f,0x32,0xe8,0x84,0xe9,0x17, +0xcb,0xf8,0xed,0x75,0xb8,0x15,0x5c,0x3a,0xd8,0xbb,0xb8,0x2e, +0x1b,0xa8,0xb2,0xe1,0x9e,0x65,0x4b,0xea,0xdd,0xec,0x57,0x14, +0xfd,0x2a,0x98,0xd2,0x1b,0x5d,0x7a,0x4f,0xaa,0x37,0x86,0xea, +0x8d,0xa2,0x7a,0xb3,0x27,0xd5,0x65,0xf5,0xaa,0x4e,0xdf,0x6f, +0xe5,0x24,0x81,0x15,0x16,0xf9,0x2e,0x51,0x95,0x9a,0x78,0xed, +0x42,0x97,0x13,0x21,0x73,0x71,0x17,0xa9,0x8b,0xbb,0xeb,0xae, +0x8d,0xf4,0xba,0xed,0x55,0xc8,0xe4,0x4b,0xc7,0xbb,0x35,0x1e, +0x67,0x5f,0xfb,0x4f,0x4f,0x05,0xa1,0xf5,0xf7,0x10,0x4d,0xa5, +0x37,0xf1,0xf4,0xd6,0x96,0x57,0x1b,0x64,0x48,0x51,0x74,0xbb, +0xa0,0xb6,0xdb,0x28,0xc6,0xcc,0x0c,0xb7,0x35,0x92,0x27,0x4a, +0xc2,0xf7,0xe2,0x87,0x02,0x56,0xec,0x95,0xef,0x8c,0xaa,0x30, +0xf5,0x0c,0xef,0xe6,0xcb,0x51,0xc9,0xc4,0xbe,0x2c,0x9f,0xd3, +0x61,0x9e,0xbd,0x9d,0xcf,0xe2,0xc9,0xb7,0x66,0xa6,0x18,0x19, +0xe7,0x07,0xe4,0xaf,0x68,0x53,0x8c,0x57,0xaf,0x4a,0xf1,0x3c, +0x6d,0x95,0xb5,0xe7,0xbc,0xfc,0x5d,0x86,0xb4,0xe6,0x8e,0x0c, +0x8c,0x3c,0xd4,0x26,0x0d,0xa9,0xdb,0x0d,0x7b,0x77,0x0a,0x97, +0x0c,0x93,0xf5,0xa8,0x6c,0xa2,0x96,0x3a,0xa1,0xe7,0x4e,0xd3, +0xce,0xec,0xdc,0xdf,0x16,0x28,0xce,0x36,0xd3,0xc1,0x7c,0xf2, +0x0a,0xf7,0x28,0x9a,0xb2,0xb3,0x93,0x0a,0xb0,0xef,0x41,0xcc, +0x34,0x58,0x8d,0xdd,0x2f,0xd7,0x2a,0x60,0x3f,0x63,0xe7,0xcc, +0x19,0x60,0x96,0xbe,0x47,0xad,0x0f,0x82,0x30,0xf5,0xe8,0xd7, +0xff,0x50,0x5a,0xbc,0xf6,0x55,0x20,0x6a,0xf5,0x65,0xbd,0xf6, +0x29,0xfd,0xe5,0xe7,0x8f,0x82,0x8f,0x42,0x11,0x7e,0x84,0x39, +0x8d,0xea,0x2c,0x78,0xfb,0xc9,0x25,0x81,0x1c,0x23,0xa6,0xc4, +0xd9,0xdb,0xe5,0xfc,0x61,0x91,0xc1,0x32,0xd1,0x59,0x4c,0x3e, +0x0c,0x86,0x93,0x38,0xcb,0x54,0xae,0x53,0x04,0xba,0x6c,0x35, +0xc7,0xd1,0xe4,0x0b,0xce,0x6e,0xa6,0x23,0x58,0x9a,0xcb,0x97, +0x8a,0x7a,0xae,0x57,0x8f,0x49,0xa2,0x6b,0x9b,0xc5,0x53,0xe9, +0x28,0x44,0x50,0xba,0x74,0xf2,0x3e,0x8a,0x34,0xc2,0x8e,0x45, +0x4c,0x32,0xe1,0x2e,0xbf,0x4e,0xf0,0xbb,0x3a,0x89,0xa2,0x64, +0x68,0x13,0x42,0x33,0xcd,0xd3,0xc7,0x27,0x64,0xf7,0xc4,0x9f, +0xba,0xd9,0xd4,0x5c,0xf7,0xd2,0x51,0xbf,0x49,0x75,0xd3,0x65, +0x0a,0x11,0xc4,0x24,0x12,0x16,0x9e,0x7e,0xf9,0x93,0x7d,0x2e, +0x8c,0xd3,0xff,0xf5,0x35,0xe1,0xf3,0xfd,0x6d,0x9b,0xc1,0x22, +0xab,0x88,0xe5,0x96,0xd7,0xfe,0x8e,0xef,0xd9,0x5a,0xb9,0x70, +0xde,0x2b,0x8f,0xcd,0xb9,0x64,0xbd,0xf2,0xcc,0x47,0xdd,0x6c, +0x4a,0x38,0xa0,0xef,0x66,0xd1,0x8b,0xa5,0x35,0x46,0xd8,0x47, +0x2b,0x80,0x7f,0x36,0x9f,0xd1,0x2e,0x71,0x50,0xe2,0xeb,0x96, +0x90,0x52,0xd0,0x1d,0xc9,0x20,0xe8,0xba,0x74,0x76,0x97,0x2c, +0x53,0x69,0xcc,0x36,0x99,0x0f,0xc7,0x51,0x09,0xf1,0x18,0xed, +0xad,0x98,0x1a,0xbd,0x8b,0x87,0x68,0x08,0xd0,0x0e,0xc4,0x70, +0xfe,0x30,0x5b,0x51,0x8c,0x6b,0x03,0x10,0x21,0x36,0x6c,0x36, +0x85,0x32,0x34,0xe9,0x32,0xd4,0x7b,0x3a,0x45,0x4d,0x49,0x95, +0x37,0xf1,0xa5,0x53,0x52,0x32,0x7a,0x47,0x1c,0x6b,0x9b,0xbb, +0x33,0x81,0xa0,0x6d,0xfc,0x23,0xd0,0x47,0x77,0x5b,0x61,0xc0, +0x17,0xfc,0xbc,0x31,0xde,0xe8,0x24,0x7a,0x13,0xa3,0x07,0xf6, +0xa8,0x62,0x92,0x65,0xc2,0x56,0xd4,0xeb,0x58,0x4f,0x93,0x68, +0x47,0x17,0x2d,0x88,0xd5,0x3a,0x33,0x84,0xe2,0x0b,0x50,0xcc, +0xf6,0x8a,0x9b,0x1c,0xc0,0xe9,0x74,0x65,0x92,0x80,0x48,0x98, +0x13,0x57,0xe9,0xa8,0x8b,0x67,0xe8,0x9e,0xdf,0xf6,0xac,0x64, +0x1c,0x22,0xa6,0x75,0xcd,0x04,0x03,0x42,0x1d,0x1c,0xe4,0x53, +0x78,0x3b,0x64,0x89,0x1f,0xad,0x08,0x2d,0x28,0x96,0x1e,0xf5, +0x91,0xa9,0xa6,0x13,0x67,0x21,0x96,0xa4,0x77,0x52,0x34,0xf3, +0x72,0x8c,0x88,0x7c,0xa8,0x10,0xcb,0xc3,0xd2,0x96,0x96,0x9f, +0x4a,0xf6,0xc8,0xa6,0x49,0xb5,0xe9,0xe9,0x29,0xf4,0xbb,0x21, +0x7d,0x85,0x4b,0xcc,0xb8,0x56,0x96,0x86,0x08,0xc4,0xec,0xa5, +0x0e,0xf0,0x05,0x1f,0x9b,0xd8,0x92,0x6b,0xe3,0xa1,0x0b,0x52, +0x8b,0x01,0x2f,0x22,0x60,0x81,0x6f,0x7d,0xfe,0x64,0xc5,0x2e, +0x5c,0x45,0xb2,0xd0,0x91,0xea,0x19,0x8c,0x99,0x0b,0xdd,0xe9, +0xad,0xd0,0x4c,0x4a,0x11,0xad,0xfd,0x26,0x05,0x1d,0x5f,0xa6, +0xf5,0x1a,0x8d,0x59,0xdf,0x6a,0xa6,0x31,0x09,0xa3,0xb8,0x08, +0xde,0x1e,0x2c,0x47,0x27,0x8f,0x39,0x86,0x33,0x79,0xbe,0x3a, +0x18,0xb2,0x3a,0x1f,0xd3,0x65,0xa7,0x37,0x1a,0x46,0x74,0xba, +0xf2,0xab,0x5a,0x25,0xb4,0x6d,0xe7,0x2b,0x5b,0x67,0xc4,0x88, +0x70,0xcb,0x93,0x99,0xab,0x59,0xb0,0xb5,0x91,0x4d,0x9b,0x16, +0x66,0x3b,0xd5,0x95,0x67,0x27,0x47,0x31,0x4e,0x32,0x06,0x4f, +0xf6,0x25,0x27,0x24,0x6f,0xaf,0x22,0x2a,0xdd,0x75,0x24,0xb8, +0x41,0x69,0x7e,0x49,0x77,0xf3,0x4d,0x7b,0x49,0x7f,0x6f,0x45, +0xc0,0x09,0xc5,0xa9,0x32,0x5e,0xa7,0xd9,0x0f,0x9e,0xd6,0xd8, +0x62,0x8d,0xfb,0xd2,0x89,0x9a,0xb1,0x57,0xab,0xa5,0x74,0xca, +0x8b,0x27,0x1b,0x35,0x51,0x3c,0x99,0x77,0x5c,0xf6,0xae,0xd1, +0x76,0xa9,0x26,0x02,0xbf,0x96,0xfb,0xa0,0x54,0xd5,0xf4,0xa3, +0x55,0xd3,0xe6,0xc3,0x6b,0xc2,0x39,0x72,0x43,0x75,0x15,0x6a, +0x1a,0x61,0x4c,0x9e,0x87,0xd5,0x50,0x29,0xd7,0x6b,0xba,0xc1, +0xff,0x23,0x05,0x0a,0xc9,0x2d,0xb6,0xae,0xc2,0x2e,0xa6,0x37, +0xff,0xfc,0xfd,0x67,0xe5,0x6b,0x13,0x73,0x3f,0x45,0xd7,0x45, +0x8e,0x62,0x06,0x6e,0xf2,0x37,0x13,0x30,0x22,0x71,0xcc,0xc4, +0x13,0xba,0x31,0x13,0xd2,0x9f,0x89,0x75,0xbd,0x43,0x37,0x63, +0x23,0xfd,0x2d,0xd5,0x28,0x88,0x08,0x9c,0x93,0xc4,0x28,0x8c, +0x38,0x04,0xad,0x37,0x0a,0x84,0x65,0x43,0x1c,0x36,0x30,0xff, +0x0a,0xff,0x34,0x46,0x41,0x77,0x14,0xb4,0x47,0xa1,0xa1,0x80, +0x8c,0x12,0x19,0xa7,0xf5,0xa9,0x19,0xa5,0x48,0xec,0x52,0xf7, +0x8e,0xe8,0xf3,0x0e,0x44,0x82,0xf1,0x6c,0xf0,0x0b,0x36,0xfc, +0x64,0x42,0xe3,0x51,0x55,0xe3,0xf2,0x76,0x5c,0x82,0xcb,0xc6, +0x52,0x67,0x82,0xc7,0x79,0x1c,0x6c,0xc9,0xb1,0x02,0x09,0x0b, +0xc5,0x68,0x65,0x4d,0x13,0x44,0xe4,0x2a,0xf0,0x49,0xf4,0xe8, +0xde,0x0a,0xbf,0xc9,0x5e,0x5d,0x85,0x64,0xab,0xd8,0xc1,0xd8, +0xc5,0x38,0x81,0xf8,0x33,0xe9,0x20,0x1a,0x05,0xf4,0xe3,0xd1, +0xea,0xe9,0x89,0xe0,0xa5,0x2b,0x54,0xd5,0x79,0x75,0xca,0xe6, +0x6b,0x49,0x02,0x45,0x66,0x69,0xb3,0x48,0x83,0x6c,0xef,0xb2, +0xda,0x60,0x11,0xe0,0x73,0x8d,0x21,0xf9,0x31,0x0d,0x5a,0x2d, +0x37,0xef,0x24,0x13,0x22,0x4b,0xc2,0xe8,0xf2,0x17,0x7e,0x6d, +0x26,0x51,0xba,0xc4,0x0c,0x8f,0x20,0x7d,0x2b,0x69,0x53,0xe0, +0xc1,0xab,0x44,0xb8,0xbd,0xc1,0xeb,0xc4,0x89,0x41,0x8a,0x44, +0x6e,0xb7,0xd4,0x6f,0x7c,0xe5,0xc1,0x7d,0x28,0x38,0x85,0x04, +0x28,0xa2,0xbf,0x32,0x05,0x19,0x4b,0xdc,0x95,0xef,0xdc,0x8b, +0x52,0x8e,0x54,0x29,0xf3,0x05,0x9d,0x26,0x94,0x73,0xf0,0x29, +0x2f,0xc1,0xd4,0x62,0xca,0xf6,0x6d,0xf2,0x99,0x18,0xa4,0x1c, +0xdf,0x98,0x90,0xd2,0xa2,0x94,0xe5,0x13,0x14,0x11,0x57,0x0a, +0x44,0x96,0x91,0x04,0x23,0x09,0x2e,0x85,0x92,0x71,0x9b,0xb9, +0x46,0xd3,0x08,0xcc,0xe3,0xa6,0x55,0x8f,0x43,0x2a,0x3f,0x4d, +0x56,0x77,0xf3,0x51,0x71,0x43,0xa2,0x64,0xfc,0x37,0xf5,0x26, +0x21,0xe1,0x1a,0xa8,0x43,0xc7,0x7e,0xf3,0xba,0xa2,0x3b,0x8b, +0x64,0x4a,0x6b,0x4e,0x0c,0xc1,0xc9,0x9f,0x56,0x6a,0x08,0x65, +0xe7,0xc9,0xa9,0xda,0x33,0xeb,0x42,0xdc,0x08,0x8e,0x45,0xa6, +0x8f,0xe0,0xee,0xa3,0x40,0x1d,0x79,0x1d,0x9f,0x47,0x51,0xa4, +0x90,0x91,0xc7,0xd0,0xcf,0x60,0x7a,0xfa,0x74,0xe5,0xa5,0x3e, +0x7d,0xf7,0x81,0xe5,0x78,0x20,0x68,0x20,0x13,0xdb,0xec,0x1e, +0xd7,0x16,0xd2,0x87,0xab,0xb7,0x88,0x5c,0x12,0xbf,0x8d,0x47, +0x59,0x6f,0xe8,0xe2,0x06,0xbc,0xe8,0xf8,0x10,0x1d,0x75,0x7b, +0xe5,0x59,0xbe,0xa0,0x08,0x52,0x36,0x22,0xc0,0x22,0x0b,0xdc, +0x30,0xff,0x79,0x29,0xbc,0xe8,0xd6,0xc8,0xc0,0x6c,0xd8,0xad, +0x7d,0x04,0xbb,0xe7,0xa0,0xd6,0x26,0x43,0x3d,0x8b,0xe2,0xa1, +0x0c,0x1f,0x66,0xee,0xae,0xf7,0x69,0x8e,0x04,0xd2,0x5f,0x3f, +0x90,0xe5,0xba,0xa1,0x55,0xae,0x45,0x94,0x60,0xf0,0x47,0x19, +0x14,0x49,0xcc,0x6c,0xcd,0x19,0x85,0xda,0xbc,0xbe,0xdb,0xb4, +0xc3,0xcb,0x20,0x10,0x53,0x58,0x2b,0x8f,0xda,0xa1,0xf8,0x13, +0xfc,0x7e,0x05,0xff,0x5e,0xc1,0xbf,0x2f,0xe1,0xdf,0x4f,0x7c, +0xda,0x46,0xd1,0xb5,0xb8,0xdd,0x84,0xce,0x1b,0x09,0x55,0xa9, +0xa4,0x09,0xa6,0x44,0xb2,0x15,0x7f,0x11,0x49,0x1a,0x5d,0xdb, +0x33,0xf2,0x8d,0x5c,0x5b,0xd4,0x52,0x10,0x16,0xfc,0x3a,0xb6, +0xf9,0xa7,0x08,0xfe,0x7d,0x1c,0xb6,0xea,0x61,0xeb,0x10,0x56, +0x7e,0x6c,0x68,0x42,0xd2,0xff,0x53,0xc4,0x9d,0x36,0x6a,0xfe, +0x44,0x91,0x20,0x91,0xad,0xb0,0x4e,0xb3,0x24,0x56,0x28,0xcd, +0xcd,0x03,0xfa,0xa7,0xae,0x95,0xd7,0x96,0xcf,0x7a,0x8f,0x7a, +0x4f,0x75,0x76,0x47,0xc4,0xce,0x64,0xf5,0x05,0xe0,0xfe,0x91, +0xbe,0x24,0x6b,0x6e,0x04,0x7a,0xfe,0xbc,0xf7,0xdb,0xb5,0x47, +0x45,0x57,0xed,0x2f,0xf4,0xf4,0xf4,0x54,0xfb,0x33,0x3d,0xf8, +0x5d,0xaf,0xa2,0xa0,0x9c,0x68,0x4a,0x72,0x24,0x0e,0x28,0xd9, +0x7c,0xac,0x9f,0xf9,0x1f,0x9f,0xd7,0xcf,0xa1,0x85,0x7f,0x69, +0x30,0xa6,0xdb,0x64,0xf5,0xc7,0x78,0xe3,0xf9,0xf5,0x53,0xc8, +0x69,0x23,0x08,0xe6,0xfe,0x39,0x9f,0x8b,0xe5,0xf0,0xae,0xbf, +0xac,0x8a,0x51,0x73,0x0a,0xff,0x46,0x86,0x80,0x3f,0xcd,0x1f, +0x96,0x19,0xf2,0xb4,0x6e,0xdd,0x29,0x03,0xfb,0x8e,0xc2,0x00, +0x3f,0xf5,0x6e,0x7e,0x55,0x87,0x97,0x8f,0xe1,0x05,0x9e,0x5f, +0xc1,0xbf,0x2f,0x7d,0xa1,0x18,0x87,0x18,0xae,0xa9,0x9a,0xad, +0xbc,0xda,0x69,0xae,0xe6,0x1c,0x3a,0xb1,0xcc,0x0a,0x40,0xf6, +0xb9,0x82,0x2d,0x6a,0x00,0x29,0x1d,0x14,0xf6,0x21,0x27,0x20, +0xf7,0xee,0xc0,0x5f,0xc9,0xc0,0x83,0x39,0xe1,0x15,0x53,0x57, +0x80,0x38,0xe4,0x1d,0x2e,0xdb,0xef,0xaf,0xa2,0xa9,0x14,0xa7, +0x46,0x48,0x0e,0xf7,0xca,0x75,0x43,0xbd,0xee,0x0b,0xd2,0x1c, +0x43,0x2a,0xb6,0x2a,0x0c,0x65,0x82,0x58,0xb8,0x02,0xbd,0xcc, +0x7a,0x2b,0x8c,0x58,0x59,0x54,0x13,0xdd,0xb2,0xe2,0xed,0x55, +0x5f,0xbc,0x58,0xc0,0xee,0xf3,0x3e,0xe2,0x75,0x84,0x69,0xa4, +0xff,0x49,0xa0,0xa9,0xdc,0xaa,0x90,0x83,0x43,0x3d,0x2c,0x2c, +0x4a,0xef,0x01,0x91,0x01,0x95,0x2c,0xbe,0xaf,0xd2,0xaa,0xdf, +0x25,0xb4,0x37,0x37,0x5e,0x4c,0x51,0x8d,0x7f,0x97,0xdc,0x7e, +0xbe,0x5e,0x78,0xb5,0xbf,0x7a,0xdd,0x76,0xad,0x4e,0xf9,0x74, +0xa9,0x48,0xdf,0x31,0xfe,0xfc,0x30,0x87,0x71,0x20,0x95,0xc5, +0x53,0x8d,0xd6,0x92,0xa2,0x96,0xd6,0x4a,0xd6,0x79,0x5c,0xc5, +0x97,0xf3,0xf9,0xf8,0x61,0xa1,0xaa,0x51,0x9d,0x05,0xf8,0x2c, +0x1b,0x51,0x75,0x4f,0xc2,0x75,0xb9,0x37,0x55,0x00,0xa9,0xcf, +0x10,0xf0,0xaa,0x0a,0x44,0xe9,0xcb,0xf9,0x63,0xb2,0xfc,0x2c, +0x26,0x83,0xaa,0x54,0xeb,0x3a,0x00,0xac,0xa2,0x01,0x5d,0x74, +0xf0,0x29,0xcc,0x4d,0x3a,0x99,0x88,0xc7,0x74,0x04,0x4a,0x84, +0x3d,0x5a,0xa4,0xb7,0xb3,0x28,0xb8,0xe2,0x13,0x95,0x5a,0x03, +0xd4,0x6a,0x4d,0xcd,0x1d,0x1e,0xe6,0x75,0x1b,0x94,0xd5,0xe6, +0x0d,0x64,0x1d,0x72,0x99,0xba,0x9c,0x3c,0x29,0x75,0x0b,0x25, +0xea,0x1e,0xa1,0xbf,0x52,0x31,0x0f,0xa1,0x91,0x1c,0xe3,0x8f, +0x92,0x1b,0x9c,0x0c,0x2a,0x9a,0x59,0x88,0xf4,0xf8,0x75,0x46, +0xd6,0x96,0x5a,0xb4,0x62,0x00,0xfc,0x25,0x49,0xc6,0xa3,0x78, +0xf3,0xe9,0x60,0xb0,0x4c,0xde,0x38,0xf2,0x91,0x92,0x87,0x2e, +0x82,0xd4,0x00,0xdf,0x25,0x76,0x60,0xd1,0x0e,0xeb,0xf4,0x32, +0x28,0x8a,0xc0,0x29,0x5b,0x63,0x66,0x06,0xcb,0xc8,0x76,0x26, +0xd5,0xd5,0x63,0xb1,0x38,0xf7,0x2d,0x1d,0x1a,0xa3,0x79,0x8d, +0xdb,0x35,0xd0,0x37,0xf5,0x99,0x71,0x3d,0xec,0xb7,0x1b,0xe1, +0xee,0x96,0xed,0x68,0xd3,0x73,0xad,0xf9,0xe0,0x76,0xfc,0xbe, +0x2d,0x90,0xd1,0x2f,0x2a,0xda,0xc1,0x0b,0xf6,0x9d,0x4d,0xd1, +0x20,0x15,0xad,0x11,0x38,0xb9,0x97,0xb4,0x88,0xc8,0x7c,0x2f, +0x6a,0x99,0xd4,0x57,0xb0,0xfe,0xac,0x66,0xfc,0xef,0x40,0x70, +0x91,0xd8,0x3f,0x7f,0x30,0xb1,0x5f,0xcd,0xff,0xe9,0xc4,0xfe, +0xe5,0x3d,0x89,0x05,0x12,0x57,0x77,0xbb,0x47,0xe8,0xd4,0x80, +0xec,0x24,0xd6,0x85,0xdb,0x9b,0xe2,0x69,0x19,0x82,0xdf,0x49, +0xca,0xa9,0x75,0x3b,0xdb,0xf5,0x7c,0x8b,0x7e,0x43,0x5b,0x7e, +0xa7,0x56,0x7c,0x89,0xab,0x94,0x04,0x17,0x3f,0xf9,0xa6,0x98, +0xa3,0xf8,0xfc,0xb2,0x23,0xb7,0x3e,0x6f,0x0e,0xf5,0x82,0xc6, +0xf3,0x0d,0x82,0x9d,0x15,0xfe,0x51,0x1d,0x29,0x7c,0x50,0x85, +0xeb,0xf7,0xaf,0xf0,0x7b,0x78,0xff,0xf0,0x0a,0x7f,0x78,0x9f, +0x0a,0xcd,0x4a,0xf2,0x1f,0xa9,0xf9,0x4e,0x8a,0x12,0xb2,0x79, +0xcf,0xf1,0xf9,0x8f,0x27,0xb2,0x55,0x46,0xa4,0x2a,0x9f,0xac, +0x17,0xf1,0x6c,0x44,0x44,0x10,0xc1,0xfb,0x0b,0xed,0x4f,0xf3, +0x59,0x45,0x67,0x1e,0xfd,0xb5,0x57,0x6f,0xf4,0x5f,0x8f,0xde, +0x9d,0x6c,0xff,0xf5,0xa8,0xb9,0x42,0x27,0xd6,0x72,0x01,0x53, +0x4a,0xdf,0xa9,0xaf,0xb6,0x1f,0x3f,0x45,0x75,0x85,0x0b,0x52, +0x4b,0xab,0xb6,0xe8,0xb5,0xae,0xcd,0xea,0xde,0xe8,0xea,0xec, +0xa2,0x8b,0x5b,0xbc,0x76,0x2b,0x39,0xae,0x12,0x0a,0xd2,0x16, +0xff,0x8c,0x19,0xb1,0x84,0xe5,0xd3,0x08,0xd9,0x0a,0x6b,0xca, +0x7d,0xf9,0xfb,0xc7,0x7f,0xe8,0xc4,0x52,0x4e,0xe4,0xe8,0x3d, +0x85,0x17,0x68,0xfc,0xe6,0xe6,0x1f,0x2f,0xc1,0xc7,0x45,0x4a, +0xef,0xdf,0x93,0x52,0xdc,0x3b,0xb6,0x4e,0xfe,0xe9,0x0c,0xfd, +0xd3,0xfb,0xce,0xd6,0xe9,0xec,0x01,0x06,0xcc,0x3f,0x9d,0xce, +0xaf,0xde,0x93,0xce,0x57,0xc9,0x10,0x96,0x3e,0xff,0x7c,0x3a, +0x5f,0xbd,0x37,0x3f,0x27,0x13,0x34,0x39,0xfe,0xc7,0x13,0x5b, +0x22,0xa3,0x5f,0xbe,0x27,0xb1,0x9f,0x4e,0xbf,0x9d,0xe6,0x89, +0x74,0xe9,0x88,0x01,0xc2,0x5a,0x73,0x94,0x12,0x12,0xb5,0xfc, +0xdc,0x3a,0xa4,0xe3,0x5c,0x5f,0xc3,0x06,0x12,0x6f,0x18,0x91, +0x40,0xd8,0xf0,0x96,0xce,0xa0,0x6f,0x49,0xb5,0x73,0xe5,0x6f, +0xa3,0x11,0xd6,0x85,0x53,0x37,0x26,0xb3,0x31,0x07,0x4c,0xc0, +0x6f,0xb3,0xe8,0xed,0x55,0xc0,0x1b,0xd4,0x7a,0x4d,0xbc,0xbd, +0x8b,0xfe,0xfd,0xdf,0xf9,0x93,0xbf,0x78,0x90,0x79,0x6f,0xfd, +0x23,0x8c,0xdb,0xf8,0x76,0x1a,0x59,0x49,0x1f,0x9f,0xe9,0xc0, +0x2e,0x6f,0xb3,0x7a,0x71,0x3f,0xfc,0xf6,0x4e,0xd4,0x82,0x9a, +0x68,0xf9,0x65,0x79,0x53,0x99,0x57,0xb5,0xc0,0x48,0x57,0xc9, +0x32,0x9e,0x7c,0x9b,0x2c,0x87,0xe4,0x30,0xa2,0xbc,0xab,0x17, +0x9c,0xbd,0xb3,0xaf,0x0d,0xcc,0x9e,0x3b,0x1e,0xa7,0x7b,0x4b, +0x7b,0x97,0x9b,0xf1,0x67,0xbc,0x8b,0x78,0xe6,0xc8,0xb8,0xea, +0x4c,0xfb,0xbf,0x94,0x1c,0x6a,0x97,0x1c,0x68,0xf3,0x8d,0x11, +0x24,0x54,0x5d,0x4d,0xe8,0xd3,0xc7,0xa8,0xea,0x2c,0x7b,0xbf, +0x53,0xd5,0x5d,0x67,0xef,0xe6,0xee,0x8a,0x4b,0xab,0x42,0xfa, +0x46,0x0c,0xf2,0x0f,0x0e,0xf8,0x48,0xae,0x82,0xcc,0xc2,0x19, +0x1d,0xa3,0x53,0xaf,0x95,0xc7,0x72,0x9c,0xfe,0x32,0x9b,0x7f, +0x4d,0x2e,0x4a,0xdc,0x8b,0x35,0xaa,0x71,0x35,0x7f,0xf9,0xea, +0x1b,0xb5,0x8a,0x2c,0x22,0xe0,0xaf,0x17,0xa4,0x57,0x04,0xbe, +0x1e,0xc8,0xb4,0xf5,0x78,0xe5,0x37,0x0e,0xd2,0x29,0xc2,0x3a, +0x77,0x1b,0xff,0x15,0x5f,0x2f,0xb0,0x77,0x12,0x36,0xd1,0x90, +0x47,0x3c,0x55,0x0e,0xa6,0x38,0x5e,0x54,0x44,0x2e,0xa4,0xd8, +0x2c,0x22,0xb5,0x7d,0x15,0xda,0x44,0xa2,0xbf,0xac,0x4c,0x16, +0xd0,0xac,0x4d,0xa3,0xa8,0x08,0xa4,0x2c,0xd2,0x7a,0xb2,0x3d, +0xcd,0x0d,0x36,0x6e,0x54,0xe5,0x18,0x89,0xa9,0x71,0xbf,0xe0, +0xb6,0xdd,0x5b,0x1c,0x87,0xa7,0xc7,0x67,0xc9,0xd9,0xd6,0x97, +0x2d,0xea,0xb5,0xfa,0xfd,0x76,0xda,0x95,0xc8,0x7b,0x4c,0xd1, +0x51,0x91,0x8c,0x5e,0xda,0x08,0xfb,0x9f,0x94,0xa5,0xf7,0x8f, +0xb8,0x50,0x17,0x20,0xda,0x69,0xbf,0xdd,0x73,0x80,0x6c,0x15, +0xfe,0x1c,0xd5,0x16,0x49,0xfb,0xba,0x09,0x72,0xea,0xa2,0x1d, +0xd1,0x3f,0xc0,0xc5,0xc4,0x5a,0x7b,0x2f,0x70,0xd3,0x7d,0x79, +0x8a,0x99,0x23,0xa1,0xc2,0x9b,0x83,0xba,0x1d,0x13,0xd9,0x38, +0xb5,0x7d,0x4b,0xe2,0xeb,0x22,0x19,0x39,0xd2,0xae,0xbe,0xb3, +0xc3,0xa4,0x83,0x83,0x17,0xaa,0xa8,0x89,0xe3,0x65,0xf6,0x57, +0xa6,0xe5,0x74,0x31,0x0c,0xca,0x8d,0x2b,0x50,0xbe,0x9d,0x69, +0x4c,0x73,0xd3,0x9d,0x0f,0x25,0x95,0x27,0xb2,0x2a,0x67,0x3d, +0x2c,0x11,0x11,0xcf,0x3a,0x8a,0x80,0x6e,0x71,0x60,0x84,0x81, +0xdf,0xae,0xf1,0x0c,0x6b,0x0c,0xca,0x14,0xfc,0xc1,0x41,0xb1, +0x80,0xca,0x33,0xe7,0xac,0x94,0x7d,0x70,0xa0,0x39,0x14,0x71, +0x0a,0x7d,0xc2,0x0b,0x4d,0x51,0xaf,0xe8,0x02,0xb7,0xdc,0x11, +0x86,0xed,0x96,0x0e,0x4b,0x5c,0x85,0x05,0x6f,0xaf,0xe6,0x4e, +0xaf,0x63,0x33,0x5c,0xb7,0x4c,0xdd,0x47,0x20,0x98,0xdf,0xf1, +0x95,0x6a,0xcc,0xb1,0x58,0xde,0xe0,0xdb,0xba,0x30,0xef,0x25, +0xf6,0xd9,0x8a,0x8c,0xcd,0x40,0x75,0x3d,0xaa,0x2b,0x9d,0x8a, +0xb6,0x6d,0xcd,0x3a,0xbf,0xc2,0x8d,0x49,0x4e,0xc6,0xaa,0x5d, +0x32,0xb9,0xa2,0xf0,0x3b,0xf5,0x76,0x09,0xbc,0xa6,0xb8,0x9d, +0x13,0xe2,0x83,0x83,0xde,0x3b,0x7a,0xd0,0x8d,0xda,0x12,0xd1, +0xfd,0x0f,0x91,0x8a,0xdc,0xf8,0x30,0x1a,0xb9,0x84,0xb3,0x5a, +0x71,0xe3,0x48,0x09,0xaf,0x10,0x55,0x37,0x6c,0x13,0xbf,0x76, +0x39,0x77,0x31,0x17,0xd9,0x3c,0x79,0x3d,0xfb,0x21,0x58,0x61, +0x42,0x52,0xfe,0x4b,0x72,0xf3,0xd2,0xfb,0xbb,0x2f,0xc9,0xb5, +0x68,0xe5,0xdc,0xb5,0xc8,0xa4,0x8a,0x02,0x5f,0xd8,0x9e,0x96, +0xb2,0xf2,0xcb,0x79,0x73,0xa7,0x92,0x46,0xea,0xf0,0x46,0x7f, +0xb2,0x79,0xa3,0x92,0x40,0xef,0x77,0x5e,0xdc,0x00,0x1b,0xb9, +0x4c,0xc7,0x37,0x39,0x8d,0x46,0xaa,0x7b,0xf1,0x06,0x7a,0x81, +0x21,0xdc,0x8b,0xf8,0xfb,0x0c,0x6a,0xc3,0xcb,0x9f,0x24,0x33, +0xf4,0xff,0xb7,0x57,0xdf,0x7c,0x2d,0x17,0x1d,0x32,0xab,0xb9, +0x4c,0xb2,0xc5,0x7c,0x96,0x25,0xdf,0x43,0xb7,0xb9,0x8d,0xba, +0x5b,0x4d,0x27,0x06,0x03,0xd9,0x00,0x93,0x95,0x02,0x2e,0x45, +0xe6,0x43,0xd2,0xe7,0xcd,0xe1,0x32,0x81,0xaa,0x79,0x8a,0x71, +0xdd,0x26,0x34,0xd9,0xe4,0x89,0xac,0x39,0xed,0x12,0x83,0xf9, +0x68,0x23,0xbf,0x96,0x97,0xa5,0xc9,0x22,0x74,0xbd,0x7a,0x88, +0x27,0x5f,0x2c,0x63,0xfe,0xfc,0xa3,0x9c,0x38,0x52,0xb4,0xc7, +0xd1,0x3b,0xb4,0x58,0x47,0x8b,0xc9,0xda,0x71,0xf3,0xb8,0x79, +0x56,0xdb,0x76,0xb0,0x4b,0x9a,0xb3,0xf9,0xe3,0xd3,0x93,0xa7, +0x1e,0x8b,0x22,0x53,0x57,0x9d,0xb7,0xe5,0x4b,0x66,0x20,0x8a, +0x22,0x1b,0xbe,0x42,0xd7,0x5e,0x51,0xaf,0xcf,0x3e,0xbe,0x84, +0x4a,0x36,0x08,0x60,0x76,0x75,0x3e,0x8d,0x33,0xa5,0xd8,0x34, +0x8f,0xf2,0x49,0xc8,0x54,0x23,0x23,0xf5,0x60,0x27,0x7e,0x3e, +0x49,0xa6,0x52,0x4f,0x68,0x66,0xe4,0xf2,0x10,0xfc,0x11,0x44, +0x13,0xc8,0xe7,0x9f,0x8e,0x5c,0x47,0x52,0x95,0x5e,0x11,0x19, +0x3b,0xe5,0xa7,0x88,0xc1,0x3e,0xae,0xb1,0xd1,0x6c,0x0f,0xad, +0xbc,0xb7,0xc3,0x78,0x35,0xbc,0x43,0xcf,0x3b,0x55,0xcd,0x31, +0x12,0x88,0xef,0xea,0xd2,0x95,0x41,0xcd,0x05,0x58,0xea,0x77, +0xd2,0x46,0xa3,0xe3,0xab,0x10,0x90,0x04,0xdc,0x33,0xc2,0x47, +0xe9,0xdb,0xad,0x5a,0xed,0xba,0x52,0x21,0x69,0xf4,0x6a,0xa3, +0xf4,0x4d,0xcd,0x6f,0x66,0xab,0xcd,0x84,0xae,0xa9,0xbf,0x5d, +0xce,0x61,0x0f,0xb1,0xda,0x78,0xb5,0xf9,0x22,0x1e,0xa6,0xab, +0x4d,0x0d,0x6f,0xcb,0x6b,0xbe,0x22,0x1a,0xfd,0x32,0x48,0xbb, +0x84,0xe3,0xeb,0x84,0x91,0x5c,0x2f,0x94,0x65,0x77,0xa4,0x79, +0xd4,0x54,0x4c,0xd0,0x79,0xc2,0x2a,0x00,0x15,0x7d,0xba,0x82, +0x75,0xec,0xe0,0x81,0xf5,0x7f,0x01,0x51,0xd3,0x86,0xa8,0x2a, +0xf9,0xf5,0xab,0xe7,0xcb,0x7e,0xfd,0x8a,0x14,0x0c,0x36,0xaf, +0x94,0xcc,0xcf,0x5e,0xbd,0x7a,0x85,0x99,0x7f,0x4c,0x86,0x13, +0x19,0x3e,0xd1,0x25,0x99,0x8b,0x5a,0x9c,0x89,0x8a,0xf8,0x6c, +0xc6,0x75,0x9e,0x25,0xc9,0xf4,0xb6,0x65,0xc7,0xfd,0xae,0xa2, +0x8d,0xd6,0xf7,0x5c,0x06,0xba,0x8e,0x1d,0x22,0xf6,0x68,0xbb, +0xb5,0xf7,0x81,0xde,0x94,0x5b,0xac,0x67,0x6a,0xfc,0xfa,0x95, +0x55,0x67,0xa1,0x98,0xae,0x7a,0x07,0x0b,0xca,0x1a,0x28,0x16, +0xcb,0x74,0xbe,0xc4,0x0f,0x67,0xdf,0x95,0x31,0xb5,0xaa,0x99, +0xa6,0xd8,0x76,0x6b,0x3b,0x5b,0x30,0x55,0x38,0x41,0x09,0xaf, +0x62,0x3c,0x1f,0x88,0xaf,0x06,0x5d,0xfc,0x1b,0x0d,0xba,0x41, +0x3b,0x38,0x0a,0x90,0xde,0x26,0x8c,0xc2,0x9d,0x65,0xb1,0x10, +0x94,0x45,0x14,0xf8,0x37,0x8a,0xad,0xb2,0x53,0x7b,0xe1,0x4c, +0xc3,0x4a,0xdc,0xf0,0x20,0x40,0x77,0xa9,0xf2,0xb2,0xdd,0x89, +0xd0,0xca,0xa1,0x94,0xa2,0x82,0x97,0x27,0x5e,0x19,0xd1,0x55, +0x3c,0x2c,0x6c,0x3d,0x69,0xf9,0x14,0xeb,0x30,0xae,0xb0,0xdc, +0x05,0xba,0x63,0x98,0x50,0xe2,0x88,0xad,0x66,0x3b,0xa6,0x04, +0xda,0x28,0x12,0xfc,0xc0,0x85,0x47,0x9b,0xfe,0x38,0x1a,0xb0, +0x7f,0xc6,0xaa,0x1a,0x6e,0x98,0xc5,0x4c,0xbd,0x2a,0xce,0xa6, +0xdd,0x7b,0xd5,0xb8,0xb3,0xbc,0xa6,0x40,0x31,0x73,0x2b,0xa3, +0x96,0xfe,0xff,0x97,0x6d,0xd0,0xd1,0xff,0x9b,0xd9,0x66,0x51, +0xe0,0xb0,0x2d,0xc9,0x7d,0xa9,0x5f,0xe0,0xdc,0xf0,0x77,0xe1, +0xdd,0xb0,0x94,0x7b,0xc3,0x7d,0xf9,0xe7,0x99,0x5e,0x17,0x83, +0xab,0x21,0x99,0xd7,0x0d,0xfc,0xdd,0xcc,0x1c,0xfe,0x23,0xd9, +0x59,0x45,0x90,0xfc,0x74,0x2b,0x16,0xc3,0x3e,0xb1,0x37,0x7b, +0x98,0x56,0xf2,0x36,0x23,0x6b,0x27,0x9b,0xad,0x1c,0xf0,0xb3, +0x8a,0xb7,0x36,0x8d,0xbc,0x67,0x8d,0xa3,0xba,0x66,0x12,0xac, +0x78,0xb2,0x3a,0x36,0xca,0x18,0x08,0xe7,0x61,0x2b,0xdb,0xa3, +0xcb,0xaa,0x93,0x00,0x1e,0x51,0x49,0x5c,0xad,0x89,0x72,0x84, +0x4f,0x31,0x7c,0x89,0x32,0xd6,0xdc,0xa7,0x01,0xa0,0x97,0xa5, +0x0d,0xb4,0x33,0xb0,0xbc,0x69,0x1d,0xfa,0xae,0x31,0xf5,0x8f, +0xea,0xf5,0xfb,0x92,0xb6,0xd8,0xc5,0x76,0x75,0x8f,0x8b,0x46, +0x59,0x52,0x75,0xa7,0xea,0x5b,0x09,0x61,0xf9,0x0d,0x32,0x6d, +0xe4,0xa8,0xc7,0x42,0xee,0xe6,0xfe,0x14,0xb9,0x21,0x9e,0x61, +0x33,0x7a,0xb8,0xa8,0x87,0xe2,0xce,0x76,0x61,0xf5,0x27,0x5f, +0xbc,0x89,0xea,0x0c,0xd7,0x23,0x37,0x28,0x49,0xf4,0xa7,0x86, +0x36,0x2a,0x4a,0xba,0x6f,0xea,0xc9,0xa1,0xa7,0xf2,0xfb,0x8d, +0x37,0x7e,0xfb,0x8d,0xe4,0xee,0x28,0x2d,0xe5,0x6f,0xc5,0x41, +0x09,0x05,0x21,0xe4,0x55,0x19,0x73,0x9e,0x8e,0x9d,0x7c,0x5f, +0xd8,0x69,0xd2,0x99,0x90,0xe6,0x92,0xcc,0x35,0xa1,0xa1,0x8c, +0xb3,0x24,0xce,0x28,0x71,0x2c,0xd4,0x3c,0xf5,0x6d,0x2e,0xf1, +0x89,0xda,0x7c,0x69,0x28,0xd5,0x34,0xbe,0x9b,0x24,0x37,0xab, +0xb6,0x35,0xc1,0xad,0x61,0xf2,0x16,0x77,0x29,0x6b,0x81,0x52, +0x97,0x62,0x93,0x79,0xa9,0xb7,0xb1,0x13,0xc8,0xba,0x4b,0x23, +0x1d,0x31,0xbd,0x73,0x97,0x5e,0x4d,0xe6,0x1d,0xee,0x87,0x69, +0x3a,0x8a,0x26,0xf3,0x3a,0x24,0x5d,0x5d,0x85,0x1d,0xd5,0xeb, +0x22,0xee,0x41,0x46,0x5f,0xc0,0x1f,0xff,0x93,0x75,0x17,0x10, +0xc3,0x53,0x3d,0x6c,0x03,0x1a,0x78,0xd8,0xea,0x2f,0x9e,0xb6, +0x62,0x89,0xf1,0xa6,0xfe,0xd9,0x64,0xae,0x3f,0x29,0x23,0xb4, +0xcb,0xd4,0xb5,0x15,0xb5,0x16,0x99,0xdb,0xad,0xda,0x8b,0x68, +0x8e,0x5b,0xdc,0x2f,0x3d,0x60,0x84,0xed,0x8b,0x06,0xc1,0x20, +0x5f,0x91,0x55,0xb8,0x89,0x7d,0x63,0xfa,0xcf,0xe0,0xa2,0xe8, +0x5b,0x0e,0x24,0xf1,0x87,0x94,0xd5,0xdd,0xc3,0xcd,0x8d,0x3d, +0x1e,0x72,0x01,0xe3,0x57,0x22,0x85,0x81,0xee,0xcc,0x03,0x53, +0x50,0x2f,0x51,0xf0,0xc4,0xfe,0xd0,0x62,0x58,0xe4,0x4e,0x3d, +0xff,0x70,0xda,0x68,0x88,0x95,0x1c,0xd5,0xd3,0xbe,0x50,0x0f, +0x7a,0x9c,0xeb,0xc1,0x1a,0xad,0xdc,0xed,0x03,0x12,0x01,0x2b, +0xb4,0xa9,0xb3,0x6e,0xe5,0x81,0x41,0xfe,0x62,0x6c,0x23,0x44, +0x0c,0xd0,0x44,0x49,0x4a,0x05,0xc9,0x82,0x59,0xc9,0xa6,0x45, +0x65,0x61,0x95,0xb2,0x6e,0x2e,0x0b,0x09,0x7a,0x07,0xa3,0x80, +0x98,0x8a,0x38,0x5d,0x66,0x95,0x8c,0x58,0x04,0xd2,0x54,0xd5, +0xe6,0x05,0x28,0xbf,0x45,0xa8,0x03,0xe9,0x0b,0xc6,0x60,0x68, +0x09,0xae,0x66,0xb0,0xc8,0x9b,0xf9,0x6c,0xbe,0x4a,0xb9,0x48, +0x4e,0x6f,0x11,0x44,0x0b,0xab,0x24,0xe8,0x39,0x8b,0x24,0x84, +0x22,0x7a,0xde,0xa6,0x0b,0x7b,0x1f,0x8b,0x9f,0x29,0x7a,0xb3, +0xa2,0x86,0x95,0xc6,0xa3,0xbd,0x7e,0xc7,0x09,0x52,0x88,0xb6, +0x38,0xe4,0x63,0x4a,0x17,0xc0,0x45,0x35,0xe0,0xfc,0x52,0x7a, +0x76,0x86,0x47,0x9b,0xd6,0x29,0xfb,0x98,0x98,0x1a,0x0f,0xdb, +0x33,0x50,0xec,0x80,0x08,0xc9,0x40,0x58,0xa4,0xdc,0x80,0xcf, +0x38,0x3a,0x11,0xa8,0x67,0xc8,0xeb,0xdd,0xf7,0x0d,0x5d,0xf0, +0x62,0xed,0x11,0xb1,0x24,0xb5,0x86,0xbe,0x6a,0x5a,0xcc,0xed, +0xeb,0x92,0x69,0x0c,0x3b,0x02,0xfb,0xac,0x19,0x9b,0x6c,0xe2, +0x22,0xc8,0x6c,0x2a,0x3d,0x4e,0x36,0xb6,0xe7,0x9b,0x58,0xaa, +0x6b,0x4a,0xb6,0x5a,0x0e,0xef,0x68,0xbf,0x8b,0xf9,0x98,0xc5, +0x76,0xf4,0xf0,0xa4,0xa7,0x03,0x4c,0x25,0x84,0xac,0x9d,0x4b, +0x50,0xca,0x8c,0x72,0xa4,0x72,0x6e,0x20,0xb4,0xf0,0xde,0x83, +0x2c,0xe3,0xcd,0x82,0x33,0x79,0x85,0x35,0x03,0xd2,0x4b,0xd1, +0xab,0x9c,0x72,0xfc,0x32,0x97,0x2b,0x78,0x07,0x19,0x6d,0xfd, +0xf9,0x64,0x5b,0xd5,0xb7,0xd5,0x15,0x4a,0x68,0x39,0xb5,0x2c, +0x6f,0xf3,0x03,0xc8,0x1c,0x40,0x51,0x9f,0x99,0x6d,0x67,0x73, +0x38,0x9f,0xc1,0x76,0x5b,0xb2,0xba,0x27,0x07,0x67,0xc6,0xac, +0xce,0xb9,0x30,0x32,0x21,0xe0,0xa4,0x6f,0x5b,0x90,0xc3,0x52, +0xfd,0x49,0x8e,0x1e,0xc3,0xa2,0x02,0x6d,0x51,0xde,0x7c,0x11, +0x29,0x4c,0xe8,0x34,0x12,0xc3,0x7e,0x84,0x47,0xe8,0xff,0xc4, +0xf1,0x07,0x49,0x35,0xac,0xee,0x96,0xf3,0x47,0x3a,0x6c,0xfb, +0x1c,0x0f,0x03,0xbc,0x5a,0x3a,0xbb,0x21,0xd7,0xf3,0x7c,0xb4, +0x54,0xe3,0x03,0x9c,0x7b,0x19,0x3c,0x0b,0xa8,0x1f,0xa3,0x6a, +0xe3,0xaf,0x67,0xf0,0xa4,0xf2,0x36,0x59,0xbe,0xa2,0x73,0x41, +0x7d,0xb9,0x4a,0x68,0x7d,0xbd,0xd2,0xa2,0xca,0x30,0xbc,0x27, +0x56,0xcd,0xbf,0x09,0xfd,0x06,0x57,0x04,0x49,0x8b,0x0f,0xef, +0x3e,0xb2,0x9c,0x7e,0xe2,0x87,0x1f,0x57,0x08,0xde,0xf1,0xf9, +0xf0,0x8a,0x7a,0xe8,0xfe,0x68,0x6c,0xaf,0x57,0x8a,0x45,0x3e, +0x29,0x2f,0x62,0x9f,0x94,0xc9,0x8d,0x8c,0x35,0xd4,0xe7,0x03, +0xf4,0x24,0x20,0x27,0x18,0xc7,0xec,0x1a,0x89,0xe7,0x5c,0x10, +0x18,0xa8,0x6a,0x36,0xc4,0xe3,0x61,0xce,0xf3,0x39,0x63,0xe7, +0x07,0xb8,0xca,0x34,0xdb,0x24,0x6d,0x2d,0xfa,0x2d,0x59,0x94, +0x24,0xd8,0xf0,0x9c,0xe4,0x4a,0x3c,0x5a,0x70,0xa3,0x56,0xe1, +0x4f,0xf2,0x99,0x0e,0xf1,0xee,0x2e,0xce,0xda,0x76,0xed,0xf6, +0xe9,0x19,0x94,0xc0,0xa0,0xae,0x37,0xe9,0xba,0x2e,0xab,0xc2, +0x2d,0xf9,0x56,0xdc,0x26,0xab,0xf2,0x32,0x98,0xdd,0x2b,0x14, +0x44,0x47,0x61,0xb9,0x12,0xaa,0x8d,0xbb,0xcb,0xf1,0xc7,0xc7, +0xe8,0xc4,0x6d,0x3a,0x7f,0x93,0x94,0x57,0x09,0xcf,0x51,0xa1, +0xa4,0xb0,0xc8,0x3d,0x38,0x90,0xdf,0xd0,0x52,0x25,0x4c,0x0e, +0x2a,0x96,0x76,0xce,0x2b,0x84,0xd2,0x4e,0x16,0x49,0xa5,0xdd, +0xe3,0xbf,0x73,0x95,0xd5,0xd6,0x17,0xac,0xa7,0x58,0xa3,0xe4, +0xd1,0x1a,0x0d,0xf5,0x1c,0x62,0xc5,0x13,0x5b,0x6d,0xe9,0x2f, +0xab,0x95,0xba,0x92,0x5a,0x24,0x5f,0x8b,0xa5,0xa8,0xf6,0xad, +0x66,0x97,0xf6,0x62,0xbe,0x63,0xbd,0x5a,0x6b,0x49,0x54,0x6d, +0x7b,0x91,0x99,0x93,0x43,0xac,0x12,0x35,0xb9,0xfd,0xc1,0x44, +0xe0,0x47,0x91,0xdb,0x3f,0x98,0x71,0x70,0x70,0x63,0x1d,0xf1, +0x60,0x11,0x63,0x93,0x80,0x1e,0xa5,0x54,0x4f,0xf9,0x5b,0x73, +0x00,0x6c,0x30,0x44,0xb5,0xd7,0xeb,0x20,0xa8,0x89,0x02,0x5a, +0xb7,0x22,0x97,0x0c,0x5c,0x88,0xcd,0x92,0xcc,0xb9,0xc4,0xd0, +0x27,0xe8,0xb8,0x52,0x87,0x7f,0x78,0x1c,0xcb,0xda,0x55,0x8c, +0x92,0x05,0x6e,0x96,0xf1,0xdb,0x50,0x7c,0xba,0x8a,0xa8,0xd3, +0xdd,0x0f,0xa4,0x96,0xf3,0xc9,0xe4,0x61,0xd1,0xe5,0x1f,0xf9, +0xe9,0x37,0xd4,0xc0,0x18,0xfc,0x36,0xae,0xdc,0xff,0x07,0x75, +0x5b,0xd7,0x5a,0xca,0x9b,0x54,0xfc,0x92,0x17,0x92,0xed,0x99, +0x85,0x32,0xe4,0xe0,0xc5,0x31,0x03,0x5b,0x05,0xd9,0xf3,0x65, +0x7b,0x7c,0x64,0x1c,0xd1,0xd5,0x23,0x1a,0xeb,0xf5,0xbe,0x04, +0xfe,0xc3,0x06,0xbd,0xd6,0x58,0x9a,0x48,0xee,0xcd,0xe4,0x2e, +0x27,0xb2,0xa0,0xc8,0xf6,0x46,0xd5,0x8c,0xc8,0xa4,0xca,0x32, +0x5b,0x3e,0xdf,0xef,0xda,0x12,0x29,0x95,0x4d,0xdb,0xc6,0x91, +0x59,0x38,0x44,0x8f,0x21,0x1c,0xbd,0x83,0x8c,0xed,0x2a,0xcc, +0xf2,0x9d,0xfc,0x46,0x61,0x0b,0x23,0x5b,0x3a,0x19,0x07,0x23, +0xf7,0xdf,0x49,0x2d,0xe9,0xe0,0xb7,0xfb,0x4a,0xf2,0x86,0x3b, +0x0b,0x03,0x0b,0xa9,0x3a,0xde,0x6d,0xf7,0x45,0xde,0x53,0xe9, +0xfd,0xa8,0x12,0xb3,0x1e,0x7f,0xdc,0x5c,0x35,0xaa,0xb8,0x06, +0x5f,0xf6,0x97,0xb9,0x8f,0x91,0x63,0x06,0x71,0xed,0x25,0x47, +0x00,0x47,0x42,0xce,0xfb,0x44,0x98,0x22,0x51,0x46,0xb0,0x07, +0xe5,0xaf,0xe9,0x5f,0x8b,0xa5,0xe5,0x63,0x1b,0xfe,0xe1,0xec, +0xf2,0x8e,0xc5,0xa4,0x64,0x68,0x67,0x6d,0x45,0x1d,0x83,0xea, +0x16,0x42,0x1b,0x65,0x75,0xb6,0xb4,0x96,0x9e,0x8d,0x4a,0x38, +0x2f,0x26,0xa7,0x45,0x83,0x26,0x6b,0x41,0x96,0x66,0xba,0xf8, +0x31,0x32,0x2e,0x78,0x70,0x08,0x1a,0x79,0xef,0x58,0xf1,0x5a, +0x2d,0xb4,0x15,0x16,0x82,0xb8,0x53,0x2b,0x8f,0x44,0xd9,0x23, +0xba,0xf6,0xe2,0x58,0x45,0x57,0x02,0x54,0xba,0xb0,0x90,0x93, +0xbb,0x02,0x77,0x05,0xe6,0xc9,0xaf,0xad,0xb0,0x7b,0x14,0x02, +0x83,0x03,0xc7,0x54,0xc9,0x56,0xce,0xe8,0xfc,0x91,0x4f,0x80, +0x12,0x5c,0x37,0xc4,0x2c,0x08,0x96,0xa3,0x64,0x99,0xe7,0x15, +0xa9,0x33,0x73,0x4c,0xd1,0x8f,0x08,0x2a,0x8f,0xe8,0x7f,0xe4, +0x96,0xb9,0x45,0x54,0x12,0xa2,0x50,0x9a,0xd9,0x5c,0x72,0x02, +0xf0,0x91,0xca,0xb1,0xa0,0xf9,0xdc,0x2b,0xc9,0x9f,0x29,0x4a, +0x7b,0x1e,0x48,0x97,0x9a,0xe3,0x55,0xb2,0xea,0xd0,0xfa,0x11, +0x33,0xcd,0x76,0x25,0xbf,0xad,0xa2,0xfd,0x12,0xae,0x9f,0xa0, +0x68,0x33,0x1e,0x8d,0x3c,0xad,0x3a,0xf4,0x69,0x55,0xb2,0x72, +0x57,0x1e,0x80,0x39,0xb7,0xf2,0x70,0xd7,0x03,0xee,0x84,0x4e, +0x79,0x66,0xf5,0x01,0x55,0x54,0x95,0x2b,0x59,0x47,0xbc,0xe1, +0xb1,0xfd,0x22,0x10,0x15,0x8b,0x09,0x17,0x03,0x3b,0x3d,0x29, +0x41,0x21,0x1c,0x2a,0xdc,0x45,0x05,0xd7,0xf1,0x1b,0x57,0x00, +0xfb,0xcc,0xfe,0xbb,0x26,0x61,0x87,0x3e,0x5f,0xba,0x6d,0x79, +0xbf,0x89,0x98,0x0b,0xd9,0x53,0x31,0x4e,0xc0,0x74,0x46,0x91, +0xdc,0xc5,0x6f,0xd2,0xf9,0x12,0xc7,0xb0,0x0e,0xb8,0x68,0xe4, +0x87,0x6d,0x9e,0x60,0x54,0x3f,0x2c,0x87,0xd6,0x95,0x39,0x5f, +0xee,0xc3,0xd4,0xc5,0x33,0x97,0xbb,0xd5,0x50,0x93,0x12,0x97, +0x95,0x06,0x5d,0xd6,0xc6,0x14,0x76,0xd9,0xb4,0x53,0x60,0x03, +0x00,0xa7,0x06,0xf9,0x23,0xcb,0x18,0x39,0x63,0x20,0x7d,0x4c, +0xf3,0x26,0x99,0x8d,0xe6,0xcb,0x6f,0xa9,0xb1,0xd8,0x01,0xb5, +0xc7,0x64,0x30,0x4e,0x57,0x35,0x51,0x9b,0x66,0xf8,0x67,0xfe, +0x16,0xfe,0x7e,0x45,0x7f,0xe7,0xf0,0xef,0x9b,0x5a,0x1f,0x17, +0x09,0xa3,0x34,0x5b,0xe0,0xe5,0xa6,0xb3,0x50,0x50,0x8e,0x29, +0x55,0x9e,0x32,0x5d,0x94,0xef,0x66,0x76,0x2e,0x6f,0xa3,0x82, +0xeb,0x15,0x9a,0xa7,0x72,0xae,0xc9,0x87,0x8c,0xa7,0x5e,0x8d, +0xad,0x8b,0x4c,0xe0,0x3b,0x6b,0xf9,0x62,0xed,0x0d,0x6d,0x0f, +0x6e,0x74,0x47,0x89,0xb7,0xbe,0xc9,0x2c,0x91,0xb7,0xb2,0x29, +0x59,0xa5,0x68,0x5f,0xbe,0xb5,0x66,0xcd,0xa7,0x1b,0xb5,0xa8, +0x56,0xa3,0x4f,0xbf,0xaf,0xa2,0x00,0x63,0xc2,0x63,0x0a,0x5f, +0xdb,0x19,0xd3,0x50,0xb4,0x08,0xa1,0xdb,0xd1,0x5c,0x46,0x40, +0x5f,0xb5,0x93,0xe3,0xae,0x8a,0x83,0xd0,0x4f,0x5a,0x5d,0x1a, +0x16,0x08,0xd3,0x6f,0xca,0x7b,0x3f,0xbf,0x5d,0x4c,0x33,0xc4, +0x22,0x31,0xad,0xd2,0xeb,0x0a,0xc8,0x60,0x3b,0x1c,0x0d,0x4b, +0x9e,0xe7,0x01,0x8d,0x96,0x75,0x1a,0x4f,0xa0,0x4d,0xbe,0x79, +0x9c,0xe9,0xfb,0x6a,0xa2,0xef,0xe0,0xa0,0xa4,0x4e,0xc4,0xe6, +0xdb,0x43,0x71,0xcb,0x87,0x00,0xc8,0x7f,0x32,0xf9,0x11,0xe6, +0x4b,0x5c,0x6b,0x63,0x6d,0xd4,0xb0,0x1d,0x32,0x51,0xc2,0x61, +0xcc,0xc4,0xda,0xeb,0xd7,0xff,0x7a,0x50,0xf3,0xb5,0xf4,0x99, +0xac,0xe8,0xa8,0xf7,0xfa,0xf5,0xeb,0xbf,0xbe,0xfe,0xd7,0xd7, +0x87,0xaf,0xeb,0xaf,0xbb,0xaf,0x9f,0x5e,0xf7,0x5e,0xf7,0x5f, +0x7b,0xaf,0xfd,0xd7,0xcd,0xd7,0xef,0x5e,0x6f,0xfb,0x47,0xb7, +0x74,0x83,0x2a,0x9d,0xa5,0xc1,0x10,0x6b,0x5e,0xf3,0x4d,0xea, +0xf5,0x75,0x37,0xb7,0xcf,0x14,0xba,0xe7,0xf5,0xe2,0x48,0xc3, +0x46,0x3a,0x6f,0xdb,0xde,0x51,0x4c,0x1f,0x91,0x49,0x56,0x21, +0x1b,0x4d,0xb6,0x5c,0x14,0xa9,0xcc,0xbe,0x41,0x6a,0xd2,0xf8, +0xc2,0x97,0x4c,0x46,0x2c,0x06,0x09,0xe3,0x6f,0x6f,0xd6,0xfc, +0xf9,0x21,0x59,0x6e,0x5e,0x11,0x08,0x54,0x97,0xf9,0x56,0x91, +0x4f,0xa1,0x33,0xf7,0x29,0x05,0x70,0x6e,0xc1,0xaf,0x50,0xf4, +0x93,0x65,0x54,0x34,0xad,0xe8,0xe9,0xc1,0xfe,0x8a,0x1c,0xe2, +0x95,0x58,0x5f,0xc0,0x70,0xa7,0xe2,0x99,0x42,0x5f,0xf3,0xfb, +0x79,0xd4,0xd6,0xcc,0x3b,0x13,0x99,0xe3,0xc8,0xce,0x26,0x40, +0xae,0xf4,0x01,0x62,0xdb,0x29,0x71,0x1f,0xf7,0x2a,0x7d,0xfb, +0x76,0x92,0xa0,0xc3,0x84,0xdd,0x4c,0x62,0x38,0x4a,0xea,0x05, +0xfd,0xa7,0x27,0x76,0xda,0xf0,0x3c,0x9b,0xb8,0x5c,0xf3,0x61, +0x96,0x02,0xbb,0x5e,0xe1,0x22,0xcd,0xc2,0x54,0xe4,0x57,0x16, +0xc9,0x02,0xb9,0xf6,0xfb,0xbc,0x08,0x90,0xae,0x8e,0xca,0xad, +0xb6,0x74,0xf6,0x77,0xf3,0xb9,0x51,0xab,0x3a,0xf5,0x5b,0xcb, +0x80,0xc2,0xa0,0x32,0xca,0x09,0xa7,0xbc,0x52,0xf8,0x66,0x81, +0x2f,0x8a,0x28,0x5e,0x82,0x3c,0x0c,0xc8,0x3b,0x9f,0x80,0x07, +0xf2,0xa5,0xc5,0x6f,0xf4,0xa8,0xf2,0x68,0x3e,0x55,0xc5,0x22, +0xbb,0x96,0xeb,0x4c,0x8b,0x9d,0xc2,0xaa,0xb7,0x58,0xf7,0x7c, +0xd6,0x4a,0x0a,0x43,0x6b,0xe8,0x7b,0x3c,0x46,0x7d,0xa7,0x11, +0xf3,0xd4,0xab,0x5e,0xd1,0x01,0xae,0xae,0x14,0xcd,0xb1,0x40, +0x96,0xd0,0x86,0x27,0x62,0x07,0x82,0x84,0x0a,0x03,0xb4,0x5b, +0x59,0xee,0xc9,0xee,0x8c,0x9d,0x11,0xe6,0x26,0x04,0xf2,0x11, +0xc6,0x39,0xb8,0x50,0xea,0x7a,0xa6,0x0a,0x59,0x3b,0x01,0xa8, +0x16,0xec,0xf6,0x23,0xa8,0xf8,0x74,0x70,0x50,0x53,0x19,0xe8, +0xce,0x82,0x93,0x14,0x2e,0x5d,0xc6,0xf5,0x17,0xe6,0xc3,0x3e, +0xd5,0xa9,0x9a,0xd4,0xe3,0xb6,0x44,0x06,0x34,0x89,0xce,0xa0, +0x2c,0xe9,0x56,0x57,0x72,0xcb,0x7b,0xf6,0x7d,0xfb,0x92,0x94, +0xc1,0x7b,0x74,0xa7,0x82,0x70,0x3a,0x49,0xec,0xdf,0x21,0xcc, +0xb7,0x52,0x81,0xd0,0x26,0x5e,0xfb,0x75,0x4e,0xb9,0xf4,0x90, +0x6b,0xc7,0x32,0xe7,0x94,0x36,0x97,0xd5,0x80,0x9b,0x65,0xbc, +0x86,0x89,0xde,0x65,0x6f,0x6e,0xdb,0xb5,0xbb,0xd5,0x6a,0xd1, +0x3e,0x3a,0x7a,0x7c,0x7c,0x6c,0x3e,0x1e,0x37,0xe7,0xcb,0xdb, +0xa3,0x56,0x10,0x04,0x47,0x90,0x57,0x13,0x6b,0x34,0x04,0x2c, +0x03,0x09,0x2f,0x2f,0x2f,0x8f,0x28,0x17,0x80,0x26,0xe9,0x6c, +0x5c,0x0d,0x84,0xb9,0x00,0x54,0x8e,0xe7,0x87,0xaf,0xbe,0x44, +0xb0,0x8b,0x23,0x72,0x18,0x81,0xe6,0x40,0x04,0x3a,0xcb,0x2a, +0xe9,0xa2,0xdc,0xa3,0xda,0x96,0x0e,0x60,0x60,0x76,0xe3,0xc5, +0x67,0xdb,0x6a,0x97,0xf8,0xf9,0x21,0x9e,0xa4,0x37,0x9b,0xb6, +0x63,0x28,0xa4,0x16,0x30,0xf8,0x6c,0x16,0x30,0x6d,0x58,0xc0, +0xc8,0x53,0x20,0xcc,0xd0,0x1f,0x0b,0xf0,0x42,0xc6,0xca,0xc9, +0xad,0x58,0x78,0xd5,0x93,0xcb,0xa0,0x8f,0x5f,0x84,0x45,0x49, +0x7e,0x15,0xc1,0xf8,0xfc,0xee,0x3b,0x6a,0xa9,0x4d,0x73,0x8f, +0xb3,0xfa,0x6c,0x0c,0xd5,0x46,0xc4,0x5b,0xfe,0x5b,0x35,0x38, +0xd0,0x99,0x5c,0xb9,0xad,0x57,0xd9,0x19,0x7d,0x8b,0x92,0x6b, +0x4c,0xa8,0x99,0x5c,0x0c,0x63,0x48,0x5e,0x49,0xf2,0xf1,0xc9, +0x18,0x74,0x52,0x3b,0x89,0xd5,0x4d,0xc9,0x56,0xe6,0x26,0x31, +0xa0,0x49,0xd4,0x76,0x49,0x4c,0x6f,0x1d,0x23,0x2f,0x8f,0x59, +0x43,0xf6,0x5d,0x06,0xd2,0x6f,0x17,0x40,0x19,0x9b,0x74,0xd5, +0x2e,0x77,0x1c,0x94,0x44,0xb4,0x90,0x6b,0x55,0x67,0x04,0x93, +0x0f,0x3d,0xde,0x41,0x21,0x98,0xdc,0x2b,0xf9,0xee,0x0a,0xcc, +0xde,0x18,0x55,0xa0,0xb0,0x18,0x56,0xa9,0x7f,0x68,0xfd,0x64, +0xbb,0x8c,0xfe,0xdd,0xb9,0x0c,0xda,0xc3,0x33,0x92,0x04,0xc5, +0xa6,0x9e,0x0f,0x5c,0x9b,0xa4,0x2b,0xef,0xe8,0xaf,0x4f,0xaf, +0xb3,0xfa,0xd1,0xad,0xef,0xe7,0x6d,0x45,0x78,0x4f,0x49,0x9c, +0x24,0x0a,0xbf,0xd4,0x56,0x9d,0xda,0x00,0xe4,0xc6,0x7b,0x21, +0x77,0x6c,0xf3,0xd9,0x2a,0x4e,0x67,0x19,0xd5,0x42,0xc7,0x6f, +0xf2,0x1b,0x88,0x70,0x6b,0x1d,0xfd,0x6b,0x7c,0x4e,0xcf,0xd4, +0x08,0x7b,0xcd,0xb7,0xb1,0x3a,0x8c,0x94,0xfc,0x91,0x6e,0x69, +0x10,0x3b,0x7f,0xd1,0x2c,0xd9,0xaa,0x6a,0x92,0x5d,0xf3,0x22, +0xd8,0xbf,0x97,0x25,0xe6,0xdf,0xd8,0xd1,0x0a,0xcb,0x3e,0x7d, +0x4d,0x36,0x83,0xcf,0x98,0x16,0xf2,0xc7,0x70,0x85,0x4d,0x19, +0xf0,0xe5,0xf8,0x6a,0xe6,0x74,0xfd,0x0b,0xb7,0xeb,0x5b,0x57, +0x33,0xd0,0x26,0xcc,0xe7,0x5a,0x8d,0x27,0x1c,0x85,0xf6,0x39, +0x56,0x10,0x61,0x1a,0x9a,0x79,0xa1,0xde,0xfa,0xaa,0x55,0x0e, +0x53,0x70,0xef,0x03,0xf4,0x98,0xd9,0x40,0x1a,0xa3,0x42,0xe7, +0x7e,0x36,0x9f,0x2e,0xa0,0x6b,0x47,0x64,0x94,0xea,0x39,0x82, +0x88,0xf3,0x34,0x82,0x28,0x65,0x45,0x47,0x43,0x3c,0x3a,0x3b, +0xaa,0x3e,0x20,0xfe,0x19,0x96,0x33,0xb5,0x65,0xdc,0xab,0xe4, +0xfc,0x62,0x87,0x5d,0xe7,0xfb,0x0f,0x33,0x9b,0x3c,0x6e,0x5a, +0x0f,0xd3,0xd5,0x85,0xeb,0x1e,0xb2,0xa7,0xe8,0xf9,0x8d,0xc2, +0xa7,0xd1,0xec,0x23,0x7d,0x68,0x04,0x1f,0x55,0x1c,0x21,0x15, +0x3e,0xa0,0x32,0x15,0x3e,0xef,0x6d,0x5a,0x1e,0x19,0x49,0xb7, +0xbb,0xa5,0x0e,0x40,0x3b,0x05,0xff,0xcc,0xbc,0x41,0x7e,0xd3, +0xad,0xd5,0xda,0x6f,0xb6,0x6d,0xf9,0x96,0xc7,0x5b,0x28,0x05, +0xb2,0xd1,0xde,0x95,0x2f,0x2f,0x81,0xda,0x56,0xd7,0xd8,0xf9, +0x55,0xac,0xc1,0x35,0xc6,0xff,0x5e,0xd6,0x50,0xdc,0x99,0x3f, +0x7d,0xff,0xd5,0x97,0xef,0xc1,0x18,0x53,0xa6,0x8c,0x2d,0x26, +0xb7,0x8c,0x29,0x3a,0xb7,0x72,0xea,0x5f,0x2c,0x92,0x59,0x6e, +0x5a,0x32,0xdb,0x5e,0x9e,0xb0,0x6d,0x35,0x88,0x26,0xf9,0xf3, +0xa5,0x9c,0xb7,0xa9,0x26,0xce,0xf4,0x8a,0xbb,0x34,0xca,0x65, +0xfc,0x9f,0xe1,0xf7,0x05,0x3c,0x8f,0x97,0x32,0x07,0x0f,0xf9, +0x2b,0x08,0x84,0x09,0xc7,0xf9,0xa4,0x90,0x86,0xc0,0x20,0x81, +0x21,0xb8,0x3f,0x9d,0x0c,0x5e,0xb1,0x15,0x93,0xb8,0xf6,0x69, +0x0c,0xd3,0xf2,0x07,0x2a,0xb0,0xa3,0x35,0xb2,0xbe,0xf7,0x6e, +0x2a,0x1f,0xff,0x96,0x39,0xca,0xd3,0xa2,0x98,0x93,0x38,0x5e, +0xb6,0xf3,0x2a,0xc0,0xda,0xe5,0xf1,0xe3,0xc1,0x01,0xff,0x4a, +0xbc,0xdc,0x07,0x74,0x2a,0x55,0x4d,0x02,0x6e,0x0f,0x72,0x23, +0x44,0xd0,0xc5,0xb2,0xbe,0x41,0xa2,0x23,0x4f,0xde,0x29,0xd1, +0xdf,0x3f,0x52,0xb4,0x1e,0x5a,0x0f,0x0b,0xbd,0xd1,0xc0,0xc4, +0xdc,0x6e,0x06,0x36,0x43,0x1a,0x5e,0x25,0xcd,0x82,0x48,0x47, +0xb0,0xc3,0x00,0x53,0xe2,0x61,0x81,0x9f,0x20,0xd1,0x87,0x28, +0x8e,0x45,0x11,0xde,0xf5,0x26,0xcb,0xb2,0xf4,0x75,0xba,0xca, +0x27,0xcf,0xe8,0xc8,0x8e,0xa8,0x76,0xee,0x2e,0x91,0x30,0xba, +0x25,0xe3,0x0b,0x45,0xcb,0x09,0x1d,0xb6,0xba,0x3c,0x47,0x15, +0xd6,0x76,0x36,0xb4,0x70,0x92,0x8b,0x18,0xfb,0x6e,0xd2,0xec, +0xb4,0xf4,0x4e,0x2d,0xbf,0xe5,0xf2,0x73,0x14,0x50,0xdc,0x63, +0x85,0xc3,0xef,0xea,0xa6,0x90,0x7d,0x14,0x3c,0xb4,0x73,0xe0, +0xce,0x55,0x23,0xed,0xd2,0x0c,0x79,0xfa,0xce,0x9f,0x91,0xb9, +0xa4,0x4e,0xcb,0x48,0xd5,0x9d,0xa1,0x3b,0xcc,0xf4,0x0f,0x9b, +0xc2,0x0a,0x6e,0x4e,0x8e,0x0a,0xfb,0x52,0x16,0x3d,0xca,0x58, +0x5d,0xa6,0x08,0x17,0xc5,0x4d,0x3d,0x49,0x49,0xdb,0xe5,0xb3, +0xdb,0xfe,0xa7,0x27,0xcf,0x74,0x32,0x62,0x72,0x46,0x2b,0x96, +0xd4,0x1e,0xfd,0x09,0x99,0x9f,0xeb,0xb5,0x22,0x7b,0x08,0x2c, +0xcf,0x73,0x1e,0x0b,0x55,0x9c,0x42,0x03,0xee,0xea,0x3e,0xca, +0xe4,0xf6,0xdb,0xe9,0x29,0xbd,0x2f,0x37,0xf6,0xe4,0x06,0x5d, +0xc0,0xf8,0x5c,0x91,0x28,0xf2,0x1a,0x53,0xbb,0x5e,0x39,0xd7, +0x44,0x09,0x83,0xfd,0xf6,0x7b,0xb1,0x8a,0x0d,0xd4,0xa7,0xf2, +0x2a,0x6c,0x9f,0xa2,0x36,0x75,0xb2,0xb8,0xba,0x49,0x2b,0x6d, +0xfc,0xd6,0x20,0x6d,0x32,0xbd,0x91,0x45,0xb6,0x35,0x7a,0xed, +0x83,0x06,0x0b,0xc2,0x4e,0xd6,0x15,0xd8,0x89,0xf9,0x53,0x0a, +0x46,0xc9,0x52,0x6f,0xb0,0x2b,0x05,0xc2,0xe9,0x16,0x7e,0x56, +0x14,0x12,0x5c,0xe1,0xe7,0x2f,0x01,0xad,0x43,0x1f,0x79,0x06, +0x63,0x9f,0xd9,0x94,0x06,0x40,0x7f,0x67,0xed,0x74,0x8c,0xd6, +0x71,0x8e,0xdd,0x82,0xbe,0xaf,0xed,0x7c,0x2b,0x8f,0x71,0x08, +0x85,0xea,0x53,0x73,0xec,0xe5,0x98,0x01,0x6e,0xa5,0x09,0x4d, +0x92,0x3b,0x7d,0xa2,0x24,0x0f,0x4f,0x00,0x25,0xbf,0x9d,0x93, +0x1a,0x4c,0xc7,0x76,0x16,0x52,0xb1,0x41,0x55,0xcb,0x19,0xc7, +0xc4,0xde,0xa8,0x79,0x6e,0x50,0x2a,0xf7,0x07,0x96,0xf2,0x10, +0xf9,0x4e,0xb1,0x75,0x5d,0x89,0xf9,0x7d,0x15,0x4a,0xe3,0x41, +0x9b,0x3b,0x8f,0x5b,0x5b,0x9c,0x05,0x29,0x7d,0xab,0xba,0x98, +0x9a,0x57,0x02,0x04,0xc9,0x0a,0x66,0xc7,0x24,0x67,0x7f,0x59, +0xb1,0xcf,0x3a,0x50,0xaf,0xc0,0xcd,0x31,0xa6,0x24,0xbe,0x5d, +0x95,0x5f,0x39,0xc7,0xb2,0x9d,0xbd,0x75,0x21,0x4e,0xef,0xb9, +0x53,0xc8,0xaa,0xb3,0x48,0xd3,0x79,0x7a,0x5f,0xc8,0xe5,0x41, +0x9e,0x24,0x62,0x47,0x4e,0x54,0x84,0x60,0xae,0xc3,0x3e,0x9f, +0x0c,0x72,0xc7,0x93,0xd3,0xab,0xfb,0x0e,0xfa,0x90,0xfd,0xc7, +0x1c,0x37,0x07,0xf9,0xc3,0x4d,0xd4,0x27,0xe8,0xfb,0xb6,0x30, +0x2c,0x98,0xd4,0xea,0x93,0x4b,0x00,0xc9,0x1d,0x0b,0xa3,0x46, +0xfc,0xc0,0x63,0x61,0xb2,0x56,0x28,0xbb,0x4e,0x7d,0xfe,0xfc, +0xd6,0x1c,0xff,0x5b,0x87,0xb8,0x76,0x13,0x51,0x99,0xe0,0x5e, +0x4c,0xb5,0xad,0xd3,0x68,0xe0,0x59,0x60,0x99,0x26,0x40,0xb8, +0x83,0x03,0xfc,0xfb,0x22,0x62,0x6d,0x80,0xcf,0xaf,0xd2,0xc1, +0x04,0x76,0xa5,0x9c,0x61,0x8f,0x33,0x77,0x1d,0x4a,0x3c,0xc2, +0x2f,0x9b,0xb9,0x3a,0xe7,0xfc,0x96,0xcd,0x12,0x2a,0x8e,0x29, +0xe6,0xf6,0xba,0x7a,0x08,0xdb,0x79,0xfc,0xac,0x14,0xcf,0xc3, +0xcd,0x73,0x6e,0xcd,0x0c,0x25,0x3e,0xd3,0x79,0x15,0xfb,0x9c, +0xe7,0x39,0x28,0xd9,0xc5,0x66,0x3c,0x56,0xbd,0x8e,0x11,0x02, +0xf5,0x8c,0x57,0xd9,0x71,0xb8,0x1a,0xb6,0x88,0x07,0x61,0x19, +0xc4,0xc3,0x71,0xe9,0x0d,0x11,0x07,0x55,0x22,0x32,0xcd,0x2e, +0x42,0x47,0x4e,0x52,0x45,0x77,0x1e,0x95,0x57,0x2f,0x98,0xb1, +0x54,0x19,0x1d,0x6c,0x34,0x75,0x9b,0x99,0x33,0x79,0x8b,0x45, +0x92,0x48,0x5d,0x35,0x33,0x12,0xc1,0x31,0xc2,0xa6,0x62,0x68, +0xc6,0x7b,0x92,0x4a,0x0e,0x4c,0x17,0xab,0x12,0x97,0x06,0x2f, +0xac,0xed,0x5f,0x55,0x51,0x92,0xbf,0x52,0xa1,0xaf,0x52,0x0a, +0x55,0x97,0x16,0x95,0xc3,0xda,0x1c,0x51,0x6a,0xf1,0xc7,0xeb, +0x72,0x94,0x4d,0xb5,0x69,0x83,0xe7,0xad,0xe5,0x32,0xab,0x52, +0x4c,0xd3,0xb7,0x0e,0xb1,0x7c,0x72,0x16,0x74,0x76,0xed,0x8d, +0xea,0xf5,0xd9,0x16,0x86,0x43,0xf1,0x5e,0x90,0xe7,0xcd,0x6f, +0xdd,0x6b,0x40,0xeb,0x82,0xb0,0x6a,0xa6,0x15,0x45,0xa8,0xa6, +0xf3,0x91,0x76,0x55,0x0d,0x62,0x47,0x9e,0xda,0x7c,0xef,0xda, +0x99,0xef,0x2c,0xcf,0x22,0xb0,0x43,0x3c,0x76,0x96,0x9e,0x49, +0xb3,0xd4,0x0a,0x01,0xd9,0x59,0x96,0xc4,0xbe,0x7a,0x44,0xec, +0x2c,0x4b,0x1d,0x5a,0xdd,0xd7,0xbb,0xcb,0x56,0xde,0xce,0x2a, +0x09,0x2d,0xdc,0xd0,0x3e,0x2c,0x76,0xce,0xa8,0x62,0xc7,0xc5, +0xab,0x2c,0x5a,0x98,0xe1,0x78,0x79,0x21,0xde,0x77,0xa2,0xcc, +0x27,0xfc,0x0e,0xb7,0xb1,0x92,0x42,0x5c,0x44,0x96,0x5f,0xcc, +0x16,0x16,0xce,0x3b,0x2e,0x69,0xab,0xee,0x61,0x7f,0xf3,0x35, +0x6c,0xbe,0x17,0xf7,0x39,0xd3,0x29,0xfd,0xfc,0xa3,0xec,0x20, +0x87,0x70,0xbf,0xb4,0x67,0x44,0x3a,0xe3,0xf0,0x77,0x9e,0x27, +0xe5,0xbf,0xe2,0x97,0x95,0x57,0x9e,0xa9,0x9a,0x98,0x6d,0x25, +0x6a,0xd3,0xbd,0x3a,0x4e,0x65,0x30,0x58,0x55,0xe2,0x25,0x07, +0x64,0x7b,0x39,0x7a,0x7a,0xaa,0xd7,0xdd,0x9c,0x91,0x2b,0x89, +0x56,0x2d,0x65,0x18,0x9e,0x9e,0xde,0x51,0x54,0x38,0xe5,0x9b, +0x04,0x3d,0x43,0x61,0x6c,0x38,0x80,0xc5,0xdf,0xeb,0xe1,0xc3, +0x20,0x1d,0xbe,0x9c,0x7d,0xf3,0x20,0x43,0x5a,0x61,0x0c,0x0e, +0x15,0x15,0xae,0x75,0x1a,0x6c,0x77,0x49,0xfa,0x0e,0x51,0xee, +0xfc,0xf6,0x0b,0xeb,0xd1,0x8e,0xc0,0x7c,0x56,0xc0,0x3c,0xb1, +0x7b,0x5d,0x67,0x85,0x74,0xd4,0xe4,0x62,0x54,0xc7,0xea,0x93, +0x43,0x10,0x8c,0xe5,0xc3,0xa2,0x6c,0x9b,0x50,0x71,0xd0,0xae, +0x8b,0xf8,0x5b,0xa3,0xf1,0x23,0x67,0xd5,0xc0,0x13,0x90,0xe4, +0x4e,0xf1,0xde,0x00,0x77,0xf2,0x1a,0xa5,0xa7,0xf4,0xa8,0xb2, +0xf8,0xe1,0x4b,0x4b,0x6d,0x6d,0x59,0xd0,0x10,0x25,0xc6,0x41, +0xee,0x0e,0x8e,0x4a,0xf4,0x6d,0xea,0x1c,0x93,0x86,0x19,0xed, +0x6b,0x2d,0x0a,0xf5,0xea,0xa3,0x94,0xd2,0xac,0x6b,0xdb,0xf4, +0x70,0xe9,0x22,0xb9,0x66,0xc1,0xf2,0xe1,0xf4,0x16,0x26,0x62, +0x34,0xdb,0x31,0xb6,0x39,0x25,0x56,0x51,0x7e,0x85,0x7d,0x4e, +0xa5,0x55,0xa1,0x18,0xc6,0x0b,0xa0,0x32,0xf9,0xa0,0xbb,0x35, +0xb6,0x3f,0xe3,0xbb,0x35,0x85,0x30,0x7a,0x11,0xb2,0xf0,0x4b, +0xc4,0x64,0xd4,0x87,0x70,0x15,0xc2,0x43,0xeb,0x3f,0x82,0x24, +0x13,0xc4,0x9e,0x7c,0xe9,0x1b,0x53,0xc2,0x1d,0x77,0x6b,0xde, +0xcc,0xbe,0xcb,0xed,0xc1,0x36,0x12,0xb6,0x78,0x75,0x32,0x0f, +0x84,0x01,0x34,0x6b,0x5e,0x77,0x24,0x3a,0xa0,0xea,0x99,0xdb, +0xa2,0x4a,0xc6,0x94,0x74,0x03,0x00,0x7f,0x41,0xfb,0xad,0x2c, +0x62,0xb3,0x73,0xef,0xdd,0x74,0xfe,0x90,0x25,0xa4,0x54,0xdb, +0x35,0x7a,0x9e,0xbf,0x49,0x96,0x35,0x41,0x8f,0x93,0x24,0x7e, +0x93,0xa8,0xe4,0x87,0x55,0x6d,0xeb,0xf6,0x92,0xc1,0x56,0xf2, +0x09,0x80,0xff,0xae,0x86,0x4d,0x1a,0xcb,0xb8,0x28,0xaa,0xbb, +0x49,0x3b,0x94,0x61,0x50,0x27,0x75,0xbe,0xf9,0xce,0x86,0x0c, +0x21,0xaf,0x47,0xcb,0xf8,0x96,0x4d,0xc5,0xa2,0xe7,0x8d,0xeb, +0xf8,0x32,0x56,0xd4,0x80,0xe0,0x25,0x17,0xaa,0xd1,0xcc,0x60, +0x50,0xbd,0x1c,0xc1,0x62,0x12,0x1b,0x8f,0x8d,0xb2,0xf7,0x43, +0x74,0xdb,0x6d,0x19,0xb9,0xa3,0x85,0x32,0xc2,0x7c,0x8b,0xc1, +0xc6,0x4d,0xbe,0xc6,0xf6,0x8a,0x2c,0x7f,0x3d,0x8b,0xcd,0x04, +0x7e,0x3d,0x78,0xb8,0x3d,0x39,0x09,0x2e,0x8e,0xa3,0xa3,0xbf, +0x24,0x83,0xff,0x9e,0xae,0xa4,0xaf,0x6e,0x73,0x99,0x3a,0x8b, +0xdf,0xa4,0xb7,0xb4,0xa3,0x42,0x2a,0x3f,0xbd,0x45,0xe9,0x47, +0x0f,0x2f,0x44,0xd6,0x6a,0xfe,0xe0,0xda,0xff,0x99,0x8a,0x65, +0xd6,0xee,0x29,0x53,0x95,0xcf,0x53,0xa9,0x10,0xfb,0x0a,0x4d, +0x57,0xeb,0x0a,0x95,0xe3,0x3a,0xa7,0xa4,0x54,0x79,0xed,0x80, +0x1c,0x88,0xaa,0xf8,0xc1,0x80,0xfa,0x33,0x64,0xcc,0x6d,0xa6, +0x23,0xa8,0x3a,0xbd,0x49,0x61,0x54,0x51,0xb6,0x95,0xc0,0xf1, +0xdc,0xb7,0x7e,0xbb,0xc7,0x5e,0x2f,0x94,0x3d,0x77,0x13,0x3b, +0xa7,0xf4,0x93,0x28,0xcc,0x50,0xd7,0x5f,0x90,0xc3,0xe2,0x08, +0x8c,0x9c,0x51,0x11,0x29,0xaa,0xf8,0xee,0x53,0x36,0x55,0xc8, +0x41,0x47,0x39,0xdf,0x24,0xd8,0x9e,0x44,0x31,0x31,0x1d,0xd9, +0x46,0x86,0x32,0x78,0xe6,0xf0,0x0e,0x3f,0xaf,0x1c,0x7d,0xcf, +0x5c,0x41,0xff,0x58,0x86,0xf8,0x5c,0xf9,0xc5,0x5c,0x4e,0x50, +0xac,0x23,0xdd,0x98,0xc3,0x8a,0xe3,0x32,0xd3,0x57,0x0e,0x17, +0x74,0x13,0x17,0x1a,0x78,0x61,0x33,0x0c,0x83,0x8a,0x6e,0xd1, +0x10,0x73,0xeb,0x70,0x80,0x63,0x5c,0xc2,0x0c,0xaa,0x2a,0x15, +0x38,0x5e,0x04,0x0c,0x87,0xb2,0xe0,0xd2,0xe6,0x7b,0x32,0x00, +0x1a,0xa9,0xeb,0xa3,0x28,0x4f,0x30,0xb5,0xf8,0x25,0x06,0x83, +0x5a,0x47,0x0b,0x74,0xa4,0x3e,0x5f,0xa6,0xb7,0xe9,0xec,0x9a, +0x7c,0xf4,0x6d,0x20,0x29,0x34,0x49,0x21,0xec,0x9d,0x80,0x0e, +0x60,0xcd,0x53,0x34,0x5a,0x3f,0x8d,0x36,0x42,0xe6,0x44,0x0b, +0xc6,0x73,0xed,0xbd,0x43,0x45,0xd4,0xae,0x31,0xdb,0xd7,0x6d, +0xc4,0x58,0xe7,0x78,0x75,0x88,0x70,0xd3,0x46,0x7c,0x2a,0x21, +0x84,0x1a,0xd6,0xed,0xd1,0x1a,0xea,0x69,0x8f,0x36,0x76,0x94, +0x4a,0x68,0x12,0x91,0xfc,0x88,0xdd,0x89,0xf4,0xd7,0x6b,0xcd, +0x5a,0x1d,0x91,0x4a,0x53,0x03,0x48,0x06,0x98,0x5c,0xaa,0xc0, +0xc7,0xef,0x60,0xa4,0xe1,0x92,0x50,0x12,0x8a,0xca,0x46,0xf6, +0xaa,0xf4,0x01,0x1b,0x45,0xf4,0xfa,0x3d,0x3b,0x78,0x2d,0xa1, +0x1b,0x10,0x63,0xc8,0x4c,0x64,0x97,0x8c,0x17,0x28,0x4b,0xd2, +0xe2,0x51,0x5d,0xc0,0x51,0x92,0x73,0x3e,0x4d,0x88,0x18,0x79, +0x73,0x7e,0xa3,0x3e,0x12,0xb0,0x2e,0x09,0xad,0x7a,0xa3,0x1c, +0x55,0xaa,0x1b,0x22,0x94,0x47,0x6a,0x87,0xbc,0x2e,0x96,0xe9, +0x5d,0xe6,0x28,0x13,0xd8,0xa8,0xd5,0x65,0xb2,0xe1,0x7f,0x65, +0xaf,0x32,0x1b,0x60,0xdb,0xfe,0xe8,0x4e,0xbe,0xac,0x89,0xfc, +0x22,0x83,0x49,0x5c,0x8a,0x1c,0xa6,0x2e,0x71,0x58,0x1c,0xb9, +0xfa,0xf9,0x61,0xb1,0x58,0x26,0x59,0xe6,0xf9,0x1d,0x26,0xaa, +0xeb,0xa9,0xa0,0x8b,0xf4,0xaa,0xce,0x8e,0x0a,0x5c,0x91,0x50, +0x3d,0xfe,0x6d,0xae,0x6d,0x01,0x94,0x69,0x1b,0x4b,0x02,0xfb, +0x7e,0x5b,0x95,0x08,0x04,0x80,0x14,0xfb,0x8f,0x86,0x09,0xf6, +0x20,0x9f,0xa4,0x93,0x29,0xbd,0x22,0xf5,0x8f,0xf2,0xa3,0x05, +0x12,0x0f,0x21,0xc5,0xd4,0x2a,0x25,0xb4,0x04,0xf8,0x92,0xb7, +0x6c,0x85,0xaf,0x15,0x4c,0x64,0x06,0x22,0x1a,0xe0,0xcd,0xe7, +0x0b,0xa1,0x66,0x12,0xc1,0x6a,0x09,0xf9,0x57,0x93,0xcf,0x0f, +0x8b,0x9a,0x6f,0x29,0x1f,0xab,0xb0,0x54,0x3e,0xc2,0x51,0x22, +0x82,0x35,0x97,0xc4,0xc0,0xca,0x03,0x49,0xd1,0xb6,0x99,0x50, +0xbc,0x29,0xc9,0xd2,0x23,0x7d,0x97,0xb3,0x5c,0x09,0xbb,0xa6, +0x6e,0x03,0xc6,0xd1,0xab,0xf5,0xd1,0x0c,0xb3,0x81,0x38,0x23, +0x70,0xae,0x96,0x13,0xd9,0xaf,0xff,0xc1,0xb7,0xb4,0xdf,0xbe, +0x14,0xbf,0xfe,0xaf,0xa8,0x75,0xf8,0xeb,0x7f,0x88,0xbb,0x78, +0x72,0x03,0xc9,0xbf,0xfe,0xc7,0x51,0x4b,0xfc,0xf2,0xb7,0x28, +0x4c,0x1a,0x67,0xf0,0xdb,0x8a,0x7e,0xf9,0xdb,0xe1,0x2f,0x7f, +0xc3,0x09,0x11,0xe3,0xe9,0xc3,0x6a,0x1d,0x21,0xc2,0x8b,0x80, +0xd6,0x94,0xc9,0xed,0x32,0x81,0x79,0x08,0x5e,0x8f,0x00,0xc3, +0xaf,0xff,0x93,0x91,0xbe,0xfa,0xbf,0xbf,0xfb,0xbe,0x05,0x6f, +0xad,0x08,0xff,0x9e,0x44,0x27,0x38,0xe7,0xd1,0x32,0x7c,0x31, +0x47,0x47,0xd9,0x3f,0xcd,0xe7,0xd6,0x51,0xfd,0x22,0x10,0x8b, +0xd0,0x52,0x66,0x16,0xa0,0xa7,0x9c,0x3e,0x47,0xab,0xc3,0x57, +0x14,0x38,0x55,0x1e,0xa9,0x0f,0xe7,0xd9,0xdd,0x32,0xc0,0x3e, +0xc7,0x27,0x6f,0x19,0xf8,0xe2,0x21,0x7a,0x0c,0x8e,0x3c,0xa8, +0xf4,0x70,0x14,0xfa,0x87,0x1e,0x43,0x1c,0xe2,0x16,0x23,0x9e, +0xdd,0x41,0xfa,0x61,0x56,0x07,0xb0,0x06,0x2e,0x4c,0xd2,0x19, +0x15,0x51,0x4c,0xef,0x3d,0xac,0x83,0xfa,0xc3,0x21,0x68,0xa9, +0x87,0x0d,0x3d,0x6c,0xc4,0x63,0x70,0xc8,0x08,0x8e,0x54,0x15, +0x0a,0x81,0x72,0x8e,0x4c,0x85,0x56,0xaa,0xd0,0x4a,0x16,0xa2, +0xe6,0x27,0xeb,0x05,0x81,0x03,0x2c,0xb9,0x1e,0x5f,0x07,0xd1, +0x22,0x40,0x59,0x07,0x48,0x7c,0x02,0xa5,0xf8,0x48,0x0f,0x2d, +0x48,0x5a,0x87,0xd1,0x22,0xe4,0x4c,0x7a,0xc2,0x4c,0x7a,0x68, +0xa1,0xea,0x8c,0x20,0xbf,0x01,0x08,0x50,0x4f,0x03,0x40,0x03, +0x30,0x88,0x51,0x0b,0x34,0x33,0x54,0x5c,0x1f,0x6d,0xb0,0xd6, +0x51,0x68,0x45,0xdf,0x1f,0xb5,0x7c,0x31,0x08,0x22,0xef,0x31, +0x3c,0x7c,0x0c,0x1b,0x40,0xd0,0x63,0x50,0x87,0x1e,0x38,0x84, +0xf4,0x23,0xaf,0x05,0x6f,0x87,0x92,0x41,0x62,0x10,0x3a,0x50, +0x0d,0x1b,0x2a,0xd4,0x50,0x4b,0x79,0x99,0x3f,0x99,0xdf,0x7a, +0xa6,0x96,0x41,0x70,0x38,0x08,0xea,0xa1,0xdf,0x18,0x00,0xdf, +0x97,0x61,0x29,0x48,0x78,0x38,0x08,0x09,0x04,0xa3,0x20,0x2e, +0xa3,0x65,0xd8,0x58,0x06,0xe2,0x55,0x04,0x3d,0xf8,0xf4,0xa4, +0xe1,0x1f,0xc3,0xa3,0x47,0xe8,0x87,0xa3,0x5f,0xff,0xa7,0x36, +0xac,0x35,0x9d,0xaf,0x43,0x43,0x83,0x28,0x1e,0x1f,0xbe,0x12, +0x56,0x96,0xbb,0xa6,0x78,0xeb,0x88,0x93,0x25,0x49,0x98,0xe1, +0xdd,0xfa,0xef,0x6e,0x59,0x05,0xca,0xf1,0x6d,0x46,0xba,0x54, +0x82,0xd0,0xc5,0x0a,0xd7,0x35,0x16,0xf9,0xcb,0x5d,0x92,0x4c, +0x40,0x2d,0xe2,0xb3,0x5c,0x78,0x3c,0x62,0x92,0x04,0xd7,0x2a, +0xc0,0xca,0x02,0xb5,0x08,0x33,0x0d,0xad,0x4c,0x46,0x83,0xc9, +0x70,0x92,0x0e,0xc7,0xb2,0xb8,0x7a,0x95,0x85,0x8d,0xa6,0xb0, +0x94,0x06,0xe4,0x99,0x89,0x11,0x0d,0x61,0xdd,0x95,0x43,0xcf, +0xa3,0x59,0xfb,0x4d,0x9a,0x3c,0x36,0xd7,0xfe,0x11,0xfd,0x8e, +0x85,0x47,0xf3,0x36,0xbd,0x6c,0x54,0xa2,0xb5,0x9a,0xa0,0x05, +0x98,0x37,0xd1,0x38,0x26,0x80,0xe2,0x90,0xa1,0xea,0x8c,0x49, +0x4c,0x00,0x81,0x93,0xb4,0xb1,0xca,0x93,0xc7,0xda,0xef,0xe7, +0xf8,0x21,0x12,0x83,0x48,0xab,0x8e,0x78,0xed,0x59,0xae,0x8c, +0x51,0x70,0xb5,0xb5,0x87,0x9d,0x0e,0x62,0x9c,0x39,0x11,0xd6, +0x55,0xd8,0x6a,0x40,0xb9,0x10,0x40,0xd6,0x24,0x52,0x14,0x0a, +0x26,0xa7,0xce,0x6b,0x13,0xa4,0x93,0x53,0x36,0x75,0x5e,0x9a, +0x20,0x99,0x06,0x11,0x70,0x9a,0xdc,0x77,0xf8,0xef,0xd6,0xe1, +0xc1,0xc1,0x3a,0xd4,0x6e,0xc2,0x03,0xe9,0x7f,0x38,0xb7,0xc2, +0xd5,0xba,0xd3,0x5b,0xe7,0x38,0xb8,0x65,0x48,0x28,0xc8,0xee, +0xca,0x7d,0x5f,0x6c,0x00,0xe5,0x46,0xa3,0xdc,0x54,0xa0,0xd4, +0x5f,0xf8,0x7a,0x9b,0x1c,0xff,0x25,0xca,0x8d,0x41,0xb9,0x75, +0x84,0x51,0xf6,0xb6,0x47,0xea,0xd8,0x7f,0xc7,0xdf,0xe0,0xc9, +0x79,0x4d,0xe7,0x3b,0xc1,0xbc,0x31,0xd5,0x14,0xd0,0x8d,0x17, +0x85,0xa2,0x35,0xf6,0x89,0xdc,0x96,0xd2,0xa1,0xf9,0xdd,0xee, +0xc9,0xde,0x56,0x3d,0x9c,0x47,0x4e,0xeb,0xb0,0x2a,0x82,0xe4, +0x42,0xc9,0x5a,0x68,0xea,0xa1,0x53,0xb6,0xfe,0x54,0x4b,0x91, +0x50,0xd8,0xfd,0xad,0x66,0x4f,0xb9,0x32,0xf0,0xa1,0xfb,0x85, +0xdd,0xae,0xeb,0xea,0x65,0xa1,0x1a,0x69,0x7a,0x35,0x83,0xd1, +0x7e,0xd9,0xe1,0x7d,0x6e,0xf0,0xb5,0xf5,0xaa,0x51,0x4e,0xcb, +0xbf,0x61,0xc1,0x98,0xe3,0xcb,0x35,0xaf,0x14,0xed,0x25,0xe2, +0x6f,0x5b,0x09,0xee,0xb7,0x62,0x33,0x5a,0xe6,0x8d,0xad,0x79, +0xa0,0x65,0x72,0xa1,0x36,0x89,0xb4,0xae,0xc8,0xb3,0x78,0xbf, +0x55,0x5c,0xf9,0x31,0x99,0x3c,0x46,0xb5,0x78,0xe1,0x88,0xed, +0x75,0x7e,0x7f,0xa5,0x72,0xed,0x70,0xef,0x09,0x51,0x96,0xc8, +0x2d,0x89,0xd9,0xae,0xea,0xed,0x52,0x2e,0xf8,0x02,0xc9,0x6e, +0x10,0x29,0xe1,0x65,0xa0,0xe2,0x79,0x03,0xc6,0x21,0xb7,0x76, +0x51,0x78,0xea,0xa0,0x58,0x90,0x05,0xe4,0xa5,0x4d,0xbd,0xf4, +0x6c,0xb8,0xbe,0x61,0x14,0x0e,0x49,0xbd,0x3b,0xb6,0xb4,0x9d, +0x69,0x83,0x8a,0x3f,0xcb,0x1b,0xc4,0xa8,0x62,0xc7,0x28,0xaf, +0xc9,0x64,0x62,0xfe,0xcb,0x74,0x8b,0x0e,0x09,0x81,0xa1,0x4e, +0x6d,0x82,0x28,0xd8,0xb1,0xcd,0x1b,0xc3,0x32,0x81,0x2e,0xaa, +0xcd,0x79,0xb0,0x72,0x9e,0xa8,0x98,0x62,0x7d,0xa8,0x7a,0x1a, +0x04,0x57,0x00,0xd3,0xa0,0x2c,0x0a,0xef,0x2e,0x77,0x80,0x2b, +0xbd,0xad,0xb5,0x84,0x04,0x88,0xb1,0xb7,0xa0,0xfd,0x8e,0x52, +0xf1,0x2d,0x39,0x09,0xf8,0x22,0xaf,0xa5,0xf5,0x31,0xc8,0xb7, +0x4b,0x5e,0x5b,0x27,0x37,0xf1,0xc3,0x84,0x42,0xd8,0xb8,0xe3, +0x57,0x53,0x10,0x01,0x3d,0x3a,0xa4,0xad,0x4b,0xf2,0x55,0x58, +0x42,0xde,0xcf,0xfa,0x85,0xb6,0x8a,0x3c,0x07,0xfc,0x6c,0xef, +0x4c,0x7f,0xa6,0x3d,0x69,0xca,0x0e,0x8d,0x02,0x7b,0xe9,0xb3, +0xdd,0x16,0x14,0x90,0xe5,0x8f,0x6d,0x82,0xcb,0x4a,0x31,0x09, +0x45,0xb5,0xf4,0xa9,0xf0,0xc2,0x0e,0x99,0xaa,0x13,0xa1,0x28, +0xf5,0x12,0x7e,0xd6,0xb0,0x08,0x35,0x99,0x69,0x1f,0x33,0x6c, +0x9e,0x86,0x36,0x53,0xa9,0x5f,0x26,0x81,0x3f,0x58,0x26,0xf1, +0xb8,0xc3,0xae,0xdc,0x26,0x41,0x34,0x09,0xf1,0x64,0x70,0x22, +0x39,0xa0,0x5a,0x03,0x4b,0x30,0xf3,0x48,0xab,0xc0,0x06,0x2d, +0x14,0xfd,0x43,0x9d,0x5c,0x77,0x21,0x42,0x82,0x08,0x6d,0x08, +0xd6,0xfb,0x61,0xa4,0x59,0x74,0x70,0x60,0xad,0x07,0x15,0xd4, +0x91,0xce,0xf6,0x91,0x2a,0x58,0x52,0x60,0x45,0x75,0xaa,0xd4, +0x87,0x05,0xbf,0x47,0x68,0xeb,0x54,0x05,0xbc,0xf7,0x91,0xe8, +0x1e,0x34,0x04,0x81,0x26,0x1a,0x68,0x42,0x40,0x13,0x0d,0xa4, +0x17,0x09,0x44,0xc2,0x21,0x8f,0x62,0x47,0x1a,0x70,0x67,0xe5, +0x48,0x15,0x76,0xcc,0xf3,0xea,0x1f,0x97,0xfa,0x5a,0x69,0xe6, +0x04,0xff,0x77,0x19,0xa1,0xd2,0x9d,0xc2,0xb3,0x03,0xd5,0x5c, +0xe7,0x55,0x68,0x1d,0x75,0xf3,0x4d,0x0e,0x32,0xcd,0x20,0xde, +0x3e,0xea,0xd5,0x1e,0x29,0x71,0x3d,0x3b,0xa9,0x2d,0x9f,0x9c, +0x9e,0x56,0xbb,0x97,0xa6,0x55,0xcb,0x45,0x67,0x5a,0xab,0x9a, +0xb0,0xb8,0x3b,0xc4,0xfb,0xcd,0x5b,0xa6,0x65,0xe4,0x7b,0x41, +0x8f,0xbb,0x40,0x9f,0x5f,0xec,0x75,0x86,0x26,0x74,0xd3,0x23, +0xb3,0xe3,0xa5,0x25,0xb1,0x75,0xb4,0xa1,0x78,0x11,0xe9,0x8d, +0x70,0x1e,0xa2,0x7a,0x7e,0x34,0xac,0x35,0xf3,0xa3,0xe6,0xad, +0x9c,0x20,0xed,0xcb,0x08,0x39,0xe4,0x5d,0x76,0xbb,0xbd,0x62, +0xb9,0xa0,0xdb,0xf7,0x1c,0x64,0x9f,0x19,0x54,0xcf,0x2d,0xbb, +0xe7,0x52,0x7b,0x9f,0xa1,0xbc,0x6c,0xe5,0x7b,0x2b,0x67,0x95, +0x63,0x8a,0x60,0x88,0xb1,0x65,0x77,0x38,0xa1,0xb8,0x3e,0x53, +0x3c,0xc6,0xf7,0x72,0x99,0x7e,0xbb,0xe2,0x56,0xcc,0xdc,0x9a, +0x96,0x12,0xe8,0x8b,0x1c,0xa2,0x28,0xe3,0x88,0x66,0x58,0x87, +0xb5,0xe9,0xca,0x43,0xd1,0xc8,0x2f,0xca,0xa5,0x38,0x0d,0xaa, +0x67,0x95,0x8e,0x39,0x87,0x1e,0xd2,0x65,0xc5,0xd3,0x93,0x59, +0xd8,0x20,0x81,0x1d,0xad,0x4a,0x82,0xa7,0x27,0xcf,0xbc,0x98, +0xf9,0x9d,0x4a,0xfb,0xbe,0xd6,0x4c,0xa4,0x0a,0x17,0x30,0x8f, +0xb6,0x44,0x33,0x08,0x5a,0x87,0xf9,0xbd,0xde,0x1f,0x93,0xc9, +0x2a,0xf6,0x7c,0xbf,0x7c,0x0a,0x44,0x64,0x26,0x65,0x87,0xde, +0xca,0x2d,0x46,0xf1,0x20,0xdb,0x10,0x47,0x86,0x30,0xe6,0x98, +0x57,0x6f,0x08,0xf7,0xeb,0x65,0x0a,0x43,0x6f,0x33,0xc1,0x5e, +0xf7,0x2d,0x7c,0x31,0x36,0xdb,0x6f,0xd9,0x86,0x23,0x4a,0xf8, +0xf2,0xeb,0x56,0xa7,0xa4,0x3f,0xcb,0x38,0xa3,0x47,0x74,0x76, +0x97,0xde,0x90,0x4b,0x26,0xca,0xa4,0x58,0x39,0x63,0xbf,0x11, +0xb6,0x2d,0xf7,0xc6,0x63,0x9f,0x3e,0x62,0x2d,0xac,0x14,0x5c, +0xce,0x54,0x2d,0xa1,0x35,0x53,0x04,0x77,0x70,0x5e,0xb8,0x84, +0x9e,0x3a,0xc4,0x3a,0x10,0xeb,0x50,0x6c,0x02,0xd8,0x94,0xd1, +0xde,0x25,0x7a,0xb7,0x6e,0xc3,0x0b,0xfc,0x1b,0xb7,0xc3,0xad, +0x20,0x83,0x93,0xde,0xe5,0x59,0x00,0x12,0x15,0xc8,0x99,0xe8, +0x73,0x1d,0x6d,0xc7,0xe9,0xe3,0x97,0xec,0x21,0x72,0x63,0x1d, +0xf1,0x59,0xd7,0x0b,0xd6,0x2e,0x9f,0xd5,0x95,0x7e,0xb4,0xb3, +0x1e,0x16,0x91,0x3a,0xef,0x93,0xc9,0xd6,0x99,0x9f,0x7d,0x1b, +0xc1,0x99,0x15,0x47,0x92,0x98,0x29,0xac,0x9d,0x9e,0x90,0x5b, +0x37,0xbd,0xd7,0xd2,0x0b,0x62,0x4c,0x90,0x3e,0x42,0xf4,0x20, +0xa3,0x93,0x8c,0xb2,0x2f,0x46,0x9e,0x93,0x20,0x64,0x5f,0x48, +0x8b,0xeb,0x4e,0xb9,0x15,0x43,0xd7,0xd2,0x93,0x28,0x62,0x96, +0x6d,0x04,0x6c,0x7e,0xf9,0x7b,0x29,0xbb,0x79,0x76,0xfd,0xd8, +0x35,0x74,0x6b,0x73,0x7d,0x8d,0xee,0x7f,0x56,0xd7,0xd7,0x4f, +0x4f,0x6e,0x57,0x95,0xa9,0x3d,0xa8,0xe2,0x31,0x49,0x66,0x1e, +0xb5,0xbc,0x5d,0xc4,0x8a,0xab,0xa4,0x75,0x84,0x9d,0x2c,0xd7, +0x82,0xf4,0x08,0x6b,0xc4,0xe1,0x1a,0x96,0x80,0xb0,0x16,0x19, +0x6e,0xa2,0xd1,0x06,0x7e,0x29,0xc8,0x5b,0xee,0x6c,0xd1,0xeb, +0x79,0xc3,0xfc,0x6e,0x5f,0x78,0xc3,0xfc,0x6e,0x1d,0xd6,0x9b, +0xea,0xdc,0x44,0xe8,0x12,0xa1,0x2a,0x12,0x5a,0x65,0x42,0x55, +0x28,0x34,0xa5,0xe0,0xd1,0x78,0x02,0xb2,0x77,0x28,0x14,0x27, +0x30,0x4a,0xe1,0x11,0x7d,0x8f,0xae,0x8f,0x26,0xbd,0x56,0xbf, +0xe3,0x72,0x28,0x52,0x02,0x0d,0x95,0xd2,0xc1,0xcc,0x18,0xb8, +0x05,0x75,0xd1,0x89,0xcc,0x18,0xb8,0x36,0xde,0xe6,0x55,0xcd, +0x56,0xf5,0x83,0x9a,0x25,0x1d,0x7e,0x15,0x87,0x1b,0xe8,0xfb, +0x92,0x3a,0xcb,0xfa,0xe2,0xd9,0xa1,0x4b,0x96,0x92,0x24,0x90, +0x7a,0xfc,0x1a,0xa1,0xbc,0xde,0x71,0x36,0xad,0x5a,0x59,0xbf, +0xe6,0x9b,0xa1,0x3a,0x9e,0xee,0x43,0xeb,0xe4,0x61,0x89,0x30, +0xc7,0x19,0x88,0xdd,0x2f,0x1c,0x55,0x70,0xa5,0x04,0xf3,0x7e, +0x15,0x4a,0x3c,0x1b,0x7e,0xd8,0x40,0x95,0xf5,0xeb,0x62,0x75, +0x8a,0x0c,0x53,0xcb,0xe7,0xb9,0xd8,0x07,0xbb,0xea,0xb2,0x0b, +0xf0,0x45,0xce,0x75,0xb7,0x4a,0xf3,0xb4,0x7b,0xcc,0x02,0x62, +0x40,0x5f,0x56,0x6f,0x21,0x90,0x34,0xb0,0x4e,0xdc,0xaf,0x7a, +0x09,0x7b,0x7d,0x70,0x50,0x86,0x9b,0x73,0x55,0xd3,0x1c,0x63, +0xcb,0x9d,0x6d,0x42,0xc8,0x0a,0x94,0x98,0x27,0x11,0x5a,0x71, +0x35,0xde,0xee,0xc0,0xb6,0x0e,0xa3,0xb7,0xa0,0xc6,0xa3,0xb7, +0x2a,0xf8,0x56,0x89,0x1a,0x67,0xdc,0x6b,0xf9,0xd4,0xdc,0xec, +0x87,0x79,0x83,0x98,0x37,0xfb,0x60,0xde,0x84,0xf6,0x5d,0x08, +0xe9,0xdf,0xc2,0x5d,0x48,0xe9,0x9a,0x40,0x54,0xf5,0x25,0x5e, +0x4a,0x85,0x47,0x1c,0xde,0xac,0x78,0x6c,0x1c,0x01,0x5e,0x9a, +0xce,0x6a,0xae,0x45,0x44,0xb7,0x70,0xca,0x4c,0xb5,0x14,0xad, +0x98,0x1a,0x7a,0x32,0x1e,0x21,0xc0,0x8f,0x87,0x9e,0x9b,0xf0, +0x15,0x1a,0x23,0x85,0xad,0xa0,0x1d,0xc2,0xa0,0xac,0x71,0x55, +0x7e,0x1b,0x6a,0x35,0x33,0xe9,0x87,0x56,0x6d,0x6e,0xc7,0x09, +0x0d,0x41,0x6d,0xe5,0x35,0x97,0xaa,0xe8,0x43,0x9a,0xb1,0x8a, +0xd3,0xc9,0x96,0x3c,0x9f,0x7d,0x45,0xa6,0x05,0xe9,0x3a,0x99, +0xbc,0x1a,0xa2,0xa3,0xc0,0x1a,0xad,0xa0,0x3f,0x9b,0x4f,0xe6, +0x96,0xe5,0xad,0x89,0x5c,0x5a,0x61,0xe2,0xb5,0xbc,0x1d,0x78, +0x7e,0xbd,0x56,0xa3,0x9e,0xbd,0xcb,0x2c,0x53,0xa9,0x3b,0x91, +0x09,0x7d,0xe4,0xfd,0x51,0x59,0xc8,0x89,0xee,0x5d,0xce,0x39, +0xf2,0x9f,0xb2,0x09,0x0e,0x5b,0x40,0xe3,0xdd,0x35,0xef,0x04, +0xec,0x9b,0xe1,0xdf,0xc4,0x47,0x03,0x3c,0xa8,0x47,0xc6,0xd4, +0xaf,0xd5,0xea,0x77,0x42,0xa6,0x00,0xa4,0xe0,0x02,0x04,0x84, +0x05,0x21,0xb3,0x9e,0x89,0xfa,0xc4,0xc8,0x13,0xa4,0xda,0xde, +0x8a,0xb0,0x16,0xcb,0xb4,0x58,0x7e,0x8b,0x48,0xed,0xee,0xe4, +0xa0,0x9b,0x03,0xf2,0xbe,0xef,0x78,0x12,0x35,0x5e,0x21,0x23, +0xbd,0x6a,0x6b,0x9e,0x17,0x9c,0x69,0x77,0x41,0xee,0x7d,0x49, +0x1b,0x2b,0xff,0x3b,0xf9,0x01,0x2c,0xff,0x4c,0x8e,0xc6,0x6c, +0x57,0xe7,0xd4,0x37,0x8a,0x97,0xe3,0xdf,0xb9,0xb6,0xf1,0x21, +0xd7,0x57,0x52,0x1b,0x70,0xb0,0xdc,0xf7,0x13,0x40,0x21,0x7b, +0x4b,0xe9,0x66,0xd3,0xb8,0xbb,0xa1,0xd3,0xd3,0xc3,0xf7,0xee, +0xe9,0x21,0xf7,0xf4,0x50,0xf5,0xf4,0x90,0x7b,0x3a,0x0f,0xf7, +0x65,0x3c,0x40,0xb8,0x49,0x3c,0x90,0xb0,0x13,0x80,0x8b,0xe1, +0xdf,0x80,0x3a,0x5c,0xa5,0x7b,0x77,0x91,0x94,0x08,0x48,0xe1, +0x37,0x92,0xcc,0x3b,0xdf,0x6f,0x2e,0x01,0xfa,0x96,0x4a,0xf8, +0xb9,0xd2,0x58,0x12,0xc5,0x65,0xe8,0x8a,0xcb,0x30,0x27,0x2e, +0xc3,0x9d,0xe2,0x32,0xdc,0x53,0x5c,0x64,0x7d,0x36,0x4b,0x87, +0xe6,0xba,0x26,0x0c,0x02,0xc9,0xdf,0xba,0x6c,0xd6,0x7f,0x3f, +0x2c,0xf8,0x52,0xa0,0x5e,0x96,0x7e,0x09,0x9c,0x7a,0xab,0xc5, +0x66,0x47,0xad,0xf1,0xda,0x53,0x75,0x36,0x3e,0xa0,0xce,0x6a, +0xe1,0x19,0x4e,0xa8,0x17,0xdc,0x3a,0xa5,0xf0,0xb0,0xbe,0x20, +0x11,0x02,0x18,0x2b,0xd8,0x9d,0xb0,0x9d,0xcc,0x96,0x8a,0xd0, +0xa4,0x52,0x34,0xbc,0x09,0x74,0xec,0x04,0x3a,0x76,0x82,0x1d, +0x9b,0x87,0x33,0xa2,0x66,0xc1,0x82,0xc0,0x36,0xef,0xb4,0x62, +0x21,0xa1,0x99,0x28,0xa1,0x99,0x90,0xd0,0x4c,0x40,0x68,0x26, +0x46,0xcc,0xbc,0xfa,0x44,0xd4,0x63,0x51,0x1f,0x18,0x41,0x21, +0x8e,0x45,0xe1,0x85,0x90,0xcf,0x3f,0x44,0xcd,0xcb,0xd3,0xe0, +0xe4,0x5c,0xbd,0xff,0x18,0x85,0xea,0xf1,0xa7,0x28,0x6c,0x06, +0x17,0x17,0x17,0xc7,0x32,0xc1,0x11,0x30,0x68,0xc8,0x0e,0x01, +0xb3,0xa1,0x9f,0x13,0x30,0xa4,0xf3,0x43,0x44,0x8a,0xe1,0x62, +0xfe,0x19,0x70,0x67,0x3b,0xf5,0xee,0x14,0x30,0x53,0xeb,0xfb, +0x88,0xd4,0xb3,0x75,0x56,0x0a,0x18,0xa2,0xd5,0xda,0x69,0x52, +0x82,0xc7,0x2d,0xba,0x14,0xb7,0xcf,0x89,0xd6,0x32,0x27,0x32, +0xdf,0xdd,0x92,0x68,0x61,0x25,0x4b,0x10,0x85,0x25,0x88,0xc2, +0x52,0x8a,0x82,0x33,0x0f,0x2d,0xe5,0x3c,0x24,0x8c,0xce,0x54, +0x40,0xde,0xbf,0xff,0xfb,0x52,0xfc,0xfb,0xbf,0xdf,0xc2,0x3f, +0x4b,0x64,0x20,0xc3,0xe9,0x7a,0xa8,0xa8,0xaa,0xeb,0xff,0x4b, +0x0e,0xbc,0xbc,0xef,0xf7,0x98,0x16,0xa8,0xee,0x25,0x6f,0x0b, +0x81,0x17,0xfc,0x00,0x2c,0xe1,0x07,0x0c,0xbe,0x76,0xac,0xbf, +0xf5,0x59,0x3e,0x3d,0xdd,0x3e,0x3d,0x0d,0xba,0xde,0xf2,0xe0, +0x20,0xbd,0xc2,0x4f,0x1e,0x97,0x51,0xea,0x8b,0x5b,0x7c,0x83, +0x3f,0xde,0x2d,0xbe,0x0d,0xf0,0x0d,0xe3,0x6b,0x0d,0xf0,0x4d, +0x36,0x57,0x8b,0x5d,0xeb,0xf4,0x14,0xda,0xec,0x2d,0x61,0x7e, +0xf3,0x45,0x3e,0xf5,0xb6,0x34,0x75,0x80,0xa9,0xbe,0xe6,0x5c, +0x8a,0x76,0xeb,0x2c,0x10,0x0e,0x03,0x7e,0xf3,0xe4,0xc8,0xdd, +0xe2,0xc9,0x19,0x71,0xe9,0x0b,0xf3,0x72,0x6b,0xbf,0x0c,0xfc, +0x92,0xda,0x9d,0xb5,0x8d,0x2d,0x8e,0x72,0x1d,0xe2,0x49,0x06, +0x4b,0xf6,0xda,0x62,0xed,0xe0,0xa9,0x5e,0x50,0xd5,0xfe,0xa5, +0x56,0x57,0xf8,0x92,0xb5,0xc4,0xe7,0x17,0x92,0x6e,0x8b,0x49, +0xae,0x84,0x5d,0x93,0x7b,0x36,0x6d,0x6e,0x1c,0x63,0x04,0xd4, +0x01,0x7a,0xf6,0x0f,0x4f,0xcf,0x2f,0x5b,0xc7,0xa0,0x84,0x30, +0xc8,0xd4,0xcf,0x0f,0xb0,0x8e,0x4c,0x57,0x90,0x7a,0x76,0x72, +0x72,0x72,0x7c,0x7e,0x2a,0xe2,0x9f,0x1f,0xe2,0xf6,0xd9,0xe9, +0xe9,0x31,0x3f,0x4e,0x63,0x20,0x33,0x69,0x5f,0x1c,0x5f,0x5c, +0x9c,0x9e,0x9d,0x88,0xf8,0xed,0xc3,0x92,0x51,0x9c,0x84,0x00, +0x3c,0x48,0xd2,0x5b,0x2c,0x1b,0x86,0x97,0xad,0xb3,0x40,0x0c, +0xd2,0xec,0x67,0xac,0xe1,0xec,0xfc,0x3c,0x68,0x9d,0x9c,0x88, +0xc1,0x24,0x1e,0x8e,0x61,0xf5,0x0f,0xbf,0xb3,0xe1,0x5d,0x32, +0x8a,0x27,0xd3,0xf9,0x6c,0x44,0xf9,0xad,0xe0,0x04,0x8a,0x23, +0x3d,0xd8,0xfd,0xf8,0x00,0xeb,0xd7,0x49,0xb2,0x6a,0x5f,0x06, +0xa7,0xa7,0xad,0xa0,0x25,0x06,0xcb,0xf9,0xe3,0xac,0x1d,0x06, +0x17,0xad,0x93,0xd6,0x31,0xa0,0x7a,0x58,0x4e,0x36,0x8f,0xf3, +0x39,0x94,0x3e,0x39,0xbd,0x3c,0x6b,0x1d,0x87,0x62,0x18,0xc3, +0x42,0x96,0x50,0x9c,0xb5,0xce,0xce,0x4e,0x5b,0x17,0x82,0xf6, +0xd5,0xcb,0x04,0x56,0xb4,0x44,0xf0,0xf1,0x69,0x0b,0x92,0xe6, +0x43,0x3a,0x85,0x68,0x87,0xc7,0xe7,0x17,0x97,0x27,0xe7,0x81, +0x18,0xce,0x97,0xf1,0x04,0x89,0x38,0x39,0x69,0x9d,0xb7,0xf0, +0x75,0x76,0x33,0x99,0x3f,0x26,0x4b,0xc6,0x75,0x7a,0x19,0x5e, +0x5e,0x84,0x94,0x9c,0xa5,0x93,0x31,0x51,0x7b,0x0a,0xd8,0xc4, +0x70,0x99,0x4e,0xb3,0x39,0xd0,0x04,0xe5,0x8e,0x51,0xaf,0x0e, +0x37,0xf1,0x4c,0xb2,0x0a,0xa5,0x92,0xb9,0x7b,0x7c,0x49,0x2f, +0x94,0x77,0x7c,0x7a,0xde,0x3a,0xa6,0xd7,0xdb,0xf9,0x64,0x94, +0xcc,0x96,0x48,0x7e,0x2b,0xb8,0x6c,0x5d,0x4a,0xa8,0xdb,0x65, +0xbc,0x69,0x87,0xf0,0xbf,0xcb,0x20,0x3c,0x97,0x29,0x49,0x82, +0xdf,0x91,0x9c,0x01,0x7e,0xf9,0x9e,0x83,0x18,0xdf,0xc5,0xe3, +0x14,0xd0,0x9c,0x1c,0x1f,0xb7,0x4e,0x19,0xcd,0x14,0xad,0xe7, +0x56,0x71,0xfb,0x32,0x0c,0x2e,0xcf,0x4e,0xb8,0xc6,0xf9,0x24, +0x7d,0x93,0x30,0xb6,0xd3,0xd3,0xcb,0xf3,0xcb,0x4b,0x06,0x9d, +0x73,0x90,0x69,0x6c,0xfd,0x39,0xf0,0x59,0xa6,0xa1,0x55,0x1c, +0xf0,0x3a,0x38,0x09,0x82,0xb0,0x45,0x69,0xcb,0x64,0x44,0xe8, +0x60,0x2a,0xa3,0xf7,0x8c,0xfa,0x0e,0x7a,0xfe,0x38,0xb8,0x38, +0x09,0xb9,0x5c,0x96,0xc4,0x5c,0x01,0x08,0xc3,0x25,0x70,0x8d, +0x13,0x91,0xd9,0xc4,0x8a,0x93,0xf3,0xe3,0x93,0xe3,0x93,0x73, +0x93,0x4a,0xad,0x45,0xce,0x9d,0x5c,0x9e,0xda,0xa9,0x89,0x9b, +0x0a,0xa3,0xe0,0xe7,0x87,0x79,0x0a,0x9d,0x78,0xda,0xba,0x3c, +0xe1,0x34,0x25,0x1c,0x67,0x97,0x97,0xa7,0xc8,0xbb,0x24,0x59, +0x2c,0xd0,0x85,0x21,0xb4,0x23,0x3c,0xbb,0xc4,0x4a,0x20,0x25, +0x1b,0x6f,0xb8,0xe2,0xcb,0xf0,0x14,0x26,0xdd,0x74,0x4a,0x15, +0x9e,0x5d,0x82,0x0c,0x9d,0x9d,0xf2,0x7b,0x62,0xbd,0xcf,0x47, +0xb7,0xb2,0xcf,0x5b,0x41,0x70,0x0c,0x2d,0x10,0x37,0x29,0xec, +0x4a,0x97,0x29,0xc8,0x6c,0x88,0x0c,0x0a,0x4f,0xce,0x04,0x48, +0x06,0x48,0x8b,0x1a,0x23,0x20,0x09,0x97,0xc0,0x34,0xfc,0x88, +0x29,0x5b,0xc9,0xae,0x6a,0x9d,0x1d,0x5f,0x9c,0xb4,0xc4,0x0d, +0x9e,0x37,0xa6,0x31,0x51,0x14,0x5e,0x82,0x48,0xdc,0xa2,0x13, +0xb7,0xc1,0x7c,0x39,0x47,0x81,0x01,0x59,0x83,0xf1,0x71,0x7b, +0x37,0xcf,0x56,0x0a,0xd7,0x71,0x78,0x06,0xa0,0x02,0x25,0x03, +0x0b,0xc1,0x0b,0x60,0xb6,0xe4,0xe4,0xe4,0xb8,0x75,0x19,0x62, +0x12,0x36,0x02,0x6a,0x08,0xb1,0x2b,0xb8,0xce,0xe3,0xd6,0xf9, +0xd9,0x05,0x3f,0x6f,0x92,0x09,0xc8,0x2e,0xd0,0x7b,0x12,0x1c, +0xc3,0xc8,0x11,0xd4,0x44,0x05,0x7d,0x37,0x9f,0x25,0x9b,0x51, +0xf2,0x28,0x07,0x2c,0x50,0x70,0x37,0x5f,0x29,0xbe,0x1d,0x5f, +0x9c,0x9f,0x04,0x18,0xb3,0x2b,0x8d,0x67,0xd8,0xdb,0xe1,0xf1, +0xc9,0xe9,0xc5,0x69,0xeb,0x84,0x92,0x6e,0xe7,0xc4,0xc5,0xe3, +0x63,0x80,0x78,0x33,0x5f,0x6e,0xa8,0xed,0x40,0x20,0xec,0xe4, +0x59,0xfc,0x4e,0xcf,0x2f,0x80,0xe4,0x40,0x4c,0x62,0xfc,0x32, +0x20,0x59,0x42,0x4a,0x78,0xdc,0x42,0xc9,0x50,0x29,0xc0,0xd9, +0xec,0x8e,0xca,0x1d,0x1f,0x03,0xbb,0x27,0xf1,0xe3,0x8c,0xa9, +0xbf,0x00,0x59,0xbe,0x3c,0x3f,0x13,0x93,0x04,0x24,0x0a,0x24, +0xef,0xe6,0x06,0x05,0x0b,0x79,0x0b,0x3a,0x46,0x4c,0x70,0x7e, +0xe3,0xa1,0x04,0x63,0x09,0x44,0xfc,0x84,0x93,0xe4,0xa8,0x3d, +0x3d,0x3f,0x03,0xb2,0xce,0x64,0x1a,0x0e,0xb2,0x10,0x98,0x0b, +0x12,0x7e,0xc9,0x49,0x9a,0x81,0x8a,0x31,0xa0,0xd7,0x2e,0x5a, +0x48,0x16,0xe5,0xd2,0x78,0x83,0xc1,0xdc,0x3a,0x86,0x81,0x29, +0x93,0x58,0x82,0x2f,0x2f,0x60,0xd0,0xe9,0xa4,0x3c,0x94,0x62, +0xda,0xe9,0xc5,0xc9,0x99,0xa4,0x51,0x8d,0x08,0x48,0x84,0xee, +0x68,0xc9,0x44,0x35,0x24,0x5a,0xe1,0x49,0xeb,0xe2,0x52,0x56, +0xab,0x04,0x13,0x12,0x82,0xe3,0x13,0x59,0x8b,0x19,0x12,0xe7, +0x17,0xc7,0xa0,0x79,0x8f,0x9d,0xe4,0x24,0x9f,0xbc,0x82,0xdd, +0xbe,0x64,0x0b,0x10,0x01,0x43,0x8b,0xd3,0x75,0x33,0xa1,0x7b, +0xc2,0x0b,0x4c,0x9c,0xa2,0x0e,0x6b,0x5d,0x04,0xf4,0x28,0xe5, +0x05,0x44,0x09,0xbb,0x12,0xa3,0xa7,0xcf,0x88,0x25,0xa7,0x67, +0xa0,0x08,0x95,0xda,0xd0,0x22,0x0b,0xca,0x7e,0x0e,0x4d,0x42, +0xdd,0x79,0x16,0x5c,0x08,0x0c,0x7f,0xf8,0x30,0xb5,0x66,0x01, +0x10,0x9a,0xf3,0xe3,0x56,0x4b,0x66,0xc8,0xa1,0x73,0x2a,0x5f, +0x95,0x16,0x69,0xb5,0x42,0x94,0x6c,0x99,0xba,0x78,0x58,0x2e, +0x26,0x09,0x0c,0x5c,0xd0,0xd1,0x30,0xe7,0x70,0xa2,0xe6,0xd2, +0xf1,0xe5,0xf9,0x05,0xc8,0x82,0x4a,0xd6,0xaa,0xe3,0x22,0xb8, +0x38,0x3f,0x07,0xee,0xc9,0xf4,0x05,0x4e,0x95,0x5c,0xe2,0xec, +0x24,0x04,0x89,0xe0,0x74,0xa3,0x28,0x4e,0x40,0x36,0x8f,0x03, +0x05,0xcf,0xca,0x82,0x65,0x1a,0x56,0xe2,0xe1,0xf9,0x31,0x06, +0xf6,0x9b,0x19,0xc1,0x02,0x06,0xc0,0xd0,0x82,0xc4,0xd9,0x0a, +0x3d,0x58,0x4d,0x71,0x06,0x6b,0x85,0x17,0xa7,0x80,0x20,0xcd, +0x56,0x9b,0xe5,0x3c,0x53,0x93,0x18,0x16,0x9d,0x0f,0x87,0x71, +0x96,0xce,0x64,0x4a,0xeb,0x52,0xcc,0xe2,0x37,0xf1,0xfd,0x5c, +0xeb,0x84,0xb3,0x8b,0x33,0x90,0x5b,0x48,0x04,0xa1,0x81,0x49, +0x08,0x04,0x10,0x7d,0x83,0x43,0x0e,0xa8,0xe2,0xd3,0x53,0x4c, +0x00,0x4d,0x4c,0x63,0xf2,0x18,0xa4,0x9e,0xde,0x46,0xcb,0x78, +0xd0,0x3e,0x0f,0x4e,0x2e,0xce,0x41,0x99,0x19,0x95,0x0c,0xaa, +0x0d,0x06,0x3c,0xbf,0x13,0xf9,0xa0,0x13,0x2e,0x8f,0x61,0x22, +0x55,0xbc,0x3d,0x39,0x86,0x01,0x00,0x5d,0xbf,0x88,0x27,0x89, +0xa5,0x2a,0x4e,0xcf,0x4e,0xcf,0xa1,0xa9,0x9c,0x4c,0x6c,0x02, +0x75,0xda,0x82,0xe1,0xc4,0x49,0x86,0x4f,0x20,0x3b,0xad,0x4b, +0xe8,0x0b,0x4a,0xb6,0xd8,0x74,0x72,0x7c,0x01,0xaa,0xe6,0x18, +0x92,0x17,0xf1,0x26,0x86,0x96,0x2d,0x78,0xe0,0x06,0xe7,0xe7, +0x62,0x81,0x87,0xdc,0x8b,0x87,0x9b,0x1b,0x6a,0x2b,0xfc,0xff, +0x18,0xc3,0xf8,0x3d,0xa0,0xbe,0x38,0xbb,0x00,0xb5,0x2f,0xd4, +0xd8,0x38,0x0b,0x03,0x90,0xa1,0xc5,0xe4,0x61,0x8a,0x73,0x74, +0xeb,0xe4,0xec,0x18,0x0a,0xcf,0x1f,0x47,0x52,0xc9,0x42,0xdd, +0x30,0x47,0xc0,0x48,0x94,0x22,0x81,0x52,0x76,0x0e,0x23,0x59, +0x36,0x14,0x84,0x06,0xc8,0x05,0xd6,0x6f,0xe4,0xcc,0xdf,0x82, +0xd9,0xf3,0x14,0x26,0x95,0xe5,0x7c,0x13,0xb3,0xe4,0xc3,0x88, +0x3a,0xc3,0x09,0x21,0x8b,0x47,0xa3,0x49,0xc2,0x60,0xd0,0x8f, +0x20,0xf7,0xe7,0x42,0x8f,0x46,0x50,0x73,0x30,0x70,0xe1,0x7d, +0x36,0x52,0x98,0xce,0x82,0x63,0x28,0x79,0x22,0x8c,0xd8,0x05, +0xa7,0x90,0x74,0x8e,0x09,0xd9,0x1d,0x0c,0x20,0x6a,0x2c,0xd0, +0x7b,0x21,0xb2,0x34,0x99,0xcd,0x60,0x44,0x00,0xc0,0xd9,0x39, +0x08,0x26,0xac,0x00,0xde,0xa0,0x72,0x03,0x25,0xdf,0x42,0xfd, +0xe0,0x8c,0x64,0x58,0x83,0x18,0x91,0x3d,0xbb,0x3c,0x0f,0x82, +0x33,0x99,0xc2,0xc3,0xfa,0x18,0x7a,0x0f,0x3a,0xcf,0x1a,0xd1, +0x2a,0x65,0x26,0x87,0xec,0xe9,0x25,0xf4,0x9a,0x23,0xde,0xa7, +0x27,0xc1,0xb9,0x30,0x83,0xfd,0xe4,0x0c,0x96,0x0b,0x17,0x68, +0x31,0x30,0x43,0x8d,0x04,0xc3,0x02,0x5f,0x12,0xd0,0x84,0xd0, +0xa4,0xcb,0x33,0x5a,0x60,0xae,0x80,0x99,0xa0,0x6d,0x40,0x9a, +0x60,0x91,0xb2,0x9a,0x4f,0xe3,0xd5,0x9c,0xf4,0xfb,0x39,0xcc, +0xde,0xc2,0x1a,0x23,0xad,0x53,0x10,0xf1,0x33,0x21,0xa7,0x52, +0x10,0x1a,0x98,0x74,0x2f,0xce,0xc4,0xe3,0x5d,0x12,0xaf,0x68, +0x0d,0x77,0x8c,0x2d,0x32,0x53,0xdd,0x39,0x4c,0x22,0xfc,0x9a, +0x4d,0xe7,0x63,0xb5,0xcc,0x03,0x51,0xb7,0x74,0xce,0x19,0x5e, +0xf7,0xf1,0xbb,0x12,0x3c,0xe8,0xfb,0xe0,0xfc,0x84,0x3f,0x59, +0xd3,0xeb,0xd0,0x9d,0x71,0xa8,0x1c,0x40,0x15,0xbf,0x8c,0x13, +0xbf,0xe6,0x40,0xb2,0xca,0x17,0x23,0xb9,0x2c,0x27,0x14,0xec, +0x33,0x43,0x3e,0x62,0xea,0xfa,0x8e,0x52,0xe0,0x87,0x42,0x0b, +0xc1,0x23,0x9b,0x3e,0xac,0x36,0x54,0x68,0x94,0xbd,0xb1,0x82, +0xba,0x24,0xa0,0x1b,0x53,0xba,0xec,0x04,0x1d,0xc9,0xb1,0x65, +0xcc,0x6d,0x70,0xf6,0xc6,0x83,0xc5,0x26,0xc8,0xdd,0xa3,0x30, +0x8e,0x26,0x4a,0x43,0xe8,0xa9,0xec,0x08,0x61,0xe1,0x1f,0x5b, +0x27,0xd1,0x22,0x9c,0xc9,0x41,0xaa,0x08,0x99,0xaa,0xc7,0x60, +0x54,0x9b,0x2d,0x80,0x68,0x62,0xd1,0x7d,0x6e,0x03,0x08,0x36, +0xc9,0x16,0xf3,0x19,0xec,0x3c,0xf9,0x86,0xc3,0xc3,0xb2,0xd7, +0x7e,0x57,0x25,0xb7,0x71,0x7b,0x31,0xfa,0x4e,0x01,0xa1,0x77, +0x6f,0x80,0xd8,0x0a,0x59,0x0d,0x82,0xfb,0x8e,0xe9,0x2e,0xc3, +0x61,0xa8,0x82,0x24,0x5b,0x99,0x1d,0x4d,0xf6,0xa6,0xc9,0x1b, +0x5c,0x99,0xa3,0xeb,0xfd,0x1e,0x5d,0xa0,0x58,0xf6,0x86,0x4e, +0x75,0x37,0xc5,0x6f,0xa2,0xde,0x0f,0xb3,0xb8,0xf1,0x2d,0x1b, +0x32,0x10,0x1a,0x90,0xe6,0xef,0x24,0xd9,0x26,0x9e,0xcd,0x23, +0x5b,0xff,0x52,0x2e,0xfb,0x17,0x6b,0xde,0xcf,0x53,0xab,0x5f, +0xfd,0x3c,0x0e,0x76,0x8d,0x8a,0xbe,0x2b,0x0d,0x9a,0xe4,0x0b, +0xca,0xe2,0xcf,0x04,0x29,0xab,0xfb,0x6f,0xb5,0x7f,0xab,0xe3, +0x93,0x0e,0xf0,0x70,0xf4,0xba,0x76,0x74,0x2b,0xfe,0xad,0x56, +0xfb,0x37,0xbf,0x0e,0x99,0x6d,0xcc,0xa4,0x7b,0x73,0x55,0x9a, +0xb6,0xeb,0xdf,0x25,0xb7,0x9f,0xaf,0x17,0xde,0xbf,0xf5,0xa0, +0xb8,0xa6,0xa1,0x5e,0x7b,0x3d,0xeb,0xe3,0x57,0x90,0x2a,0x81, +0xe3,0x93,0xa9,0xb7,0x5c,0x6c,0xb2,0x3c,0x77,0xac,0xaf,0x6f, +0x99,0x2d,0x74,0x89,0x39,0x2f,0xc0,0x01,0x73,0x32,0x09,0xa2, +0x39,0x0e,0xc2,0x98,0x92,0xc1,0xd6,0x5c,0x99,0x42,0xcd,0x39, +0xb1,0x21,0xcf,0x01,0x62,0xa2,0xfa,0x0b,0x55,0xa0,0x36,0xaa, +0x89,0x9a,0x84,0x7c,0x57,0xab,0x6b,0xfe,0x3a,0x7e,0x03,0x52, +0xcd,0xb8,0xff,0xf6,0xea,0x9b,0xaf,0x9b,0xfc,0x65,0xaf,0xf6, +0x2d,0x5d,0xaf,0xb5,0x3f,0x1a,0xf5,0x6a,0xf5,0xb4,0x5e,0xeb, +0xd7,0xb6,0xb2,0x3b,0x00,0x2b,0x64,0x6c,0x6b,0x7e,0x67,0x1e, +0xdd,0x74,0xf3,0xf4,0x29,0x41,0xf1,0x62,0xea,0x5d,0xdc,0xec, +0xb7,0x63,0xf2,0x00,0x63,0xb7,0xad,0xc8,0x07,0xcb,0xd4,0x75, +0x9c,0xc8,0x40,0xab,0x2f,0xaf,0xa2,0xaf,0x55,0x5b,0x3f,0xff, +0xe6,0x0b,0x34,0xd0,0x4c,0xe6,0x13,0x95,0x02,0x8f,0xd1,0x8b, +0x50,0x7c,0xfe,0xcd,0x97,0x1c,0x89,0x32,0x7a,0x49,0x1f,0x29, +0x9f,0xa0,0x09,0x27,0x76,0xb6,0xd5,0x13,0xf7,0xbe,0x1d,0xbd, +0xf6,0x1e,0x7d,0xac,0x7c,0xf2,0x35,0xf9,0x51,0x2e,0x05,0x4f, +0x7d,0xaa,0xfe,0xf8,0xe4,0x45,0x49,0x1e,0x5a,0x65,0xb0,0xd1, +0x61,0xbd,0x9e,0x6e,0x5f,0x46,0x69,0xbd,0x45,0xf5,0x0f,0x4b, +0x41,0x55,0xcf,0x86,0xc7,0x50,0xcf,0xb0,0xeb,0x11,0xd1,0x81, +0x08,0x83,0xb2,0x6a,0xeb,0x2d,0xff,0xe0,0xa0,0x5e,0x7f,0xe9, +0xb7,0x29,0x1f,0x63,0x9b,0x33,0xbc,0x2f,0x08,0xd6,0x38,0x38, +0xbf,0xaf,0x87,0xc0,0x59,0x23,0xcf,0x35,0x96,0xe7,0x7f,0x63, +0xff,0xdd,0x9d,0xaf,0xaf,0x5e,0xca,0xd0,0xc8,0x45,0xaa,0x5e, +0xd6,0xeb,0x78,0x5f,0xce,0xc1,0xce,0xa9,0x1a,0x9f,0x2b,0xe9, +0x28,0x6b,0x52,0xa6,0xd5,0xdf,0x45,0xea,0x4b,0xf4,0xbb,0x04, +0x94,0x8a,0x7a,0x5d,0xc5,0xde,0x84,0x82,0x43,0xe0,0x97,0x33, +0x2e,0x7c,0xfc,0xae,0x35,0x9d,0x3d,0x68,0x77,0xee,0xf9,0x66, +0x88,0x97,0x8d,0xb1,0xf6,0x52,0x90,0xcf,0x54,0xce,0xc8,0x31, +0xfc,0x31,0x10,0x83,0x3d,0x8d,0xa6,0x72,0x20,0x09,0xf8,0xb3, +0x44,0x31,0xea,0xf5,0xc5,0xd7,0x4c,0x9c,0x74,0x8a,0xf9,0x92, +0x6c,0x11,0x83,0x8e,0xb7,0x8a,0xa4,0x20,0xf9,0x40,0x14,0x0a, +0x8f,0x91,0x80,0x18,0x1d,0xd7,0xac,0x28,0xf9,0xcb,0x83,0x83, +0x95,0xca,0x8f,0xd9,0x71,0xc2,0x0a,0xcd,0xd9,0x64,0xd9,0x8e, +0xf7,0xe2,0xe6,0xe9,0x09,0x03,0x9e,0x7b,0xb1,0x98,0x01,0xe3, +0x30,0xc2,0x39,0x56,0xcc,0xa0,0xb1,0x26,0x1d,0xd3,0x58,0xc4, +0x59,0x31,0x45,0xf6,0xb0,0xc8,0x48,0x9a,0x38,0xf8,0x6b,0x9a, +0xb1,0xc3,0x3b,0x4c,0x46,0x63,0x4f,0xdf,0x1a,0xf9,0x5a,0x2f, +0x66,0x5c,0x8a,0xc4,0xea,0x26,0x4d,0x26,0xa3,0x57,0x4e,0xec, +0x2e,0x41,0x69,0x76,0xe0,0x29,0x22,0xa9,0x30,0x17,0x93,0x76, +0x55,0x8d,0xa6,0x32,0x68,0x51,0x89,0xa9,0x0a,0x29,0x39,0x68, +0xa4,0x17,0xff,0xe9,0x89,0xb1,0x72,0xcb,0x34,0x00,0x86,0xfd, +0x62,0x00,0x9c,0xa6,0x7b,0x12,0xe6,0x39,0x3d,0xdd,0x97,0xa1, +0x6d,0xa9,0x1d,0xae,0xd6,0xb1,0x55,0xbe,0x8d,0xcd,0xf8,0x73, +0xc3,0xca,0x34,0x84,0xa5,0xe6,0xa1,0x24,0xd7,0xdf,0xf7,0xb7, +0xc5,0x99,0xc1,0x57,0xda,0xe9,0xf5,0xac,0xe6,0xdb,0x5d,0xe1, +0xaa,0x1b,0xee,0x0e,0x9b,0x6d,0xa6,0x2d,0xdf,0xa1,0xc5,0x63, +0x0e,0x09,0x1d,0x39,0x0f,0x61,0x95,0xc1,0x8b,0x0d,0x54,0x7e, +0xa2,0x86,0x02,0x77,0x04,0x89,0xf4,0x31,0x7c,0x73,0x65,0xe5, +0xfe,0x5f,0x2a,0x77,0x15,0x0f,0x1a,0x59,0x42,0xbe,0xb9,0x92, +0x51,0x83,0xc3,0x7c,0xd5,0xf4,0xa7,0xf8,0x68,0x77,0xb5,0xbc, +0x86,0x79,0xf2,0x21,0xf9,0x53,0x12,0x93,0x9b,0x22,0x2b,0xe9, +0xfb,0x38,0x9d,0x98,0x24,0x32,0xb9,0x01,0x04,0x26,0x65,0xc5, +0x46,0x81,0x26,0x21,0x86,0xb6,0xf1,0xb7,0x1b,0xfc,0x7e,0xb3, +0x94,0x2e,0x7d,0xd9,0x8c,0xb3,0x2c,0x8a,0x0e,0xe7,0xe0,0xfc, +0x40,0xb3,0xf5,0xa7,0xb3,0x74,0x4a,0x86,0x6e,0x5f,0x60,0xd1, +0x9a,0xdf,0x07,0x71,0x28,0xba,0xe5,0xb2,0x0c,0x12,0x55,0xa2, +0x08,0xcf,0x7d,0x8a,0x00,0x41,0x35,0x17,0x7d,0x79,0xb1,0x33, +0x15,0x58,0xd0,0x26,0xb3,0x4a,0xd7,0x12,0xec,0x38,0x82,0x00, +0x31,0xf6,0xfb,0x31,0xbd,0x62,0x09,0xcb,0x38,0x9f,0x59,0x47, +0xa6,0xce,0x98,0x53,0x97,0x68,0xa9,0x52,0xed,0x85,0xac,0xad, +0x6b,0x25,0x57,0x2f,0x64,0xdd,0x86,0x5e,0xdd,0xe8,0x93,0x95, +0x6d,0xa7,0xc8,0xe5,0x6e,0x31,0x89,0x7c,0xc7,0x45,0x94,0xd8, +0x2e,0xf6,0x14,0x67,0x94,0xf4,0x57,0x3e,0x43,0xf5,0x1a,0x68, +0x8f,0x7c,0xbf,0x45,0x8e,0xf5,0x68,0x3e,0xd7,0x2f,0xe2,0xe0, +0xab,0x26,0xab,0x6b,0x4d,0x21,0x0e,0xa9,0xcc,0x91,0xbd,0x31, +0xa1,0x79,0x83,0xc7,0x33,0xf6,0xf1,0xb3,0x06,0x85,0xcd,0xfe, +0xd8,0xb3,0xb0,0x67,0x8f,0x49,0xb2,0xf0,0xcc,0xe1,0x32,0x8f, +0x81,0xeb,0x51,0x32,0x04,0x51,0x98,0x90,0x5b,0x82,0xa8,0xd6, +0xa4,0x60,0xac,0x32,0x6b,0x75,0x37,0x7f,0xc0,0x0d,0x58,0xf6, +0x2a,0x51,0x0e,0xee,0x70,0x38,0x18,0x00,0xf2,0x57,0x82,0xc7, +0xdf,0xbd,0x63,0x71,0xdc,0xb7,0x32,0x86,0x0f,0xcb,0x65,0x32, +0x1b,0x6e,0x58,0xfe,0xa2,0xda,0xbf,0x5a,0xa5,0xac,0x30,0x6e, +0x1b,0x34,0xd0,0x83,0x7f,0x31,0xfc,0xbb,0x81,0x7f,0x0b,0xf8, +0x37,0x83,0x7f,0xff,0xef,0xdf,0xf0,0x23,0x60,0xf8,0x07,0xff, +0x8d,0x31,0xb4,0x1b,0xfc,0xfb,0xaf,0xf0,0xef,0x7b,0xf8,0xf7, +0x2d,0xfc,0xfb,0x1c,0xfe,0xfd,0x04,0xff,0x7e,0xac,0xf5,0x9b, +0x1c,0x03,0xd2,0xc1,0x4d,0x21,0x62,0xed,0x84,0xbc,0x8f,0xe6, +0xc5,0x12,0x1a,0x9d,0xc1,0xbb,0x0a,0x4d,0x12,0x38,0x5e,0x44, +0x41,0x1a,0x83,0x2b,0xf5,0x44,0xbf,0x87,0x11,0xac,0xb9,0x4c, +0x31,0x1d,0x52,0x00,0xaf,0xa7,0xe6,0x0f,0x33,0x15,0x2e,0xc1, +0x30,0x40,0x83,0x16,0x6a,0xa4,0x80,0xd8,0x61,0xdd,0x32,0xfc, +0x0c,0x93,0x46,0xd8,0xaa,0x1b,0x5b,0x53,0xd2,0xb0,0xca,0xd4, +0x34,0x0c,0x10,0x5e,0xdf,0xc8,0x35,0x5a,0x27,0xd6,0x8d,0xcb, +0x89,0x38,0x3e,0xb4,0x10,0x01,0xd5,0x51,0xda,0x85,0x55,0x48, +0x1b,0x97,0x88,0x47,0xc7,0x78,0x0f,0x53,0xe4,0x7b,0xef,0xa2, +0x9e,0x1e,0x1d,0xb3,0xf7,0x08,0x22,0xde,0xfa,0x1c,0xda,0x0e, +0xe2,0xc5,0xa6,0xaa,0xdc,0xbc,0xf5,0xa1,0x37,0x33,0xd7,0x32, +0xb0,0xf9,0xc7,0x76,0x1c,0xcd,0xda,0x36,0x08,0x4b,0x65,0x7e, +0x3a,0xcc,0x16,0xd0,0x70,0xb4,0x5b,0x57,0x81,0xb5,0x31,0xe8, +0x9e,0xe1,0xd3,0x12,0x5d,0x93,0x26,0x43,0x0b,0x0c,0x26,0x3b, +0xd8,0x10,0x11,0x5c,0x2f,0x04,0xb5,0x54,0xfb,0xa8,0x26,0xe2, +0x49,0x7a,0x3b,0x93,0x69,0x2d,0x4c,0xbb,0xaa,0xc1,0x96,0x5e, +0x27,0x1d,0x63,0x12,0xa4,0xb0,0xb0,0x71,0xda,0x09,0xa7,0xbd, +0xb5,0xb0,0x9d,0xf6,0xc5,0x63,0x3a,0x5a,0xdd,0x45,0x75,0x7e, +0x3f,0xeb,0x8b,0xe1,0x7c,0x3a,0x8d,0x65,0xf6,0x79,0xdf,0xf4, +0x92,0x4c,0xba,0xe8,0x73,0xe0,0x3c,0x7e,0xbb,0x94,0xc6,0xb0, +0x30,0x3c,0xb3,0x87,0x1b,0x0a,0x5f,0x5c,0x13,0x32,0xf2,0x39, +0x2c,0x4c,0x3b,0xd9,0x63,0x8a,0xa6,0xa8,0xb6,0x94,0x18,0x84, +0x75,0xfd,0xe8,0x86,0x65,0x14,0x1e,0x91,0x08,0xc4,0x06,0x35, +0x58,0x77,0xe1,0xf3,0xc1,0x41,0x2d,0xc2,0x67,0x6a,0x35,0xae, +0xbb,0xb8,0x11,0xf4,0x07,0xa0,0x24,0x37,0x00,0x86,0xc9,0x07, +0x00,0x6a,0x56,0x23,0xb2,0x45,0x81,0x93,0x40,0x0a,0x4e,0x7c, +0x15,0xe4,0x0f,0xd4,0x66,0x96,0xc0,0xf8,0x6a,0x73,0xab,0x61, +0xa9,0x47,0x8d,0xab,0xdd,0xd6,0x3a,0xbc,0xbe,0xa5,0xfc,0x8f, +0x6b,0x6d,0xd9,0xca,0x20,0xd0,0xed,0xfc,0xb8,0x26,0x61,0x6f, +0x1c,0xd8,0xc5,0x4e,0xd8,0xa5,0x03,0x3b,0x80,0x7a,0xf1,0x77, +0x2e,0x7f,0xd7,0xf2,0xf7,0x87,0x5a,0xbb,0xf6,0x2f,0xd8,0x5c, +0xee,0x3e,0x8c,0xd7,0x24,0x95,0x46,0xc0,0x1e,0x7d,0x9a,0xab, +0xf9,0x97,0x78,0xad,0xf3,0x19,0x40,0xe3,0xbc,0x40,0xa5,0x86, +0xb2,0xf4,0xa8,0xd6,0xd6,0x3d,0x10,0x58,0xfd,0x17,0xd8,0x55, +0x67,0x8a,0xcc,0x46,0xa8,0x49,0xdb,0xda,0x95,0x76,0x55,0x95, +0x40,0xcb,0xbf,0x96,0xd1,0x52,0xa5,0xd9,0x7c,0x98,0x50,0xa5, +0x5b,0xa4,0xa7,0x27,0x5d,0x3b,0x5a,0xd0,0x4b,0xce,0x72,0x78, +0x8f,0x17,0x91,0x2d,0x14,0x35,0xe5,0x57,0xaa,0x6b,0xe8,0xd5, +0x03,0x3c,0xb4,0x86,0x77,0x68,0xab,0x8d,0xb6,0x57,0x4b,0x64, +0x39,0x10,0x96,0x1b,0xf9,0xe8,0x3b,0x42,0x66,0x5d,0xdc,0x1b, +0x2c,0x81,0xa3,0x7c,0x64,0x28,0x48,0x4b,0xc9,0xc3,0x6b,0x46, +0x7e,0xca,0x09,0x1f,0x7e,0x18,0x60,0x67,0xc9,0x4f,0x08,0x68, +0xd2,0x78,0xcb,0x82,0xf3,0x96,0x85,0x94,0x5e,0x0a,0x66,0xba, +0x26,0x78,0x88,0xec,0x97,0x83,0x03,0x4a,0xfa,0x38,0x94,0x2b, +0xdf,0x5a,0x8d,0x70,0xcd,0x92,0xdb,0x18,0x17,0x31,0x91,0xd4, +0xb4,0x4f,0x4f,0xb8,0xed,0x90,0x4a,0x37,0xb8,0x0a,0x8f,0x38, +0xba,0x83,0xd4,0xb3,0xbc,0xa0,0x12,0xb5,0x46,0x0d,0xcd,0x25, +0x6f,0x67,0xb8,0x97,0x09,0xae,0xa8,0x53,0xe5,0xa7,0x6f,0xac, +0xe7,0x73,0x7a,0xbf,0xa0,0x7c,0x3b,0x8c,0x4e,0x86,0x15,0x67, +0xa3,0x55,0xa6,0x58,0x89,0xaf,0xca,0xa2,0xee,0xe5,0xef,0xed, +0xe4,0x0c,0x40,0xd0,0x12,0x01,0x72,0xa6,0x0c,0x39,0xce,0x25, +0xec,0xf8,0x79,0x12,0x67,0xab,0x97,0x76,0xb0,0x4e,0xe9,0x8c, +0x2f,0xb8,0x4a,0x39,0x0a,0x77,0x3b,0x1f,0xa3,0x95,0x82,0x5b, +0xc5,0x37,0x68,0x14,0x80,0x40,0x20,0x8b,0xe5,0x93,0x74,0x3d, +0x5f,0x90,0xb6,0x9d,0x2f,0xec,0x5e,0x71,0x7c,0xff,0xd9,0x73, +0xb5,0x0a,0xdf,0xc0,0xb4,0xf2,0x72,0x4c,0x0a,0xbb,0x5c,0x9c, +0xd5,0x65,0x34,0x06,0xf9,0x46,0xf4,0xa8,0x17,0x8f,0x05,0xa0, +0x1b,0xb4,0x55,0xef,0xa9,0x4f,0xc7,0xc4,0x02,0xf6,0x0d,0xb8, +0x10,0x20,0xb5,0x73,0x25,0xcf,0xb9,0x8c,0xbb,0x6f,0x59,0x13, +0x2b,0x25,0x89,0x2d,0x94,0x6b,0x6f,0xa4,0xdb,0x6f,0x83,0x58, +0x28,0x03,0xfd,0x67,0x9a,0x20,0xab,0xaa,0xab,0xa6,0x08,0x45, +0x4c,0x5d,0xb6,0x84,0xcf,0x23,0x23,0xce,0xe7,0x16,0x08,0xaf, +0xf6,0x89,0x56,0xa9,0x5d,0x5d,0x80,0x00,0xeb,0x12,0x61,0x1b, +0x66,0x15,0x0d,0xa2,0x2a,0x71,0x41,0xdb,0xb5,0xbf,0x16,0x40, +0x9c,0x1e,0x94,0xdf,0x63,0x5e,0x45,0xa1,0x5f,0x2f,0xaf,0xc5, +0x02,0x97,0xbc,0xd3,0xcc,0xd4,0xfc,0xe5,0xba,0x14,0x09,0xf2, +0xd0,0xb4,0xce,0x02,0xba,0xcd,0xaf,0xdf,0x30,0x2a,0xa9,0xd7, +0x6d,0x7b,0xbd,0xbf,0xbe,0x43,0x87,0x97,0xbd,0x4f,0xae,0xa2, +0xbf,0xf6,0x31,0xd4,0x40,0xaf,0xfe,0xba,0xf1,0x11,0x25,0xfd, +0xeb,0xbf,0xe0,0x4f,0x00,0xff,0x5e,0x8f,0xea,0xf0,0x57,0xe0, +0x53,0xb3,0xd1,0xe5,0xb7,0x5e,0xdc,0x78,0xfb,0x31,0x00,0x1c, +0xa5,0x22,0xa7,0x15,0xb4,0xd5,0xc1,0xa0,0x5d,0xe2,0x31,0x65, +0xad,0xad,0x1f,0xbc,0x16,0xcc,0xfd,0xc3,0x32,0x18,0xce,0x6f, +0xde,0x2c,0xe7,0xd3,0xcf,0xe4,0x61,0x02,0x2d,0x14,0xe6,0xcf, +0x20,0xbc,0x00,0x98,0xf5,0x33,0x30,0xe1,0x19,0x00,0xfd,0xf0, +0x3c,0x10,0xbc,0xfc,0x79,0xb1,0x50,0xf3,0xc7,0x56,0xdc,0x5a, +0x45,0xc4,0xc2,0x29,0xf4,0xad,0x5e,0xb5,0x2d,0x00,0x30,0xa9, +0x06,0xfc,0x7c,0xbd,0x98,0xcf,0xf0,0x58,0x3a,0x9e,0x10,0xe8, +0x4d,0x35,0xe8,0x17,0xb0,0xe6,0x1a,0x11,0xd0,0xb2,0x14,0xc8, +0x5b,0x9b,0x55,0xe4,0xba,0x74,0x05,0x89,0xc0,0xbe,0xaf,0x51, +0x55,0x28,0xf9,0xd2,0x92,0x87,0x5e,0x58,0xc7,0xe5,0xe5,0xa9, +0x4f,0x38,0x64,0x10,0x68,0x67,0x3c,0x45,0xd6,0x11,0x3b,0x39, +0x82,0x29,0xac,0xee,0xe5,0x47,0x23,0x85,0xf4,0x2f,0x79,0x50, +0x17,0x33,0xd4,0x3e,0x2f,0x5f,0x53,0x7e,0x92,0x30,0x27,0x74, +0x52,0x67,0xf2,0x91,0xce,0x8a,0xfd,0xcc,0x06,0xe2,0xb6,0x04, +0x77,0x2f,0xe8,0x77,0xd2,0xab,0xe0,0xe0,0xe0,0xf6,0x2a,0xe8, +0xf8,0x2b,0x2b,0xbc,0xb6,0xad,0x11,0x1b,0xd1,0xad,0x48,0xeb, +0xb7,0xa0,0x1c,0x4a,0x71,0xdc,0x47,0x78,0xa0,0xe6,0x7f,0x5c, +0xd5,0x26,0x13,0xe0,0xbb,0x89,0x1f,0xd2,0xe1,0x29,0xb7,0x3a, +0x80,0xd8,0xb1,0x39,0x02,0xee,0x42,0x4f,0xde,0x26,0x73,0x8e, +0xad,0x7d,0x0d,0xe3,0xd7,0xf1,0x7b,0xfc,0x2e,0x6b,0xc3,0x92, +0x0b,0xfe,0x39,0x71,0xcf,0x37,0xb4,0x65,0x23,0xd8,0x57,0x0f, +0x53,0x6f,0xc3,0xc6,0x46,0x2b,0x8d,0xe0,0xfb,0x64,0xba,0xf0, +0x85,0x0d,0x62,0xe7,0x28,0x4b,0x5e,0xfe,0x51,0x61,0x79,0xd8, +0x0b,0xff,0xaa,0x1e,0x39,0xa0,0x2b,0x76,0xbe,0x9f,0xe5,0x52, +0xe9,0xf3,0x8d,0x64,0x55,0x08,0x95,0x94,0xb1,0x4d,0xd9,0x2a, +0x0a,0x64,0x3c,0xf4,0x6f,0x6e,0xda,0x15,0x76,0xe3,0x99,0xd1, +0x4a,0x1a,0xb1,0x3a,0xbf,0xa2,0x84,0x0e,0x33,0x06,0x8f,0x9b, +0x93,0xd8,0x72,0x29,0x23,0x83,0x1a,0x9b,0x50,0xd7,0x9c,0x40, +0x5e,0x0e,0x01,0xfe,0x9a,0xe1,0xbf,0xa1,0xc4,0xef,0xc9,0x70, +0xcb,0x0d,0xeb,0x28,0x23,0x27,0xd3,0xca,0xa5,0x5b,0x51,0xa6, +0x67,0x01,0xf5,0x0b,0x35,0xb6,0x9d,0x52,0xff,0x35,0x99,0x4f, +0x93,0xd5,0x72,0x53,0x00,0xd3,0xed,0x2b,0xab,0x21,0x7a,0xf7, +0x45,0x12,0xa3,0xcf,0x48,0x2b,0x9c,0x3b,0x27,0x58,0x2d,0x2b, +0xaf,0x47,0xc2,0x21,0x73,0x28,0xc1,0xaa,0x51,0x48,0xa4,0x9f, +0xcd,0x27,0xf2,0xd3,0xd2,0x42,0x30,0x68,0x83,0x5d,0x9f,0xfa, +0x71,0x99,0x2c,0x92,0x8d,0x56,0xef,0xca,0x01,0xad,0x7a,0xcf, +0x87,0x35,0xdf,0x45,0x1c,0x7e,0x9a,0x5f,0x46,0x20,0x09,0x79, +0xb1,0x18,0x73,0xe4,0xd5,0xe2,0x2e,0xb1,0x19,0x52,0xa0,0x58, +0x3d,0x35,0x33,0x82,0x44,0xcd,0x4c,0xcb,0x9b,0x1d,0x45,0x38, +0x41,0x35,0x6d,0x08,0xfb,0x1c,0x98,0x20,0xe3,0x15,0xb4,0x4e, +0x23,0x63,0xd7,0x37,0x32,0xf6,0x35,0x3a,0x41,0xe3,0xa7,0x50, +0x3f,0xb5,0xd0,0x1d,0xec,0x57,0xb0,0xa2,0x4d,0x9f,0xab,0x4d, +0x7f,0x0d,0x6f,0xea,0x29,0xab,0x5a,0x32,0xd6,0x4a,0xca,0xf1, +0x56,0x52,0x6d,0x01,0x90,0xab,0x83,0xf7,0xa3,0xf8,0xcb,0x74, +0x96,0xf0,0x94,0xb6,0x83,0x62,0xa7,0x3b,0xb0,0x84,0xb7,0x83, +0x55,0x22,0x50,0x8c,0xd8,0x0b,0xf7,0xef,0xc5,0x8d,0x22,0x8d, +0x15,0x8c,0x21,0xfa,0xbe,0x9d,0x4f,0x36,0xb7,0x3b,0x05,0xdf, +0xc1,0x27,0xc1,0x77,0x35,0xdb,0xf4,0xfe,0x73,0x98,0xff,0x21, +0x2d,0x56,0x14,0x56,0x34,0x1a,0xa8,0x53,0xc3,0xe8,0xbd,0x86, +0xbd,0x1c,0x9d,0xa9,0x21,0xd1,0xa4,0x68,0xdf,0xd3,0x3a,0x65, +0xaf,0xc1,0x6f,0xc0,0x5d,0x02,0xb7,0x4a,0x97,0xc7,0x00,0x9e, +0xd7,0xe4,0xb6,0x29,0x2d,0xe2,0x44,0x18,0x98,0xb2,0x22,0x5c, +0x9e,0x58,0x13,0x80,0x6a,0x88,0x05,0xe5,0x0b,0xb7,0x88,0xa3, +0x70,0x65,0x9a,0x0d,0xf2,0x1d,0x88,0x2b,0x62,0xb6,0x67,0x19, +0x3b,0x1f,0x66,0xdb,0x9c,0x06,0xd2,0xb2,0x22,0xb1,0xd5,0xa3, +0x93,0xc3,0x5f,0xff,0x63,0xcb,0x0e,0x55,0x29,0x22,0x30,0x3a, +0x24,0x9c,0xd0,0x60,0x88,0x97,0x6e,0xca,0xe7,0xb3,0x91,0x7e, +0x5f,0x70,0x17,0x32,0x50,0x39,0x76,0x49,0x5c,0x53,0x7e,0xb2, +0x6e,0xd3,0xd5,0xd4,0x15,0x44,0xf9,0x02,0x98,0xb8,0x55,0xe8, +0xb1,0xc6,0xdc,0x67,0xbb,0xd4,0x2c,0xfa,0xd0,0x3e,0x57,0x4f, +0x27,0xdf,0xb0,0xe0,0x0a,0x1f,0xbb,0xd8,0xbe,0x3a,0x3e,0xb5, +0xf1,0xcf,0xf3,0x54,0x34,0x65,0x53,0x9d,0x34,0xed,0xbd,0x16, +0x1b,0x6f,0x7a,0x7f,0x80,0x0b,0xd6,0xac,0xd4,0x39,0x1c,0x6b, +0xb4,0x5f,0xfe,0x2e,0x7e,0xfd,0x4f,0x90,0x07,0x34,0x82,0x92, +0x97,0x4c,0xf4,0x1c,0xf5,0x7e,0xf9,0x7b,0x10,0x41,0xee,0x2f, +0x7f,0x0f,0xe1,0xa7,0xef,0x03,0x58,0x70,0xf5,0xeb,0x7f,0xc2, +0x86,0x0f,0x1e,0x22,0x28,0x03,0x09,0xf0,0x1e,0x52,0x42,0x88, +0x09,0x96,0x0b,0x37,0x20,0xf0,0x5b,0x1b,0x3d,0xbb,0xbd,0x91, +0xf4,0x0e,0xf1,0xeb,0xdb,0x2c,0x8d,0x67,0x1e,0xd4,0x81,0x7c, +0x92,0x7e,0x15,0x01,0xd2,0x7a,0xe3,0xb0,0x4e,0x8b,0x40,0x05, +0xdd,0x80,0xd5,0xdc,0xa4,0x80,0xe1,0xb3,0xe5,0x3c,0xcb,0xc8, +0x81,0xa2,0x2f,0x92,0x9f,0x1f,0x70,0x79,0x07,0x6b,0xfd,0xa8, +0xc7,0xe0,0xa8,0x9d,0x1b,0xf2,0x11,0x54,0x36,0xfc,0x97,0xce, +0x6e,0x54,0x00,0xfe,0x72,0x54,0x06,0x89,0xe0,0x82,0x7e,0x27, +0x0f,0xf8,0x35,0xa5,0xa7,0x6f,0x13,0xcf,0x60,0xf3,0x4b,0x30, +0x93,0x60,0xa7,0x43,0xd8,0x7a,0x58,0x70,0x3c,0x5c,0x7e,0xf9, +0x3b,0xb0,0xb4,0xf1,0xcb,0xdf,0xaf,0x05,0xac,0xf2,0x7e,0xf9, +0xfb,0x55,0xd0,0x0d,0xdb,0x30,0xf8,0x7f,0xf9,0x7b,0x1a,0x19, +0x58,0xfc,0x2c,0x7a,0xa4,0x5d,0x4c,0x1e,0x66,0x64,0x41,0x3e, +0x05,0x94,0xc8,0x1e,0x3e,0x43,0x8a,0x07,0x99,0x07,0xe5,0xfd, +0xab,0xf0,0x22,0x40,0x6e,0xd9,0x10,0x7f,0x05,0xde,0xa7,0x57, +0xd9,0x21,0xd4,0x72,0x70,0x80,0x3f,0x57,0xf0,0xee,0x33,0x37, +0x7f,0xfd,0x4f,0xa7,0xa2,0xd0,0xae,0xa8,0x03,0x99,0x4e,0xbf, +0xa6,0xbe,0xf6,0x67,0x84,0x04,0xe2,0x9f,0xfa,0xf1,0x59,0xe0, +0x7f,0x0c,0x7f,0x1a,0xe8,0x07,0xf3,0x7d,0x6a,0x6d,0xec,0xa8, +0x16,0xc5,0x2b,0xd5,0xf2,0xa5,0xaa,0x7d,0x56,0xec,0x3a,0x76, +0xfd,0x5d,0xa8,0x18,0xaa,0xec,0x82,0x08,0x4f,0x12,0xa0,0x25, +0x10,0xc8,0x1e,0xfb,0x2d,0xc4,0xb3,0x38,0x16,0x6a,0xbf,0xad, +0x32,0x28,0xbd,0x0c,0x2c,0x20,0x30,0x78,0xbd,0x82,0x87,0xa0, +0x8b,0x29,0x80,0xde,0x64,0x09,0x6a,0x61,0x68,0x50,0x22,0x30, +0xa4,0x5c,0xff,0xbe,0x14,0x30,0x2b,0xec,0x11,0x4b,0xae,0x95, +0x00,0xea,0x1a,0xb2,0xdd,0x81,0x47,0xfa,0x02,0x06,0x3a,0x0d, +0x7d,0xa9,0x17,0xf4,0x80,0x74,0x41,0x41,0x89,0x78,0x72,0xec, +0x63,0x8c,0x1e,0xac,0x97,0x5f,0x42,0x7c,0x09,0x85,0x8d,0x82, +0x7d,0x81,0x2c,0xf2,0xde,0x3c,0x70,0x3d,0xe2,0x0c,0x75,0x6b, +0xd0,0x1a,0x39,0xef,0xc0,0x23,0xa9,0xbc,0x82,0xd8,0x76,0xe1, +0xa1,0xee,0xf1,0x20,0x00,0x81,0x6a,0x37,0x50,0xb4,0xda,0xf0, +0xce,0x4d,0x86,0xb2,0xd8,0x42,0xc0,0x0c,0xbf,0xbf,0xfe,0x67, +0xa7,0xa0,0xf2,0x64,0xb5,0x22,0xaf,0x72,0x5c,0x0a,0x15,0x4f, +0x4a,0x75,0xab,0x97,0x03,0x96,0x5c,0xb1,0x1a,0x76,0x7d,0x4d, +0x04,0x14,0x67,0x08,0x02,0x15,0x76,0xa3,0xa0,0x95,0xfe,0xd5, +0x2f,0x7f,0x93,0x5d,0xd7,0xa0,0x7e,0x86,0x66,0xc2,0x16,0xb7, +0x9a,0xcd,0x05,0xa6,0xba,0x62,0xa0,0x4e,0x20,0xe0,0xb9,0x81, +0x85,0xfd,0x4f,0x02,0x10,0xf3,0x10,0x07,0x21,0x4a,0xa6,0x29, +0xc6,0x51,0xa7,0x92,0xef,0x48,0x97,0x7b,0xf6,0xc7,0x7d,0x31, +0xba,0xd6,0x1a,0x38,0xde,0xb6,0x1f,0x53,0xd8,0x13,0xce,0x08, +0xd6,0x5b,0x33,0x3d,0xe6,0x02,0x5e,0x92,0xfa,0x49,0xa4,0xe8, +0xec,0x9a,0xa4,0xf5,0xc1,0xc1,0xda,0x64,0xb4,0xd7,0x9f,0xa8, +0xac,0xa7,0x27,0x95,0xf8,0xc9,0x9a,0xac,0xb6,0xb0,0x01,0x30, +0x6a,0xb1,0x11,0xf0,0x8b,0xfa,0xed,0x5a,0x68,0x66,0x42,0xab, +0x05,0xf3,0x8b,0x2b,0xcf,0xf8,0x87,0x65,0x2e,0x7a,0xc7,0xd3, +0x3d,0x0b,0x9d,0x99,0xec,0xf5,0x93,0x9e,0xee,0xe5,0x6f,0xe5, +0x74,0x6f,0x8b,0xb0,0xee,0x52,0x29,0xd8,0x66,0x76,0xd5,0x32, +0x62,0xe5,0xe0,0x0c,0x2b,0xc5,0x41,0x12,0x4a,0xeb,0x23,0x5b, +0xfe,0x4c,0x95,0xb8,0x2f,0x2a,0x5f,0x12,0x94,0x94,0x60,0xa9, +0x29,0x0e,0xae,0x3c,0x55,0xa6,0xb5,0x2e,0x55,0xaa,0xcd,0xc1, +0x55,0x71,0x91,0xd1,0xcd,0xc9,0x1d,0x76,0x00,0xbc,0xa2,0xae, +0xbc,0x0c,0x7c,0x1a,0x59,0x00,0x05,0x12,0xda,0xe5,0xa4,0x76, +0xe3,0x97,0xbf,0x5d,0x71,0xa2,0xd4,0xb1,0x8d,0xcb,0x60,0x87, +0xb0,0xc2,0xd2,0x22,0x7f,0x97,0x20,0xf7,0x9f,0xfe,0x3b,0xc4, +0x88,0xf5,0x36,0x88,0x06,0xc4,0x15,0x1e,0x29,0x5c,0x64,0x13, +0xe4,0xae,0x2e,0xd5,0xe6,0x9b,0x1a,0xc7,0x33,0xe3,0x2c,0x92, +0x0b,0x11,0x13,0xdd,0x62,0xa6,0x17,0x27,0x56,0x64,0x35,0x29, +0xe3,0x26,0x8c,0x0b,0x7e,0x52,0x17,0x8a,0x58,0x16,0xc7,0xd9, +0x1e,0x66,0x04,0x74,0x8e,0xd6,0x8b,0xfb,0xda,0xe7,0xd9,0x40, +0x65,0xa7,0x78,0xc5,0x68,0xc4,0x1f,0x07,0x86,0x88,0xfd,0xa7, +0x27,0x37,0x31,0xc4,0xc4,0xae,0xc7,0x83,0x11,0x47,0x90,0xc0, +0x34,0xa5,0xa4,0x29,0x21,0xc6,0x04,0xe0,0x1b,0xfe,0x46,0x94, +0x2b,0x38,0x77,0xa0,0x73,0xcb,0xc1,0x03,0x04,0x47,0x3b,0xa3, +0x36,0xd3,0x29,0x0d,0x97,0xa2,0x81,0x69,0x12,0x74,0x8a,0x18, +0x88,0x41,0x92,0xad,0x60,0x63,0x70,0x84,0xe6,0x53,0x12,0x54, +0x07,0xe2,0x43,0x07,0x6f,0xb1,0x4c,0xed,0xcd,0xb0,0x9d,0x51, +0xda,0x01,0x1c,0x82,0x1b,0x2b,0x33,0xa0,0xb1,0xa8,0x98,0x22, +0x45,0x47,0x88,0xed,0xc0,0xaa,0xaf,0x10,0x37,0x1d,0xe0,0x43, +0x15,0x23,0x9a,0x82,0x02,0xa2,0x8a,0xd6,0x7c,0x44,0xeb,0xd6, +0x56,0x08,0x19,0xb3,0x8f,0x3d,0x5b,0x01,0x45,0x51,0x84,0xd2, +0xf1,0xf4,0xc4,0x8f,0xd0,0xdd,0xdd,0x5e,0x2f,0x00,0x42,0x03, +0x74,0x0e,0xa1,0x9f,0xfa,0xed,0x5e,0x4f,0x2a,0x02,0x48,0x95, +0xaa,0xa0,0xdf,0xdf,0x6e,0x69,0xc1,0x4d,0xf2,0x80,0xce,0x3a, +0x96,0xf3,0x74,0x54,0xdc,0xa6,0xa8,0xb5,0x97,0x04,0xf8,0x4b, +0x10,0xe5,0x53,0xc2,0x7c,0xca,0x0f,0x05,0x98,0x1f,0x0b,0x29, +0x3f,0x15,0x52,0x7e,0x28,0xe0,0xf9,0xb1,0x90,0xf2,0x53,0xb1, +0xae,0x56,0xa1,0x54,0x21,0xe5,0xa7,0xd6,0x33,0xdb,0x2a,0x05, +0x28,0x2d,0x99,0x8b,0x75,0x88,0x4d,0xb1,0x16,0xf1,0xb6,0x58, +0x8f,0x98,0x46,0xeb,0xc3,0x75,0x7d,0x73,0xb8,0xa9,0xbf,0x3d, +0x7c,0xab,0x86,0xe9,0x2f,0x7f,0x6b,0x5d,0xe1,0xd0,0x2e,0x22, +0x0e,0x4b,0x10,0x87,0x25,0x88,0x41,0x71,0xff,0xed,0xaa,0xc0, +0xf7,0x52,0x94,0x41,0x09,0xca,0xa0,0x04,0x25,0x68,0x05,0x87, +0x58,0x41,0x54,0xfa,0x5d,0x2d,0x35,0xed,0x1e,0x4f,0xaa,0xab, +0x78,0xd6,0xf2,0x36,0x62,0xed,0x5b,0x4b,0x45,0x3a,0x65,0xcd, +0xd2,0x99,0xf7,0xf6,0xc8,0x38,0x7a,0x9c,0xfa,0x36,0x48,0xdf, +0xd9,0x97,0x1a,0xf1,0xc9,0xf3,0xfc,0x2f,0x61,0x3e,0xe5,0x87, +0x02,0xcc,0x8f,0x85,0x94,0x9f,0x0a,0x29,0x3f,0x14,0xf0,0xfc, +0x58,0x48,0xf9,0xa9,0x58,0x57,0xab,0x50,0xaa,0x90,0xf2,0x53, +0x21,0x45,0x6f,0x99,0xcd,0x4e,0x57,0xee,0x8c,0x6d,0xa8,0x6f, +0x73,0x33,0x67,0x2e,0xfb,0xcb,0xc2,0x44,0x5a,0x02,0xb0,0x6b, +0x5e,0xcd,0x81,0x17,0x77,0xaa,0x2a,0xe7,0xd9,0x3d,0xf3,0xde, +0x98,0x34,0xcd,0xe6,0x48,0x73,0x38,0x49,0x17,0x9f,0xda,0xdb, +0x22,0x2b,0x9d,0xac,0xb1,0x96,0x6c,0xee,0x53,0x06,0x8b,0xe8, +0xaa,0xf2,0x5e,0x1a,0x7f,0x5a,0xa2,0xd7,0xf8,0xf5,0x3f,0x44, +0x03,0xa3,0x02,0xf4,0x7d,0x1b,0x9e,0x3d,0x17,0x7d,0xf5,0xe9, +0x0f,0x51,0x98,0x5c,0xaa,0xfd,0xb6,0xc9,0xc8,0x47,0x77,0x5c, +0x07,0xe8,0xcb,0x0d,0x3d,0xba,0x85,0x82,0xf5,0x80,0x40,0x60, +0x61,0x95,0x78,0xc7,0xe9,0xd6,0x41,0xd2,0xc3,0x6a,0xf1,0x60, +0x4e,0x6b,0x38,0x1b,0xcd,0x1d,0xe8,0xa1,0xf9,0x06,0xb6,0xa1, +0x23,0x0c,0xe8,0x24,0x11,0x46,0xd4,0x6c,0x59,0x48,0xb8,0x40, +0x81,0x7c,0xdf,0x8a,0x84,0x2a,0x6b,0xef,0xe7,0x64,0x09,0x10, +0x46,0x85,0x36,0x7b,0xeb,0x20,0x22,0x07,0x49,0xe4,0xd5,0x4a, +0x3d,0xc3,0xbc,0xb2,0x0e,0x23,0x72,0x99,0x44,0xe9,0xea,0x19, +0x27,0xc5,0x2a,0xca,0x15,0xe1,0xec,0x0b,0xd4,0xd4,0xe0,0xc3, +0xa4,0x41,0x0c,0x83,0x29,0x83,0x58,0x86,0x13,0x86,0x8e,0x30, +0xaa,0xc1,0x9a,0xdc,0x16,0xaf,0xc7,0x71,0x32,0xb4,0x37,0x3c, +0x3c,0x8a,0xf5,0x54,0x8f,0xcc,0x67,0xe9,0xf0,0x73,0xd8,0xd6, +0x4f,0x3e,0x75,0x4e,0xc2,0xf2,0x67,0x60,0x04,0xe7,0xd9,0x2f, +0xba,0x10,0xda,0xb5,0x2e,0xe3,0xc7,0xa8,0x34,0x53,0xe9,0xf6, +0x78,0x32,0xc0,0x80,0x53,0xa5,0x4e,0x85,0x8a,0x74,0x78,0x80, +0x71,0xbe,0x42,0x4f,0xa9,0x40,0xb4,0xc0,0xc0,0xda,0xec,0xb1, +0xca,0xeb,0x35,0x9a,0x67,0xe2,0xf8,0xa2,0x79,0xce,0x31,0x87, +0xe3,0xc9,0x04,0xf6,0x41,0x5e,0xaf,0x75,0xd9,0x3c,0x15,0x27, +0xa7,0xcd,0x53,0x48,0x66,0x7b,0x88,0x30,0x38,0x0f,0xd8,0x96, +0xcb,0xd4,0xfe,0xe7,0x2c,0x2e,0x3d,0xea,0xd1,0xb9,0xf6,0x81, +0xa6,0x14,0x4b,0xe7,0xc4,0x1b,0xfb,0xcd,0x49,0x08,0xfb,0x4e, +0xc8,0x23,0x9e,0xfb,0xe9,0xf3,0xed,0x93,0x0b,0xde,0x1d,0xad, +0xc5,0xc6,0x67,0x15,0x84,0x16,0xd9,0x93,0x38,0x1b,0xc7,0xf9, +0x0c,0x58,0x59,0xdd,0xc5,0x8f,0x71,0x9a,0xe6,0x33,0xb6,0xda, +0xdf,0xa5,0x83,0x53,0x58,0x68,0x84,0x55,0x52,0x01,0x45,0x4e, +0xa3,0x61,0x1d,0xc1,0xf0,0xd1,0x73,0x9c,0xc6,0x6f,0x19,0x1d, +0x56,0xb7,0xc4,0xe9,0x05,0xb1,0xd4,0xe2,0xf4,0xe9,0xa9,0x38, +0x83,0x24,0x59,0xef,0x1e,0x38,0xcf,0x5d,0x9c,0xc7,0x22,0xbc, +0x6c,0x5e,0xba,0x38,0x2f,0x44,0x78,0xd1,0x97,0x6d,0x7e,0xc5, +0x12,0x2f,0x37,0x38,0xd6,0x55,0xf3,0xc6,0xe7,0xb4,0xa8,0x07, +0xcf,0x96,0xb8,0xeb,0xce,0x93,0x3e,0xf5,0xed,0xa0,0x59,0xb9, +0xce,0x1c,0x47,0x92,0x43,0x4d,0xe5,0xe8,0x6d,0xa5,0x53,0xb4, +0xeb,0x3a,0x48,0x5d,0x47,0x9e,0xdb,0xeb,0x8d,0x15,0x79,0x38, +0x1e,0x43,0xef,0x7b,0x6e,0xf7,0x37,0x56,0xe4,0xeb,0x78,0xdc, +0x51,0x1e,0xff,0xaf,0xa2,0x66,0xd8,0x3a,0x38,0x68,0xb6,0x8e, +0x4f,0xae,0x36,0xb0,0x15,0xbc,0x8a,0x1a,0xcd,0x93,0xd6,0xe9, +0xc1,0x41,0xa3,0xd9,0x0a,0x4f,0xae,0xd6,0x5d,0xee,0x8c,0x36, +0x01,0x9e,0x9d,0xe5,0x20,0x01,0x04,0x21,0xc3,0xf0,0x14,0x20, +0x99,0xc5,0x6d,0x49,0xa1,0x2f,0xdb,0xe7,0x34,0x6b,0x2b,0x4c, +0xf3,0xf3,0xb7,0x92,0xfc,0x2e,0xfd,0x0d,0x32,0x0e,0xc9,0x5c, +0xcd,0x05,0x5e,0x6f,0x49,0x40,0x29,0x26,0x12,0x86,0x5f,0xf2, +0x20,0x4c,0x92,0x04,0xe1,0x97,0x1c,0x88,0x64,0x44,0x69,0xff, +0x39,0x54,0xc8,0xf3,0x09,0x92,0x76,0xbb,0x66,0x3b,0xdd,0xae, +0xce,0x4a,0xdf,0x8a,0xe2,0xa9,0xb8,0x8b,0x5b,0xdd,0xc6,0xb9, +0x98,0x75,0xaa,0x83,0xd7,0xdc,0xdc,0x99,0xe5,0x40,0x25,0x62, +0xeb,0x54,0xc4,0xc5,0x6d,0x67,0x38,0xe8,0xed,0x73,0x14,0xbd, +0x9a,0xd8,0x89,0x9f,0xf7,0xbf,0x05,0xec,0x9c,0x5c,0xc0,0x4d, +0xc9,0xdb,0xca,0x85,0x48,0x9e,0xe7,0xf6,0x9e,0x3c,0xcf,0x77, +0x27,0x2f,0xc7,0xfb,0x7d,0xf6,0xf2,0xa5,0x75,0x95,0xb4,0xc6, +0xc9,0x29,0xab,0x87,0xdb,0xb4,0xb5,0xa5,0xdb,0x98,0x07,0xee, +0x35,0x33,0x2b,0x11,0x37,0x56,0x24,0xd7,0x8a,0x08,0x37,0x4d, +0x0a,0x71,0x0e,0x4e,0xd6,0xea,0xb7,0x8b,0x78,0xdc,0x31,0xb7, +0xbf,0xef,0x49,0x57,0xf5,0x18,0x6a,0xf8,0xbd,0x79,0x7c,0x7a, +0x68,0xa8,0xb1,0x60,0x54,0x4d,0x46,0x3f,0x95,0x68,0x2c,0xdf, +0x10,0x2a,0x35,0xdb,0xb6,0xac,0xa8,0x43,0x68,0x7a,0xe3,0xbd, +0xc8,0x53,0xa9,0x3e,0xeb,0x29,0xa9,0xa2,0x53,0xae,0x40,0xd7, +0x91,0x74,0xde,0xc9,0xab,0x99,0x8e,0x8b,0x80,0xcd,0xf1,0x8b, +0xd8,0xae,0x7d,0x6b,0x25,0x08,0xcb,0x93,0x35,0xe8,0xc8,0xd3, +0x53,0xf4,0x72,0x0a,0x2a,0xf0,0xf8,0xe2,0x10,0x1d,0xae,0xae, +0xeb,0x2a,0xad,0x2e,0xd3,0x70,0x6e,0x67,0x5d,0x63,0xcd,0x14, +0x3e,0xeb,0x05,0x7b,0x4e,0x54,0xba,0xcb,0xd4,0x87,0x15,0x1c, +0x07,0xe7,0x12,0x59,0x10,0x02,0xb2,0x12,0x0a,0x5a,0x50,0x5b, +0xfd,0x97,0xbf,0x21,0x4c,0xd8,0xa2,0x47,0x24,0x83,0x94,0xf2, +0xe1,0xb8,0xc1,0x19,0xa0,0xac,0xe9,0x79,0x37,0x31,0xd6,0xa4, +0xac,0xb4,0xa4,0x4b,0x4c,0x2b,0x50,0x2d,0xc3,0x9a,0x4a,0x88, +0xa1,0x3a,0x15,0x31,0x67,0x67,0x16,0x35,0x30,0x37,0xbc,0x27, +0x35,0x5a,0x12,0x0a,0x92,0x2b,0x57,0x49,0xf6,0x66,0x70,0x01, +0x5b,0xc6,0x4f,0xdd,0x8b,0x4a,0x95,0x24,0x6f,0x7d,0xed,0xe4, +0x3f,0xd0,0xfd,0x99,0xd9,0x14,0x9a,0xb4,0x1f,0x4b,0xd2,0xcc, +0x36,0xd0,0x82,0x0b,0xf3,0xd5,0xa8,0xb9,0xff,0x77,0xbb,0xcb, +0xcc,0xd1,0x1f,0x05,0xf9,0x1a,0x8b,0x7b,0x2a,0x95,0xb3,0xef, +0xee,0xec,0x79,0x4c,0xf9,0x5b,0x49,0x9d,0xee,0xdc,0x4c,0xe6, +0x29,0xcb,0x5d,0x0a,0x94,0xb6,0xe7,0xa8,0x65,0x19,0xb1,0x18, +0xc6,0x5a,0x6c,0x74,0x33,0x4a,0xb6,0xbe,0xcf,0x73,0x35,0x97, +0x68,0xc1,0x39,0x55,0x7f,0xa6,0x37,0xe1,0x85,0xca,0x3f,0xdb, +0xbd,0xf3,0xb6,0x41,0x2a,0x77,0xdf,0x79,0xa0,0x3d,0x76,0xe0, +0x76,0x91,0xf2,0xde,0xf9,0xec,0x3d,0x77,0xe2,0x0e,0x46,0xdd, +0x7b,0xe5,0xcd,0x7c,0x6f,0x22,0x4c,0xd3,0xab,0x8a,0xe6,0x64, +0x28,0xc7,0x0e,0x73,0xb5,0x4d,0x0a,0x07,0xad,0xfc,0x8c,0xd6, +0x9f,0x26,0xab,0xbb,0xf9,0x48,0x7f,0x48,0x58,0xd8,0x4f,0xdb, +0x2b,0x46,0x53,0x5c,0xda,0x26,0x60,0x85,0x3a,0x51,0xaf,0xf6, +0xd4,0x69,0x2c,0x45,0xf8,0x55,0xf8,0x35,0x58,0x6f,0xdc,0x8f, +0x64,0x22,0x3c,0x6a,0x73,0x44,0x95,0xbd,0x35,0x72,0xab,0xd3, +0x6c,0x4b,0xc3,0xb2,0x35,0x24,0x5b,0xea,0xed,0xb3,0x22,0xb4, +0x21,0x9f,0x59,0xe3,0xd9,0xa0,0xcf,0xad,0xd7,0xf2,0xb0,0xcf, +0xac,0xbf,0x5c,0x7a,0xf7,0x59,0x45,0x95,0x94,0x90,0xeb,0x21, +0xb5,0x9b,0xc5,0x7e,0x2f,0xb7,0x59,0x80,0x8c,0xbc,0x0d,0x8b, +0xb2,0x45,0xf4,0x6a,0x0a,0xcc,0xc4,0x03,0x27,0x06,0x7e,0x17, +0x8f,0xd2,0x87,0x0c,0x6d,0xee,0x67,0xf8,0x0d,0xa8,0xbd,0xda, +0xe6,0x2c,0xaf,0x6e,0xbd,0xa8,0x20,0x9e,0xae,0x0b,0x79,0x5f, +0x0c,0xe3,0xe1,0x5d,0xf2,0x4a,0x9e,0x63,0x58,0x2f,0x7c,0x96, +0x01,0x7b,0x5f,0x2b,0x2d,0x82,0x3e,0x46,0xa2,0xf8,0xcd,0x73, +0x2a,0xf6,0xfd,0xf2,0xd3,0x60,0xab,0x38,0xd6,0xe6,0xd0,0xba, +0x4c,0x32,0x8a,0x0f,0xe1,0xb8,0x92,0xa0,0x28,0x0b,0xea,0x48, +0xc4,0xaa,0x9b,0xb6,0xe8,0xc8,0x28,0xde,0x5d,0x33,0x25,0x50, +0x46,0xe1,0x14,0x0e,0x71,0x6e,0x4d,0x36,0x15,0xc2,0x62,0x4a, +0x74,0xd2,0x3c,0xd5,0x07,0x01,0xa4,0xad,0xf7,0xb0,0x2a,0xb2, +0x74,0x7c,0xd5,0x11,0xb8,0xcb,0xa7,0x5c,0x39,0xdf,0x2f,0x99, +0x2d,0xb6,0xd4,0xb4,0x1d,0x57,0x06,0xb9,0x53,0x9d,0xff,0xe3, +0x6e,0x05,0x2a,0x59,0xa2,0x14,0xa1,0x61,0x8b,0x41,0xd6,0xed, +0x15,0x68,0x38,0x7a,0xf6,0x20,0xf9,0xc7,0x12,0x98,0x7e,0xfe, +0x1c,0xf8,0xa7,0xb0,0x88,0x3b,0x2c,0x94,0x2b,0x39,0xfe,0x2e, +0xc2,0x14,0x71,0x07,0x45,0xdc,0x41,0xa1,0x5c,0xc9,0x81,0x7c, +0x11,0xa6,0xdf,0xd6,0x57,0x07,0x52,0x42,0xf2,0x96,0x4f,0xd5, +0x32,0xaa,0xd6,0x6e,0x51,0xc9,0xda,0x2d,0x6a,0x78,0x25,0xab, +0xc1,0x12,0x48,0x79,0xd7,0xf9,0x7e,0x5d,0xca,0x65,0xa1,0x43, +0x7b,0xbd,0x3d,0xd7,0x9c,0xb0,0x44,0xde,0x73,0xd5,0xd9,0x57, +0x8c,0x30,0x1a,0x60,0xbf,0x7d,0x9c,0x43,0x6d,0xe4,0x59,0xc5, +0xaf,0xfd,0xee,0xb5,0x6c,0x1a,0x7d,0x7f,0xa6,0xaa,0xfc,0xd6, +0x69,0xdf,0x35,0x19,0x6b,0xab,0x6f,0x15,0x84,0xd4,0x54,0x7e, +0xdb,0x20,0x52,0x63,0x98,0x35,0xcf,0x9e,0xee,0xfe,0x6d,0x35, +0x25,0xe3,0x0d,0xa8,0x44,0x24,0xcc,0x9a,0xc4,0x89,0x0b,0x0f, +0x37,0x37,0xc9,0xb2,0x9d,0x4b,0xfd,0x8c,0xe1,0x71,0xe3,0x69, +0x26,0x8b,0x17,0xef,0x39,0x59,0x58,0xcf,0xbe,0x69,0x9d,0x84, +0x57,0x3c,0xb7,0x54,0xe7,0x7e,0x4c,0xb7,0x0a,0x94,0xcc,0x63, +0xd7,0xdd,0xeb,0xb6,0xb7,0x63,0xfe,0x82,0x06,0xe1,0x3f,0xac, +0xdb,0x6f,0x5b,0x39,0x05,0x11,0xf0,0xf2,0xe7,0xc5,0x40,0xbb, +0xea,0x08,0xf2,0x71,0x64,0x8e,0x94,0xad,0x8e,0x57,0x2c,0x34, +0x73,0x49,0x01,0xe6,0xab,0x87,0x55,0x2c,0xdd,0x44,0xb9,0xa0, +0x32,0x43,0x9f,0xc7,0x27,0x3f,0x3f,0xa4,0x4b,0xc8,0x8a,0x67, +0xb7,0x0f,0x93,0x78,0x59,0x7d,0x20,0xef,0x10,0x4d,0x29,0xb9, +0xa2,0xb9,0x63,0xf9,0x5c,0xae,0x3a,0x2d,0x2d,0xcf,0x55,0x0d, +0xa0,0x63,0x5c,0x67,0x68,0xf0,0xc1,0xae,0xb5,0xe6,0x80,0x45, +0xdb,0x63,0xbc,0x1c,0xb9,0xa7,0xad,0x6a,0xde,0xb5,0x8c,0x8e, +0xe5,0x89,0xb0,0x7b,0xb0,0x6a,0x9b,0x37,0xba,0x07,0xab,0x56, +0x8e,0x2f,0x72,0x85,0x22,0xeb,0xb2,0x33,0x57,0xaa,0x22,0xcb, +0x38,0x3c,0x41,0x22,0x54,0xa3,0x55,0xeb,0x64,0xa3,0x00,0x35, +0x1a,0xed,0xd9,0x34,0xc9,0x0c,0x87,0x1c,0x99,0xa8,0xa2,0x16, +0xb6,0xba,0x12,0xa8,0x65,0x03,0xb5,0x41,0xd9,0x49,0xce,0xec, +0x3e,0x97,0xae,0xe4,0x54,0xc9,0x71,0xef,0x3f,0x93,0x61,0x8a, +0x7a,0xbd,0x42,0x57,0x1a,0xeb,0x3b,0xc9,0xb4,0xfd,0x04,0x68, +0x98,0x2e,0x87,0xf6,0x09,0x99,0x25,0x38,0x9c,0x25,0xaf,0xff, +0x64,0xb8,0x92,0x92,0xb1,0x2d,0x43,0xc9,0xbb,0x21,0xe4,0xdd, +0xd5,0xa7,0x8c,0x6a,0x2e,0x2a,0x7a,0xb7,0xc1,0xc8,0x73,0xdc, +0x53,0xa9,0x6e,0xdf,0x06,0xea,0x98,0x5d,0x90,0x43,0x64,0xe3, +0xab,0xc6,0x8e,0x3a,0x4e,0x4b,0x48,0x36,0xf3,0x10,0xa5,0x9b, +0x14,0xfa,0x5c,0x90,0x4c,0x56,0xd6,0x4a,0xee,0x31,0xdd,0x17, +0xeb,0x7c,0x6f,0xac,0x73,0x7d,0x80,0x9f,0xb6,0xc9,0xf8,0xba, +0x72,0x7b,0x5f,0xb3,0xbb,0xa5,0xdd,0x43,0xd4,0x7d,0x8e,0x6b, +0x4f,0x16,0x2b,0x76,0xd8,0x6b,0x15,0xaf,0x9e,0xaf,0xfb,0xcc, +0x89,0xe9,0x99,0xbe,0x1a,0x24,0x96,0x7f,0x58,0xfc,0x78,0x2e, +0x6b,0x22,0xc8,0x4b,0x5c,0x44,0xc4,0x7e,0xa8,0x2c,0x4a,0xf5, +0x7a,0x8f,0x90,0x58,0xd7,0xc9,0x1e,0xdb,0x13,0x45,0x75,0x36, +0x69,0x50,0xdd,0xa2,0xdb,0xe2,0x4a,0xb9,0x24,0x89,0x8a,0x68, +0x8a,0xde,0xf3,0xa4,0x78,0x2f,0xaa,0xa8,0x06,0x9b,0x20,0xdb, +0x27,0xc2,0xb5,0x5f,0x4a,0x95,0x86,0x70,0x79,0xe5,0x5d,0x5a, +0xf7,0x92,0x2a,0x7e,0xa2,0x21,0x95,0xcc,0x12,0xd9,0x79,0xd4, +0x2f,0xff,0xcf,0x2f,0x7f,0x8f,0xc8,0x54,0xaa,0x11,0xff,0x7f, +0xd4,0xfd,0xed,0x76,0x1b,0xb7,0xb2,0x20,0x0c,0xff,0x3f,0x57, +0x61,0xf1,0xdd,0xd1,0xdb,0x2d,0x82,0x54,0x37,0x65,0xcb,0x36, +0x99,0x16,0x97,0xe3,0x24,0x27,0x9e,0x89,0x13,0x3f,0xb1,0x77, +0xb6,0x1d,0x6e,0x6e,0x2d,0x8a,0x6c,0x59,0x6d,0xf1,0x2b,0x6c, +0x52,0x62,0xdb,0xe2,0x5a,0x67,0xd6,0xcc,0x2d,0xed,0x5f,0xcf, +0xbf,0xb9,0x80,0x93,0x5b,0x7a,0xea,0x03,0x9f,0xfd,0x41,0x52, +0x8e,0xcf,0x99,0x33,0x6b,0xef,0x58,0x6c,0xa0,0x50,0x28,0x14, +0x0a,0x85,0x02,0x50,0x28,0xf0,0xab,0x9f,0xb6,0x2f,0x78,0x40, +0x2e,0x48,0x6e,0x5a,0x48,0x4e,0x56,0x76,0x5a,0x9a,0x4c,0x09, +0x0d,0x7b,0x82,0x24,0x53,0x0f,0xbf,0x50,0x3f,0xa4,0x26,0x19, +0x3e,0x64,0x32,0x00,0x20,0x62,0x0d,0x0c,0x1f,0x04,0xab,0x13, +0x11,0x94,0x12,0x09,0x32,0xb4,0x21,0x43,0x09,0x19,0xda,0x90, +0x26,0x92,0x98,0xe5,0xaf,0x62,0xbc,0x52,0x3c,0xbc,0x50,0x84, +0x65,0x8e,0x24,0x9d,0xfe,0xd1,0xb2,0xae,0x12,0x83,0x23,0xae, +0xa4,0xc1,0x54,0x1d,0x49,0x48,0x49,0x3a,0x40,0x2a,0x7a,0x25, +0x5c,0x5d,0x96,0xca,0xc1,0x69,0x5e,0xbf,0xc7,0xd7,0xe2,0x87, +0xab,0x0a,0x6d,0xa4,0x73,0xf5,0x44,0x2b,0x87,0x61,0xee,0x72, +0x91,0x3b,0x1c,0x71,0x03,0x20,0xf5,0xec,0x28,0x81,0x32,0xc5, +0x3a,0xdd,0xe6,0x37,0xc2,0xcd,0x8b,0x79,0x60,0xc3,0x7f,0xfb, +0xd6,0x3f,0xfa,0xf6,0xad,0x00,0xcb,0x14,0x7e,0xd1,0xe5,0xe4, +0xb7,0xbe,0x8a,0x3d,0x55,0x52,0x02,0x2c,0xfa,0x6f,0xdf,0x41, +0x89,0x77,0xe2,0x1d,0x94,0x78,0xc7,0x25,0xde,0xf9,0xdb,0x8a, +0xac,0x61,0x11,0x80,0x43,0x68,0x8d,0x2e,0x15,0xf0,0xab,0x79, +0x99,0x8c,0xf1,0x00,0xb6,0x64,0xa0,0xea,0xed,0xc5,0xf5,0x57, +0x40,0xcd,0xd9,0xbf,0xff,0x53,0x3d,0x7c,0xbe,0xb5,0x82,0x0c, +0x2a,0xc8,0xa0,0x82,0x0c,0x1d,0x36,0xe0,0x57,0xa1,0x82,0xac, +0x58,0x41,0xf6,0x15,0x10,0x6f,0x2a,0x00,0x85,0x48,0x7a,0x0c, +0x28,0x5c,0x07,0xc8,0x8c,0xb7,0xf4,0x98,0x5f,0x16,0x60,0x33, +0xdf,0x05,0x02,0x54,0xa6,0x78,0x2b,0xde,0xe1,0xb3,0xbf,0x18, +0xe3,0x3b,0x8b,0xa0,0x35,0xdf,0xbe,0x8d,0x9e,0x06,0xc0,0x83, +0x08,0x26,0x6a,0x4b,0xc3,0xb5,0xcc,0xfa,0x5b,0x77,0x24,0xed, +0xce,0x94,0x79,0x1c,0xc8,0x2e,0x72,0x03,0x78,0x95,0x4c,0xca, +0x52,0x00,0xaa,0xfa,0xde,0x9e,0x2d,0x31,0x52,0xa0,0xa9,0x78, +0xb6,0x5a,0x62,0x1d,0xc5,0xaa,0xb7,0x6a,0xf6,0xb7,0x20,0x19, +0x9a,0xe3,0xef,0xbc,0x77,0xa1,0xdf,0x4c,0x31,0x4e,0xbe,0x07, +0x23,0x0b,0xf2,0x42,0xdf,0xba,0xc5,0xab,0x33,0x00,0x2e,0x28, +0xcb,0xf0,0x61,0x8e,0xb0,0x28,0x8a,0xf7,0x7f,0xb8,0xcc,0x14, +0x9a,0x0c,0x3e,0xcc,0x16,0xf2,0x50,0xe2,0xdc,0xc7,0x1b,0xda, +0xd6,0x67,0xdb,0x82,0xb3,0x32,0x1c,0x3e,0x58,0x08,0xf6,0xd3, +0xc5,0x6f,0x2d,0xc7,0x99,0xb7,0x96,0xb3,0xcc,0x3b,0xcb,0x89, +0xe6,0x9d,0x71,0x9c,0x01,0x81,0x39,0x7b,0x8b,0x9e,0x76,0xe7, +0x11,0x88,0x0e,0x14,0x46,0x19,0x0a,0x61,0xb9,0x03,0x05,0xce, +0xde,0x71,0x06,0xc8,0x11,0x94,0x46,0x81,0xa2,0x0c,0x43,0x9c, +0x39,0xd1,0xb3,0xc3,0x93,0xf4,0x7a,0x6f,0xb1,0x00,0xac,0x23, +0xdf,0x62,0x11,0x5c,0x25,0x96,0x36,0x74,0xbf,0xf6,0xd8,0x8e, +0x40,0x8e,0xf3,0x8f,0xd5,0x1e,0xcb,0x11,0x08,0x86,0xc1,0xd9, +0x9a,0xc9,0xc6,0xd7,0x2d,0x83,0x08,0x47,0x06,0x91,0x9d,0x05, +0x67,0x19,0x67,0xe0,0x83,0x97,0x41,0x84,0xc3,0x64,0xcf,0xf6, +0xe4,0xfd,0x84,0xac,0x22,0x18,0x86,0xeb,0x73,0x64,0xe2,0x35, +0x94,0xd3,0x12,0x21,0x3f,0xf2,0xf2,0x40,0xc9,0x45,0x69,0x78, +0xbd,0x6f,0x95,0x1e,0x8c,0x74,0x3e,0x5a,0x84,0xc1,0xce,0x0f, +0x03,0x6a,0x54,0xd0,0x2c,0xd0,0x9e,0xdf,0xbe,0x2b,0x76,0xce, +0xfe,0xe8,0x47,0xea,0xe4,0x72,0x94,0x95,0xa0,0x07,0x4d,0x33, +0xca,0x1c,0xf4,0xf7,0xb4,0x2a,0x6c,0xf3,0x40,0x68,0x47,0x50, +0x8d,0xee,0xad,0xc7,0x0f,0x97,0xa2,0x2f,0x79,0x56,0xc8,0x7d, +0xe7,0xf1,0xe3,0xa6,0xa6,0x23,0xc5,0xdb,0x12,0x14,0xef,0x48, +0x51,0x22,0x8a,0x77,0x25,0x28,0xde,0x92,0x4a,0xb5,0x50,0x58, +0xed,0xb3,0x4c,0x93,0xf2,0xe1,0xde,0xeb,0xd1,0xd5,0xa5,0xc6, +0xd3,0x80,0xcf,0x1d,0xf1,0xe3,0x69,0x20,0x4f,0x1a,0xed,0xf1, +0xae,0x00,0x9f,0x50,0x26,0x03,0x3e,0xa1,0x42,0x7d,0x7b,0xe6, +0x8d,0x07,0xcb,0x67,0x8b,0x61,0xc5,0xc4,0xcb,0x99,0x79,0x25, +0x59,0xa5,0x76,0x7b,0xe9,0x6c,0xb5,0x18,0xc6,0xe7,0x77,0x77, +0xfc,0xa3,0x7c,0x49,0x20,0x5f,0xb5,0x06,0x28,0xfe,0x51,0x0e, +0x25,0x0d,0x69,0x89,0x51,0x95,0x11,0xfc,0x8d,0x3c,0xe5,0x5f, +0xea,0x89,0x6c,0xf4,0x98,0xa4,0x5f,0x66,0xae,0x61,0xda,0x4b, +0xec,0xb8,0xbc,0x7f,0x9b,0x82,0xf0,0xbe,0x20,0xf9,0x34,0xbc, +0x24,0x05,0x92,0xe6,0xbd,0x84,0x53,0xc2,0xaa,0x86,0x9e,0x97, +0x6f,0xa9,0xe0,0x02,0xa7,0x7d,0xae,0x6b,0xf0,0xdb,0x0c,0x6d, +0xd5,0x29,0xd9,0xb2,0x57,0x9d,0x12,0x56,0x33,0x79,0xcf,0x3a, +0x19,0xda,0xaa,0xb3,0x64,0x1c,0x6e,0xd3,0x5b,0x5c,0xaa,0x1d, +0x18,0x0c,0x5a,0x2a,0x6d,0xc3,0xdf,0x1c,0x81,0xd9,0x1d,0x9e, +0xdf,0x72,0xb1,0xd7,0x80,0x0c,0x98,0x5b,0x53,0xca,0x44,0xd7, +0x04,0x67,0x5c,0x39,0x48,0x99,0xe8,0x2e,0xdb,0x35,0x69,0x32, +0xd0,0xd0,0x8e,0xcd,0x59,0x86,0xda,0x7d,0x31,0x59,0x45,0x37, +0xca,0x17,0x73,0xce,0xfd,0x75,0xaa,0x0b,0x56,0xed,0x4d,0x5d, +0x3c,0x9b,0x37,0xa5,0x4a,0x0f,0x70,0x3f,0xeb,0x50,0x79,0xf0, +0x31,0x99,0xac,0x96,0x57,0x83,0xb1,0x71,0x51,0xcd,0xe7,0xd8, +0x36,0x5c,0xfa,0xef,0xff,0x2f,0x99,0xff,0xae,0xed,0x49,0x6b, +0x8d,0xd6,0xb1,0x47,0x4b,0x04,0x05,0x81,0xf1,0x6e,0x54,0xc1, +0x3f,0xfe,0x87,0x2e,0xd0,0xe2,0x50,0x84,0xe4,0x40,0xff,0xc7, +0xff,0xc0,0xb3,0x75,0xbf,0xa3,0x77,0x05,0x8b,0xc4,0xdc,0x63, +0x7b,0xae,0x58,0x3a,0xb7,0x43,0x57,0x04,0x70,0x6e,0x8f,0x5b, +0xb9,0x09,0x6b,0x93,0xe5,0x9e,0xbc,0xe0,0x88,0xca,0xdc,0x2e, +0x5c,0x97,0x59,0x79,0x7a,0x67,0xe0,0xf0,0x70,0x78,0xac,0x57, +0x73,0x43,0x7e,0x24,0x4a,0xc7,0xac,0x2f,0xe5,0x80,0x26,0xe2, +0x33,0x79,0xa0,0xca,0x6f,0xe1,0x82,0x02,0x71,0xbd,0x95,0x9f, +0xcf,0xa6,0x97,0x7c,0xdf,0xf9,0x1e,0xde,0xca,0xba,0x50,0x99, +0xb7,0xb2,0xce,0x2c,0xb8,0x45,0xef,0x6e,0x64,0xa9,0x63,0x74, +0x45,0xdb,0xf2,0xd9,0x76,0xff,0xbe,0x9f,0xce,0x26,0x98,0x7d, +0x3f,0xf9,0x0e,0x8f,0x4d,0xe2,0x46,0xe8,0x45,0xb6,0xe9,0x31, +0x8d,0xf6,0x1e,0xbd,0xa4,0xca,0xe4,0xa8,0x57,0xc9,0x6a,0x6c, +0x4e,0xe2,0xc5,0x10,0xf7,0xae,0x0b,0x1b,0x9c,0xbc,0x07,0x46, +0x95,0xf4,0xd6,0x42,0x8d,0x28,0x20,0x8b,0xd7,0x8f,0xf1,0x9a, +0xd6,0x7e,0x0d,0x68,0xdb,0xe5,0x1f,0xff,0xd6,0x37,0xbe,0xe8, +0x0a,0x61,0x35,0xad,0x0a,0xe2,0x55,0x81,0x66,0x95,0x93,0xa3, +0x59,0x25,0xdb,0x9c,0x9e,0x2d,0x96,0x57,0x33,0xb0,0x7b,0xe6, +0x57,0x5b,0xb9,0x6d,0x78,0xac,0x18,0x0b,0x43,0xc3,0x30,0xd6, +0xc1,0x72,0x0f,0xe6,0xda,0xe5,0x72,0xc4,0xda,0x59,0x6e,0xa4, +0x1e,0xd0,0xc0,0x7b,0x50,0x5c,0x2a,0x1f,0xae,0xd6,0xdb,0xae, +0xf4,0xb0,0x8b,0x20,0xd9,0x52,0x79,0x6e,0xd5,0xf7,0x68,0xa6, +0x53,0x30,0xd7,0x4e,0x27,0xcf,0xf1,0x00,0xa1,0x65,0xec,0xcb, +0x7d,0xe4,0xca,0xda,0x4f,0x42,0x3b,0x2d,0x99,0x5e,0x79,0x6b, +0x79,0xd5,0x18,0x55,0x5c,0xc6,0xc7,0xc0,0xa4,0xc6,0xb5,0x5a, +0xcb,0x7c,0x3c,0x19,0x85,0x6c,0x84,0xf5,0x2d,0xc1,0x2b,0xd6, +0xfd,0x39,0x22,0x58,0xc4,0x92,0x6b,0x77,0x11,0x40,0xce,0xdb, +0x13,0x8e,0xb5,0xd5,0x4c,0x6f,0xde,0x9b,0x5f,0xcd,0xf2,0x3d, +0x04,0xa4,0x03,0x72,0xcf,0x31,0xd7,0x79,0x58,0x44,0x4f,0xe7, +0x2a,0xd7,0x6c,0x60,0xce,0x16,0x26,0x0a,0x1e,0xe6,0x0c,0x28, +0x48,0xb4,0x82,0xfb,0x91,0x52,0x44,0x8d,0x73,0x1a,0xc3,0xf1, +0x2c,0x8d,0x47,0xb5,0x22,0xc4,0x73,0xca,0xc0,0xf7,0x60,0xe6, +0x76,0x26,0xae,0xb9,0x44,0x0d,0x53,0x1b,0x1c,0xac,0xb0,0x96, +0xcf,0xfd,0x86,0x92,0x25,0x0c,0x45,0x32,0x2c,0x80,0x3c,0xa3, +0xf8,0x86,0x17,0xd0,0x5f,0xa9,0x9d,0xf5,0x0d,0x26,0x88,0x1a, +0xa5,0x37,0x66,0xf3,0x78,0x5a,0x2b,0xe4,0xfe,0x0c,0xa9,0x0a, +0xa2,0x84,0x74,0x82,0x91,0x94,0x5f,0xac,0xa6,0xa3,0x71,0xec, +0xe4,0x52,0x8a,0x18,0x0e,0x68,0x79,0x31,0xb6,0xb3,0x9e,0xcb, +0x34,0x51,0x53,0xb9,0x45,0x02,0x14,0x0c,0xd3,0xa0,0xe1,0x4a, +0xc8,0x50,0x90,0x92,0x12,0xd0,0xa3,0xb3,0xe5,0x6c,0xea,0xd0, +0xf2,0x52,0xa6,0xf1,0x63,0x35,0xa5,0x9d,0xb8,0xf5,0xe1,0x1a, +0x0e,0x1f,0x07,0x09,0x91,0x4e,0x6c,0x32,0x25,0xd1,0xb1,0x24, +0xe9,0x2f,0xc7,0xfc,0xbe,0x08,0x00,0xe0,0x20,0xcf,0xc9,0x0b, +0xb1,0xea,0x9b,0xf8,0x63,0x12,0x2f,0x42,0x3c,0x43,0x68,0x1d, +0x9f,0x88,0x10,0xfe,0xe3,0xb7,0xcc,0x4b,0xa0,0x5a,0xfc,0xd6, +0xf9,0x09,0x41,0x56,0x42,0x9d,0x30,0xd4,0xa9,0xc4,0x77,0xda, +0x97,0x1e,0x6a,0x93,0xe6,0x15,0x1e,0x53,0xeb,0xa6,0xe0,0x58, +0x4f,0x86,0xb8,0xfd,0xa6,0xd7,0x86,0x08,0xe0,0x8d,0x60,0x94, +0x93,0xbb,0x32,0xfe,0xd0,0x0f,0xde,0x48,0x57,0xe5,0x5e,0x9f, +0x5a,0xa1,0xca,0x8a,0x91,0x48,0xc4,0x07,0x79,0x8d,0x6c,0xdd, +0x12,0x59,0x4b,0xac,0xc4,0x8d,0x18,0x88,0x74,0x2e,0x2e,0xd7, +0xd6,0x23,0x3d,0xa8,0x2c,0x2e,0x33,0x3b,0x21,0xf3,0xc5,0x34, +0xb2,0xaa,0x10,0x73,0xf8,0x1b,0x4d,0x1b,0x21,0x9b,0xbc,0x74, +0x41,0x1b,0x26,0xed,0xe1,0x35,0xfe,0xb8,0x8a,0x28,0xa0,0x08, +0xa0,0x8c,0x22,0xab,0xd5,0x6f,0x0f,0x0f,0x33,0x37,0xe5,0x9d, +0xaf,0x48,0x23,0xdc,0xfc,0xe6,0x04,0x3a,0xcd,0xe1,0xe5,0x64, +0x9d,0xd5,0x33,0x97,0xb0,0x55,0x1a,0x1f,0x1f,0xf5,0xea,0x97, +0xeb,0x26,0x86,0x87,0xe7,0xa5,0xdf,0x88,0x90,0xe0,0x95,0xe5, +0xc4,0x17,0xf5,0xcb,0xcc,0xce,0x82,0xa4,0x3e,0xfb,0xe3,0x25, +0x51,0xa8,0xd1,0x69,0xae,0x42,0x21,0x8c,0x3c,0xa0,0x3f,0xaf, +0xfa,0x14,0x97,0xda,0xcd,0x8e,0x22,0x37,0xff,0xf0,0xd0,0xce, +0x0f,0xdc,0xe2,0x01,0xdd,0xd5,0xbe,0x8a,0x12,0x55,0x6b,0xa0, +0x6b,0x4d,0x0e,0xa2,0xe8,0x0a,0x32,0xb3,0x30,0x72,0x2b,0x68, +0xb8,0xf8,0x71,0x77,0xcb,0xad,0xa1,0xe1,0xd6,0x20,0x79,0xcf, +0xcc,0xf8,0x44,0xe7,0x27,0x6d,0xfb,0xe6,0x2b,0xee,0x8f,0x61, +0x70,0x99,0x51,0xbc,0x6e,0x27,0x1b,0x9f,0x29,0x91,0x65,0xe8, +0x3e,0xbc,0x7b,0xae,0xa2,0xd6,0x89,0x7c,0x12,0xd3,0xb8,0xe0, +0xbf,0x1b,0x5f,0x0c,0x38,0xc4,0x00,0x1e,0x96,0x72,0x9a,0xb8, +0xb1,0x52,0x08,0xbf,0x58,0x41,0x8f,0x21,0x6b,0x51,0x2e,0x64, +0x3b,0x51,0x2a,0x3f,0x28,0xc0,0x44,0x01,0x0e,0x22,0x2b,0x89, +0xb0,0x11,0xa0,0xdd,0xd6,0x9b,0xb2,0xb6,0x66,0x2e,0x40,0x09, +0xb7,0x5a,0x06,0xe0,0x43,0x29,0x06,0x17,0xa0,0x8c,0xdf,0x47, +0xeb,0xb0,0x9e,0x85,0x47,0x59,0x78,0x16,0xad,0x5b,0x47,0xeb, +0x56,0x3d,0x6b,0x1d,0x65,0x2d,0x79,0xb7,0xca,0x34,0x22,0x6a, +0x84,0x1d,0xf5,0x0a,0xca,0x46,0xe6,0xad,0x4c,0xde,0x66,0x90, +0x6f,0x22,0xb0,0x27,0x01,0xa6,0x7d,0xa0,0xc7,0x4f,0x68,0x9c, +0x70,0xa7,0x5d,0xf9,0x74,0x0f,0xff,0x03,0x48,0x47,0x8b,0xb8, +0xf6,0xc1,0x97,0x25,0x3f,0x48,0x74,0x67,0x8d,0x90,0x6e,0x3e, +0xea,0x22,0xb9,0x7c,0x40,0x50,0xaf,0x73,0xcf,0xa6,0xf3,0x88, +0xe1,0x64,0xfc,0x03,0xea,0x8a,0x0f,0x84,0x14,0xaf,0x33,0xe4, +0x4b,0x7e,0x1d,0xc8,0x07,0x74,0x3a,0x07,0x3c,0x03,0x4f,0xce, +0x51,0xa3,0x3c,0x7f,0xfe,0x37,0xae,0xae,0x97,0xce,0x1b,0x2d, +0x39,0xaa,0xf1,0x77,0xa8,0xe4,0x4d,0x63,0xd0,0x43,0xd4,0xef, +0xf8,0x8d,0x46,0x3a,0xef,0x28,0xd8,0x7a,0xbd,0x1f,0xe5,0x60, +0xe5,0x6d,0xbd,0x71,0x86,0xc3,0x99,0x47,0x04,0xe2,0xec,0x24, +0x67,0xd0,0xf6,0x46,0x23,0xf1,0x31,0x8f,0x5b,0x48,0xa3,0x98, +0x51,0x25,0xfd,0xbe,0x6f,0xee,0x10,0x8e,0xb3,0x8d,0xbe,0xe5, +0xae,0x95,0x0a,0xef,0x10,0x6a,0x8d,0xd2,0xa9,0xda,0xed,0x20, +0x6d,0xa9,0x09,0x6e,0x7b,0xf8,0xdd,0x5c,0xef,0xb9,0x53,0x1d, +0x9d,0x0b,0x84,0xf7,0xdb,0xeb,0x0d,0xfd,0x68,0x66,0xfb,0x15, +0xcc,0x74,0xc1,0x8c,0x0b,0xea,0x5d,0x0c,0xed,0xbd,0xba,0xdd, +0xf3,0x01,0x9b,0xb6,0xba,0x18,0x8e,0x07,0x69,0x6a,0xe7,0x0b, +0xd5,0x63,0x12,0x89,0x7e,0xad,0xd8,0x71,0x6e,0xb0,0xb7,0x30, +0x8a,0xa0,0x51,0x8e,0x10,0xcb,0xb7,0xb8,0xd7,0xef,0x54,0x95, +0xca,0x39,0x6f,0x5a,0x0f,0xf0,0xa8,0x18,0x74,0xfc,0xca,0x37, +0xcf,0x12,0xfc,0x66,0x75,0x6f,0x8a,0xc2,0x43,0x05,0x03,0x15, +0x90,0x0e,0x23,0x4d,0xc8,0xdc,0x84,0xf3,0xea,0x7c,0x64,0xab, +0x4f,0x74,0x8f,0x2e,0xcc,0x95,0x99,0xe6,0xa3,0x23,0x04,0xd9, +0x54,0xb6,0xbb,0xc4,0xa7,0xf3,0x5a,0x46,0x54,0x13,0xc3,0x32, +0xca,0xd6,0x11,0x86,0x1b,0x08,0x6c,0x0a,0x49,0x2c,0xf3,0x7d, +0x78,0x77,0xe7,0x5d,0x63,0x08,0x0d,0xef,0x94,0x9f,0x9a,0x1e, +0xd0,0x0d,0x4e,0xd0,0xa5,0xa5,0xed,0x18,0x46,0x9a,0x72,0x8a, +0x98,0x73,0x44,0x11,0x33,0xd6,0xf5,0x88,0xa2,0x76,0xd4,0x29, +0x74,0xc6,0xd1,0x50,0x64,0x94,0x10,0x62,0x42,0x88,0x09,0x1d, +0xb5,0x1c,0xc4,0x6b,0x2f,0x47,0xd7,0xfd,0x6d,0xed,0xc4,0x4b, +0xd5,0x66,0x43,0x6e,0x25,0xf7,0xbe,0x74,0x98,0xdd,0xe9,0x7c, +0xb5,0x84,0x39,0x7e,0x22,0x2e,0xa0,0xdd,0x23,0x21,0x4d,0x9c, +0x1c,0x3a,0xb6,0xb3,0x74,0xe9,0x12,0xfe,0x34,0xca,0x4b,0x70, +0x00,0xda,0x81,0xc5,0x33,0xc9,0x06,0x22,0x80,0x2a,0x8f,0x24, +0x56,0x79,0xd8,0x86,0x41,0xd7,0xf9,0x5b,0x6e,0x99,0x05,0x0e, +0xbb,0xa8,0x48,0xcf,0x9b,0xf0,0x0f,0x55,0x39,0x53,0xed,0xa3, +0xd4,0x7c,0x40,0x5d,0x0b,0x2a,0xec,0xeb,0x49,0xc7,0x1f,0x49, +0xf0,0x0f,0xfd,0x3c,0x7f,0x5e,0x4c,0x53,0xb0,0xf4,0xbd,0x91, +0xc0,0x89,0xac,0xeb,0x95,0xe7,0x0e,0x29,0xf7,0xee,0x4e,0x11, +0xc4,0xca,0x26,0x0f,0xba,0xc4,0x15,0xc8,0x70,0xe9,0x21,0xf7, +0x10,0xde,0x34,0x80,0xe1,0x7d,0x15,0x4e,0xb6,0x1c,0xfd,0xe1, +0xe1,0x7d,0xd1,0xc3,0x9a,0xb5,0xf3,0x2f,0xdc,0xe2,0x5c,0x61, +0xf9,0x81,0x42,0x03,0x3c,0xbf,0x50,0x6e,0x5a,0x32,0xd9,0x68, +0x12,0x7c,0x37,0x67,0x35,0x1d,0x64,0x65,0xd6,0x22,0x0a,0x45, +0x3c,0xc2,0xf8,0x2b,0xda,0x76,0x72,0x4e,0x76,0xf5,0x92,0xb1, +0x0f,0x13,0xfd,0x72,0x91,0xd0,0x74,0x65,0x3f,0x43,0xa5,0x5a, +0x70,0x33,0x5b,0x80,0x09,0x9e,0xbc,0x89,0xd3,0x34,0x1e,0xd3, +0x26,0xaf,0x36,0x2b,0x35,0x32,0x98,0xca,0xa9,0xae,0x5e,0xdc, +0x5c,0xc4,0xef,0xd1,0x0d,0x6a,0xcc,0xda,0xbf,0x2f,0x63,0x38, +0xab,0xf9,0x56,0xe7,0x2f,0x64,0x3e,0x3e,0x44,0x45,0x45,0x8b, +0x46,0x3c,0x26,0xe3,0x23,0x74,0x64,0xc9,0xda,0x06,0x91,0x18, +0xae,0xa3,0x1b,0x9c,0xd8,0x87,0x19,0xfc,0x05,0x59,0x44,0xc8, +0x62,0xf9,0x1b,0x28,0x2a,0x1d,0x71,0x2c,0xe3,0x08,0x0b,0x36, +0x86,0x30,0x85,0xe1,0x50,0x1d,0x66,0xaa,0xfe,0xfb,0xd8,0x45, +0xfa,0x7a,0x07,0xc6,0xb1,0x9e,0x44,0x54,0x5c,0x85,0xd7,0xe9, +0x4c,0x60,0xe6,0xfd,0x00,0x33,0xb3,0x66,0xa9,0x34,0x59,0x6f, +0xa8,0x1a,0x14,0x61,0xfe,0x5b,0x0f,0xfb,0xd4,0x78,0x0d,0x67, +0xba,0x55,0x72,0xdc,0xf4,0x2a,0x4f,0xa9,0xd6,0x0a,0x40,0x42, +0xc8,0x45,0x80,0xbe,0x0f,0xcf,0x86,0xbe,0x14,0x39,0x36,0xad, +0xb7,0x74,0xfa,0x7d,0x2d,0xff,0xdf,0xa2,0x30,0x3e,0xad,0xb0, +0xef,0x2f,0x0b,0x06,0xbe,0x5c,0x1e,0xb8,0xe6,0xbd,0x4c,0x34, +0x6f,0x07,0x4c,0xc9,0x0e,0xd2,0x16,0xb2,0xb6,0x9e,0xa2,0x7b, +0x9a,0xf8,0x32,0xb8,0x79,0x85,0xc0,0x4a,0xe6,0xd8,0xe2,0x4a, +0x07,0x5c,0xa1,0x48,0x5b,0xb8,0x2a,0xc2,0x25,0x11,0xac,0x8d, +0x5a,0x9d,0x10,0x5a,0x01,0x33,0xdd,0xe1,0x61,0xdc,0xbc,0x00, +0xfb,0xa4,0xeb,0xa5,0x21,0x7c,0xc7,0xf3,0xe6,0x02,0x20,0xf9, +0x17,0x4c,0xe9,0x3a,0x75,0xac,0x53,0x17,0xbe,0x90,0x65,0xbb, +0x68,0xde,0xa7,0x61,0x37,0x0d,0x9b,0x59,0xbb,0xf1,0x1b,0xda, +0xf2,0xa0,0xbc,0x1b,0x80,0xf0,0x88,0xea,0x88,0xd2,0x56,0x37, +0x6d,0x41,0xde,0x6f,0x68,0xb8,0xea,0xac,0x16,0xa0,0x5d,0xab, +0x82,0x6b,0x2c,0x98,0xa9,0x82,0x83,0x23,0x22,0x51,0x16,0x5c, +0x43,0xc1,0xac,0x65,0xb2,0x5a,0xbc,0x60,0xbd,0x81,0xb5,0x29, +0x9f,0x65,0x8b,0x1b,0x58,0x80,0xd2,0x22,0xaf,0xdf,0x51,0xa2, +0x50,0x39,0x2e,0x43,0x80,0xf6,0x45,0x11,0x6c,0x51,0x02,0xb6, +0x31,0x80,0x91,0xfa,0xe1,0x8a,0x97,0x4c,0x55,0x83,0x16,0x86, +0xa9,0xee,0x50,0x39,0x5c,0xcd,0x37,0x1a,0x06,0x34,0x36,0x95, +0x15,0xe2,0x60,0xba,0xc9,0xb9,0xc9,0x54,0x0d,0x5d,0x30,0x79, +0x62,0x7a,0xc4,0x95,0x1d,0x71,0x14,0x2e,0x79,0xb6,0xb3,0x6d, +0x50,0x63,0xdd,0xbd,0x01,0x58,0x1c,0xf4,0xe3,0x02,0x06,0x84, +0x52,0x7c,0x84,0xb3,0xe0,0xb8,0x33,0x32,0xef,0x61,0x1e,0x24, +0x77,0x77,0x5c,0x6c,0xa4,0xca,0x53,0x99,0x5e,0x02,0x13,0x57, +0xdf,0x78,0xf2,0x98,0xb2,0xba,0x5a,0x49,0x20,0x14,0xdc,0xd8, +0xfc,0x2c,0x2a,0xaf,0x1c,0x2b,0xa7,0x91,0xdb,0x34,0x94,0xf7, +0x83,0xa9,0xef,0x62,0x95,0x9a,0x06,0x64,0xa7,0xf1,0x5b,0x5f, +0xe0,0x5f,0xfc,0xa3,0xfe,0x85,0x34,0x0a,0x8e,0x7a,0xe0,0x4d, +0xcf,0x5a,0x32,0xb8,0x26,0x86,0x83,0x54,0x3d,0x22,0xe6,0xa1, +0xaa,0x84,0x96,0x98,0x2d,0xfd,0x45,0x2e,0x18,0xd1,0x3c,0x90, +0x3e,0x1a,0xf0,0x83,0xd7,0xa8,0xf3,0x50,0xae,0xcf,0xe0,0x07, +0xaf,0xc3,0xe6,0x2d,0xb9,0xde,0x82,0x1f,0x90,0x32,0x5a,0x9b, +0x5b,0x9a,0xeb,0x56,0x03,0x17,0xa5,0xa3,0x2c,0xca,0x5a,0x8d, +0x8c,0x1e,0xac,0x34,0x37,0x38,0x33,0xff,0xeb,0x7f,0xff,0x27, +0x93,0x04,0xf9,0xe1,0x59,0x16,0x74,0x1b,0xbf,0xb5,0x7f,0xeb, +0x14,0x1a,0x96,0x51,0x53,0xb2,0xbe,0x15,0x5d,0xf4,0x9f,0x67, +0xa3,0xb5,0x8a,0x4a,0xb2,0x0e,0xcf,0xd6,0xa5,0x45,0xd7,0xcc, +0x92,0x35,0xf0,0x82,0x8b,0xca,0xba,0x60,0xcc,0x35,0xd6,0x81, +0x7f,0xe4,0x11,0x51,0xfe,0x99,0x24,0x13,0xbe,0xc3,0x46,0x16, +0xf8,0xdd,0xdf,0x70,0x24,0x7e,0x8c,0x6c,0x4a,0x1b,0xa3,0x75, +0x47,0x7f,0x7f,0x44,0xc2,0xbb,0x6e,0x5d,0xc1,0xd9,0x28,0xeb, +0xc2,0xd8,0xcf,0x90,0xce,0xb6,0xf7,0x11,0x1f,0x34,0xf0,0x32, +0xf9,0xca,0x58,0x65,0x83,0xf0,0x62,0x3f,0x07,0xb3,0x91,0x7e, +0x50,0x7e,0x7e,0xfb,0xc0,0xca,0x23,0x6b,0xd0,0xd3,0xc3,0x36, +0xe9,0x17,0x37,0x1b,0xac,0x4c,0x75,0x1d,0x93,0x35,0x68,0xc7, +0x15,0x9a,0x74,0xf7,0xd2,0xca,0xa6,0x8a,0x2e,0x51,0x54,0xae, +0xb5,0xd4,0xbc,0x24,0x27,0xac,0xb6,0x27,0x13,0xee,0xb3,0xde, +0x92,0x45,0x68,0xc9,0xa5,0x8a,0xdf,0x63,0xd5,0xa5,0x8b,0x67, +0xa6,0x78,0x59,0x7c,0xa5,0x1d,0x77,0xfd,0xad,0x26,0x1f,0x1e, +0xf6,0xac,0x2f,0x52,0x64,0xd6,0x67,0xab,0x4f,0x93,0x0f,0xdf, +0x2d,0x39,0xf7,0x0b,0xac,0xd2,0xc2,0xc6,0x9e,0x4e,0x81,0x15, +0xe6,0x28,0x50,0xdb,0x17,0xc6,0x03,0xaa,0xa5,0xbd,0x9e,0x3a, +0x36,0xa6,0xdc,0x8a,0x0d,0x2f,0xc3,0xb3,0xae,0xa7,0xbf,0x2d, +0xfc,0xdb,0x32,0x7f,0xc9,0xb2,0x50,0x6f,0xde,0x31,0x0b,0x0c, +0x2f,0xd2,0xe4,0xe3,0x7e,0xfe,0x10,0x45,0xe6,0x79,0xe7,0xc0, +0x0a,0xf6,0xa2,0x3e,0x07,0xc9,0x76,0x58,0xe4,0xb2,0xc4,0x54, +0x07,0x02,0x74,0x6d,0x79,0x1b,0x96,0x1a,0x2c,0x74,0xe4,0xb1, +0xcb,0x5a,0x61,0x44,0xbd,0xfe,0x7d,0xcd,0x96,0xff,0xaa,0x06, +0xcb,0x4e,0x2b,0xbb,0xc2,0x68,0x19,0x47,0xf9,0xb9,0x5c,0x2c, +0xa2,0xfc,0xbc,0xdd,0x21,0x96,0xf6,0xc6,0xfd,0xde,0xa2,0x0f, +0x0b,0x59,0xeb,0x2b,0xe2,0xdf,0x8b,0x3e,0x7c,0x62,0x7c,0x2f, +0x62,0xab,0xdc,0x48,0x64,0x3f,0x8d,0x36,0x11,0x3e,0xee,0x4b, +0x67,0x0c,0xfe,0x5c,0x40,0x17,0xf8,0xaa,0x17,0x4c,0xef,0x9a, +0x75,0x43,0xae,0x87,0x71,0x63,0x6f,0xf7,0x26,0xb0,0xe3,0x0b, +0x64,0x96,0x33,0x8c,0xc3,0x84,0xb4,0x14,0x65,0xbd,0xb0,0x5b, +0x0c,0xe4,0xe2,0xd6,0x96,0x05,0xb9,0x68,0xf5,0x3e,0xbb,0xdb, +0xfc,0x26,0xe6,0xc3,0xc2,0xad,0x8a,0x74,0xa9,0xf9,0x5c,0x41, +0x56,0x6c,0x32,0x0f,0xab,0xc8,0x84,0xbc,0xf9,0x84,0x61,0xaf, +0x3e,0xd9,0xa1,0xb3,0xa8,0x3a,0x34,0x15,0x34,0xcf,0x7d,0x77, +0x73,0x47,0xa6,0xfe,0x3c,0x9f,0xcf,0xd2,0x64,0x19,0x47,0x9f, +0xc6,0xed,0xda,0xa2,0x26,0x16,0xed,0xda,0xb8,0xb6,0xd1,0x47, +0x0a,0xbf,0xaf,0x06,0xa3,0xe5,0x22,0x8e,0xf3,0x4b,0x0a,0xfb, +0x48,0xc0,0x5a,0x5e,0x28,0x70,0xd9,0x99,0x3a,0x3d,0x99,0xa6, +0x78,0xf7,0x66,0x0a,0xbc,0x40,0x2f,0x64,0xa7,0x30,0x2a,0xd3, +0x24,0xfd,0x69,0xf0,0x13,0xf4,0xc5,0xe1,0xa1,0xfc,0x99,0xf9, +0xb8,0x03,0x89,0x46,0xcb,0x40,0xbe,0x8e,0x3e,0x5d,0x47,0xd3, +0xe6,0x5a,0x4c,0x33,0xf8,0x93,0x29,0xc5,0x79,0x10,0x4d,0xd7, +0xbe,0x6d,0x10,0x4c,0xd7,0x8d,0xb5,0x5f,0x37,0x9f,0x59,0x03, +0x2c,0x84,0x66,0x10,0xfa,0x4c,0xc1,0xf3,0xab,0x64,0x3c,0x2a, +0x25,0xc3,0xa8,0xdb,0x29,0xc7,0x19,0x99,0xf2,0xe4,0xd7,0x81, +0x4a,0xb1,0x46,0xf5,0xcd,0xf7,0x9d,0x5d,0x6c,0x5c,0x42,0x4c, +0x91,0x3a,0x07,0xa9,0xd8,0x55,0x2b,0x1b,0x23,0x58,0x05,0x94, +0x85,0x4a,0x32,0xa1,0xaa,0x19,0x49,0x3b,0x65,0x17,0x82,0x1c, +0x8b,0xab,0x21,0x79,0xe1,0xb2,0x8e,0x9a,0x8f,0x8e,0xc0,0x76, +0xa9,0x83,0xf9,0x2f,0xd2,0x8c,0xbe,0xb2,0xb0,0x8e,0xb4,0x2e, +0x92,0xf7,0x57,0xcb,0x68,0x7d,0x16,0xa5,0x6b,0x71,0x31,0x5b, +0x2e,0x67,0x93,0x28,0x83,0x8f,0x0c,0x86,0x83,0xc7,0xdf,0x5f, +0x7f,0x1d,0xfa,0x75,0x02,0xeb,0x70,0xcf,0x60,0x48,0x3d,0x98, +0xad,0x9a,0xd3,0xd9,0x88,0xd6,0xd7,0xa0,0x31,0xcc,0x87,0xde, +0x42,0x52,0x42,0xf1,0x13,0xbe,0xf3,0xe5,0xcb,0x8a,0xba,0xb8, +0x66,0x59,0xb7,0x71,0x75,0xa2,0xaa,0xeb,0xe2,0xfa,0x27,0x6b, +0xe3,0x4a,0x27,0x13,0x5b,0x24,0x66,0xc3,0x63,0x7b,0x9d,0x8a, +0x2c,0x85,0x49,0x60,0x0a,0x99,0xe7,0x90,0x7b,0x0e,0xd9,0xf0, +0x17,0xfe,0xdb,0x39,0xc2,0x8d,0xf8,0x80,0xc9,0x06,0xa5,0x23, +0x42,0x7f,0x1e,0x51,0x15,0xf0,0x85,0xd5,0xc0,0x57,0x4b,0x3f, +0x69,0xbe,0xa6,0xcf,0xf3,0xa8,0xe1,0xad,0x09,0xec,0x9c,0xaf, +0xdf,0xae,0x69,0x5e,0xc9,0xe8,0x5f,0x77,0xc9,0x4b,0x71,0x83, +0x97,0x05,0x4b,0xcc,0x68,0x89,0x51,0x73,0xfd,0x35,0xe0,0xc2, +0xb0,0xa6,0x80,0x0d,0xbe,0xc0,0xc2,0x6d,0x66,0x5f,0x67,0x94, +0x94,0x51,0x52,0x86,0x49,0xeb,0x33,0xa8,0x1b,0xa1,0x5a,0x06, +0xea,0x2c,0xa3,0xa4,0xac,0x25,0xa1,0xd6,0x52,0x0f,0x53,0x7e, +0xa6,0x3f,0x32,0xdf,0x39,0x1e,0xd3,0x44,0xb0,0x51,0x71,0x1e, +0x81,0x02,0xf3,0x1c,0xb5,0x95,0x61,0x5a,0x46,0x0b,0x96,0x0e, +0x50,0x75,0xb6,0x56,0xe4,0xad,0xd1,0x65,0x1a,0x12,0x32,0x45, +0x5c,0x06,0x09,0xeb,0x73,0x8b,0x34,0x82,0x38,0xb7,0x08,0x23, +0x08,0x49,0x0a,0x65,0xca,0xdf,0x90,0xce,0xfd,0x07,0x22,0xdf, +0x3a,0x6f,0x60,0xd7,0x91,0x65,0x7f,0x0e,0x56,0xf4,0x79,0x67, +0xb4,0x26,0x13,0xb8,0x45,0x3c,0xae,0x8f,0x50,0x3e,0xb0,0x73, +0xe0,0x67,0x46,0xca,0x6b,0x31,0x9b,0x2d,0x2b,0xe4,0x0a,0xfb, +0x14,0xb3,0xf1,0xad,0xf2,0xc8,0x5e,0x40,0x49,0x49,0xc2,0x3c, +0x10,0x26,0x6a,0x34,0x45,0x2c,0x16,0xaa,0xad,0x79,0xf9,0xc1, +0x47,0xd9,0x10,0xd1,0x4d,0x02,0x8a,0xd1,0xa0,0xba,0x54,0x11, +0x4b,0x4c,0xcd,0xbf,0x22,0x84,0x77,0x49,0xe0,0x25,0x58,0x78, +0x1e,0x21,0xfb,0x0e,0xe4,0x8c,0x4f,0x4e,0xe4,0x44,0xe2,0x10, +0x25,0x7b,0x60,0x4d,0x8b,0xa9,0x8c,0xfe,0xcd,0x23,0xc3,0x03, +0x0f,0xd6,0x08,0x24,0x65,0x6a,0xd5,0xa7,0x1a,0xac,0x97,0xa0, +0x20,0x91,0x19,0x9b,0x22,0xd1,0x88,0x55,0x15,0x82,0x6c,0xf8, +0xc5,0x1a,0x14,0x49,0xb1,0xcf,0x59,0x08,0x87,0xbd,0x5e,0x16, +0xde,0xe8,0xf6,0xbb,0x3a,0x08,0xaf,0x61,0xc2,0x73,0x82,0x7d, +0xab,0xfd,0xae,0xf3,0x39,0xef,0xc4,0x09,0x4c,0xdf,0x8c,0x91, +0x84,0x83,0x87,0x19,0x8f,0x39,0xf8,0x17,0x1f,0xfb,0xd6,0xf3, +0x86,0x9c,0x05,0x61,0x01,0xa0,0x92,0xee,0xb3,0x02,0x50,0x65, +0x68,0x09,0xa0,0x11,0xdc,0x63,0x0d,0x60,0x10,0x64,0x16,0x82, +0x7b,0xdc,0xf6,0x50,0xe6,0x3c,0xaa,0x37,0x6c,0x1d,0x34,0xb6, +0x45,0xfd,0x40,0x9b,0x34,0xf7,0x31,0xe3,0x7d,0x8b,0x18,0x25, +0x43,0xec,0xcf,0x6b,0xd9,0xf0,0x64,0xbb,0xf7,0x2d,0x52,0xf7, +0x36,0xd2,0xb7,0x13,0x0a,0x29,0x81,0x24,0xcc,0xa2,0x6a,0x0b, +0x49,0xb4,0xfa,0x15,0xb4,0x16,0xb6,0xc8,0xe1,0xb3,0x2b,0xcb, +0xcd,0xf7,0x97,0xf7,0x17,0xf4,0xb2,0xa0,0x93,0x92,0x83,0xe1, +0x17,0xd4,0x72,0x60,0x3f,0x2b,0x0f,0x5c,0x1b,0xf2,0xa7,0xd5, +0xe4,0x82,0x36,0x71,0xce,0x0b,0x89,0x39,0x48,0x76,0x7c,0xcf, +0x41,0x72,0xa2,0x32,0x8a,0xac,0x8c,0xf3,0x29,0x23,0x3e,0xee, +0x35,0xea,0xfd,0xae,0xd7,0x6d,0xff,0x7d,0x54,0xff,0x7b,0xb3, +0xfb,0xf7,0xd1,0xd1,0x1d,0xfd,0xa9,0xfb,0x90,0xd6,0x8b,0xbf, +0xeb,0x53,0x3e,0xbd,0x97,0x79,0xfc,0xbe,0xe3,0xd6,0x98,0xab, +0xcb,0xa5,0x07,0xfd,0x87,0x7a,0x25,0x57,0x14,0x95,0xe3,0xf6, +0x85,0x1a,0x87,0x35,0x7e,0xba,0x10,0x9f,0x17,0x5d,0xe2,0x4b, +0x76,0x8b,0xf7,0x17,0xe7,0xd3,0xc1,0x24,0x4e,0xf1,0xc9,0x3b, +0x0f,0x8f,0x26,0x8e,0xff,0xe1,0xfd,0xff,0xee,0x20,0xf9,0xef, +0xde,0xdd,0x55,0x3a,0xfe,0xbb,0xe7,0x4b,0x97,0x94,0x0b,0x7a, +0xf9,0xce,0x65,0x74,0xbb,0xac,0xfd,0xed,0x0b,0x34,0x1d,0xc8, +0x97,0x1e,0x6a,0x06,0x88,0xe7,0xb3,0xf1,0x6c,0x51,0x52,0xb8, +0xc6,0x5e,0xd0,0x4c,0x0b,0x59,0xd3,0xcd,0x24,0x65,0xab,0xba, +0x50,0x19,0x25,0xb7,0xcb,0xba,0xb0,0x5d,0xd6,0x5b,0x3e,0xf1, +0x60,0xd3,0xcf,0xf5,0x1a,0x61,0x89,0x4a,0x30,0xab,0x3e,0x8b, +0x07,0x69,0x7c,0x3e,0xe2,0xb7,0x88,0xcb,0x3d,0xbb,0x94,0x37, +0x17,0x9d,0x8f,0x21,0x78,0x89,0xdf,0x96,0x8d,0x85,0xf6,0x59, +0x74,0x22,0x7e,0x90,0x30,0x97,0x3c,0x71,0xa8,0x60,0x30,0x7b, +0x23,0x86,0xab,0x8b,0x64,0xb8,0x05,0x8a,0xf2,0x37,0x78,0xc1, +0x72,0x0b,0x50,0x8a,0xf7,0x80,0xe3,0xf5,0x7c,0x0b,0x08,0xe4, +0xaa,0xeb,0xaf,0xdb,0x6a,0x23,0x00,0xc0,0x85,0xef,0x0d,0x03, +0x5d,0xba,0x34,0x7f,0x8b,0x8b,0xc1,0xf0,0x5a,0x27,0xe2,0x07, +0x3d,0x8a,0x30,0xdc,0x86,0x92,0x01,0xe4,0x4b,0xa1,0x94,0x32, +0x81,0x69,0x57,0x73,0xb3,0x96,0xb0,0x27,0x97,0x8e,0xc3,0x31, +0x5b,0x2d,0x75,0x0d,0xf2,0x3a,0x9d,0x00,0xa0,0x06,0xa4,0xd7, +0xac,0x0c,0x7a,0x9d,0x47,0xd4,0x20,0xb5,0x81,0x18,0xec,0xc9, +0x36,0x47,0x80,0x84,0xf5,0x72,0x48,0x01,0x12,0xdf,0x2f,0xc5, +0xd1,0x47,0xdd,0xab,0x31,0xe0,0x30,0x51,0x0f,0xf9,0xe3,0x6f, +0x5e,0xdf,0xe2,0xbb,0xcb,0x8d,0x1a,0x06,0xaa,0x45,0x37,0x82, +0x2e,0x65,0xb8,0x8f,0x2d,0xb7,0x31,0x4d,0x4c,0x4a,0xf3,0xf1, +0x4d,0xe5,0x36,0x36,0x55,0x87,0xde,0x8a,0x24,0x39,0xfc,0x54, +0x36,0xbf,0x93,0xed,0xc8,0xd3,0x24,0xb2,0xf9,0x45,0x60,0x13, +0x06,0xd3,0xac,0xd2,0xbd,0x36,0x1e,0x4c,0xe6,0xde,0xc4,0x53, +0xb7,0x53,0x68,0xfa,0xe6,0x81,0xa6,0x0f,0xda,0xf9,0x8c,0x94, +0x17,0x97,0x5a,0xa7,0x8b,0x90,0x5e,0x71,0xcd,0x8d,0x9e,0x1f, +0x86,0xe3,0xdc,0xd8,0x81,0x94,0x3c,0x4c,0x5a,0x80,0x49,0xf3, +0x30,0x3f,0x0e,0xf2,0x3a,0x1b,0x52,0xf2,0x7a,0x9d,0x5e,0x51, +0xc9,0xe9,0x0c,0x4c,0x43,0xb8,0x92,0xc8,0x69,0xcc,0x4f,0xee, +0x1e,0x52,0xcb,0xa3,0xd9,0x90,0xda,0xd2,0x1c,0xe2,0xb5,0x92, +0xf8,0xbb,0x71,0x8c,0x5f,0x3f,0xbd,0x46,0x8f,0xd2,0x69,0xda, +0x54,0xef,0x6f,0xdf,0xbc,0x17,0xf8,0x7c,0xba,0x52,0x8f,0xdb, +0x91,0x6b,0x23,0x5f,0x25,0xbc,0x6f,0xa6,0xf1,0xf2,0xd9,0x12, +0xbe,0x2e,0x56,0xcb,0xd8,0xab,0xe9,0xb2,0x35,0x21,0x41,0x3a, +0xac,0x86,0xdf,0x1b,0xb4,0xcd,0x0b,0xe8,0x9a,0x5f,0x07,0x63, +0xf4,0x60,0x4f,0x67,0xe3,0x64,0x44,0xd1,0x3b,0xd5,0x26,0x99, +0x0c,0xf8,0x62,0xe2,0xb8,0x2d,0xbb,0x4b,0x18,0x10,0x80,0x6c, +0xdd,0xb6,0xd3,0x5f,0x68,0xa7,0x52,0x5f,0x91,0x47,0x1a,0xa9, +0x24,0x4c,0x9b,0x7e,0xa5,0xb8,0xa8,0xd0,0x6a,0x26,0xe8,0x65, +0xad,0xce,0xcf,0xb0,0xaa,0x84,0x7a,0xcd,0x97,0x81,0x11,0x64, +0x0e,0x7f,0x41,0x72,0x7a,0x1d,0xdf,0xbe,0x55,0xa9,0xf8,0x81, +0x69,0x14,0xac,0x52,0xa5,0xe1,0x07,0x24,0xd6,0xf4,0xfe,0x40, +0x81,0xec,0xe8,0xd3,0xa0,0x1d,0x8a,0x8b,0x76,0x20,0x86,0xf0, +0xdf,0x08,0x7e,0xc7,0xf0,0xf7,0xb2,0x1d,0x6c,0x72,0xb3,0xde, +0x1b,0xdd,0x1b,0xae,0x2c,0xe8,0x74,0x94,0x87,0xf1,0x20,0x83, +0xf1,0x2e,0x1d,0x73,0xf9,0xa3,0xc9,0xae,0xa4,0x25,0x3a,0x5c, +0xa7,0xd0,0x06,0x92,0x71,0x5b,0xc0,0x40,0x33,0xb4,0xee,0x92, +0x9b,0x35,0xbc,0x1d,0xe5,0xee,0xd6,0x10,0x8c,0x3e,0x55,0xe7, +0x9a,0xce,0xb9,0xa6,0x57,0x18,0xd9,0x8d,0xca,0xe0,0x1e,0xb8, +0x6f,0x07,0x1b,0x4b,0x37,0x36,0x61,0xc3,0xab,0xd9,0x62,0x54, +0x7a,0xd1,0x6e,0x11,0x33,0x88,0x8c,0xb8,0x71,0x0d,0xeb,0xd4, +0x75,0x40,0xee,0x92,0xa0,0x30,0xe8,0xe5,0xe0,0x14,0xdb,0x48, +0xbf,0x5e,0xaf,0x26,0x44,0x2c,0x7d,0xd0,0x03,0xf0,0xe6,0xe8, +0x69,0xea,0xeb,0x02,0x9c,0x23,0xdd,0x9d,0xa8,0x66,0x53,0x8a, +0x7e,0x5d,0x93,0xcf,0x1c,0xf9,0x37,0x18,0x27,0x8a,0x35,0xf9, +0x83,0x49,0xa7,0x07,0x48,0x5c,0xd7,0x23,0x16,0x41,0x3c,0x2d, +0xfb,0xd0,0xef,0x68,0x0a,0xe4,0x6a,0x2c,0x57,0x9f,0x62,0x90, +0x26,0xc7,0x17,0xd7,0xf5,0x68,0xcd,0x8e,0x67,0xb3,0xc5,0xf2, +0x5f,0xa9,0xf6,0xc3,0x43,0xab,0x44,0xf5,0xe9,0x98,0x29,0xe0, +0xe9,0x6a,0x7b,0x83,0xbe,0xe1,0x42,0xef,0x82,0x0e,0xaf,0x11, +0xee,0xb5,0x62,0x13,0x39,0x31,0x58,0xe8,0x0b,0x67,0x5a,0x74, +0x84,0x36,0xda,0x51,0xad,0x46,0xe7,0x99,0xd6,0x43,0xcd,0xe6, +0x83,0x6a,0x86,0xba,0xaf,0x23,0xef,0x8f,0xff,0xd5,0x90,0x0f, +0xa9,0x1f,0x4d,0x31,0x0e,0xf9,0xba,0x94,0xaf,0x41,0xb4,0x76, +0x18,0xcb,0xaf,0x8c,0x25,0x91,0x21,0x95,0x56,0xf2,0x1f,0x22, +0x87,0xfc,0xde,0x88,0xd8,0x2e,0x6e,0x54,0x2f,0xe0,0xf7,0x08, +0x12,0x06,0x88,0x6f,0x00,0xb6,0x76,0x3d,0xba,0x39,0xba,0xee, +0x68,0x29,0x01,0x80,0x3a,0x4c,0x4a,0x75,0x80,0x89,0x3e,0xb1, +0xef,0xe5,0x28,0xc1,0x3e,0x92,0xbf,0x3f,0xa0,0x47,0xdd,0x62, +0xf9,0x8c,0xbc,0x35,0x07,0x81,0x88,0xa7,0x23,0xf9,0x3b,0x64, +0xb7,0xe4,0xf6,0xcd,0x66,0xa3,0x51,0x39,0x38,0x4c,0xb9,0xb5, +0x55,0x6e,0x2d,0x8b,0x79,0x6b,0x3c,0xc2,0x3a,0xbe,0xde,0xa0, +0xbf,0x91,0xe4,0xc7,0x86,0x77,0x0b,0x0c,0x2b,0xf0,0xfb,0x43, +0x94,0xe4,0xb8,0x20,0xaf,0x3c,0x9a,0x36,0x70,0x13,0x3e,0xa8, +0xcd,0x5f,0x2b,0xe7,0x03,0xe5,0x24,0xfd,0x8e,0xbc,0xe3,0xd7, +0xa4,0xca,0xf5,0x4d,0x4c,0xf6,0xb6,0x3e,0x3c,0x64,0x71,0x97, +0x6e,0x2c,0x16,0xe0,0xd7,0x36,0x5c,0x57,0x6d,0x35,0x73,0xa2, +0xda,0x69,0x96,0x37,0x29,0xdb,0x2a,0xd7,0xb9,0x75,0xa8,0xee, +0x3c,0xfa,0x1b,0x94,0x92,0xe7,0x54,0xcd,0xe1,0xe1,0x22,0x26, +0x69,0x72,0x63,0x1c,0x52,0xca,0x27,0x49,0xc9,0x76,0x69,0x63, +0x3c,0x9e,0x37,0x68,0xda,0xc4,0xd6,0x07,0x4d,0xa7,0x59,0xc7, +0x2d,0xe1,0x5d,0xb8,0x10,0x17,0x79,0x08,0x14,0x4a,0x5a,0x9f, +0x13,0x46,0x39,0xd8,0xa5,0xd8,0x8a,0xa9,0x30,0x23,0xca,0x1d, +0x34,0xc2,0x50,0x21,0x58,0x49,0x7d,0xc2,0xf0,0x5b,0xd4,0x89, +0x51,0xa0,0xef,0xa1,0x61,0x8e,0x9c,0x91,0xf6,0x8b,0x2a,0x33, +0x8d,0xe4,0xf8,0x89,0x70,0x3b,0x96,0x7f,0xea,0x5d,0x2d,0x56, +0x49,0x52,0x1f,0x91,0x7d,0x42,0x49,0x7e,0x9b,0xe1,0x36,0xfc, +0xd9,0x54,0x64,0xec,0x55,0xa1,0x02,0x5e,0x6f,0x41,0xaf,0x84, +0x53,0xe2,0x37,0x3c,0xd9,0xaf,0x0a,0x0b,0x7e,0x5b,0x2d,0x06, +0xcc,0xae,0x48,0x73,0x7c,0xff,0xba,0x4c,0x11,0x5d,0x5d,0xbe, +0x1e,0x0d,0x62,0x57,0xc5,0xdd,0xb9,0x7f,0x3d,0x12,0x5e,0x55, +0x62,0xa4,0xda,0xae,0x89,0x81,0x54,0x35,0xc3,0x5c,0x15,0x26, +0x62,0x16,0x65,0xdc,0xdd,0x99,0x69,0x4d,0x62,0x55,0x25,0xf3, +0x4c,0xd0,0x25,0x39,0xc3,0x29,0xe9,0x34,0xcd,0x9e,0x4e,0x41, +0x9b,0x0c,0xcb,0x03,0xc6,0x2c,0xe2,0xf9,0x6a,0x0c,0xf6,0x3c, +0x6e,0x1f,0x17,0x67,0x7f,0x5c,0x65,0xe1,0x26,0x30,0xee,0x88, +0x91,0x65,0x87,0x09,0xbc,0x55,0x7e,0x10,0x45,0x5c,0x46,0x6e, +0x2b,0x52,0xce,0x70,0xdd,0xc0,0xc4,0x26,0xc6,0x1d,0x90,0x29, +0x19,0xa7,0x80,0xbd,0x3d,0x8d,0x42,0xeb,0x29,0xa7,0xd1,0xfa, +0x68,0xb4,0xae,0x8f,0xb2,0xa3,0x11,0x6f,0x0c,0x2f,0xaf,0xe2, +0xe5,0x40,0x9f,0xe7,0x8f,0xa6,0xea,0xad,0x0b,0xc6,0x72,0x85, +0x83,0x17,0x52,0xe1,0xff,0x6a,0x8c,0x11,0xda,0xf9,0xba,0x11, +0x8d,0xd0,0x1b,0x93,0xbf,0x32,0xf8,0xca,0xe0,0xeb,0x20,0xd8, +0x38,0xc4,0x1e,0x1e,0x26,0xe9,0xf7,0xc9,0x34,0x01,0x43,0x0d, +0x30,0x3b,0xa8,0x29,0xff,0xb9,0x8d,0x7f,0x0b,0xe2,0x8d,0xb4, +0x40,0x0f,0x2c,0xaa,0xac,0xa8,0x39,0xa3,0xc5,0xe0,0xfd,0x64, +0x76,0x13,0xe3,0xfe,0x27,0x60,0x26,0x6b,0x03,0xd6,0x4b,0x60, +0x5f,0x03,0x43,0x00,0x89,0xf9,0xce,0x04,0x75,0x09,0xc5,0x7c, +0x9d,0xa0,0x65,0x4b,0x5c,0x84,0xe2,0x62,0x30,0x9e,0x5f,0x0d, +0x84,0xba,0x67,0x9f,0xd2,0xdb,0x39,0x28,0x7a,0xa8,0x6e,0xb0, +0xba,0x94,0x4b,0xa2,0xda,0x21,0x54,0x88,0x13,0xa0,0xc1,0x84, +0x82,0xa9,0xbb,0x46,0x53,0x4f,0x4d,0xd4,0x60,0xb9,0x79,0x0d, +0x7f,0x60,0xee,0xa9,0x61,0x7c,0xa1,0x8f,0x71,0xd4,0x0b,0x45, +0xd8,0x17,0x97,0x8b,0x84,0x48,0x8d,0x9a,0x4f,0xe9,0x6c,0xf0, +0x5b,0x75,0xe3,0xdf,0x58,0x6a,0x84,0xfe,0x47,0x2b,0x8f,0x00, +0x5f,0x4b,0x32,0xca,0x00,0x55,0x9e,0xa4,0x30,0x6a,0x9c,0x04, +0x78,0x46,0x7c,0x83,0x26,0x6c,0x33,0x14,0xd4,0xb3,0x51,0xf3, +0x09,0xb1,0x92,0xec,0x29,0x75,0x36,0xac,0xba,0x92,0x79,0x81, +0x34,0xdb,0x32,0x0a,0x3d,0xe8,0x11,0x37,0x8e,0x80,0xda,0xa7, +0x78,0x9e,0x14,0x3c,0x52,0x47,0x90,0xcc,0x45,0x68,0x9e,0x27, +0xe3,0x2c,0x60,0x4b,0x99,0x77,0x6d,0xfa,0x37,0x0a,0xc0,0xe4, +0x38,0x08,0xc8,0xb6,0xfe,0x1d,0xcc,0xc3,0x99,0x48,0xc5,0x52, +0x8c,0xc5,0x35,0x9d,0x6b,0x4c,0x49,0x76,0xb5,0x8f,0xf1,0xc4, +0xb5,0x63,0xd5,0xce,0xfd,0x44,0xee,0xdc,0xcf,0x22,0x65,0xb2, +0x8a,0x34,0x9a,0x35,0xd5,0x34,0x07,0x3f,0xe5,0x7c,0xb8,0x8e, +0xa0,0x87,0x1b,0x29,0xf4,0x72,0x06,0xbf,0x32,0xf8,0x95,0x09, +0x6f,0xac,0x9e,0x3e,0xc3,0xfb,0x25,0xe3,0x88,0x5b,0xa2,0x7b, +0x13,0x90,0x1d,0x79,0x90,0x6c,0x06,0xc4,0xd8,0xf7,0x1b,0xba, +0xdb,0xd1,0x3c,0x3e,0x1e,0x8b,0xf5,0x51,0x34,0x16,0x19,0xfe, +0x83,0x15,0x00,0x42,0xef,0x3a,0x4a,0x9b,0xb7,0x31,0x1e,0xe5, +0x1c,0xc3,0x8a,0x95,0x7f,0xd5,0x55,0x12,0xd8,0x90,0x58,0x7d, +0x84,0x43,0x00,0xa8,0xa9,0x73,0x81,0xb0,0x71,0x0d,0x32,0xd0, +0xcc,0xea,0x98,0x4e,0x63,0x0d,0x12,0x99,0x1c,0xd9,0x4b,0x3e, +0x3d,0xf3,0x86,0x62,0xd2,0x0b,0xfa,0xc7,0xf8,0x14,0x1d,0xfd, +0x0e,0xf1,0x37,0x19,0xfa,0xd7,0xbe,0x6f,0x6f,0xa3,0xcf,0x22, +0x75,0xda,0x24,0x66,0x58,0x0d,0x58,0x34,0xf0,0xd7,0x87,0x5a, +0x67,0x58,0x8d,0x97,0xc1,0x67,0x06,0x9f,0x58,0x17,0xcb,0x04, +0x15,0xcf,0x89,0xce,0xb3,0x21,0x2c,0x3e,0x57,0xb4,0xae,0xfa, +0x3d,0xca,0x9f,0x7c,0x92,0x42,0x42,0x17,0x66,0x1a,0x0e,0x52, +0xf4,0x7d,0xc7,0x5a,0xf4,0x0c,0x19,0x18,0xb7,0x69,0x1d,0x8f, +0xee,0xee,0x7e,0xe7,0xe3,0x03,0x4f,0xa9,0xb5,0x99,0xaf,0x3c, +0x6d,0x4c,0x39,0x87,0x7a,0x2a,0xd7,0xf5,0x80,0x7c,0xe8,0xce, +0xf9,0x1a,0x1b,0x80,0x3f,0x32,0xbf,0x8d,0x69,0x8d,0xc8,0xc3, +0xd4,0x06,0xfd,0x1b,0x61,0x1b,0xfd,0x23,0x35,0x84,0x10,0x94, +0xf3,0x33,0xca,0xc7,0x72,0x99,0x95,0xef,0x77,0x58,0x48,0x51, +0xac,0x95,0x94,0xf2,0xb0,0xb4,0xc4,0x14,0x0f,0x8a,0x59,0xfa, +0x79,0x70,0xec,0x67,0x2b,0x10,0xe8,0x9a,0x0b,0xfa,0x6d,0xfa, +0x54,0x68,0x72,0xfe,0x1b,0xdb,0xd0,0x30,0xa8,0x46,0x23,0x5d, +0x05,0x18,0x8d,0xbb,0xa1,0xbd,0x75,0x42,0x44,0x48,0x8d,0x04, +0xbf,0x6c,0x52,0xbe,0x2d,0x44,0x14,0xd9,0x45,0x91,0x2e,0x51, +0x12,0x59,0x63,0xdd,0x5d,0xb7,0xeb,0x0e,0xc1,0x0a,0x5a,0xd5, +0x69,0x22,0x98,0x14,0x48,0xb0,0xa8,0xd2,0xca,0x6c,0x6f,0xaa, +0x74,0x89,0xbd,0xa8,0x52,0xd0,0x8a,0x2a,0xad,0x75,0xf7,0xaa, +0x4f,0x43,0x1b,0xa4,0x2a,0x49,0x21,0x94,0x8a,0x76,0x2f,0x74, +0x12,0x76,0x27,0xe1,0x72,0x36,0x93,0x35,0x28,0xfd,0xbd,0x57, +0x15,0x0a,0xd8,0x20,0x93,0x29,0x0a,0x1b,0x4f,0x01,0x7b,0xe1, +0x62,0x50,0x83,0x89,0xbe,0x15,0x1e,0x56,0xed,0x7b,0xe1,0x59, +0x23,0x0e,0x82,0xef,0x72,0xa9,0xf5,0x59,0x00,0xed,0x0d,0xda, +0x6b,0xf2,0xef,0xe3,0xb1,0x49,0xd3,0xa5,0x1a,0x9c,0x72,0xee, +0xb4,0x27,0x91,0x35,0x6d,0xbc,0x36,0xe9,0xc9,0x40,0xcf,0xcc, +0x52,0xbe,0xaf,0xa8,0xe3,0x41,0xac,0x06,0x0d,0x3d,0x87,0x50, +0x66,0x65,0x91,0xb3,0x07,0x2d,0xa9,0x01,0xd3,0x14,0xe3,0xbe, +0xd0,0xa4,0x6c,0xf6,0x55,0x60,0xc9,0x81,0x1a,0xfc,0x02,0x8f, +0x28,0xd4,0x2f,0x2f,0xf1,0x69,0x1d,0x2c,0x26,0x3a,0xc9,0xda, +0x6a,0xa1,0x8b,0x16,0x96,0x13,0x87,0x81,0xc1,0x3b,0x73,0xba, +0x9e,0xbe,0xaf,0xa6,0xcc,0xb5,0x13,0xc3,0x70,0x31,0x98,0x8e, +0x66,0x13,0xcf,0x3f,0xa2,0x11,0xab,0x09,0xd5,0x75,0xb3,0x83, +0x88,0xc6,0xc9,0x94,0x1a,0x1a,0x7b,0x7d,0xba,0x0b,0xa7,0xae, +0xad,0xd9,0x75,0xab,0x8d,0x94,0x0f,0x34,0x83,0x72,0x3e,0x19, +0x5c,0x6a,0x1a,0xfd,0xd0,0xef,0x18,0x78,0x35,0x9d,0x3a,0xfe, +0xcb,0x6a,0x62,0xf5,0x85,0x0d,0x28,0x17,0x70,0x2e,0x20,0x97, +0xf6,0x37,0x66,0x6b,0x50,0x15,0x48,0xfa,0x64,0x54,0xe1,0xce, +0x90,0x4e,0x04,0x33,0x60,0xeb,0xd4,0x2f,0x6e,0xd5,0x2c,0x28, +0xae,0xd4,0x1c,0x58,0xf0,0xd4,0x74,0xe6,0x1d,0xbe,0x4a,0x08, +0xf6,0x85,0x9c,0x82,0x01,0x6e,0x8b,0xf9,0x50,0xe3,0x23,0x2e, +0x2b,0x2c,0xac,0x6c,0x00,0x48,0xa4,0xfa,0x29,0x91,0xab,0xcf, +0xbe,0x5f,0x56,0x8a,0x79,0x41,0xa5,0xe4,0xca,0x5f,0x95,0xe2, +0x4f,0x28,0x55,0xaf,0x6b,0xde,0x32,0x69,0x94,0x22,0x99,0xc8, +0x29,0x85,0xa6,0x59,0x53,0x23,0x8b,0x15,0x4e,0x79,0x54,0x0b, +0xba,0x86,0x4b,0x19,0xae,0xad,0x6b,0xe2,0x16,0x86,0x80,0x82, +0xc8,0x18,0x22,0xb3,0x20,0xb2,0x9a,0xb8,0xb2,0x20,0xe6,0x12, +0x89,0x9c,0x43,0x4d,0xba,0x2c,0x2a,0xe7,0x4e,0xba,0x14,0xa0, +0x2c,0x20,0x14,0xb1,0x12,0x9d,0x65,0x6b,0x74,0x3f,0xcf,0x69, +0xdb,0x7c,0x8a,0xea,0x36,0xa8,0xe5,0x17,0xa6,0x3b,0x23,0xc9, +0x39,0x6a,0x94,0x62,0xb1,0x91,0x20,0x7d,0xda,0x80,0xdb,0x42, +0x9f,0x52,0xff,0x05,0xfa,0x6c,0xeb,0x8f,0xe9,0x53,0xa0,0xf7, +0xa1,0xcf,0xc1,0x62,0x23,0x31,0xe6,0x56,0x15,0x75,0x96,0x31, +0xe6,0x78,0x2a,0x73,0x21,0xa2,0x8a,0x7f,0x5b,0xf4,0x18,0x89, +0xa8,0x70,0x6c,0xb1,0x4a,0xf3,0x4f,0xd7,0xac,0xd7,0x4b,0x1c, +0x67,0xc9,0x53,0xb6,0xe7,0x6c,0x54,0xbc,0xd7,0x0c,0x7d,0xa3, +0x55,0x67,0xf3,0x1d,0xd0,0x81,0x06,0xc6,0x25,0x54,0xd9,0x91, +0x24,0x24,0xdf,0xdd,0x79,0x94,0x0b,0xfa,0xfc,0x22,0xbe,0x82, +0xe9,0x69,0xb6,0x20,0x70,0xcf,0x97,0x91,0x84,0x9d,0x30,0x14, +0x4d,0x14,0x63,0xcc,0x26,0xa5,0xce,0x0b,0xe8,0x9a,0xc8,0x59, +0xb0,0xdf,0xaa,0x7c,0x03,0xae,0x21,0xe5,0x52,0xd0,0xe4,0xc0, +0x0a,0x65,0x0b,0x1a,0xc8,0x85,0x01,0x53,0x36,0x1d,0x26,0x29, +0xa1,0x98,0xcc,0x56,0x69,0x0c,0x08,0x17,0x15,0x48,0x5e,0xaa, +0x7c,0xdf,0x02,0x5f,0x55,0x51,0xfe,0x52,0x66,0xfb,0xdc,0xd1, +0x48,0x9f,0x2f,0x6e,0x66,0xc9,0xe8,0x41,0xe0,0xb7,0xf1,0x8b, +0x76,0x0f,0x16,0x31,0x6e,0x86,0xf2,0xc4,0xc7,0xab,0x4d,0x51, +0x03,0x71,0x32,0x2e,0x8d,0x55,0x4b,0xc6,0xa8,0x15,0xe4,0x6b, +0xb4,0x17,0x8a,0x51,0xd8,0x31,0x7b,0x13,0x57,0x49,0xbc,0x18, +0x2c,0x86,0x57,0x59,0xc5,0xfe,0xc4,0x70,0xb5,0x90,0xfb,0x13, +0x62,0x14,0xcf,0x41,0x47,0xf3,0xca,0x80,0x6f,0xac,0xa0,0xdf, +0x5d,0x1a,0xd1,0x1f,0x40,0xcd,0x8d,0xd1,0x08,0x85,0x29,0xc4, +0xbe,0x67,0xb8,0x94,0xa7,0xcf,0xc8,0xc2,0xc4,0x33,0x09,0xed, +0x65,0x08,0xc6,0x07,0x3a,0x69,0xca,0x38,0xb5,0x07,0x8e,0x99, +0xa9,0xd1,0x35,0x8e,0xd6,0x44,0x43,0xd2,0x95,0x4d,0x55,0x37, +0x8e,0xb8,0x1b,0x3a,0x12,0x20,0xdc,0x75,0xbd,0xdc,0x18,0x45, +0xaa,0x0d,0x8c,0x12,0x87,0xd2,0x07,0xd9,0x08,0xdc,0x15,0x18, +0x2c,0x70,0x15,0x4f,0xa4,0x0e,0xd5,0xd5,0x45,0x71,0x53,0x8f, +0x46,0xbc,0x55,0xd9,0xc1,0xcd,0xa4,0xc3,0xc3,0x21,0x6f,0x8c, +0xe2,0x3f,0x3e,0xef,0x23,0x23,0x95,0x48,0x00,0x7d,0x44,0x37, +0xd2,0xa9,0xb1,0x24,0xab,0x2e,0x83,0x68,0x54,0xf2,0xe6,0xee, +0x2e,0xf0,0xed,0x5d,0x15,0x7b,0x73,0x96,0xca,0x5a,0xdc,0xb7, +0xf8,0x8e,0x8d,0x76,0x58,0x80,0xed,0x67,0x1d,0xc4,0xdf,0x86, +0x8f,0xd8,0x37,0x8a,0x93,0x86,0x91,0xc4,0xc6,0x02,0xbf,0xa0, +0xe9,0xaa,0x5a,0x55,0x96,0x58,0x26,0xb5,0x8f,0x6a,0xe0,0xcd, +0xfd,0xda,0x55,0xca,0x32,0x71,0x63,0x9a,0xaa,0x31,0xe0,0xe6, +0x0d,0xd1,0x27,0x77,0x2b,0x14,0x06,0xd5,0x8b,0x23,0x11,0xa8, +0xde,0xe3,0x95,0x17,0x6f,0xcc,0x73,0x0c,0x79,0x29,0xf1,0x1a, +0xd9,0x6b,0x48,0x17,0x9a,0x5b,0x25,0x00,0xcf,0x35,0xef,0x88, +0xa6,0x12,0x88,0x5f,0x49,0x0a,0x24,0x11,0x3a,0x95,0xc4,0x61, +0xff,0x2d,0x4b,0x58,0xa1,0xe9,0xa2,0xbc,0x3f,0xb9,0x31,0x09, +0x46,0x86,0xf7,0x5c,0x50,0x48,0x68,0x07,0xa7,0x4a,0xb5,0xf1, +0x72,0x9b,0xf6,0x42,0xca,0xa0,0x0e,0x46,0x4a,0xb2,0xd1,0x49, +0xb1,0xb0,0x5f,0x71,0x98,0x19,0xff,0x6b,0x25,0x34,0xe4,0xdc, +0x17,0xf8,0xec,0x81,0x67,0x4a,0xdb,0xdb,0xa1,0x30,0xe6,0x96, +0x89,0xbb,0x20,0x2b,0x33,0xd6,0x49,0x94,0xd6,0x82,0xa2,0xa8, +0x6e,0x93,0x7b,0xad,0x5b,0xc8,0x9f,0x98,0x76,0x3b,0x23,0xa3, +0x6b,0x30,0xb8,0x34,0x7f,0xad,0x31,0xf4,0x33,0xff,0xcc,0x22, +0x48,0xdd,0x3e,0x4e,0x58,0x08,0xe9,0x72,0x38,0xed,0x5e,0xd0, +0x0e,0xca,0x3a,0x32,0x02,0xdc,0x1d,0xad,0x8f,0xcd,0x57,0x5b, +0xdf,0xd8,0xd7,0xf4,0x0f,0x23,0x7b,0x00,0x41,0x2b,0xa2,0x21, +0xc3,0x1e,0x71,0x93,0xf0,0xfc,0x69,0x64,0x6f,0x5f,0x22,0xb9, +0x9e,0xd9,0xd5,0xad,0x18,0xe5,0xa3,0xfb,0x8f,0x72,0xa3,0x0b, +0x69,0xf1,0x31,0x19,0xa0,0x37,0x28,0x57,0x67,0x91,0x68,0x0e, +0x84,0xc3,0xfa,0xc8,0x90,0xa5,0x7b,0x8b,0x0f,0x24,0xcd,0xd8, +0xb4,0x04,0xd8,0xf1,0xfa,0x37,0x37,0xa6,0xac,0x8e,0xa4,0x67, +0xaf,0x03,0xa1,0xcc,0x7b,0xb5,0xc1,0x65,0xda,0x8c,0x00,0xbe, +0x3d,0xa6,0xcd,0xc4,0x54,0x32,0x59,0x79,0xf6,0xb6,0xaa,0x39, +0xc7,0x96,0x94,0xde,0x7f,0xff,0x44,0x17,0x55,0x7b,0x28,0x25, +0x9a,0xe0,0x17,0x9e,0x8e,0x35,0xa8,0x35,0x58,0x1c,0xe9,0x4e, +0xca,0xb7,0xfa,0x21,0xdd,0xba,0xcf,0x43,0x82,0x50,0x76,0xe7, +0xd8,0xba,0x38,0x69,0x6b,0x58,0x28,0x4d,0xcc,0xa5,0x30,0x33, +0xf5,0xb2,0x67,0xd8,0xcc,0xb1,0x67,0xd7,0xfc,0xac,0x78,0xea, +0xc2,0x00,0xd0,0xc1,0x70,0x19,0x3a,0x75,0x6e,0xda,0x55,0x3f, +0x2a,0x50,0xa9,0x6c,0xbf,0x31,0xc0,0xc0,0x68,0x4d,0xb0,0x2e, +0x59,0x9b,0xa4,0x32,0xa4,0x8e,0x39,0xf3,0xb7,0x5c,0xc5,0xfd, +0x8e,0x74,0x52,0xa1,0xd9,0x35,0x31,0xe7,0xeb,0xa4,0x2d,0x23, +0x4b,0x11,0x43,0xc3,0x51,0x8b,0x7f,0xc3,0x8a,0xb8,0xab,0xf9, +0x04,0x0b,0x4b,0xdd,0xad,0x5c,0x1f,0xac,0x6c,0x1b,0xf2,0x17, +0x2c,0x3e,0xdb,0xa5,0x90,0x54,0x87,0xf6,0x46,0xc7,0xbf,0x1f, +0xfa,0x2a,0x88,0x15,0x74,0x65,0xea,0xbc,0xd5,0x51,0x7a,0x2a, +0x2f,0x87,0xc0,0xa8,0x83,0xe0,0x68,0x74,0x7f,0x42,0x34,0x6d, +0x85,0x93,0x95,0xc1,0x28,0xd2,0x84,0x38,0xc7,0xd8,0xd6,0x29, +0x36,0x8c,0x7d,0x3c,0xd0,0x40,0xa3,0x73,0xc8,0x02,0xcf,0xda, +0x55,0x7a,0x72,0xe6,0x66,0x34,0x97,0x0b,0x16,0xce,0xc8,0x9c, +0x70,0x47,0x7f,0xfc,0x2f,0x3d,0x12,0x92,0xf8,0x73,0xb4,0x3f, +0x14,0xd3,0x7a,0x1f,0x51,0xdc,0x7b,0x92,0x23,0x04,0x3c,0xbd, +0x51,0x79,0x43,0xe6,0x7e,0x58,0x0c,0xbc,0xc2,0xa5,0x53,0x18, +0xa3,0x6e,0xeb,0x5e,0xf8,0x34,0xb4,0xc4,0xa6,0xbe,0x09,0x57, +0xde,0x8a,0x76,0x59,0x1c,0x7d,0xda,0x58,0x46,0x32,0x87,0xec, +0x2a,0x1b,0xd5,0x94,0x43,0x12,0xc5,0xd2,0x2e,0x1d,0x05,0xe2, +0x45,0xb2,0x6b,0x74,0x4b,0xa9,0xe5,0xe1,0x4d,0x68,0xd4,0x00, +0x97,0xf7,0xb7,0x18,0x49,0xd5,0xd5,0xe8,0x51,0xee,0x06,0xb8, +0x41,0xdc,0x5b,0xdb,0x38,0x6f,0xe8,0x8e,0x43,0x3e,0x85,0xef, +0x53,0xd3,0x35,0xec,0x34,0xe2,0x1b,0xdc,0x16,0x84,0xbc,0xed, +0xc4,0x0d,0xea,0xa8,0xc6,0x9c,0x34,0xe7,0xf1,0x62,0x82,0x3e, +0x64,0x9c,0x22,0x8b,0x6b,0x7a,0x2d,0x00,0x89,0x40,0x02,0x74, +0xd4,0x46,0xd0,0x4c,0xcc,0x2e,0x2f,0xd3,0x18,0x60,0xf9,0x6f, +0x65,0x9d,0x62,0xaa,0x5a,0xaf,0xf7,0x89,0xf8,0x1b,0x03,0x69, +0x59,0xc7,0x44,0xf6,0x26,0x17,0x7e,0x93,0xef,0x92,0xc1,0xa9, +0xcb,0xa0,0x67,0xca,0x4c,0x56,0x8a,0x7a,0x42,0xe8,0xb0,0x5c, +0x1c,0xe1,0xca,0x17,0x76,0xb8,0xb3,0x72,0x2c,0xec,0xdf,0x32, +0xab,0xeb,0x2b,0xe0,0x8d,0x50,0x96,0x16,0xe6,0x9a,0x3e,0x63, +0xd3,0xd7,0xdd,0xf0,0x36,0x9a,0xad,0xed,0x6d,0x57,0x51,0x79, +0x15,0x5f,0x89,0x1f,0xd5,0xf5,0x33,0x26,0x7e,0x2b,0x9d,0x28, +0x99,0xdc,0x02,0x08,0xa5,0xfe,0x16,0x2f,0x66,0xe8,0x6d,0x5a, +0xc8,0x5d,0xc9,0xbb,0x0e,0x76,0xa2,0xbc,0xa5,0x60,0x27,0xe9, +0xf8,0x4f,0x1c,0x09,0x4b,0xd2,0xb7,0xbf,0xb2,0xc0,0x83,0x0b, +0x2a,0x2a,0xf5,0x45,0xba,0xe1,0xcf,0x26,0x37,0x6b,0x2f,0x44, +0x0c,0x5a,0xb1,0x31,0x5e,0xc6,0x98,0x94,0x1c,0x49,0xd7,0xec, +0x48,0x5a,0xc9,0x36,0x49,0x16,0x61,0xd7,0x54,0x31,0x2b,0xf7, +0x23,0x8b,0x61,0xf7,0xa4,0x8b,0x45,0xaa,0x92,0x30,0xd3,0x59, +0x8a,0x2c,0x4a,0x51,0x74,0x59,0xde,0x29,0x1f,0xb7,0x6e,0xa6, +0x7f,0x54,0xe5,0xd7,0xaa,0x68,0xb6,0x5f,0xd1,0xcc,0x14,0xcd, +0x34,0x37,0x56,0xcb,0xfd,0x0a,0x23,0xa0,0x2e,0x0e,0x1f,0x12, +0x41,0x5e,0x73,0x5a,0x3d,0x64,0xbb,0x46,0x63,0xb4,0x1f,0xf6, +0x7c,0x2e,0xb9,0xcf,0x4c,0xfb,0xc2,0xce,0xe5,0x31,0xb0,0x4a, +0x8d,0xc2,0xcc,0x21,0x7f,0x39,0x58,0xbf,0x60,0xc5,0x90,0xa2, +0x4f,0x61,0x15,0xd8,0x2f,0xf1,0x68,0x35,0x8c,0x5f,0xaf,0x26, +0x05,0xd3,0x63,0xba,0x2d,0xb8,0x05,0xd4,0x8c,0xa1,0x2d,0xf0, +0xcf,0x05,0x85,0xf7,0x99,0xcd,0x31,0xfc,0x12,0xdf,0x06,0x0c, +0xf0,0x93,0x36,0xf4,0x38,0x21,0x35,0x81,0xd9,0xcc,0x1e,0xdc, +0x87,0x28,0x51,0x8e,0x71,0x0c,0x75,0x06,0x85,0xba,0x1e,0xfc, +0x53,0x8f,0x90,0x62,0xd4,0x1c,0x88,0x86,0x37,0x18,0x3e,0xe0, +0x4d,0x1f,0x86,0x33,0xd9,0x12,0xbb,0x86,0x50,0x03,0x54,0xa5, +0x9b,0xd7,0x59,0xe4,0xe3,0x2e,0x88,0x0f,0x2f,0x6d,0x71,0x7a, +0x9e,0xc7,0xe6,0xc2,0x6d,0xd1,0xf6,0x32,0xb8,0x36,0xa2,0x26, +0x5d,0xb6,0x6b,0xa5,0x23,0x4e,0x8e,0x29,0xf6,0x7f,0x2f,0x91, +0x7b,0xdd,0xdd,0x69,0x32,0xbe,0x9a,0xad,0xe2,0xe5,0xb2,0x40, +0x88,0xd1,0xfd,0xb9,0xee,0xe6,0x3b,0x78,0x5a,0x9f,0x73,0xcf, +0xf6,0xfa,0x24,0x07,0xf4,0x3c,0x49,0xd9,0x41,0x86,0x0a,0x7a, +0x39,0x93,0xcc,0xc7,0xa0,0x7e,0xa0,0x92,0xa5,0xf5,0xc5,0xea, +0xb7,0x33,0x3b,0x03,0x1c,0xb0,0x2e,0x42,0x4c,0x33,0x16,0x19, +0x79,0x5e,0xe1,0x6f,0xf2,0x18,0xb3,0x00,0xcf,0x4c,0x10,0xb4, +0x21,0x7b,0xc2,0x3f,0x6e,0x29,0xde,0x67,0xc1,0x46,0xdc,0x26, +0xef,0xdf,0x8f,0x2b,0x5b,0x05,0xb3,0x03,0x45,0xc5,0x49,0x4f, +0x70,0x9d,0x0a,0x2a,0x39,0xc8,0x35,0x73,0xad,0x9a,0x09,0x0d, +0xd6,0x1e,0x65,0xa6,0x71,0x19,0x46,0x09,0x89,0x60,0x6a,0x0a, +0x68,0xef,0x2a,0x2c,0xb6,0x34,0x0d,0x2d,0x39,0x4b,0xc3,0x7c, +0x5b,0x35,0x18,0xbd,0x75,0x0b,0x2b,0x60,0x19,0xd3,0x11,0xfe, +0x36,0xe8,0x76,0x95,0xb9,0x00,0x89,0xa0,0xe8,0x4a,0x9b,0x9e, +0x44,0x9e,0x83,0xa4,0xa1,0xbf,0x1a,0x7c,0x09,0xeb,0xd8,0x6b, +0xc1,0x1a,0xd5,0xef,0x24,0x67,0xd7,0x50,0xf4,0xda,0x4f,0x4f, +0xea,0xb2,0xc8,0xb5,0x53,0xe4,0xda,0x2a,0x32,0x5a,0x77,0xd2, +0x16,0x88,0xf3,0xc9,0x91,0x83,0x7b,0xc3,0x0c,0x9e,0x35,0x28, +0x56,0x4f,0xeb,0x38,0x0d,0x01,0x73,0x1b,0x3a,0x30,0x38,0x9b, +0xe1,0xe1,0x43,0x10,0x55,0x75,0x4a,0x03,0x98,0x62,0x77,0x44, +0xbc,0x9e,0x0f,0xec,0x37,0xae,0xef,0x29,0x5e,0xd7,0x51,0x78, +0x3c,0xfd,0x53,0x62,0x05,0x8b,0xee,0x99,0xb3,0x77,0x8f,0x50, +0x0e,0xc8,0x71,0x34,0x2b,0xee,0xd1,0x17,0xa0,0xa2,0xeb,0x0a, +0x29,0x74,0x9a,0xfb,0x11,0x26,0x8d,0xf2,0xd9,0x06,0xa7,0x13, +0xbe,0xfb,0xa1,0x57,0xc5,0xe9,0x12,0x03,0x61,0x4f,0x4a,0x2d, +0x54,0x9d,0x2b,0xad,0x54,0xb3,0x97,0x0a,0xcb,0x59,0xb1,0x16, +0xf0,0x2f,0x8d,0xbb,0xfc,0x8a,0x94,0xbe,0x17,0x82,0x43,0xd9, +0x91,0x12,0x89,0xe8,0xdf,0x85,0xb5,0xdc,0xe7,0x32,0x68,0x61, +0x2e,0xaf,0x16,0x71,0x7a,0x35,0xc3,0xed,0x60,0x40,0x38,0x75, +0xa0,0xa8,0x98,0x05,0x2b,0x9d,0xd7,0xa5,0xfd,0xab,0x3b,0xcc, +0xa0,0xd0,0xe1,0xc1,0xa0,0xd7,0x2e,0x17,0xf1,0xef,0xab,0x78, +0x3a,0xcc,0xba,0x61,0x1b,0x7a,0x90,0xb6,0x32,0x26,0x1d,0x1f, +0x2a,0xc1,0x8a,0x38,0x28,0x41,0x1f,0x1b,0x81,0x1b,0x3c,0x06, +0x45,0x2f,0xa9,0x83,0x94,0x7a,0x98,0xee,0x26,0x83,0xad,0x87, +0x89,0x19,0xef,0xa2,0x4c,0xce,0x02,0x3f,0xef,0xa0,0xb2,0xb6, +0x16,0x6f,0xeb,0x33,0x6e,0x34,0x08,0xd2,0xe1,0xe1,0xfa,0x6b, +0xf9,0x81,0x31,0x6d,0x3d,0x4d,0x01,0x9e,0x68,0x24,0x14,0xac, +0xce,0xd4,0x03,0x7c,0x0d,0xc5,0x84,0x22,0xf3,0x51,0x6d,0xf5, +0xe8,0x9a,0x7e,0x98,0xc0,0x9d,0xf6,0x7e,0x0b,0xa2,0x21,0x3b, +0x51,0x37,0x16,0x63,0x4d,0x70,0x07,0xa8,0x05,0x21,0xf3,0xde, +0xd9,0xa1,0x94,0xfd,0xfa,0x0b,0xb1,0x97,0xb9,0x5e,0x96,0xff, +0x4d,0x32,0x7d,0x0d,0xd5,0xbc,0x8f,0x53,0xb3,0x8d,0x29,0xb3, +0xee,0xbd,0x48,0x5c,0xd0,0x1e,0xa1,0x2c,0x2d,0x6d,0xbf,0xc5, +0xc6,0x24,0xf1,0x6c,0xb3,0x1f,0x42,0xd3,0x22,0xeb,0xd2,0xba, +0x85,0x9c,0xf3,0x6d,0xe4,0x24,0xaa,0x7b,0xe1,0x96,0xdc,0x28, +0x1c,0xdd,0xae,0xcd,0x06,0x02,0xa1,0xb7,0x2f,0x5c,0x95,0xf0, +0xed,0x7b,0x74,0x4f,0x62,0x48,0xb1,0xf6,0x37,0xed,0x4a,0x5a, +0xb9,0x3e,0x9b,0x56,0xd3,0x95,0x7b,0x7a,0x9d,0xe8,0x9e,0x3f, +0x70,0x58,0xac,0x33,0x2c,0xe4,0xf6,0x76,0x93,0x1b,0xa7,0xc2, +0x1a,0xf7,0x1c,0x91,0x82,0xc6,0xbb,0x4a,0xba,0x4c,0x16,0xe9, +0xf2,0x6f,0x83,0xf1,0x35,0xef,0xad,0xce,0xc1,0x0e,0x48,0x66, +0xab,0xf4,0x75,0x72,0x31,0xd6,0x17,0x93,0x2a,0x76,0x1d,0xe5, +0xf5,0x15,0x4a,0x3c,0x47,0xd4,0xbb,0xf7,0x20,0xad,0x33,0x1b, +0x55,0x13,0x6d,0xb7,0xf3,0xc6,0xab,0x20,0x62,0x28,0xc1,0xec, +0x93,0x06,0x18,0xe5,0x6c,0x18,0xa7,0x14,0x3b,0x5f,0xe7,0x3b, +0x7b,0x98,0x43,0xb7,0x04,0x8c,0x51,0xd3,0x2a,0x46,0xec,0x54, +0xe6,0x1b,0x84,0x83,0xf9,0x1c,0x4c,0x40,0xda,0x99,0x2d,0xc2, +0x69,0x30,0xdf,0x4d,0xe7,0x9a,0x3a,0x46,0x3c,0xb0,0xed,0xaf, +0xaf,0x92,0xcb,0x25,0xef,0xd4,0x92,0x29,0x3c,0x49,0x46,0x1c, +0xbf,0x02,0x03,0x4b,0x18,0xba,0x99,0x51,0x78,0x8d,0x6b,0x9c, +0x4c,0xea,0xc3,0x42,0x92,0xdf,0xc9,0xf5,0x40,0xd7,0x53,0x7b, +0x88,0x94,0x1f,0xe5,0xb2,0x5d,0x7c,0x69,0x3c,0x1f,0x2c,0x06, +0x66,0xa7,0x3c,0xdf,0x9b,0xb2,0xcb,0x9a,0x93,0xd9,0x28,0x72, +0xd0,0x36,0x14,0xb9,0x7e,0xdb,0xad,0x4e,0xa5,0xf3,0x51,0x56, +0x0e,0x1f,0x7a,0x51,0x7e,0x39,0xe2,0xac,0x1b,0x08,0xa0,0x39, +0x67,0xd3,0x91,0x91,0x4a,0x18,0x25,0x72,0x37,0xdf,0x08,0x9b, +0x42,0xbb,0xee,0x6c,0x3f,0x06,0xd8,0x6b,0x4b,0x5f,0x6f,0xe7, +0xaf,0xeb,0x76,0x15,0xc0,0x27,0x25,0x63,0x16,0x49,0xce,0x16, +0xbe,0xfd,0x44,0xa7,0x91,0xa6,0xb2,0x06,0x1a,0x71,0x42,0x47, +0x9e,0xc2,0x40,0x53,0x23,0x23,0x45,0x41,0x12,0x37,0xc9,0x9c, +0x0f,0x20,0x6f,0x66,0xea,0x47,0x91,0xbf,0x90,0x39,0x61,0x72, +0xf9,0xc8,0xb2,0x69,0x0f,0x9a,0x14,0x30,0x00,0x16,0xd3,0x12, +0x91,0x02,0x2a,0x40,0x67,0xa7,0x00,0x4e,0xc0,0xeb,0xc0,0x40, +0xca,0xcc,0x4a,0xe9,0x60,0xbd,0xae,0xa8,0xff,0x82,0x8e,0x2a, +0x1e,0xa4,0xfb,0x44,0xa6,0x9b,0xf9,0x63,0x7c,0x89,0x79,0x73, +0xcc,0x9b,0x1c,0x1e,0xc2,0xaf,0x8e,0x8f,0x64,0x96,0x41,0xcd, +0x10,0xc3,0x2c,0x8f,0x41,0xa2,0x9f,0xcd,0x29,0x53,0x52,0xa2, +0x47,0x2c,0xf1,0x82,0x78,0x64,0x51,0xae,0x24,0x0c,0x04,0xd9, +0x34,0x59,0x4a,0x36,0xf0,0xc1,0x16,0x3d,0x28,0x24,0x88,0x3c, +0xc2,0x41,0x8e,0x67,0x6e,0xf5,0x2f,0xc9,0x5b,0xdc,0x49,0x7a, +0x26,0x2b,0xa7,0xc2,0x44,0x80,0xd1,0x0c,0x86,0x1e,0x3c,0x7e, +0xc0,0xb5,0x1d,0xf5,0x5f,0x3a,0x53,0x3f,0x31,0x19,0xd6,0x74, +0x39,0x36,0x23,0x64,0xbe,0x77,0x10,0x6a,0xe6,0x76,0x06,0x42, +0xd9,0x3d,0xd6,0x21,0xa6,0x1e,0x54,0x31,0x0c,0x0f,0x61,0x35, +0x38,0x5a,0x1e,0x83,0x51,0x44,0x0d,0xb6,0x71,0x00,0x5d,0xc8, +0x16,0xe2,0x6f,0x32,0x2f,0x60,0xd3,0x5d,0x43,0xc8,0x26,0x79, +0x64,0x73,0xe1,0xd0,0x88,0xc8,0xe6,0x80,0x6c,0x22,0x9c,0x1e, +0xd2,0xf7,0x42,0x55,0xea,0x66,0xf7,0x19,0x11,0x1d,0x0f,0x46, +0xea,0xdc,0x27,0xa7,0x5e,0x29,0xc0,0x08,0xbd,0xe5,0xc1,0xa7, +0x89,0xe6,0x76,0x73,0xe9,0xac,0x65,0xc6,0x70,0xf4,0x49,0x91, +0xd0,0x56,0xa0,0x20,0x15,0xb0,0xda,0x00,0xe2,0xf1,0x1a,0xe7, +0x15,0x4e,0xe4,0xf0,0x83,0xfa,0x0a,0xfe,0xb2,0x75,0xd0,0xce, +0x6b,0xe1,0x72,0xcd,0xc6,0xc0,0xf5,0xb0,0x1d,0xe0,0xae,0xbd, +0x99,0x74,0xe8,0x14,0x54,0x58,0x2a,0x83,0x68,0x6e,0x50,0x90, +0x12,0x57,0xdb,0x53,0x3c,0x32,0xe0,0x78,0x6e,0x0c,0xbc,0x8e, +0x91,0x43,0x32,0x2a,0x4a,0xa1,0x7b,0x26,0xb3,0x74,0xa9,0xc2, +0x14,0xed,0x5b,0x8e,0x84,0x84,0x0b,0x8e,0xe2,0x38,0x3f,0xe6, +0xaa,0xcb,0x7d,0x0b,0xc0,0x31,0x96,0x5a,0x07,0x11,0x12,0x8a, +0x5e,0xf5,0x66,0x34,0x61,0x0a,0x13,0x82,0x37,0xbc,0xd6,0x61, +0x44,0xbf,0x9b,0x6b,0x7b,0xc4,0x51,0x92,0x40,0x48,0x84,0xc9, +0xc2,0x08,0xeb,0xe7,0x83,0xd8,0xbb,0xbb,0xb0,0x53,0x30,0xb6, +0xca,0xba,0x1b,0xbb,0xee,0x75,0xf2,0xd1,0x3a,0xeb,0xe1,0x33, +0x51,0x9e,0x15,0x8e,0xb4,0x8b,0x60,0xf1,0xa8,0x57,0x9e,0x2a, +0x42,0xa3,0xc7,0xf1,0x32,0x7e,0x60,0x04,0xc3,0x3a,0x0d,0xb2, +0x71,0x45,0xf2,0xdc,0x98,0x2e,0x04,0xaa,0xd8,0x96,0x95,0xe8, +0x8f,0xb3,0x70,0x4b,0x0d,0xf7,0x38,0xba,0xe4,0xad,0x28,0x3c, +0x00,0xf3,0x9b,0xd2,0xc1,0x03,0x7f,0x0b,0xc3,0xc7,0x42,0x8f, +0xa9,0x0c,0xfb,0x32,0x89,0xe2,0x53,0x74,0xa0,0x19,0xcb,0xd1, +0x50,0x0c,0x9a,0xfd,0x4e,0x5d,0x0c,0xfc,0x5a,0x70,0x9c,0x13, +0x93,0xb4,0x11,0x25,0x11,0x56,0x76,0x79,0xc7,0x13,0x55,0x1c, +0x2b,0x45,0x1e,0xb0,0xfa,0x12,0xb1,0xee,0x5b,0x0a,0xa0,0xc2, +0x47,0xad,0x3c,0xba,0x54,0xb1,0xfb,0x57,0x72,0x50,0x59,0x09, +0x26,0x53,0xcc,0x96,0x6d,0xe7,0xb9,0x58,0xa6,0xea,0x28,0xb7, +0xea,0xd4,0x67,0x4e,0x87,0x3e,0xfb,0x1d,0x87,0xbb,0xaa,0xae, +0xcc,0xc5,0x55,0x2c,0x24,0xb3,0xf0,0x09,0xc7,0x55,0xda,0xd5, +0x97,0x52,0xda,0x25,0x9b,0xd0,0x12,0x86,0xff,0x94,0x84,0xae, +0xe0,0x8c,0x8d,0x0e,0x0c,0xb5,0x8e,0xe8,0x4f,0x16,0x05,0x62, +0x4f,0x2d,0x4b,0xb7,0xa7,0x16,0x51,0x7d,0xe1,0x49,0x1f,0x28, +0xdf,0xd9,0xff,0x2b,0x2b,0x6b,0x9d,0xa2,0x01,0x67,0xa4,0xde, +0xc4,0x93,0x21,0xbe,0xc1,0x28,0x8f,0x4c,0x17,0xea,0xba,0xed, +0x91,0x27,0x5b,0x11,0xb6,0xb5,0x43,0x42,0xeb,0x88,0xe8,0x5c, +0x1c,0xdf,0x0a,0xfd,0x13,0x6c,0xb6,0xe3,0xd6,0xbe,0x93,0x03, +0x93,0x5d,0x8f,0x46,0x8b,0xcf,0x26,0xf7,0x5e,0x35,0x35,0xa8, +0xa6,0x4d,0x41,0xa5,0x21,0x4a,0x7d,0xf3,0x9f,0x4b,0xde,0x82, +0x2e,0xbc,0x82,0xff,0x74,0xab,0xe5,0x4d,0xbc,0x2d,0xed,0xb6, +0xb4,0x09,0x97,0x12,0x7b,0x29,0x95,0x5c,0xcb,0xc8,0x5d,0x4d, +0x5f,0x8f,0x2d,0x75,0x99,0x18,0x5e,0xdf,0x23,0x7c,0x12,0x41, +0x9e,0x8b,0x39,0x1d,0x08,0xf0,0xe8,0x25,0x0c,0x4c,0xe2,0x7e, +0x38,0x24,0xac,0x8c,0xc4,0x74,0x77,0x57,0xfa,0x84,0xeb,0x79, +0xbb,0xae,0xaa,0x91,0x02,0xcd,0x15,0x15,0xee,0xd8,0x6e,0x7d, +0x3e,0x59,0x02,0x6b,0x54,0xfa,0x42,0xed,0x36,0xef,0x8e,0xe1, +0x75,0x85,0x36,0x18,0x8e,0x57,0xf8,0xe6,0x5c,0xa9,0x42,0x90, +0x79,0x46,0x27,0x28,0x1b,0x02,0xc3,0xb4,0x89,0x7b,0x29,0x88, +0x75,0x14,0xdc,0xc7,0x1e,0xda,0xea,0x19,0x65,0xd6,0x43,0xb9, +0xc5,0x50,0x57,0x79,0x4c,0x99,0x9a,0x64,0x13,0xde,0xea,0x25, +0x8f,0xaf,0xa6,0xc0,0x02,0xcc,0x3b,0x03,0xe3,0xb7,0x15,0x26, +0xbb,0xc5,0x5d,0x58,0x5a,0x55,0x2d,0x01,0x31,0xdf,0x6f,0x07, +0x0a,0x79,0xe0,0x64,0x48,0x83,0xb2,0xd4,0x56,0x92,0x95,0x93, +0xcd,0xca,0x46,0x57,0xde,0x2a,0x92,0x10,0x6c,0x23,0x33,0xc8, +0x97,0xb1,0x64,0xbe,0x84,0xe1,0xa2,0x8d,0x0d,0x56,0xcb,0x05, +0x43,0x83,0xd5,0xb5,0xba,0xaf,0xeb,0x2b,0x53,0xe3,0x4f,0x98, +0x2e,0x5e,0xd8,0x90,0x48,0xbb,0x9c,0x72,0xcc,0x5f,0xed,0xd0, +0x37,0xe8,0xff,0x4b,0xd8,0x2d,0xb2,0xe3,0xfe,0x9c,0xe9,0x22, +0x91,0xb8,0xd6,0x8b,0xc6,0xfc,0xa7,0x0d,0x18,0x8d,0xbe,0xcc, +0x86,0x51,0xd5,0x7c,0x19,0x33,0xa6,0x58,0xd5,0x5e,0x96,0x8c, +0x2c,0x56,0xa1,0xbe,0xb0,0x43,0x26,0x83,0x79,0xb9,0x17,0x0b, +0x85,0xc6,0xd1,0x9b,0x7d,0xd7,0x66,0x87,0x82,0xb7,0xca,0xf0, +0x35,0x1f,0xb5,0x6d,0x9f,0xd3,0x24,0xfa,0x81,0x1f,0x7c,0xb6, +0xc8,0x2b,0xec,0xce,0x49,0x49,0x39,0xf2,0x82,0xb3,0xeb,0x6e, +0xd0,0xbe,0x96,0x4e,0xde,0xfc,0x3c,0x12,0x5f,0x44,0xc1,0x9f, +0xe8,0x2c,0x7c,0x16,0xe1,0x2f,0x00,0xa2,0x27,0x8c,0x0c,0x6d, +0xbf,0xaf,0x06,0x8b,0xe4,0x32,0xdb,0xad,0xed,0x9c,0x0d,0xa0, +0xfc,0xee,0x8f,0x29,0x28,0xd2,0x21,0x3e,0x24,0x39,0x15,0x0b, +0x8c,0xb8,0x07,0x73,0x82,0xf4,0x3e,0x5f,0xcc,0x6e,0xf1,0x28, +0x61,0x01,0x5c,0x4a,0xa6,0x38,0x6f,0x68,0x14,0xea,0x89,0x9e, +0x0b,0x58,0x1c,0x61,0x0c,0x54,0xb1,0x8a,0x6a,0x94,0x86,0x61, +0xe1,0x30,0xc2,0x55,0x17,0x51,0x35,0x47,0xeb,0x76,0x6d,0x54, +0x4c,0xcd,0xda,0x0c,0xdc,0x70,0xf2,0xc2,0x43,0xb3,0xb2,0xd0, +0x80,0x0a,0x0d,0x5b,0x07,0x09,0x68,0x21,0x4e,0x10,0x12,0x40, +0x3e,0x60,0x46,0xbd,0xa5,0xe9,0x94,0x99,0xeb,0x23,0x09,0x64, +0x79,0xa5,0x52,0xa3,0x9a,0xf2,0x45,0x29,0x6f,0x1a,0xe9,0x32, +0x8a,0x2b,0x67,0x41,0xc7,0x47,0x10,0x3a,0x97,0xe0,0xce,0xd3, +0x30,0xf4,0x70,0x91,0xc1,0x50,0x8f,0x4c,0xcf,0x89,0x9a,0xea, +0x95,0xda,0x01,0xb7,0xe7,0xee,0xce,0x23,0xb6,0x46,0xb7,0x33, +0x58,0x18,0x83,0xda,0xb9,0x15,0x2b,0xdf,0xff,0x3a,0x42,0x96, +0x75,0x0d,0xad,0xcd,0xf9,0x6c,0xae,0x18,0x49,0xf0,0x30,0x7f, +0xa8,0x0a,0x1a,0x11,0x51,0x82,0x00,0x5c,0x89,0x76,0x20,0x25, +0x6c,0xd4,0x4c,0x71,0x10,0xfa,0xc0,0xfd,0x4a,0xfe,0x10,0xb5, +0xf2,0x05,0x25,0xd3,0x74,0xdd,0x71,0x7e,0xc7,0xe4,0x1f,0x1e, +0x7a,0xa5,0x15,0x04,0x15,0x48,0x7c,0xed,0x30,0xac,0x9d,0x02, +0x15,0x13,0xec,0x7d,0x41,0x0c,0x35,0x77,0xfd,0x45,0xc5,0x35, +0x2f,0xa6,0xd5,0xf2,0xc9,0x12,0xfc,0x67,0x64,0x24,0x27,0x02, +0xdc,0x1b,0x79,0x11,0xa9,0x90,0x09,0x56,0x62,0x9c,0xf0,0x31, +0xcf,0x5d,0x99,0xac,0x07,0x8a,0xa4,0x43,0xf2,0xbc,0x20,0x97, +0x15,0x5d,0xd0,0x29,0x76,0x81,0x64,0xb7,0xdd,0x05,0xb6,0x08, +0x6a,0x45,0xb6,0x10,0xa9,0xc6,0x24,0x16,0xec,0x64,0xb0,0x00, +0x11,0xa2,0xf1,0x2c,0x55,0x9b,0xa9,0x54,0x5f,0x72,0x5f,0x60, +0x22,0xbd,0x4f,0x88,0x5a,0x0a,0x5a,0x85,0x65,0xce,0x16,0xf2, +0x47,0xb4,0x00,0x4a,0xcf,0x16,0xec,0x6e,0x40,0x48,0x17,0xe6, +0x54,0x2f,0x3d,0x8a,0x52,0xb1,0x3a,0x8a,0x56,0x42,0x2e,0xe9, +0xd0,0xdc,0x5f,0x1d,0x21,0xd8,0x11,0x4d,0x4f,0xc7,0xa9,0x48, +0x8f,0x29,0x25,0x99,0x72,0x0a,0x98,0x53,0x40,0xce,0xa6,0xe8, +0x0b,0x6f,0x09,0xe8,0x25,0x68,0x79,0x29,0x1f,0xb3,0x22,0xe1, +0x60,0x3e,0x12,0x6b,0x31,0x16,0x02,0xfd,0xc8,0xc4,0x4d,0xb4, +0xea,0x2e,0x30,0x2c,0x9d,0x1e,0x69,0xc7,0x2b,0x30,0xc0,0x50, +0xfe,0x56,0x51,0x24,0x3b,0x84,0x19,0xe5,0x11,0xf2,0xbb,0xbb, +0x9b,0x33,0x35,0xa0,0xe8,0xaa,0x87,0xd6,0x3e,0xfa,0x0a,0x3f, +0x33,0x05,0xc3,0x0f,0x44,0x7c,0x69,0x3f,0x83,0x7f,0x47,0x59, +0x74,0x83,0x4e,0xed,0xb3,0xa6,0x7a,0xc9,0x44,0x0f,0xd1,0x75, +0x5d,0xd6,0xd3,0x58,0x8b,0x1b,0x49,0xcd,0x8c,0x69,0xb9,0x01, +0x5a,0xfc,0xce,0xac,0xf9,0x11,0xcf,0x3c,0xb1,0x68,0x3d,0x2a, +0x14,0xe1,0x96,0xd4,0x01,0xbd,0x24,0xa5,0x11,0xdd,0xf0,0xbb, +0x24,0x65,0x54,0xaf,0x6d,0xaa,0xd7,0x3b,0xa9,0x5e,0x03,0xda, +0x8c,0xa8,0xce,0x72,0x54,0x67,0x75,0x55,0x5d,0xb6,0x85,0xea, +0x90,0x9a,0x2e,0xa9,0xb6,0x8b,0xc8,0x66,0x68,0xaa,0xd7,0x48, +0xf5,0xc6,0x39,0xa0,0x23,0x27,0x26,0x7b,0x17,0x81,0xe5,0x39, +0x4e,0xef,0xee,0xec,0x1b,0x31,0xb9,0xed,0x52,0xb5,0xd4,0xe7, +0xf5,0x7d,0x20,0xf4,0x0a,0x9f,0x7e,0x8c,0x74,0x2c,0x09,0xf9, +0xad,0xe3,0x49,0x08,0x85,0xfd,0xf0,0xb0,0x70,0xad,0x43,0xed, +0x63,0x92,0xde,0xe8,0xe1,0x87,0x2a,0xbe,0x3e,0x92,0x68,0xd8, +0x7c,0x94,0x4a,0xc3,0x53,0xb8,0xba,0x6a,0x08,0xb6,0xb5,0x3a, +0x54,0xb8,0x30,0x23,0xa3,0x67,0x39,0x19,0x34,0x2a,0x5e,0xdc, +0x91,0x39,0xbb,0x56,0xb7,0x82,0x98,0xcf,0xdd,0x43,0x3f,0x6d, +0x83,0x52,0xad,0xf3,0x28,0xda,0x10,0x7c,0xe4,0xac,0x50,0xe0, +0xf1,0xab,0xc1,0xe8,0x27,0xcc,0x64,0x8a,0xb0,0xcb,0x28,0x8e, +0xa7,0x99,0xc5,0x04,0x0d,0x3f,0x3a,0xbf,0x0b,0xeb,0x26,0x28, +0xc8,0x23,0x33,0x9a,0x25,0xa6,0xfb,0x5f,0x2d,0x90,0x05,0x7d, +0x6b,0xb7,0x0b,0xf1,0x94,0x45,0x8b,0xb2,0xf6,0x97,0x46,0xdf, +0x17,0x17,0x79,0xf3,0x48,0xbd,0xbd,0xc0,0x38,0xf8,0x3c,0xc2, +0x98,0x11,0xe6,0x16,0x19,0x59,0xae,0xf3,0x6e,0x15,0x1f,0x18, +0x67,0xbb,0x2c,0x9b,0xd7,0x6a,0x85,0x73,0xf3,0x79,0xb7,0x37, +0x17,0xf4,0xbf,0x7e,0x7b,0xee,0xdb,0x97,0x43,0x46,0xcf,0x67, +0x14,0x48,0x77,0xe9,0xc6,0x30,0xaa,0xc6,0xbd,0xf6,0x37,0xdb, +0x5e,0x92,0x91,0x7c,0xe1,0x98,0x94,0x50,0xb9,0xd9,0xa9,0x18, +0x29,0x8b,0x5c,0xc7,0xce,0xf2,0x2b,0x5b,0xe8,0xec,0x9c,0x79, +0xf4,0x58,0xa7,0xf2,0x00,0xf0,0xbb,0x16,0x77,0xdb,0xa6,0xa5, +0x04,0x80,0xbe,0x96,0xbd,0xb5,0xa0,0xff,0x91,0x60,0xa9,0xd6, +0xd1,0xe6,0x81,0xfa,0x50,0x9d,0x68,0x7a,0x93,0xc5,0x73,0x3f, +0xb7,0x07,0x02,0x5d,0x77,0x8d,0x2c,0xb7,0xa5,0x8b,0x87,0x96, +0x14,0x4a,0x3d,0x90,0x9e,0x1f,0xa6,0x12,0x29,0xbd,0x7b,0xba, +0xd2,0x13,0xec,0x5a,0x98,0xb1,0x87,0x03,0xc0,0x08,0x23,0xe5, +0x5b,0x0d,0x20,0xf9,0xdf,0xd3,0x6f,0x03,0x41,0x2d,0xc1,0xa6, +0x04,0x83,0x8a,0x86,0xd6,0x5e,0x98,0x08,0x72,0x5d,0xaf,0xd5, +0x0c,0x2e,0x4c,0xda,0xb5,0x5b,0x8b,0xa2,0x9f,0x5b,0xe3,0x70, +0x3c,0x85,0x08,0x16,0xc7,0x30,0xbd,0x8e,0xcd,0x92,0xf9,0x7f, +0xff,0x53,0xfc,0xf1,0x3f,0xd5,0x9b,0x60,0x79,0x02,0x94,0x68, +0xb5,0xce,0xf0,0xc0,0xf8,0x8f,0xff,0x19,0x81,0x6d,0x19,0xd2, +0xef,0xff,0xfd,0x4f,0xb4,0xf7,0xac,0x95,0x12,0x07,0x89,0x00, +0x95,0xde,0x19,0xcd,0x1e,0xac,0xa3,0xd6,0x91,0x13,0xc5,0x01, +0xe6,0xe0,0xac,0x24,0x6d,0xa1,0x42,0x10,0x75,0x6e,0xc1,0x6c, +0x89,0xbd,0x83,0xc5,0xdd,0xdd,0xe2,0x2c,0xd4,0xc3,0xf4,0x7f, +0xff,0xb3,0xfe,0xc7,0xff,0x3c,0x5a,0x1f,0x19,0x7d,0xd3,0x90, +0x48,0xc6,0xb3,0xf7,0xde,0xc2,0x3f,0x5e,0x80,0x55,0x23,0xe0, +0xf7,0x4f,0xb9,0x56,0x31,0x3d,0xb2,0xcd,0xba,0xf5,0x4d,0x6e, +0xbc,0xbc,0xa7,0x33,0x3a,0xb1,0x6e,0xe9,0x74,0xf2,0x91,0xc5, +0xdc,0xf7,0xe2,0xe2,0xf5,0xdc,0x53,0x74,0x63,0x95,0xc9,0xe2, +0x36,0x99,0xfe,0x00,0x9a,0xc6,0x54,0x39,0x29,0x46,0x27,0xb3, +0x8e,0xb1,0xe5,0xdb,0xcc,0xea,0x31,0xc7,0xb4,0x1e,0x39,0xbc, +0xd0,0x16,0xd1,0xf1,0x64,0xc3,0x71,0x46,0x69,0x9a,0xc1,0x3b, +0x16,0xea,0xf5,0x78,0xfc,0x3e,0x9f,0x82,0x19,0x6b,0xa2,0xaf, +0x5e,0x8e,0x67,0xb3,0x85,0x13,0x5e,0x79,0x18,0x27,0x63,0x3b, +0x81,0xae,0x68,0x50,0xd1,0x26,0x07,0xb6,0x2e,0x0f,0x87,0xcd, +0xc8,0x19,0xc2,0xc3,0x67,0xe4,0xfb,0x82,0xff,0x75,0xc3,0xb5, +0xe2,0xda,0xc2,0x10,0x87,0x7d,0xb0,0x15,0x1d,0x74,0x51,0xae, +0x76,0x58,0xb7,0x40,0x73,0xc1,0x90,0xe5,0x4a,0x40,0x98,0x02, +0x58,0x4e,0x08,0x9c,0xa4,0x82,0xbe,0x9f,0x6b,0x2b,0x94,0xff, +0x1e,0xfb,0x8b,0x22,0x88,0x5d,0xd2,0x2f,0xaf,0xd6,0x0c,0xe2, +0x1a,0x6d,0x43,0x6b,0x98,0x9f,0x80,0x25,0x3f,0xc5,0xef,0x61, +0x84,0xdd,0xc4,0x8a,0x25,0xc5,0xb1,0xd5,0x20,0x76,0x23,0x7b, +0xbc,0x06,0xe8,0x57,0x66,0x54,0x15,0x18,0x21,0x21,0x38,0x8b, +0x7f,0x73,0x58,0x3a,0x6c,0x69,0x2d,0x64,0x17,0x5b,0x2b,0x42, +0xe6,0xa3,0xcd,0x35,0x14,0xe3,0x52,0x44,0xa6,0x1e,0x60,0x13, +0x08,0xdc,0x6c,0x8a,0x4f,0x64,0x35,0x1f,0xd9,0x85,0xf9,0x45, +0xe6,0xf1,0x36,0x42,0x24,0x88,0x07,0x0b,0xf5,0x4f,0xcb,0x76, +0x8d,0x5c,0xb3,0x6a,0x62,0xd0,0xee,0xf5,0xfa,0x78,0xa7,0xc5, +0xe0,0x1a,0x42,0x8f,0xbe,0x9f,0x2d,0xb2,0x30,0xd8,0x46,0x8e, +0x42,0xe7,0x2b,0x37,0xe9,0x93,0x73,0x53,0xb0,0x0c,0x5d,0xeb, +0x73,0xd1,0xb5,0x2a,0xd0,0x5d,0x7c,0x36,0xbe,0x8b,0x72,0x84, +0xc3,0xcf,0x46,0x38,0x34,0x32,0x6a,0x71,0xaf,0xd7,0x0a,0x4e, +0x5b,0xad,0x53,0xe8,0xe8,0xd3,0xc7,0x0f,0x1f,0xc2,0x87,0x68, +0x3d,0x6d,0x3d,0x7c,0xf4,0xe4,0x89,0x08,0x1f,0x06,0x27,0x0f, +0x1f,0xb7,0x9e,0x88,0xa7,0x8f,0x5b,0xf0,0xfd,0x48,0x3c,0x0d, +0x9f,0x3e,0x0e,0x4f,0x40,0xdc,0x1f,0x3e,0x0d,0x1e,0x9f,0x9c, +0x04,0xe2,0xc9,0xc9,0xa3,0x47,0x8f,0x43,0x48,0x68,0x9d,0x9c, +0x3e,0x0d,0x9f,0x9c,0x8a,0xf0,0xd1,0xa3,0xd3,0xf0,0xf1,0xa3, +0xbe,0xf2,0xde,0x5f,0xbc,0xbf,0xe0,0xb8,0xcf,0x24,0xf5,0x16, +0x93,0x4d,0xad,0xe1,0xc3,0x47,0x0f,0x1f,0x3e,0xb4,0xaa,0x87, +0x1f,0x8f,0x9e,0x9e,0x3e,0x6e,0x19,0x3a,0x82,0x00,0x70,0x3f, +0xb5,0x08,0x42,0x90,0xe0,0xf1,0xc3,0x96,0xa6,0x2c,0x6c,0x3d, +0x7d,0xf4,0xe8,0xc9,0x69,0x68,0x48,0x6c,0x41,0x72,0x88,0x68, +0x15,0xad,0xe1,0x69,0xeb,0xe4,0x61,0xf8,0xf4,0xa1,0x21,0xfa, +0x24,0x78,0xda,0x7a,0x12,0x3c,0xb6,0xa8,0x07,0x3a,0x9e,0x3c, +0x7a,0xf2,0x54,0x35,0x03,0x6a,0x7e,0x18,0x06,0x50,0xc5,0xce, +0xf6,0x5c,0x44,0xbd,0x13,0x24,0xe9,0xf1,0x63,0xf1,0xe8,0xe4, +0x29,0x14,0x7e,0x2a,0x1e,0x07,0x0f,0x83,0xc7,0xf0,0x37,0x0c, +0x5a,0xa7,0x0f,0x5b,0x80,0xfd,0xf4,0x51,0xf8,0x34,0x78,0xfa, +0x58,0x3c,0x6d,0x85,0xa7,0x8f,0x80,0x8e,0x30,0x7c,0x1a,0x3e, +0x0a,0x43,0xa8,0x05,0x08,0x3a,0x3d,0x3d,0x39,0x85,0x9c,0xa0, +0xf5,0xf4,0xe9,0x09,0x10,0xf4,0xb0,0x75,0xfa,0x24,0x40,0x3a, +0x80,0x2c,0x68,0x06,0xfe,0x80,0xb2,0x27,0x2d,0xf1,0xe4,0xf4, +0xf4,0x34,0x3c,0x85,0x1c,0x28,0x74,0x0a,0x4d,0x43,0x9e,0x3c, +0x7c,0x7a,0xfa,0xf0,0x04,0x41,0x1e,0x3f,0x3e,0x01,0xb6,0x41, +0x93,0x1e,0x9f,0x3e,0x81,0x84,0xe0,0xc9,0xc9,0xc3,0x93,0xd6, +0x43,0xc4,0xdf,0x7a,0xf2,0x08,0xd1,0x01,0x3b,0x9f,0x9e,0x3e, +0x7a,0xb8,0xb3,0x39,0x43,0x68,0x4e,0xeb,0xe1,0x43,0xe0,0x1b, +0x34,0x03,0x99,0x15,0x20,0x2b,0x82,0xd3,0xd3,0x16,0x52,0x1b, +0x40,0x1e,0xb2,0x18,0x50,0x3e,0x0a,0x1e,0x41,0x45,0x48,0xd2, +0xe3,0xd3,0x87,0xf8,0xa3,0xf5,0xa8,0xf5,0xe8,0x29,0xfe,0x38, +0x79,0x08,0x9d,0x26,0x4e,0x5a,0x8f,0x4e,0x82,0xc7,0xa7,0xe2, +0xf1,0xe9,0xa3,0xd6,0xc3,0xc7,0x88,0xe5,0x34,0x78,0x1c,0x04, +0x50,0xe6,0x24,0x0c,0x00,0xc1,0x43,0xc8,0x79,0x0a,0xd4,0xa1, +0x0a,0x3d,0x79,0x0a,0x58,0x5b,0xdc,0x1b,0xd8,0x8e,0xf0,0xe1, +0x09,0xf0,0xed,0x69,0x88,0x8c,0x3b,0x01,0xde,0x8a,0xa7,0x4f, +0x4e,0x9f,0x40,0x8d,0xc8,0x9e,0x93,0x87,0x4f,0x80,0xd5,0xe1, +0xc3,0xd6,0xe3,0xc7,0xc1,0x93,0xb0,0xa4,0x3d,0x46,0xf3,0xc1, +0x12,0x00,0x26,0x93,0xd2,0x30,0xdc,0x5a,0xeb,0x28,0x18,0x54, +0x3b,0x3d,0x47,0xdf,0x71,0xce,0xc7,0x3d,0x4a,0x7f,0x8c,0xbd, +0xa0,0x4c,0x63,0x6a,0xa7,0xe1,0x6d,0x28,0x34,0x90,0xd7,0x6b, +0x3e,0xea,0x17,0x91,0xa8,0x19,0x71,0x1b,0x0e,0x05,0xe3,0xd9, +0xa5,0x6f,0xde,0xc3,0x7a,0x76,0x58,0xba,0x29,0x0c,0xe9,0xca, +0xd2,0x08,0x22,0xf2,0x6f,0xfd,0x85,0x8e,0xe2,0xca,0x2f,0x02, +0x8b,0x45,0x18,0x81,0xc1,0xb6,0x03,0x68,0x10,0x44,0xbb,0x2e, +0x27,0xd7,0x47,0xfc,0x34,0x0f,0x54,0xcf,0xfe,0xee,0x18,0x3d, +0x7a,0xfb,0x25,0xe4,0x62,0x91,0xd1,0x20,0xf2,0x06,0xc1,0xd9, +0x20,0x44,0x97,0xaf,0x41,0x34,0x08,0xb0,0xe6,0x41,0x88,0xa8, +0x46,0x03,0x20,0x23,0x6c,0x0c,0xc0,0xc0,0x1b,0x5d,0x46,0x7f, +0xfc,0xdb,0xd9,0x68,0xd0,0xad,0x05,0xb5,0x76,0x2d,0xac,0x89, +0x61,0xc0,0x86,0xcb,0x70,0x96,0x7a,0x08,0x90,0xca,0xef,0x14, +0xe6,0x74,0xfc,0x1e,0x86,0x56,0x3e,0x58,0x8b,0x69,0x68,0xe5, +0x87,0xd6,0xbd,0xbf,0xb3,0xc8,0x90,0xf4,0x72,0xb0,0xee,0x2e, +0x82,0x6e,0xed,0x65,0x20,0x6a,0xf5,0x45,0x58,0xaf,0x3d,0xe3, +0x3f,0xf2,0xeb,0x41,0xf0,0x00,0xa4,0xfb,0x01,0x66,0x36,0xb6, +0xe6,0xe2,0x27,0xe3,0x08,0x18,0x2a,0x60,0xa8,0x40,0x42,0x05, +0x12,0xc7,0xb6,0x5c,0xfc,0xfc,0x0d,0xda,0xfa,0x25,0x68,0x01, +0x3c,0xd4,0x2c,0xfc,0x3a,0x1a,0xca,0xea,0xc2,0xa3,0x34,0x28, +0x29,0x57,0xab,0x8f,0x2e,0xe1,0x3b,0x7c,0xc0,0xc0,0xa1,0x06, +0x86,0x5f,0x3f,0x22,0x59,0x3a,0x2d,0xa0,0xb4,0x42,0x03,0x24, +0x02,0xfc,0x81,0xc0,0x81,0x06,0x56,0x0d,0xfa,0xd3,0x54,0x04, +0x22,0xa8,0x21,0xae,0x7f,0xe1,0xb8,0x65,0x46,0xde,0xad,0xae, +0x7c,0x61,0x52,0x85,0x25,0xec,0x16,0xc4,0xcf,0x26,0xd5,0xbe, +0xa5,0x6d,0x00,0x5e,0xeb,0x44,0x73,0x71,0xdb,0xe4,0x7e,0x27, +0x93,0x3a,0x7a,0xcd,0x34,0x6c,0xda,0xa4,0x98,0xfb,0xbe,0x5b, +0x96,0x55,0x39,0xda,0x95,0xc7,0xfa,0x0d,0xdd,0x35,0xf7,0xdb, +0x56,0xf6,0x06,0x53,0x9a,0x76,0x4b,0xf6,0xaa,0x20,0xd7,0xf4, +0x7c,0x05,0x56,0x36,0x57,0x50,0x76,0x0f,0x7c,0x1b,0x7e,0x97, +0x71,0x79,0xf4,0xf6,0x9d,0x70,0xc4,0x5e,0xbc,0x13,0xbe,0x0d, +0xb7,0xcd,0xf4,0x3c,0x66,0x73,0x3f,0x1c,0xf1,0x0e,0xa1,0xdc, +0x62,0x96,0x38,0x7a,0x99,0xb4,0x61,0xe4,0xed,0xd4,0x86,0xf5, +0x9d,0xaa,0x10,0x4f,0x9e,0x41,0x49,0xed,0xd4,0x86,0xdb,0x35, +0x1f,0x60,0x29,0x28,0x3f,0x29,0x3b,0x3d,0xa3,0xa9,0xfc,0xa3, +0x85,0x30,0x7a,0x0a,0xbe,0xfa,0xd4,0x42,0xb3,0x22,0xb1,0x8b, +0x47,0x8d,0xab,0xc1,0xf8,0xf2,0x8f,0x7f,0x13,0x8e,0x06,0x8b, +0xfe,0xf8,0x5f,0x8d,0x7f,0xff,0x67,0x47,0x4e,0x19,0xb8,0x02, +0x20,0xa7,0x0f,0xd7,0x58,0xa7,0x43,0x7b,0xc8,0xb2,0x5f,0x57, +0xf3,0xac,0xdf,0xbf,0x50,0x09,0xad,0x26,0x35,0x16,0x10,0x23, +0xfa,0xbd,0x56,0x7e,0x75,0xf2,0x8b,0xfe,0xf0,0xb3,0xe7,0xf4, +0x33,0x73,0xf2,0x33,0xca,0xdf,0x08,0xab,0x82,0xd7,0xcb,0x78, +0xfe,0x4d,0x0c,0xab,0xa9,0x58,0xdd,0x71,0x8c,0x72,0xb9,0x74, +0x5c,0x9f,0x2f,0x42,0x89,0x55,0x25,0x18,0x9f,0x99,0x2c,0xe3, +0x41,0xc5,0x24,0x4b,0xbc,0x8a,0x07,0x4e,0x1c,0xb3,0x8d,0x5d, +0xae,0x82,0x63,0x84,0x32,0x87,0xa0,0x82,0x63,0x1a,0x0b,0x70, +0x8c,0x7e,0x6b,0x8e,0xc9,0x2f,0xfa,0x63,0x8f,0x7f,0x4e,0x0f, +0x5c,0xb0,0x80,0xe1,0xec,0x61,0xcc,0x19,0xa1,0x0b,0x17,0x32, +0x1c,0xf7,0x00,0xfd,0xcc,0x1c,0x80,0x8c,0xf3,0xad,0xe1,0xca, +0xc9,0x6e,0x75,0x99,0xac,0x4e,0x8f,0x3c,0x4e,0x75,0xeb,0xca, +0xb8,0x2e,0xcd,0xaf,0xea,0x17,0x48,0x28,0xc7,0x38,0xda,0xa4, +0xfa,0xd1,0x05,0x1e,0x1b,0x32,0x3c,0x34,0x39,0xd6,0x2c,0x73, +0x79,0x32,0x5e,0xb4,0x15,0xa4,0x06,0xe7,0x8d,0xb4,0x39,0x0f, +0xea,0x68,0xec,0xa4,0xcd,0x85,0x80,0x8f,0x10,0xfe,0x01,0x03, +0x01,0xfe,0x09,0xfc,0xba,0x17,0x83,0xf5,0x36,0x4e,0x3d,0x28, +0xec,0x77,0x87,0xab,0xc5,0x4d,0x6c,0x83,0xd1,0x8f,0xc0,0x6f, +0xe7,0x32,0x96,0xf0,0x63,0x89,0x19,0x84,0x56,0x7e,0x61,0x32, +0xa0,0x5d,0x12,0x5a,0x2e,0xa0,0x73,0x34,0x26,0x1f,0x67,0x20, +0xeb,0x30,0x51,0x91,0x9f,0xc6,0xe3,0x4b,0x71,0x29,0x4c,0xab, +0x65,0x46,0x74,0x29,0x23,0x15,0x60,0x3e,0x3b,0x13,0x49,0x2f, +0x42,0x2b,0x5d,0x01,0x63,0xb6,0x6b,0x80,0x95,0xc2,0x6c,0x37, +0xc0,0xf6,0x2b,0x22,0xd9,0xfb,0x69,0xd1,0x5e,0x40,0x95,0xf8, +0x7e,0xc7,0x20,0xc4,0x97,0x3b,0xe6,0x41,0xbb,0xb7,0x38,0x72, +0x6c,0x2c,0xf9,0x29,0x4d,0x2c,0x7c,0x57,0xdf,0x05,0x09,0x5d, +0x90,0xd0,0xef,0x5b,0x67,0x28,0xb2,0x77,0xec,0xdb,0xdc,0x20, +0xaf,0x41,0x14,0x5d,0xc0,0xbf,0x87,0x87,0xf0,0x3b,0xa4,0xdf, +0xe1,0xc6,0x31,0x6b,0x17,0x62,0x2e,0xf4,0x25,0x69,0xb2,0x13, +0xd8,0x14,0x90,0x56,0xc2,0x83,0xba,0x37,0x38,0xfb,0xe3,0xdf, +0x7c,0x69,0x2b,0xcc,0x4d,0x61,0xee,0xb7,0x45,0x00,0x0d,0x01, +0x83,0x17,0x68,0xd5,0x48,0xfe,0x1f,0xb0,0x8b,0xb0,0xec,0x3c, +0xdc,0x58,0x4f,0x85,0x20,0x5f,0xec,0x67,0x39,0x30,0x41,0x4a, +0xe1,0xc2,0x31,0x1d,0x48,0xaa,0xbf,0x9c,0xdd,0xc0,0x8f,0x17, +0xe4,0xdd,0xf2,0xb6,0xcd,0x8b,0x8b,0xb2,0xe9,0x5c,0x3e,0xa8, +0xa0,0x5c,0xf0,0xd4,0x9b,0x0d,0x2b,0xe7,0x39,0x83,0xad,0x33, +0xb9,0x66,0x43,0x11,0xab,0x7c,0xbf,0x44,0x62,0x95,0xfc,0xd9, +0x0b,0xab,0xe1,0x65,0x11,0xab,0x7c,0xf7,0x44,0xd1,0xfa,0xa5, +0x2c,0x0f,0x45,0xb3,0x65,0x7b,0x70,0x0d,0x5f,0xc4,0xfa,0x90, +0xd8,0x8d,0xfd,0x61,0x9e,0x90,0x40,0x65,0x08,0x73,0xc0,0xfb, +0x59,0x6e,0x63,0x4c,0x8b,0xa4,0xca,0xb4,0x7c,0x0f,0x61,0x8c, +0x73,0xe4,0xd9,0x9c,0xa3,0xe1,0xfc,0x24,0x92,0x01,0x68,0x73, +0x19,0x93,0xc8,0x9b,0x07,0xcd,0xac,0x3e,0x3f,0x69,0x66,0x68, +0x98,0xcc,0xa3,0x1e,0x08,0xf8,0xa7,0x75,0x1b,0x52,0xd7,0x22, +0x6b,0x4f,0x36,0xf4,0x71,0xa2,0x3e,0xe6,0x27,0xc6,0x83,0x34, +0x9a,0xd3,0xea,0x7a,0xbe,0x98,0xe1,0x4b,0x87,0x40,0x91,0x2f, +0x50,0xab,0xce,0x7b,0x41,0xbf,0x5e,0x7b,0x8e,0x3f,0x42,0xf8, +0x81,0x23,0xa3,0xd7,0x52,0x3f,0x4e,0xfa,0x7b,0x0e,0x12,0x83, +0x55,0x09,0xbb,0x6a,0xee,0x2b,0x9d,0xa3,0x57,0x5d,0x32,0xa7, +0x20,0xa0,0xdb,0x03,0x17,0xe5,0x05,0x14,0x5f,0x1b,0x96,0x98, +0x8c,0x8c,0x6a,0xdc,0x79,0x31,0xdd,0x1a,0x4b,0xbb,0x20,0xa6, +0x0e,0x6e,0x25,0xa9,0x1a,0xb7,0xd5,0xd8,0xbd,0xf0,0x5b,0xf0, +0x6b,0x0b,0xaf,0x49,0x36,0xb8,0x0b,0xa2,0x54,0x61,0x8b,0x68, +0x49,0xcb,0x81,0x7b,0xfe,0x5e,0x5d,0x61,0x01,0x9d,0x47,0x25, +0xed,0x2a,0x74,0xd5,0x3d,0x9b,0x6c,0xa1,0xf7,0x72,0x44,0xb0, +0xa5,0x64,0x48,0x71,0xb8,0xe3,0xef,0xe0,0x4a,0x9a,0x4d,0x2e, +0x66,0xe5,0xc3,0x8b,0xb3,0xec,0xc0,0x4d,0xaa,0x66,0xce,0xe1, +0xd8,0x2f,0xf4,0x86,0x9d,0x3b,0xa8,0x38,0x1a,0x8c,0x01,0x7c, +0x4e,0xcf,0x53,0xfa,0x74,0x2c,0x9c,0x07,0xdd,0xa8,0xd3,0xce, +0xc8,0x29,0xf2,0x06,0x52,0xf8,0xa0,0xdb,0x49,0x46,0x8f,0x40, +0x7d,0xa4,0x42,0x29,0x4d,0x2a,0xbb,0x9f,0x50,0xca,0x5a,0x2c, +0x91,0x64,0x1c,0x20,0x90,0x90,0xb5,0x91,0x5f,0xf7,0x3d,0xe7, +0x2e,0xc7,0xc8,0x47,0xde,0xfc,0xe5,0xac,0x36,0x24,0xef,0x74, +0xfc,0x10,0xf9,0xb8,0x67,0x09,0xc3,0xc4,0x70,0x31,0x4b,0xad, +0x8b,0x0e,0x1c,0xb8,0x9d,0xd7,0x61,0xe6,0xe0,0x0c,0x53,0x8f, +0x1f,0x19,0x4f,0x5d,0xd4,0x40,0x8d,0x93,0x23,0x9e,0xd6,0x1b, +0xf0,0xe7,0x07,0xfe,0xf3,0xab,0x4a,0xfe,0x81,0x66,0xfb,0x5f, +0x75,0x26,0x25,0xfe,0x4a,0x89,0x3a,0x4b,0x01,0x36,0xec,0x2c, +0x2e,0x0e,0x46,0x1a,0x4a,0xd1,0x64,0x66,0x47,0xa4,0xb0,0x48, +0xcb,0xf2,0xb4,0x61,0x44,0x0d,0xa7,0x6b,0x07,0xd3,0x93,0x00, +0x9f,0x85,0x5f,0x47,0x8b,0xac,0x24,0x4b,0xb7,0x83,0xf7,0x62, +0x32,0xde,0x26,0x59,0xe3,0x16,0x48,0x0d,0xad,0x14,0xb4,0x56, +0x32,0xd2,0xa6,0x0d,0x9d,0x8a,0x44,0x91,0x47,0x43,0xbc,0x0f, +0xbb,0x72,0xcc,0x32,0xac,0xfa,0x51,0x99,0x43,0xf8,0xf1,0xc0, +0xb6,0x8d,0x34,0x98,0xe4,0x40,0x6d,0xb9,0x48,0x68,0x71,0xd0, +0x18,0xcd,0x6e,0xed,0x87,0x4c,0xad,0x5a,0xd7,0x79,0x4e,0xb8, +0xa2,0x0c,0xc9,0x27,0xc0,0x85,0x2c,0x5a,0xac,0x8f,0x8a,0x39, +0x16,0x89,0xaa,0xc5,0x8a,0x0d,0x92,0x2b,0x9a,0x01,0xf2,0xdb, +0x25,0x6b,0x35,0xff,0x8f,0x27,0xaa,0xe1,0x52,0x95,0x23,0x4a, +0xd2,0xc4,0xf1,0x3a,0x8c,0xaa,0xc1,0x91,0x9d,0x46,0x39,0x45, +0x72,0x1d,0x67,0xa9,0xe7,0x3b,0xef,0x51,0x92,0xf7,0xd7,0x8b, +0xe9,0x55,0xbc,0x48,0x96,0x2f,0x46,0xa2,0x2c,0x59,0x14,0x29, +0xb4,0x5a,0x77,0xe2,0x8b,0xa2,0x70,0x71,0xfe,0x72,0x30,0xf5, +0x4e,0x02,0x6c,0x20,0xcd,0x05,0x53,0xbe,0x00,0x63,0xf0,0xbf, +0x52,0x8f,0x73,0xa2,0x57,0xac,0x5b,0xf3,0x88,0xef,0x28,0x94, +0xc0,0x92,0x66,0xa3,0x86,0xc5,0x63,0x56,0xb7,0x6e,0x56,0x45, +0x15,0xcd,0x78,0x32,0x5f,0x66,0x15,0xe5,0x28,0xaf,0xaa,0x20, +0xfa,0x94,0x54,0x94,0xe3,0x58,0xc4,0xe5,0xc5,0xb4,0x56,0x2d, +0x16,0xc3,0x2c,0xfd,0x82,0x6b,0x2e,0x4e,0xad,0x06,0xaf,0xd6, +0x87,0xe5,0x5d,0xd7,0xd5,0x25,0x2d,0xbc,0x1e,0xfa,0xaa,0xcb, +0xe4,0xb6,0x4d,0xcc,0x2f,0xe8,0x68,0x65,0xc3,0x6d,0x5c,0x82, +0xcc,0xc3,0xa9,0x51,0x79,0xfb,0x2a,0x9b,0x4d,0x35,0xe4,0xdb, +0x83,0xd7,0xd4,0xed,0x55,0x25,0x2e,0xed,0xb4,0x6f,0x91,0x48, +0x46,0x11,0x85,0x41,0x4f,0x46,0xd6,0x1b,0x9f,0xbd,0x7e,0x47, +0x95,0x75,0xd8,0x78,0xae,0x52,0x0d,0xea,0x8e,0x72,0x13,0x90, +0xaf,0x69,0x10,0x32,0xf7,0x21,0x8d,0x4f,0x1a,0xb1,0x7c,0x0a, +0x51,0x2d,0x6f,0x7b,0x7d,0x53,0x9c,0x53,0xb0,0x34,0xc6,0xcb, +0x92,0x9e,0x8f,0x94,0x98,0x77,0xda,0x24,0x99,0xa0,0x1c,0xf4, +0x40,0x47,0x07,0x34,0x6e,0x4f,0xa4,0x68,0xe2,0xf9,0xd7,0xb8, +0x4f,0x9d,0x9f,0x63,0x4c,0x96,0xf3,0x73,0x7c,0xc8,0xd4,0xf7, +0xbb,0x5e,0x4d,0x25,0xd4,0x12,0x8e,0xc9,0x6d,0x70,0x68,0xd8, +0xc8,0xf9,0xca,0x8d,0x1d,0xbc,0xcb,0xa2,0x4a,0x00,0x52,0x60, +0x9d,0x84,0x36,0x20,0xe7,0xe7,0xbd,0x64,0xd4,0x37,0xcf,0x92, +0xea,0x96,0x93,0x1f,0x15,0x48,0x86,0x93,0x4e,0x37,0x2d,0xac, +0xcb,0x66,0x96,0x70,0x68,0xde,0x41,0x35,0xd6,0x03,0xbb,0xa5, +0x3d,0xff,0x6c,0x3c,0xde,0xb7,0xf3,0x53,0x79,0xdd,0x5d,0x36, +0xc2,0xe0,0xfc,0x4c,0x81,0x78,0xc6,0xfb,0x06,0x7b,0xcb,0xc4, +0xbd,0xbb,0x5d,0x86,0x81,0x36,0x1d,0xff,0xc9,0x1a,0xc1,0x15, +0xec,0xd7,0x8d,0xdd,0x4f,0x34,0xc4,0x3e,0x72,0x7a,0x8d,0x24, +0xce,0x22,0x85,0xd9,0x50,0x79,0xfd,0xf5,0x0c,0x84,0x53,0xcb, +0xa2,0xcc,0xef,0x5d,0xa3,0x88,0x56,0x4b,0xcf,0x35,0x4a,0x8f, +0xc9,0xab,0x12,0x98,0x3f,0x23,0x1b,0x97,0xc9,0xd8,0xb9,0xd8, +0xc6,0xdf,0x39,0xb1,0xe0,0x7f,0x95,0x4c,0x98,0x5e,0x37,0x9e, +0x76,0x07,0xca,0xc7,0x8e,0xcb,0xc3,0x98,0x91,0x88,0x1d,0x79, +0xe0,0x34,0x55,0x87,0x2d,0x09,0x41,0x4e,0x10,0x94,0x3b,0xd1, +0x67,0x69,0x87,0x20,0x2f,0x25,0x2a,0x16,0x57,0x41,0x3b,0x30, +0x25,0xd5,0xfd,0xee,0x9b,0x97,0x7a,0xad,0xc7,0x0a,0x76,0xf1, +0x5b,0x8e,0x90,0x6a,0xa6,0x2f,0x6f,0x63,0x3b,0xe4,0x3b,0x3d, +0xc0,0x4e,0x69,0x32,0xa8,0x99,0x1e,0x64,0x9d,0x8a,0x99,0xe6, +0xeb,0x56,0x97,0x20,0x90,0x1c,0xcf,0x2f,0x11,0x6e,0xae,0x82, +0xd6,0x27,0xf4,0x2e,0xbc,0x33,0xc5,0x9c,0xc7,0xe8,0x3e,0xcf, +0x4f,0x8c,0x90,0x5b,0x25,0x01,0x97,0x5e,0x5d,0xab,0xc4,0xbc, +0x88,0xe9,0xb9,0x45,0x42,0x5e,0x7e,0x4f,0xad,0xb2,0x68,0x2a, +0x89,0x92,0x4d,0xde,0x54,0xf3,0x69,0xb0,0x5c,0x2e,0x5c,0x36, +0xfd,0xf4,0x9a,0x63,0x63,0xd9,0x67,0xd5,0x00,0x44,0x9e,0xad, +0x30,0xe8,0xe9,0x21,0x70,0xa2,0xcc,0xbc,0x7a,0xce,0x24,0x16, +0xc0,0x7f,0x7a,0x5d,0x51,0x00,0x32,0xf8,0x15,0xfc,0xf9,0x60, +0x08,0x02,0x81,0x3f,0xc7,0x33,0x90,0x90,0x1c,0x8e,0x37,0x48, +0xbc,0x67,0xf6,0x1e,0x99,0x93,0x17,0x5d,0x85,0xbf,0xed,0x5d, +0xd4,0xa3,0x5a,0x2d,0xef,0x49,0x98,0x88,0x01,0x77,0xee,0x7b, +0xfb,0x65,0x76,0xa2,0x51,0x77,0xf6,0x01,0xe0,0x81,0x11,0x94, +0x44,0x96,0x87,0x1a,0xed,0x73,0x1a,0x64,0x4b,0x49,0x7b,0x9a, +0xc7,0x22,0x12,0xc8,0xa3,0x67,0x9f,0x4b,0xc8,0x85,0xa6,0x55, +0x13,0xfc,0xd3,0xeb,0xfb,0x90,0x5c,0xc9,0xa5,0x3f,0xdb,0x8a, +0x2a,0xc4,0x56,0xc3,0x40,0xdb,0x17,0x07,0x84,0x72,0x49,0x04, +0xe1,0x78,0x80,0x33,0x37,0x09,0x8b,0x4f,0xe8,0xb1,0x85,0x9c, +0x23,0x38,0xb9,0x47,0x1f,0x26,0x58,0x2f,0x42,0xc9,0xa3,0x5f, +0x4d,0x6a,0x64,0xbd,0x96,0x1f,0x45,0x5c,0xae,0x3b,0x2a,0x7f, +0xe4,0xbd,0x9d,0xf3,0x26,0x44,0xe8,0x88,0x1f,0xf2,0xc7,0x4d, +0x6a,0xba,0x12,0xc4,0xf4,0x74,0xca,0x54,0xc7,0x39,0x0d,0x04, +0x1e,0x90,0x35,0x24,0xb6,0x59,0xab,0xdb,0xc2,0x6e,0x71,0xa1, +0x6b,0xf5,0x65,0x5b,0xff,0xde,0x3e,0x82,0xde,0x14,0xb5,0x0d, +0x60,0x96,0xfa,0xa6,0x44,0xa6,0xf5,0x66,0xe2,0x25,0x2b,0x06, +0x77,0xc7,0x42,0x54,0x48,0xaf,0x71,0x34,0x05,0xb5,0xba,0x5b, +0x4c,0x2f,0xa9,0x37,0xcb,0x65,0xf4,0xbe,0x04,0x54,0xca,0xe2, +0xde,0x34,0x55,0x0a,0x9d,0x24,0x93,0x9c,0x88,0x77,0x76,0x2a, +0xa1,0xe5,0xbe,0xcc,0x75,0xe3,0xe7,0x77,0x60,0xba,0xcc,0xec, +0x0d,0x66,0xe2,0x1d,0x0b,0xc5,0x7c,0x91,0xcc,0x16,0x78,0xd0, +0x68,0x47,0x24,0x07,0x60,0x5b,0x17,0x52,0x82,0x54,0x70,0x80, +0x73,0x1e,0x2f,0x96,0x59,0x5e,0x21,0x12,0x0c,0xbb,0x46,0x95, +0x68,0x07,0x8d,0x72,0x8b,0x72,0x00,0xd2,0x6f,0x93,0xe9,0x68, +0x76,0x8b,0xbd,0xf2,0x7c,0x36,0x99,0x03,0x47,0x47,0xaf,0xb1, +0xa0,0x99,0x64,0x7c,0xcc,0x53,0x24,0xfc,0xca,0x37,0x88,0xcb, +0x95,0x9e,0x3b,0x96,0xaa,0x34,0x06,0x35,0x2c,0x35,0x28,0x8d, +0xf2,0x33,0x8c,0xd9,0xa8,0x97,0xc6,0x4b,0xfc,0xbf,0x41,0x87, +0x9c,0x9c,0x4d,0x29,0x44,0x57,0x2d,0xa5,0xc6,0x1b,0x1b,0x86, +0x08,0xfb,0xc4,0xae,0xe1,0x1c,0x25,0xb1,0x56,0x63,0x6b,0x43, +0xa1,0x56,0x1a,0xc6,0x37,0xc4,0xe8,0x3c,0xea,0xec,0x9e,0xfa, +0x92,0xc1,0x35,0x5d,0x55,0xa3,0x32,0x01,0x6f,0xa9,0x2d,0xe1, +0x28,0x04,0x6e,0x2a,0x8b,0x92,0xec,0x7a,0xab,0xcb,0x76,0x48, +0x4e,0xc9,0xd8,0xe7,0x91,0x5f,0x29,0x3e,0x7b,0xe9,0x80,0x3f, +0xd3,0xe3,0xdb,0x07,0x65,0x79,0xbf,0x5e,0xba,0xfd,0xba,0xa9, +0x32,0x8b,0x4e,0xf0,0xee,0xa1,0x61,0xae,0x2f,0xec,0x21,0xe9, +0x30,0xd2,0xb4,0x75,0x8b,0x99,0x16,0xaf,0xed,0x53,0x29,0xb6, +0x3b,0x76,0xf6,0x17,0x96,0xaa,0xc9,0x8e,0xca,0x41,0x41,0x4e, +0x75,0x6d,0x3c,0x4a,0x4b,0x1c,0x0c,0xa8,0x0d,0x64,0xb2,0xe1, +0x5b,0xcb,0xd6,0x56,0x40,0x61,0x30,0xce,0x3b,0x04,0xeb,0x5a, +0x5e,0xcd,0xe1,0x6c,0x35,0x5d,0x7e,0xdd,0x42,0xd6,0xf0,0x3c, +0xce,0x51,0xe4,0x28,0x18,0xc3,0xe1,0xe1,0x5c,0x56,0x4c,0x61, +0x0b,0xa9,0x09,0xdb,0x2c,0xb2,0x78,0x90,0xc6,0x05,0x96,0xec, +0x69,0xb3,0x86,0xbb,0x6c,0x56,0x44,0xde,0xf6,0x4a,0x16,0x15, +0xfa,0xe5,0x29,0xf9,0x52,0x13,0x41,0x96,0x5d,0x76,0xe0,0x1d, +0xaf,0x12,0x3b,0x77,0x4f,0x03,0x95,0x9a,0xc7,0xaf,0x60,0xf8, +0xd5,0x4c,0x18,0xe1,0xbb,0xef,0x7b,0x72,0xa1,0x82,0x9e,0x92, +0xc8,0x27,0x37,0xee,0x23,0x27,0x72,0xfb,0xe0,0x43,0x35,0xb1, +0x4c,0x86,0xfd,0x52,0x4c,0xc5,0xd2,0x75,0xd3,0xf6,0xec,0xf7, +0xc8,0xf6,0x65,0x06,0xe3,0xdf,0xcd,0x8d,0x55,0x3e,0x6c,0xc3, +0xff,0x31,0x86,0x28,0x4a,0xf4,0x95,0xdb,0x50,0xec,0xe6,0x8e, +0x61,0x4f,0xbe,0x98,0xbf,0x37,0xa7,0x54,0xc5,0x3b,0x99,0x85, +0x0d,0x36,0x8c,0xa2,0xed,0xba,0x71,0x92,0x2e,0xe3,0xa9,0x5a, +0x73,0x5b,0x0c,0xab,0x30,0x73,0xd9,0x4e,0xa5,0xbd,0xc5,0xa8, +0x74,0x53,0x38,0x51,0x1b,0x8f,0xa5,0xd9,0x2f,0x46,0x9d,0xf2, +0xe4,0x28,0x19,0xed,0x35,0x7a,0xb8,0x0f,0xca,0x70,0x54,0xee, +0xb4,0x98,0x23,0xb2,0x2a,0x09,0xf5,0x4b,0xf7,0xb7,0xa3,0xc4, +0xec,0x73,0x97,0x52,0xac,0x7e,0x71,0xd8,0xd2,0xfd,0xc6,0x3e, +0x9d,0xb7,0xed,0xdc,0x1d,0xea,0x78,0xd6,0x96,0x2b,0x3d,0xc0, +0x78,0x77,0x57,0x48,0x42,0x55,0x34,0x4a,0xc0,0x68,0x5c,0xa2, +0x72,0x96,0xaf,0x18,0xf3,0xd3,0xff,0x3e,0x3d,0x06,0xe9,0x76, +0xf0,0xc6,0xb5,0x02,0x2a,0xa7,0x9d,0x92,0xcd,0x67,0xeb,0xc2, +0x55,0xc9,0xa6,0x8c,0xb3,0x41,0x17,0xe8,0x1d,0xba,0x64,0x14, +0x46,0xf5,0x7a,0x6e,0xfb,0xde,0xd9,0xc1,0x11,0xff,0xc7,0x76, +0x5e,0x2c,0x5e,0x46,0x3f,0x5f,0xe0,0xf1,0x6c,0x73,0xb8,0x88, +0xd1,0xe0,0x2b,0xef,0x90,0xa0,0xef,0x5b,0xcd,0x64,0xdd,0x54, +0x8f,0xec,0x14,0x39,0x06,0x4b,0x36,0x63,0xf5,0x4e,0x6c,0x68, +0x6f,0xa6,0xe5,0x77,0xd3,0xf6,0xdc,0x4a,0x0b,0x2d,0xc7,0xbe, +0x75,0x92,0x96,0xbb,0xcf,0x43,0x86,0xf7,0xde,0xff,0xf4,0x9e, +0xa7,0xed,0xdc,0x2c,0x8d,0xd7,0x2f,0xf5,0xd2,0x51,0xd0,0xb3, +0xa8,0x2c,0xb6,0x3c,0xfb,0xf2,0x3d,0x6c,0xd9,0x8b,0xe7,0xe7, +0xf8,0xb0,0xeb,0xf2,0xfc,0xfc,0xee,0x8e,0x92,0x39,0x33,0xcc, +0x65,0x46,0xf2,0x22,0xd0,0x6c,0x8e,0xf7,0xa5,0x11,0xbf,0x0a, +0xb2,0x85,0xbf,0xc9,0xf6,0x4a,0xbb,0x5c,0x92,0x5e,0xd1,0x76, +0x3f,0xe4,0x44,0xca,0x49,0x54,0xfa,0x99,0x52,0x3b,0xe7,0x7e, +0x5b,0x42,0xca,0xdb,0x6d,0x7e,0xdb,0x60,0x24,0x50,0x79,0x91, +0xcd,0xd4,0xc6,0x09,0xe7,0x76,0x0d,0x9c,0x54,0x4c,0xd9,0x5e, +0xb1,0xe5,0x4b,0xd9,0xb6,0x10,0x13,0x5c,0xf4,0xde,0x6c,0x6a, +0x7b,0x35,0x42,0x59,0xf3,0x9b,0xa8,0x50,0x3c,0x6a,0x92,0x64, +0x13,0xf3,0xe2,0x3b,0x5c,0x4d,0x10,0x6d,0xcd,0x18,0x7f,0x7a, +0xf8,0x76,0x74,0x0a,0xc6,0xa5,0x57,0x7b,0x0f,0xc3,0x55,0x36, +0x0d,0xca,0xd3,0x3e,0x41,0x6d,0x38,0x1e,0xa4,0x29,0xa4,0x4b, +0xa4,0x6c,0xdc,0xd7,0x66,0xb0,0x32,0x04,0x4a,0x6a,0xe2,0xdf, +0xff,0x29,0xb1,0xae,0x49,0xff,0xda,0x47,0x35,0x5c,0x05,0xa4, +0x7b,0x7e,0x69,0x39,0xb5,0x69,0xc6,0x08,0xfe,0x3a,0x1f,0xe1, +0x36,0x43,0x11,0x45,0xb1,0x6c,0xa8,0xa2,0xf0,0xab,0x0b,0x1d, +0x14,0xf7,0xdd,0x53,0x8d,0x04,0xfd,0x73,0x95,0xe3,0x88,0x6e, +0x13,0xf1,0x04,0xdf,0xd7,0x23,0x28,0x59,0xa7,0x37,0xa7,0xab, +0x9f,0x92,0x19,0xd0,0x0b,0xa0,0xb2,0xbc,0x1a,0x26,0x16,0xb8, +0xa0,0x10,0xb9,0x87,0x50,0x84,0x00,0x2c,0x79,0xcd,0x5e,0x8d, +0x04,0x3d,0x57,0x6b,0x16,0xdf,0x75,0x06,0x99,0xc5,0x32,0xb8, +0x17,0xc0,0x98,0x4e,0x61,0x30,0x39,0x02,0x54,0x79,0xfc,0x23, +0x89,0x35,0xbc,0xca,0x03,0x91,0x7d,0x4e,0x4c,0x57,0x19,0x5c, +0x09,0x19,0xee,0x9e,0x91,0x19,0x86,0xac,0xaa,0x91,0xcb,0x10, +0x48,0x75,0x8d,0x92,0xfa,0xf4,0x36,0x41,0x55,0x0f,0xeb,0x0b, +0x60,0x9e,0xff,0x69,0x08,0xe6,0x62,0x8d,0x5f,0x69,0xa9,0xb5, +0x9d,0x61,0xad,0xfd,0xf3,0x40,0x13,0xbc,0x15,0xba,0xc1,0x92, +0xb9,0x59,0xab,0x26,0xc8,0x53,0xf7,0x4d,0x82,0x31,0x08,0x3f, +0xc6,0x16,0x85,0x0a,0xa4,0x26,0xb4,0x39,0xe2,0x80,0x8a,0xc0, +0xaf,0x63,0x55,0xaf,0x64,0x7c,0x45,0x8b,0x55,0xb2,0xe8,0x1a, +0xb0,0x07,0xfe,0xf6,0xaa,0xdc,0x02,0x36,0xfc,0xde,0xf5,0x22, +0x1a,0x59,0x68,0x94,0xe1,0x38,0x7a,0x1c,0xc6,0x13,0x33,0x60, +0x30,0xbb,0x31,0x98,0xa2,0x37,0x1b,0x64,0x4e,0x92,0xd1,0x68, +0x8c,0x7d,0x66,0xa4,0x50,0x95,0xad,0x91,0xcf,0x98,0x7a,0x6a, +0x81,0x4e,0xce,0xc9,0x3b,0x59,0x55,0x5c,0xaf,0xfd,0x1a,0xfc, +0xa0,0x00,0xd0,0x9f,0xec,0xd7,0x1c,0x80,0xdf,0xb9,0x80,0x99, +0xe3,0xba,0x43,0xbd,0xb1,0x9c,0xcd,0xef,0xdd,0x15,0x8d,0xdd, +0x7d,0xd1,0xf0,0xf6,0xe3,0xca,0x3e,0xdd,0x51,0x56,0xdd,0xb6, +0xfe,0xd8,0xbf,0xee,0x7c,0x97,0x04,0x5f,0xaa,0x43,0x1a,0x3b, +0x7b,0xa4,0x51,0xdd,0x25,0x18,0x48,0x6f,0x4b,0x9f,0xbc,0x2b, +0xf4,0xc9,0x7a,0x8f,0x3e,0x59,0xff,0xe9,0x3e,0xc9,0xd5,0x60, +0x75,0x50,0x50,0xde,0x27,0x7b,0xd7,0x67,0xba,0x2e,0x28,0x19, +0x26,0x27,0xad,0xea,0x5e,0x21,0xd3,0xb1,0xb2,0x4b,0xf2,0xbd, +0x20,0xec,0x5e,0xfa,0x21,0xf8,0xd5,0xee,0x92,0x1f,0xb6,0x76, +0x09,0x45,0x38,0xbc,0x77,0x9f,0xec,0xee,0x92,0x3f,0xa3,0xb2, +0xee,0xdb,0x1d,0xfb,0xd5,0xf5,0xf9,0x7d,0xc1,0xe6,0x7c,0x75, +0x6f,0xdc,0xaf,0x33,0xdc,0xbe,0xc0,0x23,0x0c,0x69,0x0e,0x11, +0xd4,0x37,0x83,0xa9,0x0c,0x05,0x84,0x11,0x7c,0x72,0x19,0x1e, +0x45,0xaa,0x34,0x86,0xa6,0x79,0x08,0x92,0x21,0x21,0xa1,0x3e, +0x5a,0x6f,0xac,0xc9,0x98,0x37,0xeb,0x1c,0x33,0x73,0x6d,0x1b, +0x1d,0xe5,0x00,0xbc,0x88,0xda,0x8a,0x85,0x4d,0xd3,0x5d,0xa8, +0x1c,0xe3,0x0b,0xcc,0xa1,0x2d,0x30,0x72,0x83,0xd6,0xb6,0xef, +0x38,0x3e,0x44,0x31,0xe2,0x00,0x4f,0xba,0xb6,0x94,0xca,0x27, +0xca,0x7e,0xa6,0x0c,0x57,0x7e,0xa2,0x53,0xe1,0x70,0x1c,0xbe, +0x2d,0xa9,0x88,0x4e,0x72,0xd6,0x66,0xd4,0x0b,0x83,0xbe,0x30, +0xa6,0x2d,0xd9,0xb3,0x7a,0x23,0x6b,0x8d,0x1b,0x93,0x44,0xd5, +0x7e,0x4e,0x91,0x04,0xba,0x16,0x58,0x4e,0xda,0xcf,0x1b,0xfa, +0x68,0xca,0x26,0xec,0x85,0x45,0xc2,0xae,0x71,0xab,0xd9,0x6a, +0x33,0x37,0x36,0xed,0x62,0xe8,0x94,0x76,0x25,0x2f,0xb8,0x6a, +0x46,0x21,0xeb,0xe6,0x55,0x41,0x71,0x77,0xb1,0xc4,0x4d,0xd4, +0x65,0x8d,0x06,0x90,0x58,0xdd,0x6c,0x0b,0xfb,0xaf,0xf7,0x78, +0x49,0xd2,0x82,0x5f,0x5b,0x78,0x7f,0x95,0x0f,0x4a,0x6a,0x9c, +0x72,0x79,0xb1,0x37,0x4e,0x29,0x44,0x0e,0x52,0x99,0x66,0x61, +0x2d,0x84,0xe9,0xdc,0x1e,0x24,0x66,0xda,0x75,0x75,0x4c,0x54, +0x5f,0xe7,0xa4,0xab,0xae,0x8b,0x52,0xe4,0x44,0x59,0xb9,0x53, +0x48,0x56,0xef,0x22,0xda,0xab,0x5d,0x85,0xba,0xab,0xb1,0xbb, +0x54,0xed,0x27,0x67,0x6e,0x43,0x14,0x76,0x27,0xd9,0x62,0xdd, +0xab,0x92,0xd8,0x55,0x5b,0x7b,0x44,0x15,0xa8,0xdb,0x5d,0xf2, +0x4a,0x45,0x63,0x36,0x5d,0xb2,0xba,0x18,0x25,0x37,0xc9,0xa8, +0x6c,0x03,0x3c,0x8f,0xf8,0xf0,0x10,0x8b,0x71,0x61,0xf7,0x22, +0x63,0x7e,0x20,0x44,0xca,0x3a,0x17,0xc5,0x21,0x14,0x7d,0x02, +0x5b,0xb1,0x1d,0x72,0x30,0x60,0xf8,0xcb,0x90,0xf0,0x03,0xed, +0x95,0x76,0xb8,0x51,0x5e,0x9b,0x17,0x8b,0x55,0x7a,0x55,0xba, +0xcc,0xa7,0x9c,0xea,0x75,0x7e,0x61,0x65,0xaf,0xe6,0x19,0x7a, +0x69,0x27,0x5e,0x34,0x68,0x0f,0x09,0x17,0x59,0xa0,0x1e,0xcd, +0x2c,0xd4,0xb8,0x8d,0x2f,0xae,0x93,0x65,0x63,0x39,0x98,0x37, +0xae,0x80,0xb6,0x31,0xd2,0xd7,0x18,0xce,0xc6,0x34,0x2b,0x2d, +0xde,0x5f,0x0c,0xbc,0x40,0xd0,0xff,0xfc,0x1a,0x6d,0x32,0xd5, +0x26,0xb3,0x55,0x1a,0xa3,0x1b,0x2e,0xd3,0x5a,0x13,0xf4,0x87, +0xe6,0x2f,0x06,0x58,0xce,0x56,0x43,0xfe,0x2e,0x81,0x10,0x17, +0x83,0xe1,0xf5,0x7b,0x0e,0x4d,0xe5,0x2e,0x22,0x4d,0x86,0xbd, +0x90,0xec,0x98,0xe4,0xc2,0x12,0x12,0x83,0xdd,0x15,0x96,0x90, +0x0e,0x1a,0xd9,0xc8,0x9b,0x24,0x4d,0x2e,0x92,0x31,0xad,0x6e, +0x6b,0x57,0x60,0x8b,0xc6,0x53,0x93,0x39,0x5c,0x2d,0x52,0x6a, +0x2c,0xf9,0x7f,0x5f,0x0d,0x92,0x05,0x4c,0xc3,0x2e,0x69,0x30, +0x53,0x43,0xdd,0x36,0x59,0xfb,0xd1,0xa2,0xcb,0xe5,0xab,0xc2, +0x85,0xb9,0x5c,0x9e,0x2e,0x62,0xf2,0x10,0x75,0x2b,0xe4,0x44, +0x55,0x21,0x7f,0xa5,0xc2,0xbe,0xef,0xd9,0xe1,0x44,0xb9,0x01, +0x60,0xd6,0xfa,0x2a,0x39,0x47,0xde,0xfb,0x3c,0x6d,0xc5,0xf9, +0xbd,0xc6,0x45,0xf1,0xee,0xfa,0xa6,0x40,0x71,0x89,0x39,0x20, +0x65,0x9c,0xfa,0xf6,0x39,0x81,0xf5,0x46,0xfd,0x4d,0x05,0x47, +0xd6,0x65,0x28,0x8e,0x7b,0xf1,0x6d,0xff,0x2f,0xc7,0xb0,0x80, +0x4e,0x97,0x90,0xda,0x6d,0x9c,0x70,0xe4,0x61,0xcb,0x8c,0x2a, +0x29,0xf4,0x8f,0xde,0x34,0xed,0x57,0x17,0xba,0x4d,0x46,0x4b, +0x10,0xb9,0x53,0xf5,0x7d,0x15,0x93,0xf5,0x89,0x09,0xdb,0xa4, +0x41,0x31,0x4e,0xc2,0xe0,0x3e,0xeb,0x78,0x90,0x49,0xd1,0x65, +0xc7,0x60,0xcf,0xef,0xd6,0xa6,0xb3,0x69,0x5c,0xa3,0x0a,0x65, +0xef,0xd1,0x8b,0x6d,0xef,0x4b,0x77,0x57,0xde,0xdb,0xc2,0x5e, +0x0a,0x61,0xb2,0xfd,0x0e,0x05,0xf6,0x2c,0xd9,0x76,0x59,0x17, +0xb1,0x18,0x9e,0x2a,0x1b,0x30,0xd7,0x78,0x65,0x0b,0x36,0x74, +0x3e,0x34,0x6f,0xb4,0x18,0xdc,0xbe,0xf5,0x24,0xa9,0xb0,0x4c, +0xc9,0x2a,0x2a,0xcc,0x2a,0x2b,0xcc,0x8a,0x15,0x2a,0xee,0x56, +0xd6,0xf8,0xce,0xaa,0x91,0x53,0x74,0x82,0xed,0x34,0xa4,0xb2, +0x50,0xbb,0x95,0x0e,0x04,0xae,0xce,0x38,0xc9,0x94,0x09,0x30, +0xe5,0xd2,0x31,0x7e,0xad,0xbe,0xfe,0x8e,0xc6,0x5e,0xaf,0x7e, +0x1c,0x1b,0x09,0xe3,0xe5,0x65,0xa6,0xb3,0xfe,0x91,0xda,0x59, +0x7e,0xad,0x48,0xd1,0x5b,0x9b,0x24,0x5b,0x13,0xe8,0x2e,0x50, +0x15,0x61,0x93,0xcb,0xb4,0x86,0x68,0x4e,0x29,0x5a,0xa8,0x68, +0xa6,0x67,0xce,0x90,0x90,0x7d,0xa5,0xca,0x03,0xef,0x2c,0x54, +0x79,0x3a,0xde,0x6d,0xa7,0x03,0x7a,0x26,0xdb,0x49,0x47,0x2c, +0xe9,0xb8,0x75,0xe9,0x50,0x5d,0x98,0x19,0x42,0xb2,0x32,0x42, +0x8c,0x12,0xb7,0x67,0xa5,0xeb,0x38,0xc3,0xb9,0x00,0x92,0x4e, +0x5a,0x11,0x1d,0x96,0xe2,0x34,0x83,0x37,0x0c,0x9e,0xb3,0x37, +0x33,0x50,0xff,0xfe,0x3d,0xcc,0xbe,0x77,0x77,0xde,0x90,0xb4, +0x12,0xc7,0x07,0x04,0x6b,0x11,0x52,0xf1,0x01,0xd9,0xc8,0x70, +0x40,0xa5,0x02,0x09,0x91,0x21,0x47,0x28,0x14,0x51,0x8b,0x0e, +0x6e,0xa8,0x86,0x57,0x0b,0xfa,0x23,0xe7,0x5f,0xcf,0x7e,0xe7, +0x0d,0xea,0x5e,0xcd,0x2b,0x09,0xc2,0x44,0x89,0x0f,0x9f,0x85, +0x55,0x64,0xd4,0x4b,0xc9,0xa8,0x97,0x92,0x11,0xec,0x43,0x05, +0xb1,0x8b,0x95,0x32,0x9f,0x96,0xd3,0xbb,0x7d,0x78,0x0d,0x09, +0xe7,0x4f,0x79,0x65,0xcf,0x17,0x08,0x31,0xc2,0x00,0xf5,0xfc, +0x5e,0x37,0x45,0x33,0x4e,0x88,0xf5,0x75,0xf9,0x8c,0x3c,0x86, +0x57,0xe5,0xb4,0xd0,0xa4,0x61,0xd0,0x37,0x8b,0xb3,0xba,0x99, +0x83,0xf1,0xf2,0xbf,0xc7,0x59,0x57,0xb2,0xda,0xf0,0xbc,0xe7, +0x19,0xe1,0xaa,0x9b,0x96,0xe2,0x5a,0xcf,0x33,0xbd,0x5d,0xcf, +0xec,0x9c,0xbe,0x6f,0x7a,0x49,0x73,0xa7,0xae,0xe9,0xfb,0x9a, +0x51,0xa3,0x90,0x58,0x1c,0xd3,0x0c,0x53,0x80,0xa1,0x06,0x04, +0xa4,0x7d,0xbf,0x6d,0x09,0x81,0x54,0xba,0xd0,0x84,0xb7,0x87, +0x87,0xc8,0x88,0x90,0xcb,0x88,0x35,0x58,0x1b,0xa8,0x9d,0xf4, +0x10,0x54,0x6c,0x0a,0x4c,0x91,0x77,0x6e,0x91,0x4c,0x84,0xa6, +0xc8,0x3b,0xb7,0x08,0xfd,0xd2,0x99,0x98,0x47,0xcc,0x3a,0xf7, +0x3e,0xe1,0x09,0x58,0xbb,0x26,0xad,0x14,0x8c,0x08,0xd9,0x56, +0x4c,0xed,0xf2,0x1c,0xdd,0x96,0xb3,0x62,0x6d,0x63,0xf7,0xad, +0x5d,0x31,0x1f,0x94,0x48,0xff,0x95,0x49,0x32,0xc5,0x67,0xa5, +0xab,0xf7,0xcf,0x81,0xfe,0x40,0x3f,0xf4,0x8a,0x91,0x8c,0x94, +0xe2,0xd4,0x31,0xc8,0x23,0xc9,0xca,0xa4,0x2f,0x78,0xe0,0x46, +0x49,0x57,0xf2,0xb4,0x2d,0x3b,0x81,0x6f,0xea,0xc5,0x7a,0xb8, +0xc6,0xaa,0xff,0xf4,0xa9,0xb7,0x11,0xf0,0x45,0xd0,0x88,0x14, +0x6a,0xa8,0xaf,0x41,0x11,0x7a,0xeb,0x2a,0x05,0x78,0x93,0x4c, +0x23,0x0f,0x6a,0x78,0x3e,0x1e,0x4c,0xe6,0xed,0x35,0xfd,0xf1, +0x4d,0x14,0xe9,0x45,0x20,0x4c,0xa4,0xe2,0x50,0xca,0x21,0xbe, +0x28,0xdb,0x56,0x3f,0xb5,0x18,0x76,0x31,0x2a,0xb5,0x07,0x80, +0x75,0x5d,0xa1,0x5f,0xa7,0xe7,0x04,0xa4,0x10,0x5a,0x91,0xba, +0xa3,0xaa,0x0a,0x5a,0x47,0x52,0x58,0x92,0x7e,0x03,0x92,0x7c, +0x9f,0x28,0x3c,0x53,0xe5,0xba,0xf4,0xd4,0x36,0x71,0x19,0xe8, +0xd6,0xd5,0xb4,0x31,0xd5,0xb4,0x49,0xf3,0xe3,0x00,0x41,0xef, +0xee,0x34,0xa7,0xe0,0x7b,0xb0,0xee,0x7a,0x9a,0xa1,0xdf,0xd2, +0x71,0x03,0xc9,0xa2,0xe2,0xad,0x95,0x64,0xf0,0x50,0x8d,0x1a, +0x0b,0x22,0xc1,0xe8,0xed,0xed,0x9b,0x59,0x32,0x7a,0x10,0xe4, +0x86,0x3c,0xda,0x3e,0xfe,0x27,0x6b,0xf4,0xa3,0x36,0xde,0x6e, +0x89,0x97,0xcd,0x77,0xfb,0xdb,0x21,0xc2,0xd8,0xfd,0xb0,0xfe, +0x18,0x65,0x45,0x73,0x93,0xc1,0x6e,0x8d,0xf5,0x8e,0x84,0x29, +0xdb,0x9c,0x9d,0xa7,0x74,0xd6,0x6a,0x5e,0xcc,0x20,0x83,0xbe, +0xbc,0x0c,0x65,0xa1,0x9f,0x50,0x21,0x47,0x4e,0x0c,0xa5,0x19, +0xb9,0x4a,0x48,0x84,0x7e,0xa1,0xe7,0xf4,0x90,0x5f,0x25,0x83, +0x13,0x77,0x2a,0xe5,0xe6,0x0d,0xcb,0x87,0x60,0x3d,0xa8,0xee, +0x71,0xd3,0xc9,0x3b,0x95,0x7b,0xa3,0xae,0x42,0x2b,0x9e,0x68, +0xd5,0xa8,0x34,0x2e,0xa3,0x8f,0x38,0x71,0x76,0x29,0x35,0xb1, +0x1d,0x3b,0xcc,0x59,0x4a,0xc9,0x52,0x4a,0xed,0x44,0x56,0x2d, +0x68,0x9d,0xaf,0x26,0x9e,0xa5,0xc6,0xa2,0x83,0xe3,0x7f,0x78, +0xd3,0xbb,0xd4,0x57,0x36,0x87,0xca,0x01,0xdd,0xb4,0x36,0xaa, +0x8b,0xc0,0xe2,0xbb,0xdb,0x32,0xb0,0xcc,0xcc,0x33,0x76,0x55, +0x64,0xb6,0xc7,0x60,0x56,0xab,0x99,0xdf,0x66,0x5a,0xa4,0xe6, +0xa3,0x73,0x4c,0x7c,0xbd,0x9a,0xcf,0x01,0x61,0xea,0x29,0xfd, +0x5d,0x9c,0x75,0x6e,0x1d,0x06,0x49,0xef,0xba,0xd2,0x7e,0x93, +0x9f,0x25,0x5d,0x47,0x1f,0x3e,0xfa,0x8d,0x68,0x16,0xf3,0xbb, +0x87,0xa3,0x37,0x24,0x14,0x69,0xf7,0xb6,0x5c,0x7a,0xf4,0xd8, +0x28,0x17,0x21,0xd5,0xe1,0x7e,0xbb,0x42,0x62,0x73,0xe5,0x73, +0x62,0xab,0x8b,0xc3,0xb8,0xa3,0xe1,0xb6,0x58,0xcd,0x71,0x85, +0x64,0x8d,0xb2,0xa3,0x9a,0x6f,0x67,0x69,0x86,0xfb,0xc5,0xe9, +0x0e,0xec,0x13,0x9d,0x58,0x32,0xc9,0x99,0xec,0xb0,0xcf,0x0f, +0xb2,0xe3,0x6b,0x68,0xaa,0x33,0x69,0x4a,0x88,0xd7,0x51,0xfd, +0xf8,0xb6,0xd0,0xd1,0x22,0xce,0x20,0xfd,0x1f,0xd3,0x7c,0xba, +0xb4,0x03,0xa2,0x9e,0x9e,0xa4,0x41,0xbd,0xdb,0x44,0x64,0x26, +0x3d,0xb3,0x6a,0xef,0x97,0xcc,0xd5,0x50,0xb0,0x84,0x66,0x28, +0xa6,0x1c,0x58,0x1c,0xa3,0x01,0xd4,0xb3,0x9c,0x9b,0xb9,0x8c, +0x7a,0xec,0xc1,0xef,0x54,0x2b,0x30,0x52,0x43,0x7b,0x6a,0xb0, +0x3d,0x95,0x95,0x2d,0xf4,0x6e,0x96,0x5f,0xaa,0x1a,0x78,0xe3, +0x7c,0xe3,0x0b,0x4b,0xe5,0x92,0xa2,0x70,0x14,0xba,0x70,0x34, +0xbe,0x50,0xfe,0x34,0x3c,0x6a,0xbe,0x55,0x4e,0x35,0x84,0x42, +0xd8,0x78,0x85,0x32,0x0e,0x8c,0x1e,0xf2,0xc5,0x9a,0xa7,0x87, +0x8c,0xff,0xc8,0x7a,0xa2,0x5e,0x20,0x4c,0xf7,0xc8,0x2f,0x9e, +0x4f,0xf1,0xbd,0x81,0x4c,0xff,0x92,0xab,0xf9,0xc8,0x5e,0x3f, +0xff,0xc2,0x69,0xd6,0x14,0x2e,0xf5,0xfd,0x8d,0xb3,0x83,0x5b, +0xb9,0xf9,0x53,0xd0,0x69,0xb9,0x68,0x88,0xac,0x34,0xc2,0xe8, +0xd3,0x5a,0xdb,0x10,0x59,0x5b,0x92,0x2a,0x12,0x77,0xee,0x13, +0x1f,0xda,0x0e,0xb3,0x36,0xb2,0x74,0xd1,0x47,0x44,0x62,0xed, +0xe4,0xdc,0x43,0x64,0x72,0x85,0x1b,0x55,0x37,0xbf,0x4d,0x65, +0xdf,0x3e,0x95,0x8e,0xa7,0xce,0xe6,0x91,0xd5,0x54,0x77,0x8e, +0x96,0x64,0x35,0x13,0xb7,0x73,0x75,0xfa,0x07,0xdd,0x35,0x2a, +0x65,0xad,0xbb,0x47,0xa5,0x64,0xdb,0x44,0x6a,0xe3,0x2b,0x67, +0x5e,0x4a,0x6e,0x17,0x09,0x22,0x39,0xcb,0x3b,0x91,0x3f,0x5b, +0x2c,0x06,0x99,0xb2,0xb7,0x15,0xeb,0x9b,0xb0,0x8e,0xcf,0xca, +0x21,0xb3,0x1c,0x64,0xa6,0x1d,0xb7,0xdc,0xf6,0x16,0x8d,0x16, +0xc7,0x53,0xdd,0x6d,0x6c,0x88,0x4f,0x4f,0x90,0x83,0x7a,0x96, +0x4b,0xcf,0xa2,0x8c,0xd2,0xab,0x6d,0x60,0x63,0xf3,0x6e,0x36, +0xbe,0xe5,0x0b,0xbc,0x5f,0x87,0x84,0x15,0x1d,0x12,0x42,0x87, +0xec,0x53,0xe7,0x96,0xf9,0xdf,0x6f,0x7b,0xdb,0x34,0xc0,0x9f, +0xc3,0x4e,0x8e,0xc8,0x3c,0xec,0xac,0xe3,0xaf,0x8f,0x5b,0x36, +0x9c,0xd7,0xd1,0xc7,0xad,0xa3,0xf9,0x60,0xfd,0xf5,0xd7,0xe1, +0xdd,0x41,0xd6,0x67,0xb4,0x7e,0x7b,0xad,0x2a,0xc8,0xf6,0xab, +0x20,0xbb,0x6f,0x05,0x99,0xaa,0x60,0x48,0xda,0x66,0xbf,0x56, +0x80,0xcd,0x01,0xff,0x4a,0xfd,0x74,0xf0,0x91,0x26,0x19,0xf3, +0x05,0xab,0xb6,0xf6,0xba,0x6b,0xb2,0xdb,0x38,0x51,0x98,0x7c, +0x5f,0xb7,0x0d,0xd1,0xf4,0x18,0x4e,0x96,0xef,0xeb,0x82,0xed, +0x4c,0xad,0x30,0xf8,0x69,0x31,0xa9,0xdd,0x58,0x30,0x6d,0x2a, +0x69,0x40,0x05,0x62,0x1d,0x8a,0x0c,0x74,0x66,0x28,0x96,0x55, +0xfe,0xdd,0x44,0xb7,0xb7,0x0e,0x22,0xa2,0x77,0x1d,0x46,0x48, +0x29,0xed,0x69,0x41,0xda,0x3a,0x90,0x89,0xeb,0x90,0xf6,0x43, +0x5c,0x31,0xed,0x51,0x05,0x90,0x0f,0x46,0xc0,0x4d,0xbc,0x58, +0xca,0x32,0xf0,0x8f,0x4f,0x65,0xbc,0x75,0xe8,0xe3,0x7b,0x7e, +0x67,0x6b,0x0c,0xea,0xba,0x8c,0x10,0x1c,0xf2,0x43,0xcc,0x84, +0x71,0x03,0x70,0x07,0x96,0x7d,0x70,0x77,0xb7,0x0e,0x0f,0xac, +0xcd,0x03,0x5c,0x85,0xea,0x29,0x81,0x6b,0x92,0x9b,0x6d,0x99, +0x24,0x36,0x93,0xc4,0xae,0x39,0x2d,0x0b,0x88,0xf2,0x30,0xca, +0x42,0x5a,0xd6,0xbb,0x63,0xa7,0x47,0x7c,0x80,0x7c,0x43,0x2c, +0x96,0x81,0x7f,0x7c,0x2a,0xe3,0x65,0x48,0x6c,0x16,0x9c,0x65, +0x4c,0x2c,0x82,0x43,0x7e,0x88,0x99,0x48,0x6c,0x06,0xc4,0x66, +0x16,0xb1,0x59,0x78,0x60,0xed,0x69,0x20,0xb1,0x7a,0xc6,0xe2, +0x9a,0x7c,0xdd,0xa3,0xcc,0x61,0x9b,0x9a,0x2e,0x31,0xca,0x4e, +0x51,0x7c,0x76,0xd2,0x50,0x64,0x2c,0x48,0x17,0xa6,0x57,0xe4, +0xbc,0xfc,0xd2,0x1d,0xa0,0xbf,0xb7,0xf5,0x83,0x62,0xaa,0x4b, +0x1f,0xb6,0x3d,0x4f,0x1f,0xb2,0xa9,0x48,0x9f,0x81,0x74,0x61, +0x7a,0x45,0x66,0x2b,0x7a,0x14,0xcf,0xf5,0xf7,0x36,0xd6,0x23, +0xe9,0x34,0x24,0x50,0x08,0x32,0xa8,0xa2,0x87,0x42,0x0d,0x86, +0x1a,0x0c,0x0a,0x29,0x17,0x38,0x94,0x14,0xd7,0xcd,0xc0,0x2d, +0x7f,0x5d,0xc0,0x59,0xff,0xdd,0xdd,0x79,0x5b,0xad,0x8e,0x1d, +0xd3,0x86,0xec,0xe1,0x8d,0xbd,0xa8,0x2c,0x56,0x79,0x70,0x00, +0xf4,0x9b,0x2e,0x8c,0xac,0x0e,0xbc,0xbb,0x3b,0x38,0x00,0xda, +0x33,0x2b,0xd3,0x70,0x8f,0x9f,0xc9,0xe0,0x47,0x34,0xd8,0xa0, +0x22,0x95,0x2b,0x6a,0xb3,0x69,0xcd,0x77,0x0e,0xd0,0xac,0xa3, +0x83,0xe8,0x13,0xbe,0x90,0x92,0x36,0xa4,0x9e,0x16,0xa0,0x9a, +0xe3,0x5b,0xfd,0x95,0x3a,0x79,0xb7,0x4e,0xde,0x14,0x3e,0xa7, +0xb7,0x69,0x6c,0x12,0xa0,0xec,0x34,0x4e,0xad,0xd2,0x71,0x0e, +0x22,0xbd,0x75,0x21,0x74,0x64,0x4f,0x5b,0xb9,0x46,0xbd,0x5e, +0x6d,0x8a,0x67,0x38,0xe8,0x9c,0x01,0xff,0xdd,0xa2,0xb9,0x4b, +0xff,0x50,0x0a,0xfd,0x73,0x5b,0x83,0x6e,0x25,0x08,0xfe,0x35, +0x25,0x58,0x8c,0xdc,0xcd,0x71,0x4c,0x12,0xbe,0xc3,0x47,0x7f, +0x3f,0x51,0x2d,0x74,0x08,0xf0,0x2d,0xde,0xda,0x94,0xf9,0x90, +0x92,0xbd,0x96,0x31,0x8a,0x7a,0xb5,0xd7,0xab,0xe9,0x08,0x6d, +0xe5,0xda,0xcb,0x99,0xfc,0xf1,0x66,0x15,0xa7,0xfc,0xeb,0x6f, +0xf1,0x68,0xaa,0x7e,0xbf,0xb9,0x02,0xae,0xf1,0xcf,0xef,0x17, +0x09,0xff,0x78,0x0d,0x8b,0xd1,0x05,0xfe,0xa4,0x57,0xee,0xb1, +0xa2,0xf3,0xd5,0x72,0x68,0x05,0xf0,0xf8,0x04,0x26,0x35,0x56, +0x5d,0xf2,0x96,0x33,0x5b,0x70,0x78,0x85,0xea,0xaf,0x6f,0x9e, +0x23,0x0c,0x06,0x02,0x21,0xf0,0x6c,0x37,0x74,0x26,0x81,0xbf, +0x07,0xd1,0x7a,0x07,0xd2,0xbb,0xab,0x84,0x82,0x93,0xc5,0x7e, +0x98,0x41,0x5b,0x76,0x95,0x21,0x20,0x59,0xe0,0x65,0x32,0x1e, +0x27,0xfc,0x0c,0xfe,0xce,0x72,0x36,0xac,0x2e,0x3e,0x5d,0xc1, +0x9a,0x6b,0x77,0x49,0x02,0x53,0x85,0x66,0xd3,0xe5,0xd5,0xce, +0x22,0x08,0x24,0x0b,0xbc,0xde,0x8f,0xbe,0xd7,0x0e,0x69,0x6f, +0x40,0x20,0xb6,0x96,0x40,0x00,0x0b,0xf6,0x23,0x2c,0xbf,0x38, +0x0c,0x66,0x49,0xa9,0x60,0xc3,0xf7,0x4a,0x7e,0xbe,0xac,0x46, +0x29,0x01,0x10,0x65,0x5a,0x14,0x0e,0x25,0xa2,0x73,0x2b,0xb6, +0x87,0x12,0x10,0x2b,0x4a,0x70,0xf3,0xdc,0x5a,0x6a,0x48,0x44, +0xd9,0x7e,0x78,0xb2,0xad,0x68,0xca,0x04,0xaa,0x0a,0x97,0x82, +0xdd,0x8a,0xb0,0x20,0x6a,0x55,0xd8,0x08,0x70,0x2b,0xaa,0x2a, +0x21,0xac,0xc2,0x68,0xc3,0xef,0x40,0x5c,0x10,0xcf,0x6a,0x9c, +0x04,0xba,0x1d,0x5d,0x5e,0x70,0x2b,0x91,0x21,0xe0,0x56,0x54, +0x25,0x22,0x5d,0x85,0xec,0xf5,0x1e,0x0d,0xcd,0x0b,0x7b,0x29, +0x2e,0x04,0xaa,0xc4,0xa2,0x27,0x13,0xb7,0x1c,0xe9,0x57,0xa3, +0xf7,0xb4,0xa6,0xe5,0xb7,0x6f,0x30,0x13,0xb1,0x46,0xb5,0xaf, +0x06,0x0f,0xbe,0xba,0x78,0xf0,0x55,0xfc,0xe0,0xab,0xb7,0x0f, +0xbe,0x7a,0x57,0x2b,0x81,0x03,0x98,0xc9,0xf1,0x57,0xa3,0xe3, +0x62,0xae,0xc4,0xf0,0x43,0xfb,0xab,0x97,0xed,0xaf,0x5e,0xd7, +0x6c,0x6d,0xfe,0xe5,0xf4,0xb8,0x8d,0xf5,0xd9,0xc5,0x05,0xbe, +0x5e,0x4d,0x97,0x4e,0x64,0x0d,0x8c,0x9e,0x71,0x33,0x5e,0xc6, +0xc8,0xc8,0x18,0x93,0x85,0x64,0x82,0x1d,0x8c,0x45,0xff,0xdb, +0x60,0xba,0x1a,0x2c,0xa8,0xce,0x18,0x66,0x3c,0xfe,0xf9,0x12, +0xdf,0xb8,0x82,0xbf,0xcf,0xe6,0x8b,0x64,0x4c,0xdf,0x98,0xfa, +0xdf,0x56,0x34,0xdb,0xfd,0xb7,0xd5,0x18,0xbf,0x9e,0xad,0xde, +0xaf,0x52,0xdc,0x13,0x79,0x1d,0xcf,0x97,0x31,0xbd,0x66,0x26, +0x6a,0x3f,0x0f,0x97,0x33,0xfe,0xf5,0xd3,0xec,0x46,0x25,0x7e, +0x1b,0x0f,0xf9,0x67,0xae,0xfa,0x7c,0x2b,0x80,0x14,0x26,0x83, +0x29,0xe0,0xfa,0xed,0xda,0xb9,0x72,0xae,0x9b,0x2b,0xe6,0x2a, +0xb9,0x3a,0xae,0x89,0x27,0x3c,0xac,0xa4,0x99,0xa1,0xf1,0xa4, +0x6a,0xa4,0x85,0x36,0x68,0x38,0xb3,0x59,0x42,0xc7,0xce,0xda, +0x7f,0x81,0xcf,0xe3,0xb9,0x20,0x70,0x98,0x73,0x85,0x74,0xc4, +0xe7,0x81,0x83,0x0f,0x69,0x70,0xd2,0x46,0x38,0x58,0xe4,0xfe, +0x33,0x48,0xad,0x04,0xd7,0x93,0x1a,0x25,0x58,0xb3,0xa1,0xe7, +0xd7,0x25,0x6c,0x0e,0x85,0x43,0x88,0x5b,0x42,0x5e,0x50,0x53, +0x2d,0x4a,0x23,0xfb,0x8b,0x5d,0x35,0x5d,0x80,0x26,0xcc,0xf4, +0x2e,0x10,0x4e,0xfd,0x2e,0x20,0x34,0x71,0x27,0x6b,0x68,0x3c, +0x01,0xdc,0x34,0xbe,0x7d,0x20,0x8d,0x08,0xaf,0x15,0x9f,0x88, +0xc0,0x7a,0x71,0x22,0xdb,0xd1,0x5c,0xa1,0x92,0xe4,0x74,0xa8, +0xbf,0xd9,0xae,0xc0,0xef,0x6c,0x17,0x37,0x09,0xd4,0x2d,0xb7, +0x2f,0x17,0x19,0xba,0x11,0x5a,0x3c,0xa4,0x41,0x69,0x7d,0x14, +0x19,0xe3,0x32,0x10,0x41,0x4a,0xf9,0xf7,0xf3,0xe5,0x3b,0xc7, +0x3a,0x37,0x4c,0xb3,0x05,0x8f,0x3a,0x80,0xf3,0x3a,0xf6,0xe3, +0x67,0xfc,0x1c,0x16,0x65,0x34,0x10,0xa4,0x71,0x1a,0x3f,0x3c, +0xd2,0xad,0x74,0xe7,0x73,0x68,0x01,0x75,0x63,0x49,0x86,0xef, +0x1f,0x3f,0x39,0x7d,0x18,0x3f,0x92,0xd7,0x3e,0x5d,0x1b,0x52, +0x3f,0x57,0x6b,0xd1,0x98,0xe1,0xe9,0x24,0xf5,0x3e,0xb4,0x6b, +0x39,0xfb,0x71,0x76,0x1b,0x2f,0x9e,0x0f,0x52,0x3c,0x6a,0x49, +0xa2,0xc7,0x8d,0xa4,0xa3,0x63,0x91,0xdc,0x50,0x3c,0x56,0xc2, +0xd9,0x03,0xe0,0xfe,0x9e,0x43,0xc9,0x2b,0x1f,0x4a,0x7e,0x45, +0x4f,0x36,0xac,0x6f,0xb0,0x1d,0xeb,0x89,0xff,0xd5,0xe3,0xfd, +0x06,0x59,0x99,0x58,0x3c,0x3e,0xb2,0x98,0x2b,0x0b,0x14,0x85, +0x44,0x09,0x76,0xb1,0x8b,0x34,0x1d,0xa5,0x7d,0x95,0xef,0x7b, +0x2e,0x53,0x47,0xae,0x32,0xdd,0xc7,0x8f,0xa9,0x3d,0xd9,0x41, +0x14,0x25,0x3e,0x05,0x5f,0xd4,0xec,0xab,0xe3,0x5a,0x20,0x52, +0xac,0x73,0xa5,0x49,0xe7,0x93,0xe0,0x69,0x98,0x82,0xd4,0x31, +0x1c,0xd7,0x0d,0xc8,0xfe,0x13,0x5b,0xb5,0xb1,0x06,0xd0,0x6d, +0x1c,0x5f,0xeb,0x4a,0x52,0x9a,0xdb,0x9c,0xbc,0x34,0x97,0x99, +0x1b,0x39,0x04,0xe2,0x8c,0x30,0x09,0x56,0x1c,0x64,0x08,0x2a, +0x47,0x99,0x0b,0xcb,0x89,0x1a,0xee,0x52,0xbd,0x68,0x67,0x4f, +0xca,0x8e,0x39,0x00,0x65,0x7e,0x89,0x73,0x00,0xbf,0xc4,0x9e, +0x3d,0x51,0xfb,0xf6,0x00,0xfa,0x71,0x36,0xbb,0x5e,0xcd,0x73, +0x05,0x38,0xb1,0xba,0x10,0xcf,0x67,0x3b,0xea,0x71,0x26,0xbd, +0x92,0xe2,0x7b,0xd6,0x5c,0x81,0x86,0xe6,0xd5,0x6d,0x14,0xf0, +0xbc,0x9f,0x2b,0xb0,0x4f,0x9d,0xa5,0x05,0x77,0xb7,0xb8,0x38, +0xd1,0x97,0xa2,0xd8,0x9b,0x82,0x0a,0x54,0xf3,0x78,0x81,0xc7, +0x55,0x40,0xc9,0xf1,0x3f,0xbe,0x3a,0xce,0x99,0x67,0xaf,0x06, +0xa3,0x34,0xfa,0x54,0x6b,0xd4,0xda,0xb5,0x9a,0x38,0x6f,0xd7, +0x1e,0xd4,0x44,0xd0,0xae,0x05,0xb5,0x4d,0x0e,0x0e,0x80,0x06, +0xed,0x72,0xdf,0xc6,0x52,0xb6,0xf7,0x46,0x7a,0x60,0xf5,0x37, +0xe2,0xd9,0x8e,0xa2,0x39,0xf0,0x8b,0x6d,0xe0,0xc5,0xa6,0x72, +0x61,0x39,0x89,0x42,0xf1,0x6f,0x76,0x16,0x2f,0x14,0x19,0xb6, +0xdd,0xe6,0x7a,0xe5,0xa6,0x30,0x30,0xd5,0xc2,0x2d,0xe6,0x05, +0xec,0x9a,0xa9,0xde,0xc8,0x68,0x5e,0x31,0x17,0x2d,0x50,0xb4, +0xf1,0xe7,0x17,0xfd,0xe1,0x5e,0x45,0xe5,0x16,0x80,0x2c,0xfb, +0xe2,0x73,0xca,0x7e,0x15,0xb6,0xee,0xee,0xc2,0x96,0x44,0xf1, +0x61,0x5f,0x14,0x61,0xbd,0x44,0x63,0x22,0x21,0x27,0x80,0xe5, +0xc7,0x7b,0x11,0xe2,0xee,0x47,0x48,0x14,0x93,0xfb,0xa1,0xe0, +0xee,0xad,0x87,0xb2,0x19,0x2f,0xef,0x49,0x80,0xdc,0xd6,0x90, +0xa5,0xe7,0xa5,0x52,0x65,0xf1,0xec,0x2c,0x0a,0x5b,0xdd,0xda, +0xab,0x97,0x30,0x94,0x9e,0xbd,0x84,0x01,0xf4,0xfa,0x5e,0xd5, +0xbd,0xb6,0x9a,0x8a,0xd5,0xfd,0x75,0xef,0xd2,0x25,0xda,0x9f, +0x99,0x8e,0x68,0x6e,0x2b,0xa9,0x96,0xdb,0x51,0x7f,0xbb,0x6f, +0x3d,0x93,0x59,0x59,0x3d,0xeb,0xdd,0x03,0xc8,0x17,0x6f,0xb7, +0x03,0xf1,0x08,0xcb,0xee,0xc5,0x36,0x63,0x4d,0x7f,0x15,0x06, +0x81,0x24,0xe6,0xdd,0x67,0xa3,0x88,0x1f,0x02,0x8a,0x87,0x80, +0xe2,0x37,0x4d,0x2a,0x5a,0x94,0xa2,0xf6,0x55,0xad,0xb8,0x37, +0x04,0x89,0x1b,0xa3,0x27,0xe7,0xb0,0xba,0x88,0x17,0xa4,0x27, +0x9d,0xa4,0xbf,0xc1,0x44,0xad,0xb5,0x23,0x68,0xc2,0xb2,0x4c, +0xd0,0x78,0x4e,0xf2,0x4b,0xa3,0xe3,0x40,0x9b,0x15,0xb3,0x2c, +0x7d,0x45,0x89,0x3f,0x62,0x08,0xb3,0x18,0x1b,0x02,0x0a,0xca, +0xc9,0x81,0x5e,0x16,0x71,0x31,0xe9,0x07,0x37,0x09,0x25,0xb8, +0xf5,0x10,0x54,0x45,0x59,0xea,0x87,0x42,0x71,0x69,0x62,0xfc, +0x98,0xa3,0xcc,0x1a,0xb1,0x62,0x52,0x42,0xb5,0x7c,0x53,0xfb, +0x65,0xbe,0x18,0x8d,0x33,0x18,0x5f,0x4e,0xf2,0xb3,0xc9,0xab, +0x09,0x0c,0x21,0x27,0x4d,0x8e,0x11,0x18,0x1b,0x05,0x26,0x32, +0x6e,0xde,0x4d,0x00,0xa1,0x2f,0x63,0xb2,0xac,0xfe,0x6f,0x55, +0x85,0x79,0x07,0xc2,0x92,0x64,0x8b,0xb5,0xb4,0x11,0xfd,0xb6, +0x2c,0x07,0x85,0x16,0x64,0xd6,0xc9,0x21,0xee,0xbc,0x73,0xd3, +0x94,0x94,0x59,0x92,0x45,0xe9,0xbf,0x29,0xf1,0x72,0x71,0x27, +0x60,0xe7,0x0e,0xc6,0xaf,0x78,0xea,0x36,0x42,0xc6,0x4f,0xa3, +0xd3,0x54,0xfe,0xf7,0xf4,0xe8,0xef,0xa3,0xba,0x99,0xcf,0x07, +0xc0,0x30,0x6d,0x2a,0x70,0x08,0xff,0xc1,0xa4,0x0d,0x43,0x62, +0xd2,0x0e,0x8d,0xb9,0x2d,0x85,0xdf,0xb6,0x31,0x65,0xd2,0x5f, +0x97,0x43,0xc7,0x2c,0xe4,0xd4,0x17,0xe9,0xac,0x08,0xe7,0xd5, +0xbe,0x7a,0xd7,0xf8,0x6a,0xd2,0xf8,0x6a,0xf4,0x46,0x6d,0xec, +0x34,0xbf,0xfa,0xf1,0xb7,0x5a,0xa1,0x96,0x04,0x4a,0xbb,0x7b, +0x4c,0xb0,0xac,0x7a,0xf1,0xfa,0x67,0x8e,0x91,0x76,0x78,0x58, +0xc7,0xe5,0x33,0xcd,0x74,0xb5,0x56,0x10,0x04,0x8d,0x20,0x84, +0xff,0xbf,0x09,0x82,0x36,0xfd,0xbf,0x09,0x49,0x80,0xb4,0x5b, +0xa0,0xe8,0x27,0x7a,0x7b,0xb9,0x5d,0x48,0x17,0x15,0x90,0x4d, +0xe2,0xa9,0x15,0x65,0x99,0x43,0xb4,0xc9,0x55,0x01,0xac,0xc8, +0x34,0x19,0x32,0x47,0xad,0x05,0x92,0xf4,0xa7,0xc1,0x4f,0x6c, +0xf3,0x77,0xc9,0xf7,0x91,0xd7,0x5e,0x55,0xd5,0x2c,0x67,0xdc, +0xb0,0xa8,0x00,0xa0,0xb3,0xb4,0x5d,0xce,0x03,0x65,0xdf,0x2d, +0x18,0x7b,0x9b,0x21,0x8c,0x4f,0xec,0x95,0x1c,0xa6,0x1d,0x43, +0x5a,0x61,0x2d,0x57,0x58,0x14,0xa2,0xa8,0x3a,0xab,0x68,0x9c, +0x1b,0x5d,0x64,0x95,0xcb,0xc2,0xdc,0xde,0x81,0xd9,0x8d,0xf7, +0x73,0x2d,0xb2,0x56,0x38,0xf4,0x9d,0x5b,0xb6,0xa8,0x8d,0x5d, +0x67,0x8d,0xc3,0x80,0xc5,0x35,0xce,0x84,0x54,0xc3,0xe7,0xf0, +0x08,0x37,0x0c,0xf2,0x3c,0x82,0xb4,0xcf,0xe3,0x51,0x0e,0xd9, +0xbe,0x3c,0x32,0xe7,0x22,0x7e,0xae,0x45,0x86,0x47,0xfc,0x5d, +0xda,0x6c,0x97,0x47,0x12,0xb0,0xc8,0xa3,0x2b,0x50,0xd2,0x7b, +0xed,0x56,0x21,0x00,0xce,0x65,0x51,0xd5,0x3e,0xca,0xf1,0x69, +0xd0,0x29,0x61,0xe5,0xc9,0x69,0xfc,0xe8,0xc8,0xcb,0x33,0x13, +0x53,0x1b,0x0a,0x25,0x5e,0xed,0x96,0xbf,0x3e,0x8b,0xc1,0x54, +0xc5,0x67,0x70,0x58,0x9d,0x75,0xf9,0x0e,0x37,0x0c,0x77,0xf1, +0xab,0x84,0x5d,0x2e,0x67,0x09,0xa8,0x44,0xf6,0x70,0xc6,0xfa, +0x42,0x3b,0xa4,0xa4,0x5a,0xf6,0xdc,0x1e,0x65,0xa3,0x35,0xb7, +0x33,0xb8,0xef,0x96,0x9e,0x3a,0x57,0xf3,0xdd,0x66,0x58,0xf2, +0x46,0x67,0x17,0x25,0x2d,0xcd,0x49,0x1b,0x81,0x69,0xa6,0x38, +0xf3,0x02,0x79,0xe7,0xe3,0x2b,0x8c,0x69,0xd4,0x03,0xed,0x21, +0x1e,0xc5,0xf8,0x90,0x35,0xfc,0x73,0x02,0xc6,0x13,0x0c,0x15, +0xf8,0xfb,0x48,0x3c,0x85,0xff,0xc2,0x27,0xf0,0x0f,0x76,0x2d, +0x3e,0x6b,0x0d,0xff,0xb6,0x42,0xfc,0xfd,0xf0,0xa4,0x05,0xff, +0xd2,0xbe,0x9c,0x08,0x1f,0xb7,0x30,0xe3,0x34,0x78,0x48,0xf9, +0x8f,0x9e,0xb6,0xe2,0x53,0xf1,0xf8,0xf1,0xe3,0x53,0xf8,0x73, +0x12,0x3e,0x82,0xb2,0xa7,0x66,0x5b,0x9c,0xea,0xa5,0x39,0xf7, +0x65,0xbc,0xbc,0x9a,0x8d,0xf0,0xdc,0xd9,0xd5,0x20,0x22,0xec, +0x8b,0x7c,0xd2,0xa3,0x62,0x52,0x58,0x92,0x76,0x12,0x58,0x69, +0x3c,0xd6,0x1c,0x6c,0x32,0xe9,0x51,0x09,0x54,0x49,0x9a,0x83, +0x0d,0xe5,0xcb,0xc1,0x45,0x09,0x27,0xf9,0x84,0xd3,0x42,0x91, +0x96,0x95,0x82,0x86,0x49,0x98,0xfb,0xb6,0xf3,0x71,0x37,0xc8, +0x25,0x98,0xcc,0xc4,0x62,0x8a,0x5d,0x2f,0x6e,0x82,0x89,0xb0, +0x5f,0xc6,0xe2,0xef,0xe5,0xa2,0x5f,0xb3,0x58,0x59,0xec,0x30, +0xf3,0xd7,0x64,0x50,0xb8,0x55,0x6c,0x70,0x99,0xec,0x6f,0x6a, +0x7e,0xe9,0x1d,0x48,0x5b,0x3c,0xcb,0xca,0x5d,0x3c,0xf8,0x6a, +0x54,0x5e,0x34,0x3c,0x88,0xac,0x15,0x71,0x69,0xe1,0x41,0x65, +0x61,0xb3,0xd4,0x39,0x3c,0xdc,0x03,0xd1,0x8b,0x07,0x5f,0xcd, +0xb7,0x20,0x52,0x0a,0xa7,0xb4,0x28,0x18,0x42,0xdb,0xda,0xae, +0x67,0x83,0x92,0xc2,0x78,0x32,0x56,0x5d,0xd4,0x4c,0xb6,0x25, +0x45,0xc1,0xf0,0xda,0x5a,0xab,0x73,0xaa,0xbf,0xa5,0xaf,0x23, +0x27,0xe3,0x7b,0x77,0x7d,0x56,0x10,0x0c,0x63,0xe9,0x15,0x86, +0x65,0xb3,0xb0,0xc3,0x6f,0x26,0x7f,0x37,0x58,0x43,0x61,0x61, +0x46,0xf9,0x5e,0x31,0xc4,0x44,0x65,0x5d,0x95,0xad,0xf1,0x37, +0x45,0x9d,0x65,0xf3,0x22,0xfa,0x44,0x8a,0xcd,0x7a,0x89,0x08, +0x5d,0x23,0x45,0xba,0x9c,0xcd,0xe1,0x9f,0xd8,0x5e,0x33,0xb2, +0x0a,0xf4,0xea,0x0c,0x51,0x37,0x20,0xea,0x05,0x7b,0x53,0x03, +0xad,0x72,0xad,0x25,0x21,0x25,0xe2,0x89,0xb1,0x54,0x58,0xd5, +0x3c,0x43,0x54,0x9a,0x96,0x89,0xaa,0xbd,0x37,0xe9,0x05,0xb4, +0xef,0x2d,0x26,0xe8,0xb8,0xbe,0xf1,0x0b,0x98,0xab,0xc6,0xa9, +0x34,0xd2,0xb7,0x0c,0x55,0x86,0xd8,0x36,0x5a,0x2d,0x3f,0x8d, +0x8a,0xd2,0xbb,0xc7,0xac,0x71,0x92,0xa9,0x40,0xb1,0x63,0xe4, +0x2a,0xb7,0x19,0x33,0x78,0x77,0x62,0xdc,0x31,0x84,0x2d,0x17, +0x99,0x2a,0x04,0xdb,0x07,0xb2,0xe3,0xf1,0x52,0x8e,0x62,0xfb, +0x70,0x76,0x9c,0x59,0xca,0x11,0x6c,0x1f,0xd4,0x05,0x6f,0x9d, +0x7f,0x29,0x0c,0x6c,0x2d,0x1a,0xbb,0x87,0xb5,0x91,0x22,0x27, +0x0a,0x7e,0x89,0xfc,0x16,0x47,0x36,0xc9,0xa6,0x33,0xba,0xc9, +0x8c,0xf8,0x93,0x23,0xdc,0xd4,0x58,0xd1,0x28,0xf9,0xf6,0x12, +0x86,0x99,0x03,0x80,0xf5,0xd5,0x02,0xdf,0xee,0x32,0xe3,0x67, +0x11,0xff,0xbe,0x8a,0xd3,0xa5,0xae,0x5c,0x7e,0xe3,0x85,0x8b, +0xf9,0x6c,0x9a,0xc6,0x6f,0xa0,0x20,0x8d,0xa5,0xe6,0x87,0xd4, +0x0e,0xe6,0xb9,0x5a,0x8c,0x05,0x86,0xc4,0xc1,0x4b,0xdd,0x36, +0xe5,0x50,0x01,0xe5,0xd5,0xd0,0xbd,0x22,0x19,0xd2,0xa6,0xf3, +0x31,0x16,0x25,0x6f,0x06,0xfc,0x61,0x8a,0x11,0x65,0x57,0xcb, +0xc9,0xf8,0x1e,0x68,0xb1,0x21,0xc7,0x58,0x86,0xf0,0xe1,0x8f, +0x1c,0xbe,0xf5,0x64,0xfc,0x59,0x0d,0x7d,0xfb,0xf2,0x47,0x6a, +0xe7,0xc6,0xf3,0xf3,0xbe,0xf8,0x93,0xd9,0x68,0x85,0xef,0x3e, +0x4f,0xdf,0xaf,0xc6,0xd0,0x8f,0xfc,0x89,0xa1,0x08,0x40,0xcd, +0x71,0x5a,0x4d,0x60,0x44,0x51,0xce,0xc0,0xf5,0x3b,0x06,0x01, +0x59,0x78,0xb5,0x5f,0x6c,0x88,0x1c,0xd2,0xe7,0xb3,0xe9,0x65, +0xf2,0x7e,0xb5,0x88,0x17,0xd1,0xa7,0x4d,0xc7,0x7c,0x35,0x93, +0x69,0x62,0x79,0x2f,0xcf,0x28,0xae,0xa8,0x18,0x52,0xbe,0x75, +0x43,0xda,0x42,0x0d,0x3f,0xe9,0x99,0x4f,0x8f,0x81,0xc4,0x5f, +0xae,0x96,0xcb,0xb9,0x20,0x4e,0x6a,0xac,0xf2,0xe1,0x38,0x09, +0x89,0x35,0x4a,0x36,0x9c,0xb3,0x27,0xfe,0x39,0xbf,0xde,0x66, +0x15,0x30,0xf4,0x42,0x0e,0x17,0xe7,0x08,0xc3,0x06,0xa6,0x07, +0x39,0xfd,0x0e,0x47,0x27,0x9e,0x0f,0x16,0x83,0x49,0x1a,0x9d, +0xb3,0xfb,0xf5,0xc8,0xfb,0x24,0xfd,0xc4,0x64,0x86,0x24,0xa5, +0x39,0xe2,0x3b,0xcd,0xbf,0x30,0xf3,0x5f,0x51,0x1e,0xbf,0x64, +0xd1,0x9c,0x90,0x18,0xbb,0x47,0xc9,0x7d,0x5f,0x9c,0x37,0x93, +0xf4,0x3b,0x72,0x50,0xb5,0xf1,0xe1,0x93,0x43,0xfc,0xfc,0xb6, +0x53,0x8b,0x6a,0x20,0x51,0x26,0x29,0x85,0xf2,0xaf,0x07,0x97, +0xb1,0x67,0xd7,0xe2,0x77,0x8b,0xc3,0x8e,0xd8,0xe6,0xe9,0x06, +0x70,0x98,0xe9,0x4f,0xc0,0xc6,0x36,0xfc,0x87,0x8e,0xfc,0x6d, +0x7b,0x3d,0x31,0xd8,0xaf,0x1c,0x2e,0x63,0x06,0xb8,0x13,0x32, +0x40,0x0c,0x1b,0x5f,0x53,0xc8,0x0f,0xc8,0x02,0x5d,0xda,0x40, +0xaf,0x81,0xb6,0xc2,0xd0,0x11,0xf1,0x00,0x3d,0x6e,0x66,0x73, +0x3a,0xae,0xc1,0x18,0x9e,0x8b,0xc1,0x30,0xae,0xf5,0x3b,0x4e, +0x6b,0x2c,0x01,0x99,0xc7,0x1c,0x34,0x56,0x13,0x84,0xd1,0xd2, +0xa7,0xcb,0x41,0x32,0x4d,0x3d,0xab,0x02,0xa1,0x01,0x5d,0x16, +0xcb,0xd9,0x7f,0x70,0x91,0xce,0xc6,0xa0,0xa8,0x5f,0x0d,0x96, +0xb0,0x7a,0x9b,0x46,0xc7,0xff,0xc0,0x66,0xa5,0xdd,0xf6,0xdf, +0x8f,0xff,0x7e,0x7c,0x9c,0x98,0xda,0x9f,0x49,0xc0,0xbf,0x2e, +0xc6,0xc5,0x4d,0x22,0x15,0xd3,0x4b,0x6e,0x59,0xe5,0x90,0xf2, +0xa5,0xb4,0x54,0xc5,0xfc,0x97,0x28,0x2f,0x80,0x0c,0x44,0x86, +0x1d,0xfd,0xd7,0x29,0x08,0x08,0xe8,0xb9,0x91,0xe7,0x66,0xfa, +0xdd,0x5a,0xad,0xed,0x26,0x09,0x1e,0x19,0xb8,0x0c,0xfc,0x46, +0x62,0xd0,0xe4,0xc0,0x92,0x5c,0xa6,0x69,0x92,0x72,0x95,0x41, +0xa3,0xd4,0xe5,0x39,0x0b,0xb8,0x6b,0x7e,0x37,0xd3,0xd5,0x05, +0x53,0xea,0x05,0xc2,0x4a,0xe6,0x7b,0x02,0x8d,0xd0,0x6f,0x9b, +0x44,0xc1,0xa1,0x90,0x65,0x15,0x20,0x07,0x8b,0xc1,0xf7,0x49, +0x3c,0x86,0x4e,0x2d,0x26,0xdd,0xdd,0xf5,0xfa,0x16,0xe9,0xdf, +0x59,0xc0,0x36,0xf9,0x56,0x7a,0xbe,0x09,0x36,0x7e,0x17,0xd2, +0xa5,0x43,0x0e,0xb5,0x1f,0xa0,0x1b,0x5d,0x6a,0x0a,0x19,0x77, +0x77,0x30,0x5a,0x0d,0x4d,0xdf,0x16,0x0a,0x6a,0xca,0x48,0xb8, +0x0b,0x14,0x15,0x6b,0xba,0x91,0xc1,0x75,0x6d,0x7a,0x6e,0x93, +0xe5,0x15,0x82,0x48,0xf4,0xa9,0xa3,0xe3,0x2c,0xe1,0x95,0xd8, +0x52,0x4c,0xae,0x6c,0x8b,0x11,0x9f,0x78,0x3a,0x9c,0x8d,0xe2, +0x17,0xa3,0xb4,0x5c,0x80,0x74,0xb6,0xdf,0x3d,0x08,0xda,0xf9, +0x44,0xbb,0x27,0x34,0x1e,0x4d,0x17,0x83,0xf9,0x9f,0x0a,0x35, +0xf1,0xaf,0x3c,0xa7,0x1d,0xa5,0x16,0x6d,0xc9,0x03,0x7e,0xc3, +0x58,0x6f,0x03,0xd3,0xe7,0xb3,0x94,0xff,0xae,0xe8,0x0f,0xc7, +0xdb,0xc1,0x5f,0xc3,0xd9,0x04,0xd6,0xa0,0xf0,0xab,0xa4,0x63, +0xdc,0x7a,0x34,0xb5,0xa4,0x02,0x43,0x41,0x7f,0x64,0x88,0xf5, +0x89,0xd2,0x2d,0x7d,0x21,0xf5,0x33,0xe7,0xde,0xdd,0x31,0xb0, +0x99,0x04,0x6c,0xce,0x49,0x0c,0x5d,0x59,0x9a,0x23,0x4b,0xd7, +0x98,0xa2,0x9a,0xdf,0x46,0x60,0xbe,0x62,0xc6,0x48,0x34,0x20, +0x23,0x0f,0xdb,0x4e,0x39,0x09,0x23,0xe4,0x3c,0x23,0xf3,0xcc, +0xe4,0x22,0xf5,0xf1,0xa7,0x6d,0xf3,0x03,0xc3,0xf4,0x19,0x7f, +0xba,0xf1,0xa5,0x5c,0x49,0xc6,0x2c,0xf6,0x65,0x7b,0x5e,0x98, +0x40,0xd1,0xe2,0x19,0x54,0x69,0x6a,0xc5,0x90,0x90,0x45,0x34, +0xf5,0x57,0x9c,0x50,0x35,0x20,0x24,0xb8,0x84,0x52,0xf8,0xf6, +0xa9,0xdf,0x6d,0xe2,0x5e,0x25,0x64,0x2a,0x33,0xeb,0xe7,0x9b, +0x78,0xb1,0x48,0x6c,0xe8,0x7c,0x7a,0x4e,0x11,0xbd,0xcc,0x17, +0x2b,0x8c,0x79,0x14,0xa8,0x99,0xc9,0xd7,0x13,0x5e,0x4f,0x3e, +0x0f,0x63,0x4c,0x62,0x3d,0x57,0x28,0x74,0x53,0xc6,0xee,0xd5, +0x78,0xce,0xae,0x09,0x83,0x07,0x26,0x72,0xf3,0x21,0x45,0x8d, +0x87,0x01,0x18,0xf5,0x55,0x4d,0x32,0x25,0x5c,0x15,0x53,0xa8, +0x31,0xca,0xc9,0x99,0xb0,0x1b,0x93,0xc6,0xe8,0x9d,0x2a,0x75, +0xd5,0xdd,0x5d,0x45,0x65,0xb2,0x51,0x07,0xee,0x18,0x39,0x6f, +0x5e,0xe2,0x4d,0x14,0x46,0x61,0xa4,0x19,0x77,0x75,0x95,0x2c, +0xc0,0x6f,0x77,0xa6,0x8d,0xa2,0xa8,0xcc,0xc6,0xa1,0xa7,0x16, +0x64,0xe5,0x60,0x31,0x3c,0xc7,0x08,0xf2,0xb3,0x45,0x54,0x48, +0xb9,0xbb,0xe3,0xe8,0xd6,0x56,0xa7,0xfd,0xd5,0x80,0x9b,0xc9, +0x83,0x1e,0xfe,0x49,0x2e,0x3d,0xa0,0xf8,0x6a,0xa0,0x4c,0x3a, +0x0b,0xcf,0xf7,0x03,0x7c,0x27,0x92,0x9f,0xf8,0xf1,0xfd,0xe5, +0xd5,0x62,0x76,0x4b,0x7b,0xd8,0xdf,0x2d,0x16,0x33,0xb0,0x59, +0xff,0xfa,0xcb,0x8f,0x0f,0x60,0xbe,0xbe,0x7a,0xc0,0xd7,0x69, +0xe3,0xd1,0x83,0x24,0x9d,0xfe,0xff,0x97,0x68,0x67,0x25,0xb0, +0xfa,0xcc,0xf5,0xb1,0x45,0x30,0xbf,0xcb,0x63,0x77,0x87,0x65, +0x20,0xbb,0x33,0x50,0x21,0x03,0x86,0x5b,0x32,0x6a,0xd7,0xa0, +0x02,0xb1,0xc0,0xa8,0x7d,0xed,0x1a,0xfd,0xa9,0x09,0x7e,0xe6, +0x45,0x59,0xb8,0xed,0x9a,0xfb,0x5d,0x13,0x16,0xa6,0xe7,0xb3, +0xb1,0x7a,0x8b,0xb6,0x56,0x9a,0x5c,0x83,0x95,0xc2,0x74,0x3a, +0x9b,0xc2,0xaa,0x64,0xfc,0x02,0x6a,0x3b,0x3f,0xb7,0xbf,0x6b, +0x22,0x5e,0x0e,0xde,0x3b,0x45,0xbf,0x83,0x04,0xbc,0xe8,0x33, +0xbe,0xfc,0x31,0x99,0x5e,0xb7,0x6b,0x57,0x8b,0xf8,0xb2,0x86, +0xb7,0x24,0xda,0x6c,0xab,0xc1,0x3f,0x3f,0x26,0x29,0x7f,0xe1, +0x8f,0x1a,0xe9,0xf1,0x1a,0xfc,0x53,0x63,0xcd,0x5e,0xc3,0x7f, +0x6b,0x4a,0xad,0x2b,0xb9,0x16,0xa8,0x0f,0xda,0xd2,0xd0,0x23, +0xeb,0xae,0x2d,0x8d,0x3c,0x21,0xad,0xbe,0xb6,0x31,0xff,0xe8, +0x0a,0x39,0xb6,0x7b,0x89,0x8e,0xd4,0x50,0x91,0x65,0xf8,0x43, +0xf7,0x53,0xdd,0x6e,0x12,0x51,0xf1,0xdd,0x38,0x46,0x77,0x76, +0x22,0x46,0xfe,0xae,0x89,0xc1,0x68,0x64,0x81,0xf2,0x08,0x69, +0xd7,0xca,0x52,0xa9,0xa6,0x57,0xc4,0x6b,0xdd,0x42,0xf3,0x09, +0x8c,0x1c,0xcf,0xf0,0x42,0x14,0xfd,0xa9,0xd9,0x8a,0xf2,0x97, +0x42,0x97,0x5b,0x0b,0xb0,0xb4,0xdc,0xa0,0x29,0x8a,0x89,0xd6, +0x2d,0x55,0x10,0xc2,0x20,0x73,0x45,0x0e,0x9d,0x88,0x31,0xfd, +0xcd,0x0c,0x5b,0x6d,0x3f,0x09,0x0a,0x89,0x02,0x24,0x7a,0x22, +0xac,0x97,0x5d,0xe6,0xfc,0x40,0x53,0x82,0x11,0x3f,0x11,0xa0, +0x99,0xc2,0x9a,0x15,0xf7,0x09,0x41,0xf7,0x24,0x23,0x0a,0xe4, +0x19,0x61,0x99,0xc2,0x62,0x09,0xd7,0x68,0xc9,0x60,0xec,0x19, +0x04,0xd6,0x7a,0x09,0x13,0x61,0x04,0x72,0xf9,0x1e,0x7e,0xf5, +0xf1,0x9e,0x96,0x42,0xe8,0x64,0x6c,0x74,0x45,0xbd,0xf3,0xe6, +0x78,0x00,0xe6,0xa8,0x85,0xb2,0xcf,0xba,0xc9,0x6d,0xe1,0x7b, +0xd9,0xc2,0xef,0x17,0xb3,0x49,0x55,0x1b,0xf7,0x6f,0x9d,0x5a, +0xd3,0xd2,0xb3,0x0b,0x54,0x34,0xd7,0x56,0x83,0xa4,0xa2,0x81, +0xf8,0xde,0x59,0x59,0xcb,0x7c,0xd3,0x34,0xbb,0x77,0x5e,0x14, +0xba,0x86,0x3a,0x25,0x19,0xe5,0xa5,0xc2,0xed,0xc9,0x4a,0x49, +0xc0,0x37,0x42,0x14,0x86,0x02,0xa3,0x5e,0x94,0x70,0x89,0xf9, +0xe3,0xd6,0x95,0xe7,0xe9,0xce,0xda,0x7c,0x6b,0xca,0xf9,0x15, +0x15,0xe3,0x8b,0x91,0x5b,0xc3,0x0b,0x13,0x3f,0xae,0x76,0x10, +0x45,0x9c,0x74,0x78,0x98,0x9b,0x45,0x24,0xa4,0x4c,0xa7,0x47, +0xef,0x64,0x92,0xcd,0x32,0x18,0xd3,0xa5,0x3c,0x5b,0x15,0x97, +0x37,0xfb,0x32,0x4d,0xe9,0x34,0xa1,0x11,0x15,0x78,0x07,0xb5, +0x7e,0x69,0xe6,0x39,0xb5,0x5a,0x93,0x5e,0x1a,0x3f,0xb7,0x14, +0x71,0xb9,0x1d,0x9f,0x03,0x02,0x6b,0x3e,0x6c,0x97,0x67,0xd9, +0xf3,0x63,0x0e,0xb3,0x6b,0xd3,0xe4,0x1b,0x92,0xa7,0xa3,0x7c, +0xf0,0xd9,0x20,0x55,0x0c,0xa2,0x88,0x42,0x36,0x2a,0x4c,0xee, +0x55,0xf2,0xc5,0x06,0xed,0x0b,0x98,0x9e,0x57,0x54,0x28,0x2f, +0x61,0xde,0xd0,0x61,0x80,0x33,0x99,0x95,0xc9,0xbc,0x3b,0x9e, +0x15,0x5a,0x7b,0x76,0xa6,0x8d,0x2f,0x5a,0x40,0x0e,0x2d,0x93, +0xa3,0x90,0x71,0x77,0x57,0xbe,0xf3,0x41,0xdb,0x1a,0xae,0xf2, +0xcf,0x61,0x34,0x7c,0x51,0x49,0x25,0xca,0x3f,0x57,0x46,0x83, +0xba,0x36,0xb0,0x85,0xff,0x05,0x1e,0x30,0x0f,0xe3,0x39,0x42, +0x6f,0xa9,0x5d,0x35,0xf4,0x12,0xc6,0x95,0x5c,0x08,0xd8,0x25, +0xb7,0xe6,0x5a,0x4d,0x8e,0x79,0xfa,0x34,0xfb,0x27,0x14,0x4d, +0x9d,0xf6,0xd7,0x94,0x61,0x2f,0xf7,0xa4,0x64,0xd3,0x3e,0xc9, +0x12,0x6d,0x55,0x52,0x82,0xb5,0x5d,0xf0,0xb6,0x5c,0xce,0xb8, +0x2d,0x2c,0x90,0xa9,0xe9,0x48,0x4c,0x62,0x9e,0x89,0x15,0x2d, +0x74,0xd5,0xc5,0xfe,0xf4,0x6f,0xa5,0x57,0xa8,0xd6,0x59,0xf4, +0x54,0x55,0x80,0xe7,0x3b,0xf9,0x4e,0xfc,0x7e,0x07,0xad,0xf7, +0x6f,0xa6,0x55,0x22,0xb7,0x13,0x83,0x16,0x6d,0x49,0x97,0xe7, +0xd3,0xad,0xce,0xf6,0x9d,0x65,0xdf,0x77,0x79,0x04,0xfb,0x50, +0x59,0xa8,0xb5,0x92,0xbe,0xd9,0xf4,0x9b,0xf8,0x72,0xb6,0x88, +0x71,0xb0,0x5a,0x86,0x53,0xf2,0x31,0xd6,0x83,0x7f,0x0b,0x48, +0x4e,0x44,0x35,0x19,0xf8,0x61,0x37,0xe2,0xe7,0x2d,0xb5,0x98, +0x29,0x7d,0x66,0x6d,0x8d,0xef,0x41,0x1e,0xc2,0xe7,0x1b,0xb3, +0xb5,0x19,0x7f,0xaa,0x01,0x9f,0x41,0xfa,0x0e,0xa2,0x0d,0x76, +0x34,0x68,0xf1,0xa4,0xe0,0x05,0x6f,0x6a,0x44,0x8e,0x28,0xa8, +0x69,0xf5,0x6a,0xb6,0x1a,0x8f,0x5e,0x0f,0x6e,0x62,0x36,0x82, +0xa3,0x8a,0x74,0x47,0x90,0xe4,0xba,0x31,0xb0,0x2b,0x63,0xb0, +0x71,0x9c,0x56,0xef,0xa9,0x99,0xed,0x15,0x99,0xd1,0xad,0x22, +0xc2,0x58,0xd6,0xb8,0x56,0xd2,0x15,0x5a,0xdb,0xbf,0x72,0x1b, +0x8e,0xf3,0x37,0xb4,0x73,0xf3,0xcd,0x6c,0x36,0x8e,0x07,0xba, +0x56,0x0c,0xcf,0xb5,0x0b,0xbd,0xc6,0xcc,0x65,0x36,0x79,0x93, +0x7b,0x75,0x79,0x99,0xac,0xcb,0x67,0x6d,0xce,0x93,0xfe,0x80, +0x4e,0x5a,0x51,0xe9,0xbd,0x66,0x3c,0xf6,0x2e,0x28,0x27,0x15, +0x8c,0x1c,0x06,0xd4,0xf9,0x2e,0x39,0x3a,0x98,0xaf,0xb5,0xf7, +0x61,0xa7,0xd9,0x3b,0x3b,0xb0,0xf4,0x91,0x8b,0xa3,0x37,0x06, +0x22,0xf7,0xb4,0x21,0x1f,0x1c,0x3e,0x5b,0xbc,0xc7,0x6a,0x16, +0xf8,0x43,0x3e,0x5f,0x98,0x9a,0x95,0x25,0x87,0x4d,0xb2,0x6a, +0xe1,0xb7,0x17,0x5a,0xb0,0xe2,0x2f,0x44,0x98,0xb1,0xa1,0x34, +0xee,0xb6,0x67,0x27,0xab,0x8a,0x84,0x53,0x87,0x06,0xe6,0x18, +0xd1,0x48,0xdc,0x9b,0x1d,0x6d,0xed,0x21,0x50,0xbf,0x93,0x07, +0xc5,0x57,0x00,0xf7,0x2e,0x8d,0x6f,0xe5,0x89,0x3c,0x38,0x6f, +0xd4,0x68,0x46,0x0d,0x67,0x18,0xd7,0xd2,0x1e,0xc3,0xda,0x9e, +0xb5,0x9b,0xe0,0xe3,0x26,0x0b,0xbe,0xb1,0x66,0x27,0xda,0x0c, +0x61,0x3d,0x40,0xf3,0x26,0x86,0x4d,0x92,0xbd,0xc4,0x6b,0x41, +0x8b,0x13,0xae,0xe0,0x8b,0x4b,0x73,0xf4,0xb1,0xad,0x5f,0x25, +0xf4,0x41,0x80,0x05,0x72,0xc8,0x5f,0xce,0x46,0xf1,0xf8,0x4f, +0xe2,0x0d,0x19,0x6f,0x9e,0x91,0x65,0x8b,0x1d,0xab,0xf9,0x3c, +0x38,0x85,0xa5,0xaa,0xa4,0x07,0xe2,0x3e,0x1d,0x44,0x65,0xfb, +0x42,0x06,0x48,0xa4,0x9a,0xec,0xe5,0x6a,0x1e,0xc7,0xe1,0xa1, +0x5c,0xd4,0xe5,0x33,0xac,0x88,0x5f,0x26,0xd5,0xff,0x64,0xe3, +0xb5,0x7b,0xc9,0xa1,0xdf,0x02,0xc2,0x25,0xdf,0x36,0x05,0xec, +0x59,0xb0,0xbb,0x98,0xb0,0x71,0xad,0x37,0x36,0xed,0xca,0x95, +0x8c,0x0d,0x61,0xaf,0x0b,0xec,0xf4,0x82,0x01,0x22,0x11,0x9a, +0xc5,0x33,0x46,0xb0,0x29,0xb5,0x37,0x34,0xe4,0x38,0x77,0xd0, +0x53,0xd8,0x52,0xc3,0x63,0x55,0xec,0x3b,0x3c,0x16,0x2a,0xec, +0xcb,0xc1,0x74,0xd2,0xb1,0x32,0x9a,0x4e,0x40,0x02,0x3e,0xfe, +0x8d,0xac,0x01,0xc5,0xc7,0xbd,0x76,0x5c,0x0d,0x4e,0x93,0x72, +0x20,0x29,0x29,0x47,0xc8,0x9b,0x64,0xf9,0xf9,0x6c,0xb8,0x5a, +0x60,0xb2,0x79,0x85,0x53,0x82,0xc1,0x00,0xef,0xc8,0xbc,0x8e, +0x2f,0xd3,0x78,0x7c,0xab,0x12,0x42,0xfe,0x88,0xe4,0xdf,0x9e, +0x45,0x4f,0xc9,0x72,0xc6,0xdd,0xa3,0xd3,0xb1,0x07,0x15,0x6a, +0x0c,0x6e,0xbd,0xa0,0x8d,0xcf,0x0a,0xf2,0xf5,0x29,0x75,0x9e, +0x74,0x79,0xb2,0x8d,0xe7,0xfd,0x7a,0xd3,0x1b,0x7e,0xcb,0xdd, +0xfd,0xdb,0xab,0xc1,0x92,0xf6,0xf0,0x84,0x75,0x24,0x2a,0xdb, +0xc9,0x47,0xd3,0xfa,0x64,0xc9,0x94,0xa2,0xd9,0xc0,0x6e,0x4e, +0xd9,0xc1,0x41,0x93,0xcf,0x3e,0x7c,0x65,0x2f,0xe7,0x51,0xd9, +0x67,0x06,0x25,0xb8,0x64,0xb6,0xdf,0x41,0xe2,0xcc,0x84,0x2b, +0x8f,0xa5,0x0d,0xb1,0x5d,0x89,0xbe,0x57,0x7b,0x71,0xd9,0xf8, +0x69,0x36,0x8d,0x1b,0x2f,0x69,0xd7,0xaf,0x1f,0xd1,0xce,0xa4, +0x9d,0x6b,0x67,0xf0,0x74,0x00,0xc2,0xc8,0x91,0x14,0xf1,0x84, +0x53,0xf7,0x9c,0xe2,0x3d,0xe4,0xd6,0x23,0x64,0x50,0xb7,0x76, +0x5c,0xab,0xe3,0x0f,0xbc,0xc2,0x48,0xa9,0x36,0xc1,0x3c,0xb3, +0xde,0xdd,0x41,0x5e,0x99,0x7f,0x81,0x05,0x6a,0x39,0x19,0x7c, +0x52,0xdb,0xa0,0x36,0xa6,0xfc,0x61,0x9f,0xf7,0x69,0x22,0xb7, +0x1b,0xff,0xf5,0xbb,0x37,0xb5,0xdc,0x8a,0x22,0xb7,0xde,0x00, +0x2d,0x82,0x7b,0xac,0x5f,0x10,0x1d,0x6e,0xcb,0xee,0x87,0xee, +0xd5,0x5f,0xf7,0x41,0x37,0xdb,0xbb,0xb5,0xaf,0x7e,0x7e,0xbd, +0x07,0x42,0xb9,0x41,0xbc,0x1f,0xca,0x6f,0xbf,0xfb,0xf1,0xbb, +0x37,0xdf,0xed,0x46,0x4a,0xbb,0xcc,0xfb,0xa1,0xfc,0xe1,0xbb, +0x67,0xdf,0xee,0x46,0xc8,0x5b,0xd5,0xfb,0x61,0x7c,0xf3,0xcb, +0xb3,0xe7,0x7b,0xd0,0xa8,0xf6,0xbb,0xf7,0x43,0xfa,0xf3,0xab, +0x37,0x2f,0x7e,0xfe,0xe9,0xf5,0x1e,0x1d,0x44,0x9b,0xe6,0x7b, +0xf6,0xd0,0xb3,0x37,0xcf,0x7f,0xd8,0x89,0x72,0x23,0xdd,0x24, +0xf0,0x64,0xc4,0xd5,0xe9,0x98,0x62,0x85,0xc3,0xc2,0x93,0x14, +0x4b,0xad,0x09,0x37,0x9b,0x46,0x67,0x89,0x4a,0x46,0xd4,0xe7, +0xe7,0x48,0x30,0x8d,0x47,0xb3,0x07,0xbb,0x88,0x47,0x2b,0x35, +0xf2,0x6c,0xb5,0x6e,0x74,0xb3,0x46,0x36,0x18,0xae,0x26,0xd6, +0xee,0x2f,0xfe,0x42,0xdf,0x04,0xfc,0xfb,0x5a,0x6e,0xbc,0x45, +0x5c,0x47,0xb3,0x74,0x97,0x4f,0x6e,0x17,0x25,0x97,0x9e,0x5d, +0x84,0x8e,0x92,0xdc,0x62,0x8e,0x0f,0x88,0x0b,0xec,0x5b,0x4b, +0x38,0x95,0xd8,0x91,0x94,0x44,0x76,0xe2,0x46,0x85,0x35,0xb6, +0x33,0x7b,0x6e,0x3d,0xc5,0x99,0x45,0xda,0x3b,0x07,0xfb,0x01, +0x97,0x9d,0x01,0xf5,0x0d,0x73,0x5e,0x8c,0x3a,0xfc,0x27,0xc7, +0x95,0xdc,0x36,0x60,0xb7,0xc0,0xb3,0xb2,0x1d,0x40,0x69,0xb9, +0x16,0x60,0xf3,0x10,0x85,0x6d,0x63,0xb3,0x07,0xac,0x58,0x51, +0x8f,0x50,0x47,0xe7,0x78,0xae,0x7d,0x0f,0xba,0xfc,0xeb,0xaf, +0xbf,0xbc,0x78,0x3e,0x9b,0x80,0x65,0x02,0x32,0xa0,0x50,0xb4, +0xe5,0x5f,0xfd,0x00,0x34,0x4a,0x44,0x1d,0x91,0x49,0xcc,0xee, +0xec,0xa4,0x5c,0x6b,0x36,0x79,0x19,0xbd,0x8c,0x61,0xf4,0x38, +0xfe,0x34,0x6a,0xfe,0xc5,0x89,0x83,0x19,0xa8,0xfc,0x68,0xaa, +0xa7,0x1c,0x84,0x85,0x46,0x49,0x40,0x6e,0x14,0x95,0x17,0x32, +0xa9,0xda,0x8e,0x6a,0xd2,0x63,0xc0,0x48,0x15,0x0f,0xba,0xf7, +0xe3,0xd9,0xc5,0x60,0xcc,0x36,0x92,0x7a,0xa5,0xbe,0xe8,0x33, +0xc7,0xe1,0x89,0x4b,0x60,0x79,0xbd,0xda,0xfc,0x0b,0x46,0x70, +0xef,0xd5,0x68,0xe2,0xaa,0x89,0xda,0x5f,0x7e,0xb7,0x5c,0xf2, +0x78,0x36,0xfb,0xcb,0xef,0x96,0x83,0x1d,0xbf,0xe4,0xfd,0x3a, +0x5e,0xdc,0x24,0x43,0xf4,0x05,0x75,0x50,0x6a,0x03,0x4d,0x83, +0x2f,0x1c,0x53,0x17,0x35,0x80,0xc7,0xe3,0x95,0xb7,0xd8,0xe5, +0x12,0x5d,0x0a,0x7c,0xf5,0x56,0x30,0x4b,0x78,0xc4,0xf6,0xf0, +0x76,0xd0,0xc2,0xc1,0x5f,0x1f,0xec,0x12,0x8a,0x8f,0x08,0x0c, +0xfd,0x61,0x30,0x1d,0x8d,0x81,0x31,0xaa,0x2f,0x85,0x49,0x63, +0x1d,0xb1,0x03,0x79,0xd9,0xa1,0xa0,0xc6,0x5f,0x96,0xf9,0xbd, +0x64,0xc4,0x5e,0xd8,0xe9,0xdc,0x50,0xa3,0xc3,0xb3,0xa7,0xdc, +0x4a,0x41,0xae,0xb2,0x98,0x77,0x06,0x63,0x13,0xaf,0x64,0xc9, +0x52,0x78,0x15,0x34,0x9f,0x0d,0xb6,0x98,0xa6,0x51,0xae,0x48, +0x73,0xa5,0xd9,0xd1,0x4c,0x21,0x50,0xfa,0x31,0x87,0xc3,0x02, +0xe2,0x2f,0x1b,0x88,0x7b,0xf5,0xf0,0xb0,0x62,0xdb,0x44,0x6e, +0xc6,0x68,0xcb,0x13,0x92,0xcc,0x56,0xbe,0xab,0x10,0x38,0x57, +0x61,0xc4,0x4a,0xcb,0xf5,0xb2,0x82,0x2b,0xf0,0x11,0xa4,0x52, +0xee,0xd5,0x9c,0x37,0x57,0x53,0x14,0x4b,0x19,0x7b,0x2f,0x85, +0x1f,0xf3,0x64,0x78,0x5d,0x7d,0x7e,0xda,0x53,0x07,0xec,0xf9, +0x13,0x75,0x8c,0xd4,0xaa,0x74,0x8e,0xe5,0x83,0x96,0x3b,0x88, +0x8f,0x24,0x7e,0x29,0xe2,0xd5,0x94,0xf9,0x9d,0x82,0xc2,0x53, +0x4c,0xf1,0x0d,0x0f,0xcd,0xb1,0xa0,0xe7,0xd6,0x23,0x34,0x70, +0x51,0x71,0x6a,0xb6,0x39,0x88,0xf4,0x61,0x59,0x39,0x26,0x04, +0xdf,0x21,0x9c,0xb9,0xe5,0x4b,0xe4,0x7e,0xf3,0xc4,0x75,0x3f, +0x0c,0xf6,0x43,0x9c,0xb4,0xb7,0xa1,0xb5,0x06,0xc8,0xa0,0xe6, +0x21,0x0d,0xf9,0x64,0x64,0x26,0x28,0xcb,0x4b,0xb7,0x84,0x51, +0xfa,0xd0,0xd3,0xd5,0x3c,0x56,0xd3,0x6d,0xcd,0x63,0xaa,0xc4, +0x67,0x4c,0xed,0x2a,0x2d,0x87,0x68,0x1b,0x8f,0x99,0x32,0x15, +0x34,0xfa,0xa7,0xc9,0x1d,0x14,0xdf,0x69,0x01,0x5a,0x01,0x4e, +0x23,0xe8,0x60,0x72,0x6b,0x2b,0x4c,0x2f,0x99,0x03,0xc8,0xfb, +0xb7,0xe3,0x3f,0xa4,0xe2,0x62,0xc3,0x4d,0xf5,0x6e,0xd3,0xdd, +0x72,0xaf,0x60,0xb8,0x26,0xa8,0xf5,0xf9,0xaf,0xb3,0xbf,0xa1, +0x79,0x2c,0x33,0xe9,0x31,0x59,0xa5,0x62,0x64,0xda,0x73,0xd4, +0x58,0xf2,0xb7,0xaf,0x7e,0xa0,0x2a,0xc8,0xc1,0xfd,0x6b,0xbc, +0x2c,0x80,0x6d,0x99,0x4f,0x4a,0x8d,0x20,0x67,0x03,0xce,0xa1, +0x14,0x5f,0xd3,0x91,0x55,0xe3,0x16,0xc0,0x16,0x1a,0x41,0x73, +0x60,0x24,0x74,0x5f,0x13,0x61,0x18,0x63,0x41,0x6b,0xff,0x3e, +0xba,0x61,0x14,0x5f,0xc6,0x60,0x24,0x8c,0xa2,0xbf,0xfc,0xde, +0xa4,0xdf,0x9e,0x4f,0x0b,0xf8,0x67,0x8b,0xf7,0xa9,0xd9,0x28, +0xed,0xd8,0x9b,0x1e,0xcb,0xab,0x78,0xea,0xd9,0x5b,0xe5,0xce, +0x92,0x9e,0x4c,0x61,0x7b,0x27,0x05,0xdf,0x96,0xe0,0x97,0x7a, +0x15,0x5e,0x8c,0xb4,0x87,0xe5,0xf1,0x34,0x57,0x39,0x12,0x76, +0x30,0x41,0xc6,0x7f,0x84,0x64,0x75,0xfc,0x25,0x14,0x7d,0xb4, +0x03,0x31,0xbe,0x21,0x97,0x71,0x7f,0x93,0x17,0x10,0xd5,0xd1, +0x1a,0x5a,0x31,0x04,0x09,0xbe,0x6f,0x4f,0xf8,0x05,0xae,0x41, +0xff,0x7a,0xc6,0xce,0x2a,0x61,0xd9,0x2e,0xfe,0x94,0xb5,0xa2, +0x87,0x18,0xfb,0xff,0x79,0x6d,0x91,0x55,0xe7,0xd1,0x0b,0x75, +0xda,0x2b,0x9c,0x73,0xe4,0x92,0xed,0xb6,0x6e,0xa1,0x15,0xda, +0x27,0x48,0xe3,0xf8,0x64,0xfb,0xd6,0xb7,0xbd,0x42,0x09,0xaa, +0x43,0xf0,0x73,0x4a,0x16,0x6d,0xe8,0xd5,0x3d,0xb7,0xec,0x0d, +0x2f,0xb7,0x71,0x0d,0x24,0xb3,0x49,0x2d,0xac,0xd9,0x94,0x12, +0x2b,0x67,0x53,0x74,0x5c,0xf3,0xed,0xa7,0xbc,0xc0,0x42,0x7a, +0xbe,0x4a,0x97,0xb3,0xc9,0xcf,0x6a,0x77,0x47,0xd6,0x42,0x26, +0xc6,0xd0,0xcd,0xd2,0x76,0x10,0x25,0xe7,0x0c,0x29,0xb9,0x63, +0xdb,0x63,0xaf,0x32,0xe9,0x50,0x26,0xaf,0x0a,0x48,0x6f,0xca, +0xbe,0x70,0x2e,0x03,0xf8,0x9f,0x74,0x99,0xd1,0x0c,0x1f,0x20, +0x25,0xb4,0x36,0x14,0x4c,0xa0,0x83,0x54,0x79,0x29,0x8c,0xc7, +0xba,0x46,0xfc,0x30,0x54,0x29,0xf4,0x51,0x14,0x21,0xda,0xae, +0x72,0x5f,0x6b,0xe3,0x17,0x79,0x0f,0x46,0x84,0xa8,0x8e,0xdf, +0xcd,0xe5,0xec,0xaf,0xf3,0xb9,0x72,0x69,0xec,0xd8,0x68,0x23, +0x22,0xfd,0x20,0x8a,0x1c,0xf4,0x87,0x87,0xdc,0x96,0x7c,0x7a, +0xd7,0xe5,0x42,0xbb,0x78,0xcf,0x81,0xf5,0x31,0x1d,0x31,0xbb, +0xab,0x75,0xab,0x0f,0xcb,0xf8,0x49,0xef,0x82,0x78,0xb9,0x43, +0x6a,0x17,0x83,0x72,0x4c,0xa1,0xe9,0x1d,0x5b,0x68,0x6c,0x54, +0x9b,0x4f,0x44,0x80,0x43,0x35,0xee,0x0f,0x48,0xfb,0x91,0x6b, +0xfd,0xd7,0xef,0xde,0xfc,0xf8,0xe2,0xf5,0x1b,0x55,0x9c,0xcc, +0xf0,0x12,0x13,0xb9,0x39,0x9a,0x29,0xc8,0x62,0x61,0x23,0x4e, +0xd5,0x16,0xb2,0x77,0x29,0xad,0x44,0xbd,0x8d,0x00,0xb0,0x89, +0xce,0x76,0xfd,0xbb,0x6c,0xd8,0x4e,0xe9,0xac,0x2f,0xfd,0x77, +0x2c,0x0c,0x7b,0x5b,0x39,0xc2,0x29,0x26,0xf6,0x44,0xc2,0xeb, +0x9d,0xca,0x49,0x35,0x6f,0x05,0x98,0xa9,0x7e,0x68,0x4d,0xae, +0x7a,0xc6,0xdf,0x7d,0x70,0xad,0xbd,0x39,0x0e,0x42,0x69,0x51, +0x88,0x31,0xde,0x2d,0xa5,0x33,0x8e,0xbd,0xd6,0x70,0x79,0x07, +0x58,0x77,0xb7,0x00,0xe6,0x4f,0x8d,0x6f,0x4f,0x87,0x9f,0xf2, +0xc5,0x81,0xc6,0x42,0xf6,0xb8,0x35,0x3d,0xef,0x81,0xde,0x71, +0xd6,0xec,0x97,0xdc,0x30,0x34,0x08,0x37,0x56,0xfb,0xef,0x6d, +0x45,0x00,0x0f,0xf7,0xa3,0x46,0x0f,0x23,0x8a,0x20,0x25,0x47, +0x81,0x69,0xe1,0x9e,0x58,0xb8,0x35,0xa5,0x23,0xea,0x7e,0xb8, +0x40,0x1f,0x69,0x3c,0xf0,0xfb,0x73,0xb1,0xcc,0x2c,0x72,0xf0, +0xe3,0x33,0xf1,0xb0,0x5a,0xd5,0x98,0x58,0xeb,0x7e,0x26,0x2e, +0xd4,0x63,0x1a,0x13,0x7e,0x7c,0x26,0x1e,0xda,0xea,0xd5,0x88, +0xe8,0xeb,0x33,0x31,0xc9,0x1d,0x5e,0x8d,0x4b,0x7e,0x7f,0x2e, +0xcf,0x71,0x63,0xd7,0x30,0x7d,0x50,0x21,0x03,0x25,0xd3,0xaf, +0x95,0x5b,0x76,0x42,0x6b,0xf2,0xb5,0x66,0x10,0x29,0xef,0xfd, +0x54,0x2a,0xa7,0xf2,0x95,0x82,0xb5,0xb0,0xfa,0x2c,0xbd,0x14, +0xfc,0x49,0xbd,0xf4,0x27,0x86,0x73,0xf0,0x85,0xc4,0x5e,0x3e, +0xca,0xf6,0x7f,0xa5,0xbc,0x1a,0x7f,0x79,0x5b,0x45,0xc8,0xa4, +0xff,0x9b,0xc6,0xc0,0xfd,0x35,0xf3,0x37,0xd9,0x8b,0xd1,0x7f, +0x8c,0x56,0x96,0x02,0xf1,0xe7,0x86,0x65,0x50,0x39,0x2c,0x25, +0xed,0x5e,0x82,0x0b,0x8c,0xdf,0x5f,0x95,0x1b,0x85,0xbc,0xf5, +0xad,0xec,0x2b,0x80,0xd5,0x61,0xc6,0xe8,0xa9,0xcd,0x7c,0x31, +0x6b,0x5d,0x56,0xe8,0x7e,0x28,0xbc,0x2e,0xd8,0x9e,0xb9,0x23, +0x1c,0x1a,0x9a,0x6f,0x66,0xaf,0x56,0xfc,0x8c,0x68,0x0f,0x8a, +0xf4,0xc5,0x96,0xd5,0x9c,0x06,0x47,0x09,0xf4,0x72,0xc8,0x73, +0xcb,0x3c,0xe4,0x40,0xbc,0x20,0xae,0x51,0xb5,0xd3,0xf8,0x96, +0xcf,0xfa,0xb7,0xd8,0x89,0x4c,0x9a,0xdf,0x51,0xb0,0x44,0x50, +0x64,0x30,0x15,0x17,0xbf,0x0a,0xd2,0x0e,0xa4,0xa4,0xd6,0x5e, +0xce,0x2a,0x13,0x1d,0x2b,0x4c,0xce,0xfe,0x6b,0x4c,0x67,0x33, +0x85,0xa2,0xce,0xa9,0xd5,0x1f,0x62,0xfb,0x16,0x56,0x6f,0x96, +0x4b,0xa9,0x74,0xc9,0x51,0xdb,0xd9,0x66,0x21,0x29,0xb1,0xea, +0x00,0x75,0x83,0x4a,0x37,0xe6,0xcf,0xc1,0x4a,0x0e,0x05,0x91, +0x4a,0x6f,0xca,0x0e,0xf1,0x6a,0xdf,0xbd,0x19,0xbc,0xaf,0x59, +0xe1,0xeb,0x97,0x83,0xc3,0x43,0x79,0xbc,0x8f,0x1f,0xd5,0x23, +0x06,0x81,0xe4,0xb9,0x3d,0x2d,0x82,0x0d,0x07,0x9c,0xb1,0x43, +0x5b,0x00,0x65,0xd2,0x9c,0x17,0xb3,0xb2,0x3d,0x95,0x99,0x59, +0xc3,0xe9,0x9b,0x49,0xe8,0x1f,0x50,0x72,0x2e,0xc0,0x87,0x26, +0x7c,0x2e,0x83,0xff,0x7e,0x8f,0xe9,0x80,0x0d,0x7f,0xdf,0xdd, +0x71,0x45,0xbb,0xac,0x77,0x21,0x6f,0x60,0x6e,0xf7,0xa8,0xf6, +0xf8,0x81,0x76,0xcd,0x7a,0xab,0x36,0xdb,0x17,0x99,0x9c,0x28, +0x74,0xbb,0xf1,0xcb,0x76,0x62,0x50,0xe4,0x2f,0x6c,0x97,0x04, +0xe9,0x8b,0xa0,0x02,0x3a,0xa8,0xb5,0x9c,0xfa,0x9e,0x5b,0xee, +0x21,0xdb,0x9b,0x43,0x7d,0x63,0x79,0x8f,0x28,0xf5,0xe6,0xe5, +0x07,0xa0,0x19,0x06,0x32,0xaa,0x02,0x8a,0x95,0x91,0x12,0xec, +0x57,0xea,0xdc,0x68,0x97,0x54,0xe7,0x99,0x50,0x22,0x80,0x30, +0x5a,0x86,0x31,0x3e,0xd2,0x4b,0x75,0x9c,0x73,0x20,0x1d,0xc4, +0x52,0xee,0x3c,0xbb,0xb3,0xcb,0x4a,0xe7,0xfe,0x6e,0xc9,0xb2, +0x6b,0x17,0xa6,0xfc,0xfa,0x8f,0xb7,0x4c,0xf6,0x93,0x18,0xda, +0x9a,0x69,0x57,0x56,0xca,0xb8,0xa8,0xc7,0x08,0x12,0x64,0x20, +0xcf,0x06,0xb9,0x19,0x44,0x9c,0x70,0xf2,0xfc,0xdd,0x34,0x54, +0xb2,0x60,0xfb,0x9e,0x55,0xa5,0xf5,0x47,0x92,0xed,0x10,0xb1, +0x27,0x1f,0x7c,0xe2,0xc1,0x67,0xd6,0x2a,0x59,0xe5,0xd6,0x4b, +0x23,0xb9,0x5c,0x63,0x57,0xb8,0xa7,0x1b,0x88,0x83,0x08,0x56, +0x74,0x18,0x57,0xe3,0xcb,0x6a,0x76,0xec,0xcb,0xef,0x8b,0xbb, +0x38,0xd4,0xbb,0x72,0x6c,0xe2,0x4d,0xff,0x7b,0x69,0x38,0xa0, +0xe3,0x95,0x75,0xa1,0x5d,0x6a,0x0d,0xe4,0xa9,0xd4,0x5c,0x7b, +0x49,0xa1,0x3e,0xe8,0xde,0xa1,0x18,0x69,0xab,0xe7,0xe2,0x03, +0xde,0x35,0x61,0xcc,0x3c,0x2b,0xc8,0xe4,0x1d,0xfa,0xa4,0xa3, +0x4a,0x17,0xf6,0x1d,0x65,0x06,0xfb,0x6f,0xed,0xa7,0x43,0x65, +0x91,0xea,0x19,0xcc,0xd5,0xa2,0xa9,0xa5,0x45,0xc5,0xec,0xfa, +0xb9,0x0c,0x94,0x13,0xdd,0x4b,0x91,0xd1,0x0a,0xe6,0x4f,0x4f, +0xcf,0xe4,0x6b,0xd1,0xd5,0x7b,0x7e,0xba,0xe4,0x1e,0x93,0xcc, +0x17,0x19,0xae,0xff,0xd1,0x2a,0xed,0xfe,0x43,0xb9,0x54,0xdf, +0xed,0x83,0x46,0xed,0x68,0x0b,0x1a,0xca,0xdb,0x3a,0xf5,0xcf, +0x0f,0xf9,0xdc,0x9e,0xb0,0xe9,0x70,0xcb,0xef,0xd1,0x09,0xfa, +0x93,0x9b,0x85,0x7d,0x61,0xee,0xe4,0x1b,0x34,0xfa,0x2c,0x37, +0x1f,0x20,0xc0,0xcc,0xbd,0x26,0xaa,0x70,0xa1,0x38,0xba,0x50, +0xba,0x3b,0xd5,0xbc,0x2b,0x5e,0x4a,0x92,0xed,0x2c,0xfa,0xa9, +0xf6,0xb6,0xf1,0xc3,0x9b,0x37,0xaf,0x1a,0x5c,0x5b,0x43,0x61, +0xae,0xe9,0xab,0xd9,0x91,0x25,0x99,0x5d,0xe5,0x69,0xd7,0xd6, +0x49,0x1b,0xdf,0x3a,0x87,0xce,0x7b,0x6f,0x96,0x50,0xda,0xdd, +0x61,0xb2,0xd8,0xc4,0x95,0x99,0x2b,0xe4,0xcd,0xea,0x6e,0x34, +0xf7,0x9c,0xcf,0x3e,0xb0,0x5c,0x1a,0x27,0x66,0x78,0xbb,0x72, +0xe1,0xb7,0xff,0xe3,0x89,0xf8,0x2f,0x40,0x82,0x2a,0x26,0xf7, +0x51,0x76,0x51,0xb4,0xe7,0x4c,0xe6,0x2e,0x32,0xbf,0x77,0x42, +0xad,0x54,0x9e,0x3b,0xd8,0x13,0x9e,0x3c,0x75,0xe0,0x03,0x1b, +0x1e,0xb8,0x6a,0xd2,0x93,0x5f,0xc5,0x05,0xa7,0xbb,0x03,0xf8, +0x19,0x95,0xa9,0x38,0x03,0x7b,0xd6,0x67,0x6d,0x81,0x7e,0x46, +0x65,0x74,0x2e,0xb5,0x6f,0x4d,0xd6,0x76,0x11,0xaf,0x70,0xe4, +0x59,0xce,0xbd,0x2b,0xa5,0x01,0x6f,0x9b,0x10,0x84,0xa8,0x58, +0xa3,0xbd,0xa7,0xf4,0x19,0x8d,0xe3,0x18,0x0d,0x7b,0xb6,0xce, +0xd9,0x76,0xfa,0x8c,0xca,0x64,0x0c,0x88,0x3d,0x6b,0xcb,0x6d, +0x22,0x7d,0x46,0x7d,0x3a,0xc6,0xc4,0xbe,0xbd,0x67,0x6f,0x35, +0x79,0x9f,0xdb,0x73,0x1c,0xc8,0xc2,0xad,0xb2,0xa2,0xf3,0xdc, +0x03,0xbc,0x7d,0x8e,0xee,0x76,0x13,0x50,0x8e,0xa4,0x82,0x80, +0x6d,0x8e,0x6e,0x14,0x68,0x25,0x7f,0xe7,0x55,0x7a,0xfc,0xcb, +0x45,0x7b,0x2e,0x40,0x8f,0xfc,0x34,0xfb,0x36,0x48,0x63,0xac, +0x11,0x76,0xdc,0x4f,0xb3,0x76,0x77,0x26,0xa5,0xd2,0x8d,0x36, +0x5e,0x08,0x00,0x01,0x18,0x9d,0xa9,0xea,0xec,0x33,0x47,0x2b, +0x9b,0x9d,0xec,0xe5,0x68,0x2a,0xbd,0x2c,0x17,0x27,0xcb,0xaf, +0xd8,0xdc,0x9d,0xb0,0xaf,0x41,0x7c,0xda,0xea,0x3f,0x4d,0xc5, +0xf9,0x5a,0x97,0x50,0x90,0x2e,0xab,0x54,0x81,0x1c,0xcb,0xa8, +0x9c,0xc5,0x37,0xb3,0x19,0xe0,0x32,0xcb,0xba,0xd1,0xa1,0x03, +0x0e,0x9a,0x94,0xab,0xc2,0x65,0x91,0x26,0x1f,0xf4,0x76,0xc8, +0xb2,0xe3,0x83,0xde,0xca,0x79,0x3d,0xc7,0xa2,0xf6,0xe5,0x3e, +0x23,0x20,0x57,0xa8,0x94,0x9d,0x1b,0x23,0x68,0xe8,0x9e,0xee, +0x38,0x98,0x7e,0xef,0xde,0x04,0x32,0x21,0x1c,0xa7,0xf1,0xad, +0xbc,0x2c,0xe4,0x1c,0xed,0x96,0xb9,0xbd,0x2a,0x56,0xe5,0xfd, +0x64,0x35,0x0a,0xa1,0x7f,0xf9,0x56,0xa8,0x48,0xcf,0x4e,0xdd, +0xea,0x04,0x6b,0x00,0x39,0x9a,0x21,0x83,0x61,0xac,0x13,0x33, +0xdf,0x93,0x43,0x7c,0x95,0xa3,0x6f,0xaf,0x90,0xd1,0x2f,0xd9, +0xec,0xd1,0xf7,0xa3,0x94,0xd7,0xad,0xc8,0xb7,0x48,0xd6,0xac, +0xa2,0x64,0xaa,0x2d,0x62,0x62,0xcd,0x1e,0x5e,0xa6,0x6a,0x47, +0x59,0x97,0x33,0xdd,0xa1,0x4a,0x57,0x76,0x50,0xb1,0x70,0xce, +0x47,0x55,0xd1,0xc6,0x5b,0xe0,0x0a,0x28,0xe7,0xa9,0x5a,0x0e, +0x54,0xea,0xaf,0x5a,0x89,0xaf,0xe8,0xb5,0x5a,0x0e,0x5a,0x5c, +0x85,0xb0,0x07,0x29,0x95,0x2c,0x66,0x16,0x1b,0x58,0xb5,0x23, +0x51,0x8e,0xc3,0x72,0x34,0xcb,0x63,0xda,0x38,0x83,0xa5,0x4a, +0xca,0xca,0x44,0x7b,0xd3,0xdf,0xf8,0x25,0x81,0x5a,0x17,0xb3, +0x19,0x6f,0xb7,0x8b,0x39,0xbe,0x78,0x36,0x5b,0xd1,0x65,0xc1, +0x45,0x3a,0x9c,0x2d,0xe2,0x08,0x33,0x9b,0xe7,0xe2,0x02,0x2a, +0xbb,0xa6,0xa0,0xab,0x82,0xb6,0xb7,0x5f,0xa1,0xf7,0x58,0xde, +0x93,0x4c,0xc0,0x22,0x9b,0x33,0x7e,0xe6,0x7b,0x83,0x26,0x07, +0x3b,0x9e,0xb3,0x94,0x08,0x58,0x99,0xe4,0x33,0x67,0xd0,0x92, +0x0f,0x9d,0x20,0xaf,0x34,0x3b,0x95,0x12,0x50,0x58,0x87,0x83, +0xa5,0x9d,0xce,0x29,0x42,0xbf,0x8c,0xa2,0x88,0x30,0x0f,0xa2, +0x5c,0x0d,0xd2,0x9f,0x6f,0xa7,0xaf,0x38,0x26,0x4f,0x66,0x00, +0xdc,0x74,0x31,0xa5,0x27,0x56,0xbe,0xe7,0x47,0x3b,0xed,0x1a, +0xe4,0x3b,0x9e,0x12,0xe0,0xe5,0x60,0x6e,0x67,0x4e,0x06,0x73, +0x99,0xf1,0x0b,0xdd,0x3a,0xb1,0xf3,0xf8,0x1e,0x8a,0x93,0xfd, +0x4b,0xf2,0xfe,0x6a,0x59,0x84,0xa1,0x64,0x45,0x41,0x32,0x86, +0x45,0xa7,0x43,0x00,0xa5,0xc8,0xec,0xef,0x60,0xbd,0x94,0xd9, +0xb9,0x78,0x33,0x50,0x51,0xff,0x7a,0x36,0x71,0x99,0x06,0xdf, +0x32,0xeb,0x05,0x74,0xea,0xfa,0xe7,0x4b,0x3b,0x37,0xe1,0x24, +0x09,0xf0,0xe3,0x00,0xb7,0x4d,0x0a,0x40,0x63,0x93,0xac,0x30, +0xc9,0x4b,0x92,0xdc,0xfd,0xf2,0xe2,0xbd,0xcc,0xfb,0xef,0x71, +0x96,0xaa,0xee,0xc7,0xe0,0xb8,0x32,0xf9,0x1b,0xa8,0x2a,0xd2, +0x42,0x40,0x12,0x2f,0xce,0xcb,0x63,0x59,0xc2,0xef,0x07,0xc9, +0x14,0x07,0xc3,0x30,0x9e,0x5d,0x3e,0x38,0xef,0x42,0x02,0xdd, +0x4a,0x72,0x53,0xe5,0x1b,0xd4,0xb7,0x8b,0xc1,0x7c,0x1e,0x8f, +0x70,0x97,0x49,0x2d,0xf1,0x31,0xc6,0xe8,0x83,0x73,0xc2,0xb9, +0xe9,0xd4,0x56,0xea,0xe2,0x6b,0xed,0x20,0x42,0x69,0x83,0xc2, +0xf1,0x7a,0x3e,0x5b,0x2c,0xd3,0xae,0x57,0x96,0xc9,0xe1,0x88, +0x0f,0x0f,0x65,0x58,0x62,0x09,0x8b,0x17,0x4d,0xf8,0x57,0xe4, +0x66,0x44,0xe7,0xbe,0x90,0x3f,0x9b,0xe7,0xf0,0xd1,0xe6,0x01, +0x13,0x9d,0x8b,0xf3,0xe6,0xaf,0xdf,0xfd,0xf2,0xfa,0xc5,0xcf, +0x3f,0x45,0xb5,0xb0,0xf9,0xa8,0xd9,0xaa,0x91,0xe9,0x80,0x3e, +0x66,0x11,0xbb,0x9a,0xc1,0x1f,0x29,0x59,0x0e,0x23,0x04,0x3d, +0xcc,0x24,0x23,0xb1,0x60,0xe0,0x66,0xba,0x00,0x81,0xda,0xe8, +0x00,0x5b,0xe9,0xe3,0x87,0x2d,0xa9,0x87,0x87,0x90,0xaa,0x31, +0x45,0x91,0x93,0xe9,0x5b,0x79,0x5e,0x01,0x71,0x47,0xdd,0x27, +0x42,0x28,0xbe,0x69,0x0f,0x08,0xea,0xe6,0xcb,0x5c,0x7b,0x4d, +0xa2,0x40,0x48,0x08,0x93,0xdd,0xe1,0x3f,0x67,0x49,0x27,0xa9, +0xd7,0x91,0x30,0x55,0x83,0xf4,0x26,0xe5,0x6a,0x70,0xa3,0xb2, +0x97,0xf4,0x45,0x82,0x3f,0x30,0x9e,0x9f,0x54,0x26,0xf2,0xda, +0x13,0xfb,0x86,0xab,0x7a,0x50,0x66,0x22,0x19,0x57,0x19,0xc1, +0x85,0x55,0x31,0x26,0xde,0xbf,0x66,0x2c,0x05,0xd5,0xf7,0x85, +0xfc,0x51,0x41,0x45,0x87,0x0e,0x09,0x22,0x8c,0x03,0x41,0xaa, +0x77,0x57,0x9f,0xc8,0xad,0x3f,0x8a,0xc9,0xda,0xd3,0x13,0x30, +0x76,0x53,0x84,0x1c,0xea,0xca,0xbc,0xb6,0x56,0x1a,0xdc,0x4f, +0x58,0x87,0xea,0x23,0x48,0xec,0xca,0xb4,0x62,0xdf,0xb4,0x3d, +0x72,0x47,0xc4,0xca,0x6d,0x4f,0xd5,0x55,0x2c,0x68,0xd4,0x8a, +0x71,0xc2,0x01,0x3c,0xa8,0x16,0xbe,0x79,0x5c,0xce,0x83,0x42, +0x21,0xb9,0x25,0x8d,0xe5,0xe4,0xfd,0x3c,0xd6,0x41,0x14,0xb7, +0x25,0xaa,0xb1,0x9e,0x7a,0x80,0xe3,0x04,0x03,0x48,0x3f,0x18, +0xe0,0xe8,0x26,0x5b,0xeb,0xc1,0x74,0xf6,0x40,0xc6,0x70,0xe3, +0x00,0xd2,0xb5,0x8e,0xba,0x6b,0x47,0xb2,0x3c,0x1e,0x51,0x3c, +0xe2,0xe9,0x87,0x6a,0xf6,0x4d,0x60,0x91,0xed,0xf2,0x50,0x22, +0x2c,0x84,0x7c,0x38,0x6b,0x75,0xa4,0xd8,0x13,0x3f,0x61,0x00, +0xc2,0xbf,0x14,0x56,0xc1,0x56,0xa7,0xcc,0x54,0x49,0x83,0xe2, +0x2b,0x67,0xf9,0xc6,0xe5,0x0c,0x6b,0xc3,0x88,0x6b,0x92,0x0a, +0x35,0xe5,0x16,0x78,0x0e,0xd2,0xc6,0xd4,0x74,0x0d,0x56,0xcf, +0xa1,0xdd,0x6f,0x97,0xe4,0xf0,0xf5,0xbf,0x3d,0x7a,0x4b,0x61, +0x47,0x4c,0x51,0x79,0x6f,0x11,0x83,0x0a,0x25,0xdb,0x1e,0x15, +0x51,0xe9,0xcc,0x31,0x3c,0x2f,0xf0,0xc5,0x81,0xfc,0xb4,0x62, +0x52,0x62,0x60,0x16,0x8e,0x4b,0x69,0x75,0xac,0x36,0x6c,0x11, +0xd3,0x46,0x9c,0xdb,0x13,0x8f,0xec,0xbd,0xc5,0x7f,0x52,0xa7, +0x51,0x9d,0x76,0xcf,0x31,0x11,0x6e,0xf7,0x51,0xda,0x17,0xea, +0x43,0xc2,0x55,0xdd,0x91,0x6e,0x36,0x2f,0xf1,0x8a,0xba,0x0e, +0x1a,0xc6,0xbf,0x0e,0x40,0x45,0x2a,0xf5,0x58,0xa6,0xb2,0x3a, +0x45,0x75,0xb5,0xd9,0x5b,0x40,0xe0,0x37,0x15,0xec,0x92,0xb6, +0x6a,0x34,0xb8,0x7c,0xbf,0xad,0x7e,0x89,0x7d,0x65,0x88,0x94, +0x2e,0xa2,0xeb,0x97,0x08,0x92,0x93,0xf9,0x45,0xa4,0x09,0x03, +0xbb,0xd2,0xd2,0x76,0x79,0x2f,0xdd,0xa9,0xe3,0xac,0x4d,0xb3, +0xdd,0xaa,0x8e,0xb7,0xb6,0xf7,0x53,0x72,0x5d,0x8f,0x2b,0x90, +0x63,0x06,0x5a,0xd7,0x66,0xf3,0x40,0x2b,0x3f,0xa6,0x9a,0x4c, +0xad,0xf3,0x26,0x87,0x6d,0xfd,0xb2,0x3a,0x9f,0xed,0x38,0x39, +0x3d,0x73,0x45,0x7a,0x76,0xa6,0xcf,0xae,0xc9,0xf9,0x5c,0xfd, +0xbf,0x27,0x37,0x0e,0x0f,0x9d,0x89,0x82,0x03,0xf9,0xd9,0xd3, +0x00,0xe9,0x83,0x2d,0x7a,0x5b,0xb3,0x40,0x6f,0x15,0x49,0xba, +0xf7,0xea,0xb4,0x83,0x3d,0xc9,0xdc,0xe8,0x7a,0x90,0x20,0xb2, +0x6b,0xa1,0x6f,0x70,0xd5,0xb7,0xcb,0x40,0x92,0x29,0x77,0x77, +0xb6,0x82,0xc0,0x23,0x92,0x65,0xb2,0xcc,0xd4,0x59,0x21,0xc9, +0xc3,0x41,0x50,0xd5,0x6f,0x6d,0xcb,0xbe,0xe6,0x5e,0x63,0x0a, +0x54,0xa7,0x51,0x46,0x57,0xa7,0x7f,0xfe,0x94,0x8d,0xb5,0x2b, +0xf1,0xe4,0x3f,0x87,0x87,0xfb,0xce,0xdd,0x5d,0x96,0xe2,0xb6, +0xb4,0x5e,0x70,0xc8,0x1e,0x30,0x0e,0xf5,0x24,0xc1,0x14,0x79, +0x86,0x76,0x3f,0xb2,0x6e,0x9a,0x7d,0x49,0xd6,0x85,0x3b,0x58, +0x87,0xab,0x0f,0xe6,0x1c,0xd5,0xaf,0x18,0x87,0xc9,0x5d,0x95, +0xfa,0xe7,0xd8,0x26,0x49,0x01,0x62,0x25,0x4d,0x7b,0xf3,0x4d, +0x32,0xcc,0x68,0x01,0x8b,0x6f,0x26,0xf8,0x17,0xd9,0x2f,0xc3, +0xf1,0x6a,0x14,0xbb,0x7c,0x5b,0xc2,0x4c,0x17,0x1b,0x2a,0x4c, +0xfb,0x0f,0xc2,0xb6,0xb3,0xbc,0xe2,0xe6,0xcb,0x85,0x95,0xe6, +0x80,0xcc,0xec,0x36,0x42,0xb2,0xe1,0x55,0xbe,0x27,0xd1,0xb6, +0xcb,0xd5,0x9f,0xae,0x8e,0x9f,0x10,0x89,0x22,0x06,0xdf,0xd0, +0xe8,0x48,0xa6,0x37,0xb3,0xeb,0x1c,0x95,0xf6,0xb5,0xb1,0x01, +0x5e,0x0f,0xb3,0xae,0x76,0xe9,0xa9,0x5a,0xb4,0xf0,0x78,0x11, +0xd7,0x62,0x14,0x18,0x48,0xef,0x99,0xc9,0xc2,0x26,0x8a,0x02, +0x9a,0xa5,0xd5,0x44,0x79,0x5c,0x54,0x86,0xb1,0x6f,0x53,0x9e, +0xba,0x2a,0xe6,0x9b,0x6b,0x62,0xd0,0x0d,0x48,0x89,0xcf,0x44, +0xcf,0xc7,0x2b,0xfb,0xcc,0x15,0xb1,0xd3,0x3b,0x29,0x7b,0xd6, +0xc9,0x8c,0xa0,0x67,0x4a,0x18,0xdf,0xed,0x55,0xbc,0xc8,0xf1, +0x60,0xb0,0x5c,0x62,0x68,0xa6,0x64,0x91,0x2e,0x9d,0xb8,0x5a, +0xfc,0x1a,0x0a,0xe5,0xfa,0x5d,0xca,0x56,0x43,0xa9,0xd7,0x6f, +0x9f,0xf7,0x38,0xa5,0x86,0xd3,0x58,0xad,0x5d,0x63,0xe5,0x56, +0xeb,0x97,0xd2,0x62,0x2d,0x55,0xc0,0x10,0x7a,0xc0,0x28,0x61, +0x82,0xa7,0x1f,0x44,0x1c,0x58,0x08,0x86,0x52,0x69,0xc3,0xe8, +0xc1,0x73,0x10,0x30,0xed,0x58,0xd5,0xdf,0x2a,0xe8,0xb7,0x28, +0xa7,0x26,0x5a,0x2d,0xc3,0x79,0x5a,0x20,0xa3,0xd6,0x7b,0xac, +0x1a,0xb5,0xe2,0xc5,0x20,0x56,0x2a,0x42,0x1e,0xda,0x28,0x24, +0xa6,0xbd,0xa0,0x2f,0x97,0x7b,0xf0,0x8b,0x05,0x97,0x8d,0x8c, +0xaf,0x4f,0x1f,0x3d,0x3a,0x79,0xa4,0x8c,0x2f,0x7a,0x36,0x13, +0xea,0x93,0x9d,0x8a,0x9f,0xb4,0x86,0xea,0x14,0x2b,0x60,0x26, +0x63,0xa6,0x2c,0xdc,0x08,0x8f,0x03,0x5b,0x85,0x7c,0x1a,0xce, +0x26,0xf3,0xd5,0x32,0x1e,0xb5,0x31,0x87,0xc7,0x29,0xfd,0xd4, +0xb7,0x4e,0xf7,0xd1,0x05,0x7c,0x63,0x87,0x11,0x69,0x0d,0xd0, +0xdd,0x53,0x15,0xb0,0xa8,0x76,0x54,0xf9,0x33,0xa6,0xac,0xa9, +0xbe,0xc1,0xc6,0x54,0xb4,0x32,0x75,0x8c,0x40,0xd3,0xad,0x7e, +0x6c,0xcc,0xfc,0xc9,0x17,0xde,0xa8,0x57,0x92,0xe9,0x7f,0x6e, +0xaf,0x24,0xd3,0xcf,0xe9,0x95,0x07,0x95,0xbd,0x62,0x3a,0xe5, +0xff,0x64,0x9f,0x7c,0xfd,0x25,0xfb,0x24,0xbd,0x5a,0x5d,0x5e, +0x8e,0xe3,0xdc,0x66,0x13,0xb5,0x7e,0x30,0x1d,0x31,0x15,0x51, +0x20,0x24,0xd8,0xc8,0x32,0xec,0x2a,0x9a,0xed,0xe3,0xcb,0x7e, +0x68,0xec,0xe2,0x9f,0xd9,0xc4,0x23,0x0c,0xf5,0xba,0xaf,0x51, +0xb0,0x69,0xdd,0x08,0xfb,0x91,0x4e,0x41,0xd0,0xbe,0x70,0x3f, +0x59,0x47,0x6c,0x4c,0x39,0xa2,0x76,0x30,0x99,0xe7,0x88,0x15, +0x53,0xf1,0x7e,0x35,0x58,0x98,0x58,0xdf,0xf9,0x55,0xd7,0xd7, +0xad,0xbb,0x3b,0x82,0xc0,0x49,0xb6,0xa7,0xe9,0x32,0x92,0xd3, +0x08,0x7d,0xd0,0x71,0x8a,0x13,0xc4,0x00,0xde,0x4e,0xf5,0x02, +0xa1,0x46,0x37,0x3e,0xd8,0xa3,0x9e,0x35,0x1a,0xd3,0x23,0xdf, +0x2f,0x94,0x39,0x50,0xa1,0x8c,0x9d,0xd9,0x83,0xb3,0xba,0xdc, +0x3d,0x55,0xfb,0x7a,0xfc,0x66,0x55,0x7f,0x83,0x33,0x6e,0x3a, +0x5b,0x2c,0xbf,0xc9,0x59,0x27,0xaa,0x5b,0xed,0x95,0xa6,0x22, +0xc2,0xa5,0x49,0xd6,0x67,0x26,0x2b,0x9a,0x59,0xbc,0xaa,0x09, +0xa4,0xc4,0x92,0x70,0x24,0x89,0xb2,0xdb,0x0c,0x34,0x5c,0x60, +0x9d,0xc9,0xa0,0xbd,0xaf,0xd1,0xba,0xf1,0xa9,0x31,0xc6,0xd1, +0x75,0x1c,0x5f,0x2e,0xc5,0x82,0xd6,0xaf,0x3c,0x15,0x47,0x98, +0xd2,0x54,0x88,0xc5,0x45,0x44,0x99,0x3a,0x01,0x87,0xec,0x00, +0xe6,0x8c,0x0b,0xd2,0x0e,0x83,0xb3,0x8b,0xbb,0x3b,0x9e,0x9a, +0x30,0x12,0xa6,0x1e,0xb2,0x08,0x75,0x71,0x36,0xb0,0xf2,0x2e, +0xb4,0x92,0x55,0x67,0x02,0x54,0x0f,0x4b,0x1f,0x57,0x41,0xbf, +0x41,0xc4,0x6a,0xbc,0x5f,0x23,0xfb,0xf7,0xfd,0x62,0xb6,0x9a, +0x1b,0xd6,0x5f,0xc4,0x57,0x83,0x9b,0xc4,0x8a,0xc9,0xbb,0xab, +0x53,0xd4,0x60,0xdc,0x68,0xfd,0x16,0xb1,0x45,0x44,0xb0,0x5d, +0x63,0x3e,0xb6,0xb7,0xf6,0xda,0x56,0xb5,0xa2,0xd7,0xd6,0x7b, +0x98,0x78,0xa4,0xf7,0x54,0x2b,0xa4,0xaa,0x40,0xc3,0x42,0xe4, +0x16,0x3a,0x24,0x78,0xd4,0x78,0x90,0x3c,0xfa,0xeb,0x78,0x27, +0xbb,0xa5,0x3e,0x79,0xfc,0x14,0x88,0xc9,0xf0,0xa5,0xb5,0x4b, +0xd3,0x7a,0xdb,0xfa,0x8d,0x3b,0x1c,0xb9,0x95,0xd5,0x39,0xb3, +0x7e,0x9f,0x7a,0x6c,0x44,0x4a,0x25,0xa0,0x45,0xba,0x9a,0x2e, +0xb7,0x15,0xc7,0x1b,0xcb,0x5b,0xe8,0xab,0xd7,0x1d,0x0a,0x43, +0x42,0x8a,0x72,0x1a,0x8f,0xc8,0x1a,0x35,0xdd,0x4f,0x1b,0x7c, +0x62,0xfb,0xea,0x40,0xf6,0xaf,0xd6,0xeb,0xd5,0x5d,0x6c,0xb6, +0x51,0x94,0x9d,0xc4,0xd6,0x6b,0xe5,0x06,0x2d,0x5e,0x99,0xb9, +0x05,0x05,0x7c,0x05,0x02,0x1b,0x11,0x2d,0x6a,0xc7,0x05,0x53, +0xce,0x20,0xb3,0x23,0xdf,0x65,0x4a,0x46,0xa0,0x05,0x6e,0xeb, +0x94,0x7c,0x76,0x06,0x03,0xa2,0x14,0x25,0xa1,0xe8,0x01,0x70, +0xdf,0xff,0x9a,0xe5,0x11,0xf1,0xc3,0x77,0x3d,0x6c,0x53,0x1d, +0xf0,0x53,0x8f,0x97,0xd9,0x2d,0x2a,0xdd,0xe5,0x2c,0x17,0xf5, +0x31,0xa7,0xb7,0xba,0xee,0x34,0xdd,0xb5,0xcc,0x6a,0xfc,0x6e, +0x57,0x6d,0x9f,0x77,0x8d,0x3e,0xb2,0x96,0x54,0x6d,0x7d,0x43, +0x9e,0x0a,0xf7,0xfa,0xa4,0xf7,0x93,0x8f,0xf9,0x29,0xaa,0xb0, +0xd6,0x08,0xaa,0x2b,0x32,0x3f,0xdb,0xd6,0x4e,0x94,0xda,0x7d, +0x22,0x6b,0x13,0xec,0x5b,0x98,0xb3,0xf0,0x2c,0x1d,0xfe,0x2c, +0x07,0xf6,0xda,0x81,0x25,0x20,0x3f,0xcd,0x70,0xc5,0x94,0xa7, +0x4c,0x65,0x4e,0x9a,0xaa,0xe9,0x86,0x59,0x1d,0xf4,0xdb,0xce, +0x3a,0x03,0x71,0xe1,0x64,0xc2,0xab,0x14,0xde,0x5d,0xda,0x51, +0x55,0x49,0x79,0x5b,0x12,0x1a,0x5e,0xae,0x62,0x58,0x74,0xf9, +0x84,0x1f,0x0f,0xa7,0xbe,0x54,0x3b,0x9c,0x1a,0xc3,0x92,0x46, +0xe9,0xa9,0xd2,0x81,0x9c,0x8a,0xc0,0x97,0x1b,0x28,0xc4,0x61, +0x58,0x47,0xe2,0x36,0xf8,0x68,0x31,0x9b,0xdf,0xbb,0xd9,0x25, +0xcd,0xdc,0x90,0x2e,0x98,0xcc,0x07,0xc3,0x7c,0x43,0x8b,0x9b, +0x31,0x8c,0xc4,0x12,0x35,0x56,0xf7,0x97,0x63,0x7c,0x49,0x70, +0x6a,0xc7,0x61,0x07,0x73,0x09,0xcc,0x0e,0xa8,0x78,0x76,0x2b, +0x66,0xab,0x25,0x7c,0x1a,0xa2,0x38,0x99,0xc2,0xde,0xf2,0x3e, +0x07,0x41,0xeb,0x21,0x40,0xa1,0xb5,0x87,0x83,0xa5,0x34,0x3c, +0xb9,0xb4,0x20,0x20,0xb5,0x94,0xe7,0x12,0x79,0xb3,0x21,0x17, +0xa3,0xdb,0xbf,0xbb,0xe3,0x14,0x69,0xcc,0x28,0x13,0x42,0xd6, +0xdf,0x45,0x7d,0xea,0xd6,0xc1,0x00,0x6d,0xd9,0x1c,0x39,0x59, +0xe4,0x5a,0xd1,0xe6,0xbf,0x39,0x6d,0x2c,0x33,0x51,0xfd,0x17, +0x98,0xc1,0x4c,0x93,0x68,0xcc,0x14,0x28,0x2b,0x71,0x72,0xf1, +0xbd,0x52,0x5a,0x76,0x26,0xb0,0xcc,0x5d,0x6d,0xe9,0x8f,0x51, +0x72,0x79,0x09,0xcb,0xb6,0x29,0x58,0x58,0x12,0x41,0xd9,0x42, +0x3c,0x64,0xb9,0x59,0x4d,0x93,0xdf,0x39,0x16,0xd5,0xef,0xab, +0xc2,0xa0,0x4c,0xd2,0xd7,0xa4,0xb3,0x4b,0x74,0xb3,0x63,0x7f, +0x29,0x38,0x19,0x9b,0x9c,0x5e,0xae,0xd5,0x45,0xb4,0x16,0x37, +0xd8,0xe4,0x8f,0xe8,0x20,0xe4,0x2d,0x1e,0x35,0x50,0x2d,0x0d, +0x8f,0xaa,0x4b,0x12,0x51,0xd8,0xb2,0xe1,0x74,0xb3,0x17,0x2a, +0xd2,0x18,0x78,0x9a,0x33,0x9d,0x25,0xce,0x8a,0xe9,0x5d,0x93, +0xdc,0xa5,0x84,0xc3,0x43,0x44,0xd1,0xc3,0x7f,0xcc,0x18,0x8c, +0xa4,0x49,0xd1,0xb6,0x5f,0xe7,0x04,0x08,0x29,0x0a,0x20,0x42, +0xf4,0x69,0xf7,0xb6,0x70,0x76,0x3a,0x79,0x64,0xf3,0x56,0xb7, +0x7d,0xf0,0x25,0xf9,0x6e,0xfb,0x6e,0x59,0xdd,0x87,0x5d,0xe1, +0x69,0x51,0xb1,0x7a,0xec,0x40,0x0e,0x75,0x7a,0x9c,0x20,0xcd, +0x07,0xd6,0x96,0x52,0x20,0x2d,0xa4,0x65,0xf9,0xee,0x4b,0x68, +0x19,0xad,0x72,0xd8,0xca,0x0a,0xb9,0xb8,0xe1,0x16,0x70,0xdd, +0xf6,0xcd,0xe3,0xc1,0x88,0x88,0xad,0xc8,0x25,0xcb,0xab,0x78, +0x61,0x1b,0xe4,0x72,0x5f,0x89,0xd2,0x05,0x21,0x38,0x8b,0x82, +0x8d,0xdc,0x7d,0x31,0x52,0x59,0x4d,0xb3,0x33,0xb6,0x8d,0x17, +0x40,0xa5,0x00,0x17,0x1a,0xc3,0x92,0x51,0xbe,0x64,0xb0,0xa3, +0xec,0x53,0x43,0xd4,0x00,0x45,0xea,0x3e,0x26,0x73,0xbb,0x33, +0x94,0xf5,0x20,0xa7,0x3b,0xda,0xf1,0xf0,0x94,0x9d,0x6f,0x48, +0xa8,0x71,0x7e,0xcd,0x97,0x7e,0x20,0x1e,0xf4,0x90,0x96,0x4b, +0x3c,0xbe,0x60,0x8d,0x23,0xcf,0x6b,0xf0,0x34,0xd9,0x3d,0x3a, +0x96,0xa0,0xbd,0x04,0x6f,0xe3,0x16,0x90,0xd7,0xea,0x89,0x1d, +0xf3,0x43,0x49,0xce,0xec,0xc2,0xdd,0x2a,0xc7,0x75,0x80,0x7e, +0x24,0x4f,0x1f,0x80,0xd1,0xea,0x40,0x2e,0x36,0x36,0xda,0x1a, +0xb2,0x6c,0x67,0x73,0xb2,0x8d,0xa0,0xa5,0x27,0xdb,0x8c,0x54, +0xd9,0x75,0x08,0x86,0x47,0xd9,0xf2,0xf5,0x3d,0xf8,0xd9,0x76, +0x73,0x60,0x2e,0xee,0x47,0xea,0x77,0xd8,0x77,0x49,0xdf,0x08, +0x2d,0x20,0x05,0x2d,0xb3,0xa4,0xd8,0xe2,0x52,0x85,0x98,0x26, +0xb0,0x74,0xa8,0x55,0x46,0x27,0xe7,0x08,0xe0,0x18,0x6b,0x78, +0x06,0x6f,0x23,0xa8,0x4d,0x57,0x93,0x8b,0x78,0x61,0x1c,0x2b, +0x74,0xae,0x3a,0xc8,0x89,0x1c,0x7b,0xd4,0xa2,0xc4,0x97,0x36, +0x5c,0x82,0xe3,0x1f,0x13,0xba,0x49,0x1b,0x6a,0x87,0x9a,0xcf, +0xb4,0xce,0xb0,0x96,0xad,0x4c,0x40,0x5d,0xe3,0x6f,0xab,0x5f, +0x1b,0xed,0x2a,0xa1,0x37,0x67,0x99,0xe4,0x8a,0xed,0x59,0x5f, +0xaf,0xaf,0x2d,0x20,0xcf,0xe5,0x4d,0xe7,0x5f,0xb0,0x23,0x0b, +0xee,0x07,0x39,0x82,0x75,0x1b,0x3b,0x7a,0x85,0x26,0x8d,0x95, +0x17,0x5b,0x3a,0x00,0x23,0xcd,0x6c,0x67,0x3e,0xd8,0xfb,0x6c, +0xbc,0xb3,0x3b,0x08,0x16,0xe8,0xe8,0x46,0x5a,0xfe,0x3b,0xaa, +0xa1,0x76,0x8d,0xaa,0xb1,0x16,0x98,0x22,0x53,0xa1,0xed,0x16, +0x4a,0x79,0x86,0xb0,0x76,0x79,0xa6,0xb1,0xf3,0x93,0x48,0xe3, +0xc1,0x02,0x6d,0x57,0x3c,0x1a,0x8d,0xce,0xde,0x8c,0x5a,0x60, +0xc0,0x7b,0xfb,0x9d,0xe4,0xc1,0x02,0xcc,0x95,0xe5,0x6c,0x0e, +0xff,0xc4,0x73,0xff,0x53,0x61,0x03,0x24,0x0a,0x61,0xda,0x43, +0x80,0x88,0x60,0xef,0xee,0x02,0x41,0x3f,0xa2,0xc0,0xa7,0x22, +0xe6,0xa0,0xba,0xd7,0xea,0xdf,0xdd,0x85,0x9d,0x9c,0x76,0xd1, +0xd2,0x44,0x3f,0x86,0x71,0x32,0xf6,0x08,0x5d,0x83,0xb0,0xf8, +0xc7,0x54,0x2d,0xd8,0x79,0x22,0x19,0xe1,0x5e,0x11,0xd3,0x57, +0xd0,0x2e,0x5a,0x2a,0x46,0xeb,0x8e,0x4f,0x30,0x78,0x5f,0xbc, +0x5e,0xef,0x47,0xaa,0x05,0xf0,0x6f,0x3d,0x42,0x64,0x7a,0x64, +0x22,0x14,0x5b,0x69,0xc3,0x42,0xe0,0x7b,0x3e,0xe2,0xb6,0xa3, +0xed,0x4f,0x87,0xee,0x42,0x1c,0xb7,0xd6,0xc5,0x05,0xac,0x14, +0x47,0x46,0x0c,0xd0,0xd5,0xea,0xff,0xe3,0xed,0x68,0x9a,0xdb, +0xd6,0x8d,0xe7,0xfe,0x0b,0x8b,0xed,0x93,0xc8,0x92,0xb6,0xec, +0x5c,0xda,0x88,0x61,0x34,0x9d,0xb4,0x6f,0xe6,0x5d,0x72,0xe8, +0xeb,0x4d,0x52,0x35,0xa4,0x4d,0x39,0xcc,0x48,0xa2,0x22,0xd1, +0x4d,0x32,0x96,0xfe,0x7b,0xf7,0x13,0x1f,0xfc,0x70,0x3c,0xef, +0x75,0x7a,0x11,0x21,0x60,0xb1,0x58,0x00,0x8b,0xc5,0x62,0xb1, +0x00,0xc6,0x63,0x84,0xe5,0xec,0xda,0xef,0x18,0xaf,0xcd,0x6d, +0x63,0x44,0xd2,0x13,0xea,0x41,0x19,0xcf,0xef,0x62,0x3a,0xda, +0x06,0x82,0x47,0x7d,0x7b,0xc4,0xa9,0x35,0x51,0x0d,0xef,0x3e, +0x10,0xc1,0x6e,0x55,0x11,0x7f,0xd8,0x76,0xf1,0x22,0x28,0x63, +0xb1,0xa4,0xfa,0x30,0xad,0x76,0xa1,0xf7,0x78,0x52,0xa9,0xdf, +0x57,0x54,0x04,0x94,0xdf,0x7b,0x97,0xe4,0x53,0x99,0xf6,0x6f, +0xca,0xfe,0xb7,0xdb,0x0d,0xbb,0xdb,0x02,0x68,0x1b,0x9e,0xee, +0xa0,0x74,0xa4,0xb6,0x43,0x05,0xe6,0x7b,0x15,0x09,0x52,0x01, +0xf6,0x89,0x93,0xd5,0x3a,0xfa,0x1f,0x71,0x48,0x37,0xd0,0x10, +0xdd,0x85,0xb6,0x48,0x80,0x47,0xbc,0x15,0x13,0xb5,0xf5,0x0f, +0xb6,0x74,0xac,0x52,0xd1,0x55,0x67,0x1c,0x9a,0xc9,0xc9,0xf3, +0x15,0x34,0x13,0x21,0xc8,0x3c,0x7f,0x6b,0x6d,0xbd,0x32,0x1d, +0x18,0x33,0x4c,0x08,0x74,0x26,0x1a,0xa7,0x08,0x48,0xdd,0xc8, +0x3a,0xef,0xa5,0x0a,0xf2,0xab,0xdd,0xd3,0xa9,0xb9,0x2a,0xca, +0xab,0x43,0x8e,0x87,0x71,0x0d,0xf5,0x57,0xe8,0x5d,0x7e,0x0a, +0x7c,0x4b,0x11,0x61,0xb4,0x9a,0xc5,0x26,0x7a,0x46,0x93,0xe2, +0xc6,0x5e,0x10,0x42,0xff,0xc8,0xa4,0x80,0x2a,0x7f,0xf1,0x99, +0x2c,0xf2,0xe5,0xae,0xf6,0x96,0xd6,0xc4,0xe9,0x20,0xa8,0x3e, +0xa9,0x6f,0x38,0x39,0x44,0xd8,0x5b,0x2f,0x39,0x09,0x14,0x4b, +0x0e,0xb8,0x7b,0xa1,0x6d,0xc7,0x59,0x34,0x4e,0x31,0x54,0xab, +0x7d,0xa5,0x21,0xad,0x6e,0x84,0x76,0x1a,0xf2,0xca,0x20,0x2b, +0x0d,0x86,0xd8,0x86,0x64,0x42,0x59,0x4f,0x2f,0x09,0x16,0xea, +0x8c,0x87,0x72,0xeb,0xda,0x27,0xa8,0x12,0x5f,0xf3,0xaa,0xf9, +0xf1,0x5e,0x9f,0x12,0x71,0x2a,0x9b,0x7f,0x55,0xbb,0x12,0x96, +0x2d,0xe1,0x8b,0x3c,0x42,0xc7,0xb2,0x79,0xcb,0x8e,0x4b,0xe0, +0xe2,0x37,0xee,0x13,0x3f,0xcc,0x92,0x76,0xb5,0x83,0xc4,0x49, +0xf6,0x75,0xb2,0x20,0xe2,0xee,0x56,0x2f,0xf1,0x18,0x4a,0x15, +0x42,0x8c,0x8c,0xd1,0x34,0xdb,0x76,0xff,0x60,0xc1,0x7a,0xcd, +0xbd,0x6e,0x2d,0xd8,0x41,0x2f,0xfa,0x5d,0xd2,0x70,0x7d,0x32, +0x39,0x49,0xce,0xbe,0xcb,0xa0,0xe0,0x49,0x46,0xe8,0x43,0x09, +0x65,0x78,0x1f,0x02,0x89,0xfb,0xbc,0x29,0x3d,0x21,0x6b,0x72, +0x09,0x24,0x30,0x6c,0xfe,0x80,0x5e,0xc4,0x78,0xc4,0x73,0x7e, +0x4b,0xae,0xa4,0x7f,0x87,0x4c,0x7e,0x59,0x5d,0x69,0xe0,0x92, +0x17,0x5d,0x7a,0x06,0x23,0x9d,0x42,0xa8,0xbf,0x66,0x8a,0x30, +0xd5,0x92,0xcf,0xe7,0x56,0xd1,0x74,0xba,0x14,0x68,0x37,0xa4, +0x41,0x36,0xdd,0x86,0xdf,0x81,0xea,0x8c,0xd4,0x61,0xfb,0x5c, +0x87,0x90,0x70,0xad,0x50,0xee,0x0d,0x6b,0xb4,0xf8,0x33,0x63, +0xdd,0xce,0x7d,0xc9,0xed,0xfb,0xcc,0xe0,0x98,0x87,0xf7,0x50, +0xe0,0x51,0x79,0x42,0xea,0x17,0x0d,0x34,0x2a,0x14,0xf5,0xa3, +0x5a,0x47,0x33,0xc9,0x6a,0x6b,0xd4,0x1c,0xf3,0x6a,0xab,0xad, +0x89,0xef,0x27,0x09,0x6e,0x87,0x17,0xa9,0x47,0x12,0x43,0x55, +0x64,0x4d,0xb5,0xc4,0x76,0x38,0x09,0xdc,0xf7,0x72,0x47,0xb5, +0xdb,0x95,0x0f,0x55,0xae,0x37,0x46,0x09,0x6a,0xe6,0x0f,0xa5, +0x0b,0x23,0x61,0x3e,0xd9,0x1d,0x12,0xdf,0xdb,0xc8,0xe9,0x97, +0x97,0x9a,0xcb,0x64,0xb7,0xbd,0xd6,0xcb,0x44,0x1c,0x09,0x8b, +0x27,0x05,0xbb,0x36,0x39,0x53,0xa4,0xf5,0x3d,0x26,0xce,0x07, +0x2b,0x4f,0x9d,0x89,0x20,0xd1,0x2c,0xf4,0x5a,0xdf,0x54,0xd1, +0xba,0x3b,0x0c,0x36,0xbe,0x1c,0x3a,0xfe,0x08,0x2c,0x66,0xb2, +0x8d,0xc7,0x23,0xc1,0x67,0x6e,0x3f,0xd5,0x1e,0x1a,0xee,0x0a, +0x1a,0xf3,0x91,0x22,0xb3,0x3b,0x6f,0x83,0x25,0xbb,0x1d,0x56, +0x77,0x3a,0xcb,0xca,0x58,0xd4,0x9a,0x1c,0x17,0x92,0xae,0x10, +0x82,0x74,0x12,0x8d,0xb3,0x90,0x20,0x6f,0xc9,0x8f,0xed,0x05, +0xc1,0xc8,0x37,0xd4,0x52,0x4b,0x91,0x47,0x1f,0x51,0x80,0xfe, +0xe0,0x6d,0x76,0x21,0x17,0xf1,0x9e,0xdd,0x0d,0x47,0x78,0x92, +0xd0,0xb2,0x8f,0xee,0x58,0xd3,0x13,0xf1,0x93,0x53,0xa6,0x20, +0x6b,0x4f,0xa8,0x5c,0x38,0xda,0xe9,0x6a,0xf7,0xa9,0x0a,0x77, +0xbe,0xec,0x5c,0xd5,0xdb,0xb3,0xd8,0x6d,0x31,0x60,0xe5,0xcd, +0xa2,0xb8,0x04,0x82,0xd5,0x3c,0xaa,0xd2,0x91,0x25,0x1a,0x57, +0x81,0x1d,0x6a,0x56,0xae,0x1e,0x06,0x8b,0x42,0xa2,0x2e,0xdf, +0x78,0x7c,0x4b,0x3c,0x9a,0x78,0xa2,0xbc,0xd3,0x27,0xd7,0xcc, +0xc9,0xef,0xee,0xe6,0xc3,0xdd,0xa0,0x1e,0x35,0x58,0x04,0xb9, +0x42,0xda,0x83,0x00,0xce,0xcb,0x7e,0xa4,0x3c,0xb0,0xa7,0x39, +0x08,0x38,0x51,0x86,0x68,0x4b,0xba,0xcf,0xd3,0x30,0xf8,0x65, +0x4f,0x2f,0xa7,0xcb,0x13,0x62,0x01,0xcb,0x3f,0xc2,0xbe,0x58, +0xa5,0x2d,0x07,0x0c,0xc4,0xc2,0x53,0xac,0xfa,0x93,0x8c,0xc7, +0xe4,0x7b,0x49,0x16,0x20,0xfc,0xaf,0xad,0x81,0xb1,0x17,0x73, +0x41,0x6d,0x4b,0xb3,0x19,0xf4,0x40,0xef,0xba,0x73,0xca,0x5a, +0xff,0x35,0xf6,0x05,0xb3,0x52,0xcf,0x5c,0x5f,0xf4,0xd4,0xf5, +0x67,0x39,0xb1,0xda,0x57,0x1d,0x7f,0x0f,0x45,0x9c,0xff,0x15, +0x04,0x11,0x20,0xd2,0xb3,0x70,0xfc,0xe1,0x0d,0x61,0xce,0xbb, +0x53,0x00,0x26,0x7e,0x46,0xe5,0xb1,0x19,0xa0,0xcc,0x1a,0x33, +0x7e,0x93,0xdf,0xbe,0x18,0x2f,0x3c,0x02,0x32,0x09,0x75,0x6d, +0x17,0x4a,0x02,0x96,0xc3,0x8e,0x46,0xed,0x06,0xa3,0x49,0x16, +0x35,0xc7,0x41,0x2e,0x71,0xd6,0x30,0x52,0xea,0x0a,0xb8,0x85, +0xf2,0x74,0xd9,0x85,0xa3,0x69,0x6f,0x97,0x1d,0x12,0xe9,0x72, +0x84,0xfe,0x5d,0x1a,0xd2,0x50,0x07,0x54,0x1d,0xab,0x37,0xf2, +0x21,0x7e,0x1a,0x08,0x12,0x34,0xcf,0x8a,0x1d,0xeb,0x03,0x92, +0x29,0xd1,0x48,0x1c,0xbf,0x81,0xce,0x11,0xe6,0xdd,0x73,0x51, +0x6b,0xf1,0x1d,0x81,0x9e,0xca,0xd3,0xd1,0x3d,0xed,0x8d,0xdf, +0x6a,0xca,0xa3,0xaa,0xd0,0xd9,0x1d,0x53,0x00,0x6d,0x37,0xda, +0x86,0x24,0x1b,0xf3,0xe1,0x3b,0x6b,0xae,0xa6,0x1d,0xe9,0x1d, +0xbb,0xc3,0x77,0x9a,0x07,0x76,0x55,0x9b,0x65,0x7e,0x3f,0x79, +0xbd,0x1d,0x6a,0xec,0x89,0x44,0x30,0xd2,0x09,0x93,0x5b,0x0f, +0x71,0x56,0x3d,0x62,0x0a,0xf5,0x0c,0xee,0xff,0xa5,0x37,0xcd, +0x8e,0xbd,0xe9,0x56,0xf6,0x8c,0xef,0xe9,0x62,0x67,0xe9,0x42, +0xef,0x7d,0xf4,0xd3,0x87,0x8c,0xec,0xc8,0xd0,0xf6,0xfe,0x24, +0xf9,0x5a,0x92,0x7f,0x07,0x6e,0x36,0xda,0x6b,0x46,0x74,0xb3, +0x92,0x34,0x6f,0x77,0x9e,0xa4,0xcd,0xdf,0x9e,0xc7,0x79,0xdd, +0xd7,0x8a,0x69,0x54,0x63,0x66,0x3e,0xc7,0xf4,0xc5,0x31,0x61, +0x25,0x45,0x92,0xff,0xda,0xe0,0x7d,0x11,0x05,0x7d,0xd8,0x8f, +0xc1,0xf1,0x50,0xb8,0xba,0x05,0xa1,0x9f,0x9f,0xcf,0x77,0x53, +0x88,0xbd,0x9b,0x16,0xf6,0x94,0x00,0x44,0x72,0xc0,0x80,0x52, +0xbe,0x34,0xf7,0x4e,0x7b,0x41,0x73,0xe5,0x59,0x6e,0x4e,0x7b, +0x45,0x49,0xd1,0x4e,0x2e,0xb2,0xc2,0x26,0xb3,0x3d,0x05,0x54, +0xaa,0xd3,0x47,0xbc,0x70,0x5b,0x0f,0x02,0x4a,0x5f,0xd2,0x12, +0xd5,0xa4,0x8e,0x5a,0xc9,0x45,0x64,0x1d,0xf6,0x4e,0x5f,0xab, +0x06,0x18,0xc1,0xc0,0x82,0xba,0x98,0x9f,0xca,0x60,0xc1,0xb3, +0xd2,0x15,0x67,0x5b,0x05,0x33,0x4b,0xb9,0xdc,0x8d,0x58,0xe0, +0x65,0xde,0x0e,0xe4,0x47,0xb2,0x85,0x3a,0x90,0xa3,0x2c,0xce, +0xe7,0x05,0xfc,0x16,0x33,0xe0,0x8b,0x7c,0xae,0x0d,0x33,0x83, +0x4f,0x5c,0xf8,0x99,0x51,0xa3,0x84,0xac,0x5e,0x9c,0x3c,0x48, +0x6b,0x30,0xc6,0x3d,0xf9,0xfe,0x59,0x3e,0xfe,0xe3,0xdb,0xc1, +0x29,0xf4,0x46,0x9e,0x21,0x81,0xa6,0xe2,0xd0,0x78,0x9c,0xdf, +0xf0,0x41,0x51,0x8c,0xe3,0x10,0xc6,0xed,0x60,0x64,0xa0,0xe2, +0x4e,0xa0,0xe6,0x0f,0xa6,0x54,0x8f,0xfb,0xfa,0x58,0xe2,0x7d, +0xe5,0x98,0x64,0xff,0xa1,0xa9,0x35,0x90,0xc9,0xda,0x58,0x7c, +0xa1,0x6f,0x3b,0x71,0x85,0x6d,0xdd,0x96,0x0d,0x8e,0x39,0xc8, +0x9f,0x27,0xd4,0x68,0x48,0x49,0x0b,0x39,0x0b,0xe1,0x38,0xc5, +0x14,0xad,0x84,0x82,0x5d,0xa0,0x3f,0xa0,0x29,0x2d,0x47,0xd9, +0x70,0x6a,0x8e,0x4f,0xf4,0xaa,0x79,0x41,0x71,0x85,0x1b,0x47, +0x6e,0x37,0x18,0x8d,0xae,0x37,0x1f,0xc8,0x5b,0x6e,0x14,0x7a, +0x53,0x04,0xa5,0xc2,0xfc,0x40,0x5f,0x97,0xe3,0xf2,0x0f,0xc6, +0xb9,0xce,0x00,0x17,0x02,0x5c,0xb4,0x81,0x39,0xc1,0xd6,0x5b, +0x2a,0xca,0xfb,0x53,0x91,0x0c,0x19,0xfe,0x5b,0x30,0xeb,0x92, +0x33,0xc0,0x6d,0x62,0x3d,0xd6,0xb1,0x75,0xb5,0x5b,0x69,0x9c, +0xaf,0x82,0x2c,0x73,0xd8,0x12,0x25,0x10,0xe6,0xc9,0x55,0x1d, +0x90,0xac,0x14,0x09,0xb5,0x16,0x6d,0x80,0x4c,0xd7,0x18,0x77, +0x7d,0x6d,0x15,0xfb,0xf2,0x4b,0x98,0x2f,0x30,0x72,0x95,0x14, +0xf2,0xf5,0x07,0x73,0x94,0x46,0x29,0x1d,0xd2,0xeb,0x78,0xbe, +0x62,0xdf,0xb0,0xe6,0x95,0x8b,0xde,0x45,0x64,0xc4,0x71,0x32, +0x52,0xec,0x9c,0x5c,0x48,0x32,0x95,0x85,0xf2,0x18,0x8a,0xa2, +0x4f,0xab,0xa4,0x28,0x22,0x37,0x6d,0xac,0xaf,0x18,0xca,0xa8, +0x4c,0x29,0xaf,0xb0,0xe5,0x29,0xc2,0x11,0xd7,0x46,0xb2,0x69, +0x83,0x61,0xe4,0xc5,0xbc,0xe7,0x25,0xed,0x5b,0x1f,0x42,0xdb, +0xd8,0xf4,0x47,0x54,0x8b,0x94,0x7c,0x24,0xbf,0x3c,0x79,0x6e, +0x0a,0x49,0x61,0x27,0x83,0x2f,0x24,0xe1,0x16,0x2b,0xdd,0xfe, +0x15,0x8f,0xca,0xac,0xa3,0xdc,0x1a,0x57,0x0d,0xed,0x6c,0xea, +0x39,0x5f,0x38,0xf3,0xae,0xb7,0xc8,0x09,0xd7,0x27,0x53,0xe6, +0x07,0x1d,0x00,0x3d,0xb3,0x9d,0xa9,0xbd,0xea,0xb9,0x3d,0xae, +0xc5,0x4c,0x9e,0x1c,0x4f,0xef,0x9b,0x3b,0x46,0xe1,0xa8,0xc6, +0x1b,0xd9,0xee,0x46,0x5c,0xdc,0xbe,0x7e,0x28,0x51,0xf1,0x96, +0x9a,0xb1,0x23,0x8c,0x77,0xce,0xb7,0xad,0xc5,0x33,0x9e,0x2e, +0x3f,0xfa,0x72,0x94,0xcc,0x70,0x89,0x9d,0xa8,0x06,0x9d,0x6b, +0x32,0x6f,0x39,0x70,0x49,0xe4,0x15,0x06,0xe3,0x17,0x10,0x24, +0x81,0x8e,0x31,0x08,0x72,0x11,0x10,0x60,0x91,0x0a,0x01,0x14, +0x8f,0xf0,0x61,0x69,0xe7,0x3e,0xd8,0xb0,0xa7,0xa1,0xb1,0x5e, +0x04,0xd5,0x29,0x88,0xf9,0xba,0x8a,0x5e,0x1a,0xba,0x74,0x67, +0x99,0xa9,0x1d,0xe7,0x8c,0x83,0x55,0x70,0x61,0xaf,0x2e,0xc7, +0x61,0xc1,0xae,0x81,0x40,0xe7,0xf0,0x92,0x5e,0x6c,0xf9,0x91, +0xed,0xc4,0x00,0x8b,0x2c,0xcb,0x80,0xb6,0xa6,0x03,0xcd,0x64, +0xa4,0xe5,0xf4,0x66,0x0a,0x43,0xca,0x95,0x32,0xbd,0x88,0x6d, +0xc6,0x4c,0xc5,0x2c,0xce,0xd3,0x4c,0xee,0xcf,0xb8,0x0b,0x3f, +0xa0,0x46,0x68,0xaa,0x38,0x1f,0x8f,0x2a,0x10,0x27,0x1f,0x43, +0xba,0xb5,0xee,0xe7,0x6d,0x9d,0xcb,0x02,0x8d,0x7b,0x11,0x52, +0x86,0x75,0x11,0xee,0x0c,0xeb,0xc3,0x3c,0x22,0xe7,0x24,0xce, +0x28,0x53,0xd5,0x4b,0xfd,0x3f,0xba,0x3d,0x9f,0x25,0x74,0x07, +0x73,0x46,0x67,0x92,0x1b,0xe6,0x2d,0x7c,0x2d,0xfb,0x25,0xb7, +0xa9,0xcc,0x90,0x61,0x1e,0x40,0xee,0x07,0x77,0x55,0x34,0xcc, +0x00,0x3d,0x34,0x7c,0x4e,0xc1,0xbf,0x38,0xc0,0xd0,0x44,0x30, +0x98,0x79,0x5f,0xe3,0xc5,0x0b,0xa0,0x80,0xf5,0xbc,0x07,0x7f, +0x25,0x07,0xc4,0xbb,0x77,0x2d,0xc8,0xeb,0xc4,0xd6,0xe0,0x3c, +0xe4,0x78,0x6b,0x5c,0x9b,0x69,0x71,0x6e,0xa1,0xf6,0x3d,0x4e, +0x22,0xc6,0xb2,0x70,0x7f,0xff,0xb4,0xe3,0x13,0xfb,0xa1,0xef, +0xf2,0x4b,0x4b,0xc4,0xbd,0xac,0xc8,0x08,0x0c,0x57,0x87,0xfd, +0xbe,0x75,0x76,0x37,0x9c,0x00,0x65,0xaf,0xee,0xa1,0x76,0xde, +0xea,0xde,0x55,0xfb,0x04,0x70,0xb7,0xfa,0x01,0x62,0x80,0x93, +0xf1,0xd0,0x02,0xa5,0x57,0x7b,0xdc,0x99,0x83,0x4f,0x4c,0xb4, +0x6c,0xb6,0x35,0x90,0x49,0x41,0xf1,0x5d,0x8e,0xfe,0x8c,0xc0, +0xd7,0x08,0x71,0xa7,0x5e,0xc9,0xdc,0x28,0x78,0xfd,0xc2,0x73, +0x79,0xba,0xcf,0x0f,0xe5,0xec,0x39,0x18,0x07,0xb3,0x60,0x8c, +0x76,0x36,0x90,0x00,0xef,0x30,0xbc,0x6d,0x30,0xf8,0x1e,0x83, +0x8f,0x18,0x9c,0x04,0x13,0x08,0x7e,0x79,0xaa,0x29,0x7e,0x82, +0xf1,0x7f,0xfc,0xf6,0xe6,0x2f,0x29,0x0c,0xe7,0xd4,0x20,0xbc, +0x79,0xda,0x33,0xc6,0x4c,0xd7,0xbb,0xa1,0x4d,0xe3,0x94,0xc8, +0x21,0x01,0x44,0x4d,0xf9,0x0d,0x1a,0x5e,0xc9,0xc0,0x55,0x37, +0x8b,0x1f,0x98,0xa5,0x83,0x58,0x56,0xc1,0x1d,0x0c,0x37,0x9f, +0xeb,0x6a,0x1f,0x06,0x41,0x84,0xd2,0x24,0x09,0x1e,0x83,0x28, +0xd1,0x72,0x3d,0x14,0x61,0x0f,0x0a,0x05,0x54,0x24,0x67,0xc4, +0x12,0x31,0x16,0x9c,0xc0,0x44,0x68,0x32,0x10,0x44,0x2b,0xbc, +0x2b,0x0f,0xf5,0x74,0xd0,0x5a,0x8f,0xe9,0xb8,0x3b,0xac,0x38, +0xb8,0x5a,0x7d,0xc6,0x91,0xf3,0x20,0x98,0x01,0xd1,0xb1,0x80, +0xdc,0x1c,0xcb,0xc3,0x36,0x87,0x95,0x85,0xd7,0x14,0x8a,0xd1, +0x29,0x0c,0x6f,0xac,0xb2,0x53,0xa8,0x56,0x44,0x01,0x17,0x94, +0x8e,0xa7,0x78,0x48,0x4c,0x39,0x36,0x46,0x1d,0x70,0x20,0x00, +0xf0,0x96,0x56,0x1a,0x61,0xfe,0xe4,0x0a,0x29,0x91,0x37,0x6a, +0x53,0xeb,0x65,0xca,0xc9,0x0b,0xcd,0xb8,0x4a,0x7f,0xe4,0xab, +0x6e,0x06,0x2f,0x62,0x9d,0x39,0x07,0x39,0xbe,0x55,0x6d,0x91, +0x45,0x6d,0xec,0x98,0x21,0x78,0x45,0xd4,0x9a,0x6e,0xd4,0xde, +0x97,0xad,0xe5,0x86,0x24,0x5c,0xe6,0x51,0x08,0x7a,0xc9,0x6c, +0x2c,0x2e,0x5a,0xd3,0x91,0x6b,0x89,0xf4,0xee,0xb5,0x78,0x8d, +0x49,0x52,0x0f,0x4b,0x60,0x45,0xc8,0x36,0xe7,0xd8,0xea,0xd6, +0x62,0xa4,0xbd,0xe8,0x53,0xbc,0xd5,0xc3,0x07,0x74,0x2c,0x2e, +0x8f,0x30,0xe8,0xd5,0xe7,0xec,0x17,0x47,0x26,0x82,0x58,0xda, +0x54,0xe2,0x7c,0x5d,0x3d,0x64,0x71,0x6c,0x32,0xc4,0x41,0x60, +0x88,0x21,0xa0,0x39,0x7f,0x00,0x62,0x56,0xd1,0xd9,0x85,0xa6, +0xdc,0x1d,0xb6,0x74,0x27,0x4d,0xd3,0x00,0xaf,0xe0,0x00,0xc1, +0x06,0x85,0x98,0xd9,0xf4,0xdd,0x4f,0xe1,0x62,0x79,0x5a,0xfe, +0xba,0x8a,0xe7,0xd1,0x4f,0xef,0xa7,0x8f,0xbc,0xdc,0x3c,0xd4, +0x5b,0x49,0xcd,0x5a,0xc9,0x32,0x2e,0x20,0xe5,0xda,0x4f,0xe1, +0x6a,0xec,0x6b,0x7a,0xcc,0x3b,0x9b,0x86,0x37,0xd1,0xbf,0xa7, +0x02,0x0d,0x05,0xd2,0x08,0x9f,0xc0,0x00,0x58,0x2e,0x21,0x00, +0x3f,0x10,0x3a,0xe2,0x4d,0x95,0x18,0xd8,0x43,0x00,0x75,0x88, +0x3f,0xc0,0x17,0x5f,0x82,0x5a,0x3e,0xbd,0xb9,0x7d,0xf3,0x57, +0xf8,0xc3,0x5f,0x89,0x78,0x2b,0x11,0x6f,0x83,0x8b,0xe0,0x3d, +0x66,0xd3,0xe5,0xf2,0x3c,0x39,0x2f,0x8f,0xe7,0xe5,0xfe,0xbc, +0x6c,0xce,0x9c,0x91,0x3f,0x6f,0xa7,0x8f,0xa9,0xad,0xba,0x63, +0x5e,0x45,0x79,0x49,0x57,0xc1,0x9e,0xa4,0x39,0xd4,0xcb,0x0a, +0x05,0x7e,0xaa,0x91,0xde,0x75,0x60,0x17,0x03,0xdb,0xd3,0x9a, +0x2c,0x81,0x68,0xdc,0xc8,0x05,0x51,0x22,0x30,0x16,0xa1,0xe6, +0x12,0x49,0x03,0x4b,0x6a,0x6e,0x9e,0x48,0xd6,0x79,0x89,0x85, +0x70,0x9a,0xfd,0x25,0x30,0xed,0xb8,0x0e,0xcc,0xca,0x15,0x40, +0xe7,0x3f,0x89,0x1c,0x33,0x87,0x68,0x78,0x81,0x19,0xac,0xd7, +0x87,0x38,0x9b,0x04,0x29,0x36,0x82,0x91,0x18,0x42,0x7a,0x4b, +0x46,0x48,0x1b,0xbb,0xfc,0x90,0x68,0xe9,0x49,0xbd,0xd9,0x9c, +0x9c,0x63,0x97,0x8c,0x3e,0xce,0x08,0x2d,0x1b,0x38,0xe4,0x00, +0x00,0xc3,0x59,0xe1,0xc4,0xfd,0x36,0x20,0x8e,0x90,0x2f,0x62, +0x61,0x19,0x23,0x88,0x84,0x0e,0x5c,0xc2,0x48,0x29,0xc1,0x24, +0x5e,0xee,0x43,0xbc,0x6c,0x36,0x0b,0x15,0x1e,0x84,0x2e,0x68, +0x87,0x28,0x8a,0xe6,0x93,0x09,0xda,0x56,0x28,0x16,0x61,0xa2, +0x08,0xa0,0x27,0xd4,0x16,0xa6,0x22,0x83,0xc8,0x1c,0x98,0x16, +0x46,0x40,0x24,0x78,0xb4,0x11,0x3c,0x24,0x29,0x30,0x71,0xac, +0x29,0x31,0xb0,0xb4,0xb4,0xb4,0x76,0x01,0x37,0x44,0x4c,0x95, +0xd2,0xb5,0x20,0xfd,0xc1,0x63,0x46,0x1e,0x16,0xc3,0x6a,0x37, +0xc0,0x56,0x55,0x5e,0x6c,0x71,0x97,0x49,0xfb,0x0f,0x1d,0x5c, +0x43,0x52,0x56,0x9f,0x2f,0xd1,0x33,0x96,0x29,0x99,0x83,0x0b, +0xfc,0x89,0x4c,0x3f,0xcb,0xad,0xd3,0x09,0x50,0x91,0x4d,0x26, +0xf0,0xf9,0xdc,0x79,0x5d,0x10,0xd9,0x05,0x04,0x79,0xb5,0xf7, +0x74,0x21,0x22,0x1b,0xc0,0xdb,0xe6,0xb5,0xc9,0x04,0xd6,0x9a, +0xa9,0x5b,0xa0,0x0a,0xf0,0xf5,0x01,0xa3,0xd3,0xe6,0xf8,0xfd, +0x99,0x87,0x10,0x0d,0x01,0x23,0xd3,0x7b,0x6a,0x83,0xb6,0x07, +0xe0,0xcf,0x75,0x20,0xe4,0x46,0x97,0x7b,0x6c,0x89,0x10,0xa4, +0x34,0xef,0x39,0x94,0x6a,0x10,0x11,0xf6,0x27,0x2b,0x06,0x5d, +0x9d,0x6e,0x6c,0xcf,0x58,0x10,0x5f,0x97,0xbe,0xe6,0xf1,0xd7, +0x1d,0xe8,0xde,0xa3,0x7c,0x9c,0xc3,0x91,0xc5,0x92,0xd7,0x6c, +0x09,0x6b,0x7e,0x2d,0xda,0xe8,0xee,0xc0,0x15,0x83,0x95,0xc0, +0xb9,0xde,0xef,0x85,0x20,0x51,0x44,0x64,0x29,0xfc,0x94,0x77, +0x66,0x2b,0x6d,0x37,0x3e,0x08,0x40,0x10,0xa1,0xb9,0xee,0xa5, +0x33,0xdb,0xfa,0x0f,0x7e,0xac,0x09,0x7c,0xee,0xe7,0x25,0xc3, +0x61,0x2a,0x53,0x63,0x88,0xd7,0x1e,0xc9,0x13,0x7b,0x35,0x3e, +0x0c,0x4d,0xcf,0x5a,0x26,0xd0,0x5b,0xff,0x41,0xc7,0x58,0x08, +0x9d,0x3e,0x55,0x1b,0x14,0xb2,0x68,0x3e,0xc7,0xcf,0x01,0x87, +0x2b,0x69,0x26,0x9c,0xd2,0x59,0xa8,0xf1,0x3e,0x21,0x6a,0x07, +0xce,0xed,0x53,0xaf,0x9d,0x3a,0x71,0xf5,0xe0,0x4d,0x9c,0xf6, +0x7e,0x0a,0xc4,0xa8,0x7e,0xe4,0x78,0x8a,0xd6,0x4e,0x9c,0x42, +0xe3,0x28,0x43,0x94,0xe3,0xb1,0xd2,0xc8,0xff,0xcf,0xe7,0xdb, +0x91,0xbb,0x20,0x3f,0x9f,0xf9,0x86,0x59,0x3a,0xc7,0x76,0xbb, +0xea,0xce,0xba,0xb4,0x28,0xb9,0x98,0x66,0x61,0xf3,0x36,0x54, +0x18,0x07,0x00,0x36,0x00,0xe1,0xfe,0xdf,0xd5,0xda,0xdb,0x1e, +0x71,0xc8,0xf0,0x2a,0xec,0x35,0x89,0x53,0x75,0x51,0xbb,0xc4, +0x32,0xec,0x94,0x93,0x3c,0x53,0x6f,0xcf,0xba,0x05,0x39,0x8c, +0x81,0x7b,0xb3,0xbc,0x56,0x69,0x1d,0xf7,0x6b,0x01,0x4b,0xb9, +0xa8,0x7b,0x58,0x02,0xa3,0xf4,0xbf,0x01,0x00,0x00,0xff,0xff, +0xe9,0x76,0xa5,0x35,0xdb,0xfb,0x08,0x00, + })) + + if err != nil { + panic("Decompression failed: " + err.Error()) + } + + var b bytes.Buffer + io.Copy(&b, gz) + gz.Close() + + return b.Bytes() +} + +func init() { + go_bindata["/scripts/stats-modules.js"] = scripts_stats_modules_js +} diff --git a/dashboard/resources/scripts-stats-scripts.js.go b/dashboard/resources/scripts-stats-scripts.js.go new file mode 100644 index 00000000000..98c7386738d --- /dev/null +++ b/dashboard/resources/scripts-stats-scripts.js.go @@ -0,0 +1,2748 @@ +package resources + +import ( + "bytes" + "compress/gzip" + "io" +) + +// scripts_stats_scripts_js returns raw, uncompressed file data. +func scripts_stats_scripts_js() []byte { + gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ +0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0xec,0xbd, +0xfb,0x7f,0xdb,0xb6,0x92,0x28,0xfe,0xfb,0xfd,0x2b,0x1c,0x9e, +0xbb,0x2e,0x69,0x41,0xb2,0xa4,0x34,0x7d,0x50,0x65,0xf4,0x49, +0xdb,0xf4,0x9c,0xdc,0x6f,0xfa,0xd8,0x36,0xa7,0xe7,0x74,0x15, +0x6d,0x3e,0x94,0x44,0xcb,0x8c,0x65,0x51,0x21,0x69,0x9b,0x8c, +0xad,0xfd,0xdb,0xbf,0xf3,0xc0,0x93,0xa2,0x64,0xa7,0xaf,0xbb, +0x7b,0x77,0x4f,0x4f,0x64,0x02,0x18,0x0c,0x06,0x83,0x01,0x30, +0x00,0x06,0x83,0xeb,0x65,0x74,0x76,0xb5,0x9e,0x97,0x69,0xb6, +0xf6,0x17,0x8f,0x45,0x99,0x6d,0xb2,0xb7,0x45,0xb6,0x0e,0x6e, +0x55,0xec,0xd1,0xf5,0xf2,0x4d,0x52,0xcc,0xe3,0x4d,0xf2,0xa6, +0x28,0x73,0xbf,0x0a,0x6e,0xf3,0xa4,0xbc,0xca,0xd7,0x47,0x55, +0x2f,0x4f,0x36,0xab,0x78,0x9e,0xf8,0x26,0xf9,0x4d,0x9e,0x08, +0xef,0x7f,0x0f,0x5e,0xbf,0xfe,0xc8,0x0b,0xb6,0x36,0x86,0x32, +0x87,0x40,0x5c,0x26,0xdf,0xaf,0xff,0x91,0xe5,0x0b,0xbf,0x10, +0xab,0x64,0x2d,0xf2,0xe4,0x3a,0xb8,0xbd,0x8e,0xf3,0xa3,0xf9, +0xba,0x8c,0xfa,0x50,0xf6,0x45,0x54,0xf4,0x8a,0xcd,0x2a,0x2d, +0x7d,0x2b,0xcb,0x9b,0x1b,0xc8,0x01,0x88,0x83,0x91,0x2c,0xb8, +0x88,0x20,0xe3,0xd8,0x47,0x70,0xf8,0x07,0x54,0x5c,0x27,0x79, +0x91,0xf8,0x41,0xd0,0x3b,0x4b,0x57,0x65,0x92,0xfb,0xba,0x42, +0x37,0x9a,0x58,0x28,0xa1,0x13,0xdd,0xf4,0xa0,0xd4,0x65,0x79, +0x8e,0x85,0x3f,0x8d,0x20,0x6a,0x1b,0x98,0xdc,0x21,0xe2,0xfa, +0x70,0x04,0xa2,0x90,0x71,0xe3,0xa2,0xf7,0x36,0x4b,0xd7,0xbe, +0xe7,0x05,0xbd,0x32,0x4f,0x2f,0x19,0xe3,0xa4,0x3f,0xed,0x15, +0xab,0x14,0x98,0xd4,0xc7,0x4c,0x2e,0x53,0x6e,0xf2,0xb4,0x4c, +0xfc,0xcb,0x62,0x09,0x5c,0x58,0xf6,0xe6,0xd9,0xfa,0x2c,0x5d, +0xf6,0xd2,0xe2,0xbb,0x6c,0x91,0x8c,0x37,0x79,0x36,0x4f,0x0a, +0x60,0x47,0xb9,0x48,0xf2,0xbc,0xa7,0x41,0x3b,0xde,0xeb,0xb5, +0x17,0x84,0x00,0x5c,0x64,0xab,0xa4,0xb7,0xca,0x96,0x84,0xc0, +0xc1,0x7b,0x19,0x5f,0x24,0xc0,0xbe,0x24,0xf1,0x17,0x9a,0xfe, +0x45,0xef,0xcd,0x9b,0xeb,0x25,0x46,0xbe,0x79,0x13,0x3d,0xea, +0x8b,0x45,0x6f,0x0d,0xc5,0x14,0xa6,0xf1,0x35,0x24,0xb1,0x1e, +0xc0,0x28,0xdd,0x2f,0xcf,0xd3,0x42,0x4c,0xa6,0xc1,0x56,0x2c, +0x1a,0x0d,0xaa,0x41,0xf2,0x2c,0x2b,0x05,0x7d,0x72,0x6b,0xa6, +0x62,0x1e,0x61,0x5c,0xef,0x3a,0x5e,0x5d,0x25,0x85,0x58,0x47, +0xf3,0xf1,0x5c,0x72,0x29,0xec,0x8f,0xce,0xb2,0xdc,0x27,0xe8, +0xde,0xe6,0xaa,0x38,0xa7,0xdc,0x81,0x48,0xa3,0xfe,0x68,0xfd, +0x34,0x1d,0x75,0x3a,0x69,0xe0,0x62,0x9f,0x4f,0xd2,0xa9,0xc4, +0xae,0x04,0x80,0x42,0x0e,0x35,0x8b,0xb8,0x8c,0xdf,0x2c,0xae, +0x40,0x74,0x50,0x62,0xb0,0xda,0x4c,0xc8,0x5a,0x54,0xd1,0x62, +0x94,0x9e,0x81,0x3c,0x01,0x63,0x9f,0xe5,0x79,0x5c,0x43,0x62, +0x80,0x24,0x54,0xd1,0x64,0x8a,0xc5,0x02,0x79,0x0b,0x49,0x9c, +0xa6,0xa0,0x62,0xd2,0x5a,0x10,0x03,0x31,0x41,0x30,0x4a,0x56, +0x45,0x72,0xa4,0xb0,0x7e,0x3f,0x7b,0x9b,0xcc,0x4b,0x44,0x7b, +0x5b,0x45,0xb7,0x5b,0xaa,0x5f,0x7a,0x94,0x02,0xcb,0x83,0x0a, +0xc0,0xa3,0x3d,0x58,0xb6,0xaa,0x1b,0x39,0x15,0x59,0x65,0xf1, +0xe2,0xcd,0x79,0x5c,0xfc,0x90,0x67,0x65,0x36,0xcf,0x56,0xfe, +0x55,0xbe,0xb2,0x5b,0x86,0xd2,0x37,0x32,0xf1,0xc7,0xe7,0xbd, +0x32,0x29,0x4a,0x82,0xd9,0xc5,0x92,0x16,0xdf,0xa4,0xab,0xc4, +0x41,0xd0,0x8f,0xa2,0x08,0xc2,0xbd,0x74,0xbd,0x48,0xaa,0xef, +0x91,0x7e,0x06,0x05,0xb1,0x4f,0x54,0x89,0x2d,0x98,0xaa,0xf3, +0x1c,0xd1,0x88,0x79,0xbc,0x5a,0xcd,0xe2,0xf9,0x05,0x89,0x2b, +0x4a,0x9e,0xf7,0xf2,0xfb,0x67,0x5f,0x87,0x47,0x5e,0x07,0x0b, +0x11,0x8b,0xc7,0x3d,0x05,0xa9,0xa5,0x0a,0x64,0x17,0xba,0x79, +0xb1,0x41,0x12,0x8a,0xcd,0xf1,0xb1,0x8f,0x7f,0x22,0xfc,0xe9, +0xe1,0x0f,0x48,0x71,0xf2,0x2a,0xa9,0x40,0x00,0x14,0x6e,0x93, +0x63,0xdb,0x42,0x09,0x12,0xea,0xe3,0xcf,0x1e,0x5a,0x8e,0xbe, +0x79,0xf1,0xf2,0x39,0x12,0x84,0x30,0xc1,0x88,0x84,0x60,0x51, +0x45,0x18,0x3a,0x5c,0xe9,0x11,0x80,0x3d,0x8d,0xfa,0x40,0x20, +0x46,0x73,0x06,0xee,0xb7,0x6d,0xe0,0x52,0x5e,0x82,0x00,0x28, +0x7d,0x77,0x95,0xe6,0x89,0xef,0x9d,0x15,0x1e,0x0e,0x26,0xf1, +0xe2,0x9b,0x5d,0x12,0x5b,0x5a,0xb8,0x2c,0x37,0x07,0x58,0x7a, +0xf4,0xb7,0x57,0xaf,0x7e,0x50,0x7c,0xa5,0x5a,0x40,0x39,0x91, +0x2e,0x0b,0x73,0x53,0x69,0xef,0xae,0x64,0xf3,0x1b,0x86,0xe7, +0xaa,0x17,0x6e,0xb2,0x02,0x84,0x1b,0x25,0x2f,0x5a,0x27,0x37, +0x47,0x5f,0x5e,0x9d,0x9d,0xc1,0xc8,0xb6,0x89,0x61,0xb0,0x7b, +0xb1,0x2e,0x11,0xae,0x77,0x0e,0xe4,0xc2,0xe0,0x37,0xf1,0x60, +0x34,0x29,0x93,0x75,0xd9,0xe5,0x6a,0x79,0x53,0x31,0xe8,0x07, +0xd8,0xdb,0x8a,0x1e,0x60,0xf4,0xa0,0x45,0xb2,0xdc,0x73,0xda, +0x34,0xb8,0x75,0xda,0x6b,0x7d,0xb5,0x02,0xb9,0x41,0x6e,0x70, +0x0e,0x2c,0xd5,0xca,0x00,0x73,0x46,0x05,0xc3,0xdb,0x06,0xfa, +0x1e,0x24,0x08,0xa0,0x2c,0xc0,0x9f,0x4e,0x54,0x49,0x46,0x5a, +0x59,0x93,0xf5,0xc2,0xca,0x69,0x95,0x83,0x65,0x50,0x75,0x50, +0x30,0x90,0xb8,0x77,0x0f,0x23,0x8e,0xe9,0x7a,0xd7,0x03,0xc4, +0xbe,0xdb,0x12,0x71,0x95,0x16,0x3f,0x5d,0xcd,0x16,0xe9,0x75, +0xba,0x48,0x7c,0x98,0xc3,0xa0,0xcd,0xca,0x74,0x7e,0x51,0x88, +0xcb,0xe0,0x16,0x7a,0x77,0x71,0x35,0xa3,0x20,0x0e,0x14,0x97, +0xc7,0xc7,0xf4,0x2d,0x09,0x7e,0x3a,0x80,0x09,0x12,0xfa,0x39, +0x32,0x5a,0x81,0x89,0xb7,0xe2,0x5a,0x80,0x2c,0x03,0x23,0x23, +0x85,0x1d,0x71,0x3e,0xa7,0x28,0xc6,0xdf,0x5b,0x64,0x97,0x31, +0x4c,0x12,0x01,0x8e,0x78,0xdd,0x01,0x8c,0x3d,0x36,0x56,0xb1, +0x88,0x7c,0x0a,0x4f,0x06,0xd3,0x2e,0x7f,0xf4,0xa7,0xc1,0x69, +0xa7,0x73,0x89,0xa3,0xd2,0x17,0xeb,0x11,0x0d,0x5c,0x6f,0xa3, +0xcb,0x51,0xb7,0xfb,0xf6,0x69,0x7f,0x14,0xf8,0xd7,0x51,0x87, +0xe1,0xd2,0x69,0xf7,0xed,0xc9,0x22,0x78,0x1a,0x31,0x01,0x90, +0xef,0xf8,0x58,0x11,0x26,0x07,0xb3,0x80,0x46,0xa6,0x6e,0x37, +0x15,0x6f,0x61,0xb0,0xed,0x74,0xde,0x7e,0x01,0x95,0xb2,0x31, +0x74,0x10,0xc3,0x17,0x12,0xc1,0x60,0x3a,0x0a,0x9a,0x08,0xd4, +0x90,0xa5,0xe2,0x77,0xb9,0x69,0xd5,0x97,0x6b,0xca,0xb2,0x58, +0x94,0x71,0x5e,0x46,0x1c,0x03,0xa4,0x89,0x02,0xd4,0x0c,0x15, +0xe4,0x3f,0x92,0x03,0x5d,0x28,0x56,0x15,0x02,0x30,0x4f,0x29, +0xe3,0x78,0x42,0x7f,0x28,0xd7,0x34,0x9c,0xe0,0x1f,0x41,0x31, +0xd3,0x76,0x02,0x7e,0x8c,0xd7,0x4b,0xd9,0x9e,0x7a,0xe4,0x63, +0xee,0xe7,0x98,0xc2,0xf4,0x8d,0x77,0x62,0x60,0xc2,0xde,0xdb, +0x6c,0x04,0x06,0xad,0xb6,0x53,0xe0,0xcb,0x78,0x96,0xac,0x08, +0x74,0xe1,0x67,0x79,0x0a,0x39,0xc4,0x0a,0xa3,0x0a,0x91,0xad, +0x16,0x84,0x47,0x40,0xef,0xe3,0x8f,0x22,0x7d,0x9f,0x88,0x4d, +0x0c,0x93,0x13,0x7e,0x45,0xdf,0xc6,0xe5,0x79,0xef,0x32,0xae, +0x7c,0x8a,0xef,0x07,0x1d,0x48,0x12,0xbe,0xb7,0x4a,0xce,0x4a, +0x0f,0xc6,0x69,0xc6,0x76,0x77,0xe7,0x41,0x75,0x4d,0x38,0x80, +0x26,0x43,0xf8,0x13,0x90,0x9e,0x40,0xb8,0x69,0x00,0x3b,0xcb, +0xca,0x32,0xbb,0x34,0x51,0x63,0x9c,0xa2,0x12,0x26,0x8f,0xe9, +0xea,0xc1,0xe4,0xb1,0x49,0xf2,0x32,0x85,0xfe,0x06,0x00,0x49, +0x2e,0x6e,0xab,0x50,0xd3,0x5a,0x87,0xb7,0x34,0x69,0x87,0x58, +0xc6,0x16,0x7a,0xce,0xa1,0xec,0x57,0x1b,0xe8,0x8e,0x09,0xe6, +0xd7,0x55,0x74,0xf3,0x8b,0x78,0x95,0x2e,0xd7,0x2a,0xca,0x9b, +0x53,0x79,0xde,0x56,0xcc,0xe2,0x22,0x59,0xa5,0xeb,0x44,0xa5, +0x48,0x5e,0x7e,0x29,0xa3,0x27,0x4c,0xfb,0x14,0xca,0x0f,0xc2, +0x87,0x55,0xc0,0x29,0xb5,0xd6,0xf5,0x79,0x70,0x05,0x1a,0xf9, +0x75,0x7d,0x9c,0x0a,0x48,0x32,0x9f,0x61,0x9c,0xa6,0x71,0xa7, +0x32,0xde,0x65,0xba,0x58,0xac,0x12,0x0f,0xa9,0xdf,0x11,0x97, +0x57,0xd8,0x6d,0x5c,0x71,0xe1,0xb1,0xa3,0x5d,0x5a,0x64,0xef, +0x81,0xf2,0xa2,0x7b,0xe4,0x62,0xdc,0x1d,0x84,0x83,0x11,0xc9, +0xd5,0xe0,0x14,0xa7,0x79,0xfc,0x1c,0xdf,0x2f,0x1f,0xb7,0xcb, +0x3c,0xbb,0xda,0x00,0xd1,0x71,0x7e,0xd1,0xa3,0x6f,0x98,0x15, +0xd2,0xe5,0x79,0xe9,0x89,0xcb,0xab,0x55,0x19,0x76,0xb1,0xf0, +0x6d,0xd8,0x02,0x76,0x93,0x2e,0x60,0xaa,0x70,0xa1,0x14,0x17, +0x97,0xeb,0x13,0x66,0xe5,0x07,0x09,0xa8,0x1c,0x6b,0x1e,0x24, +0x9f,0x7d,0x68,0xa6,0x21,0xb7,0x97,0xdd,0xc6,0x3b,0x28,0x0e, +0xc9,0xe8,0x43,0x71,0x24,0x55,0x5a,0xda,0x18,0x1a,0x62,0xb9, +0x9f,0x6c,0x53,0x4e,0xc5,0xe5,0xec,0x13,0xcd,0x03,0x64,0xb7, +0xe1,0x30,0x84,0xdc,0x4f,0xb6,0x0d,0xdd,0x26,0x8f,0xa5,0x1c, +0xe9,0x2c,0x79,0x84,0x28,0x41,0x03,0x9e,0xc8,0xce,0xce,0x8a, +0xa4,0x64,0x29,0x04,0xb1,0x8e,0xfe,0xe3,0x3f,0x7c,0x9f,0x52, +0x70,0x82,0xe2,0x0f,0x9c,0xa0,0x86,0xb0,0x0a,0x22,0x19,0x6d, +0xb6,0xb6,0x2b,0xb3,0x2c,0xa3,0x3b,0x22,0xb0,0x23,0xca,0x76, +0x9d,0x80,0x96,0x83,0x7c,0x01,0xaa,0xb6,0xf6,0xa8,0x03,0x92, +0xc7,0x44,0xc3,0xe0,0xb3,0x5e,0xae,0x12,0xc3,0xbf,0x2d,0x0e, +0xef,0x0f,0x47,0xec,0xa0,0xaa,0x9d,0xe2,0x1c,0xc4,0xdd,0xcf, +0x11,0xf5,0x0e,0x63,0xbf,0xa6,0x59,0xcd,0xe5,0x2c,0xcf,0x74, +0x92,0xb5,0xa6,0x7b,0x6f,0x60,0x1e,0x18,0xf9,0xf7,0xf0,0x4e, +0x8d,0xfb,0x30,0xec,0x53,0xd7,0x02,0x05,0x0a,0xb2,0x45,0xf7, +0x33,0xd3,0xfb,0xd6,0xeb,0xa8,0x96,0xea,0x78,0xc2,0xeb,0x60, +0xee,0x8e,0xf7,0x73,0xff,0x6f,0x2a,0x7e,0x00,0xf1,0x3f,0x73, +0x7c,0x88,0xd0,0x0c,0x20,0xec,0x6c,0x7f,0xeb,0xff,0x6c,0x43, +0xff,0x8d,0x81,0x64,0x85,0x76,0xd9,0xd8,0x23,0xda,0x24,0x83, +0xf0,0x7b,0xbb,0xc3,0x9f,0xbf,0x13,0xa0,0x0f,0x8b,0xda,0x4b, +0x41,0xe3,0x89,0x28,0x01,0xbf,0x54,0x5d,0xb3,0x88,0x02,0xe3, +0xdb,0x6d,0x48,0x00,0xdc,0x0c,0x11,0x7e,0xf7,0x68,0x04,0x5a, +0x24,0x67,0x3d,0x8e,0x14,0x5c,0xcd,0x66,0x1a,0x33,0x9c,0x86, +0xa8,0xc8,0x1a,0xae,0x04,0x8f,0x6d,0x91,0x3d,0xd0,0x8d,0x8a, +0x9b,0xb4,0x9c,0x9f,0x3b,0x6b,0x39,0x29,0xf7,0xc0,0x73,0x59, +0x74,0x97,0x73,0x90,0x32,0x50,0x4c,0x38,0x92,0x43,0x53,0x09, +0xc2,0x8b,0x5d,0xd0,0xeb,0x64,0x7b,0x81,0x0e,0x5a,0x24,0xdc, +0x88,0x61,0xd6,0xab,0xa2,0xae,0xa2,0xb7,0x57,0x83,0xf6,0x35, +0x83,0x95,0xc2,0xc5,0x88,0x40,0x72,0x1a,0x6d,0x09,0x86,0x68, +0xec,0xec,0x03,0x94,0x4d,0x4d,0x90,0x7d,0x4a,0xe6,0x0a,0xc8, +0x0c,0x36,0x28,0x8a,0x80,0x05,0xd7,0x75,0x20,0x80,0x41,0x31, +0x0e,0xda,0x26,0xbd,0xbf,0x25,0x7e,0x83,0x8a,0x8b,0x7f,0x60, +0xa1,0x04,0xe3,0xd7,0x26,0x5b,0xe9,0x06,0xca,0xf6,0x4c,0x64, +0x7a,0xcf,0xe0,0xb6,0xac,0x37,0x20,0x3c,0xf9,0x15,0xcc,0x7b, +0x82,0xb2,0xc7,0x00,0x7d,0x9d,0x84,0x8f,0x06,0xe2,0x22,0xa9, +0x43,0xb9,0x20,0x30,0x82,0x12,0xde,0xd2,0x20,0x19,0xde,0x16, +0x65,0x9e,0x5d,0x58,0x8a,0x80,0xda,0x05,0xc1,0x32,0x7a,0x38, +0xa4,0x7d,0x95,0xad,0xb2,0x7c,0x2b,0x18,0xee,0x1f,0xc8,0xa0, +0x03,0xc0,0x94,0xbe,0x15,0xd9,0x26,0x9e,0xa7,0xa5,0xee,0xb4, +0x83,0xa4,0xfb,0xc9,0x76,0x2b,0x70,0x3c,0x0c,0x6f,0xdb,0xd3, +0x58,0x6e,0x77,0x53,0xb7,0xf8,0xbf,0xb6,0xba,0x93,0xde,0xb7, +0xc3,0x80,0x12,0x86,0x97,0x06,0x03,0xfa,0xf7,0x30,0x00,0x56, +0x8a,0xab,0x03,0x35,0xa2,0x72,0x24,0x0f,0xce,0x60,0x95,0x76, +0x1f,0xe8,0x37,0x00,0xc3,0x90,0x3f,0x61,0x7f,0x7e,0x00,0xf4, +0x4f,0x34,0x51,0xb7,0xb1,0x45,0x60,0x75,0x90,0xc2,0x64,0xb5, +0x08,0x3d,0x52,0x9f,0xbc,0xdf,0x97,0x8f,0x30,0x08,0x3f,0x88, +0x85,0x6d,0x7c,0x3b,0xc4,0x0c,0xc0,0xfb,0x40,0x46,0x48,0x48, +0x66,0x02,0x42,0xff,0x83,0xfa,0xd4,0x7d,0xf0,0x0c,0x05,0x39, +0x0e,0xb5,0x1e,0xc0,0xca,0x96,0x7b,0xa0,0x1e,0xac,0x55,0x47, +0x97,0xf3,0x24,0x3b,0x16,0x73,0xdb,0x78,0xf9,0xb5,0x5c,0x57, +0xba,0xcc,0xc4,0xb1,0x77,0xa7,0x43,0xb6,0x30,0x53,0xcf,0x7a, +0xbd,0x27,0xd6,0x64,0x87,0x81,0x83,0xfd,0x13,0x7f,0x1e,0xdc, +0x3f,0xf1,0x87,0xfb,0xe7,0xfe,0x9a,0xac,0x92,0x25,0xcc,0x97, +0x7f,0xca,0xdc,0x00,0x8b,0xad,0x68,0x78,0xe2,0x26,0x41,0xdc, +0x22,0x5d,0x2f,0xc5,0xb2,0x1a,0xc8,0x29,0x62,0x96,0x5d,0xad, +0x17,0xc5,0xd8,0x0e,0xf4,0xaa,0x41,0xd8,0x07,0x90,0xe1,0x21, +0x90,0x61,0x68,0xcf,0x3a,0xab,0x1b,0xd0,0x9c,0xa8,0x28,0x99, +0x4e,0xd1,0x7e,0xd0,0xf1,0x33,0xfe,0x1c,0xf7,0x43,0x5c,0x17, +0x8a,0xd5,0x79,0x03,0x90,0x47,0x78,0x86,0xe4,0x6f,0x09,0xaa, +0x66,0x2d,0x1c,0xc4,0x7b,0x4f,0x68,0x14,0xa7,0x3f,0x3c,0xe5, +0xad,0x6e,0x84,0x82,0x8f,0x56,0xe7,0x7b,0xa6,0xa4,0x4e,0x04, +0xf5,0x94,0x53,0x43,0x77,0x75,0xd3,0x3e,0x27,0x21,0xd0,0x50, +0xce,0x30,0xdb,0x06,0xb7,0x1a,0x1a,0xef,0x6e,0x83,0xb5,0xb4, +0xed,0x4f,0xf5,0xe5,0x2c,0x53,0x0b,0x66,0x44,0xc5,0xab,0xe2, +0xe2,0x3c,0xde,0x24,0xd4,0x9b,0xa4,0x24,0x49,0xd5,0x08,0x8a, +0x28,0x22,0x2a,0xb1,0x59,0x1a,0xad,0x75,0x60,0x82,0x26,0x90, +0x1e,0xa9,0x46,0xb7,0x34,0x21,0x93,0x92,0xcc,0x73,0xf3,0x77, +0xf1,0x25,0x22,0xb5,0x3a,0x51,0xc0,0x25,0x99,0x7c,0x18,0xd2, +0x19,0x31,0xb0,0x3f,0x27,0x52,0xa7,0x33,0x62,0x40,0xe5,0xc3, +0xef,0x03,0x05,0x52,0x7d,0x4c,0x89,0x14,0xd4,0x45,0x52,0x68, +0x6f,0xe6,0x16,0x06,0xaa,0x41,0x93,0x8e,0x4d,0xce,0x96,0x91, +0xe9,0x65,0x9c,0x3e,0xba,0x6f,0x34,0xbd,0x6f,0x46,0xb6,0x97, +0x1c,0xf6,0x52,0xc9,0x1e,0xea,0xa0,0x60,0x67,0x7c,0xb3,0x07, +0x63,0x9d,0xd6,0x3e,0x02,0x3b,0xc9,0xfb,0x86,0x5d,0x07,0x48, +0x8d,0xb5,0x3b,0xa3,0x25,0x6a,0x39,0xad,0x43,0xe5,0x1f,0xa9, +0x00,0xd8,0x62,0x5c,0x3c,0xb8,0x1d,0x0a,0x82,0xff,0xf0,0x96, +0x90,0xd5,0xe2,0xfe,0x27,0x17,0xdb,0x72,0x8c,0x66,0x01,0xf2, +0xb8,0x34,0x4f,0x09,0x0e,0x6d,0x72,0x03,0x07,0x48,0x96,0x6d, +0x76,0x32,0x05,0x3f,0x61,0xf4,0x96,0xfa,0x5c,0x4b,0x22,0xb5, +0x86,0x3b,0xe2,0x9b,0xd4,0xfd,0x03,0xbd,0x85,0xc1,0x24,0xfe, +0xa1,0x8d,0x70,0xfd,0x92,0x6a,0xad,0x95,0xb0,0x3f,0xb2,0x33, +0x34,0x9a,0x80,0xff,0x86,0x0f,0x69,0x83,0x66,0x8f,0x59,0xed, +0xd1,0xe5,0x74,0xda,0xfe,0x1e,0xb3,0x72,0xb5,0x35,0x47,0x9f, +0xd0,0xe9,0xb4,0x31,0xb5,0xdb,0x4f,0x74,0xba,0xda,0x5f,0x6b, +0x57,0xec,0xfe,0xf8,0x4e,0xf3,0xd7,0x3c,0x5e,0xa4,0xb4,0xd1, +0xfa,0xc0,0x06,0xcb,0x61,0x2d,0xf6,0x41,0xea,0x4b,0x63,0xc8, +0xba,0x69,0x4a,0xe9,0x52,0x52,0x20,0x25,0xf4,0x7c,0x67,0xd0, +0x51,0x00,0x7f,0x93,0x63,0xce,0x6e,0x67,0x50,0x10,0x2c,0xea, +0x87,0x3b,0x85,0x0b,0xfb,0xc7,0x77,0x8b,0xf3,0x3f,0xb1,0x5b, +0xb4,0x8b,0xbe,0x1c,0x7f,0x35,0x8d,0x43,0x68,0x85,0x85,0x09, +0xfd,0x09,0x7d,0x42,0x12,0x42,0xa1,0x36,0x15,0xbb,0x65,0xd2, +0xf8,0x83,0x3a,0x00,0x72,0xff,0x7a,0x19,0xdd,0xa2,0x7d,0x01, +0x34,0x50,0xe8,0x0d,0x7a,0x8f,0x7b,0x43,0x4f,0x2c,0x1e,0x87, +0x96,0xa1,0x45,0xa8,0x3e,0x80,0xa8,0x0c,0x04,0x05,0xf4,0xcf, +0x88,0x77,0x21,0x7a,0x74,0xcc,0x8b,0x6d,0xd4,0x53,0x29,0x23, +0x6b,0x93,0xc2,0x9c,0xdc,0x67,0xb3,0xb7,0xfa,0x9c,0x03,0xbe, +0xa3,0x28,0x52,0xdb,0x18,0x90,0xb0,0x15,0x94,0xe7,0x1b,0x09, +0xdc,0x9a,0xcb,0x9b,0x64,0x94,0xe1,0x48,0x41,0x4d,0xbd,0x28, +0x52,0x65,0xf6,0xf0,0x30,0xcd,0x46,0x25,0x89,0x3c,0x88,0x88, +0x61,0x0e,0xa2,0xa1,0xc3,0xf8,0x88,0x7e,0x55,0xe8,0xee,0xee, +0x20,0x52,0x82,0x39,0x88,0xf3,0xbb,0xab,0xcb,0x59,0x92,0x1f, +0x26,0x8d,0x61,0x0e,0xa2,0xf9,0x32,0xcb,0x56,0x49,0x7c,0x0f, +0xaf,0x24,0xd0,0x41,0x44,0xaf,0xf2,0x24,0x69,0x6f,0x27,0xcb, +0x1e,0x01,0xa3,0x8f,0x8f,0xe1,0xd7,0xb2,0xd0,0xa0,0xfc,0xeb, +0x46,0x6d,0x0a,0x95,0xbb,0x53,0x50,0xfa,0xac,0x49,0xa6,0x06, +0x78,0xf4,0x88,0x21,0xd2,0x05,0x74,0x58,0x10,0xcc,0xc8,0x3e, +0x8a,0xd5,0x76,0x07,0x66,0xf3,0xb8,0x41,0xa3,0x3a,0xd9,0xac, +0xc4,0x1a,0x75,0xd2,0x34,0x1a,0xa0,0xed,0x4a,0x14,0xe7,0xcb, +0xab,0x4b,0x40,0xa8,0x8e,0x2b,0x47,0x68,0x05,0xc3,0x06,0x13, +0xb7,0x95,0x49,0x9d,0xa4,0x53,0xb6,0xf1,0x80,0xbc,0x68,0x06, +0x51,0x05,0x80,0x74,0x82,0xa1,0x69,0x54,0xf1,0xdf,0xad,0x91, +0x56,0xa2,0x42,0x9b,0x46,0xb4,0x33,0xeb,0xff,0xfc,0xf4,0xfd, +0x77,0x3d,0x3a,0xb9,0xf6,0xe9,0xb3,0x20,0x6e,0xa7,0x67,0xcc, +0x3b,0x66,0x36,0xf5,0x65,0x93,0xfd,0x4c,0x67,0x3e,0x93,0x16, +0x45,0xde,0xeb,0xd7,0x3d,0x2f,0x80,0xe5,0xcb,0xc6,0x98,0xf7, +0xd8,0xe6,0x31,0x12,0x0a,0x60,0xc8,0x2a,0x68,0x71,0x35,0x4f, +0x0c,0x60,0x2c,0x66,0x1a,0x34,0x96,0xd5,0x07,0x95,0x3e,0x9e, +0xc4,0xe6,0xe0,0xb2,0x13,0x41,0xe6,0xce,0x0c,0xd6,0x14,0xe9, +0x59,0x89,0x27,0xbc,0x31,0x9d,0x9b,0xf6,0xe2,0xcd,0x66,0x55, +0x13,0x06,0x11,0x6f,0xd9,0x94,0x06,0xe8,0x8d,0xe7,0x68,0xde, +0x93,0xe5,0x0e,0xc9,0x74,0xda,0x33,0xb2,0x45,0xe4,0x1b,0x93, +0x7a,0x77,0x87,0x47,0xe0,0x51,0x74,0x36,0x3e,0x0b,0xad,0x8e, +0x08,0x29,0xb8,0x37,0x1c,0x29,0x1e,0x40,0x38,0xd0,0xa7,0xd4, +0xe3,0x96,0x86,0x2f,0x76,0x6a,0x57,0x09,0xc3,0xae,0x6a,0x72, +0x36,0xdd,0x8a,0x2a,0xd8,0x86,0x6d,0x32,0x83,0x89,0x44,0xfe, +0x3c,0xbb,0x84,0x06,0x89,0x4b,0xbb,0x02,0x50,0x9b,0xd2,0x3a, +0xb1,0x9a,0xe8,0xa3,0xdc,0xeb,0x2c,0x5d,0x90,0xed,0x09,0x42, +0x20,0xb1,0xf0,0x07,0x92,0x61,0x6d,0x84,0x1f,0xc8,0x0c,0xea, +0x08,0x94,0xdf,0x6d,0x20,0xc5,0x93,0x68,0x20,0x71,0x79,0x5d, +0xdc,0xdb,0x3e,0x9b,0xf4,0xa7,0x63,0xa8,0x73,0x77,0x20,0xce, +0xa2,0x33,0x69,0xa8,0x31,0x08,0x82,0xd0,0xeb,0xa8,0x64,0xe2, +0x49,0x23,0x99,0x4e,0x29,0xf8,0x30,0xbb,0x08,0xec,0x46,0x80, +0x72,0x70,0x81,0xe7,0xb4,0xb6,0x32,0x20,0x3a,0x13,0x95,0xa8, +0xd9,0xa8,0x87,0x8c,0x85,0x90,0xca,0xa6,0xbd,0x10,0x5a,0x0b, +0x9c,0x51,0x0a,0x1a,0x2b,0x55,0xd1,0x99,0x1f,0x07,0x02,0x7a, +0x9d,0x0f,0x6d,0x5e,0x3f,0xad,0x02,0xa6,0x9d,0xb6,0xee,0x97, +0x6b,0xec,0x1f,0x00,0x5f,0x3d,0xad,0x03,0xd5,0x22,0x1c,0xab, +0x7a,0x45,0x5f,0xb2,0xf8,0x72,0x13,0xb5,0x0a,0xe0,0xec,0x69, +0x8c,0x47,0x29,0xf1,0xd3,0xd9,0x78,0x80,0x6a,0x0e,0x0f,0x15, +0x7b,0xc1,0xe3,0xee,0x8c,0x25,0x8e,0x06,0xdc,0x96,0x56,0x45, +0xb1,0x7a,0x14,0x55,0x63,0x6b,0x4c,0xaa,0x82,0x71,0x15,0x4e, +0xaa,0x69,0x38,0x99,0x52,0x5e,0xb6,0xe4,0x6a,0xcb,0x6c,0x6f, +0x9b,0x57,0x20,0x89,0x8f,0x1c,0x2c,0xc7,0xc7,0x95,0xcc,0x3b, +0x56,0x1f,0x21,0x0f,0x3c,0xd0,0x87,0xf7,0xa2,0xd3,0x34,0x78, +0x13,0xaf,0x53,0x91,0x44,0x70,0x8e,0xa0,0xe3,0x4d,0xbd,0xd0, +0x2d,0x72,0xdc,0x18,0x12,0xaa,0xc0,0xe9,0x1d,0x88,0xe5,0x23, +0xaf,0xd3,0x34,0x62,0xec,0x40,0x24,0x50,0x42,0xb6,0x35,0x8e, +0xf9,0x62,0x74,0xea,0xff,0xfb,0xdd,0xe4,0xdf,0x5f,0xbf,0x9e, +0x06,0x1f,0x9d,0xd2,0x84,0x0b,0xaa,0x90,0x5b,0x73,0xcc,0x43, +0x91,0x13,0x1e,0xe8,0x64,0x98,0xc7,0x3a,0x4c,0x60,0x11,0x83, +0x2f,0x6d,0xb1,0x86,0xb1,0x54,0xed,0xab,0x75,0xfa,0xee,0xca, +0x1a,0xe6,0xc8,0x20,0xe6,0x0c,0xad,0x5f,0x60,0x69,0x57,0x90, +0x20,0xd9,0x1c,0x24,0x7b,0x97,0xbb,0x3b,0xe8,0x3b,0xf8,0xa5, +0xcc,0x8e,0x18,0x29,0x96,0x1e,0x9d,0xdd,0xdd,0x59,0xe3,0xbc, +0xc2,0x13,0xc9,0xbf,0x77,0x77,0x16,0x89,0xd7,0xca,0xda,0xcd, +0x60,0x32,0x26,0x77,0x20,0xab,0x18,0x8f,0x86,0x69,0x0a,0x89, +0x31,0x98,0x0a,0xbe,0xe8,0x1f,0x1f,0xab,0x58,0x65,0x4c,0x22, +0x6b,0x26,0xa3,0xa9,0x72,0x97,0xe9,0xfa,0x05,0xe6,0x69,0x56, +0xef,0x83,0xaa,0xd5,0x1d,0xec,0x54,0x4b,0x57,0x01,0x6d,0xb9, +0xfa,0x02,0x8a,0x51,0xe4,0xf6,0x81,0xdc,0xeb,0x08,0x22,0x68, +0x92,0x7a,0x40,0xdd,0x00,0xf4,0xe9,0x35,0x8c,0x0d,0x88,0x03, +0x18,0x02,0xf8,0x52,0x5d,0x15,0x08,0x71,0x35,0xe2,0xea,0x4f, +0xa9,0x46,0x5c,0x35,0xaa,0x11,0x57,0x0f,0xad,0xc6,0xf5,0x53, +0x00,0xc6,0x6a,0x00,0x8e,0x7d,0xd5,0x50,0xf6,0xb3,0xd6,0x00, +0x2d,0xa4,0x9d,0xd1,0x26,0x2b,0x04,0x5a,0xd5,0x8a,0x64,0xb5, +0x4a,0x37,0x45,0x2a,0xf7,0x56,0x71,0x92,0xd7,0x93,0x3b,0xd4, +0x55,0xce,0x22,0x11,0x1a,0xae,0xaa,0xa1,0x6a,0xa4,0xb2,0x44, +0xea,0xe3,0xee,0xce,0xeb,0xf5,0x7a,0x1e,0x75,0xa5,0x95,0xb1, +0x6a,0xe9,0xcb,0xc2,0xba,0x0a,0x4e,0xf1,0x47,0x6d,0x5d,0xa2, +0x19,0x98,0xbd,0x25,0x29,0x4b,0x50,0xe0,0x1d,0x1f,0x29,0x1c, +0xb7,0xda,0x0e,0x8b,0x41,0x10,0x16,0x72,0x5c,0x07,0xac,0xdd, +0x55,0x10,0xf0,0xc6,0xa5,0xdc,0x40,0x0f,0x29,0x20,0xf7,0xd9, +0x43,0x22,0x6c,0xc0,0x94,0xcd,0x93,0x74,0xe5,0xaf,0xf0,0xc4, +0x7a,0x35,0xe4,0x98,0xb3,0x55,0x06,0xcd,0x82,0x51,0x92,0x81, +0x07,0xca,0xb5,0x8a,0xed,0x63,0x28,0xe8,0x3c,0x84,0xda,0xe1, +0x0e,0xb9,0x43,0xa0,0x57,0x1d,0xc3,0xdd,0x5b,0xa8,0x53,0xa6, +0x55,0xe4,0x96,0x47,0xaf,0x16,0x4b,0x69,0x18,0xc3,0x26,0xaf, +0xaf,0xfa,0xfd,0xfe,0xe7,0xf4,0xfb,0x8c,0x7e,0xbf,0xa4,0xdf, +0xaf,0xe8,0xf7,0x6b,0xfc,0x1d,0xf6,0xf1,0xf7,0x19,0xfc,0x0e, +0x3e,0xf9,0x0c,0x7f,0x3f,0xeb,0x3f,0x7f,0x7d,0x35,0x84,0x64, +0xfa,0x1d,0xd0,0xef,0x90,0x7e,0x1f,0xd3,0xef,0xc7,0xf4,0xfb, +0x84,0x7e,0x3f,0xa1,0xdf,0x4f,0xe9,0xf7,0x33,0xfa,0xfd,0x9c, +0x7e,0x9f,0xe1,0xef,0xf0,0x1b,0xfc,0x7d,0x42,0xbf,0x43,0x4a, +0x1d,0x42,0xea,0x63,0xc2,0xfc,0xcd,0xf3,0x6f,0xbe,0x99,0x06, +0xa7,0x23,0x36,0x77,0x34,0xb2,0x29,0xad,0xa0,0xa5,0x4d,0xb4, +0x37,0xf9,0x39,0x59,0xc6,0x47,0x2f,0xb3,0xe5,0xf4,0xc8,0xeb, +0x90,0x81,0x33,0x69,0xab,0x68,0xec,0xd7,0xc8,0x03,0x3f,0x91, +0x04,0x7f,0x9e,0xe7,0x12,0x5c,0x38,0xd6,0xd5,0xc2,0xbb,0x82, +0xfe,0x7c,0x06,0xcb,0xc0,0x85,0xf7,0x28,0xc2,0x15,0x55,0x76, +0x76,0x04,0xcb,0x57,0x54,0x46,0xe8,0x8f,0xaf,0x4b,0xe0,0x75, +0x72,0x74,0x6b,0x05,0xa4,0x45,0x76,0xd4,0x86,0x24,0xa9,0x36, +0x30,0xe9,0xe3,0xc9,0xe9,0x79,0x8a,0x16,0x17,0x14,0x7a,0x14, +0x45,0xf2,0xcb,0x42,0x82,0x0b,0xd1,0xbf,0xff,0xf8,0x32,0xf2, +0x3c,0x2b,0xb2,0xb8,0x5e,0xe2,0xae,0x6f,0x01,0xcb,0xc7,0x24, +0xfa,0x48,0x2e,0x14,0x23,0x58,0x28,0x0e,0xbc,0xa3,0xea,0x72, +0xb5,0x2e,0x22,0xb2,0xf5,0x0c,0x4f,0x4f,0x6f,0x6e,0x6e,0x7a, +0x37,0x8f,0x7b,0x59,0xbe,0x3c,0xc5,0x16,0x3a,0x85,0x9c,0x12, +0x24,0xac,0x60,0x7d,0x7b,0xd1,0x06,0x38,0xf8,0xfc,0xf3,0xcf, +0x4f,0x29,0xd5,0xfb,0xc8,0x2a,0x34,0xbe,0x82,0x25,0x67,0xbc, +0x78,0xb1,0xc6,0x73,0x93,0x27,0x36,0x35,0xb8,0xa4,0x8f,0x6e, +0xcb,0xf4,0x32,0x81,0x15,0x6a,0x0f,0xff,0x72,0x9c,0xb8,0x2a, +0xe7,0x6e,0x4c,0x0f,0x62,0x6c,0x16,0xe5,0xb0,0x88,0x80,0xa5, +0xca,0x2d,0x2e,0xb5,0x79,0x1b,0x04,0x96,0x0c,0xf0,0xfd,0x55, +0xbc,0x09,0xbd,0xd9,0x55,0x59,0x7a,0xbc,0xba,0xf7,0x8a,0x78, +0x5d,0x74,0x8b,0x24,0x4f,0xcf,0x3c,0xb3,0xa8,0x1f,0x0c,0x6c, +0x5c,0x78,0x50,0x14,0xdd,0xf2,0x99,0x45,0xa8,0x4e,0xbd,0xd9, +0x4c,0x2a,0x1c,0xf4,0x85,0x3c,0xa4,0x09,0x1f,0x0b,0x7d,0xfc, +0x14,0x7a,0x7f,0x01,0x9e,0x78,0x62,0x99,0xa7,0x0b,0x15,0xb1, +0xf8,0x0c,0xff,0xe3,0x7c,0x0e,0x90,0x7b,0xaa,0xaa,0x62,0xf5, +0xe9,0x14,0x10,0xae,0x4f,0x92,0xe5,0x37,0xd1,0xf8,0x89,0xd8, +0x39,0x35,0x05,0xba,0xdd,0x48,0xb7,0x7a,0x66,0x83,0xdc,0x14, +0x2d,0xb7,0xb4,0x5b,0x00,0x5d,0xa4,0xce,0xd6,0x37,0x32,0x61, +0xb5,0x90,0x70,0xdf,0xf3,0xfe,0xe3,0xe3,0x27,0x36,0xcb,0x78, +0x17,0xc6,0x30,0x8d,0xcf,0x6f,0xd4,0x5e,0xa5,0xc3,0xb4,0x96, +0xfd,0x2b,0x20,0xef,0xb3,0xcf,0x3e,0xf3,0x44,0xcb,0x76,0x15, +0x70,0xc0,0xdd,0x12,0x0b,0x07,0x9f,0x08,0x67,0x17,0x0d,0xb0, +0xf7,0xc5,0x6a,0x87,0x9f,0x2b,0x97,0x51,0x7d,0x13,0xe1,0xd6, +0xdd,0x6c,0x5c,0x86,0x3c,0x0d,0x08,0x67,0xab,0x52,0x1f,0x89, +0x72,0xb4,0xac,0xfc,0x67,0xc2,0xda,0xcd,0x0e,0xbd,0x79,0x9a, +0xcf,0x11,0xc4,0xec,0x62,0x87,0x4f,0xfa,0xc2,0xda,0xb5,0x56, +0x15,0xdc,0xd9,0xa4,0x0e,0x07,0xbf,0x7b,0x2b,0xd9,0xcd,0x32, +0x47,0xb4,0xd1,0x6d,0xbe,0x9c,0x85,0x93,0xc1,0xf0,0x33,0x21, +0xff,0x4d,0xb1,0x32,0xe1,0x04,0x68,0x84,0xff,0xa6,0xe2,0x7c, +0xbe,0x0a,0x27,0xf8,0xfd,0x04,0x03,0x85,0x0c,0xf4,0x9e,0x4c, +0x9d,0x0e,0x86,0xc6,0x37,0xd1,0x2d,0x8e,0xf2,0xcb,0x2c,0xaf, +0x07,0xfd,0x70,0xe2,0xfd,0x65,0x70,0xf6,0xe9,0xa7,0xb3,0x8f, +0x3d,0xe1,0xfd,0xe5,0xec,0xec,0xd3,0xb3,0x7e,0x82,0x5f,0xc3, +0x79,0xdc,0x1f,0xce,0xf1,0x6b,0xf1,0xc9,0xf0,0xd3,0xe1,0x67, +0xf8,0xf5,0xf9,0xc7,0x9f,0x7c,0x3a,0x5b,0xe0,0xd7,0x67,0xf3, +0x27,0x9f,0x7c,0x3c,0xc3,0xaf,0xe4,0xf1,0xa7,0x9f,0xce,0x87, +0xf8,0xf5,0xe9,0x19,0xfe,0x87,0x5f,0xb3,0xf9,0x6c,0x31,0xa4, +0xb8,0xc1,0xa7,0xb3,0x64,0x7e,0xe6,0x4d,0x85,0x2a,0x70,0xd8, +0x28,0x30,0x4e,0xe6,0x9f,0x26,0x9f,0xb9,0x45,0x9f,0x9d,0xcd, +0x66,0x9f,0x7e,0xe6,0x12,0xf1,0xf9,0x67,0x8b,0xb3,0xcf,0x62, +0x97,0x9c,0xb3,0xb3,0xcf,0x3f,0xfb,0xfc,0x13,0x97,0xb0,0xf9, +0x93,0x59,0x7f,0xf1,0xc4,0x25,0x71,0xfe,0xf1,0xe7,0xf3,0xcf, +0x3f,0x76,0x89,0x3d,0xfb,0x74,0xf6,0xc9,0xa2,0x41,0xf6,0xfc, +0x53,0xfc,0xcf,0xad,0xc0,0x62,0xb6,0x98,0x7d,0xb6,0xb0,0xaa, +0x82,0xa5,0x25,0x8b,0x38,0x79,0x02,0x95,0xa2,0xc3,0x91,0x02, +0x2a,0xa4,0x24,0xc7,0x9b,0xe7,0x59,0x51,0xc0,0xdf,0x45,0x1a, +0x5f,0x66,0xb8,0x9f,0xe9,0x15,0xef,0xae,0xe2,0x1c,0x93,0x60, +0x19,0x41,0x06,0x5f,0xdd,0x45,0x76,0xb3,0xb6,0xc3,0x57,0x1b, +0x8f,0x9b,0xe8,0x4b,0x3a,0xab,0xb5,0x6f,0xd2,0xe0,0xbc,0x3c, +0x6b,0xc4,0xc2,0x82,0x8c,0x26,0x09,0x28,0x30,0xce,0xfd,0x40, +0xcc,0xe4,0xa4,0x01,0x2b,0x02,0x4a,0xdd,0x0a,0xbd,0xdf,0x17, +0xc9,0xd3,0x5f,0x1d,0xa1,0x54,0x3b,0xb3,0x23,0x48,0x58,0x5a, +0x2e,0xef,0x10,0xca,0x6a,0x10,0x75,0x78,0x9b,0xab,0xf7,0xed, +0xb3,0x7f,0xbe,0xf9,0xf9,0xd9,0xcb,0xbf,0x3f,0x17,0x94,0x52, +0xef,0x4d,0xa9,0x86,0x51,0x77,0x4f,0x9e,0x7d,0x29,0x16,0xc5, +0x3d,0x9c,0x4b,0xcc,0x1a,0x69,0x20,0xea,0x81,0xa8,0x86,0xa2, +0x1e,0xee,0x10,0x06,0x69,0x8a,0x92,0x7a,0xa0,0x8b,0x06,0x58, +0x55,0x56,0x3d,0xdc,0x41,0x0e,0x83,0x97,0x85,0x5c,0xd4,0x66, +0x3f,0xe2,0x0b,0x89,0x16,0x54,0x61,0x5d,0x00,0x2c,0xb9,0xbf, +0x90,0x45,0xa8,0x68,0x2c,0x2c,0x10,0xd5,0x53,0x59,0x9c,0x86, +0x1e,0x12,0xf4,0x53,0x59,0xb4,0x86,0x1e,0x22,0x74,0x93,0x08, +0x98,0xd2,0x63,0x7b,0xa7,0x6c,0xd1,0xac,0x5a,0x37,0x5a,0xa8, +0xaa,0xe9,0xcf,0x6a,0xd8,0xd1,0xb1,0xfa,0xd3,0xc6,0x9a,0x63, +0x4b,0x1f,0x68,0x47,0x4b,0x45,0x95,0x51,0x81,0xe6,0x5f,0x33, +0xad,0x56,0x69,0xd5,0xd0,0x52,0x76,0x65,0x4c,0xa0,0x19,0xdc, +0x48,0xaa,0x87,0xbb,0x75,0xa5,0xe3,0xfb,0x95,0xb3,0x76,0x58, +0x54,0x62,0x51,0x37,0xc9,0x83,0x1a,0x55,0x56,0x45,0x2b,0x45, +0x1a,0x7c,0xd7,0x56,0xad,0xeb,0x96,0x6a,0x97,0x0e,0x76,0xea, +0x51,0x82,0x9a,0x96,0x0e,0x2d,0xb2,0x42,0x92,0x99,0x15,0x9c, +0x86,0x7b,0x37,0x6b,0x8e,0x83,0x0f,0x15,0x37,0xaf,0xa2,0xaa, +0x5b,0x9d,0x00,0x54,0xa7,0x3e,0x81,0x78,0x31,0xaf,0xa3,0x1a, +0x22,0xe0,0xb3,0x5b,0x63,0xb4,0x00,0x16,0x4a,0x5a,0x41,0x1e, +0xe5,0x27,0xc8,0xa5,0x8c,0x05,0x09,0xac,0x65,0x6c,0x3d,0x1c, +0xd9,0x75,0x93,0xbd,0x14,0x45,0xcf,0x07,0x3c,0x27,0xd0,0xbc, +0x80,0xf4,0x04,0xaa,0x36,0xaf,0x90,0x14,0x88,0xe8,0x60,0x3c, +0x46,0xd4,0xbb,0x60,0xc3,0x26,0xd8,0xd0,0x05,0x1b,0x36,0xb1, +0x0d,0x5b,0xb1,0x0d,0x9b,0xd8,0x86,0x16,0x36,0x9b,0x9f,0x34, +0x88,0x38,0x63,0x8d,0xda,0x2b,0x82,0x9a,0xb7,0x74,0x92,0x19, +0x09,0xd2,0x0c,0x18,0xd0,0xd2,0x55,0x66,0x24,0x49,0x00,0x32, +0x6c,0xe9,0x30,0x33,0x92,0x25,0x00,0x19,0xb6,0x74,0x9b,0x59, +0xab,0x34,0x25,0xeb,0xf9,0x2a,0x2b,0x92,0xa2,0x95,0x3e,0x39, +0x0a,0x02,0x95,0x44,0x94,0x0a,0x0e,0x9f,0x52,0x49,0x32,0x58, +0x53,0x6a,0xad,0x52,0x6b,0x4a,0xad,0x87,0x76,0x21,0x74,0x7e, +0x55,0x24,0xf3,0x72,0x5f,0x31,0x8f,0x54,0x15,0xbe,0xc0,0x72, +0xee,0xee,0x64,0xb1,0x4f,0xb1,0x18,0x19,0xaa,0x31,0xad,0x56, +0x69,0x35,0xa6,0x41,0x7d,0xec,0x52,0xf0,0xa2,0x54,0x9c,0xae, +0x8b,0xd6,0xe1,0xe8,0x91,0xaf,0xc7,0xa3,0xbb,0x3b,0x3d,0xd8, +0xdc,0xdd,0xe9,0xe1,0x08,0x3e,0x15,0xd3,0x1c,0xac,0x6c,0xb4, +0xb3,0x6f,0x10,0x18,0x76,0x25,0x52,0x3b,0x8b,0x34,0xee,0xd9, +0x93,0xa7,0x96,0x79,0x6a,0xc8,0xc3,0x73,0xc9,0xd6,0xa7,0x0d, +0x4f,0x75,0x04,0x6b,0x67,0xd4,0x27,0x88,0x4a,0xaf,0xf3,0xb1, +0x04,0x39,0x5b,0xa5,0x8b,0xc8,0xc3,0xf8,0x37,0xb4,0x85,0xa6, +0x20,0xde,0xa4,0x8b,0x4e,0x87,0x3b,0x38,0xcd,0x57,0xf8,0x83, +0x26,0xcb,0xa0,0xb1,0xc5,0xb9,0xc7,0x09,0x78,0x4d,0x87,0xae, +0x4e,0x29,0x99,0xeb,0xeb,0xc1,0xc9,0x4c,0x01,0x7d,0x3d,0x2a, +0xf5,0xb7,0xd2,0xe0,0x47,0x4e,0x81,0xaa,0xa8,0x43,0x93,0x20, +0x5d,0x23,0x32,0x27,0x06,0x6c,0xf4,0x45,0x4a,0x97,0xcb,0x0e, +0x22,0x85,0x37,0xae,0x6e,0xa5,0x4e,0x6c,0x03,0x87,0xf4,0xbb, +0x55,0x82,0xab,0x4a,0x06,0x96,0xa9,0x05,0xb6,0x55,0xef,0xa8, +0x8f,0x4b,0xd7,0x79,0xbc,0xbe,0x8e,0x0b,0xb5,0x4c,0xa4,0x00, +0x1b,0x3d,0xb7,0xf1,0x95,0x4f,0x2e,0x30,0x59,0x8e,0x6e,0x57, +0x39,0xac,0x97,0xca,0x1f,0x20,0x42,0xcc,0xcf,0xaf,0xd6,0x17, +0x85,0x20,0x90,0x85,0xdc,0x79,0xc3,0x7d,0x45,0x42,0xb6,0xa1, +0xe1,0x8e,0x96,0xfe,0x81,0xbe,0x6c,0x9d,0xa3,0x49,0x36,0x68, +0x33,0x7f,0xf9,0xcb,0xff,0x1e,0x78,0x81,0x3c,0xb6,0xc8,0xd1, +0x30,0x3b,0xd0,0x3b,0xdd,0x7a,0xa7,0xe9,0xad,0x2c,0xe0,0x07, +0xc6,0x8f,0x1b,0x80,0xb8,0xc9,0x43,0x88,0x9d,0x43,0x9c,0xb4, +0xd3,0x09,0x6e,0x2d,0xc2,0x08,0x04,0x37,0xb3,0x39,0x7f,0x64, +0x25,0xe9,0x52,0xe4,0x35,0x67,0x9b,0xb2,0xe1,0x14,0xaf,0x7f, +0x03,0x6d,0x5d,0x9b,0xb4,0xc7,0xd3,0xc0,0xa1,0x23,0x9a,0xd8, +0xe8,0xe6,0xe7,0x71,0xfe,0xac,0xf4,0xfb,0x81,0xd9,0xaa,0x7c, +0x0b,0x74,0xbe,0x45,0x42,0x39,0x97,0x22,0x15,0xa3,0x9e,0xbe, +0x1d,0xbd,0x05,0x5a,0x99,0x61,0x11,0xfd,0xf9,0x66,0x95,0xc5, +0xa5,0xcf,0xa0,0x93,0xb7,0x50,0x16,0x2c,0xd8,0xe3,0xef,0xf8, +0xaa,0xe3,0x22,0xb8,0xbb,0xb3,0x8b,0x66,0x29,0x90,0x49,0x23, +0x9e,0x6b,0x2e,0x2f,0x71,0x62,0xb7,0xa1,0xf0,0xd6,0x76,0x99, +0xbd,0xcc,0x6e,0x92,0xfc,0xab,0x18,0x2f,0x87,0x0b,0x09,0xf5, +0x92,0x08,0x89,0xe6,0x97,0xf2,0x6b,0x22,0xe3,0x69,0x47,0xdf, +0x29,0x47,0x1d,0x0b,0x3d,0x75,0x72,0x06,0x7a,0xbf,0x18,0x3a, +0xc1,0x85,0xa9,0xa1,0x93,0x69,0x84,0x09,0x4f,0x2f,0x46,0x17, +0x9d,0xc8,0xcd,0xcc,0xe2,0xc1,0x55,0x98,0x34,0xe8,0x9d,0xe2, +0xd8,0x34,0xd7,0x7c,0x90,0x08,0xb9,0xa9,0x2e,0xc4,0x45,0xc7, +0xc5,0xa4,0xee,0x30,0xdb,0x18,0xed,0x8c,0xfa,0xaa,0x1f,0x03, +0x18,0x2b,0x83,0x45,0x1e,0xdf,0x3c,0xcb,0xe7,0xfe,0x12,0x27, +0x6b,0xe0,0x4a,0x96,0x2f,0x0a,0x39,0xc4,0x88,0x95,0x28,0xcd, +0xd9,0x61,0x5e,0x45,0x9c,0x8a,0xd2,0x9a,0xd7,0x2a,0x30,0x80, +0x40,0x56,0xaa,0xd0,0x10,0x97,0x42,0x39,0x2c,0x6c,0x64,0xf8, +0x31,0xe8,0xe6,0x37,0x49,0xb2,0x51,0xe1,0x8f,0xa7,0x22,0xd1, +0x78,0x9e,0x40,0x40,0xe3,0xf9,0x04,0x20,0x93,0x65,0x11,0xc5, +0xf9,0xfc,0x55,0xf6,0x53,0xb2,0xa4,0x93,0x47,0x3f,0xa9,0x00, +0x44,0xe4,0x15,0x14,0xc8,0x88,0x19,0x1d,0x16,0x49,0xca,0x05, +0x5d,0x33,0x4f,0xbf,0xc0,0x9c,0x7a,0x9b,0x13,0x05,0x9f,0xf4, +0xf5,0xe4,0x7d,0x54,0x30,0xa2,0x57,0xd9,0x97,0xc9,0xfb,0x14, +0xf4,0x5d,0x3e,0xba,0xa3,0x9b,0xa7,0x98,0x07,0xb7,0x5e,0x47, +0xcb,0xde,0x8c,0x12,0xbf,0xba,0xca,0xaf,0x93,0x57,0x99,0x84, +0x59,0x0a,0x88,0x0d,0x24,0x27,0x68,0x02,0x87,0x30,0x54,0xbd, +0xbb,0xc2,0x04,0xba,0xcf,0xb9,0x93,0x3a,0x54,0xa9,0x8f,0xdb, +0x52,0x3f,0x56,0xa9,0x4f,0x30,0xd5,0x32,0xf4,0x20,0x38,0x6c, +0x83,0x9d,0x16,0xf8,0x6f,0xc0,0xfd,0x3f,0x83,0xc1,0x6e,0xad, +0x90,0xcb,0xad,0x75,0x02,0x05,0xf6,0x9f,0x22,0xab,0x44,0x56, +0xd3,0xc1,0x00,0x24,0xe2,0x99,0x53,0x84,0x4e,0x20,0xd8,0x10, +0x40,0x9f,0x88,0x07,0xc2,0x41,0xf9,0x55,0x3c,0x3f,0x4f,0x26, +0x12,0x7e,0xaa,0x76,0xd6,0x0f,0x80,0xd0,0x38,0x05,0xc3,0x8e, +0x2c,0xf4,0xc4,0x27,0x3d,0xf8,0x87,0x17,0xa7,0x83,0xcf,0xfa, +0xa4,0x18,0xbf,0x81,0x44,0xad,0x1b,0x43,0xf7,0x06,0xb1,0x28, +0x74,0x1c,0xea,0xd0,0xb8,0xed,0x0e,0x32,0x41,0xe1,0x78,0x56, +0xf8,0x39,0xac,0x7b,0x40,0x2c,0x4c,0xb8,0xe6,0xc1,0x70,0x83, +0xd6,0xc4,0x27,0x9c,0xfb,0xc4,0xcf,0x40,0xb9,0x0e,0x3a,0x10, +0xc1,0x45,0x40,0x44,0xdd,0x85,0x66,0xdc,0xd4,0x0e,0x10,0xc6, +0x75,0x6d,0x20,0xcc,0x25,0x36,0xab,0x68,0x53,0x9d,0x6c,0xaa, +0x53,0x28,0xeb,0x04,0x8a,0xeb,0x6c,0xea,0x93,0x4d,0x0d,0xa1, +0xfa,0x04,0x0b,0xdb,0xac,0x9e,0xa2,0xca,0xb9,0x91,0x07,0x06, +0xc5,0xbb,0xbc,0x84,0x00,0x10,0x55,0x9d,0x44,0x9b,0x15,0xd0, +0x86,0x7f,0x98,0xa6,0xb8,0xdf,0x8f,0xb8,0xb0,0x53,0x68,0x87, +0xb8,0x3f,0x88,0xb8,0x24,0x0a,0x0d,0xfa,0x51,0x57,0x05,0x6b, +0x08,0x0e,0x34,0x68,0x2d,0xaa,0x7e,0x04,0x79,0x4f,0xb2,0xaa, +0x03,0x99,0x4e,0xb2,0x5a,0xd4,0x10,0x31,0xe0,0x88,0x01,0x45, +0x80,0x56,0x82,0x10,0x0c,0x50,0xe3,0xb2,0x00,0xd3,0x39,0xb9, +0xc6,0xdb,0xd8,0xa0,0xcf,0x57,0xfd,0xe0,0x44,0xfe,0xed,0xf8, +0xb0,0xb4,0xab,0x31,0xcc,0x7f,0x45,0x71,0x16,0xcf,0xcb,0x2c, +0x7f,0x53,0xbc,0x8b,0x06,0xa7,0x8b,0x6e,0x6f,0xf8,0x64,0xd4, +0x7f,0x6a,0x22,0xf1,0x0c,0xda,0x40,0xf4,0xb9,0x3a,0x32,0xc6, +0xaa,0xb7,0x81,0xc1,0xb3,0x11,0xec,0x80,0x11,0x89,0x9a,0xc9, +0x0e,0x75,0xe4,0x0f,0x46,0x51,0xcd,0xb1,0x01,0xfc,0xaa,0xdf, +0x01,0x2d,0x5e,0x25,0x69,0xa2,0x6a,0x4e,0xad,0xfb,0x1d,0x50, +0xe3,0x3b,0x3a,0x95,0xab,0x00,0x3a,0x4e,0x5f,0x36,0x7b,0x19, +0xaf,0x87,0x00,0xd5,0xad,0xe7,0xc0,0xa9,0x6e,0x35,0xc7,0xb4, +0x81,0x93,0x36,0xa0,0xb4,0x81,0x4c,0x7b,0x03,0x12,0x0a,0x6b, +0xa5,0x01,0xa8,0x96,0x7d,0xa8,0x26,0x47,0x1c,0x1f,0x0f,0xa2, +0x88,0x88,0x1e,0x73,0x44,0x27,0x1a,0x9e,0x48,0xf1,0x0c,0x39, +0xe6,0x69,0xff,0xf8,0xb8,0x2f,0x81,0x68,0xad,0x80,0x91,0x5d, +0x03,0x66,0xb4,0x15,0x39,0x00,0x14,0xd6,0x12,0x55,0x4b,0x28, +0x67,0x3b,0xf5,0xa1,0x6a,0x32,0x5f,0xa7,0xd7,0xef,0x0f,0x82, +0x20,0x30,0x5a,0x13,0x0d,0x30,0x0a,0x89,0x52,0x6a,0xb8,0xef, +0xe0,0x2a,0xaf,0xdf,0x49,0x4f,0x24,0x1a,0x05,0x04,0xd5,0x7a, +0x4c,0x29,0x7e,0xda,0x19,0x04,0xcd,0xd4,0x11,0x63,0x46,0x27, +0x1e,0x93,0x6a,0x0e,0x8c,0x05,0x70,0xdc,0x9d,0x78,0x2c,0x07, +0x04,0x16,0x3c,0xd9,0xd9,0xf4,0xf1,0xfd,0x81,0x8e,0x1c,0x35, +0xa7,0xd3,0xc6,0x90,0xe7,0xc3,0x3a,0x6f,0x8e,0xab,0x65,0xd4, +0x8b,0x07,0x6d,0xa5,0xdc,0x3f,0xd4,0x34,0x50,0xb6,0x0f,0x36, +0x07,0x81,0x1a,0xbd,0xee,0x44,0xf6,0x3a,0xd9,0xcf,0x4e,0xa8, +0x9f,0xf5,0x23,0x15,0xaa,0xac,0x5e,0x87,0x69,0xfc,0xd5,0xb7, +0xc7,0x1e,0x3d,0x48,0xf5,0xed,0x51,0xaa,0xaf,0x86,0xa9,0x81, +0x0d,0x3b,0x50,0xb0,0x03,0x1b,0x96,0x76,0x36,0xde,0x9c,0xc7, +0xab,0x33,0x92,0x6d,0x29,0x85,0x0a,0xf4,0xcd,0xf9,0xd0,0x00, +0x43,0xba,0x04,0x85,0x3c,0xd1,0x67,0xa7,0x8f,0x4f,0x34,0x90, +0xf9,0x3a,0xb5,0x70,0x4b,0x58,0x18,0x0b,0xe6,0x55,0x47,0x52, +0xdf,0x2d,0x25,0x6c,0x1f,0x07,0x85,0x79,0xdd,0x91,0xa1,0x4e, +0x29,0xc7,0xbd,0xbe,0xa8,0x1e,0x1b,0xf8,0x81,0xa8,0x1f,0x1b, +0x28,0xda,0x6a,0xa8,0x1e,0x77,0x14,0x0e,0xda,0x64,0xa8,0x1f, +0x77,0x55,0xde,0xc1,0xe8,0x21,0xad,0x10,0x4d,0x68,0x68,0x1a, +0xf0,0xd8,0x34,0x10,0x34,0x32,0x0d,0x78,0x68,0x82,0x10,0xa6, +0x0d,0x39,0x6d,0xc8,0x69,0x43,0x4e,0x1b,0x72,0xda,0x63,0x4e, +0x7b,0xcc,0x69,0x8f,0x39,0xed,0xb1,0xe5,0x31,0x81,0x8f,0x6c, +0x40,0x71,0x41,0x4d,0x9f,0x75,0x37,0x6b,0x71,0x22,0xca,0xe4, +0x72,0xf3,0x4f,0xfa,0xfd,0x85,0x7e,0xbf,0x82,0x95,0x6f,0x9e, +0xad,0xfe,0x69,0x07,0x7e,0x81,0x05,0x69,0x72,0x9d,0x66,0x57, +0x45,0x44,0xb3,0x34,0x9e,0x6a,0xd7,0xf0,0x6f,0x2e,0x61,0xcd, +0xe7,0x2f,0xf0,0x29,0x37,0x44,0xd1,0x19,0x89,0xde,0x34,0x1d, +0x29,0xd3,0x20,0xbc,0x59,0xb1,0x82,0x21,0x52,0xa8,0x08,0xb4, +0x13,0x2a,0x31,0x02,0xf5,0xad,0x65,0xba,0xc6,0xa5,0x82,0x6f, +0x86,0x8a,0xfd,0x2b,0x19,0xb2,0xc7,0x91,0x47,0xcc,0xb2,0x32, +0x66,0x31,0xc3,0x61,0x3c,0x6f,0x97,0x27,0xcf,0x5e,0x58,0x75, +0xd4,0xe2,0x06,0xd5,0xa3,0xda,0x84,0x40,0x3d,0x5a,0xf6,0x70, +0x49,0xfb,0x2a,0xf3,0xab,0xce,0x0a,0x92,0x5c,0x1d,0x02,0x55, +0x1a,0xfb,0xd2,0xcc,0x4b,0xc0,0xe5,0xa0,0xfa,0xd5,0x98,0xce, +0x9b,0x54,0x7d,0x50,0xee,0xbf,0x35,0xe8,0xf8,0xa0,0xcc,0xd7, +0x5e,0x58,0xff,0xfa,0xa2,0x7f,0x86,0xdc,0xbf,0x3a,0xf3,0xe5, +0x7d,0x6d,0x71,0x99,0x5d,0x3f,0x14,0xd7,0xb7,0x87,0xdb,0xe2, +0x03,0x30,0xcd,0xbd,0x90,0xba,0x42,0x04,0x7d,0x5d,0xa2,0x00, +0x55,0x98,0xfa,0x45,0x54,0xeb,0x28,0x50,0x88,0xb5,0xd0,0x1b, +0xc0,0xc7,0x53,0x23,0xff,0x06,0xf6,0xe3,0xa9,0x68,0xac,0x21, +0x7c,0x93,0x65,0x30,0x25,0x9a,0x0c,0xb5,0x9d,0x52,0x63,0x86, +0x14,0x85,0xae,0x23,0x3b,0x28,0x44,0x11,0x29,0xcd,0x4a,0x74, +0x1c,0x36,0x1a,0x6c,0x0e,0x94,0xc2,0xab,0xb1,0x3a,0xa9,0xd6, +0x00,0x00,0x83,0x63,0xc4,0xc1,0x9a,0xfe,0xfe,0x62,0x33,0xe8, +0x2b,0x9b,0xd5,0x4f,0x6c,0x56,0xdb,0x4c,0x69,0x63,0xc9,0x01, +0x86,0x38,0xec,0x78,0x00,0x33,0xda,0x5a,0xd2,0x62,0xc1,0xaf, +0x62,0x40,0x53,0x12,0x8a,0x5d,0x49,0x78,0xbc,0x2b,0x09,0x1f, +0x5b,0x95,0x1e,0x9e,0x54,0xdd,0x9d,0x32,0x20,0xb6,0x56,0xb1, +0xbf,0xec,0x56,0x7c,0x4f,0xf5,0x0e,0xc9,0xc7,0x21,0x49,0xb8, +0xa7,0x8a,0x0f,0x92,0x13,0x47,0x12,0x5a,0xc4,0x7c,0xd0,0x2a, +0xe6,0xc3,0xe9,0x21,0xa9,0xf9,0x49,0x31,0x73,0x87,0x95,0x7f, +0x28,0x23,0xf7,0xcb,0xd5,0x0e,0x13,0x1b,0xc4,0xff,0x9e,0x92, +0xd5,0xce,0xd0,0x36,0x76,0x1a,0xf4,0x36,0xf3,0xde,0x7d,0xb0, +0x24,0x3e,0xa0,0xb1,0x96,0xbd,0x77,0x57,0xf1,0x22,0x8f,0xcb, +0x74,0x7e,0x98,0x87,0x1f,0xc4,0xa9,0x07,0xf3,0xc1,0xae,0xdf, +0xbf,0x3e,0x44,0x38,0xda,0xe8,0xfd,0xf5,0xad,0xfb,0xc0,0x56, +0xf8,0xed,0x75,0x2b,0x77,0xdb,0x6e,0xb0,0xdb,0x76,0x50,0x12, +0x1b,0x7b,0x47,0x4a,0xc3,0xc2,0x2d,0xc9,0xcb,0x18,0x95,0x9a, +0xd3,0xc9,0xbf,0xbe,0x7b,0x55,0x4e,0x4f,0x83,0xb1,0x6f,0xda, +0xd7,0x6a,0xd4,0x20,0xf4,0xc8,0x71,0x88,0x95,0xd3,0x82,0xc4, +0xae,0xe4,0xe8,0x73,0x4e,0x77,0xb2,0x95,0x3b,0xc0,0xf3,0xae, +0x81,0x07,0x74,0xb2,0x0f,0xe8,0x93,0x81,0xa3,0x38,0x46,0xf3, +0x36,0x0d,0x32,0xb2,0x3a,0xf0,0xef,0x25,0x7f,0x1f,0x22,0xf5, +0xbf,0xb9,0x3d,0x5f,0x35,0x65,0x75,0xd0,0x94,0xd5,0xe1,0x87, +0x0e,0x64,0xff,0x69,0xfa,0x61,0xec,0x85,0xd6,0x7e,0x2f,0x4f, +0xb1,0x62,0xd2,0x3a,0xfa,0x09,0x7b,0x82,0xb7,0x06,0x20,0xa3, +0x17,0x18,0xad,0xa0,0x53,0x59,0xdd,0xf3,0xd3,0x69,0x07,0xb0, +0x4e,0xed,0x3d,0x64,0x61,0xa9,0x81,0x9f,0x38,0x6a,0xe0,0xa7, +0xce,0x30,0xf8,0xec,0x0f,0x22,0xcf,0x21,0xae,0x49,0x9a,0x43, +0xd9,0x1e,0xc2,0xde,0x4b,0x03,0xd4,0x7f,0xf3,0xc2,0x65,0x8f, +0xce,0x3e,0x79,0xf5,0xb2,0xd5,0xcb,0x25,0x99,0x4f,0x6d,0x19, +0xc8,0xb6,0xd0,0xc7,0xee,0x3e,0x96,0xd5,0xd8,0xf1,0x2d,0xe8, +0x04,0x69,0x67,0xa7,0xf7,0x8f,0x5b,0xaf,0xfd,0x39,0x4b,0xac, +0x5f,0xbb,0xa0,0xfa,0x90,0xe5,0xd3,0x07,0x2c,0x96,0x3e,0x64, +0x69,0xf4,0x01,0x0b,0xa1,0x5f,0xbf,0xec,0xf9,0xb5,0x8b,0x9c, +0x3f,0x7b,0x49,0xf3,0x5f,0x79,0xf5,0xf1,0x5f,0x6d,0xd1,0xf0, +0xdf,0x51,0xbf,0xff,0xef,0xa6,0x8e,0xff,0xdf,0x51,0xaa,0xff, +0x47,0x2d,0xfe,0xcf,0xa6,0x16,0xff,0xbf,0xa5,0xdc,0xfe,0x11, +0x2a,0xaa,0x73,0x1e,0xfe,0x7b,0x68,0xa7,0x8e,0x6e,0xaa,0xb8, +0xf8,0x21,0x5a,0xe9,0xef,0x4c,0x91,0x45,0x8f,0x45,0xcd,0x07, +0x2a,0xa2,0xf7,0xe8,0x9e,0xf6,0xe9,0x77,0x12,0x93,0x2b,0x27, +0xed,0x70,0x8b,0x02,0x68,0x48,0x80,0x49,0xd1,0xe2,0x31,0x5e, +0x5b,0xe9,0x11,0x58,0xd0,0xab,0xda,0x2f,0x53,0x57,0xdb,0xa0, +0x57,0x0f,0xda,0xd3,0x6a,0x4c,0xeb,0xef,0x49,0xeb,0x2c,0xa4, +0x69,0xdb,0x56,0xdf,0x67,0xcb,0x6c,0x87,0x84,0xc7,0xc7,0x58, +0xb0,0xe3,0xa2,0xd0,0x49,0x0f,0x04,0xdf,0x66,0xcd,0x7a,0x65, +0xb2,0xc6,0xdb,0x34,0x32,0x83,0x0c,0xf9,0x3a,0x3e,0x10,0x56, +0x45,0x4d,0xed,0x71,0xbf,0x7a,0x4f,0xed,0x31,0x49,0xd5,0x9e, +0xc0,0x0e,0xd7,0x7e,0x6f,0xe5,0xf7,0xd4,0x0b,0x51,0x7e,0x50, +0xbd,0x28,0x43,0x4b,0xbd,0xac,0x2a,0x90,0x4d,0x9d,0x3c,0x50, +0x12,0xda,0x68,0x29,0xba,0xbd,0x0c,0x87,0x62,0x05,0xff,0xf0, +0x02,0xc4,0x35,0xfc,0x9b,0x87,0x9f,0x88,0x22,0xfc,0x58,0xbc, +0x83,0x7f,0x25,0xc4,0xc7,0xe1,0xa7,0x5b,0x91,0x27,0xd1,0xe4, +0xd4,0x9f,0x7c,0xfb,0xf2,0x6f,0x3f,0x7f,0xf5,0xd3,0xbf,0xbe, +0x7a,0xf6,0x6f,0x97,0xab,0xf3,0xeb,0x79,0xf1,0xae,0x8c,0xdf, +0x4f,0x83,0xd3,0xa5,0x38,0xfd,0xcb,0x5f,0xfe,0x72,0x2a,0x4e, +0xfd,0xd7,0x8b,0xa0,0x8b,0xc1,0xd7,0xc5,0x9d,0xb8,0xc3,0xb8, +0x69,0x8b,0xb9,0x03,0x9a,0xca,0xb5,0x1d,0x77,0x61,0x3c,0x1e, +0x63,0x4a,0x47,0x10,0xc6,0xb6,0x0f,0x23,0x95,0x9f,0x12,0x32, +0xd7,0x0a,0xe9,0x17,0xa8,0xc2,0x63,0xab,0x90,0xff,0xc8,0xde, +0x10,0xca,0x15,0x1a,0xb6,0x68,0x88,0x3f,0xc4,0x83,0x90,0x7c, +0xdd,0x48,0xc3,0x47,0x69,0xa3,0x87,0xae,0xc4,0x8a,0x56,0x23, +0x3d,0x20,0x99,0x16,0x69,0x4b,0x91,0x71,0xcb,0x57,0xc0,0xeb, +0xea,0xee,0x0e,0x57,0x48,0x59,0xaf,0xc6,0x8f,0x34,0x8f,0xb0, +0x4d,0xd6,0x49,0xfe,0x63,0xbc,0x48,0xaf,0x0a,0x8c,0xcb,0x30, +0x2e,0xbb,0x2a,0xed,0xb8,0x22,0x8e,0xa0,0x3d,0xc8,0x43,0xf9, +0x33,0xb4,0x53,0x86,0xb8,0xa0,0xab,0x4c,0x36,0x86,0x22,0xa1, +0x64,0xa0,0xbf,0x25,0x71,0xe4,0x9c,0x76,0x09,0xbc,0x33,0x9f, +0xe6,0x63,0x73,0x60,0x01,0xda,0x65,0x88,0x7d,0x8f,0x87,0x96, +0x34,0x87,0xb2,0x00,0x9f,0xa0,0x63,0x32,0x15,0x9b,0xe5,0x18, +0x05,0x09,0x03,0x8c,0xb5,0x97,0x9f,0x96,0x45,0x62,0x79,0x6e, +0x6a,0xeb,0x5c,0x03,0xcf,0xc8,0x8e,0x71,0xec,0x67,0x13,0xf2, +0x1e,0xc8,0x5c,0x5f,0x78,0xd3,0xbb,0xbb,0x9d,0x28,0xb6,0xbe, +0x23,0x3c,0x9c,0x8b,0xce,0xe2,0xb1,0x5d,0x14,0xee,0x46,0x06, +0x01,0x1c,0x45,0x57,0x75,0x41,0x10,0xf2,0x31,0x9f,0x75,0x10, +0x4e,0x57,0x73,0x0c,0x4d,0xee,0xa9,0x1f,0x99,0x34,0x89,0x52, +0x1f,0x88,0xbf,0x4f,0x22,0x45,0x2d,0xf9,0xb9,0xe6,0x3f,0x74, +0x15,0x89,0x9a,0x8d,0xdb,0x0c,0x6d,0xf2,0x58,0x5d,0xc8,0xd8, +0xd1,0xdb,0xdd,0x9d,0xba,0xf1,0x11,0xe9,0x38,0x75,0x22,0xee, +0x18,0x64,0x00,0xb2,0x53,0x65,0x9d,0x60,0x71,0x36,0x17,0x7d, +0xa1,0xcd,0x16,0x98,0xe9,0x16,0x7b,0xe5,0xd1,0xa5,0xf1,0xcc, +0xc7,0xf8,0x79,0x15,0xcc,0x17,0x4c,0xc2,0x9d,0x52,0x9e,0x04, +0x20,0x11,0x65,0xf4,0xf8,0x24,0xb7,0x8e,0xa5,0xba,0xa5,0xa8, +0xbb,0x79,0x60,0x1d,0x9f,0x75,0xf3,0xd6,0x98,0xd2,0x8e,0xe9, +0xb4,0xc7,0x38,0xb9,0x3a,0x3b,0x98,0x31,0xa6,0x93,0x37,0x73, +0xb5,0xc4,0x94,0xcd,0xd2,0x5b,0x62,0x5c,0x0a,0x19,0xb3,0x3d, +0x1f,0xa9,0xeb,0x35,0xa1,0xb2,0x3d,0x32,0x8c,0xf0,0x87,0x27, +0x65,0xbc,0x7e,0xdc,0x47,0x19,0xaa,0xa2,0xbc,0xe6,0x90,0xb0, +0x45,0xbf,0x9b,0xd7,0x2e,0x55,0x64,0x54,0x66,0x22,0xb0,0xb8, +0xda,0xa5,0x69,0x67,0x39,0xc6,0xb7,0x7a,0xc2,0xb2,0x51,0x3c, +0x14,0x1a,0x95,0xd0,0x12,0x78,0x93,0x11,0xbd,0x0a,0x30,0xbf, +0x41,0xe2,0x4a,0x57,0x4b,0x75,0xee,0x02,0x85,0xca,0xa2,0xca, +0xd4,0x02,0xbf,0x1e,0x93,0x69,0x55,0x5e,0x9d,0x50,0x80,0x90, +0x9a,0x3a,0x74,0x5a,0xea,0xd0,0x6d,0xa1,0x1a,0xe3,0x5a,0x4b, +0xbe,0xda,0xfc,0xaa,0x72,0xdb,0x78,0xd7,0xc6,0x2d,0x8c,0xdb, +0xee,0x19,0x35,0x70,0x64,0x95,0x3d,0xb4,0x7d,0x86,0xdc,0x44, +0x1f,0x36,0x6a,0xb8,0xb6,0xd3,0x3d,0x6b,0x42,0xa6,0xd7,0x47, +0xac,0x91,0x64,0xd3,0x98,0xa2,0xff,0x48,0x32,0xac,0xf9,0xf3, +0x3e,0x32,0xf8,0x2a,0xe1,0x5e,0x42,0x56,0x37,0x38,0xca,0x94, +0xfa,0xba,0xa1,0x58,0xcd,0x75,0xc4,0x57,0xf1,0x66,0xc4,0xbc, +0xa7,0x24,0x39,0x9e,0xad,0x6e,0xc6,0xab,0x9b,0xb0,0x79,0xaf, +0xd6,0x80,0xc9,0xe6,0x82,0xcc,0x2a,0xc3,0x7c,0xbc,0x9a,0xb7, +0x67,0x00,0x20,0xb1,0xc3,0x2b,0xcb,0xfe,0xe3,0x6a,0x65,0xe8, +0x57,0x73,0xde,0x40,0x4f,0x7a,0x03,0x35,0xeb,0x55,0x43,0x3d, +0xd6,0x56,0xc3,0x31,0xfe,0x84,0x15,0x59,0xb4,0xa8,0xd8,0x1a, +0x63,0xeb,0x61,0x58,0x0f,0xfe,0x13,0x54,0xd8,0x9d,0x3f,0x4d, +0x0f,0xc0,0x1b,0x69,0xb6,0xb0,0xd3,0xdd,0x34,0xd7,0xac,0x19, +0xf3,0x7c,0xbf,0x66,0x43,0x7d,0x01,0x2c,0x71,0x1a,0x75,0x13, +0xcf,0xa1,0x32,0x50,0x41,0xf2,0x60,0x06,0x95,0xb2,0x7c,0x95, +0xea,0x2a,0x92,0xd5,0xbe,0x66,0x35,0xf9,0x13,0x8b,0xd4,0x04, +0x24,0x5d,0x84,0x8d,0x07,0xa1,0xfe,0x16,0x7d,0x60,0x1f,0x7c, +0xf2,0xa3,0x43,0xab,0xbb,0x3b,0xe9,0xfa,0x54,0x86,0xe1,0xcf, +0xb2,0xb7,0x5c,0x65,0xb3,0x78,0xf5,0x6c,0xb5,0x39,0x8f,0x09, +0xf6,0x44,0xcf,0x68,0x08,0xf2,0xbd,0x85,0xd4,0x0a,0x63,0x45, +0x31,0xf8,0x53,0x59,0xaf,0xd0,0xba,0x77,0x95,0xa1,0x9d,0x4f, +0x46,0xa4,0xab,0x34,0x74,0x50,0xa4,0x5d,0x93,0x42,0xbb,0x31, +0x77,0xfd,0x66,0x0b,0x06,0xf7,0x34,0xcf,0xea,0x06,0xcd,0xfb, +0x0e,0x12,0xca,0xd8,0x6c,0x52,0x9d,0x18,0x24,0x88,0x23,0x9a, +0xe4,0x4a,0x76,0x08,0x5b,0x6a,0xa0,0x01,0x9a,0x12,0xe1,0xbb, +0x42,0x16,0xdc,0x23,0x1f,0xd7,0xcb,0x97,0xf0,0xfd,0x75,0x5c, +0xd0,0x14,0x4d,0xb9,0x30,0xc0,0xde,0x95,0x02,0x07,0x80,0xaf, +0x1b,0x3b,0x60,0x1c,0x85,0x4a,0x9b,0x26,0x1b,0x38,0x19,0xb4, +0x88,0xd2,0x33,0xe0,0xb1,0x14,0xa5,0x62,0x9e,0xac,0x13,0xbd, +0x73,0xcf,0x8e,0x85,0xf0,0xfd,0x3b,0x94,0x13,0xed,0x59,0x88, +0x9c,0x6f,0x20,0x60,0x8f,0xc4,0x67,0x77,0xc7,0x1d,0xa3,0x6d, +0x08,0xdc,0x6d,0xf7,0x1f,0x31,0xd6,0xbb,0x3b,0xb9,0x56,0x36, +0xd7,0xa0,0x7c,0xcb,0x8f,0x70,0x00,0x32,0xd5,0x22,0xe2,0xe4, +0xb0,0x17,0x7f,0x1a,0xe4,0xff,0x88,0x73,0x60,0x93,0x6c,0x7c, +0x10,0x69,0x87,0x3c,0x7d,0x9f,0x21,0xb3,0x9d,0xf7,0x0a,0xab, +0xbb,0xa0,0xde,0x74,0x23,0xce,0xb9,0x4f,0xd8,0xe4,0xeb,0xf3, +0x85,0x7d,0xd5,0x3d,0xf3,0xd5,0x50,0x0a,0x15,0x3d,0x50,0xcf, +0x4c,0x57,0x12,0x69,0xdc,0x55,0xdb,0x51,0xa2,0xe9,0x86,0x15, +0x06,0xce,0x23,0xe5,0x18,0x99,0xb4,0xf7,0x7b,0x7b,0x68,0x44, +0xc2,0x2c,0x95,0xc4,0x91,0x6f,0x75,0xfe,0xe0,0x8f,0xed,0x9f, +0xd4,0x06,0x92,0x79,0xd0,0x55,0xfd,0xc6,0x50,0x13,0xfc,0x4f, +0xaf,0xfd,0x6d,0xbd,0xd6,0xe5,0xaf,0x6d,0xfe,0x4f,0x3d,0x00, +0x26,0xc7,0x0f,0xee,0x01,0x2d,0xc2,0x6f,0x2e,0x42,0xff,0x79, +0x1d,0xe0,0x37,0xce,0xe1,0x1f,0xdc,0x27,0xfe,0x47,0x32,0x7f, +0x2f,0xc9,0xfc,0x00,0xd5,0xc5,0x99,0x7d,0x1a,0xc2,0xfb,0xe2, +0x32,0x5e,0x7e,0xb8,0xf4,0x72,0xc5,0x92,0x9c,0xae,0x43,0xff, +0x39,0xd2,0x9a,0xf5,0x52,0x24,0xf5,0xf8,0x58,0x7e,0xf4,0xae, +0xf2,0x15,0xad,0xcc,0xe1,0x2f,0xaa,0xec,0x1c,0x1b,0x29,0xca, +0x7a,0xf8,0x74,0x22,0x57,0x8e,0x40,0x40,0xc3,0xb2,0xf2,0x71, +0x14,0xdf,0x8f,0x90,0x73,0x0e,0xca,0xcd,0xc8,0x1a,0xfe,0x77, +0xca,0x6b,0x9d,0x16,0x76,0xa0,0xcc,0x7c,0x51,0xe1,0xce,0x4d, +0x45,0x5b,0x36,0xbe,0x72,0x2f,0x45,0x04,0x93,0xbb,0xd9,0xf1, +0xcd,0xe9,0x50,0xf9,0x5e,0xb1,0x63,0x43,0xbc,0x94,0x81,0x39, +0x6b,0x99,0x53,0xfa,0x34,0x21,0x18,0xe5,0xa0,0x76,0x7c,0x8e, +0x99,0xad,0xe7,0x7c,0xac,0x14,0x44,0xa0,0x88,0x41,0x1e,0x24, +0x8b,0x9d,0x2e,0x22,0x45,0x97,0x3a,0xaf,0xee,0xaa,0xc1,0x18, +0x3f,0xd0,0x87,0x4b,0xcf,0xc8,0x85,0x44,0x24,0xf4,0xd8,0xd7, +0x90,0x1e,0x7c,0x8e,0xf3,0x77,0x98,0xfc,0x6f,0xfe,0xb4,0x01, +0x0f,0xe6,0xcb,0x8c,0x9e,0x5c,0xec,0x71,0x59,0xe4,0x56,0x88, +0x9d,0xff,0x65,0xd8,0x4f,0xd0,0xd7,0x2f,0x79,0x98,0x51,0x2d, +0xa2,0x1e,0x57,0x92,0x69,0xca,0xd7,0x8c,0xc5,0x72,0x80,0x88, +0x91,0xad,0xb3,0xa4,0x4c,0xe7,0xde,0x07,0x0f,0x89,0x50,0x0e, +0xae,0xd5,0xc7,0x40,0x5a,0x11,0x5f,0x27,0xd4,0x9b,0x8d,0x6d, +0x87,0x1c,0x9d,0xa5,0x38,0xe0,0x96,0x03,0x5d,0xeb,0xf2,0x65, +0xae,0x13,0xe7,0x72,0x17,0xaa,0x33,0x0b,0xad,0xcf,0x2c,0x28, +0x4b,0xe8,0x1b,0x39,0xc4,0x87,0x21,0x28,0xdd,0x16,0x31,0x8a, +0xa4,0x2f,0x50,0x1c,0xfe,0x3c,0x95,0x85,0x24,0x27,0x23,0x9e, +0xd2,0x7d,0xbf,0x5f,0xa5,0xb5,0x0c,0xfe,0xb0,0x29,0xe0,0xc0, +0x04,0xc0,0x09,0x3b,0xf4,0x63,0xaf,0xa3,0x36,0x39,0x3e,0xc6, +0x81,0xbf,0x28,0xb3,0x1c,0x1a,0xb3,0xd1,0x5f,0x94,0x9e,0x8f, +0x9e,0x5a,0x8d,0xd3,0x59,0xb5,0xc3,0xdc,0xec,0x48,0xcd,0xd5, +0x01,0xe6,0x6a,0xae,0x10,0x1a,0x05,0x28,0x85,0xfd,0x61,0x05, +0xec,0x76,0x53,0xe0,0xe5,0x81,0xbe,0x64,0x2f,0x28,0xfa,0xd3, +0x03,0xeb,0x05,0x9b,0x52,0x0d,0x6f,0x07,0x9b,0x74,0xff,0x15, +0x5f,0x02,0x79,0xe0,0x48,0x42,0x8b,0x22,0x7e,0x3a,0x24,0xae, +0x12,0xf4,0x53,0x88,0x0e,0xad,0x0a,0xb1,0xac,0xc4,0xb2,0x16, +0xe8,0x86,0xf5,0xad,0xb8,0x6c,0x19,0x51,0x9c,0x49,0x8b,0x96, +0x52,0xfb,0xd6,0x2f,0xd2,0xf9,0xa5,0x33,0xf4,0x68,0x5b,0x26, +0xcc,0x48,0xa5,0x9b,0xe1,0x07,0xc9,0x90,0x2f,0xbc,0xa0,0x6f, +0xb0,0x17,0x18,0x8f,0xee,0x34,0x15,0x69,0x32,0x8d,0x43,0x26, +0x75,0x59,0xc9,0x04,0xea,0xb3,0xcb,0x5a,0x86,0x48,0x0b,0x6b, +0x1d,0x0d,0xa8,0x86,0xea,0xa8,0xf3,0xf8,0x78,0xc7,0x12,0xac, +0x0b,0x00,0x5d,0x84,0xc0,0x3b,0xf1,0x97,0x11,0x92,0xa5,0xc8, +0xbf,0x7c,0xfa,0x16,0x9f,0x71,0x0d,0x3c,0x7c,0xea,0x16,0xe7, +0x0c,0x4c,0x9c,0xbc,0x9d,0xd2,0xeb,0x2c,0xab,0xb8,0x4e,0x72, +0xf4,0xda,0x29,0xe7,0x4e,0xe4,0x0b,0xf0,0x44,0x82,0x28,0xae, +0x8c,0xfe,0x17,0xbf,0x28,0x8b,0xa8,0x99,0x52,0x87,0x41,0xaa, +0x84,0x26,0x16,0x0b,0xd4,0x46,0x66,0x70,0xed,0x27,0xf3,0xd1, +0xaf,0x22,0xd3,0x60,0x96,0xec,0xbf,0x8f,0x42,0x09,0x66,0xe3, +0xd8,0xcb,0x62,0xd9,0x04,0x4e,0x17,0x37,0xa2,0x6c,0xc6,0x3a, +0x7e,0xe1,0x4d,0x7b,0xaa,0xc5,0x50,0x2f,0x5d,0x8c,0xb5,0xdb, +0x8c,0x25,0x43,0x08,0x3d,0x3d,0x85,0x14,0xde,0xee,0x3a,0xd8, +0x58,0x8a,0x8d,0xed,0x62,0xf8,0x26,0x9a,0xa9,0x27,0x81,0x40, +0x13,0x99,0xe9,0x57,0x7f,0x84,0xf4,0xd4,0xd2,0x01,0x71,0x1a, +0x9c,0xdc,0x08,0xe9,0x9b,0x05,0x82,0xf5,0xe0,0xe4,0x5c,0xb0, +0x37,0x16,0x4a,0x1d,0x62,0xea,0x50,0xa7,0x0e,0x21,0x15,0x0b, +0x8b,0x40,0x6d,0xcd,0x13,0xa8,0xe3,0x4b,0xf2,0xcc,0xa1,0x1f, +0x60,0xb0,0x1d,0x34,0xf1,0x63,0xbc,0x1b,0x76,0x92,0x61,0xfb, +0x39,0x86,0x70,0xb3,0xa3,0x20,0x4e,0x3c,0x8e,0x27,0x7f,0x69, +0x3f,0x01,0x80,0x8f,0x50,0xd0,0x5d,0xd4,0x8b,0x4b,0x2a,0xc8, +0x4e,0x38,0xd4,0x49,0x27,0x66,0xb3,0x0e,0x9f,0xd2,0xf9,0x85, +0x3b,0x36,0xa0,0x2a,0xc2,0x8d,0x80,0xc3,0x03,0x39,0x8a,0xde, +0x19,0x22,0xee,0xee,0x38,0x4e,0xb5,0xe2,0x23,0x3b,0xa8,0xfd, +0xb2,0xc8,0xb6,0x94,0x13,0xf1,0xa3,0xc1,0x48,0x3e,0xd0,0xcc, +0xe5,0xf0,0x00,0x73,0x9e,0x96,0x82,0xbc,0x19,0xc1,0xb8,0xf2, +0xb6,0x65,0x4c,0x39,0x8f,0xd7,0xa0,0x9d,0x59,0x43,0x4a,0xea, +0x8e,0x1a,0xdd,0x6e,0xfa,0x34,0xea,0x8f,0x5a,0x47,0x8d,0x85, +0xd3,0xfd,0x17,0xf7,0x77,0x7f,0x76,0xab,0x04,0x9d,0xbb,0xa5, +0xf7,0xe1,0x23,0xcf,0x58,0x10,0x3f,0x45,0x4d,0x24,0x46,0x8d, +0x9e,0xa7,0xe2,0x7b,0xd6,0xf3,0x0d,0xd0,0xc3,0x1e,0xe1,0xdd, +0x6a,0xa8,0x67,0x24,0xeb,0xd2,0x43,0x9e,0x6b,0x24,0x92,0xdf, +0xdd,0x05,0xf2,0xaa,0xbb,0xc0,0x89,0x4e,0x35,0x94,0xe9,0x02, +0xc8,0xa6,0x91,0xdd,0x25,0xf4,0x43,0xca,0x8d,0xe2,0xc6,0x88, +0x1b,0xa7,0x33,0xc8,0xf0,0x0a,0xa0,0x0b,0x3a,0x42,0x11,0xbb, +0x6d,0x1b,0x3e,0x1a,0xb8,0x42,0x80,0x99,0xf0,0xfd,0xfa,0x16, +0x58,0xf2,0xb4,0xdb,0x32,0x4f,0x38,0xed,0x9a,0x89,0x99,0x48, +0xa9,0x85,0x06,0x50,0xe7,0x65,0xef,0x0d,0x1a,0x4d,0x67,0x50, +0xf3,0xea,0x44,0x87,0x44,0x6d,0xbe,0x71,0x0e,0x68,0xd9,0x61, +0x53,0xed,0x49,0xea,0x67,0x63,0x7f,0x6d,0x16,0xa9,0xce,0x8c, +0x5b,0x6d,0x30,0x7f,0xee,0x48,0x1a,0x8c,0x27,0xc7,0xc7,0xf4, +0x0a,0x3f,0x0e,0x11,0xa6,0x06,0x8a,0xa5,0xd9,0x48,0xd1,0xdc, +0xa8,0x3c,0x1e,0x76,0xfc,0xca,0x8a,0xcf,0x76,0xa5,0x56,0x75, +0xb5,0x59,0xd4,0x98,0xca,0xd1,0xcf,0xdc,0xa3,0x1d,0xb2,0xc7, +0x8f,0x06,0xe1,0x83,0xb9,0xa6,0x1b,0x36,0x66,0x9a,0xb9,0x9b, +0xa0,0x86,0x34,0x56,0x85,0x41,0xdb,0x06,0x6e,0xfd,0x70,0xc4, +0xf9,0xaf,0x56,0xbf,0x15,0xd3,0xfc,0xa0,0xfa,0x91,0xe4,0xee, +0x57,0xc7,0xac,0x1a,0x2b,0xb7,0x45,0x87,0x25,0xde,0x9a,0x71, +0x50,0x01,0xfd,0x5b,0xda,0x14,0x28,0xe2,0x5c,0xd6,0x53,0x4e, +0x53,0x0d,0xc7,0x38,0x9e,0x3d,0xa2,0xc9,0xc8,0x3e,0xb3,0xd1, +0xac,0x89,0xd4,0x9c,0x47,0x1a,0xae,0x28,0x2f,0x37,0x7c,0xfd, +0x56,0x3c,0x82,0x75,0x43,0x1c,0x75,0x5b,0x96,0x1e,0xc2,0xf5, +0xc1,0xd6,0xf4,0xbf,0x16,0xd8,0x07,0xed,0x62,0x83,0x6e,0x42, +0x8a,0x13,0x18,0x53,0xd0,0x4b,0xd9,0xb2,0xee,0xf8,0x8e,0x5b, +0x36,0x72,0x19,0x41,0x29,0x74,0x61,0x9a,0x00,0x1c,0x37,0x6d, +0x7a,0xa6,0xb0,0x1a,0x72,0x53,0x41,0x2e,0xb6,0x60,0xd1,0x47, +0x75,0x51,0xe3,0xa8,0xce,0x36,0x09,0x69,0x4b,0x97,0x56,0x22, +0x74,0x20,0x1a,0xf9,0xc6,0x04,0xc3,0x9c,0x9e,0x3e,0x06,0x25, +0x9f,0x4e,0x9a,0xb9,0x6e,0xf0,0xe9,0x3f,0xee,0xbb,0x4b,0x30, +0xcd,0x2d,0xf7,0xea,0xb2,0x16,0x9b,0xe8,0x96,0x5e,0x93,0x91, +0xad,0x26,0x70,0xf4,0x0b,0x77,0xfc,0x72,0x3d,0xea,0x6f,0x05, +0xad,0xbb,0xdb,0x93,0xf0,0x30,0x2e,0xb4,0x84,0x28,0x13,0xb6, +0x9b,0x43,0x74,0x41,0xfc,0x43,0x06,0x63,0xee,0x8b,0x35,0x9f, +0xd8,0x8d,0xfd,0xc6,0xe9,0x9d,0xd8,0x01,0x21,0x5b,0x11,0x1a, +0x73,0xd9,0x18,0xc6,0x96,0xd0,0xfb,0x90,0x3b,0x47,0x9b,0xc5, +0x61,0xe4,0x71,0x3e,0xdf,0x4f,0xb8,0x6d,0x54,0x63,0x63,0xa1, +0x48,0x04,0xdc,0xb2,0xcd,0xce,0x5e,0xe2,0xac,0x23,0xe7,0x62, +0x1f,0x06,0x3a,0xd6,0xdd,0x4b,0x82,0x63,0xea,0xb2,0x07,0x03, +0x1b,0x9f,0xec,0xc7,0xd1,0x30,0x4e,0x69,0xc7,0xb2,0x55,0xd6, +0x4a,0xa8,0x81,0xaa,0x07,0xd3,0xf5,0x3a,0x88,0xeb,0xa9,0x96, +0x73,0xaa,0x5a,0x6c,0xac,0xa5,0xa3,0xd5,0x69,0x2d,0xf9,0x8b, +0x09,0xd5,0xe2,0x52,0x32,0x91,0x5f,0x7c,0x0e,0xed,0x25,0x27, +0x47,0x4b,0xea,0x55,0x82,0x21,0x36,0x60,0x59,0x54,0x2b,0x22, +0x16,0x32,0xb5,0xbb,0xcd,0x4f,0x20,0xa9,0x0d,0x1f,0x29,0x9b, +0x7a,0x97,0x48,0x48,0x92,0x14,0x69,0x42,0x62,0x57,0xa5,0x00, +0xd7,0x61,0x38,0x53,0xb5,0xd4,0x1a,0x1d,0xd7,0x52,0x4d,0x6f, +0x5c,0x39,0x35,0x19,0x50,0x9d,0x68,0xe4,0xb4,0xe6,0x93,0xb9, +0xac,0x95,0x1b,0xbf,0xb1,0xeb,0xe5,0x26,0x71,0xa4,0xac,0x99, +0x9b,0x84,0x51,0x01,0x57,0xb2,0x91,0x70,0x85,0xbe,0x22,0xa9, +0xbe,0x6e,0x02,0x46,0x05,0xb2,0xea,0x6e,0x0a,0xc5,0x05,0x42, +0x8e,0xd9,0xa1,0xfc,0xbb,0x6d,0x58,0x93,0xfd,0xa8,0x56,0x9f, +0x6d,0x06,0x65,0xe4,0x27,0xfb,0x2b,0x02,0xf4,0x19,0x5e,0xcc, +0x4b,0xf9,0x7a,0xc4,0x22,0xb9,0x4e,0xe7,0xc9,0x0f,0x69,0x95, +0xac,0x7e,0xc4,0x39,0x28,0xba,0x49,0xd7,0x8b,0xec,0xa6,0xd7, +0x8c,0xbf,0xbb,0x1b,0x08,0x5c,0x35,0xa5,0xeb,0xe5,0x4f,0xa8, +0x86,0x31,0x30,0xa0,0xe9,0xdd,0x24,0xb3,0x8b,0xb4,0xfc,0xd2, +0x4a,0xb3,0x73,0x21,0xc4,0x65,0xf6,0xfe,0x60,0x72,0x71,0x28, +0x35,0x3b,0x94,0x38,0xdb,0x93,0x36,0x10,0x34,0xa1,0x46,0xcd, +0x5a,0x9c,0xee,0x54,0x01,0xe7,0xac,0x26,0x14,0x4c,0xd6,0x3b, +0x70,0xcc,0xad,0x9b,0x48,0xf2,0x5b,0xbe,0xe1,0xad,0x82,0xf2, +0xfd,0x6e,0x19,0x82,0xe5,0xc7,0xb3,0xb2,0xcc,0xd3,0xd9,0x15, +0xfa,0x8f,0x27,0x50,0x4f,0xdc,0x9c,0xc8,0x39,0xbe,0x15,0x88, +0x11,0x78,0xe2,0xbc,0x09,0x85,0xbb,0x46,0xd2,0x2b,0xe3,0x4d, +0xc7,0xdb,0x54,0x9e,0x9b,0x22,0x9d,0x2f,0x9e,0x53,0x92,0xf6, +0x94,0x86,0x28,0xcc,0x24,0x9e,0xae,0xd3,0x32,0x8d,0x57,0x30, +0x53,0xeb,0xe3,0x01,0x6a,0x7e,0x9c,0xad,0x91,0x89,0xe6,0x54, +0x80,0x2b,0xf9,0xdd,0xf7,0x5f,0x3f,0xfb,0xe9,0x6f,0xe8,0xfa, +0x0f,0x53,0x81,0x4e,0x95,0x3c,0xf6,0x5d,0x70,0xfb,0x55,0x0a, +0xcc,0xcc,0x1e,0x0e,0x0d,0x3c,0x45,0xdf,0xdd,0x31,0x3e,0x18, +0xdc,0xdc,0xdc,0x7c,0xde,0xe0,0x38,0x4d,0x94,0x28,0x56,0x2e, +0x04,0x24,0x6c,0x95,0x99,0x1e,0x34,0x8d,0x91,0xb9,0x0f,0xa2, +0xcb,0xcd,0x12,0xd9,0xa4,0x3d,0x9c,0x32,0x17,0xc9,0x7e,0xfa, +0x40,0xe2,0x1f,0x4c,0x98,0x84,0x25,0x8a,0xee,0xa3,0x24,0xb8, +0x85,0x92,0xf6,0x22,0x85,0xd4,0x07,0xe4,0xb7,0xd5,0x3b,0xb5, +0xf8,0x5b,0xb0,0x22,0xc1,0xef,0xe6,0x36,0xef,0xf5,0xcd,0x5c, +0x6d,0xc3,0x57,0xbb,0x18,0x72,0xef,0x94,0x8c,0x01,0xcc,0x7b, +0xbc,0x34,0x0e,0x07,0xa3,0x60,0x66,0x2d,0x2d,0x29,0x95,0xd6, +0xa1,0xf4,0x45,0xfb,0xc1,0x5a,0xcf,0xb2,0x36,0x25,0x92,0xd2, +0xa2,0x43,0x3f,0x21,0x76,0x44,0xa1,0xb1,0x7e,0x9c,0x89,0x13, +0x77,0x9e,0x8e,0x62,0x55,0xdd,0x72,0x37,0xcb,0xee,0xac,0xdb, +0xeb,0xe8,0x58,0x24,0xdc,0x0b,0x39,0xf1,0xa4,0x75,0x2f,0x5a, +0xae,0x7b,0x53,0x7c,0xe7,0xcb,0x61,0x49,0x10,0x22,0x2f,0x6c, +0x5f,0x4a,0xaa,0x22,0x4b,0xc7,0xd6,0x62,0x36,0xcb,0x2a,0xda, +0x72,0x1e,0x39,0x16,0xef,0xb0,0x22,0xe0,0x94,0x36,0x36,0xb3, +0x5f,0x66,0x7f,0xf7,0x50,0x8e,0xcc,0x03,0x31,0x1f,0x7a,0x14, +0xa6,0xbf,0xb5,0xfc,0xab,0x36,0x70,0x28,0xa0,0x76,0x70,0xd8, +0x44,0x34,0xdd,0xe0,0x17,0x26,0x90,0x3a,0x9b,0xb7,0x4c,0x1b, +0x24,0x94,0x6f,0x40,0x8e,0xf8,0x66,0x26,0x07,0x93,0x95,0x1d, +0x4a,0x2f,0x97,0x78,0xf8,0x13,0xf5,0x6d,0x4f,0xe1,0x7a,0xdf, +0xeb,0x80,0x9b,0x54,0x33,0x14,0x99,0x12,0x93,0x95,0x90,0xa3, +0x29,0x51,0x8a,0x0f,0x04,0xd0,0xd8,0xa4,0x0b,0x06,0x80,0x47, +0xc9,0x2a,0xb0,0xdc,0xa8,0xb2,0xb7,0x4c,0x76,0x7c,0x8a,0x66, +0xf0,0xc9,0x0a,0x79,0x01,0x30,0xf2,0x13,0x95,0x0f,0xcf,0xb6, +0xb3,0xf6,0x82,0x1e,0x3e,0x58,0xe3,0xa3,0x6b,0x52,0x45,0x95, +0x4c,0xa7,0xa3,0x33,0x74,0xaf,0xbc,0xd9,0xe0,0x43,0xd0,0x32, +0x1b,0x64,0x88,0x61,0x74,0x86,0xe0,0x2a,0x26,0x77,0xec,0x12, +0x8d,0x1a,0x7d,0xf1,0x29,0x46,0xf2,0x37,0x8a,0x47,0xa3,0xbe, +0xf4,0x68,0x9d,0x27,0x68,0x01,0xe9,0xef,0xd4,0xc6,0x71,0x37, +0x4d,0x30,0xa6,0xf6,0x07,0xaa,0x2e,0x47,0x7d,0x02,0xe0,0x18, +0x35,0xda,0x33,0x34,0xc7,0xc9,0xf7,0x14,0x22,0xf8,0xab,0xdb, +0x4a,0xfa,0x4f,0x6a,0x32,0x48,0x27,0xab,0x88,0x06,0x8f,0xd4, +0xfc,0xc5,0x35,0x57,0xf3,0x16,0xfe,0xe9,0x00,0xf6,0x1e,0x9e, +0x54,0xd1,0x07,0x9d,0x2b,0x2a,0x06,0xe9,0x99,0x8b,0xfe,0x52, +0x7a,0x99,0x6d,0xe8,0x2f,0x1f,0x21,0x4a,0x5f,0x73,0x23,0x23, +0x57,0xb2,0x98,0x75,0xb6,0xc0,0x9b,0x0b,0xd0,0xeb,0xf1,0xae, +0x11,0xae,0x07,0xbd,0xe1,0xc2,0x93,0xbc,0x44,0x40,0xb9,0x2c, +0x8e,0x70,0xf9,0xdd,0xd4,0x60,0x64,0x6e,0x03,0x1c,0xe0,0xac, +0x6f,0xb2,0x42,0x17,0x7c,0x85,0xfd,0x19,0x06,0xb1,0x4b,0xbf, +0xc0,0xe7,0x0d,0x40,0x9f,0x2f,0x4e,0x54,0x3d,0xe4,0x27,0x50, +0xaa,0xb7,0x93,0x5a,0xa6,0x49,0x83,0x7c,0xc7,0x93,0xf4,0x9c, +0x49,0x36,0x0d,0x49,0x53,0xaa,0x12,0xad,0xb2,0x1a,0xfb,0x56, +0x7d,0x4b,0x76,0x47,0x1e,0x84,0x3a,0xce,0x71,0x4a,0xbd,0x4a, +0xf0,0x32,0xc2,0x3e,0x0f,0xca,0xd0,0x60,0x36,0x34,0x4a,0x29, +0xb4,0x37,0xe9,0xc6,0xc5,0xde,0x3c,0xb2,0x87,0xba,0x92,0x47, +0x0f,0xa3,0x98,0x27,0x98,0x68,0xd9,0x6f,0x59,0x16,0xce,0x66, +0x43,0xb1,0x8c,0x34,0x89,0xf4,0x2e,0xbd,0x23,0x63,0xe2,0x26, +0xb2,0x24,0xb3,0x45,0x26,0x40,0x21,0xb2,0x05,0xb5,0x45,0x18, +0x60,0xe8,0xe1,0x01,0x90,0xe1,0x78,0x27,0x51,0x6e,0x89,0xaa, +0x4d,0x49,0x00,0xb1,0x07,0xd0,0xe6,0xac,0xc0,0x03,0x59,0x12, +0xe7,0x74,0xb6,0xd2,0xd5,0x2d,0xda,0x95,0xa5,0xd1,0x51,0x32, +0x4b,0x82,0xdc,0x81,0x97,0xa7,0x2f,0xb3,0x80,0x6b,0x4b,0x27, +0x7a,0x66,0x77,0xd1,0x2a,0x77,0x78,0x4f,0xc1,0xb3,0x99,0xf6, +0x20,0x0e,0x23,0xef,0x10,0x44,0xce,0xff,0xb5,0xb4,0x0c,0x83, +0xc0,0xd9,0xe7,0x17,0x36,0x43,0x68,0x36,0xb1,0x9a,0x0e,0xf3, +0x3a,0x92,0xd6,0x62,0xc7,0x87,0x7d,0x98,0xc6,0x61,0xde,0x66, +0x51,0x41,0x73,0x2e,0xd5,0xb8,0x7a,0x42,0x48,0x27,0x0a,0x6c, +0x3a,0xd2,0xc3,0x37,0x79,0xf1,0x23,0xcb,0x8b,0x9d,0x82,0x84, +0xbd,0x85,0xe3,0x6c,0x60,0xde,0x33,0x19,0x5b,0xd3,0xed,0x58, +0x4d,0xcc,0xf6,0x14,0x1c,0xce,0xb6,0x0e,0xee,0xbb,0x3b,0x77, +0x7e,0xdd,0x95,0xe3,0x67,0x45,0xbd,0x9e,0x37,0x84,0x99,0x19, +0xe1,0x1e,0xc4,0xe9,0x6a,0xbd,0x89,0x31,0xc7,0x9b,0x74,0x71, +0x7c,0x4c,0x2d,0xf6,0x2a,0xbd,0x4c,0xb2,0x2b,0x74,0x0a,0xdd, +0x04,0x08,0xc4,0x6e,0x1c,0x0a,0x86,0xca,0xe1,0xf4,0x39,0x09, +0xc8,0x1f,0x92,0x0c,0xb1,0x80,0x2e,0x5d,0x26,0x47,0xbb,0x68, +0xb6,0xe2,0x49,0xdf,0xa9,0x8c,0xb6,0x23,0x31,0x55,0xb9,0xca, +0x53,0xae,0xc8,0x55,0xbe,0x6a,0x98,0xc2,0xb0,0x74,0x18,0xb4, +0xb2,0x0b,0x53,0x7e,0x5b,0xad,0x30,0x10,0x72,0x1c,0xe8,0x44, +0x83,0x9d,0x57,0xa4,0xc6,0xbe,0xcc,0x98,0xdc,0x00,0x13,0xde, +0x5d,0xa5,0x20,0x86,0x66,0xf6,0x7b,0xc1,0xcb,0x4e,0x7c,0x11, +0x95,0x5e,0x80,0x03,0x24,0x48,0x99,0x79,0x99,0x32,0xc9,0x73, +0x41,0xaf,0xce,0xa9,0x36,0x86,0x88,0xb1,0xaf,0x5e,0xc6,0xc2, +0x64,0x75,0x0d,0x05,0x34,0x56,0x36,0x18,0x29,0xf2,0x39,0xbd, +0x27,0x27,0x6c,0xfb,0x91,0xe8,0x51,0x5f,0xec,0xd0,0xdb,0x45, +0x7a,0x39,0xf3,0x36,0x50,0xf9,0x49,0x3f,0xe2,0x3d,0x01,0xb4, +0xb2,0xd9,0x79,0xd0,0xaa,0x83,0xf4,0x31,0xea,0x6c,0x4d,0xfa, +0x89,0xfd,0x0e,0x0a,0x3d,0xf2,0xe5,0x7b,0x2f,0xbf,0x7f,0xf6, +0xf5,0xd1,0x8b,0x6f,0x9f,0xfd,0xf5,0x79,0x78,0xe4,0x75,0xc8, +0x74,0xe7,0x41,0xd4,0x34,0x5a,0x9a,0xc4,0x4f,0x36,0xb7,0xdc, +0x4e,0xa3,0xea,0x19,0x84,0x5b,0x9d,0xc5,0x5d,0xa9,0xff,0x4d, +0x9e,0xe9,0xb4,0x2d,0xd4,0x93,0x6b,0x98,0x07,0xf0,0x19,0x2e, +0x7a,0xb1,0x56,0x9e,0xca,0x45,0xf8,0xad,0x5f,0x40,0xc4,0x57, +0x61,0x55,0x33,0xf7,0x9f,0xa6,0x63,0x4c,0x0c,0x09,0x42,0x3d, +0xcc,0x96,0xf2,0x4e,0xe5,0x79,0xb3,0x1c,0x50,0xa3,0x2e,0xa1, +0xd9,0x57,0x4a,0xc3,0xe3,0x33,0x13,0x5b,0xad,0x93,0x59,0xc8, +0x99,0x7c,0xb2,0x92,0x2f,0x1c,0x98,0x69,0x11,0x15,0x2c,0x46, +0x21,0x93,0xe8,0xdb,0x67,0xa4,0xb6,0x26,0xa8,0x8f,0x7a,0xb4, +0x22,0x78,0xaf,0x06,0x88,0xda,0x0b,0xbd,0xb3,0xab,0xb5,0x3e, +0x57,0xaf,0x73,0x67,0xfb,0xfd,0x9a,0xdf,0x8e,0xda,0xe7,0x64, +0xdc,0xd5,0x95,0xf0,0x91,0x6a,0xf8,0xc7,0x26,0x72,0xb6,0x5a, +0x69,0x97,0x04,0xb0,0x71,0xa9,0x86,0x13,0x96,0x75,0x6c,0xa8, +0xa2,0x07,0xca,0xc5,0xf3,0x78,0x7e,0x6e,0x46,0x04,0x7e,0xae, +0x40,0xa9,0x51,0x8b,0xc5,0x73,0x84,0x7b,0x99,0x16,0x25,0xc8, +0x49,0x4e,0xa9,0x56,0x07,0xba,0x2e,0x83,0x5b,0xcd,0x98,0x09, +0x8d,0xc2,0x6a,0xf0,0x8d,0x4b,0x81,0xc9,0xdb,0x60,0xbb,0xab, +0x79,0xa8,0x4a,0x68,0x44,0xa4,0x35,0x3a,0xb3,0x7f,0xb3,0x9e, +0x76,0x76,0x6a,0x2e,0xeb,0xe1,0x3a,0x16,0x09,0xbd,0xc9,0xe9, +0xbe,0xab,0xac,0xb4,0x18,0xce,0x44,0xbf,0x8e,0x26,0x43,0x31, +0xce,0x63,0x10,0x4a,0x84,0x1a,0x4f,0x10,0x69,0xc5,0x40,0xa6, +0x5b,0x2f,0x0b,0xe3,0x83,0xa4,0xfe,0x79,0xdb,0x43,0xc7,0x17, +0x9a,0xae,0xf3,0xc9,0xc5,0xb4,0x05,0xa0,0xb2,0x5e,0x42,0x26, +0xc7,0xf1,0x15,0x3d,0x6c,0x1c,0x07,0xfc,0xb8,0x31,0x3f,0xa9, +0x4a,0x4d,0x15,0x4d,0xbc,0xcb,0xec,0xaa,0x48,0xe4,0xcb,0x49, +0xf4,0x7d,0xb5,0xc1,0xe7,0x96,0xa0,0xd7,0x5c,0xe0,0x73,0x4b, +0xb3,0x95,0xfa,0xbc,0x39,0x4f,0x92,0x15,0xfc,0x05,0xc2,0x24, +0x38,0x7c,0xc1,0xca,0x8f,0xd6,0x01,0xf0,0x49,0xf9,0x08,0x03, +0x43,0x4e,0x1f,0x26,0x15,0x8d,0xa6,0x8e,0x5c,0x41,0x20,0xf6, +0x9c,0xe1,0x18,0x4c,0x52,0x42,0xad,0x0f,0x8d,0x2f,0x51,0x52, +0xe5,0xb8,0x4c,0x5c,0x70,0x78,0x6d,0x09,0x30,0x35,0x41,0xbc, +0xdd,0xd0,0x12,0xba,0x51,0x12,0x1f,0x63,0x34,0x55,0xbb,0x59, +0x04,0x89,0xbd,0x12,0x9d,0x3d,0x97,0x3d,0xa5,0xf9,0x40,0xc2, +0x57,0x2b,0x3c,0xa6,0x27,0xf5,0x06,0x0f,0x5a,0x10,0x68,0x4e, +0x51,0xff,0xec,0xce,0x58,0xd7,0xa9,0xad,0xc8,0x5f,0x20,0x12, +0xd5,0x9e,0x38,0xb2,0xc7,0x18,0xb1,0xe1,0x20,0x1f,0x56,0x19, +0xc1,0xe1,0x13,0x21,0xc4,0x2b,0x6a,0x51,0x19,0xed,0xa9,0x56, +0xea,0x93,0x1e,0xe9,0x36,0x68,0x52,0x22,0x85,0x91,0x98,0x64, +0xf1,0x82,0x78,0x65,0xa6,0x9a,0x58,0x0e,0x4e,0x16,0x18,0x72, +0x86,0xa1,0x9c,0x91,0x6f,0x23,0x36,0x2d,0xb0,0xd7,0x49,0xee, +0xa2,0xdc,0x36,0x79,0x0a,0xe8,0xda,0x1a,0x4f,0xa2,0x7d,0x70, +0xf1,0x4d,0x25,0xef,0xeb,0xef,0xbf,0xfd,0x16,0xc1,0x7f,0x9a, +0xe7,0xd9,0x6a,0xb5,0x57,0x3c,0x6c,0xc9,0x63,0x31,0xb1,0x70, +0x20,0x44,0xe4,0xc8,0x9d,0xd0,0x6d,0xde,0x68,0x93,0x66,0x8f, +0x64,0xa9,0xc4,0xed,0x50,0xe0,0xa0,0x31,0xa8,0x54,0x86,0x92, +0x2d,0x7e,0x97,0xd0,0xc9,0x92,0xea,0xf0,0x46,0x6b,0xa4,0x31, +0x15,0x4b,0xa5,0x6e,0x68,0x48,0xb3,0x1f,0x05,0x22,0xc2,0x64, +0x4e,0x26,0x0e,0x67,0xb0,0xc8,0x4c,0x7f,0xd4,0x63,0x76,0x48, +0x54,0xd2,0x70,0x1e,0x9d,0xf3,0xe3,0xed,0xa0,0x7f,0xcb,0x2f, +0x7a,0xc3,0xfb,0x5c,0x3e,0xac,0x82,0xb9,0x43,0xbb,0x8c,0x50, +0xfe,0x6d,0x19,0x4d,0xb3,0xb3,0xb3,0xdf,0x4c,0x17,0x93,0x80, +0xac,0x3b,0x37,0xbb,0x66,0xa9,0x61,0x99,0x3a,0xd6,0x27,0x86, +0xd1,0x0c,0x19,0xd1,0x3b,0x35,0x68,0x19,0xc1,0x28,0x80,0xe3, +0x16,0x33,0x1f,0x45,0x6a,0x0a,0xbd,0xbb,0x3b,0xa7,0xb7,0x4b, +0x60,0xd8,0x4b,0xc5,0x20,0x08,0xec,0x87,0xa1,0xb6,0x07,0x97, +0xa3,0x8d,0x19,0x41,0x4e,0x49,0xcd,0xb5,0xb6,0x33,0xaf,0x40, +0xef,0x6c,0xae,0x0e,0xad,0x33,0x5f,0x32,0x8b,0xe3,0x8a,0xcb, +0xc2,0xa0,0xe3,0xee,0x5b,0x74,0x20,0xae,0x96,0x25,0x07,0x46, +0x3b,0x4b,0x0e,0x7d,0x1c,0xed,0xac,0x3b,0x5a,0x8e,0xa4,0x55, +0xab,0x48,0x5d,0xaa,0xb8,0x56,0x0f,0xa3,0x16,0xf4,0x4a,0xf2, +0xbe,0xab,0xf4,0x95,0x6f,0x2e,0x95,0xb3,0x99,0xaa,0xd9,0xa4, +0xab,0x9d,0xb4,0xda,0x4d,0x3b,0x77,0x13,0x3b,0xd6,0x55,0x1d, +0x6b,0x9b,0x0f,0x95,0x22,0xf7,0xd6,0x7a,0xa4,0xee,0x81,0x0f, +0xfa,0xfd,0x90,0x3f,0x2d,0x78,0xbc,0x85,0xed,0x20,0x6e,0xdc, +0x05,0xb7,0x40,0x71,0x77,0x1f,0xdd,0x35,0x28,0xc3,0x2c,0x68, +0xa7,0x8d,0x40,0x39,0x53,0x96,0x5d,0xd1,0x82,0xea,0x3d,0x5e, +0x84,0xea,0xb1,0x96,0x31,0x3e,0xbe,0x42,0x3b,0x91,0xea,0xba, +0x39,0x08,0x52,0x16,0xd8,0x76,0x54,0x80,0xf4,0x87,0x1c,0x5f, +0x20,0x6a,0x58,0x53,0x21,0x76,0x2b,0x19,0xed,0x4d,0x48,0xf2, +0x29,0xaa,0x98,0x60,0xf2,0x94,0x0b,0x8e,0x32,0x19,0xe2,0xda, +0x52,0xdc,0xd8,0x43,0x9b,0x5b,0x34,0xfb,0xc3,0x4c,0x63,0xf9, +0xd0,0x11,0x1e,0x4f,0xc0,0xaa,0x09,0x11,0x26,0x79,0x59,0x93, +0x4a,0x2b,0xbc,0x75,0xb6,0x86,0x21,0x9b,0x2e,0x24,0x84,0x16, +0x20,0x6f,0xa4,0x39,0xb0,0x30,0x96,0x2b,0xc3,0x36,0xe8,0x28, +0xb2,0xa9,0xdf,0xcc,0xaf,0x40,0x2f,0x5f,0x24,0x67,0xc5,0x44, +0x25,0x4e,0x99,0x08,0x49,0x9e,0x07,0x1a,0xb8,0xff,0x17,0xaf, +0xa3,0x52,0x3b,0x5e,0xa0,0xb6,0x94,0xf6,0x90,0x44,0x90,0x1d, +0x4f,0x12,0xe5,0x5c,0x32,0x9e,0xfb,0xfb,0x3c,0x30,0x8c,0xd4, +0x59,0x87,0x75,0x86,0x53,0xaa,0xfd,0x26,0x7a,0x9a,0x51,0xed, +0x7d,0x7b,0x9d,0xaa,0x03,0x11,0x9d,0xda,0xa6,0xc4,0xc9,0xb7, +0xf0,0xf0,0x40,0xf2,0x0d,0xdd,0xd8,0xcc,0x82,0xa0,0xdd,0xff, +0x09,0xb4,0xe8,0xae,0x99,0xaa,0xb9,0xe0,0x3b,0x42,0x58,0x42, +0xb1,0xdf,0x71,0x87,0x79,0x16,0x2b,0xd0,0xce,0x3a,0x94,0xcc, +0xca,0xf0,0xb8,0xf7,0x69,0x68,0xf9,0xef,0xd8,0x47,0xac,0x2c, +0x4a,0xed,0x8b,0x6c,0xdb,0x9d,0x96,0xdc,0x47,0x32,0xc2,0xfe, +0x49,0x24,0xeb,0xa2,0x5a,0x48,0x96,0x7c,0xff,0x9d,0xdb,0xd9, +0x76,0x61,0x21,0x55,0x82,0x1d,0xaa,0xa4,0xab,0x0a,0xdb,0x11, +0x3e,0xa8,0x59,0x99,0x39,0x48,0xb3,0xc0,0x2b,0x04,0x67,0xeb, +0xfa,0x96,0xd4,0xda,0xd3,0x46,0xfc,0x2d,0xa9,0x72,0x27,0xd7, +0xdc,0x3d,0x6c,0x85,0x52,0x1b,0xb9,0xd6,0x50,0xd7,0xb8,0xa3, +0xed,0xef,0xbf,0x9b,0xdd,0xc6,0xa7,0x6a,0xe0,0x09,0xfd,0x48, +0x64,0x83,0x5e,0x48,0xaa,0xdb,0x93,0xaa,0xa1,0x27,0x76,0x6f, +0x88,0xb5,0x63,0xb1,0x40,0xcd,0xb5,0xb1,0xa0,0xe9,0xca,0xe3, +0xcf,0xea,0xc4,0x5c,0x5a,0x4b,0x3f,0x4e,0xf9,0xb6,0x8b,0x3a, +0x42,0xfe,0x3d,0x2f,0x01,0x41,0x8d,0x3e,0xfc,0xfe,0x0f,0xd4, +0xfe,0x57,0x5e,0xfd,0x69,0xdf,0x77,0xa1,0xfb,0x4e,0xbb,0xcc, +0xfc,0xee,0x27,0xff,0xf0,0x53,0xe0,0xc2,0x3b,0xcf,0x93,0x33, +0x4f,0xd0,0xa6,0x49,0xbb,0xcc,0x57,0xfb,0xc4,0xbd,0x3e,0x24, +0xe9,0x37,0x07,0x05,0xdc,0xee,0x71,0xce,0x0d,0x1d,0x39,0x2f, +0xfb,0xd2,0xb4,0x0d,0xa7,0x8b,0xb1,0xf5,0xdd,0xf1,0x8e,0xbc, +0xd0,0xf3,0xe8,0x4e,0x0b,0x46,0xfe,0x1c,0xa3,0x27,0x8c,0x72, +0xec,0x84,0x9a,0x40,0xfc,0x22,0xf3,0xd8,0x0e,0x18,0x10,0x25, +0xbc,0xca,0x92,0x6e,0x6c,0x3e,0x77,0xef,0xf9,0x69,0x73,0x3b, +0x3c,0xeb,0x3f,0xf2,0x54,0x01,0x77,0x77,0xad,0x90,0x81,0x6b, +0xc4,0xb7,0xb7,0x0f,0xa0,0x9d,0xb0,0xba,0xd7,0xb3,0xd0,0x17, +0x7b,0x60,0x01,0x27,0x0d,0xf1,0x28,0x40,0x57,0x96,0xf4,0xe5, +0xa5,0x49,0xe3,0xf2,0xd2,0x54,0xa0,0x20,0x44,0x1e,0x2c,0xd8, +0x1a,0x32,0xe3,0xf5,0x3e,0x4f,0x2e,0xbd,0xb0,0x5d,0xd4,0xbc, +0xde,0xe3,0x27,0x98,0xda,0x3e,0x76,0x40,0xd3,0x77,0x16,0xfb, +0x5b,0xbf,0xb3,0x68,0x17,0x80,0x05,0x24,0xee,0x49,0x42,0xfa, +0xbb,0xf1,0x7a,0x7e,0x9e,0xc1,0x02,0x8f,0x2a,0x10,0x88,0x78, +0x7c,0xcf,0x10,0x20,0x2f,0x44,0x79,0x9d,0x18,0x5b,0xcd,0x19, +0x05,0x42,0x79,0xe6,0x87,0x8a,0x4b,0x6b,0xee,0x80,0xf8,0xd2, +0x56,0x02,0x52,0x89,0x69,0xfb,0x70,0x40,0xba,0xac,0x02,0x12, +0x4d,0x1a,0xfb,0xba,0x8c,0xe4,0x7d,0xa0,0x3d,0xba,0x8c,0x87, +0xad,0x2e,0x5f,0xb2,0xd7,0x57,0xce,0x48,0xa5,0xb1,0xad,0xfe, +0xd1,0xd6,0xfd,0xf7,0x1e,0x0e,0x1b,0xf8,0xdf,0xcc,0x96,0x2d, +0x63,0x5d,0xe3,0xbe,0x7b,0x5b,0x59,0x1f,0xdc,0x79,0xe9,0x48, +0xa5,0x8c,0x97,0x02,0x4f,0x22,0xc5,0xfa,0xa0,0x15,0x2c,0xed, +0x9f,0xf2,0xa5,0xa7,0x6f,0x41,0x6d,0x76,0xe3,0xf9,0x78,0xf7, +0x8d,0x27,0x5c,0x64,0x8d,0x5b,0x44,0x2d,0xf9,0x36,0x30,0x7e, +0xa5,0x55,0x23,0x1b,0xdb,0x55,0xc5,0x65,0x1c,0x61,0x70,0x3c, +0xb1,0xce,0x49,0xa6,0xa1,0x6d,0xde,0x0f,0x8b,0x64,0x7d,0x8a, +0x62,0xac,0xd8,0x23,0x34,0x9a,0x1f,0xb3,0xa2,0x4c,0xda,0xbc, +0x58,0xe6,0x9b,0x22,0x5a,0xaa,0xf3,0xd2,0xf9,0x79,0xba,0x5a, +0xe0,0x06,0x7e,0x21,0xd6,0x59,0xf9,0xd7,0xc8,0x5b,0xe2,0x4d, +0x16,0x24,0x61,0x13,0xf9,0x9b,0x08,0x81,0x27,0x44,0x5d,0x67, +0x30,0x0d,0xc6,0x66,0x93,0x74,0x43,0xfe,0xba,0xe4,0x29,0xf7, +0x52,0x1f,0x70,0xa7,0xf8,0xc4,0xf8,0xd2,0xeb,0x1c,0x75,0x3a, +0xc4,0x04,0x3c,0xff,0xc0,0xa7,0x56,0x41,0x87,0xde,0x18,0x90, +0x40,0x14,0x11,0x2a,0xd3,0x47,0x4f,0xa1,0x17,0x60,0x59,0x97, +0x78,0x4f,0x43,0x1f,0xb6,0x17,0xf2,0x7c,0x9d,0x0e,0x04,0x44, +0x12,0x5d,0x36,0x8f,0xd5,0x21,0x8b,0x5e,0xbc,0x22,0xd5,0x63, +0x7f,0xc3,0x32,0xec,0x7b,0x9b,0x8c,0x2a,0xdf,0xe5,0x1d,0x2d, +0xda,0xd3,0x28,0x00,0x47,0x2f,0x71,0xf6,0xd0,0x60,0x29,0xa4, +0x16,0x3c,0xbd,0x37,0xb8,0xfc,0x43,0x19,0xd1,0x4b,0x1f,0x58, +0x20,0xd0,0x4b,0x99,0x3a,0x85,0x8e,0x0d,0x12,0x5d,0x5d,0x54, +0xac,0x76,0x8e,0xf4,0xd1,0xc6,0x6b,0x49,0xd6,0x15,0xf8,0x42, +0xe8,0x21,0x62,0x2e,0x77,0xce,0xfb,0x2f,0x99,0x3c,0x44,0x18, +0x50,0x33,0x8c,0x65,0x0c,0xe1,0x09,0x42,0x87,0x39,0xc4,0x36, +0x24,0xa1,0xe7,0x14,0x49,0xf0,0xaa,0xcb,0x04,0x76,0x76,0xb1, +0x39,0x74,0x7f,0xcd,0x16,0x3d,0xb5,0x40,0xc4,0x3b,0x69,0x9b, +0xe8,0xa0,0x94,0xc2,0x70,0xcd,0x85,0x51,0x7b,0xf3,0x0d,0x94, +0x40,0xcc,0xa1,0x1d,0x5b,0x04,0x2b,0x9a,0xab,0x25,0xa2,0x5c, +0x3e,0x9a,0x1b,0x6a,0x54,0x20,0x19,0xc5,0xcf,0x71,0xcf,0xe2, +0xcd,0x1b,0x6c,0xf4,0x37,0x6f,0xa4,0x4c,0xab,0x3b,0x69,0x6e, +0x5a,0xf3,0x6e,0x1a,0x5d,0x69,0x73,0x41,0xdc,0xab,0x6d,0x85, +0x73,0x02,0x80,0x90,0x28,0x95,0x15,0xd0,0xf1,0x90,0x9b,0x5d, +0xed,0x17,0xd0,0x34,0x27,0xad,0xad,0x06,0x28,0x40,0xdf,0xf0, +0x02,0xfc,0x9d,0x8e,0xf0,0x90,0x90,0x37,0x9e,0x7d,0xd1,0xab, +0xf5,0x22,0x9a,0x39,0xd4,0x45,0x1c,0xfa,0x1a,0x0c,0x21,0xf9, +0x2d,0xf7,0xcf,0x7e,0x35,0x95,0x7b,0xae,0xa3,0xed,0xc3,0x67, +0x5d,0x4b,0x93,0x28,0x39,0x06,0x90,0x6e,0xa5,0xb3,0x46,0x5e, +0x93,0x1a,0x37,0x9b,0x73,0x3f,0x30,0x8b,0xbf,0x1d,0xef,0x9b, +0x15,0xf9,0xd9,0xac,0xc9,0xa3,0x66,0x2d,0xed,0x89,0x1d,0x48, +0xed,0xa9,0x12,0x21,0x01,0xd0,0x56,0x98,0x15,0x8c,0xd4,0xd8, +0x03,0xda,0x26,0xf3,0xd9,0x6d,0x1d,0xed,0xa2,0x48,0xdf,0x68, +0x72,0x88,0xc2,0xd7,0x9a,0xf5,0x5d,0xea,0x5b,0xd4,0x42,0x42, +0x8f,0xbc,0x1c,0x7a,0x82,0x75,0x61,0xad,0x73,0x08,0x52,0x82, +0x43,0x0f,0x6a,0xe6,0x6d,0x05,0xef,0x67,0x44,0xb7,0xb8,0x5f, +0x11,0xf2,0xae,0x85,0xb0,0xae,0x16,0x73,0x54,0x57,0xde,0xa5, +0xf6,0xe4,0x2d,0x5d,0x44,0x8d,0x7f,0x55,0x98,0x6e,0xeb,0xaa, +0xc8,0xae,0x9c,0xb5,0x9c,0x5b,0xbf,0x3a,0xb1,0x81,0xe9,0xab, +0x78,0xa3,0x93,0x90,0x1b,0xf3,0x78,0xa3,0x92,0xd0,0xde,0x44, +0xa7,0xa1,0x29,0x20,0xd9,0xba,0xd9,0xa9,0x6c,0xcb,0xe7,0xc0, +0xf0,0x55,0x37,0xbe,0x15,0x4e,0xc5,0xaa,0xf2,0x64,0x4d,0x69, +0x33,0x27,0x52,0x67,0x2c,0x5c,0x79,0x35,0x0c,0xdf,0x5e,0x6d, +0xa0,0xe7,0x25,0xca,0x70,0x9a,0x6e,0x4a,0xb5,0x78,0xad,0x24, +0x5b,0x69,0xf8,0x27,0x8d,0xa3,0xd1,0x5b,0x8f,0xb4,0x86,0xe6, +0x3f,0x6c,0xfe,0x9c,0x6b,0xa3,0xee,0x5c,0x1b,0x74,0x97,0xc6, +0x98,0x5b,0x1e,0x88,0xe2,0x74,0x98,0x2c,0xc2,0x5b,0x2a,0xe6, +0x51,0x9f,0x0b,0xc1,0x3b,0x07,0x44,0x59,0x48,0xbf,0xe2,0x1e, +0x9b,0x75,0x9f,0x7c,0xa7,0xf1,0x26,0x04,0xdd,0x19,0xd1,0x66, +0xeb,0x2a,0x85,0xa8,0xa6,0xdb,0x24,0xd2,0x72,0xdd,0x64,0x99, +0x5b,0x96,0xeb,0x2a,0x76,0xd3,0xb4,0x5c,0x57,0x09,0x8e,0x7d, +0x37,0x27,0xd0,0x34,0x22,0x2c,0xeb,0x6e,0x8e,0xc6,0x12,0x3d, +0x61,0xd9,0x76,0x73,0x34,0x7e,0x7a,0xc2,0xb6,0xec,0xe6,0x78, +0xfa,0xf6,0x84,0x34,0xec,0xa6,0xfa,0xe2,0x8f,0x36,0xe9,0xc6, +0x7e,0xd0,0x66,0xcf,0x7d,0xd8,0x60,0x0f,0x67,0x3e,0xeb,0x2c, +0xf7,0xa0,0xfd,0x1e,0xee,0x9f,0x45,0xb7,0xdb,0x3f,0xcc,0x7a, +0xaf,0x61,0xbe,0x84,0x16,0x7c,0xed,0x87,0xb5,0x7a,0xf7,0xd6, +0x6b,0xda,0xd2,0x51,0x7d,0xdc,0x4c,0x6a,0x32,0x87,0x94,0xbd, +0xe6,0x79,0xa6,0xf2,0x1a,0x8b,0xad,0xf3,0xfc,0x9e,0x76,0x7a, +0x4e,0x25,0x7f,0xad,0xad,0x1e,0xd1,0xf7,0xfb,0x9a,0xdb,0x59, +0x46,0x70,0x9c,0x65,0x9f,0x0e,0xaf,0x0b,0x41,0x55,0x5e,0x21, +0xd2,0xfb,0x1b,0x1f,0x74,0xba,0xf0,0x5b,0x0c,0xda,0x78,0x1c, +0xfa,0x1a,0x45,0xb2,0x21,0xec,0x4b,0x7c,0x4f,0x5a,0x69,0x74, +0xc4,0x2b,0x58,0xb7,0xad,0x22,0x23,0xc3,0x30,0x73,0x99,0xe1, +0x2d,0x46,0xf7,0x0f,0x24,0xd9,0x34,0x91,0x48,0x09,0xc3,0x08, +0xcb,0x62,0x02,0xbd,0xd8,0x2e,0xcc,0xe9,0x36,0xa6,0x1a,0xc1, +0xd3,0x67,0x88,0x14,0x9d,0x5c,0x6e,0x60,0x59,0x85,0x7e,0x22, +0x34,0xd2,0x74,0x5d,0xc0,0x5a,0x4b,0x22,0x15,0x5e,0x78,0x96, +0xe6,0x45,0xd9,0x25,0xbd,0xc9,0x43,0x73,0x2e,0xa0,0x37,0xa2, +0xbc,0x96,0x19,0xea,0xca,0xb9,0xef,0xac,0x0c,0x51,0x81,0x08, +0xec,0xea,0xe9,0x02,0xe2,0xd8,0x4d,0x04,0xe4,0xdd,0xb1,0x45, +0xdd,0xc9,0x6b,0x54,0xf6,0x96,0xcc,0x0d,0x15,0x95,0x23,0x1d, +0x25,0x3a,0x5d,0xa8,0x4b,0x1f,0x67,0x11,0xb0,0x6b,0x92,0x2e, +0xa6,0x08,0xd6,0x30,0x11,0x0d,0x46,0x10,0xc7,0x45,0xdd,0x56, +0x83,0x10,0x75,0x93,0x0a,0xaf,0x68,0xf3,0xd7,0x50,0xd4,0x1c, +0x57,0xa3,0xdb,0x26,0xfe,0x1a,0x6e,0xe5,0xdd,0x6d,0xcc,0x68, +0xd5,0x1d,0xe3,0x54,0x8d,0x11,0x90,0x2e,0x76,0x33,0xe8,0x4e, +0x5d,0x19,0x56,0xa6,0x35,0xaa,0x42,0x91,0x5c,0x77,0x35,0xe5, +0xb5,0xba,0xb4,0xe6,0xc4,0xad,0xe2,0x13,0x66,0xeb,0xd2,0xfd, +0xef,0x3d,0xf0,0x94,0x46,0x46,0x16,0x2d,0x67,0xbd,0x87,0xed, +0x25,0xcd,0x71,0x84,0xe5,0xa5,0x68,0x2c,0x87,0x17,0xcc,0x48, +0xba,0xac,0xdf,0x30,0x5a,0x97,0x0b,0x7e,0x5e,0xb8,0x6a,0x43, +0x4b,0xc6,0xdc,0x55,0x9b,0x9c,0xa6,0x43,0xf8,0xda,0x94,0xcc, +0x2a,0x6f,0x97,0x46,0x2a,0xca,0x10,0x6a,0x99,0x74,0x92,0x05, +0x3b,0xaa,0xf7,0x82,0x8e,0xe5,0xf0,0x20,0x89,0x0f,0xcb,0xec, +0x93,0x33,0xeb,0xa6,0x7d,0xab,0x4f,0x0a,0x6d,0xe0,0xcf,0xe7, +0x42,0x80,0x8d,0xcd,0xfd,0xa9,0x43,0xd2,0xe4,0x61,0xac,0xff, +0xf5,0x09,0x20,0x65,0xe2,0x53,0x3f,0x9e,0xf6,0xf9,0x7c,0x79, +0x6c,0x40,0x09,0x63,0x48,0x14,0x32,0x1c,0xd7,0xdb,0x36,0x84, +0x21,0xca,0xc9,0x90,0x50,0x82,0xf0,0xb6,0x47,0x23,0xed,0x21, +0x96,0x92,0x72,0xf9,0xff,0x50,0x43,0x49,0xcd,0x9b,0x0f,0xb2, +0x92,0xe4,0x42,0x76,0x0c,0xbf,0x10,0xd9,0x21,0xab,0x2f,0x48, +0x97,0xc9,0xbe,0x73,0xf6,0xbc,0x63,0x75,0xb4,0x6b,0xc7,0xc1, +0xf6,0x1a,0x96,0xe9,0x06,0xf3,0x5d,0x9a,0x71,0xa8,0xb5,0xd5, +0x08,0x23,0x61,0x1c,0x73,0x6f,0x6a,0x50,0x53,0x10,0xff,0xd1, +0x7b,0x4a,0xe3,0x1c,0x3f,0x2e,0xcd,0x39,0x3e,0x31,0xd9,0xde, +0x0c,0xf9,0x7f,0xd6,0x4c,0xed,0xa1,0x1a,0x8f,0xb6,0x56,0x6b, +0x2a,0x3a,0x07,0x8c,0xd6,0x76,0xf5,0x99,0x36,0x25,0xe8,0x43, +0xcc,0xd5,0xfe,0xc7,0x32,0xec,0x57,0x5b,0x86,0xfd,0xbe,0x76, +0x29,0x62,0x91,0x5d,0xee,0x5c,0xb3,0x80,0x56,0xd5,0xcd,0x59, +0x45,0xfb,0x6d,0x52,0x50,0xc1,0x09,0xcd,0x10,0x60,0x0d,0x2c, +0x8a,0x90,0xed,0x03,0xed,0x5e,0xa0,0x8a,0x40,0xc8,0xae,0xd9, +0x21,0x9d,0x72,0x57,0x38,0x0c,0xfd,0x91,0xe6,0x2f,0xf7,0x31, +0xe1,0x8f,0x31,0x8f,0xf1,0xb1,0xc6,0xac,0x27,0xb4,0x54,0x9a, +0x72,0x51,0xbd,0x5d,0x33,0x9a,0xa6,0x1d,0x8d,0x6b,0x5c,0x42, +0x5b,0xb7,0x6c,0x5d,0x42,0xb6,0xcf,0xd0,0x5f,0x60,0x06,0x7b, +0x66,0xdb,0x64,0x39,0x86,0xcf,0xa8,0x8a,0x15,0xaf,0xf2,0x24, +0xe1,0xe8,0xf1,0xf5,0xf2,0xcd,0x65,0x7c,0x91,0xbc,0x29,0x31, +0xca,0x45,0xa2,0xa1,0x70,0xc8,0xa5,0x78,0x52,0x1e,0x61,0xc2, +0x00,0xd5,0x80,0xc2,0x97,0xf1,0xa6,0x91,0x27,0x68,0x52,0xe2, +0x90,0x71,0x75,0xa9,0xe6,0x36,0x75,0xe7,0x1b,0x00,0x43,0x2b, +0x21,0xa4,0xdf,0x6d,0x13,0x09,0x52,0x62,0x10,0xd1,0x64,0xaa, +0x49,0xe1,0xa6,0x07,0x6d,0x61,0x11,0xb9,0x99,0x08,0x0e,0xf7, +0x0c,0xf1,0xef,0x44,0xc1,0x93,0xe5,0x13,0xba,0x8e,0x36,0xbb, +0x86,0x01,0x59,0x9d,0x2c,0x7a,0x64,0x5b,0x51,0xf0,0x8b,0x1a, +0xfe,0x9a,0x3c,0x5f,0x69,0xb5,0x42,0xa5,0x42,0x1b,0x45,0x2d, +0x5c,0x9a,0xd3,0x13,0x7e,0x8a,0x24,0xd5,0x60,0x0b,0x53,0x8f, +0x4b,0xd4,0x1c,0xad,0x89,0xe3,0xac,0xd5,0x29,0x99,0xd3,0x52, +0x94,0xcf,0x36,0xcf,0x0c,0xa8,0x11,0x2c,0xe6,0xb8,0x8b,0x3e, +0x0c,0xc9,0x8d,0x52,0x7d,0x01,0x1f,0x01,0xa8,0xc5,0xb9,0x52, +0xb4,0x2d,0x35,0xa6,0x37,0x1f,0x26,0x7d,0x7a,0x0f,0x62,0xca, +0xaf,0x42,0xe0,0x0f,0xb5,0xe6,0xae,0xe6,0x89,0x5b,0x38,0x50, +0x53,0x8f,0xc5,0x3c,0x3b,0x3b,0x42,0xcf,0x4d,0x50,0xc8,0x64, +0x31,0x0d,0x17,0x5b,0xab,0xbd,0x5d,0x23,0x76,0x34,0x6f,0xc7, +0xd1,0x01,0x37,0x0f,0xf5,0x0d,0x01,0xa0,0xe5,0x0d,0x82,0x41, +0x6f,0x2c,0x7e,0xc0,0x6e,0x0d,0x3a,0x2d,0x5d,0x21,0x18,0xc3, +0x4f,0xd8,0x6a,0x23,0x8f,0xcd,0xd5,0xbc,0x0c,0x20,0x17,0x5c, +0x49,0xa9,0xf1,0xa5,0xc5,0x37,0xe9,0x2a,0xf1,0xf1,0x6c,0x76, +0xac,0xe2,0xce,0x20,0x26,0xd4,0x05,0x96,0xe5,0x66,0x04,0x39, +0x10,0xc4,0xd0,0xb5,0x4d,0x56,0x05,0xaa,0xab,0x0c,0x53,0x9d, +0xe7,0x8d,0x64,0xb2,0xc9,0x55,0xc9,0x1b,0x49,0xf0,0x8f,0xcf, +0xa3,0xd3,0x7f,0x9f,0x3c,0xeb,0xfe,0x5b,0xdc,0x7d,0x3f,0xed, +0xbc,0x0e,0x5f,0x9f,0xbe,0x3e,0x3d,0x15,0x76,0xa9,0xaa,0x6a, +0x91,0x47,0x34,0x9c,0x9e,0x7a,0x23,0xc5,0xa4,0x3c,0x71,0x2d, +0xfd,0x2d,0x6b,0x8a,0x78,0x41,0xed,0x2f,0x70,0xf1,0x1b,0x2b, +0x95,0x09,0x07,0x18,0x8e,0x38,0x3e,0xe6,0xbf,0x34,0xe8,0xdc, +0xdd,0x79,0x6f,0x8b,0x6c,0xed,0x8d,0x2c,0x59,0x91,0x70,0xd2, +0x36,0xd2,0xc1,0x25,0x1a,0x28,0xc8,0xff,0xc7,0xf1,0x31,0xfd, +0xf9,0x99,0xe4,0xda,0x06,0xe7,0x64,0x29,0x6d,0x96,0xe5,0x49, +0x03,0x18,0x8b,0x91,0x9a,0x3b,0xd4,0xd5,0x2c,0x6d,0x39,0x5e, +0x6c,0xd0,0x7d,0x60,0xe1,0x0a,0x15,0xc4,0x58,0x0e,0x26,0x00, +0x9f,0xb4,0xe4,0x2c,0x26,0x90,0x32,0x9d,0xa2,0xc5,0x23,0xf6, +0xf4,0xc6,0x18,0x05,0xcc,0x7b,0x43,0xd0,0x2c,0xc4,0x08,0x32, +0x9e,0x60,0xca,0x94,0x46,0x21,0x81,0xe5,0x88,0x0d,0x65,0xb5, +0xce,0xdb,0xdc,0x4c,0x3b,0x80,0xbc,0xb4,0x24,0x57,0x56,0xe8, +0x52,0x7c,0x0e,0x3f,0x8e,0x4b,0x71,0x16,0xe9,0x9d,0xd7,0x3b, +0x69,0xa4,0xa0,0x44,0xec,0xf1,0xb8,0x97,0x8d,0x39,0xb9,0xaa, +0x12,0x1a,0x23,0xd4,0x76,0x36,0xd6,0x0b,0x77,0xad,0xa7,0xd1, +0x06,0x7e,0x7d,0x13,0x0e,0x46,0x48,0xc5,0xf1,0xb1,0x1a,0x57, +0x64,0x63,0x7c,0x95,0xad,0x70,0x42,0xa2,0x0e,0xa8,0xc8,0x7d, +0xd4,0x97,0x7b,0xdc,0xb2,0x7d,0x71,0xb4,0x97,0xcc,0x8b,0x6e, +0xd7,0x57,0x97,0x33,0x98,0x9e,0x81,0x63,0xfc,0x25,0xbc,0x59, +0x96,0xad,0x92,0x78,0xed,0x61,0x9c,0xfc,0x16,0xb4,0x77,0xfa, +0x35,0xfc,0x70,0xd3,0xea,0x39,0x5a,0x62,0xec,0xa1,0x2c,0xb9, +0xf3,0x85,0x23,0x85,0x8b,0xe8,0xff,0xfc,0xf4,0xfd,0x77,0x9c, +0x57,0x36,0x89,0x83,0xc0,0xc8,0x95,0x3c,0xdf,0xc5,0x6d,0x08, +0x6c,0xc5,0x78,0x3e,0x4f,0x8a,0x02,0x58,0xd6,0x48,0x0f,0x60, +0x78,0x01,0xf9,0xda,0x0a,0x45,0xc0,0xbc,0xb8,0x6e,0xce,0x57, +0x5c,0x30,0xcc,0xd1,0x90,0xd6,0x56,0xb4,0x95,0xbb,0x3c,0x90, +0xbb,0xbc,0x3f,0x77,0xb6,0xc9,0x0e,0x30,0x40,0x3f,0x86,0xa3, +0xe0,0x02,0x33,0x93,0xf2,0xed,0x21,0xef,0x15,0xa4,0x20,0x87, +0x8e,0x56,0xe9,0x2c,0x8f,0xf3,0x1a,0x0f,0x0d,0x8f,0xf8,0x8a, +0x0e,0xac,0x2c,0x40,0x87,0xa3,0x71,0xa4,0xdc,0xe1,0x22,0xaa, +0xe5,0xe8,0xb9,0xa0,0x9d,0x99,0x67,0x09,0xcc,0x8b,0x79,0x32, +0x46,0x20,0x1f,0x7f,0xca,0x1e,0xfc,0xa2,0xf3,0xc9,0x89,0x0b, +0x31,0x0d,0xc6,0x8a,0x36,0x15,0xe5,0x97,0xa4,0xf2,0xab,0x60, +0x11,0x9a,0xbb,0x4e,0xde,0x8b,0x35,0x88,0x5c,0xba,0x38,0xd2, +0x54,0x33,0x56,0xbc,0x61,0xe4,0xe2,0x45,0xca,0x83,0xb0,0x41, +0xd5,0x65,0x52,0x9c,0x1f,0x20,0x09,0x93,0x81,0x9e,0x89,0x26, +0x08,0x23,0x80,0x9a,0x3d,0x80,0xd3,0x0f,0x23,0x0c,0x33,0x31, +0x55,0x26,0xd7,0xb7,0x69,0x51,0xc0,0xfc,0x64,0x72,0x49,0xea, +0x8f,0x32,0x58,0x3d,0x03,0x3c,0x0e,0x34,0xa0,0x5e,0x95,0xa0, +0xad,0x7a,0xc4,0x71,0xab,0xe5,0xa1,0x0b,0xde,0x2b,0xfa,0x93, +0x66,0xab,0xe9,0xf6,0x5a,0xc0,0x9a,0xf5,0x7a,0x89,0x48,0xde, +0xbc,0xc1,0xab,0x58,0x0b,0xad,0x1f,0xc9,0xa1,0x58,0x87,0xd1, +0x68,0x56,0x7e,0x7a,0x28,0xf6,0x38,0xd8,0xf7,0x54,0x77,0x96, +0x7f,0x39,0x92,0xca,0x89,0xac,0xb1,0x96,0xa2,0x2d,0x5d,0x8f, +0x77,0x68,0xf6,0x3c,0xc5,0xa5,0x46,0x3a,0x57,0x87,0x68,0xce, +0xeb,0x4a,0xab,0x5d,0x81,0xfa,0x09,0xe2,0xa7,0xdd,0x0b,0x9f, +0xa5,0xc9,0x6a,0xd1,0xe2,0x5f,0x18,0xe1,0x58,0x69,0x67,0x08, +0x18,0xf9,0xb0,0xff,0x2a,0x36,0x60,0xf2,0x96,0xd7,0xc7,0x9c, +0x6e,0x89,0x34,0x91,0xd4,0x93,0xd1,0x46,0xf9,0xc1,0x30,0x2a, +0xbf,0x46,0x03,0x62,0x08,0xbd,0x01,0x65,0x20,0x94,0x8e,0xa9, +0x86,0x12,0x3c,0x29,0x01,0x88,0xad,0xfa,0xa3,0xd8,0x32,0xcf, +0x36,0x75,0x73,0xb5,0x76,0x96,0x83,0x96,0x6f,0x0f,0x43,0x1e, +0x82,0x82,0x14,0x68,0x3a,0x45,0x2c,0x9f,0x74,0xa6,0xec,0xae, +0xb2,0xe6,0xf0,0xcc,0x3c,0x3e,0x81,0x17,0xe7,0x10,0x31,0xc4, +0x0a,0xb9,0xa3,0x11,0x17,0x77,0x77,0x8c,0xd2,0x99,0x42,0xf6, +0xf1,0x73,0x31,0xe1,0x7c,0xc0,0xc7,0x69,0x04,0xe8,0x26,0x9a, +0xad,0x46,0xb4,0xcc,0x73,0x77,0x48,0x59,0x8f,0x6a,0xe2,0xf2, +0xcf,0xf0,0xae,0x59,0x4b,0x4e,0xa6,0x2a,0x6d,0x05,0x67,0xff, +0x0d,0x0d,0x60,0xe3,0x89,0x0f,0xe2,0x88,0xef,0xcb,0x6f,0xb5, +0x16,0x3e,0xeb,0xd5,0x2a,0xc4,0x94,0x62,0x0d,0xe2,0x19,0x4d, +0xd1,0x30,0x84,0x5d,0xad,0x48,0x58,0x31,0x65,0x18,0x65,0xe5, +0x39,0x2e,0xa8,0xa8,0xbb,0xba,0x73,0x75,0xd3,0x02,0x40,0x1e, +0x39,0x8f,0xd4,0x94,0x0c,0x8b,0x81,0x34,0x5e,0x1e,0x1f,0xa7, +0xb0,0x6e,0x7b,0x8b,0xce,0xd2,0x71,0x3a,0xcd,0x26,0xd9,0x55, +0xb9,0xb9,0x2a,0xe9,0x84,0x61,0xaa,0xa7,0x77,0x1d,0x4d,0x87, +0x1a,0x1c,0x3f,0xc4,0x33,0x68,0x26,0x87,0xfb,0x44,0x66,0x7a, +0x02,0x47,0x53,0x4f,0x20,0x02,0x59,0xb4,0xb8,0xc4,0xe8,0xd1, +0x40,0x30,0x36,0x75,0x04,0x1c,0xeb,0xe3,0xde,0x99,0xa7,0x67, +0x64,0xaa,0xfe,0xc4,0xbb,0x49,0xcb,0x73,0xcf,0xba,0x75,0x65, +0x5a,0x9b,0x11,0x83,0x72,0x80,0x80,0x5b,0xfe,0xd3,0xc3,0x02, +0xb2,0x75,0x6c,0xad,0xf9,0xcc,0x96,0x82,0x2a,0xbe,0x72,0xb3, +0x48,0x5a,0xcc,0x06,0x4a,0xbc,0xb1,0xd7,0x88,0xa4,0xc9,0x31, +0x4c,0xb0,0x7b,0x47,0x0c,0x35,0x7a,0xe5,0x41,0x06,0x32,0x4e, +0x2e,0xf0,0x29,0x57,0x86,0x86,0x6f,0x19,0x85,0x3b,0xd5,0x76, +0x91,0xa6,0xf5,0xcf,0xe2,0xb9,0xeb,0xe7,0xc5,0x58,0x38,0x62, +0x8a,0xdd,0xfa,0xb3,0xb7,0xe2,0x02,0x07,0x18,0x71,0x01,0xcb, +0x10,0x7e,0xfc,0x45,0x5c,0x88,0x8b,0x6b,0x25,0x11,0xb7,0x40, +0x68,0xe8,0x79,0x02,0xe9,0x0d,0x27,0xd2,0xec,0x1f,0xbf,0xb6, +0x02,0xa8,0xa0,0x85,0x71,0x0c,0xba,0xa9,0x6c,0x30,0x4e,0x1d, +0x49,0x67,0xa9,0x98,0xc7,0xf5,0xa3,0x88,0xce,0x69,0xe5,0x35, +0x35,0x9c,0xdd,0x9a,0xc8,0x79,0x89,0xda,0x57,0x85,0x40,0x5f, +0x2b,0xc7,0xbc,0x10,0xa3,0xd5,0x3a,0x2f,0x8a,0xf1,0x74,0x01, +0x12,0x8e,0x8f,0xf1,0x17,0xd1,0xc8,0x52,0x03,0x49,0xf2,0x43, +0xb5,0xcc,0x0b,0x00,0xb9,0x50,0x43,0x34,0xd6,0x3e,0xf2,0xbc, +0xd1,0xc5,0x17,0x16,0xd1,0x00,0x7b,0x11,0x5c,0x5c,0x53,0x3d, +0x80,0xdf,0xbe,0x14,0xdb,0x80,0xb3,0x71,0x35,0x2e,0xae,0x03, +0xca,0xdc,0x89,0xfc,0x8b,0xa7,0xfd,0xb1,0x77,0xc7,0xc6,0x9c, +0xd2,0xe8,0x0e,0x52,0x47,0x58,0x51,0x6a,0x30,0x80,0x9a,0xca, +0xa3,0x05,0xb4,0x7d,0x9c,0xbd,0x3d,0x3e,0x76,0xd9,0xa1,0xa1, +0x98,0x31,0xd4,0x22,0xc4,0x1a,0x6e,0x21,0xe6,0x0e,0x65,0x61, +0xfa,0xac,0xc6,0xa0,0x89,0x57,0x32,0x82,0x11,0x2a,0x62,0xb7, +0xe8,0xa5,0x15,0x38,0x15,0xd8,0x6c,0xb1,0x90,0x58,0x6c,0x21, +0x86,0x62,0x12,0x6e,0x94,0x48,0xa6,0xb6,0x74,0x3d,0x24,0x09, +0x79,0x86,0xe0,0xce,0x45,0x73,0x12,0x2e,0x92,0xec,0xc8,0x96, +0x64,0x99,0x4a,0xf1,0x7a,0xfc,0xba,0x68,0x99,0x78,0x28,0xff, +0x96,0xff,0xf4,0x08,0xbb,0x59,0x7b,0x9b,0x25,0x37,0xc6,0xeb, +0x05,0x3a,0x52,0x1c,0xf4,0x66,0xb0,0xf2,0x76,0xb3,0x5b,0x7d, +0x21,0x5d,0x95,0x7b,0xb6,0xda,0x39,0xc9,0x9d,0xd0,0xd1,0x0b, +0xe6,0xd8,0xca,0xc8,0xee,0x34,0x59,0xf0,0x48,0xc9,0xc4,0x2d, +0x17,0xa7,0xd2,0x04,0xd6,0x2b,0x9d,0xad,0x18,0x67,0xf7,0x81, +0x92,0x68,0xc9,0xf5,0x8d,0x93,0x3e,0xc6,0x5f,0x54,0xb4,0x48, +0x1b,0xe9,0x25,0xd5,0x26,0x97,0xdb,0x10,0x8c,0x73,0xab,0xff, +0xea,0xba,0xac,0xe2,0xb2,0x4c,0xd6,0xed,0x95,0xe1,0x34,0xb7, +0x36,0x18,0xc9,0x3a,0x17,0xee,0x73,0x3a,0xb0,0x1c,0xbd,0xa2, +0xf9,0x05,0xbd,0x9e,0x21,0x7e,0xd9,0xe6,0xf6,0x6d,0x3f,0xd9, +0x85,0xa4,0x64,0x35,0xce,0x82,0x34,0x1e,0xb3,0x79,0xc3,0x18, +0x47,0xb4,0xee,0x37,0xbd,0xc4,0x59,0x1a,0x90,0x3e,0x63,0x11, +0x08,0x54,0x5b,0x55,0xcc,0x56,0xed,0xeb,0x77,0x4c,0xb0,0x06, +0x2e,0xf6,0xc6,0x04,0x33,0xd7,0x5a,0xf6,0x02,0x94,0xc8,0xcb, +0x86,0x4a,0x60,0x8e,0xb2,0xec,0x71,0xc0,0x9e,0xb9,0xe8,0x10, +0xc4,0x5a,0x6d,0x6a,0x43,0x29,0x79,0x3d,0xe5,0x56,0x77,0x3a, +0x53,0x7d,0x9a,0x22,0xe9,0xa8,0x84,0x48,0xde,0x8e,0xf4,0x34, +0x06,0x22,0x3e,0x95,0x24,0xe0,0x2c,0xa6,0xe3,0x29,0xfb,0x34, +0x52,0x92,0x8e,0x89,0x3e,0x1f,0x64,0xd9,0x3d,0x36,0xd3,0xce, +0x67,0x38,0xd6,0xd5,0xf8,0x84,0xce,0x8d,0x01,0x35,0xd3,0xd1, +0x00,0x0a,0x3f,0x1e,0x63,0x0a,0x3d,0xfa,0xe3,0x59,0x0b,0xd0, +0xd5,0x62,0x57,0x3b,0xd9,0xaf,0x95,0x04,0x56,0x31,0x92,0x97, +0xbb,0xdd,0x14,0x70,0x6e,0xe9,0xf7,0x4f,0x9a,0xe4,0x4c,0x81, +0xb6,0x9c,0xe4,0xf3,0x64,0x8f,0xa0,0x40,0x0a,0x4b,0xf7,0x62, +0xa6,0xf6,0xeb,0x56,0x71,0x0d,0x98,0xd9,0x76,0xcc,0xde,0xdd, +0xb3,0x37,0xf5,0x78,0x6f,0x5a,0xea,0x46,0x68,0x38,0x74,0x51, +0x1c,0x1f,0x2f,0x66,0x13,0xfa,0x02,0xda,0x24,0x0e,0x0a,0xfa, +0x3a,0x1e,0x20,0x25,0x6e,0xb4,0x38,0xf3,0xd9,0x06,0x4f,0xf6, +0x1e,0x3c,0x6a,0x23,0xa7,0x45,0xeb,0x42,0xa9,0xfa,0x0c,0x5b, +0xe2,0xdd,0x67,0xd3,0x21,0x14,0x82,0x6c,0xe3,0xcb,0x5d,0x21, +0x5a,0x3e,0x50,0x34,0xae,0xb1,0x25,0x00,0xd7,0x4c,0x92,0xc6, +0xba,0x0f,0x7e,0x7e,0x0d,0xfd,0x29,0x5e,0x03,0x37,0x86,0x64, +0x44,0x75,0x01,0x73,0x0f,0x3f,0xa7,0x3c,0x10,0xf3,0x73,0x54, +0x84,0xa3,0xee,0x63,0xf2,0xb5,0x26,0x49,0x89,0x9e,0xf4,0x79, +0x6f,0x32,0x9a,0x28,0x3b,0x13,0x65,0x48,0x32,0x15,0xb4,0x98, +0xc3,0x0b,0xf2,0x67,0x79,0x4a,0xfc,0x44,0x33,0x91,0xf3,0x04, +0xb4,0x7a,0xe1,0x2d,0xf3,0xf8,0x9a,0xec,0xd7,0xf8,0x11,0x12, +0xcf,0x5e,0x58,0xcf,0x13,0x66,0x8c,0xbb,0xd4,0xfb,0x29,0xa1, +0xa3,0x02,0x2d,0x13,0x0c,0x62,0xa5,0x08,0xca,0xba,0xe5,0x3f, +0xcd,0xfd,0xd6,0xf7,0xee,0x3e,0x6b,0xf1,0xde,0x85,0x76,0xea, +0xbe,0x47,0x77,0x77,0x60,0x3c,0xde,0xc2,0x31,0x3b,0xad,0x04, +0x3b,0xa6,0xdf,0xb0,0x45,0xc1,0xb7,0x1a,0x5c,0xe1,0xf0,0xed, +0x40,0xb0,0x4b,0x8f,0xe6,0xfd,0x01,0x7a,0x34,0xcc,0x6f,0xa0, +0x47,0xe1,0xf0,0xed,0x40,0x83,0x1e,0xd9,0xf8,0x7b,0x28,0x91, +0xa9,0xbf,0x92,0x06,0xce,0xed,0xf3,0x9f,0x46,0xb9,0x96,0xa4, +0xd9,0x56,0x08,0xb9,0xed,0xff,0x47,0x01,0xe0,0xa7,0xca,0xcd, +0xa2,0xb7,0x3b,0x4e,0xf0,0xb1,0x32,0x01,0xc9,0x63,0xae,0x16, +0x2d,0x9c,0xe9,0xe2,0x74,0x3c,0xfb,0x62,0x9c,0x5b,0x4d,0x9a, +0x35,0x6e,0x5c,0x5e,0xad,0xe2,0x16,0x6b,0xa4,0xe6,0xda,0x16, +0xeb,0xcb,0x9d,0x0c,0x67,0xe6,0xc8,0x32,0xb1,0x11,0x0a,0xc9, +0xde,0xf5,0x2c,0xcc,0x4c,0xab,0x9d,0xd5,0x1f,0x99,0xa5,0xd3, +0xd7,0x34,0x42,0x94,0xd2,0x72,0x01,0x4b,0xd0,0x19,0x70,0xd3, +0xce,0xd9,0xfc,0x83,0x62,0x78,0xfc,0x8e,0x1a,0xfc,0xb0,0x31, +0xd3,0x69,0xbb,0xa2,0x6a,0xab,0x3e,0x48,0xa3,0xd8,0xa3,0x93, +0xe8,0x2a,0x3d,0x5c,0x27,0x69,0x60,0xdf,0xda,0x7b,0x26,0xcb, +0x24,0x6b,0x1d,0x8b,0x21,0xde,0x57,0x0f,0x95,0x96,0xb4,0xaf, +0x9a,0x67,0x6f,0x79,0xff,0x35,0xf2,0x2e,0x93,0x7c,0x1e,0x97, +0xca,0x1c,0x07,0xc7,0x39,0x00,0x9f,0x18,0x88,0x29,0x0e,0x75, +0x71,0xe9,0x70,0x7e,0x05,0x19,0xed,0xb0,0x9a,0x07,0xab,0x10, +0x2f,0x09,0xc1,0x5c,0x88,0xd6,0xae,0xb8,0x30,0xb9,0x67,0xab, +0x61,0xb5,0x8a,0x26,0x2b,0x0a,0x0a,0xd2,0x61,0x82,0xa9,0xa8, +0x78,0x7b,0xc3,0x5f,0xad,0xcc,0x26,0xa6,0x9a,0xc0,0xab,0x69, +0x54,0xd5,0x74,0x4a,0xa7,0x62,0x6a,0x8a,0x19,0x4c,0xed,0xf6, +0x82,0xb2,0x7a,0x54,0x95,0x76,0xe1,0x22,0xc2,0x7a,0x16,0x07, +0xcc,0xc9,0x3c,0xa9,0x60,0x26,0x05,0x26,0xc4,0x0d,0xb9,0x08, +0x51,0x90,0x9b,0xfd,0x2c,0xd2,0xf3,0x0d,0xca,0xc0,0x51,0xba, +0x46,0x4e,0x07,0x08,0x2d,0xfb,0x02,0x04,0xf9,0x4b,0x6b,0x17, +0x40,0xc5,0x7d,0x9d,0x0d,0xe7,0xde,0xbd,0x5d,0x4d,0x63,0x8c, +0x2a,0x61,0x15,0x54,0xa1,0x59,0xce,0x06,0xbb,0x1c,0x56,0x73, +0x65,0xd7,0xaf,0x31,0x06,0x72,0x1b,0x36,0x47,0x16,0x22,0x8b, +0xb2,0xc6,0xe5,0xde,0xac,0x2c,0x0e,0x7b,0xb3,0xfe,0x39,0xfa, +0x88,0x2a,0x4e,0xf9,0x80,0xe7,0x09,0x53,0x5e,0xbb,0x14,0x1e, +0xb9,0x27,0xb4,0xad,0x2b,0xf5,0xcd,0x31,0xf8,0xd8,0xe4,0xc9, +0x3c,0x2d,0x78,0x5e,0x45,0x07,0x9c,0xf4,0xd6,0xbe,0x99,0x49, +0xa1,0x79,0x7b,0x12,0x23,0xff,0x11,0x10,0xd3,0xe8,0x69,0x64, +0x21,0xdf,0x34,0x8e,0x94,0xd1,0x2c,0x21,0x74,0x93,0x18,0xd6, +0x45,0x46,0x50,0x7c,0x99,0xa0,0xfa,0x1c,0xfa,0xfd,0x6c,0xef, +0x8b,0x00,0x45,0x5b,0xb6,0x4e,0x2f,0xe3,0xce,0xab,0x3a,0x1b, +0x59,0x4d,0xb3,0x5d,0xf4,0xbd,0x9d,0xad,0xd9,0x93,0x30,0xd7, +0x34,0x92,0xe4,0xfa,0xb2,0x30,0x79,0x5a,0xe1,0x74,0xa4,0xd6, +0x7e,0x02,0x71,0x2d,0x5d,0x85,0x62,0xed,0xde,0x02,0x9f,0x74, +0x22,0xb0,0x79,0x78,0x8f,0xd9,0xec,0x74,0x97,0x91,0x24,0xd2, +0xe6,0xe2,0xc6,0xed,0x44,0x56,0xa3,0xf4,0x7e,0xa7,0x0e,0xd5, +0x52,0x68,0xd0,0xda,0xc1,0xd8,0xc1,0xc1,0x9e,0x7e,0x62,0x35, +0xe2,0x7f,0x96,0xbe,0x62,0x8e,0xb1,0x41,0x63,0x69,0x9d,0x2c, +0xe8,0x46,0x88,0x1c,0x9f,0x8b,0xa8,0xc8,0xae,0x48,0x8f,0x0f, +0x44,0x29,0x2d,0xd8,0xe0,0x5b,0x0a,0x88,0xf7,0xad,0xd7,0x29, +0x7a,0x7c,0xeb,0xb0,0xe8,0xd5,0x1d,0xef,0xa5,0xd7,0x29,0x65, +0x18,0x86,0x66,0xb3,0xd0,0x9d,0x5f,0xe5,0xd7,0xf7,0xe1,0xc4, +0x3b,0xaf,0x90,0xb9,0x0b,0x08,0xf1,0xce,0x2b,0xe4,0x87,0xcf, +0x5a,0xa4,0x10,0xcb,0x6e,0x04,0x4e,0xfc,0x45,0x45,0x37,0x4c, +0xd3,0xda,0x8a,0xaa,0xbb,0x8b,0x6a,0x2f,0x3d,0x5f,0x79,0x1d, +0x1f,0x23,0xd2,0x2a,0xa0,0x48,0x1f,0x63,0xd3,0x3a,0xa0,0x6b, +0xa3,0x3e,0x92,0x4a,0x01,0x41,0x81,0xba,0x4b,0x60,0x47,0x7b, +0xea,0xa0,0xf6,0x1e,0xff,0x79,0x5f,0x3d,0x2e,0x23,0x2a,0x13, +0x90,0x04,0xa7,0xc3,0x03,0x94,0x5d,0x9a,0xd0,0x91,0x0a,0x95, +0x32,0x74,0x90,0x82,0x5f,0x1e,0x44,0x41,0x8d,0x99,0x0f,0x53, +0xa0,0xc2,0x97,0x4e,0x99,0x97,0xfb,0x29,0x40,0x99,0xb1,0x16, +0xff,0x34,0xd4,0xd1,0x75,0xa0,0x62,0x42,0x7f,0xa6,0xa3,0x7d, +0x36,0x1d,0x6a,0x1c,0x6a,0x0c,0x40,0x34,0xfa,0x2c,0x50,0x40, +0xf5,0x72,0x8b,0x10,0x45,0xf2,0xfa,0x06,0xd7,0xcf,0x3d,0xdb, +0xe0,0x38,0x4f,0x9f,0x4a,0x38,0x89,0x1c,0x87,0x89,0x2c,0x21, +0x51,0x6f,0xd8,0x3e,0x5e,0x32,0xd9,0xd1,0xad,0xb9,0x53,0x43, +0x42,0x1a,0xd2,0xaf,0x50,0xac,0x0e,0x75,0xab,0xeb,0xa8,0x7f, +0xb6,0xc4,0xfd,0xa2,0xe3,0x7e,0xd9,0x8e,0xac,0x25,0x06,0xbb, +0x92,0x31,0xbd,0x0c,0x06,0x0b,0xb3,0x96,0xa2,0x24,0x88,0xa1, +0xc5,0x22,0xbd,0xc2,0x71,0xa1,0x3c,0x66,0xb4,0xe7,0x50,0x89, +0x8d,0x3c,0x92,0x47,0x7b,0x86,0xa0,0x16,0x0e,0xaa,0x25,0x84, +0x55,0x2c,0x73,0x72,0x0f,0x8a,0x16,0x3e,0xef,0xa2,0xf8,0x73, +0x86,0x31,0x53,0xa0,0x19,0xc8,0x36,0x69,0xfb,0x06,0x04,0xc4, +0x5b,0xb2,0x2a,0x77,0xa7,0x5a,0x0f,0x0a,0x99,0x4c,0xf6,0x4e, +0xc3,0xd2,0x18,0x47,0xb4,0x8d,0x20,0x2e,0x22,0x3f,0x59,0x2f, +0xba,0x14,0x08,0x4e,0xd1,0x90,0xee,0xea,0xd2,0x57,0xfb,0xdc, +0xb4,0x27,0x85,0x53,0x1b,0x28,0x18,0x4b,0x2e,0x4b,0x6d,0xb3, +0x8f,0xac,0x5d,0xd2,0xe3,0x63,0x02,0xe4,0x4d,0x52,0xcb,0x1a, +0x72,0xe6,0x3e,0x15,0x57,0x4c,0xe2,0x69,0x57,0x7e,0xcd,0x70, +0x27,0x99,0x73,0xed,0xf0,0x4a,0x5e,0xf6,0x5c,0x8c,0xe4,0x76, +0x99,0x9c,0x81,0x16,0x91,0xd9,0xfb,0x93,0x29,0xaa,0xab,0x11, +0xf5,0xa4,0xe0,0x4c,0xa3,0xb8,0x99,0x08,0xd5,0x53,0x49,0x9d, +0x68,0x71,0x72,0x61,0x77,0xc5,0x6c,0x9d,0xb4,0xe8,0xd0,0x83, +0xad,0x72,0x3a,0x04,0x9d,0x86,0x70,0x47,0x7d,0x01,0x68,0xa2, +0xa1,0x7a,0x17,0x87,0xb7,0xa4,0xad,0x83,0x20,0x43,0x81,0xbc, +0x11,0xc8,0xea,0x96,0x50,0x85,0xd3,0x4d,0x40,0x8e,0xdb,0x1a, +0x07,0x54,0x49,0x63,0xf3,0x79,0xe6,0x6e,0x3e,0xcf,0x04,0x80, +0xe0,0x43,0x23,0xc9,0xe1,0x49,0x58,0x26,0xd2,0x7a,0x7a,0x57, +0x86,0x43,0xac,0x85,0x46,0x64,0x88,0xb3,0xf6,0x3f,0x74,0x9c, +0x29,0x9f,0x6a,0x2d,0x6b,0x7b,0x62,0x00,0xe8,0x61,0x24,0x8d, +0x4c,0xd5,0xc9,0xb2,0x52,0x96,0x31,0x66,0xfd,0x07,0x6c,0x53, +0x68,0x54,0xa2,0x8b,0xe4,0xcf,0xe9,0x57,0xaa,0x38,0xcb,0xe6, +0x0e,0x8f,0x7a,0xda,0x2d,0xe3,0xe9,0x10,0x88,0x7b,0x96,0xdc, +0xb1,0xc6,0xb1,0xe1,0xda,0x98,0x4c,0x05,0xc2,0xbb,0x4c,0xd7, +0x78,0xe1,0x77,0x56,0x07,0x94,0x4e,0x66,0x4c,0x13,0x34,0xa9, +0x4e,0xd7,0x2f,0x50,0xae,0xa5,0x05,0x02,0x35,0xc1,0x74,0xca, +0xfb,0xd4,0x80,0xcc,0xbb,0x8c,0xab,0xf6,0x7c,0x71,0x75,0x38, +0x5f,0x02,0x23,0xb0,0x2a,0xf2,0x56,0x9f,0xfa,0xdb,0xa7,0x56, +0x07,0x3a,0x60,0xcc,0xe2,0x81,0xcf,0x56,0xcd,0xe4,0xe7,0x0c, +0x3e,0x9f,0xc6,0xe3,0xee,0x20,0x8c,0x9f,0xce,0xc6,0x83,0xb0, +0x0f,0x6a,0xb1,0x45,0xd2,0x7f,0xfc,0x87,0x4f,0xdb,0xea,0xdc, +0xdf,0x4f,0x87,0x40,0x0c,0x19,0xd9,0xf1,0xa6,0x38,0xa8,0x32, +0x7a,0x2f,0x17,0xe8,0xe1,0x8c,0x16,0x2d,0x00,0x80,0xab,0x58, +0xfc,0x33,0x30,0x0b,0x42,0xdd,0xef,0x66,0x35,0x6f,0x78,0xf0, +0xa6,0x42,0xcb,0xc1,0xbe,0x1e,0x5e,0x10,0x5b,0x6f,0x56,0xb7, +0x69,0xb3,0x10,0x5b,0x09,0x02,0xd8,0xf2,0x9f,0xe6,0xfe,0x45, +0x63,0xfb,0xd9,0x1d,0xdf,0x03,0x27,0xab,0x25,0x16,0x4e,0x8f, +0xb4,0xa5,0x02,0x99,0xcb,0x42,0xa1,0x24,0x42,0x59,0x63,0x52, +0xec,0x58,0x99,0xc0,0xaa,0x43,0x0a,0xba,0x51,0x6e,0x4e,0x8f, +0x90,0x4d,0x2d,0x07,0x1e,0x8d,0xb3,0x02,0x35,0xe6,0xf1,0xd0, +0x9b,0x8e,0xf6,0x70,0xce,0x1e,0x7e,0x1d,0xf6,0x14,0x36,0x7b, +0xc8,0xdc,0xf2,0x12,0x17,0x0c,0xb8,0x12,0x2b,0xf8,0x3c,0x73, +0x2b,0x7f,0x75,0x85,0xcb,0x78,0xde,0xae,0x27,0x53,0x8a,0x7d, +0x86,0x7b,0x55,0xbe,0xa9,0xfb,0x91,0xda,0x98,0xe8,0x0b,0x8a, +0x18,0xe8,0x08,0x1a,0x0e,0xdf,0xcc,0x6b,0x15,0x31,0xaf,0x45, +0x91,0xe4,0x29,0x4c,0x4a,0x84,0xa9,0x70,0x4f,0x67,0xe8,0x95, +0x4b,0x4a,0xd6,0x8e,0xe4,0x90,0x81,0xbe,0xdc,0xf8,0x73,0x5c, +0x5e,0x2f,0x04,0x94,0x56,0xa3,0x2f,0x86,0x05,0x9a,0x18,0xe3, +0x06,0x17,0x7f,0x4d,0x98,0xa6,0x69,0x54,0xa3,0xd5,0x8e,0x15, +0x35,0x80,0xa8,0x4e,0x23,0x72,0x5e,0x23,0x5c,0xa7,0x06,0x61, +0xde,0x06,0x3e,0x97,0xcd,0x93,0x81,0xed,0xbc,0xc9,0x22,0x95, +0x1d,0x58,0xea,0xd7,0x77,0x2f,0xc4,0x46,0x54,0xea,0x14,0xa8, +0x39,0x20,0x90,0x0b,0x07,0x3e,0xae,0xe4,0x3a,0x4f,0xa6,0xea, +0x78,0xda,0x39,0xd4,0x51,0x07,0xd4,0x0c,0x65,0x9d,0x1c,0xb5, +0x9f,0x7c,0xd1,0x09,0xb2,0x3d,0xfc,0xe8,0x49,0x50,0x3f,0x88, +0xdb,0xbc,0x7a,0xcf,0x94,0x48,0xa7,0x94,0x55,0x48,0x41,0x3f, +0x9e,0xbc,0x85,0x1c,0x75,0x28,0xdf,0xe5,0xe0,0xe0,0xfb,0x30, +0x15,0xc4,0x9e,0x10,0xc3,0xd0,0xff,0x65,0x7b,0x50,0x56,0x3a, +0xc8,0x83,0xc2,0x25,0xba,0x03,0x63,0x4b,0xaf,0xfa,0x62,0xd6, +0xab,0x68,0x24,0xe9,0x55,0x4f,0xf1,0x13,0xbf,0xde,0x43,0xe4, +0x7b,0x8e,0x7c,0xff,0x14,0x3f,0x69,0x88,0x11,0x55,0xc4,0x08, +0xd1,0x6f,0x46,0x45,0x16,0x4b,0x58,0x8d,0x0b,0x62,0x81,0x2c, +0x4a,0xb3,0xe0,0x82,0x0e,0xc6,0x69,0x1d,0xae,0x72,0x5d,0x4c, +0xc5,0xa6,0x57,0xc1,0x68,0x5f,0x31,0x6b,0x46,0xe9,0x17,0x8e, +0x10,0x8d,0x02,0x0e,0x4e,0xd2,0x4e,0x67,0xaa,0x79,0xf0,0x16, +0x6a,0x8e,0xe3,0x1b,0x14,0xae,0x0a,0xed,0x44,0x03,0xaa,0xde, +0x68,0x03,0xe4,0xa5,0x87,0xb3,0x41,0xa6,0xe8,0xad,0xd8,0x81, +0xd8,0x30,0x83,0x3e,0x8c,0x04,0xa7,0xf9,0xdb,0x8f,0x5f,0xb8, +0xdf,0x49,0x91,0x72,0x47,0x47,0x1a,0x16,0x40,0xf9,0x97,0xd7, +0x7c,0xdb,0x4c,0xa2,0xf4,0x3e,0x91,0xba,0xb3,0xe8,0x65,0xf9, +0x22,0xc9,0x3d,0x73,0x96,0x57,0xf7,0x43,0x72,0x62,0x56,0x0f, +0x70,0x1b,0x53,0xcc,0xeb,0xd0,0x9b,0xd7,0x46,0x2f,0xa1,0xe2, +0x7b,0x5c,0xf8,0x1e,0x8d,0x63,0x97,0x32,0xa9,0x2f,0x53,0xde, +0x2d,0xff,0x51,0xaf,0x44,0xed,0xc1,0xd1,0x52,0x05,0x17,0xc9, +0xe1,0x8d,0x0e,0x02,0x7a,0xf8,0x36,0x3d,0xe3,0xdc,0xca,0x8f, +0x83,0x4a,0x07,0x34,0xc4,0xef,0xa4,0x74,0xd8,0xdc,0x70,0x06, +0xdc,0xb2,0xdd,0x64,0x4a,0x5e,0x28,0xb2,0x0e,0x9f,0xc9,0x50, +0xe6,0x1a,0xaf,0x63,0x82,0x52,0x00,0xaa,0x45,0xd4,0x19,0x9c, +0x42,0xa7,0x8f,0xab,0xa8,0x8b,0x1f,0xa0,0xac,0xe3,0x18,0x90, +0xc4,0x6b,0xf8,0xf3,0xed,0x10,0x8d,0x2e,0x50,0x29,0xf0,0x1f, +0x34,0x37,0x49,0xc3,0x08,0x7a,0xd3,0xd9,0xb1,0x65,0xb1,0xe6, +0x7d,0xcb,0x68,0xe3,0x9a,0xe2,0x51,0xfb,0x06,0x3a,0x9e,0x5e, +0x43,0x95,0x91,0x9e,0xeb,0x40,0x5c,0x3f,0x05,0x7a,0x30,0x08, +0x54,0x41,0x10,0x68,0xea,0x44,0x92,0xe4,0xe8,0xba,0x8b,0xc4, +0x11,0x85,0xa0,0x7c,0x63,0xd4,0xa9,0x9f,0x76,0x06,0x01,0x10, +0x2b,0xc3,0x27,0x3e,0xc3,0x04,0xa3,0x6f,0x87,0xa7,0x11,0x14, +0xd7,0x95,0x6f,0xdb,0xee,0x4e,0xb0,0xb7,0x5b,0xaa,0x02,0x0e, +0xab,0xac,0x0d,0xe1,0x63,0xfa,0x48,0x2b,0xdb,0x93,0x90,0xe5, +0xf4,0xfc,0x72,0x83,0xf7,0x3d,0xac,0x2a,0x3c,0x7d,0x3a,0x30, +0x07,0xe4,0x9c,0x6f,0x6a,0xa7,0xff,0xcb,0x70,0x2c,0x2b,0x16, +0xfa,0xfc,0xd1,0x1d,0x4c,0x3b,0x32,0x2a,0x80,0xb9,0xc2,0xe4, +0x9e,0x67,0x57,0xeb,0x72,0x8a,0x44,0x5a,0x18,0x53,0x40,0x07, +0x3f,0x56,0x4c,0x5c,0xa1,0x10,0x54,0x26,0x06,0x58,0x32,0x8d, +0xe0,0xc7,0xa6,0x03,0xa9,0x20,0xde,0x58,0x87,0xf7,0xe8,0xca, +0x6d,0x0e,0xf2,0xfc,0xed,0xd0,0xca,0x5b,0x2e,0x92,0xeb,0x69, +0x64,0x1e,0xec,0xfc,0x16,0x28,0x8a,0x8b,0x02,0x1d,0x89,0x61, +0xa3,0x51,0x83,0xdf,0xdb,0xde,0xe2,0x9a,0x2c,0xa8,0x5a,0x2a, +0x72,0xbd,0x53,0x91,0xeb,0x9d,0x8a,0x5c,0xef,0x54,0xe4,0xba, +0xa5,0x22,0xd7,0xad,0x15,0xb9,0x3e,0x58,0x11,0xdd,0x8e,0xd7, +0xcd,0x16,0x6a,0x36,0x59,0x20,0x0e,0x8b,0xa7,0x6c,0x30,0xd9, +0xbb,0xae,0x47,0xff,0x4b,0xed,0x79,0x3a,0x53,0x7b,0x59,0x38, +0x76,0x2b,0xbb,0x1d,0xc5,0xba,0x8e,0xb0,0xd3,0x55,0xb8,0x7b, +0xb2,0x81,0x2c,0xaf,0xb9,0xda,0xc6,0x5f,0x6e,0x1b,0x5c,0x1c, +0x32,0xe5,0xf6,0x32,0x91,0xf8,0x0e,0xa3,0x2d,0xfe,0xf1,0xb0, +0x1f,0x85,0xb4,0x8e,0xc0,0x0e,0x1d,0xd2,0xca,0x00,0x7b,0x0f, +0xac,0x20,0xaf,0x2e,0x3d,0xea,0x33,0x10,0x89,0x37,0x00,0x84, +0xe2,0x28,0x9a,0x5d,0xf0,0x17,0xba,0x68,0x01,0x7e,0xe2,0x6a, +0x13,0xfe,0x78,0xb2,0xac,0x50,0x2d,0x13,0xec,0xc1,0x1c,0x66, +0x54,0x49,0x89,0x59,0x68,0x66,0xd6,0xfd,0x0d,0x99,0x88,0x71, +0x77,0x77,0x40,0x2c,0x65,0xd9,0xf2,0x9f,0x87,0x2c,0x3c,0xdb, +0x47,0x70,0x83,0x42,0x32,0xa4,0x65,0x95,0x2b,0x53,0x66,0x2e, +0xfc,0x9f,0xb3,0x20,0xb4,0x8b,0x34,0x63,0x33,0xda,0x80,0xe3, +0x41,0x43,0xdb,0xe8,0x2c,0xd3,0x9a,0x16,0x1f,0xb4,0x18,0x78, +0x98,0xd9,0x07,0xef,0xbf,0xf0,0x88,0x2b,0x6d,0x40,0x9a,0xb7, +0xf8,0x28,0x27,0x27,0xb1,0x31,0x18,0xbf,0x63,0xa5,0x6c,0xda, +0xec,0xca,0x53,0x67,0xb0,0xad,0x08,0xef,0xdb,0x9a,0x54,0xab, +0x0f,0xc8,0x83,0x5a,0x2c,0x5d,0x8b,0x33,0x1d,0x08,0x62,0xd9, +0x00,0x11,0x46,0x78,0xf8,0x8b,0x4e,0x9c,0x25,0xdb,0x92,0x1a, +0x99,0x08,0x39,0xa2,0x05,0x05,0x94,0x77,0x00,0x15,0x9a,0x58, +0x80,0xe4,0x16,0xd9,0xd9,0xee,0xdc,0x51,0x6f,0x14,0x1f,0x03, +0x6d,0x55,0xdf,0x7a,0x90,0x23,0xbb,0xdf,0x56,0xda,0x2e,0xb5, +0xf6,0xb5,0x7b,0x8d,0x4b,0xd8,0xe9,0x99,0x80,0x5e,0x92,0xce, +0x2f,0xd0,0xa0,0x84,0xec,0x01,0xf0,0x78,0x8c,0x6f,0x24,0x5b, +0x5a,0x91,0x75,0xb2,0x2b,0x16,0x10,0x90,0x48,0x17,0x10,0x23, +0xf1,0xea,0x4e,0x25,0x6b,0x70,0xbf,0x19,0x89,0x04,0xdc,0xaa, +0x8f,0x5f,0xd9,0x9d,0x34,0x9a,0xc3,0x2a,0x91,0x04,0x7b,0xb8, +0x52,0xa4,0xf0,0x6e,0xf5,0xe7,0x9f,0xd4,0xf9,0x9a,0x7c,0xb1, +0x3b,0x20,0x20,0x8b,0xcb,0xa4,0x79,0xee,0x78,0x68,0xbc,0x8d, +0x3c,0x95,0xcb,0x03,0x9d,0xb9,0x48,0xf9,0xa4,0x91,0xdd,0xd0, +0x8a,0x64,0xb5,0x4a,0x37,0x45,0x0a,0x40,0xbd,0x5e,0xcf,0x13, +0x37,0xa0,0x0c,0x7f,0x99,0x27,0xf1,0x45,0x44,0x9e,0xa0,0x2e, +0xd3,0x32,0x1a,0xf4,0xfb,0x42,0x17,0x7b,0xcf,0x71,0x3e,0xf9, +0xa3,0x01,0x18,0x05,0xef,0xab,0x2d,0x55,0xc6,0xa5,0x8b,0x37, +0xe5,0x68,0x02,0xac,0xc3,0xfe,0xb8,0x98,0x92,0x5f,0x53,0xfb, +0x28,0x52,0x61,0xfc,0xd5,0x22,0xc2,0xd9,0xb7,0xfa,0x6b,0xa7, +0x29,0x5d,0x4c,0xb1,0xb4,0xbb,0x6b,0xcb,0xc9,0x7c,0xd1,0x19, +0x57,0x78,0x39,0x57,0x6f,0xfa,0x63,0x52,0x07,0x35,0x87,0x96, +0x8c,0x9a,0xfb,0xe6,0x24,0x35,0x2b,0xac,0x55,0x83,0x4c,0x85, +0x8f,0xb6,0xdc,0xba,0xa9,0xac,0x9d,0x49,0x63,0xcc,0xa3,0x53, +0xd1,0x08,0xda,0xf3,0xda,0x10,0x20,0xd3,0x67,0xd4,0xb8,0x2d, +0x93,0x8d,0xd5,0xf2,0x8f,0x66,0x2d,0xb9,0x8d,0x0c,0x5e,0xad, +0xd3,0x77,0x57,0xed,0x5b,0x83,0x9c,0xb4,0x73,0x11,0xdb,0x8a, +0x96,0x5b,0x77,0xee,0x16,0x7c,0xa5,0xcd,0x3c,0xf5,0x18,0x92, +0x91,0x14,0x80,0x92,0x68,0xdf,0xb8,0xe1,0x5d,0x31,0x94,0x68, +0x0a,0xea,0x5b,0xa1,0x8c,0xff,0x03,0xf7,0xb7,0x38,0xd3,0x56, +0xfe,0x75,0xae,0x7c,0x54,0xb6,0x18,0x54,0x0d,0x48,0xc3,0x08, +0x7e,0x94,0xb9,0x95,0x11,0x90,0xf4,0xc1,0x9b,0x61,0x23,0xe5, +0x34,0x72,0x7e,0x95,0xe7,0x02,0xd8,0x4e,0xdb,0x25,0xce,0x3e, +0x58,0x17,0x47,0x4e,0xeb,0xea,0x47,0x94,0x8e,0xd2,0x0e,0xb4, +0x78,0xb2,0xe1,0xe9,0x8b,0xee,0x69,0x42,0x6e,0xdc,0x5d,0x41, +0x50,0xb9,0xe5,0x81,0x51,0xb6,0x35,0x7a,0xe7,0x2d,0x3e,0xb8, +0x09,0x05,0xc8,0xe5,0x37,0x5a,0xaf,0xa6,0xca,0x8a,0x1d,0x81, +0xb7,0x6a,0x4f,0xf2,0x56,0x46,0x22,0x30,0x5f,0xce,0xa4,0xb1, +0x7b,0x28,0xb0,0xcc,0x68,0xa0,0x1a,0x00,0xaa,0xdb,0x18,0xed, +0xd7,0xee,0x60,0x0f,0x3d,0x3e,0x5d,0x6f,0x05,0xc1,0x61,0xce, +0x36,0x38,0x8c,0x37,0x70,0x16,0x9b,0x41,0x32,0xe7,0xab,0xec, +0xaa,0xdd,0x1e,0x99,0x52,0x9a,0x4a,0x87,0x31,0x42,0x4d,0xd7, +0x29,0xde,0xcb,0x8b,0x97,0xf2,0x06,0x2f,0x4e,0xee,0x0b,0xda, +0xb2,0xba,0xe0,0xe9,0xd1,0x56,0x4d,0xe8,0x0c,0x1a,0xbf,0x26, +0xfd,0xe9,0xe9,0x10,0x4f,0xa1,0x29,0x30,0xc0,0x40,0xab,0x8a, +0x71,0xe9,0x28,0x18,0x74,0x67,0x56,0x6f,0x68,0xb6,0x6e,0x06, +0x29,0x77,0xd1,0xd0,0xf1,0xba,0xfa,0x3b,0xc6,0x31,0x9f,0x77, +0xb9,0x90,0xd0,0x36,0xfb,0x68,0x73,0x15,0xf7,0x82,0x8c,0xa3, +0x79,0xeb,0x47,0xeb,0x24,0x17,0x3b,0x6a,0x06,0x22,0xc2,0x53, +0x1e,0x52,0x41,0x3c,0xda,0x53,0x87,0x4f,0x32,0x7d,0xd3,0xb6, +0x54,0xb0,0xc4,0x84,0x59,0x0e,0x26,0xf4,0xdd,0xc4,0x1a,0x13, +0x6b,0xbd,0x33,0xfd,0x30,0xfd,0x8d,0x5c,0x5a,0x93,0x67,0x3d, +0xed,0x16,0xdb,0xd7,0xfe,0xb1,0x7b,0xd2,0x69,0x33,0xff,0x09, +0xa8,0x55,0xe5,0x6a,0xa3,0x07,0x4c,0x22,0x57,0x90,0x82,0x9b, +0x2b,0x50,0x36,0xbc,0x07,0xb4,0x24,0x6e,0x76,0xe9,0x85,0x72, +0xf2,0x39,0xcc,0x51,0x4f,0xfa,0xfd,0x29,0x7b,0xfa,0xfb,0x20, +0x5d,0x48,0x11,0xd8,0x76,0xd0,0xf5,0xf1,0x56,0x30,0xb9,0x2d, +0x89,0xfd,0xad,0x65,0xa3,0x4b,0x4e,0x9e,0x3d,0xed,0x93,0x5c, +0x7e,0xff,0x43,0x7a,0x85,0xbb,0x47,0x9b,0x22,0x37,0x05,0x9e, +0xa2,0x83,0x9d,0x45,0x87,0x12,0x27,0x17,0x1f,0x7a,0xb1,0x7b, +0x36,0x46,0xb5,0xef,0xb9,0xde,0xdf,0x1a,0x67,0xb6,0x4d,0x3e, +0xa8,0xcb,0x72,0x98,0x73,0xcb,0x7f,0xee,0x57,0xd0,0x1c,0xe8, +0x5d,0x4e,0x35,0xee,0xe7,0xa9,0xf4,0xfb,0x8a,0x6d,0xf2,0x54, +0x0e,0xff,0xd7,0x23,0x33,0x63,0xa4,0xc5,0xf7,0x74,0x27,0x16, +0xd2,0x8e,0x8f,0x1f,0xd1,0xc8,0xa9,0x47,0xd0,0x2a,0x18,0x6b, +0x35,0xaa,0xc2,0xb3,0xd7,0x45,0x76,0x39,0xf6,0xaf,0x23,0xf8, +0xbf,0x0a,0x06,0xe6,0x5d,0xea,0xeb,0x20,0xc4,0xa3,0x96,0xbd, +0xed,0x78,0x8d,0x47,0x39,0xb4,0xea,0xe6,0xac,0x7e,0x70,0x72, +0xad,0x46,0xdb,0x41,0xd2,0x7d,0x12,0x4c,0xed,0x67,0xc4,0xab, +0x5e,0x8c,0x57,0x41,0xd6,0x80,0x0b,0x75,0x7e,0x2e,0x53,0x47, +0xdd,0x5f,0x2c,0x0c,0x3d,0x56,0xc9,0xe9,0xbf,0xa8,0xa2,0xb0, +0x10,0x09,0xbb,0x9f,0x7b,0x87,0xd5,0x5b,0x02,0x7a,0xb8,0x72, +0xcb,0x38,0xb7,0xf2,0xe3,0xcf,0xba,0x26,0x67,0x89,0x81,0x19, +0xe0,0xdf,0xa7,0xed,0x2b,0x4a,0x88,0x57,0x03,0xbb,0x94,0x10, +0x36,0xd0,0x4d,0xd0,0x5f,0xda,0x46,0xbc,0xe7,0x53,0xb5,0xd9, +0xe4,0xfd,0x54,0xbc,0xc7,0xc5,0xde,0x7b,0xfb,0xc4,0x08,0x47, +0x4d,0xbc,0x5d,0xf8,0xff,0xd1,0xc0,0x26,0xef,0xc8,0xbd,0x6f, +0x5f,0xfc,0x15,0x6c,0x18,0x26,0xc1,0x21,0x38,0x8d,0x60,0x61, +0x65,0xb6,0x54,0x6c,0xbc,0xd6,0xf5,0x56,0x3d,0x26,0xb3,0xca, +0x0a,0x28,0xc6,0xf2,0xf5,0xf5,0x6b,0xae,0x33,0xe0,0x5a,0x04, +0x53,0x90,0x0a,0x74,0xc8,0x17,0x5f,0xad,0x4a,0xba,0xea,0x1c, +0xbe,0xe7,0x7c,0xff,0x82,0x44,0x4f,0x77,0x8e,0xb0,0xde,0x1b, +0x2d,0x07,0x18,0x40,0x97,0xf2,0x5a,0x07,0x34,0x1b,0x65,0xc4, +0xec,0x17,0xb2,0x06,0xce,0x19,0x18,0x20,0x39,0x74,0xcf,0xf2, +0x7d,0xb4,0x10,0x00,0xb2,0x15,0x04,0x27,0x91,0xb6,0x83,0x3a, +0x25,0x9a,0x5c,0x8e,0xea,0xe4,0x18,0x45,0xe3,0xa5,0x63,0xb4, +0x88,0xd6,0x90,0x58,0x17,0x5b,0x6a,0xcc,0x95,0x34,0xa7,0x8a, +0x17,0x81,0xc9,0xa2,0xaa,0xd4,0x92,0x4d,0x25,0xed,0xc9,0x4a, +0x02,0xc6,0xf7,0xcc,0x6f,0xcd,0x77,0x8f,0xbc,0x5c,0xb7,0x89, +0x1b,0x26,0xf8,0xc5,0x26,0x99,0x93,0x57,0x67,0x41,0xe6,0xa3, +0x20,0x19,0x14,0xc5,0xbb,0x5c,0xbb,0xbb,0x06,0xc9,0x99,0x72, +0xf4,0x43,0x9e,0xa0,0xe9,0x7b,0x1a,0x59,0xdf,0xf4,0x10,0x04, +0xfb,0xb4,0x43,0xe7,0xd0,0xe8,0xab,0x19,0xff,0xe8,0x8c,0xc2, +0x82,0x55,0x45,0x6e,0xed,0xd7,0x86,0x9a,0xd0,0xa9,0xa1,0x4c, +0xf7,0x39,0xf2,0xe2,0x88,0x91,0xc7,0xc7,0x08,0xc0,0xdf,0x3e, +0x83,0x4d,0x74,0xe2,0x94,0xcb,0xee,0x65,0x39,0xfa,0xad,0x24, +0xd7,0x8f,0xfc,0x79,0x77,0xf7,0xfd,0x8f,0x2f,0x9e,0x7f,0xf7, +0x8a,0x40,0xc9,0xa9,0x8a,0x82,0xa4,0x23,0x12,0x86,0xa4,0x4f, +0x7a,0xa5,0x86,0x92,0xc8,0x2f,0xb6,0xaf,0x3d,0x64,0xdf,0xdd, +0x79,0x67,0x39,0xce,0x58,0x32,0x79,0x99,0xa7,0x0b,0x4a,0xc5, +0x0f,0xdc,0x36,0x93,0xf1,0x65,0x5a,0xe2,0x0b,0x5a,0x58,0x0e, +0x7e,0xb1,0xcb,0x34,0x3b,0x8d,0xfd,0x29,0xcb,0x37,0x38,0x34, +0x1c,0xc7,0x8e,0x1b,0x61,0xcb,0x91,0x4e,0x13,0x81,0x46,0x39, +0xbf,0x50,0xae,0x5c,0x20,0xaf,0xd2,0xb5,0x9d,0x42,0xe7,0x17, +0xdf,0x90,0x63,0x02,0x82,0x60,0x1f,0x05,0xe8,0x9f,0xff,0xac, +0x19,0x19,0x84,0x8d,0x6c,0x3f,0x5d,0xcd,0x16,0xe9,0x75,0xba, +0xe0,0xfa,0x14,0x2a,0x64,0x78,0x84,0x40,0x3f,0xf0,0xd4,0x2f, +0xab,0xac,0xc3,0xf6,0xfb,0x20,0x04,0x2b,0x55,0x04,0xf9,0x2c, +0x3b,0xe9,0x2a,0x74,0x40,0xea,0xb4,0x31,0x95,0x8a,0xca,0x94, +0x7d,0xb7,0xe9,0xb1,0xba,0x35,0x8a,0xae,0x8d,0x58,0xc9,0xb7, +0x41,0x47,0xda,0x2a,0xa1,0x2c,0xa2,0x1d,0x86,0x31,0xd0,0x88, +0x0b,0x2c,0x61,0xd9,0x89,0xff,0x9f,0x6e,0xf5,0x4b,0x65,0x36, +0xaa,0x6f,0xe3,0xb7,0x59,0x0e,0x63,0xa9,0x54,0x90,0x77,0xd3, +0xd4,0xfb,0x4a,0x4e,0x42,0xba,0x36,0x99,0x06,0xd3,0xdd,0xb4, +0xb6,0x4c,0xcf,0xd7,0x0b,0x95,0x65,0x38,0x6d,0xa6,0xb0,0x2a, +0xa7,0x9f,0x18,0x70,0x2a,0x82,0x7e,0x4b,0x57,0xb5,0xcf,0xbd, +0x84,0x5c,0x93,0xbb,0xc8,0xe5,0x22,0x80,0x3e,0x9d,0x45,0x99, +0x4e,0x1f,0xeb,0xcf,0x70,0xa2,0x3f,0xa7,0x23,0x5d,0x4a,0x61, +0x17,0xc1,0x59,0xd8,0xb9,0x92,0x81,0xf0,0x5b,0xb8,0x5c,0x04, +0xca,0x10,0x1a,0x70,0x4a,0xef,0x30,0x78,0xbe,0xbd,0x39,0x3e, +0xde,0x30,0xc0,0x98,0x50,0xc2,0xba,0x18,0x18,0xf9,0x0a,0x05, +0x45,0x03,0xf9,0x1b,0x13,0x5b,0xa0,0x86,0x01,0xaa,0x1d,0x3a, +0x67,0xc5,0x6b,0x1c,0x9c,0x57,0xd8,0x10,0xf8,0xcc,0x01,0x17, +0xc9,0x52,0x78,0x89,0x5c,0xde,0xc5,0xa8,0x62,0xf7,0x63,0xd4, +0x10,0x06,0x23,0xbe,0xe4,0xb9,0x97,0x4a,0xac,0x8a,0x21,0xe3, +0xee,0xee,0x76,0x7b,0x88,0x02,0x82,0xd6,0x45,0x10,0xb4,0xd5, +0x6d,0x5e,0xc6,0xb3,0x64,0xd5,0x04,0x5f,0x61,0xa4,0x8d,0x98, +0x3a,0x7c,0x13,0x4a,0x0e,0x2c,0x1a,0x08,0x47,0xa0,0x97,0xe9, +0x7a,0x07,0x8e,0x47,0x26,0x0d,0x06,0xaa,0x5f,0x9c,0xae,0x9b, +0x40,0x98,0x44,0x40,0x34,0x35,0xf3,0x30,0x49,0xda,0xbb,0x7c, +0x8d,0x09,0x55,0x78,0x7a,0x6e,0x47,0x94,0xd9,0x26,0xa4,0xc7, +0x76,0x04,0x27,0x19,0x10,0xf6,0x82,0xc0,0x50,0xd2,0x11,0x42, +0xee,0x6e,0x98,0xe2,0x1c,0x20,0x2f,0x17,0xf0,0x1c,0x45,0x9a, +0x8d,0x51,0x4f,0x70,0x4e,0x32,0x3e,0xc6,0x8c,0x35,0x2e,0x3e, +0xff,0xbd,0xd8,0xbd,0x38,0x45,0x0e,0x63,0xe4,0x33,0xe0,0xf4, +0x3d,0x36,0x5e,0x64,0xf0,0xa5,0xed,0x17,0xdf,0xfd,0xf5,0xe8, +0x9b,0x67,0x2f,0x5e,0x3e,0xff,0x1a,0x5d,0xce,0x2c,0xf0,0x9d, +0xa8,0x20,0xe4,0xb7,0x5f,0x11,0xe5,0x64,0xd1,0x63,0x2d,0xf2, +0xda,0xf2,0xe5,0xc5,0x0b,0xdb,0x32,0x93,0x17,0xea,0x41,0xfd, +0x50,0xc3,0xa2,0x40,0x1b,0x8e,0x6e,0x97,0x0e,0x6d,0x8e,0x8f, +0x15,0x99,0xbe,0xf4,0xe5,0xc4,0x9e,0x28,0x6f,0xd1,0xff,0x72, +0x48,0xf5,0xc0,0x12,0x42,0x10,0xb4,0xb3,0x55,0x76,0x83,0x7f, +0xd9,0x3e,0x35,0x44,0xbf,0xe7,0x84,0x02,0x46,0x34,0x79,0xf8, +0x74,0x68,0xde,0x65,0xcb,0x33,0xa4,0x1c,0x06,0x0a,0xca,0x27, +0x1f,0x53,0x67,0x55,0x8d,0x18,0x83,0xa9,0x42,0xf2,0x08,0xef, +0x35,0x68,0x0f,0x54,0xbe,0xa2,0x1d,0xdd,0x52,0xb9,0x9e,0xc2, +0xec,0x2a,0x2b,0x5f,0x38,0xea,0x8c,0xd0,0x85,0x95,0xde,0x4c, +0x5d,0x8e,0x2d,0xb4,0x0f,0x84,0x85,0x34,0xcb,0xb5,0x6c,0xd1, +0xe4,0xeb,0xba,0x14,0x3d,0x51,0xe9,0xe8,0x35,0xb2,0x3d,0x01, +0x9d,0x48,0x8e,0xac,0xab,0xdb,0x54,0x44,0xb0,0x85,0x65,0x9f, +0xf2,0x1c,0x8e,0x7a,0x2d,0x65,0x45,0x5e,0xda,0xad,0x66,0x84, +0x08,0x53,0xb0,0xfa,0x5b,0x6e,0x26,0xd9,0x48,0xd6,0xb3,0xf9, +0xaa,0xbd,0xc4,0x40,0x56,0x68,0x2b,0x76,0x10,0x44,0xb6,0xbe, +0x23,0x47,0xd0,0x2a,0xe2,0x41,0x53,0xd1,0x62,0x9d,0x98,0xab, +0xec,0x3a,0xcd,0x6c,0xe2,0x56,0xca,0x96,0xc9,0x60,0x74,0x37, +0x6e,0x0c,0x60,0xd3,0x7b,0xe8,0x42,0x98,0x6b,0x7c,0x30,0x2d, +0x9b,0x3c,0x5b,0x69,0xa2,0x14,0x5a,0x37,0x65,0xac,0x4a,0xb8, +0x97,0xef,0x98,0xfa,0xaf,0xbe,0xff,0xee,0xa7,0x57,0xcf,0xb0, +0x27,0x3f,0x0f,0x3d,0x5a,0xfb,0x3d,0xf7,0xc4,0xcb,0xef,0x86, +0x32,0x00,0x5f,0x18,0x1c,0xf4,0x75,0x78,0xd0,0x87,0x88,0xef, +0xff,0x3a,0x54,0xe0,0xf4,0x4d,0x51,0x83,0xbe,0x15,0x07,0x01, +0x4f,0xfc,0xf0,0x42,0x46,0xfc,0xf0,0xc2,0x13,0x3f,0xfd,0xeb, +0x8f,0xaf,0x06,0x6f,0x14,0x62,0x19,0xe2,0x68,0x3b,0x72,0xe8, +0x6d,0xc5,0x37,0x7f,0xff,0xee,0xab,0x57,0x2f,0xbe,0xff,0x2e, +0xba,0x8d,0x67,0x85,0x4c,0x83,0x2f,0x4f,0xc4,0xf3,0x4c,0x87, +0xe1,0x13,0x22,0x0a,0x3c,0x19,0xe5,0x88,0x02,0xcf,0x47,0xa1, +0xfe,0x3a,0xa2,0x8c,0x65,0xc4,0xd0,0x8a,0x19,0xe2,0x0b,0x1c, +0xe9,0x4a,0xc6,0xe0,0x27,0x44,0x68,0xac,0x84,0x14,0xf8,0x24, +0x83,0xf0,0xe5,0x61,0xf7,0xcc,0x72,0x19,0x41,0xdf,0x30,0x8e, +0x65,0x4b,0x19,0x01,0x5f,0xf2,0x50,0x96,0x82,0x74,0x32,0x7b, +0xa9,0x69,0xa2,0x23,0xdb,0x0d,0xf4,0x6e,0x0e,0xc2,0x17,0x8c, +0x7b,0xb4,0xb6,0x96,0x31,0x1c,0xc0,0xcd,0x8d,0xab,0xf5,0x42, +0xc5,0xf1,0x49,0x94,0xa9,0x19,0x55,0x0c,0x0f,0xc2,0x55,0xf8, +0x1d,0xbe,0x23,0x62,0x2a,0x8a,0xf5,0xdc,0xc2,0xd8,0x5a,0x25, +0x79,0x74,0xea,0x4f,0x5e,0x7b,0xaf,0x3f,0x9a,0xde,0x4d,0x5e, +0x47,0xaf,0xbf,0x78,0xfd,0xf4,0xf5,0x7f,0xbc,0x3e,0x7e,0x7d, +0xf7,0x7a,0xfc,0x3a,0x7c,0xdd,0x79,0xdd,0x7d,0x7d,0xfa,0xfa, +0xe4,0xf5,0xbf,0xbc,0x7e,0xf4,0xfa,0xdf,0x5f,0x8b,0xd7,0xa3, +0xd7,0x93,0xd7,0xd3,0xd7,0xb7,0xaf,0xb7,0xaf,0xfd,0xd7,0xc1, +0xd1,0xb4,0x13,0x9c,0x8e,0x9a,0xc3,0xa7,0xdc,0x8e,0x28,0xc5, +0x35,0xed,0x13,0x16,0xef,0xc4,0xe2,0x1d,0x0c,0xef,0x17,0xc9, +0xba,0x80,0xc5,0x3e,0x7a,0x2c,0x2d,0x7d,0x2a,0x97,0xcd,0x42, +0x8a,0x77,0xb0,0x4a,0x5c,0xe0,0x8f,0xdc,0xc2,0x23,0xc8,0x1d, +0x43,0x35,0xf6,0x8c,0xc6,0x89,0xb0,0x66,0x1c,0x79,0x1f,0xd1, +0x03,0x51,0xe3,0x8f,0xbc,0x8f,0xe8,0xef,0xe2,0xdd,0xdd,0x5d, +0x01,0xff,0x7c,0x25,0x9c,0x93,0x12,0x57,0xd1,0x3a,0x43,0x64, +0xc5,0x07,0x5a,0x58,0x18,0xe8,0x5a,0xe3,0xc5,0xd7,0xa5,0x8e, +0x8f,0x3d,0x1f,0xb7,0xf4,0xae,0x41,0x49,0x73,0x30,0x58,0x99, +0x60,0x26,0xa7,0xc2,0x80,0xee,0x47,0x8b,0x77,0x41,0x88,0xa5, +0x63,0x4d,0x1e,0x15,0xef,0xf4,0xa6,0xdf,0x37,0x8a,0x21,0xf8, +0x8c,0xa2,0x34,0xad,0x12,0xbc,0xfa,0x14,0x9e,0x84,0xf1,0xbd, +0x8e,0xac,0xef,0xdb,0x2c,0x05,0x48,0x2f,0xe8,0x78,0xc1,0x08, +0x1f,0xab,0xb1,0xa7,0x33,0xf5,0xfc,0x50,0xeb,0x6d,0x23,0x4e, +0xe3,0x49,0x4e,0x06,0x3e,0x74,0xed,0xa5,0x5e,0xcd,0x91,0xcb, +0x2f,0x37,0x68,0xad,0xc0,0x38,0x01,0xaf,0xb3,0xf2,0x87,0x59, +0x57,0xb9,0x59,0xda,0x16,0x62,0x7b,0xb2,0x68,0x52,0x39,0xc8, +0xfb,0x91,0xb4,0x1e,0xc0,0x97,0x03,0xed,0x25,0x18,0x3a,0x21, +0x95,0x4b,0x08,0x05,0x4b,0x2f,0x0f,0x53,0x22,0x7e,0x39,0xd0, +0x74,0x9b,0xc6,0x05,0xc7,0x57,0x70,0x78,0x3d,0x02,0x1f,0x36, +0x30,0x86,0x9b,0xa8,0xe9,0x39,0x1a,0xe9,0xde,0x1f,0x3f,0x1d, +0xe4,0x14,0xa3,0x72,0x98,0x75,0xe0,0xf1,0xb1,0xcc,0xbd,0xb3, +0x42,0xb4,0xf5,0x68,0x5e,0x09,0x1a,0xd8,0xe6,0x1a,0x51,0xd3, +0xb0,0x67,0xad,0x27,0x53,0xaf,0xf7,0x2d,0xcb,0x54,0x6d,0x79, +0xfd,0xe5,0xac,0x7f,0xee,0x5f,0x9d,0xb5,0x6b,0xd9,0x36,0x41, +0x07,0xd4,0x45,0x09,0xb6,0xba,0x4f,0xf7,0x54,0x70,0xf4,0x67, +0x07,0x90,0x62,0x1d,0x40,0x7e,0xb3,0xa6,0x09,0xc8,0xb1,0x2e, +0xca,0xa5,0x7c,0x54,0x62,0x57,0x57,0xe5,0x78,0x56,0x45,0xd5, +0x8e,0x1e,0xcb,0xac,0xd3,0xd3,0xd0,0x63,0xb9,0xbd,0x89,0x97, +0x4b,0x9f,0xb4,0x1b,0x7a,0x73,0x88,0x3c,0xfc,0x1b,0xbe,0xf0, +0x04,0x1a,0xe9,0x27,0x02,0x0a,0xe9,0x83,0x96,0x36,0xfb,0x28, +0x47,0xfb,0x5e,0x1f,0x25,0xe1,0xc6,0x9e,0x2e,0xd6,0xe0,0xa4, +0x32,0x7b,0xfc,0xdc,0xb8,0x04,0xe3,0xeb,0x87,0x10,0x0b,0xda, +0x45,0xcc,0x9b,0x71,0x2a,0x70,0x0f,0x86,0x5b,0x02,0x0a,0x0d, +0x3c,0xae,0x11,0x28,0x84,0xbe,0xe0,0x2c,0x67,0xd8,0x3a,0x8e, +0x74,0x16,0x51,0x56,0x2d,0x5a,0x90,0xc9,0x37,0xd2,0x9f,0x96, +0x66,0x23,0xd5,0x09,0xdc,0x64,0x05,0x46,0x7f,0xad,0x6d,0x0c, +0xd5,0xf9,0x1d,0x8f,0x20,0x18,0xa2,0x3d,0xae,0xf1,0x62,0xc6, +0x7b,0xa9,0xfc,0x18,0xa0,0x95,0xcb,0x28,0x3b,0x0d,0xe5,0x66, +0xab,0x1a,0x8e,0x82,0x8a,0x0f,0xfc,0xf2,0x03,0x45,0xb9,0xea, +0x13,0x35,0x1e,0x57,0x77,0xeb,0x36,0x70,0xd1,0x58,0x1a,0xd8, +0xaf,0xe0,0x68,0x8f,0xd3,0xe4,0xda,0x9c,0xdf,0x96,0x93,0xef, +0xd7,0x84,0x36,0x9c,0xb4,0x74,0x96,0xc3,0x17,0xa9,0xe6,0xbc, +0x4b,0x64,0x9e,0x82,0xe3,0x48,0xfc,0xa2,0x28,0x29,0x6e,0x1c, +0x2b,0x03,0x94,0x40,0x24,0x85,0x44,0x0a,0x93,0xd7,0xa2,0x0b, +0x52,0x65,0xb6,0x56,0x3d,0xda,0xbd,0xe2,0x6b,0x1f,0xa4,0x18, +0x92,0xf7,0x16,0xe3,0xab,0x32,0xf3,0xa4,0x64,0xa6,0x85,0x5c, +0x7f,0x60,0xba,0x02,0x40,0xbf,0xcd,0x73,0x7a,0x51,0x17,0x62, +0xc7,0x2a,0x18,0x36,0x32,0xca,0xf3,0x04,0x2b,0xe3,0x11,0x7c, +0xcb,0xf1,0x82,0x20,0xbe,0x23,0x0f,0x1c,0x04,0x31,0x86,0x9f, +0x70,0xa8,0x56,0x22,0xb7,0xb8,0xb6,0xdb,0xf0,0x3a,0x6e,0x23, +0x97,0x70,0x1b,0xb5,0xcc,0xdb,0x38,0xb5,0xd2,0x52,0xdc,0x7e, +0x56,0x99,0x5d,0x6e,0xd0,0x7d,0x33,0x72,0x43,0x20,0xbf,0x6c, +0x33,0x56,0xda,0x29,0xcd,0x61,0x7a,0x99,0xe3,0x5b,0x1e,0xf8, +0xf2,0x39,0x44,0x58,0xef,0x84,0x21,0x38,0x5a,0xba,0xa1,0x5f, +0x5c,0x52,0x3c,0x10,0xae,0x13,0x79,0x74,0x72,0x7e,0x14,0x1d, +0x91,0xba,0x7d,0x34,0x3e,0xba,0xdd,0x1e,0x85,0xe4,0x96,0x69, +0xf4,0x1a,0x34,0x28,0xb5,0x85,0x4d,0xc8,0x76,0xf7,0xb0,0xa1, +0xbc,0x08,0x31,0x93,0x2c,0x33,0x10,0xba,0x67,0x14,0x8c,0x3b, +0x45,0x4f,0x6a,0xaf,0xd7,0x47,0xf8,0x32,0x2e,0xfc,0xc8,0x58, +0x2f,0xeb,0x79,0x1d,0x04,0xed,0x78,0x50,0xac,0x7c,0xd4,0xfd, +0xc7,0xe4,0xcc,0x57,0x35,0x00,0x1d,0x60,0x44,0x46,0x79,0xd2, +0x09,0x7d,0xf4,0xa8,0x8f,0x7c,0x2e,0x7a,0xd5,0x90,0x0c,0xa5, +0xf0,0x6b,0xac,0xc8,0x47,0xf4,0xe9,0xd9,0x91,0x9f,0xf5,0xaa, +0xa3,0xa7,0x47,0xf8,0xb0,0x74,0x70,0x74,0x7b,0x44,0xda,0x12, +0x60,0x87,0xf0,0x08,0x7f,0xf8,0x73,0x48,0xdf,0x43,0xac,0xec, +0xe8,0x68,0x3b,0xd2,0x04,0x21,0x0e,0xf9,0xa6,0x29,0xa4,0xf9, +0x04,0xd3,0x45,0x50,0x54,0x45,0x42,0x2a,0x90,0x12,0x8f,0x8f, +0x1f,0x71,0xe9,0x03,0x5a,0x35,0x5a,0x79,0x2b,0x27,0x1f,0x01, +0x63,0x5e,0x66,0x78,0xaf,0xd6,0x74,0xd7,0x2d,0x74,0xd7,0x44, +0x77,0xdd,0xa0,0xbb,0x46,0x5a,0x6b,0xfe,0x24,0xba,0xeb,0x7d, +0x74,0x73,0x4f,0x64,0x02,0x6a,0x26,0xa0,0x36,0x84,0x73,0xaa, +0xa2,0xbc,0xde,0xa1,0xbc,0x76,0x32,0xca,0x41,0x80,0x48,0x47, +0xdf,0xe3,0x78,0xbd,0x96,0xa4,0x0d,0x73,0x07,0x6e,0x5e,0xa2, +0x7e,0xf2,0x91,0x7c,0x42,0x0e,0xc4,0x77,0xf1,0xd1,0x34,0x38, +0x6a,0xc6,0x00,0x7a,0x3a,0x4d,0xf0,0x02,0xd1,0xc8,0x4b,0x02, +0x17,0xb0,0xdc,0x39,0x8f,0xc5,0x93,0x4f,0xb0,0xa3,0x0c,0xa8, +0x18,0x95,0x79,0x7d,0xbb,0xa3,0x4e,0x62,0x3a,0xb9,0x31,0xa2, +0xb1,0x89,0xfd,0x33,0x30,0x4f,0x82,0xed,0x3c,0x2e,0x61,0xb6, +0x49,0xe8,0x2d,0x1f,0xde,0xb2,0x48,0x68,0x7a,0x83,0x25,0x87, +0xcf,0x10,0x46,0x25,0xdb,0xad,0xa0,0xb1,0x03,0xca,0x0b,0xba, +0x61,0x0c,0x5a,0xad,0x87,0x8f,0xd2,0xe1,0xe8,0x80,0x60,0xf2, +0xe9,0x7a,0x15,0x0c,0xb4,0x3c,0x82,0x96,0xc8,0x12,0x2a,0x3f, +0xe4,0x03,0xb9,0xcc,0x74,0xf5,0x57,0x25,0xaa,0x07,0x73,0x29, +0x20,0x6f,0xe5,0xca,0x90,0xc5,0x07,0xdb,0x47,0xf7,0x4e,0x17, +0x31,0x43,0x1d,0x86,0x24,0xd9,0xc4,0x68,0x1a,0x11,0x8a,0xaf, +0xf0,0x11,0xa4,0x88,0x1f,0x43,0x04,0x62,0x31,0x1f,0xd0,0x2e, +0x9f,0x3c,0x94,0x11,0x38,0xb6,0x49,0x48,0xc2,0x07,0x98,0x7a, +0x73,0x85,0x8b,0x5e,0x51,0xc2,0x22,0xbd,0xf3,0xf9,0x0a,0x9f, +0xc7,0x3b,0xeb,0x9d,0xd3,0xef,0x9c,0x7e,0x57,0xf4,0xf4,0x03, +0xc5,0xde,0xdd,0xe1,0x9f,0xa2,0x25,0x63,0xe1,0x64,0x2c,0x1a, +0x19,0x57,0x9c,0x31,0xde,0xcd,0x08,0xca,0x12,0x67,0x5c,0xd1, +0x6f,0x4c,0xbf,0x33,0x9d,0x31,0xe7,0x8c,0x4b,0xfe,0x33,0xdb, +0xcd,0x9f,0x2f,0x65,0xfe,0x9c,0x7e,0x97,0x32,0xbf,0xb2,0x2c, +0x8e,0x3c,0xe5,0xf4,0xed,0x8a,0x27,0x7c,0xcf,0xd9,0x29,0xcf, +0x95,0x4a,0x29,0xad,0x32,0x71,0xf2,0x2e,0x73,0x5f,0x47,0x07, +0x4a,0x9b,0x25,0xdc,0xbc,0xef,0x40,0xbe,0x3b,0xf2,0x0d,0xba, +0x9e,0xb4,0xa7,0x19,0x03,0x01,0xb8,0x30,0xfd,0xaf,0x3f,0x7e, +0xff,0xf7,0x1f,0xde,0xfc,0xfc,0xec,0xc7,0x9f,0x26,0x3a,0x6d, +0x3a,0x66,0x51,0x86,0x51,0x51,0xc7,0xc9,0x99,0x57,0xde,0xae, +0x82,0x31,0x72,0xc9,0xd6,0x49,0x16,0x46,0x35,0x41,0xa2,0x11, +0x97,0x5c,0x3d,0x4d,0x27,0xb8,0x7e,0x9a,0x7a,0x81,0xd9,0x80, +0xcf,0x49,0xbf,0x97,0x57,0x57,0x1d,0xba,0xf8,0x6c,0x78,0xec, +0x37,0x18,0xd2,0x2c,0xcc,0xb2,0xb7,0x6a,0x2f,0xac,0xc9,0x0c, +0xc9,0x35,0xa8,0x6d,0xc7,0x9b,0xd0,0xe8,0x4e,0x24,0xe1,0x70, +0x04,0x05,0xd1,0x0b,0x22,0xfa,0xb4,0xed,0x40,0x25,0xe9,0xeb, +0xde,0xaa,0x06,0x7e,0xa3,0x29,0x03,0x7e,0x5d,0xb9,0x95,0x1e, +0xbb,0xdd,0x48,0x57,0x91,0x5e,0x10,0xf0,0x33,0x6a,0xb2,0x87, +0x01,0xc6,0x56,0xe3,0x73,0x4c,0x68,0xbe,0x19,0xbf,0x6c,0x3d, +0x98,0xe2,0x68,0x5c,0x02,0xaa,0xec,0x0a,0x39,0x99,0x39,0x83, +0xec,0x2f,0x14,0xc1,0xdc,0x45,0x06,0x8d,0x98,0x0c,0xd9,0xee, +0x72,0xb5,0xe5,0x75,0xe8,0x83,0xd8,0x8c,0x95,0xe0,0x10,0x21, +0x9c,0xc5,0x6b,0x50,0x56,0xf8,0xea,0xf8,0x97,0x31,0xae,0x4a, +0x81,0x02,0x5f,0xf2,0x1b,0xdf,0xef,0x36,0x26,0x8d,0x11,0xc6, +0x53,0xa6,0xcb,0xab,0x55,0x39,0xbe,0x56,0x3e,0xf0,0x75,0x14, +0x7a,0xad,0x38,0x91,0xcf,0xd7,0xcb,0xfc,0x0c,0xcf,0x2b,0xb2, +0xb1,0x77,0xd4,0x39,0xa2,0xda,0x58,0xf9,0xe4,0x62,0x0d,0xf3, +0x08,0x39,0x88,0x48,0x4e,0x2b,0x2a,0x3e,0x02,0xb4,0xde,0x47, +0x64,0x5f,0x6c,0x79,0xf2,0x50,0x9d,0x94,0x14,0xf4,0x4a,0xd4, +0xe2,0xbd,0x7c,0xb8,0xbc,0x8a,0xaa,0xb1,0x1e,0xe7,0x40,0x7d, +0xa9,0x02,0xeb,0x7c,0x8e,0xb2,0xf1,0x83,0x0d,0x68,0x16,0x51, +0xd7,0x51,0xed,0x00,0xd7,0xfb,0x80,0x07,0x53,0xf1,0xfe,0x7d, +0xf4,0xde,0x01,0x7e,0xbf,0x0f,0x78,0xa8,0x4e,0xd3,0x3d,0x7e, +0xe9,0x06,0x16,0x8a,0x5e,0x07,0x93,0x3a,0x58,0xa7,0x49,0x05, +0xe4,0x02,0xbd,0xef,0xa7,0x52,0x06,0x05,0xb0,0xeb,0xa3,0x00, +0x79,0xe3,0x05,0x1f,0xd1,0xe0,0x62,0xba,0x77,0x74,0xcb,0x3a, +0xf3,0x40,0xe9,0xcb,0x03,0x7e,0xce,0x92,0x45,0x80,0xa7,0x07, +0xaf,0x11,0x29,0x6d,0x8a,0xc2,0x81,0xb1,0xd3,0x61,0x85,0xcf, +0x59,0xa2,0xb1,0x68,0xb4,0xdf,0x28,0xa5,0x24,0x56,0xea,0xf9, +0x7b,0x67,0x6b,0xd4,0xda,0x11,0x69,0xee,0x90,0x66,0x42,0x6f, +0xcf,0x92,0x36,0x87,0x8b,0x60,0x9a,0x6f,0x36,0x79,0x72,0x1d, +0xb1,0xc2,0x16,0xe2,0xb7,0x67,0x8c,0x2c,0x59,0x3f,0xe3,0xf3, +0x62,0xdc,0xda,0xc8,0xd4,0xb3,0x41,0xaa,0x58,0x81,0x4e,0x7b, +0xf0,0x62,0x8a,0xfc,0x7b,0x77,0xa7,0x40,0xb2,0xad,0x50,0x0b, +0x86,0x5b,0x7b,0xa3,0xc4,0x20,0xa3,0x2f,0xab,0x06,0xec,0xb2, +0x24,0x5d,0x4b,0x17,0x7e,0x1a,0x04,0xed,0xe8,0x0a,0xf5,0xc6, +0xd0,0xf7,0x3f,0x7e,0xfd,0xe2,0xbb,0x67,0x2f,0xc7,0x59,0x0e, +0xeb,0x83,0x78,0x15,0xbe,0xbb,0x8a,0x71,0x4b,0x37,0xc6,0xf7, +0xc6,0x44,0x0e,0x2b,0x06,0xe9,0x6c,0x01,0x32,0x4b,0x0a,0x1b, +0x17,0xd0,0xad,0x31,0xc9,0x38,0x32,0xe2,0xc2,0x30,0x3f,0xd2, +0xc3,0x57,0x52,0x0b,0x43,0x73,0x0b,0x4d,0xe6,0x59,0x12,0x75, +0x2a,0x7e,0x77,0xf7,0xf2,0xc5,0x77,0xcf,0x9f,0xfd,0x88,0x93, +0xcd,0x23,0x02,0xba,0xbb,0xc3,0x78,0x98,0x72,0x78,0x30,0xa0, +0x87,0x98,0xf8,0x9d,0x90,0x32,0xcb,0xad,0xc3,0x56,0x4a,0x66, +0x19,0xbd,0xbb,0x43,0x27,0x12,0x26,0x3c,0x42,0x50,0xda,0xa2, +0x92,0x07,0x34,0x7f,0x5f,0xe7,0xc9,0x3c,0x5b,0xae,0xd3,0xf7, +0xc9,0x82,0x79,0x49,0x54,0xe0,0x49,0x0d,0x3f,0xf4,0xc4,0xe7, +0xfa,0xd1,0x9c,0xbd,0x47,0xf5,0xe4,0xa3,0x79,0xaa,0x7c,0x26, +0x89,0xeb,0xc0,0xe5,0x7c,0x67,0xcb,0x82,0x1a,0x55,0x28,0xc5, +0xd4,0x5f,0x32,0xdb,0x6a,0x35,0x9b,0x53,0xbc,0x0e,0xa6,0xf3, +0x31,0x9c,0x89,0x0b,0x69,0x29,0x8a,0xaf,0x20,0x8f,0x38,0x9a, +0xf0,0x4b,0x08,0xfb,0x58,0x95,0x62,0x02,0xde,0x7b,0x92,0xe9, +0x2a,0x32,0xb4,0x57,0x65,0x32,0x0e,0x06,0x20,0xc4,0x6f,0x61, +0x93,0x3e,0x53,0x89,0x41,0xb1,0x3e,0xaa,0x95,0xe0,0xea,0xae, +0x71,0xce,0xef,0x85,0xba,0xbd,0x41,0xd2,0x98,0xeb,0x55,0xbc, +0x7d,0x25,0x79,0x36,0xc9,0x69,0xb6,0x99,0xb2,0x6b,0xf5,0x40, +0x34,0xfd,0x9b,0x38,0xb3,0x89,0x9a,0x6a,0xe5,0x47,0x48,0x7b, +0xa0,0x3d,0x1a,0x50,0x75,0x86,0xdc,0x99,0xf0,0xd8,0xae,0x31, +0x18,0xb5,0x99,0x40,0x63,0x59,0x92,0x72,0x7e,0xf7,0x8c,0x76, +0xe0,0xd8,0x51,0x88,0xbc,0xe4,0xac,0x2e,0xec,0xd1,0x6d,0x3d, +0x87,0x79,0x32,0x1f,0xde,0xcf,0x01,0xed,0xb0,0xf9,0x62,0x11, +0x34,0x1a,0x8e,0xae,0x10,0x0d,0x73,0xd4,0x7a,0xa9,0x6e,0xf8, +0x0d,0x65,0x03,0x70,0xcf,0x81,0x84,0x80,0x1e,0x08,0xe5,0x2b, +0xc3,0x76,0xda,0x0f,0x14,0xe3,0x53,0xdb,0x23,0x80,0xb2,0x55, +0xe8,0x73,0x06,0xda,0xa3,0x87,0xa6,0x50,0xae,0xb3,0x75,0x9c, +0x8d,0xe3,0x47,0x8c,0xc0,0x09,0xad,0x1d,0x8f,0x05,0xb9,0x17, +0xc8,0xc8,0xa5,0xdd,0xf9,0xf7,0x09,0xa7,0x06,0x4e,0x2a,0x58, +0x49,0x80,0x30,0xe1,0x1a,0x1a,0x2f,0xea,0xe1,0xd5,0xbc,0xf7, +0xfb,0x9a,0x1f,0xa5,0xc8,0x16,0x80,0xa6,0x07,0xde,0x86,0xea, +0xd4,0x62,0x54,0x6e,0x8b,0xc8,0x59,0x30,0x3e,0x6b,0x15,0x8b, +0x33,0x47,0x20,0xb6,0xc1,0x48,0x3a,0xf3,0xdd,0xd9,0x56,0x03, +0xbc,0x67,0x4d,0x0b,0x75,0xa8,0x04,0x1e,0x56,0x52,0xcb,0x93, +0xc1,0xc5,0x63,0x3c,0x5b,0xf1,0x27,0x3a,0x46,0xc8,0x18,0x94, +0xaa,0xb3,0x60,0x4a,0xbb,0x45,0x95,0xc9,0xf2,0x9e,0xb3,0xc4, +0x95,0xce,0xf2,0x9e,0xb3,0xc4,0x95,0xc9,0x22,0x8f,0xb6,0xed, +0xee,0x4d,0x6b,0x18,0xbc,0x01,0xf5,0x5e,0x75,0xef,0x09,0x3b, +0xf8,0x87,0x1f,0xe5,0x39,0x5d,0xee,0xc8,0x72,0x3a,0x2a,0x0e, +0xae,0x31,0x85,0xea,0xf9,0x3b,0xc3,0x83,0x76,0x21,0xef,0x74, +0x7f,0x2b,0xc7,0x87,0x8d,0x00,0xd4,0xf5,0x77,0x98,0x09,0x9d, +0x5e,0x0b,0x83,0x18,0xe0,0x7f,0x50,0x4d,0x90,0xe1,0x26,0x31, +0x81,0x78,0x1f,0xc9,0x52,0x94,0xc5,0x67,0x5b,0xf5,0xbe,0x35, +0x35,0xdc,0x21,0xf8,0x5b,0xa2,0xd9,0xb4,0x11,0x31,0xaa,0x51, +0x85,0x6f,0x0f,0xd7,0xe2,0xdb,0x07,0x57,0xa4,0x0f,0xf2,0x6c, +0x2a,0xa2,0x6c,0x70,0x2c,0x2c,0x6d,0xc4,0x93,0x44,0xec,0x21, +0x3e,0xae,0x0c,0xf1,0xef,0xf7,0x10,0x1f,0x57,0x07,0x89,0x07, +0x14,0x0f,0x21,0xbe,0x0f,0xe4,0x3b,0xc4,0xbf,0x9f,0x46,0x0d, +0x2c,0x6a,0x9e,0x85,0x91,0xe5,0xe5,0xf7,0x7f,0x85,0x8e,0x69, +0xc2,0xaf,0x5e,0x7c,0xfb,0xfc,0xee,0xee,0x11,0xc6,0xbc,0x4f, +0xf2,0x0c,0x06,0x4a,0xbb,0xa2,0x18,0xc5,0x8f,0x0e,0x4a,0xae, +0xa8,0x63,0x48,0xbf,0x2f,0x74,0x64,0x20,0x4c,0xc1,0xea,0xd4, +0xd2,0xa4,0xbf,0xc7,0xce,0xd3,0x36,0x4d,0x69,0x4b,0x71,0x35, +0xde,0xe1,0xc8,0x85,0xf3,0x14,0x6a,0x22,0x30,0xc4,0x82,0x4e, +0x94,0xa0,0x95,0x80,0xca,0x3e,0xd1,0x83,0xe2,0xf1,0x71,0x73, +0x54,0x1c,0x7b,0xe6,0x1b,0x94,0x75,0x0a,0x78,0x53,0x1a,0x90, +0xa9,0xf2,0x49,0xb5,0xc9,0xd6,0x74,0x0e,0x63,0x55,0xfd,0x87, +0xef,0xff,0xf1,0xfc,0x47,0x85,0x4b,0x41,0xf8,0x36,0x38,0x67, +0x9e,0xaf,0xe2,0xcb,0x8d,0x82,0xa3,0x80,0x8f,0xcf,0xed,0xd3, +0xc4,0x0f,0x3d,0x8e,0x1f,0xcc,0xb6,0xf9,0x39,0xf6,0x55,0x3f, +0xa7,0xe7,0xa3,0xd2,0x4b,0xa6,0x1d,0x81,0xa7,0x7a,0x08,0xd8, +0xab,0x99,0x28,0x00,0xd4,0x4b,0xd4,0xb7,0xe2,0xe0,0x9a,0x5c, +0xdc,0xa8,0x48,0x35,0xe6,0x53,0x6c,0x60,0x0d,0xee,0x4d,0x45, +0x8e,0x9f,0xd2,0x07,0xbe,0x5a,0xa3,0xcd,0x8e,0x09,0x1c,0x65, +0x0a,0xd0,0xe3,0xcf,0xce,0x73,0x7d,0x76,0x2a,0x2b,0xf2,0x3f, +0xfc,0xf8,0xfd,0x0f,0xcf,0x7f,0x7c,0xf5,0xcb,0x44,0xa7,0x4d, +0x03,0x2a,0xa0,0x2f,0xe4,0xd3,0x7e,0x3a,0x9e,0x5d,0x09,0xe1, +0x2e,0xca,0x23,0xa3,0xb7,0x51,0x92,0x93,0x59,0x0f,0xfd,0x2e, +0x4f,0x88,0x25,0x04,0x43,0x16,0x35,0x9a,0x12,0x9c,0xa9,0x46, +0x58,0xe2,0x7e,0x9c,0x6c,0x3d,0x26,0x61,0xec,0x21,0x94,0x31, +0x8c,0xf5,0x27,0xbe,0x64,0x68,0xb2,0xb5,0x33,0x86,0x47,0x2a, +0x56,0x03,0x9c,0xd8,0xc6,0xd0,0xc0,0xd1,0x34,0x32,0x20,0xb0, +0xd1,0x15,0xba,0x83,0xa9,0x03,0xd0,0xcc,0xc7,0xd2,0x2e,0x9b, +0x0a,0x56,0x15,0x56,0xe4,0xc8,0x1e,0x63,0x20,0x8e,0xb5,0xb9, +0x6b,0x67,0x52,0x83,0xb0,0x9c,0x55,0x1d,0x35,0x1d,0x47,0x90, +0xeb,0xd6,0x4e,0xa5,0xf4,0x55,0x88,0xa5,0x69,0x8a,0x35,0xf0, +0x48,0xbe,0xc1,0xee,0x09,0xb9,0x5a,0x88,0x3c,0xa9,0xc1,0x92, +0x4d,0x06,0x24,0xa3,0x81,0x02,0xf5,0x9c,0xc8,0x23,0xfb,0x03, +0x14,0xf8,0xc8,0x43,0x21,0xf7,0x84,0x2b,0x1b,0xbb,0x0b,0x3d, +0x73,0x39,0x9c,0x96,0x37,0xee,0xe2,0x0d,0x56,0x5f,0x7b,0x0c, +0xb3,0xc4,0x75,0x9a,0xdc,0x7c,0x13,0xa3,0x5e,0x5e,0xdb,0x5e, +0xe8,0xc8,0x88,0x88,0x37,0xfb,0x29,0x37,0x9a,0x18,0x5d,0xe1, +0xb3,0xa7,0xb8,0x55,0x4a,0xf1,0xb4,0xe5,0x47,0x54,0xd0,0x5e, +0xbc,0xda,0x7b,0x44,0xf7,0xf0,0xd2,0x0b,0x08,0x45,0xab,0x5d, +0x47,0x8c,0xc7,0xb2,0x36,0xe8,0x8a,0x89,0x52,0x54,0x48,0xbe, +0x40,0x8c,0xc6,0x56,0xaa,0x5a,0x6d,0x67,0x3e,0x0a,0x3c,0x54, +0x1f,0x42,0x2a,0x5f,0x61,0xf3,0x78,0x86,0x0f,0x75,0x94,0x6d, +0xaa,0x3c,0xe9,0x71,0xcf,0xa2,0x5a,0x8e,0xa0,0xf8,0x61,0x06, +0xe7,0x04,0x8e,0x11,0xf1,0x05,0x40,0xb3,0xf0,0xd5,0xc6,0x62, +0x48,0xc8,0x57,0x19,0x2c,0xc2,0xf2,0x2b,0xe4,0x5f,0x40,0xcf, +0x6d,0xba,0x71,0x91,0xc5,0x5e,0x7a,0xfb,0x1f,0xf4,0x16,0x13, +0x63,0xa7,0xd2,0x98,0xf5,0xb3,0x09,0x0b,0x5b,0x30,0x89,0xe1, +0x63,0xab,0x51,0x42,0x18,0xfc,0xc8,0xf7,0x2d,0xd5,0xa3,0x61, +0x43,0x47,0xcf,0xe0,0x35,0x6d,0xe8,0xe8,0x37,0xe0,0x0d,0x72, +0x9f,0x20,0xe4,0xeb,0xa0,0x0d,0xb3,0xa7,0x76,0xb3,0x29,0x69, +0x61,0x66,0x6c,0xad,0x7d,0x7b,0x99,0x40,0x47,0x3d,0x08,0xde, +0x7a,0xf0,0xea,0x61,0x06,0x34,0x1b,0xb9,0x38,0x3e,0x2e,0x2b, +0x7a,0x0a,0x27,0x39,0x53,0x77,0x86,0x2b,0xa2,0x81,0x8e,0x29, +0xa5,0xbd,0x3b,0x1f,0x59,0x92,0xb8,0x47,0x6a,0xb7,0x5d,0x47, +0x3f,0xff,0xee,0x15,0xf4,0x8e,0xbe,0x89,0xf8,0xfb,0x0f,0x5f, +0x3f,0x7b,0xf5,0x9c,0xad,0xe8,0x24,0xc8,0x3f,0x5f,0xbc,0x8a, +0x86,0x26,0xfc,0xf5,0xf3,0x6f,0x9e,0xfd,0xfd,0xe5,0xab,0x37, +0x00,0xf7,0x2c,0xba,0x2d,0xd0,0xd4,0x6a,0x9d,0xac,0xc2,0x81, +0x55,0x98,0x6b,0xb8,0x48,0xed,0xdd,0x3c,0x50,0x95,0x58,0xa2, +0x56,0xb4,0xf6,0x6d,0x88,0x86,0x0f,0xa9,0xbb,0x3b,0x83,0xe9, +0xee,0x6e,0x22,0x73,0x4d,0xe5,0xfa,0x3f,0x8a,0x64,0x04,0x4e, +0x74,0xe4,0xa6,0xcd,0x05,0xb0,0x68,0xe4,0x6b,0x5a,0xb9,0x73, +0xfa,0x98,0xe9,0xfd,0x95,0xac,0xa7,0xaf,0x71,0x8d,0xad,0x6f, +0xf4,0x89,0x4a,0xfb,0x6b,0x12,0xe0,0x67,0x72,0x34,0x51,0x8e, +0x9d,0x50,0x13,0x88,0x2f,0x80,0x8d,0xed,0x80,0x01,0xe1,0xbd, +0xcc,0x4c,0x5f,0xab,0x1a,0x9b,0x4f,0x6b,0x7b,0x8b,0x5f,0xad, +0xd7,0x29,0x41,0xc7,0xdb,0x54,0xe8,0xbb,0x96,0x81,0x6d,0x33, +0x72,0x0b,0x32,0xb0,0x6a,0xfb,0x02,0x5f,0x56,0x69,0xdb,0x6c, +0xc2,0xdd,0x4e,0x69,0x21,0xc0,0x0f,0xb2,0xa3,0xe9,0x00,0x9d, +0x36,0x4b,0x83,0x01,0xf9,0x20,0x3b,0x6d,0xd0,0x1e,0x78,0x8d, +0x9d,0x9f,0xb8,0xa5,0xe7,0x2e,0x7f,0xb2,0x7d,0x88,0x9a,0x57, +0xe8,0xd9,0xf8,0x40,0x17,0xd2,0x6b,0x58,0x67,0x18,0x8c,0x9b, +0xe2,0xf8,0x98,0xd9,0xc2,0xb6,0x03,0xb4,0xd1,0x64,0x3f,0x8c, +0x3d,0xcf,0xae,0x8a,0xd4,0xba,0x3e,0xcd,0x9b,0x98,0xca,0x0c, +0x48,0xfa,0x13,0x93,0x3b,0x9b,0xea,0x18,0x1e,0x9f,0xea,0xe7, +0xa8,0x48,0x5d,0x62,0x90,0x3e,0x74,0xa8,0xc6,0x9a,0x2c,0xdb, +0x1a,0x82,0x4d,0x01,0x52,0xf4,0xe3,0xc7,0x87,0x34,0x54,0xc2, +0x98,0xd2,0x90,0x1f,0x85,0x7e,0x52,0x9f,0x5f,0x57,0x67,0x8b, +0xa0,0x4b,0x84,0xe7,0x79,0xcd,0x05,0x22,0x2e,0x77,0xb8,0xf4, +0x91,0x6d,0x04,0xc0,0x70,0x13,0xcc,0x38,0x95,0xdf,0x58,0xa8, +0x53,0xe7,0x22,0x9d,0xad,0x5c,0x59,0xe5,0xfa,0xfd,0xd6,0xea, +0x52,0xed,0xf6,0xd5,0xa8,0x41,0xac,0x01,0xdb,0xa5,0x8f,0xdf, +0xf9,0x6e,0x9a,0x58,0xd2,0x83,0x3e,0xf4,0x60,0x3a,0xd9,0xbe, +0x92,0x10,0x19,0x2c,0xca,0x05,0x91,0x2a,0x93,0x1e,0xe1,0x51, +0x85,0xe1,0xbb,0xe3,0xd0,0x8b,0xf1,0xe1,0x3a,0xcb,0x97,0x4d, +0x77,0x30,0x66,0x63,0x58,0x7c,0x9c,0x25,0x64,0xbf,0x46,0xd6, +0xe3,0xe1,0xf8,0xae,0xca,0xe5,0x96,0x7f,0x79,0xc2,0x66,0xf1, +0x4f,0x1d,0xf1,0x67,0x71,0x57,0xa7,0x6d,0xc9,0xcd,0x91,0xd3, +0x4d,0x38,0xd9,0xea,0x3b,0xd7,0x69,0x61,0x5f,0xf4,0xa7,0xf7, +0xb8,0x31,0xa4,0x8e,0xed,0xd7,0x54,0x1e,0x99,0xcd,0x60,0x34, +0x3f,0xc4,0x2d,0x5b,0xe2,0x51,0x9f,0x8e,0xe7,0x30,0x9d,0x1e, +0xe6,0xe6,0x8a,0x07,0xc6,0xa1,0x1d,0x73,0xbd,0x61,0x26,0xc8, +0xd6,0x0a,0x56,0xe9,0xbe,0xe4,0xfa,0x94,0x0b,0xd6,0xc8,0x2d, +0x2a,0x67,0x57,0xe9,0x9e,0xb7,0x9a,0x28,0x85,0x34,0xec,0xc5, +0x4c,0x10,0xf5,0x0f,0xb0,0x6e,0x21,0x9b,0xaa,0x3c,0xbb,0x1c, +0xab,0x0f,0xbf,0x25,0xf3,0x5e,0xab,0x17,0x04,0x8c,0xa8,0x5c, +0x7c,0xc0,0x9a,0xca,0xe6,0xe7,0xc9,0x0d,0x0f,0x38,0x19,0x19, +0xce,0x37,0x9b,0x68,0x96,0x60,0x20,0x4a,0x79,0x85,0x13,0xa8, +0x95,0xd3,0x5a,0xb7,0xd0,0x5c,0x76,0x7c,0x4c,0x60,0x7f,0xc5, +0x0e,0x64,0x57,0x8e,0xcb,0xd8,0xba,0xb5,0x77,0xa9,0xb8,0xb5, +0xa9,0xc4,0x1f,0xdc,0xbb,0x66,0xd2,0x00,0x26,0x52,0x70,0x24, +0xa9,0xce,0xde,0xaf,0x24,0x1f,0xd7,0x34,0xa0,0x54,0xa4,0x20, +0xef,0x8f,0x88,0x51,0x76,0x4c,0x14,0xe1,0x35,0xa7,0x16,0x1a, +0xda,0xaa,0x2a,0x25,0x08,0xaf,0xa3,0xd3,0x63,0x7f,0x78,0x64, +0x4e,0xca,0xb0,0xa0,0xe7,0x0b,0x30,0xe1,0x0c,0x2f,0x65,0x7f, +0x63,0xe9,0x10,0xa8,0x1d,0x04,0x72,0x37,0x5f,0x73,0x93,0x4c, +0x70,0xd6,0x78,0xdb,0x97,0x4c,0x71,0x36,0xca,0x3a,0x44,0x19, +0x7e,0x20,0xf8,0xae,0xdd,0x07,0x75,0x0b,0x4c,0x42,0xc9,0xa2, +0xce,0x89,0x9e,0x7e,0xae,0x8a,0x08,0xe7,0x7c,0x2a,0x9c,0x2f, +0x4d,0x4e,0x28,0x8d,0xde,0xd3,0xe2,0x3e,0x7a,0xff,0xa3,0x7e, +0x54,0x0b,0x73,0x5d,0x3d,0xa9,0xa3,0x94,0x4a,0xc0,0xda,0x9c, +0x8d,0xe5,0xbd,0x48,0x0a,0xf0,0x4b,0xf9,0xc1,0x34,0x54,0x84, +0x50,0xd5,0xa9,0xa0,0xf1,0xa3,0x41,0xc8,0x8f,0x82,0x39,0x3d, +0x59,0x3e,0x75,0x8f,0x0b,0x5b,0x9b,0x68,0xca,0x37,0x26,0x8d, +0x26,0x64,0x35,0x46,0x98,0x63,0xc5,0x88,0xd9,0xaa,0x2a,0x82, +0x25,0x13,0xbf,0xd8,0xac,0x7e,0xb7,0x56,0x1f,0xc8,0xb0,0x88, +0x58,0x86,0xe3,0x95,0x55,0xc0,0xd9,0x58,0x85,0x42,0x2a,0x4a, +0x3e,0x58,0xd6,0x28,0xd6,0x74,0x1a,0x88,0x6f,0xc8,0xcc,0xae, +0x74,0x5b,0xa6,0x42,0x97,0xe2,0x52,0x5b,0x0c,0xf1,0x54,0x45, +0xbf,0xf6,0x40,0xc3,0xba,0x3a,0xc9,0x2f,0x9b,0xf6,0xd9,0x55, +0x64,0xe8,0x96,0xf7,0x18,0x61,0xf8,0xe1,0x59,0x90,0x21,0xb0, +0xae,0xf6,0x79,0x26,0x8d,0x5e,0xea,0xb5,0x06,0x27,0x01,0x23, +0xf5,0x68,0x2e,0x4f,0x99,0x82,0x2f,0xfa,0x74,0x6a,0x6e,0x9d, +0x87,0xda,0x87,0x50,0xd3,0x68,0x27,0x69,0x4a,0xaf,0xdb,0xc2, +0xb2,0x8f,0xa8,0xb5,0x66,0x48,0x7b,0x56,0x65,0x9b,0x33,0xf4, +0xeb,0x87,0x55,0xa1,0xca,0x69,0x17,0x9f,0x58,0x95,0x4b,0xa8, +0xca,0x65,0xe0,0x4c,0xaf,0x53,0x1e,0x72,0x68,0x7c,0xc7,0xa0, +0x3e,0x93,0xd2,0x00,0xc2,0x5e,0x8e,0x36,0xd2,0x7a,0x16,0x4b, +0xb6,0x8d,0x66,0x6a,0x8c,0x55,0x74,0x2b,0x05,0x77,0xb5,0xae, +0xf8,0x61,0x25,0xa8,0x3f,0x0f,0x2e,0x32,0x1c,0xd9,0x89,0x72, +0xd3,0x27,0xc6,0x7b,0x27,0x0c,0x86,0xdf,0xb8,0x19,0x83,0x7f, +0x7d,0x95,0x18,0x30,0x1c,0xd9,0x3f,0x2a,0x21,0x68,0x4e,0x2b, +0x52,0x77,0xa1,0x08,0x16,0x88,0xc6,0xfc,0xd2,0x7c,0x2d,0x56, +0x63,0xe4,0xc7,0x8f,0x68,0xa2,0xd6,0x33,0x8d,0x32,0x57,0xd4, +0x55,0xb5,0xc7,0x21,0x1c,0x83,0x8c,0xf1,0x0a,0x86,0x9a,0xc6, +0x2b,0x67,0xd6,0xeb,0x92,0x90,0xbc,0xf3,0x70,0xdd,0x8e,0xc1, +0xbd,0xe5,0x7b,0xab,0x40,0xc3,0x37,0x9e,0x1f,0xcf,0x76,0xe6, +0xc6,0xa7,0x28,0x4f,0x45,0x27,0xf2,0xee,0xd0,0x9f,0x43,0x27, +0x52,0x9b,0xef,0x8d,0x77,0x9a,0xa5,0xa3,0x12,0x5e,0xf7,0xb8, +0xcb,0x20,0xc1,0xab,0x1e,0x77,0x11,0x24,0xe4,0xca,0xa7,0xb1, +0x12,0x12,0xb4,0xfc,0x71,0x16,0x43,0xaa,0x04,0x6a,0x7a,0x47, +0xcd,0x98,0xe1,0x9e,0xde,0x3e,0xa3,0xbf,0x35,0x39,0xc6,0x30, +0x8f,0x82,0x9c,0x55,0x77,0x77,0x3e,0xfc,0x46,0xd6,0x4d,0xc1, +0xb4,0xc0,0x99,0x6a,0x0c,0x3a,0x89,0x9f,0x27,0xef,0xae,0xd2, +0x3c,0xf1,0xbd,0x79,0xbc,0xbe,0x8e,0x0b,0x2f,0x08,0x7c,0xdc, +0xc1,0xc6,0x85,0x6a,0x91,0xe0,0xab,0xf9,0xbe,0x37,0xcb,0x16, +0xb5,0x17,0xe0,0x55,0x44,0xb4,0x49,0xd7,0x80,0xbd,0xb8,0x2c, +0x73,0x08,0xae,0xe2,0xa2,0xf0,0x84,0x77,0x9d,0x2c,0xe3,0x37, +0xe7,0xe9,0x62,0x91,0xac,0x75,0x9a,0xf4,0x7f,0x31,0x50,0x61, +0xb9,0xbb,0x89,0x11,0x05,0xae,0x73,0x7c,0x6f,0x91,0x82,0x5e, +0x15,0xa3,0x33,0xb0,0x75,0xb6,0x4e,0x3c,0xf6,0xc4,0x86,0x27, +0x8d,0xcb,0xa4,0xfc,0x4a,0xd6,0xc5,0x1b,0x2e,0x3c,0x7b,0x2b, +0x0f,0x4d,0xad,0xbe,0x24,0x16,0xf8,0x99,0xc0,0x80,0x60,0x86, +0x98,0x09,0x57,0x5a,0x75,0x96,0xe7,0x63,0x4e,0xe9,0xa1,0xd1, +0x78,0x5f,0xd0,0x7f,0x41,0xe8,0x53,0x24,0x19,0x73,0xd9,0xb9, +0x45,0x26,0x6d,0xd5,0x8f,0x8f,0x71,0x07,0x2a,0xeb,0x65,0x9b, +0x78,0x9e,0xe2,0xb3,0xfb,0x2a,0xe1,0x1f,0x58,0x1d,0x94,0x0b, +0x89,0x35,0xa9,0x36,0x68,0x4b,0xe1,0x24,0x43,0x07,0xe2,0x54, +0x97,0x5e,0xa0,0x54,0x11,0xc9,0x46,0xa0,0xfc,0x5c,0xcf,0x38, +0x9b,0x78,0x96,0x01,0x9c,0x87,0x17,0xb4,0x9a,0x51,0xfc,0x6e, +0xb9,0xcf,0x19,0x02,0x56,0x86,0xf4,0x32,0xc6,0x61,0x85,0x2a, +0xc2,0xba,0xd0,0x9d,0x27,0x71,0xa3,0x68,0xee,0xcd,0x99,0xad, +0x1f,0x67,0xdc,0x7d,0xf1,0x8c,0x07,0x08,0x7b,0x30,0x45,0x88, +0x9c,0xb8,0xc8,0xca,0xa6,0xe9,0x13,0x16,0x55,0x06,0xf1,0x2e, +0x71,0xf4,0xf4,0xcb,0x1f,0x45,0x1c,0x22,0xff,0x2d,0xc4,0xe5, +0x28,0xfa,0x2e,0x71,0x15,0x10,0x06,0x9d,0xa9,0x2f,0x6a,0xf8, +0xa8,0xf1,0xe3,0x26,0xaa,0x3a,0x99,0xda,0x41,0xeb,0x8b,0xf3, +0xa8,0xee,0x64,0x7a,0xe7,0xac,0xaf,0x3b,0xb0,0x91,0x41,0x34, +0x61,0xb9,0x11,0xe7,0xbf,0x97,0xac,0xed,0x88,0x5a,0x7a,0x19, +0x2f,0x9b,0x3c,0xbd,0x89,0x1c,0x12,0x2d,0x02,0x05,0x0c,0x0a, +0x5c,0xa5,0xa0,0xeb,0xab,0xa7,0xac,0x70,0x91,0xd7,0x8b,0x57, +0xe9,0x72,0x3d,0xbe,0x39,0x1d,0xaa,0xdb,0xa3,0x76,0x6c,0x08, +0x0a,0x51,0x8d,0x39,0x6b,0x99,0x13,0x56,0x94,0x8b,0x55,0xc2, +0x30,0x33,0x98,0x44,0x90,0xf7,0xe3,0x73,0xcc,0x2c,0x6f,0xa5, +0x36,0x52,0x00,0xc1,0x1e,0xde,0x54,0x9d,0x1b,0x51,0x77,0xce, +0xed,0x76,0xb8,0x5a,0x35,0x2b,0x54,0x0d,0x44,0x3d,0x68,0x43, +0x30,0xd0,0x0d,0x34,0x50,0x2d,0xa4,0xf6,0x3d,0xaa,0xe1,0x18, +0x7f,0x42,0xc8,0xab,0xa2,0x6a,0x8c,0xaa,0x87,0x61,0x3d,0xf8, +0xc3,0x9a,0x23,0xce,0xe7,0x0d,0xda,0xd9,0x81,0x39,0x56,0x35, +0xad,0x44,0x5a,0x8b,0xac,0x12,0x59,0x2d,0xe6,0x5a,0xb4,0xe6, +0x5a,0xb6,0xd2,0x1c,0xbe,0xd2,0xf5,0x3a,0xc9,0x7f,0x8c,0x17, +0xe9,0x55,0x81,0x71,0x19,0xc6,0x65,0x57,0xa5,0x1d,0x57,0xc4, +0x11,0x91,0xa1,0x5e,0x85,0xa0,0x36,0x39,0x8f,0x57,0x67,0x9b, +0x54,0x24,0x94,0xa6,0x9e,0x7a,0xb0,0x53,0x2a,0xf4,0x17,0x8c, +0x5e,0x8b,0x2b,0xed,0xbf,0xb8,0x56,0x51,0xb5,0x8e,0x22,0x47, +0x40,0x45,0x34,0x29,0x62,0x40,0x35,0x15,0x45,0x54,0xc4,0xdd, +0x22,0xfe,0x17,0xc6,0xa1,0xa7,0xf9,0x8f,0x9f,0xa6,0xc7,0xc7, +0x49,0xfc,0xb4,0xc0,0x89,0x13,0xa7,0x4a,0x4e,0x0f,0x38,0x37, +0xab,0xa0,0x45,0x60,0x79,0x57,0x97,0x09,0x8d,0x19,0x37,0x96, +0xf1,0xa8,0x14,0x54,0x91,0xba,0x2b,0x88,0xaf,0x24,0xa4,0x55, +0x94,0xe6,0x27,0xc0,0x29,0x60,0x13,0xfe,0xad,0x23,0x75,0x69, +0x8f,0x52,0x6b,0x4c,0x05,0x56,0xd6,0x98,0x5a,0x73,0xd5,0xf4, +0xa1,0x1c,0x86,0x90,0xd7,0x59,0x15,0x70,0x5d,0xf5,0x71,0x1c, +0x86,0x54,0x4a,0xed,0xe4,0xa9,0x29,0x0f,0x62,0x0c,0x98,0x19, +0x3a,0x4f,0x4d,0x79,0x28,0xa5,0x45,0x00,0xe7,0x55,0x87,0x8a, +0x9b,0xd7,0x1d,0x42,0x41,0x61,0xc8,0x40,0x61,0x3c,0xde,0xf8, +0x83,0xc4,0x8c,0x2f,0xf7,0x34,0x24,0x2d,0x17,0xa5,0xc8,0x2b, +0x91,0xd7,0xec,0x8c,0x4a,0x49,0x7d,0xc1,0x9b,0x7f,0xe4,0x06, +0x0a,0x9d,0x2d,0xee,0x0c,0x6a,0xa3,0xe2,0x26,0x45,0xab,0xe8, +0x8c,0xaf,0x85,0xe1,0xbe,0x79,0x91,0x78,0xf4,0xd6,0xbc,0x17, +0xe6,0x96,0xcb,0x60,0x44,0x71,0xfa,0x24,0x38,0x1d,0x8a,0x32, +0x7a,0x7c,0x92,0x0b,0xbb,0x27,0x76,0x4b,0x51,0xc3,0xbf,0xaa, +0x53,0x43,0x77,0x2e,0x83,0x11,0x39,0x02,0x1c,0x11,0xa6,0x45, +0x1a,0x5f,0x66,0x78,0x24,0x99,0xd7,0x4d,0x64,0xfe,0xf0,0xa4, +0x8c,0xd7,0x8f,0xfb,0x78,0x44,0x53,0x45,0x79,0xcd,0x21,0x17, +0x31,0x54,0xa9,0xee,0xe6,0x38,0x52,0xe0,0x57,0x27,0xaf,0x1d, +0xe4,0xc5,0xbb,0x2b,0x98,0x95,0xbc,0xb0,0x6c,0xa0,0x06,0x84, +0x51,0x09,0x94,0xba,0xa8,0x10,0x13,0x22,0x42,0x3c,0x0e,0x9a, +0x12,0x37,0x58,0x41,0x16,0xbb,0x8b,0xec,0x66,0x0d,0x94,0x56, +0x4d,0x4a,0xf1,0xeb,0x31,0x20,0xad,0xa3,0xbc,0x3a,0xa1,0xc0, +0x0e,0xf2,0x83,0x74,0xea,0x02,0xd0,0x14,0xf4,0x77,0x45,0xbf, +0x60,0xb7,0x39,0xbb,0x2d,0x25,0x6f,0x09,0x07,0x87,0x78,0xa0, +0x0e,0xc0,0xfe,0x20,0x39,0x25,0x6d,0x4e,0x49,0x29,0xac,0x61, +0x7e,0x64,0xef,0x6c,0x3c,0x4b,0x59,0x93,0x11,0x6e,0x38,0x2f, +0xe8,0xcb,0x9a,0x67,0x28,0x92,0xbe,0x68,0x12,0x53,0x7b,0xd4, +0x7b,0x36,0xa5,0x31,0x49,0xc4,0x6a,0xce,0x12,0x33,0x6b,0xfe, +0x11,0xcb,0x48,0x6b,0xc9,0x7a,0x7f,0x94,0x32,0x45,0x2d,0xbb, +0xf6,0x7e,0x06,0x2b,0x34,0x72,0x7e,0xf6,0x0c,0x31,0x45,0x31, +0x1a,0xf5,0x93,0x77,0x05,0x8e,0xfd,0x52,0x62,0x8d,0x66,0x90, +0x40,0x2f,0x24,0xcf,0x92,0x32,0x9d,0x7b,0xa0,0x1b,0x2c,0xd1, +0x21,0x77,0x71,0x95,0x27,0xaf,0xa8,0xde,0x04,0x0e,0x40,0xa0, +0xe4,0xf2,0x01,0x96,0x35,0xe3,0xc6,0xe3,0xaa,0x1b,0xb9,0xd3, +0x6d,0x0c,0xcb,0x0f,0x8c,0x0c,0x04,0x39,0x7a,0xc0,0xb7,0x66, +0xc6,0x35,0x8c,0xad,0xa7,0x4f,0xec,0x69,0x15,0xe2,0xba,0xd1, +0x79,0x68,0x4d,0xc1,0x10,0x13,0xd5,0x5d,0x98,0x7d,0x3b,0xe7, +0xa7,0x83,0x7e,0x08,0xc9,0x1f,0x9f,0x40,0x26,0xb1,0x67,0xa6, +0x85,0x31,0x89,0x24,0xf1,0xf8,0xf8,0x91,0x6a,0x10,0xdd,0xaa, +0xd2,0x89,0x9e,0x84,0x50,0xef,0x2d,0xd1,0x8b,0x41,0x72,0xd0, +0x90,0x6d,0x23,0x5c,0x31,0x50,0x88,0xc6,0xfd,0x70,0x60,0x4d, +0xe5,0xb4,0xd2,0xf3,0x97,0xaa,0xfd,0xd3,0xf5,0x7c,0x75,0xb5, +0x48,0x5e,0xf2,0xe5,0x2e,0x96,0x02,0x7c,0xd9,0x83,0x5c,0x26, +0xb1,0x37,0x93,0x17,0x7a,0x09,0xae,0x2e,0xf5,0x2e,0xe5,0x65, +0x30,0x9d,0x32,0x52,0x9e,0x28,0x2f,0xc9,0x11,0x52,0xf3,0xfd, +0x0d,0x49,0xd8,0xd5,0x9a,0x7c,0x23,0xa2,0xf7,0xa3,0xb7,0x53, +0xb9,0x7e,0x0a,0xac,0xac,0xcb,0x9e,0xb3,0x80,0x6d,0xcd,0x2d, +0x61,0x6c,0x04,0xb8,0xe7,0xda,0xa8,0x85,0xc1,0x29,0x49,0x3e, +0x88,0x53,0xdd,0x04,0xb6,0x70,0xf2,0x40,0xbd,0x54,0x77,0x88, +0x54,0x50,0xdd,0xcc,0x91,0xd9,0xe3,0xc5,0xc2,0x97,0x30,0x42, +0x25,0x2a,0x8b,0xf5,0x65,0xaf,0x32,0xf9,0x6a,0x27,0x0b,0xae, +0x7f,0xb6,0xee,0xc4,0xa5,0x1f,0x2c,0x05,0x7c,0xd4,0xa6,0x4b, +0xa7,0x4d,0xb7,0xce,0x11,0x8e,0xa5,0x38,0xd3,0x16,0x31,0x3e, +0x12,0xca,0xab,0x4f,0x5a,0x8d,0xde,0xdd,0x5d,0x26,0xe5,0x79, +0x86,0x57,0x9b,0xf5,0x16,0xbc,0xda,0xdd,0x94,0x1b,0x59,0x8c, +0xf5,0x0d,0x6e,0xca,0x80,0xf2,0x8e,0x51,0x13,0x8f,0xe3,0xd4, +0x46,0x8d,0xdc,0x29,0xff,0x52,0xf2,0x83,0x1e,0xda,0x89,0xac, +0xbc,0x62,0xc3,0xc1,0x46,0x3e,0x3d,0x1f,0x1f,0x1f,0x6f,0x66, +0xbd,0xf9,0x2a,0x89,0x73,0x3f,0x90,0x6c,0x9e,0x05,0x76,0xe1, +0x44,0x2a,0xd7,0x61,0x36,0xd6,0xa0,0xa1,0x53,0x2e,0x55,0x4c, +0x40,0x9f,0xde,0xc1,0x65,0x21,0x72,0xb0,0x1a,0x46,0x61,0x8d, +0x25,0xa3,0xe8,0xec,0x63,0x66,0x30,0xde,0x4a,0x0a,0xf8,0x0f, +0xb0,0x0b,0x39,0xc4,0x81,0xe3,0x63,0x2b,0xa0,0xca,0xbc,0xbb, +0x73,0xa8,0x1a,0x99,0x5d,0x39,0xda,0x30,0x76,0x18,0xcc,0xaf, +0x05,0xab,0x06,0x30,0x2c,0x2f,0xac,0x73,0x17,0x94,0x58,0x7d, +0x45,0x89,0x2d,0x78,0xf5,0x15,0x25,0x32,0xf8,0x55,0x4b,0xa2, +0x1d,0x56,0xd1,0x3a,0x49,0xc9,0x28,0x59,0x91,0xd8,0x7b,0x7c, +0x4e,0xef,0x31,0x5b,0x7c,0x8e,0xac,0x37,0xe4,0x47,0x1f,0x33, +0x10,0x63,0xe4,0x35,0x59,0x87,0x3d,0xea,0xa5,0xdd,0x82,0xee, +0x5b,0xf0,0xb6,0x43,0x8f,0xdf,0x49,0xc5,0x48,0xa1,0x97,0xf1, +0xcd,0x64,0xc9,0x70,0xb5,0x3e,0x6d,0x26,0x63,0xbc,0x50,0xeb, +0xc3,0x66,0x22,0xcd,0x10,0xac,0xc0,0xaa,0xb7,0xd2,0x60,0xee, +0xa5,0x39,0xd8,0x9a,0x51,0x1f,0xe3,0x7b,0x90,0xa0,0x9b,0x44, +0xca,0x3b,0x84,0xff,0xb8,0x6f,0x8f,0x8f,0x30,0x67,0x9c,0xf1, +0x69,0x9e,0x90,0x0d,0x12,0xdd,0xf2,0x55,0x1d,0xde,0x30,0x63, +0x8d,0x2d,0xe4,0x3f,0x82,0x56,0x6d,0x21,0xfd,0x0a,0x5c,0x77, +0x86,0xf8,0x23,0x70,0xe5,0x13,0xe2,0x0f,0x54,0x64,0x1e,0xc2, +0x3f,0x72,0x37,0x1a,0x92,0xaf,0x6a,0xb9,0xe2,0x85,0xce,0x8f, +0x95,0x09,0x75,0x8d,0xe8,0x6d,0x49,0x65,0x52,0x72,0x8b,0x2c, +0x0d,0x8d,0x34,0x92,0x34,0x84,0xa6,0x19,0x2c,0x7c,0x16,0x6d, +0x5b,0x67,0xab,0x29,0x59,0xd3,0x05,0xd1,0xb6,0xad,0x26,0x32, +0x5c,0x23,0x28,0xdc,0x32,0x14,0x34,0x8e,0x08,0xdc,0x4e,0x4a, +0x8a,0x92,0x05,0x4f,0x6f,0xc8,0xc8,0xd8,0xe3,0x63,0x8a,0x1e, +0x5f,0x6d,0x16,0xe8,0x21,0xda,0x6c,0x06,0x1e,0xc6,0x12,0x32, +0x11,0x3b,0xf0,0x7b,0x72,0x05,0x9c,0x60,0xfa,0x23,0x17,0x77, +0x1f,0xa9,0xdc,0x55,0xf4,0x66,0x22,0xc7,0x5a,0x1d,0x8e,0xc6, +0x0b,0x79,0x6b,0x1b,0xcf,0x8b,0xe9,0x77,0xf4,0x80,0x4e,0x40, +0x9b,0xf9,0xee,0x76,0xa7,0x75,0x42,0x29,0x2f,0xf9,0xf0,0x09, +0x17,0x1d,0x5d,0x9b,0x34,0xf7,0xe8,0x9a,0x92,0x23,0x79,0x70, +0xcd,0x47,0xd6,0xb2,0x0a,0x53,0x4a,0x02,0x7d,0x01,0xff,0x30, +0x9b,0xae,0x97,0x36,0xc1,0x7c,0x79,0xb2,0xb9,0x81,0xc8,0x87, +0x1c,0x7c,0x38,0x60,0xb8,0xc5,0xcc,0xf6,0xa5,0x9c,0xee,0x61, +0xf1,0x2d,0x43,0xd1,0x2c,0x6c,0xb5,0x8b,0x95,0x49,0xee,0xde, +0xec,0x6b,0x1d,0x73,0xe8,0x25,0xcf,0xd9,0xc6,0x8c,0x91,0x4e, +0x22,0x5a,0x25,0xc3,0xaa,0x89,0xc6,0x15,0x36,0x6b,0x84,0x58, +0xb9,0xa1,0x77,0x6a,0xc4,0x33,0xf4,0xc3,0xea,0xf5,0x61,0x54, +0x88,0xdf,0xc0,0x8c,0xfd,0x15,0xa0,0xc9,0x1c,0x0f,0x9f,0x9a, +0x15,0xe1,0xc3,0x1a,0x43,0x12,0x9f,0x8b,0x36,0x2b,0xb2,0x73, +0x7e,0xa3,0xc8,0xa1,0x8b,0x3e,0xe4,0x73,0x52,0x6b,0x58,0xf4, +0xf5,0x35,0xa0,0x02,0x75,0x04,0xff,0x71,0x24,0x7c,0x40,0x9c, +0x23,0xe1,0x56,0x65,0x76,0xe5,0x5c,0x9f,0xa2,0xd8,0x60,0x28, +0xf2,0xf2,0x0e,0x92,0x7d,0x00,0x83,0x5b,0xd2,0x56,0xd0,0xf8, +0x94,0x6c,0x3a,0x88,0xa3,0xba,0xa9,0x73,0x9f,0xd0,0x0d,0x0a, +0x93,0x8b,0x7c,0x69,0xd9,0x37,0x9e,0x7c,0x93,0xa6,0xea,0x4c, +0xc6,0x08,0x6f,0xe8,0x50,0x54,0xdd,0xe5,0xc1,0xeb,0xa2,0x15, +0x39,0xe9,0x62,0xa5,0x93,0x48,0x62,0x1f,0x04,0xbe,0x09,0xa0, +0x6f,0x2c,0xc3,0x2c,0x0d,0xa5,0xd5,0x53,0xbf,0x11,0x43,0xf0, +0xd7,0xb6,0x07,0x50,0x5f,0x95,0xc4,0x8c,0x77,0x0e,0xb3,0x28, +0x6a,0xd7,0x00,0x2b,0x46,0xdf,0xbd,0xb2,0x5d,0xa2,0x18,0x87, +0x02,0xdf,0x22,0x02,0x1d,0xec,0xb8,0x47,0xf2,0xbe,0x6a,0x43, +0xab,0x9a,0x36,0xb8,0x9b,0xd9,0x3e,0x65,0x12,0x3b,0xc3,0xab, +0x75,0xca,0xd7,0xb3,0x73,0x69,0x39,0xe1,0xf1,0x84,0xfc,0x63, +0xa8,0xa3,0x31,0xdb,0x12,0x65,0xf7,0x80,0xec,0x60,0x11,0xfa, +0x58,0x4c,0x9f,0x1f,0x62,0xc0,0x95,0xe7,0x91,0x6d,0xab,0xb0, +0xaf,0xaf,0x60,0x7e,0xa9,0x51,0x73,0xe3,0xae,0x6c,0xad,0xfc, +0x41,0x42,0xdc,0x2a,0xc0,0x80,0x46,0xd6,0x4f,0x7b,0xa2,0xf0, +0x9d,0x30,0xb5,0xb8,0xea,0x38,0x0e,0xa8,0x2b,0x22,0x56,0x9c, +0x2b,0x24,0xca,0x2f,0x91,0x45,0x34,0x75,0x46,0x57,0x52,0x20, +0x66,0x57,0x50,0x56,0x28,0x28,0xdc,0x57,0xa3,0x95,0x94,0x13, +0x45,0x4a,0x8b,0x98,0x30,0xa4,0x2d,0x25,0x16,0xb0,0x93,0xb3, +0x45,0x46,0x78,0x0c,0xdd,0xd3,0x8c,0x56,0xe5,0x24,0xd3,0x6c, +0x51,0x19,0x3d,0x64,0xa4,0x6b,0x0e,0x75,0x6c,0xcb,0x20,0xed, +0xa3,0xf6,0x0c,0xa1,0xb7,0x8d,0x29,0x1a,0xe7,0x40,0x39,0x93, +0xba,0xf3,0xe7,0xdd,0xdd,0xf3,0x6f,0x7f,0x78,0xf5,0x8b,0x3c, +0xf7,0x27,0x88,0x1e,0x9b,0x3f,0xb0,0x82,0x20,0xa3,0x38,0x20, +0x92,0x2a,0x2d,0x15,0x10,0x7c,0xf2,0x6d,0x72,0x59,0x22,0xbe, +0x81,0xa8,0xe7,0x62,0x3d,0x09,0x07,0x0f,0xd0,0x05,0x76,0x26, +0x68,0xa3,0x13,0x28,0x4e,0xd9,0xba,0xf5,0x28,0xfd,0xc2,0x41, +0xb4,0xab,0x4e,0x34,0x8c,0x03,0xf0,0x84,0x11,0x46,0x33,0xaa, +0xd6,0xf1,0x31,0xfd,0x69,0xd1,0x07,0x5c,0x7b,0x06,0x3e,0x80, +0x74,0xe2,0x1e,0x29,0x3b,0x03,0x66,0x86,0xfa,0x7b,0x40,0xb5, +0x68,0x35,0x52,0x40,0xc6,0x01,0x15,0xf0,0x7b,0x20,0x27,0xfd, +0x39,0x3e,0x7e,0x84,0x60,0xe3,0x3d,0xee,0x1d,0xa8,0xe1,0x82, +0x90,0x52,0xef,0xee,0x64,0xe5,0xbb,0xdd,0xa9,0x34,0x07,0x43, +0xd3,0xed,0x3f,0xe8,0x10,0x96,0x8b,0xb6,0x9e,0x0a,0x41,0xad, +0xd7,0xd1,0x90,0xe9,0x50,0xbe,0xf1,0xd0,0x8a,0xf3,0x50,0x15, +0x9d,0xd9,0xcb,0x53,0x78,0x41,0x67,0xec,0x6c,0x02,0x69,0x8e, +0xe9,0xe5,0x07,0x1b,0x5b,0x53,0x1a,0x9d,0xcd,0xf3,0x61,0xbd, +0x3a,0xa1,0xa7,0x9c,0x74,0xa1,0x93,0x82,0xde,0xfc,0x6a,0x96, +0xce,0xbb,0xe9,0xba,0x9b,0x5d,0xd1,0x0b,0xf4,0x98,0x8d,0x9b, +0x09,0xd6,0x1a,0x68,0xe3,0x41,0x07,0x85,0x5b,0xfb,0xc5,0xb5, +0x64,0xe3,0x27,0xab,0x78,0x53,0x24,0xd6,0x19,0x38,0xdb,0x02, +0xe1,0xe1,0xbc,0x38,0x13,0x09,0x9d,0x4e,0x90,0x0d,0x9d,0x8d, +0x8f,0xad,0x80,0x30,0x9a,0xdf,0xfb,0x20,0xa3,0x95,0x35,0xbd, +0x99,0xa3,0xea,0xe0,0xd4,0x48,0x14,0x25,0xf4,0x8e,0x47,0x7d, +0xb9,0x97,0x81,0x79,0x46,0x84,0x82,0xde,0x1a,0x71,0x51,0x04, +0xd2,0x6e,0x8d,0xd2,0x7a,0x86,0x1a,0x56,0x83,0xe9,0x13,0x37, +0x25,0xce,0x22,0x45,0x7a,0x97,0x6d,0x13,0x4e,0x75,0x61,0xfd, +0xa7,0x67,0x01,0x17,0x38,0xe0,0x6b,0x15,0x58,0xb5,0xb3,0xa7, +0xe8,0xbb,0xe4,0x2c,0x1a,0x80,0x76,0xc6,0xa8,0x89,0x6b,0x67, +0xea,0xe9,0x0b,0x8a,0x6a,0x1e,0xf9,0xe3,0xa6,0x02,0x26,0xe0, +0xe0,0x27,0xc7,0x8d,0x7a,0x1a,0xc9,0x18,0xe9,0x1a,0x64,0x8f, +0x02,0x3d,0x1a,0x80,0xd8,0x9f,0x8d,0x7d,0xc2,0xcb,0x52,0xc9, +0xab,0x1d,0x2d,0xa2,0x42,0x57,0x99,0x0b,0x27,0xfe,0x69,0x5e, +0x04,0xa1,0xac,0xc2,0xd6,0x32,0xad,0xec,0x19,0xff,0xa6,0xc4, +0xd2,0x86,0xad,0x2c,0x77,0x96,0x03,0xc6,0xb2,0x56,0x3f,0x32, +0xb2,0x49,0x2c,0x96,0x37,0x3b,0xf5,0x23,0x25,0xd4,0x30,0x44, +0x11,0xe7,0x61,0x19,0xa0,0x93,0x66,0xe4,0x26,0x80,0xa0,0x11, +0x8e,0xcc,0x45,0x44,0x13,0xb3,0xe8,0xbd,0x0f,0xaa,0x92,0x7c, +0x09,0x97,0x22,0x30,0x1d,0x46,0x10,0x8c,0x87,0x36,0xf0,0x70, +0x81,0x29,0x1f,0xfd,0x18,0xeb,0x5c,0x94,0x1a,0xf2,0xbd,0x9d, +0x0d,0x5a,0xa1,0xd8,0x7d,0x5e,0x53,0x13,0x48,0x72,0x74,0x6b, +0x90,0x3d,0x15,0x3d,0xb3,0x08,0x73,0x2a,0xd1,0x08,0xd3,0xe8, +0xff,0xcf,0xde,0xf7,0xff,0xb6,0x71,0x2b,0x89,0xff,0xfe,0xf9, +0x2b,0x92,0x6d,0x90,0xee,0x7e,0x44,0xcb,0x52,0x52,0x1f,0x70, +0xeb,0x6e,0x85,0x5e,0x9b,0x7e,0xc1,0xb5,0xf7,0x8a,0x26,0xaf, +0xef,0x00,0xc3,0x08,0x14,0x6b,0x15,0xa9,0x91,0x25,0x9f,0x24, +0xcb,0x72,0xe4,0xfd,0xdf,0x8f,0xf3,0x8d,0x1c,0xee,0x72,0x65, +0x39,0x75,0xd1,0x03,0xde,0x2b,0x50,0x47,0x4b,0xce,0x0c,0xbf, +0x0f,0x87,0xc3,0xe1,0x4c,0xc6,0x56,0x4e,0x08,0xaa,0x6e,0x7a, +0xc9,0x0c,0x00,0xe7,0x2d,0x8f,0x85,0x63,0x50,0xc1,0x1a,0xe7, +0xc8,0x95,0xb8,0x23,0xd3,0x8c,0x25,0x14,0x8a,0x67,0xe9,0xf8, +0x2d,0xf6,0x09,0xad,0x4c,0x7f,0x44,0x83,0xcf,0xbb,0x3b,0xb7, +0x66,0x09,0x06,0x7b,0x44,0x2f,0x20,0x1a,0xec,0x46,0x0a,0x96, +0x43,0x4b,0x37,0xb0,0xc1,0xc5,0x78,0xef,0x6e,0xc0,0x94,0x2f, +0x5e,0x5e,0xb3,0x44,0x9c,0x56,0xe5,0xda,0x2d,0xd2,0x70,0x5d, +0x1d,0xb4,0xf6,0xdc,0x9a,0x6b,0xef,0x19,0x35,0xa1,0xed,0x8a, +0xce,0x4e,0xd7,0x6e,0x52,0x16,0xee,0x2d,0x0a,0xbf,0xe8,0x52, +0x11,0xba,0x1c,0x9b,0x51,0xe1,0x76,0x58,0x6c,0x30,0x92,0x07, +0xef,0x18,0x70,0x0a,0x07,0xec,0x74,0x1d,0x61,0xa7,0x76,0xa9, +0x33,0xe3,0x8c,0x19,0xd3,0x7a,0x06,0xec,0x01,0x95,0xc1,0x2a, +0xc8,0xac,0x2d,0x01,0x5e,0x41,0x17,0xee,0x85,0x6c,0x64,0x97, +0x81,0x2b,0xfc,0xb7,0x20,0x4d,0xa9,0x07,0xfa,0xb6,0xd8,0xd7, +0xf8,0x24,0x78,0x31,0x1b,0xd1,0x0f,0x7c,0xb9,0x74,0xea,0xde, +0x03,0x15,0xfa,0x71,0xfe,0x20,0x15,0xf8,0x62,0x87,0xa9,0x79, +0xed,0x9d,0xbc,0x21,0xb3,0xe7,0xbc,0x7b,0xd2,0xa9,0x3d,0x54, +0x4e,0xb3,0xe3,0x17,0x95,0x2b,0xa5,0x10,0x3a,0x59,0x7e,0x38, +0x49,0x85,0x1e,0x07,0x15,0xcb,0x38,0x85,0x92,0x51,0x7b,0xec, +0x1c,0x78,0x0b,0xcd,0x47,0xec,0x5f,0xf1,0xdd,0x1c,0x75,0xc2, +0x29,0x39,0xae,0x21,0x0f,0xd7,0x2b,0x01,0x43,0x97,0xda,0x29, +0xbc,0xd7,0x11,0x6f,0xdc,0x8d,0x1c,0xe7,0x79,0xbb,0x9e,0xe3, +0xbc,0x6f,0x07,0x39,0x94,0x92,0xca,0x23,0x4a,0xc9,0xfa,0x96, +0xde,0x4a,0x02,0xd6,0x7a,0x56,0x7a,0x04,0x70,0xfc,0x98,0x9d, +0xba,0x7a,0x29,0x91,0x8f,0xe4,0x3b,0xbe,0x94,0x2a,0x28,0xbc, +0x54,0xdd,0x8f,0x3f,0xe0,0xa1,0x43,0x12,0x98,0x33,0xbe,0x72, +0xaf,0xe8,0x0c,0x4a,0x1e,0x29,0x8d,0x23,0xee,0x87,0xde,0x4d, +0x86,0xfe,0x71,0x2f,0xdb,0x83,0xea,0xbb,0x25,0x82,0x0b,0xed, +0x47,0x6f,0xf6,0xf8,0xdc,0x62,0x1f,0x15,0xd7,0x85,0x6d,0x54, +0x00,0x20,0xa0,0x82,0xbd,0x18,0x52,0xf1,0xdd,0x7d,0x5f,0x5d, +0x8c,0x0a,0x1f,0xe0,0x28,0xd2,0x00,0x84,0x24,0xe5,0x29,0x37, +0x4c,0x13,0x44,0x7a,0x35,0x1f,0xd5,0x1a,0x63,0xc7,0xa7,0x5e, +0x8d,0xb5,0xac,0x1d,0x13,0x84,0x51,0xf0,0x67,0xff,0xe8,0x68, +0xb2,0x68,0x2e,0x79,0xf8,0x54,0x46,0x23,0xf9,0xae,0x8e,0x60, +0xb9,0xcc,0x26,0x9a,0xeb,0xdb,0x18,0x9a,0x64,0x36,0xd0,0x7c, +0x67,0x46,0xd0,0x5c,0x66,0x03,0x8d,0x9f,0x80,0x37,0x51,0x28, +0x23,0x52,0x8c,0xed,0xa0,0x68,0x09,0x36,0x9d,0xa0,0xdd,0x83, +0x0b,0x7b,0xaa,0xf4,0x33,0x55,0x4d,0x3c,0x35,0x7b,0xd4,0x14, +0xe0,0xa1,0x43,0x42,0xe7,0xa7,0x31,0x4f,0x8e,0xca,0xe0,0x3c, +0x7f,0xda,0x37,0xbe,0x12,0xf9,0x0e,0x57,0x56,0xce,0x63,0xfc, +0x77,0xaa,0x12,0xd5,0x2c,0x4c,0xac,0xf8,0x1d,0x1f,0x99,0xce, +0xc6,0xdc,0x34,0x3a,0xaf,0x4a,0x86,0xa6,0x47,0x3d,0x6e,0x04, +0x4f,0x1a,0x7e,0x66,0xd2,0xd3,0x33,0xa6,0x19,0x62,0xc2,0x65, +0x19,0xcd,0xd6,0x0d,0x86,0x0d,0x29,0x38,0x6a,0x08,0x4e,0x20, +0x88,0x04,0x4c,0x7c,0x04,0x01,0x82,0xc9,0xdf,0x16,0xb9,0x22, +0x5c,0x68,0x7b,0xa1,0x78,0x15,0xec,0x85,0xe1,0xe5,0x55,0x87, +0xe1,0x07,0x90,0xc6,0x87,0x13,0xf1,0x55,0xa4,0xe0,0x21,0xfe, +0xdb,0x45,0x05,0xc1,0x7e,0xb9,0xf8,0xf0,0xf5,0xf2,0xfd,0xf5, +0xa5,0xed,0x2d,0x3b,0x11,0x22,0x25,0xaf,0xce,0xc3,0xb5,0x03, +0x2f,0xf7,0xc2,0x89,0x0a,0x29,0xe1,0x42,0xc1,0x94,0x60,0x0d, +0x10,0x96,0x4c,0x3e,0xf8,0x52,0x33,0x17,0x3e,0x71,0xc7,0xdd, +0xa9,0x78,0x03,0x70,0x34,0xb0,0x23,0x51,0x7b,0xb4,0x03,0x49, +0x3c,0x48,0x03,0xfe,0x37,0x77,0x0a,0xaf,0x70,0xf7,0xa5,0x3a, +0xf1,0xeb,0x28,0xdf,0x2f,0xec,0x09,0x84,0x22,0x5a,0xa8,0xdf, +0x1c,0x35,0x63,0xe5,0xd8,0x9a,0xeb,0x16,0x79,0x1e,0xce,0xcf, +0xee,0xed,0x91,0xd2,0x11,0xa3,0x56,0xba,0xbd,0xcf,0x85,0x5b, +0x21,0x2a,0x58,0x81,0xb0,0xcb,0x9d,0xad,0x31,0xbe,0x4c,0xb0, +0x23,0x86,0x7a,0x2a,0xaa,0x29,0xfe,0x8d,0xe6,0xa3,0xcd,0xf2, +0xe5,0x5a,0xb5,0x85,0xc6,0x54,0xd5,0xbf,0x25,0x61,0x5f,0xab, +0xc8,0x56,0x21,0xf7,0xc0,0x54,0x8f,0x68,0x2c,0x03,0xf6,0x39, +0x5c,0xd8,0x5a,0xa4,0xa3,0xae,0xb8,0x0a,0x41,0xb9,0x12,0x1f, +0xe8,0xe0,0xd8,0x0e,0xce,0x88,0x2d,0xc4,0xda,0x90,0x9f,0xf9, +0x38,0x52,0x4e,0x63,0xd7,0x3b,0xaf,0xf9,0xb8,0xf5,0x16,0xc8, +0x76,0xca,0x0d,0xb0,0x3a,0x16,0xb1,0x52,0x4a,0xbe,0x7e,0x2b, +0x0a,0x42,0x6b,0xd0,0x17,0xed,0xa0,0x30,0x6a,0x1a,0xf4,0x25, +0x83,0x36,0xca,0x51,0x30,0x5f,0xb4,0x91,0xb3,0xb0,0x9a,0xd6, +0x49,0x6b,0xb1,0x6b,0x7a,0xe6,0x39,0x72,0xbe,0xcd,0xf8,0xfd, +0x9b,0x72,0x70,0x5d,0xc8,0x0e,0x59,0x72,0xd8,0xa2,0x02,0xff, +0xc2,0xf7,0xff,0xa3,0x45,0x42,0xc2,0x58,0x34,0x12,0xa8,0x0c, +0xad,0x84,0x30,0xa0,0x61,0x87,0x98,0x70,0x60,0x95,0x8e,0x68, +0x5b,0xc3,0x12,0x2c,0xa9,0x76,0x79,0x6a,0x57,0x3a,0xec,0xd2, +0x61,0xa4,0x09,0x96,0x69,0x33,0xe2,0x16,0xcf,0x7b,0xb4,0x16, +0xfe,0x86,0x49,0xab,0xc1,0xb6,0x93,0x24,0x75,0xe9,0x89,0xdb, +0x58,0xab,0x09,0xa5,0x56,0x2a,0x6e,0xca,0x61,0x35,0x41,0x50, +0xae,0x08,0xa1,0x35,0x1a,0x89,0xc9,0x95,0x8f,0xdd,0xb2,0x8a, +0x0c,0x4e,0x8c,0xb0,0xe6,0x8d,0x2e,0xdf,0xd1,0x54,0xb9,0x55, +0x3d,0x98,0xd3,0xa1,0x55,0x77,0xf0,0x5b,0x45,0x96,0x92,0xaa, +0x7a,0xf8,0xa7,0xc3,0x69,0x32,0xbc,0xa6,0x49,0x49,0x8a,0x66, +0x18,0x27,0x71,0x6b,0xe8,0xcd,0xc4,0xd3,0x3a,0x41,0xf7,0x6a, +0x53,0xef,0x75,0xb8,0xf0,0xe7,0x45,0x1d,0xf6,0xa8,0xcf,0x3c, +0xb7,0xb3,0x65,0xc6,0x38,0xff,0xaa,0x3f,0xe8,0xdc,0xe6,0xa1, +0x94,0x68,0x05,0x14,0x9b,0xd1,0x1b,0x74,0x1c,0xfe,0xd9,0xfc, +0x3c,0x84,0x91,0x00,0x2b,0x41,0x22,0x86,0xee,0xfb,0x1d,0x7c, +0x91,0x05,0x5b,0x2a,0x24,0xc3,0x6f,0x4a,0xe6,0x3d,0xd4,0x26, +0xda,0x62,0xb2,0xe7,0xcf,0x79,0x26,0xd4,0xf6,0x6a,0xcf,0x99, +0xfd,0xce,0x8c,0x44,0x82,0x7d,0x18,0x9c,0xe7,0x43,0x7f,0x55, +0x91,0x88,0x5a,0x87,0x0f,0x86,0x47,0xe9,0xe8,0x01,0x71,0xc9, +0x95,0x8e,0x64,0x76,0xe0,0xe2,0x73,0x82,0x8c,0x8a,0x42,0x39, +0xd8,0xe6,0xae,0x00,0x02,0xa8,0x1a,0x41,0xbe,0x0e,0xaf,0x35, +0x23,0xd8,0x7e,0xec,0xd0,0xc2,0xf2,0x34,0x3a,0x91,0xf5,0xe5, +0x72,0xab,0x46,0xa4,0xb4,0x07,0xac,0x62,0x42,0xf0,0x65,0x7a, +0x1a,0xb1,0x32,0x5d,0x6e,0xa5,0x02,0xbe,0x1d,0x56,0x1a,0x82, +0x32,0xcf,0x20,0xb4,0x06,0x7d,0x4c,0xae,0x7c,0x80,0xa6,0xc3, +0x08,0x03,0x24,0xd3,0x45,0xa4,0x06,0x59,0x48,0xad,0xda,0xc2, +0x3e,0x1d,0x5e,0x86,0x13,0xc7,0x54,0x61,0x5e,0x44,0xdb,0xea, +0xd2,0x5c,0x32,0x17,0x1b,0x89,0x8c,0x75,0x58,0xb9,0xa1,0x8c, +0xc7,0x05,0xd7,0x04,0x3f,0x57,0x72,0x98,0x5e,0xb5,0x86,0xd9, +0x3a,0xb0,0xe8,0x40,0x98,0x94,0xa2,0x43,0x09,0xd3,0x17,0x1d, +0xa4,0x57,0xad,0x21,0xbb,0x0e,0x5f,0x0c,0x5e,0xd6,0x75,0xfb, +0x4c,0x20,0x00,0x6f,0xf5,0x42,0xf0,0xe9,0x55,0x34,0x06,0xd8, +0x03,0xd6,0x43,0x58,0xa4,0x93,0x9e,0xb7,0xc1,0x1a,0xd0,0x45, +0xd5,0x43,0x84,0x1d,0x56,0x96,0x12,0xc5,0xb9,0x30,0x2d,0x9c, +0xbb,0xd2,0x54,0x22,0x17,0x87,0x73,0x3b,0xdc,0x4b,0x71,0xb2, +0x53,0x76,0x85,0xdb,0x44,0x28,0x0f,0x14,0xe8,0xa9,0xbe,0xcf, +0x2e,0xea,0xfb,0xe2,0xa2,0xbe,0x4f,0xae,0xeb,0xfb,0x4e,0x9d, +0x42,0x76,0xb3,0x3b,0x09,0x54,0xc6,0x46,0xaf,0x14,0xc2,0x4c, +0x3e,0x28,0x6a,0x19,0xd9,0xbe,0x4a,0xd8,0x32,0xb4,0xb2,0x75, +0x54,0x9c,0x9d,0xad,0x23,0x84,0xe1,0xcf,0x90,0x4a,0x18,0xfb, +0x8c,0x4d,0x61,0x85,0x0c,0x7f,0x56,0xfe,0x02,0x92,0x2e,0x2a, +0xf7,0xea,0x04,0x09,0xa4,0xa9,0x15,0x1c,0x4d,0x57,0x17,0xb6, +0xef,0xcb,0x94,0xbc,0x6f,0xb2,0xff,0x57,0xa5,0xfe,0x63,0x9b, +0x3a,0x72,0x25,0xf8,0xb1,0x6c,0x24,0xcd,0x74,0xd2,0x7a,0x62, +0xcb,0x9b,0x2c,0x66,0x23,0x49,0x6b,0xf8,0xc8,0xfc,0x36,0x54, +0x40,0x8e,0x57,0xac,0x35,0xf1,0xc2,0x66,0xb1,0x78,0x4b,0x95, +0xa5,0xe3,0x12,0x6c,0xca,0x68,0xb7,0x6f,0xc0,0x99,0xb7,0x21, +0xcd,0x17,0x7b,0x2d,0xe1,0x57,0x94,0x9b,0xc7,0x3a,0x3b,0xf1, +0x75,0x40,0xec,0x88,0x60,0xfc,0x11,0x67,0xfc,0x78,0xc7,0x1b, +0x22,0x05,0xfe,0x83,0x8a,0x13,0xd1,0x0d,0x8e,0x5e,0xb2,0xa7, +0x49,0xf5,0x3e,0x3c,0xc3,0x10,0x93,0x83,0x94,0xb4,0x99,0x98, +0x11,0xb8,0x55,0xf4,0x0b,0x28,0xb4,0x91,0x4f,0xb7,0x74,0x1c, +0x02,0x0f,0x37,0xbc,0x2f,0xb3,0x03,0x43,0x71,0xb0,0x85,0x04, +0xf1,0x6f,0xc3,0x05,0xe8,0xd0,0xbc,0x33,0x53,0xe3,0xa3,0x5a, +0xd3,0x03,0x52,0x78,0x96,0x5e,0xd8,0x3f,0x47,0xfd,0xf3,0xce, +0x47,0xfc,0xe7,0xf8,0x45,0x07,0x82,0x3e,0x18,0xc8,0xe9,0x14, +0xef,0x8e,0x5f,0x18,0x7b,0x8e,0x00,0x97,0x71,0x58,0x47,0x1f, +0x9e,0x0a,0xc2,0x1f,0x72,0x25,0x7c,0x62,0xea,0x2b,0xec,0x05, +0xf1,0x20,0xf0,0x0b,0xea,0xde,0xa4,0xa6,0x2b,0xb0,0xdb,0x07, +0x37,0x97,0xe9,0x78,0x55,0xcc,0x1c,0x33,0xf3,0x0e,0x5e,0x9e, +0x3f,0x1f,0xaf,0xc8,0xed,0x0d,0x54,0x4a,0xdb,0xd2,0xeb,0xf0, +0x34,0xdf,0x31,0x38,0xc0,0x30,0x65,0xd6,0x62,0x05,0xdd,0xaa, +0xc3,0x4d,0xab,0x2a,0x53,0x23,0x6c,0xb3,0xa7,0xff,0x9f,0xba, +0xb1,0xca,0xf8,0xb8,0xb9,0x32,0xab,0x8f,0x05,0xab,0x3b,0x4e, +0x91,0x01,0x02,0x73,0x5c,0x15,0x9e,0x17,0x7a,0x5b,0xfe,0xd5, +0xc7,0x4e,0x71,0xd2,0xb1,0xb9,0xcf,0x9f,0xaf,0xb9,0xca,0x91, +0xea,0x22,0xa6,0x54,0x97,0x5f,0x76,0x73,0xe8,0x52,0x33,0xe7, +0x81,0xab,0xdd,0xfa,0x91,0x8b,0x34,0x18,0x8b,0xd5,0xc7,0x53, +0xef,0xe1,0x7b,0x07,0x4f,0xc4,0x81,0x09,0xbd,0xc7,0x58,0xf1, +0xa4,0x16,0x73,0x7e,0xbf,0xc8,0x9b,0x69,0xfe,0x54,0x5c,0xfa, +0xe5,0x5a,0x0d,0x9a,0xe3,0xdf,0xca,0xd0,0x71,0x3a,0x0d,0xce, +0xd3,0xb9,0x0a,0x56,0x12,0xe8,0x05,0x94,0x2b,0xa2,0x43,0x4f, +0xed,0x66,0x14,0x9e,0x3d,0x2b,0x6d,0x11,0x55,0x9c,0x51,0x18, +0x5e,0x73,0xc8,0x11,0x5d,0x9d,0x66,0xef,0x3b,0x9a,0xd7,0x21, +0x5f,0xec,0x3b,0x6e,0x2b,0x7e,0x76,0x00,0x97,0xe2,0x68,0xa9, +0xeb,0x19,0xdd,0x2d,0x10,0x02,0xab,0xf7,0xd9,0x62,0x56,0x65, +0xbc,0xa6,0x04,0xb0,0x76,0x71,0x17,0x08,0x1b,0x32,0x50,0x91, +0xfb,0x03,0xef,0xea,0x2a,0xc0,0x61,0x65,0x34,0x53,0x34,0x6d, +0x3c,0xb3,0xa6,0x82,0x8d,0x6b,0x79,0x9d,0x0e,0x56,0x81,0x33, +0xe1,0x08,0x3c,0x2f,0xd1,0x3a,0xc2,0xac,0x4d,0x8b,0x3c,0x53, +0x1a,0xe4,0x66,0x25,0xe8,0x72,0x63,0xcb,0xeb,0xb7,0xf0,0x7a, +0xc3,0x38,0xe0,0x6d,0x1d,0xb0,0x59,0xac,0x50,0xa4,0xf9,0xe4, +0x20,0x3b,0x7d,0x13,0x69,0x53,0x08,0xac,0x60,0x1f,0xac,0x43, +0x6e,0x46,0x65,0x12,0x64,0x1e,0x36,0xea,0x01,0xaf,0x5e,0xa6, +0xe4,0x50,0xc1,0x4c,0x0b,0x6c,0x25,0x35,0xe5,0xd6,0x9d,0x47, +0x55,0xce,0xa1,0xcb,0xdf,0xda,0x26,0x0b,0x9b,0xea,0xff,0x04, +0xd3,0x95,0x74,0x90,0x76,0x89,0x17,0xe1,0x0e,0x48,0xdb,0xaa, +0xfd,0x8a,0x6e,0x81,0xe4,0x63,0x4f,0x82,0x69,0x11,0x3b,0xe3, +0x47,0x1d,0x91,0xc4,0x56,0x96,0x26,0xb0,0xf8,0xa6,0xea,0x4f, +0xdd,0x57,0x29,0x68,0x7f,0x94,0xf3,0xf9,0x5b,0x4d,0x8e,0xa3, +0x68,0x39,0x1f,0x38,0x42,0x85,0x11,0x65,0x0e,0x08,0x5e,0x7f, +0xd1,0x4b,0xb0,0xfd,0x07,0xba,0xc4,0xb9,0x78,0x3c,0x67,0xae, +0x78,0x26,0xf3,0x0e,0xdb,0xdb,0xe1,0xaf,0xf3,0xea,0x14,0x9d, +0x26,0x06,0x57,0xa7,0x74,0x94,0xb3,0x95,0x71,0x6e,0x47,0x8b, +0x9a,0x4b,0xd2,0x4c,0x6b,0x2c,0x1f,0x85,0xc5,0xc2,0xe6,0x15, +0x67,0xb2,0xf8,0x84,0xac,0x98,0x16,0xda,0x05,0xcd,0x51,0x7f, +0xc0,0xe2,0x6a,0xde,0x2b,0x8a,0xe9,0x80,0xa4,0x55,0x27,0xd0, +0xd6,0x58,0x31,0x35,0xe6,0xb1,0x79,0x71,0x4d,0x59,0xd9,0xae, +0x23,0x75,0x14,0xd5,0xc4,0x6f,0x4e,0xf0,0xbd,0xec,0x57,0x66, +0xa1,0xcf,0xf9,0x9e,0x53,0x02,0x06,0x3c,0x09,0x19,0x30,0xa2, +0xc9,0x83,0x1e,0x41,0x88,0x2d,0x24,0x78,0x7e,0xcb,0xf3,0x07, +0xde,0xdb,0x36,0xe6,0x90,0x19,0x33,0x06,0xb9,0x6c,0x11,0xc4, +0x33,0xf6,0xbc,0x0d,0xf1,0x09,0x50,0x08,0x3c,0xeb,0x99,0xfe, +0x39,0x19,0xcb,0xac,0x9c,0xdf,0xce,0xa7,0xe1,0xd4,0x6a,0x13, +0x7c,0xd1,0x3c,0x81,0x96,0x49,0xff,0x04,0x8f,0x48,0xa7,0x96, +0x3c,0x20,0x03,0x35,0x0c,0x6f,0x89,0xbf,0xba,0xd7,0xf3,0xd5, +0x64,0x3a,0x5e,0xc3,0xa1,0x15,0x7d,0x60,0x3b,0x10,0xca,0x76, +0x95,0x16,0x78,0xb4,0x4e,0x81,0xf7,0xbf,0x35,0x71,0x44,0x83, +0x9f,0x7a,0x03,0xd4,0x21,0x04,0x54,0x5c,0x58,0xb9,0x2a,0x25, +0xaa,0x60,0x96,0x49,0x61,0x0e,0xdc,0x37,0xdc,0x76,0x53,0x67, +0x36,0x19,0x32,0xec,0x61,0x72,0xd5,0x0d,0x50,0xd5,0xa7,0x6f, +0x64,0xb1,0x42,0xdc,0x1d,0xac,0xe2,0x60,0x0f,0xdd,0xcb,0xc8, +0x8e,0xe0,0xaa,0x88,0xd0,0x37,0xef,0xeb,0x2c,0x93,0xdd,0x88, +0x4e,0x6c,0x3a,0x70,0x4e,0xc7,0x28,0xdf,0x5f,0x71,0x83,0x09, +0x80,0x92,0xc1,0x4c,0xb3,0x6d,0x5b,0xe3,0xfd,0xaf,0x98,0x4c, +0xa8,0xb9,0x3c,0xe1,0xaf,0x8a,0x46,0xaf,0x98,0x71,0x9b,0x0c, +0x5a,0x7c,0xd1,0xb1,0xa2,0xb4,0x97,0x9a,0xef,0xee,0xd6,0x52, +0x0d,0x81,0xe2,0xb0,0x3f,0x7b,0x46,0xc8,0x6d,0xc4,0xab,0x8f, +0xad,0x7b,0xb0,0x82,0x71,0x16,0x61,0x8f,0x2e,0x00,0xb4,0x57, +0xb1,0x41,0xf3,0x80,0xd6,0xdc,0x27,0x56,0x34,0x00,0xff,0x5a, +0x39,0x41,0x5a,0xb4,0x57,0x50,0xc0,0x70,0xda,0xa2,0x0b,0xf0, +0x0f,0xe9,0x49,0x5c,0xa4,0x9f,0xb8,0xde,0x28,0x91,0xcc,0x4c, +0xe8,0x37,0x9d,0xbc,0xe8,0x63,0xa3,0x6e,0x71,0xc7,0xc1,0x0d, +0x2e,0xda,0xb1,0xa0,0xcf,0x63,0xb9,0xfc,0x16,0x25,0xc8,0xa2, +0x7e,0xc1,0x1d,0x44,0x40,0x15,0x1f,0xba,0xcd,0x7c,0x7d,0x7b, +0xac,0x6e,0x83,0x4f,0xce,0x75,0x1f,0xc1,0x2d,0xad,0x92,0x45, +0xe1,0x33,0x58,0x74,0xcd,0x2b,0xde,0x59,0x70,0x45,0xcc,0x1a, +0x14,0xf7,0x4b,0x14,0x25,0xa7,0x41,0xb0,0xd0,0xd8,0xa5,0x2f, +0x1d,0xae,0x56,0xf8,0x70,0x1b,0x3b,0xf1,0xee,0x0e,0x3a,0xd0, +0x7e,0x60,0xe7,0x85,0x04,0x2c,0x59,0x30,0xf3,0x73,0x45,0xc8, +0x1d,0x18,0x60,0x30,0x1f,0x47,0xde,0x0b,0x98,0x77,0x77,0x4e, +0x37,0xa3,0xe2,0xa3,0x0c,0x1a,0x7a,0x94,0xbc,0x26,0xf2,0x65, +0xbe,0x09,0xb5,0x7b,0x3b,0x95,0x1e,0xdc,0xf0,0xf9,0x74,0x19, +0x04,0x37,0xf3,0x25,0xa7,0x32,0x2a,0x40,0xef,0x81,0xf7,0x7b, +0x74,0xd3,0x81,0xd7,0x7b,0x80,0xa4,0xb4,0xcd,0x44,0x2b,0xcb, +0x21,0xbd,0x0a,0xc2,0xf9,0x1e,0x48,0x1a,0x40,0x85,0x36,0xa2, +0xc5,0x88,0x43,0x46,0xa5,0xa3,0xff,0x1e,0x46,0x1c,0x20,0x99, +0x36,0x22,0x45,0x48,0x43,0x7a,0x15,0xc6,0x0a,0x3e,0xb0,0xe2, +0x08,0x2b,0x35,0x27,0xc4,0x58,0xd5,0x31,0xa7,0x0a,0x22,0x01, +0x3f,0xc2,0x7d,0xa3,0x90,0xe7,0x1b,0xc7,0x20,0x52,0xf0,0x81, +0x7d,0x83,0xb0,0xd2,0x3b,0x72,0x91,0xd7,0xec,0x1f,0xbe,0xcf, +0x93,0x1e,0x62,0xd6,0x71,0x58,0x17,0x11,0xb0,0xdc,0xb3,0x08, +0x6e,0x27,0xd6,0x4d,0x94,0xe7,0xca,0x79,0xf8,0x15,0xb1,0xbb, +0x15,0x26,0x0a,0x2d,0xf7,0xc2,0x01,0x79,0x57,0xba,0x5c,0x09, +0x07,0x6c,0xec,0x41,0x57,0x64,0x9d,0xad,0x27,0xc6,0x77,0x45, +0x41,0x74,0xe7,0xc3,0x88,0x6d,0xe4,0x8e,0x56,0x68,0x6d,0xf8, +0x86,0xb6,0x25,0xce,0xf2,0x81,0x97,0x50,0x8a,0xb7,0x7a,0xd2, +0x2a,0xb5,0x6a,0x0b,0xcf,0x7c,0xe0,0x30,0x2b,0x66,0xed,0xe9, +0xab,0xd4,0xaa,0x3d,0xa8,0xf3,0xa1,0x37,0x51,0x9a,0xff,0xfb, +0x32,0x82,0xf4,0xaa,0x25,0x6a,0xf5,0xa1,0x37,0x75,0xfe,0x96, +0xc5,0xf7,0x91,0xba,0x62,0x89,0x47,0xce,0x7e,0xe8,0x25,0x8b, +0x22,0xae,0x2f,0x55,0x98,0x78,0xfb,0x3d,0x07,0x01,0xb8,0x9b, +0x8e,0x60,0x8e,0x17,0x3b,0xb9,0xe5,0xe0,0xcb,0x0d,0xe5,0x45, +0xfd,0xe7,0xc5,0xa8,0x9c,0xc5,0x5f,0x48,0x43,0x4e,0xca,0xcf, +0x3d,0xe0,0xa4,0x25,0x96,0x5c,0xee,0xf1,0x15,0x6e,0xb4,0xf8, +0x45,0xae,0xd4,0x55,0x36,0x55,0xf4,0x69,0xbf,0x66,0xf4,0x8f, +0x34,0xf7,0x19,0xfd,0x63,0x31,0x63,0xe5,0xc8,0x75,0xb5,0xaf, +0xd7,0x7c,0xcd,0xe0,0x8f,0x21,0x07,0xd3,0x3e,0x55,0x9b,0xa5, +0x37,0xbd,0xad,0xdf,0x63,0x66,0xe0,0x5a,0xc9,0xaf,0xa8,0x3f, +0x40,0xa8,0x6c,0x4f,0x9b,0x0e,0xe7,0x10,0x2d,0x1b,0xfd,0xe2, +0x82,0x33,0xfa,0x46,0x2e,0xfe,0xaa,0xb9,0x74,0x4c,0x13,0x50, +0x50,0x24,0xc8,0xd9,0xe6,0xf0,0x20,0xa0,0xe1,0x77,0x32,0x70, +0x0f,0x3b,0x3c,0xfb,0x80,0x40,0x10,0x69,0xf8,0xf9,0x73,0x2c, +0x80,0x14,0x02,0x29,0xd4,0xc7,0x10,0xc4,0xb9,0x0f,0xa4,0x5d, +0x33,0xc6,0x27,0xd8,0xd0,0xd1,0x38,0xe0,0x4d,0xe7,0x57,0xd7, +0x6b,0x37,0xb4,0x40,0x84,0xa2,0xdb,0xad,0xb7,0xf4,0x63,0x20, +0x3f,0x52,0x84,0xd4,0x0f,0xee,0x54,0x23,0x51,0x24,0x05,0x8f, +0xde,0x0e,0x64,0x54,0x82,0x97,0xc5,0x72,0x7e,0x01,0xe2,0x2d, +0x17,0x96,0x05,0xc3,0xa0,0x00,0x1a,0xd5,0x92,0xd7,0x70,0x5b, +0xb3,0x5a,0x5c,0x2f,0x2f,0xca,0x46,0x87,0x52,0xb2,0xc4,0xdd, +0x43,0x73,0xaf,0x5a,0x0b,0xe0,0x60,0x8a,0x40,0x03,0xfa,0x87, +0x3b,0x37,0xef,0x35,0xdc,0x7a,0x42,0x78,0x01,0x44,0x7c,0xeb, +0x43,0x4c,0x90,0xce,0x04,0xad,0x19,0xde,0x2c,0x4b,0xfe,0x86, +0xd7,0xb9,0x6f,0x2f,0x87,0x1f,0xca,0xb7,0x6b,0x48,0xdb,0xf2, +0x6b,0x25,0x1e,0x06,0xae,0xd2,0xf4,0x1c,0xba,0x35,0x6c,0x2b, +0xa9,0xd0,0x5c,0x23,0x29,0x52,0xf2,0x2e,0x9c,0x5f,0x25,0x9c, +0xc7,0xd4,0x44,0x66,0x24,0x82,0x35,0x1a,0xa8,0xde,0xef,0x21, +0x1a,0x19,0xa8,0x46,0x90,0x71,0x65,0x3a,0x58,0xfd,0x34,0xb4, +0x77,0x1e,0x83,0xe7,0x85,0xdb,0x6b,0xbc,0xeb,0xe0,0xf0,0x1a, +0xae,0x39,0x61,0x0c,0xf5,0xd6,0xf6,0x30,0x9a,0xf8,0x78,0x79, +0x68,0x8b,0xa2,0xe8,0x7b,0xdb,0x14,0xc5,0x68,0x6d,0x15,0x31, +0xae,0xc0,0x6d,0xf9,0xbd,0x0c,0x87,0x99,0x1d,0xb8,0xf9,0xd6, +0x1c,0x87,0xfc,0x36,0x28,0xda,0x0d,0x67,0xe3,0x68,0xe8,0x4c, +0xaf,0x5f,0x70,0xf2,0x5e,0xf2,0x92,0x22,0xe3,0xe7,0x4b,0xdd, +0x70,0xf7,0x60,0x4f,0x8a,0x0b,0x1f,0xc2,0x92,0xba,0xe7,0x92, +0x30,0xc9,0x55,0xb6,0x40,0x66,0xee,0x57,0x7d,0x9c,0xd5,0x1c, +0x89,0xc0,0x70,0xbf,0xe9,0x6e,0x57,0x50,0xda,0x9b,0x77,0xbf, +0xd1,0x8b,0x75,0x3f,0x1b,0x4d,0x17,0x15,0xc2,0x6b,0x70,0x20, +0x64,0xe8,0x9c,0x49,0x54,0xb0,0x65,0xb0,0x41,0x78,0xe1,0x7d, +0x17,0x14,0x52,0x0f,0x0a,0xd7,0xc2,0x1d,0xa5,0x36,0x31,0xed, +0xed,0x43,0xba,0xc6,0x23,0x50,0x8f,0x46,0xfc,0x66,0x34,0x1f, +0x2f,0x45,0xf6,0x57,0x35,0xbb,0x79,0xde,0xd5,0xbc,0xce,0xab, +0x3c,0x13,0xbc,0x53,0x23,0xaf,0x1c,0xfc,0x46,0xdf,0xfd,0x8c, +0x3c,0x96,0x07,0xa3,0x86,0x9d,0xb7,0x63,0x48,0xe1,0x71,0x91, +0xd4,0x0d,0x77,0x4c,0x7e,0x63,0x04,0x81,0x5a,0xea,0xf3,0x69, +0x13,0xa4,0x95,0xb3,0x30,0xbc,0x0c,0xf7,0x7a,0x39,0xd3,0x7b, +0x33,0xcd,0x4b,0x1e,0xc6,0xee,0x5b,0x2c,0x41,0xd4,0x9c,0x28, +0x0e,0x70,0x06,0xcd,0x1a,0xfa,0xfd,0x56,0xb1,0x0a,0xf5,0xd2, +0xf3,0x2d,0xcf,0x1a,0x06,0x0a,0xd6,0x9e,0x06,0x1b,0x5e,0xaf, +0x17,0x70,0xf1,0x2d,0x65,0xca,0xd9,0x13,0x8d,0x30,0x7a,0x24, +0x96,0xf4,0xc4,0x06,0xa3,0xc7,0xc6,0x0f,0x3d,0x91,0x2e,0x5c, +0x04,0x9f,0x40,0xc0,0x00,0x6f,0x69,0xe5,0x52,0xa7,0x4d,0x86, +0xf3,0xd1,0x2c,0x4c,0x9a,0x2e,0xbc,0x8b,0x1c,0x53,0xce,0xdc, +0x16,0x3a,0x5d,0x4f,0x87,0x33,0xb8,0x34,0x2f,0x67,0x9a,0x67, +0x63,0x6c,0x9c,0x7d,0xf2,0xc9,0x5e,0x9e,0xde,0xc6,0x30,0xa8, +0xf3,0xec,0x29,0x87,0x2f,0x50,0xd2,0xfd,0xfd,0x6b,0x64,0xd0, +0x9a,0xb5,0x95,0x9c,0x2e,0xf9,0x92,0xf8,0x2f,0x74,0xf1,0xae, +0x86,0x91,0xea,0x97,0x86,0x3b,0xc0,0x7a,0x39,0xbd,0x70,0x6b, +0xce,0x0f,0x45,0x96,0x05,0xec,0x8b,0x6a,0xf0,0x00,0x8e,0xdf, +0xda,0x5a,0x82,0xb3,0xcd,0x15,0xaf,0x5f,0xe9,0xbd,0x73,0xe5, +0x8f,0x34,0x99,0x28,0x04,0x65,0x3c,0xb4,0xe9,0x84,0xa5,0xdb, +0x7e,0x55,0x37,0x53,0x04,0xab,0x87,0xfb,0x5a,0x7e,0xe5,0x4c, +0x15,0xed,0x2f,0x89,0x24,0xc8,0x8e,0xf8,0x00,0x7f,0x50,0xaf, +0xc7,0x27,0xad,0x06,0x6a,0x14,0x45,0x34,0xbd,0xc0,0xb7,0xa9, +0x40,0x3b,0xaf,0xd1,0xee,0xd5,0x68,0x83,0xf1,0x43,0x80,0x0f, +0x01,0x18,0x7c,0xaf,0xa7,0xe1,0xaa,0x02,0x46,0xe4,0x3b,0x5f, +0xcf,0x4a,0x1e,0x2f,0x34,0x02,0x09,0x56,0x9d,0x8b,0x83,0x05, +0x59,0xb5,0x0e,0xbe,0x12,0x93,0x4c,0xdf,0xc1,0x52,0x4f,0xd7, +0xc1,0xe8,0x60,0x6c,0x3a,0x0e,0x9b,0xf1,0x65,0x3f,0x08,0xd6, +0x52,0x6b,0xe2,0x29,0xb9,0xd8,0x1a,0x15,0x41,0x29,0xe6,0x1d, +0x47,0x6e,0xa1,0xf3,0x0b,0xf1,0x6a,0xfb,0xaf,0x73,0xdc,0x57, +0x7b,0x7a,0x43,0xd4,0x7e,0x9c,0xa3,0x76,0xac,0x78,0xd7,0xdd, +0xf6,0xbf,0xec,0x0d,0xc8,0x81,0x6b,0x39,0x9d,0xa5,0x47,0x90, +0x92,0x75,0x10,0xcd,0x0e,0xc6,0xda,0x42,0xdc,0x36,0x20,0x6e, +0x15,0xc4,0x12,0x68,0xbc,0xf8,0x4a,0x75,0x9d,0x82,0xed,0x40, +0xde,0x91,0xca,0xf3,0x78,0xef,0x80,0xb2,0xe0,0x51,0x37,0x87, +0x88,0xb7,0x82,0xc8,0x93,0x5c,0x30,0xbd,0x6f,0xe8,0x51,0xb1, +0xc3,0xc9,0x33,0x43,0xcb,0xb2,0x35,0xcf,0x9c,0xa5,0x4c,0xa5, +0x77,0xe1,0x14,0xaa,0x4f,0xc7,0xf6,0x29,0x43,0xcc,0x49,0x85, +0x82,0xd4,0x2c,0xe3,0x28,0x9d,0x75,0x96,0x59,0x16,0x6e,0x08, +0x4d,0x58,0xca,0x38,0x4a,0xd7,0x9d,0x77,0x59,0x8c,0x67,0xe9, +0x3e,0x39,0x74,0x7d,0x3f,0x8c,0x5f,0xb0,0x27,0xad,0x1d,0x3a, +0xbe,0xc8,0x13,0xb9,0xeb,0xac,0xa7,0xd3,0x67,0x02,0xf1,0x37, +0x11,0x4d,0xcf,0x6b,0x81,0x45,0xff,0x7a,0x75,0xe1,0xc1,0x6d, +0x53,0x6e,0x4e,0xa3,0xf3,0xd6,0xfb,0xb8,0x86,0xa0,0x81,0x32, +0xd8,0xc2,0xbb,0xb5,0xe8,0xe3,0xd6,0xe1,0x93,0xab,0x07,0x35, +0x38,0x5c,0x7e,0x42,0x29,0x94,0x74,0x78,0xff,0xf4,0x42,0x9b, +0x32,0x0a,0x8c,0x54,0x56,0x3c,0xe7,0xb3,0xb1,0x1f,0x54,0x13, +0x37,0x4c,0xd9,0x5d,0x33,0x93,0xac,0x36,0xef,0x23,0xd9,0x36, +0x35,0x73,0xbb,0xf1,0x53,0x9f,0x2d,0xfb,0x33,0xde,0xcb,0xb7, +0x6f,0xea,0x0f,0xdc,0x13,0x50,0xb8,0x61,0x0c,0x22,0x97,0xd6, +0x7a,0x63,0xba,0x08,0x0f,0xa4,0x0f,0x56,0x38,0xd0,0xb4,0x04, +0x50,0x82,0x0f,0xa8,0xab,0xcc,0xec,0x8f,0xeb,0x1f,0x98,0x1a, +0xc4,0x1a,0x62,0x6f,0x6e,0x74,0xa0,0x77,0xe1,0xef,0x80,0x4e, +0xc3,0x98,0x6f,0x64,0x7c,0xe4,0xa6,0xd1,0xd9,0x87,0xf3,0x22, +0xb4,0x25,0xf8,0xda,0xca,0xc7,0xa2,0x38,0x30,0xa3,0xca,0xec, +0xaa,0x40,0x7f,0x10,0x94,0x2a,0xe6,0x20,0x11,0xc9,0x51,0xb7, +0xee,0x32,0xd4,0x22,0xe1,0xe7,0x61,0xfd,0x08,0x8f,0x26,0xe0, +0x5f,0x37,0x23,0x88,0xd4,0xa5,0x92,0x43,0x79,0x83,0x91,0x03, +0xa3,0xec,0x37,0xc4,0xdf,0xa9,0xa8,0xc8,0x18,0x84,0x6a,0x10, +0x99,0x3a,0x5a,0x54,0x96,0x88,0x6b,0xe5,0xe6,0x07,0x22,0x69, +0x36,0x74,0xe8,0xb8,0x29,0x36,0xb2,0xe9,0x4d,0xe0,0xa7,0xdf, +0xf3,0xe0,0x4b,0x4c,0xfe,0xa4,0x83,0x5d,0xa8,0x09,0x4b,0xd1, +0x45,0x9d,0x18,0x4d,0x37,0x5d,0x88,0x29,0x91,0x64,0xde,0x85, +0x86,0x93,0xc5,0x4b,0x0c,0x23,0xab,0xb0,0x24,0x38,0x85,0xc5, +0x8a,0x46,0xa6,0x48,0x5c,0xc4,0x89,0xab,0x05,0x39,0x0b,0xb0, +0x19,0xcb,0x72,0x86,0xc1,0xbe,0x5d,0xd8,0x09,0xb3,0xf1,0x4b, +0x1d,0x3d,0xaf,0xe8,0x7a,0x11,0x3a,0x87,0xb4,0x48,0x15,0xa4, +0x3d,0xfb,0xdd,0xdd,0xdd,0x60,0xb0,0x3c,0x5f,0x8c,0x84,0xba, +0x08,0x00,0xfb,0x16,0x70,0x52,0x03,0xb4,0x2d,0x5b,0x82,0x12, +0xcc,0xd6,0x07,0xb6,0x91,0x04,0x2b,0xe1,0x96,0x71,0xaa,0x3e, +0xc8,0x19,0xa8,0x2c,0xc1,0xee,0xaf,0x9c,0x9c,0x85,0xc2,0x38, +0xc4,0x17,0x20,0xe1,0x42,0x8d,0x0b,0x8e,0x81,0x0c,0x91,0x17, +0xf3,0xd3,0x80,0x20,0xc3,0xd6,0xe9,0xc1,0x26,0xb6,0xc9,0x78, +0xb2,0x6c,0x78,0x6e,0x04,0xd4,0xc1,0x1d,0xa1,0xfb,0xe8,0x32, +0x71,0xbb,0x7a,0x9b,0x07,0x35,0x2b,0xe9,0xfb,0xe2,0xbb,0x90, +0x40,0x46,0x45,0x13,0xc1,0xf2,0xc7,0xb6,0x3a,0xa3,0x0d,0x3d, +0x98,0xac,0x6a,0x07,0x4b,0x25,0x76,0x21,0xc7,0xd2,0xcb,0x90, +0xc5,0x17,0x76,0xf4,0xd8,0x20,0x1f,0x3a,0xee,0x09,0xa8,0xfa, +0x8a,0xb2,0x15,0x55,0x90,0x6c,0xdc,0xb2,0x8c,0x10,0x1d,0x8f, +0xef,0xa7,0x3a,0x1e,0x3f,0x94,0x2c,0x7b,0xaa,0x0a,0x45,0x3e, +0xfb,0xa7,0xb0,0xff,0x83,0x16,0xf5,0xd4,0x1d,0x68,0x71,0x19, +0xe2,0x81,0x1d,0xf2,0x9c,0x6b,0x9e,0x41,0xc4,0xd1,0x46,0xaa, +0x01,0xc0,0x1b,0x2b,0x45,0x02,0x0a,0x02,0x8d,0xe0,0xa9,0x8e, +0xb9,0x97,0xfa,0x7d,0x77,0x97,0xd2,0x17,0xf5,0x34,0xb9,0x18, +0xa3,0x90,0x59,0x3a,0x9d,0x7d,0x4a,0x92,0xfe,0x00,0x0a,0x20, +0x0f,0x9a,0xf0,0x4b,0x06,0x05,0xb2,0xd8,0xfd,0x13,0xfa,0x3f, +0x49,0xeb,0xc3,0x4d,0xf4,0xea,0xe3,0xac,0x4b,0x09,0xc7,0xb9, +0xca,0x30,0xe2,0xac,0x00,0xaa,0xb2,0x30,0x99,0x45,0x36,0xec, +0x41,0x0a,0xff,0xaa,0x54,0x04,0x12,0xe7,0x35,0xbe,0xab,0xdd, +0xd3,0xfb,0xb4,0xaa,0x98,0x12,0xfb,0xfa,0x4e,0xbd,0x66,0x32, +0x13,0x71,0x4c,0x29,0xf7,0x32,0x27,0x22,0x61,0xa2,0x84,0xc0, +0x75,0xf2,0x10,0x25,0xcb,0x57,0xe6,0x44,0x0f,0x6c,0x95,0x67, +0x0f,0x3e,0x34,0x8f,0xdf,0x5a,0xd5,0x4b,0x5d,0x56,0x01,0xcf, +0x16,0x70,0xb4,0xda,0xb8,0xf8,0x7b,0x2a,0x39,0xc3,0x21,0x81, +0x4f,0x07,0x20,0x09,0x94,0x05,0xbb,0xcc,0x46,0xf3,0x07,0x4a, +0xa4,0xcc,0x09,0xb0,0xb2,0xa7,0x10,0x98,0x0e,0x80,0x6c,0xff, +0x24,0x97,0x8b,0xeb,0x55,0x09,0xa9,0x89,0x8a,0x75,0xbb,0x11, +0x7d,0x16,0x2a,0x74,0xc2,0x80,0x9e,0x96,0x6f,0x22,0x78,0xc6, +0x5b,0x55,0x4d,0xb0,0xa4,0x4c,0x1a,0x60,0x74,0x64,0x0b,0x6a, +0x1e,0x55,0xd2,0xf5,0xfa,0x91,0x0a,0x62,0x09,0xb6,0x56,0x92, +0xd9,0x54,0x68,0x4d,0xf5,0x3a,0x88,0xcb,0xcc,0x86,0x0d,0x61, +0xac,0x66,0xd6,0x1b,0x61,0x2a,0x07,0x47,0xf6,0x11,0x96,0xf1, +0xa7,0x44,0x41,0xee,0x51,0x00,0xe3,0x33,0xf1,0x93,0x09,0xbf, +0x40,0xcb,0x95,0x63,0x84,0x3d,0xb4,0x58,0x39,0xaf,0x0c,0x79, +0xb8,0x14,0x33,0x2e,0x21,0x6d,0x56,0x81,0xea,0x85,0xe2,0x27, +0x1f,0xae,0x7a,0x09,0xf9,0x92,0x8f,0x06,0x5d,0xdc,0xdc,0xaf, +0xbb,0x8e,0xe0,0x8a,0x3e,0xa2,0x81,0xdc,0x54,0x05,0x44,0xb0, +0xdd,0x99,0xe9,0x80,0x53,0xc1,0x26,0x82,0xef,0xa0,0x36,0x0d, +0x02,0xa1,0xe4,0x88,0x57,0x25,0x56,0xec,0x1d,0xaa,0x4d,0x84, +0x3e,0x07,0xf4,0x4f,0x17,0x20,0x0a,0xb4,0x82,0xa5,0x84,0x42, +0xab,0x21,0x30,0xd2,0xec,0x80,0x8c,0x64,0xe1,0x4f,0x95,0xd3, +0xdd,0x8b,0xab,0x08,0xae,0x26,0xb4,0xf9,0xe3,0x42,0x22,0x2a, +0xf3,0xe0,0xd9,0xf7,0x9f,0x5d,0x21,0x9a,0x57,0xfb,0xab,0x14, +0x7a,0x49,0xaa,0x55,0xc6,0x93,0x42,0xf5,0xeb,0x5e,0x42,0x18, +0xf7,0x35,0x38,0x11,0x19,0x8a,0x83,0x0a,0x7f,0x29,0xa6,0x2d, +0x39,0x2f,0x03,0xdb,0x2f,0x04,0x26,0x33,0x30,0x7c,0xfe,0x84, +0xa0,0xca,0xbe,0xcb,0x5d,0x8f,0x62,0xe1,0xa4,0x58,0x67,0x6b, +0x4a,0x4b,0x92,0x2c,0x80,0x27,0x43,0x52,0x88,0x89,0x03,0x13, +0xb0,0x81,0xcd,0xc3,0xbe,0x5d,0xbc,0xfb,0x5d,0xb9,0x16,0xb0, +0x25,0xa0,0x9d,0xac,0x4d,0x1d,0xa4,0xf6,0x0f,0x4d,0x0f,0x07, +0x98,0x47,0xfa,0x16,0x57,0x68,0xad,0x6f,0xb1,0xd1,0x57,0xf7, +0x95,0xe4,0xdb,0xc2,0xf7,0x83,0x72,0xee,0x00,0xc8,0xc6,0xb1, +0x63,0x01,0xc7,0x0e,0xda,0x44,0x6c,0xbe,0x3d,0x5d,0xc8,0x1a, +0x5e,0xf0,0x19,0xc4,0xbd,0x11,0xde,0x64,0x83,0x4d,0x2e,0x0e, +0x8d,0x2a,0xb3,0xc0,0x33,0x08,0xd7,0x6b,0xb4,0x98,0x97,0x79, +0x4c,0xf8,0x18,0xae,0xab,0x2a,0x5c,0x97,0xe0,0x5b,0x3d,0xe2, +0x68,0x6c,0xa3,0x23,0xd0,0x2b,0x76,0xe3,0x20,0x34,0x15,0x08, +0x4a,0xde,0x26,0xeb,0x00,0x96,0x65,0x5e,0xf0,0x97,0x9d,0x7c, +0xd5,0xd9,0x64,0xe8,0xd1,0x0b,0x38,0x6b,0xba,0x22,0xf7,0x5d, +0x93,0x72,0x68,0xc5,0xa1,0xd5,0x8a,0xe3,0x8b,0xcb,0xe7,0x01, +0x3a,0x7c,0xad,0xc0,0x47,0xf1,0x95,0x0e,0xe6,0x24,0x65,0xd1, +0x1e,0xa9,0xcf,0xc7,0x6e,0x3e,0xfa,0xf3,0x05,0xc5,0xbc,0xfb, +0xbf,0xa7,0xed,0xc7,0x47,0x5a,0x87,0x29,0x39,0xbd,0x26,0x00, +0x6b,0x4a,0x01,0x01,0xdc,0xa1,0xc1,0xd4,0x15,0x02,0x0f,0xd2, +0xe1,0xeb,0x93,0xea,0x01,0xc7,0x54,0x3c,0xa3,0xaa,0x03,0x2a, +0x1e,0xe0,0x9a,0x35,0x08,0x8f,0xa0,0xe1,0xf9,0xf3,0x93,0x6e, +0x0d,0xf6,0x5d,0x1a,0x28,0x8d,0xac,0xae,0x43,0xeb,0x55,0x40, +0x58,0xb9,0xc7,0xd2,0xf4,0xdf,0xa3,0xe8,0xd7,0xaa,0xfd,0xd6, +0x5a,0xc6,0xb4,0x7b,0x61,0x6d,0xff,0xa9,0x95,0xf3,0xed,0x33, +0xec,0x2f,0xd1,0xa9,0x87,0x67,0x95,0x7f,0xa9,0xd4,0xff,0x39, +0x54,0xea,0x7f,0x99,0xfe,0xbc,0x7d,0x51,0x03,0x2e,0xb9,0x08, +0xfe,0x97,0x2e,0xf7,0x91,0x75,0xb9,0xcd,0x9b,0x81,0x16,0x6d, +0x95,0x46,0x62,0xb9,0xa4,0x0d,0x85,0xb2,0x9b,0x08,0x5f,0xaf, +0x6e,0xe7,0x17,0x51,0x67,0xb1,0x62,0x85,0x78,0x33,0x9c,0x82, +0x4b,0x83,0x9e,0xe5,0xa4,0xcb,0x2e,0x68,0x49,0x6d,0x2b,0x7e, +0x84,0x58,0x33,0x76,0x4c,0x06,0xa9,0xd6,0x91,0x64,0x4e,0xc0, +0x63,0xbd,0x0a,0xdf,0x4b,0x64,0xb9,0x5d,0xd2,0x6f,0xa6,0x97, +0x70,0xc0,0x4e,0x81,0x9c,0xe9,0xf7,0xc8,0xe3,0xf6,0xb2,0x08, +0x1b,0x64,0x9c,0xee,0xe9,0xb4,0x51,0xd6,0x57,0xbd,0x01,0x55, +0x25,0x77,0xa5,0xe8,0xa6,0x05,0xf3,0x63,0xb5,0x79,0xaf,0x7b, +0x02,0xa2,0x16,0xc1,0xd5,0x08,0xdc,0x7d,0x38,0x91,0xad,0x11, +0x0a,0xfa,0xaa,0xc6,0x71,0x6f,0x0a,0xb5,0x7c,0x3b,0xe9,0xd5, +0x00,0x7c,0xd4,0x8f,0xd7,0x9d,0xab,0x2e,0x6f,0x39,0x10,0x49, +0x4e,0xaf,0x60,0x82,0xb1,0x7c,0xc7,0x82,0xc8,0xf6,0x94,0x19, +0xa8,0x8b,0x57,0x05,0xcb,0x1a,0xf7,0x8a,0x6a,0xa8,0x98,0x28, +0x90,0x29,0x60,0xe7,0x0f,0x6f,0x7e,0xfe,0xc9,0xf6,0xe9,0xd5, +0x6c,0x68,0xe7,0xe1,0xf1,0x93,0xc9,0xb2,0x1c,0x17,0xc7,0xef, +0x4d,0xf2,0x64,0x3b,0x9b,0xce,0x3f,0xe4,0xf8,0x9d,0xc8,0x2c, +0xfb,0xfc,0x4b,0x4b,0xe0,0x09,0x31,0xb3,0xe4,0xf3,0xce,0x4d, +0xe7,0xf3,0xe4,0xc9,0xe7,0x9d,0xcf,0x99,0x65,0xd9,0x94,0x09, +0xa5,0xf8,0xcd,0xc1,0xc2,0x83,0x03,0x57,0xb0,0xe7,0x2e,0x3b, +0xc9,0x57,0x49,0xc7,0x26,0x74,0x92,0x2f,0x8f,0xed,0x3f,0x5f, +0x25,0xf7,0x29,0xed,0x25,0x26,0xac,0xea,0x9a,0x5a,0x2f,0x98, +0xc6,0xde,0xc5,0x35,0x75,0xb7,0x53,0x6e,0x08,0x06,0x8e,0xcf, +0xbd,0xfe,0xed,0xfb,0x54,0x14,0xcf,0xb9,0x4b,0xfd,0x06,0x47, +0xd6,0x65,0x34,0x16,0x8a,0x87,0x51,0xa2,0x1d,0x03,0x63,0x3d, +0x39,0xb3,0x11,0xa4,0xda,0xac,0x6f,0x8a,0x1b,0x78,0x42,0xca, +0x23,0x6a,0x7f,0x2c,0x59,0x54,0x2a,0xf0,0x69,0x29,0x8d,0xa2, +0xfd,0x97,0xc6,0xd1,0xf0,0xea,0x0a,0x8f,0x00,0x56,0x30,0xe6, +0x1a,0xae,0x41,0xe3,0x61,0xa7,0xff,0x7a,0x5b,0x70,0x30,0xa8, +0x7a,0xfc,0xe9,0x53,0x9b,0x07,0xc1,0xf3,0xd0,0xdb,0x30,0x58, +0xec,0xa7,0x7d,0x8c,0x27,0xdd,0x37,0x52,0x0b,0xc3,0xc5,0x66, +0x35,0x01,0xbc,0x2b,0xd1,0x06,0x2d,0x85,0x46,0x1e,0x5b,0x09, +0x48,0xa3,0xeb,0x9d,0x63,0xbb,0x35,0xd2,0x33,0xd1,0x96,0x47, +0x9b,0x5d,0x2b,0x2d,0x72,0x01,0x68,0x62,0x45,0xff,0xc9,0x1a, +0xf7,0x87,0x69,0xb1,0x1f,0x55,0x05,0x8d,0xc7,0xbe,0xa8,0x06, +0xfa,0x93,0x94,0xc5,0xc1,0xa1,0xb4,0xfb,0x47,0xd4,0xc6,0x37, +0x85,0x57,0x11,0xdb,0x05,0xa9,0x94,0xc3,0xe6,0xaa,0xd0,0x5a, +0x61,0x2b,0xca,0xc5,0xb5,0xbe,0x66,0x23,0x87,0xd2,0xa0,0x56, +0x34,0x6d,0xcc,0xf2,0xcf,0x51,0x0a,0xb3,0x46,0xd4,0x95,0x58, +0x9f,0x39,0x16,0xd2,0x44,0xf6,0xa4,0xd1,0x72,0x78,0x93,0x5e, +0x4c,0x86,0x4b,0x30,0xb0,0x5b,0xde,0xfa,0x33,0x3c,0xa6,0xa5, +0xa4,0x70,0x91,0x52,0x8c,0xb4,0x35,0xd7,0x0d,0xf7,0x62,0x53, +0x76,0x2a,0x9c,0x49,0xe7,0x0f,0xdc,0x2e,0x83,0x63,0xbe,0xb3, +0x20,0x74,0x21,0x60,0x7f,0xa4,0xee,0x76,0x40,0x6d,0xa2,0xa9, +0xda,0x44,0x71,0x43,0xda,0xd5,0x28,0x50,0x72,0xce,0x3b,0x31, +0x98,0x18,0x5e,0x0c,0x21,0x52,0x6e,0xb9,0x5c,0x2a,0xd8,0x12, +0x7c,0xd7,0x5b,0x78,0x78,0x3d,0x18,0xa8,0x51,0xa0,0x72,0xa8, +0x45,0x81,0xc6,0xb7,0x4e,0x1e,0xd4,0x7d,0x54,0xe9,0xe8,0xa5, +0x49,0xec,0xc9,0xa0,0x1c,0x4f,0xe7,0x25,0xb8,0x07,0x82,0x75, +0xb3,0x18,0x3f,0xb1,0xd3,0x70,0x01,0xfa,0x96,0x01,0x94,0x90, +0xcb,0x57,0x06,0x81,0x9a,0xaf,0x67,0x43,0xbc,0xd1,0x85,0x78, +0xd9,0x49,0xb9,0xbe,0x18,0x25,0x06,0x1e,0xc2,0x8f,0x89,0x6c, +0x9a,0xbc,0xb2,0x49,0xbf,0xf5,0x6d,0x62,0xf2,0x6c,0xb2,0x5e, +0x5f,0x29,0x45,0x39,0x7e,0xab,0xc1,0x81,0xd8,0x83,0xf3,0xeb, +0x2b,0x17,0x20,0x9f,0x6f,0x79,0xed,0xc0,0xac,0x30,0xa8,0xbe, +0x6d,0xc5,0xcc,0xee,0xe3,0xc9,0xb1,0xdb,0xc3,0x9e,0xf4,0xf0, +0x98,0xb6,0xf4,0xc2,0x61,0x92,0xe4,0xa9,0x60,0x40,0xf2,0x78, +0x3a,0x5b,0x6b,0x07,0xf1,0x4e,0x83,0x9b,0xc0,0xa6,0xbe,0xa9, +0x32,0x43,0x70,0xbf,0x2f,0xa6,0x73,0xa4,0xac,0xc2,0x5e,0xd9, +0x59,0xfd,0x9f,0xe5,0x2d,0x04,0x21,0x80,0x6d,0x8f,0xd5,0xed, +0xe5,0x6c,0xac,0x7c,0xcc,0xc2,0x27,0x69,0x49,0x75,0xe5,0x05, +0xc3,0x60,0xb6,0xe5,0xe8,0xbf,0xa0,0xb1,0x43,0x7d,0x33,0xa4, +0x7a,0x3a,0x12,0x87,0x34,0x8f,0xab,0x94,0x24,0x19,0x37,0xb3, +0x7b,0xb5,0xb8,0xb2,0x2c,0x81,0xd3,0xeb,0x6d,0xb1,0x83,0x8a, +0xe4,0xa1,0xf7,0x9a,0x12,0x9e,0x05,0xe9,0xd4,0xaa,0xfd,0x8b, +0x15,0x11,0xa6,0xdb,0x8e,0xab,0x93,0x10,0x78,0x1f,0xb5,0xad, +0xc5,0x01,0x84,0xbc,0xd4,0x95,0x92,0xba,0x32,0x03,0x73,0x5c, +0x4b,0x1a,0x3d,0x52,0x86,0xa8,0xe9,0xee,0x7a,0x39,0xcb,0x15, +0x2e,0x5d,0x49,0x3c,0xeb,0xa2,0x92,0x37,0x65,0x35,0xa3,0xe0, +0xda,0xb1,0xa2,0x98,0x86,0x79,0xf2,0xcb,0xdf,0x5e,0xbf,0x49, +0x0c,0xcc,0xe3,0x72,0xb9,0xca,0x77,0x09,0xee,0x98,0xf3,0xf5, +0xd1,0x1b,0x3b,0x5b,0x93,0x3c,0x81,0xcb,0x4d,0x78,0x16,0x60, +0x4b,0x3e,0xde,0x1e,0xdd,0xdc,0xdc,0x1c,0xc1,0xc6,0x79,0x64, +0x0b,0x23,0x6e,0x3c,0x4a,0xaa,0x4a,0xaa,0x69,0x79,0x74,0xb9, +0x2e,0x6d,0xe7,0xb5,0xb5,0xaf,0x59,0x49,0xa9,0xc5,0xb7,0xaf, +0x7e,0x7a,0xf5,0xe6,0xd5,0xe3,0xd4,0x23,0x98,0x75,0xaf,0xd7, +0x43,0xdb,0xa5,0xf6,0xcf,0x03,0xe7,0x9d,0x43,0x79,0xc8,0xa8, +0x03,0xd2,0xea,0xf1,0xc6,0xbd,0xe2,0xd0,0x1e,0x44,0xb1,0x48, +0x8e,0x37,0xfd,0x63,0x38,0x24,0x1d,0x27,0x46,0x95,0x44,0xe9, +0x98,0x70,0x9c,0xf8,0x5d,0x01,0x86,0x82,0xa9,0x8f,0x28,0x9e, +0xf4,0xaa,0xcb,0xbd,0x6b,0xa5,0x99,0xcb,0xcb,0xc5,0xfc,0x2c, +0xf9,0xef,0xa3,0x5f,0xc9,0x10,0xbc,0x1c,0x1d,0xfd,0x63,0xba, +0x9e,0x24,0xe7,0x66,0x67,0xab,0x02,0x7d,0x96,0x73,0xdf,0x19, +0xfb,0x6d,0x47,0x34,0xa7,0x55,0x51,0x55,0xe7,0x71,0x46,0x05, +0xa0,0x2b,0x60,0x4c,0xf3,0xf7,0xbf,0x42,0x70,0xfb,0xc4,0x10, +0xff,0xb2,0xdc,0x8b,0x24,0xde,0xd4,0xf2,0xac,0x25,0xe4,0xfc, +0xb2,0x5c,0x80,0x9b,0x4d,0x7d,0x9d,0x18,0x66,0x64,0xbb,0xf0, +0xbb,0x7b,0x33,0x29,0x71,0x0d,0x9a,0x9d,0xdd,0xce,0xaf,0x20, +0xba,0xc8,0xdf,0xed,0x2c,0x4a,0x80,0xf7,0xaf,0xa8,0xdd,0xdd, +0xc9,0xfa,0x72,0x96,0x18,0x10,0xd2,0x96,0x8b,0xd9,0xcc,0xee, +0x32,0x09,0x56,0xe8,0x9b,0xf5,0x72,0x06,0x67,0xf3,0xc5,0x7a, +0x52,0x2e,0x6f,0xa6,0x2b,0x7b,0x3c,0xff,0x44,0x12,0x15,0x35, +0x84,0xf3,0x52,0x95,0x07,0xcc,0x78,0x75,0xb1,0xb8,0x82,0x26, +0x0b,0x7f,0x4e,0x90,0xe4,0x6f,0x60,0xb3,0xa2,0x5a,0x89,0x50, +0x86,0x60,0x8c,0x83,0x08,0x3c,0xd7,0x0d,0xa9,0x23,0xed,0xf4, +0x20,0xb0,0x2e,0x8f,0x46,0x9a,0xcc,0x70,0xe8,0xec,0x21,0x05, +0xa6,0x8a,0x3d,0xb8,0x5c,0xe3,0x4b,0xaa,0xb4,0x76,0xfe,0xa6, +0x32,0xba,0x04,0x8c,0xa4,0xd0,0xe1,0x89,0xe1,0xf4,0xb1,0xad, +0xfd,0xe2,0xc6,0xce,0x80,0xe2,0xec,0xdc,0x3c,0xeb,0x96,0x60, +0xd7,0xc1,0xae,0x54,0x38,0x43,0xd9,0xee,0xdb,0x9d,0x6b,0x4b, +0xcf,0xde,0x33,0xe2,0x1e,0xb4,0x48,0x28,0xbd,0x4e,0x90,0xae, +0x74,0x08,0x1a,0x1c,0xa7,0xd8,0xed,0xf1,0xfb,0xe5,0xf0,0x6a, +0x82,0x56,0xfc,0x81,0xc8,0xc0,0xc9,0xbe,0xd5,0xb8,0xbd,0xa6, +0xb0,0xab,0xd6,0x2f,0x2d,0x70,0xa7,0xf5,0x3b,0x3a,0x89,0x1a, +0x2c,0x5d,0x94,0xb3,0x9c,0xeb,0xf0,0x1e,0x08,0x02,0xb7,0x18, +0xda,0x7d,0x76,0x49,0x6c,0x6f,0x87,0xdd,0x9b,0xd7,0x6b,0x69, +0x59,0x44,0xa0,0x10,0x16,0x15,0x90,0xdc,0xe0,0x8b,0x30,0x52, +0xd1,0xc9,0x78,0x53,0x2e,0xd7,0xe2,0x65,0xf5,0x65,0xcf,0x4c, +0x16,0xcb,0x8f,0xf2,0xd9,0x3f,0x61,0xdf,0x40,0xcf,0xd2,0x68, +0x35,0xa4,0x9c,0xec,0x48,0x61,0xf1,0x75,0x46,0x3b,0x0e,0x57, +0x27,0x3b,0x52,0x25,0x9f,0x72,0xff,0xb8,0xf9,0x52,0x45,0x91, +0x8b,0xe4,0x33,0x98,0xd6,0xf3,0x8b,0xdb,0xc4,0x68,0x80,0xdf, +0xa6,0xab,0xe9,0xbb,0xa9,0xdd,0x0c,0x6f,0x0b,0x5c,0x91,0x47, +0x98,0x7a,0xb4,0x9a,0x80,0xb5,0x12,0x03,0xae,0x87,0xef,0x66, +0x65,0x03,0x10,0x53,0x8f,0x26,0x76,0x09,0x3a,0x40,0xc0,0x7a, +0x03,0xc9,0x9a,0x91,0x1d,0x40,0x64,0x59,0x6e,0xca,0xa1,0x5d, +0x46,0x9a,0x0e,0xce,0x83,0x07,0xd2,0xc1,0xca,0x38,0x2a,0x76, +0x25,0xfc,0x50,0x53,0xd9,0x7b,0xb6,0x6a,0x47,0x78,0x3e,0x5a, +0xdc,0xf8,0x3e,0xd5,0x68,0xff,0x08,0x6f,0x22,0x22,0x58,0x3c, +0x7a,0x0e,0xe9,0xd9,0x0d,0x8a,0x89,0xf5,0x92,0x8d,0xae,0x7f, +0x9a,0x74,0xb1,0xae,0x78,0xef,0x64,0xd9,0x85,0x5d,0x9e,0xce, +0x84,0xac,0x8e,0x68,0xc7,0xf8,0x24,0x33,0x6a,0xc1,0xdb,0x45, +0xd3,0x56,0x14,0x7b,0x8d,0xd2,0xb5,0xd5,0x68,0x54,0xe5,0xee, +0x62,0x4e,0xe7,0xd0,0x48,0x97,0x3e,0x23,0x0b,0x25,0xdc,0x55, +0xd6,0x3f,0x82,0xc2,0xd2,0x2e,0xd4,0x34,0x4e,0xd0,0x34,0x70, +0x4e,0x7a,0x3d,0x64,0x7f,0x4e,0x0a,0x8d,0x31,0x37,0xe9,0xc2, +0x9d,0x98,0x44,0x90,0x63,0x51,0xbe,0x29,0x38,0x11,0x45,0xf1, +0x17,0x72,0x55,0xd0,0xef,0x55,0x6c,0x33,0xc1,0xae,0xaa,0x90, +0xa8,0x1d,0x5b,0xfe,0x74,0x6e,0x35,0xe1,0x4c,0x84,0xaf,0xbb, +0xf3,0xb3,0x13,0x7b,0x52,0xef,0xf5,0xce,0x2b,0x6f,0x65,0xc1, +0xc0,0xb7,0x0d,0xdf,0x7e,0xe4,0xad,0xca,0xf5,0x3e,0xbb,0xb7, +0xa2,0x23,0x09,0x17,0x65,0xc6,0xd3,0x72,0x66,0x25,0x0f,0xe4, +0x67,0x56,0x78,0x90,0x92,0xb7,0x0e,0x9b,0xcd,0x04,0x09,0xf9, +0x67,0x8b,0x2f,0x9e,0x02,0x7f,0x1e,0x6e,0x6d,0x03,0x7a,0x66, +0x3e,0xbd,0x28,0xbd,0x17,0xad,0xde,0xde,0x62,0xe8,0x28,0x46, +0x2b,0xb4,0x0b,0x01,0x99,0xac,0x6c,0xa3,0x4a,0xbd,0x80,0xe8, +0x35,0xd2,0x0c,0xf6,0x79,0xe4,0xbc,0x72,0xd9,0x6e,0xa4,0xb6, +0x9b,0x7e,0xf9,0xef,0xce,0x15,0x57,0xf2,0x59,0xaf,0xf7,0xed, +0x7f,0xbc,0xf8,0xc2,0x6e,0x37,0x9f,0x7d,0xf7,0xdd,0x37,0xbd, +0x5e,0x0f,0x7e,0x5d,0x7c,0xd1,0xeb,0xbd,0x78,0xa1,0x7e,0x41, +0x7f,0x91,0x2d,0x0a,0xfb,0x6a,0xb1,0x0d,0xa4,0x60,0x42,0xf0, +0x0b,0xdd,0x26,0xe5,0xff,0x66,0xa8,0x16,0x3f,0x51,0xfd,0x9e, +0xa4,0xf6,0xd4,0x0c,0x63,0x41,0x08,0xb7,0x0e,0xc1,0xfe,0xd2, +0x3e,0x5d,0x08,0x79,0x47,0xbe,0x1b,0xe4,0x26,0x3b,0x41,0x03, +0x30,0x32,0x49,0x86,0x06,0xfa,0x10,0x26,0xf7,0x41,0x92,0x13, +0x97,0x7c,0x07,0xbe,0x26,0xda,0x60,0xc0,0xc6,0xa1,0x41,0xe7, +0xb3,0x97,0x2f,0x5f,0xda,0xea,0x52,0x2a,0xae,0x16,0xc9,0xea, +0x57,0xf6,0x3f,0x67,0x83,0xc3,0xed,0x81,0xc8,0xdc,0x76,0x5c, +0xc0,0xc6,0x20,0x18,0xaa,0x2a,0xe2,0xb0,0x66,0xb7,0xcd,0x77, +0xbe,0xb7,0x88,0xa8,0x9d,0xba,0xdb,0x17,0x3a,0x79,0xef,0x10, +0x9b,0x5b,0x07,0x7a,0x5b,0x9b,0x74,0x12,0xb6,0xe9,0xa8,0x5f, +0x89,0x55,0x11,0xd7,0xfb,0x65,0x65,0xa8,0x17,0x18,0x93,0xa7, +0xc7,0xfe,0xb9,0x54,0xb9,0x11,0x23,0xa7,0x02,0x9f,0xd6,0xc6, +0x4f,0x6d,0x4c,0x85,0xae,0x75,0xa4,0x01,0x27,0x3d,0x69,0x81, +0x8c,0x11,0xcc,0x4f,0x1c,0x0d,0xbb,0x09,0x9f,0xfe,0x6f,0x00, +0x00,0x00,0xff,0xff,0xdf,0x8a,0x9a,0x2e,0x24,0xda,0x01,0x00, + })) + + if err != nil { + panic("Decompression failed: " + err.Error()) + } + + var b bytes.Buffer + io.Copy(&b, gz) + gz.Close() + + return b.Bytes() +} + +func init() { + go_bindata["/scripts/stats-scripts.js"] = scripts_stats_scripts_js +} diff --git a/dashboard/resources/stats.html.go b/dashboard/resources/stats.html.go new file mode 100644 index 00000000000..9dd24ef8825 --- /dev/null +++ b/dashboard/resources/stats.html.go @@ -0,0 +1,76 @@ +package resources + +import ( + "bytes" + "compress/gzip" + "io" +) + +// stats_html returns raw, uncompressed file data. +func stats_html() []byte { + gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ +0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0x8c,0x54, +0xcb,0x6e,0xdb,0x30,0x10,0xbc,0xe7,0x2b,0x36,0xba,0xd3,0x3c, +0x14,0x45,0x92,0x82,0x12,0x90,0x16,0x39,0xe4,0x56,0xa0,0x28, +0xd0,0x22,0x08,0x02,0x9a,0x5c,0x5b,0x4c,0x25,0x2e,0x4b,0xae, +0xec,0xfa,0xef,0x4b,0x4a,0x56,0xa2,0x3c,0x5a,0xd8,0x07,0x6b, +0xc5,0xe5,0xcc,0x3e,0x66,0x6c,0x75,0x6e,0xc9,0xf0,0x21,0x20, +0xb4,0xdc,0x77,0xcd,0x99,0x3a,0x17,0xe2,0xce,0x6d,0xa0,0x63, +0xb8,0xbd,0x81,0x8b,0xfb,0x06,0xc6,0x8f,0x2a,0x59,0x30,0x9d, +0x4e,0xa9,0xae,0x3c,0x89,0xc7,0x94,0x6f,0x08,0x87,0x57,0xd3, +0xe3,0x72,0x7a,0x5c,0x54,0x0d,0xa8,0xf3,0x3b,0xf4,0xd6,0x6d, +0xee,0x85,0x78,0x66,0x5b,0x52,0x9d,0xc0,0xf6,0x1f,0x9a,0xcb, +0x53,0x68,0xfe,0x85,0xdf,0xf2,0x91,0xa2,0x1c,0x34,0xef,0xe0, +0x47,0xa0,0x10,0x2f,0xc0,0xa5,0x0e,0x6a,0x5b,0x82,0x1c,0xf6, +0xc8,0x1a,0x4c,0xab,0x63,0x42,0xae,0xab,0x81,0x37,0x22,0x77, +0xbb,0x48,0xb5,0xcc,0x41,0xe0,0xef,0xc1,0xed,0xea,0xea,0x87, +0xf8,0x7e,0x2d,0xbe,0x50,0x1f,0x34,0xbb,0x75,0x87,0x15,0x18, +0xf2,0x8c,0x3e,0xe3,0x6e,0x6f,0x6a,0xb4,0x5b,0x9c,0x91,0xec, +0xb8,0xc3,0x06,0xd9,0x58,0xf8,0x1c,0x69,0x9f,0x30,0x2a,0x39, +0x9d,0x2d,0x98,0xbd,0xee,0xb1,0xae,0x2c,0x26,0x13,0x5d,0x60, +0x47,0x7e,0xc1,0x57,0xbd,0xbd,0xb8,0x73,0xb8,0x0f,0x14,0x79, +0x71,0x6b,0xef,0x2c,0xb7,0xb5,0xc5,0x9d,0x33,0x28,0xc6,0x97, +0x19,0x97,0x67,0x86,0xaf,0x9d,0x36,0x08,0x1b,0x9d,0xb3,0xe4, +0x57,0xf9,0x0b,0xb4,0xb7,0xa0,0x43,0xe8,0x50,0x30,0x0d,0xa6, +0x15,0x63,0x22,0xf8,0x2d,0x38,0x0f,0xdc,0x22,0x44,0x22,0x06, +0xeb,0x22,0x1a,0xa6,0x78,0x80,0xb2,0xac,0xb3,0x27,0x6d,0x3a, +0xe7,0x7f,0x41,0xc4,0xae,0xae,0x12,0x1f,0x3a,0x4c,0x2d,0x62, +0xee,0xa5,0x8d,0xb8,0x99,0x4f,0x64,0xaf,0x9d,0x5f,0x99,0x94, +0xb7,0x7e,0xa6,0xe4,0xbc,0x63,0xb5,0x26,0x7b,0x00,0xbf,0x15, +0xb9,0x72,0x5d,0x95,0x9d,0x7c,0x63,0xcd,0x69,0xd1,0xe9,0x0b, +0x7b,0x1e,0x0b,0xaa,0x30,0xeb,0xb8,0x1e,0x17,0xd8,0x66,0xf4, +0xa1,0x6a,0x7e,0xd2,0x00,0x3a,0x22,0x0c,0xc9,0xe5,0xb6,0xb5, +0x07,0x95,0x38,0x92,0xdf,0x36,0x34,0xb0,0xd5,0x8c,0x56,0xc9, +0xe3,0x01,0x4c,0xb8,0xb8,0xca,0x7b,0x40,0x9d,0x10,0x94,0x3e, +0x36,0x5b,0x14,0xfd,0x24,0xe5,0x82,0x77,0x65,0xa8,0x97,0x55, +0x33,0x84,0x6d,0xd4,0x16,0xe1,0x40,0x43,0x9c,0xe1,0x4a,0xea, +0x06,0x98,0xc0,0xf5,0x21,0xd2,0xee,0x98,0xc3,0x3f,0x01,0xa3, +0x43,0x6f,0x70,0xa5,0x64,0x98,0x07,0x59,0x58,0xec,0x9d,0xd1, +0xae,0x9e,0x47,0x9b,0x04,0x87,0x14,0x4d,0x9e,0x8e,0xf6,0x18, +0x1f,0x72,0xfd,0x40,0x3e,0x6b,0x9a,0x24,0xa6,0x8f,0x22,0xb5, +0xae,0x7f,0x0a,0x56,0xc5,0xc5,0x79,0xac,0x11,0x74,0x1a,0xc7, +0x63,0x22,0xff,0x41,0x76,0x6e,0x3d,0x45,0xab,0x3e,0xcb,0xf2, +0x96,0xe5,0xfd,0x8e,0xe1,0xda,0xda,0x69,0xca,0xe4,0x18,0x81, +0xe2,0xe8,0x18,0x67,0x74,0x71,0xe8,0x6c,0x3d,0x68,0x31,0x8b, +0x30,0xfd,0x98,0x32,0xcc,0xba,0x1d,0x38,0x5b,0xc4,0xb5,0x0f, +0x69,0x14,0xb7,0xe8,0x5d,0x0c,0x3b,0x09,0x3e,0x6b,0x2d,0xf3, +0xc5,0xe6,0xd9,0x51,0xcb,0x19,0xa6,0x38,0xc9,0x11,0x2e,0x7a, +0xb2,0x43,0x36,0xd4,0xab,0xa6,0x4f,0x41,0x1e,0xdf,0x5e,0x21, +0x95,0x2c,0x26,0x1c,0x5d,0x59,0xfe,0x14,0xff,0x06,0x00,0x00, +0xff,0xff,0x95,0x89,0x83,0x4d,0x24,0x05,0x00,0x00, + })) + + if err != nil { + panic("Decompression failed: " + err.Error()) + } + + var b bytes.Buffer + io.Copy(&b, gz) + gz.Close() + + return b.Bytes() +} + +func init() { + go_bindata["/stats.html"] = stats_html +} diff --git a/dashboard/resources/styles-main.css.go b/dashboard/resources/styles-main.css.go new file mode 100644 index 00000000000..8896b86473b --- /dev/null +++ b/dashboard/resources/styles-main.css.go @@ -0,0 +1,216 @@ +package resources + +import ( + "bytes" + "compress/gzip" + "io" +) + +// styles_main_css returns raw, uncompressed file data. +func styles_main_css() []byte { + gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ +0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0xe4,0x5a, +0x6d,0x6f,0xe3,0xb8,0x11,0xfe,0x2b,0x42,0x0e,0x8b,0x8b,0x0b, +0x2b,0x90,0xe5,0x97,0xdc,0xca,0x38,0xa0,0x7b,0xb9,0xa6,0x2d, +0xd0,0x2b,0x0a,0xdc,0x7d,0x28,0x50,0xf4,0x03,0x2d,0xd1,0x36, +0xb1,0x92,0xa8,0x52,0x74,0x5e,0x56,0xc8,0x7f,0xef,0x90,0x14, +0x25,0x52,0xa2,0x62,0xc9,0xd9,0x2d,0x8a,0x8b,0x17,0x8b,0xc8, +0x34,0x39,0x9c,0x79,0x38,0x2f,0xcf,0xd0,0xde,0xd1,0xe4,0xb9, +0xca,0x10,0x3b,0x90,0x3c,0x0a,0x5e,0x6e,0x30,0x8f,0x13,0x3f, +0xa6,0x39,0x47,0x24,0xc7,0xac,0xda,0xa1,0xf8,0xf3,0x81,0xd1, +0x53,0x2e,0x06,0x53,0xca,0xa2,0xef,0xf6,0xfb,0xfd,0x76,0x47, +0x59,0x82,0x59,0xb4,0x28,0x9e,0xbc,0x92,0xa6,0x24,0xf1,0xbe, +0x4b,0x92,0xa4,0x1e,0xf5,0x19,0x4a,0xc8,0xa9,0x8c,0xd6,0xc5, +0x13,0x8c,0x3c,0xf9,0xe5,0x11,0x25,0xf4,0x31,0x62,0x87,0x1d, +0xba,0x0e,0xe6,0xe2,0xdf,0xcd,0x62,0xf5,0x31,0x08,0x67,0x5e, +0xe0,0x09,0x01,0x4b,0x98,0xb7,0x87,0xfd,0xfc,0x3d,0xca,0x48, +0xfa,0x1c,0x5d,0xfd,0x05,0xa7,0x0f,0x98,0x93,0x18,0x79,0x7f, +0xc7,0x27,0x7c,0x35,0x6f,0xde,0xcf,0x3f,0x31,0x82,0xd2,0x79, +0x89,0xf2,0xd2,0x2f,0x31,0x23,0xfb,0x2d,0x7d,0xc0,0x6c,0x9f, +0x82,0xf4,0x23,0x49,0x12,0x9c,0xab,0xfd,0xc8,0x17,0x92,0x1f, +0xa2,0x5a,0x19,0x18,0xd9,0xfa,0x19,0xfd,0xe2,0xbb,0x3f,0x2a, +0x68,0x49,0x38,0xa1,0x79,0xc4,0x70,0x8a,0x38,0x79,0xc0,0x2f, +0xa8,0xaa,0xed,0x0c,0x17,0xb7,0x9b,0x4f,0x77,0x5b,0x8e,0x9f, +0xb8,0x9f,0xe0,0x98,0x32,0x24,0x27,0xe6,0x34,0x87,0x49,0xd1, +0x51,0x6c,0x3d,0x47,0x11,0x8a,0xc5,0xaa,0xaa,0x3b,0x0b,0xf0, +0xc2,0x2c,0x05,0x04,0x5f,0x48,0x5e,0x9c,0xf8,0xbf,0xf8,0x73, +0x81,0x7f,0x14,0x93,0xfe,0x5d,0x19,0x98,0x90,0xbc,0xc4,0xbc, +0x86,0x21,0x84,0xff,0x26,0x46,0xeb,0x99,0x46,0x39,0xe8,0x00, +0xdb,0x00,0x06,0xe6,0xe0,0x68,0x21,0xde,0x16,0x28,0x49,0xc0, +0x32,0x3f,0xc5,0x7b,0x2e,0x81,0xd7,0x03,0x8c,0x1c,0x8e,0x6a, +0xe4,0x88,0xe5,0x63,0x08,0xcf,0x2f,0xc7,0xb0,0x6a,0x05,0x84, +0x61,0xf7,0x00,0x7e,0xa5,0x27,0x16,0x63,0xef,0x57,0xc0,0xd9, +0xfb,0x07,0xa3,0x57,0xf3,0x29,0x47,0x22,0x25,0x3d,0xaa,0xcd, +0xd6,0x41,0xb0,0xd5,0x9e,0x05,0x66,0x86,0x41,0xab,0x59,0xe3, +0x6a,0xbb,0x13,0xe7,0x34,0xaf,0x12,0x52,0x16,0x29,0x7a,0x06, +0x48,0x04,0x6a,0xfe,0x2e,0xa5,0xf1,0xe7,0x66,0xee,0x06,0xb0, +0x59,0x08,0x35,0x95,0x30,0x38,0x39,0x58,0x93,0x01,0x30,0x06, +0x0c,0x2b,0x6d,0x45,0xbd,0xf7,0x0a,0xf6,0x96,0xa2,0x6a,0xc3, +0x17,0x37,0xab,0xf0,0x87,0xf5,0xed,0x62,0x15,0x7e,0x54,0x67, +0x8a,0x52,0x72,0xc8,0xa3,0x18,0xe7,0x1c,0xb3,0xed,0xe3,0x91, +0x70,0xec,0x97,0x05,0x8a,0x31,0x9c,0xf0,0x23,0x43,0xc5,0x16, +0xce,0x57,0x58,0x98,0xd6,0x13,0x33,0xf0,0xb0,0x14,0x6f,0xe3, +0x13,0x2b,0xc1,0x3b,0x0a,0x4a,0xe4,0xba,0x5e,0x20,0x70,0x06, +0x50,0x14,0x88,0x81,0xd8,0xce,0xb1,0x09,0x05,0x41,0xb9,0xdd, +0x67,0xc2,0xfd,0x13,0x60,0x05,0x78,0xa5,0x38,0xe6,0xd2,0xa1, +0x94,0x8b,0x3a,0x46,0x4b,0xc7,0x20,0xed,0x8f,0xf5,0x06,0x34, +0xea,0xa6,0x0f,0x39,0xdd,0x2e,0x50,0x4e,0x17,0xae,0xd7,0x73, +0xfd,0xff,0x26,0x9c,0xcd,0xdd,0x3e,0x19,0xae,0x67,0xd6,0xa9, +0x99,0xcf,0x7e,0x99,0xa1,0x34,0xad,0x0c,0x37,0x6b,0x8f,0xda, +0x5b,0x04,0x3d,0x9f,0xb5,0x04,0xf9,0x05,0x23,0xa0,0xf2,0xb3, +0x23,0xdd,0xc0,0xa9,0xfd,0x74,0xf7,0x69,0x6b,0x24,0x1f,0x79, +0x7a,0xb5,0x29,0xc2,0xad,0x96,0x67,0xb4,0xd4,0xc2,0x75,0xac, +0xf6,0xf7,0x08,0xc3,0xdb,0x0d,0x4a,0xea,0x45,0x05,0x2d,0x44, +0x6c,0x1b,0xd3,0xa2,0xef,0x96,0xcb,0xa5,0x23,0x08,0xb5,0x7d, +0x0b,0x69,0xac,0xce,0x24,0x68,0x07,0x9e,0x70,0xe2,0x78,0xcb, +0x69,0x11,0x2d,0x3f,0xc2,0x47,0x5f,0x7c,0x02,0xe9,0xe0,0x29, +0xfa,0x08,0x2f,0xd3,0x90,0x4e,0x10,0x1b,0x27,0x14,0x48,0xe8, +0x05,0x6a,0xdd,0x9c,0xa0,0xe3,0x44,0xa6,0x21,0x4b,0x61,0x1f, +0x33,0x46,0x99,0x67,0x8f,0x89,0x44,0x0e,0xae,0xa8,0x73,0xda, +0xfd,0xfd,0xf2,0x6e,0xb5,0xb4,0xe2,0xe4,0x36,0x08,0x3a,0x72, +0x72,0xca,0xe3,0x63,0xf5,0x48,0x12,0x7e,0x54,0x61,0xa5,0x23, +0xc8,0xf4,0x60,0xe9,0xe7,0x7b,0xca,0xb2,0x88,0x51,0x8e,0x38, +0xbe,0x5e,0xad,0x13,0x7c,0x98,0x39,0x60,0xa8,0x63,0x56,0xa0, +0xe1,0xaf,0xdb,0x18,0x96,0xa9,0x4a,0x9a,0xdd,0xc1,0x59,0x4c, +0x0c,0xb6,0x2a,0x71,0x09,0x64,0x6d,0xed,0x6c,0x55,0xe5,0xac, +0x4a,0x8a,0xba,0xbd,0xed,0x4e,0x55,0x9f,0x7a,0x2e,0xe3,0xe4, +0x8a,0x73,0xc2,0xc5,0xa4,0xaa,0xd6,0x23,0xe8,0x09,0x17,0x9f, +0x3a,0x65,0x0f,0x69,0x2e,0xce,0x62,0x4f,0x58,0x56,0x19,0x80, +0x18,0x72,0x8f,0x18,0x81,0x7b,0x69,0xd8,0x83,0xe0,0x43,0xbf, +0x36,0x4d,0x2e,0x6f,0x96,0xe8,0x1b,0x99,0xa0,0x2c,0xb7,0xc6, +0x18,0x9b,0xf0,0xeb,0xb3,0x3d,0x08,0x1f,0x07,0xaf,0xb9,0x16, +0xf9,0x13,0xb1,0xb9,0x34,0x15,0xd4,0x55,0x0f,0x2a,0xf9,0xce, +0xa5,0x47,0xf9,0x25,0x0c,0x5f,0x07,0x1f,0xe6,0x42,0xd4,0xcc, +0x1c,0x13,0x16,0xcc,0x05,0x2b,0x98,0xcd,0x5c,0x5b,0x28,0xc9, +0xed,0x4e,0x42,0xba,0x90,0xe1,0x05,0x72,0x91,0x27,0x96,0xdb, +0x0b,0x21,0x15,0x4e,0x5f,0x74,0xe9,0x6e,0xfd,0x05,0xda,0xec, +0xfe,0xb2,0x3d,0x49,0xa1,0x12,0x44,0x05,0xa3,0x07,0x92,0x44, +0x3f,0xff,0xf3,0xaf,0x19,0x3a,0xe0,0xdf,0x74,0x78,0xdc,0xfc, +0x42,0x62,0x46,0x4b,0xba,0xe7,0x37,0x8d,0xb4,0x92,0x23,0xc6, +0xef,0x04,0x56,0x25,0x67,0x3f,0x7e,0x2f,0x44,0xc2,0xeb,0xfb, +0xb9,0x87,0xf3,0xc4,0x18,0x4e,0xe4,0x0b,0x86,0xff,0x5c,0x2f, +0xfc,0x4d,0x90,0x88,0xa0,0x49,0x6e,0x82,0xb7,0xa9,0x60,0x91, +0xee,0x1c,0x38,0x1c,0x46,0xf3,0x23,0x1f,0x92,0xdf,0x89,0xd3, +0xf6,0xfd,0x93,0x66,0x4c,0x3a,0xb6,0x85,0xc3,0x19,0xbe,0x77, +0xa1,0xa7,0x09,0x9d,0x3c,0x8e,0x76,0x29,0xae,0xbe,0xa6,0x30, +0x8f,0x0b,0x17,0xf6,0x78,0xa2,0x78,0x56,0x9b,0x7b,0x4e,0x45, +0x81,0x59,0x8c,0x4a,0x6c,0xe6,0x52,0xa8,0x5c,0x56,0xe5,0x97, +0xb4,0xa3,0x91,0x2e,0x99,0x43,0x87,0xb8,0x6a,0xb6,0xa4,0xd0, +0xb4,0xb9,0xd3,0xc2,0x20,0x2d,0x2d,0xf1,0xb0,0x08,0x57,0xa0, +0x93,0xfa,0x66,0xb3,0x71,0xe8,0xae,0x9e,0x93,0xca,0x62,0x23, +0x3f,0x4c,0xd1,0x69,0xd3,0x63,0x74,0x2e,0xad,0x36,0x5d,0x22, +0x18,0x74,0x39,0x8c,0x42,0x8f,0x16,0x7d,0x25,0xd5,0x00,0xe7, +0xa9,0x9d,0x87,0x96,0xcb,0x0f,0x43,0x06,0x35,0x2b,0x2a,0x77, +0x5d,0x1f,0x9c,0x6f,0xec,0x25,0x8a,0x98,0x2e,0x4d,0xa2,0x36, +0x9a,0x75,0x69,0xd1,0xd4,0xa5,0x9e,0x9a,0xa2,0x8e,0x53,0x20, +0x9c,0xb6,0xaa,0x81,0x73,0xef,0xda,0x71,0x22,0xc8,0xbc,0x25, +0xf7,0xe3,0x23,0x49,0x93,0xf9,0xe0,0x11,0x99,0xb3,0x2a,0x0b, +0xca,0xc5,0xeb,0xd2,0x53,0x34,0x42,0x78,0x3b,0xa9,0xea,0x1f, +0x66,0xb7,0xf1,0xd2,0xa5,0x85,0xe1,0x07,0x82,0x1f,0xeb,0x77, +0x09,0x70,0x6d,0x7a,0xe8,0x31,0x93,0x7e,0xcd,0x55,0x72,0xdb, +0xc4,0x50,0x13,0x15,0x19,0x09,0x6e,0x02,0xb2,0x7a,0x43,0x17, +0xd6,0x44,0x05,0x05,0xfe,0x4c,0xf8,0x33,0x3c,0x65,0x50,0xe0, +0xb4,0xab,0x4b,0x5f,0xb5,0x78,0x83,0xd4,0x16,0x6c,0xa3,0x10, +0xbd,0x30,0x1d,0x78,0xe3,0x76,0x68,0xdc,0xb1,0x2e,0x39,0xd5, +0x4d,0xd6,0x62,0x1d,0x64,0xe5,0x76,0xf8,0x13,0xc7,0x5a,0x4e, +0x32,0x81,0xfb,0xfe,0x94,0x4b,0x1f,0x8a,0x30,0xe4,0x0e,0x60, +0x67,0x3e,0x3d,0xf1,0xed,0xb8,0x69,0x13,0x0e,0xca,0x7a,0xe3, +0x67,0xb8,0x2c,0xa1,0x32,0x54,0x76,0x0f,0x13,0x4e,0x3b,0x7b, +0x5b,0xa4,0xe2,0xb7,0xa5,0x87,0xac,0xdc,0xb2,0x5c,0xd9,0xa7, +0xec,0x6e,0x64,0x4c,0x1a,0xd6,0xf5,0x6e,0xf5,0x98,0x92,0x92, +0x57,0x96,0xeb,0x4c,0x6e,0xaf,0x9b,0xfe,0x5c,0x56,0x1f,0xb3, +0xd6,0xf4,0x7c,0xd6,0xbd,0x7f,0xfd,0xa8,0x5a,0x1b,0x9c,0x38, +0xb8,0xfb,0x9f,0x3e,0xdd,0x2f,0xef,0xef,0x07,0x56,0xa3,0x9b, +0x84,0x30,0x58,0x49,0xa1,0xb5,0x70,0x93,0xde,0xee,0x0a,0xce, +0x54,0x6b,0xaf,0x81,0x86,0x8d,0x39,0xf6,0xca,0x87,0x43,0xb5, +0x78,0x20,0x25,0xd9,0x91,0x54,0x38,0xb8,0x7c,0x04,0x10,0xa1, +0xf6,0xa7,0x50,0x6d,0x93,0x57,0x95,0x57,0x32,0x2a,0xb3,0x1a, +0xa9,0x94,0x25,0x9b,0xa4,0xd7,0x4e,0x07,0x02,0xc9,0xd5,0x17, +0x8f,0xd8,0x4d,0x6a,0x6c,0xee,0x28,0x35,0x15,0x4c,0x63,0x14, +0xce,0x63,0x64,0x41,0x91,0x1a,0xa1,0x48,0x8d,0xa6,0x10,0xd1, +0x69,0x9d,0x3b,0xd8,0x35,0xde,0x5f,0xcb,0x64,0xf4,0xb1,0x6c, +0x8e,0xa1,0xce,0xf4,0xda,0xc1,0x5b,0xaa,0x2f,0xfc,0xc1,0x1c, +0xae,0x91,0x95,0x8f,0x67,0x90,0x75,0x37,0x6c,0x75,0xce,0x14, +0x8d,0x89,0xa3,0xd5,0x72,0x2b,0x69,0x85,0xab,0x0f,0x7f,0x30, +0xd2,0x55,0x4e,0xea,0xa7,0x05,0x99,0x87,0x37,0x46,0x12,0x50, +0x34,0x7c,0x4e,0x8e,0xf8,0xa0,0x77,0x40,0x3a,0x68,0x81,0x64, +0x78,0x1b,0x0b,0xab,0x33,0xc7,0x21,0xba,0xed,0x5d,0x8c,0x8c, +0x05,0x3a,0xdf,0x94,0xa2,0x96,0x25,0x72,0xa1,0x9a,0xb9,0xdb, +0xed,0xea,0x69,0xb0,0xd9,0x37,0x39,0x82,0x56,0x73,0xd8,0xc1, +0xb2,0x31,0x34,0x6c,0xbc,0xb5,0x6c,0x84,0xc1,0x49,0x26,0xc2, +0xfc,0x11,0x16,0x8e,0xf0,0x4d,0xcf,0xfa,0xc8,0x2f,0x10,0x3f, +0x56,0x7d,0xf3,0x54,0xa7,0x2a,0xd4,0xd7,0x15,0x5a,0x19,0x5c, +0x23,0x22,0xee,0xb9,0xd6,0xfd,0x7a,0x30,0x69,0x57,0x4f,0x5e, +0x38,0x7e,0x05,0xfa,0x3d,0x7e,0xfb,0x12,0x3d,0x60,0x87,0xb1, +0x4a,0x85,0xf5,0xaa,0xb1,0xd6,0x5f,0xb7,0xfd,0xbf,0xb4,0xb5, +0x77,0xdd,0xec,0x08,0x06,0x21,0xdc,0x8e,0xa9,0xfe,0xde,0x4a, +0xfc,0xed,0x19,0xe4,0x46,0xc9,0xb3,0x0e,0x50,0xc9,0xdd,0x84, +0xe3,0x05,0x1b,0x21,0x3b,0xa4,0xa6,0x44,0xe1,0xad,0xf2,0x1c, +0x6a,0xbe,0x0e,0xa6,0xcd,0x29,0x86,0x5c,0x53,0xfa,0x8b,0xc9, +0xdd,0xc4,0xfb,0xf3,0x4d,0xa4,0x72,0xe2,0xc9,0xec,0xc0,0xfd, +0xdd,0x82,0xe5,0xb9,0x72,0xae,0x52,0xcd,0x6a,0x8f,0x26,0xd8, +0xea,0xc1,0xdf,0xf9,0x84,0xe9,0xa2,0x49,0x42,0x0c,0x23,0x9b, +0xfa,0x34,0x39,0xcb,0x24,0xc9,0x77,0x40,0xbe,0x28,0xd0,0xfa, +0xf9,0xd5,0xdf,0xc8,0x0e,0x2b,0xee,0xe9,0xfd,0x42,0x73,0x7a, +0x35,0xbf,0xa3,0x27,0x46,0x30,0x9b,0x67,0xf0,0x4e,0x5e,0x2b, +0x0f,0x75,0xda,0xe6,0xcd,0x33,0xa8,0xe0,0xcb,0xbb,0xe7,0xfe, +0xf1,0xf4,0xee,0x0a,0xf5,0xbd,0xf3,0x16,0x38,0xa9,0xa0,0x06, +0x17,0x1c,0xc0,0xa5,0x45,0x6d,0x2c,0xb0,0x23,0x44,0x34,0x60, +0x8f,0xaf,0x91,0x4e,0x76,0x8c,0xb3,0x82,0xeb,0xdb,0x90,0xa6, +0x1d,0x31,0x50,0xee,0xdf,0xf8,0x0f,0x11,0xd0,0xc9,0xa6,0xa8, +0xad,0x47,0x93,0x85,0x61,0xf5,0xbb,0x1d,0x82,0xb0,0x26,0x5c, +0x7f,0xd8,0xb6,0x0d,0xf2,0xcb,0x1f,0x33,0x0c,0xd4,0xdf,0xbb, +0xce,0xd0,0x93,0xaf,0xac,0x03,0x26,0x5b,0x3c,0xcd,0xaa,0x11, +0x3b,0x4a,0x3a,0xdf,0x42,0x72,0xa9,0xb9,0x52,0x8c,0x8c,0x48, +0x5f,0x22,0xfb,0x2d,0x1a,0xbb,0x70,0xb0,0xb1,0x0b,0xff,0x77, +0x8d,0xdd,0x38,0x62,0xd6,0xc2,0x11,0xbc,0x13,0x28,0x3a,0x95, +0xa4,0xa9,0x1b,0x6f,0xcc,0x26,0xda,0xa7,0xbe,0xd1,0x5d,0xc1, +0xff,0x1f,0x8e,0x83,0x2e,0xd5,0x43,0xf6,0x77,0x04,0x48,0x9b, +0xc1,0x60,0xe0,0x0d,0x19,0x6c,0x7d,0xc6,0xdf,0x5c,0x50,0xae, +0x9b,0xc2,0x7b,0x6e,0xf5,0x58,0x6f,0x7d,0xbb,0x9c,0xcb,0x6a, +0xc8,0x38,0xd7,0x99,0xa6,0x5d,0x5f,0x8a,0x53,0xb7,0x37,0xf4, +0xb0,0x97,0xd1,0x06,0x87,0x80,0x1e,0x69,0x90,0x80,0xb9,0x95, +0x2a,0x39,0xe2,0xa5,0x77,0x0c,0xcd,0xa2,0xea,0xb7,0x9d,0x82, +0xf8,0x16,0x01,0xf1,0xfa,0xde,0x83,0x32,0x07,0x2f,0x96,0x5f, +0xd3,0xb5,0x1d,0xdb,0x62,0x63,0x7f,0xa1,0xeb,0x96,0x63,0x5d, +0xa7,0x50,0xe6,0x03,0xc5,0xcb,0xdc,0x3f,0x71,0xd0,0xac,0x30, +0x6c,0x1a,0xe7,0x45,0xd8,0xe9,0xe0,0x57,0x43,0xda,0xba,0x76, +0x51,0xdd,0x6f,0xa7,0xf3,0x1d,0xbf,0xdc,0x68,0x7f,0x97,0xcb, +0xe5,0x6b,0x90,0xaa,0xe7,0x03,0xb0,0xd5,0x63,0x75,0xf9,0xcf, +0x6e,0x3a,0x8d,0x7f,0x43,0xdd,0xea,0x8a,0xaa,0xbb,0x64,0x93, +0x8a,0x8f,0x54,0xca,0x7c,0x36,0x7e,0xcf,0xe4,0xde,0x79,0xa3, +0x6e,0x37,0x9b,0x1b,0x59,0xb5,0xbf,0x7c,0x62,0xe6,0x77,0x48, +0x5f,0x83,0x5f,0x2b,0x4d,0xcd,0xef,0x33,0x52,0xc4,0x71,0x1e, +0x3f,0x9f,0x4b,0x6d,0xa6,0x89,0xea,0x6e,0xf6,0x77,0xd9,0xc4, +0x75,0xcd,0xd4,0x6e,0xfa,0x9f,0x13,0xc4,0x7c,0xd5,0xd8,0xd4, +0xc6,0x4b,0x60,0xdc,0xd5,0x59,0xb1,0x55,0x87,0x7c,0xf3,0x1b, +0xa8,0x0b,0xf6,0x83,0xa4,0xed,0xba,0x74,0x8e,0x57,0x41,0x10, +0x86,0x97,0xc8,0xa3,0x50,0x27,0x0f,0xae,0xdf,0xa0,0xdc,0xdf, +0xdf,0x05,0xc1,0x40,0x0f,0xff,0xba,0xc8,0x03,0xc3,0x38,0x77, +0x48,0x0c,0x82,0x9f,0x7f,0x0a,0x57,0x93,0x24,0xd6,0x9e,0xe8, +0x3f,0xa0,0xf4,0x84,0x9d,0x09,0x6b,0x62,0x0b,0xd2,0xf3,0x59, +0xc3,0x31,0xce,0x30,0xc7,0x7e,0x9a,0x51,0x49,0xe1,0x95,0xd9, +0xea,0x51,0x86,0x96,0x5d,0x6d,0x8d,0xf5,0xef,0xa2,0x6b,0xe9, +0xa1,0x61,0x54,0x51,0x13,0xcb,0xf7,0x8c,0xc3,0x99,0x6a,0xa8, +0x8a,0x08,0x94,0x43,0xef,0x0f,0x95,0xe3,0xe2,0x78,0xc2,0x46, +0x6d,0x8b,0xf8,0x2e,0x5d,0x6f,0xe0,0xf6,0xe0,0xfd,0xf9,0x9e, +0x05,0xc4,0x19,0xef,0x53,0xf4,0xc0,0xe1,0x7d,0x13,0x1b,0xa8, +0x81,0xfc,0xbb,0xd6,0xe9,0x77,0x14,0xd9,0xb0,0x32,0xc6,0x04, +0x92,0xd2,0x25,0xd7,0x36,0x5d,0xff,0x6f,0x00,0x00,0x00,0xff, +0xff,0xb7,0x8e,0x2c,0x40,0x78,0x2e,0x00,0x00, + })) + + if err != nil { + panic("Decompression failed: " + err.Error()) + } + + var b bytes.Buffer + io.Copy(&b, gz) + gz.Close() + + return b.Bytes() +} + +func init() { + go_bindata["/styles/main.css"] = styles_main_css +} diff --git a/dashboard/resources/views-browser.html.go b/dashboard/resources/views-browser.html.go new file mode 100644 index 00000000000..42b8095f340 --- /dev/null +++ b/dashboard/resources/views-browser.html.go @@ -0,0 +1,169 @@ +package resources + +import ( + "bytes" + "compress/gzip" + "io" +) + +// views_browser_html returns raw, uncompressed file data. +func views_browser_html() []byte { + gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ +0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0xc4,0x58, +0x7b,0x6f,0xa3,0x48,0x12,0xff,0x7b,0xee,0x53,0xf4,0x70,0xd2, +0x24,0x91,0x00,0xf3,0x6a,0xc0,0x19,0xdb,0xa3,0x9b,0xdc,0x48, +0x39,0x29,0x73,0x73,0xba,0x1d,0x45,0x5a,0xad,0x46,0x51,0x1b, +0x3a,0x86,0x0d,0x06,0x44,0xb7,0xed,0x78,0xb3,0xf9,0xee,0x5b, +0x45,0x63,0x07,0xf0,0x23,0xc9,0xee,0xec,0x6e,0x22,0x9b,0x7e, +0x54,0x57,0xfd,0xaa,0xba,0xba,0xeb,0x67,0x46,0x71,0xba,0x24, +0x51,0xc6,0x84,0x18,0x6b,0x5c,0x46,0xb1,0x11,0x15,0xb9,0x64, +0x69,0xce,0x2b,0x52,0x77,0xa7,0x55,0xb1,0x12,0xd0,0x79,0x78, +0x88,0x8a,0x6c,0x31,0xcf,0xc5,0xe3,0x23,0xb4,0xcb,0x8a,0x2f, +0x53,0xbe,0xaa,0xdb,0x82,0x2d,0xf9,0xe3,0xa3,0x36,0xf9,0x07, +0x81,0xbf,0xd1,0x5b,0xc3,0x50,0x8d,0xbe,0xde,0xb2,0x28,0x8b, +0xe5,0x46,0x6b,0xd3,0x31,0x78,0x55,0x15,0x55,0xb3,0xf4,0xd8, +0x2a,0x23,0x2f,0x64,0x94,0x68,0x93,0xd1,0x00,0x04,0x5e,0x20, +0x8e,0x4e,0xf0,0x5c,0xb6,0x34,0xe3,0xdf,0x17,0x98,0x5a,0x55, +0xa9,0xe4,0x44,0x26,0xa9,0x20,0x4b,0x96,0x2d,0xf8,0x87,0x27, +0x65,0x2f,0xd7,0x7d,0x9b,0x56,0xf3,0x9e,0xee,0xd1,0x74,0x21, +0x65,0x91,0x77,0x56,0x34,0x43,0xad,0xb6,0x21,0xe6,0x2c,0xcb, +0x3a,0x23,0x65,0x95,0xce,0x59,0xb5,0x26,0x9b,0xe0,0x2b,0xdd, +0x5b,0xa8,0xa3,0x81,0x92,0x9b,0xec,0xc3,0xd9,0x6a,0x1a,0xc6, +0xe4,0xd5,0x71,0x27,0x69,0xdc,0x08,0xe1,0x1e,0xfe,0xc5,0x9b, +0xf1,0x09,0xad,0x9d,0x3f,0xe3,0xd5,0xef,0xf3,0x46,0xe5,0xec, +0xdf,0xe2,0x0f,0x79,0xad,0x43,0x09,0x67,0x31,0xf8,0x23,0x8a, +0x2c,0x8d,0xdb,0x58,0x59,0x37,0x93,0x58,0x74,0xa7,0x91,0x7c, +0x66,0x44,0x59,0x1a,0xdd,0x8d,0x35,0xec,0x9f,0x9e,0x35,0x23, +0xb5,0xd8,0xc3,0x2d,0xcb,0x04,0x3f,0x3f,0xa9,0xc5,0xe3,0x54, +0xb0,0x69,0xc6,0xe3,0x93,0xc7,0x9f,0x78,0x8e,0xad,0x1b,0x5c, +0xf0,0xad,0x9f,0xb2,0x62,0x39,0x23,0xe0,0x90,0x48,0x8b,0x7c, +0xac,0xd9,0xa6,0xad,0x91,0xfb,0x79,0x96,0x83,0xb2,0x44,0xca, +0xf2,0x7c,0x30,0x58,0xad,0x56,0xe6,0xca,0x35,0x8b,0x6a,0x36, +0x70,0x2c,0xcb,0x1a,0x80,0x7c,0x23,0x72,0x7e,0x9f,0xa5,0xf9, +0xdd,0x3e,0x41,0x7b,0x38,0x1c,0x0e,0xea,0x59,0x10,0x1d,0x6b, +0x56,0x79,0xaf,0x91,0xb5,0x7a,0x76,0x8c,0xc3,0x0d,0x02,0xd7, +0xca,0x92,0xff,0x4b,0x94,0x3c,0x92,0xff,0x67,0x32,0x2d,0xc6, +0xda,0xfd,0xe7,0x34,0xff,0x11,0x3e,0x1a,0xc1,0xdb,0xe5,0x63, +0x81,0x0a,0x88,0x45,0x02,0xd7,0x74,0xa9,0x4f,0x7c,0x00,0xa8, +0xdc,0xa9,0xe3,0x31,0xab,0x8a,0x45,0x0e,0x5b,0x9e,0xf3,0x15, +0xe9,0x49,0x01,0xc6,0x73,0x51,0xb2,0x88,0x8f,0xb5,0x8d,0x9d, +0xbe,0xef,0x25,0x93,0x09,0x81,0xd5,0x9f,0xa9,0xe9,0x04,0xba, +0xeb,0x9a,0x8e,0xe3,0x67,0x8e,0x63,0x7a,0x4e,0xa8,0xab,0x47, +0x64,0x9b,0xd4,0x77,0x74,0xf5,0xed,0x99,0xd6,0x90,0x36,0x6d, +0x6a,0xfa,0x34,0xd0,0x2d,0x35,0x6f,0xb4,0xa4,0x0c,0x25,0x65, +0x19,0xb5,0xc4,0x95,0x1d,0x98,0x01,0xa8,0xf6,0x4c,0xcf,0x76, +0x13,0x2f,0x34,0xa9,0xed,0x75,0x30,0x44,0x8e,0xe9,0x58,0x43, +0xdd,0xd2,0x3d,0x50,0x12,0x0c,0x6d,0x78,0x7a,0x42,0x35,0x0d, +0x0f,0xff,0x2f,0x51,0x83,0x37,0xcc,0x6c,0xd0,0x67,0x81,0x10, +0x35,0x69,0xe8,0xd4,0x66,0xdd,0xda,0xac,0x5d,0x9b,0xa5,0xb5, +0x59,0x4f,0xb7,0x4c,0x0b,0x84,0x6a,0xd3,0x17,0xae,0x63,0xd2, +0xc0,0x07,0xd4,0x2e,0xa8,0x75,0x41,0x8a,0x82,0x0b,0xba,0x6b, +0x99,0x14,0x9c,0xed,0xa1,0x00,0x55,0x96,0xe3,0x02,0x6a,0xc7, +0xb4,0x3c,0x1f,0xd4,0xb8,0x43,0x68,0x86,0x20,0x08,0x29,0xe1, +0x0f,0xaf,0x30,0x40,0x8e,0xee,0x04,0x60,0x2a,0x88,0x0c,0xcb, +0x0c,0xa8,0xad,0xe3,0x37,0xac,0xb3,0x03,0x17,0x84,0x02,0x3f, +0x6c,0xda,0xb8,0x6c,0x78,0x81,0x70,0x42,0xb4,0xea,0xd1,0x10, +0x20,0x80,0x69,0x80,0xe3,0x01,0x9c,0x56,0xa8,0x7f,0xd1,0x06, +0xbd,0x0d,0xc1,0xec,0x6a,0x5f,0x72,0xec,0xe0,0x61,0x60,0x31, +0x9e,0x94,0xe7,0x52,0xf9,0xe9,0xb0,0x80,0x3c,0x9e,0x95,0x3f, +0x37,0xb7,0xdf,0xbc,0x79,0xf3,0xaa,0x94,0x86,0x0d,0x7a,0x41, +0x4a,0x3f,0x49,0xed,0x4f,0x69,0xb0,0xfa,0x94,0xc9,0x2e,0xc6, +0xda,0xd6,0xc3,0xa5,0x0d,0x3b,0x1a,0x86,0x4b,0xc7,0x5d,0xda, +0xb0,0x51,0x6e,0x10,0x59,0xba,0x4a,0xb5,0x26,0xcd,0xf0,0x7f, +0x5f,0xf2,0x5d,0x7b,0x90,0x02,0x7e,0x70,0x6d,0x0f,0x4d,0xdb, +0x1a,0x5e,0x87,0x11,0xe6,0x04,0x48,0xb5,0x73,0x12,0x4d,0x5e, +0xb8,0xbe,0xe9,0xda,0x75,0x46,0x29,0x93,0xb4,0xd6,0xb0,0xb1, +0x5f,0xef,0x6d,0x07,0x99,0xed,0xa3,0x42,0x75,0x14,0x9c,0x44, +0xc1,0x4b,0x1c,0x37,0x69,0xe0,0xed,0x81,0xb2,0xcf,0xf4,0x25, +0xb5,0x20,0xcb,0xe8,0x25,0xa4,0xa2,0x63,0x07,0x97,0x4a,0x69, +0x64,0x34,0xab,0x0d,0x4f,0x6f,0x64,0x75,0x05,0xd2,0x76,0x94, +0x51,0xc7,0xf4,0x01,0x95,0xed,0x9a,0x1b,0x00,0x7a,0x07,0xce, +0x16,0xed,0xb1,0x04,0xec,0xdf,0xda,0x0d,0x2b,0x32,0xd0,0xc3, +0xfe,0xcd,0x92,0xe6,0xe5,0x42,0x12,0xb9,0x2e,0x61,0xab,0x24, +0xbf,0x97,0x75,0x2a,0xce,0x8b,0x98,0x67,0x6a,0xf1,0xff,0x70, +0x0d,0x8e,0x41,0x0d,0xe1,0xd5,0x58,0x13,0xeb,0x3c,0xba,0x2a, +0x22,0x4c,0x97,0x1c,0xf3,0x54,0xb2,0x69,0x9a,0xc7,0x1c,0x12, +0x25,0x0c,0x43,0x8d,0x0c,0xf6,0x56,0xfe,0xba,0xfb,0x5d,0x28, +0x07,0x96,0xfe,0xf6,0x69,0xc1,0xfe,0xbf,0x99,0x64,0x00,0x65, +0xf2,0x03,0xb4,0xbb,0x04,0xe4,0x58,0x29,0x9b,0x16,0xf1,0xfa, +0x58,0xb9,0xcd,0x52,0xd1,0x2f,0x9a,0x23,0x89,0xb9,0x4f,0x22, +0x9e,0x65,0x25,0x1c,0xd3,0x34,0x9f,0xc1,0xf1,0xd0,0xea,0x3e, +0x26,0x7b,0xd3,0x57,0xfb,0x23,0xb1,0x54,0xd6,0x4d,0xb5,0x32, +0xee,0x28,0xcf,0xd9,0x9c,0x37,0xc5,0x54,0x9b,0xfc,0x17,0x3a, +0xa3,0x81,0x3c,0x2c,0x2d,0x65,0xb6,0x15,0xfe,0xfa,0xf5,0xea, +0xa8,0x2c,0x8b,0x70,0x67,0xc4,0x56,0xfe,0x5d,0x3e,0x15,0xe5, +0x76,0x05,0x34,0xb6,0xb8,0x46,0x12,0x43,0xd0,0x52,0x54,0x61, +0x5c,0x2b,0x5e,0x72,0x26,0xc7,0xda,0x1d,0x5f,0x93,0x34,0x27, +0x18,0x05,0xf2,0x2b,0x29,0x2a,0x50,0xf6,0x71,0x7d,0x7e,0x02, +0xc3,0x27,0xda,0x76,0x4d,0x03,0x60,0x02,0x57,0x5e,0xab,0xb6, +0xcb,0x6a,0x01,0xa5,0x3d,0x4e,0x2b,0xb8,0x57,0x8a,0x6a,0x0d, +0x65,0x1d,0x56,0x99,0xd0,0xff,0xa6,0x91,0xa4,0xe2,0xb7,0x63, +0xed,0x9f,0x83,0xa5,0x3d,0x80,0x41,0xf1,0xf0,0x80,0x53,0xf0, +0x01,0x4a,0x4e,0x92,0x74,0x96,0x64,0xf0,0x91,0x93,0xd6,0x30, +0x66,0x76,0xc7,0xe1,0x8d,0xd3,0x60,0x50,0xac,0x52,0xa0,0x42, +0x04,0x2f,0xd1,0xb7,0x6f,0x71,0x05,0xbf,0x2f,0xd3,0xaa,0x4e, +0x4d,0xad,0x1f,0x40,0x05,0xba,0x71,0x15,0xf7,0x7a,0xbb,0xde, +0x58,0x25,0x1c,0x34,0x20,0x6a,0x60,0x55,0x32,0x9d,0x73,0x52, +0xf1,0x0c,0xb4,0x2c,0x39,0x89,0x99,0xe4,0x38,0x02,0x5e,0x3d, +0x74,0x0d,0x98,0x62,0x31,0x15,0xb2,0x82,0x3d,0x3f,0xb5,0x74, +0xd2,0x9b,0x03,0xd3,0xf2,0x3f,0x78,0x2a,0xbe,0xdc,0x9e,0x9e, +0x18,0x27,0x67,0x67,0xf8,0x8b,0x03,0xbc,0x00,0x4d,0x1b,0xde, +0x76,0x08,0x4c,0xcc,0x6f,0xd9,0x22,0x93,0x3b,0xfb,0x9f,0x17, +0x39,0xc0,0x7b,0x37,0x8f,0x99,0x48,0xde,0xb7,0x75,0xec,0x0b, +0x4e,0xa7,0xbf,0x37,0xbb,0x9b,0x34,0xd1,0x76,0x24,0xf7,0x4a, +0xc3,0x6d,0xc0,0xa5,0x3a,0x77,0x3b,0x31,0x87,0x7d,0xdd,0xaf, +0x65,0x5b,0xe9,0xfa,0x91,0xae,0xb9,0x5f,0xfb,0x10,0x2b,0xf5, +0x37,0xa0,0x0d,0x6f,0x94,0x6e,0x61,0x44,0x9a,0x7c,0xc5,0xd6, +0xbc,0xba,0xf9,0xde,0x84,0xef,0x10,0xe6,0x5e,0xad,0x7c,0x1d, +0xf7,0x83,0xbb,0xda,0x73,0x9f,0x2d,0x94,0x4f,0x52,0x07,0x0b, +0xe5,0xde,0x68,0x6e,0x4b,0x94,0x37,0x84,0x62,0x80,0x55,0x82, +0x5a,0x34,0x33,0x7c,0xd3,0x72,0xa9,0x5e,0x7f,0x5f,0x41,0xb9, +0xb1,0xa1,0x68,0xb8,0xd4,0x0c,0x2d,0xb7,0x33,0x15,0x75,0xc8, +0x5f,0x9b,0x28,0x5a,0x8a,0x24,0x3e,0x47,0x21,0x0f,0xc1,0xca, +0x40,0x3d,0x45,0x4b,0x34,0x83,0x5a,0xe5,0xb8,0x50,0x18,0xeb, +0xc7,0xd3,0xf8,0x73,0xe4,0xf3,0x82,0xd6,0x9e,0xe8,0x50,0x6b, +0x87,0x9e,0xab,0x43,0xe5,0x1c,0x02,0x89,0x6b,0x7a,0x6d,0x5f, +0x37,0xe5,0xef,0x78,0x74,0x1c,0x28,0xd9,0xc0,0x65,0x91,0x18, +0xf8,0x4e,0xd6,0x8e,0x8e,0x07,0xac,0x3b,0xa0,0xba,0x07,0x6c, +0xcf,0x6f,0x4f,0xfc,0x51,0xf6,0x7c,0x30,0x34,0xc6,0x36,0x06, +0x57,0x50,0xda,0x83,0xd0,0x45,0x54,0x94,0xd2,0xd6,0x44,0xb3, +0x2f,0x46,0x4b,0x6d,0xd3,0x6e,0x8c,0x5f,0xe0,0x12,0x1b,0x10, +0x51,0xe4,0x3b,0xfa,0xa6,0x17,0x98,0xbe,0xde,0xf1,0xf4,0x68, +0x70,0x14,0x5f,0x38,0x34,0x7b,0xe4,0x42,0xdc,0x5e,0x35,0xbb, +0x2a,0xf7,0xcd,0xec,0x8e,0x76,0xcb,0xd5,0x40,0x56,0x9b,0x4a, +0xb4,0x2d,0x3f,0xd0,0xc6,0xb3,0x72,0x98,0x3a,0xec,0xfc,0xb6, +0x55,0x2f,0x74,0x76,0xca,0x33,0xf0,0x17,0x56,0x71,0x46,0xca, +0x0c,0x4e,0x54,0x52,0x64,0x31,0x32,0x96,0x4f,0x48,0x5c,0x08, +0xc3,0x3b,0x9a,0x60,0xdd,0x25,0x6c,0x0a,0x3f,0x8c,0x09,0xcb, +0x63,0x02,0xc5,0x50,0xbd,0x57,0x21,0x09,0xaf,0x78,0x9b,0xfa, +0x08,0xb8,0xd8,0x33,0x7e,0x8d,0x73,0x3b,0x1c,0x07,0x2f,0xae, +0x84,0xe5,0x33,0x38,0xb3,0x22,0x29,0x56,0x48,0x3b,0x90,0x7e, +0xf4,0x7e,0x20,0x6c,0xc0,0xf4,0xc6,0xfb,0xae,0xf0,0x79,0x29, +0xd7,0xbd,0xc5,0x87,0x05,0x8d,0x39,0x17,0x82,0xcd,0x60,0x5f, +0xb6,0x6f,0xb5,0x6e,0x9a,0x21,0x2c,0x95,0x9d,0xa8,0xed,0x09, +0xe4,0x5e,0xc5,0x71,0xca,0xb2,0x62,0xf6,0x12,0x08,0x4a,0xf2, +0x09,0xc3,0xce,0x0a,0xfc,0xc3,0x70,0xd4,0xc1,0x05,0x26,0x01, +0xd7,0x3b,0xdb,0xbc,0xbf,0x52,0x21,0xfb,0xb0,0x6b,0x65,0x17, +0xe2,0x31,0xe3,0x8a,0xde,0x89,0x03,0xc6,0x5f,0x48,0x32,0x1b, +0x4a,0xa9,0x28,0x23,0xb9,0xa8,0x91,0x89,0xdd,0x77,0x57,0x1d, +0xcd,0xac,0xe1,0x2e,0x3f,0xb3,0x25,0x13,0x51,0x95,0x96,0xf2, +0x7c,0x59,0xa4,0xf1,0xa9,0x75,0xf6,0x5e,0x9b,0x5c,0xb0,0x1c, +0x58,0x60,0x87,0x85,0x1f,0xdb,0x82,0x5e,0x7a,0xef,0xbc,0x7b, +0x69,0x1e,0xbf,0x05,0x00,0x00,0xff,0xff,0xd1,0x7f,0x20,0xf2, +0xee,0x14,0x00,0x00, + })) + + if err != nil { + panic("Decompression failed: " + err.Error()) + } + + var b bytes.Buffer + io.Copy(&b, gz) + gz.Close() + + return b.Bytes() +} + +func init() { + go_bindata["/views/browser.html"] = views_browser_html +} diff --git a/dashboard/resources/views-stats.html.go b/dashboard/resources/views-stats.html.go new file mode 100644 index 00000000000..f62994bf692 --- /dev/null +++ b/dashboard/resources/views-stats.html.go @@ -0,0 +1,95 @@ +package resources + +import ( + "bytes" + "compress/gzip" + "io" +) + +// views_stats_html returns raw, uncompressed file data. +func views_stats_html() []byte { + gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ +0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0xd4,0x56, +0x5b,0x8f,0xe3,0x34,0x14,0x7e,0xdf,0x5f,0x61,0x99,0x87,0x05, +0x29,0x37,0x27,0xbd,0x4c,0xaa,0xa6,0x12,0x0b,0xe2,0x89,0xe1, +0x81,0x95,0x90,0x78,0x42,0xae,0xe3,0x69,0xac,0x71,0x9d,0x60, +0xbb,0xcd,0xcc,0x96,0xfe,0x77,0x4e,0xe2,0xa4,0x4c,0x93,0x10, +0x2d,0x62,0x5f,0x76,0x2e,0x72,0xce,0xed,0x3b,0x9f,0x4f,0x3f, +0x3b,0xdd,0xe6,0xe2,0x8c,0x98,0xa4,0xc6,0x64,0x98,0x5b,0x96, +0xfb,0xac,0x54,0x96,0x0a,0xc5,0x35,0x6a,0x4d,0x63,0xa9,0x35, +0xe8,0x72,0x61,0xa5,0x3c,0x1d,0x95,0xb9,0x5e,0xe1,0xd9,0xd2, +0xbd,0xe4,0xbf,0x09,0x23,0xf6,0x42,0x0a,0xfb,0x7a,0xbd,0xe2, +0xdd,0x3b,0x04,0x3f,0x23,0xac,0x7d,0x99,0xbf,0x76,0xb1,0xc9, +0xf8,0x53,0xa9,0x8f,0xd4,0xfa,0x86,0x4b,0xce,0x6c,0xa9,0xdf, +0xa4,0x4e,0xa6,0xf7,0x79,0xbe,0xb0,0xfc,0x88,0xee,0x5d,0x07, +0x4d,0xab,0x02,0x23,0x75,0xf0,0x99,0x14,0xec,0x39,0xc3,0xa6, +0x28,0xeb,0x3f,0x5a,0xef,0xb7,0xdf,0x0d,0x80,0x5b,0x70,0x73, +0x3e,0xa0,0x33,0xd7,0x46,0x94,0x2a,0xc3,0x24,0x20,0x18,0xbd, +0x1c,0xa5,0x82,0x56,0x85,0xb5,0xd5,0x26,0x0c,0xeb,0xba,0x0e, +0xea,0x24,0x28,0xf5,0x21,0x8c,0xa3,0x28,0x0a,0x21,0xbf,0x4b, +0xd9,0xbc,0x48,0xa1,0x9e,0xa7,0x12,0x49,0x9a,0xa6,0x61,0x1b, +0x85,0xd4,0x0c,0x47,0xd5,0x0b,0x46,0xaf,0x6e,0x1d,0x11,0xa8, +0x34,0x37,0x5c,0x9f,0xf9,0xf7,0xa6,0x82,0x1d,0xfc,0x4a,0xad, +0x28,0x33,0xfc,0xf2,0x28,0xd4,0xef,0xf0,0x8f,0xd1,0x59,0xf0, +0xfa,0x43,0xd9,0x80,0xa0,0x08,0xad,0x9a,0xbf,0x20,0x8a,0xd6, +0x18,0x71,0xd5,0x0c,0xdf,0xdf,0x53,0xf6,0x7c,0xd0,0xe5,0x49, +0xe5,0x19,0x56,0xbc,0x46,0x83,0x2c,0xe0,0xb9,0x31,0x15,0x65, +0x3c,0xc3,0x7d,0x9f,0xa9,0x19,0x54,0xd4,0x16,0xe8,0x49,0x48, +0x99,0xe1,0x6f,0xc8,0x8f,0xf0,0xfb,0x01,0x23,0x00,0x7c,0x4c, +0x22,0x1f,0x80,0x52,0xc2,0x7c,0xb2,0x0a,0x56,0x31,0xf1,0x22, +0x3f,0x69,0x1c,0x0b,0x8f,0x24,0xc1,0x62,0xbd,0xe8,0x2d,0xb7, +0xb0,0xc8,0xeb,0xd2,0x5c,0xd4,0xbb,0x8b,0x76,0xcb,0xa8,0xb9, +0x71,0x7e,0xff,0xae,0xa6,0x03,0xfe,0x61,0x75,0xeb,0x96,0x3c, +0x24,0xde,0xa2,0x45,0x77,0x94,0xbc,0x9e,0xdb,0x27,0x04,0x34, +0xbd,0xc5,0x3a,0x88,0x93,0x94,0xf9,0x69,0xb0,0x24,0x29,0xd0, +0x24,0x8d,0xbd,0xf4,0xd7,0xc1,0x9a,0xac,0x7a,0xc3,0x2d,0x23, +0x02,0x1f,0xe3,0x28,0x58,0x3c,0x00,0xe9,0x38,0x58,0xaf,0x1e, +0x00,0xb7,0x7b,0x62,0x1d,0x96,0xe7,0xea,0xbc,0x16,0xab,0x37, +0xdc,0xf2,0x31,0x71,0x39,0xae,0xbb,0x77,0xe3,0xf1,0x09,0x87, +0x13,0x53,0x6e,0xa4,0xb3,0x7b,0x77,0x2f,0xed,0x10,0xb4,0xfd, +0x3f,0xd4,0xde,0x9e,0xc0,0x91,0xda,0x5b,0xef,0xd7,0xaf,0xf6, +0x65,0x1a,0x2c,0xd2,0x55,0xb3,0x2c,0xa3,0x64,0x56,0xf1,0x83, +0xcc,0xcf,0x54,0xbd,0x06,0x06,0x2d,0x61,0x47,0x17,0x0f,0x4f, +0x40,0x2d,0x72,0x5b,0x64,0xd8,0x81,0x63,0x54,0x70,0x71,0x28, +0x2c,0x4c,0x2d,0x09,0x08,0x21,0x93,0x9f,0xf0,0x3d,0x62,0x0c, +0x89,0x4d,0xe1,0x17,0x86,0x85,0x53,0x90,0xa4,0xf1,0x17,0x80, +0x75,0x7a,0x9c,0x91,0xe3,0xd0,0x1c,0x2a,0xd3,0xdd,0xb5,0x03, +0x88,0x22,0xde,0xfd,0x54,0x4a,0x59,0xd6,0xf0,0xe6,0xf8,0x99, +0x5a,0xae,0xd8,0xeb,0x36,0x04,0xe7,0xbc,0xc8,0x5b,0xa8,0x7f, +0xde,0x39,0x18,0x09,0xf8,0x78,0xa5,0x2b,0x1f,0x76,0xf8,0x6f, +0x24,0xa5,0x30,0x76,0x96,0x23,0xc4,0x27,0x08,0xb6,0x67,0x08, +0x31,0x2e,0x65,0x45,0xf3,0x5c,0xa8,0x43,0x3b,0xfe,0xc6,0x6e, +0x94,0xd5,0xd9,0xc3,0x9a,0x82,0xd3,0x7c,0x62,0xd0,0x36,0xbf, +0x63,0xa4,0xe8,0x91,0xfb,0x4d,0x2a,0xec,0x73,0xf7,0x48,0x59, +0x01,0x3b,0x46,0xbf,0x80,0x73,0x1b,0xda,0xcf,0x28,0xbf,0x4d, +0xe5,0x36,0xdd,0x61,0x15,0x78,0xc6,0x4c,0xb6,0xb6,0x79,0x0b, +0x4f,0xc1,0xeb,0xe6,0xfe,0xd0,0xbc,0xe2,0x14,0xf4,0xf2,0xd4, +0xcf,0x45,0x28,0xd4,0x3f,0x9b,0x89,0xd3,0xd3,0x31,0xdb,0x5d, +0x2e,0x7d,0x56,0xd0,0xec,0xeb,0x7a,0x9d,0xde,0x44,0x9f,0x3e, +0x19,0x68,0x83,0xa3,0x6b,0xef,0xcf,0x13,0xd5,0xdc,0x5d,0x6d, +0xe0,0xdd,0xa0,0xcb,0xfb,0x37,0x7e,0x10,0x0c,0xe7,0xea,0xfd, +0xe6,0xc6,0x31,0xe8,0xc6,0x12,0xb0,0x93,0xd6,0x5c,0x59,0xb4, +0x45,0xb1,0x87,0xee,0x4a,0x4a,0x4d,0xd5,0x81,0xcf,0xd7,0x2c, +0x07,0x35,0x9a,0xe7,0x73,0x05,0xbb,0x0c,0x91,0x08,0xbe,0xf6, +0x4c,0xdc,0xe3,0xb3,0x9b,0xeb,0x70,0xfc,0x33,0x95,0x27,0xb8, +0x9b,0xde,0x0c,0x71,0xd8,0xe1,0x2f,0xa4,0x4e,0xc7,0x3d,0xd7, +0x1b,0x82,0xe0,0x3b,0xd7,0xd1,0xcc,0x74,0xfa,0x17,0xf9,0x84, +0x56,0x8f,0xe4,0x31,0x96,0x02,0x38,0x1b,0xc5,0x4f,0x9e,0xab, +0xee,0xb1,0x5b,0xfe,0x0e,0x00,0x00,0xff,0xff,0xf4,0x18,0x69, +0x45,0x2a,0x0a,0x00,0x00, + })) + + if err != nil { + panic("Decompression failed: " + err.Error()) + } + + var b bytes.Buffer + io.Copy(&b, gz) + gz.Close() + + return b.Bytes() +} + +func init() { + go_bindata["/views/stats.html"] = views_stats_html +} From 8acaf71a593f23d8a9ead225b28959ada83acc6e Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Mon, 7 Oct 2013 23:33:16 -0700 Subject: [PATCH 196/247] feat(dashboard): introduce the in memory handler the in memory handler gives etcd the ability to serve a dashboard without on disk resources. This is the first time we are using the /mod/ path too. TODO: cleanup the mod stuff so it isn't hacked into etcd_handlers. --- etcd_modules.go | 42 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 42 insertions(+) create mode 100644 etcd_modules.go diff --git a/etcd_modules.go b/etcd_modules.go new file mode 100644 index 00000000000..a0397bd76b1 --- /dev/null +++ b/etcd_modules.go @@ -0,0 +1,42 @@ +package main + +import ( + "bytes" + "net/http" + "os" + "time" + + "github.com/coreos/etcd/dashboard/resources" +) + +func DashboardMemoryFileServer(w http.ResponseWriter, req *http.Request) { + path := req.URL.Path + if len(path) == 0 { + path = "index.html" + } + + b, ok := resources.File("/" + path) + + if ok == false { + http.Error(w, path+": File not found", http.StatusNotFound) + return + } + + http.ServeContent(w, req, path, time.Time{}, bytes.NewReader(b)) + return +} + +// DashboardHttpHandler either uses the compiled in virtual filesystem for the +// dashboard assets or if ETCD_DASHBOARD_DIR is set uses that as the source of +// assets. +func DashboardHttpHandler(prefix string) (handler http.Handler) { + handler = http.HandlerFunc(DashboardMemoryFileServer) + + // Serve the dashboard from a filesystem if the magic env variable is enabled + dashDir := os.Getenv("ETCD_DASHBOARD_DIR") + if len(dashDir) != 0 { + handler = http.FileServer(http.Dir(dashDir)) + } + + return http.StripPrefix(prefix, handler) +} From 6e278294afdac149aa0dcc295c51f1d7ff399ada Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Tue, 8 Oct 2013 11:04:48 -0700 Subject: [PATCH 197/247] fix(dashboard): add d3 to bower --- dashboard/bower.json | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dashboard/bower.json b/dashboard/bower.json index 51ed4e5adce..642e8f9fc3d 100644 --- a/dashboard/bower.json +++ b/dashboard/bower.json @@ -10,7 +10,8 @@ "angular-route": "~1.2.0-rc.2", "angular-resource": "~1.2.0-rc.2", "angular-cookies": "~1.2.0-rc.2", - "angular-sanitize": "~1.2.0-rc.2" + "angular-sanitize": "~1.2.0-rc.2", + "d3": "~3.3.6" }, "devDependencies": { "angular-mocks": "~1.2.0-rc.2", From 13f7257dd7a550649e60a8db229c46a7c91afddc Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Tue, 8 Oct 2013 11:06:22 -0700 Subject: [PATCH 198/247] fix(dashboard): add moment to bower dependencies --- dashboard/bower.json | 3 +- dashboard/build | 16 +- .../resources/scripts-browser-modules.js.go | 4746 ++++++++--------- .../resources/scripts-stats-modules.js.go | 4541 +++++++--------- 4 files changed, 4272 insertions(+), 5034 deletions(-) diff --git a/dashboard/bower.json b/dashboard/bower.json index 642e8f9fc3d..881b4e7e7af 100644 --- a/dashboard/bower.json +++ b/dashboard/bower.json @@ -11,7 +11,8 @@ "angular-resource": "~1.2.0-rc.2", "angular-cookies": "~1.2.0-rc.2", "angular-sanitize": "~1.2.0-rc.2", - "d3": "~3.3.6" + "d3": "~3.3.6", + "moment": "~2.3.0" }, "devDependencies": { "angular-mocks": "~1.2.0-rc.2", diff --git a/dashboard/build b/dashboard/build index 0ae99eb45db..848eaa2505c 100755 --- a/dashboard/build +++ b/dashboard/build @@ -1,11 +1,17 @@ -#!/bin/sh +#!/bin/bash -e +DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" +cd ${DIR} +git clean -x -f dist + +npm install +bower install grunt build -git clean -x -f dashboard/dist +export GOPATH="${DIR}/../" -for i in `find dashboard/dist -type f`; do - file=$(echo $i | sed 's#dashboard/dist/##g' | sed 's#/#-#g') +for i in `find dist -type f`; do + file=$(echo $i | sed 's#dist/##g' | sed 's#/#-#g') go build github.com/jteeuwen/go-bindata - ./go-bindata -pkg "resources" -toc -out dashboard/resources/$file.go -prefix dashboard/dist $i + ./go-bindata -pkg "resources" -toc -out resources/$file.go -prefix dist $i done diff --git a/dashboard/resources/scripts-browser-modules.js.go b/dashboard/resources/scripts-browser-modules.js.go index da47a3537f8..5b4a8a6a7ae 100644 --- a/dashboard/resources/scripts-browser-modules.js.go +++ b/dashboard/resources/scripts-browser-modules.js.go @@ -6919,2569 +6919,2189 @@ func scripts_browser_modules_js() []byte { 0x85,0xbb,0x44,0x17,0x93,0x2c,0xed,0x95,0x3a,0x4e,0xfa,0x37, 0xec,0x48,0x38,0x20,0x13,0xbc,0xb6,0xc2,0x23,0xfa,0x99,0x2c, 0x70,0xa4,0x9a,0x47,0xe7,0xf8,0xa4,0x57,0x73,0xfc,0xff,0x50, -0xf7,0xfe,0xff,0x6d,0xdb,0xc8,0xa2,0xe8,0xcf,0xef,0xfc,0x15, -0x36,0x4f,0xd6,0x21,0x23,0x58,0xb6,0x9c,0xb4,0xdd,0x52,0xa1, -0x75,0xd3,0x38,0x69,0x73,0x6e,0x9c,0xe4,0xc6,0xce,0x66,0xb7, -0xb2,0xea,0x8f,0x6c,0xc9,0x36,0x1b,0x59,0x72,0x45,0xa9,0x49, -0x6a,0xe9,0xfd,0xed,0x0f,0xf3,0x05,0xc0,0x00,0x24,0x65,0x25, -0xed,0x9e,0xfb,0xce,0x6e,0x63,0x11,0xc0,0x60,0x30,0xf8,0x3e, -0x18,0x0c,0x66,0xce,0x61,0x89,0x82,0xb0,0x79,0x28,0xa5,0xc4, -0xd0,0x2a,0xed,0x66,0x80,0x4d,0xf9,0xb8,0x94,0x87,0xc9,0xbe, -0xc3,0xb2,0x35,0x51,0x72,0x0b,0x94,0x77,0x30,0x4f,0xf5,0x30, -0x93,0xe3,0xaf,0x72,0xb4,0xae,0x37,0x0e,0x0b,0x7b,0x81,0x5d, -0x4c,0xcf,0xf5,0x71,0xc0,0x78,0x89,0xe1,0x06,0xe1,0x33,0x81, -0x4e,0x53,0x93,0x31,0x18,0xf5,0xb5,0x20,0x14,0xc4,0x27,0x21, -0x60,0xcb,0x93,0xea,0x60,0x53,0x21,0xaa,0xc0,0xa8,0x76,0xf5, -0x72,0x71,0xcf,0x37,0xb6,0xc2,0x06,0x43,0x31,0x89,0x03,0x6e, -0xb4,0xc3,0x51,0x17,0x65,0x8e,0xc3,0x29,0x18,0x94,0x83,0x81, -0x34,0xbf,0x79,0xd9,0x2f,0x66,0x4c,0x24,0x1b,0x01,0x94,0xac, -0xb6,0x44,0x08,0xa3,0x4d,0x04,0xbd,0xe1,0x2a,0xe2,0xd1,0x79, -0x6c,0x12,0x94,0x5e,0x31,0x55,0xbc,0xf4,0x10,0x9e,0x90,0xfc, -0xc7,0xd2,0xf7,0xa1,0x81,0x6c,0x02,0x2e,0x85,0x4f,0x8a,0xb7, -0x4e,0x03,0x33,0xa6,0x26,0x97,0x5a,0x4f,0xd3,0x73,0xb6,0x3e, -0x74,0x95,0x17,0x74,0xc6,0x7f,0x31,0xc8,0x1a,0x0d,0xaf,0x0d, -0xda,0x1a,0xaa,0x43,0x03,0x0a,0x78,0x0f,0xc8,0xa4,0x6e,0xd1, -0xb0,0x78,0xea,0x8f,0xaf,0x65,0xd2,0x2c,0xe6,0xe7,0xe7,0x43, -0xf0,0x6a,0x62,0x4a,0xd0,0xa3,0xe8,0x66,0x32,0x2e,0x48,0x5b, -0xc9,0x2b,0x25,0xcb,0xbc,0x52,0x58,0x01,0x67,0xf8,0x91,0x1a, -0x47,0xce,0xd1,0xb6,0x1c,0x3c,0xb1,0x01,0x29,0xcf,0xd1,0xba, -0x8e,0x2a,0x35,0xbc,0xc1,0x10,0xb4,0x25,0x4d,0x61,0x3f,0x92, -0xf8,0x2e,0x5b,0x8b,0xd2,0xac,0xf7,0xc7,0x0e,0xf4,0x86,0x1b, -0x6a,0x76,0x86,0x05,0x60,0x4d,0x63,0x72,0x27,0x4e,0x12,0x6f, -0xa8,0x78,0x76,0x71,0xbd,0x51,0x0e,0x87,0x65,0x19,0xde,0xda, -0xda,0xf4,0x4e,0x53,0x01,0xac,0x37,0xd1,0x83,0xea,0xeb,0x2c, -0xd7,0xb0,0xad,0xdf,0xcb,0xbd,0x46,0x62,0x2b,0xdb,0x89,0x77, -0x4a,0xb3,0x73,0x90,0x24,0x3f,0x4d,0x94,0xbf,0xc7,0x81,0xa8, -0xbb,0xae,0x47,0xc1,0x43,0x4c,0x6d,0x8f,0x2c,0x5d,0x41,0x55, -0xd4,0xf0,0x3b,0x44,0x20,0x28,0x49,0x57,0xa2,0x71,0x2b,0x6a, -0x3e,0x5b,0x97,0xe3,0x5c,0xa5,0x89,0x23,0x57,0xad,0xa2,0x2c, -0x0b,0x2b,0x9a,0x54,0x14,0x29,0x47,0xe9,0x72,0x5f,0x4d,0xc6, -0xc0,0xbb,0xc3,0x0d,0x55,0x4d,0xf1,0x72,0xe5,0x94,0x5b,0x3a, -0x66,0x7f,0xa3,0x39,0x2d,0x64,0x97,0xbc,0xad,0x80,0xb5,0x38, -0x55,0xed,0x81,0x8e,0x00,0x94,0x7f,0xb0,0x43,0x2b,0xdb,0x6f, -0x9f,0x3c,0x7d,0x96,0xed,0xdc,0x2e,0x77,0x8c,0x65,0x12,0x4f, -0x2b,0x63,0x1d,0x97,0x4d,0xa4,0xf0,0x05,0x17,0x95,0x76,0x79, -0x3d,0x87,0x1e,0x55,0x1f,0xaf,0x86,0x63,0x1b,0x45,0x7b,0x3f, -0x44,0x05,0xfe,0xd4,0x82,0xd4,0x04,0x6c,0x7f,0x69,0x76,0x9e, -0x57,0x71,0xfc,0x5e,0x2c,0x76,0x11,0x59,0xe1,0x79,0x19,0x66, -0xf4,0x68,0x7c,0x84,0x92,0x75,0x08,0x4c,0x96,0xe8,0x20,0x5e, -0xef,0x1d,0x7d,0xbe,0x3e,0x9b,0x8c,0xbc,0x93,0x68,0x53,0x24, -0xe8,0x71,0x3e,0x1c,0x0f,0xaa,0x80,0x6c,0x74,0x0c,0x0f,0x8f, -0xde,0x6b,0xd4,0xd9,0xce,0x2f,0x50,0x42,0x0c,0x3a,0xc7,0x45, -0xd2,0x89,0x9b,0x20,0x4c,0xb1,0x7c,0x37,0xee,0x58,0x6e,0x1f, -0xb3,0x57,0x93,0xca,0x5a,0x50,0x66,0x05,0x4c,0xc4,0x45,0x82, -0x11,0x3f,0x49,0xaf,0xe3,0x58,0x83,0x40,0xe3,0xc0,0x02,0x58, -0x77,0x1a,0x11,0x80,0x45,0x9e,0x87,0x0d,0x54,0x84,0x86,0x7b, -0xfb,0x6d,0x3d,0xf8,0x7b,0x59,0x4d,0xe3,0x76,0x3d,0x7c,0xe4, -0xca,0xc8,0xd0,0x8f,0x45,0x57,0x56,0x00,0xed,0x6a,0x88,0xb6, -0x45,0x7b,0x19,0xc1,0xd9,0xde,0x42,0x5b,0x8a,0x70,0x50,0xc9, -0x3e,0x68,0xd8,0xf1,0xd1,0x80,0x6b,0x7b,0xea,0xd5,0x86,0x6d, -0xe6,0x44,0x4c,0xef,0x15,0x97,0x1e,0xc2,0x33,0xa8,0x1c,0x07, -0x16,0xb7,0x33,0x83,0x42,0xbe,0x5a,0x58,0x7a,0x06,0xf6,0x85, -0xf1,0x13,0xac,0x90,0x62,0x5d,0x16,0x0b,0x16,0x99,0xf0,0xbc, -0x68,0xde,0xe0,0xac,0x7a,0x6a,0x1c,0xc1,0x6e,0x13,0x81,0x49, -0x22,0x87,0x55,0x17,0x93,0x7a,0xc1,0x14,0xd8,0x04,0x37,0x3c, -0xe2,0xb8,0xc3,0x2e,0xca,0x3c,0xd1,0x82,0x71,0x18,0x96,0xf0, -0xd2,0xf3,0x76,0x78,0x33,0xec,0xfb,0x7a,0x2f,0x7c,0x96,0xa9, -0xe4,0xfd,0x31,0x4d,0xb9,0x23,0x00,0xda,0x42,0xfe,0xf1,0xf4, -0xed,0xb3,0xc3,0xd7,0xff,0x78,0x76,0x90,0x45,0xf7,0xee,0xb9, -0x50,0x64,0xd1,0x97,0xdc,0xea,0x50,0x74,0x64,0xda,0xe8,0x0b, -0x8f,0x10,0xab,0xb9,0x35,0x58,0x24,0xdc,0xc1,0x76,0xad,0x13, -0x83,0x2e,0xfe,0xfc,0xc3,0x0f,0x9f,0x75,0xdb,0x2a,0xf7,0xc9, -0x4a,0xf3,0x1c,0xf1,0x62,0xf0,0x7c,0xec,0xbe,0x51,0x22,0x2a, -0x23,0x5e,0x9f,0xfd,0xaa,0x83,0xa3,0xab,0x42,0x4d,0xf5,0x3f, -0xec,0x9d,0x17,0xf8,0x7a,0xf2,0x22,0xd7,0x38,0xf4,0x40,0x15, -0x21,0x37,0x48,0x8d,0xea,0x02,0xb7,0x07,0xf9,0x4c,0xcf,0xc4, -0x20,0xb6,0x3e,0xd8,0x8b,0x07,0x30,0xcd,0x4f,0x8a,0x46,0x3e, -0xd6,0x7f,0xe2,0xe6,0x83,0x4e,0x82,0xea,0x58,0x45,0x03,0x49, -0xd0,0xbf,0x67,0x9f,0x31,0x01,0x80,0x1e,0x80,0xfe,0x57,0x82, -0x2f,0x3c,0x9f,0x8f,0x5f,0xe2,0x83,0x9b,0xec,0xf6,0x5e,0x3e, -0x48,0xd9,0x02,0x0d,0xba,0x7d,0xdf,0x44,0xdc,0xac,0xe0,0xe0, -0x77,0x54,0x1c,0xe5,0x81,0xaf,0x73,0x47,0x12,0x8c,0x5b,0x10, -0x1c,0xc3,0x43,0xd9,0xfb,0xa7,0xb9,0xe6,0xa1,0x4e,0x21,0xea, -0xf4,0x1c,0x44,0x10,0xd8,0xd0,0xa7,0x5d,0x6a,0xcf,0x8d,0xb3, -0xcf,0x1b,0xa7,0xf9,0xe0,0xb4,0x77,0x1f,0xfd,0xa2,0x4b,0x63, -0x28,0x0e,0x1d,0x5e,0x15,0xea,0x66,0xcb,0xac,0xdf,0xf8,0xa9, -0x0d,0xec,0xf5,0x44,0x6f,0x70,0xdc,0x23,0x19,0xd7,0x89,0xc3, -0xde,0x32,0x87,0x68,0x17,0x9f,0xc8,0x0e,0xcc,0xa4,0x8d,0x1e, -0xea,0x25,0xe5,0xbb,0xbc,0xf0,0x7a,0x4a,0x2f,0x84,0xb2,0x0d, -0xbb,0x5e,0x22,0xda,0x6a,0xf1,0x1b,0xb9,0x1b,0xf4,0xbb,0x31, -0x1c,0x2e,0x61,0xd8,0x35,0x60,0x46,0x86,0x21,0x43,0xfa,0xcd, -0x20,0x95,0x39,0xc0,0xb3,0x56,0x1c,0x8e,0xbc,0x8a,0x9a,0xd8, -0x3a,0x18,0x3b,0x43,0x2c,0x45,0x0c,0x06,0xa9,0x97,0x53,0xd6, -0x7b,0xc9,0x6e,0xfd,0x33,0xdd,0x1d,0x76,0xe0,0xc5,0x9d,0x34, -0xee,0x9e,0xdc,0x3b,0xf9,0x08,0x8f,0x40,0x4e,0x62,0xfb,0xad, -0x07,0x99,0x42,0x03,0x44,0x26,0x9c,0x24,0x30,0xe4,0x56,0x0f, -0xaa,0x7c,0x40,0xc3,0x8a,0xc7,0xcd,0x7d,0x18,0x38,0x35,0x63, -0xe8,0xfe,0x46,0x71,0x35,0x99,0x8f,0x06,0xf8,0xd2,0x5a,0x0f, -0x3a,0xdb,0xa6,0xf0,0xdc,0xfa,0x7e,0x7c,0xaa,0xe9,0x3d,0x55, -0x1b,0xa7,0x58,0xc5,0xd3,0xe4,0xbe,0x18,0xa0,0xaa,0x34,0xda, -0x74,0x7d,0xd8,0x8e,0xb2,0xeb,0x9a,0xcc,0xbd,0xdf,0xb7,0x23, -0xcf,0xeb,0x5e,0x3b,0x06,0xe9,0x31,0x8a,0x66,0xdf,0x7e,0xd0, -0xeb,0xf3,0x87,0x43,0x30,0xa6,0xba,0xf4,0x45,0x09,0x4f,0x2d, -0xd9,0x31,0x4c,0x7e,0xc7,0xda,0xdb,0x78,0x69,0x0b,0x94,0x2d, -0x41,0x83,0x71,0x1b,0xb2,0x85,0x03,0x1d,0x4a,0xfe,0xd2,0x85, -0x44,0x40,0xb9,0x01,0x6a,0xbb,0x4f,0x39,0x84,0xba,0x7b,0x0b, -0x75,0x06,0x04,0xa1,0xe2,0x70,0x3e,0x99,0x17,0x68,0x8b,0xc7, -0x2c,0x6e,0x70,0x21,0x0c,0x25,0x08,0x9a,0x5d,0xf8,0xf5,0x14, -0xcd,0x2e,0xf5,0xda,0xce,0x05,0xe3,0xcb,0xfc,0xc3,0x50,0xd2, -0x9b,0xf8,0x45,0x65,0x2e,0xe8,0x4d,0x27,0xf1,0xbd,0x58,0x84, -0x43,0x94,0x5e,0xe6,0xdf,0x0d,0x4e,0x4b,0x67,0x50,0xa0,0xa6, -0x4e,0x98,0xeb,0x16,0x94,0xb9,0xcf,0xc0,0x4f,0x20,0x0e,0xe7, -0xad,0xad,0xe8,0x5e,0xb4,0x09,0x33,0xd3,0x19,0x32,0xd0,0xac, -0xbc,0x87,0x9b,0x7d,0x3e,0x6a,0xe8,0x76,0x90,0x00,0x6e,0x69, -0x34,0x5b,0x0e,0x05,0x8b,0x5e,0xc9,0x02,0x28,0xee,0x40,0xf6, -0xf4,0xef,0xb7,0x2a,0x27,0xd6,0x64,0xa1,0x99,0xbf,0xdb,0x66, -0x3f,0xfe,0xe4,0x6e,0x11,0xff,0x92,0x19,0x73,0xb0,0x56,0xe5, -0x72,0xc2,0x91,0xc4,0x43,0xd3,0x41,0xd0,0xd4,0x8f,0x64,0x7f, -0x3c,0x34,0x56,0x44,0x06,0x32,0x26,0x66,0x1b,0x59,0xb6,0x7d, -0x69,0xf1,0x53,0x72,0x78,0x87,0xcd,0x6a,0x33,0x26,0x09,0x0e, -0xb8,0x4c,0x02,0x77,0x6d,0x6a,0x4f,0xb1,0xb1,0x82,0xba,0x64, -0x39,0x1c,0x45,0x7c,0x46,0xa3,0xd8,0x6f,0x46,0xae,0x14,0xa5, -0x59,0x0b,0x0f,0x12,0xc3,0x0a,0x2a,0x69,0xf1,0x71,0xe6,0x33, -0x25,0x62,0x37,0x39,0x11,0x75,0x72,0x8b,0x3f,0x5b,0x5b,0xf8, -0x43,0xfc,0x38,0x59,0xa1,0x8a,0xbd,0x6a,0x50,0x72,0x6e,0xa8, -0xa5,0x7b,0x2c,0x7f,0x71,0x1b,0xcc,0x6f,0xe0,0xf9,0x5f,0x74, -0x30,0x07,0x67,0x46,0x9a,0x37,0x2a,0xd0,0xce,0xfc,0xc6,0x14, -0xa1,0xf4,0xba,0xd5,0x9f,0x0e,0xd1,0x68,0x4d,0x7f,0x04,0x6c, -0xf5,0xa0,0xb9,0xf1,0xae,0x18,0x6e,0xdc,0x37,0x3b,0xa5,0x5c, -0xc1,0x36,0x66,0x93,0x0d,0xf2,0x64,0xf2,0x79,0x63,0x3e,0xce, -0xf5,0x61,0x12,0x96,0xe7,0xa2,0xb9,0xf1,0x96,0x71,0xa5,0x1b, -0x7a,0x65,0x53,0x1b,0xb6,0x28,0x48,0x66,0xc5,0x42,0xb1,0x0e, -0xba,0x26,0x69,0x57,0xb7,0xee,0xad,0xe6,0x08,0x2c,0xd0,0xb2, -0xb6,0x83,0x36,0x5b,0x4b,0x31,0x15,0x65,0xbb,0x24,0xab,0xc6, -0x0d,0x4d,0xc7,0xb8,0x62,0xcc,0xe0,0xc8,0x0c,0x24,0x45,0xd4, -0xc2,0xbc,0x7a,0x15,0xee,0x34,0xe0,0xc7,0x57,0x98,0x0e,0x36, -0x0b,0x9e,0x63,0x3b,0xd9,0x41,0x12,0x77,0x69,0x20,0xc5,0x4a, -0x70,0x59,0xe1,0x89,0xa8,0x56,0xcd,0xd6,0xea,0x59,0x7a,0xfb, -0x6f,0x99,0xa6,0xd4,0x48,0x95,0xbd,0xa4,0x82,0xa1,0xc9,0xe2, -0x5c,0x12,0x06,0x89,0x3a,0xda,0x0d,0x25,0x93,0x1b,0x42,0x7b, -0x30,0xd9,0xb0,0x09,0xe6,0xc3,0xb3,0x00,0xfa,0xf1,0x0a,0x64, -0x3c,0x26,0x09,0x94,0x1d,0xe8,0x4b,0x36,0x68,0xd2,0x0e,0x88, -0xc8,0x2c,0x2e,0x14,0xd7,0x50,0x47,0xa2,0x29,0xb4,0xa0,0xbf, -0x50,0xa6,0xc4,0xf6,0x24,0x25,0x5d,0xfa,0x34,0xe3,0xed,0x5b, -0x9c,0x6d,0x8c,0x26,0x26,0xc9,0xb0,0x77,0x9d,0xd0,0xbb,0xed, -0x12,0xea,0xf8,0xad,0x90,0x89,0x13,0x39,0xaa,0x58,0x38,0x29, -0x9d,0x97,0xcc,0x99,0x8c,0x47,0x2b,0x31,0x19,0xf8,0x0f,0x29, -0xa7,0xc1,0xe0,0xa6,0x68,0xb0,0xcd,0xee,0x36,0x8e,0xed,0x96, -0x07,0x76,0x9d,0x0f,0xc0,0x64,0xff,0x66,0x5c,0xc2,0xbb,0x58, -0x84,0xe8,0x7c,0xa2,0x26,0x83,0x41,0x90,0x0d,0xee,0x6e,0x80, -0x1a,0x76,0x33,0x9d,0x84,0xa3,0x64,0xb1,0xa0,0x53,0xd0,0xaa, -0x6b,0x81,0xca,0x3b,0x80,0x75,0x3b,0x8c,0xa0,0xc5,0xf8,0xcb, -0x44,0x49,0xe1,0x68,0x41,0x60,0x60,0x49,0xfc,0xd1,0xe1,0x21, -0xe1,0xbe,0x67,0x58,0xfc,0x2b,0xcc,0x0d,0x7a,0x6b,0x53,0xb0, -0x1a,0xeb,0xa3,0xac,0xc7,0x9e,0x49,0x58,0x27,0x62,0x3b,0xba, -0x9a,0x7c,0xfc,0xc2,0x8b,0xad,0x2f,0x7c,0xf8,0xd9,0xa4,0x42, -0x4a,0xb7,0x49,0x06,0x6d,0x37,0xbc,0xd9,0xe9,0x44,0xe2,0x22, -0x38,0x4a,0x23,0x73,0x25,0x1f,0xf5,0x9c,0x51,0xdb,0xf1,0xe5, -0xf6,0x15,0x98,0x14,0xb0,0xd7,0xe3,0x3f,0xe5,0xc1,0xcd,0xcb, -0xbf,0xa3,0x1e,0x50,0xc8,0x97,0xd4,0xc3,0x52,0x9e,0x7a,0x55, -0x5a,0x55,0x0f,0xf4,0x77,0xf6,0x15,0x0a,0x2d,0x95,0xcd,0x0e, -0xb8,0x3c,0xb9,0x07,0xc6,0x14,0xe0,0xe6,0x9a,0xbe,0x92,0x5b, -0xfb,0x89,0xfa,0x5c,0xf4,0x49,0x4e,0xae,0x4d,0xb4,0xd9,0x72, -0x6c,0x94,0xd7,0x02,0x0a,0xbd,0xa5,0x39,0x61,0xca,0x39,0x38, -0xb6,0xc6,0x72,0x51,0x77,0x41,0x59,0xa4,0xc2,0xb3,0x1b,0x39, -0x6d,0x67,0x0a,0x96,0x20,0x9e,0x41,0xc6,0xe1,0x08,0xad,0x64, -0x7d,0xe9,0x2d,0xab,0x2f,0x09,0x15,0x4a,0x70,0x84,0xce,0x7b, -0xcd,0xe9,0x1e,0xd7,0x84,0x9a,0xdd,0x20,0xcd,0x03,0xa1,0xa4, -0xee,0x83,0x3b,0xb5,0xe6,0x0c,0xea,0xa7,0xe2,0x39,0x27,0x5e, -0x7e,0x0d,0xc9,0xf1,0xd2,0xb1,0x15,0xda,0x14,0xd6,0x31,0xa3, -0x31,0x09,0xad,0xb0,0x83,0xe0,0x61,0x84,0xb9,0x24,0x23,0x5c, -0x7c,0x47,0xa6,0xd9,0x13,0x93,0x03,0x97,0x0c,0x64,0xf8,0xbd, -0xae,0xb5,0xf9,0x4b,0xe3,0x50,0x3a,0x4c,0xca,0xf3,0xcc,0xc7, -0x63,0x8d,0x22,0xe7,0xfb,0x39,0x5a,0x46,0xf6,0x93,0xbb,0x79, -0x4f,0x5e,0x69,0x05,0x4c,0x48,0x58,0x09,0xf4,0xa0,0x14,0x46, -0xc2,0xeb,0x95,0x12,0xed,0x2a,0xae,0x68,0x94,0xac,0xdc,0x80, -0xd4,0xfe,0xdd,0x68,0x93,0x9d,0x71,0x82,0x92,0x6e,0x2d,0x50, -0x27,0x02,0xa3,0x73,0x25,0xf7,0xe9,0x4d,0xba,0xa8,0x70,0x3c, -0x52,0x55,0x7f,0x48,0x63,0x2c,0x41,0xa2,0xdf,0x89,0x15,0x37, -0x56,0x41,0x8b,0xe2,0x71,0xc9,0x8b,0x4b,0x54,0x19,0x6b,0x53, -0xdc,0x45,0x7b,0xc0,0x62,0xdf,0xd1,0xd5,0x7a,0x26,0x6f,0x32, -0xe9,0xa2,0x27,0xab,0x40,0xc6,0xe3,0xb0,0xd4,0xf8,0x44,0x8c, -0x44,0x54,0xba,0xd4,0x72,0x69,0x8a,0xf0,0xbb,0xeb,0x6c,0x0a, -0xa3,0xc6,0x83,0x13,0x80,0x52,0xeb,0x83,0x2c,0xbc,0xee,0x0a, -0x64,0xa5,0x6c,0xf2,0x9b,0xdd,0x5d,0x37,0x19,0x7f,0x91,0xb3, -0x71,0x85,0x8c,0xb2,0x58,0xa9,0xda,0xb0,0x5a,0x0b,0xda,0x8d, -0x20,0x73,0xa5,0xe3,0x6a,0xd0,0xcb,0xee,0x06,0x01,0xf7,0x17, -0xea,0x6e,0x30,0x6a,0x69,0x59,0x79,0xf7,0x69,0x68,0x33,0x8e, -0xfd,0xac,0x85,0x08,0x5e,0xd9,0xe8,0x0d,0xf3,0xff,0xa8,0xe6, -0xd4,0x73,0x2d,0x0b,0xc2,0xa5,0x86,0xd2,0x71,0x5f,0xd5,0x2a, -0x6e,0x5c,0xd7,0xde,0xf1,0xc9,0x95,0xdb,0x61,0xf4,0x1e,0xf8, -0x8b,0x1a,0xd2,0x03,0x10,0x17,0x93,0x89,0xef,0xd2,0xc2,0x5e, -0x29,0xf6,0x2e,0xbc,0x37,0xfa,0xe7,0xc2,0x6c,0x81,0x98,0xc6, -0x25,0x86,0xd1,0x53,0xe7,0x03,0x9b,0x25,0xee,0x8d,0x18,0x19, -0x2d,0x21,0x40,0x52,0xb4,0x2f,0xce,0xa7,0xf9,0x8d,0x7f,0xb1, -0x10,0x28,0x87,0x88,0xca,0xc9,0x84,0xea,0x87,0xb4,0x6b,0xcb, -0xff,0xc9,0xb4,0x08,0xdc,0x75,0xec,0x68,0xa6,0xc3,0x60,0x8e, -0xb2,0xcc,0xbd,0x9a,0x65,0x8f,0x92,0x94,0xf2,0x6e,0x3a,0xa2, -0x24,0x78,0x21,0xa5,0x73,0x65,0x4e,0x84,0x86,0x37,0x26,0x6d, -0x9f,0x3a,0x3d,0x02,0x66,0xb1,0xc8,0xab,0xc8,0xb7,0x20,0x2f, -0x26,0xaf,0xc9,0x67,0x61,0xc5,0xd3,0x07,0x41,0xff,0xd2,0xac, -0x9f,0x5e,0xe3,0x08,0xc5,0x98,0x92,0x32,0x99,0x55,0x76,0x61, -0xad,0x32,0xbe,0x75,0x79,0xfd,0xe6,0xf8,0xc5,0xeb,0x57,0x47, -0xfe,0x0b,0x6d,0xb8,0x1a,0x88,0x3b,0xe9,0x49,0xd1,0xe8,0x17, -0xe6,0xaa,0x20,0xe9,0x50,0xcc,0xe5,0x74,0x32,0xbf,0x71,0x77, -0x05,0x60,0x6a,0x56,0x7f,0xe8,0x7d,0x04,0x82,0x56,0xd2,0x8b, -0x7f,0x4f,0x06,0xbd,0x07,0x68,0xe3,0x87,0x0d,0xcd,0x07,0x29, -0xbe,0xfc,0xd7,0x8b,0x3f,0xc1,0xd7,0xe0,0xe2,0xa2,0x82,0xca, -0x0e,0xee,0x29,0x90,0x2a,0xd0,0x01,0xd7,0x07,0x8e,0x43,0xfb, -0x9a,0xe5,0xd6,0x7b,0x1c,0xc6,0x96,0x35,0x56,0xbd,0xab,0xee, -0x46,0xd4,0x94,0xba,0xd9,0xec,0x63,0x8d,0x5e,0xc0,0x0a,0x89, -0x17,0xc4,0xf6,0xc9,0x31,0x3f,0x2f,0x76,0x4d,0x6c,0xa7,0x07, -0x4f,0x17,0xbe,0x42,0x37,0x06,0x44,0xa8,0x63,0xd5,0x7c,0xfc, -0x61,0x3c,0xf9,0x38,0xe6,0x10,0x18,0x34,0x23,0xa7,0x57,0xec, -0xab,0xd2,0x48,0x5a,0x2f,0x5d,0x8d,0xbc,0xfa,0xc1,0x7e,0x76, -0x81,0x3a,0xb6,0x67,0x93,0xf9,0x78,0x40,0xd7,0x3f,0x85,0x7b, -0x07,0x0a,0xa9,0xf9,0x38,0x17,0xda,0x39,0x02,0xd5,0xa9,0x72, -0x84,0x9c,0xfa,0x94,0x9c,0x26,0xb7,0x5e,0x99,0xd5,0x99,0xb2, -0xda,0xfc,0x99,0x8f,0x6d,0x49,0x94,0x68,0xae,0x9e,0x93,0x43, -0x16,0xcc,0x55,0x97,0x2f,0x22,0xe1,0x0d,0x9e,0x28,0x55,0x3e, -0xa7,0xb2,0xef,0xac,0xec,0x62,0x01,0x8c,0x0c,0x3f,0x69,0xf3, -0xca,0xb5,0x1b,0xf5,0xd6,0x96,0x1f,0x6f,0xcc,0xac,0x27,0x4c, -0x18,0x85,0x6b,0x68,0x33,0xa6,0x78,0x29,0xd9,0x29,0xd1,0xb1, -0x78,0xb2,0x44,0xfa,0x5d,0x74,0xb3,0xaf,0x01,0x78,0x07,0xf6, -0x4e,0x52,0x65,0x9f,0x5b,0x18,0x9a,0x4a,0x10,0x1e,0x69,0x34, -0x94,0xb8,0x5e,0xff,0x00,0x0f,0xcd,0x9d,0x8d,0xa8,0x21,0xae, -0x6a,0x92,0x46,0xb4,0xd1,0x89,0xda,0x7e,0xcd,0xa1,0xa9,0x5c, -0x1e,0xb1,0xe0,0xb2,0xab,0xa8,0xd8,0x03,0x17,0xe9,0x61,0xc6, -0xa0,0xa1,0xc1,0x78,0x4e,0x64,0x98,0x2b,0x54,0xa9,0xe6,0x6a, -0xd8,0x86,0xab,0x7b,0xa6,0xe4,0xda,0x2f,0x14,0xf8,0x99,0xeb, -0x26,0x23,0xad,0xa9,0x7d,0x48,0x56,0xdb,0x5e,0x38,0xdd,0x93, -0x75,0x8e,0x26,0xfc,0x14,0xce,0x3e,0xe8,0x3f,0xca,0xc7,0x97, -0xa3,0x21,0x43,0x52,0xbd,0xac,0x5a,0x93,0xeb,0x5e,0x4e,0xc1, -0xb7,0x6a,0xb7,0x5e,0xb7,0xf3,0x23,0xeb,0xf0,0xa6,0xdf,0x0e, -0x86,0xea,0x31,0xd2,0x76,0xf8,0xe4,0x88,0xb3,0x0f,0x7f,0x3a, -0xf1,0x97,0x8d,0x6f,0x9f,0x72,0x9a,0x27,0x16,0x19,0x7b,0x05, -0x77,0xef,0x86,0x86,0xf0,0x3c,0xac,0xbe,0x43,0x93,0xd4,0xf3, -0x42,0x6b,0xf1,0x78,0x19,0x3b,0xe5,0x12,0xf5,0xbe,0x9e,0x8a, -0x7a,0x55,0x8e,0x7c,0xf7,0x68,0x2b,0x20,0x19,0xfa,0x9c,0x8e, -0x27,0x7e,0x8f,0xd7,0xe8,0x24,0x7f,0x61,0xf3,0x78,0xbd,0x20, -0x37,0x8a,0xb8,0x5c,0x8d,0x84,0x14,0x9b,0xed,0x08,0x39,0x64, -0x5b,0x63,0x95,0x63,0xc4,0xbd,0x0f,0x03,0x49,0x12,0xa0,0x6d, -0x9f,0xaf,0x18,0x17,0x70,0x63,0x59,0x80,0x26,0xde,0x06,0x3b, -0x9b,0x29,0x3d,0x66,0x6b,0xfb,0x67,0x33,0x43,0x18,0xb8,0x58, -0x8a,0x23,0x9a,0x42,0x52,0x31,0xbd,0xc6,0xb3,0x71,0xe6,0x14, -0xea,0xb0,0x48,0xd4,0x60,0x94,0x0e,0x8e,0xb1,0x92,0xb5,0xca, -0x2a,0x7a,0x8b,0xec,0x8f,0x8a,0xd8,0x54,0x49,0x85,0x44,0x2e, -0x16,0x36,0x2d,0x43,0x93,0x38,0x21,0x40,0xe9,0x11,0x6c,0x38, -0x42,0xbf,0xb0,0xbb,0xf1,0xcc,0x07,0xf2,0x51,0xbe,0xde,0xfb, -0xf3,0x2d,0xb4,0xb5,0x85,0xf8,0xca,0xae,0x9f,0xd1,0x4d,0x60, -0x79,0x9c,0x20,0x74,0x30,0x32,0x98,0x67,0x5b,0x31,0x30,0x2c, -0xa8,0x69,0x09,0xac,0x09,0x15,0xf7,0x23,0xb0,0x54,0x68,0x83, -0x4c,0x84,0xf9,0x5b,0x0d,0x3f,0xe1,0x83,0xf4,0xcb,0x37,0xe4, -0xc7,0xc3,0x04,0xb9,0xc4,0x20,0xac,0x90,0x3b,0x7b,0x29,0x6f, -0x17,0x29,0xea,0x05,0x4a,0x9c,0x49,0xee,0x6c,0x2a,0x8e,0x57, -0x76,0x86,0x4a,0xb3,0x34,0x8e,0xfa,0x67,0x9a,0x8b,0x10,0x74, -0x14,0xd9,0x6d,0x14,0xa5,0xe0,0xee,0x30,0x20,0xb6,0xd0,0x1c, -0xa7,0xe6,0x93,0xc4,0x4b,0xea,0xf0,0x69,0x35,0xdd,0x54,0x14, -0xb4,0x0b,0x16,0xcf,0x79,0x15,0x4e,0xf0,0x50,0x04,0xb7,0x51, -0x20,0x3a,0x07,0x4c,0x9d,0x02,0xbd,0xf6,0xc2,0x1d,0x07,0xed, -0x00,0x45,0x42,0xcf,0xbd,0x0a,0x36,0xf5,0x8a,0x1a,0x70,0x46, -0x20,0x30,0x9c,0x65,0x9b,0x2d,0xb8,0x77,0xb6,0xbe,0xc0,0x41, -0xb3,0x16,0x38,0xc3,0xe7,0xe3,0xad,0x2d,0xf3,0xd2,0xd0,0x51, -0x91,0xe0,0x6e,0x61,0xf3,0xca,0x29,0xd7,0xed,0x25,0xd6,0x1f, -0x1a,0x62,0x78,0xc1,0xb7,0xab,0x2e,0xf0,0xd8,0x21,0x32,0x72, -0x21,0x97,0xd8,0x68,0x24,0x23,0xa1,0xb5,0x41,0x2f,0x39,0x5c, -0x86,0xae,0x83,0xec,0x49,0xfa,0x89,0xff,0x27,0x8a,0x79,0xc8, -0x18,0xab,0xdc,0xd5,0x99,0xd9,0x4f,0x69,0x5d,0x35,0x44,0x5d, -0xe5,0xd5,0x14,0xdf,0x3e,0x65,0x1f,0x4a,0x37,0xce,0x15,0x17, -0x52,0xe5,0x7a,0x50,0x0f,0x74,0x4d,0x1f,0x8d,0xac,0xfe,0x0a, -0x25,0x7f,0x70,0x37,0x52,0x29,0x5f,0x33,0x05,0x03,0x24,0xc3, -0x91,0xf7,0xc3,0xe7,0xb0,0x92,0xa8,0xc3,0x1e,0x0b,0xe0,0x4c, -0x0e,0xb7,0x6e,0x80,0x05,0xde,0xc9,0xae,0x0f,0x0c,0x72,0xb4, -0x70,0x9c,0xca,0x99,0x6d,0x63,0x13,0xdd,0xd6,0x66,0xfc,0xd8, -0xb5,0x52,0xf6,0x11,0xef,0x18,0xdc,0x4d,0x9d,0xca,0xee,0x4a, -0xcd,0xd1,0xcc,0x8b,0x4d,0x36,0xdd,0x5b,0x0c,0x7b,0xf1,0xcc, -0xf9,0xe5,0xd3,0x7b,0xb8,0xf2,0xb9,0x5d,0xb6,0x6d,0xa0,0x7a, -0x10,0xd9,0x81,0x93,0xf9,0x24,0xd8,0x6c,0x60,0x54,0xbc,0x92, -0x3a,0x7f,0xd0,0x08,0x94,0x59,0x96,0x55,0x12,0xde,0x96,0x03, -0x4c,0x7c,0x2f,0x16,0x26,0xb0,0xc4,0xe5,0x21,0x1b,0xe4,0x85, -0x3e,0xbb,0x96,0x3a,0x96,0x16,0x8f,0x8c,0xfe,0x06,0xcf,0x37, -0x4b,0xeb,0x0a,0x8b,0x41,0xcc,0xfd,0x71,0x6d,0x13,0x9b,0xf1, -0x57,0x1a,0x6f,0x54,0x1a,0x23,0x36,0x04,0xa6,0x96,0x52,0xd2, -0xc1,0x30,0x7d,0xac,0xc7,0x90,0x3b,0xb9,0x18,0x2f,0x09,0xa2, -0x49,0x3a,0xde,0xa8,0xd2,0xeb,0x9a,0x7d,0x9c,0x0e,0x14,0xea, -0xf1,0x4a,0x85,0xe9,0x0f,0x5b,0xd2,0xa6,0x68,0xa1,0x65,0x92, -0x7a,0xed,0xb5,0x1a,0x5b,0xa7,0x12,0x9d,0x3e,0xd3,0x27,0x62, -0xb1,0xce,0x76,0xe5,0x62,0x9e,0x95,0x06,0x35,0x2f,0x46,0x02, -0x66,0xdf,0x65,0x6e,0xbb,0x4f,0x98,0xdf,0xd0,0x12,0xe1,0xe4, -0x0c,0x11,0x76,0x5d,0x16,0x6f,0x0a,0xad,0x9e,0x6f,0xde,0x4e, -0x41,0xc2,0x0d,0x22,0xec,0x71,0xe6,0xf0,0x75,0x62,0x7f,0x13, -0xcb,0xcc,0x0d,0x48,0xaa,0x73,0x63,0x56,0xf3,0xa6,0xb5,0x89, -0xd2,0x9f,0x38,0x61,0xa3,0xb2,0xd8,0x4a,0x51,0x38,0xa1,0x79, -0x9c,0xa5,0x72,0x30,0x61,0xcc,0x32,0xdc,0x1d,0xb3,0xae,0x5f, -0x70,0x15,0xb9,0xe4,0x18,0xd8,0xcf,0x17,0xf2,0x28,0x2c,0x9a, -0xf2,0x91,0x19,0x01,0x33,0xd8,0xf6,0x09,0xcb,0x2d,0x15,0xe3, -0xb5,0x6e,0xd0,0x1a,0x41,0x66,0xb8,0xe4,0x0c,0x4a,0xc2,0xda, -0x6d,0x86,0x5d,0xa6,0xb9,0xf0,0x4a,0x82,0xfc,0xc6,0xab,0x42, -0x15,0x62,0x4a,0x12,0xc9,0x0b,0xe0,0x19,0x5f,0x05,0xfa,0x0d, -0x5e,0x63,0xaf,0x50,0x6e,0x20,0x38,0x09,0x6e,0x94,0x10,0x82, -0x46,0x2a,0x55,0x9b,0x30,0xb4,0x7a,0xfa,0xc8,0x23,0x89,0xf1, -0xc1,0x4c,0x1d,0x83,0x9e,0x36,0x2d,0xc4,0xe5,0x52,0x70,0x6b, -0x4b,0xe0,0x21,0xcd,0xe5,0xaa,0x5c,0x5e,0x9e,0x24,0x44,0x9c, -0x0f,0x1c,0x56,0xb0,0x1f,0x25,0x51,0xc2,0x19,0xa1,0x04,0xee, -0x80,0xbd,0x46,0x05,0x51,0xa0,0x99,0xeb,0x0e,0xa3,0xe3,0x41, -0x25,0xde,0xf0,0x24,0x16,0x94,0x61,0x57,0xf4,0x00,0x09,0x0c, -0x44,0x3c,0xe0,0x09,0x72,0xdd,0xa2,0xd7,0x19,0x8a,0xde,0xa5, -0xa8,0xd4,0x08,0x3d,0x39,0x47,0x38,0x0b,0x13,0xac,0xa1,0xab, -0x0f,0x0f,0x2c,0x47,0x58,0x48,0x00,0x35,0xf2,0xaa,0xf6,0x2c, -0xaa,0xe6,0x9b,0x5b,0x0f,0x3c,0x2e,0x34,0x95,0x88,0x94,0x5e, -0x38,0x2d,0x25,0x6e,0xd5,0x0c,0x28,0x58,0x7a,0x6d,0xde,0x91, -0x8d,0x8a,0x0e,0xf0,0xac,0xce,0x4f,0x5a,0x37,0x75,0x78,0xa6, -0x9b,0x9b,0x21,0x6f,0x24,0x72,0xa4,0xe3,0xb1,0x1a,0x8d,0x76, -0x58,0x3d,0x6f,0x52,0x24,0xa5,0xca,0xeb,0x8e,0x4d,0x6c,0x61, -0xe6,0x50,0x8a,0x3b,0x55,0xbb,0x6e,0x25,0x95,0xcb,0x7a,0x52, -0xb1,0x7e,0x01,0x4a,0x18,0x5d,0x25,0xac,0xd6,0x56,0x09,0xaa, -0x6a,0x93,0xdd,0x11,0xee,0x69,0x78,0x02,0xc0,0xfa,0xb9,0x25, -0x19,0x70,0x92,0x94,0x2c,0xd6,0x31,0xf9,0x60,0xb3,0x6e,0x5d, -0xad,0x6e,0xea,0x91,0xd3,0x8d,0xb8,0x5f,0x6c,0x9c,0x62,0x0f, -0x9e,0x26,0x1d,0x48,0xdf,0x60,0xc5,0xdb,0xa4,0xc3,0x9a,0xb7, -0x65,0xb5,0x5d,0x5f,0xf9,0x36,0x30,0x7b,0xe7,0x2a,0xe0,0xd9, -0xbc,0xf3,0x16,0xec,0x84,0x1e,0x22,0x5b,0x36,0xc5,0xa8,0x74, -0x57,0x78,0xe4,0x52,0x96,0xe9,0xb2,0xda,0xe1,0x26,0xf5,0x5b, -0x3c,0xa5,0x88,0xa4,0x6f,0x7a,0xca,0xb2,0xb4,0x3e,0x4a,0xe3, -0xc3,0x8b,0xd9,0xaa,0xb0,0xbc,0x8e,0x29,0x2e,0xb5,0xa5,0x31, -0x6c,0x11,0x02,0x7f,0xd7,0x63,0x6d,0x73,0x2e,0xf3,0xef,0xac, -0x80,0x69,0x54,0x5f,0x3b,0x1e,0xf4,0xdf,0x7b,0x09,0x7a,0x8b, -0x2e,0xef,0x6b,0x59,0xb7,0x6b,0x67,0x95,0x7f,0x1e,0x35,0xac, -0xc7,0xb2,0xd7,0x6b,0xbb,0xb5,0x00,0x84,0xab,0xe6,0xb5,0x9e, -0x8b,0x65,0x53,0xa0,0x89,0x90,0x00,0x97,0x2c,0x0f,0x90,0x3c, -0xbc,0x02,0x04,0xec,0x9d,0xf9,0xfb,0xa8,0xbd,0xf4,0xf9,0xd3, -0x22,0x00,0x79,0x50,0x76,0xaa,0xa8,0x14,0x6b,0x2a,0x4a,0x5c, -0xa2,0x38,0xfe,0xca,0x03,0xb1,0xd1,0x96,0x76,0x3a,0x78,0x55, -0xa7,0x54,0x7b,0xfc,0xf4,0xce,0x9b,0xac,0x3e,0x00,0x9c,0x74, -0xb7,0xb7,0x16,0xcb,0xe6,0x4d,0xe8,0xb5,0x78,0xb6,0x80,0x65, -0xbb,0x83,0xa3,0xa0,0xcd,0xba,0xf5,0xa5,0x9b,0xb5,0xae,0x54, -0xec,0xb5,0x59,0xc5,0xbe,0x6d,0xd9,0x1c,0xb9,0x83,0x59,0x1d, -0x46,0x2f,0x37,0x09,0xcd,0xcc,0xa4,0x01,0x05,0xd8,0xf2,0x99, -0x31,0x31,0xc3,0x39,0xb9,0xeb,0xe0,0x2d,0xb4,0xb4,0x89,0x78, -0x87,0x50,0x1c,0x94,0x84,0x1a,0xa4,0x3c,0x6d,0x57,0x9e,0x25, -0x50,0xa9,0x3b,0xf1,0xcf,0xef,0x78,0x40,0xdb,0x58,0x89,0x17, -0xd4,0x2b,0x49,0xf7,0xdf,0xda,0xb3,0xaa,0x38,0xf4,0x2d,0xe9, -0xb8,0xc5,0xed,0x52,0x21,0x4c,0x54,0x9a,0xef,0xcf,0x90,0x02, -0x1a,0x3a,0xfe,0x19,0x11,0x32,0x46,0x5e,0x3a,0x7a,0xab,0x35, -0x49,0xf6,0xf8,0xf8,0xc5,0xad,0xe6,0x57,0xb7,0xf2,0xa4,0xff, -0x25,0x6d,0x48,0x14,0xb2,0x91,0xc2,0xea,0x53,0x24,0xfa,0xa9, -0xe3,0xe6,0xb8,0xb3,0x61,0xef,0x1a,0x2d,0xab,0x0a,0xaa,0x10, -0xce,0x09,0x2b,0x41,0x9e,0xf4,0x95,0x7e,0x7c,0x31,0x27,0xc5, -0x25,0x4b,0x36,0xba,0x06,0xf6,0xee,0x5c,0xfb,0x0a,0x01,0xb7, -0x10,0xf9,0x3b,0x53,0x79,0xf2,0xae,0xcc,0xe4,0xb6,0x12,0x05, -0xba,0x0c,0x36,0x21,0xb1,0x65,0x19,0x45,0x26,0x23,0xbd,0x13, -0x17,0x6b,0x9b,0x2d,0xe5,0x73,0x61,0xc6,0x87,0x7a,0xe0,0xc5, -0x83,0xc7,0x94,0x13,0x70,0x26,0x2a,0x3c,0x45,0xad,0x91,0x13, -0x17,0x0f,0xc8,0xeb,0xdf,0xe1,0x55,0xf3,0x81,0xe0,0x30,0x9e, -0x94,0x49,0x75,0x93,0xd9,0xf7,0x98,0x26,0x02,0xd2,0xf3,0xcc, -0x84,0x4a,0xca,0x53,0x3c,0xae,0x4d,0x3a,0x68,0x4f,0x19,0xcb, -0x3e,0xae,0x89,0x05,0x53,0xea,0x30,0x0d,0x7f,0x8b,0x73,0x33, -0x9a,0x34,0x16,0x71,0xa1,0x00,0xf7,0x9b,0xf2,0x5a,0x53,0xd5, -0x5d,0xaf,0x3a,0x21,0x8f,0x71,0x0b,0x61,0x2c,0xca,0x59,0xe3, -0x6b,0x6f,0x39,0x22,0xa1,0x8d,0xc5,0xa4,0xff,0xe3,0x2e,0x5b, -0x7b,0xa5,0x8b,0x84,0x2a,0xdb,0x78,0x41,0x91,0x7c,0x1d,0xf8, -0xbb,0x90,0x26,0xf2,0x08,0x5f,0xb6,0x3d,0x7c,0x9e,0xc1,0xb5, -0x12,0x49,0xc1,0x2d,0x86,0xb0,0xc1,0x66,0xc4,0x0c,0x15,0x59, -0xd6,0xb1,0xac,0x57,0xca,0x16,0x57,0x5f,0x5a,0xc1,0x04,0x73, -0x83,0xba,0xe3,0x4b,0xbf,0x87,0xe5,0x4b,0xb3,0x24,0x35,0xdd, -0xd0,0x09,0xee,0x50,0x2a,0x81,0xbd,0x9b,0xb8,0x0a,0x08,0x79, -0x07,0x47,0x5a,0x13,0x44,0xcc,0x17,0x5b,0x02,0x33,0x37,0xf3, -0x47,0x6e,0x82,0xdf,0xda,0x1b,0x6b,0x63,0xde,0xdd,0xdd,0x13, -0xaf,0xd0,0x25,0x58,0x6d,0x63,0xae,0xa4,0x5b,0x22,0xaf,0xd5, -0xb0,0x63,0xf8,0x2a,0x66,0x5d,0xbb,0x63,0x78,0xce,0x4a,0xe0, -0x8a,0xae,0xed,0x41,0x7b,0x7e,0x81,0x7e,0x27,0x23,0x70,0x7e, -0x9e,0x64,0xb9,0xae,0x39,0x21,0xd7,0xc8,0xc8,0x02,0x47,0xec, -0x71,0x5e,0xd8,0x84,0x66,0x2b,0xb7,0x76,0x45,0xb0,0x4a,0x72, -0x62,0xc1,0xa4,0x38,0xb2,0x09,0xe6,0x63,0xd4,0x1c,0x16,0x27, -0x5a,0xbf,0x13,0x55,0x50,0xe2,0x82,0x14,0xdc,0x3a,0xd8,0x35, -0xc0,0x6a,0x40,0x74,0x86,0x35,0xc7,0xe6,0xcd,0x96,0xbc,0x85, -0xcc,0xfc,0x8e,0xf6,0x4d,0xad,0x75,0xbc,0x8d,0xc1,0x4b,0x0a, -0xdf,0x42,0x83,0x42,0x30,0x3e,0x89,0x2e,0xb7,0x34,0xbf,0x8b, -0xe6,0x5f,0xd2,0x0c,0xd6,0x1f,0x1e,0xd9,0x72,0x3c,0xc5,0x8c, -0x4a,0xb4,0x97,0x53,0x98,0x70,0xaf,0xac,0xd3,0xca,0x64,0x31, -0x6c,0xee,0xb6,0xbb,0x5a,0x47,0x80,0x40,0x62,0xd4,0xdc,0x0b, -0x5f,0xa9,0xda,0x2a,0x1f,0xd5,0x69,0x52,0x2d,0xf5,0x12,0xad, -0x39,0xc5,0xff,0xfa,0x3f,0xf3,0x21,0x3a,0xaf,0xb8,0x99,0x9f, -0x8d,0xf2,0xe2,0xea,0xd9,0x27,0x0d,0xa3,0x21,0x9e,0xbc,0x79, -0x11,0x6b,0xce,0x7e,0x3e,0xea,0xeb,0x35,0x28,0xd8,0x97,0xe0, -0xc9,0x7f,0x7f,0xe0,0xf1,0xf6,0x0c,0x0a,0x16,0x26,0x2c,0x98, -0x3a,0x9b,0x4c,0x66,0xba,0xf0,0x3e,0x5a,0xe1,0x88,0xc9,0x8b, -0xb9,0xb2,0x48,0x14,0xe7,0x31,0x8c,0xaa,0x40,0x4f,0x37,0x81, -0x57,0xba,0x90,0x28,0xb1,0xea,0x0e,0xf7,0x1f,0x63,0x15,0x37, -0xd0,0xef,0x34,0x29,0x79,0x81,0x0b,0xbf,0xfd,0xff,0x05,0xaf, -0xfd,0x74,0x77,0x6e,0x44,0xef,0x8e,0x9f,0x6f,0xff,0x3d,0x6a, -0x77,0xc7,0x97,0x27,0x27,0x29,0x1a,0x56,0xeb,0xa9,0xae,0xb1, -0xb1,0xa6,0x3f,0x61,0xe8,0x6d,0x8b,0xf0,0x27,0x11,0x68,0x9a, -0x4f,0xd5,0x74,0xd1,0x18,0x0b,0xba,0xeb,0xb7,0x7c,0x24,0xd5, -0x2b,0xc9,0x78,0xb8,0xb1,0x99,0x5f,0x83,0xcf,0xe5,0xfe,0x78, -0xd6,0x5e,0x62,0x59,0xb0,0x92,0x5b,0x10,0x7a,0xb8,0xb6,0x7c, -0xbc,0x83,0xe4,0xee,0xdf,0x17,0x37,0x99,0xdc,0x04,0x5c,0x71, -0x61,0x1e,0xee,0x36,0x9a,0x83,0xc8,0x1f,0x3b,0x2a,0x22,0xf7, -0xb9,0xc6,0x03,0x21,0xbf,0x9f,0x37,0x8d,0x75,0xef,0x9e,0xf5, -0x72,0x68,0x20,0xa2,0xa4,0x6d,0x52,0xaf,0x27,0x83,0xf9,0x68, -0x48,0xaf,0xec,0x39,0x51,0x81,0x35,0xe2,0xa8,0xa7,0x1b,0x95, -0x1d,0x90,0x88,0x8c,0x4a,0x58,0x6f,0xb2,0x2f,0xff,0x7f,0x2b, -0x5b,0x63,0xe2,0x97,0xff,0xbf,0x89,0xeb,0xd0,0xe1,0x18,0x7d, -0x68,0x4f,0xf3,0xa3,0xe1,0x25,0xf6,0x1e,0xea,0xb9,0xf0,0x1a, -0x65,0x13,0x69,0x70,0x81,0x0a,0xbd,0x5e,0xf2,0xac,0x59,0x86, -0x9d,0xbf,0xed,0x7d,0xbb,0x73,0x99,0x83,0x3b,0x4b,0x19,0xf9, -0xf0,0x00,0x23,0x33,0x2f,0x72,0xef,0x07,0x8c,0x6c,0x44,0xe2, -0xd6,0xb6,0x02,0xfd,0x8d,0x66,0xda,0x31,0xf6,0xe8,0x46,0xe7, -0x2b,0x42,0x4a,0xa4,0xab,0x6f,0x20,0x54,0x14,0xf0,0x68,0x17, -0x0b,0xf8,0x5f,0x3e,0x29,0x4f,0x30,0x32,0xf5,0x49,0x79,0xb4, -0x73,0xa9,0xdb,0xc7,0x8f,0x7b,0x8a,0x80,0xca,0x8f,0xd4,0x28, -0x43,0x8a,0x3a,0x91,0x8e,0xd6,0x08,0xbd,0x8a,0xbc,0x9d,0xcc, -0xf5,0xb4,0x32,0x1a,0x85,0x8a,0x5d,0x44,0x17,0xac,0xc8,0x64, -0xe2,0x33,0x0b,0x80,0xd1,0x06,0x2a,0x33,0x1f,0x68,0x77,0x04, -0x93,0xe6,0xd3,0xd1,0x1b,0xf0,0x01,0x8d,0x5a,0xfe,0xae,0x14, -0xee,0x6e,0xe3,0x81,0x06,0x7c,0xad,0xa3,0xab,0x68,0x56,0xd6, -0x2d,0x14,0xf9,0x67,0x92,0x4a,0x35,0x7a,0x76,0xe9,0x53,0xc4, -0x8c,0xb0,0xc5,0x30,0x71,0x18,0x88,0x62,0x78,0xc3,0x1b,0xe0, -0x11,0xdb,0x18,0x31,0x16,0xe9,0x19,0x7b,0xdb,0xd1,0x84,0x55, -0x95,0xc4,0x48,0xac,0x16,0xbb,0x8c,0x0d,0x74,0xfe,0xc9,0xf2, -0x47,0x5e,0x3c,0xf7,0xb9,0x39,0xcd,0x16,0x8a,0x13,0x06,0x48, -0x2c,0x34,0x31,0x64,0x14,0xc4,0x63,0xd6,0xe8,0x25,0xf0,0xd6, -0x96,0xee,0x5f,0xbe,0x3c,0x13,0x5e,0xce,0xc9,0x97,0x28,0xd5, -0x8e,0xd2,0x8a,0xf9,0x99,0x9e,0x84,0x31,0xd8,0xfd,0x27,0xf6, -0x0e,0xf1,0xba,0xa6,0xe4,0x26,0x7f,0xcb,0x66,0x95,0x5e,0xc0, -0x86,0x03,0xa6,0xd4,0x35,0xc7,0xe5,0x0c,0x46,0x71,0x7b,0x98, -0x88,0xa6,0x99,0x6e,0xe5,0x1e,0xb1,0x46,0xe7,0x40,0x03,0x03, -0xbf,0x6d,0x5f,0x92,0x18,0x70,0x0a,0x03,0x04,0xef,0x8c,0x69, -0xa8,0x48,0xcf,0xf6,0xdc,0x69,0xa2,0xa9,0x0f,0x9e,0x3d,0x7f, -0xf2,0xee,0xe5,0xf1,0xe9,0x93,0xa7,0x28,0x17,0xb4,0xdd,0x1a, -0xb4,0xb3,0x68,0x62,0x8a,0x50,0xce,0x38,0xe2,0x55,0xbf,0xf8, -0x61,0x32,0xf8,0x0c,0x36,0x7b,0xdf,0xbc,0x3e,0x3a,0x5e,0xbc, -0x79,0xa7,0xff,0x81,0x67,0xfd,0xe4,0xde,0x4e,0xce,0x06,0x63, -0xe8,0x94,0x7a,0x3d,0x9c,0x5d,0x4d,0x06,0x49,0xdb,0xd4,0xa5, -0x3b,0xc6,0xc3,0x9f,0x43,0xdd,0x52,0xfd,0x3d,0xd5,0x7f,0xa8, -0xfa,0x8f,0xd8,0xee,0x22,0x34,0x33,0xdb,0xd9,0x52,0x68,0x90, -0x89,0xc6,0x06,0x8e,0xa1,0xd5,0xde,0xe3,0x1f,0xa5,0xe4,0x9a, -0xa2,0xff,0xc8,0x20,0xc8,0xfa,0x0f,0xc9,0x79,0xfc,0xc3,0x14, -0x7f,0xf6,0x52,0x72,0x03,0x6f,0x87,0x49,0x7f,0x2f,0x41,0xf7, -0x53,0x80,0x5f,0x93,0x02,0x85,0x67,0x9a,0x1e,0x91,0xdd,0x1e, -0x50,0x64,0xae,0x16,0x68,0x1d,0x18,0x98,0x96,0xe2,0x62,0xf7, -0x18,0xd8,0xa6,0xec,0x99,0x94,0x87,0xd6,0x85,0xbd,0x87,0xa4, -0xe3,0x70,0xa4,0xdc,0xa4,0x1d,0x22,0xa1,0x95,0x5a,0xa2,0x08, -0x69,0x9b,0xfd,0xe3,0xfb,0x5e,0xe2,0xd9,0x7f,0xbd,0xbf,0x09, -0xc4,0xd1,0x59,0x7f,0xa0,0x9b,0xa8,0x90,0x82,0xdc,0xf9,0x0d, -0xbc,0x19,0x7e,0x64,0xbd,0xd8,0x17,0x1b,0x5d,0x2a,0x41,0x51, -0x8b,0x6f,0x98,0x26,0xdf,0x40,0x92,0x7b,0x0a,0x45,0xb4,0xb7, -0xbb,0x4b,0x97,0x23,0x52,0xa5,0x76,0xc7,0xd1,0x97,0x17,0x2f, -0xc6,0x85,0xde,0xe6,0xce,0x87,0x4f,0xfb,0xa3,0x51,0x06,0xe8, -0x34,0x93,0x4b,0x31,0x93,0x0b,0x3b,0x8a,0xf9,0xac,0xef,0x43, -0x63,0x75,0x53,0x1e,0x2b,0xac,0xdc,0xd1,0xe9,0xf6,0x52,0x32, -0xa8,0xcd,0xc3,0x1f,0x60,0x12,0x05,0x7b,0x8c,0xe6,0x4e,0x2f, -0xf2,0x4b,0xd0,0x17,0x99,0x96,0xe7,0x56,0x66,0xd0,0xb8,0xa8, -0xad,0xad,0x72,0x5c,0xd3,0x64,0x5d,0x2c,0xea,0xe7,0xa9,0xc5, -0x8f,0x2e,0x32,0xfe,0x44,0x21,0x98,0x7f,0xb1,0x70,0xd2,0x20, -0x7f,0x92,0x55,0x2e,0x63,0x11,0xf5,0x0c,0x59,0x29,0xd0,0x8b, -0x12,0xb7,0x8b,0x0b,0xbb,0x72,0x4c,0x5c,0xec,0x5a,0x87,0x96, -0x37,0xb7,0x56,0xa0,0x52,0x96,0x4b,0x46,0xce,0x1a,0x3b,0x49, -0xe1,0xb2,0xd1,0xd4,0x3c,0xd1,0x3b,0xb3,0x21,0x08,0x34,0xca, -0x2d,0x18,0xb5,0xab,0x7c,0x93,0x08,0x85,0xd5,0x28,0xe1,0x59, -0x9a,0x98,0x24,0x5c,0x83,0xc8,0x93,0xdf,0xe4,0x3a,0x2f,0x86, -0x19,0x32,0x09,0xa2,0x80,0xa4,0x39,0x03,0x13,0x55,0x15,0xa6, -0xf4,0xcc,0x32,0x90,0xd9,0xb5,0x11,0xcb,0x34,0x88,0x68,0x0d, -0xbe,0xc7,0x41,0x10,0xdd,0xe2,0x00,0x81,0x83,0x97,0x61,0x70, -0x8c,0x9a,0x10,0x26,0x6c,0x66,0x9b,0x9b,0xfe,0x08,0x4b,0xea, -0xe7,0xcd,0xf9,0xc5,0x25,0x4c,0x1b,0xe8,0x35,0xb8,0xb9,0x30, -0x10,0x1b,0xe7,0x48,0xf3,0x7c,0x8a,0x7e,0x53,0x9a,0x1b,0x76, -0x5e,0x19,0x12,0x61,0x76,0x81,0x4e,0x79,0x1f,0x9e,0xf4,0x8f, -0x71,0xe2,0x98,0x7b,0x0e,0x08,0xc2,0xd5,0x46,0x30,0xca,0x23, -0xd4,0x34,0xd3,0x7b,0xfd,0xe4,0xec,0x57,0x54,0x4f,0xaf,0x24, -0xbe,0x04,0xa6,0xb9,0x38,0x1f,0x4f,0x2c,0xe5,0x0d,0xd9,0xae, -0x5d,0xc5,0xb1,0xcd,0x6c,0xeb,0x82,0x76,0x20,0xcb,0xf5,0x48, -0xec,0xe0,0xcd,0x30,0x4c,0x85,0x37,0x4f,0x29,0x79,0xbe,0x73, -0x7b,0x1d,0x0b,0x30,0x6c,0x73,0x67,0xfc,0x6b,0x0f,0x9a,0x9c, -0x0c,0x0d,0x35,0xfa,0x1d,0x9d,0x19,0xa8,0x98,0xd7,0x92,0xc5, -0x02,0x4e,0xd4,0x09,0x0f,0x6d,0xdb,0x99,0xc0,0xb0,0x0f,0xa7, -0x45,0xa2,0x4a,0x5b,0x5f,0xe6,0x43,0x0a,0x43,0x55,0xa5,0x6d, -0xb3,0xaa,0xd8,0x21,0xcd,0x35,0x2a,0x54,0xd8,0x34,0xfc,0x4d, -0x17,0x00,0x6d,0xe7,0xe2,0x96,0x3c,0xf8,0x2a,0x56,0x91,0xda, -0x99,0x2f,0xac,0x75,0x79,0x2b,0x18,0x37,0x08,0x1b,0xed,0x2a, -0x35,0x94,0x2a,0x51,0xda,0x32,0x86,0xdb,0x95,0x69,0x7f,0x38, -0xdb,0xce,0x26,0x70,0x62,0xd1,0x9c,0x36,0xd9,0x14,0x16,0x5b, -0x24,0xaf,0xd3,0xde,0x9e,0xe8,0xf1,0x3a,0x3c,0xf3,0xf4,0x2a, -0x40,0xdb,0x8d,0x81,0x34,0xbb,0x0b,0x23,0xb0,0xdb,0x28,0xcd, -0x4b,0x4d,0x90,0x5e,0xfb,0x32,0x7f,0x63,0x36,0x85,0xe1,0x93, -0x05,0xbf,0xc4,0xb6,0x63,0x58,0x74,0x3e,0x5b,0xcf,0xc5,0x82, -0x22,0xe0,0x41,0x8f,0xad,0x0f,0x1c,0x19,0xc5,0x1e,0x3f,0x18, -0xe4,0xf0,0xd1,0xa7,0x55,0xe6,0xc0,0xf2,0xaf,0x8c,0xb1,0x8a, -0xf3,0xac,0xe5,0x0c,0x6b,0x70,0x59,0x2e,0xc6,0xb5,0xea,0x92, -0xdc,0xa3,0x7b,0xcc,0x4e,0x76,0xab,0xb9,0xb9,0xf4,0x96,0xb8, -0x92,0x34,0xfa,0xf1,0xd9,0x71,0xb4,0x54,0x45,0xff,0xf7,0xa1, -0x8b,0x03,0x86,0x46,0x47,0xfe,0x06,0x87,0x06,0x1f,0x52,0xf1, -0x94,0x83,0x63,0x31,0x4b,0x8e,0x1c,0xc0,0xc1,0xb3,0x97,0xcf, -0x8e,0x9f,0xe9,0x8c,0x11,0x3d,0x23,0x88,0xca,0x49,0x4b,0x05, -0x43,0xd3,0x9e,0xd1,0x50,0x00,0xc5,0xb3,0xd5,0x46,0x72,0x98, -0xeb,0x6f,0xa3,0x29,0xa8,0x60,0x72,0xda,0x38,0x08,0x28,0x37, -0x0c,0x32,0xb7,0x7e,0x98,0x28,0x76,0x83,0xef,0x7a,0x42,0xaf, -0x21,0xba,0x3d,0x81,0x61,0xe2,0xa6,0xe7,0xeb,0x46,0x8c,0x83, -0xd4,0xc4,0xf2,0xe9,0xc8,0x49,0xba,0x81,0x99,0xdd,0xca,0x7d, -0x22,0x15,0x16,0x8c,0x70,0xb3,0xe0,0x61,0x43,0x9d,0xf0,0xce, -0xa8,0x4a,0xc3,0x91,0x8b,0x8e,0x56,0x20,0x08,0x11,0x8f,0x61, -0xe6,0xf0,0xa2,0xca,0xc0,0xa2,0xfe,0xd9,0x85,0x3d,0xc7,0x28, -0x77,0x3c,0xc1,0x78,0x79,0x5a,0xb1,0xbb,0xa7,0x8e,0x64,0xfb, -0xdd,0x3b,0x27,0xef,0x77,0xc4,0xd9,0x19,0x09,0x49,0x6e,0x37, -0x85,0x47,0x8e,0xe8,0x97,0x13,0x70,0x29,0x1e,0x25,0xc4,0x98, -0x12,0xc4,0xd6,0x16,0xfe,0x7a,0xae,0x3b,0xa2,0xf8,0x97,0x45, -0x57,0x03,0x9f,0x9c,0xf4,0x12,0x7d,0x04,0x43,0x80,0x46,0x14, -0x9f,0x9c,0xbc,0x5f,0xdc,0x4b,0x4c,0x76,0xd8,0xd7,0xf4,0x4c, -0xb3,0x54,0x11,0x1b,0x05,0xcf,0x5b,0x60,0x9f,0x85,0x8a,0xcd, -0x51,0x0c,0xc3,0xa7,0x3c,0x7d,0xec,0x87,0xe3,0xa0,0x3e,0x22, -0x9a,0xf9,0xe7,0xf6,0x4c,0xf9,0x3c,0x57,0xd4,0xd3,0x55,0xe6, -0xd4,0xb6,0x05,0x2e,0xdd,0x9c,0x35,0x7c,0x55,0x61,0x61,0x3a, -0x94,0xde,0x35,0x11,0xbd,0x94,0x1e,0x13,0xf1,0x1c,0x71,0xf1, -0x62,0xab,0x31,0xda,0xe3,0x70,0xdc,0x25,0x7d,0x12,0xf2,0x7b, -0xd4,0x89,0x5d,0xc7,0x65,0x95,0xa7,0xf8,0x52,0x5d,0x65,0x4b, -0xea,0xe6,0x33,0xa5,0xb9,0x16,0x54,0xd1,0x25,0x78,0x31,0xb7, -0x78,0x1b,0xd1,0xbd,0x16,0x18,0x40,0x5d,0x85,0x28,0xde,0xe9, -0x24,0x2b,0x90,0xd9,0xa6,0xc2,0x0b,0x73,0x35,0xd2,0x5b,0x4b, -0x3e,0xbe,0x3c,0x1a,0xf5,0x8b,0xab,0xa1,0x5e,0xc5,0xfa,0xb9, -0x95,0x36,0x44,0x3b,0xfa,0x68,0x07,0x11,0xe2,0x64,0x07,0xc1, -0xd4,0xcf,0xd3,0x80,0xb8,0x65,0x75,0x57,0xee,0x34,0xee,0xed, -0xa0,0x23,0xf6,0x8a,0xa4,0x93,0x66,0xdc,0xc9,0x4e,0x3e,0x36, -0xe2,0x7b,0x8b,0x93,0x4e,0x92,0x68,0xb8,0xa6,0x06,0xa4,0xd9, -0xd1,0xac,0x82,0x3f,0x39,0x69,0x6a,0xa0,0x1d,0x80,0x32,0xc3, -0xe0,0xa6,0xfe,0x64,0xeb,0x8f,0x10,0x64,0xf1,0x16,0x0b,0x9e, -0x7c,0xcc,0x85,0x65,0x5e,0x08,0xc7,0x97,0x17,0x23,0x8e,0xbd, -0x28,0x0d,0x54,0xc1,0xb2,0xbb,0xec,0x39,0xe9,0x1b,0xfd,0x34, -0xad,0x64,0xaf,0x46,0x34,0x15,0x08,0xa4,0xec,0xc1,0x95,0x9d, -0x0d,0xc6,0x37,0x46,0xa9,0x5d,0xf3,0x8f,0x4e,0xd6,0x42,0x2b, -0x3b,0x3c,0x24,0xe7,0x4f,0x21,0x20,0x5c,0xaa,0x5b,0xbb,0xe4, -0xa4,0x94,0x1d,0xf4,0x26,0x09,0x83,0x3b,0x19,0xdf,0xc3,0xd5, -0x49,0xcf,0x80,0xdf,0x73,0xd2,0xb6,0xb7,0x29,0xb0,0x92,0xf1, -0xd8,0x81,0x4f,0xb8,0x3a,0x30,0x62,0x08,0xbd,0xa7,0x8f,0x8b, -0x1c,0xc5,0x9e,0x10,0x8b,0x6f,0x76,0x5f,0xb8,0xc8,0x43,0x1c, -0x41,0x9a,0xc8,0xec,0x76,0x32,0xcd,0x2f,0x73,0xdc,0x62,0x66, -0x57,0x29,0xa2,0x99,0x0e,0x2f,0x87,0x9f,0x6e,0xf0,0x7b,0x49, -0xca,0xed,0x1a,0xb0,0xf9,0x81,0xed,0x90,0x71,0xcd,0x20,0x35, -0x83,0x3f,0xae,0xa3,0xe3,0x6e,0x9c,0x34,0x7b,0x09,0x2c,0x01, -0x27,0x27,0x30,0xda,0x45,0xd2,0xc9,0x4e,0xd2,0x49,0xc1,0x83, -0x6f,0x12,0x77,0x4f,0x3a,0x8b,0x93,0x07,0xbd,0xa4,0xa3,0x01, -0xc5,0xec,0x2f,0x60,0x4c,0xa2,0x02,0x84,0x79,0xdc,0xe0,0x94, -0x23,0xf0,0x7d,0x98,0x53,0x07,0x63,0x0d,0x57,0x52,0x17,0x01, -0xcd,0x99,0x2c,0x7a,0x50,0x9d,0xda,0x76,0xf6,0xfc,0xf8,0x06, -0x8a,0xdc,0x7f,0xbb,0x62,0x40,0xf0,0xbb,0x69,0x3e,0xe1,0x45, -0x07,0x90,0x91,0xe1,0x5f,0x54,0xee,0x8e,0xa2,0x46,0x6c,0x92, -0x41,0xf9,0x17,0x53,0x12,0x3d,0x35,0x3b,0xa9,0x4c,0xc1,0xe8, -0x54,0xcf,0x97,0x46,0x0c,0x04,0xe9,0x83,0x0b,0xd8,0x9e,0xec, -0x44,0x1a,0x47,0xdc,0xfd,0x65,0x07,0x1c,0x14,0xeb,0xb4,0x28, -0x11,0x79,0xd8,0xcd,0x87,0x6c,0xbf,0x93,0x9d,0x7b,0xd0,0x32, -0xae,0x05,0xa1,0x8b,0x9a,0xd4,0x1f,0x99,0xb7,0xca,0xc3,0x8a, -0x3d,0xbb,0xd2,0x8b,0x0a,0x38,0x5a,0xb1,0xdd,0xda,0x89,0xf2, -0x08,0x88,0x80,0x6c,0x4e,0x58,0x82,0x9b,0x09,0x0a,0xc3,0xc0, -0x64,0xaa,0xe4,0xb5,0xa0,0xb3,0x01,0x02,0xcf,0x13,0x04,0xdb, -0x85,0xd8,0x9e,0x95,0xa0,0x4c,0x87,0x60,0x35,0xfa,0xf5,0xf8, -0x68,0xd8,0x9f,0x9e,0x5f,0x11,0x33,0x00,0x70,0xb8,0xb3,0xc2, -0xbe,0xe2,0x8f,0x40,0xc2,0x96,0xf0,0xbe,0x4b,0x9c,0x17,0x39, -0x6f,0x80,0xe1,0x95,0xe1,0xba,0x04,0x69,0x58,0x8a,0x33,0x68, -0xd3,0xc1,0x20,0x4b,0x9a,0x76,0x95,0x97,0x98,0xa4,0x54,0xd3, -0x9d,0xa8,0xcd,0x14,0x4a,0x94,0x92,0x52,0x8a,0x3d,0x9e,0xf0, -0xc8,0x15,0xb4,0xc9,0x1c,0x86,0x46,0xc3,0xd3,0x43,0xc3,0xb0, -0xac,0x70,0xa2,0xf9,0xe4,0xe9,0xc7,0x5c,0xba,0xad,0x62,0x76, -0xf3,0x56,0x00,0x63,0x2b,0xa2,0x62,0x91,0x3d,0x06,0x0a,0x1c, -0xf7,0x34,0x1b,0x02,0x77,0x76,0xd3,0xc9,0x84,0x6c,0x74,0x83, -0xfc,0x18,0x6e,0xf9,0xf1,0x86,0x5b,0x41,0x02,0xcc,0x66,0x3a, -0xf4,0x92,0x68,0x39,0x62,0xe7,0xf8,0x13,0xf4,0xf4,0x5b,0xeb, -0x3a,0x20,0x70,0x0a,0x60,0x0b,0x50,0x16,0xbd,0x92,0xc8,0xf5, -0x51,0x40,0x59,0xc4,0xaa,0xda,0x87,0x00,0xfa,0x08,0xb0,0xcb, -0x09,0x09,0x9c,0x70,0xb5,0xc1,0xf5,0x41,0x2f,0x36,0x1a,0x27, -0x0f,0x10,0xe8,0x4b,0x5a,0x0b,0x90,0x57,0x82,0x4f,0x21,0xab, -0x02,0x49,0x13,0x25,0xd0,0x60,0x4d,0xcc,0xcd,0x32,0xcc,0x41, -0x54,0xde,0xcb,0x64,0x32,0xb9,0x19,0x63,0x73,0xa7,0x9b,0xd7, -0x1e,0xb4,0xb3,0x67,0x82,0x3a,0x40,0xd9,0xb5,0x50,0xfb,0xd9, -0xcf,0xdb,0x8d,0x46,0x6e,0x89,0xe1,0x87,0x1b,0x60,0x11,0x09, -0x38,0x86,0x08,0xd6,0xe6,0xf1,0x25,0x6c,0x7d,0x7a,0x45,0x9d, -0x5c,0x6c,0x5c,0x77,0xf3,0x5e,0x67,0x30,0x2c,0x89,0xbb,0x21, -0x3e,0x49,0xe1,0x6f,0x1b,0x05,0x0a,0xbf,0xc3,0xc5,0x56,0xec, -0x76,0x8e,0xe6,0xd8,0xbc,0x1a,0xb1,0xa3,0x84,0x12,0xdd,0xa2, -0x3c,0xbf,0xd1,0xc7,0xc6,0x21,0x49,0x20,0x8d,0xb9,0xfb,0x4f, -0x33,0x32,0x83,0xcc,0xb1,0xa8,0x70,0x99,0x51,0x97,0x34,0xd9, -0x6a,0x3b,0x9a,0x9e,0x23,0x0d,0x59,0xb2,0x2d,0xd6,0xbc,0x47, -0x00,0x7a,0xf5,0x82,0x48,0x13,0xdc,0xda,0xe2,0xd7,0x6a,0x08, -0x03,0x23,0x99,0x3b,0x15,0x81,0x5c,0x58,0xb3,0x31,0x9b,0x08, -0xe2,0xcf,0x53,0x1d,0xab,0x9b,0xf1,0x7c,0xf8,0x16,0x63,0x49, -0x09,0xd9,0xec,0x9c,0x8c,0x11,0xb1,0xe1,0xf9,0x57,0x24,0x7a, -0x03,0x28,0x51,0x6e,0x8c,0x35,0xc1,0x4b,0x76,0x7f,0xa0,0x77, -0x11,0xb0,0xef,0x8e,0x40,0xef,0xb0,0x05,0x22,0xa4,0x08,0xce, -0xd2,0x80,0x77,0xb1,0xc0,0x90,0x6e,0x4c,0x51,0x3c,0xba,0x1f, -0x5d,0x85,0x89,0x8c,0xce,0x1f,0x81,0x06,0x64,0x84,0xa6,0xad, -0x08,0xa7,0xf2,0x9b,0x0e,0x09,0x57,0xd4,0x7e,0x5c,0x65,0x33, -0xe1,0xd1,0x3d,0xe0,0x11,0x76,0x7e,0x1c,0x24,0x25,0x1d,0x3b, -0x3b,0xb0,0xd9,0xe4,0x85,0x68,0x08,0xab,0x44,0xd3,0x24,0x49, -0xb3,0xc0,0xa6,0x8c,0x65,0xa4,0x5d,0xab,0x93,0xd4,0xa1,0xd5, -0x9c,0x4a,0x88,0xa9,0xd4,0x6d,0x01,0x11,0x89,0x88,0xe0,0x62, -0x92,0x44,0x60,0xc7,0x73,0x3c,0x2d,0x33,0x60,0x4e,0x21,0x10, -0x1e,0x25,0xc6,0xfa,0xa3,0x6f,0xb9,0x41,0x49,0x33,0x0c,0xac, -0xed,0xe7,0x4e,0x96,0x4c,0x21,0x9e,0xcb,0xed,0xf1,0xd6,0x30, -0x63,0x04,0x5d,0xc9,0x8c,0x39,0x4d,0xa6,0x5e,0x66,0x1b,0x99, -0x18,0xab,0x8e,0x5d,0x5e,0xf0,0x25,0x26,0x45,0xa6,0x2e,0x32, -0x1f,0xff,0x3e,0xf9,0xe0,0xf4,0x98,0x94,0x63,0xc0,0xed,0x35, -0x0e,0x52,0x65,0x42,0x49,0x47,0x9c,0xf3,0x6d,0x24,0x78,0x86, -0x0d,0x6f,0x7d,0xbc,0x4e,0x81,0x97,0xbd,0x21,0x62,0xb0,0x62, -0xe1,0xe1,0x7e,0x47,0x27,0x99,0x8a,0x02,0x38,0x45,0xe6,0x14, -0xdf,0x7e,0x49,0xb2,0x89,0xb3,0xc0,0x81,0x97,0x74,0xce,0x21, -0x71,0x57,0x54,0x9b,0x8b,0x44,0xd4,0x23,0xf4,0xd5,0x70,0x5c, -0x5d,0xba,0x2d,0x2f,0x73,0x2e,0x3b,0x64,0x72,0x9d,0xeb,0x8e, -0x3a,0x61,0x63,0x78,0x0d,0x03,0xa2,0x2f,0xcd,0x72,0x56,0xd1, -0x98,0x58,0x2d,0xb6,0xe6,0xbd,0x52,0xfb,0xe3,0xf8,0xec,0x8f, -0x46,0xb1,0x79,0x9e,0x55,0x2a,0x93,0x13,0x6e,0x71,0x55,0x0c, -0x16,0x41,0xae,0xba,0x6b,0x01,0x1c,0xaa,0x3c,0x06,0x71,0x49, -0x92,0x6b,0x94,0xb7,0x24,0xdd,0xb1,0x26,0xf1,0x4a,0x42,0xa2, -0x1c,0xb9,0x96,0x48,0x3b,0xf0,0x2c,0x53,0xaa,0xa6,0x6c,0x0d, -0xf4,0x28,0x2b,0x13,0xc8,0x59,0x64,0x04,0x46,0x94,0x04,0x67, -0xee,0xf6,0x81,0x5b,0x76,0x53,0x0e,0x95,0x21,0x1d,0xf6,0x60, -0xf6,0x11,0x5b,0x23,0xe4,0x7f,0xb4,0xe3,0x6e,0xb2,0xbf,0x4d, -0xde,0x98,0xb2,0x8a,0xed,0xb9,0xb4,0xac,0x30,0x6f,0xa3,0x33, -0x91,0x8a,0xbc,0x39,0x9c,0x10,0xcf,0xc6,0x97,0x3f,0xa9,0x5b, -0x15,0xca,0xcb,0x90,0xe5,0x69,0xdc,0xea,0xc5,0xd7,0x33,0xc6, -0x20,0xb6,0xdd,0xb2,0xa8,0x30,0x13,0xbd,0x58,0x30,0x7b,0x06, -0xbb,0x38,0xfa,0x1d,0x17,0x45,0x73,0xac,0xa5,0xe0,0x76,0x29, -0xf1,0xdf,0x2e,0xe5,0x03,0x64,0xb9,0x42,0xd3,0x8e,0x6e,0x68, -0xba,0x15,0xa2,0x3c,0x77,0x12,0x31,0xcd,0xc8,0xc0,0xc8,0x56, -0x1b,0x7f,0x67,0xa9,0x3c,0x36,0x17,0x74,0xa0,0x57,0x39,0x2e, -0xa0,0x68,0xbc,0x32,0x61,0x01,0x1f,0x1b,0xf5,0xba,0xa4,0x47, -0x11,0xf8,0xd0,0x91,0xf4,0x86,0x30,0x06,0x9b,0x3b,0xe3,0x80, -0xb1,0x05,0x4e,0x87,0x99,0xe6,0x83,0x64,0x07,0xd5,0x84,0x33, -0x09,0xdb,0x6d,0x01,0x75,0x0e,0xb3,0x38,0x94,0xa2,0x24,0x38, -0x2c,0xf2,0xd0,0x2a,0xf3,0x03,0xf3,0xce,0xa6,0x39,0x44,0x26, -0xf2,0xd4,0x8b,0xd9,0xc8,0xd3,0x68,0x44,0xb7,0x51,0xe5,0x5d, -0x16,0xfa,0xe5,0x96,0x9a,0x3c,0xe5,0x71,0x45,0xac,0x81,0xf4, -0x98,0xe2,0x65,0xdb,0xf5,0xa6,0x14,0x98,0x55,0x7b,0x52,0x7c, -0x1e,0x9f,0xc7,0x82,0xc3,0xd1,0xf3,0xdb,0x34,0xb6,0x84,0x45, -0x15,0x1d,0x33,0x80,0x82,0xa9,0x27,0x73,0x33,0x5d,0xcb,0x5e, -0xe9,0x4c,0x8b,0x35,0x14,0x27,0x5b,0xc7,0x41,0x7b,0x9a,0x7c, -0xa8,0xa3,0xb6,0x14,0xd7,0xf7,0xe3,0x4b,0x50,0x93,0x35,0x22, -0x54,0x42,0x5f,0xeb,0x11,0xcb,0x59,0xbf,0x51,0xab,0xac,0x08, -0x86,0x8e,0xb1,0x7c,0x77,0x0b,0x5f,0xee,0x25,0xab,0xce,0xef, -0x42,0x60,0x7f,0x8a,0x94,0xd4,0x6c,0xcd,0x84,0x7f,0x1c,0xa8, -0xe2,0xff,0x75,0x77,0x54,0x21,0xc7,0xcb,0xcb,0x19,0xaf,0xd9, -0x15,0xab,0xa7,0x08,0xf3,0xca,0xee,0x36,0x31,0x0a,0x6b,0xf6, -0x37,0xd8,0x57,0xe0,0x14,0x60,0xb7,0x96,0x15,0xbe,0xa3,0xd8, -0x88,0xeb,0x3a,0x5e,0xa3,0xa8,0xf1,0x14,0x19,0x4e,0xc5,0xb7, -0x1a,0x62,0xef,0xaa,0xb5,0xf4,0x6a,0xdd,0x3d,0x91,0x95,0x7b, -0x5d,0x5c,0xe6,0xfc,0x3e,0x21,0x2a,0xe1,0xee,0xc9,0xb4,0x57, -0xc8,0xe2,0x83,0x0e,0xb6,0xec,0x0c,0xd3,0x1a,0xc5,0xfa,0x0e, -0xab,0x9a,0xd2,0x1c,0x99,0x69,0xad,0x42,0x62,0x43,0x1c,0xe4, -0x9b,0xde,0x82,0x66,0x62,0xa8,0xc7,0x65,0x4c,0x4e,0xd7,0xbc, -0x94,0xf4,0xa4,0x08,0x46,0x56,0xb7,0x0a,0x06,0x2e,0x5a,0x2d, -0x59,0xdc,0xb4,0xec,0x20,0x55,0xfa,0xe9,0x94,0x5a,0x95,0xe5, -0x0c,0xae,0x01,0xbf,0x20,0xef,0x12,0xba,0x22,0x70,0xaf,0x55, -0xe9,0x07,0x0b,0x5d,0xa2,0x07,0x4e,0xb0,0x02,0xc0,0xdf,0x3d, -0x5f,0x58,0x2a,0x70,0xaf,0xb5,0xe4,0xa7,0xe6,0xa5,0xa1,0xb4, -0xbc,0xcb,0xd7,0x5c,0xad,0x8f,0xbb,0xb6,0x34,0x00,0x54,0xc5, -0xa6,0xd0,0xcc,0x4a,0x94,0x99,0x61,0xa0,0xbc,0x48,0x3d,0x1b, -0x5e,0x80,0xd0,0xa1,0xd0,0x5d,0x94,0x60,0xb0,0xe6,0xfa,0xa4, -0xee,0xb2,0xc5,0xb2,0x7a,0x59,0x49,0x30,0x7e,0xc7,0x05,0x8b, -0x61,0xff,0x45,0x1a,0x45,0xb4,0x03,0xb5,0x45,0xe5,0xdd,0xfa, -0x98,0xc0,0x6c,0xf2,0x5f,0xc5,0x64,0xcc,0x6e,0x35,0x89,0x8d, -0x41,0xd5,0xbc,0xac,0x42,0x53,0x0f,0x52,0x9d,0x9a,0xde,0x8d, -0xd9,0x21,0xb8,0xfd,0x22,0xe5,0xcb,0x44,0xc1,0x27,0xa9,0x40, -0x58,0x82,0x37,0xd2,0x96,0xaa,0x5d,0x07,0x4d,0xd7,0x8b,0xbc, -0x03,0xe7,0x04,0x97,0x76,0x19,0x70,0xdb,0x23,0x76,0x9b,0xc4, -0x0f,0x36,0xf9,0xb8,0x43,0x22,0x9f,0xf9,0xec,0x2e,0x27,0x8f, -0xe7,0x62,0x80,0x3b,0x3b,0xb7,0xbd,0x2f,0x15,0x46,0xd7,0xea, -0x49,0x96,0xdb,0xf2,0xe9,0x64,0xf2,0x21,0x07,0x5b,0xfc,0x65, -0xa5,0xc7,0x73,0x97,0xe4,0xcb,0xab,0x34,0xe7,0xfb,0xb1,0xf0, -0x1d,0xe4,0x0a,0x71,0x13,0xa7,0x4a,0x41,0x34,0xf0,0x32,0xbc, -0x72,0x83,0xed,0x17,0x3a,0x3f,0x32,0x1c,0x13,0xc0,0xc3,0x1b, -0x95,0x42,0xd0,0x57,0xab,0x31,0x9f,0xcf,0xe9,0x89,0x54,0x24, -0x67,0xca,0xe8,0x9e,0x56,0xb3,0xb3,0xa6,0xcc,0x26,0x27,0x20, -0x02,0xd1,0x08,0x1e,0x52,0x86,0xe1,0xb7,0x47,0x1e,0x2a,0x15, -0x0e,0x5d,0xcf,0x8d,0xfb,0x26,0x09,0x61,0x9e,0xca,0x1c,0xb0, -0x6d,0x56,0x16,0xce,0x1a,0x03,0x5c,0x2b,0xb8,0x12,0x4b,0xcb, -0x37,0x4d,0x25,0x74,0x49,0xc7,0xa3,0xa7,0x5c,0x5e,0xca,0xdc, -0x9f,0x07,0x06,0x7b,0x0a,0x17,0x94,0xdc,0xda,0x12,0x5b,0x41, -0x0b,0x67,0x25,0x42,0xab,0xdb,0xc5,0x43,0xad,0xeb,0xec,0x63, -0xb1,0xd5,0x96,0xfd,0x51,0x05,0x92,0x74,0xaa,0x48,0x4d,0xfd, -0xfa,0x55,0x65,0x94,0x6d,0x46,0x2b,0x1d,0x1a,0x2c,0xf7,0x47, -0x0b,0xa3,0x01,0xf5,0x27,0xd1,0x46,0xa8,0x0d,0x35,0x1f,0x3f, -0x03,0x71,0x9f,0xae,0x7f,0xd5,0x15,0xb1,0x25,0x5a,0x2c,0x53, -0x36,0xce,0xb2,0xb1,0xa6,0xa5,0xfa,0x83,0xc1,0x1b,0x3d,0x1d, -0x9f,0x8f,0xc3,0x57,0x99,0xbc,0xba,0xaf,0x68,0xda,0x32,0xd1, -0x9b,0x99,0x9f,0xcb,0xf8,0xcd,0xf0,0x7b,0xc9,0x87,0xa1,0x03, -0x6f,0x10,0x27,0xa7,0x45,0x25,0x80,0xe9,0x4a,0xd3,0x18,0xfe, -0xc1,0x95,0x9e,0x99,0x82,0x8e,0x4b,0x9c,0xb8,0xe6,0xf2,0x19, -0x7d,0x7a,0x1e,0x00,0x13,0x37,0x31,0xd8,0x96,0x15,0x8b,0xc3, -0x91,0x0e,0x91,0x52,0xb4,0x5d,0x2c,0xa4,0x1d,0x4e,0xa2,0xc2, -0x70,0xd3,0xa0,0x6e,0x60,0x53,0xf9,0x6d,0x9d,0x71,0x10,0x67, -0x80,0xe9,0x09,0xa2,0x54,0x6c,0xe9,0xd8,0x1d,0x6a,0x3a,0xb9, -0x86,0x2d,0xc2,0x48,0x92,0x48,0xdd,0x54,0xdd,0xcc,0x7d,0xac, -0xc6,0xbd,0x93,0x87,0x31,0xf3,0x77,0x19,0x6b,0x7d,0x8f,0x35, -0x17,0x7c,0xaa,0x78,0xdc,0x7a,0x08,0xe0,0xc1,0xc0,0x5f,0x76, -0x27,0x78,0xdd,0xff,0x30,0x04,0x6b,0x46,0x85,0xf5,0xbf,0xaf, -0x26,0x67,0xbf,0xc2,0xe0,0x25,0x5b,0x69,0x05,0xf8,0xc5,0xa5, -0x13,0xaa,0x8a,0xf8,0x11,0x7e,0xb6,0xdb,0xce,0x1f,0x93,0x5d, -0x33,0xf3,0xee,0x0c,0x6c,0x50,0x9c,0xfd,0xda,0xc5,0xc8,0x6e, -0xde,0x83,0x7b,0x77,0xd3,0x72,0x3a,0xde,0xb1,0xe6,0xc0,0xdf, -0xbe,0xc1,0x67,0x56,0x31,0x7c,0x2a,0xcd,0x5e,0x0c,0x46,0xfe, -0x02,0x0d,0xa9,0x28,0x61,0x85,0x07,0xe6,0xb3,0xfe,0xa5,0xd2, -0xff,0xd0,0x4c,0x17,0x9c,0x80,0xf4,0x42,0xda,0x9f,0x7e,0x26, -0xb7,0xae,0x14,0x6d,0x9b,0xd3,0xf9,0x3c,0xe4,0x14,0xf6,0xad, -0xf0,0xcc,0xd8,0x04,0xe7,0xe8,0x1e,0xbe,0x81,0x6d,0x17,0xb3, -0xfe,0xf9,0x87,0x26,0x0c,0x5f,0xb0,0x5b,0x97,0x8f,0x35,0xdf, -0x36,0xb4,0xa0,0x32,0xb1,0xd7,0x4e,0x90,0xa6,0x67,0xe3,0x01, -0x50,0x04,0xd6,0x63,0x44,0x6a,0xd2,0x36,0x97,0x64,0x94,0x5e, -0x2a,0x6d,0x6b,0x4b,0x82,0xc3,0x5d,0xf7,0x25,0xbd,0xbc,0x0c, -0x90,0x5a,0xa2,0xb1,0x86,0xd9,0xef,0x93,0x7c,0x50,0xc2,0xb5, -0x58,0x6c,0x6e,0x62,0x32,0x01,0x2d,0x16,0x84,0x1a,0xf7,0x34, -0x93,0x1f,0xb9,0x16,0xb4,0x4e,0x4a,0xda,0xdd,0xc5,0xcc,0x0a, -0x6b,0x9f,0x1c,0x1f,0xbf,0x65,0x73,0x03,0xe1,0x0d,0x3d,0xee, -0x13,0x83,0xc9,0xfc,0x6c,0x34,0xfc,0x3f,0xf3,0xc9,0x0c,0xf5, -0x00,0xc0,0x0a,0x12,0xbe,0x09,0x93,0x31,0xf3,0xf1,0x6f,0x2e, -0x24,0x67,0x4d,0x29,0xb3,0xa6,0x2e,0xcc,0x0d,0x0a,0x97,0xbf, -0xc9,0xa0,0x66,0x39,0x91,0x54,0x5e,0x82,0xe9,0x02,0xe4,0xd9, -0x78,0x96,0xcf,0x60,0xe1,0xb2,0xe2,0x58,0x1e,0x25,0xe4,0x98, -0x62,0x6b,0xcb,0x0b,0x9a,0x8a,0xb3,0x3d,0x69,0x1a,0x1f,0x81, -0x78,0x8b,0x5b,0x59,0x0c,0x26,0x33,0xd6,0x6f,0x26,0x05,0x8c, -0xe6,0x75,0x46,0x13,0x0e,0x1b,0x80,0xe7,0xfe,0xe4,0x4b,0xbf, -0xb6,0x8e,0xda,0xcf,0x76,0xb9,0x9b,0xbb,0x3a,0xa4,0x37,0x2a, -0xce,0x03,0x69,0xdb,0xdb,0x78,0x75,0x44,0x50,0xf4,0xd2,0x35, -0x0f,0x51,0xe4,0xfb,0x99,0x4e,0x6f,0xe7,0x1a,0xd6,0xd4,0x4d, -0xf3,0xc5,0xae,0xa2,0x20,0x02,0x23,0xf4,0x68,0xd7,0x5e,0x64, -0x86,0x7c,0xb4,0x31,0x19,0xef,0x26,0xba,0xbe,0x24,0xe5,0xa2, -0x61,0x0a,0x6f,0xe5,0xf1,0x76,0x07,0xa6,0x1b,0x4e,0x5f,0x37, -0x1c,0xcb,0xa2,0x8b,0x0d,0x2a,0xc5,0x27,0xaf,0xb7,0x6c,0x63, -0x66,0x9c,0x75,0x58,0x00,0xac,0xd0,0xfe,0xfc,0x41,0xaf,0x47, -0x7a,0x0e,0x54,0x4e,0xa0,0x04,0xb2,0x23,0x01,0xd5,0xba,0x24, -0xcd,0x07,0xc9,0xe3,0x13,0xb0,0x3f,0x0c,0xfa,0x17,0xc5,0x83, -0xa8,0x21,0x73,0x37,0xa2,0xee,0x2f,0xfb,0xbd,0x07,0xfb,0x7a, -0xf9,0xc9,0xa5,0x88,0xac,0x3f,0x62,0x0b,0xce,0xf6,0xc2,0x13, -0xa4,0xa4,0x33,0xba,0x3b,0xa0,0x42,0x9e,0xbe,0x3e,0x3c,0x7c, -0xf6,0xea,0xd8,0x0c,0xf9,0xc8,0xbb,0xe8,0x7f,0x7a,0xf0,0xe4, -0xf8,0x89,0x97,0x64,0x87,0x19,0xd6,0xd1,0xb5,0x3e,0x06,0xe3, -0x60,0x6c,0x62,0xd1,0x60,0xda,0x93,0x54,0x6c,0xeb,0x97,0x07, -0xf3,0x96,0x1c,0x84,0x78,0xd8,0x04,0xd8,0x51,0xaf,0x2f,0xe2, -0xe8,0xf1,0xe6,0xf6,0x76,0x94,0x74,0xd8,0x70,0x9c,0x9f,0xb6, -0xbd,0xbd,0xaf,0x09,0xc2,0x20,0x0e,0xae,0xd8,0x12,0x33,0xb9, -0x26,0x11,0x49,0x10,0x81,0xcb,0x29,0xdf,0x4b,0x03,0xb3,0xf8, -0x88,0x3d,0x84,0x81,0x8e,0xb1,0x69,0x7c,0x97,0x4a,0xa6,0x08, -0x1e,0x26,0x8a,0xbb,0x13,0x9e,0x4c,0xfc,0xf0,0xec,0xc7,0x17, -0xaf,0x7e,0x3c,0x7d,0xf6,0xea,0xe0,0xf4,0xf8,0xc9,0x8f,0xcf, -0xb8,0x69,0x48,0xbb,0x0a,0xb2,0x6b,0x4a,0x49,0x52,0x8b,0xb8, -0x48,0xb4,0xc8,0xc0,0xc6,0x86,0x89,0x32,0x72,0xe0,0xaa,0x32, -0xc9,0x76,0xc6,0x6e,0xcf,0x3e,0xc1,0xb5,0x11,0xa6,0x4b,0x7c, -0x6c,0xb2,0x59,0xcb,0x84,0x0a,0x40,0x41,0xa2,0x95,0x26,0x0b, -0x1a,0x8f,0x8e,0x9f,0xbc,0x3d,0xfe,0x0b,0xa9,0x0c,0xf1,0x29, -0x6f,0xc7,0x92,0x94,0xd2,0xa7,0x9e,0x36,0x15,0x1d,0xfc,0x58, -0x6f,0xa6,0xff,0x41,0x77,0x62,0x7a,0xd8,0xee,0x92,0x89,0x88, -0x0e,0xc0,0xa4,0x01,0x45,0xbb,0xdc,0x93,0x38,0x09,0x2d,0x64, -0x14,0x85,0x70,0xec,0x11,0xee,0xcb,0x07,0x31,0xbc,0xa2,0x40, -0xdc,0xc8,0xe2,0x1b,0x25,0xec,0x42,0x9f,0xf8,0x66,0xf9,0x1f, -0x52,0x09,0xdb,0x3c,0x32,0x3b,0xbe,0xd2,0x8c,0x04,0x27,0x4f, -0x37,0x3e,0xf6,0x8b,0x0d,0xbd,0xea,0x9e,0x8d,0x50,0xd9,0x1a, -0x61,0x36,0x66,0x1a,0xe2,0x62,0x02,0xfe,0xd2,0x34,0x65,0x1b, -0xb8,0x0d,0x83,0x81,0x1a,0xac,0x1e,0x28,0x61,0x47,0x38,0x2a, -0x89,0x31,0xc5,0x86,0x59,0xca,0x39,0x94,0xc8,0x27,0x3b,0x15, -0x3b,0x82,0x75,0x97,0x99,0x0f,0x06,0xc3,0xf1,0x9b,0xe9,0x50, -0x37,0xeb,0x78,0x38,0xfd,0xe9,0xf8,0xf0,0x25,0x6b,0xa1,0x5b, -0x45,0x95,0xc7,0xa0,0xa0,0xb2,0x35,0x9a,0xb5,0x61,0x7a,0xfb, -0xe0,0xc7,0x78,0x0b,0xec,0x22,0xa1,0x31,0x58,0x20,0x03,0x1b, -0x53,0xf8,0x60,0xad,0x86,0x86,0xa0,0xbc,0x2d,0x2c,0xaf,0x7f, -0x7d,0xd3,0x16,0x4b,0xcd,0xab,0xd7,0xaf,0x4e,0x9f,0xbc,0x7c, -0xf3,0xd3,0x93,0x57,0xef,0x0e,0x9f,0xbd,0x7d,0xf1,0xb4,0xb4, -0x07,0x7b,0x28,0xa3,0xad,0xff,0x64,0x1f,0x23,0xd8,0x69,0x4f, -0x75,0xd9,0xa8,0x16,0xd7,0x88,0xda,0x91,0xd4,0xc1,0x11,0x55, -0x73,0x91,0xfb,0x18,0x79,0x09,0x91,0x3e,0xd3,0x75,0xc4,0xfd, -0xf5,0x7e,0x9a,0x83,0x24,0xf1,0x6c,0x7e,0xc1,0x5b,0xe0,0xe5, -0x58,0xb3,0xcd,0x68,0x07,0x61,0x3e,0xb3,0xbb,0x1f,0x28,0x0a, -0x03,0x8c,0xd2,0xff,0x90,0xbd,0xb0,0xfe,0x7e,0x71,0xcf,0x75, -0x4c,0x2a,0x6c,0xa9,0x72,0xe3,0xbd,0xd5,0x11,0xd5,0x7b,0x28, -0x78,0x99,0xc7,0xb2,0xca,0x9b,0x85,0x4e,0xc5,0x53,0x1c,0x91, -0x82,0xb0,0xf4,0xbd,0x58,0xb4,0x36,0xa1,0x47,0x7d,0x56,0x08, -0x54,0xec,0x34,0xb1,0x38,0x87,0x80,0x6a,0xc2,0x1e,0x08,0x8f, -0xc8,0x6d,0x7a,0xe5,0x3d,0x31,0x1e,0xe3,0xe0,0x1e,0xa4,0x4c, -0x27,0x5a,0x2a,0x31,0x65,0x3e,0x41,0xce,0x64,0x44,0x4a,0x7d, -0xf3,0x69,0x2e,0xc2,0x7a,0x2d,0xdf,0xd4,0xab,0xf2,0x26,0x75, -0x13,0xad,0x33,0xef,0xde,0xbe,0x30,0x2b,0x8c,0x21,0x70,0x83, -0x09,0x44,0x9b,0x1a,0xf0,0x71,0x3f,0x8b,0xee,0xd3,0x57,0xe5, -0xa0,0x62,0x20,0x0d,0x03,0x5c,0x0f,0x7c,0x63,0xab,0x76,0xa2, -0x9d,0xfd,0x28,0x8d,0xf4,0x8e,0xa0,0x8f,0x09,0x9a,0x17,0xf0, -0x9a,0x7f,0x65,0x93,0xaf,0xd7,0x8c,0x3b,0xb2,0x1d,0x31,0x0a, -0x8a,0x02,0x56,0x29,0xcb,0x4c,0x9f,0xc5,0x76,0xa0,0x68,0x1a, -0x70,0x31,0x11,0xca,0xc6,0xbc,0xcc,0x71,0x61,0x15,0xd5,0x23, -0x88,0x84,0xa5,0x85,0xc1,0xda,0x52,0xf5,0x3a,0xd6,0x40,0x80, -0xab,0x0a,0xf3,0xed,0xd8,0x15,0x5c,0x3c,0xb0,0x1f,0xf5,0xf8, -0x14,0xf7,0x6b,0xa5,0xd3,0x45,0xf5,0xc0,0x4f,0x70,0x58,0xdb, -0xeb,0x29,0x15,0xae,0xe8,0xd9,0xce,0x2f,0x8f,0xc9,0x45,0xc2, -0xc7,0x74,0xbb,0xd7,0x48,0x62,0xf2,0x83,0xc0,0x41,0x0c,0x3c, -0xc8,0x00,0xa0,0x93,0x82,0x6e,0x5d,0xf7,0x97,0xa8,0xf7,0x20, -0x42,0x97,0x0b,0xf7,0xbb,0xbf,0xdc,0xef,0x3d,0xb8,0x9f,0x2c, -0x34,0xcf,0x72,0x52,0xe8,0xac,0x49,0x27,0x79,0x40,0xfe,0x9e, -0x77,0xd0,0xef,0xf3,0xfe,0x8e,0xf2,0x77,0x39,0x2e,0x4b,0x73, -0x3e,0xa2,0x40,0xe2,0x78,0x76,0x94,0x60,0xd8,0xb3,0x1d,0x47, -0x4e,0x48,0x00,0xd0,0x07,0x44,0xe8,0xa2,0x98,0x0c,0x8a,0xb9, -0x0f,0x31,0x9a,0x98,0xd8,0x52,0x83,0x9a,0x8c,0xe1,0xfe,0x09, -0x24,0xec,0xa8,0xea,0xed,0xdf,0x92,0xb7,0xa3,0x7c,0x6e,0x2a, -0xdb,0x01,0xf6,0x05,0xbd,0x41,0x68,0x5e,0x45,0x63,0x95,0xfc, -0x14,0x24,0x9e,0x74,0x31,0xe6,0xa4,0x8b,0x30,0xbd,0x1e,0xc0, -0xb8,0x29,0x02,0x4f,0x14,0xe3,0x8b,0xd9,0xcd,0x02,0x94,0x05, -0x8a,0x4e,0x92,0xea,0x06,0xd8,0x59,0x5c,0xf7,0xf3,0xd1,0x6c, -0x92,0x2e,0x66,0xc3,0x51,0xba,0xf8,0xcf,0x64,0x27,0x57,0x35, -0x4b,0x27,0xb4,0xc6,0x2f,0x27,0xff,0xb9,0xfd,0xff,0x2e,0x36, -0x16,0x9b,0xa8,0x85,0x28,0xcf,0x4b,0x99,0x39,0xd4,0x46,0xfd, -0xe9,0xb0,0xaf,0xce,0xa6,0x60,0x70,0x4a,0x5d,0x4d,0x55,0x7e, -0x7d,0xa9,0x3e,0x9e,0x4d,0xa3,0x44,0xf9,0x87,0xb6,0x1f,0xe4, -0x39,0xd1,0xe5,0xd6,0xb9,0xd0,0x62,0x8b,0x1a,0x0c,0xd4,0x60, -0xa6,0x46,0xfa,0xb8,0xa0,0x66,0x67,0x93,0xc1,0x67,0x35,0x1b, -0xa8,0xd9,0xc5,0x64,0x32,0x53,0xb3,0x2b,0xfd,0xdf,0xb0,0xaf, -0x83,0x65,0xac,0x2f,0xbc,0x33,0xa5,0x43,0x3b,0xbd,0x51,0xd3, -0x59,0x09,0xda,0xc2,0xf9,0xc2,0x71,0xb8,0xfa,0x5e,0x85,0x76, -0x55,0x4d,0x82,0x03,0xf0,0x9d,0x98,0xbd,0xcc,0xca,0x35,0xe2, -0x60,0x00,0xc6,0xe6,0x14,0x98,0x7d,0x3b,0x1f,0xe9,0x83,0x56, -0x01,0x4e,0xe6,0x10,0x35,0x1e,0xe6,0xd4,0x79,0x9f,0x8c,0xc0, -0x9c,0xe3,0x2d,0x15,0x5c,0x0a,0xab,0x41,0xae,0x7f,0xf3,0xdf, -0xd5,0x60,0xa4,0xf0,0x7d,0xd6,0x10,0x7e,0x0c,0x1c,0xb4,0x9c, -0x86,0xbb,0x6a,0xa9,0xab,0x3d,0x75,0xf5,0x50,0x5d,0x3d,0x52, -0x57,0xdf,0xa8,0xab,0x6f,0x15,0x3d,0x41,0x52,0x57,0xd4,0xea, -0xd0,0x61,0x63,0xa0,0xe3,0x46,0x69,0x8a,0xe6,0xfa,0x8c,0xfa, -0xbb,0xd2,0xfd,0x78,0xa3,0x91,0x91,0x8b,0x1a,0x55,0x90,0x49, -0x23,0xbd,0x52,0x69,0xee,0x43,0xcd,0x47,0xb0,0x6a,0xf9,0x27, -0xf9,0x2f,0x6d,0x4d,0x57,0x69,0xd5,0x3f,0xd3,0x03,0xa7,0x7f, -0x3e,0x9d,0x8c,0x3f,0x5f,0xab,0x33,0x75,0x36,0xc8,0xf5,0xbf, -0x89,0x3a,0xcb,0x2f,0x71,0x44,0xe5,0x50,0x71,0x70,0x03,0x8d, -0xd5,0xbd,0x18,0xab,0xe1,0xb5,0xae,0x18,0xdc,0xd9,0xe3,0x30, -0x03,0xca,0x3f,0x9c,0x0d,0xd8,0xb2,0x37,0xd6,0xa1,0x3f,0xfd, -0xa0,0x7e,0x53,0xd3,0xf9,0xd9,0x67,0x85,0x43,0x40,0x15,0xaa, -0xd0,0x9c,0x82,0x2a,0xae,0xe1,0x2c,0x53,0xdc,0xf4,0xc7,0x0a, -0x58,0xb9,0x0f,0x43,0xf8,0x99,0x8c,0x2f,0x95,0xe6,0xed,0xf4, -0x3f,0x3d,0xe6,0x72,0x7d,0xbc,0x9d,0xcd,0xd4,0x5c,0xe9,0x25, -0x0f,0xea,0x18,0x6c,0xa0,0x6e,0x68,0x99,0x76,0x01,0xdb,0x05, -0x64,0x18,0x4f,0xcc,0x8a,0x72,0x53,0xc8,0x49,0xe3,0x8f,0x95, -0xa0,0x19,0x6b,0xc6,0x2a,0xa8,0xdc,0xd3,0xb6,0xe8,0x48,0x38, -0xd3,0x87,0x1f,0xe8,0x3f,0x78,0x16,0x03,0x6d,0x74,0x35,0x1d, -0x5e,0xa8,0x91,0xae,0xce,0x60,0x58,0x9c,0xab,0x62,0x7a,0xae, -0xe6,0xc5,0x50,0xb7,0x87,0x21,0x8f,0xb2,0x97,0x69,0x33,0x98, -0x45,0x97,0x60,0x87,0x8c,0xf4,0x36,0xa3,0xff,0xce,0x54,0xff, -0x53,0xae,0x89,0xbe,0xd4,0x93,0x74,0x32,0x55,0x67,0x13,0x74, -0x12,0x7c,0x3e,0x1c,0x8d,0x6e,0xe0,0x2d,0x92,0x6e,0x3d,0xf8, -0xd6,0x8d,0x7a,0x8e,0xdf,0x60,0x37,0x4f,0xc1,0x4d,0x19,0xae, -0x05,0x13,0xfc,0x5b,0xe0,0x1f,0x68,0x76,0xb8,0xf5,0xe8,0x9f, -0xcf,0x40,0xba,0x38,0x1d,0x14,0x38,0x78,0x2f,0x34,0x3b,0xc5, -0xe3,0xb1,0xd0,0xbf,0xf9,0xe5,0xd5,0x0c,0xab,0x32,0xd2,0x94, -0xaa,0x2b,0xc0,0x3b,0x54,0x79,0x01,0xfd,0x8a,0x31,0xf0,0x67, -0xde,0xbf,0x1c,0xaa,0xf1,0x04,0x2b,0x3c,0x9e,0x7c,0x9c,0xf6, -0xc1,0x8e,0xcd,0x48,0xff,0xfb,0x5d,0x81,0x58,0x15,0xff,0x40, -0x69,0xd3,0x39,0xb8,0x28,0x64,0xa3,0xf5,0x78,0xf3,0x02,0x24, -0x16,0x57,0x7d,0x08,0xd3,0x28,0xe8,0xc3,0xe8,0x98,0x5f,0x5f, -0x83,0xa8,0x47,0x07,0x2e,0x87,0x7a,0xa5,0xc9,0x67,0x7a,0x8c, -0x83,0xba,0x28,0xb6,0x9b,0x6e,0x04,0x62,0x69,0x7e,0x27,0x5a, -0x3e,0xe6,0x83,0xd9,0x15,0x8c,0x0d,0xcb,0xd0,0x66,0x75,0x86, -0xa8,0xf4,0x04,0xaa,0x34,0x3a,0x61,0x76,0xca,0x48,0x75,0x7b, -0x09,0xd9,0x27,0x91,0x3b,0xb1,0xdb,0x88,0x1d,0xb3,0x55,0x91, -0x19,0xec,0x7f,0x8c,0x60,0xa7,0x8d,0xe0,0x26,0xd4,0x37,0x88, -0x02,0x3b,0xf6,0xcb,0x17,0xaf,0xfe,0xf7,0xbf,0x4e,0xdf,0xbd, -0x7d,0xe9,0x96,0xf2,0xf2,0x46,0x10,0xf3,0x4e,0x90,0x74,0xba, -0x4f,0xb6,0x7f,0xee,0x6f,0xff,0xb1,0xbb,0xfd,0x7d,0xf3,0xf4, -0x6f,0x0d,0xbd,0xfd,0xfd,0xaf,0xe4,0xe4,0xe8,0x81,0x5e,0xfc, -0x8b,0x93,0xe6,0x49,0xfb,0x44,0x9d,0xc4,0x27,0xc9,0xc9,0xed, -0xc9,0xf2,0xe4,0xf1,0xc9,0x7e,0x6f,0x47,0x1d,0x3e,0x79,0xf1, -0xf2,0xf8,0xb5,0xdb,0x64,0x18,0xcf,0x4e,0x3b,0xd4,0x6d,0x00, -0x6e,0x9f,0x9b,0x16,0xe5,0x1a,0x9b,0x78,0x16,0x12,0xa2,0x84, -0xb6,0x35,0xe4,0x09,0xef,0x4a,0xf4,0xac,0x9e,0xf6,0x3f,0xa2, -0x78,0x81,0x0e,0xd4,0xdd,0x9e,0xfa,0x88,0x1c,0x45,0x56,0xc1, -0x64,0x20,0x7f,0x02,0x0f,0x63,0xe1,0x25,0x10,0xdd,0x05,0xa0, -0xe8,0xa5,0x7c,0xeb,0xc2,0x14,0x80,0xea,0x94,0x85,0x6e,0x52, -0x64,0xc6,0x69,0x6d,0x3a,0xd1,0xea,0xe2,0x99,0xd1,0x0c,0x5b, -0x30,0x69,0x27,0xf0,0x92,0xc5,0x1c,0x54,0x95,0xb1,0x7f,0x99, -0x59,0x57,0xf9,0xf4,0x30,0x2a,0x8b,0xb8,0x35,0xf0,0xdd,0x0e, -0xd8,0x17,0x23,0x25,0x29,0x92,0x1c,0x51,0x6d,0xf8,0x9c,0x08, -0x85,0x59,0x1d,0xf7,0x3c,0x91,0x75,0x69,0xc2,0xf8,0x86,0x67, -0x33,0x26,0x07,0x49,0xe1,0xa2,0x7e,0x24,0x80,0x12,0x1f,0x5d, -0xe9,0x0c,0xed,0x75,0x13,0xbc,0xdd,0xb1,0x19,0x60,0xf6,0x6b, -0x5c,0x09,0x36,0xb7,0x23,0x03,0x0f,0xb8,0x8d,0xf0,0x70,0x6e, -0x7a,0x35,0xa4,0x9d,0x84,0x1e,0x8e,0xc3,0x5b,0xae,0x21,0x2e, -0xb7,0x36,0xe2,0xab,0xee,0x79,0x41,0x8c,0xca,0x71,0x38,0x3b, -0xda,0xd7,0xe1,0xf5,0xed,0x5b,0x09,0x11,0x20,0x7d,0xca,0x8f, -0x94,0xf5,0x60,0xd1,0x03,0xc1,0x85,0x02,0x8f,0x51,0xf4,0x92, -0x98,0x5f,0x07,0x09,0x89,0xb8,0x40,0x6d,0x5f,0x06,0xf3,0xfb, -0x3e,0xd2,0x95,0x87,0xce,0x38,0xb7,0x58,0xad,0xee,0x19,0xbd, -0xdd,0x75,0x26,0x3e,0x4e,0x9b,0x43,0x38,0x18,0x9d,0xa2,0x46, -0x7c,0x2c,0x32,0xa8,0xba,0x9b,0x10,0x07,0xe3,0x99,0x63,0x24, -0x25,0xbf,0x53,0xb9,0x91,0x88,0x04,0xf3,0xbc,0xc9,0xbe,0xe2, -0xff,0x6d,0xae,0x2b,0xc0,0x2f,0xa4,0xf8,0xc0,0x84,0x2f,0x30, -0x9b,0xb3,0xc9,0x4b,0x38,0xae,0x3c,0x85,0xe3,0x4a,0xd2,0x4b, -0xd4,0xa9,0x9e,0x18,0xcf,0xc0,0x3c,0x5d,0x2c,0xf1,0xe9,0xc9, -0xc1,0xb7,0x20,0x5e,0x29,0xa6,0x82,0xe6,0x5d,0x3d,0x96,0x99, -0x17,0x47,0xfd,0x0b,0xd6,0xe3,0x35,0x36,0x2e,0x3a,0x65,0x29, -0x27,0xbd,0x79,0xb7,0x15,0xa0,0x85,0xf4,0x56,0x37,0x23,0xbc, -0x79,0x03,0xed,0x48,0x77,0xb6,0xa1,0x37,0xec,0xeb,0xe4,0x43, -0x2b,0x15,0x29,0xeb,0xa9,0xb2,0x1a,0x9c,0x7b,0xf0,0x5a,0x68, -0xba,0x0e,0x91,0x20,0xf0,0xe0,0xa1,0xe7,0x75,0xa4,0xc8,0x44, -0x92,0x62,0x93,0x82,0xf0,0xc4,0x02,0xde,0xf3,0x0f,0xa3,0x5e, -0xdb,0xab,0x8d,0x18,0x20,0x7a,0x76,0xf5,0xe9,0xc1,0x91,0xed, -0x52,0x7e,0xdc,0x5e,0xc4,0xa2,0x00,0x65,0x01,0xfd,0x26,0x4e, -0x96,0x74,0x39,0x70,0x56,0x4c,0x46,0xa8,0x5d,0x00,0xaf,0x50, -0xc7,0x7a,0x95,0xa4,0xb5,0x17,0x97,0xde,0x9d,0xdc,0x95,0xfe, -0x84,0x01,0x41,0xd5,0xd7,0x29,0x85,0xe1,0x64,0x14,0x02,0x63, -0x08,0x6e,0x6d,0x05,0x48,0x49,0x26,0xc7,0xb0,0xf6,0xc5,0xdb, -0x99,0x26,0x03,0x90,0x9d,0x36,0xfd,0x1b,0x74,0x99,0x98,0x80, -0x68,0xcb,0x8f,0x52,0x34,0x33,0xc0,0x24,0xc2,0x0f,0x8c,0x41, -0x9a,0x07,0xe3,0x38,0x4b,0x52,0x50,0x98,0xae,0xd4,0xbd,0x1d, -0xa2,0x47,0x00,0x77,0xdc,0xb7,0x27,0x6b,0x13,0xd1,0xee,0x59, -0x8f,0x8b,0xe4,0xf7,0x33,0x5c,0x04,0xbe,0x7f,0x7b,0x9e,0x0f, -0x47,0x03,0xfb,0xcc,0x4f,0x44,0xa1,0x2d,0x5b,0x47,0xfa,0x33, -0x01,0x2c,0xc9,0x17,0xf1,0x61,0x15,0x24,0x7e,0x1f,0xd2,0xa7, -0x83,0xa7,0xda,0x4f,0xba,0x1b,0x7d,0x6a,0x4a,0x09,0xf8,0xfe, -0xd0,0xd1,0x74,0x50,0xca,0xe8,0x0b,0x4c,0x4a,0x14,0x95,0x4b, -0x22,0x38,0x9f,0x9e,0x8f,0xf9,0xec,0x0a,0x40,0xcc,0x33,0x70, -0x6f,0x8d,0x13,0x83,0xd7,0x3c,0x81,0xc5,0x07,0xd0,0x75,0x25, -0xb8,0xe1,0x43,0x42,0x85,0x17,0x83,0xa2,0x7a,0x00,0xd9,0xe4, -0xa4,0xb3,0xb9,0x9b,0x86,0x91,0xb2,0x27,0x2c,0x1e,0xa7,0x0f, -0x89,0x51,0xe8,0x50,0xd3,0x2f,0x89,0xbe,0xc2,0x96,0xf6,0x16, -0xb5,0x6c,0x45,0x9a,0x6e,0x6f,0x7c,0xd9,0xbe,0x84,0x9b,0x26, -0xfa,0x9d,0xe3,0x8f,0x79,0xa7,0x0e,0x98,0xaf,0xaf,0x27,0x63, -0xd0,0x2c,0x2e,0x77,0x8c,0x5f,0x8e,0xff,0xd4,0xab,0x45,0xba, -0xc6,0x7b,0xbc,0x71,0x99,0xb5,0xa5,0xe7,0x3d,0x68,0xde,0x43, -0x73,0x7f,0x1a,0xd8,0x6d,0x02,0xb2,0xe5,0x18,0x43,0x87,0x73, -0xd3,0x4d,0x62,0x44,0x14,0x45,0x49,0x7a,0x6a,0x8d,0x5e,0x10, -0x12,0x0b,0x48,0xc8,0x5b,0xa9,0x97,0x8f,0x61,0x14,0xef,0x33, -0x9c,0x26,0x6e,0x1a,0x69,0x3d,0xbe,0x5d,0xb5,0x3f,0x10,0x4c, -0x2f,0xb3,0x7a,0xdc,0x34,0xae,0xb8,0x61,0xa6,0xeb,0x36,0x7b, -0x38,0x98,0x88,0x8f,0x0f,0x67,0x05,0xc5,0xd6,0x4c,0x09,0xce, -0xe2,0x64,0x4e,0x6c,0x1e,0xa3,0x66,0x42,0x30,0xb8,0x39,0x31, -0x30,0xbe,0x75,0xca,0xf7,0xab,0xb8,0x56,0x0e,0x8e,0xa5,0xc6, -0x7a,0xfd,0xfb,0x70,0x3a,0xcd,0x25,0x74,0x18,0x1f,0x2c,0x44, -0x87,0x61,0xb6,0xd2,0x9c,0xc7,0xe7,0xac,0x2e,0xdd,0x6e,0x78, -0xdd,0x1e,0xdb,0x4f,0xb7,0x9c,0x97,0xdd,0x2b,0x0c,0xba,0x31, -0x61,0x8f,0x8d,0xb9,0x05,0xe5,0xf0,0xe8,0x8d,0xdc,0x05,0x78, -0xa8,0xd1,0x34,0x70,0x4f,0xb2,0x4b,0x55,0x72,0x39,0xfc,0x25, -0xa6,0x54,0x62,0x16,0x8c,0x33,0x25,0x2b,0x43,0x4f,0x06,0x78, -0xad,0x5a,0x2c,0x6a,0x0a,0xe3,0x4a,0x6d,0xfa,0x73,0xe4,0x94, -0x0c,0x1a,0x12,0x0a,0xe1,0xd7,0x6c,0x2c,0x5c,0x2b,0x8e,0x87, -0xfe,0x4e,0x0b,0x5e,0x6e,0x2a,0x78,0x1c,0x60,0x0c,0xc4,0xe3, -0xf3,0xa7,0x70,0x2e,0x9b,0x4c,0xb3,0x52,0xcc,0x62,0x11,0xc1, -0x9b,0x83,0x48,0x74,0xda,0x3b,0x07,0xee,0x36,0x0f,0xbc,0xc8, -0x86,0xc3,0xcb,0x29,0x18,0x1f,0x88,0x4b,0x78,0x58,0xf3,0x8f, -0x2c,0x84,0xf1,0xc5,0x0e,0xdc,0xbc,0xe2,0xbb,0x90,0x38,0xd2, -0x47,0x88,0x0d,0x78,0x67,0x6a,0xbd,0x15,0x6d,0xe4,0xc5,0xf8, -0xfe,0x6c,0x03,0xcf,0xe5,0x51,0xd8,0xc7,0x82,0x60,0xd4,0x16, -0xf0,0xba,0x43,0x30,0xc8,0xfe,0x0e,0x54,0x4a,0xd0,0xd3,0x0d, -0xbc,0xf1,0xe8,0x02,0xe8,0x09,0x48,0x1a,0xb1,0x32,0x22,0x3d, -0x2b,0x37,0x1c,0x6e,0x1a,0xf9,0xe1,0x48,0x09,0x4c,0x4f,0xad, -0x35,0xeb,0x34,0xaa,0x8c,0x8e,0xd4,0x79,0x7f,0x3c,0x9e,0x8c, -0xf3,0xf3,0xfe,0xe8,0x85,0x2e,0xed,0xf4,0x54,0x86,0x23,0x35, -0x9c,0xf5,0x2f,0xbd,0xac,0xcf,0x74,0x04,0xba,0x05,0xba,0x78, -0x09,0x1e,0x2c,0x23,0x38,0xe4,0x44,0x60,0xa7,0x23,0x25,0x5e, -0x4d,0xff,0x79,0x99,0x17,0x14,0x82,0x8f,0x08,0xd7,0xf1,0x48, -0xff,0x89,0x68,0x65,0x8f,0xe0,0x6f,0x64,0x96,0x75,0x33,0xae, -0x51,0x82,0x90,0x32,0xa3,0x87,0xdc,0x5d,0xca,0x4c,0x9e,0x31, -0x4f,0x9d,0x3a,0xf6,0xef,0x06,0x4e,0x37,0x50,0x6f,0xf4,0x9b, -0xad,0x0b,0x12,0x8c,0xbf,0xee,0x7e,0x2c,0xdb,0x8f,0x42,0x2a, -0x8c,0xa3,0x86,0xc8,0x7d,0x47,0x60,0x8f,0x45,0x80,0x1e,0xb2, -0xc5,0x93,0xaa,0x58,0x2c,0xe9,0x0d,0xb6,0xb5,0xad,0xa1,0x0b, -0x46,0xa4,0x0b,0x9d,0x46,0xf8,0x13,0xc9,0x85,0xf2,0x6d,0xa9, -0xcb,0xe5,0x33,0xae,0x6a,0x86,0xa6,0x3c,0x4c,0xec,0xda,0x52, -0x07,0xa1,0x1c,0x32,0x7f,0xc8,0x69,0x12,0x30,0xfe,0x78,0x02, -0xb5,0x76,0x85,0x5f,0x40,0x24,0x1a,0x67,0x35,0xea,0x57,0x6c, -0xee,0xca,0x1c,0xc6,0x11,0xc0,0x28,0x35,0x81,0xa1,0x87,0x9c, -0xd4,0x51,0xd0,0xa2,0x6e,0xe9,0xb0,0x04,0x67,0xb4,0xbc,0x3f, -0x8a,0xe5,0xe1,0xd6,0x6d,0xc4,0x3a,0x52,0xcf,0x40,0xca,0xdf, -0x85,0x50,0x0f,0x55,0xa7,0x18,0xa1,0x97,0xb0,0xb4,0x05,0x75, -0x4f,0x49,0x3b,0x40,0xa0,0x64,0x0b,0x10,0x7e,0x0d,0x2f,0xb9, -0x86,0xcf,0xa7,0x93,0xeb,0xba,0x3a,0xae,0x5f,0x3b,0xa9,0x7c, -0x88,0xef,0x3f,0x92,0xa0,0xae,0x0e,0x49,0x4d,0x05,0xe1,0x16, -0xa8,0xaa,0x66,0x89,0xab,0x9a,0xec,0x9d,0x17,0xa5,0xae,0xc1, -0x4e,0xc9,0x07,0xe1,0xa8,0xf0,0x7b,0xb2,0x76,0x24,0x80,0x0b, -0x1a,0x83,0xa1,0xd4,0x50,0x2f,0x2a,0x5a,0x89,0xda,0xc7,0x2f, -0x2b,0x6c,0xd3,0x3b,0x4b,0x4b,0xc4,0x96,0xf3,0x0f,0x58,0x18, -0x5f,0x0c,0xfc,0x12,0x5e,0xd8,0xfa,0x44,0xd1,0x66,0x96,0x51, -0xd4,0xd6,0x56,0xb0,0x8b,0x30,0x24,0xc7,0xbf,0x9a,0x8f,0x46, -0x26,0x4a,0x36,0x99,0x9e,0xd3,0x95,0x6d,0x36,0x2f,0x1f,0x6f, -0xd6,0x6d,0x34,0xb3,0xa6,0x29,0x8b,0xa8,0xd4,0x76,0xba,0xd4, -0xbf,0xba,0xf1,0xbc,0x52,0xc5,0xa6,0x57,0x0c,0x9f,0x8a,0x85, -0xb8,0x9a,0x8f,0x0f,0x80,0x34,0x37,0xdf,0x4a,0xab,0x93,0xe4, -0xfe,0x18,0x60,0xae,0xbe,0x5b,0xdf,0xa8,0xa1,0xa3,0x7a,0xf2, -0x49,0x90,0xba,0x06,0x42,0x05,0x5c,0x89,0x0a,0xa2,0xbb,0xb5, -0xed,0x22,0x41,0x7b,0x60,0x97,0x69,0x8e,0x99,0xc2,0x11,0x16, -0x9f,0x7b,0x0d,0xe0,0x6d,0x66,0x55,0x63,0xde,0x9f,0xcf,0x06, -0xad,0xdc,0x9d,0xc9,0x76,0x1a,0x59,0x0c,0x74,0x2c,0x47,0x29, -0x61,0xb1,0xa8,0x96,0x7c,0xa0,0x58,0xc3,0x5f,0xfc,0x03,0x8c, -0xae,0x5d,0x4c,0x54,0xc5,0xe2,0x1f,0xe4,0xb1,0xa0,0x3e,0x0f, -0x2c,0xf0,0x4b,0x13,0x8f,0x2b,0x4a,0x37,0x15,0xbd,0xd0,0xf3, -0x8a,0x0f,0x02,0x32,0xe7,0xca,0x54,0x51,0x65,0xf3,0x28,0xc4, -0xca,0x4f,0xf0,0xe9,0x27,0xca,0xd7,0x0c,0x63,0xef,0x5b,0xd9, -0x28,0xb9,0xd0,0x62,0xb0,0xd4,0x07,0x37,0xcf,0x52,0xfd,0x1a, -0x96,0xc8,0x74,0x56,0x01,0x85,0x79,0xbb,0xa0,0x11,0x6b,0x6a, -0xe8,0x2f,0x17,0xeb,0xd3,0xbf,0x92,0x5e,0x63,0xe1,0x3c,0x15, -0xf4,0xd4,0x15,0x00,0x96,0x8b,0xc2,0x4e,0x7c,0x7e,0x07,0xad, -0x5f,0x5e,0x4d,0x91,0x23,0x90,0xc4,0x84,0xf6,0x40,0x6b,0xe2, -0x45,0x67,0x27,0xde,0xb1,0xaf,0x64,0x50,0x74,0x1d,0x2a,0x4b, -0xa5,0xd6,0xd2,0x37,0x19,0xff,0x30,0xbc,0x98,0x4c,0xf1,0x12, -0x46,0x30,0x4e,0xf9,0x1f,0x43,0x3b,0xf9,0x57,0x80,0x04,0x43, -0xd4,0xd9,0x6c,0xd2,0x01,0x59,0x89,0xd7,0x2b,0x4a,0x71,0x5b, -0xba,0x66,0x54,0xc3,0x8a,0xac,0x22,0x0f,0xe0,0xc3,0xca,0xac, -0xac,0xc6,0x9f,0xaa,0xc0,0x57,0x90,0x7e,0x07,0xd1,0x0e,0x3b, -0x30,0xb4,0xc7,0x9f,0x6f,0x86,0x2f,0x48,0xa8,0x91,0x79,0x43, -0xc1,0x6c,0xab,0x57,0x93,0xf9,0x68,0x70,0xd4,0xff,0x7d,0x48, -0x4c,0x70,0x56,0x13,0xef,0x0d,0x24,0x3e,0x37,0xee,0xca,0xc2, -0x08,0x6c,0x04,0x86,0x1e,0xeb,0x64,0x6a,0x4e,0xbc,0xc2,0x09, -0x9d,0x3a,0x22,0xc2,0x07,0xf7,0xf6,0xa0,0x6a,0xc5,0xbf,0x2c, -0x86,0xe3,0x87,0xee,0x28,0xb9,0xf9,0x61,0x32,0x81,0xa7,0x81, -0x06,0x39,0xbc,0x9b,0xbc,0x0b,0xbd,0xc5,0x4c,0x79,0x96,0x21, -0xcb,0x3d,0xbf,0xb8,0xc8,0x3f,0x55,0xef,0xda,0x94,0x96,0x74, -0xe0,0x75,0x6a,0xea,0xc5,0x95,0x17,0xbd,0x23,0xc2,0x23,0xa5, -0xa0,0x14,0x55,0x62,0x72,0x08,0xd0,0xa6,0xfb,0xe4,0xe0,0x53, -0x67,0xb0,0xde,0x2f,0x64,0x1f,0x32,0x4e,0x4a,0x76,0xf4,0xd1, -0x87,0x0f,0x47,0xc7,0x0e,0xc2,0x51,0x80,0xf7,0xaa,0xc5,0x50, -0x23,0x19,0x3c,0x99,0x5e,0x42,0x31,0x53,0xf8,0x60,0x1d,0xb9, -0xc2,0x9d,0x2c,0xc9,0x4d,0xa7,0x28,0x85,0x3c,0x46,0xed,0xe9, -0x13,0x7f,0x68,0x86,0xb9,0x23,0xa1,0x2c,0xee,0x34,0x96,0xd1, -0xa6,0x20,0xe5,0x95,0x61,0x81,0x49,0x9b,0x1f,0x88,0x3b,0xbe, -0xa3,0xae,0x5d,0x00,0xea,0xb5,0x43,0xd0,0xc5,0x22,0x5e,0x3f, -0x77,0xd6,0x05,0x37,0x78,0x01,0x38,0x09,0x6a,0x84,0x61,0x49, -0x5d,0x7d,0x6f,0x0e,0x5b,0x7e,0x56,0x56,0x21,0x01,0x21,0x0b, -0x78,0xfb,0x95,0x91,0xb2,0x41,0x68,0x1d,0xc0,0x7d,0x13,0x3c, -0xe1,0x73,0x2f,0xd1,0x59,0x50,0xb4,0x84,0x3f,0xf0,0xd5,0x85, -0xbb,0xfa,0x58,0xd5,0xaf,0x0c,0xbd,0xb9,0x0b,0x19,0x02,0xe4, -0xe8,0xcf,0xe6,0x4f,0xe2,0x6d,0x11,0xde,0xb0,0x21,0xab,0x0e, -0x3b,0xa2,0xfa,0x6c,0x8e,0x4c,0x2c,0x55,0x6c,0x15,0x67,0x9d, -0x0e,0xc2,0xbc,0x3d,0x45,0x0e,0xfb,0x90,0x34,0xef,0xb8,0x1a, -0xe2,0xd8,0xda,0xe2,0x43,0x5d,0x98,0xe0,0x8e,0x76,0x02,0x7d, -0x72,0x2b,0xf1,0xca,0x5e,0xf2,0xe8,0x17,0x40,0xe8,0xea,0x6b, -0xc5,0x02,0x1c,0x0b,0xd8,0xbb,0x1a,0x61,0xe9,0x73,0x6f,0xc4, -0xda,0x55,0x2f,0x32,0x12,0x42,0x9e,0x0b,0x64,0x7c,0x89,0x01, -0x61,0x84,0xee,0xf0,0x0c,0xd6,0x02,0x2a,0xf9,0x0d,0x0b,0x39, -0x0a,0x2e,0x7a,0x4a,0x22,0x35,0xb8,0x56,0x85,0xbe,0x83,0x6b, -0xa1,0x92,0x5c,0x4e,0x6f,0x27,0x6d,0x91,0xe0,0xac,0xb5,0x02, -0x4d,0x74,0xfd,0x9b,0x05,0x96,0x5a,0x7d,0xb3,0x6d,0x14,0xc7, -0xe3,0x80,0x29,0xa9,0x46,0x48,0x42,0xb2,0x70,0x3f,0xe3,0x27, -0x78,0xce,0xaf,0x1a,0x83,0x81,0xaa,0xa4,0x79,0xfb,0x9f,0x70, -0x1c,0xcd,0x6f,0x93,0xc3,0x5a,0x0a,0xe0,0xdf,0xae,0xa0,0xa7, -0xe2,0x38,0xe3,0xcb,0xe8,0x84,0xc5,0x45,0x42,0x3d,0x1d,0xfe, -0x3e,0x9c,0xa2,0xe0,0xb3,0x86,0x7c,0x7b,0x4b,0x1d,0x92,0xce, -0x37,0xdb,0xfa,0xc4,0x33,0xb2,0x42,0x6f,0xfd,0xcd,0xd2,0xfd, -0x8f,0x57,0xfd,0x19,0xca,0xf0,0x94,0xb8,0x12,0x75,0xb6,0x6b, -0x32,0x71,0xb3,0xe4,0x72,0x39,0x47,0x1b,0x2b,0x2e,0x0e,0x9a, -0x74,0xf7,0x91,0x18,0x7e,0x39,0x44,0x25,0xef,0x0c,0x2a,0x70, -0x71,0x72,0xd2,0x06,0xe2,0xdc,0x86,0xcb,0xd7,0xd2,0x8e,0xd8, -0x0e,0xa3,0xef,0x46,0x2f,0x2e,0xb6,0x5f,0x4d,0xc6,0xc3,0x6d, -0xb4,0xab,0x12,0xf5,0x32,0x94,0x4c,0xca,0x54,0x99,0x40,0xdb, -0x01,0xa8,0x72,0x60,0xd9,0x70,0xc3,0x69,0x7b,0xce,0xb4,0xbd, -0x4e,0x6d,0x64,0xd0,0x40,0x9d,0x68,0x27,0x6a,0xc0,0x07,0x38, -0x3c,0xc7,0x58,0x49,0x30,0xed,0xac,0x68,0x37,0xb2,0x4a,0xbf, -0x40,0x80,0x0a,0x25,0x83,0x5b,0x23,0x06,0x95,0x98,0xc2,0xcb, -0xbe,0xd8,0x37,0xce,0xec,0x9f,0x28,0x82,0xf3,0x86,0x5e,0x45, -0x40,0xc6,0xfa,0x17,0xa2,0x03,0xb1,0xec,0x7a,0xe8,0xde,0xbc, -0x5b,0x07,0xdd,0x64,0xed,0xda,0xa2,0x81,0xea,0x3b,0x11,0xb2, -0x80,0x78,0x3d,0x94,0x6c,0xa1,0xfa,0x4e,0xa4,0x28,0x65,0x5e, -0x0f,0xe5,0x4f,0xcf,0x9e,0x1c,0xdc,0x8d,0x90,0x44,0xd5,0xeb, -0x61,0x3c,0x7e,0xfb,0xe4,0xe9,0x1a,0x34,0x1a,0x79,0xf7,0x7a, -0x48,0xd9,0x39,0xf2,0x1a,0x1d,0x84,0x42,0xf3,0x35,0x7b,0x08, -0x5c,0xa1,0xdc,0x89,0x72,0xc9,0x6a,0x12,0x68,0x06,0xd4,0x5b, -0xd3,0x21,0x46,0x98,0xde,0x86,0x9b,0x14,0xb1,0xac,0x29,0x3f, -0x19,0x67,0x67,0xc5,0x92,0x0c,0xa8,0x4f,0x4f,0x81,0x60,0x9c, -0x8f,0x4e,0x06,0x3b,0x1d,0x0e,0xe6,0x66,0xe6,0xc9,0x65,0xdd, -0xad,0xcd,0xee,0xed,0xdb,0xf9,0xfc,0x5a,0x48,0x7f,0xe1,0x0b, -0x74,0x13,0xe0,0xf7,0x88,0x05,0x6f,0x19,0x95,0xd1,0xac,0x94, -0xf2,0xb1,0xb8,0x28,0xbf,0x88,0x65,0x16,0xbc,0x4a,0xf2,0xb3, -0x79,0x3a,0x20,0x3e,0x70,0x22,0x8e,0x70,0x26,0xb2,0xcd,0x94, -0x64,0x32,0xd2,0x7a,0x68,0x95,0x89,0x5d,0xbf,0x9c,0xf2,0xce, -0xc2,0xfc,0xce,0xe6,0x7a,0xc0,0x55,0x77,0x40,0x3d,0xd7,0x38, -0x2f,0x06,0x6d,0xfa,0x09,0x5a,0x25,0x10,0x03,0x76,0x4a,0x6d, -0x56,0x25,0x01,0x64,0xce,0xb5,0x04,0x1b,0x42,0x94,0xc4,0xc6, -0x4e,0x06,0x6c,0x9a,0xa2,0x01,0x66,0x66,0x1b,0x41,0x9b,0x5b, -0xdd,0x83,0x4e,0x85,0xfb,0x2b,0x46,0x91,0xf2,0xaf,0x35,0xfa, -0x09,0x23,0x02,0x0c,0xcf,0x36,0x18,0xb3,0xbf,0x3b,0x19,0xd5, -0x9a,0x65,0x38,0x46,0x2f,0x86,0x7a,0xf6,0x78,0xfa,0x34,0x66, -0xff,0x85,0x8d,0x83,0x9f,0x58,0xb0,0x1e,0x4d,0xfd,0x96,0x03, -0xb0,0xba,0x52,0x0c,0x48,0x95,0xc2,0xfc,0x8a,0xa3,0xea,0xf9, -0x28,0xb4,0x06,0x97,0x01,0x55,0x34,0xe9,0x2e,0x47,0x93,0xb3, -0xfe,0x88,0x78,0x24,0xf6,0xea,0x51,0xa1,0x33,0x87,0x93,0x44, -0x55,0xc0,0x26,0xbe,0x65,0x5b,0x63,0xa0,0xb6,0xc2,0x29,0x9a, -0xf4,0x86,0x46,0xfa,0xb1,0x47,0xc3,0xe9,0xef,0xf9,0xf9,0xf0, -0xf9,0x64,0xea,0xa1,0xb4,0x0c,0x9a,0x05,0x9f,0x7a,0xac,0x2e, -0xac,0x00,0xd6,0xa2,0x36,0x70,0xbb,0xce,0x4c,0xf1,0x6a,0x51, -0x30,0x8d,0x70,0x32,0x4d,0xa7,0x56,0x83,0x96,0x2e,0xfe,0x7a, -0x9a,0x2f,0xc1,0x37,0x5a,0xba,0x41,0x7f,0xe2,0x17,0x7e,0xa6, -0x2f,0x95,0x8b,0xa3,0x35,0xe2,0x0e,0xe4,0x55,0x97,0x82,0x16, -0x7f,0x55,0xa2,0xb5,0xb2,0xb3,0x0e,0x76,0xbc,0x37,0xb4,0xe8, -0xe0,0xee,0x29,0x38,0x29,0xf0,0x29,0x8b,0xda,0xce,0x61,0xd4, -0xc7,0x8a,0xa1,0xc9,0x05,0xe6,0xa6,0xc2,0x64,0x70,0x69,0x64, -0x68,0xe4,0x13,0x69,0x90,0x9b,0x14,0xcd,0x0c,0x02,0xb3,0x3e, -0x06,0x38,0x04,0x10,0x85,0x24,0x10,0xf5,0xea,0xd6,0x56,0x8d, -0xd8,0x84,0x85,0x31,0x96,0xf3,0xd4,0x51,0x4e,0x94,0xef,0x2f, -0x08,0x94,0x6a,0x30,0x42,0xa1,0xd5,0xeb,0xb2,0x81,0x2b,0xb5, -0xa3,0x1e,0x95,0x2c,0xab,0x39,0x6d,0xce,0xc7,0x30,0x2c,0x4f, -0x49,0x43,0xbb,0xd0,0x1f,0x37,0xf9,0xf9,0x87,0xfa,0xfb,0xd3, -0xae,0xb9,0x60,0x0f,0x6f,0xd4,0x7b,0x49,0x52,0xa1,0xc8,0x96, -0x04,0x17,0xf1,0x19,0xe3,0xe7,0x21,0x5e,0x4f,0x59,0xd2,0x2e, -0x2d,0x78,0xa6,0x51,0x12,0xd7,0x86,0xee,0x5a,0x30,0xf6,0xcb, -0x51,0x16,0xb8,0xbc,0x70,0xda,0x66,0xf3,0x10,0xd9,0xcb,0xb2, -0x6a,0x4c,0x68,0xd1,0x74,0xf5,0xe0,0x0c,0x8e,0x2f,0x99,0x1f, -0xa6,0x8d,0xeb,0xcb,0x30,0x48,0x4b,0xee,0x28,0xdb,0xb0,0xab, -0x06,0xf8,0x74,0x36,0x6d,0x88,0x53,0x1e,0xee,0x41,0xcd,0x06, -0x25,0xb4,0x74,0x2b,0x1a,0xca,0x5e,0x7a,0xfa,0x2b,0x8f,0xa8, -0xba,0x5c,0x79,0x5c,0x91,0x60,0xf6,0x54,0x16,0x29,0xad,0xaa, -0x0a,0x3c,0x6e,0xcb,0x34,0xd0,0xb7,0xc6,0x74,0x3a,0x98,0xbd, -0x91,0x35,0x00,0x2e,0xc0,0xab,0xc4,0xdc,0xf8,0x14,0xa9,0xad, -0x85,0xeb,0x25,0x77,0x01,0xf9,0xe5,0xf5,0xf8,0xb7,0x14,0x5c, -0xae,0xb8,0x2b,0xde,0xaf,0xba,0x9f,0xef,0x0d,0xb9,0xd8,0x89, -0x8d,0x2b,0x21,0x4f,0xf0,0x65,0xda,0x98,0x13,0x9b,0xe7,0x62, -0xa5,0xe2,0x38,0xf0,0x4f,0x64,0xdc,0x66,0x25,0xe6,0x03,0x96, -0x82,0x00,0xee,0xc7,0xe1,0xac,0x04,0xb6,0x62,0x3f,0xa9,0x64, -0x82,0x3c,0x01,0x9c,0x47,0x29,0xbd,0x39,0xc0,0xa2,0x41,0x04, -0xb0,0x82,0x46,0xbd,0x72,0x68,0x80,0x28,0xb1,0x44,0x08,0xf3, -0x1a,0x0e,0xda,0xea,0xf7,0xa1,0x9f,0xb0,0xe1,0xc5,0x50,0x33, -0x09,0x83,0xec,0xde,0x6f,0x4d,0xfc,0x06,0x0b,0x87,0x1a,0xe8, -0xc9,0xf4,0xb2,0x70,0x82,0xd2,0xb6,0x14,0x7a,0xf8,0x56,0x79, -0xc1,0xa2,0xb9,0x3c,0xd2,0x23,0x2b,0x2c,0x25,0x29,0x23,0xbd, -0x57,0x63,0xe3,0xc6,0x06,0xaf,0x6a,0x11,0x8b,0x0c,0xb7,0xb9, -0x46,0x91,0x10,0x0d,0xdd,0x34,0xc9,0xca,0x10,0xb8,0xbd,0x31, -0x26,0x63,0x0d,0x7d,0xc6,0xce,0x34,0x96,0xb7,0x0c,0x07,0x88, -0xe9,0x68,0x0b,0x6d,0x1a,0x04,0x08,0xfe,0xd2,0x9e,0x48,0x4a, -0xad,0xa6,0xfb,0x37,0x76,0x7c,0x56,0x45,0x93,0xdd,0xd5,0x3e, -0x55,0xb5,0xe8,0x02,0xc6,0xde,0x7f,0x5f,0x5d,0xb8,0xe8,0x10, -0xbd,0xf5,0xd7,0xa5,0xbc,0x7b,0xe4,0x0a,0x71,0x5b,0xa7,0x54, -0x0b,0xab,0x13,0x64,0x71,0xdc,0x4a,0xdd,0xfa,0x34,0x2e,0xe5, -0x20,0x57,0x84,0xf0,0x9c,0x6d,0x43,0x4e,0x5b,0xd0,0xea,0xbe, -0x11,0xfc,0x46,0x1c,0x08,0xae,0x27,0x60,0x8a,0x11,0xe7,0xbc, -0xd8,0x4d,0x31,0xb2,0x76,0x37,0x05,0xc5,0xb5,0x44,0xda,0x4f, -0xd6,0x1c,0xd2,0xd3,0x79,0x31,0x9b,0x5c,0xbf,0x36,0xd2,0x1d, -0x2e,0x05,0x59,0x8c,0x73,0x3f,0xc9,0xf2,0x41,0x18,0x1d,0x30, -0x52,0x2c,0xb1,0xed,0x92,0x56,0x19,0x2b,0x94,0xf1,0x53,0x01, -0xd6,0xa6,0xec,0x29,0xef,0x31,0x40,0x72,0x6b,0xf3,0x0c,0x26, -0x1a,0x8a,0xd0,0x4a,0x28,0xbd,0x81,0xf6,0x0b,0xa3,0xa5,0x30, -0x1a,0xd9,0x12,0x21,0xe0,0xa8,0x32,0xe8,0xd1,0x31,0xc9,0x70, -0xda,0x31,0xea,0x6b,0x29,0x84,0x50,0x7b,0x30,0x43,0x44,0x0d, -0x08,0x37,0x67,0x93,0x77,0x37,0x37,0x46,0xa5,0xb1,0x2d,0xd1, -0x66,0x48,0xfa,0x66,0x96,0x79,0xe8,0xb7,0xb6,0xa8,0x2e,0x61, -0x7c,0xc7,0x6f,0x85,0xb4,0xfc,0xce,0x81,0xd6,0x63,0xbc,0x62, -0xf6,0x4f,0xeb,0xa2,0x0f,0xab,0xda,0x13,0x5d,0xb0,0xc6,0xc1, -0x25,0xb5,0x8f,0xc1,0x28,0xa6,0xe0,0xf6,0x4e,0xd6,0x87,0x0c, -0x2e,0xd9,0x4e,0x48,0x80,0x47,0x75,0x82,0x5e,0x88,0x44,0xe7, -0xfe,0xf8,0xec,0xf8,0xe5,0x8b,0xa3,0x63,0x93,0x1d,0xd9,0xf0, -0x0a,0x16,0xb9,0x39,0x98,0x18,0xc8,0x72,0x66,0x37,0x9c,0xea, -0x39,0xe4,0xf8,0x82,0xb9,0x44,0x2b,0x46,0xd0,0xb0,0xb9,0x4d, -0xf6,0xf5,0xbb,0x24,0x6c,0xbb,0x72,0xd7,0x67,0xfd,0x1d,0x81, -0x61,0x6d,0x2e,0x47,0x79,0xd9,0xd4,0x9a,0x48,0xe8,0xbc,0x53, -0xbb,0xa9,0x86,0x5c,0x80,0xdb,0xea,0xcf,0xc5,0xe6,0x6a,0x77, -0xfc,0xbb,0x2f,0xae,0xad,0x36,0xc7,0x66,0x8b,0x39,0x0a,0x32, -0x2b,0x8b,0x77,0x1c,0x6b,0x9d,0xe1,0x42,0x05,0x58,0x5f,0x5a, -0x60,0xac,0xce,0x3f,0x5b,0x7d,0xfe,0x91,0x0a,0x3f,0xd5,0x87, -0x03,0x8b,0x05,0xf9,0x71,0xb1,0x3d,0xaf,0x81,0xde,0x53,0xd6, -0xec,0x55,0x18,0x74,0x72,0x08,0x97,0xa2,0xfe,0x5f,0xcc,0x45, -0xe8,0x36,0x5c,0x8f,0x1a,0x3b,0x8d,0x40,0x57,0xcc,0xcc,0x02, -0x57,0xc3,0x35,0xb1,0x50,0x6d,0x2a,0x67,0xd4,0x97,0xe1,0xd2, -0xeb,0x91,0xc5,0xa3,0xbf,0xbf,0x16,0xcb,0x44,0x90,0x03,0x81, -0xaf,0xc4,0x43,0xcb,0xaa,0xc5,0x44,0xab,0xee,0x57,0xe2,0x82, -0x75,0xcc,0x62,0x82,0xc0,0x57,0xe2,0x41,0x51,0xaf,0x45,0x84, -0xa1,0xaf,0xc4,0xc4,0x12,0x5e,0x8b,0x8b,0xc3,0x5f,0xdb,0xe6, -0x20,0xd8,0x75,0x8d,0xde,0xaf,0x19,0x03,0x15,0xdb,0xaf,0x48, -0xad,0xba,0xa1,0x75,0xe9,0x76,0x65,0x50,0x05,0xc9,0x7e,0x6a, -0x17,0xa7,0xea,0x93,0x82,0x38,0x58,0x7d,0xd5,0xba,0xb4,0xfb, -0x27,0xd7,0xa5,0x3f,0x31,0x9d,0x77,0xff,0xa2,0x61,0xaf,0xc8, -0x7e,0xfb,0xff,0xc8,0xf1,0xea,0xf4,0xe5,0xe5,0x12,0xc1,0x51, -0xff,0x93,0xe6,0xc0,0x97,0xaf,0xcc,0x3f,0x7c,0x7e,0x31,0xf8, -0xf7,0xac,0xca,0x3c,0x20,0xfe,0xdc,0xb4,0xdc,0xad,0x9d,0x96, -0x4c,0x7b,0x9c,0xc3,0x01,0xe3,0xb7,0x37,0xd5,0x4c,0x21,0x89, -0xbe,0x0d,0x7f,0xa5,0x61,0x35,0xcb,0xca,0x36,0x94,0x93,0x8a, -0x6c,0xe2,0x5c,0x56,0xea,0x7e,0x9d,0xf9,0x53,0x89,0xf7,0x0c, -0xae,0x70,0x70,0x6a,0x1e,0x4f,0xde,0xcc,0x67,0x18,0xec,0xea, -0x2c,0x3d,0xb5,0xe2,0x34,0x67,0xc1,0x61,0x04,0xc6,0x01,0xf2, -0xe0,0x98,0x07,0x2d,0x30,0x9c,0x62,0xab,0x19,0x6f,0x03,0x74, -0xd7,0xbf,0x82,0x4f,0x24,0xd2,0x92,0xb6,0x81,0x45,0x82,0x32, -0x87,0xa9,0x7c,0xf8,0x35,0x90,0x49,0xa5,0x6b,0x67,0x01,0x1e, -0xba,0x6b,0x5e,0xf7,0x8c,0xe9,0x09,0x53,0xd0,0xd4,0x9b,0x39, -0xfd,0x01,0xb6,0x03,0x70,0x6b,0xed,0xb8,0x75,0x56,0xc9,0x31, -0xe2,0x6c,0x77,0x90,0x64,0xac,0xc2,0x13,0x79,0x9d,0x66,0xf1, -0xd7,0x60,0x45,0x85,0x02,0xe7,0xdb,0x9c,0x3b,0x24,0x8e,0x9e, -0x1d,0xf7,0x2f,0xdd,0x63,0x28,0x28,0x76,0x6b,0x8b,0xaf,0xf7, -0x21,0x50,0x3f,0x63,0x00,0x88,0xef,0xed,0xf1,0x10,0xec,0x5a, -0xc0,0x9b,0x3b,0x28,0x02,0xa8,0x1a,0xcd,0xe1,0x30,0xab,0x92, -0xa9,0x4c,0xdc,0x19,0xce,0xbe,0x4c,0x62,0xb7,0xa6,0xe1,0xbd, -0x00,0x5d,0x9a,0xd0,0xbd,0x0c,0xfc,0x7d,0x0e,0xf1,0x1a,0x1b, -0x7c,0x2f,0x16,0x54,0xd0,0x5d,0xdc,0xbb,0xe2,0x17,0x98,0xab, -0x35,0xaa,0xc9,0xe3,0xa0,0x6b,0x7a,0x51,0x9a,0xd4,0x45,0x46, -0x25,0x0a,0x5b,0x6f,0xf4,0x56,0x2f,0x94,0x18,0x0c,0xf9,0x53, -0xa9,0x92,0xc0,0xba,0x08,0x4c,0x85,0xe9,0x23,0x1b,0xbe,0x11, -0xea,0x21,0xab,0xab,0x83,0x7d,0x23,0xb4,0x47,0xcc,0xf2,0x16, -0xaf,0x76,0x7f,0xcf,0xc3,0x2a,0xf0,0x80,0x8f,0x43,0xf1,0xae, -0x51,0x1d,0x36,0x42,0xc5,0x00,0xd4,0xb3,0x05,0xdc,0x2c,0x0c, -0x07,0x58,0xc6,0x69,0xf3,0xba,0x7f,0x13,0xb8,0x8b,0xf7,0xe5, -0x17,0x77,0x75,0x59,0xe5,0xde,0xdf,0xa9,0x38,0x76,0xdd,0x85, -0x29,0x3c,0xff,0x91,0xc8,0x64,0xbd,0x11,0x83,0xa2,0x99,0xb4, -0xb6,0x50,0xc2,0x85,0x3d,0x86,0x90,0x7a,0x0c,0x84,0xcd,0xc0, -0xc2,0x20,0x6c,0x09,0x2f,0x2d,0xb9,0x9b,0x86,0xda,0x26,0x58, -0x2d,0xb3,0xaa,0xe5,0xfe,0xc8,0x97,0xa6,0x24,0x62,0xcd,0x76, -0x48,0xb0,0x0d,0xbe,0xb2,0x54,0x6e,0x2a,0xbf,0x5c,0x9c,0xc9, -0xd5,0x2b,0x76,0x8d,0x7a,0xba,0x83,0xd8,0xcc,0xf4,0x89,0x0e, -0xec,0x6a,0xfc,0xb5,0x2b,0x3b,0xf4,0xe5,0xf3,0xb2,0x14,0x07, -0x7b,0x97,0xe7,0x26,0xbc,0xf4,0xff,0xa2,0x15,0x4e,0xd3,0xf1, -0xa6,0xe4,0xa1,0x9b,0xbc,0x47,0xd1,0xca,0xb5,0xd6,0x28,0xb4, -0x17,0xdd,0x77,0x2c,0x8c,0x28,0xea,0x39,0xfb,0x15,0xde,0x9a, -0x10,0x66,0xda,0x15,0x38,0xfa,0x8e,0xf5,0xa4,0x6d,0x72,0x97, -0xe4,0x8e,0x9c,0x60,0x3c,0xfc,0xaf,0xb3,0x86,0x72,0x96,0xfa, -0x1d,0xcc,0x5f,0x45,0x0b,0xb1,0x8a,0xaa,0xc9,0x07,0x10,0xc8, -0x83,0x25,0xae,0xec,0x8b,0x16,0x32,0x3c,0xc1,0xfc,0xe9,0xed, -0x19,0x75,0x2d,0x3a,0x56,0xe6,0x67,0x73,0xae,0xb1,0xc9,0xfc, -0x25,0xd3,0xf5,0xdf,0xbd,0xa4,0x7d,0xf9,0x54,0xae,0x5c,0xef, -0xd6,0x41,0x63,0x24,0xda,0xe4,0xb8,0x6f,0x55,0xa7,0xfe,0xf9, -0x29,0x1f,0xc8,0x84,0x5d,0x87,0x0b,0xbd,0x47,0xcf,0xe8,0x4f, -0xb0,0x0b,0x83,0x6b,0x46,0xf3,0x96,0xde,0xa1,0xb1,0x77,0xb9, -0xa1,0x81,0x00,0xb7,0xf7,0x9a,0x4d,0xbf,0x22,0x3b,0xa8,0x50, -0xfa,0x92,0x6a,0x92,0x8a,0x57,0x92,0x24,0x95,0x45,0x6f,0xa3, -0x7f,0x6e,0xff,0x74,0x7c,0xfc,0x66,0x9b,0x4a,0xdb,0x36,0x98, -0x23,0xfb,0x34,0x3b,0x13,0x23,0xb3,0x63,0x34,0xed,0x52,0x1b, -0xb5,0x4c,0xc4,0x3d,0x74,0xa8,0xbd,0x59,0x41,0x69,0xe7,0x0e, -0x96,0x45,0x12,0x57,0xc5,0xae,0xa0,0x36,0xab,0x2f,0x68,0xee, -0x7a,0xc1,0x9e,0x6e,0x72,0x66,0x4e,0xdc,0xf4,0xf6,0xc7,0x05, -0xba,0xec,0xff,0x37,0x13,0xf1,0xff,0x03,0x12,0x4c,0x36,0x96, -0xa3,0xdc,0x45,0xd1,0x9a,0x3b,0x99,0x7f,0xc8,0x7c,0xee,0x99, -0x5a,0xa9,0xbd,0x77,0x90,0x1b,0x1e,0xdf,0x3a,0xd0,0x85,0x0d, -0x4d,0x5c,0xb3,0xe9,0x71,0xa8,0x7c,0xe0,0xf4,0x25,0x80,0x5f, -0x51,0x98,0xb1,0x33,0xb0,0x66,0x79,0x42,0x04,0xfa,0x15,0x85, -0xe1,0xbd,0xd4,0xba,0x25,0x09,0x71,0x11,0x9d,0x70,0xf8,0x2e, -0xe7,0x8b,0x0b,0xc5,0x09,0x2f,0x59,0x08,0x44,0x54,0x2e,0x51, -0xca,0x94,0xbe,0xa2,0x72,0x64,0xa3,0x61,0xcd,0xda,0x79,0x62, -0xa7,0xaf,0x28,0x8c,0x6d,0x40,0xac,0x59,0x5a,0x20,0x44,0xfa, -0x8a,0xf2,0xac,0x8d,0x89,0x75,0x7b,0x4f,0x8a,0x9a,0xe2,0xaf, -0xed,0x39,0x32,0x64,0xe1,0x17,0x59,0xd3,0x79,0xfe,0x05,0xde, -0x3a,0x57,0x77,0x77,0x13,0x50,0x8d,0xa4,0x86,0x80,0x55,0x8a, -0x6e,0xe4,0x45,0x2b,0x40,0xc7,0x1a,0xff,0x7c,0x68,0x0f,0x0c, -0xf4,0x70,0xd0,0xc9,0x6d,0x80,0xc6,0xa1,0x45,0xd8,0xf6,0x83, -0xee,0xec,0xee,0x6d,0x4a,0x95,0x82,0x36,0x3a,0x08,0x68,0x02, -0xc0,0x3a,0x53,0xdd,0xdd,0x67,0x40,0x2b,0xb1,0x9d,0xa4,0xe5, -0xe8,0x0a,0xbd,0xa8,0x1e,0x4e,0x42,0xaf,0xd8,0xbd,0x9d,0x90, -0xcf,0x20,0x6e,0x57,0xea,0x4f,0x63,0x76,0x7a,0xd6,0x65,0x5d, -0xe8,0xfa,0x4d,0x65,0x32,0x04,0x4d,0x86,0xf9,0x44,0xbb,0x39, -0x61,0x80,0xdf,0x58,0xe2,0x45,0x87,0x35,0x38,0xe8,0x62,0xae, -0x4a,0x8f,0x45,0x9a,0x74,0xd1,0xdb,0x46,0xce,0x8e,0x2e,0x7a, -0x6b,0xf7,0xf5,0xa0,0x89,0x7c,0x77,0xa9,0x75,0x33,0x20,0xc8, -0x54,0xd9,0x9c,0xc2,0x21,0x2c,0xa8,0xa7,0x7b,0x0a,0xa6,0xcf, -0xfd,0x97,0x40,0xce,0x84,0xe3,0x18,0x1d,0x44,0xc2,0x43,0x20, -0xef,0x6a,0xb7,0x4a,0xed,0xd5,0x34,0x55,0xa8,0x27,0x6b,0x51, -0x28,0xfb,0x95,0x08,0x53,0x91,0xb1,0x8c,0x5d,0xa9,0x04,0xeb, -0x00,0xc9,0x9a,0x21,0x81,0x65,0x68,0xa7,0xd7,0xec,0xf7,0xa8, -0x10,0x5f,0xa7,0xe8,0xdb,0x2d,0x25,0xf4,0x2a,0x84,0x3d,0xf6, -0x7d,0x94,0xd1,0xba,0x55,0x61,0x8d,0xb8,0x64,0x63,0x25,0xd3, -0x88,0x88,0xb1,0x69,0xd6,0xd0,0x32,0x35,0x12,0x65,0x9b,0xcf, -0x75,0x87,0xc9,0x5d,0xdb,0x41,0xe5,0xcc,0x81,0x8e,0xaa,0xa1, -0x8d,0x44,0xe0,0x06,0x28,0xd0,0x54,0xad,0x06,0xaa,0xd4,0x57, -0xad,0xc5,0x57,0xd6,0x5a,0xad,0x06,0x2d,0x9f,0x42,0x48,0x83, -0x14,0x73,0x96,0x13,0xcb,0x15,0xac,0x93,0x48,0x54,0xe3,0x10, -0x8a,0x66,0x21,0xa6,0xa5,0x37,0x59,0xea,0x46,0x59,0xd5,0xd0, -0x5e,0xf6,0xc0,0x7a,0x6b,0xc9,0x50,0x2b,0xf8,0x87,0xcb,0x58, -0x2a,0x32,0xfc,0x3d,0x9f,0xcc,0xf1,0xb1,0xe0,0xb4,0x38,0x07, -0x27,0x05,0x90,0xd8,0x3c,0x55,0x67,0xba,0xb0,0x0f,0x68,0x74, -0x55,0xa1,0x78,0xfb,0x0d,0x68,0x8f,0x85,0x9a,0x64,0x4a,0x1f, -0xb2,0x29,0xe1,0x35,0xbd,0x1b,0x74,0x29,0xd0,0xf1,0x94,0x64, -0x86,0x80,0x48,0x44,0x9d,0x39,0x87,0x16,0x75,0xe8,0x14,0x6a, -0xa5,0xc9,0x58,0x8c,0x80,0xc1,0x7a,0xde,0x9f,0xc9,0x78,0x8a, -0x51,0xe6,0x5e,0x22,0x33,0x44,0xd8,0x9b,0x0a,0x75,0xd5,0x2f, -0x5e,0x7f,0x1c,0xbf,0x21,0x9b,0x3c,0x9f,0x1d,0x80,0x1f,0xaf, -0xc6,0x7d,0xf0,0x00,0xfa,0x9c,0x3d,0xa8,0x8a,0x12,0x8c,0x13, -0x55,0x02,0x38,0xec,0xdf,0xc8,0x44,0xb0,0x5b,0x4d,0x09,0x6f, -0xf1,0xd5,0x89,0x4c,0xa3,0x77,0x28,0x5e,0xf2,0x5b,0x30,0x7f, -0x5d,0x86,0xc1,0x68,0x43,0x01,0x1a,0x7d,0xf6,0x08,0xc0,0x18, -0x4e,0x7e,0xa6,0xcf,0x4b,0x9f,0x65,0x2a,0xbc,0x0c,0x34,0xd4, -0x1f,0x4d,0xae,0xfd,0x46,0xd3,0x61,0x4e,0x7a,0x41,0xfe,0x72, -0x64,0x2a,0xbb,0xd0,0x61,0x00,0x70,0x7b,0x5b,0x01,0x34,0x72, -0xd1,0x06,0x13,0x3f,0x92,0xa4,0xee,0xe7,0x87,0xf7,0x9c,0xf6, -0xbf,0x87,0x9f,0x0b,0xd3,0xfd,0x60,0x1c,0x97,0xa3,0x7f,0xd0, -0x45,0x65,0x76,0x10,0xe0,0x88,0x57,0xa7,0xd5,0xb6,0x2c,0xf5, -0xf7,0x46,0x3e,0x86,0xc9,0x70,0x3e,0x9c,0x5c,0x6c,0x9c,0x76, -0x74,0x04,0xbe,0x4a,0xf2,0x63,0xe9,0x75,0xcf,0x29,0xd8,0x06, -0xbf,0x19,0x0e,0x40,0xca,0x64,0x8e,0xf8,0x60,0x63,0x74,0xe3, -0x14,0x71,0x2e,0xdb,0x91,0x75,0x17,0x1a,0x6d,0x66,0x30,0xda, -0x74,0xe6,0xe1,0xa7,0x9b,0xc9,0x74,0x56,0x74,0xe2,0xaa,0x44, -0x32,0x47,0xbc,0xb5,0xc5,0x66,0x89,0x19,0x16,0x1e,0x9a,0xd0, -0x57,0xe6,0x27,0x64,0xa7,0x89,0xe2,0xcf,0xe6,0xa9,0x0e,0xa4, -0x34,0x61,0xb2,0x53,0x75,0xda,0xfc,0xc7,0xb3,0xb7,0x47,0x2f, -0x5e,0xbf,0xca,0xa2,0x56,0xf3,0x9b,0xe6,0x5e,0x84,0xac,0x03, -0xe8,0x98,0x65,0xa4,0x6a,0xa6,0x7f,0x8c,0xbb,0x5e,0xd9,0x10, -0xe0,0x2d,0x70,0xda,0x67,0x4b,0x2c,0xe4,0x96,0x2b,0xbf,0x40, -0x69,0xe9,0x26,0xd4,0x32,0x81,0x80,0x1c,0xa9,0x5b,0x5b,0x3a, -0xd6,0x62,0xca,0x32,0x2f,0x31,0x11,0x69,0x71,0x09,0xb1,0xf5, -0xa8,0x05,0x50,0xec,0x05,0x2d,0xcb,0x1a,0x2e,0xe4,0x9e,0xbd, -0xe6,0xd9,0xae,0x62,0x08,0x97,0xdc,0xa6,0x9f,0xfd,0x1c,0xbd, -0x16,0x6a,0x3c,0xa6,0x04,0xd6,0x26,0xa5,0x62,0x14,0xba,0x33, -0xec,0x29,0x74,0x87,0x08,0xf6,0xfc,0x78,0x31,0xe1,0x67,0x4f, -0xa4,0x1b,0x6e,0xca,0x81,0x31,0x93,0xb1,0x5d,0x65,0x00,0x57, -0xa2,0x60,0x88,0xfc,0xf2,0x92,0x21,0x17,0xf8,0x51,0x54,0xfc, -0x51,0x43,0x45,0x1b,0x2f,0x09,0x32,0xb0,0x03,0x81,0x4b,0xef, -0x5d,0x7d,0xc2,0xa2,0x3f,0xb4,0xc9,0xea,0xdc,0xa6,0x40,0x37, -0x65,0xd0,0x42,0x1d,0x4e,0x4b,0xed,0xa2,0x41,0xfd,0x04,0x65, -0x98,0x3e,0xd2,0x91,0x1d,0x8e,0x2b,0xf7,0x4d,0x1a,0xa3,0x3a, -0x22,0x14,0x2e,0x35,0x55,0xe7,0x43,0xf2,0x61,0xa5,0x46,0x39, -0x19,0xf0,0xc0,0x52,0xe8,0xe5,0x71,0x75,0x1b,0x94,0x32,0xb1, -0x48,0x1a,0xf2,0xf1,0xfb,0x3c,0x5a,0x83,0xd0,0x6e,0x4b,0x16, -0xd1,0x3a,0x05,0x3e,0xa6,0x86,0x60,0x40,0x7a,0xa3,0x0f,0xb3, -0x1b,0x79,0xad,0x8d,0xf1,0x64,0x83,0x6d,0xb8,0x91,0x01,0xe9, -0xa8,0x6d,0xde,0xda,0xe1,0x58,0x1e,0x0d,0xd0,0x1e,0x31,0xb9, -0x46,0xae,0x6e,0xbe,0x6b,0x7d,0xc8,0xf6,0xdb,0x90,0x11,0x96, -0x4c,0x3e,0xec,0xef,0xb5,0x79,0xd8,0x63,0x7b,0xea,0x09,0x08, -0xce,0x34,0xc1,0xac,0x82,0x5c,0x4e,0xa9,0x51,0x99,0x06,0xd3, -0xae,0x94,0x94,0x38,0x95,0x33,0x28,0x0d,0x2c,0xae,0x31,0x15, -0x66,0xcb,0x2d,0xb5,0x39,0xb8,0xbc,0x40,0x6a,0x3a,0x0e,0x6b, -0xec,0xd1,0x9e,0xa4,0x15,0x29,0xf4,0xfc,0x6f,0x8d,0xde,0x32, -0xd8,0x01,0x53,0x56,0xdd,0x5b,0xd8,0x40,0xa5,0x9c,0x69,0x8c, -0x59,0x4c,0x3c,0xb5,0x18,0xdc,0x17,0x80,0x2b,0x28,0x0a,0x0a, -0x9b,0x94,0x60,0x98,0x85,0xec,0x52,0x8a,0x8e,0xb5,0x8c,0x2d, -0x60,0x5a,0xaa,0x53,0xb9,0xf1,0x70,0xef,0x4d,0xff,0x9b,0x3a, -0x0d,0xcb,0x94,0x3d,0x47,0x44,0xf8,0xdd,0x87,0x71,0x7f,0x51, -0x1f,0x22,0xae,0xfa,0x8e,0xf4,0x93,0xe9,0x88,0x57,0x5e,0xeb, -0x74,0xc5,0xe8,0x6b,0x53,0x2f,0x91,0x66,0x79,0xac,0x5a,0xb2, -0xda,0xe5,0xe5,0x6a,0xb9,0xf6,0x00,0x01,0xf7,0x76,0x90,0xb1, -0x83,0xab,0xd5,0xf6,0x36,0xe5,0xef,0xa5,0xe6,0x4b,0xad,0x3b, -0x86,0x70,0xd1,0x05,0x74,0xbd,0x8a,0x81,0xe4,0x25,0xfe,0x25, -0xa3,0x09,0x0c,0xbb,0xe2,0xd1,0x76,0xf6,0x45,0x6b,0xa7,0xb5, -0xb3,0x36,0xfe,0x7c,0xf7,0x52,0x47,0xa2,0xed,0xf5,0x16,0xb9, -0x4e,0x4c,0x05,0xf0,0x9c,0x01,0x47,0xdd,0xc4,0x1e,0xd8,0xc5, -0x8f,0xa8,0x46,0x56,0xeb,0xb4,0x49,0x66,0x5b,0xff,0xda,0x35, -0x9f,0xf8,0x38,0xde,0x9e,0xa9,0x20,0xbb,0x3b,0x63,0xb0,0xe3, -0x52,0xbe,0x76,0xfd,0x5f,0xb3,0x35,0xb6,0xb6,0xbc,0x8d,0xc2, -0x7a,0x83,0xb5,0xdb,0x00,0xae,0x07,0x2b,0xd6,0x6d,0xdb,0x04, -0x56,0x54,0xc4,0x74,0xaf,0xd5,0x69,0x9b,0x6b,0x92,0xb9,0xb4, -0xe5,0x00,0x41,0xc8,0xd7,0xea,0xbe,0x81,0x53,0xdf,0x5d,0x0c, -0x12,0xc7,0x2c,0x16,0x72,0x81,0x80,0x2b,0x92,0x59,0x3e,0xfb, -0x6c,0xee,0x0a,0x71,0x3c,0x38,0x6f,0xca,0x61,0xbf,0xa5,0x82, -0xbf,0xa6,0x5e,0x23,0x0a,0x4c,0xa7,0x61,0x42,0xc7,0xc6,0x7f, -0xfd,0x96,0x0d,0xa5,0x9b,0xe1,0x49,0x3f,0x5b,0x5b,0xeb,0xee, -0xdd,0x1d,0x1a,0xc5,0x29,0x73,0x2f,0x30,0x65,0x37,0x09,0x87, -0x71,0x49,0x30,0x86,0x36,0x03,0xbe,0x1f,0x9a,0x6e,0xfc,0xf9, -0xaf,0x6c,0xba,0xd6,0x1d,0x4d,0x07,0xa7,0x0f,0x6a,0x39,0x2c, -0xdf,0x34,0x1c,0x44,0x77,0x4c,0xec,0x9f,0x6b,0x36,0x26,0x45, -0x13,0xcb,0x34,0xad,0xdd,0x6e,0xdc,0x60,0x6e,0x15,0x10,0xed, -0xe6,0x8c,0x7f,0x21,0xff,0x72,0x3e,0x9a,0x0f,0x86,0x7e,0xbb, -0x19,0xf7,0x36,0xa5,0xfa,0x6f,0xb6,0x52,0xef,0x78,0x45,0xd5, -0xe7,0x83,0x95,0x6d,0x01,0x4e,0xec,0x6c,0xb7,0x90,0x87,0xb7, -0xbe,0x4b,0x19,0x6d,0x5a,0xbd,0xfc,0xf9,0x9e,0x31,0xb3,0x8c, -0xdd,0xd8,0x2c,0x71,0x76,0xe4,0xe3,0xdf,0x27,0x1f,0x02,0x2a, -0xe5,0xb3,0xb1,0x3e,0x3c,0x0f,0x13,0x4f,0xbb,0xec,0x56,0xad, -0xf6,0xe0,0x7a,0x11,0xce,0x62,0x68,0x18,0xc8,0xca,0xcc,0x38, -0xb3,0xb3,0xa2,0x00,0x6c,0x69,0x3d,0x51,0x31,0x65,0x65,0x33, -0xf6,0xe4,0xb6,0xdd,0x3c,0x15,0x4b,0xdc,0x33,0x31,0xdd,0x0d, -0x40,0x49,0x42,0x44,0xdf,0x8c,0xe6,0xf2,0xce,0x15,0xb0,0xa3, -0x9f,0x94,0x35,0xcb,0xa4,0x86,0x20,0xcf,0xed,0x88,0xef,0xe3, -0xd5,0x70,0x1a,0xb4,0x01,0x3b,0xab,0xcc,0xa7,0xc5,0xcc,0xb3, -0xab,0x45,0xde,0x50,0x30,0x35,0xe9,0x60,0xb2,0x99,0x4a,0xdd, -0x5e,0x7a,0xda,0xa5,0x98,0x08,0xb6,0xb1,0x28,0x8d,0x68,0x71, -0x8b,0x7a,0x95,0xb4,0x88,0xa3,0x8a,0x66,0x84,0xc8,0x41,0x38, -0x1c,0x41,0xc8,0xfd,0x36,0x10,0xa7,0x39,0x04,0x47,0x29,0xf3, -0x30,0x76,0xf2,0x6c,0xee,0x12,0xed,0x50,0xd4,0xfb,0x1a,0xfa, -0x05,0xe5,0x58,0x45,0x51,0x33,0xd8,0xa7,0x15,0x34,0xd4,0xa7, -0x35,0x4e,0x8d,0x76,0xe1,0x05,0x23,0x56,0xc6,0x42,0x1e,0xf0, -0x28,0x38,0x4c,0xbb,0xbb,0x3d,0x3e,0xee,0xe9,0x2f,0x1a,0xb8, -0xc4,0x64,0x3c,0xfe,0xf6,0x9b,0x6f,0x1e,0x7e,0x63,0x98,0xaf, -0x43,0x30,0x74,0xa0,0xcb,0xe3,0x4e,0x85,0x20,0x9e,0xa1,0xda, -0xe5,0x02,0xa8,0x91,0x21,0x91,0x33,0x6f,0xb7,0x76,0x76,0xe5, -0x12,0x72,0x0b,0xfe,0xc1,0xe6,0xb3,0xe1,0x20,0x85,0x14,0x9a, -0xa7,0xf8,0x69,0x5f,0x9d,0xae,0xb3,0x16,0xd0,0x8b,0x1d,0x42, -0x64,0x57,0x80,0xce,0x9a,0x4b,0x01,0x0d,0xd5,0xb6,0xc9,0xbf, -0x4f,0x94,0x35,0x4d,0x58,0xf3,0x98,0x86,0x56,0xa2,0x8e,0x10, -0x58,0xba,0xcd,0xc7,0xd2,0xed,0x9f,0xf4,0xe0,0x0d,0x7b,0x25, -0x1f,0xff,0xf7,0xf6,0x4a,0x3e,0xfe,0x9a,0x5e,0xd9,0xa8,0xed, -0x15,0xd7,0x29,0xff,0x37,0xfb,0xe4,0xf1,0x5f,0xd9,0x27,0xc5, -0xd5,0xfc,0xe2,0x62,0x34,0x0c,0x84,0x4d,0x58,0xfb,0xfe,0x78, -0x40,0x54,0x64,0xbb,0x8a,0xc1,0x06,0x82,0xb1,0xab,0xa9,0xb6, -0x9e,0x9c,0x7d,0x64,0x76,0xe1,0x67,0x72,0xcd,0x1e,0xc0,0x1b, -0x89,0x45,0x41,0xac,0xf5,0x76,0xab,0x97,0xd9,0x18,0x00,0xed, -0x29,0x3f,0x48,0x6b,0xc4,0xd2,0xe5,0x43,0x6a,0xfb,0xd7,0x37, -0x01,0xb1,0x6a,0xac,0x2e,0xe7,0xfd,0xa9,0xb3,0xf5,0x1d,0x9e, -0xba,0x1e,0xef,0x2d,0x16,0x08,0x01,0x9b,0x6c,0xd7,0xd2,0xe5, -0x46,0xce,0x76,0x2b,0xd1,0x6b,0x9c,0x69,0x09,0x6c,0x00,0x12, -0xa7,0xc6,0xbb,0xca,0xcc,0x6e,0x70,0xd8,0x63,0xdc,0x1a,0x8d, -0x26,0x93,0x0f,0xf3,0x9b,0x17,0x86,0x1d,0xa8,0x59,0x8c,0xbd, -0xdd,0x83,0x92,0x3a,0xd4,0x3d,0x75,0x72,0x3d,0xf2,0x59,0xd5, -0x5b,0xc2,0x8e,0x5b,0x4c,0xa6,0xb3,0x1f,0x02,0xee,0xc4,0x74, -0xab,0x3c,0x69,0x1a,0x22,0x7c,0x9a,0xb8,0x3c,0xb7,0x59,0xe1, -0xce,0x12,0xd7,0x6d,0x20,0x15,0x9c,0x84,0x37,0x92,0x30,0x39, -0x65,0xbf,0xff,0xe8,0x01,0x2d,0xef,0xa7,0xeb,0x32,0xad,0xcb, -0x04,0x2b,0xe3,0x14,0x5d,0x47,0xc3,0x8b,0x99,0x9a,0xe2,0xf9, -0x95,0xb6,0xe2,0x0c,0x62,0x9a,0x06,0xb1,0x3a,0xcb,0x30,0xd1, -0x46,0xc0,0x94,0xed,0xeb,0x3d,0xe3,0x0c,0x57,0x87,0xfe,0xfe, -0xd9,0x62,0x41,0x5b,0x13,0x58,0xc2,0xb4,0x53,0x16,0xa0,0xce, -0xf6,0xfb,0x22,0xed,0xcc,0x2e,0xb2,0xe6,0x4e,0x00,0xcb,0xa1, -0xd1,0x47,0x45,0xe0,0xf7,0x7f,0xe8,0x31,0x16,0x91,0xc0,0x86, -0x3b,0x18,0xbd,0x82,0xba,0xb6,0x3f,0x1b,0x5e,0xf5,0x7f,0xcf, -0x85,0x51,0xde,0xbb,0x7a,0xc5,0xcc,0xc6,0xa5,0x5d,0xe0,0x32, -0x62,0x89,0x10,0xb6,0xe3,0xf8,0xc7,0x74,0x65,0xb7,0xad,0x5c, -0x57,0xec,0xe1,0x7a,0x0d,0x1e,0x0f,0x17,0x3e,0x53,0x0b,0x5e, -0x2b,0x80,0xb3,0x50,0xc1,0x49,0x07,0x47,0x1e,0x56,0x5e,0x0f, -0x3d,0xfc,0xf5,0xd4,0x93,0xfd,0x5c,0xb7,0x31,0xf9,0x02,0x71, -0x09,0x09,0xb3,0xbb,0xb8,0xaf,0xa7,0xe2,0x1b,0x44,0x1c,0xc1, -0xd1,0xea,0x94,0xda,0x7e,0x9d,0x72,0x24,0x22,0xb3,0x26,0x00, -0x4b,0x3a,0x1f,0xcf,0x56,0x65,0x87,0x27,0xcb,0x2b,0xe8,0x6b, -0x34,0x3c,0x0a,0x5b,0x88,0x14,0x06,0xea,0x70,0x80,0xec,0xa8, -0xeb,0x7e,0x94,0xf0,0xa9,0xd5,0xc7,0x03,0xee,0x5f,0xbb,0xb0, -0xd7,0x77,0xb1,0x93,0xa3,0x18,0x46,0x89,0xd8,0xd7,0x5a,0x09, -0x2d,0xbc,0x99,0xf9,0xa8,0x57,0xe0,0x2b,0x3d,0x62,0x33,0xa4, -0xc5,0x88,0x5c,0x20,0x66,0x5f,0x27,0xb6,0xd9,0x31,0x53,0x3e, -0xd0,0xcb,0xc0,0xc7,0x06,0x46,0xef,0xef,0xeb,0x19,0x51,0x89, -0x12,0x51,0x74,0x35,0x70,0x2f,0x79,0x4c,0xe3,0x11,0xf0,0xeb, -0x70,0xa3,0x95,0x62,0x19,0xfa,0xd3,0x4e,0x98,0xc9,0x47,0x58, -0x75,0x67,0x93,0xc0,0xec,0x63,0xb0,0x70,0x75,0xfc,0x7d,0xba, -0x23,0xf8,0x6a,0x08,0xa7,0x75,0xf2,0xf3,0x8e,0x5b,0x90,0xc4, -0x99,0x2a,0xb5,0x4f,0xe4,0x31,0x73,0xb7,0x87,0x0b,0xbf,0xe7, -0x96,0x5b,0x96,0xef,0x0e,0x1b,0xbb,0xf5,0x05,0xb9,0xcf,0x54, -0x88,0xa2,0x8c,0xf8,0x09,0xd9,0x4d,0xcd,0xe0,0xea,0x4d,0x0b, -0x2e,0xd3,0xf5,0xcf,0xac,0x2f,0x0f,0x0f,0x34,0x02,0xc2,0x7d, -0x86,0x0a,0xc6,0x34,0xc3,0x2b,0x53,0xd4,0xd8,0xec,0x37,0xd4, -0xd4,0xbb,0xbd,0xd4,0x3b,0x68,0x00,0x2e,0xd8,0x4d,0xe8,0x98, -0x42,0xe2,0xa5,0x3b,0x8a,0xaa,0xc8,0x2f,0x47,0xc2,0x76,0x1c, -0x14,0xac,0x4f,0x5d,0x09,0xe2,0x87,0xdb,0xa9,0xbf,0xaa,0x1e, -0x5e,0x89,0xad,0x8a,0x4a,0xd9,0xbd,0xd2,0x83,0x1c,0xab,0xdd, -0x84,0x25,0x28,0xd8,0xc2,0xfa,0x20,0x09,0x72,0xf0,0xc1,0x74, -0x72,0xf3,0xc5,0xd5,0xae,0xa8,0xe6,0x12,0xd7,0x02,0xf4,0xaa, -0x1b,0xa0,0x2b,0x4b,0x63,0x08,0x89,0x18,0x6a,0xb4,0xdc,0x5f, -0x8c,0xc0,0x95,0xe0,0x58,0x1a,0x62,0xd7,0xfc,0x12,0xf8,0xc9, -0x1d,0xe9,0x09,0x00,0x7e,0xea,0x75,0xd0,0x11,0x45,0xd1,0x68, -0xf7,0x96,0x04,0x1d,0x08,0x6d,0xa7,0x00,0xda,0xd6,0x3e,0xef, -0xcf,0x98,0xf3,0xa4,0xdc,0x0a,0x81,0xcc,0x59,0x9e,0x72,0x84, -0x7c,0x43,0x60,0xa4,0x3b,0x59,0x2c,0x28,0x86,0xb9,0x19,0xc3, -0x43,0x70,0xf9,0x1d,0x58,0x4f,0xfd,0x32,0x08,0x20,0xe5,0xea, -0xf0,0x66,0x11,0xd4,0x22,0xa5,0xdf,0x60,0x35,0xe6,0x44,0x58, -0xfe,0x4b,0x8d,0x41,0x8d,0xc6,0x68,0xdc,0x16,0xc8,0x85,0x78, -0xa9,0xe0,0xb0,0x14,0xcf,0x9d,0xb9,0x3e,0xe7,0xce,0x57,0xf4, -0xc7,0x20,0xbf,0xb8,0xd0,0xe7,0xb6,0xb1,0x66,0xb1,0x18,0x41, -0xd5,0x49,0xbc,0x45,0xe3,0x66,0x3e,0xce,0x7f,0x23,0x63,0x54, -0xbf,0xcd,0x4b,0x93,0x32,0x2f,0x8e,0x70,0xcd,0xae,0x58,0x9b, -0x3d,0x06,0xcc,0xc0,0xb1,0x71,0x72,0x80,0xb0,0x4b,0xae,0xdb, -0xa5,0x1d,0x36,0xfe,0xc8,0x36,0x5b,0x24,0xe3,0x31,0x13,0x55, -0xac,0xf0,0xb0,0x74,0x31,0x11,0x25,0x99,0x0d,0xc5,0x3b,0x61, -0xa8,0x2a,0x86,0xba,0x4d,0x03,0xde,0x99,0x71,0xd6,0x6c,0xef, -0x96,0xe4,0x0e,0x46,0x6c,0x6d,0x01,0x8a,0x2e,0xfc,0x71,0x73, -0x30,0x63,0x96,0x22,0x95,0xee,0x39,0x35,0x04,0x0f,0x05,0x3d, -0x84,0x30,0x28,0x7b,0x5b,0x79,0xa2,0x4e,0x9a,0xd9,0x24,0xeb, -0x96,0x37,0x5f,0xdc,0xee,0x52,0x79,0x4b,0x74,0x1f,0x74,0x45, -0x6c,0x87,0x8a,0xe8,0xb1,0x4d,0x9e,0xea,0xe8,0x9d,0xa0,0x08, -0x2d,0x6b,0xf3,0x28,0x60,0x0e,0x69,0x56,0x2d,0x7e,0x69,0x09, -0xae,0x95,0xa7,0x2d,0x17,0x48,0xd9,0x5d,0x6b,0xe9,0x56,0x97, -0xca,0x79,0x34,0x19,0x01,0xb1,0x30,0x5d,0x32,0xbb,0x1a,0x4e, -0x25,0x47,0xce,0x82,0x25,0x8c,0x57,0x88,0x60,0x3f,0xdb,0x5d, -0xb2,0xf8,0xc5,0x8d,0xca,0x7a,0x9a,0xbd,0xb9,0xed,0xd4,0x00, -0x6a,0x07,0x70,0xa9,0x32,0x34,0x32,0xaa,0xcf,0x0c,0xd2,0xcc, -0x3e,0x56,0xc4,0x4c,0x50,0xa0,0xee,0x8f,0xfc,0x46,0x76,0x86, -0xe1,0x1e,0x78,0xbb,0x43,0x91,0x47,0x6c,0x18,0x7d,0x47,0x42, -0x44,0xe9,0x51,0xc2,0x8a,0x20,0xb1,0xee,0x21,0x3b,0x2e,0xe1, -0xfe,0x82,0x56,0x1c,0xbe,0xb0,0x81,0xeb,0x64,0xff,0xee,0x98, -0x41,0xbb,0x39,0x3c,0xc7,0x2d,0x21,0x8f,0x1a,0xb9,0x34,0xfa, -0x61,0x46,0xce,0xe4,0xcc,0x97,0x95,0xc3,0x41,0xc0,0x7a,0xc9, -0xb3,0x37,0x60,0x78,0x3c,0xe0,0xd3,0xc6,0xd2,0x72,0x43,0x82, -0x77,0x76,0x57,0xdb,0x00,0x5a,0x79,0xb5,0x4d,0x48,0x0d,0x5f, -0x07,0x60,0x70,0x97,0xcd,0xee,0xf7,0xf4,0x67,0xea,0xa7,0xe8, -0xbd,0xb8,0x97,0x99,0xef,0x56,0xcf,0x27,0x7d,0xa9,0xec,0x00, -0x29,0xad,0x32,0x33,0x34,0x2e,0xce,0x4b,0x88,0xab,0x02,0x8d, -0x0e,0x73,0xcc,0x68,0x07,0x9a,0x00,0x1e,0xb3,0x06,0x97,0xf0, -0x12,0x41,0x34,0x9e,0x5f,0x9f,0x0d,0xa7,0x4e,0xb3,0xc2,0xa6, -0x9a,0x9b,0x9c,0xcc,0xe3,0x47,0x05,0x25,0x09,0xf3,0x70,0x39, -0xcc,0x7f,0x88,0xe8,0xe4,0xa9,0x2e,0x5d,0x97,0xbc,0x6f,0xd7, -0x0c,0x71,0x6e,0x25,0x02,0x1a,0x16,0x7f,0x6a,0xbe,0x96,0x56, -0x57,0xc2,0x4a,0x67,0x89,0xe4,0x1a,0xf9,0x6c,0x62,0x0f,0xd8, -0x02,0x28,0xf6,0xdb,0x06,0xfb,0xb1,0xa4,0x7e,0x10,0xd0,0x6b, -0xab,0xd8,0xb6,0x27,0x34,0xe6,0x55,0x5e,0xac,0x68,0x7f,0xb0, -0x34,0xb3,0xba,0xed,0x35,0xbb,0x4f,0xbc,0x3b,0xa9,0x83,0x40, -0x86,0xb6,0xad,0xa3,0xd0,0xdf,0x31,0xf5,0x94,0x25,0x9a,0xba, -0x0a,0x30,0x43,0xa6,0x41,0xdb,0x29,0xe5,0x8a,0x1d,0x61,0x69, -0x75,0xa2,0x63,0xf3,0xf3,0xcc,0xe2,0x81,0x0c,0xa9,0x3f,0x3a, -0xb6,0xb7,0xdb,0x6b,0x37,0xd4,0x14,0x0c,0xde,0x4b,0x3f,0xc9, -0xe8,0xc4,0x7f,0x36,0xb9,0xd1,0x7f,0x86,0x37,0xc9,0x6d,0x49, -0x00,0x92,0xb5,0xf4,0xae,0x07,0x00,0x19,0xc2,0x2e,0x16,0xbb, -0xe4,0xf9,0x3f,0xdb,0x4d,0x30,0x8b,0xbb,0xa8,0xee,0xee,0xf5, -0x16,0x8b,0x56,0x3b,0x58,0x5c,0xec,0x60,0xc2,0x8f,0xf3,0x61, -0x3e,0x8a,0x11,0xdd,0x36,0x62,0x49,0x76,0xb0,0x58,0xcd,0xe6, -0xa9,0x7c,0x00,0xb2,0x22,0xa2,0xaf,0xb4,0xb8,0xd8,0x51,0x31, -0xf8,0xd4,0x4e,0x10,0x06,0xde,0x8b,0x37,0x1a,0xbd,0xcc,0xd4, -0x40,0xff,0x6d,0x64,0x80,0xcc,0x4e,0x4c,0x80,0x22,0x26,0xed, -0xbc,0x64,0xf8,0x9e,0xae,0xb8,0xa5,0xb5,0xfd,0xf1,0xb9,0x7f, -0x0e,0x07,0xd1,0xba,0x3a,0xd3,0x07,0xc5,0x81,0x1b,0x06,0xa0, -0x6a,0xb5,0xb5,0x05,0xb0,0x94,0xdd,0xf4,0x3b,0xc4,0x9b,0xe6, -0x76,0x31,0xbc,0xd0,0x23,0xea,0xda,0x25,0x9e,0xfc,0x62,0x0a, -0x66,0x03,0xc0,0x93,0xaa,0x3b,0xe2,0xb6,0x13,0x51,0xd5,0xdf, -0x3e,0x20,0xc1,0xb2,0xaa,0x80,0x3f,0x0e,0x55,0xbc,0x10,0xca, -0x4a,0x2c,0xb1,0x3e,0x44,0xab,0x3b,0xe7,0x5d,0x16,0x66,0xd1, -0xaf,0x2a,0x2a,0xd1,0x94,0x9f,0x7b,0x46,0xf2,0xb1,0x4c,0x17, -0x6c,0x93,0xfe,0xed,0xe8,0x82,0xd4,0x6d,0x35,0x68,0x08,0x8f, -0x36,0x28,0xc5,0xa2,0x2d,0xa8,0x80,0x7c,0x6b,0x91,0xc0,0x15, -0x20,0x9d,0x38,0x3e,0xac,0x83,0xfe,0x11,0x7d,0x99,0x0b,0x34, -0x40,0xb7,0xc4,0x2b,0x12,0x3d,0x46,0xbc,0x03,0x13,0xb6,0xf5, -0x1d,0x57,0x3a,0x8e,0xa7,0x28,0x73,0x33,0x82,0x66,0x54,0xf2, -0x5c,0x83,0x66,0x24,0x04,0x06,0xcf,0x93,0xe0,0xea,0x95,0xe8, -0x80,0x98,0x7a,0x42,0x74,0x67,0x82,0x70,0x0a,0x81,0x8c,0x1a, -0x59,0xc9,0x5f,0x2a,0x23,0xdf,0xb8,0x9e,0x17,0xb3,0x8d,0xb3, -0xe1,0xc6,0x4d,0x1f,0x1e,0xe3,0x5a,0xea,0x37,0x40,0xbb,0xbc, -0x88,0x7c,0x41,0x11,0x62,0x74,0x8c,0xc5,0x45,0x72,0x0b,0x22, -0xc5,0x0b,0x67,0x20,0x04,0x43,0x28,0x51,0x00,0x8e,0xff,0xec, -0x57,0x94,0xc8,0x0f,0xaf,0x27,0xde,0xc9,0x1a,0x47,0xba,0x5e, -0xa8,0xae,0x8c,0x6e,0x38,0x2a,0x44,0x38,0xab,0x97,0x94,0xa4, -0xf9,0x4a,0xfa,0x90,0x77,0xa1,0xa1,0xe2,0x2c,0xc8,0xa6,0x08, -0x2a,0x68,0x5f,0x6e,0x48,0xc7,0x1a,0x81,0x98,0x06,0xb5,0x32, -0x50,0x48,0x03,0x5f,0x24,0x42,0xb2,0x5f,0x59,0x45,0x2f,0x31, -0x16,0xec,0x8c,0xc1,0x70,0x24,0xc5,0x13,0x58,0x89,0x8f,0xfd, -0x7c,0x76,0xf7,0x5d,0x9f,0x21,0xa2,0x18,0xce,0x8e,0xf3,0xeb, -0xa1,0x3e,0xb5,0xc4,0x2b,0xc7,0x08,0x3e,0xcb,0xa6,0x2b,0x3b, -0x2a,0x81,0x8a,0xbf,0x90,0x2e,0x7e,0x68,0x48,0xba,0xc3,0x0e, -0x10,0xc7,0xd9,0x4f,0x55,0x17,0x89,0x6b,0xf5,0x56,0x8d,0x31, -0x58,0x55,0x10,0x31,0x0c,0x8c,0xd9,0x6c,0x14,0xf6,0x0f,0x14, -0x6c,0xcc,0xdc,0x9b,0xab,0x05,0x37,0xe9,0x99,0xbd,0x53,0x33, -0xaa,0x4f,0xc6,0x2f,0xc9,0x49,0x77,0x59,0xf3,0x77,0x9c,0x51, -0xf7,0x21,0x7f,0x65,0x60,0x0f,0x01,0x97,0xfb,0xfe,0x6c,0xe8, -0x2d,0xb2,0x36,0x17,0x43,0xea,0x01,0xdb,0x1f,0x80,0x16,0x31, -0x3c,0xf1,0xec,0xec,0xa2,0x2a,0xe9,0x81,0xce,0xe4,0x97,0x55, -0x5e,0x0d,0x24,0x79,0xc9,0xb2,0x62,0x32,0xe2,0x2b,0x84,0xc9, -0xc7,0xcc,0x20,0x6c,0x9b,0x92,0x17,0x8b,0xa0,0x68,0x7c,0x5d, -0xaa,0x69,0xb7,0xa4,0xe9,0x6c,0xe6,0x1a,0xfe,0x5a,0x73,0xce, -0x40,0x1d,0xb4,0xcf,0x76,0xac,0x13,0xb6,0x0d,0x94,0xb4,0xb0, -0x86,0x67,0x3f,0x3b,0xd7,0xdd,0xde,0xa7,0x76,0xf7,0x33,0x8b, -0xa3,0x13,0x9f,0xeb,0x02,0xa7,0x66,0x4c,0x70,0xfd,0x92,0x9a, -0x46,0xd5,0x45,0xdd,0x55,0xeb,0x24,0xe5,0xac,0xae,0x46,0xb3, -0x69,0x3f,0x1f,0x99,0xd6,0x04,0xff,0x49,0x8c,0x5b,0x8c,0x45, -0xec,0x11,0x65,0xa9,0x4a,0x9c,0xa4,0x16,0x87,0x1d,0x6c,0x02, -0xe7,0x95,0xa3,0x23,0xbf,0xbe,0x1e,0x0e,0xf2,0xbe,0xb1,0x18, -0xc5,0xa8,0x69,0x7c,0x18,0xba,0x20,0x52,0xef,0x27,0xd7,0x37, -0xca,0xd7,0x36,0x12,0xfd,0xb2,0xaa,0xb9,0x6c,0x76,0xd7,0x6b, -0x95,0x83,0x88,0x22,0xf5,0xd9,0xc9,0x80,0x6d,0xdb,0x9c,0x6d, -0xa0,0x75,0x1f,0x12,0x3b,0xb5,0x95,0xc7,0xce,0x04,0x90,0x24, -0x8d,0xbd,0xd6,0xb7,0x55,0x74,0xea,0x0e,0xb5,0x8d,0xcf,0x8f, -0x8e,0x5f,0xe9,0x21,0x66,0xb3,0x6d,0x6d,0x6d,0x32,0x3e,0x6b, -0xfd,0xd4,0xf4,0x50,0x7d,0x57,0xe0,0x9c,0x4f,0x0c,0x32,0x77, -0xf3,0x56,0x5b,0xb2,0xec,0xb0,0x49,0xa9,0xb3,0xdc,0x1a,0x0b, -0x5c,0x93,0x50,0x21,0x29,0x2f,0x42,0x3a,0x1d,0x97,0xc6,0x34, -0x46,0xc8,0x5d,0xd4,0x63,0x5b,0xb1,0x30,0x92,0x85,0x5a,0x6c, -0x29,0xd4,0xe8,0x43,0x0a,0x40,0x1f,0x3c,0x1c,0x2e,0xa8,0x22, -0x5e,0x71,0xb9,0x21,0x16,0x4f,0x5c,0xb4,0x9c,0xd3,0x1d,0x27, -0x79,0xc2,0xf1,0x24,0xca,0x64,0x64,0xe1,0x86,0x4a,0x85,0x83, -0x98,0x6e,0x22,0x5d,0x55,0xc8,0xfd,0xb2,0x64,0xaa,0xb7,0xe2, -0xac,0x1b,0x0c,0xc0,0xdc,0xdb,0x45,0xe1,0x04,0xa4,0x0f,0xf3, -0xc0,0x4a,0x27,0x8e,0x68,0x38,0x04,0x96,0xa8,0xe9,0x49,0x3e, -0x4c,0x9f,0x09,0x91,0xba,0xfe,0x85,0x37,0x6e,0x71,0x8c,0x2a, -0x6f,0x29,0x2f,0xf5,0xc9,0x36,0x8d,0xe4,0xc7,0xad,0x4e,0x7d, -0x37,0x18,0x8d,0x1a,0x28,0x02,0x55,0x21,0xdd,0x43,0x00,0xe1, -0xd9,0x0f,0x99,0x07,0xd2,0x34,0xd7,0x0b,0x1c,0x33,0x43,0x78, -0x25,0x5d,0xa5,0x69,0x18,0xbd,0x18,0xa3,0xe7,0x74,0x76,0x21, -0x16,0xd1,0xfa,0x87,0xd8,0xbb,0xbd,0x76,0xa0,0x80,0x01,0x58, -0x68,0x8b,0x35,0xfa,0x24,0x5b,0x5b,0xa8,0x7b,0x89,0x02,0x20, -0x08,0x9b,0xd6,0x80,0xd8,0xa5,0x35,0x50,0x1b,0x70,0x36,0xb5, -0x1a,0xe8,0x65,0x75,0x4e,0x3e,0xea,0xaf,0x23,0x5e,0xb0,0x07, -0xf5,0x4c,0xea,0xa2,0xb7,0xa5,0x3e,0x4b,0x41,0x6c,0x5f,0x3e, -0xfd,0x33,0x14,0x51,0xfe,0x35,0x08,0x42,0x40,0xa0,0xa7,0x2b, -0xf4,0xe1,0x2d,0x61,0xc2,0xef,0x94,0x06,0x63,0x3d,0xa3,0xe1, -0x74,0x56,0x43,0x99,0x93,0x65,0x7c,0x95,0xde,0x3e,0xcb,0x2e, -0x3c,0x02,0x32,0xfe,0x2a,0x8b,0x2e,0x0c,0x09,0x50,0x0e,0x29, -0x1a,0x85,0x0d,0x86,0x9b,0x2c,0x70,0x8e,0xb5,0xa3,0x44,0x9c, -0x61,0xb8,0xd4,0x9e,0x1e,0x2d,0x98,0xa7,0x3c,0x5c,0x28,0x1a, -0xef,0x76,0x49,0x21,0x11,0x8d,0x23,0x54,0x5f,0xd2,0x20,0x87, -0x5a,0xc3,0xea,0x38,0xbe,0x91,0x1e,0xf1,0xe3,0x44,0xe0,0x4f, -0xeb,0x56,0x6c,0x3a,0xb9,0x01,0x32,0x39,0x1a,0x88,0x23,0x1f, -0xe8,0x14,0x61,0xfd,0x9e,0x33,0x5b,0x0b,0x7e,0x04,0x2a,0x2a, -0x8f,0x4f,0xf7,0x4c,0x6f,0x7c,0xad,0x24,0x0f,0xab,0x82,0x6f, -0x77,0x6c,0x01,0x78,0xdb,0xe8,0x1a,0x12,0x45,0xcc,0x37,0x9f, -0x89,0x73,0xb5,0xed,0x88,0x7e,0xec,0x6e,0x3e,0xe3,0x3e,0x70, -0x9d,0x87,0x43,0xe6,0xcf,0x93,0x57,0xd9,0xa1,0x56,0x9c,0x88, -0x04,0x03,0x9d,0x7a,0x73,0xab,0x20,0xce,0xb1,0x47,0x44,0xa1, -0x79,0x83,0xfb,0xdf,0xd2,0x9b,0xf6,0xc6,0xde,0x76,0x2b,0x69, -0xc6,0x57,0x74,0xb1,0x38,0xba,0xa0,0xbf,0x8f,0x6a,0xfa,0x60, -0x20,0x8b,0x35,0x34,0xbc,0x9e,0x44,0x5d,0x4b,0xd4,0xef,0x80, -0xbb,0x46,0x67,0x66,0xc4,0xdc,0x55,0x22,0xe7,0x2d,0xf7,0x49, -0xbc,0xfb,0xad,0x70,0xce,0x2b,0xbd,0x15,0xe3,0xac,0x86,0xcc, -0xf4,0x8e,0xe9,0x37,0x21,0xc2,0x52,0x67,0xaa,0x7f,0x34,0x03, -0x7b,0x11,0x67,0xf8,0x43,0x7a,0x0c,0x42,0x43,0x61,0x63,0x57, -0x2f,0xfa,0xfd,0xc5,0xa2,0xb5,0xa3,0x63,0x5b,0x3b,0x67,0xee, -0x95,0x80,0x8e,0xa4,0x0f,0x0b,0x8a,0xf9,0xda,0x7d,0xef,0xb5, -0x97,0x6e,0xae,0x7e,0xd6,0xb7,0xaf,0xbd,0x12,0x75,0x16,0x26, -0x9f,0x65,0x67,0x2e,0x99,0xe4,0x29,0x9a,0xa5,0x2a,0x5e,0x81, -0xc1,0x6d,0xf3,0x10,0x90,0xfb,0x12,0x8f,0xa8,0x36,0x75,0x33, -0x48,0x3e,0x4b,0x9c,0xc2,0x5e,0xf1,0x31,0x9f,0xe9,0x81,0x60, -0x61,0x35,0xbb,0xd8,0x2f,0x86,0x51,0x97,0x76,0xa5,0x0d,0xca, -0xd6,0x8b,0x52,0x47,0x39,0xdb,0x46,0x3c,0x03,0x63,0xde,0x02, -0xf2,0x15,0x8a,0x42,0x05,0xe4,0x66,0xd6,0xe8,0x77,0xce,0xf4, -0xdf,0xb3,0x54,0x8f,0x8b,0x7e,0xc7,0x34,0x4c,0xaa,0x7f,0x1a, -0x67,0x7e,0x66,0xe0,0x28,0x75,0x56,0x2f,0x8e,0x1d,0xd2,0x5a, -0x8c,0x8d,0x8a,0x7c,0x6f,0x87,0x97,0xcf,0x3e,0xdd,0x88,0x42, -0x9b,0xec,0x86,0x44,0x37,0x15,0x7d,0x6d,0x6d,0xf5,0x9b,0xf4, -0x50,0x14,0xe2,0xe8,0x0b,0xe2,0xae,0xf5,0xcc,0x00,0xc6,0x1d, -0x41,0x6d,0x00,0x52,0xf2,0xcb,0xf1,0x64,0x3a,0x04,0x7b,0xe5, -0x90,0xe4,0x42,0x20,0x69,0x8d,0x78,0xb3,0xb6,0x02,0x5f,0xdd, -0xb7,0xa5,0xb8,0x33,0xd7,0xba,0x81,0x0c,0x8e,0x46,0x90,0xbf, -0x4f,0x18,0xa1,0x21,0x26,0x75,0xf9,0x2d,0x84,0x50,0x8a,0x39, -0x0b,0x12,0xce,0x48,0x05,0xfa,0x29,0x88,0xd2,0xfa,0xb0,0x36, -0x14,0xb3,0xe9,0x1c,0xbd,0x9a,0x9f,0x61,0xdc,0x99,0x8c,0x43, -0xb5,0x1b,0x88,0x06,0xd5,0x9b,0xa7,0xa8,0x2d,0xb7,0x19,0x7b, -0x5b,0x04,0xa6,0xea,0xfd,0x01,0x7f,0xe5,0x88,0xeb,0x3f,0xb5, -0xca,0x75,0x16,0xf8,0x8c,0x81,0xcf,0x42,0x60,0x4a,0x70,0xf5, -0xe6,0x8a,0xd2,0xf5,0x54,0xc2,0x53,0x86,0x82,0x67,0x34,0x74, -0x51,0x17,0x60,0x57,0x39,0x8d,0x75,0x68,0x5d,0xd3,0xad,0x38, -0xcf,0x7b,0x51,0x96,0x89,0x61,0x09,0x2b,0x10,0xe4,0xe9,0x1b, -0x76,0x80,0xb3,0x62,0xa4,0xae,0x35,0x73,0x03,0x28,0xba,0x86, -0xb8,0xed,0x6d,0xc7,0xd8,0x0f,0x7f,0x8b,0xfb,0x5d,0x88,0xec, -0xa9,0x33,0xfe,0xf5,0x27,0x73,0xd2,0x4e,0xda,0xf8,0x48,0xaf, -0xa4,0xf9,0x0a,0x7d,0x43,0x9c,0x57,0x9f,0xf9,0x2e,0x24,0xa3, -0xd1,0x50,0x9b,0x06,0x3b,0x25,0x9f,0x71,0x32,0x96,0x05,0xeb, -0xb1,0x2e,0x0a,0x7f,0x82,0x92,0x92,0x04,0xd5,0xb4,0xa1,0xbe, -0x2c,0x28,0xc3,0x32,0xb9,0xbc,0x33,0x57,0x9e,0x41,0xb8,0x49, -0xb5,0xe1,0x6c,0xa6,0xc1,0x20,0x72,0x69,0xfd,0x79,0x71,0xfb, -0x4e,0x6e,0x62,0xd7,0xd8,0x18,0x60,0xd6,0xa2,0x8d,0x3a,0x92, -0xbf,0xcd,0x3d,0x2d,0x05,0x75,0xe6,0x36,0x83,0xdf,0x70,0x85, -0xeb,0xf6,0xcc,0xed,0x2f,0x6b,0x54,0x66,0x25,0xe6,0xd6,0x6a, -0x6a,0x98,0xce,0xc6,0x9e,0xf3,0x17,0x67,0xba,0xf4,0xe6,0x75, -0x42,0xea,0x64,0xf2,0xfe,0x60,0x26,0x40,0xc5,0x6e,0x67,0x6b, -0x6f,0xf8,0xdc,0x0a,0xd5,0x62,0x22,0x8f,0x9f,0xa7,0x57,0xed, -0x1d,0x9b,0xf1,0xe6,0x04,0x2c,0xb2,0xb5,0x36,0xa9,0xb8,0xf1, -0x64,0x30,0x04,0xc6,0x9b,0x6b,0x46,0x7a,0x30,0xde,0x3b,0xdf, -0x90,0x8b,0x27,0x3c,0xe5,0xf1,0xe8,0xaf,0xa3,0x28,0x86,0x53, -0x6e,0xa3,0xaa,0xd5,0xad,0xc9,0xbc,0xe3,0xc0,0x52,0xb1,0x17, -0x06,0xab,0x16,0x10,0xa9,0xc8,0xcc,0x31,0xfd,0x49,0x45,0xe8, -0x0f,0x5a,0x52,0xf5,0x07,0x2c,0x8f,0xfa,0x87,0x56,0x3b,0xe9, -0xb0,0x61,0x8c,0x53,0xe3,0xb4,0x1b,0xe5,0x45,0xd4,0x20,0x73, -0x15,0x95,0x34,0x94,0xe9,0xce,0x32,0x5b,0x3b,0xca,0xd9,0x88, -0x7a,0xd1,0x92,0x94,0xba,0x84,0xbe,0x82,0x3b,0x03,0x69,0x9e, -0xc3,0x4b,0x5a,0xd9,0xf2,0x9b,0xae,0x13,0x23,0x28,0x72,0x38, -0x8c,0xf0,0x66,0x3a,0x32,0x99,0xec,0x6a,0xb9,0xd3,0xdc,0xd1, -0x53,0x4a,0xae,0x32,0x95,0x88,0x5d,0xc6,0xcc,0x2c,0xb3,0xb0, -0x4f,0x13,0xb9,0xcf,0xe1,0x12,0xbe,0x86,0x8d,0x30,0xa9,0xac, -0x7c,0xbc,0x99,0xeb,0xe5,0xe4,0x55,0x8c,0x56,0xeb,0x9e,0x8f, -0x26,0x7d,0x3e,0xa0,0x51,0x2f,0xea,0x94,0x7a,0x5e,0x84,0x3a, -0xc3,0xe9,0x30,0x6f,0xa2,0x6e,0x12,0x65,0xe4,0xad,0x6a,0x55, -0xff,0x6f,0xee,0x2e,0x16,0xfc,0xd5,0xd2,0x7b,0x46,0x69,0x93, -0xab,0x1f,0x5b,0xe0,0x2d,0x7b,0x95,0xd6,0x54,0x66,0xc9,0xb0, -0x0e,0x90,0xab,0xc1,0x25,0x8b,0x06,0x19,0x74,0x0f,0xd5,0xbf, -0x53,0xf0,0x0d,0x07,0x58,0x9a,0x10,0x06,0x32,0x8f,0x27,0x60, -0x78,0x41,0x33,0x60,0x15,0xfe,0xe0,0x37,0xf8,0x81,0x78,0xd9, -0xd6,0x02,0x7b,0x27,0x76,0x02,0xe7,0x3a,0xc5,0x5b,0xab,0xda, -0x8c,0x87,0x73,0x07,0x35,0xae,0xd0,0x11,0xb1,0x92,0x85,0xf3, -0xf3,0xf9,0x35,0xbd,0xd8,0x8f,0x7d,0x95,0x5f,0x3c,0x22,0x8e, -0xf9,0x44,0x86,0x60,0x70,0x3a,0xac,0x56,0xad,0x73,0x97,0xe1, -0x08,0xc8,0x77,0x75,0x83,0x89,0xf0,0xd5,0x7d,0x9d,0x8f,0x95, -0xc6,0x1d,0xf4,0x83,0x8e,0xd1,0x23,0x19,0x1e,0x2d,0x60,0x7a, -0x3e,0x86,0x9b,0x39,0xfd,0xd3,0x40,0x5a,0x2e,0x46,0x13,0x4d, -0x26,0x7e,0xb2,0xee,0x72,0xf2,0x00,0x80,0xb7,0x01,0xa2,0x65, -0xb4,0x92,0xa9,0x51,0xc0,0xfc,0xc2,0xed,0xb0,0x38,0xef,0xdf, -0x0c,0xd3,0xdb,0x68,0x2b,0x4a,0xa3,0x2d,0x90,0xb3,0xe9,0x15, -0xe0,0x31,0x7c,0x8f,0x66,0xf0,0xb9,0x0f,0x9f,0x97,0xf0,0x79, -0x3f,0xba,0xaf,0x3f,0x7f,0x9b,0x4f,0x30,0xfe,0x3e,0xc4,0xff, -0xe7,0xa7,0xbd,0xef,0xda,0x7a,0x3a,0xb7,0x2d,0xc2,0xe6,0x7c, -0x4c,0x18,0x33,0x73,0xde,0x8d,0x5d,0x1a,0xa5,0x24,0x82,0x04, -0xbd,0xd4,0x0c,0x3f,0xe9,0x86,0x37,0x64,0xc0,0xa9,0x9b,0x96, -0x1f,0xbd,0x4b,0x47,0x0d,0x3e,0x05,0x97,0x30,0x34,0x7f,0x9d, -0xe4,0xe3,0x38,0x8a,0x12,0x58,0x4d,0x54,0x74,0x19,0x25,0xca, -0x94,0xeb,0xa1,0x88,0x2b,0x50,0x18,0x40,0x83,0x64,0x01,0x58, -0x12,0xc2,0x02,0x1b,0x18,0x2f,0x9a,0x04,0xa4,0xa3,0x0d,0xbc, -0x5c,0x0f,0xcd,0xeb,0xa0,0x53,0xf3,0x4c,0x47,0xde,0xb0,0xc2, -0xe4,0x0a,0xfa,0x8c,0x22,0x3b,0x51,0x94,0x6a,0xa2,0x1b,0x0c, -0xd2,0x9c,0x0e,0x6f,0x46,0x7d,0x7d,0xb2,0xf0,0x9a,0xc2,0x60, -0x14,0x85,0x81,0xc5,0x2a,0xb7,0x85,0x9a,0x8a,0x18,0xc0,0x2e, -0xa6,0xc3,0x2b,0x1e,0x5c,0xa6,0x84,0x8c,0xd1,0x4c,0x38,0xbd, -0x00,0x80,0x95,0x56,0x9c,0x61,0xfe,0xe6,0xaa,0x53,0x12,0x6f, -0xd6,0xb6,0x9d,0x92,0x29,0x25,0x77,0x4d,0xc6,0x5e,0xfb,0x2e, -0x5d,0x75,0x3b,0x79,0x01,0x6b,0x2a,0x1e,0x72,0x7c,0xca,0xc3, -0x25,0x0b,0xdb,0x58,0x88,0x21,0xe8,0x44,0x14,0x6c,0x37,0x46, -0xde,0x97,0x9d,0xb2,0x85,0x24,0x38,0xe6,0xe1,0x97,0xee,0x25, -0x7b,0xb1,0xd8,0x0d,0xb6,0x23,0x29,0x89,0xf4,0xec,0x5a,0xac, -0x23,0x92,0x34,0x8f,0x25,0xa0,0x22,0x28,0x9b,0x13,0xb2,0xba, -0x53,0x16,0xd2,0x2e,0x8d,0x2b,0xde,0x7c,0xf0,0x14,0xf4,0x8a, -0x87,0x53,0x3d,0xe9,0x8d,0xca,0xd9,0x0b,0xb1,0x26,0xea,0x65, -0xe9,0x22,0x67,0xdd,0xeb,0x7c,0x90,0x35,0x1a,0x36,0x43,0x23, -0x8a,0x2c,0x31,0x08,0xd4,0xa1,0x1f,0x0d,0x91,0xe6,0xf8,0x76, -0x61,0x36,0xbc,0xbe,0x19,0xa1,0x4d,0x9a,0xd9,0x4c,0x8f,0x15, -0x98,0x20,0xd0,0xa0,0x3a,0x26,0xdd,0x79,0xfc,0xb7,0xb8,0x7b, -0x52,0x9c,0x1c,0xf5,0x1a,0x9d,0xe4,0x6f,0xfb,0x3b,0x97,0x74, -0xdc,0xbc,0x99,0x8c,0x38,0x35,0x0b,0x92,0x79,0x5e,0xe8,0x94, -0x6d,0x3f,0x85,0xaa,0x31,0x9e,0xa0,0x33,0xef,0x6c,0x27,0x6e, -0x26,0xbf,0xec,0x30,0xb4,0x2e,0x10,0x67,0xf8,0x7d,0x3d,0x01, -0x4e,0x4e,0xf4,0x87,0xfe,0xa3,0xbf,0xa6,0x60,0xa9,0x12,0x3e, -0xc6,0xfa,0x03,0x78,0x88,0xff,0x47,0xff,0x82,0x27,0xa8,0x93, -0xf9,0xde,0xee,0xde,0xdf,0x75,0x80,0x7e,0x39,0xe2,0x7b,0x8e, -0xf8,0x3e,0x5a,0x32,0xde,0x69,0xb6,0x73,0x72,0xb2,0xb8,0xbf, -0x38,0x99,0x2e,0x4e,0xc6,0x8b,0x93,0xd9,0x82,0x32,0xd2,0xcf, -0xf7,0x3b,0x97,0x6d,0x57,0x75,0x21,0x5e,0x85,0xf5,0x12,0x4d, -0xc1,0x16,0xdc,0x1c,0x46,0xc9,0x0a,0x16,0xfc,0xb6,0x89,0xf4, -0xcc,0x81,0x2d,0x2d,0x6c,0x45,0x6b,0xd2,0x0a,0x84,0xf3,0x86, -0x0d,0x44,0xf1,0x82,0xd1,0x8d,0x4d,0x2e,0x5e,0x69,0xf4,0x91, -0x9a,0x9a,0x27,0xe1,0x73,0x9e,0x72,0x10,0xa2,0xd9,0x57,0x81, -0x99,0x8e,0x2b,0xc1,0xf4,0xe4,0x02,0xb4,0xb8,0xc7,0xeb,0x98, -0x7d,0x44,0x43,0x07,0xcc,0xe8,0xf4,0xf4,0xa6,0x91,0xdd,0x8f, -0xda,0xd0,0x08,0x76,0xc5,0x60,0xd2,0x83,0x35,0x82,0xdb,0x58, -0x8e,0x07,0x65,0x4a,0x57,0x93,0x8b,0x8b,0x42,0x3c,0xbb,0x24, -0xf4,0x8d,0x0c,0xd1,0x92,0x80,0x83,0xf5,0xff,0x09,0xce,0x2d, -0x4e,0xd4,0x6f,0x35,0xcb,0x11,0x8c,0x8b,0x06,0x0f,0x19,0xbb, -0x10,0x31,0x1d,0x70,0x84,0xe1,0x52,0xa2,0xfb,0x8d,0x93,0x71, -0x0c,0xc6,0x66,0xb3,0xd8,0xc0,0xeb,0x45,0x57,0x73,0x87,0xb0, -0x14,0x75,0xee,0xdf,0x07,0xd9,0x0a,0xc6,0x02,0x4c,0x92,0x68, -0xe8,0xfb,0xd8,0x16,0xb6,0x22,0xb5,0xc8,0x04,0x4c,0x80,0x51, -0x23,0x62,0x3c,0xa6,0x11,0x3c,0x24,0x6d,0x3d,0x88,0x1b,0x26, -0xa5,0xa1,0x87,0x34,0xb7,0xb4,0xe9,0x02,0x6a,0x88,0x06,0x56, -0xca,0x9c,0x05,0x31,0x00,0xcf,0x8c,0x3c,0x2c,0x76,0xa8,0x35, -0xf5,0xb0,0xca,0xfb,0x67,0x23,0xb8,0x65,0x32,0xfd,0x07,0xfa, -0xad,0x31,0x32,0xab,0xb7,0xcb,0xe4,0x16,0xca,0xe4,0xcc,0xd1, -0x52,0x07,0x12,0xdb,0xcf,0x6c,0x75,0x5a,0x69,0x2a,0xb2,0xfb, -0xf7,0xf5,0xcf,0xaf,0x25,0xef,0x82,0x30,0x5c,0xf4,0x42,0x9e, -0x8f,0x3d,0x5e,0x08,0xc9,0xd6,0xe0,0xa1,0x78,0xed,0xfe,0x7d, -0x7d,0xd6,0x6c,0xcb,0x02,0xcd,0x02,0x7e,0x7a,0x03,0xd1,0xed, -0xd9,0xf4,0xf3,0x2d,0x4d,0x21,0x9c,0x02,0x76,0x4d,0xaf,0xa8, -0x0d,0xc8,0x1e,0xf4,0xf8,0x3c,0x8d,0x98,0xdc,0x64,0x79,0x0e, -0x2d,0x11,0xeb,0x55,0x9a,0xee,0x1c,0x86,0x46,0x20,0xc2,0xc3, -0x1f,0xa5,0x18,0x68,0x3a,0xdd,0xca,0x9e,0xa1,0x20,0x32,0x97, -0x7e,0x4a,0xf3,0xaf,0x3c,0xd1,0x3d,0xa7,0x7c,0x94,0x43,0xac, -0xc5,0x9c,0xd7,0x5e,0x09,0x9b,0xfc,0xa6,0x68,0xcb,0xbb,0xeb, -0x51,0x51,0x5b,0x09,0xd8,0xeb,0xfd,0x5e,0x88,0x94,0x41,0x84, -0x92,0xc2,0xab,0x7e,0x69,0xb7,0x32,0xed,0x46,0xef,0x00,0x10, -0x22,0xb6,0xe6,0x5e,0x4a,0xbb,0xad,0xef,0xf0,0xe3,0x14,0xc1, -0x3b,0x7e,0x5e,0x14,0x1c,0xb6,0x79,0x6b,0x8c,0xc1,0xec,0x11, -0xbb,0xd8,0x9b,0x80,0x63,0x68,0x74,0x6b,0xa9,0x74,0x6f,0xfd, -0x0e,0x7a,0xb1,0xfa,0xab,0xb8,0xca,0x2f,0x60,0x91,0x05,0xf1, -0x39,0xfc,0xdc,0xc0,0x74,0x45,0xce,0x84,0x52,0x4a,0x07,0x35, -0xba,0x27,0x04,0xee,0x40,0x58,0x9f,0x5a,0x77,0xeb,0x84,0xd3, -0x83,0xb7,0x71,0x3a,0xfb,0x14,0x80,0xd1,0xa8,0x91,0xc3,0x2b, -0x5a,0xb7,0x71,0x32,0x8d,0x9b,0x19,0xa0,0xdc,0xda,0x32,0x34, -0x52,0x78,0xb1,0xd8,0xdd,0x94,0x07,0xf2,0xc5,0x82,0x2c,0xcc, -0xe2,0x3b,0xb6,0xdd,0x5e,0x79,0xd7,0xc5,0x43,0xc9,0xd2,0x36, -0x0b,0x89,0xb7,0x75,0x85,0x61,0x02,0x40,0x03,0x20,0xee,0xbf, -0xae,0xd6,0xde,0xf5,0x88,0x20,0xc3,0xab,0xb0,0xd7,0x24,0xa2, -0xea,0xcc,0x76,0xb1,0x64,0x58,0x94,0xa3,0x6e,0xb1,0xb7,0xd3, -0x72,0x41,0x62,0x60,0xc0,0xdd,0x2c,0x9d,0x55,0x82,0xe7,0x7e, -0x01,0x30,0x97,0x0b,0xbc,0x87,0x23,0x50,0x70,0x4c,0xd6,0xfa, -0x96,0xf0,0x08,0x7e,0xd3,0x1f,0x1c,0x4f,0x3e,0x0c,0xad,0xb2, -0xdb,0x7c,0x3c,0x2b,0x5f,0x4f,0xba,0xf9,0x06,0x0f,0xde,0x7e, -0x1e,0x4e,0x27,0xcf,0x73,0x8d,0x1e,0xf9,0x1e,0xd7,0x12,0xfd, -0x84,0xf3,0x0b,0x93,0x91,0x93,0xe9,0x20,0x1f,0xf7,0x47,0xf9, -0x1f,0x43,0x51,0x8a,0xde,0x29,0xf2,0xc9,0x60,0x55,0x31,0x58, -0x9f,0x51,0x1f,0xbc,0xde,0x34,0x19,0x45,0x45,0x69,0x8c,0x47, -0x14,0xf7,0x12,0x2c,0xfa,0xf5,0x2f,0x87,0xa0,0xce,0x67,0x23, -0x0f,0x27,0xe8,0x5b,0x86,0xcd,0x2e,0xea,0x06,0x1f,0x9e,0x7f, -0x00,0x23,0xd8,0xfa,0x14,0xf4,0xd1,0x1a,0x6b,0xe4,0xae,0x41, -0xd4,0x1c,0xe7,0x30,0x1c,0x18,0x2b,0x7f,0x03,0x63,0xda,0x8f, -0xf5,0x4b,0xa6,0xd7,0x58,0xb7,0xc1,0x0b,0x78,0x7f,0x91,0xd9, -0x30,0xc9,0x57,0xde,0xe9,0xa3,0x7e,0xe1,0xb2,0xa8,0xcf,0xc3, -0x3e,0x5c,0x19,0xfa,0x99,0x9a,0x10,0x0b,0xfa,0x93,0xd7,0xfa, -0xc8,0x77,0x55,0x4e,0xc6,0x68,0x48,0xff,0x38,0x1c,0x7e,0x28, -0x27,0x43,0x2c,0xa4,0x0e,0xfa,0x9f,0xcb,0x89,0x03,0x90,0x25, -0xed,0xaa,0x2b,0xbd,0x88,0x95,0x13,0x21,0x16,0xcb,0xcd,0xc7, -0xf3,0xd9,0xb0,0xa2,0x60,0x8c,0x47,0xcd,0xce,0xa1,0x6e,0x90, -0x41,0x19,0x82,0xe2,0x09,0xc7,0x68,0x94,0xd7,0x81,0x89,0x44, -0x0d,0xdb,0xa6,0xa1,0x8a,0x0f,0x56,0x32,0xd3,0x36,0xe4,0xfa, -0xfe,0xd4,0x43,0xd3,0x90,0xa1,0x46,0x6b,0xf8,0xf0,0x81,0xf9, -0xfe,0x76,0xf8,0xe8,0x01,0x53,0xdd,0x78,0xf8,0xed,0xf0,0x9b, -0x07,0x58,0x41,0xc6,0x81,0x0d,0xd1,0x80,0xbf,0x8d,0xef,0x1e, -0x60,0x9b,0x19,0xe4,0xd4,0xbe,0x0d,0xfa,0x6d,0xb4,0xf6,0x1e, -0x60,0x7f,0xd8,0x6c,0xb3,0x3e,0xdc,0x94,0x51,0xe8,0x6c,0x7e, -0xa6,0x37,0x83,0x58,0xba,0x87,0xa0,0xe1,0x81,0x62,0x4a,0xab, -0x7f,0x4b,0x32,0xd2,0xb3,0xc0,0xae,0x61,0x9c,0x83,0x5c,0xb6, -0x0f,0x87,0xfa,0x33,0xfd,0xc7,0xd9,0x82,0x0d,0xe1,0x22,0x23, -0x64,0x89,0x30,0x83,0xb5,0x9f,0x98,0x9d,0xd9,0xcf,0x44,0x95, -0x73,0xe1,0x1a,0xf0,0xfa,0x82,0x33,0x71,0x48,0xe7,0xe1,0xaf, -0x44,0x09,0x6f,0x3d,0xfd,0xb3,0xe2,0x2d,0xa8,0x75,0xc6,0xa4, -0x1b,0x6e,0xe7,0xd7,0xee,0x3e,0x45,0x74,0x9c,0xfe,0x2d,0x43, -0xa4,0x42,0x40,0xc0,0x51,0x0e,0x9d,0x37,0xfb,0x29,0x95,0x2d, -0x75,0xbc,0x0c,0xcc,0xd7,0xd1,0xfb,0x9f,0x8c,0x60,0xe0,0x24, -0xc3,0xaf,0x84,0x58,0x7f,0x58,0x66,0x6a,0x27,0x0c,0x1c,0xed, -0x46,0x0d,0xfa,0x34,0x4d,0x46,0x21,0xcf,0xd2,0xf1,0xeb,0xe9, -0xd1,0xfc,0x0c,0x5d,0x25,0x99,0x49,0x09,0xce,0x28,0x79,0x8a, -0x5f,0x4f,0xae,0x95,0x1d,0x53,0x79,0xf1,0x64,0x00,0xfa,0x5d, -0x8a,0x6e,0x4d,0xde,0xdd,0xe8,0x3e,0xd6,0xed,0x43,0x0c,0x2e, -0x3d,0x6f,0xc4,0x31,0x44,0xf3,0xc3,0x1f,0xc3,0x06,0x89,0x3f, -0x26,0x69,0x96,0xb9,0x34,0x08,0x9a,0x79,0x2b,0x72,0x60,0x44, -0x5b,0x66,0x04,0xeb,0x86,0xd7,0x1a,0xbe,0xc9,0xda,0x3f,0x71, -0x83,0xc2,0xde,0x18,0x7f,0x60,0x08,0x4e,0x54,0x0c,0x98,0x17, -0x0b,0xc2,0x04,0xfd,0x6c,0x26,0x29,0x64,0xa3,0xef,0x38,0xe1, -0x79,0x0d,0x51,0xf0,0x15,0x27,0x09,0xd2,0x47,0x65,0x41,0x5d, -0x63,0xfb,0x91,0xe0,0x84,0x10,0x05,0x10,0x66,0x02,0xc5,0xef, -0xd8,0x7d,0x25,0x3c,0x45,0x24,0xb8,0x57,0x97,0xcd,0x72,0x83, -0x22,0x26,0xdd,0x03,0xcd,0xb9,0x88,0x04,0x9c,0x55,0x75,0x71, -0x75,0xe0,0x6a,0x01,0x3d,0x5c,0x09,0xac,0x93,0xf4,0x51,0x6c, -0x44,0xf9,0xf4,0xdc,0xad,0x56,0x16,0x5e,0xb2,0x80,0xda,0xf4, -0x25,0x98,0x94,0x5d,0xa2,0x05,0x31,0x76,0x0d,0x56,0xba,0x00, -0xfc,0x72,0xa4,0xf8,0x6a,0x70,0x3a,0x44,0xa2,0x0a,0xd2,0xbc, -0x6f,0xd1,0xcb,0x8a,0x3d,0x35,0xd0,0xf5,0x7f,0x3a,0x41,0x59, -0x17,0x1f,0xf6,0x41,0x5b,0x23,0x33,0x56,0x25,0x18,0xda,0x1c, -0x21,0x28,0x93,0xb9,0x46,0x62,0xbd,0x8e,0x83,0xfc,0xe2,0x82, -0x32,0xe8,0x69,0xed,0x67,0xd8,0x0e,0x32,0xc0,0xc3,0x23,0xd0, -0xcc,0x84,0xe9,0xc8,0x1a,0x29,0x2c,0x70,0x8c,0x05,0x21,0xfc, -0x6a,0xa1,0xa5,0x57,0xa9,0x4d,0x7e,0xfe,0xb0,0xa7,0xbf,0x17, -0x8b,0x4d,0x0f,0x68,0x36,0x79,0xa1,0xe7,0x96,0x05,0x05,0x77, -0x1c,0x22,0x0a,0x72,0x24,0xba,0x5f,0xb1,0xc8,0x46,0xc3,0x4c, -0x5d,0x0a,0x3a,0xc2,0x5d,0x2b,0xd9,0xfd,0x81,0xf6,0xc7,0x39, -0xfc,0x45,0x81,0x13,0x7f,0x91,0x41,0x84,0x8f,0x43,0x70,0xa6, -0x83,0x51,0xba,0xd5,0x5f,0x42,0x98,0xbc,0x36,0xdb,0x33,0x27, -0x88,0x22,0x8a,0x7b,0x3b,0x2a,0xba,0xd7,0x8a,0x92,0x36,0x42, -0x22,0xfc,0x13,0x70,0xf7,0xac,0x0f,0x9a,0x18,0xa3,0x2b,0x73, -0xae,0x39,0x39,0xeb,0x15,0xb9,0xe8,0x32,0x6a,0x9d,0xc0,0x5f, -0xe6,0xde,0x0a,0xe1,0x2b,0xc8,0x94,0x9b,0x39,0xf6,0x38,0x45, -0x84,0x2c,0x00,0x2c,0xd0,0x28,0x40,0x53,0x21,0x5b,0xc0,0x6f, -0x91,0x8c,0x8e,0x82,0xc4,0x21,0xbe,0xc3,0x75,0x1e,0xc0,0x61, -0xba,0xf8,0x25,0x64,0x41,0xeb,0x21,0x94,0xf2,0x61,0xbc,0x4c, -0x48,0x42,0xd7,0x07,0xe8,0x65,0xa2,0x5c,0x56,0x83,0x70,0x4a, -0x38,0x7e,0x56,0xd7,0x20,0xd7,0xfd,0x0f,0x43,0xf4,0x33,0x76, -0x01,0x1e,0x65,0x8c,0x82,0xc9,0x1c,0x2d,0x53,0xcf,0xf4,0xf1, -0xda,0x2a,0xa2,0x43,0xb2,0x7d,0xcc,0x13,0xc1,0x1e,0xac,0x8f, -0xdc,0x08,0x99,0x7d,0xc7,0xb0,0x59,0xa4,0x57,0x85,0x08,0xcd, -0xa1,0x42,0xcf,0x03,0xe7,0x1f,0x64,0xc3,0xf5,0x22,0x32,0x37, -0x6e,0x6d,0xca,0xde,0xda,0xb3,0xf9,0x29,0x7d,0x49,0xcb,0x4e, -0x17,0x33,0x0b,0x76,0x1d,0x3c,0x33,0xf6,0x67,0xd2,0xae,0x41, -0xae,0x2e,0x31,0x27,0xf3,0xeb,0x19,0xaf,0x57,0x17,0x7a,0xc1, -0x06,0x3e,0x93,0x31,0x88,0x07,0x96,0x6d,0xf1,0xa4,0xca,0x5e, -0xf3,0x10,0x5e,0x30,0x8b,0x48,0x2f,0xfa,0xa9,0x18,0xfa,0xc9, -0x1c,0x7b,0xcd,0x65,0x89,0xf7,0xbf,0xbc,0xe3,0x70,0xb1,0x7a, -0x14,0xcf,0x67,0xe7,0xfa,0x2f,0xac,0x92,0x54,0x23,0x21,0xea, -0xe7,0x13,0x05,0x2e,0x31,0x01,0x99,0xea,0x9a,0x0b,0xd3,0x07, -0xd5,0x28,0x59,0x72,0x05,0x39,0x1f,0x15,0x4a,0xa4,0x25,0x76, -0xea,0x63,0xcb,0xf9,0x6f,0xef,0xe8,0xee,0xdb,0x80,0x27,0xa5, -0x07,0x77,0xae,0xcf,0xcd,0x3c,0xa7,0x73,0x0c,0x1a,0xbd,0x1e, -0x4e,0xcf,0x2d,0x03,0x7c,0x0e,0xa1,0xe1,0x80,0x6e,0xa3,0xb2, -0x46,0x05,0x1c,0x9d,0x58,0x32,0x6b,0x03,0x0e,0x7a,0xda,0xcb, -0xb5,0xb5,0x65,0x2f,0xc5,0xbc,0x78,0x18,0xfa,0x94,0xd7,0x8b, -0xde,0xcf,0x76,0x3b,0x82,0x43,0xf1,0xb3,0xa4,0x8e,0x9b,0xf1, -0x13,0x12,0x22,0xc3,0xd5,0x0b,0xf6,0xa4,0x17,0xe3,0x43,0xdc, -0xef,0x80,0x0d,0xa4,0x0d,0xd1,0xc9,0xe3,0x59,0xaf,0x38,0x86, -0x3f,0xcd,0x77,0xc7,0x4f,0x05,0x50,0xa3,0x05,0xcf,0xcf,0xc1, -0xe9,0x9e,0x8e,0x47,0xa0,0x24,0xc4,0xfb,0x2f,0x0d,0x8c,0x39, -0xc4,0xc5,0xdf,0xcb,0x61,0xff,0xc6,0xc5,0x77,0x1e,0x7e,0xfb, -0x6d,0xfa,0xf0,0xdb,0x6f,0xe4,0x86,0xe4,0x43,0xdc,0x8a,0x5b, -0x6a,0x88,0xf8,0xdb,0xa3,0xad,0x2d,0x68,0x3c,0xfc,0x6e,0xed, -0xee,0x6a,0x56,0xda,0xa6,0xec,0xee,0x8a,0x3d,0xc8,0x3b,0xdd, -0xb0,0x37,0x84,0x09,0x87,0xdb,0x9a,0xe9,0xe8,0x6f,0x6d,0x6d, -0xef,0xe9,0xac,0xfa,0xf3,0xe6,0xa2,0x69,0x52,0x40,0x3f,0x8a, -0x3f,0x21,0xa5,0xdf,0x3d,0x7c,0xfd,0xea,0xf8,0xa7,0xde,0x63, -0x5d,0x8e,0x08,0xee,0xb7,0x5a,0x1d,0xfc,0x4a,0x31,0xf2,0xe0, -0xc9,0xf1,0xb3,0xde,0xe3,0x16,0x83,0x60,0x68,0x5f,0xb6,0x2c, -0x46,0xff,0xeb,0xd9,0x93,0xb7,0x3d,0x25,0x90,0x24,0x1d,0x80, -0x24,0x0c,0x3f,0xbd,0x7e,0xf7,0xd6,0x15,0x82,0xa1,0xfd,0xbd, -0x87,0x1d,0xf8,0x20,0x80,0xc3,0x17,0xaf,0xde,0x41,0x21,0x96, -0x0e,0x0a,0xef,0x7f,0xf3,0x7d,0x87,0x3e,0x09,0xec,0xe8,0xd9, -0xd3,0xd7,0xaf,0x0e,0x1c,0x18,0x87,0x01,0x8c,0x3e,0x0d,0xb6, -0x97,0x2f,0x5f,0x84,0xb0,0x32,0x72,0xff,0xfb,0xef,0x01,0xb1, -0x8d,0x48,0xb7,0x5b,0x8a,0x1a,0xea,0xd4,0x34,0xcf,0x41,0xff, -0xf3,0xeb,0x0b,0xe8,0x27,0xdd,0x64,0x50,0xb7,0x7d,0x93,0xb0, -0x58,0x98,0xaf,0x7d,0xa8,0x60,0x22,0x9b,0x14,0x23,0x94,0xdf, -0xe4,0x99,0xf9,0x90,0x6c,0x09,0x3d,0xe0,0x06,0x5f,0x3f,0xfa, -0x6c,0xa2,0x19,0x8d,0xe7,0xa3,0xfe,0x65,0xe1,0x8e,0xae,0xe4, -0x88,0x40,0x63,0xc9,0x6e,0xd1,0x58,0x70,0xba,0xd9,0x52,0xf3, -0xf1,0xbc,0x18,0xd2,0x21,0xbe,0x48,0xbb,0x3d,0x0e,0xe3,0xe2, -0x0d,0x41,0x53,0x4a,0xba,0xbd,0xa7,0xce,0xaf,0x34,0xd6,0x97, -0x9a,0x89,0x87,0x01,0x92,0xee,0xa2,0xf5,0x03,0x02,0xd4,0x78, -0x72,0x52,0x01,0xc6,0xae,0x4b,0x49,0x1f,0x9e,0x62,0x9e,0xe3, -0x8a,0x83,0x45,0x15,0xc3,0x29,0x6b,0x0a,0x83,0xdf,0x08,0x1d, -0xb5,0x94,0x23,0xf8,0x1f,0x90,0x10,0x5f,0x87,0x57,0x91,0xfa, -0xb4,0x47,0x49,0xc0,0xfc,0xd9,0x40,0xc6,0xb7,0xdf,0xc8,0x17, -0x5f,0x32,0x5f,0x0c,0x8c,0x84,0xdf,0x4a,0x8f,0x77,0x35,0xc3, -0x49,0x51,0x58,0x63,0x1b,0x92,0xe4,0xda,0x48,0x5b,0xa1,0x10, -0xec,0x39,0x2f,0xdc,0x1c,0x1b,0x54,0x04,0x7a,0x06,0xee,0xdb, -0xce,0x67,0x3e,0x89,0x92,0xf4,0x5d,0x3b,0x67,0xbc,0x56,0x94, -0x09,0xb2,0x27,0x0c,0x2b,0x86,0xbd,0xce,0x48,0x2a,0x98,0x0b, -0x2b,0xb0,0x90,0xd7,0xdd,0xfa,0x1b,0xcc,0xd1,0xd6,0xf0,0x3d, -0x20,0x60,0x8d,0xb6,0xa3,0x24,0xd5,0x20,0xe2,0x68,0x36,0xe9, -0x0f,0x00,0x59,0x6c,0xed,0xdc,0x14,0xfe,0xfd,0x75,0xa1,0xb9, -0xc6,0xb3,0x29,0x68,0xfb,0xaa,0x11,0x17,0x4a,0xc6,0xf3,0x16, -0x8b,0xd8,0x8f,0x40,0x41,0xaf,0x21,0x2c,0x09,0xa0,0x71,0xcb, -0x62,0xfc,0x41,0x92,0x23,0x66,0x3e,0x96,0xe4,0x80,0x1b,0x5e, -0x94,0xe4,0xd5,0x81,0xc3,0x51,0x50,0x27,0x1d,0xc0,0xee,0x99, -0x3b,0xfd,0x57,0x58,0xb9,0x7e,0xc5,0x32,0xd4,0x18,0x2e,0x77, -0x40,0x5c,0x38,0x43,0x3d,0x67,0x9d,0xc1,0xed,0xad,0xa4,0xfd, -0xb8,0x29,0x90,0xf7,0xb6,0xb6,0x34,0x33,0x75,0x88,0x76,0xe3, -0x13,0x12,0x60,0xff,0x36,0xcf,0xa7,0xba,0xe9,0x9a,0x3b,0x00, -0xb6,0x13,0x35,0x3e,0x08,0xe1,0xb4,0xb5,0x4d,0x23,0x30,0x2c, -0xf1,0x0d,0x26,0x90,0x61,0x36,0x65,0x7f,0x23,0x26,0xab,0x6f, -0x7c,0x36,0x41,0xd5,0x20,0x20,0x02,0x52,0x32,0x4d,0x5c,0x2c, -0x94,0x85,0x10,0x6d,0x1b,0x9e,0xae,0x71,0xad,0x41,0x27,0x2c, -0x7f,0x0c,0xfd,0xcb,0x9a,0x47,0x74,0x6a,0xc6,0xda,0x65,0x95, -0x03,0x03,0x38,0xec,0x26,0xa6,0xc7,0xd0,0xf1,0xea,0xd7,0x0c, -0x03,0xe6,0xb0,0x00,0x8d,0x53,0x97,0xb1,0xd1,0x02,0x5b,0xd0, -0x08,0xa0,0xff,0x74,0xc6,0x78,0x91,0x63,0x31,0xe1,0x34,0x6f, -0xff,0xba,0xbf,0xdb,0xf6,0xc9,0x27,0xf4,0xc6,0x62,0xd9,0xaf, -0xe6,0x12,0x5c,0xe7,0xf0,0x6b,0x05,0xb7,0xc4,0x68,0x26,0x1a, -0xf1,0xb2,0xaa,0x79,0xf6,0xeb,0xd6,0x96,0x7f,0x12,0xa2,0x8e, -0xc3,0x3e,0xdc,0xdc,0x4d,0x34,0xc0,0x76,0x8b,0x15,0xc8,0x7e, -0xdd,0xde,0x5e,0x6a,0xc6,0x64,0x59,0xdd,0xc8,0x6e,0x84,0x90, -0xaf,0x30,0x9a,0xc4,0x20,0xb1,0xa7,0x39,0xe6,0x1f,0xdf,0x88, -0xb1,0x6e,0xe2,0xed,0x4b,0xbc,0x73,0xd2,0xa5,0x5b,0xcc,0x9d, -0xa4,0x43,0xf1,0xf6,0xd8,0xf0,0xcb,0x49,0x77,0x71,0xd2,0xbb, -0xb7,0x73,0xa9,0x34,0x0b,0x95,0x06,0x89,0x27,0x27,0x14,0xef, -0xb3,0xbd,0x54,0xf0,0x73,0x9f,0xbb,0x14,0xc7,0x38,0x7b,0x64, -0x63,0x8e,0x90,0x89,0xb8,0x08,0xe8,0xb5,0x4c,0x59,0xe5,0x7b, -0x7e,0x4f,0x53,0xdf,0x3e,0xdb,0x26,0x1c,0x98,0xdf,0x9d,0x61, -0x4c,0x6a,0xaf,0xb3,0x3a,0x39,0xad,0x69,0x37,0x03,0x50,0x7e, -0xc5,0x0a,0x47,0xf6,0x5b,0x21,0xc0,0x89,0x22,0x79,0x86,0x74, -0xf4,0x51,0x72,0x23,0x33,0x98,0x84,0xfa,0xa5,0xa1,0xa3,0x63, -0xd2,0x2c,0x27,0xcb,0xbc,0x2b,0xbf,0x6c,0x17,0x6f,0x0e,0x58, -0xce,0xe3,0x1a,0x9d,0x00,0x8d,0x54,0xc7,0xe4,0x33,0x1d,0x7d, -0xdd,0x34,0x7b,0x4d,0xd2,0xe1,0x46,0xce,0x86,0x9f,0x6e,0xfa, -0x63,0x5e,0xe6,0x0d,0xfb,0x7f,0xcd,0x22,0xe4,0x84,0x11,0xb8, -0x16,0xa2,0x30,0x2c,0x7d,0x35,0x29,0x59,0x7d,0xb7,0xd7,0x62, -0xd3,0x3b,0x5f,0x92,0x9a,0x32,0xcd,0xc6,0x13,0x72,0x88,0x55, -0x74,0x42,0x0e,0x21,0x92,0xe7,0xc1,0xf8,0x72,0xa2,0x97,0x44, -0x9d,0xfb,0xb9,0xeb,0xe2,0x70,0xc4,0x43,0xc6,0xe6,0xc8,0x83, -0x63,0x88,0xc5,0x02,0x7f,0x97,0x64,0x27,0xe0,0x1b,0xec,0x44, -0xf4,0x52,0x1f,0x8e,0x05,0x67,0x5f,0x00,0x5e,0x3a,0x69,0xf6, -0x7a,0x6b,0xab,0x1a,0x6e,0x36,0x2c,0x0c,0xc1,0x49,0x3b,0xe1, -0x56,0xe7,0xe1,0x6e,0xa6,0x4f,0x65,0x4e,0xb5,0xa2,0x3e,0xec, -0xbe,0x7f,0x15,0x51,0x2a,0xdf,0xce,0xdc,0x23,0x36,0x04,0xf4, -0x76,0x8e,0x37,0xe4,0xf3,0xf5,0x72,0xf8,0x49,0x63,0xa1,0x4b, -0x86,0x19,0xfc,0x35,0x52,0x7c,0xd2,0xf4,0x30,0xaa,0xe5,0x98, -0xc4,0x6a,0xe5,0x07,0xfa,0x7f,0x56,0x59,0x1b,0x95,0xf0,0x30, -0xfb,0xf1,0xd5,0x74,0x38,0x3c,0xc8,0x2f,0xf5,0xf1,0x9a,0xf4, -0xbb,0xff,0xa5,0xff,0xc7,0xea,0xe0,0x3f,0xea,0xff,0xf1,0xe7, -0xa5,0xfe,0x5f,0x45,0xee,0xe7,0x93,0xf9,0x34,0xcc,0x2c,0x73, -0xcb,0xec,0x55,0xf9,0x8f,0xf2,0x4f,0x32,0xfb,0x11,0x83,0x1f, -0xb9,0x0f,0xf3,0x55,0x4d,0xfd,0xeb,0xb1,0xfe,0x2d,0x55,0xe1, -0xf0,0xf0,0x90,0x73,0x1d,0xba,0xcf,0xc1,0xc0,0x7e,0x88,0xaf, -0x41,0x05,0xd2,0xf7,0x93,0xe9,0x80,0x10,0xf5,0x19,0xf0,0x89, -0x85,0x2a,0x6f,0xde,0x86,0x4b,0x1d,0x25,0xcd,0xd3,0xeb,0xe1, -0x34,0x1f,0xe4,0xc3,0x6b,0xec,0x25,0x42,0xf1,0xcf,0xaa,0x36, -0x37,0xef,0x37,0x0f,0x99,0xe0,0x9f,0xb9,0x9c,0x9f,0x7f,0xae, -0x81,0xfe,0x63,0x32,0x66,0x7c,0xc7,0x55,0x10,0xa6,0xda,0xb6, -0xad,0xf8,0x43,0xf4,0x85,0xed,0x08,0xfe,0xf8,0xe9,0x27,0xfe, -0xb8,0xba,0xe2,0x8f,0xeb,0x6b,0xfe,0x28,0x0a,0xd3,0x7a,0x06, -0x9f,0x69,0x2f,0x93,0xd7,0x64,0x35,0x39,0x4d,0x46,0xfe,0xfd, -0x68,0x7e,0xcd,0xc7,0x7b,0xf3,0x6b,0x3e,0x86,0xfc,0xfb,0xac, -0xba,0x4f,0x5f,0x4f,0x8f,0x3f,0x4e,0xb8,0x47,0x59,0x7f,0xc5, -0xbd,0x82,0x10,0xba,0x29,0x53,0x98,0x08,0x37,0xcf,0x48,0x4b, -0xc2,0x28,0xa5,0xf1,0xca,0x80,0x63,0xdf,0xb1,0x91,0xa8,0xac, -0xa3,0x77,0x74,0x15,0xe5,0xa0,0x0f,0x2b,0x0e,0xfd,0xdc,0xbc, -0x87,0x24,0x9d,0x05,0xf9,0x3a,0xab,0x4f,0x1b,0xb5,0x35,0xbc, -0x98,0xff,0xe3,0xfc,0x6a,0x3e,0xfe,0x90,0xc5,0xe5,0x7e,0x69, -0x0e,0x3f,0x0d,0xcf,0x0d,0xf0,0x62,0xd1,0xed,0x25,0x70,0x81, -0x0b,0x86,0x16,0x8a,0x2c,0xe6,0x7c,0x0d,0x5d,0x34,0x6f,0x93, -0x84,0x80,0xf3,0x3e,0x85,0x44,0x7d,0x64,0xd7,0xd9,0x34,0xbf, -0xa1,0x76,0x95,0xce,0x69,0xa4,0xdf,0x8d,0xf8,0xdb,0xdd,0x07, -0x88,0xa6,0xab,0xd9,0x9a,0x06,0x09,0x27,0x28,0xbc,0x67,0xf7, -0xb2,0xa8,0x01,0x92,0x5b,0x8a,0xdd,0xed,0x75,0xb6,0x39,0x73, -0xca,0xbf,0xde,0x65,0x02,0x14,0x7a,0x4c,0xa6,0x08,0xa1,0x96, -0x72,0xf5,0x20,0x8b,0x6a,0xde,0x1a,0x02,0xba,0x05,0x70,0x2e, -0x9b,0x91,0xce,0xb6,0x19,0xe6,0x95,0x6b,0x8b,0x9b,0x6e,0x51, -0x4a,0xb6,0x63,0x72,0x3a,0x96,0xc4,0x1e,0x12,0x3e,0x12,0xb3, -0x3c,0x95,0x96,0x6e,0xcd,0x1c,0xb5,0x89,0x3b,0xaa,0x9c,0xba, -0xfd,0x6c,0xe5,0x8c,0x23,0x39,0x3b,0x4e,0x37,0xc6,0xa7,0xa8, -0xfc,0x7e,0xa7,0xb2,0xe4,0x7e,0xea,0x0e,0x95,0xde,0x99,0x8a, -0x08,0x96,0x94,0x1c,0x88,0xc9,0xb4,0xaa,0x4e,0x28,0x08,0xf0, -0xaa,0xe4,0x55,0xe8,0xe0,0xc0,0x21,0x2a,0xa3,0x32,0xd4,0x0c, -0xcc,0x31,0xbb,0x1e,0x11,0xcc,0x65,0xbf,0x64,0x94,0x35,0x78, -0x19,0x1a,0xb1,0x0c,0xed,0x7f,0xfb,0xf7,0x4e,0xeb,0xfb,0xdd, -0xdd,0x74,0x6f,0xf8,0x30,0x40,0x65,0x17,0x06,0x5e,0xb0,0xef, -0x42,0x2c,0x73,0x8b,0x45,0xd1,0x90,0x9f,0x17,0x6f,0xae,0x57, -0x77,0x94,0x86,0x38,0xac,0xc0,0xf5,0x53,0x69,0x35,0x12,0xab, -0x92,0x4f,0x14,0x8a,0x4b,0x6a,0x89,0xba,0x16,0x8b,0x58,0xd0, -0xf3,0x24,0x43,0xa9,0xcd,0x59,0x88,0x55,0xcf,0xcf,0xc9,0x52, -0x92,0xda,0x9c,0xd5,0xbb,0x55,0x58,0xba,0x13,0xb7,0x30,0x22, -0xb8,0xdc,0x8d,0xa3,0xdd,0x26,0xe8,0x69,0x95,0x7a,0xf9,0x9f, -0xae,0x51,0x07,0xf6,0xc1,0x3f,0x66,0x11,0x7a,0xf3,0x15,0xf9, -0xe4,0x0e,0x62,0x10,0xe8,0xe3,0xf9,0xbb,0xe3,0xa7,0xa0,0x4e, -0x61,0x62,0x66,0x7f,0x5c,0x67,0xf5,0xeb,0x5d,0xb9,0x86,0x77, -0x2f,0xe4,0x83,0xbb,0x76,0xd7,0x70,0xb9,0xc7,0x65,0x23,0xa3, -0xc5,0xb9,0x98,0x9f,0xe9,0x65,0x53,0x9f,0xb9,0x5a,0xfc,0xa0, -0xcd,0xee,0x4e,0xc4,0x2e,0xf8,0x5b,0x97,0x60,0x27,0x98,0xb5, -0xa8,0xc4,0x05,0xfe,0x07,0x82,0xd9,0xf5,0xd1,0xae,0x5e,0x1f, -0xd1,0x09,0xbf,0x0d,0x75,0x31,0x2f,0x5f,0x10,0xc8,0x4d,0x01, -0x3a,0x11,0x5a,0xc6,0xf7,0x71,0xc9,0x47,0x20,0x48,0x54,0xe7, -0xf3,0x29,0xf8,0xa5,0x47,0xeb,0x1d,0x20,0x97,0x3c,0x9e,0xbc, -0x2b,0x86,0xea,0x22,0xff,0x04,0xb3,0x58,0x7d,0x44,0x05,0x2a, -0x3a,0xcd,0xc3,0x95,0x80,0x9e,0xdf,0xf8,0x4b,0xa4,0xb1,0xbc, -0x7d,0xd3,0xf6,0x34,0x9d,0x89,0x05,0xca,0x4c,0x7c,0xd3,0xa9, -0xfb,0xdc,0xf3,0x09,0xaa,0x69,0xd7,0xc7,0x50,0x14,0x38,0xd9, -0x85,0x99,0x56,0xa2,0x72,0x34,0x2d,0x7d,0xba,0x39,0x98,0x38, -0x4f,0xef,0x5f,0x4a,0x4d,0x40,0x65,0xd9,0x5c,0x59,0x3f,0xec, -0xe0,0x78,0x83,0xf1,0xaa,0x63,0x55,0x6b,0x17,0x57,0x93,0xa8, -0xf5,0x7d,0xd4,0xd0,0xe1,0x34,0xda,0xdb,0xa5,0x0f,0xf8,0x17, -0x16,0x88,0xcb,0x47,0xc7,0xca,0xfe,0xa1,0xde,0x28,0xe9,0x4d, -0xd2,0x00,0x66,0xa9,0x44,0xcf,0xf0,0xc2,0xfd,0xb1,0xf9,0xe3, -0x8f,0xf4,0x68,0x13,0xbe,0xdf,0xbb,0xcf,0x67,0x1d,0x68,0xd2, -0xcc,0xae,0x94,0xd0,0x3d,0xef,0x41,0xe3,0xc1,0xd4,0x2b,0x86, -0xac,0x89,0xc2,0x4c,0x2d,0xfd,0xf3,0x4c,0x3d,0xd2,0x03,0x2b, -0xb5,0xe7,0xfc,0xda,0xc5,0xc9,0xf4,0x50,0x66,0x8a,0x1a,0x50, -0xe5,0xdf,0x53,0xb4,0xc6,0x3b,0xa0,0x43,0x4b,0x6a,0x00,0x86, -0xae,0x75,0x74,0x00,0x5a,0xa7,0x81,0x87,0x93,0x53,0xc0,0xd4, -0x1c,0x4c,0x3e,0xa6,0xa4,0xc1,0x92,0x09,0xb0,0x8f,0x00,0x06, -0x94,0xd9,0x52,0xb6,0xb6,0xb8,0xe0,0xc7,0x7e,0x66,0x8a,0x6f, -0x34,0xd4,0xdd,0x15,0xbe,0xbc,0x24,0xea,0xed,0x20,0xf3,0x30, -0x05,0xc1,0x8f,0xce,0x53,0xbe,0xe9,0x81,0x0c,0x8a,0x40,0xa5, -0x1c,0x55,0xb1,0x15,0x41,0x9a,0x0d,0x26,0x65,0x08,0x3d,0xa6, -0xec,0xd0,0xcf,0xaa,0x47,0x81,0x18,0xc7,0x14,0x13,0x8e,0x81, -0x32,0xd6,0xfd,0xe0,0x5a,0x01,0xd1,0x27,0x62,0x3a,0x57,0xca, -0xa7,0xc1,0xc3,0x14,0xce,0x4c,0x3c,0xcb,0x9a,0xcb,0x0a,0x37, -0x33,0x77,0xcb,0x05,0xc9,0xd6,0x60,0xfe,0x00,0x10,0xf0,0x65, -0x07,0x09,0xf5,0x25,0x0c,0xed,0xf4,0x02,0x84,0x4e,0xbb,0xf4, -0xc4,0xe5,0xe1,0x7e,0x5e,0x9e,0x7a,0xfa,0xcc,0xdf,0x68,0xe4, -0x89,0x8c,0xa0,0xb5,0x06,0x3e,0x64,0xd3,0xe4,0x64,0x50,0xa0, -0xfd,0x1d,0x0b,0x1c,0xaa,0x73,0x94,0xd1,0x77,0xe0,0x6e,0x23, -0xef,0xb4,0xd2,0xdd,0xd4,0x2b,0x95,0xb2,0xe0,0x96,0xd9,0xe0, -0x6d,0x27,0x16,0x9b,0xc0,0x62,0xb1,0x9b,0xec,0x7c,0xbb,0xcb, -0x2b,0xa5,0xd9,0x22,0xeb,0x20,0xff,0x06,0x90,0x6e,0x57,0x8a, -0xfd,0xfd,0xa5,0x23,0xda,0x3b,0x85,0x6f,0xf8,0x48,0xa4,0xb9, -0xa7,0xda,0xd5,0x95,0x1f,0xe6,0xc9,0xd5,0x35,0xb8,0x8f,0x6d, -0xdb,0x62,0x17,0x8b,0xf0,0x9a,0xb7,0x5a,0x01,0xcd,0xb2,0x24, -0xa2,0xe7,0xb2,0x6e,0x95,0x1a,0x5a,0x78,0x73,0x4d,0xcc,0x64, -0x29,0x16,0x66,0x55,0x95,0x36,0x59,0x39,0x3b,0x6e,0xa8,0xa5, -0x68,0x52,0x24,0xa9,0x80,0xb6,0x4a,0x26,0x3d,0x55,0xbd,0xdd, -0x78,0x5e,0xd7,0xab,0xb7,0x83,0xb2,0xe1,0x27,0x67,0xa7,0x49, -0xf6,0x91,0x6e,0x80,0x8f,0x3c,0x68,0x9f,0xeb,0x2e,0xa1,0xe5, -0x58,0xb9,0x48,0x33,0xd8,0x5d,0x0c,0x8d,0xed,0x5e,0x6a,0x72, -0x96,0xb3,0x05,0x79,0x38,0x83,0x2f,0x4a,0x3c,0xe0,0x8a,0x11, -0x87,0xf1,0xc4,0x8a,0x82,0xc2,0x2b,0x9e,0x3e,0xd8,0xf8,0x15, -0xf3,0x1b,0x2f,0x3f,0xe0,0x69,0x32,0xed,0xb8,0xb8,0x8c,0x52, -0xc3,0xa9,0x19,0x89,0x59,0xe8,0xdc,0x52,0x7c,0xc8,0x51,0xf7, -0xf4,0xee,0x35,0x9e,0xce,0x67,0x59,0x14,0x35,0xec,0x08,0xe1, -0x38,0xd2,0xcb,0xe2,0xf7,0x48,0x46,0xca,0x3c,0x9b,0xcc,0xfa, -0xa3,0x37,0xae,0x58,0x06,0x22,0x1d,0x13,0x22,0xc1,0x17,0xc2, -0x19,0xac,0x17,0xb4,0x55,0xd4,0xc9,0x44,0xe1,0x7c,0x88,0xeb, -0x45,0xfe,0x78,0x26,0x6f,0x4e,0x70,0xd6,0x0b,0x7e,0x06,0x4d, -0xa7,0x88,0x6a,0x67,0xf1,0x1a,0xc2,0x9f,0xda,0x93,0xa8,0xc1, -0x02,0xca,0xfe,0xd4,0x62,0xa6,0xba,0x96,0x6d,0xe2,0xb0,0xd1, -0x48,0x10,0x99,0xf4,0x02,0xc7,0x99,0x8c,0xcc,0x7b,0x17,0x44, -0xde,0xb6,0xaf,0xc4,0x0d,0x1c,0x5d,0xb5,0x33,0xb4,0x6d,0x73, -0x53,0x14,0x0a,0xd8,0x57,0x94,0xd3,0x10,0x01,0xab,0xdb,0x53, -0xdd,0x11,0x8d,0xac,0x0a,0xb6,0x52,0x3c,0x4c,0x2c,0x5e,0x47, -0x96,0xd4,0x29,0x8f,0xb4,0x56,0x5a,0xaa,0x11,0x0b,0xe7,0xb0, -0x4a,0x74,0xd8,0x55,0xab,0x0f,0xd1,0x72,0x94,0x9a,0x29,0x6c, -0xd1,0x9a,0x0b,0xb7,0x4d,0xaf,0x3b,0xd6,0x28,0xb4,0x2d,0x60, -0xbc,0xbb,0xb8,0x4c,0x0e,0xdf,0xed,0xea,0x76,0x52,0xde,0xa8, -0xbe,0xb3,0xe3,0x58,0x0b,0x53,0x1e,0xeb,0x5c,0x0e,0x73,0x7f, -0xdd,0xda,0x13,0x7b,0x71,0xdf,0x6c,0x32,0xad,0x3d,0x3f,0x1f, -0xbe,0xb6,0xdd,0xda,0x6a,0xc1,0x06,0x15,0x00,0x97,0xb3,0x67, -0xbc,0x77,0x97,0xd7,0x40,0x55,0xa9,0xbf,0x2c,0xef,0xde,0x3c, -0xa1,0x8f,0xbb,0x05,0x2c,0xe4,0x2d,0x47,0x7c,0xd2,0x4d,0x16, -0xf0,0xd3,0xd3,0x3f,0xdd,0xb8,0xfb,0xcb,0x49,0xef,0xa4,0xdb, -0x7b,0x90,0x9c,0xf4,0x20,0xb6,0x99,0xec,0x5c,0x06,0x6f,0x79, -0xf4,0x92,0x72,0xd3,0x52,0x37,0x7b,0xea,0xe6,0xa1,0xba,0x79, -0x64,0x91,0xde,0xb4,0x16,0x8b,0x9b,0x3d,0xfd,0xef,0xa1,0xfe, -0xf7,0x68,0x99,0xc8,0x2b,0x1b,0x21,0x89,0xaa,0xa4,0xa2,0xbb, -0x7d,0xa2,0x29,0xf9,0xe5,0xde,0x83,0x46,0xa7,0x19,0x27,0x8b, -0xee,0x49,0xef,0x76,0xd9,0x83,0xeb,0x97,0x93,0x93,0x7b,0x5b, -0xe1,0x15,0x4c,0x69,0xdd,0x2c,0xaf,0xfa,0xc0,0x9d,0x51,0x5b, -0xa9,0xb3,0x61,0xc1,0xb7,0x07,0x0a,0x9f,0xfa,0x1e,0x4f,0x7e, -0x18,0x82,0x3e,0x90,0x39,0xa4,0x1c,0x41,0xa4,0x51,0x58,0xb2, -0x0b,0x95,0x99,0x36,0xc1,0x8e,0x11,0xde,0x22,0xcb,0xf3,0xa3, -0x38,0x92,0xee,0xee,0xe8,0x3e,0xe3,0x07,0x92,0xe6,0xce,0x24, -0x7f,0x1c,0x22,0x27,0x66,0x46,0x50,0x91,0xed,0x2a,0x47,0x78, -0xe6,0x6c,0xc7,0x98,0xee,0xae,0x51,0x09,0x70,0x79,0x12,0x91, -0x5f,0x97,0xe3,0xaa,0x03,0x0b,0xe6,0xaa,0x2d,0x47,0xa2,0x30, -0x17,0x2a,0x22,0x0e,0x06,0xa5,0xa0,0xb3,0x91,0xc9,0x62,0xc2, -0xa9,0xa7,0x7c,0xd0,0xd6,0xee,0x83,0x00,0xba,0xe2,0x6e,0x5c, -0x05,0x20,0xe4,0x00,0x5f,0x22,0x52,0xfc,0x08,0x55,0xf4,0xe1, -0x62,0x21,0x02,0xfb,0x12,0x18,0xad,0x61,0xb8,0x34,0x1f,0x91, -0x1b,0x10,0xa2,0x1a,0x49,0xd2,0xe6,0xf6,0x3e,0x0f,0x87,0xcd, -0x62,0x21,0xc0,0x56,0x8d,0xc4,0xe0,0x24,0xcc,0x0b,0xbc,0xdb, -0x51,0x71,0xf2,0x64,0x79,0x31,0xc1,0x5d,0xca,0xdb,0x8f,0x60, -0xfc,0xf1,0x3b,0x39,0x1a,0x2f,0x8f,0xda,0xf9,0x3e,0x99,0x9b, -0x33,0x29,0x70,0x47,0x67,0x59,0x82,0x0b,0x40,0x03,0xa5,0xeb, -0xad,0x70,0xbb,0xd5,0x6b,0x30,0xc8,0xb7,0xbd,0xc5,0x22,0xda, -0x88,0x58,0x4c,0xb1,0x34,0x43,0xef,0x91,0xf0,0xd5,0x5d,0x4c, -0x50,0xc0,0x4e,0x36,0xca,0x3d,0x1a,0x04,0xf6,0x46,0x26,0xe1, -0x76,0x7b,0x8c,0xef,0x0e,0xf1,0xae,0x58,0xba,0x34,0x86,0x68, -0xe3,0xe7,0x28,0x59,0x39,0xe8,0xcc,0x7a,0x85,0x4e,0xc8,0x2b, -0x66,0x11,0xa3,0xad,0x6e,0x72,0x5c,0x99,0xfd,0x16,0x47,0x56, -0x20,0x68,0x6e,0xbd,0x99,0xf7,0x8b,0x9b,0x57,0xc3,0xd9,0x7f, -0x15,0x93,0xb1,0x68,0x77,0x16,0xe7,0x50,0x9e,0xcc,0xe9,0xeb, -0x75,0xca,0x74,0xa4,0x8c,0xa8,0x22,0x29,0x6e,0x70,0x1a,0x08, -0xa1,0xd3,0x88,0x08,0x76,0x2a,0x82,0x88,0xbd,0x53,0x3f,0x50, -0x52,0x5f,0xa3,0xb9,0xe3,0x56,0x7e,0x3a,0xd6,0x98,0x6b,0xf7, -0xba,0xf5,0x1f,0x10,0x08,0xdd,0xe5,0x4a,0x02,0xd9,0x37,0x88, -0x31,0x06,0x14,0xd0,0x56,0x7d,0xd8,0xa8,0x92,0xb2,0x85,0x0a, -0xce,0xa6,0x56,0xf1,0x67,0x75,0xad,0x06,0xea,0x4a,0x1d,0xaa, -0x42,0x5d,0xb3,0xce,0x2e,0x1e,0x35,0x6d,0xd6,0x00,0xc2,0x70, -0xe1,0xad,0xef,0xbf,0xdb,0xdd,0xff,0xbc,0xb5,0x85,0x67,0xc6, -0x42,0x30,0xd1,0x9f,0xa9,0xbe,0x7e,0x59,0xf6,0xd0,0x5a,0x55, -0x82,0xd1,0xcd,0x0b,0xec,0xe8,0x9a,0x17,0x57,0x75,0x25,0x4a, -0x86,0xbf,0x54,0xa8,0x27,0xe5,0xa0,0x2b,0x00,0xa3,0x15,0x42, -0x36,0xf6,0x2b,0x7b,0x9b,0xe6,0x18,0x28,0x33,0xb1,0x1c,0x12, -0x40,0x69,0x94,0x99,0xdc,0x4d,0x16,0x47,0xf8,0x92,0xf9,0xb2, -0xc9,0x7e,0x8c,0x17,0xea,0x53,0x34,0x4f,0x08,0x97,0x15,0x9d, -0x10,0x61,0xad,0x5d,0x5b,0xc7,0xdc,0xd7,0xd4,0x45,0x16,0x36, -0x9f,0xe9,0xb3,0x17,0x4c,0xd9,0x27,0x97,0x13,0x9e,0x55,0x8a, -0x9f,0x6c,0xb0,0x37,0x95,0xa3,0xf9,0xc5,0x45,0xfe,0x09,0x7d, -0x93,0x6a,0x2c,0x43,0xbe,0x86,0x96,0x57,0xcb,0xd3,0xe1,0x08, -0x0d,0xd3,0xa0,0x6e,0x16,0x65,0x06,0x79,0xcd,0xe6,0xa6,0x8f, -0x81,0xb1,0x1b,0x44,0x1e,0x13,0x20,0x10,0x78,0x4f,0x29,0x7c, -0x0c,0x54,0x34,0x99,0x07,0xa7,0x17,0x18,0x53,0x7c,0xb6,0x62, -0xb5,0xdd,0x65,0xde,0x64,0xa7,0x35,0x7c,0x98,0xd8,0x5b,0x21, -0x82,0xe4,0x34,0x3c,0xc3,0xd3,0xc3,0x08,0x8a,0x67,0x28,0x8c, -0xc7,0xe7,0x1b,0x14,0x8d,0x20,0x3b,0x7b,0x8f,0xcc,0xab,0x2c, -0x8a,0x05,0x80,0x9d,0x87,0xdf,0x7e,0x93,0x90,0x65,0xda,0x47, -0xdf,0xec,0xdb,0x47,0x0f,0xdd,0xa8,0x88,0xcc,0x4b,0x28,0xb0, -0x68,0x0f,0x4a,0x5f,0x84,0x1a,0xd2,0xae,0x23,0x1d,0xa7,0xc1, -0x65,0xd4,0x75,0x64,0x48,0x64,0x78,0x2c,0x13,0x92,0xae,0x00, -0x7a,0x6f,0x6f,0xdf,0x45,0x5c,0x45,0x44,0x34,0x43,0xd2,0x3b, -0x8e,0x6e,0x34,0x40,0xc0,0x6f,0xf6,0x5d,0xc4,0x20,0xc2,0x5a, -0x50,0x69,0x36,0xfa,0x10,0xe0,0x1e,0xea,0x18,0x1b,0x71,0x18, -0x29,0x59,0xa5,0xdd,0x84,0x31,0x63,0x6d,0x01,0xe2,0x33,0x64, -0xd1,0x3f,0x9f,0x23,0x6a,0x01,0xdf,0x82,0xe9,0x5e,0x2f,0xf3, -0x3b,0x08,0x63,0x1f,0xf6,0x32,0xd9,0x0f,0xfb,0xbb,0x14,0xfd, -0xa8,0x87,0xc3,0x5c,0x95,0x86,0x1d,0xcf,0x4c,0xcd,0xcc,0x90, -0xc9,0x56,0x3b,0x2c,0x60,0x32,0x90,0xc4,0x89,0xd4,0x42,0xc0, -0x55,0x16,0xca,0xad,0x60,0xea,0x05,0x41,0x96,0x4c,0xd1,0x15, -0xdc,0xe0,0xd7,0x79,0x31,0x1b,0x0e,0x98,0xb5,0x43,0xe3,0x90, -0x15,0xc0,0xdb,0x01,0x42,0x68,0x83,0xe3,0x89,0x0d,0x57,0xe7, -0xa1,0xc7,0x32,0x9f,0x63,0x3b,0xa9,0x82,0x5c,0xfb,0x43,0x30, -0xf3,0x1f,0x07,0xb1,0xdb,0xd9,0x77,0x09,0xd0,0xb1,0xfd,0xdd, -0x7e,0x90,0x52,0x86,0x6d,0x00,0xac,0x5f,0x05,0xa3,0x0f,0x0e, -0x6a,0x37,0x4d,0x7d,0x96,0x8a,0x23,0xee,0x60,0x91,0x2d,0x51, -0xb7,0xd0,0x5e,0x42,0xb9,0xd9,0xc7,0xe1,0x04,0x92,0x71,0xb2, -0xf3,0x1d,0x0d,0xe8,0x34,0x00,0xf9,0x8c,0xa9,0x9e,0xc0,0xa9, -0x24,0x43,0x45,0x09,0x90,0x27,0x3a,0xad,0x6a,0xa7,0x20,0xd2, -0x2c,0xcc,0x40,0xf1,0x93,0xc1,0x40,0x59,0xbc,0x6a,0x50,0x5e, -0xa9,0xa9,0x08,0xa9,0x3d,0x2d,0x9a,0x3b,0x10,0x39,0x53,0xa8, -0xc3,0xbf,0x69,0x65,0x8f,0xea,0x02,0x83,0xbe,0xdc,0x1e,0x34, -0xe2,0xc1,0x7e,0x15,0xdd,0x9d,0xef,0x52,0x5a,0x01,0x58,0x38, -0xfa,0xdd,0x83,0x18,0x70,0x6f,0xb7,0x92,0x46,0xcc,0x85,0x04, -0xc3,0x86,0x1e,0x4d,0x61,0x29,0x8d,0x96,0xba,0xc5,0x66,0x75, -0x22,0xd9,0xdd,0x0e,0x46,0xc0,0x9f,0xed,0x96,0xc3,0xeb,0x41, -0x78,0xb1,0x52,0x80,0x0b,0xa5,0xda,0xc4,0xa5,0xbf,0xe7,0x05, -0xaf,0x55,0xab,0x98,0x1c,0x56,0xbe,0xb1,0x7c,0x97,0xb9,0xf9, -0xb6,0x0c,0x8d,0xdb,0x9f,0xdc,0x19,0x66,0x6b,0x6b,0xb5,0x72, -0x31,0xbb,0x42,0xcb,0x98,0x7f,0xa1,0x71,0xc3,0xe7,0x9e,0xf8, -0x56,0x68,0x0a,0xef,0x2e,0x93,0xb4,0x66,0x23,0x14,0xcc,0x60, -0x4e,0x88,0x2a,0x44,0x50,0x49,0xf3,0x46,0x1f,0x01,0xc5,0x36, -0x88,0xcf,0xbf,0xb0,0xb4,0x82,0x2b,0xef,0xb3,0x46,0xe2,0x50, -0xc4,0xfb,0x66,0x2d,0xcf,0x03,0x37,0x47,0x49,0x4a,0xed,0xd3, -0x31,0x8c,0x16,0x2b,0x2d,0x55,0xf0,0x64,0xe1,0x31,0x32,0x5d, -0x83,0x73,0x4d,0x6b,0x19,0x52,0x05,0xc4,0xf8,0xdd,0x17,0xf0, -0x4e,0x3f,0xe2,0x93,0x0b,0x8d,0xf2,0x88,0xde,0x5e,0xc0,0x23, -0x73,0x32,0x74,0x65,0x35,0x1f,0xf9,0xe1,0xb9,0x1f,0x6e,0xe8, -0xcd,0x47,0xbc,0x6e,0x61,0x25,0x30,0x18,0x1a,0xf3,0xd9,0x39, -0xbf,0xc3,0xcf,0x0b,0x10,0x6b,0x46,0xfa,0x4f,0x94,0x3a,0xcb, -0x31,0xe2,0x72,0xbd,0xc3,0xaf,0xd3,0x07,0x7a,0x2b,0x1b,0xce, -0xa2,0x86,0xce,0xda,0x00,0x8d,0x54,0xc3,0x8c,0x54,0xbd,0xe0, -0xa3,0x07,0xe4,0xf8,0x37,0xb5,0xb9,0x2f,0xbd,0xdc,0x49,0x30, -0x7e,0xcd,0xb3,0xcc,0x1f,0x5d,0x15,0x6d,0xf5,0xec,0x23,0x49, -0x5b,0xcf,0xba,0xa7,0xec,0xf0,0x1e,0x97,0x40,0x6a,0xd0,0x3f, -0x29,0x44,0x01,0xea,0xa2,0x7f,0x8e,0x66,0x54,0x2b,0xca,0x89, -0xfa,0x65,0x6b,0x74,0xa6,0xb0,0x06,0x14,0xb6,0x43,0x99,0x83, -0x72,0x7e,0x44,0xe3,0x9c,0x1a,0xb2,0x6a,0x56,0xa1,0xd1,0x09, -0x3d,0xd8,0x91,0xd6,0x6b,0xb9,0x92,0x27,0x4b,0xf3,0x02,0x96, -0xc2,0x2a,0x57,0xff,0x78,0xf6,0xf6,0xe8,0xc5,0xeb,0x57,0x59, -0xb4,0xd7,0x7c,0xd8,0xdc,0xe5,0x7d,0x99,0xde,0xf1,0xe1,0xa7, -0x82,0x5b,0x9f,0x6c,0x57,0xe1,0x85,0x4b,0xd6,0x52,0x70,0xa9, -0x92,0xed,0x29,0x10,0x03,0x65,0x0f,0x15,0xdd,0x44,0x64,0x8f, -0x14,0x5d,0x99,0x67,0xdf,0x28,0x71,0x81,0x9e,0x7d,0xeb,0x54, -0xb0,0xe1,0xf5,0xb2,0xd5,0x7a,0xce,0x04,0xd5,0x96,0xb9,0xbc, -0xc6,0x24,0x78,0xaa,0x09,0xbf,0xfa,0x48,0x74,0x33,0x99,0xce, -0x0a,0x15,0x1c,0x95,0xb2,0x9d,0x5f,0x4e,0x76,0x3a,0x30,0xc0, -0x4f,0xe2,0xf8,0x64,0xbb,0x73,0x32,0x68,0x24,0x3b,0x39,0x43, -0xc1,0xc6,0x7e,0x74,0xd3,0x1f,0x0b,0x68,0x0d,0xa3,0x67,0x6a, -0x27,0x8d,0x4f,0x06,0x0f,0x92,0x93,0xa6,0xfe,0x86,0x1c,0x27, -0x29,0xfe,0xe8,0x78,0xfe,0x3a,0x69,0x42,0xc2,0xed,0xc3,0x65, -0xd2,0x49,0x3a,0x3b,0x0a,0x4e,0xb2,0xdc,0x49,0xa6,0xd4,0x58, -0xe3,0x79,0x03,0x88,0xf4,0x7f,0xdd,0xdd,0xed,0xef,0x55,0xb3, -0xf7,0x20,0xf9,0x17,0xe1,0xb6,0xe1,0xc3,0x20,0x7c,0x80,0xe1, -0x63,0x2f,0xee,0xa7,0x3b,0xf2,0x1c,0x01,0x09,0x0b,0x17,0x7e, -0x9f,0xdc,0xdb,0x51,0xa1,0x5c,0x1a,0x2a,0xd6,0xed,0xfe,0x02, -0x42,0x31,0x90,0x91,0xc5,0x27,0x27,0x1a,0xcb,0xe1,0x64,0x71, -0x78,0xd8,0x81,0xff,0x2f,0x0e,0x26,0x8b,0x83,0x03,0xfc,0xd3, -0x81,0xff,0x2f,0x06,0x83,0x41,0x67,0xa0,0x7f,0x26,0x9d,0xc5, -0xc7,0xee,0x64,0xf1,0xb1,0xd7,0x59,0xbc,0xd7,0xbf,0xef,0xf5, -0x2f,0x6a,0x94,0x2c,0xf8,0xcf,0xe2,0xf2,0x32,0xbe,0xbc,0xbc, -0x04,0x0a,0x7e,0xfc,0x31,0xfe,0xf1,0xc7,0x1f,0xe1,0x6b,0xb8, -0x78,0xb6,0xe8,0x2f,0x9e,0x2c,0xae,0xae,0x3a,0x8b,0x9f,0x7e, -0xea,0x2c,0xae,0xaf,0x3b,0x8b,0xa2,0xe8,0x2c,0x8e,0x8e,0x3a, -0x47,0x9d,0xc5,0x3f,0x17,0x7f,0xfc,0xd1,0x59,0xfc,0xfc,0x73, -0x67,0x81,0x72,0xb9,0x4a,0x8d,0xc9,0x4a,0x72,0x5f,0x1e,0x2f, -0x5e,0xbe,0xec,0xc0,0xff,0x17,0xa3,0xdb,0x96,0x7a,0xb4,0x84, -0xec,0x75,0x6a,0x65,0xd9,0xce,0xc9,0xe0,0x64,0xa0,0xbb,0xa6, -0x5e,0x97,0x10,0x40,0x34,0x9e,0x87,0x4b,0x09,0x14,0xa6,0xfb, -0xa9,0x4e,0x1d,0x92,0x33,0x3f,0xf2,0x92,0xad,0xb6,0x63,0xb6, -0xd3,0x6d,0x9c,0x6c,0xf7,0x3a,0x08,0xf3,0xad,0x07,0x03,0xca, -0x87,0x3a,0x59,0xf7,0x56,0xef,0x41,0xf7,0x7e,0x7f,0xfb,0x8f, -0x93,0xf9,0xee,0xee,0x93,0xdd,0x6d,0xfd,0xf3,0xcd,0xf3,0xe7, -0xfa,0xef,0x77,0xbb,0x10,0x38,0xf8,0x0e,0x02,0xcf,0xbf,0xc7, -0xc0,0xf3,0x83,0xa7,0x10,0x38,0x78,0x8e,0x81,0xe7,0xcf,0x9e, -0xf7,0x1a,0x8b,0xae,0x06,0xfd,0x16,0x53,0x77,0xbf,0xd5,0xa0, -0x3b,0xbd,0x46,0x7c,0x52,0x3c,0xe8,0xf8,0xd1,0xbd,0x46,0xa2, -0x09,0xd8,0x5b,0xee,0xe4,0xaa,0x2c,0xf8,0xc8,0x76,0x7e,0xd6, -0x58,0x80,0x4e,0x68,0xaa,0xb4,0x03,0x7f,0x7d,0xc0,0x6c,0xe7, -0xb8,0x94,0x93,0xf5,0x1a,0x75,0x15,0x4e,0xb8,0x8a,0xba,0xe0, -0x26,0xb7,0x24,0xcf,0x05,0x3b,0xf3,0x8a,0x07,0x3a,0xe1,0xd1, -0x72,0x9b,0x26,0xd4,0xc9,0x60,0x1b,0xfe,0xe8,0xfe,0x7c,0x0f, -0xbf,0xf7,0xcc,0xc7,0x36,0xc6,0xc1,0x97,0xfe,0x48,0xe2,0xf8, -0x78,0xb1,0x91,0x60,0x30,0x4e,0xc5,0x5f,0x28,0x44,0xf7,0xa8, -0xfe,0x2f,0xe1,0xff,0xc7,0x21,0xf5,0x10,0x79,0x0f,0x29,0x60, -0xd1,0x26,0xaa,0x3f,0x6d,0x1f,0x1e,0x6e,0x1f,0x1c,0x1c,0xff, -0xf4,0x53,0x7a,0x7d,0x9d,0x16,0xc5,0xcf,0x91,0x32,0x82,0xa7, -0xac,0x2b,0x00,0x22,0x85,0x6a,0x27,0xdb,0xdd,0xf7,0xbd,0xf7, -0xef,0xbd,0xc0,0xf6,0x33,0x1d,0x44,0x40,0xd0,0xf0,0xea,0x29, -0x23,0x58,0xca,0xba,0xdd,0xc8,0x60,0x6d,0x1e,0x45,0x6a,0x07, -0x29,0x47,0x72,0xec,0x1f,0xdb,0x34,0x3b,0x3d,0xe5,0xa0,0xab, -0x61,0x1d,0x48,0x29,0x9d,0x93,0x64,0xfc,0x4e,0x8f,0xaf,0x7e, -0x02,0x5d,0x43,0x3d,0x77,0xa8,0x5d,0x16,0xd8,0x24,0x3b,0x97, -0x39,0xbc,0xe3,0xfd,0xf4,0x99,0x36,0x98,0xc2,0x6e,0xd7,0x45, -0x86,0xb6,0x4f,0x17,0x3f,0xc1,0x49,0x6d,0xc1,0xda,0x42,0x8b, -0x23,0x3a,0x0c,0xe9,0xb0,0x3b,0x19,0x45,0xe6,0x8d,0xc5,0x02, -0x6d,0x0e,0xe6,0x33,0x91,0xf8,0x1c,0x77,0x1b,0xbd,0x5c,0xcb, -0x0c,0x69,0x4b,0x1d,0x99,0xaf,0xa1,0x5e,0xf4,0x59,0x67,0xf1, -0xdb,0xe1,0x23,0x85,0xa5,0xa5,0x60,0xe7,0x42,0x69,0xd6,0xb4, -0x48,0xff,0xfe,0xed,0x23,0xfd,0x89,0x57,0x8d,0x45,0xba,0xf7, -0xcd,0xf7,0x7b,0xc3,0x6f,0x15,0xb0,0x91,0x1a,0xa6,0xf5,0x8d, -0x06,0xfb,0x76,0xa9,0xc4,0x2b,0xe8,0xec,0xf6,0xba,0x48,0x23, -0x71,0x6a,0xd3,0x87,0xd8,0x34,0x32,0x9f,0xd7,0x90,0x04,0x45, -0xe9,0xe3,0x67,0x1a,0xc1,0x09,0x54,0x9f,0x3e,0x52,0x7c,0x9b, -0xab,0x0e,0xe0,0x17,0x52,0x3e,0xa6,0xf4,0x86,0x57,0xbd,0x4f, -0x23,0xdd,0x91,0xef,0xf1,0xfb,0x30,0xe5,0x17,0xb8,0xea,0x73, -0x1a,0x01,0x4b,0xab,0x33,0x1c,0x60,0x16,0xe6,0x6a,0x23,0x35, -0xa4,0x7c,0x88,0xec,0x99,0xcd,0x8a,0xc1,0xcb,0x4b,0x4a,0x23, -0xc0,0x1f,0x7f,0xb4,0xa9,0x18,0x01,0x9e,0x55,0xe4,0x93,0xed, -0xec,0x56,0x67,0x9a,0x5c,0x20,0xbf,0x2d,0x0b,0xd0,0x79,0xcc, -0xf1,0xc0,0xc3,0xce,0xf1,0x82,0x5a,0x08,0x52,0x76,0x57,0x2a, -0x43,0x51,0xb4,0x5f,0x7c,0xa0,0xdf,0x0f,0x3b,0x6b,0x60,0xc7, -0x06,0x86,0xc2,0xc1,0xc1,0xc6,0xc7,0x8d,0xf7,0x1b,0x87,0x1b, -0x07,0x1b,0x03,0xdb,0xe1,0x1b,0xba,0xc3,0x6f,0xf4,0x41,0xce, -0x88,0xba,0xb3,0x08,0xd2,0x7f,0xda,0xb8,0xda,0xb8,0xde,0x28, -0x00,0xde,0x83,0xac,0xba,0x9e,0xd3,0x03,0xa3,0xce,0xc0,0x8f, -0xb1,0xcf,0xd0,0xfa,0x8f,0xa5,0x3a,0x3c,0x14,0x60,0xc1,0xfb, -0x08,0x69,0x3d,0x87,0xf4,0x41,0x8f,0xae,0xc0,0xa9,0x02,0x19, -0x2b,0x24,0x58,0xc4,0xf0,0x25,0x28,0x82,0xdc,0x07,0x75,0x44, -0x92,0xc1,0x89,0x25,0x0e,0x87,0x5a,0x10,0x7b,0x60,0x5d,0xea, -0xf1,0x56,0x0f,0x85,0xe9,0x83,0xb5,0x88,0x34,0xf2,0x35,0x3d, -0x75,0x02,0x4a,0x07,0x5f,0x88,0xa1,0xaa,0xad,0x06,0x5f,0x8a, -0x24,0xc8,0xff,0xb1,0xae,0x92,0x00,0x0f,0xb5,0x7c,0x5f,0x07, -0xc0,0x23,0x13,0x60,0xfe,0xf5,0xaf,0x0a,0x20,0xcf,0x24,0x0b, -0xe6,0xa0,0xb3,0x3e,0xbc,0x11,0x56,0x7b,0x98,0xeb,0x8b,0xf2, -0xa9,0x47,0x9c,0xe7,0x8b,0x32,0x7d,0xa3,0x33,0xe9,0x49,0xbd, -0x4e,0x0e,0xa7,0xfc,0x66,0x49,0x04,0xdd,0xc6,0x55,0x0d,0x64, -0x86,0xca,0x65,0x0d,0xe0,0xaa,0x42,0x90,0x34,0xbd,0xc2,0xac, -0x93,0x4b,0xac,0x02,0x82,0x3a,0xd8,0xd6,0xee,0xe8,0x1d,0x43, -0xe0,0x8f,0x35,0xb0,0x77,0x14,0x85,0x34,0xd6,0x9a,0xf5,0xe2, -0x21,0x05,0xf8,0x9f,0xdd,0x41,0x07,0x83,0xf5,0xeb,0xc0,0xcc, -0x84,0xe6,0x47,0x19,0x44,0x0a,0x0a,0x1e,0x63,0x3a,0xe5,0xb1, -0xca,0x0e,0x04,0x37,0x77,0x35,0xa6,0x27,0x7f,0x09,0xa6,0x96, -0xc6,0xf4,0x53,0x1d,0x26,0xce,0xa4,0x0f,0x2f,0x77,0x40,0xfc, -0xad,0xb5,0xb7,0x58,0xb4,0xf6,0x96,0x7a,0xd7,0xaa,0x5b,0x1f, -0x4d,0x91,0x4b,0xbd,0xc9,0xd5,0xc0,0xf0,0x86,0x0b,0x30,0x47, -0x55,0x30,0xa8,0xef,0xc5,0xd8,0xdc,0xfe,0x1c,0x27,0x3b,0x7a, -0x38,0x40,0x9e,0xaa,0x4c,0x7e,0xf7,0xae,0xc0,0x90,0xe0,0x80, -0x3a,0x5a,0x03,0x49,0x39,0xbb,0x7a,0xa8,0xb3,0xfe,0x9c,0x06, -0x76,0xf8,0xfa,0xd9,0x36,0xc2,0x02,0x3b,0x03,0xe6,0xf6,0xb3, -0xa8,0x61,0xcf,0xfe,0xbb,0xfb,0x7d,0xf4,0xe3,0xb1,0xdd,0x87, -0x78,0x78,0x38,0x7a,0xd6,0xa8,0x20,0xb5,0x8f,0xb2,0xf1,0xbd, -0xa4,0x11,0xa5,0x51,0x65,0x3a,0xa8,0xbb,0x21,0xe1,0x3f,0xff, -0x3b,0x8a,0x6f,0xed,0x3e,0xd0,0x14,0xd0,0xba,0xf3,0x47,0x5d, -0xaf,0x01,0xfe,0x27,0x67,0x67,0x38,0xc9,0xfe,0x58,0x09,0x05, -0xfe,0x19,0x00,0xea,0x9f,0x75,0x40,0x9a,0x39,0xfa,0x04,0x42, -0x0b,0x05,0x5e,0x9b,0x81,0xa1,0xa5,0xed,0x4d,0x33,0xac,0x62, -0xab,0xd4,0x21,0xb3,0x92,0x8b,0xcf,0x30,0x49,0xef,0x35,0x51, -0xaf,0x1d,0xb2,0x06,0xf6,0x59,0x6f,0x9e,0x95,0x92,0xc8,0x0f, -0x42,0xa5,0x56,0x4e,0xde,0x88,0x26,0x51,0x2f,0xcc,0x12,0x57, -0xc3,0xf6,0xc0,0x5c,0x08,0xea,0x47,0x4a,0x66,0x43,0x94,0xec, -0x45,0xaf,0x2c,0x35,0xef,0x65,0xce,0x06,0x60,0x4d,0x61,0x7b, -0x54,0x58,0x55,0x72,0x13,0x1e,0x71,0xac,0xc6,0x00,0x20,0x7a, -0xf4,0x1a,0x13,0x7b,0xe6,0x81,0xb2,0x34,0x82,0x58,0x0c,0x67, -0xae,0xbf,0xa4,0xe8,0x13,0x0d,0xea,0xe4,0xa4,0x4f,0x01,0xc6, -0x73,0x38,0x0d,0xa2,0x59,0xfe,0x09,0xf4,0x55,0x18,0xe4,0x07, -0x88,0x0e,0x74,0x38,0xa8,0x89,0x42,0x00,0xe5,0x57,0xdd,0xe8, -0x34,0x6a,0x70,0xc4,0x52,0xb1,0xa9,0xae,0x34,0xfa,0xaf,0xfe, -0x78,0xde,0x9f,0x7e,0x3e,0x7d,0x3e,0x3c,0x9b,0xe2,0xc7,0x61, -0x7f,0x7a,0x7e,0x75,0xfa,0xe4,0x66,0x9a,0x8f,0xf4,0xf7,0xe7, -0xd3,0xff,0x9a,0x8f,0x87,0xfa,0xcf,0xe8,0xf3,0xe9,0x93,0xf9, -0xe5,0xbc,0x98,0x9d,0x1e,0x0d,0x6f,0x66,0x43,0xb8,0x0f,0x3b, -0x7d,0xad,0xd9,0x79,0xf8,0x7d,0x35,0xf9,0x9d,0x22,0x0e,0x86, -0xe7,0xf8,0x61,0x39,0xbd,0xd3,0xc8,0x18,0xdc,0x72,0x75,0xbc, -0x0e,0xe4,0x5d,0x94,0xde,0xb5,0x36,0xb8,0x7a,0x96,0x3a,0x1c, -0x74,0x48,0x22,0x90,0x07,0x94,0x01,0x5d,0x86,0x2a,0x20,0x0a, -0x68,0x02,0x82,0x80,0x14,0x20,0x03,0x28,0xa8,0x28,0x9c,0x10, -0xdd,0x41,0x01,0x02,0x79,0x64,0x88,0x07,0x45,0x22,0x33,0x44, -0xbe,0xb2,0x06,0x35,0x73,0x10,0x1e,0x2a,0xd4,0xf8,0x21,0x75, -0x40,0x81,0x11,0x73,0x82,0x03,0xc4,0x30,0x2e,0xeb,0xf6,0x48, -0x51,0xb8,0xb5,0xe7,0x54,0x15,0x4a,0x50,0x68,0x8f,0x0a,0xee, -0x47,0x8c,0x48,0x14,0x0c,0xe7,0x74,0xf7,0xf4,0x21,0x29,0xef, -0x25,0x54,0x64,0x16,0xfd,0x12,0x35,0x1c,0x83,0x5c,0xe0,0x9d, -0x12,0x9a,0x7d,0x5f,0xf8,0x09,0xc4,0xd9,0x71,0xaa,0xaa,0x2a, -0xaa,0xf4,0x94,0xce,0x3d,0x97,0x6b,0xe2,0x6b,0x39,0x7a,0x2c, -0x57,0x99,0x97,0x5e,0xc8,0xba,0x96,0xb1,0xae,0xb6,0x75,0x57, -0x9a,0x15,0x23,0x8d,0x8e,0xe6,0x63,0xfd,0x71,0xaa,0xcf,0x6f, -0xf0,0x73,0x3c,0x1f,0x16,0xf0,0xfb,0x7e,0x38,0x18,0xd3,0xd7, -0xf1,0x95,0xde,0xec,0xe0,0xe3,0xb9,0xde,0x59,0xf5,0xcf,0x51, -0x5f,0x23,0x81,0x63,0x8d,0xec,0x50,0x8b,0xae,0xb6,0x37,0x0d, -0x44,0x97,0x2f,0xb5,0x7a,0x82,0x08,0x1e,0x51,0x9a,0x12,0x20, -0x03,0x68,0x80,0xf2,0xa1,0x64,0x28,0x14,0x4a,0xac,0x2c,0xec, -0x8e,0xf1,0xe3,0x81,0x55,0x15,0xab,0x57,0x4b,0x28,0x54,0x97, -0xa9,0x8b,0xd4,0x25,0xea,0x02,0x75,0x79,0xba,0xb8,0xca,0xd2, -0x0e,0xa5,0x51,0xd4,0xba,0xb2,0x34,0x90,0x28,0xc9,0xbb,0x62, -0x77,0x99,0x39,0xfa,0xae,0xd1,0xea,0xe5,0xb6,0xe3,0xd5,0x8b, -0xb5,0x23,0xf6,0xbb,0x70,0xc0,0x7a,0x60,0xe1,0x90,0xa5,0xe1, -0xda,0xea,0x25,0x48,0x69,0x5e,0x1a,0xb5,0xf6,0xcc,0x50,0x1e, -0xb7,0xfe,0x99,0xa4,0x3e,0x1d,0x4e,0x3d,0xfe,0xb8,0x0e,0x29, -0xfa,0xb2,0x91,0x1d,0xe6,0xa6,0xb1,0x2d,0x5b,0x52,0x8e,0x6e, -0xff,0x05,0x7a,0x7a,0xfb,0xf2,0x38,0x8d,0xae,0xd2,0xeb,0xeb, -0x8d,0x27,0x91,0x7a,0x99,0x46,0x87,0x87,0x3b,0x07,0x07,0x3b, -0xf8,0xac,0x4e,0xbd,0xc4,0xf0,0x21,0x1c,0x8a,0x4d,0x84,0x1f, -0xb3,0xf1,0xf2,0x18,0x23,0x75,0x2c,0x1c,0xba,0xd4,0x46,0x90, -0xa6,0xf7,0x6d,0xbf,0x34,0x67,0x07,0xc4,0xd8,0x0c,0x61,0xd3, -0x05,0x54,0x13,0x1f,0x1a,0x4d,0x70,0x18,0xe7,0x3d,0x04,0xb7, -0xb5,0x55,0x07,0xd8,0x9c,0x4d,0xde,0x81,0xef,0x51,0xb2,0xbf, -0x82,0x2e,0xe4,0x56,0x63,0x0e,0x32,0x38,0x4d,0x47,0xa8,0xc3, -0x42,0xff,0x77,0x70,0x00,0x72,0xe2,0x81,0x54,0xae,0x0c,0x5e, -0x1c,0xb1,0xf6,0x4f,0x0b,0xfc,0xd3,0xd4,0x56,0x20,0x23,0x3a, -0x12,0xc5,0x76,0x13,0x14,0xbc,0x25,0x4c,0xc3,0xeb,0x21,0xbe, -0x14,0xbc,0x81,0xe7,0xb0,0x14,0x87,0x6f,0x6e,0x7d,0xa3,0x32, -0xa0,0xc4,0xf7,0x64,0x16,0x03,0x6b,0xeb,0xbf,0xd5,0x4e,0x77, -0xba,0xfd,0x9b,0xde,0x49,0xb3,0x73,0xdd,0xd1,0x7f,0x76,0xf4, -0x9c,0xe1,0x54,0x57,0x8e,0xb1,0x9b,0x49,0x56,0x34,0xf3,0x02, -0xf1,0x3a,0x9f,0x2d,0x90,0x0a,0x06,0xa3,0x38,0xa1,0x13,0xdd, -0x5c,0x6b,0x0e,0xf3,0xcd,0x61,0x94,0xda,0x98,0x3e,0xc4,0x3c, -0x39,0xd4,0x9d,0x7a,0x7a,0xde,0xd7,0xcc,0xcb,0xa0,0x3f,0x4d, -0x6f,0x0b,0x3d,0xbc,0x0e,0x40,0xa6,0xd3,0x3d,0x9e,0xe8,0xd1, -0xb6,0xd1,0x9f,0xf5,0x70,0x50,0x80,0xfd,0x10,0x13,0x7f,0x3d, -0x99,0x82,0x41,0x71,0x99,0xf4,0x1e,0x45,0x3e,0xd0,0xbc,0x1b, -0x5d,0x13,0x0f,0x66,0x04,0x28,0xcb,0xbf,0xf4,0xf8,0x1d,0x4e, -0x25,0x3a,0x48,0xa3,0x3c,0xdd,0x97,0xfa,0xb3,0xb7,0xe1,0x67, -0x05,0x2a,0x9e,0x8d,0x74,0x3b,0x44,0x2f,0x51,0x20,0xc5,0xd4, -0xc9,0xd1,0xa6,0x42,0x63,0x19,0x6c,0xb2,0x99,0x61,0xe5,0x58, -0xab,0xf2,0x18,0x84,0x79,0x3a,0xff,0x1f,0x73,0x57,0xbf,0xd3, -0x48,0x8e,0xc4,0x5f,0x05,0x5a,0x0a,0x8a,0x49,0x37,0x24,0x0c, -0x03,0xab,0x66,0x9a,0xd6,0xdc,0x8d,0x74,0xda,0x93,0x98,0x59, -0x89,0xf9,0xe7,0x06,0x21,0x94,0xd9,0xc0,0x80,0xb4,0x81,0x15, -0x61,0x75,0xcb,0x91,0xbc,0xfb,0xb9,0xbe,0xec,0x2a,0xb7,0x3b, -0x30,0x8c,0xb4,0x5a,0x21,0x91,0xc4,0x2e,0xdb,0xd5,0x6e,0x7f, -0x94,0xcb,0x55,0xbf,0x62,0x5c,0x54,0xb2,0x94,0x80,0x0a,0x6b, -0x79,0xa9,0x17,0xda,0x78,0xa6,0x7e,0xba,0x42,0xdb,0x9a,0xba, -0xf0,0x42,0xd0,0xc0,0x0b,0xa5,0xbf,0x7b,0x9e,0xeb,0x62,0xb0, -0xd8,0x98,0x7e,0xbb,0x43,0x9d,0xde,0x74,0xe3,0xca,0xcf,0x70, -0x51,0x40,0x82,0x6a,0x6f,0xba,0x21,0xca,0xbd,0xb9,0xff,0x35, -0x98,0x09,0xe0,0x29,0x2a,0xfb,0xa6,0xf4,0x86,0xfc,0xf7,0x6b, -0xcc,0xc3,0xd7,0x85,0xda,0xbf,0xe9,0x06,0x6a,0xd1,0x66,0x33, -0x4c,0x27,0x11,0xf8,0x04,0x6b,0x23,0x85,0xdf,0xc9,0x09,0x55, -0xc6,0x62,0xf3,0x23,0x64,0x91,0x06,0xf0,0xf1,0x11,0x73,0xd0, -0x9e,0xc4,0x77,0x9a,0xe1,0x3f,0x74,0x5c,0xd6,0x06,0x29,0xb5, -0x20,0xea,0xf6,0xaa,0xae,0xec,0x8c,0xc8,0x5f,0xda,0xb9,0x2f, -0x6b,0x92,0xfb,0x3d,0x4e,0xda,0x01,0xe8,0xf6,0x05,0x02,0x17, -0xfb,0x9b,0x28,0xe3,0xa3,0x00,0xc6,0x25,0xcf,0x41,0x0e,0x44, -0x42,0x4a,0xf4,0x0c,0xc7,0x40,0x7a,0x3c,0x6e,0x0b,0x7a,0x8b, -0x7e,0xd4,0x43,0x7d,0xc5,0x9a,0x27,0xe0,0x3b,0x6c,0xfa,0xe0, -0x85,0x47,0x6e,0xb6,0x15,0x8f,0x0b,0xcf,0x23,0x67,0x06,0x25, -0x65,0xda,0xd7,0xc9,0xe6,0xc9,0x54,0x71,0xfd,0x1f,0xcc,0x8a, -0xf8,0x9c,0x92,0x5d,0x63,0xb2,0xdc,0xd4,0xc7,0x2a,0x93,0xc8, -0x35,0xf4,0xd3,0x77,0xcf,0xdd,0xe2,0xe1,0x2a,0x59,0x90,0x7b, -0x48,0x51,0x37,0x6b,0x01,0x67,0x94,0x09,0x88,0x32,0x0b,0x8a, -0x5b,0x11,0xb8,0xea,0xd9,0x9f,0x8f,0xa4,0x6a,0x63,0xf1,0xa2, -0x7e,0x22,0x0f,0x43,0x9f,0x5e,0x1f,0xf8,0x34,0x85,0xbd,0x52, -0x87,0xd0,0xc6,0xa4,0xcd,0xd6,0x59,0x7d,0x97,0xcf,0x8a,0x06, -0x70,0xdb,0xf9,0x82,0xd7,0xae,0xaf,0xa5,0xc2,0x6e,0x29,0xc9, -0x95,0x21,0x2c,0xb9,0x5f,0x29,0x8e,0x56,0x7c,0x99,0x40,0x04, -0xd6,0xc9,0x48,0x46,0xc6,0x52,0xe8,0x9f,0xa3,0x10,0x23,0x95, -0xe9,0xc7,0xd3,0xc5,0x0d,0xc1,0x12,0x95,0x17,0x57,0xfc,0xc6, -0xfd,0x1e,0x5b,0x63,0x31,0xf6,0x9a,0xa8,0xe9,0xa3,0xa4,0x5b, -0x7f,0xc0,0x61,0x73,0xab,0x32,0x0a,0xc8,0x2f,0xe1,0x15,0xef, -0xa8,0x8f,0x7e,0x84,0xe3,0xc6,0xf0,0x4c,0x7e,0x55,0xf5,0xe6, -0x38,0x30,0x35,0x8e,0x5c,0x67,0x1e,0xc8,0x3c,0xc8,0x8a,0x01, -0x31,0xe3,0x43,0xf8,0x53,0x7a,0xc7,0xe0,0xc1,0xa0,0x44,0xa1, -0x0b,0x38,0x1b,0xa8,0x96,0xc9,0xbd,0x7f,0xfa,0xfc,0x41,0x34, -0x58,0xdc,0x7c,0xbb,0xf5,0xeb,0xd2,0x03,0xdd,0x03,0xfd,0xbc, -0xb8,0xf3,0xf3,0x5f,0xee,0xa1,0x9b,0x60,0x85,0x12,0xe0,0xde, -0xd9,0x32,0x42,0xe2,0xad,0x35,0x1d,0x3c,0x50,0xaa,0x3e,0x34, -0x1b,0x8b,0x32,0xe8,0x14,0xe1,0x9c,0xd7,0x52,0x41,0xfb,0xb4, -0xe2,0x8e,0x20,0xb3,0x70,0x84,0xdf,0x0a,0x10,0x91,0x4c,0x03, -0xe8,0x6f,0x52,0x23,0x6d,0xf7,0x54,0x49,0x30,0x6a,0x30,0x48, -0xd5,0x94,0x47,0xa6,0xe0,0x4d,0xcf,0x25,0xbd,0xb6,0x7e,0x76, -0xed,0x10,0x3a,0x01,0x14,0x32,0x60,0xc2,0x88,0x16,0xe4,0x93, -0xf3,0xb6,0x9a,0xd4,0x93,0xf8,0x1c,0x4f,0x8f,0x30,0x99,0x6a, -0x52,0xd0,0x10,0x0d,0xba,0x7a,0xba,0x6d,0xec,0xc0,0x6b,0x93, -0x83,0xfe,0x2a,0x9c,0x33,0x37,0x39,0xec,0x45,0xc9,0x79,0x0b, -0x93,0xc7,0x08,0x00,0x52,0x6e,0x91,0x14,0x8c,0x18,0x01,0x44, -0x01,0x36,0x47,0xc1,0x94,0xde,0x58,0x0e,0x98,0x87,0xc3,0x80, -0x98,0x3d,0x4f,0x27,0xaf,0xdc,0x0c,0x0e,0x89,0x11,0x84,0x1d, -0xb9,0xb5,0x65,0x21,0x05,0xe2,0x12,0x59,0xfa,0xbf,0x9d,0x22, -0x18,0x15,0xb3,0xb5,0xaf,0x2f,0xe6,0xda,0x71,0x0d,0x1f,0xc4, -0xa4,0xe9,0x41,0x69,0x8f,0x9f,0x69,0xcf,0x9f,0x1d,0x4e,0xd2, -0xc4,0x37,0x3e,0x71,0x96,0x26,0xee,0xfb,0xc4,0xeb,0x34,0xf1, -0xad,0x4f,0x9c,0xa7,0x89,0x07,0x3e,0x71,0x91,0x26,0x1e,0xfa, -0xc4,0xff,0xa6,0x89,0x3f,0x61,0xa0,0x63,0xcf,0x3d,0x59,0x4c, -0x75,0xe2,0x19,0xa8,0x79,0x00,0x86,0x62,0xe8,0xd2,0x99,0xe0, -0xce,0x23,0x24,0x1b,0xa2,0xce,0xfb,0x6a,0x08,0xa0,0x8d,0x2d, -0xd6,0xf1,0x3b,0xd6,0x1e,0x66,0x22,0x44,0x25,0x81,0x7e,0x60, -0xe3,0x97,0x30,0x41,0x09,0xba,0x86,0x2f,0x9b,0xc3,0xb5,0x73, -0xce,0xf6,0x48,0x5b,0xeb,0x84,0x6a,0xb1,0x51,0x23,0x83,0x09, -0xd4,0x21,0xbe,0xc6,0x23,0x05,0xa0,0xc8,0x20,0x85,0x6d,0x40, -0x22,0xcc,0x63,0x2e,0x4a,0x05,0xec,0xb3,0xdf,0x70,0xa9,0x61, -0x02,0x61,0x08,0x4b,0x48,0x53,0x5c,0xde,0x16,0xae,0x4e,0xd1, -0x13,0xf3,0xe8,0x85,0xe5,0x7d,0xd3,0x35,0x46,0xe2,0x5e,0x4b, -0x60,0xee,0x32,0x36,0x72,0x54,0xc3,0xce,0x05,0x40,0x36,0xba, -0x3a,0xa1,0xa7,0x64,0xd3,0x27,0x1f,0x20,0x7c,0x81,0x3d,0x09, -0xc5,0xae,0xc0,0xe8,0xfc,0x54,0x54,0x7d,0xa5,0x5a,0xfc,0xdb, -0x84,0x27,0x4b,0x52,0x11,0xc0,0x38,0xc3,0xd2,0x2a,0xb5,0xd4, -0xeb,0x0d,0x32,0xa9,0xe3,0xf1,0x12,0xa9,0x2a,0xfb,0xa1,0xb3, -0x4c,0xaf,0x29,0x2d,0xc4,0xfe,0x94,0xd3,0xa0,0xca,0x56,0x20, -0xcf,0x27,0x08,0x57,0x76,0x54,0x55,0x37,0x2e,0x40,0x53,0x23, -0x01,0x02,0xdb,0xc1,0xc1,0x9e,0x1b,0xb4,0x90,0xd9,0xb1,0x51, -0xc6,0x1b,0x4f,0x71,0xaf,0x35,0x2c,0x79,0x64,0x9a,0x44,0x02, -0x85,0x30,0x0d,0xb6,0x93,0x16,0xd5,0x19,0xb7,0x30,0xf0,0x02, -0xb3,0xc6,0x77,0x48,0xd9,0xb2,0xf2,0x13,0x81,0x42,0x4b,0x2a, -0x5c,0x67,0xb1,0x48,0xc1,0xbb,0x6c,0x1e,0xda,0xc5,0x69,0xfc, -0xc5,0xc4,0x25,0xcf,0x6e,0x86,0x94,0x18,0xc9,0x61,0x2b,0x95, -0x26,0x65,0xf4,0x34,0x6c,0x0c,0xac,0x83,0xcf,0x40,0xc8,0xf3, -0x8c,0x1c,0xc4,0xb5,0xa2,0xe1,0x1f,0x47,0x9d,0xf9,0x74,0xd5, -0xa5,0x1b,0xb1,0xb1,0x1e,0x86,0xe4,0x60,0x55,0x08,0x45,0xc4, -0x02,0x2f,0x79,0x34,0x44,0xf8,0x33,0x53,0xbb,0x02,0x70,0x26, -0x0b,0x3c,0xb0,0xe7,0x5f,0x95,0x02,0xae,0xdf,0x27,0x97,0x21, -0xb3,0xfe,0xcc,0x8a,0x77,0x4f,0x38,0x17,0x77,0x58,0x3c,0x86, -0x63,0xdf,0x06,0xea,0x09,0x58,0x51,0x20,0x66,0x23,0x1b,0x67, -0xff,0x3a,0xf9,0x7c,0xfe,0xe5,0x4b,0x81,0xd5,0xaf,0x15,0xfa, -0x88,0xef,0x36,0x1a,0x94,0x1a,0xab,0x47,0x28,0xfe,0xf3,0xe9, -0xa7,0x5e,0x06,0x2d,0x60,0xa1,0xea,0x3d,0x92,0x6b,0x4a,0x65, -0x36,0x73,0xf6,0xf9,0x3c,0xda,0xc0,0x9c,0x9e,0x9e,0x7d,0x39, -0x27,0xee,0xd0,0x12,0x35,0xbd,0x60,0x99,0xe3,0x61,0x82,0x07, -0xd4,0xd9,0x5c,0xee,0x5b,0x83,0x82,0xb6,0x94,0x08,0x0f,0xe5, -0x3c,0x5c,0xbf,0xcd,0xd5,0xdd,0xdb,0x3c,0x5e,0x78,0x61,0xba, -0xbe,0x51,0x3a,0x5f,0x89,0xdb,0x5f,0xe6,0x81,0x82,0x43,0x20, -0x0d,0x01,0x3f,0x69,0x4f,0x3f,0x9f,0x42,0xf8,0xa6,0xcb,0xde, -0x0b,0xfb,0x8b,0x69,0xcb,0x97,0x91,0x0c,0xce,0x98,0xe2,0x80, -0x32,0x55,0x39,0xd4,0x16,0xab,0x6a,0xee,0x70,0xbe,0x2c,0x78, -0xe1,0x37,0x28,0x2d,0xc8,0x4e,0xd7,0xb9,0xe3,0x31,0x48,0xbb, -0xb8,0x99,0x8b,0x05,0x73,0x86,0x9f,0x68,0x31,0x4c,0x75,0xfc, -0x7e,0x85,0x60,0xe9,0x38,0xc9,0xde,0x3f,0xf4,0x3e,0x80,0xc2, -0x1d,0xf6,0x83,0xf7,0xe1,0xd7,0x75,0x17,0x4d,0xa8,0x33,0x41, -0x5b,0xbc,0x67,0xa8,0x4a,0xf5,0xb4,0xcd,0xe6,0x84,0x23,0x39, -0xa5,0x87,0x25,0x9c,0xc0,0xa7,0x0a,0x5f,0x8d,0x4f,0xbf,0x66, -0x60,0xa1,0x51,0x81,0xa2,0x84,0xb0,0x1a,0xdd,0x7d,0x35,0xac, -0x3f,0xfa,0xa6,0x36,0x1e,0xd0,0x86,0xe1,0xbc,0x38,0xd5,0x46, -0x0d,0x24,0x97,0xa2,0x3a,0x0a,0x4d,0xfa,0xff,0x08,0xbb,0xa9, -0xff,0xda,0xf4,0xb9,0xb9,0xd9,0x2d,0x6e,0x08,0x08,0x38,0x6c, -0xa4,0x5d,0xa7,0x79,0xb1,0x81,0xf2,0xb9,0x38,0x2e,0xf8,0x9c, -0xbe,0x60,0x39,0x71,0xdc,0x5d,0x0b,0x26,0xfe,0x9b,0xf3,0x5b, -0x05,0x86,0xe1,0xc0,0x9f,0x32,0x8b,0x7b,0x4a,0x39,0x5d,0xa0, -0x80,0xc9,0x6c,0x47,0x15,0x82,0x2f,0x18,0x74,0x07,0x7a,0x6a, -0xc8,0xda,0x8e,0xc3,0x29,0x5d,0x63,0x6b,0x4b,0x83,0xe3,0xd1, -0x4f,0x75,0xa0,0xc5,0x70,0x21,0x71,0x61,0xa6,0x2b,0xda,0x0a, -0x1a,0xe1,0xef,0x61,0x9c,0x50,0xec,0x8c,0xec,0x06,0x58,0x92, -0x71,0x17,0x7a,0x4b,0xfa,0xdf,0xcb,0x25,0x9b,0x7d,0x49,0x42, -0x8b,0x4a,0x90,0x66,0xff,0xcd,0xde,0xe5,0x5b,0x6e,0x48,0x23, -0xc0,0xbb,0x11,0xb6,0x67,0x92,0x44,0x6d,0xd9,0x4c,0xf6,0xb6, -0xb5,0x11,0x09,0xb1,0x46,0xdf,0x01,0xb7,0x4d,0x59,0x3c,0x51, -0xd6,0xdc,0x96,0x1f,0x35,0x48,0x53,0x99,0xd5,0xd6,0x8b,0x0d, -0xf7,0x0f,0x2a,0x7e,0x44,0x35,0x84,0xa2,0x91,0xc6,0xf7,0x7b, -0x87,0xc6,0xb9,0x5d,0xf5,0x1a,0xaa,0x6e,0x9f,0xad,0x6d,0x41, -0x88,0x86,0xaa,0x67,0xd7,0xb7,0x27,0xdd,0xcf,0xcd,0xda,0x1e, -0x0e,0x5a,0xe6,0x5d,0x00,0x10,0xf0,0xa7,0x1e,0xec,0x5f,0x7c, -0x50,0xa8,0x4d,0xfa,0x4e,0x4c,0xf8,0xc2,0x7b,0x00,0x32,0xd8, -0x47,0x83,0x45,0x9f,0xcd,0xf1,0xcf,0xc4,0x06,0x7e,0x36,0x1d, -0xec,0x0a,0xc9,0xe0,0xcf,0xbe,0xd1,0x4a,0x46,0x90,0xdb,0x45, -0x83,0x43,0xb6,0x01,0xec,0x25,0x3a,0x18,0xef,0xff,0xe4,0xa9, -0x20,0xd5,0xc9,0x08,0x67,0xf5,0x5b,0x1d,0x82,0x50,0x85,0x85, -0xe7,0xca,0x4f,0x9d,0x38,0x39,0x00,0x7f,0xcd,0x2a,0xe7,0x12, -0x39,0x24,0xce,0x49,0x1a,0x60,0xbe,0x11,0x2c,0xe4,0x58,0x16, -0xd0,0xeb,0x1c,0xc9,0xaf,0x3b,0xd7,0x7f,0xcc,0xa7,0xb7,0x7e, -0x2c,0x0f,0xad,0xab,0x22,0xb7,0xfd,0xf1,0x4e,0x19,0x64,0xe5, -0x5b,0xc6,0x3a,0x81,0x56,0x76,0x73,0x57,0xa6,0x35,0x75,0x95, -0xc2,0x71,0x4e,0x37,0x91,0xd3,0x00,0xfb,0x15,0x27,0x30,0x0f, -0x80,0x38,0x32,0xe0,0x05,0xf8,0xb9,0x46,0xca,0x56,0xc0,0x6d, -0x62,0x4d,0x62,0x75,0x70,0x0c,0x75,0xb4,0x85,0xa8,0xa3,0x8b, -0xba,0x9a,0x70,0x92,0x28,0xb0,0x8b,0x7a,0xac,0x52,0x3e,0xf8, -0x9a,0xea,0x89,0x2a,0x85,0x09,0x7b,0x9c,0xc0,0x3a,0xf4,0xa2, -0x3e,0x54,0x09,0x54,0x49,0x6c,0xc2,0x6c,0x1f,0xbc,0x67,0xe8, -0x2e,0x96,0xe7,0x16,0xf8,0x62,0x9c,0x17,0x28,0x23,0x48,0x08, -0x8b,0xac,0x3c,0x11,0xe2,0x5b,0xa8,0x39,0x2f,0x92,0xc5,0xda, -0x2d,0xd4,0xbd,0x33,0xe2,0x1f,0x2d,0x03,0x63,0x99,0x46,0xcb, -0xe5,0x33,0x94,0x6f,0x85,0x72,0x55,0xce,0xb4,0x80,0xa5,0xbc, -0x68,0xc0,0xef,0x4c,0x2f,0xbc,0x2c,0xf4,0x69,0x37,0xb5,0x5a, -0xa5,0x19,0xbf,0x35,0xe3,0x61,0xa3,0x1c,0x75,0xad,0x17,0xb1, -0xea,0x3f,0x16,0x08,0xc0,0xdb,0xef,0x69,0x46,0xea,0xa0,0xca, -0x13,0xae,0xfc,0x64,0x87,0x0f,0xba,0x70,0xef,0x63,0xf3,0xd3, -0x15,0xae,0xa2,0xe5,0xf7,0xf8,0xfd,0xf4,0xfa,0x6a,0x11,0xb7, -0xc6,0x9e,0xab,0x8b,0x1e,0xda,0xe3,0xa3,0x8c,0x7d,0x54,0x0f, -0x23,0x4b,0x3c,0xe2,0x49,0x18,0x8d,0xdf,0x27,0x22,0xff,0x68, -0xb7,0xa3,0x02,0x4b,0x14,0xc1,0xf5,0x28,0x92,0x9f,0x48,0x3c, -0x2c,0xf5,0xac,0xdd,0x5d,0xc5,0xbd,0xc6,0x5b,0x29,0x34,0x0a, -0xe6,0x61,0x34,0xf1,0xea,0xf4,0x2c,0xc8,0x48,0xae,0x6b,0x36, -0x44,0xc6,0x77,0xc5,0x35,0xbb,0x56,0x7b,0xd4,0x98,0x01,0x1b, -0x69,0x8d,0xaf,0x75,0x07,0x30,0x72,0x24,0xce,0x32,0xfc,0x2a, -0x76,0xc9,0x8c,0x0a,0x09,0xf3,0x51,0x19,0xba,0x49,0x4d,0xb8, -0x5a,0xd7,0xc6,0xb0,0x2d,0x34,0x42,0x4b,0x7c,0x6d,0x2c,0xda, -0x24,0x93,0x77,0x86,0xba,0x6b,0x43,0xe6,0x25,0x56,0xd6,0xf7, -0xda,0xc5,0xdc,0x98,0x19,0x7a,0x99,0x22,0x30,0x18,0x37,0xa5, -0xd4,0xca,0x30,0xc8,0x39,0x7e,0x25,0xc8,0xf4,0xe3,0x0b,0x04, -0x0b,0xe2,0x9d,0x17,0x40,0x4a,0x23,0x0f,0xd8,0xb4,0xf1,0x96, -0xbb,0x8e,0xa4,0xa7,0x89,0xdb,0x11,0x59,0xd0,0x6f,0xa8,0x7e, -0xb5,0x9c,0xe0,0x1a,0xf2,0xde,0x1f,0x4c,0xee,0xb3,0xf2,0x56, -0xc2,0x4c,0xce,0x31,0x8a,0x9a,0xc1,0xff,0x89,0xad,0xfc,0xc8, -0xac,0x27,0x96,0xdb,0xe3,0x91,0x95,0xbd,0x6c,0x2e,0x30,0xf5, -0x8f,0x4b,0xbf,0x42,0x5e,0xfe,0xb5,0x5c,0xbd,0x7b,0x96,0xab, -0xd3,0xe9,0xfc,0x2f,0xe6,0xc9,0xbf,0xc9,0x67,0xb8,0x9a,0x6b, -0x6b,0x8f,0xbb,0x87,0x6b,0x75,0x69,0x85,0xbf,0x44,0xc3,0x92, -0x85,0x7a,0xc0,0xa1,0x74,0x8c,0x74,0xb4,0x2a,0xe1,0x57,0x5f, -0xe9,0xf4,0xcf,0x1f,0xa8,0x14,0x49,0x8e,0xa1,0x3e,0x53,0xe9, -0xff,0x8c,0xc2,0x44,0x2d,0xce,0x24,0x93,0x37,0x89,0x80,0xae, -0xd7,0x63,0xf1,0xad,0xd5,0xeb,0x36,0x11,0xea,0xcd,0x45,0x9c, -0x53,0x78,0x4d,0x5b,0xe7,0x6b,0xcb,0xeb,0xf7,0x33,0xf0,0xb5, -0xae,0x0c,0x28,0x0a,0x94,0xf0,0x6e,0x72,0x10,0x0a,0x1f,0x8c, -0xb7,0xf5,0x1a,0xcc,0x8c,0x37,0x6a,0xd3,0x92,0x83,0xeb,0x98, -0x83,0xbe,0x6f,0x36,0x82,0xd5,0xfc,0xa2,0x53,0x51,0x2a,0xc1, -0x51,0x25,0x15,0x35,0x50,0xcc,0xbd,0xd4,0x33,0x41,0x81,0x87, -0xf5,0x0c,0x62,0xe8,0xd9,0x7f,0x37,0x67,0xb6,0x3b,0x2a,0xf0, -0xd1,0x8c,0xe8,0x7c,0x81,0x7f,0xde,0xd1,0xc5,0xe6,0xc3,0xe5, -0x6c,0xc3,0xaf,0x43,0xa0,0x9f,0x9e,0xfe,0xb6,0x01,0x9d,0x4d, -0xf5,0x04,0xcd,0x59,0xb7,0xa2,0x4e,0xff,0x73,0xc5,0xbc,0x2a, -0xaa,0x73,0xd9,0x8d,0xac,0x8a,0xd7,0xd3,0xc5,0xfb,0xdf,0x6e, -0xbe,0xf9,0xb9,0x03,0xee,0x3b,0xf4,0x2a,0x7b,0x0c,0x34,0xd8, -0xbb,0x5b,0x9f,0x4f,0xcd,0x89,0xcf,0xd5,0xe3,0x12,0x50,0xaa, -0xcc,0x89,0x84,0x08,0x06,0x07,0x63,0x94,0x69,0x64,0x63,0xcc, -0xf4,0x81,0xde,0x36,0x95,0xdc,0x55,0xea,0xd3,0x15,0x09,0x46, -0xec,0xa9,0xde,0x2b,0x77,0xa0,0xd7,0x3c,0x61,0x4a,0x0c,0xf3, -0x07,0x22,0x14,0x62,0x7b,0x0e,0x4b,0xb8,0x5f,0x3a,0x3e,0x4c, -0xb8,0xd1,0x24,0x37,0x2f,0x22,0x13,0x41,0x40,0x02,0x38,0x84, -0x20,0x21,0x31,0x12,0x04,0xdd,0x20,0xf7,0x32,0x0b,0x2d,0x35, -0xea,0x26,0x19,0x47,0xa1,0x39,0x21,0xd8,0x1b,0xe5,0x24,0xf5, -0xd1,0x61,0x17,0xe5,0xf8,0x7b,0xb4,0xac,0x3d,0x0a,0x6b,0x8f, -0xc4,0x95,0x32,0xe9,0x7f,0x39,0x63,0x93,0x72,0xff,0xd5,0x0d, -0xda,0x8b,0x74,0xd5,0x12,0x02,0xed,0xea,0x87,0x43,0x2f,0x13, -0x7c,0x23,0xb9,0x66,0xb0,0x1e,0xd3,0xe5,0x87,0xdb,0x54,0x5d, -0x05,0x59,0x2e,0x3e,0x5b,0x7f,0x6b,0xd9,0xe7,0x82,0xc4,0x57, -0x36,0x18,0x70,0x1f,0x7a,0x1a,0x04,0x89,0x3d,0xa8,0x1b,0x86, -0x6e,0x74,0x58,0x65,0x5f,0xb1,0x1b,0x1c,0xf6,0x31,0x00,0xb5, -0x67,0xc6,0x19,0x67,0xc5,0x67,0xce,0x71,0xd1,0xb7,0xa8,0x23, -0x2f,0xcb,0xe5,0x61,0x1d,0x7e,0xc5,0xe4,0xc1,0x61,0xcb,0x97, -0xc1,0xd8,0xce,0x21,0xf8,0xb3,0xe8,0x45,0xe1,0xfb,0x24,0x27, -0x8e,0x6e,0x8a,0xb2,0x6c,0xa7,0x16,0x52,0xd8,0xbf,0xa8,0xae, -0x8c,0xe5,0x89,0xaa,0x9e,0x56,0x38,0x69,0x8b,0x6a,0xe5,0x15, -0x0e,0x7a,0xba,0xee,0xbb,0x71,0x32,0xe0,0x5a,0xb4,0x34,0x22, -0xfd,0x30,0x7e,0xef,0xbb,0xf4,0xa2,0x7d,0x60,0xe5,0x18,0x12, -0x34,0xef,0x88,0xa9,0xb1,0xf4,0x72,0xb8,0x0a,0xf9,0x52,0x68, -0xc7,0x98,0x0f,0x25,0x8b,0x71,0x64,0xc1,0x75,0xaf,0xaf,0xa0, -0x3b,0xca,0xb5,0xc3,0x9e,0x8f,0x85,0x00,0x4a,0x15,0xe1,0x5c, -0x72,0x75,0x8b,0x27,0x96,0xc6,0xfc,0x54,0x99,0x1c,0x35,0x3a, -0x49,0x50,0x04,0x30,0x0e,0x75,0x3e,0x02,0xb3,0xc4,0x9f,0x3c, -0x38,0x35,0x05,0x27,0x29,0xa2,0x87,0xbb,0x7f,0x9f,0x7e,0xfa, -0xd8,0xe8,0x84,0x70,0x87,0x91,0xdc,0x0e,0xa9,0x4b,0xca,0x46, -0xb6,0x72,0x7d,0x51,0xc4,0x21,0xd1,0x53,0x55,0x87,0xa0,0x39, -0xd9,0x50,0xda,0x14,0x58,0xdd,0x98,0x29,0xd0,0x8b,0xef,0x06, -0xd3,0x66,0xc1,0x47,0x05,0xd2,0x66,0x4a,0xfc,0x51,0x62,0x58, -0x76,0xa1,0x79,0x98,0x1e,0xc1,0x3f,0x6b,0xff,0xa0,0x7f,0x0c, -0xc0,0x77,0x56,0x32,0x82,0xd6,0x49,0x53,0x10,0x72,0x14,0xd6, -0x22,0x74,0x52,0xf6,0x20,0x86,0xc2,0x0f,0x45,0x35,0xa8,0x14, -0x37,0xcd,0x71,0xb8,0xe9,0x13,0x0a,0x11,0xd8,0x94,0x6a,0x4d, -0xe3,0x4d,0xf9,0x22,0x94,0x8f,0xff,0x07,0x7b,0xfb,0xf8,0xd4, -0xa3,0x48,0x1e,0x71,0xa8,0x90,0x18,0xfb,0x04,0xfe,0x0d,0xde, -0x8c,0xb9,0x43,0x14,0xf1,0x8c,0xb1,0x9c,0x98,0x17,0x19,0x42, -0xf0,0x31,0x98,0xec,0x31,0x92,0x55,0xe4,0x04,0x33,0x76,0x01, -0x86,0x14,0xe9,0x29,0x1b,0xff,0xd3,0xfa,0x9a,0xbb,0x38,0x09, -0xa5,0xc3,0xa1,0x1b,0x41,0x85,0xd6,0xdc,0x01,0x6e,0x74,0x5f, -0xed,0x08,0x77,0xf9,0xed,0xf8,0x6e,0x47,0xe4,0xa7,0xbc,0x6d, -0xcc,0xfb,0x3d,0xcb,0x6e,0xc4,0x3e,0xcb,0xdb,0xca,0x6b,0xea, -0x22,0x32,0xee,0x05,0x29,0xd6,0xe3,0x59,0x75,0x9d,0xe8,0xea, -0x44,0xdb,0x76,0x79,0x7f,0x79,0xfb,0xeb,0x65,0x83,0x67,0x10, -0xd1,0x90,0x1a,0xe8,0xb1,0x48,0x54,0x6e,0xc6,0x0a,0x8c,0x56, -0x26,0xe0,0x10,0x85,0xec,0xc4,0xe0,0x57,0x2e,0x52,0x82,0x21, -0xa0,0xae,0x95,0x55,0x9b,0xae,0xfc,0xc1,0x3b,0x97,0xd4,0x68, -0x40,0x91,0x1c,0xf5,0xf7,0x37,0x44,0x9f,0x4f,0x66,0x97,0xea, -0x7c,0xca,0xc5,0x39,0xa6,0x3b,0x58,0x4a,0xd1,0x4c,0xa3,0x1c, -0x9a,0xe5,0xc3,0xef,0xb9,0x75,0x79,0x15,0xc7,0xd5,0x5a,0x8e, -0xab,0x1e,0x8e,0xab,0xe7,0x39,0xfe,0xe1,0x2d,0xd5,0xee,0x10, -0x08,0xbc,0x83,0x2e,0x95,0x8b,0x57,0x57,0x8b,0x00,0x34,0x54, -0x77,0xb0,0xb6,0xb6,0x96,0xe2,0x9c,0x2b,0xa6,0xdf,0xa1,0x51, -0xdc,0x33,0xe3,0xea,0x8d,0xa6,0x74,0x7e,0xa2,0x2c,0x32,0xf7, -0xe4,0x22,0x60,0x2e,0x62,0x80,0xf8,0x20,0xee,0x2f,0x86,0x4e, -0x7c,0x80,0x92,0x5c,0xf6,0x97,0x71,0x0c,0x8d,0x67,0x33,0x69, -0xf2,0x0b,0x9a,0x9e,0xcd,0xe3,0xab,0xe6,0x08,0xc2,0x97,0xd4, -0x2b,0xf7,0xcf,0x2e,0x2c,0xc7,0x96,0x20,0xdc,0x41,0x8f,0xba, -0x5a,0x2a,0x82,0xf7,0x33,0x63,0x67,0xba,0x38,0x95,0x4c,0x46, -0x31,0x52,0x29,0xef,0xc6,0x6d,0x51,0xc1,0x21,0xce,0x77,0xdc, -0x2f,0xc5,0x08,0x01,0xb6,0x16,0x28,0x3b,0x2f,0x46,0xc5,0x7f, -0x28,0x83,0x17,0xc3,0x96,0x47,0x3e,0x84,0x67,0xc2,0x64,0x78, -0xc6,0x16,0x27,0x09,0x44,0x37,0xc2,0x24,0x7c,0xb4,0xe5,0x92, -0x1f,0x61,0xb9,0x64,0xb6,0x5a,0x88,0x33,0x16,0x09,0x5a,0xfc, -0x3f,0x82,0x70,0x3d,0x54,0x3f,0x91,0xb7,0xfc,0x19,0x5b,0x90, -0xe2,0x32,0x55,0x21,0x46,0x0e,0xc4,0x9d,0x2c,0x7e,0x19,0x7f, -0x28,0xbc,0x9c,0x13,0x5d,0xdb,0xf2,0x38,0x11,0x2e,0x9f,0x9c, -0x5a,0x67,0xdd,0x80,0x5d,0x56,0x16,0x1f,0xe9,0xa6,0x07,0x80, -0x02,0x84,0x9a,0x32,0x03,0xd8,0x74,0x63,0x67,0x80,0x23,0xc9, -0xa7,0x53,0x6d,0x81,0xe2,0x47,0x51,0xd2,0xc5,0x8f,0x4b,0xcf, -0xf9,0x30,0x5e,0x01,0x8c,0x08,0x07,0x5d,0x67,0xcf,0x20,0xe3, -0x8a,0x2a,0x2c,0xfe,0x7a,0xa4,0xee,0xd2,0x5e,0x31,0x9a,0xec, -0x99,0x74,0x63,0xee,0x84,0xd6,0x1f,0xe5,0x53,0xaf,0xbd,0x33, -0xcc,0x86,0xaf,0x0d,0xfd,0x1a,0x4c,0xc6,0xe1,0xc2,0xd1,0xcb, -0xa5,0xb4,0xd1,0x84,0xac,0x31,0x78,0xe5,0xb6,0x05,0xa8,0x6c, -0x21,0xf7,0x2b,0x28,0xcb,0x8b,0x7a,0x8f,0xbe,0x82,0x16,0xf5, -0x0d,0x7d,0xbd,0xf7,0x5f,0x81,0x2a,0x9e,0x26,0xa0,0x82,0x91, -0x04,0xae,0x74,0x11,0x7a,0xa9,0x1d,0x5a,0x7c,0x25,0x5e,0x1f, -0x4b,0x0d,0x2a,0xe5,0x5f,0x7f,0x57,0xee,0x26,0x79,0x79,0x6b, -0x8b,0x3e,0x77,0xa6,0xf3,0x59,0x4b,0x5f,0xe1,0xe2,0x10,0xaa, -0x28,0xa2,0xd7,0x08,0x07,0xb4,0x2d,0x05,0xc3,0x89,0x9a,0x54, -0xd7,0x66,0xc8,0x01,0x39,0x48,0xfa,0x8d,0xe8,0xf6,0x8e,0x5a, -0xde,0x6c,0x9a,0xcd,0xf1,0xd6,0x96,0x66,0x85,0xbb,0x75,0x45, -0xb8,0x66,0x92,0xba,0xa2,0x98,0x9d,0x74,0x64,0xfc,0x7f,0x00, -0x00,0x00,0xff,0xff,0xc9,0xc8,0xe4,0xfd,0x32,0x22,0x06,0x00, +0xf7,0xf6,0x7d,0x6d,0x1c,0xc9,0xc2,0xe8,0xdf,0xf7,0xf9,0x14, +0x30,0xc7,0xc1,0x1a,0xab,0x11,0x12,0x76,0x9c,0xcd,0xc8,0x83, +0x8e,0x63,0xec,0xc4,0x7b,0x8d,0x9d,0xc7,0x90,0xf5,0xd9,0x08, +0x85,0x9f,0x40,0x02,0x26,0x06,0x89,0x68,0x44,0x1c,0x07,0xe9, +0x7e,0xf6,0x5b,0x6f,0xdd,0x5d,0xdd,0x33,0x02,0x9c,0xdd,0x73, +0x9e,0x7b,0x37,0x6b,0x34,0xd3,0x5d,0x5d,0x5d,0xd3,0x2f,0xd5, +0xd5,0xd5,0xd5,0x55,0x27,0xc8,0xa2,0xf0,0xdd,0x5e,0x94,0x32, +0x6a,0x68,0x55,0x56,0x33,0xc4,0x66,0x42,0x5c,0x26,0xc0,0xe4, +0xee,0x61,0xb9,0x2f,0x31,0x7a,0x09,0xd4,0x67,0x30,0x2f,0x60, +0x98,0xe9,0xf1,0x57,0x3b,0x5a,0xef,0x37,0x0e,0x4b,0x77,0x80, +0x5d,0xce,0x4e,0x60,0x3b,0x60,0xa3,0xc4,0x48,0x83,0xc8,0x9e, +0x00,0xf2,0xcc,0x74,0x82,0x4e,0x7d,0x1d,0x08,0xbf,0xd2,0x95, +0x10,0xf4,0xe5,0xc9,0xdf,0xe0,0x72,0x31,0xa9,0xa4,0xa4,0x6e, +0x3d,0xbb,0x78,0x10,0x3a,0x5b,0x11,0x87,0xa1,0x94,0x25,0x2f, +0x7e,0xb4,0xe3,0x56,0x97,0x74,0x8e,0xe3,0x19,0x3a,0x94,0xc3, +0x81,0x74,0x7d,0xf5,0x66,0x58,0xce,0x85,0x48,0x71,0x02,0xa8, +0x45,0x6d,0x8d,0x10,0x47,0x9b,0x7a,0x0d,0x86,0xab,0x4a,0xa7, +0xe0,0xb1,0x69,0x54,0x7b,0xcd,0x54,0x09,0xf2,0x63,0x78,0x46, +0xf2,0xbf,0x96,0x61,0x0c,0x0d,0x12,0x13,0x88,0x15,0x3e,0x2f, +0xdf,0x7b,0x0b,0xcc,0x06,0x37,0xb9,0xb6,0x7a,0x9a,0x9d,0x88, +0xf7,0xa1,0xf3,0xa2,0xe4,0x3d,0xfe,0xeb,0x51,0xde,0x6c,0x06, +0x6d,0xd0,0x05,0xa8,0x1e,0x0f,0x28,0x94,0x3d,0xb0,0x90,0xb9, +0x21,0xc7,0xe2,0x59,0x38,0xbe,0x96,0x69,0xab,0xbc,0x3e,0x39, +0x19,0x63,0x54,0x13,0x5b,0x03,0x8c,0xa2,0xab,0xe9,0xa4,0x64, +0x6b,0xa5,0xa0,0x96,0x3c,0x0f,0x6a,0x11,0x03,0x9c,0xf1,0x27, +0x6e,0x1c,0x3d,0x47,0xbb,0x7a,0xf0,0x34,0x2c,0x48,0x75,0x8e, +0xae,0xea,0xa8,0x4a,0xc3,0x5b,0x0c,0x51,0x5b,0xf2,0x14,0x0e, +0x13,0x59,0xee,0x72,0x5f,0x51,0x99,0xf5,0xe1,0xd8,0xc1,0xde, +0xf0,0x43,0xcd,0xcd,0xb0,0x08,0xac,0x65,0x5d,0xee,0x34,0xd2, +0x34,0x18,0x2a,0x81,0x5f,0xdc,0x60,0x94,0xe3,0x66,0x59,0xbf, +0x6f,0x6c,0xac,0x07,0xbb,0xa9,0x08,0x36,0x98,0xe8,0xd1,0xe7, +0x43,0x91,0x4b,0x5c,0xd6,0x1f,0x14,0x41,0x23,0x89,0x97,0xed, +0x34,0xd8,0xa5,0xb9,0x39,0xc8,0x9a,0x9f,0x16,0xe9,0xdf,0x1b, +0x91,0xaa,0x7b,0x55,0x8f,0x62,0x84,0x98,0x95,0x3d,0xb2,0xf4, +0x15,0xd5,0x51,0x23,0xf7,0x10,0x91,0xa0,0x34,0xbb,0x15,0x8d, +0xe7,0xa8,0xc5,0xfc,0xbe,0x12,0xe7,0x6d,0x96,0x38,0x9a,0x6b, +0x95,0x55,0x5d,0x58,0xd9,0xe2,0xaa,0xd8,0x38,0x0a,0xea,0x7d, +0x3b,0x9d,0xa0,0xec,0x8e,0x27,0x54,0x2b,0xaa,0xd7,0x9c,0x53, +0x2f,0xe9,0x54,0xfc,0x47,0x90,0xb4,0x48,0x5c,0x0a,0x96,0x02, +0xb1,0xe2,0x34,0x2b,0x37,0x74,0x0c,0x60,0xc2,0x8d,0x1d,0x79, +0xd9,0x7e,0xff,0xfc,0xc5,0xcb,0x7c,0xeb,0x66,0xb9,0x65,0x3d, +0x93,0x04,0x56,0x19,0xf7,0x09,0xd9,0xc4,0x06,0x5f,0x78,0x50, +0xe9,0xd8,0xeb,0x09,0xf6,0xa8,0xf9,0x74,0x3e,0x9e,0xb8,0x24, +0x5e,0xfb,0x31,0x29,0x8a,0xa7,0x16,0xe5,0xa6,0xe8,0xfb,0x0b, +0xc4,0x79,0xe1,0xe2,0xf4,0xbc,0x58,0xb4,0x09,0x59,0x19,0x44, +0x19,0x16,0xf4,0xe4,0x7c,0x84,0xb3,0xe1,0x0d,0x5d,0x96,0xc0, +0x2b,0x1d,0xef,0xed,0x7f,0xbe,0x3c,0x9e,0x5e,0x04,0x3b,0xd1, +0x96,0xca,0x80,0x71,0x3e,0x9e,0x8c,0xea,0x80,0x5c,0x72,0x03, +0x2f,0x1e,0x7d,0x00,0xd4,0xf9,0xd6,0x2f,0x58,0x43,0x03,0x6d, +0x8e,0xcb,0xb4,0xd7,0x68,0xa1,0x32,0xc5,0xc9,0xdd,0xb4,0x62, +0xf9,0x75,0xcc,0x1d,0x4d,0x1a,0xe7,0x41,0x59,0x0c,0x30,0x09, +0x17,0x2b,0x46,0xc2,0x2c,0xe0,0xe3,0xf4,0x05,0x91,0xc5,0x81, +0x03,0x70,0xe1,0x34,0x12,0x04,0x4b,0x82,0x08,0x1b,0x64,0x08, +0x8d,0xe7,0xf6,0x9b,0x30,0xf8,0x07,0xf9,0x8a,0xc6,0xed,0x07, +0xf8,0x38,0x94,0x91,0xa5,0x9f,0xaa,0xae,0xfd,0x00,0xf2,0xab, +0xa1,0xda,0x96,0xfc,0x65,0x44,0x7b,0x7b,0x07,0xed,0x28,0xa2, +0x41,0xa5,0xfb,0xa0,0xe9,0xc6,0x47,0x13,0x8f,0xed,0xb9,0x57, +0x9b,0xae,0x99,0x53,0x35,0xbd,0x6f,0x39,0xf4,0x50,0x91,0x41, +0xf5,0x38,0x70,0xb8,0xbd,0x1b,0x14,0x8e,0xd5,0x22,0xda,0x33, +0xf4,0x2f,0x4c,0x8f,0xe8,0x85,0x94,0xbe,0x65,0xb1,0x10,0x95, +0x89,0xcc,0x8b,0xd6,0x15,0xcd,0xaa,0x17,0x36,0x10,0xec,0x26, +0x13,0x98,0xa6,0x7a,0x58,0xf5,0x29,0x6b,0x10,0x4d,0x81,0x75, +0x0c,0xc3,0xa3,0xb6,0x3b,0x12,0xa2,0x2c,0x50,0x2d,0xd8,0x80, +0x61,0xa9,0xb0,0x9e,0xf7,0xe3,0xab,0xf1,0x30,0xb4,0x7b,0x91, +0xbd,0x4c,0xad,0xec,0x4f,0x79,0xc6,0x6f,0x01,0xc8,0x17,0xf2, +0xf7,0x47,0xef,0x5f,0xee,0xbd,0xfb,0xc7,0xcb,0xdd,0x3c,0x79, +0xf0,0xc0,0xbf,0x25,0x0e,0x7d,0x25,0xac,0x0e,0x27,0x27,0xb6, +0x8d,0xbe,0x70,0x0b,0x71,0xbb,0xb4,0x86,0x4c,0xc2,0x6f,0x6c, +0xef,0xb5,0x63,0x80,0xea,0x4f,0x3e,0x7e,0xf7,0x19,0xda,0xd6, +0xf8,0x47,0x31,0x9a,0x97,0x84,0xd7,0xa3,0x57,0x13,0xff,0x4c, +0x1a,0x51,0x9d,0xf0,0xee,0xf8,0x57,0x78,0xbd,0x38,0x2f,0xcd, +0x0c,0xfe,0x51,0xef,0xbc,0xa6,0xdb,0x93,0xa7,0x05,0xe0,0x80, +0x81,0xaa,0xde,0xfc,0x20,0xb5,0xa6,0x0b,0xd2,0x1e,0x1c,0x33, +0x3d,0x57,0x83,0xd8,0xc5,0x60,0x2f,0x1f,0xe1,0x34,0x3f,0x2c, +0x9b,0xc5,0x04,0xfe,0x34,0x5a,0x8f,0x7a,0x29,0x99,0x63,0x95, +0x4d,0x22,0x01,0x7e,0x8f,0x3f,0x53,0x06,0x02,0x3d,0x42,0xfb, +0xaf,0x94,0x6e,0x78,0xbe,0x9a,0xbc,0xa1,0x0b,0x37,0xf9,0xcd, +0x83,0x62,0x94,0x89,0x07,0x1a,0x0a,0xfb,0xbe,0x4e,0xb8,0xc5, +0xc0,0x21,0xec,0xa8,0x46,0x52,0x44,0xb1,0xce,0x3d,0x49,0x38, +0x6e,0x51,0x71,0x8c,0x17,0x65,0x1f,0x1e,0x15,0x20,0x43,0x1d, +0x61,0xd2,0xd1,0x09,0xaa,0x20,0xa8,0xa1,0x8f,0xfa,0xdc,0x9e, +0x6b,0xc7,0x9f,0xd7,0x8e,0x8a,0xd1,0xd1,0xe0,0x21,0xc5,0x45, +0xd7,0xce,0x50,0x3c,0x3a,0x3a,0x2a,0x84,0x66,0xcb,0x5d,0xdc, +0xf8,0x99,0x7b,0xd9,0x1e,0xa8,0xde,0x90,0xb4,0x27,0x3a,0xad, +0xd7,0x88,0x7b,0xcb,0x6e,0xa2,0x7d,0x7a,0xaa,0x3b,0x30,0xd7, +0x3e,0x7a,0xb8,0x97,0x4c,0x18,0xf2,0x22,0xe8,0x29,0x60,0x84, +0xba,0x0d,0xfb,0x41,0x26,0xf9,0x6a,0x09,0x1b,0xb9,0x1f,0xf5, +0xbb,0x75,0x1c,0xae,0x61,0x24,0x34,0x60,0xce,0x8e,0x21,0x63, +0xfa,0xed,0x20,0xd5,0x25,0x30,0xb2,0x56,0x23,0x1e,0x79,0x35, +0x5f,0xe2,0xbe,0xc1,0xfa,0x19,0x12,0x2d,0x62,0x34,0x48,0x83, +0x92,0xfa,0xbb,0x97,0x12,0xd6,0x3f,0x87,0xee,0x70,0x03,0xaf, +0xd1,0xcb,0x1a,0xfd,0xc3,0x07,0x87,0x9f,0xf0,0x12,0xc8,0x61, +0xc3,0x3d,0xc3,0x20,0x33,0xe4,0x80,0xc8,0xbe,0xa7,0x29,0x0e, +0xb9,0xdb,0x07,0x55,0x31,0xe2,0x61,0x25,0xe3,0xe6,0x21,0x0e, +0x9c,0x15,0x63,0xe8,0xe1,0x5a,0x79,0x3e,0xbd,0xbe,0x18,0xd1, +0x4d,0x6b,0x18,0x74,0xae,0x4d,0xf1,0xba,0xf5,0xc3,0xc6,0x11, +0xd0,0x7b,0x64,0xd6,0x8e,0xe8,0x13,0x8f,0xd2,0x87,0x6a,0x80, +0x9a,0xca,0x68,0x83,0xef,0x11,0x3f,0xca,0xbe,0x6b,0x72,0x7f, +0x7f,0xdf,0x8d,0xbc,0xa0,0x7b,0xdd,0x18,0xe4,0xcb,0x28,0x20, +0xbe,0x7d,0x07,0xfc,0xf9,0xe3,0x1e,0x3a,0x53,0x5d,0x86,0xaa, +0x84,0x17,0x8e,0xec,0x06,0x4e,0x7e,0x2f,0xda,0xbb,0x74,0xed, +0x0b,0x54,0x3c,0x41,0xa3,0x73,0x1b,0xf6,0x85,0x83,0x1d,0xca, +0xf1,0xd2,0x95,0x46,0xc0,0xf8,0x01,0xea,0xba,0xcf,0x78,0x84, +0xd0,0xbd,0xa5,0x39,0x46,0x82,0xc8,0x70,0xb8,0x98,0x5e,0x97, +0xe4,0x8b,0xc7,0x32,0x37,0x3c,0x10,0xc6,0x1a,0x14,0xcd,0xfe, +0xfd,0xdd,0x8c,0xdc,0x2e,0x0d,0xba,0x3e,0x04,0xe3,0x9b,0xe2, +0xe3,0x58,0xd3,0x9b,0x86,0x55,0xe5,0xfe,0x35,0x98,0x4e,0xea, +0x79,0xb1,0x88,0x87,0x28,0xdf,0xcc,0xbf,0x1b,0x9c,0x59,0x67, +0x54,0x21,0x50,0xa7,0xdc,0x75,0x2b,0xca,0xfc,0x63,0x14,0x27, +0x90,0x86,0xf3,0xc6,0x46,0xf2,0x20,0x59,0xc7,0x99,0xe9,0x1d, +0x19,0x80,0x28,0x1f,0xe0,0x96,0x98,0x8f,0x00,0xdd,0x8d,0x32, +0x30,0x2c,0x0d,0x88,0xe5,0x58,0xb1,0xea,0x95,0x3c,0x82,0x92, +0x0e,0x94,0x48,0xff,0x61,0xab,0x4a,0xe6,0x8a,0x22,0x3c,0xf3, +0xdb,0x5d,0x89,0xe3,0xcf,0xe1,0x16,0xe9,0x2f,0xbb,0x31,0x47, +0x6f,0x55,0xbe,0x24,0x6e,0x49,0x02,0x34,0x3d,0x02,0xcd,0xc2, +0x44,0x89,0xc7,0xc3,0x63,0x45,0x15,0x60,0x67,0x62,0xae,0x91, +0x75,0xdb,0x57,0x98,0x9f,0xd1,0xc3,0x3b,0x6e,0x56,0x57,0x30, +0x4d,0x69,0xc0,0xe5,0x1a,0xb8,0xef,0x72,0x07,0x46,0x9c,0x15, +0xac,0xca,0xd6,0xc3,0x51,0xa5,0xe7,0x3c,0x8a,0xc3,0x66,0x94, +0x8f,0xe2,0x3c,0xe7,0xe1,0x41,0x63,0xb8,0x85,0x4a,0x66,0x3e, +0xde,0x7d,0xa6,0x46,0xec,0x27,0x27,0xa1,0x4e,0x6f,0xe8,0x67, +0x63,0x83,0x7e,0x58,0x1e,0x67,0x2f,0x54,0x8d,0xe0,0x33,0x38, +0xbb,0xb0,0xd4,0xf2,0x39,0x56,0xc8,0xdc,0x46,0xd7,0x57,0x78, +0xfd,0x2f,0xd9,0xbd,0xc6,0x60,0x46,0x20,0x1b,0x95,0xe4,0x67, +0x7e,0x6d,0x46,0x50,0xc0,0xb7,0x86,0xb3,0x31,0x39,0xad,0x19, +0x5e,0xa0,0x58,0x3d,0x6a,0xad,0xfd,0x54,0x8e,0xd7,0x1e,0xda, +0x95,0x52,0x73,0xb0,0xb5,0xf9,0x74,0x8d,0x23,0x99,0x7c,0x5e, +0xbb,0x9e,0x14,0xb0,0x99,0x44,0xf6,0x5c,0xb6,0xd6,0xde,0x0b, +0xae,0x6c,0x0d,0x38,0x9b,0x59,0x73,0x55,0x61,0xb6,0x18,0x16, +0x2a,0x3e,0xe8,0x9b,0xa4,0x5b,0xdf,0xba,0x37,0x20,0x11,0x38, +0xa0,0xe5,0xca,0x0e,0x5a,0xef,0x2c,0xd5,0x54,0xd4,0xed,0x92, +0xde,0x36,0x6e,0x78,0x3a,0x36,0x6a,0xc6,0x0c,0x8d,0xcc,0x48, +0x53,0xc4,0x2d,0x2c,0xdc,0xab,0xf4,0xbb,0x81,0x30,0xbd,0xc6, +0x75,0xb0,0x65,0x78,0x5e,0xec,0x94,0x00,0x49,0xd2,0xa5,0x91, +0x16,0x2b,0x25,0xb6,0x22,0x13,0xd1,0xdc,0x36,0x5b,0xeb,0x67, +0xe9,0xcd,0x7f,0xcb,0x34,0xe5,0x46,0xaa,0xed,0x25,0x13,0x0d, +0x4d,0x51,0xe7,0xb2,0x32,0x48,0x7d,0xa3,0x5b,0x50,0x72,0xbd, +0x20,0x74,0x47,0xd3,0x35,0x97,0x61,0x1f,0x02,0x0f,0xa0,0x9f, +0xce,0x51,0xc7,0x63,0xb3,0xd0,0xd8,0x81,0x9f,0x74,0x83,0xa6, +0xdd,0x88,0x88,0xdc,0xe1,0x22,0x75,0x0d,0x77,0x24,0xb9,0x42, +0x8b,0xfa,0x8b,0x74,0x4a,0xe2,0x4f,0x52,0xd3,0x05,0xbb,0x99, +0x60,0xdd,0x92,0x62,0x13,0x72,0x31,0xc9,0x8e,0xbd,0x57,0x29, +0xbd,0xbb,0x3e,0x63,0x95,0xbc,0x15,0x0b,0x71,0xaa,0x44,0x9d, +0x08,0xa7,0xb5,0xf3,0x5a,0x38,0xd3,0xe9,0xe4,0x25,0x26,0xc7, +0xf8,0x21,0xd5,0x3c,0x1c,0xdc,0x9c,0x8c,0xbe,0xd9,0xfd,0xc2, +0xb1,0xd9,0x09,0xc0,0x2e,0x8b,0x11,0xba,0xec,0x5f,0x6f,0x54, +0xf0,0x2e,0x16,0x31,0xba,0x90,0xa8,0xe9,0x68,0x14,0x15,0xc3, +0xb3,0x1b,0xa4,0x46,0xc2,0x4c,0xa7,0xf1,0x28,0x59,0x2c,0x78, +0x17,0x74,0xdb,0xb1,0x40,0xed,0x19,0xc0,0x7d,0x3b,0x8c,0xa1, +0xd5,0xf8,0xcb,0x55,0x4d,0xf1,0x68,0x21,0x60,0x14,0x49,0xc2, +0xd1,0x11,0x20,0x91,0xbe,0x17,0x58,0xfa,0xab,0xdc,0x0d,0x06, +0xbc,0x29,0xe2,0xc6,0xb0,0x95,0x0d,0xc4,0x33,0x0d,0xeb,0x55, +0x6c,0xfb,0xe7,0xd3,0x4f,0x5f,0x78,0xb0,0xf5,0x85,0x17,0x3f, +0x5b,0x5c,0x49,0xe5,0x34,0xc9,0xa2,0xed,0xc7,0x27,0x3b,0xbd, +0x44,0x1d,0x04,0x27,0x59,0x62,0x8f,0xe4,0x93,0x81,0x77,0x6a, +0x3b,0x39,0xdb,0x3c,0x47,0x97,0x02,0xee,0x78,0xfc,0x87,0x22, +0x3a,0x79,0xf9,0xef,0xf8,0x0e,0xac,0xe4,0x4b,0xbe,0xc3,0x51, +0x9e,0x05,0x9f,0x74,0xdb,0x77,0x50,0xbc,0xb3,0xbf,0x60,0xd0, +0x52,0xdb,0xec,0x88,0x2b,0xd0,0x7b,0x50,0x4a,0x89,0x61,0xae, +0xf9,0x29,0xbd,0x71,0x8f,0x64,0xcf,0xc5,0x8f,0x1c,0xe4,0xda, +0x26,0xdb,0x25,0xc7,0x25,0x05,0x2d,0x60,0x28,0x5a,0x9a,0x57, +0xa6,0x9c,0x60,0x60,0x6b,0xaa,0x97,0x6c,0x17,0x8c,0x43,0xaa, +0x22,0xbb,0x71,0xd0,0x76,0xa1,0x60,0x89,0xea,0x19,0x12,0x1c, +0xf6,0xc9,0x4b,0xd6,0x97,0x9e,0xb2,0x86,0x9a,0x50,0x65,0x04, +0xc7,0xe8,0x82,0xdb,0x9c,0xfe,0x72,0x4d,0x6c,0xd9,0x8d,0xda, +0x3c,0x54,0x4a,0x42,0x1f,0xdc,0x69,0x35,0x67,0x51,0xbf,0x50, +0xd7,0x39,0xe9,0xf0,0x6b,0xcc,0x81,0x97,0x0e,0x9c,0xd2,0xa6, +0x74,0x81,0x19,0xad,0x4b,0x68,0x43,0x1d,0x84,0x17,0x23,0xec, +0x21,0x19,0xe3,0x92,0x33,0x32,0x10,0x4f,0x6c,0x09,0x62,0x19, +0x24,0xf0,0x07,0x5d,0xeb,0xca,0x57,0xc6,0xa1,0x0e,0x98,0x54, +0x14,0x79,0x88,0xc7,0x39,0x45,0x2e,0x76,0x0a,0xf2,0x8c,0x1c, +0x66,0xf7,0x8b,0x81,0x3e,0xd2,0x8a,0x84,0x90,0xf8,0x23,0x28, +0x82,0x52,0x9c,0x88,0xb7,0x57,0x2a,0xb4,0x9b,0x46,0x4d,0xa3, +0xe4,0xd5,0x06,0xe4,0xf6,0xef,0x27,0xeb,0x12,0x8c,0x13,0x8d, +0x74,0x57,0x02,0xf5,0x12,0x74,0x3a,0x57,0x09,0x9f,0xde,0xe2, +0x83,0x0a,0x2f,0x23,0xd5,0xf5,0x87,0x76,0xc6,0x12,0x65,0x86, +0x9d,0x58,0x73,0x62,0x15,0xb5,0x28,0x6d,0x97,0x82,0xb4,0xd4, +0x54,0xb1,0xb6,0xd4,0x59,0x74,0x00,0xac,0xd6,0x1d,0xf8,0xac, +0x97,0xfa,0x24,0x93,0x0f,0x7a,0xf2,0x1a,0x64,0x32,0x0e,0x2b, +0x8d,0xcf,0xc4,0x68,0x44,0x95,0x43,0x2d,0x9f,0x67,0x18,0xbf, +0x3f,0xce,0xe6,0x77,0xb2,0x78,0xf0,0x0a,0x50,0x6e,0x7d,0xd4, +0x85,0xaf,0x3a,0x02,0xb9,0x55,0x37,0xf9,0x75,0xbb,0xed,0x27, +0xe3,0x2f,0x7a,0x36,0xde,0xa2,0xa3,0x2c,0x6f,0x35,0x6d,0xb8, +0xdd,0x0a,0xda,0x8f,0x20,0x7b,0xa4,0xe3,0xbf,0x60,0x90,0xdf, +0x0d,0x82,0xe1,0x2f,0xcc,0xdd,0x60,0xdc,0xd2,0xfa,0xe3,0xfd, +0xa3,0xa5,0xcd,0x06,0xf6,0x73,0x1e,0x22,0x84,0xb3,0xf1,0x1d, +0xe6,0xff,0x5f,0x35,0x27,0xcc,0xb5,0x3c,0x7a,0xaf,0x34,0x14, +0xa4,0xfd,0xa5,0x56,0xf1,0xe3,0x7a,0xe5,0x19,0x9f,0xe6,0xdc, +0x1e,0x63,0x70,0xc1,0x5f,0x7d,0x21,0x5f,0x00,0xf1,0x29,0xb9, +0x7a,0xae,0x30,0xf6,0x5a,0xb5,0x77,0x19,0xdc,0xd1,0x3f,0x51, +0x6e,0x0b,0xd4,0x34,0xae,0x08,0x8c,0x81,0x39,0x1f,0xfa,0x2c, +0xf1,0x77,0xc4,0xd8,0x69,0x09,0x03,0xb2,0xa1,0x7d,0x79,0x32, +0x2b,0xae,0xc2,0x83,0x85,0xc8,0x38,0x44,0x7d,0x9c,0xce,0xa8, +0xbf,0x48,0x7b,0x6f,0xfd,0x3f,0xbb,0x16,0xc1,0xb3,0x8e,0x2d, +0x10,0x3a,0x2c,0xe6,0x24,0xcf,0xfd,0xad,0x59,0x89,0x28,0xc9, +0x39,0x3f,0xcd,0x2e,0x38,0x0b,0x6f,0x48,0x41,0xa9,0xdc,0xab, +0xd0,0xe8,0xc4,0xa4,0x1b,0x52,0x07,0x23,0x60,0xde,0x50,0x65, +0x0d,0xc7,0x16,0x14,0x66,0xf2,0x8e,0x63,0x16,0xd6,0x5c,0x7d, +0x50,0xf4,0x2f,0x2d,0xff,0x0c,0x1a,0x47,0x19,0xc6,0x54,0x8c, +0xc9,0x9c,0xb1,0x8b,0x58,0x95,0xc9,0xa9,0xcb,0xbb,0x1f,0x0f, +0x5e,0xbf,0x7b,0xbb,0x1f,0xde,0xd0,0xc6,0xa3,0x81,0x46,0x2f, +0x3b,0x2c,0x9b,0xc3,0xd2,0x1e,0x15,0xa4,0x3d,0x4e,0x39,0x9b, +0x4d,0xaf,0xaf,0xfc,0x59,0x01,0xba,0x9a,0x85,0x07,0x58,0x47, +0xf0,0xd5,0x69,0x7a,0xe9,0xef,0xe1,0x68,0xf0,0x88,0x7c,0xfc, +0x88,0xa3,0xf9,0x28,0x27,0xd4,0xff,0x06,0xe9,0x87,0x74,0x1b, +0x5c,0x1d,0x54,0x70,0xdd,0xd1,0x39,0x05,0x51,0x85,0x36,0xe0, +0xb0,0xe1,0xd8,0x73,0xb7,0x59,0x6e,0x82,0xcb,0x61,0xe2,0x59, +0xe3,0xb6,0x7b,0xd5,0xfd,0x84,0x9b,0x12,0x9a,0xcd,0x5d,0xd6, +0x18,0x44,0xa2,0x90,0xba,0x41,0xec,0xae,0x1c,0xcb,0xf5,0x62, +0xdf,0xc4,0x6e,0x7a,0xc8,0x74,0x91,0x23,0x74,0xeb,0x40,0x84, +0x3b,0xd6,0x5c,0x4f,0x3e,0x4e,0xa6,0x9f,0x26,0xf2,0x86,0x0e, +0xcd,0x38,0xe8,0x95,0xc4,0xaa,0xb4,0x9a,0xd6,0x33,0xff,0x45, +0xc1,0xf7,0xe1,0x7a,0x76,0x4a,0x36,0xb6,0xc7,0xd3,0xeb,0xc9, +0x88,0x8f,0x7f,0x4a,0x7f,0x0f,0x14,0x73,0x8b,0x49,0xa1,0xac, +0x73,0x14,0xaa,0x23,0xe3,0x09,0x39,0x0a,0x29,0x39,0x4a,0x6f, +0x82,0x3a,0xeb,0x0b,0xe5,0x2b,0xcb,0xe7,0x21,0xb6,0x25,0x53, +0x02,0x52,0xbd,0x64,0xc7,0x22,0x98,0xff,0x5c,0x39,0x88,0xc4, +0x3b,0x78,0xaa,0x56,0x7d,0x9d,0xca,0xdd,0xb3,0x72,0xcc,0x02, +0x05,0x19,0xb9,0xd2,0x16,0xd4,0xeb,0x16,0xea,0x8d,0x8d,0x30, +0xdd,0xba,0x59,0x4f,0x85,0x30,0x7e,0x5f,0x41,0x9b,0x75,0xc5, +0xcb,0xd9,0xde,0x88,0x4e,0xd4,0x93,0x15,0xd2,0xef,0xa2,0x5b, +0x62,0x0d,0xe0,0x3d,0xb0,0x9f,0x34,0x55,0xee,0xba,0x85,0xa5, +0xa9,0x02,0x11,0x90,0xc6,0x43,0x49,0xbe,0xeb,0x1f,0x18,0xa1, +0xb9,0xb7,0x96,0x34,0xd5,0x51,0x4d,0xda,0x4c,0xd6,0x7a,0x49, +0x37,0xfc,0x72,0x6c,0x2a,0x5f,0x46,0x31,0x5c,0x09,0x15,0xd5, +0x08,0xc0,0x55,0x7e,0x5c,0x30,0x6a,0x68,0x74,0x9e,0x93,0x58, +0xe1,0x8a,0x4c,0xaa,0xe5,0x33,0x5c,0xc3,0xad,0xba,0xa6,0xe4, +0xdb,0x2f,0x56,0xf8,0xd9,0xe3,0x26,0xab,0xad,0x59,0x79,0x91, +0x6c,0x65,0x7b,0xd1,0x74,0x4f,0xef,0xb3,0x35,0x91,0xab,0x70, +0xee,0x42,0xff,0x7e,0x31,0x39,0xbb,0x18,0x0b,0x24,0x7f,0x97, +0x33,0x6b,0xf2,0xdd,0x2b,0x39,0x74,0x57,0xed,0x26,0xe8,0x76, +0xb9,0x64,0x1d,0x9f,0xf4,0xbb,0xc1,0x50,0x3f,0x46,0xba,0x1e, +0x9f,0x1e,0x71,0xee,0xe2,0x4f,0xaf,0xf1,0x65,0xe3,0x3b,0xa4, +0x9c,0xe7,0x89,0x43,0x26,0x51,0xc1,0xfd,0xbd,0xa1,0x31,0x5e, +0x0f,0x5b,0xdd,0xa1,0x69,0x16,0x44,0xa1,0x75,0x78,0x82,0x82, +0xbd,0x6a,0x8d,0xb0,0xae,0x67,0xea,0xbb,0x6a,0x47,0xbe,0xbf, +0xb4,0x15,0x91,0x8c,0x7d,0xce,0xdb,0x93,0xb0,0xc7,0x57,0xd8, +0x24,0x7f,0x61,0xf3,0x04,0xbd,0xa0,0x17,0x8a,0x46,0xf5,0x33, +0x52,0x36,0x6c,0x76,0x23,0x64,0x4f,0x7c,0x8d,0xd5,0x8e,0x11, +0x7f,0x3f,0x0c,0x35,0x49,0x88,0xb6,0x7b,0x72,0xcb,0xb8,0xc0, +0x13,0xcb,0x12,0x2d,0xf1,0xd6,0x24,0xd8,0x4c,0xe5,0x32,0x5b, +0x37,0xdc,0x9b,0x59,0xc2,0x30,0xc4,0x52,0x23,0xe1,0x29,0xa4, +0x0d,0xd3,0x57,0x44,0x36,0xce,0xbd,0x41,0x1d,0x55,0x49,0x16, +0x8c,0x3a,0xc0,0x31,0x7d,0xe4,0x4a,0x63,0x15,0x58,0x22,0x87, +0x17,0x65,0xc3,0x7e,0x92,0x89,0x89,0x5c,0x2c,0x5c,0x5e,0x4e, +0x2e,0x71,0x62,0x80,0xca,0x25,0xd8,0x78,0x84,0x7e,0x61,0x77, +0xd3,0x9e,0x0f,0xf5,0xa3,0x72,0xbc,0xf7,0xaf,0xb7,0xd0,0xc6, +0x06,0xe1,0xab,0x86,0x7e,0xa6,0x30,0x81,0xd5,0x71,0x42,0xd0, +0xd1,0xc8,0x10,0x99,0xed,0x96,0x81,0xe1,0x40,0x6d,0x4b,0xd0, +0x97,0x70,0x75,0xdf,0xa3,0x48,0x45,0x3e,0xc8,0xd4,0xbb,0x3c, +0x9b,0xf1,0x1f,0x74,0x21,0xfd,0xec,0x47,0x8e,0xe3,0x61,0x5f, +0xa5,0xc6,0xe8,0xdd,0x90,0x74,0xf6,0x46,0x9f,0x2e,0x72,0xd2, +0x6b,0xd2,0x38,0xb3,0xde,0xd9,0x7e,0x38,0x1d,0xd9,0x59,0x2a, +0x2d,0x6b,0xbc,0x18,0x1e,0x83,0x14,0xa1,0xe8,0x28,0xf3,0x9b, +0x24,0xc9,0x30,0xdc,0x61,0x44,0x6c,0x09,0x12,0x27,0xc8,0x49, +0xea,0x26,0x75,0x7c,0xb5,0x9a,0x4f,0x2a,0x4a,0x5e,0x05,0xcb, +0x57,0xc2,0x85,0x53,0xda,0x14,0xe1,0x69,0x14,0xaa,0xce,0x11, +0x53,0xaf,0xa4,0xa8,0xbd,0x78,0xc6,0xc1,0x2b,0x40,0x99,0xf2, +0x75,0xaf,0x52,0x5c,0xbd,0x92,0x05,0x9c,0x55,0x08,0x8c,0xe7, +0xf9,0x7a,0x07,0xcf,0x9d,0x5d,0x2c,0x70,0xb4,0xac,0x45,0xc9, +0xf0,0xd5,0x64,0x63,0xc3,0xde,0x34,0xf4,0x54,0xa4,0xb4,0x5a, +0xb8,0xb2,0x7a,0xca,0xf5,0x07,0xa9,0x8b,0x87,0x46,0x18,0x5e, +0xcb,0xe9,0xaa,0x7f,0x79,0xe6,0x11,0x59,0xbd,0x90,0xcf,0x6c, +0x36,0xd3,0x0b,0x65,0xb5,0xc1,0x37,0x39,0x7c,0x81,0xbe,0x87, +0x1c,0x68,0xfa,0x59,0xfe,0x67,0x8a,0x65,0xc8,0x58,0xaf,0xdc, +0xf5,0x85,0x25,0x4e,0xe9,0xaa,0xcf,0x50,0xdf,0xaa,0x8f,0xa6, +0xe4,0xf4,0x29,0xff,0x58,0x39,0x71,0xae,0x39,0x90,0xaa,0x7e, +0x07,0xf7,0x40,0xdf,0xf6,0xd1,0x85,0xb3,0x5f,0xe1,0xec,0x8f, +0xfe,0x44,0x2a,0x93,0x63,0xa6,0x68,0x80,0xe4,0x34,0xf2,0xbe, +0xfb,0x1c,0x7f,0x24,0xd9,0xb0,0x37,0x14,0x70,0xae,0x87,0x5b, +0x3f,0xc2,0x82,0xf7,0x64,0xef,0x0f,0x8c,0x7a,0xb4,0x78,0x9c, +0xea,0x99,0xed,0x52,0x53,0x68,0x6b,0x3b,0x7e,0x1c,0xaf,0xd4, +0x7d,0x24,0x2b,0x86,0x74,0x53,0xaf,0xb6,0xbb,0x32,0xbb,0x35, +0x0b,0x52,0xd3,0x75,0x7f,0x17,0xc3,0x1d,0x3c,0x4b,0x79,0x7d, +0xf5,0x1e,0x8f,0x7c,0x6e,0x96,0x5d,0xf7,0x52,0x3f,0x88,0xdc, +0xc0,0xc9,0x43,0x12,0x5c,0x31,0x74,0x2a,0x5e,0x4b,0x5d,0x38, +0x68,0x14,0xca,0x3c,0xcf,0x6b,0x09,0xef,0xea,0x01,0xa6,0x9e, +0x17,0x0b,0xfb,0xb2,0x24,0xf6,0x90,0x8f,0x8a,0x12,0xf6,0xae, +0x95,0x8e,0x65,0xe6,0x91,0xf3,0xdf,0xe8,0xfa,0x66,0x85,0xaf, +0x88,0x1a,0xc4,0x9e,0x1f,0xaf,0x6c,0x62,0x3b,0xfe,0x2a,0xe3, +0x8d,0x6b,0x13,0xc4,0x96,0xc0,0xcc,0x51,0xca,0x36,0x18,0xb6, +0x8f,0x61,0x0c,0xf9,0x9d,0x8b,0x8d,0x92,0xa0,0x9a,0xa4,0x17, +0x8c,0x2a,0xe0,0x6b,0xee,0x72,0x3a,0x52,0x08,0xe3,0x95,0x2b, +0x83,0x07,0x57,0xd3,0xba,0x6a,0xa1,0x65,0x9a,0x05,0xed,0x75, +0x3b,0xb6,0x5e,0x2d,0x3a,0xd8,0xd3,0xa7,0x8a,0x59,0xe7,0x6d, +0xcd,0xcc,0xf3,0xca,0xa0,0x16,0x66,0xa4,0x60,0x76,0x7c,0xe1, +0xae,0x7f,0xc4,0xf9,0x8d,0x2d,0x11,0x4f,0xce,0x18,0x61,0xdf, +0x17,0x09,0xa6,0xd0,0xed,0xf3,0x2d,0x58,0x29,0x58,0xb9,0xc1, +0x84,0x3d,0xcb,0x3d,0xbe,0x5e,0x23,0x5c,0xc4,0x72,0x7b,0x02, +0x92,0x41,0x69,0x2a,0x6a,0xef,0xb4,0xb6,0x48,0xfb,0xd3,0x48, +0xc5,0xa9,0x2c,0xb5,0x52,0x12,0x4f,0x68,0x19,0x67,0x99,0x1e, +0x4c,0x94,0xb2,0x8c,0x57,0xc7,0xbc,0x1f,0x56,0x5c,0x47,0x2e, +0x07,0x06,0x0e,0xcb,0xc5,0x32,0x8a,0xa8,0xa6,0x42,0x64,0x56, +0xc1,0x8c,0xbe,0x7d,0xe2,0x7a,0x2b,0xd5,0x04,0xad,0x1b,0xb5, +0x46,0x54,0x18,0x0f,0x39,0xa3,0x9a,0xe8,0xeb,0xd6,0xe3,0x2e, +0x03,0x29,0xbc,0x96,0xa0,0xb0,0xf1,0xea,0x50,0xc5,0x98,0xd2, +0x54,0xcb,0x02,0xb4,0xc7,0x37,0x91,0x7d,0x43,0xd0,0xd8,0xb7, +0x18,0x37,0x30,0x9c,0x06,0xb7,0x46,0x08,0x51,0x23,0x55,0x3e, +0x9b,0x31,0x74,0x06,0xb0,0xe5,0xd1,0xc4,0x84,0x60,0xf6,0x1b, +0xa3,0x9e,0xb6,0x2d,0x24,0xf5,0xf2,0xeb,0xc6,0x86,0xc2,0xc3, +0x96,0xcb,0x75,0xa5,0x82,0x32,0x69,0x8c,0xb8,0x18,0x79,0xac, +0xe8,0x3f,0x4a,0xa3,0xc4,0x3d,0x42,0x05,0xdc,0x03,0x07,0x8d, +0x8a,0xaa,0x40,0x3b,0xd7,0x3d,0x46,0x2f,0x83,0x6a,0xbc,0xf1, +0x4e,0x2c,0xaa,0xc3,0x71,0xf4,0x08,0x09,0x0e,0x44,0xda,0xe0, +0x29,0x72,0x3d,0xd3,0xeb,0x8d,0x55,0xef,0x72,0x52,0x66,0x95, +0x9e,0x52,0x22,0x9e,0x85,0x29,0x7d,0xa1,0xff,0x1e,0x19,0x58, +0x9e,0xb0,0x98,0x00,0x6e,0xe4,0xdb,0xda,0xb3,0xac,0x9b,0x6f, +0x9e,0x1f,0x04,0x52,0x68,0xa6,0x11,0x19,0x60,0x9c,0x8e,0x12, +0xcf,0x35,0x23,0x0a,0x96,0x41,0x9b,0xf7,0x74,0xa3,0x52,0x00, +0x3c,0x67,0xf3,0x93,0xad,0x9a,0x3a,0x32,0xd3,0xed,0xc9,0x50, +0x30,0x12,0x25,0xd1,0xcb,0x58,0xcd,0x66,0x37,0xfe,0xbc,0x60, +0x52,0xa4,0x95,0x8f,0x87,0x8e,0x4d,0x5d,0x65,0x76,0x53,0x4a, +0x2b,0x55,0x77,0x15,0x27,0xd5,0x6c,0x3d,0xad,0xe1,0x5f,0x88, +0x12,0x47,0x57,0x05,0xab,0xf3,0x55,0x42,0xa6,0xda,0xec,0x77, +0x44,0x7a,0x1a,0xaf,0x00,0x88,0x7d,0x6e,0x45,0x07,0x9c,0xa6, +0x15,0x8f,0x75,0x42,0x3e,0xfa,0xac,0xbb,0xaf,0x55,0x37,0xf7, +0xc8,0xd1,0x5a,0x63,0x58,0xae,0x1d,0x51,0x0f,0x1e,0xa5,0x3d, +0xcc,0x5f,0x13,0xc3,0xdb,0xb4,0x27,0x96,0xb7,0x55,0xb3,0xdd, +0xd0,0xf8,0x36,0x72,0x7b,0xe7,0x3f,0x20,0xf0,0x79,0x17,0x30, +0xec,0x94,0x2f,0x22,0x3b,0x31,0xc5,0x9a,0x74,0xd7,0x44,0xe4, +0x32,0x4e,0xe8,0x72,0xd6,0xe1,0x36,0xf7,0x29,0xed,0x52,0x54, +0xd6,0xd7,0x03,0xe3,0x44,0xda,0x10,0xa5,0x8d,0xe1,0x25,0x62, +0x55,0x5c,0x5f,0xcf,0x56,0x97,0xb9,0xda,0x04,0xb6,0x8c,0x81, +0xbf,0x19,0x88,0xb5,0xb9,0xd4,0xf9,0x37,0x31,0xc0,0xb4,0xa6, +0xaf,0xbd,0x00,0xfa,0x6f,0x83,0x94,0xa2,0x45,0x57,0xd7,0xb5, +0xbc,0xdf,0x77,0xb3,0x2a,0xdc,0x8f,0x5a,0xd1,0x63,0x39,0x18, +0x74,0x3d,0x2f,0x40,0xe5,0xaa,0xbd,0xad,0xe7,0x53,0xc5,0x15, +0x68,0xaa,0x34,0xc0,0x15,0xcf,0x03,0xac,0x0f,0xaf,0x01,0x41, +0x7f,0x67,0xe1,0x3a,0xea,0x0e,0x7d,0xfe,0x65,0x15,0x80,0xde, +0x28,0x7b,0x53,0x54,0x4e,0xb5,0x1f,0xca,0x52,0xa2,0xda,0xfe, +0xea,0x0d,0xb1,0xb5,0x96,0xf6,0x36,0x78,0x75,0xbb,0x54,0xb7, +0xfd,0x0c,0xf6,0x9b,0x62,0x3e,0x80,0x92,0x74,0x7f,0x70,0x2f, +0x91,0x2d,0x98,0xd0,0xf7,0x92,0xd9,0x22,0x91,0xed,0x0e,0x89, +0x82,0x17,0xeb,0xce,0x97,0x2e,0xd6,0xf0,0x51,0x8d,0xa0,0xcd, +0x6a,0xd6,0x6d,0x27,0xe6,0xe8,0x15,0xcc,0xd9,0x30,0x06,0xa5, +0x59,0x69,0x66,0x27,0x0d,0x1a,0xc0,0x56,0xf7,0x8c,0xa9,0x1d, +0xce,0xe9,0x5d,0x1b,0x6f,0x65,0xa5,0xcd,0xc4,0x7b,0x84,0x6a, +0xa3,0xa4,0xcc,0x20,0xf5,0x6e,0xbb,0x76,0x2f,0x41,0x46,0xdd, +0x69,0xb8,0x7f,0xa7,0x0d,0xda,0xda,0xad,0x78,0xd1,0xbc,0x92, +0x6d,0xff,0x9d,0x3f,0xab,0x9a,0x4d,0xdf,0x92,0xb7,0x5b,0xd2, +0x2e,0x35,0xca,0x44,0x03,0x72,0x7f,0x4e,0x14,0xf0,0xd0,0x09, +0xf7,0x88,0x58,0x30,0x09,0xf2,0x29,0x5a,0xad,0xcd,0x72,0xdb, +0xc7,0x2f,0x6e,0xb5,0xf0,0x73,0x6b,0x77,0xfa,0x5f,0xd2,0x86, +0x4c,0xa1,0x38,0x29,0xac,0xdf,0x45,0x52,0x9c,0x3a,0x69,0x8e, +0x3b,0x1b,0xf6,0xae,0xd1,0x72,0x5b,0x45,0x35,0xca,0x39,0xe5, +0x25,0x28,0xd0,0xbe,0xf2,0x4f,0xa8,0xe6,0xe4,0xb4,0x74,0x29, +0x4e,0xd7,0xd0,0xdf,0x9d,0x6f,0x5f,0xa5,0xe0,0x56,0x2a,0x7f, +0xef,0x2a,0x4f,0x9f,0x95,0xd9,0xd2,0x4e,0xa3,0xc0,0x87,0xc1, +0xf6,0x4d,0x2d,0x59,0xd6,0x90,0xc9,0x6a,0xef,0xd4,0xc1,0xda, +0x7a,0xc7,0x84,0x52,0x98,0x8d,0xa1,0x1e,0x45,0xf1,0x90,0x31, +0xe5,0x15,0x9c,0xa9,0x89,0x77,0x51,0xf7,0x28,0x49,0xcc,0x03, +0xcb,0x86,0x67,0x78,0xf5,0x72,0x20,0x06,0x8c,0x67,0x63,0x52, +0x68,0x32,0x77,0x1f,0xd3,0x26,0x60,0x7e,0x91,0xdb,0xb7,0x8a, +0xf1,0x94,0x8c,0x6b,0x9b,0x8f,0xd6,0x53,0xd6,0xb3,0x8f,0x6f, +0x62,0x25,0x94,0x7a,0x4c,0xe3,0xdf,0x1a,0x85,0x1d,0x4d,0x80, +0x45,0x1d,0x28,0xe0,0xf9,0xa6,0x3e,0xd6,0x34,0xab,0x8e,0x57, +0xbd,0x92,0xc7,0x86,0x85,0xb0,0x1e,0xe5,0x9c,0xf3,0xb5,0xf7, +0x92,0x90,0xf2,0xc2,0x62,0xf3,0xff,0x71,0x97,0xaf,0xbd,0xca, +0x41,0x42,0x9d,0x6f,0xbc,0xa8,0x4a,0x39,0x0e,0xfc,0x5d,0x69, +0x13,0x65,0x84,0x2f,0xbb,0x01,0xbe,0xc0,0xe1,0x5a,0x85,0xa4, +0xe8,0x14,0x43,0xf9,0x60,0xb3,0x6a,0x86,0x9a,0x22,0xf7,0xf1, +0xac,0x57,0x29,0xd6,0xa8,0x3f,0xb4,0xc2,0x09,0xe6,0x07,0x75, +0x2f,0xd4,0x7e,0x8f,0xab,0x87,0x66,0x69,0x66,0xbb,0xa1,0x17, +0x9d,0xa1,0xd4,0x02,0x07,0x27,0x71,0x35,0x10,0xfa,0x0c,0x8e, +0xad,0x26,0x98,0x98,0x2f,0xf6,0x04,0x66,0x4f,0xe6,0xf7,0xfd, +0x04,0xbf,0x71,0x27,0xd6,0xd6,0xbd,0xbb,0x3f,0x27,0xbe,0xc5, +0x96,0xe0,0x76,0x1f,0x73,0x15,0xdb,0x12,0x7d,0xac,0x46,0x1d, +0x23,0x47,0x31,0xf7,0xf5,0x3b,0x46,0xfb,0xac,0x14,0x8f,0xe8, +0xba,0x01,0x74,0x10,0x17,0xe8,0x77,0x76,0x02,0x17,0x96,0x49, +0x97,0xf7,0x75,0x27,0xe4,0x1b,0x99,0x44,0xe0,0x44,0x22,0xce, +0x2b,0x9f,0xd0,0xe2,0xe5,0xd6,0x71,0x04,0x67,0x24,0xa7,0x18, +0x26,0xa7,0xb1,0x4f,0xb0,0x10,0x23,0x48,0x58,0x92,0xe9,0xe2, +0x4e,0xd4,0x41,0xa9,0x03,0x52,0x0c,0xeb,0xe0,0x78,0x80,0xb3, +0x80,0xe8,0x8d,0x57,0x6c,0x9b,0xd7,0x3b,0xfa,0x14,0x32,0x0f, +0x3b,0x3a,0x74,0xb5,0xd6,0x0b,0x16,0x86,0x20,0x2b,0xbe,0x0b, +0x8d,0x06,0xc1,0x74,0x25,0xba,0xda,0xd2,0x72,0x2f,0x5a,0x7e, +0xd9,0x32,0x18,0x1e,0x02,0xb2,0xf5,0x78,0x6a,0x08,0x2a,0xd5, +0x5e,0xde,0x60,0xc2,0xdf,0xb2,0xce,0x6a,0xb3,0xd5,0xb0,0xb9, +0xdb,0xef,0xea,0x2a,0x02,0x14,0x12,0x6b,0xe6,0x5e,0x86,0x46, +0xd5,0xce,0xf8,0x68,0x95,0x25,0xd5,0x12,0x58,0x34,0x48,0x8a, +0x7f,0xff,0xdf,0xd7,0x63,0x0a,0x5e,0x71,0x75,0x7d,0x7c,0x51, +0x94,0xe7,0x2f,0xff,0x00,0x18,0x80,0x78,0xfe,0xe3,0xeb,0x06, +0x48,0xf6,0xd7,0x17,0x43,0xe0,0x41,0xd1,0xba,0x84,0x57,0xfe, +0x87,0xa3,0x40,0xb6,0x17,0x50,0xf4,0x30,0xe1,0xc0,0xcc,0xf1, +0x74,0x3a,0x87,0xca,0x87,0xe4,0x85,0xa3,0xc1,0x51,0xcc,0x8d, +0x43,0x62,0xa4,0x8c,0x15,0x54,0x15,0x7a,0x3e,0x09,0x3c,0x87, +0x4a,0x92,0xd4,0x99,0x3b,0x3c,0x7c,0x46,0x9f,0xb8,0x46,0x71, +0xa7,0xd9,0xc8,0x0b,0x43,0xf8,0xed,0xfc,0x27,0xde,0xf6,0x83, +0xee,0x5c,0x4b,0x7e,0x3a,0x78,0xb5,0xf9,0xb7,0xa4,0xdb,0x9f, +0x9c,0x1d,0x1e,0x66,0xe4,0x58,0x6d,0x60,0xfa,0xd6,0xc7,0x1a, +0x3c,0xe2,0xd0,0xdb,0x54,0xef,0x7f,0xa8,0x97,0x96,0x7d,0x34, +0x2d,0x9f,0x4c,0xa9,0x68,0xbb,0x7e,0x23,0x5b,0x52,0xe0,0x24, +0x93,0xf1,0xda,0x7a,0x71,0x89,0x31,0x97,0x87,0x93,0x79,0x77, +0x49,0x75,0x21,0x27,0x77,0x20,0x7c,0x71,0x6d,0xf9,0x6c,0x8b, +0xc8,0xdd,0x79,0xa8,0x4e,0x32,0xa5,0x09,0xe4,0xc3,0x95,0x7b, +0xb8,0x9b,0xe4,0x1a,0x55,0xfe,0xd4,0x51,0x09,0x87,0xcf,0xb5, +0x11,0x08,0xe5,0xfe,0xbc,0x6d,0xac,0x07,0x0f,0x5c,0x94,0x43, +0x0b,0x91,0xa4,0x5d,0x9b,0x7b,0x39,0x1d,0x5d,0x5f,0x8c,0xf9, +0x96,0xbd,0x64,0x1a,0xf4,0x46,0x9c,0x0c,0xa0,0x51,0x25,0x00, +0x89,0x2a,0x68,0x94,0xf7,0x26,0x77,0xf3,0xff,0xb7,0xaa,0x37, +0x26,0xb9,0xf9,0xff,0x9b,0x3a,0x0e,0x1d,0x4f,0x28,0x86,0xf6, +0xac,0xd8,0x1f,0x9f,0x51,0xef,0x91,0x9d,0x8b,0xf0,0x28,0x97, +0xc9,0x83,0x0b,0x4d,0xe8,0x81,0xe5,0x39,0xb7,0x0c,0x5b,0x5f, +0x6d,0x3f,0xdd,0x3a,0x2b,0x30,0x9c,0xa5,0x4e,0x7c,0xbc,0x4b, +0x89,0x79,0x90,0xb8,0xfd,0x1d,0x25,0x36,0x13,0x75,0x6a,0x5b, +0x83,0xfe,0x0a,0x84,0x76,0x4a,0xdd,0xbf,0x82,0x72,0x65,0x4c, +0x89,0x0e,0xf5,0x8d,0x84,0xaa,0x0a,0x9e,0xb4,0xa9,0x82,0xff, +0x0c,0x49,0x79,0x4e,0x89,0x59,0x48,0xca,0x93,0xad,0x33,0x68, +0x9f,0x30,0xed,0x05,0x01,0x9a,0x30,0x11,0x50,0xc6,0x14,0xf5, +0x12,0x48,0x06,0x84,0xc1,0x87,0xbc,0x9f,0x5e,0xc3,0xb4,0xb2, +0x16,0x85,0x46,0x42,0x44,0x97,0x62,0xc8,0x64,0xd3,0x73,0x07, +0x40,0xc9,0x16,0x2a,0xb7,0x0f,0xe4,0x77,0x84,0xb2,0xae,0x67, +0x17,0x3f,0x62,0x0c,0x68,0xb2,0xf2,0xf7,0xb5,0x48,0x77,0xdb, +0x08,0x34,0x18,0x6b,0x9d,0x42,0x45,0x8b,0xb1,0x6e,0x69,0x38, +0x3e,0x93,0x36,0xaa,0x81,0xd9,0x05,0xbb,0x88,0x39,0x63,0x6b, +0xe0,0xc4,0x11,0x20,0x4e,0x91,0x05,0x6f,0x44,0x5b,0x6c,0xeb, +0xc4,0x58,0xe5,0xe7,0x12,0x6d,0x07,0x08,0xab,0xab,0x49,0x90, +0x38,0x2b,0x76,0x9d,0x1a,0xd9,0xfc,0xb3,0xe7,0x8f,0xa2,0x7c, +0x15,0x4a,0x73,0x20,0x16,0xaa,0x1d,0x06,0x6a,0x2c,0x80,0x18, +0x76,0x0a,0x12,0x08,0x6b,0x7c,0x13,0x78,0x63,0x03,0xfa,0x57, +0x0e,0xcf,0x54,0x94,0x73,0x8e,0x25,0xca,0x5f,0xc7,0x79,0xe5, +0xf5,0x31,0x4c,0xc2,0x06,0xfa,0xfd,0x67,0xf1,0x8e,0xf0,0xfa, +0xa6,0x94,0x26,0x7f,0x2f,0x6e,0x95,0x5e,0xe3,0x82,0x83,0xae, +0xd4,0x41,0xe2,0xf2,0x0e,0xa3,0xa4,0x3d,0x6c,0x42,0xcb,0x4e, +0xb7,0x6a,0x8f,0x38,0xa7,0x73,0x68,0x81,0x41,0xcf,0xae,0x2f, +0x59,0x0d,0x38,0xc3,0x01,0x42,0x67,0xc6,0x3c,0x54,0x74,0x64, +0x7b,0xe9,0x34,0xd5,0xd4,0xbb,0x2f,0x5f,0x3d,0xff,0xe9,0xcd, +0xc1,0xd1,0xf3,0x17,0xa4,0x17,0x74,0xdd,0x1a,0xb5,0xb3,0x6a, +0x62,0x4e,0x30,0xde,0x39,0xe2,0xf9,0xb0,0xfc,0x6e,0x3a,0xfa, +0x8c,0x3e,0x7b,0x7f,0x7c,0xb7,0x7f,0xb0,0xf8,0xf1,0x27,0xf8, +0x87,0x91,0xf5,0xd3,0x07,0x5b,0x85,0x38,0x8c,0xe1,0x5d,0xea, +0xe5,0x78,0x7e,0x3e,0x1d,0xa5,0x5d,0xfb,0x2d,0xfd,0x09,0x6d, +0xfe,0x3c,0xea,0x8e,0x19,0x6e,0x9b,0xe1,0x63,0x33,0x7c,0x22, +0x7e,0x17,0xb1,0x99,0xc5,0xcf,0x96,0x21,0x87,0x4c,0x3c,0x36, +0x68,0x0c,0xdd,0x1e,0x3d,0xfe,0x49,0xc6,0xa1,0x29,0x86,0x4f, +0x2c,0x82,0x7c,0xf8,0x98,0x83,0xc7,0x3f,0xce,0xe8,0x67,0x3b, +0xe3,0x30,0xf0,0x6e,0x98,0x0c,0xb7,0x53,0x0a,0x3f,0x85,0xf8, +0x81,0x14,0xac,0x3c,0x07,0x7a,0x54,0x71,0xb7,0x41,0xd1,0xa5, +0x3a,0x68,0x75,0x60,0x61,0x3a,0x46,0xaa,0xdd,0x16,0x60,0x97, +0xb3,0x6d,0x73,0x1e,0xbb,0x10,0xf6,0x01,0x92,0x9e,0xc7,0x91, +0x49,0x93,0xf6,0x98,0x84,0x4e,0xe6,0x88,0x62,0xa4,0x5d,0x89, +0x8f,0x1f,0x46,0x89,0x97,0xf8,0xf5,0xe1,0x22,0xd0,0x48,0x8e, +0x87,0x23,0x68,0xa2,0x52,0x2b,0x72,0xaf,0xaf,0xf0,0xce,0xf0, +0x13,0x17,0xc5,0xbe,0x5c,0xeb,0x73,0x0d,0x86,0x5b,0x7c,0xcd, +0x36,0xf9,0x1a,0x91,0x3c,0x30,0xa4,0xa2,0xbd,0x69,0x2f,0x7d, +0x89,0xc4,0x54,0xda,0x9d,0x46,0x5f,0x51,0xbe,0x9e,0x94,0xb0, +0xcc,0x9d,0x8c,0x5f,0x0c,0x2f,0x2e,0x72,0x44,0x07,0x42,0x2e, +0xa7,0x4c,0x4f,0xdd,0x28,0x96,0xbd,0x7e,0x08,0x4d,0x9f,0x9b, +0xc9,0x58,0x11,0xe3,0x8e,0x5e,0x7f,0x90,0xb1,0x43,0x6d,0x19, +0xfe,0x08,0x93,0x1a,0x5c,0x63,0x40,0x3a,0x3d,0x2d,0xce,0xd0, +0x5e,0x64,0x56,0x9d,0x5b,0xb9,0x45,0xe3,0x93,0x36,0x36,0xaa, +0x69,0x2d,0x5b,0x74,0xb1,0x58,0x3d,0x4f,0x1d,0x7e,0x0a,0x91, +0xf1,0x2f,0x54,0x42,0xe5,0x17,0x0b,0xaf,0x0d,0x0a,0x27,0x59, +0x2d,0x1b,0x4b,0xb8,0x67,0xd8,0x4b,0x01,0x30,0x25,0x69,0x17, +0xff,0xee,0xeb,0xb1,0x69,0x0d,0xdf,0x3a,0xcc,0xde,0x3c,0xaf, +0x20,0xa3,0x2c,0x9f,0x4d,0x92,0x35,0x75,0x92,0x21,0xb6,0xd1, +0x02,0x99,0xe8,0x27,0xbb,0x20,0x28,0x34,0xc6,0x33,0x8c,0x95, +0x5c,0xbe,0xc5,0x84,0x22,0x37,0x4a,0x65,0x96,0xa6,0x36,0x8b, +0x78,0x10,0x47,0xf2,0x9b,0x5e,0x16,0xe5,0x38,0x27,0x21,0x41, +0x55,0x90,0xb6,0xe6,0xe8,0xa2,0xaa,0xc6,0x95,0x9e,0x65,0x03, +0xb9,0xe3,0x8d,0x54,0xa7,0x45,0xc4,0x3c,0xf8,0x81,0xbc,0xa2, +0xea,0x96,0x06,0x08,0x6e,0xbc,0xac,0x80,0x63,0xcd,0x84,0x28, +0x63,0x3d,0x5f,0x5f,0x0f,0x47,0x58,0xba,0x7a,0xde,0x9c,0x9c, +0x9e,0xe1,0xb4,0xc1,0x5e,0xc3,0x93,0x0b,0x0b,0xb1,0x76,0x42, +0x34,0x5f,0xcf,0x28,0x6e,0x4a,0x6b,0xcd,0xcd,0x2b,0x4b,0x22, +0xce,0x2e,0xb4,0x29,0x1f,0xe2,0x95,0xfe,0x09,0x4d,0x1c,0x7b, +0xce,0x81,0xaf,0x78,0xb4,0x11,0x8d,0xf2,0x84,0x2c,0xcd,0x60, +0xad,0x9f,0x1e,0xff,0x4a,0xe6,0xe9,0xb5,0xc4,0x57,0xc0,0x40, +0x8a,0x0b,0xf1,0x34,0xb4,0xbe,0x21,0x6f,0x3b,0x2e,0x4e,0x6d, +0xe6,0x5a,0x17,0xad,0x03,0x45,0xaf,0xc7,0x6a,0x87,0x60,0x86, +0x51,0x2e,0xde,0x79,0xca,0x38,0xf2,0x9d,0x5f,0xeb,0x44,0x81, +0xe1,0x9a,0x3b,0x97,0x5f,0xb7,0xd1,0x94,0x6c,0x6c,0xa8,0x8b, +0xdf,0x29,0x98,0x81,0x69,0x08,0x2f,0x59,0x2c,0x70,0x47,0x9d, +0xca,0xd0,0x76,0x9d,0x89,0x02,0xfb,0x78,0x56,0xa6,0xa6,0xb2, +0xf4,0xe5,0x21,0xa4,0x72,0x54,0x55,0x59,0x36,0xeb,0xaa,0x1d, +0xf3,0x5c,0xe3,0x4a,0x95,0x4f,0xc3,0xdf,0xa0,0x02,0x6c,0x3b, +0x9f,0xb6,0x94,0xc1,0x57,0xc3,0x45,0x56,0xce,0x7c,0xe5,0xad, +0x2b,0xe0,0x60,0xd2,0x20,0xe2,0xb4,0xab,0xd2,0x50,0xa6,0x42, +0x69,0xc7,0x3a,0x6e,0x37,0xb6,0xfd,0x71,0x6f,0x3b,0x9f,0xe2, +0x8e,0x05,0x24,0x6d,0xf6,0x29,0xac,0x96,0x48,0xe1,0xd3,0xc1, +0x9a,0x18,0xc8,0x3a,0x32,0xf3,0x80,0x0b,0xf0,0x72,0x63,0x21, +0xed,0xea,0x22,0x08,0xdc,0x32,0xca,0xf3,0x12,0x08,0x02,0xde, +0x97,0x87,0x0b,0xb3,0xad,0x8c,0xae,0x2c,0x84,0x35,0x76,0xbd, +0xc0,0x02,0xe5,0xdc,0x77,0x2e,0x16,0x9c,0x80,0x17,0x7a,0xdc, +0xf7,0xe0,0x96,0x51,0xad,0xf1,0xa3,0x51,0x81,0x0f,0x43,0xe6, +0x32,0xbb,0x4e,0x7e,0x15,0x8c,0x75,0x92,0xe7,0x4a,0xc9,0x70, +0x05,0x2e,0x27,0xc5,0xf8,0x56,0x5d,0x72,0x78,0xf4,0x40,0xd8, +0xc9,0x6f,0x40,0x9a,0xcb,0x6e,0x58,0x2a,0xc9,0x92,0xef,0x5f, +0x1e,0x24,0x4b,0x53,0x0e,0x7f,0x1f,0xfb,0x34,0x14,0x68,0x20, +0xf1,0x37,0xdc,0x34,0x84,0x90,0x46,0xa6,0x1c,0x6e,0x8b,0x45, +0x73,0xe4,0x01,0x76,0x5f,0xbe,0x79,0x79,0xf0,0x12,0x0a,0x26, +0x7c,0x8d,0x20,0xa9,0x66,0x2d,0x0d,0x0e,0x4d,0xb7,0x47,0x23, +0x05,0x94,0xcc,0x56,0x97,0x28,0xef,0xf2,0xfd,0x2e,0x99,0x5f, +0x0d,0x4e,0x4e,0x97,0x86,0x2f,0xc6,0x0f,0x83,0xdc,0xf3,0x0f, +0x9b,0x24,0x61,0xf0,0x7d,0x4f,0x00,0x0f,0x81,0xf6,0x44,0x81, +0x49,0x9a,0x5e,0x8e,0x1b,0x29,0x0d,0x73,0x53,0x27,0xa7,0x93, +0x24,0xe9,0x07,0x66,0x7e,0xa3,0xd7,0x89,0x4c,0x79,0x30,0xa2, +0xc5,0x42,0x86,0x0d,0x77,0xc2,0x4f,0xd6,0x54,0x1a,0xb7,0x5c, +0xbc,0xb5,0x42,0x45,0x88,0xba,0x0c,0x73,0x8d,0x37,0xaa,0x2c, +0x2c,0xd9,0x9f,0x9d,0xba,0x7d,0x8c,0xf1,0xdb,0x13,0x4a,0xd7, +0xbb,0x15,0xb7,0x7a,0x42,0xa2,0xf8,0xef,0xde,0x3a,0xfc,0xb0, +0xa5,0xf6,0xce,0x44,0x48,0x7a,0xb3,0xae,0x22,0x72,0x24,0xbf, +0x1c,0x62,0x48,0xf1,0x24,0x65,0xc1,0x94,0x21,0x36,0x36,0xe8, +0x37,0x08,0xdd,0x91,0x34,0x7e,0x59,0xf4,0x01,0xf8,0xf0,0x70, +0x90,0xc2,0x16,0x8c,0x00,0x9a,0x49,0xe3,0xf0,0xf0,0xc3,0xe2, +0x41,0x6a,0x8b,0xe3,0xba,0x06,0x33,0xcd,0x51,0xc5,0x62,0x14, +0x5e,0x6f,0xc1,0x75,0x16,0x3f,0xec,0x9a,0xd4,0x30,0xb2,0xcb, +0x83,0x6d,0x3f,0x6e,0x07,0x61,0x8b,0x68,0xe7,0x9f,0x5f,0x33, +0xf5,0xf5,0x5c,0xf5,0x9d,0xfe,0x63,0x8e,0x5c,0x5b,0x10,0xeb, +0x96,0xa2,0xf1,0xad,0x0a,0x07,0xd3,0xe3,0xfc,0xbe,0x4d,0x18, +0x64,0x7c,0x99,0x48,0xe6,0x88,0x4f,0x57,0x4b,0x8d,0xb5,0x1e, +0xc7,0xed,0x2e,0xdb,0x93,0x70,0xdc,0xa3,0x5e,0xc3,0x77,0x5c, +0x5e,0xbb,0x8b,0xaf,0x7c,0xab,0x6e,0x49,0x68,0x3e,0x5b,0x9b, +0x6f,0x41,0x93,0x9c,0x61,0x14,0x73,0x87,0xb7,0x99,0x3c,0xe8, +0xa0,0x03,0xd4,0xdb,0x10,0x35,0xb6,0x7a,0xe9,0x2d,0xc8,0x5c, +0x53,0xd1,0x81,0xb9,0xb9,0x80,0xa5,0xa5,0x98,0x9c,0xed,0x5f, +0x0c,0xcb,0xf3,0x31,0x70,0xb1,0x61,0xe1,0xb4,0x0d,0xc9,0x16, +0x6c,0xed,0x30,0x41,0xed,0xec,0xf0,0x35,0x0b,0xcb,0x34,0x31, +0x6d,0x59,0xdf,0x95,0x5b,0xcd,0x07,0x5b,0x14,0x88,0xbd,0x26, +0xeb,0xb0,0xd5,0xe8,0xe5,0x87,0x9f,0x9a,0x8d,0x07,0x8b,0xc3, +0x5e,0x9a,0x02,0x5c,0x0b,0x00,0x79,0x76,0xb4,0xea,0xe0,0x0f, +0x0f,0x5b,0x00,0xb4,0x85,0x50,0x76,0x18,0x5c,0xad,0xde,0xd9, +0x86,0x23,0x84,0x44,0xbc,0xc5,0x42,0x26,0x9f,0x48,0x61,0x79, +0xf0,0x46,0xe3,0x2b,0x48,0x51,0xdb,0x5e,0xd2,0x06,0x9a,0x88, +0xed,0x2e,0x07,0x5e,0xfb,0xc6,0x3f,0x2d,0xa7,0xd9,0x5b,0xa1, +0x9a,0x8a,0x14,0x52,0x6e,0xe3,0x2a,0xc1,0x06,0x1b,0x57,0xd6, +0xa8,0x1d,0xe4,0x47,0xaf,0x6b,0x61,0xce,0x8e,0x17,0xc9,0xe5, +0x51,0x29,0x08,0x97,0xe6,0xc6,0xb1,0x9c,0x8c,0x8b,0xa3,0xdd, +0x24,0x63,0xf0,0x3b,0xe3,0x07,0xc4,0x9d,0x60,0x06,0xfc,0x5e, +0xb0,0xb5,0xbd,0xcb,0x41,0x4e,0x26,0x63,0x07,0x1f,0xf1,0xe8, +0xc0,0xaa,0x21,0x60,0x4d,0x9f,0x94,0x05,0xa9,0x3d,0x31,0x95, +0xee,0xec,0xbe,0xf6,0x89,0x7b,0x34,0x82,0x80,0xc8,0xfc,0x66, +0x3a,0x2b,0xce,0x0a,0x5a,0x62,0xe6,0xe7,0x19,0xa1,0x99,0x8d, +0xcf,0xc6,0x7f,0x5c,0xd1,0xf3,0x92,0x8d,0xdb,0x01,0xb0,0xf5, +0x51,0xfc,0x90,0xc9,0x97,0x61,0x6e,0x8e,0x7f,0x7c,0x47,0x37, +0xfa,0x8d,0xb4,0x35,0x48,0x91,0x05,0x1c,0x1e,0xe2,0x68,0x57, +0x59,0x87,0x5b,0x69,0x2f,0xc3,0x08,0xbe,0x69,0xa3,0x7f,0xd8, +0x5b,0x1c,0x3e,0x1a,0xa4,0x3d,0x00,0x54,0xb3,0xbf,0xc4,0x31, +0x49,0x06,0x10,0xf6,0x72,0x83,0x37,0x8e,0xa0,0xfb,0x61,0xde, +0x1c,0x4c,0x2c,0x5c,0xd9,0x5c,0x04,0x2d,0x67,0xf2,0xe4,0x51, +0x7d,0x6e,0xd7,0xfb,0xf3,0x93,0x13,0x28,0x0e,0xff,0xed,0xab, +0x41,0xc5,0xef,0xba,0x7d,0xc4,0x1b,0x1d,0x48,0x46,0x4e,0x7f, +0xc9,0xb8,0x3b,0x49,0x9a,0x0d,0x9b,0x8d,0xc6,0xbf,0x94,0x93, +0xc2,0xd4,0xec,0x65,0x3a,0x87,0x92,0x33,0x98,0x2f,0xcd,0x06, +0x12,0x04,0x1b,0x17,0xf4,0x3d,0xd9,0x4b,0x00,0x47,0xa3,0xff, +0xcb,0x16,0x06,0x28,0x86,0xbc,0x24,0x55,0x65,0x24,0xcc,0x87, +0x6e,0xbf,0xc3,0xad,0x07,0xd8,0x32,0xbe,0x05,0xb1,0x8b,0x5a, +0xdc,0x1f,0x79,0xc0,0xe5,0x91,0x63,0xcf,0xcf,0x81,0xa9,0x60, +0xa0,0x15,0xd7,0xad,0xbd,0xa4,0x48,0x90,0x08,0x2c,0xe6,0x95, +0x25,0xb4,0x98,0x90,0x32,0x0c,0x5d,0xa6,0x6a,0x59,0x0b,0x3b, +0x1b,0x21,0x68,0x3f,0xc1,0xb0,0x7d,0x4c,0x1d,0x38,0x0d,0xca, +0x6c,0x8c,0x5e,0xa3,0xdf,0x4d,0xf6,0xc7,0xc3,0xd9,0xc9,0x39, +0x0b,0x03,0x08,0x47,0x2b,0x2b,0xae,0x2b,0xe1,0x08,0x64,0x6c, +0xa9,0xac,0xbb,0x2c,0x79,0x71,0xf0,0x06,0x1c,0x5e,0x39,0xf1, +0x25,0xcc,0xa3,0x5a,0xbc,0x43,0x9b,0x1e,0xbd,0x8a,0xa6,0xa9, +0x6d,0x82,0xcc,0x34,0xe3,0x2f,0xdd,0x4a,0xba,0x42,0xa1,0x46, +0xa9,0x29,0xe5,0xd4,0x83,0xa9,0x8c,0x5c,0x45,0x9b,0x2e,0x61, +0x69,0xb4,0x32,0x3d,0x36,0x8c,0xe8,0x0a,0xa7,0x20,0x27,0xcf, +0x3e,0x15,0x3a,0x6c,0x95,0x88,0x9b,0x37,0x0a,0x98,0x5a,0x91, +0x0c,0x8b,0xdc,0x36,0x50,0xe1,0x78,0x00,0x62,0x08,0x9e,0xd9, +0xcd,0xa6,0x53,0xf6,0xd1,0x8d,0xfa,0x63,0x3c,0xe5,0xa7,0x13, +0x6e,0x83,0x19,0x38,0x9b,0x79,0xd3,0xcb,0xaa,0xe5,0x44,0x82, +0xe3,0x4f,0x29,0xd2,0xef,0xca,0xd0,0x01,0x51,0x50,0x00,0x57, +0x81,0x71,0xe8,0x8d,0x46,0x0e,0x5b,0x01,0xe3,0x10,0x9b,0xfa, +0x18,0x02,0x14,0x23,0xc0,0xb1,0x13,0x56,0x38,0x11,0xb7,0x21, +0xfe,0x00,0xcc,0x06,0x70,0xca,0x00,0xc1,0xbe,0x64,0x5e,0x40, +0xb2,0x12,0x3e,0x2a,0x5d,0x15,0x6a,0x9a,0x38,0x83,0x07,0x6b, +0x6a,0x4f,0x96,0x71,0x0e,0x92,0xf1,0x5e,0xae,0xb3,0x39,0xcc, +0x98,0xb8,0x3b,0x5d,0xbf,0x0c,0xa0,0xbd,0x3f,0x13,0xb2,0x01, +0xca,0x2f,0x95,0xd9,0xcf,0x4e,0xd1,0x6d,0x36,0x0b,0x47,0x8c, +0x5c,0xdc,0x40,0x8f,0x48,0x28,0x31,0x24,0xc8,0x9b,0x27,0x67, +0xb8,0xf4,0x01,0x47,0x9d,0x9e,0xae,0x5d,0xf6,0x8b,0x41,0x6f, +0x34,0xae,0xa8,0xbb,0x31,0x3d,0xcd,0xf0,0x6f,0x97,0x14,0x0a, +0xbf,0xe3,0xc1,0x56,0xc3,0xaf,0x1c,0xad,0x89,0xbd,0x35,0xe2, +0x46,0x09,0x67,0x7a,0xa6,0x7c,0x7d,0x05,0xdb,0xc6,0x31,0x6b, +0x20,0xad,0xbb,0xfb,0x3f,0xe6,0xec,0x06,0x59,0x52,0xc9,0xe0, +0x32,0xe7,0x2e,0x69,0x89,0xd7,0x76,0x72,0x3d,0xc7,0x16,0xb2, +0xec,0x5b,0xac,0xf5,0x80,0x01,0x80,0x7b,0x61,0xa2,0x7d,0xdd, +0xd8,0x90,0xdb,0x6a,0x04,0x83,0x23,0x59,0x3a,0x95,0x80,0xfc, +0x3b,0x88,0x31,0xeb,0x04,0x12,0xce,0x53,0x48,0x85,0x66,0x3c, +0x19,0xbf,0xa7,0x54,0x36,0x42,0xb6,0x2b,0xa7,0x60,0x24,0x6c, +0xb4,0xff,0x55,0x99,0xc1,0x00,0x4a,0x8d,0x1f,0x63,0x2d,0x8c, +0x92,0x3d,0x1c,0xc1,0x2a,0x82,0xfe,0xdd,0x09,0xe8,0x27,0x6a, +0x81,0x84,0x28,0xc2,0xbd,0x34,0xe2,0x5d,0x2c,0xe8,0x0d,0x1a, +0x53,0x55,0x4f,0xe1,0x47,0x6f,0xc3,0xc4,0x4e,0xe7,0xf7,0xd1, +0x02,0x32,0x21,0xd7,0x56,0x8c,0xd3,0x84,0x4d,0x47,0x84,0x1b, +0x6e,0x3f,0xf9,0x64,0x3b,0xe1,0x29,0x3c,0xe0,0x3e,0x75,0x7e, +0x23,0xca,0x4a,0x7b,0x6e,0x76,0x50,0xb3,0xe9,0x03,0xd1,0x18, +0xd6,0xa8,0xa6,0x49,0xd3,0x56,0x49,0x4d,0xd9,0xd0,0x89,0x8e, +0x57,0xa7,0x99,0x47,0x0b,0x92,0x4a,0x8c,0xa9,0xd2,0x6d,0x11, +0x11,0xa9,0x4a,0x90,0x6a,0xd2,0x54,0x61,0xa7,0x7d,0x3c,0xb3, +0x19,0x74,0xa7,0x10,0x29,0x8f,0x52,0xeb,0xfd,0x31,0xf4,0xdc, +0x60,0xb4,0x1b,0x06,0xb1,0xf6,0xf3,0x3b,0x4b,0xa1,0x90,0xf6, +0xe5,0x6e,0x7b,0x6b,0x85,0x31,0x86,0xae,0x15,0xc6,0xbc,0x25, +0xd3,0x20,0x77,0x8d,0xcc,0x82,0x55,0xcf,0xb1,0x17,0xba,0x89, +0xc9,0x89,0x99,0x4f,0x2c,0x26,0xbf,0x4f,0x3f,0x7a,0x3b,0x26, +0xe3,0x05,0x70,0x77,0x8c,0x43,0x54,0xd9,0xb7,0xb4,0xa7,0xf6, +0xf9,0x2e,0x11,0x23,0xc3,0xc6,0xa7,0x3e,0x41,0xa7,0xe0,0xcd, +0xde,0x18,0x31,0x7a,0xb1,0x08,0x70,0xff,0xc4,0x3b,0x99,0x9a, +0x0a,0x24,0x47,0x97,0x54,0xcf,0x61,0x4d,0xba,0x89,0xf3,0x28, +0x80,0x97,0x0e,0xce,0xa1,0x71,0xd7,0x7c,0xb6,0x54,0x49,0xa8, +0x2f,0x28,0x56,0xc3,0x41,0x7d,0xed,0xae,0xbe,0xdc,0x87,0xec, +0xd0,0xd9,0xab,0x42,0x77,0xac,0x52,0x36,0xc6,0xc7,0x30,0xa8, +0xfa,0x02,0x91,0xb3,0x8e,0xc6,0xd4,0x59,0xb1,0xb5,0x1e,0x54, +0xda,0x9f,0xc6,0xe7,0xf0,0xe2,0xa2,0x61,0xaf,0x67,0x55,0xea, +0x94,0x8c,0x1b,0xe2,0x8a,0x11,0x13,0x94,0x4f,0xf7,0x2d,0x40, +0x43,0x55,0xc6,0x20,0xb1,0x24,0xcd,0xa3,0x02,0x96,0x74,0x07, +0x4f,0x12,0x4e,0xc2,0xaa,0x1c,0xcd,0x4b,0xb4,0x1f,0x78,0xd1, +0x29,0xd5,0x53,0x76,0x0f,0xf4,0xa4,0x2b,0x53,0xc8,0x45,0x65, +0x84,0x4e,0x94,0x94,0x64,0xee,0xd7,0x81,0x1b,0x09,0x53,0x8e, +0x1f,0xc3,0x36,0xec,0xd1,0xec,0x63,0xb1,0x46,0xe9,0xff,0x78, +0xc5,0x5d,0x97,0x78,0x9b,0xb2,0x30,0xe5,0x35,0xcb,0x73,0x85, +0xad,0x88,0x6c,0x03,0x85,0xd8,0x44,0xde,0x6e,0x4e,0x58,0x66, +0x93,0xc3,0x9f,0xcc,0x73,0x85,0x2a,0x1b,0x72,0x32,0x8d,0xe7, +0x5e,0x72,0x3c,0x63,0x1d,0x62,0xbb,0x25,0x8b,0x2b,0xb3,0xc9, +0x8b,0x85,0x88,0x67,0xb8,0x8a,0x53,0xdc,0x71,0x55,0xb5,0xa4, +0x3a,0x0a,0x6e,0x96,0x1a,0xff,0xcd,0x52,0x5f,0x40,0xd6,0x1c, +0x9a,0x57,0x74,0x4b,0xd3,0x8d,0x52,0xe5,0xf9,0x9d,0x88,0x6d, +0x46,0x01,0x26,0xb1,0xda,0xc6,0x3b,0xcb,0xf4,0xb6,0xb9,0xe4, +0x0d,0xbd,0x29,0x88,0x81,0x92,0xf3,0xca,0x54,0x14,0x7c,0xe2, +0xd4,0xeb,0x8c,0x2f,0x45,0xd0,0x45,0x47,0xb6,0x1b,0xa2,0x14, +0x6a,0xee,0x5c,0x5e,0xac,0x2f,0x70,0xde,0xcc,0xb4,0x1e,0xa5, +0x5b,0x64,0x26,0x9c,0x6b,0xd8,0x7e,0x07,0xa9,0xf3,0x98,0xd5, +0xa6,0x94,0x34,0xc1,0x71,0x95,0x7b,0xce,0x98,0x1f,0x85,0x77, +0x71,0xcd,0xa1,0x0a,0x71,0xa4,0x5e,0x2a,0xc6,0x91,0x46,0x13, +0x3e,0x8d,0xaa,0xae,0xb2,0xd8,0x2f,0x37,0xdc,0xe4,0x99,0x8c, +0x2b,0x16,0x0d,0x74,0xc4,0x94,0xa0,0x58,0x3b,0x98,0x52,0xe8, +0x56,0xed,0x79,0xf9,0x79,0x72,0xd2,0x50,0x12,0x0e,0xcc,0x6f, +0xdb,0xd8,0x1a,0x96,0x4c,0x74,0xec,0x00,0x8a,0xa6,0x9e,0x2e, +0x2d,0x74,0x2d,0x07,0x95,0x3d,0x2d,0x7d,0xa1,0xda,0xd9,0x7a, +0x09,0x3a,0xb0,0xe4,0x23,0x1b,0xb5,0xa5,0x3a,0xbe,0x9f,0x9c, +0xa1,0x99,0xac,0x55,0xa1,0x32,0xfa,0x95,0x11,0xb1,0xbc,0xf7, +0x1b,0x73,0x9b,0x17,0xc1,0x38,0x30,0x56,0x18,0x6e,0xe1,0xcb, +0xa3,0x64,0xad,0x8a,0xbb,0x10,0xf9,0x9f,0x62,0x23,0x35,0xf7, +0x65,0x2a,0x3e,0x0e,0x7e,0xe2,0xff,0xf1,0x70,0x54,0xb1,0xc4, +0x2b,0xec,0x4c,0x78,0x76,0x0d,0xf7,0x54,0xef,0xc2,0xd9,0xfd, +0x22,0xc6,0xef,0x20,0xfe,0x46,0xeb,0x0a,0xee,0x02,0xdc,0xd2, +0x72,0x4b,0xec,0x28,0x71,0xe2,0x7a,0x9f,0xa8,0x51,0xdc,0x78, +0x86,0x1d,0xa7,0xd2,0x5d,0x0d,0xb5,0x76,0xad,0xf4,0xf4,0xea, +0xc2,0x3d,0xb1,0x97,0x7b,0xa8,0x2e,0xf7,0x71,0x9f,0x08,0x95, +0x0a,0xf7,0x64,0xdb,0x2b,0x16,0xf1,0xd1,0x06,0x5b,0x77,0x86, +0x6d,0x8d,0xf2,0xfe,0x01,0xab,0x5a,0xda,0x1d,0x99,0x6d,0xad, +0x52,0x63,0x23,0x1c,0x1c,0x9b,0xde,0x81,0xe6,0x6a,0xa8,0x37, +0xaa,0x98,0xbc,0xad,0x79,0x25,0xeb,0x79,0x19,0x8d,0xac,0x7e, +0x1d,0x0c,0x1e,0xb4,0x3a,0xb2,0xa4,0x69,0x25,0x40,0xaa,0x8e, +0xd3,0xa9,0xad,0x2a,0xab,0x05,0x7c,0x03,0x7e,0x41,0xd9,0x25, +0x76,0x45,0x14,0x5e,0xab,0x36,0x0e,0x16,0x85,0x44,0x8f,0x82, +0x60,0x45,0x80,0xbf,0x07,0xb1,0xb0,0x4c,0x14,0x5e,0x6b,0x29, +0x57,0xcd,0x2b,0x43,0x69,0x79,0x57,0xac,0xb9,0x95,0x31,0xee, +0xba,0xda,0x01,0x50,0x9d,0x98,0xc2,0x33,0x2b,0x35,0x76,0x86, +0xa1,0xf1,0x22,0xf7,0x6c,0x7c,0x00,0xc2,0x9b,0x42,0x7f,0x50, +0x42,0xaf,0x2b,0x8e,0x4f,0x56,0x1d,0xb6,0x38,0x51,0x2f,0xaf, +0x28,0xc6,0xef,0x38,0x60,0xb1,0xe2,0xbf,0xca,0xe3,0x84,0x6e, +0x64,0xb6,0x68,0x82,0x53,0x1f,0xfb,0x32,0x9f,0xfe,0xbd,0x9c, +0x4e,0x24,0xac,0x26,0x8b,0x31,0x64,0x9a,0x97,0xd7,0x58,0xea, +0x61,0xae,0x37,0xd3,0xbb,0xb2,0x2b,0x84,0xb4,0x5f,0x62,0x42, +0x9d,0x28,0xc6,0x24,0x55,0x08,0x2b,0xf0,0x56,0xdb,0x52,0xb7, +0xea,0x90,0xeb,0x7a,0x55,0x76,0xe4,0x83,0xe0,0xf2,0x2a,0x83, +0x61,0x7b,0xd4,0x6a,0x93,0x86,0xaf,0x2d,0xd9,0xee,0xb0,0xca, +0xe7,0x7a,0x7e,0x57,0x90,0xc7,0x13,0x35,0xc0,0xbd,0x9f,0xdb, +0xc1,0x97,0x2a,0xa3,0x57,0xda,0x49,0x56,0xdb,0xf2,0xc5,0x74, +0xfa,0xb1,0x40,0x5f,0xfc,0x55,0xa3,0xc7,0x13,0x9f,0x15,0xea, +0xab,0x40,0xf2,0xfd,0x54,0x86,0x01,0x72,0x95,0xba,0x49,0x72, +0xb5,0x22,0x1a,0x65,0x19,0xe1,0xdc,0xe8,0xfb,0x85,0xf7,0x8f, +0x02,0x27,0x04,0xc8,0xf0,0x26,0xa3,0x10,0x8a,0xd5,0x6a,0xdd, +0xe7,0x4b,0x7e,0xaa,0x0d,0xc9,0x85,0x32,0x3e,0xa7,0x05,0x71, +0xd6,0xd6,0xd9,0x92,0x0c,0x42,0xa0,0x1a,0x21,0x40,0x2a,0x30, +0x72,0xf7,0x28,0x40,0x65,0xe2,0xa1,0x1b,0x84,0x71,0x5f,0x67, +0x25,0xcc,0x0b,0x5d,0x02,0x97,0xcd,0xda,0xca,0xc5,0x62,0x40, +0xbe,0x0a,0x8f,0xc4,0xb2,0xea,0x49,0x53,0x05,0x5d,0xda,0x0b, +0xe8,0xa9,0xd6,0x97,0x89,0xf4,0x17,0x80,0xe1,0x9a,0x22,0x15, +0xa5,0x37,0xae,0xc6,0x4e,0xd4,0xc2,0x79,0x85,0xd0,0xfa,0x76, +0x09,0x50,0xc3,0x37,0x87,0x58,0xdc,0x67,0xeb,0xfe,0xa8,0x03, +0x49,0x7b,0x75,0xa4,0x66,0xe1,0xf7,0xd5,0x15,0xd4,0x6d,0xc6, +0x9c,0x8e,0x1c,0x96,0x87,0xa3,0x45,0xd0,0xa0,0xf9,0x93,0x6a, +0x23,0xb2,0x86,0xba,0x9e,0xbc,0x44,0x75,0x1f,0x7c,0x7f,0xdd, +0x11,0xb1,0x23,0x5a,0xb1,0x29,0x97,0xe6,0xc4,0x58,0xdb,0x52, +0xc3,0xd1,0xe8,0x47,0x98,0x8e,0xaf,0x26,0xf1,0xad,0x4c,0xe1, +0xee,0xb7,0x34,0x6d,0x95,0xe8,0xf5,0x3c,0x2c,0x65,0xe3,0x66, +0x84,0xbd,0x14,0xc2,0xf0,0x86,0x37,0x4a,0xd3,0xd3,0xa2,0x16, +0xc0,0x76,0xa5,0x6d,0x8c,0x70,0xe3,0xca,0xd7,0x4c,0xd1,0xc6, +0xa5,0x91,0xfa,0xe6,0x0a,0x05,0x7d,0xbe,0x1e,0x80,0x13,0x37, +0xb5,0xd8,0x96,0x35,0xcc,0x61,0x1f,0xde,0xd8,0x28,0xda,0x31, +0x0b,0xed,0x87,0x93,0xa9,0xb0,0xd2,0x34,0x9a,0x1b,0xb8,0x5c, +0xb9,0x5b,0x67,0x03,0xc4,0x59,0x60,0xbe,0x82,0xa8,0x0d,0x5b, +0x7a,0x6e,0x85,0x9a,0x4d,0x2f,0x71,0x89,0xb0,0x9a,0x24,0x36, +0x37,0x35,0x57,0xd7,0x21,0x56,0x1b,0xde,0x29,0xc0,0x98,0x87, +0xab,0x8c,0xf3,0xbe,0x27,0x96,0x0b,0x21,0x55,0x32,0x6e,0x03, +0x04,0x78,0x61,0xe0,0xdf,0x76,0x26,0x78,0x39,0xfc,0x38,0x46, +0x6f,0x46,0xa5,0x8b,0xbf,0x6f,0xa6,0xc7,0xbf,0xe2,0xe0,0x65, +0x5f,0x69,0x25,0xc6,0xc5,0xe5,0x1d,0xaa,0x49,0xe4,0x12,0x7e, +0xde,0xee,0x16,0xcf,0xd8,0xaf,0x99,0xbd,0x77,0x86,0x3e,0x28, +0x8e,0x7f,0xed,0x53,0x62,0xbf,0x18,0xe0,0xb9,0xbb,0x6d,0x39, +0x48,0xf7,0xa2,0x39,0xca,0xb7,0x3f,0xd2,0x35,0xab,0x06,0x3e, +0x1a,0x10,0x2f,0x46,0x17,0x21,0x83,0xc6,0x5c,0xd2,0xb0,0xe2, +0x05,0xf3,0xf9,0xf0,0xcc,0xc0,0x3f,0x72,0xd3,0x85,0x3b,0x20, +0x60,0xa4,0xc3,0xd9,0x67,0x0e,0xeb,0xca,0xc9,0xae,0x39,0x7d, +0xcc,0x43,0xc9,0x91,0xd8,0x0a,0x2f,0xad,0x4f,0x70,0x49,0x1e, +0xd0,0x1d,0xd8,0x6e,0x39,0x1f,0x9e,0x7c,0x6c,0xe1,0xf0,0x45, +0xbf,0x75,0xc5,0x04,0xe4,0xb6,0xb1,0x03,0xd5,0x99,0x83,0x6e, +0x4a,0x34,0xbd,0x9c,0x8c,0x90,0x22,0xf4,0x1e,0xa3,0x72,0xd3, +0xae,0x3d,0x24,0xe3,0xfc,0x4a,0x6d,0x1b,0x1b,0x1a,0x1c,0xcf, +0xba,0xcf,0xf8,0xe6,0x65,0x84,0xd4,0x11,0x4d,0x5f,0x98,0xff, +0x3e,0x2d,0x46,0x15,0x5c,0x8b,0xc5,0xfa,0x3a,0x65,0x33,0xd0, +0x62,0xc1,0xa8,0x69,0x4d,0xb3,0xe5,0x49,0x6a,0x21,0xef,0xa4, +0x6c,0xdd,0x5d,0xce,0x9d,0xb2,0xf6,0xf9,0xc1,0xc1,0x7b,0x71, +0x37,0x10,0x9f,0xd0,0xd3,0x3a,0x31,0x9a,0x5e,0x1f,0x5f,0x8c, +0xff,0xf7,0xf5,0x74,0x4e,0x76,0x00,0xe8,0x05,0x89,0xee,0x84, +0xe9,0x94,0xeb,0xc9,0x6f,0xfe,0x4d,0xcf,0x9a,0x4a,0x61,0xa0, +0x2e,0x2e,0x8d,0x06,0x97,0xbf,0xe9,0x57,0x10,0x39,0x89,0x54, +0x61,0xc1,0x7c,0x00,0xf2,0x72,0x32,0x2f,0xe6,0xc8,0xb8,0x9c, +0x3a,0x56,0x46,0x09,0x07,0xa6,0xd8,0xd8,0x08,0x5e,0xed,0x87, +0x8b,0x3f,0x69,0x1e,0x1f,0x91,0x7a,0x4b,0x5a,0x59,0x0d,0x26, +0x3b,0xd6,0xaf,0xa6,0x25,0x8e,0xe6,0xfb,0x8c,0x26,0x1a,0x36, +0x08,0x2f,0xfd,0x29,0x87,0x7e,0x5d,0x48,0xda,0xc9,0xdb,0xd2, +0xcd,0x7d,0x78,0x83,0x85,0x4a,0xca,0x60,0xde,0xe6,0x26,0x1d, +0x1d,0x31,0x14,0xdf,0x74,0x2d,0x62,0x14,0xc5,0x4e,0x0e,0xf9, +0xdd,0x02,0x60,0xed,0xb7,0x81,0x5c,0xec,0x3f,0x14,0x55,0x60, +0x8c,0x9e,0xfc,0xda,0xab,0xc2,0x58,0x8e,0x17,0x26,0x1b,0xdd, +0x04,0xbe,0x97,0xb5,0x5c,0x3c,0x4c,0xf1,0xae,0x3c,0x9d,0xee, +0xe0,0x74,0xa3,0xe9,0xeb,0x87,0x63,0x55,0x75,0xb1,0xc6,0xb5, +0x84,0xe4,0x0d,0x96,0x5d,0x2a,0x4c,0xb3,0x8e,0x2a,0x40,0x0e, +0x1d,0xce,0x1f,0x8a,0x7a,0x04,0x73,0xa0,0x76,0x02,0xa5,0x58, +0x9c,0x08,0xa8,0xb7,0x25,0x69,0x3d,0x4a,0x9f,0x1d,0xa2,0xff, +0x61,0xb4,0xbf,0x28,0x1f,0x25,0x4d,0x5d,0xba,0x99,0xf4,0x7f, +0xd9,0x19,0x3c,0xda,0x01,0xf6,0x53,0x68,0x15,0xd9,0xf0,0x42, +0x3c,0x38,0xbb,0x03,0x4f,0xd4,0x92,0xce,0xf9,0xec,0x80,0x2b, +0x79,0xf1,0x6e,0x6f,0xef,0xe5,0xdb,0x03,0x3b,0xe4,0x93,0xe0, +0xa0,0xff,0xc5,0xee,0xf3,0x83,0xe7,0x41,0x96,0x1b,0x66,0xf4, +0x8d,0xbe,0xf5,0xe9,0xb5,0x11,0x8d,0x4d,0xaa,0x1a,0x5d,0x7b, +0xb2,0x89,0xed,0x6a,0xf6,0x60,0xef,0x92,0xa3,0x12,0x8f,0x9a, +0x80,0x3a,0xea,0xdd,0x69,0x23,0x79,0xb6,0xbe,0xb9,0x99,0xa4, +0x3d,0x71,0x1c,0x17,0xe6,0x6d,0x6e,0xee,0x00,0x41,0xf4,0x4a, +0x83,0xab,0xe1,0x88,0x99,0x5e,0xb2,0x8a,0x24,0x4a,0x20,0x76, +0x2a,0xe7,0xd2,0x28,0x2c,0x3e,0x91,0x08,0x61,0x68,0x63,0x6c, +0x1b,0xdf,0xe7,0xb2,0x2b,0x82,0xc7,0xa9,0x91,0xee,0xc4,0x2b, +0x13,0xdf,0xbd,0xfc,0xfe,0xf5,0xdb,0xef,0x8f,0x5e,0xbe,0xdd, +0x3d,0x3a,0x78,0xfe,0xfd,0x4b,0x69,0x1a,0xb6,0xae,0xc2,0xe2, +0x40,0x29,0x6b,0x6a,0x09,0x17,0xab,0x16,0x05,0xd8,0xfa,0x30, +0x31,0x56,0x0f,0x5c,0x57,0x27,0xfb,0xce,0x68,0x0f,0xdc,0x15, +0x5c,0x97,0x60,0xbb,0x24,0xc4,0xa6,0x9b,0xb5,0x4a,0xa8,0x02, +0x54,0x24,0x3a,0x6d,0xb2,0xa2,0x71,0xff,0xe0,0xf9,0xfb,0x83, +0x7f,0x23,0x95,0x31,0x3e,0x13,0xac,0x58,0x9a,0x52,0x7e,0x84, +0x69,0x53,0xd3,0xc1,0xcf,0x60,0x31,0xfd,0x5f,0x7c,0x26,0x06, +0xc3,0xb6,0xcd,0x2e,0x22,0x7a,0x08,0x93,0x45,0x14,0xb5,0xa5, +0x27,0x69,0x12,0x3a,0xc8,0x24,0x89,0xe1,0x24,0x22,0xdc,0x97, +0x0f,0x62,0xbc,0x45,0x41,0xb8,0x49,0xc4,0xb7,0x46,0xd8,0x25, +0xec,0xf8,0xe6,0xc5,0x9f,0xda,0x08,0xdb,0x5e,0x32,0x3b,0x38, +0x07,0x41,0x42,0xb2,0x67,0x6b,0x9f,0x86,0xe5,0x1a,0x70,0xdd, +0xe3,0x0b,0x32,0xb6,0x26,0x98,0xb5,0x39,0x40,0x9c,0x4e,0x31, +0x5e,0x1a,0x50,0xb6,0x46,0xcb,0x30,0x3a,0xa8,0xa1,0xcf,0x43, +0x23,0xec,0x84,0x46,0x25,0x0b,0xa6,0xd4,0x30,0x4b,0x3d,0x87, +0x52,0x7d,0x65,0xa7,0x66,0x45,0x70,0xe1,0x32,0x8b,0xd1,0x68, +0x3c,0xf9,0x71,0x36,0x86,0x66,0x9d,0x8c,0x67,0x3f,0x1c,0xec, +0xbd,0x11,0x2b,0x74,0x67,0xa8,0xf2,0x0c,0x0d,0x54,0x36,0x2e, +0xe6,0x5d,0x9c,0xde,0x21,0xf8,0x01,0x9d,0x02,0xfb,0x44,0x6c, +0x0c,0x51,0xc8,0xe0,0xc2,0x14,0x5f,0x58,0x5b,0x41,0x43,0x54, +0xdf,0x06,0xd5,0x37,0xbc,0xbc,0xea,0x2a,0x56,0xf3,0xf6,0xdd, +0xdb,0xa3,0xe7,0x6f,0x7e,0xfc,0xe1,0xf9,0xdb,0x9f,0xf6,0x5e, +0xbe,0x7f,0xfd,0xa2,0xb2,0x06,0x07,0x28,0x93,0x8d,0xff,0x90, +0x18,0x23,0xd4,0x69,0x2f,0xa0,0x6e,0x32,0x8b,0x6b,0x26,0xdd, +0x44,0xdb,0xe0,0xa8,0x4f,0xf3,0x89,0x3b,0x94,0x78,0x86,0x89, +0xa1,0xd0,0xb5,0x2f,0xfd,0xf5,0x61,0x56,0xa0,0x26,0xf1,0xf8, +0xfa,0x54,0x96,0xc0,0xb3,0x09,0x88,0xcd,0xe4,0x07,0xe1,0x7a, +0xee,0x56,0x3f,0x34,0x14,0x46,0x18,0x03,0xff,0x48,0xbc,0x70, +0xf1,0x7e,0x69,0xcd,0xf5,0x42,0x2a,0x2e,0xa9,0x7a,0xe1,0xbd, +0x81,0x84,0xfa,0x35,0x14,0xa3,0xcc,0x53,0x5d,0xd5,0xc5,0x02, +0x72,0x69,0x17,0xc7,0xa4,0x10,0x2c,0x3f,0x2f,0x16,0x9d,0x75, +0xec,0xd1,0x50,0x14,0x42,0x13,0x3b,0x20,0x96,0xe6,0x10,0x52, +0xcd,0xd8,0x23,0xe5,0x11,0x87,0x4d,0xaf,0x3d,0x27,0xa6,0x6d, +0x1c,0x9e,0x83,0x54,0xe9,0x24,0x4f,0x25,0xb6,0xce,0xe7,0x24, +0x99,0x5c,0xb0,0x51,0xdf,0xf5,0xac,0x50,0xef,0xc0,0xcb,0xd7, +0x81,0x2b,0xaf,0x73,0x37,0x31,0x9f,0xf9,0xe9,0xfd,0x6b,0xcb, +0x61,0x2c,0x81,0x6b,0x42,0x20,0xf9,0xd4,0xc0,0x87,0x87,0x79, +0xf2,0x90,0x9f,0x6a,0x07,0x95,0x00,0x01,0x0c,0x4a,0x3d,0xf8, +0x4c,0xad,0xda,0x4b,0xb6,0x76,0x92,0x2c,0x81,0x15,0x01,0xb6, +0x09,0x20,0x0b,0x04,0xcd,0x7f,0x6b,0x93,0xdf,0xaf,0x19,0xb7, +0x74,0x3b,0x52,0x12,0x56,0x85,0xa2,0x52,0x9e,0xdb,0x3e,0x6b, +0xb8,0x81,0x02,0x34,0x10,0x33,0x51,0xc6,0xc6,0xc2,0xe6,0xa4, +0xb2,0x9a,0xcf,0x63,0x88,0x54,0xb4,0x85,0x11,0x6f,0xa9,0xbb, +0x1d,0x6b,0x21,0x30,0x54,0x85,0x7d,0xf6,0xe2,0x0a,0x31,0x0f, +0xea,0x47,0x18,0x9f,0xea,0x7c,0xad,0xb2,0xbb,0xa8,0x1f,0xf8, +0x29,0x0d,0x6b,0x77,0x3c,0x65,0x62,0x8e,0x9e,0x6f,0xfd,0xf2, +0x8c,0x43,0x24,0x7c,0xca,0x36,0x07,0xcd,0xb4,0xc1,0x71,0x10, +0xe4,0x95,0x5e,0x1e,0xe5,0x08,0xd0,0xcb,0xd0,0xb6,0xae,0xff, +0x4b,0x32,0x78,0x94,0x50,0xc8,0x85,0x87,0xfd,0x5f,0x1e,0x0e, +0x1e,0x3d,0x4c,0x17,0x20,0xb3,0x1c,0x96,0x50,0x34,0xed,0xa5, +0x8f,0x38,0xde,0xf3,0x16,0xc5,0x7d,0xde,0xd9,0x32,0xe1,0x2a, +0x27,0x75,0x81,0xe4,0xa3,0x2a,0x64,0x89,0x67,0xcb,0x28,0x81, +0x3d,0xdf,0xf2,0xe4,0xc4,0x04,0x20,0x7d,0x48,0x04,0x54,0x25, +0x64,0x70,0xca,0x43,0x4c,0x01,0x62,0x1a,0x8e,0x1a,0xb2,0x64, +0x8c,0xd7,0x4f,0x24,0x61,0xcb,0xd4,0x2f,0xff,0x8e,0xbc,0x2d, +0x13,0x4a,0x53,0xf9,0x16,0x8a,0x2f,0x14,0x0d,0x02,0x64,0x15, +0xc0,0xaa,0xe5,0x29,0xcc,0x3c,0xec,0x53,0xca,0x61,0x9f,0x60, +0x06,0x03,0x84,0xf1,0x53,0x04,0xaf,0x28,0x36,0x4e,0xe7,0x57, +0x0b,0x34,0x16,0x28,0x7b,0x69,0x06,0x0d,0xb0,0xb5,0xb8,0x1c, +0x16,0x17,0xf3,0x69,0xb6,0x98,0x8f,0x2f,0xb2,0xc5,0x7f,0xa4, +0x5b,0x85,0x59,0xc1,0x3a,0xb1,0x35,0x7e,0x39,0xfc,0x8f,0xcd, +0xff,0x67,0xb1,0xb6,0x58,0x27,0x2b,0x44,0xbd,0x5f,0xca,0xed, +0xa6,0x36,0x19,0xce,0xc6,0x43,0x73,0x3c,0x43,0x87,0x53,0xe6, +0x7c,0x66,0x8a,0xcb,0x33,0xf3,0xe9,0x78,0x96,0xa4,0x26,0xdc, +0xb4,0x7d,0xa7,0xf7,0x89,0xbe,0x34,0x94,0x22,0x8f,0x2d,0x66, +0x34,0x32,0xa3,0xb9,0xb9,0x80,0xed,0x82,0x99,0x1f,0x4f,0x47, +0x9f,0xcd,0x7c,0x64,0xe6,0xa7,0xd3,0xe9,0xdc,0xcc,0xcf,0xe1, +0xff,0xe3,0x21,0xbc,0x56,0xb1,0xbe,0x0e,0xf6,0x94,0x1e,0xed, +0xec,0xca,0xcc,0xe6,0x15,0x68,0x07,0x17,0x2a,0xc7,0xf1,0xe8, +0xfb,0x36,0xb4,0xb7,0x7d,0x49,0xb4,0x01,0xbe,0x13,0x73,0x50, +0xd8,0xf8,0x46,0x1c,0x8d,0xd0,0xd9,0x9c,0x41,0xb7,0x6f,0x27, +0x17,0xb0,0xd1,0x2a,0x31,0xc8,0x1c,0xa1,0xa6,0xcd,0x9c,0x39, +0x19,0xb2,0x13,0x98,0x13,0x3a,0xa5,0xc2,0x43,0x61,0x33,0x2a, +0xe0,0xb7,0xf8,0xdd,0x8c,0x2e,0x0c,0xdd,0xcf,0x1a,0xe3,0x8f, +0x85,0xc3,0x96,0x03,0xb8,0xf3,0x8e,0x39,0xdf,0x36,0xe7,0x8f, +0xcd,0xf9,0x13,0x73,0xfe,0xb5,0x39,0x7f,0x6a,0xf8,0x0a,0x92, +0x39,0xe7,0x56,0xc7,0x0e,0x9b,0x20,0x1d,0x57,0x06,0x28,0xba, +0x86,0x3d,0xea,0xef,0x06,0xfa,0xf1,0x0a,0x90,0x71,0x88,0x1a, +0x53,0xb2,0x4b,0x23,0xe0,0x54,0x20,0x7d,0x98,0xeb,0x0b,0xe4, +0x5a,0xe1,0x4e,0xfe,0x4b,0x5b,0xd3,0x7f,0xb4,0x19,0x1e,0xc3, +0xc0,0x19,0x9e,0xcc,0xa6,0x93,0xcf,0x97,0xe6,0xd8,0x1c,0x8f, +0x0a,0xf8,0x37,0x35,0xc7,0xc5,0x19,0x8d,0xa8,0x02,0x3f,0x1c, +0xc3,0x40,0xd3,0xe7,0x9e,0x4e,0xcc,0xf8,0x12,0x3e,0x0c,0xcf, +0xec,0x69,0x98,0x21,0xe5,0x1f,0x8f,0x47,0xe2,0xd9,0x9b,0xbe, +0x61,0x38,0xfb,0x68,0x7e,0x33,0xb3,0xeb,0xe3,0xcf,0x86,0x86, +0x80,0x29,0x4d,0x09,0x92,0x82,0x29,0x2f,0x71,0x2f,0x53,0x5e, +0x0d,0x27,0x06,0x45,0xb9,0x8f,0x63,0xfc,0x99,0x4e,0xce,0x0c, +0xc8,0x76,0xf0,0x0f,0xc6,0x5c,0x01,0xdb,0xdb,0xf9,0xdc,0x5c, +0x1b,0x60,0x79,0xf8,0x8d,0xd1,0x02,0xea,0x87,0x96,0x6d,0x17, +0xf4,0x5d,0xc0,0x8e,0xf1,0xd4,0xac,0xa8,0x36,0x85,0x9e,0x34, +0xe1,0x58,0x89,0x9a,0x71,0xc5,0x58,0x45,0x93,0x7b,0x5e,0x16, +0x3d,0x09,0xc7,0xb0,0xf9,0xc1,0xfe,0xc3,0x6b,0x31,0xd8,0x46, +0xe7,0xb3,0xf1,0xa9,0xb9,0x80,0xcf,0x19,0x8d,0xcb,0x13,0x53, +0xce,0x4e,0xcc,0x75,0x39,0x86,0xf6,0xb0,0xe4,0x71,0xf1,0x2a, +0x6d,0x16,0xb3,0xea,0x12,0xea,0x90,0x0b,0x58,0x66,0xe0,0xef, +0xdc,0x0c,0xff,0x28,0x80,0xe8,0x33,0x98,0xa4,0xd3,0x99,0x39, +0x9e,0x52,0x90,0xe0,0x93,0xf1,0xc5,0xc5,0x15,0xde,0x45,0x82, +0xd6,0xc3,0x67,0x68,0xd4,0x13,0x7a,0x46,0xbf,0x79,0x06,0x4f, +0xca,0x88,0x17,0x4c,0xe9,0x6f,0x49,0x7f,0xb0,0xd9,0xf1,0xd4, +0x63,0x78,0x32,0x47,0xed,0xe2,0x6c,0x54,0xd2,0xe0,0x3d,0x05, +0x71,0x4a,0xc6,0x63,0x09,0xbf,0xc5,0xd9,0xf9,0x9c,0x3e,0xe5, +0x02,0x28,0x35,0xe7,0x88,0x77,0x6c,0x8a,0x12,0xfb,0x95,0x52, +0xf0,0xcf,0xf5,0xf0,0x6c,0x6c,0x26,0x53,0xfa,0xe0,0xc9,0xf4, +0xd3,0x6c,0x88,0x7e,0x6c,0x2e,0xe0,0xdf,0xef,0x06,0xd5,0xaa, +0xf4,0x07,0x6b,0x9b,0x5d,0x63,0x88,0x42,0x71,0x5a,0x4f,0x27, +0x2f,0x48,0x62,0x79,0x3e,0xc4,0x77,0x1e,0x05,0x43,0x1c,0x1d, +0xd7,0x97,0x97,0xa8,0xea,0x81,0x97,0xb3,0x31,0x70,0x9a,0x62, +0x0e,0x63,0x1c,0xcd,0x45,0xa9,0xdd,0xa0,0x11,0x58,0xa4,0xf9, +0x9d,0x69,0xf9,0x54,0x8c,0xe6,0xe7,0x38,0x36,0x9c,0x40,0x9b, +0xaf,0x72,0x44,0x05,0x13,0xa8,0xd6,0xe9,0x84,0x5d,0x29,0x13, +0xd3,0x1f,0xa4,0xec,0x9f,0x44,0xaf,0xc4,0x7e,0x21,0xf6,0xc2, +0x56,0x4d,0x61,0xf4,0xff,0x71,0x81,0x2b,0x6d,0x82,0x27,0xa1, +0xa1,0x43,0x14,0x5c,0xb1,0xdf,0xbc,0x7e,0xfb,0x7f,0xff,0xf3, +0xe8,0xa7,0xf7,0x6f,0x3c,0x2b,0xaf,0x2e,0x04,0x0d,0x59,0x09, +0xd2,0x5e,0xff,0xf9,0xe6,0xcf,0xc3,0xcd,0x3f,0xdb,0x9b,0xdf, +0xb6,0x8e,0xbe,0x6a,0xc2,0xf2,0xf7,0x9f,0xe9,0xe1,0xfe,0x23, +0x60,0xfe,0xe5,0x61,0xeb,0xb0,0x7b,0x68,0x0e,0x1b,0x87,0xe9, +0xe1,0xcd,0xe1,0xf2,0xf0,0xd9,0xe1,0xce,0x60,0xcb,0xec,0x3d, +0x7f,0xfd,0xe6,0xe0,0x9d,0x5f,0x64,0x04,0xcf,0x56,0x37,0xb6, +0x6d,0x40,0x69,0x5f,0x9a,0x96,0xf4,0x1a,0xeb,0xb4,0x17,0x52, +0xaa,0x84,0xae,0x73,0xe4,0x89,0xf7,0x4a,0x60,0x56,0xcf,0x86, +0x9f,0x48,0xbd,0xc0,0x1b,0xea,0xfe,0xc0,0x7c,0x22,0x89,0x22, +0xaf,0x11,0x32,0x48,0x3e,0xc1,0x8b,0xb1,0x78,0x13,0x88,0xcf, +0x02,0x48,0xf5,0x52,0x3d,0x75,0x11,0x0a,0xd0,0x74,0xca,0x41, +0xb7,0x38,0x31,0x97,0xbc,0x2e,0xef,0x68,0xa1,0x7a,0x11,0x34, +0xe3,0x16,0x4c,0xbb,0x29,0xde,0x64,0xb1,0x1b,0x55,0x63,0xfd, +0x5f,0xe6,0x2e,0x54,0x3e,0x5f,0x8c,0xca,0x13,0x69,0x0d,0xba, +0xb7,0x83,0xfe,0xc5,0xd8,0x48,0x8a,0x35,0x47,0xfc,0x35,0xb2, +0x4f,0xc4,0xca,0x9c,0x8d,0x7b,0x91,0xea,0x6f,0x69,0xe1,0xf8, +0xc6,0x6b,0x33,0xb6,0x04,0x6b,0xe1,0x92,0x61,0xa2,0x80,0xd2, +0x10,0x5d,0x65,0x0f,0x1d,0x74,0x13,0xde,0xdd,0x71,0x05,0x70, +0xf6,0x03,0xae,0x94,0x9a,0xdb,0x93,0x41,0x1b,0xdc,0x66,0xbc, +0x39,0xb7,0xbd,0x1a,0xd3,0xce,0x4a,0x0f,0x2f,0xe1,0x2d,0xef, +0xa1,0x2e,0x17,0x93,0x2d,0x58,0x9a,0xf8,0x2e,0x9c,0x0d,0x9a, +0x8b,0x07,0x32,0x33,0x98,0xac,0x33,0xb4,0x22,0x9b,0xce,0x5b, +0x47,0x86,0xee,0xfa,0x43,0xc7,0x03,0x93,0xa2,0x7b,0x87,0x3f, +0xe2,0xcd,0x98,0xfc,0x39,0x87,0xb3,0xb0,0xb7,0x64,0xcc,0xbb, +0xe3,0x5f,0x39,0xe3,0x1d,0x5d,0x10,0x56,0x39,0x78,0x4a,0xcd, +0x59,0xf6,0xbc,0x5a,0x65,0xe2,0x66,0x2b,0xf7,0x68,0x69,0xf3, +0x65,0xca,0x8b,0xe2,0x64,0xac,0x53,0x29,0x01,0xcd,0x0e,0x4e, +0x86,0x73,0x9d,0xce,0x29,0x66,0x3e,0x95,0xe3,0x6f,0x4b,0x44, +0xcb,0xa6,0x98,0xf0,0x92,0x9a,0x07,0x08,0xd3,0x61,0xad,0xc5, +0x83,0xe5,0x57,0x72,0x30,0xaf,0x6a,0xb0,0x67,0xf3,0x0c,0x80, +0x61,0xb3,0x54,0x26,0xb2,0x43,0xce,0x78,0x3f,0x1e,0x5d,0x87, +0x24,0xcf,0x28,0x25,0xc8,0x7e,0x8f,0x5c,0xb5,0x0a,0x43,0xc9, +0x96,0x02,0xe2,0x25,0x01,0x01,0x94,0x22,0xd9,0x2f,0x7f,0x1f, +0xcf,0x3e,0xeb,0xdc,0x31,0x26,0x48,0xe6,0xfe,0xf4,0x32,0x6c, +0x34,0x78,0x97,0xac,0xd7,0xac,0x86,0xd1,0xb9,0xa2,0x99,0x11, +0x00,0xb4,0xa6,0xa8,0x01,0xba,0xf0,0xc9,0x16,0x13,0xdf,0x3d, +0x95,0xee,0x97,0x9b,0xa8,0x92,0x87,0xa1,0x35,0x6c,0xf7,0xd3, +0x2d,0x04,0x4e,0xfe,0x0e,0x6f,0xe1,0xba,0x41,0x40,0x7b,0x6d, +0x73,0x14,0x5c,0x06,0xf5,0x31,0x9b,0x8e,0x7f,0xd5,0x3e,0x1b, +0x8e,0x7a,0x90,0x90,0xe1,0xf0,0x0c,0x53,0x1b,0x64,0x8d,0x76, +0x84,0x4b,0xce,0x15,0x7a,0xa8,0x3e,0xfe,0x95,0x96,0xf6,0xb5, +0x76,0x4a,0x1e,0x1b,0x8e,0xe4,0x0a,0x69,0xe2,0x4e,0xa1,0x93, +0x75,0x7b,0xf1,0x60,0xfc,0x07,0x3a,0x43,0x2a,0x7b,0x8d,0xba, +0x4c,0x66,0xef,0x1b,0x1b,0xfc,0xdb,0x12,0x58,0xbc,0xd0,0xcc, +0x4f,0x79,0x98,0x91,0x1f,0xe1,0x95,0x30,0x7a,0x6c,0x1d,0xc1, +0x4b,0xc6,0x13,0x26,0x3f,0x32,0x47,0xad,0x7f,0xbc,0x7c,0xbf, +0xff,0xfa,0xdd,0xdb,0x3c,0xe9,0xb4,0xbe,0x6e,0x6d,0xb3,0x97, +0xa4,0x31,0x8e,0xad,0xa3,0x96,0xfc,0x58,0x2b,0x90,0xe0,0x56, +0x2c,0xce,0x6b,0x74,0x89,0x47,0x26,0x36,0xa4,0xed,0x45,0x83, +0x72,0xba,0x0e,0x89,0xdf,0x84,0x2f,0x7a,0xa4,0x6e,0x6c,0x40, +0xaa,0xc3,0x94,0xe7,0x41,0x66,0xaa,0xf2,0x1a,0x15,0xc4,0x4e, +0x51,0x8b,0x50,0xa2,0x5c,0xcf,0xf3,0xa6,0x7f,0x0b,0xe3,0xd7, +0x5a,0x9f,0xaf,0x2e,0xdb,0xf9,0x7a,0xb5,0xfe,0x17,0x6d,0x0d, +0xad,0x13,0x34,0x3f,0x96,0x6a,0x0c,0x9d,0x92,0x0d,0x0c,0x9d, +0xb2,0x61,0x0c,0x09,0x61,0x26,0xb2,0xf2,0xb0,0x45,0x8f,0xad, +0x87,0x2e,0xb1,0x1c,0xd1,0xd0,0xa1,0x1e,0x34,0xaa,0xe2,0x8f, +0x35,0xf1,0xee,0xef,0xae,0x99,0x6f,0xa1,0x0c,0x38,0x10,0x0c, +0xd2,0x51,0x4f,0x45,0xf7,0x08,0xa7,0x32,0x54,0x2d,0x6e,0x43, +0xef,0xea,0x13,0x6f,0xec,0xaa,0xef,0xdd,0x71,0xb8,0x07,0x28, +0xd1,0x93,0xbc,0xcc,0x31,0x0d,0xee,0x27,0xac,0xc3,0xf6,0x11, +0x24,0xf6,0x24,0xad,0xda,0x37,0x59,0x63,0x4c,0xf1,0x9d,0xa1, +0xf2,0x48,0xa1,0x23,0x31,0x29,0x8a,0x92,0xce,0x01,0xa8,0x16, +0x3e,0x0e,0xab,0x6f,0x83,0x4a,0xa1,0xd4,0x99,0xab,0x96,0x30, +0x41,0xe4,0x16,0x18,0xf0,0x20,0xb2,0x9c,0xce,0x13,0xe6,0x53, +0xa8,0xba,0x24,0xb7,0x99,0x1c,0x03,0x69,0xed,0x53,0x31,0x3f, +0x5f,0x9b,0x4c,0xd7,0xd0,0x21,0x26,0x88,0xe1,0xac,0x03,0x4c, +0xa0,0xd1,0xb8,0x2c,0x8d,0xe5,0x8b,0xd1,0x05,0xfc,0x8a,0xc5, +0x4d,0x7d,0xf3,0x5d,0x8e,0x2f,0xa7,0x61,0x1b,0x0a,0xc2,0x3c, +0x76,0xe8,0xb2,0xb3,0xdd,0x95,0x61,0x4f,0xed,0x09,0x13,0x10, +0xcf,0x68,0x41,0x4a,0x0b,0xd8,0x29,0x37,0xaa,0xd0,0x60,0xdb, +0x95,0xb3,0xac,0x60,0x23,0xb5,0xa1,0x3a,0x47,0xa8,0x00,0x22, +0x49,0xd9,0x57,0x69,0x73,0xdc,0x49,0x11,0x35,0x3d,0x8f,0xb5, +0x11,0xd0,0x9e,0x66,0x35,0x39,0x74,0xd8,0x75,0x9f,0xde,0xb2, +0xd8,0x11,0x53,0x5e,0xdf,0x5b,0xd4,0x40,0x95,0x92,0x59,0x83, +0x8a,0xd8,0x74,0x6e,0x31,0xbe,0xe6,0xbd,0x2e,0xaf,0xa2,0xc3, +0x46,0xde,0x77,0x00,0x9c,0x8c,0x3a,0xb3,0xa1,0x3a,0xd6,0x09, +0x0e,0x88,0x69,0x69,0x8e,0xf4,0xc2,0x23,0xbd,0x37,0xfb,0x1f, +0xea,0x34,0xaa,0x53,0xf7,0x1c,0x13,0x11,0x76,0x1f,0xa5,0xfd, +0x9b,0xfa,0x90,0x70,0xad,0xee,0xc8,0x30,0x5b,0xac,0x4b,0x2b, +0xbc,0x0e,0x1d,0x1a,0xd3,0xd3,0x3a,0xb0,0x48,0xcb,0x1e,0xeb, +0x58,0x56,0x4d,0x34,0xa4,0xe5,0xbd,0x07,0x08,0x9e,0x9a,0x60, +0x41,0x8e,0x42,0xb3,0xb9,0xc9,0xe5,0x07,0x99,0x7d,0x32,0xf7, +0x1d,0x43,0xc4,0x74,0x95,0xcf,0xee,0x60,0x20,0x05,0x99,0xff, +0x96,0xd1,0x84,0x0e,0x10,0xa1,0x11,0x46,0xe3,0xf9,0x17,0xf1, +0x4e,0xe7,0x1a,0x6c,0xf2,0xf9,0x6e,0x56,0xc7,0x2e,0x4f,0xee, +0xc7,0xe4,0x7a,0x0d,0xb9,0x89,0xc0,0x39,0x68,0xff,0xc5,0xe2, +0x81,0x63,0x7e,0x4c,0x35,0x89,0x5a,0x47,0xe2,0x6e,0xfc,0xdf, +0xcb,0xf3,0x59,0x8e,0x93,0xe5,0x99,0x2b,0x72,0xab,0x33,0xbd, +0xf6,0x7c,0xce,0x5f,0xe5,0xff,0xf7,0x6c,0x8d,0x8d,0x8d,0x60, +0xa1,0x70,0x46,0x06,0x6e,0x19,0x20,0x7e,0x70,0x0b,0xdf,0x76, +0x4d,0x20,0xdf,0x6c,0x9b,0xee,0x7e,0x9d,0xb6,0x7e,0x4f,0x32, +0x97,0xae,0x1e,0x24,0x88,0xe4,0x5a,0xe8,0x1b,0xf4,0xe4,0x75, +0x97,0x80,0x24,0x29,0x8b,0x85,0x66,0x10,0xc5,0x68,0x8c,0xca, +0xf8,0xcf,0x81,0x93,0x19,0x6f,0xa4,0x13,0xf7,0x5b,0xa6,0xe4, +0x6b,0xee,0x35,0xa6,0xc0,0x76,0x1a,0x65,0xf4,0x5c,0xfa,0x5f, +0x5f,0xb2,0xb1,0x76,0x3b,0x3c,0xf9,0x67,0x63,0xe3,0xbe,0x6b, +0x77,0x8f,0x47,0x71,0x26,0xd2,0x0b,0x4e,0xd9,0x75,0xc6,0x21, +0x6b,0x39,0xcc,0x24,0x1c,0xcf,0xb8,0x0f,0x80,0xa6,0x83,0x97, +0x7f,0x63,0xd3,0x75,0xee,0x68,0x3a,0xdc,0x7d,0x70,0xcb,0x51, +0xfd,0xb6,0xe1,0x30,0xb9,0x67,0x53,0xff,0xb5,0x66,0x13,0x52, +0x80,0x58,0xa1,0xe9,0xde,0xed,0x26,0x0d,0xe6,0xb9,0x80,0x6a, +0xb7,0xa3,0x96,0xf8,0xcb,0x2a,0x49,0x7e,0xe1,0x30,0xe5,0x41, +0xbb,0x59,0xad,0x49,0xe5,0xfb,0xd7,0x3b,0x59,0xb0,0xbd,0xe2, +0xcf,0x97,0x8d,0x95,0x6b,0x01,0xc9,0xec,0x6d,0x76,0x48,0x86, +0x77,0x47,0xe2,0x82,0x36,0xab,0x67,0x7f,0xe1,0x81,0x2b,0x06, +0x5f,0x23,0xf0,0x25,0xcd,0x0e,0xbe,0xbe,0x19,0x52,0x29,0x1e, +0x0e,0x25,0x74,0xe5,0x59,0x99,0xd3,0x9e,0x99,0x9b,0xc6,0x2d, +0xd5,0x66,0x3b,0x15,0x4b,0x72,0xfc,0x58,0x7f,0xe3,0xd2,0xba, +0x47,0x74,0x33,0x1c,0xc5,0xd2,0xd5,0x44,0xc9,0x65,0xcd,0x9e, +0x78,0x16,0xa2,0xbc,0x3e,0xbf,0x0c,0xd2,0x16,0xdb,0x38,0x72, +0x37,0x20,0x25,0x29,0x13,0x7d,0x75,0x71,0x7d,0xf2,0x31,0xa4, +0x99,0x8e,0x27,0xef,0x59,0x27,0x37,0x04,0x1b,0x04,0x12,0xbe, +0x4f,0xe7,0xe3,0x59,0xd4,0x06,0x72,0x06,0x5a,0xcc,0x4a,0xcd, +0xad,0x8a,0xf2,0x25,0x0a,0xb3,0x7c,0x42,0x9a,0xf6,0x28,0xdb, +0x4e,0xa5,0xfe,0x20,0x3b,0xea,0x73,0x4a,0x82,0xcb,0x58,0x92, +0x25,0xcc,0xdc,0x92,0x41,0x2d,0x2d,0x6a,0xab,0x82,0x76,0xbb, +0x8c,0x12,0x3d,0xbe,0xd1,0x59,0x29,0x12,0xb7,0x2e,0x9e,0x3b, +0xf9,0x4a,0x99,0xf0,0x40,0x3b,0x79,0xd0,0x91,0xb1,0x5d,0x31, +0x3f,0xac,0xa0,0x5f,0x51,0x4e,0x9f,0xa8,0xbe,0x8c,0xa2,0x1f, +0x63,0x43,0xfd,0x71,0x8f,0x5d,0xa3,0x63,0xbc,0x1b,0x1b,0x47, +0xce,0x99,0x1b,0xca,0x28,0x34,0x4c,0xfb,0xed,0x81,0x6c,0xf7, +0xe0,0x89,0x07,0xae,0x84,0x54,0x7b,0xfa,0xf5,0xd7,0x8f,0xbf, +0xb6,0xc2,0xd7,0x1e,0xfa,0x7b,0x80,0xfa,0xa4,0x53,0xf7,0xc8, +0x97,0x09,0xca,0x39,0xd5,0x0a,0xb8,0x91,0x31,0x53,0x0a,0x6f, +0x76,0xb6,0xda,0x9a,0x85,0xdc,0xa0,0xda,0xf9,0x1a,0xe3,0x29, +0x61,0x0e,0xcf,0x53,0x7a,0x74,0x57,0xd8,0xee,0xc3,0x0b,0xf8, +0xfa,0x05,0x23,0x72,0x1c,0xa0,0x77,0x4f,0x56,0xc0,0x43,0xb5, +0x6b,0xcb,0xef,0xc8,0xcd,0x3d,0xfb,0x0e,0x32,0xa6,0xa5,0x95, +0xa9,0x63,0x04,0x8e,0x6e,0xfb,0xb0,0xf4,0xeb,0x67,0x4b,0x4c, +0x61,0xa1,0x57,0x8a,0xc9,0xff,0x6c,0xaf,0x14,0x93,0xbf,0xd2, +0x2b,0x6b,0x2b,0x7b,0xc5,0x77,0xca,0xff,0xc9,0x3e,0x79,0xf6, +0xef,0xec,0x93,0xf2,0xfc,0xfa,0xf4,0xf4,0x62,0x1c,0x29,0x9b, +0xe8,0xeb,0x87,0x93,0x91,0x8b,0x3e,0x27,0x60,0x23,0x25,0xd8, +0xad,0xf8,0x6c,0x98,0x9c,0x43,0x12,0x76,0xf1,0x67,0x7a,0x69, +0x03,0x74,0xa5,0x0e,0x05,0x8b,0xd6,0x9b,0x9d,0x41,0xee,0x52, +0x10,0x74,0x60,0xc2,0xd7,0xdc,0x3a,0xec,0xb5,0xc9,0x44,0xed, +0xf0,0xf2,0x2a,0x22,0xd6,0x4c,0xcc,0xd9,0xf5,0x70,0x36,0x72, +0x4c,0x21,0xde,0x75,0x3d,0xdb,0x5e,0x2c,0x08,0x02,0x17,0xd9, +0xbe,0xa3,0xcb,0x8f,0x9c,0xcd,0x4e,0x0a,0x3c,0xce,0xb6,0x04, +0x35,0x00,0xab,0x53,0x1b,0x6d,0x63,0x67,0x37,0x3c,0x4e,0x52, +0x91,0x21,0x2e,0xa6,0xd3,0x8f,0xd7,0x57,0xaf,0xad,0x38,0xb0, +0x82,0x19,0x07,0xab,0x87,0xbe,0xa8,0x91,0xad,0xd2,0xeb,0x49, +0x10,0xfb,0x25,0xae,0xb8,0x18,0x89,0xf7,0xbb,0x48,0x3a,0xb1, +0xdd,0xaa,0x77,0x9a,0x96,0x88,0x90,0x26,0xa9,0xcf,0x2f,0x56, +0xb4,0xb2,0x34,0x56,0x2d,0x20,0x35,0x92,0x44,0x30,0x92,0x28, +0x5b,0x82,0x7d,0x9e,0x90,0x62,0xbd,0x18,0x66,0xf7,0x15,0x5a, +0x97,0x29,0x7d,0x8c,0xba,0x5d,0x3f,0x3e,0x9d,0x9b,0x19,0xed, +0x5f,0x79,0x29,0xce,0x31,0xa5,0x65,0x11,0x9b,0xe3,0x9c,0x32, +0x5d,0x02,0x4e,0xd9,0x21,0xde,0xf4,0x60,0xc7,0xa1,0x3b,0xc7, +0x8b,0x05,0x2f,0x4d,0xc0,0x07,0x86,0x6e,0xca,0x22,0xd4,0xf1, +0xce,0x50,0xe5,0x1d,0x3b,0x26,0x6b,0xfd,0xa0,0x50,0x3d,0x3c, +0xfa,0xb8,0x0a,0x7a,0x86,0x21,0x26,0x81,0x03,0xa4,0x7f,0xe9, +0xac,0xd9,0x37,0xfd,0xf1,0xf8,0x7c,0xf8,0x7b,0x31,0xad,0xb9, +0x0d,0xbb,0xa2,0x53,0xec,0x64,0x5c,0x3a,0xfe,0x96,0xb3,0x44, +0xc4,0x37,0x06,0xbc,0xf8,0x98,0xdd,0xda,0x6b,0xb7,0xb2,0x15, +0xef,0x46,0xe6,0xee,0x7e,0x20,0xbe,0x67,0xbf,0x42,0x58,0x85, +0x0f,0xd6,0xe5,0x99,0x02,0x0d,0x3c,0x09,0xac,0xc6,0x51,0x43, +0x03,0x37,0x0c,0x61,0xa9,0x1b,0x18,0x4d,0xe7,0xc3,0x52,0x65, +0xa4,0x22,0xed,0xd2,0xb2,0x9e,0xa9,0x67,0xd4,0x70,0x44,0x3b, +0xab,0x23,0x6e,0xfa,0xfb,0xd4,0xa3,0x11,0x59,0x96,0x80,0x12, +0xe9,0xf5,0x64,0x7e,0x5b,0xf1,0xf4,0xe6,0x56,0xfa,0x9a,0xcd, +0x80,0xc2,0x0e,0x21,0xe5,0xf0,0xd7,0x1c,0xcc,0xc9,0x5b,0x15, +0x93,0xfa,0xfe,0xf6,0xdd,0x81,0xf4,0xaf,0xe3,0xeb,0xab,0xbb, +0xd8,0xab,0x51,0xac,0x9c,0xc4,0xd2,0xeb,0x4a,0x05,0x6d,0x6a, +0x2e,0xa6,0x9f,0x80,0x01,0x9f,0xc3,0x80,0xcd,0x39,0x40,0xba, +0x68,0x5c,0x30,0x65,0x07,0x32,0xbb,0x12,0xcb,0xb8,0x18,0x01, +0x17,0xf8,0xd4,0xa4,0xe4,0x9d,0x1d,0x98,0x10,0xb5,0x28,0x09, +0x45,0x1f,0x80,0x07,0xe9,0x33,0x1e,0x8f,0x88,0x1f,0xde,0x9b, +0x9d,0x8c,0xea,0x80,0x47,0x37,0x5f,0xa6,0x9f,0x90,0xe9,0xce, +0xa7,0x7c,0xa0,0xb1,0x8a,0x6f,0xf5,0xc2,0x65,0xba,0xa7,0xc4, +0x6a,0x7c,0xcf,0x56,0xa9,0xcf,0x7b,0x9e,0x1f,0xa9,0x2d,0x15, +0x70,0x62,0xa2,0x8b,0xb5,0x45,0x14,0xf0,0x1c,0x7a,0x26,0x30, +0xf6,0xd2,0xf5,0xfb,0xbd,0x46,0x7b,0x75,0x45,0xfe,0x31,0x53, +0x9a,0x28,0xab,0x7d,0x22,0x69,0x13,0xe4,0x5b,0x58,0xb3,0xf0, +0x78,0x1f,0x7e,0xe6,0x43,0xbd,0x77,0xe0,0x11,0x10,0x2f,0x33, +0x5c,0x31,0xe5,0x59,0x51,0x99,0x93,0x26,0x76,0xb9,0xe1,0xa6, +0x6e,0x0f,0xb2,0x60,0x9f,0x81,0xb8,0x70,0x31,0xe1,0x5d,0x0a, +0x6b,0x97,0xee,0xa8,0xaa,0xa6,0xbc,0x1e,0x09,0x9b,0x8d,0xa8, +0x62,0xd8,0x74,0xa5,0x84,0x3f,0x34,0xe9,0xff,0xd7,0xbe,0x23, +0xa8,0xb1,0x53,0xf3,0x51,0x6e,0xa9,0x0c,0x20,0x27,0xa6,0x9d, +0x8a,0x02,0x85,0x5a,0x18,0xbd,0x36,0xa2,0x26,0x6c,0x36,0xbd, +0xfa,0xe2,0xcf,0xae,0xf9,0xcc,0x25,0xf1,0x02,0xb2,0xd5,0x88, +0xd0,0x55,0x95,0x31,0x8c,0x44,0x0d,0x35,0x66,0xf7,0xa7,0x17, +0x18,0xb4,0x49,0x49,0xa6,0xc5,0x04,0xc4,0x25,0xb4,0xbe,0x40, +0x6b,0x65,0xb4,0x7e,0x84,0x57,0x4f,0x14,0x27,0xa3,0xe8,0x28, +0x7a,0x0e,0x82,0x76,0x53,0x00,0xef,0x48,0xe2,0x89,0xab,0x08, +0x9e,0x5c,0xda,0x10,0x90,0xdd,0xca,0x73,0x89,0x58,0x6c,0xf0, +0x93,0x88,0x13,0x16,0x0b,0x4e,0x11,0x61,0xc6,0x8a,0x10,0x52, +0x7f,0x0f,0xf9,0x69,0x58,0x87,0x5c,0x3c,0x93,0xcf,0x91,0xc5, +0x22,0xfa,0x8a,0x8c,0x7f,0x23,0x6e,0x2c,0x99,0xc8,0xfe,0x2b, +0x8d,0xc1,0x8d,0x26,0x68,0xfc,0x12,0x28,0x95,0x04,0xb9,0x68, +0x24,0x42,0xdb,0xce,0x02,0xb6,0xb9,0xd7,0xb7,0xf4,0xc7,0xa8, +0x38,0x3d,0x85,0x6d,0xdb,0x04,0x24,0x2c,0x41,0x50,0xb7,0x11, +0xef,0xf0,0xb8,0xb9,0x9e,0x14,0xbf,0xe5,0xfc,0x73,0x5d,0x99, +0x94,0x45,0xb9,0x4f,0x3c,0xbb,0x86,0x37,0x07,0xf2,0x97,0x85, +0x43,0xb3,0x0a,0x81,0x70,0x2c,0xd7,0xaf,0xd2,0x1e,0x9b,0x3c, +0xa0,0x6d,0x6b,0x57,0x6b,0xf4,0x15,0x87,0x47,0xd6,0x25,0x44, +0x54,0x54,0x36,0x9c,0xee,0x75,0xa1,0xa6,0x1c,0x43,0x9b,0x46, +0xa2,0xb3,0xe0,0x5c,0xb1,0xbc,0x3b,0x92,0x7b,0x94,0x80,0xb1, +0x8b,0xc6,0x93,0x3e,0xfe,0xf1,0x73,0xd0,0xc6,0x8f,0xcf,0xbc, +0x86,0xa6,0x81,0x10,0x32,0x14,0xd0,0xe8,0x98,0x0a,0xa8,0xde, +0x36,0x81,0xa6,0x93,0x67,0x36,0xab,0xba,0xf5,0xc1,0x97,0xb4, +0xfb,0x74,0x52,0x73,0x1f,0x88,0xbb,0xa2,0xe1,0x86,0x8a,0xea, +0xb1,0x75,0x99,0xea,0xe4,0x04,0x47,0x2c,0xf1,0x2a,0xa3,0x40, +0x24,0xa4,0x79,0xbd,0xf6,0xa5,0xa3,0x84,0x56,0x99,0xb6,0x52, +0x21,0x17,0x8f,0xfd,0xa4,0x3b,0x60,0x9e,0x8c,0x74,0x21,0xd0, +0xaf,0x13,0xe8,0x12,0x51,0x0b,0xe4,0xa2,0x57,0xa2,0x74,0xba, +0xcd,0x98,0xee,0xe4,0xa8,0x6f,0x20,0xaa,0xfd,0xa8,0x5c,0x4d, +0x73,0x30,0xb7,0xbd,0x15,0xc0,0xca,0x01,0x5c,0xf9,0x18,0x1e, +0x19,0xf5,0x5b,0x86,0x75,0xd5,0x8d,0xf4,0x21,0x76,0x82,0x22, +0x75,0x7f,0x16,0x57,0x79,0xe8,0x05,0xa7,0xa1,0x0e,0x62,0x48, +0xe3,0xd1,0xb0,0x72,0xbe,0x27,0x21,0xe1,0xfc,0x24,0x15,0x3b, +0x90,0x06,0xf4,0x90,0x1b,0x97,0x78,0x7c,0xc1,0x1c,0xc7,0xde, +0x6c,0xc1,0x7b,0x9c,0xc1,0xd1,0xb1,0x80,0xf6,0x8b,0x41,0x5e, +0x83,0x3c,0x69,0x16,0x91,0xb3,0x71,0x1a,0x39,0xec,0xff,0xde, +0x53,0x8b,0xfb,0x00,0x89,0xb2,0xeb,0x0e,0xeb,0xf3,0x9c,0x76, +0x07,0xd6,0x3d,0x8e,0x93,0x86,0x94,0xec,0xec,0x4f,0xb6,0x11, +0xb4,0xf6,0x64,0x9b,0x91,0x5a,0xb9,0x0e,0xc1,0xe8,0xa6,0x29, +0x27,0xc3,0x63,0x16,0xe6,0xc0,0x5a,0x3c,0xc8,0xed,0x73,0x67, +0x10,0x92,0xbe,0x34,0x6e,0x80,0x54,0xb8,0x0c,0x8c,0x14,0xc7, +0x13,0xd4,0x27,0xf0,0xe8,0xb0,0xbb,0x8c,0x6e,0x64,0x08,0x10, +0x08,0x6b,0x14,0x96,0x44,0x21,0x48,0x26,0xd7,0x97,0xc7,0xe3, +0x99,0x37,0xac,0x70,0xb9,0xf6,0x20,0x27,0x0f,0xe4,0x51,0x45, +0x49,0x2a,0x32,0x5c,0x81,0xf3,0x1f,0x13,0x7a,0x45,0x86,0x37, +0x06,0xf1,0x22,0x90,0xe5,0x19,0x6a,0xdb,0xca,0x04,0x34,0x1d, +0xfe,0xcc,0x3e,0x2d,0x9d,0xa9,0x84,0x53,0xce,0x32,0xc9,0x2b, +0xd4,0xb3,0xa9,0xdb,0x5f,0x2b,0xa0,0x46,0xd8,0x36,0xd4,0x8f, +0x15,0xeb,0x83,0x88,0x5e,0xf7,0x89,0x5d,0xb7,0x41,0x13,0x59, +0xe5,0xf5,0x2d,0xed,0x8f,0x17,0xac,0x6f,0x6f,0x7b,0x10,0xf7, +0x59,0x76,0x67,0x6b,0x10,0x2c,0xd0,0x75,0xdf,0xa8,0xcc,0x77, +0xec,0x77,0xea,0x1a,0xed,0xb7,0x2a,0x30,0x4b,0xa6,0x45,0xdb, +0xab,0x94,0x6a,0x78,0xc2,0xb2,0xfa,0xcc,0x54,0x79,0x04,0x75, +0x78,0xb0,0x40,0x16,0x8e,0x8e,0xcd,0xcd,0xee,0xbd,0x1b,0x6a, +0x86,0xfe,0x57,0x7c,0x13,0x89,0x69,0xe8,0x7c,0x8a,0x71,0xb8, +0xc7,0x57,0xe9,0x4d,0x45,0xff,0x91,0x77,0x60,0xd5,0x43,0x80, +0x9c,0x60,0x17,0x8b,0x36,0xdb,0x93,0xe6,0xed,0x94,0x8a,0xf8, +0x73,0x6a,0xf2,0xe2,0xd5,0xe9,0x46,0xcc,0xc5,0x0d,0x26,0x7a, +0x38,0x19,0x17,0x17,0x0d,0x42,0xb7,0x49,0x58,0xd2,0x2d,0xaa, +0x16,0xc4,0x3c,0x53,0x8c,0x50,0x55,0xc4,0xf4,0x55,0x98,0x8b, +0x1b,0x15,0xa3,0x3f,0xba,0x29,0xc1,0xf4,0xe1,0xb1,0xd9,0x1c, +0xe4,0xf6,0x0b,0xe0,0x6f,0x33,0x47,0x64,0x6e,0x62,0x22,0x14, +0x0b,0x69,0x27,0x81,0xa6,0x25,0xbd,0x41,0x59,0x25,0x8c,0x5d, +0x80,0x0f,0xe1,0x3e,0x1c,0x35,0xeb,0x86,0x42,0x13,0xfa,0x61, +0x80,0x96,0x56,0x1b,0x1b,0x08,0xcb,0xc5,0x6d,0xbf,0x63,0xba, +0xf3,0xe4,0xea,0x52,0x5a,0x3e,0xe6,0xf5,0x6a,0x16,0x8f,0x1a, +0xc5,0x40,0xd8,0x40,0xf0,0xb4,0xee,0x88,0xb8,0xeb,0x35,0x54, +0xab,0x0f,0x1f,0x88,0x60,0xfd,0xa9,0x14,0x34,0x3e,0xb6,0xf0, +0x22,0x28,0xa7,0xb0,0xa4,0xef,0x61,0x5a,0xfd,0x3e,0xef,0xac, +0xb4,0x4c,0xbf,0xae,0xaa,0x34,0xc5,0xb8,0xbd,0xb0,0x3b,0xf4, +0x81,0x03,0x08,0x8b,0x7b,0xed,0x4a,0xec,0xc9,0x53,0xea,0x4a, +0x04,0x8d,0xe1,0x9d,0x7f,0x5c,0x61,0xda,0x8a,0x0a,0x2c,0x77, +0x2f,0x12,0xe4,0x03,0xd8,0x24,0x4e,0x36,0xeb,0x68,0x7e,0xc4, +0x4f,0xf6,0xfc,0x0c,0xd1,0x2d,0xe9,0x84,0x04,0x6f,0x1c,0xe8, +0x0d,0x13,0xb5,0xf5,0x1d,0x27,0x3a,0x5e,0xa6,0xa8,0x4a,0x33, +0x8a,0x66,0xb2,0xf1,0xbc,0x07,0xcd,0x44,0x08,0x0e,0x9e,0xe7, +0xd1,0xc9,0x2b,0xd3,0x81,0x29,0xab,0x09,0x91,0x6b,0xbf,0x04, +0x64,0xad,0xc8,0xfc,0x48,0x61,0x43,0x82,0x44,0x90,0xaf,0x5d, +0x5e,0x97,0xf3,0xb5,0x63,0x74,0xa6,0x57,0x96,0xe3,0x91,0xa3, +0x9e,0xdc,0xd4,0x94,0x49,0xa8,0x28,0x22,0x8c,0x5e,0xb0,0x38, +0x4d,0x6f,0x50,0xa3,0x78,0x3a,0xb0,0x96,0x20,0xfc,0x46,0x1a, +0x05,0x94,0xf8,0x8f,0x7f,0x25,0x85,0xfc,0xf8,0x72,0x1a,0xec, +0xac,0x69,0xa4,0x9f,0xa3,0x7f,0x7d,0xf1,0xfd,0x40,0xf6,0x10, +0x3e,0x26,0x1c,0x67,0x81,0x5c,0xc9,0x0f,0xfa,0x28,0x34,0xb6, +0x9b,0x45,0xdd,0x14,0x43,0x45,0xed,0x2b,0x0d,0xe9,0x45,0x23, +0x54,0xd3,0x90,0x51,0x06,0x29,0x69,0xf0,0x89,0x55,0x48,0xee, +0x29,0xaf,0xe9,0x25,0xc1,0x42,0x9d,0x31,0x1a,0x5f,0x68,0xf5, +0x04,0x7d,0xc4,0xa7,0x61,0x31,0xbf,0xfb,0xa8,0xcf,0x12,0x51, +0x8e,0xe7,0x07,0xc5,0xe5,0x18,0xef,0xa9,0xdd,0x3a,0x46,0xc8, +0xd1,0x1a,0x9f,0xd8,0x71,0x0d,0x5c,0xfd,0xa9,0x0e,0xdd,0xc1, +0x43,0xd2,0x6f,0x76,0x90,0x38,0x29,0x7e,0x64,0xfa,0x44,0x5c, +0x67,0x70,0xdb,0x18,0x43,0xae,0x42,0x88,0x71,0x60,0xcc,0xe7, +0x17,0x71,0xff,0x60,0xc5,0x36,0x96,0xb5,0x3d,0x59,0xf0,0x93, +0x5e,0xc4,0x3b,0xba,0x22,0x82,0xbb,0x30,0xf6,0xf5,0x2d,0xa6, +0xcb,0x20,0xdf,0x49,0x41,0xbc,0xea,0xc7,0x4f,0x2e,0xce,0x0c, +0x3a,0x98,0x0b,0x98,0xac,0x2b,0x25,0x90,0x2d,0x89,0xc0,0x80, +0x77,0x1d,0x3b,0xbd,0x36,0x59,0x92,0xee,0x92,0xf3,0x5e,0x5d, +0x57,0x95,0x1b,0x68,0xf2,0x7c,0xdc,0x92,0x68,0xbc,0x4c,0xa6, +0x9f,0x72,0x8b,0xb0,0x6b,0x6b,0x5e,0x2c,0xa2,0xaa,0xd7,0xb1, +0x6a,0xa0,0xdd,0x91,0x06,0xc5,0xec,0x29,0xfc,0x25,0x48,0xce, +0x48,0x1d,0xb6,0xcf,0x66,0x03,0x32,0x36,0x2d,0x94,0xeb,0x67, +0xbb,0xf7,0x73,0x73,0xdd,0xaf,0x7d,0xa6,0xbd,0x93,0x3b,0x1c, +0xbd,0x06,0xdd,0x0c,0xb1,0x63,0x42,0xbe,0x2f,0x5d,0xd1,0xa8, +0x50,0xd5,0x5d,0x5f,0x9d,0x66,0x52,0xd4,0x7f,0xd1,0x7c,0x36, +0x2c,0x2e,0x6c,0x6b,0xc2,0x27,0x59,0xdc,0x6a,0x2c,0x52,0x8f, +0x18,0x47,0x55,0xea,0x35,0xb5,0x34,0xec,0x70,0x11,0x38,0xa9, +0x1d,0x1d,0xc5,0xe5,0xe5,0x78,0x54,0x38,0x8f,0x84,0x82,0x9a, +0xc7,0x87,0xa5,0x0b,0x13,0x61,0x3d,0xb9,0xbc,0x32,0xa1,0xb1, +0x91,0xea,0x97,0xdb,0x9a,0xcb,0x15,0xf7,0xbd,0x56,0x3b,0x88, +0xac,0x8b,0x24,0x07,0xb6,0xe9,0x4a,0x76,0x91,0xd6,0x1d,0xcc, +0xec,0xad,0xfc,0x78,0xea,0x4c,0xba,0x85,0x9e,0x35,0x82,0xd6, +0x77,0x9f,0xe8,0xad,0x1d,0x56,0x36,0x3e,0x9a,0xce,0xc0,0x14, +0x7b,0x0b,0x43,0xcc,0x15,0xdb,0xd8,0x58,0x17,0x7c,0xf6,0xc3, +0x5d,0x0f,0xad,0xee,0x0a,0x9a,0xf3,0xa9,0x45,0xe6,0x0f,0xde, +0x56,0xd6,0xac,0x3b,0x6c,0x5a,0xe9,0x2c,0xcf,0x63,0x51,0x6a, +0x52,0x16,0x24,0x55,0x26,0x04,0xf9,0xc4,0x1a,0xb3,0x06,0x41, +0xb6,0xc9,0x8c,0xed,0x16,0xc6,0x48,0xec,0x98,0x5b,0x8a,0x0c, +0xfa,0x88,0x02,0x34,0x07,0x8f,0x87,0x0b,0x59,0x88,0xd7,0x1c, +0x6e,0x28,0xe6,0x49,0x4c,0xcb,0x07,0xcf,0xf0,0x9a,0x27,0x1a, +0x4f,0xaa,0x4e,0x41,0x16,0x2f,0xa8,0x5c,0x39,0xaa,0xe9,0xa6, +0x3a,0x3c,0x81,0x5e,0x2f,0x1d,0x92,0x9a,0x16,0xb0,0xe2,0x68, +0x34,0x00,0x8b,0x60,0x15,0x65,0x9f,0x29,0x6d,0xf2,0x98,0xe2, +0x89,0xc6,0x4d,0x60,0x85,0x9a,0x81,0x96,0xc3,0x60,0x4f,0x48, +0xd4,0x0d,0x4f,0x83,0x71,0x4b,0x63,0xd4,0x04,0xac,0xbc,0xd2, +0x27,0x9b,0x3c,0x92,0x9f,0x75,0x7a,0xab,0xbb,0xc1,0x1a,0xd4, +0x60,0x15,0x64,0x09,0xe9,0xef,0x01,0x2c,0x16,0xa1,0xf0,0xc0, +0x86,0xe6,0xc0,0xe0,0x44,0x18,0xa2,0x13,0xe9,0x3a,0x43,0xc3, +0xe4,0xf5,0x84,0xee,0xc9,0xad,0x4d,0x6d,0xa0,0x39,0x67,0x67, +0xd9,0x1f,0x74,0x23,0xfb,0x0b,0xc4,0xc2,0x4b,0xac,0x35,0x27, +0xd9,0xd8,0xf0,0x01,0x45,0xe8,0x76,0xfb,0xcc,0x87,0x19,0x41, +0x32,0x79,0xf7,0x1c,0x49,0x36,0x2b,0x0d,0xd0,0xab,0xd6,0x9c, +0xb2,0xd5,0xbf,0x8f,0x7a,0xc1,0x6d,0xd4,0x73,0x6d,0x8a,0x1e, +0x38,0xd8,0x2a,0x59,0xec,0x2b,0x66,0xff,0x0a,0x45,0x5c,0xfe, +0x1e,0x04,0x11,0x20,0xd2,0xd3,0x57,0xe6,0xf0,0x8e,0x30,0x15, +0x3f,0x06,0xc0,0xc4,0xcc,0x68,0x3c,0x9b,0xaf,0xa0,0xcc,0xeb, +0x32,0xfe,0x92,0xd9,0xbe,0xe8,0x2e,0x02,0x02,0x24,0x68,0x44, +0x8d,0xea,0xc2,0x92,0x80,0xf5,0xb0,0x9d,0x51,0xdc,0x60,0xd6, +0xc1,0xe1,0xea,0x51,0xa2,0xf6,0x30,0x52,0x2b,0xba,0x2f,0xa4, +0x32,0xd5,0xe1,0xc2,0xc9,0x74,0xb4,0xcb,0xf6,0x88,0xec,0xc8, +0xb3,0xf6,0x90,0x86,0x24,0xd4,0x15,0xa2,0x8e,0x72,0x71,0x4d, +0x2e,0xe8,0x69,0x22,0xc8,0xa3,0xa5,0x12,0xaf,0x91,0x21,0x99, +0x92,0x8c,0xc4,0x61,0x12,0x6c,0x1d,0x39,0x22,0x1e,0xbd,0x78, +0xb1,0xf6,0xaa,0x88,0xcc,0xa8,0xac,0x94,0x74,0xf5,0xd9,0xf5, +0xc6,0x5f,0xd5,0xe4,0xd1,0xa7,0xd0,0xd5,0x1d,0x57,0x01,0x9d, +0x36,0xfa,0x86,0x24,0x15,0xf3,0xd5,0x67,0x96,0x5c,0x5d,0x3b, +0x2e,0xd9,0x17,0x1e,0xad,0x03,0x97,0x45,0x3c,0x64,0xfe,0x75, +0xf2,0x6a,0x3b,0xd4,0xa9,0x13,0x89,0x60,0xa4,0x93,0x62,0x51, +0x55,0x88,0xf3,0xe2,0x11,0x53,0xe8,0x42,0x52,0xff,0x4f,0xf4, +0xa6,0x3b,0xb0,0x77,0xdd,0xca,0x86,0xf1,0x35,0x5d,0xac,0xb6, +0x2e,0xe4,0x90,0xb7,0x9e,0x3e,0x1c,0xc8,0x8a,0x87,0xc6,0xc7, +0x93,0x64,0x6a,0x49,0xe6,0x1d,0x78,0xd6,0xa8,0x6e,0x81,0xcb, +0x59,0x25,0x49,0xde,0x7a,0x9d,0xa4,0xb3,0x5f,0x15,0x5e,0xd2, +0xd9,0x5a,0xab,0x68,0xd1,0x34,0xab,0xb1,0x30,0x5f,0x63,0xfa, +0x4d,0xa9,0xb0,0xcc,0xb1,0x19,0xee,0xa3,0x03,0x28,0x73,0x4c, +0x3f,0x6c,0xc6,0xa0,0x0c,0x14,0xd6,0xda,0xc0,0xf4,0x87,0x8b, +0x45,0x67,0x0b,0x52,0x3b,0x5b,0xc7,0xfe,0x92,0x00,0x24,0xf2, +0x83,0x03,0xa5,0x72,0xdd,0x20,0x6c,0xef,0x11,0x34,0xd7,0x30, +0x1f,0xba,0xcb,0x5e,0xa9,0x39,0x8e,0xb3,0x8f,0xf3,0x63,0x9f, +0xcd,0xfa,0x14,0xbc,0xfe,0x4c,0xee,0xcd,0xec,0x3d,0x40,0xe9, +0x4b,0xda,0xa2,0xba,0xdc,0xf5,0x28,0xfb,0x38,0xf5,0xf6,0x7a, +0x12,0xe2,0xd9,0xc1,0x72,0x6c,0xe7,0xa4,0xcf,0xab,0xd2,0x1a, +0x17,0x1b,0x24,0x99,0xa7,0x5c,0xbc,0x96,0x1e,0xa7,0xdd,0x00, +0xf2,0x2d,0xa9,0x42,0x15,0xe4,0x7a,0xde,0x1c,0xf6,0x8e,0xe1, +0xef,0x71,0x06,0xe3,0x62,0xd8,0xb3,0x0d,0x93,0xc1,0x4f,0xf3, +0x38,0x2c,0x8c,0x12,0x25,0x14,0x0d,0xd2,0xbe,0x9b,0x4e,0xd1, +0x03,0xb2,0xc3,0xd8,0xac,0x29,0xc7,0xae,0xc6,0x54,0xa5,0x2d, +0x89,0x6c,0x0a,0x4d,0xc5,0x4f,0x1b,0x1b,0xc3,0xd6,0xd9,0xc5, +0xf4,0x78,0x88,0xed,0x2f,0x4f,0x98,0x76,0x09,0x33,0x03,0x05, +0x77,0x02,0x75,0x2f,0x98,0xc3,0xee,0x4e,0x5e,0x40,0x35,0x98, +0xe5,0xdf,0x50,0xd3,0x6a,0xa3,0xc2,0x3a,0x85,0x2f,0xf4,0x6d, +0x25,0xed,0xd8,0xb7,0x6e,0xa4,0x83,0xe3,0x11,0x14,0xae,0x13, +0x56,0x69,0x48,0x59,0x7d,0xb9,0x0a,0xa1,0x6c,0x62,0x8e,0xa3, +0x8c,0x63,0xb6,0x80,0x7e,0x41,0xc1,0x91,0x91,0x37,0x94,0xf3, +0xd9,0x35,0x05,0x58,0x3a,0xa6,0xb4,0x63,0x9d,0x46,0x56,0x37, +0x98,0x8c,0x96,0x37,0x2f,0xc8,0x58,0x6e,0xbd,0x11,0x2c,0x11, +0x94,0x0b,0xeb,0x03,0xfd,0xea,0x11,0x37,0x7c,0xe1,0x6c,0xeb, +0x1c,0xf0,0xb1,0x00,0x1f,0xc7,0xc0,0x9c,0xe1,0xbf,0x5b,0x3e, +0x94,0x8f,0xa7,0x52,0x99,0x32,0xfc,0x7a,0xcc,0x43,0x97,0x6c, +0x01,0xda,0xc6,0x1b,0xac,0x63,0xeb,0xda,0x6e,0xa5,0x79,0x3e, +0x48,0xf2,0x5c,0x0d,0x4b,0xe4,0x40,0x58,0x66,0x68,0xc5,0x01, +0x29,0x4a,0x89,0xf0,0xd5,0x22,0x0d,0xb0,0x0f,0x48,0x48,0xdb, +0xdc,0xf4,0x82,0xfd,0xf8,0xb7,0xc6,0xb0,0x8f,0x89,0x03,0x73, +0x2c,0xbf,0xe1,0x64,0x4e,0xbb,0x69,0x97,0xee,0xe8,0x55,0x0c, +0x5f,0xb1,0x6f,0x58,0xf2,0x1a,0x8a,0xdc,0x45,0x64,0x34,0x9b, +0x66,0xdd,0x62,0xe7,0xec,0x63,0xc9,0xa6,0xba,0x90,0x1f,0x43, +0x55,0xf4,0x13,0xd5,0x94,0xa6,0x1c,0x53,0x1c,0xc3,0x9d,0xb1, +0xa2,0x8c,0xea,0x94,0xfa,0x8e,0x7d,0x7d,0x16,0xe1,0x3a,0x7f, +0x8d,0x14,0xb3,0x0d,0x86,0x89,0x4b,0x6b,0xe2,0x61,0x9b,0x7b, +0x7a,0xd5,0xf0,0x8d,0x4d,0x2f,0x22,0x5a,0x74,0xc9,0x44,0x12, +0x7d,0x75,0x07,0x2c,0xcd,0x2f,0x06,0xbf,0x11,0x87,0xeb,0x0f, +0xec,0xe9,0xaf,0x18,0x54,0xe6,0x15,0xe1,0xd6,0x59,0x6a,0xd8, +0xce,0xa6,0x9e,0x0b,0x99,0x33,0x1f,0x7a,0x0b,0x9f,0xd0,0x26, +0x99,0xb2,0x3e,0xd8,0x09,0x50,0xb3,0xda,0xb9,0xaf,0xb7,0x72, +0x6e,0x8d,0x65,0x31,0x93,0x27,0x4e,0xeb,0xeb,0xd6,0x8e,0xf5, +0xc6,0x3a,0xbc,0x91,0x77,0x24,0xaa,0x6e,0x32,0x1d,0x8d,0x51, +0xf0,0x96,0x2f,0x63,0x3b,0x98,0xe0,0x9a,0x6f,0x2c,0xc5,0x33, +0x9e,0xea,0x78,0x0c,0xf9,0x28,0xa9,0xe1,0x8c,0x5f,0xa8,0x56, +0xda,0xd6,0xe4,0xc1,0x76,0x60,0x69,0x68,0xed,0xed,0x27,0xcf, +0x7d,0xa8,0xf7,0xc4,0xce,0x31,0x78,0xe4,0x2a,0xe0,0x81,0x59, +0x2a,0x3c,0xec,0x52,0x98,0xab,0x84,0xb9,0x5d,0x32,0xf0,0xcb, +0xf3,0x84,0xa6,0xc6,0x51,0x3f,0x29,0xca,0x4a,0x1c,0x63,0x4d, +0x43,0x95,0xee,0x3c,0x77,0x5f,0xc7,0x25,0x9b,0xc9,0x20,0x59, +0xb2,0x51,0x97,0xb2,0x57,0xf0,0x7b,0x20,0x90,0x39,0x82,0xac, +0x5b,0x5b,0x7e,0xdd,0x77,0x62,0x82,0x55,0x8e,0xc7,0x09,0x9d, +0x4c,0x27,0xb6,0x90,0xe3,0x96,0x5b,0xad,0x2d,0x98,0x52,0x9a, +0xcb,0xd4,0x22,0xf6,0x05,0x5d,0x68,0x35,0x5c,0xa7,0x99,0xdc, +0x57,0x78,0x08,0xbf,0x42,0x8c,0xb0,0xb9,0x62,0x7b,0xbc,0x5e, +0x00,0x3b,0x79,0xdb,0x20,0xaf,0x74,0xaf,0x2e,0xa6,0x43,0xd9, +0xa0,0x71,0x2f,0x42,0xce,0x6a,0x59,0x84,0x3b,0xc3,0x9b,0x30, +0xaf,0x93,0x6d,0x12,0x17,0x94,0xa5,0xea,0xb6,0xfe,0x5f,0x6f, +0x2f,0x16,0xf2,0xd4,0x81,0x35,0xa3,0xb2,0xc8,0xad,0x1e,0x5b, +0x6f,0xaf,0x2b,0x1a,0xea,0xc0,0xe8,0x27,0x77,0x64,0x78,0xbf, +0xd3,0xb5,0xe0,0x5a,0x44,0xc3,0x02,0xd0,0x43,0xab,0xaf,0x29, +0x84,0x7e,0x03,0x1c,0x4d,0x04,0x83,0x85,0x27,0x53,0x0c,0x2b, +0x0f,0x02,0x58,0x9d,0xb7,0x51,0xb9,0x1f,0x5e,0x75,0xb5,0x20, +0x61,0x0a,0xbd,0xc2,0x79,0x95,0xdd,0xad,0xb3,0x6c,0xa6,0xcd, +0xb9,0x87,0x9a,0xd4,0xd8,0x88,0x38,0xcd,0xc2,0xc9,0xc9,0xf5, +0x25,0x5f,0xd8,0x6f,0x84,0x16,0xbf,0xb4,0x45,0x9c,0xc8,0x8e, +0x8c,0xc0,0x70,0x77,0x58,0x6f,0x5a,0xe7,0x0f,0xc3,0x09,0x50, +0xce,0xea,0x46,0xd3,0x4b,0x4f,0xc4,0x65,0x31,0x31,0x80,0x3b, +0xea,0x07,0x48,0x21,0x37,0x95,0x7f,0xe4,0x94,0x5f,0x4c,0xf0, +0x64,0x0e,0x7e,0x9a,0x44,0xcb,0xe9,0xc5,0x14,0xc8,0xa4,0x47, +0x31,0x5d,0x4e,0x1f,0x21,0xf0,0x26,0x42,0x74,0xac,0x51,0x32, +0x37,0x0a,0x7a,0x5f,0xb8,0x19,0x97,0x27,0xc3,0xab,0x71,0x76, +0x93,0x6c,0x24,0x99,0x78,0x1e,0x34,0xc9,0x33,0x7c,0x46,0xd7, +0x80,0x26,0x41,0xcf,0x71,0xe4,0x10,0xd0,0x3c,0x4c,0x1e,0xc2, +0x23,0xba,0x6c,0xc2,0xf4,0x87,0x98,0xfe,0x1f,0x7f,0x6c,0x7f, +0xd3,0x85,0xe9,0xdc,0x75,0x08,0x5b,0xd7,0x13,0xc6,0x98,0xdb, +0xfd,0x6e,0xc3,0xe7,0x71,0x4e,0xaa,0x48,0x78,0x8f,0x41,0x12, +0xd1,0xc1,0x89,0x90,0xa1,0xfd,0xba,0xf6,0x93,0xa6,0xec,0x82, +0x2b,0x18,0x9c,0x6b,0x0e,0xe4,0x26,0x1c,0x32,0xd8,0xd6,0x1b, +0xa0,0x68,0xd4,0xa0,0xb0,0x80,0x16,0xc9,0x82,0xa3,0x95,0x12, +0x16,0x5c,0xc0,0x84,0x69,0x32,0x10,0x24,0x5b,0x78,0xcd,0x0f, +0xed,0xe5,0xa0,0x23,0x7b,0x4b,0x47,0x9f,0xb0,0xe2,0xe4,0x8a, +0xfa,0x8c,0x13,0xd1,0xe9,0x26,0x10,0xdd,0x14,0x10,0xe7,0xcd, +0x24,0x68,0x0a,0x8b,0x31,0xf2,0xb0,0xec,0x97,0x50,0xfb,0x21, +0x16,0xb0,0x4f,0xf9,0x78,0x89,0x87,0xd8,0x94,0xd2,0x31,0xda, +0x09,0x07,0x0c,0xc0,0xfa,0x56,0xf9,0x1c,0x2e,0xae,0x90,0x93, +0x06,0xb3,0xb6,0xeb,0x8d,0x4c,0x39,0xbb,0x6f,0x0b,0x0e,0xba, +0x77,0x99,0xaa,0xbb,0xc9,0x8b,0x58,0x33,0x75,0x8f,0xe3,0x8f, +0x22,0x66,0x59,0xd4,0xc6,0x4a,0x0d,0xc1,0x3b,0xa2,0x68,0xb9, +0xb1,0xfa,0xbe,0xfc,0x48,0xdc,0x3b,0xe3,0x36,0x8f,0x83,0x04, +0x1c,0xa9,0xd0,0xfa,0xd1,0x72,0xa4,0x35,0x91,0x81,0x5b,0x8b, +0xfb,0xa8,0x24,0xed,0x5d,0x09,0xfc,0x10,0xd2,0xcd,0x29,0x5d, +0xdd,0x91,0x28,0x69,0xd1,0x1b,0x0c,0x5b,0x6f,0x8c,0x5e,0xa0, +0x5d,0xf1,0x78,0x06,0x93,0xde,0x9a,0x9c,0xbd,0x56,0x3c,0x11, +0xd8,0xd2,0x69,0x21,0xb6,0xd7,0xc5,0x28,0x6f,0x36,0x5d,0x81, +0x66,0x92,0x38,0x62,0x08,0xa8,0xc7,0x3f,0x00,0x91,0x15,0x74, +0x75,0xc1,0x86,0xf0,0xd9,0x1f,0xcf,0xe7,0x30,0x56,0x70,0x82, +0x60,0x83,0x42,0x4a,0xb6,0xf5,0xec,0xab,0x46,0xff,0xb0,0x3c, +0xdc,0x1f,0x34,0x7b,0xe9,0x57,0xe8,0x16,0x4f,0x85,0x11,0xc3, +0xdc,0x3c,0xca,0x96,0x79,0x01,0x39,0x9b,0x61,0x0e,0x7f,0xc6, +0x64,0xca,0x41,0xbf,0xb6,0x1a,0xad,0xf4,0x97,0x2d,0x81,0x86, +0x0a,0x69,0x86,0x3f,0x4c,0x30,0x3e,0x2f,0x3c,0xc0,0x1f,0x78, +0x9a,0xc1,0x13,0x0a,0x0a,0x87,0x13,0x78,0x40,0x19,0xe2,0xff, +0x82,0xdf,0x39,0x26,0x5c,0x6f,0xb7,0xb7,0xff,0x06,0x2f,0xfc, +0x2b,0x09,0xdf,0x4a,0xc2,0xb7,0xc9,0x52,0xf0,0xce,0xf2,0xad, +0xc3,0xc3,0xc5,0xc3,0xc5,0xe1,0x6c,0x71,0x38,0x59,0x1c,0xce, +0x17,0x5c,0x90,0x7f,0xbe,0xdd,0x3a,0xeb,0xfa,0x4f,0x57,0xea, +0x55,0xe4,0x97,0x18,0x42,0xc7,0x94,0xd2,0x1c,0xd6,0xc8,0x0a, +0x19,0x7e,0xd7,0x26,0xe6,0x5e,0x15,0x81,0x3b,0x74,0x9b,0x5c, +0xd3,0x9a,0xa9,0x77,0xce,0x04,0x14,0x29,0x86,0xd1,0x6f,0xd8, +0x52,0xc2,0x69,0x60,0x4b,0xcd,0xcd,0x93,0xca,0x3e,0xcf,0x78, +0x08,0xd5,0xec,0xb7,0x81,0xd9,0x8e,0xab,0xc0,0x0c,0x34,0x03, +0x5a,0x3c,0x10,0x3e,0xe6,0xee,0xd0,0xf0,0x06,0x33,0x39,0x3a, +0xba,0x6a,0xe6,0x0f,0x93,0x6e,0xe0,0x99,0x5a,0x48,0x8f,0xbd, +0xb0,0x33,0xcd,0x7a,0x3c,0x18,0x5b,0xbb,0x99,0x9e,0x9e,0x96, +0xea,0xd6,0x25,0xa3,0x6f,0xb2,0xc3,0x6b,0x56,0x70,0x88,0xfd, +0x3f,0xc3,0x79,0xe6,0xc4,0xfd,0xb6,0x82,0x1d,0xe1,0xb8,0x68, +0xca,0x90,0x71,0x8c,0x48,0xe8,0xc0,0x2d,0x8c,0xd4,0x92,0x3c, +0x6c,0x1e,0x4e,0x1a,0x8d,0xa3,0xa3,0x79,0xde,0xb0,0xf0,0xc0, +0x74,0x41,0x3a,0x44,0x56,0xd4,0x7b,0xf8,0x10,0x75,0x2b,0x94, +0x8a,0x30,0x69,0x0a,0xd0,0x0f,0xa9,0x2d,0xdc,0x87,0xac,0x44, +0xa6,0x60,0x22,0x8c,0x80,0x48,0xf0,0xd8,0x46,0x08,0x90,0x74, +0x61,0x10,0x37,0x6d,0x4e,0x13,0x86,0xb4,0xb4,0xb4,0xed,0x02, +0x6e,0x08,0xf6,0x19,0x65,0xf7,0x82,0xf4,0x82,0xb7,0x8c,0x02, +0x2c,0x6e,0xa8,0xb5,0x60,0x58,0x15,0xe8,0x4b,0x10,0x6d,0x29, +0xa5,0xff,0xd0,0xbe,0xb5,0x41,0xc2,0xea,0xcd,0x32,0xbd,0xc1, +0x3a,0xa5,0x70,0xb2,0x84,0x97,0xd4,0xf5,0x33,0x8e,0x48,0x20, +0xd9,0x00,0x15,0xf9,0xc3,0x87,0xf0,0xf3,0x6b,0xec,0x12,0x8a, +0x86,0x0b,0x06,0x6b,0x9b,0x04,0xb2,0x10,0x91,0x0d,0xe0,0xb1, +0x7a,0xed,0xe1,0x43,0xd8,0x6b,0x76,0x75,0x85,0x96,0x81,0x1f, +0x5d,0x61,0x72,0x77,0x3e,0xfb,0x7c,0xc3,0x53,0x88,0xa6,0x80, +0xe3,0xe9,0x35,0x5f,0x83,0xba,0x07,0x18,0x9f,0x47,0x89,0x90, +0x9b,0x2e,0x4f,0xc8,0x95,0xd8,0x18,0x03,0xd9,0xe1,0x99,0xc3, +0xd8,0x2a,0x44,0x64,0xf8,0x93,0x16,0x03,0xa7,0x6c,0xea,0x74, +0xcf,0x58,0x11,0x25,0x99,0x23,0x9e,0x7f,0xd5,0x89,0x4e,0x05, +0x6e,0x82,0x12,0x8a,0x17,0x4b,0x59,0x77,0x24,0x6c,0xcb,0xdb, +0xaa,0x9d,0xec,0x0e,0xa3,0x62,0xe5,0x47,0xe0,0x5a,0x1f,0xf6, +0x42,0xe2,0xa2,0xb6,0x91,0xa6,0xf0,0x7c,0x58,0x59,0xad,0x6c, +0xbb,0xf1,0x3d,0x00,0x82,0x68,0x38,0x6f,0x2f,0x95,0xd5,0x36, +0x8c,0xa0,0x7d,0x44,0xe0,0xbd,0xb0,0x2c,0x29,0x0e,0xbb,0xb2, +0x34,0x36,0xd0,0xeb,0x11,0xcb,0x1c,0xb0,0x97,0x86,0x56,0xc6, +0xf5,0x09,0x7e,0x66,0x68,0xa4,0x4a,0xfe,0xb2,0xcb,0xf3,0xe2, +0x14,0x99,0x2c,0xaa,0xcf,0xf1,0xe7,0x0a,0xa7,0x2b,0x49,0x26, +0x9c,0x53,0xd9,0xa8,0xf1,0x39,0x21,0x4a,0x07,0xca,0xf9,0xd4, +0x7d,0x97,0x4e,0xdc,0x3d,0x04,0x0b,0xa7,0x77,0x4f,0x81,0x18, +0xad,0x19,0x39,0x5e,0xa2,0xf5,0x0b,0xa7,0xd0,0xb8,0x9e,0x4f, +0x28,0x36,0x85,0xa5,0x91,0xdf,0x17,0x8b,0xf6,0xba,0xde,0x90, +0x2f,0x16,0x12,0x96,0x64,0x4a,0xf7,0x32,0xab,0xab,0x2e,0x6d, +0x4a,0x96,0xae,0x59,0x58,0xbd,0x0d,0x1f,0x8c,0x13,0x00,0x1b, +0x80,0x70,0xff,0xfb,0xbe,0x3a,0x38,0x1e,0x51,0x64,0x04,0x1f, +0x1c,0x34,0x89,0xfa,0x74,0x11,0xbb,0x44,0x33,0xac,0xea,0x31, +0x37,0xd4,0xdb,0x59,0xb5,0x22,0x35,0x30,0xf0,0x6c,0x96,0xf7, +0x2a,0xd1,0x6d,0xbf,0x08,0x58,0xea,0x45,0xd9,0xc3,0x13,0xa8, +0x24,0x26,0x15,0x24,0xcb,0x19,0xfe,0x5c,0x0d,0x47,0x07,0xd3, +0x8f,0x63,0x67,0xec,0x06,0xa2,0x47,0xf5,0x78,0xd2,0xcf,0x37, +0xbc,0xef,0xf6,0xf3,0x78,0x36,0x7d,0x55,0x00,0x7a,0x92,0x7b, +0x7c,0x4b,0x0c,0x53,0x29,0xaf,0x22,0x53,0x4e,0x67,0x23,0x0c, +0x20,0x59,0xfc,0x39,0x56,0xb5,0xc0,0x4a,0x51,0x4c,0x47,0xb7, +0x55,0x43,0xdf,0x83,0xee,0x2e,0x1b,0x69,0x4b,0x50,0xd4,0xd4, +0x26,0x78,0x54,0x75,0x6f,0xc4,0x43,0x26,0x9a,0xf3,0xb9,0xc4, +0xbd,0x29,0xb9,0xa4,0x84,0x01,0x72,0x5a,0x80,0xcc,0x0d,0x6b, +0xe2,0xc9,0xc7,0x77,0x30,0x6d,0x60,0x17,0xf4,0xc9,0xa6,0x4a, +0xd0,0x38,0x46,0x2d,0x69,0x1e,0xc3,0xee,0xf5,0x6c,0xc8,0xbc, +0x47,0x1e,0xac,0x7d,0xc9,0xec,0x92,0xbe,0x6d,0xf4,0x1a,0xef, +0x5f,0xe4,0xee,0x9d,0xf5,0x2b,0x3f,0xc1,0x56,0xbf,0xf4,0x45, +0xcc,0xe7,0x31,0xfa,0xec,0x8f,0x0a,0xb5,0x30,0x15,0xed,0x27, +0x2f,0x61,0xcb,0x77,0x5e,0xcd,0xa6,0x64,0xcc,0xff,0x34,0x1e, +0x7f,0xac,0x66,0x63,0x2a,0xe6,0x8e,0x86,0x9f,0xab,0x99,0x23, +0xd4,0x25,0xb5,0xcd,0x39,0x30,0xb1,0x6a,0x26,0xa6,0x52,0xbd, +0xc5,0x04,0xc3,0xa2,0x56,0x2b,0xa6,0x74,0xb2,0xec,0x1c,0x43, +0x83,0x8c,0xaa,0x10,0x9c,0xce,0x38,0x2e,0x2e,0x8a,0x55,0x60, +0x2a,0x13,0x60,0xbb,0x3c,0x54,0xe9,0xc2,0x4a,0x6e,0xdb,0xc6, +0x70,0x62,0x80,0xa6,0xa9,0xdf,0x9a,0x9d,0xf1,0xe3,0x47,0xf6, +0xf9,0xe9,0xf8,0xc9,0x23,0xa1,0xba,0xf9,0xf8,0xe9,0xf8,0xeb, +0x47,0xf4,0x81,0x82,0x83,0x1a,0xa2,0x89,0x7f,0x9b,0xdf,0x3c, +0xa2,0x36,0xb3,0xc8,0xb9,0x7d,0x9b,0xfc,0xdb,0xec,0x6c,0x3f, +0xa2,0xfe,0x70,0xc5,0xe6,0x43,0x3c,0x29,0xe3,0xb7,0xe3,0xeb, +0x63,0x58,0x0c,0x74,0x88,0x00,0x19,0x1e,0xa4,0xa6,0x74,0xf6, +0xb7,0xac,0x23,0x3d,0x8e,0xdc,0x1a,0x36,0x0a,0xd4,0xcb,0x0e, +0x71,0x53,0x7f,0x4c,0xa1,0x54,0xac,0xee,0x3c,0x86,0x4b,0xac, +0x92,0x25,0xa1,0x02,0xce,0x7d,0x62,0x7e,0xec,0x1e,0x53,0x53, +0x2d,0x45,0x3c,0xe0,0xdd,0xa9,0x14,0x92,0x37,0x28,0x23,0x4f, +0xa9,0x19,0x7a,0xb2,0x87,0xc7,0xe5,0x7b,0x34,0xeb,0x6c,0xb0, +0x6d,0xb8,0x9b,0x5f,0xed,0x1d,0x4e,0xe8,0x79,0xfb,0x5b,0x81, +0xc8,0x94,0x82,0x40,0x92,0x3c,0xba,0x60,0xf6,0x73,0xae,0x38, +0xea,0x78,0x13,0x79,0xaf,0xe3,0xfb,0x3f,0x39,0xc3,0xe0,0x4e, +0x46,0x6e,0x09,0x89,0xfd,0xb0,0x2e,0xd4,0x4d,0x05,0x38,0x69, +0x27,0x4d,0x7e,0xb4,0x4d,0xc6,0x6f,0xea,0x73,0x46,0xa3,0x77, +0xb3,0xfd,0xeb,0xe3,0xf9,0x6c,0x78,0x32,0xb7,0x93,0xf2,0xd5, +0x6c,0x7a,0x29,0x53,0xfc,0x72,0x7a,0x69,0xdc,0x98,0x2a,0xca, +0xe7,0xec,0xa3,0x97,0x4f,0x4d,0x38,0x3e,0xff,0x3b,0x11,0x70, +0xf9,0x7a,0x23,0x8d,0x21,0x9e,0x1f,0xe1,0x18,0xb6,0x48,0xc2, +0x31,0xc9,0xb3,0xcc,0xe7,0xe1,0xab,0x9d,0xb7,0xaa,0x04,0x25, +0x74,0x75,0x41,0x74,0x6e,0x78,0x09,0xf0,0x2d,0xb1,0xfe,0x69, +0x34,0xf9,0x3d,0x18,0xe3,0x8f,0x2c,0xc1,0xa9,0x69,0x20,0xe6, +0xc5,0x82,0x31,0x51,0xbc,0x11,0x99,0xa4,0x58,0x8c,0x9f,0x1b, +0xa9,0xcc,0x6b,0x4c,0xc2,0x27,0x8c,0xc0,0x8a,0xc5,0xb8,0x2e, +0x8a,0x9c,0xe9,0x1e,0x52,0x9a,0x10,0xaa,0x02,0xc6,0xcc,0xa0, +0xf4,0xdc,0xf0,0x4f,0xa9,0x4c,0x11,0x0d,0x1e,0x7c,0xcb,0x7a, +0xb5,0x41,0x09,0x13,0x7a,0x01,0xbe,0x56,0x89,0x88,0xb3,0xee, +0x5b,0xfc,0x37,0xc8,0x67,0x21,0x3d,0xf2,0x11,0xf4,0x4d,0x3a, +0x72,0xb8,0x55,0xe5,0xf3,0x75,0xb7,0x95,0xba,0xf0,0x8a,0x03, +0xd4,0x56,0xa8,0xc1,0xe4,0xe2,0x1a,0x2d,0xaa,0xb1,0x57,0x60, +0xe5,0x03,0xc0,0x2f,0x47,0x4a,0xb7,0x06,0x67,0x63,0x22,0xaa, +0x64,0xcb,0xfb,0x0e,0xdf,0xac,0xd8,0x36,0xa3,0x29,0x46,0xab, +0x23,0x5d,0x97,0x0d,0x0b,0x05,0x73,0x21,0xb7,0x4e,0x25,0x04, +0xda,0x6e,0x21,0xb8,0x90,0x0b,0x0f,0xc3,0xbf,0xbb,0xc5,0xe9, +0x29,0x17,0x80,0x69,0x1d,0x16,0xd8,0x8c,0x0a,0xe0,0xc5,0x23, +0xb4,0xcc,0xb4,0x91,0xd4,0x20,0x5d,0x14,0x8e,0x0d,0x45,0x88, +0xdc,0x5a,0xe8,0x00,0x97,0x5a,0x97,0xeb,0x0f,0xdb,0xf0,0xbc, +0x58,0xac,0x07,0x40,0xf3,0xe9,0x6b,0x98,0x5b,0x0e,0x34,0x5d, +0x47,0xf5,0xb0,0x4b,0xc2,0x12,0x18,0x61,0x9d,0xaa,0x6c,0x36, +0xed,0xd4,0xe5,0x57,0x4f,0xb8,0x0a,0x56,0x6d,0xd7,0x07,0x5e, +0x1f,0xaf,0xf1,0x2f,0x29,0x9c,0xe4,0x89,0xfd,0x21,0x7c,0x1a, +0xcf,0xc6,0xa3,0x9c,0x92,0xa0,0xd5,0xdf,0xe0,0x3b,0x9e,0x7b, +0x36,0x54,0x18,0x92,0x46,0x2b,0x2d,0x1f,0x6c,0x71,0x58,0xa5, +0x2e,0x41,0x12,0xfc,0xf3,0x8b,0x02,0x00,0xcb,0x3e,0xa5,0xc0, +0xc7,0x9c,0x80,0x24,0x77,0x61,0xf7,0x2d,0x65,0x5f,0x50,0x43, +0x86,0x3c,0xd9,0x73,0x2b,0x82,0xaf,0x21,0x53,0x2f,0xe6,0xd4, +0xe3,0x9c,0x10,0x8b,0x00,0xc8,0xa0,0x49,0x81,0x66,0x62,0xb1, +0x40,0xee,0x22,0x59,0x1b,0x05,0x8d,0x43,0x3d,0xc7,0x7c,0x1e, +0xc1,0x71,0xba,0x84,0x35,0xe4,0x51,0xeb,0x11,0x94,0x09,0x61, +0x82,0x42,0x44,0x42,0x3f,0x04,0x18,0xe4,0xaa,0x5e,0x31,0x83, +0xf0,0x46,0x38,0x61,0x51,0xdf,0x20,0xe8,0x22,0xfd,0x4d,0x51, +0xce,0x1b,0xa7,0xc5,0xf8,0x62,0x64,0x0d,0x4c,0x40,0xde,0xa3, +0x9d,0xed,0x78,0xe6,0x0c,0xd1,0x31,0xdb,0xc7,0x20,0xc2,0x35, +0x18,0xb6,0xdc,0x04,0x99,0x7f,0x23,0xb0,0x79,0x02,0x5c,0x21, +0xe1,0xc0,0xf2,0x58,0x6c,0xbd,0x52,0x8c,0xf8,0x45,0x62,0x4f, +0xdc,0xba,0x5c,0xbc,0xb3,0xed,0xca,0x73,0xfe,0x92,0xd9,0x4e, +0x9f,0x0a,0x2b,0x71,0xfd,0x14,0x3f,0x62,0xae,0xfd,0x1a,0x14, +0xe6,0x8c,0x4a,0x8a,0xbc,0x9e,0x0b,0xbf,0x3a,0x05,0x86,0x8d, +0x72,0xa6,0x60,0x50,0x17,0x2c,0xbb,0xea,0x4a,0x95,0x3b,0xe6, +0x61,0xbc,0xe8,0x15,0x91,0x6f,0xf4,0x73,0x35,0xfc,0x93,0x7b, +0xf1,0x5a,0xea,0x52,0xf7,0x7f,0x65,0xc5,0x91,0x6a,0x61,0x14, +0x5f,0xcf,0x4f,0xe0,0x2f,0x72,0x49,0xfe,0x22,0xa5,0xea,0x97, +0x1d,0x05,0xb1,0x98,0x88,0x4c,0x73,0x29,0x95,0x51,0x64,0xfc, +0xa5,0x7c,0xa0,0x94,0xe3,0x4a,0x25,0x44,0x93,0x9b,0xfa,0xd4, +0x72,0xe1,0xdd,0x3b,0x3e,0xfb,0xb6,0xe0,0x69,0xe5,0xc2,0x9d, +0xef,0x73,0x3b,0xcf,0x79,0x1f,0xf3,0x6a,0x3a,0x7b,0x31,0x1d, +0xcf,0x4e,0x9c,0x00,0x7c,0x82,0x6f,0xe3,0x11,0x9f,0x46,0xe5, +0xcd,0x1a,0x38,0xde,0xb1,0xe4,0xce,0x05,0x1c,0xf6,0x74,0x50, +0x6a,0x63,0xc3,0x1d,0x8a,0x05,0xe9,0x38,0xf4,0xb9,0x6c,0x90, +0xbc,0x93,0xb7,0x7b,0x4a,0x42,0x09,0x8b,0x64,0x5e,0x9a,0x09, +0x33,0x52,0x26,0xc3,0x7f,0x17,0xae,0x49,0xaf,0x27,0x7b,0xb4, +0xde,0xa1,0x18,0xc8,0x0b,0xa2,0xd7,0xc7,0x8b,0x5d,0x71,0x03, +0xff,0xb4,0x7e,0x3a,0x78,0xa1,0x80,0x9a,0x1d,0xbc,0x7e,0xde, +0x82,0xd6,0x83,0xf4,0x5d,0x0e,0x50,0x1d,0xe1,0xfd,0x27,0x00, +0x53,0x09,0x75,0xf0,0xf7,0x66,0x3c,0xbc,0xf2,0xe9,0xbd,0xc7, +0x4f,0x9f,0x66,0x8f,0x9f,0x7e,0xad,0x17,0xa4,0x10,0xe2,0x46, +0x9d,0x52,0x63,0xc2,0x57,0x4f,0x36,0x36,0xb0,0xf1,0xe8,0xb9, +0xd3,0x6e,0x83,0x28,0xed,0x72,0xda,0x6d,0xb5,0x06,0x05,0xbb, +0x9b,0x4b,0xd9,0xad,0xcb,0x7b,0x17,0x84,0x8e,0xe1,0xc6,0xc6, +0xe6,0x36,0x14,0x85,0xc7,0xab,0xd3,0x96,0xcd,0x41,0xfb,0x28, +0x79,0xc4,0x9c,0x61,0x7f,0xef,0xdd,0xdb,0x83,0x1f,0x06,0xcf, +0xa0,0x1e,0xf5,0xba,0xd3,0xe9,0xf4,0xe8,0x29,0xa3,0xc4,0xdd, +0xe7,0x07,0x2f,0x07,0xcf,0x3a,0x02,0x42,0x6f,0x3b,0xba,0x65, +0x29,0xf9,0x9f,0x2f,0x9f,0xbf,0x1f,0x18,0x85,0x24,0xed,0x21, +0x24,0x63,0xf8,0xe1,0xdd,0x4f,0xef,0x7d,0x25,0xf4,0xb6,0xb3, +0xfd,0xb8,0x87,0x0f,0x0c,0xb0,0xf7,0xfa,0xed,0x4f,0x58,0x89, +0xa3,0x83,0xdf,0x77,0xbe,0xfe,0xb6,0xc7,0x8f,0x0c,0xb6,0xff, +0xf2,0xc5,0xbb,0xb7,0xbb,0x1e,0x4c,0xde,0x11,0x8c,0x1f,0x2d, +0xb6,0x37,0x6f,0x5e,0xc7,0xb0,0x3a,0x71,0xe7,0xdb,0x6f,0x11, +0xb1,0x4b,0xc8,0x36,0x3b,0x86,0x1b,0xea,0xc8,0x36,0xcf,0xee, +0xf0,0xf3,0xbb,0x53,0xec,0x27,0x68,0x32,0xfc,0xb6,0x1d,0x9b, +0xb1,0x58,0xd8,0xa7,0x1d,0xfc,0xc0,0x54,0x37,0x29,0x25,0x98, +0xb0,0xc9,0x73,0xfb,0xa0,0xc5,0x12,0xbe,0xc0,0x0d,0xdc,0x17, +0x23,0x0c,0x81,0xa0,0xf1,0xea,0x62,0x78,0x56,0xfa,0xad,0x2b, +0xfd,0x22,0x96,0xfc,0x86,0x7c,0x05,0x67,0xeb,0x1d,0x73,0x3d, +0xb9,0x2e,0xc7,0xbc,0x89,0x2f,0xb3,0xfe,0x40,0xde,0x89,0x79, +0xe3,0xab,0xad,0x25,0xdb,0xdc,0xe6,0x90,0x4a,0x6f,0x40,0x88, +0xc7,0x01,0x92,0xb5,0xc9,0xfb,0x01,0x03,0x02,0x9e,0x82,0x4d, +0x80,0xa9,0xeb,0x32,0xb6,0x87,0xe7,0x94,0x57,0xc4,0x71,0xa8, +0xaa,0x72,0x3c,0x13,0x4b,0x61,0x0c,0x58,0x0c,0x49,0x4b,0x3d, +0x82,0xff,0x81,0x19,0x8d,0xcb,0xf8,0x28,0x12,0x76,0x7b,0x9c, +0x85,0xc2,0x9f,0x7b,0xc9,0xe5,0xf4,0x9b,0xe4,0xe2,0x33,0x91, +0x8b,0x51,0x90,0x08,0x5b,0xe9,0x19,0x46,0xc0,0xe2,0x24,0xfa, +0x62,0xf7,0xa6,0xc9,0x75,0x89,0xee,0x83,0x62,0xb0,0x57,0xc2, +0xb8,0x25,0x35,0xfa,0x10,0xec,0x19,0x0e,0x5a,0x1b,0x92,0xa8, +0x49,0x6f,0xbb,0x39,0x13,0xb4,0xa2,0xce,0xd0,0x3d,0x61,0x45, +0x31,0xea,0x75,0x41,0x52,0x23,0x5c,0x38,0x85,0x85,0x3e,0xee, +0x86,0x67,0xf4,0x46,0xbb,0x42,0xee,0x41,0x05,0x6b,0xb2,0x99, +0xa4,0x19,0x80,0xa8,0xad,0xd9,0x74,0x38,0x42,0x64,0x3e,0x22, +0x70,0x19,0x9e,0x5f,0x97,0x2d,0x0c,0x6d,0x82,0xd6,0xbe,0x2e, +0x8e,0x08,0xfb,0xce,0x5b,0x2c,0x1a,0x61,0x02,0x29,0x7a,0x2d, +0x61,0x69,0x04,0x4d,0x4b,0x96,0xe0,0x8f,0xb2,0x3c,0x31,0xd7, +0x13,0x4d,0x8e,0x0b,0x30,0xbc,0x0a,0x1c,0xb7,0x82,0x90,0x45, +0xe1,0x2a,0x0a,0x6f,0xff,0x8a,0x9c,0xeb,0x57,0x8e,0x7c,0x32, +0xc1,0xc3,0x1d,0x8a,0x10,0x4c,0x76,0xce,0x18,0xb8,0xc2,0xdb, +0xca,0xf2,0x7d,0x46,0x85,0x7c,0x80,0x11,0x09,0x4b,0x0e,0x60, +0x9f,0xb2,0x02,0xfb,0xb7,0xeb,0x62,0x06,0x4d,0xd7,0xda,0x42, +0xb0,0xad,0xa4,0xf9,0x51,0x29,0xa7,0x9d,0x6f,0x1a,0x85,0x61, +0x49,0x77,0x30,0x91,0x0c,0xbb,0x28,0x87,0x0b,0x31,0x3b,0x7d, +0x93,0xbd,0x09,0x99,0x06,0x21,0x11,0x98,0x93,0x03,0x71,0x0d, +0x65,0x2c,0x44,0x68,0xbb,0x78,0x75,0x4d,0xbe,0x1a,0x6d,0xc2, +0x8a,0x67,0xd8,0xbf,0x62,0x79,0xc4,0xbb,0x66,0xfa,0xba,0xbc, +0x76,0x60,0xa0,0x84,0x6d,0xe3,0x23,0x43,0xc7,0x9b,0x5f,0x73, +0x7a,0xb1,0x9b,0x05,0x6c,0x9c,0x55,0x05,0x9b,0x1d,0x74,0x05, +0x4d,0x00,0xf0,0xa7,0x37,0xa1,0x83,0x1c,0x87,0x89,0xa6,0x79, +0xf7,0xd7,0x9d,0x76,0x37,0x24,0x9f,0xd1,0x5b,0x87,0x65,0xbf, +0xda,0x43,0x70,0x28,0x11,0x7e,0x15,0x9e,0x12,0x93,0x97,0x68, +0xc2,0x2b,0xa6,0xe6,0xf9,0xaf,0x1b,0x1b,0xe1,0x4e,0x88,0x3b, +0x8e,0xfa,0x70,0xbd,0x9d,0x02,0xc0,0x66,0x47,0x0c,0xc8,0x7e, +0xdd,0xdc,0x5c,0x82,0x60,0xb2,0xac,0x6f,0x64,0x3f,0x42,0x38, +0x6c,0x35,0x4f,0x62,0xd4,0xd8,0xf3,0x1c,0x0b,0xb7,0x6f,0x2c, +0x58,0x4b,0xf8,0x92,0xad,0xc3,0x3e,0x9f,0x62,0x6e,0xa5,0x3d, +0x4e,0x77,0xdb,0x86,0x5f,0x0e,0xfb,0x8b,0xc3,0xc1,0x03,0x8c, +0x61,0x08,0x2d,0x10,0x65,0x1e,0x1e,0x72,0x7a,0x28,0xf6,0x72, +0xc5,0xaf,0x42,0xe9,0x52,0x6d,0xe3,0xdc,0x96,0x4d,0x24,0x42, +0x21,0xe2,0x34,0xa2,0xd7,0x09,0x65,0xb5,0xf7,0xf9,0x03,0x4b, +0x7d,0x77,0x6d,0x9b,0x71,0x50,0x79,0xbf,0x87,0xb1,0xb9,0x83, +0xde,0xed,0xd9,0xd9,0x8a,0x76,0xb3,0x00,0xd5,0x5b,0xac,0xb8, +0x65,0xbf,0x51,0x0a,0x9c,0x24,0xd1,0x7b,0x48,0x4f,0x1f,0x67, +0x37,0x73,0x8b,0x49,0x99,0x5f,0x5a,0x3a,0x7a,0x36,0xcf,0x49, +0xb2,0x22,0xbb,0xca,0xcd,0x76,0x75,0xe7,0x40,0xf4,0x3c,0xbe, +0xd1,0x19,0xd0,0x6a,0x75,0x6c,0x39,0xdb,0xd1,0x97,0x2d,0xbb, +0xd6,0xa4,0x3d,0x69,0xe4,0x7c,0xfc,0xc7,0xd5,0x70,0x22,0x6c, +0xde,0x8a,0xff,0x97,0xa2,0x42,0x4e,0x05,0x81,0x6f,0x21,0x7e, +0x47,0xd6,0xb7,0x22,0x27,0x5f,0xdd,0xed,0x2b,0xb1,0xc1,0xca, +0x97,0x66,0xb6,0x4e,0xbb,0xf0,0xc4,0x12,0x62,0x1d,0x9d,0x58, +0x42,0xa9,0xe4,0x65,0x30,0xbe,0x99,0x02,0x4b,0x84,0xd2,0xaf, +0x7c,0x17,0xc7,0x23,0x1e,0x0b,0xb6,0x2e,0x02,0x38,0x81,0x58, +0x2c,0xe8,0x97,0x83,0x36,0xe7,0x5f,0x53,0x27,0x5e,0x4c,0xa1, +0x23,0xe2,0xb1,0xe0,0xfd,0x0b,0xe0,0x4d,0x27,0x0a,0x03,0x5c, +0x0f,0x47,0xf1,0x6f,0xa5,0x09,0xba,0xa9,0xb4,0xba,0x0c,0x77, +0x3b,0x7d,0x6a,0x4b,0x9a,0x5b,0xbe,0x07,0x5d,0x91,0xdd,0x41, +0x94,0x29,0x36,0x73,0x7f,0x89,0x8d,0x00,0x83,0x95,0x83,0x42, +0x31,0x92,0x59,0x0c,0x60,0xe1,0x43,0x86,0x39,0xfe,0xb5,0x5a, +0x7c,0xb6,0xf4,0xb0,0xa6,0xe5,0x94,0x25,0x66,0xe5,0xbb,0xf0, +0x3f,0x67,0xac,0x4d,0x46,0x78,0x54,0xfc,0xe0,0x7c,0x36,0x1e, +0xef,0x16,0x67,0xb0,0xbd,0x66,0xfb,0xee,0x7f,0xc2,0xff,0xc4, +0x1c,0xfc,0x7b,0xf8,0x9f,0x3c,0x9e,0xc1,0xff,0x6a,0x4a,0xbf, +0x9a,0x5e,0xcf,0xe2,0xc2,0xba,0xb4,0x2e,0x5e,0x57,0x7e,0xbf, +0xf8,0x43,0x17,0xdf,0x17,0xf0,0x7d,0xff,0x60,0x9f,0xea,0xa9, +0x7f,0x37,0x81,0xdf,0xca,0x27,0xec,0xed,0xed,0x49,0xa9,0x3d, +0xff,0x38,0x1a,0xb9,0x07,0xf5,0x34,0xaa,0x41,0xfa,0x61,0x3a, +0x1b,0x31,0xa2,0xa1,0x00,0x3e,0x77,0x50,0xd5,0xc5,0xdb,0x4a, +0xa9,0x17,0x69,0xeb,0xe8,0x72,0x3c,0x2b,0x46,0xc5,0xf8,0x92, +0x7a,0x89,0x51,0xfc,0x57,0x5d,0x9b,0xdb,0xfb,0x9b,0x7b,0x42, +0xf0,0xcf,0x52,0xcf,0xcf,0x3f,0xaf,0x80,0xfe,0x73,0x3a,0x11, +0x7c,0x07,0x75,0x10,0xf6,0xb3,0x5d,0x5b,0xc9,0x83,0xea,0x0b, +0xd7,0x11,0xf2,0xf0,0xc3,0x0f,0xf2,0x70,0x7e,0x2e,0x0f,0x97, +0x97,0xf2,0x50,0x96,0xb6,0xf5,0x2c,0x3e,0xdb,0x5e,0xb6,0xac, +0x2d,0x6a,0x4b,0xda,0x82,0xf2,0xfb,0xc9,0xfe,0xda,0x87,0x0f, +0xf6,0xd7,0x3e,0x8c,0xe5,0xf7,0x65,0x7d,0x9f,0xbe,0x9b,0x1d, +0x7c,0x9a,0x4a,0x8f,0x8a,0xfd,0x8a,0xbf,0x05,0xa1,0x6c,0x53, +0x66,0x38,0x11,0xae,0x5e,0xb2,0x95,0x84,0x35,0x4a,0x13,0xce, +0x40,0x63,0xdf,0x8b,0x91,0x64,0xac,0x83,0x41,0xb1,0x30,0xe4, +0xb9,0x3e,0x44,0x9b,0x4b,0xf3,0xee,0xb1,0x76,0x16,0xf5,0xeb, +0x62,0x3e,0x6d,0xcd,0xd6,0xe8,0x60,0xfe,0xcf,0x93,0xf3,0xeb, +0xc9,0xc7,0xbc,0x51,0xed,0x97,0xd6,0xf8,0x8f,0xf1,0x89,0x05, +0x5e,0x2c,0xfa,0x83,0x14,0x0f,0x70,0xd1,0xd1,0x42,0x99,0x37, +0xa4,0x5c,0x13,0xaa,0x96,0x65,0x92,0x11,0x48,0xd9,0x17,0x98, +0x09,0x5b,0x76,0x28,0x06,0xf2,0x86,0x69,0x1b,0x8c,0x35,0x26, +0xda,0xef,0x66,0xe3,0x69,0xfb,0x11,0xa1,0xe9,0x83,0x58,0xd3, +0x64,0xe5,0x04,0xbf,0x6f,0xbb,0xb5,0x2c,0x69,0xa2,0xe6,0x96, +0x53,0xdb,0x83,0xde,0xa6,0x14,0xce,0xe4,0x37,0x38,0x4c,0xc0, +0x4a,0x0f,0xd8,0x15,0x21,0x7e,0xa5,0xe6,0x1e,0xec,0x51,0x2d, +0xe0,0x21,0x68,0x5b,0x80,0xfb,0xb2,0x39,0xdb,0x6c,0xdb,0x61, +0x5e,0xcb,0x5b,0xfc,0x74,0x4b,0x32,0xf6,0x1d,0x53,0xf0,0xb6, +0xa4,0x11,0x20,0x91,0x2d,0xb1,0xe8,0x53,0x99,0x75,0x83,0x70, +0xd4,0x65,0xe9,0xa8,0x76,0xea,0x0e,0xf3,0x5b,0x67,0x1c,0xeb, +0xd9,0x69,0xba,0x09,0x3e,0xc3,0xf5,0x0f,0x7b,0xb5,0x35,0x0f, +0x33,0xbf,0xa9,0x0c,0xf6,0x54,0x4c,0xb0,0xa6,0x64,0x57,0x4d, +0xa6,0xdb,0xbe,0x89,0x14,0x01,0xc1,0x27,0x05,0x1f,0xb4,0xbb, +0xeb,0x11,0x55,0x51,0x59,0x6a,0x46,0x76,0x9b,0xbd,0x1a,0x11, +0xce,0xe5,0xb0,0x66,0xd2,0x35,0x04,0x05,0x9a,0x0d,0xfd,0xb6, +0xf3,0xf4,0x6f,0xbd,0xce,0xb7,0xed,0x76,0xb6,0x3d,0x7e,0x9c, +0x76,0xff,0x57,0x80,0xcb,0x71,0x06,0xe1,0xd8,0x77,0x61,0xd6, +0x94,0x28,0xae,0x68,0xe9,0x2f,0xca,0x1f,0x2f,0x6f,0xef,0x29, +0x80,0xd8,0xab,0xc1,0xf5,0x43,0x85,0x1d,0x29,0xb6,0x14,0x12, +0x45,0xfa,0x92,0x95,0x44,0x5d,0x2a,0x2e,0x16,0x75,0x3d,0x2b, +0x51,0x56,0x96,0x2c,0x15,0xdb,0x0b,0x4b,0x8a,0x9a,0x64,0x65, +0xc9,0xfa,0xe5,0x2a,0xae,0xdd,0xeb,0x5b,0x04,0x11,0x9e,0xee, +0x36,0x92,0x76,0x0b,0x0d,0xb5,0x2a,0xdd,0xfc,0x5f,0xbe,0x51, +0x47,0xee,0xc6,0x3f,0x15,0x51,0x86,0xf3,0x35,0xe5,0xf4,0x12, +0x62,0x11,0xc0,0xfe,0xfc,0xa7,0x83,0x17,0x68,0x4f,0x61,0x53, +0xe6,0x7f,0x5e,0xe6,0xab,0x19,0x5e,0xf5,0x0b,0xef,0xe6,0xe4, +0xa3,0xbb,0x96,0xd7,0x98,0xdf,0x13,0xdf,0xc8,0x99,0x3b,0xbb, +0x78,0x88,0x1d,0xb9,0xd1,0xe6,0x96,0x27,0x96,0x17,0xc2,0xb5, +0x4b,0xc9,0x13,0x22,0x5b,0xd4,0xe2,0xc2,0xf8,0x03,0xd1,0xf4, +0xfa,0xe4,0xd8,0xd7,0x27,0xb4,0x39,0x73,0xcd,0xf1,0xa9,0x4f, +0x65,0xe5,0x84,0x40,0xaf,0x0a,0xd8,0x89,0xd8,0x32,0x2f,0x08, +0xb2,0xa1,0x39,0x63,0x41,0x9c,0xd1,0x9c,0x5c,0xcf,0x66,0x20, +0xaf,0x93,0xfb,0x0e,0x54,0x4c,0x1e,0x4c,0x7f,0x2a,0x31,0x24, +0xef,0x1f,0x38,0x8d,0xcd,0x27,0xb2,0xa0,0xe2,0xed,0x3c,0x9e, +0x09,0xc0,0x04,0xa7,0x5f,0x26,0x4d,0x14,0xee,0xeb,0xae,0xa7, +0x79,0x53,0xac,0x50,0xe6,0xea,0x99,0xb7,0xdd,0xd6,0x74,0xc3, +0xd1,0x0e,0xfb,0x50,0xd2,0x38,0x39,0xce,0xcc,0xac,0xa8,0x9a, +0xcc,0xbc,0x0f,0x9a,0x43,0x88,0x0b,0x0c,0xff,0xb5,0xda,0x04, +0x6d,0x96,0xed,0x99,0xf5,0xe3,0x1e,0x8d,0x37,0x1c,0xaf,0x90, +0x6a,0x3a,0x6d,0x62,0x27,0x49,0xe7,0x5b,0x0a,0xa4,0x9f,0x25, +0xdb,0x6d,0x7e,0xc0,0x7f,0x71,0x85,0xc4,0x3e,0x7a,0x4e,0xf9, +0x8f,0xdf,0x4d,0xaa,0xde,0x34,0x8b,0x60,0x96,0x46,0xf5,0x8c, +0x70,0xee,0x4f,0xad,0xef,0xbf,0xe7,0x5b,0x9b,0xf8,0xfc,0xc1, +0x3f,0xbe,0xec,0x61,0x93,0xe6,0x8e,0x55,0x62,0xf7,0x7c,0x40, +0x93,0x07,0xfb,0x5d,0x0d,0x2c,0x9a,0x1a,0x2a,0xd4,0x81,0x9f, +0x97,0xe6,0x09,0x0c,0xac,0xcc,0x6d,0xf4,0x57,0x32,0x27,0xdb, +0x43,0xb9,0xad,0x6a,0xc4,0x1f,0xff,0x81,0x93,0x01,0xef,0x88, +0x77,0x2d,0x99,0x05,0x18,0xfb,0xd6,0x81,0x17,0x6c,0x9d,0x26, +0xed,0x4e,0x8e,0x10,0x53,0x6b,0x34,0xfd,0x94,0xb1,0x09,0x4b, +0xae,0xc0,0x3e,0x21,0x18,0x52,0xe6,0x6a,0xd9,0xd8,0x90,0x8a, +0x9f,0x85,0x85,0x39,0xbd,0xd9,0x34,0x77,0x7f,0xf0,0xd9,0x19, +0x53,0xef,0x06,0x59,0x80,0x29,0x7a,0xfd,0x94,0xfa,0xe1,0x33, +0xb4,0x4c,0x1e,0xaa,0x20,0xab,0x1c,0x53,0xb3,0x16,0x61,0x9e, +0x7b,0x4d,0xab,0x10,0x30,0xa6,0xdc,0xd0,0xcf,0xeb,0x47,0x81, +0x1a,0xc7,0x9c,0x12,0x8f,0x81,0x2a,0xd6,0x9d,0xe8,0x5c,0x81, +0xd0,0xa7,0x6a,0x3a,0xd7,0x2a,0xa8,0x31,0xc2,0x14,0xcd,0x4c, +0xda,0xcc,0xda,0xd3,0x0a,0x3f,0x33,0xdb,0xd5,0x8a,0x74,0x6b, +0x88,0x80,0x80,0x08,0xe4,0xb4,0x83,0xb5,0xfa,0x1a,0x86,0x97, +0x7a,0x05,0xc2,0xdb,0x5d,0xbe,0xe3,0xf2,0x78,0xa7,0xa8,0x4e, +0x3d,0xd8,0xf4,0x37,0x9b,0x45,0xaa,0x13,0x98,0xd7,0xe0,0x83, +0x6e,0x9a,0x82,0x3d,0x0a,0x74,0xbf,0x11,0x8d,0x43,0x7d,0x89, +0x2a,0xfa,0x1e,0x1e,0x6e,0x14,0xbd,0x4e,0xd6,0xce,0x82,0x5a, +0xb9,0x08,0x2d,0x99,0x4d,0x59,0x76,0x1a,0x6a,0x11,0x58,0x2c, +0xda,0xe9,0xd6,0xd3,0xb6,0x70,0x4a,0xbb,0x44,0xae,0x82,0xfc, +0x0a,0x21,0xfd,0xaa,0xd4,0x08,0xd7,0x97,0x9e,0x6a,0xef,0x0c, +0x9f,0xf1,0x21,0xd5,0xfe,0x9e,0x56,0x72,0x57,0xb9,0x99,0xa7, +0xb9,0x6b,0x74,0x20,0xdb,0x75,0xd5,0x2e,0x16,0xf1,0x39,0x6f, +0xbd,0x05,0x9a,0x13,0x49,0x54,0xcf,0xe5,0xfd,0x3a,0x3b,0xb4, +0xf8,0xe8,0x9a,0xa5,0xc9,0x4a,0x2a,0xce,0xaa,0x3a,0x73,0xb2, +0x6a,0x71,0x5a,0x50,0x2b,0xc9,0x6c,0x49,0x52,0x03,0xed,0xac, +0x4c,0x06,0xa6,0x7e,0xb9,0xd1,0xc6,0x16,0x2b,0x96,0x83,0xaa, +0xe7,0x27,0xef,0xa8,0x49,0xf7,0x11,0x34,0xc0,0x27,0x19,0xb4, +0xaf,0xa0,0x4b,0x98,0x1d,0x1b,0x9f,0x68,0x07,0xbb,0x4f,0xe1, +0xb1,0x3d,0xc8,0x6c,0xc9,0x6a,0xb1,0xa8,0x8c,0x14,0x08,0x75, +0x89,0xbb,0xf2,0x61,0x2c,0x61,0x3c,0x77,0xba,0xa0,0xf8,0x8c, +0x67,0x88,0x4e,0x7e,0xd5,0xfc,0xa6,0xd3,0x0f,0xbc,0x9b,0xcc, +0x2b,0x2e,0xb1,0x51,0x6e,0x38,0x33,0x67,0x3d,0x0b,0x6f,0x5c, +0xca,0x8f,0x05,0x19,0x9f,0xde,0xcd,0xe3,0x79,0x83,0x96,0x27, +0x49,0xd3,0x8d,0x10,0x49,0x63,0xc3,0x2c,0xb9,0x90,0x64,0xd5, +0xcc,0xf3,0xe9,0x7c,0x78,0xf1,0xa3,0xaf,0x56,0x80,0xd8,0xc8, +0x84,0x49,0x08,0xb5,0x70,0x16,0xeb,0x29,0x2f,0x15,0xab,0x94, +0xa2,0xb8,0x41,0x24,0x7e,0x51,0x3c,0x9b,0xeb,0xa3,0x13,0x9a, +0xf5,0x4a,0x9e,0x21,0xdf,0x29,0xea,0xb3,0xf3,0xc6,0x3d,0xb4, +0x3f,0x2b,0xb7,0xa2,0x16,0x0b,0x5a,0xfb,0x73,0x8b,0xd9,0xcf, +0x75,0x62,0x93,0xbc,0x5b,0x93,0x04,0x55,0x08,0xa3,0xe3,0x73, +0x21,0xab,0xf4,0x6e,0xa3,0xce,0xdb,0xf5,0x95,0x3a,0x82,0xe3, +0xb3,0x76,0x81,0x76,0x6d,0x6e,0xab,0x22,0x0d,0xfb,0x2d,0xf5, +0x34,0xd5,0x8b,0x33,0xee,0xa9,0xef,0x88,0x66,0x5e,0x07,0x5b, +0xab,0x1f,0x66,0x11,0xaf,0xa7,0x6b,0xea,0x55,0x47,0x5a,0x27, +0xab,0x7c,0x91,0x68,0xe7,0xe8,0x93,0x78,0xb7,0x6b,0x6e,0xdf, +0x45,0xeb,0x51,0x6a,0xa7,0xb0,0x43,0x6b,0x4f,0xdc,0xd6,0x83, +0xee,0xb8,0x47,0xa5,0x5d,0x05,0x13,0x1c,0xc6,0xe5,0x7a,0xf8, +0x6e,0xd6,0xb7,0x93,0x09,0x46,0xf5,0x9d,0x1d,0x27,0x66,0x98, +0x7a,0x5b,0xe7,0x4b,0xd8,0x03,0xec,0xce,0xb6,0x5a,0x8b,0x87, +0x76,0x91,0xe9,0x6c,0x87,0xe5,0xe8,0xba,0xed,0xc6,0x46,0x07, +0x17,0xa8,0x08,0xb8,0x5a,0x3c,0x97,0xb5,0xbb,0xca,0x03,0x4d, +0xad,0x01,0xb3,0x3e,0x7c,0x0b,0xb4,0x3e,0xfe,0x18,0xb0,0xd4, +0xc7,0x1c,0x8d,0xc3,0x7e,0xba,0xc0,0x9f,0x01,0xfc,0xf4,0x1b, +0xfd,0x5f,0x0e,0x07,0x87,0xfd,0xc1,0xa3,0xf4,0x70,0x80,0xa9, +0xad,0x74,0xeb,0x2c,0xba,0xcc,0x03,0x2c,0xe5,0xaa,0x63,0xae, +0xb6,0xcd,0xd5,0x63,0x73,0xf5,0xc4,0x21,0xbd,0xea,0x2c,0x16, +0x57,0xdb,0xf0,0xef,0x31,0xfc,0x7b,0xb2,0x4c,0xf5,0x99,0x8d, +0x52,0x45,0xd5,0x52,0xd1,0xdf,0x3c,0x04,0x4a,0x7e,0x79,0xf0, +0xa8,0xd9,0x6b,0x35,0xd2,0x45,0xff,0x70,0x70,0xb3,0x1c,0xe0, +0xf9,0xcb,0xe1,0xe1,0x83,0x8d,0xf8,0x0c,0xa6,0xc2,0x37,0xab, +0x5c,0x1f,0xa5,0x33,0x6e,0x2b,0x73,0x3c,0x2e,0xe5,0xf8,0xc0, +0xd0,0x5d,0xdf,0x83,0xe9,0x77,0x63,0x34,0x08,0xb2,0x9b,0x94, +0x7d,0x4c,0xb4,0x16,0x4b,0x8e,0x51,0xd9,0x69,0x13,0xad,0x18, +0xf1,0x31,0xb2,0xde,0x3f,0xaa,0x2d,0x69,0x7b,0x0b,0xfa,0x4c, +0x6e,0x48,0xda,0x43,0x93,0xe2,0x59,0x8c,0x9c,0x85,0x19,0x45, +0x45,0xde,0x36,0x9e,0xf0,0xdc,0x3b,0x8f,0xb1,0xdd,0xbd,0xc2, +0x26,0xc0,0x97,0x49,0x55,0x79,0xa8,0xc7,0x7f,0x0e,0x32,0xcc, +0xdb,0x96,0x1c,0x8d,0xc2,0x9e,0xa8,0xa8,0x34,0x1c,0x94,0x8a, +0xce,0x66,0xae,0xab,0x89,0xa7,0x9e,0x09,0x41,0x3b,0xed,0x47, +0x11,0x74,0xcd,0xe1,0xb8,0x89,0x40,0x38,0x00,0xbe,0x46,0x64, +0xe4,0x16,0xaa,0xea,0xc3,0xc5,0x42,0xbd,0xec,0x68,0x60,0x72, +0x87,0xe1,0xf3,0x42,0x44,0x7e,0x40,0xa8,0xcf,0x48,0xd3,0xae, +0xb4,0xf7,0x49,0x3c,0x6c,0x16,0x0b,0x05,0x76,0xdb,0x48,0x8c, +0x76,0xc2,0xc2,0xe0,0xfd,0x8a,0x4a,0x93,0x27,0x2f,0xca,0x29, +0xad,0x52,0xc1,0x7a,0x84,0xe3,0x4f,0x2e,0xca,0xf1,0x78,0x79, +0xd2,0x2d,0x76,0xd8,0xdf,0x9c,0xcd,0xc1,0x43,0x3a,0x27,0x12, +0x9c,0x22,0x1a,0xac,0x1d,0x96,0xc2,0xcd,0xce,0xa0,0x29,0x20, +0x4f,0x07,0x8b,0x45,0xb2,0x96,0x88,0x9a,0x62,0x69,0x87,0xde, +0x13,0x15,0xab,0xbb,0x9c,0x92,0x86,0x9d,0x9d,0x94,0x07,0x34, +0x28,0xec,0xcd,0x5c,0xc3,0xb5,0x07,0x82,0xef,0x0e,0xfd,0xae, +0x62,0x5d,0x80,0x21,0x59,0xfb,0x39,0x49,0x6f,0x1d,0x74,0x96, +0x5f,0x51,0x10,0xf2,0x9a,0x59,0x24,0x68,0xeb,0x9b,0x9c,0x38, +0x73,0xd8,0xe2,0x24,0x0a,0x44,0xcd,0x0d,0x8b,0xf9,0xb0,0xbc, +0x7a,0x3b,0x9e,0xff,0xbd,0x9c,0x4e,0x54,0xbb,0x8b,0x3a,0x87, +0xcb,0xe4,0xde,0x60,0xaf,0x57,0xa5,0x23,0x13,0x44,0x35,0x59, +0x8d,0xa6,0xe4,0xa1,0x16,0x3a,0x4b,0x98,0x60,0x6f,0x23,0x48, +0xd8,0x7b,0xab,0x07,0x4a,0x16,0x9a,0x34,0xf7,0x3c,0xe7,0xe7, +0x6d,0x8d,0x3d,0x77,0x5f,0xc5,0xff,0x11,0x81,0x32,0x5e,0xae, +0x25,0x50,0x82,0x83,0x58,0x6f,0x40,0x11,0x6d,0xf5,0x9b,0x8d, +0x3a,0x2d,0x5b,0x6c,0xe1,0x6c,0xbf,0xaa,0xf1,0xd9,0x5c,0x9a, +0x91,0x39,0x37,0x7b,0xa6,0x34,0x97,0x62,0xb4,0x4b,0x5b,0x4d, +0x57,0x34,0x82,0xb0,0x52,0x78,0xe7,0xdb,0x6f,0xda,0x3b,0x9f, +0x37,0x36,0x68,0xcf,0x58,0x2a,0x21,0xfa,0x33,0x7f,0x6f,0x58, +0x97,0xdb,0xb4,0xd6,0xd5,0x60,0x8d,0xf3,0x22,0x47,0xba,0xf6, +0xca,0xd5,0xaa,0x1a,0xb5,0xc0,0x5f,0xa9,0x34,0xd0,0x72,0xf0, +0x19,0x80,0x35,0x0b,0x61,0x27,0xfb,0xb5,0xbd,0xcd,0x73,0x0c, +0xad,0x99,0x44,0x0f,0x89,0xa0,0x3c,0xca,0x6c,0xe9,0x96,0xa8, +0x23,0x42,0xd5,0x7c,0xd5,0x67,0x3f,0xa5,0x2b,0xfb,0x29,0x9e, +0x27,0x8c,0xcb,0xa9,0x4e,0x98,0xb0,0x4e,0xdb,0x7d,0x63,0x11, +0x9a,0xea,0x92,0x08,0x5b,0xcc,0x61,0xef,0x85,0x53,0xf6,0xf9, +0xd9,0x54,0x66,0x95,0x91,0x3b,0x1b,0x12,0x4e,0x65,0xff,0xfa, +0xf4,0xb4,0xf8,0x83,0x62,0x93,0x02,0x96,0xb1,0x9c,0x43,0xeb, +0xb3,0xe5,0xd9,0xf8,0x82,0x3c,0xd3,0x90,0x71,0x16,0x17,0x46, +0x7d,0xcd,0xfa,0x7a,0x88,0x41,0xb0,0x5b,0x44,0x81,0x10,0xa0, +0x10,0x04,0x77,0x29,0x42,0x0c,0x5c,0x35,0xfb,0x07,0xe7,0x2b, +0x18,0x33,0xba,0xb7,0xe2,0xcc,0xdd,0x75,0xd9,0x74,0xab,0x33, +0x7e,0x9c,0xba,0x63,0x21,0x86,0x94,0x3c,0xda,0xc3,0xf3,0xcd, +0x08,0x4e,0x17,0x28,0x4a,0xa7,0xfb,0x1b,0x9c,0x4c,0x20,0x5b, +0xdb,0x4f,0xec,0xb5,0x2c,0x4e,0x45,0x80,0xad,0xc7,0x4f,0xbf, +0x4e,0xd9,0x35,0xed,0x93,0xaf,0x77,0xdc,0xad,0x87,0x7e,0x52, +0x26,0xf6,0x2a,0x14,0xba,0xb4,0x47,0xab,0x2f,0x46,0x8d,0x79, +0x97,0x09,0xa4,0x01,0xb8,0x4e,0xba,0x4c,0x2c,0x89,0x02,0x4f, +0x75,0x62,0xd6,0x39,0x42,0x6f,0x6f,0xef,0xf8,0x84,0xf3,0x84, +0x89,0x16,0x48,0xbe,0xc8,0xd1,0x4f,0x46,0x04,0xf8,0xf5,0x8e, +0x4f,0x18,0x25,0xf4,0x15,0x5c,0x9b,0x4b,0xde,0x43,0xb8,0xc7, +0x90,0xe2,0x12,0xf6,0x12,0xa3,0x3f,0xa9,0x9d,0x0a,0x66,0xfa, +0x5a,0x84,0xf8,0x8c,0x45,0xe0,0xe7,0x73,0xc2,0x2d,0x10,0xba, +0x30,0xdd,0x1e,0xe4,0x61,0x07,0x51,0xea,0xe3,0x41,0xae,0xfb, +0x61,0xa7,0xcd,0xc9,0x4f,0x06,0x34,0xcc,0x4d,0x65,0xd8,0xc9, +0xcc,0x04,0x61,0x86,0x7d,0xb6,0xba,0x61,0x81,0x93,0x81,0x35, +0x4e,0x6c,0x17,0x82,0xb1,0xb2,0x48,0x6f,0x85,0x53,0x2f,0x7a, +0x15,0xcd,0x14,0x9f,0xc1,0x8d,0x7e,0xbd,0x2e,0xe7,0xe3,0x91, +0x88,0x76,0xe4,0x1d,0xb2,0x06,0x78,0x33,0x42,0x88,0x6d,0x70, +0x30,0x75,0xef,0xf5,0x65,0xf8,0xb6,0xcc,0xe7,0x86,0x9b,0x54, +0x51,0xa9,0x9d,0x31,0xfa,0xf9,0x6f,0x44,0xa9,0x9b,0xf9,0x37, +0x29,0xd2,0xb1,0xf9,0xcd,0x4e,0x94,0x53,0x85,0x6d,0x22,0x6c, +0xf8,0x09,0xd6,0x20,0x1c,0xed,0x6e,0x5a,0xb0,0x97,0x6a,0x24, +0xd2,0xc1,0xaa,0x58,0x6a,0x6e,0xb0,0xbd,0x94,0x75,0x73,0x88, +0xc3,0x2b,0x24,0x1b,0xe9,0xd6,0x37,0x3c,0xa0,0xb3,0x08,0xe4, +0x33,0xe5,0x06,0x0a,0xa7,0x8a,0x0e,0x95,0x34,0x40,0x81,0xea, +0xb4,0xae,0x9d,0xa2,0x44,0xcb,0x98,0x91,0xe2,0xe7,0xa3,0x91, +0x71,0x78,0xcd,0xa8,0xca,0xa9,0xb9,0x0a,0x6d,0x3e,0xad,0x9a, +0x3b,0x52,0x39,0xf3,0x5b,0x4f,0x7e,0xb3,0xda,0x1e,0x85,0x0a, +0xa3,0xbe,0xdc,0x1c,0x35,0x1b,0xa3,0x9d,0x3a,0xba,0x7b,0xdf, +0x64,0xcc,0x01,0x44,0x39,0xfa,0xcd,0xa3,0x06,0xe2,0xde,0xec, +0xa4,0xcd,0x86,0x54,0x12,0x0d,0x1b,0xbe,0x35,0x45,0xb5,0x34, +0x3b,0xe6,0x86,0x9a,0xd5,0xab,0x64,0xdb,0x3d,0x4a,0xc0,0x3f, +0x9b,0x1d,0x8f,0x37,0x80,0x08,0x52,0xb5,0x02,0x17,0x6b,0x75, +0x99,0xcb,0x70,0xcd,0x8b,0xae,0xab,0xd6,0x09,0x39,0x62,0x7d, +0xe3,0xe4,0x2e,0x7b,0xf4,0xed,0x04,0x1a,0xbf,0x3e,0xf9,0x3d, +0xcc,0xc6,0xc6,0xed,0xd6,0xc5,0x12,0x0b,0x2d,0x17,0xf9,0x85, +0xc7,0x8d,0xec,0x7b,0x1a,0x37,0xca,0x54,0xb8,0xbd,0x4c,0xb3, +0x15,0x0b,0xa1,0x12,0x06,0x0b,0x46,0x54,0xa3,0x82,0x4a,0x5b, +0x57,0xb0,0x05,0x54,0xcb,0x20,0xdd,0xff,0xa2,0xda,0x4a,0xf9, +0xf8,0x50,0x34,0x52,0x9b,0x22,0x59,0x37,0x57,0xca,0x3c,0x78, +0x72,0x94,0x66,0xdc,0x3e,0x3d,0x2b,0x68,0x89,0xd5,0x52,0x8d, +0x4c,0x16,0x6f,0x23,0xb3,0x7b,0x48,0xae,0xd9,0x4a,0x81,0xd4, +0x20,0x31,0x61,0xf7,0x45,0xb2,0xd3,0xf7,0x74,0xe7,0x02,0x50, +0xee,0xf3,0xe5,0x0b,0xbc,0x65,0xce,0x9e,0xae,0x9c,0xe9,0xa3, +0xdc,0x3c,0x0f,0xdf,0x9b,0xb0,0xf8,0xa8,0xeb,0x2d,0x62,0x05, +0x86,0x43,0xe3,0x7a,0x7e,0x22,0x17,0xf1,0x8b,0x12,0xd5,0x9a, +0x09,0xfc,0x49,0x32,0xef,0x3a,0x46,0x9d,0xae,0xf7,0xe4,0x7a, +0xfa,0x08,0x96,0xb2,0xf1,0x3c,0x69,0x42,0xd1,0x26,0x9a,0xa4, +0x5a,0x61,0xa4,0xee,0x0a,0x1f,0xdf,0x20,0xa7,0xbf,0x99,0x2b, +0x7d,0x16,0x94,0x4e,0xa3,0xf1,0x6b,0xef,0x65,0x7e,0xef,0x3f, +0xd1,0x7d,0x9e,0xbb,0x25,0xe9,0xbe,0x73,0xd5,0x5d,0x76,0xbc, +0x90,0xcb,0x20,0x2b,0xd0,0x3f,0x2f,0x55,0x05,0xe6,0x74,0x78, +0x42,0x7e,0x54,0x6b,0xea,0x49,0x86,0x55,0x77,0x74,0xb6,0xb2, +0x26,0x56,0xb6,0xc5,0x85,0xa3,0x7a,0xbe,0x27,0xef,0x9c,0x00, +0x59,0x37,0xab,0xc8,0xeb,0x04,0x0c,0x76,0xa2,0xf5,0x52,0x73, +0xf2,0x74,0x69,0xaf,0xc0,0xf2,0xbb,0x29,0xcc,0x3f,0x5e,0xbe, +0xdf,0x7f,0xfd,0xee,0x6d,0x9e,0x6c,0xb7,0x1e,0xb7,0xda,0xb2, +0x2e,0xf3,0x45,0x3e,0x7a,0x34,0x78,0xea,0x93,0xb7,0x0d,0x1d, +0xb8,0xe4,0x1d,0x83,0x87,0x2a,0xf9,0xb6,0x41,0x35,0x50,0xfe, +0xd8,0xf0,0x49,0x44,0xfe,0xc4,0xf0,0x91,0x79,0xfe,0xb5,0x51, +0x07,0xe8,0xf9,0x53,0x6f,0x83,0x8d,0xd7,0x97,0x9d,0xd9,0x73, +0xae,0xa8,0x76,0xc2,0xe5,0x25,0x65,0xe1,0x5d,0x4d,0xfc,0x85, +0x2d,0xd1,0xd5,0x74,0x36,0x2f,0x4d,0xb4,0x55,0xca,0xb7,0x7e, +0x39,0xdc,0xea,0xe1,0x00,0x3f,0x6c,0x34,0x0e,0x37,0x7b,0x87, +0xa3,0x66,0xba,0x55,0x08,0x14,0x2e,0xec,0xfb,0x57,0xc3,0x89, +0x82,0x06,0x18,0x98,0xa9,0xbd,0xac,0x71,0x38,0x7a,0x94,0x1e, +0xb6,0xe0,0x19,0x4b,0x1c,0x66,0xf4,0x03,0xe9,0xf2,0x74,0xd8, +0xc2,0x8c,0x9b,0xc7,0xcb,0xb4,0x97,0xf6,0xb6,0x0c,0xee,0x64, +0xa5,0x93,0x6c,0xad,0x0d,0xc0,0xf3,0x23,0x22,0x82,0xff,0xf7, +0xdb,0x9b,0xdf,0x9a,0xd6,0xe0,0x51,0xfa,0x4f,0xc6,0xed,0xde, +0xf7,0xa2,0xf7,0x5d,0x7a,0x3f,0x08,0xd2,0x7e,0xb8,0xa3,0xcc, +0x3e,0x92,0xb0,0xf0,0xef,0x1f,0xd2,0x07,0x5b,0x26,0xd6,0x4b, +0xe3,0x87,0xf5,0xfb,0xbf,0xa0,0x52,0x0c,0x75,0x64,0x8d,0xc3, +0x43,0xc0,0xb2,0x37,0x5d,0xec,0xed,0xf5,0xf0,0xbf,0xc5,0xee, +0x74,0xb1,0xbb,0x4b,0x7f,0x7a,0xf8,0xdf,0x62,0x34,0x1a,0xf5, +0x46,0xf0,0x33,0xed,0x2d,0x3e,0xf5,0xa7,0x8b,0x4f,0x83,0xde, +0xe2,0x03,0xfc,0x7e,0x80,0x5f,0xb2,0x28,0x59,0xc8,0x9f,0xc5, +0xd9,0x59,0xe3,0xec,0xec,0x0c,0x29,0xf8,0xfe,0xfb,0xc6,0xf7, +0xdf,0x7f,0x8f,0x4f,0xe3,0xc5,0xcb,0xc5,0x70,0xf1,0x7c,0x71, +0x7e,0xde,0x5b,0xfc,0xf0,0x43,0x6f,0x71,0x79,0xd9,0x5b,0x94, +0x65,0x6f,0xb1,0xbf,0xdf,0xdb,0xef,0x2d,0xfe,0x6b,0xf1,0xe7, +0x9f,0xbd,0xc5,0xcf,0x3f,0xf7,0x16,0xa4,0x97,0xab,0x35,0x99, +0xac,0x25,0xf7,0xcd,0xc1,0xe2,0xcd,0x9b,0x1e,0xfe,0xb7,0xb8, +0xb8,0xe9,0x98,0x27,0x4b,0x2c,0xbe,0xca,0xae,0x2c,0xdf,0x3a, +0x1c,0x1d,0x8e,0xa0,0x6b,0x56,0x1b,0x13,0x22,0x08,0xe0,0x79, +0xbc,0xd4,0x40,0x71,0x7e,0x98,0xeb,0xed,0x21,0xa5,0xf0,0x93, +0x20,0xdb,0x99,0x3b,0xe6,0x5b,0xfd,0xe6,0xe1,0xe6,0xa0,0x47, +0x30,0x4f,0x03,0x18,0xb4,0x3e,0x84,0x6c,0xe8,0xad,0xc1,0xa3, +0xfe,0xc3,0xe1,0xe6,0x9f,0x87,0xd7,0xed,0xf6,0xf3,0xf6,0x26, +0xfc,0x7c,0xfd,0xea,0x15,0xfc,0xfd,0xa6,0x8d,0x2f,0xbb,0xdf, +0xe0,0xcb,0xab,0x6f,0xe9,0xe5,0xd5,0xee,0x0b,0x7c,0xd9,0x7d, +0x45,0x2f,0xaf,0x5e,0xbe,0x1a,0x34,0x17,0x7d,0x00,0x7d,0x4a, +0xb9,0xed,0xa7,0x00,0xba,0x35,0x68,0x36,0x0e,0xcb,0x47,0xbd, +0x30,0x79,0xd0,0x4c,0x81,0x80,0xed,0xe5,0x56,0x61,0xaa,0x8a, +0x8f,0x7c,0xeb,0x67,0xc0,0x82,0x74,0x62,0x53,0x65,0x3d,0xfc, +0x1b,0x02,0xe6,0x5b,0x07,0x95,0x92,0x62,0xd8,0x08,0x9f,0x70, +0x28,0x9f,0x08,0x15,0xb7,0xa4,0x25,0x65,0x2e,0xb8,0x99,0x57, +0x3e,0x82,0x8c,0x27,0xcb,0x4d,0x9e,0x50,0x87,0xa3,0x4d,0xfc, +0x03,0xfd,0xf9,0x01,0x7f,0x1f,0xd8,0x87,0x4d,0x4a,0xc3,0x27, +0x78,0x48,0x1b,0x8d,0x83,0xc5,0x5a,0x4a,0xaf,0x8d,0x4c,0xfd, +0xc5,0x4a,0xa0,0x47,0xe1,0xff,0xa9,0xfc,0xd7,0x88,0xa9,0xc7, +0xc4,0x07,0x44,0x81,0xa8,0x36,0xc9,0xfc,0x69,0x73,0x6f,0x6f, +0x73,0x77,0xf7,0xe0,0x87,0x1f,0xb2,0xcb,0xcb,0xac,0x2c,0x7f, +0x4e,0x8c,0x55,0x3c,0xe5,0x7d,0x05,0x90,0x18,0x32,0x3b,0xd9, +0xec,0x7f,0x18,0x7c,0xf8,0x10,0xbc,0x6c,0xbe,0x84,0x57,0x02, +0x44,0x13,0xaf,0x81,0xb1,0x8a,0xa5,0xbc,0xdf,0x4f,0x2c,0xd6, +0xd6,0x7e,0x62,0xb6,0x88,0x72,0x22,0xc7,0xfd,0x71,0x4d,0xb3, +0x35,0x30,0x1e,0xba,0x1e,0xd6,0x83,0x54,0xf2,0x25,0x4b,0xa7, +0x6f,0x0d,0xe4,0xe8,0x27,0x32,0x36,0x84,0xb9,0xc3,0xed,0xb2, +0xa0,0x26,0xd9,0x3a,0x2b,0xf0,0x22,0xef,0x1f,0x9f,0x79,0x81, +0x29,0xdd,0x72,0x5d,0xe6,0xe4,0xfc,0x74,0xf1,0x03,0xee,0xd4, +0x16,0x62,0x2d,0xb4,0xd8,0xe7,0xcd,0x10,0xbc,0xfb,0x9d,0x51, +0x62,0x2f,0x59,0x2c,0xc8,0xe9,0x60,0x31,0x57,0x99,0xaf,0x68, +0xb5,0x01,0x76,0xad,0x0b,0x64,0x1d,0xb3,0x6f,0x9f,0xc6,0xc0, +0xf4,0xc5,0x68,0xf1,0xe9,0xf8,0x89,0xa1,0xda,0x32,0x74,0x74, +0x61,0x40,0x34,0x2d,0xb3,0xbf,0x3d,0x7d,0x02,0x8f,0x74,0xd4, +0x58,0x66,0xdb,0x5f,0x7f,0xbb,0x3d,0x7e,0x6a,0x50,0x8c,0x04, +0x98,0xce,0xd7,0x00,0xf6,0x74,0x69,0xd4,0x35,0xe8,0xfc,0xe6, +0xb2,0xcc,0x12,0xb5,0x6b,0x83,0x4d,0x6c,0x96,0xd8,0xc7,0x4b, +0xcc,0xc2,0xaa,0x60,0xfb,0x99,0x25,0xb8,0x03,0x85,0xdd,0x47, +0x46,0x97,0x73,0xcd,0x2e,0xfe,0x62,0xce,0xa7,0x8c,0x2f,0xf1, +0x9a,0x0f,0x59,0x02,0x1d,0xf9,0x81,0x9e,0xf7,0x32,0xb9,0x82, +0x6b,0x3e,0x67,0x09,0x8a,0xb4,0x50,0x60,0x97,0x8a,0x88,0x54, +0x9b,0x98,0x31,0x97,0x23,0x64,0x2f,0x5d,0x51,0x7a,0x3d,0x3b, +0xe3,0x3c,0x06,0xfc,0xfe,0x7b,0x97,0x4b,0x09,0x18,0x5a,0x45, +0xdf,0xd9,0xce,0x6f,0xa0,0xd0,0xf4,0x94,0xe4,0x6d,0x5d,0x01, +0x94,0xb1,0xdb,0x83,0x00,0xbb,0xa4,0x2b,0x6a,0xf1,0x95,0x8b, +0xfb,0x5a,0x05,0x8a,0x93,0xc3,0xea,0x23,0x03,0x7f,0x5c,0x59, +0x23,0x47,0x36,0x38,0x14,0x76,0x77,0xd7,0x3e,0xad,0x7d,0x58, +0xdb,0x5b,0xdb,0x5d,0x1b,0xb9,0x0e,0x5f,0x83,0x0e,0xbf,0x82, +0x8d,0x9c,0x55,0x75,0xe7,0x09,0xe6,0xff,0xb0,0x76,0xbe,0x76, +0xb9,0x56,0x22,0x7c,0x00,0x59,0x77,0x3c,0x07,0x03,0x63,0x95, +0x87,0x1f,0xeb,0xa0,0xa1,0xb3,0x34,0x7b,0x7b,0x0a,0x2a,0xba, +0x1f,0xa1,0xbd,0xe7,0xb0,0x3d,0xe8,0xfe,0x39,0x06,0x55,0x60, +0x67,0x85,0x0c,0x4b,0x18,0xbe,0x04,0x45,0x54,0x7a,0x77,0x15, +0x8d,0xec,0x70,0x62,0x49,0xa3,0x61,0x25,0x88,0xdb,0xaf,0x2e, +0x61,0xb8,0xad,0x86,0xa2,0xfc,0xd1,0xbd,0x88,0xb4,0xea,0x35, +0x98,0x39,0x11,0xa5,0xa3,0x2f,0xc4,0x50,0xd7,0x56,0xa3,0x2f, +0x45,0x12,0x95,0xff,0xb4,0xea,0x23,0x11,0x1e,0xbf,0xf2,0xc3, +0x2a,0x00,0x19,0x98,0x08,0xf3,0xcf,0x7f,0xd6,0x00,0x05,0x2e, +0x59,0xa8,0x04,0x6f,0xf5,0xf1,0x8e,0xb0,0xd9,0xa6,0x52,0x5f, +0x54,0xce,0x3c,0x91,0x32,0x5f,0x54,0xe8,0x6b,0x28,0x04,0x73, +0xfa,0x3e,0x25,0xbc,0xed,0x9b,0x23,0x11,0x4d,0x1b,0x6f,0x6b, +0x20,0x3b,0x54,0xce,0x56,0x00,0xde,0x56,0x09,0x91,0x06,0x0c, +0xe6,0x3e,0xa5,0x14,0x13,0x50,0xd4,0xe1,0xaa,0x76,0x47,0xef, +0x58,0x02,0xbf,0x5f,0x01,0x7b,0x47,0x55,0x44,0xe3,0x4a,0xb7, +0x5e,0x32,0xa4,0x10,0xff,0xcb,0x3b,0xe8,0x10,0xb0,0xe1,0x2a, +0x30,0x3b,0xa1,0xe5,0x52,0x06,0x93,0x42,0x7a,0xc7,0x06,0x6f, +0xf2,0xc4,0x62,0x07,0x5f,0xd7,0xdb,0x80,0xe9,0xf9,0xbf,0x05, +0x53,0x07,0x30,0xfd,0xb0,0x0a,0x93,0x14,0x82,0xbd,0xcb,0x1d, +0x10,0x5f,0x75,0xb6,0x17,0x8b,0xce,0xf6,0x12,0x16,0xad,0x55, +0xec,0xd1,0x56,0xb9,0x84,0x35,0x6e,0x05,0x8c,0xac,0xb7,0x08, +0xb3,0x5f,0x07,0x43,0xe6,0x5e,0x82,0xcd,0x2f,0xcf,0x8d,0x74, +0x0b,0x86,0x03,0x96,0xa9,0x2b,0x14,0x76,0xef,0x2d,0x18,0x52, +0x1a,0x50,0xfb,0xf7,0x40,0x52,0x2d,0x6e,0x1e,0x43,0xd1,0x9f, +0xb3,0xc8,0x0f,0xdf,0x30,0xdf,0x24,0x58,0x94,0x66,0xd0,0xdd, +0x7e,0x9e,0x34,0xdd,0xd6,0xbf,0xbd,0x33,0xa4,0x38,0x1e,0x9b, +0x43,0x4c,0xc7,0x8b,0xa3,0xc7,0xcd,0x1a,0x52,0x87,0xa4,0x1a, +0xdf,0x4e,0x9b,0x49,0x96,0xd4,0xe6,0xa3,0xb5,0x1b,0x11,0xfe, +0xf3,0x7f,0x47,0xf5,0x9d,0xf6,0x23,0xa0,0x80,0xf9,0xce,0x9f, +0xab,0x7a,0x0d,0xf1,0x3f,0x3f,0x3e,0xa6,0x49,0xf6,0xe7,0xad, +0x50,0x18,0x9f,0x01,0xa1,0xfe,0x6b,0x15,0x10,0xc8,0x46,0x7f, +0xa0,0xce,0xc2,0x60,0xd4,0x66,0x94,0x67,0x79,0x79,0x03,0x79, +0x55,0x2d,0x95,0xf0,0x66,0x39,0xb9,0x7a,0x8c,0xb3,0x60,0xad, +0x49,0x06,0xdd,0x58,0x32,0x70,0xd7,0x7a,0x8b,0xbc,0x92,0xc5, +0x71,0x10,0x6a,0x8d,0x72,0x8a,0x66,0x32,0x4d,0x06,0x71,0x91, +0x46,0x3d,0xec,0x00,0xdd,0x85,0x90,0x79,0xa4,0x96,0x35,0x54, +0xcd,0x41,0xf2,0xad,0xb5,0x16,0x83,0xdc,0xfb,0x00,0x5c,0x51, +0xd9,0x36,0x57,0x56,0x97,0xdd,0xc2,0x4b,0x1c,0xb7,0x63,0x40, +0x10,0x18,0xbd,0xd6,0xc5,0x9e,0xbd,0xa0,0xac,0x9d,0x20,0x96, +0xe3,0xb9,0xef,0x2f,0xad,0xf9,0x24,0x87,0x3a,0x05,0x9b,0x53, +0xa0,0xf3,0x1c,0xc9,0xc3,0x64,0x51,0x7f,0x22,0x7d,0x35,0x0e, +0xf9,0x11,0xa2,0x87,0x1d,0x8e,0x56,0xa2,0xf8,0x42,0xea,0xab, +0x7e,0x72,0x94,0x34,0x25,0x61,0x69,0xc4,0x55,0x57,0x96,0xfc, +0x7d,0x38,0xb9,0x1e,0xce,0x3e,0x1f,0xbd,0x1a,0x1f,0xcf,0xe8, +0x61,0x6f,0x38,0x3b,0x39,0x3f,0x7a,0x7e,0x35,0x2b,0x2e,0xe0, +0xf9,0xf3,0xd1,0xdf,0xaf,0x27,0x63,0xf8,0x73,0xf1,0xf9,0xe8, +0xf9,0xf5,0xd9,0x75,0x39,0x3f,0xda,0x1f,0x5f,0xcd,0xc7,0x78, +0x1c,0x76,0xf4,0x0e,0xa4,0x79,0xfc,0x7d,0x3b,0xfd,0x9d,0x13, +0x76,0xc7,0x27,0xf4,0xe0,0x04,0xbd,0xa3,0xc4,0x3a,0xdc,0xf2, +0xdf,0x78,0x19,0xa9,0xbb,0x38,0xbf,0xef,0x7c,0x70,0x0d,0x1c, +0x75,0x34,0xe8,0x88,0x44,0x24,0x0f,0x29,0x43,0xba,0x2c,0x55, +0x48,0x14,0xd2,0x84,0x04,0x21,0x29,0x48,0x06,0x52,0x50,0x53, +0x39,0x23,0xba,0x83,0x02,0x02,0x0a,0xc8,0x50,0x17,0x8a,0x54, +0x61,0x4c,0x7c,0xeb,0x1c,0x6a,0x16,0xa8,0x3b,0x34,0x64,0xf0, +0xc3,0xd6,0x80,0x0a,0x23,0x95,0xc4,0x00,0x88,0x71,0x5a,0xde, +0x1f,0xb0,0x9d,0x70,0x67,0xdb,0x5b,0x2a,0x54,0xa0,0xc8,0x1f, +0x15,0x1e,0x8f,0x58,0x8d,0x28,0x3a,0xce,0xe9,0x6f,0xc3,0x1e, +0xa9,0x18,0xa4,0x5c,0x65,0x9e,0xfc,0x92,0x34,0xbd,0x7c,0x5c, +0xd2,0x91,0x12,0xb9,0x7d,0x5f,0x84,0x19,0x2c,0xd9,0x49,0xae, +0xa9,0xab,0xaa,0x72,0x95,0xce,0x5f,0x97,0x6b,0xd1,0x6d,0x39, +0xbe,0x2c,0x57,0x5b,0x96,0x6f,0xc8,0xfa,0x96,0x71,0xa1,0xb6, +0xa1,0x2b,0x2d,0xc7,0xc8,0x92,0xfd,0xeb,0x09,0x3c,0x1c,0xc1, +0xf6,0x0d,0x7f,0x0e,0xae,0xc7,0x25,0xfe,0x7e,0x18,0x8f,0x26, +0xfc,0x74,0x70,0x0e,0x8b,0x1d,0x3e,0xbc,0x82,0x95,0x15,0x7e, +0xf6,0x87,0x80,0x04,0x77,0x35,0xba,0x43,0x1d,0xba,0x95,0xbd, +0x69,0x21,0xfa,0x72,0xa6,0x35,0x50,0x44,0xc8,0x88,0x02,0x4a, +0x90,0x0c,0xa4,0x01,0xeb,0xc7,0x9a,0xb1,0x52,0xac,0xb1,0xb6, +0xb2,0x3b,0xc6,0x4f,0x00,0x56,0x57,0x2d,0x70,0x4b,0xac,0x14, +0xea,0x84,0x2a,0xa1,0x46,0xa8,0x10,0xea,0x83,0xea,0x6a,0x6b, +0xdb,0xd3,0x4e,0x51,0x57,0xd5,0x05,0x40,0xaa,0xa6,0xe0,0x84, +0xdd,0x17,0x96,0xe4,0xbb,0x46,0x6b,0x50,0xda,0x8d,0xd7,0x20, +0xd5,0x8d,0xd8,0x6f,0xe2,0x01,0x1b,0x80,0xc5,0x43,0x96,0x87, +0x6b,0x67,0x90,0x12,0xa5,0x45,0x65,0xd4,0xba,0x3d,0x43,0x75, +0xdc,0x86,0x7b,0x92,0xd5,0xf9,0xb8,0xeb,0x09,0xc7,0x75,0x4c, +0xd1,0x97,0x8d,0xec,0xb8,0x34,0x8f,0x6d,0xdd,0x92,0x7a,0x74, +0x87,0x37,0xd0,0xb3,0x9b,0x37,0x07,0x59,0x72,0x9e,0x5d,0x5e, +0xae,0x3d,0x4f,0xcc,0x9b,0x2c,0xd9,0xdb,0xdb,0xda,0xdd,0xdd, +0xa2,0x5b,0x75,0xe6,0x0d,0xbd,0xef,0xe1,0x9e,0xd8,0x26,0x84, +0x29,0x6b,0x6f,0x0e,0x28,0x11,0x52,0x71,0xd3,0x65,0xd6,0xa2, +0x3c,0x58,0xb7,0xc3,0xda,0xbc,0x1f,0x10,0xeb,0x33,0x44,0x5c, +0x17,0xf0,0x97,0x84,0xd0,0xe4,0x82,0xc3,0x06,0xef,0x61,0xb8, +0x8d,0x8d,0x55,0x80,0xad,0xf9,0xf4,0x27,0x8c,0x3d,0xca,0xfe, +0x57,0x28,0x84,0xdc,0xed,0x98,0xa3,0x02,0xde,0xd0,0x11,0xbf, +0x61,0x01,0xff,0xdf,0xdd,0x45,0x35,0xf1,0x48,0xdb,0x56,0x46, +0x17,0x8e,0xc4,0xf8,0xa7,0x83,0xf1,0x69,0x56,0x7e,0x40,0xce, +0x74,0xa4,0x46,0xfc,0x26,0x18,0xbc,0x4a,0x98,0xc5,0xa7,0x43, +0x72,0x26,0x78,0x85,0xd7,0x61,0x39,0x8d,0xee,0xdc,0x86,0x4e, +0x65,0xd0,0x86,0xef,0xf9,0xbc,0x81,0xa2,0x6d,0x78,0x57,0x3b, +0xdb,0xea,0x0f,0xaf,0x06,0x87,0xad,0xde,0x65,0x0f,0xfe,0x6c, +0xc1,0x9c,0x91,0x5c,0x5f,0x8f,0xf5,0x9b,0xc9,0x5e,0x34,0x8b, +0x92,0xf0,0xfa,0x98,0x2d,0x98,0x8b,0x0e,0xa3,0x24,0xa3,0x97, +0x5c,0x5d,0x82,0x84,0xf9,0xe3,0x5e,0x92,0xb9,0x94,0x21,0xa6, +0x3c,0xdf,0x83,0x4e,0x3d,0x3a,0x19,0x82,0xf0,0x32,0x1a,0xce, +0xb2,0x9b,0x12,0x86,0xd7,0x2e,0xaa,0x74,0xfa,0x07,0x53,0x18, +0x6d,0x6b,0xc3,0xf9,0x80,0x06,0x05,0xfa,0x0f,0xb1,0xe9,0x97, +0xd3,0x19,0x3a,0x14,0xd7,0x59,0x1f,0x48,0xe3,0x83,0xcd,0xbb, +0xd6,0xb7,0xe9,0xe8,0x46,0x80,0x8b,0xfc,0x13,0xc6,0xef,0x78, +0xa6,0xd1,0x61,0x1e,0x97,0xe9,0xbf,0x81,0xc7,0xc1,0x5a,0x58, +0x14,0xa9,0x78,0x79,0x01,0xed,0x90,0xbc,0x21,0x7d,0x94,0x50, +0xa7,0x47,0x9b,0x89,0x9d,0x65,0x88,0xcb,0x66,0x81,0xd5,0x63, +0xad,0x2e,0x62,0x10,0x95,0xe9,0x89,0x5f,0x54,0x36,0x94,0x40, +0x84,0x99,0xed,0xd4,0x23,0x6d,0x3b,0x93,0xdd,0x9c,0x92,0x69, +0x4d,0x96,0x80,0x10,0xf4,0x15,0x08,0xa5,0x57,0x40,0x73,0x96, +0x7c,0x55,0xae,0x0d,0xcf,0xa6,0xa4,0xd2,0x1b,0xae,0x9d,0xc2, +0x0c,0xb7,0xfa,0x47,0xd4,0xec,0x0d,0xd7,0xac,0x6e,0xef,0x12, +0xde,0xbe,0x1a,0x59,0x87,0xa7,0xa4,0xeb,0x1b,0x72,0x0f,0xc1, +0xf3,0x39,0xe5,0x51,0x77,0x91,0xf2,0x6f,0xb8,0x46,0x4a,0xb4, +0xd1,0x88,0xd2,0x59,0x04,0xde,0x23,0x6c,0xac,0xef,0xdb,0xdb, +0x63,0x64,0x22,0x36,0x7f,0xc6,0x2c,0x56,0x00,0x7e,0xfe,0x4c, +0x39,0x64,0x4e,0x02,0x8d,0x16,0xd0,0xef,0x1a,0xae,0xd6,0x04, +0x29,0x36,0x20,0xaa,0xb6,0xaa,0x46,0xd6,0x67,0xf0,0xfb,0x36, +0xee,0xfd,0xaa,0x94,0x76,0xf7,0x93,0xf6,0x2b,0x54,0xed,0x5b, +0x17,0xb8,0xd4,0xde,0x0c,0xe9,0x3f,0x05,0x7d,0x5c,0xca,0x1c, +0x94,0x40,0x24,0xac,0x43,0xaf,0xa1,0x18,0x41,0x77,0xda,0xbd, +0x84,0x7b,0x11,0x46,0x3d,0xe2,0x4b,0x6e,0xf9,0x02,0x39,0xc2, +0xe6,0x1f,0x61,0x3c,0xf6,0x60,0x5b,0xd1,0x58,0x02,0x8d,0x92, +0xe9,0x74,0x94,0x71,0x5b,0x47,0x8b,0xa7,0x40,0x79,0xfe,0xff, +0xd5,0x28,0xf1,0xdf,0x69,0xb3,0x33,0x4a,0xb6,0x07,0xf5,0x1e, +0x65,0x14,0xb9,0x86,0x5f,0xa1,0x79,0xa6,0xe5,0xfc,0x34,0x62, +0xc8,0x2b,0x40,0x49,0x35,0x1b,0x3a,0x9c,0x51,0x16,0x20,0xca, +0x2a,0xc8,0x2f,0x45,0x78,0x53,0x2f,0x7c,0xfd,0xcc,0xaa,0x36, +0x11,0x2f,0xb2,0x1b,0xbe,0x60,0x08,0xe9,0xd9,0x53,0x48,0x53, +0xbe,0x57,0x32,0x17,0xda,0x98,0x95,0xd9,0x3a,0x6b,0xd5,0xd9, +0xb3,0x82,0x41,0xbf,0xed,0x72,0xbe,0x1b,0xf2,0x57,0xa3,0x7c, +0xb7,0x18,0xbe,0xc9,0xe0,0x58,0xee,0x31,0xc7,0xd1,0xf2,0x9d, +0x89,0x40,0x68,0x9c,0x4c,0x60,0x6c,0x2b,0x45,0xd7,0x73,0x94, +0xc7,0x48,0x65,0xf9,0x71,0x73,0x54,0xb0,0x5b,0x22,0x73,0x74, +0x2a,0x3d,0x0e,0x6b,0x6c,0x46,0xc5,0xe4,0xd2,0x44,0xc6,0x3f, +0x86,0x0f,0xfd,0xd1,0x0f,0x5b,0xba,0x34,0x5e,0x40,0xbe,0x0f, +0xad,0x74,0x44,0xdd,0xfd,0x57,0x28,0xce,0x03,0x9a,0xf9,0x5a, +0x55,0xb6,0xde,0x76,0x44,0xb5,0x3d,0xd5,0x35,0x1f,0x14,0x7c, +0xc8,0x52,0x1c,0x62,0xfa,0x8f,0x80,0x5d,0x7a,0xc5,0xde,0x21, +0xf0,0x12,0x45,0x37,0xc0,0xc5,0x3e,0xd5,0x44,0xc7,0xfe,0xf1, +0xf7,0x3b,0xd1,0xa0,0x2c,0xce,0x26,0xc0,0x97,0xe6,0x7c,0x0c, +0xf4,0xba,0x9c,0xc2,0xfc,0xb7,0xc7,0xd0,0xb9,0x33,0x42,0x71, +0xee,0xde,0xc5,0x30,0xc2,0xc6,0x5b,0xcb,0x2b,0xfe,0x40,0x19, +0xbd,0xab,0xd6,0x17,0x15,0xa7,0x53,0xec,0xe7,0x3c,0xb3,0x08, +0x7a,0x37,0x4b,0x69,0x08,0xb6,0x0a,0x27,0xf7,0x5b,0xce,0x45, +0xa4,0xc0,0xa0,0xf7,0x37,0x8b,0x91,0x97,0x7b,0x46,0xe2,0x6c, +0x1a,0x02,0x4f,0xd5,0x9c,0xc7,0x96,0xe0,0xf9,0x8a,0x33,0x7a, +0x6d,0xfc,0x9c,0xf6,0x1a,0xd8,0x08,0xa8,0x90,0x41,0x0b,0x46, +0x32,0x20,0xef,0x0c,0x7a,0x9b,0x9d,0xac,0xe3,0xbf,0xe3,0xe6, +0x33,0x4e,0xa6,0x8c,0x15,0x34,0x0c,0x43,0x37,0x3d,0xd3,0x47, +0xd4,0x80,0xe7,0x41,0x0e,0x5d,0x57,0x91,0x9c,0xcb,0x20,0x47, +0x2e,0x51,0x4a,0x5e,0x19,0xe4,0x89,0x03,0x00,0x5b,0xae,0x8c, +0x0a,0x7a,0x17,0x01,0x0c,0x81,0x26,0x47,0xce,0x92,0x3e,0x30, +0x1c,0x08,0x3e,0x8e,0x02,0x62,0xae,0xf8,0x3a,0xdb,0xe5,0xc1, +0xe0,0xb0,0x31,0x82,0xa8,0x21,0x37,0x36,0x42,0x8f,0x02,0x9e, +0x45,0x1a,0xf8,0xaf,0x95,0x38,0x9b,0x62,0x31,0xf6,0x85,0x62, +0x69,0xaf,0x9d,0xe1,0x0f,0x13,0x19,0xb4,0xa0,0xad,0x4f,0xbe, +0x69,0x1b,0xf6,0x0e,0x7b,0x71,0xe2,0x63,0x48,0x1c,0xc5,0x89, +0x4f,0x20,0xf1,0x3c,0x4e,0xfc,0x1a,0x12,0x2f,0xe3,0xc4,0xa7, +0x90,0x58,0xc6,0x89,0xdf,0x40,0xe2,0xa7,0x38,0xf1,0x6f,0x14, +0xe8,0x18,0xa8,0x67,0x83,0xa9,0x4a,0x3c,0x03,0x35,0x0f,0xd0, +0x4e,0x8c,0x6e,0x74,0x46,0x7e,0xe7,0xc9,0x25,0x1b,0x79,0x9d, +0x07,0x34,0xec,0xa0,0x4d,0x0c,0xd6,0xe9,0x99,0xb0,0xbb,0x99, +0x88,0x51,0x49,0xb0,0x1d,0xc4,0xf6,0xc5,0x4d,0x50,0x76,0x5d, +0x23,0x67,0xcd,0xee,0xd4,0xb9,0xce,0xf4,0x48,0x1b,0xeb,0x38, +0xb4,0x54,0x69,0x20,0x83,0x59,0x57,0x87,0xd4,0x8d,0x5d,0xe5, +0x40,0x51,0x9c,0x14,0xf6,0x9c,0x27,0xc2,0x7a,0x9f,0x8b,0x16, +0x81,0x5c,0xd9,0xcf,0xa5,0x54,0x23,0x72,0x61,0x88,0x2c,0x24, +0x4f,0xc6,0x93,0x24,0xcd,0x62,0xef,0x89,0xf5,0xde,0x0b,0xcd, +0x2c,0xaf,0xda,0x22,0x49,0xab,0x45,0x6e,0xee,0x6a,0x4c,0xe4, +0x18,0x43,0xeb,0x08,0x5d,0x36,0xa6,0x59,0x04,0xcf,0xc9,0x41, +0x9b,0xec,0x62,0xf8,0x82,0x70,0x27,0xe4,0x9b,0x82,0xa2,0xf3, +0x73,0x51,0xf5,0xc8,0x58,0xa0,0x37,0xf1,0xcb,0xa2,0x54,0x72, +0x60,0x5c,0x43,0xd2,0x32,0x36,0xd4,0x5b,0x19,0x64,0x52,0xc7, +0xe3,0x65,0x50,0x55,0x76,0xb7,0xc2,0xa6,0x6f,0x29,0x6d,0x81, +0x61,0x97,0x93,0x93,0xca,0xd6,0xba,0x3c,0xef,0x90,0xbb,0xb2, +0xee,0xe6,0x66,0x91,0x3a,0xd7,0xd4,0x04,0x40,0x8e,0xed,0x70, +0x63,0x2f,0x15,0x86,0x2e,0xb3,0x7d,0xa5,0xe2,0x6f,0x3c,0xf6, +0x7b,0xad,0xdd,0x92,0x7b,0xa2,0x59,0x24,0x50,0x1e,0xa6,0xd1, +0x74,0x32,0xf4,0xea,0x4c,0x4b,0x18,0x5e,0x02,0x0b,0x6d,0xef, +0x08,0xb2,0x27,0xca,0x4f,0x72,0x14,0x6a,0xb8,0x70,0x56,0xeb, +0x8b,0x14,0x2f,0x97,0x5d,0xba,0x7a,0x69,0x1a,0xff,0x1c,0xc4, +0x25,0xaf,0x5d,0x0c,0x39,0xd1,0x83,0xe3,0x52,0x6a,0xab,0xb4, +0xa3,0x27,0x17,0x5b,0x60,0x1d,0x7c,0x06,0x43,0x9e,0xd7,0xc8, +0x41,0x82,0x95,0xec,0xfe,0x24,0xea,0xcc,0xbb,0xd3,0x2a,0x5c, +0x53,0x6c,0xf5,0x28,0x24,0x87,0xa8,0x42,0x38,0x22,0x16,0x5e, +0x92,0x27,0x3b,0x84,0x3f,0x6a,0xb0,0x2b,0x07,0xce,0x6c,0x80, +0x87,0xe6,0xfc,0x4b,0x63,0x9d,0xeb,0xaf,0x92,0xcb,0x88,0x58, +0xd8,0xb3,0xd2,0xd9,0x13,0xcd,0xc5,0x96,0x88,0xc7,0xb8,0xed, +0x5b,0x23,0x3d,0x81,0x28,0x0a,0xac,0xd5,0xc8,0x5a,0xff,0xfb, +0xbd,0x83,0xc1,0xcf,0x3f,0x27,0x84,0xfe,0x56,0xa1,0x8f,0xe9, +0xee,0x79,0x7b,0xd2,0xc0,0xe8,0x11,0x8b,0xbf,0xde,0x7f,0xb7, +0x92,0xc0,0xd0,0x61,0xa1,0x6a,0x3d,0x96,0x6b,0x8c,0xb2,0x9a, +0xe9,0x1f,0x0c,0xbc,0x09,0xcc,0xfe,0x7e,0xff,0xe7,0x01,0x53, +0x47,0x86,0xa8,0xf1,0x01,0xcb,0x25,0x6d,0x26,0x64,0x40,0xf5, +0x2f,0xed,0x79,0xab,0x53,0xd0,0x1a,0x1b,0xe1,0xc1,0x5c,0xba, +0xe3,0xb7,0x4b,0x75,0xf6,0x76,0xe9,0x0f,0xbc,0x28,0x5d,0x9f, +0x28,0x0d,0x96,0xf6,0xd6,0x5f,0xcd,0x07,0xb9,0xfb,0x80,0x3c, +0x04,0x60,0xd2,0xee,0x1f,0xec,0x63,0xf8,0xa6,0xf1,0xca,0x03, +0xfb,0xa3,0x61,0x4f,0x0e,0x23,0xc5,0x39,0x63,0xec,0x07,0x54, +0xa0,0x4c,0x43,0x1b,0xac,0xaa,0xb9,0x23,0xf9,0x96,0xe1,0xb9, +0x77,0x54,0x5a,0xb0,0x99,0x6e,0x9a,0xee,0xb4,0x51,0xda,0xa5, +0xc5,0xdc,0x1a,0x30,0xd7,0xd0,0xe3,0x0d,0x86,0x19,0xc7,0xd5, +0x29,0x39,0x4b,0xa7,0x49,0xf6,0x7c,0xbe,0xf2,0x03,0x94,0xdf, +0x61,0x18,0xbc,0xf3,0x93,0xdb,0x0e,0x9a,0x48,0x67,0x42,0xa6, +0x78,0x77,0x40,0x19,0xf5,0xb5,0xf9,0x7a,0x47,0x22,0x39,0xc5, +0x9b,0x25,0x9a,0xc0,0xfb,0xca,0xbf,0x9a,0xec,0x7e,0x83,0x81, +0x45,0x46,0x05,0x0a,0x12,0xc3,0x6a,0x54,0xd7,0x55,0xc7,0x7f, +0xf4,0x49,0xad,0xdf,0xa0,0x35,0xdc,0x7e,0x71,0xa8,0x8d,0x1a, +0x58,0x2e,0x25,0x75,0x14,0x59,0xf4,0x5f,0xbb,0xd5,0x14,0x1e, +0xf3,0x55,0xb7,0xdc,0xc2,0x25,0xae,0x81,0x0e,0x70,0xc4,0x46, +0x3b,0x8b,0xf3,0x7c,0x05,0xe6,0xae,0x38,0x2e,0xf4,0x9d,0x50, +0xd0,0x74,0x52,0x69,0xae,0x52,0x80,0xff,0x3f,0x4e,0xef,0xa6, +0x23,0x18,0x37,0xfc,0x31,0xb1,0xb4,0xa6,0x98,0x61,0x49,0x02, +0xa6,0x90,0xed,0x55,0x08,0x50,0xd0,0xe9,0x0e,0xf4,0xd4,0xb0, +0xbc,0x9d,0x86,0x53,0xcc,0x63,0xb3,0x10,0x86,0xc6,0x23,0x4c, +0x75,0x84,0xa5,0x70,0x21,0x9e,0x31,0xf3,0x11,0xed,0x26,0x56, +0x22,0xcf,0x6e,0x9c,0x70,0xec,0x8c,0xda,0x05,0xd0,0xb0,0x6d, +0x17,0x5d,0x96,0x84,0xf7,0xc5,0x42,0xac,0xbe,0x6c,0x42,0x8f, +0x94,0x20,0xf9,0x93,0xc7,0xdb,0xe3,0xaf,0xa5,0x22,0xed,0x01, +0x3e,0x6d,0x52,0x7d,0x41,0x92,0x55,0x5b,0xe6,0x9d,0xed,0x47, +0xda,0x88,0x84,0x49,0xe3,0x67,0xf4,0xdb,0xa6,0x0c,0x9e,0x38, +0xeb,0x32,0x2c,0xdf,0xcc,0x09,0x66,0x33,0xe0,0xb6,0x20,0x36, +0xcc,0xe6,0x2a,0x7e,0xc4,0x66,0x03,0x8b,0x7a,0x18,0x68,0xf7, +0x0a,0x4c,0x9a,0x6e,0xa9,0x6e,0xd8,0xac,0xb6,0xd9,0xad,0x35, +0x58,0xa0,0x86,0x6a,0xd9,0xdb,0xeb,0xb3,0xcd,0x2f,0xd5,0x86, +0x2d,0xec,0xb4,0xcc,0x5b,0xe8,0x3f,0x00,0x76,0x3d,0xd4,0xbe, +0xf4,0xa1,0x88,0xcd,0xb6,0x9d,0xb5,0xe0,0x73,0xfd,0x80,0x60, +0xb8,0x8e,0x3a,0x83,0xbe,0x30,0x07,0xbe,0x49,0xec,0xfb,0xc2, +0x74,0x34,0x2b,0x64,0x7b,0xbf,0xb0,0x47,0x37,0xed,0x08,0x4a, +0xb7,0xc8,0xde,0x50,0x4c,0x00,0x57,0x02,0x3d,0x6d,0x3f,0xf9, +0x1b,0x40,0x61,0x6a,0x6a,0x47,0xb8,0xa8,0xdf,0x32,0x17,0x84, +0xca,0x31,0x9e,0x53,0x98,0x3a,0x7e,0x72,0xa0,0xfb,0xb5,0x50, +0x39,0x17,0xc9,0x21,0x7e,0x4e,0xf2,0x00,0x83,0x4a,0xa8,0x50, +0x2a,0xb2,0x80,0xe6,0x73,0x2c,0xbf,0xb6,0xce,0xaf,0x2f,0x87, +0x13,0x18,0xcb,0x8d,0xf0,0xa6,0xa2,0xd4,0xfd,0x76,0xaa,0x0c, +0xb2,0xea,0x6b,0x26,0x9c,0x08,0x6b,0x57,0xf3,0xd4,0xc4,0x98, +0xaa,0x4a,0x61,0x3f,0xa7,0x73,0x4f,0xa9,0xf3,0xfa,0xe5,0x27, +0xb0,0x0c,0x00,0x3f,0x32,0xb0,0x03,0x60,0xae,0xb1,0xb2,0x15, +0xdd,0x36,0x89,0x26,0x71,0xf3,0xe9,0x0e,0xe2,0xe8,0x25,0x56, +0x1d,0x9d,0x64,0x9b,0x1d,0x49,0xb2,0x0a,0xec,0x24,0x6b,0xab, +0x94,0x5d,0xc0,0x94,0x75,0x54,0x29,0x4a,0xd8,0x96,0x04,0xd1, +0xa1,0x27,0xd9,0x37,0x2a,0x81,0x91,0xf8,0x2a,0x82,0xe5,0x43, +0xd6,0x0c,0xdd,0xc4,0xf6,0xbb,0xad,0xfb,0x62,0x9a,0x17,0x24, +0x23,0xd8,0x10,0x16,0xb5,0xf2,0x84,0x8b,0x6f,0xa1,0xe6,0xbc, +0x95,0x2c,0x6e,0x5d,0x42,0xd3,0x67,0x81,0xf8,0xc7,0x6c,0xa0, +0x6d,0xa7,0xd1,0x62,0x71,0x07,0xe4,0xd7,0x16,0x72,0x69,0x46, +0x5a,0xc0,0x52,0x97,0x68,0xf0,0xda,0x99,0x66,0xbc,0x22,0xf4, +0xe9,0x5b,0x6a,0x99,0x4a,0x0b,0xae,0xad,0x05,0x17,0x6c,0xd4, +0x3d,0xdd,0xf0,0x12,0xb1,0x6a,0x3f,0x11,0x08,0xf0,0xb2,0xdf, +0xcd,0x88,0xd5,0x41,0x9b,0x00,0xb8,0x84,0xc9,0x8e,0x3f,0x7c, +0xe0,0xbe,0x8a,0xcc,0x77,0xa7,0xc4,0x45,0xcd,0x97,0x5c,0xfb, +0x59,0x79,0x55,0x8b,0xa9,0x0d,0xec,0xb9,0xaa,0xde,0x43,0x57, +0x5c,0x51,0xa6,0x36,0xca,0x1a,0x9e,0x24,0x19,0xf1,0x2c,0x8c, +0xfa,0xe7,0x8e,0x95,0x7f,0xf4,0xad,0xa3,0x84,0x4a,0x24,0xee, +0xe6,0x91,0x07,0xdf,0xb3,0xf1,0xb0,0xd4,0xb7,0x56,0x57,0x95, +0xf4,0xaf,0x5c,0x56,0x72,0x95,0xa2,0x79,0x18,0x4f,0xbc,0x2c, +0xde,0x0b,0x8a,0x27,0xd7,0x5b,0x16,0x44,0xf1,0xef,0x4a,0x3c, +0x3b,0x53,0x6b,0x54,0x5b,0xfc,0x35,0x32,0x8f,0xcf,0x74,0x03, +0x88,0xe3,0x48,0x9a,0x65,0xf4,0x68,0xcd,0x92,0xc5,0x29,0x24, +0xce,0x47,0x65,0xe8,0x66,0x31,0x11,0xb7,0xce,0x02,0xc3,0x36, +0x57,0x09,0xb3,0xf8,0x2c,0xb0,0x68,0xb3,0x99,0xb2,0x32,0x64, +0x55,0x1b,0x32,0x90,0x58,0x45,0xdf,0x1b,0x32,0xf3,0xc0,0xcc, +0x10,0x64,0x0a,0x47,0xa0,0x5f,0x94,0x62,0x2b,0x43,0x27,0xe7, +0x00,0x27,0xa8,0x69,0xc7,0x7b,0x08,0x16,0x4c,0xbb,0x30,0x40, +0x4e,0xe3,0x0b,0xb0,0x71,0xe5,0x3d,0x69,0x3a,0x96,0x9e,0x3a, +0x69,0xcb,0xca,0x82,0xb0,0xa0,0x02,0xb7,0xec,0x10,0x0f,0x79, +0x0e,0x1b,0x93,0x59,0xad,0xbc,0x15,0x11,0x53,0x77,0x2f,0x8a, +0xab,0xa1,0xbf,0x91,0xa9,0x7c,0x33,0xe0,0x27,0x21,0xb5,0x3b, +0xcd,0x50,0xf6,0x0a,0x73,0x91,0xa8,0xef,0xc6,0xc0,0x21,0xc7, +0xff,0xb3,0x54,0x3d,0xbb,0x93,0xaa,0xfd,0xe1,0xe5,0xff,0x30, +0x4d,0xd0,0x93,0x77,0x50,0x75,0xa9,0xad,0x3d,0xa6,0xf3,0x73, +0x75,0x68,0x45,0x6f,0x56,0xc3,0x52,0xeb,0xe9,0x81,0x86,0xd2, +0x0e,0xc1,0x31,0x57,0xa2,0x47,0x40,0x3a,0xfc,0xe3,0x5f,0x40, +0x4a,0x20,0x3b,0x88,0x2f,0x40,0xfa,0x67,0xa0,0x30,0x51,0xcc, +0x99,0x65,0xf2,0x3c,0x12,0xd0,0x35,0x3f,0xb6,0x57,0x6b,0x35, +0xdf,0x66,0x40,0xbd,0xb8,0xd8,0xbb,0x29,0xc2,0xd3,0x6e,0xbb, +0x6a,0x2b,0xfc,0xfb,0x0e,0xef,0xb5,0xa9,0x71,0x4e,0x14,0x38, +0xe1,0x59,0xe7,0xa9,0x2b,0xfc,0xb4,0xfd,0x48,0xf3,0x60,0x21, +0x3c,0x57,0x8b,0x96,0xdd,0xb8,0xb6,0x25,0xe8,0xfb,0x7a,0x6e, +0x7d,0x35,0xdf,0x6b,0x57,0x14,0x4b,0x70,0x8c,0x64,0x93,0x2b, +0x48,0x2e,0x41,0xea,0xe9,0x90,0xc0,0x23,0x7a,0x06,0x6b,0xe8, +0xb9,0xfa,0x6c,0x2e,0x58,0xee,0xb8,0xc0,0xdb,0x60,0x44,0xd7, +0x17,0x78,0x31,0xe5,0x83,0xcd,0xf9,0x78,0xb4,0x06,0x7c,0x08, +0xf5,0xd3,0xc3,0x8b,0x35,0x6c,0x6c,0xc6,0xe3,0x34,0x67,0x55, +0x44,0x95,0xf6,0x17,0xc4,0xc2,0x15,0xd5,0xbe,0xac,0xb0,0x5c, +0xf1,0x7c,0x58,0x3e,0xbf,0x28,0xce,0x60,0xee,0xe0,0xed,0x1d, +0xee,0xca,0x15,0x06,0x1a,0x72,0xb9,0x5b,0xef,0x4f,0x83,0x1d, +0x5f,0x9a,0xb5,0x0d,0x3a,0xa9,0x0a,0x76,0x24,0x0c,0xf0,0xd5, +0xd3,0x36,0xc9,0x34,0x76,0x61,0xac,0x69,0x03,0xbd,0x6c,0x2a, +0xb9,0xcb,0xe8,0xdd,0x15,0x0b,0x46,0x72,0x51,0x7d,0xa5,0xdc, +0x41,0x97,0xe6,0xd9,0xa5,0x44,0xa3,0x7e,0x43,0x44,0x42,0xec, +0x8a,0xcd,0x12,0xad,0x97,0xa9,0x6c,0x26,0xd2,0x66,0xa7,0x6e, +0x5e,0x78,0x22,0x9c,0x80,0x84,0xde,0x10,0x9c,0x84,0x24,0x8e, +0x20,0xf8,0x04,0x79,0x25,0xb1,0x58,0x53,0xae,0x4e,0x92,0x69, +0x14,0x06,0x3b,0x84,0xf0,0x44,0x39,0x4a,0xfd,0x9c,0x52,0x13, +0xd5,0xd1,0xf7,0x39,0x24,0xed,0xb3,0x25,0xed,0x33,0x53,0xa5, +0x4c,0xfa,0xef,0x4f,0x58,0xc7,0x3c,0xf9,0xcb,0x15,0x86,0x07, +0xe9,0xaa,0x26,0xf2,0xb3,0xab,0x3f,0x8e,0x6e,0x99,0x50,0x8f, +0xd4,0x55,0x43,0x78,0x82,0x26,0xff,0xe6,0x11,0xa3,0xdb,0xc4, +0xac,0xd4,0x7f,0xdb,0xea,0xda,0x6a,0xbf,0x0b,0x13,0xff,0x62, +0x85,0xce,0xed,0xc3,0x8a,0x0a,0x51,0x62,0x77,0xea,0x86,0x46, +0xda,0xfc,0x66,0xb3,0xb6,0x8b,0xd3,0xaf,0xbe,0x59,0x45,0x00, +0x62,0xaf,0x19,0x67,0x92,0xe5,0xbf,0xb9,0x8e,0x8a,0x55,0x4c, +0x9d,0x68,0x59,0x2c,0xbe,0xc9,0xdc,0x9b,0x4f,0xfe,0xea,0x9b, +0x9e,0x1c,0x06,0x53,0x3d,0xdf,0xe0,0x7d,0x16,0xcd,0x14,0xbe, +0x4c,0x72,0x92,0xe8,0xa6,0x24,0xcb,0x56,0xb0,0xb0,0xc2,0xfe, +0x5e,0xb8,0x6a,0x2c,0x4f,0x14,0x7a,0xe6,0x70,0xb6,0x2e,0xc6, +0x2a,0x1c,0x0e,0x5b,0x3a,0x5b,0x75,0xe2,0x14,0xf8,0xd6,0x62, +0xd6,0x48,0xf0,0x0d,0xff,0xbc,0xea,0xd0,0x8b,0xd7,0x81,0x65, +0x2a,0x1e,0x41,0xeb,0xef,0x61,0x6a,0x57,0x7a,0x75,0x6e,0x15, +0xea,0x4b,0x91,0x1d,0x63,0x7d,0x28,0x59,0x8a,0x23,0x8b,0x37, +0xf7,0x56,0x15,0x4c,0xbb,0x75,0xf5,0xc8,0xc5,0xc7,0xc4,0xfa, +0x93,0x4a,0xdc,0xbe,0xe4,0x74,0x42,0x3b,0x96,0x3c,0x78,0x55, +0x99,0x12,0x35,0x3a,0x4a,0x50,0x00,0x38,0x0e,0x75,0x3e,0xf9, +0x65,0xf1,0xaf,0x32,0x38,0x35,0x84,0x24,0x29,0xa0,0xf9,0xf4, +0xef,0xfb,0xef,0xde,0xe6,0x3a,0xc1,0x9d,0x61,0x44,0xa7,0x43, +0xea,0x90,0x32,0xb7,0x4b,0xb9,0x3e,0x28,0x92,0x90,0xe8,0xb1, +0xaa,0xc3,0x3a,0x73,0x0a,0x43,0x69,0x73,0x60,0xf5,0xc0,0x4c, +0x81,0x3b,0xbe,0x1a,0x4c,0x5b,0x04,0x1f,0x15,0x48,0x5b,0x20, +0xe9,0xc5,0x50,0x58,0x76,0x0b,0x33,0x1f,0x76,0xf1,0x4f,0x68, +0xff,0xa0,0x5f,0xbe,0xc2,0xab,0xb3,0x36,0xc3,0x69,0x9d,0x34, +0x04,0x3b,0x8e,0x22,0x2c,0x16,0xce,0x96,0x7d,0xea,0x43,0xe1, +0xbb,0xa2,0xda,0xa7,0x94,0x54,0x2d,0x71,0xb8,0xf9,0x17,0x0b, +0xb1,0xaf,0x29,0x55,0x9b,0x76,0x37,0x05,0x45,0x38,0x9f,0xfe, +0x7e,0xb5,0xfd,0x84,0xbe,0xba,0xe9,0xc1,0xbd,0x1b,0x2a,0x02, +0xa6,0x36,0xc1,0x3f,0x5f,0x3d,0x6e,0x4b,0x83,0x28,0xe0,0x91, +0xb8,0x72,0x12,0x5a,0xec,0x10,0xc2,0x9f,0xaf,0x3a,0xdb,0xe2, +0xc8,0xca,0x53,0x42,0x19,0x5b,0xe8,0x85,0x94,0xe0,0x39,0x9b, +0xfe,0x32,0x7f,0xad,0x3b,0x38,0x71,0xa5,0xdd,0xa6,0x9b,0x7c, +0x0a,0xdd,0x72,0x06,0xb8,0x56,0xed,0xda,0x26,0xad,0xf2,0x8f, +0x7c,0xdf,0x36,0xf9,0x9a,0xf2,0xa3,0xc0,0xbc,0x1f,0x48,0x4e, +0x9b,0x72,0x65,0xf9,0x91,0xba,0x35,0x75,0xe4,0x09,0x07,0x41, +0x4a,0xf4,0x78,0xa1,0xba,0xce,0xea,0xea,0xac,0xb6,0x6d,0x3c, +0x1b,0x4f,0x4e,0xc6,0x39,0xed,0x41,0xac,0x86,0x34,0xf0,0x3c, +0xe6,0x81,0xcc,0xba,0x47,0x10,0x68,0x65,0x9c,0x1b,0x22,0x97, +0x1d,0x19,0xfc,0xda,0x83,0x14,0x67,0x08,0xa8,0xb1,0x8a,0x6a, +0x33,0x35,0xff,0xe2,0x99,0x4b,0x6c,0x34,0xa0,0x40,0xba,0xab, +0xdb,0x1b,0xa3,0xcf,0x47,0xb3,0x4b,0x35,0x3e,0xe7,0xd2,0x1c, +0xd3,0x0d,0x6c,0x4b,0xf1,0x4c,0xe3,0x1c,0x9e,0xe5,0x8d,0x2f, +0x39,0x75,0xf9,0x4b,0x14,0x6f,0xde,0x4a,0xf1,0xe6,0x0a,0x8a, +0x37,0xef,0xa6,0xf8,0x5f,0x5e,0x52,0xc3,0x15,0x82,0xfc,0xee, +0xd0,0x95,0xca,0xf2,0x2f,0xa3,0x25,0xff,0x33,0x8c,0xdb,0x59, +0x5b,0x87,0x96,0xe2,0x92,0x6b,0x4d,0xbf,0x5d,0xa5,0xb4,0x66, +0x7a,0xee,0x4d,0xa6,0x74,0x30,0x51,0xca,0x9a,0x73,0x72,0x2b, +0x60,0x96,0x3e,0x40,0xbc,0x13,0xf7,0xcb,0x46,0x6a,0xef,0x00, +0x45,0xb9,0x72,0x5f,0x26,0x15,0xcf,0x78,0x61,0x26,0x4f,0x7e, +0xeb,0x4c,0x2f,0xcc,0x93,0xa3,0x66,0xef,0x83,0x2f,0xc2,0x6b, +0xcf,0x9f,0x53,0xc7,0x8e,0x43,0x00,0x77,0x06,0xdd,0xac,0x6a, +0xa9,0xd8,0xbb,0x5f,0x30,0x76,0x86,0xe5,0xbe,0xcd,0x14,0x27, +0x46,0x2a,0xe5,0x59,0xbb,0x97,0x6c,0xe2,0x26,0x0e,0x1a,0xee, +0xc7,0xa4,0x49,0xfe,0xb5,0x4a,0x92,0x9d,0xcb,0x66,0xf2,0x4f, +0xce,0x10,0x66,0xd8,0x93,0x91,0x8f,0xe1,0x99,0x28,0x19,0xbf, +0xb1,0x47,0x93,0x04,0xa3,0x1b,0x51,0x12,0x7d,0xda,0x62,0x21, +0x9f,0xb0,0x58,0x08,0x59,0x3d,0x8c,0x33,0xe6,0x01,0x7a,0xf4, +0xb7,0x89,0xd1,0x7a,0x18,0x3f,0x83,0xf7,0xe4,0xd7,0xd7,0x60, +0x8b,0xdb,0xa9,0x8a,0x21,0x72,0x30,0xee,0x64,0xf2,0x63,0x7b, +0x37,0x01,0x39,0xc7,0x5f,0x6d,0xab,0x77,0x13,0x91,0xd6,0x27, +0xc7,0xd6,0x59,0x05,0xda,0x65,0xd5,0xba,0x47,0x2a,0x56,0xf8, +0x9f,0x40,0xa1,0xc6,0xd4,0xf8,0x6b,0x2a,0xc2,0x19,0x90,0xb2, +0xe4,0x53,0x41,0x9b,0x90,0xf8,0x91,0x18,0x3e,0xf8,0x49,0xe3, +0x7d,0x3e,0x8e,0x57,0xf4,0x45,0x44,0x83,0xae,0xb2,0x66,0xb0, +0x71,0xc5,0xa6,0x63,0xfe,0x7a,0xa4,0x6e,0xf1,0x5a,0xd1,0xec, +0x6c,0x07,0xe9,0x81,0xb9,0x13,0x59,0x7f,0x98,0x9b,0x95,0xf6, +0xce,0x38,0x1b,0x8e,0x73,0x7e,0xfb,0xaa,0xd3,0x76,0x07,0x8e, +0x20,0x97,0xf2,0x42,0xe3,0xb2,0xda,0x78,0x2b,0xb7,0x97,0xa0, +0xca,0x16,0x73,0x8f,0x51,0x59,0x9e,0x64,0xdb,0xfc,0x88,0x5a, +0xd4,0xc7,0xfc,0x38,0x83,0x47,0x84,0xf2,0xbb,0x09,0x44,0xd0, +0xb4,0x81,0x2b,0x53,0xef,0x79,0xa9,0xd7,0x08,0xdd,0x2b,0x09, +0x7f,0x34,0xda,0xa7,0x14,0x74,0x7f,0x55,0xee,0x66,0x79,0x79, +0x63,0x83,0x7f,0x5b,0xc3,0xcb,0x51,0x8f,0x1f,0xf1,0xe0,0x10, +0x51,0x24,0xfe,0xd6,0x88,0x04,0xb4,0x35,0xd6,0x85,0x13,0x57, +0xa9,0x8e,0xcd,0x88,0x02,0xbe,0x20,0x09,0x0b,0xd1,0x64,0xca, +0x35,0xaf,0xe7,0xf9,0x7a,0x7b,0x63,0x43,0x93,0x22,0xcd,0xba, +0x64,0xb7,0x66,0x36,0x75,0xc9,0x31,0x3b,0x79,0xcb,0xf8,0xff, +0x06,0x00,0x00,0xff,0xff,0x07,0x1f,0x31,0x88,0x88,0xcc,0x05, +0x00, })) if err != nil { diff --git a/dashboard/resources/scripts-stats-modules.js.go b/dashboard/resources/scripts-stats-modules.js.go index d544eff6c5f..747a4d067b4 100644 --- a/dashboard/resources/scripts-stats-modules.js.go +++ b/dashboard/resources/scripts-stats-modules.js.go @@ -11277,2471 +11277,2082 @@ func scripts_stats_modules_js() []byte { 0x27,0xc2,0xb5,0x5f,0x4a,0x95,0x86,0x70,0x79,0xe5,0x5d,0x5a, 0xf7,0x92,0x2a,0x7e,0xa2,0x21,0x95,0xcc,0x12,0xd9,0x79,0xd4, 0x2f,0xff,0xcf,0x2f,0x7f,0x8f,0xc8,0x54,0xaa,0x11,0xff,0x7f, -0xd4,0xfd,0xed,0x76,0x1b,0xb7,0xb2,0x20,0x0c,0xff,0x3f,0x57, -0x61,0xf1,0xdd,0xd1,0xdb,0x2d,0x82,0x54,0x37,0x65,0xcb,0x36, -0x99,0x16,0x97,0xe3,0x24,0x27,0x9e,0x89,0x13,0x3f,0xb1,0x77, -0xb6,0x1d,0x6e,0x6e,0x2d,0x8a,0x6c,0x59,0x6d,0xf1,0x2b,0x6c, -0x52,0x62,0xdb,0xe2,0x5a,0x67,0xd6,0xcc,0x2d,0xed,0x5f,0xcf, -0xbf,0xb9,0x80,0x93,0x5b,0x7a,0xea,0x03,0x9f,0xfd,0x41,0x52, -0x8e,0xcf,0x99,0x33,0x6b,0xef,0x58,0x6c,0xa0,0x50,0x28,0x14, -0x0a,0x85,0x02,0x50,0x28,0xf0,0xab,0x9f,0xb6,0x2f,0x78,0x40, -0x2e,0x48,0x6e,0x5a,0x48,0x4e,0x56,0x76,0x5a,0x9a,0x4c,0x09, -0x0d,0x7b,0x82,0x24,0x53,0x0f,0xbf,0x50,0x3f,0xa4,0x26,0x19, -0x3e,0x64,0x32,0x00,0x20,0x62,0x0d,0x0c,0x1f,0x04,0xab,0x13, -0x11,0x94,0x12,0x09,0x32,0xb4,0x21,0x43,0x09,0x19,0xda,0x90, -0x26,0x92,0x98,0xe5,0xaf,0x62,0xbc,0x52,0x3c,0xbc,0x50,0x84, -0x65,0x8e,0x24,0x9d,0xfe,0xd1,0xb2,0xae,0x12,0x83,0x23,0xae, -0xa4,0xc1,0x54,0x1d,0x49,0x48,0x49,0x3a,0x40,0x2a,0x7a,0x25, -0x5c,0x5d,0x96,0xca,0xc1,0x69,0x5e,0xbf,0xc7,0xd7,0xe2,0x87, -0xab,0x0a,0x6d,0xa4,0x73,0xf5,0x44,0x2b,0x87,0x61,0xee,0x72, -0x91,0x3b,0x1c,0x71,0x03,0x20,0xf5,0xec,0x28,0x81,0x32,0xc5, -0x3a,0xdd,0xe6,0x37,0xc2,0xcd,0x8b,0x79,0x60,0xc3,0x7f,0xfb, -0xd6,0x3f,0xfa,0xf6,0xad,0x00,0xcb,0x14,0x7e,0xd1,0xe5,0xe4, -0xb7,0xbe,0x8a,0x3d,0x55,0x52,0x02,0x2c,0xfa,0x6f,0xdf,0x41, -0x89,0x77,0xe2,0x1d,0x94,0x78,0xc7,0x25,0xde,0xf9,0xdb,0x8a, -0xac,0x61,0x11,0x80,0x43,0x68,0x8d,0x2e,0x15,0xf0,0xab,0x79, -0x99,0x8c,0xf1,0x00,0xb6,0x64,0xa0,0xea,0xed,0xc5,0xf5,0x57, -0x40,0xcd,0xd9,0xbf,0xff,0x53,0x3d,0x7c,0xbe,0xb5,0x82,0x0c, -0x2a,0xc8,0xa0,0x82,0x0c,0x1d,0x36,0xe0,0x57,0xa1,0x82,0xac, -0x58,0x41,0xf6,0x15,0x10,0x6f,0x2a,0x00,0x85,0x48,0x7a,0x0c, -0x28,0x5c,0x07,0xc8,0x8c,0xb7,0xf4,0x98,0x5f,0x16,0x60,0x33, -0xdf,0x05,0x02,0x54,0xa6,0x78,0x2b,0xde,0xe1,0xb3,0xbf,0x18, -0xe3,0x3b,0x8b,0xa0,0x35,0xdf,0xbe,0x8d,0x9e,0x06,0xc0,0x83, -0x08,0x26,0x6a,0x4b,0xc3,0xb5,0xcc,0xfa,0x5b,0x77,0x24,0xed, -0xce,0x94,0x79,0x1c,0xc8,0x2e,0x72,0x03,0x78,0x95,0x4c,0xca, -0x52,0x00,0xaa,0xfa,0xde,0x9e,0x2d,0x31,0x52,0xa0,0xa9,0x78, -0xb6,0x5a,0x62,0x1d,0xc5,0xaa,0xb7,0x6a,0xf6,0xb7,0x20,0x19, -0x9a,0xe3,0xef,0xbc,0x77,0xa1,0xdf,0x4c,0x31,0x4e,0xbe,0x07, -0x23,0x0b,0xf2,0x42,0xdf,0xba,0xc5,0xab,0x33,0x00,0x2e,0x28, -0xcb,0xf0,0x61,0x8e,0xb0,0x28,0x8a,0xf7,0x7f,0xb8,0xcc,0x14, -0x9a,0x0c,0x3e,0xcc,0x16,0xf2,0x50,0xe2,0xdc,0xc7,0x1b,0xda, -0xd6,0x67,0xdb,0x82,0xb3,0x32,0x1c,0x3e,0x58,0x08,0xf6,0xd3, -0xc5,0x6f,0x2d,0xc7,0x99,0xb7,0x96,0xb3,0xcc,0x3b,0xcb,0x89, -0xe6,0x9d,0x71,0x9c,0x01,0x81,0x39,0x7b,0x8b,0x9e,0x76,0xe7, -0x11,0x88,0x0e,0x14,0x46,0x19,0x0a,0x61,0xb9,0x03,0x05,0xce, -0xde,0x71,0x06,0xc8,0x11,0x94,0x46,0x81,0xa2,0x0c,0x43,0x9c, -0x39,0xd1,0xb3,0xc3,0x93,0xf4,0x7a,0x6f,0xb1,0x00,0xac,0x23, -0xdf,0x62,0x11,0x5c,0x25,0x96,0x36,0x74,0xbf,0xf6,0xd8,0x8e, -0x40,0x8e,0xf3,0x8f,0xd5,0x1e,0xcb,0x11,0x08,0x86,0xc1,0xd9, -0x9a,0xc9,0xc6,0xd7,0x2d,0x83,0x08,0x47,0x06,0x91,0x9d,0x05, -0x67,0x19,0x67,0xe0,0x83,0x97,0x41,0x84,0xc3,0x64,0xcf,0xf6, -0xe4,0xfd,0x84,0xac,0x22,0x18,0x86,0xeb,0x73,0x64,0xe2,0x35, -0x94,0xd3,0x12,0x21,0x3f,0xf2,0xf2,0x40,0xc9,0x45,0x69,0x78, -0xbd,0x6f,0x95,0x1e,0x8c,0x74,0x3e,0x5a,0x84,0xc1,0xce,0x0f, -0x03,0x6a,0x54,0xd0,0x2c,0xd0,0x9e,0xdf,0xbe,0x2b,0x76,0xce, -0xfe,0xe8,0x47,0xea,0xe4,0x72,0x94,0x95,0xa0,0x07,0x4d,0x33, -0xca,0x1c,0xf4,0xf7,0xb4,0x2a,0x6c,0xf3,0x40,0x68,0x47,0x50, -0x8d,0xee,0xad,0xc7,0x0f,0x97,0xa2,0x2f,0x79,0x56,0xc8,0x7d, -0xe7,0xf1,0xe3,0xa6,0xa6,0x23,0xc5,0xdb,0x12,0x14,0xef,0x48, -0x51,0x22,0x8a,0x77,0x25,0x28,0xde,0x92,0x4a,0xb5,0x50,0x58, -0xed,0xb3,0x4c,0x93,0xf2,0xe1,0xde,0xeb,0xd1,0xd5,0xa5,0xc6, -0xd3,0x80,0xcf,0x1d,0xf1,0xe3,0x69,0x20,0x4f,0x1a,0xed,0xf1, -0xae,0x00,0x9f,0x50,0x26,0x03,0x3e,0xa1,0x42,0x7d,0x7b,0xe6, -0x8d,0x07,0xcb,0x67,0x8b,0x61,0xc5,0xc4,0xcb,0x99,0x79,0x25, -0x59,0xa5,0x76,0x7b,0xe9,0x6c,0xb5,0x18,0xc6,0xe7,0x77,0x77, -0xfc,0xa3,0x7c,0x49,0x20,0x5f,0xb5,0x06,0x28,0xfe,0x51,0x0e, -0x25,0x0d,0x69,0x89,0x51,0x95,0x11,0xfc,0x8d,0x3c,0xe5,0x5f, -0xea,0x89,0x6c,0xf4,0x98,0xa4,0x5f,0x66,0xae,0x61,0xda,0x4b, -0xec,0xb8,0xbc,0x7f,0x9b,0x82,0xf0,0xbe,0x20,0xf9,0x34,0xbc, -0x24,0x05,0x92,0xe6,0xbd,0x84,0x53,0xc2,0xaa,0x86,0x9e,0x97, -0x6f,0xa9,0xe0,0x02,0xa7,0x7d,0xae,0x6b,0xf0,0xdb,0x0c,0x6d, -0xd5,0x29,0xd9,0xb2,0x57,0x9d,0x12,0x56,0x33,0x79,0xcf,0x3a, -0x19,0xda,0xaa,0xb3,0x64,0x1c,0x6e,0xd3,0x5b,0x5c,0xaa,0x1d, -0x18,0x0c,0x5a,0x2a,0x6d,0xc3,0xdf,0x1c,0x81,0xd9,0x1d,0x9e, -0xdf,0x72,0xb1,0xd7,0x80,0x0c,0x98,0x5b,0x53,0xca,0x44,0xd7, -0x04,0x67,0x5c,0x39,0x48,0x99,0xe8,0x2e,0xdb,0x35,0x69,0x32, -0xd0,0xd0,0x8e,0xcd,0x59,0x86,0xda,0x7d,0x31,0x59,0x45,0x37, -0xca,0x17,0x73,0xce,0xfd,0x75,0xaa,0x0b,0x56,0xed,0x4d,0x5d, -0x3c,0x9b,0x37,0xa5,0x4a,0x0f,0x70,0x3f,0xeb,0x50,0x79,0xf0, -0x31,0x99,0xac,0x96,0x57,0x83,0xb1,0x71,0x51,0xcd,0xe7,0xd8, -0x36,0x5c,0xfa,0xef,0xff,0x2f,0x99,0xff,0xae,0xed,0x49,0x6b, -0x8d,0xd6,0xb1,0x47,0x4b,0x04,0x05,0x81,0xf1,0x6e,0x54,0xc1, -0x3f,0xfe,0x87,0x2e,0xd0,0xe2,0x50,0x84,0xe4,0x40,0xff,0xc7, -0xff,0xc0,0xb3,0x75,0xbf,0xa3,0x77,0x05,0x8b,0xc4,0xdc,0x63, -0x7b,0xae,0x58,0x3a,0xb7,0x43,0x57,0x04,0x70,0x6e,0x8f,0x5b, -0xb9,0x09,0x6b,0x93,0xe5,0x9e,0xbc,0xe0,0x88,0xca,0xdc,0x2e, -0x5c,0x97,0x59,0x79,0x7a,0x67,0xe0,0xf0,0x70,0x78,0xac,0x57, -0x73,0x43,0x7e,0x24,0x4a,0xc7,0xac,0x2f,0xe5,0x80,0x26,0xe2, -0x33,0x79,0xa0,0xca,0x6f,0xe1,0x82,0x02,0x71,0xbd,0x95,0x9f, -0xcf,0xa6,0x97,0x7c,0xdf,0xf9,0x1e,0xde,0xca,0xba,0x50,0x99, -0xb7,0xb2,0xce,0x2c,0xb8,0x45,0xef,0x6e,0x64,0xa9,0x63,0x74, -0x45,0xdb,0xf2,0xd9,0x76,0xff,0xbe,0x9f,0xce,0x26,0x98,0x7d, -0x3f,0xf9,0x0e,0x8f,0x4d,0xe2,0x46,0xe8,0x45,0xb6,0xe9,0x31, -0x8d,0xf6,0x1e,0xbd,0xa4,0xca,0xe4,0xa8,0x57,0xc9,0x6a,0x6c, -0x4e,0xe2,0xc5,0x10,0xf7,0xae,0x0b,0x1b,0x9c,0xbc,0x07,0x46, -0x95,0xf4,0xd6,0x42,0x8d,0x28,0x20,0x8b,0xd7,0x8f,0xf1,0x9a, -0xd6,0x7e,0x0d,0x68,0xdb,0xe5,0x1f,0xff,0xd6,0x37,0xbe,0xe8, -0x0a,0x61,0x35,0xad,0x0a,0xe2,0x55,0x81,0x66,0x95,0x93,0xa3, -0x59,0x25,0xdb,0x9c,0x9e,0x2d,0x96,0x57,0x33,0xb0,0x7b,0xe6, -0x57,0x5b,0xb9,0x6d,0x78,0xac,0x18,0x0b,0x43,0xc3,0x30,0xd6, -0xc1,0x72,0x0f,0xe6,0xda,0xe5,0x72,0xc4,0xda,0x59,0x6e,0xa4, -0x1e,0xd0,0xc0,0x7b,0x50,0x5c,0x2a,0x1f,0xae,0xd6,0xdb,0xae, -0xf4,0xb0,0x8b,0x20,0xd9,0x52,0x79,0x6e,0xd5,0xf7,0x68,0xa6, -0x53,0x30,0xd7,0x4e,0x27,0xcf,0xf1,0x00,0xa1,0x65,0xec,0xcb, -0x7d,0xe4,0xca,0xda,0x4f,0x42,0x3b,0x2d,0x99,0x5e,0x79,0x6b, -0x79,0xd5,0x18,0x55,0x5c,0xc6,0xc7,0xc0,0xa4,0xc6,0xb5,0x5a, -0xcb,0x7c,0x3c,0x19,0x85,0x6c,0x84,0xf5,0x2d,0xc1,0x2b,0xd6, -0xfd,0x39,0x22,0x58,0xc4,0x92,0x6b,0x77,0x11,0x40,0xce,0xdb, -0x13,0x8e,0xb5,0xd5,0x4c,0x6f,0xde,0x9b,0x5f,0xcd,0xf2,0x3d, -0x04,0xa4,0x03,0x72,0xcf,0x31,0xd7,0x79,0x58,0x44,0x4f,0xe7, -0x2a,0xd7,0x6c,0x60,0xce,0x16,0x26,0x0a,0x1e,0xe6,0x0c,0x28, -0x48,0xb4,0x82,0xfb,0x91,0x52,0x44,0x8d,0x73,0x1a,0xc3,0xf1, -0x2c,0x8d,0x47,0xb5,0x22,0xc4,0x73,0xca,0xc0,0xf7,0x60,0xe6, -0x76,0x26,0xae,0xb9,0x44,0x0d,0x53,0x1b,0x1c,0xac,0xb0,0x96, -0xcf,0xfd,0x86,0x92,0x25,0x0c,0x45,0x32,0x2c,0x80,0x3c,0xa3, -0xf8,0x86,0x17,0xd0,0x5f,0xa9,0x9d,0xf5,0x0d,0x26,0x88,0x1a, -0xa5,0x37,0x66,0xf3,0x78,0x5a,0x2b,0xe4,0xfe,0x0c,0xa9,0x0a, -0xa2,0x84,0x74,0x82,0x91,0x94,0x5f,0xac,0xa6,0xa3,0x71,0xec, -0xe4,0x52,0x8a,0x18,0x0e,0x68,0x79,0x31,0xb6,0xb3,0x9e,0xcb, -0x34,0x51,0x53,0xb9,0x45,0x02,0x14,0x0c,0xd3,0xa0,0xe1,0x4a, -0xc8,0x50,0x90,0x92,0x12,0xd0,0xa3,0xb3,0xe5,0x6c,0xea,0xd0, -0xf2,0x52,0xa6,0xf1,0x63,0x35,0xa5,0x9d,0xb8,0xf5,0xe1,0x1a, -0x0e,0x1f,0x07,0x09,0x91,0x4e,0x6c,0x32,0x25,0xd1,0xb1,0x24, -0xe9,0x2f,0xc7,0xfc,0xbe,0x08,0x00,0xe0,0x20,0xcf,0xc9,0x0b, -0xb1,0xea,0x9b,0xf8,0x63,0x12,0x2f,0x42,0x3c,0x43,0x68,0x1d, -0x9f,0x88,0x10,0xfe,0xe3,0xb7,0xcc,0x4b,0xa0,0x5a,0xfc,0xd6, -0xf9,0x09,0x41,0x56,0x42,0x9d,0x30,0xd4,0xa9,0xc4,0x77,0xda, -0x97,0x1e,0x6a,0x93,0xe6,0x15,0x1e,0x53,0xeb,0xa6,0xe0,0x58, -0x4f,0x86,0xb8,0xfd,0xa6,0xd7,0x86,0x08,0xe0,0x8d,0x60,0x94, -0x93,0xbb,0x32,0xfe,0xd0,0x0f,0xde,0x48,0x57,0xe5,0x5e,0x9f, -0x5a,0xa1,0xca,0x8a,0x91,0x48,0xc4,0x07,0x79,0x8d,0x6c,0xdd, -0x12,0x59,0x4b,0xac,0xc4,0x8d,0x18,0x88,0x74,0x2e,0x2e,0xd7, -0xd6,0x23,0x3d,0xa8,0x2c,0x2e,0x33,0x3b,0x21,0xf3,0xc5,0x34, -0xb2,0xaa,0x10,0x73,0xf8,0x1b,0x4d,0x1b,0x21,0x9b,0xbc,0x74, -0x41,0x1b,0x26,0xed,0xe1,0x35,0xfe,0xb8,0x8a,0x28,0xa0,0x08, -0xa0,0x8c,0x22,0xab,0xd5,0x6f,0x0f,0x0f,0x33,0x37,0xe5,0x9d, -0xaf,0x48,0x23,0xdc,0xfc,0xe6,0x04,0x3a,0xcd,0xe1,0xe5,0x64, -0x9d,0xd5,0x33,0x97,0xb0,0x55,0x1a,0x1f,0x1f,0xf5,0xea,0x97, -0xeb,0x26,0x86,0x87,0xe7,0xa5,0xdf,0x88,0x90,0xe0,0x95,0xe5, -0xc4,0x17,0xf5,0xcb,0xcc,0xce,0x82,0xa4,0x3e,0xfb,0xe3,0x25, -0x51,0xa8,0xd1,0x69,0xae,0x42,0x21,0x8c,0x3c,0xa0,0x3f,0xaf, -0xfa,0x14,0x97,0xda,0xcd,0x8e,0x22,0x37,0xff,0xf0,0xd0,0xce, -0x0f,0xdc,0xe2,0x01,0xdd,0xd5,0xbe,0x8a,0x12,0x55,0x6b,0xa0, -0x6b,0x4d,0x0e,0xa2,0xe8,0x0a,0x32,0xb3,0x30,0x72,0x2b,0x68, -0xb8,0xf8,0x71,0x77,0xcb,0xad,0xa1,0xe1,0xd6,0x20,0x79,0xcf, -0xcc,0xf8,0x44,0xe7,0x27,0x6d,0xfb,0xe6,0x2b,0xee,0x8f,0x61, -0x70,0x99,0x51,0xbc,0x6e,0x27,0x1b,0x9f,0x29,0x91,0x65,0xe8, -0x3e,0xbc,0x7b,0xae,0xa2,0xd6,0x89,0x7c,0x12,0xd3,0xb8,0xe0, -0xbf,0x1b,0x5f,0x0c,0x38,0xc4,0x00,0x1e,0x96,0x72,0x9a,0xb8, -0xb1,0x52,0x08,0xbf,0x58,0x41,0x8f,0x21,0x6b,0x51,0x2e,0x64, -0x3b,0x51,0x2a,0x3f,0x28,0xc0,0x44,0x01,0x0e,0x22,0x2b,0x89, -0xb0,0x11,0xa0,0xdd,0xd6,0x9b,0xb2,0xb6,0x66,0x2e,0x40,0x09, -0xb7,0x5a,0x06,0xe0,0x43,0x29,0x06,0x17,0xa0,0x8c,0xdf,0x47, -0xeb,0xb0,0x9e,0x85,0x47,0x59,0x78,0x16,0xad,0x5b,0x47,0xeb, -0x56,0x3d,0x6b,0x1d,0x65,0x2d,0x79,0xb7,0xca,0x34,0x22,0x6a, -0x84,0x1d,0xf5,0x0a,0xca,0x46,0xe6,0xad,0x4c,0xde,0x66,0x90, -0x6f,0x22,0xb0,0x27,0x01,0xa6,0x7d,0xa0,0xc7,0x4f,0x68,0x9c, -0x70,0xa7,0x5d,0xf9,0x74,0x0f,0xff,0x03,0x48,0x47,0x8b,0xb8, -0xf6,0xc1,0x97,0x25,0x3f,0x48,0x74,0x67,0x8d,0x90,0x6e,0x3e, -0xea,0x22,0xb9,0x7c,0x40,0x50,0xaf,0x73,0xcf,0xa6,0xf3,0x88, -0xe1,0x64,0xfc,0x03,0xea,0x8a,0x0f,0x84,0x14,0xaf,0x33,0xe4, -0x4b,0x7e,0x1d,0xc8,0x07,0x74,0x3a,0x07,0x3c,0x03,0x4f,0xce, -0x51,0xa3,0x3c,0x7f,0xfe,0x37,0xae,0xae,0x97,0xce,0x1b,0x2d, -0x39,0xaa,0xf1,0x77,0xa8,0xe4,0x4d,0x63,0xd0,0x43,0xd4,0xef, -0xf8,0x8d,0x46,0x3a,0xef,0x28,0xd8,0x7a,0xbd,0x1f,0xe5,0x60, -0xe5,0x6d,0xbd,0x71,0x86,0xc3,0x99,0x47,0x04,0xe2,0xec,0x24, -0x67,0xd0,0xf6,0x46,0x23,0xf1,0x31,0x8f,0x5b,0x48,0xa3,0x98, -0x51,0x25,0xfd,0xbe,0x6f,0xee,0x10,0x8e,0xb3,0x8d,0xbe,0xe5, -0xae,0x95,0x0a,0xef,0x10,0x6a,0x8d,0xd2,0xa9,0xda,0xed,0x20, -0x6d,0xa9,0x09,0x6e,0x7b,0xf8,0xdd,0x5c,0xef,0xb9,0x53,0x1d, -0x9d,0x0b,0x84,0xf7,0xdb,0xeb,0x0d,0xfd,0x68,0x66,0xfb,0x15, -0xcc,0x74,0xc1,0x8c,0x0b,0xea,0x5d,0x0c,0xed,0xbd,0xba,0xdd, -0xf3,0x01,0x9b,0xb6,0xba,0x18,0x8e,0x07,0x69,0x6a,0xe7,0x0b, -0xd5,0x63,0x12,0x89,0x7e,0xad,0xd8,0x71,0x6e,0xb0,0xb7,0x30, -0x8a,0xa0,0x51,0x8e,0x10,0xcb,0xb7,0xb8,0xd7,0xef,0x54,0x95, -0xca,0x39,0x6f,0x5a,0x0f,0xf0,0xa8,0x18,0x74,0xfc,0xca,0x37, -0xcf,0x12,0xfc,0x66,0x75,0x6f,0x8a,0xc2,0x43,0x05,0x03,0x15, -0x90,0x0e,0x23,0x4d,0xc8,0xdc,0x84,0xf3,0xea,0x7c,0x64,0xab, -0x4f,0x74,0x8f,0x2e,0xcc,0x95,0x99,0xe6,0xa3,0x23,0x04,0xd9, -0x54,0xb6,0xbb,0xc4,0xa7,0xf3,0x5a,0x46,0x54,0x13,0xc3,0x32, -0xca,0xd6,0x11,0x86,0x1b,0x08,0x6c,0x0a,0x49,0x2c,0xf3,0x7d, -0x78,0x77,0xe7,0x5d,0x63,0x08,0x0d,0xef,0x94,0x9f,0x9a,0x1e, -0xd0,0x0d,0x4e,0xd0,0xa5,0xa5,0xed,0x18,0x46,0x9a,0x72,0x8a, -0x98,0x73,0x44,0x11,0x33,0xd6,0xf5,0x88,0xa2,0x76,0xd4,0x29, -0x74,0xc6,0xd1,0x50,0x64,0x94,0x10,0x62,0x42,0x88,0x09,0x1d, -0xb5,0x1c,0xc4,0x6b,0x2f,0x47,0xd7,0xfd,0x6d,0xed,0xc4,0x4b, -0xd5,0x66,0x43,0x6e,0x25,0xf7,0xbe,0x74,0x98,0xdd,0xe9,0x7c, -0xb5,0x84,0x39,0x7e,0x22,0x2e,0xa0,0xdd,0x23,0x21,0x4d,0x9c, -0x1c,0x3a,0xb6,0xb3,0x74,0xe9,0x12,0xfe,0x34,0xca,0x4b,0x70, -0x00,0xda,0x81,0xc5,0x33,0xc9,0x06,0x22,0x80,0x2a,0x8f,0x24, -0x56,0x79,0xd8,0x86,0x41,0xd7,0xf9,0x5b,0x6e,0x99,0x05,0x0e, -0xbb,0xa8,0x48,0xcf,0x9b,0xf0,0x0f,0x55,0x39,0x53,0xed,0xa3, -0xd4,0x7c,0x40,0x5d,0x0b,0x2a,0xec,0xeb,0x49,0xc7,0x1f,0x49, -0xf0,0x0f,0xfd,0x3c,0x7f,0x5e,0x4c,0x53,0xb0,0xf4,0xbd,0x91, -0xc0,0x89,0xac,0xeb,0x95,0xe7,0x0e,0x29,0xf7,0xee,0x4e,0x11, -0xc4,0xca,0x26,0x0f,0xba,0xc4,0x15,0xc8,0x70,0xe9,0x21,0xf7, -0x10,0xde,0x34,0x80,0xe1,0x7d,0x15,0x4e,0xb6,0x1c,0xfd,0xe1, -0xe1,0x7d,0xd1,0xc3,0x9a,0xb5,0xf3,0x2f,0xdc,0xe2,0x5c,0x61, -0xf9,0x81,0x42,0x03,0x3c,0xbf,0x50,0x6e,0x5a,0x32,0xd9,0x68, -0x12,0x7c,0x37,0x67,0x35,0x1d,0x64,0x65,0xd6,0x22,0x0a,0x45, -0x3c,0xc2,0xf8,0x2b,0xda,0x76,0x72,0x4e,0x76,0xf5,0x92,0xb1, -0x0f,0x13,0xfd,0x72,0x91,0xd0,0x74,0x65,0x3f,0x43,0xa5,0x5a, -0x70,0x33,0x5b,0x80,0x09,0x9e,0xbc,0x89,0xd3,0x34,0x1e,0xd3, -0x26,0xaf,0x36,0x2b,0x35,0x32,0x98,0xca,0xa9,0xae,0x5e,0xdc, -0x5c,0xc4,0xef,0xd1,0x0d,0x6a,0xcc,0xda,0xbf,0x2f,0x63,0x38, -0xab,0xf9,0x56,0xe7,0x2f,0x64,0x3e,0x3e,0x44,0x45,0x45,0x8b, -0x46,0x3c,0x26,0xe3,0x23,0x74,0x64,0xc9,0xda,0x06,0x91,0x18, -0xae,0xa3,0x1b,0x9c,0xd8,0x87,0x19,0xfc,0x05,0x59,0x44,0xc8, -0x62,0xf9,0x1b,0x28,0x2a,0x1d,0x71,0x2c,0xe3,0x08,0x0b,0x36, -0x86,0x30,0x85,0xe1,0x50,0x1d,0x66,0xaa,0xfe,0xfb,0xd8,0x45, -0xfa,0x7a,0x07,0xc6,0xb1,0x9e,0x44,0x54,0x5c,0x85,0xd7,0xe9, -0x4c,0x60,0xe6,0xfd,0x00,0x33,0xb3,0x66,0xa9,0x34,0x59,0x6f, -0xa8,0x1a,0x14,0x61,0xfe,0x5b,0x0f,0xfb,0xd4,0x78,0x0d,0x67, -0xba,0x55,0x72,0xdc,0xf4,0x2a,0x4f,0xa9,0xd6,0x0a,0x40,0x42, -0xc8,0x45,0x80,0xbe,0x0f,0xcf,0x86,0xbe,0x14,0x39,0x36,0xad, -0xb7,0x74,0xfa,0x7d,0x2d,0xff,0xdf,0xa2,0x30,0x3e,0xad,0xb0, -0xef,0x2f,0x0b,0x06,0xbe,0x5c,0x1e,0xb8,0xe6,0xbd,0x4c,0x34, -0x6f,0x07,0x4c,0xc9,0x0e,0xd2,0x16,0xb2,0xb6,0x9e,0xa2,0x7b, -0x9a,0xf8,0x32,0xb8,0x79,0x85,0xc0,0x4a,0xe6,0xd8,0xe2,0x4a, -0x07,0x5c,0xa1,0x48,0x5b,0xb8,0x2a,0xc2,0x25,0x11,0xac,0x8d, -0x5a,0x9d,0x10,0x5a,0x01,0x33,0xdd,0xe1,0x61,0xdc,0xbc,0x00, -0xfb,0xa4,0xeb,0xa5,0x21,0x7c,0xc7,0xf3,0xe6,0x02,0x20,0xf9, -0x17,0x4c,0xe9,0x3a,0x75,0xac,0x53,0x17,0xbe,0x90,0x65,0xbb, -0x68,0xde,0xa7,0x61,0x37,0x0d,0x9b,0x59,0xbb,0xf1,0x1b,0xda, -0xf2,0xa0,0xbc,0x1b,0x80,0xf0,0x88,0xea,0x88,0xd2,0x56,0x37, -0x6d,0x41,0xde,0x6f,0x68,0xb8,0xea,0xac,0x16,0xa0,0x5d,0xab, -0x82,0x6b,0x2c,0x98,0xa9,0x82,0x83,0x23,0x22,0x51,0x16,0x5c, -0x43,0xc1,0xac,0x65,0xb2,0x5a,0xbc,0x60,0xbd,0x81,0xb5,0x29, -0x9f,0x65,0x8b,0x1b,0x58,0x80,0xd2,0x22,0xaf,0xdf,0x51,0xa2, -0x50,0x39,0x2e,0x43,0x80,0xf6,0x45,0x11,0x6c,0x51,0x02,0xb6, -0x31,0x80,0x91,0xfa,0xe1,0x8a,0x97,0x4c,0x55,0x83,0x16,0x86, -0xa9,0xee,0x50,0x39,0x5c,0xcd,0x37,0x1a,0x06,0x34,0x36,0x95, -0x15,0xe2,0x60,0xba,0xc9,0xb9,0xc9,0x54,0x0d,0x5d,0x30,0x79, -0x62,0x7a,0xc4,0x95,0x1d,0x71,0x14,0x2e,0x79,0xb6,0xb3,0x6d, -0x50,0x63,0xdd,0xbd,0x01,0x58,0x1c,0xf4,0xe3,0x02,0x06,0x84, -0x52,0x7c,0x84,0xb3,0xe0,0xb8,0x33,0x32,0xef,0x61,0x1e,0x24, -0x77,0x77,0x5c,0x6c,0xa4,0xca,0x53,0x99,0x5e,0x02,0x13,0x57, -0xdf,0x78,0xf2,0x98,0xb2,0xba,0x5a,0x49,0x20,0x14,0xdc,0xd8, -0xfc,0x2c,0x2a,0xaf,0x1c,0x2b,0xa7,0x91,0xdb,0x34,0x94,0xf7, -0x83,0xa9,0xef,0x62,0x95,0x9a,0x06,0x64,0xa7,0xf1,0x5b,0x5f, -0xe0,0x5f,0xfc,0xa3,0xfe,0x85,0x34,0x0a,0x8e,0x7a,0xe0,0x4d, -0xcf,0x5a,0x32,0xb8,0x26,0x86,0x83,0x54,0x3d,0x22,0xe6,0xa1, -0xaa,0x84,0x96,0x98,0x2d,0xfd,0x45,0x2e,0x18,0xd1,0x3c,0x90, -0x3e,0x1a,0xf0,0x83,0xd7,0xa8,0xf3,0x50,0xae,0xcf,0xe0,0x07, -0xaf,0xc3,0xe6,0x2d,0xb9,0xde,0x82,0x1f,0x90,0x32,0x5a,0x9b, -0x5b,0x9a,0xeb,0x56,0x03,0x17,0xa5,0xa3,0x2c,0xca,0x5a,0x8d, -0x8c,0x1e,0xac,0x34,0x37,0x38,0x33,0xff,0xeb,0x7f,0xff,0x27, -0x93,0x04,0xf9,0xe1,0x59,0x16,0x74,0x1b,0xbf,0xb5,0x7f,0xeb, -0x14,0x1a,0x96,0x51,0x53,0xb2,0xbe,0x15,0x5d,0xf4,0x9f,0x67, -0xa3,0xb5,0x8a,0x4a,0xb2,0x0e,0xcf,0xd6,0xa5,0x45,0xd7,0xcc, -0x92,0x35,0xf0,0x82,0x8b,0xca,0xba,0x60,0xcc,0x35,0xd6,0x81, -0x7f,0xe4,0x11,0x51,0xfe,0x99,0x24,0x13,0xbe,0xc3,0x46,0x16, -0xf8,0xdd,0xdf,0x70,0x24,0x7e,0x8c,0x6c,0x4a,0x1b,0xa3,0x75, -0x47,0x7f,0x7f,0x44,0xc2,0xbb,0x6e,0x5d,0xc1,0xd9,0x28,0xeb, -0xc2,0xd8,0xcf,0x90,0xce,0xb6,0xf7,0x11,0x1f,0x34,0xf0,0x32, -0xf9,0xca,0x58,0x65,0x83,0xf0,0x62,0x3f,0x07,0xb3,0x91,0x7e, -0x50,0x7e,0x7e,0xfb,0xc0,0xca,0x23,0x6b,0xd0,0xd3,0xc3,0x36, -0xe9,0x17,0x37,0x1b,0xac,0x4c,0x75,0x1d,0x93,0x35,0x68,0xc7, -0x15,0x9a,0x74,0xf7,0xd2,0xca,0xa6,0x8a,0x2e,0x51,0x54,0xae, -0xb5,0xd4,0xbc,0x24,0x27,0xac,0xb6,0x27,0x13,0xee,0xb3,0xde, -0x92,0x45,0x68,0xc9,0xa5,0x8a,0xdf,0x63,0xd5,0xa5,0x8b,0x67, -0xa6,0x78,0x59,0x7c,0xa5,0x1d,0x77,0xfd,0xad,0x26,0x1f,0x1e, -0xf6,0xac,0x2f,0x52,0x64,0xd6,0x67,0xab,0x4f,0x93,0x0f,0xdf, -0x2d,0x39,0xf7,0x0b,0xac,0xd2,0xc2,0xc6,0x9e,0x4e,0x81,0x15, -0xe6,0x28,0x50,0xdb,0x17,0xc6,0x03,0xaa,0xa5,0xbd,0x9e,0x3a, -0x36,0xa6,0xdc,0x8a,0x0d,0x2f,0xc3,0xb3,0xae,0xa7,0xbf,0x2d, -0xfc,0xdb,0x32,0x7f,0xc9,0xb2,0x50,0x6f,0xde,0x31,0x0b,0x0c, -0x2f,0xd2,0xe4,0xe3,0x7e,0xfe,0x10,0x45,0xe6,0x79,0xe7,0xc0, -0x0a,0xf6,0xa2,0x3e,0x07,0xc9,0x76,0x58,0xe4,0xb2,0xc4,0x54, -0x07,0x02,0x74,0x6d,0x79,0x1b,0x96,0x1a,0x2c,0x74,0xe4,0xb1, -0xcb,0x5a,0x61,0x44,0xbd,0xfe,0x7d,0xcd,0x96,0xff,0xaa,0x06, -0xcb,0x4e,0x2b,0xbb,0xc2,0x68,0x19,0x47,0xf9,0xb9,0x5c,0x2c, -0xa2,0xfc,0xbc,0xdd,0x21,0x96,0xf6,0xc6,0xfd,0xde,0xa2,0x0f, -0x0b,0x59,0xeb,0x2b,0xe2,0xdf,0x8b,0x3e,0x7c,0x62,0x7c,0x2f, -0x62,0xab,0xdc,0x48,0x64,0x3f,0x8d,0x36,0x11,0x3e,0xee,0x4b, -0x67,0x0c,0xfe,0x5c,0x40,0x17,0xf8,0xaa,0x17,0x4c,0xef,0x9a, -0x75,0x43,0xae,0x87,0x71,0x63,0x6f,0xf7,0x26,0xb0,0xe3,0x0b, -0x64,0x96,0x33,0x8c,0xc3,0x84,0xb4,0x14,0x65,0xbd,0xb0,0x5b, -0x0c,0xe4,0xe2,0xd6,0x96,0x05,0xb9,0x68,0xf5,0x3e,0xbb,0xdb, -0xfc,0x26,0xe6,0xc3,0xc2,0xad,0x8a,0x74,0xa9,0xf9,0x5c,0x41, -0x56,0x6c,0x32,0x0f,0xab,0xc8,0x84,0xbc,0xf9,0x84,0x61,0xaf, -0x3e,0xd9,0xa1,0xb3,0xa8,0x3a,0x34,0x15,0x34,0xcf,0x7d,0x77, -0x73,0x47,0xa6,0xfe,0x3c,0x9f,0xcf,0xd2,0x64,0x19,0x47,0x9f, -0xc6,0xed,0xda,0xa2,0x26,0x16,0xed,0xda,0xb8,0xb6,0xd1,0x47, -0x0a,0xbf,0xaf,0x06,0xa3,0xe5,0x22,0x8e,0xf3,0x4b,0x0a,0xfb, -0x48,0xc0,0x5a,0x5e,0x28,0x70,0xd9,0x99,0x3a,0x3d,0x99,0xa6, -0x78,0xf7,0x66,0x0a,0xbc,0x40,0x2f,0x64,0xa7,0x30,0x2a,0xd3, -0x24,0xfd,0x69,0xf0,0x13,0xf4,0xc5,0xe1,0xa1,0xfc,0x99,0xf9, -0xb8,0x03,0x89,0x46,0xcb,0x40,0xbe,0x8e,0x3e,0x5d,0x47,0xd3, -0xe6,0x5a,0x4c,0x33,0xf8,0x93,0x29,0xc5,0x79,0x10,0x4d,0xd7, -0xbe,0x6d,0x10,0x4c,0xd7,0x8d,0xb5,0x5f,0x37,0x9f,0x59,0x03, -0x2c,0x84,0x66,0x10,0xfa,0x4c,0xc1,0xf3,0xab,0x64,0x3c,0x2a, -0x25,0xc3,0xa8,0xdb,0x29,0xc7,0x19,0x99,0xf2,0xe4,0xd7,0x81, -0x4a,0xb1,0x46,0xf5,0xcd,0xf7,0x9d,0x5d,0x6c,0x5c,0x42,0x4c, -0x91,0x3a,0x07,0xa9,0xd8,0x55,0x2b,0x1b,0x23,0x58,0x05,0x94, -0x85,0x4a,0x32,0xa1,0xaa,0x19,0x49,0x3b,0x65,0x17,0x82,0x1c, -0x8b,0xab,0x21,0x79,0xe1,0xb2,0x8e,0x9a,0x8f,0x8e,0xc0,0x76, -0xa9,0x83,0xf9,0x2f,0xd2,0x8c,0xbe,0xb2,0xb0,0x8e,0xb4,0x2e, -0x92,0xf7,0x57,0xcb,0x68,0x7d,0x16,0xa5,0x6b,0x71,0x31,0x5b, -0x2e,0x67,0x93,0x28,0x83,0x8f,0x0c,0x86,0x83,0xc7,0xdf,0x5f, -0x7f,0x1d,0xfa,0x75,0x02,0xeb,0x70,0xcf,0x60,0x48,0x3d,0x98, -0xad,0x9a,0xd3,0xd9,0x88,0xd6,0xd7,0xa0,0x31,0xcc,0x87,0xde, -0x42,0x52,0x42,0xf1,0x13,0xbe,0xf3,0xe5,0xcb,0x8a,0xba,0xb8, -0x66,0x59,0xb7,0x71,0x75,0xa2,0xaa,0xeb,0xe2,0xfa,0x27,0x6b, -0xe3,0x4a,0x27,0x13,0x5b,0x24,0x66,0xc3,0x63,0x7b,0x9d,0x8a, -0x2c,0x85,0x49,0x60,0x0a,0x99,0xe7,0x90,0x7b,0x0e,0xd9,0xf0, -0x17,0xfe,0xdb,0x39,0xc2,0x8d,0xf8,0x80,0xc9,0x06,0xa5,0x23, -0x42,0x7f,0x1e,0x51,0x15,0xf0,0x85,0xd5,0xc0,0x57,0x4b,0x3f, -0x69,0xbe,0xa6,0xcf,0xf3,0xa8,0xe1,0xad,0x09,0xec,0x9c,0xaf, -0xdf,0xae,0x69,0x5e,0xc9,0xe8,0x5f,0x77,0xc9,0x4b,0x71,0x83, -0x97,0x05,0x4b,0xcc,0x68,0x89,0x51,0x73,0xfd,0x35,0xe0,0xc2, -0xb0,0xa6,0x80,0x0d,0xbe,0xc0,0xc2,0x6d,0x66,0x5f,0x67,0x94, -0x94,0x51,0x52,0x86,0x49,0xeb,0x33,0xa8,0x1b,0xa1,0x5a,0x06, -0xea,0x2c,0xa3,0xa4,0xac,0x25,0xa1,0xd6,0x52,0x0f,0x53,0x7e, -0xa6,0x3f,0x32,0xdf,0x39,0x1e,0xd3,0x44,0xb0,0x51,0x71,0x1e, -0x81,0x02,0xf3,0x1c,0xb5,0x95,0x61,0x5a,0x46,0x0b,0x96,0x0e, -0x50,0x75,0xb6,0x56,0xe4,0xad,0xd1,0x65,0x1a,0x12,0x32,0x45, -0x5c,0x06,0x09,0xeb,0x73,0x8b,0x34,0x82,0x38,0xb7,0x08,0x23, -0x08,0x49,0x0a,0x65,0xca,0xdf,0x90,0xce,0xfd,0x07,0x22,0xdf, -0x3a,0x6f,0x60,0xd7,0x91,0x65,0x7f,0x0e,0x56,0xf4,0x79,0x67, -0xb4,0x26,0x13,0xb8,0x45,0x3c,0xae,0x8f,0x50,0x3e,0xb0,0x73, -0xe0,0x67,0x46,0xca,0x6b,0x31,0x9b,0x2d,0x2b,0xe4,0x0a,0xfb, -0x14,0xb3,0xf1,0xad,0xf2,0xc8,0x5e,0x40,0x49,0x49,0xc2,0x3c, -0x10,0x26,0x6a,0x34,0x45,0x2c,0x16,0xaa,0xad,0x79,0xf9,0xc1, -0x47,0xd9,0x10,0xd1,0x4d,0x02,0x8a,0xd1,0xa0,0xba,0x54,0x11, -0x4b,0x4c,0xcd,0xbf,0x22,0x84,0x77,0x49,0xe0,0x25,0x58,0x78, -0x1e,0x21,0xfb,0x0e,0xe4,0x8c,0x4f,0x4e,0xe4,0x44,0xe2,0x10, -0x25,0x7b,0x60,0x4d,0x8b,0xa9,0x8c,0xfe,0xcd,0x23,0xc3,0x03, -0x0f,0xd6,0x08,0x24,0x65,0x6a,0xd5,0xa7,0x1a,0xac,0x97,0xa0, -0x20,0x91,0x19,0x9b,0x22,0xd1,0x88,0x55,0x15,0x82,0x6c,0xf8, -0xc5,0x1a,0x14,0x49,0xb1,0xcf,0x59,0x08,0x87,0xbd,0x5e,0x16, -0xde,0xe8,0xf6,0xbb,0x3a,0x08,0xaf,0x61,0xc2,0x73,0x82,0x7d, -0xab,0xfd,0xae,0xf3,0x39,0xef,0xc4,0x09,0x4c,0xdf,0x8c,0x91, -0x84,0x83,0x87,0x19,0x8f,0x39,0xf8,0x17,0x1f,0xfb,0xd6,0xf3, -0x86,0x9c,0x05,0x61,0x01,0xa0,0x92,0xee,0xb3,0x02,0x50,0x65, -0x68,0x09,0xa0,0x11,0xdc,0x63,0x0d,0x60,0x10,0x64,0x16,0x82, -0x7b,0xdc,0xf6,0x50,0xe6,0x3c,0xaa,0x37,0x6c,0x1d,0x34,0xb6, -0x45,0xfd,0x40,0x9b,0x34,0xf7,0x31,0xe3,0x7d,0x8b,0x18,0x25, -0x43,0xec,0xcf,0x6b,0xd9,0xf0,0x64,0xbb,0xf7,0x2d,0x52,0xf7, -0x36,0xd2,0xb7,0x13,0x0a,0x29,0x81,0x24,0xcc,0xa2,0x6a,0x0b, -0x49,0xb4,0xfa,0x15,0xb4,0x16,0xb6,0xc8,0xe1,0xb3,0x2b,0xcb, -0xcd,0xf7,0x97,0xf7,0x17,0xf4,0xb2,0xa0,0x93,0x92,0x83,0xe1, -0x17,0xd4,0x72,0x60,0x3f,0x2b,0x0f,0x5c,0x1b,0xf2,0xa7,0xd5, -0xe4,0x82,0x36,0x71,0xce,0x0b,0x89,0x39,0x48,0x76,0x7c,0xcf, -0x41,0x72,0xa2,0x32,0x8a,0xac,0x8c,0xf3,0x29,0x23,0x3e,0xee, -0x35,0xea,0xfd,0xae,0xd7,0x6d,0xff,0x7d,0x54,0xff,0x7b,0xb3, -0xfb,0xf7,0xd1,0xd1,0x1d,0xfd,0xa9,0xfb,0x90,0xd6,0x8b,0xbf, -0xeb,0x53,0x3e,0xbd,0x97,0x79,0xfc,0xbe,0xe3,0xd6,0x98,0xab, -0xcb,0xa5,0x07,0xfd,0x87,0x7a,0x25,0x57,0x14,0x95,0xe3,0xf6, -0x85,0x1a,0x87,0x35,0x7e,0xba,0x10,0x9f,0x17,0x5d,0xe2,0x4b, -0x76,0x8b,0xf7,0x17,0xe7,0xd3,0xc1,0x24,0x4e,0xf1,0xc9,0x3b, -0x0f,0x8f,0x26,0x8e,0xff,0xe1,0xfd,0xff,0xee,0x20,0xf9,0xef, -0xde,0xdd,0x55,0x3a,0xfe,0xbb,0xe7,0x4b,0x97,0x94,0x0b,0x7a, -0xf9,0xce,0x65,0x74,0xbb,0xac,0xfd,0xed,0x0b,0x34,0x1d,0xc8, -0x97,0x1e,0x6a,0x06,0x88,0xe7,0xb3,0xf1,0x6c,0x51,0x52,0xb8, -0xc6,0x5e,0xd0,0x4c,0x0b,0x59,0xd3,0xcd,0x24,0x65,0xab,0xba, -0x50,0x19,0x25,0xb7,0xcb,0xba,0xb0,0x5d,0xd6,0x5b,0x3e,0xf1, -0x60,0xd3,0xcf,0xf5,0x1a,0x61,0x89,0x4a,0x30,0xab,0x3e,0x8b, -0x07,0x69,0x7c,0x3e,0xe2,0xb7,0x88,0xcb,0x3d,0xbb,0x94,0x37, -0x17,0x9d,0x8f,0x21,0x78,0x89,0xdf,0x96,0x8d,0x85,0xf6,0x59, -0x74,0x22,0x7e,0x90,0x30,0x97,0x3c,0x71,0xa8,0x60,0x30,0x7b, -0x23,0x86,0xab,0x8b,0x64,0xb8,0x05,0x8a,0xf2,0x37,0x78,0xc1, -0x72,0x0b,0x50,0x8a,0xf7,0x80,0xe3,0xf5,0x7c,0x0b,0x08,0xe4, -0xaa,0xeb,0xaf,0xdb,0x6a,0x23,0x00,0xc0,0x85,0xef,0x0d,0x03, -0x5d,0xba,0x34,0x7f,0x8b,0x8b,0xc1,0xf0,0x5a,0x27,0xe2,0x07, -0x3d,0x8a,0x30,0xdc,0x86,0x92,0x01,0xe4,0x4b,0xa1,0x94,0x32, -0x81,0x69,0x57,0x73,0xb3,0x96,0xb0,0x27,0x97,0x8e,0xc3,0x31, -0x5b,0x2d,0x75,0x0d,0xf2,0x3a,0x9d,0x00,0xa0,0x06,0xa4,0xd7, -0xac,0x0c,0x7a,0x9d,0x47,0xd4,0x20,0xb5,0x81,0x18,0xec,0xc9, -0x36,0x47,0x80,0x84,0xf5,0x72,0x48,0x01,0x12,0xdf,0x2f,0xc5, -0xd1,0x47,0xdd,0xab,0x31,0xe0,0x30,0x51,0x0f,0xf9,0xe3,0x6f, -0x5e,0xdf,0xe2,0xbb,0xcb,0x8d,0x1a,0x06,0xaa,0x45,0x37,0x82, -0x2e,0x65,0xb8,0x8f,0x2d,0xb7,0x31,0x4d,0x4c,0x4a,0xf3,0xf1, -0x4d,0xe5,0x36,0x36,0x55,0x87,0xde,0x8a,0x24,0x39,0xfc,0x54, -0x36,0xbf,0x93,0xed,0xc8,0xd3,0x24,0xb2,0xf9,0x45,0x60,0x13, -0x06,0xd3,0xac,0xd2,0xbd,0x36,0x1e,0x4c,0xe6,0xde,0xc4,0x53, -0xb7,0x53,0x68,0xfa,0xe6,0x81,0xa6,0x0f,0xda,0xf9,0x8c,0x94, -0x17,0x97,0x5a,0xa7,0x8b,0x90,0x5e,0x71,0xcd,0x8d,0x9e,0x1f, -0x86,0xe3,0xdc,0xd8,0x81,0x94,0x3c,0x4c,0x5a,0x80,0x49,0xf3, -0x30,0x3f,0x0e,0xf2,0x3a,0x1b,0x52,0xf2,0x7a,0x9d,0x5e,0x51, -0xc9,0xe9,0x0c,0x4c,0x43,0xb8,0x92,0xc8,0x69,0xcc,0x4f,0xee, -0x1e,0x52,0xcb,0xa3,0xd9,0x90,0xda,0xd2,0x1c,0xe2,0xb5,0x92, -0xf8,0xbb,0x71,0x8c,0x5f,0x3f,0xbd,0x46,0x8f,0xd2,0x69,0xda, -0x54,0xef,0x6f,0xdf,0xbc,0x17,0xf8,0x7c,0xba,0x52,0x8f,0xdb, -0x91,0x6b,0x23,0x5f,0x25,0xbc,0x6f,0xa6,0xf1,0xf2,0xd9,0x12, -0xbe,0x2e,0x56,0xcb,0xd8,0xab,0xe9,0xb2,0x35,0x21,0x41,0x3a, -0xac,0x86,0xdf,0x1b,0xb4,0xcd,0x0b,0xe8,0x9a,0x5f,0x07,0x63, -0xf4,0x60,0x4f,0x67,0xe3,0x64,0x44,0xd1,0x3b,0xd5,0x26,0x99, -0x0c,0xf8,0x62,0xe2,0xb8,0x2d,0xbb,0x4b,0x18,0x10,0x80,0x6c, -0xdd,0xb6,0xd3,0x5f,0x68,0xa7,0x52,0x5f,0x91,0x47,0x1a,0xa9, -0x24,0x4c,0x9b,0x7e,0xa5,0xb8,0xa8,0xd0,0x6a,0x26,0xe8,0x65, -0xad,0xce,0xcf,0xb0,0xaa,0x84,0x7a,0xcd,0x97,0x81,0x11,0x64, -0x0e,0x7f,0x41,0x72,0x7a,0x1d,0xdf,0xbe,0x55,0xa9,0xf8,0x81, -0x69,0x14,0xac,0x52,0xa5,0xe1,0x07,0x24,0xd6,0xf4,0xfe,0x40, -0x81,0xec,0xe8,0xd3,0xa0,0x1d,0x8a,0x8b,0x76,0x20,0x86,0xf0, -0xdf,0x08,0x7e,0xc7,0xf0,0xf7,0xb2,0x1d,0x6c,0x72,0xb3,0xde, -0x1b,0xdd,0x1b,0xae,0x2c,0xe8,0x74,0x94,0x87,0xf1,0x20,0x83, -0xf1,0x2e,0x1d,0x73,0xf9,0xa3,0xc9,0xae,0xa4,0x25,0x3a,0x5c, -0xa7,0xd0,0x06,0x92,0x71,0x5b,0xc0,0x40,0x33,0xb4,0xee,0x92, -0x9b,0x35,0xbc,0x1d,0xe5,0xee,0xd6,0x10,0x8c,0x3e,0x55,0xe7, -0x9a,0xce,0xb9,0xa6,0x57,0x18,0xd9,0x8d,0xca,0xe0,0x1e,0xb8, -0x6f,0x07,0x1b,0x4b,0x37,0x36,0x61,0xc3,0xab,0xd9,0x62,0x54, -0x7a,0xd1,0x6e,0x11,0x33,0x88,0x8c,0xb8,0x71,0x0d,0xeb,0xd4, -0x75,0x40,0xee,0x92,0xa0,0x30,0xe8,0xe5,0xe0,0x14,0xdb,0x48, -0xbf,0x5e,0xaf,0x26,0x44,0x2c,0x7d,0xd0,0x03,0xf0,0xe6,0xe8, -0x69,0xea,0xeb,0x02,0x9c,0x23,0xdd,0x9d,0xa8,0x66,0x53,0x8a, -0x7e,0x5d,0x93,0xcf,0x1c,0xf9,0x37,0x18,0x27,0x8a,0x35,0xf9, -0x83,0x49,0xa7,0x07,0x48,0x5c,0xd7,0x23,0x16,0x41,0x3c,0x2d, -0xfb,0xd0,0xef,0x68,0x0a,0xe4,0x6a,0x2c,0x57,0x9f,0x62,0x90, -0x26,0xc7,0x17,0xd7,0xf5,0x68,0xcd,0x8e,0x67,0xb3,0xc5,0xf2, -0x5f,0xa9,0xf6,0xc3,0x43,0xab,0x44,0xf5,0xe9,0x98,0x29,0xe0, -0xe9,0x6a,0x7b,0x83,0xbe,0xe1,0x42,0xef,0x82,0x0e,0xaf,0x11, -0xee,0xb5,0x62,0x13,0x39,0x31,0x58,0xe8,0x0b,0x67,0x5a,0x74, -0x84,0x36,0xda,0x51,0xad,0x46,0xe7,0x99,0xd6,0x43,0xcd,0xe6, -0x83,0x6a,0x86,0xba,0xaf,0x23,0xef,0x8f,0xff,0xd5,0x90,0x0f, -0xa9,0x1f,0x4d,0x31,0x0e,0xf9,0xba,0x94,0xaf,0x41,0xb4,0x76, -0x18,0xcb,0xaf,0x8c,0x25,0x91,0x21,0x95,0x56,0xf2,0x1f,0x22, -0x87,0xfc,0xde,0x88,0xd8,0x2e,0x6e,0x54,0x2f,0xe0,0xf7,0x08, -0x12,0x06,0x88,0x6f,0x00,0xb6,0x76,0x3d,0xba,0x39,0xba,0xee, -0x68,0x29,0x01,0x80,0x3a,0x4c,0x4a,0x75,0x80,0x89,0x3e,0xb1, -0xef,0xe5,0x28,0xc1,0x3e,0x92,0xbf,0x3f,0xa0,0x47,0xdd,0x62, -0xf9,0x8c,0xbc,0x35,0x07,0x81,0x88,0xa7,0x23,0xf9,0x3b,0x64, -0xb7,0xe4,0xf6,0xcd,0x66,0xa3,0x51,0x39,0x38,0x4c,0xb9,0xb5, -0x55,0x6e,0x2d,0x8b,0x79,0x6b,0x3c,0xc2,0x3a,0xbe,0xde,0xa0, -0xbf,0x91,0xe4,0xc7,0x86,0x77,0x0b,0x0c,0x2b,0xf0,0xfb,0x43, -0x94,0xe4,0xb8,0x20,0xaf,0x3c,0x9a,0x36,0x70,0x13,0x3e,0xa8, -0xcd,0x5f,0x2b,0xe7,0x03,0xe5,0x24,0xfd,0x8e,0xbc,0xe3,0xd7, -0xa4,0xca,0xf5,0x4d,0x4c,0xf6,0xb6,0x3e,0x3c,0x64,0x71,0x97, -0x6e,0x2c,0x16,0xe0,0xd7,0x36,0x5c,0x57,0x6d,0x35,0x73,0xa2, -0xda,0x69,0x96,0x37,0x29,0xdb,0x2a,0xd7,0xb9,0x75,0xa8,0xee, -0x3c,0xfa,0x1b,0x94,0x92,0xe7,0x54,0xcd,0xe1,0xe1,0x22,0x26, -0x69,0x72,0x63,0x1c,0x52,0xca,0x27,0x49,0xc9,0x76,0x69,0x63, -0x3c,0x9e,0x37,0x68,0xda,0xc4,0xd6,0x07,0x4d,0xa7,0x59,0xc7, -0x2d,0xe1,0x5d,0xb8,0x10,0x17,0x79,0x08,0x14,0x4a,0x5a,0x9f, -0x13,0x46,0x39,0xd8,0xa5,0xd8,0x8a,0xa9,0x30,0x23,0xca,0x1d, -0x34,0xc2,0x50,0x21,0x58,0x49,0x7d,0xc2,0xf0,0x5b,0xd4,0x89, -0x51,0xa0,0xef,0xa1,0x61,0x8e,0x9c,0x91,0xf6,0x8b,0x2a,0x33, -0x8d,0xe4,0xf8,0x89,0x70,0x3b,0x96,0x7f,0xea,0x5d,0x2d,0x56, -0x49,0x52,0x1f,0x91,0x7d,0x42,0x49,0x7e,0x9b,0xe1,0x36,0xfc, -0xd9,0x54,0x64,0xec,0x55,0xa1,0x02,0x5e,0x6f,0x41,0xaf,0x84, -0x53,0xe2,0x37,0x3c,0xd9,0xaf,0x0a,0x0b,0x7e,0x5b,0x2d,0x06, -0xcc,0xae,0x48,0x73,0x7c,0xff,0xba,0x4c,0x11,0x5d,0x5d,0xbe, -0x1e,0x0d,0x62,0x57,0xc5,0xdd,0xb9,0x7f,0x3d,0x12,0x5e,0x55, -0x62,0xa4,0xda,0xae,0x89,0x81,0x54,0x35,0xc3,0x5c,0x15,0x26, -0x62,0x16,0x65,0xdc,0xdd,0x99,0x69,0x4d,0x62,0x55,0x25,0xf3, -0x4c,0xd0,0x25,0x39,0xc3,0x29,0xe9,0x34,0xcd,0x9e,0x4e,0x41, -0x9b,0x0c,0xcb,0x03,0xc6,0x2c,0xe2,0xf9,0x6a,0x0c,0xf6,0x3c, -0x6e,0x1f,0x17,0x67,0x7f,0x5c,0x65,0xe1,0x26,0x30,0xee,0x88, -0x91,0x65,0x87,0x09,0xbc,0x55,0x7e,0x10,0x45,0x5c,0x46,0x6e, -0x2b,0x52,0xce,0x70,0xdd,0xc0,0xc4,0x26,0xc6,0x1d,0x90,0x29, -0x19,0xa7,0x80,0xbd,0x3d,0x8d,0x42,0xeb,0x29,0xa7,0xd1,0xfa, -0x68,0xb4,0xae,0x8f,0xb2,0xa3,0x11,0x6f,0x0c,0x2f,0xaf,0xe2, -0xe5,0x40,0x9f,0xe7,0x8f,0xa6,0xea,0xad,0x0b,0xc6,0x72,0x85, -0x83,0x17,0x52,0xe1,0xff,0x6a,0x8c,0x11,0xda,0xf9,0xba,0x11, -0x8d,0xd0,0x1b,0x93,0xbf,0x32,0xf8,0xca,0xe0,0xeb,0x20,0xd8, -0x38,0xc4,0x1e,0x1e,0x26,0xe9,0xf7,0xc9,0x34,0x01,0x43,0x0d, -0x30,0x3b,0xa8,0x29,0xff,0xb9,0x8d,0x7f,0x0b,0xe2,0x8d,0xb4, -0x40,0x0f,0x2c,0xaa,0xac,0xa8,0x39,0xa3,0xc5,0xe0,0xfd,0x64, -0x76,0x13,0xe3,0xfe,0x27,0x60,0x26,0x6b,0x03,0xd6,0x4b,0x60, -0x5f,0x03,0x43,0x00,0x89,0xf9,0xce,0x04,0x75,0x09,0xc5,0x7c, -0x9d,0xa0,0x65,0x4b,0x5c,0x84,0xe2,0x62,0x30,0x9e,0x5f,0x0d, -0x84,0xba,0x67,0x9f,0xd2,0xdb,0x39,0x28,0x7a,0xa8,0x6e,0xb0, -0xba,0x94,0x4b,0xa2,0xda,0x21,0x54,0x88,0x13,0xa0,0xc1,0x84, -0x82,0xa9,0xbb,0x46,0x53,0x4f,0x4d,0xd4,0x60,0xb9,0x79,0x0d, -0x7f,0x60,0xee,0xa9,0x61,0x7c,0xa1,0x8f,0x71,0xd4,0x0b,0x45, -0xd8,0x17,0x97,0x8b,0x84,0x48,0x8d,0x9a,0x4f,0xe9,0x6c,0xf0, -0x5b,0x75,0xe3,0xdf,0x58,0x6a,0x84,0xfe,0x47,0x2b,0x8f,0x00, -0x5f,0x4b,0x32,0xca,0x00,0x55,0x9e,0xa4,0x30,0x6a,0x9c,0x04, -0x78,0x46,0x7c,0x83,0x26,0x6c,0x33,0x14,0xd4,0xb3,0x51,0xf3, -0x09,0xb1,0x92,0xec,0x29,0x75,0x36,0xac,0xba,0x92,0x79,0x81, -0x34,0xdb,0x32,0x0a,0x3d,0xe8,0x11,0x37,0x8e,0x80,0xda,0xa7, -0x78,0x9e,0x14,0x3c,0x52,0x47,0x90,0xcc,0x45,0x68,0x9e,0x27, -0xe3,0x2c,0x60,0x4b,0x99,0x77,0x6d,0xfa,0x37,0x0a,0xc0,0xe4, -0x38,0x08,0xc8,0xb6,0xfe,0x1d,0xcc,0xc3,0x99,0x48,0xc5,0x52, -0x8c,0xc5,0x35,0x9d,0x6b,0x4c,0x49,0x76,0xb5,0x8f,0xf1,0xc4, -0xb5,0x63,0xd5,0xce,0xfd,0x44,0xee,0xdc,0xcf,0x22,0x65,0xb2, -0x8a,0x34,0x9a,0x35,0xd5,0x34,0x07,0x3f,0xe5,0x7c,0xb8,0x8e, -0xa0,0x87,0x1b,0x29,0xf4,0x72,0x06,0xbf,0x32,0xf8,0x95,0x09, -0x6f,0xac,0x9e,0x3e,0xc3,0xfb,0x25,0xe3,0x88,0x5b,0xa2,0x7b, -0x13,0x90,0x1d,0x79,0x90,0x6c,0x06,0xc4,0xd8,0xf7,0x1b,0xba, -0xdb,0xd1,0x3c,0x3e,0x1e,0x8b,0xf5,0x51,0x34,0x16,0x19,0xfe, -0x83,0x15,0x00,0x42,0xef,0x3a,0x4a,0x9b,0xb7,0x31,0x1e,0xe5, -0x1c,0xc3,0x8a,0x95,0x7f,0xd5,0x55,0x12,0xd8,0x90,0x58,0x7d, -0x84,0x43,0x00,0xa8,0xa9,0x73,0x81,0xb0,0x71,0x0d,0x32,0xd0, -0xcc,0xea,0x98,0x4e,0x63,0x0d,0x12,0x99,0x1c,0xd9,0x4b,0x3e, -0x3d,0xf3,0x86,0x62,0xd2,0x0b,0xfa,0xc7,0xf8,0x14,0x1d,0xfd, -0x0e,0xf1,0x37,0x19,0xfa,0xd7,0xbe,0x6f,0x6f,0xa3,0xcf,0x22, -0x75,0xda,0x24,0x66,0x58,0x0d,0x58,0x34,0xf0,0xd7,0x87,0x5a, -0x67,0x58,0x8d,0x97,0xc1,0x67,0x06,0x9f,0x58,0x17,0xcb,0x04, -0x15,0xcf,0x89,0xce,0xb3,0x21,0x2c,0x3e,0x57,0xb4,0xae,0xfa, -0x3d,0xca,0x9f,0x7c,0x92,0x42,0x42,0x17,0x66,0x1a,0x0e,0x52, -0xf4,0x7d,0xc7,0x5a,0xf4,0x0c,0x19,0x18,0xb7,0x69,0x1d,0x8f, -0xee,0xee,0x7e,0xe7,0xe3,0x03,0x4f,0xa9,0xb5,0x99,0xaf,0x3c, -0x6d,0x4c,0x39,0x87,0x7a,0x2a,0xd7,0xf5,0x80,0x7c,0xe8,0xce, -0xf9,0x1a,0x1b,0x80,0x3f,0x32,0xbf,0x8d,0x69,0x8d,0xc8,0xc3, -0xd4,0x06,0xfd,0x1b,0x61,0x1b,0xfd,0x23,0x35,0x84,0x10,0x94, -0xf3,0x33,0xca,0xc7,0x72,0x99,0x95,0xef,0x77,0x58,0x48,0x51, -0xac,0x95,0x94,0xf2,0xb0,0xb4,0xc4,0x14,0x0f,0x8a,0x59,0xfa, -0x79,0x70,0xec,0x67,0x2b,0x10,0xe8,0x9a,0x0b,0xfa,0x6d,0xfa, -0x54,0x68,0x72,0xfe,0x1b,0xdb,0xd0,0x30,0xa8,0x46,0x23,0x5d, -0x05,0x18,0x8d,0xbb,0xa1,0xbd,0x75,0x42,0x44,0x48,0x8d,0x04, -0xbf,0x6c,0x52,0xbe,0x2d,0x44,0x14,0xd9,0x45,0x91,0x2e,0x51, -0x12,0x59,0x63,0xdd,0x5d,0xb7,0xeb,0x0e,0xc1,0x0a,0x5a,0xd5, -0x69,0x22,0x98,0x14,0x48,0xb0,0xa8,0xd2,0xca,0x6c,0x6f,0xaa, -0x74,0x89,0xbd,0xa8,0x52,0xd0,0x8a,0x2a,0xad,0x75,0xf7,0xaa, -0x4f,0x43,0x1b,0xa4,0x2a,0x49,0x21,0x94,0x8a,0x76,0x2f,0x74, -0x12,0x76,0x27,0xe1,0x72,0x36,0x93,0x35,0x28,0xfd,0xbd,0x57, -0x15,0x0a,0xd8,0x20,0x93,0x29,0x0a,0x1b,0x4f,0x01,0x7b,0xe1, -0x62,0x50,0x83,0x89,0xbe,0x15,0x1e,0x56,0xed,0x7b,0xe1,0x59, -0x23,0x0e,0x82,0xef,0x72,0xa9,0xf5,0x59,0x00,0xed,0x0d,0xda, -0x6b,0xf2,0xef,0xe3,0xb1,0x49,0xd3,0xa5,0x1a,0x9c,0x72,0xee, -0xb4,0x27,0x91,0x35,0x6d,0xbc,0x36,0xe9,0xc9,0x40,0xcf,0xcc, -0x52,0xbe,0xaf,0xa8,0xe3,0x41,0xac,0x06,0x0d,0x3d,0x87,0x50, -0x66,0x65,0x91,0xb3,0x07,0x2d,0xa9,0x01,0xd3,0x14,0xe3,0xbe, -0xd0,0xa4,0x6c,0xf6,0x55,0x60,0xc9,0x81,0x1a,0xfc,0x02,0x8f, -0x28,0xd4,0x2f,0x2f,0xf1,0x69,0x1d,0x2c,0x26,0x3a,0xc9,0xda, -0x6a,0xa1,0x8b,0x16,0x96,0x13,0x87,0x81,0xc1,0x3b,0x73,0xba, -0x9e,0xbe,0xaf,0xa6,0xcc,0xb5,0x13,0xc3,0x70,0x31,0x98,0x8e, -0x66,0x13,0xcf,0x3f,0xa2,0x11,0xab,0x09,0xd5,0x75,0xb3,0x83, -0x88,0xc6,0xc9,0x94,0x1a,0x1a,0x7b,0x7d,0xba,0x0b,0xa7,0xae, -0xad,0xd9,0x75,0xab,0x8d,0x94,0x0f,0x34,0x83,0x72,0x3e,0x19, -0x5c,0x6a,0x1a,0xfd,0xd0,0xef,0x18,0x78,0x35,0x9d,0x3a,0xfe, -0xcb,0x6a,0x62,0xf5,0x85,0x0d,0x28,0x17,0x70,0x2e,0x20,0x97, -0xf6,0x37,0x66,0x6b,0x50,0x15,0x48,0xfa,0x64,0x54,0xe1,0xce, -0x90,0x4e,0x04,0x33,0x60,0xeb,0xd4,0x2f,0x6e,0xd5,0x2c,0x28, -0xae,0xd4,0x1c,0x58,0xf0,0xd4,0x74,0xe6,0x1d,0xbe,0x4a,0x08, -0xf6,0x85,0x9c,0x82,0x01,0x6e,0x8b,0xf9,0x50,0xe3,0x23,0x2e, -0x2b,0x2c,0xac,0x6c,0x00,0x48,0xa4,0xfa,0x29,0x91,0xab,0xcf, -0xbe,0x5f,0x56,0x8a,0x79,0x41,0xa5,0xe4,0xca,0x5f,0x95,0xe2, -0x4f,0x28,0x55,0xaf,0x6b,0xde,0x32,0x69,0x94,0x22,0x99,0xc8, -0x29,0x85,0xa6,0x59,0x53,0x23,0x8b,0x15,0x4e,0x79,0x54,0x0b, -0xba,0x86,0x4b,0x19,0xae,0xad,0x6b,0xe2,0x16,0x86,0x80,0x82, -0xc8,0x18,0x22,0xb3,0x20,0xb2,0x9a,0xb8,0xb2,0x20,0xe6,0x12, -0x89,0x9c,0x43,0x4d,0xba,0x2c,0x2a,0xe7,0x4e,0xba,0x14,0xa0, -0x2c,0x20,0x14,0xb1,0x12,0x9d,0x65,0x6b,0x74,0x3f,0xcf,0x69, -0xdb,0x7c,0x8a,0xea,0x36,0xa8,0xe5,0x17,0xa6,0x3b,0x23,0xc9, -0x39,0x6a,0x94,0x62,0xb1,0x91,0x20,0x7d,0xda,0x80,0xdb,0x42, -0x9f,0x52,0xff,0x05,0xfa,0x6c,0xeb,0x8f,0xe9,0x53,0xa0,0xf7, -0xa1,0xcf,0xc1,0x62,0x23,0x31,0xe6,0x56,0x15,0x75,0x96,0x31, -0xe6,0x78,0x2a,0x73,0x21,0xa2,0x8a,0x7f,0x5b,0xf4,0x18,0x89, -0xa8,0x70,0x6c,0xb1,0x4a,0xf3,0x4f,0xd7,0xac,0xd7,0x4b,0x1c, -0x67,0xc9,0x53,0xb6,0xe7,0x6c,0x54,0xbc,0xd7,0x0c,0x7d,0xa3, -0x55,0x67,0xf3,0x1d,0xd0,0x81,0x06,0xc6,0x25,0x54,0xd9,0x91, -0x24,0x24,0xdf,0xdd,0x79,0x94,0x0b,0xfa,0xfc,0x22,0xbe,0x82, -0xe9,0x69,0xb6,0x20,0x70,0xcf,0x97,0x91,0x84,0x9d,0x30,0x14, -0x4d,0x14,0x63,0xcc,0x26,0xa5,0xce,0x0b,0xe8,0x9a,0xc8,0x59, -0xb0,0xdf,0xaa,0x7c,0x03,0xae,0x21,0xe5,0x52,0xd0,0xe4,0xc0, -0x0a,0x65,0x0b,0x1a,0xc8,0x85,0x01,0x53,0x36,0x1d,0x26,0x29, -0xa1,0x98,0xcc,0x56,0x69,0x0c,0x08,0x17,0x15,0x48,0x5e,0xaa, -0x7c,0xdf,0x02,0x5f,0x55,0x51,0xfe,0x52,0x66,0xfb,0xdc,0xd1, -0x48,0x9f,0x2f,0x6e,0x66,0xc9,0xe8,0x41,0xe0,0xb7,0xf1,0x8b, -0x76,0x0f,0x16,0x31,0x6e,0x86,0xf2,0xc4,0xc7,0xab,0x4d,0x51, -0x03,0x71,0x32,0x2e,0x8d,0x55,0x4b,0xc6,0xa8,0x15,0xe4,0x6b, -0xb4,0x17,0x8a,0x51,0xd8,0x31,0x7b,0x13,0x57,0x49,0xbc,0x18, -0x2c,0x86,0x57,0x59,0xc5,0xfe,0xc4,0x70,0xb5,0x90,0xfb,0x13, -0x62,0x14,0xcf,0x41,0x47,0xf3,0xca,0x80,0x6f,0xac,0xa0,0xdf, -0x5d,0x1a,0xd1,0x1f,0x40,0xcd,0x8d,0xd1,0x08,0x85,0x29,0xc4, -0xbe,0x67,0xb8,0x94,0xa7,0xcf,0xc8,0xc2,0xc4,0x33,0x09,0xed, -0x65,0x08,0xc6,0x07,0x3a,0x69,0xca,0x38,0xb5,0x07,0x8e,0x99, -0xa9,0xd1,0x35,0x8e,0xd6,0x44,0x43,0xd2,0x95,0x4d,0x55,0x37, -0x8e,0xb8,0x1b,0x3a,0x12,0x20,0xdc,0x75,0xbd,0xdc,0x18,0x45, -0xaa,0x0d,0x8c,0x12,0x87,0xd2,0x07,0xd9,0x08,0xdc,0x15,0x18, -0x2c,0x70,0x15,0x4f,0xa4,0x0e,0xd5,0xd5,0x45,0x71,0x53,0x8f, -0x46,0xbc,0x55,0xd9,0xc1,0xcd,0xa4,0xc3,0xc3,0x21,0x6f,0x8c, -0xe2,0x3f,0x3e,0xef,0x23,0x23,0x95,0x48,0x00,0x7d,0x44,0x37, -0xd2,0xa9,0xb1,0x24,0xab,0x2e,0x83,0x68,0x54,0xf2,0xe6,0xee, -0x2e,0xf0,0xed,0x5d,0x15,0x7b,0x73,0x96,0xca,0x5a,0xdc,0xb7, -0xf8,0x8e,0x8d,0x76,0x58,0x80,0xed,0x67,0x1d,0xc4,0xdf,0x86, -0x8f,0xd8,0x37,0x8a,0x93,0x86,0x91,0xc4,0xc6,0x02,0xbf,0xa0, -0xe9,0xaa,0x5a,0x55,0x96,0x58,0x26,0xb5,0x8f,0x6a,0xe0,0xcd, -0xfd,0xda,0x55,0xca,0x32,0x71,0x63,0x9a,0xaa,0x31,0xe0,0xe6, -0x0d,0xd1,0x27,0x77,0x2b,0x14,0x06,0xd5,0x8b,0x23,0x11,0xa8, -0xde,0xe3,0x95,0x17,0x6f,0xcc,0x73,0x0c,0x79,0x29,0xf1,0x1a, -0xd9,0x6b,0x48,0x17,0x9a,0x5b,0x25,0x00,0xcf,0x35,0xef,0x88, -0xa6,0x12,0x88,0x5f,0x49,0x0a,0x24,0x11,0x3a,0x95,0xc4,0x61, -0xff,0x2d,0x4b,0x58,0xa1,0xe9,0xa2,0xbc,0x3f,0xb9,0x31,0x09, -0x46,0x86,0xf7,0x5c,0x50,0x48,0x68,0x07,0xa7,0x4a,0xb5,0xf1, -0x72,0x9b,0xf6,0x42,0xca,0xa0,0x0e,0x46,0x4a,0xb2,0xd1,0x49, -0xb1,0xb0,0x5f,0x71,0x98,0x19,0xff,0x6b,0x25,0x34,0xe4,0xdc, -0x17,0xf8,0xec,0x81,0x67,0x4a,0xdb,0xdb,0xa1,0x30,0xe6,0x96, -0x89,0xbb,0x20,0x2b,0x33,0xd6,0x49,0x94,0xd6,0x82,0xa2,0xa8, -0x6e,0x93,0x7b,0xad,0x5b,0xc8,0x9f,0x98,0x76,0x3b,0x23,0xa3, -0x6b,0x30,0xb8,0x34,0x7f,0xad,0x31,0xf4,0x33,0xff,0xcc,0x22, -0x48,0xdd,0x3e,0x4e,0x58,0x08,0xe9,0x72,0x38,0xed,0x5e,0xd0, -0x0e,0xca,0x3a,0x32,0x02,0xdc,0x1d,0xad,0x8f,0xcd,0x57,0x5b, -0xdf,0xd8,0xd7,0xf4,0x0f,0x23,0x7b,0x00,0x41,0x2b,0xa2,0x21, -0xc3,0x1e,0x71,0x93,0xf0,0xfc,0x69,0x64,0x6f,0x5f,0x22,0xb9, -0x9e,0xd9,0xd5,0xad,0x18,0xe5,0xa3,0xfb,0x8f,0x72,0xa3,0x0b, -0x69,0xf1,0x31,0x19,0xa0,0x37,0x28,0x57,0x67,0x91,0x68,0x0e, -0x84,0xc3,0xfa,0xc8,0x90,0xa5,0x7b,0x8b,0x0f,0x24,0xcd,0xd8, -0xb4,0x04,0xd8,0xf1,0xfa,0x37,0x37,0xa6,0xac,0x8e,0xa4,0x67, -0xaf,0x03,0xa1,0xcc,0x7b,0xb5,0xc1,0x65,0xda,0x8c,0x00,0xbe, -0x3d,0xa6,0xcd,0xc4,0x54,0x32,0x59,0x79,0xf6,0xb6,0xaa,0x39, -0xc7,0x96,0x94,0xde,0x7f,0xff,0x44,0x17,0x55,0x7b,0x28,0x25, -0x9a,0xe0,0x17,0x9e,0x8e,0x35,0xa8,0x35,0x58,0x1c,0xe9,0x4e, -0xca,0xb7,0xfa,0x21,0xdd,0xba,0xcf,0x43,0x82,0x50,0x76,0xe7, -0xd8,0xba,0x38,0x69,0x6b,0x58,0x28,0x4d,0xcc,0xa5,0x30,0x33, -0xf5,0xb2,0x67,0xd8,0xcc,0xb1,0x67,0xd7,0xfc,0xac,0x78,0xea, -0xc2,0x00,0xd0,0xc1,0x70,0x19,0x3a,0x75,0x6e,0xda,0x55,0x3f, -0x2a,0x50,0xa9,0x6c,0xbf,0x31,0xc0,0xc0,0x68,0x4d,0xb0,0x2e, -0x59,0x9b,0xa4,0x32,0xa4,0x8e,0x39,0xf3,0xb7,0x5c,0xc5,0xfd, -0x8e,0x74,0x52,0xa1,0xd9,0x35,0x31,0xe7,0xeb,0xa4,0x2d,0x23, -0x4b,0x11,0x43,0xc3,0x51,0x8b,0x7f,0xc3,0x8a,0xb8,0xab,0xf9, -0x04,0x0b,0x4b,0xdd,0xad,0x5c,0x1f,0xac,0x6c,0x1b,0xf2,0x17, -0x2c,0x3e,0xdb,0xa5,0x90,0x54,0x87,0xf6,0x46,0xc7,0xbf,0x1f, -0xfa,0x2a,0x88,0x15,0x74,0x65,0xea,0xbc,0xd5,0x51,0x7a,0x2a, -0x2f,0x87,0xc0,0xa8,0x83,0xe0,0x68,0x74,0x7f,0x42,0x34,0x6d, -0x85,0x93,0x95,0xc1,0x28,0xd2,0x84,0x38,0xc7,0xd8,0xd6,0x29, -0x36,0x8c,0x7d,0x3c,0xd0,0x40,0xa3,0x73,0xc8,0x02,0xcf,0xda, -0x55,0x7a,0x72,0xe6,0x66,0x34,0x97,0x0b,0x16,0xce,0xc8,0x9c, -0x70,0x47,0x7f,0xfc,0x2f,0x3d,0x12,0x92,0xf8,0x73,0xb4,0x3f, -0x14,0xd3,0x7a,0x1f,0x51,0xdc,0x7b,0x92,0x23,0x04,0x3c,0xbd, -0x51,0x79,0x43,0xe6,0x7e,0x58,0x0c,0xbc,0xc2,0xa5,0x53,0x18, -0xa3,0x6e,0xeb,0x5e,0xf8,0x34,0xb4,0xc4,0xa6,0xbe,0x09,0x57, -0xde,0x8a,0x76,0x59,0x1c,0x7d,0xda,0x58,0x46,0x32,0x87,0xec, -0x2a,0x1b,0xd5,0x94,0x43,0x12,0xc5,0xd2,0x2e,0x1d,0x05,0xe2, -0x45,0xb2,0x6b,0x74,0x4b,0xa9,0xe5,0xe1,0x4d,0x68,0xd4,0x00, -0x97,0xf7,0xb7,0x18,0x49,0xd5,0xd5,0xe8,0x51,0xee,0x06,0xb8, -0x41,0xdc,0x5b,0xdb,0x38,0x6f,0xe8,0x8e,0x43,0x3e,0x85,0xef, -0x53,0xd3,0x35,0xec,0x34,0xe2,0x1b,0xdc,0x16,0x84,0xbc,0xed, -0xc4,0x0d,0xea,0xa8,0xc6,0x9c,0x34,0xe7,0xf1,0x62,0x82,0x3e, -0x64,0x9c,0x22,0x8b,0x6b,0x7a,0x2d,0x00,0x89,0x40,0x02,0x74, -0xd4,0x46,0xd0,0x4c,0xcc,0x2e,0x2f,0xd3,0x18,0x60,0xf9,0x6f, -0x65,0x9d,0x62,0xaa,0x5a,0xaf,0xf7,0x89,0xf8,0x1b,0x03,0x69, -0x59,0xc7,0x44,0xf6,0x26,0x17,0x7e,0x93,0xef,0x92,0xc1,0xa9, -0xcb,0xa0,0x67,0xca,0x4c,0x56,0x8a,0x7a,0x42,0xe8,0xb0,0x5c, -0x1c,0xe1,0xca,0x17,0x76,0xb8,0xb3,0x72,0x2c,0xec,0xdf,0x32, -0xab,0xeb,0x2b,0xe0,0x8d,0x50,0x96,0x16,0xe6,0x9a,0x3e,0x63, -0xd3,0xd7,0xdd,0xf0,0x36,0x9a,0xad,0xed,0x6d,0x57,0x51,0x79, -0x15,0x5f,0x89,0x1f,0xd5,0xf5,0x33,0x26,0x7e,0x2b,0x9d,0x28, -0x99,0xdc,0x02,0x08,0xa5,0xfe,0x16,0x2f,0x66,0xe8,0x6d,0x5a, -0xc8,0x5d,0xc9,0xbb,0x0e,0x76,0xa2,0xbc,0xa5,0x60,0x27,0xe9, -0xf8,0x4f,0x1c,0x09,0x4b,0xd2,0xb7,0xbf,0xb2,0xc0,0x83,0x0b, -0x2a,0x2a,0xf5,0x45,0xba,0xe1,0xcf,0x26,0x37,0x6b,0x2f,0x44, -0x0c,0x5a,0xb1,0x31,0x5e,0xc6,0x98,0x94,0x1c,0x49,0xd7,0xec, -0x48,0x5a,0xc9,0x36,0x49,0x16,0x61,0xd7,0x54,0x31,0x2b,0xf7, -0x23,0x8b,0x61,0xf7,0xa4,0x8b,0x45,0xaa,0x92,0x30,0xd3,0x59, -0x8a,0x2c,0x4a,0x51,0x74,0x59,0xde,0x29,0x1f,0xb7,0x6e,0xa6, -0x7f,0x54,0xe5,0xd7,0xaa,0x68,0xb6,0x5f,0xd1,0xcc,0x14,0xcd, -0x34,0x37,0x56,0xcb,0xfd,0x0a,0x23,0xa0,0x2e,0x0e,0x1f,0x12, -0x41,0x5e,0x73,0x5a,0x3d,0x64,0xbb,0x46,0x63,0xb4,0x1f,0xf6, -0x7c,0x2e,0xb9,0xcf,0x4c,0xfb,0xc2,0xce,0xe5,0x31,0xb0,0x4a, -0x8d,0xc2,0xcc,0x21,0x7f,0x39,0x58,0xbf,0x60,0xc5,0x90,0xa2, -0x4f,0x61,0x15,0xd8,0x2f,0xf1,0x68,0x35,0x8c,0x5f,0xaf,0x26, -0x05,0xd3,0x63,0xba,0x2d,0xb8,0x05,0xd4,0x8c,0xa1,0x2d,0xf0, -0xcf,0x05,0x85,0xf7,0x99,0xcd,0x31,0xfc,0x12,0xdf,0x06,0x0c, -0xf0,0x93,0x36,0xf4,0x38,0x21,0x35,0x81,0xd9,0xcc,0x1e,0xdc, -0x87,0x28,0x51,0x8e,0x71,0x0c,0x75,0x06,0x85,0xba,0x1e,0xfc, -0x53,0x8f,0x90,0x62,0xd4,0x1c,0x88,0x86,0x37,0x18,0x3e,0xe0, -0x4d,0x1f,0x86,0x33,0xd9,0x12,0xbb,0x86,0x50,0x03,0x54,0xa5, -0x9b,0xd7,0x59,0xe4,0xe3,0x2e,0x88,0x0f,0x2f,0x6d,0x71,0x7a, -0x9e,0xc7,0xe6,0xc2,0x6d,0xd1,0xf6,0x32,0xb8,0x36,0xa2,0x26, -0x5d,0xb6,0x6b,0xa5,0x23,0x4e,0x8e,0x29,0xf6,0x7f,0x2f,0x91, -0x7b,0xdd,0xdd,0x69,0x32,0xbe,0x9a,0xad,0xe2,0xe5,0xb2,0x40, -0x88,0xd1,0xfd,0xb9,0xee,0xe6,0x3b,0x78,0x5a,0x9f,0x73,0xcf, -0xf6,0xfa,0x24,0x07,0xf4,0x3c,0x49,0xd9,0x41,0x86,0x0a,0x7a, -0x39,0x93,0xcc,0xc7,0xa0,0x7e,0xa0,0x92,0xa5,0xf5,0xc5,0xea, -0xb7,0x33,0x3b,0x03,0x1c,0xb0,0x2e,0x42,0x4c,0x33,0x16,0x19, -0x79,0x5e,0xe1,0x6f,0xf2,0x18,0xb3,0x00,0xcf,0x4c,0x10,0xb4, -0x21,0x7b,0xc2,0x3f,0x6e,0x29,0xde,0x67,0xc1,0x46,0xdc,0x26, -0xef,0xdf,0x8f,0x2b,0x5b,0x05,0xb3,0x03,0x45,0xc5,0x49,0x4f, -0x70,0x9d,0x0a,0x2a,0x39,0xc8,0x35,0x73,0xad,0x9a,0x09,0x0d, -0xd6,0x1e,0x65,0xa6,0x71,0x19,0x46,0x09,0x89,0x60,0x6a,0x0a, -0x68,0xef,0x2a,0x2c,0xb6,0x34,0x0d,0x2d,0x39,0x4b,0xc3,0x7c, -0x5b,0x35,0x18,0xbd,0x75,0x0b,0x2b,0x60,0x19,0xd3,0x11,0xfe, -0x36,0xe8,0x76,0x95,0xb9,0x00,0x89,0xa0,0xe8,0x4a,0x9b,0x9e, -0x44,0x9e,0x83,0xa4,0xa1,0xbf,0x1a,0x7c,0x09,0xeb,0xd8,0x6b, -0xc1,0x1a,0xd5,0xef,0x24,0x67,0xd7,0x50,0xf4,0xda,0x4f,0x4f, -0xea,0xb2,0xc8,0xb5,0x53,0xe4,0xda,0x2a,0x32,0x5a,0x77,0xd2, -0x16,0x88,0xf3,0xc9,0x91,0x83,0x7b,0xc3,0x0c,0x9e,0x35,0x28, -0x56,0x4f,0xeb,0x38,0x0d,0x01,0x73,0x1b,0x3a,0x30,0x38,0x9b, -0xe1,0xe1,0x43,0x10,0x55,0x75,0x4a,0x03,0x98,0x62,0x77,0x44, -0xbc,0x9e,0x0f,0xec,0x37,0xae,0xef,0x29,0x5e,0xd7,0x51,0x78, -0x3c,0xfd,0x53,0x62,0x05,0x8b,0xee,0x99,0xb3,0x77,0x8f,0x50, -0x0e,0xc8,0x71,0x34,0x2b,0xee,0xd1,0x17,0xa0,0xa2,0xeb,0x0a, -0x29,0x74,0x9a,0xfb,0x11,0x26,0x8d,0xf2,0xd9,0x06,0xa7,0x13, -0xbe,0xfb,0xa1,0x57,0xc5,0xe9,0x12,0x03,0x61,0x4f,0x4a,0x2d, -0x54,0x9d,0x2b,0xad,0x54,0xb3,0x97,0x0a,0xcb,0x59,0xb1,0x16, -0xf0,0x2f,0x8d,0xbb,0xfc,0x8a,0x94,0xbe,0x17,0x82,0x43,0xd9, -0x91,0x12,0x89,0xe8,0xdf,0x85,0xb5,0xdc,0xe7,0x32,0x68,0x61, -0x2e,0xaf,0x16,0x71,0x7a,0x35,0xc3,0xed,0x60,0x40,0x38,0x75, -0xa0,0xa8,0x98,0x05,0x2b,0x9d,0xd7,0xa5,0xfd,0xab,0x3b,0xcc, -0xa0,0xd0,0xe1,0xc1,0xa0,0xd7,0x2e,0x17,0xf1,0xef,0xab,0x78, -0x3a,0xcc,0xba,0x61,0x1b,0x7a,0x90,0xb6,0x32,0x26,0x1d,0x1f, -0x2a,0xc1,0x8a,0x38,0x28,0x41,0x1f,0x1b,0x81,0x1b,0x3c,0x06, -0x45,0x2f,0xa9,0x83,0x94,0x7a,0x98,0xee,0x26,0x83,0xad,0x87, -0x89,0x19,0xef,0xa2,0x4c,0xce,0x02,0x3f,0xef,0xa0,0xb2,0xb6, -0x16,0x6f,0xeb,0x33,0x6e,0x34,0x08,0xd2,0xe1,0xe1,0xfa,0x6b, -0xf9,0x81,0x31,0x6d,0x3d,0x4d,0x01,0x9e,0x68,0x24,0x14,0xac, -0xce,0xd4,0x03,0x7c,0x0d,0xc5,0x84,0x22,0xf3,0x51,0x6d,0xf5, -0xe8,0x9a,0x7e,0x98,0xc0,0x9d,0xf6,0x7e,0x0b,0xa2,0x21,0x3b, -0x51,0x37,0x16,0x63,0x4d,0x70,0x07,0xa8,0x05,0x21,0xf3,0xde, -0xd9,0xa1,0x94,0xfd,0xfa,0x0b,0xb1,0x97,0xb9,0x5e,0x96,0xff, -0x4d,0x32,0x7d,0x0d,0xd5,0xbc,0x8f,0x53,0xb3,0x8d,0x29,0xb3, -0xee,0xbd,0x48,0x5c,0xd0,0x1e,0xa1,0x2c,0x2d,0x6d,0xbf,0xc5, -0xc6,0x24,0xf1,0x6c,0xb3,0x1f,0x42,0xd3,0x22,0xeb,0xd2,0xba, -0x85,0x9c,0xf3,0x6d,0xe4,0x24,0xaa,0x7b,0xe1,0x96,0xdc,0x28, -0x1c,0xdd,0xae,0xcd,0x06,0x02,0xa1,0xb7,0x2f,0x5c,0x95,0xf0, -0xed,0x7b,0x74,0x4f,0x62,0x48,0xb1,0xf6,0x37,0xed,0x4a,0x5a, -0xb9,0x3e,0x9b,0x56,0xd3,0x95,0x7b,0x7a,0x9d,0xe8,0x9e,0x3f, -0x70,0x58,0xac,0x33,0x2c,0xe4,0xf6,0x76,0x93,0x1b,0xa7,0xc2, -0x1a,0xf7,0x1c,0x91,0x82,0xc6,0xbb,0x4a,0xba,0x4c,0x16,0xe9, -0xf2,0x6f,0x83,0xf1,0x35,0xef,0xad,0xce,0xc1,0x0e,0x48,0x66, -0xab,0xf4,0x75,0x72,0x31,0xd6,0x17,0x93,0x2a,0x76,0x1d,0xe5, -0xf5,0x15,0x4a,0x3c,0x47,0xd4,0xbb,0xf7,0x20,0xad,0x33,0x1b, -0x55,0x13,0x6d,0xb7,0xf3,0xc6,0xab,0x20,0x62,0x28,0xc1,0xec, -0x93,0x06,0x18,0xe5,0x6c,0x18,0xa7,0x14,0x3b,0x5f,0xe7,0x3b, -0x7b,0x98,0x43,0xb7,0x04,0x8c,0x51,0xd3,0x2a,0x46,0xec,0x54, -0xe6,0x1b,0x84,0x83,0xf9,0x1c,0x4c,0x40,0xda,0x99,0x2d,0xc2, -0x69,0x30,0xdf,0x4d,0xe7,0x9a,0x3a,0x46,0x3c,0xb0,0xed,0xaf, -0xaf,0x92,0xcb,0x25,0xef,0xd4,0x92,0x29,0x3c,0x49,0x46,0x1c, -0xbf,0x02,0x03,0x4b,0x18,0xba,0x99,0x51,0x78,0x8d,0x6b,0x9c, -0x4c,0xea,0xc3,0x42,0x92,0xdf,0xc9,0xf5,0x40,0xd7,0x53,0x7b, -0x88,0x94,0x1f,0xe5,0xb2,0x5d,0x7c,0x69,0x3c,0x1f,0x2c,0x06, -0x66,0xa7,0x3c,0xdf,0x9b,0xb2,0xcb,0x9a,0x93,0xd9,0x28,0x72, -0xd0,0x36,0x14,0xb9,0x7e,0xdb,0xad,0x4e,0xa5,0xf3,0x51,0x56, -0x0e,0x1f,0x7a,0x51,0x7e,0x39,0xe2,0xac,0x1b,0x08,0xa0,0x39, -0x67,0xd3,0x91,0x91,0x4a,0x18,0x25,0x72,0x37,0xdf,0x08,0x9b, -0x42,0xbb,0xee,0x6c,0x3f,0x06,0xd8,0x6b,0x4b,0x5f,0x6f,0xe7, -0xaf,0xeb,0x76,0x15,0xc0,0x27,0x25,0x63,0x16,0x49,0xce,0x16, -0xbe,0xfd,0x44,0xa7,0x91,0xa6,0xb2,0x06,0x1a,0x71,0x42,0x47, -0x9e,0xc2,0x40,0x53,0x23,0x23,0x45,0x41,0x12,0x37,0xc9,0x9c, -0x0f,0x20,0x6f,0x66,0xea,0x47,0x91,0xbf,0x90,0x39,0x61,0x72, -0xf9,0xc8,0xb2,0x69,0x0f,0x9a,0x14,0x30,0x00,0x16,0xd3,0x12, -0x91,0x02,0x2a,0x40,0x67,0xa7,0x00,0x4e,0xc0,0xeb,0xc0,0x40, -0xca,0xcc,0x4a,0xe9,0x60,0xbd,0xae,0xa8,0xff,0x82,0x8e,0x2a, -0x1e,0xa4,0xfb,0x44,0xa6,0x9b,0xf9,0x63,0x7c,0x89,0x79,0x73, -0xcc,0x9b,0x1c,0x1e,0xc2,0xaf,0x8e,0x8f,0x64,0x96,0x41,0xcd, -0x10,0xc3,0x2c,0x8f,0x41,0xa2,0x9f,0xcd,0x29,0x53,0x52,0xa2, -0x47,0x2c,0xf1,0x82,0x78,0x64,0x51,0xae,0x24,0x0c,0x04,0xd9, -0x34,0x59,0x4a,0x36,0xf0,0xc1,0x16,0x3d,0x28,0x24,0x88,0x3c, -0xc2,0x41,0x8e,0x67,0x6e,0xf5,0x2f,0xc9,0x5b,0xdc,0x49,0x7a, -0x26,0x2b,0xa7,0xc2,0x44,0x80,0xd1,0x0c,0x86,0x1e,0x3c,0x7e, -0xc0,0xb5,0x1d,0xf5,0x5f,0x3a,0x53,0x3f,0x31,0x19,0xd6,0x74, -0x39,0x36,0x23,0x64,0xbe,0x77,0x10,0x6a,0xe6,0x76,0x06,0x42, -0xd9,0x3d,0xd6,0x21,0xa6,0x1e,0x54,0x31,0x0c,0x0f,0x61,0x35, -0x38,0x5a,0x1e,0x83,0x51,0x44,0x0d,0xb6,0x71,0x00,0x5d,0xc8, -0x16,0xe2,0x6f,0x32,0x2f,0x60,0xd3,0x5d,0x43,0xc8,0x26,0x79, -0x64,0x73,0xe1,0xd0,0x88,0xc8,0xe6,0x80,0x6c,0x22,0x9c,0x1e, -0xd2,0xf7,0x42,0x55,0xea,0x66,0xf7,0x19,0x11,0x1d,0x0f,0x46, -0xea,0xdc,0x27,0xa7,0x5e,0x29,0xc0,0x08,0xbd,0xe5,0xc1,0xa7, -0x89,0xe6,0x76,0x73,0xe9,0xac,0x65,0xc6,0x70,0xf4,0x49,0x91, -0xd0,0x56,0xa0,0x20,0x15,0xb0,0xda,0x00,0xe2,0xf1,0x1a,0xe7, -0x15,0x4e,0xe4,0xf0,0x83,0xfa,0x0a,0xfe,0xb2,0x75,0xd0,0xce, -0x6b,0xe1,0x72,0xcd,0xc6,0xc0,0xf5,0xb0,0x1d,0xe0,0xae,0xbd, -0x99,0x74,0xe8,0x14,0x54,0x58,0x2a,0x83,0x68,0x6e,0x50,0x90, -0x12,0x57,0xdb,0x53,0x3c,0x32,0xe0,0x78,0x6e,0x0c,0xbc,0x8e, -0x91,0x43,0x32,0x2a,0x4a,0xa1,0x7b,0x26,0xb3,0x74,0xa9,0xc2, -0x14,0xed,0x5b,0x8e,0x84,0x84,0x0b,0x8e,0xe2,0x38,0x3f,0xe6, -0xaa,0xcb,0x7d,0x0b,0xc0,0x31,0x96,0x5a,0x07,0x11,0x12,0x8a, -0x5e,0xf5,0x66,0x34,0x61,0x0a,0x13,0x82,0x37,0xbc,0xd6,0x61, -0x44,0xbf,0x9b,0x6b,0x7b,0xc4,0x51,0x92,0x40,0x48,0x84,0xc9, -0xc2,0x08,0xeb,0xe7,0x83,0xd8,0xbb,0xbb,0xb0,0x53,0x30,0xb6, -0xca,0xba,0x1b,0xbb,0xee,0x75,0xf2,0xd1,0x3a,0xeb,0xe1,0x33, -0x51,0x9e,0x15,0x8e,0xb4,0x8b,0x60,0xf1,0xa8,0x57,0x9e,0x2a, -0x42,0xa3,0xc7,0xf1,0x32,0x7e,0x60,0x04,0xc3,0x3a,0x0d,0xb2, -0x71,0x45,0xf2,0xdc,0x98,0x2e,0x04,0xaa,0xd8,0x96,0x95,0xe8, -0x8f,0xb3,0x70,0x4b,0x0d,0xf7,0x38,0xba,0xe4,0xad,0x28,0x3c, -0x00,0xf3,0x9b,0xd2,0xc1,0x03,0x7f,0x0b,0xc3,0xc7,0x42,0x8f, -0xa9,0x0c,0xfb,0x32,0x89,0xe2,0x53,0x74,0xa0,0x19,0xcb,0xd1, -0x50,0x0c,0x9a,0xfd,0x4e,0x5d,0x0c,0xfc,0x5a,0x70,0x9c,0x13, -0x93,0xb4,0x11,0x25,0x11,0x56,0x76,0x79,0xc7,0x13,0x55,0x1c, -0x2b,0x45,0x1e,0xb0,0xfa,0x12,0xb1,0xee,0x5b,0x0a,0xa0,0xc2, -0x47,0xad,0x3c,0xba,0x54,0xb1,0xfb,0x57,0x72,0x50,0x59,0x09, -0x26,0x53,0xcc,0x96,0x6d,0xe7,0xb9,0x58,0xa6,0xea,0x28,0xb7, -0xea,0xd4,0x67,0x4e,0x87,0x3e,0xfb,0x1d,0x87,0xbb,0xaa,0xae, -0xcc,0xc5,0x55,0x2c,0x24,0xb3,0xf0,0x09,0xc7,0x55,0xda,0xd5, -0x97,0x52,0xda,0x25,0x9b,0xd0,0x12,0x86,0xff,0x94,0x84,0xae, -0xe0,0x8c,0x8d,0x0e,0x0c,0xb5,0x8e,0xe8,0x4f,0x16,0x05,0x62, -0x4f,0x2d,0x4b,0xb7,0xa7,0x16,0x51,0x7d,0xe1,0x49,0x1f,0x28, -0xdf,0xd9,0xff,0x2b,0x2b,0x6b,0x9d,0xa2,0x01,0x67,0xa4,0xde, -0xc4,0x93,0x21,0xbe,0xc1,0x28,0x8f,0x4c,0x17,0xea,0xba,0xed, -0x91,0x27,0x5b,0x11,0xb6,0xb5,0x43,0x42,0xeb,0x88,0xe8,0x5c, -0x1c,0xdf,0x0a,0xfd,0x13,0x6c,0xb6,0xe3,0xd6,0xbe,0x93,0x03, -0x93,0x5d,0x8f,0x46,0x8b,0xcf,0x26,0xf7,0x5e,0x35,0x35,0xa8, -0xa6,0x4d,0x41,0xa5,0x21,0x4a,0x7d,0xf3,0x9f,0x4b,0xde,0x82, -0x2e,0xbc,0x82,0xff,0x74,0xab,0xe5,0x4d,0xbc,0x2d,0xed,0xb6, -0xb4,0x09,0x97,0x12,0x7b,0x29,0x95,0x5c,0xcb,0xc8,0x5d,0x4d, -0x5f,0x8f,0x2d,0x75,0x99,0x18,0x5e,0xdf,0x23,0x7c,0x12,0x41, -0x9e,0x8b,0x39,0x1d,0x08,0xf0,0xe8,0x25,0x0c,0x4c,0xe2,0x7e, -0x38,0x24,0xac,0x8c,0xc4,0x74,0x77,0x57,0xfa,0x84,0xeb,0x79, -0xbb,0xae,0xaa,0x91,0x02,0xcd,0x15,0x15,0xee,0xd8,0x6e,0x7d, -0x3e,0x59,0x02,0x6b,0x54,0xfa,0x42,0xed,0x36,0xef,0x8e,0xe1, -0x75,0x85,0x36,0x18,0x8e,0x57,0xf8,0xe6,0x5c,0xa9,0x42,0x90, -0x79,0x46,0x27,0x28,0x1b,0x02,0xc3,0xb4,0x89,0x7b,0x29,0x88, -0x75,0x14,0xdc,0xc7,0x1e,0xda,0xea,0x19,0x65,0xd6,0x43,0xb9, -0xc5,0x50,0x57,0x79,0x4c,0x99,0x9a,0x64,0x13,0xde,0xea,0x25, -0x8f,0xaf,0xa6,0xc0,0x02,0xcc,0x3b,0x03,0xe3,0xb7,0x15,0x26, -0xbb,0xc5,0x5d,0x58,0x5a,0x55,0x2d,0x01,0x31,0xdf,0x6f,0x07, -0x0a,0x79,0xe0,0x64,0x48,0x83,0xb2,0xd4,0x56,0x92,0x95,0x93, -0xcd,0xca,0x46,0x57,0xde,0x2a,0x92,0x10,0x6c,0x23,0x33,0xc8, -0x97,0xb1,0x64,0xbe,0x84,0xe1,0xa2,0x8d,0x0d,0x56,0xcb,0x05, -0x43,0x83,0xd5,0xb5,0xba,0xaf,0xeb,0x2b,0x53,0xe3,0x4f,0x98, -0x2e,0x5e,0xd8,0x90,0x48,0xbb,0x9c,0x72,0xcc,0x5f,0xed,0xd0, -0x37,0xe8,0xff,0x4b,0xd8,0x2d,0xb2,0xe3,0xfe,0x9c,0xe9,0x22, -0x91,0xb8,0xd6,0x8b,0xc6,0xfc,0xa7,0x0d,0x18,0x8d,0xbe,0xcc, -0x86,0x51,0xd5,0x7c,0x19,0x33,0xa6,0x58,0xd5,0x5e,0x96,0x8c, -0x2c,0x56,0xa1,0xbe,0xb0,0x43,0x26,0x83,0x79,0xb9,0x17,0x0b, -0x85,0xc6,0xd1,0x9b,0x7d,0xd7,0x66,0x87,0x82,0xb7,0xca,0xf0, -0x35,0x1f,0xb5,0x6d,0x9f,0xd3,0x24,0xfa,0x81,0x1f,0x7c,0xb6, -0xc8,0x2b,0xec,0xce,0x49,0x49,0x39,0xf2,0x82,0xb3,0xeb,0x6e, -0xd0,0xbe,0x96,0x4e,0xde,0xfc,0x3c,0x12,0x5f,0x44,0xc1,0x9f, -0xe8,0x2c,0x7c,0x16,0xe1,0x2f,0x00,0xa2,0x27,0x8c,0x0c,0x6d, -0xbf,0xaf,0x06,0x8b,0xe4,0x32,0xdb,0xad,0xed,0x9c,0x0d,0xa0, -0xfc,0xee,0x8f,0x29,0x28,0xd2,0x21,0x3e,0x24,0x39,0x15,0x0b, -0x8c,0xb8,0x07,0x73,0x82,0xf4,0x3e,0x5f,0xcc,0x6e,0xf1,0x28, -0x61,0x01,0x5c,0x4a,0xa6,0x38,0x6f,0x68,0x14,0xea,0x89,0x9e, -0x0b,0x58,0x1c,0x61,0x0c,0x54,0xb1,0x8a,0x6a,0x94,0x86,0x61, -0xe1,0x30,0xc2,0x55,0x17,0x51,0x35,0x47,0xeb,0x76,0x6d,0x54, -0x4c,0xcd,0xda,0x0c,0xdc,0x70,0xf2,0xc2,0x43,0xb3,0xb2,0xd0, -0x80,0x0a,0x0d,0x5b,0x07,0x09,0x68,0x21,0x4e,0x10,0x12,0x40, -0x3e,0x60,0x46,0xbd,0xa5,0xe9,0x94,0x99,0xeb,0x23,0x09,0x64, -0x79,0xa5,0x52,0xa3,0x9a,0xf2,0x45,0x29,0x6f,0x1a,0xe9,0x32, -0x8a,0x2b,0x67,0x41,0xc7,0x47,0x10,0x3a,0x97,0xe0,0xce,0xd3, -0x30,0xf4,0x70,0x91,0xc1,0x50,0x8f,0x4c,0xcf,0x89,0x9a,0xea, -0x95,0xda,0x01,0xb7,0xe7,0xee,0xce,0x23,0xb6,0x46,0xb7,0x33, -0x58,0x18,0x83,0xda,0xb9,0x15,0x2b,0xdf,0xff,0x3a,0x42,0x96, -0x75,0x0d,0xad,0xcd,0xf9,0x6c,0xae,0x18,0x49,0xf0,0x30,0x7f, -0xa8,0x0a,0x1a,0x11,0x51,0x82,0x00,0x5c,0x89,0x76,0x20,0x25, -0x6c,0xd4,0x4c,0x71,0x10,0xfa,0xc0,0xfd,0x4a,0xfe,0x10,0xb5, -0xf2,0x05,0x25,0xd3,0x74,0xdd,0x71,0x7e,0xc7,0xe4,0x1f,0x1e, -0x7a,0xa5,0x15,0x04,0x15,0x48,0x7c,0xed,0x30,0xac,0x9d,0x02, -0x15,0x13,0xec,0x7d,0x41,0x0c,0x35,0x77,0xfd,0x45,0xc5,0x35, -0x2f,0xa6,0xd5,0xf2,0xc9,0x12,0xfc,0x67,0x64,0x24,0x27,0x02, -0xdc,0x1b,0x79,0x11,0xa9,0x90,0x09,0x56,0x62,0x9c,0xf0,0x31, -0xcf,0x5d,0x99,0xac,0x07,0x8a,0xa4,0x43,0xf2,0xbc,0x20,0x97, -0x15,0x5d,0xd0,0x29,0x76,0x81,0x64,0xb7,0xdd,0x05,0xb6,0x08, -0x6a,0x45,0xb6,0x10,0xa9,0xc6,0x24,0x16,0xec,0x64,0xb0,0x00, -0x11,0xa2,0xf1,0x2c,0x55,0x9b,0xa9,0x54,0x5f,0x72,0x5f,0x60, -0x22,0xbd,0x4f,0x88,0x5a,0x0a,0x5a,0x85,0x65,0xce,0x16,0xf2, -0x47,0xb4,0x00,0x4a,0xcf,0x16,0xec,0x6e,0x40,0x48,0x17,0xe6, -0x54,0x2f,0x3d,0x8a,0x52,0xb1,0x3a,0x8a,0x56,0x42,0x2e,0xe9, -0xd0,0xdc,0x5f,0x1d,0x21,0xd8,0x11,0x4d,0x4f,0xc7,0xa9,0x48, -0x8f,0x29,0x25,0x99,0x72,0x0a,0x98,0x53,0x40,0xce,0xa6,0xe8, -0x0b,0x6f,0x09,0xe8,0x25,0x68,0x79,0x29,0x1f,0xb3,0x22,0xe1, -0x60,0x3e,0x12,0x6b,0x31,0x16,0x02,0xfd,0xc8,0xc4,0x4d,0xb4, -0xea,0x2e,0x30,0x2c,0x9d,0x1e,0x69,0xc7,0x2b,0x30,0xc0,0x50, -0xfe,0x56,0x51,0x24,0x3b,0x84,0x19,0xe5,0x11,0xf2,0xbb,0xbb, -0x9b,0x33,0x35,0xa0,0xe8,0xaa,0x87,0xd6,0x3e,0xfa,0x0a,0x3f, -0x33,0x05,0xc3,0x0f,0x44,0x7c,0x69,0x3f,0x83,0x7f,0x47,0x59, -0x74,0x83,0x4e,0xed,0xb3,0xa6,0x7a,0xc9,0x44,0x0f,0xd1,0x75, -0x5d,0xd6,0xd3,0x58,0x8b,0x1b,0x49,0xcd,0x8c,0x69,0xb9,0x01, -0x5a,0xfc,0xce,0xac,0xf9,0x11,0xcf,0x3c,0xb1,0x68,0x3d,0x2a, -0x14,0xe1,0x96,0xd4,0x01,0xbd,0x24,0xa5,0x11,0xdd,0xf0,0xbb, -0x24,0x65,0x54,0xaf,0x6d,0xaa,0xd7,0x3b,0xa9,0x5e,0x03,0xda, -0x8c,0xa8,0xce,0x72,0x54,0x67,0x75,0x55,0x5d,0xb6,0x85,0xea, -0x90,0x9a,0x2e,0xa9,0xb6,0x8b,0xc8,0x66,0x68,0xaa,0xd7,0x48, -0xf5,0xc6,0x39,0xa0,0x23,0x27,0x26,0x7b,0x17,0x81,0xe5,0x39, -0x4e,0xef,0xee,0xec,0x1b,0x31,0xb9,0xed,0x52,0xb5,0xd4,0xe7, -0xf5,0x7d,0x20,0xf4,0x0a,0x9f,0x7e,0x8c,0x74,0x2c,0x09,0xf9, -0xad,0xe3,0x49,0x08,0x85,0xfd,0xf0,0xb0,0x70,0xad,0x43,0xed, -0x63,0x92,0xde,0xe8,0xe1,0x87,0x2a,0xbe,0x3e,0x92,0x68,0xd8, -0x7c,0x94,0x4a,0xc3,0x53,0xb8,0xba,0x6a,0x08,0xb6,0xb5,0x3a, -0x54,0xb8,0x30,0x23,0xa3,0x67,0x39,0x19,0x34,0x2a,0x5e,0xdc, -0x91,0x39,0xbb,0x56,0xb7,0x82,0x98,0xcf,0xdd,0x43,0x3f,0x6d, -0x83,0x52,0xad,0xf3,0x28,0xda,0x10,0x7c,0xe4,0xac,0x50,0xe0, -0xf1,0xab,0xc1,0xe8,0x27,0xcc,0x64,0x8a,0xb0,0xcb,0x28,0x8e, -0xa7,0x99,0xc5,0x04,0x0d,0x3f,0x3a,0xbf,0x0b,0xeb,0x26,0x28, -0xc8,0x23,0x33,0x9a,0x25,0xa6,0xfb,0x5f,0x2d,0x90,0x05,0x7d, -0x6b,0xb7,0x0b,0xf1,0x94,0x45,0x8b,0xb2,0xf6,0x97,0x46,0xdf, -0x17,0x17,0x79,0xf3,0x48,0xbd,0xbd,0xc0,0x38,0xf8,0x3c,0xc2, -0x98,0x11,0xe6,0x16,0x19,0x59,0xae,0xf3,0x6e,0x15,0x1f,0x18, -0x67,0xbb,0x2c,0x9b,0xd7,0x6a,0x85,0x73,0xf3,0x79,0xb7,0x37, -0x17,0xf4,0xbf,0x7e,0x7b,0xee,0xdb,0x97,0x43,0x46,0xcf,0x67, -0x14,0x48,0x77,0xe9,0xc6,0x30,0xaa,0xc6,0xbd,0xf6,0x37,0xdb, -0x5e,0x92,0x91,0x7c,0xe1,0x98,0x94,0x50,0xb9,0xd9,0xa9,0x18, -0x29,0x8b,0x5c,0xc7,0xce,0xf2,0x2b,0x5b,0xe8,0xec,0x9c,0x79, -0xf4,0x58,0xa7,0xf2,0x00,0xf0,0xbb,0x16,0x77,0xdb,0xa6,0xa5, -0x04,0x80,0xbe,0x96,0xbd,0xb5,0xa0,0xff,0x91,0x60,0xa9,0xd6, -0xd1,0xe6,0x81,0xfa,0x50,0x9d,0x68,0x7a,0x93,0xc5,0x73,0x3f, -0xb7,0x07,0x02,0x5d,0x77,0x8d,0x2c,0xb7,0xa5,0x8b,0x87,0x96, -0x14,0x4a,0x3d,0x90,0x9e,0x1f,0xa6,0x12,0x29,0xbd,0x7b,0xba, -0xd2,0x13,0xec,0x5a,0x98,0xb1,0x87,0x03,0xc0,0x08,0x23,0xe5, -0x5b,0x0d,0x20,0xf9,0xdf,0xd3,0x6f,0x03,0x41,0x2d,0xc1,0xa6, -0x04,0x83,0x8a,0x86,0xd6,0x5e,0x98,0x08,0x72,0x5d,0xaf,0xd5, -0x0c,0x2e,0x4c,0xda,0xb5,0x5b,0x8b,0xa2,0x9f,0x5b,0xe3,0x70, -0x3c,0x85,0x08,0x16,0xc7,0x30,0xbd,0x8e,0xcd,0x92,0xf9,0x7f, -0xff,0x53,0xfc,0xf1,0x3f,0xd5,0x9b,0x60,0x79,0x02,0x94,0x68, -0xb5,0xce,0xf0,0xc0,0xf8,0x8f,0xff,0x19,0x81,0x6d,0x19,0xd2, -0xef,0xff,0xfd,0x4f,0xb4,0xf7,0xac,0x95,0x12,0x07,0x89,0x00, -0x95,0xde,0x19,0xcd,0x1e,0xac,0xa3,0xd6,0x91,0x13,0xc5,0x01, -0xe6,0xe0,0xac,0x24,0x6d,0xa1,0x42,0x10,0x75,0x6e,0xc1,0x6c, -0x89,0xbd,0x83,0xc5,0xdd,0xdd,0xe2,0x2c,0xd4,0xc3,0xf4,0x7f, -0xff,0xb3,0xfe,0xc7,0xff,0x3c,0x5a,0x1f,0x19,0x7d,0xd3,0x90, -0x48,0xc6,0xb3,0xf7,0xde,0xc2,0x3f,0x5e,0x80,0x55,0x23,0xe0, -0xf7,0x4f,0xb9,0x56,0x31,0x3d,0xb2,0xcd,0xba,0xf5,0x4d,0x6e, -0xbc,0xbc,0xa7,0x33,0x3a,0xb1,0x6e,0xe9,0x74,0xf2,0x91,0xc5, -0xdc,0xf7,0xe2,0xe2,0xf5,0xdc,0x53,0x74,0x63,0x95,0xc9,0xe2, -0x36,0x99,0xfe,0x00,0x9a,0xc6,0x54,0x39,0x29,0x46,0x27,0xb3, -0x8e,0xb1,0xe5,0xdb,0xcc,0xea,0x31,0xc7,0xb4,0x1e,0x39,0xbc, -0xd0,0x16,0xd1,0xf1,0x64,0xc3,0x71,0x46,0x69,0x9a,0xc1,0x3b, -0x16,0xea,0xf5,0x78,0xfc,0x3e,0x9f,0x82,0x19,0x6b,0xa2,0xaf, -0x5e,0x8e,0x67,0xb3,0x85,0x13,0x5e,0x79,0x18,0x27,0x63,0x3b, -0x81,0xae,0x68,0x50,0xd1,0x26,0x07,0xb6,0x2e,0x0f,0x87,0xcd, -0xc8,0x19,0xc2,0xc3,0x67,0xe4,0xfb,0x82,0xff,0x75,0xc3,0xb5, -0xe2,0xda,0xc2,0x10,0x87,0x7d,0xb0,0x15,0x1d,0x74,0x51,0xae, -0x76,0x58,0xb7,0x40,0x73,0xc1,0x90,0xe5,0x4a,0x40,0x98,0x02, -0x58,0x4e,0x08,0x9c,0xa4,0x82,0xbe,0x9f,0x6b,0x2b,0x94,0xff, -0x1e,0xfb,0x8b,0x22,0x88,0x5d,0xd2,0x2f,0xaf,0xd6,0x0c,0xe2, -0x1a,0x6d,0x43,0x6b,0x98,0x9f,0x80,0x25,0x3f,0xc5,0xef,0x61, -0x84,0xdd,0xc4,0x8a,0x25,0xc5,0xb1,0xd5,0x20,0x76,0x23,0x7b, -0xbc,0x06,0xe8,0x57,0x66,0x54,0x15,0x18,0x21,0x21,0x38,0x8b, -0x7f,0x73,0x58,0x3a,0x6c,0x69,0x2d,0x64,0x17,0x5b,0x2b,0x42, -0xe6,0xa3,0xcd,0x35,0x14,0xe3,0x52,0x44,0xa6,0x1e,0x60,0x13, -0x08,0xdc,0x6c,0x8a,0x4f,0x64,0x35,0x1f,0xd9,0x85,0xf9,0x45, -0xe6,0xf1,0x36,0x42,0x24,0x88,0x07,0x0b,0xf5,0x4f,0xcb,0x76, -0x8d,0x5c,0xb3,0x6a,0x62,0xd0,0xee,0xf5,0xfa,0x78,0xa7,0xc5, -0xe0,0x1a,0x42,0x8f,0xbe,0x9f,0x2d,0xb2,0x30,0xd8,0x46,0x8e, -0x42,0xe7,0x2b,0x37,0xe9,0x93,0x73,0x53,0xb0,0x0c,0x5d,0xeb, -0x73,0xd1,0xb5,0x2a,0xd0,0x5d,0x7c,0x36,0xbe,0x8b,0x72,0x84, -0xc3,0xcf,0x46,0x38,0x34,0x32,0x6a,0x71,0xaf,0xd7,0x0a,0x4e, -0x5b,0xad,0x53,0xe8,0xe8,0xd3,0xc7,0x0f,0x1f,0xc2,0x87,0x68, -0x3d,0x6d,0x3d,0x7c,0xf4,0xe4,0x89,0x08,0x1f,0x06,0x27,0x0f, -0x1f,0xb7,0x9e,0x88,0xa7,0x8f,0x5b,0xf0,0xfd,0x48,0x3c,0x0d, -0x9f,0x3e,0x0e,0x4f,0x40,0xdc,0x1f,0x3e,0x0d,0x1e,0x9f,0x9c, -0x04,0xe2,0xc9,0xc9,0xa3,0x47,0x8f,0x43,0x48,0x68,0x9d,0x9c, -0x3e,0x0d,0x9f,0x9c,0x8a,0xf0,0xd1,0xa3,0xd3,0xf0,0xf1,0xa3, -0xbe,0xf2,0xde,0x5f,0xbc,0xbf,0xe0,0xb8,0xcf,0x24,0xf5,0x16, -0x93,0x4d,0xad,0xe1,0xc3,0x47,0x0f,0x1f,0x3e,0xb4,0xaa,0x87, -0x1f,0x8f,0x9e,0x9e,0x3e,0x6e,0x19,0x3a,0x82,0x00,0x70,0x3f, -0xb5,0x08,0x42,0x90,0xe0,0xf1,0xc3,0x96,0xa6,0x2c,0x6c,0x3d, -0x7d,0xf4,0xe8,0xc9,0x69,0x68,0x48,0x6c,0x41,0x72,0x88,0x68, -0x15,0xad,0xe1,0x69,0xeb,0xe4,0x61,0xf8,0xf4,0xa1,0x21,0xfa, -0x24,0x78,0xda,0x7a,0x12,0x3c,0xb6,0xa8,0x07,0x3a,0x9e,0x3c, -0x7a,0xf2,0x54,0x35,0x03,0x6a,0x7e,0x18,0x06,0x50,0xc5,0xce, -0xf6,0x5c,0x44,0xbd,0x13,0x24,0xe9,0xf1,0x63,0xf1,0xe8,0xe4, -0x29,0x14,0x7e,0x2a,0x1e,0x07,0x0f,0x83,0xc7,0xf0,0x37,0x0c, -0x5a,0xa7,0x0f,0x5b,0x80,0xfd,0xf4,0x51,0xf8,0x34,0x78,0xfa, -0x58,0x3c,0x6d,0x85,0xa7,0x8f,0x80,0x8e,0x30,0x7c,0x1a,0x3e, -0x0a,0x43,0xa8,0x05,0x08,0x3a,0x3d,0x3d,0x39,0x85,0x9c,0xa0, -0xf5,0xf4,0xe9,0x09,0x10,0xf4,0xb0,0x75,0xfa,0x24,0x40,0x3a, -0x80,0x2c,0x68,0x06,0xfe,0x80,0xb2,0x27,0x2d,0xf1,0xe4,0xf4, -0xf4,0x34,0x3c,0x85,0x1c,0x28,0x74,0x0a,0x4d,0x43,0x9e,0x3c, -0x7c,0x7a,0xfa,0xf0,0x04,0x41,0x1e,0x3f,0x3e,0x01,0xb6,0x41, -0x93,0x1e,0x9f,0x3e,0x81,0x84,0xe0,0xc9,0xc9,0xc3,0x93,0xd6, -0x43,0xc4,0xdf,0x7a,0xf2,0x08,0xd1,0x01,0x3b,0x9f,0x9e,0x3e, -0x7a,0xb8,0xb3,0x39,0x43,0x68,0x4e,0xeb,0xe1,0x43,0xe0,0x1b, -0x34,0x03,0x99,0x15,0x20,0x2b,0x82,0xd3,0xd3,0x16,0x52,0x1b, -0x40,0x1e,0xb2,0x18,0x50,0x3e,0x0a,0x1e,0x41,0x45,0x48,0xd2, -0xe3,0xd3,0x87,0xf8,0xa3,0xf5,0xa8,0xf5,0xe8,0x29,0xfe,0x38, -0x79,0x08,0x9d,0x26,0x4e,0x5a,0x8f,0x4e,0x82,0xc7,0xa7,0xe2, -0xf1,0xe9,0xa3,0xd6,0xc3,0xc7,0x88,0xe5,0x34,0x78,0x1c,0x04, -0x50,0xe6,0x24,0x0c,0x00,0xc1,0x43,0xc8,0x79,0x0a,0xd4,0xa1, -0x0a,0x3d,0x79,0x0a,0x58,0x5b,0xdc,0x1b,0xd8,0x8e,0xf0,0xe1, -0x09,0xf0,0xed,0x69,0x88,0x8c,0x3b,0x01,0xde,0x8a,0xa7,0x4f, -0x4e,0x9f,0x40,0x8d,0xc8,0x9e,0x93,0x87,0x4f,0x80,0xd5,0xe1, -0xc3,0xd6,0xe3,0xc7,0xc1,0x93,0xb0,0xa4,0x3d,0x46,0xf3,0xc1, -0x12,0x00,0x26,0x93,0xd2,0x30,0xdc,0x5a,0xeb,0x28,0x18,0x54, -0x3b,0x3d,0x47,0xdf,0x71,0xce,0xc7,0x3d,0x4a,0x7f,0x8c,0xbd, -0xa0,0x4c,0x63,0x6a,0xa7,0xe1,0x6d,0x28,0x34,0x90,0xd7,0x6b, -0x3e,0xea,0x17,0x91,0xa8,0x19,0x71,0x1b,0x0e,0x05,0xe3,0xd9, -0xa5,0x6f,0xde,0xc3,0x7a,0x76,0x58,0xba,0x29,0x0c,0xe9,0xca, -0xd2,0x08,0x22,0xf2,0x6f,0xfd,0x85,0x8e,0xe2,0xca,0x2f,0x02, -0x8b,0x45,0x18,0x81,0xc1,0xb6,0x03,0x68,0x10,0x44,0xbb,0x2e, -0x27,0xd7,0x47,0xfc,0x34,0x0f,0x54,0xcf,0xfe,0xee,0x18,0x3d, -0x7a,0xfb,0x25,0xe4,0x62,0x91,0xd1,0x20,0xf2,0x06,0xc1,0xd9, -0x20,0x44,0x97,0xaf,0x41,0x34,0x08,0xb0,0xe6,0x41,0x88,0xa8, -0x46,0x03,0x20,0x23,0x6c,0x0c,0xc0,0xc0,0x1b,0x5d,0x46,0x7f, -0xfc,0xdb,0xd9,0x68,0xd0,0xad,0x05,0xb5,0x76,0x2d,0xac,0x89, -0x61,0xc0,0x86,0xcb,0x70,0x96,0x7a,0x08,0x90,0xca,0xef,0x14, -0xe6,0x74,0xfc,0x1e,0x86,0x56,0x3e,0x58,0x8b,0x69,0x68,0xe5, -0x87,0xd6,0xbd,0xbf,0xb3,0xc8,0x90,0xf4,0x72,0xb0,0xee,0x2e, -0x82,0x6e,0xed,0x65,0x20,0x6a,0xf5,0x45,0x58,0xaf,0x3d,0xe3, -0x3f,0xf2,0xeb,0x41,0xf0,0x00,0xa4,0xfb,0x01,0x66,0x36,0xb6, -0xe6,0xe2,0x27,0xe3,0x08,0x18,0x2a,0x60,0xa8,0x40,0x42,0x05, -0x12,0xc7,0xb6,0x5c,0xfc,0xfc,0x0d,0xda,0xfa,0x25,0x68,0x01, -0x3c,0xd4,0x2c,0xfc,0x3a,0x1a,0xca,0xea,0xc2,0xa3,0x34,0x28, -0x29,0x57,0xab,0x8f,0x2e,0xe1,0x3b,0x7c,0xc0,0xc0,0xa1,0x06, -0x86,0x5f,0x3f,0x22,0x59,0x3a,0x2d,0xa0,0xb4,0x42,0x03,0x24, -0x02,0xfc,0x81,0xc0,0x81,0x06,0x56,0x0d,0xfa,0xd3,0x54,0x04, -0x22,0xa8,0x21,0xae,0x7f,0xe1,0xb8,0x65,0x46,0xde,0xad,0xae, -0x7c,0x61,0x52,0x85,0x25,0xec,0x16,0xc4,0xcf,0x26,0xd5,0xbe, -0xa5,0x6d,0x00,0x5e,0xeb,0x44,0x73,0x71,0xdb,0xe4,0x7e,0x27, -0x93,0x3a,0x7a,0xcd,0x34,0x6c,0xda,0xa4,0x98,0xfb,0xbe,0x5b, -0x96,0x55,0x39,0xda,0x95,0xc7,0xfa,0x0d,0xdd,0x35,0xf7,0xdb, -0x56,0xf6,0x06,0x53,0x9a,0x76,0x4b,0xf6,0xaa,0x20,0xd7,0xf4, -0x7c,0x05,0x56,0x36,0x57,0x50,0x76,0x0f,0x7c,0x1b,0x7e,0x97, -0x71,0x79,0xf4,0xf6,0x9d,0x70,0xc4,0x5e,0xbc,0x13,0xbe,0x0d, -0xb7,0xcd,0xf4,0x3c,0x66,0x73,0x3f,0x1c,0xf1,0x0e,0xa1,0xdc, -0x62,0x96,0x38,0x7a,0x99,0xb4,0x61,0xe4,0xed,0xd4,0x86,0xf5, -0x9d,0xaa,0x10,0x4f,0x9e,0x41,0x49,0xed,0xd4,0x86,0xdb,0x35, -0x1f,0x60,0x29,0x28,0x3f,0x29,0x3b,0x3d,0xa3,0xa9,0xfc,0xa3, -0x85,0x30,0x7a,0x0a,0xbe,0xfa,0xd4,0x42,0xb3,0x22,0xb1,0x8b, -0x47,0x8d,0xab,0xc1,0xf8,0xf2,0x8f,0x7f,0x13,0x8e,0x06,0x8b, -0xfe,0xf8,0x5f,0x8d,0x7f,0xff,0x67,0x47,0x4e,0x19,0xb8,0x02, -0x20,0xa7,0x0f,0xd7,0x58,0xa7,0x43,0x7b,0xc8,0xb2,0x5f,0x57, -0xf3,0xac,0xdf,0xbf,0x50,0x09,0xad,0x26,0x35,0x16,0x10,0x23, -0xfa,0xbd,0x56,0x7e,0x75,0xf2,0x8b,0xfe,0xf0,0xb3,0xe7,0xf4, -0x33,0x73,0xf2,0x33,0xca,0xdf,0x08,0xab,0x82,0xd7,0xcb,0x78, -0xfe,0x4d,0x0c,0xab,0xa9,0x58,0xdd,0x71,0x8c,0x72,0xb9,0x74, -0x5c,0x9f,0x2f,0x42,0x89,0x55,0x25,0x18,0x9f,0x99,0x2c,0xe3, -0x41,0xc5,0x24,0x4b,0xbc,0x8a,0x07,0x4e,0x1c,0xb3,0x8d,0x5d, -0xae,0x82,0x63,0x84,0x32,0x87,0xa0,0x82,0x63,0x1a,0x0b,0x70, -0x8c,0x7e,0x6b,0x8e,0xc9,0x2f,0xfa,0x63,0x8f,0x7f,0x4e,0x0f, -0x5c,0xb0,0x80,0xe1,0xec,0x61,0xcc,0x19,0xa1,0x0b,0x17,0x32, -0x1c,0xf7,0x00,0xfd,0xcc,0x1c,0x80,0x8c,0xf3,0xad,0xe1,0xca, -0xc9,0x6e,0x75,0x99,0xac,0x4e,0x8f,0x3c,0x4e,0x75,0xeb,0xca, -0xb8,0x2e,0xcd,0xaf,0xea,0x17,0x48,0x28,0xc7,0x38,0xda,0xa4, -0xfa,0xd1,0x05,0x1e,0x1b,0x32,0x3c,0x34,0x39,0xd6,0x2c,0x73, -0x79,0x32,0x5e,0xb4,0x15,0xa4,0x06,0xe7,0x8d,0xb4,0x39,0x0f, -0xea,0x68,0xec,0xa4,0xcd,0x85,0x80,0x8f,0x10,0xfe,0x01,0x03, -0x01,0xfe,0x09,0xfc,0xba,0x17,0x83,0xf5,0x36,0x4e,0x3d,0x28, -0xec,0x77,0x87,0xab,0xc5,0x4d,0x6c,0x83,0xd1,0x8f,0xc0,0x6f, -0xe7,0x32,0x96,0xf0,0x63,0x89,0x19,0x84,0x56,0x7e,0x61,0x32, -0xa0,0x5d,0x12,0x5a,0x2e,0xa0,0x73,0x34,0x26,0x1f,0x67,0x20, -0xeb,0x30,0x51,0x91,0x9f,0xc6,0xe3,0x4b,0x71,0x29,0x4c,0xab, -0x65,0x46,0x74,0x29,0x23,0x15,0x60,0x3e,0x3b,0x13,0x49,0x2f, -0x42,0x2b,0x5d,0x01,0x63,0xb6,0x6b,0x80,0x95,0xc2,0x6c,0x37, -0xc0,0xf6,0x2b,0x22,0xd9,0xfb,0x69,0xd1,0x5e,0x40,0x95,0xf8, -0x7e,0xc7,0x20,0xc4,0x97,0x3b,0xe6,0x41,0xbb,0xb7,0x38,0x72, -0x6c,0x2c,0xf9,0x29,0x4d,0x2c,0x7c,0x57,0xdf,0x05,0x09,0x5d, -0x90,0xd0,0xef,0x5b,0x67,0x28,0xb2,0x77,0xec,0xdb,0xdc,0x20, -0xaf,0x41,0x14,0x5d,0xc0,0xbf,0x87,0x87,0xf0,0x3b,0xa4,0xdf, -0xe1,0xc6,0x31,0x6b,0x17,0x62,0x2e,0xf4,0x25,0x69,0xb2,0x13, -0xd8,0x14,0x90,0x56,0xc2,0x83,0xba,0x37,0x38,0xfb,0xe3,0xdf, -0x7c,0x69,0x2b,0xcc,0x4d,0x61,0xee,0xb7,0x45,0x00,0x0d,0x01, -0x83,0x17,0x68,0xd5,0x48,0xfe,0x1f,0xb0,0x8b,0xb0,0xec,0x3c, -0xdc,0x58,0x4f,0x85,0x20,0x5f,0xec,0x67,0x39,0x30,0x41,0x4a, -0xe1,0xc2,0x31,0x1d,0x48,0xaa,0xbf,0x9c,0xdd,0xc0,0x8f,0x17, -0xe4,0xdd,0xf2,0xb6,0xcd,0x8b,0x8b,0xb2,0xe9,0x5c,0x3e,0xa8, -0xa0,0x5c,0xf0,0xd4,0x9b,0x0d,0x2b,0xe7,0x39,0x83,0xad,0x33, -0xb9,0x66,0x43,0x11,0xab,0x7c,0xbf,0x44,0x62,0x95,0xfc,0xd9, -0x0b,0xab,0xe1,0x65,0x11,0xab,0x7c,0xf7,0x44,0xd1,0xfa,0xa5, -0x2c,0x0f,0x45,0xb3,0x65,0x7b,0x70,0x0d,0x5f,0xc4,0xfa,0x90, -0xd8,0x8d,0xfd,0x61,0x9e,0x90,0x40,0x65,0x08,0x73,0xc0,0xfb, -0x59,0x6e,0x63,0x4c,0x8b,0xa4,0xca,0xb4,0x7c,0x0f,0x61,0x8c, -0x73,0xe4,0xd9,0x9c,0xa3,0xe1,0xfc,0x24,0x92,0x01,0x68,0x73, -0x19,0x93,0xc8,0x9b,0x07,0xcd,0xac,0x3e,0x3f,0x69,0x66,0x68, -0x98,0xcc,0xa3,0x1e,0x08,0xf8,0xa7,0x75,0x1b,0x52,0xd7,0x22, -0x6b,0x4f,0x36,0xf4,0x71,0xa2,0x3e,0xe6,0x27,0xc6,0x83,0x34, -0x9a,0xd3,0xea,0x7a,0xbe,0x98,0xe1,0x4b,0x87,0x40,0x91,0x2f, -0x50,0xab,0xce,0x7b,0x41,0xbf,0x5e,0x7b,0x8e,0x3f,0x42,0xf8, -0x81,0x23,0xa3,0xd7,0x52,0x3f,0x4e,0xfa,0x7b,0x0e,0x12,0x83, -0x55,0x09,0xbb,0x6a,0xee,0x2b,0x9d,0xa3,0x57,0x5d,0x32,0xa7, -0x20,0xa0,0xdb,0x03,0x17,0xe5,0x05,0x14,0x5f,0x1b,0x96,0x98, -0x8c,0x8c,0x6a,0xdc,0x79,0x31,0xdd,0x1a,0x4b,0xbb,0x20,0xa6, -0x0e,0x6e,0x25,0xa9,0x1a,0xb7,0xd5,0xd8,0xbd,0xf0,0x5b,0xf0, -0x6b,0x0b,0xaf,0x49,0x36,0xb8,0x0b,0xa2,0x54,0x61,0x8b,0x68, -0x49,0xcb,0x81,0x7b,0xfe,0x5e,0x5d,0x61,0x01,0x9d,0x47,0x25, -0xed,0x2a,0x74,0xd5,0x3d,0x9b,0x6c,0xa1,0xf7,0x72,0x44,0xb0, -0xa5,0x64,0x48,0x71,0xb8,0xe3,0xef,0xe0,0x4a,0x9a,0x4d,0x2e, -0x66,0xe5,0xc3,0x8b,0xb3,0xec,0xc0,0x4d,0xaa,0x66,0xce,0xe1, -0xd8,0x2f,0xf4,0x86,0x9d,0x3b,0xa8,0x38,0x1a,0x8c,0x01,0x7c, -0x4e,0xcf,0x53,0xfa,0x74,0x2c,0x9c,0x07,0xdd,0xa8,0xd3,0xce, -0xc8,0x29,0xf2,0x06,0x52,0xf8,0xa0,0xdb,0x49,0x46,0x8f,0x40, -0x7d,0xa4,0x42,0x29,0x4d,0x2a,0xbb,0x9f,0x50,0xca,0x5a,0x2c, -0x91,0x64,0x1c,0x20,0x90,0x90,0xb5,0x91,0x5f,0xf7,0x3d,0xe7, -0x2e,0xc7,0xc8,0x47,0xde,0xfc,0xe5,0xac,0x36,0x24,0xef,0x74, -0xfc,0x10,0xf9,0xb8,0x67,0x09,0xc3,0xc4,0x70,0x31,0x4b,0xad, -0x8b,0x0e,0x1c,0xb8,0x9d,0xd7,0x61,0xe6,0xe0,0x0c,0x53,0x8f, -0x1f,0x19,0x4f,0x5d,0xd4,0x40,0x8d,0x93,0x23,0x9e,0xd6,0x1b, -0xf0,0xe7,0x07,0xfe,0xf3,0xab,0x4a,0xfe,0x81,0x66,0xfb,0x5f, -0x75,0x26,0x25,0xfe,0x4a,0x89,0x3a,0x4b,0x01,0x36,0xec,0x2c, -0x2e,0x0e,0x46,0x1a,0x4a,0xd1,0x64,0x66,0x47,0xa4,0xb0,0x48, -0xcb,0xf2,0xb4,0x61,0x44,0x0d,0xa7,0x6b,0x07,0xd3,0x93,0x00, -0x9f,0x85,0x5f,0x47,0x8b,0xac,0x24,0x4b,0xb7,0x83,0xf7,0x62, -0x32,0xde,0x26,0x59,0xe3,0x16,0x48,0x0d,0xad,0x14,0xb4,0x56, -0x32,0xd2,0xa6,0x0d,0x9d,0x8a,0x44,0x91,0x47,0x43,0xbc,0x0f, -0xbb,0x72,0xcc,0x32,0xac,0xfa,0x51,0x99,0x43,0xf8,0xf1,0xc0, -0xb6,0x8d,0x34,0x98,0xe4,0x40,0x6d,0xb9,0x48,0x68,0x71,0xd0, -0x18,0xcd,0x6e,0xed,0x87,0x4c,0xad,0x5a,0xd7,0x79,0x4e,0xb8, -0xa2,0x0c,0xc9,0x27,0xc0,0x85,0x2c,0x5a,0xac,0x8f,0x8a,0x39, -0x16,0x89,0xaa,0xc5,0x8a,0x0d,0x92,0x2b,0x9a,0x01,0xf2,0xdb, -0x25,0x6b,0x35,0xff,0x8f,0x27,0xaa,0xe1,0x52,0x95,0x23,0x4a, -0xd2,0xc4,0xf1,0x3a,0x8c,0xaa,0xc1,0x91,0x9d,0x46,0x39,0x45, -0x72,0x1d,0x67,0xa9,0xe7,0x3b,0xef,0x51,0x92,0xf7,0xd7,0x8b, -0xe9,0x55,0xbc,0x48,0x96,0x2f,0x46,0xa2,0x2c,0x59,0x14,0x29, -0xb4,0x5a,0x77,0xe2,0x8b,0xa2,0x70,0x71,0xfe,0x72,0x30,0xf5, -0x4e,0x02,0x6c,0x20,0xcd,0x05,0x53,0xbe,0x00,0x63,0xf0,0xbf, -0x52,0x8f,0x73,0xa2,0x57,0xac,0x5b,0xf3,0x88,0xef,0x28,0x94, -0xc0,0x92,0x66,0xa3,0x86,0xc5,0x63,0x56,0xb7,0x6e,0x56,0x45, -0x15,0xcd,0x78,0x32,0x5f,0x66,0x15,0xe5,0x28,0xaf,0xaa,0x20, -0xfa,0x94,0x54,0x94,0xe3,0x58,0xc4,0xe5,0xc5,0xb4,0x56,0x2d, -0x16,0xc3,0x2c,0xfd,0x82,0x6b,0x2e,0x4e,0xad,0x06,0xaf,0xd6, -0x87,0xe5,0x5d,0xd7,0xd5,0x25,0x2d,0xbc,0x1e,0xfa,0xaa,0xcb, -0xe4,0xb6,0x4d,0xcc,0x2f,0xe8,0x68,0x65,0xc3,0x6d,0x5c,0x82, -0xcc,0xc3,0xa9,0x51,0x79,0xfb,0x2a,0x9b,0x4d,0x35,0xe4,0xdb, -0x83,0xd7,0xd4,0xed,0x55,0x25,0x2e,0xed,0xb4,0x6f,0x91,0x48, -0x46,0x11,0x85,0x41,0x4f,0x46,0xd6,0x1b,0x9f,0xbd,0x7e,0x47, -0x95,0x75,0xd8,0x78,0xae,0x52,0x0d,0xea,0x8e,0x72,0x13,0x90, -0xaf,0x69,0x10,0x32,0xf7,0x21,0x8d,0x4f,0x1a,0xb1,0x7c,0x0a, -0x51,0x2d,0x6f,0x7b,0x7d,0x53,0x9c,0x53,0xb0,0x34,0xc6,0xcb, -0x92,0x9e,0x8f,0x94,0x98,0x77,0xda,0x24,0x99,0xa0,0x1c,0xf4, -0x40,0x47,0x07,0x34,0x6e,0x4f,0xa4,0x68,0xe2,0xf9,0xd7,0xb8, -0x4f,0x9d,0x9f,0x63,0x4c,0x96,0xf3,0x73,0x7c,0xc8,0xd4,0xf7, -0xbb,0x5e,0x4d,0x25,0xd4,0x12,0x8e,0xc9,0x6d,0x70,0x68,0xd8, -0xc8,0xf9,0xca,0x8d,0x1d,0xbc,0xcb,0xa2,0x4a,0x00,0x52,0x60, -0x9d,0x84,0x36,0x20,0xe7,0xe7,0xbd,0x64,0xd4,0x37,0xcf,0x92, -0xea,0x96,0x93,0x1f,0x15,0x48,0x86,0x93,0x4e,0x37,0x2d,0xac, -0xcb,0x66,0x96,0x70,0x68,0xde,0x41,0x35,0xd6,0x03,0xbb,0xa5, -0x3d,0xff,0x6c,0x3c,0xde,0xb7,0xf3,0x53,0x79,0xdd,0x5d,0x36, -0xc2,0xe0,0xfc,0x4c,0x81,0x78,0xc6,0xfb,0x06,0x7b,0xcb,0xc4, -0xbd,0xbb,0x5d,0x86,0x81,0x36,0x1d,0xff,0xc9,0x1a,0xc1,0x15, -0xec,0xd7,0x8d,0xdd,0x4f,0x34,0xc4,0x3e,0x72,0x7a,0x8d,0x24, -0xce,0x22,0x85,0xd9,0x50,0x79,0xfd,0xf5,0x0c,0x84,0x53,0xcb, -0xa2,0xcc,0xef,0x5d,0xa3,0x88,0x56,0x4b,0xcf,0x35,0x4a,0x8f, -0xc9,0xab,0x12,0x98,0x3f,0x23,0x1b,0x97,0xc9,0xd8,0xb9,0xd8, -0xc6,0xdf,0x39,0xb1,0xe0,0x7f,0x95,0x4c,0x98,0x5e,0x37,0x9e, -0x76,0x07,0xca,0xc7,0x8e,0xcb,0xc3,0x98,0x91,0x88,0x1d,0x79, -0xe0,0x34,0x55,0x87,0x2d,0x09,0x41,0x4e,0x10,0x94,0x3b,0xd1, -0x67,0x69,0x87,0x20,0x2f,0x25,0x2a,0x16,0x57,0x41,0x3b,0x30, -0x25,0xd5,0xfd,0xee,0x9b,0x97,0x7a,0xad,0xc7,0x0a,0x76,0xf1, -0x5b,0x8e,0x90,0x6a,0xa6,0x2f,0x6f,0x63,0x3b,0xe4,0x3b,0x3d, -0xc0,0x4e,0x69,0x32,0xa8,0x99,0x1e,0x64,0x9d,0x8a,0x99,0xe6, -0xeb,0x56,0x97,0x20,0x90,0x1c,0xcf,0x2f,0x11,0x6e,0xae,0x82, -0xd6,0x27,0xf4,0x2e,0xbc,0x33,0xc5,0x9c,0xc7,0xe8,0x3e,0xcf, -0x4f,0x8c,0x90,0x5b,0x25,0x01,0x97,0x5e,0x5d,0xab,0xc4,0xbc, -0x88,0xe9,0xb9,0x45,0x42,0x5e,0x7e,0x4f,0xad,0xb2,0x68,0x2a, -0x89,0x92,0x4d,0xde,0x54,0xf3,0x69,0xb0,0x5c,0x2e,0x5c,0x36, -0xfd,0xf4,0x9a,0x63,0x63,0xd9,0x67,0xd5,0x00,0x44,0x9e,0xad, -0x30,0xe8,0xe9,0x21,0x70,0xa2,0xcc,0xbc,0x7a,0xce,0x24,0x16, -0xc0,0x7f,0x7a,0x5d,0x51,0x00,0x32,0xf8,0x15,0xfc,0xf9,0x60, -0x08,0x02,0x81,0x3f,0xc7,0x33,0x90,0x90,0x1c,0x8e,0x37,0x48, -0xbc,0x67,0xf6,0x1e,0x99,0x93,0x17,0x5d,0x85,0xbf,0xed,0x5d, -0xd4,0xa3,0x5a,0x2d,0xef,0x49,0x98,0x88,0x01,0x77,0xee,0x7b, -0xfb,0x65,0x76,0xa2,0x51,0x77,0xf6,0x01,0xe0,0x81,0x11,0x94, -0x44,0x96,0x87,0x1a,0xed,0x73,0x1a,0x64,0x4b,0x49,0x7b,0x9a, -0xc7,0x22,0x12,0xc8,0xa3,0x67,0x9f,0x4b,0xc8,0x85,0xa6,0x55, -0x13,0xfc,0xd3,0xeb,0xfb,0x90,0x5c,0xc9,0xa5,0x3f,0xdb,0x8a, -0x2a,0xc4,0x56,0xc3,0x40,0xdb,0x17,0x07,0x84,0x72,0x49,0x04, -0xe1,0x78,0x80,0x33,0x37,0x09,0x8b,0x4f,0xe8,0xb1,0x85,0x9c, -0x23,0x38,0xb9,0x47,0x1f,0x26,0x58,0x2f,0x42,0xc9,0xa3,0x5f, -0x4d,0x6a,0x64,0xbd,0x96,0x1f,0x45,0x5c,0xae,0x3b,0x2a,0x7f, -0xe4,0xbd,0x9d,0xf3,0x26,0x44,0xe8,0x88,0x1f,0xf2,0xc7,0x4d, -0x6a,0xba,0x12,0xc4,0xf4,0x74,0xca,0x54,0xc7,0x39,0x0d,0x04, -0x1e,0x90,0x35,0x24,0xb6,0x59,0xab,0xdb,0xc2,0x6e,0x71,0xa1, -0x6b,0xf5,0x65,0x5b,0xff,0xde,0x3e,0x82,0xde,0x14,0xb5,0x0d, -0x60,0x96,0xfa,0xa6,0x44,0xa6,0xf5,0x66,0xe2,0x25,0x2b,0x06, -0x77,0xc7,0x42,0x54,0x48,0xaf,0x71,0x34,0x05,0xb5,0xba,0x5b, -0x4c,0x2f,0xa9,0x37,0xcb,0x65,0xf4,0xbe,0x04,0x54,0xca,0xe2, -0xde,0x34,0x55,0x0a,0x9d,0x24,0x93,0x9c,0x88,0x77,0x76,0x2a, -0xa1,0xe5,0xbe,0xcc,0x75,0xe3,0xe7,0x77,0x60,0xba,0xcc,0xec, -0x0d,0x66,0xe2,0x1d,0x0b,0xc5,0x7c,0x91,0xcc,0x16,0x78,0xd0, -0x68,0x47,0x24,0x07,0x60,0x5b,0x17,0x52,0x82,0x54,0x70,0x80, -0x73,0x1e,0x2f,0x96,0x59,0x5e,0x21,0x12,0x0c,0xbb,0x46,0x95, -0x68,0x07,0x8d,0x72,0x8b,0x72,0x00,0xd2,0x6f,0x93,0xe9,0x68, -0x76,0x8b,0xbd,0xf2,0x7c,0x36,0x99,0x03,0x47,0x47,0xaf,0xb1, -0xa0,0x99,0x64,0x7c,0xcc,0x53,0x24,0xfc,0xca,0x37,0x88,0xcb, -0x95,0x9e,0x3b,0x96,0xaa,0x34,0x06,0x35,0x2c,0x35,0x28,0x8d, -0xf2,0x33,0x8c,0xd9,0xa8,0x97,0xc6,0x4b,0xfc,0xbf,0x41,0x87, -0x9c,0x9c,0x4d,0x29,0x44,0x57,0x2d,0xa5,0xc6,0x1b,0x1b,0x86, -0x08,0xfb,0xc4,0xae,0xe1,0x1c,0x25,0xb1,0x56,0x63,0x6b,0x43, -0xa1,0x56,0x1a,0xc6,0x37,0xc4,0xe8,0x3c,0xea,0xec,0x9e,0xfa, -0x92,0xc1,0x35,0x5d,0x55,0xa3,0x32,0x01,0x6f,0xa9,0x2d,0xe1, -0x28,0x04,0x6e,0x2a,0x8b,0x92,0xec,0x7a,0xab,0xcb,0x76,0x48, -0x4e,0xc9,0xd8,0xe7,0x91,0x5f,0x29,0x3e,0x7b,0xe9,0x80,0x3f, -0xd3,0xe3,0xdb,0x07,0x65,0x79,0xbf,0x5e,0xba,0xfd,0xba,0xa9, -0x32,0x8b,0x4e,0xf0,0xee,0xa1,0x61,0xae,0x2f,0xec,0x21,0xe9, -0x30,0xd2,0xb4,0x75,0x8b,0x99,0x16,0xaf,0xed,0x53,0x29,0xb6, -0x3b,0x76,0xf6,0x17,0x96,0xaa,0xc9,0x8e,0xca,0x41,0x41,0x4e, -0x75,0x6d,0x3c,0x4a,0x4b,0x1c,0x0c,0xa8,0x0d,0x64,0xb2,0xe1, -0x5b,0xcb,0xd6,0x56,0x40,0x61,0x30,0xce,0x3b,0x04,0xeb,0x5a, -0x5e,0xcd,0xe1,0x6c,0x35,0x5d,0x7e,0xdd,0x42,0xd6,0xf0,0x3c, -0xce,0x51,0xe4,0x28,0x18,0xc3,0xe1,0xe1,0x5c,0x56,0x4c,0x61, -0x0b,0xa9,0x09,0xdb,0x2c,0xb2,0x78,0x90,0xc6,0x05,0x96,0xec, -0x69,0xb3,0x86,0xbb,0x6c,0x56,0x44,0xde,0xf6,0x4a,0x16,0x15, -0xfa,0xe5,0x29,0xf9,0x52,0x13,0x41,0x96,0x5d,0x76,0xe0,0x1d, -0xaf,0x12,0x3b,0x77,0x4f,0x03,0x95,0x9a,0xc7,0xaf,0x60,0xf8, -0xd5,0x4c,0x18,0xe1,0xbb,0xef,0x7b,0x72,0xa1,0x82,0x9e,0x92, -0xc8,0x27,0x37,0xee,0x23,0x27,0x72,0xfb,0xe0,0x43,0x35,0xb1, -0x4c,0x86,0xfd,0x52,0x4c,0xc5,0xd2,0x75,0xd3,0xf6,0xec,0xf7, -0xc8,0xf6,0x65,0x06,0xe3,0xdf,0xcd,0x8d,0x55,0x3e,0x6c,0xc3, -0xff,0x31,0x86,0x28,0x4a,0xf4,0x95,0xdb,0x50,0xec,0xe6,0x8e, -0x61,0x4f,0xbe,0x98,0xbf,0x37,0xa7,0x54,0xc5,0x3b,0x99,0x85, -0x0d,0x36,0x8c,0xa2,0xed,0xba,0x71,0x92,0x2e,0xe3,0xa9,0x5a, -0x73,0x5b,0x0c,0xab,0x30,0x73,0xd9,0x4e,0xa5,0xbd,0xc5,0xa8, -0x74,0x53,0x38,0x51,0x1b,0x8f,0xa5,0xd9,0x2f,0x46,0x9d,0xf2, -0xe4,0x28,0x19,0xed,0x35,0x7a,0xb8,0x0f,0xca,0x70,0x54,0xee, -0xb4,0x98,0x23,0xb2,0x2a,0x09,0xf5,0x4b,0xf7,0xb7,0xa3,0xc4, -0xec,0x73,0x97,0x52,0xac,0x7e,0x71,0xd8,0xd2,0xfd,0xc6,0x3e, -0x9d,0xb7,0xed,0xdc,0x1d,0xea,0x78,0xd6,0x96,0x2b,0x3d,0xc0, -0x78,0x77,0x57,0x48,0x42,0x55,0x34,0x4a,0xc0,0x68,0x5c,0xa2, -0x72,0x96,0xaf,0x18,0xf3,0xd3,0xff,0x3e,0x3d,0x06,0xe9,0x76, -0xf0,0xc6,0xb5,0x02,0x2a,0xa7,0x9d,0x92,0xcd,0x67,0xeb,0xc2, -0x55,0xc9,0xa6,0x8c,0xb3,0x41,0x17,0xe8,0x1d,0xba,0x64,0x14, -0x46,0xf5,0x7a,0x6e,0xfb,0xde,0xd9,0xc1,0x11,0xff,0xc7,0x76, -0x5e,0x2c,0x5e,0x46,0x3f,0x5f,0xe0,0xf1,0x6c,0x73,0xb8,0x88, -0xd1,0xe0,0x2b,0xef,0x90,0xa0,0xef,0x5b,0xcd,0x64,0xdd,0x54, -0x8f,0xec,0x14,0x39,0x06,0x4b,0x36,0x63,0xf5,0x4e,0x6c,0x68, -0x6f,0xa6,0xe5,0x77,0xd3,0xf6,0xdc,0x4a,0x0b,0x2d,0xc7,0xbe, -0x75,0x92,0x96,0xbb,0xcf,0x43,0x86,0xf7,0xde,0xff,0xf4,0x9e, -0xa7,0xed,0xdc,0x2c,0x8d,0xd7,0x2f,0xf5,0xd2,0x51,0xd0,0xb3, -0xa8,0x2c,0xb6,0x3c,0xfb,0xf2,0x3d,0x6c,0xd9,0x8b,0xe7,0xe7, -0xf8,0xb0,0xeb,0xf2,0xfc,0xfc,0xee,0x8e,0x92,0x39,0x33,0xcc, -0x65,0x46,0xf2,0x22,0xd0,0x6c,0x8e,0xf7,0xa5,0x11,0xbf,0x0a, -0xb2,0x85,0xbf,0xc9,0xf6,0x4a,0xbb,0x5c,0x92,0x5e,0xd1,0x76, -0x3f,0xe4,0x44,0xca,0x49,0x54,0xfa,0x99,0x52,0x3b,0xe7,0x7e, -0x5b,0x42,0xca,0xdb,0x6d,0x7e,0xdb,0x60,0x24,0x50,0x79,0x91, -0xcd,0xd4,0xc6,0x09,0xe7,0x76,0x0d,0x9c,0x54,0x4c,0xd9,0x5e, -0xb1,0xe5,0x4b,0xd9,0xb6,0x10,0x13,0x5c,0xf4,0xde,0x6c,0x6a, -0x7b,0x35,0x42,0x59,0xf3,0x9b,0xa8,0x50,0x3c,0x6a,0x92,0x64, -0x13,0xf3,0xe2,0x3b,0x5c,0x4d,0x10,0x6d,0xcd,0x18,0x7f,0x7a, -0xf8,0x76,0x74,0x0a,0xc6,0xa5,0x57,0x7b,0x0f,0xc3,0x55,0x36, -0x0d,0xca,0xd3,0x3e,0x41,0x6d,0x38,0x1e,0xa4,0x29,0xa4,0x4b, -0xa4,0x6c,0xdc,0xd7,0x66,0xb0,0x32,0x04,0x4a,0x6a,0xe2,0xdf, -0xff,0x29,0xb1,0xae,0x49,0xff,0xda,0x47,0x35,0x5c,0x05,0xa4, -0x7b,0x7e,0x69,0x39,0xb5,0x69,0xc6,0x08,0xfe,0x3a,0x1f,0xe1, -0x36,0x43,0x11,0x45,0xb1,0x6c,0xa8,0xa2,0xf0,0xab,0x0b,0x1d, -0x14,0xf7,0xdd,0x53,0x8d,0x04,0xfd,0x73,0x95,0xe3,0x88,0x6e, -0x13,0xf1,0x04,0xdf,0xd7,0x23,0x28,0x59,0xa7,0x37,0xa7,0xab, -0x9f,0x92,0x19,0xd0,0x0b,0xa0,0xb2,0xbc,0x1a,0x26,0x16,0xb8, -0xa0,0x10,0xb9,0x87,0x50,0x84,0x00,0x2c,0x79,0xcd,0x5e,0x8d, -0x04,0x3d,0x57,0x6b,0x16,0xdf,0x75,0x06,0x99,0xc5,0x32,0xb8, -0x17,0xc0,0x98,0x4e,0x61,0x30,0x39,0x02,0x54,0x79,0xfc,0x23, -0x89,0x35,0xbc,0xca,0x03,0x91,0x7d,0x4e,0x4c,0x57,0x19,0x5c, -0x09,0x19,0xee,0x9e,0x91,0x19,0x86,0xac,0xaa,0x91,0xcb,0x10, -0x48,0x75,0x8d,0x92,0xfa,0xf4,0x36,0x41,0x55,0x0f,0xeb,0x0b, -0x60,0x9e,0xff,0x69,0x08,0xe6,0x62,0x8d,0x5f,0x69,0xa9,0xb5, -0x9d,0x61,0xad,0xfd,0xf3,0x40,0x13,0xbc,0x15,0xba,0xc1,0x92, -0xb9,0x59,0xab,0x26,0xc8,0x53,0xf7,0x4d,0x82,0x31,0x08,0x3f, -0xc6,0x16,0x85,0x0a,0xa4,0x26,0xb4,0x39,0xe2,0x80,0x8a,0xc0, -0xaf,0x63,0x55,0xaf,0x64,0x7c,0x45,0x8b,0x55,0xb2,0xe8,0x1a, -0xb0,0x07,0xfe,0xf6,0xaa,0xdc,0x02,0x36,0xfc,0xde,0xf5,0x22, -0x1a,0x59,0x68,0x94,0xe1,0x38,0x7a,0x1c,0xc6,0x13,0x33,0x60, -0x30,0xbb,0x31,0x98,0xa2,0x37,0x1b,0x64,0x4e,0x92,0xd1,0x68, -0x8c,0x7d,0x66,0xa4,0x50,0x95,0xad,0x91,0xcf,0x98,0x7a,0x6a, -0x81,0x4e,0xce,0xc9,0x3b,0x59,0x55,0x5c,0xaf,0xfd,0x1a,0xfc, -0xa0,0x00,0xd0,0x9f,0xec,0xd7,0x1c,0x80,0xdf,0xb9,0x80,0x99, -0xe3,0xba,0x43,0xbd,0xb1,0x9c,0xcd,0xef,0xdd,0x15,0x8d,0xdd, -0x7d,0xd1,0xf0,0xf6,0xe3,0xca,0x3e,0xdd,0x51,0x56,0xdd,0xb6, -0xfe,0xd8,0xbf,0xee,0x7c,0x97,0x04,0x5f,0xaa,0x43,0x1a,0x3b, -0x7b,0xa4,0x51,0xdd,0x25,0x18,0x48,0x6f,0x4b,0x9f,0xbc,0x2b, -0xf4,0xc9,0x7a,0x8f,0x3e,0x59,0xff,0xe9,0x3e,0xc9,0xd5,0x60, -0x75,0x50,0x50,0xde,0x27,0x7b,0xd7,0x67,0xba,0x2e,0x28,0x19, -0x26,0x27,0xad,0xea,0x5e,0x21,0xd3,0xb1,0xb2,0x4b,0xf2,0xbd, -0x20,0xec,0x5e,0xfa,0x21,0xf8,0xd5,0xee,0x92,0x1f,0xb6,0x76, -0x09,0x45,0x38,0xbc,0x77,0x9f,0xec,0xee,0x92,0x3f,0xa3,0xb2, -0xee,0xdb,0x1d,0xfb,0xd5,0xf5,0xf9,0x7d,0xc1,0xe6,0x7c,0x75, -0x6f,0xdc,0xaf,0x33,0xdc,0xbe,0xc0,0x23,0x0c,0x69,0x0e,0x11, -0xd4,0x37,0x83,0xa9,0x0c,0x05,0x84,0x11,0x7c,0x72,0x19,0x1e, -0x45,0xaa,0x34,0x86,0xa6,0x79,0x08,0x92,0x21,0x21,0xa1,0x3e, -0x5a,0x6f,0xac,0xc9,0x98,0x37,0xeb,0x1c,0x33,0x73,0x6d,0x1b, -0x1d,0xe5,0x00,0xbc,0x88,0xda,0x8a,0x85,0x4d,0xd3,0x5d,0xa8, -0x1c,0xe3,0x0b,0xcc,0xa1,0x2d,0x30,0x72,0x83,0xd6,0xb6,0xef, -0x38,0x3e,0x44,0x31,0xe2,0x00,0x4f,0xba,0xb6,0x94,0xca,0x27, -0xca,0x7e,0xa6,0x0c,0x57,0x7e,0xa2,0x53,0xe1,0x70,0x1c,0xbe, -0x2d,0xa9,0x88,0x4e,0x72,0xd6,0x66,0xd4,0x0b,0x83,0xbe,0x30, -0xa6,0x2d,0xd9,0xb3,0x7a,0x23,0x6b,0x8d,0x1b,0x93,0x44,0xd5, -0x7e,0x4e,0x91,0x04,0xba,0x16,0x58,0x4e,0xda,0xcf,0x1b,0xfa, -0x68,0xca,0x26,0xec,0x85,0x45,0xc2,0xae,0x71,0xab,0xd9,0x6a, -0x33,0x37,0x36,0xed,0x62,0xe8,0x94,0x76,0x25,0x2f,0xb8,0x6a, -0x46,0x21,0xeb,0xe6,0x55,0x41,0x71,0x77,0xb1,0xc4,0x4d,0xd4, -0x65,0x8d,0x06,0x90,0x58,0xdd,0x6c,0x0b,0xfb,0xaf,0xf7,0x78, -0x49,0xd2,0x82,0x5f,0x5b,0x78,0x7f,0x95,0x0f,0x4a,0x6a,0x9c, -0x72,0x79,0xb1,0x37,0x4e,0x29,0x44,0x0e,0x52,0x99,0x66,0x61, -0x2d,0x84,0xe9,0xdc,0x1e,0x24,0x66,0xda,0x75,0x75,0x4c,0x54, -0x5f,0xe7,0xa4,0xab,0xae,0x8b,0x52,0xe4,0x44,0x59,0xb9,0x53, -0x48,0x56,0xef,0x22,0xda,0xab,0x5d,0x85,0xba,0xab,0xb1,0xbb, -0x54,0xed,0x27,0x67,0x6e,0x43,0x14,0x76,0x27,0xd9,0x62,0xdd, -0xab,0x92,0xd8,0x55,0x5b,0x7b,0x44,0x15,0xa8,0xdb,0x5d,0xf2, -0x4a,0x45,0x63,0x36,0x5d,0xb2,0xba,0x18,0x25,0x37,0xc9,0xa8, -0x6c,0x03,0x3c,0x8f,0xf8,0xf0,0x10,0x8b,0x71,0x61,0xf7,0x22, -0x63,0x7e,0x20,0x44,0xca,0x3a,0x17,0xc5,0x21,0x14,0x7d,0x02, -0x5b,0xb1,0x1d,0x72,0x30,0x60,0xf8,0xcb,0x90,0xf0,0x03,0xed, -0x95,0x76,0xb8,0x51,0x5e,0x9b,0x17,0x8b,0x55,0x7a,0x55,0xba, -0xcc,0xa7,0x9c,0xea,0x75,0x7e,0x61,0x65,0xaf,0xe6,0x19,0x7a, -0x69,0x27,0x5e,0x34,0x68,0x0f,0x09,0x17,0x59,0xa0,0x1e,0xcd, -0x2c,0xd4,0xb8,0x8d,0x2f,0xae,0x93,0x65,0x63,0x39,0x98,0x37, -0xae,0x80,0xb6,0x31,0xd2,0xd7,0x18,0xce,0xc6,0x34,0x2b,0x2d, -0xde,0x5f,0x0c,0xbc,0x40,0xd0,0xff,0xfc,0x1a,0x6d,0x32,0xd5, -0x26,0xb3,0x55,0x1a,0xa3,0x1b,0x2e,0xd3,0x5a,0x13,0xf4,0x87, -0xe6,0x2f,0x06,0x58,0xce,0x56,0x43,0xfe,0x2e,0x81,0x10,0x17, -0x83,0xe1,0xf5,0x7b,0x0e,0x4d,0xe5,0x2e,0x22,0x4d,0x86,0xbd, -0x90,0xec,0x98,0xe4,0xc2,0x12,0x12,0x83,0xdd,0x15,0x96,0x90, -0x0e,0x1a,0xd9,0xc8,0x9b,0x24,0x4d,0x2e,0x92,0x31,0xad,0x6e, -0x6b,0x57,0x60,0x8b,0xc6,0x53,0x93,0x39,0x5c,0x2d,0x52,0x6a, -0x2c,0xf9,0x7f,0x5f,0x0d,0x92,0x05,0x4c,0xc3,0x2e,0x69,0x30, -0x53,0x43,0xdd,0x36,0x59,0xfb,0xd1,0xa2,0xcb,0xe5,0xab,0xc2, -0x85,0xb9,0x5c,0x9e,0x2e,0x62,0xf2,0x10,0x75,0x2b,0xe4,0x44, -0x55,0x21,0x7f,0xa5,0xc2,0xbe,0xef,0xd9,0xe1,0x44,0xb9,0x01, -0x60,0xd6,0xfa,0x2a,0x39,0x47,0xde,0xfb,0x3c,0x6d,0xc5,0xf9, -0xbd,0xc6,0x45,0xf1,0xee,0xfa,0xa6,0x40,0x71,0x89,0x39,0x20, -0x65,0x9c,0xfa,0xf6,0x39,0x81,0xf5,0x46,0xfd,0x4d,0x05,0x47, -0xd6,0x65,0x28,0x8e,0x7b,0xf1,0x6d,0xff,0x2f,0xc7,0xb0,0x80, -0x4e,0x97,0x90,0xda,0x6d,0x9c,0x70,0xe4,0x61,0xcb,0x8c,0x2a, -0x29,0xf4,0x8f,0xde,0x34,0xed,0x57,0x17,0xba,0x4d,0x46,0x4b, -0x10,0xb9,0x53,0xf5,0x7d,0x15,0x93,0xf5,0x89,0x09,0xdb,0xa4, -0x41,0x31,0x4e,0xc2,0xe0,0x3e,0xeb,0x78,0x90,0x49,0xd1,0x65, -0xc7,0x60,0xcf,0xef,0xd6,0xa6,0xb3,0x69,0x5c,0xa3,0x0a,0x65, -0xef,0xd1,0x8b,0x6d,0xef,0x4b,0x77,0x57,0xde,0xdb,0xc2,0x5e, -0x0a,0x61,0xb2,0xfd,0x0e,0x05,0xf6,0x2c,0xd9,0x76,0x59,0x17, -0xb1,0x18,0x9e,0x2a,0x1b,0x30,0xd7,0x78,0x65,0x0b,0x36,0x74, -0x3e,0x34,0x6f,0xb4,0x18,0xdc,0xbe,0xf5,0x24,0xa9,0xb0,0x4c, -0xc9,0x2a,0x2a,0xcc,0x2a,0x2b,0xcc,0x8a,0x15,0x2a,0xee,0x56, -0xd6,0xf8,0xce,0xaa,0x91,0x53,0x74,0x82,0xed,0x34,0xa4,0xb2, -0x50,0xbb,0x95,0x0e,0x04,0xae,0xce,0x38,0xc9,0x94,0x09,0x30, -0xe5,0xd2,0x31,0x7e,0xad,0xbe,0xfe,0x8e,0xc6,0x5e,0xaf,0x7e, -0x1c,0x1b,0x09,0xe3,0xe5,0x65,0xa6,0xb3,0xfe,0x91,0xda,0x59, -0x7e,0xad,0x48,0xd1,0x5b,0x9b,0x24,0x5b,0x13,0xe8,0x2e,0x50, -0x15,0x61,0x93,0xcb,0xb4,0x86,0x68,0x4e,0x29,0x5a,0xa8,0x68, -0xa6,0x67,0xce,0x90,0x90,0x7d,0xa5,0xca,0x03,0xef,0x2c,0x54, -0x79,0x3a,0xde,0x6d,0xa7,0x03,0x7a,0x26,0xdb,0x49,0x47,0x2c, -0xe9,0xb8,0x75,0xe9,0x50,0x5d,0x98,0x19,0x42,0xb2,0x32,0x42, -0x8c,0x12,0xb7,0x67,0xa5,0xeb,0x38,0xc3,0xb9,0x00,0x92,0x4e, -0x5a,0x11,0x1d,0x96,0xe2,0x34,0x83,0x37,0x0c,0x9e,0xb3,0x37, -0x33,0x50,0xff,0xfe,0x3d,0xcc,0xbe,0x77,0x77,0xde,0x90,0xb4, -0x12,0xc7,0x07,0x04,0x6b,0x11,0x52,0xf1,0x01,0xd9,0xc8,0x70, -0x40,0xa5,0x02,0x09,0x91,0x21,0x47,0x28,0x14,0x51,0x8b,0x0e, -0x6e,0xa8,0x86,0x57,0x0b,0xfa,0x23,0xe7,0x5f,0xcf,0x7e,0xe7, -0x0d,0xea,0x5e,0xcd,0x2b,0x09,0xc2,0x44,0x89,0x0f,0x9f,0x85, -0x55,0x64,0xd4,0x4b,0xc9,0xa8,0x97,0x92,0x11,0xec,0x43,0x05, -0xb1,0x8b,0x95,0x32,0x9f,0x96,0xd3,0xbb,0x7d,0x78,0x0d,0x09, -0xe7,0x4f,0x79,0x65,0xcf,0x17,0x08,0x31,0xc2,0x00,0xf5,0xfc, -0x5e,0x37,0x45,0x33,0x4e,0x88,0xf5,0x75,0xf9,0x8c,0x3c,0x86, -0x57,0xe5,0xb4,0xd0,0xa4,0x61,0xd0,0x37,0x8b,0xb3,0xba,0x99, -0x83,0xf1,0xf2,0xbf,0xc7,0x59,0x57,0xb2,0xda,0xf0,0xbc,0xe7, -0x19,0xe1,0xaa,0x9b,0x96,0xe2,0x5a,0xcf,0x33,0xbd,0x5d,0xcf, -0xec,0x9c,0xbe,0x6f,0x7a,0x49,0x73,0xa7,0xae,0xe9,0xfb,0x9a, -0x51,0xa3,0x90,0x58,0x1c,0xd3,0x0c,0x53,0x80,0xa1,0x06,0x04, -0xa4,0x7d,0xbf,0x6d,0x09,0x81,0x54,0xba,0xd0,0x84,0xb7,0x87, -0x87,0xc8,0x88,0x90,0xcb,0x88,0x35,0x58,0x1b,0xa8,0x9d,0xf4, -0x10,0x54,0x6c,0x0a,0x4c,0x91,0x77,0x6e,0x91,0x4c,0x84,0xa6, -0xc8,0x3b,0xb7,0x08,0xfd,0xd2,0x99,0x98,0x47,0xcc,0x3a,0xf7, -0x3e,0xe1,0x09,0x58,0xbb,0x26,0xad,0x14,0x8c,0x08,0xd9,0x56, -0x4c,0xed,0xf2,0x1c,0xdd,0x96,0xb3,0x62,0x6d,0x63,0xf7,0xad, -0x5d,0x31,0x1f,0x94,0x48,0xff,0x95,0x49,0x32,0xc5,0x67,0xa5, -0xab,0xf7,0xcf,0x81,0xfe,0x40,0x3f,0xf4,0x8a,0x91,0x8c,0x94, -0xe2,0xd4,0x31,0xc8,0x23,0xc9,0xca,0xa4,0x2f,0x78,0xe0,0x46, -0x49,0x57,0xf2,0xb4,0x2d,0x3b,0x81,0x6f,0xea,0xc5,0x7a,0xb8, -0xc6,0xaa,0xff,0xf4,0xa9,0xb7,0x11,0xf0,0x45,0xd0,0x88,0x14, -0x6a,0xa8,0xaf,0x41,0x11,0x7a,0xeb,0x2a,0x05,0x78,0x93,0x4c, -0x23,0x0f,0x6a,0x78,0x3e,0x1e,0x4c,0xe6,0xed,0x35,0xfd,0xf1, -0x4d,0x14,0xe9,0x45,0x20,0x4c,0xa4,0xe2,0x50,0xca,0x21,0xbe, -0x28,0xdb,0x56,0x3f,0xb5,0x18,0x76,0x31,0x2a,0xb5,0x07,0x80, -0x75,0x5d,0xa1,0x5f,0xa7,0xe7,0x04,0xa4,0x10,0x5a,0x91,0xba, -0xa3,0xaa,0x0a,0x5a,0x47,0x52,0x58,0x92,0x7e,0x03,0x92,0x7c, -0x9f,0x28,0x3c,0x53,0xe5,0xba,0xf4,0xd4,0x36,0x71,0x19,0xe8, -0xd6,0xd5,0xb4,0x31,0xd5,0xb4,0x49,0xf3,0xe3,0x00,0x41,0xef, -0xee,0x34,0xa7,0xe0,0x7b,0xb0,0xee,0x7a,0x9a,0xa1,0xdf,0xd2, -0x71,0x03,0xc9,0xa2,0xe2,0xad,0x95,0x64,0xf0,0x50,0x8d,0x1a, -0x0b,0x22,0xc1,0xe8,0xed,0xed,0x9b,0x59,0x32,0x7a,0x10,0xe4, -0x86,0x3c,0xda,0x3e,0xfe,0x27,0x6b,0xf4,0xa3,0x36,0xde,0x6e, -0x89,0x97,0xcd,0x77,0xfb,0xdb,0x21,0xc2,0xd8,0xfd,0xb0,0xfe, -0x18,0x65,0x45,0x73,0x93,0xc1,0x6e,0x8d,0xf5,0x8e,0x84,0x29, -0xdb,0x9c,0x9d,0xa7,0x74,0xd6,0x6a,0x5e,0xcc,0x20,0x83,0xbe, -0xbc,0x0c,0x65,0xa1,0x9f,0x50,0x21,0x47,0x4e,0x0c,0xa5,0x19, -0xb9,0x4a,0x48,0x84,0x7e,0xa1,0xe7,0xf4,0x90,0x5f,0x25,0x83, -0x13,0x77,0x2a,0xe5,0xe6,0x0d,0xcb,0x87,0x60,0x3d,0xa8,0xee, -0x71,0xd3,0xc9,0x3b,0x95,0x7b,0xa3,0xae,0x42,0x2b,0x9e,0x68, -0xd5,0xa8,0x34,0x2e,0xa3,0x8f,0x38,0x71,0x76,0x29,0x35,0xb1, -0x1d,0x3b,0xcc,0x59,0x4a,0xc9,0x52,0x4a,0xed,0x44,0x56,0x2d, -0x68,0x9d,0xaf,0x26,0x9e,0xa5,0xc6,0xa2,0x83,0xe3,0x7f,0x78, -0xd3,0xbb,0xd4,0x57,0x36,0x87,0xca,0x01,0xdd,0xb4,0x36,0xaa, -0x8b,0xc0,0xe2,0xbb,0xdb,0x32,0xb0,0xcc,0xcc,0x33,0x76,0x55, -0x64,0xb6,0xc7,0x60,0x56,0xab,0x99,0xdf,0x66,0x5a,0xa4,0xe6, -0xa3,0x73,0x4c,0x7c,0xbd,0x9a,0xcf,0x01,0x61,0xea,0x29,0xfd, -0x5d,0x9c,0x75,0x6e,0x1d,0x06,0x49,0xef,0xba,0xd2,0x7e,0x93, -0x9f,0x25,0x5d,0x47,0x1f,0x3e,0xfa,0x8d,0x68,0x16,0xf3,0xbb, -0x87,0xa3,0x37,0x24,0x14,0x69,0xf7,0xb6,0x5c,0x7a,0xf4,0xd8, -0x28,0x17,0x21,0xd5,0xe1,0x7e,0xbb,0x42,0x62,0x73,0xe5,0x73, -0x62,0xab,0x8b,0xc3,0xb8,0xa3,0xe1,0xb6,0x58,0xcd,0x71,0x85, -0x64,0x8d,0xb2,0xa3,0x9a,0x6f,0x67,0x69,0x86,0xfb,0xc5,0xe9, -0x0e,0xec,0x13,0x9d,0x58,0x32,0xc9,0x99,0xec,0xb0,0xcf,0x0f, -0xb2,0xe3,0x6b,0x68,0xaa,0x33,0x69,0x4a,0x88,0xd7,0x51,0xfd, -0xf8,0xb6,0xd0,0xd1,0x22,0xce,0x20,0xfd,0x1f,0xd3,0x7c,0xba, -0xb4,0x03,0xa2,0x9e,0x9e,0xa4,0x41,0xbd,0xdb,0x44,0x64,0x26, -0x3d,0xb3,0x6a,0xef,0x97,0xcc,0xd5,0x50,0xb0,0x84,0x66,0x28, -0xa6,0x1c,0x58,0x1c,0xa3,0x01,0xd4,0xb3,0x9c,0x9b,0xb9,0x8c, -0x7a,0xec,0xc1,0xef,0x54,0x2b,0x30,0x52,0x43,0x7b,0x6a,0xb0, -0x3d,0x95,0x95,0x2d,0xf4,0x6e,0x96,0x5f,0xaa,0x1a,0x78,0xe3, -0x7c,0xe3,0x0b,0x4b,0xe5,0x92,0xa2,0x70,0x14,0xba,0x70,0x34, -0xbe,0x50,0xfe,0x34,0x3c,0x6a,0xbe,0x55,0x4e,0x35,0x84,0x42, -0xd8,0x78,0x85,0x32,0x0e,0x8c,0x1e,0xf2,0xc5,0x9a,0xa7,0x87, -0x8c,0xff,0xc8,0x7a,0xa2,0x5e,0x20,0x4c,0xf7,0xc8,0x2f,0x9e, -0x4f,0xf1,0xbd,0x81,0x4c,0xff,0x92,0xab,0xf9,0xc8,0x5e,0x3f, -0xff,0xc2,0x69,0xd6,0x14,0x2e,0xf5,0xfd,0x8d,0xb3,0x83,0x5b, -0xb9,0xf9,0x53,0xd0,0x69,0xb9,0x68,0x88,0xac,0x34,0xc2,0xe8, -0xd3,0x5a,0xdb,0x10,0x59,0x5b,0x92,0x2a,0x12,0x77,0xee,0x13, -0x1f,0xda,0x0e,0xb3,0x36,0xb2,0x74,0xd1,0x47,0x44,0x62,0xed, -0xe4,0xdc,0x43,0x64,0x72,0x85,0x1b,0x55,0x37,0xbf,0x4d,0x65, -0xdf,0x3e,0x95,0x8e,0xa7,0xce,0xe6,0x91,0xd5,0x54,0x77,0x8e, -0x96,0x64,0x35,0x13,0xb7,0x73,0x75,0xfa,0x07,0xdd,0x35,0x2a, -0x65,0xad,0xbb,0x47,0xa5,0x64,0xdb,0x44,0x6a,0xe3,0x2b,0x67, -0x5e,0x4a,0x6e,0x17,0x09,0x22,0x39,0xcb,0x3b,0x91,0x3f,0x5b, -0x2c,0x06,0x99,0xb2,0xb7,0x15,0xeb,0x9b,0xb0,0x8e,0xcf,0xca, -0x21,0xb3,0x1c,0x64,0xa6,0x1d,0xb7,0xdc,0xf6,0x16,0x8d,0x16, -0xc7,0x53,0xdd,0x6d,0x6c,0x88,0x4f,0x4f,0x90,0x83,0x7a,0x96, -0x4b,0xcf,0xa2,0x8c,0xd2,0xab,0x6d,0x60,0x63,0xf3,0x6e,0x36, -0xbe,0xe5,0x0b,0xbc,0x5f,0x87,0x84,0x15,0x1d,0x12,0x42,0x87, -0xec,0x53,0xe7,0x96,0xf9,0xdf,0x6f,0x7b,0xdb,0x34,0xc0,0x9f, -0xc3,0x4e,0x8e,0xc8,0x3c,0xec,0xac,0xe3,0xaf,0x8f,0x5b,0x36, -0x9c,0xd7,0xd1,0xc7,0xad,0xa3,0xf9,0x60,0xfd,0xf5,0xd7,0xe1, -0xdd,0x41,0xd6,0x67,0xb4,0x7e,0x7b,0xad,0x2a,0xc8,0xf6,0xab, -0x20,0xbb,0x6f,0x05,0x99,0xaa,0x60,0x48,0xda,0x66,0xbf,0x56, -0x80,0xcd,0x01,0xff,0x4a,0xfd,0x74,0xf0,0x91,0x26,0x19,0xf3, -0x05,0xab,0xb6,0xf6,0xba,0x6b,0xb2,0xdb,0x38,0x51,0x98,0x7c, -0x5f,0xb7,0x0d,0xd1,0xf4,0x18,0x4e,0x96,0xef,0xeb,0x82,0xed, -0x4c,0xad,0x30,0xf8,0x69,0x31,0xa9,0xdd,0x58,0x30,0x6d,0x2a, -0x69,0x40,0x05,0x62,0x1d,0x8a,0x0c,0x74,0x66,0x28,0x96,0x55, -0xfe,0xdd,0x44,0xb7,0xb7,0x0e,0x22,0xa2,0x77,0x1d,0x46,0x48, -0x29,0xed,0x69,0x41,0xda,0x3a,0x90,0x89,0xeb,0x90,0xf6,0x43, -0x5c,0x31,0xed,0x51,0x05,0x90,0x0f,0x46,0xc0,0x4d,0xbc,0x58, -0xca,0x32,0xf0,0x8f,0x4f,0x65,0xbc,0x75,0xe8,0xe3,0x7b,0x7e, -0x67,0x6b,0x0c,0xea,0xba,0x8c,0x10,0x1c,0xf2,0x43,0xcc,0x84, -0x71,0x03,0x70,0x07,0x96,0x7d,0x70,0x77,0xb7,0x0e,0x0f,0xac, -0xcd,0x03,0x5c,0x85,0xea,0x29,0x81,0x6b,0x92,0x9b,0x6d,0x99, -0x24,0x36,0x93,0xc4,0xae,0x39,0x2d,0x0b,0x88,0xf2,0x30,0xca, -0x42,0x5a,0xd6,0xbb,0x63,0xa7,0x47,0x7c,0x80,0x7c,0x43,0x2c, -0x96,0x81,0x7f,0x7c,0x2a,0xe3,0x65,0x48,0x6c,0x16,0x9c,0x65, -0x4c,0x2c,0x82,0x43,0x7e,0x88,0x99,0x48,0x6c,0x06,0xc4,0x66, -0x16,0xb1,0x59,0x78,0x60,0xed,0x69,0x20,0xb1,0x7a,0xc6,0xe2, -0x9a,0x7c,0xdd,0xa3,0xcc,0x61,0x9b,0x9a,0x2e,0x31,0xca,0x4e, -0x51,0x7c,0x76,0xd2,0x50,0x64,0x2c,0x48,0x17,0xa6,0x57,0xe4, -0xbc,0xfc,0xd2,0x1d,0xa0,0xbf,0xb7,0xf5,0x83,0x62,0xaa,0x4b, -0x1f,0xb6,0x3d,0x4f,0x1f,0xb2,0xa9,0x48,0x9f,0x81,0x74,0x61, -0x7a,0x45,0x66,0x2b,0x7a,0x14,0xcf,0xf5,0xf7,0x36,0xd6,0x23, -0xe9,0x34,0x24,0x50,0x08,0x32,0xa8,0xa2,0x87,0x42,0x0d,0x86, -0x1a,0x0c,0x0a,0x29,0x17,0x38,0x94,0x14,0xd7,0xcd,0xc0,0x2d, -0x7f,0x5d,0xc0,0x59,0xff,0xdd,0xdd,0x79,0x5b,0xad,0x8e,0x1d, -0xd3,0x86,0xec,0xe1,0x8d,0xbd,0xa8,0x2c,0x56,0x79,0x70,0x00, -0xf4,0x9b,0x2e,0x8c,0xac,0x0e,0xbc,0xbb,0x3b,0x38,0x00,0xda, -0x33,0x2b,0xd3,0x70,0x8f,0x9f,0xc9,0xe0,0x47,0x34,0xd8,0xa0, -0x22,0x95,0x2b,0x6a,0xb3,0x69,0xcd,0x77,0x0e,0xd0,0xac,0xa3, -0x83,0xe8,0x13,0xbe,0x90,0x92,0x36,0xa4,0x9e,0x16,0xa0,0x9a, -0xe3,0x5b,0xfd,0x95,0x3a,0x79,0xb7,0x4e,0xde,0x14,0x3e,0xa7, -0xb7,0x69,0x6c,0x12,0xa0,0xec,0x34,0x4e,0xad,0xd2,0x71,0x0e, -0x22,0xbd,0x75,0x21,0x74,0x64,0x4f,0x5b,0xb9,0x46,0xbd,0x5e, -0x6d,0x8a,0x67,0x38,0xe8,0x9c,0x01,0xff,0xdd,0xa2,0xb9,0x4b, -0xff,0x50,0x0a,0xfd,0x73,0x5b,0x83,0x6e,0x25,0x08,0xfe,0x35, -0x25,0x58,0x8c,0xdc,0xcd,0x71,0x4c,0x12,0xbe,0xc3,0x47,0x7f, -0x3f,0x51,0x2d,0x74,0x08,0xf0,0x2d,0xde,0xda,0x94,0xf9,0x90, -0x92,0xbd,0x96,0x31,0x8a,0x7a,0xb5,0xd7,0xab,0xe9,0x08,0x6d, -0xe5,0xda,0xcb,0x99,0xfc,0xf1,0x66,0x15,0xa7,0xfc,0xeb,0x6f, -0xf1,0x68,0xaa,0x7e,0xbf,0xb9,0x02,0xae,0xf1,0xcf,0xef,0x17, -0x09,0xff,0x78,0x0d,0x8b,0xd1,0x05,0xfe,0xa4,0x57,0xee,0xb1, -0xa2,0xf3,0xd5,0x72,0x68,0x05,0xf0,0xf8,0x04,0x26,0x35,0x56, -0x5d,0xf2,0x96,0x33,0x5b,0x70,0x78,0x85,0xea,0xaf,0x6f,0x9e, -0x23,0x0c,0x06,0x02,0x21,0xf0,0x6c,0x37,0x74,0x26,0x81,0xbf, -0x07,0xd1,0x7a,0x07,0xd2,0xbb,0xab,0x84,0x82,0x93,0xc5,0x7e, -0x98,0x41,0x5b,0x76,0x95,0x21,0x20,0x59,0xe0,0x65,0x32,0x1e, -0x27,0xfc,0x0c,0xfe,0xce,0x72,0x36,0xac,0x2e,0x3e,0x5d,0xc1, -0x9a,0x6b,0x77,0x49,0x02,0x53,0x85,0x66,0xd3,0xe5,0xd5,0xce, -0x22,0x08,0x24,0x0b,0xbc,0xde,0x8f,0xbe,0xd7,0x0e,0x69,0x6f, -0x40,0x20,0xb6,0x96,0x40,0x00,0x0b,0xf6,0x23,0x2c,0xbf,0x38, -0x0c,0x66,0x49,0xa9,0x60,0xc3,0xf7,0x4a,0x7e,0xbe,0xac,0x46, -0x29,0x01,0x10,0x65,0x5a,0x14,0x0e,0x25,0xa2,0x73,0x2b,0xb6, -0x87,0x12,0x10,0x2b,0x4a,0x70,0xf3,0xdc,0x5a,0x6a,0x48,0x44, -0xd9,0x7e,0x78,0xb2,0xad,0x68,0xca,0x04,0xaa,0x0a,0x97,0x82, -0xdd,0x8a,0xb0,0x20,0x6a,0x55,0xd8,0x08,0x70,0x2b,0xaa,0x2a, -0x21,0xac,0xc2,0x68,0xc3,0xef,0x40,0x5c,0x10,0xcf,0x6a,0x9c, -0x04,0xba,0x1d,0x5d,0x5e,0x70,0x2b,0x91,0x21,0xe0,0x56,0x54, -0x25,0x22,0x5d,0x85,0xec,0xf5,0x1e,0x0d,0xcd,0x0b,0x7b,0x29, -0x2e,0x04,0xaa,0xc4,0xa2,0x27,0x13,0xb7,0x1c,0xe9,0x57,0xa3, -0xf7,0xb4,0xa6,0xe5,0xb7,0x6f,0x30,0x13,0xb1,0x46,0xb5,0xaf, -0x06,0x0f,0xbe,0xba,0x78,0xf0,0x55,0xfc,0xe0,0xab,0xb7,0x0f, -0xbe,0x7a,0x57,0x2b,0x81,0x03,0x98,0xc9,0xf1,0x57,0xa3,0xe3, -0x62,0xae,0xc4,0xf0,0x43,0xfb,0xab,0x97,0xed,0xaf,0x5e,0xd7, -0x6c,0x6d,0xfe,0xe5,0xf4,0xb8,0x8d,0xf5,0xd9,0xc5,0x05,0xbe, -0x5e,0x4d,0x97,0x4e,0x64,0x0d,0x8c,0x9e,0x71,0x33,0x5e,0xc6, -0xc8,0xc8,0x18,0x93,0x85,0x64,0x82,0x1d,0x8c,0x45,0xff,0xdb, -0x60,0xba,0x1a,0x2c,0xa8,0xce,0x18,0x66,0x3c,0xfe,0xf9,0x12, -0xdf,0xb8,0x82,0xbf,0xcf,0xe6,0x8b,0x64,0x4c,0xdf,0x98,0xfa, -0xdf,0x56,0x34,0xdb,0xfd,0xb7,0xd5,0x18,0xbf,0x9e,0xad,0xde, -0xaf,0x52,0xdc,0x13,0x79,0x1d,0xcf,0x97,0x31,0xbd,0x66,0x26, -0x6a,0x3f,0x0f,0x97,0x33,0xfe,0xf5,0xd3,0xec,0x46,0x25,0x7e, -0x1b,0x0f,0xf9,0x67,0xae,0xfa,0x7c,0x2b,0x80,0x14,0x26,0x83, -0x29,0xe0,0xfa,0xed,0xda,0xb9,0x72,0xae,0x9b,0x2b,0xe6,0x2a, -0xb9,0x3a,0xae,0x89,0x27,0x3c,0xac,0xa4,0x99,0xa1,0xf1,0xa4, -0x6a,0xa4,0x85,0x36,0x68,0x38,0xb3,0x59,0x42,0xc7,0xce,0xda, -0x7f,0x81,0xcf,0xe3,0xb9,0x20,0x70,0x98,0x73,0x85,0x74,0xc4, -0xe7,0x81,0x83,0x0f,0x69,0x70,0xd2,0x46,0x38,0x58,0xe4,0xfe, -0x33,0x48,0xad,0x04,0xd7,0x93,0x1a,0x25,0x58,0xb3,0xa1,0xe7, -0xd7,0x25,0x6c,0x0e,0x85,0x43,0x88,0x5b,0x42,0x5e,0x50,0x53, -0x2d,0x4a,0x23,0xfb,0x8b,0x5d,0x35,0x5d,0x80,0x26,0xcc,0xf4, -0x2e,0x10,0x4e,0xfd,0x2e,0x20,0x34,0x71,0x27,0x6b,0x68,0x3c, -0x01,0xdc,0x34,0xbe,0x7d,0x20,0x8d,0x08,0xaf,0x15,0x9f,0x88, -0xc0,0x7a,0x71,0x22,0xdb,0xd1,0x5c,0xa1,0x92,0xe4,0x74,0xa8, -0xbf,0xd9,0xae,0xc0,0xef,0x6c,0x17,0x37,0x09,0xd4,0x2d,0xb7, -0x2f,0x17,0x19,0xba,0x11,0x5a,0x3c,0xa4,0x41,0x69,0x7d,0x14, -0x19,0xe3,0x32,0x10,0x41,0x4a,0xf9,0xf7,0xf3,0xe5,0x3b,0xc7, -0x3a,0x37,0x4c,0xb3,0x05,0x8f,0x3a,0x80,0xf3,0x3a,0xf6,0xe3, -0x67,0xfc,0x1c,0x16,0x65,0x34,0x10,0xa4,0x71,0x1a,0x3f,0x3c, -0xd2,0xad,0x74,0xe7,0x73,0x68,0x01,0x75,0x63,0x49,0x86,0xef, -0x1f,0x3f,0x39,0x7d,0x18,0x3f,0x92,0xd7,0x3e,0x5d,0x1b,0x52, -0x3f,0x57,0x6b,0xd1,0x98,0xe1,0xe9,0x24,0xf5,0x3e,0xb4,0x6b, -0x39,0xfb,0x71,0x76,0x1b,0x2f,0x9e,0x0f,0x52,0x3c,0x6a,0x49, -0xa2,0xc7,0x8d,0xa4,0xa3,0x63,0x91,0xdc,0x50,0x3c,0x56,0xc2, -0xd9,0x03,0xe0,0xfe,0x9e,0x43,0xc9,0x2b,0x1f,0x4a,0x7e,0x45, -0x4f,0x36,0xac,0x6f,0xb0,0x1d,0xeb,0x89,0xff,0xd5,0xe3,0xfd, -0x06,0x59,0x99,0x58,0x3c,0x3e,0xb2,0x98,0x2b,0x0b,0x14,0x85, -0x44,0x09,0x76,0xb1,0x8b,0x34,0x1d,0xa5,0x7d,0x95,0xef,0x7b, -0x2e,0x53,0x47,0xae,0x32,0xdd,0xc7,0x8f,0xa9,0x3d,0xd9,0x41, -0x14,0x25,0x3e,0x05,0x5f,0xd4,0xec,0xab,0xe3,0x5a,0x20,0x52, -0xac,0x73,0xa5,0x49,0xe7,0x93,0xe0,0x69,0x98,0x82,0xd4,0x31, -0x1c,0xd7,0x0d,0xc8,0xfe,0x13,0x5b,0xb5,0xb1,0x06,0xd0,0x6d, -0x1c,0x5f,0xeb,0x4a,0x52,0x9a,0xdb,0x9c,0xbc,0x34,0x97,0x99, -0x1b,0x39,0x04,0xe2,0x8c,0x30,0x09,0x56,0x1c,0x64,0x08,0x2a, -0x47,0x99,0x0b,0xcb,0x89,0x1a,0xee,0x52,0xbd,0x68,0x67,0x4f, -0xca,0x8e,0x39,0x00,0x65,0x7e,0x89,0x73,0x00,0xbf,0xc4,0x9e, -0x3d,0x51,0xfb,0xf6,0x00,0xfa,0x71,0x36,0xbb,0x5e,0xcd,0x73, -0x05,0x38,0xb1,0xba,0x10,0xcf,0x67,0x3b,0xea,0x71,0x26,0xbd, -0x92,0xe2,0x7b,0xd6,0x5c,0x81,0x86,0xe6,0xd5,0x6d,0x14,0xf0, -0xbc,0x9f,0x2b,0xb0,0x4f,0x9d,0xa5,0x05,0x77,0xb7,0xb8,0x38, -0xd1,0x97,0xa2,0xd8,0x9b,0x82,0x0a,0x54,0xf3,0x78,0x81,0xc7, -0x55,0x40,0xc9,0xf1,0x3f,0xbe,0x3a,0xce,0x99,0x67,0xaf,0x06, -0xa3,0x34,0xfa,0x54,0x6b,0xd4,0xda,0xb5,0x9a,0x38,0x6f,0xd7, -0x1e,0xd4,0x44,0xd0,0xae,0x05,0xb5,0x4d,0x0e,0x0e,0x80,0x06, -0xed,0x72,0xdf,0xc6,0x52,0xb6,0xf7,0x46,0x7a,0x60,0xf5,0x37, -0xe2,0xd9,0x8e,0xa2,0x39,0xf0,0x8b,0x6d,0xe0,0xc5,0xa6,0x72, -0x61,0x39,0x89,0x42,0xf1,0x6f,0x76,0x16,0x2f,0x14,0x19,0xb6, -0xdd,0xe6,0x7a,0xe5,0xa6,0x30,0x30,0xd5,0xc2,0x2d,0xe6,0x05, -0xec,0x9a,0xa9,0xde,0xc8,0x68,0x5e,0x31,0x17,0x2d,0x50,0xb4, -0xf1,0xe7,0x17,0xfd,0xe1,0x5e,0x45,0xe5,0x16,0x80,0x2c,0xfb, -0xe2,0x73,0xca,0x7e,0x15,0xb6,0xee,0xee,0xc2,0x96,0x44,0xf1, -0x61,0x5f,0x14,0x61,0xbd,0x44,0x63,0x22,0x21,0x27,0x80,0xe5, -0xc7,0x7b,0x11,0xe2,0xee,0x47,0x48,0x14,0x93,0xfb,0xa1,0xe0, -0xee,0xad,0x87,0xb2,0x19,0x2f,0xef,0x49,0x80,0xdc,0xd6,0x90, -0xa5,0xe7,0xa5,0x52,0x65,0xf1,0xec,0x2c,0x0a,0x5b,0xdd,0xda, -0xab,0x97,0x30,0x94,0x9e,0xbd,0x84,0x01,0xf4,0xfa,0x5e,0xd5, -0xbd,0xb6,0x9a,0x8a,0xd5,0xfd,0x75,0xef,0xd2,0x25,0xda,0x9f, -0x99,0x8e,0x68,0x6e,0x2b,0xa9,0x96,0xdb,0x51,0x7f,0xbb,0x6f, -0x3d,0x93,0x59,0x59,0x3d,0xeb,0xdd,0x03,0xc8,0x17,0x6f,0xb7, -0x03,0xf1,0x08,0xcb,0xee,0xc5,0x36,0x63,0x4d,0x7f,0x15,0x06, -0x81,0x24,0xe6,0xdd,0x67,0xa3,0x88,0x1f,0x02,0x8a,0x87,0x80, -0xe2,0x37,0x4d,0x2a,0x5a,0x94,0xa2,0xf6,0x55,0xad,0xb8,0x37, -0x04,0x89,0x1b,0xa3,0x27,0xe7,0xb0,0xba,0x88,0x17,0xa4,0x27, -0x9d,0xa4,0xbf,0xc1,0x44,0xad,0xb5,0x23,0x68,0xc2,0xb2,0x4c, -0xd0,0x78,0x4e,0xf2,0x4b,0xa3,0xe3,0x40,0x9b,0x15,0xb3,0x2c, -0x7d,0x45,0x89,0x3f,0x62,0x08,0xb3,0x18,0x1b,0x02,0x0a,0xca, -0xc9,0x81,0x5e,0x16,0x71,0x31,0xe9,0x07,0x37,0x09,0x25,0xb8, -0xf5,0x10,0x54,0x45,0x59,0xea,0x87,0x42,0x71,0x69,0x62,0xfc, -0x98,0xa3,0xcc,0x1a,0xb1,0x62,0x52,0x42,0xb5,0x7c,0x53,0xfb, -0x65,0xbe,0x18,0x8d,0x33,0x18,0x5f,0x4e,0xf2,0xb3,0xc9,0xab, -0x09,0x0c,0x21,0x27,0x4d,0x8e,0x11,0x18,0x1b,0x05,0x26,0x32, -0x6e,0xde,0x4d,0x00,0xa1,0x2f,0x63,0xb2,0xac,0xfe,0x6f,0x55, -0x85,0x79,0x07,0xc2,0x92,0x64,0x8b,0xb5,0xb4,0x11,0xfd,0xb6, -0x2c,0x07,0x85,0x16,0x64,0xd6,0xc9,0x21,0xee,0xbc,0x73,0xd3, -0x94,0x94,0x59,0x92,0x45,0xe9,0xbf,0x29,0xf1,0x72,0x71,0x27, -0x60,0xe7,0x0e,0xc6,0xaf,0x78,0xea,0x36,0x42,0xc6,0x4f,0xa3, -0xd3,0x54,0xfe,0xf7,0xf4,0xe8,0xef,0xa3,0xba,0x99,0xcf,0x07, -0xc0,0x30,0x6d,0x2a,0x70,0x08,0xff,0xc1,0xa4,0x0d,0x43,0x62, -0xd2,0x0e,0x8d,0xb9,0x2d,0x85,0xdf,0xb6,0x31,0x65,0xd2,0x5f, -0x97,0x43,0xc7,0x2c,0xe4,0xd4,0x17,0xe9,0xac,0x08,0xe7,0xd5, -0xbe,0x7a,0xd7,0xf8,0x6a,0xd2,0xf8,0x6a,0xf4,0x46,0x6d,0xec, -0x34,0xbf,0xfa,0xf1,0xb7,0x5a,0xa1,0x96,0x04,0x4a,0xbb,0x7b, -0x4c,0xb0,0xac,0x7a,0xf1,0xfa,0x67,0x8e,0x91,0x76,0x78,0x58, -0xc7,0xe5,0x33,0xcd,0x74,0xb5,0x56,0x10,0x04,0x8d,0x20,0x84, -0xff,0xbf,0x09,0x82,0x36,0xfd,0xbf,0x09,0x49,0x80,0xb4,0x5b, -0xa0,0xe8,0x27,0x7a,0x7b,0xb9,0x5d,0x48,0x17,0x15,0x90,0x4d, -0xe2,0xa9,0x15,0x65,0x99,0x43,0xb4,0xc9,0x55,0x01,0xac,0xc8, -0x34,0x19,0x32,0x47,0xad,0x05,0x92,0xf4,0xa7,0xc1,0x4f,0x6c, -0xf3,0x77,0xc9,0xf7,0x91,0xd7,0x5e,0x55,0xd5,0x2c,0x67,0xdc, -0xb0,0xa8,0x00,0xa0,0xb3,0xb4,0x5d,0xce,0x03,0x65,0xdf,0x2d, -0x18,0x7b,0x9b,0x21,0x8c,0x4f,0xec,0x95,0x1c,0xa6,0x1d,0x43, -0x5a,0x61,0x2d,0x57,0x58,0x14,0xa2,0xa8,0x3a,0xab,0x68,0x9c, -0x1b,0x5d,0x64,0x95,0xcb,0xc2,0xdc,0xde,0x81,0xd9,0x8d,0xf7, -0x73,0x2d,0xb2,0x56,0x38,0xf4,0x9d,0x5b,0xb6,0xa8,0x8d,0x5d, -0x67,0x8d,0xc3,0x80,0xc5,0x35,0xce,0x84,0x54,0xc3,0xe7,0xf0, -0x08,0x37,0x0c,0xf2,0x3c,0x82,0xb4,0xcf,0xe3,0x51,0x0e,0xd9, -0xbe,0x3c,0x32,0xe7,0x22,0x7e,0xae,0x45,0x86,0x47,0xfc,0x5d, -0xda,0x6c,0x97,0x47,0x12,0xb0,0xc8,0xa3,0x2b,0x50,0xd2,0x7b, -0xed,0x56,0x21,0x00,0xce,0x65,0x51,0xd5,0x3e,0xca,0xf1,0x69, -0xd0,0x29,0x61,0xe5,0xc9,0x69,0xfc,0xe8,0xc8,0xcb,0x33,0x13, -0x53,0x1b,0x0a,0x25,0x5e,0xed,0x96,0xbf,0x3e,0x8b,0xc1,0x54, -0xc5,0x67,0x70,0x58,0x9d,0x75,0xf9,0x0e,0x37,0x0c,0x77,0xf1, -0xab,0x84,0x5d,0x2e,0x67,0x09,0xa8,0x44,0xf6,0x70,0xc6,0xfa, -0x42,0x3b,0xa4,0xa4,0x5a,0xf6,0xdc,0x1e,0x65,0xa3,0x35,0xb7, -0x33,0xb8,0xef,0x96,0x9e,0x3a,0x57,0xf3,0xdd,0x66,0x58,0xf2, -0x46,0x67,0x17,0x25,0x2d,0xcd,0x49,0x1b,0x81,0x69,0xa6,0x38, -0xf3,0x02,0x79,0xe7,0xe3,0x2b,0x8c,0x69,0xd4,0x03,0xed,0x21, -0x1e,0xc5,0xf8,0x90,0x35,0xfc,0x73,0x02,0xc6,0x13,0x0c,0x15, -0xf8,0xfb,0x48,0x3c,0x85,0xff,0xc2,0x27,0xf0,0x0f,0x76,0x2d, -0x3e,0x6b,0x0d,0xff,0xb6,0x42,0xfc,0xfd,0xf0,0xa4,0x05,0xff, -0xd2,0xbe,0x9c,0x08,0x1f,0xb7,0x30,0xe3,0x34,0x78,0x48,0xf9, -0x8f,0x9e,0xb6,0xe2,0x53,0xf1,0xf8,0xf1,0xe3,0x53,0xf8,0x73, -0x12,0x3e,0x82,0xb2,0xa7,0x66,0x5b,0x9c,0xea,0xa5,0x39,0xf7, -0x65,0xbc,0xbc,0x9a,0x8d,0xf0,0xdc,0xd9,0xd5,0x20,0x22,0xec, -0x8b,0x7c,0xd2,0xa3,0x62,0x52,0x58,0x92,0x76,0x12,0x58,0x69, -0x3c,0xd6,0x1c,0x6c,0x32,0xe9,0x51,0x09,0x54,0x49,0x9a,0x83, -0x0d,0xe5,0xcb,0xc1,0x45,0x09,0x27,0xf9,0x84,0xd3,0x42,0x91, -0x96,0x95,0x82,0x86,0x49,0x98,0xfb,0xb6,0xf3,0x71,0x37,0xc8, -0x25,0x98,0xcc,0xc4,0x62,0x8a,0x5d,0x2f,0x6e,0x82,0x89,0xb0, -0x5f,0xc6,0xe2,0xef,0xe5,0xa2,0x5f,0xb3,0x58,0x59,0xec,0x30, -0xf3,0xd7,0x64,0x50,0xb8,0x55,0x6c,0x70,0x99,0xec,0x6f,0x6a, -0x7e,0xe9,0x1d,0x48,0x5b,0x3c,0xcb,0xca,0x5d,0x3c,0xf8,0x6a, -0x54,0x5e,0x34,0x3c,0x88,0xac,0x15,0x71,0x69,0xe1,0x41,0x65, -0x61,0xb3,0xd4,0x39,0x3c,0xdc,0x03,0xd1,0x8b,0x07,0x5f,0xcd, -0xb7,0x20,0x52,0x0a,0xa7,0xb4,0x28,0x18,0x42,0xdb,0xda,0xae, -0x67,0x83,0x92,0xc2,0x78,0x32,0x56,0x5d,0xd4,0x4c,0xb6,0x25, -0x45,0xc1,0xf0,0xda,0x5a,0xab,0x73,0xaa,0xbf,0xa5,0xaf,0x23, -0x27,0xe3,0x7b,0x77,0x7d,0x56,0x10,0x0c,0x63,0xe9,0x15,0x86, -0x65,0xb3,0xb0,0xc3,0x6f,0x26,0x7f,0x37,0x58,0x43,0x61,0x61, -0x46,0xf9,0x5e,0x31,0xc4,0x44,0x65,0x5d,0x95,0xad,0xf1,0x37, -0x45,0x9d,0x65,0xf3,0x22,0xfa,0x44,0x8a,0xcd,0x7a,0x89,0x08, -0x5d,0x23,0x45,0xba,0x9c,0xcd,0xe1,0x9f,0xd8,0x5e,0x33,0xb2, -0x0a,0xf4,0xea,0x0c,0x51,0x37,0x20,0xea,0x05,0x7b,0x53,0x03, -0xad,0x72,0xad,0x25,0x21,0x25,0xe2,0x89,0xb1,0x54,0x58,0xd5, -0x3c,0x43,0x54,0x9a,0x96,0x89,0xaa,0xbd,0x37,0xe9,0x05,0xb4, -0xef,0x2d,0x26,0xe8,0xb8,0xbe,0xf1,0x0b,0x98,0xab,0xc6,0xa9, -0x34,0xd2,0xb7,0x0c,0x55,0x86,0xd8,0x36,0x5a,0x2d,0x3f,0x8d, -0x8a,0xd2,0xbb,0xc7,0xac,0x71,0x92,0xa9,0x40,0xb1,0x63,0xe4, -0x2a,0xb7,0x19,0x33,0x78,0x77,0x62,0xdc,0x31,0x84,0x2d,0x17, -0x99,0x2a,0x04,0xdb,0x07,0xb2,0xe3,0xf1,0x52,0x8e,0x62,0xfb, -0x70,0x76,0x9c,0x59,0xca,0x11,0x6c,0x1f,0xd4,0x05,0x6f,0x9d, -0x7f,0x29,0x0c,0x6c,0x2d,0x1a,0xbb,0x87,0xb5,0x91,0x22,0x27, -0x0a,0x7e,0x89,0xfc,0x16,0x47,0x36,0xc9,0xa6,0x33,0xba,0xc9, -0x8c,0xf8,0x93,0x23,0xdc,0xd4,0x58,0xd1,0x28,0xf9,0xf6,0x12, -0x86,0x99,0x03,0x80,0xf5,0xd5,0x02,0xdf,0xee,0x32,0xe3,0x67, -0x11,0xff,0xbe,0x8a,0xd3,0xa5,0xae,0x5c,0x7e,0xe3,0x85,0x8b, -0xf9,0x6c,0x9a,0xc6,0x6f,0xa0,0x20,0x8d,0xa5,0xe6,0x87,0xd4, -0x0e,0xe6,0xb9,0x5a,0x8c,0x05,0x86,0xc4,0xc1,0x4b,0xdd,0x36, -0xe5,0x50,0x01,0xe5,0xd5,0xd0,0xbd,0x22,0x19,0xd2,0xa6,0xf3, -0x31,0x16,0x25,0x6f,0x06,0xfc,0x61,0x8a,0x11,0x65,0x57,0xcb, -0xc9,0xf8,0x1e,0x68,0xb1,0x21,0xc7,0x58,0x86,0xf0,0xe1,0x8f, -0x1c,0xbe,0xf5,0x64,0xfc,0x59,0x0d,0x7d,0xfb,0xf2,0x47,0x6a, -0xe7,0xc6,0xf3,0xf3,0xbe,0xf8,0x93,0xd9,0x68,0x85,0xef,0x3e, -0x4f,0xdf,0xaf,0xc6,0xd0,0x8f,0xfc,0x89,0xa1,0x08,0x40,0xcd, -0x71,0x5a,0x4d,0x60,0x44,0x51,0xce,0xc0,0xf5,0x3b,0x06,0x01, -0x59,0x78,0xb5,0x5f,0x6c,0x88,0x1c,0xd2,0xe7,0xb3,0xe9,0x65, -0xf2,0x7e,0xb5,0x88,0x17,0xd1,0xa7,0x4d,0xc7,0x7c,0x35,0x93, -0x69,0x62,0x79,0x2f,0xcf,0x28,0xae,0xa8,0x18,0x52,0xbe,0x75, -0x43,0xda,0x42,0x0d,0x3f,0xe9,0x99,0x4f,0x8f,0x81,0xc4,0x5f, -0xae,0x96,0xcb,0xb9,0x20,0x4e,0x6a,0xac,0xf2,0xe1,0x38,0x09, -0x89,0x35,0x4a,0x36,0x9c,0xb3,0x27,0xfe,0x39,0xbf,0xde,0x66, -0x15,0x30,0xf4,0x42,0x0e,0x17,0xe7,0x08,0xc3,0x06,0xa6,0x07, -0x39,0xfd,0x0e,0x47,0x27,0x9e,0x0f,0x16,0x83,0x49,0x1a,0x9d, -0xb3,0xfb,0xf5,0xc8,0xfb,0x24,0xfd,0xc4,0x64,0x86,0x24,0xa5, -0x39,0xe2,0x3b,0xcd,0xbf,0x30,0xf3,0x5f,0x51,0x1e,0xbf,0x64, -0xd1,0x9c,0x90,0x18,0xbb,0x47,0xc9,0x7d,0x5f,0x9c,0x37,0x93, -0xf4,0x3b,0x72,0x50,0xb5,0xf1,0xe1,0x93,0x43,0xfc,0xfc,0xb6, -0x53,0x8b,0x6a,0x20,0x51,0x26,0x29,0x85,0xf2,0xaf,0x07,0x97, -0xb1,0x67,0xd7,0xe2,0x77,0x8b,0xc3,0x8e,0xd8,0xe6,0xe9,0x06, -0x70,0x98,0xe9,0x4f,0xc0,0xc6,0x36,0xfc,0x87,0x8e,0xfc,0x6d, -0x7b,0x3d,0x31,0xd8,0xaf,0x1c,0x2e,0x63,0x06,0xb8,0x13,0x32, -0x40,0x0c,0x1b,0x5f,0x53,0xc8,0x0f,0xc8,0x02,0x5d,0xda,0x40, -0xaf,0x81,0xb6,0xc2,0xd0,0x11,0xf1,0x00,0x3d,0x6e,0x66,0x73, -0x3a,0xae,0xc1,0x18,0x9e,0x8b,0xc1,0x30,0xae,0xf5,0x3b,0x4e, -0x6b,0x2c,0x01,0x99,0xc7,0x1c,0x34,0x56,0x13,0x84,0xd1,0xd2, -0xa7,0xcb,0x41,0x32,0x4d,0x3d,0xab,0x02,0xa1,0x01,0x5d,0x16, -0xcb,0xd9,0x7f,0x70,0x91,0xce,0xc6,0xa0,0xa8,0x5f,0x0d,0x96, -0xb0,0x7a,0x9b,0x46,0xc7,0xff,0xc0,0x66,0xa5,0xdd,0xf6,0xdf, -0x8f,0xff,0x7e,0x7c,0x9c,0x98,0xda,0x9f,0x49,0xc0,0xbf,0x2e, -0xc6,0xc5,0x4d,0x22,0x15,0xd3,0x4b,0x6e,0x59,0xe5,0x90,0xf2, -0xa5,0xb4,0x54,0xc5,0xfc,0x97,0x28,0x2f,0x80,0x0c,0x44,0x86, -0x1d,0xfd,0xd7,0x29,0x08,0x08,0xe8,0xb9,0x91,0xe7,0x66,0xfa, -0xdd,0x5a,0xad,0xed,0x26,0x09,0x1e,0x19,0xb8,0x0c,0xfc,0x46, -0x62,0xd0,0xe4,0xc0,0x92,0x5c,0xa6,0x69,0x92,0x72,0x95,0x41, -0xa3,0xd4,0xe5,0x39,0x0b,0xb8,0x6b,0x7e,0x37,0xd3,0xd5,0x05, -0x53,0xea,0x05,0xc2,0x4a,0xe6,0x7b,0x02,0x8d,0xd0,0x6f,0x9b, -0x44,0xc1,0xa1,0x90,0x65,0x15,0x20,0x07,0x8b,0xc1,0xf7,0x49, -0x3c,0x86,0x4e,0x2d,0x26,0xdd,0xdd,0xf5,0xfa,0x16,0xe9,0xdf, -0x59,0xc0,0x36,0xf9,0x56,0x7a,0xbe,0x09,0x36,0x7e,0x17,0xd2, -0xa5,0x43,0x0e,0xb5,0x1f,0xa0,0x1b,0x5d,0x6a,0x0a,0x19,0x77, -0x77,0x30,0x5a,0x0d,0x4d,0xdf,0x16,0x0a,0x6a,0xca,0x48,0xb8, -0x0b,0x14,0x15,0x6b,0xba,0x91,0xc1,0x75,0x6d,0x7a,0x6e,0x93, -0xe5,0x15,0x82,0x48,0xf4,0xa9,0xa3,0xe3,0x2c,0xe1,0x95,0xd8, -0x52,0x4c,0xae,0x6c,0x8b,0x11,0x9f,0x78,0x3a,0x9c,0x8d,0xe2, -0x17,0xa3,0xb4,0x5c,0x80,0x74,0xb6,0xdf,0x3d,0x08,0xda,0xf9, -0x44,0xbb,0x27,0x34,0x1e,0x4d,0x17,0x83,0xf9,0x9f,0x0a,0x35, -0xf1,0xaf,0x3c,0xa7,0x1d,0xa5,0x16,0x6d,0xc9,0x03,0x7e,0xc3, -0x58,0x6f,0x03,0xd3,0xe7,0xb3,0x94,0xff,0xae,0xe8,0x0f,0xc7, -0xdb,0xc1,0x5f,0xc3,0xd9,0x04,0xd6,0xa0,0xf0,0xab,0xa4,0x63, -0xdc,0x7a,0x34,0xb5,0xa4,0x02,0x43,0x41,0x7f,0x64,0x88,0xf5, -0x89,0xd2,0x2d,0x7d,0x21,0xf5,0x33,0xe7,0xde,0xdd,0x31,0xb0, -0x99,0x04,0x6c,0xce,0x49,0x0c,0x5d,0x59,0x9a,0x23,0x4b,0xd7, -0x98,0xa2,0x9a,0xdf,0x46,0x60,0xbe,0x62,0xc6,0x48,0x34,0x20, -0x23,0x0f,0xdb,0x4e,0x39,0x09,0x23,0xe4,0x3c,0x23,0xf3,0xcc, -0xe4,0x22,0xf5,0xf1,0xa7,0x6d,0xf3,0x03,0xc3,0xf4,0x19,0x7f, -0xba,0xf1,0xa5,0x5c,0x49,0xc6,0x2c,0xf6,0x65,0x7b,0x5e,0x98, -0x40,0xd1,0xe2,0x19,0x54,0x69,0x6a,0xc5,0x90,0x90,0x45,0x34, -0xf5,0x57,0x9c,0x50,0x35,0x20,0x24,0xb8,0x84,0x52,0xf8,0xf6, -0xa9,0xdf,0x6d,0xe2,0x5e,0x25,0x64,0x2a,0x33,0xeb,0xe7,0x9b, -0x78,0xb1,0x48,0x6c,0xe8,0x7c,0x7a,0x4e,0x11,0xbd,0xcc,0x17, -0x2b,0x8c,0x79,0x14,0xa8,0x99,0xc9,0xd7,0x13,0x5e,0x4f,0x3e, -0x0f,0x63,0x4c,0x62,0x3d,0x57,0x28,0x74,0x53,0xc6,0xee,0xd5, -0x78,0xce,0xae,0x09,0x83,0x07,0x26,0x72,0xf3,0x21,0x45,0x8d, -0x87,0x01,0x18,0xf5,0x55,0x4d,0x32,0x25,0x5c,0x15,0x53,0xa8, -0x31,0xca,0xc9,0x99,0xb0,0x1b,0x93,0xc6,0xe8,0x9d,0x2a,0x75, -0xd5,0xdd,0x5d,0x45,0x65,0xb2,0x51,0x07,0xee,0x18,0x39,0x6f, -0x5e,0xe2,0x4d,0x14,0x46,0x61,0xa4,0x19,0x77,0x75,0x95,0x2c, -0xc0,0x6f,0x77,0xa6,0x8d,0xa2,0xa8,0xcc,0xc6,0xa1,0xa7,0x16, -0x64,0xe5,0x60,0x31,0x3c,0xc7,0x08,0xf2,0xb3,0x45,0x54,0x48, -0xb9,0xbb,0xe3,0xe8,0xd6,0x56,0xa7,0xfd,0xd5,0x80,0x9b,0xc9, -0x83,0x1e,0xfe,0x49,0x2e,0x3d,0xa0,0xf8,0x6a,0xa0,0x4c,0x3a, -0x0b,0xcf,0xf7,0x03,0x7c,0x27,0x92,0x9f,0xf8,0xf1,0xfd,0xe5, -0xd5,0x62,0x76,0x4b,0x7b,0xd8,0xdf,0x2d,0x16,0x33,0xb0,0x59, -0xff,0xfa,0xcb,0x8f,0x0f,0x60,0xbe,0xbe,0x7a,0xc0,0xd7,0x69, -0xe3,0xd1,0x83,0x24,0x9d,0xfe,0xff,0x97,0x68,0x67,0x25,0xb0, -0xfa,0xcc,0xf5,0xb1,0x45,0x30,0xbf,0xcb,0x63,0x77,0x87,0x65, -0x20,0xbb,0x33,0x50,0x21,0x03,0x86,0x5b,0x32,0x6a,0xd7,0xa0, -0x02,0xb1,0xc0,0xa8,0x7d,0xed,0x1a,0xfd,0xa9,0x09,0x7e,0xe6, -0x45,0x59,0xb8,0xed,0x9a,0xfb,0x5d,0x13,0x16,0xa6,0xe7,0xb3, -0xb1,0x7a,0x8b,0xb6,0x56,0x9a,0x5c,0x83,0x95,0xc2,0x74,0x3a, -0x9b,0xc2,0xaa,0x64,0xfc,0x02,0x6a,0x3b,0x3f,0xb7,0xbf,0x6b, -0x22,0x5e,0x0e,0xde,0x3b,0x45,0xbf,0x83,0x04,0xbc,0xe8,0x33, -0xbe,0xfc,0x31,0x99,0x5e,0xb7,0x6b,0x57,0x8b,0xf8,0xb2,0x86, -0xb7,0x24,0xda,0x6c,0xab,0xc1,0x3f,0x3f,0x26,0x29,0x7f,0xe1, -0x8f,0x1a,0xe9,0xf1,0x1a,0xfc,0x53,0x63,0xcd,0x5e,0xc3,0x7f, -0x6b,0x4a,0xad,0x2b,0xb9,0x16,0xa8,0x0f,0xda,0xd2,0xd0,0x23, -0xeb,0xae,0x2d,0x8d,0x3c,0x21,0xad,0xbe,0xb6,0x31,0xff,0xe8, -0x0a,0x39,0xb6,0x7b,0x89,0x8e,0xd4,0x50,0x91,0x65,0xf8,0x43, -0xf7,0x53,0xdd,0x6e,0x12,0x51,0xf1,0xdd,0x38,0x46,0x77,0x76, -0x22,0x46,0xfe,0xae,0x89,0xc1,0x68,0x64,0x81,0xf2,0x08,0x69, -0xd7,0xca,0x52,0xa9,0xa6,0x57,0xc4,0x6b,0xdd,0x42,0xf3,0x09, -0x8c,0x1c,0xcf,0xf0,0x42,0x14,0xfd,0xa9,0xd9,0x8a,0xf2,0x97, -0x42,0x97,0x5b,0x0b,0xb0,0xb4,0xdc,0xa0,0x29,0x8a,0x89,0xd6, -0x2d,0x55,0x10,0xc2,0x20,0x73,0x45,0x0e,0x9d,0x88,0x31,0xfd, -0xcd,0x0c,0x5b,0x6d,0x3f,0x09,0x0a,0x89,0x02,0x24,0x7a,0x22, -0xac,0x97,0x5d,0xe6,0xfc,0x40,0x53,0x82,0x11,0x3f,0x11,0xa0, -0x99,0xc2,0x9a,0x15,0xf7,0x09,0x41,0xf7,0x24,0x23,0x0a,0xe4, -0x19,0x61,0x99,0xc2,0x62,0x09,0xd7,0x68,0xc9,0x60,0xec,0x19, -0x04,0xd6,0x7a,0x09,0x13,0x61,0x04,0x72,0xf9,0x1e,0x7e,0xf5, -0xf1,0x9e,0x96,0x42,0xe8,0x64,0x6c,0x74,0x45,0xbd,0xf3,0xe6, -0x78,0x00,0xe6,0xa8,0x85,0xb2,0xcf,0xba,0xc9,0x6d,0xe1,0x7b, -0xd9,0xc2,0xef,0x17,0xb3,0x49,0x55,0x1b,0xf7,0x6f,0x9d,0x5a, -0xd3,0xd2,0xb3,0x0b,0x54,0x34,0xd7,0x56,0x83,0xa4,0xa2,0x81, -0xf8,0xde,0x59,0x59,0xcb,0x7c,0xd3,0x34,0xbb,0x77,0x5e,0x14, -0xba,0x86,0x3a,0x25,0x19,0xe5,0xa5,0xc2,0xed,0xc9,0x4a,0x49, -0xc0,0x37,0x42,0x14,0x86,0x02,0xa3,0x5e,0x94,0x70,0x89,0xf9, -0xe3,0xd6,0x95,0xe7,0xe9,0xce,0xda,0x7c,0x6b,0xca,0xf9,0x15, -0x15,0xe3,0x8b,0x91,0x5b,0xc3,0x0b,0x13,0x3f,0xae,0x76,0x10, -0x45,0x9c,0x74,0x78,0x98,0x9b,0x45,0x24,0xa4,0x4c,0xa7,0x47, -0xef,0x64,0x92,0xcd,0x32,0x18,0xd3,0xa5,0x3c,0x5b,0x15,0x97, -0x37,0xfb,0x32,0x4d,0xe9,0x34,0xa1,0x11,0x15,0x78,0x07,0xb5, -0x7e,0x69,0xe6,0x39,0xb5,0x5a,0x93,0x5e,0x1a,0x3f,0xb7,0x14, -0x71,0xb9,0x1d,0x9f,0x03,0x02,0x6b,0x3e,0x6c,0x97,0x67,0xd9, -0xf3,0x63,0x0e,0xb3,0x6b,0xd3,0xe4,0x1b,0x92,0xa7,0xa3,0x7c, -0xf0,0xd9,0x20,0x55,0x0c,0xa2,0x88,0x42,0x36,0x2a,0x4c,0xee, -0x55,0xf2,0xc5,0x06,0xed,0x0b,0x98,0x9e,0x57,0x54,0x28,0x2f, -0x61,0xde,0xd0,0x61,0x80,0x33,0x99,0x95,0xc9,0xbc,0x3b,0x9e, -0x15,0x5a,0x7b,0x76,0xa6,0x8d,0x2f,0x5a,0x40,0x0e,0x2d,0x93, -0xa3,0x90,0x71,0x77,0x57,0xbe,0xf3,0x41,0xdb,0x1a,0xae,0xf2, -0xcf,0x61,0x34,0x7c,0x51,0x49,0x25,0xca,0x3f,0x57,0x46,0x83, -0xba,0x36,0xb0,0x85,0xff,0x05,0x1e,0x30,0x0f,0xe3,0x39,0x42, -0x6f,0xa9,0x5d,0x35,0xf4,0x12,0xc6,0x95,0x5c,0x08,0xd8,0x25, -0xb7,0xe6,0x5a,0x4d,0x8e,0x79,0xfa,0x34,0xfb,0x27,0x14,0x4d, -0x9d,0xf6,0xd7,0x94,0x61,0x2f,0xf7,0xa4,0x64,0xd3,0x3e,0xc9, -0x12,0x6d,0x55,0x52,0x82,0xb5,0x5d,0xf0,0xb6,0x5c,0xce,0xb8, -0x2d,0x2c,0x90,0xa9,0xe9,0x48,0x4c,0x62,0x9e,0x89,0x15,0x2d, -0x74,0xd5,0xc5,0xfe,0xf4,0x6f,0xa5,0x57,0xa8,0xd6,0x59,0xf4, -0x54,0x55,0x80,0xe7,0x3b,0xf9,0x4e,0xfc,0x7e,0x07,0xad,0xf7, -0x6f,0xa6,0x55,0x22,0xb7,0x13,0x83,0x16,0x6d,0x49,0x97,0xe7, -0xd3,0xad,0xce,0xf6,0x9d,0x65,0xdf,0x77,0x79,0x04,0xfb,0x50, -0x59,0xa8,0xb5,0x92,0xbe,0xd9,0xf4,0x9b,0xf8,0x72,0xb6,0x88, -0x71,0xb0,0x5a,0x86,0x53,0xf2,0x31,0xd6,0x83,0x7f,0x0b,0x48, -0x4e,0x44,0x35,0x19,0xf8,0x61,0x37,0xe2,0xe7,0x2d,0xb5,0x98, -0x29,0x7d,0x66,0x6d,0x8d,0xef,0x41,0x1e,0xc2,0xe7,0x1b,0xb3, -0xb5,0x19,0x7f,0xaa,0x01,0x9f,0x41,0xfa,0x0e,0xa2,0x0d,0x76, -0x34,0x68,0xf1,0xa4,0xe0,0x05,0x6f,0x6a,0x44,0x8e,0x28,0xa8, -0x69,0xf5,0x6a,0xb6,0x1a,0x8f,0x5e,0x0f,0x6e,0x62,0x36,0x82, -0xa3,0x8a,0x74,0x47,0x90,0xe4,0xba,0x31,0xb0,0x2b,0x63,0xb0, -0x71,0x9c,0x56,0xef,0xa9,0x99,0xed,0x15,0x99,0xd1,0xad,0x22, -0xc2,0x58,0xd6,0xb8,0x56,0xd2,0x15,0x5a,0xdb,0xbf,0x72,0x1b, -0x8e,0xf3,0x37,0xb4,0x73,0xf3,0xcd,0x6c,0x36,0x8e,0x07,0xba, -0x56,0x0c,0xcf,0xb5,0x0b,0xbd,0xc6,0xcc,0x65,0x36,0x79,0x93, -0x7b,0x75,0x79,0x99,0xac,0xcb,0x67,0x6d,0xce,0x93,0xfe,0x80, -0x4e,0x5a,0x51,0xe9,0xbd,0x66,0x3c,0xf6,0x2e,0x28,0x27,0x15, -0x8c,0x1c,0x06,0xd4,0xf9,0x2e,0x39,0x3a,0x98,0xaf,0xb5,0xf7, -0x61,0xa7,0xd9,0x3b,0x3b,0xb0,0xf4,0x91,0x8b,0xa3,0x37,0x06, -0x22,0xf7,0xb4,0x21,0x1f,0x1c,0x3e,0x5b,0xbc,0xc7,0x6a,0x16, -0xf8,0x43,0x3e,0x5f,0x98,0x9a,0x95,0x25,0x87,0x4d,0xb2,0x6a, -0xe1,0xb7,0x17,0x5a,0xb0,0xe2,0x2f,0x44,0x98,0xb1,0xa1,0x34, -0xee,0xb6,0x67,0x27,0xab,0x8a,0x84,0x53,0x87,0x06,0xe6,0x18, -0xd1,0x48,0xdc,0x9b,0x1d,0x6d,0xed,0x21,0x50,0xbf,0x93,0x07, -0xc5,0x57,0x00,0xf7,0x2e,0x8d,0x6f,0xe5,0x89,0x3c,0x38,0x6f, -0xd4,0x68,0x46,0x0d,0x67,0x18,0xd7,0xd2,0x1e,0xc3,0xda,0x9e, -0xb5,0x9b,0xe0,0xe3,0x26,0x0b,0xbe,0xb1,0x66,0x27,0xda,0x0c, -0x61,0x3d,0x40,0xf3,0x26,0x86,0x4d,0x92,0xbd,0xc4,0x6b,0x41, -0x8b,0x13,0xae,0xe0,0x8b,0x4b,0x73,0xf4,0xb1,0xad,0x5f,0x25, -0xf4,0x41,0x80,0x05,0x72,0xc8,0x5f,0xce,0x46,0xf1,0xf8,0x4f, -0xe2,0x0d,0x19,0x6f,0x9e,0x91,0x65,0x8b,0x1d,0xab,0xf9,0x3c, -0x38,0x85,0xa5,0xaa,0xa4,0x07,0xe2,0x3e,0x1d,0x44,0x65,0xfb, -0x42,0x06,0x48,0xa4,0x9a,0xec,0xe5,0x6a,0x1e,0xc7,0xe1,0xa1, -0x5c,0xd4,0xe5,0x33,0xac,0x88,0x5f,0x26,0xd5,0xff,0x64,0xe3, -0xb5,0x7b,0xc9,0xa1,0xdf,0x02,0xc2,0x25,0xdf,0x36,0x05,0xec, -0x59,0xb0,0xbb,0x98,0xb0,0x71,0xad,0x37,0x36,0xed,0xca,0x95, -0x8c,0x0d,0x61,0xaf,0x0b,0xec,0xf4,0x82,0x01,0x22,0x11,0x9a, -0xc5,0x33,0x46,0xb0,0x29,0xb5,0x37,0x34,0xe4,0x38,0x77,0xd0, -0x53,0xd8,0x52,0xc3,0x63,0x55,0xec,0x3b,0x3c,0x16,0x2a,0xec, -0xcb,0xc1,0x74,0xd2,0xb1,0x32,0x9a,0x4e,0x40,0x02,0x3e,0xfe, -0x8d,0xac,0x01,0xc5,0xc7,0xbd,0x76,0x5c,0x0d,0x4e,0x93,0x72, -0x20,0x29,0x29,0x47,0xc8,0x9b,0x64,0xf9,0xf9,0x6c,0xb8,0x5a, -0x60,0xb2,0x79,0x85,0x53,0x82,0xc1,0x00,0xef,0xc8,0xbc,0x8e, -0x2f,0xd3,0x78,0x7c,0xab,0x12,0x42,0xfe,0x88,0xe4,0xdf,0x9e, -0x45,0x4f,0xc9,0x72,0xc6,0xdd,0xa3,0xd3,0xb1,0x07,0x15,0x6a, -0x0c,0x6e,0xbd,0xa0,0x8d,0xcf,0x0a,0xf2,0xf5,0x29,0x75,0x9e, -0x74,0x79,0xb2,0x8d,0xe7,0xfd,0x7a,0xd3,0x1b,0x7e,0xcb,0xdd, -0xfd,0xdb,0xab,0xc1,0x92,0xf6,0xf0,0x84,0x75,0x24,0x2a,0xdb, -0xc9,0x47,0xd3,0xfa,0x64,0xc9,0x94,0xa2,0xd9,0xc0,0x6e,0x4e, -0xd9,0xc1,0x41,0x93,0xcf,0x3e,0x7c,0x65,0x2f,0xe7,0x51,0xd9, -0x67,0x06,0x25,0xb8,0x64,0xb6,0xdf,0x41,0xe2,0xcc,0x84,0x2b, -0x8f,0xa5,0x0d,0xb1,0x5d,0x89,0xbe,0x57,0x7b,0x71,0xd9,0xf8, -0x69,0x36,0x8d,0x1b,0x2f,0x69,0xd7,0xaf,0x1f,0xd1,0xce,0xa4, -0x9d,0x6b,0x67,0xf0,0x74,0x00,0xc2,0xc8,0x91,0x14,0xf1,0x84, -0x53,0xf7,0x9c,0xe2,0x3d,0xe4,0xd6,0x23,0x64,0x50,0xb7,0x76, -0x5c,0xab,0xe3,0x0f,0xbc,0xc2,0x48,0xa9,0x36,0xc1,0x3c,0xb3, -0xde,0xdd,0x41,0x5e,0x99,0x7f,0x81,0x05,0x6a,0x39,0x19,0x7c, -0x52,0xdb,0xa0,0x36,0xa6,0xfc,0x61,0x9f,0xf7,0x69,0x22,0xb7, -0x1b,0xff,0xf5,0xbb,0x37,0xb5,0xdc,0x8a,0x22,0xb7,0xde,0x00, -0x2d,0x82,0x7b,0xac,0x5f,0x10,0x1d,0x6e,0xcb,0xee,0x87,0xee, -0xd5,0x5f,0xf7,0x41,0x37,0xdb,0xbb,0xb5,0xaf,0x7e,0x7e,0xbd, -0x07,0x42,0xb9,0x41,0xbc,0x1f,0xca,0x6f,0xbf,0xfb,0xf1,0xbb, -0x37,0xdf,0xed,0x46,0x4a,0xbb,0xcc,0xfb,0xa1,0xfc,0xe1,0xbb, -0x67,0xdf,0xee,0x46,0xc8,0x5b,0xd5,0xfb,0x61,0x7c,0xf3,0xcb, -0xb3,0xe7,0x7b,0xd0,0xa8,0xf6,0xbb,0xf7,0x43,0xfa,0xf3,0xab, -0x37,0x2f,0x7e,0xfe,0xe9,0xf5,0x1e,0x1d,0x44,0x9b,0xe6,0x7b, -0xf6,0xd0,0xb3,0x37,0xcf,0x7f,0xd8,0x89,0x72,0x23,0xdd,0x24, -0xf0,0x64,0xc4,0xd5,0xe9,0x98,0x62,0x85,0xc3,0xc2,0x93,0x14, -0x4b,0xad,0x09,0x37,0x9b,0x46,0x67,0x89,0x4a,0x46,0xd4,0xe7, -0xe7,0x48,0x30,0x8d,0x47,0xb3,0x07,0xbb,0x88,0x47,0x2b,0x35, -0xf2,0x6c,0xb5,0x6e,0x74,0xb3,0x46,0x36,0x18,0xae,0x26,0xd6, -0xee,0x2f,0xfe,0x42,0xdf,0x04,0xfc,0xfb,0x5a,0x6e,0xbc,0x45, -0x5c,0x47,0xb3,0x74,0x97,0x4f,0x6e,0x17,0x25,0x97,0x9e,0x5d, -0x84,0x8e,0x92,0xdc,0x62,0x8e,0x0f,0x88,0x0b,0xec,0x5b,0x4b, -0x38,0x95,0xd8,0x91,0x94,0x44,0x76,0xe2,0x46,0x85,0x35,0xb6, -0x33,0x7b,0x6e,0x3d,0xc5,0x99,0x45,0xda,0x3b,0x07,0xfb,0x01, -0x97,0x9d,0x01,0xf5,0x0d,0x73,0x5e,0x8c,0x3a,0xfc,0x27,0xc7, -0x95,0xdc,0x36,0x60,0xb7,0xc0,0xb3,0xb2,0x1d,0x40,0x69,0xb9, -0x16,0x60,0xf3,0x10,0x85,0x6d,0x63,0xb3,0x07,0xac,0x58,0x51, -0x8f,0x50,0x47,0xe7,0x78,0xae,0x7d,0x0f,0xba,0xfc,0xeb,0xaf, -0xbf,0xbc,0x78,0x3e,0x9b,0x80,0x65,0x02,0x32,0xa0,0x50,0xb4, -0xe5,0x5f,0xfd,0x00,0x34,0x4a,0x44,0x1d,0x91,0x49,0xcc,0xee, -0xec,0xa4,0x5c,0x6b,0x36,0x79,0x19,0xbd,0x8c,0x61,0xf4,0x38, -0xfe,0x34,0x6a,0xfe,0xc5,0x89,0x83,0x19,0xa8,0xfc,0x68,0xaa, -0xa7,0x1c,0x84,0x85,0x46,0x49,0x40,0x6e,0x14,0x95,0x17,0x32, -0xa9,0xda,0x8e,0x6a,0xd2,0x63,0xc0,0x48,0x15,0x0f,0xba,0xf7, -0xe3,0xd9,0xc5,0x60,0xcc,0x36,0x92,0x7a,0xa5,0xbe,0xe8,0x33, -0xc7,0xe1,0x89,0x4b,0x60,0x79,0xbd,0xda,0xfc,0x0b,0x46,0x70, -0xef,0xd5,0x68,0xe2,0xaa,0x89,0xda,0x5f,0x7e,0xb7,0x5c,0xf2, -0x78,0x36,0xfb,0xcb,0xef,0x96,0x83,0x1d,0xbf,0xe4,0xfd,0x3a, -0x5e,0xdc,0x24,0x43,0xf4,0x05,0x75,0x50,0x6a,0x03,0x4d,0x83, -0x2f,0x1c,0x53,0x17,0x35,0x80,0xc7,0xe3,0x95,0xb7,0xd8,0xe5, -0x12,0x5d,0x0a,0x7c,0xf5,0x56,0x30,0x4b,0x78,0xc4,0xf6,0xf0, -0x76,0xd0,0xc2,0xc1,0x5f,0x1f,0xec,0x12,0x8a,0x8f,0x08,0x0c, -0xfd,0x61,0x30,0x1d,0x8d,0x81,0x31,0xaa,0x2f,0x85,0x49,0x63, -0x1d,0xb1,0x03,0x79,0xd9,0xa1,0xa0,0xc6,0x5f,0x96,0xf9,0xbd, -0x64,0xc4,0x5e,0xd8,0xe9,0xdc,0x50,0xa3,0xc3,0xb3,0xa7,0xdc, -0x4a,0x41,0xae,0xb2,0x98,0x77,0x06,0x63,0x13,0xaf,0x64,0xc9, -0x52,0x78,0x15,0x34,0x9f,0x0d,0xb6,0x98,0xa6,0x51,0xae,0x48, -0x73,0xa5,0xd9,0xd1,0x4c,0x21,0x50,0xfa,0x31,0x87,0xc3,0x02, -0xe2,0x2f,0x1b,0x88,0x7b,0xf5,0xf0,0xb0,0x62,0xdb,0x44,0x6e, -0xc6,0x68,0xcb,0x13,0x92,0xcc,0x56,0xbe,0xab,0x10,0x38,0x57, -0x61,0xc4,0x4a,0xcb,0xf5,0xb2,0x82,0x2b,0xf0,0x11,0xa4,0x52, -0xee,0xd5,0x9c,0x37,0x57,0x53,0x14,0x4b,0x19,0x7b,0x2f,0x85, -0x1f,0xf3,0x64,0x78,0x5d,0x7d,0x7e,0xda,0x53,0x07,0xec,0xf9, -0x13,0x75,0x8c,0xd4,0xaa,0x74,0x8e,0xe5,0x83,0x96,0x3b,0x88, -0x8f,0x24,0x7e,0x29,0xe2,0xd5,0x94,0xf9,0x9d,0x82,0xc2,0x53, -0x4c,0xf1,0x0d,0x0f,0xcd,0xb1,0xa0,0xe7,0xd6,0x23,0x34,0x70, -0x51,0x71,0x6a,0xb6,0x39,0x88,0xf4,0x61,0x59,0x39,0x26,0x04, -0xdf,0x21,0x9c,0xb9,0xe5,0x4b,0xe4,0x7e,0xf3,0xc4,0x75,0x3f, -0x0c,0xf6,0x43,0x9c,0xb4,0xb7,0xa1,0xb5,0x06,0xc8,0xa0,0xe6, -0x21,0x0d,0xf9,0x64,0x64,0x26,0x28,0xcb,0x4b,0xb7,0x84,0x51, -0xfa,0xd0,0xd3,0xd5,0x3c,0x56,0xd3,0x6d,0xcd,0x63,0xaa,0xc4, -0x67,0x4c,0xed,0x2a,0x2d,0x87,0x68,0x1b,0x8f,0x99,0x32,0x15, -0x34,0xfa,0xa7,0xc9,0x1d,0x14,0xdf,0x69,0x01,0x5a,0x01,0x4e, -0x23,0xe8,0x60,0x72,0x6b,0x2b,0x4c,0x2f,0x99,0x03,0xc8,0xfb, -0xb7,0xe3,0x3f,0xa4,0xe2,0x62,0xc3,0x4d,0xf5,0x6e,0xd3,0xdd, -0x72,0xaf,0x60,0xb8,0x26,0xa8,0xf5,0xf9,0xaf,0xb3,0xbf,0xa1, -0x79,0x2c,0x33,0xe9,0x31,0x59,0xa5,0x62,0x64,0xda,0x73,0xd4, -0x58,0xf2,0xb7,0xaf,0x7e,0xa0,0x2a,0xc8,0xc1,0xfd,0x6b,0xbc, -0x2c,0x80,0x6d,0x99,0x4f,0x4a,0x8d,0x20,0x67,0x03,0xce,0xa1, -0x14,0x5f,0xd3,0x91,0x55,0xe3,0x16,0xc0,0x16,0x1a,0x41,0x73, -0x60,0x24,0x74,0x5f,0x13,0x61,0x18,0x63,0x41,0x6b,0xff,0x3e, -0xba,0x61,0x14,0x5f,0xc6,0x60,0x24,0x8c,0xa2,0xbf,0xfc,0xde, -0xa4,0xdf,0x9e,0x4f,0x0b,0xf8,0x67,0x8b,0xf7,0xa9,0xd9,0x28, -0xed,0xd8,0x9b,0x1e,0xcb,0xab,0x78,0xea,0xd9,0x5b,0xe5,0xce, -0x92,0x9e,0x4c,0x61,0x7b,0x27,0x05,0xdf,0x96,0xe0,0x97,0x7a, -0x15,0x5e,0x8c,0xb4,0x87,0xe5,0xf1,0x34,0x57,0x39,0x12,0x76, -0x30,0x41,0xc6,0x7f,0x84,0x64,0x75,0xfc,0x25,0x14,0x7d,0xb4, -0x03,0x31,0xbe,0x21,0x97,0x71,0x7f,0x93,0x17,0x10,0xd5,0xd1, -0x1a,0x5a,0x31,0x04,0x09,0xbe,0x6f,0x4f,0xf8,0x05,0xae,0x41, -0xff,0x7a,0xc6,0xce,0x2a,0x61,0xd9,0x2e,0xfe,0x94,0xb5,0xa2, -0x87,0x18,0xfb,0xff,0x79,0x6d,0x91,0x55,0xe7,0xd1,0x0b,0x75, -0xda,0x2b,0x9c,0x73,0xe4,0x92,0xed,0xb6,0x6e,0xa1,0x15,0xda, -0x27,0x48,0xe3,0xf8,0x64,0xfb,0xd6,0xb7,0xbd,0x42,0x09,0xaa, -0x43,0xf0,0x73,0x4a,0x16,0x6d,0xe8,0xd5,0x3d,0xb7,0xec,0x0d, -0x2f,0xb7,0x71,0x0d,0x24,0xb3,0x49,0x2d,0xac,0xd9,0x94,0x12, -0x2b,0x67,0x53,0x74,0x5c,0xf3,0xed,0xa7,0xbc,0xc0,0x42,0x7a, -0xbe,0x4a,0x97,0xb3,0xc9,0xcf,0x6a,0x77,0x47,0xd6,0x42,0x26, -0xc6,0xd0,0xcd,0xd2,0x76,0x10,0x25,0xe7,0x0c,0x29,0xb9,0x63, -0xdb,0x63,0xaf,0x32,0xe9,0x50,0x26,0xaf,0x0a,0x48,0x6f,0xca, -0xbe,0x70,0x2e,0x03,0xf8,0x9f,0x74,0x99,0xd1,0x0c,0x1f,0x20, -0x25,0xb4,0x36,0x14,0x4c,0xa0,0x83,0x54,0x79,0x29,0x8c,0xc7, -0xba,0x46,0xfc,0x30,0x54,0x29,0xf4,0x51,0x14,0x21,0xda,0xae, -0x72,0x5f,0x6b,0xe3,0x17,0x79,0x0f,0x46,0x84,0xa8,0x8e,0xdf, -0xcd,0xe5,0xec,0xaf,0xf3,0xb9,0x72,0x69,0xec,0xd8,0x68,0x23, -0x22,0xfd,0x20,0x8a,0x1c,0xf4,0x87,0x87,0xdc,0x96,0x7c,0x7a, -0xd7,0xe5,0x42,0xbb,0x78,0xcf,0x81,0xf5,0x31,0x1d,0x31,0xbb, -0xab,0x75,0xab,0x0f,0xcb,0xf8,0x49,0xef,0x82,0x78,0xb9,0x43, -0x6a,0x17,0x83,0x72,0x4c,0xa1,0xe9,0x1d,0x5b,0x68,0x6c,0x54, -0x9b,0x4f,0x44,0x80,0x43,0x35,0xee,0x0f,0x48,0xfb,0x91,0x6b, -0xfd,0xd7,0xef,0xde,0xfc,0xf8,0xe2,0xf5,0x1b,0x55,0x9c,0xcc, -0xf0,0x12,0x13,0xb9,0x39,0x9a,0x29,0xc8,0x62,0x61,0x23,0x4e, -0xd5,0x16,0xb2,0x77,0x29,0xad,0x44,0xbd,0x8d,0x00,0xb0,0x89, -0xce,0x76,0xfd,0xbb,0x6c,0xd8,0x4e,0xe9,0xac,0x2f,0xfd,0x77, -0x2c,0x0c,0x7b,0x5b,0x39,0xc2,0x29,0x26,0xf6,0x44,0xc2,0xeb, -0x9d,0xca,0x49,0x35,0x6f,0x05,0x98,0xa9,0x7e,0x68,0x4d,0xae, -0x7a,0xc6,0xdf,0x7d,0x70,0xad,0xbd,0x39,0x0e,0x42,0x69,0x51, -0x88,0x31,0xde,0x2d,0xa5,0x33,0x8e,0xbd,0xd6,0x70,0x79,0x07, -0x58,0x77,0xb7,0x00,0xe6,0x4f,0x8d,0x6f,0x4f,0x87,0x9f,0xf2, -0xc5,0x81,0xc6,0x42,0xf6,0xb8,0x35,0x3d,0xef,0x81,0xde,0x71, -0xd6,0xec,0x97,0xdc,0x30,0x34,0x08,0x37,0x56,0xfb,0xef,0x6d, -0x45,0x00,0x0f,0xf7,0xa3,0x46,0x0f,0x23,0x8a,0x20,0x25,0x47, -0x81,0x69,0xe1,0x9e,0x58,0xb8,0x35,0xa5,0x23,0xea,0x7e,0xb8, -0x40,0x1f,0x69,0x3c,0xf0,0xfb,0x73,0xb1,0xcc,0x2c,0x72,0xf0, -0xe3,0x33,0xf1,0xb0,0x5a,0xd5,0x98,0x58,0xeb,0x7e,0x26,0x2e, -0xd4,0x63,0x1a,0x13,0x7e,0x7c,0x26,0x1e,0xda,0xea,0xd5,0x88, -0xe8,0xeb,0x33,0x31,0xc9,0x1d,0x5e,0x8d,0x4b,0x7e,0x7f,0x2e, -0xcf,0x71,0x63,0xd7,0x30,0x7d,0x50,0x21,0x03,0x25,0xd3,0xaf, -0x95,0x5b,0x76,0x42,0x6b,0xf2,0xb5,0x66,0x10,0x29,0xef,0xfd, -0x54,0x2a,0xa7,0xf2,0x95,0x82,0xb5,0xb0,0xfa,0x2c,0xbd,0x14, -0xfc,0x49,0xbd,0xf4,0x27,0x86,0x73,0xf0,0x85,0xc4,0x5e,0x3e, -0xca,0xf6,0x7f,0xa5,0xbc,0x1a,0x7f,0x79,0x5b,0x45,0xc8,0xa4, -0xff,0x9b,0xc6,0xc0,0xfd,0x35,0xf3,0x37,0xd9,0x8b,0xd1,0x7f, -0x8c,0x56,0x96,0x02,0xf1,0xe7,0x86,0x65,0x50,0x39,0x2c,0x25, -0xed,0x5e,0x82,0x0b,0x8c,0xdf,0x5f,0x95,0x1b,0x85,0xbc,0xf5, -0xad,0xec,0x2b,0x80,0xd5,0x61,0xc6,0xe8,0xa9,0xcd,0x7c,0x31, -0x6b,0x5d,0x56,0xe8,0x7e,0x28,0xbc,0x2e,0xd8,0x9e,0xb9,0x23, -0x1c,0x1a,0x9a,0x6f,0x66,0xaf,0x56,0xfc,0x8c,0x68,0x0f,0x8a, -0xf4,0xc5,0x96,0xd5,0x9c,0x06,0x47,0x09,0xf4,0x72,0xc8,0x73, -0xcb,0x3c,0xe4,0x40,0xbc,0x20,0xae,0x51,0xb5,0xd3,0xf8,0x96, -0xcf,0xfa,0xb7,0xd8,0x89,0x4c,0x9a,0xdf,0x51,0xb0,0x44,0x50, -0x64,0x30,0x15,0x17,0xbf,0x0a,0xd2,0x0e,0xa4,0xa4,0xd6,0x5e, -0xce,0x2a,0x13,0x1d,0x2b,0x4c,0xce,0xfe,0x6b,0x4c,0x67,0x33, -0x85,0xa2,0xce,0xa9,0xd5,0x1f,0x62,0xfb,0x16,0x56,0x6f,0x96, -0x4b,0xa9,0x74,0xc9,0x51,0xdb,0xd9,0x66,0x21,0x29,0xb1,0xea, -0x00,0x75,0x83,0x4a,0x37,0xe6,0xcf,0xc1,0x4a,0x0e,0x05,0x91, -0x4a,0x6f,0xca,0x0e,0xf1,0x6a,0xdf,0xbd,0x19,0xbc,0xaf,0x59, -0xe1,0xeb,0x97,0x83,0xc3,0x43,0x79,0xbc,0x8f,0x1f,0xd5,0x23, -0x06,0x81,0xe4,0xb9,0x3d,0x2d,0x82,0x0d,0x07,0x9c,0xb1,0x43, -0x5b,0x00,0x65,0xd2,0x9c,0x17,0xb3,0xb2,0x3d,0x95,0x99,0x59, -0xc3,0xe9,0x9b,0x49,0xe8,0x1f,0x50,0x72,0x2e,0xc0,0x87,0x26, -0x7c,0x2e,0x83,0xff,0x7e,0x8f,0xe9,0x80,0x0d,0x7f,0xdf,0xdd, -0x71,0x45,0xbb,0xac,0x77,0x21,0x6f,0x60,0x6e,0xf7,0xa8,0xf6, -0xf8,0x81,0x76,0xcd,0x7a,0xab,0x36,0xdb,0x17,0x99,0x9c,0x28, -0x74,0xbb,0xf1,0xcb,0x76,0x62,0x50,0xe4,0x2f,0x6c,0x97,0x04, -0xe9,0x8b,0xa0,0x02,0x3a,0xa8,0xb5,0x9c,0xfa,0x9e,0x5b,0xee, -0x21,0xdb,0x9b,0x43,0x7d,0x63,0x79,0x8f,0x28,0xf5,0xe6,0xe5, -0x07,0xa0,0x19,0x06,0x32,0xaa,0x02,0x8a,0x95,0x91,0x12,0xec, -0x57,0xea,0xdc,0x68,0x97,0x54,0xe7,0x99,0x50,0x22,0x80,0x30, -0x5a,0x86,0x31,0x3e,0xd2,0x4b,0x75,0x9c,0x73,0x20,0x1d,0xc4, -0x52,0xee,0x3c,0xbb,0xb3,0xcb,0x4a,0xe7,0xfe,0x6e,0xc9,0xb2, -0x6b,0x17,0xa6,0xfc,0xfa,0x8f,0xb7,0x4c,0xf6,0x93,0x18,0xda, -0x9a,0x69,0x57,0x56,0xca,0xb8,0xa8,0xc7,0x08,0x12,0x64,0x20, -0xcf,0x06,0xb9,0x19,0x44,0x9c,0x70,0xf2,0xfc,0xdd,0x34,0x54, -0xb2,0x60,0xfb,0x9e,0x55,0xa5,0xf5,0x47,0x92,0xed,0x10,0xb1, -0x27,0x1f,0x7c,0xe2,0xc1,0x67,0xd6,0x2a,0x59,0xe5,0xd6,0x4b, -0x23,0xb9,0x5c,0x63,0x57,0xb8,0xa7,0x1b,0x88,0x83,0x08,0x56, -0x74,0x18,0x57,0xe3,0xcb,0x6a,0x76,0xec,0xcb,0xef,0x8b,0xbb, -0x38,0xd4,0xbb,0x72,0x6c,0xe2,0x4d,0xff,0x7b,0x69,0x38,0xa0, -0xe3,0x95,0x75,0xa1,0x5d,0x6a,0x0d,0xe4,0xa9,0xd4,0x5c,0x7b, -0x49,0xa1,0x3e,0xe8,0xde,0xa1,0x18,0x69,0xab,0xe7,0xe2,0x03, -0xde,0x35,0x61,0xcc,0x3c,0x2b,0xc8,0xe4,0x1d,0xfa,0xa4,0xa3, -0x4a,0x17,0xf6,0x1d,0x65,0x06,0xfb,0x6f,0xed,0xa7,0x43,0x65, -0x91,0xea,0x19,0xcc,0xd5,0xa2,0xa9,0xa5,0x45,0xc5,0xec,0xfa, -0xb9,0x0c,0x94,0x13,0xdd,0x4b,0x91,0xd1,0x0a,0xe6,0x4f,0x4f, -0xcf,0xe4,0x6b,0xd1,0xd5,0x7b,0x7e,0xba,0xe4,0x1e,0x93,0xcc, -0x17,0x19,0xae,0xff,0xd1,0x2a,0xed,0xfe,0x43,0xb9,0x54,0xdf, -0xed,0x83,0x46,0xed,0x68,0x0b,0x1a,0xca,0xdb,0x3a,0xf5,0xcf, -0x0f,0xf9,0xdc,0x9e,0xb0,0xe9,0x70,0xcb,0xef,0xd1,0x09,0xfa, -0x93,0x9b,0x85,0x7d,0x61,0xee,0xe4,0x1b,0x34,0xfa,0x2c,0x37, -0x1f,0x20,0xc0,0xcc,0xbd,0x26,0xaa,0x70,0xa1,0x38,0xba,0x50, -0xba,0x3b,0xd5,0xbc,0x2b,0x5e,0x4a,0x92,0xed,0x2c,0xfa,0xa9, -0xf6,0xb6,0xf1,0xc3,0x9b,0x37,0xaf,0x1a,0x5c,0x5b,0x43,0x61, -0xae,0xe9,0xab,0xd9,0x91,0x25,0x99,0x5d,0xe5,0x69,0xd7,0xd6, -0x49,0x1b,0xdf,0x3a,0x87,0xce,0x7b,0x6f,0x96,0x50,0xda,0xdd, -0x61,0xb2,0xd8,0xc4,0x95,0x99,0x2b,0xe4,0xcd,0xea,0x6e,0x34, -0xf7,0x9c,0xcf,0x3e,0xb0,0x5c,0x1a,0x27,0x66,0x78,0xbb,0x72, -0xe1,0xb7,0xff,0xe3,0x89,0xf8,0x2f,0x40,0x82,0x2a,0x26,0xf7, -0x51,0x76,0x51,0xb4,0xe7,0x4c,0xe6,0x2e,0x32,0xbf,0x77,0x42, -0xad,0x54,0x9e,0x3b,0xd8,0x13,0x9e,0x3c,0x75,0xe0,0x03,0x1b, -0x1e,0xb8,0x6a,0xd2,0x93,0x5f,0xc5,0x05,0xa7,0xbb,0x03,0xf8, -0x19,0x95,0xa9,0x38,0x03,0x7b,0xd6,0x67,0x6d,0x81,0x7e,0x46, -0x65,0x74,0x2e,0xb5,0x6f,0x4d,0xd6,0x76,0x11,0xaf,0x70,0xe4, -0x59,0xce,0xbd,0x2b,0xa5,0x01,0x6f,0x9b,0x10,0x84,0xa8,0x58, -0xa3,0xbd,0xa7,0xf4,0x19,0x8d,0xe3,0x18,0x0d,0x7b,0xb6,0xce, -0xd9,0x76,0xfa,0x8c,0xca,0x64,0x0c,0x88,0x3d,0x6b,0xcb,0x6d, -0x22,0x7d,0x46,0x7d,0x3a,0xc6,0xc4,0xbe,0xbd,0x67,0x6f,0x35, -0x79,0x9f,0xdb,0x73,0x1c,0xc8,0xc2,0xad,0xb2,0xa2,0xf3,0xdc, -0x03,0xbc,0x7d,0x8e,0xee,0x76,0x13,0x50,0x8e,0xa4,0x82,0x80, -0x6d,0x8e,0x6e,0x14,0x68,0x25,0x7f,0xe7,0x55,0x7a,0xfc,0xcb, -0x45,0x7b,0x2e,0x40,0x8f,0xfc,0x34,0xfb,0x36,0x48,0x63,0xac, -0x11,0x76,0xdc,0x4f,0xb3,0x76,0x77,0x26,0xa5,0xd2,0x8d,0x36, -0x5e,0x08,0x00,0x01,0x18,0x9d,0xa9,0xea,0xec,0x33,0x47,0x2b, -0x9b,0x9d,0xec,0xe5,0x68,0x2a,0xbd,0x2c,0x17,0x27,0xcb,0xaf, -0xd8,0xdc,0x9d,0xb0,0xaf,0x41,0x7c,0xda,0xea,0x3f,0x4d,0xc5, -0xf9,0x5a,0x97,0x50,0x90,0x2e,0xab,0x54,0x81,0x1c,0xcb,0xa8, -0x9c,0xc5,0x37,0xb3,0x19,0xe0,0x32,0xcb,0xba,0xd1,0xa1,0x03, -0x0e,0x9a,0x94,0xab,0xc2,0x65,0x91,0x26,0x1f,0xf4,0x76,0xc8, -0xb2,0xe3,0x83,0xde,0xca,0x79,0x3d,0xc7,0xa2,0xf6,0xe5,0x3e, -0x23,0x20,0x57,0xa8,0x94,0x9d,0x1b,0x23,0x68,0xe8,0x9e,0xee, -0x38,0x98,0x7e,0xef,0xde,0x04,0x32,0x21,0x1c,0xa7,0xf1,0xad, -0xbc,0x2c,0xe4,0x1c,0xed,0x96,0xb9,0xbd,0x2a,0x56,0xe5,0xfd, -0x64,0x35,0x0a,0xa1,0x7f,0xf9,0x56,0xa8,0x48,0xcf,0x4e,0xdd, -0xea,0x04,0x6b,0x00,0x39,0x9a,0x21,0x83,0x61,0xac,0x13,0x33, -0xdf,0x93,0x43,0x7c,0x95,0xa3,0x6f,0xaf,0x90,0xd1,0x2f,0xd9, -0xec,0xd1,0xf7,0xa3,0x94,0xd7,0xad,0xc8,0xb7,0x48,0xd6,0xac, -0xa2,0x64,0xaa,0x2d,0x62,0x62,0xcd,0x1e,0x5e,0xa6,0x6a,0x47, -0x59,0x97,0x33,0xdd,0xa1,0x4a,0x57,0x76,0x50,0xb1,0x70,0xce, -0x47,0x55,0xd1,0xc6,0x5b,0xe0,0x0a,0x28,0xe7,0xa9,0x5a,0x0e, -0x54,0xea,0xaf,0x5a,0x89,0xaf,0xe8,0xb5,0x5a,0x0e,0x5a,0x5c, -0x85,0xb0,0x07,0x29,0x95,0x2c,0x66,0x16,0x1b,0x58,0xb5,0x23, -0x51,0x8e,0xc3,0x72,0x34,0xcb,0x63,0xda,0x38,0x83,0xa5,0x4a, -0xca,0xca,0x44,0x7b,0xd3,0xdf,0xf8,0x25,0x81,0x5a,0x17,0xb3, -0x19,0x6f,0xb7,0x8b,0x39,0xbe,0x78,0x36,0x5b,0xd1,0x65,0xc1, -0x45,0x3a,0x9c,0x2d,0xe2,0x08,0x33,0x9b,0xe7,0xe2,0x02,0x2a, -0xbb,0xa6,0xa0,0xab,0x82,0xb6,0xb7,0x5f,0xa1,0xf7,0x58,0xde, -0x93,0x4c,0xc0,0x22,0x9b,0x33,0x7e,0xe6,0x7b,0x83,0x26,0x07, -0x3b,0x9e,0xb3,0x94,0x08,0x58,0x99,0xe4,0x33,0x67,0xd0,0x92, -0x0f,0x9d,0x20,0xaf,0x34,0x3b,0x95,0x12,0x50,0x58,0x87,0x83, -0xa5,0x9d,0xce,0x29,0x42,0xbf,0x8c,0xa2,0x88,0x30,0x0f,0xa2, -0x5c,0x0d,0xd2,0x9f,0x6f,0xa7,0xaf,0x38,0x26,0x4f,0x66,0x00, -0xdc,0x74,0x31,0xa5,0x27,0x56,0xbe,0xe7,0x47,0x3b,0xed,0x1a, -0xe4,0x3b,0x9e,0x12,0xe0,0xe5,0x60,0x6e,0x67,0x4e,0x06,0x73, -0x99,0xf1,0x0b,0xdd,0x3a,0xb1,0xf3,0xf8,0x1e,0x8a,0x93,0xfd, -0x4b,0xf2,0xfe,0x6a,0x59,0x84,0xa1,0x64,0x45,0x41,0x32,0x86, -0x45,0xa7,0x43,0x00,0xa5,0xc8,0xec,0xef,0x60,0xbd,0x94,0xd9, -0xb9,0x78,0x33,0x50,0x51,0xff,0x7a,0x36,0x71,0x99,0x06,0xdf, -0x32,0xeb,0x05,0x74,0xea,0xfa,0xe7,0x4b,0x3b,0x37,0xe1,0x24, -0x09,0xf0,0xe3,0x00,0xb7,0x4d,0x0a,0x40,0x63,0x93,0xac,0x30, -0xc9,0x4b,0x92,0xdc,0xfd,0xf2,0xe2,0xbd,0xcc,0xfb,0xef,0x71, -0x96,0xaa,0xee,0xc7,0xe0,0xb8,0x32,0xf9,0x1b,0xa8,0x2a,0xd2, -0x42,0x40,0x12,0x2f,0xce,0xcb,0x63,0x59,0xc2,0xef,0x07,0xc9, -0x14,0x07,0xc3,0x30,0x9e,0x5d,0x3e,0x38,0xef,0x42,0x02,0xdd, -0x4a,0x72,0x53,0xe5,0x1b,0xd4,0xb7,0x8b,0xc1,0x7c,0x1e,0x8f, -0x70,0x97,0x49,0x2d,0xf1,0x31,0xc6,0xe8,0x83,0x73,0xc2,0xb9, -0xe9,0xd4,0x56,0xea,0xe2,0x6b,0xed,0x20,0x42,0x69,0x83,0xc2, -0xf1,0x7a,0x3e,0x5b,0x2c,0xd3,0xae,0x57,0x96,0xc9,0xe1,0x88, -0x0f,0x0f,0x65,0x58,0x62,0x09,0x8b,0x17,0x4d,0xf8,0x57,0xe4, -0x66,0x44,0xe7,0xbe,0x90,0x3f,0x9b,0xe7,0xf0,0xd1,0xe6,0x01, -0x13,0x9d,0x8b,0xf3,0xe6,0xaf,0xdf,0xfd,0xf2,0xfa,0xc5,0xcf, -0x3f,0x45,0xb5,0xb0,0xf9,0xa8,0xd9,0xaa,0x91,0xe9,0x80,0x3e, -0x66,0x11,0xbb,0x9a,0xc1,0x1f,0x29,0x59,0x0e,0x23,0x04,0x3d, -0xcc,0x24,0x23,0xb1,0x60,0xe0,0x66,0xba,0x00,0x81,0xda,0xe8, -0x00,0x5b,0xe9,0xe3,0x87,0x2d,0xa9,0x87,0x87,0x90,0xaa,0x31, -0x45,0x91,0x93,0xe9,0x5b,0x79,0x5e,0x01,0x71,0x47,0xdd,0x27, -0x42,0x28,0xbe,0x69,0x0f,0x08,0xea,0xe6,0xcb,0x5c,0x7b,0x4d, -0xa2,0x40,0x48,0x08,0x93,0xdd,0xe1,0x3f,0x67,0x49,0x27,0xa9, -0xd7,0x91,0x30,0x55,0x83,0xf4,0x26,0xe5,0x6a,0x70,0xa3,0xb2, -0x97,0xf4,0x45,0x82,0x3f,0x30,0x9e,0x9f,0x54,0x26,0xf2,0xda, -0x13,0xfb,0x86,0xab,0x7a,0x50,0x66,0x22,0x19,0x57,0x19,0xc1, -0x85,0x55,0x31,0x26,0xde,0xbf,0x66,0x2c,0x05,0xd5,0xf7,0x85, -0xfc,0x51,0x41,0x45,0x87,0x0e,0x09,0x22,0x8c,0x03,0x41,0xaa, -0x77,0x57,0x9f,0xc8,0xad,0x3f,0x8a,0xc9,0xda,0xd3,0x13,0x30, -0x76,0x53,0x84,0x1c,0xea,0xca,0xbc,0xb6,0x56,0x1a,0xdc,0x4f, -0x58,0x87,0xea,0x23,0x48,0xec,0xca,0xb4,0x62,0xdf,0xb4,0x3d, -0x72,0x47,0xc4,0xca,0x6d,0x4f,0xd5,0x55,0x2c,0x68,0xd4,0x8a, -0x71,0xc2,0x01,0x3c,0xa8,0x16,0xbe,0x79,0x5c,0xce,0x83,0x42, -0x21,0xb9,0x25,0x8d,0xe5,0xe4,0xfd,0x3c,0xd6,0x41,0x14,0xb7, -0x25,0xaa,0xb1,0x9e,0x7a,0x80,0xe3,0x04,0x03,0x48,0x3f,0x18, -0xe0,0xe8,0x26,0x5b,0xeb,0xc1,0x74,0xf6,0x40,0xc6,0x70,0xe3, -0x00,0xd2,0xb5,0x8e,0xba,0x6b,0x47,0xb2,0x3c,0x1e,0x51,0x3c, -0xe2,0xe9,0x87,0x6a,0xf6,0x4d,0x60,0x91,0xed,0xf2,0x50,0x22, -0x2c,0x84,0x7c,0x38,0x6b,0x75,0xa4,0xd8,0x13,0x3f,0x61,0x00, -0xc2,0xbf,0x14,0x56,0xc1,0x56,0xa7,0xcc,0x54,0x49,0x83,0xe2, -0x2b,0x67,0xf9,0xc6,0xe5,0x0c,0x6b,0xc3,0x88,0x6b,0x92,0x0a, -0x35,0xe5,0x16,0x78,0x0e,0xd2,0xc6,0xd4,0x74,0x0d,0x56,0xcf, -0xa1,0xdd,0x6f,0x97,0xe4,0xf0,0xf5,0xbf,0x3d,0x7a,0x4b,0x61, -0x47,0x4c,0x51,0x79,0x6f,0x11,0x83,0x0a,0x25,0xdb,0x1e,0x15, -0x51,0xe9,0xcc,0x31,0x3c,0x2f,0xf0,0xc5,0x81,0xfc,0xb4,0x62, -0x52,0x62,0x60,0x16,0x8e,0x4b,0x69,0x75,0xac,0x36,0x6c,0x11, -0xd3,0x46,0x9c,0xdb,0x13,0x8f,0xec,0xbd,0xc5,0x7f,0x52,0xa7, -0x51,0x9d,0x76,0xcf,0x31,0x11,0x6e,0xf7,0x51,0xda,0x17,0xea, -0x43,0xc2,0x55,0xdd,0x91,0x6e,0x36,0x2f,0xf1,0x8a,0xba,0x0e, -0x1a,0xc6,0xbf,0x0e,0x40,0x45,0x2a,0xf5,0x58,0xa6,0xb2,0x3a, -0x45,0x75,0xb5,0xd9,0x5b,0x40,0xe0,0x37,0x15,0xec,0x92,0xb6, -0x6a,0x34,0xb8,0x7c,0xbf,0xad,0x7e,0x89,0x7d,0x65,0x88,0x94, -0x2e,0xa2,0xeb,0x97,0x08,0x92,0x93,0xf9,0x45,0xa4,0x09,0x03, -0xbb,0xd2,0xd2,0x76,0x79,0x2f,0xdd,0xa9,0xe3,0xac,0x4d,0xb3, -0xdd,0xaa,0x8e,0xb7,0xb6,0xf7,0x53,0x72,0x5d,0x8f,0x2b,0x90, -0x63,0x06,0x5a,0xd7,0x66,0xf3,0x40,0x2b,0x3f,0xa6,0x9a,0x4c, -0xad,0xf3,0x26,0x87,0x6d,0xfd,0xb2,0x3a,0x9f,0xed,0x38,0x39, -0x3d,0x73,0x45,0x7a,0x76,0xa6,0xcf,0xae,0xc9,0xf9,0x5c,0xfd, -0xbf,0x27,0x37,0x0e,0x0f,0x9d,0x89,0x82,0x03,0xf9,0xd9,0xd3, -0x00,0xe9,0x83,0x2d,0x7a,0x5b,0xb3,0x40,0x6f,0x15,0x49,0xba, -0xf7,0xea,0xb4,0x83,0x3d,0xc9,0xdc,0xe8,0x7a,0x90,0x20,0xb2, -0x6b,0xa1,0x6f,0x70,0xd5,0xb7,0xcb,0x40,0x92,0x29,0x77,0x77, -0xb6,0x82,0xc0,0x23,0x92,0x65,0xb2,0xcc,0xd4,0x59,0x21,0xc9, -0xc3,0x41,0x50,0xd5,0x6f,0x6d,0xcb,0xbe,0xe6,0x5e,0x63,0x0a, -0x54,0xa7,0x51,0x46,0x57,0xa7,0x7f,0xfe,0x94,0x8d,0xb5,0x2b, -0xf1,0xe4,0x3f,0x87,0x87,0xfb,0xce,0xdd,0x5d,0x96,0xe2,0xb6, -0xb4,0x5e,0x70,0xc8,0x1e,0x30,0x0e,0xf5,0x24,0xc1,0x14,0x79, -0x86,0x76,0x3f,0xb2,0x6e,0x9a,0x7d,0x49,0xd6,0x85,0x3b,0x58, -0x87,0xab,0x0f,0xe6,0x1c,0xd5,0xaf,0x18,0x87,0xc9,0x5d,0x95, -0xfa,0xe7,0xd8,0x26,0x49,0x01,0x62,0x25,0x4d,0x7b,0xf3,0x4d, -0x32,0xcc,0x68,0x01,0x8b,0x6f,0x26,0xf8,0x17,0xd9,0x2f,0xc3, -0xf1,0x6a,0x14,0xbb,0x7c,0x5b,0xc2,0x4c,0x17,0x1b,0x2a,0x4c, -0xfb,0x0f,0xc2,0xb6,0xb3,0xbc,0xe2,0xe6,0xcb,0x85,0x95,0xe6, -0x80,0xcc,0xec,0x36,0x42,0xb2,0xe1,0x55,0xbe,0x27,0xd1,0xb6, -0xcb,0xd5,0x9f,0xae,0x8e,0x9f,0x10,0x89,0x22,0x06,0xdf,0xd0, -0xe8,0x48,0xa6,0x37,0xb3,0xeb,0x1c,0x95,0xf6,0xb5,0xb1,0x01, -0x5e,0x0f,0xb3,0xae,0x76,0xe9,0xa9,0x5a,0xb4,0xf0,0x78,0x11, -0xd7,0x62,0x14,0x18,0x48,0xef,0x99,0xc9,0xc2,0x26,0x8a,0x02, -0x9a,0xa5,0xd5,0x44,0x79,0x5c,0x54,0x86,0xb1,0x6f,0x53,0x9e, -0xba,0x2a,0xe6,0x9b,0x6b,0x62,0xd0,0x0d,0x48,0x89,0xcf,0x44, -0xcf,0xc7,0x2b,0xfb,0xcc,0x15,0xb1,0xd3,0x3b,0x29,0x7b,0xd6, -0xc9,0x8c,0xa0,0x67,0x4a,0x18,0xdf,0xed,0x55,0xbc,0xc8,0xf1, -0x60,0xb0,0x5c,0x62,0x68,0xa6,0x64,0x91,0x2e,0x9d,0xb8,0x5a, -0xfc,0x1a,0x0a,0xe5,0xfa,0x5d,0xca,0x56,0x43,0xa9,0xd7,0x6f, -0x9f,0xf7,0x38,0xa5,0x86,0xd3,0x58,0xad,0x5d,0x63,0xe5,0x56, -0xeb,0x97,0xd2,0x62,0x2d,0x55,0xc0,0x10,0x7a,0xc0,0x28,0x61, -0x82,0xa7,0x1f,0x44,0x1c,0x58,0x08,0x86,0x52,0x69,0xc3,0xe8, -0xc1,0x73,0x10,0x30,0xed,0x58,0xd5,0xdf,0x2a,0xe8,0xb7,0x28, -0xa7,0x26,0x5a,0x2d,0xc3,0x79,0x5a,0x20,0xa3,0xd6,0x7b,0xac, -0x1a,0xb5,0xe2,0xc5,0x20,0x56,0x2a,0x42,0x1e,0xda,0x28,0x24, -0xa6,0xbd,0xa0,0x2f,0x97,0x7b,0xf0,0x8b,0x05,0x97,0x8d,0x8c, -0xaf,0x4f,0x1f,0x3d,0x3a,0x79,0xa4,0x8c,0x2f,0x7a,0x36,0x13, -0xea,0x93,0x9d,0x8a,0x9f,0xb4,0x86,0xea,0x14,0x2b,0x60,0x26, -0x63,0xa6,0x2c,0xdc,0x08,0x8f,0x03,0x5b,0x85,0x7c,0x1a,0xce, -0x26,0xf3,0xd5,0x32,0x1e,0xb5,0x31,0x87,0xc7,0x29,0xfd,0xd4, -0xb7,0x4e,0xf7,0xd1,0x05,0x7c,0x63,0x87,0x11,0x69,0x0d,0xd0, -0xdd,0x53,0x15,0xb0,0xa8,0x76,0x54,0xf9,0x33,0xa6,0xac,0xa9, -0xbe,0xc1,0xc6,0x54,0xb4,0x32,0x75,0x8c,0x40,0xd3,0xad,0x7e, -0x6c,0xcc,0xfc,0xc9,0x17,0xde,0xa8,0x57,0x92,0xe9,0x7f,0x6e, -0xaf,0x24,0xd3,0xcf,0xe9,0x95,0x07,0x95,0xbd,0x62,0x3a,0xe5, -0xff,0x64,0x9f,0x7c,0xfd,0x25,0xfb,0x24,0xbd,0x5a,0x5d,0x5e, -0x8e,0xe3,0xdc,0x66,0x13,0xb5,0x7e,0x30,0x1d,0x31,0x15,0x51, -0x20,0x24,0xd8,0xc8,0x32,0xec,0x2a,0x9a,0xed,0xe3,0xcb,0x7e, -0x68,0xec,0xe2,0x9f,0xd9,0xc4,0x23,0x0c,0xf5,0xba,0xaf,0x51, -0xb0,0x69,0xdd,0x08,0xfb,0x91,0x4e,0x41,0xd0,0xbe,0x70,0x3f, -0x59,0x47,0x6c,0x4c,0x39,0xa2,0x76,0x30,0x99,0xe7,0x88,0x15, -0x53,0xf1,0x7e,0x35,0x58,0x98,0x58,0xdf,0xf9,0x55,0xd7,0xd7, -0xad,0xbb,0x3b,0x82,0xc0,0x49,0xb6,0xa7,0xe9,0x32,0x92,0xd3, -0x08,0x7d,0xd0,0x71,0x8a,0x13,0xc4,0x00,0xde,0x4e,0xf5,0x02, -0xa1,0x46,0x37,0x3e,0xd8,0xa3,0x9e,0x35,0x1a,0xd3,0x23,0xdf, -0x2f,0x94,0x39,0x50,0xa1,0x8c,0x9d,0xd9,0x83,0xb3,0xba,0xdc, -0x3d,0x55,0xfb,0x7a,0xfc,0x66,0x55,0x7f,0x83,0x33,0x6e,0x3a, -0x5b,0x2c,0xbf,0xc9,0x59,0x27,0xaa,0x5b,0xed,0x95,0xa6,0x22, -0xc2,0xa5,0x49,0xd6,0x67,0x26,0x2b,0x9a,0x59,0xbc,0xaa,0x09, -0xa4,0xc4,0x92,0x70,0x24,0x89,0xb2,0xdb,0x0c,0x34,0x5c,0x60, -0x9d,0xc9,0xa0,0xbd,0xaf,0xd1,0xba,0xf1,0xa9,0x31,0xc6,0xd1, -0x75,0x1c,0x5f,0x2e,0xc5,0x82,0xd6,0xaf,0x3c,0x15,0x47,0x98, -0xd2,0x54,0x88,0xc5,0x45,0x44,0x99,0x3a,0x01,0x87,0xec,0x00, -0xe6,0x8c,0x0b,0xd2,0x0e,0x83,0xb3,0x8b,0xbb,0x3b,0x9e,0x9a, -0x30,0x12,0xa6,0x1e,0xb2,0x08,0x75,0x71,0x36,0xb0,0xf2,0x2e, -0xb4,0x92,0x55,0x67,0x02,0x54,0x0f,0x4b,0x1f,0x57,0x41,0xbf, -0x41,0xc4,0x6a,0xbc,0x5f,0x23,0xfb,0xf7,0xfd,0x62,0xb6,0x9a, -0x1b,0xd6,0x5f,0xc4,0x57,0x83,0x9b,0xc4,0x8a,0xc9,0xbb,0xab, -0x53,0xd4,0x60,0xdc,0x68,0xfd,0x16,0xb1,0x45,0x44,0xb0,0x5d, -0x63,0x3e,0xb6,0xb7,0xf6,0xda,0x56,0xb5,0xa2,0xd7,0xd6,0x7b, -0x98,0x78,0xa4,0xf7,0x54,0x2b,0xa4,0xaa,0x40,0xc3,0x42,0xe4, -0x16,0x3a,0x24,0x78,0xd4,0x78,0x90,0x3c,0xfa,0xeb,0x78,0x27, -0xbb,0xa5,0x3e,0x79,0xfc,0x14,0x88,0xc9,0xf0,0xa5,0xb5,0x4b, -0xd3,0x7a,0xdb,0xfa,0x8d,0x3b,0x1c,0xb9,0x95,0xd5,0x39,0xb3, -0x7e,0x9f,0x7a,0x6c,0x44,0x4a,0x25,0xa0,0x45,0xba,0x9a,0x2e, -0xb7,0x15,0xc7,0x1b,0xcb,0x5b,0xe8,0xab,0xd7,0x1d,0x0a,0x43, -0x42,0x8a,0x72,0x1a,0x8f,0xc8,0x1a,0x35,0xdd,0x4f,0x1b,0x7c, -0x62,0xfb,0xea,0x40,0xf6,0xaf,0xd6,0xeb,0xd5,0x5d,0x6c,0xb6, -0x51,0x94,0x9d,0xc4,0xd6,0x6b,0xe5,0x06,0x2d,0x5e,0x99,0xb9, -0x05,0x05,0x7c,0x05,0x02,0x1b,0x11,0x2d,0x6a,0xc7,0x05,0x53, -0xce,0x20,0xb3,0x23,0xdf,0x65,0x4a,0x46,0xa0,0x05,0x6e,0xeb, -0x94,0x7c,0x76,0x06,0x03,0xa2,0x14,0x25,0xa1,0xe8,0x01,0x70, -0xdf,0xff,0x9a,0xe5,0x11,0xf1,0xc3,0x77,0x3d,0x6c,0x53,0x1d, -0xf0,0x53,0x8f,0x97,0xd9,0x2d,0x2a,0xdd,0xe5,0x2c,0x17,0xf5, -0x31,0xa7,0xb7,0xba,0xee,0x34,0xdd,0xb5,0xcc,0x6a,0xfc,0x6e, -0x57,0x6d,0x9f,0x77,0x8d,0x3e,0xb2,0x96,0x54,0x6d,0x7d,0x43, -0x9e,0x0a,0xf7,0xfa,0xa4,0xf7,0x93,0x8f,0xf9,0x29,0xaa,0xb0, -0xd6,0x08,0xaa,0x2b,0x32,0x3f,0xdb,0xd6,0x4e,0x94,0xda,0x7d, -0x22,0x6b,0x13,0xec,0x5b,0x98,0xb3,0xf0,0x2c,0x1d,0xfe,0x2c, -0x07,0xf6,0xda,0x81,0x25,0x20,0x3f,0xcd,0x70,0xc5,0x94,0xa7, -0x4c,0x65,0x4e,0x9a,0xaa,0xe9,0x86,0x59,0x1d,0xf4,0xdb,0xce, -0x3a,0x03,0x71,0xe1,0x64,0xc2,0xab,0x14,0xde,0x5d,0xda,0x51, -0x55,0x49,0x79,0x5b,0x12,0x1a,0x5e,0xae,0x62,0x58,0x74,0xf9, -0x84,0x1f,0x0f,0xa7,0xbe,0x54,0x3b,0x9c,0x1a,0xc3,0x92,0x46, -0xe9,0xa9,0xd2,0x81,0x9c,0x8a,0xc0,0x97,0x1b,0x28,0xc4,0x61, -0x58,0x47,0xe2,0x36,0xf8,0x68,0x31,0x9b,0xdf,0xbb,0xd9,0x25, -0xcd,0xdc,0x90,0x2e,0x98,0xcc,0x07,0xc3,0x7c,0x43,0x8b,0x9b, -0x31,0x8c,0xc4,0x12,0x35,0x56,0xf7,0x97,0x63,0x7c,0x49,0x70, -0x6a,0xc7,0x61,0x07,0x73,0x09,0xcc,0x0e,0xa8,0x78,0x76,0x2b, -0x66,0xab,0x25,0x7c,0x1a,0xa2,0x38,0x99,0xc2,0xde,0xf2,0x3e, -0x07,0x41,0xeb,0x21,0x40,0xa1,0xb5,0x87,0x83,0xa5,0x34,0x3c, -0xb9,0xb4,0x20,0x20,0xb5,0x94,0xe7,0x12,0x79,0xb3,0x21,0x17, -0xa3,0xdb,0xbf,0xbb,0xe3,0x14,0x69,0xcc,0x28,0x13,0x42,0xd6, -0xdf,0x45,0x7d,0xea,0xd6,0xc1,0x00,0x6d,0xd9,0x1c,0x39,0x59, -0xe4,0x5a,0xd1,0xe6,0xbf,0x39,0x6d,0x2c,0x33,0x51,0xfd,0x17, -0x98,0xc1,0x4c,0x93,0x68,0xcc,0x14,0x28,0x2b,0x71,0x72,0xf1, -0xbd,0x52,0x5a,0x76,0x26,0xb0,0xcc,0x5d,0x6d,0xe9,0x8f,0x51, -0x72,0x79,0x09,0xcb,0xb6,0x29,0x58,0x58,0x12,0x41,0xd9,0x42, -0x3c,0x64,0xb9,0x59,0x4d,0x93,0xdf,0x39,0x16,0xd5,0xef,0xab, -0xc2,0xa0,0x4c,0xd2,0xd7,0xa4,0xb3,0x4b,0x74,0xb3,0x63,0x7f, -0x29,0x38,0x19,0x9b,0x9c,0x5e,0xae,0xd5,0x45,0xb4,0x16,0x37, -0xd8,0xe4,0x8f,0xe8,0x20,0xe4,0x2d,0x1e,0x35,0x50,0x2d,0x0d, -0x8f,0xaa,0x4b,0x12,0x51,0xd8,0xb2,0xe1,0x74,0xb3,0x17,0x2a, -0xd2,0x18,0x78,0x9a,0x33,0x9d,0x25,0xce,0x8a,0xe9,0x5d,0x93, -0xdc,0xa5,0x84,0xc3,0x43,0x44,0xd1,0xc3,0x7f,0xcc,0x18,0x8c, -0xa4,0x49,0xd1,0xb6,0x5f,0xe7,0x04,0x08,0x29,0x0a,0x20,0x42, -0xf4,0x69,0xf7,0xb6,0x70,0x76,0x3a,0x79,0x64,0xf3,0x56,0xb7, -0x7d,0xf0,0x25,0xf9,0x6e,0xfb,0x6e,0x59,0xdd,0x87,0x5d,0xe1, -0x69,0x51,0xb1,0x7a,0xec,0x40,0x0e,0x75,0x7a,0x9c,0x20,0xcd, -0x07,0xd6,0x96,0x52,0x20,0x2d,0xa4,0x65,0xf9,0xee,0x4b,0x68, -0x19,0xad,0x72,0xd8,0xca,0x0a,0xb9,0xb8,0xe1,0x16,0x70,0xdd, -0xf6,0xcd,0xe3,0xc1,0x88,0x88,0xad,0xc8,0x25,0xcb,0xab,0x78, -0x61,0x1b,0xe4,0x72,0x5f,0x89,0xd2,0x05,0x21,0x38,0x8b,0x82, -0x8d,0xdc,0x7d,0x31,0x52,0x59,0x4d,0xb3,0x33,0xb6,0x8d,0x17, -0x40,0xa5,0x00,0x17,0x1a,0xc3,0x92,0x51,0xbe,0x64,0xb0,0xa3, -0xec,0x53,0x43,0xd4,0x00,0x45,0xea,0x3e,0x26,0x73,0xbb,0x33, -0x94,0xf5,0x20,0xa7,0x3b,0xda,0xf1,0xf0,0x94,0x9d,0x6f,0x48, -0xa8,0x71,0x7e,0xcd,0x97,0x7e,0x20,0x1e,0xf4,0x90,0x96,0x4b, -0x3c,0xbe,0x60,0x8d,0x23,0xcf,0x6b,0xf0,0x34,0xd9,0x3d,0x3a, -0x96,0xa0,0xbd,0x04,0x6f,0xe3,0x16,0x90,0xd7,0xea,0x89,0x1d, -0xf3,0x43,0x49,0xce,0xec,0xc2,0xdd,0x2a,0xc7,0x75,0x80,0x7e, -0x24,0x4f,0x1f,0x80,0xd1,0xea,0x40,0x2e,0x36,0x36,0xda,0x1a, -0xb2,0x6c,0x67,0x73,0xb2,0x8d,0xa0,0xa5,0x27,0xdb,0x8c,0x54, -0xd9,0x75,0x08,0x86,0x47,0xd9,0xf2,0xf5,0x3d,0xf8,0xd9,0x76, -0x73,0x60,0x2e,0xee,0x47,0xea,0x77,0xd8,0x77,0x49,0xdf,0x08, -0x2d,0x20,0x05,0x2d,0xb3,0xa4,0xd8,0xe2,0x52,0x85,0x98,0x26, -0xb0,0x74,0xa8,0x55,0x46,0x27,0xe7,0x08,0xe0,0x18,0x6b,0x78, -0x06,0x6f,0x23,0xa8,0x4d,0x57,0x93,0x8b,0x78,0x61,0x1c,0x2b, -0x74,0xae,0x3a,0xc8,0x89,0x1c,0x7b,0xd4,0xa2,0xc4,0x97,0x36, -0x5c,0x82,0xe3,0x1f,0x13,0xba,0x49,0x1b,0x6a,0x87,0x9a,0xcf, -0xb4,0xce,0xb0,0x96,0xad,0x4c,0x40,0x5d,0xe3,0x6f,0xab,0x5f, -0x1b,0xed,0x2a,0xa1,0x37,0x67,0x99,0xe4,0x8a,0xed,0x59,0x5f, -0xaf,0xaf,0x2d,0x20,0xcf,0xe5,0x4d,0xe7,0x5f,0xb0,0x23,0x0b, -0xee,0x07,0x39,0x82,0x75,0x1b,0x3b,0x7a,0x85,0x26,0x8d,0x95, -0x17,0x5b,0x3a,0x00,0x23,0xcd,0x6c,0x67,0x3e,0xd8,0xfb,0x6c, -0xbc,0xb3,0x3b,0x08,0x16,0xe8,0xe8,0x46,0x5a,0xfe,0x3b,0xaa, -0xa1,0x76,0x8d,0xaa,0xb1,0x16,0x98,0x22,0x53,0xa1,0xed,0x16, -0x4a,0x79,0x86,0xb0,0x76,0x79,0xa6,0xb1,0xf3,0x93,0x48,0xe3, -0xc1,0x02,0x6d,0x57,0x3c,0x1a,0x8d,0xce,0xde,0x8c,0x5a,0x60, -0xc0,0x7b,0xfb,0x9d,0xe4,0xc1,0x02,0xcc,0x95,0xe5,0x6c,0x0e, -0xff,0xc4,0x73,0xff,0x53,0x61,0x03,0x24,0x0a,0x61,0xda,0x43, -0x80,0x88,0x60,0xef,0xee,0x02,0x41,0x3f,0xa2,0xc0,0xa7,0x22, -0xe6,0xa0,0xba,0xd7,0xea,0xdf,0xdd,0x85,0x9d,0x9c,0x76,0xd1, -0xd2,0x44,0x3f,0x86,0x71,0x32,0xf6,0x08,0x5d,0x83,0xb0,0xf8, -0xc7,0x54,0x2d,0xd8,0x79,0x22,0x19,0xe1,0x5e,0x11,0xd3,0x57, -0xd0,0x2e,0x5a,0x2a,0x46,0xeb,0x8e,0x4f,0x30,0x78,0x5f,0xbc, -0x5e,0xef,0x47,0xaa,0x05,0xf0,0x6f,0x3d,0x42,0x64,0x7a,0x64, -0x22,0x14,0x5b,0x69,0xc3,0x42,0xe0,0x7b,0x3e,0xe2,0xb6,0xa3, -0xed,0x4f,0x87,0xee,0x42,0x1c,0xb7,0xd6,0xc5,0x05,0xac,0x14, -0x47,0x46,0x0c,0xd0,0xd5,0xea,0xff,0xe3,0xed,0x68,0x9a,0xdb, -0xd6,0x8d,0xe7,0xfe,0x0b,0x8b,0xed,0x93,0xc8,0x92,0xb6,0xec, -0x5c,0xda,0x88,0x61,0x34,0x9d,0xb4,0x6f,0xe6,0x5d,0x72,0xe8, -0xeb,0x4d,0x52,0x35,0xa4,0x4d,0x39,0xcc,0x48,0xa2,0x22,0xd1, -0x4d,0x32,0x96,0xfe,0x7b,0xf7,0x13,0x1f,0xfc,0x70,0x3c,0xef, -0x75,0x7a,0x11,0x21,0x60,0xb1,0x58,0x00,0x8b,0xc5,0x62,0xb1, -0x00,0xc6,0x63,0x84,0xe5,0xec,0xda,0xef,0x18,0xaf,0xcd,0x6d, -0x63,0x44,0xd2,0x13,0xea,0x41,0x19,0xcf,0xef,0x62,0x3a,0xda, -0x06,0x82,0x47,0x7d,0x7b,0xc4,0xa9,0x35,0x51,0x0d,0xef,0x3e, -0x10,0xc1,0x6e,0x55,0x11,0x7f,0xd8,0x76,0xf1,0x22,0x28,0x63, -0xb1,0xa4,0xfa,0x30,0xad,0x76,0xa1,0xf7,0x78,0x52,0xa9,0xdf, -0x57,0x54,0x04,0x94,0xdf,0x7b,0x97,0xe4,0x53,0x99,0xf6,0x6f, -0xca,0xfe,0xb7,0xdb,0x0d,0xbb,0xdb,0x02,0x68,0x1b,0x9e,0xee, -0xa0,0x74,0xa4,0xb6,0x43,0x05,0xe6,0x7b,0x15,0x09,0x52,0x01, -0xf6,0x89,0x93,0xd5,0x3a,0xfa,0x1f,0x71,0x48,0x37,0xd0,0x10, -0xdd,0x85,0xb6,0x48,0x80,0x47,0xbc,0x15,0x13,0xb5,0xf5,0x0f, -0xb6,0x74,0xac,0x52,0xd1,0x55,0x67,0x1c,0x9a,0xc9,0xc9,0xf3, -0x15,0x34,0x13,0x21,0xc8,0x3c,0x7f,0x6b,0x6d,0xbd,0x32,0x1d, -0x18,0x33,0x4c,0x08,0x74,0x26,0x1a,0xa7,0x08,0x48,0xdd,0xc8, -0x3a,0xef,0xa5,0x0a,0xf2,0xab,0xdd,0xd3,0xa9,0xb9,0x2a,0xca, -0xab,0x43,0x8e,0x87,0x71,0x0d,0xf5,0x57,0xe8,0x5d,0x7e,0x0a, -0x7c,0x4b,0x11,0x61,0xb4,0x9a,0xc5,0x26,0x7a,0x46,0x93,0xe2, -0xc6,0x5e,0x10,0x42,0xff,0xc8,0xa4,0x80,0x2a,0x7f,0xf1,0x99, -0x2c,0xf2,0xe5,0xae,0xf6,0x96,0xd6,0xc4,0xe9,0x20,0xa8,0x3e, -0xa9,0x6f,0x38,0x39,0x44,0xd8,0x5b,0x2f,0x39,0x09,0x14,0x4b, -0x0e,0xb8,0x7b,0xa1,0x6d,0xc7,0x59,0x34,0x4e,0x31,0x54,0xab, -0x7d,0xa5,0x21,0xad,0x6e,0x84,0x76,0x1a,0xf2,0xca,0x20,0x2b, -0x0d,0x86,0xd8,0x86,0x64,0x42,0x59,0x4f,0x2f,0x09,0x16,0xea, -0x8c,0x87,0x72,0xeb,0xda,0x27,0xa8,0x12,0x5f,0xf3,0xaa,0xf9, -0xf1,0x5e,0x9f,0x12,0x71,0x2a,0x9b,0x7f,0x55,0xbb,0x12,0x96, -0x2d,0xe1,0x8b,0x3c,0x42,0xc7,0xb2,0x79,0xcb,0x8e,0x4b,0xe0, -0xe2,0x37,0xee,0x13,0x3f,0xcc,0x92,0x76,0xb5,0x83,0xc4,0x49, -0xf6,0x75,0xb2,0x20,0xe2,0xee,0x56,0x2f,0xf1,0x18,0x4a,0x15, -0x42,0x8c,0x8c,0xd1,0x34,0xdb,0x76,0xff,0x60,0xc1,0x7a,0xcd, -0xbd,0x6e,0x2d,0xd8,0x41,0x2f,0xfa,0x5d,0xd2,0x70,0x7d,0x32, -0x39,0x49,0xce,0xbe,0xcb,0xa0,0xe0,0x49,0x46,0xe8,0x43,0x09, -0x65,0x78,0x1f,0x02,0x89,0xfb,0xbc,0x29,0x3d,0x21,0x6b,0x72, -0x09,0x24,0x30,0x6c,0xfe,0x80,0x5e,0xc4,0x78,0xc4,0x73,0x7e, -0x4b,0xae,0xa4,0x7f,0x87,0x4c,0x7e,0x59,0x5d,0x69,0xe0,0x92, -0x17,0x5d,0x7a,0x06,0x23,0x9d,0x42,0xa8,0xbf,0x66,0x8a,0x30, -0xd5,0x92,0xcf,0xe7,0x56,0xd1,0x74,0xba,0x14,0x68,0x37,0xa4, -0x41,0x36,0xdd,0x86,0xdf,0x81,0xea,0x8c,0xd4,0x61,0xfb,0x5c, -0x87,0x90,0x70,0xad,0x50,0xee,0x0d,0x6b,0xb4,0xf8,0x33,0x63, -0xdd,0xce,0x7d,0xc9,0xed,0xfb,0xcc,0xe0,0x98,0x87,0xf7,0x50, -0xe0,0x51,0x79,0x42,0xea,0x17,0x0d,0x34,0x2a,0x14,0xf5,0xa3, -0x5a,0x47,0x33,0xc9,0x6a,0x6b,0xd4,0x1c,0xf3,0x6a,0xab,0xad, -0x89,0xef,0x27,0x09,0x6e,0x87,0x17,0xa9,0x47,0x12,0x43,0x55, -0x64,0x4d,0xb5,0xc4,0x76,0x38,0x09,0xdc,0xf7,0x72,0x47,0xb5, -0xdb,0x95,0x0f,0x55,0xae,0x37,0x46,0x09,0x6a,0xe6,0x0f,0xa5, -0x0b,0x23,0x61,0x3e,0xd9,0x1d,0x12,0xdf,0xdb,0xc8,0xe9,0x97, -0x97,0x9a,0xcb,0x64,0xb7,0xbd,0xd6,0xcb,0x44,0x1c,0x09,0x8b, -0x27,0x05,0xbb,0x36,0x39,0x53,0xa4,0xf5,0x3d,0x26,0xce,0x07, -0x2b,0x4f,0x9d,0x89,0x20,0xd1,0x2c,0xf4,0x5a,0xdf,0x54,0xd1, -0xba,0x3b,0x0c,0x36,0xbe,0x1c,0x3a,0xfe,0x08,0x2c,0x66,0xb2, -0x8d,0xc7,0x23,0xc1,0x67,0x6e,0x3f,0xd5,0x1e,0x1a,0xee,0x0a, -0x1a,0xf3,0x91,0x22,0xb3,0x3b,0x6f,0x83,0x25,0xbb,0x1d,0x56, -0x77,0x3a,0xcb,0xca,0x58,0xd4,0x9a,0x1c,0x17,0x92,0xae,0x10, -0x82,0x74,0x12,0x8d,0xb3,0x90,0x20,0x6f,0xc9,0x8f,0xed,0x05, -0xc1,0xc8,0x37,0xd4,0x52,0x4b,0x91,0x47,0x1f,0x51,0x80,0xfe, -0xe0,0x6d,0x76,0x21,0x17,0xf1,0x9e,0xdd,0x0d,0x47,0x78,0x92, -0xd0,0xb2,0x8f,0xee,0x58,0xd3,0x13,0xf1,0x93,0x53,0xa6,0x20, -0x6b,0x4f,0xa8,0x5c,0x38,0xda,0xe9,0x6a,0xf7,0xa9,0x0a,0x77, -0xbe,0xec,0x5c,0xd5,0xdb,0xb3,0xd8,0x6d,0x31,0x60,0xe5,0xcd, -0xa2,0xb8,0x04,0x82,0xd5,0x3c,0xaa,0xd2,0x91,0x25,0x1a,0x57, -0x81,0x1d,0x6a,0x56,0xae,0x1e,0x06,0x8b,0x42,0xa2,0x2e,0xdf, -0x78,0x7c,0x4b,0x3c,0x9a,0x78,0xa2,0xbc,0xd3,0x27,0xd7,0xcc, -0xc9,0xef,0xee,0xe6,0xc3,0xdd,0xa0,0x1e,0x35,0x58,0x04,0xb9, -0x42,0xda,0x83,0x00,0xce,0xcb,0x7e,0xa4,0x3c,0xb0,0xa7,0x39, -0x08,0x38,0x51,0x86,0x68,0x4b,0xba,0xcf,0xd3,0x30,0xf8,0x65, -0x4f,0x2f,0xa7,0xcb,0x13,0x62,0x01,0xcb,0x3f,0xc2,0xbe,0x58, -0xa5,0x2d,0x07,0x0c,0xc4,0xc2,0x53,0xac,0xfa,0x93,0x8c,0xc7, -0xe4,0x7b,0x49,0x16,0x20,0xfc,0xaf,0xad,0x81,0xb1,0x17,0x73, -0x41,0x6d,0x4b,0xb3,0x19,0xf4,0x40,0xef,0xba,0x73,0xca,0x5a, -0xff,0x35,0xf6,0x05,0xb3,0x52,0xcf,0x5c,0x5f,0xf4,0xd4,0xf5, -0x67,0x39,0xb1,0xda,0x57,0x1d,0x7f,0x0f,0x45,0x9c,0xff,0x15, -0x04,0x11,0x20,0xd2,0xb3,0x70,0xfc,0xe1,0x0d,0x61,0xce,0xbb, -0x53,0x00,0x26,0x7e,0x46,0xe5,0xb1,0x19,0xa0,0xcc,0x1a,0x33, -0x7e,0x93,0xdf,0xbe,0x18,0x2f,0x3c,0x02,0x32,0x09,0x75,0x6d, -0x17,0x4a,0x02,0x96,0xc3,0x8e,0x46,0xed,0x06,0xa3,0x49,0x16, -0x35,0xc7,0x41,0x2e,0x71,0xd6,0x30,0x52,0xea,0x0a,0xb8,0x85, -0xf2,0x74,0xd9,0x85,0xa3,0x69,0x6f,0x97,0x1d,0x12,0xe9,0x72, -0x84,0xfe,0x5d,0x1a,0xd2,0x50,0x07,0x54,0x1d,0xab,0x37,0xf2, -0x21,0x7e,0x1a,0x08,0x12,0x34,0xcf,0x8a,0x1d,0xeb,0x03,0x92, -0x29,0xd1,0x48,0x1c,0xbf,0x81,0xce,0x11,0xe6,0xdd,0x73,0x51, -0x6b,0xf1,0x1d,0x81,0x9e,0xca,0xd3,0xd1,0x3d,0xed,0x8d,0xdf, -0x6a,0xca,0xa3,0xaa,0xd0,0xd9,0x1d,0x53,0x00,0x6d,0x37,0xda, -0x86,0x24,0x1b,0xf3,0xe1,0x3b,0x6b,0xae,0xa6,0x1d,0xe9,0x1d, -0xbb,0xc3,0x77,0x9a,0x07,0x76,0x55,0x9b,0x65,0x7e,0x3f,0x79, -0xbd,0x1d,0x6a,0xec,0x89,0x44,0x30,0xd2,0x09,0x93,0x5b,0x0f, -0x71,0x56,0x3d,0x62,0x0a,0xf5,0x0c,0xee,0xff,0xa5,0x37,0xcd, -0x8e,0xbd,0xe9,0x56,0xf6,0x8c,0xef,0xe9,0x62,0x67,0xe9,0x42, -0xef,0x7d,0xf4,0xd3,0x87,0x8c,0xec,0xc8,0xd0,0xf6,0xfe,0x24, -0xf9,0x5a,0x92,0x7f,0x07,0x6e,0x36,0xda,0x6b,0x46,0x74,0xb3, -0x92,0x34,0x6f,0x77,0x9e,0xa4,0xcd,0xdf,0x9e,0xc7,0x79,0xdd, -0xd7,0x8a,0x69,0x54,0x63,0x66,0x3e,0xc7,0xf4,0xc5,0x31,0x61, -0x25,0x45,0x92,0xff,0xda,0xe0,0x7d,0x11,0x05,0x7d,0xd8,0x8f, -0xc1,0xf1,0x50,0xb8,0xba,0x05,0xa1,0x9f,0x9f,0xcf,0x77,0x53, -0x88,0xbd,0x9b,0x16,0xf6,0x94,0x00,0x44,0x72,0xc0,0x80,0x52, -0xbe,0x34,0xf7,0x4e,0x7b,0x41,0x73,0xe5,0x59,0x6e,0x4e,0x7b, -0x45,0x49,0xd1,0x4e,0x2e,0xb2,0xc2,0x26,0xb3,0x3d,0x05,0x54, -0xaa,0xd3,0x47,0xbc,0x70,0x5b,0x0f,0x02,0x4a,0x5f,0xd2,0x12, -0xd5,0xa4,0x8e,0x5a,0xc9,0x45,0x64,0x1d,0xf6,0x4e,0x5f,0xab, -0x06,0x18,0xc1,0xc0,0x82,0xba,0x98,0x9f,0xca,0x60,0xc1,0xb3, -0xd2,0x15,0x67,0x5b,0x05,0x33,0x4b,0xb9,0xdc,0x8d,0x58,0xe0, -0x65,0xde,0x0e,0xe4,0x47,0xb2,0x85,0x3a,0x90,0xa3,0x2c,0xce, -0xe7,0x05,0xfc,0x16,0x33,0xe0,0x8b,0x7c,0xae,0x0d,0x33,0x83, -0x4f,0x5c,0xf8,0x99,0x51,0xa3,0x84,0xac,0x5e,0x9c,0x3c,0x48, -0x6b,0x30,0xc6,0x3d,0xf9,0xfe,0x59,0x3e,0xfe,0xe3,0xdb,0xc1, -0x29,0xf4,0x46,0x9e,0x21,0x81,0xa6,0xe2,0xd0,0x78,0x9c,0xdf, -0xf0,0x41,0x51,0x8c,0xe3,0x10,0xc6,0xed,0x60,0x64,0xa0,0xe2, -0x4e,0xa0,0xe6,0x0f,0xa6,0x54,0x8f,0xfb,0xfa,0x58,0xe2,0x7d, -0xe5,0x98,0x64,0xff,0xa1,0xa9,0x35,0x90,0xc9,0xda,0x58,0x7c, -0xa1,0x6f,0x3b,0x71,0x85,0x6d,0xdd,0x96,0x0d,0x8e,0x39,0xc8, -0x9f,0x27,0xd4,0x68,0x48,0x49,0x0b,0x39,0x0b,0xe1,0x38,0xc5, -0x14,0xad,0x84,0x82,0x5d,0xa0,0x3f,0xa0,0x29,0x2d,0x47,0xd9, -0x70,0x6a,0x8e,0x4f,0xf4,0xaa,0x79,0x41,0x71,0x85,0x1b,0x47, -0x6e,0x37,0x18,0x8d,0xae,0x37,0x1f,0xc8,0x5b,0x6e,0x14,0x7a, -0x53,0x04,0xa5,0xc2,0xfc,0x40,0x5f,0x97,0xe3,0xf2,0x0f,0xc6, -0xb9,0xce,0x00,0x17,0x02,0x5c,0xb4,0x81,0x39,0xc1,0xd6,0x5b, -0x2a,0xca,0xfb,0x53,0x91,0x0c,0x19,0xfe,0x5b,0x30,0xeb,0x92, -0x33,0xc0,0x6d,0x62,0x3d,0xd6,0xb1,0x75,0xb5,0x5b,0x69,0x9c, -0xaf,0x82,0x2c,0x73,0xd8,0x12,0x25,0x10,0xe6,0xc9,0x55,0x1d, -0x90,0xac,0x14,0x09,0xb5,0x16,0x6d,0x80,0x4c,0xd7,0x18,0x77, -0x7d,0x6d,0x15,0xfb,0xf2,0x4b,0x98,0x2f,0x30,0x72,0x95,0x14, -0xf2,0xf5,0x07,0x73,0x94,0x46,0x29,0x1d,0xd2,0xeb,0x78,0xbe, -0x62,0xdf,0xb0,0xe6,0x95,0x8b,0xde,0x45,0x64,0xc4,0x71,0x32, -0x52,0xec,0x9c,0x5c,0x48,0x32,0x95,0x85,0xf2,0x18,0x8a,0xa2, -0x4f,0xab,0xa4,0x28,0x22,0x37,0x6d,0xac,0xaf,0x18,0xca,0xa8, -0x4c,0x29,0xaf,0xb0,0xe5,0x29,0xc2,0x11,0xd7,0x46,0xb2,0x69, -0x83,0x61,0xe4,0xc5,0xbc,0xe7,0x25,0xed,0x5b,0x1f,0x42,0xdb, -0xd8,0xf4,0x47,0x54,0x8b,0x94,0x7c,0x24,0xbf,0x3c,0x79,0x6e, -0x0a,0x49,0x61,0x27,0x83,0x2f,0x24,0xe1,0x16,0x2b,0xdd,0xfe, -0x15,0x8f,0xca,0xac,0xa3,0xdc,0x1a,0x57,0x0d,0xed,0x6c,0xea, -0x39,0x5f,0x38,0xf3,0xae,0xb7,0xc8,0x09,0xd7,0x27,0x53,0xe6, -0x07,0x1d,0x00,0x3d,0xb3,0x9d,0xa9,0xbd,0xea,0xb9,0x3d,0xae, -0xc5,0x4c,0x9e,0x1c,0x4f,0xef,0x9b,0x3b,0x46,0xe1,0xa8,0xc6, -0x1b,0xd9,0xee,0x46,0x5c,0xdc,0xbe,0x7e,0x28,0x51,0xf1,0x96, -0x9a,0xb1,0x23,0x8c,0x77,0xce,0xb7,0xad,0xc5,0x33,0x9e,0x2e, -0x3f,0xfa,0x72,0x94,0xcc,0x70,0x89,0x9d,0xa8,0x06,0x9d,0x6b, -0x32,0x6f,0x39,0x70,0x49,0xe4,0x15,0x06,0xe3,0x17,0x10,0x24, -0x81,0x8e,0x31,0x08,0x72,0x11,0x10,0x60,0x91,0x0a,0x01,0x14, -0x8f,0xf0,0x61,0x69,0xe7,0x3e,0xd8,0xb0,0xa7,0xa1,0xb1,0x5e, -0x04,0xd5,0x29,0x88,0xf9,0xba,0x8a,0x5e,0x1a,0xba,0x74,0x67, -0x99,0xa9,0x1d,0xe7,0x8c,0x83,0x55,0x70,0x61,0xaf,0x2e,0xc7, -0x61,0xc1,0xae,0x81,0x40,0xe7,0xf0,0x92,0x5e,0x6c,0xf9,0x91, -0xed,0xc4,0x00,0x8b,0x2c,0xcb,0x80,0xb6,0xa6,0x03,0xcd,0x64, -0xa4,0xe5,0xf4,0x66,0x0a,0x43,0xca,0x95,0x32,0xbd,0x88,0x6d, -0xc6,0x4c,0xc5,0x2c,0xce,0xd3,0x4c,0xee,0xcf,0xb8,0x0b,0x3f, -0xa0,0x46,0x68,0xaa,0x38,0x1f,0x8f,0x2a,0x10,0x27,0x1f,0x43, -0xba,0xb5,0xee,0xe7,0x6d,0x9d,0xcb,0x02,0x8d,0x7b,0x11,0x52, -0x86,0x75,0x11,0xee,0x0c,0xeb,0xc3,0x3c,0x22,0xe7,0x24,0xce, -0x28,0x53,0xd5,0x4b,0xfd,0x3f,0xba,0x3d,0x9f,0x25,0x74,0x07, -0x73,0x46,0x67,0x92,0x1b,0xe6,0x2d,0x7c,0x2d,0xfb,0x25,0xb7, -0xa9,0xcc,0x90,0x61,0x1e,0x40,0xee,0x07,0x77,0x55,0x34,0xcc, -0x00,0x3d,0x34,0x7c,0x4e,0xc1,0xbf,0x38,0xc0,0xd0,0x44,0x30, -0x98,0x79,0x5f,0xe3,0xc5,0x0b,0xa0,0x80,0xf5,0xbc,0x07,0x7f, -0x25,0x07,0xc4,0xbb,0x77,0x2d,0xc8,0xeb,0xc4,0xd6,0xe0,0x3c, -0xe4,0x78,0x6b,0x5c,0x9b,0x69,0x71,0x6e,0xa1,0xf6,0x3d,0x4e, -0x22,0xc6,0xb2,0x70,0x7f,0xff,0xb4,0xe3,0x13,0xfb,0xa1,0xef, -0xf2,0x4b,0x4b,0xc4,0xbd,0xac,0xc8,0x08,0x0c,0x57,0x87,0xfd, -0xbe,0x75,0x76,0x37,0x9c,0x00,0x65,0xaf,0xee,0xa1,0x76,0xde, -0xea,0xde,0x55,0xfb,0x04,0x70,0xb7,0xfa,0x01,0x62,0x80,0x93, -0xf1,0xd0,0x02,0xa5,0x57,0x7b,0xdc,0x99,0x83,0x4f,0x4c,0xb4, -0x6c,0xb6,0x35,0x90,0x49,0x41,0xf1,0x5d,0x8e,0xfe,0x8c,0xc0, -0xd7,0x08,0x71,0xa7,0x5e,0xc9,0xdc,0x28,0x78,0xfd,0xc2,0x73, -0x79,0xba,0xcf,0x0f,0xe5,0xec,0x39,0x18,0x07,0xb3,0x60,0x8c, -0x76,0x36,0x90,0x00,0xef,0x30,0xbc,0x6d,0x30,0xf8,0x1e,0x83, -0x8f,0x18,0x9c,0x04,0x13,0x08,0x7e,0x79,0xaa,0x29,0x7e,0x82, -0xf1,0x7f,0xfc,0xf6,0xe6,0x2f,0x29,0x0c,0xe7,0xd4,0x20,0xbc, -0x79,0xda,0x33,0xc6,0x4c,0xd7,0xbb,0xa1,0x4d,0xe3,0x94,0xc8, -0x21,0x01,0x44,0x4d,0xf9,0x0d,0x1a,0x5e,0xc9,0xc0,0x55,0x37, -0x8b,0x1f,0x98,0xa5,0x83,0x58,0x56,0xc1,0x1d,0x0c,0x37,0x9f, -0xeb,0x6a,0x1f,0x06,0x41,0x84,0xd2,0x24,0x09,0x1e,0x83,0x28, -0xd1,0x72,0x3d,0x14,0x61,0x0f,0x0a,0x05,0x54,0x24,0x67,0xc4, -0x12,0x31,0x16,0x9c,0xc0,0x44,0x68,0x32,0x10,0x44,0x2b,0xbc, -0x2b,0x0f,0xf5,0x74,0xd0,0x5a,0x8f,0xe9,0xb8,0x3b,0xac,0x38, -0xb8,0x5a,0x7d,0xc6,0x91,0xf3,0x20,0x98,0x01,0xd1,0xb1,0x80, -0xdc,0x1c,0xcb,0xc3,0x36,0x87,0x95,0x85,0xd7,0x14,0x8a,0xd1, -0x29,0x0c,0x6f,0xac,0xb2,0x53,0xa8,0x56,0x44,0x01,0x17,0x94, -0x8e,0xa7,0x78,0x48,0x4c,0x39,0x36,0x46,0x1d,0x70,0x20,0x00, -0xf0,0x96,0x56,0x1a,0x61,0xfe,0xe4,0x0a,0x29,0x91,0x37,0x6a, -0x53,0xeb,0x65,0xca,0xc9,0x0b,0xcd,0xb8,0x4a,0x7f,0xe4,0xab, -0x6e,0x06,0x2f,0x62,0x9d,0x39,0x07,0x39,0xbe,0x55,0x6d,0x91, -0x45,0x6d,0xec,0x98,0x21,0x78,0x45,0xd4,0x9a,0x6e,0xd4,0xde, -0x97,0xad,0xe5,0x86,0x24,0x5c,0xe6,0x51,0x08,0x7a,0xc9,0x6c, -0x2c,0x2e,0x5a,0xd3,0x91,0x6b,0x89,0xf4,0xee,0xb5,0x78,0x8d, -0x49,0x52,0x0f,0x4b,0x60,0x45,0xc8,0x36,0xe7,0xd8,0xea,0xd6, -0x62,0xa4,0xbd,0xe8,0x53,0xbc,0xd5,0xc3,0x07,0x74,0x2c,0x2e, -0x8f,0x30,0xe8,0xd5,0xe7,0xec,0x17,0x47,0x26,0x82,0x58,0xda, -0x54,0xe2,0x7c,0x5d,0x3d,0x64,0x71,0x6c,0x32,0xc4,0x41,0x60, -0x88,0x21,0xa0,0x39,0x7f,0x00,0x62,0x56,0xd1,0xd9,0x85,0xa6, -0xdc,0x1d,0xb6,0x74,0x27,0x4d,0xd3,0x00,0xaf,0xe0,0x00,0xc1, -0x06,0x85,0x98,0xd9,0xf4,0xdd,0x4f,0xe1,0x62,0x79,0x5a,0xfe, -0xba,0x8a,0xe7,0xd1,0x4f,0xef,0xa7,0x8f,0xbc,0xdc,0x3c,0xd4, -0x5b,0x49,0xcd,0x5a,0xc9,0x32,0x2e,0x20,0xe5,0xda,0x4f,0xe1, -0x6a,0xec,0x6b,0x7a,0xcc,0x3b,0x9b,0x86,0x37,0xd1,0xbf,0xa7, -0x02,0x0d,0x05,0xd2,0x08,0x9f,0xc0,0x00,0x58,0x2e,0x21,0x00, -0x3f,0x10,0x3a,0xe2,0x4d,0x95,0x18,0xd8,0x43,0x00,0x75,0x88, -0x3f,0xc0,0x17,0x5f,0x82,0x5a,0x3e,0xbd,0xb9,0x7d,0xf3,0x57, -0xf8,0xc3,0x5f,0x89,0x78,0x2b,0x11,0x6f,0x83,0x8b,0xe0,0x3d, -0x66,0xd3,0xe5,0xf2,0x3c,0x39,0x2f,0x8f,0xe7,0xe5,0xfe,0xbc, -0x6c,0xce,0x9c,0x91,0x3f,0x6f,0xa7,0x8f,0xa9,0xad,0xba,0x63, -0x5e,0x45,0x79,0x49,0x57,0xc1,0x9e,0xa4,0x39,0xd4,0xcb,0x0a, -0x05,0x7e,0xaa,0x91,0xde,0x75,0x60,0x17,0x03,0xdb,0xd3,0x9a, -0x2c,0x81,0x68,0xdc,0xc8,0x05,0x51,0x22,0x30,0x16,0xa1,0xe6, -0x12,0x49,0x03,0x4b,0x6a,0x6e,0x9e,0x48,0xd6,0x79,0x89,0x85, -0x70,0x9a,0xfd,0x25,0x30,0xed,0xb8,0x0e,0xcc,0xca,0x15,0x40, -0xe7,0x3f,0x89,0x1c,0x33,0x87,0x68,0x78,0x81,0x19,0xac,0xd7, -0x87,0x38,0x9b,0x04,0x29,0x36,0x82,0x91,0x18,0x42,0x7a,0x4b, -0x46,0x48,0x1b,0xbb,0xfc,0x90,0x68,0xe9,0x49,0xbd,0xd9,0x9c, -0x9c,0x63,0x97,0x8c,0x3e,0xce,0x08,0x2d,0x1b,0x38,0xe4,0x00, -0x00,0xc3,0x59,0xe1,0xc4,0xfd,0x36,0x20,0x8e,0x90,0x2f,0x62, -0x61,0x19,0x23,0x88,0x84,0x0e,0x5c,0xc2,0x48,0x29,0xc1,0x24, -0x5e,0xee,0x43,0xbc,0x6c,0x36,0x0b,0x15,0x1e,0x84,0x2e,0x68, -0x87,0x28,0x8a,0xe6,0x93,0x09,0xda,0x56,0x28,0x16,0x61,0xa2, -0x08,0xa0,0x27,0xd4,0x16,0xa6,0x22,0x83,0xc8,0x1c,0x98,0x16, -0x46,0x40,0x24,0x78,0xb4,0x11,0x3c,0x24,0x29,0x30,0x71,0xac, -0x29,0x31,0xb0,0xb4,0xb4,0xb4,0x76,0x01,0x37,0x44,0x4c,0x95, -0xd2,0xb5,0x20,0xfd,0xc1,0x63,0x46,0x1e,0x16,0xc3,0x6a,0x37, -0xc0,0x56,0x55,0x5e,0x6c,0x71,0x97,0x49,0xfb,0x0f,0x1d,0x5c, -0x43,0x52,0x56,0x9f,0x2f,0xd1,0x33,0x96,0x29,0x99,0x83,0x0b, -0xfc,0x89,0x4c,0x3f,0xcb,0xad,0xd3,0x09,0x50,0x91,0x4d,0x26, -0xf0,0xf9,0xdc,0x79,0x5d,0x10,0xd9,0x05,0x04,0x79,0xb5,0xf7, -0x74,0x21,0x22,0x1b,0xc0,0xdb,0xe6,0xb5,0xc9,0x04,0xd6,0x9a, -0xa9,0x5b,0xa0,0x0a,0xf0,0xf5,0x01,0xa3,0xd3,0xe6,0xf8,0xfd, -0x99,0x87,0x10,0x0d,0x01,0x23,0xd3,0x7b,0x6a,0x83,0xb6,0x07, -0xe0,0xcf,0x75,0x20,0xe4,0x46,0x97,0x7b,0x6c,0x89,0x10,0xa4, -0x34,0xef,0x39,0x94,0x6a,0x10,0x11,0xf6,0x27,0x2b,0x06,0x5d, -0x9d,0x6e,0x6c,0xcf,0x58,0x10,0x5f,0x97,0xbe,0xe6,0xf1,0xd7, -0x1d,0xe8,0xde,0xa3,0x7c,0x9c,0xc3,0x91,0xc5,0x92,0xd7,0x6c, -0x09,0x6b,0x7e,0x2d,0xda,0xe8,0xee,0xc0,0x15,0x83,0x95,0xc0, -0xb9,0xde,0xef,0x85,0x20,0x51,0x44,0x64,0x29,0xfc,0x94,0x77, -0x66,0x2b,0x6d,0x37,0x3e,0x08,0x40,0x10,0xa1,0xb9,0xee,0xa5, -0x33,0xdb,0xfa,0x0f,0x7e,0xac,0x09,0x7c,0xee,0xe7,0x25,0xc3, -0x61,0x2a,0x53,0x63,0x88,0xd7,0x1e,0xc9,0x13,0x7b,0x35,0x3e, -0x0c,0x4d,0xcf,0x5a,0x26,0xd0,0x5b,0xff,0x41,0xc7,0x58,0x08, -0x9d,0x3e,0x55,0x1b,0x14,0xb2,0x68,0x3e,0xc7,0xcf,0x01,0x87, -0x2b,0x69,0x26,0x9c,0xd2,0x59,0xa8,0xf1,0x3e,0x21,0x6a,0x07, -0xce,0xed,0x53,0xaf,0x9d,0x3a,0x71,0xf5,0xe0,0x4d,0x9c,0xf6, -0x7e,0x0a,0xc4,0xa8,0x7e,0xe4,0x78,0x8a,0xd6,0x4e,0x9c,0x42, -0xe3,0x28,0x43,0x94,0xe3,0xb1,0xd2,0xc8,0xff,0xcf,0xe7,0xdb, -0x91,0xbb,0x20,0x3f,0x9f,0xf9,0x86,0x59,0x3a,0xc7,0x76,0xbb, -0xea,0xce,0xba,0xb4,0x28,0xb9,0x98,0x66,0x61,0xf3,0x36,0x54, -0x18,0x07,0x00,0x36,0x00,0xe1,0xfe,0xdf,0xd5,0xda,0xdb,0x1e, -0x71,0xc8,0xf0,0x2a,0xec,0x35,0x89,0x53,0x75,0x51,0xbb,0xc4, -0x32,0xec,0x94,0x93,0x3c,0x53,0x6f,0xcf,0xba,0x05,0x39,0x8c, -0x81,0x7b,0xb3,0xbc,0x56,0x69,0x1d,0xf7,0x6b,0x01,0x4b,0xb9, -0xa8,0x7b,0x58,0x02,0xa3,0xf4,0xbf,0x01,0x00,0x00,0xff,0xff, -0xe9,0x76,0xa5,0x35,0xdb,0xfb,0x08,0x00, +0xcc,0xbd,0xeb,0x76,0x1b,0xb7,0x92,0x28,0xfc,0xfb,0x9b,0xa7, +0xb0,0xfa,0x8c,0x95,0x6e,0xb1,0x49,0x75,0x53,0x96,0x6c,0x93, +0x6e,0x71,0x25,0xce,0xce,0x97,0xcc,0xc9,0xed,0x8b,0xb3,0xb3, +0xed,0x30,0xdc,0x5a,0xbc,0xb4,0xa4,0xb6,0x78,0x0b,0x9b,0x94, +0xba,0x63,0x71,0xad,0x39,0x6b,0xe6,0x95,0xf6,0xaf,0xef,0xdf, +0x3c,0xc0,0xe4,0x95,0x4e,0x5d,0x70,0xed,0x0b,0x45,0x39,0x39, +0x33,0x67,0x25,0x16,0x1b,0x40,0xa1,0x50,0x28,0x14,0x0a,0x05, +0xa0,0x00,0xf0,0xab,0x9f,0xa6,0x2f,0x78,0x40,0x2e,0x48,0x76, +0x5c,0x48,0x4e,0x56,0x66,0x5c,0x9a,0xcc,0x09,0x0d,0x7b,0x82, +0x24,0x73,0x17,0x43,0xa8,0x1f,0x52,0x1d,0x0d,0x01,0x11,0x0d, +0x00,0x88,0x58,0x01,0x43,0x80,0x60,0x55,0x24,0x82,0x52,0x24, +0x41,0x86,0x26,0x64,0x28,0x20,0x43,0x13,0x52,0xdf,0x24,0x66, +0xf8,0xab,0x68,0xaf,0x14,0x17,0x0f,0x14,0x61,0x9e,0x23,0x41, +0xa7,0x77,0xb4,0x6e,0xc8,0xc8,0xe0,0x88,0x0b,0x69,0x32,0x55, +0x47,0x02,0x52,0x90,0x0e,0x90,0x92,0x5e,0x01,0xd7,0x10,0xb9, +0x0a,0x70,0x8a,0xd7,0x57,0xf8,0x5a,0xfc,0x78,0x53,0xa3,0x8d, +0x54,0xaa,0x1a,0x68,0x45,0x37,0x2c,0x1c,0x2e,0xb2,0xbb,0x23, +0x2e,0x00,0xa4,0xae,0x79,0x4b,0xa0,0x88,0x31,0x76,0xb7,0xf9, +0x8d,0x70,0xfd,0x62,0x1e,0xd8,0xf0,0x9f,0xbf,0xf5,0x8e,0x3e, +0x7f,0xeb,0x83,0x65,0x0a,0x5f,0x74,0x38,0xf9,0xad,0x27,0xef, +0x9e,0xaa,0xc8,0x01,0x16,0xfd,0xe7,0xef,0x20,0xc7,0x3b,0xff, +0x1d,0xe4,0x78,0xc7,0x39,0xde,0x79,0xbb,0xb2,0x64,0x30,0x09, +0xc0,0x2e,0x94,0xa1,0x4b,0x05,0x7c,0xb5,0x2e,0x93,0x29,0x6e, +0xc0,0x56,0x74,0x54,0xb5,0xbc,0x98,0x3d,0x05,0x6a,0xce,0xff, +0xf3,0x1f,0xf2,0xe1,0xf3,0x9d,0x05,0xe4,0x50,0x40,0x0e,0x05, +0xe4,0xe8,0xb0,0x01,0x5f,0xa5,0x02,0xf2,0x72,0x01,0xf9,0x53, +0x20,0x5e,0x17,0x00,0x0a,0x91,0xf4,0x18,0x50,0x98,0x05,0xc8, +0x8c,0xb7,0xf4,0x98,0x5f,0x1e,0x60,0x35,0xdf,0x05,0x3e,0xa8, +0x4c,0xff,0xad,0xff,0x0e,0x9f,0xfd,0xc5,0x3b,0xbe,0xf3,0x08, +0x6a,0xf3,0xf9,0xdb,0xe8,0x65,0x00,0x3c,0x88,0x60,0xa0,0x36, +0x34,0x5c,0x5b,0xcf,0xbf,0x55,0x43,0xd2,0xea,0x4c,0x95,0xc7, +0x81,0x68,0x22,0xfb,0x02,0xaf,0x8a,0x41,0x59,0x08,0x40,0x5d, +0xdb,0x9b,0xa3,0x25,0xde,0x14,0xa8,0x0b,0x5e,0x6c,0xd6,0x58, +0x46,0xb9,0xe8,0x9d,0x9a,0xfd,0x2d,0x48,0x86,0xe2,0xf8,0x3b, +0xf7,0x5d,0xe8,0xb5,0x52,0xbc,0x27,0xdf,0x85,0x9e,0x05,0x69, +0xa1,0x67,0x9c,0xe2,0x55,0x09,0x00,0x17,0x54,0x25,0x78,0x30, +0x46,0x18,0x14,0xc5,0xfb,0x3f,0x5c,0xa6,0x33,0xcd,0x86,0xef, +0x17,0x2b,0xb1,0x29,0x71,0xe1,0xe1,0x09,0x6d,0x23,0xd8,0x31, +0xe0,0x8c,0x04,0x8b,0x0f,0x06,0x82,0xfd,0x74,0xf1,0x5b,0xc3, +0x71,0xe6,0xad,0xe1,0x2c,0xf3,0xce,0x70,0xa2,0x79,0xa7,0x1d, +0x67,0x40,0x60,0xce,0xdf,0xa2,0xa7,0xdd,0x45,0x04,0xa2,0x03, +0x99,0x51,0x86,0x42,0x98,0xee,0x40,0x86,0xf3,0x77,0x9c,0x00, +0x72,0x04,0xb9,0x51,0xa0,0x28,0x41,0x13,0xa7,0x77,0xf4,0xcc, +0xeb,0x49,0xfa,0xfd,0xb7,0x98,0x01,0xe6,0x91,0x6f,0x31,0x0b, +0xce,0x12,0x2b,0x2b,0xba,0x5f,0x7d,0x4c,0x47,0x20,0xcb,0xf9, +0xc7,0xa8,0x8f,0xe1,0x08,0x04,0xdd,0xe0,0x3c,0x63,0xb2,0xf1, +0x75,0xcb,0x20,0xc2,0x9e,0x41,0x64,0xe7,0xc1,0x79,0xce,0x09, +0xf8,0xe0,0x65,0x10,0x61,0x37,0xd9,0xb3,0x3e,0x45,0x3f,0x21, +0x23,0x0b,0x5e,0xc3,0xf5,0x31,0x32,0xf1,0x06,0xf2,0x29,0x89, +0x10,0x81,0xa2,0x3c,0x50,0x74,0x59,0x1a,0xde,0xec,0x5b,0xa4, +0x0b,0x3d,0x9d,0xb7,0x16,0xa1,0xb3,0xf3,0xc3,0x80,0x0a,0x15, +0x54,0x0b,0xb4,0xe7,0xe7,0xef,0xca,0x8d,0xb3,0x3f,0xfa,0x89, +0xdc,0xb9,0x9c,0xe4,0x15,0xe8,0x41,0xd3,0x4c,0x72,0x0b,0xfd, +0x23,0xad,0x0a,0xd3,0x3c,0xf0,0x95,0x23,0xa8,0x42,0xf7,0xd6, +0xe5,0x87,0x4b,0xd1,0x97,0x3c,0x2f,0xa5,0xbe,0x73,0xf9,0x71, +0x53,0xdd,0x90,0xfe,0xdb,0x0a,0x14,0xef,0x48,0x51,0x22,0x8a, +0x77,0x15,0x28,0xde,0x92,0x4a,0x35,0x50,0x18,0xf5,0x33,0x4c, +0x93,0xea,0xee,0xde,0xef,0xd3,0xd1,0xa5,0xe6,0xcb,0x80,0xf7, +0x1d,0x31,0xf0,0x32,0x10,0x3b,0x8d,0x66,0x7f,0x97,0x80,0x2f, +0x28,0x91,0x01,0x5f,0x50,0xa6,0x81,0x39,0xf2,0xc6,0xc3,0xf5, +0xa7,0xab,0x71,0xcd,0xc0,0xcb,0x89,0x45,0x25,0x59,0xa7,0x76, +0xfb,0xe9,0x62,0xb3,0x1a,0xc7,0x17,0xf7,0xf7,0xfc,0x51,0x3d, +0x25,0x10,0xaf,0x5a,0x03,0x14,0x7f,0x54,0x43,0x09,0x43,0x5a, +0x60,0x94,0x79,0x7c,0x0e,0x23,0x4f,0xf9,0x4b,0x3e,0x91,0x8d, +0x1e,0x93,0xf4,0xa5,0xc7,0x1a,0xa6,0xbd,0xc2,0x8e,0x2b,0xfa, +0xb7,0x49,0x08,0xf7,0x4f,0x24,0x9f,0xba,0x97,0xa0,0x40,0xd0, +0xbc,0x97,0x70,0x0a,0x58,0x59,0xd1,0x8b,0xea,0x25,0x15,0x9c, +0xe0,0x74,0x2e,0x54,0x09,0x5e,0x87,0xa1,0x8d,0x32,0x05,0x5b, +0xf6,0x2a,0x53,0xc0,0x2a,0x26,0xef,0x59,0x26,0x43,0x1b,0x65, +0x56,0xf4,0xc3,0x5d,0x7a,0x8b,0x73,0x75,0x02,0x8d,0x41,0x49, +0xa5,0x69,0xf8,0xeb,0x2d,0x30,0xb3,0xc1,0x8b,0x4b,0x2e,0xe6, +0x1c,0x90,0x01,0x0b,0x73,0x4a,0x11,0x69,0x9b,0xe0,0x8c,0xab, +0x00,0x29,0x22,0xed,0x69,0xbb,0x22,0x4d,0x5c,0x34,0xf4,0xc0, +0xe2,0x2c,0x43,0x3d,0x7c,0x30,0x59,0xde,0x6e,0x54,0xcc,0x66, +0xed,0xfb,0xab,0x58,0x1b,0xac,0xde,0x9b,0xba,0xbc,0x37,0xaf, +0x73,0x55,0x6e,0xe0,0x7e,0xd4,0xa6,0xf2,0xf0,0xb7,0x64,0xb6, +0x59,0x5f,0x0f,0xa7,0xda,0x45,0xb5,0x98,0x62,0xda,0x70,0xe9, +0x7f,0xfe,0xff,0x64,0xfe,0xdb,0xb6,0x27,0xcd,0x35,0xda,0xc7, +0x2e,0x4d,0x11,0x24,0x04,0xde,0x77,0x23,0x33,0xfe,0xfe,0xbf, +0x54,0x86,0x36,0x5f,0x45,0x48,0x0e,0xf4,0xbf,0xff,0x2f,0xdc, +0x5b,0xf7,0xba,0x6a,0x55,0xb0,0x4c,0xcc,0x23,0x96,0xe7,0xca, +0xb9,0x0b,0x2b,0x74,0x65,0x00,0xeb,0xf4,0xb8,0x91,0x9a,0xb0, +0x36,0x59,0xef,0xc9,0x0b,0xbe,0x51,0x99,0xeb,0x85,0xf3,0x32, +0x23,0x4d,0xad,0x0c,0x1c,0x1e,0x8e,0x8f,0xd5,0x6c,0x6e,0xcc, +0x8f,0x44,0xa9,0x3b,0xeb,0x2b,0x39,0xa0,0x88,0xf8,0x48,0x1e, +0xc8,0xfc,0x3b,0xb8,0x20,0x41,0x6c,0x6f,0xe5,0xd7,0x8b,0xf9, +0x25,0x9f,0x77,0x7e,0x84,0xb7,0xb2,0xca,0x54,0xe5,0xad,0xac, +0x12,0x4b,0x6e,0xd1,0x0f,0x57,0xb2,0xd2,0x31,0xba,0xa6,0x6e, +0xc5,0x64,0xb3,0x7d,0xaf,0xe6,0x8b,0x19,0x26,0x3f,0x4e,0xbe, +0xc3,0x63,0x1d,0xb9,0xf5,0xd5,0x24,0x5b,0xb7,0x98,0x42,0xfb, +0x88,0x56,0x92,0x79,0x0a,0xd4,0xcb,0x68,0xd9,0x37,0x67,0xf1, +0x6a,0x8c,0x6b,0xd7,0xa5,0x05,0x4e,0x5e,0x03,0xa3,0x42,0xfa, +0x99,0x2f,0x7b,0x14,0x90,0xc5,0xf3,0xc7,0x38,0xa3,0xb9,0x5f, +0x13,0xea,0x76,0xf9,0xfb,0xbf,0x0e,0xb4,0x2f,0xba,0x44,0x58, +0x4f,0xab,0x84,0xf8,0xbe,0x44,0xb3,0x4c,0x29,0xd0,0x2c,0xa3, +0x4d,0x4e,0x2f,0x56,0xeb,0xeb,0x05,0xd8,0x3d,0xcb,0xeb,0x9d, +0xdc,0xd6,0x3c,0x96,0x8c,0x85,0xae,0xa1,0x19,0x6b,0x61,0x79, +0x04,0x73,0xcd,0x7c,0x05,0x62,0xcd,0x24,0xfb,0xa6,0x1e,0xd0, +0xc0,0x7b,0x50,0x5c,0x29,0x1f,0xb6,0xd6,0xdb,0xad,0xf4,0xb0, +0x89,0x20,0xda,0x50,0x79,0x76,0xd1,0x8f,0xa8,0xa6,0x95,0xb1, +0x50,0x4f,0x2b,0xcd,0xf2,0x00,0xa1,0x69,0xec,0x37,0xfb,0xc8, +0x95,0xb1,0x9e,0x84,0x76,0x5a,0x32,0xbf,0x76,0x33,0x71,0xd4, +0x18,0x55,0x5c,0xce,0xdb,0xc0,0xa4,0xc6,0x95,0x5a,0xcb,0x3d, +0xdc,0x19,0x85,0x64,0x84,0xf5,0x0c,0xc1,0x2b,0x97,0xfd,0x31, +0x22,0x58,0xc6,0x52,0xa8,0x77,0x19,0x40,0x8c,0xdb,0x33,0xbe, +0x6b,0xab,0x95,0xde,0x5e,0xe9,0xaf,0x56,0xf5,0x1a,0x02,0xd2, +0x01,0xa9,0x17,0x98,0x6a,0x3d,0x2c,0xa2,0x86,0x73,0x99,0xaa, +0x17,0x30,0x17,0x2b,0x7d,0x0b,0x1e,0xa6,0x0c,0xe9,0x92,0x68, +0x09,0xf7,0x35,0xc5,0xf8,0x0e,0xa7,0x34,0xc7,0xd3,0x45,0x1a, +0x4f,0x9c,0x32,0xc4,0x6b,0x4a,0xc0,0xf7,0x60,0x96,0x66,0x22, +0xce,0xb9,0x7c,0x07,0x63,0x9b,0x7c,0x59,0xa1,0x53,0x4c,0xfd, +0x8c,0xa2,0x05,0x0c,0xdd,0x64,0x58,0x02,0xf9,0x94,0xee,0x37, +0x1c,0x41,0x7b,0xa5,0x66,0xd2,0x67,0x18,0xe1,0x3b,0x14,0xdf, +0x5c,0x2c,0xe3,0xb9,0x53,0x4a,0xfd,0x0e,0x62,0x25,0x44,0x05, +0xe9,0x04,0x23,0x28,0x1f,0x6d,0xe6,0x93,0x69,0x6c,0xa5,0x52, +0x8c,0x3f,0x1e,0xd2,0xf4,0x62,0x6a,0x26,0xbd,0x16,0x71,0xbe, +0x23,0x53,0xcb,0x04,0x48,0x18,0xa6,0x41,0xc1,0x55,0x90,0x21, +0x21,0x05,0x25,0xa0,0x47,0x17,0xeb,0xc5,0xdc,0xa2,0xe5,0x1b, +0x11,0xc7,0x8f,0xd5,0x54,0x36,0xe2,0xce,0x87,0x6b,0xf8,0xfa, +0x38,0x88,0x88,0x54,0x64,0x8b,0x29,0x89,0x8e,0x05,0x49,0xff, +0x7c,0xcc,0xef,0x8b,0x00,0x00,0x76,0xf2,0x82,0xbc,0x10,0xab, +0x3e,0x8b,0x7f,0x4b,0xe2,0x55,0x88,0x7b,0x08,0xed,0xe3,0x13, +0x3f,0x84,0x7f,0xfc,0x96,0x79,0x05,0x54,0x9b,0xdf,0x3a,0x3f, +0x21,0xc8,0x5a,0xa8,0x13,0x86,0x3a,0x13,0xf8,0xce,0x06,0xc2, +0x43,0x6d,0xd6,0xba,0xc6,0x6d,0x6a,0x55,0x15,0xec,0xeb,0xc9, +0x18,0x97,0xdf,0xd4,0xdc,0x10,0x01,0xdc,0x09,0xf4,0x72,0x72, +0x57,0xc6,0x0f,0xf5,0xe0,0x8d,0x70,0x55,0xee,0x0f,0xa8,0x16, +0x32,0xaf,0x3f,0xf1,0x13,0xff,0xbd,0x38,0x46,0x96,0xb5,0xfd, +0xbc,0xed,0x6f,0xfc,0x5b,0x7f,0xe8,0xa7,0x4b,0xff,0x32,0x33, +0x1e,0xe9,0x41,0x65,0x71,0x99,0x9b,0x11,0xb9,0xe7,0xcf,0x23, +0xa3,0x08,0x7f,0x09,0xbf,0xd1,0xbc,0x19,0xb2,0xc9,0x4b,0x07, +0xb4,0x61,0xd0,0x1e,0xdf,0xe0,0xc7,0x75,0x44,0x17,0x8a,0x00, +0xca,0x28,0x32,0x6a,0xfd,0xf6,0xf0,0x30,0xb7,0x63,0xde,0x79, +0x92,0x34,0xc2,0xcd,0x6f,0x4e,0xa0,0xd3,0x1c,0x1e,0x4e,0x56, +0x49,0x7d,0x7d,0x08,0x5b,0xc6,0xf1,0xf6,0x51,0xbf,0x71,0x99, +0xb5,0xf0,0x7a,0x78,0x9e,0xfa,0x4d,0x08,0x09,0x1e,0x59,0x4e, +0x3c,0xbf,0x71,0x99,0x9b,0x49,0x10,0x35,0x60,0x7f,0xbc,0x24, +0x0a,0x15,0x3a,0xc5,0x55,0xc8,0x84,0x37,0x0f,0xa8,0xe0,0xf5, +0x80,0xee,0xa5,0xb6,0x93,0xa3,0xc8,0x4e,0x3f,0x3c,0x34,0xd3, +0x03,0x3b,0x7b,0x40,0x67,0xb5,0xaf,0xa3,0x44,0x96,0x1a,0xa8, +0x52,0x93,0x83,0x28,0xba,0x86,0xc4,0x3c,0x8c,0xec,0x02,0x9a, +0x36,0x7e,0x5c,0xdd,0xb2,0x4b,0x68,0xda,0x25,0x08,0xde,0x33, +0x33,0x3e,0xd0,0xfe,0x49,0xc7,0x3c,0xf9,0x8a,0xeb,0x63,0x78, +0xb9,0xcc,0x24,0xce,0x3a,0xc9,0xd6,0x63,0x4a,0x44,0x1e,0x3a, +0x0f,0x6f,0xef,0xab,0xc8,0x79,0x22,0xef,0xc4,0x34,0x47,0xfc, +0xbb,0xf5,0xfc,0x21,0x5f,0x31,0x80,0x9b,0xa5,0x1c,0xe7,0xdf, +0x1a,0x31,0x84,0xdf,0xdf,0x40,0x8b,0x21,0x6b,0x51,0x2e,0x44, +0x3d,0x51,0x2a,0xdf,0x4b,0xc0,0x44,0x02,0x0e,0x23,0x23,0x8a, +0xb0,0x11,0xa0,0x59,0xd7,0xdb,0xaa,0xba,0xe6,0x36,0x40,0x05, +0xb7,0xda,0x1a,0xe0,0x7d,0x25,0x06,0x1b,0xa0,0x8a,0xdf,0x47, +0x59,0xd8,0xc8,0xc3,0xa3,0x3c,0x3c,0x8f,0xb2,0xf6,0x51,0xd6, +0x6e,0xe4,0xed,0xa3,0xbc,0x2d,0xce,0x56,0xe9,0x4a,0x44,0xcd, +0xb0,0x2b,0x5f,0x41,0xd9,0x8a,0xb4,0x8d,0x4e,0xdb,0x0e,0x8b, +0x55,0x04,0xf6,0x24,0xc0,0xb4,0xf7,0xf4,0xf8,0x09,0xf5,0x13, +0x6e,0xb4,0x6b,0x8f,0xce,0xe1,0xbf,0x07,0xe9,0x68,0x13,0xd7, +0xde,0x7b,0x22,0xe7,0x7b,0x81,0xee,0xbc,0x19,0xd2,0xc9,0x47, +0x95,0xa5,0x90,0x0e,0x08,0x1a,0x0d,0x6e,0xd9,0x74,0x19,0x31, +0x9c,0xb8,0xff,0x80,0x9a,0xe2,0x3d,0x21,0xc5,0xe3,0x0c,0xc5, +0x9c,0xaf,0x02,0xf1,0x80,0x4e,0xf7,0x80,0x47,0xe0,0xd9,0x05, +0x6a,0x94,0xd7,0xaf,0xff,0xc6,0xc5,0xf5,0xd3,0x65,0xb3,0x2d, +0x7a,0x35,0x7e,0x87,0x52,0xde,0x14,0x06,0xd5,0x45,0xbd,0xae, +0xd7,0x6c,0xa6,0xcb,0xae,0x84,0x6d,0x34,0x06,0x51,0x01,0x56, +0x9c,0xd6,0x9b,0xe6,0xd8,0x9d,0xb9,0x47,0x20,0xce,0x6e,0x72, +0x0e,0x75,0x6f,0x36,0x13,0x0f,0xd3,0xb8,0x86,0xd4,0x8b,0x19, +0x55,0x32,0x18,0x78,0xfa,0x0c,0xe1,0x34,0xdf,0xaa,0x53,0xee, +0x4a,0xa9,0xf0,0x0a,0xa1,0xd2,0x28,0xdd,0xba,0xd5,0x0e,0xd2, +0x96,0x8a,0xe0,0x8e,0x8b,0xe1,0x56,0xb6,0xe7,0x4a,0x75,0x74, +0xe1,0x23,0xbc,0xd7,0xc9,0xb6,0xf4,0xd1,0xca,0xf7,0xcb,0x98, +0xab,0x8c,0x39,0x67,0x54,0xab,0x18,0xca,0x7b,0x75,0xb7,0xe7, +0x03,0x56,0x6d,0x33,0x1a,0x4f,0x87,0x69,0x6a,0xa6,0xfb,0xb2, +0xc5,0x04,0x12,0xf5,0x5a,0xb1,0xe5,0xdc,0x60,0x2e,0x61,0x94, +0x41,0xa3,0x02,0x21,0x86,0x6f,0x71,0x7f,0xd0,0xad,0xcb,0x55, +0x70,0xde,0x34,0x1e,0xe0,0x91,0x77,0xd0,0xf1,0x2b,0xdf,0x3c, +0x4a,0xf0,0x9b,0xd5,0xfd,0x39,0x0a,0x0f,0x65,0x0c,0xe4,0x85, +0x74,0x78,0xd3,0x84,0x48,0x4d,0x38,0xad,0xc1,0x5b,0xb6,0x6a, +0x47,0xf7,0x68,0xa4,0x8f,0xcc,0xb4,0x4e,0x8f,0x10,0x64,0x5b, +0x5b,0xef,0x0a,0x9f,0xce,0x1b,0x71,0xa3,0x9a,0x3f,0xae,0xa2, +0x2c,0x8b,0xf0,0xba,0x81,0xc0,0xa4,0x90,0xc4,0xb2,0xd8,0x86, +0xf7,0xf7,0xee,0x0d,0x5e,0xa1,0xe1,0x9e,0xf1,0x53,0xd3,0x43, +0x3a,0xc1,0x09,0xba,0xb4,0xb2,0x1e,0xe3,0x48,0x51,0x4e,0x37, +0xe6,0x1c,0xd1,0x8d,0x19,0x59,0x23,0xa2,0x5b,0x3b,0x1a,0x74, +0x75,0xc6,0xd1,0xd8,0xcf,0x29,0x22,0xc4,0x88,0x10,0x23,0xba, +0x72,0x3a,0x88,0xc7,0x5e,0x8e,0x6e,0x06,0xbb,0xea,0x89,0x87, +0xaa,0xf5,0x82,0xdc,0x46,0xac,0x7d,0xa9,0x6b,0x76,0xe7,0xcb, +0xcd,0x1a,0xc6,0xf8,0x99,0x3f,0x82,0x7a,0x4f,0x7c,0x61,0xe2, +0x14,0xd0,0xb1,0x9d,0xa5,0x72,0x57,0xf0,0xa7,0x59,0x9d,0x83, +0x2f,0xa0,0x1d,0x1a,0x3c,0x13,0x6c,0x20,0x02,0xa8,0xf0,0x48, +0x60,0x15,0x9b,0x6d,0x78,0xe9,0x3a,0x87,0xc5,0x92,0x59,0x60, +0xb1,0x8b,0xb2,0xf4,0xdd,0x19,0x7f,0xc8,0xc2,0x99,0x6a,0x0f, +0xa5,0xe6,0x3d,0xea,0x5a,0x50,0x61,0xaf,0x66,0x5d,0x6f,0x22, +0xc0,0xdf,0x0f,0x8a,0xfc,0xf9,0x6a,0x9e,0x82,0xa5,0xef,0x4e, +0x7c,0x1c,0xc8,0x7a,0x6e,0x75,0xea,0x98,0x52,0xef,0xef,0x25, +0x41,0xac,0x6c,0x8a,0xa0,0x6b,0x9c,0x81,0x8c,0xd7,0x2e,0x72, +0x0f,0xe1,0x75,0x05,0x18,0xde,0x93,0xd7,0xc9,0x56,0xa3,0x3f, +0x3c,0x7c,0x2c,0x7a,0x98,0xb3,0x76,0xff,0x89,0x6b,0x5c,0xc8, +0x2c,0x02,0x28,0x34,0xc0,0xf3,0x91,0x74,0xd3,0x12,0xd1,0x5a, +0x93,0xe0,0xbb,0x39,0x9b,0xf9,0x30,0xaf,0xb2,0x16,0x51,0x28, +0xe2,0x09,0xde,0xbf,0xa2,0x6c,0x27,0x6b,0x67,0x57,0x4d,0x19, +0x07,0x30,0xd0,0xaf,0x57,0x09,0x0d,0x57,0xe6,0x33,0x54,0xb2, +0x06,0xb7,0x8b,0x15,0x98,0xe0,0xc9,0x8f,0x71,0x9a,0xc6,0x53, +0x5a,0xe4,0x55,0x66,0xa5,0x42,0x06,0x43,0x39,0x95,0xd5,0x8f, +0x5b,0xab,0xf8,0x0a,0xdd,0xa0,0xa6,0xac,0xfd,0x07,0xe2,0x0e, +0x67,0x39,0xde,0xaa,0xf4,0x95,0x48,0xc7,0x87,0xa8,0x28,0x6b, +0xd9,0x88,0xc7,0x68,0x7c,0x84,0x8e,0x2c,0x59,0xd3,0x20,0xf2, +0xc7,0x59,0x74,0x8b,0x03,0xfb,0x38,0x87,0x5f,0x90,0x45,0x84, +0x2c,0xe7,0xbf,0x85,0xac,0xc2,0x11,0xc7,0x30,0x8e,0x30,0x63, +0x73,0x0c,0x43,0x18,0x76,0xd5,0x71,0x2e,0xcb,0x7f,0x8c,0x5d, +0xa4,0x8e,0x77,0xe0,0x3d,0xd6,0xb3,0x88,0xb2,0xcb,0xeb,0x75, +0xba,0x33,0x18,0x79,0xdf,0xc3,0xc8,0xac,0x58,0x2a,0x4c,0xd6, +0x5b,0x2a,0x06,0x45,0x98,0x7f,0x1b,0xe1,0x80,0x2a,0xaf,0xe0, +0x74,0xb3,0x0a,0x8e,0xeb,0x56,0xe5,0x21,0xd5,0x98,0x01,0x08, +0x08,0x31,0x09,0x50,0xe7,0xe1,0xd9,0xd0,0x17,0x22,0xc7,0xa6, +0xf5,0x8e,0x46,0x7f,0xac,0xe5,0xff,0x73,0x14,0xc6,0x67,0x35, +0xf6,0xfd,0x65,0xc9,0xc0,0x17,0xd3,0x03,0xdb,0xbc,0x17,0x91, +0xfa,0xed,0x80,0x39,0xd9,0x41,0xca,0x42,0x56,0xd6,0x53,0xf4, +0x48,0x13,0x5f,0x5c,0x6e,0x5e,0x23,0xb0,0x82,0x39,0xa6,0xb8, +0xd2,0x06,0x57,0xe8,0xa7,0x6d,0x9c,0x15,0xe1,0x94,0x08,0xe6, +0x46,0xed,0x6e,0x08,0xb5,0x80,0x91,0xee,0xf0,0x30,0x6e,0x8d, +0xc0,0x3e,0xe9,0xb9,0x69,0x08,0xe1,0x78,0xd9,0x5a,0x01,0x24, +0x7f,0xc1,0x90,0xae,0x62,0xa7,0x2a,0x76,0xe5,0xf9,0x22,0x6f, +0x0f,0xcd,0xfb,0x34,0xec,0xa5,0x61,0x2b,0xef,0x34,0x7f,0x46, +0x5b,0x1e,0x94,0x77,0x13,0x10,0x1e,0x51,0x19,0x51,0xda,0xee, +0xa5,0x6d,0x48,0xfb,0x19,0x0d,0x57,0x95,0xd4,0x06,0xb4,0x99, +0xcc,0x98,0x61,0xc6,0x5c,0x66,0x1c,0x1e,0x11,0x89,0x22,0x63, +0x06,0x19,0xf3,0xb6,0x4e,0x6a,0xf3,0x84,0xf5,0x16,0xe6,0xa6, +0xbc,0x97,0xed,0xdf,0xc2,0x04,0x94,0x26,0x79,0x83,0xae,0x14, +0x85,0xda,0x7e,0x19,0x02,0xb4,0xe7,0x97,0xc1,0x56,0x15,0x60, +0x5b,0x0d,0x18,0xc9,0x0f,0x5b,0xbc,0x44,0xac,0xec,0xb4,0xd0, +0x4d,0x55,0x83,0x8a,0xee,0xaa,0xc3,0x68,0x18,0x50,0xdf,0x94, +0x56,0x88,0x85,0xe9,0xb6,0xe0,0x26,0x53,0xd7,0x75,0xc1,0xe4, +0x89,0xe9,0x11,0x57,0x76,0xc4,0x91,0xb8,0xc4,0xde,0xce,0xae, +0x4e,0x8d,0x65,0xf7,0x87,0x60,0x71,0xd0,0xc7,0x08,0x3a,0x84, +0x54,0x7c,0x84,0xb3,0xe4,0xb8,0x33,0xd1,0xef,0x61,0x1e,0x24, +0xf7,0xf7,0x9c,0x6d,0x22,0xf3,0x53,0x9e,0x7e,0x02,0x03,0xd7, +0x40,0x7b,0xf2,0xe8,0xbc,0xaa,0x58,0x41,0x20,0x64,0xdc,0x9a, +0xfc,0x2c,0x2b,0xaf,0x02,0x2b,0xe7,0x91,0x5d,0x35,0x94,0xf7, +0x83,0xb9,0x67,0x63,0x15,0x9a,0x06,0x64,0xa7,0xf9,0xf3,0xc0, +0xc7,0x5f,0xfc,0x91,0x7f,0x21,0x8e,0x2e,0x47,0x3d,0x70,0xe7, +0xe7,0x6d,0x71,0xb9,0x26,0x5e,0x07,0x29,0x5b,0xc4,0x5f,0x86, +0xb2,0x10,0x9a,0x62,0xb6,0x55,0x88,0x5c,0x30,0xa2,0x65,0x20, +0x7c,0x34,0xe0,0x83,0xe7,0xa8,0xcb,0x50,0xcc,0xcf,0xe0,0x83, +0xe7,0x61,0xcb,0xb6,0x98,0x6f,0xc1,0x07,0xc4,0x4c,0x32,0x7d, +0x4a,0x33,0x6b,0x37,0x71,0x52,0x3a,0xc9,0xa3,0xbc,0xdd,0xcc, +0xe9,0xc1,0x4a,0x7d,0x82,0x33,0xf7,0x5e,0xfd,0xe7,0x3f,0x98, +0x24,0x48,0x0f,0xcf,0xf3,0xa0,0xd7,0xfc,0xb9,0xf3,0x73,0xb7, +0x54,0xb1,0x9c,0xaa,0x92,0x0f,0x8c,0xdb,0x45,0xff,0x71,0x3e, +0xc9,0xe4,0xad,0x24,0x59,0x78,0x9e,0x55,0x66,0xcd,0x98,0x25, +0x19,0xf0,0x82,0xb3,0x8a,0xb2,0xa0,0xcf,0x35,0xb3,0xc0,0x3b, +0x72,0x89,0x28,0xef,0x5c,0x90,0x09,0xe1,0xb0,0x99,0x07,0x5e, +0xef,0x67,0xec,0x89,0xbf,0x45,0x26,0xa5,0xcd,0x49,0xd6,0x55, +0xe1,0xdf,0x90,0xf0,0x9e,0x5d,0x56,0x70,0x3e,0xc9,0x7b,0xd0, +0xf7,0x73,0xa4,0xb3,0xe3,0xfe,0x86,0x0f,0x1a,0xb8,0xb9,0x78, +0x65,0xac,0xb6,0x42,0x78,0xb0,0x9f,0x2f,0xb3,0x11,0x7e,0x50, +0x5e,0x71,0xf9,0xc0,0x48,0x23,0x6b,0xd0,0x55,0xdd,0x36,0x19, +0x94,0x17,0x1b,0x8c,0x44,0x79,0x1c,0x93,0x35,0x68,0xd7,0x16, +0x9a,0xf4,0xe1,0xa9,0x95,0x49,0x15,0x1d,0xa2,0xa8,0x9d,0x6b, +0xc9,0x71,0x49,0x0c,0x58,0x1d,0x57,0x44,0x3c,0x66,0xbe,0x25, +0xb2,0xd0,0x94,0x4b,0x66,0x7f,0xc4,0xac,0x4b,0x65,0xcf,0x75, +0xf6,0xaa,0xfb,0x95,0x1e,0x38,0xeb,0x6f,0x54,0xf9,0xf0,0xb0, +0x6f,0x84,0x48,0x91,0x19,0xc1,0xf6,0x80,0x06,0x1f,0x3e,0x5b, +0x72,0xe1,0x95,0x58,0xa5,0x84,0x8d,0x3d,0x9d,0x02,0xe3,0x9a, +0xa3,0x40,0x2e,0x5f,0x68,0x0f,0xa8,0xb6,0xf2,0x7a,0xea,0x9a, +0x98,0x0a,0x33,0x36,0x3c,0x0c,0xcf,0xba,0x9e,0x7e,0xdb,0xf8, +0xdb,0xd6,0xbf,0x64,0x59,0xc8,0x37,0xef,0x98,0x05,0x9a,0x17, +0x69,0xf2,0xdb,0x7e,0xfe,0x10,0x65,0xe6,0xb9,0x17,0xc0,0x0a, +0xf6,0xa2,0xbe,0x00,0xc9,0xb6,0x58,0x64,0xb3,0x44,0x17,0x07, +0x02,0x74,0x63,0x78,0x1b,0x56,0x1a,0x2c,0xb4,0xe5,0xf1,0x90, +0xb5,0xc2,0x88,0xfa,0x83,0xc7,0x9a,0x2d,0xff,0xb7,0x1a,0x2c, +0x0f,0x5a,0xd9,0x35,0x46,0xcb,0x34,0x2a,0x8e,0xe5,0xfe,0x2a, +0x2a,0x8e,0xdb,0x5d,0x62,0x69,0x7f,0x3a,0xe8,0xaf,0x06,0x30, +0x91,0x35,0x42,0x11,0x7f,0xaf,0x06,0x10,0xc4,0xfb,0xbd,0x88, +0xad,0x62,0x21,0x91,0xfd,0x34,0x3a,0x44,0xf8,0x74,0x20,0x9c, +0x31,0x38,0xb8,0x82,0x26,0xf0,0x64,0x2b,0xe8,0xd6,0xd5,0xf3, +0x86,0x42,0x0b,0xe3,0xc2,0xde,0xc3,0x8b,0xc0,0x96,0x2f,0x90, +0x9e,0xce,0x30,0x0e,0x7d,0xa5,0xa5,0x5f,0xd5,0x0a,0x0f,0x8b, +0x81,0x98,0xdc,0x9a,0xb2,0x20,0x26,0xad,0xee,0x47,0x37,0x9b, +0xd7,0xc2,0x74,0x98,0xb8,0xd5,0x91,0x2e,0x34,0x9f,0x2d,0xc8, +0x92,0x4d,0xfa,0x61,0x15,0x11,0x51,0x34,0x9f,0xf0,0xda,0xab, +0x0f,0xe6,0xd5,0x59,0x54,0x1c,0x9a,0x0a,0x8a,0xe7,0x9e,0xbd, +0xb8,0x23,0x62,0xbf,0x5b,0x2e,0x17,0x69,0xb2,0x8e,0xa3,0x0f, +0xd3,0x8e,0xb3,0x72,0xfc,0x55,0xc7,0x99,0x3a,0x5b,0xb5,0xa5, +0xf0,0xeb,0x66,0x38,0x59,0xaf,0xe2,0xb8,0x38,0xa5,0x30,0xb7, +0x04,0x8c,0xe9,0x85,0x04,0x17,0x8d,0xa9,0xe2,0x93,0x79,0x8a, +0x67,0x6f,0xe6,0xc0,0x0b,0xf4,0x42,0xb6,0x32,0xa3,0x32,0x4d, +0xd2,0x6f,0x87,0xdf,0x42,0x5b,0x1c,0x1e,0x8a,0xcf,0xdc,0xc3, +0x15,0x48,0x34,0x5a,0x86,0xe2,0x75,0xf4,0x79,0x16,0xcd,0x5b, +0x99,0x3f,0xcf,0xe1,0x27,0x97,0x8a,0xf3,0x20,0x9a,0x67,0x9e, +0x69,0x10,0xcc,0xb3,0x66,0xe6,0x35,0x74,0x30,0x6f,0x82,0x85, +0xd0,0x0a,0x42,0x8f,0x29,0x78,0x7d,0x9d,0x4c,0x27,0x95,0x64, +0x68,0x75,0x3b,0xe7,0x7b,0x46,0xe6,0x3c,0xf8,0x75,0xa1,0x50, +0x2c,0x51,0x86,0xf9,0xbc,0xb3,0x8d,0x8d,0x73,0xf8,0x73,0xa4, +0xce,0x42,0xea,0x3f,0x54,0x2a,0x1b,0x23,0x58,0x04,0xe4,0x85, +0x42,0x72,0x5f,0x16,0x33,0x11,0x76,0xca,0x43,0x08,0x0a,0x2c, +0xae,0x87,0xe4,0x89,0x4b,0x16,0xb5,0x4e,0x8f,0xc0,0x76,0x69, +0x80,0xf9,0xef,0xa7,0x39,0x85,0xf2,0xb0,0x81,0xb4,0xae,0x92, +0xab,0xeb,0x75,0x94,0x9d,0x47,0x69,0xe6,0x8f,0x16,0xeb,0xf5, +0x62,0x16,0xe5,0x10,0xc8,0xa1,0x3b,0xb8,0x1c,0x7e,0xf5,0x2a, +0xf4,0x1a,0x04,0xd6,0xe5,0x96,0xc1,0x2b,0xf5,0x60,0xb4,0x6a, +0xcd,0x17,0x13,0x9a,0x5f,0x83,0xc6,0xd0,0x01,0xb5,0x84,0x24, +0x85,0xe2,0x5b,0x7c,0xe7,0xcb,0x13,0x05,0xf5,0x70,0xce,0x92, +0x75,0x70,0x76,0x22,0x8b,0xeb,0xe1,0xfc,0x27,0xef,0xe0,0x4c, +0x27,0xf7,0x77,0x48,0xcc,0x96,0xfb,0x76,0x96,0xfa,0x79,0x0a, +0x83,0xc0,0x1c,0x12,0x2f,0x20,0xf5,0x02,0x92,0xe1,0x17,0xfe, +0x3d,0xd8,0xc3,0xb5,0xf8,0x80,0xc9,0x06,0xb9,0x23,0x42,0x7f, +0x11,0x51,0x11,0x10,0xc2,0x62,0x20,0xd4,0x56,0x4f,0x9a,0x67, +0x14,0xbc,0x88,0x9a,0x6e,0x46,0x60,0x17,0x7c,0xfc,0x36,0xa3, +0x71,0x25,0xa7,0xbf,0xf6,0x94,0x97,0xee,0x0d,0x5e,0x97,0x2c, +0x31,0xad,0x25,0x26,0xad,0xec,0x15,0xe0,0xc2,0x6b,0x4d,0x01, +0x1b,0x84,0xc0,0xc2,0x6d,0xe5,0xaf,0x72,0x8a,0xca,0x29,0x2a, +0xc7,0xa8,0xec,0x1c,0xca,0x46,0xa8,0xb6,0x86,0x3a,0xcf,0x29, +0x2a,0x6f,0x0b,0xa8,0x4c,0xe8,0x61,0x4a,0xcf,0x55,0x20,0xf7, +0xac,0xed,0x31,0x45,0x04,0x1b,0x15,0x17,0x11,0x28,0x30,0xd7, +0x52,0x5b,0x39,0xc6,0xe5,0x34,0x61,0xe9,0x02,0x55,0xe7,0x99, +0x24,0x2f,0x43,0x97,0x69,0x88,0xc8,0x25,0x71,0x39,0x44,0x64, +0x17,0x06,0x69,0x04,0x71,0x61,0x10,0x46,0x10,0x82,0x14,0x4a, +0x14,0xdf,0x10,0xcf,0xed,0x07,0x22,0xdf,0xbe,0x68,0x62,0xd3, +0x91,0x65,0x7f,0x01,0x56,0xf4,0x45,0x77,0x92,0x91,0x09,0xdc, +0x26,0x1e,0x37,0x26,0x28,0x1f,0xd8,0x38,0xf0,0x99,0x93,0xf2, +0x5a,0x2d,0x16,0xeb,0x1a,0xb9,0xc2,0x36,0xc5,0x64,0x7c,0xab, +0x3c,0x32,0x27,0x50,0x42,0x92,0x30,0x0d,0x84,0x89,0x2a,0x4d, +0x37,0x16,0xfb,0xb2,0xae,0x45,0xf9,0xc1,0x47,0xd9,0x10,0xd1, +0x6d,0x02,0x8a,0x51,0xa3,0xba,0x94,0x37,0x96,0xe8,0x92,0x7f, +0x42,0x08,0xf7,0x92,0xc0,0x2b,0xb0,0xf0,0x38,0x42,0xf6,0x1d, +0xc8,0x19,0xef,0x9c,0x88,0x81,0xc4,0x22,0x4a,0xb4,0x40,0x46, +0x93,0xa9,0x9c,0xfe,0x16,0x91,0xe1,0x86,0x07,0x6b,0x04,0x92, +0x32,0x39,0xeb,0x93,0x15,0x56,0x53,0x50,0x90,0xc8,0x9c,0x4d, +0x91,0x68,0xc2,0xaa,0x0a,0x41,0xb6,0xfc,0x62,0x0d,0x8a,0xa4, +0xbf,0xcf,0x5e,0x08,0x5f,0x7b,0xbd,0x2e,0xbd,0xd1,0xed,0xf5, +0xd4,0x25,0xbc,0x9a,0x09,0xaf,0x09,0xf6,0xad,0xf2,0xbb,0x2e, +0xa6,0xbc,0xf3,0x4f,0x60,0xf8,0x66,0x8c,0x24,0x1c,0xdc,0xcd, +0xb8,0xcf,0xc1,0x5f,0x7c,0xec,0x5b,0x8d,0x1b,0x62,0x14,0x84, +0x09,0x80,0x8c,0x7a,0xcc,0x0c,0x40,0xe6,0xa1,0x29,0x80,0x42, +0xf0,0x88,0x39,0x80,0x46,0x90,0x1b,0x08,0x1e,0x71,0xda,0x43, +0x9a,0xf3,0xa8,0xde,0xb0,0x76,0x50,0xd9,0x36,0xb5,0x03,0x2d, +0xd2,0x3c,0xc6,0x8c,0xf7,0x0c,0x62,0xa4,0x0c,0xb1,0x3f,0xaf, +0x61,0xc3,0x93,0xed,0x3e,0x30,0x48,0xdd,0xdb,0x48,0xdf,0x4d, +0x28,0xc4,0x04,0x82,0x30,0x83,0xaa,0x1d,0x24,0xd1,0xec,0xd7, +0xa7,0xb9,0xb0,0x41,0x0e,0xef,0x5d,0x19,0x6e,0xbe,0x3f,0x5c, +0x8d,0xe8,0x65,0x41,0x2b,0xa6,0x00,0xc3,0x2f,0xa8,0x15,0xc0, +0xbe,0x93,0x1e,0xb8,0x26,0xe4,0xb7,0x9b,0xd9,0x88,0x16,0x71, +0x2e,0x4a,0x91,0x05,0x48,0x76,0x7c,0x2f,0x40,0x72,0xa4,0x34, +0x8a,0x8c,0x84,0x8b,0x39,0x23,0x3e,0xee,0x37,0x1b,0x83,0x9e, +0xdb,0xeb,0xfc,0x32,0x69,0xfc,0xd2,0xea,0xfd,0x32,0x39,0xba, +0xa7,0x9f,0x86,0x07,0x71,0xfd,0xf8,0x2f,0x03,0x4a,0xa7,0xf7, +0x32,0x8f,0xaf,0xba,0x76,0x89,0x85,0xb2,0x6c,0x7a,0xd0,0x7f, +0xa8,0x5f,0x71,0x44,0x51,0x3a,0x6e,0x8f,0x64,0x3f,0x74,0xf8, +0xe9,0x42,0x7c,0x5e,0x74,0x8d,0x2f,0xd9,0xad,0xae,0x46,0x17, +0xf3,0xe1,0x2c,0x4e,0xf1,0xc9,0x3b,0x17,0xb7,0x26,0x8e,0xff, +0xee,0xfe,0x8f,0x7b,0x88,0xfe,0xc5,0xbd,0xbf,0x4e,0xa7,0xbf, +0xb8,0x9e,0x70,0x49,0x19,0xd1,0xcb,0x77,0x36,0xa3,0x3b,0x55, +0xf5,0xef,0x8c,0xd0,0x74,0x20,0x5f,0x7a,0x28,0x19,0x20,0x5e, +0x2f,0xa6,0x8b,0x55,0x45,0x66,0x87,0xbd,0xa0,0x99,0x16,0xb2, +0xa6,0x5b,0x49,0xca,0x56,0x75,0xa9,0x30,0x8a,0xee,0x54,0x35, +0x61,0xa7,0xaa,0xb5,0x3c,0xe2,0xc1,0x76,0x50,0x68,0x35,0xc2, +0x12,0x55,0x60,0x96,0x6d,0x16,0x0f,0xd3,0xf8,0x62,0xc2,0x6f, +0x11,0x57,0x7b,0x76,0x49,0x6f,0x2e,0xda,0x1f,0x43,0xf0,0x0a, +0xbf,0x2d,0x13,0x0b,0xad,0xb3,0xa8,0x48,0x0c,0x90,0x30,0x57, +0x3c,0x71,0x28,0x61,0x30,0x79,0xeb,0x8f,0x37,0xa3,0x64,0xbc, +0x03,0x8a,0xd2,0xb7,0x78,0xc0,0x72,0x07,0x50,0x8a,0xe7,0x80, +0xe3,0x6c,0xb9,0x03,0x04,0x52,0xe5,0xf1,0xd7,0x5d,0xa5,0x11, +0x00,0xe0,0xc2,0xf7,0x86,0x81,0x2e,0x95,0x9b,0xc3,0xfe,0x68, +0x38,0xbe,0x51,0x91,0x18,0xa0,0x47,0x11,0xc6,0xbb,0x50,0x32, +0x80,0x78,0x29,0x94,0x62,0x66,0x30,0xec,0x2a,0x6e,0x3a,0x09, +0x7b,0x72,0xa9,0x7b,0x38,0x16,0x9b,0xb5,0x2a,0x41,0x1c,0xa7, +0xf3,0x01,0xa8,0x09,0xf1,0x8e,0x91,0x40,0xaf,0xf3,0xf8,0x0e, +0xc4,0x36,0x11,0x83,0x39,0xd8,0x16,0x08,0x10,0xb0,0x6e,0x01, +0x29,0x40,0xe2,0xfb,0xa5,0xd8,0xfb,0xa8,0x79,0x15,0x06,0xec, +0x26,0xf2,0x21,0x7f,0xfc,0xe6,0xf9,0x2d,0xbe,0xbb,0xdc,0x74, +0xf0,0xa2,0x5a,0x74,0x23,0xe8,0x51,0x82,0xfd,0xd8,0x72,0x07, +0xe3,0xfc,0x59,0x65,0x3a,0xbe,0xa9,0xdc,0xc1,0xaa,0xaa,0xab, +0xb7,0x22,0x41,0x0e,0x3f,0x95,0xcd,0xef,0x64,0x5b,0xf2,0x34, +0x8b,0x4c,0x7e,0x11,0xd8,0x8c,0xc1,0x14,0xab,0x54,0xab,0x4d, +0x87,0xb3,0xa5,0x3b,0x73,0xe5,0xe9,0x14,0x1a,0xbe,0xb9,0xa3, +0xa9,0x8d,0x76,0xde,0x23,0xe5,0xc9,0xa5,0xd2,0xe9,0x7e,0x48, +0xaf,0xb8,0x16,0x7a,0xcf,0x97,0xe3,0x69,0xa1,0xef,0x40,0x4c, +0x11,0x26,0x2d,0xc1,0xa4,0x45,0x98,0xaf,0x87,0x45,0x9d,0x0d, +0x31,0x45,0xbd,0x4e,0xaf,0xa8,0x14,0x74,0x06,0xc6,0x21,0x5c, +0xc5,0xcd,0x69,0xcc,0x4f,0x6e,0x1e,0x52,0xcb,0x93,0xc5,0x98, +0xea,0xd2,0x1a,0xe3,0xb1,0x92,0xf8,0x2f,0xd3,0x18,0x43,0xdf, +0xbe,0x41,0x8f,0xd2,0x79,0xda,0x92,0xef,0x6f,0xdf,0x5e,0xf9, +0xf8,0x7c,0xba,0x54,0x8f,0xbb,0x91,0x2b,0x23,0x5f,0x46,0x5c, +0xb5,0xd2,0x78,0xfd,0xe9,0x1a,0x42,0xa3,0xcd,0x3a,0x76,0x1d, +0x95,0xd7,0xf1,0x05,0x48,0x97,0xd5,0xf0,0x95,0x46,0xdb,0x1a, +0x41,0xd3,0xfc,0x34,0x9c,0xa2,0x07,0x7b,0xba,0x98,0x26,0x13, +0xba,0xbd,0x53,0x2e,0x92,0x89,0x0b,0x5f,0xf4,0x3d,0x6e,0xeb, +0xde,0x1a,0x3a,0x04,0x20,0xcb,0x3a,0x66,0xfc,0x57,0xca,0xa9, +0xd4,0x93,0xe4,0x91,0x46,0xaa,0xb8,0xa6,0x4d,0xbd,0x52,0x5c, +0x56,0x68,0x8e,0xbe,0xf4,0xd2,0x69,0xf0,0x33,0xac,0x32,0xa2, +0xe1,0x78,0xe2,0x62,0x04,0x91,0xc2,0x21,0x88,0x4e,0x6f,0xe2, +0xbb,0xb7,0x32,0x16,0x03,0x18,0x47,0x97,0x55,0xca,0x38,0x0c, +0x40,0xa4,0xa3,0xd6,0x07,0x4a,0x64,0x47,0x1f,0x86,0x9d,0xd0, +0x1f,0x75,0x02,0x7f,0x0c,0xff,0x26,0xf0,0x1d,0xc3,0xef,0x65, +0x27,0xd8,0x16,0x46,0xbd,0x1f,0x55,0x6b,0xd8,0xb2,0xa0,0xe2, +0x51,0x1e,0xa6,0xc3,0x1c,0xfa,0xbb,0x70,0xcc,0xe5,0x40,0x8b, +0x5d,0x49,0x2b,0x74,0xb8,0x8a,0xa1,0x05,0x24,0xed,0xb6,0x80, +0x17,0xcd,0xd0,0xbc,0x4b,0x2c,0xd6,0xf0,0x72,0x94,0xbd,0x5a, +0x43,0x30,0x6a,0x57,0x9d,0x4b,0xba,0xe0,0x92,0xbe,0xc7,0x9b, +0xdd,0x28,0x0f,0xae,0x81,0x7b,0xe6,0x65,0x63,0xe9,0xd6,0x24, +0x6c,0x7c,0xbd,0x58,0x4d,0x2a,0x0f,0xda,0xad,0x62,0x06,0x11, +0x37,0x6e,0xdc,0xc0,0x3c,0x35,0x0b,0xc8,0x5d,0x12,0x14,0x06, +0xbd,0x1c,0x9c,0x62,0x1d,0xe9,0xeb,0xcd,0x66,0x46,0xc4,0x52, +0x80,0x1e,0x80,0xd7,0x5b,0x4f,0x73,0x4f,0x65,0xe0,0x14,0xe1, +0xee,0x44,0x25,0xeb,0x5c,0xf4,0x75,0x43,0x3e,0x73,0xe4,0xdf, +0xa0,0x9d,0x28,0x32,0xf2,0x07,0x13,0x4e,0x0f,0x10,0x99,0x35, +0x22,0x16,0x41,0xdc,0x2d,0x7b,0x3f,0xe8,0x2a,0x0a,0xc4,0x6c, +0xac,0x50,0x9e,0x64,0x90,0x22,0xc7,0xf3,0x6f,0x1a,0x51,0xc6, +0x8e,0x67,0x8b,0xd5,0xfa,0xff,0xa5,0xd2,0x0f,0x0f,0x8d,0x1c, +0xf5,0xbb,0x63,0x3a,0x83,0xab,0x8a,0xed,0x0f,0x07,0x9a,0x0b, +0xfd,0x11,0x6d,0x5e,0x23,0xdc,0x1b,0xc9,0x26,0x72,0x62,0x30, +0xd0,0x97,0xf6,0xb4,0x68,0x0b,0x6d,0xf2,0x40,0xb1,0x0a,0x9d, +0xab,0x6b,0x0f,0x25,0xeb,0x00,0x95,0x0c,0x65,0xdf,0x44,0xee, +0xef,0xff,0xde,0x14,0x0f,0xa9,0x1f,0xcd,0xf1,0x1e,0xf2,0xac, +0x92,0xaf,0x41,0x94,0x59,0x8c,0xe5,0x57,0xc6,0x92,0x48,0x93, +0x4a,0x33,0xf9,0xf7,0x91,0x45,0x7e,0x7f,0x42,0x6c,0xf7,0x6f, +0x65,0x2b,0x60,0x78,0x02,0x11,0x43,0xc4,0x37,0x04,0x5b,0xbb, +0x11,0xdd,0x1e,0xdd,0x74,0x95,0x94,0x00,0x40,0x03,0x06,0xa5, +0x06,0xc0,0x44,0x1f,0xd8,0xf7,0x72,0x92,0x60,0x1b,0x89,0xef, +0xf7,0xe8,0x51,0xb7,0x5a,0x7f,0x4a,0xde,0x9a,0xc3,0xc0,0x8f, +0xe7,0x13,0xf1,0x1d,0xb2,0x5b,0x72,0xe7,0x76,0xbb,0x55,0xa8, +0x2c,0x1c,0x3a,0x5f,0x66,0xe4,0xcb,0x44,0x36,0x37,0xc3,0x2d, +0xac,0xe3,0x9b,0x2d,0xfa,0x1b,0x09,0x7e,0x6c,0x79,0xb5,0x40, +0xb3,0x02,0xc3,0xef,0xa3,0xa4,0xc0,0x05,0x71,0xe4,0x51,0xd7, +0x81,0xab,0xf0,0x5e,0x2e,0xfe,0x1a,0x29,0xef,0x29,0x25,0x19, +0x74,0xc5,0x19,0xbf,0x16,0x15,0xae,0x4e,0x62,0xb2,0xb7,0xf5, +0xe1,0x21,0x8b,0xbb,0x70,0x63,0x31,0x00,0x5f,0x99,0x70,0x3d, +0xb9,0xd4,0xcc,0x91,0x72,0xa5,0x59,0x9c,0xa4,0xec,0xc8,0x54, +0xeb,0xd4,0xa1,0x3c,0xf3,0xe8,0x6d,0x51,0x4a,0x5e,0x53,0x31, +0x87,0x87,0xab,0x98,0xa4,0xc9,0xbe,0xe3,0x90,0x62,0x3e,0x08, +0x4a,0x76,0x4b,0x1b,0xe3,0x71,0xdd,0x61,0xcb,0x24,0xb6,0x31, +0x6c,0x59,0xd5,0x3a,0x6e,0xfb,0xee,0xc8,0x86,0x18,0x15,0x21, +0x50,0x28,0x69,0x7e,0x4e,0x18,0x45,0x67,0x17,0x62,0xeb,0xcf, +0x7d,0xdd,0xa3,0xec,0x4e,0xe3,0x6b,0x2a,0x7c,0x56,0x52,0x1f, +0xf0,0xfa,0x2d,0x6a,0xc4,0x28,0x50,0xe7,0xd0,0x30,0x45,0x8c, +0x48,0xfb,0xdd,0x2a,0x33,0x8f,0x44,0xff,0x89,0x70,0x39,0x96, +0x3f,0xd5,0xaa,0x16,0xab,0x24,0xa1,0x8f,0xc8,0x3e,0xa1,0x28, +0xaf,0xc3,0x70,0x5b,0x0e,0xb6,0x24,0x19,0x7b,0x15,0x28,0x81, +0xb3,0x1d,0xe8,0xa5,0x70,0x0a,0xfc,0x9a,0x27,0xfb,0x15,0x61, +0xc0,0xef,0x2a,0x45,0x83,0x99,0x05,0x29,0x8e,0xef,0x5f,0x96, +0xce,0xa2,0x8a,0x2b,0x96,0xa3,0x40,0xcc,0xa2,0xb8,0x39,0xf7, +0x2f,0x47,0xc0,0xcb,0x42,0xb4,0x54,0x9b,0x25,0x31,0x90,0x2c, +0x66,0x5c,0x28,0x42,0xdf,0x98,0x45,0x09,0xf7,0xf7,0x7a,0x58, +0x13,0x58,0x65,0xce,0x22,0x13,0x54,0x4e,0x4e,0xb0,0x72,0x5a, +0x55,0x33,0x87,0x53,0xd0,0x26,0xe3,0xea,0x0b,0x63,0x56,0xf1, +0x72,0x33,0x05,0x7b,0x1e,0x97,0x8f,0xcb,0xa3,0x3f,0xce,0xb2, +0x70,0x11,0x18,0x57,0xc4,0xc8,0xb2,0xc3,0x08,0x5e,0x2a,0x3f, +0x88,0x22,0xce,0x23,0x96,0x15,0x29,0x65,0x9c,0x35,0x31,0xb2, +0x85,0xf7,0x0e,0x88,0x98,0x9c,0x63,0xc0,0xde,0x9e,0x47,0xa1, +0xf1,0x94,0xd3,0x24,0x3b,0x9a,0x64,0x8d,0x49,0x7e,0x34,0xe1, +0x85,0xe1,0xf5,0x75,0xbc,0x1e,0xaa,0xfd,0xfc,0xc9,0x5c,0xbe, +0x75,0xc1,0x58,0xae,0xb1,0xf3,0x42,0x2c,0xfc,0x2f,0xfb,0x18, +0xa1,0x5d,0x66,0xcd,0x68,0x82,0xde,0x98,0x1c,0xca,0x21,0x94, +0x43,0xe8,0x20,0xd8,0x5a,0xc4,0x1e,0x1e,0x26,0xe9,0x17,0xc9, +0x3c,0x01,0x43,0x0d,0x30,0x5b,0xa8,0x29,0xfd,0xb5,0x89,0x7f, +0x07,0xe2,0xad,0xb0,0x40,0x0f,0x0c,0xaa,0x8c,0x5b,0x73,0x26, +0xab,0xe1,0xd5,0x6c,0x71,0x1b,0xe3,0xfa,0x27,0x60,0x26,0x6b, +0x03,0xe6,0x4b,0x60,0x5f,0x03,0x43,0x00,0x89,0x0e,0xe7,0x3e, +0x35,0x09,0xdd,0xf9,0x3a,0x43,0xcb,0x96,0xb8,0x08,0xd9,0xfd, +0xe1,0x74,0x79,0x3d,0xf4,0xe5,0x39,0xfb,0x94,0xde,0xce,0x41, +0xd1,0x43,0x75,0x83,0xc5,0xa5,0x9c,0x13,0xd5,0x0e,0xa1,0x42, +0x9c,0x00,0x0d,0x26,0x14,0x0c,0xdd,0x0e,0x0d,0x3d,0x8e,0xef, +0xc0,0x74,0xf3,0x06,0x7e,0x60,0xec,0x71,0xf0,0x7e,0xa1,0xdf, +0xe2,0xa8,0x1f,0xfa,0xe1,0xc0,0xbf,0x5c,0x25,0x44,0x6a,0xd4, +0x7a,0x49,0x7b,0x83,0x9f,0xcb,0x13,0xff,0xda,0x52,0x23,0xf4, +0x5f,0x1b,0x69,0x04,0xf8,0x46,0x90,0x51,0x05,0x28,0xd3,0x04, +0x85,0x51,0xf3,0x24,0xc0,0x3d,0xe2,0x5b,0x34,0x61,0x5b,0xa1, +0x4f,0x2d,0x1b,0xb5,0x5e,0x10,0x2b,0xc9,0x9e,0x92,0x7b,0xc3, +0xb2,0x29,0x99,0x17,0x48,0xb3,0x29,0xa3,0xd0,0x82,0x2e,0x71, +0xe3,0x08,0xa8,0x7d,0x89,0xfb,0x49,0xc1,0xa9,0xdc,0x82,0x64, +0x2e,0x42,0xf5,0x5c,0x71,0xcf,0x02,0xd6,0x94,0x79,0xd7,0xa1, +0xbf,0x51,0x00,0x26,0xc7,0x41,0x40,0xb6,0xf5,0xaf,0x60,0x1e, +0x2e,0xfc,0xd4,0x5f,0xfb,0x53,0xff,0x86,0xf6,0x35,0xe6,0x24, +0xbb,0xca,0xc7,0x78,0x66,0xdb,0xb1,0x72,0xe5,0x7e,0x26,0x56, +0xee,0x17,0x91,0x34,0x59,0xfd,0x34,0x5a,0xb4,0xe4,0x30,0x07, +0x9f,0x62,0x3c,0xcc,0x22,0x68,0xe1,0x66,0x0a,0xad,0x9c,0xc3, +0x57,0x0e,0x5f,0xb9,0xef,0x4e,0xe5,0xd3,0x67,0x78,0xbe,0x64, +0x1a,0x71,0x4d,0x54,0x6b,0x02,0xb2,0x23,0x17,0xa2,0x75,0x87, +0x98,0x7a,0x5e,0x53,0x35,0x3b,0x9a,0xc7,0xc7,0x53,0x3f,0x3b, +0x8a,0xa6,0x7e,0x8e,0x7f,0xb0,0x00,0x40,0xe8,0xde,0x44,0x69, +0xeb,0x2e,0xc6,0xad,0x9c,0x63,0x98,0xb1,0xf2,0x57,0x43,0x46, +0x81,0x0d,0x89,0xc5,0x47,0xd8,0x05,0x80,0x9a,0x06,0x67,0x08, +0x9b,0x37,0x20,0x03,0xad,0xbc,0x81,0xf1,0xd4,0xd7,0x20,0x92, +0xc9,0x11,0xad,0xe4,0xd1,0x33,0x6f,0x28,0x26,0xfd,0x60,0x70, +0x8c,0x4f,0xd1,0xd1,0x77,0x88,0xdf,0x64,0xe8,0xdf,0x78,0x9e, +0xb9,0x8c,0xbe,0x88,0xe4,0x6e,0x93,0xbf,0xc0,0x62,0xc0,0xa2, +0x81,0x5f,0x0f,0x4a,0x5d,0x60,0x31,0x6e,0x0e,0xc1,0x1c,0x82, +0x58,0x16,0xcb,0x04,0x65,0x2f,0x88,0xce,0xa7,0x63,0x98,0x7c, +0x6e,0x68,0x5e,0xf5,0x6b,0x54,0xdc,0xf9,0x24,0x85,0x84,0x2e, +0xcc,0xd4,0x1d,0x84,0xe8,0x7b,0x96,0xb5,0xe8,0x6a,0x32,0xf0, +0xde,0xa6,0x2c,0x9e,0xdc,0xdf,0xff,0xca,0xdb,0x07,0xae,0x54, +0x6b,0x0b,0x4f,0x7a,0xda,0xe8,0x7c,0x16,0xf5,0x94,0xaf,0xe7, +0x02,0xf9,0xd0,0x9c,0xcb,0x0c,0x2b,0x80,0x1f,0xb9,0xd7,0xc1, +0xb8,0x66,0xe4,0x62,0x6c,0x93,0xfe,0x46,0x58,0x47,0xef,0x48, +0x76,0x21,0x04,0xe5,0xf4,0x9c,0xd2,0x31,0x5f,0x6e,0xa4,0x7b, +0x5d,0x16,0x52,0x14,0x6b,0x29,0xa5,0xdc,0x2d,0x0d,0x31,0xc5, +0x8d,0x62,0x96,0x7e,0xee,0x1c,0xfb,0xd9,0x0a,0x04,0x9a,0x71, +0x46,0xaf,0x43,0x41,0x89,0xa6,0xe0,0xbf,0xb1,0x0b,0x0d,0x83, +0x2a,0x34,0xc2,0x55,0x80,0xd1,0xd8,0x0b,0xda,0x3b,0x07,0x44, +0x84,0x54,0x48,0x30,0x64,0x92,0xf2,0x79,0xe9,0x46,0x91,0x87, +0x28,0x52,0x39,0x2a,0x6e,0xd6,0xc8,0x7a,0x59,0xa7,0x61,0x11, +0x2c,0xa1,0x65,0x99,0xfa,0x06,0x93,0x12,0x09,0x06,0x55,0x4a, +0x99,0xed,0x4d,0x95,0xca,0xb1,0x17,0x55,0x12,0x5a,0x52,0xa5, +0xb4,0xee,0x5e,0xe5,0x29,0x68,0x8d,0x54,0x46,0x49,0x84,0x42, +0xd1,0xee,0x85,0x4e,0xc0,0x3e,0x48,0xb8,0x18,0xcd,0x44,0x09, +0x52,0x7f,0xef,0x55,0x84,0x04,0xd6,0xc8,0x44,0x8c,0xc4,0xc6, +0x43,0xc0,0x5e,0xb8,0x18,0x54,0x63,0xa2,0xb0,0xc4,0xc3,0xaa, +0x7d,0x2f,0x3c,0x19,0xe2,0x20,0xf8,0x1e,0xe7,0xca,0xce,0x03, +0xa8,0x6f,0xd0,0xc9,0xc8,0xbf,0x8f,0xfb,0x26,0x0d,0x97,0xb2, +0x73,0x8a,0xb1,0xd3,0x1c,0x44,0x32,0x5a,0x78,0x6d,0xd1,0x93, +0x81,0xae,0x1e,0xa5,0x3c,0x4f,0x52,0xc7,0x9d,0x58,0x76,0x1a, +0x7a,0x0e,0xa1,0xca,0xca,0x22,0x67,0x0f,0x9a,0x52,0x03,0xa6, +0x39,0xde,0xfb,0x42,0x83,0xb2,0x5e,0x57,0x81,0x29,0x07,0x6a, +0xf0,0x11,0x6e,0x51,0xc8,0x2f,0x37,0xf1,0x68,0x1e,0xec,0xcf, +0x54,0x94,0xb1,0xd4,0x42,0x07,0x2d,0x0c,0x27,0x0e,0x0d,0x83, +0x67,0xe6,0x54,0x39,0x03,0x4f,0x0e,0x99,0x99,0x75,0x87,0xe1, +0x6a,0x38,0x9f,0x2c,0x66,0xae,0x77,0x44,0x3d,0x56,0x11,0xaa, +0xca,0x66,0x07,0x11,0x85,0x93,0x29,0xd5,0x34,0xf6,0x07,0x74, +0x16,0x4e,0x1e,0x5b,0x33,0xcb,0x96,0x0b,0x29,0xef,0x69,0x04, +0xe5,0x74,0x32,0xb8,0xe4,0x30,0xfa,0x7e,0xd0,0xd5,0xf0,0x72, +0x38,0xb5,0xfc,0x97,0xe5,0xc0,0xea,0xf9,0x26,0xa0,0x98,0xc0, +0xd9,0x80,0x9c,0xdb,0xdb,0xea,0xa5,0x41,0x99,0x21,0x19,0x90, +0x51,0x85,0x2b,0x43,0x2a,0x12,0xcc,0x80,0x9d,0x43,0xbf,0x7f, +0x27,0x47,0x41,0xff,0x5a,0x8e,0x81,0x25,0x4f,0x4d,0x6b,0xdc, +0xe1,0xa3,0x84,0x60,0x5f,0x88,0x21,0x18,0xe0,0x76,0x98,0x0f, +0x0e,0x6f,0x71,0x19,0xd7,0xc2,0x8a,0x0a,0x80,0x44,0xca,0x4f, +0x81,0x5c,0x06,0x07,0x5e,0x55,0x2e,0xe6,0x05,0xe5,0x12,0x33, +0x7f,0x99,0x8b,0x83,0x90,0xab,0xd1,0x50,0xbc,0x65,0xd2,0x28, +0x46,0x30,0x91,0x63,0x4a,0x55,0x33,0x86,0x46,0x16,0x2b,0x1c, +0xf2,0xa8,0x14,0x74,0x0d,0x17,0x32,0xec,0x64,0x8e,0x7f,0x07, +0x5d,0x40,0x42,0xe4,0x0c,0x91,0x1b,0x10,0xb9,0xe3,0x5f,0x1b, +0x10,0x4b,0x81,0x44,0x8c,0xa1,0x3a,0x5e,0x64,0x15,0x63,0x27, +0x1d,0x0a,0x90,0x16,0x10,0x8a,0x58,0x85,0xce,0x32,0x35,0xba, +0x57,0xe4,0xb4,0x69,0x3e,0x45,0x0d,0x13,0xd4,0xf0,0x0b,0x53, +0x8d,0x91,0x14,0x1c,0x35,0x2a,0xb1,0x98,0x48,0x90,0x3e,0x65, +0xc0,0xed,0xa0,0x4f,0xaa,0xff,0x12,0x7d,0xa6,0xf5,0xc7,0xf4, +0x49,0xd0,0xc7,0xd0,0x67,0x61,0x31,0x91,0x68,0x73,0xab,0x8e, +0x3a,0xc3,0x18,0xb3,0x3c,0x95,0x39,0x13,0x51,0xc5,0xdf,0x06, +0x3d,0x5a,0x22,0x6a,0x1c,0x5b,0x8c,0xdc,0xfc,0x69,0x9b,0xf5, +0x6a,0x8a,0x63,0x4d,0x79,0xaa,0xd6,0x9c,0xb5,0x8a,0x77,0x5b, +0xa1,0xa7,0xb5,0xea,0x62,0xf9,0x00,0x74,0xa0,0x80,0x71,0x0a, +0x55,0xb5,0x25,0x09,0xd1,0xf7,0xf7,0x2e,0xa5,0x82,0x3e,0x1f, +0xc5,0xd7,0x30,0x3c,0x2d,0x56,0x04,0xee,0x7a,0xe2,0x26,0x61, +0xeb,0x1a,0x8a,0x16,0x8a,0x31,0x26,0x93,0x52,0xe7,0x09,0xb4, +0xe3,0x17,0x2c,0xd8,0xcf,0x65,0xba,0x06,0x57,0x90,0x62,0x2a, +0xa8,0x53,0x60,0x86,0xb2,0x03,0x0d,0xa4,0x42,0x87,0xa9,0x1a, +0x0e,0x93,0x94,0x50,0xcc,0x16,0x9b,0x34,0x06,0x84,0xab,0x1a, +0x24,0xdf,0xc8,0x74,0xcf,0x00,0xdf,0xd4,0x51,0xfe,0x8d,0x48, +0xf6,0xb8,0xa1,0x91,0x3e,0xcf,0xbf,0x5d,0x24,0x93,0x27,0x81, +0xd7,0xc1,0x10,0xad,0x1e,0xac,0x62,0x5c,0x0c,0xe5,0x81,0x8f, +0x67,0x9b,0xbe,0x03,0xe2,0xa4,0x5d,0x1a,0xeb,0xa6,0x8c,0x51, +0x3b,0x28,0x96,0x68,0x4e,0x14,0xa3,0xb0,0xab,0xd7,0x26,0xae, +0x93,0x78,0x35,0x5c,0x8d,0xaf,0xf3,0x9a,0xf5,0x89,0xf1,0x66, +0x25,0xd6,0x27,0xfc,0x49,0xbc,0x04,0x1d,0xcd,0x33,0x03,0x3e, +0xb1,0x82,0x7e,0x77,0x69,0x44,0x3f,0x80,0x9a,0x2b,0xa3,0x10, +0xfa,0x3a,0x13,0xfb,0x9e,0xe1,0x54,0x9e,0x82,0x91,0x81,0x89, +0x47,0x12,0x5a,0xcb,0xf0,0x19,0x1f,0xe8,0xa4,0x39,0xe3,0x54, +0x1e,0x38,0x7a,0xa4,0x46,0xd7,0x38,0x9a,0x13,0x8d,0x49,0x57, +0xb6,0x64,0xd9,0xd8,0xe3,0x6e,0x69,0x4b,0x80,0x70,0x37,0xd4, +0x74,0x63,0x12,0xc9,0x3a,0x30,0x4a,0xec,0x4a,0xef,0x45,0x25, +0x70,0x55,0x60,0xb8,0xc2,0x59,0x3c,0x91,0x3a,0x96,0x47,0x17, +0xfd,0xdb,0x46,0x34,0xe1,0xa5,0xca,0x2e,0x2e,0x26,0x1d,0x1e, +0x8e,0x79,0x61,0x14,0xff,0x78,0xbc,0x8e,0x8c,0x54,0x22,0x01, +0x14,0x88,0x6e,0x85,0x53,0x63,0x45,0x52,0x43,0x5c,0xa2,0x51, +0xcb,0x9b,0xfb,0xfb,0xc0,0x33,0x57,0x55,0xcc,0xc5,0x59,0xca, +0x6b,0x70,0xdf,0xe0,0x3b,0x56,0xda,0x62,0x01,0xd6,0x9f,0x75, +0x10,0x87,0x35,0x1f,0xb1,0x6d,0x24,0x27,0x35,0x23,0x89,0x8d, +0x25,0x7e,0x41,0xd5,0x65,0xb1,0x32,0x2f,0xb1,0x4c,0x68,0x1f, +0x59,0xc1,0xdb,0xc7,0xd5,0xab,0x92,0x65,0xfe,0xad,0xae,0xaa, +0xc2,0x80,0x8b,0x37,0x44,0x9f,0x58,0xad,0x90,0x18,0x64,0x2b, +0x4e,0xfc,0x40,0xb6,0x1e,0xcf,0xbc,0x78,0x61,0x9e,0xef,0x90, +0x17,0x12,0xaf,0x90,0xbd,0x81,0x78,0x5f,0x71,0xab,0x02,0xe0, +0xb5,0xe2,0x1d,0xd1,0x54,0x01,0xf1,0x13,0x49,0x81,0x20,0x42, +0xc5,0x92,0x38,0xec,0xbf,0x64,0x09,0x33,0x34,0x95,0x95,0xd7, +0x27,0xb7,0x3a,0x42,0xcb,0xf0,0x9e,0x13,0x0a,0x01,0x6d,0xe1, +0x94,0xb1,0x26,0x5e,0xae,0xd3,0x5e,0x48,0x19,0xd4,0xc2,0x48, +0x51,0x26,0x3a,0x21,0x16,0xe6,0x2b,0x0e,0x0b,0xed,0x7f,0x2d, +0x85,0x86,0x9c,0xfb,0x02,0x8f,0x3d,0xf0,0x74,0x6e,0x73,0x39, +0x14,0xfa,0xdc,0x3a,0xb1,0x27,0x64,0x55,0xc6,0x3a,0x89,0x52, +0xe6,0xd3,0x2d,0xaa,0xbb,0xe4,0x5e,0xe9,0x16,0xf2,0x27,0xa6, +0xd5,0xce,0x48,0xeb,0x1a,0xbc,0x5c,0x9a,0x43,0x19,0x5e,0xfd, +0xcc,0x9f,0x79,0x04,0xb1,0xbb,0xfb,0x09,0x0b,0x21,0x1d,0x0e, +0xa7,0xd5,0x0b,0x5a,0x41,0xc9,0x22,0x2d,0xc0,0xbd,0x49,0x76, +0xac,0x43,0x1d,0x75,0x62,0x5f,0xd1,0x3f,0x8e,0xcc,0x0e,0x04, +0xb5,0x88,0xc6,0x0c,0x7b,0xc4,0x55,0xc2,0xfd,0xa7,0x89,0xb9, +0x7c,0x89,0xe4,0xba,0x7a,0x55,0xb7,0xa6,0x97,0x4f,0x1e,0xdf, +0xcb,0xb5,0x2e,0xa4,0xc9,0xc7,0x6c,0x88,0xde,0xa0,0x5c,0x9c, +0x41,0xa2,0xde,0x10,0x0e,0x1b,0x13,0x4d,0x96,0x6a,0x2d,0xde, +0x90,0xd4,0x7d,0xd3,0x10,0x60,0xcb,0xeb,0x5f,0x9f,0x98,0x32, +0x1a,0x92,0x9e,0xbd,0x0e,0x7c,0x69,0xde,0xcb,0x05,0x2e,0x5d, +0x67,0x04,0xf0,0xcc,0x3e,0xad,0x07,0xa6,0x8a,0xc1,0xca,0x35, +0x97,0x55,0xf5,0x3e,0xb6,0xa0,0xf4,0xf1,0xeb,0x27,0x2a,0xab, +0x5c,0x43,0xa9,0xd0,0x04,0x3f,0xf0,0x70,0xac,0x40,0x8d,0xce, +0x62,0x49,0x77,0x52,0xbd,0xd4,0x0f,0xf1,0xc6,0x79,0x1e,0x12, +0x84,0xaa,0x33,0xc7,0xc6,0xc1,0x49,0x53,0xc3,0x42,0x6e,0x62, +0x2e,0x5d,0x33,0xd3,0xa8,0x7a,0x86,0x4d,0x6f,0x7b,0xf6,0xf4, +0x67,0xcd,0x53,0x17,0x1a,0x80,0x36,0x86,0xab,0xd0,0xc9,0x7d, +0xd3,0x9e,0xfc,0xa8,0x41,0x25,0x93,0xbd,0xe6,0x10,0x2f,0x46, +0x6b,0x81,0x75,0xc9,0xda,0x24,0x15,0x57,0xea,0xe8,0x3d,0x7f, +0xc3,0x55,0xdc,0xeb,0x0a,0x27,0x15,0x1a,0x5d,0x13,0xbd,0xbf, +0x4e,0xda,0x32,0x32,0x14,0x31,0x54,0x1c,0xb5,0xf8,0x67,0xac, +0x88,0x7b,0x8a,0x4f,0x30,0xb1,0x54,0xcd,0xca,0xe5,0xc1,0xcc, +0xb6,0x29,0xbe,0x60,0xf2,0xd9,0xa9,0x84,0xa4,0x32,0x94,0x37, +0x3a,0xfe,0xbe,0x1f,0xc8,0x4b,0xac,0xa0,0x29,0x53,0xeb,0xad, +0x8e,0xca,0x5d,0x79,0xd1,0x05,0x26,0x5d,0x04,0x47,0xa3,0xfb, +0x03,0xa2,0xe9,0x48,0x9c,0xac,0x0c,0x26,0x91,0x22,0xc4,0xda, +0xc6,0x36,0x76,0xb1,0xa1,0xef,0xe3,0x86,0x06,0x1a,0x9d,0x63, +0x16,0x78,0xd6,0xae,0xc2,0x93,0xb3,0x30,0xa2,0xd9,0x5c,0x30, +0x70,0x46,0x7a,0x87,0x3b,0xfa,0xfd,0xdf,0x55,0x4f,0x48,0xe2, +0x8f,0xd1,0xfe,0x90,0x4d,0xe9,0x7d,0x44,0xf1,0xe8,0x41,0x8e, +0x10,0xf0,0xf0,0x46,0xf9,0x35,0x99,0xfb,0x61,0xd1,0xf0,0x12, +0x97,0x8a,0x61,0x8c,0xaa,0xae,0x7b,0xe1,0x53,0xd0,0x02,0x9b, +0x0c,0x13,0xae,0xa2,0x15,0x6d,0xb3,0x38,0xfa,0xb0,0x35,0x8c, +0x64,0xbe,0xb2,0xab,0xaa,0x57,0x53,0x0a,0x49,0x14,0x4b,0xbb, +0x70,0x14,0x88,0x57,0xc9,0x43,0xbd,0x5b,0x48,0x2d,0x77,0x6f, +0x42,0x23,0x3b,0xb8,0x38,0xbf,0xc5,0x48,0xea,0x8e,0x46,0x4f, +0x0a,0x27,0xc0,0x35,0xe2,0x7e,0x66,0xe2,0xbc,0xa5,0x33,0x0e, +0xc5,0x18,0x3e,0x4f,0x4d,0xc7,0xb0,0xd3,0x88,0x4f,0x70,0x1b, +0x10,0xe2,0xb4,0x13,0x57,0xa8,0x2b,0x2b,0x73,0xd2,0x5a,0xc6, +0xab,0x19,0xfa,0x90,0x71,0x8c,0xc8,0xae,0xe8,0x35,0x00,0x04, +0x02,0x01,0xd0,0x95,0x0b,0x41,0x0b,0x7f,0x71,0x79,0x99,0xc6, +0x00,0xcb,0xbf,0xb5,0x65,0xfa,0x73,0x59,0x7b,0xb5,0x4e,0xc4, +0x61,0xbc,0x48,0xcb,0xd8,0x26,0x32,0x17,0xb9,0x30,0x4c,0xbe, +0x4b,0x1a,0xa7,0xca,0x83,0x9e,0x29,0x0b,0x51,0x28,0xea,0x09, +0x5f,0x5d,0xcb,0xc5,0x37,0x5c,0x79,0xbe,0x79,0xdd,0x59,0x35, +0x16,0xf6,0x6f,0x59,0x34,0xd4,0x11,0xf0,0x66,0x28,0x72,0xfb, +0xfa,0x98,0x3e,0x63,0x53,0xc7,0xdd,0xf0,0x34,0x9a,0xa9,0xed, +0x4d,0x57,0x51,0x71,0x14,0x5f,0x8a,0x1f,0x95,0xf5,0x1d,0x46, +0x7e,0x2e,0x9c,0x28,0x99,0xdc,0x12,0x08,0xc5,0xfe,0x1c,0xaf, +0x16,0xe8,0x6d,0x5a,0x4a,0xdd,0x88,0xb3,0x0e,0x66,0xa4,0x38, +0xa5,0x60,0x46,0xa9,0xfb,0x9f,0xf8,0x26,0x2c,0x41,0xdf,0xfe, +0xca,0x02,0x37,0x2e,0x28,0xab,0xd0,0x17,0xe9,0x96,0x83,0x2d, +0xae,0xd6,0x5e,0x88,0x18,0xb4,0x66,0x61,0xbc,0x8a,0x31,0x29, +0x39,0x92,0x66,0xec,0x48,0x5a,0xcb,0x36,0x41,0x16,0x61,0x57, +0x54,0x31,0x2b,0xf7,0x23,0x8b,0x61,0xf7,0xa4,0x8b,0x45,0xaa, +0x96,0x30,0xdd,0x58,0x92,0x2c,0x8a,0x91,0x74,0x19,0xde,0x29, +0xbf,0xed,0x5c,0x4c,0xff,0x4d,0xe6,0xcf,0x64,0xd6,0x7c,0xbf, +0xac,0xb9,0xce,0x9a,0x2b,0x6e,0x6c,0xd6,0xfb,0x65,0x46,0x40, +0x95,0x1d,0x02,0x02,0x41,0x51,0x73,0x1a,0x2d,0x64,0xba,0x46, +0xe3,0x6d,0x3f,0xec,0xf9,0x5c,0x71,0x9e,0x99,0xd6,0x85,0xad, +0xc3,0x63,0x60,0x95,0x6a,0x85,0x59,0x40,0xfe,0xcd,0x30,0xfb, +0x8a,0x15,0x43,0x8a,0x3e,0x85,0x75,0x60,0x3f,0xc4,0x93,0xcd, +0x38,0x7e,0xb3,0x99,0x95,0x4c,0x8f,0xf9,0xae,0xcb,0x2d,0xa0, +0x64,0xbc,0xda,0x02,0x7f,0x46,0x74,0xbd,0xcf,0x62,0x89,0xd7, +0x2f,0xf1,0x69,0xc0,0x00,0x83,0xb4,0xa0,0xc7,0x11,0xa9,0xbe, +0x98,0x4d,0xaf,0xc1,0xbd,0x8f,0x12,0xe9,0x18,0xc7,0x50,0xe7, +0x90,0xa9,0xe7,0xc2,0x9f,0x46,0x84,0x14,0xa3,0xe6,0x40,0x34, +0xbc,0xc0,0xf0,0x1e,0x4f,0xfa,0x30,0x9c,0x4e,0x16,0xd8,0x15, +0x84,0xec,0xa0,0x32,0x5e,0xbf,0xce,0x22,0x1e,0x77,0x41,0x7c, +0x78,0x68,0x8b,0xe3,0x8b,0x3c,0xd6,0x07,0x6e,0xcb,0xb6,0x97, +0xc6,0xb5,0xf5,0x1d,0xe1,0xb2,0xed,0x54,0xf6,0x38,0xd1,0xa7, +0xd8,0xff,0xbd,0x42,0xee,0x55,0x73,0xa7,0xc9,0xf4,0x7a,0xb1, +0x89,0xd7,0xeb,0x12,0x21,0x5a,0xf7,0x17,0x9a,0x9b,0xcf,0xe0, +0x29,0x7d,0xce,0x2d,0xdb,0x1f,0x90,0x1c,0xd0,0xf3,0x24,0x55, +0x1b,0x19,0xf2,0xd2,0xcb,0x85,0x60,0x3e,0x5e,0xea,0x07,0x2a, +0x59,0x58,0x5f,0xac,0x7e,0xbb,0x8b,0x73,0xc0,0x01,0xf3,0x22, +0xc4,0xb4,0x60,0x91,0x11,0xfb,0x15,0xde,0xb6,0x88,0x31,0x0f, +0x70,0xcf,0x04,0x41,0x9b,0xa2,0x25,0xbc,0xe3,0xb6,0xe4,0x7d, +0x1e,0x6c,0xfd,0xbb,0xe4,0xea,0x6a,0x5a,0x5b,0x2b,0x18,0x1d, +0xe8,0x56,0x9c,0xf4,0x04,0xe7,0xa9,0xa0,0x92,0x83,0x42,0x35, +0x33,0x59,0x4d,0xa8,0xb0,0xf2,0x28,0xd3,0x95,0xcb,0xf1,0x96, +0x90,0x08,0x86,0xa6,0x80,0xd6,0xae,0xc2,0x72,0x4d,0xd3,0xd0, +0x90,0xb3,0x34,0x2c,0xd6,0x55,0x81,0xd1,0x5b,0xb7,0x30,0x03, +0x16,0x77,0x3a,0xc2,0x6f,0x93,0x4e,0x57,0xe9,0x03,0x90,0x08, +0x8a,0xae,0xb4,0xe9,0x49,0xe4,0x5a,0x48,0x9a,0x2a,0xd4,0xe4, +0x43,0x58,0xc7,0x6e,0x1b,0xe6,0xa8,0x5e,0x37,0x39,0xbf,0x81, +0xac,0x37,0x5e,0x7a,0xd2,0x10,0x59,0x6e,0xac,0x2c,0x37,0x46, +0x96,0x49,0xd6,0x4d,0xdb,0x20,0xce,0x27,0x47,0x16,0xee,0x2d, +0x33,0x78,0xd1,0xa4,0xbb,0x7a,0xda,0xc7,0x69,0x08,0x98,0x3b, +0xd0,0x80,0xc1,0xf9,0x02,0x37,0x1f,0x82,0xa8,0xae,0x51,0x9a, +0xc0,0x14,0xb3,0x21,0xe2,0x6c,0x39,0x34,0xdf,0xb8,0x7e,0xa4, +0x78,0xdd,0x44,0xe1,0xf1,0xfc,0x0f,0x89,0x15,0x4c,0xba,0x17, +0xd6,0xda,0x3d,0x42,0x59,0x20,0xc7,0xd1,0xa2,0xbc,0x46,0x5f, +0x82,0x8a,0x6e,0x6a,0xa4,0xd0,0xaa,0xee,0x6f,0x30,0x68,0x54, +0x8f,0x36,0x38,0x9c,0xf0,0xd9,0x0f,0x35,0x2b,0x4e,0xd7,0x78, +0x11,0xf6,0xac,0xd2,0x42,0x55,0xa9,0xc2,0x4a,0xd5,0x6b,0xa9, +0x30,0x9d,0xf5,0x33,0x1f,0xfe,0x52,0xbf,0x2b,0xce,0x48,0x29, +0xbc,0xf2,0xf9,0x2a,0x3b,0x52,0x22,0x11,0xfd,0x5d,0x19,0xd3, +0x7d,0xce,0x83,0x16,0xe6,0xfa,0x7a,0x15,0xa7,0xd7,0x0b,0x5c, +0x0e,0x06,0x84,0x73,0x0b,0x8a,0xb2,0x19,0xb0,0xc2,0x79,0x5d, +0xd8,0xbf,0xaa,0xc1,0x34,0x0a,0x75,0x3d,0x18,0xb4,0xda,0xe5, +0x2a,0xfe,0x75,0x13,0xcf,0xc7,0x79,0x2f,0xec,0x40,0x0b,0xd2, +0x52,0xc6,0xac,0xeb,0x41,0x21,0x58,0x10,0x5f,0x4a,0x30,0xc0, +0x4a,0xe0,0x02,0x8f,0x46,0xd1,0x4f,0x1a,0x20,0xa5,0x2e,0xc6, +0xdb,0xd1,0x60,0xeb,0x61,0x64,0xce,0xab,0x28,0xb3,0xf3,0xc0, +0x2b,0x3a,0xa8,0x64,0xc6,0xe4,0x2d,0x3b,0xe7,0x4a,0x83,0x20, +0x1d,0x1e,0x66,0xaf,0x44,0x00,0xef,0xb4,0x75,0x15,0x05,0xb8, +0xa3,0x91,0xd0,0x65,0x75,0xba,0x1c,0xe0,0x6b,0xe8,0xcf,0xe8, +0x66,0x3e,0x2a,0xad,0x11,0xdd,0xd0,0x87,0xbe,0xb8,0xd3,0x5c, +0x6f,0x41,0x34,0x64,0x27,0xaa,0xca,0xe2,0x5d,0x13,0xdc,0x00, +0x72,0x42,0xc8,0xbc,0xb7,0x56,0x28,0x45,0xbb,0xfe,0x40,0xec, +0x65,0xae,0x57,0xa5,0x7f,0x96,0xcc,0xdf,0x40,0x31,0x57,0x71, +0xaa,0x97,0x31,0x45,0xd2,0xa3,0x27,0x89,0x2b,0x5a,0x23,0x14, +0xb9,0x85,0xed,0xb7,0xda,0xea,0x28,0x1e,0x6d,0xf6,0x43,0xa8, +0x6b,0x64,0x1c,0x5a,0x37,0x90,0x73,0xba,0x89,0x9c,0x44,0x75, +0x2f,0xdc,0x82,0x1b,0xa5,0xad,0xdb,0x4c,0x2f,0x20,0x10,0x7a, +0xf3,0xc0,0x55,0x05,0xdf,0xbe,0x40,0xf7,0x24,0x86,0xf4,0x33, +0x6f,0xdb,0xa9,0xa5,0x95,0xcb,0x33,0x69,0xd5,0x4d,0xb9,0xa7, +0xd7,0x89,0x6a,0xf9,0x03,0x8b,0xc5,0x2a,0xc1,0x40,0x6e,0x2e, +0x37,0xd9,0xf7,0x54,0x18,0xfd,0x9e,0x6f,0xa4,0xa0,0xfe,0x2e, +0xa3,0x2e,0x93,0x55,0xba,0xfe,0xdb,0x70,0x7a,0xc3,0x6b,0xab, +0x4b,0xb0,0x03,0x92,0xc5,0x26,0x7d,0x93,0x8c,0xa6,0xea,0x60, +0x52,0xcd,0xaa,0xa3,0x38,0xbe,0x42,0x91,0x17,0x88,0xfa,0xe1, +0x35,0x48,0x63,0xcf,0x46,0x96,0x44,0xcb,0xed,0xbc,0xf0,0xea, +0x13,0x31,0x14,0xa1,0xd7,0x49,0x03,0xbc,0xe5,0x6c,0x1c,0xa7, +0x74,0x77,0xbe,0x4a,0xb7,0xd6,0x30,0xc7,0x76,0x0e,0xe8,0xa3, +0xba,0x56,0x8c,0xd8,0x2a,0xcc,0xd3,0x08,0x87,0xcb,0x25,0x98, +0x80,0xb4,0x32,0x5b,0x86,0x53,0x60,0x9e,0x1d,0xcf,0x25,0x75, +0xb5,0x78,0x60,0xdd,0xdf,0x5c,0x27,0x97,0x6b,0x5e,0xa9,0x25, +0x53,0x78,0x96,0x4c,0xf8,0xfe,0x0a,0xbc,0x58,0x42,0xd3,0xcd, +0x8c,0xc2,0x63,0x5c,0xd3,0x64,0xd6,0x18,0x97,0xa2,0xbc,0x6e, +0xa1,0x05,0x7a,0xae,0x5c,0x43,0xa4,0xf4,0xa8,0x90,0x6c,0xe3, +0x4b,0xe3,0xe5,0x70,0x35,0xd4,0x2b,0xe5,0xc5,0xd6,0x14,0x4d, +0xd6,0x9a,0x2d,0x26,0x91,0x85,0xb6,0x29,0xc9,0xf5,0x3a,0x76, +0x71,0x32,0x9e,0xb7,0xb2,0x0a,0xf8,0xd0,0x8b,0xf2,0xcf,0x23, +0xce,0x38,0x81,0x00,0x9a,0x73,0x31,0x9f,0x68,0xa9,0x84,0x5e, +0x22,0x56,0xf3,0xb5,0xb0,0x49,0xb4,0x59,0x77,0xf7,0x36,0xc0, +0x5e,0x4b,0xfa,0x6a,0x39,0x3f,0x6b,0x98,0x45,0x00,0x9f,0xa4, +0x8c,0x19,0x24,0x59,0x4b,0xf8,0xe6,0x13,0x9d,0x5a,0x9a,0xaa, +0x2a,0xa8,0xc5,0x09,0x1d,0x79,0x4a,0x1d,0x4d,0xf6,0x8c,0x14, +0x05,0xc9,0xbf,0x4d,0x96,0xbc,0x01,0x79,0xbb,0x90,0x1f,0x65, +0xfe,0x42,0xe2,0x8c,0xc9,0xe5,0x2d,0xcb,0x96,0xd9,0x69,0x52, +0xc0,0x00,0x58,0x74,0x4d,0xfc,0x14,0x50,0x01,0x3a,0x33,0x06, +0x70,0x02,0x5e,0x0b,0x06,0x62,0x16,0x46,0x4c,0x17,0xcb,0xb5, +0x45,0xfd,0x07,0x74,0x54,0x71,0x21,0xde,0x23,0x32,0xed,0xc4, +0xaf,0xe3,0x4b,0x4c,0x5b,0x62,0xda,0xec,0xf0,0x10,0xbe,0xba, +0x1e,0x92,0x59,0x05,0xb5,0x40,0x0c,0x8b,0x22,0x06,0x81,0x7e, +0xb1,0xa4,0x44,0x41,0x89,0xea,0xb1,0xc4,0x0b,0xe2,0x91,0x41, +0xb9,0x94,0x30,0x10,0x64,0x5d,0x65,0x21,0xd9,0xc0,0x07,0x53, +0xf4,0x20,0x93,0x4f,0xe4,0x11,0x0e,0x72,0x3c,0xb3,0x8b,0xff, +0x86,0xbc,0xc5,0xad,0xa8,0x4f,0x45,0xe1,0x94,0x99,0x08,0xd0, +0x9a,0x41,0xd3,0x83,0xdb,0x0f,0x38,0xb7,0xa3,0xf6,0x4b,0x17, +0xf2,0x13,0xa3,0x61,0x4e,0x57,0x60,0x33,0x42,0x16,0x5b,0x07, +0xa1,0x16,0x76,0x63,0x20,0x94,0xd9,0x62,0x5d,0x62,0xea,0x41, +0x1d,0xc3,0x70,0x13,0x56,0x81,0xa3,0xe5,0x31,0x9c,0x44,0x54, +0x61,0x13,0x07,0xd0,0x85,0x6c,0x21,0xfe,0x26,0xcb,0x12,0x36, +0xd5,0x34,0x84,0x6c,0x56,0x44,0xb6,0xf4,0x2d,0x1a,0x11,0xd9, +0x12,0x90,0xcd,0x7c,0xab,0x85,0xd4,0xb9,0x50,0x19,0xbb,0x7d, +0x78,0x8f,0x88,0xb6,0x07,0x23,0xb9,0xef,0x53,0x50,0xaf,0x74, +0xc1,0x08,0xbd,0xe5,0xc1,0xbb,0x89,0xfa,0x74,0x73,0xe5,0xa8, +0xa5,0xfb,0x70,0xf4,0x41,0x92,0xd0,0x91,0xa0,0x20,0x15,0x30, +0xdb,0x00,0xe2,0xf1,0x18,0xe7,0x35,0x0e,0xe4,0xf0,0x41,0x6d, +0x05,0xbf,0x6c,0x1d,0x74,0x8a,0x5a,0xb8,0x5a,0xb3,0x31,0x70, +0x23,0xec,0x04,0xb8,0x6a,0xaf,0x07,0x1d,0xda,0x05,0xf5,0x0d, +0x95,0x41,0x34,0x37,0xe9,0x92,0x12,0x5b,0xdb,0xd3,0x7d,0x64, +0xc0,0xf1,0x42,0x1f,0x78,0x13,0x23,0x87,0xc4,0xad,0x28,0xa5, +0xe6,0x99,0x2d,0xd2,0xb5,0xbc,0xa6,0x68,0xdf,0x7c,0x24,0x24, +0x9c,0x71,0x12,0xc7,0xc5,0x3e,0x57,0x9f,0xef,0x73,0x00,0x8e, +0x31,0x57,0x16,0x44,0x48,0x28,0x7a,0xd5,0xeb,0xde,0x84,0x31, +0x4c,0x08,0x9e,0xf0,0xca,0xc2,0x88,0xbe,0x5b,0x99,0xd9,0xe3, +0x28,0xca,0x47,0x48,0x84,0xc9,0xc3,0x08,0xcb,0xe7,0x8d,0xd8, +0xfb,0xfb,0xb0,0x5b,0x32,0xb6,0xaa,0x9a,0x1b,0x9b,0xee,0x4d, +0xf2,0x9b,0xb1,0xd7,0xc3,0x7b,0xa2,0x3c,0x2a,0x1c,0x29,0x17, +0xc1,0xf2,0x56,0xaf,0xd8,0x55,0x84,0x4a,0x4f,0xe3,0x75,0xfc, +0x44,0x0b,0x86,0xb1,0x1b,0x64,0xe2,0x8a,0xc4,0xbe,0x31,0x1d, +0x08,0x94,0x77,0x5b,0xd6,0xa2,0x3f,0xce,0xc3,0x1d,0x25,0x3c, +0x62,0xeb,0x92,0x97,0xa2,0x70,0x03,0xcc,0x6b,0x09,0x07,0x0f, +0xfc,0xf6,0x35,0x1f,0x4b,0x2d,0x26,0x13,0xcc,0xc3,0x24,0x92, +0x4f,0xd1,0x81,0x62,0x2c,0xdf,0x86,0xa2,0xd1,0xec,0xb7,0xeb, +0xa2,0xe1,0x33,0x9f,0xef,0x39,0xd1,0x51,0x5b,0xbf,0xe2,0x86, +0x95,0x87,0xbc,0xe3,0x89,0x2a,0xbe,0x2b,0x45,0x6c,0xb0,0x7a, +0x02,0xb1,0x6a,0x5b,0xba,0x40,0x85,0xb7,0x5a,0xb9,0x77,0xc9, +0x6c,0x8f,0x2f,0xe4,0xa0,0xb6,0x10,0x8c,0xa6,0x3b,0x5b,0x76, +0xed,0xe7,0x62,0x9e,0xba,0xad,0xdc,0xba,0x5d,0x9f,0x25,0x6d, +0xfa,0xec,0xb7,0x1d,0x6e,0xab,0xba,0x2a,0x17,0x57,0x7f,0x25, +0x98,0x85,0x4f,0x38,0x6e,0xd2,0x9e,0x3a,0x94,0xd2,0xa9,0x58, +0x84,0x16,0x30,0xfc,0x53,0x71,0x75,0x05,0x27,0x6c,0xd5,0xc5, +0x50,0x59,0x44,0x3f,0x79,0x14,0xf8,0x7b,0x6a,0x59,0x3a,0x3d, +0xb5,0x8a,0x1a,0x2b,0x57,0xf8,0x40,0x79,0xd6,0xfa,0x5f,0x55, +0x5e,0x63,0x17,0x0d,0x38,0x23,0xf4,0x26,0xee,0x0c,0xf1,0x09, +0x46,0xb1,0x65,0xba,0x92,0xc7,0x6d,0x8f,0x5c,0x51,0x8b,0xb0, +0xa3,0x1c,0x12,0xda,0x47,0x44,0xe7,0xea,0xf8,0xce,0x57,0x9f, +0x60,0xb3,0x1d,0xb7,0xf7,0x1d,0x1c,0x98,0xec,0x46,0x34,0x59, +0x7d,0x34,0xb9,0x8f,0x2a,0xa9,0x49,0x25,0x6d,0x4b,0x2a,0x0d, +0x51,0xaa,0x93,0xff,0x9c,0xf3,0x0e,0x74,0xe1,0x35,0xfc,0x53, +0xb5,0x16,0x27,0xf1,0x76,0xd4,0xdb,0xd0,0x26,0x9c,0xcb,0xdf, +0x4b,0xa9,0x14,0x6a,0x46,0xee,0x6a,0xea,0x78,0x6c,0xa5,0xcb, +0xc4,0xf8,0xe6,0x11,0xd7,0x27,0x11,0xe4,0x85,0xbf,0xa4,0x0d, +0x01,0xee,0xbd,0x84,0x81,0x49,0xdc,0x0f,0x87,0x80,0x15,0x37, +0x31,0xdd,0xdf,0x57,0x3e,0xe1,0x7a,0xd1,0x69,0xc8,0x62,0x84, +0x40,0x73,0x41,0xa5,0x33,0xb6,0x3b,0x9f,0x4f,0x16,0xc0,0x0a, +0x95,0x3a,0x50,0xbb,0xcb,0xbb,0x63,0x7c,0x53,0xa3,0x0d,0xc6, +0xd3,0x0d,0xbe,0x39,0x57,0xa9,0x10,0x44,0x9a,0xd6,0x09,0xd2, +0x86,0xc0,0x6b,0xda,0xfc,0x47,0x29,0x88,0x2c,0x0a,0x1e,0x63, +0x0f,0xed,0xf4,0x8c,0xd2,0xf3,0xa1,0xc2,0x64,0xa8,0x27,0x3d, +0xa6,0x74,0x49,0xa2,0x0a,0x6f,0xd5,0x94,0xc7,0x93,0x43,0x60, +0x09,0xe6,0x9d,0x86,0xf1,0x3a,0x12,0x93,0x59,0xe3,0x1e,0x4c, +0xad,0xea,0xa6,0x80,0x98,0xee,0x75,0x02,0x89,0x3c,0xb0,0x12, +0x84,0x41,0x59,0x69,0x2b,0x89,0xc2,0xc9,0x66,0x65,0xa3,0xab, +0x68,0x15,0x09,0x08,0xb6,0x91,0x19,0xe4,0xcf,0xb1,0x64,0xfe, +0x0c,0xc3,0x45,0x19,0x1b,0xac,0x96,0x4b,0x86,0x06,0xab,0x6b, +0x79,0x5e,0xd7,0x93,0xa6,0xc6,0x1f,0x30,0x5d,0xdc,0xb0,0x29, +0x90,0xf6,0x38,0xe6,0x98,0x43,0x9d,0xd0,0xd3,0xe8,0xff,0xaf, +0xb0,0x5b,0x44,0xc3,0xfd,0x31,0xd3,0x45,0x20,0xb1,0xad,0x17, +0x85,0xf9,0x0f,0x1b,0x30,0x0a,0x7d,0x95,0x0d,0x23,0x8b,0xf9, +0x73,0xcc,0x98,0x72,0x51,0x7b,0x59,0x32,0x22,0x5b,0x8d,0xfa, +0xc2,0x06,0x99,0x0d,0x97,0xd5,0x5e,0x2c,0x74,0x35,0x8e,0x5a, +0xec,0xbb,0xd1,0x2b,0x14,0xbc,0x54,0x86,0xaf,0xf9,0xc8,0x65, +0xfb,0x82,0x26,0x51,0x0f,0xfc,0xe0,0xb3,0x45,0x6e,0x69,0x75, +0x4e,0x48,0xca,0x91,0x1b,0x9c,0xdf,0xf4,0x82,0xce,0x8d,0x70, +0xf2,0xe6,0xe7,0x91,0xf8,0x20,0x0a,0x7e,0xa2,0xb3,0xf0,0x79, +0x84,0x5f,0x00,0x44,0x4f,0x18,0x69,0xda,0x7e,0xdd,0x0c,0x57, +0xc9,0x65,0xfe,0xb0,0xb6,0xb3,0x16,0x80,0x8a,0xab,0x3f,0x3a, +0xa3,0x9f,0x8e,0xf1,0x21,0xc9,0xb9,0xbf,0xc2,0x1b,0xf7,0x60, +0x4c,0x10,0xde,0xe7,0xab,0xc5,0x1d,0x6e,0x25,0xac,0x80,0x4b, +0xc9,0x1c,0xc7,0x0d,0x85,0x42,0x3e,0xd1,0x33,0x82,0xc9,0x11, +0xde,0x81,0xea,0x6f,0x22,0x87,0xe2,0xf0,0x5a,0x38,0xbc,0xe1, +0xaa,0x87,0xa8,0x5a,0x93,0xac,0xe3,0x4c,0xca,0xb1,0x79,0x87, +0x81,0x9b,0x56,0x5a,0x78,0xa8,0x67,0x16,0x0a,0x50,0xa2,0x61, +0xeb,0x20,0x01,0x2d,0xc4,0x11,0xbe,0x00,0x10,0x0f,0x98,0x51, +0x6b,0x29,0x3a,0x45,0x62,0x76,0x24,0x80,0x0c,0xaf,0x54,0xaa, +0x54,0x4b,0xbc,0x28,0xe5,0xce,0x23,0x95,0x47,0x72,0xe5,0x3c, +0xe8,0x7a,0x08,0x42,0xfb,0x12,0xdc,0x78,0x0a,0x86,0x1e,0x2e, +0xd2,0x18,0x1a,0x91,0x6e,0x39,0xdf,0x91,0xad,0xe2,0x1c,0x70, +0x7d,0xee,0xef,0x5d,0x62,0x6b,0x74,0xb7,0x80,0x89,0x31,0xa8, +0x9d,0x3b,0x7f,0xe3,0x79,0xaf,0x22,0x64,0x59,0x4f,0xd3,0xda, +0x5a,0x2e,0x96,0x92,0x91,0x04,0x0f,0xe3,0x87,0x2c,0xa0,0x19, +0x11,0x25,0x08,0xc0,0x85,0x28,0x07,0x52,0xc2,0x46,0xd5,0xf4, +0x0f,0x42,0x0f,0xb8,0x5f,0xcb,0x1f,0xa2,0x56,0xbc,0xa0,0xa4, +0xab,0xae,0x1a,0xce,0xeb,0xea,0xf4,0xc3,0x43,0xb7,0xb2,0x80, +0xa0,0x06,0x89,0xa7,0x1c,0x86,0x95,0x53,0xa0,0x64,0x82,0xb9, +0x2e,0x88,0x57,0xcd,0xdd,0xfc,0xa9,0xe2,0x5a,0x14,0xd3,0x7a, +0xf9,0x64,0x09,0xfe,0x23,0x32,0x52,0x10,0x01,0x6e,0x8d,0xa2, +0x88,0xd4,0xc8,0x04,0x2b,0x31,0x8e,0xf8,0xad,0xc8,0x5d,0x11, +0xad,0x3a,0x8a,0xa0,0x43,0xf0,0xbc,0x24,0x97,0x35,0x4d,0xd0, +0x2d,0x37,0x81,0x60,0xb7,0xd9,0x04,0xa6,0x08,0x2a,0x45,0xb6, +0xf2,0x53,0x85,0xc9,0x5f,0xb1,0x93,0xc1,0x0a,0x44,0x88,0xfa, +0xb3,0x50,0x6d,0xba,0x50,0x75,0xc8,0x7d,0x85,0x91,0xf4,0x3e, +0x21,0x6a,0x29,0xa8,0x15,0xe6,0x39,0x5f,0x89,0x8f,0x68,0x05, +0x94,0x9e,0xaf,0xd8,0xdd,0x80,0x90,0xae,0xf4,0xae,0x5e,0x7a, +0x14,0xa5,0xfe,0xe6,0x28,0xda,0xf8,0x62,0x4a,0x87,0xe6,0xfe, +0xe6,0x08,0xc1,0x8e,0x68,0x78,0x3a,0x4e,0xfd,0xf4,0x98,0x62, +0x92,0x39,0xc7,0x80,0x39,0x05,0xe4,0x6c,0xcb,0xbe,0xf0,0x86, +0x80,0x5e,0x82,0x96,0x17,0xf2,0xb1,0x28,0x13,0x0e,0xe6,0x23, +0xb1,0x16,0xef,0x42,0xa0,0x8f,0xdc,0xbf,0x8d,0x36,0xbd,0x15, +0x5e,0x4b,0xa7,0x7a,0xda,0xf1,0x06,0x0c,0x30,0x94,0xbf,0x4d, +0x14,0x89,0x06,0x61,0x46,0xb9,0x84,0xfc,0xfe,0xfe,0xf6,0x5c, +0x76,0x28,0x3a,0xea,0xa1,0xb4,0x8f,0x3a,0xc2,0xcf,0x4c,0xc1, +0xeb,0x07,0x22,0x3e,0xb4,0x9f,0xc3,0xdf,0x49,0x1e,0xdd,0xa2, +0x53,0xfb,0xa2,0x25,0x5f,0x32,0x51,0x5d,0x34,0x6b,0x88,0x72, +0x9a,0x99,0x7f,0x2b,0xa8,0x59,0x30,0x2d,0xb7,0x40,0x8b,0xd7, +0x5d,0xb4,0x7e,0xc3,0x3d,0x4f,0xcc,0xda,0x88,0x4a,0x59,0xb8, +0x26,0x0d,0x40,0x2f,0x48,0x69,0x46,0xb7,0xfc,0x2e,0x49,0x15, +0xd5,0x99,0x49,0x75,0xf6,0x20,0xd5,0x19,0xa0,0xcd,0x89,0xea, +0xbc,0x40,0x75,0xde,0x90,0xc5,0xe5,0x3b,0xa8,0x0e,0xa9,0xea, +0x82,0x6a,0x33,0x8b,0xa8,0x86,0xa2,0x3a,0x43,0xaa,0xb7,0xd6, +0x06,0x1d,0x39,0x31,0x99,0xab,0x08,0x2c,0xcf,0x71,0x7a,0x7f, +0x6f,0x9e,0x88,0x29,0x2c,0x97,0xca,0xa9,0x3e,0xcf,0xef,0x03, +0x5f,0xcd,0xf0,0xe9,0x63,0xa2,0xee,0x92,0x10,0x61,0x75,0x9f, +0x84,0x2f,0xb1,0x1f,0x1e,0x96,0x8e,0x75,0xc8,0x75,0x4c,0xd2, +0x1b,0x7d,0x0c,0xc8,0xec,0xd9,0x91,0x40,0xc3,0xe6,0xa3,0x50, +0x1a,0xae,0xc4,0xd5,0x93,0x5d,0xb0,0xa3,0xd4,0xa1,0xc4,0x85, +0x09,0x39,0x3d,0xcb,0xc9,0xa0,0x51,0xf9,0xe0,0x8e,0x48,0x79, +0x68,0x76,0xeb,0x13,0xf3,0xb9,0x79,0xe8,0xd3,0x34,0x28,0xe5, +0x3c,0x8f,0x6e,0x1b,0x82,0x40,0xc1,0x0a,0x05,0x1e,0x7f,0x3f, +0x9c,0x7c,0x8b,0x89,0x4c,0x11,0x36,0x19,0xdd,0xe3,0xa9,0x47, +0x31,0x9f,0xba,0x1f,0xed,0xdf,0x85,0x0d,0x7d,0x29,0xc8,0xa9, +0xee,0xcd,0x02,0xd3,0xe3,0x8f,0x16,0x88,0x8c,0x9e,0xb1,0xda, +0x85,0x78,0xaa,0x6e,0x8b,0x32,0xd6,0x97,0x26,0x5f,0x94,0x27, +0x79,0xcb,0x48,0xbe,0xbd,0xc0,0x38,0x78,0x3f,0x42,0x9b,0x11, +0xfa,0x14,0x19,0x59,0xae,0xcb,0x5e,0x1d,0x1f,0x18,0x67,0xa7, +0x2a,0x99,0xe7,0x6a,0xa5,0x7d,0xf3,0x65,0xaf,0xbf,0xf4,0xe9, +0xbf,0x41,0x67,0xe9,0x99,0x87,0x43,0x26,0xaf,0x17,0x74,0x91, +0xee,0xda,0xbe,0xc3,0xa8,0x1e,0x77,0xe6,0x6d,0x77,0xbd,0x24, +0x23,0xf8,0xc2,0x77,0x52,0x42,0xe1,0x7a,0xa5,0x62,0x22,0x2d, +0x72,0x75,0x77,0x96,0x57,0x5b,0x43,0x6b,0xe5,0xcc,0xa5,0xc7, +0x3a,0xa5,0x07,0x80,0xd7,0x33,0xb8,0xdb,0xd1,0x35,0x25,0x00, +0xf4,0xb5,0xec,0x67,0x3e,0xfd,0x47,0x82,0x25,0x6b,0x47,0x8b, +0x07,0x32,0x20,0x1b,0x51,0xb7,0x26,0x8b,0xe7,0x7e,0x6e,0x0f, +0x04,0x9a,0xf5,0xb4,0x2c,0x77,0x84,0x8b,0x87,0x92,0x14,0x8a, +0x3d,0x10,0x9e,0x1f,0xba,0x10,0x21,0xbd,0x7b,0xba,0xd2,0x13, +0x6c,0xe6,0xeb,0xbe,0x87,0x1d,0x40,0x0b,0x23,0xa5,0x1b,0x15, +0x20,0xf9,0xdf,0xd3,0x6f,0x03,0x41,0x0d,0xc1,0xa6,0x08,0x8d, +0x8a,0xba,0xd6,0x5e,0x98,0x08,0x32,0x6b,0x38,0x8e,0xc6,0x85, +0x51,0x0f,0xad,0xd6,0xa2,0xe8,0x17,0xe6,0x38,0x7c,0x9f,0x42, +0x04,0x93,0x63,0x18,0x5e,0xa7,0x7a,0xca,0xfc,0x1f,0xff,0xf0, +0x7f,0xff,0x37,0xf9,0x26,0x58,0x91,0x00,0x29,0x5a,0xed,0x73, +0xdc,0x30,0xfe,0xfd,0xdf,0x22,0xb0,0x2d,0x43,0xfa,0xfe,0x8f, +0x7f,0xa0,0xbd,0x67,0xcc,0x94,0xf8,0x92,0x08,0x50,0xe9,0xdd, +0xc9,0xe2,0x49,0x16,0xb5,0x8f,0xac,0x5b,0x1c,0x60,0x0c,0xce, +0x2b,0xe2,0x56,0xf2,0x0a,0xa2,0xee,0x1d,0x98,0x2d,0xb1,0x7b, +0xb0,0xba,0xbf,0x5f,0x9d,0x87,0xaa,0x9b,0xfe,0xc7,0x3f,0x1a, +0xbf,0xff,0xdb,0x51,0x76,0xa4,0xf5,0x4d,0x53,0x20,0x99,0x2e, +0xae,0xdc,0x95,0x77,0xbc,0x02,0xab,0xc6,0x87,0xef,0x6f,0x0b, +0xb5,0x62,0x7a,0x44,0x9d,0x55,0xed,0x5b,0x5c,0x79,0x71,0x4e, +0x67,0x72,0x62,0x9c,0xd2,0xe9,0x16,0x6f,0x16,0xb3,0xdf,0x8b, +0x8b,0xb3,0xa5,0x2b,0xe9,0xc6,0x22,0x93,0xd5,0x5d,0x32,0xff, +0x12,0x34,0x8d,0x2e,0x72,0x56,0xbe,0x9d,0xcc,0xd8,0xc6,0x16, +0x6f,0x33,0xcb,0xc7,0x1c,0xd3,0x46,0x64,0xf1,0x42,0x59,0x44, +0xc7,0xb3,0x2d,0xdf,0x33,0x4a,0xc3,0x0c,0x9e,0xb1,0x90,0xaf, +0xc7,0x63,0xf8,0x62,0x0e,0x66,0xac,0xbe,0x7d,0xf5,0x72,0xba, +0x58,0xac,0xac,0xeb,0x95,0xc7,0x71,0x32,0x35,0x23,0xe8,0x88, +0x06,0x65,0x6d,0xf1,0xc5,0xd6,0xd5,0xd7,0x61,0x33,0x72,0x86, +0x70,0xf1,0x19,0xf9,0x81,0xcf,0x7f,0xed,0xeb,0x5a,0x71,0x6e, +0xa1,0x89,0xc3,0x36,0xd8,0x89,0x0e,0x9a,0xa8,0x50,0x3a,0xcc, +0x5b,0xa0,0xba,0x60,0xc8,0x72,0x21,0x20,0x4c,0x01,0x4c,0x27, +0x7c,0x1c,0xa4,0x82,0x81,0x57,0xa8,0x2b,0xe4,0xff,0x02,0xdb, +0x8b,0x6e,0x10,0xbb,0xa4,0x2f,0xd7,0x69,0x05,0xb1,0x43,0xcb, +0xd0,0x0a,0xe6,0x5b,0x60,0xc9,0xb7,0xf1,0x15,0xf4,0xb0,0xdb, +0x58,0xb2,0xa4,0xdc,0xb7,0x9a,0xc4,0x6e,0x64,0x8f,0xdb,0x04, +0xfd,0xca,0x8c,0xaa,0x03,0x23,0x24,0x04,0x67,0xf0,0x6f,0x09, +0x53,0x87,0x1d,0xb5,0x85,0xe4,0x72,0x6d,0xfd,0x90,0xf9,0x68, +0x72,0x0d,0xc5,0xb8,0x12,0x91,0x2e,0x07,0xd8,0x04,0x02,0xb7, +0x98,0xe3,0x13,0x59,0xad,0x53,0x33,0x33,0xbf,0xc8,0x3c,0xdd, +0x45,0x88,0x00,0x71,0x61,0xa2,0xfe,0x61,0xdd,0x71,0xc8,0x35, +0xcb,0xf1,0x87,0x9d,0x7e,0x7f,0x80,0x67,0x5a,0x34,0xae,0x31, +0xb4,0xe8,0xd5,0x62,0x95,0x87,0xc1,0x2e,0x72,0x24,0x3a,0x4f, +0xba,0x49,0x9f,0x5c,0xe8,0x8c,0x55,0xe8,0xda,0x1f,0x8b,0xae, +0x5d,0x83,0x6e,0xf4,0xd1,0xf8,0x46,0xd5,0x08,0xc7,0x1f,0x8d, +0x70,0xac,0x65,0xd4,0xe0,0x5e,0xbf,0x1d,0x9c,0xb5,0xdb,0x67, +0xd0,0xd0,0x67,0xcf,0x9f,0x3d,0x83,0x80,0xdf,0x7e,0xd9,0x7e, +0x76,0xfa,0xe2,0x85,0x1f,0x3e,0x0b,0x4e,0x9e,0x3d,0x6f,0xbf, +0xf0,0x5f,0x3e,0x6f,0x43,0xf8,0xd4,0x7f,0x19,0xbe,0x7c,0x1e, +0x9e,0x80,0xb8,0x3f,0x7b,0x19,0x3c,0x3f,0x39,0x09,0xfc,0x17, +0x27,0xa7,0xa7,0xcf,0x43,0x88,0x68,0x9f,0x9c,0xbd,0x0c,0x5f, +0x9c,0xf9,0xe1,0xe9,0xe9,0x59,0xf8,0xfc,0x74,0x20,0xbd,0xf7, +0x57,0x57,0x23,0xbe,0xf7,0x99,0xa4,0xde,0x60,0xb2,0x2e,0x35, +0x7c,0x76,0xfa,0xec,0xd9,0x33,0xa3,0x78,0xf8,0x38,0x7d,0x79, +0xf6,0xbc,0xad,0xe9,0x08,0x02,0xc0,0xfd,0xd2,0x20,0x08,0x41, +0x82,0xe7,0xcf,0xda,0x8a,0xb2,0xb0,0xfd,0xf2,0xf4,0xf4,0xc5, +0x59,0xa8,0x49,0x6c,0x43,0x74,0x88,0x68,0x25,0xad,0xe1,0x59, +0xfb,0xe4,0x59,0xf8,0xf2,0x99,0x26,0xfa,0x24,0x78,0xd9,0x7e, +0x11,0x3c,0x37,0xa8,0x07,0x3a,0x5e,0x9c,0xbe,0x78,0x29,0xab, +0x01,0x25,0x3f,0x0b,0x03,0x28,0xe2,0xc1,0xfa,0x8c,0xa2,0xfe, +0x09,0x92,0xf4,0xfc,0xb9,0x7f,0x7a,0xf2,0x12,0x32,0xbf,0xf4, +0x9f,0x07,0xcf,0x82,0xe7,0xf0,0x1b,0x06,0xed,0xb3,0x67,0x6d, +0xc0,0x7e,0x76,0x1a,0xbe,0x0c,0x5e,0x3e,0xf7,0x5f,0xb6,0xc3, +0xb3,0x53,0xa0,0x23,0x0c,0x5f,0x86,0xa7,0x61,0x08,0xa5,0x00, +0x41,0x67,0x67,0x27,0x67,0x90,0x12,0xb4,0x5f,0xbe,0x3c,0x01, +0x82,0x9e,0xb5,0xcf,0x5e,0x04,0x48,0x07,0x90,0x05,0xd5,0xc0, +0x0f,0xc8,0x7b,0xd2,0xf6,0x5f,0x9c,0x9d,0x9d,0x85,0x67,0x90, +0x02,0x99,0xce,0xa0,0x6a,0xc8,0x93,0x67,0x2f,0xcf,0x9e,0x9d, +0x20,0xc8,0xf3,0xe7,0x27,0xc0,0x36,0xa8,0xd2,0xf3,0xb3,0x17, +0x10,0x11,0xbc,0x38,0x79,0x76,0xd2,0x7e,0x86,0xf8,0xdb,0x2f, +0x4e,0x11,0x1d,0xb0,0xf3,0xe5,0xd9,0xe9,0xb3,0x07,0xab,0x33, +0x86,0xea,0xb4,0x9f,0x3d,0x03,0xbe,0x41,0x35,0x90,0x59,0x01, +0xb2,0x22,0x38,0x3b,0x6b,0x23,0xb5,0x01,0xa4,0x21,0x8b,0x01, +0xe5,0x69,0x70,0x0a,0x05,0x21,0x49,0xcf,0xcf,0x9e,0xe1,0x47, +0xfb,0xb4,0x7d,0xfa,0x12,0x3f,0x4e,0x9e,0x41,0xa3,0xf9,0x27, +0xed,0xd3,0x93,0xe0,0xf9,0x99,0xff,0xfc,0xec,0xb4,0xfd,0xec, +0x39,0x62,0x39,0x0b,0x9e,0x07,0x01,0xe4,0x39,0x09,0x03,0x40, +0xf0,0x0c,0x52,0x5e,0x02,0x75,0xa8,0x42,0x4f,0x5e,0x02,0xd6, +0x36,0xb7,0x06,0xd6,0x23,0x7c,0x76,0x02,0x7c,0x7b,0x19,0x22, +0xe3,0x4e,0x80,0xb7,0xfe,0xcb,0x17,0x67,0x2f,0xa0,0x44,0x64, +0xcf,0xc9,0xb3,0x17,0xc0,0xea,0xf0,0x59,0xfb,0xf9,0xf3,0xe0, +0x45,0x58,0x51,0x1f,0xad,0xf9,0x60,0x0a,0x00,0x83,0x49,0xe5, +0x35,0xdc,0x4a,0xeb,0x48,0x18,0x54,0x3b,0x7d,0x4b,0xdf,0x71, +0xca,0x6f,0x7b,0xe4,0xfe,0x2d,0x76,0x83,0x2a,0x8d,0xa9,0x9c, +0x86,0x77,0xa1,0x50,0x40,0x6e,0xbf,0x75,0x3a,0x28,0x23,0x91, +0x23,0xe2,0x2e,0x1c,0x12,0xc6,0x35,0x73,0xdf,0x5e,0xc1,0x7c, +0x76,0x5c,0xb9,0x28,0x0c,0xf1,0xd2,0xd2,0x08,0x22,0xf2,0x6f, +0xfd,0x81,0xb6,0xe2,0xaa,0x0f,0x02,0xfb,0xab,0x30,0x02,0x83, +0xed,0x01,0xa0,0x61,0x10,0x3d,0x74,0x38,0xb9,0x31,0xe1,0xa7, +0x79,0xa0,0x78,0xf6,0x77,0xc7,0xdb,0xa3,0x77,0x1f,0x42,0x2e, +0x67,0x99,0x0c,0x23,0x77,0x18,0x9c,0x0f,0x43,0x74,0xf9,0x1a, +0x46,0xc3,0x00,0x4b,0x1e,0x86,0x88,0x6a,0x32,0x04,0x32,0xc2, +0xe6,0x10,0x0c,0xbc,0xc9,0x65,0xf4,0xfb,0xbf,0x9e,0x4f,0x86, +0x3d,0x27,0x70,0x3a,0x4e,0xe8,0xf8,0xe3,0x80,0x0d,0x97,0xf1, +0x22,0x75,0x11,0x20,0x15,0xe1,0x14,0xc6,0x74,0x0c,0x8f,0x43, +0x23,0x1d,0xac,0xc5,0x34,0x34,0xd2,0x43,0xe3,0xdc,0xdf,0x79, +0xa4,0x49,0xfa,0x66,0x98,0xf5,0x56,0x41,0xcf,0xf9,0x26,0xf0, +0x9d,0xc6,0x2a,0x6c,0x38,0x9f,0xf2,0x8f,0x08,0x3d,0x09,0x9e, +0x80,0x74,0x3f,0xc1,0xc4,0xe6,0xce,0x54,0x0c,0x32,0x8e,0x80, +0xa1,0x02,0x86,0x0a,0x04,0x54,0x20,0x70,0xec,0x4a,0xc5,0xe0, +0xcf,0x50,0xd7,0x3f,0x83,0x16,0xc0,0x43,0xd5,0xc2,0xd0,0xd1, +0x58,0x14,0x17,0x1e,0xa5,0x41,0x45,0x3e,0xa7,0x31,0xb9,0x84, +0x70,0xf8,0x84,0x81,0x43,0x05,0x0c,0x5f,0x5f,0x23,0x59,0x2a, +0x2e,0xa0,0xb8,0x52,0x05,0x04,0x02,0xfc,0x40,0xe0,0x40,0x01, +0xcb,0x0a,0xfd,0x61,0x2a,0x02,0x3f,0x70,0x10,0xd7,0x3f,0xf1, +0xbd,0x65,0x5a,0xde,0x8d,0xa6,0xfc,0x4a,0xc7,0xfa,0x86,0xb0, +0x1b,0x10,0xdf,0xe9,0x58,0xf3,0x94,0xb6,0x06,0x78,0xa3,0x22, +0xf5,0xc1,0x6d,0x9d,0xfa,0x17,0x11,0xd5,0x55,0x73,0xa6,0x71, +0xcb,0x24,0x45,0x9f,0xf7,0xdd,0x31,0xad,0x2a,0xd0,0x2e,0x3d, +0xd6,0x6f,0xe9,0xac,0xb9,0xd7,0x31,0x92,0xb7,0x18,0xd3,0x32, +0x6b,0xb2,0x57,0x01,0x85,0xaa,0x17,0x0b,0x30,0x92,0xb9,0x80, +0xaa,0x73,0xe0,0xbb,0xf0,0xdb,0x8c,0x2b,0xa2,0x37,0xcf,0x84, +0x23,0xf6,0xf2,0x99,0xf0,0x5d,0xb8,0x4d,0xa6,0x17,0x31,0xeb, +0xf3,0xe1,0x88,0x77,0x0c,0xf9,0x56,0x8b,0xc4,0xd2,0xcb,0xa4, +0x0d,0x23,0xf7,0x41,0x6d,0xd8,0x78,0x50,0x15,0xe2,0xce,0x33, +0x28,0xa9,0x07,0xb5,0xe1,0x6e,0xcd,0x07,0x58,0x4a,0xca,0x4f, +0xc8,0x4e,0x5f,0x6b,0x2a,0xef,0x68,0xe5,0x6b,0x3d,0x05,0xa1, +0x01,0xd5,0x50,0xcf,0x48,0xcc,0xec,0x51,0xf3,0x7a,0x38,0xbd, +0xfc,0xfd,0x5f,0x7d,0x4b,0x83,0x45,0xbf,0xff,0x7b,0xf3,0x3f, +0xff,0xd1,0x15,0x43,0x06,0xce,0x00,0xc8,0xe9,0xc3,0x36,0xd6, +0x69,0xd3,0x1e,0x92,0xcc,0xd7,0xd5,0x5c,0xe3,0xfb,0x07,0xca, +0xa1,0xd4,0xa4,0xc2,0x02,0x62,0x44,0xdf,0x99,0xf4,0xab,0x13, +0x21,0xfa,0xe1,0x67,0xcf,0xe9,0x33,0xb7,0xd2,0x73,0x4a,0xdf, +0xfa,0x46,0x01,0x6f,0xd6,0xf1,0xf2,0xb3,0x18,0x66,0x53,0xb1, +0x3c,0xe3,0x18,0x15,0x52,0x69,0xbb,0xbe,0x98,0x85,0x22,0xeb, +0x72,0x30,0x3e,0x3d,0x58,0xc6,0xc3,0x9a,0x41,0x96,0x78,0x15, +0x0f,0xad,0x7b,0xcc,0xb6,0x66,0xbe,0x1a,0x8e,0x11,0xca,0x02, +0x82,0x1a,0x8e,0x29,0x2c,0xc0,0x31,0xfa,0x56,0x1c,0x13,0x21, +0xfa,0x31,0xfb,0x3f,0xc7,0x07,0x36,0x58,0xc0,0x70,0x66,0x37, +0xe6,0x84,0xd0,0x86,0x0b,0x19,0x8e,0x5b,0x80,0x3e,0x73,0x0b, +0x20,0xe7,0x74,0xa3,0xbb,0x72,0xb4,0x5d,0x5c,0x2e,0x8a,0x53, +0x3d,0x8f,0x63,0xed,0xb2,0x72,0x2e,0x4b,0xf1,0xab,0xfe,0x05, +0x12,0x4a,0xd1,0x8e,0x36,0xa9,0x7a,0x74,0x81,0xfb,0x86,0xb8, +0x1e,0x9a,0x1c,0x6b,0xd6,0x85,0x34,0x71,0x5f,0xb4,0x71,0x49, +0x0d,0x8e,0x1b,0x69,0x6b,0x19,0x34,0xd0,0xd8,0x49,0x5b,0x2b, +0x1f,0x02,0x21,0xfc,0x01,0x03,0x01,0xfe,0x04,0x5e,0xc3,0x8d, +0xc1,0x7a,0x9b,0xa6,0x2e,0x64,0xf6,0x7a,0xe3,0xcd,0xea,0x36, +0x36,0xc1,0xe8,0x23,0xf0,0x3a,0x85,0x84,0x35,0x7c,0xac,0x31, +0x81,0xd0,0x8a,0x10,0x46,0x03,0xda,0x35,0xa1,0xe5,0x0c,0x2a, +0x45,0x61,0xf2,0x70,0x04,0x32,0x36,0x13,0x25,0xf9,0x69,0x3c, +0xbd,0xf4,0x2f,0x7d,0x5d,0x6b,0x91,0x10,0x5d,0x8a,0x9b,0x0a, +0x30,0x9d,0x9d,0x89,0x84,0x17,0xa1,0x11,0x2f,0x81,0x31,0xd9, +0x36,0xc0,0x2a,0x61,0x76,0x1b,0x60,0xfb,0x65,0x11,0xec,0xfd, +0xb0,0xea,0xac,0xa0,0x48,0x7c,0xbf,0x63,0x18,0xe2,0xcb,0x1d, +0xcb,0xa0,0xd3,0x5f,0x1d,0x59,0x36,0x96,0x08,0x0a,0x13,0x0b, +0xdf,0xd5,0xb7,0x41,0x42,0x1b,0x24,0xf4,0x06,0xc6,0x1e,0x8a, +0x68,0x1d,0xf3,0x34,0x37,0xc8,0x6b,0x10,0x45,0x23,0xf8,0x7b, +0x78,0x08,0xdf,0x21,0x7d,0x87,0x5b,0xcb,0xac,0x5d,0xf9,0x4b, +0x5f,0x1d,0x92,0x26,0x3b,0x81,0x4d,0x01,0x61,0x25,0x3c,0x69, +0xb8,0xc3,0xf3,0xdf,0xff,0xd5,0x13,0xb6,0xc2,0x52,0x67,0xe6, +0x76,0x5b,0x05,0x50,0x11,0x30,0x78,0x81,0x56,0x85,0xe4,0xff, +0x03,0xbb,0x08,0xf3,0x2e,0xc3,0xad,0xf1,0x54,0x08,0xf2,0xc5, +0x7c,0x96,0x03,0x23,0x84,0x14,0xae,0x2c,0xd3,0x81,0xa4,0xfa, +0xcf,0xb3,0x1b,0xf8,0xf1,0x82,0xa2,0x5b,0xde,0xae,0x71,0x71, +0x55,0x35,0x9c,0x8b,0x07,0x15,0xa4,0x0b,0x9e,0x7c,0xb3,0x61, +0x63,0x3d,0x67,0xb0,0x73,0x24,0x57,0x6c,0x28,0x63,0x15,0xef, +0x97,0x08,0xac,0x82,0x3f,0x7b,0x61,0xd5,0xbc,0x2c,0x63,0x15, +0xef,0x9e,0x48,0x5a,0xff,0x2c,0xcb,0x43,0xd2,0x6c,0xd8,0x1e, +0x5c,0xc2,0x9f,0x62,0x7d,0x08,0xec,0xda,0xfe,0xd0,0x4f,0x48, +0xa0,0x32,0x84,0x31,0xe0,0x6a,0x51,0x58,0x18,0x53,0x22,0x29, +0x13,0x0d,0xdf,0x43,0xe8,0xe3,0x7c,0xf3,0x6c,0xc1,0xd1,0x70, +0x79,0x12,0x89,0x0b,0x68,0x0b,0x09,0xb3,0xc8,0x5d,0x06,0xad, +0xbc,0xb1,0x3c,0x69,0xe5,0x68,0x98,0x2c,0xa3,0x3e,0x08,0xf8, +0x87,0xac,0x03,0xb1,0x99,0x9f,0x77,0x66,0x5b,0x0a,0x9c,0xc8, +0xc0,0xf2,0x44,0x7b,0x90,0x46,0x4b,0x9a,0x5d,0x2f,0x57,0x0b, +0x7c,0xe9,0x10,0x28,0xf2,0x7c,0xd4,0xaa,0xcb,0x7e,0x30,0x68, +0x38,0xaf,0xf1,0x23,0x84,0x0f,0xec,0x19,0xfd,0xb6,0xfc,0x38, +0x19,0xec,0xd9,0x49,0x34,0x56,0x29,0xec,0xb2,0xba,0xdf,0xab, +0x14,0x35,0xeb,0x12,0x29,0x25,0x01,0xdd,0x7d,0x71,0x51,0x51, +0x40,0xf1,0xb5,0x61,0x81,0x49,0xcb,0xa8,0xc2,0x5d,0x14,0xd3, +0x9d,0x77,0x69,0x97,0xc4,0xd4,0xc2,0x2d,0x25,0x55,0xe1,0x36, +0x2a,0xbb,0x17,0x7e,0x03,0x3e,0x33,0xf0,0xea,0x68,0x8d,0xbb, +0x24,0x4a,0x35,0xb6,0x88,0x92,0xb4,0x02,0xb8,0xeb,0xed,0xd5, +0x14,0x06,0xd0,0x45,0x54,0x51,0xaf,0x52,0x53,0x3d,0xb2,0xca, +0x06,0x7a,0xb7,0x40,0x04,0x5b,0x4a,0x9a,0x14,0x8b,0x3b,0xde, +0x03,0x5c,0x49,0xf3,0xd9,0x68,0x51,0xdd,0xbd,0x38,0xc9,0xbc, +0xb8,0x49,0x96,0xcc,0x29,0x7c,0xf7,0x0b,0xbd,0x61,0x67,0x77, +0x2a,0xbe,0x0d,0x46,0x03,0xbe,0xa6,0xe7,0x29,0x3d,0xda,0x16, +0x2e,0x82,0x6e,0xe5,0x6e,0x67,0x64,0x65,0xf9,0x11,0x62,0x78, +0xa3,0xdb,0x8a,0x46,0x8f,0x40,0xb5,0xa5,0x42,0x31,0x2d,0xca, +0xbb,0x9f,0x50,0x8a,0x52,0x0c,0x91,0x64,0x1c,0x20,0x90,0x90, +0xb4,0x15,0xa1,0xc7,0xee,0x73,0x57,0x63,0xe4,0x2d,0x6f,0x0e, +0x59,0xb3,0x0d,0xc1,0x3b,0x75,0x7f,0x88,0x78,0xdc,0xb3,0x82, +0x61,0xfe,0x78,0xb5,0x48,0x8d,0x83,0x0e,0x7c,0x71,0x3b,0xcf, +0xc3,0xf4,0xc6,0x19,0xc6,0x1e,0x9f,0x6a,0x4f,0x5d,0xd4,0x40, +0xcd,0x93,0x23,0x1e,0xd6,0x9b,0xf0,0xf3,0x25,0xff,0xfc,0x24, +0xa3,0xbf,0xa4,0xd1,0xfe,0x27,0x95,0x48,0x91,0x3f,0x51,0xa4, +0x4a,0x92,0x80,0x4d,0x33,0x89,0xb3,0x83,0x91,0x86,0x52,0x34, +0x5b,0x98,0x37,0x52,0x18,0xa4,0xe5,0x45,0xda,0xf0,0x46,0x0d, +0xab,0x69,0x87,0xf3,0x93,0x00,0x9f,0x85,0xcf,0xa2,0x55,0x5e, +0x91,0xa4,0xea,0xc1,0x6b,0x31,0x39,0x2f,0x93,0x64,0xb8,0x04, +0xe2,0xa0,0x95,0x82,0xd6,0x4a,0x4e,0xda,0xb4,0xa9,0x62,0x91, +0x28,0xf2,0x68,0x88,0xf7,0x61,0x57,0x81,0x59,0x9a,0x55,0x5f, +0x4b,0x73,0x08,0x03,0x4f,0x4c,0xdb,0x48,0x81,0x09,0x0e,0x38, +0xeb,0x55,0x42,0x93,0x83,0xe6,0x64,0x71,0x67,0x3e,0x64,0x6a, +0x94,0x9a,0x15,0x39,0x61,0x8b,0x32,0x44,0x9f,0x00,0x17,0xf2, +0x68,0x95,0x1d,0x95,0x53,0x0c,0x12,0x65,0x8d,0x25,0x1b,0x04, +0x57,0x14,0x03,0x44,0xd8,0x26,0x6b,0xb3,0xfc,0x3f,0x4f,0x54, +0xd3,0xa6,0xaa,0x40,0x94,0xa0,0x89,0xef,0xeb,0xd0,0xaa,0x06, +0x7b,0x76,0x1a,0x15,0x14,0xc9,0x4d,0x9c,0xa7,0xae,0x67,0xbd, +0x47,0x49,0xde,0x5f,0x5f,0xcd,0xaf,0xe3,0x55,0xb2,0xfe,0x6a, +0xe2,0x57,0x45,0xfb,0x65,0x0a,0x8d,0xda,0x9d,0x78,0x7e,0x59, +0xb8,0x38,0x7d,0x3d,0x9c,0xbb,0x27,0x01,0x56,0x90,0xc6,0x82, +0x39,0x1f,0x80,0xd1,0xf8,0xbf,0x97,0x8f,0x73,0xa2,0x57,0xac, +0x5d,0xf2,0x84,0xcf,0x28,0x54,0xc0,0x92,0x66,0xa3,0x8a,0xc5, +0x53,0x56,0xb7,0x76,0x52,0x4d,0x11,0xad,0x78,0xb6,0x5c,0xe7, +0x35,0xf9,0x28,0xad,0x2e,0x23,0xfa,0x94,0xd4,0xe4,0xe3,0xbb, +0x88,0xab,0xb3,0x29,0xad,0x5a,0xce,0x86,0x49,0xea,0x05,0xd7, +0xc2,0x3d,0xb5,0x0a,0xbc,0x5e,0x1f,0x56,0x37,0x5d,0x4f,0xe5, +0x34,0xf0,0xba,0xe8,0xab,0x2e,0xa2,0x3b,0x26,0x31,0x3f,0xa0, +0xa3,0x95,0x09,0xb7,0xb5,0x09,0xd2,0x0f,0xa7,0x46,0xd5,0xf5, +0xab,0xad,0x36,0x95,0x50,0xac,0x0f,0x1e,0x53,0x37,0x67,0x95, +0x38,0xb5,0x53,0xbe,0x45,0x7e,0x32,0x89,0xe8,0x1a,0xf4,0x64, +0x62,0xbc,0xf1,0xd9,0x1f,0x74,0x65,0x5e,0x8b,0x8d,0x17,0x32, +0x56,0xa3,0xee,0x4a,0x37,0x01,0xf1,0x9a,0x06,0x21,0xb3,0x1f, +0xd2,0xf8,0xa0,0x10,0x8b,0xa7,0x10,0xe5,0xf4,0xb6,0x3f,0xd0, +0xd9,0x39,0x06,0x73,0xe3,0x7d,0x59,0xc2,0xf3,0x91,0x22,0x8b, +0x4e,0x9b,0x24,0x13,0x94,0x82,0x1e,0xe8,0xe8,0x80,0xc6,0xf5, +0x89,0x24,0x4d,0x3c,0xfe,0x6a,0xf7,0xa9,0x8b,0x0b,0xbc,0x93, +0xe5,0xe2,0x02,0x1f,0x32,0xf5,0xbc,0x9e,0xeb,0xc8,0x08,0x27, +0xe1,0x3b,0xb9,0x35,0x0e,0x05,0x1b,0x59,0xa1,0x42,0xdf,0xc1, +0xb3,0x2c,0x32,0x07,0x20,0x05,0xd6,0x09,0x68,0x0d,0x72,0x71, +0xd1,0x4f,0x26,0x03,0xfd,0x2c,0xa9,0xaa,0x39,0xf9,0x51,0x81, +0x64,0x58,0xf1,0x74,0xd2,0xc2,0x38,0x6c,0x66,0x08,0x87,0xe2, +0x1d,0x14,0x63,0x3c,0xb0,0x5b,0xd9,0xf2,0x9f,0x4e,0xa7,0xfb, +0x36,0x7e,0x2a,0x8e,0xbb,0x8b,0x4a,0x68,0x9c,0x1f,0x29,0x10, +0x9f,0xf2,0xba,0xc1,0xde,0x32,0xf1,0xe8,0x66,0x17,0xd7,0x40, +0xeb,0x86,0xff,0x60,0xf4,0xe0,0x1a,0xf6,0xab,0xca,0xee,0x27, +0x1a,0xfe,0x3e,0x72,0x7a,0x83,0x24,0x2e,0x22,0x89,0x59,0x53, +0x79,0xf3,0x6a,0x01,0xc2,0xa9,0x64,0x51,0xa4,0xf7,0x6f,0x50, +0x44,0xeb,0xa5,0xe7,0x06,0xa5,0x47,0xa7,0xd5,0x09,0xcc,0x1f, +0x91,0x8d,0xcb,0x64,0x6a,0x1d,0x6c,0xe3,0x70,0x41,0x2c,0xf8, +0xaf,0x94,0x09,0xdd,0xea,0xda,0xd3,0xee,0x40,0xfa,0xd8,0x71, +0x7e,0xe8,0x33,0x02,0xb1,0x25,0x0f,0x1c,0x27,0xcb,0x30,0x25, +0x21,0x28,0x08,0x82,0x74,0x27,0xfa,0x28,0xed,0x10,0x14,0xa5, +0x44,0xde,0xc5,0x55,0xd2,0x0e,0x4c,0x49,0x7d,0xbb,0x7b,0xfa, +0xa5,0x5e,0xe3,0xb1,0x82,0x87,0xf8,0x2d,0x7a,0x48,0x3d,0xd3, +0xd7,0x77,0xb1,0x79,0xe5,0x3b,0x3d,0xc0,0x4e,0x71,0xe2,0x52, +0x33,0xd5,0xc9,0xba,0x35,0x23,0xcd,0xab,0x76,0x8f,0x20,0x90, +0x1c,0xd7,0xab,0x10,0x6e,0x2e,0x82,0xe6,0x27,0xf4,0x2e,0xbc, +0x35,0xc4,0x5c,0xc4,0xe8,0x3e,0xcf,0x4f,0x8c,0x90,0x5b,0x25, +0x01,0x57,0x1e,0x5d,0xab,0xc5,0xbc,0x8a,0xe9,0xb9,0x45,0x42, +0x5e,0x7d,0x4e,0xad,0x36,0x6b,0x2a,0x88,0x12,0x55,0xde,0xd6, +0xf3,0x69,0xb8,0x5e,0xaf,0x6c,0x36,0x7d,0xfb,0x86,0xef,0xc6, +0x32,0xf7,0xaa,0x01,0x88,0x3c,0x5b,0xa1,0xd3,0xd3,0x43,0xe0, +0x44,0x99,0x7e,0xf5,0x9c,0x49,0x2c,0x81,0x7f,0xfb,0xa6,0x26, +0x03,0x24,0xf0,0x2b,0xf8,0xcb,0xe1,0x18,0x04,0x02,0x3f,0xa7, +0x0b,0x90,0x90,0x02,0x8e,0x1f,0x91,0x78,0x57,0xaf,0x3d,0x32, +0x27,0x47,0x3d,0x89,0xbf,0xe3,0x8e,0x1a,0x91,0xe3,0x14,0x3d, +0x09,0x13,0x7f,0xc8,0x8d,0x7b,0x65,0xbe,0xcc,0x4e,0x34,0xaa, +0xc6,0x3e,0x00,0x3c,0xd0,0x83,0x92,0xc8,0xf0,0x50,0xa3,0x75, +0x4e,0x8d,0x6c,0x2d,0x68,0x4f,0x8b,0x58,0xfc,0x04,0xd2,0xe8, +0xd9,0xe7,0x0a,0x72,0xa1,0x6a,0xf5,0x04,0x7f,0xfb,0xe6,0x31, +0x24,0xd7,0x72,0xe9,0x8f,0xd6,0xa2,0x0e,0xb1,0x51,0x31,0xd0, +0xf6,0xe5,0x0e,0x21,0x5d,0x12,0x41,0x38,0x9e,0xe0,0xc8,0x4d, +0xc2,0xe2,0x11,0x7a,0xac,0x21,0xa7,0xf8,0x1c,0xdd,0xa7,0x80, +0xbe,0xac,0x17,0xa1,0xc4,0xd6,0xaf,0x22,0x35,0x32,0x5e,0xcb, +0x8f,0x22,0xce,0xd7,0x9b,0x54,0x3f,0xf2,0xde,0x29,0x78,0x13, +0x22,0x74,0xc4,0x0f,0xf9,0xe3,0x22,0x35,0x1d,0x09,0x62,0x7a, +0xba,0x55,0xaa,0xe3,0x82,0x3a,0x02,0x77,0x48,0x07,0x89,0x6d, +0x39,0x0d,0x53,0xd8,0x0d,0x2e,0xf4,0x8c,0xb6,0xec,0xa8,0xef, +0xdd,0x3d,0xe8,0xc7,0xb2,0xb6,0x01,0xcc,0x42,0xdf,0x54,0xc8, +0xb4,0x5a,0x4c,0xbc,0x64,0xc5,0x60,0xaf,0x58,0xf8,0x35,0xd2, +0xab,0x1d,0x4d,0x41,0xad,0x3e,0x2c,0xa6,0x97,0xd4,0x9a,0xd5, +0x32,0xfa,0x58,0x02,0x6a,0x65,0x71,0x6f,0x9a,0x6a,0x85,0x4e, +0x90,0x49,0x4e,0xc4,0x0f,0x36,0x2a,0xa1,0xe5,0xb6,0x2c,0x34, +0xe3,0xc7,0x37,0x60,0xba,0xce,0xcd,0x05,0x66,0xe2,0x1d,0x0b, +0xc5,0x72,0x95,0x2c,0x56,0xb8,0xd1,0x68,0xde,0x48,0x0e,0xc0, +0xa6,0x2e,0xa4,0x08,0xa1,0xe0,0x00,0xe7,0x32,0x5e,0xad,0xf3, +0xa2,0x42,0x24,0x18,0x76,0x8d,0xaa,0xd0,0x0e,0x0a,0xe5,0x0e, +0xe5,0x00,0xa4,0xdf,0x25,0xf3,0xc9,0xe2,0x0e,0x5b,0xe5,0xf5, +0x62,0xb6,0x04,0x8e,0x4e,0xde,0x60,0x46,0x3d,0xc8,0x78,0x98, +0x26,0x49,0xf8,0x89,0x4f,0x10,0x57,0x2b,0x3d,0xbb,0x2f,0xd5, +0x69,0x0c,0xaa,0x58,0xaa,0x51,0x6a,0xe5,0xa7,0x19,0xb3,0x95, +0x2f,0x8d,0x57,0xf8,0x7f,0x83,0x0e,0x39,0x39,0x9f,0xd3,0x15, +0x5d,0x4e,0x4a,0x95,0xd7,0x36,0x0c,0x11,0xf6,0x81,0x5d,0xc3, +0xf9,0x96,0x44,0xc7,0x61,0x6b,0x43,0xa2,0x96,0x1a,0xc6,0xd3, +0xc4,0xa8,0x34,0x6a,0xec,0xbe,0x0c,0x89,0xcb,0x35,0x6d,0x55, +0x23,0x13,0x01,0x6f,0xa5,0x2d,0x61,0x29,0x04,0xae,0x2a,0x8b, +0x92,0x68,0x7a,0xa3,0xc9,0x1e,0x90,0x9c,0x8a,0xbe,0xcf,0x3d, +0xbf,0x56,0x7c,0xf6,0xd2,0x01,0x7f,0xa4,0xc5,0x77,0x77,0xca, +0xea,0x76,0xbd,0xb4,0xdb,0x75,0x5b,0x67,0x16,0x9d,0xe0,0xd9, +0x43,0xcd,0x5c,0xcf,0x37,0xbb,0xa4,0xc5,0x48,0x5d,0xd7,0x1d, +0x66,0x5a,0x9c,0x99,0xbb,0x52,0x6c,0x77,0x3c,0xd8,0x5e,0x98, +0xcb,0x11,0x0d,0x55,0x80,0x82,0x94,0xfa,0xd2,0xb8,0x97,0x56, +0x38,0x18,0x50,0x1d,0xc8,0x64,0xc3,0xb7,0x96,0x8d,0xa5,0x80, +0x52,0x67,0x5c,0x76,0x09,0xd6,0xb6,0xbc,0x5a,0xe3,0xc5,0x66, +0xbe,0x7e,0xd5,0x46,0xd6,0xf0,0x38,0xce,0xb7,0xc8,0xd1,0x65, +0x0c,0x87,0x87,0x4b,0x51,0x30,0x5d,0x5b,0x48,0x55,0xd8,0x65, +0x91,0xc5,0xc3,0x34,0x2e,0xb1,0x64,0x4f,0x9b,0x35,0x7c,0xc8, +0x66,0x45,0xe4,0x1d,0xb7,0x62,0x52,0xa1,0x5e,0x9e,0x12,0x2f, +0x35,0x11,0x64,0xd5,0x61,0x07,0x5e,0xf1,0xaa,0xb0,0x73,0xf7, +0x34,0x50,0xa9,0x7a,0xfc,0x0a,0x86,0x57,0xcf,0x84,0x09,0xbe, +0xfb,0xbe,0x27,0x17,0x6a,0xe8,0xa9,0xb8,0xf9,0xe4,0xd6,0x7e, +0xe4,0x44,0x2c,0x1f,0xbc,0xaf,0x27,0x96,0xc9,0x30,0x5f,0x8a, +0xa9,0x99,0xba,0x6e,0x3b,0xae,0xf9,0x1e,0xd9,0xbe,0xcc,0x60, +0xfc,0x0f,0x73,0x63,0x53,0xbc,0xb6,0xe1,0xbf,0x8d,0x21,0x92, +0x12,0x75,0xe4,0x36,0xf4,0x1f,0xe6,0x8e,0x66,0x4f,0x31,0x9b, +0xb7,0x37,0xa7,0x64,0xc1,0x0f,0x32,0x0b,0x2b,0xac,0x19,0x45, +0xcb,0x75,0xd3,0x24,0x5d,0xc7,0x73,0x39,0xe7,0x36,0x18,0x56, +0x63,0xe6,0xb2,0x9d,0x4a,0x6b,0x8b,0x51,0xe5,0xa2,0x70,0x22, +0x17,0x1e,0x2b,0x93,0xbf,0x9a,0x74,0xab,0xa3,0xa3,0x64,0xb2, +0x57,0xef,0xe1,0x36,0xa8,0xc2,0x51,0xbb,0xd2,0xa2,0xb7,0xc8, +0xea,0x24,0xd4,0xab,0x5c,0xdf,0x8e,0x12,0xbd,0xce,0x5d,0x49, +0xb1,0xfc,0xe2,0x6b,0x4b,0xf7,0xeb,0xfb,0xb4,0xdf,0xf6,0xe0, +0xea,0x50,0xd7,0x35,0x96,0x5c,0xe9,0x01,0xc6,0xfb,0xfb,0x52, +0x14,0xaa,0xa2,0x49,0x02,0x46,0xe3,0x1a,0x95,0xb3,0x78,0xc5, +0x98,0x9f,0xfe,0xf7,0xe8,0x31,0x48,0xbb,0x81,0xb7,0xb6,0x15, +0x50,0x3b,0xec,0x54,0x2c,0x3e,0x1b,0x07,0xae,0x2a,0x16,0x65, +0xac,0x05,0xba,0x40,0xad,0xd0,0x25,0x93,0x30,0x6a,0x34,0x0a, +0xcb,0xf7,0xd6,0x0a,0x8e,0xff,0xdf,0xb6,0xf2,0x62,0xf0,0x32, +0xfa,0x6e,0x84,0xdb,0xb3,0xad,0xf1,0x2a,0x46,0x83,0xaf,0xba, +0x41,0x82,0x81,0x67,0x54,0x93,0x75,0x53,0x23,0x32,0x63,0x44, +0x1f,0xac,0x58,0x8c,0x55,0x2b,0xb1,0xa1,0xb9,0x98,0x56,0x5c, +0x4d,0xdb,0x73,0x29,0x2d,0x34,0x1c,0xfb,0xb2,0x24,0xad,0x76, +0x9f,0x87,0x04,0xf7,0xca,0xfb,0x70,0xc5,0xc3,0x76,0x61,0x94, +0xc6,0xe3,0x97,0x6a,0xea,0xe8,0xd3,0xb3,0xa8,0x2c,0xb6,0x3c, +0xfa,0xf2,0x39,0x6c,0xd1,0x8a,0x17,0x17,0xf8,0xb0,0xeb,0xfa, +0xe2,0xe2,0xfe,0x9e,0xa2,0x39,0x31,0x2c,0x24,0x46,0xe2,0x20, +0xd0,0x62,0x89,0xe7,0xa5,0x11,0xbf,0xbc,0x64,0x0b,0xbf,0xc9, +0xf6,0x4a,0x7b,0x9c,0x93,0x5e,0xd1,0xb6,0x03,0x62,0x20,0xe5, +0x28,0xca,0xfd,0xa9,0x54,0x3b,0x17,0x5e,0x47,0x40,0x8a,0xd3, +0x6d,0x5e,0x47,0x63,0x24,0x50,0x71,0x90,0x4d,0x97,0xc6,0x11, +0x17,0x66,0x09,0x1c,0x55,0x8e,0xd9,0x5d,0xb0,0xe1,0x4b,0xd9, +0x31,0x10,0x13,0x5c,0x74,0xa5,0x17,0xb5,0x5d,0x87,0x50,0x3a, +0x5e,0x0b,0x15,0x8a,0x4b,0x55,0x12,0x6c,0x62,0x5e,0xfc,0x05, +0x67,0x13,0x44,0x5b,0x2b,0xc6,0x4f,0x17,0xdf,0x8e,0x4e,0xc1, +0xb8,0x74,0x9d,0x2b,0xe8,0xae,0xa2,0x6a,0x90,0x9f,0xd6,0x09, +0x9c,0xf1,0x74,0x98,0xa6,0x10,0x2f,0x90,0xb2,0x71,0xef,0x2c, +0x60,0x66,0x08,0x94,0x38,0xfe,0x7f,0xfe,0x43,0x60,0xcd,0x48, +0xff,0x9a,0x5b,0x35,0x5c,0x04,0xc4,0xbb,0x5e,0x65,0x3e,0xb9, +0x68,0xc6,0x08,0xfe,0xba,0x9c,0xe0,0x32,0x43,0x19,0x45,0x39, +0x6f,0x28,0x6f,0xe1,0x97,0x07,0x3a,0xe8,0xde,0x77,0x57,0x56, +0x12,0xf4,0xcf,0x75,0x81,0x23,0xaa,0x4e,0xc4,0x13,0x7c,0x5f, +0x8f,0xa0,0x44,0x99,0xee,0x92,0x8e,0x7e,0x0a,0x66,0x40,0x2b, +0x80,0xca,0x72,0x1d,0x8c,0x2c,0x71,0x41,0x22,0xb2,0x37,0xa1, +0x08,0x01,0x58,0xf2,0x8a,0xbd,0x0a,0x09,0x7a,0xae,0x3a,0x06, +0xdf,0x55,0x02,0x99,0xc5,0xe2,0x72,0x2f,0x80,0xd1,0x8d,0xc2, +0x60,0xa2,0x07,0xc8,0xfc,0xf8,0x23,0x88,0xd5,0xbc,0x2a,0x02, +0x91,0x7d,0x4e,0x4c,0x97,0x09,0x5c,0x08,0x19,0xee,0xae,0x96, +0x19,0x86,0xac,0x2b,0x91,0xf3,0x10,0x48,0x7d,0x89,0x82,0xfa, +0xf4,0x2e,0x41,0x55,0x0f,0xf3,0x0b,0x60,0x9e,0xf7,0x61,0x0c, +0xe6,0xa2,0xc3,0xaf,0xb4,0x38,0x1d,0xab,0x5b,0x2b,0xff,0x3c, +0xd0,0x04,0x6f,0x7d,0x55,0x61,0xc1,0xdc,0xbc,0xed,0xf8,0xe4, +0xa9,0xfb,0x63,0x82,0x77,0x10,0xfe,0x16,0x1b,0x14,0x4a,0x10, +0xc7,0x57,0xe6,0x88,0x05,0xea,0x07,0x5e,0x03,0x8b,0xfa,0x5e, +0xdc,0xaf,0x68,0xb0,0x4a,0x64,0xcd,0x00,0x7b,0xe0,0xed,0x2e, +0xca,0xce,0x60,0xc2,0xef,0x5d,0x2e,0xa2,0x11,0x99,0x26,0x39, +0xf6,0xa3,0xe7,0x61,0x3c,0xd3,0x1d,0x06,0x93,0x9b,0xc3,0x39, +0x7a,0xb3,0x41,0xe2,0x2c,0x99,0x4c,0xa6,0xd8,0x66,0x5a,0x0a, +0x65,0x5e,0x87,0x7c,0xc6,0xe4,0x53,0x0b,0xb4,0x73,0x4e,0xde, +0xc9,0xb2,0xe0,0x86,0xf3,0x53,0xf0,0xa5,0x04,0x40,0x7f,0xb2, +0x9f,0x0a,0x00,0x5e,0x77,0x04,0x23,0xc7,0x4d,0x97,0x5a,0x63, +0xbd,0x58,0x3e,0xba,0x29,0x9a,0x0f,0xb7,0x45,0xd3,0xdd,0x8f, +0x2b,0xfb,0x34,0x47,0x55,0x71,0xbb,0xda,0x63,0xff,0xb2,0x8b, +0x4d,0x12,0xfc,0x59,0x0d,0xd2,0x7c,0xb0,0x45,0x9a,0xf5,0x4d, +0x82,0x17,0xe9,0xed,0x68,0x93,0x77,0xa5,0x36,0xc9,0xf6,0x68, +0x93,0xec,0x0f,0xb7,0x49,0xa1,0x04,0xa3,0x81,0x82,0xea,0x36, +0xd9,0xbb,0x3c,0xdd,0x74,0x41,0x45,0x37,0x39,0x69,0xd7,0xb7, +0x0a,0x99,0x8e,0xb5,0x4d,0x52,0x6c,0x05,0xdf,0x6c,0xa5,0x2f, +0x83,0x9f,0xcc,0x26,0xf9,0x72,0x67,0x93,0xd0,0x0d,0x87,0x8f, +0x6e,0x93,0x87,0x9b,0xe4,0x8f,0xa8,0xac,0xc7,0x36,0xc7,0x7e, +0x65,0x7d,0x7c,0x5b,0xb0,0x39,0x5f,0xdf,0x1a,0x8f,0x6b,0x0c, +0xbb,0x2d,0x70,0x0b,0x43,0x98,0x43,0x04,0xf5,0xd9,0x70,0x2e, +0xae,0x02,0xc2,0x1b,0x7c,0x0a,0x09,0x2e,0xdd,0x54,0xa9,0x0d, +0x4d,0xfd,0x10,0x24,0x43,0x42,0x44,0x63,0x92,0x6d,0x8d,0xc1, +0x98,0x17,0xeb,0x2c,0x33,0x33,0x33,0x8d,0x8e,0x6a,0x00,0x9e, +0x44,0xed,0xc4,0xc2,0xa6,0xe9,0x43,0xa8,0x2c,0xe3,0x0b,0xcc, +0xa1,0x1d,0x30,0x62,0x81,0xd6,0xb4,0xef,0xf8,0x7e,0x88,0xf2, +0x8d,0x03,0x3c,0xe8,0x9a,0x52,0x2a,0x9e,0x28,0xfb,0x8e,0x12, +0x6c,0xf9,0x89,0xce,0x7c,0x8b,0xe3,0x10,0x36,0xa4,0x22,0x3a, +0x29,0x58,0x9b,0x51,0x3f,0x0c,0x06,0xbe,0x36,0x6d,0xc9,0x9e, +0x55,0x0b,0x59,0x19,0x2e,0x4c,0x12,0x55,0xfb,0x39,0x45,0x12, +0x68,0xe6,0x63,0x3e,0x61,0x3f,0x6f,0x29,0xd0,0x12,0x55,0xd8, +0x0b,0x8b,0x80,0xcd,0x70,0xa9,0xd9,0xa8,0x33,0x57,0x36,0xed, +0xe1,0xd5,0x29,0x9d,0x5a,0x5e,0x70,0xd1,0x8c,0x42,0x94,0xcd, +0xb3,0x82,0xf2,0xea,0x62,0x85,0x9b,0xa8,0xcd,0x1a,0x05,0x20, +0xb0,0xda,0xc9,0x06,0xf6,0x9f,0x1e,0xf1,0x92,0xa4,0x01,0x9f, +0x19,0x78,0x7f,0x12,0x0f,0x4a,0x2a,0x9c,0x62,0x7a,0xb1,0x37, +0x4e,0x21,0x44,0x16,0x52,0x11,0x67,0x60,0x2d,0x5d,0xd3,0xb9, +0xfb,0x92,0x98,0x79,0xcf,0xd6,0x31,0x51,0x23,0x2b,0x48,0x57, +0x43,0x65,0xa5,0x9b,0x13,0x45,0xe1,0x56,0x26,0x51,0xbc,0x8d, +0x68,0xaf,0x7a,0x95,0xca,0xae,0xc7,0x6e,0x53,0xb5,0x9f,0x9c, +0xd9,0x15,0x91,0xd8,0xad,0x68,0x83,0x75,0xdf,0x57,0xdc,0x5d, +0xb5,0xb3,0x45,0x64,0x86,0x86,0xd9,0x24,0xdf,0xcb,0xdb,0x98, +0x75,0x93,0x6c,0x46,0x93,0xe4,0x36,0x99,0x54,0x2d,0x80,0x17, +0x11,0x1f,0x1e,0x62,0x36,0xce,0x6c,0x1f,0x64,0x2c,0x76,0x84, +0x48,0x5a,0xe7,0x7e,0xb9,0x0b,0x45,0x1f,0xc0,0x56,0xec,0x84, +0x7c,0x19,0x30,0xfc,0x32,0x24,0x7c,0xa0,0xbd,0xd2,0x09,0xb7, +0xd2,0x6b,0x73,0xb4,0xda,0xa4,0xd7,0x95,0xd3,0x7c,0x4a,0xa9, +0x9f,0xe7,0x97,0x66,0xf6,0x72,0x9c,0xa1,0x97,0x76,0xe2,0x55, +0x93,0xd6,0x90,0x70,0x92,0x05,0xea,0x51,0x8f,0x42,0xcd,0xbb, +0x78,0x74,0x93,0xac,0x9b,0xeb,0xe1,0xb2,0x79,0x0d,0xb4,0x4d, +0x91,0xbe,0xe6,0x78,0x31,0xa5,0x51,0x69,0x75,0x35,0x1a,0xba, +0x81,0x4f,0xff,0x79,0x0e,0x2d,0x32,0x39,0xb3,0xc5,0x26,0x8d, +0xd1,0x0d,0x97,0x69,0x75,0x7c,0xfa,0xa1,0xf1,0x8b,0x01,0xd6, +0x8b,0xcd,0x98,0xc3,0x15,0x10,0xfe,0x68,0x38,0xbe,0xb9,0xe2, +0xab,0xa9,0xec,0x49,0xa4,0x4e,0x30,0x27,0x92,0x5d,0x1d,0x5d, +0x9a,0x42,0xe2,0x65,0x77,0xa5,0x29,0xa4,0x85,0x46,0x54,0xf2, +0x36,0x49,0x93,0x51,0x32,0xa5,0xd9,0xad,0x73,0x0d,0xb6,0x68, +0x3c,0xd7,0x89,0xe3,0xcd,0x2a,0xa5,0xca,0x92,0xff,0xf7,0xf5, +0x30,0x59,0xc1,0x30,0x6c,0x93,0x06,0x23,0x35,0x94,0x6d,0x92, +0xb5,0x1f,0x2d,0x2a,0x5f,0xb1,0x28,0x9c,0x98,0x8b,0xe9,0xe9, +0x2a,0x26,0x0f,0x51,0xbb,0x40,0x8e,0x94,0x05,0x72,0x28,0xf5, +0xcd,0xf3,0x9e,0x5d,0x8e,0x14,0x0b,0x00,0x7a,0xae,0x2f,0xa3, +0x0b,0xe4,0x5d,0x15,0x69,0x2b,0x8f,0xef,0x0e,0x67,0xc5,0xb3, +0xeb,0xdb,0x12,0xc5,0x15,0xe6,0x80,0x90,0x71,0x6a,0xdb,0xd7, +0x04,0xd6,0x9f,0x0c,0xb6,0x35,0x1c,0xc9,0xaa,0x50,0x1c,0xf7, +0xe3,0xbb,0xc1,0x3f,0x1f,0xc3,0x04,0x3a,0x5d,0x43,0x6c,0xaf, +0x79,0xc2,0x37,0x0f,0x1b,0x66,0x54,0x45,0xa6,0xbf,0xf7,0xe7, +0xe9,0xa0,0x3e,0xd3,0x5d,0x32,0x59,0x83,0xc8,0x9d,0xc9,0xf0, +0x75,0x4c,0xd6,0x27,0x46,0xec,0x92,0x06,0xc9,0x38,0x01,0x83, +0xeb,0xac,0xd3,0x61,0x2e,0x44,0x97,0x1d,0x83,0x5d,0xaf,0xe7, +0xcc,0x17,0xf3,0xd8,0xa1,0x02,0x45,0xeb,0xd1,0x8b,0x6d,0x57, +0x95,0xab,0x2b,0x57,0xa6,0xb0,0x57,0x42,0xe8,0x64,0xaf,0x4b, +0x17,0x7b,0x56,0x2c,0xbb,0x64,0x65,0x2c,0x9a,0xa7,0xd2,0x06, +0x2c,0x54,0x5e,0xda,0x82,0x4d,0x95,0x0e,0xd5,0x9b,0xac,0x86, +0x77,0x6f,0x5d,0x41,0x2a,0x4c,0x53,0xf2,0x9a,0x02,0xf3,0xda, +0x02,0xf3,0x72,0x81,0x92,0xbb,0xb5,0x25,0xbe,0x33,0x4a,0xe4, +0x18,0x15,0x61,0x3a,0x0d,0xc9,0x24,0xd4,0x6e,0x95,0x1d,0x81, +0x8b,0xd3,0x4e,0x32,0x55,0x02,0x4c,0xa9,0xb4,0x8d,0xef,0x34, +0xb2,0xbf,0x50,0xdf,0xeb,0x37,0x8e,0x63,0x2d,0x61,0x3c,0xbd, +0xcc,0x55,0xd2,0xdf,0x53,0x33,0xc9,0x73,0xca,0x14,0xbd,0x35, +0x49,0x32,0x35,0x81,0x6a,0x02,0x59,0x10,0x56,0xb9,0x4a,0x6b, +0xf8,0xad,0x39,0xdd,0x16,0xea,0xb7,0xd2,0x73,0xab,0x4b,0x88, +0xb6,0x92,0xf9,0x81,0x77,0x06,0xaa,0x22,0x1d,0xef,0x76,0xd3, +0x01,0x2d,0x93,0x3f,0x48,0x47,0x2c,0xe8,0xb8,0xb3,0xe9,0x90, +0x4d,0x98,0x6b,0x42,0xf2,0x2a,0x42,0xb4,0x12,0x37,0x47,0xa5, +0x9b,0x38,0xc7,0xb1,0x00,0xa2,0x4e,0xda,0x11,0x6d,0x96,0xe2, +0x30,0x83,0x27,0x0c,0x5e,0xb3,0x37,0x33,0x50,0x7f,0x75,0x05, +0xa3,0xef,0xfd,0xbd,0x3b,0x26,0xad,0xc4,0xf7,0x03,0x82,0xb5, +0x08,0xb1,0xf8,0x80,0x6c,0xa4,0x39,0x20,0x63,0x81,0x84,0x48, +0x93,0xe3,0x4b,0x14,0x51,0x9b,0x36,0x6e,0xa8,0x84,0xef,0x57, +0xf4,0x23,0xc6,0x5f,0xd7,0x7c,0xe7,0x0d,0xca,0xde,0x2c,0x6b, +0x09,0xc2,0x48,0x81,0x0f,0x9f,0x85,0x95,0x64,0x34,0x2a,0xc9, +0x68,0x54,0x92,0x11,0xec,0x43,0x05,0xb1,0x8b,0x95,0x32,0xef, +0x96,0xd3,0xbb,0x7d,0x78,0x0c,0x09,0xc7,0x4f,0x71,0x64,0xcf, +0xf3,0x11,0x62,0x82,0x17,0xd4,0xf3,0x7b,0xdd,0x74,0x9b,0x71, +0x42,0xac,0x6f,0x88,0x67,0xe4,0xf1,0x7a,0x55,0x8e,0x0b,0x75, +0x1c,0x5e,0xfa,0x66,0x70,0x56,0x55,0x73,0x38,0x5d,0xff,0xcf, +0x38,0xef,0x09,0x56,0x6b,0x9e,0xf7,0x5d,0x2d,0x5c,0x0d,0x5d, +0x53,0x9c,0xeb,0xb9,0xba,0xb5,0x1b,0xb9,0x99,0x32,0xf0,0x74, +0x2b,0x29,0xee,0x34,0x14,0x7d,0xaf,0x18,0x35,0x0a,0x89,0xc1, +0x31,0xc5,0x30,0x09,0x18,0x2a,0x40,0x40,0x3a,0xf0,0x3a,0x86, +0x10,0x08,0xa5,0x0b,0x55,0x78,0x7b,0x78,0x88,0x8c,0x08,0x39, +0x8f,0x9f,0x81,0xb5,0x81,0xda,0x49,0x75,0x41,0xc9,0xa6,0x40, +0x67,0x79,0x67,0x67,0xc9,0xfd,0x50,0x67,0x79,0x67,0x67,0xa1, +0x2f,0x95,0x88,0x69,0xc4,0xac,0x0b,0xf7,0x03,0xee,0x80,0x75, +0x1c,0x61,0xa5,0xe0,0x8d,0x90,0x1d,0xc9,0xd4,0x1e,0x8f,0xd1, +0x1d,0x31,0x2a,0x3a,0x5b,0xb3,0x6d,0xcd,0x82,0x79,0xa3,0x44, +0xf8,0xaf,0xcc,0x92,0x39,0x3e,0x2b,0x5d,0xbf,0x7e,0x0e,0xf4, +0x07,0xea,0xa1,0x57,0xbc,0xc9,0x48,0x2a,0x4e,0x75,0x07,0x79, +0x24,0x58,0x99,0x0c,0x7c,0xee,0xb8,0x51,0xd2,0x13,0x3c,0xed, +0x88,0x46,0xe0,0x93,0x7a,0xb1,0xea,0xae,0xb1,0x6c,0x3f,0xb5, +0xeb,0xad,0x05,0x7c,0x15,0x34,0x23,0x89,0x1a,0xca,0x6b,0xd2, +0x0d,0xbd,0x0d,0x19,0x03,0xbc,0x49,0xe6,0x91,0x0b,0x25,0xbc, +0x9e,0x0e,0x67,0xcb,0x4e,0x46,0x3f,0x9e,0xbe,0x45,0x7a,0x15, +0xf8,0xfa,0xa6,0xe2,0x50,0xc8,0x21,0xbe,0x28,0xdb,0x91,0x9f, +0x4a,0x0c,0x7b,0x78,0x2b,0xb5,0x0b,0x80,0x0d,0x55,0xa0,0xd7, +0xa0,0xe7,0x04,0x84,0x10,0x1a,0x37,0x75,0x47,0x75,0x05,0xb4, +0x8f,0x84,0xb0,0x24,0x83,0x26,0x44,0x79,0x1e,0x51,0x78,0x2e, +0xf3,0xf5,0xe8,0xa9,0x6d,0xe2,0x32,0xd0,0xad,0x8a,0xe9,0x60, +0xac,0xae,0x93,0xe2,0xc7,0x01,0x82,0xde,0xdf,0x2b,0x4e,0x41, +0x78,0x98,0xf5,0x5c,0xc5,0xd0,0xcf,0x69,0xbb,0x81,0x64,0x51, +0xf2,0xd6,0x88,0xd2,0x78,0xa8,0x44,0x85,0x05,0x91,0xe0,0xed, +0xed,0x9d,0xdb,0x45,0x32,0x79,0x12,0x14,0xba,0x3c,0xda,0x3e, +0xde,0x07,0xa3,0xf7,0xa3,0x36,0xde,0x6d,0x89,0x57,0x8d,0x77, +0xfb,0xdb,0x21,0xbe,0xb6,0xfb,0x61,0xfe,0x31,0xc9,0xcb,0xe6, +0x26,0x83,0xdd,0x69,0xeb,0x1d,0x09,0x93,0xb6,0x39,0x3b,0x4f, +0xa9,0xa4,0xcd,0xb2,0x9c,0x40,0x06,0x7d,0x75,0x1e,0x4a,0x42, +0x3f,0xa1,0x52,0x8a,0x18,0x18,0x2a,0x13,0x0a,0x85,0x90,0x08, +0xfd,0x40,0xcf,0xe9,0x21,0xbf,0x2a,0x3a,0x27,0xae,0x54,0x8a, +0xc5,0x1b,0x96,0x0f,0x9f,0xf5,0xa0,0x3c,0xc7,0x4d,0x3b,0xef, +0x94,0xef,0x47,0x79,0x14,0x5a,0xf2,0x44,0xa9,0x46,0xa9,0x71, +0x19,0x7d,0xc4,0x91,0x8b,0x4b,0xa1,0x89,0xcd,0xbb,0xc3,0xac, +0xa9,0x94,0xc8,0x25,0xd5,0x4e,0x64,0x94,0x82,0xd6,0xf9,0x66, +0xe6,0x1a,0x6a,0x2c,0x3a,0x38,0xfe,0xbb,0x3b,0xbf,0x4f,0x3d, +0x69,0x73,0xc8,0x14,0xd0,0x4d,0x99,0x56,0x5d,0x04,0x16,0xdf, +0xdf,0x55,0x81,0xe5,0x7a,0x9c,0x31,0x8b,0x22,0xb3,0x3d,0x06, +0xb3,0x5a,0x8e,0xfc,0x26,0xd3,0x22,0x39,0x1e,0x5d,0x60,0xe4, +0x9b,0xcd,0x72,0x09,0x08,0x53,0x57,0xea,0xef,0xf2,0xa8,0x73, +0x67,0x31,0x48,0x78,0xd7,0x55,0xb6,0x9b,0x08,0x56,0x34,0x1d, +0x05,0x3c,0xf4,0x1b,0x51,0x2c,0xe6,0x77,0x0f,0x27,0x3f,0x92, +0x50,0xa4,0xbd,0xbb,0x6a,0xe9,0x51,0x7d,0xa3,0x5a,0x84,0x64, +0x83,0x7b,0x9d,0x1a,0x89,0x2d,0xe4,0x2f,0x88,0xad,0xca,0x0e, +0xfd,0x8e,0xba,0xdb,0x6a,0xb3,0xc4,0x19,0x92,0xd1,0xcb,0x8e, +0x1c,0xcf,0x4c,0x52,0x0c,0xf7,0xca,0xc3,0x1d,0xd8,0x27,0x2a, +0xb2,0x62,0x90,0xd3,0xc9,0xe1,0x80,0x1f,0x64,0xc7,0xd7,0xd0, +0x64,0x63,0xd2,0x90,0x10,0x67,0x51,0xe3,0xf8,0xae,0xd4,0xd0, +0x7e,0x9c,0x43,0xfc,0xdf,0xe7,0xc5,0x78,0x61,0x07,0x44,0x7d, +0x35,0x48,0x83,0x7a,0x37,0x89,0xc8,0x75,0x7c,0x6e,0x94,0x3e, +0xa8,0x18,0xab,0x21,0x63,0x05,0xcd,0x90,0x4d,0x3a,0xb0,0x58, +0x46,0x03,0xa8,0x67,0x31,0x36,0x73,0x1e,0xf9,0xd8,0x83,0xd7, +0xad,0x57,0x60,0xa4,0x86,0xf6,0xd4,0x60,0x7b,0x2a,0x2b,0x53, +0xe8,0xed,0x24,0xaf,0x52,0x35,0xf0,0xc2,0xf9,0xd6,0xf3,0x0d, +0x95,0x4b,0x8a,0xc2,0x52,0xe8,0xbe,0xa5,0xf1,0x7d,0xe9,0x4f, +0xc3,0xbd,0xe6,0x73,0xe9,0x54,0x43,0x28,0x7c,0x13,0xaf,0x2f, +0x8d,0x03,0xad,0x87,0x3c,0x3f,0xe3,0xe1,0x21,0xe7,0x1f,0x51, +0x4e,0xd4,0x0f,0x7c,0xdd,0x3c,0x22,0xc4,0xe3,0x29,0xbe,0x37, +0x90,0xab,0x2f,0x31,0x9b,0x8f,0xcc,0xf9,0xf3,0x0f,0x1c,0x67, +0x0c,0xe1,0x42,0xdf,0xdf,0x5a,0x2b,0xb8,0xb5,0x8b,0x3f,0x25, +0x9d,0x56,0xb8,0x0d,0x91,0x95,0x46,0x18,0x7d,0xc8,0x94,0x0d, +0x91,0x77,0x04,0xa9,0x7e,0x62,0x8f,0x7d,0xfe,0xfb,0x8e,0xc5, +0xac,0xad,0xc8,0x5d,0xf6,0x11,0x11,0x58,0xbb,0x05,0xf7,0x10, +0x11,0x5d,0xe3,0x46,0xd5,0x2b,0x2e,0x53,0x99,0xa7,0x4f,0x85, +0xe3,0xa9,0xb5,0x78,0x64,0x54,0xd5,0x1e,0xa3,0x05,0x59,0xad, +0xc4,0x6e,0x5c,0x15,0xff,0x5e,0x35,0x8d,0x8c,0xc9,0x54,0xf3, +0xc8,0x98,0x7c,0x97,0x48,0x6d,0x3d,0xe9,0xcc,0x4b,0xd1,0x9d, +0x32,0x41,0x24,0x67,0x45,0x27,0xf2,0x4f,0x57,0xab,0x61,0x2e, +0xed,0x6d,0xc9,0xfa,0x16,0xcc,0xe3,0xf3,0x6a,0xc8,0xbc,0x00, +0x99,0x2b,0xc7,0x2d,0xbb,0xbe,0x65,0xa3,0xc5,0xf2,0x54,0xb7, +0x2b,0x1b,0xe2,0xd3,0x13,0xe4,0xa0,0x9e,0x17,0xe2,0xf3,0x28, +0xa7,0xf8,0x7a,0x1b,0x58,0xdb,0xbc,0xdb,0xad,0x67,0xf8,0x02, +0xef,0xd7,0x20,0x61,0x4d,0x83,0x84,0xd0,0x20,0xfb,0x94,0xb9, +0x63,0xfc,0xf7,0x3a,0xee,0x2e,0x0d,0xf0,0xc7,0xb0,0x93,0x23, +0x32,0x77,0x3b,0x63,0xfb,0xeb,0xb7,0x1d,0x0b,0xce,0x59,0xf4, +0xdb,0xce,0xde,0x7c,0x90,0xbd,0x7a,0x15,0xde,0x1f,0xe4,0x03, +0x46,0xeb,0x75,0x32,0x59,0x40,0xbe,0x5f,0x01,0xf9,0x63,0x0b, +0xc8,0x65,0x01,0x63,0xd2,0x36,0xfb,0xd5,0x02,0x6c,0x0e,0xf8, +0x2b,0xf4,0xd3,0xc1,0x6f,0x34,0xc8,0xe8,0x10,0xcc,0xda,0x3a, +0x59,0x4f,0x27,0x77,0x70,0xa0,0xd0,0xe9,0x9e,0xaa,0x1b,0xa2, +0xe9,0x33,0x9c,0xc8,0x3f,0x50,0x19,0x3b,0xb9,0x9c,0x61,0xf0, +0xd3,0x62,0x42,0xbb,0xb1,0x60,0x9a,0x54,0x52,0x87,0x0a,0xfc, +0x2c,0xf4,0x73,0xd0,0x99,0xa1,0xbf,0xae,0xf3,0xef,0x26,0xba, +0xdd,0x2c,0x88,0x88,0xde,0x2c,0x8c,0x90,0x52,0x5a,0xd3,0x82, +0xb8,0x2c,0x10,0x91,0x59,0x48,0xeb,0x21,0xb6,0x98,0xf6,0xa9, +0x00,0x48,0x07,0x23,0xe0,0x36,0x5e,0xad,0x45,0x1e,0xf8,0xe3, +0x51,0x1e,0x37,0x0b,0x3d,0x7c,0xcf,0xef,0x3c,0xc3,0x4b,0x5d, +0xd7,0x11,0x82,0x43,0x7a,0x88,0x89,0xd0,0x6f,0x00,0xee,0xc0, +0xb0,0x0f,0xee,0xef,0xb3,0xf0,0xc0,0x58,0x3c,0xc0,0x59,0xa8, +0x1a,0x12,0xb8,0x24,0xb1,0xd8,0x96,0x0b,0x62,0x73,0x41,0x6c, +0xc6,0x71,0x79,0x40,0x94,0x87,0x51,0x1e,0xd2,0xb4,0xde,0xee, +0x3b,0x7d,0xe2,0x03,0xa4,0x6b,0x62,0x31,0x0f,0xfc,0xf1,0x28, +0x8f,0x9b,0x23,0xb1,0x79,0x70,0x9e,0x33,0xb1,0x08,0x0e,0xe9, +0x21,0x26,0x22,0xb1,0x39,0x10,0x9b,0x1b,0xc4,0xe6,0xe1,0x81, +0xb1,0xa6,0x81,0xc4,0xaa,0x11,0x8b,0x4b,0xf2,0x54,0x8b,0x32, +0x87,0x4d,0x6a,0x7a,0xc4,0x28,0x33,0x46,0xf2,0xd9,0x8a,0x43, +0x91,0x31,0x20,0x6d,0x98,0x7e,0x99,0xf3,0x22,0xa4,0x1a,0x40, +0x85,0x77,0xb5,0x83,0x64,0xaa,0x4d,0x1f,0xd6,0xbd,0x48,0x1f, +0xb2,0xa9,0x4c,0x9f,0x86,0xb4,0x61,0xfa,0x65,0x66,0x4b,0x7a, +0x24,0xcf,0x55,0x78,0x17,0xeb,0x91,0x74,0xea,0x12,0x28,0x04, +0x39,0x14,0xd1,0x47,0xa1,0x06,0x43,0x0d,0x3a,0x85,0x90,0x0b, +0xec,0x4a,0x92,0xeb,0xba,0xe3,0x56,0xbf,0x2e,0x60,0xcd,0xff, +0xee,0xef,0xdd,0x9d,0x56,0xc7,0x03,0xc3,0x86,0x68,0xe1,0xad, +0x39,0xa9,0x2c,0x17,0x79,0x70,0x00,0xf4,0xeb,0x26,0x8c,0x8c, +0x06,0xbc,0xbf,0x3f,0x38,0x00,0xda,0x73,0x23,0x51,0x73,0x8f, +0x9f,0xc9,0xe0,0x47,0x34,0xd8,0xa0,0x22,0x95,0xeb,0x3b,0x8b, +0xb9,0xe3,0x59,0x1b,0x68,0xc6,0xd6,0x41,0xf4,0x01,0x5f,0x48, +0x49,0x9b,0x42,0x4f,0xfb,0xa0,0x9a,0xe3,0x3b,0x15,0x4a,0xad, +0xb4,0x3b,0x2b,0x6d,0x0e,0xc1,0xf9,0x5d,0x1a,0xeb,0x08,0xc8, +0x3b,0x8f,0x53,0x23,0x77,0x5c,0x80,0x48,0xef,0x6c,0x08,0x75, +0xb3,0xa7,0xa9,0x5c,0xa3,0x7e,0xdf,0x99,0xe3,0x1e,0x0e,0x3a, +0x67,0xc0,0xbf,0x3b,0x34,0x77,0xe9,0x0f,0xc5,0xd0,0x9f,0x3b, +0x07,0x9a,0x95,0x20,0xf8,0x6b,0x4e,0xb0,0x78,0x73,0x37,0xdf, +0x63,0x92,0xf0,0x19,0x3e,0xfa,0xfd,0x40,0xa5,0xd0,0x26,0xc0, +0xe7,0x78,0x6a,0x53,0xa4,0x43,0x4c,0xfe,0x46,0xdc,0x51,0xd4, +0x77,0xde,0x6c,0xe6,0x13,0xb4,0x95,0x9d,0x6f,0x16,0xe2,0xe3, +0xc7,0x4d,0x9c,0xf2,0xd7,0xdf,0xe2,0xc9,0x5c,0x7e,0xff,0x78, +0x0d,0x5c,0xe3,0xcf,0x2f,0x56,0x09,0x7f,0xbc,0x81,0xc9,0xe8, +0x0a,0x3f,0xe9,0x95,0x7b,0x2c,0xe8,0x62,0xb3,0x1e,0x1b,0x17, +0x78,0x7c,0x00,0x93,0x1a,0x8b,0xae,0x78,0xcb,0x99,0x2d,0x38, +0x3c,0x42,0xf5,0xd7,0x1f,0x5f,0x23,0x0c,0x5e,0x04,0x42,0xe0, +0xf9,0xc3,0xd0,0xb9,0x00,0xfe,0x02,0x44,0xeb,0x1d,0x48,0xef, +0x43,0x39,0x24,0x9c,0xc8,0xf6,0xe5,0x02,0xea,0xf2,0x50,0x1e, +0x02,0x12,0x19,0xbe,0x49,0xa6,0xd3,0x84,0x9f,0xc1,0x7f,0x30, +0x9f,0x09,0xab,0xb2,0xcf,0x37,0x30,0xe7,0x7a,0x38,0x27,0x81, +0xc9,0x4c,0x8b,0xf9,0xfa,0xfa,0xc1,0x2c,0x08,0x24,0x32,0xbc, +0xd9,0x8f,0xbe,0x37,0x16,0x69,0x3f,0x82,0x40,0xec,0xcc,0x81, +0x00,0x06,0xec,0x6f,0x30,0xfd,0xe2,0x6b,0x30,0x2b,0x72,0x05, +0x5b,0x3e,0x57,0xf2,0xdd,0x65,0x3d,0x4a,0x01,0x80,0x28,0xd3, +0xb2,0x70,0x48,0x11,0x5d,0x1a,0x77,0x7b,0x48,0x01,0x31,0x6e, +0x09,0x6e,0x5d,0x18,0x53,0x0d,0x81,0x28,0xdf,0x0f,0x4f,0xbe, +0x13,0x4d,0x95,0x40,0xd5,0xe1,0x92,0xb0,0x3b,0x11,0x96,0x44, +0xad,0x0e,0x1b,0x01,0xee,0x44,0x55,0x27,0x84,0x75,0x18,0x4d, +0xf8,0x07,0x10,0x97,0xc4,0xb3,0x1e,0x27,0x81,0xee,0x46,0x57, +0x14,0xdc,0x5a,0x64,0x08,0xb8,0x13,0x55,0x85,0x48,0xd7,0x21, +0x7b,0xb3,0x47,0x45,0x8b,0xc2,0x5e,0x89,0x0b,0x81,0x6a,0xb1, +0xa8,0xc1,0xc4,0xce,0x47,0xfa,0x55,0xeb,0x3d,0xa5,0x69,0xf9, +0xed,0x1b,0x4c,0x44,0xac,0x91,0xf3,0x74,0xf8,0xe4,0xe9,0xe8, +0xc9,0xd3,0xf8,0xc9,0xd3,0xb7,0x4f,0x9e,0xbe,0x73,0x2a,0xe0, +0x00,0x66,0x76,0xfc,0x74,0x72,0x5c,0x4e,0x15,0x18,0xbe,0xec, +0x3c,0xfd,0xa6,0xf3,0xf4,0x8d,0x63,0x6a,0xf3,0x3f,0x4f,0x8f, +0x9b,0x58,0x3f,0x1d,0x8d,0xf0,0xf5,0x6a,0x3a,0x74,0x22,0x4a, +0x60,0xf4,0x8c,0x9b,0xf1,0x32,0x46,0x46,0xc6,0x98,0x0c,0x24, +0x33,0x6c,0x60,0xcc,0xfa,0x2f,0xc3,0xf9,0x66,0xb8,0xa2,0x32, +0x63,0x18,0xf1,0xf8,0xf3,0x1b,0x7c,0xe3,0x0a,0x7e,0x3f,0x5d, +0xae,0x92,0x29,0x85,0x31,0xf6,0x5f,0x36,0x34,0xda,0xfd,0xcb, +0x66,0x8a,0xa1,0x4f,0x37,0x57,0x9b,0x14,0xd7,0x44,0xde,0xc4, +0xcb,0x75,0x4c,0xaf,0x99,0xf9,0xce,0x77,0xe3,0xf5,0x82,0xbf, +0xbe,0x5d,0xdc,0xca,0xc8,0xcf,0xe3,0x31,0x7f,0x16,0x8a,0x2f, +0xd6,0x02,0x48,0x61,0x32,0x98,0x02,0x2e,0xdf,0x2c,0x9d,0x0b, +0xe7,0xb2,0xb9,0x60,0x2e,0x92,0x8b,0xe3,0x92,0x78,0xc0,0xc3, +0x42,0x5a,0x39,0x1a,0x4f,0xb2,0x44,0x9a,0x68,0x83,0x86,0xd3, +0x8b,0x25,0xb4,0xed,0xac,0xfc,0x17,0x78,0x3f,0x9e,0x33,0x02, +0x87,0x39,0xd5,0x17,0x8e,0xf8,0xdc,0x71,0xf0,0x21,0x0d,0x8e, +0xda,0xfa,0x16,0x16,0xb1,0xfe,0x0c,0x52,0x2b,0xc0,0xd5,0xa0, +0x46,0x11,0xc6,0x68,0xe8,0x7a,0x0d,0x01,0x5b,0x40,0x61,0x11, +0x62,0xe7,0x10,0x07,0xd4,0x64,0x8d,0xd2,0xc8,0x0c,0xb1,0xab, +0xa6,0x0d,0xd0,0x82,0x91,0xde,0x06,0xc2,0xa1,0xdf,0x06,0x84, +0x2a,0x3e,0xc8,0x1a,0xea,0x4f,0x00,0x37,0x8f,0xef,0x9e,0x08, +0x23,0xc2,0x6d,0xc7,0x27,0x7e,0x60,0xbc,0x38,0x91,0x3f,0x50, +0x5d,0x5f,0x46,0x89,0xe1,0x50,0x85,0xd9,0xae,0xc0,0x70,0xfe, +0x10,0x37,0x09,0xd4,0xce,0xb7,0x2f,0x17,0x19,0xba,0x19,0x1a, +0x3c,0xa4,0x4e,0x69,0x04,0xca,0x8c,0xb1,0x19,0x88,0x20,0x95, +0xfc,0xfb,0xee,0xf2,0x9d,0x65,0x9d,0x6b,0xa6,0x99,0x82,0x47, +0x0d,0xc0,0x69,0x5d,0xf3,0xf1,0x33,0x7e,0x0e,0x8b,0x12,0x9a, +0x08,0xd2,0x3c,0x8b,0x9f,0x1d,0xa9,0x5a,0xda,0xe3,0x39,0xd4, +0x80,0x9a,0xb1,0x22,0xc1,0xf3,0x8e,0x5f,0x9c,0x3d,0x8b,0x4f, +0xc5,0xb1,0x4f,0xdb,0x86,0x54,0xcf,0xd5,0x1a,0x34,0xe6,0xb8, +0x3b,0x49,0xad,0x0f,0xf5,0x5a,0x2f,0xbe,0x5e,0xdc,0xc5,0xab, +0xd7,0xc3,0x14,0xb7,0x5a,0x92,0xe8,0x79,0x33,0xe9,0xaa,0xbb, +0x48,0x6e,0xe9,0x3e,0x56,0xc2,0xd9,0x07,0xe0,0xc1,0x9e,0x5d, +0xc9,0xad,0xee,0x4a,0x5e,0x4d,0x4b,0x36,0x8d,0x30,0xd8,0x8e, +0x8d,0xc4,0x7b,0xfa,0x7c,0xbf,0x4e,0x56,0x25,0x16,0xcf,0x8f, +0x0c,0xe6,0x8a,0x0c,0x65,0x21,0x91,0x82,0x5d,0x6e,0x22,0x45, +0x47,0x65,0x5b,0x15,0xdb,0x9e,0xf3,0x34,0x90,0xab,0x4c,0xf7, +0xf1,0x73,0xaa,0x4f,0x7e,0x10,0x45,0x89,0x47,0x97,0x2f,0x2a, +0xf6,0x35,0x70,0x2e,0x10,0x49,0xd6,0xd9,0xd2,0xa4,0xd2,0x49, +0xf0,0x14,0x4c,0x49,0xea,0x18,0x8e,0xcb,0x06,0x64,0xff,0x85, +0xb5,0xda,0x1a,0x1d,0xe8,0x2e,0x8e,0x6f,0x54,0x21,0x29,0x8d, +0x6d,0x56,0x5a,0x5a,0x48,0x2c,0xf4,0x1c,0x02,0xb1,0x7a,0x98, +0x00,0x2b,0x77,0x32,0x04,0x15,0xbd,0xcc,0x86,0xe5,0x48,0x05, +0x77,0x29,0x5f,0xb4,0x33,0x07,0x65,0xcb,0x1c,0x80,0x3c,0x3f, +0xc4,0x05,0x80,0x1f,0x62,0xd7,0x1c,0xa8,0x3d,0xb3,0x03,0x7d, +0xbd,0x58,0xdc,0x6c,0x96,0x85,0x0c,0x1c,0x59,0x9f,0x89,0xc7, +0xb3,0x07,0xca,0xb1,0x06,0xbd,0x8a,0xec,0x7b,0x96,0x5c,0x83, +0x86,0xc6,0xd5,0x5d,0x14,0xf0,0xb8,0x5f,0xc8,0xb0,0x4f,0x99, +0x95,0x19,0x1f,0xae,0x71,0x79,0xa0,0xaf,0x44,0xb1,0x37,0x05, +0x35,0xa8,0x96,0xf1,0x0a,0xb7,0xab,0x80,0x92,0xe3,0xbf,0x3f, +0x3d,0x2e,0x98,0x67,0xdf,0x0f,0x27,0x69,0xf4,0xc1,0x69,0x3a, +0x1d,0xc7,0xf1,0x2f,0x3a,0xce,0x13,0xc7,0x0f,0x3a,0x4e,0xe0, +0x6c,0x0b,0x70,0x00,0x34,0xec,0x54,0xfb,0x36,0x56,0xb2,0xbd, +0x3f,0x51,0x1d,0x6b,0xb0,0xf5,0x3f,0x7d,0x20,0x6b,0x01,0x7c, +0xb4,0x0b,0xbc,0x5c,0x55,0xce,0x2c,0x06,0x51,0xc8,0xfe,0xd9, +0x83,0xd9,0x4b,0x59,0xc6,0x1d,0xbb,0xba,0x6e,0xb5,0x29,0x0c, +0x4c,0x35,0x70,0xfb,0xcb,0x12,0x76,0xc5,0x54,0x77,0xa2,0x35, +0xaf,0xbf,0xf4,0xdb,0xa0,0x68,0xe3,0x8f,0xcf,0xfa,0xe5,0xa3, +0xb2,0x8a,0x25,0x00,0x91,0xf7,0xab,0x8f,0xc9,0xfb,0x34,0x6c, +0xdf,0xdf,0x87,0x6d,0x81,0xe2,0xfd,0xbe,0x28,0xc2,0x46,0x85, +0xc6,0x44,0x42,0x4e,0x00,0xcb,0xd7,0x8f,0x22,0xc4,0x5e,0x8f, +0x10,0x28,0x66,0x8f,0x43,0xc1,0xcd,0xdb,0x08,0x45,0x35,0xbe, +0x79,0x24,0x01,0x62,0x59,0x43,0xe4,0x5e,0x56,0x4a,0x95,0xc1, +0xb3,0xf3,0x28,0x6c,0xf7,0x9c,0xef,0xbf,0x81,0xae,0xf4,0xe9, +0x37,0xd0,0x81,0xde,0x3c,0xaa,0xb8,0x37,0x46,0x55,0xb1,0xb8, +0xbf,0xee,0x9d,0xbb,0x42,0xfb,0x33,0xd3,0x11,0xcd,0x5d,0x2d, +0xd5,0x62,0x39,0xea,0x6f,0x8f,0x2d,0x67,0xb6,0xa8,0x2a,0x27, +0x7b,0xb8,0x03,0x79,0xfe,0xdb,0xdd,0x40,0xdc,0xc3,0xf2,0x47, +0xb1,0x4d,0x5b,0xd3,0x4f,0xc3,0x20,0x10,0xc4,0xbc,0xfb,0x68, +0x14,0xf1,0x33,0x40,0xf1,0x0c,0x50,0xfc,0xac,0x48,0x45,0x8b, +0xd2,0x77,0x9e,0x3a,0xe5,0xb5,0x21,0x88,0xdc,0x6a,0x3d,0xb9, +0x84,0xd9,0x45,0xbc,0x22,0x3d,0x69,0x45,0xfd,0x0d,0x06,0x6a, +0xa5,0x1d,0x41,0x13,0x56,0x25,0x82,0xc6,0xb3,0xa2,0xbf,0xd1, +0x3a,0x0e,0xb4,0x59,0x39,0xc9,0xd0,0x57,0x14,0xf9,0x35,0x5e, +0x61,0x16,0x63,0x45,0x40,0x41,0x59,0x29,0xd0,0xca,0x7e,0x5c, +0x8e,0xfa,0xd2,0x8e,0x42,0x09,0x6e,0x3f,0x03,0x55,0x51,0x15, +0xfb,0xbe,0x94,0x5d,0x98,0x18,0x5f,0x17,0x28,0x33,0x7a,0xac, +0x3f,0xab,0xa0,0x5a,0xbc,0xa9,0xfd,0x4d,0x31,0x1b,0xf5,0x33, +0xe8,0x5f,0x56,0xf4,0xa7,0xb3,0xef,0x67,0xd0,0x85,0xac,0x38, +0xd1,0x47,0xa0,0x6f,0x94,0x98,0xc8,0xb8,0x79,0x35,0x01,0x84, +0xbe,0x8a,0xc9,0xa2,0xf8,0xbf,0xd5,0x65,0xe6,0x15,0x08,0x43, +0x92,0x0d,0xd6,0xd2,0x42,0xf4,0xdb,0xaa,0x14,0x14,0x5a,0x90, +0x59,0x2b,0x85,0xb8,0xf3,0xce,0x8e,0x93,0x52,0x66,0x48,0x16, +0xc5,0xff,0x2c,0xc5,0xcb,0xc6,0x9d,0x80,0x9d,0x3b,0x9c,0x7e, +0xcf,0x43,0xb7,0x16,0x32,0x7e,0x1a,0x9d,0x86,0xf2,0x5f,0xd2, +0xa3,0x5f,0x26,0x0d,0x3d,0x9e,0x0f,0x81,0x61,0xca,0x54,0xe0, +0x2b,0xfc,0x87,0xb3,0x0e,0x74,0x89,0x59,0x27,0xd4,0xe6,0xb6, +0x10,0x7e,0xd3,0xc6,0x14,0x51,0x7f,0x5d,0x8f,0x2d,0xb3,0x90, +0x63,0xbf,0x4a,0x17,0x65,0x38,0xd7,0x79,0xfa,0xae,0xf9,0x74, +0xd6,0x7c,0x3a,0xf9,0x51,0x2e,0xec,0xb4,0x9e,0x7e,0xfd,0xb3, +0x53,0x2a,0x25,0x81,0xdc,0xf6,0x1a,0x13,0x4c,0xab,0xbe,0x7a, +0xf3,0x1d,0xdf,0x91,0x76,0x78,0xd8,0xc0,0xe9,0x33,0x8d,0x74, +0x4e,0x3b,0x08,0x82,0x66,0x10,0xc2,0xff,0x3f,0x06,0x41,0x87, +0xfe,0x6f,0x41,0x14,0x20,0xed,0x95,0x28,0xfa,0x96,0xde,0x5e, +0xee,0x94,0xe2,0xfd,0x1a,0xc8,0x16,0xf1,0xd4,0xb8,0x65,0x99, +0xaf,0x68,0x13,0xb3,0x02,0x98,0x91,0x29,0x32,0x44,0x8a,0x9c, +0x0b,0x24,0xe9,0xb7,0xc3,0x6f,0xd9,0xe6,0xef,0x91,0xef,0x23, +0xcf,0xbd,0xea,0x8a,0x59,0x2f,0xb8,0x62,0x51,0x09,0x40,0x25, +0x29,0xbb,0x9c,0x3b,0xca,0xbe,0x4b,0x30,0xe6,0x32,0x43,0x18, +0x9f,0x98,0x33,0x39,0x8c,0x3b,0x86,0xb8,0xd2,0x5c,0xae,0x34, +0x29,0x44,0x51,0xb5,0x66,0xd1,0x38,0x36,0xda,0xc8,0x6a,0xa7, +0x85,0x85,0xb5,0x03,0xbd,0x1a,0xef,0x15,0x6a,0x64,0xcc,0x70, +0x28,0x5c,0x98,0xb6,0xc8,0x85,0x5d,0x6b,0x8e,0xc3,0x80,0xe5, +0x39,0xce,0x8c,0x54,0xc3,0xc7,0xf0,0x08,0x17,0x0c,0x8a,0x3c, +0x82,0xb8,0x8f,0xe3,0x51,0x01,0xd9,0xbe,0x3c,0xd2,0xfb,0x22, +0x5e,0xa1,0x46,0x9a,0x47,0x1c,0xae,0xac,0xb6,0xcd,0x23,0x01, +0x58,0xe6,0xd1,0x35,0x28,0xe9,0xbd,0x56,0xab,0x10,0x00,0xc7, +0xb2,0xa8,0x6e,0x1d,0xe5,0xf8,0x2c,0xe8,0x56,0xb0,0xf2,0xe4, +0x2c,0x3e,0x3d,0x72,0x8b,0xcc,0xc4,0xd8,0xa6,0x44,0x89,0x47, +0xbb,0xc5,0xd7,0x47,0x31,0x98,0x8a,0xf8,0x08,0x0e,0xcb,0xbd, +0x2e,0xcf,0xe2,0x86,0xe6,0x2e,0x86,0x2a,0xd8,0x65,0x73,0x96, +0x80,0x2a,0x64,0x0f,0x47,0xac,0x3f,0x69,0x85,0x94,0x54,0xcb, +0x9e,0xcb,0xa3,0x6c,0xb4,0x16,0x56,0x06,0xf7,0x5d,0xd2,0x93, +0xfb,0x6a,0x9e,0x5d,0x0d,0x43,0xde,0x68,0xef,0xa2,0xa2,0xa6, +0x05,0x69,0x23,0x30,0xc5,0x14,0x6b,0x5c,0x20,0xef,0x7c,0x7c, +0x85,0x31,0x8d,0xfa,0xa0,0x3d,0xfc,0xd3,0x18,0x1f,0xb2,0x86, +0x3f,0x27,0x60,0x3c,0x41,0x57,0x81,0xdf,0x53,0xff,0x25,0xfc, +0x0b,0x5f,0xc0,0x1f,0x6c,0x5a,0x7c,0xd6,0x1a,0xfe,0xb6,0x43, +0xfc,0x7e,0x76,0xd2,0x86,0xbf,0xb4,0x2e,0xe7,0x87,0xcf,0xdb, +0x98,0x70,0x16,0x3c,0xa3,0xf4,0xd3,0x97,0xed,0xf8,0xcc,0x7f, +0xfe,0xfc,0xf9,0x19,0xfc,0x9c,0x84,0xa7,0x90,0xf7,0x4c,0x2f, +0x8b,0x53,0xb9,0x34,0xe6,0x7e,0x13,0xaf,0xaf,0x17,0x13,0xdc, +0x77,0xb6,0x35,0x88,0x1f,0x0e,0xfc,0x62,0xd4,0x69,0x39,0x2a, +0xac,0x88,0x3b,0x09,0x8c,0x38,0xee,0x6b,0x16,0x36,0x11,0x75, +0x5a,0x01,0x55,0x11,0x67,0x61,0x43,0xf9,0xb2,0x70,0x51,0xc4, +0x49,0x31,0xe2,0xac,0x94,0xa5,0x6d,0xc4,0xa0,0x61,0x12,0x16, +0xc2,0x66,0x3a,0xae,0x06,0xd9,0x04,0x93,0x99,0x58,0x8e,0x31, +0xcb,0xc5,0x45,0x30,0x3f,0x1c,0x54,0xb1,0xf8,0x0b,0x31,0xe9, +0x57,0x2c,0x96,0x16,0x3b,0x8c,0xfc,0x8e,0xb8,0x14,0x6e,0x13, +0x6b,0x5c,0x3a,0xf9,0x33,0xc7,0xab,0x3c,0x03,0x69,0x8a,0x67, +0x55,0xbe,0xd1,0x93,0xa7,0x93,0xea,0xac,0xe1,0x41,0x64,0xcc, +0x88,0x2b,0x33,0x0f,0x6b,0x33,0xeb,0xa9,0xce,0xe1,0xe1,0x1e, +0x88,0xbe,0x7a,0xf2,0x74,0xb9,0x03,0x91,0x54,0x38,0x95,0x59, +0xc1,0x10,0xda,0x55,0x77,0x35,0x1a,0x54,0x64,0xc6,0x9d,0xb1, +0xfa,0xac,0x7a,0xb0,0xad,0xc8,0x0a,0x86,0xd7,0xce,0x52,0xad, +0x5d,0xfd,0x1d,0x6d,0x1d,0x59,0x09,0x5f,0xd8,0xf3,0xb3,0x92, +0x60,0x68,0x4b,0xaf,0xd4,0x2d,0x5b,0xa5,0x15,0x7e,0x3d,0xf8, +0xdb,0x97,0x35,0x94,0x26,0x66,0x94,0xee,0x96,0xaf,0x98,0xa8, +0x2d,0xab,0xb6,0x36,0xde,0xb6,0xac,0xb3,0x4c,0x5e,0x44,0x1f, +0x48,0xb1,0x19,0x2f,0x11,0xa1,0x6b,0xa4,0x9f,0xae,0x17,0x4b, +0xf8,0x13,0x9b,0x73,0x46,0x56,0x81,0x6e,0x83,0x21,0x1a,0x1a, +0x44,0xbe,0x60,0xaf,0x4b,0xa0,0x59,0xae,0x31,0x25,0xa4,0x48, +0xdc,0x31,0x16,0x0a,0xab,0x9e,0x67,0x88,0x4a,0xd1,0x32,0x93, +0xa5,0xf7,0x67,0xfd,0x80,0xd6,0xbd,0xfd,0x19,0x3a,0xae,0x6f, +0xbd,0x12,0xe6,0xba,0x7e,0x2a,0x8c,0xf4,0x1d,0x5d,0x95,0x21, +0x76,0xf5,0x56,0xc3,0x4f,0xa3,0x26,0xf7,0xc3,0x7d,0x56,0x3b, +0xc9,0xd4,0xa0,0x78,0xa0,0xe7,0x4a,0xb7,0x19,0xdd,0x79,0x1f, +0xc4,0xf8,0x40,0x17,0x36,0x5c,0x64,0xea,0x10,0xec,0xee,0xc8, +0x96,0xc7,0x4b,0x35,0x8a,0xdd,0xdd,0xd9,0x72,0x66,0xa9,0x46, +0xb0,0xbb,0x53,0x97,0xbc,0x75,0xfe,0xa9,0xd4,0xb1,0x95,0x68, +0x3c,0xdc,0xad,0xb5,0x14,0x59,0xb7,0xe0,0x57,0xc8,0x6f,0xb9, +0x67,0x93,0x6c,0x5a,0xbd,0x9b,0xcc,0x88,0x3f,0xd8,0xc3,0x75, +0x89,0x35,0x95,0x12,0x6f,0x2f,0xe1,0x35,0x73,0x00,0x90,0x5d, +0xaf,0xf0,0xed,0x2e,0xdd,0x7f,0x56,0xf1,0xaf,0x9b,0x38,0x5d, +0xab,0xc2,0x45,0x18,0x0f,0x5c,0x2c,0x17,0xf3,0x34,0xfe,0x11, +0x32,0x52,0x5f,0x6a,0xbd,0x4f,0xcd,0xcb,0x3c,0x37,0xab,0xa9, +0x8f,0x57,0xe2,0xe0,0xa1,0x6e,0x93,0x72,0x28,0x80,0xd2,0x1c, +0x74,0xaf,0x48,0xc6,0xb4,0xe8,0x7c,0x8c,0x59,0xc9,0x9b,0x01, +0x3f,0x74,0x36,0xa2,0xec,0x7a,0x3d,0x9b,0x3e,0x02,0x2d,0x56, +0xe4,0x18,0xf3,0x10,0x3e,0xfc,0x28,0xe0,0xcb,0x66,0xd3,0x8f, +0xaa,0xe8,0xdb,0x6f,0xbe,0xa6,0x7a,0x6e,0x5d,0xaf,0xe8,0x8b, +0xbf,0x5a,0x2c,0xc4,0x61,0xb0,0x25,0x2e,0xa4,0x2f,0x36,0xe9, +0x5f,0xe7,0x93,0x78,0x95,0x8e,0xf1,0x90,0x14,0x26,0xb6,0x2e, +0x7c,0xba,0x98,0x2a,0x5e,0xa1,0x43,0x1e,0xf9,0xdf,0xd3,0x55, +0xa8,0x11,0x7d,0x1a,0x4e,0x23,0xdf,0x8d,0xde,0x73,0x82,0xb8, +0x2e,0x54,0xa7,0x7c,0x01,0x45,0x72,0xd2,0x17,0xa2,0x70,0x23, +0x11,0xef,0xf6,0x8c,0x34,0x5a,0xba,0xeb,0xd3,0xa7,0x83,0x34, +0x66,0x2c,0x45,0xf8,0x20,0xe6,0xc0,0x75,0x33,0x9e,0x63,0x7c, +0x35,0xe1,0x96,0x44,0xe8,0x79,0xf6,0xf5,0x30,0xfd,0xee,0x6e, +0x2e,0xaf,0x2b,0xd7,0x00,0x76,0xbc,0x3f,0xa7,0x99,0xfb,0x17, +0xbc,0x17,0x6c,0x96,0x20,0xb6,0x87,0x05,0xc0,0x37,0xc3,0xa5, +0x99,0x08,0xea,0x5a,0x24,0xfc,0x10,0x4f,0x36,0x36,0xc9,0x2b, +0x8a,0xb1,0x92,0x7f,0xc0,0x73,0xe3,0x65,0x18,0x8a,0x96,0x14, +0xf0,0xb3,0x3c,0x26,0x01,0x14,0x23,0x92,0xff,0x72,0x1b,0xaf, +0x72,0x33,0x15,0x1f,0x46,0x97,0xd4,0xbf,0x59,0xcc,0x6c,0xa6, +0x41,0x58,0x24,0x7d,0x05,0x8d,0x9a,0x7d,0x77,0x69,0xa6,0x26, +0x1c,0x25,0x00,0xbe,0x1e,0xa6,0xeb,0x0a,0xa0,0xa9,0x8e,0x96, +0x98,0x52,0x4a,0x16,0xcd,0x9f,0x70,0x48,0xa4,0xfd,0xcf,0x38, +0x4f,0x65,0xf3,0xe3,0x8b,0x79,0x22,0xfa,0x33,0x28,0x2a,0x52, +0x42,0xd0,0x42,0x3f,0x57,0xff,0x42,0xf7,0x8b,0xc5,0xe8,0xbd, +0x92,0x5c,0xf8,0x7e,0x92,0xcc,0x61,0x88,0x9d,0x8f,0xf1,0x8a, +0xe9,0x8b,0x1e,0x44,0x74,0x50,0x3c,0xed,0x58,0xe1,0xda,0x74, +0xb7,0xc2,0xab,0x33,0x26,0x11,0x00,0xf9,0x7c,0xb4,0xd4,0xeb, +0xe0,0x94,0xf5,0x82,0x70,0x6e,0xbb,0xce,0x06,0x48,0xbf,0x04, +0xdd,0x32,0xd1,0x77,0x95,0xc7,0xd9,0x72,0xb1,0x5a,0xa7,0x3d, +0xb7,0x2a,0x71,0xb6,0x98,0x6c,0xa6,0x31,0x9e,0x93,0xc6,0xdf, +0x96,0x80,0x3d,0x3c,0x74,0xc5,0x57,0x64,0x27,0x44,0x17,0x78, +0x50,0x88,0x3e,0x5b,0x17,0x10,0xe8,0x70,0x87,0x89,0x2e,0xfc, +0x8b,0xd6,0x4f,0x7f,0xf9,0xe1,0xcd,0x57,0xdf,0x7d,0x1b,0x39, +0x61,0xeb,0xb4,0xd5,0x76,0xc8,0x0c,0xa1,0xdb,0xa5,0x2f,0x5a, +0xe2,0x47,0x48,0x96,0xc5,0x08,0x9f,0xd6,0xfb,0xd6,0x8b,0x15, +0xca,0x3a,0xdd,0x47,0x8f,0x2f,0x20,0xe0,0x92,0xd3,0x01,0xd6, +0x92,0x1e,0xd2,0x32,0x25,0xf5,0xf0,0x10,0x62,0x15,0x26,0x7c, +0x07,0xc5,0x48,0xf4,0x8c,0x34,0xb7,0x84,0x58,0x9d,0xbe,0x43, +0x28,0x3e,0x22,0x00,0x08,0x1a,0x3a,0xa4,0xef,0x34,0xc6,0x8b, +0x83,0x05,0x84,0x4e,0xee,0xf2,0x8f,0xb8,0x40,0x18,0xf0,0xc8, +0x12,0xf8,0x3a,0x31,0x51,0x8c,0x0f,0x19,0xf0,0xcc,0x73,0x82, +0x1f,0x1e,0x14,0x20,0x94,0x89,0xc7,0x0d,0xce,0xe7,0xea,0xd4, +0xf3,0x5c,0x28,0x3f,0x17,0xfc,0xd8,0x22,0x82,0xfb,0x46,0xc1, +0x18,0xf9,0xf8,0x92,0x31,0x17,0x14,0x3f,0xf0,0xc5,0x47,0x0d, +0x15,0xdd,0x0b,0xec,0xca,0x11,0xde,0xc5,0x3f,0xb5,0x5f,0xe1, +0xab,0x6e,0x13,0x71,0x0d,0xcd,0x66,0xba,0xa6,0x77,0xb6,0xac, +0x37,0x39,0x20,0x47,0x4f,0xa4,0x75,0x94,0xd2,0xe0,0x76,0xc2, +0x32,0x64,0x1b,0x41,0x64,0x4f,0xc4,0x95,0xdb,0xa6,0xe3,0xd2, +0x19,0x22,0x2c,0x5c,0x51,0xc2,0x77,0xc3,0x53,0xaf,0xa5,0x2b, +0xab,0xb1,0xcf,0x50,0x29,0x7c,0x3d,0x72,0x35,0x0f,0x4a,0x99, +0x70,0x4d,0x40,0xe4,0x13,0xb6,0x31,0xeb,0xa0,0xbf,0xac,0x56, +0x8b,0x55,0xe4,0xb0,0x9e,0x7a,0x82,0xfd,0x04,0x7d,0xdc,0x9f, +0x0c,0xb1,0x77,0x3f,0xb9,0x4b,0xd6,0xd7,0x4f,0xe6,0x0b,0xe8, +0x83,0xc9,0x3a,0x19,0x4e,0xf9,0xe6,0x77,0x07,0x98,0xc6,0x79, +0x49,0x96,0xa7,0x93,0x29,0xfc,0x26,0xf3,0xf7,0xf5,0xec,0x9b, +0xc5,0xb3,0x85,0xcd,0x43,0x81,0xb0,0x74,0xb3,0xd7,0x79,0xbb, +0x2b,0xc4,0x9e,0xf8,0x89,0xd7,0x4e,0x8c,0xde,0xe3,0xcd,0xd6, +0x96,0x3a,0x65,0xa6,0x0a,0x1a,0x24,0x5f,0x39,0x49,0x34,0xec, +0x13,0x51,0x1a,0x3e,0x5c,0x22,0xa8,0x00,0x22,0xc9,0xd1,0xbe, +0xc4,0x73,0x90,0x36,0xa6,0xa6,0xa7,0xb1,0xba,0x16,0xed,0x5e, +0xa7,0x22,0x85,0xce,0xf3,0xee,0xd3,0x5a,0x12,0x3b,0x62,0x8a, +0xaa,0x5b,0x8b,0x18,0x54,0xca,0xd9,0x71,0x29,0x8b,0x8c,0x67, +0x8e,0x1d,0x04,0xd8,0x92,0x07,0x22,0xe8,0xad,0xaf,0x57,0x8b, +0x3b,0x5a,0xae,0x43,0xcb,0x80,0x1a,0xd3,0x35,0x1a,0x56,0x59, +0x74,0x88,0x69,0xeb,0x5f,0x98,0x03,0x8f,0x68,0xbd,0xd5,0x7f, +0x51,0xa3,0x51,0x99,0x66,0xcb,0x31,0x11,0x76,0xf3,0x51,0xdc, +0x9f,0xd4,0x86,0x84,0xab,0xbe,0x21,0xed,0x64,0x71,0x01,0x72, +0x49,0xd7,0x41,0xc5,0xf8,0xeb,0x00,0x54,0xa4,0x54,0x8f,0x55, +0x2a,0xab,0x5b,0x56,0x57,0xdb,0xbd,0x05,0x04,0xbe,0x29,0x63, +0x8f,0xb4,0x55,0xb3,0xc9,0xf9,0x07,0x1d,0xf9,0xe5,0xef,0x2b, +0x43,0xa4,0x74,0x11,0xdd,0xa0,0x42,0x90,0xac,0xc4,0x3f,0x45, +0x9a,0x2e,0x71,0x80,0xbf,0x68,0x4d,0xe2,0xf5,0xa3,0x74,0xa7, +0x3a,0xa9,0x36,0xcf,0x1f,0x56,0x75,0xbc,0x0f,0xb3,0x9f,0x92, +0xeb,0xb9,0x5c,0x80,0xe8,0x33,0xc6,0xcd,0x13,0x52,0xf9,0x31, +0xd5,0x64,0x6a,0x5d,0x94,0x5e,0x5e,0xfd,0x33,0x74,0xfe,0x17, +0xe2,0x9d,0x45,0x1a,0x82,0xb9,0x20,0x35,0x3a,0x53,0xb0,0xa7, +0x53,0x3e,0x56,0xff,0xef,0xc9,0x8d,0xc3,0x43,0x6b,0xa0,0xe0, +0xa7,0x33,0xcc,0x61,0x80,0xf4,0xc1,0x0e,0xbd,0xad,0x58,0x20, +0xea,0x2c,0x59,0xb7,0x5f,0xa3,0x1d,0xec,0x49,0xe6,0x56,0x95, +0x83,0x04,0x91,0x5d,0x0b,0x6d,0x33,0x34,0x5f,0x45,0xad,0x31, +0x90,0x44,0xcc,0xfd,0xbd,0xa9,0x20,0xf4,0xbd,0x6f,0xba,0xe1, +0x40,0xce,0xeb,0xda,0xad,0x63,0xd8,0xd7,0xdc,0x6a,0x4c,0x81, +0x6c,0x34,0x4a,0xe8,0xa9,0xf8,0x8f,0x1f,0xb2,0xc9,0xfd,0x53, +0x90,0xc3,0x3f,0x87,0x87,0xfb,0x8e,0xdd,0x3d,0x96,0xe2,0x8e, +0xb0,0x5e,0xb0,0xcb,0x1e,0x30,0x0e,0x31,0x96,0x43,0x4f,0x42, +0x79,0xc6,0x79,0x00,0xb0,0x6e,0x9e,0xff,0x99,0xac,0x0b,0x1f, +0x60,0x1d,0xce,0x3e,0x98,0x73,0x54,0xbe,0x64,0x1c,0x46,0xf7, +0x64,0xec,0x1f,0x63,0x9b,0x20,0x05,0x88,0x15,0x34,0xed,0xcd, +0x37,0xc1,0x30,0xad,0x05,0x0c,0xbe,0xa1,0xd1,0x37,0x5f,0x0f, +0x61,0x7e,0x41,0xf6,0xcb,0x78,0xba,0x31,0x2f,0xb9,0x44,0xc2, +0xc4,0x1d,0x20,0x1f,0x4a,0xf5,0x3f,0x08,0x3b,0xd6,0xf4,0x8a, +0xab,0x2f,0x26,0x56,0x8a,0x03,0x22,0xb1,0xd7,0x0c,0xc9,0x86, +0x97,0xe9,0xf2,0x6a,0x91,0x4e,0xb5,0xfa,0x53,0xc5,0xf1,0xdb, +0x37,0x51,0xc4,0xe0,0x5b,0xea,0x1d,0xc9,0xfc,0x76,0x71,0x53, +0xa0,0x72,0x46,0x4b,0x25,0xac,0xa7,0x00,0x34,0x8d,0x68,0xce, +0xcc,0xac,0xd1,0x77,0xc4,0xb6,0x61,0x88,0x4c,0x71,0x2e,0x86, +0x95,0x4d,0xbf,0x50,0x4b,0x8d,0x9c,0xb9,0xab,0x7a,0x38,0x9a, +0xa5,0xf5,0x44,0xb9,0x9c,0xb5,0xc7,0xd9,0x3a,0x94,0xd6,0xe7, +0xc0,0xc0,0x13,0xa7,0x4f,0xb8,0x19,0x90,0x12,0x8f,0x89,0x5e, +0x4e,0x37,0xe3,0x1b,0x9b,0x66,0x18,0xe7,0x0c,0xad,0xb2,0xbb, +0x4c,0x66,0x04,0x5a,0xf4,0x03,0xc6,0x77,0x77,0x1d,0xaf,0x0a, +0x3c,0xc0,0xfb,0xd9,0x52,0xff,0x32,0x59,0xa5,0xa6,0xb6,0x4a, +0xd2,0xbf,0xd0,0x29,0x50,0x4a,0xf5,0x7a,0x94,0x2c,0xbb,0x52, +0x7f,0xd0,0xb9,0xe8,0x73,0x8c,0x83,0xc3,0x98,0xd3,0x71,0x58, +0xb9,0x39,0x83,0x4a,0x5a,0x8c,0xa9,0x0a,0xde,0xfc,0xcb,0x28, +0xf1,0x25,0x20,0xfc,0x20,0xe2,0xc0,0x42,0xd0,0x94,0x0a,0x1b, +0x46,0x75,0x9e,0x83,0x80,0x69,0xc7,0xa2,0xfe,0x56,0x43,0xbf, +0x41,0x39,0x55,0xd1,0xa8,0x19,0x8e,0xd3,0x3e,0x32,0x2a,0xdb, +0x63,0xd6,0xa8,0x14,0xef,0xe1,0xe1,0x85,0x9c,0xa8,0x93,0x8d, +0x42,0x62,0x4a,0x67,0x50,0x69,0xba,0x07,0x5f,0x2c,0xb8,0xe2, +0x0d,0xa3,0xb3,0xd3,0xd3,0x93,0x53,0xcf,0xf4,0x82,0x86,0xf2, +0x44,0xa3,0x62,0x90,0xe6,0x50,0xdd,0x72,0x01,0xcc,0x64,0x4c, +0x14,0x99,0x9b,0xe1,0x71,0x60,0xaa,0x90,0x0f,0x63,0xf1,0x12, +0x5d,0x07,0x53,0xb8,0x9f,0xd2,0xe7,0x56,0xca,0xdd,0x3e,0xba, +0x80,0xae,0x34,0x12,0x88,0x94,0x06,0xe8,0xed,0xa9,0x0a,0x58, +0x54,0xbb,0x32,0xff,0x39,0x53,0xd6,0x92,0x61,0xba,0xf2,0x8c, +0x69,0x65,0xea,0x18,0x81,0xa2,0x5b,0x7e,0x6c,0xf5,0xf8,0xc9, +0x27,0x02,0xa9,0x55,0x92,0xf9,0x7f,0x6d,0xab,0x24,0xf3,0x8f, +0x69,0x95,0x27,0xb5,0xad,0xa2,0x1b,0xe5,0xbf,0xb3,0x4d,0x5e, +0xfd,0x99,0x6d,0x92,0x5e,0x6f,0x2e,0x2f,0xa7,0x71,0x61,0xb1, +0x49,0xdc,0x43,0x3a,0x61,0x2a,0xa2,0xc0,0x17,0x60,0x13,0xc3, +0xb0,0xab,0xa9,0xb6,0x87,0x1b,0x46,0x68,0xec,0xe2,0xcf,0x62, +0xe6,0x12,0x86,0x46,0xc3,0x53,0x28,0xd8,0xb4,0x6e,0x86,0x83, +0x48,0xc5,0x20,0xe8,0xc0,0xb7,0x83,0xf2,0x29,0x32,0x15,0x4d, +0xd4,0x0e,0x67,0xcb,0x02,0xb1,0xfe,0xdc,0xbf,0xda,0x0c,0x57, +0x93,0xda,0xeb,0x28,0x5e,0xb5,0xef,0xef,0x09,0x02,0x07,0xd9, +0xbe,0xa2,0x4b,0x4b,0x4e,0x33,0xf4,0x40,0xc7,0x49,0x4e,0x10, +0x03,0xc4,0x45,0x45,0x81,0xbe,0xa3,0x3f,0xf0,0xe7,0x9e,0xb0, +0x21,0xa6,0xe4,0x3b,0xf6,0x95,0x34,0x07,0x6a,0x94,0xb1,0x35, +0x7a,0x70,0x52,0x8f,0x9b,0xa7,0x6e,0x5d,0x4f,0x3c,0xea,0xbb, +0xc5,0x11,0x37,0x5d,0xac,0xd6,0x9f,0x15,0xac,0x13,0xd9,0xac, +0xe6,0x4c,0x53,0x12,0x61,0xd3,0xe4,0xda,0x4f,0x76,0x8a,0x91, +0xc5,0xad,0x1b,0x40,0x2a,0x2c,0x09,0x4b,0x92,0x28,0xb9,0xc3, +0x40,0xe3,0x15,0x96,0x99,0x0c,0x3b,0xfb,0x1a,0xad,0x78,0xf1, +0x2f,0x54,0x46,0xaf,0xc5,0xe3,0xed,0xd4,0x7c,0x67,0xb5,0x18, +0x8a,0x23,0x8c,0x69,0x49,0xc4,0xfe,0x28,0xa2,0x44,0x15,0x41, +0xcf,0xc8,0xe1,0xe3,0xaa,0xa4,0x1d,0x86,0xe7,0xa3,0xfb,0x7b, +0x1e,0x9a,0x40,0x0f,0x0c,0x55,0x97,0x45,0xa8,0xd1,0xf9,0xd0, +0x48,0x1b,0x29,0x25,0x2b,0xdf,0xc4,0xa2,0x72,0x58,0xfa,0xb8, +0x08,0xfa,0x06,0x11,0x73,0x78,0xbd,0x46,0xb4,0x2f,0xbf,0x09, +0xa6,0xe8,0x1d,0xc5,0xd7,0xc3,0xdb,0x04,0x26,0x75,0xb2,0xa9, +0x1e,0x6a,0x14,0xd9,0x19,0xb7,0x4a,0xbf,0x89,0xd7,0xa5,0xf8, +0x3d,0x40,0x6d,0x3e,0x76,0x76,0xb6,0xda,0x4e,0xb5,0xa2,0xe6, +0xd6,0x7b,0x98,0x78,0xa4,0xf7,0x64,0x2d,0x84,0xaa,0x40,0xc3, +0xc2,0x2f,0x4c,0x74,0x48,0xf0,0xa8,0xf2,0x20,0x79,0xf4,0x6b, +0x6e,0xa0,0x14,0x72,0x7d,0x00,0x69,0xba,0x1e,0xa6,0x46,0x82, +0x27,0xac,0x5d,0x1a,0xd6,0x3b,0xc6,0x37,0xae,0x70,0x14,0x66, +0x56,0x17,0xcc,0xfa,0x7d,0xca,0x31,0x11,0x49,0x95,0x20,0x9e, +0x04,0xdd,0x95,0xdd,0xfb,0xb0,0x93,0xbe,0x46,0xc3,0xa2,0x30, +0x24,0xa4,0x28,0xa7,0xf1,0x84,0xac,0x51,0xdd,0xfc,0xb4,0xc0, +0xe7,0xef,0x9e,0x1d,0x88,0xf6,0x55,0x7a,0xbd,0xbe,0x89,0xf5, +0x32,0x8a,0xb4,0x93,0xd8,0x7a,0xad,0x5d,0xa0,0xf5,0xfc,0xe9, +0xe2,0x0e,0x14,0x30,0x5e,0xa4,0x1e,0x11,0x2d,0x72,0xc5,0x05, +0x63,0xce,0x21,0xb1,0x2b,0xef,0x09,0x9d,0x80,0x16,0xb8,0x6b, +0x50,0xf4,0xf9,0x39,0x74,0x88,0x4a,0x94,0x84,0xa2,0x0f,0xc0, +0x03,0xef,0x15,0xcb,0x23,0xe2,0x87,0x70,0x23,0xec,0x50,0x19, +0xf0,0xa9,0xfa,0xcb,0xe2,0x0e,0x95,0xee,0x7a,0xc1,0x1b,0x1a, +0x75,0x7a,0xab,0x67,0x0f,0xd3,0x3d,0xc3,0xac,0xc6,0x70,0xa7, +0x6e,0xf9,0xbc,0xa7,0xf5,0x91,0x31,0xa5,0xea,0x88,0x2b,0x04, +0x78,0xb5,0x08,0xcc,0x4f,0xd2,0xfb,0xd6,0x6b,0x00,0x66,0xf9, +0x7a,0xae,0x11,0xd4,0x17,0xa4,0x3f,0x3b,0xc6,0x4a,0x94,0x5c, +0x7d,0x22,0x6b,0x13,0xec,0x5b,0x18,0xb3,0xae,0xe3,0x21,0x0e, +0x5d,0xeb,0xa1,0x39,0x77,0x60,0x09,0x28,0x0e,0x33,0x5c,0x30, +0xa5,0x49,0x53,0x99,0xa3,0xe6,0x72,0xb8,0x61,0x56,0x07,0x83, +0x8e,0x35,0xcf,0x40,0x5c,0x38,0x98,0xf0,0x2c,0x85,0x57,0x97, +0x1e,0x28,0xaa,0x22,0xbf,0x29,0x09,0x4d,0xb7,0x50,0x30,0x4c, +0xba,0x3c,0xc2,0x8f,0x9b,0x53,0x7f,0x56,0x3d,0xac,0x12,0xc3, +0x8a,0x4a,0xa9,0xa1,0xd2,0x82,0x9c,0xfb,0x81,0x27,0x16,0x50, +0x88,0xc3,0x30,0x8f,0xc4,0x65,0xf0,0xc9,0x6a,0xb1,0x7c,0x74, +0xb5,0x2b,0xaa,0xb9,0x25,0x5d,0x30,0x5b,0x0e,0xc7,0xc5,0x8a, +0x96,0x17,0x63,0x18,0x89,0x21,0x6a,0xac,0xee,0x2f,0xa7,0x30, +0x75,0x30,0xdf,0x98,0x4e,0xe6,0x60,0x2e,0x81,0xd9,0x01,0x05, +0x2f,0xee,0xf0,0x9d,0x0c,0x08,0x6a,0xa2,0x38,0x1a,0x4d,0x47, +0xb1,0xce,0x41,0xd0,0xaa,0x0b,0x78,0x3d,0xde,0x71,0x15,0x86, +0x27,0xe7,0xf6,0x09,0x48,0x4e,0xe5,0x39,0x47,0xd1,0x6c,0xd0, +0x9d,0x88,0x23,0xee,0xef,0x39,0x46,0x18,0x33,0xd2,0x84,0x10, +0xe5,0xf7,0x50,0x9f,0xda,0x65,0x30,0x40,0x47,0x54,0x47,0x0c, +0x16,0x85,0x5a,0x74,0xf8,0xb7,0xa0,0x8d,0x45,0x22,0xaa,0xff, +0x12,0x33,0x98,0x69,0x02,0x8d,0x1e,0x02,0x45,0x21,0x56,0xaa, +0xdf,0x1f,0xf0,0xb4,0x33,0x81,0x69,0xee,0x66,0x47,0x7b,0x4c, +0x92,0xcb,0x4b,0x98,0xb6,0xcd,0xc1,0xc2,0x12,0x08,0xaa,0x26, +0xe2,0x21,0xcb,0xcd,0x66,0x9e,0xfc,0x1a,0xf1,0xcf,0xa6,0xd4, +0x29,0x93,0xf4,0x0d,0xe9,0xec,0x0a,0xdd,0x6c,0xd9,0x5f,0x12, +0x0e,0x2f,0xa7,0x12,0x10,0x4a,0xe5,0xea,0x51,0x5a,0x63,0x13, +0x1f,0xd1,0x41,0xe8,0x75,0xcd,0x15,0x7d,0x43,0xc3,0xa3,0xea, +0x12,0x44,0x94,0x96,0x6c,0x38,0x5e,0xaf,0x85,0xfa,0x29,0x3e, +0x62,0x5e,0x30,0x9d,0x05,0xce,0x9a,0xe1,0x5d,0x91,0xdc,0xa3, +0x88,0xc3,0x43,0x44,0xd1,0xc7,0x3f,0xba,0x0f,0x46,0xc2,0xa4, +0xe8,0xe8,0x15,0x1a,0x17,0x21,0x84,0x28,0xe0,0x9d,0x4e,0x94, +0xc1,0x68,0x6d,0xdf,0x5a,0xe9,0xe4,0x9e,0xcd,0x4b,0xdd,0xe6, +0xc6,0x97,0xe0,0xbb,0xfd,0x7e,0xab,0x6a,0x3e,0x6c,0x0a,0x57, +0x89,0x8a,0xd1,0x62,0x07,0xa2,0xab,0x93,0x0f,0x70,0xca,0xcf, +0xd8,0x96,0xa4,0x40,0x58,0x48,0xeb,0xea,0xd5,0x97,0xd0,0x30, +0x5a,0x45,0xb7,0x15,0x05,0x72,0x76,0xcd,0x2d,0xe0,0xfa,0xcc, +0x20,0x8a,0x3b,0x23,0x22,0xd6,0x20,0x8b,0xf5,0x75,0xbc,0x32, +0x0d,0x72,0xb1,0xae,0x44,0xf1,0x3e,0x21,0x38,0x8f,0x82,0xad, +0x58,0x7d,0xd1,0x52,0x59,0x4f,0xb3,0xd5,0xb7,0xb5,0x17,0x40, +0xad,0x00,0x97,0x2a,0xc3,0x92,0x51,0x3d,0x65,0x38,0x30,0x9a, +0x91,0x2a,0x22,0x3b,0x28,0x52,0xf7,0x5b,0xb2,0xac,0x7a,0x4c, +0x57,0x0c,0x77,0xb4,0xe2,0xe1,0x4a,0x3b,0x5f,0x93,0xe0,0x70, +0xba,0xe3,0x09,0x3f,0x10,0x37,0xf0,0x54,0x23,0xd3,0x59,0x0e, +0xd6,0x38,0x62,0xbf,0x06,0x77,0x93,0xed,0xad,0x63,0x01,0xda, +0x4f,0x06,0x51,0x05,0x72,0xa7,0x91,0xa8,0x1a,0x1a,0x92,0xb3, +0x18,0xd9,0x4b,0xe5,0x38,0x0f,0xe0,0xba,0xa4,0x6a,0xb3,0x3e, +0x8a,0x68,0x76,0x20,0x26,0x1b,0x5b,0x65,0x0d,0x19,0xb6,0xb3, +0xde,0xd9,0x46,0xd0,0xca,0x9d,0x6d,0x46,0x2a,0xed,0x3a,0x04, +0xc3,0xad,0x6c,0xee,0x14,0x48,0x74,0xc7,0x4e,0x81,0xb1,0x78, +0x10,0xc9,0xef,0x70,0x60,0x93,0xbe,0xf5,0x95,0x80,0x94,0xb4, +0x0c,0x48,0x8a,0xd2,0x09,0x46,0x15,0x58,0x3a,0xe4,0x2c,0xa3, +0x5b,0x70,0x04,0xb0,0x8c,0x35,0xdc,0x83,0x37,0x11,0x38,0x7c, +0x3e,0x49,0x3b,0x56,0xa8,0x54,0xb9,0x91,0x13,0x59,0xf6,0xa8, +0x41,0x89,0x27,0x6c,0xb8,0x04,0xfb,0x3f,0x46,0xf4,0x92,0x0e, +0x94,0x0e,0x25,0x9f,0x2b,0x9d,0x61,0x4c,0x5b,0x99,0x80,0x86, +0xc2,0xdf,0x91,0x5f,0x5b,0xe5,0x2a,0xa1,0x16,0x67,0x99,0xe4, +0x9a,0xe5,0x59,0x4f,0xcd,0xaf,0x0d,0x20,0xd7,0xe6,0x0d,0xb5, +0x63,0xc9,0xfb,0xa0,0x40,0xaf,0xaa,0x62,0x57,0x4d,0xd0,0x84, +0xad,0xf2,0xd5,0x0e,0xfe,0x5f,0xae,0x16,0xb3,0xdd,0xbc,0x07, +0x73,0x9f,0x6d,0x77,0xf6,0x06,0xc1,0x0c,0x5d,0x55,0x47,0xc3, +0x7d,0x47,0xd6,0xd3,0x2c,0x51,0xd6,0xd5,0x00,0x93,0x64,0x4a, +0xb4,0xbd,0x52,0x2e,0x57,0x13,0xd6,0xa9,0x4e,0xd4,0x66,0x7e, +0x12,0x29,0x3c,0x98,0xa1,0x63,0x4b,0x47,0xb3,0xd9,0xdd,0x9b, +0x51,0xfc,0x10,0x40,0xbd,0x3f,0x6e,0x69,0xfd,0x23,0xc2,0xbb, +0x04,0x11,0x20,0x22,0xd8,0xfb,0xfb,0xc0,0xa7,0x8f,0x28,0xf0, +0x28,0x8b,0xde,0xa7,0xee,0xb7,0x07,0xf7,0xf7,0x61,0xb7,0xa0, +0x5c,0x94,0x30,0xd1,0xc7,0x38,0x4e,0xa6,0x2e,0xa1,0x6b,0xf2, +0xf3,0x4b,0xc7,0x54,0x2c,0xbe,0x4e,0x98,0x4c,0x70,0xa9,0x88, +0xe9,0x2b,0x29,0x17,0x25,0x15,0x93,0xac,0xeb,0xf1,0x13,0x05, +0xf0,0xd9,0x68,0x0c,0x22,0x59,0x03,0xf8,0xdb,0x88,0x10,0x99, +0xea,0x98,0x08,0xc5,0x46,0xda,0xd8,0x5a,0x69,0xf1,0x3e,0xa0, +0xad,0x82,0x3b,0xdc,0x3a,0x0e,0x3f,0xec,0x79,0x38,0xae,0xac, +0xfb,0x23,0x7c,0x72,0x46,0x8b,0x01,0x7a,0x5a,0x1d,0x1e,0x22, +0x2c,0x67,0x97,0xed,0x8e,0xf1,0x92,0xdd,0x3a,0x46,0x28,0x7a, +0x42,0x5d,0xab,0xe2,0x71,0x45,0xd1,0x32,0x36,0x10,0xdc,0xab, +0xda,0x22,0xee,0xea,0x15,0xaa,0xfa,0xcd,0x07,0x22,0xd8,0xac, +0x2a,0xe2,0x77,0x8b,0x1e,0x5e,0x04,0xa5,0x16,0x2c,0xa9,0x3e, +0x4c,0xab,0x9e,0xe7,0x5d,0xa5,0x52,0xe9,0x57,0x15,0xe5,0x01, +0xe5,0xc8,0x54,0xe3,0x32,0x3f,0xc2,0xa2,0x82,0xc4,0xf6,0x34, +0x9e,0x5e,0x52,0x53,0x22,0x68,0x11,0x9e,0x9e,0x18,0x34,0x94, +0xb6,0x41,0x05,0xe6,0xdb,0x8b,0x04,0x51,0x01,0x76,0x89,0x13, +0x93,0x75,0x74,0x3f,0xe2,0x2f,0xb9,0x7f,0x86,0xe8,0xb6,0xb4, +0x43,0x02,0x32,0x62,0x4d,0x98,0x88,0xd7,0x0f,0xec,0xe8,0x68, +0x9b,0xa2,0x6c,0xcd,0x18,0x34,0xcb,0x0b,0xae,0x1f,0xa2,0x99, +0x08,0x41,0xe1,0xf9,0xb4,0xb0,0xf3,0xca,0x74,0x60,0x4c,0x3d, +0x21,0xd0,0x98,0xb8,0x36,0x45,0x40,0xd2,0x8b,0x4c,0x4b,0x0a, +0x3b,0x12,0x38,0x02,0xf9,0x93,0xd9,0x26,0x5d,0x3f,0x19,0xc5, +0x4f,0x96,0x74,0x4f,0xbf,0xa2,0x1e,0x04,0x74,0x16,0xa7,0x8e, +0xbd,0x50,0x44,0x18,0xb5,0x61,0x71,0xe9,0x7d,0xc0,0x15,0xc5, +0xcb,0x81,0xf4,0x04,0xe1,0x10,0xad,0x28,0xa0,0xc5,0x3f,0x7a, +0x4f,0x0b,0xf2,0xf1,0x6c,0x61,0xcd,0xac,0x49,0xd2,0x41,0x51, +0x91,0xdd,0x44,0xab,0x0a,0xe8,0x0f,0xf1,0x41,0x2d,0x75,0x73, +0x12,0xd8,0x95,0xfc,0x61,0x6e,0x85,0x16,0xfd,0x66,0x71,0x6d, +0x8a,0xa1,0x0a,0xfc,0x15,0x8c,0xd4,0xa6,0x11,0x2e,0xd3,0x90, +0x53,0x06,0x2d,0xd2,0xe0,0x17,0x2f,0x21,0xa9,0xaf,0xa8,0xa2, +0x95,0xd4,0x85,0x6d,0x68,0xb9,0xe1,0x5b,0xfa,0x85,0x4a,0xdc, +0x0d,0x93,0xf5,0xc3,0x5b,0x7d,0x92,0x08,0x71,0xdc,0x0e,0x66, +0x2d,0xee,0x4e,0x19,0xa1,0x1b,0xb2,0x79,0xc7,0x8e,0x4b,0xe0, +0xe2,0xc1,0x6e,0x2c,0x8a,0xa4,0x9e,0xec,0x20,0x71,0x22,0xfb, +0x85,0xdf,0x27,0xe2,0xc2,0xc1,0x2e,0x19,0x43,0xad,0x42,0x88, +0x51,0x30,0xd6,0xeb,0x69,0xb1,0x7d,0xb0,0x60,0x7f,0xb1,0xc4, +0x88,0x54,0xee,0x2c,0xe8,0x4e,0x2f,0xcc,0x3b,0x7f,0xcd,0xf5, +0xe1,0x4b,0xbd,0xa5,0xeb,0x32,0xd8,0x77,0x22,0x23,0xb4,0xa1, +0xf8,0x82,0xe6,0x15,0x4e,0x3d,0xc3,0x75,0x6c,0x29,0x59,0x95, +0x4b,0x40,0x82,0xc0,0x0e,0xe9,0x29,0xc4,0x08,0x26,0x46,0xbd, +0xa0,0x23,0x0f,0x03,0xdb,0x65,0x95,0xb5,0x81,0x49,0x9e,0xb7, +0xad,0xe8,0x8c,0xf4,0x7a,0xe5,0xe2,0x4e,0x9d,0x2e,0xee,0xca, +0x92,0xef,0xef,0x0b,0x45,0x1f,0x60,0xd1,0x40,0xbb,0x22,0x0d, +0xb2,0xc9,0x5d,0x78,0xbc,0x8c,0x16,0xa9,0x43,0xfe,0x34,0x5d, +0x48,0x68,0x4a,0x28,0xd5,0xce,0x72,0xee,0xa7,0xfa,0xba,0xf1, +0x38,0x68,0x70,0x1e,0x29,0x1c,0x3d,0x97,0x2e,0xcd,0x95,0x32, +0x21,0xea,0xe7,0xd5,0x30,0x15,0x8a,0x7a,0xa8,0xd6,0xf8,0x6a, +0x24,0x65,0xd5,0x35,0x5a,0xaf,0x86,0xc9,0x54,0x72,0x13,0xaa, +0x24,0x71,0x1b,0xb2,0x48,0x2d,0xe2,0x2b,0xaa,0x3c,0xbd,0x52, +0x4b,0x62,0x87,0x83,0xc0,0xb8,0x52,0x3a,0x92,0xd9,0x2c,0x9e, +0x24,0xd6,0x89,0x56,0xc0,0xc7,0xf2,0x21,0xe9,0xc2,0x48,0x18, +0x4f,0x66,0x4b,0xdf,0x76,0x36,0x32,0xda,0x65,0x17,0xbb,0x54, +0x76,0xdd,0x6a,0x95,0x42,0xc4,0x91,0x30,0x77,0x92,0x60,0x4d, +0x95,0xb3,0x8b,0xb4,0x9e,0x63,0x62,0xaf,0xb6,0xf2,0xd4,0x98, +0x08,0xe2,0x75,0x5c,0x8b,0xfb,0xaa,0x8a,0xda,0xdb,0xa1,0x96, +0xf9,0xe8,0x3a,0x03,0x5d,0xec,0x5b,0x10,0x31,0x95,0xed,0xf0, +0xf0,0x40,0xe0,0x93,0x15,0x57,0x2d,0x54,0xdf,0x14,0xd4,0xe7, +0x3d,0x89,0x4c,0x6f,0xbc,0xd5,0x96,0x6c,0x36,0xd8,0xa2,0xd4, +0x58,0x5a,0xc7,0xa2,0xd5,0x64,0x78,0x90,0x94,0x95,0x10,0xa4, +0x93,0x6a,0xec,0xb8,0x04,0x19,0x90,0x1b,0xdb,0x0e,0xc5,0x48, +0xea,0x98,0x39,0x45,0x0e,0x7d,0x44,0x01,0xba,0x83,0x17,0xc5, +0x85,0x3c,0xc4,0x2b,0x36,0x37,0x0c,0xe5,0x49,0x4a,0x4b,0x4d, +0x97,0x8c,0x95,0x27,0x92,0x27,0xa3,0x4c,0x81,0xac,0x38,0xa0, +0x72,0xe1,0xb8,0x4c,0xb7,0x48,0xe3,0xa2,0x6c,0xf0,0x78,0xa9, +0x90,0x54,0x70,0x40,0x9a,0xa3,0x05,0x01,0x4c,0xac,0x51,0x14, +0x67,0x40,0x30,0x99,0x47,0x53,0xda,0xd3,0x44,0xe3,0x24,0xb0, +0x44,0xcd,0xc0,0xb4,0xc3,0x60,0x4e,0x48,0xd4,0x0d,0x2f,0x2d, +0xb9,0x25,0x19,0xf5,0x2d,0x55,0x5e,0x6a,0x93,0x26,0x4b,0xf2, +0xab,0xb0,0x57,0xdf,0x0c,0xd2,0xa1,0x06,0x8b,0x20,0x4f,0x48, +0x7d,0x0e,0xe0,0xfe,0xde,0x36,0x1e,0xd8,0xd1,0x1c,0x14,0x9c, +0x30,0x86,0x68,0x47,0xba,0xca,0xd1,0xd0,0xf9,0x6a,0x0e,0x33, +0x5c,0x40,0xcb,0x13,0x6d,0xf1,0xe8,0x27,0x61,0xef,0x0f,0xba, +0x05,0xff,0x0b,0xc4,0xc2,0x43,0xac,0x74,0x27,0x39,0x3c,0x24, +0xd7,0x4b,0x5a,0x00,0xc2,0xb0,0xe4,0x06,0xc6,0x22,0x99,0xb7, +0x85,0x57,0x90,0x89,0xb8,0x5a,0x07,0xf4,0xb2,0x37,0xa7,0x98, +0xea,0xef,0xb3,0xbc,0xa0,0x26,0xea,0x91,0xe9,0x8a,0xde,0x35, +0xdd,0x59,0x52,0x36,0xfb,0x92,0xd5,0x1f,0xa1,0x88,0xf3,0xef, +0x41,0x10,0x01,0x22,0x3d,0x7d,0xc3,0x1d,0x5e,0x11,0xa6,0xbb, +0x00,0x82,0x09,0x37,0xa3,0x78,0xb5,0xae,0xa1,0x4c,0xaf,0x65, +0x7c,0x94,0xdb,0xbe,0x58,0xbb,0xb0,0x08,0x88,0xc4,0x57,0x79, +0xe9,0x42,0x92,0x80,0xe5,0xcc,0xc4,0x29,0xca,0xb2,0x71,0x4a, +0x96,0x63,0xad,0x94,0x18,0x73,0x18,0x51,0x2a,0xde,0xe6,0x4f, +0x79,0xca,0xe2,0xc2,0xd1,0xb4,0xb5,0xcb,0xfe,0x88,0x78,0x59, +0xfb,0xa4,0x7a,0x93,0x86,0x2c,0xd4,0x1a,0x53,0x47,0xdb,0x8d, +0xe9,0x62,0xb3,0x1a,0xc7,0xd4,0x11,0xc4,0xa7,0xa4,0x12,0xe6, +0x1c,0x4b,0x24,0x53,0x44,0x23,0x71,0x18,0x05,0x53,0x47,0x8a, +0xe0,0x80,0x36,0x6b,0x97,0x49,0xc1,0x8d,0x4a,0x5a,0x49,0xcb, +0x5c,0xb5,0xc6,0xc7,0xae,0xe4,0x51,0x55,0xe8,0xe8,0x8e,0x2a, +0x80,0x76,0x1b,0x35,0x23,0x69,0x89,0x79,0x99,0xb3,0xe5,0xaa, +0xf8,0x08,0xd4,0x61,0x2c,0x8d,0x03,0xb3,0xa4,0x28,0x32,0x7f, +0x9c,0xbc,0xca,0x06,0x55,0xcb,0x89,0x44,0x30,0xd2,0x89,0xcf, +0x37,0x96,0x89,0xd3,0xe6,0x11,0x53,0x38,0xe1,0xc7,0x27,0x8b, +0x12,0xf4,0x7f,0xa6,0x35,0xd5,0x86,0xbd,0x6a,0x56,0x76,0x8c, +0xaf,0x68,0x62,0x63,0xea,0x32,0x9e,0xe2,0x0d,0x1d,0x95,0xf4, +0xa1,0x20,0x1b,0x3a,0xb4,0xb8,0x3d,0x49,0xae,0x96,0xfc,0x0e, +0x55,0x47,0x8a,0xad,0x0b,0x7c,0x97,0x7b,0x95,0x64,0x79,0x9b, +0xe3,0x24,0xed,0xfd,0xe2,0x92,0xf6,0x38,0x5e,0xae,0x0d,0x3f, +0x00,0x23,0x8e,0x7b,0x35,0x66,0xe6,0x63,0x4c,0xbf,0x1a,0x4b, +0x58,0xfe,0xc8,0x1f,0xbe,0x59,0x0f,0xc7,0x37,0xfe,0x88,0x7e, +0xd8,0x8d,0xc1,0x70,0x50,0x78,0x12,0x80,0xd2,0x1f,0xde,0xdf, +0x87,0xc7,0x10,0x1b,0x1e,0x8f,0xf4,0x21,0x01,0x88,0xe4,0x0f, +0x05,0x4a,0xf9,0xba,0x43,0xeb,0xb0,0x17,0xb0,0x6b,0x18,0x0d, +0xd5,0x61,0x2f,0xcf,0x1f,0x15,0x93,0x47,0xd1,0x48,0x27,0xf3, +0x7a,0x0a,0xbe,0xfd,0xf6,0x2d,0x74,0xe1,0x48,0x9e,0x03,0x14, +0x6d,0x49,0x53,0x54,0x95,0x7a,0x50,0x48,0x1e,0x79,0xda,0x5f, +0x2f,0xbd,0x4b,0xf0,0x81,0x2b,0x05,0x0b,0xe6,0xe2,0x30,0x8d, +0x9d,0x3e,0x8f,0x4a,0x4f,0x38,0xdb,0xc0,0xe9,0x68,0xca,0x39, +0x0a,0x90,0x74,0x2d,0x48,0xbe,0x54,0xca,0x80,0x3c,0x88,0x1a, +0xc3,0xde,0x08,0xfe,0x8e,0x3a,0x20,0x17,0xc3,0x9e,0x64,0x4c, +0x07,0x7e,0x1a,0x23,0x3b,0x33,0x5a,0x94,0x90,0xd5,0x8a,0xfb, +0x6c,0xb1,0x00,0xeb,0x7d,0xae,0x30,0x36,0x2a,0xf2,0xfd,0x10, +0x5f,0xfd,0x25,0x5b,0x1a,0x85,0xb6,0x58,0xd0,0x80,0xbd,0xe2, +0xeb,0xf0,0x70,0xd8,0xba,0x9a,0x2e,0x46,0x43,0xe4,0xbf,0xf8, +0xc2,0xb8,0x19,0xf4,0x0c,0x34,0xdc,0x09,0x54,0x05,0x30,0x25, +0xb9,0x9a,0x2f,0x56,0x31,0xde,0x83,0x8b,0x49,0x3a,0x84,0x2b, +0xad,0x8e,0x18,0xac,0xd5,0x82,0x2f,0xb4,0x6d,0x29,0x6e,0xa4, +0xb9,0x5b,0x58,0x83,0x63,0x09,0xb2,0xc7,0x09,0xb9,0x68,0x48, +0x49,0x7d,0x71,0x14,0xc2,0xf0,0x89,0x19,0x15,0x12,0x46,0xec, +0x01,0xfd,0x1a,0x97,0xd2,0x86,0xa8,0x1b,0xd2,0xf5,0x6a,0x83, +0x6b,0x3a,0xfe,0x88,0xe2,0x46,0x66,0x1c,0x79,0xdd,0x60,0x34, +0x7a,0xde,0xbc,0x26,0x67,0xb9,0x03,0xd7,0x1a,0x22,0x28,0x15, +0xc6,0x07,0xfa,0x35,0x25,0x6e,0xf8,0x5a,0xf9,0xd6,0x29,0xe0, +0x91,0x00,0x1e,0x15,0x81,0x39,0x41,0xd7,0x5b,0x54,0x94,0xb7, +0xa7,0x3c,0xd1,0x65,0x38,0x38,0x62,0xd1,0x25,0x5f,0x80,0xc0, +0xd7,0x0e,0xeb,0xc8,0x5d,0xd9,0xac,0xd4,0xcf,0x07,0x4e,0x14, +0x19,0x62,0x89,0x1a,0x08,0xf3,0x0c,0xa5,0x39,0x20,0xb2,0x52, +0x24,0xd4,0x5a,0x58,0x03,0xb4,0x74,0x8d,0x71,0xcd,0xa6,0x36, +0xec,0xe3,0x5f,0xdd,0x61,0x1f,0x23,0x07,0xfe,0x48,0xfc,0xda, +0x9d,0xd9,0xeb,0x7a,0x5d,0x3a,0xa3,0x57,0x72,0x7c,0xc5,0xb6, +0x61,0xcb,0x6b,0x28,0xec,0x2e,0x22,0xa3,0xd1,0xf0,0x0f,0x24, +0x76,0x4e,0x1e,0x89,0x64,0x2a,0x0b,0xf5,0x31,0x14,0x45,0x3f, +0x85,0x92,0x3c,0x8f,0xbc,0xb4,0xbb,0xfc,0x1e,0x20,0x2e,0x94, +0x51,0x99,0xa2,0xbc,0x91,0x2e,0x4f,0x22,0x3c,0xe0,0xda,0x88, +0x6c,0x92,0x61,0x18,0xb9,0x95,0x2e,0x1e,0x92,0xdd,0x8b,0xa5, +0xab,0x99,0x4d,0x01,0x61,0x5a,0x74,0xc9,0x45,0xf2,0xd7,0x8d, +0xe5,0xa5,0xe0,0x8f,0xf4,0x60,0xf0,0x2b,0x69,0xb8,0xfe,0x40, +0xee,0xfe,0x0a,0x87,0xca,0xa8,0x64,0xdc,0x2a,0x4f,0x0d,0xd9, +0xd8,0xd4,0x72,0xb6,0x72,0xe6,0x4d,0x6f,0xa1,0x27,0x4c,0x97, +0x4c,0x31,0x3e,0xc8,0x0e,0x50,0x31,0xda,0xa9,0xda,0x4b,0x3b, +0xb7,0xc2,0xb3,0x98,0xc9,0x9b,0xc6,0x33,0xeb,0x8d,0x28,0x63, +0xec,0x38,0x70,0x0f,0x20,0x04,0xaa,0xf8,0x80,0x8b,0x9b,0x2f, +0x26,0x31,0x1a,0xde,0xa2,0x66,0xec,0x07,0x63,0x1d,0xf3,0x2d, +0x5a,0xf1,0x8c,0xa7,0x2c,0x8f,0xb6,0x1e,0xa5,0x65,0x38,0x5f, +0x0f,0x54,0xb5,0xbe,0x35,0x91,0x35,0x1d,0xd8,0xfa,0x34,0xf6, +0xf6,0x1d,0xe5,0x16,0x80,0x57,0xdd,0x8b,0xac,0x78,0x97,0x3d, +0x15,0x81,0x17,0xd9,0x6f,0xe4,0xad,0xf9,0xa0,0x1e,0xe1,0x87, +0xb5,0x9d,0x33,0xd0,0xc3,0xf3,0x9c,0xba,0xc6,0x45,0xdf,0x49, +0x52,0xa7,0x81,0x81,0x41,0x35,0x0d,0x65,0xba,0xa3,0x48,0xd5, +0x8e,0x73,0x36,0x9c,0x81,0xb3,0x65,0xa7,0x2e,0xc3,0x5f,0x41, +0xcf,0x81,0xc0,0xe6,0xb0,0x92,0x76,0x72,0xfe,0x40,0x37,0xa2, +0x83,0x45,0xc6,0xb1,0x43,0x3b,0xd3,0x8e,0xcc,0xa4,0xb4,0xe5, +0x71,0xeb,0x18,0xba,0x94,0xa9,0x65,0x2a,0x11,0xeb,0x8c,0x91, +0x54,0xb3,0x38,0x4e,0x33,0xb9,0x5f,0xe0,0x26,0x7c,0x8d,0x19, +0x21,0x53,0x85,0xef,0xf1,0x01,0x5f,0x71,0xc7,0x57,0x14,0x4e, +0x17,0x43,0x31,0x41,0xe3,0x56,0x84,0x94,0x7a,0x5b,0x84,0x1b, +0x43,0xbb,0x30,0x1f,0x90,0x6f,0x12,0x67,0x14,0x43,0xd5,0xae, +0xf6,0x3f,0x08,0xee,0xef,0xc5,0x57,0x08,0x63,0x46,0x69,0x90, +0xab,0x97,0xad,0x6f,0x37,0xa5,0x15,0x6a,0xcb,0xe9,0x27,0x52, +0x64,0xfc,0x55,0x9e,0xf4,0xae,0x06,0x37,0x4d,0x34,0xcc,0x00, +0x2d,0x54,0x7f,0x4c,0xc1,0xbe,0x37,0x40,0xd1,0x44,0x30,0x98, +0x79,0xbe,0x78,0xbd,0x98,0x5f,0x82,0x01,0xb6,0xae,0xf0,0x2e, +0x10,0xe7,0xc3,0xcb,0x57,0x2d,0xf8,0x38,0xb1,0xde,0x1a,0xde, +0x3b,0x75,0x7e,0xb7,0xca,0xb3,0x99,0x26,0xe7,0x1a,0x6a,0x5e, +0xe1,0x23,0xa2,0x56,0x16,0xc6,0xe3,0xcd,0x8c,0x0f,0xec,0xbb, +0xb6,0xc7,0x2f,0x4d,0x11,0xe7,0x62,0x46,0x46,0x60,0x38,0x3b, +0xac,0x76,0xad,0xd3,0x9b,0xe1,0x04,0x28,0xf6,0xea,0x26,0x8b, +0x99,0x26,0x42,0x3c,0xe6,0x5d,0x68,0x07,0x88,0x01,0x49,0x36, +0x9f,0xa1,0x0e,0xe8,0x8d,0xea,0x86,0x71,0xd3,0x1b,0x7d,0x0a, +0xd7,0x65,0xef,0x08,0x81,0xf1,0x31,0xeb,0x46,0x28,0x9d,0x92, +0x99,0x29,0x78,0xfb,0xc2,0x87,0x38,0x1d,0x0f,0x97,0x71,0xe7, +0x83,0x73,0xe8,0x74,0x9c,0x43,0x5c,0x67,0x03,0x0d,0xf0,0x0a, +0xbf,0xa7,0x6b,0xfc,0x3c,0xc7,0xcf,0x2b,0xfc,0xfc,0xc4,0xf9, +0x04,0x3e,0x7f,0xdd,0x2c,0x28,0xfe,0x13,0x8c,0xff,0x1f,0x59, +0xfb,0x79,0x17,0xba,0x73,0x57,0x21,0x6c,0x6d,0xe6,0x8c,0x31, +0x92,0xf3,0x5d,0x57,0xa7,0x71,0x8a,0x67,0x90,0x00,0xaa,0x26, +0xce,0x80,0xf1,0x92,0x0c,0x9c,0x75,0xb3,0xfa,0x81,0x51,0xda, +0x69,0x88,0x59,0x70,0x09,0x43,0xeb,0xfd,0x22,0x99,0xbb,0x8e, +0xe3,0xa1,0x36,0xf1,0x9d,0x2b,0xc7,0xf3,0x65,0xb9,0x16,0x0a, +0xb7,0x02,0x85,0x04,0x94,0x48,0xee,0x11,0x8b,0xc7,0x58,0x70, +0x00,0x13,0x4a,0x93,0x81,0x20,0x5a,0xc2,0x9b,0xfa,0x50,0x1e, +0x0e,0xba,0x90,0xa7,0x74,0xca,0x97,0x60,0x5a,0x6d,0xc6,0x91, +0x3d,0xc7,0xe9,0x00,0xd1,0x0d,0x01,0xd2,0x5a,0xc5,0xcb,0xe9, +0x10,0x66,0x16,0x16,0x2b,0x24,0x46,0xa3,0x30,0x7c,0x1a,0x56, +0x0f,0xa1,0xb2,0x22,0x12,0xb0,0x4f,0xe9,0x03,0x7a,0x2b,0x93, +0x3d,0xe7,0xe4,0x1a,0xa3,0xec,0x70,0xa0,0x00,0xfc,0xa5,0xe8, +0x61,0xf6,0xe0,0x0a,0x29,0x9e,0xd5,0x6b,0xbb,0xda,0xc9,0x94, +0x93,0xfb,0x32,0xe3,0xa0,0xfb,0x90,0xab,0xba,0xea,0xbc,0x88, +0xb5,0x63,0x9c,0xe3,0xc8,0x92,0xa2,0xca,0x22,0x1e,0x1b,0xcb, +0x10,0x3c,0x23,0x2a,0x0c,0x37,0x72,0xbd,0x2f,0xba,0xe8,0xf3, +0x90,0x83,0xd3,0x3c,0xfa,0x82,0x56,0x52,0x1b,0x8b,0xfd,0xc2, +0x70,0x64,0xae,0x44,0x5a,0xd7,0x5a,0xec,0xb3,0x24,0x29,0xcf, +0x4a,0x60,0x45,0x68,0x6d,0xce,0x58,0xab,0xbb,0x10,0x8b,0xb4, +0xc0,0x66,0xee,0x44,0xc9,0xe4,0x35,0xfa,0x15,0xc7,0x2b,0xe8, +0xf4,0xd2,0xe5,0xec,0x2b,0x43,0x27,0x82,0x5a,0xba,0x4c,0x84, +0xef,0x75,0x32,0x89,0x1a,0x0d,0x95,0xa1,0xe1,0x38,0x8a,0x18, +0x02,0xea,0xf1,0x0f,0x40,0x74,0x12,0x3a,0xba,0xb0,0x8e,0x67, +0x4b,0x5c,0x44,0x7e,0x13,0xaf,0xd7,0x20,0x2b,0xd8,0x41,0x90, +0xa1,0xf8,0x3e,0xd6,0xf1,0xab,0xa7,0x6e,0xff,0x97,0xf4,0x97, +0x37,0x83,0x46,0xcf,0x7b,0x7a,0x7e,0x7c,0xe5,0x1b,0x0f,0xba, +0x62,0x6a,0x54,0x48,0x16,0xfd,0x02,0x52,0x9a,0x76,0x0a,0x57, +0x63,0xbe,0xf8,0x06,0x05,0x28,0x3a,0x76,0x5b,0xde,0xdf,0x8f, +0x05,0x34,0x5e,0x5e,0x8f,0x3d,0xfc,0x13,0xe8,0x00,0xbf,0xfc, +0x02,0x1f,0xf0,0x07,0xbe,0x56,0xf0,0x85,0x86,0xc2,0x2f,0x73, +0xf8,0x40,0x1b,0xe2,0xff,0x81,0x5f,0x7c,0x0b,0xe7,0x97,0x4d, +0x3b,0x68,0xbf,0x80,0x00,0xff,0x8a,0x88,0x97,0x22,0xe2,0xa5, +0xb3,0x15,0x78,0x57,0xd1,0xf1,0x2f,0xbf,0xdc,0x7f,0x72,0xff, +0xcb,0xea,0xfe,0x97,0xf9,0xfd,0x2f,0xeb,0x7b,0xce,0xc8,0x3f, +0x2f,0x8f,0xaf,0xba,0xba,0xea,0xc6,0xf2,0x2a,0xea,0xcb,0xc9, +0x70,0x3d,0xc4,0x57,0x9b,0x88,0x1d,0xd2,0xc9,0x0a,0x15,0x7e, +0x57,0x46,0x46,0x7a,0x29,0x02,0x67,0xe8,0x32,0xba,0x82,0x9b, +0xac,0x81,0xa8,0xdf,0x00,0x45,0x86,0xc2,0xe8,0xbb,0x32,0x97, +0xd0,0x34,0x30,0xa5,0x66,0xf6,0x78,0x62,0x9e,0xe7,0x6b,0x08, +0x83,0xed,0xbb,0xc0,0x64,0xc3,0x95,0x60,0x06,0xa6,0x02,0xba, +0xff,0x67,0xa1,0xc7,0xd4,0x19,0x1a,0x9e,0x60,0x3a,0x17,0x17, +0xcb,0x46,0xf4,0x89,0xd3,0x45,0x26,0x28,0x8d,0x21,0x48,0x2f, +0xe8,0x08,0xc1,0x63,0x53,0x1e,0x7c,0x59,0xba,0xba,0x10,0x53, +0xee,0x61,0x12,0xfa,0x46,0x44,0x68,0x79,0x81,0x43,0xf8,0xff, +0x33,0x9c,0x56,0x4e,0xdc,0x6e,0x35,0xea,0x08,0xe5,0xa2,0x21, +0x44,0x46,0x29,0x22,0x41,0x07,0x4e,0x61,0x44,0x29,0xce,0x27, +0x8d,0x5f,0xe6,0xae,0x7b,0x71,0xb1,0x8e,0x5c,0x09,0x0f,0x4a, +0x17,0xac,0x43,0x54,0x45,0xbd,0x4f,0x3e,0xc1,0xb5,0x15,0x8a, +0x45,0x18,0xcf,0x03,0xe8,0x4f,0x88,0x17,0xaa,0x22,0xb5,0xc8, +0x0c,0x98,0x02,0x46,0x40,0x24,0xf0,0x48,0x26,0x58,0x48,0xba, +0x20,0xc4,0x0d,0x99,0xd2,0x00,0x91,0x16,0x9c,0x96,0x4d,0xc0, +0x8c,0x68,0x50,0xa5,0xe4,0x5c,0x90,0x02,0x78,0xca,0xc8,0xc2, +0xa2,0x44,0xad,0x05,0x62,0x95,0x0c,0x47,0x53,0xdc,0x65,0x92, +0xed,0x87,0xfe,0xad,0x2e,0x19,0xab,0x1f,0xb6,0xde,0x07,0x2c, +0x53,0x64,0x76,0xb6,0x10,0xf0,0x54,0x3b,0xa3,0x44,0x02,0xc9, +0x3e,0x50,0x11,0x7d,0xf2,0x09,0xfc,0xbc,0x2f,0x5e,0x09,0x45, +0xe2,0x02,0x8a,0x3c,0x99,0x5b,0xb6,0x10,0x91,0x0d,0xe0,0xc5, +0xe5,0xb5,0x4f,0x3e,0x81,0xb9,0x66,0xd7,0x2c,0x50,0x2a,0xf0, +0x8b,0x25,0x46,0x77,0xd7,0xab,0xfc,0x03,0x77,0x21,0xea,0x02, +0x4a,0xa7,0x57,0xd4,0x06,0xd7,0x1e,0x40,0x3e,0x2f,0x1c,0x41, +0xae,0xb7,0x1d,0xd3,0x7b,0xe5,0xa0,0xa5,0x79,0xcf,0x21,0x96, +0x0b,0x22,0x42,0xfc,0x69,0x15,0x03,0xbb,0xac,0xa7,0xd6,0x9e, +0xb1,0x20,0x8a,0xf2,0x2f,0xb8,0xff,0x95,0x3b,0x3a,0x65,0xf8, +0x60,0xe5,0x30,0x74,0xb1,0xc8,0xab,0xb6,0x84,0x65,0x7e,0x59, +0xb4,0xb2,0xdd,0x41,0x2a,0x6a,0x2b,0x81,0x63,0xbd,0xdd,0x0a, +0x8e,0x2f,0x11,0xd1,0x4a,0xe1,0xf5,0xb0,0x34,0x5a,0x49,0xbe, +0xf1,0x39,0x00,0x82,0x70,0xd5,0x6d,0x2f,0xa5,0xd1,0xb6,0xf0, +0xc4,0x21,0x81,0xf7,0xec,0xbc,0xb4,0x70,0xd8,0x15,0x43,0xa3, +0x8b,0xb7,0x1e,0xb1,0xcd,0x01,0x73,0x69,0xe0,0xf2,0x92,0xdf, +0x7b,0x5f,0xa1,0x93,0x2a,0x3f,0xf3,0x79,0x9d,0x5c,0xa2,0x92, +0xc5,0xe5,0x73,0xfc,0xc1,0xdb,0xd2,0xd8,0x32,0xe1,0x94,0xd2, +0x44,0x8d,0xf7,0x09,0xd1,0x3a,0x30,0x2e,0x9f,0xda,0x77,0xe8, +0xc4,0xd9,0x83,0x35,0x70,0xea,0xeb,0x29,0x10,0xa3,0x74,0x23, +0xc7,0x43,0xb4,0x7a,0xe0,0x14,0x34,0x1e,0x44,0x88,0xf2,0xf0, +0x50,0xd2,0xc8,0xe1,0xfb,0xfb,0xe0,0xc0,0x9c,0x90,0xdf,0xdf, +0x4f,0xe2,0x69,0xbc,0x8e,0xe9,0x18,0x5b,0x30,0x28,0x8f,0xba, +0x34,0x29,0xd9,0x2a,0xb6,0xf0,0xf2,0x36,0x54,0x18,0x3b,0x00, +0x32,0x80,0x70,0xff,0x79,0xb5,0xb6,0xb6,0x47,0x0c,0x32,0xac, +0x0a,0x5b,0x2c,0x31,0xaa,0x2e,0xcc,0x2e,0xb1,0x32,0x6c,0x94, +0xe3,0x7f,0xa0,0xd6,0xae,0x7d,0xfb,0x92,0x25,0xed,0x20,0x10, +0x73,0x95,0xc2,0x69,0xbf,0x02,0xb0,0x28,0x17,0x6d,0x0f,0x4d, +0xa0,0xd7,0xfd,0xdf,0x01,0x00,0x00,0xff,0xff,0x26,0x9f,0xa4, +0xcb,0x31,0xa6,0x08,0x00, })) if err != nil { From 5e604b7a1836de4ee667c97787b44e39234f7ad5 Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Tue, 8 Oct 2013 11:17:12 -0700 Subject: [PATCH 199/247] chore(dashboard): move to the mod directory --- {dashboard => mod/dashboard}/.bowerrc | 0 {dashboard => mod/dashboard}/.editorconfig | 0 {dashboard => mod/dashboard}/.gitattributes | 0 {dashboard => mod/dashboard}/.gitignore | 0 {dashboard => mod/dashboard}/.jshintignore | 0 {dashboard => mod/dashboard}/.jshintrc | 0 {dashboard => mod/dashboard}/.travis.yml | 0 {dashboard => mod/dashboard}/Gruntfile.js | 0 {dashboard => mod/dashboard}/LICENSE | 0 {dashboard => mod/dashboard}/README.md | 0 {dashboard => mod/dashboard}/app/.buildignore | 0 {dashboard => mod/dashboard}/app/browser.html | 0 {dashboard => mod/dashboard}/app/index.html | 0 .../dashboard}/app/scripts/common/services/etcd.js | 0 {dashboard => mod/dashboard}/app/scripts/controllers/browser.js | 0 {dashboard => mod/dashboard}/app/scripts/controllers/stats.js | 0 .../dashboard}/app/scripts/ng-time-relative.min.js | 0 {dashboard => mod/dashboard}/app/scripts/vega.js | 0 {dashboard => mod/dashboard}/app/stats.html | 0 {dashboard => mod/dashboard}/app/styles/bootstrap.css | 0 {dashboard => mod/dashboard}/app/styles/etcd-widgets.css | 0 {dashboard => mod/dashboard}/app/styles/main.css | 0 {dashboard => mod/dashboard}/app/views/browser.html | 0 {dashboard => mod/dashboard}/app/views/stats.html | 0 {dashboard => mod/dashboard}/bower.json | 0 {dashboard => mod/dashboard}/build | 2 +- {dashboard => mod/dashboard}/karma-e2e.conf.js | 0 {dashboard => mod/dashboard}/karma.conf.js | 0 {dashboard => mod/dashboard}/package.json | 0 {dashboard => mod/dashboard}/resources/bindata-file.go | 0 {dashboard => mod/dashboard}/resources/bindata-toc.go | 0 {dashboard => mod/dashboard}/resources/browser.html.go | 0 {dashboard => mod/dashboard}/resources/index.html.go | 0 .../dashboard}/resources/scripts-browser-modules.js.go | 0 .../dashboard}/resources/scripts-browser-scripts.js.go | 0 .../dashboard}/resources/scripts-stats-modules.js.go | 0 .../dashboard}/resources/scripts-stats-scripts.js.go | 0 {dashboard => mod/dashboard}/resources/stats.html.go | 0 {dashboard => mod/dashboard}/resources/styles-main.css.go | 0 {dashboard => mod/dashboard}/resources/views-browser.html.go | 0 {dashboard => mod/dashboard}/resources/views-stats.html.go | 0 {dashboard => mod/dashboard}/test/.jshintrc | 0 {dashboard => mod/dashboard}/test/runner.html | 0 {dashboard => mod/dashboard}/test/spec/controllers/main.js | 0 44 files changed, 1 insertion(+), 1 deletion(-) rename {dashboard => mod/dashboard}/.bowerrc (100%) rename {dashboard => mod/dashboard}/.editorconfig (100%) rename {dashboard => mod/dashboard}/.gitattributes (100%) rename {dashboard => mod/dashboard}/.gitignore (100%) rename {dashboard => mod/dashboard}/.jshintignore (100%) rename {dashboard => mod/dashboard}/.jshintrc (100%) rename {dashboard => mod/dashboard}/.travis.yml (100%) rename {dashboard => mod/dashboard}/Gruntfile.js (100%) rename {dashboard => mod/dashboard}/LICENSE (100%) rename {dashboard => mod/dashboard}/README.md (100%) rename {dashboard => mod/dashboard}/app/.buildignore (100%) rename {dashboard => mod/dashboard}/app/browser.html (100%) rename {dashboard => mod/dashboard}/app/index.html (100%) rename {dashboard => mod/dashboard}/app/scripts/common/services/etcd.js (100%) rename {dashboard => mod/dashboard}/app/scripts/controllers/browser.js (100%) rename {dashboard => mod/dashboard}/app/scripts/controllers/stats.js (100%) rename {dashboard => mod/dashboard}/app/scripts/ng-time-relative.min.js (100%) rename {dashboard => mod/dashboard}/app/scripts/vega.js (100%) rename {dashboard => mod/dashboard}/app/stats.html (100%) rename {dashboard => mod/dashboard}/app/styles/bootstrap.css (100%) rename {dashboard => mod/dashboard}/app/styles/etcd-widgets.css (100%) rename {dashboard => mod/dashboard}/app/styles/main.css (100%) rename {dashboard => mod/dashboard}/app/views/browser.html (100%) rename {dashboard => mod/dashboard}/app/views/stats.html (100%) rename {dashboard => mod/dashboard}/bower.json (100%) rename {dashboard => mod/dashboard}/build (92%) rename {dashboard => mod/dashboard}/karma-e2e.conf.js (100%) rename {dashboard => mod/dashboard}/karma.conf.js (100%) rename {dashboard => mod/dashboard}/package.json (100%) rename {dashboard => mod/dashboard}/resources/bindata-file.go (100%) rename {dashboard => mod/dashboard}/resources/bindata-toc.go (100%) rename {dashboard => mod/dashboard}/resources/browser.html.go (100%) rename {dashboard => mod/dashboard}/resources/index.html.go (100%) rename {dashboard => mod/dashboard}/resources/scripts-browser-modules.js.go (100%) rename {dashboard => mod/dashboard}/resources/scripts-browser-scripts.js.go (100%) rename {dashboard => mod/dashboard}/resources/scripts-stats-modules.js.go (100%) rename {dashboard => mod/dashboard}/resources/scripts-stats-scripts.js.go (100%) rename {dashboard => mod/dashboard}/resources/stats.html.go (100%) rename {dashboard => mod/dashboard}/resources/styles-main.css.go (100%) rename {dashboard => mod/dashboard}/resources/views-browser.html.go (100%) rename {dashboard => mod/dashboard}/resources/views-stats.html.go (100%) rename {dashboard => mod/dashboard}/test/.jshintrc (100%) rename {dashboard => mod/dashboard}/test/runner.html (100%) rename {dashboard => mod/dashboard}/test/spec/controllers/main.js (100%) diff --git a/dashboard/.bowerrc b/mod/dashboard/.bowerrc similarity index 100% rename from dashboard/.bowerrc rename to mod/dashboard/.bowerrc diff --git a/dashboard/.editorconfig b/mod/dashboard/.editorconfig similarity index 100% rename from dashboard/.editorconfig rename to mod/dashboard/.editorconfig diff --git a/dashboard/.gitattributes b/mod/dashboard/.gitattributes similarity index 100% rename from dashboard/.gitattributes rename to mod/dashboard/.gitattributes diff --git a/dashboard/.gitignore b/mod/dashboard/.gitignore similarity index 100% rename from dashboard/.gitignore rename to mod/dashboard/.gitignore diff --git a/dashboard/.jshintignore b/mod/dashboard/.jshintignore similarity index 100% rename from dashboard/.jshintignore rename to mod/dashboard/.jshintignore diff --git a/dashboard/.jshintrc b/mod/dashboard/.jshintrc similarity index 100% rename from dashboard/.jshintrc rename to mod/dashboard/.jshintrc diff --git a/dashboard/.travis.yml b/mod/dashboard/.travis.yml similarity index 100% rename from dashboard/.travis.yml rename to mod/dashboard/.travis.yml diff --git a/dashboard/Gruntfile.js b/mod/dashboard/Gruntfile.js similarity index 100% rename from dashboard/Gruntfile.js rename to mod/dashboard/Gruntfile.js diff --git a/dashboard/LICENSE b/mod/dashboard/LICENSE similarity index 100% rename from dashboard/LICENSE rename to mod/dashboard/LICENSE diff --git a/dashboard/README.md b/mod/dashboard/README.md similarity index 100% rename from dashboard/README.md rename to mod/dashboard/README.md diff --git a/dashboard/app/.buildignore b/mod/dashboard/app/.buildignore similarity index 100% rename from dashboard/app/.buildignore rename to mod/dashboard/app/.buildignore diff --git a/dashboard/app/browser.html b/mod/dashboard/app/browser.html similarity index 100% rename from dashboard/app/browser.html rename to mod/dashboard/app/browser.html diff --git a/dashboard/app/index.html b/mod/dashboard/app/index.html similarity index 100% rename from dashboard/app/index.html rename to mod/dashboard/app/index.html diff --git a/dashboard/app/scripts/common/services/etcd.js b/mod/dashboard/app/scripts/common/services/etcd.js similarity index 100% rename from dashboard/app/scripts/common/services/etcd.js rename to mod/dashboard/app/scripts/common/services/etcd.js diff --git a/dashboard/app/scripts/controllers/browser.js b/mod/dashboard/app/scripts/controllers/browser.js similarity index 100% rename from dashboard/app/scripts/controllers/browser.js rename to mod/dashboard/app/scripts/controllers/browser.js diff --git a/dashboard/app/scripts/controllers/stats.js b/mod/dashboard/app/scripts/controllers/stats.js similarity index 100% rename from dashboard/app/scripts/controllers/stats.js rename to mod/dashboard/app/scripts/controllers/stats.js diff --git a/dashboard/app/scripts/ng-time-relative.min.js b/mod/dashboard/app/scripts/ng-time-relative.min.js similarity index 100% rename from dashboard/app/scripts/ng-time-relative.min.js rename to mod/dashboard/app/scripts/ng-time-relative.min.js diff --git a/dashboard/app/scripts/vega.js b/mod/dashboard/app/scripts/vega.js similarity index 100% rename from dashboard/app/scripts/vega.js rename to mod/dashboard/app/scripts/vega.js diff --git a/dashboard/app/stats.html b/mod/dashboard/app/stats.html similarity index 100% rename from dashboard/app/stats.html rename to mod/dashboard/app/stats.html diff --git a/dashboard/app/styles/bootstrap.css b/mod/dashboard/app/styles/bootstrap.css similarity index 100% rename from dashboard/app/styles/bootstrap.css rename to mod/dashboard/app/styles/bootstrap.css diff --git a/dashboard/app/styles/etcd-widgets.css b/mod/dashboard/app/styles/etcd-widgets.css similarity index 100% rename from dashboard/app/styles/etcd-widgets.css rename to mod/dashboard/app/styles/etcd-widgets.css diff --git a/dashboard/app/styles/main.css b/mod/dashboard/app/styles/main.css similarity index 100% rename from dashboard/app/styles/main.css rename to mod/dashboard/app/styles/main.css diff --git a/dashboard/app/views/browser.html b/mod/dashboard/app/views/browser.html similarity index 100% rename from dashboard/app/views/browser.html rename to mod/dashboard/app/views/browser.html diff --git a/dashboard/app/views/stats.html b/mod/dashboard/app/views/stats.html similarity index 100% rename from dashboard/app/views/stats.html rename to mod/dashboard/app/views/stats.html diff --git a/dashboard/bower.json b/mod/dashboard/bower.json similarity index 100% rename from dashboard/bower.json rename to mod/dashboard/bower.json diff --git a/dashboard/build b/mod/dashboard/build similarity index 92% rename from dashboard/build rename to mod/dashboard/build index 848eaa2505c..37d90464110 100755 --- a/dashboard/build +++ b/mod/dashboard/build @@ -8,7 +8,7 @@ npm install bower install grunt build -export GOPATH="${DIR}/../" +export GOPATH="${DIR}/../../" for i in `find dist -type f`; do file=$(echo $i | sed 's#dist/##g' | sed 's#/#-#g') diff --git a/dashboard/karma-e2e.conf.js b/mod/dashboard/karma-e2e.conf.js similarity index 100% rename from dashboard/karma-e2e.conf.js rename to mod/dashboard/karma-e2e.conf.js diff --git a/dashboard/karma.conf.js b/mod/dashboard/karma.conf.js similarity index 100% rename from dashboard/karma.conf.js rename to mod/dashboard/karma.conf.js diff --git a/dashboard/package.json b/mod/dashboard/package.json similarity index 100% rename from dashboard/package.json rename to mod/dashboard/package.json diff --git a/dashboard/resources/bindata-file.go b/mod/dashboard/resources/bindata-file.go similarity index 100% rename from dashboard/resources/bindata-file.go rename to mod/dashboard/resources/bindata-file.go diff --git a/dashboard/resources/bindata-toc.go b/mod/dashboard/resources/bindata-toc.go similarity index 100% rename from dashboard/resources/bindata-toc.go rename to mod/dashboard/resources/bindata-toc.go diff --git a/dashboard/resources/browser.html.go b/mod/dashboard/resources/browser.html.go similarity index 100% rename from dashboard/resources/browser.html.go rename to mod/dashboard/resources/browser.html.go diff --git a/dashboard/resources/index.html.go b/mod/dashboard/resources/index.html.go similarity index 100% rename from dashboard/resources/index.html.go rename to mod/dashboard/resources/index.html.go diff --git a/dashboard/resources/scripts-browser-modules.js.go b/mod/dashboard/resources/scripts-browser-modules.js.go similarity index 100% rename from dashboard/resources/scripts-browser-modules.js.go rename to mod/dashboard/resources/scripts-browser-modules.js.go diff --git a/dashboard/resources/scripts-browser-scripts.js.go b/mod/dashboard/resources/scripts-browser-scripts.js.go similarity index 100% rename from dashboard/resources/scripts-browser-scripts.js.go rename to mod/dashboard/resources/scripts-browser-scripts.js.go diff --git a/dashboard/resources/scripts-stats-modules.js.go b/mod/dashboard/resources/scripts-stats-modules.js.go similarity index 100% rename from dashboard/resources/scripts-stats-modules.js.go rename to mod/dashboard/resources/scripts-stats-modules.js.go diff --git a/dashboard/resources/scripts-stats-scripts.js.go b/mod/dashboard/resources/scripts-stats-scripts.js.go similarity index 100% rename from dashboard/resources/scripts-stats-scripts.js.go rename to mod/dashboard/resources/scripts-stats-scripts.js.go diff --git a/dashboard/resources/stats.html.go b/mod/dashboard/resources/stats.html.go similarity index 100% rename from dashboard/resources/stats.html.go rename to mod/dashboard/resources/stats.html.go diff --git a/dashboard/resources/styles-main.css.go b/mod/dashboard/resources/styles-main.css.go similarity index 100% rename from dashboard/resources/styles-main.css.go rename to mod/dashboard/resources/styles-main.css.go diff --git a/dashboard/resources/views-browser.html.go b/mod/dashboard/resources/views-browser.html.go similarity index 100% rename from dashboard/resources/views-browser.html.go rename to mod/dashboard/resources/views-browser.html.go diff --git a/dashboard/resources/views-stats.html.go b/mod/dashboard/resources/views-stats.html.go similarity index 100% rename from dashboard/resources/views-stats.html.go rename to mod/dashboard/resources/views-stats.html.go diff --git a/dashboard/test/.jshintrc b/mod/dashboard/test/.jshintrc similarity index 100% rename from dashboard/test/.jshintrc rename to mod/dashboard/test/.jshintrc diff --git a/dashboard/test/runner.html b/mod/dashboard/test/runner.html similarity index 100% rename from dashboard/test/runner.html rename to mod/dashboard/test/runner.html diff --git a/dashboard/test/spec/controllers/main.js b/mod/dashboard/test/spec/controllers/main.js similarity index 100% rename from dashboard/test/spec/controllers/main.js rename to mod/dashboard/test/spec/controllers/main.js From ae05a9f0038e147d97938dae20a7a7bc524a14bb Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Tue, 8 Oct 2013 11:22:48 -0700 Subject: [PATCH 200/247] feat(mods): add README file --- mod/README.md | 10 ++++++++++ 1 file changed, 10 insertions(+) create mode 100644 mod/README.md diff --git a/mod/README.md b/mod/README.md new file mode 100644 index 00000000000..5fd9c1782a6 --- /dev/null +++ b/mod/README.md @@ -0,0 +1,10 @@ +## Etcd modules + +etcd modules (mods) are higher order pieces of functionality that only +speak to the client etcd API and are presented in the `/mod` HTTP path +of the etcd service. + +The basic idea is that etcd can ship things like dashboards, master +election APIs and other helpful services that would normally have to +stand up and talk to an etcd cluster directly in the binary. It is a +convienence and hopefully eases complexity in deployments. From 26c24dd889aa5f90dd3a48ad1bb9493aa67f47ab Mon Sep 17 00:00:00 2001 From: kelseyhightower Date: Tue, 8 Oct 2013 14:41:39 -0700 Subject: [PATCH 201/247] Add confd to list of projects using etcd --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index adc0db9b283..e32b1e78b17 100644 --- a/README.md +++ b/README.md @@ -574,6 +574,7 @@ See [CONTRIBUTING](https://github.com/coreos/etcd/blob/master/CONTRIBUTING.md) f - [garethr/hiera-etcd](https://github.com/garethr/hiera-etcd) - Puppet hiera backend using etcd - [mattn/etcd-vim](https://github.com/mattn/etcd-vim) - SET and GET keys from inside vim - [mattn/etcdenv](https://github.com/mattn/etcdenv) - "env" shebang with etcd integration +- [kelseyhightower/confd](https://github.com/kelseyhightower/confd) - Manage local app config files using templates and data from etcd ## FAQ From 0c676ac93a1b3ca1f9072e782acda7ec7f6e9a80 Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Wed, 9 Oct 2013 13:54:44 -0700 Subject: [PATCH 202/247] feat(mod): introduce the /etcd/mod namespace introduce the `/etcd/mod` namespace and add the dashboard into it. --- mod/README.md | 2 +- etcd_modules.go => mod/dashboard/dashboard.go | 14 ++++++++------ mod/mod.go | 15 +++++++++++++++ 3 files changed, 24 insertions(+), 7 deletions(-) rename etcd_modules.go => mod/dashboard/dashboard.go (70%) create mode 100644 mod/mod.go diff --git a/mod/README.md b/mod/README.md index 5fd9c1782a6..2dde929752b 100644 --- a/mod/README.md +++ b/mod/README.md @@ -1,7 +1,7 @@ ## Etcd modules etcd modules (mods) are higher order pieces of functionality that only -speak to the client etcd API and are presented in the `/mod` HTTP path +speak to the client etcd API and are presented in the `/etcd/mod` HTTP path of the etcd service. The basic idea is that etcd can ship things like dashboards, master diff --git a/etcd_modules.go b/mod/dashboard/dashboard.go similarity index 70% rename from etcd_modules.go rename to mod/dashboard/dashboard.go index a0397bd76b1..ff9ed43c890 100644 --- a/etcd_modules.go +++ b/mod/dashboard/dashboard.go @@ -1,15 +1,16 @@ -package main +package dashboard import ( "bytes" + "fmt" "net/http" "os" "time" - "github.com/coreos/etcd/dashboard/resources" + "github.com/coreos/etcd/mod/dashboard/resources" ) -func DashboardMemoryFileServer(w http.ResponseWriter, req *http.Request) { +func memoryFileServer(w http.ResponseWriter, req *http.Request) { path := req.URL.Path if len(path) == 0 { path = "index.html" @@ -29,8 +30,9 @@ func DashboardMemoryFileServer(w http.ResponseWriter, req *http.Request) { // DashboardHttpHandler either uses the compiled in virtual filesystem for the // dashboard assets or if ETCD_DASHBOARD_DIR is set uses that as the source of // assets. -func DashboardHttpHandler(prefix string) (handler http.Handler) { - handler = http.HandlerFunc(DashboardMemoryFileServer) +func HttpHandler() (handler http.Handler) { + fmt.Println("hello world") + handler = http.HandlerFunc(memoryFileServer) // Serve the dashboard from a filesystem if the magic env variable is enabled dashDir := os.Getenv("ETCD_DASHBOARD_DIR") @@ -38,5 +40,5 @@ func DashboardHttpHandler(prefix string) (handler http.Handler) { handler = http.FileServer(http.Dir(dashDir)) } - return http.StripPrefix(prefix, handler) + return handler } diff --git a/mod/mod.go b/mod/mod.go new file mode 100644 index 00000000000..f764d6c2f23 --- /dev/null +++ b/mod/mod.go @@ -0,0 +1,15 @@ +// mod is the entry point to all of the etcd modules. +package mod + +import ( + "net/http" + "github.com/coreos/etcd/mod/dashboard" +) + +var ServeMux *http.Handler + +func init() { + // TODO: Use a Gorilla mux to handle this in 0.2 and remove the strip + handler := http.StripPrefix("/etcd/mod/dashboard/", dashboard.HttpHandler()) + ServeMux = &handler +} From bb275e3f458d53e6f964477e0a307345a9075e61 Mon Sep 17 00:00:00 2001 From: Antonio Terreno Date: Thu, 10 Oct 2013 13:43:51 +0100 Subject: [PATCH 203/247] Including clojure library --- README.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/README.md b/README.md index e32b1e78b17..247537a298e 100644 --- a/README.md +++ b/README.md @@ -557,6 +557,10 @@ See [CONTRIBUTING](https://github.com/coreos/etcd/blob/master/CONTRIBUTING.md) f - [jdarcy/etcd-api](https://github.com/jdarcy/etcd-api) +**Clojure libraries** + +- [aterreno/etcd-clojure](https://github.com/aterreno/etcd-clojure) + **Chef Integration** - [coderanger/etcd-chef](https://github.com/coderanger/etcd-chef) From 55356465c67b81cf034a4ffc7c5321b9e479d13b Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Thu, 10 Oct 2013 11:32:39 -0700 Subject: [PATCH 204/247] feat(mod/dashboard): add cleaner urls for stats and browser you can hit them at: - http://localhost:4001/etcd/mod/dashboard/stats - http://localhost:4001/etcd/mod/dashboard/browser --- mod/dashboard/dashboard.go | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/mod/dashboard/dashboard.go b/mod/dashboard/dashboard.go index ff9ed43c890..508d64b7ea7 100644 --- a/mod/dashboard/dashboard.go +++ b/mod/dashboard/dashboard.go @@ -5,25 +5,32 @@ import ( "fmt" "net/http" "os" + "path" "time" "github.com/coreos/etcd/mod/dashboard/resources" ) func memoryFileServer(w http.ResponseWriter, req *http.Request) { - path := req.URL.Path - if len(path) == 0 { - path = "index.html" + upath := req.URL.Path + if len(upath) == 0 { + upath = "index.html" } - b, ok := resources.File("/" + path) + // TODO: use the new mux to do this work + dir, file := path.Split(upath) + if file == "browser" || file == "stats" { + file = file + ".html" + } + upath = path.Join(dir, file) + b, ok := resources.File("/" + upath) if ok == false { - http.Error(w, path+": File not found", http.StatusNotFound) + http.Error(w, upath+": File not found", http.StatusNotFound) return } - http.ServeContent(w, req, path, time.Time{}, bytes.NewReader(b)) + http.ServeContent(w, req, upath, time.Time{}, bytes.NewReader(b)) return } From 26806a10fc8d91a7eda1ef965a9ed6cea61a5b1c Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Thu, 10 Oct 2013 12:19:01 -0700 Subject: [PATCH 205/247] fix(mod/dashboard): use deleteKey() not delete_key() small bug I overlooked while refactoring all of the dashboard function names to be more "javascripty" --- mod/dashboard/app/scripts/controllers/browser.js | 2 +- mod/dashboard/app/views/browser.html | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/mod/dashboard/app/scripts/controllers/browser.js b/mod/dashboard/app/scripts/controllers/browser.js index ecb42022223..5032ecc7ec6 100644 --- a/mod/dashboard/app/scripts/controllers/browser.js +++ b/mod/dashboard/app/scripts/controllers/browser.js @@ -113,7 +113,7 @@ angular.module('etcdBrowser', ['ngRoute', 'etcd', 'timeRelative']) }).error(function (data, status, headers, config) { //TODO: remove loader //show errors - $scope.showBrowseError('Error: Could not delete the key'); + $scope.showBrowseError('Could not delete the key'); }); }; diff --git a/mod/dashboard/app/views/browser.html b/mod/dashboard/app/views/browser.html index 8a2504cf890..beb081f4fb6 100644 --- a/mod/dashboard/app/views/browser.html +++ b/mod/dashboard/app/views/browser.html @@ -64,7 +64,7 @@
- From b819bb1c8ca9ff3fca8536d211057bb38fbf7290 Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Thu, 10 Oct 2013 12:34:08 -0700 Subject: [PATCH 206/247] chore(mod/dashboard): remove debug line Oops, this debug line made it into the tree. Remove it. --- mod/dashboard/dashboard.go | 1 - 1 file changed, 1 deletion(-) diff --git a/mod/dashboard/dashboard.go b/mod/dashboard/dashboard.go index 508d64b7ea7..8a7a36edd58 100644 --- a/mod/dashboard/dashboard.go +++ b/mod/dashboard/dashboard.go @@ -38,7 +38,6 @@ func memoryFileServer(w http.ResponseWriter, req *http.Request) { // dashboard assets or if ETCD_DASHBOARD_DIR is set uses that as the source of // assets. func HttpHandler() (handler http.Handler) { - fmt.Println("hello world") handler = http.HandlerFunc(memoryFileServer) // Serve the dashboard from a filesystem if the magic env variable is enabled From fa1b00cf917d2b1c359bc64b7e3cbc5f0fa5b5a6 Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Thu, 10 Oct 2013 12:41:20 -0700 Subject: [PATCH 207/247] chore(mod/dashboard): remove fmt package Always build before pushing to origin master! Sorry for this noise. I deleted the debug message but not fmt. --- mod/dashboard/dashboard.go | 1 - 1 file changed, 1 deletion(-) diff --git a/mod/dashboard/dashboard.go b/mod/dashboard/dashboard.go index 8a7a36edd58..7941fc7a3e4 100644 --- a/mod/dashboard/dashboard.go +++ b/mod/dashboard/dashboard.go @@ -2,7 +2,6 @@ package dashboard import ( "bytes" - "fmt" "net/http" "os" "path" From 677590feb7ba0d28f9c170995f2dfb8f43ea9cb5 Mon Sep 17 00:00:00 2001 From: Rob Szumski Date: Thu, 10 Oct 2013 13:32:37 -0700 Subject: [PATCH 208/247] docs(dashboard): add instructions to get started --- mod/dashboard/README.md | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/mod/dashboard/README.md b/mod/dashboard/README.md index 45f27af950e..f360837861d 100644 --- a/mod/dashboard/README.md +++ b/mod/dashboard/README.md @@ -6,8 +6,18 @@ http://yeoman.io/ -### Hacking +### Install NPM locally ``` -grunt server +npm install ``` + +### Install Bower + +``` +bower install +``` + +### View in Browser + +http://localhost:4001/etcd/mod/dashboard/ From 4005312ee51ed25774718d2b48b1237441907e5e Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Thu, 10 Oct 2013 14:38:35 -0700 Subject: [PATCH 209/247] feat(scripts): add build-release script --- scripts/build-release | 39 +++++++++++++++++++++++++++++++++++++++ 1 file changed, 39 insertions(+) create mode 100755 scripts/build-release diff --git a/scripts/build-release b/scripts/build-release new file mode 100755 index 00000000000..c376ae57582 --- /dev/null +++ b/scripts/build-release @@ -0,0 +1,39 @@ +#!/bin/sh -e + +VER=$1 + +function build { + proj=${1} + ver=${2} + + if [ ! -d ${proj} ]; then + git clone https://github.com/coreos/${proj} + fi + + cd ${proj} + git checkout master + git fetch --all + git reset --hard origin/master + git checkout $ver + ./build + cd - +} + +function package { + proj=${1} + target=${2} + + cp ${proj}/${proj} ${target} + cp ${proj}/README.md ${target}/README-${proj}.md +} + +build etcd ${VER} +build etcdctl ${VER} + +TARGET="etcd-$VER-$(uname -s)-$(uname -m)" +mkdir ${TARGET} + +package etcd ${TARGET} +package etcdctl ${TARGET} + +tar cvvfz ${TARGET}.tar.gz ${TARGET} From 3f5c39afd9e4a7180d3236fd72a5643ee190b04f Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Thu, 10 Oct 2013 15:28:19 -0700 Subject: [PATCH 210/247] fix(mod/dashboard): bump the resources bump all of the resources to get the new delete_key fix. --- .../resources/scripts-browser-modules.js.go | 4754 +++++++++-------- .../resources/scripts-browser-scripts.js.go | 74 +- .../resources/scripts-stats-modules.js.go | 4549 +++++++++------- mod/dashboard/resources/views-browser.html.go | 282 +- 4 files changed, 5222 insertions(+), 4437 deletions(-) diff --git a/mod/dashboard/resources/scripts-browser-modules.js.go b/mod/dashboard/resources/scripts-browser-modules.js.go index 5b4a8a6a7ae..d48ef578ca1 100644 --- a/mod/dashboard/resources/scripts-browser-modules.js.go +++ b/mod/dashboard/resources/scripts-browser-modules.js.go @@ -6919,2189 +6919,2577 @@ func scripts_browser_modules_js() []byte { 0x85,0xbb,0x44,0x17,0x93,0x2c,0xed,0x95,0x3a,0x4e,0xfa,0x37, 0xec,0x48,0x38,0x20,0x13,0xbc,0xb6,0xc2,0x23,0xfa,0x99,0x2c, 0x70,0xa4,0x9a,0x47,0xe7,0xf8,0xa4,0x57,0x73,0xfc,0xff,0x50, -0xf7,0xf6,0x7d,0x6d,0x1c,0xc9,0xc2,0xe8,0xdf,0xf7,0xf9,0x14, -0x30,0xc7,0xc1,0x1a,0xab,0x11,0x12,0x76,0x9c,0xcd,0xc8,0x83, -0x8e,0x63,0xec,0xc4,0x7b,0x8d,0x9d,0xc7,0x90,0xf5,0xd9,0x08, -0x85,0x9f,0x40,0x02,0x26,0x06,0x89,0x68,0x44,0x1c,0x07,0xe9, -0x7e,0xf6,0x5b,0x6f,0xdd,0x5d,0xdd,0x33,0x02,0x9c,0xdd,0x73, -0x9e,0x7b,0x37,0x6b,0x34,0xd3,0x5d,0x5d,0x5d,0xd3,0x2f,0xd5, -0xd5,0xd5,0xd5,0x55,0x27,0xc8,0xa2,0xf0,0xdd,0x5e,0x94,0x32, -0x6a,0x68,0x55,0x56,0x33,0xc4,0x66,0x42,0x5c,0x26,0xc0,0xe4, -0xee,0x61,0xb9,0x2f,0x31,0x7a,0x09,0xd4,0x67,0x30,0x2f,0x60, -0x98,0xe9,0xf1,0x57,0x3b,0x5a,0xef,0x37,0x0e,0x4b,0x77,0x80, -0x5d,0xce,0x4e,0x60,0x3b,0x60,0xa3,0xc4,0x48,0x83,0xc8,0x9e, -0x00,0xf2,0xcc,0x74,0x82,0x4e,0x7d,0x1d,0x08,0xbf,0xd2,0x95, -0x10,0xf4,0xe5,0xc9,0xdf,0xe0,0x72,0x31,0xa9,0xa4,0xa4,0x6e, -0x3d,0xbb,0x78,0x10,0x3a,0x5b,0x11,0x87,0xa1,0x94,0x25,0x2f, -0x7e,0xb4,0xe3,0x56,0x97,0x74,0x8e,0xe3,0x19,0x3a,0x94,0xc3, -0x81,0x74,0x7d,0xf5,0x66,0x58,0xce,0x85,0x48,0x71,0x02,0xa8, -0x45,0x6d,0x8d,0x10,0x47,0x9b,0x7a,0x0d,0x86,0xab,0x4a,0xa7, -0xe0,0xb1,0x69,0x54,0x7b,0xcd,0x54,0x09,0xf2,0x63,0x78,0x46, -0xf2,0xbf,0x96,0x61,0x0c,0x0d,0x12,0x13,0x88,0x15,0x3e,0x2f, -0xdf,0x7b,0x0b,0xcc,0x06,0x37,0xb9,0xb6,0x7a,0x9a,0x9d,0x88, -0xf7,0xa1,0xf3,0xa2,0xe4,0x3d,0xfe,0xeb,0x51,0xde,0x6c,0x06, -0x6d,0xd0,0x05,0xa8,0x1e,0x0f,0x28,0x94,0x3d,0xb0,0x90,0xb9, -0x21,0xc7,0xe2,0x59,0x38,0xbe,0x96,0x69,0xab,0xbc,0x3e,0x39, -0x19,0x63,0x54,0x13,0x5b,0x03,0x8c,0xa2,0xab,0xe9,0xa4,0x64, -0x6b,0xa5,0xa0,0x96,0x3c,0x0f,0x6a,0x11,0x03,0x9c,0xf1,0x27, -0x6e,0x1c,0x3d,0x47,0xbb,0x7a,0xf0,0x34,0x2c,0x48,0x75,0x8e, -0xae,0xea,0xa8,0x4a,0xc3,0x5b,0x0c,0x51,0x5b,0xf2,0x14,0x0e, -0x13,0x59,0xee,0x72,0x5f,0x51,0x99,0xf5,0xe1,0xd8,0xc1,0xde, -0xf0,0x43,0xcd,0xcd,0xb0,0x08,0xac,0x65,0x5d,0xee,0x34,0xd2, -0x34,0x18,0x2a,0x81,0x5f,0xdc,0x60,0x94,0xe3,0x66,0x59,0xbf, -0x6f,0x6c,0xac,0x07,0xbb,0xa9,0x08,0x36,0x98,0xe8,0xd1,0xe7, -0x43,0x91,0x4b,0x5c,0xd6,0x1f,0x14,0x41,0x23,0x89,0x97,0xed, -0x34,0xd8,0xa5,0xb9,0x39,0xc8,0x9a,0x9f,0x16,0xe9,0xdf,0x1b, -0x91,0xaa,0x7b,0x55,0x8f,0x62,0x84,0x98,0x95,0x3d,0xb2,0xf4, -0x15,0xd5,0x51,0x23,0xf7,0x10,0x91,0xa0,0x34,0xbb,0x15,0x8d, -0xe7,0xa8,0xc5,0xfc,0xbe,0x12,0xe7,0x6d,0x96,0x38,0x9a,0x6b, -0x95,0x55,0x5d,0x58,0xd9,0xe2,0xaa,0xd8,0x38,0x0a,0xea,0x7d, -0x3b,0x9d,0xa0,0xec,0x8e,0x27,0x54,0x2b,0xaa,0xd7,0x9c,0x53, -0x2f,0xe9,0x54,0xfc,0x47,0x90,0xb4,0x48,0x5c,0x0a,0x96,0x02, -0xb1,0xe2,0x34,0x2b,0x37,0x74,0x0c,0x60,0xc2,0x8d,0x1d,0x79, -0xd9,0x7e,0xff,0xfc,0xc5,0xcb,0x7c,0xeb,0x66,0xb9,0x65,0x3d, -0x93,0x04,0x56,0x19,0xf7,0x09,0xd9,0xc4,0x06,0x5f,0x78,0x50, -0xe9,0xd8,0xeb,0x09,0xf6,0xa8,0xf9,0x74,0x3e,0x9e,0xb8,0x24, -0x5e,0xfb,0x31,0x29,0x8a,0xa7,0x16,0xe5,0xa6,0xe8,0xfb,0x0b, -0xc4,0x79,0xe1,0xe2,0xf4,0xbc,0x58,0xb4,0x09,0x59,0x19,0x44, -0x19,0x16,0xf4,0xe4,0x7c,0x84,0xb3,0xe1,0x0d,0x5d,0x96,0xc0, -0x2b,0x1d,0xef,0xed,0x7f,0xbe,0x3c,0x9e,0x5e,0x04,0x3b,0xd1, -0x96,0xca,0x80,0x71,0x3e,0x9e,0x8c,0xea,0x80,0x5c,0x72,0x03, -0x2f,0x1e,0x7d,0x00,0xd4,0xf9,0xd6,0x2f,0x58,0x43,0x03,0x6d, -0x8e,0xcb,0xb4,0xd7,0x68,0xa1,0x32,0xc5,0xc9,0xdd,0xb4,0x62, -0xf9,0x75,0xcc,0x1d,0x4d,0x1a,0xe7,0x41,0x59,0x0c,0x30,0x09, -0x17,0x2b,0x46,0xc2,0x2c,0xe0,0xe3,0xf4,0x05,0x91,0xc5,0x81, -0x03,0x70,0xe1,0x34,0x12,0x04,0x4b,0x82,0x08,0x1b,0x64,0x08, -0x8d,0xe7,0xf6,0x9b,0x30,0xf8,0x07,0xf9,0x8a,0xc6,0xed,0x07, -0xf8,0x38,0x94,0x91,0xa5,0x9f,0xaa,0xae,0xfd,0x00,0xf2,0xab, -0xa1,0xda,0x96,0xfc,0x65,0x44,0x7b,0x7b,0x07,0xed,0x28,0xa2, -0x41,0xa5,0xfb,0xa0,0xe9,0xc6,0x47,0x13,0x8f,0xed,0xb9,0x57, -0x9b,0xae,0x99,0x53,0x35,0xbd,0x6f,0x39,0xf4,0x50,0x91,0x41, -0xf5,0x38,0x70,0xb8,0xbd,0x1b,0x14,0x8e,0xd5,0x22,0xda,0x33, -0xf4,0x2f,0x4c,0x8f,0xe8,0x85,0x94,0xbe,0x65,0xb1,0x10,0x95, -0x89,0xcc,0x8b,0xd6,0x15,0xcd,0xaa,0x17,0x36,0x10,0xec,0x26, -0x13,0x98,0xa6,0x7a,0x58,0xf5,0x29,0x6b,0x10,0x4d,0x81,0x75, -0x0c,0xc3,0xa3,0xb6,0x3b,0x12,0xa2,0x2c,0x50,0x2d,0xd8,0x80, -0x61,0xa9,0xb0,0x9e,0xf7,0xe3,0xab,0xf1,0x30,0xb4,0x7b,0x91, -0xbd,0x4c,0xad,0xec,0x4f,0x79,0xc6,0x6f,0x01,0xc8,0x17,0xf2, -0xf7,0x47,0xef,0x5f,0xee,0xbd,0xfb,0xc7,0xcb,0xdd,0x3c,0x79, -0xf0,0xc0,0xbf,0x25,0x0e,0x7d,0x25,0xac,0x0e,0x27,0x27,0xb6, -0x8d,0xbe,0x70,0x0b,0x71,0xbb,0xb4,0x86,0x4c,0xc2,0x6f,0x6c, -0xef,0xb5,0x63,0x80,0xea,0x4f,0x3e,0x7e,0xf7,0x19,0xda,0xd6, -0xf8,0x47,0x31,0x9a,0x97,0x84,0xd7,0xa3,0x57,0x13,0xff,0x4c, -0x1a,0x51,0x9d,0xf0,0xee,0xf8,0x57,0x78,0xbd,0x38,0x2f,0xcd, -0x0c,0xfe,0x51,0xef,0xbc,0xa6,0xdb,0x93,0xa7,0x05,0xe0,0x80, -0x81,0xaa,0xde,0xfc,0x20,0xb5,0xa6,0x0b,0xd2,0x1e,0x1c,0x33, -0x3d,0x57,0x83,0xd8,0xc5,0x60,0x2f,0x1f,0xe1,0x34,0x3f,0x2c, -0x9b,0xc5,0x04,0xfe,0x34,0x5a,0x8f,0x7a,0x29,0x99,0x63,0x95, -0x4d,0x22,0x01,0x7e,0x8f,0x3f,0x53,0x06,0x02,0x3d,0x42,0xfb, -0xaf,0x94,0x6e,0x78,0xbe,0x9a,0xbc,0xa1,0x0b,0x37,0xf9,0xcd, -0x83,0x62,0x94,0x89,0x07,0x1a,0x0a,0xfb,0xbe,0x4e,0xb8,0xc5, -0xc0,0x21,0xec,0xa8,0x46,0x52,0x44,0xb1,0xce,0x3d,0x49,0x38, -0x6e,0x51,0x71,0x8c,0x17,0x65,0x1f,0x1e,0x15,0x20,0x43,0x1d, -0x61,0xd2,0xd1,0x09,0xaa,0x20,0xa8,0xa1,0x8f,0xfa,0xdc,0x9e, -0x6b,0xc7,0x9f,0xd7,0x8e,0x8a,0xd1,0xd1,0xe0,0x21,0xc5,0x45, -0xd7,0xce,0x50,0x3c,0x3a,0x3a,0x2a,0x84,0x66,0xcb,0x5d,0xdc, -0xf8,0x99,0x7b,0xd9,0x1e,0xa8,0xde,0x90,0xb4,0x27,0x3a,0xad, -0xd7,0x88,0x7b,0xcb,0x6e,0xa2,0x7d,0x7a,0xaa,0x3b,0x30,0xd7, -0x3e,0x7a,0xb8,0x97,0x4c,0x18,0xf2,0x22,0xe8,0x29,0x60,0x84, -0xba,0x0d,0xfb,0x41,0x26,0xf9,0x6a,0x09,0x1b,0xb9,0x1f,0xf5, -0xbb,0x75,0x1c,0xae,0x61,0x24,0x34,0x60,0xce,0x8e,0x21,0x63, -0xfa,0xed,0x20,0xd5,0x25,0x30,0xb2,0x56,0x23,0x1e,0x79,0x35, -0x5f,0xe2,0xbe,0xc1,0xfa,0x19,0x12,0x2d,0x62,0x34,0x48,0x83, -0x92,0xfa,0xbb,0x97,0x12,0xd6,0x3f,0x87,0xee,0x70,0x03,0xaf, -0xd1,0xcb,0x1a,0xfd,0xc3,0x07,0x87,0x9f,0xf0,0x12,0xc8,0x61, -0xc3,0x3d,0xc3,0x20,0x33,0xe4,0x80,0xc8,0xbe,0xa7,0x29,0x0e, -0xb9,0xdb,0x07,0x55,0x31,0xe2,0x61,0x25,0xe3,0xe6,0x21,0x0e, -0x9c,0x15,0x63,0xe8,0xe1,0x5a,0x79,0x3e,0xbd,0xbe,0x18,0xd1, -0x4d,0x6b,0x18,0x74,0xae,0x4d,0xf1,0xba,0xf5,0xc3,0xc6,0x11, -0xd0,0x7b,0x64,0xd6,0x8e,0xe8,0x13,0x8f,0xd2,0x87,0x6a,0x80, -0x9a,0xca,0x68,0x83,0xef,0x11,0x3f,0xca,0xbe,0x6b,0x72,0x7f, -0x7f,0xdf,0x8d,0xbc,0xa0,0x7b,0xdd,0x18,0xe4,0xcb,0x28,0x20, -0xbe,0x7d,0x07,0xfc,0xf9,0xe3,0x1e,0x3a,0x53,0x5d,0x86,0xaa, -0x84,0x17,0x8e,0xec,0x06,0x4e,0x7e,0x2f,0xda,0xbb,0x74,0xed, -0x0b,0x54,0x3c,0x41,0xa3,0x73,0x1b,0xf6,0x85,0x83,0x1d,0xca, -0xf1,0xd2,0x95,0x46,0xc0,0xf8,0x01,0xea,0xba,0xcf,0x78,0x84, -0xd0,0xbd,0xa5,0x39,0x46,0x82,0xc8,0x70,0xb8,0x98,0x5e,0x97, -0xe4,0x8b,0xc7,0x32,0x37,0x3c,0x10,0xc6,0x1a,0x14,0xcd,0xfe, -0xfd,0xdd,0x8c,0xdc,0x2e,0x0d,0xba,0x3e,0x04,0xe3,0x9b,0xe2, -0xe3,0x58,0xd3,0x9b,0x86,0x55,0xe5,0xfe,0x35,0x98,0x4e,0xea, -0x79,0xb1,0x88,0x87,0x28,0xdf,0xcc,0xbf,0x1b,0x9c,0x59,0x67, -0x54,0x21,0x50,0xa7,0xdc,0x75,0x2b,0xca,0xfc,0x63,0x14,0x27, -0x90,0x86,0xf3,0xc6,0x46,0xf2,0x20,0x59,0xc7,0x99,0xe9,0x1d, -0x19,0x80,0x28,0x1f,0xe0,0x96,0x98,0x8f,0x00,0xdd,0x8d,0x32, -0x30,0x2c,0x0d,0x88,0xe5,0x58,0xb1,0xea,0x95,0x3c,0x82,0x92, -0x0e,0x94,0x48,0xff,0x61,0xab,0x4a,0xe6,0x8a,0x22,0x3c,0xf3, -0xdb,0x5d,0x89,0xe3,0xcf,0xe1,0x16,0xe9,0x2f,0xbb,0x31,0x47, -0x6f,0x55,0xbe,0x24,0x6e,0x49,0x02,0x34,0x3d,0x02,0xcd,0xc2, -0x44,0x89,0xc7,0xc3,0x63,0x45,0x15,0x60,0x67,0x62,0xae,0x91, -0x75,0xdb,0x57,0x98,0x9f,0xd1,0xc3,0x3b,0x6e,0x56,0x57,0x30, -0x4d,0x69,0xc0,0xe5,0x1a,0xb8,0xef,0x72,0x07,0x46,0x9c,0x15, -0xac,0xca,0xd6,0xc3,0x51,0xa5,0xe7,0x3c,0x8a,0xc3,0x66,0x94, -0x8f,0xe2,0x3c,0xe7,0xe1,0x41,0x63,0xb8,0x85,0x4a,0x66,0x3e, -0xde,0x7d,0xa6,0x46,0xec,0x27,0x27,0xa1,0x4e,0x6f,0xe8,0x67, -0x63,0x83,0x7e,0x58,0x1e,0x67,0x2f,0x54,0x8d,0xe0,0x33,0x38, -0xbb,0xb0,0xd4,0xf2,0x39,0x56,0xc8,0xdc,0x46,0xd7,0x57,0x78, -0xfd,0x2f,0xd9,0xbd,0xc6,0x60,0x46,0x20,0x1b,0x95,0xe4,0x67, -0x7e,0x6d,0x46,0x50,0xc0,0xb7,0x86,0xb3,0x31,0x39,0xad,0x19, -0x5e,0xa0,0x58,0x3d,0x6a,0xad,0xfd,0x54,0x8e,0xd7,0x1e,0xda, -0x95,0x52,0x73,0xb0,0xb5,0xf9,0x74,0x8d,0x23,0x99,0x7c,0x5e, -0xbb,0x9e,0x14,0xb0,0x99,0x44,0xf6,0x5c,0xb6,0xd6,0xde,0x0b, -0xae,0x6c,0x0d,0x38,0x9b,0x59,0x73,0x55,0x61,0xb6,0x18,0x16, -0x2a,0x3e,0xe8,0x9b,0xa4,0x5b,0xdf,0xba,0x37,0x20,0x11,0x38, -0xa0,0xe5,0xca,0x0e,0x5a,0xef,0x2c,0xd5,0x54,0xd4,0xed,0x92, -0xde,0x36,0x6e,0x78,0x3a,0x36,0x6a,0xc6,0x0c,0x8d,0xcc,0x48, -0x53,0xc4,0x2d,0x2c,0xdc,0xab,0xf4,0xbb,0x81,0x30,0xbd,0xc6, -0x75,0xb0,0x65,0x78,0x5e,0xec,0x94,0x00,0x49,0xd2,0xa5,0x91, -0x16,0x2b,0x25,0xb6,0x22,0x13,0xd1,0xdc,0x36,0x5b,0xeb,0x67, -0xe9,0xcd,0x7f,0xcb,0x34,0xe5,0x46,0xaa,0xed,0x25,0x13,0x0d, -0x4d,0x51,0xe7,0xb2,0x32,0x48,0x7d,0xa3,0x5b,0x50,0x72,0xbd, -0x20,0x74,0x47,0xd3,0x35,0x97,0x61,0x1f,0x02,0x0f,0xa0,0x9f, -0xce,0x51,0xc7,0x63,0xb3,0xd0,0xd8,0x81,0x9f,0x74,0x83,0xa6, -0xdd,0x88,0x88,0xdc,0xe1,0x22,0x75,0x0d,0x77,0x24,0xb9,0x42, -0x8b,0xfa,0x8b,0x74,0x4a,0xe2,0x4f,0x52,0xd3,0x05,0xbb,0x99, -0x60,0xdd,0x92,0x62,0x13,0x72,0x31,0xc9,0x8e,0xbd,0x57,0x29, -0xbd,0xbb,0x3e,0x63,0x95,0xbc,0x15,0x0b,0x71,0xaa,0x44,0x9d, -0x08,0xa7,0xb5,0xf3,0x5a,0x38,0xd3,0xe9,0xe4,0x25,0x26,0xc7, -0xf8,0x21,0xd5,0x3c,0x1c,0xdc,0x9c,0x8c,0xbe,0xd9,0xfd,0xc2, -0xb1,0xd9,0x09,0xc0,0x2e,0x8b,0x11,0xba,0xec,0x5f,0x6f,0x54, -0xf0,0x2e,0x16,0x31,0xba,0x90,0xa8,0xe9,0x68,0x14,0x15,0xc3, -0xb3,0x1b,0xa4,0x46,0xc2,0x4c,0xa7,0xf1,0x28,0x59,0x2c,0x78, -0x17,0x74,0xdb,0xb1,0x40,0xed,0x19,0xc0,0x7d,0x3b,0x8c,0xa1, -0xd5,0xf8,0xcb,0x55,0x4d,0xf1,0x68,0x21,0x60,0x14,0x49,0xc2, -0xd1,0x11,0x20,0x91,0xbe,0x17,0x58,0xfa,0xab,0xdc,0x0d,0x06, -0xbc,0x29,0xe2,0xc6,0xb0,0x95,0x0d,0xc4,0x33,0x0d,0xeb,0x55, -0x6c,0xfb,0xe7,0xd3,0x4f,0x5f,0x78,0xb0,0xf5,0x85,0x17,0x3f, -0x5b,0x5c,0x49,0xe5,0x34,0xc9,0xa2,0xed,0xc7,0x27,0x3b,0xbd, -0x44,0x1d,0x04,0x27,0x59,0x62,0x8f,0xe4,0x93,0x81,0x77,0x6a, -0x3b,0x39,0xdb,0x3c,0x47,0x97,0x02,0xee,0x78,0xfc,0x87,0x22, -0x3a,0x79,0xf9,0xef,0xf8,0x0e,0xac,0xe4,0x4b,0xbe,0xc3,0x51, -0x9e,0x05,0x9f,0x74,0xdb,0x77,0x50,0xbc,0xb3,0xbf,0x60,0xd0, -0x52,0xdb,0xec,0x88,0x2b,0xd0,0x7b,0x50,0x4a,0x89,0x61,0xae, -0xf9,0x29,0xbd,0x71,0x8f,0x64,0xcf,0xc5,0x8f,0x1c,0xe4,0xda, -0x26,0xdb,0x25,0xc7,0x25,0x05,0x2d,0x60,0x28,0x5a,0x9a,0x57, -0xa6,0x9c,0x60,0x60,0x6b,0xaa,0x97,0x6c,0x17,0x8c,0x43,0xaa, -0x22,0xbb,0x71,0xd0,0x76,0xa1,0x60,0x89,0xea,0x19,0x12,0x1c, -0xf6,0xc9,0x4b,0xd6,0x97,0x9e,0xb2,0x86,0x9a,0x50,0x65,0x04, -0xc7,0xe8,0x82,0xdb,0x9c,0xfe,0x72,0x4d,0x6c,0xd9,0x8d,0xda, -0x3c,0x54,0x4a,0x42,0x1f,0xdc,0x69,0x35,0x67,0x51,0xbf,0x50, -0xd7,0x39,0xe9,0xf0,0x6b,0xcc,0x81,0x97,0x0e,0x9c,0xd2,0xa6, -0x74,0x81,0x19,0xad,0x4b,0x68,0x43,0x1d,0x84,0x17,0x23,0xec, -0x21,0x19,0xe3,0x92,0x33,0x32,0x10,0x4f,0x6c,0x09,0x62,0x19, -0x24,0xf0,0x07,0x5d,0xeb,0xca,0x57,0xc6,0xa1,0x0e,0x98,0x54, -0x14,0x79,0x88,0xc7,0x39,0x45,0x2e,0x76,0x0a,0xf2,0x8c,0x1c, -0x66,0xf7,0x8b,0x81,0x3e,0xd2,0x8a,0x84,0x90,0xf8,0x23,0x28, -0x82,0x52,0x9c,0x88,0xb7,0x57,0x2a,0xb4,0x9b,0x46,0x4d,0xa3, -0xe4,0xd5,0x06,0xe4,0xf6,0xef,0x27,0xeb,0x12,0x8c,0x13,0x8d, -0x74,0x57,0x02,0xf5,0x12,0x74,0x3a,0x57,0x09,0x9f,0xde,0xe2, -0x83,0x0a,0x2f,0x23,0xd5,0xf5,0x87,0x76,0xc6,0x12,0x65,0x86, -0x9d,0x58,0x73,0x62,0x15,0xb5,0x28,0x6d,0x97,0x82,0xb4,0xd4, -0x54,0xb1,0xb6,0xd4,0x59,0x74,0x00,0xac,0xd6,0x1d,0xf8,0xac, -0x97,0xfa,0x24,0x93,0x0f,0x7a,0xf2,0x1a,0x64,0x32,0x0e,0x2b, -0x8d,0xcf,0xc4,0x68,0x44,0x95,0x43,0x2d,0x9f,0x67,0x18,0xbf, -0x3f,0xce,0xe6,0x77,0xb2,0x78,0xf0,0x0a,0x50,0x6e,0x7d,0xd4, -0x85,0xaf,0x3a,0x02,0xb9,0x55,0x37,0xf9,0x75,0xbb,0xed,0x27, -0xe3,0x2f,0x7a,0x36,0xde,0xa2,0xa3,0x2c,0x6f,0x35,0x6d,0xb8, -0xdd,0x0a,0xda,0x8f,0x20,0x7b,0xa4,0xe3,0xbf,0x60,0x90,0xdf, -0x0d,0x82,0xe1,0x2f,0xcc,0xdd,0x60,0xdc,0xd2,0xfa,0xe3,0xfd, -0xa3,0xa5,0xcd,0x06,0xf6,0x73,0x1e,0x22,0x84,0xb3,0xf1,0x1d, -0xe6,0xff,0x5f,0x35,0x27,0xcc,0xb5,0x3c,0x7a,0xaf,0x34,0x14, -0xa4,0xfd,0xa5,0x56,0xf1,0xe3,0x7a,0xe5,0x19,0x9f,0xe6,0xdc, -0x1e,0x63,0x70,0xc1,0x5f,0x7d,0x21,0x5f,0x00,0xf1,0x29,0xb9, -0x7a,0xae,0x30,0xf6,0x5a,0xb5,0x77,0x19,0xdc,0xd1,0x3f,0x51, -0x6e,0x0b,0xd4,0x34,0xae,0x08,0x8c,0x81,0x39,0x1f,0xfa,0x2c, -0xf1,0x77,0xc4,0xd8,0x69,0x09,0x03,0xb2,0xa1,0x7d,0x79,0x32, -0x2b,0xae,0xc2,0x83,0x85,0xc8,0x38,0x44,0x7d,0x9c,0xce,0xa8, -0xbf,0x48,0x7b,0x6f,0xfd,0x3f,0xbb,0x16,0xc1,0xb3,0x8e,0x2d, -0x10,0x3a,0x2c,0xe6,0x24,0xcf,0xfd,0xad,0x59,0x89,0x28,0xc9, -0x39,0x3f,0xcd,0x2e,0x38,0x0b,0x6f,0x48,0x41,0xa9,0xdc,0xab, -0xd0,0xe8,0xc4,0xa4,0x1b,0x52,0x07,0x23,0x60,0xde,0x50,0x65, -0x0d,0xc7,0x16,0x14,0x66,0xf2,0x8e,0x63,0x16,0xd6,0x5c,0x7d, -0x50,0xf4,0x2f,0x2d,0xff,0x0c,0x1a,0x47,0x19,0xc6,0x54,0x8c, -0xc9,0x9c,0xb1,0x8b,0x58,0x95,0xc9,0xa9,0xcb,0xbb,0x1f,0x0f, -0x5e,0xbf,0x7b,0xbb,0x1f,0xde,0xd0,0xc6,0xa3,0x81,0x46,0x2f, -0x3b,0x2c,0x9b,0xc3,0xd2,0x1e,0x15,0xa4,0x3d,0x4e,0x39,0x9b, -0x4d,0xaf,0xaf,0xfc,0x59,0x01,0xba,0x9a,0x85,0x07,0x58,0x47, -0xf0,0xd5,0x69,0x7a,0xe9,0xef,0xe1,0x68,0xf0,0x88,0x7c,0xfc, -0x88,0xa3,0xf9,0x28,0x27,0xd4,0xff,0x06,0xe9,0x87,0x74,0x1b, -0x5c,0x1d,0x54,0x70,0xdd,0xd1,0x39,0x05,0x51,0x85,0x36,0xe0, -0xb0,0xe1,0xd8,0x73,0xb7,0x59,0x6e,0x82,0xcb,0x61,0xe2,0x59, -0xe3,0xb6,0x7b,0xd5,0xfd,0x84,0x9b,0x12,0x9a,0xcd,0x5d,0xd6, -0x18,0x44,0xa2,0x90,0xba,0x41,0xec,0xae,0x1c,0xcb,0xf5,0x62, -0xdf,0xc4,0x6e,0x7a,0xc8,0x74,0x91,0x23,0x74,0xeb,0x40,0x84, -0x3b,0xd6,0x5c,0x4f,0x3e,0x4e,0xa6,0x9f,0x26,0xf2,0x86,0x0e, -0xcd,0x38,0xe8,0x95,0xc4,0xaa,0xb4,0x9a,0xd6,0x33,0xff,0x45, -0xc1,0xf7,0xe1,0x7a,0x76,0x4a,0x36,0xb6,0xc7,0xd3,0xeb,0xc9, -0x88,0x8f,0x7f,0x4a,0x7f,0x0f,0x14,0x73,0x8b,0x49,0xa1,0xac, -0x73,0x14,0xaa,0x23,0xe3,0x09,0x39,0x0a,0x29,0x39,0x4a,0x6f, -0x82,0x3a,0xeb,0x0b,0xe5,0x2b,0xcb,0xe7,0x21,0xb6,0x25,0x53, -0x02,0x52,0xbd,0x64,0xc7,0x22,0x98,0xff,0x5c,0x39,0x88,0xc4, -0x3b,0x78,0xaa,0x56,0x7d,0x9d,0xca,0xdd,0xb3,0x72,0xcc,0x02, -0x05,0x19,0xb9,0xd2,0x16,0xd4,0xeb,0x16,0xea,0x8d,0x8d,0x30, -0xdd,0xba,0x59,0x4f,0x85,0x30,0x7e,0x5f,0x41,0x9b,0x75,0xc5, -0xcb,0xd9,0xde,0x88,0x4e,0xd4,0x93,0x15,0xd2,0xef,0xa2,0x5b, -0x62,0x0d,0xe0,0x3d,0xb0,0x9f,0x34,0x55,0xee,0xba,0x85,0xa5, -0xa9,0x02,0x11,0x90,0xc6,0x43,0x49,0xbe,0xeb,0x1f,0x18,0xa1, -0xb9,0xb7,0x96,0x34,0xd5,0x51,0x4d,0xda,0x4c,0xd6,0x7a,0x49, -0x37,0xfc,0x72,0x6c,0x2a,0x5f,0x46,0x31,0x5c,0x09,0x15,0xd5, -0x08,0xc0,0x55,0x7e,0x5c,0x30,0x6a,0x68,0x74,0x9e,0x93,0x58, -0xe1,0x8a,0x4c,0xaa,0xe5,0x33,0x5c,0xc3,0xad,0xba,0xa6,0xe4, -0xdb,0x2f,0x56,0xf8,0xd9,0xe3,0x26,0xab,0xad,0x59,0x79,0x91, -0x6c,0x65,0x7b,0xd1,0x74,0x4f,0xef,0xb3,0x35,0x91,0xab,0x70, -0xee,0x42,0xff,0x7e,0x31,0x39,0xbb,0x18,0x0b,0x24,0x7f,0x97, -0x33,0x6b,0xf2,0xdd,0x2b,0x39,0x74,0x57,0xed,0x26,0xe8,0x76, -0xb9,0x64,0x1d,0x9f,0xf4,0xbb,0xc1,0x50,0x3f,0x46,0xba,0x1e, -0x9f,0x1e,0x71,0xee,0xe2,0x4f,0xaf,0xf1,0x65,0xe3,0x3b,0xa4, -0x9c,0xe7,0x89,0x43,0x26,0x51,0xc1,0xfd,0xbd,0xa1,0x31,0x5e, -0x0f,0x5b,0xdd,0xa1,0x69,0x16,0x44,0xa1,0x75,0x78,0x82,0x82, -0xbd,0x6a,0x8d,0xb0,0xae,0x67,0xea,0xbb,0x6a,0x47,0xbe,0xbf, -0xb4,0x15,0x91,0x8c,0x7d,0xce,0xdb,0x93,0xb0,0xc7,0x57,0xd8, -0x24,0x7f,0x61,0xf3,0x04,0xbd,0xa0,0x17,0x8a,0x46,0xf5,0x33, -0x52,0x36,0x6c,0x76,0x23,0x64,0x4f,0x7c,0x8d,0xd5,0x8e,0x11, -0x7f,0x3f,0x0c,0x35,0x49,0x88,0xb6,0x7b,0x72,0xcb,0xb8,0xc0, -0x13,0xcb,0x12,0x2d,0xf1,0xd6,0x24,0xd8,0x4c,0xe5,0x32,0x5b, -0x37,0xdc,0x9b,0x59,0xc2,0x30,0xc4,0x52,0x23,0xe1,0x29,0xa4, -0x0d,0xd3,0x57,0x44,0x36,0xce,0xbd,0x41,0x1d,0x55,0x49,0x16, -0x8c,0x3a,0xc0,0x31,0x7d,0xe4,0x4a,0x63,0x15,0x58,0x22,0x87, -0x17,0x65,0xc3,0x7e,0x92,0x89,0x89,0x5c,0x2c,0x5c,0x5e,0x4e, -0x2e,0x71,0x62,0x80,0xca,0x25,0xd8,0x78,0x84,0x7e,0x61,0x77, -0xd3,0x9e,0x0f,0xf5,0xa3,0x72,0xbc,0xf7,0xaf,0xb7,0xd0,0xc6, -0x06,0xe1,0xab,0x86,0x7e,0xa6,0x30,0x81,0xd5,0x71,0x42,0xd0, -0xd1,0xc8,0x10,0x99,0xed,0x96,0x81,0xe1,0x40,0x6d,0x4b,0xd0, -0x97,0x70,0x75,0xdf,0xa3,0x48,0x45,0x3e,0xc8,0xd4,0xbb,0x3c, -0x9b,0xf1,0x1f,0x74,0x21,0xfd,0xec,0x47,0x8e,0xe3,0x61,0x5f, -0xa5,0xc6,0xe8,0xdd,0x90,0x74,0xf6,0x46,0x9f,0x2e,0x72,0xd2, -0x6b,0xd2,0x38,0xb3,0xde,0xd9,0x7e,0x38,0x1d,0xd9,0x59,0x2a, -0x2d,0x6b,0xbc,0x18,0x1e,0x83,0x14,0xa1,0xe8,0x28,0xf3,0x9b, -0x24,0xc9,0x30,0xdc,0x61,0x44,0x6c,0x09,0x12,0x27,0xc8,0x49, -0xea,0x26,0x75,0x7c,0xb5,0x9a,0x4f,0x2a,0x4a,0x5e,0x05,0xcb, -0x57,0xc2,0x85,0x53,0xda,0x14,0xe1,0x69,0x14,0xaa,0xce,0x11, -0x53,0xaf,0xa4,0xa8,0xbd,0x78,0xc6,0xc1,0x2b,0x40,0x99,0xf2, -0x75,0xaf,0x52,0x5c,0xbd,0x92,0x05,0x9c,0x55,0x08,0x8c,0xe7, -0xf9,0x7a,0x07,0xcf,0x9d,0x5d,0x2c,0x70,0xb4,0xac,0x45,0xc9, -0xf0,0xd5,0x64,0x63,0xc3,0xde,0x34,0xf4,0x54,0xa4,0xb4,0x5a, -0xb8,0xb2,0x7a,0xca,0xf5,0x07,0xa9,0x8b,0x87,0x46,0x18,0x5e, -0xcb,0xe9,0xaa,0x7f,0x79,0xe6,0x11,0x59,0xbd,0x90,0xcf,0x6c, -0x36,0xd3,0x0b,0x65,0xb5,0xc1,0x37,0x39,0x7c,0x81,0xbe,0x87, -0x1c,0x68,0xfa,0x59,0xfe,0x67,0x8a,0x65,0xc8,0x58,0xaf,0xdc, -0xf5,0x85,0x25,0x4e,0xe9,0xaa,0xcf,0x50,0xdf,0xaa,0x8f,0xa6, -0xe4,0xf4,0x29,0xff,0x58,0x39,0x71,0xae,0x39,0x90,0xaa,0x7e, -0x07,0xf7,0x40,0xdf,0xf6,0xd1,0x85,0xb3,0x5f,0xe1,0xec,0x8f, -0xfe,0x44,0x2a,0x93,0x63,0xa6,0x68,0x80,0xe4,0x34,0xf2,0xbe, -0xfb,0x1c,0x7f,0x24,0xd9,0xb0,0x37,0x14,0x70,0xae,0x87,0x5b, -0x3f,0xc2,0x82,0xf7,0x64,0xef,0x0f,0x8c,0x7a,0xb4,0x78,0x9c, -0xea,0x99,0xed,0x52,0x53,0x68,0x6b,0x3b,0x7e,0x1c,0xaf,0xd4, -0x7d,0x24,0x2b,0x86,0x74,0x53,0xaf,0xb6,0xbb,0x32,0xbb,0x35, -0x0b,0x52,0xd3,0x75,0x7f,0x17,0xc3,0x1d,0x3c,0x4b,0x79,0x7d, -0xf5,0x1e,0x8f,0x7c,0x6e,0x96,0x5d,0xf7,0x52,0x3f,0x88,0xdc, -0xc0,0xc9,0x43,0x12,0x5c,0x31,0x74,0x2a,0x5e,0x4b,0x5d,0x38, -0x68,0x14,0xca,0x3c,0xcf,0x6b,0x09,0xef,0xea,0x01,0xa6,0x9e, -0x17,0x0b,0xfb,0xb2,0x24,0xf6,0x90,0x8f,0x8a,0x12,0xf6,0xae, -0x95,0x8e,0x65,0xe6,0x91,0xf3,0xdf,0xe8,0xfa,0x66,0x85,0xaf, -0x88,0x1a,0xc4,0x9e,0x1f,0xaf,0x6c,0x62,0x3b,0xfe,0x2a,0xe3, -0x8d,0x6b,0x13,0xc4,0x96,0xc0,0xcc,0x51,0xca,0x36,0x18,0xb6, -0x8f,0x61,0x0c,0xf9,0x9d,0x8b,0x8d,0x92,0xa0,0x9a,0xa4,0x17, -0x8c,0x2a,0xe0,0x6b,0xee,0x72,0x3a,0x52,0x08,0xe3,0x95,0x2b, -0x83,0x07,0x57,0xd3,0xba,0x6a,0xa1,0x65,0x9a,0x05,0xed,0x75, -0x3b,0xb6,0x5e,0x2d,0x3a,0xd8,0xd3,0xa7,0x8a,0x59,0xe7,0x6d, -0xcd,0xcc,0xf3,0xca,0xa0,0x16,0x66,0xa4,0x60,0x76,0x7c,0xe1, -0xae,0x7f,0xc4,0xf9,0x8d,0x2d,0x11,0x4f,0xce,0x18,0x61,0xdf, -0x17,0x09,0xa6,0xd0,0xed,0xf3,0x2d,0x58,0x29,0x58,0xb9,0xc1, -0x84,0x3d,0xcb,0x3d,0xbe,0x5e,0x23,0x5c,0xc4,0x72,0x7b,0x02, -0x92,0x41,0x69,0x2a,0x6a,0xef,0xb4,0xb6,0x48,0xfb,0xd3,0x48, -0xc5,0xa9,0x2c,0xb5,0x52,0x12,0x4f,0x68,0x19,0x67,0x99,0x1e, -0x4c,0x94,0xb2,0x8c,0x57,0xc7,0xbc,0x1f,0x56,0x5c,0x47,0x2e, -0x07,0x06,0x0e,0xcb,0xc5,0x32,0x8a,0xa8,0xa6,0x42,0x64,0x56, -0xc1,0x8c,0xbe,0x7d,0xe2,0x7a,0x2b,0xd5,0x04,0xad,0x1b,0xb5, -0x46,0x54,0x18,0x0f,0x39,0xa3,0x9a,0xe8,0xeb,0xd6,0xe3,0x2e, -0x03,0x29,0xbc,0x96,0xa0,0xb0,0xf1,0xea,0x50,0xc5,0x98,0xd2, -0x54,0xcb,0x02,0xb4,0xc7,0x37,0x91,0x7d,0x43,0xd0,0xd8,0xb7, -0x18,0x37,0x30,0x9c,0x06,0xb7,0x46,0x08,0x51,0x23,0x55,0x3e, -0x9b,0x31,0x74,0x06,0xb0,0xe5,0xd1,0xc4,0x84,0x60,0xf6,0x1b, -0xa3,0x9e,0xb6,0x2d,0x24,0xf5,0xf2,0xeb,0xc6,0x86,0xc2,0xc3, -0x96,0xcb,0x75,0xa5,0x82,0x32,0x69,0x8c,0xb8,0x18,0x79,0xac, -0xe8,0x3f,0x4a,0xa3,0xc4,0x3d,0x42,0x05,0xdc,0x03,0x07,0x8d, -0x8a,0xaa,0x40,0x3b,0xd7,0x3d,0x46,0x2f,0x83,0x6a,0xbc,0xf1, -0x4e,0x2c,0xaa,0xc3,0x71,0xf4,0x08,0x09,0x0e,0x44,0xda,0xe0, -0x29,0x72,0x3d,0xd3,0xeb,0x8d,0x55,0xef,0x72,0x52,0x66,0x95, -0x9e,0x52,0x22,0x9e,0x85,0x29,0x7d,0xa1,0xff,0x1e,0x19,0x58, -0x9e,0xb0,0x98,0x00,0x6e,0xe4,0xdb,0xda,0xb3,0xac,0x9b,0x6f, -0x9e,0x1f,0x04,0x52,0x68,0xa6,0x11,0x19,0x60,0x9c,0x8e,0x12, -0xcf,0x35,0x23,0x0a,0x96,0x41,0x9b,0xf7,0x74,0xa3,0x52,0x00, -0x3c,0x67,0xf3,0x93,0xad,0x9a,0x3a,0x32,0xd3,0xed,0xc9,0x50, -0x30,0x12,0x25,0xd1,0xcb,0x58,0xcd,0x66,0x37,0xfe,0xbc,0x60, -0x52,0xa4,0x95,0x8f,0x87,0x8e,0x4d,0x5d,0x65,0x76,0x53,0x4a, -0x2b,0x55,0x77,0x15,0x27,0xd5,0x6c,0x3d,0xad,0xe1,0x5f,0x88, -0x12,0x47,0x57,0x05,0xab,0xf3,0x55,0x42,0xa6,0xda,0xec,0x77, -0x44,0x7a,0x1a,0xaf,0x00,0x88,0x7d,0x6e,0x45,0x07,0x9c,0xa6, -0x15,0x8f,0x75,0x42,0x3e,0xfa,0xac,0xbb,0xaf,0x55,0x37,0xf7, -0xc8,0xd1,0x5a,0x63,0x58,0xae,0x1d,0x51,0x0f,0x1e,0xa5,0x3d, -0xcc,0x5f,0x13,0xc3,0xdb,0xb4,0x27,0x96,0xb7,0x55,0xb3,0xdd, -0xd0,0xf8,0x36,0x72,0x7b,0xe7,0x3f,0x20,0xf0,0x79,0x17,0x30, -0xec,0x94,0x2f,0x22,0x3b,0x31,0xc5,0x9a,0x74,0xd7,0x44,0xe4, -0x32,0x4e,0xe8,0x72,0xd6,0xe1,0x36,0xf7,0x29,0xed,0x52,0x54, -0xd6,0xd7,0x03,0xe3,0x44,0xda,0x10,0xa5,0x8d,0xe1,0x25,0x62, -0x55,0x5c,0x5f,0xcf,0x56,0x97,0xb9,0xda,0x04,0xb6,0x8c,0x81, -0xbf,0x19,0x88,0xb5,0xb9,0xd4,0xf9,0x37,0x31,0xc0,0xb4,0xa6, -0xaf,0xbd,0x00,0xfa,0x6f,0x83,0x94,0xa2,0x45,0x57,0xd7,0xb5, -0xbc,0xdf,0x77,0xb3,0x2a,0xdc,0x8f,0x5a,0xd1,0x63,0x39,0x18, -0x74,0x3d,0x2f,0x40,0xe5,0xaa,0xbd,0xad,0xe7,0x53,0xc5,0x15, -0x68,0xaa,0x34,0xc0,0x15,0xcf,0x03,0xac,0x0f,0xaf,0x01,0x41, -0x7f,0x67,0xe1,0x3a,0xea,0x0e,0x7d,0xfe,0x65,0x15,0x80,0xde, -0x28,0x7b,0x53,0x54,0x4e,0xb5,0x1f,0xca,0x52,0xa2,0xda,0xfe, -0xea,0x0d,0xb1,0xb5,0x96,0xf6,0x36,0x78,0x75,0xbb,0x54,0xb7, -0xfd,0x0c,0xf6,0x9b,0x62,0x3e,0x80,0x92,0x74,0x7f,0x70,0x2f, -0x91,0x2d,0x98,0xd0,0xf7,0x92,0xd9,0x22,0x91,0xed,0x0e,0x89, -0x82,0x17,0xeb,0xce,0x97,0x2e,0xd6,0xf0,0x51,0x8d,0xa0,0xcd, -0x6a,0xd6,0x6d,0x27,0xe6,0xe8,0x15,0xcc,0xd9,0x30,0x06,0xa5, -0x59,0x69,0x66,0x27,0x0d,0x1a,0xc0,0x56,0xf7,0x8c,0xa9,0x1d, -0xce,0xe9,0x5d,0x1b,0x6f,0x65,0xa5,0xcd,0xc4,0x7b,0x84,0x6a, -0xa3,0xa4,0xcc,0x20,0xf5,0x6e,0xbb,0x76,0x2f,0x41,0x46,0xdd, -0x69,0xb8,0x7f,0xa7,0x0d,0xda,0xda,0xad,0x78,0xd1,0xbc,0x92, -0x6d,0xff,0x9d,0x3f,0xab,0x9a,0x4d,0xdf,0x92,0xb7,0x5b,0xd2, -0x2e,0x35,0xca,0x44,0x03,0x72,0x7f,0x4e,0x14,0xf0,0xd0,0x09, -0xf7,0x88,0x58,0x30,0x09,0xf2,0x29,0x5a,0xad,0xcd,0x72,0xdb, -0xc7,0x2f,0x6e,0xb5,0xf0,0x73,0x6b,0x77,0xfa,0x5f,0xd2,0x86, -0x4c,0xa1,0x38,0x29,0xac,0xdf,0x45,0x52,0x9c,0x3a,0x69,0x8e, -0x3b,0x1b,0xf6,0xae,0xd1,0x72,0x5b,0x45,0x35,0xca,0x39,0xe5, -0x25,0x28,0xd0,0xbe,0xf2,0x4f,0xa8,0xe6,0xe4,0xb4,0x74,0x29, -0x4e,0xd7,0xd0,0xdf,0x9d,0x6f,0x5f,0xa5,0xe0,0x56,0x2a,0x7f, -0xef,0x2a,0x4f,0x9f,0x95,0xd9,0xd2,0x4e,0xa3,0xc0,0x87,0xc1, -0xf6,0x4d,0x2d,0x59,0xd6,0x90,0xc9,0x6a,0xef,0xd4,0xc1,0xda, -0x7a,0xc7,0x84,0x52,0x98,0x8d,0xa1,0x1e,0x45,0xf1,0x90,0x31, -0xe5,0x15,0x9c,0xa9,0x89,0x77,0x51,0xf7,0x28,0x49,0xcc,0x03, -0xcb,0x86,0x67,0x78,0xf5,0x72,0x20,0x06,0x8c,0x67,0x63,0x52, -0x68,0x32,0x77,0x1f,0xd3,0x26,0x60,0x7e,0x91,0xdb,0xb7,0x8a, -0xf1,0x94,0x8c,0x6b,0x9b,0x8f,0xd6,0x53,0xd6,0xb3,0x8f,0x6f, -0x62,0x25,0x94,0x7a,0x4c,0xe3,0xdf,0x1a,0x85,0x1d,0x4d,0x80, -0x45,0x1d,0x28,0xe0,0xf9,0xa6,0x3e,0xd6,0x34,0xab,0x8e,0x57, -0xbd,0x92,0xc7,0x86,0x85,0xb0,0x1e,0xe5,0x9c,0xf3,0xb5,0xf7, -0x92,0x90,0xf2,0xc2,0x62,0xf3,0xff,0x71,0x97,0xaf,0xbd,0xca, -0x41,0x42,0x9d,0x6f,0xbc,0xa8,0x4a,0x39,0x0e,0xfc,0x5d,0x69, -0x13,0x65,0x84,0x2f,0xbb,0x01,0xbe,0xc0,0xe1,0x5a,0x85,0xa4, -0xe8,0x14,0x43,0xf9,0x60,0xb3,0x6a,0x86,0x9a,0x22,0xf7,0xf1, -0xac,0x57,0x29,0xd6,0xa8,0x3f,0xb4,0xc2,0x09,0xe6,0x07,0x75, -0x2f,0xd4,0x7e,0x8f,0xab,0x87,0x66,0x69,0x66,0xbb,0xa1,0x17, -0x9d,0xa1,0xd4,0x02,0x07,0x27,0x71,0x35,0x10,0xfa,0x0c,0x8e, -0xad,0x26,0x98,0x98,0x2f,0xf6,0x04,0x66,0x4f,0xe6,0xf7,0xfd, -0x04,0xbf,0x71,0x27,0xd6,0xd6,0xbd,0xbb,0x3f,0x27,0xbe,0xc5, -0x96,0xe0,0x76,0x1f,0x73,0x15,0xdb,0x12,0x7d,0xac,0x46,0x1d, -0x23,0x47,0x31,0xf7,0xf5,0x3b,0x46,0xfb,0xac,0x14,0x8f,0xe8, -0xba,0x01,0x74,0x10,0x17,0xe8,0x77,0x76,0x02,0x17,0x96,0x49, -0x97,0xf7,0x75,0x27,0xe4,0x1b,0x99,0x44,0xe0,0x44,0x22,0xce, -0x2b,0x9f,0xd0,0xe2,0xe5,0xd6,0x71,0x04,0x67,0x24,0xa7,0x18, -0x26,0xa7,0xb1,0x4f,0xb0,0x10,0x23,0x48,0x58,0x92,0xe9,0xe2, -0x4e,0xd4,0x41,0xa9,0x03,0x52,0x0c,0xeb,0xe0,0x78,0x80,0xb3, -0x80,0xe8,0x8d,0x57,0x6c,0x9b,0xd7,0x3b,0xfa,0x14,0x32,0x0f, -0x3b,0x3a,0x74,0xb5,0xd6,0x0b,0x16,0x86,0x20,0x2b,0xbe,0x0b, -0x8d,0x06,0xc1,0x74,0x25,0xba,0xda,0xd2,0x72,0x2f,0x5a,0x7e, -0xd9,0x32,0x18,0x1e,0x02,0xb2,0xf5,0x78,0x6a,0x08,0x2a,0xd5, -0x5e,0xde,0x60,0xc2,0xdf,0xb2,0xce,0x6a,0xb3,0xd5,0xb0,0xb9, -0xdb,0xef,0xea,0x2a,0x02,0x14,0x12,0x6b,0xe6,0x5e,0x86,0x46, -0xd5,0xce,0xf8,0x68,0x95,0x25,0xd5,0x12,0x58,0x34,0x48,0x8a, -0x7f,0xff,0xdf,0xd7,0x63,0x0a,0x5e,0x71,0x75,0x7d,0x7c,0x51, -0x94,0xe7,0x2f,0xff,0x00,0x18,0x80,0x78,0xfe,0xe3,0xeb,0x06, -0x48,0xf6,0xd7,0x17,0x43,0xe0,0x41,0xd1,0xba,0x84,0x57,0xfe, -0x87,0xa3,0x40,0xb6,0x17,0x50,0xf4,0x30,0xe1,0xc0,0xcc,0xf1, -0x74,0x3a,0x87,0xca,0x87,0xe4,0x85,0xa3,0xc1,0x51,0xcc,0x8d, -0x43,0x62,0xa4,0x8c,0x15,0x54,0x15,0x7a,0x3e,0x09,0x3c,0x87, -0x4a,0x92,0xd4,0x99,0x3b,0x3c,0x7c,0x46,0x9f,0xb8,0x46,0x71, -0xa7,0xd9,0xc8,0x0b,0x43,0xf8,0xed,0xfc,0x27,0xde,0xf6,0x83, -0xee,0x5c,0x4b,0x7e,0x3a,0x78,0xb5,0xf9,0xb7,0xa4,0xdb,0x9f, -0x9c,0x1d,0x1e,0x66,0xe4,0x58,0x6d,0x60,0xfa,0xd6,0xc7,0x1a, -0x3c,0xe2,0xd0,0xdb,0x54,0xef,0x7f,0xa8,0x97,0x96,0x7d,0x34, -0x2d,0x9f,0x4c,0xa9,0x68,0xbb,0x7e,0x23,0x5b,0x52,0xe0,0x24, -0x93,0xf1,0xda,0x7a,0x71,0x89,0x31,0x97,0x87,0x93,0x79,0x77, -0x49,0x75,0x21,0x27,0x77,0x20,0x7c,0x71,0x6d,0xf9,0x6c,0x8b, -0xc8,0xdd,0x79,0xa8,0x4e,0x32,0xa5,0x09,0xe4,0xc3,0x95,0x7b, -0xb8,0x9b,0xe4,0x1a,0x55,0xfe,0xd4,0x51,0x09,0x87,0xcf,0xb5, -0x11,0x08,0xe5,0xfe,0xbc,0x6d,0xac,0x07,0x0f,0x5c,0x94,0x43, -0x0b,0x91,0xa4,0x5d,0x9b,0x7b,0x39,0x1d,0x5d,0x5f,0x8c,0xf9, -0x96,0xbd,0x64,0x1a,0xf4,0x46,0x9c,0x0c,0xa0,0x51,0x25,0x00, -0x89,0x2a,0x68,0x94,0xf7,0x26,0x77,0xf3,0xff,0xb7,0xaa,0x37, -0x26,0xb9,0xf9,0xff,0x9b,0x3a,0x0e,0x1d,0x4f,0x28,0x86,0xf6, -0xac,0xd8,0x1f,0x9f,0x51,0xef,0x91,0x9d,0x8b,0xf0,0x28,0x97, -0xc9,0x83,0x0b,0x4d,0xe8,0x81,0xe5,0x39,0xb7,0x0c,0x5b,0x5f, -0x6d,0x3f,0xdd,0x3a,0x2b,0x30,0x9c,0xa5,0x4e,0x7c,0xbc,0x4b, -0x89,0x79,0x90,0xb8,0xfd,0x1d,0x25,0x36,0x13,0x75,0x6a,0x5b, -0x83,0xfe,0x0a,0x84,0x76,0x4a,0xdd,0xbf,0x82,0x72,0x65,0x4c, -0x89,0x0e,0xf5,0x8d,0x84,0xaa,0x0a,0x9e,0xb4,0xa9,0x82,0xff, -0x0c,0x49,0x79,0x4e,0x89,0x59,0x48,0xca,0x93,0xad,0x33,0x68, -0x9f,0x30,0xed,0x05,0x01,0x9a,0x30,0x11,0x50,0xc6,0x14,0xf5, -0x12,0x48,0x06,0x84,0xc1,0x87,0xbc,0x9f,0x5e,0xc3,0xb4,0xb2, -0x16,0x85,0x46,0x42,0x44,0x97,0x62,0xc8,0x64,0xd3,0x73,0x07, -0x40,0xc9,0x16,0x2a,0xb7,0x0f,0xe4,0x77,0x84,0xb2,0xae,0x67, -0x17,0x3f,0x62,0x0c,0x68,0xb2,0xf2,0xf7,0xb5,0x48,0x77,0xdb, -0x08,0x34,0x18,0x6b,0x9d,0x42,0x45,0x8b,0xb1,0x6e,0x69,0x38, -0x3e,0x93,0x36,0xaa,0x81,0xd9,0x05,0xbb,0x88,0x39,0x63,0x6b, -0xe0,0xc4,0x11,0x20,0x4e,0x91,0x05,0x6f,0x44,0x5b,0x6c,0xeb, -0xc4,0x58,0xe5,0xe7,0x12,0x6d,0x07,0x08,0xab,0xab,0x49,0x90, -0x38,0x2b,0x76,0x9d,0x1a,0xd9,0xfc,0xb3,0xe7,0x8f,0xa2,0x7c, -0x15,0x4a,0x73,0x20,0x16,0xaa,0x1d,0x06,0x6a,0x2c,0x80,0x18, -0x76,0x0a,0x12,0x08,0x6b,0x7c,0x13,0x78,0x63,0x03,0xfa,0x57, -0x0e,0xcf,0x54,0x94,0x73,0x8e,0x25,0xca,0x5f,0xc7,0x79,0xe5, -0xf5,0x31,0x4c,0xc2,0x06,0xfa,0xfd,0x67,0xf1,0x8e,0xf0,0xfa, -0xa6,0x94,0x26,0x7f,0x2f,0x6e,0x95,0x5e,0xe3,0x82,0x83,0xae, -0xd4,0x41,0xe2,0xf2,0x0e,0xa3,0xa4,0x3d,0x6c,0x42,0xcb,0x4e, -0xb7,0x6a,0x8f,0x38,0xa7,0x73,0x68,0x81,0x41,0xcf,0xae,0x2f, -0x59,0x0d,0x38,0xc3,0x01,0x42,0x67,0xc6,0x3c,0x54,0x74,0x64, -0x7b,0xe9,0x34,0xd5,0xd4,0xbb,0x2f,0x5f,0x3d,0xff,0xe9,0xcd, -0xc1,0xd1,0xf3,0x17,0xa4,0x17,0x74,0xdd,0x1a,0xb5,0xb3,0x6a, -0x62,0x4e,0x30,0xde,0x39,0xe2,0xf9,0xb0,0xfc,0x6e,0x3a,0xfa, -0x8c,0x3e,0x7b,0x7f,0x7c,0xb7,0x7f,0xb0,0xf8,0xf1,0x27,0xf8, -0x87,0x91,0xf5,0xd3,0x07,0x5b,0x85,0x38,0x8c,0xe1,0x5d,0xea, -0xe5,0x78,0x7e,0x3e,0x1d,0xa5,0x5d,0xfb,0x2d,0xfd,0x09,0x6d, -0xfe,0x3c,0xea,0x8e,0x19,0x6e,0x9b,0xe1,0x63,0x33,0x7c,0x22, -0x7e,0x17,0xb1,0x99,0xc5,0xcf,0x96,0x21,0x87,0x4c,0x3c,0x36, -0x68,0x0c,0xdd,0x1e,0x3d,0xfe,0x49,0xc6,0xa1,0x29,0x86,0x4f, -0x2c,0x82,0x7c,0xf8,0x98,0x83,0xc7,0x3f,0xce,0xe8,0x67,0x3b, -0xe3,0x30,0xf0,0x6e,0x98,0x0c,0xb7,0x53,0x0a,0x3f,0x85,0xf8, -0x81,0x14,0xac,0x3c,0x07,0x7a,0x54,0x71,0xb7,0x41,0xd1,0xa5, -0x3a,0x68,0x75,0x60,0x61,0x3a,0x46,0xaa,0xdd,0x16,0x60,0x97, -0xb3,0x6d,0x73,0x1e,0xbb,0x10,0xf6,0x01,0x92,0x9e,0xc7,0x91, -0x49,0x93,0xf6,0x98,0x84,0x4e,0xe6,0x88,0x62,0xa4,0x5d,0x89, -0x8f,0x1f,0x46,0x89,0x97,0xf8,0xf5,0xe1,0x22,0xd0,0x48,0x8e, -0x87,0x23,0x68,0xa2,0x52,0x2b,0x72,0xaf,0xaf,0xf0,0xce,0xf0, -0x13,0x17,0xc5,0xbe,0x5c,0xeb,0x73,0x0d,0x86,0x5b,0x7c,0xcd, -0x36,0xf9,0x1a,0x91,0x3c,0x30,0xa4,0xa2,0xbd,0x69,0x2f,0x7d, -0x89,0xc4,0x54,0xda,0x9d,0x46,0x5f,0x51,0xbe,0x9e,0x94,0xb0, -0xcc,0x9d,0x8c,0x5f,0x0c,0x2f,0x2e,0x72,0x44,0x07,0x42,0x2e, -0xa7,0x4c,0x4f,0xdd,0x28,0x96,0xbd,0x7e,0x08,0x4d,0x9f,0x9b, -0xc9,0x58,0x11,0xe3,0x8e,0x5e,0x7f,0x90,0xb1,0x43,0x6d,0x19, -0xfe,0x08,0x93,0x1a,0x5c,0x63,0x40,0x3a,0x3d,0x2d,0xce,0xd0, -0x5e,0x64,0x56,0x9d,0x5b,0xb9,0x45,0xe3,0x93,0x36,0x36,0xaa, -0x69,0x2d,0x5b,0x74,0xb1,0x58,0x3d,0x4f,0x1d,0x7e,0x0a,0x91, -0xf1,0x2f,0x54,0x42,0xe5,0x17,0x0b,0xaf,0x0d,0x0a,0x27,0x59, -0x2d,0x1b,0x4b,0xb8,0x67,0xd8,0x4b,0x01,0x30,0x25,0x69,0x17, -0xff,0xee,0xeb,0xb1,0x69,0x0d,0xdf,0x3a,0xcc,0xde,0x3c,0xaf, -0x20,0xa3,0x2c,0x9f,0x4d,0x92,0x35,0x75,0x92,0x21,0xb6,0xd1, -0x02,0x99,0xe8,0x27,0xbb,0x20,0x28,0x34,0xc6,0x33,0x8c,0x95, -0x5c,0xbe,0xc5,0x84,0x22,0x37,0x4a,0x65,0x96,0xa6,0x36,0x8b, -0x78,0x10,0x47,0xf2,0x9b,0x5e,0x16,0xe5,0x38,0x27,0x21,0x41, -0x55,0x90,0xb6,0xe6,0xe8,0xa2,0xaa,0xc6,0x95,0x9e,0x65,0x03, -0xb9,0xe3,0x8d,0x54,0xa7,0x45,0xc4,0x3c,0xf8,0x81,0xbc,0xa2, -0xea,0x96,0x06,0x08,0x6e,0xbc,0xac,0x80,0x63,0xcd,0x84,0x28, -0x63,0x3d,0x5f,0x5f,0x0f,0x47,0x58,0xba,0x7a,0xde,0x9c,0x9c, -0x9e,0xe1,0xb4,0xc1,0x5e,0xc3,0x93,0x0b,0x0b,0xb1,0x76,0x42, -0x34,0x5f,0xcf,0x28,0x6e,0x4a,0x6b,0xcd,0xcd,0x2b,0x4b,0x22, -0xce,0x2e,0xb4,0x29,0x1f,0xe2,0x95,0xfe,0x09,0x4d,0x1c,0x7b, -0xce,0x81,0xaf,0x78,0xb4,0x11,0x8d,0xf2,0x84,0x2c,0xcd,0x60, -0xad,0x9f,0x1e,0xff,0x4a,0xe6,0xe9,0xb5,0xc4,0x57,0xc0,0x40, -0x8a,0x0b,0xf1,0x34,0xb4,0xbe,0x21,0x6f,0x3b,0x2e,0x4e,0x6d, -0xe6,0x5a,0x17,0xad,0x03,0x45,0xaf,0xc7,0x6a,0x87,0x60,0x86, -0x51,0x2e,0xde,0x79,0xca,0x38,0xf2,0x9d,0x5f,0xeb,0x44,0x81, -0xe1,0x9a,0x3b,0x97,0x5f,0xb7,0xd1,0x94,0x6c,0x6c,0xa8,0x8b, -0xdf,0x29,0x98,0x81,0x69,0x08,0x2f,0x59,0x2c,0x70,0x47,0x9d, -0xca,0xd0,0x76,0x9d,0x89,0x02,0xfb,0x78,0x56,0xa6,0xa6,0xb2, -0xf4,0xe5,0x21,0xa4,0x72,0x54,0x55,0x59,0x36,0xeb,0xaa,0x1d, -0xf3,0x5c,0xe3,0x4a,0x95,0x4f,0xc3,0xdf,0xa0,0x02,0x6c,0x3b, -0x9f,0xb6,0x94,0xc1,0x57,0xc3,0x45,0x56,0xce,0x7c,0xe5,0xad, -0x2b,0xe0,0x60,0xd2,0x20,0xe2,0xb4,0xab,0xd2,0x50,0xa6,0x42, -0x69,0xc7,0x3a,0x6e,0x37,0xb6,0xfd,0x71,0x6f,0x3b,0x9f,0xe2, -0x8e,0x05,0x24,0x6d,0xf6,0x29,0xac,0x96,0x48,0xe1,0xd3,0xc1, -0x9a,0x18,0xc8,0x3a,0x32,0xf3,0x80,0x0b,0xf0,0x72,0x63,0x21, -0xed,0xea,0x22,0x08,0xdc,0x32,0xca,0xf3,0x12,0x08,0x02,0xde, -0x97,0x87,0x0b,0xb3,0xad,0x8c,0xae,0x2c,0x84,0x35,0x76,0xbd, -0xc0,0x02,0xe5,0xdc,0x77,0x2e,0x16,0x9c,0x80,0x17,0x7a,0xdc, -0xf7,0xe0,0x96,0x51,0xad,0xf1,0xa3,0x51,0x81,0x0f,0x43,0xe6, -0x32,0xbb,0x4e,0x7e,0x15,0x8c,0x75,0x92,0xe7,0x4a,0xc9,0x70, -0x05,0x2e,0x27,0xc5,0xf8,0x56,0x5d,0x72,0x78,0xf4,0x40,0xd8, -0xc9,0x6f,0x40,0x9a,0xcb,0x6e,0x58,0x2a,0xc9,0x92,0xef,0x5f, -0x1e,0x24,0x4b,0x53,0x0e,0x7f,0x1f,0xfb,0x34,0x14,0x68,0x20, -0xf1,0x37,0xdc,0x34,0x84,0x90,0x46,0xa6,0x1c,0x6e,0x8b,0x45, -0x73,0xe4,0x01,0x76,0x5f,0xbe,0x79,0x79,0xf0,0x12,0x0a,0x26, -0x7c,0x8d,0x20,0xa9,0x66,0x2d,0x0d,0x0e,0x4d,0xb7,0x47,0x23, -0x05,0x94,0xcc,0x56,0x97,0x28,0xef,0xf2,0xfd,0x2e,0x99,0x5f, -0x0d,0x4e,0x4e,0x97,0x86,0x2f,0xc6,0x0f,0x83,0xdc,0xf3,0x0f, -0x9b,0x24,0x61,0xf0,0x7d,0x4f,0x00,0x0f,0x81,0xf6,0x44,0x81, -0x49,0x9a,0x5e,0x8e,0x1b,0x29,0x0d,0x73,0x53,0x27,0xa7,0x93, -0x24,0xe9,0x07,0x66,0x7e,0xa3,0xd7,0x89,0x4c,0x79,0x30,0xa2, -0xc5,0x42,0x86,0x0d,0x77,0xc2,0x4f,0xd6,0x54,0x1a,0xb7,0x5c, -0xbc,0xb5,0x42,0x45,0x88,0xba,0x0c,0x73,0x8d,0x37,0xaa,0x2c, -0x2c,0xd9,0x9f,0x9d,0xba,0x7d,0x8c,0xf1,0xdb,0x13,0x4a,0xd7, -0xbb,0x15,0xb7,0x7a,0x42,0xa2,0xf8,0xef,0xde,0x3a,0xfc,0xb0, -0xa5,0xf6,0xce,0x44,0x48,0x7a,0xb3,0xae,0x22,0x72,0x24,0xbf, -0x1c,0x62,0x48,0xf1,0x24,0x65,0xc1,0x94,0x21,0x36,0x36,0xe8, -0x37,0x08,0xdd,0x91,0x34,0x7e,0x59,0xf4,0x01,0xf8,0xf0,0x70, -0x90,0xc2,0x16,0x8c,0x00,0x9a,0x49,0xe3,0xf0,0xf0,0xc3,0xe2, -0x41,0x6a,0x8b,0xe3,0xba,0x06,0x33,0xcd,0x51,0xc5,0x62,0x14, -0x5e,0x6f,0xc1,0x75,0x16,0x3f,0xec,0x9a,0xd4,0x30,0xb2,0xcb, -0x83,0x6d,0x3f,0x6e,0x07,0x61,0x8b,0x68,0xe7,0x9f,0x5f,0x33, -0xf5,0xf5,0x5c,0xf5,0x9d,0xfe,0x63,0x8e,0x5c,0x5b,0x10,0xeb, -0x96,0xa2,0xf1,0xad,0x0a,0x07,0xd3,0xe3,0xfc,0xbe,0x4d,0x18, -0x64,0x7c,0x99,0x48,0xe6,0x88,0x4f,0x57,0x4b,0x8d,0xb5,0x1e, -0xc7,0xed,0x2e,0xdb,0x93,0x70,0xdc,0xa3,0x5e,0xc3,0x77,0x5c, -0x5e,0xbb,0x8b,0xaf,0x7c,0xab,0x6e,0x49,0x68,0x3e,0x5b,0x9b, -0x6f,0x41,0x93,0x9c,0x61,0x14,0x73,0x87,0xb7,0x99,0x3c,0xe8, -0xa0,0x03,0xd4,0xdb,0x10,0x35,0xb6,0x7a,0xe9,0x2d,0xc8,0x5c, -0x53,0xd1,0x81,0xb9,0xb9,0x80,0xa5,0xa5,0x98,0x9c,0xed,0x5f, -0x0c,0xcb,0xf3,0x31,0x70,0xb1,0x61,0xe1,0xb4,0x0d,0xc9,0x16, -0x6c,0xed,0x30,0x41,0xed,0xec,0xf0,0x35,0x0b,0xcb,0x34,0x31, -0x6d,0x59,0xdf,0x95,0x5b,0xcd,0x07,0x5b,0x14,0x88,0xbd,0x26, -0xeb,0xb0,0xd5,0xe8,0xe5,0x87,0x9f,0x9a,0x8d,0x07,0x8b,0xc3, -0x5e,0x9a,0x02,0x5c,0x0b,0x00,0x79,0x76,0xb4,0xea,0xe0,0x0f, -0x0f,0x5b,0x00,0xb4,0x85,0x50,0x76,0x18,0x5c,0xad,0xde,0xd9, -0x86,0x23,0x84,0x44,0xbc,0xc5,0x42,0x26,0x9f,0x48,0x61,0x79, -0xf0,0x46,0xe3,0x2b,0x48,0x51,0xdb,0x5e,0xd2,0x06,0x9a,0x88, -0xed,0x2e,0x07,0x5e,0xfb,0xc6,0x3f,0x2d,0xa7,0xd9,0x5b,0xa1, -0x9a,0x8a,0x14,0x52,0x6e,0xe3,0x2a,0xc1,0x06,0x1b,0x57,0xd6, -0xa8,0x1d,0xe4,0x47,0xaf,0x6b,0x61,0xce,0x8e,0x17,0xc9,0xe5, -0x51,0x29,0x08,0x97,0xe6,0xc6,0xb1,0x9c,0x8c,0x8b,0xa3,0xdd, -0x24,0x63,0xf0,0x3b,0xe3,0x07,0xc4,0x9d,0x60,0x06,0xfc,0x5e, -0xb0,0xb5,0xbd,0xcb,0x41,0x4e,0x26,0x63,0x07,0x1f,0xf1,0xe8, -0xc0,0xaa,0x21,0x60,0x4d,0x9f,0x94,0x05,0xa9,0x3d,0x31,0x95, -0xee,0xec,0xbe,0xf6,0x89,0x7b,0x34,0x82,0x80,0xc8,0xfc,0x66, -0x3a,0x2b,0xce,0x0a,0x5a,0x62,0xe6,0xe7,0x19,0xa1,0x99,0x8d, -0xcf,0xc6,0x7f,0x5c,0xd1,0xf3,0x92,0x8d,0xdb,0x01,0xb0,0xf5, -0x51,0xfc,0x90,0xc9,0x97,0x61,0x6e,0x8e,0x7f,0x7c,0x47,0x37, -0xfa,0x8d,0xb4,0x35,0x48,0x91,0x05,0x1c,0x1e,0xe2,0x68,0x57, -0x59,0x87,0x5b,0x69,0x2f,0xc3,0x08,0xbe,0x69,0xa3,0x7f,0xd8, -0x5b,0x1c,0x3e,0x1a,0xa4,0x3d,0x00,0x54,0xb3,0xbf,0xc4,0x31, -0x49,0x06,0x10,0xf6,0x72,0x83,0x37,0x8e,0xa0,0xfb,0x61,0xde, -0x1c,0x4c,0x2c,0x5c,0xd9,0x5c,0x04,0x2d,0x67,0xf2,0xe4,0x51, -0x7d,0x6e,0xd7,0xfb,0xf3,0x93,0x13,0x28,0x0e,0xff,0xed,0xab, -0x41,0xc5,0xef,0xba,0x7d,0xc4,0x1b,0x1d,0x48,0x46,0x4e,0x7f, -0xc9,0xb8,0x3b,0x49,0x9a,0x0d,0x9b,0x8d,0xc6,0xbf,0x94,0x93, -0xc2,0xd4,0xec,0x65,0x3a,0x87,0x92,0x33,0x98,0x2f,0xcd,0x06, -0x12,0x04,0x1b,0x17,0xf4,0x3d,0xd9,0x4b,0x00,0x47,0xa3,0xff, -0xcb,0x16,0x06,0x28,0x86,0xbc,0x24,0x55,0x65,0x24,0xcc,0x87, -0x6e,0xbf,0xc3,0xad,0x07,0xd8,0x32,0xbe,0x05,0xb1,0x8b,0x5a, -0xdc,0x1f,0x79,0xc0,0xe5,0x91,0x63,0xcf,0xcf,0x81,0xa9,0x60, -0xa0,0x15,0xd7,0xad,0xbd,0xa4,0x48,0x90,0x08,0x2c,0xe6,0x95, -0x25,0xb4,0x98,0x90,0x32,0x0c,0x5d,0xa6,0x6a,0x59,0x0b,0x3b, -0x1b,0x21,0x68,0x3f,0xc1,0xb0,0x7d,0x4c,0x1d,0x38,0x0d,0xca, -0x6c,0x8c,0x5e,0xa3,0xdf,0x4d,0xf6,0xc7,0xc3,0xd9,0xc9,0x39, -0x0b,0x03,0x08,0x47,0x2b,0x2b,0xae,0x2b,0xe1,0x08,0x64,0x6c, -0xa9,0xac,0xbb,0x2c,0x79,0x71,0xf0,0x06,0x1c,0x5e,0x39,0xf1, -0x25,0xcc,0xa3,0x5a,0xbc,0x43,0x9b,0x1e,0xbd,0x8a,0xa6,0xa9, -0x6d,0x82,0xcc,0x34,0xe3,0x2f,0xdd,0x4a,0xba,0x42,0xa1,0x46, -0xa9,0x29,0xe5,0xd4,0x83,0xa9,0x8c,0x5c,0x45,0x9b,0x2e,0x61, -0x69,0xb4,0x32,0x3d,0x36,0x8c,0xe8,0x0a,0xa7,0x20,0x27,0xcf, -0x3e,0x15,0x3a,0x6c,0x95,0x88,0x9b,0x37,0x0a,0x98,0x5a,0x91, -0x0c,0x8b,0xdc,0x36,0x50,0xe1,0x78,0x00,0x62,0x08,0x9e,0xd9, -0xcd,0xa6,0x53,0xf6,0xd1,0x8d,0xfa,0x63,0x3c,0xe5,0xa7,0x13, -0x6e,0x83,0x19,0x38,0x9b,0x79,0xd3,0xcb,0xaa,0xe5,0x44,0x82, -0xe3,0x4f,0x29,0xd2,0xef,0xca,0xd0,0x01,0x51,0x50,0x00,0x57, -0x81,0x71,0xe8,0x8d,0x46,0x0e,0x5b,0x01,0xe3,0x10,0x9b,0xfa, -0x18,0x02,0x14,0x23,0xc0,0xb1,0x13,0x56,0x38,0x11,0xb7,0x21, -0xfe,0x00,0xcc,0x06,0x70,0xca,0x00,0xc1,0xbe,0x64,0x5e,0x40, -0xb2,0x12,0x3e,0x2a,0x5d,0x15,0x6a,0x9a,0x38,0x83,0x07,0x6b, -0x6a,0x4f,0x96,0x71,0x0e,0x92,0xf1,0x5e,0xae,0xb3,0x39,0xcc, -0x98,0xb8,0x3b,0x5d,0xbf,0x0c,0xa0,0xbd,0x3f,0x13,0xb2,0x01, -0xca,0x2f,0x95,0xd9,0xcf,0x4e,0xd1,0x6d,0x36,0x0b,0x47,0x8c, -0x5c,0xdc,0x40,0x8f,0x48,0x28,0x31,0x24,0xc8,0x9b,0x27,0x67, -0xb8,0xf4,0x01,0x47,0x9d,0x9e,0xae,0x5d,0xf6,0x8b,0x41,0x6f, -0x34,0xae,0xa8,0xbb,0x31,0x3d,0xcd,0xf0,0x6f,0x97,0x14,0x0a, -0xbf,0xe3,0xc1,0x56,0xc3,0xaf,0x1c,0xad,0x89,0xbd,0x35,0xe2, -0x46,0x09,0x67,0x7a,0xa6,0x7c,0x7d,0x05,0xdb,0xc6,0x31,0x6b, -0x20,0xad,0xbb,0xfb,0x3f,0xe6,0xec,0x06,0x59,0x52,0xc9,0xe0, -0x32,0xe7,0x2e,0x69,0x89,0xd7,0x76,0x72,0x3d,0xc7,0x16,0xb2, -0xec,0x5b,0xac,0xf5,0x80,0x01,0x80,0x7b,0x61,0xa2,0x7d,0xdd, -0xd8,0x90,0xdb,0x6a,0x04,0x83,0x23,0x59,0x3a,0x95,0x80,0xfc, -0x3b,0x88,0x31,0xeb,0x04,0x12,0xce,0x53,0x48,0x85,0x66,0x3c, -0x19,0xbf,0xa7,0x54,0x36,0x42,0xb6,0x2b,0xa7,0x60,0x24,0x6c, -0xb4,0xff,0x55,0x99,0xc1,0x00,0x4a,0x8d,0x1f,0x63,0x2d,0x8c, -0x92,0x3d,0x1c,0xc1,0x2a,0x82,0xfe,0xdd,0x09,0xe8,0x27,0x6a, -0x81,0x84,0x28,0xc2,0xbd,0x34,0xe2,0x5d,0x2c,0xe8,0x0d,0x1a, -0x53,0x55,0x4f,0xe1,0x47,0x6f,0xc3,0xc4,0x4e,0xe7,0xf7,0xd1, -0x02,0x32,0x21,0xd7,0x56,0x8c,0xd3,0x84,0x4d,0x47,0x84,0x1b, -0x6e,0x3f,0xf9,0x64,0x3b,0xe1,0x29,0x3c,0xe0,0x3e,0x75,0x7e, -0x23,0xca,0x4a,0x7b,0x6e,0x76,0x50,0xb3,0xe9,0x03,0xd1,0x18, -0xd6,0xa8,0xa6,0x49,0xd3,0x56,0x49,0x4d,0xd9,0xd0,0x89,0x8e, -0x57,0xa7,0x99,0x47,0x0b,0x92,0x4a,0x8c,0xa9,0xd2,0x6d,0x11, -0x11,0xa9,0x4a,0x90,0x6a,0xd2,0x54,0x61,0xa7,0x7d,0x3c,0xb3, -0x19,0x74,0xa7,0x10,0x29,0x8f,0x52,0xeb,0xfd,0x31,0xf4,0xdc, -0x60,0xb4,0x1b,0x06,0xb1,0xf6,0xf3,0x3b,0x4b,0xa1,0x90,0xf6, -0xe5,0x6e,0x7b,0x6b,0x85,0x31,0x86,0xae,0x15,0xc6,0xbc,0x25, -0xd3,0x20,0x77,0x8d,0xcc,0x82,0x55,0xcf,0xb1,0x17,0xba,0x89, -0xc9,0x89,0x99,0x4f,0x2c,0x26,0xbf,0x4f,0x3f,0x7a,0x3b,0x26, -0xe3,0x05,0x70,0x77,0x8c,0x43,0x54,0xd9,0xb7,0xb4,0xa7,0xf6, -0xf9,0x2e,0x11,0x23,0xc3,0xc6,0xa7,0x3e,0x41,0xa7,0xe0,0xcd, -0xde,0x18,0x31,0x7a,0xb1,0x08,0x70,0xff,0xc4,0x3b,0x99,0x9a, -0x0a,0x24,0x47,0x97,0x54,0xcf,0x61,0x4d,0xba,0x89,0xf3,0x28, -0x80,0x97,0x0e,0xce,0xa1,0x71,0xd7,0x7c,0xb6,0x54,0x49,0xa8, -0x2f,0x28,0x56,0xc3,0x41,0x7d,0xed,0xae,0xbe,0xdc,0x87,0xec, -0xd0,0xd9,0xab,0x42,0x77,0xac,0x52,0x36,0xc6,0xc7,0x30,0xa8, -0xfa,0x02,0x91,0xb3,0x8e,0xc6,0xd4,0x59,0xb1,0xb5,0x1e,0x54, -0xda,0x9f,0xc6,0xe7,0xf0,0xe2,0xa2,0x61,0xaf,0x67,0x55,0xea, -0x94,0x8c,0x1b,0xe2,0x8a,0x11,0x13,0x94,0x4f,0xf7,0x2d,0x40, -0x43,0x55,0xc6,0x20,0xb1,0x24,0xcd,0xa3,0x02,0x96,0x74,0x07, -0x4f,0x12,0x4e,0xc2,0xaa,0x1c,0xcd,0x4b,0xb4,0x1f,0x78,0xd1, -0x29,0xd5,0x53,0x76,0x0f,0xf4,0xa4,0x2b,0x53,0xc8,0x45,0x65, -0x84,0x4e,0x94,0x94,0x64,0xee,0xd7,0x81,0x1b,0x09,0x53,0x8e, -0x1f,0xc3,0x36,0xec,0xd1,0xec,0x63,0xb1,0x46,0xe9,0xff,0x78, -0xc5,0x5d,0x97,0x78,0x9b,0xb2,0x30,0xe5,0x35,0xcb,0x73,0x85, -0xad,0x88,0x6c,0x03,0x85,0xd8,0x44,0xde,0x6e,0x4e,0x58,0x66, -0x93,0xc3,0x9f,0xcc,0x73,0x85,0x2a,0x1b,0x72,0x32,0x8d,0xe7, -0x5e,0x72,0x3c,0x63,0x1d,0x62,0xbb,0x25,0x8b,0x2b,0xb3,0xc9, -0x8b,0x85,0x88,0x67,0xb8,0x8a,0x53,0xdc,0x71,0x55,0xb5,0xa4, -0x3a,0x0a,0x6e,0x96,0x1a,0xff,0xcd,0x52,0x5f,0x40,0xd6,0x1c, -0x9a,0x57,0x74,0x4b,0xd3,0x8d,0x52,0xe5,0xf9,0x9d,0x88,0x6d, -0x46,0x01,0x26,0xb1,0xda,0xc6,0x3b,0xcb,0xf4,0xb6,0xb9,0xe4, -0x0d,0xbd,0x29,0x88,0x81,0x92,0xf3,0xca,0x54,0x14,0x7c,0xe2, -0xd4,0xeb,0x8c,0x2f,0x45,0xd0,0x45,0x47,0xb6,0x1b,0xa2,0x14, -0x6a,0xee,0x5c,0x5e,0xac,0x2f,0x70,0xde,0xcc,0xb4,0x1e,0xa5, -0x5b,0x64,0x26,0x9c,0x6b,0xd8,0x7e,0x07,0xa9,0xf3,0x98,0xd5, -0xa6,0x94,0x34,0xc1,0x71,0x95,0x7b,0xce,0x98,0x1f,0x85,0x77, -0x71,0xcd,0xa1,0x0a,0x71,0xa4,0x5e,0x2a,0xc6,0x91,0x46,0x13, -0x3e,0x8d,0xaa,0xae,0xb2,0xd8,0x2f,0x37,0xdc,0xe4,0x99,0x8c, -0x2b,0x16,0x0d,0x74,0xc4,0x94,0xa0,0x58,0x3b,0x98,0x52,0xe8, -0x56,0xed,0x79,0xf9,0x79,0x72,0xd2,0x50,0x12,0x0e,0xcc,0x6f, -0xdb,0xd8,0x1a,0x96,0x4c,0x74,0xec,0x00,0x8a,0xa6,0x9e,0x2e, -0x2d,0x74,0x2d,0x07,0x95,0x3d,0x2d,0x7d,0xa1,0xda,0xd9,0x7a, -0x09,0x3a,0xb0,0xe4,0x23,0x1b,0xb5,0xa5,0x3a,0xbe,0x9f,0x9c, -0xa1,0x99,0xac,0x55,0xa1,0x32,0xfa,0x95,0x11,0xb1,0xbc,0xf7, -0x1b,0x73,0x9b,0x17,0xc1,0x38,0x30,0x56,0x18,0x6e,0xe1,0xcb, -0xa3,0x64,0xad,0x8a,0xbb,0x10,0xf9,0x9f,0x62,0x23,0x35,0xf7, -0x65,0x2a,0x3e,0x0e,0x7e,0xe2,0xff,0xf1,0x70,0x54,0xb1,0xc4, -0x2b,0xec,0x4c,0x78,0x76,0x0d,0xf7,0x54,0xef,0xc2,0xd9,0xfd, -0x22,0xc6,0xef,0x20,0xfe,0x46,0xeb,0x0a,0xee,0x02,0xdc,0xd2, -0x72,0x4b,0xec,0x28,0x71,0xe2,0x7a,0x9f,0xa8,0x51,0xdc,0x78, -0x86,0x1d,0xa7,0xd2,0x5d,0x0d,0xb5,0x76,0xad,0xf4,0xf4,0xea, -0xc2,0x3d,0xb1,0x97,0x7b,0xa8,0x2e,0xf7,0x71,0x9f,0x08,0x95, -0x0a,0xf7,0x64,0xdb,0x2b,0x16,0xf1,0xd1,0x06,0x5b,0x77,0x86, -0x6d,0x8d,0xf2,0xfe,0x01,0xab,0x5a,0xda,0x1d,0x99,0x6d,0xad, -0x52,0x63,0x23,0x1c,0x1c,0x9b,0xde,0x81,0xe6,0x6a,0xa8,0x37, -0xaa,0x98,0xbc,0xad,0x79,0x25,0xeb,0x79,0x19,0x8d,0xac,0x7e, -0x1d,0x0c,0x1e,0xb4,0x3a,0xb2,0xa4,0x69,0x25,0x40,0xaa,0x8e, -0xd3,0xa9,0xad,0x2a,0xab,0x05,0x7c,0x03,0x7e,0x41,0xd9,0x25, -0x76,0x45,0x14,0x5e,0xab,0x36,0x0e,0x16,0x85,0x44,0x8f,0x82, -0x60,0x45,0x80,0xbf,0x07,0xb1,0xb0,0x4c,0x14,0x5e,0x6b,0x29, -0x57,0xcd,0x2b,0x43,0x69,0x79,0x57,0xac,0xb9,0x95,0x31,0xee, -0xba,0xda,0x01,0x50,0x9d,0x98,0xc2,0x33,0x2b,0x35,0x76,0x86, -0xa1,0xf1,0x22,0xf7,0x6c,0x7c,0x00,0xc2,0x9b,0x42,0x7f,0x50, -0x42,0xaf,0x2b,0x8e,0x4f,0x56,0x1d,0xb6,0x38,0x51,0x2f,0xaf, -0x28,0xc6,0xef,0x38,0x60,0xb1,0xe2,0xbf,0xca,0xe3,0x84,0x6e, -0x64,0xb6,0x68,0x82,0x53,0x1f,0xfb,0x32,0x9f,0xfe,0xbd,0x9c, -0x4e,0x24,0xac,0x26,0x8b,0x31,0x64,0x9a,0x97,0xd7,0x58,0xea, -0x61,0xae,0x37,0xd3,0xbb,0xb2,0x2b,0x84,0xb4,0x5f,0x62,0x42, -0x9d,0x28,0xc6,0x24,0x55,0x08,0x2b,0xf0,0x56,0xdb,0x52,0xb7, -0xea,0x90,0xeb,0x7a,0x55,0x76,0xe4,0x83,0xe0,0xf2,0x2a,0x83, -0x61,0x7b,0xd4,0x6a,0x93,0x86,0xaf,0x2d,0xd9,0xee,0xb0,0xca, -0xe7,0x7a,0x7e,0x57,0x90,0xc7,0x13,0x35,0xc0,0xbd,0x9f,0xdb, -0xc1,0x97,0x2a,0xa3,0x57,0xda,0x49,0x56,0xdb,0xf2,0xc5,0x74, -0xfa,0xb1,0x40,0x5f,0xfc,0x55,0xa3,0xc7,0x13,0x9f,0x15,0xea, -0xab,0x40,0xf2,0xfd,0x54,0x86,0x01,0x72,0x95,0xba,0x49,0x72, -0xb5,0x22,0x1a,0x65,0x19,0xe1,0xdc,0xe8,0xfb,0x85,0xf7,0x8f, -0x02,0x27,0x04,0xc8,0xf0,0x26,0xa3,0x10,0x8a,0xd5,0x6a,0xdd, -0xe7,0x4b,0x7e,0xaa,0x0d,0xc9,0x85,0x32,0x3e,0xa7,0x05,0x71, -0xd6,0xd6,0xd9,0x92,0x0c,0x42,0xa0,0x1a,0x21,0x40,0x2a,0x30, -0x72,0xf7,0x28,0x40,0x65,0xe2,0xa1,0x1b,0x84,0x71,0x5f,0x67, -0x25,0xcc,0x0b,0x5d,0x02,0x97,0xcd,0xda,0xca,0xc5,0x62,0x40, -0xbe,0x0a,0x8f,0xc4,0xb2,0xea,0x49,0x53,0x05,0x5d,0xda,0x0b, -0xe8,0xa9,0xd6,0x97,0x89,0xf4,0x17,0x80,0xe1,0x9a,0x22,0x15, -0xa5,0x37,0xae,0xc6,0x4e,0xd4,0xc2,0x79,0x85,0xd0,0xfa,0x76, -0x09,0x50,0xc3,0x37,0x87,0x58,0xdc,0x67,0xeb,0xfe,0xa8,0x03, -0x49,0x7b,0x75,0xa4,0x66,0xe1,0xf7,0xd5,0x15,0xd4,0x6d,0xc6, -0x9c,0x8e,0x1c,0x96,0x87,0xa3,0x45,0xd0,0xa0,0xf9,0x93,0x6a, -0x23,0xb2,0x86,0xba,0x9e,0xbc,0x44,0x75,0x1f,0x7c,0x7f,0xdd, -0x11,0xb1,0x23,0x5a,0xb1,0x29,0x97,0xe6,0xc4,0x58,0xdb,0x52, -0xc3,0xd1,0xe8,0x47,0x98,0x8e,0xaf,0x26,0xf1,0xad,0x4c,0xe1, -0xee,0xb7,0x34,0x6d,0x95,0xe8,0xf5,0x3c,0x2c,0x65,0xe3,0x66, -0x84,0xbd,0x14,0xc2,0xf0,0x86,0x37,0x4a,0xd3,0xd3,0xa2,0x16, -0xc0,0x76,0xa5,0x6d,0x8c,0x70,0xe3,0xca,0xd7,0x4c,0xd1,0xc6, -0xa5,0x91,0xfa,0xe6,0x0a,0x05,0x7d,0xbe,0x1e,0x80,0x13,0x37, -0xb5,0xd8,0x96,0x35,0xcc,0x61,0x1f,0xde,0xd8,0x28,0xda,0x31, -0x0b,0xed,0x87,0x93,0xa9,0xb0,0xd2,0x34,0x9a,0x1b,0xb8,0x5c, -0xb9,0x5b,0x67,0x03,0xc4,0x59,0x60,0xbe,0x82,0xa8,0x0d,0x5b, -0x7a,0x6e,0x85,0x9a,0x4d,0x2f,0x71,0x89,0xb0,0x9a,0x24,0x36, -0x37,0x35,0x57,0xd7,0x21,0x56,0x1b,0xde,0x29,0xc0,0x98,0x87, -0xab,0x8c,0xf3,0xbe,0x27,0x96,0x0b,0x21,0x55,0x32,0x6e,0x03, -0x04,0x78,0x61,0xe0,0xdf,0x76,0x26,0x78,0x39,0xfc,0x38,0x46, -0x6f,0x46,0xa5,0x8b,0xbf,0x6f,0xa6,0xc7,0xbf,0xe2,0xe0,0x65, -0x5f,0x69,0x25,0xc6,0xc5,0xe5,0x1d,0xaa,0x49,0xe4,0x12,0x7e, -0xde,0xee,0x16,0xcf,0xd8,0xaf,0x99,0xbd,0x77,0x86,0x3e,0x28, -0x8e,0x7f,0xed,0x53,0x62,0xbf,0x18,0xe0,0xb9,0xbb,0x6d,0x39, -0x48,0xf7,0xa2,0x39,0xca,0xb7,0x3f,0xd2,0x35,0xab,0x06,0x3e, -0x1a,0x10,0x2f,0x46,0x17,0x21,0x83,0xc6,0x5c,0xd2,0xb0,0xe2, -0x05,0xf3,0xf9,0xf0,0xcc,0xc0,0x3f,0x72,0xd3,0x85,0x3b,0x20, -0x60,0xa4,0xc3,0xd9,0x67,0x0e,0xeb,0xca,0xc9,0xae,0x39,0x7d, -0xcc,0x43,0xc9,0x91,0xd8,0x0a,0x2f,0xad,0x4f,0x70,0x49,0x1e, -0xd0,0x1d,0xd8,0x6e,0x39,0x1f,0x9e,0x7c,0x6c,0xe1,0xf0,0x45, -0xbf,0x75,0xc5,0x04,0xe4,0xb6,0xb1,0x03,0xd5,0x99,0x83,0x6e, -0x4a,0x34,0xbd,0x9c,0x8c,0x90,0x22,0xf4,0x1e,0xa3,0x72,0xd3, -0xae,0x3d,0x24,0xe3,0xfc,0x4a,0x6d,0x1b,0x1b,0x1a,0x1c,0xcf, -0xba,0xcf,0xf8,0xe6,0x65,0x84,0xd4,0x11,0x4d,0x5f,0x98,0xff, -0x3e,0x2d,0x46,0x15,0x5c,0x8b,0xc5,0xfa,0x3a,0x65,0x33,0xd0, -0x62,0xc1,0xa8,0x69,0x4d,0xb3,0xe5,0x49,0x6a,0x21,0xef,0xa4, -0x6c,0xdd,0x5d,0xce,0x9d,0xb2,0xf6,0xf9,0xc1,0xc1,0x7b,0x71, -0x37,0x10,0x9f,0xd0,0xd3,0x3a,0x31,0x9a,0x5e,0x1f,0x5f,0x8c, -0xff,0xf7,0xf5,0x74,0x4e,0x76,0x00,0xe8,0x05,0x89,0xee,0x84, -0xe9,0x94,0xeb,0xc9,0x6f,0xfe,0x4d,0xcf,0x9a,0x4a,0x61,0xa0, -0x2e,0x2e,0x8d,0x06,0x97,0xbf,0xe9,0x57,0x10,0x39,0x89,0x54, -0x61,0xc1,0x7c,0x00,0xf2,0x72,0x32,0x2f,0xe6,0xc8,0xb8,0x9c, -0x3a,0x56,0x46,0x09,0x07,0xa6,0xd8,0xd8,0x08,0x5e,0xed,0x87, -0x8b,0x3f,0x69,0x1e,0x1f,0x91,0x7a,0x4b,0x5a,0x59,0x0d,0x26, -0x3b,0xd6,0xaf,0xa6,0x25,0x8e,0xe6,0xfb,0x8c,0x26,0x1a,0x36, -0x08,0x2f,0xfd,0x29,0x87,0x7e,0x5d,0x48,0xda,0xc9,0xdb,0xd2, -0xcd,0x7d,0x78,0x83,0x85,0x4a,0xca,0x60,0xde,0xe6,0x26,0x1d, -0x1d,0x31,0x14,0xdf,0x74,0x2d,0x62,0x14,0xc5,0x4e,0x0e,0xf9, -0xdd,0x02,0x60,0xed,0xb7,0x81,0x5c,0xec,0x3f,0x14,0x55,0x60, -0x8c,0x9e,0xfc,0xda,0xab,0xc2,0x58,0x8e,0x17,0x26,0x1b,0xdd, -0x04,0xbe,0x97,0xb5,0x5c,0x3c,0x4c,0xf1,0xae,0x3c,0x9d,0xee, -0xe0,0x74,0xa3,0xe9,0xeb,0x87,0x63,0x55,0x75,0xb1,0xc6,0xb5, -0x84,0xe4,0x0d,0x96,0x5d,0x2a,0x4c,0xb3,0x8e,0x2a,0x40,0x0e, -0x1d,0xce,0x1f,0x8a,0x7a,0x04,0x73,0xa0,0x76,0x02,0xa5,0x58, -0x9c,0x08,0xa8,0xb7,0x25,0x69,0x3d,0x4a,0x9f,0x1d,0xa2,0xff, -0x61,0xb4,0xbf,0x28,0x1f,0x25,0x4d,0x5d,0xba,0x99,0xf4,0x7f, -0xd9,0x19,0x3c,0xda,0x01,0xf6,0x53,0x68,0x15,0xd9,0xf0,0x42, -0x3c,0x38,0xbb,0x03,0x4f,0xd4,0x92,0xce,0xf9,0xec,0x80,0x2b, -0x79,0xf1,0x6e,0x6f,0xef,0xe5,0xdb,0x03,0x3b,0xe4,0x93,0xe0, -0xa0,0xff,0xc5,0xee,0xf3,0x83,0xe7,0x41,0x96,0x1b,0x66,0xf4, -0x8d,0xbe,0xf5,0xe9,0xb5,0x11,0x8d,0x4d,0xaa,0x1a,0x5d,0x7b, -0xb2,0x89,0xed,0x6a,0xf6,0x60,0xef,0x92,0xa3,0x12,0x8f,0x9a, -0x80,0x3a,0xea,0xdd,0x69,0x23,0x79,0xb6,0xbe,0xb9,0x99,0xa4, -0x3d,0x71,0x1c,0x17,0xe6,0x6d,0x6e,0xee,0x00,0x41,0xf4,0x4a, -0x83,0xab,0xe1,0x88,0x99,0x5e,0xb2,0x8a,0x24,0x4a,0x20,0x76, -0x2a,0xe7,0xd2,0x28,0x2c,0x3e,0x91,0x08,0x61,0x68,0x63,0x6c, -0x1b,0xdf,0xe7,0xb2,0x2b,0x82,0xc7,0xa9,0x91,0xee,0xc4,0x2b, -0x13,0xdf,0xbd,0xfc,0xfe,0xf5,0xdb,0xef,0x8f,0x5e,0xbe,0xdd, -0x3d,0x3a,0x78,0xfe,0xfd,0x4b,0x69,0x1a,0xb6,0xae,0xc2,0xe2, -0x40,0x29,0x6b,0x6a,0x09,0x17,0xab,0x16,0x05,0xd8,0xfa,0x30, -0x31,0x56,0x0f,0x5c,0x57,0x27,0xfb,0xce,0x68,0x0f,0xdc,0x15, -0x5c,0x97,0x60,0xbb,0x24,0xc4,0xa6,0x9b,0xb5,0x4a,0xa8,0x02, -0x54,0x24,0x3a,0x6d,0xb2,0xa2,0x71,0xff,0xe0,0xf9,0xfb,0x83, -0x7f,0x23,0x95,0x31,0x3e,0x13,0xac,0x58,0x9a,0x52,0x7e,0x84, -0x69,0x53,0xd3,0xc1,0xcf,0x60,0x31,0xfd,0x5f,0x7c,0x26,0x06, -0xc3,0xb6,0xcd,0x2e,0x22,0x7a,0x08,0x93,0x45,0x14,0xb5,0xa5, -0x27,0x69,0x12,0x3a,0xc8,0x24,0x89,0xe1,0x24,0x22,0xdc,0x97, -0x0f,0x62,0xbc,0x45,0x41,0xb8,0x49,0xc4,0xb7,0x46,0xd8,0x25, -0xec,0xf8,0xe6,0xc5,0x9f,0xda,0x08,0xdb,0x5e,0x32,0x3b,0x38, -0x07,0x41,0x42,0xb2,0x67,0x6b,0x9f,0x86,0xe5,0x1a,0x70,0xdd, -0xe3,0x0b,0x32,0xb6,0x26,0x98,0xb5,0x39,0x40,0x9c,0x4e,0x31, -0x5e,0x1a,0x50,0xb6,0x46,0xcb,0x30,0x3a,0xa8,0xa1,0xcf,0x43, -0x23,0xec,0x84,0x46,0x25,0x0b,0xa6,0xd4,0x30,0x4b,0x3d,0x87, -0x52,0x7d,0x65,0xa7,0x66,0x45,0x70,0xe1,0x32,0x8b,0xd1,0x68, -0x3c,0xf9,0x71,0x36,0x86,0x66,0x9d,0x8c,0x67,0x3f,0x1c,0xec, -0xbd,0x11,0x2b,0x74,0x67,0xa8,0xf2,0x0c,0x0d,0x54,0x36,0x2e, -0xe6,0x5d,0x9c,0xde,0x21,0xf8,0x01,0x9d,0x02,0xfb,0x44,0x6c, -0x0c,0x51,0xc8,0xe0,0xc2,0x14,0x5f,0x58,0x5b,0x41,0x43,0x54, -0xdf,0x06,0xd5,0x37,0xbc,0xbc,0xea,0x2a,0x56,0xf3,0xf6,0xdd, -0xdb,0xa3,0xe7,0x6f,0x7e,0xfc,0xe1,0xf9,0xdb,0x9f,0xf6,0x5e, -0xbe,0x7f,0xfd,0xa2,0xb2,0x06,0x07,0x28,0x93,0x8d,0xff,0x90, -0x18,0x23,0xd4,0x69,0x2f,0xa0,0x6e,0x32,0x8b,0x6b,0x26,0xdd, -0x44,0xdb,0xe0,0xa8,0x4f,0xf3,0x89,0x3b,0x94,0x78,0x86,0x89, -0xa1,0xd0,0xb5,0x2f,0xfd,0xf5,0x61,0x56,0xa0,0x26,0xf1,0xf8, -0xfa,0x54,0x96,0xc0,0xb3,0x09,0x88,0xcd,0xe4,0x07,0xe1,0x7a, -0xee,0x56,0x3f,0x34,0x14,0x46,0x18,0x03,0xff,0x48,0xbc,0x70, -0xf1,0x7e,0x69,0xcd,0xf5,0x42,0x2a,0x2e,0xa9,0x7a,0xe1,0xbd, -0x81,0x84,0xfa,0x35,0x14,0xa3,0xcc,0x53,0x5d,0xd5,0xc5,0x02, -0x72,0x69,0x17,0xc7,0xa4,0x10,0x2c,0x3f,0x2f,0x16,0x9d,0x75, -0xec,0xd1,0x50,0x14,0x42,0x13,0x3b,0x20,0x96,0xe6,0x10,0x52, -0xcd,0xd8,0x23,0xe5,0x11,0x87,0x4d,0xaf,0x3d,0x27,0xa6,0x6d, -0x1c,0x9e,0x83,0x54,0xe9,0x24,0x4f,0x25,0xb6,0xce,0xe7,0x24, -0x99,0x5c,0xb0,0x51,0xdf,0xf5,0xac,0x50,0xef,0xc0,0xcb,0xd7, -0x81,0x2b,0xaf,0x73,0x37,0x31,0x9f,0xf9,0xe9,0xfd,0x6b,0xcb, -0x61,0x2c,0x81,0x6b,0x42,0x20,0xf9,0xd4,0xc0,0x87,0x87,0x79, -0xf2,0x90,0x9f,0x6a,0x07,0x95,0x00,0x01,0x0c,0x4a,0x3d,0xf8, -0x4c,0xad,0xda,0x4b,0xb6,0x76,0x92,0x2c,0x81,0x15,0x01,0xb6, -0x09,0x20,0x0b,0x04,0xcd,0x7f,0x6b,0x93,0xdf,0xaf,0x19,0xb7, -0x74,0x3b,0x52,0x12,0x56,0x85,0xa2,0x52,0x9e,0xdb,0x3e,0x6b, -0xb8,0x81,0x02,0x34,0x10,0x33,0x51,0xc6,0xc6,0xc2,0xe6,0xa4, -0xb2,0x9a,0xcf,0x63,0x88,0x54,0xb4,0x85,0x11,0x6f,0xa9,0xbb, -0x1d,0x6b,0x21,0x30,0x54,0x85,0x7d,0xf6,0xe2,0x0a,0x31,0x0f, -0xea,0x47,0x18,0x9f,0xea,0x7c,0xad,0xb2,0xbb,0xa8,0x1f,0xf8, -0x29,0x0d,0x6b,0x77,0x3c,0x65,0x62,0x8e,0x9e,0x6f,0xfd,0xf2, -0x8c,0x43,0x24,0x7c,0xca,0x36,0x07,0xcd,0xb4,0xc1,0x71,0x10, -0xe4,0x95,0x5e,0x1e,0xe5,0x08,0xd0,0xcb,0xd0,0xb6,0xae,0xff, -0x4b,0x32,0x78,0x94,0x50,0xc8,0x85,0x87,0xfd,0x5f,0x1e,0x0e, -0x1e,0x3d,0x4c,0x17,0x20,0xb3,0x1c,0x96,0x50,0x34,0xed,0xa5, -0x8f,0x38,0xde,0xf3,0x16,0xc5,0x7d,0xde,0xd9,0x32,0xe1,0x2a, -0x27,0x75,0x81,0xe4,0xa3,0x2a,0x64,0x89,0x67,0xcb,0x28,0x81, -0x3d,0xdf,0xf2,0xe4,0xc4,0x04,0x20,0x7d,0x48,0x04,0x54,0x25, -0x64,0x70,0xca,0x43,0x4c,0x01,0x62,0x1a,0x8e,0x1a,0xb2,0x64, -0x8c,0xd7,0x4f,0x24,0x61,0xcb,0xd4,0x2f,0xff,0x8e,0xbc,0x2d, -0x13,0x4a,0x53,0xf9,0x16,0x8a,0x2f,0x14,0x0d,0x02,0x64,0x15, -0xc0,0xaa,0xe5,0x29,0xcc,0x3c,0xec,0x53,0xca,0x61,0x9f,0x60, -0x06,0x03,0x84,0xf1,0x53,0x04,0xaf,0x28,0x36,0x4e,0xe7,0x57, -0x0b,0x34,0x16,0x28,0x7b,0x69,0x06,0x0d,0xb0,0xb5,0xb8,0x1c, -0x16,0x17,0xf3,0x69,0xb6,0x98,0x8f,0x2f,0xb2,0xc5,0x7f,0xa4, -0x5b,0x85,0x59,0xc1,0x3a,0xb1,0x35,0x7e,0x39,0xfc,0x8f,0xcd, -0xff,0x67,0xb1,0xb6,0x58,0x27,0x2b,0x44,0xbd,0x5f,0xca,0xed, -0xa6,0x36,0x19,0xce,0xc6,0x43,0x73,0x3c,0x43,0x87,0x53,0xe6, -0x7c,0x66,0x8a,0xcb,0x33,0xf3,0xe9,0x78,0x96,0xa4,0x26,0xdc, -0xb4,0x7d,0xa7,0xf7,0x89,0xbe,0x34,0x94,0x22,0x8f,0x2d,0x66, -0x34,0x32,0xa3,0xb9,0xb9,0x80,0xed,0x82,0x99,0x1f,0x4f,0x47, -0x9f,0xcd,0x7c,0x64,0xe6,0xa7,0xd3,0xe9,0xdc,0xcc,0xcf,0xe1, -0xff,0xe3,0x21,0xbc,0x56,0xb1,0xbe,0x0e,0xf6,0x94,0x1e,0xed, -0xec,0xca,0xcc,0xe6,0x15,0x68,0x07,0x17,0x2a,0xc7,0xf1,0xe8, -0xfb,0x36,0xb4,0xb7,0x7d,0x49,0xb4,0x01,0xbe,0x13,0x73,0x50, -0xd8,0xf8,0x46,0x1c,0x8d,0xd0,0xd9,0x9c,0x41,0xb7,0x6f,0x27, -0x17,0xb0,0xd1,0x2a,0x31,0xc8,0x1c,0xa1,0xa6,0xcd,0x9c,0x39, -0x19,0xb2,0x13,0x98,0x13,0x3a,0xa5,0xc2,0x43,0x61,0x33,0x2a, -0xe0,0xb7,0xf8,0xdd,0x8c,0x2e,0x0c,0xdd,0xcf,0x1a,0xe3,0x8f, -0x85,0xc3,0x96,0x03,0xb8,0xf3,0x8e,0x39,0xdf,0x36,0xe7,0x8f, -0xcd,0xf9,0x13,0x73,0xfe,0xb5,0x39,0x7f,0x6a,0xf8,0x0a,0x92, -0x39,0xe7,0x56,0xc7,0x0e,0x9b,0x20,0x1d,0x57,0x06,0x28,0xba, -0x86,0x3d,0xea,0xef,0x06,0xfa,0xf1,0x0a,0x90,0x71,0x88,0x1a, -0x53,0xb2,0x4b,0x23,0xe0,0x54,0x20,0x7d,0x98,0xeb,0x0b,0xe4, -0x5a,0xe1,0x4e,0xfe,0x4b,0x5b,0xd3,0x7f,0xb4,0x19,0x1e,0xc3, -0xc0,0x19,0x9e,0xcc,0xa6,0x93,0xcf,0x97,0xe6,0xd8,0x1c,0x8f, -0x0a,0xf8,0x37,0x35,0xc7,0xc5,0x19,0x8d,0xa8,0x02,0x3f,0x1c, -0xc3,0x40,0xd3,0xe7,0x9e,0x4e,0xcc,0xf8,0x12,0x3e,0x0c,0xcf, -0xec,0x69,0x98,0x21,0xe5,0x1f,0x8f,0x47,0xe2,0xd9,0x9b,0xbe, -0x61,0x38,0xfb,0x68,0x7e,0x33,0xb3,0xeb,0xe3,0xcf,0x86,0x86, -0x80,0x29,0x4d,0x09,0x92,0x82,0x29,0x2f,0x71,0x2f,0x53,0x5e, -0x0d,0x27,0x06,0x45,0xb9,0x8f,0x63,0xfc,0x99,0x4e,0xce,0x0c, -0xc8,0x76,0xf0,0x0f,0xc6,0x5c,0x01,0xdb,0xdb,0xf9,0xdc,0x5c, -0x1b,0x60,0x79,0xf8,0x8d,0xd1,0x02,0xea,0x87,0x96,0x6d,0x17, -0xf4,0x5d,0xc0,0x8e,0xf1,0xd4,0xac,0xa8,0x36,0x85,0x9e,0x34, -0xe1,0x58,0x89,0x9a,0x71,0xc5,0x58,0x45,0x93,0x7b,0x5e,0x16, -0x3d,0x09,0xc7,0xb0,0xf9,0xc1,0xfe,0xc3,0x6b,0x31,0xd8,0x46, -0xe7,0xb3,0xf1,0xa9,0xb9,0x80,0xcf,0x19,0x8d,0xcb,0x13,0x53, -0xce,0x4e,0xcc,0x75,0x39,0x86,0xf6,0xb0,0xe4,0x71,0xf1,0x2a, -0x6d,0x16,0xb3,0xea,0x12,0xea,0x90,0x0b,0x58,0x66,0xe0,0xef, -0xdc,0x0c,0xff,0x28,0x80,0xe8,0x33,0x98,0xa4,0xd3,0x99,0x39, -0x9e,0x52,0x90,0xe0,0x93,0xf1,0xc5,0xc5,0x15,0xde,0x45,0x82, -0xd6,0xc3,0x67,0x68,0xd4,0x13,0x7a,0x46,0xbf,0x79,0x06,0x4f, -0xca,0x88,0x17,0x4c,0xe9,0x6f,0x49,0x7f,0xb0,0xd9,0xf1,0xd4, -0x63,0x78,0x32,0x47,0xed,0xe2,0x6c,0x54,0xd2,0xe0,0x3d,0x05, -0x71,0x4a,0xc6,0x63,0x09,0xbf,0xc5,0xd9,0xf9,0x9c,0x3e,0xe5, -0x02,0x28,0x35,0xe7,0x88,0x77,0x6c,0x8a,0x12,0xfb,0x95,0x52, -0xf0,0xcf,0xf5,0xf0,0x6c,0x6c,0x26,0x53,0xfa,0xe0,0xc9,0xf4, -0xd3,0x6c,0x88,0x7e,0x6c,0x2e,0xe0,0xdf,0xef,0x06,0xd5,0xaa, -0xf4,0x07,0x6b,0x9b,0x5d,0x63,0x88,0x42,0x71,0x5a,0x4f,0x27, -0x2f,0x48,0x62,0x79,0x3e,0xc4,0x77,0x1e,0x05,0x43,0x1c,0x1d, -0xd7,0x97,0x97,0xa8,0xea,0x81,0x97,0xb3,0x31,0x70,0x9a,0x62, -0x0e,0x63,0x1c,0xcd,0x45,0xa9,0xdd,0xa0,0x11,0x58,0xa4,0xf9, -0x9d,0x69,0xf9,0x54,0x8c,0xe6,0xe7,0x38,0x36,0x9c,0x40,0x9b, -0xaf,0x72,0x44,0x05,0x13,0xa8,0xd6,0xe9,0x84,0x5d,0x29,0x13, -0xd3,0x1f,0xa4,0xec,0x9f,0x44,0xaf,0xc4,0x7e,0x21,0xf6,0xc2, -0x56,0x4d,0x61,0xf4,0xff,0x71,0x81,0x2b,0x6d,0x82,0x27,0xa1, -0xa1,0x43,0x14,0x5c,0xb1,0xdf,0xbc,0x7e,0xfb,0x7f,0xff,0xf3, -0xe8,0xa7,0xf7,0x6f,0x3c,0x2b,0xaf,0x2e,0x04,0x0d,0x59,0x09, -0xd2,0x5e,0xff,0xf9,0xe6,0xcf,0xc3,0xcd,0x3f,0xdb,0x9b,0xdf, -0xb6,0x8e,0xbe,0x6a,0xc2,0xf2,0xf7,0x9f,0xe9,0xe1,0xfe,0x23, -0x60,0xfe,0xe5,0x61,0xeb,0xb0,0x7b,0x68,0x0e,0x1b,0x87,0xe9, -0xe1,0xcd,0xe1,0xf2,0xf0,0xd9,0xe1,0xce,0x60,0xcb,0xec,0x3d, -0x7f,0xfd,0xe6,0xe0,0x9d,0x5f,0x64,0x04,0xcf,0x56,0x37,0xb6, -0x6d,0x40,0x69,0x5f,0x9a,0x96,0xf4,0x1a,0xeb,0xb4,0x17,0x52, -0xaa,0x84,0xae,0x73,0xe4,0x89,0xf7,0x4a,0x60,0x56,0xcf,0x86, -0x9f,0x48,0xbd,0xc0,0x1b,0xea,0xfe,0xc0,0x7c,0x22,0x89,0x22, -0xaf,0x11,0x32,0x48,0x3e,0xc1,0x8b,0xb1,0x78,0x13,0x88,0xcf, -0x02,0x48,0xf5,0x52,0x3d,0x75,0x11,0x0a,0xd0,0x74,0xca,0x41, -0xb7,0x38,0x31,0x97,0xbc,0x2e,0xef,0x68,0xa1,0x7a,0x11,0x34, -0xe3,0x16,0x4c,0xbb,0x29,0xde,0x64,0xb1,0x1b,0x55,0x63,0xfd, -0x5f,0xe6,0x2e,0x54,0x3e,0x5f,0x8c,0xca,0x13,0x69,0x0d,0xba, -0xb7,0x83,0xfe,0xc5,0xd8,0x48,0x8a,0x35,0x47,0xfc,0x35,0xb2, -0x4f,0xc4,0xca,0x9c,0x8d,0x7b,0x91,0xea,0x6f,0x69,0xe1,0xf8, -0xc6,0x6b,0x33,0xb6,0x04,0x6b,0xe1,0x92,0x61,0xa2,0x80,0xd2, -0x10,0x5d,0x65,0x0f,0x1d,0x74,0x13,0xde,0xdd,0x71,0x05,0x70, -0xf6,0x03,0xae,0x94,0x9a,0xdb,0x93,0x41,0x1b,0xdc,0x66,0xbc, -0x39,0xb7,0xbd,0x1a,0xd3,0xce,0x4a,0x0f,0x2f,0xe1,0x2d,0xef, -0xa1,0x2e,0x17,0x93,0x2d,0x58,0x9a,0xf8,0x2e,0x9c,0x0d,0x9a, -0x8b,0x07,0x32,0x33,0x98,0xac,0x33,0xb4,0x22,0x9b,0xce,0x5b, -0x47,0x86,0xee,0xfa,0x43,0xc7,0x03,0x93,0xa2,0x7b,0x87,0x3f, -0xe2,0xcd,0x98,0xfc,0x39,0x87,0xb3,0xb0,0xb7,0x64,0xcc,0xbb, -0xe3,0x5f,0x39,0xe3,0x1d,0x5d,0x10,0x56,0x39,0x78,0x4a,0xcd, -0x59,0xf6,0xbc,0x5a,0x65,0xe2,0x66,0x2b,0xf7,0x68,0x69,0xf3, -0x65,0xca,0x8b,0xe2,0x64,0xac,0x53,0x29,0x01,0xcd,0x0e,0x4e, -0x86,0x73,0x9d,0xce,0x29,0x66,0x3e,0x95,0xe3,0x6f,0x4b,0x44, -0xcb,0xa6,0x98,0xf0,0x92,0x9a,0x07,0x08,0xd3,0x61,0xad,0xc5, -0x83,0xe5,0x57,0x72,0x30,0xaf,0x6a,0xb0,0x67,0xf3,0x0c,0x80, -0x61,0xb3,0x54,0x26,0xb2,0x43,0xce,0x78,0x3f,0x1e,0x5d,0x87, -0x24,0xcf,0x28,0x25,0xc8,0x7e,0x8f,0x5c,0xb5,0x0a,0x43,0xc9, -0x96,0x02,0xe2,0x25,0x01,0x01,0x94,0x22,0xd9,0x2f,0x7f,0x1f, -0xcf,0x3e,0xeb,0xdc,0x31,0x26,0x48,0xe6,0xfe,0xf4,0x32,0x6c, -0x34,0x78,0x97,0xac,0xd7,0xac,0x86,0xd1,0xb9,0xa2,0x99,0x11, -0x00,0xb4,0xa6,0xa8,0x01,0xba,0xf0,0xc9,0x16,0x13,0xdf,0x3d, -0x95,0xee,0x97,0x9b,0xa8,0x92,0x87,0xa1,0x35,0x6c,0xf7,0xd3, -0x2d,0x04,0x4e,0xfe,0x0e,0x6f,0xe1,0xba,0x41,0x40,0x7b,0x6d, -0x73,0x14,0x5c,0x06,0xf5,0x31,0x9b,0x8e,0x7f,0xd5,0x3e,0x1b, -0x8e,0x7a,0x90,0x90,0xe1,0xf0,0x0c,0x53,0x1b,0x64,0x8d,0x76, -0x84,0x4b,0xce,0x15,0x7a,0xa8,0x3e,0xfe,0x95,0x96,0xf6,0xb5, -0x76,0x4a,0x1e,0x1b,0x8e,0xe4,0x0a,0x69,0xe2,0x4e,0xa1,0x93, -0x75,0x7b,0xf1,0x60,0xfc,0x07,0x3a,0x43,0x2a,0x7b,0x8d,0xba, -0x4c,0x66,0xef,0x1b,0x1b,0xfc,0xdb,0x12,0x58,0xbc,0xd0,0xcc, -0x4f,0x79,0x98,0x91,0x1f,0xe1,0x95,0x30,0x7a,0x6c,0x1d,0xc1, -0x4b,0xc6,0x13,0x26,0x3f,0x32,0x47,0xad,0x7f,0xbc,0x7c,0xbf, -0xff,0xfa,0xdd,0xdb,0x3c,0xe9,0xb4,0xbe,0x6e,0x6d,0xb3,0x97, -0xa4,0x31,0x8e,0xad,0xa3,0x96,0xfc,0x58,0x2b,0x90,0xe0,0x56, -0x2c,0xce,0x6b,0x74,0x89,0x47,0x26,0x36,0xa4,0xed,0x45,0x83, -0x72,0xba,0x0e,0x89,0xdf,0x84,0x2f,0x7a,0xa4,0x6e,0x6c,0x40, -0xaa,0xc3,0x94,0xe7,0x41,0x66,0xaa,0xf2,0x1a,0x15,0xc4,0x4e, -0x51,0x8b,0x50,0xa2,0x5c,0xcf,0xf3,0xa6,0x7f,0x0b,0xe3,0xd7, -0x5a,0x9f,0xaf,0x2e,0xdb,0xf9,0x7a,0xb5,0xfe,0x17,0x6d,0x0d, -0xad,0x13,0x34,0x3f,0x96,0x6a,0x0c,0x9d,0x92,0x0d,0x0c,0x9d, -0xb2,0x61,0x0c,0x09,0x61,0x26,0xb2,0xf2,0xb0,0x45,0x8f,0xad, -0x87,0x2e,0xb1,0x1c,0xd1,0xd0,0xa1,0x1e,0x34,0xaa,0xe2,0x8f, -0x35,0xf1,0xee,0xef,0xae,0x99,0x6f,0xa1,0x0c,0x38,0x10,0x0c, -0xd2,0x51,0x4f,0x45,0xf7,0x08,0xa7,0x32,0x54,0x2d,0x6e,0x43, -0xef,0xea,0x13,0x6f,0xec,0xaa,0xef,0xdd,0x71,0xb8,0x07,0x28, -0xd1,0x93,0xbc,0xcc,0x31,0x0d,0xee,0x27,0xac,0xc3,0xf6,0x11, -0x24,0xf6,0x24,0xad,0xda,0x37,0x59,0x63,0x4c,0xf1,0x9d,0xa1, -0xf2,0x48,0xa1,0x23,0x31,0x29,0x8a,0x92,0xce,0x01,0xa8,0x16, -0x3e,0x0e,0xab,0x6f,0x83,0x4a,0xa1,0xd4,0x99,0xab,0x96,0x30, -0x41,0xe4,0x16,0x18,0xf0,0x20,0xb2,0x9c,0xce,0x13,0xe6,0x53, -0xa8,0xba,0x24,0xb7,0x99,0x1c,0x03,0x69,0xed,0x53,0x31,0x3f, -0x5f,0x9b,0x4c,0xd7,0xd0,0x21,0x26,0x88,0xe1,0xac,0x03,0x4c, -0xa0,0xd1,0xb8,0x2c,0x8d,0xe5,0x8b,0xd1,0x05,0xfc,0x8a,0xc5, -0x4d,0x7d,0xf3,0x5d,0x8e,0x2f,0xa7,0x61,0x1b,0x0a,0xc2,0x3c, -0x76,0xe8,0xb2,0xb3,0xdd,0x95,0x61,0x4f,0xed,0x09,0x13,0x10, -0xcf,0x68,0x41,0x4a,0x0b,0xd8,0x29,0x37,0xaa,0xd0,0x60,0xdb, -0x95,0xb3,0xac,0x60,0x23,0xb5,0xa1,0x3a,0x47,0xa8,0x00,0x22, -0x49,0xd9,0x57,0x69,0x73,0xdc,0x49,0x11,0x35,0x3d,0x8f,0xb5, -0x11,0xd0,0x9e,0x66,0x35,0x39,0x74,0xd8,0x75,0x9f,0xde,0xb2, -0xd8,0x11,0x53,0x5e,0xdf,0x5b,0xd4,0x40,0x95,0x92,0x59,0x83, -0x8a,0xd8,0x74,0x6e,0x31,0xbe,0xe6,0xbd,0x2e,0xaf,0xa2,0xc3, -0x46,0xde,0x77,0x00,0x9c,0x8c,0x3a,0xb3,0xa1,0x3a,0xd6,0x09, -0x0e,0x88,0x69,0x69,0x8e,0xf4,0xc2,0x23,0xbd,0x37,0xfb,0x1f, -0xea,0x34,0xaa,0x53,0xf7,0x1c,0x13,0x11,0x76,0x1f,0xa5,0xfd, -0x9b,0xfa,0x90,0x70,0xad,0xee,0xc8,0x30,0x5b,0xac,0x4b,0x2b, -0xbc,0x0e,0x1d,0x1a,0xd3,0xd3,0x3a,0xb0,0x48,0xcb,0x1e,0xeb, -0x58,0x56,0x4d,0x34,0xa4,0xe5,0xbd,0x07,0x08,0x9e,0x9a,0x60, -0x41,0x8e,0x42,0xb3,0xb9,0xc9,0xe5,0x07,0x99,0x7d,0x32,0xf7, -0x1d,0x43,0xc4,0x74,0x95,0xcf,0xee,0x60,0x20,0x05,0x99,0xff, -0x96,0xd1,0x84,0x0e,0x10,0xa1,0x11,0x46,0xe3,0xf9,0x17,0xf1, -0x4e,0xe7,0x1a,0x6c,0xf2,0xf9,0x6e,0x56,0xc7,0x2e,0x4f,0xee, -0xc7,0xe4,0x7a,0x0d,0xb9,0x89,0xc0,0x39,0x68,0xff,0xc5,0xe2, -0x81,0x63,0x7e,0x4c,0x35,0x89,0x5a,0x47,0xe2,0x6e,0xfc,0xdf, -0xcb,0xf3,0x59,0x8e,0x93,0xe5,0x99,0x2b,0x72,0xab,0x33,0xbd, -0xf6,0x7c,0xce,0x5f,0xe5,0xff,0xf7,0x6c,0x8d,0x8d,0x8d,0x60, -0xa1,0x70,0x46,0x06,0x6e,0x19,0x20,0x7e,0x70,0x0b,0xdf,0x76, -0x4d,0x20,0xdf,0x6c,0x9b,0xee,0x7e,0x9d,0xb6,0x7e,0x4f,0x32, -0x97,0xae,0x1e,0x24,0x88,0xe4,0x5a,0xe8,0x1b,0xf4,0xe4,0x75, -0x97,0x80,0x24,0x29,0x8b,0x85,0x66,0x10,0xc5,0x68,0x8c,0xca, -0xf8,0xcf,0x81,0x93,0x19,0x6f,0xa4,0x13,0xf7,0x5b,0xa6,0xe4, -0x6b,0xee,0x35,0xa6,0xc0,0x76,0x1a,0x65,0xf4,0x5c,0xfa,0x5f, -0x5f,0xb2,0xb1,0x76,0x3b,0x3c,0xf9,0x67,0x63,0xe3,0xbe,0x6b, -0x77,0x8f,0x47,0x71,0x26,0xd2,0x0b,0x4e,0xd9,0x75,0xc6,0x21, -0x6b,0x39,0xcc,0x24,0x1c,0xcf,0xb8,0x0f,0x80,0xa6,0x83,0x97, -0x7f,0x63,0xd3,0x75,0xee,0x68,0x3a,0xdc,0x7d,0x70,0xcb,0x51, -0xfd,0xb6,0xe1,0x30,0xb9,0x67,0x53,0xff,0xb5,0x66,0x13,0x52, -0x80,0x58,0xa1,0xe9,0xde,0xed,0x26,0x0d,0xe6,0xb9,0x80,0x6a, -0xb7,0xa3,0x96,0xf8,0xcb,0x2a,0x49,0x7e,0xe1,0x30,0xe5,0x41, -0xbb,0x59,0xad,0x49,0xe5,0xfb,0xd7,0x3b,0x59,0xb0,0xbd,0xe2, -0xcf,0x97,0x8d,0x95,0x6b,0x01,0xc9,0xec,0x6d,0x76,0x48,0x86, -0x77,0x47,0xe2,0x82,0x36,0xab,0x67,0x7f,0xe1,0x81,0x2b,0x06, -0x5f,0x23,0xf0,0x25,0xcd,0x0e,0xbe,0xbe,0x19,0x52,0x29,0x1e, -0x0e,0x25,0x74,0xe5,0x59,0x99,0xd3,0x9e,0x99,0x9b,0xc6,0x2d, -0xd5,0x66,0x3b,0x15,0x4b,0x72,0xfc,0x58,0x7f,0xe3,0xd2,0xba, -0x47,0x74,0x33,0x1c,0xc5,0xd2,0xd5,0x44,0xc9,0x65,0xcd,0x9e, -0x78,0x16,0xa2,0xbc,0x3e,0xbf,0x0c,0xd2,0x16,0xdb,0x38,0x72, -0x37,0x20,0x25,0x29,0x13,0x7d,0x75,0x71,0x7d,0xf2,0x31,0xa4, -0x99,0x8e,0x27,0xef,0x59,0x27,0x37,0x04,0x1b,0x04,0x12,0xbe, -0x4f,0xe7,0xe3,0x59,0xd4,0x06,0x72,0x06,0x5a,0xcc,0x4a,0xcd, -0xad,0x8a,0xf2,0x25,0x0a,0xb3,0x7c,0x42,0x9a,0xf6,0x28,0xdb, -0x4e,0xa5,0xfe,0x20,0x3b,0xea,0x73,0x4a,0x82,0xcb,0x58,0x92, -0x25,0xcc,0xdc,0x92,0x41,0x2d,0x2d,0x6a,0xab,0x82,0x76,0xbb, -0x8c,0x12,0x3d,0xbe,0xd1,0x59,0x29,0x12,0xb7,0x2e,0x9e,0x3b, -0xf9,0x4a,0x99,0xf0,0x40,0x3b,0x79,0xd0,0x91,0xb1,0x5d,0x31, -0x3f,0xac,0xa0,0x5f,0x51,0x4e,0x9f,0xa8,0xbe,0x8c,0xa2,0x1f, -0x63,0x43,0xfd,0x71,0x8f,0x5d,0xa3,0x63,0xbc,0x1b,0x1b,0x47, -0xce,0x99,0x1b,0xca,0x28,0x34,0x4c,0xfb,0xed,0x81,0x6c,0xf7, -0xe0,0x89,0x07,0xae,0x84,0x54,0x7b,0xfa,0xf5,0xd7,0x8f,0xbf, -0xb6,0xc2,0xd7,0x1e,0xfa,0x7b,0x80,0xfa,0xa4,0x53,0xf7,0xc8, -0x97,0x09,0xca,0x39,0xd5,0x0a,0xb8,0x91,0x31,0x53,0x0a,0x6f, -0x76,0xb6,0xda,0x9a,0x85,0xdc,0xa0,0xda,0xf9,0x1a,0xe3,0x29, -0x61,0x0e,0xcf,0x53,0x7a,0x74,0x57,0xd8,0xee,0xc3,0x0b,0xf8, -0xfa,0x05,0x23,0x72,0x1c,0xa0,0x77,0x4f,0x56,0xc0,0x43,0xb5, -0x6b,0xcb,0xef,0xc8,0xcd,0x3d,0xfb,0x0e,0x32,0xa6,0xa5,0x95, -0xa9,0x63,0x04,0x8e,0x6e,0xfb,0xb0,0xf4,0xeb,0x67,0x4b,0x4c, -0x61,0xa1,0x57,0x8a,0xc9,0xff,0x6c,0xaf,0x14,0x93,0xbf,0xd2, -0x2b,0x6b,0x2b,0x7b,0xc5,0x77,0xca,0xff,0xc9,0x3e,0x79,0xf6, -0xef,0xec,0x93,0xf2,0xfc,0xfa,0xf4,0xf4,0x62,0x1c,0x29,0x9b, -0xe8,0xeb,0x87,0x93,0x91,0x8b,0x3e,0x27,0x60,0x23,0x25,0xd8, -0xad,0xf8,0x6c,0x98,0x9c,0x43,0x12,0x76,0xf1,0x67,0x7a,0x69, -0x03,0x74,0xa5,0x0e,0x05,0x8b,0xd6,0x9b,0x9d,0x41,0xee,0x52, -0x10,0x74,0x60,0xc2,0xd7,0xdc,0x3a,0xec,0xb5,0xc9,0x44,0xed, -0xf0,0xf2,0x2a,0x22,0xd6,0x4c,0xcc,0xd9,0xf5,0x70,0x36,0x72, -0x4c,0x21,0xde,0x75,0x3d,0xdb,0x5e,0x2c,0x08,0x02,0x17,0xd9, -0xbe,0xa3,0xcb,0x8f,0x9c,0xcd,0x4e,0x0a,0x3c,0xce,0xb6,0x04, -0x35,0x00,0xab,0x53,0x1b,0x6d,0x63,0x67,0x37,0x3c,0x4e,0x52, -0x91,0x21,0x2e,0xa6,0xd3,0x8f,0xd7,0x57,0xaf,0xad,0x38,0xb0, -0x82,0x19,0x07,0xab,0x87,0xbe,0xa8,0x91,0xad,0xd2,0xeb,0x49, -0x10,0xfb,0x25,0xae,0xb8,0x18,0x89,0xf7,0xbb,0x48,0x3a,0xb1, -0xdd,0xaa,0x77,0x9a,0x96,0x88,0x90,0x26,0xa9,0xcf,0x2f,0x56, -0xb4,0xb2,0x34,0x56,0x2d,0x20,0x35,0x92,0x44,0x30,0x92,0x28, -0x5b,0x82,0x7d,0x9e,0x90,0x62,0xbd,0x18,0x66,0xf7,0x15,0x5a, -0x97,0x29,0x7d,0x8c,0xba,0x5d,0x3f,0x3e,0x9d,0x9b,0x19,0xed, -0x5f,0x79,0x29,0xce,0x31,0xa5,0x65,0x11,0x9b,0xe3,0x9c,0x32, -0x5d,0x02,0x4e,0xd9,0x21,0xde,0xf4,0x60,0xc7,0xa1,0x3b,0xc7, -0x8b,0x05,0x2f,0x4d,0xc0,0x07,0x86,0x6e,0xca,0x22,0xd4,0xf1, -0xce,0x50,0xe5,0x1d,0x3b,0x26,0x6b,0xfd,0xa0,0x50,0x3d,0x3c, -0xfa,0xb8,0x0a,0x7a,0x86,0x21,0x26,0x81,0x03,0xa4,0x7f,0xe9, -0xac,0xd9,0x37,0xfd,0xf1,0xf8,0x7c,0xf8,0x7b,0x31,0xad,0xb9, -0x0d,0xbb,0xa2,0x53,0xec,0x64,0x5c,0x3a,0xfe,0x96,0xb3,0x44, -0xc4,0x37,0x06,0xbc,0xf8,0x98,0xdd,0xda,0x6b,0xb7,0xb2,0x15, -0xef,0x46,0xe6,0xee,0x7e,0x20,0xbe,0x67,0xbf,0x42,0x58,0x85, -0x0f,0xd6,0xe5,0x99,0x02,0x0d,0x3c,0x09,0xac,0xc6,0x51,0x43, -0x03,0x37,0x0c,0x61,0xa9,0x1b,0x18,0x4d,0xe7,0xc3,0x52,0x65, -0xa4,0x22,0xed,0xd2,0xb2,0x9e,0xa9,0x67,0xd4,0x70,0x44,0x3b, -0xab,0x23,0x6e,0xfa,0xfb,0xd4,0xa3,0x11,0x59,0x96,0x80,0x12, -0xe9,0xf5,0x64,0x7e,0x5b,0xf1,0xf4,0xe6,0x56,0xfa,0x9a,0xcd, -0x80,0xc2,0x0e,0x21,0xe5,0xf0,0xd7,0x1c,0xcc,0xc9,0x5b,0x15, -0x93,0xfa,0xfe,0xf6,0xdd,0x81,0xf4,0xaf,0xe3,0xeb,0xab,0xbb, -0xd8,0xab,0x51,0xac,0x9c,0xc4,0xd2,0xeb,0x4a,0x05,0x6d,0x6a, -0x2e,0xa6,0x9f,0x80,0x01,0x9f,0xc3,0x80,0xcd,0x39,0x40,0xba, -0x68,0x5c,0x30,0x65,0x07,0x32,0xbb,0x12,0xcb,0xb8,0x18,0x01, -0x17,0xf8,0xd4,0xa4,0xe4,0x9d,0x1d,0x98,0x10,0xb5,0x28,0x09, -0x45,0x1f,0x80,0x07,0xe9,0x33,0x1e,0x8f,0x88,0x1f,0xde,0x9b, -0x9d,0x8c,0xea,0x80,0x47,0x37,0x5f,0xa6,0x9f,0x90,0xe9,0xce, -0xa7,0x7c,0xa0,0xb1,0x8a,0x6f,0xf5,0xc2,0x65,0xba,0xa7,0xc4, -0x6a,0x7c,0xcf,0x56,0xa9,0xcf,0x7b,0x9e,0x1f,0xa9,0x2d,0x15, -0x70,0x62,0xa2,0x8b,0xb5,0x45,0x14,0xf0,0x1c,0x7a,0x26,0x30, -0xf6,0xd2,0xf5,0xfb,0xbd,0x46,0x7b,0x75,0x45,0xfe,0x31,0x53, -0x9a,0x28,0xab,0x7d,0x22,0x69,0x13,0xe4,0x5b,0x58,0xb3,0xf0, -0x78,0x1f,0x7e,0xe6,0x43,0xbd,0x77,0xe0,0x11,0x10,0x2f,0x33, -0x5c,0x31,0xe5,0x59,0x51,0x99,0x93,0x26,0x76,0xb9,0xe1,0xa6, -0x6e,0x0f,0xb2,0x60,0x9f,0x81,0xb8,0x70,0x31,0xe1,0x5d,0x0a, -0x6b,0x97,0xee,0xa8,0xaa,0xa6,0xbc,0x1e,0x09,0x9b,0x8d,0xa8, -0x62,0xd8,0x74,0xa5,0x84,0x3f,0x34,0xe9,0xff,0xd7,0xbe,0x23, -0xa8,0xb1,0x53,0xf3,0x51,0x6e,0xa9,0x0c,0x20,0x27,0xa6,0x9d, -0x8a,0x02,0x85,0x5a,0x18,0xbd,0x36,0xa2,0x26,0x6c,0x36,0xbd, -0xfa,0xe2,0xcf,0xae,0xf9,0xcc,0x25,0xf1,0x02,0xb2,0xd5,0x88, -0xd0,0x55,0x95,0x31,0x8c,0x44,0x0d,0x35,0x66,0xf7,0xa7,0x17, -0x18,0xb4,0x49,0x49,0xa6,0xc5,0x04,0xc4,0x25,0xb4,0xbe,0x40, -0x6b,0x65,0xb4,0x7e,0x84,0x57,0x4f,0x14,0x27,0xa3,0xe8,0x28, -0x7a,0x0e,0x82,0x76,0x53,0x00,0xef,0x48,0xe2,0x89,0xab,0x08, -0x9e,0x5c,0xda,0x10,0x90,0xdd,0xca,0x73,0x89,0x58,0x6c,0xf0, -0x93,0x88,0x13,0x16,0x0b,0x4e,0x11,0x61,0xc6,0x8a,0x10,0x52, -0x7f,0x0f,0xf9,0x69,0x58,0x87,0x5c,0x3c,0x93,0xcf,0x91,0xc5, -0x22,0xfa,0x8a,0x8c,0x7f,0x23,0x6e,0x2c,0x99,0xc8,0xfe,0x2b, -0x8d,0xc1,0x8d,0x26,0x68,0xfc,0x12,0x28,0x95,0x04,0xb9,0x68, -0x24,0x42,0xdb,0xce,0x02,0xb6,0xb9,0xd7,0xb7,0xf4,0xc7,0xa8, -0x38,0x3d,0x85,0x6d,0xdb,0x04,0x24,0x2c,0x41,0x50,0xb7,0x11, -0xef,0xf0,0xb8,0xb9,0x9e,0x14,0xbf,0xe5,0xfc,0x73,0x5d,0x99, -0x94,0x45,0xb9,0x4f,0x3c,0xbb,0x86,0x37,0x07,0xf2,0x97,0x85, -0x43,0xb3,0x0a,0x81,0x70,0x2c,0xd7,0xaf,0xd2,0x1e,0x9b,0x3c, -0xa0,0x6d,0x6b,0x57,0x6b,0xf4,0x15,0x87,0x47,0xd6,0x25,0x44, -0x54,0x54,0x36,0x9c,0xee,0x75,0xa1,0xa6,0x1c,0x43,0x9b,0x46, -0xa2,0xb3,0xe0,0x5c,0xb1,0xbc,0x3b,0x92,0x7b,0x94,0x80,0xb1, -0x8b,0xc6,0x93,0x3e,0xfe,0xf1,0x73,0xd0,0xc6,0x8f,0xcf,0xbc, -0x86,0xa6,0x81,0x10,0x32,0x14,0xd0,0xe8,0x98,0x0a,0xa8,0xde, -0x36,0x81,0xa6,0x93,0x67,0x36,0xab,0xba,0xf5,0xc1,0x97,0xb4, -0xfb,0x74,0x52,0x73,0x1f,0x88,0xbb,0xa2,0xe1,0x86,0x8a,0xea, -0xb1,0x75,0x99,0xea,0xe4,0x04,0x47,0x2c,0xf1,0x2a,0xa3,0x40, -0x24,0xa4,0x79,0xbd,0xf6,0xa5,0xa3,0x84,0x56,0x99,0xb6,0x52, -0x21,0x17,0x8f,0xfd,0xa4,0x3b,0x60,0x9e,0x8c,0x74,0x21,0xd0, -0xaf,0x13,0xe8,0x12,0x51,0x0b,0xe4,0xa2,0x57,0xa2,0x74,0xba, -0xcd,0x98,0xee,0xe4,0xa8,0x6f,0x20,0xaa,0xfd,0xa8,0x5c,0x4d, -0x73,0x30,0xb7,0xbd,0x15,0xc0,0xca,0x01,0x5c,0xf9,0x18,0x1e, -0x19,0xf5,0x5b,0x86,0x75,0xd5,0x8d,0xf4,0x21,0x76,0x82,0x22, -0x75,0x7f,0x16,0x57,0x79,0xe8,0x05,0xa7,0xa1,0x0e,0x62,0x48, -0xe3,0xd1,0xb0,0x72,0xbe,0x27,0x21,0xe1,0xfc,0x24,0x15,0x3b, -0x90,0x06,0xf4,0x90,0x1b,0x97,0x78,0x7c,0xc1,0x1c,0xc7,0xde, -0x6c,0xc1,0x7b,0x9c,0xc1,0xd1,0xb1,0x80,0xf6,0x8b,0x41,0x5e, -0x83,0x3c,0x69,0x16,0x91,0xb3,0x71,0x1a,0x39,0xec,0xff,0xde, -0x53,0x8b,0xfb,0x00,0x89,0xb2,0xeb,0x0e,0xeb,0xf3,0x9c,0x76, -0x07,0xd6,0x3d,0x8e,0x93,0x86,0x94,0xec,0xec,0x4f,0xb6,0x11, -0xb4,0xf6,0x64,0x9b,0x91,0x5a,0xb9,0x0e,0xc1,0xe8,0xa6,0x29, -0x27,0xc3,0x63,0x16,0xe6,0xc0,0x5a,0x3c,0xc8,0xed,0x73,0x67, -0x10,0x92,0xbe,0x34,0x6e,0x80,0x54,0xb8,0x0c,0x8c,0x14,0xc7, -0x13,0xd4,0x27,0xf0,0xe8,0xb0,0xbb,0x8c,0x6e,0x64,0x08,0x10, -0x08,0x6b,0x14,0x96,0x44,0x21,0x48,0x26,0xd7,0x97,0xc7,0xe3, -0x99,0x37,0xac,0x70,0xb9,0xf6,0x20,0x27,0x0f,0xe4,0x51,0x45, -0x49,0x2a,0x32,0x5c,0x81,0xf3,0x1f,0x13,0x7a,0x45,0x86,0x37, -0x06,0xf1,0x22,0x90,0xe5,0x19,0x6a,0xdb,0xca,0x04,0x34,0x1d, -0xfe,0xcc,0x3e,0x2d,0x9d,0xa9,0x84,0x53,0xce,0x32,0xc9,0x2b, -0xd4,0xb3,0xa9,0xdb,0x5f,0x2b,0xa0,0x46,0xd8,0x36,0xd4,0x8f, -0x15,0xeb,0x83,0x88,0x5e,0xf7,0x89,0x5d,0xb7,0x41,0x13,0x59, -0xe5,0xf5,0x2d,0xed,0x8f,0x17,0xac,0x6f,0x6f,0x7b,0x10,0xf7, -0x59,0x76,0x67,0x6b,0x10,0x2c,0xd0,0x75,0xdf,0xa8,0xcc,0x77, -0xec,0x77,0xea,0x1a,0xed,0xb7,0x2a,0x30,0x4b,0xa6,0x45,0xdb, -0xab,0x94,0x6a,0x78,0xc2,0xb2,0xfa,0xcc,0x54,0x79,0x04,0x75, -0x78,0xb0,0x40,0x16,0x8e,0x8e,0xcd,0xcd,0xee,0xbd,0x1b,0x6a, -0x86,0xfe,0x57,0x7c,0x13,0x89,0x69,0xe8,0x7c,0x8a,0x71,0xb8, -0xc7,0x57,0xe9,0x4d,0x45,0xff,0x91,0x77,0x60,0xd5,0x43,0x80, -0x9c,0x60,0x17,0x8b,0x36,0xdb,0x93,0xe6,0xed,0x94,0x8a,0xf8, -0x73,0x6a,0xf2,0xe2,0xd5,0xe9,0x46,0xcc,0xc5,0x0d,0x26,0x7a, -0x38,0x19,0x17,0x17,0x0d,0x42,0xb7,0x49,0x58,0xd2,0x2d,0xaa, -0x16,0xc4,0x3c,0x53,0x8c,0x50,0x55,0xc4,0xf4,0x55,0x98,0x8b, -0x1b,0x15,0xa3,0x3f,0xba,0x29,0xc1,0xf4,0xe1,0xb1,0xd9,0x1c, -0xe4,0xf6,0x0b,0xe0,0x6f,0x33,0x47,0x64,0x6e,0x62,0x22,0x14, -0x0b,0x69,0x27,0x81,0xa6,0x25,0xbd,0x41,0x59,0x25,0x8c,0x5d, -0x80,0x0f,0xe1,0x3e,0x1c,0x35,0xeb,0x86,0x42,0x13,0xfa,0x61, -0x80,0x96,0x56,0x1b,0x1b,0x08,0xcb,0xc5,0x6d,0xbf,0x63,0xba, -0xf3,0xe4,0xea,0x52,0x5a,0x3e,0xe6,0xf5,0x6a,0x16,0x8f,0x1a, -0xc5,0x40,0xd8,0x40,0xf0,0xb4,0xee,0x88,0xb8,0xeb,0x35,0x54, -0xab,0x0f,0x1f,0x88,0x60,0xfd,0xa9,0x14,0x34,0x3e,0xb6,0xf0, -0x22,0x28,0xa7,0xb0,0xa4,0xef,0x61,0x5a,0xfd,0x3e,0xef,0xac, -0xb4,0x4c,0xbf,0xae,0xaa,0x34,0xc5,0xb8,0xbd,0xb0,0x3b,0xf4, -0x81,0x03,0x08,0x8b,0x7b,0xed,0x4a,0xec,0xc9,0x53,0xea,0x4a, -0x04,0x8d,0xe1,0x9d,0x7f,0x5c,0x61,0xda,0x8a,0x0a,0x2c,0x77, -0x2f,0x12,0xe4,0x03,0xd8,0x24,0x4e,0x36,0xeb,0x68,0x7e,0xc4, -0x4f,0xf6,0xfc,0x0c,0xd1,0x2d,0xe9,0x84,0x04,0x6f,0x1c,0xe8, -0x0d,0x13,0xb5,0xf5,0x1d,0x27,0x3a,0x5e,0xa6,0xa8,0x4a,0x33, -0x8a,0x66,0xb2,0xf1,0xbc,0x07,0xcd,0x44,0x08,0x0e,0x9e,0xe7, -0xd1,0xc9,0x2b,0xd3,0x81,0x29,0xab,0x09,0x91,0x6b,0xbf,0x04, -0x64,0xad,0xc8,0xfc,0x48,0x61,0x43,0x82,0x44,0x90,0xaf,0x5d, -0x5e,0x97,0xf3,0xb5,0x63,0x74,0xa6,0x57,0x96,0xe3,0x91,0xa3, -0x9e,0xdc,0xd4,0x94,0x49,0xa8,0x28,0x22,0x8c,0x5e,0xb0,0x38, -0x4d,0x6f,0x50,0xa3,0x78,0x3a,0xb0,0x96,0x20,0xfc,0x46,0x1a, -0x05,0x94,0xf8,0x8f,0x7f,0x25,0x85,0xfc,0xf8,0x72,0x1a,0xec, -0xac,0x69,0xa4,0x9f,0xa3,0x7f,0x7d,0xf1,0xfd,0x40,0xf6,0x10, -0x3e,0x26,0x1c,0x67,0x81,0x5c,0xc9,0x0f,0xfa,0x28,0x34,0xb6, -0x9b,0x45,0xdd,0x14,0x43,0x45,0xed,0x2b,0x0d,0xe9,0x45,0x23, -0x54,0xd3,0x90,0x51,0x06,0x29,0x69,0xf0,0x89,0x55,0x48,0xee, -0x29,0xaf,0xe9,0x25,0xc1,0x42,0x9d,0x31,0x1a,0x5f,0x68,0xf5, -0x04,0x7d,0xc4,0xa7,0x61,0x31,0xbf,0xfb,0xa8,0xcf,0x12,0x51, -0x8e,0xe7,0x07,0xc5,0xe5,0x18,0xef,0xa9,0xdd,0x3a,0x46,0xc8, -0xd1,0x1a,0x9f,0xd8,0x71,0x0d,0x5c,0xfd,0xa9,0x0e,0xdd,0xc1, -0x43,0xd2,0x6f,0x76,0x90,0x38,0x29,0x7e,0x64,0xfa,0x44,0x5c, -0x67,0x70,0xdb,0x18,0x43,0xae,0x42,0x88,0x71,0x60,0xcc,0xe7, -0x17,0x71,0xff,0x60,0xc5,0x36,0x96,0xb5,0x3d,0x59,0xf0,0x93, -0x5e,0xc4,0x3b,0xba,0x22,0x82,0xbb,0x30,0xf6,0xf5,0x2d,0xa6, -0xcb,0x20,0xdf,0x49,0x41,0xbc,0xea,0xc7,0x4f,0x2e,0xce,0x0c, -0x3a,0x98,0x0b,0x98,0xac,0x2b,0x25,0x90,0x2d,0x89,0xc0,0x80, -0x77,0x1d,0x3b,0xbd,0x36,0x59,0x92,0xee,0x92,0xf3,0x5e,0x5d, -0x57,0x95,0x1b,0x68,0xf2,0x7c,0xdc,0x92,0x68,0xbc,0x4c,0xa6, -0x9f,0x72,0x8b,0xb0,0x6b,0x6b,0x5e,0x2c,0xa2,0xaa,0xd7,0xb1, -0x6a,0xa0,0xdd,0x91,0x06,0xc5,0xec,0x29,0xfc,0x25,0x48,0xce, -0x48,0x1d,0xb6,0xcf,0x66,0x03,0x32,0x36,0x2d,0x94,0xeb,0x67, -0xbb,0xf7,0x73,0x73,0xdd,0xaf,0x7d,0xa6,0xbd,0x93,0x3b,0x1c, -0xbd,0x06,0xdd,0x0c,0xb1,0x63,0x42,0xbe,0x2f,0x5d,0xd1,0xa8, -0x50,0xd5,0x5d,0x5f,0x9d,0x66,0x52,0xd4,0x7f,0xd1,0x7c,0x36, -0x2c,0x2e,0x6c,0x6b,0xc2,0x27,0x59,0xdc,0x6a,0x2c,0x52,0x8f, -0x18,0x47,0x55,0xea,0x35,0xb5,0x34,0xec,0x70,0x11,0x38,0xa9, -0x1d,0x1d,0xc5,0xe5,0xe5,0x78,0x54,0x38,0x8f,0x84,0x82,0x9a, -0xc7,0x87,0xa5,0x0b,0x13,0x61,0x3d,0xb9,0xbc,0x32,0xa1,0xb1, -0x91,0xea,0x97,0xdb,0x9a,0xcb,0x15,0xf7,0xbd,0x56,0x3b,0x88, -0xac,0x8b,0x24,0x07,0xb6,0xe9,0x4a,0x76,0x91,0xd6,0x1d,0xcc, -0xec,0xad,0xfc,0x78,0xea,0x4c,0xba,0x85,0x9e,0x35,0x82,0xd6, -0x77,0x9f,0xe8,0xad,0x1d,0x56,0x36,0x3e,0x9a,0xce,0xc0,0x14, -0x7b,0x0b,0x43,0xcc,0x15,0xdb,0xd8,0x58,0x17,0x7c,0xf6,0xc3, -0x5d,0x0f,0xad,0xee,0x0a,0x9a,0xf3,0xa9,0x45,0xe6,0x0f,0xde, -0x56,0xd6,0xac,0x3b,0x6c,0x5a,0xe9,0x2c,0xcf,0x63,0x51,0x6a, -0x52,0x16,0x24,0x55,0x26,0x04,0xf9,0xc4,0x1a,0xb3,0x06,0x41, -0xb6,0xc9,0x8c,0xed,0x16,0xc6,0x48,0xec,0x98,0x5b,0x8a,0x0c, -0xfa,0x88,0x02,0x34,0x07,0x8f,0x87,0x0b,0x59,0x88,0xd7,0x1c, -0x6e,0x28,0xe6,0x49,0x4c,0xcb,0x07,0xcf,0xf0,0x9a,0x27,0x1a, -0x4f,0xaa,0x4e,0x41,0x16,0x2f,0xa8,0x5c,0x39,0xaa,0xe9,0xa6, -0x3a,0x3c,0x81,0x5e,0x2f,0x1d,0x92,0x9a,0x16,0xb0,0xe2,0x68, -0x34,0x00,0x8b,0x60,0x15,0x65,0x9f,0x29,0x6d,0xf2,0x98,0xe2, -0x89,0xc6,0x4d,0x60,0x85,0x9a,0x81,0x96,0xc3,0x60,0x4f,0x48, -0xd4,0x0d,0x4f,0x83,0x71,0x4b,0x63,0xd4,0x04,0xac,0xbc,0xd2, -0x27,0x9b,0x3c,0x92,0x9f,0x75,0x7a,0xab,0xbb,0xc1,0x1a,0xd4, -0x60,0x15,0x64,0x09,0xe9,0xef,0x01,0x2c,0x16,0xa1,0xf0,0xc0, -0x86,0xe6,0xc0,0xe0,0x44,0x18,0xa2,0x13,0xe9,0x3a,0x43,0xc3, -0xe4,0xf5,0x84,0xee,0xc9,0xad,0x4d,0x6d,0xa0,0x39,0x67,0x67, -0xd9,0x1f,0x74,0x23,0xfb,0x0b,0xc4,0xc2,0x4b,0xac,0x35,0x27, -0xd9,0xd8,0xf0,0x01,0x45,0xe8,0x76,0xfb,0xcc,0x87,0x19,0x41, -0x32,0x79,0xf7,0x1c,0x49,0x36,0x2b,0x0d,0xd0,0xab,0xd6,0x9c, -0xb2,0xd5,0xbf,0x8f,0x7a,0xc1,0x6d,0xd4,0x73,0x6d,0x8a,0x1e, -0x38,0xd8,0x2a,0x59,0xec,0x2b,0x66,0xff,0x0a,0x45,0x5c,0xfe, -0x1e,0x04,0x11,0x20,0xd2,0xd3,0x57,0xe6,0xf0,0x8e,0x30,0x15, -0x3f,0x06,0xc0,0xc4,0xcc,0x68,0x3c,0x9b,0xaf,0xa0,0xcc,0xeb, -0x32,0xfe,0x92,0xd9,0xbe,0xe8,0x2e,0x02,0x02,0x24,0x68,0x44, -0x8d,0xea,0xc2,0x92,0x80,0xf5,0xb0,0x9d,0x51,0xdc,0x60,0xd6, -0xc1,0xe1,0xea,0x51,0xa2,0xf6,0x30,0x52,0x2b,0xba,0x2f,0xa4, -0x32,0xd5,0xe1,0xc2,0xc9,0x74,0xb4,0xcb,0xf6,0x88,0xec,0xc8, -0xb3,0xf6,0x90,0x86,0x24,0xd4,0x15,0xa2,0x8e,0x72,0x71,0x4d, -0x2e,0xe8,0x69,0x22,0xc8,0xa3,0xa5,0x12,0xaf,0x91,0x21,0x99, -0x92,0x8c,0xc4,0x61,0x12,0x6c,0x1d,0x39,0x22,0x1e,0xbd,0x78, -0xb1,0xf6,0xaa,0x88,0xcc,0xa8,0xac,0x94,0x74,0xf5,0xd9,0xf5, -0xc6,0x5f,0xd5,0xe4,0xd1,0xa7,0xd0,0xd5,0x1d,0x57,0x01,0x9d, -0x36,0xfa,0x86,0x24,0x15,0xf3,0xd5,0x67,0x96,0x5c,0x5d,0x3b, -0x2e,0xd9,0x17,0x1e,0xad,0x03,0x97,0x45,0x3c,0x64,0xfe,0x75, -0xf2,0x6a,0x3b,0xd4,0xa9,0x13,0x89,0x60,0xa4,0x93,0x62,0x51, -0x55,0x88,0xf3,0xe2,0x11,0x53,0xe8,0x42,0x52,0xff,0x4f,0xf4, -0xa6,0x3b,0xb0,0x77,0xdd,0xca,0x86,0xf1,0x35,0x5d,0xac,0xb6, -0x2e,0xe4,0x90,0xb7,0x9e,0x3e,0x1c,0xc8,0x8a,0x87,0xc6,0xc7, -0x93,0x64,0x6a,0x49,0xe6,0x1d,0x78,0xd6,0xa8,0x6e,0x81,0xcb, -0x59,0x25,0x49,0xde,0x7a,0x9d,0xa4,0xb3,0x5f,0x15,0x5e,0xd2, -0xd9,0x5a,0xab,0x68,0xd1,0x34,0xab,0xb1,0x30,0x5f,0x63,0xfa, -0x4d,0xa9,0xb0,0xcc,0xb1,0x19,0xee,0xa3,0x03,0x28,0x73,0x4c, -0x3f,0x6c,0xc6,0xa0,0x0c,0x14,0xd6,0xda,0xc0,0xf4,0x87,0x8b, -0x45,0x67,0x0b,0x52,0x3b,0x5b,0xc7,0xfe,0x92,0x00,0x24,0xf2, -0x83,0x03,0xa5,0x72,0xdd,0x20,0x6c,0xef,0x11,0x34,0xd7,0x30, -0x1f,0xba,0xcb,0x5e,0xa9,0x39,0x8e,0xb3,0x8f,0xf3,0x63,0x9f, -0xcd,0xfa,0x14,0xbc,0xfe,0x4c,0xee,0xcd,0xec,0x3d,0x40,0xe9, -0x4b,0xda,0xa2,0xba,0xdc,0xf5,0x28,0xfb,0x38,0xf5,0xf6,0x7a, -0x12,0xe2,0xd9,0xc1,0x72,0x6c,0xe7,0xa4,0xcf,0xab,0xd2,0x1a, -0x17,0x1b,0x24,0x99,0xa7,0x5c,0xbc,0x96,0x1e,0xa7,0xdd,0x00, -0xf2,0x2d,0xa9,0x42,0x15,0xe4,0x7a,0xde,0x1c,0xf6,0x8e,0xe1, -0xef,0x71,0x06,0xe3,0x62,0xd8,0xb3,0x0d,0x93,0xc1,0x4f,0xf3, -0x38,0x2c,0x8c,0x12,0x25,0x14,0x0d,0xd2,0xbe,0x9b,0x4e,0xd1, -0x03,0xb2,0xc3,0xd8,0xac,0x29,0xc7,0xae,0xc6,0x54,0xa5,0x2d, -0x89,0x6c,0x0a,0x4d,0xc5,0x4f,0x1b,0x1b,0xc3,0xd6,0xd9,0xc5, -0xf4,0x78,0x88,0xed,0x2f,0x4f,0x98,0x76,0x09,0x33,0x03,0x05, -0x77,0x02,0x75,0x2f,0x98,0xc3,0xee,0x4e,0x5e,0x40,0x35,0x98, -0xe5,0xdf,0x50,0xd3,0x6a,0xa3,0xc2,0x3a,0x85,0x2f,0xf4,0x6d, -0x25,0xed,0xd8,0xb7,0x6e,0xa4,0x83,0xe3,0x11,0x14,0xae,0x13, -0x56,0x69,0x48,0x59,0x7d,0xb9,0x0a,0xa1,0x6c,0x62,0x8e,0xa3, -0x8c,0x63,0xb6,0x80,0x7e,0x41,0xc1,0x91,0x91,0x37,0x94,0xf3, -0xd9,0x35,0x05,0x58,0x3a,0xa6,0xb4,0x63,0x9d,0x46,0x56,0x37, -0x98,0x8c,0x96,0x37,0x2f,0xc8,0x58,0x6e,0xbd,0x11,0x2c,0x11, -0x94,0x0b,0xeb,0x03,0xfd,0xea,0x11,0x37,0x7c,0xe1,0x6c,0xeb, -0x1c,0xf0,0xb1,0x00,0x1f,0xc7,0xc0,0x9c,0xe1,0xbf,0x5b,0x3e, -0x94,0x8f,0xa7,0x52,0x99,0x32,0xfc,0x7a,0xcc,0x43,0x97,0x6c, -0x01,0xda,0xc6,0x1b,0xac,0x63,0xeb,0xda,0x6e,0xa5,0x79,0x3e, -0x48,0xf2,0x5c,0x0d,0x4b,0xe4,0x40,0x58,0x66,0x68,0xc5,0x01, -0x29,0x4a,0x89,0xf0,0xd5,0x22,0x0d,0xb0,0x0f,0x48,0x48,0xdb, -0xdc,0xf4,0x82,0xfd,0xf8,0xb7,0xc6,0xb0,0x8f,0x89,0x03,0x73, -0x2c,0xbf,0xe1,0x64,0x4e,0xbb,0x69,0x97,0xee,0xe8,0x55,0x0c, -0x5f,0xb1,0x6f,0x58,0xf2,0x1a,0x8a,0xdc,0x45,0x64,0x34,0x9b, -0x66,0xdd,0x62,0xe7,0xec,0x63,0xc9,0xa6,0xba,0x90,0x1f,0x43, -0x55,0xf4,0x13,0xd5,0x94,0xa6,0x1c,0x53,0x1c,0xc3,0x9d,0xb1, -0xa2,0x8c,0xea,0x94,0xfa,0x8e,0x7d,0x7d,0x16,0xe1,0x3a,0x7f, -0x8d,0x14,0xb3,0x0d,0x86,0x89,0x4b,0x6b,0xe2,0x61,0x9b,0x7b, -0x7a,0xd5,0xf0,0x8d,0x4d,0x2f,0x22,0x5a,0x74,0xc9,0x44,0x12, -0x7d,0x75,0x07,0x2c,0xcd,0x2f,0x06,0xbf,0x11,0x87,0xeb,0x0f, -0xec,0xe9,0xaf,0x18,0x54,0xe6,0x15,0xe1,0xd6,0x59,0x6a,0xd8, -0xce,0xa6,0x9e,0x0b,0x99,0x33,0x1f,0x7a,0x0b,0x9f,0xd0,0x26, -0x99,0xb2,0x3e,0xd8,0x09,0x50,0xb3,0xda,0xb9,0xaf,0xb7,0x72, -0x6e,0x8d,0x65,0x31,0x93,0x27,0x4e,0xeb,0xeb,0xd6,0x8e,0xf5, -0xc6,0x3a,0xbc,0x91,0x77,0x24,0xaa,0x6e,0x32,0x1d,0x8d,0x51, -0xf0,0x96,0x2f,0x63,0x3b,0x98,0xe0,0x9a,0x6f,0x2c,0xc5,0x33, -0x9e,0xea,0x78,0x0c,0xf9,0x28,0xa9,0xe1,0x8c,0x5f,0xa8,0x56, -0xda,0xd6,0xe4,0xc1,0x76,0x60,0x69,0x68,0xed,0xed,0x27,0xcf, -0x7d,0xa8,0xf7,0xc4,0xce,0x31,0x78,0xe4,0x2a,0xe0,0x81,0x59, -0x2a,0x3c,0xec,0x52,0x98,0xab,0x84,0xb9,0x5d,0x32,0xf0,0xcb, -0xf3,0x84,0xa6,0xc6,0x51,0x3f,0x29,0xca,0x4a,0x1c,0x63,0x4d, -0x43,0x95,0xee,0x3c,0x77,0x5f,0xc7,0x25,0x9b,0xc9,0x20,0x59, -0xb2,0x51,0x97,0xb2,0x57,0xf0,0x7b,0x20,0x90,0x39,0x82,0xac, -0x5b,0x5b,0x7e,0xdd,0x77,0x62,0x82,0x55,0x8e,0xc7,0x09,0x9d, -0x4c,0x27,0xb6,0x90,0xe3,0x96,0x5b,0xad,0x2d,0x98,0x52,0x9a, -0xcb,0xd4,0x22,0xf6,0x05,0x5d,0x68,0x35,0x5c,0xa7,0x99,0xdc, -0x57,0x78,0x08,0xbf,0x42,0x8c,0xb0,0xb9,0x62,0x7b,0xbc,0x5e, -0x00,0x3b,0x79,0xdb,0x20,0xaf,0x74,0xaf,0x2e,0xa6,0x43,0xd9, -0xa0,0x71,0x2f,0x42,0xce,0x6a,0x59,0x84,0x3b,0xc3,0x9b,0x30, -0xaf,0x93,0x6d,0x12,0x17,0x94,0xa5,0xea,0xb6,0xfe,0x5f,0x6f, -0x2f,0x16,0xf2,0xd4,0x81,0x35,0xa3,0xb2,0xc8,0xad,0x1e,0x5b, -0x6f,0xaf,0x2b,0x1a,0xea,0xc0,0xe8,0x27,0x77,0x64,0x78,0xbf, -0xd3,0xb5,0xe0,0x5a,0x44,0xc3,0x02,0xd0,0x43,0xab,0xaf,0x29, -0x84,0x7e,0x03,0x1c,0x4d,0x04,0x83,0x85,0x27,0x53,0x0c,0x2b, -0x0f,0x02,0x58,0x9d,0xb7,0x51,0xb9,0x1f,0x5e,0x75,0xb5,0x20, -0x61,0x0a,0xbd,0xc2,0x79,0x95,0xdd,0xad,0xb3,0x6c,0xa6,0xcd, -0xb9,0x87,0x9a,0xd4,0xd8,0x88,0x38,0xcd,0xc2,0xc9,0xc9,0xf5, -0x25,0x5f,0xd8,0x6f,0x84,0x16,0xbf,0xb4,0x45,0x9c,0xc8,0x8e, -0x8c,0xc0,0x70,0x77,0x58,0x6f,0x5a,0xe7,0x0f,0xc3,0x09,0x50, -0xce,0xea,0x46,0xd3,0x4b,0x4f,0xc4,0x65,0x31,0x31,0x80,0x3b, -0xea,0x07,0x48,0x21,0x37,0x95,0x7f,0xe4,0x94,0x5f,0x4c,0xf0, -0x64,0x0e,0x7e,0x9a,0x44,0xcb,0xe9,0xc5,0x14,0xc8,0xa4,0x47, -0x31,0x5d,0x4e,0x1f,0x21,0xf0,0x26,0x42,0x74,0xac,0x51,0x32, -0x37,0x0a,0x7a,0x5f,0xb8,0x19,0x97,0x27,0xc3,0xab,0x71,0x76, -0x93,0x6c,0x24,0x99,0x78,0x1e,0x34,0xc9,0x33,0x7c,0x46,0xd7, -0x80,0x26,0x41,0xcf,0x71,0xe4,0x10,0xd0,0x3c,0x4c,0x1e,0xc2, -0x23,0xba,0x6c,0xc2,0xf4,0x87,0x98,0xfe,0x1f,0x7f,0x6c,0x7f, -0xd3,0x85,0xe9,0xdc,0x75,0x08,0x5b,0xd7,0x13,0xc6,0x98,0xdb, -0xfd,0x6e,0xc3,0xe7,0x71,0x4e,0xaa,0x48,0x78,0x8f,0x41,0x12, -0xd1,0xc1,0x89,0x90,0xa1,0xfd,0xba,0xf6,0x93,0xa6,0xec,0x82, -0x2b,0x18,0x9c,0x6b,0x0e,0xe4,0x26,0x1c,0x32,0xd8,0xd6,0x1b, -0xa0,0x68,0xd4,0xa0,0xb0,0x80,0x16,0xc9,0x82,0xa3,0x95,0x12, -0x16,0x5c,0xc0,0x84,0x69,0x32,0x10,0x24,0x5b,0x78,0xcd,0x0f, -0xed,0xe5,0xa0,0x23,0x7b,0x4b,0x47,0x9f,0xb0,0xe2,0xe4,0x8a, -0xfa,0x8c,0x13,0xd1,0xe9,0x26,0x10,0xdd,0x14,0x10,0xe7,0xcd, -0x24,0x68,0x0a,0x8b,0x31,0xf2,0xb0,0xec,0x97,0x50,0xfb,0x21, -0x16,0xb0,0x4f,0xf9,0x78,0x89,0x87,0xd8,0x94,0xd2,0x31,0xda, -0x09,0x07,0x0c,0xc0,0xfa,0x56,0xf9,0x1c,0x2e,0xae,0x90,0x93, -0x06,0xb3,0xb6,0xeb,0x8d,0x4c,0x39,0xbb,0x6f,0x0b,0x0e,0xba, -0x77,0x99,0xaa,0xbb,0xc9,0x8b,0x58,0x33,0x75,0x8f,0xe3,0x8f, -0x22,0x66,0x59,0xd4,0xc6,0x4a,0x0d,0xc1,0x3b,0xa2,0x68,0xb9, -0xb1,0xfa,0xbe,0xfc,0x48,0xdc,0x3b,0xe3,0x36,0x8f,0x83,0x04, -0x1c,0xa9,0xd0,0xfa,0xd1,0x72,0xa4,0x35,0x91,0x81,0x5b,0x8b, -0xfb,0xa8,0x24,0xed,0x5d,0x09,0xfc,0x10,0xd2,0xcd,0x29,0x5d, -0xdd,0x91,0x28,0x69,0xd1,0x1b,0x0c,0x5b,0x6f,0x8c,0x5e,0xa0, -0x5d,0xf1,0x78,0x06,0x93,0xde,0x9a,0x9c,0xbd,0x56,0x3c,0x11, -0xd8,0xd2,0x69,0x21,0xb6,0xd7,0xc5,0x28,0x6f,0x36,0x5d,0x81, -0x66,0x92,0x38,0x62,0x08,0xa8,0xc7,0x3f,0x00,0x91,0x15,0x74, -0x75,0xc1,0x86,0xf0,0xd9,0x1f,0xcf,0xe7,0x30,0x56,0x70,0x82, -0x60,0x83,0x42,0x4a,0xb6,0xf5,0xec,0xab,0x46,0xff,0xb0,0x3c, -0xdc,0x1f,0x34,0x7b,0xe9,0x57,0xe8,0x16,0x4f,0x85,0x11,0xc3, -0xdc,0x3c,0xca,0x96,0x79,0x01,0x39,0x9b,0x61,0x0e,0x7f,0xc6, -0x64,0xca,0x41,0xbf,0xb6,0x1a,0xad,0xf4,0x97,0x2d,0x81,0x86, -0x0a,0x69,0x86,0x3f,0x4c,0x30,0x3e,0x2f,0x3c,0xc0,0x1f,0x78, -0x9a,0xc1,0x13,0x0a,0x0a,0x87,0x13,0x78,0x40,0x19,0xe2,0xff, -0x82,0xdf,0x39,0x26,0x5c,0x6f,0xb7,0xb7,0xff,0x06,0x2f,0xfc, -0x2b,0x09,0xdf,0x4a,0xc2,0xb7,0xc9,0x52,0xf0,0xce,0xf2,0xad, -0xc3,0xc3,0xc5,0xc3,0xc5,0xe1,0x6c,0x71,0x38,0x59,0x1c,0xce, -0x17,0x5c,0x90,0x7f,0xbe,0xdd,0x3a,0xeb,0xfa,0x4f,0x57,0xea, -0x55,0xe4,0x97,0x18,0x42,0xc7,0x94,0xd2,0x1c,0xd6,0xc8,0x0a, -0x19,0x7e,0xd7,0x26,0xe6,0x5e,0x15,0x81,0x3b,0x74,0x9b,0x5c, -0xd3,0x9a,0xa9,0x77,0xce,0x04,0x14,0x29,0x86,0xd1,0x6f,0xd8, -0x52,0xc2,0x69,0x60,0x4b,0xcd,0xcd,0x93,0xca,0x3e,0xcf,0x78, -0x08,0xd5,0xec,0xb7,0x81,0xd9,0x8e,0xab,0xc0,0x0c,0x34,0x03, -0x5a,0x3c,0x10,0x3e,0xe6,0xee,0xd0,0xf0,0x06,0x33,0x39,0x3a, -0xba,0x6a,0xe6,0x0f,0x93,0x6e,0xe0,0x99,0x5a,0x48,0x8f,0xbd, -0xb0,0x33,0xcd,0x7a,0x3c,0x18,0x5b,0xbb,0x99,0x9e,0x9e,0x96, -0xea,0xd6,0x25,0xa3,0x6f,0xb2,0xc3,0x6b,0x56,0x70,0x88,0xfd, -0x3f,0xc3,0x79,0xe6,0xc4,0xfd,0xb6,0x82,0x1d,0xe1,0xb8,0x68, -0xca,0x90,0x71,0x8c,0x48,0xe8,0xc0,0x2d,0x8c,0xd4,0x92,0x3c, -0x6c,0x1e,0x4e,0x1a,0x8d,0xa3,0xa3,0x79,0xde,0xb0,0xf0,0xc0, -0x74,0x41,0x3a,0x44,0x56,0xd4,0x7b,0xf8,0x10,0x75,0x2b,0x94, -0x8a,0x30,0x69,0x0a,0xd0,0x0f,0xa9,0x2d,0xdc,0x87,0xac,0x44, -0xa6,0x60,0x22,0x8c,0x80,0x48,0xf0,0xd8,0x46,0x08,0x90,0x74, -0x61,0x10,0x37,0x6d,0x4e,0x13,0x86,0xb4,0xb4,0xb4,0xed,0x02, -0x6e,0x08,0xf6,0x19,0x65,0xf7,0x82,0xf4,0x82,0xb7,0x8c,0x02, -0x2c,0x6e,0xa8,0xb5,0x60,0x58,0x15,0xe8,0x4b,0x10,0x6d,0x29, -0xa5,0xff,0xd0,0xbe,0xb5,0x41,0xc2,0xea,0xcd,0x32,0xbd,0xc1, -0x3a,0xa5,0x70,0xb2,0x84,0x97,0xd4,0xf5,0x33,0x8e,0x48,0x20, -0xd9,0x00,0x15,0xf9,0xc3,0x87,0xf0,0xf3,0x6b,0xec,0x12,0x8a, -0x86,0x0b,0x06,0x6b,0x9b,0x04,0xb2,0x10,0x91,0x0d,0xe0,0xb1, -0x7a,0xed,0xe1,0x43,0xd8,0x6b,0x76,0x75,0x85,0x96,0x81,0x1f, -0x5d,0x61,0x72,0x77,0x3e,0xfb,0x7c,0xc3,0x53,0x88,0xa6,0x80, -0xe3,0xe9,0x35,0x5f,0x83,0xba,0x07,0x18,0x9f,0x47,0x89,0x90, -0x9b,0x2e,0x4f,0xc8,0x95,0xd8,0x18,0x03,0xd9,0xe1,0x99,0xc3, -0xd8,0x2a,0x44,0x64,0xf8,0x93,0x16,0x03,0xa7,0x6c,0xea,0x74, -0xcf,0x58,0x11,0x25,0x99,0x23,0x9e,0x7f,0xd5,0x89,0x4e,0x05, -0x6e,0x82,0x12,0x8a,0x17,0x4b,0x59,0x77,0x24,0x6c,0xcb,0xdb, -0xaa,0x9d,0xec,0x0e,0xa3,0x62,0xe5,0x47,0xe0,0x5a,0x1f,0xf6, -0x42,0xe2,0xa2,0xb6,0x91,0xa6,0xf0,0x7c,0x58,0x59,0xad,0x6c, -0xbb,0xf1,0x3d,0x00,0x82,0x68,0x38,0x6f,0x2f,0x95,0xd5,0x36, -0x8c,0xa0,0x7d,0x44,0xe0,0xbd,0xb0,0x2c,0x29,0x0e,0xbb,0xb2, -0x34,0x36,0xd0,0xeb,0x11,0xcb,0x1c,0xb0,0x97,0x86,0x56,0xc6, -0xf5,0x09,0x7e,0x66,0x68,0xa4,0x4a,0xfe,0xb2,0xcb,0xf3,0xe2, -0x14,0x99,0x2c,0xaa,0xcf,0xf1,0xe7,0x0a,0xa7,0x2b,0x49,0x26, -0x9c,0x53,0xd9,0xa8,0xf1,0x39,0x21,0x4a,0x07,0xca,0xf9,0xd4, -0x7d,0x97,0x4e,0xdc,0x3d,0x04,0x0b,0xa7,0x77,0x4f,0x81,0x18, -0xad,0x19,0x39,0x5e,0xa2,0xf5,0x0b,0xa7,0xd0,0xb8,0x9e,0x4f, -0x28,0x36,0x85,0xa5,0x91,0xdf,0x17,0x8b,0xf6,0xba,0xde,0x90, -0x2f,0x16,0x12,0x96,0x64,0x4a,0xf7,0x32,0xab,0xab,0x2e,0x6d, -0x4a,0x96,0xae,0x59,0x58,0xbd,0x0d,0x1f,0x8c,0x13,0x00,0x1b, -0x80,0x70,0xff,0xfb,0xbe,0x3a,0x38,0x1e,0x51,0x64,0x04,0x1f, -0x1c,0x34,0x89,0xfa,0x74,0x11,0xbb,0x44,0x33,0xac,0xea,0x31, -0x37,0xd4,0xdb,0x59,0xb5,0x22,0x35,0x30,0xf0,0x6c,0x96,0xf7, -0x2a,0xd1,0x6d,0xbf,0x08,0x58,0xea,0x45,0xd9,0xc3,0x13,0xa8, -0x24,0x26,0x15,0x24,0xcb,0x19,0xfe,0x5c,0x0d,0x47,0x07,0xd3, -0x8f,0x63,0x67,0xec,0x06,0xa2,0x47,0xf5,0x78,0xd2,0xcf,0x37, -0xbc,0xef,0xf6,0xf3,0x78,0x36,0x7d,0x55,0x00,0x7a,0x92,0x7b, -0x7c,0x4b,0x0c,0x53,0x29,0xaf,0x22,0x53,0x4e,0x67,0x23,0x0c, -0x20,0x59,0xfc,0x39,0x56,0xb5,0xc0,0x4a,0x51,0x4c,0x47,0xb7, -0x55,0x43,0xdf,0x83,0xee,0x2e,0x1b,0x69,0x4b,0x50,0xd4,0xd4, -0x26,0x78,0x54,0x75,0x6f,0xc4,0x43,0x26,0x9a,0xf3,0xb9,0xc4, -0xbd,0x29,0xb9,0xa4,0x84,0x01,0x72,0x5a,0x80,0xcc,0x0d,0x6b, -0xe2,0xc9,0xc7,0x77,0x30,0x6d,0x60,0x17,0xf4,0xc9,0xa6,0x4a, -0xd0,0x38,0x46,0x2d,0x69,0x1e,0xc3,0xee,0xf5,0x6c,0xc8,0xbc, -0x47,0x1e,0xac,0x7d,0xc9,0xec,0x92,0xbe,0x6d,0xf4,0x1a,0xef, -0x5f,0xe4,0xee,0x9d,0xf5,0x2b,0x3f,0xc1,0x56,0xbf,0xf4,0x45, -0xcc,0xe7,0x31,0xfa,0xec,0x8f,0x0a,0xb5,0x30,0x15,0xed,0x27, -0x2f,0x61,0xcb,0x77,0x5e,0xcd,0xa6,0x64,0xcc,0xff,0x34,0x1e, -0x7f,0xac,0x66,0x63,0x2a,0xe6,0x8e,0x86,0x9f,0xab,0x99,0x23, -0xd4,0x25,0xb5,0xcd,0x39,0x30,0xb1,0x6a,0x26,0xa6,0x52,0xbd, -0xc5,0x04,0xc3,0xa2,0x56,0x2b,0xa6,0x74,0xb2,0xec,0x1c,0x43, -0x83,0x8c,0xaa,0x10,0x9c,0xce,0x38,0x2e,0x2e,0x8a,0x55,0x60, -0x2a,0x13,0x60,0xbb,0x3c,0x54,0xe9,0xc2,0x4a,0x6e,0xdb,0xc6, -0x70,0x62,0x80,0xa6,0xa9,0xdf,0x9a,0x9d,0xf1,0xe3,0x47,0xf6, -0xf9,0xe9,0xf8,0xc9,0x23,0xa1,0xba,0xf9,0xf8,0xe9,0xf8,0xeb, -0x47,0xf4,0x81,0x82,0x83,0x1a,0xa2,0x89,0x7f,0x9b,0xdf,0x3c, -0xa2,0x36,0xb3,0xc8,0xb9,0x7d,0x9b,0xfc,0xdb,0xec,0x6c,0x3f, -0xa2,0xfe,0x70,0xc5,0xe6,0x43,0x3c,0x29,0xe3,0xb7,0xe3,0xeb, -0x63,0x58,0x0c,0x74,0x88,0x00,0x19,0x1e,0xa4,0xa6,0x74,0xf6, -0xb7,0xac,0x23,0x3d,0x8e,0xdc,0x1a,0x36,0x0a,0xd4,0xcb,0x0e, -0x71,0x53,0x7f,0x4c,0xa1,0x54,0xac,0xee,0x3c,0x86,0x4b,0xac, -0x92,0x25,0xa1,0x02,0xce,0x7d,0x62,0x7e,0xec,0x1e,0x53,0x53, -0x2d,0x45,0x3c,0xe0,0xdd,0xa9,0x14,0x92,0x37,0x28,0x23,0x4f, -0xa9,0x19,0x7a,0xb2,0x87,0xc7,0xe5,0x7b,0x34,0xeb,0x6c,0xb0, -0x6d,0xb8,0x9b,0x5f,0xed,0x1d,0x4e,0xe8,0x79,0xfb,0x5b,0x81, -0xc8,0x94,0x82,0x40,0x92,0x3c,0xba,0x60,0xf6,0x73,0xae,0x38, -0xea,0x78,0x13,0x79,0xaf,0xe3,0xfb,0x3f,0x39,0xc3,0xe0,0x4e, -0x46,0x6e,0x09,0x89,0xfd,0xb0,0x2e,0xd4,0x4d,0x05,0x38,0x69, -0x27,0x4d,0x7e,0xb4,0x4d,0xc6,0x6f,0xea,0x73,0x46,0xa3,0x77, -0xb3,0xfd,0xeb,0xe3,0xf9,0x6c,0x78,0x32,0xb7,0x93,0xf2,0xd5, -0x6c,0x7a,0x29,0x53,0xfc,0x72,0x7a,0x69,0xdc,0x98,0x2a,0xca, -0xe7,0xec,0xa3,0x97,0x4f,0x4d,0x38,0x3e,0xff,0x3b,0x11,0x70, -0xf9,0x7a,0x23,0x8d,0x21,0x9e,0x1f,0xe1,0x18,0xb6,0x48,0xc2, -0x31,0xc9,0xb3,0xcc,0xe7,0xe1,0xab,0x9d,0xb7,0xaa,0x04,0x25, -0x74,0x75,0x41,0x74,0x6e,0x78,0x09,0xf0,0x2d,0xb1,0xfe,0x69, -0x34,0xf9,0x3d,0x18,0xe3,0x8f,0x2c,0xc1,0xa9,0x69,0x20,0xe6, -0xc5,0x82,0x31,0x51,0xbc,0x11,0x99,0xa4,0x58,0x8c,0x9f,0x1b, -0xa9,0xcc,0x6b,0x4c,0xc2,0x27,0x8c,0xc0,0x8a,0xc5,0xb8,0x2e, -0x8a,0x9c,0xe9,0x1e,0x52,0x9a,0x10,0xaa,0x02,0xc6,0xcc,0xa0, -0xf4,0xdc,0xf0,0x4f,0xa9,0x4c,0x11,0x0d,0x1e,0x7c,0xcb,0x7a, -0xb5,0x41,0x09,0x13,0x7a,0x01,0xbe,0x56,0x89,0x88,0xb3,0xee, -0x5b,0xfc,0x37,0xc8,0x67,0x21,0x3d,0xf2,0x11,0xf4,0x4d,0x3a, -0x72,0xb8,0x55,0xe5,0xf3,0x75,0xb7,0x95,0xba,0xf0,0x8a,0x03, -0xd4,0x56,0xa8,0xc1,0xe4,0xe2,0x1a,0x2d,0xaa,0xb1,0x57,0x60, -0xe5,0x03,0xc0,0x2f,0x47,0x4a,0xb7,0x06,0x67,0x63,0x22,0xaa, -0x64,0xcb,0xfb,0x0e,0xdf,0xac,0xd8,0x36,0xa3,0x29,0x46,0xab, -0x23,0x5d,0x97,0x0d,0x0b,0x05,0x73,0x21,0xb7,0x4e,0x25,0x04, -0xda,0x6e,0x21,0xb8,0x90,0x0b,0x0f,0xc3,0xbf,0xbb,0xc5,0xe9, -0x29,0x17,0x80,0x69,0x1d,0x16,0xd8,0x8c,0x0a,0xe0,0xc5,0x23, -0xb4,0xcc,0xb4,0x91,0xd4,0x20,0x5d,0x14,0x8e,0x0d,0x45,0x88, -0xdc,0x5a,0xe8,0x00,0x97,0x5a,0x97,0xeb,0x0f,0xdb,0xf0,0xbc, -0x58,0xac,0x07,0x40,0xf3,0xe9,0x6b,0x98,0x5b,0x0e,0x34,0x5d, -0x47,0xf5,0xb0,0x4b,0xc2,0x12,0x18,0x61,0x9d,0xaa,0x6c,0x36, -0xed,0xd4,0xe5,0x57,0x4f,0xb8,0x0a,0x56,0x6d,0xd7,0x07,0x5e, -0x1f,0xaf,0xf1,0x2f,0x29,0x9c,0xe4,0x89,0xfd,0x21,0x7c,0x1a, -0xcf,0xc6,0xa3,0x9c,0x92,0xa0,0xd5,0xdf,0xe0,0x3b,0x9e,0x7b, -0x36,0x54,0x18,0x92,0x46,0x2b,0x2d,0x1f,0x6c,0x71,0x58,0xa5, -0x2e,0x41,0x12,0xfc,0xf3,0x8b,0x02,0x00,0xcb,0x3e,0xa5,0xc0, -0xc7,0x9c,0x80,0x24,0x77,0x61,0xf7,0x2d,0x65,0x5f,0x50,0x43, -0x86,0x3c,0xd9,0x73,0x2b,0x82,0xaf,0x21,0x53,0x2f,0xe6,0xd4, -0xe3,0x9c,0x10,0x8b,0x00,0xc8,0xa0,0x49,0x81,0x66,0x62,0xb1, -0x40,0xee,0x22,0x59,0x1b,0x05,0x8d,0x43,0x3d,0xc7,0x7c,0x1e, -0xc1,0x71,0xba,0x84,0x35,0xe4,0x51,0xeb,0x11,0x94,0x09,0x61, -0x82,0x42,0x44,0x42,0x3f,0x04,0x18,0xe4,0xaa,0x5e,0x31,0x83, -0xf0,0x46,0x38,0x61,0x51,0xdf,0x20,0xe8,0x22,0xfd,0x4d,0x51, -0xce,0x1b,0xa7,0xc5,0xf8,0x62,0x64,0x0d,0x4c,0x40,0xde,0xa3, -0x9d,0xed,0x78,0xe6,0x0c,0xd1,0x31,0xdb,0xc7,0x20,0xc2,0x35, -0x18,0xb6,0xdc,0x04,0x99,0x7f,0x23,0xb0,0x79,0x02,0x5c,0x21, -0xe1,0xc0,0xf2,0x58,0x6c,0xbd,0x52,0x8c,0xf8,0x45,0x62,0x4f, -0xdc,0xba,0x5c,0xbc,0xb3,0xed,0xca,0x73,0xfe,0x92,0xd9,0x4e, -0x9f,0x0a,0x2b,0x71,0xfd,0x14,0x3f,0x62,0xae,0xfd,0x1a,0x14, -0xe6,0x8c,0x4a,0x8a,0xbc,0x9e,0x0b,0xbf,0x3a,0x05,0x86,0x8d, -0x72,0xa6,0x60,0x50,0x17,0x2c,0xbb,0xea,0x4a,0x95,0x3b,0xe6, -0x61,0xbc,0xe8,0x15,0x91,0x6f,0xf4,0x73,0x35,0xfc,0x93,0x7b, -0xf1,0x5a,0xea,0x52,0xf7,0x7f,0x65,0xc5,0x91,0x6a,0x61,0x14, -0x5f,0xcf,0x4f,0xe0,0x2f,0x72,0x49,0xfe,0x22,0xa5,0xea,0x97, -0x1d,0x05,0xb1,0x98,0x88,0x4c,0x73,0x29,0x95,0x51,0x64,0xfc, -0xa5,0x7c,0xa0,0x94,0xe3,0x4a,0x25,0x44,0x93,0x9b,0xfa,0xd4, -0x72,0xe1,0xdd,0x3b,0x3e,0xfb,0xb6,0xe0,0x69,0xe5,0xc2,0x9d, -0xef,0x73,0x3b,0xcf,0x79,0x1f,0xf3,0x6a,0x3a,0x7b,0x31,0x1d, -0xcf,0x4e,0x9c,0x00,0x7c,0x82,0x6f,0xe3,0x11,0x9f,0x46,0xe5, -0xcd,0x1a,0x38,0xde,0xb1,0xe4,0xce,0x05,0x1c,0xf6,0x74,0x50, -0x6a,0x63,0xc3,0x1d,0x8a,0x05,0xe9,0x38,0xf4,0xb9,0x6c,0x90, -0xbc,0x93,0xb7,0x7b,0x4a,0x42,0x09,0x8b,0x64,0x5e,0x9a,0x09, -0x33,0x52,0x26,0xc3,0x7f,0x17,0xae,0x49,0xaf,0x27,0x7b,0xb4, -0xde,0xa1,0x18,0xc8,0x0b,0xa2,0xd7,0xc7,0x8b,0x5d,0x71,0x03, -0xff,0xb4,0x7e,0x3a,0x78,0xa1,0x80,0x9a,0x1d,0xbc,0x7e,0xde, -0x82,0xd6,0x83,0xf4,0x5d,0x0e,0x50,0x1d,0xe1,0xfd,0x27,0x00, -0x53,0x09,0x75,0xf0,0xf7,0x66,0x3c,0xbc,0xf2,0xe9,0xbd,0xc7, -0x4f,0x9f,0x66,0x8f,0x9f,0x7e,0xad,0x17,0xa4,0x10,0xe2,0x46, -0x9d,0x52,0x63,0xc2,0x57,0x4f,0x36,0x36,0xb0,0xf1,0xe8,0xb9, -0xd3,0x6e,0x83,0x28,0xed,0x72,0xda,0x6d,0xb5,0x06,0x05,0xbb, -0x9b,0x4b,0xd9,0xad,0xcb,0x7b,0x17,0x84,0x8e,0xe1,0xc6,0xc6, -0xe6,0x36,0x14,0x85,0xc7,0xab,0xd3,0x96,0xcd,0x41,0xfb,0x28, -0x79,0xc4,0x9c,0x61,0x7f,0xef,0xdd,0xdb,0x83,0x1f,0x06,0xcf, -0xa0,0x1e,0xf5,0xba,0xd3,0xe9,0xf4,0xe8,0x29,0xa3,0xc4,0xdd, -0xe7,0x07,0x2f,0x07,0xcf,0x3a,0x02,0x42,0x6f,0x3b,0xba,0x65, -0x29,0xf9,0x9f,0x2f,0x9f,0xbf,0x1f,0x18,0x85,0x24,0xed,0x21, -0x24,0x63,0xf8,0xe1,0xdd,0x4f,0xef,0x7d,0x25,0xf4,0xb6,0xb3, -0xfd,0xb8,0x87,0x0f,0x0c,0xb0,0xf7,0xfa,0xed,0x4f,0x58,0x89, -0xa3,0x83,0xdf,0x77,0xbe,0xfe,0xb6,0xc7,0x8f,0x0c,0xb6,0xff, -0xf2,0xc5,0xbb,0xb7,0xbb,0x1e,0x4c,0xde,0x11,0x8c,0x1f,0x2d, -0xb6,0x37,0x6f,0x5e,0xc7,0xb0,0x3a,0x71,0xe7,0xdb,0x6f,0x11, -0xb1,0x4b,0xc8,0x36,0x3b,0x86,0x1b,0xea,0xc8,0x36,0xcf,0xee, -0xf0,0xf3,0xbb,0x53,0xec,0x27,0x68,0x32,0xfc,0xb6,0x1d,0x9b, -0xb1,0x58,0xd8,0xa7,0x1d,0xfc,0xc0,0x54,0x37,0x29,0x25,0x98, -0xb0,0xc9,0x73,0xfb,0xa0,0xc5,0x12,0xbe,0xc0,0x0d,0xdc,0x17, -0x23,0x0c,0x81,0xa0,0xf1,0xea,0x62,0x78,0x56,0xfa,0xad,0x2b, -0xfd,0x22,0x96,0xfc,0x86,0x7c,0x05,0x67,0xeb,0x1d,0x73,0x3d, -0xb9,0x2e,0xc7,0xbc,0x89,0x2f,0xb3,0xfe,0x40,0xde,0x89,0x79, -0xe3,0xab,0xad,0x25,0xdb,0xdc,0xe6,0x90,0x4a,0x6f,0x40,0x88, -0xc7,0x01,0x92,0xb5,0xc9,0xfb,0x01,0x03,0x02,0x9e,0x82,0x4d, -0x80,0xa9,0xeb,0x32,0xb6,0x87,0xe7,0x94,0x57,0xc4,0x71,0xa8, -0xaa,0x72,0x3c,0x13,0x4b,0x61,0x0c,0x58,0x0c,0x49,0x4b,0x3d, -0x82,0xff,0x81,0x19,0x8d,0xcb,0xf8,0x28,0x12,0x76,0x7b,0x9c, -0x85,0xc2,0x9f,0x7b,0xc9,0xe5,0xf4,0x9b,0xe4,0xe2,0x33,0x91, -0x8b,0x51,0x90,0x08,0x5b,0xe9,0x19,0x46,0xc0,0xe2,0x24,0xfa, -0x62,0xf7,0xa6,0xc9,0x75,0x89,0xee,0x83,0x62,0xb0,0x57,0xc2, -0xb8,0x25,0x35,0xfa,0x10,0xec,0x19,0x0e,0x5a,0x1b,0x92,0xa8, -0x49,0x6f,0xbb,0x39,0x13,0xb4,0xa2,0xce,0xd0,0x3d,0x61,0x45, -0x31,0xea,0x75,0x41,0x52,0x23,0x5c,0x38,0x85,0x85,0x3e,0xee, -0x86,0x67,0xf4,0x46,0xbb,0x42,0xee,0x41,0x05,0x6b,0xb2,0x99, -0xa4,0x19,0x80,0xa8,0xad,0xd9,0x74,0x38,0x42,0x64,0x3e,0x22, -0x70,0x19,0x9e,0x5f,0x97,0x2d,0x0c,0x6d,0x82,0xd6,0xbe,0x2e, -0x8e,0x08,0xfb,0xce,0x5b,0x2c,0x1a,0x61,0x02,0x29,0x7a,0x2d, -0x61,0x69,0x04,0x4d,0x4b,0x96,0xe0,0x8f,0xb2,0x3c,0x31,0xd7, -0x13,0x4d,0x8e,0x0b,0x30,0xbc,0x0a,0x1c,0xb7,0x82,0x90,0x45, -0xe1,0x2a,0x0a,0x6f,0xff,0x8a,0x9c,0xeb,0x57,0x8e,0x7c,0x32, -0xc1,0xc3,0x1d,0x8a,0x10,0x4c,0x76,0xce,0x18,0xb8,0xc2,0xdb, -0xca,0xf2,0x7d,0x46,0x85,0x7c,0x80,0x11,0x09,0x4b,0x0e,0x60, -0x9f,0xb2,0x02,0xfb,0xb7,0xeb,0x62,0x06,0x4d,0xd7,0xda,0x42, -0xb0,0xad,0xa4,0xf9,0x51,0x29,0xa7,0x9d,0x6f,0x1a,0x85,0x61, -0x49,0x77,0x30,0x91,0x0c,0xbb,0x28,0x87,0x0b,0x31,0x3b,0x7d, -0x93,0xbd,0x09,0x99,0x06,0x21,0x11,0x98,0x93,0x03,0x71,0x0d, -0x65,0x2c,0x44,0x68,0xbb,0x78,0x75,0x4d,0xbe,0x1a,0x6d,0xc2, -0x8a,0x67,0xd8,0xbf,0x62,0x79,0xc4,0xbb,0x66,0xfa,0xba,0xbc, -0x76,0x60,0xa0,0x84,0x6d,0xe3,0x23,0x43,0xc7,0x9b,0x5f,0x73, -0x7a,0xb1,0x9b,0x05,0x6c,0x9c,0x55,0x05,0x9b,0x1d,0x74,0x05, -0x4d,0x00,0xf0,0xa7,0x37,0xa1,0x83,0x1c,0x87,0x89,0xa6,0x79, -0xf7,0xd7,0x9d,0x76,0x37,0x24,0x9f,0xd1,0x5b,0x87,0x65,0xbf, -0xda,0x43,0x70,0x28,0x11,0x7e,0x15,0x9e,0x12,0x93,0x97,0x68, -0xc2,0x2b,0xa6,0xe6,0xf9,0xaf,0x1b,0x1b,0xe1,0x4e,0x88,0x3b, -0x8e,0xfa,0x70,0xbd,0x9d,0x02,0xc0,0x66,0x47,0x0c,0xc8,0x7e, -0xdd,0xdc,0x5c,0x82,0x60,0xb2,0xac,0x6f,0x64,0x3f,0x42,0x38, -0x6c,0x35,0x4f,0x62,0xd4,0xd8,0xf3,0x1c,0x0b,0xb7,0x6f,0x2c, -0x58,0x4b,0xf8,0x92,0xad,0xc3,0x3e,0x9f,0x62,0x6e,0xa5,0x3d, -0x4e,0x77,0xdb,0x86,0x5f,0x0e,0xfb,0x8b,0xc3,0xc1,0x03,0x8c, -0x61,0x08,0x2d,0x10,0x65,0x1e,0x1e,0x72,0x7a,0x28,0xf6,0x72, -0xc5,0xaf,0x42,0xe9,0x52,0x6d,0xe3,0xdc,0x96,0x4d,0x24,0x42, -0x21,0xe2,0x34,0xa2,0xd7,0x09,0x65,0xb5,0xf7,0xf9,0x03,0x4b, -0x7d,0x77,0x6d,0x9b,0x71,0x50,0x79,0xbf,0x87,0xb1,0xb9,0x83, -0xde,0xed,0xd9,0xd9,0x8a,0x76,0xb3,0x00,0xd5,0x5b,0xac,0xb8, -0x65,0xbf,0x51,0x0a,0x9c,0x24,0xd1,0x7b,0x48,0x4f,0x1f,0x67, -0x37,0x73,0x8b,0x49,0x99,0x5f,0x5a,0x3a,0x7a,0x36,0xcf,0x49, -0xb2,0x22,0xbb,0xca,0xcd,0x76,0x75,0xe7,0x40,0xf4,0x3c,0xbe, -0xd1,0x19,0xd0,0x6a,0x75,0x6c,0x39,0xdb,0xd1,0x97,0x2d,0xbb, -0xd6,0xa4,0x3d,0x69,0xe4,0x7c,0xfc,0xc7,0xd5,0x70,0x22,0x6c, -0xde,0x8a,0xff,0x97,0xa2,0x42,0x4e,0x05,0x81,0x6f,0x21,0x7e, -0x47,0xd6,0xb7,0x22,0x27,0x5f,0xdd,0xed,0x2b,0xb1,0xc1,0xca, -0x97,0x66,0xb6,0x4e,0xbb,0xf0,0xc4,0x12,0x62,0x1d,0x9d,0x58, -0x42,0xa9,0xe4,0x65,0x30,0xbe,0x99,0x02,0x4b,0x84,0xd2,0xaf, -0x7c,0x17,0xc7,0x23,0x1e,0x0b,0xb6,0x2e,0x02,0x38,0x81,0x58, -0x2c,0xe8,0x97,0x83,0x36,0xe7,0x5f,0x53,0x27,0x5e,0x4c,0xa1, -0x23,0xe2,0xb1,0xe0,0xfd,0x0b,0xe0,0x4d,0x27,0x0a,0x03,0x5c, -0x0f,0x47,0xf1,0x6f,0xa5,0x09,0xba,0xa9,0xb4,0xba,0x0c,0x77, -0x3b,0x7d,0x6a,0x4b,0x9a,0x5b,0xbe,0x07,0x5d,0x91,0xdd,0x41, -0x94,0x29,0x36,0x73,0x7f,0x89,0x8d,0x00,0x83,0x95,0x83,0x42, -0x31,0x92,0x59,0x0c,0x60,0xe1,0x43,0x86,0x39,0xfe,0xb5,0x5a, -0x7c,0xb6,0xf4,0xb0,0xa6,0xe5,0x94,0x25,0x66,0xe5,0xbb,0xf0, -0x3f,0x67,0xac,0x4d,0x46,0x78,0x54,0xfc,0xe0,0x7c,0x36,0x1e, -0xef,0x16,0x67,0xb0,0xbd,0x66,0xfb,0xee,0x7f,0xc2,0xff,0xc4, -0x1c,0xfc,0x7b,0xf8,0x9f,0x3c,0x9e,0xc1,0xff,0x6a,0x4a,0xbf, -0x9a,0x5e,0xcf,0xe2,0xc2,0xba,0xb4,0x2e,0x5e,0x57,0x7e,0xbf, -0xf8,0x43,0x17,0xdf,0x17,0xf0,0x7d,0xff,0x60,0x9f,0xea,0xa9, -0x7f,0x37,0x81,0xdf,0xca,0x27,0xec,0xed,0xed,0x49,0xa9,0x3d, -0xff,0x38,0x1a,0xb9,0x07,0xf5,0x34,0xaa,0x41,0xfa,0x61,0x3a, -0x1b,0x31,0xa2,0xa1,0x00,0x3e,0x77,0x50,0xd5,0xc5,0xdb,0x4a, -0xa9,0x17,0x69,0xeb,0xe8,0x72,0x3c,0x2b,0x46,0xc5,0xf8,0x92, -0x7a,0x89,0x51,0xfc,0x57,0x5d,0x9b,0xdb,0xfb,0x9b,0x7b,0x42, -0xf0,0xcf,0x52,0xcf,0xcf,0x3f,0xaf,0x80,0xfe,0x73,0x3a,0x11, -0x7c,0x07,0x75,0x10,0xf6,0xb3,0x5d,0x5b,0xc9,0x83,0xea,0x0b, -0xd7,0x11,0xf2,0xf0,0xc3,0x0f,0xf2,0x70,0x7e,0x2e,0x0f,0x97, -0x97,0xf2,0x50,0x96,0xb6,0xf5,0x2c,0x3e,0xdb,0x5e,0xb6,0xac, -0x2d,0x6a,0x4b,0xda,0x82,0xf2,0xfb,0xc9,0xfe,0xda,0x87,0x0f, -0xf6,0xd7,0x3e,0x8c,0xe5,0xf7,0x65,0x7d,0x9f,0xbe,0x9b,0x1d, -0x7c,0x9a,0x4a,0x8f,0x8a,0xfd,0x8a,0xbf,0x05,0xa1,0x6c,0x53, -0x66,0x38,0x11,0xae,0x5e,0xb2,0x95,0x84,0x35,0x4a,0x13,0xce, -0x40,0x63,0xdf,0x8b,0x91,0x64,0xac,0x83,0x41,0xb1,0x30,0xe4, -0xb9,0x3e,0x44,0x9b,0x4b,0xf3,0xee,0xb1,0x76,0x16,0xf5,0xeb, -0x62,0x3e,0x6d,0xcd,0xd6,0xe8,0x60,0xfe,0xcf,0x93,0xf3,0xeb, -0xc9,0xc7,0xbc,0x51,0xed,0x97,0xd6,0xf8,0x8f,0xf1,0x89,0x05, -0x5e,0x2c,0xfa,0x83,0x14,0x0f,0x70,0xd1,0xd1,0x42,0x99,0x37, -0xa4,0x5c,0x13,0xaa,0x96,0x65,0x92,0x11,0x48,0xd9,0x17,0x98, -0x09,0x5b,0x76,0x28,0x06,0xf2,0x86,0x69,0x1b,0x8c,0x35,0x26, -0xda,0xef,0x66,0xe3,0x69,0xfb,0x11,0xa1,0xe9,0x83,0x58,0xd3, -0x64,0xe5,0x04,0xbf,0x6f,0xbb,0xb5,0x2c,0x69,0xa2,0xe6,0x96, -0x53,0xdb,0x83,0xde,0xa6,0x14,0xce,0xe4,0x37,0x38,0x4c,0xc0, -0x4a,0x0f,0xd8,0x15,0x21,0x7e,0xa5,0xe6,0x1e,0xec,0x51,0x2d, -0xe0,0x21,0x68,0x5b,0x80,0xfb,0xb2,0x39,0xdb,0x6c,0xdb,0x61, -0x5e,0xcb,0x5b,0xfc,0x74,0x4b,0x32,0xf6,0x1d,0x53,0xf0,0xb6, -0xa4,0x11,0x20,0x91,0x2d,0xb1,0xe8,0x53,0x99,0x75,0x83,0x70, -0xd4,0x65,0xe9,0xa8,0x76,0xea,0x0e,0xf3,0x5b,0x67,0x1c,0xeb, -0xd9,0x69,0xba,0x09,0x3e,0xc3,0xf5,0x0f,0x7b,0xb5,0x35,0x0f, -0x33,0xbf,0xa9,0x0c,0xf6,0x54,0x4c,0xb0,0xa6,0x64,0x57,0x4d, -0xa6,0xdb,0xbe,0x89,0x14,0x01,0xc1,0x27,0x05,0x1f,0xb4,0xbb, -0xeb,0x11,0x55,0x51,0x59,0x6a,0x46,0x76,0x9b,0xbd,0x1a,0x11, -0xce,0xe5,0xb0,0x66,0xd2,0x35,0x04,0x05,0x9a,0x0d,0xfd,0xb6, -0xf3,0xf4,0x6f,0xbd,0xce,0xb7,0xed,0x76,0xb6,0x3d,0x7e,0x9c, -0x76,0xff,0x57,0x80,0xcb,0x71,0x06,0xe1,0xd8,0x77,0x61,0xd6, -0x94,0x28,0xae,0x68,0xe9,0x2f,0xca,0x1f,0x2f,0x6f,0xef,0x29, -0x80,0xd8,0xab,0xc1,0xf5,0x43,0x85,0x1d,0x29,0xb6,0x14,0x12, -0x45,0xfa,0x92,0x95,0x44,0x5d,0x2a,0x2e,0x16,0x75,0x3d,0x2b, -0x51,0x56,0x96,0x2c,0x15,0xdb,0x0b,0x4b,0x8a,0x9a,0x64,0x65, -0xc9,0xfa,0xe5,0x2a,0xae,0xdd,0xeb,0x5b,0x04,0x11,0x9e,0xee, -0x36,0x92,0x76,0x0b,0x0d,0xb5,0x2a,0xdd,0xfc,0x5f,0xbe,0x51, -0x47,0xee,0xc6,0x3f,0x15,0x51,0x86,0xf3,0x35,0xe5,0xf4,0x12, -0x62,0x11,0xc0,0xfe,0xfc,0xa7,0x83,0x17,0x68,0x4f,0x61,0x53, -0xe6,0x7f,0x5e,0xe6,0xab,0x19,0x5e,0xf5,0x0b,0xef,0xe6,0xe4, -0xa3,0xbb,0x96,0xd7,0x98,0xdf,0x13,0xdf,0xc8,0x99,0x3b,0xbb, -0x78,0x88,0x1d,0xb9,0xd1,0xe6,0x96,0x27,0x96,0x17,0xc2,0xb5, -0x4b,0xc9,0x13,0x22,0x5b,0xd4,0xe2,0xc2,0xf8,0x03,0xd1,0xf4, -0xfa,0xe4,0xd8,0xd7,0x27,0xb4,0x39,0x73,0xcd,0xf1,0xa9,0x4f, -0x65,0xe5,0x84,0x40,0xaf,0x0a,0xd8,0x89,0xd8,0x32,0x2f,0x08, -0xb2,0xa1,0x39,0x63,0x41,0x9c,0xd1,0x9c,0x5c,0xcf,0x66,0x20, -0xaf,0x93,0xfb,0x0e,0x54,0x4c,0x1e,0x4c,0x7f,0x2a,0x31,0x24, -0xef,0x1f,0x38,0x8d,0xcd,0x27,0xb2,0xa0,0xe2,0xed,0x3c,0x9e, -0x09,0xc0,0x04,0xa7,0x5f,0x26,0x4d,0x14,0xee,0xeb,0xae,0xa7, -0x79,0x53,0xac,0x50,0xe6,0xea,0x99,0xb7,0xdd,0xd6,0x74,0xc3, -0xd1,0x0e,0xfb,0x50,0xd2,0x38,0x39,0xce,0xcc,0xac,0xa8,0x9a, -0xcc,0xbc,0x0f,0x9a,0x43,0x88,0x0b,0x0c,0xff,0xb5,0xda,0x04, -0x6d,0x96,0xed,0x99,0xf5,0xe3,0x1e,0x8d,0x37,0x1c,0xaf,0x90, -0x6a,0x3a,0x6d,0x62,0x27,0x49,0xe7,0x5b,0x0a,0xa4,0x9f,0x25, -0xdb,0x6d,0x7e,0xc0,0x7f,0x71,0x85,0xc4,0x3e,0x7a,0x4e,0xf9, -0x8f,0xdf,0x4d,0xaa,0xde,0x34,0x8b,0x60,0x96,0x46,0xf5,0x8c, -0x70,0xee,0x4f,0xad,0xef,0xbf,0xe7,0x5b,0x9b,0xf8,0xfc,0xc1, -0x3f,0xbe,0xec,0x61,0x93,0xe6,0x8e,0x55,0x62,0xf7,0x7c,0x40, -0x93,0x07,0xfb,0x5d,0x0d,0x2c,0x9a,0x1a,0x2a,0xd4,0x81,0x9f, -0x97,0xe6,0x09,0x0c,0xac,0xcc,0x6d,0xf4,0x57,0x32,0x27,0xdb, -0x43,0xb9,0xad,0x6a,0xc4,0x1f,0xff,0x81,0x93,0x01,0xef,0x88, -0x77,0x2d,0x99,0x05,0x18,0xfb,0xd6,0x81,0x17,0x6c,0x9d,0x26, -0xed,0x4e,0x8e,0x10,0x53,0x6b,0x34,0xfd,0x94,0xb1,0x09,0x4b, -0xae,0xc0,0x3e,0x21,0x18,0x52,0xe6,0x6a,0xd9,0xd8,0x90,0x8a, -0x9f,0x85,0x85,0x39,0xbd,0xd9,0x34,0x77,0x7f,0xf0,0xd9,0x19, -0x53,0xef,0x06,0x59,0x80,0x29,0x7a,0xfd,0x94,0xfa,0xe1,0x33, -0xb4,0x4c,0x1e,0xaa,0x20,0xab,0x1c,0x53,0xb3,0x16,0x61,0x9e, -0x7b,0x4d,0xab,0x10,0x30,0xa6,0xdc,0xd0,0xcf,0xeb,0x47,0x81, -0x1a,0xc7,0x9c,0x12,0x8f,0x81,0x2a,0xd6,0x9d,0xe8,0x5c,0x81, -0xd0,0xa7,0x6a,0x3a,0xd7,0x2a,0xa8,0x31,0xc2,0x14,0xcd,0x4c, -0xda,0xcc,0xda,0xd3,0x0a,0x3f,0x33,0xdb,0xd5,0x8a,0x74,0x6b, -0x88,0x80,0x80,0x08,0xe4,0xb4,0x83,0xb5,0xfa,0x1a,0x86,0x97, -0x7a,0x05,0xc2,0xdb,0x5d,0xbe,0xe3,0xf2,0x78,0xa7,0xa8,0x4e, -0x3d,0xd8,0xf4,0x37,0x9b,0x45,0xaa,0x13,0x98,0xd7,0xe0,0x83, -0x6e,0x9a,0x82,0x3d,0x0a,0x74,0xbf,0x11,0x8d,0x43,0x7d,0x89, -0x2a,0xfa,0x1e,0x1e,0x6e,0x14,0xbd,0x4e,0xd6,0xce,0x82,0x5a, -0xb9,0x08,0x2d,0x99,0x4d,0x59,0x76,0x1a,0x6a,0x11,0x58,0x2c, -0xda,0xe9,0xd6,0xd3,0xb6,0x70,0x4a,0xbb,0x44,0xae,0x82,0xfc, -0x0a,0x21,0xfd,0xaa,0xd4,0x08,0xd7,0x97,0x9e,0x6a,0xef,0x0c, -0x9f,0xf1,0x21,0xd5,0xfe,0x9e,0x56,0x72,0x57,0xb9,0x99,0xa7, -0xb9,0x6b,0x74,0x20,0xdb,0x75,0xd5,0x2e,0x16,0xf1,0x39,0x6f, -0xbd,0x05,0x9a,0x13,0x49,0x54,0xcf,0xe5,0xfd,0x3a,0x3b,0xb4, -0xf8,0xe8,0x9a,0xa5,0xc9,0x4a,0x2a,0xce,0xaa,0x3a,0x73,0xb2, -0x6a,0x71,0x5a,0x50,0x2b,0xc9,0x6c,0x49,0x52,0x03,0xed,0xac, -0x4c,0x06,0xa6,0x7e,0xb9,0xd1,0xc6,0x16,0x2b,0x96,0x83,0xaa, -0xe7,0x27,0xef,0xa8,0x49,0xf7,0x11,0x34,0xc0,0x27,0x19,0xb4, -0xaf,0xa0,0x4b,0x98,0x1d,0x1b,0x9f,0x68,0x07,0xbb,0x4f,0xe1, -0xb1,0x3d,0xc8,0x6c,0xc9,0x6a,0xb1,0xa8,0x8c,0x14,0x08,0x75, -0x89,0xbb,0xf2,0x61,0x2c,0x61,0x3c,0x77,0xba,0xa0,0xf8,0x8c, -0x67,0x88,0x4e,0x7e,0xd5,0xfc,0xa6,0xd3,0x0f,0xbc,0x9b,0xcc, -0x2b,0x2e,0xb1,0x51,0x6e,0x38,0x33,0x67,0x3d,0x0b,0x6f,0x5c, -0xca,0x8f,0x05,0x19,0x9f,0xde,0xcd,0xe3,0x79,0x83,0x96,0x27, -0x49,0xd3,0x8d,0x10,0x49,0x63,0xc3,0x2c,0xb9,0x90,0x64,0xd5, -0xcc,0xf3,0xe9,0x7c,0x78,0xf1,0xa3,0xaf,0x56,0x80,0xd8,0xc8, -0x84,0x49,0x08,0xb5,0x70,0x16,0xeb,0x29,0x2f,0x15,0xab,0x94, -0xa2,0xb8,0x41,0x24,0x7e,0x51,0x3c,0x9b,0xeb,0xa3,0x13,0x9a, -0xf5,0x4a,0x9e,0x21,0xdf,0x29,0xea,0xb3,0xf3,0xc6,0x3d,0xb4, -0x3f,0x2b,0xb7,0xa2,0x16,0x0b,0x5a,0xfb,0x73,0x8b,0xd9,0xcf, -0x75,0x62,0x93,0xbc,0x5b,0x93,0x04,0x55,0x08,0xa3,0xe3,0x73, -0x21,0xab,0xf4,0x6e,0xa3,0xce,0xdb,0xf5,0x95,0x3a,0x82,0xe3, -0xb3,0x76,0x81,0x76,0x6d,0x6e,0xab,0x22,0x0d,0xfb,0x2d,0xf5, -0x34,0xd5,0x8b,0x33,0xee,0xa9,0xef,0x88,0x66,0x5e,0x07,0x5b, -0xab,0x1f,0x66,0x11,0xaf,0xa7,0x6b,0xea,0x55,0x47,0x5a,0x27, -0xab,0x7c,0x91,0x68,0xe7,0xe8,0x93,0x78,0xb7,0x6b,0x6e,0xdf, -0x45,0xeb,0x51,0x6a,0xa7,0xb0,0x43,0x6b,0x4f,0xdc,0xd6,0x83, -0xee,0xb8,0x47,0xa5,0x5d,0x05,0x13,0x1c,0xc6,0xe5,0x7a,0xf8, -0x6e,0xd6,0xb7,0x93,0x09,0x46,0xf5,0x9d,0x1d,0x27,0x66,0x98, -0x7a,0x5b,0xe7,0x4b,0xd8,0x03,0xec,0xce,0xb6,0x5a,0x8b,0x87, -0x76,0x91,0xe9,0x6c,0x87,0xe5,0xe8,0xba,0xed,0xc6,0x46,0x07, -0x17,0xa8,0x08,0xb8,0x5a,0x3c,0x97,0xb5,0xbb,0xca,0x03,0x4d, -0xad,0x01,0xb3,0x3e,0x7c,0x0b,0xb4,0x3e,0xfe,0x18,0xb0,0xd4, -0xc7,0x1c,0x8d,0xc3,0x7e,0xba,0xc0,0x9f,0x01,0xfc,0xf4,0x1b, -0xfd,0x5f,0x0e,0x07,0x87,0xfd,0xc1,0xa3,0xf4,0x70,0x80,0xa9, -0xad,0x74,0xeb,0x2c,0xba,0xcc,0x03,0x2c,0xe5,0xaa,0x63,0xae, -0xb6,0xcd,0xd5,0x63,0x73,0xf5,0xc4,0x21,0xbd,0xea,0x2c,0x16, -0x57,0xdb,0xf0,0xef,0x31,0xfc,0x7b,0xb2,0x4c,0xf5,0x99,0x8d, -0x52,0x45,0xd5,0x52,0xd1,0xdf,0x3c,0x04,0x4a,0x7e,0x79,0xf0, -0xa8,0xd9,0x6b,0x35,0xd2,0x45,0xff,0x70,0x70,0xb3,0x1c,0xe0, -0xf9,0xcb,0xe1,0xe1,0x83,0x8d,0xf8,0x0c,0xa6,0xc2,0x37,0xab, -0x5c,0x1f,0xa5,0x33,0x6e,0x2b,0x73,0x3c,0x2e,0xe5,0xf8,0xc0, -0xd0,0x5d,0xdf,0x83,0xe9,0x77,0x63,0x34,0x08,0xb2,0x9b,0x94, -0x7d,0x4c,0xb4,0x16,0x4b,0x8e,0x51,0xd9,0x69,0x13,0xad,0x18, -0xf1,0x31,0xb2,0xde,0x3f,0xaa,0x2d,0x69,0x7b,0x0b,0xfa,0x4c, -0x6e,0x48,0xda,0x43,0x93,0xe2,0x59,0x8c,0x9c,0x85,0x19,0x45, -0x45,0xde,0x36,0x9e,0xf0,0xdc,0x3b,0x8f,0xb1,0xdd,0xbd,0xc2, -0x26,0xc0,0x97,0x49,0x55,0x79,0xa8,0xc7,0x7f,0x0e,0x32,0xcc, -0xdb,0x96,0x1c,0x8d,0xc2,0x9e,0xa8,0xa8,0x34,0x1c,0x94,0x8a, -0xce,0x66,0xae,0xab,0x89,0xa7,0x9e,0x09,0x41,0x3b,0xed,0x47, -0x11,0x74,0xcd,0xe1,0xb8,0x89,0x40,0x38,0x00,0xbe,0x46,0x64, -0xe4,0x16,0xaa,0xea,0xc3,0xc5,0x42,0xbd,0xec,0x68,0x60,0x72, -0x87,0xe1,0xf3,0x42,0x44,0x7e,0x40,0xa8,0xcf,0x48,0xd3,0xae, -0xb4,0xf7,0x49,0x3c,0x6c,0x16,0x0b,0x05,0x76,0xdb,0x48,0x8c, -0x76,0xc2,0xc2,0xe0,0xfd,0x8a,0x4a,0x93,0x27,0x2f,0xca,0x29, -0xad,0x52,0xc1,0x7a,0x84,0xe3,0x4f,0x2e,0xca,0xf1,0x78,0x79, -0xd2,0x2d,0x76,0xd8,0xdf,0x9c,0xcd,0xc1,0x43,0x3a,0x27,0x12, -0x9c,0x22,0x1a,0xac,0x1d,0x96,0xc2,0xcd,0xce,0xa0,0x29,0x20, -0x4f,0x07,0x8b,0x45,0xb2,0x96,0x88,0x9a,0x62,0x69,0x87,0xde, -0x13,0x15,0xab,0xbb,0x9c,0x92,0x86,0x9d,0x9d,0x94,0x07,0x34, -0x28,0xec,0xcd,0x5c,0xc3,0xb5,0x07,0x82,0xef,0x0e,0xfd,0xae, -0x62,0x5d,0x80,0x21,0x59,0xfb,0x39,0x49,0x6f,0x1d,0x74,0x96, -0x5f,0x51,0x10,0xf2,0x9a,0x59,0x24,0x68,0xeb,0x9b,0x9c,0x38, -0x73,0xd8,0xe2,0x24,0x0a,0x44,0xcd,0x0d,0x8b,0xf9,0xb0,0xbc, -0x7a,0x3b,0x9e,0xff,0xbd,0x9c,0x4e,0x54,0xbb,0x8b,0x3a,0x87, -0xcb,0xe4,0xde,0x60,0xaf,0x57,0xa5,0x23,0x13,0x44,0x35,0x59, -0x8d,0xa6,0xe4,0xa1,0x16,0x3a,0x4b,0x98,0x60,0x6f,0x23,0x48, -0xd8,0x7b,0xab,0x07,0x4a,0x16,0x9a,0x34,0xf7,0x3c,0xe7,0xe7, -0x6d,0x8d,0x3d,0x77,0x5f,0xc5,0xff,0x11,0x81,0x32,0x5e,0xae, -0x25,0x50,0x82,0x83,0x58,0x6f,0x40,0x11,0x6d,0xf5,0x9b,0x8d, -0x3a,0x2d,0x5b,0x6c,0xe1,0x6c,0xbf,0xaa,0xf1,0xd9,0x5c,0x9a, -0x91,0x39,0x37,0x7b,0xa6,0x34,0x97,0x62,0xb4,0x4b,0x5b,0x4d, -0x57,0x34,0x82,0xb0,0x52,0x78,0xe7,0xdb,0x6f,0xda,0x3b,0x9f, -0x37,0x36,0x68,0xcf,0x58,0x2a,0x21,0xfa,0x33,0x7f,0x6f,0x58, -0x97,0xdb,0xb4,0xd6,0xd5,0x60,0x8d,0xf3,0x22,0x47,0xba,0xf6, -0xca,0xd5,0xaa,0x1a,0xb5,0xc0,0x5f,0xa9,0x34,0xd0,0x72,0xf0, -0x19,0x80,0x35,0x0b,0x61,0x27,0xfb,0xb5,0xbd,0xcd,0x73,0x0c, -0xad,0x99,0x44,0x0f,0x89,0xa0,0x3c,0xca,0x6c,0xe9,0x96,0xa8, -0x23,0x42,0xd5,0x7c,0xd5,0x67,0x3f,0xa5,0x2b,0xfb,0x29,0x9e, -0x27,0x8c,0xcb,0xa9,0x4e,0x98,0xb0,0x4e,0xdb,0x7d,0x63,0x11, -0x9a,0xea,0x92,0x08,0x5b,0xcc,0x61,0xef,0x85,0x53,0xf6,0xf9, -0xd9,0x54,0x66,0x95,0x91,0x3b,0x1b,0x12,0x4e,0x65,0xff,0xfa, -0xf4,0xb4,0xf8,0x83,0x62,0x93,0x02,0x96,0xb1,0x9c,0x43,0xeb, -0xb3,0xe5,0xd9,0xf8,0x82,0x3c,0xd3,0x90,0x71,0x16,0x17,0x46, -0x7d,0xcd,0xfa,0x7a,0x88,0x41,0xb0,0x5b,0x44,0x81,0x10,0xa0, -0x10,0x04,0x77,0x29,0x42,0x0c,0x5c,0x35,0xfb,0x07,0xe7,0x2b, -0x18,0x33,0xba,0xb7,0xe2,0xcc,0xdd,0x75,0xd9,0x74,0xab,0x33, -0x7e,0x9c,0xba,0x63,0x21,0x86,0x94,0x3c,0xda,0xc3,0xf3,0xcd, -0x08,0x4e,0x17,0x28,0x4a,0xa7,0xfb,0x1b,0x9c,0x4c,0x20,0x5b, -0xdb,0x4f,0xec,0xb5,0x2c,0x4e,0x45,0x80,0xad,0xc7,0x4f,0xbf, -0x4e,0xd9,0x35,0xed,0x93,0xaf,0x77,0xdc,0xad,0x87,0x7e,0x52, -0x26,0xf6,0x2a,0x14,0xba,0xb4,0x47,0xab,0x2f,0x46,0x8d,0x79, -0x97,0x09,0xa4,0x01,0xb8,0x4e,0xba,0x4c,0x2c,0x89,0x02,0x4f, -0x75,0x62,0xd6,0x39,0x42,0x6f,0x6f,0xef,0xf8,0x84,0xf3,0x84, -0x89,0x16,0x48,0xbe,0xc8,0xd1,0x4f,0x46,0x04,0xf8,0xf5,0x8e, -0x4f,0x18,0x25,0xf4,0x15,0x5c,0x9b,0x4b,0xde,0x43,0xb8,0xc7, -0x90,0xe2,0x12,0xf6,0x12,0xa3,0x3f,0xa9,0x9d,0x0a,0x66,0xfa, -0x5a,0x84,0xf8,0x8c,0x45,0xe0,0xe7,0x73,0xc2,0x2d,0x10,0xba, -0x30,0xdd,0x1e,0xe4,0x61,0x07,0x51,0xea,0xe3,0x41,0xae,0xfb, -0x61,0xa7,0xcd,0xc9,0x4f,0x06,0x34,0xcc,0x4d,0x65,0xd8,0xc9, -0xcc,0x04,0x61,0x86,0x7d,0xb6,0xba,0x61,0x81,0x93,0x81,0x35, -0x4e,0x6c,0x17,0x82,0xb1,0xb2,0x48,0x6f,0x85,0x53,0x2f,0x7a, -0x15,0xcd,0x14,0x9f,0xc1,0x8d,0x7e,0xbd,0x2e,0xe7,0xe3,0x91, -0x88,0x76,0xe4,0x1d,0xb2,0x06,0x78,0x33,0x42,0x88,0x6d,0x70, -0x30,0x75,0xef,0xf5,0x65,0xf8,0xb6,0xcc,0xe7,0x86,0x9b,0x54, -0x51,0xa9,0x9d,0x31,0xfa,0xf9,0x6f,0x44,0xa9,0x9b,0xf9,0x37, -0x29,0xd2,0xb1,0xf9,0xcd,0x4e,0x94,0x53,0x85,0x6d,0x22,0x6c, -0xf8,0x09,0xd6,0x20,0x1c,0xed,0x6e,0x5a,0xb0,0x97,0x6a,0x24, -0xd2,0xc1,0xaa,0x58,0x6a,0x6e,0xb0,0xbd,0x94,0x75,0x73,0x88, -0xc3,0x2b,0x24,0x1b,0xe9,0xd6,0x37,0x3c,0xa0,0xb3,0x08,0xe4, -0x33,0xe5,0x06,0x0a,0xa7,0x8a,0x0e,0x95,0x34,0x40,0x81,0xea, -0xb4,0xae,0x9d,0xa2,0x44,0xcb,0x98,0x91,0xe2,0xe7,0xa3,0x91, -0x71,0x78,0xcd,0xa8,0xca,0xa9,0xb9,0x0a,0x6d,0x3e,0xad,0x9a, -0x3b,0x52,0x39,0xf3,0x5b,0x4f,0x7e,0xb3,0xda,0x1e,0x85,0x0a, -0xa3,0xbe,0xdc,0x1c,0x35,0x1b,0xa3,0x9d,0x3a,0xba,0x7b,0xdf, -0x64,0xcc,0x01,0x44,0x39,0xfa,0xcd,0xa3,0x06,0xe2,0xde,0xec, -0xa4,0xcd,0x86,0x54,0x12,0x0d,0x1b,0xbe,0x35,0x45,0xb5,0x34, -0x3b,0xe6,0x86,0x9a,0xd5,0xab,0x64,0xdb,0x3d,0x4a,0xc0,0x3f, -0x9b,0x1d,0x8f,0x37,0x80,0x08,0x52,0xb5,0x02,0x17,0x6b,0x75, -0x99,0xcb,0x70,0xcd,0x8b,0xae,0xab,0xd6,0x09,0x39,0x62,0x7d, -0xe3,0xe4,0x2e,0x7b,0xf4,0xed,0x04,0x1a,0xbf,0x3e,0xf9,0x3d, -0xcc,0xc6,0xc6,0xed,0xd6,0xc5,0x12,0x0b,0x2d,0x17,0xf9,0x85, -0xc7,0x8d,0xec,0x7b,0x1a,0x37,0xca,0x54,0xb8,0xbd,0x4c,0xb3, -0x15,0x0b,0xa1,0x12,0x06,0x0b,0x46,0x54,0xa3,0x82,0x4a,0x5b, -0x57,0xb0,0x05,0x54,0xcb,0x20,0xdd,0xff,0xa2,0xda,0x4a,0xf9, -0xf8,0x50,0x34,0x52,0x9b,0x22,0x59,0x37,0x57,0xca,0x3c,0x78, -0x72,0x94,0x66,0xdc,0x3e,0x3d,0x2b,0x68,0x89,0xd5,0x52,0x8d, -0x4c,0x16,0x6f,0x23,0xb3,0x7b,0x48,0xae,0xd9,0x4a,0x81,0xd4, -0x20,0x31,0x61,0xf7,0x45,0xb2,0xd3,0xf7,0x74,0xe7,0x02,0x50, -0xee,0xf3,0xe5,0x0b,0xbc,0x65,0xce,0x9e,0xae,0x9c,0xe9,0xa3, -0xdc,0x3c,0x0f,0xdf,0x9b,0xb0,0xf8,0xa8,0xeb,0x2d,0x62,0x05, -0x86,0x43,0xe3,0x7a,0x7e,0x22,0x17,0xf1,0x8b,0x12,0xd5,0x9a, -0x09,0xfc,0x49,0x32,0xef,0x3a,0x46,0x9d,0xae,0xf7,0xe4,0x7a, -0xfa,0x08,0x96,0xb2,0xf1,0x3c,0x69,0x42,0xd1,0x26,0x9a,0xa4, -0x5a,0x61,0xa4,0xee,0x0a,0x1f,0xdf,0x20,0xa7,0xbf,0x99,0x2b, -0x7d,0x16,0x94,0x4e,0xa3,0xf1,0x6b,0xef,0x65,0x7e,0xef,0x3f, -0xd1,0x7d,0x9e,0xbb,0x25,0xe9,0xbe,0x73,0xd5,0x5d,0x76,0xbc, -0x90,0xcb,0x20,0x2b,0xd0,0x3f,0x2f,0x55,0x05,0xe6,0x74,0x78, -0x42,0x7e,0x54,0x6b,0xea,0x49,0x86,0x55,0x77,0x74,0xb6,0xb2, -0x26,0x56,0xb6,0xc5,0x85,0xa3,0x7a,0xbe,0x27,0xef,0x9c,0x00, -0x59,0x37,0xab,0xc8,0xeb,0x04,0x0c,0x76,0xa2,0xf5,0x52,0x73, -0xf2,0x74,0x69,0xaf,0xc0,0xf2,0xbb,0x29,0xcc,0x3f,0x5e,0xbe, -0xdf,0x7f,0xfd,0xee,0x6d,0x9e,0x6c,0xb7,0x1e,0xb7,0xda,0xb2, -0x2e,0xf3,0x45,0x3e,0x7a,0x34,0x78,0xea,0x93,0xb7,0x0d,0x1d, -0xb8,0xe4,0x1d,0x83,0x87,0x2a,0xf9,0xb6,0x41,0x35,0x50,0xfe, -0xd8,0xf0,0x49,0x44,0xfe,0xc4,0xf0,0x91,0x79,0xfe,0xb5,0x51, -0x07,0xe8,0xf9,0x53,0x6f,0x83,0x8d,0xd7,0x97,0x9d,0xd9,0x73, -0xae,0xa8,0x76,0xc2,0xe5,0x25,0x65,0xe1,0x5d,0x4d,0xfc,0x85, -0x2d,0xd1,0xd5,0x74,0x36,0x2f,0x4d,0xb4,0x55,0xca,0xb7,0x7e, -0x39,0xdc,0xea,0xe1,0x00,0x3f,0x6c,0x34,0x0e,0x37,0x7b,0x87, -0xa3,0x66,0xba,0x55,0x08,0x14,0x2e,0xec,0xfb,0x57,0xc3,0x89, -0x82,0x06,0x18,0x98,0xa9,0xbd,0xac,0x71,0x38,0x7a,0x94,0x1e, -0xb6,0xe0,0x19,0x4b,0x1c,0x66,0xf4,0x03,0xe9,0xf2,0x74,0xd8, -0xc2,0x8c,0x9b,0xc7,0xcb,0xb4,0x97,0xf6,0xb6,0x0c,0xee,0x64, -0xa5,0x93,0x6c,0xad,0x0d,0xc0,0xf3,0x23,0x22,0x82,0xff,0xf7, -0xdb,0x9b,0xdf,0x9a,0xd6,0xe0,0x51,0xfa,0x4f,0xc6,0xed,0xde, -0xf7,0xa2,0xf7,0x5d,0x7a,0x3f,0x08,0xd2,0x7e,0xb8,0xa3,0xcc, -0x3e,0x92,0xb0,0xf0,0xef,0x1f,0xd2,0x07,0x5b,0x26,0xd6,0x4b, -0xe3,0x87,0xf5,0xfb,0xbf,0xa0,0x52,0x0c,0x75,0x64,0x8d,0xc3, -0x43,0xc0,0xb2,0x37,0x5d,0xec,0xed,0xf5,0xf0,0xbf,0xc5,0xee, -0x74,0xb1,0xbb,0x4b,0x7f,0x7a,0xf8,0xdf,0x62,0x34,0x1a,0xf5, -0x46,0xf0,0x33,0xed,0x2d,0x3e,0xf5,0xa7,0x8b,0x4f,0x83,0xde, -0xe2,0x03,0xfc,0x7e,0x80,0x5f,0xb2,0x28,0x59,0xc8,0x9f,0xc5, -0xd9,0x59,0xe3,0xec,0xec,0x0c,0x29,0xf8,0xfe,0xfb,0xc6,0xf7, -0xdf,0x7f,0x8f,0x4f,0xe3,0xc5,0xcb,0xc5,0x70,0xf1,0x7c,0x71, -0x7e,0xde,0x5b,0xfc,0xf0,0x43,0x6f,0x71,0x79,0xd9,0x5b,0x94, -0x65,0x6f,0xb1,0xbf,0xdf,0xdb,0xef,0x2d,0xfe,0x6b,0xf1,0xe7, -0x9f,0xbd,0xc5,0xcf,0x3f,0xf7,0x16,0xa4,0x97,0xab,0x35,0x99, -0xac,0x25,0xf7,0xcd,0xc1,0xe2,0xcd,0x9b,0x1e,0xfe,0xb7,0xb8, -0xb8,0xe9,0x98,0x27,0x4b,0x2c,0xbe,0xca,0xae,0x2c,0xdf,0x3a, -0x1c,0x1d,0x8e,0xa0,0x6b,0x56,0x1b,0x13,0x22,0x08,0xe0,0x79, -0xbc,0xd4,0x40,0x71,0x7e,0x98,0xeb,0xed,0x21,0xa5,0xf0,0x93, -0x20,0xdb,0x99,0x3b,0xe6,0x5b,0xfd,0xe6,0xe1,0xe6,0xa0,0x47, -0x30,0x4f,0x03,0x18,0xb4,0x3e,0x84,0x6c,0xe8,0xad,0xc1,0xa3, -0xfe,0xc3,0xe1,0xe6,0x9f,0x87,0xd7,0xed,0xf6,0xf3,0xf6,0x26, -0xfc,0x7c,0xfd,0xea,0x15,0xfc,0xfd,0xa6,0x8d,0x2f,0xbb,0xdf, -0xe0,0xcb,0xab,0x6f,0xe9,0xe5,0xd5,0xee,0x0b,0x7c,0xd9,0x7d, -0x45,0x2f,0xaf,0x5e,0xbe,0x1a,0x34,0x17,0x7d,0x00,0x7d,0x4a, -0xb9,0xed,0xa7,0x00,0xba,0x35,0x68,0x36,0x0e,0xcb,0x47,0xbd, -0x30,0x79,0xd0,0x4c,0x81,0x80,0xed,0xe5,0x56,0x61,0xaa,0x8a, -0x8f,0x7c,0xeb,0x67,0xc0,0x82,0x74,0x62,0x53,0x65,0x3d,0xfc, -0x1b,0x02,0xe6,0x5b,0x07,0x95,0x92,0x62,0xd8,0x08,0x9f,0x70, -0x28,0x9f,0x08,0x15,0xb7,0xa4,0x25,0x65,0x2e,0xb8,0x99,0x57, -0x3e,0x82,0x8c,0x27,0xcb,0x4d,0x9e,0x50,0x87,0xa3,0x4d,0xfc, -0x03,0xfd,0xf9,0x01,0x7f,0x1f,0xd8,0x87,0x4d,0x4a,0xc3,0x27, -0x78,0x48,0x1b,0x8d,0x83,0xc5,0x5a,0x4a,0xaf,0x8d,0x4c,0xfd, -0xc5,0x4a,0xa0,0x47,0xe1,0xff,0xa9,0xfc,0xd7,0x88,0xa9,0xc7, -0xc4,0x07,0x44,0x81,0xa8,0x36,0xc9,0xfc,0x69,0x73,0x6f,0x6f, -0x73,0x77,0xf7,0xe0,0x87,0x1f,0xb2,0xcb,0xcb,0xac,0x2c,0x7f, -0x4e,0x8c,0x55,0x3c,0xe5,0x7d,0x05,0x90,0x18,0x32,0x3b,0xd9, -0xec,0x7f,0x18,0x7c,0xf8,0x10,0xbc,0x6c,0xbe,0x84,0x57,0x02, -0x44,0x13,0xaf,0x81,0xb1,0x8a,0xa5,0xbc,0xdf,0x4f,0x2c,0xd6, -0xd6,0x7e,0x62,0xb6,0x88,0x72,0x22,0xc7,0xfd,0x71,0x4d,0xb3, -0x35,0x30,0x1e,0xba,0x1e,0xd6,0x83,0x54,0xf2,0x25,0x4b,0xa7, -0x6f,0x0d,0xe4,0xe8,0x27,0x32,0x36,0x84,0xb9,0xc3,0xed,0xb2, -0xa0,0x26,0xd9,0x3a,0x2b,0xf0,0x22,0xef,0x1f,0x9f,0x79,0x81, -0x29,0xdd,0x72,0x5d,0xe6,0xe4,0xfc,0x74,0xf1,0x03,0xee,0xd4, -0x16,0x62,0x2d,0xb4,0xd8,0xe7,0xcd,0x10,0xbc,0xfb,0x9d,0x51, -0x62,0x2f,0x59,0x2c,0xc8,0xe9,0x60,0x31,0x57,0x99,0xaf,0x68, -0xb5,0x01,0x76,0xad,0x0b,0x64,0x1d,0xb3,0x6f,0x9f,0xc6,0xc0, -0xf4,0xc5,0x68,0xf1,0xe9,0xf8,0x89,0xa1,0xda,0x32,0x74,0x74, -0x61,0x40,0x34,0x2d,0xb3,0xbf,0x3d,0x7d,0x02,0x8f,0x74,0xd4, -0x58,0x66,0xdb,0x5f,0x7f,0xbb,0x3d,0x7e,0x6a,0x50,0x8c,0x04, -0x98,0xce,0xd7,0x00,0xf6,0x74,0x69,0xd4,0x35,0xe8,0xfc,0xe6, -0xb2,0xcc,0x12,0xb5,0x6b,0x83,0x4d,0x6c,0x96,0xd8,0xc7,0x4b, -0xcc,0xc2,0xaa,0x60,0xfb,0x99,0x25,0xb8,0x03,0x85,0xdd,0x47, -0x46,0x97,0x73,0xcd,0x2e,0xfe,0x62,0xce,0xa7,0x8c,0x2f,0xf1, -0x9a,0x0f,0x59,0x02,0x1d,0xf9,0x81,0x9e,0xf7,0x32,0xb9,0x82, -0x6b,0x3e,0x67,0x09,0x8a,0xb4,0x50,0x60,0x97,0x8a,0x88,0x54, -0x9b,0x98,0x31,0x97,0x23,0x64,0x2f,0x5d,0x51,0x7a,0x3d,0x3b, -0xe3,0x3c,0x06,0xfc,0xfe,0x7b,0x97,0x4b,0x09,0x18,0x5a,0x45, -0xdf,0xd9,0xce,0x6f,0xa0,0xd0,0xf4,0x94,0xe4,0x6d,0x5d,0x01, -0x94,0xb1,0xdb,0x83,0x00,0xbb,0xa4,0x2b,0x6a,0xf1,0x95,0x8b, -0xfb,0x5a,0x05,0x8a,0x93,0xc3,0xea,0x23,0x03,0x7f,0x5c,0x59, -0x23,0x47,0x36,0x38,0x14,0x76,0x77,0xd7,0x3e,0xad,0x7d,0x58, -0xdb,0x5b,0xdb,0x5d,0x1b,0xb9,0x0e,0x5f,0x83,0x0e,0xbf,0x82, -0x8d,0x9c,0x55,0x75,0xe7,0x09,0xe6,0xff,0xb0,0x76,0xbe,0x76, -0xb9,0x56,0x22,0x7c,0x00,0x59,0x77,0x3c,0x07,0x03,0x63,0x95, -0x87,0x1f,0xeb,0xa0,0xa1,0xb3,0x34,0x7b,0x7b,0x0a,0x2a,0xba, -0x1f,0xa1,0xbd,0xe7,0xb0,0x3d,0xe8,0xfe,0x39,0x06,0x55,0x60, -0x67,0x85,0x0c,0x4b,0x18,0xbe,0x04,0x45,0x54,0x7a,0x77,0x15, -0x8d,0xec,0x70,0x62,0x49,0xa3,0x61,0x25,0x88,0xdb,0xaf,0x2e, -0x61,0xb8,0xad,0x86,0xa2,0xfc,0xd1,0xbd,0x88,0xb4,0xea,0x35, -0x98,0x39,0x11,0xa5,0xa3,0x2f,0xc4,0x50,0xd7,0x56,0xa3,0x2f, -0x45,0x12,0x95,0xff,0xb4,0xea,0x23,0x11,0x1e,0xbf,0xf2,0xc3, -0x2a,0x00,0x19,0x98,0x08,0xf3,0xcf,0x7f,0xd6,0x00,0x05,0x2e, -0x59,0xa8,0x04,0x6f,0xf5,0xf1,0x8e,0xb0,0xd9,0xa6,0x52,0x5f, -0x54,0xce,0x3c,0x91,0x32,0x5f,0x54,0xe8,0x6b,0x28,0x04,0x73, -0xfa,0x3e,0x25,0xbc,0xed,0x9b,0x23,0x11,0x4d,0x1b,0x6f,0x6b, -0x20,0x3b,0x54,0xce,0x56,0x00,0xde,0x56,0x09,0x91,0x06,0x0c, -0xe6,0x3e,0xa5,0x14,0x13,0x50,0xd4,0xe1,0xaa,0x76,0x47,0xef, -0x58,0x02,0xbf,0x5f,0x01,0x7b,0x47,0x55,0x44,0xe3,0x4a,0xb7, -0x5e,0x32,0xa4,0x10,0xff,0xcb,0x3b,0xe8,0x10,0xb0,0xe1,0x2a, -0x30,0x3b,0xa1,0xe5,0x52,0x06,0x93,0x42,0x7a,0xc7,0x06,0x6f, -0xf2,0xc4,0x62,0x07,0x5f,0xd7,0xdb,0x80,0xe9,0xf9,0xbf,0x05, -0x53,0x07,0x30,0xfd,0xb0,0x0a,0x93,0x14,0x82,0xbd,0xcb,0x1d, -0x10,0x5f,0x75,0xb6,0x17,0x8b,0xce,0xf6,0x12,0x16,0xad,0x55, -0xec,0xd1,0x56,0xb9,0x84,0x35,0x6e,0x05,0x8c,0xac,0xb7,0x08, -0xb3,0x5f,0x07,0x43,0xe6,0x5e,0x82,0xcd,0x2f,0xcf,0x8d,0x74, -0x0b,0x86,0x03,0x96,0xa9,0x2b,0x14,0x76,0xef,0x2d,0x18,0x52, -0x1a,0x50,0xfb,0xf7,0x40,0x52,0x2d,0x6e,0x1e,0x43,0xd1,0x9f, -0xb3,0xc8,0x0f,0xdf,0x30,0xdf,0x24,0x58,0x94,0x66,0xd0,0xdd, -0x7e,0x9e,0x34,0xdd,0xd6,0xbf,0xbd,0x33,0xa4,0x38,0x1e,0x9b, -0x43,0x4c,0xc7,0x8b,0xa3,0xc7,0xcd,0x1a,0x52,0x87,0xa4,0x1a, -0xdf,0x4e,0x9b,0x49,0x96,0xd4,0xe6,0xa3,0xb5,0x1b,0x11,0xfe, -0xf3,0x7f,0x47,0xf5,0x9d,0xf6,0x23,0xa0,0x80,0xf9,0xce,0x9f, -0xab,0x7a,0x0d,0xf1,0x3f,0x3f,0x3e,0xa6,0x49,0xf6,0xe7,0xad, -0x50,0x18,0x9f,0x01,0xa1,0xfe,0x6b,0x15,0x10,0xc8,0x46,0x7f, -0xa0,0xce,0xc2,0x60,0xd4,0x66,0x94,0x67,0x79,0x79,0x03,0x79, -0x55,0x2d,0x95,0xf0,0x66,0x39,0xb9,0x7a,0x8c,0xb3,0x60,0xad, -0x49,0x06,0xdd,0x58,0x32,0x70,0xd7,0x7a,0x8b,0xbc,0x92,0xc5, -0x71,0x10,0x6a,0x8d,0x72,0x8a,0x66,0x32,0x4d,0x06,0x71,0x91, -0x46,0x3d,0xec,0x00,0xdd,0x85,0x90,0x79,0xa4,0x96,0x35,0x54, -0xcd,0x41,0xf2,0xad,0xb5,0x16,0x83,0xdc,0xfb,0x00,0x5c,0x51, -0xd9,0x36,0x57,0x56,0x97,0xdd,0xc2,0x4b,0x1c,0xb7,0x63,0x40, -0x10,0x18,0xbd,0xd6,0xc5,0x9e,0xbd,0xa0,0xac,0x9d,0x20,0x96, -0xe3,0xb9,0xef,0x2f,0xad,0xf9,0x24,0x87,0x3a,0x05,0x9b,0x53, -0xa0,0xf3,0x1c,0xc9,0xc3,0x64,0x51,0x7f,0x22,0x7d,0x35,0x0e, -0xf9,0x11,0xa2,0x87,0x1d,0x8e,0x56,0xa2,0xf8,0x42,0xea,0xab, -0x7e,0x72,0x94,0x34,0x25,0x61,0x69,0xc4,0x55,0x57,0x96,0xfc, -0x7d,0x38,0xb9,0x1e,0xce,0x3e,0x1f,0xbd,0x1a,0x1f,0xcf,0xe8, -0x61,0x6f,0x38,0x3b,0x39,0x3f,0x7a,0x7e,0x35,0x2b,0x2e,0xe0, -0xf9,0xf3,0xd1,0xdf,0xaf,0x27,0x63,0xf8,0x73,0xf1,0xf9,0xe8, -0xf9,0xf5,0xd9,0x75,0x39,0x3f,0xda,0x1f,0x5f,0xcd,0xc7,0x78, -0x1c,0x76,0xf4,0x0e,0xa4,0x79,0xfc,0x7d,0x3b,0xfd,0x9d,0x13, -0x76,0xc7,0x27,0xf4,0xe0,0x04,0xbd,0xa3,0xc4,0x3a,0xdc,0xf2, -0xdf,0x78,0x19,0xa9,0xbb,0x38,0xbf,0xef,0x7c,0x70,0x0d,0x1c, -0x75,0x34,0xe8,0x88,0x44,0x24,0x0f,0x29,0x43,0xba,0x2c,0x55, -0x48,0x14,0xd2,0x84,0x04,0x21,0x29,0x48,0x06,0x52,0x50,0x53, -0x39,0x23,0xba,0x83,0x02,0x02,0x0a,0xc8,0x50,0x17,0x8a,0x54, -0x61,0x4c,0x7c,0xeb,0x1c,0x6a,0x16,0xa8,0x3b,0x34,0x64,0xf0, -0xc3,0xd6,0x80,0x0a,0x23,0x95,0xc4,0x00,0x88,0x71,0x5a,0xde, -0x1f,0xb0,0x9d,0x70,0x67,0xdb,0x5b,0x2a,0x54,0xa0,0xc8,0x1f, -0x15,0x1e,0x8f,0x58,0x8d,0x28,0x3a,0xce,0xe9,0x6f,0xc3,0x1e, -0xa9,0x18,0xa4,0x5c,0x65,0x9e,0xfc,0x92,0x34,0xbd,0x7c,0x5c, -0xd2,0x91,0x12,0xb9,0x7d,0x5f,0x84,0x19,0x2c,0xd9,0x49,0xae, -0xa9,0xab,0xaa,0x72,0x95,0xce,0x5f,0x97,0x6b,0xd1,0x6d,0x39, -0xbe,0x2c,0x57,0x5b,0x96,0x6f,0xc8,0xfa,0x96,0x71,0xa1,0xb6, -0xa1,0x2b,0x2d,0xc7,0xc8,0x92,0xfd,0xeb,0x09,0x3c,0x1c,0xc1, -0xf6,0x0d,0x7f,0x0e,0xae,0xc7,0x25,0xfe,0x7e,0x18,0x8f,0x26, -0xfc,0x74,0x70,0x0e,0x8b,0x1d,0x3e,0xbc,0x82,0x95,0x15,0x7e, -0xf6,0x87,0x80,0x04,0x77,0x35,0xba,0x43,0x1d,0xba,0x95,0xbd, -0x69,0x21,0xfa,0x72,0xa6,0x35,0x50,0x44,0xc8,0x88,0x02,0x4a, -0x90,0x0c,0xa4,0x01,0xeb,0xc7,0x9a,0xb1,0x52,0xac,0xb1,0xb6, -0xb2,0x3b,0xc6,0x4f,0x00,0x56,0x57,0x2d,0x70,0x4b,0xac,0x14, -0xea,0x84,0x2a,0xa1,0x46,0xa8,0x10,0xea,0x83,0xea,0x6a,0x6b, -0xdb,0xd3,0x4e,0x51,0x57,0xd5,0x05,0x40,0xaa,0xa6,0xe0,0x84, -0xdd,0x17,0x96,0xe4,0xbb,0x46,0x6b,0x50,0xda,0x8d,0xd7,0x20, -0xd5,0x8d,0xd8,0x6f,0xe2,0x01,0x1b,0x80,0xc5,0x43,0x96,0x87, -0x6b,0x67,0x90,0x12,0xa5,0x45,0x65,0xd4,0xba,0x3d,0x43,0x75, -0xdc,0x86,0x7b,0x92,0xd5,0xf9,0xb8,0xeb,0x09,0xc7,0x75,0x4c, -0xd1,0x97,0x8d,0xec,0xb8,0x34,0x8f,0x6d,0xdd,0x92,0x7a,0x74, -0x87,0x37,0xd0,0xb3,0x9b,0x37,0x07,0x59,0x72,0x9e,0x5d,0x5e, -0xae,0x3d,0x4f,0xcc,0x9b,0x2c,0xd9,0xdb,0xdb,0xda,0xdd,0xdd, -0xa2,0x5b,0x75,0xe6,0x0d,0xbd,0xef,0xe1,0x9e,0xd8,0x26,0x84, -0x29,0x6b,0x6f,0x0e,0x28,0x11,0x52,0x71,0xd3,0x65,0xd6,0xa2, -0x3c,0x58,0xb7,0xc3,0xda,0xbc,0x1f,0x10,0xeb,0x33,0x44,0x5c, -0x17,0xf0,0x97,0x84,0xd0,0xe4,0x82,0xc3,0x06,0xef,0x61,0xb8, -0x8d,0x8d,0x55,0x80,0xad,0xf9,0xf4,0x27,0x8c,0x3d,0xca,0xfe, -0x57,0x28,0x84,0xdc,0xed,0x98,0xa3,0x02,0xde,0xd0,0x11,0xbf, -0x61,0x01,0xff,0xdf,0xdd,0x45,0x35,0xf1,0x48,0xdb,0x56,0x46, -0x17,0x8e,0xc4,0xf8,0xa7,0x83,0xf1,0x69,0x56,0x7e,0x40,0xce, -0x74,0xa4,0x46,0xfc,0x26,0x18,0xbc,0x4a,0x98,0xc5,0xa7,0x43, -0x72,0x26,0x78,0x85,0xd7,0x61,0x39,0x8d,0xee,0xdc,0x86,0x4e, -0x65,0xd0,0x86,0xef,0xf9,0xbc,0x81,0xa2,0x6d,0x78,0x57,0x3b, -0xdb,0xea,0x0f,0xaf,0x06,0x87,0xad,0xde,0x65,0x0f,0xfe,0x6c, -0xc1,0x9c,0x91,0x5c,0x5f,0x8f,0xf5,0x9b,0xc9,0x5e,0x34,0x8b, -0x92,0xf0,0xfa,0x98,0x2d,0x98,0x8b,0x0e,0xa3,0x24,0xa3,0x97, -0x5c,0x5d,0x82,0x84,0xf9,0xe3,0x5e,0x92,0xb9,0x94,0x21,0xa6, -0x3c,0xdf,0x83,0x4e,0x3d,0x3a,0x19,0x82,0xf0,0x32,0x1a,0xce, -0xb2,0x9b,0x12,0x86,0xd7,0x2e,0xaa,0x74,0xfa,0x07,0x53,0x18, -0x6d,0x6b,0xc3,0xf9,0x80,0x06,0x05,0xfa,0x0f,0xb1,0xe9,0x97, -0xd3,0x19,0x3a,0x14,0xd7,0x59,0x1f,0x48,0xe3,0x83,0xcd,0xbb, -0xd6,0xb7,0xe9,0xe8,0x46,0x80,0x8b,0xfc,0x13,0xc6,0xef,0x78, -0xa6,0xd1,0x61,0x1e,0x97,0xe9,0xbf,0x81,0xc7,0xc1,0x5a,0x58, -0x14,0xa9,0x78,0x79,0x01,0xed,0x90,0xbc,0x21,0x7d,0x94,0x50, -0xa7,0x47,0x9b,0x89,0x9d,0x65,0x88,0xcb,0x66,0x81,0xd5,0x63, -0xad,0x2e,0x62,0x10,0x95,0xe9,0x89,0x5f,0x54,0x36,0x94,0x40, -0x84,0x99,0xed,0xd4,0x23,0x6d,0x3b,0x93,0xdd,0x9c,0x92,0x69, -0x4d,0x96,0x80,0x10,0xf4,0x15,0x08,0xa5,0x57,0x40,0x73,0x96, -0x7c,0x55,0xae,0x0d,0xcf,0xa6,0xa4,0xd2,0x1b,0xae,0x9d,0xc2, -0x0c,0xb7,0xfa,0x47,0xd4,0xec,0x0d,0xd7,0xac,0x6e,0xef,0x12, -0xde,0xbe,0x1a,0x59,0x87,0xa7,0xa4,0xeb,0x1b,0x72,0x0f,0xc1, -0xf3,0x39,0xe5,0x51,0x77,0x91,0xf2,0x6f,0xb8,0x46,0x4a,0xb4, -0xd1,0x88,0xd2,0x59,0x04,0xde,0x23,0x6c,0xac,0xef,0xdb,0xdb, -0x63,0x64,0x22,0x36,0x7f,0xc6,0x2c,0x56,0x00,0x7e,0xfe,0x4c, -0x39,0x64,0x4e,0x02,0x8d,0x16,0xd0,0xef,0x1a,0xae,0xd6,0x04, -0x29,0x36,0x20,0xaa,0xb6,0xaa,0x46,0xd6,0x67,0xf0,0xfb,0x36, -0xee,0xfd,0xaa,0x94,0x76,0xf7,0x93,0xf6,0x2b,0x54,0xed,0x5b, -0x17,0xb8,0xd4,0xde,0x0c,0xe9,0x3f,0x05,0x7d,0x5c,0xca,0x1c, -0x94,0x40,0x24,0xac,0x43,0xaf,0xa1,0x18,0x41,0x77,0xda,0xbd, -0x84,0x7b,0x11,0x46,0x3d,0xe2,0x4b,0x6e,0xf9,0x02,0x39,0xc2, -0xe6,0x1f,0x61,0x3c,0xf6,0x60,0x5b,0xd1,0x58,0x02,0x8d,0x92, -0xe9,0x74,0x94,0x71,0x5b,0x47,0x8b,0xa7,0x40,0x79,0xfe,0xff, -0xd5,0x28,0xf1,0xdf,0x69,0xb3,0x33,0x4a,0xb6,0x07,0xf5,0x1e, -0x65,0x14,0xb9,0x86,0x5f,0xa1,0x79,0xa6,0xe5,0xfc,0x34,0x62, -0xc8,0x2b,0x40,0x49,0x35,0x1b,0x3a,0x9c,0x51,0x16,0x20,0xca, -0x2a,0xc8,0x2f,0x45,0x78,0x53,0x2f,0x7c,0xfd,0xcc,0xaa,0x36, -0x11,0x2f,0xb2,0x1b,0xbe,0x60,0x08,0xe9,0xd9,0x53,0x48,0x53, -0xbe,0x57,0x32,0x17,0xda,0x98,0x95,0xd9,0x3a,0x6b,0xd5,0xd9, -0xb3,0x82,0x41,0xbf,0xed,0x72,0xbe,0x1b,0xf2,0x57,0xa3,0x7c, -0xb7,0x18,0xbe,0xc9,0xe0,0x58,0xee,0x31,0xc7,0xd1,0xf2,0x9d, -0x89,0x40,0x68,0x9c,0x4c,0x60,0x6c,0x2b,0x45,0xd7,0x73,0x94, -0xc7,0x48,0x65,0xf9,0x71,0x73,0x54,0xb0,0x5b,0x22,0x73,0x74, -0x2a,0x3d,0x0e,0x6b,0x6c,0x46,0xc5,0xe4,0xd2,0x44,0xc6,0x3f, -0x86,0x0f,0xfd,0xd1,0x0f,0x5b,0xba,0x34,0x5e,0x40,0xbe,0x0f, -0xad,0x74,0x44,0xdd,0xfd,0x57,0x28,0xce,0x03,0x9a,0xf9,0x5a, -0x55,0xb6,0xde,0x76,0x44,0xb5,0x3d,0xd5,0x35,0x1f,0x14,0x7c, -0xc8,0x52,0x1c,0x62,0xfa,0x8f,0x80,0x5d,0x7a,0xc5,0xde,0x21, -0xf0,0x12,0x45,0x37,0xc0,0xc5,0x3e,0xd5,0x44,0xc7,0xfe,0xf1, -0xf7,0x3b,0xd1,0xa0,0x2c,0xce,0x26,0xc0,0x97,0xe6,0x7c,0x0c, -0xf4,0xba,0x9c,0xc2,0xfc,0xb7,0xc7,0xd0,0xb9,0x33,0x42,0x71, -0xee,0xde,0xc5,0x30,0xc2,0xc6,0x5b,0xcb,0x2b,0xfe,0x40,0x19, -0xbd,0xab,0xd6,0x17,0x15,0xa7,0x53,0xec,0xe7,0x3c,0xb3,0x08, -0x7a,0x37,0x4b,0x69,0x08,0xb6,0x0a,0x27,0xf7,0x5b,0xce,0x45, -0xa4,0xc0,0xa0,0xf7,0x37,0x8b,0x91,0x97,0x7b,0x46,0xe2,0x6c, -0x1a,0x02,0x4f,0xd5,0x9c,0xc7,0x96,0xe0,0xf9,0x8a,0x33,0x7a, -0x6d,0xfc,0x9c,0xf6,0x1a,0xd8,0x08,0xa8,0x90,0x41,0x0b,0x46, -0x32,0x20,0xef,0x0c,0x7a,0x9b,0x9d,0xac,0xe3,0xbf,0xe3,0xe6, -0x33,0x4e,0xa6,0x8c,0x15,0x34,0x0c,0x43,0x37,0x3d,0xd3,0x47, -0xd4,0x80,0xe7,0x41,0x0e,0x5d,0x57,0x91,0x9c,0xcb,0x20,0x47, -0x2e,0x51,0x4a,0x5e,0x19,0xe4,0x89,0x03,0x00,0x5b,0xae,0x8c, -0x0a,0x7a,0x17,0x01,0x0c,0x81,0x26,0x47,0xce,0x92,0x3e,0x30, -0x1c,0x08,0x3e,0x8e,0x02,0x62,0xae,0xf8,0x3a,0xdb,0xe5,0xc1, -0xe0,0xb0,0x31,0x82,0xa8,0x21,0x37,0x36,0x42,0x8f,0x02,0x9e, -0x45,0x1a,0xf8,0xaf,0x95,0x38,0x9b,0x62,0x31,0xf6,0x85,0x62, -0x69,0xaf,0x9d,0xe1,0x0f,0x13,0x19,0xb4,0xa0,0xad,0x4f,0xbe, -0x69,0x1b,0xf6,0x0e,0x7b,0x71,0xe2,0x63,0x48,0x1c,0xc5,0x89, -0x4f,0x20,0xf1,0x3c,0x4e,0xfc,0x1a,0x12,0x2f,0xe3,0xc4,0xa7, -0x90,0x58,0xc6,0x89,0xdf,0x40,0xe2,0xa7,0x38,0xf1,0x6f,0x14, -0xe8,0x18,0xa8,0x67,0x83,0xa9,0x4a,0x3c,0x03,0x35,0x0f,0xd0, -0x4e,0x8c,0x6e,0x74,0x46,0x7e,0xe7,0xc9,0x25,0x1b,0x79,0x9d, -0x07,0x34,0xec,0xa0,0x4d,0x0c,0xd6,0xe9,0x99,0xb0,0xbb,0x99, -0x88,0x51,0x49,0xb0,0x1d,0xc4,0xf6,0xc5,0x4d,0x50,0x76,0x5d, -0x23,0x67,0xcd,0xee,0xd4,0xb9,0xce,0xf4,0x48,0x1b,0xeb,0x38, -0xb4,0x54,0x69,0x20,0x83,0x59,0x57,0x87,0xd4,0x8d,0x5d,0xe5, -0x40,0x51,0x9c,0x14,0xf6,0x9c,0x27,0xc2,0x7a,0x9f,0x8b,0x16, -0x81,0x5c,0xd9,0xcf,0xa5,0x54,0x23,0x72,0x61,0x88,0x2c,0x24, -0x4f,0xc6,0x93,0x24,0xcd,0x62,0xef,0x89,0xf5,0xde,0x0b,0xcd, -0x2c,0xaf,0xda,0x22,0x49,0xab,0x45,0x6e,0xee,0x6a,0x4c,0xe4, -0x18,0x43,0xeb,0x08,0x5d,0x36,0xa6,0x59,0x04,0xcf,0xc9,0x41, -0x9b,0xec,0x62,0xf8,0x82,0x70,0x27,0xe4,0x9b,0x82,0xa2,0xf3, -0x73,0x51,0xf5,0xc8,0x58,0xa0,0x37,0xf1,0xcb,0xa2,0x54,0x72, -0x60,0x5c,0x43,0xd2,0x32,0x36,0xd4,0x5b,0x19,0x64,0x52,0xc7, -0xe3,0x65,0x50,0x55,0x76,0xb7,0xc2,0xa6,0x6f,0x29,0x6d,0x81, -0x61,0x97,0x93,0x93,0xca,0xd6,0xba,0x3c,0xef,0x90,0xbb,0xb2, -0xee,0xe6,0x66,0x91,0x3a,0xd7,0xd4,0x04,0x40,0x8e,0xed,0x70, -0x63,0x2f,0x15,0x86,0x2e,0xb3,0x7d,0xa5,0xe2,0x6f,0x3c,0xf6, -0x7b,0xad,0xdd,0x92,0x7b,0xa2,0x59,0x24,0x50,0x1e,0xa6,0xd1, -0x74,0x32,0xf4,0xea,0x4c,0x4b,0x18,0x5e,0x02,0x0b,0x6d,0xef, -0x08,0xb2,0x27,0xca,0x4f,0x72,0x14,0x6a,0xb8,0x70,0x56,0xeb, -0x8b,0x14,0x2f,0x97,0x5d,0xba,0x7a,0x69,0x1a,0xff,0x1c,0xc4, -0x25,0xaf,0x5d,0x0c,0x39,0xd1,0x83,0xe3,0x52,0x6a,0xab,0xb4, -0xa3,0x27,0x17,0x5b,0x60,0x1d,0x7c,0x06,0x43,0x9e,0xd7,0xc8, -0x41,0x82,0x95,0xec,0xfe,0x24,0xea,0xcc,0xbb,0xd3,0x2a,0x5c, -0x53,0x6c,0xf5,0x28,0x24,0x87,0xa8,0x42,0x38,0x22,0x16,0x5e, -0x92,0x27,0x3b,0x84,0x3f,0x6a,0xb0,0x2b,0x07,0xce,0x6c,0x80, -0x87,0xe6,0xfc,0x4b,0x63,0x9d,0xeb,0xaf,0x92,0xcb,0x88,0x58, -0xd8,0xb3,0xd2,0xd9,0x13,0xcd,0xc5,0x96,0x88,0xc7,0xb8,0xed, -0x5b,0x23,0x3d,0x81,0x28,0x0a,0xac,0xd5,0xc8,0x5a,0xff,0xfb, -0xbd,0x83,0xc1,0xcf,0x3f,0x27,0x84,0xfe,0x56,0xa1,0x8f,0xe9, -0xee,0x79,0x7b,0xd2,0xc0,0xe8,0x11,0x8b,0xbf,0xde,0x7f,0xb7, -0x92,0xc0,0xd0,0x61,0xa1,0x6a,0x3d,0x96,0x6b,0x8c,0xb2,0x9a, -0xe9,0x1f,0x0c,0xbc,0x09,0xcc,0xfe,0x7e,0xff,0xe7,0x01,0x53, -0x47,0x86,0xa8,0xf1,0x01,0xcb,0x25,0x6d,0x26,0x64,0x40,0xf5, -0x2f,0xed,0x79,0xab,0x53,0xd0,0x1a,0x1b,0xe1,0xc1,0x5c,0xba, -0xe3,0xb7,0x4b,0x75,0xf6,0x76,0xe9,0x0f,0xbc,0x28,0x5d,0x9f, -0x28,0x0d,0x96,0xf6,0xd6,0x5f,0xcd,0x07,0xb9,0xfb,0x80,0x3c, -0x04,0x60,0xd2,0xee,0x1f,0xec,0x63,0xf8,0xa6,0xf1,0xca,0x03, -0xfb,0xa3,0x61,0x4f,0x0e,0x23,0xc5,0x39,0x63,0xec,0x07,0x54, -0xa0,0x4c,0x43,0x1b,0xac,0xaa,0xb9,0x23,0xf9,0x96,0xe1,0xb9, -0x77,0x54,0x5a,0xb0,0x99,0x6e,0x9a,0xee,0xb4,0x51,0xda,0xa5, -0xc5,0xdc,0x1a,0x30,0xd7,0xd0,0xe3,0x0d,0x86,0x19,0xc7,0xd5, -0x29,0x39,0x4b,0xa7,0x49,0xf6,0x7c,0xbe,0xf2,0x03,0x94,0xdf, -0x61,0x18,0xbc,0xf3,0x93,0xdb,0x0e,0x9a,0x48,0x67,0x42,0xa6, -0x78,0x77,0x40,0x19,0xf5,0xb5,0xf9,0x7a,0x47,0x22,0x39,0xc5, -0x9b,0x25,0x9a,0xc0,0xfb,0xca,0xbf,0x9a,0xec,0x7e,0x83,0x81, -0x45,0x46,0x05,0x0a,0x12,0xc3,0x6a,0x54,0xd7,0x55,0xc7,0x7f, -0xf4,0x49,0xad,0xdf,0xa0,0x35,0xdc,0x7e,0x71,0xa8,0x8d,0x1a, -0x58,0x2e,0x25,0x75,0x14,0x59,0xf4,0x5f,0xbb,0xd5,0x14,0x1e, -0xf3,0x55,0xb7,0xdc,0xc2,0x25,0xae,0x81,0x0e,0x70,0xc4,0x46, -0x3b,0x8b,0xf3,0x7c,0x05,0xe6,0xae,0x38,0x2e,0xf4,0x9d,0x50, -0xd0,0x74,0x52,0x69,0xae,0x52,0x80,0xff,0x3f,0x4e,0xef,0xa6, -0x23,0x18,0x37,0xfc,0x31,0xb1,0xb4,0xa6,0x98,0x61,0x49,0x02, -0xa6,0x90,0xed,0x55,0x08,0x50,0xd0,0xe9,0x0e,0xf4,0xd4,0xb0, -0xbc,0x9d,0x86,0x53,0xcc,0x63,0xb3,0x10,0x86,0xc6,0x23,0x4c, -0x75,0x84,0xa5,0x70,0x21,0x9e,0x31,0xf3,0x11,0xed,0x26,0x56, -0x22,0xcf,0x6e,0x9c,0x70,0xec,0x8c,0xda,0x05,0xd0,0xb0,0x6d, -0x17,0x5d,0x96,0x84,0xf7,0xc5,0x42,0xac,0xbe,0x6c,0x42,0x8f, -0x94,0x20,0xf9,0x93,0xc7,0xdb,0xe3,0xaf,0xa5,0x22,0xed,0x01, -0x3e,0x6d,0x52,0x7d,0x41,0x92,0x55,0x5b,0xe6,0x9d,0xed,0x47, -0xda,0x88,0x84,0x49,0xe3,0x67,0xf4,0xdb,0xa6,0x0c,0x9e,0x38, -0xeb,0x32,0x2c,0xdf,0xcc,0x09,0x66,0x33,0xe0,0xb6,0x20,0x36, -0xcc,0xe6,0x2a,0x7e,0xc4,0x66,0x03,0x8b,0x7a,0x18,0x68,0xf7, -0x0a,0x4c,0x9a,0x6e,0xa9,0x6e,0xd8,0xac,0xb6,0xd9,0xad,0x35, -0x58,0xa0,0x86,0x6a,0xd9,0xdb,0xeb,0xb3,0xcd,0x2f,0xd5,0x86, -0x2d,0xec,0xb4,0xcc,0x5b,0xe8,0x3f,0x00,0x76,0x3d,0xd4,0xbe, -0xf4,0xa1,0x88,0xcd,0xb6,0x9d,0xb5,0xe0,0x73,0xfd,0x80,0x60, -0xb8,0x8e,0x3a,0x83,0xbe,0x30,0x07,0xbe,0x49,0xec,0xfb,0xc2, -0x74,0x34,0x2b,0x64,0x7b,0xbf,0xb0,0x47,0x37,0xed,0x08,0x4a, -0xb7,0xc8,0xde,0x50,0x4c,0x00,0x57,0x02,0x3d,0x6d,0x3f,0xf9, -0x1b,0x40,0x61,0x6a,0x6a,0x47,0xb8,0xa8,0xdf,0x32,0x17,0x84, -0xca,0x31,0x9e,0x53,0x98,0x3a,0x7e,0x72,0xa0,0xfb,0xb5,0x50, -0x39,0x17,0xc9,0x21,0x7e,0x4e,0xf2,0x00,0x83,0x4a,0xa8,0x50, -0x2a,0xb2,0x80,0xe6,0x73,0x2c,0xbf,0xb6,0xce,0xaf,0x2f,0x87, -0x13,0x18,0xcb,0x8d,0xf0,0xa6,0xa2,0xd4,0xfd,0x76,0xaa,0x0c, -0xb2,0xea,0x6b,0x26,0x9c,0x08,0x6b,0x57,0xf3,0xd4,0xc4,0x98, -0xaa,0x4a,0x61,0x3f,0xa7,0x73,0x4f,0xa9,0xf3,0xfa,0xe5,0x27, -0xb0,0x0c,0x00,0x3f,0x32,0xb0,0x03,0x60,0xae,0xb1,0xb2,0x15, -0xdd,0x36,0x89,0x26,0x71,0xf3,0xe9,0x0e,0xe2,0xe8,0x25,0x56, -0x1d,0x9d,0x64,0x9b,0x1d,0x49,0xb2,0x0a,0xec,0x24,0x6b,0xab, -0x94,0x5d,0xc0,0x94,0x75,0x54,0x29,0x4a,0xd8,0x96,0x04,0xd1, -0xa1,0x27,0xd9,0x37,0x2a,0x81,0x91,0xf8,0x2a,0x82,0xe5,0x43, -0xd6,0x0c,0xdd,0xc4,0xf6,0xbb,0xad,0xfb,0x62,0x9a,0x17,0x24, -0x23,0xd8,0x10,0x16,0xb5,0xf2,0x84,0x8b,0x6f,0xa1,0xe6,0xbc, -0x95,0x2c,0x6e,0x5d,0x42,0xd3,0x67,0x81,0xf8,0xc7,0x6c,0xa0, -0x6d,0xa7,0xd1,0x62,0x71,0x07,0xe4,0xd7,0x16,0x72,0x69,0x46, -0x5a,0xc0,0x52,0x97,0x68,0xf0,0xda,0x99,0x66,0xbc,0x22,0xf4, -0xe9,0x5b,0x6a,0x99,0x4a,0x0b,0xae,0xad,0x05,0x17,0x6c,0xd4, -0x3d,0xdd,0xf0,0x12,0xb1,0x6a,0x3f,0x11,0x08,0xf0,0xb2,0xdf, -0xcd,0x88,0xd5,0x41,0x9b,0x00,0xb8,0x84,0xc9,0x8e,0x3f,0x7c, -0xe0,0xbe,0x8a,0xcc,0x77,0xa7,0xc4,0x45,0xcd,0x97,0x5c,0xfb, -0x59,0x79,0x55,0x8b,0xa9,0x0d,0xec,0xb9,0xaa,0xde,0x43,0x57, -0x5c,0x51,0xa6,0x36,0xca,0x1a,0x9e,0x24,0x19,0xf1,0x2c,0x8c, -0xfa,0xe7,0x8e,0x95,0x7f,0xf4,0xad,0xa3,0x84,0x4a,0x24,0xee, -0xe6,0x91,0x07,0xdf,0xb3,0xf1,0xb0,0xd4,0xb7,0x56,0x57,0x95, -0xf4,0xaf,0x5c,0x56,0x72,0x95,0xa2,0x79,0x18,0x4f,0xbc,0x2c, -0xde,0x0b,0x8a,0x27,0xd7,0x5b,0x16,0x44,0xf1,0xef,0x4a,0x3c, -0x3b,0x53,0x6b,0x54,0x5b,0xfc,0x35,0x32,0x8f,0xcf,0x74,0x03, -0x88,0xe3,0x48,0x9a,0x65,0xf4,0x68,0xcd,0x92,0xc5,0x29,0x24, -0xce,0x47,0x65,0xe8,0x66,0x31,0x11,0xb7,0xce,0x02,0xc3,0x36, -0x57,0x09,0xb3,0xf8,0x2c,0xb0,0x68,0xb3,0x99,0xb2,0x32,0x64, -0x55,0x1b,0x32,0x90,0x58,0x45,0xdf,0x1b,0x32,0xf3,0xc0,0xcc, -0x10,0x64,0x0a,0x47,0xa0,0x5f,0x94,0x62,0x2b,0x43,0x27,0xe7, -0x00,0x27,0xa8,0x69,0xc7,0x7b,0x08,0x16,0x4c,0xbb,0x30,0x40, -0x4e,0xe3,0x0b,0xb0,0x71,0xe5,0x3d,0x69,0x3a,0x96,0x9e,0x3a, -0x69,0xcb,0xca,0x82,0xb0,0xa0,0x02,0xb7,0xec,0x10,0x0f,0x79, -0x0e,0x1b,0x93,0x59,0xad,0xbc,0x15,0x11,0x53,0x77,0x2f,0x8a, -0xab,0xa1,0xbf,0x91,0xa9,0x7c,0x33,0xe0,0x27,0x21,0xb5,0x3b, -0xcd,0x50,0xf6,0x0a,0x73,0x91,0xa8,0xef,0xc6,0xc0,0x21,0xc7, -0xff,0xb3,0x54,0x3d,0xbb,0x93,0xaa,0xfd,0xe1,0xe5,0xff,0x30, -0x4d,0xd0,0x93,0x77,0x50,0x75,0xa9,0xad,0x3d,0xa6,0xf3,0x73, -0x75,0x68,0x45,0x6f,0x56,0xc3,0x52,0xeb,0xe9,0x81,0x86,0xd2, -0x0e,0xc1,0x31,0x57,0xa2,0x47,0x40,0x3a,0xfc,0xe3,0x5f,0x40, -0x4a,0x20,0x3b,0x88,0x2f,0x40,0xfa,0x67,0xa0,0x30,0x51,0xcc, -0x99,0x65,0xf2,0x3c,0x12,0xd0,0x35,0x3f,0xb6,0x57,0x6b,0x35, -0xdf,0x66,0x40,0xbd,0xb8,0xd8,0xbb,0x29,0xc2,0xd3,0x6e,0xbb, -0x6a,0x2b,0xfc,0xfb,0x0e,0xef,0xb5,0xa9,0x71,0x4e,0x14,0x38, -0xe1,0x59,0xe7,0xa9,0x2b,0xfc,0xb4,0xfd,0x48,0xf3,0x60,0x21, -0x3c,0x57,0x8b,0x96,0xdd,0xb8,0xb6,0x25,0xe8,0xfb,0x7a,0x6e, -0x7d,0x35,0xdf,0x6b,0x57,0x14,0x4b,0x70,0x8c,0x64,0x93,0x2b, -0x48,0x2e,0x41,0xea,0xe9,0x90,0xc0,0x23,0x7a,0x06,0x6b,0xe8, -0xb9,0xfa,0x6c,0x2e,0x58,0xee,0xb8,0xc0,0xdb,0x60,0x44,0xd7, -0x17,0x78,0x31,0xe5,0x83,0xcd,0xf9,0x78,0xb4,0x06,0x7c,0x08, -0xf5,0xd3,0xc3,0x8b,0x35,0x6c,0x6c,0xc6,0xe3,0x34,0x67,0x55, -0x44,0x95,0xf6,0x17,0xc4,0xc2,0x15,0xd5,0xbe,0xac,0xb0,0x5c, -0xf1,0x7c,0x58,0x3e,0xbf,0x28,0xce,0x60,0xee,0xe0,0xed,0x1d, -0xee,0xca,0x15,0x06,0x1a,0x72,0xb9,0x5b,0xef,0x4f,0x83,0x1d, -0x5f,0x9a,0xb5,0x0d,0x3a,0xa9,0x0a,0x76,0x24,0x0c,0xf0,0xd5, -0xd3,0x36,0xc9,0x34,0x76,0x61,0xac,0x69,0x03,0xbd,0x6c,0x2a, -0xb9,0xcb,0xe8,0xdd,0x15,0x0b,0x46,0x72,0x51,0x7d,0xa5,0xdc, -0x41,0x97,0xe6,0xd9,0xa5,0x44,0xa3,0x7e,0x43,0x44,0x42,0xec, -0x8a,0xcd,0x12,0xad,0x97,0xa9,0x6c,0x26,0xd2,0x66,0xa7,0x6e, -0x5e,0x78,0x22,0x9c,0x80,0x84,0xde,0x10,0x9c,0x84,0x24,0x8e, -0x20,0xf8,0x04,0x79,0x25,0xb1,0x58,0x53,0xae,0x4e,0x92,0x69, -0x14,0x06,0x3b,0x84,0xf0,0x44,0x39,0x4a,0xfd,0x9c,0x52,0x13, -0xd5,0xd1,0xf7,0x39,0x24,0xed,0xb3,0x25,0xed,0x33,0x53,0xa5, -0x4c,0xfa,0xef,0x4f,0x58,0xc7,0x3c,0xf9,0xcb,0x15,0x86,0x07, -0xe9,0xaa,0x26,0xf2,0xb3,0xab,0x3f,0x8e,0x6e,0x99,0x50,0x8f, -0xd4,0x55,0x43,0x78,0x82,0x26,0xff,0xe6,0x11,0xa3,0xdb,0xc4, -0xac,0xd4,0x7f,0xdb,0xea,0xda,0x6a,0xbf,0x0b,0x13,0xff,0x62, -0x85,0xce,0xed,0xc3,0x8a,0x0a,0x51,0x62,0x77,0xea,0x86,0x46, -0xda,0xfc,0x66,0xb3,0xb6,0x8b,0xd3,0xaf,0xbe,0x59,0x45,0x00, -0x62,0xaf,0x19,0x67,0x92,0xe5,0xbf,0xb9,0x8e,0x8a,0x55,0x4c, -0x9d,0x68,0x59,0x2c,0xbe,0xc9,0xdc,0x9b,0x4f,0xfe,0xea,0x9b, -0x9e,0x1c,0x06,0x53,0x3d,0xdf,0xe0,0x7d,0x16,0xcd,0x14,0xbe, -0x4c,0x72,0x92,0xe8,0xa6,0x24,0xcb,0x56,0xb0,0xb0,0xc2,0xfe, -0x5e,0xb8,0x6a,0x2c,0x4f,0x14,0x7a,0xe6,0x70,0xb6,0x2e,0xc6, -0x2a,0x1c,0x0e,0x5b,0x3a,0x5b,0x75,0xe2,0x14,0xf8,0xd6,0x62, -0xd6,0x48,0xf0,0x0d,0xff,0xbc,0xea,0xd0,0x8b,0xd7,0x81,0x65, -0x2a,0x1e,0x41,0xeb,0xef,0x61,0x6a,0x57,0x7a,0x75,0x6e,0x15, -0xea,0x4b,0x91,0x1d,0x63,0x7d,0x28,0x59,0x8a,0x23,0x8b,0x37, -0xf7,0x56,0x15,0x4c,0xbb,0x75,0xf5,0xc8,0xc5,0xc7,0xc4,0xfa, -0x93,0x4a,0xdc,0xbe,0xe4,0x74,0x42,0x3b,0x96,0x3c,0x78,0x55, -0x99,0x12,0x35,0x3a,0x4a,0x50,0x00,0x38,0x0e,0x75,0x3e,0xf9, -0x65,0xf1,0xaf,0x32,0x38,0x35,0x84,0x24,0x29,0xa0,0xf9,0xf4, -0xef,0xfb,0xef,0xde,0xe6,0x3a,0xc1,0x9d,0x61,0x44,0xa7,0x43, -0xea,0x90,0x32,0xb7,0x4b,0xb9,0x3e,0x28,0x92,0x90,0xe8,0xb1, -0xaa,0xc3,0x3a,0x73,0x0a,0x43,0x69,0x73,0x60,0xf5,0xc0,0x4c, -0x81,0x3b,0xbe,0x1a,0x4c,0x5b,0x04,0x1f,0x15,0x48,0x5b,0x20, -0xe9,0xc5,0x50,0x58,0x76,0x0b,0x33,0x1f,0x76,0xf1,0x4f,0x68, -0xff,0xa0,0x5f,0xbe,0xc2,0xab,0xb3,0x36,0xc3,0x69,0x9d,0x34, -0x04,0x3b,0x8e,0x22,0x2c,0x16,0xce,0x96,0x7d,0xea,0x43,0xe1, -0xbb,0xa2,0xda,0xa7,0x94,0x54,0x2d,0x71,0xb8,0xf9,0x17,0x0b, -0xb1,0xaf,0x29,0x55,0x9b,0x76,0x37,0x05,0x45,0x38,0x9f,0xfe, -0x7e,0xb5,0xfd,0x84,0xbe,0xba,0xe9,0xc1,0xbd,0x1b,0x2a,0x02, -0xa6,0x36,0xc1,0x3f,0x5f,0x3d,0x6e,0x4b,0x83,0x28,0xe0,0x91, -0xb8,0x72,0x12,0x5a,0xec,0x10,0xc2,0x9f,0xaf,0x3a,0xdb,0xe2, -0xc8,0xca,0x53,0x42,0x19,0x5b,0xe8,0x85,0x94,0xe0,0x39,0x9b, -0xfe,0x32,0x7f,0xad,0x3b,0x38,0x71,0xa5,0xdd,0xa6,0x9b,0x7c, -0x0a,0xdd,0x72,0x06,0xb8,0x56,0xed,0xda,0x26,0xad,0xf2,0x8f, -0x7c,0xdf,0x36,0xf9,0x9a,0xf2,0xa3,0xc0,0xbc,0x1f,0x48,0x4e, -0x9b,0x72,0x65,0xf9,0x91,0xba,0x35,0x75,0xe4,0x09,0x07,0x41, -0x4a,0xf4,0x78,0xa1,0xba,0xce,0xea,0xea,0xac,0xb6,0x6d,0x3c, -0x1b,0x4f,0x4e,0xc6,0x39,0xed,0x41,0xac,0x86,0x34,0xf0,0x3c, -0xe6,0x81,0xcc,0xba,0x47,0x10,0x68,0x65,0x9c,0x1b,0x22,0x97, -0x1d,0x19,0xfc,0xda,0x83,0x14,0x67,0x08,0xa8,0xb1,0x8a,0x6a, -0x33,0x35,0xff,0xe2,0x99,0x4b,0x6c,0x34,0xa0,0x40,0xba,0xab, -0xdb,0x1b,0xa3,0xcf,0x47,0xb3,0x4b,0x35,0x3e,0xe7,0xd2,0x1c, -0xd3,0x0d,0x6c,0x4b,0xf1,0x4c,0xe3,0x1c,0x9e,0xe5,0x8d,0x2f, -0x39,0x75,0xf9,0x4b,0x14,0x6f,0xde,0x4a,0xf1,0xe6,0x0a,0x8a, -0x37,0xef,0xa6,0xf8,0x5f,0x5e,0x52,0xc3,0x15,0x82,0xfc,0xee, -0xd0,0x95,0xca,0xf2,0x2f,0xa3,0x25,0xff,0x33,0x8c,0xdb,0x59, -0x5b,0x87,0x96,0xe2,0x92,0x6b,0x4d,0xbf,0x5d,0xa5,0xb4,0x66, -0x7a,0xee,0x4d,0xa6,0x74,0x30,0x51,0xca,0x9a,0x73,0x72,0x2b, -0x60,0x96,0x3e,0x40,0xbc,0x13,0xf7,0xcb,0x46,0x6a,0xef,0x00, -0x45,0xb9,0x72,0x5f,0x26,0x15,0xcf,0x78,0x61,0x26,0x4f,0x7e, -0xeb,0x4c,0x2f,0xcc,0x93,0xa3,0x66,0xef,0x83,0x2f,0xc2,0x6b, -0xcf,0x9f,0x53,0xc7,0x8e,0x43,0x00,0x77,0x06,0xdd,0xac,0x6a, -0xa9,0xd8,0xbb,0x5f,0x30,0x76,0x86,0xe5,0xbe,0xcd,0x14,0x27, -0x46,0x2a,0xe5,0x59,0xbb,0x97,0x6c,0xe2,0x26,0x0e,0x1a,0xee, -0xc7,0xa4,0x49,0xfe,0xb5,0x4a,0x92,0x9d,0xcb,0x66,0xf2,0x4f, -0xce,0x10,0x66,0xd8,0x93,0x91,0x8f,0xe1,0x99,0x28,0x19,0xbf, -0xb1,0x47,0x93,0x04,0xa3,0x1b,0x51,0x12,0x7d,0xda,0x62,0x21, -0x9f,0xb0,0x58,0x08,0x59,0x3d,0x8c,0x33,0xe6,0x01,0x7a,0xf4, -0xb7,0x89,0xd1,0x7a,0x18,0x3f,0x83,0xf7,0xe4,0xd7,0xd7,0x60, -0x8b,0xdb,0xa9,0x8a,0x21,0x72,0x30,0xee,0x64,0xf2,0x63,0x7b, -0x37,0x01,0x39,0xc7,0x5f,0x6d,0xab,0x77,0x13,0x91,0xd6,0x27, -0xc7,0xd6,0x59,0x05,0xda,0x65,0xd5,0xba,0x47,0x2a,0x56,0xf8, -0x9f,0x40,0xa1,0xc6,0xd4,0xf8,0x6b,0x2a,0xc2,0x19,0x90,0xb2, -0xe4,0x53,0x41,0x9b,0x90,0xf8,0x91,0x18,0x3e,0xf8,0x49,0xe3, -0x7d,0x3e,0x8e,0x57,0xf4,0x45,0x44,0x83,0xae,0xb2,0x66,0xb0, -0x71,0xc5,0xa6,0x63,0xfe,0x7a,0xa4,0x6e,0xf1,0x5a,0xd1,0xec, -0x6c,0x07,0xe9,0x81,0xb9,0x13,0x59,0x7f,0x98,0x9b,0x95,0xf6, -0xce,0x38,0x1b,0x8e,0x73,0x7e,0xfb,0xaa,0xd3,0x76,0x07,0x8e, -0x20,0x97,0xf2,0x42,0xe3,0xb2,0xda,0x78,0x2b,0xb7,0x97,0xa0, -0xca,0x16,0x73,0x8f,0x51,0x59,0x9e,0x64,0xdb,0xfc,0x88,0x5a, -0xd4,0xc7,0xfc,0x38,0x83,0x47,0x84,0xf2,0xbb,0x09,0x44,0xd0, -0xb4,0x81,0x2b,0x53,0xef,0x79,0xa9,0xd7,0x08,0xdd,0x2b,0x09, -0x7f,0x34,0xda,0xa7,0x14,0x74,0x7f,0x55,0xee,0x66,0x79,0x79, -0x63,0x83,0x7f,0x5b,0xc3,0xcb,0x51,0x8f,0x1f,0xf1,0xe0,0x10, -0x51,0x24,0xfe,0xd6,0x88,0x04,0xb4,0x35,0xd6,0x85,0x13,0x57, -0xa9,0x8e,0xcd,0x88,0x02,0xbe,0x20,0x09,0x0b,0xd1,0x64,0xca, -0x35,0xaf,0xe7,0xf9,0x7a,0x7b,0x63,0x43,0x93,0x22,0xcd,0xba, -0x64,0xb7,0x66,0x36,0x75,0xc9,0x31,0x3b,0x79,0xcb,0xf8,0xff, -0x06,0x00,0x00,0xff,0xff,0x07,0x1f,0x31,0x88,0x88,0xcc,0x05, -0x00, +0xf7,0xf6,0xfd,0x6d,0xdb,0xc8,0xc2,0xe8,0xdf,0xf7,0x7c,0x0a, +0x9b,0x27,0xeb,0x90,0x11,0x2c,0x5b,0x4e,0xda,0x6e,0xa9,0xd0, +0x7a,0xd2,0xbc,0xb4,0x39,0x4f,0x9c,0xe4,0x89,0x9d,0xed,0x6e, +0x65,0xd5,0x3f,0xd9,0xa2,0x6d,0x36,0xb6,0xe4,0x8a,0x52,0x93, +0xd4,0xd2,0xfd,0xec,0x17,0xf3,0x02,0x60,0x00,0x92,0xb2,0xd2, +0x76,0xcf,0x73,0xcf,0x6e,0x63,0x91,0xc0,0x60,0x30,0xc4,0xcb, +0x60,0x30,0x18,0xcc,0x9c,0x01,0x8b,0x82,0x77,0x73,0x51,0x4a, +0x89,0xa1,0x55,0x59,0xcd,0x00,0x9b,0xf2,0x71,0x29,0x0f,0x93, +0xbd,0x87,0x65,0xbf,0x44,0xc9,0x25,0x50,0x9e,0xc1,0x3c,0xd5, +0xc3,0x4c,0x8e,0xbf,0xda,0xd1,0xba,0xde,0x38,0x2c,0xed,0x01, +0x76,0x39,0x3d,0xd3,0xdb,0x01,0x13,0x25,0x86,0x1b,0x84,0xf7, +0x04,0x3a,0x4f,0x4d,0xc6,0xe0,0xd4,0xd7,0x82,0xd0,0x2b,0x5e, +0x09,0x01,0x5f,0x9e,0xf4,0x0d,0x36,0x17,0x92,0x4a,0x4c,0xea, +0xd6,0xb3,0x8b,0x7b,0xbe,0xb3,0x15,0x76,0x18,0x8a,0x59,0xfc, +0xe2,0x46,0x3b,0x6c,0x75,0x51,0xe7,0x98,0x4f,0xc1,0xa1,0x1c, +0x0c,0xa4,0xf9,0xcd,0xab,0x61,0x39,0x63,0x22,0xd9,0x09,0xa0, +0x14,0xb5,0x25,0x42,0x18,0x6d,0xe2,0xd5,0x1b,0xae,0x22,0x1d, +0x83,0xc7,0x26,0x41,0xed,0x35,0x53,0xc5,0xcb,0x0f,0xe1,0x09, +0xc9,0x7f,0x2c,0xfd,0x18,0x1a,0x28,0x26,0x20,0x2b,0x7c,0x52, +0xbe,0x73,0x16,0x98,0x31,0x35,0xb9,0xb4,0x7a,0x9a,0x9e,0xb1, +0xf7,0xa1,0xcb,0xa2,0xa4,0x3d,0xfe,0xcb,0x51,0xd6,0x6a,0x79, +0x6d,0xd0,0xd5,0x50,0x3d,0x1a,0x50,0x20,0x7b,0x40,0x21,0x75, +0x8b,0x8e,0xc5,0x53,0x7f,0x7c,0x2d,0x93,0x76,0x39,0x3f,0x3b, +0xcb,0x21,0xaa,0x89,0xa9,0x41,0x8f,0xa2,0x9b,0xc9,0xb8,0x24, +0x6b,0x25,0xaf,0x96,0x2c,0xf3,0x6a,0x61,0x03,0x9c,0xfc,0x23, +0x35,0x8e,0x9c,0xa3,0x5d,0x39,0x78,0x62,0x03,0x52,0x9d,0xa3, +0x4d,0x1d,0x55,0x69,0x78,0x83,0x21,0x68,0x4b,0x9a,0xc2,0x7e, +0x22,0xc9,0x5d,0xf6,0x2b,0x2a,0xb3,0xde,0x1f,0x3b,0xd0,0x1b, +0x6e,0xa8,0xd9,0x19,0x16,0x80,0xb5,0x8d,0xcb,0x9d,0x38,0x49, +0xbc,0xa1,0xe2,0xf9,0xc5,0xf5,0x46,0x39,0x6c,0x96,0xe5,0xfb, +0xd6,0xd6,0xa6,0xb7,0x9b,0x0a,0x60,0xbd,0x89,0x1e,0x7c,0xbe, +0x2e,0x72,0x0d,0xcb,0xfa,0xbd,0xc2,0x6b,0x24,0xf6,0xb2,0x9d, +0x78,0xbb,0x34,0x3b,0x07,0x49,0xf3,0xd3,0x46,0xfd,0x7b,0x1c, +0xa8,0xba,0x9b,0x7a,0x14,0x22,0xc4,0x34,0xf6,0xc8,0xd2,0x55, +0x54,0x47,0x0d,0xdf,0x43,0x04,0x82,0x92,0x74,0x25,0x1a,0xc7, +0x51,0x8b,0xd9,0xba,0x12,0xe7,0x2a,0x4b,0x1c,0xc9,0xb5,0xca, +0xaa,0x2e,0xac,0x6c,0x53,0x55,0x64,0x1c,0xa5,0xeb,0x7d,0x3d, +0x19,0x83,0xec,0x0e,0x27,0x54,0x0d,0xd5,0x4b,0xce,0x29,0x97, +0x74,0x2c,0xfe,0x56,0x4b,0x5a,0x28,0x2e,0x79,0x4b,0x01,0x5b, +0x71,0xaa,0xc6,0x0d,0x1d,0x01,0x28,0x7f,0x63,0x87,0x5e,0xb6, +0xdf,0x3d,0x79,0xfa,0x3c,0xdb,0xb9,0x5d,0xee,0x18,0xcf,0x24, +0x9e,0x55,0xc6,0x3a,0x21,0x9b,0xc8,0xe0,0x0b,0x0e,0x2a,0x2d, +0x7b,0x3d,0x83,0x1e,0x55,0x1f,0x2f,0xf3,0xb1,0x4d,0xa2,0xb5, +0x1f,0x92,0x82,0x78,0x6a,0x41,0x6e,0x02,0xbe,0xbf,0xb4,0x38, +0xcf,0x5c,0x1c,0x9f,0x17,0x8b,0x5d,0x44,0x56,0x7a,0x51,0x86, +0x19,0x3d,0x3a,0x1f,0xa1,0x6c,0xfd,0x06,0x2e,0x4b,0xf4,0x2b, +0x1e,0xef,0x1d,0x7e,0xbe,0x3e,0x9d,0x5c,0x79,0x3b,0xd1,0xb6, +0xc8,0xd0,0xe3,0x3c,0x1f,0x8f,0xea,0x80,0x6c,0x72,0x0c,0x17, +0x8f,0x7e,0xd4,0xa8,0xb3,0x9d,0x9f,0xa1,0x86,0x18,0x6c,0x8e, +0xcb,0xa4,0x17,0xb7,0x41,0x99,0x62,0xe5,0x6e,0x5c,0xb1,0xdc, +0x3a,0x66,0x8f,0x26,0x95,0xf5,0xa0,0xcc,0x06,0x98,0x88,0x8b, +0x14,0x23,0x7e,0x96,0xe6,0xe3,0xf8,0x05,0x81,0xc5,0x81,0x05, +0xb0,0xe1,0x34,0x22,0x00,0x8b,0xbc,0x08,0x1b,0x68,0x08,0x0d, +0xe7,0xf6,0xdb,0x7a,0xf0,0x0f,0xb2,0x86,0xc6,0xed,0x7b,0xf8, +0x28,0x94,0x91,0xa1,0x1f,0xab,0xae,0xfd,0x00,0xf4,0xab,0x21, +0xda,0x16,0xfd,0x65,0x04,0x7b,0x7b,0x0b,0x6d,0x29,0xc2,0x41, +0x25,0xfb,0xa0,0x65,0xc7,0x47,0x0b,0x8e,0xed,0xa9,0x57,0x5b, +0xb6,0x99,0x13,0x31,0xbd,0x57,0x1c,0x7a,0x88,0xc8,0xa0,0x72, +0x1c,0x58,0xdc,0xce,0x0d,0x0a,0xc5,0x6a,0x61,0xed,0x19,0xf8, +0x17,0xc6,0x47,0xf0,0x42,0x8a,0xdf,0xb2,0x58,0xb0,0xca,0x84, +0xe7,0x45,0xfb,0x06,0x67,0xd5,0x53,0x13,0x08,0x76,0x9b,0x08, +0x4c,0x12,0x39,0xac,0xfa,0x98,0x35,0x08,0xa6,0xc0,0x26,0x84, +0xe1,0x11,0xdb,0x1d,0x0e,0x51,0xe6,0xa9,0x16,0x4c,0xc0,0xb0, +0x84,0x59,0xcf,0xbb,0xfc,0x26,0x1f,0xfa,0x76,0x2f,0xbc,0x97, +0xa9,0x95,0xfd,0x31,0x4f,0xb9,0x2d,0x00,0xfa,0x42,0xfe,0xfe, +0xe4,0xdd,0xf3,0x83,0x37,0xff,0x78,0xfe,0x2c,0x8b,0xee,0xdd, +0x73,0x6f,0x91,0x45,0x5f,0x09,0xab,0x43,0xc9,0x91,0x69,0xa3, +0x2f,0xdc,0x42,0xac,0x96,0xd6,0x80,0x49,0xb8,0x8d,0xed,0x5a, +0x3b,0x06,0x5d,0xfd,0xd9,0x87,0xef,0x3e,0xeb,0xb6,0x55,0xee, +0x91,0x8d,0xe6,0x39,0xe1,0xe5,0xe8,0xc5,0xd8,0x3d,0xa3,0x46, +0x54,0x26,0xbc,0x39,0xfd,0x45,0xbf,0x5e,0x5d,0x96,0x6a,0xaa, +0xff,0x61,0xef,0xbc,0xc4,0xdb,0x93,0xe7,0x85,0xc6,0xa1,0x07, +0xaa,0x78,0x73,0x83,0xd4,0x98,0x2e,0x70,0x7b,0x50,0xcc,0xf4, +0x4c,0x0c,0x62,0x1b,0x83,0xbd,0x7c,0x00,0xd3,0xfc,0xb8,0x6c, +0x15,0x63,0xfd,0x27,0x6e,0x3f,0xe8,0x25,0x68,0x8e,0x55,0xb6, +0x90,0x04,0xfd,0x7b,0xfa,0x19,0x33,0x00,0xe8,0x01,0xd8,0x7f, +0x25,0x78,0xc3,0xf3,0xc5,0xf8,0x15,0x5e,0xb8,0xc9,0x6e,0xef, +0x15,0xa3,0x94,0x3d,0xd0,0x60,0xd8,0xf7,0x4d,0xc4,0xcd,0x06, +0x0e,0x7e,0x47,0xc5,0x51,0x11,0xc4,0x3a,0x77,0x24,0xc1,0xb8, +0x05,0xc5,0x31,0x5c,0x94,0xbd,0x7f,0x52,0x68,0x19,0xea,0x04, +0x92,0x4e,0xce,0x40,0x05,0x81,0x0d,0x7d,0xd2,0xa7,0xf6,0xdc, +0x38,0xfd,0xbc,0x71,0x52,0x8c,0x4e,0x06,0xf7,0x31,0x2e,0xba, +0x74,0x86,0xe2,0xd0,0xe1,0x51,0xa1,0x6e,0xb6,0xcc,0xc6,0x8d, +0x9f,0xda,0x97,0xbd,0x81,0xe8,0x0d,0x4e,0x7b,0x24,0xd3,0x7a, +0x71,0xd8,0x5b,0x66,0x13,0xed,0xd2,0x13,0xd9,0x81,0x99,0xf4, +0xd1,0x43,0xbd,0xa4,0xfc,0x90,0x17,0x5e,0x4f,0x69,0x46,0x28, +0xdb,0xb0,0xef,0x65,0xa2,0xaf,0x16,0xbf,0x91,0xfb,0x41,0xbf, +0x1b,0xc7,0xe1,0x12,0x86,0x43,0x03,0x66,0xe4,0x18,0x32,0xa4, +0xdf,0x0c,0x52,0x59,0x02,0x22,0x6b,0xc5,0xe1,0xc8,0xab,0xf9, +0x12,0xfb,0x0d,0xc6,0xcf,0x10,0x6b,0x11,0x83,0x41,0xea,0x95, +0x94,0xdf,0xbd,0xe4,0xb0,0xfe,0x99,0xee,0x0e,0x3b,0xf0,0xe2, +0x5e,0x1a,0xf7,0x8f,0xef,0x1d,0x7f,0x84,0x4b,0x20,0xc7,0xb1, +0x7d,0xd6,0x83,0x4c,0xa1,0x03,0x22,0xf3,0x9e,0x24,0x30,0xe4, +0x56,0x0f,0xaa,0x62,0x44,0xc3,0x8a,0xc7,0xcd,0x7d,0x18,0x38, +0x0d,0x63,0xe8,0xfe,0x46,0x79,0x39,0x99,0x5f,0x8d,0xf0,0xa6, +0xb5,0x1e,0x74,0xb6,0x4d,0xe1,0xba,0xf5,0xfd,0xf8,0x44,0xd3, +0x7b,0xa2,0x36,0x4e,0xf0,0x13,0x4f,0x92,0xfb,0x62,0x80,0xaa, +0xca,0x68,0xd3,0xdf,0xc3,0x7e,0x94,0x5d,0xd7,0x64,0xee,0xfe, +0xbe,0x1d,0x79,0x5e,0xf7,0xda,0x31,0x48,0x97,0x51,0xb4,0xf8, +0xf6,0x9d,0xe6,0xcf,0x1f,0x0e,0xc0,0x99,0xea,0xd2,0x57,0x25, +0x3c,0xb5,0x64,0xc7,0x30,0xf9,0x9d,0x68,0x6f,0xd3,0xa5,0x2f, +0x50,0xf6,0x04,0x0d,0xce,0x6d,0xc8,0x17,0x0e,0x74,0x28,0xc5, +0x4b,0x17,0x1a,0x01,0xe5,0x06,0xa8,0xed,0x3e,0xe5,0x10,0xea, +0xee,0x2d,0xd5,0x29,0x10,0x84,0x86,0xc3,0xc5,0x64,0x5e,0xa2, +0x2f,0x1e,0xc3,0xdc,0xe0,0x40,0x18,0x6a,0x10,0x34,0xbb,0xf7, +0x37,0x53,0x74,0xbb,0x34,0xe8,0xba,0x10,0x8c,0xaf,0x8a,0x0f, +0xb9,0xa4,0x37,0xf1,0xab,0xca,0xdc,0xab,0x37,0x9d,0xc4,0xf3, +0x62,0x11,0x0e,0x51,0xba,0x99,0x7f,0x37,0x38,0xb1,0xce,0xa0, +0x42,0x4d,0x9d,0x70,0xd7,0x2d,0x28,0x73,0x8f,0x41,0x9c,0x40, +0x1c,0xce,0x5b,0x5b,0xd1,0xbd,0x68,0x13,0x66,0xa6,0x73,0x64, +0xa0,0x45,0x79,0x0f,0x37,0xc7,0x7c,0xd4,0xd0,0xdd,0x20,0x03, +0xc2,0xd2,0x68,0xb1,0x1c,0x2a,0x16,0xbd,0x92,0x05,0x50,0xdc, +0x81,0x1c,0xe9,0xdf,0x6f,0x55,0xce,0x6c,0x28,0x42,0x33,0x7f, +0xb7,0xcb,0x71,0xfc,0x29,0xdc,0x22,0xfe,0x25,0x37,0xe6,0xe0, +0xad,0xca,0x95,0x84,0x2d,0x89,0x87,0xa6,0x87,0xa0,0xa9,0x9f, +0xc8,0xf1,0x78,0x68,0xac,0x88,0x02,0xe4,0x4c,0xcc,0x36,0xb2, +0x6c,0xfb,0x0a,0xf3,0x53,0x72,0x78,0x87,0xcd,0x6a,0x0b,0x26, +0x09,0x0e,0xb8,0x4c,0x02,0xf7,0x6d,0xee,0x40,0xb1,0xb3,0x82, +0xa6,0x6c,0x39,0x1c,0x45,0x7a,0x46,0xa3,0xd8,0x6f,0x46,0xfe, +0x28,0xca,0xb3,0x1e,0x1e,0x24,0x86,0x15,0x54,0x12,0xf3,0x71, +0xee,0x33,0x25,0x62,0x37,0x39,0x11,0x75,0x72,0x8b,0x3f,0x5b, +0x5b,0xf8,0x43,0xf2,0x38,0x79,0xa1,0x8a,0xbd,0xcf,0xa0,0xec, +0xc2,0x50,0x4b,0xe7,0x58,0x3e,0x73,0x1b,0xcd,0x6f,0xe0,0xfa, +0x5f,0xf4,0x6c,0x0e,0xc1,0x8c,0xb4,0x6c,0x54,0xa2,0x9f,0xf9, +0x8d,0x29,0x42,0x69,0xbe,0x35,0x9c,0xe6,0xe8,0xb4,0x66,0x78, +0x05,0x62,0xf5,0xa8,0xbd,0xf1,0xbe,0xcc,0x37,0xee,0x9b,0x95, +0x52,0x72,0xb0,0x8d,0xd9,0x64,0x83,0x22,0x99,0x7c,0xde,0x98, +0x8f,0x0b,0xbd,0x99,0x04,0xf6,0x5c,0xb6,0x37,0xde,0x31,0xae, +0x74,0x43,0x73,0x36,0xb5,0x61,0xab,0x82,0x6c,0x36,0x2c,0x14, +0x7c,0xd0,0x35,0x49,0xb7,0xbe,0x75,0x6f,0xb5,0x44,0x60,0x81, +0x96,0x8d,0x1d,0xb4,0xd9,0x59,0x8a,0xa9,0x28,0xdb,0x25,0x59, +0x35,0x6e,0x68,0x3a,0xc6,0x35,0x63,0x06,0x47,0x66,0xa0,0x29, +0xa2,0x16,0x66,0xee,0x55,0xba,0xdd,0x80,0x9f,0x5e,0xe3,0x3a, +0xd8,0x30,0x3c,0x27,0x76,0x72,0x80,0x24,0xee,0xd2,0x40,0x8b, +0x95,0x20,0x5b,0xe1,0x89,0xa8,0x56,0xcd,0xd6,0xfa,0x59,0x7a, +0xfb,0x6f,0x99,0xa6,0xd4,0x48,0xb5,0xbd,0xa4,0x82,0xa1,0xc9, +0xea,0x5c,0x52,0x06,0x89,0x6f,0xb4,0x0b,0x4a,0x26,0x17,0x84, +0xee,0x68,0xb2,0x61,0x33,0xcc,0x83,0xe7,0x01,0xf4,0xe3,0x25, +0xe8,0x78,0x4c,0x16,0x18,0x3b,0xd0,0x93,0x6c,0xd0,0xa4,0x1b, +0x10,0x91,0x59,0x5c,0xa8,0xae,0xa1,0x8e,0x44,0x57,0x68,0x41, +0x7f,0xa1,0x4e,0x89,0xfd,0x49,0x4a,0xba,0xf4,0x6e,0xc6,0x5b, +0xb7,0xb8,0xd8,0x18,0x5d,0x4c,0x92,0x63,0xef,0x26,0xa5,0x77, +0xd7,0x65,0x34,0xc9,0x5b,0xa1,0x10,0x27,0x4a,0xd4,0x89,0x70, +0x52,0x3b,0x2f,0x85,0x33,0x99,0x8e,0x5e,0x62,0x32,0x88,0x1f, +0x52,0xcd,0x83,0xc1,0x4d,0xc9,0xe0,0x9b,0xdd,0x2d,0x1c,0xdb, +0x1d,0x0f,0xec,0xba,0x18,0x81,0xcb,0xfe,0xcd,0xb8,0x82,0x77, +0xb1,0x08,0xd1,0xf9,0x44,0x4d,0x46,0xa3,0xa0,0x18,0x9c,0xdd, +0x00,0x35,0x1c,0x66,0x3a,0x09,0x47,0xc9,0x62,0x41,0xbb,0xa0, +0x55,0xc7,0x02,0xb5,0x67,0x00,0xeb,0x76,0x18,0x41,0x8b,0xf1, +0x97,0x89,0x9a,0xc2,0xd1,0x82,0xc0,0x20,0x92,0xf8,0xa3,0xc3, +0x43,0xc2,0x7d,0xcf,0xb0,0xf8,0x57,0xb8,0x1b,0xf4,0x78,0x53, +0xc0,0x8d,0xf5,0x56,0xd6,0x13,0xcf,0x24,0xac,0x53,0xb1,0x1d, +0x5e,0x4e,0x3e,0x7e,0xe1,0xc1,0xd6,0x17,0x5e,0xfc,0x6c,0x53, +0x25,0x95,0xd3,0x24,0x83,0xb6,0x1f,0x9e,0xec,0xf4,0x22,0x71, +0x10,0x1c,0xa5,0x91,0x39,0x92,0x8f,0x06,0xce,0xa9,0xed,0xf8, +0x62,0xfb,0x12,0x5c,0x0a,0xd8,0xe3,0xf1,0x1f,0x8a,0xe0,0xe4, +0xe5,0xdf,0xf1,0x1d,0x50,0xc9,0x97,0x7c,0x87,0xa5,0x3c,0xf5, +0x3e,0x69,0xd5,0x77,0x60,0xbc,0xb3,0x3f,0x60,0xd0,0x52,0xdb, +0xec,0x80,0xcb,0xd3,0x7b,0x60,0x4a,0x09,0x61,0xae,0xe9,0x29, +0xb9,0xb5,0x8f,0x68,0xcf,0x45,0x8f,0x14,0xe4,0xda,0x24,0x9b, +0x25,0xc7,0x26,0x79,0x2d,0xa0,0x30,0x5a,0x9a,0x53,0xa6,0x9c, +0x41,0x60,0x6b,0xac,0x17,0x6d,0x17,0x94,0x45,0x2a,0x22,0xbb, +0x51,0xd0,0x76,0xa6,0x60,0x09,0xea,0x19,0x14,0x1c,0x0e,0xd1, +0x4b,0xd6,0x97,0x9e,0xb2,0xfa,0x9a,0x50,0x61,0x04,0x47,0xe8, +0xbc,0xdb,0x9c,0xee,0x72,0x4d,0x68,0xd9,0x0d,0xda,0x3c,0x50, +0x4a,0xea,0x3e,0xb8,0xd3,0x6a,0xce,0xa0,0x7e,0x2a,0xae,0x73, +0xe2,0xe1,0x57,0x4e,0x81,0x97,0x8e,0xac,0xd2,0xa6,0xb4,0x81, +0x19,0x8d,0x4b,0x68,0x85,0x1d,0x04,0x17,0x23,0xcc,0x21,0x19, +0xe1,0xe2,0x33,0x32,0x2d,0x9e,0x98,0x12,0xc8,0x32,0x50,0xe0, +0xf7,0xba,0xd6,0x96,0xaf,0x8c,0x43,0x19,0x30,0xa9,0x28,0x32, +0x1f,0x8f,0x75,0x8a,0x5c,0xec,0x17,0xe8,0x19,0xd9,0xcf,0xee, +0x17,0x03,0x79,0xa4,0x15,0x08,0x21,0xe1,0x47,0x60,0x04,0xa5, +0x30,0x11,0x6e,0xaf,0x54,0x68,0x57,0x71,0x4d,0xa3,0x64,0xd5, +0x06,0xa4,0xf6,0xef,0x47,0x9b,0x1c,0x8c,0x13,0x8c,0x74,0x1b, +0x81,0x7a,0x11,0x38,0x9d,0xab,0x84,0x4f,0x6f,0xd3,0x41,0x85, +0x93,0x91,0xea,0xfa,0x43,0x3a,0x63,0x09,0x32,0xfd,0x4e,0xac, +0x39,0xb1,0x0a,0x5a,0x14,0xb7,0x4b,0x5e,0x5a,0xa2,0xaa,0x58, +0xdb,0xe2,0x2c,0xda,0x03,0x16,0xeb,0x8e,0xfe,0xac,0xe7,0xf2, +0x24,0x93,0x0e,0x7a,0xb2,0x1a,0x64,0x3c,0x0e,0x2b,0x8d,0x4f, +0xc4,0x48,0x44,0x95,0x43,0x2d,0x97,0xa7,0x08,0xbf,0x3b,0xce, +0xa6,0x77,0xb4,0x78,0x70,0x0a,0x50,0x6a,0x7d,0xd0,0x85,0x37, +0x1d,0x81,0xac,0xd4,0x4d,0x7e,0xb5,0xbb,0xeb,0x26,0xe3,0xcf, +0x72,0x36,0xae,0xd0,0x51,0x96,0x2b,0x4d,0x1b,0x56,0x5b,0x41, +0xbb,0x11,0x64,0x8e,0x74,0xdc,0x17,0x0c,0xb2,0xbb,0x41,0x20, +0xfc,0x85,0xba,0x1b,0x8c,0x5a,0x5a,0x7e,0xbc,0x7b,0x34,0xb4, +0x99,0xc0,0x7e,0xd6,0x43,0x04,0x73,0x36,0xba,0xc3,0xfc,0x3f, +0xaa,0x39,0xf5,0x5c,0xcb,0x82,0xf7,0x4a,0x43,0xe9,0xb4,0x3f, +0xd4,0x2a,0x6e,0x5c,0x37,0x9e,0xf1,0x49,0xce,0xed,0x30,0x7a, +0x17,0xfc,0xc5,0x17,0xd2,0x05,0x10,0x97,0x92,0x89,0xe7,0x0a, +0x63,0xaf,0x55,0x7b,0x97,0xde,0x1d,0xfd,0x33,0xe1,0xb6,0x40, +0x4c,0xe3,0x8a,0xc0,0xe8,0x99,0xf3,0x81,0xcf,0x12,0x77,0x47, +0x8c,0x9c,0x96,0x10,0x20,0x19,0xda,0x97,0x67,0xd3,0xe2,0xc6, +0x3f,0x58,0x08,0x8c,0x43,0xc4,0xc7,0xc9,0x8c,0xfa,0x8b,0xb4, +0x6b,0xeb,0xff,0xc9,0xb5,0x08,0x9c,0x75,0xec,0x68,0xa1,0xc3, +0x60,0x8e,0xb2,0xcc,0xdd,0x9a,0xe5,0x88,0x92,0x94,0xf3,0x7e, +0x7a,0x45,0x59,0x70,0x43,0x4a,0x97,0xca,0x9c,0x0a,0x0d,0x4f, +0x4c,0xba,0x3e,0x75,0x7a,0x04,0xcc,0x62,0x51,0x56,0x51,0x6c, +0x41,0x66,0x26,0x6f,0x28,0x66,0x61,0xcd,0xd5,0x07,0x41,0xff, +0xd2,0xf0,0x4f,0xaf,0x71,0x84,0x61,0x4c,0xc5,0x98,0xcc,0x1a, +0xbb,0xb0,0x55,0x19,0x9f,0xba,0xbc,0x79,0x7b,0xf4,0xf2,0xcd, +0xeb,0x43,0xff,0x86,0x36,0x1c,0x0d,0xc4,0xbd,0xf4,0xb8,0x6c, +0x0d,0x4b,0x73,0x54,0x90,0xf4,0x28,0xe5,0x62,0x3a,0x99,0xdf, +0xb8,0xb3,0x02,0x70,0x35,0xab,0x1f,0xf4,0x3a,0x02,0xaf,0x56, +0xd3,0x8b,0x7f,0x8f,0x47,0x83,0x07,0xe8,0xe3,0x87,0x1d,0xcd, +0x07,0x39,0xbe,0xfe,0xd7,0x4b,0x3f,0xc6,0xdb,0xe0,0xe2,0xa0, +0x82,0xea,0x0e,0xce,0x29,0x90,0x2a,0xb0,0x01,0xd7,0x1b,0x8e, +0x03,0x7b,0x9b,0xe5,0xd6,0xbb,0x1c,0xc6,0x9e,0x35,0x56,0xdd, +0xab,0xee,0x47,0xd4,0x94,0xba,0xd9,0xec,0x65,0x8d,0x41,0x20, +0x0a,0x89,0x1b,0xc4,0xf6,0xca,0x31,0x5f,0x2f,0x76,0x4d,0x6c, +0xa7,0x07,0x4f,0x17,0x3e,0x42,0x37,0x0e,0x44,0xa8,0x63,0xd5, +0x7c,0xfc,0x61,0x3c,0xf9,0x38,0xe6,0x37,0x70,0x68,0x46,0x41, +0xaf,0x38,0x56,0xa5,0xd1,0xb4,0x5e,0xb8,0x2f,0xf2,0xbe,0x0f, +0xd6,0xb3,0x73,0xb4,0xb1,0x3d,0x9d,0xcc,0xc7,0x23,0x3a,0xfe, +0x29,0xdd,0x3d,0x50,0xc8,0x2d,0xc6,0x85,0xb0,0xce,0x11,0xa8, +0x4e,0x94,0x23,0xe4,0xc4,0xa7,0xe4,0x24,0xb9,0xf5,0xea,0xac, +0x2f,0x94,0x35,0x96,0xcf,0x7c,0x6c,0x4b,0xa2,0x44,0x4b,0xf5, +0x9c,0x1d,0x8a,0x60,0xee,0x73,0xf9,0x20,0x12,0xee,0xe0,0x89, +0x5a,0xe5,0x75,0x2a,0x7b,0xcf,0xca,0x32,0x0b,0x10,0x64,0xf8, +0x4a,0x9b,0x57,0xaf,0x5d,0xa8,0xb7,0xb6,0xfc,0x74,0xe3,0x66, +0x3d,0x61,0xc2,0xe8,0xbd,0x81,0x36,0xe3,0x8a,0x97,0xb2,0x9d, +0x11,0x1d,0xab,0x27,0x2b,0xa4,0xdf,0x45,0x37,0xc7,0x1a,0x80, +0x7b,0x60,0xef,0x25,0x55,0xf6,0xba,0x85,0xa1,0xa9,0x02,0xe1, +0x91,0x46,0x43,0x89,0xbf,0xeb,0x1f,0x10,0xa1,0xb9,0xb7,0x11, +0xb5,0xc4,0x51,0x4d,0xd2,0x8a,0x36,0x7a,0x51,0xd7,0xff,0x72, +0x68,0x2a,0x57,0x46,0x30,0x5c,0x0e,0x15,0x15,0x7b,0xe0,0x22, +0x3f,0x2c,0x18,0x34,0x34,0x38,0xcf,0x89,0x8c,0x70,0x85,0x26, +0xd5,0xfc,0x19,0xb6,0xe1,0x9a,0xae,0x29,0xb9,0xf6,0x0b,0x15, +0x7e,0xe6,0xb8,0xc9,0x68,0x6b,0x1a,0x2f,0x92,0x35,0xb6,0x17, +0x4e,0xf7,0x64,0x9d,0xad,0x09,0x5f,0x85,0xb3,0x17,0xfa,0x0f, +0x8b,0xf1,0xc5,0x55,0xce,0x90,0xf4,0x5d,0xd6,0xac,0xc9,0x75, +0x2f,0xe7,0xe0,0x5d,0xb5,0x5b,0xaf,0xdb,0xf9,0x92,0x75,0x78, +0xd2,0x6f,0x07,0x43,0xfd,0x18,0xe9,0x3a,0x7c,0x72,0xc4,0xd9, +0x8b,0x3f,0xbd,0xf8,0xcb,0xc6,0xb7,0x4f,0x39,0xcd,0x13,0x8b, +0x8c,0xa3,0x82,0xbb,0x7b,0x43,0x39,0x5c,0x0f,0x6b,0xee,0xd0, +0x24,0xf5,0xa2,0xd0,0x5a,0x3c,0x5e,0xc1,0x5e,0xb5,0x46,0xbd, +0xae,0xa7,0xe2,0xbb,0x6a,0x47,0xbe,0xbb,0xb4,0x15,0x90,0x0c, +0x7d,0x4e,0xdb,0x13,0xbf,0xc7,0x1b,0x6c,0x92,0xbf,0xb0,0x79, +0xbc,0x5e,0x90,0x0b,0x45,0x5c,0xfd,0x8c,0x84,0x0c,0x9b,0xed, +0x08,0x39,0x60,0x5f,0x63,0xb5,0x63,0xc4,0xdd,0x0f,0x03,0x4d, +0x12,0xa0,0xed,0x9e,0xad,0x18,0x17,0x70,0x62,0x59,0x82,0x25, +0xde,0x06,0x07,0x9b,0xa9,0x5c,0x66,0xeb,0xfa,0x7b,0x33,0x43, +0x18,0x84,0x58,0x8a,0x23,0x9a,0x42,0xd2,0x30,0xbd,0x21,0xb2, +0x71,0xe6,0x0c,0xea,0xb0,0x4a,0xb4,0x60,0x94,0x01,0x8e,0xf1, +0x23,0x1b,0x8d,0x55,0xf4,0x12,0x39,0xbc,0x2a,0x63,0xf3,0x49, +0x2a,0x24,0x72,0xb1,0xb0,0x79,0x19,0xba,0xc4,0x09,0x01,0x2a, +0x97,0x60,0xc3,0x11,0xfa,0x85,0xdd,0x8d,0x7b,0x3e,0xd0,0x8f, +0xf2,0xf1,0xde,0x9f,0x6f,0xa1,0xad,0x2d,0xc4,0x57,0x0d,0xfd, +0x8c,0x61,0x02,0xab,0xe3,0x04,0xa1,0x83,0x91,0xc1,0x32,0xdb, +0x8a,0x81,0x61,0x41,0x4d,0x4b,0xe0,0x97,0x50,0x75,0xdf,0x83, +0x48,0x85,0x3e,0xc8,0xc4,0x3b,0x3f,0xab,0xfc,0x13,0x5e,0x48, +0xbf,0x78,0x4b,0x71,0x3c,0xcc,0x2b,0xd7,0x18,0xbc,0x2b,0x94, +0xce,0x5e,0xc9,0xd3,0x45,0x4a,0x7a,0x89,0x1a,0x67,0xd2,0x3b, +0x9b,0x0f,0xc7,0x23,0x3b,0x43,0xa5,0x61,0x8d,0x57,0xc3,0x53, +0x2d,0x45,0x08,0x3a,0xca,0xec,0x36,0x8a,0x52,0x08,0x77,0x18, +0x10,0x5b,0x6a,0x89,0x53,0xcb,0x49,0xe2,0x26,0x75,0x78,0xb5, +0x9a,0x4e,0x2a,0x4a,0x5a,0x05,0xcb,0x17,0xcc,0x85,0x13,0xdc, +0x14,0xc1,0x69,0x14,0xa8,0xce,0x01,0x53,0xaf,0xc4,0xa8,0xbd, +0x70,0xc6,0x41,0x2b,0x40,0x99,0xd0,0x75,0xaf,0x92,0x5d,0xbd, +0xa2,0x05,0x9c,0x51,0x08,0xe4,0xb3,0x6c,0xb3,0x03,0xe7,0xce, +0x36,0x16,0x38,0x58,0xd6,0x82,0x64,0xf8,0x62,0xbc,0xb5,0x65, +0x6e,0x1a,0x3a,0x2a,0x12,0x5c,0x2d,0x6c,0x59,0x39,0xe5,0xfa, +0x83,0xc4,0xc6,0x43,0x43,0x0c,0x2f,0xf9,0x74,0xd5,0xbd,0x3c, +0x76,0x88,0x8c,0x5e,0xc8,0x65,0xb6,0x5a,0xc9,0x95,0xb0,0xda, +0xa0,0x9b,0x1c,0xae,0x40,0xdf,0x41,0x0e,0x24,0xfd,0x24,0xff, +0x13,0xc5,0x3c,0x64,0x8c,0x57,0xee,0xfa,0xc2,0x1c,0xa7,0xb4, +0xe9,0x33,0xc4,0xb7,0xca,0xa3,0x29,0x3e,0x7d,0xca,0x3e,0x54, +0x4e,0x9c,0x6b,0x0e,0xa4,0xaa,0xdf,0x41,0x3d,0xd0,0x37,0x7d, +0x74,0x65,0xed,0x57,0x28,0xfb,0x83,0x3b,0x91,0x4a,0xf9,0x98, +0x29,0x18,0x20,0x19,0x8e,0xbc,0xef,0x3e,0x87,0x1f,0x89,0x36, +0xec,0xb1,0x00,0xce,0xe4,0x70,0xeb,0x07,0x58,0xe0,0x9e,0xec, +0xfa,0xc0,0xa0,0x47,0x0b,0xc7,0xa9,0x9c,0xd9,0x36,0x35,0xd1, +0x6d,0x6d,0xc6,0x8f,0xe5,0x95,0xb2,0x8f,0x78,0xc5,0xe0,0x6e, +0xea,0xd5,0x76,0x57,0x6a,0xb6,0x66,0x5e,0x6a,0xb2,0xe9,0xee, +0x62,0xd8,0x83,0x67,0x2e,0x2f,0xaf,0xde,0xc3,0x91,0xcf,0xed, +0xb2,0x6b,0x5f,0xea,0x07,0x91,0x1d,0x38,0x99,0x4f,0x82,0x2d, +0x06,0x4e,0xc5,0x6b,0xa9,0xf3,0x07,0x8d,0x40,0x99,0x65,0x59, +0x2d,0xe1,0x5d,0x39,0xc0,0xc4,0xf3,0x62,0x61,0x5e,0x96,0xc8, +0x1e,0xb2,0x51,0x51,0xea,0xbd,0x6b,0xa5,0x63,0x89,0x79,0x64, +0xf4,0x37,0xb8,0xbe,0x59,0xe1,0x2b,0xac,0x06,0x31,0xe7,0xc7, +0x8d,0x4d,0x6c,0xc6,0x5f,0x65,0xbc,0x51,0x6d,0x8c,0xd8,0x10, +0x98,0x5a,0x4a,0xc9,0x06,0xc3,0xf4,0xb1,0x1e,0x43,0x6e,0xe7, +0x62,0xa2,0x24,0x88,0x26,0xe9,0x79,0xa3,0x4a,0xf3,0x35,0x7b, +0x39,0x1d,0x28,0xd4,0xe3,0x95,0x2a,0xd3,0x0f,0xb6,0xa6,0x4d, +0xd1,0x42,0xcb,0x24,0xf5,0xda,0x6b,0x35,0xb6,0x5e,0x2d,0x3a, +0xbd,0xa7,0x4f,0x04,0xb3,0xce,0x76,0x25,0x33,0xcf,0x2a,0x83, +0x9a,0x99,0x91,0x80,0xd9,0x77,0x85,0xbb,0xee,0x11,0xe6,0x37, +0xb4,0x44,0x38,0x39,0x43,0x84,0x7d,0x57,0xc4,0x9b,0x42,0xab, +0xe7,0x9b,0xb7,0x52,0x90,0x72,0x83,0x08,0x7b,0x9c,0x39,0x7c, +0xbd,0xd8,0x5f,0xc4,0x32,0x73,0x02,0x92,0xea,0xd2,0x58,0xd4, +0xdc,0x69,0x6d,0xa3,0xf6,0x27,0x4e,0xd8,0xa9,0x2c,0xb6,0x52, +0x14,0x4e,0x68,0x1e,0x67,0xa9,0x1c,0x4c,0x98,0xb2,0x0c,0x57, +0xc7,0xac,0xef,0x57,0x5c,0x47,0x2e,0x05,0x06,0xf6,0xcb,0x85, +0x32,0x0a,0xab,0xa6,0x7c,0x64,0x46,0xc1,0x0c,0xbe,0x7d,0xc2, +0x7a,0x2b,0xd5,0x78,0xad,0x1b,0xb4,0x46,0x50,0x18,0x0e,0x39, +0x83,0x9a,0xf0,0xeb,0x36,0xc3,0x2e,0xd3,0x52,0x78,0x2d,0x41, +0x7e,0xe3,0xd5,0xa1,0x0a,0x31,0x25,0x89,0x94,0x05,0x70,0x8f, +0xaf,0x02,0xfb,0x06,0xaf,0xb1,0x57,0x18,0x37,0x10,0x9c,0x04, +0x37,0x46,0x08,0x41,0x23,0x55,0x3e,0x9b,0x30,0x74,0x06,0x7a, +0xcb,0x23,0x89,0xf1,0xc1,0xcc,0x37,0x06,0x3d,0x6d,0x5a,0x88, +0xeb,0xa5,0xd7,0xad,0x2d,0x81,0x87,0x2c,0x97,0xeb,0x4a,0x79, +0x65,0x92,0x10,0x71,0x31,0x72,0x58,0xc1,0x7f,0x94,0x44,0x09, +0x7b,0x84,0x0a,0xb8,0x03,0xf6,0x1a,0x15,0x54,0x81,0x66,0xae, +0x3b,0x8c,0x4e,0x06,0x95,0x78,0xc3,0x9d,0x58,0x50,0x87,0xe5, +0xe8,0x01,0x12,0x18,0x88,0xb8,0xc1,0x13,0xe4,0x3a,0xa6,0xd7, +0xcb,0x45,0xef,0x52,0x52,0x6a,0x94,0x9e,0x5c,0x22,0x9c,0x85, +0x09,0x7e,0xa1,0xfb,0x1e,0x1e,0x58,0x8e,0xb0,0x90,0x00,0x6a, +0xe4,0x55,0xed,0x59,0xd6,0xcd,0x37,0xc7,0x0f,0x3c,0x29,0x34, +0x95,0x88,0x94,0x66,0x9c,0x96,0x12,0xc7,0x35,0x03,0x0a,0x96, +0x5e,0x9b,0xf7,0x64,0xa3,0x62,0x00,0x3c,0x6b,0xf3,0x93,0x36, +0x4d,0x1d,0x9e,0xe9,0xe6,0x64,0xc8,0x1b,0x89,0x9c,0xe8,0x64, +0xac,0x56,0xab,0x1b,0x7e,0x9e,0x37,0x29,0x92,0xca,0xc7,0xeb, +0x8e,0x4d,0x6c,0x65,0x66,0x53,0x8a,0x2b,0x55,0xb7,0x89,0x93, +0x4a,0xb6,0x9e,0xd4,0xf0,0x2f,0x40,0x09,0xa3,0xab,0x82,0xd5, +0xfa,0x2a,0x41,0x53,0x6d,0xf2,0x3b,0xc2,0x3d,0x0d,0x57,0x00, +0xd8,0x3e,0xb7,0xa2,0x03,0x4e,0x92,0x8a,0xc7,0x3a,0x26,0x1f, +0x7c,0xd6,0xad,0x6b,0xd5,0x4d,0x3d,0x72,0xb2,0x11,0x0f,0xcb, +0x8d,0x13,0xec,0xc1,0x93,0xa4,0x07,0xf9,0x1b,0x6c,0x78,0x9b, +0xf4,0xd8,0xf2,0xb6,0x6a,0xb6,0xeb,0x1b,0xdf,0x06,0x6e,0xef, +0xdc,0x07,0x78,0x3e,0xef,0x3c,0x86,0x9d,0xd0,0x45,0x64,0x2b, +0xa6,0x18,0x93,0xee,0x9a,0x88,0x5c,0xca,0x0a,0x5d,0xd6,0x3a, +0xdc,0xe4,0x7e,0x8d,0xbb,0x14,0x91,0xf5,0xd5,0x40,0x59,0x91, +0xd6,0x47,0x69,0x62,0x78,0xb1,0x58,0x15,0xd6,0xd7,0x33,0xd5, +0xa5,0xb6,0x36,0x86,0x2d,0x43,0xe0,0x6f,0x06,0x6c,0x6d,0xce, +0x75,0xfe,0x9d,0x0d,0x30,0x8d,0xe9,0x6b,0xcf,0x83,0xfe,0xfb, +0x20,0xc1,0x68,0xd1,0xd5,0x75,0x2d,0xeb,0xf7,0xed,0xac,0xf2, +0xf7,0xa3,0x46,0xf4,0x58,0x0e,0x06,0x5d,0xc7,0x0b,0x40,0xb9, +0x6a,0x6e,0xeb,0xb9,0x54,0x76,0x05,0x9a,0x08,0x0d,0x70,0xc5, +0xf3,0x00,0xe9,0xc3,0x6b,0x40,0xc0,0xdf,0x99,0xbf,0x8e,0xda, +0x43,0x9f,0x3f,0xad,0x02,0x90,0x1b,0x65,0x67,0x8a,0x4a,0xa9, +0xe6,0x43,0x49,0x4a,0x14,0xdb,0x5f,0xb9,0x21,0x36,0xd6,0xd2, +0xce,0x06,0xaf,0x6e,0x97,0x6a,0xb7,0x9f,0xde,0x7e,0x93,0xcd, +0x07,0x40,0x92,0xee,0x0f,0xd6,0x12,0xd9,0xbc,0x09,0xbd,0x96, +0xcc,0x16,0x88,0x6c,0x77,0x48,0x14,0xb4,0x58,0x77,0xbe,0x74, +0xb1,0xd6,0x1f,0x15,0x7b,0x6d,0x56,0xb3,0x6e,0x5b,0x31,0x47, +0xae,0x60,0xd6,0x86,0xd1,0x2b,0x4d,0x4a,0x33,0x33,0x69,0xc0, +0x00,0xb6,0xba,0x67,0x4c,0xcc,0x70,0x4e,0xee,0xda,0x78,0x0b, +0x2b,0x6d,0x22,0xde,0x21,0x14,0x1b,0x25,0x61,0x06,0x29,0x77, +0xdb,0xb5,0x7b,0x09,0x34,0xea,0x4e,0xfc,0xfd,0x3b,0x6e,0xd0, +0x36,0x56,0xe2,0x05,0xf3,0x4a,0xb2,0xfd,0xb7,0xfe,0xac,0x6a, +0x36,0x7d,0x4b,0xda,0x6e,0x71,0xbb,0xd4,0x28,0x13,0x95,0x96, +0xfb,0x33,0xa4,0x80,0x86,0x8e,0xbf,0x47,0x84,0x82,0x91,0x97, +0x8f,0xd1,0x6a,0x4d,0x96,0xdd,0x3e,0x7e,0x71,0xab,0xf9,0x9f, +0x5b,0xbb,0xd3,0xff,0x92,0x36,0x24,0x0a,0xd9,0x49,0x61,0xfd, +0x2e,0x12,0xe3,0xd4,0x71,0x73,0xdc,0xd9,0xb0,0x77,0x8d,0x96, +0x55,0x15,0xd5,0x28,0xe7,0x84,0x97,0x20,0x4f,0xfb,0x4a,0x3f, +0xbe,0x9a,0x93,0xd2,0x92,0x25,0x3b,0x5d,0x03,0x7f,0x77,0xae, +0x7d,0x85,0x82,0x5b,0xa8,0xfc,0x9d,0xab,0x3c,0x79,0x56,0x66, +0x4a,0x5b,0x8d,0x02,0x1d,0x06,0x9b,0x37,0xb1,0x64,0x19,0x43, +0x26,0xa3,0xbd,0x13,0x07,0x6b,0x9b,0x1d,0xe5,0x4b,0x61,0x26, +0x86,0x7a,0x10,0xc5,0x83,0xc7,0x94,0x53,0x70,0x26,0x2a,0xdc, +0x45,0xad,0x51,0x12,0x99,0x07,0x94,0xf5,0xcf,0xf0,0xea,0xe5, +0x40,0x08,0x18,0x4f,0xc6,0xa4,0xba,0xc9,0xec,0x7d,0x4c,0x93, +0x00,0xf9,0x45,0x66,0xde,0x2a,0xc6,0x53,0x3c,0xae,0x4d,0x3e, +0x58,0x4f,0x19,0xcf,0x3e,0xae,0x89,0x85,0x50,0xea,0x30,0xe5, +0xbf,0xc6,0x85,0x19,0x4d,0x1a,0x8b,0x38,0x50,0x80,0xf3,0x4d, +0x79,0xac,0xa9,0x9a,0x8e,0x57,0x9d,0x92,0xc7,0x84,0x85,0x30, +0x1e,0xe5,0xac,0xf3,0xb5,0x77,0x9c,0x90,0xd0,0xc2,0x62,0xf2, +0xff,0x71,0x97,0xaf,0xbd,0xca,0x41,0x42,0x9d,0x6f,0xbc,0xa0, +0x4a,0x3e,0x0e,0xfc,0x4d,0x68,0x13,0x79,0x84,0x2f,0xbb,0x1e, +0x3e,0xcf,0xe1,0x5a,0x85,0xa4,0xe0,0x14,0x43,0xf8,0x60,0x33, +0x6a,0x86,0x9a,0x22,0xeb,0x78,0xd6,0xab,0x14,0x8b,0xeb,0x0f, +0xad,0x60,0x82,0xb9,0x41,0xdd,0xf3,0xb5,0xdf,0x79,0xf5,0xd0, +0x2c,0x49,0x4d,0x37,0xf4,0x82,0x33,0x94,0x5a,0x60,0xef,0x24, +0xae,0x06,0x42,0x9e,0xc1,0x91,0xd5,0x04,0x11,0xf3,0xc5,0x9e, +0xc0,0xcc,0xc9,0xfc,0xa1,0x9b,0xe0,0xb7,0xf6,0xc4,0xda,0xb8, +0x77,0x77,0xe7,0xc4,0x2b,0x6c,0x09,0x56,0xfb,0x98,0xab,0xd8, +0x96,0xc8,0x63,0x35,0xec,0x18,0x3e,0x8a,0x59,0xd7,0xef,0x18, +0xee,0xb3,0x12,0x38,0xa2,0xeb,0x7a,0xd0,0x5e,0x5c,0xa0,0xdf, +0xc8,0x09,0x9c,0x5f,0x26,0x59,0xae,0xeb,0x4e,0xc8,0x35,0x32, +0x8a,0xc0,0x11,0x47,0x9c,0x17,0x3e,0xa1,0xd9,0xcb,0xad,0xe5, +0x08,0xd6,0x48,0x4e,0x30,0x4c,0x4a,0x23,0x9f,0x60,0x3e,0x46, +0x2d,0x61,0x71,0xa6,0x8d,0x3b,0x51,0x07,0x25,0x0e,0x48,0x21, +0xac,0x83,0xe5,0x01,0xd6,0x02,0xa2,0x97,0x37,0x6c,0x9b,0x37, +0x3b,0xf2,0x14,0x32,0xf3,0x3b,0xda,0x77,0xb5,0xd6,0xf3,0x16, +0x06,0x2f,0x2b,0xbc,0x0b,0x0d,0x06,0xc1,0x78,0x25,0xba,0xda, +0xd2,0x7c,0x2f,0x9a,0x7f,0xc9,0x32,0x58,0x3f,0x78,0x64,0xcb, +0xf1,0x14,0x33,0x2a,0xd1,0x5e,0xce,0x60,0xc2,0xdd,0xb2,0x4e, +0x6b,0xb3,0xc5,0xb0,0xb9,0xdb,0xef,0x6a,0x13,0x01,0x02,0x89, +0x31,0x73,0x2f,0x7d,0xa3,0x6a,0x6b,0x7c,0xd4,0x64,0x49,0xb5, +0xd4,0x2c,0x5a,0x4b,0x8a,0xff,0xf5,0x7f,0xe6,0x39,0x06,0xaf, +0xb8,0x99,0x9f,0x5e,0x15,0xe5,0xe5,0xf3,0x4f,0x1a,0x46,0x43, +0x3c,0x79,0xfb,0x32,0xd6,0x92,0xfd,0xfc,0x6a,0xa8,0x79,0x50, +0xb0,0x2e,0xc1,0x95,0xff,0xe1,0xc8,0x93,0xed,0x19,0x14,0x3c, +0x4c,0x58,0x30,0x75,0x3a,0x99,0xcc,0x74,0xe5,0x43,0xf4,0xc2, +0x11,0x53,0x14,0x73,0x65,0x91,0x28,0x2e,0x63,0x04,0x55,0x81, +0x9e,0x4e,0x02,0x2f,0x75,0x25,0x51,0x62,0xcd,0x1d,0xee,0x3f, +0xc6,0x4f,0xdc,0xc0,0xb8,0xd3,0x64,0xe4,0x05,0x21,0xfc,0xf6, +0xff,0x17,0xdc,0xf6,0xd3,0xdd,0xb9,0x11,0xbd,0x3f,0x7a,0xb1, +0xfd,0xf7,0xa8,0xdb,0x1f,0x5f,0x1c,0x1f,0xa7,0xe8,0x58,0x6d, +0xa0,0xfa,0xc6,0xc7,0x9a,0x7e,0x84,0xa1,0xb7,0x2d,0xde,0x3f, +0x89,0x97,0xb6,0x79,0x54,0x6d,0x97,0x8c,0xa9,0x60,0xbb,0x7e, +0xcb,0x5b,0x52,0xcd,0x49,0xc6,0xf9,0xc6,0x66,0x71,0x0d,0x31, +0x97,0x87,0xe3,0x59,0x77,0x89,0x75,0x01,0x27,0xb7,0x20,0x74, +0x71,0x6d,0xf9,0x78,0x07,0xc9,0xdd,0xbf,0x2f,0x4e,0x32,0xb9, +0x09,0xf8,0xc3,0x85,0x7b,0xb8,0xdb,0x68,0x0e,0x2a,0x7f,0xec, +0xa8,0x88,0xc2,0xe7,0x9a,0x08,0x84,0x7c,0x7f,0xde,0x34,0xd6, +0xbd,0x7b,0x36,0xca,0xa1,0x81,0x88,0x92,0xae,0xc9,0xbd,0x9e, +0x8c,0xe6,0x57,0x39,0xdd,0xb2,0xe7,0x4c,0x05,0xde,0x88,0xa3, +0x81,0x6e,0x54,0x0e,0x40,0x22,0x0a,0x2a,0xe1,0xbd,0xc9,0xde, +0xfc,0xff,0xb5,0xea,0x8d,0x89,0x6f,0xfe,0xff,0x2a,0x8e,0x43, +0xf3,0x31,0xc6,0xd0,0x9e,0x16,0x87,0xf9,0x05,0xf6,0x1e,0xda, +0xb9,0x30,0x8f,0xb2,0x99,0x34,0xb8,0xc0,0x84,0x5e,0xb3,0x3c, +0xeb,0x96,0x61,0xe7,0x6f,0x7b,0x5f,0xef,0x5c,0x14,0x10,0xce, +0x52,0x26,0x3e,0x7c,0x86,0x89,0x99,0x97,0xb8,0xf7,0x1d,0x26, +0xb6,0x22,0x71,0x6a,0x5b,0x83,0xfe,0x46,0x0b,0xed,0x98,0x7a, +0x78,0xa3,0xcb,0x95,0x21,0x25,0x32,0xd4,0x37,0x10,0x2a,0x2a, +0x78,0xb4,0x8b,0x15,0xfc,0x2f,0x9f,0x94,0x27,0x98,0x98,0xfa, +0xa4,0x3c,0xda,0xb9,0xd0,0xed,0xe3,0xa7,0x3d,0x45,0x40,0xe5, +0x27,0x6a,0x94,0x21,0x45,0xbd,0x48,0x27,0x6b,0x84,0xde,0x87, +0xbc,0x9b,0xcc,0xf5,0xb4,0x32,0x16,0x85,0x8a,0x43,0x44,0x97, +0x6c,0xc8,0x64,0xd2,0x33,0x0b,0x80,0xc9,0x06,0x2a,0x33,0x0f, +0xe8,0x77,0x04,0xb3,0xe6,0xd3,0xab,0xb7,0x10,0x03,0x1a,0xad, +0xfc,0x5d,0x2d,0xdc,0xdd,0x26,0x02,0x0d,0xc4,0x5a,0xc7,0x50, +0xd1,0x6c,0xac,0x5b,0x2a,0x8a,0xcf,0x24,0x8d,0x6a,0xf4,0xec, +0xd2,0xbb,0x88,0x19,0x61,0x8b,0x61,0xe2,0x30,0x10,0xa5,0xf0, +0x82,0x37,0xc2,0x2d,0xb6,0x71,0x62,0x2c,0xf2,0x33,0x8e,0xb6, +0xa3,0x09,0xab,0xab,0x89,0x91,0x58,0x2b,0x76,0x99,0x1a,0xd8, +0xfc,0x93,0xe7,0x8f,0xa2,0x7c,0xe1,0x4b,0x73,0x5a,0x2c,0x14, +0x3b,0x0c,0xd0,0x58,0x68,0x62,0xc8,0x29,0x88,0x27,0xac,0xd1, +0x4d,0xe0,0xad,0x2d,0xdd,0xbf,0x7c,0x78,0x26,0xa2,0x9c,0x53, +0x2c,0x51,0xfa,0x3a,0xca,0x2b,0xe7,0xa7,0x7a,0x12,0xc6,0xe0, +0xf7,0x9f,0xc4,0x3b,0xc4,0xeb,0x9a,0x92,0x9b,0xfc,0x1d,0xbb, +0x55,0x7a,0x09,0x0b,0x0e,0xb8,0x52,0xd7,0x12,0x97,0x73,0x18, +0xc5,0xed,0x61,0x12,0xda,0x66,0xba,0x55,0x7b,0xc4,0x3a,0x9d, +0x03,0x0b,0x0c,0x7c,0xb6,0x7d,0x49,0x6a,0xc0,0x29,0x0c,0x10, +0x3c,0x33,0xa6,0xa1,0x22,0x23,0xdb,0x73,0xa7,0x89,0xa6,0x7e, +0xf6,0xfc,0xc5,0x93,0xf7,0xaf,0x8e,0x4e,0x9e,0x3c,0x45,0xbd, +0xa0,0xed,0xd6,0xa0,0x9d,0x45,0x13,0x53,0x82,0x72,0xce,0x11, +0x2f,0x87,0xe5,0x77,0x93,0xd1,0x67,0xf0,0xd9,0xfb,0xf6,0xcd, +0xe1,0xd1,0xe2,0xed,0x7b,0xfd,0x0f,0x22,0xeb,0x27,0xf7,0x76, +0x0a,0x76,0x18,0x43,0xbb,0xd4,0xeb,0x7c,0x76,0x39,0x19,0x25, +0x5d,0xf3,0x2d,0xfd,0x31,0x6e,0xfe,0x1c,0xea,0x8e,0x1a,0xee, +0xa9,0xe1,0x43,0x35,0x7c,0xc4,0x7e,0x17,0xa1,0x99,0xd9,0xcf, +0x96,0x42,0x87,0x4c,0x34,0x36,0x70,0x0c,0xad,0x8e,0x1e,0xff, +0x28,0xa5,0xd0,0x14,0xc3,0x47,0x06,0x41,0x36,0x7c,0x48,0xc1, +0xe3,0x1f,0xa6,0xf8,0xb3,0x97,0x52,0x18,0x78,0x3b,0x4c,0x86, +0x7b,0x09,0x86,0x9f,0x02,0xfc,0x9a,0x14,0xa8,0x3c,0xd3,0xf4, +0x88,0xe2,0x76,0x83,0x22,0x4b,0x75,0xc0,0xea,0xc0,0xc0,0x74, +0x14,0x57,0xbb,0xc7,0xc0,0x36,0x67,0xcf,0xe4,0x3c,0xb4,0x21, +0xec,0x3d,0x24,0x3d,0x87,0x23,0xe5,0x26,0xed,0x11,0x09,0x9d, +0xd4,0x12,0x45,0x48,0xbb,0x1c,0x1f,0xdf,0x8f,0x12,0xcf,0xf1, +0xeb,0xfd,0x45,0x20,0x8e,0x4e,0x87,0x23,0xdd,0x44,0xa5,0x54, +0xe4,0xce,0x6f,0xe0,0xce,0xf0,0x23,0x1b,0xc5,0xbe,0xdc,0xe8, +0x53,0x0d,0x8a,0x5a,0x7c,0xc3,0x34,0xf9,0x06,0x92,0x3c,0x50, +0xa8,0xa2,0xbd,0xdd,0x5d,0xba,0x12,0x91,0xaa,0xb4,0x3b,0x8e, +0xbe,0xa2,0x7c,0x39,0x2e,0xf5,0x32,0x77,0x96,0x3f,0x1d,0x5e, +0x5d,0x65,0x80,0x4e,0x0b,0xb9,0x94,0x32,0x39,0xb7,0xa3,0x98, +0xf7,0xfa,0x3e,0x34,0x7e,0x6e,0xca,0x63,0x85,0x8d,0x3b,0x7a, +0xfd,0x41,0x4a,0x0e,0xb5,0x79,0xf8,0x03,0x4c,0xa2,0x60,0x8d, +0xd1,0xd2,0xe9,0x79,0x71,0x01,0xf6,0x22,0xd3,0xea,0xdc,0xca, +0x0c,0x1a,0x97,0xb4,0xb5,0x55,0x4d,0x6b,0x9b,0xa2,0x8b,0x45, +0xf3,0x3c,0xb5,0xf8,0x31,0x44,0xc6,0x9f,0xa8,0x04,0xcb,0x2f, +0x16,0x4e,0x1b,0xe4,0x4f,0xb2,0x5a,0x36,0x16,0x51,0xcf,0x90, +0x97,0x02,0xcd,0x94,0xb8,0x5d,0xdc,0xbb,0xab,0xc7,0xa4,0xc5, +0xae,0x75,0x88,0xbd,0x39,0x5e,0x81,0x46,0x59,0x2e,0x1b,0x25, +0x6b,0xec,0x24,0x85,0x6c,0xa3,0xad,0x65,0xa2,0xf7,0x66,0x41, +0x10,0x68,0x94,0x63,0x18,0x8d,0x5c,0xbe,0x4d,0x84,0x02,0x37, +0x4a,0x78,0x96,0x26,0x26,0x0b,0x79,0x10,0x45,0xf2,0x9b,0x5c, +0x17,0x65,0x9e,0xa1,0x90,0x20,0x2a,0x48,0xda,0x33,0x70,0x51, +0x55,0xe3,0x4a,0xcf,0xb0,0x81,0xcc,0xf2,0x46,0xac,0xd3,0x20, +0x22,0x1e,0x7c,0x8f,0x5f,0x41,0x75,0x8b,0x03,0x04,0x36,0x5e, +0x46,0xc0,0x31,0x66,0x42,0x98,0xb1,0x99,0x6d,0x6e,0xfa,0x23, +0x2c,0x69,0x9e,0x37,0x67,0xe7,0x17,0x30,0x6d,0xa0,0xd7,0xe0, +0xe4,0xc2,0x40,0x6c,0x9c,0x21,0xcd,0xf3,0x29,0xc6,0x4d,0x69, +0x6f,0xd8,0x79,0x65,0x48,0x84,0xd9,0x05,0x36,0xe5,0x43,0xb8, +0xd2,0x3f,0xc6,0x89,0x63,0xce,0x39,0xe0,0x15,0x8e,0x36,0x82, +0x51,0x1e,0xa1,0xa5,0x99,0x5e,0xeb,0x27,0xa7,0xbf,0xa0,0x79, +0x7a,0x2d,0xf1,0x15,0x30,0x2d,0xc5,0xf9,0x78,0x62,0xa9,0x6f, +0xc8,0x76,0x2d,0x17,0xc7,0x36,0xb3,0xad,0x0b,0xd6,0x81,0xac, +0xd7,0x23,0xb5,0x83,0x37,0xc3,0x30,0x17,0xee,0x3c,0xa5,0x14, +0xf9,0xce,0xad,0x75,0xac,0xc0,0xb0,0xcd,0x9d,0xf1,0xaf,0xdd, +0x68,0x72,0x36,0x34,0xd4,0xd5,0x6f,0x18,0xcc,0x40,0xc5,0xcc, +0x4b,0x16,0x0b,0xd8,0x51,0x27,0x3c,0xb4,0x6d,0x67,0x82,0xc0, +0x9e,0x4f,0xcb,0x44,0x55,0x96,0xbe,0xcc,0x87,0x14,0x8e,0xaa, +0x2a,0xcb,0x66,0x5d,0xb5,0x39,0xcd,0x35,0xaa,0x54,0xf8,0x34, +0xfc,0x55,0x57,0x00,0x6d,0xe7,0xd2,0x96,0x3c,0xf8,0x6a,0xb8, +0x48,0xe3,0xcc,0x17,0xde,0xba,0x3c,0x0e,0xc6,0x0d,0xc2,0x4e, +0xbb,0x2a,0x0d,0xa5,0x2a,0x94,0x76,0x8c,0xe3,0x76,0x65,0xda, +0x1f,0xf6,0xb6,0xb3,0x09,0xec,0x58,0xb4,0xa4,0x4d,0x3e,0x85, +0xc5,0x12,0xc9,0x7c,0xda,0x5b,0x13,0x3d,0x59,0x87,0x67,0x9e, +0xe6,0x02,0xb4,0xdc,0x18,0x48,0xb3,0xba,0x30,0x02,0xbb,0x8c, +0xd2,0xbc,0xd4,0x04,0x69,0xde,0x97,0xf9,0x0b,0xb3,0xa9,0x0c, +0xaf,0x2c,0xf8,0x35,0x76,0x9d,0xc0,0xa2,0xcb,0xd9,0xef,0x5c, +0x2c,0x28,0x01,0x2e,0xf4,0xd8,0xef,0x81,0x2d,0xa3,0x58,0xe3, +0x47,0xa3,0x02,0x1e,0x86,0xc4,0x65,0x9e,0x59,0xf9,0x95,0x31, +0xd6,0x49,0x9e,0x8d,0x92,0x61,0x03,0x2e,0x2b,0xc5,0xb8,0x56, +0x5d,0x52,0x78,0x74,0x4f,0xd8,0xc9,0x6e,0xb5,0x34,0x97,0xde, +0x92,0x54,0x92,0x46,0xdf,0x3f,0x3f,0x8a,0x96,0xaa,0x1c,0xfe, +0x96,0xbb,0x34,0x10,0x68,0x74,0xe2,0xaf,0xb0,0x69,0xf0,0x21, +0x15,0x4f,0x39,0xd8,0x16,0xb3,0xe6,0xc8,0x01,0x3c,0x7b,0xfe, +0xea,0xf9,0xd1,0x73,0x5d,0x30,0xa2,0x6b,0x04,0x51,0x35,0x6b, +0xa9,0x60,0x68,0xda,0x3d,0x1a,0x2a,0xa0,0x78,0xb6,0xda,0x44, +0x7e,0xe7,0xef,0xb7,0xc9,0xf4,0xaa,0x60,0x72,0xda,0x34,0x78, +0x51,0x6e,0x18,0x64,0x8e,0x7f,0x98,0x24,0x0e,0x83,0xef,0x7a, +0x42,0xf3,0x10,0xdd,0x9e,0x20,0x30,0x71,0xd3,0xf3,0x71,0x23, +0xa6,0x41,0x6e,0x62,0xe5,0x74,0x94,0x24,0xdd,0xc0,0xcc,0x6e, +0xe5,0x3a,0x91,0x0a,0x0f,0x46,0xb8,0x58,0xf0,0xb0,0xa1,0x4e, +0x78,0x6f,0x4c,0xa5,0x61,0xcb,0x45,0x5b,0x2b,0x50,0x84,0x88, +0xcb,0x30,0x73,0xb8,0x51,0x65,0x60,0xd1,0xfe,0xec,0xdc,0xee, +0x63,0x94,0xdb,0x9e,0x60,0xba,0xdc,0xad,0xd8,0xd5,0x53,0x27, +0xb2,0xff,0xee,0x9d,0xe3,0x1f,0x77,0xc4,0xde,0x19,0x09,0x49, +0x6e,0x37,0x45,0x44,0x8e,0xe8,0xe7,0x63,0x08,0x29,0x1e,0x25, +0x24,0x98,0x12,0xc4,0xd6,0x16,0xfe,0x7a,0xa1,0x3b,0xa2,0xf8, +0xe7,0x45,0x5f,0x03,0x1f,0x1f,0x0f,0x12,0xbd,0x05,0x43,0x80, +0x56,0x14,0x1f,0x1f,0xff,0xb8,0xb8,0x97,0x98,0xe2,0xb0,0xae, +0xe9,0x99,0x66,0xa9,0x22,0x31,0x0a,0xae,0xb7,0xc0,0x3a,0x0b, +0x1f,0x36,0x47,0x35,0x0c,0xef,0xf2,0xf4,0xb6,0x1f,0xb6,0x83, +0x7a,0x8b,0x68,0xe6,0x9f,0x5b,0x33,0xe5,0xf5,0x5c,0xf1,0x9d, +0xee,0x63,0x4e,0x6c,0x5b,0x20,0xeb,0xe6,0xa2,0xe1,0xad,0x0a, +0x0b,0xd3,0xa3,0xfc,0xbe,0x49,0x18,0xa4,0x74,0x99,0x88,0xe7, +0x88,0x4b,0x17,0x4b,0x8d,0xb1,0x1e,0x87,0xed,0x2e,0xd9,0x93, +0x50,0xdc,0xa3,0x5e,0xec,0x3a,0x2e,0xab,0xdd,0xc5,0x57,0xbe, +0x55,0xb6,0xa4,0x6e,0x3e,0x53,0x9b,0x6b,0x41,0x15,0x5d,0x40, +0x14,0x73,0x8b,0xb7,0x15,0xdd,0xeb,0x80,0x03,0xd4,0x55,0x88, +0xe2,0x9d,0x5e,0xb2,0x02,0x99,0x6d,0x2a,0x3c,0x30,0x57,0x57, +0x7a,0x69,0x29,0xc6,0x17,0x87,0x57,0xc3,0xf2,0x32,0xd7,0x5c, +0x6c,0x58,0x58,0x6d,0x43,0xb4,0xa3,0xb7,0x76,0x90,0x20,0x76, +0x76,0xf0,0x9a,0xfa,0x65,0x5a,0x90,0xb6,0xac,0xef,0xca,0x9d, +0xd6,0xbd,0x1d,0x0c,0xc4,0x5e,0x93,0x75,0xdc,0x8e,0x7b,0xd9, +0xf1,0xc7,0x56,0x7c,0x6f,0x71,0xdc,0x4b,0x12,0x0d,0xd7,0xd6, +0x80,0x34,0x3b,0xda,0x75,0xf0,0xc7,0xc7,0x6d,0x0d,0xb4,0x03, +0x50,0x66,0x18,0xdc,0x34,0xef,0x6c,0xfd,0x11,0x82,0x22,0xde, +0x62,0xc1,0x93,0x8f,0xa5,0xb0,0xcc,0x7b,0xc3,0xf1,0xe5,0xa5, +0x88,0x6d,0x2f,0x6a,0x03,0x55,0xc0,0x76,0x97,0x03,0xa7,0x7d, +0xa3,0x9f,0xb6,0xd5,0xec,0x35,0xa8,0xa6,0x02,0x85,0x94,0xdd, +0xb8,0x72,0xb0,0xc1,0xf8,0xc6,0x18,0xb5,0x6b,0xf9,0xd1,0xe9, +0x5a,0x88,0xb3,0xc3,0x45,0x72,0x7e,0x14,0x0a,0xc2,0xa5,0xba, +0xb5,0x2c,0x27,0xa5,0xe2,0x60,0x37,0x49,0x18,0xdc,0xce,0xf8, +0x1e,0x72,0x27,0x3d,0x03,0x7e,0x2b,0xc8,0xda,0xde,0xe6,0x00, +0x27,0xe3,0xb1,0x03,0x8f,0x70,0x74,0x60,0xd4,0x10,0x7a,0x4d, +0x1f,0x97,0x05,0xaa,0x3d,0x21,0x15,0xef,0xec,0xbe,0x74,0x89, +0x07,0x38,0x82,0x34,0x91,0xd9,0xed,0x64,0x5a,0x5c,0x14,0xb8, +0xc4,0xcc,0x2e,0x53,0x44,0x33,0xcd,0x2f,0xf2,0x4f,0x37,0xf8, +0xbc,0x24,0xe3,0x76,0x0d,0xd8,0xfe,0xc0,0x7e,0xc8,0xf8,0xcb, +0x20,0x37,0x83,0x3f,0xae,0xa3,0xe3,0x7e,0x9c,0xb4,0x07,0x09, +0xb0,0x80,0xe3,0x63,0x18,0xed,0x22,0xeb,0x78,0x27,0xe9,0xa5, +0x10,0xc1,0x37,0x89,0xfb,0xc7,0xbd,0xc5,0xf1,0x83,0x41,0xd2, +0xd3,0x80,0x62,0xf6,0x97,0x30,0x26,0xd1,0x00,0xc2,0x5c,0x6e, +0x70,0xc6,0x11,0x78,0x3f,0xcc,0x99,0x83,0xb1,0x85,0x2b,0x99, +0x8b,0x80,0xe5,0x4c,0x16,0x3d,0xa8,0xcf,0xed,0x3a,0x7f,0x7e, +0x7c,0x02,0x45,0xe1,0xbf,0x5d,0x35,0xa0,0xf8,0xdd,0x34,0x8f, +0x70,0xa3,0x03,0xc8,0xc8,0xf0,0x2f,0x1a,0x77,0x47,0x51,0x2b, +0x36,0xd9,0x60,0xfc,0x8b,0x39,0x89,0x9e,0x9a,0xbd,0x54,0xe6, +0x60,0x72,0xaa,0xe7,0x4b,0x2b,0x06,0x82,0xf4,0xc6,0x05,0x7c, +0x4f,0xf6,0x22,0x8d,0x23,0xee,0xff,0xbc,0x03,0x01,0x8a,0x75, +0x5e,0x94,0x88,0x32,0x1c,0xe6,0x43,0xb6,0xdf,0xf1,0xce,0x3d, +0x68,0x19,0xd7,0x82,0xd0,0x45,0x6d,0xea,0x8f,0xcc,0xe3,0xf2, +0xc0,0xb1,0x67,0x97,0x9a,0xa9,0x40,0xa0,0x15,0xdb,0xad,0xbd, +0xa8,0x88,0x80,0x08,0x28,0xe6,0x94,0x25,0xb8,0x98,0xa0,0x32, +0x0c,0x5c,0xa6,0x4a,0x59,0x0b,0x3a,0x1b,0x20,0x70,0x3f,0x41, +0xb0,0x7d,0x48,0x1d,0x58,0x0d,0xca,0x34,0x07,0xaf,0xd1,0x6f, +0xc6,0x87,0xf9,0x70,0x7a,0x76,0x49,0xc2,0x00,0xc0,0xe1,0xca, +0x0a,0xeb,0x8a,0x3f,0x02,0x09,0x5b,0xc2,0xeb,0x2e,0x49,0x5e, +0x14,0xbc,0x01,0x86,0x57,0x86,0x7c,0x09,0xf2,0xb0,0x16,0xe7, +0xd0,0xa6,0x87,0xaf,0xac,0x69,0xda,0x55,0x5e,0x66,0x92,0xd2, +0x97,0xee,0x44,0x5d,0xa6,0x50,0xa2,0x94,0x94,0x52,0xea,0xd1, +0x84,0x47,0xae,0xa0,0x4d,0x96,0x30,0x34,0x1a,0x99,0x1e,0x1a, +0x86,0x75,0x85,0x13,0x2d,0x27,0x4f,0x3f,0x16,0x32,0x6c,0x15, +0x8b,0x9b,0xb7,0x02,0x18,0x5b,0x11,0x0d,0x8b,0xec,0x36,0x50, +0xe0,0xb8,0xa7,0xc5,0x10,0x38,0xb3,0x9b,0x4e,0x26,0xe4,0xa3, +0x1b,0xf4,0xc7,0x70,0xca,0x8f,0x27,0xdc,0x0a,0x32,0x60,0x36, +0xd3,0xa6,0x97,0x54,0xcb,0x11,0x07,0xc7,0x9f,0x60,0xa4,0xdf, +0xc6,0xd0,0x01,0x41,0x50,0x00,0x5b,0x81,0xb2,0xe8,0x95,0x44, +0xae,0xb7,0x02,0xca,0x22,0x56,0xf5,0x31,0x04,0x30,0x46,0x80, +0x65,0x27,0xa4,0x70,0x42,0x6e,0x83,0xfc,0x41,0x33,0x1b,0x8d, +0x93,0x07,0x08,0xf4,0x25,0xf1,0x02,0x94,0x95,0xe0,0x51,0xe8, +0xaa,0x40,0xd3,0x44,0x19,0x34,0x58,0x13,0x73,0xb2,0x0c,0x73, +0x10,0x8d,0xf7,0x32,0x99,0x4d,0x61,0xc6,0xd8,0xdd,0xe9,0xe6, +0xb5,0x07,0xed,0xfc,0x99,0xa0,0x0d,0x50,0x76,0x2d,0xcc,0x7e, +0xf6,0x8b,0x6e,0xab,0x55,0x58,0x62,0xf8,0xe2,0x06,0x78,0x44, +0x02,0x89,0x21,0x02,0xde,0x3c,0xbe,0x80,0xa5,0x4f,0x73,0xd4, +0xc9,0xf9,0xc6,0x75,0xbf,0x18,0xf4,0x46,0x79,0x45,0xdd,0x0d, +0xe9,0x49,0x0a,0x7f,0xbb,0xa8,0x50,0xf8,0x0d,0x0e,0xb6,0x62, +0xb7,0x72,0xb4,0xc7,0xe6,0xd6,0x88,0x1d,0x25,0x94,0xe9,0x98, +0xf2,0xfc,0x46,0x6f,0x1b,0x73,0xd2,0x40,0x1a,0x77,0xf7,0x9f, +0x66,0xe4,0x06,0x99,0x53,0xd1,0xe0,0x32,0xa3,0x2e,0x69,0xb3, +0xd7,0x76,0x74,0x3d,0x47,0x16,0xb2,0xe4,0x5b,0xac,0x7d,0x8f, +0x00,0x34,0xf7,0x82,0x44,0xf3,0xba,0xb5,0xc5,0xb7,0xd5,0x10, +0x06,0x46,0x32,0x77,0x2a,0x02,0xb9,0x77,0x2d,0xc6,0x6c,0x22, +0x88,0x3f,0x4f,0x75,0xaa,0x6e,0xc6,0xb3,0xfc,0x1d,0xa6,0x92, +0x11,0xb2,0x59,0x39,0x19,0x23,0x62,0xc3,0xfd,0xaf,0xc8,0xf4, +0x06,0x50,0xa2,0xdc,0x18,0x6b,0x43,0x94,0xec,0xe1,0x48,0xaf, +0x22,0xe0,0xdf,0x1d,0x81,0xde,0x63,0x0b,0x44,0x48,0x11,0xec, +0xa5,0x01,0xef,0x62,0x81,0x6f,0xba,0x31,0x45,0xf5,0x18,0x7e, +0x74,0x15,0x26,0x72,0x3a,0x7f,0x08,0x16,0x90,0x11,0xba,0xb6, +0x22,0x9c,0xca,0x6f,0x3a,0x24,0x5c,0x51,0xfb,0xf1,0x27,0x9b, +0x09,0x8f,0xe1,0x01,0x0f,0xb1,0xf3,0xe3,0x20,0x2b,0xe9,0xd9, +0xd9,0x81,0xcd,0x26,0x0f,0x44,0x43,0x58,0x25,0x9a,0x26,0x49, +0xda,0x25,0x36,0x65,0x2c,0x13,0x2d,0xaf,0x4e,0x52,0x87,0x56, +0x4b,0x2a,0x21,0xa6,0x4a,0xb7,0x05,0x44,0x24,0x22,0x81,0xab, +0x49,0x12,0x81,0x1d,0xf7,0xf1,0xc4,0x66,0xc0,0x9d,0x42,0xa0, +0x3c,0x4a,0x8c,0xf7,0x47,0xdf,0x73,0x83,0x92,0x6e,0x18,0xd8, +0xda,0xcf,0xed,0x2c,0x99,0x42,0xdc,0x97,0xdb,0xed,0xad,0x11, +0xc6,0x08,0xba,0x56,0x18,0x73,0x96,0x4c,0x83,0xcc,0x36,0x32, +0x09,0x56,0x3d,0xcb,0x5e,0xf0,0x26,0x26,0x25,0xa6,0x2e,0xb1, +0x18,0xff,0x36,0xf9,0xe0,0xec,0x98,0x94,0x13,0xc0,0xed,0x31, +0x0e,0x52,0x65,0xde,0x92,0x9e,0xd8,0xe7,0xdb,0x44,0x88,0x0c, +0x1b,0x9e,0xfa,0x78,0x9d,0x02,0x37,0x7b,0x43,0xc4,0xe0,0xc5, +0xc2,0xc3,0xfd,0x9e,0x76,0x32,0x35,0x15,0x70,0x8e,0x2c,0x29, +0x9e,0xfd,0x9a,0x64,0x13,0x67,0x41,0x00,0x2f,0x19,0x9c,0x43, +0xe2,0xae,0xf9,0x6c,0xae,0x12,0x51,0x5f,0x61,0xac,0x86,0xa3, +0xfa,0xda,0x6d,0x7d,0x99,0x0b,0xd9,0x21,0xb3,0x9b,0x42,0x77, +0x34,0x29,0x1b,0xc3,0x63,0x18,0x50,0x7d,0x69,0x91,0xb3,0x8e, +0xc6,0xc4,0x5a,0xb1,0xb5,0xef,0x55,0xda,0x1f,0xc7,0xe7,0xf0, +0xea,0x2a,0x36,0xd7,0xb3,0x2a,0x75,0x72,0xc6,0x2d,0x72,0xc5, +0x80,0x09,0xf2,0xa7,0xbb,0x16,0xc0,0xa1,0xca,0x63,0x10,0x59, +0x92,0xe4,0x51,0x1e,0x4b,0xba,0x83,0x27,0x31,0x27,0x21,0x55, +0x8e,0xe4,0x25,0xd2,0x0f,0x3c,0xeb,0x94,0xea,0x29,0x5b,0x03, +0x3d,0xea,0xca,0x04,0x72,0x56,0x19,0x81,0x13,0x25,0x21,0x99, +0xbb,0x75,0xe0,0x96,0xc3,0x94,0xc3,0xc7,0x90,0x0d,0x7b,0x30, +0xfb,0x48,0xac,0x11,0xfa,0x3f,0x5a,0x71,0x37,0x39,0xde,0x26, +0x2f,0x4c,0x59,0xcd,0xf2,0x5c,0x61,0x2b,0x2c,0xdb,0xe8,0x42, +0x64,0x22,0x6f,0x36,0x27,0x24,0xb3,0xf1,0xe1,0x4f,0xea,0xb8, +0x42,0x95,0x0d,0x59,0x99,0xc6,0x71,0x2f,0x3e,0x9e,0x31,0x0e, +0xb1,0xed,0x92,0x45,0x95,0x99,0xe4,0xc5,0x82,0xc5,0x33,0x58, +0xc5,0x31,0xee,0xb8,0xa8,0x9a,0x53,0x2d,0x05,0xb7,0x4b,0x89, +0xff,0x76,0x29,0x2f,0x20,0x4b,0x0e,0x4d,0x2b,0xba,0xa1,0xe9, +0x56,0xa8,0xf2,0xdc,0x4e,0xc4,0x34,0x23,0x03,0xa3,0x58,0x6d, +0xe2,0x9d,0xa5,0x72,0xdb,0x5c,0xd2,0x86,0x5e,0x15,0xc8,0x40, +0xd1,0x79,0x65,0xc2,0x0a,0x3e,0x76,0xea,0x75,0x41,0x97,0x22, +0xf0,0xa2,0x23,0xd9,0x0d,0x61,0x0a,0x36,0x77,0xc6,0x2f,0xc6, +0x17,0x38,0x6d,0x66,0xda,0x0f,0x92,0x1d,0x34,0x13,0xce,0x24, +0x6c,0xbf,0x03,0xd4,0x39,0xcc,0x62,0x53,0x8a,0x9a,0xe0,0xb0, +0xca,0x03,0x6b,0xcc,0x0f,0xc2,0x3b,0xbb,0xe6,0x10,0x85,0x28, +0x52,0x2f,0x16,0xa3,0x48,0xa3,0x11,0x9d,0x46,0x55,0x57,0x59, +0xe8,0x97,0x5b,0x6a,0xf2,0x94,0xc7,0x15,0x89,0x06,0x32,0x62, +0x8a,0x57,0x6c,0xd7,0x9b,0x52,0xe0,0x56,0xed,0x49,0xf9,0x79, +0x7c,0x16,0x0b,0x09,0x47,0xcf,0x6f,0xd3,0xd8,0x12,0x16,0x4d, +0x74,0xcc,0x00,0x0a,0xa6,0x9e,0x2c,0xcd,0x74,0x2d,0x07,0x95, +0x3d,0x2d,0x7e,0xa1,0xd8,0xd9,0x3a,0x09,0xda,0xb3,0xe4,0x43, +0x1b,0xb5,0xa5,0x38,0xbe,0x1f,0x5f,0x80,0x99,0xac,0x51,0xa1, +0x12,0xfa,0xc6,0x88,0x58,0xce,0xfb,0x8d,0x5a,0xe5,0x45,0x30, +0x0c,0x8c,0xe5,0x87,0x5b,0xf8,0xf2,0x28,0x59,0x4d,0x71,0x17, +0x02,0xff,0x53,0x64,0xa4,0x66,0xbf,0x4c,0xc4,0xc7,0x81,0x4f, +0xfc,0xbf,0x1e,0x8e,0x2a,0x94,0x78,0x99,0x9d,0x31,0xcf,0xae, +0xe1,0x9e,0xe2,0x9d,0x39,0xbb,0x5b,0xc4,0xe8,0x5d,0x8b,0xbf, +0xc1,0xba,0x02,0xbb,0x00,0xbb,0xb4,0xac,0x88,0x1d,0xc5,0x4e, +0x5c,0xd7,0x89,0x1a,0x45,0x8d,0xa7,0xc8,0x71,0x2a,0xde,0xd5, +0x10,0x6b,0x57,0xa3,0xa7,0x57,0x1b,0xee,0x89,0xbc,0xdc,0xeb, +0xea,0x32,0x17,0xf7,0x09,0x51,0x89,0x70,0x4f,0xa6,0xbd,0x42, +0x11,0x1f,0x6c,0xb0,0x65,0x67,0x98,0xd6,0x28,0xd7,0x0f,0x58, +0xd5,0x96,0xee,0xc8,0x4c,0x6b,0x95,0x12,0x1b,0xe2,0xa0,0xd8, +0xf4,0x16,0x34,0x13,0x43,0x3d,0xae,0x62,0x72,0xb6,0xe6,0x95, +0xac,0x27,0x65,0x30,0xb2,0xfa,0x75,0x30,0x70,0xd0,0x6a,0xc9, +0xe2,0xa6,0xe5,0x00,0xa9,0x32,0x4e,0xa7,0xb4,0xaa,0xac,0x16, +0x70,0x0d,0xf8,0x05,0x65,0x97,0xd0,0x15,0x41,0x78,0xad,0xda, +0x38,0x58,0x18,0x12,0x3d,0x08,0x82,0x15,0x00,0xfe,0xe6,0xc5, +0xc2,0x52,0x41,0x78,0xad,0x25,0x5f,0x35,0xaf,0x0c,0xa5,0xe5, +0x5d,0xb1,0xe6,0x1a,0x63,0xdc,0x75,0xa5,0x03,0xa0,0x3a,0x31, +0x85,0x66,0x56,0xa2,0xcc,0x0c,0x03,0xe3,0x45,0xea,0xd9,0xf0, +0x00,0x84,0x36,0x85,0xee,0xa0,0x04,0x5f,0x1b,0x8e,0x4f,0x9a, +0x0e,0x5b,0xac,0xa8,0x97,0x55,0x14,0xe3,0x77,0x1c,0xb0,0x18, +0xf1,0x5f,0xe4,0x51,0x42,0x37,0x30,0x5b,0x54,0xde,0xa9,0x8f, +0x79,0x99,0x4d,0xfe,0xab,0x9c,0x8c,0x39,0xac,0x26,0x89,0x31, +0x68,0x9a,0x97,0xd5,0x58,0xea,0x41,0xae,0x33,0xd3,0xbb,0x31, +0x2b,0x04,0xb7,0x5f,0xa4,0x7c,0x9d,0x28,0xc4,0x24,0x15,0x08, +0x2b,0xf0,0x46,0xdb,0x52,0xb7,0xea,0xa0,0xeb,0x7a,0x51,0x76, +0xe4,0x82,0xe0,0xd2,0x2a,0x03,0x61,0x7b,0xc4,0x6a,0x93,0xf8, +0xaf,0x6d,0xde,0xee,0x90,0xca,0x67,0x3e,0xbb,0x2b,0xc8,0xe3, +0x99,0x18,0xe0,0xce,0xcf,0xed,0xe0,0x4b,0x95,0xd1,0x8d,0x76, +0x92,0xd5,0xb6,0x7c,0x3a,0x99,0x7c,0x28,0xc0,0x17,0x7f,0xd5, +0xe8,0xf1,0xcc,0x65,0xf9,0xfa,0x2a,0x2d,0xf9,0x7e,0x2c,0xfd, +0x00,0xb9,0x42,0xdd,0xc4,0xb9,0x52,0x11,0x0d,0xb2,0x0c,0x73, +0x6e,0xf0,0xfd,0x42,0xfb,0x47,0x86,0x63,0x02,0x78,0x78,0xa3, +0x51,0x08,0xc6,0x6a,0x35,0xee,0xf3,0x39,0x3f,0x91,0x86,0xe4, +0x4c,0x19,0x9d,0xd3,0x6a,0x71,0xd6,0xd4,0xd9,0xe6,0x0c,0x44, +0x20,0x1a,0xc1,0x43,0xca,0x30,0x7c,0xf7,0xc8,0x43,0xa5,0xc2, +0xa1,0xeb,0x85,0x71,0xdf,0x24,0x25,0xcc,0x53,0x59,0x02,0x96, +0xcd,0xda,0xca,0xd9,0x62,0x80,0xbf,0x0a,0x8e,0xc4,0xd2,0xea, +0x49,0x53,0x05,0x5d,0xd2,0xf3,0xe8,0xa9,0xd6,0x97,0xb2,0xf4, +0xe7,0x81,0xc1,0x9a,0xc2,0x15,0x25,0xb7,0xb6,0xc6,0x4e,0xd0, +0xc2,0x59,0x85,0xd0,0xfa,0x76,0xf1,0x50,0xeb,0x6f,0xf6,0xb1, +0xd8,0xcf,0x96,0xfd,0x51,0x07,0x92,0xf4,0xea,0x48,0x4d,0xfd, +0xef,0xab,0x2b,0x28,0xdb,0x8c,0x38,0x1d,0x3a,0x2c,0xf7,0x47, +0x0b,0xa3,0x01,0xf3,0x27,0xd1,0x46,0x68,0x0d,0x35,0x1f,0x3f, +0x07,0x75,0x9f,0xfe,0xfe,0xba,0x23,0x62,0x4b,0xb4,0x60,0x53, +0x36,0xcd,0x8a,0xb1,0xa6,0xa5,0x86,0xa3,0xd1,0x5b,0x3d,0x1d, +0x5f,0x8c,0xc3,0x5b,0x99,0xcc,0xdd,0x57,0x34,0x6d,0x95,0xe8, +0xcd,0xcc,0x2f,0x65,0xe2,0x66,0xf8,0xbd,0xe4,0xc3,0xd0,0x86, +0x37,0x48,0x93,0xd3,0xa2,0x16,0xc0,0x74,0xa5,0x69,0x0c,0x7f, +0xe3,0x4a,0xd7,0x4c,0xc1,0xc6,0x25,0x4e,0x5c,0x73,0xf9,0x82, +0x3e,0x5d,0x0f,0x80,0x89,0x9b,0x18,0x6c,0xcb,0x1a,0xe6,0x70, +0xa8,0xdf,0xc8,0x28,0xda,0x32,0x0b,0xe9,0x87,0x93,0xa8,0x30, +0xd2,0x34,0x98,0x1b,0xd8,0x5c,0xbe,0x5b,0x67,0x02,0xc4,0x19, +0x60,0xba,0x82,0x28,0x0d,0x5b,0x7a,0x76,0x85,0x9a,0x4e,0xae, +0x61,0x89,0x30,0x9a,0x24,0x32,0x37,0x55,0x37,0x73,0x1f,0xab, +0x09,0xef,0xe4,0x61,0xcc,0xfc,0x55,0xc6,0x7a,0xdf,0x63,0xcb, +0x05,0x9f,0x2a,0x1e,0xb7,0x1e,0x02,0xb8,0x30,0xf0,0x97,0x9d, +0x09,0x5e,0x0f,0x3f,0xe4,0xe0,0xcd,0xa8,0xb4,0xf1,0xf7,0xd5, +0xe4,0xf4,0x17,0x18,0xbc,0xe4,0x2b,0xad,0x84,0xb8,0xb8,0xb4, +0x43,0x55,0x11,0x5f,0xc2,0xcf,0x76,0xbb,0xc5,0x63,0xf2,0x6b, +0x66,0xee,0x9d,0x81,0x0f,0x8a,0xd3,0x5f,0xfa,0x98,0xd8,0x2f, +0x06,0x70,0xee,0x6e,0x5a,0x4e,0xa7,0x3b,0xd1,0x1c,0xe4,0xdb, +0xb7,0x78,0xcd,0x2a,0x86,0x47,0xa5,0xc5,0x8b,0xd1,0x95,0xcf, +0xa0,0x21,0x17,0x35,0xac,0x70,0xc1,0x7c,0x36,0xbc,0x50,0xfa, +0x1f,0xba,0xe9,0x82,0x1d,0x90,0x66,0xa4,0xc3,0xe9,0x67,0x0a, +0xeb,0x4a,0xc9,0xb6,0x39,0x5d,0xcc,0x43,0xce,0xe1,0xd8,0x0a, +0xcf,0x8d,0x4f,0x70,0x4e,0x1e,0xe0,0x1d,0xd8,0x6e,0x39,0x1b, +0x9e,0x7d,0x68,0xc3,0xf0,0x05,0xbf,0x75,0xc5,0x58,0xcb,0x6d, +0xb9,0x05,0x95,0x99,0x83,0x6e,0x82,0x34,0x3d,0x1f,0x8f,0x80, +0x22,0xf0,0x1e,0x23,0x72,0x93,0xae,0x39,0x24,0xa3,0xfc,0x4a, +0x6d,0x5b,0x5b,0x12,0x1c,0xce,0xba,0x2f,0xe8,0xe6,0x65,0x80, +0xd4,0x12,0x8d,0x5f,0x98,0xfd,0x36,0x29,0x46,0x15,0x5c,0x8b, +0xc5,0xe6,0x26,0x66,0x13,0xd0,0x62,0x41,0xa8,0x71,0x4d,0x33, +0xe5,0x51,0x6a,0x41,0xef,0xa4,0x64,0xdd,0x5d,0xce,0xac,0xb2, +0xf6,0xc9,0xd1,0xd1,0x3b,0x76,0x37,0x10,0x9e,0xd0,0xe3,0x3a, +0x31,0x9a,0xcc,0x4f,0xaf,0xf2,0xff,0x33,0x9f,0xcc,0xd0,0x0e, +0x00,0xbc,0x20,0xe1,0x9d,0x30,0x99,0x32,0x1f,0xff,0xea,0xde, +0xe4,0xac,0xa9,0x14,0xd6,0xd4,0x85,0xa5,0xc1,0xe0,0xf2,0x57, +0xf9,0xaa,0x45,0x4e,0x24,0x95,0x59,0x30,0x1d,0x80,0x3c,0x1f, +0xcf,0x8a,0x19,0x30,0x2e,0xab,0x8e,0xe5,0x51,0x42,0x81,0x29, +0xb6,0xb6,0xbc,0x57,0xf3,0xe1,0xec,0x4f,0x9a,0xc6,0x47,0xa0, +0xde,0xe2,0x56,0x16,0x83,0xc9,0x8c,0xf5,0x9b,0x49,0x09,0xa3, +0x79,0x9d,0xd1,0x84,0xc3,0x06,0xe0,0xb9,0x3f,0xf9,0xd0,0xaf, +0xab,0x93,0xf6,0xb3,0x5d,0xee,0xe6,0xbe,0x7e,0xd3,0x0b,0x15, +0x97,0x81,0xbc,0xed,0x6d,0x3c,0x3a,0x22,0x28,0xba,0xe9,0x5a, +0x84,0x28,0x8a,0xfd,0x4c,0xe7,0x77,0x0b,0x0d,0x6b,0xbe,0x4d, +0xcb,0xc5,0xee,0x43,0x41,0x05,0x46,0xe8,0xd1,0xaf,0xbd,0x28, +0x0c,0xe5,0x68,0x61,0x32,0xd1,0x4d,0xf4,0xf7,0x92,0x96,0x8b, +0x86,0x29,0xdc,0x95,0xc7,0xd3,0x1d,0x98,0x6e,0x38,0x7d,0xdd, +0x70,0xac,0xaa,0x2e,0x36,0xa8,0x16,0x9f,0xbc,0xc1,0xb2,0x8b, +0x85,0x71,0xd6,0x61,0x05,0xc0,0xa1,0xfd,0xf9,0x83,0x51,0x8f, +0xf4,0x1c,0xa8,0x9d,0x40,0x09,0x14,0x47,0x02,0xea,0x6d,0x49, +0xda,0x0f,0x92,0xc7,0xc7,0xe0,0x7f,0x18,0xec,0x2f,0xca,0x07, +0x51,0x4b,0x96,0x6e,0x45,0xfd,0x9f,0xf7,0x07,0x0f,0xf6,0x35, +0xfb,0x29,0xa4,0x8a,0x6c,0x78,0xc5,0x1e,0x9c,0xed,0x81,0x27, +0x68,0x49,0x67,0x74,0x76,0x40,0x95,0x3c,0x7d,0x73,0x70,0xf0, +0xfc,0xf5,0x91,0x19,0xf2,0x91,0x77,0xd0,0xff,0xf4,0xd9,0x93, +0xa3,0x27,0x5e,0x96,0x1d,0x66,0xf8,0x8d,0xae,0xf5,0xf1,0x35, +0x0e,0xc6,0x26,0x56,0x0d,0xae,0x3d,0xc9,0xc4,0xb6,0x99,0x3d, +0x98,0xbb,0xe4,0xa0,0xc4,0xc3,0x26,0xc0,0x8e,0x7a,0x73,0x1e, +0x47,0x8f,0x37,0xb7,0xb7,0xa3,0xa4,0xc7,0x8e,0xe3,0xfc,0xbc, +0xed,0xed,0x7d,0x4d,0x10,0xbe,0xe2,0xe0,0x8a,0x2d,0x31,0x93, +0x6b,0x52,0x91,0x04,0x09,0xc8,0x4e,0xf9,0x5c,0x1a,0x84,0xc5, +0x47,0x1c,0x21,0x0c,0x6c,0x8c,0x4d,0xe3,0xbb,0x5c,0x72,0x45, +0xf0,0x30,0x51,0xdc,0x9d,0x70,0x65,0xe2,0xbb,0xe7,0xdf,0xbf, +0x7c,0xfd,0xfd,0xc9,0xf3,0xd7,0xcf,0x4e,0x8e,0x9e,0x7c,0xff, +0x9c,0x9b,0x86,0xac,0xab,0xa0,0xb8,0xa6,0x94,0x34,0xb5,0x88, +0x8b,0x54,0x8b,0x0c,0x6c,0x7c,0x98,0x28,0xa3,0x07,0xae,0xab, +0x93,0x7c,0x67,0xec,0x0e,0xec,0x15,0x5c,0x9b,0x60,0xba,0xc4, +0xc7,0x26,0x9b,0xb5,0x4a,0xa8,0x00,0x14,0x24,0x5a,0x6d,0xb2, +0xa0,0xf1,0xf0,0xe8,0xc9,0xbb,0xa3,0xbf,0x90,0xca,0x10,0x9f, +0xf2,0x56,0x2c,0x49,0x29,0x3d,0xea,0x69,0x53,0xd3,0xc1,0x8f, +0xf5,0x62,0xfa,0x1f,0x74,0x26,0xa6,0x87,0xed,0x2e,0xb9,0x88, +0xe8,0x01,0x4c,0x1a,0x50,0xb4,0xcb,0x3d,0x89,0x93,0xd0,0x42, +0x46,0x51,0x08,0xc7,0x11,0xe1,0xbe,0x7c,0x10,0xc3,0x2d,0x0a, +0xc4,0x8d,0x22,0xbe,0x31,0xc2,0x2e,0xf5,0x8e,0x6f,0x56,0xfc, +0x2e,0x8d,0xb0,0xcd,0x25,0xb3,0xa3,0x4b,0x2d,0x48,0x70,0xf6, +0x74,0xe3,0xe3,0xb0,0xdc,0xd0,0x5c,0xf7,0xf4,0x0a,0x8d,0xad, +0x11,0x66,0x63,0xa6,0x21,0xce,0x27,0x10,0x2f,0x4d,0x53,0xb6, +0x81,0xcb,0x30,0x38,0xa8,0xc1,0xcf,0x03,0x23,0xec,0x08,0x47, +0x25,0x09,0xa6,0xd8,0x30,0x4b,0x39,0x87,0x12,0x79,0x65,0xa7, +0x66,0x45,0xb0,0xe1,0x32,0x8b,0xd1,0x28,0x1f,0xbf,0x9d,0xe6, +0xba,0x59,0xc7,0xf9,0xf4,0x87,0xa3,0x83,0x57,0x6c,0x85,0x6e, +0x0d,0x55,0x1e,0x83,0x81,0xca,0xd6,0xd5,0xac,0x0b,0xd3,0xdb, +0x07,0x3f,0xc2,0x53,0x60,0x97,0x08,0x8d,0xc1,0x0a,0x19,0x58, +0x98,0xc2,0x0b,0x6b,0x0d,0x34,0x04,0xf5,0x6d,0x61,0x7d,0xc3, +0xeb,0x9b,0xae,0x60,0x35,0xaf,0xdf,0xbc,0x3e,0x79,0xf2,0xea, +0xed,0x0f,0x4f,0x5e,0xbf,0x3f,0x78,0xfe,0xee,0xe5,0xd3,0xca, +0x1a,0xec,0xa1,0x8c,0xb6,0xfe,0x93,0x63,0x8c,0x60,0xa7,0x3d, +0xd5,0x75,0xa3,0x59,0x5c,0x2b,0xea,0x46,0xd2,0x06,0x47,0x7c, +0x9a,0x4b,0xdc,0xc7,0xc4,0x0b,0x48,0xf4,0x85,0xae,0x43,0xee, +0xaf,0x1f,0xa7,0x05,0x68,0x12,0x4f,0xe7,0xe7,0xbc,0x04,0x5e, +0x8c,0xb5,0xd8,0x8c,0x7e,0x10,0xe6,0x33,0xbb,0xfa,0x81,0xa1, +0x30,0xc0,0x28,0xfd,0x0f,0xc5,0x0b,0x1b,0xef,0x17,0xd7,0x5c, +0x27,0xa4,0xc2,0x92,0x2a,0x17,0xde,0x5b,0x9d,0x50,0xbf,0x86, +0x42,0x94,0x79,0xac,0xab,0xba,0x58,0xe8,0x5c,0xdc,0xc5,0x11, +0x29,0x08,0x4b,0xcf,0x8b,0x45,0x67,0x13,0x7a,0xd4,0x17,0x85, +0xc0,0xc4,0x4e,0x13,0x8b,0x73,0x08,0xa8,0x26,0xec,0x81,0xf2, +0x88,0xc2,0xa6,0xd7,0x9e,0x13,0xe3,0x36,0x0e,0xce,0x41,0xaa, +0x74,0xa2,0xa7,0x12,0x53,0xe7,0x13,0x94,0x4c,0xae,0xc8,0xa8, +0x6f,0x3e,0x2d,0xc4,0xbb,0xe6,0xe5,0x9b,0x9a,0x2b,0x6f,0x52, +0x37,0x11,0x9f,0x79,0xff,0xee,0xa5,0xe1,0x30,0x86,0xc0,0x0d, +0x26,0x10,0x7d,0x6a,0xc0,0xc3,0xfd,0x2c,0xba,0x4f,0x4f,0xb5, +0x83,0x8a,0x81,0x34,0x0c,0x48,0x3d,0xf0,0x8c,0xad,0xda,0x8b, +0x76,0xf6,0xa3,0x34,0xd2,0x2b,0x82,0xde,0x26,0x68,0x59,0xc0, +0x6b,0xfe,0x95,0x4d,0xbe,0x5e,0x33,0xee,0xc8,0x76,0xc4,0x24, +0xa8,0x0a,0x44,0xa5,0x2c,0x33,0x7d,0x16,0xdb,0x81,0xa2,0x69, +0x40,0x66,0x22,0x8c,0x8d,0x99,0xcd,0x71,0x65,0x35,0x9f,0x47, +0x10,0x09,0x6b,0x0b,0x03,0xde,0x52,0x77,0x3b,0xd6,0x40,0x40, +0xa8,0x0a,0xf3,0xec,0xc4,0x15,0x64,0x1e,0xd8,0x8f,0x7a,0x7c, +0x8a,0xf3,0xb5,0xca,0xee,0xa2,0x7e,0xe0,0x27,0x38,0xac,0xed, +0xf1,0x94,0x0a,0x39,0x7a,0xb6,0xf3,0xf3,0x63,0x0a,0x91,0xf0, +0x31,0xdd,0x1e,0xb4,0x92,0x98,0xe2,0x20,0xf0,0x2b,0xbe,0x3c, +0xc8,0x00,0xa0,0x97,0x82,0x6d,0x5d,0xff,0xe7,0x68,0xf0,0x20, +0xc2,0x90,0x0b,0xf7,0xfb,0x3f,0xdf,0x1f,0x3c,0xb8,0x9f,0x2c, +0xb4,0xcc,0x72,0x5c,0xea,0xa2,0x49,0x2f,0x79,0x40,0xf1,0x9e, +0x77,0x30,0xee,0xf3,0xfe,0x8e,0xf2,0x57,0x39,0xae,0x4b,0x4b, +0x3e,0xa2,0x42,0x92,0x78,0x76,0x94,0x10,0xd8,0xb3,0x1d,0x47, +0x4e,0x48,0x00,0xd0,0x07,0x44,0xe8,0xaa,0x98,0x0c,0x4a,0xb9, +0x0f,0x29,0x9a,0x98,0xd8,0x52,0x83,0x96,0x8c,0xe1,0xfa,0x09, +0x24,0xec,0xa8,0xfa,0xe5,0xdf,0x92,0xb7,0xa3,0x7c,0x69,0x2a, +0xdb,0x01,0xf1,0x05,0xa3,0x41,0x68,0x59,0x45,0x63,0x95,0xf2, +0x14,0x64,0x1e,0xf7,0x31,0xe5,0xb8,0x8f,0x30,0x83,0x01,0xc0, +0xb8,0x29,0x02,0x57,0x14,0xe3,0xf3,0xd9,0xcd,0x02,0x8c,0x05, +0xca,0x5e,0x92,0xea,0x06,0xd8,0x59,0x5c,0x0f,0x8b,0xab,0xd9, +0x24,0x5d,0xcc,0xf2,0xab,0x74,0xf1,0x9f,0xc9,0x4e,0xa1,0x1a, +0x58,0x27,0xb4,0xc6,0xcf,0xc7,0xff,0xb9,0xfd,0xff,0x2e,0x36, +0x16,0x9b,0x68,0x85,0x28,0xf7,0x4b,0x99,0xd9,0xd4,0x46,0xc3, +0x69,0x3e,0x54,0xa7,0x53,0x70,0x38,0xa5,0x2e,0xa7,0xaa,0xb8, +0xbe,0x50,0x1f,0x4f,0xa7,0x51,0xa2,0xfc,0x4d,0xdb,0x77,0x72, +0x9f,0xe8,0x4a,0xeb,0x52,0xe8,0xb1,0x45,0x8d,0x46,0x6a,0x34, +0x53,0x57,0x7a,0xbb,0xa0,0x66,0xa7,0x93,0xd1,0x67,0x35,0x1b, +0xa9,0xd9,0xf9,0x64,0x32,0x53,0xb3,0x4b,0xfd,0x5f,0x3e,0xd4, +0xaf,0x55,0xac,0x2f,0xbd,0x3d,0xa5,0x43,0x3b,0xbd,0x51,0xd3, +0x59,0x05,0xda,0xc2,0xf9,0xca,0x71,0x38,0xfa,0x5e,0x85,0x76, +0xd5,0x97,0x04,0x1b,0xe0,0x3b,0x31,0x7b,0x85,0x95,0x6b,0xc4, +0xd1,0x08,0x9c,0xcd,0x29,0x70,0xfb,0x76,0x76,0xa5,0x37,0x5a, +0x25,0x04,0x99,0x43,0xd4,0xb8,0x99,0x53,0x67,0x43,0x72,0x02, +0x73,0x86,0xa7,0x54,0x70,0x28,0xac,0x46,0x85,0xfe,0x2d,0x7e, +0x53,0xa3,0x2b,0x85,0xf7,0xb3,0x72,0xf8,0x31,0x70,0xd0,0x72, +0x1a,0xee,0xb2,0xa3,0x2e,0xf7,0xd4,0xe5,0x43,0x75,0xf9,0x48, +0x5d,0x7e,0xa5,0x2e,0xbf,0x56,0x74,0x05,0x49,0x5d,0x52,0xab, +0x43,0x87,0x8d,0x81,0x8e,0x1b,0xa5,0x29,0x9a,0xeb,0x3d,0xea, +0x6f,0x4a,0xf7,0xe3,0x8d,0x46,0x46,0x21,0x6a,0x54,0x49,0x2e, +0x8d,0x34,0xa7,0xd2,0xd2,0x87,0x9a,0x5f,0x01,0xd7,0xf2,0x77, +0xf2,0x5f,0xda,0x9a,0xee,0xa3,0xd5,0xf0,0x54,0x0f,0x9c,0xe1, +0xd9,0x74,0x32,0xfe,0x7c,0xad,0x4e,0xd5,0xe9,0xa8,0xd0,0xff, +0x26,0xea,0xb4,0xb8,0xc0,0x11,0x55,0xc0,0x87,0x43,0x18,0x68, +0xfc,0xdc,0xf3,0xb1,0xca,0xaf,0xf5,0x87,0xc1,0x99,0x3d,0x0e, +0x33,0xa0,0xfc,0xc3,0xe9,0x88,0x3d,0x7b,0xe3,0x37,0x0c,0xa7, +0x1f,0xd4,0xaf,0x6a,0x3a,0x3f,0xfd,0xac,0x70,0x08,0xa8,0x52, +0x95,0x5a,0x52,0x50,0xe5,0x35,0xec,0x65,0xca,0x9b,0xe1,0x58, +0x81,0x28,0xf7,0x21,0x87,0x9f,0xc9,0xf8,0x42,0x69,0xd9,0x4e, +0xff,0xd3,0x63,0xae,0xd0,0xdb,0xdb,0xd9,0x4c,0xcd,0x95,0x66, +0x79,0xf0,0x8d,0xc1,0x02,0xea,0x86,0x96,0x69,0x17,0xf0,0x5d, +0x40,0x8e,0xf1,0xc4,0xac,0xa8,0x36,0x85,0x9c,0x34,0xfe,0x58, +0x09,0x9a,0xb1,0x61,0xac,0x82,0xc9,0x3d,0x2d,0x8b,0x8e,0x84, +0x53,0xbd,0xf9,0x81,0xfe,0x83,0x6b,0x31,0xd0,0x46,0x97,0xd3, +0xfc,0x5c,0x5d,0xe9,0xcf,0x19,0xe5,0xe5,0x99,0x2a,0xa7,0x67, +0x6a,0x5e,0xe6,0xba,0x3d,0x0c,0x79,0x54,0xbc,0x4a,0x9b,0xc1, +0x2c,0xba,0x04,0x3b,0xe4,0x4a,0x2f,0x33,0xfa,0xef,0x4c,0x0d, +0x3f,0x15,0x9a,0xe8,0x0b,0x3d,0x49,0x27,0x53,0x75,0x3a,0xc1, +0x20,0xc1,0x67,0xf9,0xd5,0xd5,0x0d,0xdc,0x45,0xd2,0xad,0x07, +0xcf,0xba,0x51,0xcf,0xf0,0x19,0xfc,0xe6,0x29,0x38,0x29,0x43, +0x5e,0x30,0xc1,0xbf,0x25,0xfe,0x81,0x66,0x87,0x53,0x8f,0xe1, +0xd9,0x0c,0xb4,0x8b,0xd3,0x51,0x89,0x83,0xf7,0x5c,0x8b,0x53, +0x3c,0x1e,0x4b,0xfd,0x5b,0x5c,0x5c,0xce,0xf0,0x53,0xae,0x34, +0xa5,0xea,0x12,0xf0,0xe6,0xaa,0x28,0xa1,0x5f,0x31,0x05,0xfe, +0xcc,0x87,0x17,0xb9,0x1a,0x4f,0xf0,0x83,0xc7,0x93,0x8f,0xd3, +0x21,0xf8,0xb1,0xb9,0xd2,0xff,0x7e,0x53,0xa0,0x56,0xc5,0x3f, +0x50,0xdb,0x74,0x0e,0x21,0x0a,0xd9,0x69,0x3d,0x9e,0xbc,0x00, +0x89,0xe5,0xe5,0x10,0xde,0x69,0x14,0x0c,0x61,0x74,0xcc,0xaf, +0xaf,0x41,0xd5,0xa3,0x5f,0x2e,0x72,0xcd,0x69,0x8a,0x99,0x1e, +0xe3,0x60,0x2e,0x8a,0xed,0xa6,0x1b,0x81,0x44,0x9a,0xdf,0x88, +0x96,0x8f,0xc5,0x68,0x76,0x09,0x63,0xc3,0x0a,0xb4,0x59,0x93, +0x23,0x2a,0x3d,0x81,0x6a,0x9d,0x4e,0x98,0x95,0x32,0x52,0xfd, +0x41,0x42,0xfe,0x49,0xe4,0x4a,0xec,0x16,0x62,0x27,0x6c,0xd5, +0x14,0x06,0xff,0x1f,0x57,0xb0,0xd2,0x46,0x70,0x12,0xea,0x3b, +0x44,0x81,0x15,0xfb,0xd5,0xcb,0xd7,0xff,0xfb,0x5f,0x27,0xef, +0xdf,0xbd,0x72,0xac,0xbc,0xba,0x10,0xc4,0xbc,0x12,0x24,0xbd, +0xfe,0x93,0xed,0x9f,0x86,0xdb,0xbf,0xef,0x6e,0x7f,0xdb,0x3e, +0xf9,0x5b,0x4b,0x2f,0x7f,0xff,0x2b,0x39,0x3e,0x7c,0xa0,0x99, +0x7f,0x79,0xdc,0x3e,0xee,0x1e,0xab,0xe3,0xf8,0x38,0x39,0xbe, +0x3d,0x5e,0x1e,0x3f,0x3e,0xde,0x1f,0xec,0xa8,0x83,0x27,0x2f, +0x5f,0x1d,0xbd,0x71,0x8b,0x0c,0xe3,0xd9,0xe9,0x86,0xb6,0x0d, +0x20,0xed,0x73,0xd3,0xa2,0x5e,0x63,0x13,0xf7,0x42,0x42,0x95, +0xd0,0xb5,0x8e,0x3c,0xe1,0x5e,0x89,0x9e,0xd5,0xd3,0xe1,0x47, +0x54,0x2f,0xd0,0x86,0xba,0x3f,0x50,0x1f,0x51,0xa2,0xc8,0x6a, +0x84,0x0c,0x94,0x4f,0xe0,0x62,0x2c,0xdc,0x04,0xa2,0xb3,0x00, +0x54,0xbd,0x54,0x4f,0x5d,0x98,0x02,0x30,0x9d,0xb2,0xd0,0x6d, +0x4a,0xcc,0x38,0xaf,0x4b,0x3b,0x5a,0x5d,0x3d,0x0b,0x9a,0x61, +0x0b,0x26,0xdd,0x04,0x6e,0xb2,0x98,0x8d,0xaa,0x32,0xfe,0x2f, +0x33,0x1b,0x2a,0x9f,0x2e,0x46,0x65,0x11,0xb7,0x06,0xde,0xdb, +0x01,0xff,0x62,0x64,0x24,0x45,0x9a,0x23,0xfa,0x1a,0xde,0x27, +0x42,0x65,0xd6,0xc6,0xbd,0x48,0xe4,0xb7,0xb4,0x61,0x7c,0xc3, +0xb5,0x19,0x53,0x82,0xb4,0x70,0xd1,0x30,0x12,0x40,0x89,0x8f, +0xae,0xb2,0x87,0xf6,0xba,0x09,0xee,0xee,0xd8,0x02,0x30,0xfb, +0x35,0xae,0x04,0x9b,0xdb,0x91,0x81,0x1b,0xdc,0x56,0xb8,0x39, +0x37,0xbd,0x1a,0xd2,0x4e,0x4a,0x0f,0x27,0xe1,0x2d,0xd7,0x50, +0x97,0x5b,0x1f,0xf1,0x75,0xe7,0xbc,0xa0,0x46,0xe5,0x34,0x9c, +0x1d,0xdd,0xeb,0xf0,0xf8,0xf6,0x9d,0x84,0x08,0x90,0x3e,0xe5, +0x4b,0xca,0x7a,0xb0,0xe8,0x81,0xe0,0xde,0x82,0x88,0x51,0x74, +0x93,0x98,0x6f,0x07,0x09,0x8d,0xb8,0x40,0x6d,0x6f,0x06,0xf3, +0xfd,0x3e,0xb2,0x95,0x87,0xce,0x38,0xb3,0x58,0xad,0xed,0x19, +0xdd,0xdd,0x75,0x2e,0x3e,0x4e,0xda,0x39,0x6c,0x8c,0x4e,0xd0, +0x22,0x3e,0x16,0x05,0x54,0xd3,0x49,0x88,0x83,0xf1,0xdc,0x31, +0x92,0x91,0xdf,0x89,0x5c,0x48,0x44,0x86,0xb9,0xde,0x64,0x6f, +0xf1,0xff,0x3a,0xd7,0x1f,0xc0,0x37,0xa4,0x78,0xc3,0x84,0x37, +0x30,0xdb,0xb3,0xc9,0x2b,0xd8,0xae,0x3c,0x85,0xed,0x4a,0x32, +0x48,0xd4,0x89,0x9e,0x18,0xcf,0xc1,0x3d,0x5d,0x2c,0xf1,0xe9, +0xc9,0xc1,0xa7,0x20,0x5e,0x2d,0xe6,0x03,0xcd,0xbd,0x7a,0xac, +0xb3,0x28,0x0f,0x87,0xe7,0x6c,0xc7,0x6b,0x7c,0x5c,0xf4,0xaa, +0x5a,0x4e,0xba,0xf3,0x6e,0x3f,0x80,0x18,0xe9,0xad,0x6e,0x46, +0xb8,0xf3,0x06,0xd6,0x91,0x6e,0x6f,0x43,0x77,0xd8,0xd7,0x29, +0x87,0x5e,0x2a,0x52,0xb6,0x53,0x65,0x33,0x38,0x77,0xe1,0xb5, +0xd4,0x74,0x1d,0x20,0x41,0x10,0xc1,0x43,0xcf,0xeb,0x48,0x91, +0x8b,0x24,0xc5,0x2e,0x05,0xe1,0x8a,0x05,0xdc,0xe7,0xcf,0xa3, +0x41,0xd7,0xfb,0x1a,0x31,0x40,0xf4,0xec,0x1a,0xd2,0x85,0x23, +0xdb,0xa5,0x7c,0xb9,0xbd,0x8c,0x45,0x05,0xca,0x02,0xfa,0x4d, +0x9c,0x2c,0xe9,0x70,0xe0,0xb4,0x9c,0x5c,0xa1,0x75,0x01,0xdc, +0x42,0x1d,0x6b,0x2e,0x49,0xbc,0x17,0x59,0xef,0x4e,0xe1,0x6a, +0x7f,0xc2,0x80,0x60,0xea,0xeb,0x8c,0xc2,0x70,0x32,0x0a,0x85, +0x31,0xbc,0x6e,0x6d,0x05,0x48,0x49,0x27,0xc7,0xb0,0xf6,0xc6, +0xdb,0xa9,0x26,0x03,0x90,0x9d,0xb4,0xfd,0x13,0x74,0x99,0x99, +0x80,0x6a,0xcb,0x4f,0x52,0x34,0x33,0xc0,0x25,0xc2,0x77,0x8c, +0x41,0xba,0x07,0xe3,0x34,0x4b,0x52,0x50,0x99,0xfe,0xa8,0x7b, +0x3b,0x44,0x8f,0x00,0xee,0xb9,0x67,0x4f,0xd7,0x26,0x92,0xdd, +0xb5,0x1e,0x97,0xc8,0xf7,0x67,0xb8,0x0a,0xbc,0xff,0xf6,0xa2, +0xc8,0xaf,0x46,0xf6,0x9a,0x9f,0x48,0x42,0x5f,0xb6,0x8e,0xf4, +0xe7,0x02,0x58,0x92,0x2f,0xd2,0xc3,0x4f,0x90,0xf8,0x7d,0x48, +0x9f,0x0e,0x9e,0x6a,0x3f,0xe8,0x6e,0xf4,0xa9,0xa9,0x64,0xe0, +0xfd,0x43,0x47,0xd3,0xb3,0x4a,0x41,0x5f,0x61,0x52,0xa1,0xa8, +0x5a,0x13,0xc1,0xf9,0xf4,0x7c,0x2c,0x66,0x97,0x00,0x62,0xae, +0x81,0x7b,0x3c,0x4e,0x0c,0x5e,0x73,0x05,0x16,0x2f,0x40,0x37, +0xd5,0xe0,0x86,0x0f,0x29,0x15,0x5e,0x8e,0xca,0xfa,0x01,0x64, +0xb3,0x93,0xde,0xe6,0x6e,0x1a,0x26,0xca,0x9e,0xb0,0x78,0x9c, +0x3d,0x24,0x26,0x61,0x40,0x4d,0xbf,0x26,0x7a,0x0a,0x5b,0xda, +0x63,0x6a,0xd9,0x8a,0x3c,0xdd,0xde,0x78,0xb3,0x7d,0x09,0x27, +0x4d,0xf4,0x3b,0xc7,0x1f,0x73,0x4f,0x1d,0x30,0x5f,0x5f,0x4f, +0xc6,0x60,0x59,0x5c,0xed,0x18,0xbf,0x1e,0xff,0xaa,0x57,0x87, +0x6c,0x8d,0xf7,0x78,0xe1,0x32,0xbc,0x65,0xe0,0x5d,0x68,0xde, +0x43,0x77,0x7f,0x1a,0xd8,0x2d,0x02,0xb2,0xe5,0x18,0x43,0x8f, +0x4b,0xd3,0x49,0x62,0x44,0x14,0x45,0x49,0x7a,0x62,0x9d,0x5e, +0x10,0x12,0x0b,0x48,0xc8,0x3b,0xa9,0x57,0x8e,0x61,0x14,0xaf, +0x33,0x9c,0x27,0x4e,0x1a,0x89,0x1f,0xdf,0xae,0x5a,0x1f,0x08, +0x66,0x90,0x59,0x3b,0x6e,0x1a,0x57,0xdc,0x30,0xd3,0x75,0x9b, +0x3d,0x1c,0x4c,0x24,0xc7,0x87,0xb3,0x82,0x52,0x1b,0xa6,0x04, +0x17,0x71,0x3a,0x27,0x76,0x8f,0xd1,0x30,0x21,0x18,0xdc,0xec, +0x18,0x18,0xdf,0x3a,0xf5,0xfb,0x9f,0xb8,0x56,0x09,0x4e,0xa5, +0xc6,0x7a,0xf3,0x5b,0x3e,0x9d,0x16,0x12,0x3a,0x4c,0x0f,0x18, +0xd1,0x41,0x58,0xac,0x32,0xe7,0xf1,0x3a,0xab,0xcb,0xb7,0x0b, +0x5e,0x7f,0xc0,0xfe,0xd3,0xad,0xe4,0x65,0xd7,0x0a,0x83,0x6e, +0x4c,0xd8,0x63,0xe3,0x6e,0x41,0x39,0x3c,0x7a,0x21,0x77,0x2f, +0x3c,0xd4,0x68,0x1a,0xb8,0x2b,0xd9,0x95,0x4f,0x72,0x25,0x7c, +0x16,0x53,0xa9,0x31,0x0b,0xc6,0x99,0x92,0x1f,0x43,0x57,0x06, +0x98,0x57,0x2d,0x16,0x0d,0x95,0xf1,0x47,0x6d,0xfa,0x73,0xe4, +0x84,0x1c,0x1a,0x12,0x0a,0x11,0xd7,0x6c,0x2c,0x42,0x2b,0x8e, +0x73,0x7f,0xa5,0x85,0x28,0x37,0x35,0x32,0x0e,0x08,0x06,0xe2, +0xf2,0xf9,0x53,0xd8,0x97,0x4d,0xa6,0x59,0x25,0x65,0xb1,0x88, +0xe0,0xce,0x41,0x24,0x3a,0xed,0xbd,0x03,0x77,0x8b,0x07,0x1e, +0x64,0xc3,0xe6,0xe5,0x04,0x9c,0x0f,0xc4,0x15,0x3c,0x6c,0xf9, +0x47,0x1e,0xc2,0xf8,0x60,0x07,0x4e,0x5e,0xf1,0x5e,0x48,0x1c, +0xe9,0x2d,0xc4,0x06,0xdc,0x33,0xb5,0xd1,0x8a,0x36,0x8a,0x72, +0x7c,0x7f,0xb6,0x81,0xfb,0xf2,0x28,0xec,0x63,0x41,0x30,0x5a, +0x0b,0x78,0xdd,0x21,0x04,0x64,0x7f,0x05,0xaa,0x64,0xe8,0xe9, +0x06,0xd1,0x78,0x74,0x05,0x74,0x05,0x24,0x8d,0xd8,0x18,0x91, +0xae,0x95,0x1b,0x09,0x37,0x8d,0xfc,0xf7,0x48,0x09,0x4c,0x4f, +0xad,0x37,0xeb,0x34,0xaa,0x4d,0x8e,0xd4,0xd9,0x70,0x3c,0x9e, +0x8c,0x8b,0xb3,0xe1,0xd5,0x4b,0x5d,0xdb,0xc9,0x89,0x7c,0x8f, +0x54,0x3e,0x1b,0x5e,0x78,0x45,0x9f,0xeb,0x04,0x0c,0x0b,0x74, +0xfe,0x0a,0x22,0x58,0x46,0xb0,0xc9,0x89,0xc0,0x4f,0x47,0x4a, +0xb2,0x9a,0xfe,0xf3,0xaa,0x28,0xe9,0x0d,0x1e,0x22,0xe4,0xe3, +0x91,0xfe,0x13,0x11,0x67,0x8f,0xe0,0x6f,0x64,0xd8,0xba,0x19, +0xd7,0xa8,0x41,0x48,0x59,0xd0,0x43,0xe9,0x2e,0x65,0x21,0xcf, +0xb8,0xa7,0x4e,0x9d,0xf8,0x77,0x03,0xbb,0x1b,0xf8,0x6e,0x8c, +0x9b,0xad,0x2b,0x12,0x82,0xbf,0xee,0x7e,0xac,0xdb,0x4f,0x42, +0x2a,0x4c,0xa0,0x86,0xc8,0x3d,0x47,0xe0,0x8f,0x45,0x80,0x1e, +0xb0,0xc7,0x93,0xba,0x54,0xac,0xe9,0x2d,0xb6,0xb5,0xfd,0x42, +0xf7,0x1a,0x91,0x2d,0x74,0x1a,0xe1,0x4f,0x04,0xfe,0xf4,0xa0, +0xef,0xca,0x48,0xb2,0xcc,0x77,0x95,0xce,0x97,0x17,0xba,0xea, +0x45,0x9b,0xea,0x80,0xb1,0x5c,0xa6,0x09,0x42,0x39,0x64,0xfe, +0xe0,0xd3,0x24,0x60,0xfa,0xd1,0x04,0xbe,0xdf,0x55,0x7e,0x0e, +0x89,0xe8,0xa6,0xd5,0x18,0x62,0xb1,0xe3,0x2b,0xb3,0x2d,0x47, +0x00,0x63,0xde,0x04,0x2e,0x1f,0x0a,0x32,0x4c,0x41,0xdf,0xba, +0x95,0x6d,0x13,0xec,0xd6,0x8a,0xe1,0x55,0x2c,0xb7,0xb9,0x6e, +0x49,0xd6,0x89,0x7a,0x2e,0x52,0xf9,0x3e,0xbc,0x0d,0xd0,0x88, +0x8a,0x11,0x7a,0x19,0x4b,0x5b,0x51,0xff,0x84,0xec,0x04,0x04, +0x4a,0xf6,0x05,0xe1,0x7f,0xe1,0x05,0x7f,0xe1,0x8b,0xe9,0xe4, +0xba,0xe9,0x1b,0xd7,0xff,0x3a,0x69,0x86,0x88,0x37,0x41,0x92, +0xe0,0x5b,0x1d,0x92,0x86,0x0f,0x84,0xf3,0xa0,0xba,0x2f,0x4b, +0xdc,0xa7,0xc9,0xde,0x79,0x59,0xe9,0x1a,0xec,0x94,0x62,0x14, +0x8e,0x0a,0xbf,0x27,0x1b,0x47,0x02,0x04,0xa3,0x31,0x18,0x2a, +0x0d,0xf5,0xb2,0xa6,0x95,0xa8,0x7d,0xfc,0xba,0xc2,0x36,0xbd, +0xb3,0xb6,0x44,0x2c,0x3e,0xff,0x00,0x16,0xf9,0x72,0xe4,0xd7, +0xf0,0xd2,0x7e,0x4f,0x14,0x6d,0x66,0x19,0x25,0x6d,0x6d,0x05, +0xeb,0x09,0x43,0x72,0xfa,0xeb,0xf9,0xd5,0x95,0x49,0x92,0x4d, +0xa6,0x67,0x77,0x6d,0x9b,0xcd,0xab,0x1b,0x9d,0x75,0x1b,0xcd, +0x70,0x37,0x65,0x11,0x55,0xda,0x4e,0xd7,0xfa,0x57,0x37,0x9e, +0x57,0xab,0x58,0xfe,0xca,0xfc,0xa9,0x60,0xc9,0xf5,0x12,0x7d, +0x00,0xa4,0xe5,0xfa,0x4e,0x5a,0x9f,0x25,0x57,0xca,0x00,0x73, +0xfd,0x29,0xfb,0x46,0x03,0x1d,0xf5,0x93,0x4f,0x82,0x34,0x35, +0x10,0x9a,0xe2,0x4a,0x54,0x90,0xdc,0x6f,0x6c,0x17,0x09,0x3a, +0x00,0x0f,0x4d,0x73,0x2c,0x14,0x8e,0xb0,0xf8,0xcc,0x6b,0x00, +0x6f,0x59,0xab,0x1b,0xf3,0xfe,0x7c,0x36,0x68,0xe5,0x3a,0x4d, +0x5e,0xd4,0xc8,0x77,0xa0,0x13,0x3e,0x2a,0x19,0x8b,0x45,0xbd, +0x0e,0x04,0x15,0x1c,0x3e,0xf3,0x0f,0x30,0xba,0x76,0x31,0x49, +0x35,0xcc,0x3f,0x28,0x63,0x41,0x7d,0x69,0x58,0xe0,0x97,0xce, +0x1e,0x57,0xd4,0x6e,0x3e,0xf4,0x5c,0xcf,0x2b,0xde,0x12,0xc8, +0x92,0x2b,0x73,0xc5,0x27,0x9b,0xeb,0x21,0x56,0x93,0x82,0x97, +0x40,0x51,0xd3,0x66,0x44,0x7c,0xdf,0xdf,0x46,0x25,0x98,0x16, +0x83,0xa5,0x3e,0xb8,0xb9,0xa0,0xea,0x7f,0x61,0x85,0x4c,0xe7, +0x1f,0x50,0x38,0xba,0x0b,0x1a,0xb1,0xe1,0x0b,0x7d,0x76,0xb1, +0x3e,0xfd,0x2b,0xe9,0x35,0xbe,0xce,0x53,0x41,0x4f,0x53,0x05, +0xe0,0xc3,0x28,0xec,0xc4,0x17,0x77,0xd0,0xfa,0xe5,0x9f,0x29, +0x4a,0x04,0x3a,0x99,0xd0,0x33,0x68,0x43,0xba,0xe8,0xec,0xc4, +0xdb,0x00,0x56,0x5c,0x8b,0xae,0x43,0x65,0xa5,0xd6,0x46,0xfa, +0x26,0xe3,0xef,0xf2,0xf3,0xc9,0x14,0x8f,0x63,0x84,0xe0,0x54, +0xfc,0x9e,0xdb,0xc9,0xbf,0x02,0x24,0x18,0xa2,0xce,0x7b,0x93, +0x7e,0x91,0x1f,0xf1,0x66,0x45,0x2d,0x6e,0x49,0xd7,0x22,0x6b, +0xf8,0x21,0xab,0xc8,0x03,0xf8,0xf0,0x63,0x56,0x7e,0xc6,0x9f, +0xfa,0x80,0x3f,0x40,0xfa,0x1d,0x44,0x3b,0xec,0x20,0xda,0x1e, +0x7d,0xbe,0xc9,0x5f,0x92,0x7a,0x23,0xf3,0x86,0x82,0x59,0x56, +0x2f,0x27,0xf3,0xab,0xd1,0xe1,0xf0,0xb7,0x9c,0xc4,0xe1,0xac, +0x21,0xdd,0x1b,0x48,0xbc,0x83,0xdc,0x95,0x95,0x11,0xd8,0x15, +0xb8,0x7c,0x6c,0xd2,0xae,0x39,0x45,0x0b,0x67,0xf4,0x9a,0x88, +0x08,0xaf,0xde,0xdb,0x2d,0xab,0x55,0x04,0xb3,0x42,0x8e,0xaf, +0xbc,0xa3,0x0e,0xe7,0xbb,0xc9,0x04,0x2e,0x09,0x1a,0xe4,0x70, +0x83,0xf2,0x2e,0xf4,0x16,0x33,0x95,0x59,0x86,0x22,0xf7,0xfc, +0xfc,0xbc,0xf8,0x54,0xbf,0x6a,0x53,0x5e,0xd2,0x83,0x7b,0xaa, +0xa9,0x97,0x56,0x65,0x7a,0x87,0x84,0x47,0xea,0x43,0x29,0xa9, +0x22,0xe4,0x10,0xa0,0xcd,0xf7,0xc9,0xc1,0x4b,0xcf,0xe0,0xc7, +0x5f,0x68,0x41,0x64,0x9a,0xd4,0xf1,0xe8,0x4d,0x10,0x6f,0x93, +0x8e,0x1c,0x84,0xa3,0x00,0x4f,0x58,0xcb,0x5c,0x23,0x19,0x3d, +0x99,0x5e,0x40,0x35,0x53,0x78,0x60,0x6b,0xb9,0xd2,0xed,0x31, +0x29,0x60,0xa7,0xa8,0x85,0x62,0x47,0xed,0xe9,0xbd,0x7f,0xe8, +0x90,0xb9,0x27,0xa1,0x2c,0xee,0x34,0x96,0xc9,0xa6,0x22,0xe5, +0xd5,0x61,0x81,0xc9,0xae,0x1f,0x88,0x3b,0xba,0xe3,0x5b,0xfb, +0x00,0x34,0xe8,0x86,0xa0,0x8b,0x45,0xbc,0x7e,0xe9,0xac,0x0f, +0x01,0xf1,0x02,0x70,0x52,0xd9,0x08,0x17,0x93,0xfa,0xf3,0xbd, +0x39,0x6c,0xe5,0x59,0xf9,0x09,0x09,0xa8,0x5b,0x20,0xee,0xaf, +0x4c,0x94,0x0d,0x42,0x7c,0x00,0xd7,0x4d,0x88,0x89,0xcf,0xbd, +0x44,0x7b,0x41,0xd1,0x12,0xfe,0xc0,0x57,0xe7,0xee,0x10,0x64, +0x55,0xbf,0x32,0xf4,0xe6,0x2e,0x14,0x08,0x90,0x63,0x64,0x9b, +0x3f,0x89,0xb7,0x43,0x78,0xc3,0x86,0xac,0xdb,0xec,0x88,0xcf, +0x67,0xc7,0x64,0x82,0x55,0xb1,0x7f,0x9c,0x75,0x3a,0x08,0xcb, +0x0e,0x14,0x85,0xee,0x43,0xd2,0xbc,0xed,0x6a,0x88,0x63,0x6b, +0x8b,0x37,0x75,0x61,0x86,0xdb,0xda,0x09,0xf4,0xc9,0xad,0xc4, +0x2b,0x7b,0xc9,0xa3,0x5f,0x00,0x61,0xd0,0xaf,0x15,0x0c,0x38, +0x16,0xb0,0x77,0x35,0xc2,0xd2,0x97,0xde,0x48,0xb4,0xab,0x67, +0x32,0x12,0x42,0xee,0x0b,0x64,0x7a,0x45,0x00,0x61,0x84,0x6e, +0xf3,0x0c,0x7e,0x03,0x6a,0xe5,0x0d,0x0b,0x79,0x15,0x1c,0xf9, +0x54,0x94,0x6b,0x70,0xc0,0x0a,0x7d,0x07,0x07,0x44,0x15,0x0d, +0x9d,0x5e,0x4e,0xba,0x22,0xc3,0xf9,0x6d,0x05,0x9a,0xe8,0x20, +0x38,0x0b,0x7c,0xb6,0xfa,0x0e,0xdc,0x28,0x8d,0xc7,0x01,0x53, +0x52,0x8f,0x90,0xd4,0x65,0xe1,0x7a,0xc6,0x97,0xf1,0x5c,0x84, +0x35,0x06,0x03,0xa3,0x49,0xe3,0x05,0x20,0xe1,0x34,0x9a,0xdf, +0xa6,0x84,0xf5,0x19,0xc0,0xbf,0x7d,0x41,0x4f,0xcd,0x76,0xc6, +0xd7,0xd6,0x09,0xdf,0x8b,0x84,0x7a,0x9a,0xff,0x96,0x4f,0x51, +0x05,0xda,0x40,0xbe,0x3d,0xaf,0x0e,0x49,0xe7,0x33,0x6e,0xbd, +0xe3,0xb9,0xb2,0xea,0x6f,0xfd,0xcc,0x7a,0xfe,0x8f,0x97,0xc3, +0x19,0x6a,0xf3,0x94,0x38,0x1c,0x75,0x5e,0x6c,0x32,0x71,0xc6, +0xe4,0x4a,0xb9,0x90,0x1b,0x2b,0x8e,0x10,0xda,0x74,0x0a,0x92, +0x18,0x79,0x39,0x44,0x25,0x4f,0x0f,0x6a,0x70,0x71,0x76,0xd2, +0x05,0xe2,0xdc,0x82,0xcb,0x07,0xd4,0x8e,0xd8,0x1e,0xa3,0xef, +0x47,0x2f,0xcf,0xb7,0x5f,0x4f,0xc6,0xf9,0x36,0x7a,0x58,0x89, +0x06,0x19,0xea,0x28,0x65,0xae,0xcc,0xa0,0xe5,0x00,0x8c,0x3a, +0xb0,0x6e,0x38,0xeb,0xb4,0x3d,0x67,0xda,0x5e,0xe7,0xb6,0x32, +0x68,0xa0,0x5e,0xb4,0x13,0xb5,0xe0,0x01,0x42,0x9f,0x63,0xaa, +0x24,0x98,0x56,0x56,0xf4,0x20,0x59,0x67,0x69,0x20,0x40,0x85, +0xb9,0xc1,0xad,0x51,0x88,0x4a,0x4c,0xe1,0xb1,0x5f,0xec,0xbb, +0x69,0xf6,0x77,0x14,0xc1,0x7e,0x43,0x73,0x11,0xd0,0xb6,0xfe, +0x85,0xe8,0x40,0x41,0xbb,0x1e,0xba,0xb7,0xef,0xd7,0x41,0x37, +0x59,0xfb,0x6b,0xd1,0x55,0xf5,0x9d,0x08,0x59,0x55,0xbc,0x1e, +0x4a,0xf6,0x55,0x7d,0x27,0x52,0xd4,0x37,0xaf,0x87,0xf2,0x87, +0xe7,0x4f,0x9e,0xdd,0x8d,0x90,0x94,0xd6,0xeb,0x61,0x3c,0x7a, +0xf7,0xe4,0xe9,0x1a,0x34,0x1a,0xcd,0xf7,0x7a,0x48,0x39,0x4c, +0xf2,0x1a,0x1d,0x84,0xea,0xf3,0x35,0x7b,0x08,0x82,0xa2,0xdc, +0x89,0x72,0xc9,0x06,0x13,0xe8,0x10,0xd4,0xe3,0xe9,0x90,0x22, +0x9c,0x70,0xc3,0x99,0x8a,0x60,0x6b,0xca,0xcf,0xc6,0xd9,0x59, +0xc3,0x92,0x01,0xf5,0xc9,0x09,0x10,0x8c,0xf3,0xd1,0xe9,0x60, +0xa7,0xf9,0x68,0x6e,0x66,0x9e,0x64,0xeb,0x8e,0x37,0xbb,0x5b, +0x70,0x67,0xf3,0x6b,0xa1,0xfd,0x85,0x27,0xb0,0x52,0x80,0xdf, +0x43,0x56,0xbc,0x65,0x54,0x47,0xbb,0x56,0xcb,0xc7,0xea,0xa2, +0xe2,0x3c,0x96,0x45,0xf0,0x50,0xc9,0x2f,0xe6,0x59,0x83,0xf8, +0xc0,0x89,0xd8,0xc2,0x99,0xc4,0x2e,0x53,0x92,0xc9,0x44,0x1b, +0xab,0x55,0x66,0xf6,0xfd,0x7a,0xaa,0x2b,0x0b,0xcb,0x3b,0xeb, +0xc1,0xd6,0x1d,0x06,0x0d,0x5c,0xbc,0xa3,0xb5,0x90,0x68,0xc0, +0x41,0x57,0xff,0x01,0xc7,0xfd,0x44,0x68,0x0b,0xdc,0xc1,0xb6, +0x74,0x12,0x9b,0x94,0x29,0xb8,0x89,0x61,0x5d,0x6f,0x91,0xf6, +0xb6,0x4b,0x3f,0x41,0x6b,0x07,0xea,0xc5,0x5e,0xa5,0x2f,0xea, +0x34,0x8b,0x2c,0x11,0x57,0x60,0x43,0x88,0x8a,0x3a,0xda,0xe9, +0x96,0x7d,0xca,0x83,0xbe,0xb4,0xd6,0x0d,0xbd,0x9a,0x00,0x5b, +0x8c,0x22,0xe5,0x5f,0xeb,0x56,0x14,0x46,0x1a,0xb8,0xb6,0x6d, +0x31,0x66,0x7f,0xd5,0x33,0xc6,0x3b,0xcb,0x70,0xec,0x9f,0xe7, +0x7a,0x56,0x7a,0x16,0x3b,0x66,0x5d,0x87,0x05,0x89,0x2f,0x71, +0xb0,0xa5,0x4e,0xf3,0x52,0x06,0xb0,0xfa,0xa3,0x18,0x90,0x3e, +0x0a,0xcb,0x2b,0x4e,0x6a,0x96,0xcf,0xd0,0xdf,0x5c,0x06,0x54, +0xd1,0x64,0xbe,0xb8,0x9a,0x9c,0x0e,0xaf,0x48,0xf6,0xe2,0xb8, +0x21,0x35,0x56,0x79,0x38,0xf9,0x54,0x0d,0x6c,0xe2,0xfb,0xce, +0x35,0x2e,0x70,0x6b,0xc2,0xae,0xc9,0x78,0x6b,0x64,0x81,0x7b, +0x98,0x4f,0x7f,0x2b,0xce,0xf2,0x17,0x93,0xa9,0x87,0xd2,0x0a, +0x7e,0x16,0x7c,0xea,0x89,0xd0,0xc0,0x59,0xac,0xcf,0x6e,0x90, +0xa2,0x9d,0x23,0xe4,0xd5,0x2a,0x66,0x9a,0x39,0xe4,0xfc,0x4e, +0xad,0x06,0xad,0x1c,0x2d,0x0e,0xb4,0xbc,0x83,0xb7,0xc0,0x74, +0x83,0xfe,0xc0,0x77,0x08,0x4d,0x5f,0x2a,0x97,0x46,0xbc,0xe7, +0x0e,0xe4,0x75,0xc7,0x8e,0x16,0x7f,0x5d,0xa6,0xf5,0xe3,0xb3, +0x0e,0x76,0x3c,0x99,0xb4,0xe8,0xe0,0x4c,0x2b,0xd8,0x81,0xf0, +0xee,0x8d,0xda,0xce,0x61,0xd4,0xdb,0x95,0xdc,0x94,0x02,0x87, +0x56,0x61,0x36,0x04,0x4d,0x32,0x34,0xf2,0x4e,0x57,0x66,0x97, +0x20,0xd1,0x0e,0x2d,0x08,0xbf,0xd6,0x55,0x42,0x16,0x6f,0xa6, +0x1e,0xc3,0x9e,0x83,0xaa,0x04,0x10,0xbd,0x49,0x20,0xea,0xfc, +0xad,0xad,0x06,0xad,0x0d,0xeb,0x82,0xac,0xe0,0xab,0x93,0xdc, +0x49,0x82,0xcf,0x37,0x28,0xd7,0x60,0x84,0x4a,0xeb,0x97,0x05, +0x03,0x57,0x69,0x6e,0x3d,0x78,0x59,0x55,0x74,0xd2,0x9e,0x8f, +0x61,0xf4,0x9e,0x90,0xa9,0x78,0xa9,0x1f,0x6e,0x8a,0xb3,0x0f, +0xcd,0xc7,0xb7,0x7d,0x73,0xd2,0x1f,0x1e,0xed,0x0f,0x92,0xa4, +0xc6,0xa2,0x2e,0x09,0x2c,0x02,0x32,0xc6,0xcf,0x33,0xa1,0x99, +0xb2,0xa4,0x5b,0xe1,0x8b,0xa6,0x51,0x12,0xd7,0x86,0xee,0x54, +0x32,0xf6,0xeb,0x51,0x16,0xb8,0xca,0x5f,0x6d,0xb3,0x79,0x88, +0xec,0x59,0x5d,0x3d,0x26,0x74,0xad,0xba,0x7a,0x0c,0x07,0xbb, +0xa7,0xcc,0x7f,0xa7,0x75,0xf3,0xcb,0x30,0x48,0x97,0xf2,0xa8, +0x5a,0xb1,0xcc,0x05,0x82,0x4b,0x9b,0x36,0x44,0xce,0x00,0xc7, +0xb0,0x66,0x1d,0x13,0xe6,0xc2,0x35,0x0d,0x65,0xcf,0x5c,0x7d, +0x06,0x25,0x3e,0x5d,0x32,0x28,0x57,0x25,0xf8,0x5f,0x95,0x55, +0x4a,0xf7,0xae,0x02,0x8f,0x5b,0xb2,0x0d,0x74,0x7f,0xa0,0x8c, +0x02,0x47,0xa0,0xe3,0xb9,0x16,0xa0,0x24,0x3f,0x64,0x56,0x5f, +0xe4,0x6e,0x05,0x8a,0xd8,0xe7,0x2b,0xd7,0xfc,0x0c,0x85,0x2c, +0xb9,0x47,0xbf,0xb9,0xd2,0xec,0xba,0x0d,0xbb,0x3d,0x17,0xdb, +0x4e,0xed,0x85,0x9f,0x5f,0x25,0xfb,0x2c,0x54,0x72,0x78,0xe4, +0x13,0x13,0xf0,0xfb,0x60,0x6e,0x62,0xb3,0x34,0x76,0x82,0x1b, +0x64,0xee,0xf8,0xf6,0xcb,0xbb,0xe1,0xdf,0x52,0x71,0xb5,0x01, +0x5c,0xf5,0xfe,0xa7,0xfb,0xe5,0xde,0x52,0xa8,0xa2,0xd8,0x84, +0x64,0xf2,0xd4,0x86,0x66,0x88,0x70,0x26,0xf6,0x82,0xe1,0x90, +0x9c,0x06,0x71,0x9e,0x4c,0xf8,0xb1,0xc4,0x3c,0x00,0x27,0x0b, +0xe0,0xbe,0xcf,0x67,0x15,0xb0,0x15,0xab,0x66,0xad,0x0c,0xe9, +0xa9,0x2f,0x3d,0x4a,0xe9,0xee,0x06,0x56,0x0d,0x0a,0x94,0x15, +0x34,0x6a,0xc6,0xa7,0x01,0xa2,0xc4,0x12,0x21,0xdc,0x94,0x38, +0x68,0x6b,0x27,0x89,0xf1,0xd6,0xf2,0xf3,0x5c,0x8b,0x42,0xa3, +0xec,0xde,0xaf,0x6d,0x7c,0x06,0x4f,0x91,0x1a,0xe8,0xc9,0xf4, +0xa2,0x74,0x6a,0xe6,0xae,0x54,0x19,0xf9,0xde,0x8d,0xc1,0x33, +0xbc,0x54,0x88,0x54,0xc6,0xb8,0x1b,0xe2,0x06,0xaf,0xea,0xd0, +0x06,0x03,0xce,0xc2,0x8d,0x41,0x26,0x3a,0x0c,0x6a,0x93,0xb7, +0x26,0x08,0x1f,0x64,0x5c,0xef,0x1a,0xfa,0x8c,0xbf,0x6e,0xac, +0x6f,0x19,0x0e,0x10,0xd3,0xd1,0x16,0xda,0x34,0x08,0x10,0xfc, +0xa5,0x3d,0x91,0x54,0x5a,0x4d,0xf7,0x6f,0xec,0xa4,0xc9,0x9a, +0x26,0xbb,0xab,0x7d,0xea,0xbe,0xa2,0x0f,0x18,0x07,0xff,0x7d, +0xdf,0xc2,0x55,0x87,0xe8,0x6d,0xdc,0x33,0xe5,0x9d,0xc2,0xd7, +0x28,0x2b,0x7b,0x95,0xaf,0xb0,0x16,0x55,0x16,0xc7,0xad,0xbc, +0xa3,0x90,0xc6,0x95,0x12,0x14,0xd2,0x11,0xae,0x05,0x6e,0x78, +0x0c,0x77,0x36,0x2d,0x6e,0x84,0x54,0x15,0x07,0x6a,0xff,0x09, +0xb8,0xb4,0xc4,0x39,0x2f,0x84,0x01,0x4c,0x6c,0x14,0x06,0xc0, +0x00,0x30,0x91,0x7e,0xa8,0xb5,0x1c,0xf8,0x74,0x5e,0xce,0x26, +0xd7,0x6f,0x8c,0x6e,0x8c,0x6b,0x41,0x09,0xe9,0xcc,0xcf,0xb2, +0xd2,0x1e,0x26,0x07,0xe2,0x22,0xeb,0xbb,0xfb,0x64,0x9d,0xc7, +0x86,0x79,0x7c,0xe5,0x82,0xad,0x52,0x07,0xca,0xbb,0x54,0x91, +0xdc,0xda,0x32,0xa3,0x89,0x86,0x22,0xb4,0x12,0x4a,0xaf,0xff, +0xc3,0xd2,0xd8,0x78,0x5c,0x5d,0xd9,0x1a,0xe1,0xc5,0x51,0x65, +0xd0,0x63,0x80,0x97,0x7c,0xda,0x33,0x66,0x80,0x29,0xbc,0xa1, +0x15,0x66,0x86,0x88,0x5a,0xf0,0xde,0x9e,0x4d,0xde,0xdf,0xdc, +0x18,0xd3,0xd0,0xae,0x44,0x9b,0x21,0xe9,0x9b,0x59,0xe6,0xa1, +0xdf,0xda,0xa2,0x6f,0x09,0xd3,0x7b,0x7e,0x2b,0xa4,0xd5,0xfb, +0x22,0xc4,0x8f,0xf1,0x80,0xde,0xd7,0x75,0x88,0x3e,0xac,0x6b, +0x4f,0x0c,0x65,0x1b,0x07,0x47,0xfc,0x3e,0x06,0x63,0xd6,0x83, +0xd2,0x09,0x79,0x71,0x32,0xb8,0x64,0x3b,0x21,0x01,0x1e,0xd5, +0x09,0x46,0x73,0x12,0x9d,0xfb,0xfd,0xf3,0xa3,0x57,0x2f,0x0f, +0x8f,0x4c,0x71,0xdc,0x6c,0xd4,0x6c,0x04,0xda,0xa3,0x89,0x81, +0xac,0x16,0x76,0xc3,0xa9,0x79,0x1f,0x10,0x9f,0xb3,0x90,0x6b, +0x95,0x30,0x1a,0xb6,0xb0,0xd9,0xbe,0x75,0x9c,0x84,0xed,0xd6, +0x0a,0x2d,0xcf,0xc9,0xfa,0x49,0x60,0x58,0x5b,0x48,0x53,0x5e, +0x31,0xb5,0x26,0x12,0xda,0xd5,0x35,0x2e,0xaa,0xa1,0x14,0xe0, +0x96,0xfa,0x33,0xb1,0xb8,0xda,0x15,0xff,0xee,0x63,0x7f,0x6b, +0x0b,0xb3,0xd9,0x61,0x89,0x82,0xdc,0xf3,0xe2,0x09,0xd1,0x5a, +0x3b,0xd5,0xd0,0x90,0xd8,0xd7,0x89,0x18,0xef,0xfd,0xcf,0x57, +0xef,0xf2,0xa4,0xb9,0x54,0xfd,0xde,0xc6,0x62,0xc1,0xed,0x84, +0x58,0x9e,0xd7,0x40,0xef,0x19,0xbd,0x0e,0x6a,0x1c,0x63,0x39, +0x84,0x4b,0xf1,0xfd,0x5f,0x2c,0x45,0xe8,0x36,0x5c,0x8f,0x1a, +0x3b,0x8d,0xc0,0xd2,0xce,0xcc,0x02,0xf7,0x85,0x6b,0x62,0xa1, +0xaf,0xa9,0x9d,0x51,0x5f,0x86,0x4b,0xf3,0x23,0x8b,0x47,0x3f, +0xff,0x51,0x2c,0x13,0x41,0x0e,0xbc,0xfc,0x41,0x3c,0xc4,0x56, +0x2d,0x26,0xe2,0xba,0x7f,0x10,0x17,0xf0,0x31,0x8b,0x09,0x5e, +0xfe,0x20,0x1e,0x54,0x94,0x5b,0x44,0xf8,0xf6,0x07,0x31,0xb1, +0x7e,0xdc,0xe2,0xe2,0xf7,0x3f,0xda,0xe6,0xa0,0x16,0x77,0x8d, +0x3e,0x6c,0x18,0x03,0x35,0xcb,0xaf,0xc8,0xad,0x3b,0xdf,0x76, +0xf9,0x96,0x33,0xa8,0x92,0x34,0x5c,0x8d,0xcc,0xa9,0x7e,0xa7, +0x10,0x6c,0xe2,0xbe,0x98,0x2f,0xed,0xfe,0x49,0xbe,0xf4,0x27, +0xa6,0xf3,0xee,0x5f,0x34,0xec,0x15,0xf9,0xc1,0xff,0x1f,0x39, +0x5e,0xdd,0xbd,0x03,0xc9,0x22,0x38,0xe9,0x7f,0xd2,0x1c,0xf8, +0x72,0xce,0xfc,0xdd,0xe7,0x97,0xa3,0x7f,0x0f,0x57,0xe6,0x01, +0xf1,0xe7,0xa6,0xe5,0x6e,0xe3,0xb4,0x64,0xda,0xe3,0x02,0x36, +0x18,0xbf,0xbe,0xad,0x17,0x0a,0x49,0xc1,0x6f,0xe4,0x2b,0x0d, +0xab,0x45,0x56,0xf6,0x45,0x9d,0xd4,0x14,0x13,0xfb,0xb2,0x4a, +0xf7,0xeb,0xc2,0x9f,0x2a,0xb2,0x67,0x70,0x00,0x86,0x53,0xf3, +0x68,0xf2,0x76,0x3e,0xc3,0xd7,0xbe,0x2e,0x32,0x50,0x2b,0x76, +0x73,0x16,0x1c,0x46,0x60,0x1c,0x20,0x0f,0xb6,0x79,0xd0,0x02, +0xf9,0x14,0x5b,0xcd,0x44,0x6d,0x20,0x4b,0x89,0x15,0x72,0x22, +0x91,0x96,0x74,0x0d,0x2c,0x12,0x94,0x39,0x4c,0xd5,0xcd,0xaf, +0x81,0x4c,0x6a,0x43,0x64,0x0b,0xf0,0x30,0xec,0xf5,0xba,0x7b, +0x4c,0x4f,0x99,0x82,0x2e,0xf3,0xcc,0xee,0x0f,0xb0,0x3d,0x83, +0xf0,0xe0,0x4e,0x5a,0x67,0x83,0x26,0xa3,0xb4,0x77,0x1b,0x49, +0xc6,0x2a,0x22,0xba,0x37,0xd9,0x65,0xff,0x11,0xac,0x68,0x8e, +0xe1,0x62,0xc4,0x73,0x87,0xc4,0xd1,0xf3,0xa3,0xe1,0x85,0xbb, +0x54,0x06,0xd5,0x6e,0x6d,0xb1,0x71,0x04,0xbc,0x34,0xcf,0x18, +0x00,0x62,0xab,0x07,0xdc,0x04,0xbb,0x16,0xf0,0xe6,0x0e,0xaa, +0x00,0xea,0x46,0x73,0x38,0xcc,0xea,0x74,0x2a,0x13,0xb7,0x87, +0xb3,0x37,0xbc,0x38,0x3c,0x6c,0x78,0xfa,0x41,0x47,0x43,0x74, +0xfa,0x04,0x7f,0x5f,0x40,0xba,0xc6,0x06,0xcf,0x8b,0x05,0x55, +0x74,0x97,0xf4,0xae,0xf8,0x26,0xeb,0x6a,0x7b,0x74,0x8a,0xdc, +0xe8,0x9a,0x5e,0xd4,0x26,0x2d,0xb9,0xd1,0x04,0xc5,0x7e,0x37, +0x46,0xfd,0x17,0x26,0x20,0x86,0xfc,0xa9,0x34,0xe8,0x60,0x4b, +0x0e,0xa6,0xc2,0xf4,0x91,0x7d,0xbf,0x11,0xc6,0x35,0xab,0x3f, +0x07,0xfb,0x46,0xd8,0xde,0x18,0xf6,0x16,0x37,0x47,0xf6,0x62, +0xef,0x14,0x30,0xac,0xdc,0x28,0xc1,0xc8,0xf6,0x38,0x14,0xef, +0x1a,0xd5,0x61,0x23,0xd4,0x0c,0x40,0x3d,0x5b,0x20,0x5c,0x45, +0x3e,0xc2,0x3a,0x4e,0xda,0xd7,0xc3,0x9b,0x20,0xec,0xbe,0xaf, +0xbf,0xb8,0xab,0xcb,0x6a,0xd7,0xfe,0x5e,0xcd,0xb6,0xeb,0x2e, +0x4c,0xe1,0xfe,0x8f,0x54,0x26,0xeb,0x8d,0x18,0x54,0xcd,0xa4, +0x8d,0x95,0x12,0x2e,0xec,0x31,0x84,0xd4,0x63,0x20,0x6c,0x06, +0x56,0x06,0x61,0x4b,0x78,0x79,0xc9,0xdd,0x34,0x34,0x36,0xc1, +0x6a,0x9d,0x55,0xa3,0xf4,0x47,0x31,0x49,0x25,0x11,0x6b,0xb6, +0x43,0x82,0x6d,0xf0,0x07,0x6b,0xe5,0xa6,0xf2,0xeb,0xc5,0x99, +0x5c,0xcf,0xb1,0xcf,0xea,0x8d,0xfb,0x1d,0xc4,0x66,0xa6,0x77, +0x74,0xe0,0x9f,0xe4,0xaf,0xe5,0xec,0xd0,0x97,0x2f,0xaa,0x5a, +0x1c,0xec,0x5d,0x9e,0x9b,0xe0,0x31,0xe1,0x8b,0x38,0x9c,0xa6, +0xe3,0x6d,0x25,0xd2,0x39,0x45,0xe1,0x22,0xce,0xb5,0xd6,0x28, +0xb4,0xc7,0xf9,0x77,0x30,0x46,0x54,0xf5,0x9c,0xfe,0x02,0x37, +0x75,0x08,0x33,0xad,0x0a,0x9c,0x7c,0x07,0x3f,0xe9,0x82,0xc5, +0xe7,0x1b,0xb4,0xdd,0x8c,0xb9,0x04,0xda,0xac,0x33,0xce,0x8a, +0x36,0xd2,0xc0,0x90,0x51,0xdc,0x7a,0xac,0x95,0xcb,0x34,0x2f, +0x6c,0x3e,0x73,0x2d,0x05,0x73,0x55,0x93,0x0f,0xa0,0xa7,0x07, +0x47,0x67,0xd9,0x17,0xf1,0x37,0xdc,0xd8,0xfc,0xe9,0x55,0x1b, +0x0d,0x4d,0x7a,0x56,0x15,0x68,0x4b,0xae,0xb1,0xf6,0xfc,0x25, +0xb3,0xf8,0xdf,0xcd,0xe9,0xbe,0x7c,0x86,0xd7,0xb2,0xc1,0x75, +0xd0,0x18,0x45,0x37,0xc5,0x45,0x5c,0xd5,0xa9,0x7f,0x9e,0x13, +0x04,0xaa,0x62,0xd7,0xe1,0xc2,0x98,0xd4,0xf3,0xa9,0x14,0x2c, +0xce,0x10,0xf9,0xd2,0xb8,0x2a,0x70,0x68,0xec,0x09,0x75,0xe8, +0x7f,0xc1,0x2d,0xc9,0x46,0x16,0xa8,0x29,0x6e,0x26,0x95,0x13, +0x7e,0x48,0x59,0x5e,0x4b,0x92,0xb4,0xc0,0xbd,0x8d,0xfe,0xb9, +0xfd,0xc3,0xd1,0xd1,0xdb,0x6d,0xaa,0x6d,0xdb,0x60,0x8e,0xec, +0xcd,0xf7,0x4c,0x8c,0xcc,0x9e,0x31,0x5f,0x4c,0x6d,0xd2,0x32, +0x11,0xa7,0xeb,0xa1,0x49,0x6c,0x0d,0xa5,0xbd,0x3b,0x24,0x19, +0x49,0x5c,0x9d,0x14,0x83,0x26,0xc2,0xbe,0xfe,0xb9,0xef,0xbd, +0x0e,0x74,0x93,0xb3,0xcc,0xe2,0xa6,0xb7,0x3f,0x2e,0x92,0xf4, +0xdf,0x4f,0xc4,0xff,0x0f,0x48,0x30,0xc5,0x58,0xbd,0x72,0x17, +0x45,0x6b,0x2e,0x70,0xfe,0xde,0xf3,0x85,0xe7,0xc9,0xa6,0xf1, +0x38,0x42,0xae,0x83,0x7c,0x18,0x41,0xe7,0x38,0x34,0x71,0xcd, +0x5a,0xc8,0x6f,0xd5,0x7d,0xa8,0xaf,0x18,0xfc,0x03,0x95,0x19, +0x37,0x0e,0x6b,0xd6,0x27,0x34,0xa3,0x7f,0xa0,0x32,0x3c,0xae, +0x5a,0xb7,0x26,0xa1,0x45,0xa2,0x8d,0x0f,0x1f,0xf1,0x7c,0x71, +0xa5,0x38,0xe1,0xa5,0x64,0x81,0x88,0xaa,0x35,0x4a,0x55,0xd3, +0x1f,0xf8,0x38,0x72,0x81,0xb1,0xe6,0xd7,0x79,0xda,0xa8,0x3f, +0x50,0x19,0xbb,0xd8,0x58,0xb3,0xb6,0x40,0xb7,0xf4,0x07,0xea, +0xb3,0x2e,0x3c,0xd6,0xed,0x3d,0xa9,0x81,0x8a,0xff,0x68,0xcf, +0x91,0x9f,0x10,0xbf,0xca,0x86,0xce,0xf3,0xcf,0xf5,0xd6,0x39, +0xd1,0xbb,0x9b,0x80,0x7a,0x24,0x0d,0x04,0xac,0xb2,0xf2,0xa3, +0x20,0x65,0x01,0x3a,0xbe,0x46,0xc1,0x7b,0xf9,0xc0,0xff,0x11, +0xbf,0x3a,0x75,0x0e,0xd0,0x98,0x5b,0x84,0x5d,0xff,0xd5,0x6d, +0xe9,0xbd,0x45,0xa9,0x56,0xff,0x46,0xfb,0x03,0x4d,0x00,0x38, +0xbf,0x6a,0x3a,0x12,0x0d,0x68,0x25,0xb1,0x93,0x4c,0x3c,0x5d, +0xa5,0xe7,0xf5,0xc3,0x49,0x18,0x6b,0xbb,0x0b,0x29,0xf2,0x6e, +0xc9,0xed,0x4a,0xa3,0x74,0x2c,0x4e,0x77,0xe5,0x6c,0x84,0x62, +0xbf,0xa9,0x4c,0x81,0xa0,0xc9,0xb0,0x9c,0x68,0x37,0xa7,0x23, +0xf0,0x1b,0x4b,0x5c,0x93,0xb1,0xfe,0x1c,0x5d,0xca,0x65,0xe5, +0x06,0x4e,0x9b,0xce,0x7f,0xbb,0x28,0xd9,0xd1,0xf9,0x6f,0xe3, +0xba,0x1e,0x34,0x91,0x1f,0x8d,0xb6,0x69,0x06,0x04,0x85,0x6a, +0x9b,0x53,0xc4,0xdb,0x05,0x9b,0x7f,0xcf,0xba,0xf6,0x85,0x7f, +0xbd,0xca,0x79,0xc8,0x1c,0x63,0xfc,0x4d,0xb8,0x5d,0xe5,0x9d, +0xf8,0xd6,0xd9,0xfc,0x9a,0xa6,0x0a,0x8d,0x84,0x2d,0x0a,0x65, +0x9f,0x12,0xe1,0x89,0x33,0x96,0xa9,0x2b,0x2d,0x80,0x1d,0x20, +0x39,0x8b,0x24,0xb0,0x0c,0xdd,0x20,0x9b,0xf5,0x1e,0x6f,0x19, +0x34,0x59,0x39,0xf7,0x2b,0x19,0x83,0x1a,0x1d,0x90,0xbd,0x74, +0x66,0x4c,0x8e,0x55,0xf8,0x45,0x5c,0xb3,0x71,0x42,0x6a,0x34, +0xc7,0xd8,0x34,0x6b,0x98,0xd8,0x1a,0x45,0xb3,0x2d,0xe7,0xba, +0xc3,0x94,0x6e,0xec,0xa0,0x6a,0xe1,0xc0,0x40,0xd7,0xd0,0x46, +0x9a,0x71,0x03,0x14,0x98,0xe9,0xd6,0x03,0x35,0x18,0xeb,0xd6, +0x03,0xd7,0x9a,0xec,0x36,0x56,0x5e,0x35,0xdc,0xad,0x07,0xad, +0x6e,0x59,0xc8,0x88,0x16,0x4b,0x56,0x33,0xab,0xad,0xd1,0xa4, +0xd5,0xa8,0xc7,0x21,0x8c,0xd5,0x42,0x4c,0x4b,0x6f,0x66,0x35, +0x0d,0xc9,0xba,0x79,0xb0,0x1c,0x80,0x27,0xdd,0x8a,0xd3,0x5c, +0x88,0xd5,0x97,0xb1,0x66,0x25,0xff,0xad,0x98,0xcc,0xf1,0xba, +0xe6,0xb4,0x3c,0x83,0x80,0x11,0x90,0xd9,0x3e,0x51,0xa7,0xba, +0xb2,0x0f,0xe8,0x00,0x57,0xa1,0x8a,0xfc,0x2d,0x58,0xa0,0x85, +0xd6,0x68,0x4a,0xef,0xc8,0x29,0x83,0x76,0xff,0x22,0x07,0x46, +0x09,0x65,0x99,0xf1,0x22,0x32,0xd1,0xee,0xce,0xa1,0x45,0x3b, +0x3c,0x85,0x96,0x6d,0x32,0x15,0x13,0x60,0x64,0x9f,0x0d,0x67, +0x32,0x9d,0x52,0x94,0x39,0xdb,0xc8,0x0c,0x11,0xf6,0xb4,0x43, +0x5d,0x0e,0xcb,0x37,0x1f,0xc7,0x6f,0xc9,0x2b,0xd2,0x67,0x07, +0xe0,0xa7,0xab,0xf1,0x10,0xa2,0xb1,0xbe,0xe0,0x68,0xb6,0xa2, +0x06,0x13,0xd0,0x96,0x00,0x0e,0x86,0x37,0x32,0x13,0x7c,0x88, +0x53,0xc6,0x3b,0xbc,0xf7,0x23,0xf3,0xe8,0x26,0x90,0x97,0xfd, +0x0e,0x5c,0x91,0x57,0x61,0x30,0xd9,0x50,0x80,0x0e,0xb8,0x3d, +0x02,0x30,0x85,0xb3,0x9f,0xeb,0xc1,0xff,0x59,0xe6,0xc2,0x6c, +0x30,0xd4,0x1f,0x4e,0xae,0xfd,0x46,0xd3,0xef,0x9c,0xf5,0x92, +0x62,0x17,0xc9,0x5c,0x0e,0x67,0xc4,0x00,0x10,0x82,0xb8,0x06, +0xe8,0xca,0x25,0x1b,0x4c,0x7c,0x4d,0x95,0xba,0x9f,0x5d,0x1f, +0x70,0xde,0xff,0xce,0x3f,0x97,0xa6,0xfb,0xc1,0x51,0x31,0x27, +0x7f,0xa7,0xab,0xca,0xec,0x20,0xc0,0x11,0xaf,0x4e,0xea,0xfd, +0x8a,0xea,0xe7,0x8d,0x62,0x0c,0x93,0xe1,0x2c,0x9f,0x9c,0x6f, +0x9c,0xf4,0x74,0x02,0xde,0x0b,0xf3,0x53,0xe9,0x7e,0xd5,0x09, +0xf8,0x69,0xbf,0xc9,0x47,0xa0,0xa9,0x32,0xfa,0x00,0xf0,0xf7, +0xba,0x71,0x82,0x38,0x97,0xdd,0xc8,0x86,0x6e,0x8d,0x36,0x33, +0x18,0x6d,0xba,0x70,0xfe,0xe9,0x66,0x32,0x9d,0x95,0xbd,0xb8, +0x2e,0x93,0x5c,0x43,0x6f,0x6d,0xb1,0x8b,0x68,0x86,0x85,0x2b, +0x39,0xf4,0x94,0xf9,0x19,0xd9,0x49,0xa2,0xf8,0xb1,0x7d,0xa2, +0x5f,0x52,0x9a,0x30,0xd9,0x89,0x3a,0x69,0xff,0xe3,0xf9,0xbb, +0xc3,0x97,0x6f,0x5e,0x67,0x51,0xa7,0xfd,0x55,0x7b,0x2f,0x42, +0x39,0x03,0xec,0xd4,0x32,0x32,0x57,0xd3,0x3f,0x26,0x74,0xb2, +0x6c,0x08,0x88,0xdc,0x38,0x1d,0xb2,0x2f,0x1c,0x0a,0x91,0x56, +0x9c,0xa3,0xc6,0x75,0x13,0xbe,0x32,0x81,0x17,0x39,0x52,0xb7, +0xb6,0x74,0xaa,0xc5,0x94,0x65,0x5e,0x66,0x22,0xf2,0xe2,0x0a, +0x62,0x1b,0xdd,0x0c,0xa0,0x38,0x22,0x5d,0x96,0xb5,0xdc,0x9b, +0xbb,0x78,0x5c,0x64,0xbb,0x8a,0x21,0x5c,0x76,0x97,0x7e,0xf6, +0x0b,0x8c,0x20,0xa9,0xf1,0x98,0x1a,0xd8,0x22,0x95,0xaa,0x51, +0x18,0x5a,0x72,0xa0,0x30,0x34,0x25,0xf8,0x56,0x64,0x66,0xc2, +0x17,0xcf,0xc8,0x3c,0xde,0xd4,0x03,0x63,0x26,0x63,0x1f,0xd7, +0x00,0xae,0x44,0xc5,0x90,0xf8,0xe5,0x35,0x43,0x29,0x88,0x69, +0xa9,0xf8,0xa1,0x81,0x8a,0x2e,0x1e,0x34,0x64,0xe0,0x89,0x03, +0x59,0xef,0x5d,0x7d,0xc2,0x7a,0x42,0xf4,0x8f,0xeb,0x8c,0xd5, +0xa1,0x9b,0x32,0x68,0xa1,0x1e,0xe7,0xa5,0x96,0x69,0x50,0x3f, +0x41,0x1d,0xa6,0x8f,0x74,0x62,0x8f,0xd3,0xaa,0x7d,0x93,0xc6, +0x68,0xd2,0x08,0x95,0x4b,0x6b,0xd7,0x79,0x4e,0xf1,0xc4,0xd4, +0x55,0x41,0x2e,0x54,0xb0,0x16,0xba,0xfb,0x5d,0xdf,0x06,0x95, +0x42,0xac,0xd6,0x86,0x72,0x7c,0x43,0x92,0x78,0x10,0x7a,0xce, +0xc9,0x22,0xe2,0x53,0x10,0xef,0x2b,0x07,0x67,0xde,0x1b,0x43, +0x98,0xdd,0x28,0x98,0x6d,0x8c,0x27,0x1b,0xec,0x45,0x8f,0x9c, +0x79,0x47,0x5d,0x73,0xdb,0x11,0xc7,0xf2,0xd5,0x08,0x7d,0x43, +0x53,0x98,0xea,0xfa,0xe6,0xbb,0xd6,0x3b,0x72,0xbf,0x0d,0x19, +0x61,0xc5,0xe9,0xc6,0xfe,0x5e,0x97,0x87,0x3d,0xb6,0xa7,0x9e, +0x80,0x10,0xd8,0x14,0x1c,0x5b,0x48,0x76,0x4a,0x8d,0xca,0x34, +0x98,0x76,0xa5,0xac,0xc4,0x99,0xad,0x41,0x6d,0xe0,0xf3,0x8e, +0xa9,0x30,0x4b,0x6e,0xa5,0xcd,0x21,0xfc,0x08,0x52,0xd3,0x73, +0x58,0x63,0x8f,0xf6,0x24,0xad,0xc9,0xa1,0x0b,0x98,0x6b,0xf4, +0x96,0xc1,0x0e,0x98,0xb2,0xfa,0xde,0xc2,0x06,0xaa,0x94,0x4c, +0x63,0x2c,0x62,0xd2,0xa9,0xc5,0xe0,0xcc,0x01,0xc2,0x72,0xd1, +0xab,0xf0,0x0f,0x0a,0xae,0x71,0xc8,0x47,0xa8,0xe8,0x58,0x2b, +0x05,0x03,0xa6,0xa5,0x3a,0x91,0x0b,0x0f,0xf7,0xde,0xf4,0xbf, +0xa9,0xd3,0xb0,0x4e,0xd9,0x73,0x44,0x84,0xdf,0x7d,0x98,0xf6, +0x17,0xf5,0x21,0xe2,0x6a,0xee,0x48,0x3f,0x9b,0xf6,0x83,0x55, +0x5e,0xa7,0x3f,0x8c,0x9e,0x36,0x35,0x8b,0x34,0xec,0xb1,0x8e, +0x65,0x75,0xab,0xec,0x6a,0xb9,0xf6,0x00,0x81,0x50,0x83,0x50, +0xb0,0x87,0xdc,0x6a,0x7b,0x9b,0xca,0x0f,0x52,0xf3,0xa4,0xd6, +0x1d,0x43,0xc8,0x74,0x01,0xdd,0xa0,0x66,0x20,0x79,0x99,0x7f, +0xc9,0x68,0x02,0x27,0xbb,0xb8,0x0f,0x9e,0x7d,0x11,0xef,0xb4, +0x9e,0xee,0xc6,0x9f,0xef,0x66,0x75,0xa4,0x07,0x5f,0x8f,0xc9, +0xf5,0x62,0xaa,0x80,0xe7,0x0c,0x04,0x4d,0x27,0xf1,0xc0,0x32, +0x3f,0xa2,0x1a,0x45,0xad,0x93,0x36,0xb9,0xd0,0xfd,0x6b,0x79, +0x3e,0xc9,0x71,0xbc,0x3c,0x53,0x45,0x76,0x75,0xc6,0xd7,0x9e, +0xcb,0xf9,0xa3,0xfc,0x7f,0xcd,0xd6,0xd8,0xda,0xf2,0x16,0x0a, +0x1b,0x99,0xd7,0x2e,0x03,0xc8,0x0f,0x56,0xf0,0x6d,0xdb,0x04, +0x56,0xaf,0xc4,0x74,0xaf,0xd5,0x69,0x9b,0x6b,0x92,0xb9,0xb4, +0xf5,0x00,0x41,0x28,0xd7,0xea,0xbe,0x81,0x2d,0xe2,0x5d,0x02, +0x12,0xa7,0x2c,0x16,0x92,0x41,0xc0,0x79,0xca,0xac,0x98,0x7d, +0x36,0x07,0x8b,0x38,0x1e,0x5c,0x64,0xeb,0xb0,0xdf,0x52,0x21, +0x5f,0x53,0xaf,0x11,0x05,0xa6,0xd3,0x30,0xa3,0x67,0xd3,0xff, +0xf8,0x92,0x0d,0xb5,0x9b,0xe1,0x49,0x3f,0x5b,0x5b,0xeb,0xae, +0xdd,0x3d,0x1a,0xc5,0x29,0x4b,0x2f,0x30,0x65,0x37,0x09,0x87, +0x09,0x0f,0x31,0x86,0x36,0x03,0xb9,0x1f,0x9a,0x6e,0xfc,0xf9, +0xaf,0x6c,0xba,0xce,0x1d,0x4d,0x07,0xbb,0x0f,0x6a,0x39,0xac, +0xdf,0x34,0x1c,0x24,0xf7,0x4c,0xea,0x9f,0x6b,0x36,0x26,0x45, +0x13,0xcb,0x34,0xad,0xdd,0x6e,0xdc,0x60,0x8e,0x0b,0x88,0x76, +0x73,0xee,0xd7,0x50,0x7e,0x39,0xbb,0x9a,0x8f,0x72,0xbf,0xdd, +0x4c,0xa8,0xa1,0xca,0xf7,0x6f,0x76,0x52,0x6f,0x7b,0x45,0x9f, +0xcf,0x1b,0x2b,0xdb,0x02,0x9c,0xd9,0xdb,0xee,0xa0,0x0c,0x6f, +0xe3,0xc8,0x32,0xda,0xb4,0x9e,0xfd,0xf9,0x51,0x4a,0xb3,0x8c, +0x43,0x0a,0x2d,0x71,0x76,0x14,0xe3,0xdf,0x26,0x1f,0x02,0x2a, +0xe5,0xd5,0xb3,0x21,0x5c,0x31,0x2b,0x1b,0x6e,0x40,0x16,0xa8, +0x2e,0x47,0xd7,0x4c,0x56,0xc1,0xc6,0x85,0x9d,0x1f,0x0b,0x10, +0x4b,0x9b,0x89,0x8a,0xa9,0x28,0x87,0x14,0xa0,0x10,0xfa,0xe6, +0xba,0x59,0xe2,0xae,0x9a,0xe9,0x6e,0x00,0x4a,0x12,0x22,0xfa, +0xe6,0x6a,0x2e,0x0f,0x68,0x01,0x3b,0xc6,0xac,0x59,0xb3,0x4e, +0x6a,0x08,0x8a,0xa2,0x8f,0xf8,0x3e,0x5e,0xe6,0xd3,0xa0,0x0d, +0x38,0x70,0x68,0x31,0x2d,0x67,0x9e,0x67,0x33,0x8a,0x4c,0x83, +0xb9,0x49,0x0f,0xb3,0xcd,0x54,0xea,0x0f,0xd2,0x93,0x3e,0xa5, +0x44,0xb0,0x8c,0x45,0x69,0x44,0xcc,0x2d,0x1a,0xd4,0xd2,0x22, +0xb6,0x2a,0x5a,0x10,0xa2,0x60,0xed,0xb0,0x05,0xa1,0x50,0xe8, +0x40,0x9c,0x96,0x10,0x1c,0xa5,0x2c,0xc3,0xd8,0xc9,0xb3,0xb9, +0x4b,0xb4,0x43,0x55,0x3f,0x36,0xd0,0x2f,0x28,0xc7,0x4f,0x14, +0x5f,0x06,0xeb,0xb4,0x82,0x86,0xfa,0xb4,0xc6,0xae,0xd1,0x32, +0x5e,0x70,0x23,0x66,0x7c,0x14,0x4e,0xd0,0x1e,0x03,0x44,0x80, +0xdd,0x01,0x6f,0xf7,0xf4,0x13,0x0d,0x5c,0x12,0x32,0x1e,0x7f, +0xfd,0xd5,0x57,0x0f,0xbf,0x32,0xc2,0xd7,0x01,0xb8,0x84,0xd0, +0xf5,0x71,0xa7,0xc2,0x2b,0xee,0xa1,0xba,0xd5,0x0a,0xa8,0x91, +0x21,0x93,0x0b,0x6f,0x77,0x76,0x76,0x25,0x0b,0xb9,0x85,0x58, +0x6d,0xf3,0x59,0x3e,0x4a,0x21,0x87,0xe6,0x29,0x3e,0xda,0x9b, +0xab,0xeb,0xf0,0x02,0xba,0xf5,0x43,0x88,0x2c,0x07,0xe8,0xad, +0xc9,0x0a,0x68,0xa8,0x76,0x4d,0xf9,0x7d,0xa2,0xac,0x6d,0xde, +0xb5,0x8c,0x69,0x68,0x25,0xea,0x08,0x81,0xa5,0xdb,0x3c,0x2c, +0xdd,0xfa,0x49,0x97,0xe6,0xb0,0x57,0x8a,0xf1,0x7f,0x6f,0xaf, +0x14,0xe3,0x3f,0xd2,0x2b,0x1b,0x8d,0xbd,0xe2,0x3a,0xe5,0xff, +0x66,0x9f,0x3c,0xfe,0x2b,0xfb,0xa4,0xbc,0x9c,0x9f,0x9f,0x5f, +0xe5,0x81,0xb2,0x09,0xbf,0x7e,0x38,0x1e,0x11,0x15,0xd9,0xae, +0x62,0xb0,0x91,0x10,0xec,0x1a,0x3e,0x5b,0x4f,0xce,0x21,0x0a, +0xbb,0xf0,0x33,0xb9,0xe6,0x68,0xec,0xad,0xc4,0xa2,0x20,0xd1, +0x7a,0xbb,0x33,0xc8,0x6c,0x0a,0x80,0x0e,0x94,0xff,0x4a,0x3c, +0x62,0xe9,0xca,0x21,0xb5,0xc3,0xeb,0x9b,0x80,0x58,0x35,0x56, +0x17,0xf3,0xe1,0xd4,0x79,0x5b,0x0f,0x77,0x5d,0x8f,0xf7,0x16, +0x0b,0x84,0x80,0x45,0xb6,0x6f,0xe9,0x72,0x23,0x67,0xbb,0x93, +0x68,0x1e,0x67,0x5a,0x02,0x1b,0x80,0xd4,0xa9,0xf1,0xae,0x32, +0xb3,0x1b,0x82,0x27,0x99,0x10,0x53,0x57,0x93,0xc9,0x87,0xf9, +0xcd,0x4b,0x23,0x0e,0x34,0x30,0x63,0x6f,0xf5,0xa0,0xac,0x1e, +0x75,0x4f,0x93,0x5e,0x8f,0xe2,0x87,0x0d,0xfe,0x63,0x09,0x4b, +0x6e,0x39,0x99,0xce,0xbe,0x0b,0xc4,0x13,0xd3,0xaf,0x72,0xab, +0x69,0xa8,0xf0,0x89,0xe2,0x0a,0xdd,0x6a,0x85,0x4b,0x4b,0xdc, +0xb4,0x82,0xd4,0x88,0x12,0xde,0x50,0xc2,0xec,0x94,0x80,0xce, +0x30,0x1c,0x5d,0x31,0x4c,0xd7,0x95,0x5a,0x97,0x09,0x7e,0x8c, +0xb3,0x96,0xbd,0xca,0xcf,0x67,0x6a,0x8a,0x1b,0x58,0x5a,0x8b, +0x33,0x48,0x69,0x1b,0xc4,0xea,0x34,0xc3,0x4c,0x9b,0x00,0x73, +0x76,0xa8,0x17,0x8d,0x53,0x64,0x0f,0xc3,0xfd,0xd3,0xc5,0x82, +0xd6,0x26,0x70,0x46,0x6a,0xe7,0x2c,0x40,0x9d,0xee,0x0f,0x45, +0xde,0xa9,0xe5,0xb2,0xe6,0x50,0x00,0xeb,0xa1,0xe1,0x47,0x55, +0xe0,0xb3,0x1e,0x63,0x11,0x29,0x6c,0xb8,0x83,0x31,0x42,0xab, +0x6b,0xfa,0xd3,0xfc,0x72,0xf8,0x5b,0x21,0xdc,0x22,0xdf,0xd5, +0x29,0x66,0x36,0x2e,0x2d,0x83,0xcb,0x48,0x24,0x42,0xd8,0x9e, +0x93,0x1f,0xd3,0x95,0xbd,0xb6,0x92,0xaf,0xd8,0xcd,0xf5,0x1a, +0x32,0x1e,0x32,0x3e,0xf3,0x15,0xcc,0x2b,0x40,0xb2,0x50,0xc1, +0x4e,0x07,0x07,0x1e,0x7e,0xbc,0x1e,0x79,0xf8,0xeb,0x99,0x38, +0xfb,0xa5,0x6e,0x63,0x8a,0xcb,0xe2,0x32,0x12,0x16,0x77,0x71, +0x5d,0x4f,0xc5,0x33,0xa8,0x38,0x82,0xad,0xd5,0x09,0x35,0xfd, +0x3a,0xf5,0x48,0x44,0x86,0x27,0x80,0x48,0x3a,0x1f,0xcf,0x56, +0x15,0x87,0x6b,0xcf,0x2b,0xe8,0x6b,0xb5,0x3c,0x0a,0x3b,0x88, +0x14,0xc6,0x69,0x3e,0x42,0x71,0xd4,0x75,0x3f,0x6a,0xf8,0xd4, +0xea,0xed,0x01,0xf7,0xaf,0x65,0xec,0xcd,0x5d,0xec,0xf4,0x28, +0x46,0x50,0x22,0xf1,0xb5,0x51,0x43,0x0b,0xf7,0x6e,0x3e,0x6a, +0x0e,0x7c,0xa9,0x07,0x6c,0x86,0xb4,0x18,0x95,0x0b,0xa4,0xec, +0xeb,0xcc,0x2e,0x07,0xc9,0x2a,0x46,0x9a,0x0b,0x7c,0x6c,0x61, +0xf2,0xfe,0xbe,0x9e,0x10,0xb5,0x28,0x11,0x45,0x5f,0x03,0x0f, +0x92,0xc7,0x34,0x1e,0x01,0xbf,0x7e,0x6f,0x75,0x52,0xac,0x43, +0x3f,0xda,0xf9,0x32,0xf9,0x08,0x5c,0x77,0x36,0x09,0x1c,0x6f, +0x06,0x8c,0xab,0xe7,0xaf,0xd3,0x3d,0x21,0x57,0xc3,0x7b,0xda, +0xa4,0x3f,0xef,0x39,0x7e,0x24,0xf6,0x54,0xa9,0xbd,0x66,0x8f, +0x85,0xfb,0x03,0x64,0xfc,0x5e,0x88,0x74,0x59,0xbf,0xdb,0x6c, +0xec,0x36,0x57,0xe4,0x1e,0x53,0xa1,0x8a,0x32,0xea,0x27,0x14, +0x37,0xb5,0x80,0xab,0x17,0x2d,0x38,0x79,0xd7,0x3f,0xb3,0xa1, +0xdc,0x3c,0xd0,0x08,0x08,0xd7,0x19,0xaa,0x18,0xf3,0x8c,0xac, +0x4c,0x49,0x63,0xb3,0xde,0x50,0x53,0xef,0x0e,0x52,0x6f,0xa3, +0x01,0xb8,0x60,0x35,0xa1,0x6d,0x0a,0xa9,0x97,0xee,0xa8,0xaa, +0xa6,0xbc,0x1c,0x09,0xdb,0x71,0x50,0xb1,0xde,0x75,0x25,0x88, +0x1f,0x4e,0xa7,0xfe,0xaa,0xef,0xf0,0x6a,0xec,0xd4,0x7c,0x94, +0x5d,0x2b,0x3d,0xc8,0xb1,0xda,0x4d,0x58,0x83,0x82,0x2d,0xac, +0x37,0x92,0xa0,0x07,0x1f,0x4d,0x27,0x37,0x5f,0xfc,0xd9,0x35, +0x9f,0xb9,0x44,0x5e,0x80,0x11,0x8e,0x03,0x74,0x55,0x6d,0x0c, +0x21,0x11,0x43,0x8d,0xd8,0xfd,0xf9,0x15,0x84,0x75,0x1c,0x4b, +0x57,0xf8,0x5a,0x5e,0x82,0x98,0xc5,0x57,0x7a,0x02,0xa8,0xc9, +0x7c,0xa6,0x5f,0x1d,0x51,0x94,0x8c,0x9e,0x87,0x49,0xd1,0x81, +0xd0,0x76,0x0a,0xa0,0x77,0xf3,0xb3,0xe1,0x8c,0x25,0x4f,0x2a, +0xad,0x10,0xc8,0xec,0xe5,0xa9,0x44,0x28,0x37,0x04,0x6e,0xd2, +0x93,0xc5,0x82,0x52,0x58,0x9a,0x31,0x32,0x04,0xd7,0xdf,0x03, +0x7e,0xea,0xd7,0x41,0x00,0x29,0x7f,0x0e,0x2f,0x16,0xc1,0x57, +0xa4,0xf4,0x1b,0x70,0x63,0xce,0x04,0xf6,0x5f,0x69,0x0c,0x6a, +0x34,0x46,0xe3,0x96,0x40,0xae,0xc4,0xcb,0x85,0xe0,0xb1,0xb8, +0xef,0x2c,0xf4,0x3e,0x77,0xbe,0xa2,0x3f,0x46,0xc5,0xf9,0xb9, +0xde,0xb7,0x8d,0xb5,0x88,0xc5,0x08,0xea,0x76,0xe2,0x1d,0x1a, +0x37,0xf3,0x71,0xf1,0x2b,0xf9,0xe3,0xfa,0x75,0x5e,0x99,0x94, +0x45,0x79,0x88,0x3c,0xbb,0x86,0x37,0x7b,0x02,0x98,0x81,0x63, +0xf7,0xf0,0x00,0x61,0x59,0xae,0x5b,0xa5,0x1d,0x36,0x7e,0xc8, +0x36,0x3b,0xa4,0xe3,0x31,0x13,0x55,0x70,0x78,0x60,0x5d,0x4c, +0x44,0x45,0x67,0x43,0xe9,0x4e,0x19,0xaa,0xca,0x3c,0x1f,0x87, +0xb2,0x33,0xe3,0x6c,0x58,0xde,0x2d,0xc9,0x3d,0x4c,0xd8,0xda, +0x02,0x14,0x7d,0xf8,0xe3,0xe6,0x60,0xc6,0x22,0x45,0x2a,0x43, +0xa5,0x6a,0x08,0x1e,0x0a,0x7a,0x08,0xe1,0xab,0xec,0x6d,0xe5, +0xa9,0x3a,0x69,0x66,0x93,0xae,0x5b,0x9e,0x7c,0x71,0xbb,0x4b, +0x4b,0x2f,0xd1,0x7d,0xd0,0x15,0xb1,0x1d,0x2a,0xa2,0xc7,0x36, +0x79,0xaa,0x63,0x7c,0x88,0x32,0xf4,0x6d,0xce,0xa3,0x80,0x25, +0xa4,0x59,0xbd,0xfa,0xa5,0x23,0x84,0x56,0x9e,0xb6,0x5c,0x21, +0x15,0x77,0xad,0xa5,0x5b,0x5d,0x5a,0xf2,0xd1,0x64,0x04,0xc4, +0xc2,0xfd,0xc9,0xec,0x32,0x9f,0x4a,0x89,0x9c,0x15,0x4b,0x98, +0xae,0x10,0xc1,0x7e,0xb6,0xbb,0x64,0xf5,0x8b,0x1b,0x95,0xcd, +0x34,0x7b,0x73,0xdb,0x99,0x01,0x34,0x0e,0xe0,0xca,0xc7,0xd0, +0xc8,0xa8,0xdf,0x33,0xc8,0x40,0x07,0xf8,0x21,0x66,0x82,0x02, +0x75,0xbf,0x17,0x37,0xb2,0x33,0x8c,0xf4,0xc0,0xcb,0x1d,0xaa, +0x3c,0x62,0x23,0xe7,0x3b,0x12,0x22,0xca,0x8f,0x12,0x36,0x04, +0x89,0x75,0x0f,0xd9,0x71,0x09,0xe7,0x17,0xc4,0x71,0xf8,0xc0, +0x06,0x8e,0x93,0xfd,0xb3,0x63,0x06,0xed,0x17,0x70,0xa5,0xb7, +0x82,0x3c,0x6a,0x15,0xd2,0x71,0x88,0x19,0x39,0x93,0x53,0x5f, +0x57,0x0e,0xfb,0x00,0x1b,0xb1,0xd0,0x9e,0x80,0xe1,0xee,0x80, +0x37,0x1b,0x4b,0x2b,0x0d,0x09,0xd9,0xd9,0x1d,0x6d,0x03,0x68, +0xed,0xd1,0x36,0x21,0x35,0x72,0x1d,0x80,0xc1,0x59,0x36,0x87, +0x42,0xd4,0x8f,0xa9,0x9f,0xa3,0xd7,0xe2,0x41,0x66,0x9e,0x3b, +0x03,0x9f,0xf4,0xa5,0xb2,0x03,0xa4,0xc2,0x65,0x66,0xe8,0xde, +0x9d,0x59,0x88,0xfb,0x04,0x1a,0x1d,0x66,0x97,0xd1,0x0d,0x2c, +0x01,0x3c,0x61,0x0d,0x0e,0xe1,0x25,0x82,0x68,0x3c,0xbf,0x3e, +0xcd,0xa7,0xce,0xb2,0xc2,0xe6,0x9a,0x93,0x9c,0xcc,0x93,0x47, +0x05,0x25,0x09,0xcb,0x70,0x05,0xcc,0x7f,0x48,0xe8,0x15,0xa9, +0xae,0x5d,0xd7,0xbc,0x6f,0x79,0x86,0xd8,0xb7,0x12,0x01,0x2d, +0x8b,0x3f,0x35,0x4f,0x4b,0x6b,0x2b,0x61,0xb5,0xb3,0x44,0x72, +0x83,0x7e,0x36,0xb1,0x1b,0x6c,0x01,0x14,0xfb,0x6d,0x83,0xfd, +0x58,0x31,0x3f,0x08,0xe8,0xb5,0x9f,0xd8,0xb5,0x1b,0x34,0x96, +0x55,0x5e,0xae,0x68,0x7f,0xf0,0x56,0xb3,0xba,0xed,0xb5,0xb8, +0x4f,0xb2,0x3b,0x99,0x83,0x40,0x81,0xae,0xfd,0x46,0x61,0xbf, +0x63,0xbe,0x53,0xd6,0x68,0xbe,0x55,0x80,0x19,0x32,0x0d,0xda, +0x5e,0xa5,0x54,0xec,0x08,0x4b,0xeb,0x33,0x9d,0x98,0x5f,0x64, +0x16,0x0f,0x14,0x48,0xfd,0xd1,0xb1,0xbd,0xdd,0x5d,0xbb,0xa1, +0xa6,0x10,0x72,0x40,0xc6,0xac,0x1e,0x4e,0xb5,0xb4,0x32,0x9b, +0xdc,0xe8,0x3f,0xf9,0x4d,0x72,0x5b,0x51,0x80,0x64,0x1d,0xbd, +0xea,0x01,0x40,0x86,0xb0,0x8b,0xc5,0xae,0xc2,0x87,0x6c,0x37, +0xc1,0x22,0xee,0xa0,0xba,0xbf,0x37,0x58,0x2c,0x3a,0xdd,0x80, +0xb9,0xd8,0xc1,0x84,0x0f,0x67,0x79,0x71,0x15,0x23,0xba,0x6d, +0xc4,0x92,0xec,0x60,0xb5,0x5a,0xcc,0x53,0xc5,0x08,0x74,0x45, +0x44,0x5f,0x85,0xb9,0xd8,0x51,0x31,0xfa,0xd4,0x4d,0x10,0x06, +0xee,0x9c,0xb7,0x5a,0x83,0xcc,0x7c,0x81,0xfe,0xdb,0xca,0x00, +0x99,0x9d,0x98,0x00,0x45,0x42,0xda,0x59,0x25,0xf4,0x00,0x1d, +0x71,0xcb,0x78,0x07,0xe3,0x33,0x7f,0x1f,0x0e,0xaa,0x75,0x75, +0xaa,0x37,0x8a,0x23,0x37,0x0c,0xc0,0xd4,0x6a,0x6b,0x0b,0x60, +0xa9,0xb8,0xe9,0x77,0x48,0x37,0xcd,0xed,0x52,0x98,0xd1,0x23, +0xea,0x46,0x16,0x4f,0x31,0x4a,0x85,0xb0,0x01,0xe0,0x49,0xdd, +0x19,0x71,0xd7,0xa9,0xa8,0x9a,0x4f,0x1f,0x90,0x60,0xf9,0xa9, +0x80,0x3f,0x0e,0x4d,0xbc,0x10,0xca,0x6a,0x2c,0xf1,0x7b,0x88, +0x56,0xb7,0xcf,0xbb,0x28,0x0d,0xd3,0xaf,0xab,0x2a,0xd1,0x94, +0x9f,0x79,0x61,0x0a,0xb0,0x4e,0xf7,0xda,0x25,0x63,0xdd,0xab, +0x73,0xb2,0xcd,0xd5,0xa0,0x21,0x3c,0xba,0xe1,0x14,0x4c,0x5b, +0x50,0x01,0xe5,0xd6,0x22,0x81,0x3f,0x80,0x2f,0x44,0xf2,0x99, +0x93,0xee,0x15,0x7a,0x32,0x07,0x68,0x80,0x6e,0x89,0x47,0x24, +0x7a,0x8c,0x78,0x1b,0x26,0x6c,0xeb,0x3b,0x8e,0x74,0x9c,0x4c, +0x51,0x95,0x66,0x04,0xcd,0x68,0xe4,0xb9,0x06,0xcd,0x48,0x08, +0x0c,0x9e,0x27,0xc1,0xd1,0x2b,0xd1,0x01,0x29,0xcd,0x84,0xe8, +0xce,0x04,0xdd,0x14,0x02,0x19,0x33,0xb2,0x4a,0xec,0x5a,0x46, +0xbe,0x71,0x3d,0x2f,0x67,0x1b,0xa7,0xf9,0xc6,0xcd,0x10,0x2e, +0xf4,0x5a,0xea,0x37,0xc0,0x14,0xbd,0x8c,0x7c,0x45,0x11,0x62, +0x74,0x82,0xc5,0x79,0x72,0x0b,0x2a,0xc5,0x73,0xe7,0x64,0x04, +0xdf,0x50,0xa3,0x00,0x12,0xff,0xe9,0x2f,0xa8,0x91,0xcf,0xaf, +0x27,0xde,0xce,0x1a,0x47,0xba,0x66,0x54,0x97,0xc6,0x90,0x1c, +0x0d,0x22,0x9c,0xe7,0x4c,0xca,0xd2,0x72,0x25,0x3d,0xc8,0xb3, +0xd0,0xd0,0x70,0x16,0x74,0x53,0x04,0x15,0xb4,0x2f,0x37,0xa4, +0x13,0x8d,0x40,0x4d,0x83,0x56,0x19,0xa8,0xa4,0x81,0x27,0x52, +0x21,0xd9,0xa7,0xac,0xa6,0x97,0x18,0x0b,0x76,0xc6,0x28,0xbf, +0x92,0xea,0x09,0xfc,0x88,0x8f,0xc3,0x62,0x76,0xf7,0x59,0x9f, +0x21,0xa2,0xcc,0x67,0x47,0xc5,0x75,0xae,0x77,0x2d,0xf1,0xca, +0x31,0x82,0x57,0xbb,0xe9,0xc8,0x8e,0x6a,0xa0,0xea,0xcf,0x65, +0x90,0x25,0x1a,0x92,0x6e,0xb3,0x03,0xc4,0x71,0xf1,0x13,0xd5, +0x47,0xe2,0x3a,0x83,0x55,0x63,0x0c,0xb8,0x0a,0x22,0x86,0x81, +0x31,0x9b,0x5d,0x85,0xfd,0x03,0x15,0x9b,0x40,0x03,0xe6,0x68, +0xc1,0x4d,0x7a,0x16,0xef,0xd4,0x8c,0xbe,0x27,0xe3,0xdb,0xe8, +0x64,0xbb,0xac,0xe5,0x3b,0x2e,0xa8,0xfb,0x90,0x9f,0x32,0xf0, +0xa9,0x80,0xec,0x7e,0x38,0xcb,0x3d,0x26,0x6b,0x4b,0x31,0xa4, +0x1e,0xb0,0xc3,0x11,0x58,0x11,0xc3,0x7d,0xd0,0xde,0x2e,0x9a, +0x92,0x3e,0xd3,0x85,0xfc,0xba,0xaa,0xdc,0x40,0x92,0x97,0x2c, +0x6b,0x26,0x23,0x5e,0x59,0x98,0x7c,0xcc,0x0c,0xc2,0xae,0xa9, +0x79,0xb1,0x08,0xaa,0xc6,0xab,0xa8,0x9a,0x76,0x4b,0x9a,0x2e, +0x66,0x8e,0xe1,0xaf,0xb5,0xe4,0x0c,0xd4,0x41,0xfb,0x6c,0xc7, +0x3a,0x63,0xdb,0x40,0x49,0x2f,0x6d,0xb8,0xf7,0xb3,0x73,0xdd, +0xad,0x7d,0x6a,0x77,0x3f,0xb3,0x38,0x7a,0xf1,0x99,0xae,0x70, +0x6a,0xc6,0x04,0x7f,0x5f,0xd2,0xd0,0xa8,0xba,0xaa,0xbb,0xbe, +0x3a,0x49,0xb9,0xa8,0xfb,0xa2,0xd9,0x74,0x58,0x5c,0x99,0xd6, +0x84,0x08,0x56,0x8c,0x5b,0x8c,0x45,0xec,0x11,0x65,0xa9,0x4a, +0x9c,0xa6,0x16,0x87,0x1d,0x2c,0x02,0x67,0xb5,0xa3,0xa3,0xb8, +0xbe,0xce,0x47,0xc5,0xd0,0x78,0x9d,0x62,0xd4,0x34,0x3e,0x0c, +0x5d,0x90,0xa8,0xd7,0x93,0xeb,0x1b,0xe5,0x5b,0x1b,0x89,0x7e, +0x59,0xd5,0x5c,0xb6,0xb8,0xeb,0xb5,0xda,0x41,0x44,0x89,0x7a, +0xef,0x64,0xc0,0xb6,0x6d,0xc9,0x2e,0xd0,0xba,0x0f,0x99,0xbd, +0xc6,0x8f,0xc7,0xce,0x04,0x90,0x24,0x8d,0xbd,0xd6,0xb7,0x9f, +0xe8,0xcc,0x1d,0x1a,0x1b,0x9f,0x6f,0x28,0xbf,0xd6,0x43,0xcc, +0x16,0xdb,0xda,0xda,0x64,0x7c,0xd6,0x83,0xaa,0xe9,0xa1,0xe6, +0xae,0xc0,0x39,0x9f,0x18,0x64,0xee,0xe4,0xad,0xb1,0x66,0xd9, +0x61,0x93,0x4a,0x67,0x39,0x1e,0x0b,0x52,0x93,0x30,0x21,0xa9, +0x32,0x21,0x9d,0x8f,0xac,0x31,0x8d,0x11,0x72,0x17,0xed,0xd8, +0x56,0x30,0x46,0xf2,0x72,0x8b,0x2d,0x85,0x16,0x7d,0x48,0x01, +0xd8,0x83,0x87,0xc3,0x05,0x4d,0xc4,0x6b,0x0e,0x37,0x04,0xf3, +0x44,0xa6,0xe5,0xc2,0x1e,0x39,0xcd,0x13,0x8e,0x27,0x51,0x27, +0x23,0x0b,0x17,0x54,0xaa,0x1c,0xd4,0x74,0x13,0x19,0x2c,0x44, +0xae,0x97,0x15,0x77,0xbf,0x35,0x7b,0xdd,0x60,0x00,0x16,0xde, +0x2a,0x0a,0x3b,0x20,0xbd,0x99,0x07,0x51,0x3a,0x71,0x44,0xc3, +0x26,0xb0,0x42,0xcd,0x40,0xca,0x61,0x7a,0x4f,0x88,0xd4,0x0d, +0xcf,0xbd,0x71,0x8b,0x63,0x54,0x79,0xac,0xbc,0xd2,0x27,0xdb, +0x34,0x92,0x1f,0x77,0x7a,0xcd,0xdd,0x60,0x2c,0x6a,0xa0,0x0a, +0x34,0x85,0x74,0x17,0x01,0x44,0x6c,0x45,0x14,0x1e,0xc8,0xd2, +0x5c,0x33,0x38,0x16,0x86,0xf0,0x48,0xba,0xce,0xd2,0x30,0x7a, +0x39,0xc6,0x28,0xf6,0x1c,0xc4,0x2d,0x22,0xfe,0x87,0xd8,0xfb, +0x83,0x6e,0x60,0x80,0x01,0x58,0x68,0x89,0x35,0xf6,0x24,0x5b, +0x5b,0x68,0x7b,0x89,0x0a,0x20,0x78,0x37,0xad,0x01,0xa9,0x4b, +0xeb,0xe4,0x36,0x90,0x6c,0x1a,0x2d,0xd0,0xab,0xe6,0x9c,0xbc, +0xd5,0x5f,0x47,0xbd,0x60,0x37,0xea,0x99,0xb4,0x45,0xef,0x4a, +0x7b,0x96,0x92,0xc4,0xbe,0x62,0xfa,0x67,0x28,0xa2,0xf2,0x6b, +0x10,0x84,0x80,0x40,0x4f,0x5f,0xd8,0xc3,0x5b,0xc2,0x44,0xe4, +0x2f,0x0d,0xc6,0x76,0x46,0xf9,0x74,0xd6,0x40,0x99,0xd3,0x65, +0xfc,0x21,0xbb,0x7d,0xd6,0x5d,0x78,0x04,0x64,0xfc,0x54,0x55, +0x5d,0x18,0x12,0xa0,0x1e,0x32,0x34,0x0a,0x1b,0x0c,0x17,0x59, +0x90,0x1c,0x1b,0x47,0x89,0xd8,0xc3,0x70,0xad,0x03,0x3d,0x5a, +0xb0,0x4c,0x75,0xb8,0x50,0x32,0x1e,0xed,0x92,0x41,0x22,0x7a, +0x52,0xa8,0x3f,0xa4,0x41,0x09,0xb5,0x41,0xd4,0x71,0x72,0x23, +0xdd,0xf8,0xc7,0x89,0xc0,0x8f,0x36,0xb0,0xdb,0x74,0x72,0x03, +0x64,0x72,0x32,0x10,0x47,0x51,0xe8,0x29,0xc1,0x46,0x9e,0x67, +0xb1,0x16,0x42,0x29,0xd4,0x7c,0x3c,0xde,0xf3,0x33,0xbd,0xf1, +0x47,0x35,0x79,0xf8,0x29,0x78,0x77,0xc7,0x56,0x80,0xa7,0x8d, +0xae,0x21,0x51,0xc5,0x7c,0xf3,0x99,0x24,0x57,0xdb,0x8e,0x18, +0x49,0xf0,0xe6,0x33,0xae,0x03,0xd7,0x45,0x38,0x64,0xfe,0x3c, +0x79,0xb5,0x1d,0x6a,0xd5,0x89,0x48,0x30,0xd0,0xa9,0x17,0xb7, +0x1a,0xe2,0x9c,0x78,0x44,0x14,0x9a,0x0b,0xbb,0xff,0x2d,0xbd, +0x69,0x0f,0xec,0x6d,0xb7,0x92,0x65,0x7c,0x4d,0x17,0x8b,0xad, +0x0b,0x46,0x46,0xa9,0xa7,0x4f,0x78,0xd8,0xc1,0xf3,0xc8,0xe0, +0x78,0x12,0x6d,0x2d,0xd1,0xbe,0x03,0xce,0x1a,0x9d,0x4f,0x12, +0x73,0x56,0x89,0x92,0xb7,0x5c,0x27,0xf1,0xec,0xb7,0x26,0x3c, +0xb2,0x8c,0x17,0x8d,0xb3,0x1a,0x0a,0xd3,0x3d,0xa6,0x5f,0x85, +0x0a,0x4b,0x9d,0xaa,0xe1,0xe1,0x0c,0x9c,0x4b,0x9c,0xe2,0x0f, +0x99,0x31,0x08,0x03,0x85,0x8d,0x5d,0xcd,0xf4,0x87,0x8b,0x45, +0x67,0x47,0xa7,0x76,0x76,0x4e,0xdd,0x2d,0x01,0x9d,0x48,0x0f, +0x16,0x14,0xcb,0x75,0x87,0xde,0x6d,0x2f,0xdd,0x5c,0xc3,0x6c, +0x68,0x6f,0x7b,0x25,0xea,0x34,0xcc,0x3e,0xcd,0x4e,0x5d,0x36, +0xe9,0x53,0xb4,0x48,0x55,0xbe,0x06,0xa7,0xdd,0xe6,0x22,0x20, +0xf7,0x25,0x6e,0x51,0x6d,0xee,0x66,0x90,0x7d,0x9a,0x38,0x83, +0xbd,0xf2,0x63,0x31,0xd3,0x03,0xc1,0xc2,0x6a,0x71,0x71,0x58, +0xe6,0x51,0x9f,0x56,0xa5,0x0d,0x2a,0x36,0x88,0x52,0x47,0x39, +0xfb,0x57,0x3c,0x05,0x87,0xe0,0x02,0xf2,0x35,0xaa,0x42,0x05, +0xe4,0x66,0xd6,0x1a,0xf6,0x4e,0xf5,0xdf,0xd3,0x54,0x8f,0x8b, +0x61,0xcf,0x34,0x4c,0xaa,0x7f,0x5a,0xa7,0x7e,0x61,0x90,0x28, +0x75,0x51,0x2f,0x8d,0x43,0x02,0x5b,0x8c,0xad,0x9a,0x72,0xef, +0xf2,0x8b,0xe7,0x9f,0x6e,0x44,0xa5,0x6d,0x8e,0xc4,0xa2,0x9b, +0x8a,0x9e,0xb6,0xb6,0x86,0x6d,0xba,0x28,0x0a,0x69,0xf4,0x04, +0x69,0xd7,0x7a,0x66,0x80,0xe0,0x8e,0xa0,0xf6,0x05,0x72,0x8a, +0x8b,0xf1,0x64,0x9a,0x83,0xcf,0x73,0xc8,0x72,0x6f,0xa0,0x69, +0x8d,0x78,0xb1,0xb6,0x0a,0x5f,0xdd,0xb7,0x95,0xb4,0x53,0xd7, +0xba,0x81,0x0e,0x8e,0x46,0x90,0xbf,0x4e,0x18,0xa5,0x21,0x66, +0xf5,0xf9,0x2e,0x84,0xb0,0x89,0x39,0x0d,0x32,0x4e,0xc9,0x04, +0xfa,0x29,0xa8,0xd2,0x86,0xc0,0x1b,0xca,0xd9,0x74,0x8e,0x71, +0xe5,0x4f,0x31,0xed,0x54,0xa6,0xa1,0xd5,0x0d,0x24,0x83,0xe5, +0xcd,0x53,0xb4,0x96,0xdb,0x8c,0xbd,0x25,0x02,0x73,0xf5,0xfa, +0x80,0xbf,0x72,0xc4,0x0d,0x9f,0x5a,0xe3,0x3a,0x0b,0x7c,0xca, +0xc0,0xa7,0x21,0x30,0x65,0xb8,0xef,0xe6,0x0f,0xa5,0xe3,0xa9, +0x84,0xa7,0x0c,0xbd,0x9e,0xd2,0xd0,0x45,0x5b,0x80,0x5d,0xe5, +0x2c,0xd6,0xa1,0x75,0x4d,0xb7,0xe2,0x3c,0x1f,0x44,0x59,0x26, +0x86,0x25,0x70,0x20,0x28,0x33,0x34,0xe2,0x00,0x17,0xc5,0x44, +0xfd,0xd5,0x2c,0x0d,0xa0,0xea,0x1a,0xd2,0xb6,0xb7,0x9d,0x60, +0x9f,0xff,0x1a,0x0f,0xfb,0x90,0x38,0x50,0xa7,0xfc,0xeb,0x4f, +0xe6,0xa4,0x9b,0x74,0x29,0x5c,0x5a,0x68,0xf9,0x0a,0x7d,0x43, +0x92,0xd7,0x90,0xe5,0x2e,0x24,0xa3,0xd5,0x52,0x9b,0x06,0x3b, +0x65,0x9f,0x72,0x36,0xd6,0x05,0xfc,0x58,0x57,0x85,0x3f,0x41, +0x4d,0x49,0x82,0x66,0xda,0xf0,0xbd,0xac,0x28,0xc3,0x3a,0xb9, +0xbe,0x53,0x57,0x9f,0x41,0xb8,0x49,0x5f,0xc3,0xc5,0x4c,0x83, +0x41,0xe2,0xd2,0x46,0x3e,0xe3,0xf6,0x9d,0xdc,0xc4,0xae,0xb1, +0xf1,0x85,0x45,0x0b,0xf4,0x57,0xf6,0xfc,0xd7,0xb9,0x67,0xa5, +0xa0,0x4e,0xdd,0x62,0xf0,0x2b,0x72,0xb8,0xfe,0xc0,0x9c,0xfe, +0xb2,0x45,0x65,0x56,0x11,0x6e,0xad,0xa5,0x86,0xe9,0x6c,0xec, +0x39,0x9f,0x39,0xd3,0xa1,0x37,0xf3,0x09,0x69,0x93,0xc9,0xeb, +0x83,0x99,0x00,0x35,0xab,0x9d,0xfd,0x7a,0x23,0xe7,0xd6,0x98, +0x16,0x13,0x79,0x7c,0x3d,0xbd,0x6e,0xed,0xd8,0x8c,0x37,0x27, +0xe0,0xd5,0xad,0xb3,0x49,0xd5,0x8d,0x27,0xa3,0x1c,0x04,0x6f, +0xfe,0x32,0xb2,0x83,0xf1,0xee,0xf9,0x86,0x52,0x3c,0xe1,0xa9, +0x8e,0x47,0x9f,0x8f,0xa2,0x1a,0x4e,0xb9,0x85,0xaa,0xd1,0xb6, +0x26,0xf3,0xb6,0x03,0x4b,0xc5,0x91,0x1c,0xac,0x59,0x40,0xa4, +0x22,0x33,0xc7,0xf4,0x23,0x55,0xa1,0x1f,0x88,0xa5,0xea,0x07, +0x60,0x8f,0xfa,0x87,0xb8,0x9d,0x0c,0xfa,0x30,0xc6,0xa9,0x71, +0xd2,0x8f,0x8a,0x32,0x6a,0x91,0x6f,0x8b,0x5a,0x1a,0xaa,0x74, +0x67,0x99,0xfd,0x3a,0x2a,0xd9,0x8a,0x06,0xd1,0x92,0x8c,0xba, +0x84,0xbd,0x82,0xdb,0x03,0x69,0x99,0xc3,0xcb,0x5a,0xd9,0xf2, +0x9b,0xae,0x13,0x23,0xa8,0x32,0xcf,0x23,0x3c,0x99,0x8e,0x4c, +0x21,0xcb,0x2d,0x77,0xda,0x3b,0x7a,0x4a,0x49,0x2e,0x53,0x8b, +0xd8,0x15,0xcc,0x0c,0x9b,0x85,0x75,0x9a,0xc8,0x7d,0x01,0x87, +0xf0,0x0d,0x62,0x84,0xc9,0x65,0xe3,0xe3,0xcd,0x42,0xb3,0x93, +0xd7,0x31,0xba,0xb8,0x7b,0x71,0x35,0x19,0xf2,0x06,0x8d,0x7a, +0x51,0xe7,0x34,0xcb,0x22,0xd4,0x19,0xce,0x86,0x79,0x13,0x6d, +0x93,0xa8,0x20,0x2f,0x55,0xab,0xfa,0x7f,0x73,0x77,0xb1,0xe0, +0xa7,0x8e,0x5e,0x33,0x2a,0x8b,0x5c,0xf3,0xd8,0x82,0x78,0xe5, +0xab,0xac,0xa6,0x32,0x4b,0x86,0x0d,0x41,0x5d,0x0f,0x2e,0x45, +0x34,0x28,0xa0,0x7b,0xa8,0xf9,0x9e,0x82,0xef,0x38,0xc0,0xd2, +0x84,0x30,0x50,0x78,0x3c,0x01,0xc7,0x0b,0x5a,0x00,0xab,0x89, +0xc8,0xbf,0xc1,0x17,0xc4,0xab,0xbe,0x16,0x38,0x3e,0xb4,0x53, +0x38,0x37,0x19,0xde,0x5a,0xd3,0x66,0xdc,0x9c,0x3b,0xa8,0x71, +0x8d,0x8d,0x88,0xd5,0x2c,0x9c,0x9d,0xcd,0xaf,0xe9,0xc6,0x7e, +0xec,0x9b,0xfc,0xe2,0x16,0x71,0xcc,0x3b,0x32,0x04,0x83,0xdd, +0x61,0xbd,0x69,0x9d,0x3b,0x0c,0x47,0x40,0x3e,0xab,0x1b,0x4d, +0x44,0xb4,0xf4,0xeb,0x62,0xac,0x34,0xee,0xa0,0x1f,0x74,0x8a, +0x1e,0xc9,0x70,0x69,0x01,0xf3,0x8b,0x31,0x9c,0xcc,0xe9,0x9f, +0x16,0xd2,0x72,0x7e,0x35,0xd1,0x64,0xe2,0x23,0xdb,0x2e,0x27, +0x0f,0x00,0x78,0x1b,0x20,0x3a,0xc6,0x2a,0x99,0x1a,0x05,0xdc, +0x2f,0xdc,0xe6,0xe5,0xd9,0xf0,0x26,0x4f,0x6f,0xa3,0xad,0x28, +0x8d,0xb6,0x40,0xcf,0xa6,0x39,0xc0,0x63,0x78,0xbe,0x9a,0xc1, +0xe3,0x3e,0x3c,0x5e,0xc0,0xe3,0xfd,0xe8,0xbe,0x7e,0xfc,0x75, +0x3e,0xc1,0xf4,0xfb,0x90,0xfe,0x9f,0x9f,0xf6,0xbe,0xe9,0xea, +0xe9,0xdc,0xb5,0x08,0xdb,0xf3,0x31,0x61,0xcc,0xcc,0x7e,0x37, +0x76,0x79,0x94,0x93,0x08,0x12,0x34,0xab,0xc9,0x3f,0xe9,0x86, +0x37,0x64,0xc0,0xae,0x9b,0xd8,0x8f,0x5e,0xa5,0xa3,0x16,0xef, +0x82,0x2b,0x18,0x38,0xfa,0x68,0x94,0x00,0x37,0x51,0xd1,0x45, +0x94,0x28,0x53,0xaf,0x87,0x22,0xae,0x41,0x61,0x00,0x0d,0x92, +0x05,0x60,0x49,0x08,0x0b,0x2c,0x60,0xcc,0x34,0x09,0x48,0x27, +0x1b,0x78,0xc9,0x0f,0xcd,0xed,0xa0,0x13,0x73,0x4d,0x47,0x9e, +0xb0,0xc2,0xe4,0x0a,0xfa,0x8c,0x12,0x7b,0x51,0x94,0x6a,0xa2, +0x5b,0x0c,0xd2,0x9e,0xe6,0x37,0x57,0x43,0xbd,0xb3,0xf0,0x9a, +0xc2,0x60,0x14,0x95,0x81,0x7b,0x2b,0xb7,0x84,0x9a,0x0f,0x31, +0x80,0x7d,0xcc,0x87,0x5b,0x3c,0xc8,0xa6,0x84,0x8e,0xd1,0x4c, +0x38,0xcd,0x00,0xc0,0xd3,0x2b,0xce,0x30,0x7f,0x71,0xd5,0x39, +0x89,0x37,0x6b,0xbb,0xce,0xc8,0x94,0xb2,0xfb,0xa6,0xe0,0xa0, +0x7b,0x97,0xad,0xba,0x9d,0xbc,0x80,0x35,0x15,0x17,0x39,0x3e, +0x15,0x21,0xcb,0xc2,0x36,0x16,0x6a,0x08,0xda,0x11,0x05,0xcb, +0x8d,0xd1,0xf7,0x65,0x27,0xec,0x4e,0x09,0xb6,0x79,0xf8,0xa4, +0x7b,0xc9,0x1e,0x2c,0xf6,0x83,0xe5,0x48,0x6a,0x22,0x3d,0xbf, +0x16,0xeb,0xa8,0x24,0xcd,0x65,0x09,0xf8,0x10,0xd4,0xcd,0x09, +0x5d,0xdd,0x09,0x2b,0x69,0x97,0x26,0x18,0x72,0x31,0x7a,0x0a, +0x76,0xc5,0xf9,0x54,0x4f,0x7a,0x63,0x72,0xf6,0x52,0xf0,0x44, +0xcd,0x96,0xce,0x8b,0x4f,0x26,0xe4,0x6e,0xd6,0x6a,0xd9,0x02, +0xad,0x28,0xb2,0xc4,0x20,0x50,0x8f,0x7e,0x34,0x44,0x5a,0xe0, +0xdd,0x85,0x59,0x7e,0x7d,0x73,0x85,0x3e,0x69,0x66,0x33,0x3d, +0x56,0x60,0x82,0x40,0x83,0xea,0x94,0x74,0xe7,0xf1,0xdf,0xe2, +0xfe,0x71,0x79,0x7c,0x38,0x68,0xf5,0x92,0xbf,0xed,0xef,0x5c, +0xd0,0x76,0xf3,0x66,0x72,0xc5,0xb9,0x59,0x90,0xcd,0xf3,0x42, +0xe7,0x6c,0xfb,0x39,0xf4,0x19,0xe3,0x09,0x86,0x53,0xcf,0x76, +0xe2,0x76,0xf2,0xf3,0x0e,0x43,0xeb,0x0a,0x71,0x86,0xdf,0xd7, +0x13,0xe0,0xf8,0x58,0x3f,0xe8,0x3f,0xfa,0x69,0x0a,0x6e,0x2d, +0xe1,0x61,0xac,0x1f,0x40,0x86,0xf8,0x7f,0xf4,0x2f,0x44,0x93, +0x3a,0x9e,0xef,0xed,0xee,0xfd,0x5d,0xbf,0xd0,0x2f,0x27,0x7c, +0xcb,0x09,0xdf,0x46,0x4b,0xc6,0x3b,0xcd,0x76,0x8e,0x8f,0x17, +0xf7,0x17,0xc7,0xd3,0xc5,0xf1,0x78,0x71,0x3c,0x5b,0x50,0x41, +0xfa,0xf9,0x76,0xe7,0xa2,0xeb,0x3e,0x5d,0xa8,0x57,0x81,0x5f, +0xa2,0xdf,0xd8,0x92,0x9b,0xc3,0x18,0x59,0x01,0xc3,0xef,0x9a, +0x44,0xcf,0x77,0xd8,0xd2,0xc2,0xd6,0xb4,0x26,0x71,0x20,0x9c, +0x37,0xec,0x4d,0x8a,0x19,0x46,0x3f,0x36,0xa5,0x98,0xd3,0xe8, +0x2d,0x35,0x35,0x4f,0xc2,0xfb,0x3c,0xe5,0x20,0x44,0xb3,0xaf, +0x02,0x33,0x1d,0x57,0x81,0x19,0x48,0x06,0xb4,0xb8,0xc7,0x7c, +0xcc,0x5e,0xa2,0xa1,0x0d,0x66,0x74,0x72,0x72,0xd3,0xca,0xee, +0x47,0x5d,0x68,0x04,0xcb,0x31,0x98,0xf4,0x80,0x47,0x70,0x1b, +0xcb,0xf1,0xa0,0x4c,0xed,0x6a,0x72,0x7e,0x5e,0x8a,0x6b,0x97, +0x84,0xbe,0x95,0x21,0x5a,0x52,0x70,0xb0,0xfd,0x3f,0xc1,0x39, +0xe6,0x44,0xfd,0xd6,0xc0,0x8e,0x60,0x5c,0xb4,0x78,0xc8,0x58, +0x46,0xc4,0x74,0xc0,0x16,0x86,0x6b,0x89,0xee,0xb7,0x8e,0xc7, +0x31,0x78,0xa6,0xcd,0x62,0x03,0xaf,0x99,0xae,0x96,0x0e,0x81, +0x15,0xf5,0xee,0xdf,0x07,0xdd,0x0a,0xa6,0x02,0x4c,0x92,0x68, +0xe8,0xfb,0xd8,0x16,0xf6,0x43,0x1a,0x91,0x09,0x98,0x00,0xa3, +0x46,0xc4,0x78,0x4c,0x23,0x78,0x48,0xba,0x7a,0x10,0xb7,0x4c, +0x4e,0x4b,0x0f,0x69,0x6e,0x69,0xd3,0x05,0xd4,0x10,0x2d,0xfc, +0x28,0xb3,0x17,0xc4,0x17,0xb8,0x66,0xe4,0x61,0xb1,0x43,0xad, +0xad,0x87,0x55,0x31,0x3c,0xbd,0x82,0x53,0x26,0xd3,0x7f,0x60, +0xdf,0x1a,0xa3,0xb0,0x7a,0xbb,0x4c,0x6e,0xa1,0x4e,0x2e,0x1c, +0x2d,0xf5,0x4b,0x62,0xfb,0x99,0x3d,0x57,0x2b,0x4d,0x45,0x76, +0xff,0xbe,0xfe,0xf9,0xa5,0x12,0xa1,0x10,0x86,0x8b,0x66,0xe4, +0xc5,0xd8,0x93,0x85,0x90,0x6c,0x0d,0x1e,0xaa,0xd7,0xee,0xdf, +0xd7,0x7b,0xcd,0xae,0xac,0xd0,0x30,0xf0,0x93,0x1b,0x48,0xee, +0xce,0xa6,0x9f,0x6f,0x69,0x0a,0xe1,0x14,0xb0,0x3c,0xbd,0xe6, +0x6b,0x40,0xf7,0xa0,0xc7,0xe7,0x49,0xc4,0xe4,0x26,0xcb,0x33, +0x68,0x89,0x58,0x73,0x69,0x3a,0x73,0xc8,0x8d,0x42,0x84,0x87, +0x3f,0x6a,0x31,0xd0,0xfd,0xba,0xd5,0x3d,0x43,0x45,0xe4,0x72, +0xfd,0x84,0xe6,0x5f,0x75,0xa2,0x7b,0x81,0xfd,0xa8,0x84,0xe0, +0xc5,0x5c,0xd6,0x1e,0x09,0x9b,0xf2,0xa6,0x6a,0x2b,0xbb,0xeb, +0x51,0xd1,0xf8,0x11,0xb0,0xd6,0xfb,0xbd,0x10,0x29,0x83,0x08, +0x35,0x85,0x97,0xc3,0xca,0x6a,0x65,0xda,0x8d,0xee,0x01,0x20, +0x44,0x6c,0xdd,0xbd,0x54,0x56,0x5b,0x3f,0x68,0xc8,0x09,0x82, +0xf7,0xfc,0xb2,0xa8,0x38,0xec,0xf2,0xd2,0x18,0x83,0xdb,0x23, +0x0e,0xd3,0x37,0x81,0x10,0xda,0x18,0x1a,0x53,0xe9,0xde,0xfa, +0x0d,0xec,0x62,0xf5,0x53,0x79,0x59,0x9c,0x03,0x93,0x05,0xf5, +0x39,0xfc,0x60,0x30,0x56,0x94,0x4c,0x28,0xa7,0xb2,0x51,0xa3, +0x73,0x42,0x90,0x0e,0x84,0xf7,0xa9,0x75,0x97,0x4e,0xd8,0x3d, +0x78,0x0b,0xa7,0xf3,0x4f,0x01,0x18,0x8d,0x19,0x39,0xdc,0xa2, +0x75,0x0b,0x27,0xd3,0xb8,0x99,0x01,0xca,0xad,0x2d,0x43,0x23, +0xbd,0x2f,0x16,0xbb,0x9b,0x72,0x43,0xbe,0x58,0x90,0x3b,0x5a, +0xbc,0xc7,0xb6,0x3b,0xa8,0xae,0xba,0xb8,0x29,0x59,0xda,0x66, +0x21,0xf5,0xb6,0xfe,0x60,0x98,0x00,0xd0,0x00,0x88,0xfb,0xaf, +0xfb,0x6a,0xef,0x78,0x44,0x90,0xe1,0x7d,0xb0,0xd7,0x24,0xe2, +0xd3,0x59,0xec,0x62,0xcd,0xb0,0xa8,0x47,0xdd,0x62,0x6f,0xa7, +0xd5,0x8a,0xc4,0xc0,0x80,0xb3,0x59,0xda,0xab,0x04,0xd7,0xfd, +0x02,0x60,0xae,0x17,0x64,0x0f,0x47,0xa0,0x90,0x98,0xac,0xf7, +0x2d,0x11,0x3b,0xfd,0x66,0x38,0x3a,0x9a,0x7c,0xc8,0xad,0xb1, +0xdb,0x7c,0x3c,0xab,0x1e,0x4f,0xba,0xf9,0x06,0xf7,0xdd,0x7e, +0xca,0xa7,0x93,0x17,0x85,0x46,0x8f,0x72,0x8f,0x6b,0x89,0x61, +0xc2,0xe5,0x85,0x7f,0xc9,0xc9,0x74,0x54,0x8c,0x87,0x57,0xc5, +0xef,0xb9,0xa8,0x45,0xaf,0x14,0xc5,0x64,0xb4,0xaa,0x1a,0xfc, +0x9e,0xab,0x21,0x44,0xce,0x69,0x33,0x8a,0x9a,0xda,0x18,0x8f, +0xa8,0xee,0x15,0x78,0xf4,0x1b,0x5e,0xe4,0x60,0xce,0x67,0x13, +0x0f,0x26,0x18,0x9f,0x86,0x7d,0x34,0xea,0x06,0xcf,0xcf,0x3e, +0x80,0xc7,0x6c,0xbd,0x0b,0xfa,0x68,0x3d,0x3b,0x72,0xd7,0x20, +0x6a,0x4e,0x73,0x18,0x9e,0x19,0x2f,0x7f,0x23,0xe3,0xda,0x8f, +0xed,0x4b,0xa6,0xd7,0xf8,0x6d,0xa3,0x97,0x70,0xff,0x22,0xb3, +0xef,0xa4,0x5f,0x79,0xaf,0xb7,0xfa,0xa5,0x2b,0xa2,0x3e,0xe7, +0x43,0x38,0x32,0xf4,0x0b,0xb5,0x21,0x15,0xec,0x27,0xaf,0xf5, +0x96,0xef,0xb2,0x9a,0x8d,0xc9,0x90,0xff,0x31,0xcf,0x3f,0x54, +0xb3,0x21,0x15,0x72,0x47,0xc3,0xcf,0xd5,0xcc,0x11,0xe8,0x92, +0x76,0xd5,0xa5,0x66,0x62,0xd5,0x4c,0x48,0xc5,0x7a,0x8b,0xf1, +0x7c,0x96,0xd7,0x54,0x8c,0xe9,0x68,0xd9,0x99,0xeb,0x06,0x19, +0x55,0x21,0x28,0x9d,0x70,0x5c,0x5d,0x15,0x4d,0x60,0x22,0x53, +0xc3,0x76,0x69,0xa8,0xe2,0x85,0x95,0xcc,0xb4,0x8d,0xa2,0x44, +0x0f,0x4d,0x4b,0xbe,0xb5,0x3a,0xf9,0xc3,0x07,0xe6,0xf9,0xeb, +0xfc,0xd1,0x03,0xa6,0xba,0xf5,0xf0,0xeb,0xfc,0xab,0x07,0xf8, +0x81,0x8c,0x03,0x1b,0xa2,0x05,0x7f,0x5b,0xdf,0x3c,0xc0,0x36, +0x33,0xc8,0xa9,0x7d,0x5b,0xf4,0xdb,0xea,0xec,0x3d,0xc0,0xfe, +0xb0,0xc5,0x66,0x43,0x38,0x29,0xa3,0xb7,0xd3,0xf9,0xa9,0x5e, +0x0c,0x62,0x19,0x62,0x82,0x86,0x07,0xaa,0x29,0xad,0xfd,0x2d, +0xe9,0x48,0x4f,0x03,0xbf,0x86,0x71,0x01,0x7a,0xd9,0x21,0x6c, +0xea,0x4f,0xf5,0x1f,0xe7,0x38,0x36,0x84,0x8b,0x8c,0x92,0x25, +0xc2,0x02,0xd6,0x7f,0x62,0x76,0x6a,0x1f,0x13,0x55,0x2d,0x85, +0x3c,0xe0,0xcd,0x39,0x17,0xe2,0x37,0x5d,0x86,0x9f,0x12,0x25, +0x22,0xfe,0x0c,0x4f,0xcb,0x77,0x60,0xd6,0x19,0x93,0x6d,0xb8, +0x9d,0x5f,0xbb,0xfb,0x94,0xd0,0x73,0xf6,0xb7,0x0c,0x91,0x0a, +0x05,0x01,0x27,0x39,0x74,0xde,0xec,0xa7,0x5c,0xf6,0xd4,0xf1, +0x2a,0x70,0x5f,0x47,0xf7,0x7f,0x32,0x82,0x81,0x9d,0x0c,0xdf, +0x12,0x62,0xfb,0x61,0x59,0xa8,0x9b,0x30,0x70,0xb4,0x1b,0xb5, +0xe8,0xd1,0x34,0x19,0xbd,0x79,0x6e,0x91,0xdf,0x4c,0x0f,0xe7, +0xa7,0x18,0x6e,0xc9,0x4c,0x4a,0x08,0x68,0xc9,0x53,0xfc,0x7a, +0x72,0xad,0xec,0x98,0x2a,0xca,0x27,0x23,0xb0,0xef,0x52,0x74, +0x6a,0xf2,0xfe,0x46,0xf7,0xb1,0x6e,0x1f,0x12,0x70,0xe9,0x7a, +0x23,0x8e,0x21,0x9a,0x1f,0xfe,0x18,0x36,0x48,0xfc,0x31,0x49, +0xb3,0xcc,0xe5,0xc1,0xab,0x99,0xb7,0xa2,0x04,0x26,0x74,0x65, +0x41,0xf0,0x6e,0x78,0xad,0xe1,0xdb,0x6c,0xfd,0x13,0xb7,0xe8, +0xdd,0x1b,0xe3,0x0f,0x0c,0xc1,0x89,0x8a,0x01,0xf3,0x62,0x41, +0x98,0xa0,0x9f,0xcd,0x24,0x85,0x62,0xf4,0x1c,0x27,0x3c,0xaf, +0x21,0x09,0x9e,0xe2,0x24,0x41,0xfa,0xa8,0x2e,0xf8,0xd6,0xd8, +0x3e,0x24,0x38,0x21,0x44,0x05,0x84,0x99,0x40,0xf1,0x39,0x76, +0x4f,0x09,0x4f,0x11,0x09,0xee,0x7d,0xcb,0x66,0xb5,0x41,0x11, +0x93,0xee,0x81,0xf6,0x5c,0x24,0x02,0xce,0xba,0x6f,0x71,0xdf, +0xc0,0x9f,0x05,0xf4,0xf0,0x47,0xe0,0x37,0xc9,0x38,0xc7,0x46, +0x95,0x4f,0xd7,0xdd,0x1a,0x75,0xe1,0x15,0x0f,0xa8,0x6d,0x5f, +0x83,0x49,0xc5,0x25,0x5a,0x50,0x63,0x37,0x60,0xa5,0x03,0xc0, +0x2f,0x47,0x8a,0xb7,0x06,0xa7,0x39,0x12,0x55,0x92,0xe5,0x7d, +0x87,0x6e,0x56,0xec,0xa9,0x91,0xfe,0xfe,0xa7,0x13,0xd4,0x75, +0xf1,0x66,0x1f,0xac,0x35,0x32,0xe3,0x55,0x82,0xa1,0xcd,0x16, +0x82,0x0a,0x99,0x63,0x24,0xb6,0xeb,0x78,0x56,0x9c,0x9f,0x53, +0x01,0x3d,0xad,0xfd,0x02,0xdb,0x41,0x01,0xb8,0x78,0x04,0x96, +0x99,0x30,0x1d,0xd9,0x22,0x85,0x15,0x8e,0xb1,0x20,0x84,0x6f, +0x2d,0x74,0x34,0x97,0xda,0xe4,0xeb,0x0f,0x7b,0xfa,0x79,0xb1, +0xd8,0xf4,0x80,0x66,0x93,0x97,0x7a,0x6e,0x59,0x50,0x88,0xdd, +0x21,0x92,0xa0,0x44,0xa2,0xfb,0x15,0xab,0x6c,0xb5,0xcc,0xd4, +0xa5,0x57,0x47,0xb8,0x6b,0x25,0xbb,0x3e,0xd0,0xfa,0x38,0x87, +0xbf,0xa8,0x70,0xe2,0x27,0x72,0x88,0xf0,0x31,0x87,0x80,0x3c, +0x98,0xa4,0x5b,0xfd,0x15,0xbc,0x53,0xe4,0x67,0xbb,0xe7,0x04, +0x55,0x44,0x79,0x6f,0x47,0x45,0xf7,0x3a,0x51,0xd2,0x45,0x48, +0x84,0x7f,0x02,0x21,0xa3,0xf5,0x46,0x13,0x53,0xf4,0xc7,0x9c, +0x69,0x49,0xce,0x46,0x56,0x2e,0xfb,0x8c,0x5a,0x67,0xf0,0x93, +0x39,0xb7,0x42,0xf8,0x1a,0x32,0xe5,0x62,0x8e,0x3d,0x4e,0x09, +0xa1,0x08,0x00,0x0c,0x1a,0x15,0x68,0x2a,0x14,0x0b,0xf8,0x2e, +0x92,0xb1,0x51,0x90,0x38,0xc4,0x73,0xc8,0xe7,0x01,0x1c,0xa6, +0x8b,0x5f,0x43,0x16,0xb4,0x1e,0x42,0x29,0x1f,0xc6,0x2b,0x84, +0x24,0xf4,0x7d,0x80,0x41,0x26,0xea,0x65,0x33,0x08,0x67,0x84, +0xe3,0x17,0x75,0x0d,0x72,0x3d,0xfc,0x90,0x63,0xac,0xb2,0x73, +0x08,0x3f,0x63,0x0c,0x4c,0xe6,0xe8,0xc6,0x7a,0xa6,0xb7,0xd7, +0xd6,0x10,0x1d,0xb2,0xed,0x65,0x9e,0x08,0xd6,0x60,0xbd,0xe5, +0x46,0xc8,0xec,0x1b,0x86,0xcd,0x22,0xcd,0x15,0x22,0x74,0x87, +0x0a,0x3d,0x0f,0x92,0x7f,0x50,0x0c,0xf9,0x45,0x64,0x4e,0xdc, +0xba,0x54,0xbc,0xb3,0x67,0xcb,0x53,0xfe,0x92,0xd8,0x4e,0x1f, +0x0b,0x0b,0x71,0x1d,0xa2,0x3b,0x0e,0x67,0xd2,0xaf,0x41,0xa1, +0x2e,0xb0,0x24,0xcb,0xeb,0x19,0xf3,0xab,0x73,0xcd,0xb0,0x41, +0xce,0x64,0x0c,0xe2,0x82,0x65,0x57,0x5c,0xa9,0xb2,0xc7,0x3c, +0x84,0x17,0xdc,0x22,0xd2,0x8d,0x7e,0xaa,0x86,0x7e,0x32,0x27, +0x5e,0x73,0x5d,0xe2,0xfe,0x2f,0xaf,0x38,0x5c,0xad,0x1e,0xc5, +0xf3,0xd9,0x99,0xfe,0x0b,0x5c,0x92,0xbe,0x48,0xa8,0xfa,0x79, +0x47,0x81,0x2c,0x26,0x20,0x53,0x5d,0x73,0x65,0x7a,0xa3,0x1a, +0x25,0x4b,0x45,0xd7,0x92,0xe8,0x33,0xb9,0x34,0x55,0x4d,0x04, +0x26,0x96,0x01,0x60,0xfb,0xf9,0x37,0xf0,0xe8,0x04,0xdc,0x80, +0x27,0x95,0x6b,0x77,0xae,0xe7,0xcd,0x6c,0xa7,0xdd,0x0c,0xba, +0xbe,0xce,0xa7,0x67,0x56,0x0c,0x3e,0x83,0xb7,0x7c,0x44,0x67, +0x52,0x59,0xab,0x06,0x8e,0xf6,0x2d,0x99,0xf5,0x04,0x07,0xfd, +0xed,0x95,0xda,0xda,0xb2,0x47,0x63,0x5e,0x3a,0x4c,0x00,0x2a, +0xeb,0x25,0xef,0x67,0xbb,0x3d,0x21,0xa7,0xf8,0x45,0x52,0x27, +0xd3,0xf8,0x19,0x09,0x91,0xe1,0xbe,0x0b,0x56,0xa6,0x97,0xe3, +0x03,0x5c,0xf5,0x40,0x18,0xa4,0x65,0xd1,0x69,0xe5,0xd9,0xba, +0x38,0x86,0x3f,0xed,0xf7,0x47,0x4f,0x05,0x50,0xab,0x03,0x97, +0xd0,0x21,0x7c,0x9f,0x4e,0x47,0xa0,0x24,0xc4,0xfb,0x2f,0x0d, +0x8c,0x25,0xc4,0xf1,0xdf,0xab,0x7c,0x78,0xe3,0xd2,0x7b,0x0f, +0xbf,0xfe,0x3a,0x7d,0xf8,0xf5,0x57,0x72,0x59,0xf2,0x21,0x6e, +0xc5,0x59,0x35,0x24,0xfc,0xed,0xd1,0xd6,0x16,0x34,0x1e,0x3e, +0x77,0x76,0x77,0xb5,0x40,0x6d,0x73,0x76,0x77,0xc5,0x4a,0xe4, +0xed,0x71,0x38,0x80,0xc2,0x84,0xdf,0xbb,0x5a,0xf4,0x18,0x6e, +0x6d,0x6d,0xef,0xe9,0xa2,0xfa,0xf1,0xe6,0xbc,0x6d,0x72,0xc0, +0x4a,0x8a,0x1f,0x21,0x67,0xd8,0x3f,0x78,0xf3,0xfa,0xe8,0x87, +0xc1,0x63,0x5d,0x8f,0x78,0xdd,0xef,0x74,0x7a,0xf8,0x94,0x62, +0xe2,0xb3,0x27,0x47,0xcf,0x07,0x8f,0x3b,0x0c,0x82,0x6f,0xfb, +0xb2,0x65,0x31,0xf9,0x5f,0xcf,0x9f,0xbc,0x1b,0x28,0x81,0x24, +0xe9,0x01,0x24,0x61,0xf8,0xe1,0xcd,0xfb,0x77,0xae,0x12,0x7c, +0xdb,0xdf,0x7b,0xd8,0x83,0x07,0x02,0x38,0x78,0xf9,0xfa,0x3d, +0x54,0x62,0xe9,0xa0,0xf7,0xfd,0xaf,0xbe,0xed,0xd1,0x23,0x81, +0x1d,0x3e,0x7f,0xfa,0xe6,0xf5,0x33,0x07,0xc6,0xef,0x00,0x46, +0x8f,0x06,0xdb,0xab,0x57,0x2f,0x43,0x58,0x99,0xb8,0xff,0xed, +0xb7,0x80,0xd8,0x26,0xa4,0xdb,0x1d,0x45,0x0d,0x75,0x62,0x9a, +0xe7,0xd9,0xf0,0xf3,0x9b,0x73,0xe8,0x27,0xdd,0x64,0xf0,0x6d, +0xfb,0x26,0x63,0xb1,0x30,0x4f,0xfb,0xf0,0x81,0x89,0x6c,0x52, +0x4c,0x50,0x7e,0x93,0x67,0xe6,0x41,0x0a,0x27,0x74,0x8d,0x1b, +0xc2,0x03,0xe9,0x1d,0x8a,0x16,0x37,0x5e,0x5c,0x0d,0x2f,0x4a, +0xb7,0x81,0xa5,0xd8,0x05,0x1a,0x4b,0x76,0x8b,0x2e,0x83,0xd3, +0xcd,0x8e,0x9a,0x8f,0xe7,0x65,0x4e,0x5b,0xf9,0x32,0xed,0x0f, +0xf8,0x1d,0x59,0x38,0xbc,0x9a,0x5a,0xd2,0xed,0x3d,0x75,0x76, +0xa9,0xb1,0xbe,0xd2,0xa2,0x3c,0x0c,0x90,0x74,0x17,0x99,0x08, +0x01,0x6a,0x3c,0x05,0x19,0x02,0x63,0xd7,0xa5,0x64,0x15,0x4f, +0x29,0x2f,0x90,0xef,0x60,0x55,0x65,0x3e,0x65,0x7b,0x61,0x08, +0x35,0xa1,0x93,0x96,0x72,0x04,0xff,0x03,0x32,0xe2,0xeb,0xf0, +0x40,0x52,0xef,0xf9,0x28,0x0b,0x44,0x40,0xfb,0x92,0xf1,0x19, +0x38,0x4a,0xc7,0x17,0x2c,0x1d,0x83,0x38,0xe1,0xb7,0xd2,0xe3, +0x5d,0x2d,0x76,0x52,0x12,0x7e,0xb1,0x7d,0x93,0xe4,0xda,0x44, +0xfb,0x41,0x21,0xd8,0x0b,0x66,0xdf,0x9c,0x1a,0x7c,0x08,0xf4, +0x0c,0x9c,0xba,0x9d,0xcd,0x7c,0x12,0x25,0xe9,0xbb,0x76,0xce, +0x78,0xad,0x28,0x33,0x64,0x4f,0x18,0x81,0x0c,0x7b,0x9d,0x91, +0xd4,0x88,0x18,0x56,0x6d,0x21,0x0f,0xbd,0xf5,0x33,0x38,0xa5, +0x6d,0x90,0x7e,0x40,0xcd,0x1a,0x6d,0x47,0x49,0xaa,0x41,0xc4, +0x06,0x6d,0x32,0x1c,0x01,0xb2,0xd8,0x7a,0xbb,0x29,0xfd,0x53, +0xec,0x52,0xcb,0x8e,0xa7,0x53,0xb0,0xf9,0x55,0x57,0x5c,0x29, +0xb9,0xd0,0x5b,0x2c,0x62,0x3f,0x01,0xd5,0xbd,0x86,0xb0,0x24, +0x80,0xc6,0x85,0x8b,0xf1,0x07,0x59,0x8e,0x98,0xf9,0x58,0x92, +0x03,0x01,0x7d,0x51,0x9f,0xd7,0x04,0x0e,0x1b,0x42,0x9d,0xf5, +0x0c,0xd6,0xd0,0xc2,0x59,0xc1,0x02,0xe7,0xfa,0x05,0xeb,0x50, +0x63,0x38,0xe2,0x01,0xa5,0xe1,0x0c,0xad,0x9d,0x75,0x01,0xb7, +0xc2,0x92,0x0d,0xe4,0xa6,0x40,0x3e,0xd8,0xda,0xd2,0x22,0xd5, +0x01,0x7a,0x8f,0x4f,0x48,0x8d,0xfd,0xeb,0xbc,0x98,0xea,0xa6, +0x6b,0xef,0x00,0xd8,0x4e,0xd4,0xfa,0x20,0x54,0xd4,0xd6,0x43, +0x8d,0xc0,0xb0,0xc4,0x9b,0x98,0x40,0x86,0x59,0x9a,0xfd,0xe5, +0x98,0x7c,0xbf,0xf1,0x0e,0x05,0x0d,0x84,0x80,0x08,0xc8,0xc9, +0x34,0x71,0xb1,0x30,0x19,0x42,0xb4,0x5d,0xb8,0xc0,0xc6,0x5f, +0x0d,0x96,0x61,0xc5,0x63,0xe8,0x5f,0xb6,0x3f,0xa2,0xbd,0x33, +0x7e,0x5d,0x56,0x3b,0x30,0x40,0xce,0x6e,0x63,0x7e,0x0c,0x1d, +0xaf,0x7e,0xc9,0xf0,0xc5,0x6c,0x19,0xa0,0x71,0x9a,0x0a,0xb6, +0x3a,0xe0,0x11,0x1a,0x01,0xf4,0x9f,0xde,0x18,0x8f,0x73,0x2c, +0x26,0x9c,0xe6,0xdd,0x5f,0xf6,0x77,0xbb,0x3e,0xf9,0x84,0xde, +0xf8,0x2d,0xfb,0xc5,0x1c,0x85,0xeb,0x12,0xfe,0x57,0xc1,0x59, +0x31,0x3a,0x8b,0x46,0xbc,0x6c,0x70,0x9e,0xfd,0xb2,0xb5,0xe5, +0xef,0x87,0xa8,0xe3,0xb0,0x0f,0x37,0x77,0x13,0x0d,0xb0,0xdd, +0x61,0x33,0xb2,0x5f,0xb6,0xb7,0x97,0x5a,0x30,0x59,0xd6,0x37, +0xb2,0x1b,0x21,0x14,0x5e,0x8c,0x26,0x31,0xe8,0xed,0x69,0x8e, +0xf9,0x9b,0x38,0x12,0xaf,0xdb,0x78,0x06,0x13,0xef,0x1c,0xf7, +0xe9,0x2c,0x73,0x27,0xe9,0x51,0xba,0xdd,0x3c,0xfc,0x7c,0xdc, +0x5f,0x1c,0x0f,0xee,0xed,0x5c,0x28,0x2d,0x48,0xa5,0x41,0xe6, +0xf1,0x31,0xa5,0xfb,0xc2,0x2f,0x55,0xfc,0xc2,0x97,0x31,0xc5, +0x66,0xce,0x6e,0xdc,0x58,0x2e,0x64,0x22,0xce,0x03,0x7a,0xad, +0x50,0x56,0x7b,0xab,0xdf,0xb3,0xd7,0xb7,0x97,0xb7,0x09,0x07, +0x96,0x77,0x3b,0x19,0x93,0x3b,0xe8,0xad,0xce,0x4e,0x1b,0xda, +0xcd,0x00,0x54,0xef,0xb2,0xc2,0xc6,0xfd,0x56,0xa8,0x71,0xa2, +0x48,0xee,0x24,0x1d,0x7d,0x94,0xdd,0xca,0x0c,0x26,0x61,0x84, +0x69,0xe8,0xe8,0x99,0x3c,0x2b,0xcf,0xb2,0x04,0xcb,0xf7,0xdb, +0xc5,0xcd,0x03,0xd6,0xf6,0xb8,0x46,0x27,0x40,0xa3,0xdb,0x31, +0xe5,0x4c,0x47,0x5f,0xb7,0xcd,0x5a,0x93,0xf4,0xb8,0x91,0xb3, +0xfc,0xd3,0xcd,0x70,0xcc,0x6c,0xde,0x6c,0x02,0xae,0x59,0x91, +0x9c,0x30,0x02,0xd7,0x42,0xf4,0x0e,0xac,0xaf,0x21,0x27,0x6b, +0xee,0xf6,0x46,0x6c,0x7a,0xe5,0x4b,0x52,0x53,0xa7,0x59,0x78, +0x42,0x09,0xb1,0x8e,0x4e,0x28,0x21,0x14,0xf3,0x3c,0x18,0x5f, +0x4d,0x34,0x4b,0xd4,0xa5,0x5f,0xb8,0x2e,0x0e,0x47,0x3c,0x14, +0x6c,0x5f,0x79,0x70,0x0c,0xb1,0x58,0xe0,0xef,0x92,0xbc,0x05, +0x7c,0x85,0x9d,0x88,0xf1,0xee,0xc3,0xb1,0xe0,0xbc,0x0c,0xc0, +0x7d,0x27,0x2d,0x5e,0x6f,0x6d,0xd5,0xc3,0xcd,0xf2,0xd2,0x10, +0x9c,0x74,0x13,0x6e,0x75,0x1e,0xee,0x66,0xfa,0xd4,0x96,0x54, +0x2b,0xbe,0x27,0x51,0x77,0x12,0xa5,0x8a,0xed,0xcc,0x5d,0x65, +0x43,0x40,0x6f,0xe5,0x78,0x4b,0x61,0x62,0x2f,0xf2,0x4f,0x1a, +0x0b,0x1d,0x35,0xcc,0xe0,0xaf,0xd1,0xe5,0x93,0xbd,0x87,0x31, +0x30,0xc7,0x2c,0x36,0x2e,0x7f,0xa6,0xff,0x67,0x4d,0xb6,0xd1, +0x14,0x0f,0x8b,0x1f,0x5d,0x4e,0xf3,0xfc,0x59,0x71,0xa1,0x37, +0xd9,0x64,0xe5,0xfd,0x2f,0xfd,0x3f,0x36,0x0a,0xff,0x5e,0xff, +0x8f,0x1f,0x2f,0xf4,0xff,0x6a,0x4a,0xbf,0x98,0xcc,0xa7,0x61, +0x61,0x59,0x5a,0x16,0xaf,0x2b,0x7f,0x58,0x7c,0x92,0xc5,0x0f, +0x19,0xfc,0xd0,0x3d,0x98,0xa7,0x7a,0xea,0xdf,0x8c,0xf5,0x6f, +0xe5,0x13,0x0e,0x0e,0x0e,0xb8,0xd4,0x81,0x7b,0x1c,0x8d,0xec, +0x83,0x78,0x1a,0xd5,0x20,0xfd,0x71,0x32,0x1d,0x11,0xa2,0x21, +0x03,0x3e,0xb1,0x50,0xd5,0xc5,0xdb,0x48,0xa9,0x57,0x49,0xfb, +0xe4,0x3a,0x9f,0x16,0xa3,0x22,0xbf,0xc6,0x5e,0x22,0x14,0xff, +0xac,0x6b,0x73,0x73,0x8b,0xf3,0x80,0x09,0xfe,0x89,0xeb,0xf9, +0xe9,0xa7,0x06,0xe8,0xdf,0x27,0x63,0xc6,0x77,0x54,0x07,0x61, +0x3e,0xdb,0xb6,0x15,0x3f,0x88,0xbe,0xb0,0x1d,0xc1,0x0f,0x3f, +0xfc,0xc0,0x0f,0x97,0x97,0xfc,0x70,0x7d,0xcd,0x0f,0x65,0x69, +0x5a,0xcf,0xe0,0x33,0xed,0x65,0xca,0x9a,0xa2,0xa6,0xa4,0x29, +0xc8,0xbf,0x1f,0xcd,0xaf,0x79,0xf8,0xd1,0xfc,0x9a,0x87,0x9c, +0x7f,0x9f,0xd7,0xf7,0xe9,0x9b,0xe9,0xd1,0xc7,0x09,0xf7,0x28, +0x5b,0xb1,0xb8,0xbb,0x10,0xc2,0x42,0x65,0x0a,0x13,0xe1,0xe6, +0x39,0xd9,0x4a,0x18,0xd3,0x34,0xe6,0x0c,0x38,0xf6,0x9d,0x18, +0x89,0x26,0x3b,0x7a,0x45,0x57,0x51,0x01,0x56,0xb1,0x62,0xd3, +0xcf,0xcd,0x7b,0x40,0x3a,0x5a,0xd0,0xb2,0xb3,0x11,0xb5,0x31, +0x5e,0xc3,0xe3,0xf9,0xdf,0xcf,0x2e,0xe7,0xe3,0x0f,0x59,0x5c, +0xed,0x97,0x76,0xfe,0x29,0x3f,0x33,0xc0,0x8b,0x45,0x7f,0x90, +0xc0,0x31,0x2e,0xb8,0x5b,0x28,0xb3,0x98,0xcb,0xb5,0x74,0xd5, +0xbc,0x4c,0x12,0x02,0x2e,0xfb,0x14,0x32,0xf5,0x96,0x5d,0x17, +0xd3,0xf2,0x86,0xda,0x55,0xba,0xa4,0xd1,0x81,0xb7,0xe2,0xaf, +0x77,0x1f,0x20,0x9a,0xbe,0x16,0x6b,0x5a,0xa4,0x9c,0xa0,0xf7, +0x3d,0xbb,0x96,0x45,0x2d,0xd0,0xdf,0x52,0xea,0xee,0xa0,0xb7, +0xcd,0x85,0x53,0xfe,0xf5,0x8e,0x14,0xa0,0xd2,0x23,0x72,0x48, +0x08,0x5f,0x29,0xb9,0x07,0xf9,0x55,0xf3,0x78,0x08,0x58,0x18, +0xc0,0xbe,0x6c,0x46,0x96,0xdb,0x66,0x98,0xd7,0xf2,0x16,0x37, +0xdd,0xa2,0xd4,0xa8,0x6a,0x70,0x5b,0x12,0x7b,0x48,0x78,0x4b, +0xcc,0x5a,0x55,0x62,0xdd,0x5a,0x38,0xea,0x92,0x74,0x54,0x3b, +0x75,0x87,0xd9,0xca,0x19,0x47,0xda,0x76,0x9c,0x6e,0x8c,0x8f, +0x55,0x45,0xc3,0x5e,0x6d,0xcd,0xc3,0xd4,0x6d,0x2a,0xbd,0x3d, +0x15,0x11,0x2c,0x29,0x79,0x26,0x26,0xd3,0xaa,0x6f,0x42,0x45, +0x80,0xf7,0x49,0xde,0x07,0x3d,0x7b,0xe6,0x10,0x55,0x51,0x19, +0x6a,0x46,0x66,0x9b,0xdd,0x8c,0x08,0xe6,0xb2,0x5f,0x33,0xea, +0x1a,0xbc,0x02,0xad,0x58,0xbe,0xed,0x7f,0xfd,0xf7,0x5e,0xe7, +0xdb,0xdd,0xdd,0x74,0x2f,0x7f,0x18,0xa0,0xb2,0x8c,0x81,0x19, +0xf6,0x5d,0x88,0x65,0x69,0xc1,0x14,0x0d,0xf9,0x45,0xf9,0xf6, +0x7a,0x75,0x47,0x69,0x88,0x83,0x1a,0x5c,0x3f,0x54,0xb8,0x91, +0xe0,0x4a,0x3e,0x51,0xa8,0x2e,0x69,0x24,0xea,0x5a,0x30,0xb1, +0xa0,0xe7,0x49,0x87,0xd2,0x58,0xb2,0x14,0x5c,0xcf,0x2f,0xc9, +0x5a,0x92,0xc6,0x92,0xf5,0xab,0x55,0x58,0xbb,0x53,0xb7,0x30, +0x22,0x38,0xe2,0x8d,0xa3,0xdd,0x36,0x58,0x6b,0x55,0x7a,0xf9, +0x9f,0xae,0x51,0x47,0xf6,0xda,0x3f,0x16,0x11,0xd6,0xf3,0x35, +0xe5,0xe4,0x0a,0x62,0x10,0xe8,0xed,0xf9,0xfb,0xa3,0xa7,0x60, +0x54,0x61,0x52,0x66,0xbf,0x5f,0x67,0xcd,0xfc,0xae,0xfa,0x85, +0x77,0x33,0xf2,0xd1,0x5d,0xab,0x6b,0xc8,0xee,0x91,0x6d,0x64, +0xc4,0x9c,0xcb,0xf9,0xa9,0x66,0x9b,0x7a,0xcf,0xd5,0xe1,0x6b, +0x6d,0x76,0x75,0x22,0x71,0xc1,0x5f,0xba,0x84,0x38,0xc1,0xa2, +0x45,0x2d,0x2e,0x88,0x42,0x10,0xcc,0xae,0x8f,0x96,0x7b,0x7d, +0xc4,0xb8,0xfd,0xf6,0xad,0x8f,0x65,0xf9,0x98,0x40,0x2e,0x0a, +0xd0,0x89,0xd0,0x32,0x7e,0x58,0x4c,0xde,0x02,0x41,0xa6,0x3a, +0x9b,0x4f,0x21,0x94,0x3d,0xfa,0xf0,0x00,0xbd,0xe4,0xd1,0xe4, +0x7d,0x99,0xab,0xf3,0xe2,0x13,0xcc,0x62,0xf5,0x11,0xcd,0xa8, +0x68,0x37,0x0f,0x07,0x03,0x7a,0x7e,0xe3,0x2f,0x91,0xc6,0x5a, +0xf7,0x4d,0xdb,0xd3,0xb4,0x27,0x16,0x28,0x33,0xf1,0x4c,0xbb, +0xee,0x33,0x2f,0x8c,0xa8,0xa6,0x5d,0x6f,0x43,0x51,0xe1,0x64, +0x19,0x33,0x71,0xa2,0x6a,0x32,0xb1,0x3e,0xdd,0x1c,0x4c,0x9c, +0x67,0xfd,0x2f,0xb5,0x26,0x60,0xb8,0x6c,0x0e,0xae,0x1f,0xf6, +0x70,0xbc,0xc1,0x78,0xd5,0xa9,0xaa,0xb3,0x8b,0xdc,0x24,0xea, +0x7c,0x1b,0xb5,0xf4,0x7b,0x1a,0xed,0xed,0xd2,0x03,0xfc,0x0b, +0x2b,0x44,0xf6,0xd1,0xb3,0x27,0x00,0xf0,0xdd,0xa8,0xe9,0x4d, +0xd2,0x00,0x66,0xa9,0x44,0xcf,0x30,0xe3,0xfe,0xd8,0xfe,0xfe, +0x7b,0xba,0xba,0x09,0xcf,0x3f,0xba,0xc7,0xe7,0x3d,0x68,0xd2, +0xcc,0x72,0x4a,0xe8,0x9e,0x1f,0xc1,0xee,0xc1,0x7c,0x57,0x0c, +0x45,0x13,0x85,0x85,0x3a,0xfa,0xe7,0xb9,0x7a,0xa4,0x07,0x56, +0x6a,0xf7,0xf9,0x8d,0xcc,0xc9,0xf4,0x50,0x66,0xaa,0x1a,0xd1, +0xc7,0xff,0x48,0xc9,0x1a,0xef,0x88,0x36,0x2d,0xa9,0x01,0xc8, +0x5d,0xeb,0xe8,0x17,0x68,0x9d,0x16,0x6e,0x4e,0x4e,0x00,0x53, +0x7b,0x34,0xf9,0x98,0x92,0x1d,0x4b,0x26,0xc0,0x3e,0x02,0x18, +0x50,0x66,0x6b,0xd9,0xda,0xe2,0x8a,0x1f,0xfb,0x85,0x29,0xbd, +0xd5,0x52,0x77,0x7f,0xf0,0xc5,0x05,0x51,0x6f,0x07,0x99,0x87, +0x29,0x78,0xfd,0xe8,0x82,0xeb,0x9b,0x1e,0xc8,0xa0,0x0a,0x34, +0xcd,0x51,0x35,0x4b,0x11,0xe4,0xd9,0xd7,0xa4,0x0a,0xa1,0xc7, +0x94,0x1d,0xfa,0x59,0xfd,0x28,0x10,0xe3,0x98,0x52,0xc2,0x31, +0x50,0xc5,0xba,0x1f,0x1c,0x2b,0x20,0xfa,0x44,0x4c,0xe7,0x5a, +0xfd,0x34,0xc4,0x99,0xc2,0x99,0x89,0x7b,0x59,0x73,0x58,0xe1, +0x66,0xe6,0x6e,0xb5,0x22,0xd9,0x1a,0x2c,0x1f,0x00,0x02,0x3e, +0xec,0x20,0xa5,0xbe,0x84,0xa1,0x95,0x5e,0x80,0xd0,0x6e,0x97, +0x2e,0xba,0x3c,0xdc,0x2f,0xaa,0x53,0x4f,0xef,0xf9,0x5b,0xad, +0x22,0x91,0x09,0xc4,0x6b,0xe0,0x41,0x36,0x4d,0x41,0x6e,0x05, +0xba,0xdf,0xb0,0xc2,0xa1,0xbe,0x44,0x15,0x7d,0x0f,0xce,0x36, +0x8a,0x5e,0x27,0xdd,0x4d,0xbd,0x5a,0xa9,0x08,0x2e,0x99,0x2d, +0x5e,0x76,0x62,0xb1,0x08,0x2c,0x16,0xbb,0xc9,0xce,0xd7,0xbb, +0xcc,0x29,0xcd,0x12,0xd9,0x04,0xf9,0x37,0x80,0x74,0xab,0x52, +0xec,0xaf,0x2f,0x3d,0xd1,0xde,0x29,0x3c,0xc3,0x43,0x22,0x9d, +0x3e,0x35,0x72,0x57,0xbe,0x9e,0x27,0xb9,0x6b,0x70,0x2a,0xdb, +0xb5,0xd5,0x2e,0x16,0xe1,0x61,0x6f,0xbd,0x19,0x9a,0x15,0x49, +0x44,0xcf,0x65,0xfd,0x3a,0x63,0xb4,0xf0,0xfc,0x9a,0x84,0xc9, +0x4a,0x2a,0xcc,0xaa,0x3a,0x9b,0xb2,0x6a,0x71,0x5c,0x50,0x2b, +0xc9,0x64,0x4e,0x52,0x03,0x6d,0x4d,0x4d,0x06,0xaa,0x7e,0xb9, +0xf1,0x02,0xb5,0xd7,0x2f,0x07,0x55,0xf7,0x4f,0xce,0x5b,0x93, +0xec,0x23,0xdd,0x00,0x1f,0x79,0xd0,0xbe,0xd0,0x5d,0x42,0xec, +0x58,0xb9,0x44,0x33,0xd8,0x5d,0x0a,0x8d,0xed,0x41,0x6a,0x4a, +0x56,0x8b,0x05,0x65,0xb8,0x80,0xaf,0x4a,0x7c,0xc6,0x1f,0x46, +0x12,0xc6,0x13,0xab,0x0a,0x0a,0x8f,0x78,0x86,0xe0,0xe9,0x57, +0xcc,0x6f,0x3c,0xfc,0x80,0x0b,0xca,0xb4,0xe2,0x22,0x1b,0xa5, +0x86,0x53,0x33,0x52,0xb3,0xd0,0xbe,0xa5,0xfc,0x50,0xa0,0x05, +0xea,0xdd,0x3c,0x9e,0xf6,0x67,0x59,0x14,0xb5,0xec,0x08,0xe1, +0x34,0xb2,0xce,0xe2,0x5b,0x49,0x46,0xcb,0x3c,0x9b,0xcc,0x86, +0x57,0x6f,0x5d,0xb5,0x0c,0x44,0x96,0x26,0x44,0x82,0xaf,0x84, +0x33,0x58,0xcf,0x69,0xa9,0x68,0xd2,0x89,0xc2,0xfe,0x10,0xf9, +0x45,0xf1,0x78,0x26,0x4f,0x4e,0x70,0xd6,0x0b,0x79,0x06,0x1d, +0xa8,0x88,0xcf,0xce,0xe2,0x35,0x94,0x3f,0x8d,0x3b,0x51,0x83, +0x05,0x4c,0xfe,0xa9,0xc5,0xcc,0xe7,0x5a,0xb1,0x89,0xdf,0x8d, +0x5d,0x82,0x28,0xa4,0x19,0x1c,0x17,0x32,0x3a,0xef,0x5d,0x50, +0x79,0xdb,0xbe,0x12,0x27,0x70,0x74,0xd4,0xce,0xd0,0xb6,0xcd, +0x4d,0x55,0xa8,0x60,0x5f,0x51,0x4f,0x4b,0xbc,0x58,0x0b,0x9f, +0xfa,0x8e,0x68,0x65,0x75,0xb0,0xb5,0xea,0x61,0x12,0xf1,0x7a, +0xb2,0xa6,0x5e,0x75,0xa4,0x75,0xd2,0xca,0x17,0xb1,0x72,0x0e, +0x3f,0x89,0x36,0xbb,0x6a,0xf5,0x26,0x5a,0x8e,0x52,0x33,0x85, +0x2d,0x5a,0x73,0xe0,0xb6,0xe9,0x75,0xc7,0x1a,0x95,0x76,0x05, +0x8c,0x77,0x16,0x97,0xc9,0xe1,0xbb,0x5d,0xdf,0x4e,0xca,0x1b, +0xd5,0x77,0x76,0x1c,0xdb,0x62,0xca,0x6d,0x9d,0x2b,0x61,0xce, +0xaf,0x3b,0x7b,0x62,0x2d,0x1e,0x9a,0x45,0xa6,0xb3,0xe7,0x97, +0xc3,0x3b,0xb7,0x5b,0x5b,0x1d,0x58,0xa0,0x02,0xe0,0x6a,0xf1, +0x8c,0xd7,0xee,0x2a,0x0f,0x54,0xb5,0x56,0xcc,0xf2,0xec,0xcd, +0x53,0xfa,0xb8,0x53,0xc0,0x52,0x9e,0x72,0xc4,0xc7,0xfd,0x64, +0x01,0x3f,0x03,0xfd,0xd3,0x8f,0xfb,0x3f,0x1f,0x0f,0x8e,0xfb, +0x83,0x07,0xc9,0xf1,0x00,0x52,0xdb,0xc9,0xce,0x45,0x70,0xa3, +0x47,0xb3,0x94,0x9b,0x8e,0xba,0xd9,0x53,0x37,0x0f,0xd5,0xcd, +0x23,0x8b,0xf4,0xa6,0xb3,0x58,0xdc,0xec,0xe9,0x7f,0x0f,0xf5, +0xbf,0x47,0xcb,0x44,0x1e,0xd9,0x08,0x4d,0x54,0x2d,0x15,0xfd, +0xed,0x63,0x4d,0xc9,0xcf,0xf7,0x1e,0xb4,0x7a,0xed,0x38,0x59, +0xf4,0x8f,0x07,0xb7,0xcb,0x01,0x1c,0xbf,0x1c,0x1f,0xdf,0xdb, +0x0a,0x8f,0x60,0x2a,0x7c,0xb3,0xca,0xf5,0x41,0x3a,0xa3,0xb6, +0x52,0xa7,0x79,0xc9,0xa7,0x07,0x0a,0x2f,0xfc,0x1e,0x4d,0xbe, +0xcb,0xc1,0x2a,0xc8,0x6c,0x52,0x0e,0x21,0xd1,0x98,0x2d,0x59, +0x46,0x65,0xa6,0x4d,0xb0,0x62,0x84,0xa7,0xc8,0x72,0xff,0x28, +0xb6,0xa4,0xbb,0x3b,0xba,0xcf,0xf8,0x9a,0xa4,0x39,0x33,0x29, +0x1e,0x87,0xc8,0x49,0x98,0x11,0x54,0x64,0xbb,0xca,0x11,0x9e, +0x39,0x0f,0x32,0xa6,0xbb,0x1b,0x4c,0x02,0x5c,0x99,0x44,0x94, +0xd7,0xf5,0xb8,0xcf,0x01,0x86,0xb9,0x6a,0xc9,0x91,0x28,0xcc, +0x81,0x8a,0x48,0x83,0x41,0x29,0xe8,0x6c,0x65,0xb2,0x9a,0x70, +0xea,0x29,0x1f,0xb4,0xb3,0xfb,0x20,0x80,0xae,0x39,0x1b,0x57, +0x01,0x08,0x85,0xc1,0x97,0x88,0x14,0x5f,0x45,0x15,0x7d,0xb8, +0x58,0x88,0x97,0x7d,0x09,0x8c,0x3e,0x31,0x5c,0x9e,0x8f,0xc8, +0x0d,0x08,0xf1,0x19,0x49,0xd2,0xe5,0xf6,0x3e,0x0b,0x87,0xcd, +0x62,0x21,0xc0,0x56,0x8d,0xc4,0x60,0x27,0xcc,0x0c,0xde,0xad, +0xa8,0x38,0x79,0xb2,0xa2,0x9c,0xe0,0x2a,0xe5,0xad,0x47,0x30, +0xfe,0xf8,0xb6,0x1c,0x8d,0x97,0x47,0xdd,0x62,0x9f,0x9c,0xce, +0x99,0x1c,0x38,0xa3,0xb3,0x22,0xc1,0x39,0xa0,0x81,0xda,0xf5, +0x52,0xb8,0xdd,0x19,0xb4,0x18,0xe4,0xeb,0xc1,0x62,0x11,0x6d, +0x44,0xac,0xa6,0x58,0x9a,0xa1,0xf7,0x48,0x44,0xec,0x2e,0x27, +0xa8,0x60,0x27,0x4f,0xe5,0x1e,0x0d,0x02,0x7b,0x2b,0x93,0x70, +0xbb,0x03,0xc6,0x77,0x87,0x7a,0x57,0xb0,0x2e,0x8d,0x21,0xda, +0xf8,0x29,0x4a,0x56,0x0e,0x3a,0xc3,0xaf,0x30,0x14,0x79,0xcd, +0x2c,0x62,0xb4,0xf5,0x4d,0x8e,0x9c,0xd9,0x6f,0x71,0x14,0x05, +0x82,0xe6,0xd6,0x8b,0xf9,0xb0,0xbc,0x79,0x9d,0xcf,0xfe,0xab, +0x9c,0x8c,0x45,0xbb,0xb3,0x3a,0x87,0xca,0x64,0xce,0x6a,0xaf, +0x57,0xa5,0x23,0x65,0x44,0x35,0x59,0x71,0x8b,0xf3,0x40,0x09, +0x9d,0x46,0x44,0xb0,0x33,0x14,0x44,0xec,0xbd,0xe6,0x81,0x92, +0xfa,0x76,0xcd,0x3d,0xc7,0xf9,0x69,0x5b,0x63,0x8e,0xdd,0x9b, +0xf8,0x3f,0x20,0x10,0x16,0xcc,0xb5,0x04,0x72,0x84,0x10,0xe3, +0x12,0x28,0xa0,0xad,0x7e,0xb3,0x51,0xa7,0x65,0x0b,0xcd,0x9c, +0xcd,0x57,0xc5,0x9f,0xd5,0xb5,0x1a,0xa9,0x4b,0x75,0xa0,0x4a, +0x75,0xcd,0x96,0xbb,0xb8,0xd5,0xb4,0x45,0x03,0x08,0x23,0x85, +0x77,0xbe,0xfd,0x66,0x77,0xff,0xf3,0xd6,0x16,0xee,0x19,0x4b, +0x21,0x44,0x7f,0xa6,0xef,0xf5,0xeb,0xb2,0x9b,0xd6,0xba,0x1a, +0x8c,0x6d,0x5e,0xe0,0x4d,0xd7,0xdc,0xbb,0x6a,0xaa,0x51,0x0a, +0xfc,0x95,0x4a,0x3d,0x2d,0x07,0x1d,0x01,0x18,0xab,0x10,0xf2, +0xb4,0x5f,0xdb,0xdb,0x34,0xc7,0xc0,0x98,0x89,0xf5,0x90,0x00, +0x4a,0xa3,0xcc,0x94,0x6e,0xb3,0x3a,0xc2,0xd7,0xcc,0x57,0x1d, +0xf7,0x63,0xba,0x30,0x9f,0xa2,0x79,0x42,0xb8,0xac,0xea,0x84, +0x08,0xeb,0xec,0xda,0x6f,0x2c,0x7c,0x7b,0x5d,0x14,0x61,0x8b, +0x99,0xde,0x7b,0xc1,0x94,0x7d,0x72,0x31,0xe1,0x59,0xa5,0xf8, +0xe2,0x06,0xc7,0x54,0x39,0x9c,0x9f,0x9f,0x17,0x9f,0x30,0x42, +0xa9,0xc6,0x92,0xf3,0x31,0xb4,0x3c,0x5a,0x9e,0xe6,0x57,0xe8, +0x9e,0x06,0x6d,0xb3,0xa8,0x30,0xe8,0x6b,0x36,0x37,0x7d,0x0c, +0x8c,0xdd,0x20,0xf2,0x84,0x00,0x81,0xc0,0xbb,0x50,0xe1,0x63, +0xa0,0xaa,0xc9,0x49,0x38,0xdd,0xc3,0x98,0xe2,0xe5,0x15,0x6b, +0xf3,0x2e,0xcb,0x26,0x3b,0x9d,0xfc,0x61,0x62,0x4f,0x85,0x08, +0x92,0xf3,0x70,0x0f,0x4f,0xd7,0x23,0x28,0x9d,0xa1,0x30,0x1d, +0x2f,0x71,0x50,0x32,0x82,0xec,0xec,0x3d,0x32,0x77,0xb3,0x28, +0x15,0x00,0x76,0x1e,0x7e,0xfd,0x55,0x42,0xfe,0x69,0x1f,0x7d, +0xb5,0x6f,0xaf,0x3e,0xf4,0xa3,0x32,0x32,0xf7,0xa1,0xc0,0xaf, +0x3d,0x18,0x7d,0x11,0x6a,0xc8,0xbb,0x8e,0x74,0x9a,0x06,0x97, +0x49,0xd7,0x91,0x21,0x91,0xe1,0xb1,0x4e,0xc8,0xba,0x04,0xe8, +0xbd,0xbd,0x7d,0x97,0x70,0x19,0x11,0xd1,0x0c,0x49,0xb7,0x39, +0xfa,0xd1,0x08,0x01,0xbf,0xda,0x77,0x09,0xa3,0x08,0xbf,0x82, +0x6a,0xb3,0xc9,0x07,0x00,0xf7,0x50,0xa7,0xd8,0x84,0x83,0x48, +0xc9,0x4f,0xda,0x4d,0x18,0x33,0x7e,0x2d,0x40,0x7c,0x86,0x22, +0xfa,0xe7,0x73,0x44,0x2d,0xe0,0xfb,0x31,0xdd,0x1b,0x64,0x7e, +0x07,0x61,0xea,0xc3,0x41,0x26,0xfb,0x61,0x7f,0x97,0x92,0x1f, +0x0d,0x70,0x98,0xab,0xca,0xb0,0xe3,0x99,0xa9,0x85,0x19,0x72, +0xdc,0x6a,0x87,0x05,0x4c,0x06,0xd2,0x38,0x91,0x59,0x08,0x04, +0xcc,0x42,0xbd,0x15,0x4c,0xbd,0xe0,0x95,0x35,0x53,0x74,0x04, +0x37,0xfa,0x65,0x5e,0xce,0xf2,0x11,0x8b,0x76,0xe8,0x22,0xb2, +0x06,0x78,0x3b,0x40,0x08,0x6d,0x70,0x34,0xb1,0xef,0xf5,0x65, +0xe8,0xca,0xcc,0xe7,0xd8,0x4e,0xaa,0xa0,0xd4,0x7e,0x0e,0xce, +0xfe,0xe3,0x20,0x75,0x3b,0xfb,0x26,0x01,0x3a,0xb6,0xbf,0xd9, +0x0f,0x72,0xaa,0xb0,0x2d,0x80,0xf5,0x3f,0xc1,0x58,0x85,0x83, +0xd9,0x4d,0x5b,0xef,0xa5,0xe2,0x88,0x3b,0x58,0x14,0x4b,0xd4, +0x2d,0xb4,0x97,0x30,0x6e,0xf6,0x71,0x38,0x85,0x64,0x9c,0xec, +0x7c,0x43,0x03,0x3a,0x0d,0x40,0x3e,0x63,0xae,0xa7,0x70,0xaa, +0xe8,0x50,0x51,0x03,0xe4,0xa9,0x4e,0xeb,0xda,0x29,0x48,0x34, +0x8c,0x19,0x28,0x7e,0x32,0x1a,0x29,0x8b,0x57,0x8d,0xaa,0x9c, +0x9a,0xaa,0x90,0xd6,0xd3,0xa2,0xb9,0x03,0x95,0x33,0xbd,0xf5, +0xf8,0x37,0xad,0xed,0x51,0x5d,0x61,0xd0,0x97,0xdb,0xa3,0x56, +0x3c,0xda,0xaf,0xa3,0xbb,0xf7,0x4d,0x4a,0x1c,0x80,0x95,0xa3, +0xdf,0x3c,0x88,0x01,0xf7,0x76,0x27,0x69,0xc5,0x5c,0x49,0x30, +0x6c,0xe8,0xea,0x14,0xd6,0xd2,0xea,0xa8,0x5b,0x6c,0x56,0xa7, +0x92,0xdd,0xed,0x61,0x02,0xfc,0xd9,0xee,0x38,0xbc,0x1e,0x84, +0x97,0x2a,0x15,0xb8,0x50,0xab,0xcd,0x5c,0xfa,0x6b,0x5e,0x70, +0x67,0xb5,0x4e,0xc8,0x61,0xe3,0x1b,0x2b,0x77,0x99,0x93,0x6f, +0x2b,0xd0,0xb8,0xf5,0xc9,0xed,0x61,0xb6,0xb6,0x56,0x1b,0x17, +0x73,0x40,0xb4,0x8c,0xe5,0x17,0x1a,0x37,0xbc,0xef,0x89,0x6f, +0x85,0xa5,0xf0,0xee,0x32,0x49,0x1b,0x16,0x42,0x21,0x0c,0x16, +0x84,0xa8,0x46,0x05,0x95,0xb4,0x6f,0xf4,0x16,0x50,0x2c,0x83, +0x78,0x09,0x0c,0x6b,0x2b,0xf9,0xe3,0x7d,0xd1,0x48,0x6c,0x8a, +0x78,0xdd,0x6c,0x94,0x79,0xe0,0xe4,0x28,0x49,0xa9,0x7d,0x7a, +0x46,0xd0,0x62,0xa3,0xa5,0x1a,0x99,0x2c,0xdc,0x46,0xa6,0x6b, +0x48,0xae,0x69,0xa3,0x40,0xaa,0x80,0x18,0xbf,0xfb,0x02,0xd9, +0xe9,0x7b,0xbc,0x72,0xa1,0x51,0x1e,0xd2,0xdd,0x0b,0xb8,0x6a, +0x4e,0xee,0xae,0xac,0xe5,0x23,0x5f,0x3f,0xf7,0xdf,0x5b,0x7a, +0xf1,0x11,0x77,0x5c,0xd8,0x08,0x0c,0x86,0xc6,0x7c,0x76,0xc6, +0xb7,0xf1,0x8b,0x12,0xd4,0x9a,0x91,0xfe,0x13,0xa5,0xce,0x7f, +0x8c,0x38,0x5c,0xef,0xf1,0x1d,0xf5,0x91,0x5e,0xca,0xf2,0x59, +0xd4,0xd2,0x45,0x5b,0x60,0x91,0x6a,0x84,0x91,0xba,0x7b,0x7c, +0x74,0x8d,0x1c,0xff,0xa6,0xb6,0xf4,0x85,0x57,0x3a,0x09,0xc6, +0xaf,0xb9,0x9c,0xf9,0xbd,0xfb,0x44,0xfb,0x79,0xf6,0xaa,0xa4, +0xfd,0xce,0xa6,0x0b,0xed,0x70,0x2b,0x97,0x40,0x1a,0xd0,0x3f, +0x29,0x45,0x05,0xea,0x7c,0x78,0x86,0xce,0x54,0x6b,0xea,0x89, +0x86,0x55,0x9f,0x74,0xa6,0xb2,0x16,0x54,0xb6,0x43,0x85,0x83, +0x7a,0xbe,0x47,0x17,0x9d,0x1a,0xb2,0x6e,0x56,0xa1,0xeb,0x09, +0x3d,0xd8,0x91,0xd6,0x6b,0xc9,0xc9,0x93,0xa5,0xb9,0x07,0x4b, +0xef,0xaa,0x50,0xff,0x78,0xfe,0xee,0xf0,0xe5,0x9b,0xd7,0x59, +0xb4,0xd7,0x7e,0xd8,0xee,0xf0,0xba,0x4c,0xb7,0xf9,0xf0,0x51, +0xc1,0xa9,0x4f,0xb6,0xab,0xf0,0xc0,0x25,0xeb,0x28,0x38,0x54, +0xc9,0xf6,0x14,0xa8,0x81,0xb2,0x87,0x8a,0x4e,0x22,0xb2,0x47, +0x8a,0x8e,0xcc,0xb3,0xaf,0x94,0x38,0x40,0xcf,0xbe,0x76,0x26, +0xd8,0x70,0x87,0xd9,0x5a,0x3d,0x67,0x82,0x6a,0x2b,0x5c,0x5e, +0x63,0x16,0x5c,0xd8,0x84,0x5f,0xbd,0x25,0xba,0x99,0x4c,0x67, +0xa5,0x0a,0xb6,0x4a,0xd9,0xce,0xcf,0xc7,0x3b,0x3d,0x18,0xe0, +0xc7,0x71,0x7c,0xbc,0xdd,0x3b,0x1e,0xb5,0x92,0x9d,0x82,0xa1, +0x60,0x61,0x3f,0xbc,0x19,0x8e,0x05,0xb4,0x86,0xd1,0x33,0xb5, +0x97,0xc6,0xc7,0xa3,0x07,0xc9,0x71,0x5b,0x3f,0x43,0x89,0xe3, +0x14,0x7f,0x74,0x3a,0x3f,0x1d,0xb7,0x21,0xe3,0xf6,0xe1,0x32, +0xe9,0x25,0xbd,0x1d,0x05,0x3b,0x59,0xee,0x24,0x53,0x6b,0xac, +0xf1,0xbc,0x05,0x44,0xfa,0xbf,0xfe,0xee,0xf6,0xb7,0xaa,0x3d, +0x78,0x90,0xfc,0x8b,0x70,0xdb,0xf7,0x83,0xe0,0xfd,0x19,0xbe, +0x1f,0x79,0x69,0x3f,0xdc,0x51,0xe6,0x10,0x48,0x58,0xb8,0xf7, +0x1f,0x93,0x7b,0x3b,0x2a,0xd4,0x4b,0xc3,0x87,0xf5,0xfb,0x3f, +0x83,0x52,0x0c,0x74,0x64,0xf1,0xf1,0xb1,0xc6,0x72,0x30,0x59, +0x1c,0x1c,0xf4,0xe0,0xff,0x8b,0x67,0x93,0xc5,0xb3,0x67,0xf8, +0xa7,0x07,0xff,0x5f,0x8c,0x46,0xa3,0xde,0x48,0xff,0x4c,0x7a, +0x8b,0x8f,0xfd,0xc9,0xe2,0xe3,0xa0,0xb7,0xf8,0x51,0xff,0xfe, +0xa8,0x7f,0xd1,0xa2,0x64,0xc1,0x7f,0x16,0x17,0x17,0xf1,0xc5, +0xc5,0x05,0x50,0xf0,0xfd,0xf7,0xf1,0xf7,0xdf,0x7f,0x0f,0x4f, +0xf9,0xe2,0xf9,0x62,0xb8,0x78,0xb2,0xb8,0xbc,0xec,0x2d,0x7e, +0xf8,0xa1,0xb7,0xb8,0xbe,0xee,0x2d,0xca,0xb2,0xb7,0x38,0x3c, +0xec,0x1d,0xf6,0x16,0xff,0x5c,0xfc,0xfe,0x7b,0x6f,0xf1,0xd3, +0x4f,0xbd,0x05,0xea,0xe5,0x6a,0x2d,0x26,0x6b,0xc9,0x7d,0x75, +0xb4,0x78,0xf5,0xaa,0x07,0xff,0x5f,0x5c,0xdd,0x76,0xd4,0xa3, +0x25,0x14,0x6f,0x32,0x2b,0xcb,0x76,0x8e,0x47,0xc7,0x23,0xdd, +0x35,0xcd,0xb6,0x84,0x00,0xa2,0xf1,0x3c,0x5c,0x4a,0xa0,0x30, +0xdf,0xcf,0x75,0xe6,0x90,0x5c,0xf8,0x91,0x97,0x6d,0xad,0x1d, +0xb3,0x9d,0x7e,0xeb,0x78,0x7b,0xd0,0x43,0x98,0xaf,0x3d,0x18, +0x30,0x3e,0xd4,0xd9,0xba,0xb7,0x06,0x0f,0xfa,0xf7,0x87,0xdb, +0xbf,0x1f,0xcf,0x77,0x77,0x9f,0xec,0x6e,0xeb,0x9f,0xaf,0x5e, +0xbc,0xd0,0x7f,0xbf,0xd9,0x85,0x97,0x67,0xdf,0xc0,0xcb,0x8b, +0x6f,0xf1,0xe5,0xc5,0xb3,0xa7,0xf0,0xf2,0xec,0x05,0xbe,0xbc, +0x78,0xfe,0x62,0xd0,0x5a,0xf4,0x35,0xe8,0xd7,0x98,0xbb,0xfb, +0xb5,0x06,0xdd,0x19,0xb4,0xe2,0xe3,0xf2,0x41,0xcf,0x4f,0x1e, +0xb4,0x12,0x4d,0xc0,0xde,0x72,0xa7,0x50,0x55,0xc5,0x47,0xb6, +0xf3,0x93,0xc6,0x02,0x74,0x42,0x53,0xa5,0x3d,0xf8,0xeb,0x03, +0x66,0x3b,0x47,0x95,0x92,0x6c,0xd7,0xa8,0x3f,0xe1,0x98,0x3f, +0x51,0x57,0xdc,0xe6,0x96,0xe4,0xb9,0x60,0x67,0x5e,0xf9,0x40, +0x67,0x3c,0x5a,0x6e,0xd3,0x84,0x3a,0x1e,0x6d,0xc3,0x1f,0xdd, +0x9f,0x3f,0xc2,0xef,0x3d,0xf3,0xb0,0x8d,0x69,0xf0,0xa4,0x1f, +0x92,0x38,0x3e,0x5a,0x6c,0x24,0xf8,0x1a,0xa7,0xe2,0x2f,0x54, +0xa2,0x7b,0x54,0xff,0x97,0xf0,0xff,0xe3,0x90,0x7a,0x48,0xbc, +0x87,0x14,0xb0,0x6a,0x13,0xcd,0x9f,0xb6,0x0f,0x0e,0xb6,0x9f, +0x3d,0x3b,0xfa,0xe1,0x87,0xf4,0xfa,0x3a,0x2d,0xcb,0x9f,0x22, +0x65,0x14,0x4f,0x59,0x5f,0x00,0x44,0x0a,0xcd,0x4e,0xb6,0xfb, +0x3f,0x0e,0x7e,0xfc,0xd1,0x7b,0xd9,0x7e,0xae,0x5f,0x11,0x10, +0x2c,0xbc,0x06,0xca,0x28,0x96,0xb2,0x7e,0x3f,0x32,0x58,0xdb, +0x87,0x91,0xda,0x41,0xca,0x91,0x1c,0xfb,0xc7,0x36,0xcd,0xce, +0x40,0x39,0xe8,0x7a,0x58,0x07,0x52,0xc9,0xe7,0x2c,0x99,0xbe, +0x33,0xe0,0xa3,0x9f,0xc0,0xd6,0x50,0xcf,0x1d,0x6a,0x97,0x05, +0x36,0xc9,0xce,0x45,0x01,0xb7,0x79,0x3f,0x7d,0xa6,0x05,0xa6, +0xb4,0xcb,0x75,0x99,0xa1,0x07,0xd4,0xc5,0x0f,0xb0,0x53,0x5b, +0xb0,0xb5,0xd0,0xe2,0x90,0x36,0x43,0xfa,0xdd,0xed,0x8c,0x22, +0x73,0xc7,0x62,0x81,0x9e,0x07,0x8b,0x99,0xc8,0x7c,0x81,0xab, +0x8d,0x66,0xd7,0xb2,0x40,0xda,0x51,0x87,0xe6,0x29,0xd7,0x4c, +0x9f,0x6d,0x16,0xbf,0xce,0x1f,0x29,0xac,0x2d,0x05,0x6f,0x17, +0x4a,0x8b,0xa6,0x65,0xfa,0xf7,0xaf,0x1f,0xe9,0x47,0x3c,0x6a, +0x2c,0xd3,0xbd,0xaf,0xbe,0xdd,0xcb,0xbf,0x56,0x20,0x46,0x6a, +0x98,0xce,0x57,0x1a,0xec,0xeb,0xa5,0x12,0x77,0xa1,0xb3,0xdb, +0xeb,0x32,0x8d,0xc4,0xae,0x4d,0x6f,0x62,0xd3,0xc8,0x3c,0x5e, +0x43,0x16,0x54,0xa5,0xb7,0x9f,0x69,0x04,0x3b,0x50,0xbd,0xfb, +0x48,0xf1,0x86,0xae,0x7a,0x06,0xbf,0x90,0xf3,0x31,0xa5,0x9b, +0xbc,0xea,0xc7,0x34,0xd2,0x1d,0xf9,0x23,0x3e,0x1f,0xa4,0x7c, +0x0f,0x57,0x7d,0x4e,0x23,0x10,0x69,0x75,0x81,0x67,0x58,0x84, +0xa5,0xda,0x48,0xe5,0x54,0x0e,0x91,0x3d,0xb7,0x45,0xf1,0xf5, +0xe2,0x82,0xf2,0x08,0xf0,0xfb,0xef,0x6d,0x2e,0x26,0x40,0x7c, +0x15,0x79,0x71,0x3b,0xbb,0xd5,0x85,0x26,0xe7,0x28,0x6f,0xcb, +0x0a,0x74,0x19,0xb3,0x3d,0xf0,0xb0,0x73,0xba,0xa0,0x16,0x5e, +0xa9,0xb8,0xab,0x95,0xa1,0x28,0xd9,0xaf,0x3e,0xb0,0xef,0x87, +0x95,0x35,0xf0,0x66,0x03,0x43,0xe1,0xd9,0xb3,0x8d,0x8f,0x1b, +0x3f,0x6e,0x1c,0x6c,0x3c,0xdb,0x18,0xd9,0x0e,0xdf,0xd0,0x1d, +0x7e,0xa3,0x37,0x72,0x46,0xd5,0x9d,0x45,0x90,0xff,0xc3,0xc6, +0xe5,0xc6,0xf5,0x46,0x09,0xf0,0x1e,0x64,0xdd,0xf1,0x9c,0x1e, +0x18,0x4d,0x6e,0x7e,0x8c,0x97,0x86,0xce,0x7f,0x2c,0xd5,0xc1, +0x81,0x00,0x0b,0xee,0x47,0x48,0x1f,0x3a,0x64,0x0f,0x7a,0x78, +0x09,0xa1,0x15,0xc8,0x65,0x21,0xc1,0x22,0x86,0x2f,0x41,0x11, +0x94,0x7e,0xd6,0x44,0x24,0xb9,0x9d,0x58,0xe2,0x70,0x68,0x04, +0xb1,0x1b,0xd6,0xa5,0x1e,0x6f,0xcd,0x50,0x98,0x3f,0x5a,0x8b, +0x48,0xa3,0x5f,0xd3,0x53,0x27,0xa0,0x74,0xf4,0x85,0x18,0xea, +0xda,0x6a,0xf4,0xa5,0x48,0x82,0xf2,0x1f,0x9b,0x3e,0x12,0xe0, +0xe1,0x2b,0x7f,0x6c,0x02,0xe0,0x91,0x09,0x30,0xff,0xfa,0x57, +0x0d,0x90,0xe7,0x98,0x05,0x4b,0xd0,0x5e,0x1f,0xee,0x08,0xab, +0x3d,0x2c,0xf5,0x45,0xe5,0xd4,0x23,0x2e,0xf3,0x45,0x85,0xbe, +0xd2,0x85,0xf4,0xa4,0x5e,0xa7,0x84,0x33,0x7e,0xb3,0x24,0x82, +0x6d,0xe3,0xaa,0x06,0x32,0x43,0xe5,0xa2,0x01,0x70,0x55,0x25, +0x48,0x9a,0xe6,0x30,0xeb,0x94,0x12,0x5c,0x40,0x50,0x07,0xcb, +0xda,0x1d,0xbd,0x63,0x08,0xfc,0xbe,0x01,0xf6,0x8e,0xaa,0x90, +0xc6,0x46,0xe7,0x5e,0x3c,0xa4,0x00,0xff,0xf3,0x3b,0xe8,0x60, +0xb0,0x61,0x13,0x98,0x99,0xd0,0x7c,0x29,0x83,0x48,0x41,0xc5, +0x63,0x4c,0xbb,0x3c,0x36,0xd9,0x81,0xd7,0xcd,0x5d,0x8d,0xe9, +0xc9,0x5f,0x82,0xa9,0xa3,0x31,0xfd,0xd0,0x84,0x89,0x0b,0xe9, +0xcd,0xcb,0x1d,0x10,0x7f,0xeb,0xec,0x2d,0x16,0x9d,0xbd,0xa5, +0x5e,0xb5,0x9a,0xf8,0xa3,0xa9,0x72,0xa9,0x17,0xb9,0x06,0x18, +0x5e,0x70,0x01,0xe6,0xb0,0x0e,0x06,0xed,0xbd,0x18,0x9b,0x5b, +0x9f,0xe3,0x64,0x47,0x0f,0x07,0x28,0x53,0x57,0xc8,0xef,0xde, +0x15,0x18,0x12,0x1c,0x50,0x87,0x6b,0x20,0xa9,0x16,0x57,0x0f, +0x75,0xd1,0x9f,0xd2,0xc0,0x1b,0xdf,0x30,0xdb,0x46,0x58,0x10, +0x67,0xc0,0xe9,0x7e,0x16,0xb5,0xec,0xde,0x7f,0x77,0x7f,0x88, +0xd1,0x3c,0xb6,0x87,0x90,0x0e,0x17,0x47,0x4f,0x5b,0x35,0xa4, +0x0e,0x51,0x37,0xbe,0x97,0xb4,0xa2,0x34,0xaa,0xcd,0x07,0x73, +0x37,0x24,0xfc,0xa7,0x7f,0x47,0xf5,0x9d,0xdd,0x07,0x9a,0x02, +0xe2,0x3b,0xbf,0x37,0xf5,0x1a,0xe0,0x7f,0x72,0x7a,0x8a,0x93, +0xec,0xf7,0x95,0x50,0x10,0xa5,0x01,0xa0,0xfe,0xd9,0x04,0xa4, +0x85,0xa3,0x4f,0xa0,0xb4,0x50,0x10,0xbb,0x19,0x04,0x5a,0x5a, +0xde,0xb4,0xc0,0x2a,0x96,0x4a,0xfd,0x66,0x38,0xb9,0x78,0x0c, +0xb3,0xf4,0x5a,0x13,0x0d,0xba,0xa1,0x68,0x60,0xaf,0xf5,0x16, +0x59,0x25,0x8b,0xa2,0x21,0xd4,0x5a,0xe5,0x14,0xad,0x68,0x12, +0x0d,0xc2,0x22,0x71,0x3d,0xec,0x00,0x9c,0x86,0xa0,0x7d,0xa4, +0x14,0x36,0x44,0xcd,0x5e,0xf2,0xca,0x5a,0x8b,0x41,0xe6,0x3c, +0x01,0x36,0x54,0xb6,0x47,0x95,0xd5,0x65,0xb7,0xe1,0x12,0xc7, +0x6a,0x0c,0x00,0xa2,0x47,0xaf,0x71,0xb4,0x67,0x2e,0x28,0x4b, +0x57,0x88,0x65,0x3e,0x73,0xfd,0x25,0x55,0x9f,0xe8,0x56,0xa7, +0x20,0x7b,0x0a,0x70,0xa1,0xc3,0x79,0x90,0xcc,0xfa,0x4f,0xa0, +0xaf,0xc6,0x2d,0x3f,0x40,0xf4,0xa0,0xc3,0xc1,0x4c,0x14,0x5e, +0x50,0x7f,0xd5,0x8f,0x4e,0xa2,0x16,0x27,0x2c,0x15,0x3b,0xec, +0x4a,0xa3,0xff,0x1a,0x8e,0xe7,0xc3,0xe9,0xe7,0x93,0x17,0xf9, +0xe9,0x14,0x1f,0x0e,0x86,0xd3,0xb3,0xcb,0x93,0x27,0x37,0xd3, +0xe2,0x4a,0x3f,0x7f,0x3e,0xf9,0xaf,0xf9,0x38,0xd7,0x7f,0xae, +0x3e,0x9f,0x3c,0x99,0x5f,0xcc,0xcb,0xd9,0xc9,0x61,0x7e,0x33, +0xcb,0xe1,0x3c,0xec,0xe4,0x8d,0x16,0xe7,0xe1,0xf7,0xf5,0xe4, +0x37,0x4a,0x78,0x96,0x9f,0xe1,0x83,0x95,0xf4,0x4e,0x22,0xe3, +0x76,0xcb,0x7d,0xe3,0x75,0xa0,0xef,0xa2,0xfc,0xbe,0xf5,0xc4, +0x35,0xb0,0xd4,0xe1,0xa0,0x43,0x12,0x81,0x3c,0xa0,0x0c,0xe8, +0x32,0x54,0x01,0x51,0x40,0x13,0x10,0x04,0xa4,0x00,0x19,0x40, +0x41,0x4d,0xe5,0x84,0xe8,0x0e,0x0a,0x10,0xc8,0x23,0x43,0x5c, +0x28,0x12,0x85,0x21,0xf1,0xb5,0x75,0xab,0x59,0x80,0xf2,0x50, +0xa1,0xc5,0x0f,0x99,0x03,0x0a,0x8c,0x58,0x12,0xc2,0x20,0x86, +0x69,0x59,0x7f,0x40,0x86,0xc2,0x9d,0x3d,0x67,0xaa,0x50,0x81, +0x42,0xaf,0x54,0x70,0x3e,0x62,0x54,0xa2,0xe0,0x3e,0xa7,0xbf, +0xa7,0x37,0x49,0xc5,0x20,0xa1,0x2a,0xb3,0xe8,0xe7,0xa8,0xe5, +0x04,0xe4,0x12,0xcf,0x94,0xd0,0xf9,0xfb,0xc2,0xcf,0x20,0xc9, +0x8e,0x73,0x55,0x5d,0x55,0x95,0xab,0x74,0xee,0xba,0x5c,0x1b, +0x6f,0xcb,0xd1,0x65,0xb9,0xda,0xb2,0x74,0x43,0xd6,0xb5,0x8c, +0x0d,0xb8,0xad,0xbb,0xd2,0x70,0x8c,0x34,0x3a,0x9c,0x8f,0xf5, +0xc3,0x89,0xde,0xbf,0xc1,0xcf,0xd1,0x3c,0x2f,0xe1,0xf7,0xc7, +0x7c,0x34,0xa6,0xa7,0xa3,0x4b,0xbd,0xd8,0xc1,0xc3,0x0b,0xbd, +0xb2,0xea,0x9f,0xc3,0xa1,0x46,0x02,0xdb,0x1a,0xd9,0xa1,0x16, +0x5d,0x63,0x6f,0x1a,0x88,0x3e,0x1f,0x6a,0x0d,0x04,0x11,0x3c, +0xa2,0x34,0x25,0x40,0x06,0xd0,0x00,0xf5,0x43,0xcd,0x50,0x29, +0xd4,0x58,0x5b,0xd9,0x1d,0xe3,0xc7,0x03,0xab,0xab,0x56,0x73, +0x4b,0xa8,0x54,0xd7,0xa9,0xab,0xd4,0x35,0xea,0x0a,0x75,0x7d, +0xba,0xba,0xda,0xda,0x0e,0xa4,0x6b,0xd4,0xa6,0xba,0x34,0x90, +0xa8,0xc9,0x3b,0x62,0x77,0x85,0x39,0xf9,0xae,0xd1,0xea,0x95, +0xb6,0xe3,0xd5,0x4b,0xb5,0x23,0xf6,0x9b,0x70,0xc0,0x7a,0x60, +0xe1,0x90,0xa5,0xe1,0xda,0x19,0x24,0x48,0x69,0x51,0x19,0xb5, +0x76,0xcf,0x50,0x1d,0xb7,0xfe,0x9e,0xa4,0x39,0x1f,0x76,0x3d, +0xfe,0xb8,0x0e,0x29,0xfa,0xb2,0x91,0x1d,0x96,0xa6,0xb1,0x2d, +0x5b,0x52,0x8e,0x6e,0xff,0x06,0x7a,0x7a,0xfb,0xea,0x28,0x8d, +0x2e,0xd3,0xeb,0xeb,0x8d,0x27,0x91,0x7a,0x95,0x46,0x07,0x07, +0x3b,0xcf,0x9e,0xed,0xe0,0xb5,0x3a,0xf5,0x0a,0xdf,0x0f,0x60, +0x53,0x6c,0x12,0xfc,0x94,0x8d,0x57,0x47,0x98,0xa8,0x53,0x61, +0xd3,0xa5,0x36,0x82,0x3c,0xbd,0x6e,0xfb,0xb5,0x39,0x3f,0x20, +0xc6,0x67,0x08,0xbb,0x2e,0xa0,0x2f,0xf1,0xa1,0xd1,0x05,0x87, +0x09,0xe1,0x43,0x70,0x5b,0x5b,0x4d,0x80,0xed,0xd9,0xe4,0x3d, +0x44,0x20,0x25,0xff,0x2b,0x18,0x48,0x6e,0x35,0xe6,0xa0,0x80, +0xb3,0x74,0x84,0x6f,0x58,0xe8,0xff,0x9e,0x3d,0x03,0x3d,0xf1, +0x48,0x1a,0x57,0x06,0x37,0x8e,0xd8,0xfa,0xa7,0x03,0x51,0x6a, +0x1a,0x3f,0x20,0x23,0x3a,0x12,0xc5,0x7e,0x13,0x14,0xdc,0x25, +0x4c,0xc3,0xe3,0x21,0x3e,0x14,0xbc,0x81,0xeb,0xb0,0x94,0x86, +0x77,0x6e,0x7d,0xa7,0x32,0x60,0xc4,0xf7,0x64,0x16,0x83,0x68, +0xeb,0xdf,0xd5,0x4e,0x77,0xfa,0xc3,0x9b,0xc1,0x71,0xbb,0x77, +0xdd,0xd3,0x7f,0x76,0xf4,0x9c,0xe1,0x5c,0x57,0x8f,0xf1,0x9e, +0x49,0xbe,0x34,0x8b,0x12,0xf1,0xba,0xc8,0x2d,0x90,0x0b,0x0e, +0xa3,0x38,0xa3,0x17,0xdd,0x5c,0x6b,0x09,0xf3,0xed,0x41,0x94, +0xda,0x94,0x21,0xa4,0x3c,0x39,0xd0,0x9d,0x7a,0x72,0x36,0xd4, +0xc2,0xcb,0x68,0x38,0x4d,0x6f,0x4b,0x3d,0xbc,0x9e,0x81,0x4e, +0xa7,0x7f,0xf4,0xff,0x31,0x77,0x35,0x3a,0x6d,0x24,0x49,0xf8, +0x55,0xcc,0x48,0x46,0x1e,0x3c,0x43,0x6c,0xc2,0xc2,0x6a,0xc8, +0x30,0xca,0x5d,0x4e,0xa7,0x3d,0x2d,0xc9,0x4a,0x44,0x3a,0x6d, +0x10,0x42,0xce,0xda,0x04,0xa4,0x18,0xaf,0xb0,0x57,0xb7,0x1c, +0xf6,0xbb,0x5f,0xd7,0x5f,0x77,0x55,0x4f,0x8f,0x21,0x44,0x5a, +0x9d,0x22,0x85,0x71,0x77,0x75,0x77,0x4d,0x4f,0xff,0x54,0x57, +0x57,0x7d,0xb5,0x70,0xa3,0xad,0x37,0x59,0x5d,0xe2,0xa0,0x00, +0xfc,0x10,0x49,0x9f,0x2f,0xee,0x01,0x56,0x5c,0x67,0xfd,0x1b, +0x55,0x3e,0xd0,0xbd,0xbd,0x0b,0x49,0x07,0x18,0x01,0x2a,0xf2, +0xab,0x1b,0xbf,0xb3,0x7b,0x5d,0x1d,0xe4,0x51,0x99,0x8b,0x9f, +0xdd,0xe3,0x65,0xcf,0x16,0x05,0x2e,0xfe,0xf1,0xd5,0xf5,0x43, +0xf6,0x33,0x2a,0xa4,0x98,0x3b,0x3d,0xda,0x8a,0x18,0x2c,0x83, +0x81,0x9b,0x99,0x56,0x8f,0xb5,0x54,0xdc,0x20,0x2c,0xd3,0x30, +0x3a,0x2a,0x59,0x4a,0x40,0x85,0x95,0x7c,0xd4,0x2b,0x6d,0x3c, +0x53,0x3d,0x5e,0xa3,0x6d,0x4d,0x95,0x39,0x21,0xa8,0xef,0x84, +0xd2,0xdf,0x1d,0xcf,0x55,0xd6,0x5f,0xf6,0x26,0x5f,0x16,0xa8, +0xd3,0x9b,0xf4,0xae,0xdd,0x0c,0x17,0x05,0x24,0xa8,0xf6,0x26, +0x3d,0x51,0xee,0xcd,0xdd,0xaf,0xfe,0x54,0x60,0x4f,0x51,0xd9, +0x37,0xa1,0x2f,0xe4,0x9e,0x6f,0x30,0x0f,0x3f,0x17,0x6a,0xff, +0x26,0x3d,0xd4,0xa2,0x4d,0xa7,0x98,0x4e,0x22,0xf0,0x19,0xd6, +0x46,0x0a,0xbf,0xb3,0x33,0xaa,0x8c,0xc5,0xe6,0x07,0xc8,0x22, +0x0d,0xe0,0xc3,0x03,0xe6,0xa0,0x3d,0x89,0xeb,0x34,0xc3,0xbf, +0xef,0xb8,0xa4,0x0d,0x52,0x6c,0x41,0xd4,0xee,0x55,0x5d,0xd9, +0x05,0x91,0x3f,0xb7,0x73,0x9f,0xd7,0x24,0xf7,0x7b,0x98,0xb4, +0x7d,0xd0,0xed,0x0b,0x10,0x2e,0xf6,0x37,0x51,0x86,0x57,0x01, +0xa4,0x4b,0x9e,0x83,0x1c,0x8e,0x84,0x94,0xe8,0x09,0x8e,0x81, +0xf4,0x74,0xd4,0x64,0xf4,0x15,0xdd,0xa8,0x87,0xfa,0xb2,0x2d, +0x6f,0xc0,0x77,0xd8,0xf4,0x87,0x17,0x1e,0xb9,0xd9,0x56,0x3c, +0x2e,0x1d,0x8f,0x9c,0xe9,0x95,0x94,0x71,0x5f,0x47,0x9b,0x27, +0x53,0x85,0xf5,0xbf,0x3f,0xcd,0xc2,0x7b,0x4a,0x76,0x85,0xc9, +0x72,0x53,0x1f,0xaa,0x8c,0xe2,0xd7,0xd0,0x4f,0xd7,0x3d,0x8b, +0xe5,0xea,0x3a,0x5a,0x90,0x3b,0x48,0x51,0x37,0x6b,0x01,0x67, +0x94,0x09,0x88,0x32,0x0b,0x0a,0x5b,0x11,0xb8,0xea,0xd9,0x9f, +0x0f,0xa4,0x6a,0x63,0xf1,0xa2,0x7a,0x24,0x0f,0x43,0x97,0x5e, +0x1d,0xb9,0x34,0x85,0xbd,0x52,0xf9,0x00,0xc7,0xa4,0xcd,0xd6, +0x59,0x5d,0x97,0xcf,0x8a,0x06,0xd0,0xdb,0xf9,0x82,0xd7,0xae, +0xaf,0x85,0xc2,0x6e,0x29,0xc8,0x95,0xc1,0x2f,0xb9,0x9f,0x29, +0x9a,0x56,0xf8,0x98,0x40,0x04,0xd6,0xc9,0x48,0x46,0xc6,0x52, +0xe8,0x9f,0xa3,0x70,0x23,0x95,0xe9,0xc7,0xe3,0xd5,0x2d,0xc1, +0x12,0x15,0x57,0xd7,0xfc,0xc5,0xdd,0x1e,0x5b,0x61,0x31,0xf6, +0x9a,0xa8,0xe8,0x4f,0x41,0xb7,0xfe,0x80,0xc3,0x96,0x6f,0x8a, +0x20,0x20,0x3f,0x87,0x57,0xbc,0xa3,0x3e,0xf9,0x1e,0x8e,0x6b, +0xc3,0x33,0xf9,0x55,0x55,0x3b,0x23,0xcf,0xd4,0x28,0x70,0x9d, +0x78,0x21,0xf3,0x22,0x1b,0x86,0xc5,0x0c,0x2f,0xe1,0x4e,0xe9, +0x2d,0x83,0x07,0x83,0x12,0x85,0x2e,0xe0,0x6c,0xa0,0x5a,0x44, +0xf7,0xfe,0xf1,0xfb,0x7b,0xd1,0x60,0x79,0xfb,0xe5,0xce,0xad, +0x4b,0x2b,0xba,0x07,0xfa,0x69,0xb9,0x70,0xf3,0x5f,0xee,0xa1, +0x6b,0x6f,0x85,0xe2,0x41,0xdf,0xd9,0x32,0x42,0xa2,0xae,0xd5, +0x2d,0x54,0x50,0xaa,0xde,0x37,0x1b,0x8a,0x32,0xe8,0x14,0xa1, +0x9d,0x57,0x52,0x41,0xf3,0xb8,0xe1,0x8e,0x20,0xb3,0x70,0x84, +0xdf,0xf2,0x10,0x91,0x4c,0x03,0xe8,0x6f,0x52,0x23,0x6d,0xf7, +0x54,0x89,0x37,0x6a,0x30,0x78,0xd5,0x94,0x47,0xa6,0xe0,0x75, +0xc7,0x25,0xbd,0xb6,0x7e,0xce,0x9b,0x01,0x74,0x02,0x28,0x64, +0xc0,0x84,0x11,0x2d,0xc8,0xc7,0x97,0x4d,0x39,0xae,0xc6,0xe1, +0x3d,0x1e,0x1f,0x60,0x32,0x55,0xa4,0xa0,0x21,0x1a,0x74,0xf5, +0xcc,0xf7,0xb0,0x03,0x6f,0x4c,0x0e,0xfa,0xab,0x70,0xce,0xdc, +0xe4,0xb0,0x17,0x25,0xe7,0x2d,0x4d,0x1e,0x23,0x00,0x48,0xb9, +0x65,0x54,0x30,0x60,0x04,0x10,0x05,0xd8,0x1c,0x79,0x53,0x7a, +0x63,0x39,0x60,0x5e,0x0e,0xc3,0x62,0x76,0xbc,0x9d,0x7c,0x72, +0x33,0x38,0x24,0x52,0x10,0x76,0xe4,0xee,0xae,0x85,0x14,0x08, +0x4b,0x64,0xe1,0xfe,0xed,0x67,0xde,0xa8,0x98,0xad,0x7d,0x5d, +0xb1,0xbc,0x19,0x55,0xf0,0x87,0x98,0x34,0x3d,0x28,0xed,0xf1, +0x3b,0x1d,0xb8,0xb3,0xc3,0x59,0x9c,0xf8,0xda,0x25,0x4e,0xe3, +0xc4,0x43,0x97,0x78,0x13,0x27,0xfe,0xe0,0x12,0xe7,0x71,0xe2, +0x91,0x4b,0x5c,0xc6,0x89,0xc7,0x2e,0xf1,0x3f,0x71,0xe2,0x8f, +0x18,0xee,0xd8,0x71,0x4f,0x16,0x53,0xad,0xa8,0x06,0x6a,0x1e, +0x80,0xa1,0x18,0xba,0x74,0x46,0xe8,0xf3,0x08,0xc9,0x86,0xd8, +0xf3,0xae,0x1a,0x02,0x68,0x63,0x8b,0x75,0x7c,0xc6,0xda,0xfd, +0x4c,0x84,0xd8,0x24,0xd0,0x0f,0x6c,0xfc,0xe2,0x27,0x28,0x41, +0xd7,0xf0,0x65,0xb3,0xbf,0x76,0x4e,0xd9,0x1e,0x69,0x6b,0x1d, +0x5f,0x2d,0x36,0x6a,0x64,0x30,0x81,0x3a,0xc4,0xcf,0x78,0xa2, +0x00,0x14,0x19,0xa4,0xb0,0xf1,0x48,0x84,0x69,0xcc,0x45,0xa9, +0x80,0x7d,0xf6,0x6b,0x2e,0x35,0x88,0x20,0x0c,0x61,0x09,0xa9, +0xb3,0xd9,0x5d,0x96,0x57,0x31,0x7a,0x62,0x1a,0xbd,0xb0,0xb8, +0xaf,0xdb,0xc6,0x48,0xdc,0x6b,0x11,0xcc,0x5d,0xc2,0x46,0x8e, +0x6a,0xd8,0xbf,0x02,0xc8,0xc6,0xbc,0x8a,0xe8,0x29,0xd9,0xf4, +0xc9,0x3b,0x08,0x62,0x60,0x4f,0x42,0xa1,0x2b,0x30,0x46,0x3f, +0x15,0x55,0x8f,0x54,0x8b,0xfb,0x9a,0xf0,0x66,0x51,0x2a,0xc2, +0x18,0x27,0x58,0xda,0xc4,0x96,0x7a,0x9d,0xa1,0x26,0x75,0x54, +0x5e,0x22,0x55,0x65,0xdf,0xb5,0x96,0xe9,0x2d,0xa5,0x85,0xd8, +0x9d,0x72,0x6a,0x54,0xd9,0x0a,0xf0,0xf9,0x18,0xe1,0xca,0x4e, +0xca,0xf2,0x36,0xf7,0x00,0xd5,0x48,0x80,0xc0,0x76,0x70,0xb0, +0xe7,0x06,0x2d,0x70,0x76,0x68,0x94,0x51,0xc7,0x63,0xf4,0x6b, +0x0d,0x4e,0x1e,0x98,0x26,0x91,0x40,0xe1,0x4c,0x83,0xed,0xa4, +0xc5,0x76,0xc6,0x2d,0x0c,0xbc,0xc0,0xac,0xf1,0x1d,0x52,0x36, +0xac,0xfc,0x44,0xa0,0xd0,0x82,0x0a,0x57,0x49,0x2c,0x52,0xf0, +0x2e,0x9b,0xfb,0x76,0x71,0x1a,0x7f,0x32,0xd1,0xc9,0x93,0x9b, +0x21,0x25,0x06,0x72,0xd8,0x4a,0xa5,0x49,0x19,0x3d,0x35,0x1b, +0x03,0xeb,0x10,0x34,0x10,0xf8,0x3c,0x21,0x07,0x71,0xad,0x68, +0xf8,0xc7,0xb1,0x67,0x3e,0x5c,0xb7,0xe9,0x86,0x6c,0xac,0x87, +0x81,0x39,0x58,0x15,0x42,0x71,0xb1,0xc0,0x4b,0x1e,0x0d,0x11, +0xfe,0x4c,0xd4,0xae,0x00,0x9c,0xc9,0x02,0x0f,0xec,0xf9,0x37, +0x85,0x40,0xec,0x77,0xc9,0x65,0xc8,0xac,0x3b,0xb3,0xe2,0xdd, +0x13,0xce,0xc5,0x7d,0x16,0x8f,0xe1,0xd8,0xd7,0x43,0x3d,0x01, +0x2b,0x0a,0xc4,0x6c,0xa4,0x77,0xf1,0xcf,0xb3,0x8f,0x97,0x9f, +0x3e,0x65,0x58,0xfd,0x56,0xa1,0x8f,0xf8,0x6e,0x82,0x41,0xa9, +0xb1,0x7a,0x84,0xe2,0x3f,0x9d,0x7f,0xe8,0x64,0xd0,0x02,0x16, +0xaa,0xde,0x23,0xb9,0xa6,0x50,0x66,0x33,0x17,0x1f,0x2f,0x83, +0x0d,0xcc,0xf9,0xf9,0xc5,0xa7,0x4b,0xe2,0x0e,0x2d,0x51,0xe3, +0x0b,0x96,0x39,0x1e,0x26,0x78,0x40,0x5d,0xcc,0xe5,0xbe,0xd5, +0x2b,0x68,0x0b,0x89,0xf3,0x50,0xcc,0xfd,0xf5,0xdb,0x5c,0xdd, +0xbd,0xcd,0xc3,0x85,0x17,0xa6,0xeb,0x1b,0xa5,0xcb,0x8d,0xb8, +0xfd,0x25,0x5e,0xc8,0x3b,0x04,0xd2,0x10,0x70,0x93,0xf6,0xfc, +0xe3,0x39,0x04,0x71,0x9a,0x75,0x5e,0xd8,0x5f,0x4d,0x1a,0xbe, +0x8c,0x64,0x70,0xc6,0x18,0x07,0x94,0xa9,0x8a,0x81,0xb6,0x58, +0x55,0x73,0x87,0xf3,0x65,0xc1,0xf3,0xbf,0x41,0x69,0x41,0x76, +0xba,0x79,0x7e,0x3a,0x02,0x69,0x17,0x37,0x73,0xb1,0x60,0x4e, +0xf0,0x13,0x2c,0x86,0xa9,0x8e,0xdf,0xaf,0xe1,0x1d,0x68,0x92, +0xbd,0x5d,0x75,0xbe,0x80,0xc2,0x1d,0x76,0x83,0x77,0xf5,0xdb, +0xb6,0x8b,0x26,0xd4,0x99,0xa0,0x2d,0xde,0x13,0x54,0x85,0x7a, +0xdb,0x7a,0x67,0xcc,0xf1,0x9c,0xe2,0xc3,0x12,0x4e,0xe0,0x73, +0x85,0xaf,0xc6,0xa7,0x5f,0x33,0xb0,0xd0,0xa8,0x40,0x51,0x42, +0x70,0x8d,0xf6,0xbe,0xea,0xd7,0x1f,0x7d,0x53,0x1b,0x0e,0x68, +0x03,0x7f,0x5e,0x9c,0x68,0xa3,0x06,0x92,0x4b,0x51,0x1d,0x85, +0x26,0xfd,0x7f,0xf8,0xdd,0xd4,0x3d,0xd6,0x5d,0x6e,0x6e,0x76, +0x8b,0x1b,0x00,0x02,0x0e,0x1b,0x69,0x57,0x71,0x5e,0x68,0xa0, +0x78,0x2a,0x9a,0x0b,0xbe,0xa7,0x2b,0x58,0x8c,0x73,0xee,0xae, +0x25,0x13,0xff,0x9f,0xf3,0x5b,0x7a,0x86,0xe1,0xc0,0x1f,0x33, +0x8b,0x7b,0x4a,0x31,0x59,0xa2,0x80,0xc9,0x6c,0x07,0x15,0x82, +0x2b,0xe8,0x75,0x07,0x7a,0x6a,0xc8,0xda,0x8e,0xc3,0x29,0x5e, +0x63,0x2b,0x4b,0x83,0xe3,0xd1,0x4d,0x75,0xa0,0xc5,0xa0,0x21, +0x61,0x61,0xa6,0x2b,0xda,0x12,0x1a,0xe1,0x67,0x3f,0x4e,0x28, +0x82,0x46,0x72,0x03,0x2c,0xc8,0xb8,0x0b,0xbd,0x25,0xdd,0xef, +0xf5,0x9a,0xcd,0xbe,0x24,0xa1,0x41,0x25,0x48,0x7d,0xf8,0xfa, +0x60,0xf6,0x03,0x37,0xa4,0x11,0xe0,0xf3,0x21,0xb6,0x67,0x92, +0x44,0x6d,0x59,0x8f,0x0f,0xf6,0xb4,0x11,0x09,0xb1,0x46,0xcf, +0x80,0xdb,0xa6,0x2c,0x9e,0x28,0x6b,0x6e,0xcb,0x0f,0x6b,0xa4, +0x29,0xcd,0x6a,0xeb,0xc4,0x86,0xfb,0x95,0x8a,0x22,0x51,0x0e, +0xa0,0x68,0xa0,0x71,0xfd,0xde,0xa2,0xc9,0xf3,0x57,0xea,0x33, +0x94,0xed,0x3e,0xdb,0xda,0x82,0x10,0x0d,0x54,0xcf,0x6e,0x6f, +0x4f,0xba,0x9f,0x9b,0xb5,0x3d,0xec,0xb5,0xcc,0xaf,0x00,0x40, +0xc0,0x9d,0x7a,0xb0,0x7f,0xf1,0x45,0xa1,0x36,0xe9,0x3b,0x31, +0xe1,0xf3,0xdf,0x01,0xc8,0x60,0x1f,0xf5,0x16,0x7d,0x36,0xc7, +0xbd,0x13,0x1b,0xf8,0xd9,0x74,0xb0,0x2b,0x24,0x83,0x3f,0xfb, +0x45,0x4b,0x19,0x41,0xf9,0x2b,0x34,0x38,0x64,0x1b,0xc0,0x4e, +0xa2,0xa3,0xd1,0xe1,0x8f,0x8e,0x0a,0x52,0x73,0x19,0xe1,0xac, +0x7e,0xab,0x7c,0x28,0x2a,0xbf,0xf0,0x5c,0xbb,0xa9,0x13,0x26, +0x07,0xe0,0xaf,0x59,0xe5,0x5c,0x24,0x87,0x84,0x39,0x49,0x03, +0xcc,0x35,0x82,0x85,0x72,0x96,0x05,0xf4,0x3a,0x47,0xf2,0xeb, +0xfe,0xcd,0x1f,0xf3,0xc9,0x9d,0x1b,0xcb,0x03,0xeb,0xaa,0xc8, +0x6d,0xbf,0x5f,0x28,0x83,0xac,0x74,0xcb,0x58,0x27,0xd0,0xca, +0x6e,0x9e,0x17,0x71,0x4d,0x6d,0xa5,0x70,0x98,0xd3,0x75,0xe0, +0xd4,0xc3,0x7e,0x85,0x09,0xcc,0x03,0x20,0x8c,0x0c,0xf8,0x00, +0x6e,0xae,0x91,0xb2,0x15,0x70,0x9b,0x58,0x93,0x58,0x1e,0x9d, +0x42,0x1d,0x4d,0x26,0xea,0xe8,0xac,0x2a,0xc7,0x9c,0x24,0x0a, +0xec,0xac,0x1a,0xa9,0x94,0x77,0xae,0xa6,0x6a,0xac,0x4a,0x61, +0xc2,0x01,0x27,0xb0,0x0e,0x3d,0xab,0x8e,0x55,0x02,0x55,0x12, +0x9a,0x30,0xdb,0x07,0xef,0x19,0xba,0x8b,0xe5,0xbd,0x05,0xbe, +0x18,0xe7,0x05,0xca,0x08,0x12,0xc2,0x22,0x29,0x4f,0xf8,0xf8, +0x16,0x6a,0xce,0x8b,0x64,0xb1,0x75,0x0b,0xcd,0xdf,0x18,0xf1, +0x8f,0x96,0x81,0x91,0x4c,0xa3,0xf5,0xfa,0x09,0xca,0x1f,0x84, +0x72,0x53,0x4c,0xb5,0x80,0xa5,0xbc,0x68,0xc0,0xef,0x4c,0x2f, +0xbc,0x2c,0xf4,0x69,0x37,0xb5,0x4a,0xa5,0x19,0xbf,0x35,0xe3, +0x61,0xa3,0x1c,0x75,0xad,0x17,0xb1,0xea,0x3f,0x16,0x08,0xc0, +0xdb,0xef,0x71,0x4a,0xea,0xa0,0xd2,0x11,0x6e,0xdc,0x64,0x87, +0x3f,0x74,0xe1,0xde,0xc5,0xe6,0x87,0x6b,0x5c,0x45,0x8b,0x6f, +0xf1,0xfb,0xe9,0xf4,0xd5,0x22,0x6e,0x8d,0x3d,0x57,0x1b,0x3d, +0xb4,0xc3,0x47,0x19,0xfb,0xa8,0x1a,0x04,0x96,0x78,0xc4,0x93, +0x30,0x1a,0x9e,0xc7,0x22,0xff,0x68,0xb7,0xa3,0x0c,0x4b,0x64, +0xde,0xf5,0x28,0x90,0x9f,0x49,0x54,0x2c,0xf5,0xae,0xed,0x5d, +0x25,0x7f,0x89,0xb7,0x92,0x6f,0x14,0xcc,0xc3,0x68,0xe2,0x55, +0xf1,0x59,0x90,0x91,0x5c,0xb7,0x6c,0x88,0x8c,0xef,0x8a,0x6b, +0x76,0xa5,0xf6,0xa8,0x11,0x03,0x36,0xd2,0x1a,0x5f,0xe9,0x0e, +0x60,0xe4,0x48,0x9c,0x65,0xf8,0x28,0x76,0xc9,0x8c,0x0a,0x09, +0xf3,0x51,0x19,0xba,0x49,0x4d,0xb8,0x5a,0x57,0xc6,0xb0,0xcd, +0x37,0x42,0x4b,0x7c,0x65,0x2c,0xda,0x24,0x93,0x77,0x86,0xaa, +0x6d,0x43,0xe6,0x24,0x56,0xd6,0xf7,0xda,0xc5,0xdc,0x98,0x19, +0x3a,0x99,0xc2,0x33,0x18,0x36,0xa5,0xd8,0xca,0xd0,0xcb,0x39, +0x6e,0x25,0x48,0xf4,0xe3,0x33,0x04,0x0b,0xe2,0x9d,0x17,0x40, +0x4a,0x23,0x0f,0xd8,0xb8,0xf1,0x86,0xbb,0x8e,0xa4,0xa7,0x71, +0xbe,0x2f,0xb2,0xa0,0xdb,0x50,0xdd,0x6a,0x39,0xc6,0x35,0xe4, +0xad,0x3b,0x98,0xdc,0x27,0xe5,0xad,0x88,0x99,0x94,0x63,0x14, +0x35,0x83,0xff,0x47,0xb6,0xf2,0x43,0xb3,0x9e,0x58,0x6e,0x4f, +0x87,0x56,0xf6,0xb2,0xb9,0xc0,0xd4,0xdf,0x66,0x6e,0x85,0x9c, +0xfd,0xb5,0x5c,0xbd,0x79,0x92,0xab,0xf3,0xc9,0xfc,0x2f,0xe6, +0xc9,0x7d,0xc9,0x27,0xb8,0x9a,0x6b,0x6b,0x8f,0xc5,0xea,0x46, +0x5d,0x5a,0xe1,0x2f,0xd1,0xb0,0x24,0xa1,0x1e,0x70,0x28,0x9d, +0x22,0x1d,0xad,0x4a,0xf8,0xe8,0x2a,0x9d,0xfc,0xf9,0x1d,0x95, +0x22,0xc9,0x29,0xd4,0x67,0x2a,0xfd,0xaf,0x51,0x98,0xa8,0xc5, +0x99,0x64,0xf2,0x3a,0x12,0xd0,0xf5,0x7a,0x2c,0xbe,0xb5,0x7a, +0xdd,0x26,0x42,0xbd,0xb9,0x88,0x73,0x0a,0xaf,0x69,0xdb,0x7c, +0x6d,0x79,0xfd,0x7e,0x02,0xbe,0x36,0x2f,0x3c,0x8a,0x02,0x25, +0xbc,0x19,0x1f,0xf9,0xc2,0x47,0xa3,0x3d,0xbd,0x06,0x33,0xe3, +0xb5,0xda,0xb4,0xe4,0xe0,0x3a,0xe2,0xd0,0xef,0x3b,0xb5,0x60, +0x35,0x3f,0xeb,0x54,0x14,0x4b,0x70,0x54,0x49,0x49,0x0d,0x64, +0x73,0x27,0xf5,0x8c,0x51,0xe0,0x61,0x3d,0x83,0x18,0x7a,0x76, +0xdf,0xcd,0x99,0xed,0x8e,0x0a,0xbc,0x37,0x23,0x3a,0x5d,0xe0, +0xef,0x0b,0xba,0xd8,0x5c,0xcd,0xa6,0x3d,0xb7,0x0e,0x81,0x7e, +0x7a,0xf2,0xb5,0x07,0x9d,0x4d,0xf5,0x78,0xcd,0x59,0xbb,0xa2, +0x56,0xff,0x73,0xc5,0xbc,0x2a,0xaa,0x73,0xd9,0xad,0xac,0x8a, +0x37,0x93,0xe5,0xdb,0xaf,0xb7,0x5f,0xdc,0xdc,0x01,0xf7,0x1d, +0xfa,0x94,0x1d,0x06,0x1a,0xec,0xdd,0xad,0xcf,0xa7,0xe6,0xc4, +0x97,0x57,0xa3,0x02,0x50,0xaa,0xcc,0x89,0x84,0x08,0xfa,0x47, +0x23,0x94,0x69,0x64,0x63,0x4c,0xf4,0x81,0xde,0x36,0x95,0xdc, +0x55,0xe8,0xd3,0x15,0x09,0x46,0xec,0xa9,0xde,0x29,0x77,0xa0, +0xd7,0x3c,0x61,0x4a,0x0c,0xd2,0x07,0x22,0x14,0x62,0x3b,0x0e, +0x4b,0xb8,0x5f,0xe6,0x7c,0x98,0xc8,0x87,0xe3,0xd4,0xbc,0x08, +0x4c,0x78,0x01,0x09,0xe0,0x10,0xbc,0x84,0xc4,0x48,0x10,0x74, +0x83,0xdc,0xc9,0x2c,0xb4,0x54,0xab,0x9b,0x64,0x1c,0x85,0xe6, +0x84,0x60,0x6f,0x94,0xa3,0xd4,0x87,0x1c,0xbb,0x28,0xc5,0xdf, +0x83,0x65,0xed,0x41,0x58,0x7b,0x20,0xae,0x94,0x49,0xff,0xf3, +0x19,0x1b,0x17,0x87,0x2f,0x6e,0xd0,0x5e,0xa4,0xab,0x96,0x10, +0x68,0x57,0xbf,0x1c,0x7a,0x99,0xe0,0x17,0x49,0x35,0x83,0xf5, +0x98,0x2e,0x3f,0xde,0xa3,0xea,0x4a,0xc8,0xca,0xc3,0xbb,0x75, +0xb7,0x96,0x7c,0x2f,0x48,0x7c,0x61,0x83,0x1e,0xf7,0xa1,0xa3, +0x41,0x90,0xd8,0xbd,0xba,0x61,0x90,0x0f,0x8f,0xcb,0xe4,0x27, +0xce,0xfb,0xc7,0x5d,0x0c,0x40,0xed,0x89,0x71,0xc6,0x59,0xe1, +0x9d,0x53,0x5c,0x74,0x2d,0xea,0xc8,0xcb,0x7a,0x7d,0x5c,0xf9, +0x5f,0x21,0xb9,0x7f,0xdc,0xf0,0x65,0x30,0xb6,0x73,0x0c,0xfe, +0x2c,0x7a,0x51,0xf8,0x36,0xc9,0x89,0x63,0x9c,0xa2,0x2c,0xdb, +0xaa,0x85,0x14,0xf6,0xcf,0xaa,0x2b,0x61,0x79,0xa2,0xaa,0xa7, +0x15,0x4e,0xda,0xa2,0x5a,0x79,0x85,0x83,0x9e,0xae,0xba,0x6e, +0x9c,0x0c,0xb8,0x16,0x2d,0x8d,0x48,0x3f,0x08,0xcf,0x5d,0x97, +0x5e,0xb4,0x0f,0x6c,0x72,0x86,0x04,0x4d,0x3b,0x62,0x6a,0x2c, +0xbd,0x14,0xae,0x42,0xba,0x14,0xda,0x31,0xa6,0x03,0xca,0x62, +0x34,0x59,0x70,0xdd,0xeb,0x2a,0x98,0x9f,0xa4,0xda,0x61,0xcf, +0xc7,0x4c,0x00,0xa5,0x32,0x7f,0x2e,0xb9,0xbe,0xc3,0x13,0x4b, +0x6d,0x7e,0xaa,0x4c,0x8e,0x1d,0x1d,0x25,0x28,0x02,0x18,0x87, +0x3a,0x1f,0x81,0x59,0xc2,0x4f,0x1e,0x9c,0x9a,0x82,0x93,0x14, +0xd1,0x6a,0xf1,0xaf,0xf3,0x0f,0xef,0x6b,0x9d,0xe0,0xef,0x30, +0xa2,0xdb,0x21,0x75,0x49,0x59,0xcb,0x56,0xae,0x2f,0x8a,0x38, +0x30,0x7a,0xac,0xea,0x10,0x34,0x27,0x1b,0x50,0x9b,0xc2,0xab, +0x1b,0x33,0x05,0xfa,0xf0,0xed,0x90,0xda,0x2c,0xf8,0xa8,0x70, +0xda,0x4c,0x89,0x3f,0x0a,0x0c,0xce,0x2e,0x34,0xab,0xc9,0x09, +0xfc,0x67,0xed,0x1f,0xf4,0x8f,0x3e,0xf8,0xce,0x4a,0x86,0xd7, +0x3a,0x69,0x0a,0x42,0x8e,0xc2,0x5a,0x84,0x4e,0xca,0x1e,0x85, +0x80,0xf8,0xbe,0xa8,0x06,0x95,0xe2,0xa6,0x39,0x1a,0x37,0xfd, +0x85,0x42,0x04,0x36,0xa5,0x5a,0xd3,0x78,0x53,0xae,0x08,0xe5, +0xe3,0xff,0xfd,0x83,0x43,0x7c,0xeb,0x61,0x20,0x0f,0x38,0x54, +0x48,0x8c,0x7d,0x02,0xff,0xf5,0x5f,0x8f,0xb8,0x43,0x14,0xf1, +0x94,0xb1,0x9c,0x98,0x17,0x19,0x42,0xf0,0xa7,0x3f,0x3e,0x60, +0x24,0xab,0xc0,0x09,0x66,0xbc,0x02,0x18,0x52,0xa4,0xa7,0x6c, +0xfc,0x9f,0xd6,0xd7,0xd4,0xc5,0x89,0x2f,0xed,0x0f,0xdd,0x08, +0x2a,0xb4,0xe5,0x0e,0xb0,0xd7,0xfe,0xb4,0x43,0xdc,0xe5,0xf7, +0xc2,0xb7,0x1d,0x92,0x9f,0xf2,0x9e,0x31,0xef,0x77,0x2c,0xe7, +0x43,0xf6,0x59,0xde,0x53,0x5e,0x53,0x57,0x81,0x71,0x27,0x48, +0xb1,0x1e,0xcf,0xaa,0xeb,0x44,0x57,0x27,0xda,0xb6,0xd9,0xfd, +0xec,0xee,0xb7,0x59,0x8d,0x67,0x10,0xd1,0x90,0x1a,0xe8,0xb1, +0x40,0x54,0xec,0x84,0x0a,0x8c,0x56,0xc6,0xe3,0x10,0xf9,0xec, +0xc8,0xe0,0x57,0x2e,0x52,0xbc,0x21,0xa0,0xae,0x95,0x55,0x9b, +0x79,0xf1,0x9d,0x77,0x2e,0xb1,0xd1,0x80,0x22,0x39,0xe9,0xee, +0x6f,0x88,0x41,0x1f,0xcd,0x2e,0xd5,0xf9,0x94,0x8b,0x73,0x4c, +0x77,0xb0,0x94,0xa2,0x99,0x46,0x39,0x34,0xcb,0x07,0xdf,0x72, +0xeb,0xf2,0x22,0x8e,0xcb,0xad,0x1c,0x97,0x1d,0x1c,0x97,0x4f, +0x73,0xfc,0xdd,0x5b,0xaa,0xdd,0x21,0x10,0x78,0x07,0x5d,0x2a, +0x97,0x2f,0xae,0x16,0x01,0x68,0xa8,0x6e,0x6f,0x6d,0x6d,0x2d, +0xc5,0x39,0x57,0x4c,0xbf,0x7d,0xa3,0xb8,0x67,0x86,0xd5,0x1b, +0x4d,0xe9,0xdc,0x44,0x59,0x26,0xee,0xc9,0x45,0xc0,0x5c,0x86, +0x30,0xf1,0x5e,0xdc,0x5f,0x0e,0x72,0xf1,0x01,0x8a,0x72,0xd9, +0x5f,0x26,0x67,0x68,0x3c,0x9b,0x49,0x93,0x5f,0xd0,0xf4,0x6c, +0x1e,0x5f,0x35,0x07,0x10,0xbe,0xa8,0x5e,0xb9,0x7f,0xce,0xfd, +0x72,0x6c,0x09,0xfc,0x1d,0xf4,0xb0,0xad,0xa5,0x22,0x78,0x3f, +0x33,0x76,0x26,0xcb,0x73,0xc9,0x64,0x14,0x23,0x95,0xf2,0x66, +0xd4,0x64,0x25,0x1c,0xe2,0x5c,0xc7,0xfd,0x92,0x0d,0x11,0x60, +0x6b,0x89,0xb2,0xf3,0x72,0x98,0xfd,0x4a,0x19,0xbc,0x18,0x36, +0x3c,0xf2,0x21,0x3c,0x13,0x26,0xc3,0x3b,0x36,0x38,0x49,0x20, +0xba,0x11,0x26,0xe1,0xab,0xad,0xd7,0xfc,0x0a,0xeb,0x35,0xb3, +0xd5,0x40,0x9c,0xb1,0x40,0xd0,0xe0,0xff,0x43,0x08,0xd7,0x43, +0xf5,0x13,0x79,0xc3,0x7f,0x43,0x0b,0x52,0x5c,0xa6,0x2a,0xc4, +0xc8,0x81,0xb8,0x93,0xd9,0x2f,0xa3,0x77,0x99,0x93,0x73,0x82, +0x6b,0x5b,0x1a,0x27,0x22,0x4f,0x27,0xc7,0xd6,0x59,0xb7,0x60, +0x97,0x95,0xc4,0x47,0xba,0xed,0x00,0xa0,0x00,0xa1,0xa6,0x48, +0x00,0x36,0xdd,0xda,0x19,0x90,0x93,0xe4,0xd3,0xaa,0x36,0x43, +0xf1,0x23,0x2b,0xe8,0xe2,0x27,0x8f,0xcf,0xf9,0x30,0x5e,0x01, +0x8c,0x08,0x07,0x5d,0x6b,0xcf,0x20,0xe3,0x8a,0xd2,0x2f,0xfe, +0x7a,0xa4,0xbe,0xa2,0xbd,0x62,0x38,0x3e,0x30,0xe9,0xc6,0xdc, +0x09,0xad,0x3f,0x8a,0xc7,0x4e,0x7b,0x67,0x98,0x0d,0x9f,0x6b, +0xfa,0xd5,0x1f,0x8f,0xfc,0x85,0xa3,0x93,0x4b,0x69,0xa3,0xf1, +0x59,0x23,0xf0,0xca,0x6d,0x32,0x50,0xd9,0x42,0xee,0x67,0x50, +0x96,0x67,0xd5,0x01,0x3d,0x82,0x16,0xf5,0x35,0x3d,0xde,0xbb, +0x47,0xa0,0x0a,0xa7,0x09,0xa8,0x60,0x28,0x81,0x2b,0xf3,0x00, +0xbd,0xd4,0x0c,0x2c,0xbe,0x12,0xaf,0x8f,0x85,0x06,0x95,0x72, +0x9f,0xbf,0x2d,0x77,0x93,0xbc,0xbc,0xbb,0x4b,0x7f,0xf7,0x27, +0xf3,0x69,0x43,0x8f,0x70,0x71,0x08,0x55,0x64,0xc1,0x6b,0x84, +0x03,0xda,0x16,0x82,0xe1,0x44,0x4d,0xaa,0x6b,0x33,0xe4,0x80, +0x1c,0x24,0xdd,0x46,0x74,0xb7,0xa0,0x96,0x77,0xea,0x7a,0x67, +0xb4,0xbb,0xab,0x59,0xe1,0x6e,0xdd,0x10,0xae,0x99,0xa4,0x6e, +0x28,0x66,0x27,0x1d,0x19,0xff,0x17,0x00,0x00,0xff,0xff,0x67, +0x2f,0x5b,0x4f,0xc4,0x23,0x06,0x00, })) if err != nil { diff --git a/mod/dashboard/resources/scripts-browser-scripts.js.go b/mod/dashboard/resources/scripts-browser-scripts.js.go index ab389cd7d00..661289c1a5f 100644 --- a/mod/dashboard/resources/scripts-browser-scripts.js.go +++ b/mod/dashboard/resources/scripts-browser-scripts.js.go @@ -177,43 +177,43 @@ func scripts_browser_scripts_js() []byte { 0xdf,0xac,0x64,0x92,0x93,0x76,0x76,0xed,0x1b,0xed,0xf3,0x31, 0xe3,0x4d,0x31,0x1c,0x05,0x9e,0xe3,0xe0,0xbc,0xd5,0x51,0xbb, 0xdd,0x1f,0x8a,0xfb,0x6f,0xac,0xe5,0xa1,0x42,0x3d,0x65,0xba, -0x41,0xcd,0xcc,0x4f,0x1a,0xbc,0x51,0xab,0x32,0x37,0x59,0xe0, -0xa6,0x15,0x34,0x57,0x4a,0x2d,0xd6,0xd5,0x91,0xe7,0xf9,0x97, -0xc7,0xc3,0x57,0xd5,0x87,0x6e,0x89,0x61,0x80,0x86,0x2c,0xe9, -0x94,0xb6,0x6a,0x64,0x47,0x84,0x64,0x6a,0xde,0x10,0x0a,0xb9, -0xc4,0xfd,0x17,0x2b,0x35,0x59,0xd5,0xc3,0x3e,0x3d,0x6f,0x9e, -0xc8,0xe2,0xce,0x1b,0x81,0xf3,0xe1,0x93,0x75,0xd4,0x1e,0x66, -0xe5,0xc0,0x6d,0x44,0x3d,0x40,0x8e,0x89,0x1d,0x84,0xb1,0x6b, -0x5e,0xcd,0xbc,0xf5,0xd8,0xa9,0xe7,0x16,0x3f,0xc5,0x0d,0xb6, -0x7b,0x53,0x72,0x38,0x9d,0xf5,0x18,0x56,0xc5,0x6c,0x4e,0xec, -0x48,0xcc,0xf0,0x30,0xfd,0xfe,0xa7,0xcc,0xc6,0xe2,0xdf,0x48, -0xe2,0x1e,0xaf,0x27,0xe4,0x2d,0xc5,0x74,0x40,0x5c,0x94,0xb5, -0xbf,0x09,0xd2,0x64,0x68,0xa8,0x0e,0xed,0xdb,0x38,0x1a,0x93, -0x41,0xe9,0x90,0xb9,0xce,0xb8,0xcf,0xa4,0xd7,0x24,0xdb,0x48, -0x50,0x39,0x3d,0x27,0x4e,0x48,0x22,0xcb,0x88,0x1d,0x9c,0x1e, -0x46,0xa3,0x17,0x2f,0xa9,0x30,0xbb,0xd7,0x78,0x25,0x2b,0x51, -0x17,0xff,0x19,0x2a,0x62,0xc7,0x74,0x87,0xd9,0xd0,0x93,0xd2, -0x6d,0xef,0xa9,0x45,0xce,0x2e,0x61,0xaf,0xaa,0xd7,0xa8,0xf7, -0xdf,0xe6,0x07,0x1f,0x67,0xfb,0xcf,0x5c,0xc8,0x1c,0x08,0x20, -0x29,0x83,0x10,0xf1,0x75,0xdd,0xe1,0x87,0xc0,0xa7,0x97,0xe1, -0x8b,0xe7,0x18,0x41,0xcc,0x37,0x26,0xdd,0x62,0x42,0xe3,0x47, -0x4f,0xb2,0xce,0xf9,0x0e,0x2e,0xd0,0x76,0xdd,0xab,0x9d,0x93, -0x92,0x54,0xb5,0x2c,0x28,0xad,0xf0,0xfb,0xd6,0x5e,0x2e,0xe8, -0xd2,0x12,0xed,0xbd,0x21,0xcd,0x61,0xa0,0xd2,0x9a,0xed,0x40, -0xb5,0xee,0xfb,0x34,0xfb,0x92,0x97,0x68,0xf6,0x1d,0x3b,0xdd, -0x6b,0x72,0x59,0x66,0x45,0x9b,0x63,0x98,0x3b,0x39,0xf1,0xd6, -0x58,0xee,0x9a,0x9d,0xfb,0xd8,0x8b,0x2b,0xdf,0xb9,0x19,0xbd, -0xec,0x0d,0xbd,0xd5,0xf8,0xb7,0x2f,0x7a,0xef,0x4d,0x31,0x90, -0x42,0x5e,0x91,0xb2,0xcb,0xc7,0x25,0x54,0xab,0x83,0x42,0x06, -0xcf,0x60,0xdc,0x25,0xaf,0xb5,0x07,0xe6,0x80,0x04,0x7c,0xa6, -0x70,0x2f,0x4b,0x59,0x2d,0x10,0xff,0x39,0x30,0x16,0xb8,0xb5, -0x06,0x8b,0x42,0x9a,0xcb,0xd0,0x02,0xab,0x67,0xb9,0x5b,0x62, -0x73,0x8e,0x4d,0x19,0xcc,0xd5,0x0a,0x7e,0x9f,0xcf,0xcd,0x1e, -0x2d,0xb0,0x93,0x13,0x19,0x3d,0x64,0xc7,0x79,0x6e,0xe0,0xf6, -0x19,0xfb,0xbd,0xe1,0x86,0xcc,0xc2,0xc4,0xf9,0xfe,0xbd,0x65, -0x46,0x2b,0x6c,0x6d,0x68,0x6b,0x23,0x38,0x58,0x6d,0x36,0x66, -0x87,0x16,0x35,0xdd,0x88,0xc7,0xff,0x0d,0x00,0x00,0xff,0xff, -0xb0,0x30,0xf2,0xd3,0x4b,0x1b,0x00,0x00, +0x41,0x8d,0xd1,0x6e,0x55,0xe6,0x26,0xfc,0xdd,0x98,0x82,0xae, +0x4a,0x39,0xc5,0xba,0xca,0xf1,0x3c,0xff,0xf2,0x40,0xf8,0xaa, +0xc2,0xd0,0xad,0x2d,0x0c,0xd0,0x90,0x25,0x9d,0x9a,0x56,0x8d, +0xec,0x6c,0x90,0x4c,0xcd,0xe3,0x41,0x21,0x97,0xb8,0xf8,0x62, +0xa5,0x26,0xab,0x7a,0xd8,0x99,0xe7,0xcd,0x13,0xe9,0xdb,0x79, +0x1c,0x70,0xce,0x7b,0xb2,0x80,0xda,0xc3,0xac,0x1c,0xb8,0x86, +0xa8,0x07,0xc8,0x31,0xb1,0x13,0x30,0x76,0xcd,0x73,0x19,0x33, +0x3c,0xd3,0x80,0x9d,0x7a,0x6e,0xf1,0x53,0xdc,0x60,0xbb,0x37, +0x25,0x87,0xb7,0x59,0x8f,0x61,0x55,0xcc,0xe6,0xc4,0x8e,0xc4, +0x0c,0x0f,0xf3,0xee,0x7f,0xca,0x6c,0x2c,0xfe,0x8d,0x24,0xee, +0xf1,0x7a,0x42,0xde,0x52,0x4c,0x07,0xc4,0x45,0x3d,0xfb,0x9b, +0x20,0x4d,0x86,0xa6,0xe9,0xd0,0x3e,0x8a,0xa3,0x23,0x19,0x94, +0x0e,0x99,0x6b,0x89,0xfb,0x4c,0x7a,0xdd,0xb1,0x8d,0x04,0x95, +0xd3,0x3b,0xe2,0x84,0x24,0xb2,0x8c,0xd8,0xc1,0xe9,0x61,0x34, +0x7a,0xf1,0x92,0x2a,0xb2,0x7b,0x86,0x57,0xb2,0x12,0x75,0xf1, +0x9f,0xa1,0xea,0x75,0x4c,0x97,0x97,0x0d,0xbd,0x25,0xdd,0xf6, +0xde,0x58,0xe4,0xec,0x12,0xf6,0xaa,0x7a,0x1d,0x7a,0xff,0x51, +0x7e,0xf0,0x55,0xb6,0xff,0xbe,0x85,0xcc,0x81,0x00,0x92,0x32, +0x08,0x11,0x5f,0xd7,0x1d,0x7e,0x08,0x7c,0x7a,0x12,0xbe,0x78, +0x8e,0xd9,0xc3,0x7c,0x63,0xc4,0x2d,0x26,0x34,0x77,0xf4,0x24, +0xeb,0x9c,0xef,0xe0,0x02,0xfd,0xd6,0x3d,0xd7,0x39,0x29,0x49, +0x55,0xcb,0x82,0xd2,0x0a,0xbf,0x6f,0xed,0xad,0x82,0x6e,0x2b, +0xd1,0xde,0xe3,0xd1,0x1c,0x06,0x2a,0xad,0xd9,0x0e,0x54,0xeb, +0x3e,0x4c,0xb3,0x2f,0x79,0x82,0x66,0xdf,0xb1,0xd3,0xbd,0xee, +0x96,0x65,0x56,0xb4,0x39,0xa6,0xb8,0x93,0x13,0x6f,0x8d,0xe5, +0xae,0xcb,0xb9,0x8f,0xbd,0xb8,0xf2,0x2d,0x9b,0xd1,0x93,0xde, +0xd0,0x23,0x8d,0x7f,0xf4,0xa2,0x87,0xde,0x14,0x93,0x28,0xe4, +0x15,0x29,0xbb,0x7c,0x5c,0x42,0xb5,0x3a,0x28,0x64,0xf0,0x0c, +0xc6,0x5d,0xf2,0x5a,0x7b,0x60,0x0e,0x48,0xc0,0x67,0x0a,0x17, +0xb2,0x94,0xd5,0x02,0xf1,0x9f,0x03,0x63,0x81,0xeb,0x6a,0xb0, +0x28,0xa4,0xb9,0x05,0x2d,0xb0,0x7a,0x96,0xbb,0x25,0x36,0xe7, +0xd8,0x94,0xc1,0x5c,0xad,0xe0,0xf7,0xf9,0xdc,0xec,0xd1,0x02, +0x3b,0x39,0x91,0xd1,0x0b,0x76,0x9c,0xe7,0x06,0x6e,0xdf,0xaf, +0xdf,0x1b,0x6e,0xc8,0x2c,0x8c,0x9a,0xef,0xdf,0x5b,0x66,0xb4, +0xc2,0xd6,0x86,0xb6,0x36,0x82,0x83,0xd5,0x66,0x63,0x76,0x68, +0x51,0xd3,0x55,0x78,0xfc,0xdf,0x00,0x00,0x00,0xff,0xff,0x3f, +0x2a,0x63,0x77,0x44,0x1b,0x00,0x00, })) if err != nil { diff --git a/mod/dashboard/resources/scripts-stats-modules.js.go b/mod/dashboard/resources/scripts-stats-modules.js.go index 747a4d067b4..fa4a209b006 100644 --- a/mod/dashboard/resources/scripts-stats-modules.js.go +++ b/mod/dashboard/resources/scripts-stats-modules.js.go @@ -11277,2082 +11277,2479 @@ func scripts_stats_modules_js() []byte { 0x27,0xc2,0xb5,0x5f,0x4a,0x95,0x86,0x70,0x79,0xe5,0x5d,0x5a, 0xf7,0x92,0x2a,0x7e,0xa2,0x21,0x95,0xcc,0x12,0xd9,0x79,0xd4, 0x2f,0xff,0xcf,0x2f,0x7f,0x8f,0xc8,0x54,0xaa,0x11,0xff,0x7f, -0xcc,0xbd,0xeb,0x76,0x1b,0xb7,0x92,0x28,0xfc,0xfb,0x9b,0xa7, -0xb0,0xfa,0x8c,0x95,0x6e,0xb1,0x49,0x75,0x53,0x96,0x6c,0x93, -0x6e,0x71,0x25,0xce,0xce,0x97,0xcc,0xc9,0xed,0x8b,0xb3,0xb3, -0xed,0x30,0xdc,0x5a,0xbc,0xb4,0xa4,0xb6,0x78,0x0b,0x9b,0x94, -0xba,0x63,0x71,0xad,0x39,0x6b,0xe6,0x95,0xf6,0xaf,0xef,0xdf, -0x3c,0xc0,0xe4,0x95,0x4e,0x5d,0x70,0xed,0x0b,0x45,0x39,0x39, -0x33,0x67,0x25,0x16,0x1b,0x40,0xa1,0x50,0x28,0x14,0x0a,0x05, -0xa0,0x00,0xf0,0xab,0x9f,0xa6,0x2f,0x78,0x40,0x2e,0x48,0x76, -0x5c,0x48,0x4e,0x56,0x66,0x5c,0x9a,0xcc,0x09,0x0d,0x7b,0x82, -0x24,0x73,0x17,0x43,0xa8,0x1f,0x52,0x1d,0x0d,0x01,0x11,0x0d, -0x00,0x88,0x58,0x01,0x43,0x80,0x60,0x55,0x24,0x82,0x52,0x24, -0x41,0x86,0x26,0x64,0x28,0x20,0x43,0x13,0x52,0xdf,0x24,0x66, -0xf8,0xab,0x68,0xaf,0x14,0x17,0x0f,0x14,0x61,0x9e,0x23,0x41, -0xa7,0x77,0xb4,0x6e,0xc8,0xc8,0xe0,0x88,0x0b,0x69,0x32,0x55, -0x47,0x02,0x52,0x90,0x0e,0x90,0x92,0x5e,0x01,0xd7,0x10,0xb9, -0x0a,0x70,0x8a,0xd7,0x57,0xf8,0x5a,0xfc,0x78,0x53,0xa3,0x8d, -0x54,0xaa,0x1a,0x68,0x45,0x37,0x2c,0x1c,0x2e,0xb2,0xbb,0x23, -0x2e,0x00,0xa4,0xae,0x79,0x4b,0xa0,0x88,0x31,0x76,0xb7,0xf9, -0x8d,0x70,0xfd,0x62,0x1e,0xd8,0xf0,0x9f,0xbf,0xf5,0x8e,0x3e, -0x7f,0xeb,0x83,0x65,0x0a,0x5f,0x74,0x38,0xf9,0xad,0x27,0xef, -0x9e,0xaa,0xc8,0x01,0x16,0xfd,0xe7,0xef,0x20,0xc7,0x3b,0xff, -0x1d,0xe4,0x78,0xc7,0x39,0xde,0x79,0xbb,0xb2,0x64,0x30,0x09, -0xc0,0x2e,0x94,0xa1,0x4b,0x05,0x7c,0xb5,0x2e,0x93,0x29,0x6e, -0xc0,0x56,0x74,0x54,0xb5,0xbc,0x98,0x3d,0x05,0x6a,0xce,0xff, -0xf3,0x1f,0xf2,0xe1,0xf3,0x9d,0x05,0xe4,0x50,0x40,0x0e,0x05, -0xe4,0xe8,0xb0,0x01,0x5f,0xa5,0x02,0xf2,0x72,0x01,0xf9,0x53, -0x20,0x5e,0x17,0x00,0x0a,0x91,0xf4,0x18,0x50,0x98,0x05,0xc8, -0x8c,0xb7,0xf4,0x98,0x5f,0x1e,0x60,0x35,0xdf,0x05,0x3e,0xa8, -0x4c,0xff,0xad,0xff,0x0e,0x9f,0xfd,0xc5,0x3b,0xbe,0xf3,0x08, -0x6a,0xf3,0xf9,0xdb,0xe8,0x65,0x00,0x3c,0x88,0x60,0xa0,0x36, -0x34,0x5c,0x5b,0xcf,0xbf,0x55,0x43,0xd2,0xea,0x4c,0x95,0xc7, -0x81,0x68,0x22,0xfb,0x02,0xaf,0x8a,0x41,0x59,0x08,0x40,0x5d, -0xdb,0x9b,0xa3,0x25,0xde,0x14,0xa8,0x0b,0x5e,0x6c,0xd6,0x58, -0x46,0xb9,0xe8,0x9d,0x9a,0xfd,0x2d,0x48,0x86,0xe2,0xf8,0x3b, -0xf7,0x5d,0xe8,0xb5,0x52,0xbc,0x27,0xdf,0x85,0x9e,0x05,0x69, -0xa1,0x67,0x9c,0xe2,0x55,0x09,0x00,0x17,0x54,0x25,0x78,0x30, -0x46,0x18,0x14,0xc5,0xfb,0x3f,0x5c,0xa6,0x33,0xcd,0x86,0xef, -0x17,0x2b,0xb1,0x29,0x71,0xe1,0xe1,0x09,0x6d,0x23,0xd8,0x31, -0xe0,0x8c,0x04,0x8b,0x0f,0x06,0x82,0xfd,0x74,0xf1,0x5b,0xc3, -0x71,0xe6,0xad,0xe1,0x2c,0xf3,0xce,0x70,0xa2,0x79,0xa7,0x1d, -0x67,0x40,0x60,0xce,0xdf,0xa2,0xa7,0xdd,0x45,0x04,0xa2,0x03, -0x99,0x51,0x86,0x42,0x98,0xee,0x40,0x86,0xf3,0x77,0x9c,0x00, -0x72,0x04,0xb9,0x51,0xa0,0x28,0x41,0x13,0xa7,0x77,0xf4,0xcc, -0xeb,0x49,0xfa,0xfd,0xb7,0x98,0x01,0xe6,0x91,0x6f,0x31,0x0b, -0xce,0x12,0x2b,0x2b,0xba,0x5f,0x7d,0x4c,0x47,0x20,0xcb,0xf9, -0xc7,0xa8,0x8f,0xe1,0x08,0x04,0xdd,0xe0,0x3c,0x63,0xb2,0xf1, -0x75,0xcb,0x20,0xc2,0x9e,0x41,0x64,0xe7,0xc1,0x79,0xce,0x09, -0xf8,0xe0,0x65,0x10,0x61,0x37,0xd9,0xb3,0x3e,0x45,0x3f,0x21, -0x23,0x0b,0x5e,0xc3,0xf5,0x31,0x32,0xf1,0x06,0xf2,0x29,0x89, -0x10,0x81,0xa2,0x3c,0x50,0x74,0x59,0x1a,0xde,0xec,0x5b,0xa4, -0x0b,0x3d,0x9d,0xb7,0x16,0xa1,0xb3,0xf3,0xc3,0x80,0x0a,0x15, -0x54,0x0b,0xb4,0xe7,0xe7,0xef,0xca,0x8d,0xb3,0x3f,0xfa,0x89, -0xdc,0xb9,0x9c,0xe4,0x15,0xe8,0x41,0xd3,0x4c,0x72,0x0b,0xfd, -0x23,0xad,0x0a,0xd3,0x3c,0xf0,0x95,0x23,0xa8,0x42,0xf7,0xd6, -0xe5,0x87,0x4b,0xd1,0x97,0x3c,0x2f,0xa5,0xbe,0x73,0xf9,0x71, -0x53,0xdd,0x90,0xfe,0xdb,0x0a,0x14,0xef,0x48,0x51,0x22,0x8a, -0x77,0x15,0x28,0xde,0x92,0x4a,0x35,0x50,0x18,0xf5,0x33,0x4c, -0x93,0xea,0xee,0xde,0xef,0xd3,0xd1,0xa5,0xe6,0xcb,0x80,0xf7, -0x1d,0x31,0xf0,0x32,0x10,0x3b,0x8d,0x66,0x7f,0x97,0x80,0x2f, -0x28,0x91,0x01,0x5f,0x50,0xa6,0x81,0x39,0xf2,0xc6,0xc3,0xf5, -0xa7,0xab,0x71,0xcd,0xc0,0xcb,0x89,0x45,0x25,0x59,0xa7,0x76, -0xfb,0xe9,0x62,0xb3,0x1a,0xc7,0x17,0xf7,0xf7,0xfc,0x51,0x3d, -0x25,0x10,0xaf,0x5a,0x03,0x14,0x7f,0x54,0x43,0x09,0x43,0x5a, -0x60,0x94,0x79,0x7c,0x0e,0x23,0x4f,0xf9,0x4b,0x3e,0x91,0x8d, -0x1e,0x93,0xf4,0xa5,0xc7,0x1a,0xa6,0xbd,0xc2,0x8e,0x2b,0xfa, -0xb7,0x49,0x08,0xf7,0x4f,0x24,0x9f,0xba,0x97,0xa0,0x40,0xd0, -0xbc,0x97,0x70,0x0a,0x58,0x59,0xd1,0x8b,0xea,0x25,0x15,0x9c, -0xe0,0x74,0x2e,0x54,0x09,0x5e,0x87,0xa1,0x8d,0x32,0x05,0x5b, -0xf6,0x2a,0x53,0xc0,0x2a,0x26,0xef,0x59,0x26,0x43,0x1b,0x65, -0x56,0xf4,0xc3,0x5d,0x7a,0x8b,0x73,0x75,0x02,0x8d,0x41,0x49, -0xa5,0x69,0xf8,0xeb,0x2d,0x30,0xb3,0xc1,0x8b,0x4b,0x2e,0xe6, -0x1c,0x90,0x01,0x0b,0x73,0x4a,0x11,0x69,0x9b,0xe0,0x8c,0xab, -0x00,0x29,0x22,0xed,0x69,0xbb,0x22,0x4d,0x5c,0x34,0xf4,0xc0, -0xe2,0x2c,0x43,0x3d,0x7c,0x30,0x59,0xde,0x6e,0x54,0xcc,0x66, -0xed,0xfb,0xab,0x58,0x1b,0xac,0xde,0x9b,0xba,0xbc,0x37,0xaf, -0x73,0x55,0x6e,0xe0,0x7e,0xd4,0xa6,0xf2,0xf0,0xb7,0x64,0xb6, -0x59,0x5f,0x0f,0xa7,0xda,0x45,0xb5,0x98,0x62,0xda,0x70,0xe9, -0x7f,0xfe,0xff,0x64,0xfe,0xdb,0xb6,0x27,0xcd,0x35,0xda,0xc7, -0x2e,0x4d,0x11,0x24,0x04,0xde,0x77,0x23,0x33,0xfe,0xfe,0xbf, -0x54,0x86,0x36,0x5f,0x45,0x48,0x0e,0xf4,0xbf,0xff,0x2f,0xdc, -0x5b,0xf7,0xba,0x6a,0x55,0xb0,0x4c,0xcc,0x23,0x96,0xe7,0xca, -0xb9,0x0b,0x2b,0x74,0x65,0x00,0xeb,0xf4,0xb8,0x91,0x9a,0xb0, -0x36,0x59,0xef,0xc9,0x0b,0xbe,0x51,0x99,0xeb,0x85,0xf3,0x32, -0x23,0x4d,0xad,0x0c,0x1c,0x1e,0x8e,0x8f,0xd5,0x6c,0x6e,0xcc, -0x8f,0x44,0xa9,0x3b,0xeb,0x2b,0x39,0xa0,0x88,0xf8,0x48,0x1e, -0xc8,0xfc,0x3b,0xb8,0x20,0x41,0x6c,0x6f,0xe5,0xd7,0x8b,0xf9, -0x25,0x9f,0x77,0x7e,0x84,0xb7,0xb2,0xca,0x54,0xe5,0xad,0xac, -0x12,0x4b,0x6e,0xd1,0x0f,0x57,0xb2,0xd2,0x31,0xba,0xa6,0x6e, -0xc5,0x64,0xb3,0x7d,0xaf,0xe6,0x8b,0x19,0x26,0x3f,0x4e,0xbe, -0xc3,0x63,0x1d,0xb9,0xf5,0xd5,0x24,0x5b,0xb7,0x98,0x42,0xfb, -0x88,0x56,0x92,0x79,0x0a,0xd4,0xcb,0x68,0xd9,0x37,0x67,0xf1, -0x6a,0x8c,0x6b,0xd7,0xa5,0x05,0x4e,0x5e,0x03,0xa3,0x42,0xfa, -0x99,0x2f,0x7b,0x14,0x90,0xc5,0xf3,0xc7,0x38,0xa3,0xb9,0x5f, -0x13,0xea,0x76,0xf9,0xfb,0xbf,0x0e,0xb4,0x2f,0xba,0x44,0x58, -0x4f,0xab,0x84,0xf8,0xbe,0x44,0xb3,0x4c,0x29,0xd0,0x2c,0xa3, -0x4d,0x4e,0x2f,0x56,0xeb,0xeb,0x05,0xd8,0x3d,0xcb,0xeb,0x9d, -0xdc,0xd6,0x3c,0x96,0x8c,0x85,0xae,0xa1,0x19,0x6b,0x61,0x79, -0x04,0x73,0xcd,0x7c,0x05,0x62,0xcd,0x24,0xfb,0xa6,0x1e,0xd0, -0xc0,0x7b,0x50,0x5c,0x29,0x1f,0xb6,0xd6,0xdb,0xad,0xf4,0xb0, -0x89,0x20,0xda,0x50,0x79,0x76,0xd1,0x8f,0xa8,0xa6,0x95,0xb1, -0x50,0x4f,0x2b,0xcd,0xf2,0x00,0xa1,0x69,0xec,0x37,0xfb,0xc8, -0x95,0xb1,0x9e,0x84,0x76,0x5a,0x32,0xbf,0x76,0x33,0x71,0xd4, -0x18,0x55,0x5c,0xce,0xdb,0xc0,0xa4,0xc6,0x95,0x5a,0xcb,0x3d, -0xdc,0x19,0x85,0x64,0x84,0xf5,0x0c,0xc1,0x2b,0x97,0xfd,0x31, -0x22,0x58,0xc6,0x52,0xa8,0x77,0x19,0x40,0x8c,0xdb,0x33,0xbe, -0x6b,0xab,0x95,0xde,0x5e,0xe9,0xaf,0x56,0xf5,0x1a,0x02,0xd2, -0x01,0xa9,0x17,0x98,0x6a,0x3d,0x2c,0xa2,0x86,0x73,0x99,0xaa, -0x17,0x30,0x17,0x2b,0x7d,0x0b,0x1e,0xa6,0x0c,0xe9,0x92,0x68, -0x09,0xf7,0x35,0xc5,0xf8,0x0e,0xa7,0x34,0xc7,0xd3,0x45,0x1a, -0x4f,0x9c,0x32,0xc4,0x6b,0x4a,0xc0,0xf7,0x60,0x96,0x66,0x22, -0xce,0xb9,0x7c,0x07,0x63,0x9b,0x7c,0x59,0xa1,0x53,0x4c,0xfd, -0x8c,0xa2,0x05,0x0c,0xdd,0x64,0x58,0x02,0xf9,0x94,0xee,0x37, -0x1c,0x41,0x7b,0xa5,0x66,0xd2,0x67,0x18,0xe1,0x3b,0x14,0xdf, -0x5c,0x2c,0xe3,0xb9,0x53,0x4a,0xfd,0x0e,0x62,0x25,0x44,0x05, -0xe9,0x04,0x23,0x28,0x1f,0x6d,0xe6,0x93,0x69,0x6c,0xa5,0x52, -0x8c,0x3f,0x1e,0xd2,0xf4,0x62,0x6a,0x26,0xbd,0x16,0x71,0xbe, -0x23,0x53,0xcb,0x04,0x48,0x18,0xa6,0x41,0xc1,0x55,0x90,0x21, -0x21,0x05,0x25,0xa0,0x47,0x17,0xeb,0xc5,0xdc,0xa2,0xe5,0x1b, -0x11,0xc7,0x8f,0xd5,0x54,0x36,0xe2,0xce,0x87,0x6b,0xf8,0xfa, -0x38,0x88,0x88,0x54,0x64,0x8b,0x29,0x89,0x8e,0x05,0x49,0xff, -0x7c,0xcc,0xef,0x8b,0x00,0x00,0x76,0xf2,0x82,0xbc,0x10,0xab, -0x3e,0x8b,0x7f,0x4b,0xe2,0x55,0x88,0x7b,0x08,0xed,0xe3,0x13, -0x3f,0x84,0x7f,0xfc,0x96,0x79,0x05,0x54,0x9b,0xdf,0x3a,0x3f, -0x21,0xc8,0x5a,0xa8,0x13,0x86,0x3a,0x13,0xf8,0xce,0x06,0xc2, -0x43,0x6d,0xd6,0xba,0xc6,0x6d,0x6a,0x55,0x15,0xec,0xeb,0xc9, -0x18,0x97,0xdf,0xd4,0xdc,0x10,0x01,0xdc,0x09,0xf4,0x72,0x72, -0x57,0xc6,0x0f,0xf5,0xe0,0x8d,0x70,0x55,0xee,0x0f,0xa8,0x16, -0x32,0xaf,0x3f,0xf1,0x13,0xff,0xbd,0x38,0x46,0x96,0xb5,0xfd, -0xbc,0xed,0x6f,0xfc,0x5b,0x7f,0xe8,0xa7,0x4b,0xff,0x32,0x33, -0x1e,0xe9,0x41,0x65,0x71,0x99,0x9b,0x11,0xb9,0xe7,0xcf,0x23, -0xa3,0x08,0x7f,0x09,0xbf,0xd1,0xbc,0x19,0xb2,0xc9,0x4b,0x07, -0xb4,0x61,0xd0,0x1e,0xdf,0xe0,0xc7,0x75,0x44,0x17,0x8a,0x00, -0xca,0x28,0x32,0x6a,0xfd,0xf6,0xf0,0x30,0xb7,0x63,0xde,0x79, -0x92,0x34,0xc2,0xcd,0x6f,0x4e,0xa0,0xd3,0x1c,0x1e,0x4e,0x56, -0x49,0x7d,0x7d,0x08,0x5b,0xc6,0xf1,0xf6,0x51,0xbf,0x71,0x99, -0xb5,0xf0,0x7a,0x78,0x9e,0xfa,0x4d,0x08,0x09,0x1e,0x59,0x4e, -0x3c,0xbf,0x71,0x99,0x9b,0x49,0x10,0x35,0x60,0x7f,0xbc,0x24, -0x0a,0x15,0x3a,0xc5,0x55,0xc8,0x84,0x37,0x0f,0xa8,0xe0,0xf5, -0x80,0xee,0xa5,0xb6,0x93,0xa3,0xc8,0x4e,0x3f,0x3c,0x34,0xd3, -0x03,0x3b,0x7b,0x40,0x67,0xb5,0xaf,0xa3,0x44,0x96,0x1a,0xa8, -0x52,0x93,0x83,0x28,0xba,0x86,0xc4,0x3c,0x8c,0xec,0x02,0x9a, -0x36,0x7e,0x5c,0xdd,0xb2,0x4b,0x68,0xda,0x25,0x08,0xde,0x33, -0x33,0x3e,0xd0,0xfe,0x49,0xc7,0x3c,0xf9,0x8a,0xeb,0x63,0x78, -0xb9,0xcc,0x24,0xce,0x3a,0xc9,0xd6,0x63,0x4a,0x44,0x1e,0x3a, -0x0f,0x6f,0xef,0xab,0xc8,0x79,0x22,0xef,0xc4,0x34,0x47,0xfc, -0xbb,0xf5,0xfc,0x21,0x5f,0x31,0x80,0x9b,0xa5,0x1c,0xe7,0xdf, -0x1a,0x31,0x84,0xdf,0xdf,0x40,0x8b,0x21,0x6b,0x51,0x2e,0x44, -0x3d,0x51,0x2a,0xdf,0x4b,0xc0,0x44,0x02,0x0e,0x23,0x23,0x8a, -0xb0,0x11,0xa0,0x59,0xd7,0xdb,0xaa,0xba,0xe6,0x36,0x40,0x05, -0xb7,0xda,0x1a,0xe0,0x7d,0x25,0x06,0x1b,0xa0,0x8a,0xdf,0x47, -0x59,0xd8,0xc8,0xc3,0xa3,0x3c,0x3c,0x8f,0xb2,0xf6,0x51,0xd6, -0x6e,0xe4,0xed,0xa3,0xbc,0x2d,0xce,0x56,0xe9,0x4a,0x44,0xcd, -0xb0,0x2b,0x5f,0x41,0xd9,0x8a,0xb4,0x8d,0x4e,0xdb,0x0e,0x8b, -0x55,0x04,0xf6,0x24,0xc0,0xb4,0xf7,0xf4,0xf8,0x09,0xf5,0x13, -0x6e,0xb4,0x6b,0x8f,0xce,0xe1,0xbf,0x07,0xe9,0x68,0x13,0xd7, -0xde,0x7b,0x22,0xe7,0x7b,0x81,0xee,0xbc,0x19,0xd2,0xc9,0x47, -0x95,0xa5,0x90,0x0e,0x08,0x1a,0x0d,0x6e,0xd9,0x74,0x19,0x31, -0x9c,0xb8,0xff,0x80,0x9a,0xe2,0x3d,0x21,0xc5,0xe3,0x0c,0xc5, -0x9c,0xaf,0x02,0xf1,0x80,0x4e,0xf7,0x80,0x47,0xe0,0xd9,0x05, -0x6a,0x94,0xd7,0xaf,0xff,0xc6,0xc5,0xf5,0xd3,0x65,0xb3,0x2d, -0x7a,0x35,0x7e,0x87,0x52,0xde,0x14,0x06,0xd5,0x45,0xbd,0xae, -0xd7,0x6c,0xa6,0xcb,0xae,0x84,0x6d,0x34,0x06,0x51,0x01,0x56, -0x9c,0xd6,0x9b,0xe6,0xd8,0x9d,0xb9,0x47,0x20,0xce,0x6e,0x72, -0x0e,0x75,0x6f,0x36,0x13,0x0f,0xd3,0xb8,0x86,0xd4,0x8b,0x19, -0x55,0x32,0x18,0x78,0xfa,0x0c,0xe1,0x34,0xdf,0xaa,0x53,0xee, -0x4a,0xa9,0xf0,0x0a,0xa1,0xd2,0x28,0xdd,0xba,0xd5,0x0e,0xd2, -0x96,0x8a,0xe0,0x8e,0x8b,0xe1,0x56,0xb6,0xe7,0x4a,0x75,0x74, -0xe1,0x23,0xbc,0xd7,0xc9,0xb6,0xf4,0xd1,0xca,0xf7,0xcb,0x98, -0xab,0x8c,0x39,0x67,0x54,0xab,0x18,0xca,0x7b,0x75,0xb7,0xe7, -0x03,0x56,0x6d,0x33,0x1a,0x4f,0x87,0x69,0x6a,0xa6,0xfb,0xb2, -0xc5,0x04,0x12,0xf5,0x5a,0xb1,0xe5,0xdc,0x60,0x2e,0x61,0x94, -0x41,0xa3,0x02,0x21,0x86,0x6f,0x71,0x7f,0xd0,0xad,0xcb,0x55, -0x70,0xde,0x34,0x1e,0xe0,0x91,0x77,0xd0,0xf1,0x2b,0xdf,0x3c, -0x4a,0xf0,0x9b,0xd5,0xfd,0x39,0x0a,0x0f,0x65,0x0c,0xe4,0x85, -0x74,0x78,0xd3,0x84,0x48,0x4d,0x38,0xad,0xc1,0x5b,0xb6,0x6a, -0x47,0xf7,0x68,0xa4,0x8f,0xcc,0xb4,0x4e,0x8f,0x10,0x64,0x5b, -0x5b,0xef,0x0a,0x9f,0xce,0x1b,0x71,0xa3,0x9a,0x3f,0xae,0xa2, -0x2c,0x8b,0xf0,0xba,0x81,0xc0,0xa4,0x90,0xc4,0xb2,0xd8,0x86, -0xf7,0xf7,0xee,0x0d,0x5e,0xa1,0xe1,0x9e,0xf1,0x53,0xd3,0x43, -0x3a,0xc1,0x09,0xba,0xb4,0xb2,0x1e,0xe3,0x48,0x51,0x4e,0x37, -0xe6,0x1c,0xd1,0x8d,0x19,0x59,0x23,0xa2,0x5b,0x3b,0x1a,0x74, -0x75,0xc6,0xd1,0xd8,0xcf,0x29,0x22,0xc4,0x88,0x10,0x23,0xba, -0x72,0x3a,0x88,0xc7,0x5e,0x8e,0x6e,0x06,0xbb,0xea,0x89,0x87, -0xaa,0xf5,0x82,0xdc,0x46,0xac,0x7d,0xa9,0x6b,0x76,0xe7,0xcb, -0xcd,0x1a,0xc6,0xf8,0x99,0x3f,0x82,0x7a,0x4f,0x7c,0x61,0xe2, -0x14,0xd0,0xb1,0x9d,0xa5,0x72,0x57,0xf0,0xa7,0x59,0x9d,0x83, -0x2f,0xa0,0x1d,0x1a,0x3c,0x13,0x6c,0x20,0x02,0xa8,0xf0,0x48, -0x60,0x15,0x9b,0x6d,0x78,0xe9,0x3a,0x87,0xc5,0x92,0x59,0x60, -0xb1,0x8b,0xb2,0xf4,0xdd,0x19,0x7f,0xc8,0xc2,0x99,0x6a,0x0f, -0xa5,0xe6,0x3d,0xea,0x5a,0x50,0x61,0xaf,0x66,0x5d,0x6f,0x22, -0xc0,0xdf,0x0f,0x8a,0xfc,0xf9,0x6a,0x9e,0x82,0xa5,0xef,0x4e, -0x7c,0x1c,0xc8,0x7a,0x6e,0x75,0xea,0x98,0x52,0xef,0xef,0x25, -0x41,0xac,0x6c,0x8a,0xa0,0x6b,0x9c,0x81,0x8c,0xd7,0x2e,0x72, -0x0f,0xe1,0x75,0x05,0x18,0xde,0x93,0xd7,0xc9,0x56,0xa3,0x3f, -0x3c,0x7c,0x2c,0x7a,0x98,0xb3,0x76,0xff,0x89,0x6b,0x5c,0xc8, -0x2c,0x02,0x28,0x34,0xc0,0xf3,0x91,0x74,0xd3,0x12,0xd1,0x5a, -0x93,0xe0,0xbb,0x39,0x9b,0xf9,0x30,0xaf,0xb2,0x16,0x51,0x28, -0xe2,0x09,0xde,0xbf,0xa2,0x6c,0x27,0x6b,0x67,0x57,0x4d,0x19, -0x07,0x30,0xd0,0xaf,0x57,0x09,0x0d,0x57,0xe6,0x33,0x54,0xb2, -0x06,0xb7,0x8b,0x15,0x98,0xe0,0xc9,0x8f,0x71,0x9a,0xc6,0x53, -0x5a,0xe4,0x55,0x66,0xa5,0x42,0x06,0x43,0x39,0x95,0xd5,0x8f, -0x5b,0xab,0xf8,0x0a,0xdd,0xa0,0xa6,0xac,0xfd,0x07,0xe2,0x0e, -0x67,0x39,0xde,0xaa,0xf4,0x95,0x48,0xc7,0x87,0xa8,0x28,0x6b, -0xd9,0x88,0xc7,0x68,0x7c,0x84,0x8e,0x2c,0x59,0xd3,0x20,0xf2, -0xc7,0x59,0x74,0x8b,0x03,0xfb,0x38,0x87,0x5f,0x90,0x45,0x84, -0x2c,0xe7,0xbf,0x85,0xac,0xc2,0x11,0xc7,0x30,0x8e,0x30,0x63, -0x73,0x0c,0x43,0x18,0x76,0xd5,0x71,0x2e,0xcb,0x7f,0x8c,0x5d, -0xa4,0x8e,0x77,0xe0,0x3d,0xd6,0xb3,0x88,0xb2,0xcb,0xeb,0x75, -0xba,0x33,0x18,0x79,0xdf,0xc3,0xc8,0xac,0x58,0x2a,0x4c,0xd6, -0x5b,0x2a,0x06,0x45,0x98,0x7f,0x1b,0xe1,0x80,0x2a,0xaf,0xe0, -0x74,0xb3,0x0a,0x8e,0xeb,0x56,0xe5,0x21,0xd5,0x98,0x01,0x08, -0x08,0x31,0x09,0x50,0xe7,0xe1,0xd9,0xd0,0x17,0x22,0xc7,0xa6, -0xf5,0x8e,0x46,0x7f,0xac,0xe5,0xff,0x73,0x14,0xc6,0x67,0x35, -0xf6,0xfd,0x65,0xc9,0xc0,0x17,0xd3,0x03,0xdb,0xbc,0x17,0x91, -0xfa,0xed,0x80,0x39,0xd9,0x41,0xca,0x42,0x56,0xd6,0x53,0xf4, -0x48,0x13,0x5f,0x5c,0x6e,0x5e,0x23,0xb0,0x82,0x39,0xa6,0xb8, -0xd2,0x06,0x57,0xe8,0xa7,0x6d,0x9c,0x15,0xe1,0x94,0x08,0xe6, -0x46,0xed,0x6e,0x08,0xb5,0x80,0x91,0xee,0xf0,0x30,0x6e,0x8d, -0xc0,0x3e,0xe9,0xb9,0x69,0x08,0xe1,0x78,0xd9,0x5a,0x01,0x24, -0x7f,0xc1,0x90,0xae,0x62,0xa7,0x2a,0x76,0xe5,0xf9,0x22,0x6f, -0x0f,0xcd,0xfb,0x34,0xec,0xa5,0x61,0x2b,0xef,0x34,0x7f,0x46, -0x5b,0x1e,0x94,0x77,0x13,0x10,0x1e,0x51,0x19,0x51,0xda,0xee, -0xa5,0x6d,0x48,0xfb,0x19,0x0d,0x57,0x95,0xd4,0x06,0xb4,0x99, -0xcc,0x98,0x61,0xc6,0x5c,0x66,0x1c,0x1e,0x11,0x89,0x22,0x63, -0x06,0x19,0xf3,0xb6,0x4e,0x6a,0xf3,0x84,0xf5,0x16,0xe6,0xa6, -0xbc,0x97,0xed,0xdf,0xc2,0x04,0x94,0x26,0x79,0x83,0xae,0x14, -0x85,0xda,0x7e,0x19,0x02,0xb4,0xe7,0x97,0xc1,0x56,0x15,0x60, -0x5b,0x0d,0x18,0xc9,0x0f,0x5b,0xbc,0x44,0xac,0xec,0xb4,0xd0, -0x4d,0x55,0x83,0x8a,0xee,0xaa,0xc3,0x68,0x18,0x50,0xdf,0x94, -0x56,0x88,0x85,0xe9,0xb6,0xe0,0x26,0x53,0xd7,0x75,0xc1,0xe4, -0x89,0xe9,0x11,0x57,0x76,0xc4,0x91,0xb8,0xc4,0xde,0xce,0xae, -0x4e,0x8d,0x65,0xf7,0x87,0x60,0x71,0xd0,0xc7,0x08,0x3a,0x84, -0x54,0x7c,0x84,0xb3,0xe4,0xb8,0x33,0xd1,0xef,0x61,0x1e,0x24, -0xf7,0xf7,0x9c,0x6d,0x22,0xf3,0x53,0x9e,0x7e,0x02,0x03,0xd7, -0x40,0x7b,0xf2,0xe8,0xbc,0xaa,0x58,0x41,0x20,0x64,0xdc,0x9a, -0xfc,0x2c,0x2b,0xaf,0x02,0x2b,0xe7,0x91,0x5d,0x35,0x94,0xf7, -0x83,0xb9,0x67,0x63,0x15,0x9a,0x06,0x64,0xa7,0xf9,0xf3,0xc0, -0xc7,0x5f,0xfc,0x91,0x7f,0x21,0x8e,0x2e,0x47,0x3d,0x70,0xe7, -0xe7,0x6d,0x71,0xb9,0x26,0x5e,0x07,0x29,0x5b,0xc4,0x5f,0x86, -0xb2,0x10,0x9a,0x62,0xb6,0x55,0x88,0x5c,0x30,0xa2,0x65,0x20, -0x7c,0x34,0xe0,0x83,0xe7,0xa8,0xcb,0x50,0xcc,0xcf,0xe0,0x83, -0xe7,0x61,0xcb,0xb6,0x98,0x6f,0xc1,0x07,0xc4,0x4c,0x32,0x7d, -0x4a,0x33,0x6b,0x37,0x71,0x52,0x3a,0xc9,0xa3,0xbc,0xdd,0xcc, -0xe9,0xc1,0x4a,0x7d,0x82,0x33,0xf7,0x5e,0xfd,0xe7,0x3f,0x98, -0x24,0x48,0x0f,0xcf,0xf3,0xa0,0xd7,0xfc,0xb9,0xf3,0x73,0xb7, -0x54,0xb1,0x9c,0xaa,0x92,0x0f,0x8c,0xdb,0x45,0xff,0x71,0x3e, -0xc9,0xe4,0xad,0x24,0x59,0x78,0x9e,0x55,0x66,0xcd,0x98,0x25, -0x19,0xf0,0x82,0xb3,0x8a,0xb2,0xa0,0xcf,0x35,0xb3,0xc0,0x3b, -0x72,0x89,0x28,0xef,0x5c,0x90,0x09,0xe1,0xb0,0x99,0x07,0x5e, -0xef,0x67,0xec,0x89,0xbf,0x45,0x26,0xa5,0xcd,0x49,0xd6,0x55, -0xe1,0xdf,0x90,0xf0,0x9e,0x5d,0x56,0x70,0x3e,0xc9,0x7b,0xd0, -0xf7,0x73,0xa4,0xb3,0xe3,0xfe,0x86,0x0f,0x1a,0xb8,0xb9,0x78, -0x65,0xac,0xb6,0x42,0x78,0xb0,0x9f,0x2f,0xb3,0x11,0x7e,0x50, -0x5e,0x71,0xf9,0xc0,0x48,0x23,0x6b,0xd0,0x55,0xdd,0x36,0x19, -0x94,0x17,0x1b,0x8c,0x44,0x79,0x1c,0x93,0x35,0x68,0xd7,0x16, -0x9a,0xf4,0xe1,0xa9,0x95,0x49,0x15,0x1d,0xa2,0xa8,0x9d,0x6b, -0xc9,0x71,0x49,0x0c,0x58,0x1d,0x57,0x44,0x3c,0x66,0xbe,0x25, -0xb2,0xd0,0x94,0x4b,0x66,0x7f,0xc4,0xac,0x4b,0x65,0xcf,0x75, -0xf6,0xaa,0xfb,0x95,0x1e,0x38,0xeb,0x6f,0x54,0xf9,0xf0,0xb0, -0x6f,0x84,0x48,0x91,0x19,0xc1,0xf6,0x80,0x06,0x1f,0x3e,0x5b, -0x72,0xe1,0x95,0x58,0xa5,0x84,0x8d,0x3d,0x9d,0x02,0xe3,0x9a, -0xa3,0x40,0x2e,0x5f,0x68,0x0f,0xa8,0xb6,0xf2,0x7a,0xea,0x9a, -0x98,0x0a,0x33,0x36,0x3c,0x0c,0xcf,0xba,0x9e,0x7e,0xdb,0xf8, -0xdb,0xd6,0xbf,0x64,0x59,0xc8,0x37,0xef,0x98,0x05,0x9a,0x17, -0x69,0xf2,0xdb,0x7e,0xfe,0x10,0x65,0xe6,0xb9,0x17,0xc0,0x0a, -0xf6,0xa2,0xbe,0x00,0xc9,0xb6,0x58,0x64,0xb3,0x44,0x17,0x07, -0x02,0x74,0x63,0x78,0x1b,0x56,0x1a,0x2c,0xb4,0xe5,0xf1,0x90, -0xb5,0xc2,0x88,0xfa,0x83,0xc7,0x9a,0x2d,0xff,0xb7,0x1a,0x2c, -0x0f,0x5a,0xd9,0x35,0x46,0xcb,0x34,0x2a,0x8e,0xe5,0xfe,0x2a, -0x2a,0x8e,0xdb,0x5d,0x62,0x69,0x7f,0x3a,0xe8,0xaf,0x06,0x30, -0x91,0x35,0x42,0x11,0x7f,0xaf,0x06,0x10,0xc4,0xfb,0xbd,0x88, -0xad,0x62,0x21,0x91,0xfd,0x34,0x3a,0x44,0xf8,0x74,0x20,0x9c, -0x31,0x38,0xb8,0x82,0x26,0xf0,0x64,0x2b,0xe8,0xd6,0xd5,0xf3, -0x86,0x42,0x0b,0xe3,0xc2,0xde,0xc3,0x8b,0xc0,0x96,0x2f,0x90, -0x9e,0xce,0x30,0x0e,0x7d,0xa5,0xa5,0x5f,0xd5,0x0a,0x0f,0x8b, -0x81,0x98,0xdc,0x9a,0xb2,0x20,0x26,0xad,0xee,0x47,0x37,0x9b, -0xd7,0xc2,0x74,0x98,0xb8,0xd5,0x91,0x2e,0x34,0x9f,0x2d,0xc8, -0x92,0x4d,0xfa,0x61,0x15,0x11,0x51,0x34,0x9f,0xf0,0xda,0xab, -0x0f,0xe6,0xd5,0x59,0x54,0x1c,0x9a,0x0a,0x8a,0xe7,0x9e,0xbd, -0xb8,0x23,0x62,0xbf,0x5b,0x2e,0x17,0x69,0xb2,0x8e,0xa3,0x0f, -0xd3,0x8e,0xb3,0x72,0xfc,0x55,0xc7,0x99,0x3a,0x5b,0xb5,0xa5, -0xf0,0xeb,0x66,0x38,0x59,0xaf,0xe2,0xb8,0x38,0xa5,0x30,0xb7, -0x04,0x8c,0xe9,0x85,0x04,0x17,0x8d,0xa9,0xe2,0x93,0x79,0x8a, -0x67,0x6f,0xe6,0xc0,0x0b,0xf4,0x42,0xb6,0x32,0xa3,0x32,0x4d, -0xd2,0x6f,0x87,0xdf,0x42,0x5b,0x1c,0x1e,0x8a,0xcf,0xdc,0xc3, -0x15,0x48,0x34,0x5a,0x86,0xe2,0x75,0xf4,0x79,0x16,0xcd,0x5b, -0x99,0x3f,0xcf,0xe1,0x27,0x97,0x8a,0xf3,0x20,0x9a,0x67,0x9e, -0x69,0x10,0xcc,0xb3,0x66,0xe6,0x35,0x74,0x30,0x6f,0x82,0x85, -0xd0,0x0a,0x42,0x8f,0x29,0x78,0x7d,0x9d,0x4c,0x27,0x95,0x64, -0x68,0x75,0x3b,0xe7,0x7b,0x46,0xe6,0x3c,0xf8,0x75,0xa1,0x50, -0x2c,0x51,0x86,0xf9,0xbc,0xb3,0x8d,0x8d,0x73,0xf8,0x73,0xa4, -0xce,0x42,0xea,0x3f,0x54,0x2a,0x1b,0x23,0x58,0x04,0xe4,0x85, -0x42,0x72,0x5f,0x16,0x33,0x11,0x76,0xca,0x43,0x08,0x0a,0x2c, -0xae,0x87,0xe4,0x89,0x4b,0x16,0xb5,0x4e,0x8f,0xc0,0x76,0x69, -0x80,0xf9,0xef,0xa7,0x39,0x85,0xf2,0xb0,0x81,0xb4,0xae,0x92, -0xab,0xeb,0x75,0x94,0x9d,0x47,0x69,0xe6,0x8f,0x16,0xeb,0xf5, -0x62,0x16,0xe5,0x10,0xc8,0xa1,0x3b,0xb8,0x1c,0x7e,0xf5,0x2a, -0xf4,0x1a,0x04,0xd6,0xe5,0x96,0xc1,0x2b,0xf5,0x60,0xb4,0x6a, -0xcd,0x17,0x13,0x9a,0x5f,0x83,0xc6,0xd0,0x01,0xb5,0x84,0x24, -0x85,0xe2,0x5b,0x7c,0xe7,0xcb,0x13,0x05,0xf5,0x70,0xce,0x92, -0x75,0x70,0x76,0x22,0x8b,0xeb,0xe1,0xfc,0x27,0xef,0xe0,0x4c, -0x27,0xf7,0x77,0x48,0xcc,0x96,0xfb,0x76,0x96,0xfa,0x79,0x0a, -0x83,0xc0,0x1c,0x12,0x2f,0x20,0xf5,0x02,0x92,0xe1,0x17,0xfe, -0x3d,0xd8,0xc3,0xb5,0xf8,0x80,0xc9,0x06,0xb9,0x23,0x42,0x7f, -0x11,0x51,0x11,0x10,0xc2,0x62,0x20,0xd4,0x56,0x4f,0x9a,0x67, -0x14,0xbc,0x88,0x9a,0x6e,0x46,0x60,0x17,0x7c,0xfc,0x36,0xa3, -0x71,0x25,0xa7,0xbf,0xf6,0x94,0x97,0xee,0x0d,0x5e,0x97,0x2c, -0x31,0xad,0x25,0x26,0xad,0xec,0x15,0xe0,0xc2,0x6b,0x4d,0x01, -0x1b,0x84,0xc0,0xc2,0x6d,0xe5,0xaf,0x72,0x8a,0xca,0x29,0x2a, -0xc7,0xa8,0xec,0x1c,0xca,0x46,0xa8,0xb6,0x86,0x3a,0xcf,0x29, -0x2a,0x6f,0x0b,0xa8,0x4c,0xe8,0x61,0x4a,0xcf,0x55,0x20,0xf7, -0xac,0xed,0x31,0x45,0x04,0x1b,0x15,0x17,0x11,0x28,0x30,0xd7, -0x52,0x5b,0x39,0xc6,0xe5,0x34,0x61,0xe9,0x02,0x55,0xe7,0x99, -0x24,0x2f,0x43,0x97,0x69,0x88,0xc8,0x25,0x71,0x39,0x44,0x64, -0x17,0x06,0x69,0x04,0x71,0x61,0x10,0x46,0x10,0x82,0x14,0x4a, -0x14,0xdf,0x10,0xcf,0xed,0x07,0x22,0xdf,0xbe,0x68,0x62,0xd3, -0x91,0x65,0x7f,0x01,0x56,0xf4,0x45,0x77,0x92,0x91,0x09,0xdc, -0x26,0x1e,0x37,0x26,0x28,0x1f,0xd8,0x38,0xf0,0x99,0x93,0xf2, -0x5a,0x2d,0x16,0xeb,0x1a,0xb9,0xc2,0x36,0xc5,0x64,0x7c,0xab, -0x3c,0x32,0x27,0x50,0x42,0x92,0x30,0x0d,0x84,0x89,0x2a,0x4d, -0x37,0x16,0xfb,0xb2,0xae,0x45,0xf9,0xc1,0x47,0xd9,0x10,0xd1, -0x6d,0x02,0x8a,0x51,0xa3,0xba,0x94,0x37,0x96,0xe8,0x92,0x7f, -0x42,0x08,0xf7,0x92,0xc0,0x2b,0xb0,0xf0,0x38,0x42,0xf6,0x1d, -0xc8,0x19,0xef,0x9c,0x88,0x81,0xc4,0x22,0x4a,0xb4,0x40,0x46, -0x93,0xa9,0x9c,0xfe,0x16,0x91,0xe1,0x86,0x07,0x6b,0x04,0x92, -0x32,0x39,0xeb,0x93,0x15,0x56,0x53,0x50,0x90,0xc8,0x9c,0x4d, -0x91,0x68,0xc2,0xaa,0x0a,0x41,0xb6,0xfc,0x62,0x0d,0x8a,0xa4, -0xbf,0xcf,0x5e,0x08,0x5f,0x7b,0xbd,0x2e,0xbd,0xd1,0xed,0xf5, -0xd4,0x25,0xbc,0x9a,0x09,0xaf,0x09,0xf6,0xad,0xf2,0xbb,0x2e, -0xa6,0xbc,0xf3,0x4f,0x60,0xf8,0x66,0x8c,0x24,0x1c,0xdc,0xcd, -0xb8,0xcf,0xc1,0x5f,0x7c,0xec,0x5b,0x8d,0x1b,0x62,0x14,0x84, -0x09,0x80,0x8c,0x7a,0xcc,0x0c,0x40,0xe6,0xa1,0x29,0x80,0x42, -0xf0,0x88,0x39,0x80,0x46,0x90,0x1b,0x08,0x1e,0x71,0xda,0x43, -0x9a,0xf3,0xa8,0xde,0xb0,0x76,0x50,0xd9,0x36,0xb5,0x03,0x2d, -0xd2,0x3c,0xc6,0x8c,0xf7,0x0c,0x62,0xa4,0x0c,0xb1,0x3f,0xaf, -0x61,0xc3,0x93,0xed,0x3e,0x30,0x48,0xdd,0xdb,0x48,0xdf,0x4d, -0x28,0xc4,0x04,0x82,0x30,0x83,0xaa,0x1d,0x24,0xd1,0xec,0xd7, -0xa7,0xb9,0xb0,0x41,0x0e,0xef,0x5d,0x19,0x6e,0xbe,0x3f,0x5c, -0x8d,0xe8,0x65,0x41,0x2b,0xa6,0x00,0xc3,0x2f,0xa8,0x15,0xc0, -0xbe,0x93,0x1e,0xb8,0x26,0xe4,0xb7,0x9b,0xd9,0x88,0x16,0x71, -0x2e,0x4a,0x91,0x05,0x48,0x76,0x7c,0x2f,0x40,0x72,0xa4,0x34, -0x8a,0x8c,0x84,0x8b,0x39,0x23,0x3e,0xee,0x37,0x1b,0x83,0x9e, -0xdb,0xeb,0xfc,0x32,0x69,0xfc,0xd2,0xea,0xfd,0x32,0x39,0xba, -0xa7,0x9f,0x86,0x07,0x71,0xfd,0xf8,0x2f,0x03,0x4a,0xa7,0xf7, -0x32,0x8f,0xaf,0xba,0x76,0x89,0x85,0xb2,0x6c,0x7a,0xd0,0x7f, -0xa8,0x5f,0x71,0x44,0x51,0x3a,0x6e,0x8f,0x64,0x3f,0x74,0xf8, -0xe9,0x42,0x7c,0x5e,0x74,0x8d,0x2f,0xd9,0xad,0xae,0x46,0x17, -0xf3,0xe1,0x2c,0x4e,0xf1,0xc9,0x3b,0x17,0xb7,0x26,0x8e,0xff, -0xee,0xfe,0x8f,0x7b,0x88,0xfe,0xc5,0xbd,0xbf,0x4e,0xa7,0xbf, -0xb8,0x9e,0x70,0x49,0x19,0xd1,0xcb,0x77,0x36,0xa3,0x3b,0x55, -0xf5,0xef,0x8c,0xd0,0x74,0x20,0x5f,0x7a,0x28,0x19,0x20,0x5e, -0x2f,0xa6,0x8b,0x55,0x45,0x66,0x87,0xbd,0xa0,0x99,0x16,0xb2, -0xa6,0x5b,0x49,0xca,0x56,0x75,0xa9,0x30,0x8a,0xee,0x54,0x35, -0x61,0xa7,0xaa,0xb5,0x3c,0xe2,0xc1,0x76,0x50,0x68,0x35,0xc2, -0x12,0x55,0x60,0x96,0x6d,0x16,0x0f,0xd3,0xf8,0x62,0xc2,0x6f, -0x11,0x57,0x7b,0x76,0x49,0x6f,0x2e,0xda,0x1f,0x43,0xf0,0x0a, -0xbf,0x2d,0x13,0x0b,0xad,0xb3,0xa8,0x48,0x0c,0x90,0x30,0x57, -0x3c,0x71,0x28,0x61,0x30,0x79,0xeb,0x8f,0x37,0xa3,0x64,0xbc, -0x03,0x8a,0xd2,0xb7,0x78,0xc0,0x72,0x07,0x50,0x8a,0xe7,0x80, -0xe3,0x6c,0xb9,0x03,0x04,0x52,0xe5,0xf1,0xd7,0x5d,0xa5,0x11, -0x00,0xe0,0xc2,0xf7,0x86,0x81,0x2e,0x95,0x9b,0xc3,0xfe,0x68, -0x38,0xbe,0x51,0x91,0x18,0xa0,0x47,0x11,0xc6,0xbb,0x50,0x32, -0x80,0x78,0x29,0x94,0x62,0x66,0x30,0xec,0x2a,0x6e,0x3a,0x09, -0x7b,0x72,0xa9,0x7b,0x38,0x16,0x9b,0xb5,0x2a,0x41,0x1c,0xa7, -0xf3,0x01,0xa8,0x09,0xf1,0x8e,0x91,0x40,0xaf,0xf3,0xf8,0x0e, -0xc4,0x36,0x11,0x83,0x39,0xd8,0x16,0x08,0x10,0xb0,0x6e,0x01, -0x29,0x40,0xe2,0xfb,0xa5,0xd8,0xfb,0xa8,0x79,0x15,0x06,0xec, -0x26,0xf2,0x21,0x7f,0xfc,0xe6,0xf9,0x2d,0xbe,0xbb,0xdc,0x74, -0xf0,0xa2,0x5a,0x74,0x23,0xe8,0x51,0x82,0xfd,0xd8,0x72,0x07, -0xe3,0xfc,0x59,0x65,0x3a,0xbe,0xa9,0xdc,0xc1,0xaa,0xaa,0xab, -0xb7,0x22,0x41,0x0e,0x3f,0x95,0xcd,0xef,0x64,0x5b,0xf2,0x34, -0x8b,0x4c,0x7e,0x11,0xd8,0x8c,0xc1,0x14,0xab,0x54,0xab,0x4d, -0x87,0xb3,0xa5,0x3b,0x73,0xe5,0xe9,0x14,0x1a,0xbe,0xb9,0xa3, -0xa9,0x8d,0x76,0xde,0x23,0xe5,0xc9,0xa5,0xd2,0xe9,0x7e,0x48, -0xaf,0xb8,0x16,0x7a,0xcf,0x97,0xe3,0x69,0xa1,0xef,0x40,0x4c, -0x11,0x26,0x2d,0xc1,0xa4,0x45,0x98,0xaf,0x87,0x45,0x9d,0x0d, -0x31,0x45,0xbd,0x4e,0xaf,0xa8,0x14,0x74,0x06,0xc6,0x21,0x5c, -0xc5,0xcd,0x69,0xcc,0x4f,0x6e,0x1e,0x52,0xcb,0x93,0xc5,0x98, -0xea,0xd2,0x1a,0xe3,0xb1,0x92,0xf8,0x2f,0xd3,0x18,0x43,0xdf, -0xbe,0x41,0x8f,0xd2,0x79,0xda,0x92,0xef,0x6f,0xdf,0x5e,0xf9, -0xf8,0x7c,0xba,0x54,0x8f,0xbb,0x91,0x2b,0x23,0x5f,0x46,0x5c, -0xb5,0xd2,0x78,0xfd,0xe9,0x1a,0x42,0xa3,0xcd,0x3a,0x76,0x1d, -0x95,0xd7,0xf1,0x05,0x48,0x97,0xd5,0xf0,0x95,0x46,0xdb,0x1a, -0x41,0xd3,0xfc,0x34,0x9c,0xa2,0x07,0x7b,0xba,0x98,0x26,0x13, -0xba,0xbd,0x53,0x2e,0x92,0x89,0x0b,0x5f,0xf4,0x3d,0x6e,0xeb, -0xde,0x1a,0x3a,0x04,0x20,0xcb,0x3a,0x66,0xfc,0x57,0xca,0xa9, -0xd4,0x93,0xe4,0x91,0x46,0xaa,0xb8,0xa6,0x4d,0xbd,0x52,0x5c, -0x56,0x68,0x8e,0xbe,0xf4,0xd2,0x69,0xf0,0x33,0xac,0x32,0xa2, -0xe1,0x78,0xe2,0x62,0x04,0x91,0xc2,0x21,0x88,0x4e,0x6f,0xe2, -0xbb,0xb7,0x32,0x16,0x03,0x18,0x47,0x97,0x55,0xca,0x38,0x0c, -0x40,0xa4,0xa3,0xd6,0x07,0x4a,0x64,0x47,0x1f,0x86,0x9d,0xd0, -0x1f,0x75,0x02,0x7f,0x0c,0xff,0x26,0xf0,0x1d,0xc3,0xef,0x65, -0x27,0xd8,0x16,0x46,0xbd,0x1f,0x55,0x6b,0xd8,0xb2,0xa0,0xe2, -0x51,0x1e,0xa6,0xc3,0x1c,0xfa,0xbb,0x70,0xcc,0xe5,0x40,0x8b, -0x5d,0x49,0x2b,0x74,0xb8,0x8a,0xa1,0x05,0x24,0xed,0xb6,0x80, -0x17,0xcd,0xd0,0xbc,0x4b,0x2c,0xd6,0xf0,0x72,0x94,0xbd,0x5a, -0x43,0x30,0x6a,0x57,0x9d,0x4b,0xba,0xe0,0x92,0xbe,0xc7,0x9b, -0xdd,0x28,0x0f,0xae,0x81,0x7b,0xe6,0x65,0x63,0xe9,0xd6,0x24, -0x6c,0x7c,0xbd,0x58,0x4d,0x2a,0x0f,0xda,0xad,0x62,0x06,0x11, -0x37,0x6e,0xdc,0xc0,0x3c,0x35,0x0b,0xc8,0x5d,0x12,0x14,0x06, -0xbd,0x1c,0x9c,0x62,0x1d,0xe9,0xeb,0xcd,0x66,0x46,0xc4,0x52, -0x80,0x1e,0x80,0xd7,0x5b,0x4f,0x73,0x4f,0x65,0xe0,0x14,0xe1, -0xee,0x44,0x25,0xeb,0x5c,0xf4,0x75,0x43,0x3e,0x73,0xe4,0xdf, -0xa0,0x9d,0x28,0x32,0xf2,0x07,0x13,0x4e,0x0f,0x10,0x99,0x35, -0x22,0x16,0x41,0xdc,0x2d,0x7b,0x3f,0xe8,0x2a,0x0a,0xc4,0x6c, -0xac,0x50,0x9e,0x64,0x90,0x22,0xc7,0xf3,0x6f,0x1a,0x51,0xc6, -0x8e,0x67,0x8b,0xd5,0xfa,0xff,0xa5,0xd2,0x0f,0x0f,0x8d,0x1c, -0xf5,0xbb,0x63,0x3a,0x83,0xab,0x8a,0xed,0x0f,0x07,0x9a,0x0b, -0xfd,0x11,0x6d,0x5e,0x23,0xdc,0x1b,0xc9,0x26,0x72,0x62,0x30, -0xd0,0x97,0xf6,0xb4,0x68,0x0b,0x6d,0xf2,0x40,0xb1,0x0a,0x9d, -0xab,0x6b,0x0f,0x25,0xeb,0x00,0x95,0x0c,0x65,0xdf,0x44,0xee, -0xef,0xff,0xde,0x14,0x0f,0xa9,0x1f,0xcd,0xf1,0x1e,0xf2,0xac, -0x92,0xaf,0x41,0x94,0x59,0x8c,0xe5,0x57,0xc6,0x92,0x48,0x93, -0x4a,0x33,0xf9,0xf7,0x91,0x45,0x7e,0x7f,0x42,0x6c,0xf7,0x6f, -0x65,0x2b,0x60,0x78,0x02,0x11,0x43,0xc4,0x37,0x04,0x5b,0xbb, -0x11,0xdd,0x1e,0xdd,0x74,0x95,0x94,0x00,0x40,0x03,0x06,0xa5, -0x06,0xc0,0x44,0x1f,0xd8,0xf7,0x72,0x92,0x60,0x1b,0x89,0xef, -0xf7,0xe8,0x51,0xb7,0x5a,0x7f,0x4a,0xde,0x9a,0xc3,0xc0,0x8f, -0xe7,0x13,0xf1,0x1d,0xb2,0x5b,0x72,0xe7,0x76,0xbb,0x55,0xa8, -0x2c,0x1c,0x3a,0x5f,0x66,0xe4,0xcb,0x44,0x36,0x37,0xc3,0x2d, -0xac,0xe3,0x9b,0x2d,0xfa,0x1b,0x09,0x7e,0x6c,0x79,0xb5,0x40, -0xb3,0x02,0xc3,0xef,0xa3,0xa4,0xc0,0x05,0x71,0xe4,0x51,0xd7, -0x81,0xab,0xf0,0x5e,0x2e,0xfe,0x1a,0x29,0xef,0x29,0x25,0x19, -0x74,0xc5,0x19,0xbf,0x16,0x15,0xae,0x4e,0x62,0xb2,0xb7,0xf5, -0xe1,0x21,0x8b,0xbb,0x70,0x63,0x31,0x00,0x5f,0x99,0x70,0x3d, -0xb9,0xd4,0xcc,0x91,0x72,0xa5,0x59,0x9c,0xa4,0xec,0xc8,0x54, -0xeb,0xd4,0xa1,0x3c,0xf3,0xe8,0x6d,0x51,0x4a,0x5e,0x53,0x31, -0x87,0x87,0xab,0x98,0xa4,0xc9,0xbe,0xe3,0x90,0x62,0x3e,0x08, -0x4a,0x76,0x4b,0x1b,0xe3,0x71,0xdd,0x61,0xcb,0x24,0xb6,0x31, -0x6c,0x59,0xd5,0x3a,0x6e,0xfb,0xee,0xc8,0x86,0x18,0x15,0x21, -0x50,0x28,0x69,0x7e,0x4e,0x18,0x45,0x67,0x17,0x62,0xeb,0xcf, -0x7d,0xdd,0xa3,0xec,0x4e,0xe3,0x6b,0x2a,0x7c,0x56,0x52,0x1f, -0xf0,0xfa,0x2d,0x6a,0xc4,0x28,0x50,0xe7,0xd0,0x30,0x45,0x8c, -0x48,0xfb,0xdd,0x2a,0x33,0x8f,0x44,0xff,0x89,0x70,0x39,0x96, -0x3f,0xd5,0xaa,0x16,0xab,0x24,0xa1,0x8f,0xc8,0x3e,0xa1,0x28, -0xaf,0xc3,0x70,0x5b,0x0e,0xb6,0x24,0x19,0x7b,0x15,0x28,0x81, -0xb3,0x1d,0xe8,0xa5,0x70,0x0a,0xfc,0x9a,0x27,0xfb,0x15,0x61, -0xc0,0xef,0x2a,0x45,0x83,0x99,0x05,0x29,0x8e,0xef,0x5f,0x96, -0xce,0xa2,0x8a,0x2b,0x96,0xa3,0x40,0xcc,0xa2,0xb8,0x39,0xf7, -0x2f,0x47,0xc0,0xcb,0x42,0xb4,0x54,0x9b,0x25,0x31,0x90,0x2c, -0x66,0x5c,0x28,0x42,0xdf,0x98,0x45,0x09,0xf7,0xf7,0x7a,0x58, -0x13,0x58,0x65,0xce,0x22,0x13,0x54,0x4e,0x4e,0xb0,0x72,0x5a, -0x55,0x33,0x87,0x53,0xd0,0x26,0xe3,0xea,0x0b,0x63,0x56,0xf1, -0x72,0x33,0x05,0x7b,0x1e,0x97,0x8f,0xcb,0xa3,0x3f,0xce,0xb2, -0x70,0x11,0x18,0x57,0xc4,0xc8,0xb2,0xc3,0x08,0x5e,0x2a,0x3f, -0x88,0x22,0xce,0x23,0x96,0x15,0x29,0x65,0x9c,0x35,0x31,0xb2, -0x85,0xf7,0x0e,0x88,0x98,0x9c,0x63,0xc0,0xde,0x9e,0x47,0xa1, -0xf1,0x94,0xd3,0x24,0x3b,0x9a,0x64,0x8d,0x49,0x7e,0x34,0xe1, -0x85,0xe1,0xf5,0x75,0xbc,0x1e,0xaa,0xfd,0xfc,0xc9,0x5c,0xbe, -0x75,0xc1,0x58,0xae,0xb1,0xf3,0x42,0x2c,0xfc,0x2f,0xfb,0x18, -0xa1,0x5d,0x66,0xcd,0x68,0x82,0xde,0x98,0x1c,0xca,0x21,0x94, -0x43,0xe8,0x20,0xd8,0x5a,0xc4,0x1e,0x1e,0x26,0xe9,0x17,0xc9, -0x3c,0x01,0x43,0x0d,0x30,0x5b,0xa8,0x29,0xfd,0xb5,0x89,0x7f, -0x07,0xe2,0xad,0xb0,0x40,0x0f,0x0c,0xaa,0x8c,0x5b,0x73,0x26, -0xab,0xe1,0xd5,0x6c,0x71,0x1b,0xe3,0xfa,0x27,0x60,0x26,0x6b, -0x03,0xe6,0x4b,0x60,0x5f,0x03,0x43,0x00,0x89,0x0e,0xe7,0x3e, -0x35,0x09,0xdd,0xf9,0x3a,0x43,0xcb,0x96,0xb8,0x08,0xd9,0xfd, -0xe1,0x74,0x79,0x3d,0xf4,0xe5,0x39,0xfb,0x94,0xde,0xce,0x41, -0xd1,0x43,0x75,0x83,0xc5,0xa5,0x9c,0x13,0xd5,0x0e,0xa1,0x42, -0x9c,0x00,0x0d,0x26,0x14,0x0c,0xdd,0x0e,0x0d,0x3d,0x8e,0xef, -0xc0,0x74,0xf3,0x06,0x7e,0x60,0xec,0x71,0xf0,0x7e,0xa1,0xdf, -0xe2,0xa8,0x1f,0xfa,0xe1,0xc0,0xbf,0x5c,0x25,0x44,0x6a,0xd4, -0x7a,0x49,0x7b,0x83,0x9f,0xcb,0x13,0xff,0xda,0x52,0x23,0xf4, -0x5f,0x1b,0x69,0x04,0xf8,0x46,0x90,0x51,0x05,0x28,0xd3,0x04, -0x85,0x51,0xf3,0x24,0xc0,0x3d,0xe2,0x5b,0x34,0x61,0x5b,0xa1, -0x4f,0x2d,0x1b,0xb5,0x5e,0x10,0x2b,0xc9,0x9e,0x92,0x7b,0xc3, -0xb2,0x29,0x99,0x17,0x48,0xb3,0x29,0xa3,0xd0,0x82,0x2e,0x71, -0xe3,0x08,0xa8,0x7d,0x89,0xfb,0x49,0xc1,0xa9,0xdc,0x82,0x64, -0x2e,0x42,0xf5,0x5c,0x71,0xcf,0x02,0xd6,0x94,0x79,0xd7,0xa1, -0xbf,0x51,0x00,0x26,0xc7,0x41,0x40,0xb6,0xf5,0xaf,0x60,0x1e, -0x2e,0xfc,0xd4,0x5f,0xfb,0x53,0xff,0x86,0xf6,0x35,0xe6,0x24, -0xbb,0xca,0xc7,0x78,0x66,0xdb,0xb1,0x72,0xe5,0x7e,0x26,0x56, -0xee,0x17,0x91,0x34,0x59,0xfd,0x34,0x5a,0xb4,0xe4,0x30,0x07, -0x9f,0x62,0x3c,0xcc,0x22,0x68,0xe1,0x66,0x0a,0xad,0x9c,0xc3, -0x57,0x0e,0x5f,0xb9,0xef,0x4e,0xe5,0xd3,0x67,0x78,0xbe,0x64, -0x1a,0x71,0x4d,0x54,0x6b,0x02,0xb2,0x23,0x17,0xa2,0x75,0x87, -0x98,0x7a,0x5e,0x53,0x35,0x3b,0x9a,0xc7,0xc7,0x53,0x3f,0x3b, -0x8a,0xa6,0x7e,0x8e,0x7f,0xb0,0x00,0x40,0xe8,0xde,0x44,0x69, -0xeb,0x2e,0xc6,0xad,0x9c,0x63,0x98,0xb1,0xf2,0x57,0x43,0x46, -0x81,0x0d,0x89,0xc5,0x47,0xd8,0x05,0x80,0x9a,0x06,0x67,0x08, -0x9b,0x37,0x20,0x03,0xad,0xbc,0x81,0xf1,0xd4,0xd7,0x20,0x92, -0xc9,0x11,0xad,0xe4,0xd1,0x33,0x6f,0x28,0x26,0xfd,0x60,0x70, -0x8c,0x4f,0xd1,0xd1,0x77,0x88,0xdf,0x64,0xe8,0xdf,0x78,0x9e, -0xb9,0x8c,0xbe,0x88,0xe4,0x6e,0x93,0xbf,0xc0,0x62,0xc0,0xa2, -0x81,0x5f,0x0f,0x4a,0x5d,0x60,0x31,0x6e,0x0e,0xc1,0x1c,0x82, -0x58,0x16,0xcb,0x04,0x65,0x2f,0x88,0xce,0xa7,0x63,0x98,0x7c, -0x6e,0x68,0x5e,0xf5,0x6b,0x54,0xdc,0xf9,0x24,0x85,0x84,0x2e, -0xcc,0xd4,0x1d,0x84,0xe8,0x7b,0x96,0xb5,0xe8,0x6a,0x32,0xf0, -0xde,0xa6,0x2c,0x9e,0xdc,0xdf,0xff,0xca,0xdb,0x07,0xae,0x54, -0x6b,0x0b,0x4f,0x7a,0xda,0xe8,0x7c,0x16,0xf5,0x94,0xaf,0xe7, -0x02,0xf9,0xd0,0x9c,0xcb,0x0c,0x2b,0x80,0x1f,0xb9,0xd7,0xc1, -0xb8,0x66,0xe4,0x62,0x6c,0x93,0xfe,0x46,0x58,0x47,0xef,0x48, -0x76,0x21,0x04,0xe5,0xf4,0x9c,0xd2,0x31,0x5f,0x6e,0xa4,0x7b, -0x5d,0x16,0x52,0x14,0x6b,0x29,0xa5,0xdc,0x2d,0x0d,0x31,0xc5, -0x8d,0x62,0x96,0x7e,0xee,0x1c,0xfb,0xd9,0x0a,0x04,0x9a,0x71, -0x46,0xaf,0x43,0x41,0x89,0xa6,0xe0,0xbf,0xb1,0x0b,0x0d,0x83, -0x2a,0x34,0xc2,0x55,0x80,0xd1,0xd8,0x0b,0xda,0x3b,0x07,0x44, -0x84,0x54,0x48,0x30,0x64,0x92,0xf2,0x79,0xe9,0x46,0x91,0x87, -0x28,0x52,0x39,0x2a,0x6e,0xd6,0xc8,0x7a,0x59,0xa7,0x61,0x11, -0x2c,0xa1,0x65,0x99,0xfa,0x06,0x93,0x12,0x09,0x06,0x55,0x4a, -0x99,0xed,0x4d,0x95,0xca,0xb1,0x17,0x55,0x12,0x5a,0x52,0xa5, -0xb4,0xee,0x5e,0xe5,0x29,0x68,0x8d,0x54,0x46,0x49,0x84,0x42, -0xd1,0xee,0x85,0x4e,0xc0,0x3e,0x48,0xb8,0x18,0xcd,0x44,0x09, -0x52,0x7f,0xef,0x55,0x84,0x04,0xd6,0xc8,0x44,0x8c,0xc4,0xc6, -0x43,0xc0,0x5e,0xb8,0x18,0x54,0x63,0xa2,0xb0,0xc4,0xc3,0xaa, -0x7d,0x2f,0x3c,0x19,0xe2,0x20,0xf8,0x1e,0xe7,0xca,0xce,0x03, -0xa8,0x6f,0xd0,0xc9,0xc8,0xbf,0x8f,0xfb,0x26,0x0d,0x97,0xb2, -0x73,0x8a,0xb1,0xd3,0x1c,0x44,0x32,0x5a,0x78,0x6d,0xd1,0x93, -0x81,0xae,0x1e,0xa5,0x3c,0x4f,0x52,0xc7,0x9d,0x58,0x76,0x1a, -0x7a,0x0e,0xa1,0xca,0xca,0x22,0x67,0x0f,0x9a,0x52,0x03,0xa6, -0x39,0xde,0xfb,0x42,0x83,0xb2,0x5e,0x57,0x81,0x29,0x07,0x6a, -0xf0,0x11,0x6e,0x51,0xc8,0x2f,0x37,0xf1,0x68,0x1e,0xec,0xcf, -0x54,0x94,0xb1,0xd4,0x42,0x07,0x2d,0x0c,0x27,0x0e,0x0d,0x83, -0x67,0xe6,0x54,0x39,0x03,0x4f,0x0e,0x99,0x99,0x75,0x87,0xe1, -0x6a,0x38,0x9f,0x2c,0x66,0xae,0x77,0x44,0x3d,0x56,0x11,0xaa, -0xca,0x66,0x07,0x11,0x85,0x93,0x29,0xd5,0x34,0xf6,0x07,0x74, -0x16,0x4e,0x1e,0x5b,0x33,0xcb,0x96,0x0b,0x29,0xef,0x69,0x04, -0xe5,0x74,0x32,0xb8,0xe4,0x30,0xfa,0x7e,0xd0,0xd5,0xf0,0x72, -0x38,0xb5,0xfc,0x97,0xe5,0xc0,0xea,0xf9,0x26,0xa0,0x98,0xc0, -0xd9,0x80,0x9c,0xdb,0xdb,0xea,0xa5,0x41,0x99,0x21,0x19,0x90, -0x51,0x85,0x2b,0x43,0x2a,0x12,0xcc,0x80,0x9d,0x43,0xbf,0x7f, -0x27,0x47,0x41,0xff,0x5a,0x8e,0x81,0x25,0x4f,0x4d,0x6b,0xdc, -0xe1,0xa3,0x84,0x60,0x5f,0x88,0x21,0x18,0xe0,0x76,0x98,0x0f, -0x0e,0x6f,0x71,0x19,0xd7,0xc2,0x8a,0x0a,0x80,0x44,0xca,0x4f, -0x81,0x5c,0x06,0x07,0x5e,0x55,0x2e,0xe6,0x05,0xe5,0x12,0x33, -0x7f,0x99,0x8b,0x83,0x90,0xab,0xd1,0x50,0xbc,0x65,0xd2,0x28, -0x46,0x30,0x91,0x63,0x4a,0x55,0x33,0x86,0x46,0x16,0x2b,0x1c, -0xf2,0xa8,0x14,0x74,0x0d,0x17,0x32,0xec,0x64,0x8e,0x7f,0x07, -0x5d,0x40,0x42,0xe4,0x0c,0x91,0x1b,0x10,0xb9,0xe3,0x5f,0x1b, -0x10,0x4b,0x81,0x44,0x8c,0xa1,0x3a,0x5e,0x64,0x15,0x63,0x27, -0x1d,0x0a,0x90,0x16,0x10,0x8a,0x58,0x85,0xce,0x32,0x35,0xba, -0x57,0xe4,0xb4,0x69,0x3e,0x45,0x0d,0x13,0xd4,0xf0,0x0b,0x53, -0x8d,0x91,0x14,0x1c,0x35,0x2a,0xb1,0x98,0x48,0x90,0x3e,0x65, -0xc0,0xed,0xa0,0x4f,0xaa,0xff,0x12,0x7d,0xa6,0xf5,0xc7,0xf4, -0x49,0xd0,0xc7,0xd0,0x67,0x61,0x31,0x91,0x68,0x73,0xab,0x8e, -0x3a,0xc3,0x18,0xb3,0x3c,0x95,0x39,0x13,0x51,0xc5,0xdf,0x06, -0x3d,0x5a,0x22,0x6a,0x1c,0x5b,0x8c,0xdc,0xfc,0x69,0x9b,0xf5, -0x6a,0x8a,0x63,0x4d,0x79,0xaa,0xd6,0x9c,0xb5,0x8a,0x77,0x5b, -0xa1,0xa7,0xb5,0xea,0x62,0xf9,0x00,0x74,0xa0,0x80,0x71,0x0a, -0x55,0xb5,0x25,0x09,0xd1,0xf7,0xf7,0x2e,0xa5,0x82,0x3e,0x1f, -0xc5,0xd7,0x30,0x3c,0x2d,0x56,0x04,0xee,0x7a,0xe2,0x26,0x61, -0xeb,0x1a,0x8a,0x16,0x8a,0x31,0x26,0x93,0x52,0xe7,0x09,0xb4, -0xe3,0x17,0x2c,0xd8,0xcf,0x65,0xba,0x06,0x57,0x90,0x62,0x2a, -0xa8,0x53,0x60,0x86,0xb2,0x03,0x0d,0xa4,0x42,0x87,0xa9,0x1a, -0x0e,0x93,0x94,0x50,0xcc,0x16,0x9b,0x34,0x06,0x84,0xab,0x1a, -0x24,0xdf,0xc8,0x74,0xcf,0x00,0xdf,0xd4,0x51,0xfe,0x8d,0x48, -0xf6,0xb8,0xa1,0x91,0x3e,0xcf,0xbf,0x5d,0x24,0x93,0x27,0x81, -0xd7,0xc1,0x10,0xad,0x1e,0xac,0x62,0x5c,0x0c,0xe5,0x81,0x8f, -0x67,0x9b,0xbe,0x03,0xe2,0xa4,0x5d,0x1a,0xeb,0xa6,0x8c,0x51, -0x3b,0x28,0x96,0x68,0x4e,0x14,0xa3,0xb0,0xab,0xd7,0x26,0xae, -0x93,0x78,0x35,0x5c,0x8d,0xaf,0xf3,0x9a,0xf5,0x89,0xf1,0x66, -0x25,0xd6,0x27,0xfc,0x49,0xbc,0x04,0x1d,0xcd,0x33,0x03,0x3e, -0xb1,0x82,0x7e,0x77,0x69,0x44,0x3f,0x80,0x9a,0x2b,0xa3,0x10, -0xfa,0x3a,0x13,0xfb,0x9e,0xe1,0x54,0x9e,0x82,0x91,0x81,0x89, -0x47,0x12,0x5a,0xcb,0xf0,0x19,0x1f,0xe8,0xa4,0x39,0xe3,0x54, -0x1e,0x38,0x7a,0xa4,0x46,0xd7,0x38,0x9a,0x13,0x8d,0x49,0x57, -0xb6,0x64,0xd9,0xd8,0xe3,0x6e,0x69,0x4b,0x80,0x70,0x37,0xd4, -0x74,0x63,0x12,0xc9,0x3a,0x30,0x4a,0xec,0x4a,0xef,0x45,0x25, -0x70,0x55,0x60,0xb8,0xc2,0x59,0x3c,0x91,0x3a,0x96,0x47,0x17, -0xfd,0xdb,0x46,0x34,0xe1,0xa5,0xca,0x2e,0x2e,0x26,0x1d,0x1e, -0x8e,0x79,0x61,0x14,0xff,0x78,0xbc,0x8e,0x8c,0x54,0x22,0x01, -0x14,0x88,0x6e,0x85,0x53,0x63,0x45,0x52,0x43,0x5c,0xa2,0x51, -0xcb,0x9b,0xfb,0xfb,0xc0,0x33,0x57,0x55,0xcc,0xc5,0x59,0xca, -0x6b,0x70,0xdf,0xe0,0x3b,0x56,0xda,0x62,0x01,0xd6,0x9f,0x75, -0x10,0x87,0x35,0x1f,0xb1,0x6d,0x24,0x27,0x35,0x23,0x89,0x8d, -0x25,0x7e,0x41,0xd5,0x65,0xb1,0x32,0x2f,0xb1,0x4c,0x68,0x1f, -0x59,0xc1,0xdb,0xc7,0xd5,0xab,0x92,0x65,0xfe,0xad,0xae,0xaa, -0xc2,0x80,0x8b,0x37,0x44,0x9f,0x58,0xad,0x90,0x18,0x64,0x2b, -0x4e,0xfc,0x40,0xb6,0x1e,0xcf,0xbc,0x78,0x61,0x9e,0xef,0x90, -0x17,0x12,0xaf,0x90,0xbd,0x81,0x78,0x5f,0x71,0xab,0x02,0xe0, -0xb5,0xe2,0x1d,0xd1,0x54,0x01,0xf1,0x13,0x49,0x81,0x20,0x42, -0xc5,0x92,0x38,0xec,0xbf,0x64,0x09,0x33,0x34,0x95,0x95,0xd7, -0x27,0xb7,0x3a,0x42,0xcb,0xf0,0x9e,0x13,0x0a,0x01,0x6d,0xe1, -0x94,0xb1,0x26,0x5e,0xae,0xd3,0x5e,0x48,0x19,0xd4,0xc2,0x48, -0x51,0x26,0x3a,0x21,0x16,0xe6,0x2b,0x0e,0x0b,0xed,0x7f,0x2d, -0x85,0x86,0x9c,0xfb,0x02,0x8f,0x3d,0xf0,0x74,0x6e,0x73,0x39, -0x14,0xfa,0xdc,0x3a,0xb1,0x27,0x64,0x55,0xc6,0x3a,0x89,0x52, -0xe6,0xd3,0x2d,0xaa,0xbb,0xe4,0x5e,0xe9,0x16,0xf2,0x27,0xa6, -0xd5,0xce,0x48,0xeb,0x1a,0xbc,0x5c,0x9a,0x43,0x19,0x5e,0xfd, -0xcc,0x9f,0x79,0x04,0xb1,0xbb,0xfb,0x09,0x0b,0x21,0x1d,0x0e, -0xa7,0xd5,0x0b,0x5a,0x41,0xc9,0x22,0x2d,0xc0,0xbd,0x49,0x76, -0xac,0x43,0x1d,0x75,0x62,0x5f,0xd1,0x3f,0x8e,0xcc,0x0e,0x04, -0xb5,0x88,0xc6,0x0c,0x7b,0xc4,0x55,0xc2,0xfd,0xa7,0x89,0xb9, -0x7c,0x89,0xe4,0xba,0x7a,0x55,0xb7,0xa6,0x97,0x4f,0x1e,0xdf, -0xcb,0xb5,0x2e,0xa4,0xc9,0xc7,0x6c,0x88,0xde,0xa0,0x5c,0x9c, -0x41,0xa2,0xde,0x10,0x0e,0x1b,0x13,0x4d,0x96,0x6a,0x2d,0xde, -0x90,0xd4,0x7d,0xd3,0x10,0x60,0xcb,0xeb,0x5f,0x9f,0x98,0x32, -0x1a,0x92,0x9e,0xbd,0x0e,0x7c,0x69,0xde,0xcb,0x05,0x2e,0x5d, -0x67,0x04,0xf0,0xcc,0x3e,0xad,0x07,0xa6,0x8a,0xc1,0xca,0x35, -0x97,0x55,0xf5,0x3e,0xb6,0xa0,0xf4,0xf1,0xeb,0x27,0x2a,0xab, -0x5c,0x43,0xa9,0xd0,0x04,0x3f,0xf0,0x70,0xac,0x40,0x8d,0xce, -0x62,0x49,0x77,0x52,0xbd,0xd4,0x0f,0xf1,0xc6,0x79,0x1e,0x12, -0x84,0xaa,0x33,0xc7,0xc6,0xc1,0x49,0x53,0xc3,0x42,0x6e,0x62, -0x2e,0x5d,0x33,0xd3,0xa8,0x7a,0x86,0x4d,0x6f,0x7b,0xf6,0xf4, -0x67,0xcd,0x53,0x17,0x1a,0x80,0x36,0x86,0xab,0xd0,0xc9,0x7d, -0xd3,0x9e,0xfc,0xa8,0x41,0x25,0x93,0xbd,0xe6,0x10,0x2f,0x46, -0x6b,0x81,0x75,0xc9,0xda,0x24,0x15,0x57,0xea,0xe8,0x3d,0x7f, -0xc3,0x55,0xdc,0xeb,0x0a,0x27,0x15,0x1a,0x5d,0x13,0xbd,0xbf, -0x4e,0xda,0x32,0x32,0x14,0x31,0x54,0x1c,0xb5,0xf8,0x67,0xac, -0x88,0x7b,0x8a,0x4f,0x30,0xb1,0x54,0xcd,0xca,0xe5,0xc1,0xcc, -0xb6,0x29,0xbe,0x60,0xf2,0xd9,0xa9,0x84,0xa4,0x32,0x94,0x37, -0x3a,0xfe,0xbe,0x1f,0xc8,0x4b,0xac,0xa0,0x29,0x53,0xeb,0xad, -0x8e,0xca,0x5d,0x79,0xd1,0x05,0x26,0x5d,0x04,0x47,0xa3,0xfb, -0x03,0xa2,0xe9,0x48,0x9c,0xac,0x0c,0x26,0x91,0x22,0xc4,0xda, -0xc6,0x36,0x76,0xb1,0xa1,0xef,0xe3,0x86,0x06,0x1a,0x9d,0x63, -0x16,0x78,0xd6,0xae,0xc2,0x93,0xb3,0x30,0xa2,0xd9,0x5c,0x30, -0x70,0x46,0x7a,0x87,0x3b,0xfa,0xfd,0xdf,0x55,0x4f,0x48,0xe2, -0x8f,0xd1,0xfe,0x90,0x4d,0xe9,0x7d,0x44,0xf1,0xe8,0x41,0x8e, -0x10,0xf0,0xf0,0x46,0xf9,0x35,0x99,0xfb,0x61,0xd1,0xf0,0x12, -0x97,0x8a,0x61,0x8c,0xaa,0xae,0x7b,0xe1,0x53,0xd0,0x02,0x9b, -0x0c,0x13,0xae,0xa2,0x15,0x6d,0xb3,0x38,0xfa,0xb0,0x35,0x8c, -0x64,0xbe,0xb2,0xab,0xaa,0x57,0x53,0x0a,0x49,0x14,0x4b,0xbb, -0x70,0x14,0x88,0x57,0xc9,0x43,0xbd,0x5b,0x48,0x2d,0x77,0x6f, -0x42,0x23,0x3b,0xb8,0x38,0xbf,0xc5,0x48,0xea,0x8e,0x46,0x4f, -0x0a,0x27,0xc0,0x35,0xe2,0x7e,0x66,0xe2,0xbc,0xa5,0x33,0x0e, -0xc5,0x18,0x3e,0x4f,0x4d,0xc7,0xb0,0xd3,0x88,0x4f,0x70,0x1b, -0x10,0xe2,0xb4,0x13,0x57,0xa8,0x2b,0x2b,0x73,0xd2,0x5a,0xc6, -0xab,0x19,0xfa,0x90,0x71,0x8c,0xc8,0xae,0xe8,0x35,0x00,0x04, -0x02,0x01,0xd0,0x95,0x0b,0x41,0x0b,0x7f,0x71,0x79,0x99,0xc6, -0x00,0xcb,0xbf,0xb5,0x65,0xfa,0x73,0x59,0x7b,0xb5,0x4e,0xc4, -0x61,0xbc,0x48,0xcb,0xd8,0x26,0x32,0x17,0xb9,0x30,0x4c,0xbe, -0x4b,0x1a,0xa7,0xca,0x83,0x9e,0x29,0x0b,0x51,0x28,0xea,0x09, -0x5f,0x5d,0xcb,0xc5,0x37,0x5c,0x79,0xbe,0x79,0xdd,0x59,0x35, -0x16,0xf6,0x6f,0x59,0x34,0xd4,0x11,0xf0,0x66,0x28,0x72,0xfb, -0xfa,0x98,0x3e,0x63,0x53,0xc7,0xdd,0xf0,0x34,0x9a,0xa9,0xed, -0x4d,0x57,0x51,0x71,0x14,0x5f,0x8a,0x1f,0x95,0xf5,0x1d,0x46, -0x7e,0x2e,0x9c,0x28,0x99,0xdc,0x12,0x08,0xc5,0xfe,0x1c,0xaf, -0x16,0xe8,0x6d,0x5a,0x4a,0xdd,0x88,0xb3,0x0e,0x66,0xa4,0x38, -0xa5,0x60,0x46,0xa9,0xfb,0x9f,0xf8,0x26,0x2c,0x41,0xdf,0xfe, -0xca,0x02,0x37,0x2e,0x28,0xab,0xd0,0x17,0xe9,0x96,0x83,0x2d, -0xae,0xd6,0x5e,0x88,0x18,0xb4,0x66,0x61,0xbc,0x8a,0x31,0x29, -0x39,0x92,0x66,0xec,0x48,0x5a,0xcb,0x36,0x41,0x16,0x61,0x57, -0x54,0x31,0x2b,0xf7,0x23,0x8b,0x61,0xf7,0xa4,0x8b,0x45,0xaa, -0x96,0x30,0xdd,0x58,0x92,0x2c,0x8a,0x91,0x74,0x19,0xde,0x29, -0xbf,0xed,0x5c,0x4c,0xff,0x4d,0xe6,0xcf,0x64,0xd6,0x7c,0xbf, -0xac,0xb9,0xce,0x9a,0x2b,0x6e,0x6c,0xd6,0xfb,0x65,0x46,0x40, -0x95,0x1d,0x02,0x02,0x41,0x51,0x73,0x1a,0x2d,0x64,0xba,0x46, -0xe3,0x6d,0x3f,0xec,0xf9,0x5c,0x71,0x9e,0x99,0xd6,0x85,0xad, -0xc3,0x63,0x60,0x95,0x6a,0x85,0x59,0x40,0xfe,0xcd,0x30,0xfb, -0x8a,0x15,0x43,0x8a,0x3e,0x85,0x75,0x60,0x3f,0xc4,0x93,0xcd, -0x38,0x7e,0xb3,0x99,0x95,0x4c,0x8f,0xf9,0xae,0xcb,0x2d,0xa0, -0x64,0xbc,0xda,0x02,0x7f,0x46,0x74,0xbd,0xcf,0x62,0x89,0xd7, -0x2f,0xf1,0x69,0xc0,0x00,0x83,0xb4,0xa0,0xc7,0x11,0xa9,0xbe, -0x98,0x4d,0xaf,0xc1,0xbd,0x8f,0x12,0xe9,0x18,0xc7,0x50,0xe7, -0x90,0xa9,0xe7,0xc2,0x9f,0x46,0x84,0x14,0xa3,0xe6,0x40,0x34, -0xbc,0xc0,0xf0,0x1e,0x4f,0xfa,0x30,0x9c,0x4e,0x16,0xd8,0x15, -0x84,0xec,0xa0,0x32,0x5e,0xbf,0xce,0x22,0x1e,0x77,0x41,0x7c, -0x78,0x68,0x8b,0xe3,0x8b,0x3c,0xd6,0x07,0x6e,0xcb,0xb6,0x97, -0xc6,0xb5,0xf5,0x1d,0xe1,0xb2,0xed,0x54,0xf6,0x38,0xd1,0xa7, -0xd8,0xff,0xbd,0x42,0xee,0x55,0x73,0xa7,0xc9,0xf4,0x7a,0xb1, -0x89,0xd7,0xeb,0x12,0x21,0x5a,0xf7,0x17,0x9a,0x9b,0xcf,0xe0, -0x29,0x7d,0xce,0x2d,0xdb,0x1f,0x90,0x1c,0xd0,0xf3,0x24,0x55, -0x1b,0x19,0xf2,0xd2,0xcb,0x85,0x60,0x3e,0x5e,0xea,0x07,0x2a, -0x59,0x58,0x5f,0xac,0x7e,0xbb,0x8b,0x73,0xc0,0x01,0xf3,0x22, -0xc4,0xb4,0x60,0x91,0x11,0xfb,0x15,0xde,0xb6,0x88,0x31,0x0f, -0x70,0xcf,0x04,0x41,0x9b,0xa2,0x25,0xbc,0xe3,0xb6,0xe4,0x7d, -0x1e,0x6c,0xfd,0xbb,0xe4,0xea,0x6a,0x5a,0x5b,0x2b,0x18,0x1d, -0xe8,0x56,0x9c,0xf4,0x04,0xe7,0xa9,0xa0,0x92,0x83,0x42,0x35, -0x33,0x59,0x4d,0xa8,0xb0,0xf2,0x28,0xd3,0x95,0xcb,0xf1,0x96, -0x90,0x08,0x86,0xa6,0x80,0xd6,0xae,0xc2,0x72,0x4d,0xd3,0xd0, -0x90,0xb3,0x34,0x2c,0xd6,0x55,0x81,0xd1,0x5b,0xb7,0x30,0x03, -0x16,0x77,0x3a,0xc2,0x6f,0x93,0x4e,0x57,0xe9,0x03,0x90,0x08, -0x8a,0xae,0xb4,0xe9,0x49,0xe4,0x5a,0x48,0x9a,0x2a,0xd4,0xe4, -0x43,0x58,0xc7,0x6e,0x1b,0xe6,0xa8,0x5e,0x37,0x39,0xbf,0x81, -0xac,0x37,0x5e,0x7a,0xd2,0x10,0x59,0x6e,0xac,0x2c,0x37,0x46, -0x96,0x49,0xd6,0x4d,0xdb,0x20,0xce,0x27,0x47,0x16,0xee,0x2d, -0x33,0x78,0xd1,0xa4,0xbb,0x7a,0xda,0xc7,0x69,0x08,0x98,0x3b, -0xd0,0x80,0xc1,0xf9,0x02,0x37,0x1f,0x82,0xa8,0xae,0x51,0x9a, -0xc0,0x14,0xb3,0x21,0xe2,0x6c,0x39,0x34,0xdf,0xb8,0x7e,0xa4, -0x78,0xdd,0x44,0xe1,0xf1,0xfc,0x0f,0x89,0x15,0x4c,0xba,0x17, -0xd6,0xda,0x3d,0x42,0x59,0x20,0xc7,0xd1,0xa2,0xbc,0x46,0x5f, -0x82,0x8a,0x6e,0x6a,0xa4,0xd0,0xaa,0xee,0x6f,0x30,0x68,0x54, -0x8f,0x36,0x38,0x9c,0xf0,0xd9,0x0f,0x35,0x2b,0x4e,0xd7,0x78, -0x11,0xf6,0xac,0xd2,0x42,0x55,0xa9,0xc2,0x4a,0xd5,0x6b,0xa9, -0x30,0x9d,0xf5,0x33,0x1f,0xfe,0x52,0xbf,0x2b,0xce,0x48,0x29, -0xbc,0xf2,0xf9,0x2a,0x3b,0x52,0x22,0x11,0xfd,0x5d,0x19,0xd3, -0x7d,0xce,0x83,0x16,0xe6,0xfa,0x7a,0x15,0xa7,0xd7,0x0b,0x5c, -0x0e,0x06,0x84,0x73,0x0b,0x8a,0xb2,0x19,0xb0,0xc2,0x79,0x5d, -0xd8,0xbf,0xaa,0xc1,0x34,0x0a,0x75,0x3d,0x18,0xb4,0xda,0xe5, -0x2a,0xfe,0x75,0x13,0xcf,0xc7,0x79,0x2f,0xec,0x40,0x0b,0xd2, -0x52,0xc6,0xac,0xeb,0x41,0x21,0x58,0x10,0x5f,0x4a,0x30,0xc0, -0x4a,0xe0,0x02,0x8f,0x46,0xd1,0x4f,0x1a,0x20,0xa5,0x2e,0xc6, -0xdb,0xd1,0x60,0xeb,0x61,0x64,0xce,0xab,0x28,0xb3,0xf3,0xc0, -0x2b,0x3a,0xa8,0x64,0xc6,0xe4,0x2d,0x3b,0xe7,0x4a,0x83,0x20, -0x1d,0x1e,0x66,0xaf,0x44,0x00,0xef,0xb4,0x75,0x15,0x05,0xb8, -0xa3,0x91,0xd0,0x65,0x75,0xba,0x1c,0xe0,0x6b,0xe8,0xcf,0xe8, -0x66,0x3e,0x2a,0xad,0x11,0xdd,0xd0,0x87,0xbe,0xb8,0xd3,0x5c, -0x6f,0x41,0x34,0x64,0x27,0xaa,0xca,0xe2,0x5d,0x13,0xdc,0x00, -0x72,0x42,0xc8,0xbc,0xb7,0x56,0x28,0x45,0xbb,0xfe,0x40,0xec, -0x65,0xae,0x57,0xa5,0x7f,0x96,0xcc,0xdf,0x40,0x31,0x57,0x71, -0xaa,0x97,0x31,0x45,0xd2,0xa3,0x27,0x89,0x2b,0x5a,0x23,0x14, -0xb9,0x85,0xed,0xb7,0xda,0xea,0x28,0x1e,0x6d,0xf6,0x43,0xa8, -0x6b,0x64,0x1c,0x5a,0x37,0x90,0x73,0xba,0x89,0x9c,0x44,0x75, -0x2f,0xdc,0x82,0x1b,0xa5,0xad,0xdb,0x4c,0x2f,0x20,0x10,0x7a, -0xf3,0xc0,0x55,0x05,0xdf,0xbe,0x40,0xf7,0x24,0x86,0xf4,0x33, -0x6f,0xdb,0xa9,0xa5,0x95,0xcb,0x33,0x69,0xd5,0x4d,0xb9,0xa7, -0xd7,0x89,0x6a,0xf9,0x03,0x8b,0xc5,0x2a,0xc1,0x40,0x6e,0x2e, -0x37,0xd9,0xf7,0x54,0x18,0xfd,0x9e,0x6f,0xa4,0xa0,0xfe,0x2e, -0xa3,0x2e,0x93,0x55,0xba,0xfe,0xdb,0x70,0x7a,0xc3,0x6b,0xab, -0x4b,0xb0,0x03,0x92,0xc5,0x26,0x7d,0x93,0x8c,0xa6,0xea,0x60, -0x52,0xcd,0xaa,0xa3,0x38,0xbe,0x42,0x91,0x17,0x88,0xfa,0xe1, -0x35,0x48,0x63,0xcf,0x46,0x96,0x44,0xcb,0xed,0xbc,0xf0,0xea, -0x13,0x31,0x14,0xa1,0xd7,0x49,0x03,0xbc,0xe5,0x6c,0x1c,0xa7, -0x74,0x77,0xbe,0x4a,0xb7,0xd6,0x30,0xc7,0x76,0x0e,0xe8,0xa3, -0xba,0x56,0x8c,0xd8,0x2a,0xcc,0xd3,0x08,0x87,0xcb,0x25,0x98, -0x80,0xb4,0x32,0x5b,0x86,0x53,0x60,0x9e,0x1d,0xcf,0x25,0x75, -0xb5,0x78,0x60,0xdd,0xdf,0x5c,0x27,0x97,0x6b,0x5e,0xa9,0x25, -0x53,0x78,0x96,0x4c,0xf8,0xfe,0x0a,0xbc,0x58,0x42,0xd3,0xcd, -0x8c,0xc2,0x63,0x5c,0xd3,0x64,0xd6,0x18,0x97,0xa2,0xbc,0x6e, -0xa1,0x05,0x7a,0xae,0x5c,0x43,0xa4,0xf4,0xa8,0x90,0x6c,0xe3, -0x4b,0xe3,0xe5,0x70,0x35,0xd4,0x2b,0xe5,0xc5,0xd6,0x14,0x4d, -0xd6,0x9a,0x2d,0x26,0x91,0x85,0xb6,0x29,0xc9,0xf5,0x3a,0x76, -0x71,0x32,0x9e,0xb7,0xb2,0x0a,0xf8,0xd0,0x8b,0xf2,0xcf,0x23, -0xce,0x38,0x81,0x00,0x9a,0x73,0x31,0x9f,0x68,0xa9,0x84,0x5e, -0x22,0x56,0xf3,0xb5,0xb0,0x49,0xb4,0x59,0x77,0xf7,0x36,0xc0, -0x5e,0x4b,0xfa,0x6a,0x39,0x3f,0x6b,0x98,0x45,0x00,0x9f,0xa4, -0x8c,0x19,0x24,0x59,0x4b,0xf8,0xe6,0x13,0x9d,0x5a,0x9a,0xaa, -0x2a,0xa8,0xc5,0x09,0x1d,0x79,0x4a,0x1d,0x4d,0xf6,0x8c,0x14, -0x05,0xc9,0xbf,0x4d,0x96,0xbc,0x01,0x79,0xbb,0x90,0x1f,0x65, -0xfe,0x42,0xe2,0x8c,0xc9,0xe5,0x2d,0xcb,0x96,0xd9,0x69,0x52, -0xc0,0x00,0x58,0x74,0x4d,0xfc,0x14,0x50,0x01,0x3a,0x33,0x06, -0x70,0x02,0x5e,0x0b,0x06,0x62,0x16,0x46,0x4c,0x17,0xcb,0xb5, -0x45,0xfd,0x07,0x74,0x54,0x71,0x21,0xde,0x23,0x32,0xed,0xc4, -0xaf,0xe3,0x4b,0x4c,0x5b,0x62,0xda,0xec,0xf0,0x10,0xbe,0xba, -0x1e,0x92,0x59,0x05,0xb5,0x40,0x0c,0x8b,0x22,0x06,0x81,0x7e, -0xb1,0xa4,0x44,0x41,0x89,0xea,0xb1,0xc4,0x0b,0xe2,0x91,0x41, -0xb9,0x94,0x30,0x10,0x64,0x5d,0x65,0x21,0xd9,0xc0,0x07,0x53, -0xf4,0x20,0x93,0x4f,0xe4,0x11,0x0e,0x72,0x3c,0xb3,0x8b,0xff, -0x86,0xbc,0xc5,0xad,0xa8,0x4f,0x45,0xe1,0x94,0x99,0x08,0xd0, -0x9a,0x41,0xd3,0x83,0xdb,0x0f,0x38,0xb7,0xa3,0xf6,0x4b,0x17, -0xf2,0x13,0xa3,0x61,0x4e,0x57,0x60,0x33,0x42,0x16,0x5b,0x07, -0xa1,0x16,0x76,0x63,0x20,0x94,0xd9,0x62,0x5d,0x62,0xea,0x41, -0x1d,0xc3,0x70,0x13,0x56,0x81,0xa3,0xe5,0x31,0x9c,0x44,0x54, -0x61,0x13,0x07,0xd0,0x85,0x6c,0x21,0xfe,0x26,0xcb,0x12,0x36, -0xd5,0x34,0x84,0x6c,0x56,0x44,0xb6,0xf4,0x2d,0x1a,0x11,0xd9, -0x12,0x90,0xcd,0x7c,0xab,0x85,0xd4,0xb9,0x50,0x19,0xbb,0x7d, -0x78,0x8f,0x88,0xb6,0x07,0x23,0xb9,0xef,0x53,0x50,0xaf,0x74, -0xc1,0x08,0xbd,0xe5,0xc1,0xbb,0x89,0xfa,0x74,0x73,0xe5,0xa8, -0xa5,0xfb,0x70,0xf4,0x41,0x92,0xd0,0x91,0xa0,0x20,0x15,0x30, -0xdb,0x00,0xe2,0xf1,0x18,0xe7,0x35,0x0e,0xe4,0xf0,0x41,0x6d, -0x05,0xbf,0x6c,0x1d,0x74,0x8a,0x5a,0xb8,0x5a,0xb3,0x31,0x70, -0x23,0xec,0x04,0xb8,0x6a,0xaf,0x07,0x1d,0xda,0x05,0xf5,0x0d, -0x95,0x41,0x34,0x37,0xe9,0x92,0x12,0x5b,0xdb,0xd3,0x7d,0x64, -0xc0,0xf1,0x42,0x1f,0x78,0x13,0x23,0x87,0xc4,0xad,0x28,0xa5, -0xe6,0x99,0x2d,0xd2,0xb5,0xbc,0xa6,0x68,0xdf,0x7c,0x24,0x24, -0x9c,0x71,0x12,0xc7,0xc5,0x3e,0x57,0x9f,0xef,0x73,0x00,0x8e, -0x31,0x57,0x16,0x44,0x48,0x28,0x7a,0xd5,0xeb,0xde,0x84,0x31, -0x4c,0x08,0x9e,0xf0,0xca,0xc2,0x88,0xbe,0x5b,0x99,0xd9,0xe3, -0x28,0xca,0x47,0x48,0x84,0xc9,0xc3,0x08,0xcb,0xe7,0x8d,0xd8, -0xfb,0xfb,0xb0,0x5b,0x32,0xb6,0xaa,0x9a,0x1b,0x9b,0xee,0x4d, -0xf2,0x9b,0xb1,0xd7,0xc3,0x7b,0xa2,0x3c,0x2a,0x1c,0x29,0x17, -0xc1,0xf2,0x56,0xaf,0xd8,0x55,0x84,0x4a,0x4f,0xe3,0x75,0xfc, -0x44,0x0b,0x86,0xb1,0x1b,0x64,0xe2,0x8a,0xc4,0xbe,0x31,0x1d, -0x08,0x94,0x77,0x5b,0xd6,0xa2,0x3f,0xce,0xc3,0x1d,0x25,0x3c, -0x62,0xeb,0x92,0x97,0xa2,0x70,0x03,0xcc,0x6b,0x09,0x07,0x0f, -0xfc,0xf6,0x35,0x1f,0x4b,0x2d,0x26,0x13,0xcc,0xc3,0x24,0x92, -0x4f,0xd1,0x81,0x62,0x2c,0xdf,0x86,0xa2,0xd1,0xec,0xb7,0xeb, -0xa2,0xe1,0x33,0x9f,0xef,0x39,0xd1,0x51,0x5b,0xbf,0xe2,0x86, -0x95,0x87,0xbc,0xe3,0x89,0x2a,0xbe,0x2b,0x45,0x6c,0xb0,0x7a, -0x02,0xb1,0x6a,0x5b,0xba,0x40,0x85,0xb7,0x5a,0xb9,0x77,0xc9, -0x6c,0x8f,0x2f,0xe4,0xa0,0xb6,0x10,0x8c,0xa6,0x3b,0x5b,0x76, -0xed,0xe7,0x62,0x9e,0xba,0xad,0xdc,0xba,0x5d,0x9f,0x25,0x6d, -0xfa,0xec,0xb7,0x1d,0x6e,0xab,0xba,0x2a,0x17,0x57,0x7f,0x25, -0x98,0x85,0x4f,0x38,0x6e,0xd2,0x9e,0x3a,0x94,0xd2,0xa9,0x58, -0x84,0x16,0x30,0xfc,0x53,0x71,0x75,0x05,0x27,0x6c,0xd5,0xc5, -0x50,0x59,0x44,0x3f,0x79,0x14,0xf8,0x7b,0x6a,0x59,0x3a,0x3d, -0xb5,0x8a,0x1a,0x2b,0x57,0xf8,0x40,0x79,0xd6,0xfa,0x5f,0x55, -0x5e,0x63,0x17,0x0d,0x38,0x23,0xf4,0x26,0xee,0x0c,0xf1,0x09, -0x46,0xb1,0x65,0xba,0x92,0xc7,0x6d,0x8f,0x5c,0x51,0x8b,0xb0, -0xa3,0x1c,0x12,0xda,0x47,0x44,0xe7,0xea,0xf8,0xce,0x57,0x9f, -0x60,0xb3,0x1d,0xb7,0xf7,0x1d,0x1c,0x98,0xec,0x46,0x34,0x59, -0x7d,0x34,0xb9,0x8f,0x2a,0xa9,0x49,0x25,0x6d,0x4b,0x2a,0x0d, -0x51,0xaa,0x93,0xff,0x9c,0xf3,0x0e,0x74,0xe1,0x35,0xfc,0x53, -0xb5,0x16,0x27,0xf1,0x76,0xd4,0xdb,0xd0,0x26,0x9c,0xcb,0xdf, -0x4b,0xa9,0x14,0x6a,0x46,0xee,0x6a,0xea,0x78,0x6c,0xa5,0xcb, -0xc4,0xf8,0xe6,0x11,0xd7,0x27,0x11,0xe4,0x85,0xbf,0xa4,0x0d, -0x01,0xee,0xbd,0x84,0x81,0x49,0xdc,0x0f,0x87,0x80,0x15,0x37, -0x31,0xdd,0xdf,0x57,0x3e,0xe1,0x7a,0xd1,0x69,0xc8,0x62,0x84, -0x40,0x73,0x41,0xa5,0x33,0xb6,0x3b,0x9f,0x4f,0x16,0xc0,0x0a, -0x95,0x3a,0x50,0xbb,0xcb,0xbb,0x63,0x7c,0x53,0xa3,0x0d,0xc6, -0xd3,0x0d,0xbe,0x39,0x57,0xa9,0x10,0x44,0x9a,0xd6,0x09,0xd2, -0x86,0xc0,0x6b,0xda,0xfc,0x47,0x29,0x88,0x2c,0x0a,0x1e,0x63, -0x0f,0xed,0xf4,0x8c,0xd2,0xf3,0xa1,0xc2,0x64,0xa8,0x27,0x3d, -0xa6,0x74,0x49,0xa2,0x0a,0x6f,0xd5,0x94,0xc7,0x93,0x43,0x60, -0x09,0xe6,0x9d,0x86,0xf1,0x3a,0x12,0x93,0x59,0xe3,0x1e,0x4c, -0xad,0xea,0xa6,0x80,0x98,0xee,0x75,0x02,0x89,0x3c,0xb0,0x12, -0x84,0x41,0x59,0x69,0x2b,0x89,0xc2,0xc9,0x66,0x65,0xa3,0xab, -0x68,0x15,0x09,0x08,0xb6,0x91,0x19,0xe4,0xcf,0xb1,0x64,0xfe, -0x0c,0xc3,0x45,0x19,0x1b,0xac,0x96,0x4b,0x86,0x06,0xab,0x6b, -0x79,0x5e,0xd7,0x93,0xa6,0xc6,0x1f,0x30,0x5d,0xdc,0xb0,0x29, -0x90,0xf6,0x38,0xe6,0x98,0x43,0x9d,0xd0,0xd3,0xe8,0xff,0xaf, -0xb0,0x5b,0x44,0xc3,0xfd,0x31,0xd3,0x45,0x20,0xb1,0xad,0x17, -0x85,0xf9,0x0f,0x1b,0x30,0x0a,0x7d,0x95,0x0d,0x23,0x8b,0xf9, -0x73,0xcc,0x98,0x72,0x51,0x7b,0x59,0x32,0x22,0x5b,0x8d,0xfa, -0xc2,0x06,0x99,0x0d,0x97,0xd5,0x5e,0x2c,0x74,0x35,0x8e,0x5a, -0xec,0xbb,0xd1,0x2b,0x14,0xbc,0x54,0x86,0xaf,0xf9,0xc8,0x65, -0xfb,0x82,0x26,0x51,0x0f,0xfc,0xe0,0xb3,0x45,0x6e,0x69,0x75, -0x4e,0x48,0xca,0x91,0x1b,0x9c,0xdf,0xf4,0x82,0xce,0x8d,0x70, -0xf2,0xe6,0xe7,0x91,0xf8,0x20,0x0a,0x7e,0xa2,0xb3,0xf0,0x79, -0x84,0x5f,0x00,0x44,0x4f,0x18,0x69,0xda,0x7e,0xdd,0x0c,0x57, -0xc9,0x65,0xfe,0xb0,0xb6,0xb3,0x16,0x80,0x8a,0xab,0x3f,0x3a, -0xa3,0x9f,0x8e,0xf1,0x21,0xc9,0xb9,0xbf,0xc2,0x1b,0xf7,0x60, -0x4c,0x10,0xde,0xe7,0xab,0xc5,0x1d,0x6e,0x25,0xac,0x80,0x4b, -0xc9,0x1c,0xc7,0x0d,0x85,0x42,0x3e,0xd1,0x33,0x82,0xc9,0x11, -0xde,0x81,0xea,0x6f,0x22,0x87,0xe2,0xf0,0x5a,0x38,0xbc,0xe1, -0xaa,0x87,0xa8,0x5a,0x93,0xac,0xe3,0x4c,0xca,0xb1,0x79,0x87, -0x81,0x9b,0x56,0x5a,0x78,0xa8,0x67,0x16,0x0a,0x50,0xa2,0x61, -0xeb,0x20,0x01,0x2d,0xc4,0x11,0xbe,0x00,0x10,0x0f,0x98,0x51, -0x6b,0x29,0x3a,0x45,0x62,0x76,0x24,0x80,0x0c,0xaf,0x54,0xaa, -0x54,0x4b,0xbc,0x28,0xe5,0xce,0x23,0x95,0x47,0x72,0xe5,0x3c, -0xe8,0x7a,0x08,0x42,0xfb,0x12,0xdc,0x78,0x0a,0x86,0x1e,0x2e, -0xd2,0x18,0x1a,0x91,0x6e,0x39,0xdf,0x91,0xad,0xe2,0x1c,0x70, -0x7d,0xee,0xef,0x5d,0x62,0x6b,0x74,0xb7,0x80,0x89,0x31,0xa8, -0x9d,0x3b,0x7f,0xe3,0x79,0xaf,0x22,0x64,0x59,0x4f,0xd3,0xda, -0x5a,0x2e,0x96,0x92,0x91,0x04,0x0f,0xe3,0x87,0x2c,0xa0,0x19, -0x11,0x25,0x08,0xc0,0x85,0x28,0x07,0x52,0xc2,0x46,0xd5,0xf4, -0x0f,0x42,0x0f,0xb8,0x5f,0xcb,0x1f,0xa2,0x56,0xbc,0xa0,0xa4, -0xab,0xae,0x1a,0xce,0xeb,0xea,0xf4,0xc3,0x43,0xb7,0xb2,0x80, -0xa0,0x06,0x89,0xa7,0x1c,0x86,0x95,0x53,0xa0,0x64,0x82,0xb9, -0x2e,0x88,0x57,0xcd,0xdd,0xfc,0xa9,0xe2,0x5a,0x14,0xd3,0x7a, -0xf9,0x64,0x09,0xfe,0x23,0x32,0x52,0x10,0x01,0x6e,0x8d,0xa2, -0x88,0xd4,0xc8,0x04,0x2b,0x31,0x8e,0xf8,0xad,0xc8,0x5d,0x11, -0xad,0x3a,0x8a,0xa0,0x43,0xf0,0xbc,0x24,0x97,0x35,0x4d,0xd0, -0x2d,0x37,0x81,0x60,0xb7,0xd9,0x04,0xa6,0x08,0x2a,0x45,0xb6, -0xf2,0x53,0x85,0xc9,0x5f,0xb1,0x93,0xc1,0x0a,0x44,0x88,0xfa, -0xb3,0x50,0x6d,0xba,0x50,0x75,0xc8,0x7d,0x85,0x91,0xf4,0x3e, -0x21,0x6a,0x29,0xa8,0x15,0xe6,0x39,0x5f,0x89,0x8f,0x68,0x05, -0x94,0x9e,0xaf,0xd8,0xdd,0x80,0x90,0xae,0xf4,0xae,0x5e,0x7a, -0x14,0xa5,0xfe,0xe6,0x28,0xda,0xf8,0x62,0x4a,0x87,0xe6,0xfe, -0xe6,0x08,0xc1,0x8e,0x68,0x78,0x3a,0x4e,0xfd,0xf4,0x98,0x62, -0x92,0x39,0xc7,0x80,0x39,0x05,0xe4,0x6c,0xcb,0xbe,0xf0,0x86, -0x80,0x5e,0x82,0x96,0x17,0xf2,0xb1,0x28,0x13,0x0e,0xe6,0x23, -0xb1,0x16,0xef,0x42,0xa0,0x8f,0xdc,0xbf,0x8d,0x36,0xbd,0x15, -0x5e,0x4b,0xa7,0x7a,0xda,0xf1,0x06,0x0c,0x30,0x94,0xbf,0x4d, -0x14,0x89,0x06,0x61,0x46,0xb9,0x84,0xfc,0xfe,0xfe,0xf6,0x5c, -0x76,0x28,0x3a,0xea,0xa1,0xb4,0x8f,0x3a,0xc2,0xcf,0x4c,0xc1, -0xeb,0x07,0x22,0x3e,0xb4,0x9f,0xc3,0xdf,0x49,0x1e,0xdd,0xa2, -0x53,0xfb,0xa2,0x25,0x5f,0x32,0x51,0x5d,0x34,0x6b,0x88,0x72, -0x9a,0x99,0x7f,0x2b,0xa8,0x59,0x30,0x2d,0xb7,0x40,0x8b,0xd7, -0x5d,0xb4,0x7e,0xc3,0x3d,0x4f,0xcc,0xda,0x88,0x4a,0x59,0xb8, -0x26,0x0d,0x40,0x2f,0x48,0x69,0x46,0xb7,0xfc,0x2e,0x49,0x15, -0xd5,0x99,0x49,0x75,0xf6,0x20,0xd5,0x19,0xa0,0xcd,0x89,0xea, -0xbc,0x40,0x75,0xde,0x90,0xc5,0xe5,0x3b,0xa8,0x0e,0xa9,0xea, -0x82,0x6a,0x33,0x8b,0xa8,0x86,0xa2,0x3a,0x43,0xaa,0xb7,0xd6, -0x06,0x1d,0x39,0x31,0x99,0xab,0x08,0x2c,0xcf,0x71,0x7a,0x7f, -0x6f,0x9e,0x88,0x29,0x2c,0x97,0xca,0xa9,0x3e,0xcf,0xef,0x03, -0x5f,0xcd,0xf0,0xe9,0x63,0xa2,0xee,0x92,0x10,0x61,0x75,0x9f, -0x84,0x2f,0xb1,0x1f,0x1e,0x96,0x8e,0x75,0xc8,0x75,0x4c,0xd2, -0x1b,0x7d,0x0c,0xc8,0xec,0xd9,0x91,0x40,0xc3,0xe6,0xa3,0x50, -0x1a,0xae,0xc4,0xd5,0x93,0x5d,0xb0,0xa3,0xd4,0xa1,0xc4,0x85, -0x09,0x39,0x3d,0xcb,0xc9,0xa0,0x51,0xf9,0xe0,0x8e,0x48,0x79, -0x68,0x76,0xeb,0x13,0xf3,0xb9,0x79,0xe8,0xd3,0x34,0x28,0xe5, -0x3c,0x8f,0x6e,0x1b,0x82,0x40,0xc1,0x0a,0x05,0x1e,0x7f,0x3f, -0x9c,0x7c,0x8b,0x89,0x4c,0x11,0x36,0x19,0xdd,0xe3,0xa9,0x47, -0x31,0x9f,0xba,0x1f,0xed,0xdf,0x85,0x0d,0x7d,0x29,0xc8,0xa9, -0xee,0xcd,0x02,0xd3,0xe3,0x8f,0x16,0x88,0x8c,0x9e,0xb1,0xda, -0x85,0x78,0xaa,0x6e,0x8b,0x32,0xd6,0x97,0x26,0x5f,0x94,0x27, -0x79,0xcb,0x48,0xbe,0xbd,0xc0,0x38,0x78,0x3f,0x42,0x9b,0x11, -0xfa,0x14,0x19,0x59,0xae,0xcb,0x5e,0x1d,0x1f,0x18,0x67,0xa7, -0x2a,0x99,0xe7,0x6a,0xa5,0x7d,0xf3,0x65,0xaf,0xbf,0xf4,0xe9, -0xbf,0x41,0x67,0xe9,0x99,0x87,0x43,0x26,0xaf,0x17,0x74,0x91, -0xee,0xda,0xbe,0xc3,0xa8,0x1e,0x77,0xe6,0x6d,0x77,0xbd,0x24, -0x23,0xf8,0xc2,0x77,0x52,0x42,0xe1,0x7a,0xa5,0x62,0x22,0x2d, -0x72,0x75,0x77,0x96,0x57,0x5b,0x43,0x6b,0xe5,0xcc,0xa5,0xc7, -0x3a,0xa5,0x07,0x80,0xd7,0x33,0xb8,0xdb,0xd1,0x35,0x25,0x00, -0xf4,0xb5,0xec,0x67,0x3e,0xfd,0x47,0x82,0x25,0x6b,0x47,0x8b, -0x07,0x32,0x20,0x1b,0x51,0xb7,0x26,0x8b,0xe7,0x7e,0x6e,0x0f, -0x04,0x9a,0xf5,0xb4,0x2c,0x77,0x84,0x8b,0x87,0x92,0x14,0x8a, -0x3d,0x10,0x9e,0x1f,0xba,0x10,0x21,0xbd,0x7b,0xba,0xd2,0x13, -0x6c,0xe6,0xeb,0xbe,0x87,0x1d,0x40,0x0b,0x23,0xa5,0x1b,0x15, -0x20,0xf9,0xdf,0xd3,0x6f,0x03,0x41,0x0d,0xc1,0xa6,0x08,0x8d, -0x8a,0xba,0xd6,0x5e,0x98,0x08,0x32,0x6b,0x38,0x8e,0xc6,0x85, -0x51,0x0f,0xad,0xd6,0xa2,0xe8,0x17,0xe6,0x38,0x7c,0x9f,0x42, -0x04,0x93,0x63,0x18,0x5e,0xa7,0x7a,0xca,0xfc,0x1f,0xff,0xf0, -0x7f,0xff,0x37,0xf9,0x26,0x58,0x91,0x00,0x29,0x5a,0xed,0x73, -0xdc,0x30,0xfe,0xfd,0xdf,0x22,0xb0,0x2d,0x43,0xfa,0xfe,0x8f, -0x7f,0xa0,0xbd,0x67,0xcc,0x94,0xf8,0x92,0x08,0x50,0xe9,0xdd, -0xc9,0xe2,0x49,0x16,0xb5,0x8f,0xac,0x5b,0x1c,0x60,0x0c,0xce, -0x2b,0xe2,0x56,0xf2,0x0a,0xa2,0xee,0x1d,0x98,0x2d,0xb1,0x7b, -0xb0,0xba,0xbf,0x5f,0x9d,0x87,0xaa,0x9b,0xfe,0xc7,0x3f,0x1a, -0xbf,0xff,0xdb,0x51,0x76,0xa4,0xf5,0x4d,0x53,0x20,0x99,0x2e, -0xae,0xdc,0x95,0x77,0xbc,0x02,0xab,0xc6,0x87,0xef,0x6f,0x0b, -0xb5,0x62,0x7a,0x44,0x9d,0x55,0xed,0x5b,0x5c,0x79,0x71,0x4e, -0x67,0x72,0x62,0x9c,0xd2,0xe9,0x16,0x6f,0x16,0xb3,0xdf,0x8b, -0x8b,0xb3,0xa5,0x2b,0xe9,0xc6,0x22,0x93,0xd5,0x5d,0x32,0xff, -0x12,0x34,0x8d,0x2e,0x72,0x56,0xbe,0x9d,0xcc,0xd8,0xc6,0x16, -0x6f,0x33,0xcb,0xc7,0x1c,0xd3,0x46,0x64,0xf1,0x42,0x59,0x44, -0xc7,0xb3,0x2d,0xdf,0x33,0x4a,0xc3,0x0c,0x9e,0xb1,0x90,0xaf, -0xc7,0x63,0xf8,0x62,0x0e,0x66,0xac,0xbe,0x7d,0xf5,0x72,0xba, -0x58,0xac,0xac,0xeb,0x95,0xc7,0x71,0x32,0x35,0x23,0xe8,0x88, -0x06,0x65,0x6d,0xf1,0xc5,0xd6,0xd5,0xd7,0x61,0x33,0x72,0x86, -0x70,0xf1,0x19,0xf9,0x81,0xcf,0x7f,0xed,0xeb,0x5a,0x71,0x6e, -0xa1,0x89,0xc3,0x36,0xd8,0x89,0x0e,0x9a,0xa8,0x50,0x3a,0xcc, -0x5b,0xa0,0xba,0x60,0xc8,0x72,0x21,0x20,0x4c,0x01,0x4c,0x27, -0x7c,0x1c,0xa4,0x82,0x81,0x57,0xa8,0x2b,0xe4,0xff,0x02,0xdb, -0x8b,0x6e,0x10,0xbb,0xa4,0x2f,0xd7,0x69,0x05,0xb1,0x43,0xcb, -0xd0,0x0a,0xe6,0x5b,0x60,0xc9,0xb7,0xf1,0x15,0xf4,0xb0,0xdb, -0x58,0xb2,0xa4,0xdc,0xb7,0x9a,0xc4,0x6e,0x64,0x8f,0xdb,0x04, -0xfd,0xca,0x8c,0xaa,0x03,0x23,0x24,0x04,0x67,0xf0,0x6f,0x09, -0x53,0x87,0x1d,0xb5,0x85,0xe4,0x72,0x6d,0xfd,0x90,0xf9,0x68, -0x72,0x0d,0xc5,0xb8,0x12,0x91,0x2e,0x07,0xd8,0x04,0x02,0xb7, -0x98,0xe3,0x13,0x59,0xad,0x53,0x33,0x33,0xbf,0xc8,0x3c,0xdd, -0x45,0x88,0x00,0x71,0x61,0xa2,0xfe,0x61,0xdd,0x71,0xc8,0x35, -0xcb,0xf1,0x87,0x9d,0x7e,0x7f,0x80,0x67,0x5a,0x34,0xae,0x31, -0xb4,0xe8,0xd5,0x62,0x95,0x87,0xc1,0x2e,0x72,0x24,0x3a,0x4f, -0xba,0x49,0x9f,0x5c,0xe8,0x8c,0x55,0xe8,0xda,0x1f,0x8b,0xae, -0x5d,0x83,0x6e,0xf4,0xd1,0xf8,0x46,0xd5,0x08,0xc7,0x1f,0x8d, -0x70,0xac,0x65,0xd4,0xe0,0x5e,0xbf,0x1d,0x9c,0xb5,0xdb,0x67, -0xd0,0xd0,0x67,0xcf,0x9f,0x3d,0x83,0x80,0xdf,0x7e,0xd9,0x7e, -0x76,0xfa,0xe2,0x85,0x1f,0x3e,0x0b,0x4e,0x9e,0x3d,0x6f,0xbf, -0xf0,0x5f,0x3e,0x6f,0x43,0xf8,0xd4,0x7f,0x19,0xbe,0x7c,0x1e, -0x9e,0x80,0xb8,0x3f,0x7b,0x19,0x3c,0x3f,0x39,0x09,0xfc,0x17, -0x27,0xa7,0xa7,0xcf,0x43,0x88,0x68,0x9f,0x9c,0xbd,0x0c,0x5f, -0x9c,0xf9,0xe1,0xe9,0xe9,0x59,0xf8,0xfc,0x74,0x20,0xbd,0xf7, -0x57,0x57,0x23,0xbe,0xf7,0x99,0xa4,0xde,0x60,0xb2,0x2e,0x35, -0x7c,0x76,0xfa,0xec,0xd9,0x33,0xa3,0x78,0xf8,0x38,0x7d,0x79, -0xf6,0xbc,0xad,0xe9,0x08,0x02,0xc0,0xfd,0xd2,0x20,0x08,0x41, -0x82,0xe7,0xcf,0xda,0x8a,0xb2,0xb0,0xfd,0xf2,0xf4,0xf4,0xc5, -0x59,0xa8,0x49,0x6c,0x43,0x74,0x88,0x68,0x25,0xad,0xe1,0x59, -0xfb,0xe4,0x59,0xf8,0xf2,0x99,0x26,0xfa,0x24,0x78,0xd9,0x7e, -0x11,0x3c,0x37,0xa8,0x07,0x3a,0x5e,0x9c,0xbe,0x78,0x29,0xab, -0x01,0x25,0x3f,0x0b,0x03,0x28,0xe2,0xc1,0xfa,0x8c,0xa2,0xfe, -0x09,0x92,0xf4,0xfc,0xb9,0x7f,0x7a,0xf2,0x12,0x32,0xbf,0xf4, -0x9f,0x07,0xcf,0x82,0xe7,0xf0,0x1b,0x06,0xed,0xb3,0x67,0x6d, -0xc0,0x7e,0x76,0x1a,0xbe,0x0c,0x5e,0x3e,0xf7,0x5f,0xb6,0xc3, -0xb3,0x53,0xa0,0x23,0x0c,0x5f,0x86,0xa7,0x61,0x08,0xa5,0x00, -0x41,0x67,0x67,0x27,0x67,0x90,0x12,0xb4,0x5f,0xbe,0x3c,0x01, -0x82,0x9e,0xb5,0xcf,0x5e,0x04,0x48,0x07,0x90,0x05,0xd5,0xc0, -0x0f,0xc8,0x7b,0xd2,0xf6,0x5f,0x9c,0x9d,0x9d,0x85,0x67,0x90, -0x02,0x99,0xce,0xa0,0x6a,0xc8,0x93,0x67,0x2f,0xcf,0x9e,0x9d, -0x20,0xc8,0xf3,0xe7,0x27,0xc0,0x36,0xa8,0xd2,0xf3,0xb3,0x17, -0x10,0x11,0xbc,0x38,0x79,0x76,0xd2,0x7e,0x86,0xf8,0xdb,0x2f, -0x4e,0x11,0x1d,0xb0,0xf3,0xe5,0xd9,0xe9,0xb3,0x07,0xab,0x33, -0x86,0xea,0xb4,0x9f,0x3d,0x03,0xbe,0x41,0x35,0x90,0x59,0x01, -0xb2,0x22,0x38,0x3b,0x6b,0x23,0xb5,0x01,0xa4,0x21,0x8b,0x01, -0xe5,0x69,0x70,0x0a,0x05,0x21,0x49,0xcf,0xcf,0x9e,0xe1,0x47, -0xfb,0xb4,0x7d,0xfa,0x12,0x3f,0x4e,0x9e,0x41,0xa3,0xf9,0x27, -0xed,0xd3,0x93,0xe0,0xf9,0x99,0xff,0xfc,0xec,0xb4,0xfd,0xec, -0x39,0x62,0x39,0x0b,0x9e,0x07,0x01,0xe4,0x39,0x09,0x03,0x40, -0xf0,0x0c,0x52,0x5e,0x02,0x75,0xa8,0x42,0x4f,0x5e,0x02,0xd6, -0x36,0xb7,0x06,0xd6,0x23,0x7c,0x76,0x02,0x7c,0x7b,0x19,0x22, -0xe3,0x4e,0x80,0xb7,0xfe,0xcb,0x17,0x67,0x2f,0xa0,0x44,0x64, -0xcf,0xc9,0xb3,0x17,0xc0,0xea,0xf0,0x59,0xfb,0xf9,0xf3,0xe0, -0x45,0x58,0x51,0x1f,0xad,0xf9,0x60,0x0a,0x00,0x83,0x49,0xe5, -0x35,0xdc,0x4a,0xeb,0x48,0x18,0x54,0x3b,0x7d,0x4b,0xdf,0x71, -0xca,0x6f,0x7b,0xe4,0xfe,0x2d,0x76,0x83,0x2a,0x8d,0xa9,0x9c, -0x86,0x77,0xa1,0x50,0x40,0x6e,0xbf,0x75,0x3a,0x28,0x23,0x91, -0x23,0xe2,0x2e,0x1c,0x12,0xc6,0x35,0x73,0xdf,0x5e,0xc1,0x7c, -0x76,0x5c,0xb9,0x28,0x0c,0xf1,0xd2,0xd2,0x08,0x22,0xf2,0x6f, -0xfd,0x81,0xb6,0xe2,0xaa,0x0f,0x02,0xfb,0xab,0x30,0x02,0x83, -0xed,0x01,0xa0,0x61,0x10,0x3d,0x74,0x38,0xb9,0x31,0xe1,0xa7, -0x79,0xa0,0x78,0xf6,0x77,0xc7,0xdb,0xa3,0x77,0x1f,0x42,0x2e, -0x67,0x99,0x0c,0x23,0x77,0x18,0x9c,0x0f,0x43,0x74,0xf9,0x1a, -0x46,0xc3,0x00,0x4b,0x1e,0x86,0x88,0x6a,0x32,0x04,0x32,0xc2, -0xe6,0x10,0x0c,0xbc,0xc9,0x65,0xf4,0xfb,0xbf,0x9e,0x4f,0x86, -0x3d,0x27,0x70,0x3a,0x4e,0xe8,0xf8,0xe3,0x80,0x0d,0x97,0xf1, -0x22,0x75,0x11,0x20,0x15,0xe1,0x14,0xc6,0x74,0x0c,0x8f,0x43, -0x23,0x1d,0xac,0xc5,0x34,0x34,0xd2,0x43,0xe3,0xdc,0xdf,0x79, -0xa4,0x49,0xfa,0x66,0x98,0xf5,0x56,0x41,0xcf,0xf9,0x26,0xf0, -0x9d,0xc6,0x2a,0x6c,0x38,0x9f,0xf2,0x8f,0x08,0x3d,0x09,0x9e, -0x80,0x74,0x3f,0xc1,0xc4,0xe6,0xce,0x54,0x0c,0x32,0x8e,0x80, -0xa1,0x02,0x86,0x0a,0x04,0x54,0x20,0x70,0xec,0x4a,0xc5,0xe0, -0xcf,0x50,0xd7,0x3f,0x83,0x16,0xc0,0x43,0xd5,0xc2,0xd0,0xd1, -0x58,0x14,0x17,0x1e,0xa5,0x41,0x45,0x3e,0xa7,0x31,0xb9,0x84, -0x70,0xf8,0x84,0x81,0x43,0x05,0x0c,0x5f,0x5f,0x23,0x59,0x2a, -0x2e,0xa0,0xb8,0x52,0x05,0x04,0x02,0xfc,0x40,0xe0,0x40,0x01, -0xcb,0x0a,0xfd,0x61,0x2a,0x02,0x3f,0x70,0x10,0xd7,0x3f,0xf1, -0xbd,0x65,0x5a,0xde,0x8d,0xa6,0xfc,0x4a,0xc7,0xfa,0x86,0xb0, -0x1b,0x10,0xdf,0xe9,0x58,0xf3,0x94,0xb6,0x06,0x78,0xa3,0x22, -0xf5,0xc1,0x6d,0x9d,0xfa,0x17,0x11,0xd5,0x55,0x73,0xa6,0x71, -0xcb,0x24,0x45,0x9f,0xf7,0xdd,0x31,0xad,0x2a,0xd0,0x2e,0x3d, -0xd6,0x6f,0xe9,0xac,0xb9,0xd7,0x31,0x92,0xb7,0x18,0xd3,0x32, -0x6b,0xb2,0x57,0x01,0x85,0xaa,0x17,0x0b,0x30,0x92,0xb9,0x80, -0xaa,0x73,0xe0,0xbb,0xf0,0xdb,0x8c,0x2b,0xa2,0x37,0xcf,0x84, -0x23,0xf6,0xf2,0x99,0xf0,0x5d,0xb8,0x4d,0xa6,0x17,0x31,0xeb, -0xf3,0xe1,0x88,0x77,0x0c,0xf9,0x56,0x8b,0xc4,0xd2,0xcb,0xa4, -0x0d,0x23,0xf7,0x41,0x6d,0xd8,0x78,0x50,0x15,0xe2,0xce,0x33, -0x28,0xa9,0x07,0xb5,0xe1,0x6e,0xcd,0x07,0x58,0x4a,0xca,0x4f, -0xc8,0x4e,0x5f,0x6b,0x2a,0xef,0x68,0xe5,0x6b,0x3d,0x05,0xa1, -0x01,0xd5,0x50,0xcf,0x48,0xcc,0xec,0x51,0xf3,0x7a,0x38,0xbd, -0xfc,0xfd,0x5f,0x7d,0x4b,0x83,0x45,0xbf,0xff,0x7b,0xf3,0x3f, -0xff,0xd1,0x15,0x43,0x06,0xce,0x00,0xc8,0xe9,0xc3,0x36,0xd6, -0x69,0xd3,0x1e,0x92,0xcc,0xd7,0xd5,0x5c,0xe3,0xfb,0x07,0xca, -0xa1,0xd4,0xa4,0xc2,0x02,0x62,0x44,0xdf,0x99,0xf4,0xab,0x13, -0x21,0xfa,0xe1,0x67,0xcf,0xe9,0x33,0xb7,0xd2,0x73,0x4a,0xdf, -0xfa,0x46,0x01,0x6f,0xd6,0xf1,0xf2,0xb3,0x18,0x66,0x53,0xb1, -0x3c,0xe3,0x18,0x15,0x52,0x69,0xbb,0xbe,0x98,0x85,0x22,0xeb, -0x72,0x30,0x3e,0x3d,0x58,0xc6,0xc3,0x9a,0x41,0x96,0x78,0x15, -0x0f,0xad,0x7b,0xcc,0xb6,0x66,0xbe,0x1a,0x8e,0x11,0xca,0x02, -0x82,0x1a,0x8e,0x29,0x2c,0xc0,0x31,0xfa,0x56,0x1c,0x13,0x21, -0xfa,0x31,0xfb,0x3f,0xc7,0x07,0x36,0x58,0xc0,0x70,0x66,0x37, -0xe6,0x84,0xd0,0x86,0x0b,0x19,0x8e,0x5b,0x80,0x3e,0x73,0x0b, -0x20,0xe7,0x74,0xa3,0xbb,0x72,0xb4,0x5d,0x5c,0x2e,0x8a,0x53, -0x3d,0x8f,0x63,0xed,0xb2,0x72,0x2e,0x4b,0xf1,0xab,0xfe,0x05, -0x12,0x4a,0xd1,0x8e,0x36,0xa9,0x7a,0x74,0x81,0xfb,0x86,0xb8, -0x1e,0x9a,0x1c,0x6b,0xd6,0x85,0x34,0x71,0x5f,0xb4,0x71,0x49, -0x0d,0x8e,0x1b,0x69,0x6b,0x19,0x34,0xd0,0xd8,0x49,0x5b,0x2b, -0x1f,0x02,0x21,0xfc,0x01,0x03,0x01,0xfe,0x04,0x5e,0xc3,0x8d, -0xc1,0x7a,0x9b,0xa6,0x2e,0x64,0xf6,0x7a,0xe3,0xcd,0xea,0x36, -0x36,0xc1,0xe8,0x23,0xf0,0x3a,0x85,0x84,0x35,0x7c,0xac,0x31, -0x81,0xd0,0x8a,0x10,0x46,0x03,0xda,0x35,0xa1,0xe5,0x0c,0x2a, -0x45,0x61,0xf2,0x70,0x04,0x32,0x36,0x13,0x25,0xf9,0x69,0x3c, -0xbd,0xf4,0x2f,0x7d,0x5d,0x6b,0x91,0x10,0x5d,0x8a,0x9b,0x0a, -0x30,0x9d,0x9d,0x89,0x84,0x17,0xa1,0x11,0x2f,0x81,0x31,0xd9, -0x36,0xc0,0x2a,0x61,0x76,0x1b,0x60,0xfb,0x65,0x11,0xec,0xfd, -0xb0,0xea,0xac,0xa0,0x48,0x7c,0xbf,0x63,0x18,0xe2,0xcb,0x1d, -0xcb,0xa0,0xd3,0x5f,0x1d,0x59,0x36,0x96,0x08,0x0a,0x13,0x0b, -0xdf,0xd5,0xb7,0x41,0x42,0x1b,0x24,0xf4,0x06,0xc6,0x1e,0x8a, -0x68,0x1d,0xf3,0x34,0x37,0xc8,0x6b,0x10,0x45,0x23,0xf8,0x7b, -0x78,0x08,0xdf,0x21,0x7d,0x87,0x5b,0xcb,0xac,0x5d,0xf9,0x4b, -0x5f,0x1d,0x92,0x26,0x3b,0x81,0x4d,0x01,0x61,0x25,0x3c,0x69, -0xb8,0xc3,0xf3,0xdf,0xff,0xd5,0x13,0xb6,0xc2,0x52,0x67,0xe6, -0x76,0x5b,0x05,0x50,0x11,0x30,0x78,0x81,0x56,0x85,0xe4,0xff, -0x03,0xbb,0x08,0xf3,0x2e,0xc3,0xad,0xf1,0x54,0x08,0xf2,0xc5, -0x7c,0x96,0x03,0x23,0x84,0x14,0xae,0x2c,0xd3,0x81,0xa4,0xfa, -0xcf,0xb3,0x1b,0xf8,0xf1,0x82,0xa2,0x5b,0xde,0xae,0x71,0x71, -0x55,0x35,0x9c,0x8b,0x07,0x15,0xa4,0x0b,0x9e,0x7c,0xb3,0x61, -0x63,0x3d,0x67,0xb0,0x73,0x24,0x57,0x6c,0x28,0x63,0x15,0xef, -0x97,0x08,0xac,0x82,0x3f,0x7b,0x61,0xd5,0xbc,0x2c,0x63,0x15, -0xef,0x9e,0x48,0x5a,0xff,0x2c,0xcb,0x43,0xd2,0x6c,0xd8,0x1e, -0x5c,0xc2,0x9f,0x62,0x7d,0x08,0xec,0xda,0xfe,0xd0,0x4f,0x48, -0xa0,0x32,0x84,0x31,0xe0,0x6a,0x51,0x58,0x18,0x53,0x22,0x29, -0x13,0x0d,0xdf,0x43,0xe8,0xe3,0x7c,0xf3,0x6c,0xc1,0xd1,0x70, -0x79,0x12,0x89,0x0b,0x68,0x0b,0x09,0xb3,0xc8,0x5d,0x06,0xad, -0xbc,0xb1,0x3c,0x69,0xe5,0x68,0x98,0x2c,0xa3,0x3e,0x08,0xf8, -0x87,0xac,0x03,0xb1,0x99,0x9f,0x77,0x66,0x5b,0x0a,0x9c,0xc8, -0xc0,0xf2,0x44,0x7b,0x90,0x46,0x4b,0x9a,0x5d,0x2f,0x57,0x0b, -0x7c,0xe9,0x10,0x28,0xf2,0x7c,0xd4,0xaa,0xcb,0x7e,0x30,0x68, -0x38,0xaf,0xf1,0x23,0x84,0x0f,0xec,0x19,0xfd,0xb6,0xfc,0x38, -0x19,0xec,0xd9,0x49,0x34,0x56,0x29,0xec,0xb2,0xba,0xdf,0xab, -0x14,0x35,0xeb,0x12,0x29,0x25,0x01,0xdd,0x7d,0x71,0x51,0x51, -0x40,0xf1,0xb5,0x61,0x81,0x49,0xcb,0xa8,0xc2,0x5d,0x14,0xd3, -0x9d,0x77,0x69,0x97,0xc4,0xd4,0xc2,0x2d,0x25,0x55,0xe1,0x36, -0x2a,0xbb,0x17,0x7e,0x03,0x3e,0x33,0xf0,0xea,0x68,0x8d,0xbb, -0x24,0x4a,0x35,0xb6,0x88,0x92,0xb4,0x02,0xb8,0xeb,0xed,0xd5, -0x14,0x06,0xd0,0x45,0x54,0x51,0xaf,0x52,0x53,0x3d,0xb2,0xca, -0x06,0x7a,0xb7,0x40,0x04,0x5b,0x4a,0x9a,0x14,0x8b,0x3b,0xde, -0x03,0x5c,0x49,0xf3,0xd9,0x68,0x51,0xdd,0xbd,0x38,0xc9,0xbc, -0xb8,0x49,0x96,0xcc,0x29,0x7c,0xf7,0x0b,0xbd,0x61,0x67,0x77, -0x2a,0xbe,0x0d,0x46,0x03,0xbe,0xa6,0xe7,0x29,0x3d,0xda,0x16, -0x2e,0x82,0x6e,0xe5,0x6e,0x67,0x64,0x65,0xf9,0x11,0x62,0x78, -0xa3,0xdb,0x8a,0x46,0x8f,0x40,0xb5,0xa5,0x42,0x31,0x2d,0xca, -0xbb,0x9f,0x50,0x8a,0x52,0x0c,0x91,0x64,0x1c,0x20,0x90,0x90, -0xb4,0x15,0xa1,0xc7,0xee,0x73,0x57,0x63,0xe4,0x2d,0x6f,0x0e, -0x59,0xb3,0x0d,0xc1,0x3b,0x75,0x7f,0x88,0x78,0xdc,0xb3,0x82, -0x61,0xfe,0x78,0xb5,0x48,0x8d,0x83,0x0e,0x7c,0x71,0x3b,0xcf, -0xc3,0xf4,0xc6,0x19,0xc6,0x1e,0x9f,0x6a,0x4f,0x5d,0xd4,0x40, -0xcd,0x93,0x23,0x1e,0xd6,0x9b,0xf0,0xf3,0x25,0xff,0xfc,0x24, -0xa3,0xbf,0xa4,0xd1,0xfe,0x27,0x95,0x48,0x91,0x3f,0x51,0xa4, -0x4a,0x92,0x80,0x4d,0x33,0x89,0xb3,0x83,0x91,0x86,0x52,0x34, -0x5b,0x98,0x37,0x52,0x18,0xa4,0xe5,0x45,0xda,0xf0,0x46,0x0d, -0xab,0x69,0x87,0xf3,0x93,0x00,0x9f,0x85,0xcf,0xa2,0x55,0x5e, -0x91,0xa4,0xea,0xc1,0x6b,0x31,0x39,0x2f,0x93,0x64,0xb8,0x04, -0xe2,0xa0,0x95,0x82,0xd6,0x4a,0x4e,0xda,0xb4,0xa9,0x62,0x91, -0x28,0xf2,0x68,0x88,0xf7,0x61,0x57,0x81,0x59,0x9a,0x55,0x5f, -0x4b,0x73,0x08,0x03,0x4f,0x4c,0xdb,0x48,0x81,0x09,0x0e,0x38, -0xeb,0x55,0x42,0x93,0x83,0xe6,0x64,0x71,0x67,0x3e,0x64,0x6a, -0x94,0x9a,0x15,0x39,0x61,0x8b,0x32,0x44,0x9f,0x00,0x17,0xf2, -0x68,0x95,0x1d,0x95,0x53,0x0c,0x12,0x65,0x8d,0x25,0x1b,0x04, -0x57,0x14,0x03,0x44,0xd8,0x26,0x6b,0xb3,0xfc,0x3f,0x4f,0x54, -0xd3,0xa6,0xaa,0x40,0x94,0xa0,0x89,0xef,0xeb,0xd0,0xaa,0x06, -0x7b,0x76,0x1a,0x15,0x14,0xc9,0x4d,0x9c,0xa7,0xae,0x67,0xbd, -0x47,0x49,0xde,0x5f,0x5f,0xcd,0xaf,0xe3,0x55,0xb2,0xfe,0x6a, -0xe2,0x57,0x45,0xfb,0x65,0x0a,0x8d,0xda,0x9d,0x78,0x7e,0x59, -0xb8,0x38,0x7d,0x3d,0x9c,0xbb,0x27,0x01,0x56,0x90,0xc6,0x82, -0x39,0x1f,0x80,0xd1,0xf8,0xbf,0x97,0x8f,0x73,0xa2,0x57,0xac, -0x5d,0xf2,0x84,0xcf,0x28,0x54,0xc0,0x92,0x66,0xa3,0x8a,0xc5, -0x53,0x56,0xb7,0x76,0x52,0x4d,0x11,0xad,0x78,0xb6,0x5c,0xe7, -0x35,0xf9,0x28,0xad,0x2e,0x23,0xfa,0x94,0xd4,0xe4,0xe3,0xbb, -0x88,0xab,0xb3,0x29,0xad,0x5a,0xce,0x86,0x49,0xea,0x05,0xd7, -0xc2,0x3d,0xb5,0x0a,0xbc,0x5e,0x1f,0x56,0x37,0x5d,0x4f,0xe5, -0x34,0xf0,0xba,0xe8,0xab,0x2e,0xa2,0x3b,0x26,0x31,0x3f,0xa0, -0xa3,0x95,0x09,0xb7,0xb5,0x09,0xd2,0x0f,0xa7,0x46,0xd5,0xf5, -0xab,0xad,0x36,0x95,0x50,0xac,0x0f,0x1e,0x53,0x37,0x67,0x95, -0x38,0xb5,0x53,0xbe,0x45,0x7e,0x32,0x89,0xe8,0x1a,0xf4,0x64, -0x62,0xbc,0xf1,0xd9,0x1f,0x74,0x65,0x5e,0x8b,0x8d,0x17,0x32, -0x56,0xa3,0xee,0x4a,0x37,0x01,0xf1,0x9a,0x06,0x21,0xb3,0x1f, -0xd2,0xf8,0xa0,0x10,0x8b,0xa7,0x10,0xe5,0xf4,0xb6,0x3f,0xd0, -0xd9,0x39,0x06,0x73,0xe3,0x7d,0x59,0xc2,0xf3,0x91,0x22,0x8b, -0x4e,0x9b,0x24,0x13,0x94,0x82,0x1e,0xe8,0xe8,0x80,0xc6,0xf5, -0x89,0x24,0x4d,0x3c,0xfe,0x6a,0xf7,0xa9,0x8b,0x0b,0xbc,0x93, -0xe5,0xe2,0x02,0x1f,0x32,0xf5,0xbc,0x9e,0xeb,0xc8,0x08,0x27, -0xe1,0x3b,0xb9,0x35,0x0e,0x05,0x1b,0x59,0xa1,0x42,0xdf,0xc1, -0xb3,0x2c,0x32,0x07,0x20,0x05,0xd6,0x09,0x68,0x0d,0x72,0x71, -0xd1,0x4f,0x26,0x03,0xfd,0x2c,0xa9,0xaa,0x39,0xf9,0x51,0x81, -0x64,0x58,0xf1,0x74,0xd2,0xc2,0x38,0x6c,0x66,0x08,0x87,0xe2, -0x1d,0x14,0x63,0x3c,0xb0,0x5b,0xd9,0xf2,0x9f,0x4e,0xa7,0xfb, -0x36,0x7e,0x2a,0x8e,0xbb,0x8b,0x4a,0x68,0x9c,0x1f,0x29,0x10, -0x9f,0xf2,0xba,0xc1,0xde,0x32,0xf1,0xe8,0x66,0x17,0xd7,0x40, -0xeb,0x86,0xff,0x60,0xf4,0xe0,0x1a,0xf6,0xab,0xca,0xee,0x27, -0x1a,0xfe,0x3e,0x72,0x7a,0x83,0x24,0x2e,0x22,0x89,0x59,0x53, -0x79,0xf3,0x6a,0x01,0xc2,0xa9,0x64,0x51,0xa4,0xf7,0x6f,0x50, -0x44,0xeb,0xa5,0xe7,0x06,0xa5,0x47,0xa7,0xd5,0x09,0xcc,0x1f, -0x91,0x8d,0xcb,0x64,0x6a,0x1d,0x6c,0xe3,0x70,0x41,0x2c,0xf8, -0xaf,0x94,0x09,0xdd,0xea,0xda,0xd3,0xee,0x40,0xfa,0xd8,0x71, -0x7e,0xe8,0x33,0x02,0xb1,0x25,0x0f,0x1c,0x27,0xcb,0x30,0x25, -0x21,0x28,0x08,0x82,0x74,0x27,0xfa,0x28,0xed,0x10,0x14,0xa5, -0x44,0xde,0xc5,0x55,0xd2,0x0e,0x4c,0x49,0x7d,0xbb,0x7b,0xfa, -0xa5,0x5e,0xe3,0xb1,0x82,0x87,0xf8,0x2d,0x7a,0x48,0x3d,0xd3, -0xd7,0x77,0xb1,0x79,0xe5,0x3b,0x3d,0xc0,0x4e,0x71,0xe2,0x52, -0x33,0xd5,0xc9,0xba,0x35,0x23,0xcd,0xab,0x76,0x8f,0x20,0x90, -0x1c,0xd7,0xab,0x10,0x6e,0x2e,0x82,0xe6,0x27,0xf4,0x2e,0xbc, -0x35,0xc4,0x5c,0xc4,0xe8,0x3e,0xcf,0x4f,0x8c,0x90,0x5b,0x25, -0x01,0x57,0x1e,0x5d,0xab,0xc5,0xbc,0x8a,0xe9,0xb9,0x45,0x42, -0x5e,0x7d,0x4e,0xad,0x36,0x6b,0x2a,0x88,0x12,0x55,0xde,0xd6, -0xf3,0x69,0xb8,0x5e,0xaf,0x6c,0x36,0x7d,0xfb,0x86,0xef,0xc6, -0x32,0xf7,0xaa,0x01,0x88,0x3c,0x5b,0xa1,0xd3,0xd3,0x43,0xe0, -0x44,0x99,0x7e,0xf5,0x9c,0x49,0x2c,0x81,0x7f,0xfb,0xa6,0x26, -0x03,0x24,0xf0,0x2b,0xf8,0xcb,0xe1,0x18,0x04,0x02,0x3f,0xa7, -0x0b,0x90,0x90,0x02,0x8e,0x1f,0x91,0x78,0x57,0xaf,0x3d,0x32, -0x27,0x47,0x3d,0x89,0xbf,0xe3,0x8e,0x1a,0x91,0xe3,0x14,0x3d, -0x09,0x13,0x7f,0xc8,0x8d,0x7b,0x65,0xbe,0xcc,0x4e,0x34,0xaa, -0xc6,0x3e,0x00,0x3c,0xd0,0x83,0x92,0xc8,0xf0,0x50,0xa3,0x75, -0x4e,0x8d,0x6c,0x2d,0x68,0x4f,0x8b,0x58,0xfc,0x04,0xd2,0xe8, -0xd9,0xe7,0x0a,0x72,0xa1,0x6a,0xf5,0x04,0x7f,0xfb,0xe6,0x31, -0x24,0xd7,0x72,0xe9,0x8f,0xd6,0xa2,0x0e,0xb1,0x51,0x31,0xd0, -0xf6,0xe5,0x0e,0x21,0x5d,0x12,0x41,0x38,0x9e,0xe0,0xc8,0x4d, -0xc2,0xe2,0x11,0x7a,0xac,0x21,0xa7,0xf8,0x1c,0xdd,0xa7,0x80, -0xbe,0xac,0x17,0xa1,0xc4,0xd6,0xaf,0x22,0x35,0x32,0x5e,0xcb, -0x8f,0x22,0xce,0xd7,0x9b,0x54,0x3f,0xf2,0xde,0x29,0x78,0x13, -0x22,0x74,0xc4,0x0f,0xf9,0xe3,0x22,0x35,0x1d,0x09,0x62,0x7a, -0xba,0x55,0xaa,0xe3,0x82,0x3a,0x02,0x77,0x48,0x07,0x89,0x6d, -0x39,0x0d,0x53,0xd8,0x0d,0x2e,0xf4,0x8c,0xb6,0xec,0xa8,0xef, -0xdd,0x3d,0xe8,0xc7,0xb2,0xb6,0x01,0xcc,0x42,0xdf,0x54,0xc8, -0xb4,0x5a,0x4c,0xbc,0x64,0xc5,0x60,0xaf,0x58,0xf8,0x35,0xd2, -0xab,0x1d,0x4d,0x41,0xad,0x3e,0x2c,0xa6,0x97,0xd4,0x9a,0xd5, -0x32,0xfa,0x58,0x02,0x6a,0x65,0x71,0x6f,0x9a,0x6a,0x85,0x4e, -0x90,0x49,0x4e,0xc4,0x0f,0x36,0x2a,0xa1,0xe5,0xb6,0x2c,0x34, -0xe3,0xc7,0x37,0x60,0xba,0xce,0xcd,0x05,0x66,0xe2,0x1d,0x0b, -0xc5,0x72,0x95,0x2c,0x56,0xb8,0xd1,0x68,0xde,0x48,0x0e,0xc0, -0xa6,0x2e,0xa4,0x08,0xa1,0xe0,0x00,0xe7,0x32,0x5e,0xad,0xf3, -0xa2,0x42,0x24,0x18,0x76,0x8d,0xaa,0xd0,0x0e,0x0a,0xe5,0x0e, -0xe5,0x00,0xa4,0xdf,0x25,0xf3,0xc9,0xe2,0x0e,0x5b,0xe5,0xf5, -0x62,0xb6,0x04,0x8e,0x4e,0xde,0x60,0x46,0x3d,0xc8,0x78,0x98, -0x26,0x49,0xf8,0x89,0x4f,0x10,0x57,0x2b,0x3d,0xbb,0x2f,0xd5, -0x69,0x0c,0xaa,0x58,0xaa,0x51,0x6a,0xe5,0xa7,0x19,0xb3,0x95, -0x2f,0x8d,0x57,0xf8,0x7f,0x83,0x0e,0x39,0x39,0x9f,0xd3,0x15, -0x5d,0x4e,0x4a,0x95,0xd7,0x36,0x0c,0x11,0xf6,0x81,0x5d,0xc3, -0xf9,0x96,0x44,0xc7,0x61,0x6b,0x43,0xa2,0x96,0x1a,0xc6,0xd3, -0xc4,0xa8,0x34,0x6a,0xec,0xbe,0x0c,0x89,0xcb,0x35,0x6d,0x55, -0x23,0x13,0x01,0x6f,0xa5,0x2d,0x61,0x29,0x04,0xae,0x2a,0x8b, -0x92,0x68,0x7a,0xa3,0xc9,0x1e,0x90,0x9c,0x8a,0xbe,0xcf,0x3d, -0xbf,0x56,0x7c,0xf6,0xd2,0x01,0x7f,0xa4,0xc5,0x77,0x77,0xca, -0xea,0x76,0xbd,0xb4,0xdb,0x75,0x5b,0x67,0x16,0x9d,0xe0,0xd9, -0x43,0xcd,0x5c,0xcf,0x37,0xbb,0xa4,0xc5,0x48,0x5d,0xd7,0x1d, -0x66,0x5a,0x9c,0x99,0xbb,0x52,0x6c,0x77,0x3c,0xd8,0x5e,0x98, -0xcb,0x11,0x0d,0x55,0x80,0x82,0x94,0xfa,0xd2,0xb8,0x97,0x56, -0x38,0x18,0x50,0x1d,0xc8,0x64,0xc3,0xb7,0x96,0x8d,0xa5,0x80, -0x52,0x67,0x5c,0x76,0x09,0xd6,0xb6,0xbc,0x5a,0xe3,0xc5,0x66, -0xbe,0x7e,0xd5,0x46,0xd6,0xf0,0x38,0xce,0xb7,0xc8,0xd1,0x65, -0x0c,0x87,0x87,0x4b,0x51,0x30,0x5d,0x5b,0x48,0x55,0xd8,0x65, -0x91,0xc5,0xc3,0x34,0x2e,0xb1,0x64,0x4f,0x9b,0x35,0x7c,0xc8, -0x66,0x45,0xe4,0x1d,0xb7,0x62,0x52,0xa1,0x5e,0x9e,0x12,0x2f, -0x35,0x11,0x64,0xd5,0x61,0x07,0x5e,0xf1,0xaa,0xb0,0x73,0xf7, -0x34,0x50,0xa9,0x7a,0xfc,0x0a,0x86,0x57,0xcf,0x84,0x09,0xbe, -0xfb,0xbe,0x27,0x17,0x6a,0xe8,0xa9,0xb8,0xf9,0xe4,0xd6,0x7e, -0xe4,0x44,0x2c,0x1f,0xbc,0xaf,0x27,0x96,0xc9,0x30,0x5f,0x8a, -0xa9,0x99,0xba,0x6e,0x3b,0xae,0xf9,0x1e,0xd9,0xbe,0xcc,0x60, -0xfc,0x0f,0x73,0x63,0x53,0xbc,0xb6,0xe1,0xbf,0x8d,0x21,0x92, -0x12,0x75,0xe4,0x36,0xf4,0x1f,0xe6,0x8e,0x66,0x4f,0x31,0x9b, -0xb7,0x37,0xa7,0x64,0xc1,0x0f,0x32,0x0b,0x2b,0xac,0x19,0x45, -0xcb,0x75,0xd3,0x24,0x5d,0xc7,0x73,0x39,0xe7,0x36,0x18,0x56, -0x63,0xe6,0xb2,0x9d,0x4a,0x6b,0x8b,0x51,0xe5,0xa2,0x70,0x22, -0x17,0x1e,0x2b,0x93,0xbf,0x9a,0x74,0xab,0xa3,0xa3,0x64,0xb2, -0x57,0xef,0xe1,0x36,0xa8,0xc2,0x51,0xbb,0xd2,0xa2,0xb7,0xc8, -0xea,0x24,0xd4,0xab,0x5c,0xdf,0x8e,0x12,0xbd,0xce,0x5d,0x49, -0xb1,0xfc,0xe2,0x6b,0x4b,0xf7,0xeb,0xfb,0xb4,0xdf,0xf6,0xe0, -0xea,0x50,0xd7,0x35,0x96,0x5c,0xe9,0x01,0xc6,0xfb,0xfb,0x52, -0x14,0xaa,0xa2,0x49,0x02,0x46,0xe3,0x1a,0x95,0xb3,0x78,0xc5, -0x98,0x9f,0xfe,0xf7,0xe8,0x31,0x48,0xbb,0x81,0xb7,0xb6,0x15, -0x50,0x3b,0xec,0x54,0x2c,0x3e,0x1b,0x07,0xae,0x2a,0x16,0x65, -0xac,0x05,0xba,0x40,0xad,0xd0,0x25,0x93,0x30,0x6a,0x34,0x0a, -0xcb,0xf7,0xd6,0x0a,0x8e,0xff,0xdf,0xb6,0xf2,0x62,0xf0,0x32, -0xfa,0x6e,0x84,0xdb,0xb3,0xad,0xf1,0x2a,0x46,0x83,0xaf,0xba, -0x41,0x82,0x81,0x67,0x54,0x93,0x75,0x53,0x23,0x32,0x63,0x44, -0x1f,0xac,0x58,0x8c,0x55,0x2b,0xb1,0xa1,0xb9,0x98,0x56,0x5c, -0x4d,0xdb,0x73,0x29,0x2d,0x34,0x1c,0xfb,0xb2,0x24,0xad,0x76, -0x9f,0x87,0x04,0xf7,0xca,0xfb,0x70,0xc5,0xc3,0x76,0x61,0x94, -0xc6,0xe3,0x97,0x6a,0xea,0xe8,0xd3,0xb3,0xa8,0x2c,0xb6,0x3c, -0xfa,0xf2,0x39,0x6c,0xd1,0x8a,0x17,0x17,0xf8,0xb0,0xeb,0xfa, -0xe2,0xe2,0xfe,0x9e,0xa2,0x39,0x31,0x2c,0x24,0x46,0xe2,0x20, -0xd0,0x62,0x89,0xe7,0xa5,0x11,0xbf,0xbc,0x64,0x0b,0xbf,0xc9, -0xf6,0x4a,0x7b,0x9c,0x93,0x5e,0xd1,0xb6,0x03,0x62,0x20,0xe5, -0x28,0xca,0xfd,0xa9,0x54,0x3b,0x17,0x5e,0x47,0x40,0x8a,0xd3, -0x6d,0x5e,0x47,0x63,0x24,0x50,0x71,0x90,0x4d,0x97,0xc6,0x11, -0x17,0x66,0x09,0x1c,0x55,0x8e,0xd9,0x5d,0xb0,0xe1,0x4b,0xd9, -0x31,0x10,0x13,0x5c,0x74,0xa5,0x17,0xb5,0x5d,0x87,0x50,0x3a, -0x5e,0x0b,0x15,0x8a,0x4b,0x55,0x12,0x6c,0x62,0x5e,0xfc,0x05, -0x67,0x13,0x44,0x5b,0x2b,0xc6,0x4f,0x17,0xdf,0x8e,0x4e,0xc1, -0xb8,0x74,0x9d,0x2b,0xe8,0xae,0xa2,0x6a,0x90,0x9f,0xd6,0x09, -0x9c,0xf1,0x74,0x98,0xa6,0x10,0x2f,0x90,0xb2,0x71,0xef,0x2c, -0x60,0x66,0x08,0x94,0x38,0xfe,0x7f,0xfe,0x43,0x60,0xcd,0x48, -0xff,0x9a,0x5b,0x35,0x5c,0x04,0xc4,0xbb,0x5e,0x65,0x3e,0xb9, -0x68,0xc6,0x08,0xfe,0xba,0x9c,0xe0,0x32,0x43,0x19,0x45,0x39, -0x6f,0x28,0x6f,0xe1,0x97,0x07,0x3a,0xe8,0xde,0x77,0x57,0x56, -0x12,0xf4,0xcf,0x75,0x81,0x23,0xaa,0x4e,0xc4,0x13,0x7c,0x5f, -0x8f,0xa0,0x44,0x99,0xee,0x92,0x8e,0x7e,0x0a,0x66,0x40,0x2b, -0x80,0xca,0x72,0x1d,0x8c,0x2c,0x71,0x41,0x22,0xb2,0x37,0xa1, -0x08,0x01,0x58,0xf2,0x8a,0xbd,0x0a,0x09,0x7a,0xae,0x3a,0x06, -0xdf,0x55,0x02,0x99,0xc5,0xe2,0x72,0x2f,0x80,0xd1,0x8d,0xc2, -0x60,0xa2,0x07,0xc8,0xfc,0xf8,0x23,0x88,0xd5,0xbc,0x2a,0x02, -0x91,0x7d,0x4e,0x4c,0x97,0x09,0x5c,0x08,0x19,0xee,0xae,0x96, -0x19,0x86,0xac,0x2b,0x91,0xf3,0x10,0x48,0x7d,0x89,0x82,0xfa, -0xf4,0x2e,0x41,0x55,0x0f,0xf3,0x0b,0x60,0x9e,0xf7,0x61,0x0c, -0xe6,0xa2,0xc3,0xaf,0xb4,0x38,0x1d,0xab,0x5b,0x2b,0xff,0x3c, -0xd0,0x04,0x6f,0x7d,0x55,0x61,0xc1,0xdc,0xbc,0xed,0xf8,0xe4, -0xa9,0xfb,0x63,0x82,0x77,0x10,0xfe,0x16,0x1b,0x14,0x4a,0x10, -0xc7,0x57,0xe6,0x88,0x05,0xea,0x07,0x5e,0x03,0x8b,0xfa,0x5e, -0xdc,0xaf,0x68,0xb0,0x4a,0x64,0xcd,0x00,0x7b,0xe0,0xed,0x2e, -0xca,0xce,0x60,0xc2,0xef,0x5d,0x2e,0xa2,0x11,0x99,0x26,0x39, -0xf6,0xa3,0xe7,0x61,0x3c,0xd3,0x1d,0x06,0x93,0x9b,0xc3,0x39, -0x7a,0xb3,0x41,0xe2,0x2c,0x99,0x4c,0xa6,0xd8,0x66,0x5a,0x0a, -0x65,0x5e,0x87,0x7c,0xc6,0xe4,0x53,0x0b,0xb4,0x73,0x4e,0xde, -0xc9,0xb2,0xe0,0x86,0xf3,0x53,0xf0,0xa5,0x04,0x40,0x7f,0xb2, -0x9f,0x0a,0x00,0x5e,0x77,0x04,0x23,0xc7,0x4d,0x97,0x5a,0x63, -0xbd,0x58,0x3e,0xba,0x29,0x9a,0x0f,0xb7,0x45,0xd3,0xdd,0x8f, -0x2b,0xfb,0x34,0x47,0x55,0x71,0xbb,0xda,0x63,0xff,0xb2,0x8b, -0x4d,0x12,0xfc,0x59,0x0d,0xd2,0x7c,0xb0,0x45,0x9a,0xf5,0x4d, -0x82,0x17,0xe9,0xed,0x68,0x93,0x77,0xa5,0x36,0xc9,0xf6,0x68, -0x93,0xec,0x0f,0xb7,0x49,0xa1,0x04,0xa3,0x81,0x82,0xea,0x36, -0xd9,0xbb,0x3c,0xdd,0x74,0x41,0x45,0x37,0x39,0x69,0xd7,0xb7, -0x0a,0x99,0x8e,0xb5,0x4d,0x52,0x6c,0x05,0xdf,0x6c,0xa5,0x2f, -0x83,0x9f,0xcc,0x26,0xf9,0x72,0x67,0x93,0xd0,0x0d,0x87,0x8f, -0x6e,0x93,0x87,0x9b,0xe4,0x8f,0xa8,0xac,0xc7,0x36,0xc7,0x7e, -0x65,0x7d,0x7c,0x5b,0xb0,0x39,0x5f,0xdf,0x1a,0x8f,0x6b,0x0c, -0xbb,0x2d,0x70,0x0b,0x43,0x98,0x43,0x04,0xf5,0xd9,0x70,0x2e, -0xae,0x02,0xc2,0x1b,0x7c,0x0a,0x09,0x2e,0xdd,0x54,0xa9,0x0d, -0x4d,0xfd,0x10,0x24,0x43,0x42,0x44,0x63,0x92,0x6d,0x8d,0xc1, -0x98,0x17,0xeb,0x2c,0x33,0x33,0x33,0x8d,0x8e,0x6a,0x00,0x9e, -0x44,0xed,0xc4,0xc2,0xa6,0xe9,0x43,0xa8,0x2c,0xe3,0x0b,0xcc, -0xa1,0x1d,0x30,0x62,0x81,0xd6,0xb4,0xef,0xf8,0x7e,0x88,0xf2, -0x8d,0x03,0x3c,0xe8,0x9a,0x52,0x2a,0x9e,0x28,0xfb,0x8e,0x12, -0x6c,0xf9,0x89,0xce,0x7c,0x8b,0xe3,0x10,0x36,0xa4,0x22,0x3a, -0x29,0x58,0x9b,0x51,0x3f,0x0c,0x06,0xbe,0x36,0x6d,0xc9,0x9e, -0x55,0x0b,0x59,0x19,0x2e,0x4c,0x12,0x55,0xfb,0x39,0x45,0x12, -0x68,0xe6,0x63,0x3e,0x61,0x3f,0x6f,0x29,0xd0,0x12,0x55,0xd8, -0x0b,0x8b,0x80,0xcd,0x70,0xa9,0xd9,0xa8,0x33,0x57,0x36,0xed, -0xe1,0xd5,0x29,0x9d,0x5a,0x5e,0x70,0xd1,0x8c,0x42,0x94,0xcd, -0xb3,0x82,0xf2,0xea,0x62,0x85,0x9b,0xa8,0xcd,0x1a,0x05,0x20, -0xb0,0xda,0xc9,0x06,0xf6,0x9f,0x1e,0xf1,0x92,0xa4,0x01,0x9f, -0x19,0x78,0x7f,0x12,0x0f,0x4a,0x2a,0x9c,0x62,0x7a,0xb1,0x37, -0x4e,0x21,0x44,0x16,0x52,0x11,0x67,0x60,0x2d,0x5d,0xd3,0xb9, -0xfb,0x92,0x98,0x79,0xcf,0xd6,0x31,0x51,0x23,0x2b,0x48,0x57, -0x43,0x65,0xa5,0x9b,0x13,0x45,0xe1,0x56,0x26,0x51,0xbc,0x8d, -0x68,0xaf,0x7a,0x95,0xca,0xae,0xc7,0x6e,0x53,0xb5,0x9f,0x9c, -0xd9,0x15,0x91,0xd8,0xad,0x68,0x83,0x75,0xdf,0x57,0xdc,0x5d, -0xb5,0xb3,0x45,0x64,0x86,0x86,0xd9,0x24,0xdf,0xcb,0xdb,0x98, -0x75,0x93,0x6c,0x46,0x93,0xe4,0x36,0x99,0x54,0x2d,0x80,0x17, -0x11,0x1f,0x1e,0x62,0x36,0xce,0x6c,0x1f,0x64,0x2c,0x76,0x84, -0x48,0x5a,0xe7,0x7e,0xb9,0x0b,0x45,0x1f,0xc0,0x56,0xec,0x84, -0x7c,0x19,0x30,0xfc,0x32,0x24,0x7c,0xa0,0xbd,0xd2,0x09,0xb7, -0xd2,0x6b,0x73,0xb4,0xda,0xa4,0xd7,0x95,0xd3,0x7c,0x4a,0xa9, -0x9f,0xe7,0x97,0x66,0xf6,0x72,0x9c,0xa1,0x97,0x76,0xe2,0x55, -0x93,0xd6,0x90,0x70,0x92,0x05,0xea,0x51,0x8f,0x42,0xcd,0xbb, -0x78,0x74,0x93,0xac,0x9b,0xeb,0xe1,0xb2,0x79,0x0d,0xb4,0x4d, -0x91,0xbe,0xe6,0x78,0x31,0xa5,0x51,0x69,0x75,0x35,0x1a,0xba, -0x81,0x4f,0xff,0x79,0x0e,0x2d,0x32,0x39,0xb3,0xc5,0x26,0x8d, -0xd1,0x0d,0x97,0x69,0x75,0x7c,0xfa,0xa1,0xf1,0x8b,0x01,0xd6, -0x8b,0xcd,0x98,0xc3,0x15,0x10,0xfe,0x68,0x38,0xbe,0xb9,0xe2, -0xab,0xa9,0xec,0x49,0xa4,0x4e,0x30,0x27,0x92,0x5d,0x1d,0x5d, -0x9a,0x42,0xe2,0x65,0x77,0xa5,0x29,0xa4,0x85,0x46,0x54,0xf2, -0x36,0x49,0x93,0x51,0x32,0xa5,0xd9,0xad,0x73,0x0d,0xb6,0x68, -0x3c,0xd7,0x89,0xe3,0xcd,0x2a,0xa5,0xca,0x92,0xff,0xf7,0xf5, -0x30,0x59,0xc1,0x30,0x6c,0x93,0x06,0x23,0x35,0x94,0x6d,0x92, -0xb5,0x1f,0x2d,0x2a,0x5f,0xb1,0x28,0x9c,0x98,0x8b,0xe9,0xe9, -0x2a,0x26,0x0f,0x51,0xbb,0x40,0x8e,0x94,0x05,0x72,0x28,0xf5, -0xcd,0xf3,0x9e,0x5d,0x8e,0x14,0x0b,0x00,0x7a,0xae,0x2f,0xa3, -0x0b,0xe4,0x5d,0x15,0x69,0x2b,0x8f,0xef,0x0e,0x67,0xc5,0xb3, -0xeb,0xdb,0x12,0xc5,0x15,0xe6,0x80,0x90,0x71,0x6a,0xdb,0xd7, -0x04,0xd6,0x9f,0x0c,0xb6,0x35,0x1c,0xc9,0xaa,0x50,0x1c,0xf7, -0xe3,0xbb,0xc1,0x3f,0x1f,0xc3,0x04,0x3a,0x5d,0x43,0x6c,0xaf, -0x79,0xc2,0x37,0x0f,0x1b,0x66,0x54,0x45,0xa6,0xbf,0xf7,0xe7, -0xe9,0xa0,0x3e,0xd3,0x5d,0x32,0x59,0x83,0xc8,0x9d,0xc9,0xf0, -0x75,0x4c,0xd6,0x27,0x46,0xec,0x92,0x06,0xc9,0x38,0x01,0x83, -0xeb,0xac,0xd3,0x61,0x2e,0x44,0x97,0x1d,0x83,0x5d,0xaf,0xe7, -0xcc,0x17,0xf3,0xd8,0xa1,0x02,0x45,0xeb,0xd1,0x8b,0x6d,0x57, -0x95,0xab,0x2b,0x57,0xa6,0xb0,0x57,0x42,0xe8,0x64,0xaf,0x4b, -0x17,0x7b,0x56,0x2c,0xbb,0x64,0x65,0x2c,0x9a,0xa7,0xd2,0x06, -0x2c,0x54,0x5e,0xda,0x82,0x4d,0x95,0x0e,0xd5,0x9b,0xac,0x86, -0x77,0x6f,0x5d,0x41,0x2a,0x4c,0x53,0xf2,0x9a,0x02,0xf3,0xda, -0x02,0xf3,0x72,0x81,0x92,0xbb,0xb5,0x25,0xbe,0x33,0x4a,0xe4, -0x18,0x15,0x61,0x3a,0x0d,0xc9,0x24,0xd4,0x6e,0x95,0x1d,0x81, -0x8b,0xd3,0x4e,0x32,0x55,0x02,0x4c,0xa9,0xb4,0x8d,0xef,0x34, -0xb2,0xbf,0x50,0xdf,0xeb,0x37,0x8e,0x63,0x2d,0x61,0x3c,0xbd, -0xcc,0x55,0xd2,0xdf,0x53,0x33,0xc9,0x73,0xca,0x14,0xbd,0x35, -0x49,0x32,0x35,0x81,0x6a,0x02,0x59,0x10,0x56,0xb9,0x4a,0x6b, -0xf8,0xad,0x39,0xdd,0x16,0xea,0xb7,0xd2,0x73,0xab,0x4b,0x88, -0xb6,0x92,0xf9,0x81,0x77,0x06,0xaa,0x22,0x1d,0xef,0x76,0xd3, -0x01,0x2d,0x93,0x3f,0x48,0x47,0x2c,0xe8,0xb8,0xb3,0xe9,0x90, -0x4d,0x98,0x6b,0x42,0xf2,0x2a,0x42,0xb4,0x12,0x37,0x47,0xa5, -0x9b,0x38,0xc7,0xb1,0x00,0xa2,0x4e,0xda,0x11,0x6d,0x96,0xe2, -0x30,0x83,0x27,0x0c,0x5e,0xb3,0x37,0x33,0x50,0x7f,0x75,0x05, -0xa3,0xef,0xfd,0xbd,0x3b,0x26,0xad,0xc4,0xf7,0x03,0x82,0xb5, -0x08,0xb1,0xf8,0x80,0x6c,0xa4,0x39,0x20,0x63,0x81,0x84,0x48, -0x93,0xe3,0x4b,0x14,0x51,0x9b,0x36,0x6e,0xa8,0x84,0xef,0x57, -0xf4,0x23,0xc6,0x5f,0xd7,0x7c,0xe7,0x0d,0xca,0xde,0x2c,0x6b, -0x09,0xc2,0x48,0x81,0x0f,0x9f,0x85,0x95,0x64,0x34,0x2a,0xc9, -0x68,0x54,0x92,0x11,0xec,0x43,0x05,0xb1,0x8b,0x95,0x32,0xef, -0x96,0xd3,0xbb,0x7d,0x78,0x0c,0x09,0xc7,0x4f,0x71,0x64,0xcf, -0xf3,0x11,0x62,0x82,0x17,0xd4,0xf3,0x7b,0xdd,0x74,0x9b,0x71, -0x42,0xac,0x6f,0x88,0x67,0xe4,0xf1,0x7a,0x55,0x8e,0x0b,0x75, -0x1c,0x5e,0xfa,0x66,0x70,0x56,0x55,0x73,0x38,0x5d,0xff,0xcf, -0x38,0xef,0x09,0x56,0x6b,0x9e,0xf7,0x5d,0x2d,0x5c,0x0d,0x5d, -0x53,0x9c,0xeb,0xb9,0xba,0xb5,0x1b,0xb9,0x99,0x32,0xf0,0x74, -0x2b,0x29,0xee,0x34,0x14,0x7d,0xaf,0x18,0x35,0x0a,0x89,0xc1, -0x31,0xc5,0x30,0x09,0x18,0x2a,0x40,0x40,0x3a,0xf0,0x3a,0x86, -0x10,0x08,0xa5,0x0b,0x55,0x78,0x7b,0x78,0x88,0x8c,0x08,0x39, -0x8f,0x9f,0x81,0xb5,0x81,0xda,0x49,0x75,0x41,0xc9,0xa6,0x40, -0x67,0x79,0x67,0x67,0xc9,0xfd,0x50,0x67,0x79,0x67,0x67,0xa1, -0x2f,0x95,0x88,0x69,0xc4,0xac,0x0b,0xf7,0x03,0xee,0x80,0x75, -0x1c,0x61,0xa5,0xe0,0x8d,0x90,0x1d,0xc9,0xd4,0x1e,0x8f,0xd1, -0x1d,0x31,0x2a,0x3a,0x5b,0xb3,0x6d,0xcd,0x82,0x79,0xa3,0x44, -0xf8,0xaf,0xcc,0x92,0x39,0x3e,0x2b,0x5d,0xbf,0x7e,0x0e,0xf4, -0x07,0xea,0xa1,0x57,0xbc,0xc9,0x48,0x2a,0x4e,0x75,0x07,0x79, -0x24,0x58,0x99,0x0c,0x7c,0xee,0xb8,0x51,0xd2,0x13,0x3c,0xed, -0x88,0x46,0xe0,0x93,0x7a,0xb1,0xea,0xae,0xb1,0x6c,0x3f,0xb5, -0xeb,0xad,0x05,0x7c,0x15,0x34,0x23,0x89,0x1a,0xca,0x6b,0xd2, -0x0d,0xbd,0x0d,0x19,0x03,0xbc,0x49,0xe6,0x91,0x0b,0x25,0xbc, -0x9e,0x0e,0x67,0xcb,0x4e,0x46,0x3f,0x9e,0xbe,0x45,0x7a,0x15, -0xf8,0xfa,0xa6,0xe2,0x50,0xc8,0x21,0xbe,0x28,0xdb,0x91,0x9f, -0x4a,0x0c,0x7b,0x78,0x2b,0xb5,0x0b,0x80,0x0d,0x55,0xa0,0xd7, -0xa0,0xe7,0x04,0x84,0x10,0x1a,0x37,0x75,0x47,0x75,0x05,0xb4, -0x8f,0x84,0xb0,0x24,0x83,0x26,0x44,0x79,0x1e,0x51,0x78,0x2e, -0xf3,0xf5,0xe8,0xa9,0x6d,0xe2,0x32,0xd0,0xad,0x8a,0xe9,0x60, -0xac,0xae,0x93,0xe2,0xc7,0x01,0x82,0xde,0xdf,0x2b,0x4e,0x41, -0x78,0x98,0xf5,0x5c,0xc5,0xd0,0xcf,0x69,0xbb,0x81,0x64,0x51, -0xf2,0xd6,0x88,0xd2,0x78,0xa8,0x44,0x85,0x05,0x91,0xe0,0xed, -0xed,0x9d,0xdb,0x45,0x32,0x79,0x12,0x14,0xba,0x3c,0xda,0x3e, -0xde,0x07,0xa3,0xf7,0xa3,0x36,0xde,0x6d,0x89,0x57,0x8d,0x77, -0xfb,0xdb,0x21,0xbe,0xb6,0xfb,0x61,0xfe,0x31,0xc9,0xcb,0xe6, -0x26,0x83,0xdd,0x69,0xeb,0x1d,0x09,0x93,0xb6,0x39,0x3b,0x4f, -0xa9,0xa4,0xcd,0xb2,0x9c,0x40,0x06,0x7d,0x75,0x1e,0x4a,0x42, -0x3f,0xa1,0x52,0x8a,0x18,0x18,0x2a,0x13,0x0a,0x85,0x90,0x08, -0xfd,0x40,0xcf,0xe9,0x21,0xbf,0x2a,0x3a,0x27,0xae,0x54,0x8a, -0xc5,0x1b,0x96,0x0f,0x9f,0xf5,0xa0,0x3c,0xc7,0x4d,0x3b,0xef, -0x94,0xef,0x47,0x79,0x14,0x5a,0xf2,0x44,0xa9,0x46,0xa9,0x71, -0x19,0x7d,0xc4,0x91,0x8b,0x4b,0xa1,0x89,0xcd,0xbb,0xc3,0xac, -0xa9,0x94,0xc8,0x25,0xd5,0x4e,0x64,0x94,0x82,0xd6,0xf9,0x66, -0xe6,0x1a,0x6a,0x2c,0x3a,0x38,0xfe,0xbb,0x3b,0xbf,0x4f,0x3d, -0x69,0x73,0xc8,0x14,0xd0,0x4d,0x99,0x56,0x5d,0x04,0x16,0xdf, -0xdf,0x55,0x81,0xe5,0x7a,0x9c,0x31,0x8b,0x22,0xb3,0x3d,0x06, -0xb3,0x5a,0x8e,0xfc,0x26,0xd3,0x22,0x39,0x1e,0x5d,0x60,0xe4, -0x9b,0xcd,0x72,0x09,0x08,0x53,0x57,0xea,0xef,0xf2,0xa8,0x73, -0x67,0x31,0x48,0x78,0xd7,0x55,0xb6,0x9b,0x08,0x56,0x34,0x1d, -0x05,0x3c,0xf4,0x1b,0x51,0x2c,0xe6,0x77,0x0f,0x27,0x3f,0x92, -0x50,0xa4,0xbd,0xbb,0x6a,0xe9,0x51,0x7d,0xa3,0x5a,0x84,0x64, -0x83,0x7b,0x9d,0x1a,0x89,0x2d,0xe4,0x2f,0x88,0xad,0xca,0x0e, -0xfd,0x8e,0xba,0xdb,0x6a,0xb3,0xc4,0x19,0x92,0xd1,0xcb,0x8e, -0x1c,0xcf,0x4c,0x52,0x0c,0xf7,0xca,0xc3,0x1d,0xd8,0x27,0x2a, -0xb2,0x62,0x90,0xd3,0xc9,0xe1,0x80,0x1f,0x64,0xc7,0xd7,0xd0, -0x64,0x63,0xd2,0x90,0x10,0x67,0x51,0xe3,0xf8,0xae,0xd4,0xd0, -0x7e,0x9c,0x43,0xfc,0xdf,0xe7,0xc5,0x78,0x61,0x07,0x44,0x7d, -0x35,0x48,0x83,0x7a,0x37,0x89,0xc8,0x75,0x7c,0x6e,0x94,0x3e, -0xa8,0x18,0xab,0x21,0x63,0x05,0xcd,0x90,0x4d,0x3a,0xb0,0x58, -0x46,0x03,0xa8,0x67,0x31,0x36,0x73,0x1e,0xf9,0xd8,0x83,0xd7, -0xad,0x57,0x60,0xa4,0x86,0xf6,0xd4,0x60,0x7b,0x2a,0x2b,0x53, -0xe8,0xed,0x24,0xaf,0x52,0x35,0xf0,0xc2,0xf9,0xd6,0xf3,0x0d, -0x95,0x4b,0x8a,0xc2,0x52,0xe8,0xbe,0xa5,0xf1,0x7d,0xe9,0x4f, -0xc3,0xbd,0xe6,0x73,0xe9,0x54,0x43,0x28,0x7c,0x13,0xaf,0x2f, -0x8d,0x03,0xad,0x87,0x3c,0x3f,0xe3,0xe1,0x21,0xe7,0x1f,0x51, -0x4e,0xd4,0x0f,0x7c,0xdd,0x3c,0x22,0xc4,0xe3,0x29,0xbe,0x37, -0x90,0xab,0x2f,0x31,0x9b,0x8f,0xcc,0xf9,0xf3,0x0f,0x1c,0x67, -0x0c,0xe1,0x42,0xdf,0xdf,0x5a,0x2b,0xb8,0xb5,0x8b,0x3f,0x25, -0x9d,0x56,0xb8,0x0d,0x91,0x95,0x46,0x18,0x7d,0xc8,0x94,0x0d, -0x91,0x77,0x04,0xa9,0x7e,0x62,0x8f,0x7d,0xfe,0xfb,0x8e,0xc5, -0xac,0xad,0xc8,0x5d,0xf6,0x11,0x11,0x58,0xbb,0x05,0xf7,0x10, -0x11,0x5d,0xe3,0x46,0xd5,0x2b,0x2e,0x53,0x99,0xa7,0x4f,0x85, -0xe3,0xa9,0xb5,0x78,0x64,0x54,0xd5,0x1e,0xa3,0x05,0x59,0xad, -0xc4,0x6e,0x5c,0x15,0xff,0x5e,0x35,0x8d,0x8c,0xc9,0x54,0xf3, -0xc8,0x98,0x7c,0x97,0x48,0x6d,0x3d,0xe9,0xcc,0x4b,0xd1,0x9d, -0x32,0x41,0x24,0x67,0x45,0x27,0xf2,0x4f,0x57,0xab,0x61,0x2e, -0xed,0x6d,0xc9,0xfa,0x16,0xcc,0xe3,0xf3,0x6a,0xc8,0xbc,0x00, -0x99,0x2b,0xc7,0x2d,0xbb,0xbe,0x65,0xa3,0xc5,0xf2,0x54,0xb7, -0x2b,0x1b,0xe2,0xd3,0x13,0xe4,0xa0,0x9e,0x17,0xe2,0xf3,0x28, -0xa7,0xf8,0x7a,0x1b,0x58,0xdb,0xbc,0xdb,0xad,0x67,0xf8,0x02, -0xef,0xd7,0x20,0x61,0x4d,0x83,0x84,0xd0,0x20,0xfb,0x94,0xb9, -0x63,0xfc,0xf7,0x3a,0xee,0x2e,0x0d,0xf0,0xc7,0xb0,0x93,0x23, -0x32,0x77,0x3b,0x63,0xfb,0xeb,0xb7,0x1d,0x0b,0xce,0x59,0xf4, -0xdb,0xce,0xde,0x7c,0x90,0xbd,0x7a,0x15,0xde,0x1f,0xe4,0x03, -0x46,0xeb,0x75,0x32,0x59,0x40,0xbe,0x5f,0x01,0xf9,0x63,0x0b, -0xc8,0x65,0x01,0x63,0xd2,0x36,0xfb,0xd5,0x02,0x6c,0x0e,0xf8, -0x2b,0xf4,0xd3,0xc1,0x6f,0x34,0xc8,0xe8,0x10,0xcc,0xda,0x3a, -0x59,0x4f,0x27,0x77,0x70,0xa0,0xd0,0xe9,0x9e,0xaa,0x1b,0xa2, -0xe9,0x33,0x9c,0xc8,0x3f,0x50,0x19,0x3b,0xb9,0x9c,0x61,0xf0, -0xd3,0x62,0x42,0xbb,0xb1,0x60,0x9a,0x54,0x52,0x87,0x0a,0xfc, -0x2c,0xf4,0x73,0xd0,0x99,0xa1,0xbf,0xae,0xf3,0xef,0x26,0xba, -0xdd,0x2c,0x88,0x88,0xde,0x2c,0x8c,0x90,0x52,0x5a,0xd3,0x82, -0xb8,0x2c,0x10,0x91,0x59,0x48,0xeb,0x21,0xb6,0x98,0xf6,0xa9, -0x00,0x48,0x07,0x23,0xe0,0x36,0x5e,0xad,0x45,0x1e,0xf8,0xe3, -0x51,0x1e,0x37,0x0b,0x3d,0x7c,0xcf,0xef,0x3c,0xc3,0x4b,0x5d, -0xd7,0x11,0x82,0x43,0x7a,0x88,0x89,0xd0,0x6f,0x00,0xee,0xc0, -0xb0,0x0f,0xee,0xef,0xb3,0xf0,0xc0,0x58,0x3c,0xc0,0x59,0xa8, -0x1a,0x12,0xb8,0x24,0xb1,0xd8,0x96,0x0b,0x62,0x73,0x41,0x6c, -0xc6,0x71,0x79,0x40,0x94,0x87,0x51,0x1e,0xd2,0xb4,0xde,0xee, -0x3b,0x7d,0xe2,0x03,0xa4,0x6b,0x62,0x31,0x0f,0xfc,0xf1,0x28, -0x8f,0x9b,0x23,0xb1,0x79,0x70,0x9e,0x33,0xb1,0x08,0x0e,0xe9, -0x21,0x26,0x22,0xb1,0x39,0x10,0x9b,0x1b,0xc4,0xe6,0xe1,0x81, -0xb1,0xa6,0x81,0xc4,0xaa,0x11,0x8b,0x4b,0xf2,0x54,0x8b,0x32, -0x87,0x4d,0x6a,0x7a,0xc4,0x28,0x33,0x46,0xf2,0xd9,0x8a,0x43, -0x91,0x31,0x20,0x6d,0x98,0x7e,0x99,0xf3,0x22,0xa4,0x1a,0x40, -0x85,0x77,0xb5,0x83,0x64,0xaa,0x4d,0x1f,0xd6,0xbd,0x48,0x1f, -0xb2,0xa9,0x4c,0x9f,0x86,0xb4,0x61,0xfa,0x65,0x66,0x4b,0x7a, -0x24,0xcf,0x55,0x78,0x17,0xeb,0x91,0x74,0xea,0x12,0x28,0x04, -0x39,0x14,0xd1,0x47,0xa1,0x06,0x43,0x0d,0x3a,0x85,0x90,0x0b, -0xec,0x4a,0x92,0xeb,0xba,0xe3,0x56,0xbf,0x2e,0x60,0xcd,0xff, -0xee,0xef,0xdd,0x9d,0x56,0xc7,0x03,0xc3,0x86,0x68,0xe1,0xad, -0x39,0xa9,0x2c,0x17,0x79,0x70,0x00,0xf4,0xeb,0x26,0x8c,0x8c, -0x06,0xbc,0xbf,0x3f,0x38,0x00,0xda,0x73,0x23,0x51,0x73,0x8f, -0x9f,0xc9,0xe0,0x47,0x34,0xd8,0xa0,0x22,0x95,0xeb,0x3b,0x8b, -0xb9,0xe3,0x59,0x1b,0x68,0xc6,0xd6,0x41,0xf4,0x01,0x5f,0x48, -0x49,0x9b,0x42,0x4f,0xfb,0xa0,0x9a,0xe3,0x3b,0x15,0x4a,0xad, -0xb4,0x3b,0x2b,0x6d,0x0e,0xc1,0xf9,0x5d,0x1a,0xeb,0x08,0xc8, -0x3b,0x8f,0x53,0x23,0x77,0x5c,0x80,0x48,0xef,0x6c,0x08,0x75, -0xb3,0xa7,0xa9,0x5c,0xa3,0x7e,0xdf,0x99,0xe3,0x1e,0x0e,0x3a, -0x67,0xc0,0xbf,0x3b,0x34,0x77,0xe9,0x0f,0xc5,0xd0,0x9f,0x3b, -0x07,0x9a,0x95,0x20,0xf8,0x6b,0x4e,0xb0,0x78,0x73,0x37,0xdf, -0x63,0x92,0xf0,0x19,0x3e,0xfa,0xfd,0x40,0xa5,0xd0,0x26,0xc0, -0xe7,0x78,0x6a,0x53,0xa4,0x43,0x4c,0xfe,0x46,0xdc,0x51,0xd4, -0x77,0xde,0x6c,0xe6,0x13,0xb4,0x95,0x9d,0x6f,0x16,0xe2,0xe3, -0xc7,0x4d,0x9c,0xf2,0xd7,0xdf,0xe2,0xc9,0x5c,0x7e,0xff,0x78, -0x0d,0x5c,0xe3,0xcf,0x2f,0x56,0x09,0x7f,0xbc,0x81,0xc9,0xe8, -0x0a,0x3f,0xe9,0x95,0x7b,0x2c,0xe8,0x62,0xb3,0x1e,0x1b,0x17, -0x78,0x7c,0x00,0x93,0x1a,0x8b,0xae,0x78,0xcb,0x99,0x2d,0x38, -0x3c,0x42,0xf5,0xd7,0x1f,0x5f,0x23,0x0c,0x5e,0x04,0x42,0xe0, -0xf9,0xc3,0xd0,0xb9,0x00,0xfe,0x02,0x44,0xeb,0x1d,0x48,0xef, -0x43,0x39,0x24,0x9c,0xc8,0xf6,0xe5,0x02,0xea,0xf2,0x50,0x1e, -0x02,0x12,0x19,0xbe,0x49,0xa6,0xd3,0x84,0x9f,0xc1,0x7f,0x30, -0x9f,0x09,0xab,0xb2,0xcf,0x37,0x30,0xe7,0x7a,0x38,0x27,0x81, -0xc9,0x4c,0x8b,0xf9,0xfa,0xfa,0xc1,0x2c,0x08,0x24,0x32,0xbc, -0xd9,0x8f,0xbe,0x37,0x16,0x69,0x3f,0x82,0x40,0xec,0xcc,0x81, -0x00,0x06,0xec,0x6f,0x30,0xfd,0xe2,0x6b,0x30,0x2b,0x72,0x05, -0x5b,0x3e,0x57,0xf2,0xdd,0x65,0x3d,0x4a,0x01,0x80,0x28,0xd3, -0xb2,0x70,0x48,0x11,0x5d,0x1a,0x77,0x7b,0x48,0x01,0x31,0x6e, -0x09,0x6e,0x5d,0x18,0x53,0x0d,0x81,0x28,0xdf,0x0f,0x4f,0xbe, -0x13,0x4d,0x95,0x40,0xd5,0xe1,0x92,0xb0,0x3b,0x11,0x96,0x44, -0xad,0x0e,0x1b,0x01,0xee,0x44,0x55,0x27,0x84,0x75,0x18,0x4d, -0xf8,0x07,0x10,0x97,0xc4,0xb3,0x1e,0x27,0x81,0xee,0x46,0x57, -0x14,0xdc,0x5a,0x64,0x08,0xb8,0x13,0x55,0x85,0x48,0xd7,0x21, -0x7b,0xb3,0x47,0x45,0x8b,0xc2,0x5e,0x89,0x0b,0x81,0x6a,0xb1, -0xa8,0xc1,0xc4,0xce,0x47,0xfa,0x55,0xeb,0x3d,0xa5,0x69,0xf9, -0xed,0x1b,0x4c,0x44,0xac,0x91,0xf3,0x74,0xf8,0xe4,0xe9,0xe8, -0xc9,0xd3,0xf8,0xc9,0xd3,0xb7,0x4f,0x9e,0xbe,0x73,0x2a,0xe0, -0x00,0x66,0x76,0xfc,0x74,0x72,0x5c,0x4e,0x15,0x18,0xbe,0xec, -0x3c,0xfd,0xa6,0xf3,0xf4,0x8d,0x63,0x6a,0xf3,0x3f,0x4f,0x8f, -0x9b,0x58,0x3f,0x1d,0x8d,0xf0,0xf5,0x6a,0x3a,0x74,0x22,0x4a, -0x60,0xf4,0x8c,0x9b,0xf1,0x32,0x46,0x46,0xc6,0x98,0x0c,0x24, -0x33,0x6c,0x60,0xcc,0xfa,0x2f,0xc3,0xf9,0x66,0xb8,0xa2,0x32, -0x63,0x18,0xf1,0xf8,0xf3,0x1b,0x7c,0xe3,0x0a,0x7e,0x3f,0x5d, -0xae,0x92,0x29,0x85,0x31,0xf6,0x5f,0x36,0x34,0xda,0xfd,0xcb, -0x66,0x8a,0xa1,0x4f,0x37,0x57,0x9b,0x14,0xd7,0x44,0xde,0xc4, -0xcb,0x75,0x4c,0xaf,0x99,0xf9,0xce,0x77,0xe3,0xf5,0x82,0xbf, -0xbe,0x5d,0xdc,0xca,0xc8,0xcf,0xe3,0x31,0x7f,0x16,0x8a,0x2f, -0xd6,0x02,0x48,0x61,0x32,0x98,0x02,0x2e,0xdf,0x2c,0x9d,0x0b, -0xe7,0xb2,0xb9,0x60,0x2e,0x92,0x8b,0xe3,0x92,0x78,0xc0,0xc3, -0x42,0x5a,0x39,0x1a,0x4f,0xb2,0x44,0x9a,0x68,0x83,0x86,0xd3, -0x8b,0x25,0xb4,0xed,0xac,0xfc,0x17,0x78,0x3f,0x9e,0x33,0x02, -0x87,0x39,0xd5,0x17,0x8e,0xf8,0xdc,0x71,0xf0,0x21,0x0d,0x8e, -0xda,0xfa,0x16,0x16,0xb1,0xfe,0x0c,0x52,0x2b,0xc0,0xd5,0xa0, -0x46,0x11,0xc6,0x68,0xe8,0x7a,0x0d,0x01,0x5b,0x40,0x61,0x11, -0x62,0xe7,0x10,0x07,0xd4,0x64,0x8d,0xd2,0xc8,0x0c,0xb1,0xab, -0xa6,0x0d,0xd0,0x82,0x91,0xde,0x06,0xc2,0xa1,0xdf,0x06,0x84, -0x2a,0x3e,0xc8,0x1a,0xea,0x4f,0x00,0x37,0x8f,0xef,0x9e,0x08, -0x23,0xc2,0x6d,0xc7,0x27,0x7e,0x60,0xbc,0x38,0x91,0x3f,0x50, -0x5d,0x5f,0x46,0x89,0xe1,0x50,0x85,0xd9,0xae,0xc0,0x70,0xfe, -0x10,0x37,0x09,0xd4,0xce,0xb7,0x2f,0x17,0x19,0xba,0x19,0x1a, -0x3c,0xa4,0x4e,0x69,0x04,0xca,0x8c,0xb1,0x19,0x88,0x20,0x95, -0xfc,0xfb,0xee,0xf2,0x9d,0x65,0x9d,0x6b,0xa6,0x99,0x82,0x47, -0x0d,0xc0,0x69,0x5d,0xf3,0xf1,0x33,0x7e,0x0e,0x8b,0x12,0x9a, -0x08,0xd2,0x3c,0x8b,0x9f,0x1d,0xa9,0x5a,0xda,0xe3,0x39,0xd4, -0x80,0x9a,0xb1,0x22,0xc1,0xf3,0x8e,0x5f,0x9c,0x3d,0x8b,0x4f, -0xc5,0xb1,0x4f,0xdb,0x86,0x54,0xcf,0xd5,0x1a,0x34,0xe6,0xb8, -0x3b,0x49,0xad,0x0f,0xf5,0x5a,0x2f,0xbe,0x5e,0xdc,0xc5,0xab, -0xd7,0xc3,0x14,0xb7,0x5a,0x92,0xe8,0x79,0x33,0xe9,0xaa,0xbb, -0x48,0x6e,0xe9,0x3e,0x56,0xc2,0xd9,0x07,0xe0,0xc1,0x9e,0x5d, -0xc9,0xad,0xee,0x4a,0x5e,0x4d,0x4b,0x36,0x8d,0x30,0xd8,0x8e, -0x8d,0xc4,0x7b,0xfa,0x7c,0xbf,0x4e,0x56,0x25,0x16,0xcf,0x8f, -0x0c,0xe6,0x8a,0x0c,0x65,0x21,0x91,0x82,0x5d,0x6e,0x22,0x45, -0x47,0x65,0x5b,0x15,0xdb,0x9e,0xf3,0x34,0x90,0xab,0x4c,0xf7, -0xf1,0x73,0xaa,0x4f,0x7e,0x10,0x45,0x89,0x47,0x97,0x2f,0x2a, -0xf6,0x35,0x70,0x2e,0x10,0x49,0xd6,0xd9,0xd2,0xa4,0xd2,0x49, -0xf0,0x14,0x4c,0x49,0xea,0x18,0x8e,0xcb,0x06,0x64,0xff,0x85, -0xb5,0xda,0x1a,0x1d,0xe8,0x2e,0x8e,0x6f,0x54,0x21,0x29,0x8d, -0x6d,0x56,0x5a,0x5a,0x48,0x2c,0xf4,0x1c,0x02,0xb1,0x7a,0x98, -0x00,0x2b,0x77,0x32,0x04,0x15,0xbd,0xcc,0x86,0xe5,0x48,0x05, -0x77,0x29,0x5f,0xb4,0x33,0x07,0x65,0xcb,0x1c,0x80,0x3c,0x3f, -0xc4,0x05,0x80,0x1f,0x62,0xd7,0x1c,0xa8,0x3d,0xb3,0x03,0x7d, -0xbd,0x58,0xdc,0x6c,0x96,0x85,0x0c,0x1c,0x59,0x9f,0x89,0xc7, -0xb3,0x07,0xca,0xb1,0x06,0xbd,0x8a,0xec,0x7b,0x96,0x5c,0x83, -0x86,0xc6,0xd5,0x5d,0x14,0xf0,0xb8,0x5f,0xc8,0xb0,0x4f,0x99, -0x95,0x19,0x1f,0xae,0x71,0x79,0xa0,0xaf,0x44,0xb1,0x37,0x05, -0x35,0xa8,0x96,0xf1,0x0a,0xb7,0xab,0x80,0x92,0xe3,0xbf,0x3f, -0x3d,0x2e,0x98,0x67,0xdf,0x0f,0x27,0x69,0xf4,0xc1,0x69,0x3a, -0x1d,0xc7,0xf1,0x2f,0x3a,0xce,0x13,0xc7,0x0f,0x3a,0x4e,0xe0, -0x6c,0x0b,0x70,0x00,0x34,0xec,0x54,0xfb,0x36,0x56,0xb2,0xbd, -0x3f,0x51,0x1d,0x6b,0xb0,0xf5,0x3f,0x7d,0x20,0x6b,0x01,0x7c, -0xb4,0x0b,0xbc,0x5c,0x55,0xce,0x2c,0x06,0x51,0xc8,0xfe,0xd9, -0x83,0xd9,0x4b,0x59,0xc6,0x1d,0xbb,0xba,0x6e,0xb5,0x29,0x0c, -0x4c,0x35,0x70,0xfb,0xcb,0x12,0x76,0xc5,0x54,0x77,0xa2,0x35, -0xaf,0xbf,0xf4,0xdb,0xa0,0x68,0xe3,0x8f,0xcf,0xfa,0xe5,0xa3, -0xb2,0x8a,0x25,0x00,0x91,0xf7,0xab,0x8f,0xc9,0xfb,0x34,0x6c, -0xdf,0xdf,0x87,0x6d,0x81,0xe2,0xfd,0xbe,0x28,0xc2,0x46,0x85, -0xc6,0x44,0x42,0x4e,0x00,0xcb,0xd7,0x8f,0x22,0xc4,0x5e,0x8f, -0x10,0x28,0x66,0x8f,0x43,0xc1,0xcd,0xdb,0x08,0x45,0x35,0xbe, -0x79,0x24,0x01,0x62,0x59,0x43,0xe4,0x5e,0x56,0x4a,0x95,0xc1, -0xb3,0xf3,0x28,0x6c,0xf7,0x9c,0xef,0xbf,0x81,0xae,0xf4,0xe9, -0x37,0xd0,0x81,0xde,0x3c,0xaa,0xb8,0x37,0x46,0x55,0xb1,0xb8, -0xbf,0xee,0x9d,0xbb,0x42,0xfb,0x33,0xd3,0x11,0xcd,0x5d,0x2d, -0xd5,0x62,0x39,0xea,0x6f,0x8f,0x2d,0x67,0xb6,0xa8,0x2a,0x27, -0x7b,0xb8,0x03,0x79,0xfe,0xdb,0xdd,0x40,0xdc,0xc3,0xf2,0x47, -0xb1,0x4d,0x5b,0xd3,0x4f,0xc3,0x20,0x10,0xc4,0xbc,0xfb,0x68, -0x14,0xf1,0x33,0x40,0xf1,0x0c,0x50,0xfc,0xac,0x48,0x45,0x8b, -0xd2,0x77,0x9e,0x3a,0xe5,0xb5,0x21,0x88,0xdc,0x6a,0x3d,0xb9, -0x84,0xd9,0x45,0xbc,0x22,0x3d,0x69,0x45,0xfd,0x0d,0x06,0x6a, -0xa5,0x1d,0x41,0x13,0x56,0x25,0x82,0xc6,0xb3,0xa2,0xbf,0xd1, -0x3a,0x0e,0xb4,0x59,0x39,0xc9,0xd0,0x57,0x14,0xf9,0x35,0x5e, -0x61,0x16,0x63,0x45,0x40,0x41,0x59,0x29,0xd0,0xca,0x7e,0x5c, -0x8e,0xfa,0xd2,0x8e,0x42,0x09,0x6e,0x3f,0x03,0x55,0x51,0x15, -0xfb,0xbe,0x94,0x5d,0x98,0x18,0x5f,0x17,0x28,0x33,0x7a,0xac, -0x3f,0xab,0xa0,0x5a,0xbc,0xa9,0xfd,0x4d,0x31,0x1b,0xf5,0x33, -0xe8,0x5f,0x56,0xf4,0xa7,0xb3,0xef,0x67,0xd0,0x85,0xac,0x38, -0xd1,0x47,0xa0,0x6f,0x94,0x98,0xc8,0xb8,0x79,0x35,0x01,0x84, -0xbe,0x8a,0xc9,0xa2,0xf8,0xbf,0xd5,0x65,0xe6,0x15,0x08,0x43, -0x92,0x0d,0xd6,0xd2,0x42,0xf4,0xdb,0xaa,0x14,0x14,0x5a,0x90, -0x59,0x2b,0x85,0xb8,0xf3,0xce,0x8e,0x93,0x52,0x66,0x48,0x16, -0xc5,0xff,0x2c,0xc5,0xcb,0xc6,0x9d,0x80,0x9d,0x3b,0x9c,0x7e, -0xcf,0x43,0xb7,0x16,0x32,0x7e,0x1a,0x9d,0x86,0xf2,0x5f,0xd2, -0xa3,0x5f,0x26,0x0d,0x3d,0x9e,0x0f,0x81,0x61,0xca,0x54,0xe0, -0x2b,0xfc,0x87,0xb3,0x0e,0x74,0x89,0x59,0x27,0xd4,0xe6,0xb6, -0x10,0x7e,0xd3,0xc6,0x14,0x51,0x7f,0x5d,0x8f,0x2d,0xb3,0x90, -0x63,0xbf,0x4a,0x17,0x65,0x38,0xd7,0x79,0xfa,0xae,0xf9,0x74, -0xd6,0x7c,0x3a,0xf9,0x51,0x2e,0xec,0xb4,0x9e,0x7e,0xfd,0xb3, -0x53,0x2a,0x25,0x81,0xdc,0xf6,0x1a,0x13,0x4c,0xab,0xbe,0x7a, -0xf3,0x1d,0xdf,0x91,0x76,0x78,0xd8,0xc0,0xe9,0x33,0x8d,0x74, -0x4e,0x3b,0x08,0x82,0x66,0x10,0xc2,0xff,0x3f,0x06,0x41,0x87, -0xfe,0x6f,0x41,0x14,0x20,0xed,0x95,0x28,0xfa,0x96,0xde,0x5e, -0xee,0x94,0xe2,0xfd,0x1a,0xc8,0x16,0xf1,0xd4,0xb8,0x65,0x99, -0xaf,0x68,0x13,0xb3,0x02,0x98,0x91,0x29,0x32,0x44,0x8a,0x9c, -0x0b,0x24,0xe9,0xb7,0xc3,0x6f,0xd9,0xe6,0xef,0x91,0xef,0x23, -0xcf,0xbd,0xea,0x8a,0x59,0x2f,0xb8,0x62,0x51,0x09,0x40,0x25, -0x29,0xbb,0x9c,0x3b,0xca,0xbe,0x4b,0x30,0xe6,0x32,0x43,0x18, -0x9f,0x98,0x33,0x39,0x8c,0x3b,0x86,0xb8,0xd2,0x5c,0xae,0x34, -0x29,0x44,0x51,0xb5,0x66,0xd1,0x38,0x36,0xda,0xc8,0x6a,0xa7, -0x85,0x85,0xb5,0x03,0xbd,0x1a,0xef,0x15,0x6a,0x64,0xcc,0x70, -0x28,0x5c,0x98,0xb6,0xc8,0x85,0x5d,0x6b,0x8e,0xc3,0x80,0xe5, -0x39,0xce,0x8c,0x54,0xc3,0xc7,0xf0,0x08,0x17,0x0c,0x8a,0x3c, -0x82,0xb8,0x8f,0xe3,0x51,0x01,0xd9,0xbe,0x3c,0xd2,0xfb,0x22, -0x5e,0xa1,0x46,0x9a,0x47,0x1c,0xae,0xac,0xb6,0xcd,0x23,0x01, -0x58,0xe6,0xd1,0x35,0x28,0xe9,0xbd,0x56,0xab,0x10,0x00,0xc7, -0xb2,0xa8,0x6e,0x1d,0xe5,0xf8,0x2c,0xe8,0x56,0xb0,0xf2,0xe4, -0x2c,0x3e,0x3d,0x72,0x8b,0xcc,0xc4,0xd8,0xa6,0x44,0x89,0x47, -0xbb,0xc5,0xd7,0x47,0x31,0x98,0x8a,0xf8,0x08,0x0e,0xcb,0xbd, -0x2e,0xcf,0xe2,0x86,0xe6,0x2e,0x86,0x2a,0xd8,0x65,0x73,0x96, -0x80,0x2a,0x64,0x0f,0x47,0xac,0x3f,0x69,0x85,0x94,0x54,0xcb, -0x9e,0xcb,0xa3,0x6c,0xb4,0x16,0x56,0x06,0xf7,0x5d,0xd2,0x93, -0xfb,0x6a,0x9e,0x5d,0x0d,0x43,0xde,0x68,0xef,0xa2,0xa2,0xa6, -0x05,0x69,0x23,0x30,0xc5,0x14,0x6b,0x5c,0x20,0xef,0x7c,0x7c, -0x85,0x31,0x8d,0xfa,0xa0,0x3d,0xfc,0xd3,0x18,0x1f,0xb2,0x86, -0x3f,0x27,0x60,0x3c,0x41,0x57,0x81,0xdf,0x53,0xff,0x25,0xfc, -0x0b,0x5f,0xc0,0x1f,0x6c,0x5a,0x7c,0xd6,0x1a,0xfe,0xb6,0x43, -0xfc,0x7e,0x76,0xd2,0x86,0xbf,0xb4,0x2e,0xe7,0x87,0xcf,0xdb, -0x98,0x70,0x16,0x3c,0xa3,0xf4,0xd3,0x97,0xed,0xf8,0xcc,0x7f, -0xfe,0xfc,0xf9,0x19,0xfc,0x9c,0x84,0xa7,0x90,0xf7,0x4c,0x2f, -0x8b,0x53,0xb9,0x34,0xe6,0x7e,0x13,0xaf,0xaf,0x17,0x13,0xdc, -0x77,0xb6,0x35,0x88,0x1f,0x0e,0xfc,0x62,0xd4,0x69,0x39,0x2a, -0xac,0x88,0x3b,0x09,0x8c,0x38,0xee,0x6b,0x16,0x36,0x11,0x75, -0x5a,0x01,0x55,0x11,0x67,0x61,0x43,0xf9,0xb2,0x70,0x51,0xc4, -0x49,0x31,0xe2,0xac,0x94,0xa5,0x6d,0xc4,0xa0,0x61,0x12,0x16, -0xc2,0x66,0x3a,0xae,0x06,0xd9,0x04,0x93,0x99,0x58,0x8e,0x31, -0xcb,0xc5,0x45,0x30,0x3f,0x1c,0x54,0xb1,0xf8,0x0b,0x31,0xe9, -0x57,0x2c,0x96,0x16,0x3b,0x8c,0xfc,0x8e,0xb8,0x14,0x6e,0x13, -0x6b,0x5c,0x3a,0xf9,0x33,0xc7,0xab,0x3c,0x03,0x69,0x8a,0x67, -0x55,0xbe,0xd1,0x93,0xa7,0x93,0xea,0xac,0xe1,0x41,0x64,0xcc, -0x88,0x2b,0x33,0x0f,0x6b,0x33,0xeb,0xa9,0xce,0xe1,0xe1,0x1e, -0x88,0xbe,0x7a,0xf2,0x74,0xb9,0x03,0x91,0x54,0x38,0x95,0x59, -0xc1,0x10,0xda,0x55,0x77,0x35,0x1a,0x54,0x64,0xc6,0x9d,0xb1, -0xfa,0xac,0x7a,0xb0,0xad,0xc8,0x0a,0x86,0xd7,0xce,0x52,0xad, -0x5d,0xfd,0x1d,0x6d,0x1d,0x59,0x09,0x5f,0xd8,0xf3,0xb3,0x92, -0x60,0x68,0x4b,0xaf,0xd4,0x2d,0x5b,0xa5,0x15,0x7e,0x3d,0xf8, -0xdb,0x97,0x35,0x94,0x26,0x66,0x94,0xee,0x96,0xaf,0x98,0xa8, -0x2d,0xab,0xb6,0x36,0xde,0xb6,0xac,0xb3,0x4c,0x5e,0x44,0x1f, -0x48,0xb1,0x19,0x2f,0x11,0xa1,0x6b,0xa4,0x9f,0xae,0x17,0x4b, -0xf8,0x13,0x9b,0x73,0x46,0x56,0x81,0x6e,0x83,0x21,0x1a,0x1a, -0x44,0xbe,0x60,0xaf,0x4b,0xa0,0x59,0xae,0x31,0x25,0xa4,0x48, -0xdc,0x31,0x16,0x0a,0xab,0x9e,0x67,0x88,0x4a,0xd1,0x32,0x93, -0xa5,0xf7,0x67,0xfd,0x80,0xd6,0xbd,0xfd,0x19,0x3a,0xae,0x6f, -0xbd,0x12,0xe6,0xba,0x7e,0x2a,0x8c,0xf4,0x1d,0x5d,0x95,0x21, -0x76,0xf5,0x56,0xc3,0x4f,0xa3,0x26,0xf7,0xc3,0x7d,0x56,0x3b, -0xc9,0xd4,0xa0,0x78,0xa0,0xe7,0x4a,0xb7,0x19,0xdd,0x79,0x1f, -0xc4,0xf8,0x40,0x17,0x36,0x5c,0x64,0xea,0x10,0xec,0xee,0xc8, -0x96,0xc7,0x4b,0x35,0x8a,0xdd,0xdd,0xd9,0x72,0x66,0xa9,0x46, -0xb0,0xbb,0x53,0x97,0xbc,0x75,0xfe,0xa9,0xd4,0xb1,0x95,0x68, -0x3c,0xdc,0xad,0xb5,0x14,0x59,0xb7,0xe0,0x57,0xc8,0x6f,0xb9, -0x67,0x93,0x6c,0x5a,0xbd,0x9b,0xcc,0x88,0x3f,0xd8,0xc3,0x75, -0x89,0x35,0x95,0x12,0x6f,0x2f,0xe1,0x35,0x73,0x00,0x90,0x5d, -0xaf,0xf0,0xed,0x2e,0xdd,0x7f,0x56,0xf1,0xaf,0x9b,0x38,0x5d, -0xab,0xc2,0x45,0x18,0x0f,0x5c,0x2c,0x17,0xf3,0x34,0xfe,0x11, -0x32,0x52,0x5f,0x6a,0xbd,0x4f,0xcd,0xcb,0x3c,0x37,0xab,0xa9, -0x8f,0x57,0xe2,0xe0,0xa1,0x6e,0x93,0x72,0x28,0x80,0xd2,0x1c, -0x74,0xaf,0x48,0xc6,0xb4,0xe8,0x7c,0x8c,0x59,0xc9,0x9b,0x01, -0x3f,0x74,0x36,0xa2,0xec,0x7a,0x3d,0x9b,0x3e,0x02,0x2d,0x56, -0xe4,0x18,0xf3,0x10,0x3e,0xfc,0x28,0xe0,0xcb,0x66,0xd3,0x8f, -0xaa,0xe8,0xdb,0x6f,0xbe,0xa6,0x7a,0x6e,0x5d,0xaf,0xe8,0x8b, -0xbf,0x5a,0x2c,0xc4,0x61,0xb0,0x25,0x2e,0xa4,0x2f,0x36,0xe9, -0x5f,0xe7,0x93,0x78,0x95,0x8e,0xf1,0x90,0x14,0x26,0xb6,0x2e, -0x7c,0xba,0x98,0x2a,0x5e,0xa1,0x43,0x1e,0xf9,0xdf,0xd3,0x55, -0xa8,0x11,0x7d,0x1a,0x4e,0x23,0xdf,0x8d,0xde,0x73,0x82,0xb8, -0x2e,0x54,0xa7,0x7c,0x01,0x45,0x72,0xd2,0x17,0xa2,0x70,0x23, -0x11,0xef,0xf6,0x8c,0x34,0x5a,0xba,0xeb,0xd3,0xa7,0x83,0x34, -0x66,0x2c,0x45,0xf8,0x20,0xe6,0xc0,0x75,0x33,0x9e,0x63,0x7c, -0x35,0xe1,0x96,0x44,0xe8,0x79,0xf6,0xf5,0x30,0xfd,0xee,0x6e, -0x2e,0xaf,0x2b,0xd7,0x00,0x76,0xbc,0x3f,0xa7,0x99,0xfb,0x17, -0xbc,0x17,0x6c,0x96,0x20,0xb6,0x87,0x05,0xc0,0x37,0xc3,0xa5, -0x99,0x08,0xea,0x5a,0x24,0xfc,0x10,0x4f,0x36,0x36,0xc9,0x2b, -0x8a,0xb1,0x92,0x7f,0xc0,0x73,0xe3,0x65,0x18,0x8a,0x96,0x14, -0xf0,0xb3,0x3c,0x26,0x01,0x14,0x23,0x92,0xff,0x72,0x1b,0xaf, -0x72,0x33,0x15,0x1f,0x46,0x97,0xd4,0xbf,0x59,0xcc,0x6c,0xa6, -0x41,0x58,0x24,0x7d,0x05,0x8d,0x9a,0x7d,0x77,0x69,0xa6,0x26, -0x1c,0x25,0x00,0xbe,0x1e,0xa6,0xeb,0x0a,0xa0,0xa9,0x8e,0x96, -0x98,0x52,0x4a,0x16,0xcd,0x9f,0x70,0x48,0xa4,0xfd,0xcf,0x38, -0x4f,0x65,0xf3,0xe3,0x8b,0x79,0x22,0xfa,0x33,0x28,0x2a,0x52, -0x42,0xd0,0x42,0x3f,0x57,0xff,0x42,0xf7,0x8b,0xc5,0xe8,0xbd, -0x92,0x5c,0xf8,0x7e,0x92,0xcc,0x61,0x88,0x9d,0x8f,0xf1,0x8a, -0xe9,0x8b,0x1e,0x44,0x74,0x50,0x3c,0xed,0x58,0xe1,0xda,0x74, -0xb7,0xc2,0xab,0x33,0x26,0x11,0x00,0xf9,0x7c,0xb4,0xd4,0xeb, -0xe0,0x94,0xf5,0x82,0x70,0x6e,0xbb,0xce,0x06,0x48,0xbf,0x04, -0xdd,0x32,0xd1,0x77,0x95,0xc7,0xd9,0x72,0xb1,0x5a,0xa7,0x3d, -0xb7,0x2a,0x71,0xb6,0x98,0x6c,0xa6,0x31,0x9e,0x93,0xc6,0xdf, -0x96,0x80,0x3d,0x3c,0x74,0xc5,0x57,0x64,0x27,0x44,0x17,0x78, -0x50,0x88,0x3e,0x5b,0x17,0x10,0xe8,0x70,0x87,0x89,0x2e,0xfc, -0x8b,0xd6,0x4f,0x7f,0xf9,0xe1,0xcd,0x57,0xdf,0x7d,0x1b,0x39, -0x61,0xeb,0xb4,0xd5,0x76,0xc8,0x0c,0xa1,0xdb,0xa5,0x2f,0x5a, -0xe2,0x47,0x48,0x96,0xc5,0x08,0x9f,0xd6,0xfb,0xd6,0x8b,0x15, -0xca,0x3a,0xdd,0x47,0x8f,0x2f,0x20,0xe0,0x92,0xd3,0x01,0xd6, -0x92,0x1e,0xd2,0x32,0x25,0xf5,0xf0,0x10,0x62,0x15,0x26,0x7c, -0x07,0xc5,0x48,0xf4,0x8c,0x34,0xb7,0x84,0x58,0x9d,0xbe,0x43, -0x28,0x3e,0x22,0x00,0x08,0x1a,0x3a,0xa4,0xef,0x34,0xc6,0x8b, -0x83,0x05,0x84,0x4e,0xee,0xf2,0x8f,0xb8,0x40,0x18,0xf0,0xc8, -0x12,0xf8,0x3a,0x31,0x51,0x8c,0x0f,0x19,0xf0,0xcc,0x73,0x82, -0x1f,0x1e,0x14,0x20,0x94,0x89,0xc7,0x0d,0xce,0xe7,0xea,0xd4, -0xf3,0x5c,0x28,0x3f,0x17,0xfc,0xd8,0x22,0x82,0xfb,0x46,0xc1, -0x18,0xf9,0xf8,0x92,0x31,0x17,0x14,0x3f,0xf0,0xc5,0x47,0x0d, -0x15,0xdd,0x0b,0xec,0xca,0x11,0xde,0xc5,0x3f,0xb5,0x5f,0xe1, -0xab,0x6e,0x13,0x71,0x0d,0xcd,0x66,0xba,0xa6,0x77,0xb6,0xac, -0x37,0x39,0x20,0x47,0x4f,0xa4,0x75,0x94,0xd2,0xe0,0x76,0xc2, -0x32,0x64,0x1b,0x41,0x64,0x4f,0xc4,0x95,0xdb,0xa6,0xe3,0xd2, -0x19,0x22,0x2c,0x5c,0x51,0xc2,0x77,0xc3,0x53,0xaf,0xa5,0x2b, -0xab,0xb1,0xcf,0x50,0x29,0x7c,0x3d,0x72,0x35,0x0f,0x4a,0x99, -0x70,0x4d,0x40,0xe4,0x13,0xb6,0x31,0xeb,0xa0,0xbf,0xac,0x56, -0x8b,0x55,0xe4,0xb0,0x9e,0x7a,0x82,0xfd,0x04,0x7d,0xdc,0x9f, -0x0c,0xb1,0x77,0x3f,0xb9,0x4b,0xd6,0xd7,0x4f,0xe6,0x0b,0xe8, -0x83,0xc9,0x3a,0x19,0x4e,0xf9,0xe6,0x77,0x07,0x98,0xc6,0x79, -0x49,0x96,0xa7,0x93,0x29,0xfc,0x26,0xf3,0xf7,0xf5,0xec,0x9b, -0xc5,0xb3,0x85,0xcd,0x43,0x81,0xb0,0x74,0xb3,0xd7,0x79,0xbb, -0x2b,0xc4,0x9e,0xf8,0x89,0xd7,0x4e,0x8c,0xde,0xe3,0xcd,0xd6, -0x96,0x3a,0x65,0xa6,0x0a,0x1a,0x24,0x5f,0x39,0x49,0x34,0xec, -0x13,0x51,0x1a,0x3e,0x5c,0x22,0xa8,0x00,0x22,0xc9,0xd1,0xbe, -0xc4,0x73,0x90,0x36,0xa6,0xa6,0xa7,0xb1,0xba,0x16,0xed,0x5e, -0xa7,0x22,0x85,0xce,0xf3,0xee,0xd3,0x5a,0x12,0x3b,0x62,0x8a, -0xaa,0x5b,0x8b,0x18,0x54,0xca,0xd9,0x71,0x29,0x8b,0x8c,0x67, -0x8e,0x1d,0x04,0xd8,0x92,0x07,0x22,0xe8,0xad,0xaf,0x57,0x8b, -0x3b,0x5a,0xae,0x43,0xcb,0x80,0x1a,0xd3,0x35,0x1a,0x56,0x59, -0x74,0x88,0x69,0xeb,0x5f,0x98,0x03,0x8f,0x68,0xbd,0xd5,0x7f, -0x51,0xa3,0x51,0x99,0x66,0xcb,0x31,0x11,0x76,0xf3,0x51,0xdc, -0x9f,0xd4,0x86,0x84,0xab,0xbe,0x21,0xed,0x64,0x71,0x01,0x72, -0x49,0xd7,0x41,0xc5,0xf8,0xeb,0x00,0x54,0xa4,0x54,0x8f,0x55, -0x2a,0xab,0x5b,0x56,0x57,0xdb,0xbd,0x05,0x04,0xbe,0x29,0x63, -0x8f,0xb4,0x55,0xb3,0xc9,0xf9,0x07,0x1d,0xf9,0xe5,0xef,0x2b, -0x43,0xa4,0x74,0x11,0xdd,0xa0,0x42,0x90,0xac,0xc4,0x3f,0x45, -0x9a,0x2e,0x71,0x80,0xbf,0x68,0x4d,0xe2,0xf5,0xa3,0x74,0xa7, -0x3a,0xa9,0x36,0xcf,0x1f,0x56,0x75,0xbc,0x0f,0xb3,0x9f,0x92, -0xeb,0xb9,0x5c,0x80,0xe8,0x33,0xc6,0xcd,0x13,0x52,0xf9,0x31, -0xd5,0x64,0x6a,0x5d,0x94,0x5e,0x5e,0xfd,0x33,0x74,0xfe,0x17, -0xe2,0x9d,0x45,0x1a,0x82,0xb9,0x20,0x35,0x3a,0x53,0xb0,0xa7, -0x53,0x3e,0x56,0xff,0xef,0xc9,0x8d,0xc3,0x43,0x6b,0xa0,0xe0, -0xa7,0x33,0xcc,0x61,0x80,0xf4,0xc1,0x0e,0xbd,0xad,0x58,0x20, -0xea,0x2c,0x59,0xb7,0x5f,0xa3,0x1d,0xec,0x49,0xe6,0x56,0x95, -0x83,0x04,0x91,0x5d,0x0b,0x6d,0x33,0x34,0x5f,0x45,0xad,0x31, -0x90,0x44,0xcc,0xfd,0xbd,0xa9,0x20,0xf4,0xbd,0x6f,0xba,0xe1, -0x40,0xce,0xeb,0xda,0xad,0x63,0xd8,0xd7,0xdc,0x6a,0x4c,0x81, -0x6c,0x34,0x4a,0xe8,0xa9,0xf8,0x8f,0x1f,0xb2,0xc9,0xfd,0x53, -0x90,0xc3,0x3f,0x87,0x87,0xfb,0x8e,0xdd,0x3d,0x96,0xe2,0x8e, -0xb0,0x5e,0xb0,0xcb,0x1e,0x30,0x0e,0x31,0x96,0x43,0x4f,0x42, -0x79,0xc6,0x79,0x00,0xb0,0x6e,0x9e,0xff,0x99,0xac,0x0b,0x1f, -0x60,0x1d,0xce,0x3e,0x98,0x73,0x54,0xbe,0x64,0x1c,0x46,0xf7, -0x64,0xec,0x1f,0x63,0x9b,0x20,0x05,0x88,0x15,0x34,0xed,0xcd, -0x37,0xc1,0x30,0xad,0x05,0x0c,0xbe,0xa1,0xd1,0x37,0x5f,0x0f, -0x61,0x7e,0x41,0xf6,0xcb,0x78,0xba,0x31,0x2f,0xb9,0x44,0xc2, -0xc4,0x1d,0x20,0x1f,0x4a,0xf5,0x3f,0x08,0x3b,0xd6,0xf4,0x8a, -0xab,0x2f,0x26,0x56,0x8a,0x03,0x22,0xb1,0xd7,0x0c,0xc9,0x86, -0x97,0xe9,0xf2,0x6a,0x91,0x4e,0xb5,0xfa,0x53,0xc5,0xf1,0xdb, -0x37,0x51,0xc4,0xe0,0x5b,0xea,0x1d,0xc9,0xfc,0x76,0x71,0x53, -0xa0,0x72,0x46,0x4b,0x25,0xac,0xa7,0x00,0x34,0x8d,0x68,0xce, -0xcc,0xac,0xd1,0x77,0xc4,0xb6,0x61,0x88,0x4c,0x71,0x2e,0x86, -0x95,0x4d,0xbf,0x50,0x4b,0x8d,0x9c,0xb9,0xab,0x7a,0x38,0x9a, -0xa5,0xf5,0x44,0xb9,0x9c,0xb5,0xc7,0xd9,0x3a,0x94,0xd6,0xe7, -0xc0,0xc0,0x13,0xa7,0x4f,0xb8,0x19,0x90,0x12,0x8f,0x89,0x5e, -0x4e,0x37,0xe3,0x1b,0x9b,0x66,0x18,0xe7,0x0c,0xad,0xb2,0xbb, -0x4c,0x66,0x04,0x5a,0xf4,0x03,0xc6,0x77,0x77,0x1d,0xaf,0x0a, -0x3c,0xc0,0xfb,0xd9,0x52,0xff,0x32,0x59,0xa5,0xa6,0xb6,0x4a, -0xd2,0xbf,0xd0,0x29,0x50,0x4a,0xf5,0x7a,0x94,0x2c,0xbb,0x52, -0x7f,0xd0,0xb9,0xe8,0x73,0x8c,0x83,0xc3,0x98,0xd3,0x71,0x58, -0xb9,0x39,0x83,0x4a,0x5a,0x8c,0xa9,0x0a,0xde,0xfc,0xcb,0x28, -0xf1,0x25,0x20,0xfc,0x20,0xe2,0xc0,0x42,0xd0,0x94,0x0a,0x1b, -0x46,0x75,0x9e,0x83,0x80,0x69,0xc7,0xa2,0xfe,0x56,0x43,0xbf, -0x41,0x39,0x55,0xd1,0xa8,0x19,0x8e,0xd3,0x3e,0x32,0x2a,0xdb, -0x63,0xd6,0xa8,0x14,0xef,0xe1,0xe1,0x85,0x9c,0xa8,0x93,0x8d, -0x42,0x62,0x4a,0x67,0x50,0x69,0xba,0x07,0x5f,0x2c,0xb8,0xe2, -0x0d,0xa3,0xb3,0xd3,0xd3,0x93,0x53,0xcf,0xf4,0x82,0x86,0xf2, -0x44,0xa3,0x62,0x90,0xe6,0x50,0xdd,0x72,0x01,0xcc,0x64,0x4c, -0x14,0x99,0x9b,0xe1,0x71,0x60,0xaa,0x90,0x0f,0x63,0xf1,0x12, -0x5d,0x07,0x53,0xb8,0x9f,0xd2,0xe7,0x56,0xca,0xdd,0x3e,0xba, -0x80,0xae,0x34,0x12,0x88,0x94,0x06,0xe8,0xed,0xa9,0x0a,0x58, -0x54,0xbb,0x32,0xff,0x39,0x53,0xd6,0x92,0x61,0xba,0xf2,0x8c, -0x69,0x65,0xea,0x18,0x81,0xa2,0x5b,0x7e,0x6c,0xf5,0xf8,0xc9, -0x27,0x02,0xa9,0x55,0x92,0xf9,0x7f,0x6d,0xab,0x24,0xf3,0x8f, -0x69,0x95,0x27,0xb5,0xad,0xa2,0x1b,0xe5,0xbf,0xb3,0x4d,0x5e, -0xfd,0x99,0x6d,0x92,0x5e,0x6f,0x2e,0x2f,0xa7,0x71,0x61,0xb1, -0x49,0xdc,0x43,0x3a,0x61,0x2a,0xa2,0xc0,0x17,0x60,0x13,0xc3, -0xb0,0xab,0xa9,0xb6,0x87,0x1b,0x46,0x68,0xec,0xe2,0xcf,0x62, -0xe6,0x12,0x86,0x46,0xc3,0x53,0x28,0xd8,0xb4,0x6e,0x86,0x83, -0x48,0xc5,0x20,0xe8,0xc0,0xb7,0x83,0xf2,0x29,0x32,0x15,0x4d, -0xd4,0x0e,0x67,0xcb,0x02,0xb1,0xfe,0xdc,0xbf,0xda,0x0c,0x57, -0x93,0xda,0xeb,0x28,0x5e,0xb5,0xef,0xef,0x09,0x02,0x07,0xd9, -0xbe,0xa2,0x4b,0x4b,0x4e,0x33,0xf4,0x40,0xc7,0x49,0x4e,0x10, -0x03,0xc4,0x45,0x45,0x81,0xbe,0xa3,0x3f,0xf0,0xe7,0x9e,0xb0, -0x21,0xa6,0xe4,0x3b,0xf6,0x95,0x34,0x07,0x6a,0x94,0xb1,0x35, -0x7a,0x70,0x52,0x8f,0x9b,0xa7,0x6e,0x5d,0x4f,0x3c,0xea,0xbb, -0xc5,0x11,0x37,0x5d,0xac,0xd6,0x9f,0x15,0xac,0x13,0xd9,0xac, -0xe6,0x4c,0x53,0x12,0x61,0xd3,0xe4,0xda,0x4f,0x76,0x8a,0x91, -0xc5,0xad,0x1b,0x40,0x2a,0x2c,0x09,0x4b,0x92,0x28,0xb9,0xc3, -0x40,0xe3,0x15,0x96,0x99,0x0c,0x3b,0xfb,0x1a,0xad,0x78,0xf1, -0x2f,0x54,0x46,0xaf,0xc5,0xe3,0xed,0xd4,0x7c,0x67,0xb5,0x18, -0x8a,0x23,0x8c,0x69,0x49,0xc4,0xfe,0x28,0xa2,0x44,0x15,0x41, -0xcf,0xc8,0xe1,0xe3,0xaa,0xa4,0x1d,0x86,0xe7,0xa3,0xfb,0x7b, -0x1e,0x9a,0x40,0x0f,0x0c,0x55,0x97,0x45,0xa8,0xd1,0xf9,0xd0, -0x48,0x1b,0x29,0x25,0x2b,0xdf,0xc4,0xa2,0x72,0x58,0xfa,0xb8, -0x08,0xfa,0x06,0x11,0x73,0x78,0xbd,0x46,0xb4,0x2f,0xbf,0x09, -0xa6,0xe8,0x1d,0xc5,0xd7,0xc3,0xdb,0x04,0x26,0x75,0xb2,0xa9, -0x1e,0x6a,0x14,0xd9,0x19,0xb7,0x4a,0xbf,0x89,0xd7,0xa5,0xf8, -0x3d,0x40,0x6d,0x3e,0x76,0x76,0xb6,0xda,0x4e,0xb5,0xa2,0xe6, -0xd6,0x7b,0x98,0x78,0xa4,0xf7,0x64,0x2d,0x84,0xaa,0x40,0xc3, -0xc2,0x2f,0x4c,0x74,0x48,0xf0,0xa8,0xf2,0x20,0x79,0xf4,0x6b, -0x6e,0xa0,0x14,0x72,0x7d,0x00,0x69,0xba,0x1e,0xa6,0x46,0x82, -0x27,0xac,0x5d,0x1a,0xd6,0x3b,0xc6,0x37,0xae,0x70,0x14,0x66, -0x56,0x17,0xcc,0xfa,0x7d,0xca,0x31,0x11,0x49,0x95,0x20,0x9e, -0x04,0xdd,0x95,0xdd,0xfb,0xb0,0x93,0xbe,0x46,0xc3,0xa2,0x30, -0x24,0xa4,0x28,0xa7,0xf1,0x84,0xac,0x51,0xdd,0xfc,0xb4,0xc0, -0xe7,0xef,0x9e,0x1d,0x88,0xf6,0x55,0x7a,0xbd,0xbe,0x89,0xf5, -0x32,0x8a,0xb4,0x93,0xd8,0x7a,0xad,0x5d,0xa0,0xf5,0xfc,0xe9, -0xe2,0x0e,0x14,0x30,0x5e,0xa4,0x1e,0x11,0x2d,0x72,0xc5,0x05, -0x63,0xce,0x21,0xb1,0x2b,0xef,0x09,0x9d,0x80,0x16,0xb8,0x6b, -0x50,0xf4,0xf9,0x39,0x74,0x88,0x4a,0x94,0x84,0xa2,0x0f,0xc0, -0x03,0xef,0x15,0xcb,0x23,0xe2,0x87,0x70,0x23,0xec,0x50,0x19, -0xf0,0xa9,0xfa,0xcb,0xe2,0x0e,0x95,0xee,0x7a,0xc1,0x1b,0x1a, -0x75,0x7a,0xab,0x67,0x0f,0xd3,0x3d,0xc3,0xac,0xc6,0x70,0xa7, -0x6e,0xf9,0xbc,0xa7,0xf5,0x91,0x31,0xa5,0xea,0x88,0x2b,0x04, -0x78,0xb5,0x08,0xcc,0x4f,0xd2,0xfb,0xd6,0x6b,0x00,0x66,0xf9, -0x7a,0xae,0x11,0xd4,0x17,0xa4,0x3f,0x3b,0xc6,0x4a,0x94,0x5c, -0x7d,0x22,0x6b,0x13,0xec,0x5b,0x18,0xb3,0xae,0xe3,0x21,0x0e, -0x5d,0xeb,0xa1,0x39,0x77,0x60,0x09,0x28,0x0e,0x33,0x5c,0x30, -0xa5,0x49,0x53,0x99,0xa3,0xe6,0x72,0xb8,0x61,0x56,0x07,0x83, -0x8e,0x35,0xcf,0x40,0x5c,0x38,0x98,0xf0,0x2c,0x85,0x57,0x97, -0x1e,0x28,0xaa,0x22,0xbf,0x29,0x09,0x4d,0xb7,0x50,0x30,0x4c, -0xba,0x3c,0xc2,0x8f,0x9b,0x53,0x7f,0x56,0x3d,0xac,0x12,0xc3, -0x8a,0x4a,0xa9,0xa1,0xd2,0x82,0x9c,0xfb,0x81,0x27,0x16,0x50, -0x88,0xc3,0x30,0x8f,0xc4,0x65,0xf0,0xc9,0x6a,0xb1,0x7c,0x74, -0xb5,0x2b,0xaa,0xb9,0x25,0x5d,0x30,0x5b,0x0e,0xc7,0xc5,0x8a, -0x96,0x17,0x63,0x18,0x89,0x21,0x6a,0xac,0xee,0x2f,0xa7,0x30, -0x75,0x30,0xdf,0x98,0x4e,0xe6,0x60,0x2e,0x81,0xd9,0x01,0x05, -0x2f,0xee,0xf0,0x9d,0x0c,0x08,0x6a,0xa2,0x38,0x1a,0x4d,0x47, -0xb1,0xce,0x41,0xd0,0xaa,0x0b,0x78,0x3d,0xde,0x71,0x15,0x86, -0x27,0xe7,0xf6,0x09,0x48,0x4e,0xe5,0x39,0x47,0xd1,0x6c,0xd0, -0x9d,0x88,0x23,0xee,0xef,0x39,0x46,0x18,0x33,0xd2,0x84,0x10, -0xe5,0xf7,0x50,0x9f,0xda,0x65,0x30,0x40,0x47,0x54,0x47,0x0c, -0x16,0x85,0x5a,0x74,0xf8,0xb7,0xa0,0x8d,0x45,0x22,0xaa,0xff, -0x12,0x33,0x98,0x69,0x02,0x8d,0x1e,0x02,0x45,0x21,0x56,0xaa, -0xdf,0x1f,0xf0,0xb4,0x33,0x81,0x69,0xee,0x66,0x47,0x7b,0x4c, -0x92,0xcb,0x4b,0x98,0xb6,0xcd,0xc1,0xc2,0x12,0x08,0xaa,0x26, -0xe2,0x21,0xcb,0xcd,0x66,0x9e,0xfc,0x1a,0xf1,0xcf,0xa6,0xd4, -0x29,0x93,0xf4,0x0d,0xe9,0xec,0x0a,0xdd,0x6c,0xd9,0x5f,0x12, -0x0e,0x2f,0xa7,0x12,0x10,0x4a,0xe5,0xea,0x51,0x5a,0x63,0x13, -0x1f,0xd1,0x41,0xe8,0x75,0xcd,0x15,0x7d,0x43,0xc3,0xa3,0xea, -0x12,0x44,0x94,0x96,0x6c,0x38,0x5e,0xaf,0x85,0xfa,0x29,0x3e, -0x62,0x5e,0x30,0x9d,0x05,0xce,0x9a,0xe1,0x5d,0x91,0xdc,0xa3, -0x88,0xc3,0x43,0x44,0xd1,0xc7,0x3f,0xba,0x0f,0x46,0xc2,0xa4, -0xe8,0xe8,0x15,0x1a,0x17,0x21,0x84,0x28,0xe0,0x9d,0x4e,0x94, -0xc1,0x68,0x6d,0xdf,0x5a,0xe9,0xe4,0x9e,0xcd,0x4b,0xdd,0xe6, -0xc6,0x97,0xe0,0xbb,0xfd,0x7e,0xab,0x6a,0x3e,0x6c,0x0a,0x57, -0x89,0x8a,0xd1,0x62,0x07,0xa2,0xab,0x93,0x0f,0x70,0xca,0xcf, -0xd8,0x96,0xa4,0x40,0x58,0x48,0xeb,0xea,0xd5,0x97,0xd0,0x30, -0x5a,0x45,0xb7,0x15,0x05,0x72,0x76,0xcd,0x2d,0xe0,0xfa,0xcc, -0x20,0x8a,0x3b,0x23,0x22,0xd6,0x20,0x8b,0xf5,0x75,0xbc,0x32, -0x0d,0x72,0xb1,0xae,0x44,0xf1,0x3e,0x21,0x38,0x8f,0x82,0xad, -0x58,0x7d,0xd1,0x52,0x59,0x4f,0xb3,0xd5,0xb7,0xb5,0x17,0x40, -0xad,0x00,0x97,0x2a,0xc3,0x92,0x51,0x3d,0x65,0x38,0x30,0x9a, -0x91,0x2a,0x22,0x3b,0x28,0x52,0xf7,0x5b,0xb2,0xac,0x7a,0x4c, -0x57,0x0c,0x77,0xb4,0xe2,0xe1,0x4a,0x3b,0x5f,0x93,0xe0,0x70, -0xba,0xe3,0x09,0x3f,0x10,0x37,0xf0,0x54,0x23,0xd3,0x59,0x0e, -0xd6,0x38,0x62,0xbf,0x06,0x77,0x93,0xed,0xad,0x63,0x01,0xda, -0x4f,0x06,0x51,0x05,0x72,0xa7,0x91,0xa8,0x1a,0x1a,0x92,0xb3, -0x18,0xd9,0x4b,0xe5,0x38,0x0f,0xe0,0xba,0xa4,0x6a,0xb3,0x3e, -0x8a,0x68,0x76,0x20,0x26,0x1b,0x5b,0x65,0x0d,0x19,0xb6,0xb3, -0xde,0xd9,0x46,0xd0,0xca,0x9d,0x6d,0x46,0x2a,0xed,0x3a,0x04, -0xc3,0xad,0x6c,0xee,0x14,0x48,0x74,0xc7,0x4e,0x81,0xb1,0x78, -0x10,0xc9,0xef,0x70,0x60,0x93,0xbe,0xf5,0x95,0x80,0x94,0xb4, -0x0c,0x48,0x8a,0xd2,0x09,0x46,0x15,0x58,0x3a,0xe4,0x2c,0xa3, -0x5b,0x70,0x04,0xb0,0x8c,0x35,0xdc,0x83,0x37,0x11,0x38,0x7c, -0x3e,0x49,0x3b,0x56,0xa8,0x54,0xb9,0x91,0x13,0x59,0xf6,0xa8, -0x41,0x89,0x27,0x6c,0xb8,0x04,0xfb,0x3f,0x46,0xf4,0x92,0x0e, -0x94,0x0e,0x25,0x9f,0x2b,0x9d,0x61,0x4c,0x5b,0x99,0x80,0x86, -0xc2,0xdf,0x91,0x5f,0x5b,0xe5,0x2a,0xa1,0x16,0x67,0x99,0xe4, -0x9a,0xe5,0x59,0x4f,0xcd,0xaf,0x0d,0x20,0xd7,0xe6,0x0d,0xb5, -0x63,0xc9,0xfb,0xa0,0x40,0xaf,0xaa,0x62,0x57,0x4d,0xd0,0x84, -0xad,0xf2,0xd5,0x0e,0xfe,0x5f,0xae,0x16,0xb3,0xdd,0xbc,0x07, -0x73,0x9f,0x6d,0x77,0xf6,0x06,0xc1,0x0c,0x5d,0x55,0x47,0xc3, -0x7d,0x47,0xd6,0xd3,0x2c,0x51,0xd6,0xd5,0x00,0x93,0x64,0x4a, -0xb4,0xbd,0x52,0x2e,0x57,0x13,0xd6,0xa9,0x4e,0xd4,0x66,0x7e, -0x12,0x29,0x3c,0x98,0xa1,0x63,0x4b,0x47,0xb3,0xd9,0xdd,0x9b, -0x51,0xfc,0x10,0x40,0xbd,0x3f,0x6e,0x69,0xfd,0x23,0xc2,0xbb, -0x04,0x11,0x20,0x22,0xd8,0xfb,0xfb,0xc0,0xa7,0x8f,0x28,0xf0, -0x28,0x8b,0xde,0xa7,0xee,0xb7,0x07,0xf7,0xf7,0x61,0xb7,0xa0, -0x5c,0x94,0x30,0xd1,0xc7,0x38,0x4e,0xa6,0x2e,0xa1,0x6b,0xf2, -0xf3,0x4b,0xc7,0x54,0x2c,0xbe,0x4e,0x98,0x4c,0x70,0xa9,0x88, -0xe9,0x2b,0x29,0x17,0x25,0x15,0x93,0xac,0xeb,0xf1,0x13,0x05, -0xf0,0xd9,0x68,0x0c,0x22,0x59,0x03,0xf8,0xdb,0x88,0x10,0x99, -0xea,0x98,0x08,0xc5,0x46,0xda,0xd8,0x5a,0x69,0xf1,0x3e,0xa0, -0xad,0x82,0x3b,0xdc,0x3a,0x0e,0x3f,0xec,0x79,0x38,0xae,0xac, -0xfb,0x23,0x7c,0x72,0x46,0x8b,0x01,0x7a,0x5a,0x1d,0x1e,0x22, -0x2c,0x67,0x97,0xed,0x8e,0xf1,0x92,0xdd,0x3a,0x46,0x28,0x7a, -0x42,0x5d,0xab,0xe2,0x71,0x45,0xd1,0x32,0x36,0x10,0xdc,0xab, -0xda,0x22,0xee,0xea,0x15,0xaa,0xfa,0xcd,0x07,0x22,0xd8,0xac, -0x2a,0xe2,0x77,0x8b,0x1e,0x5e,0x04,0xa5,0x16,0x2c,0xa9,0x3e, -0x4c,0xab,0x9e,0xe7,0x5d,0xa5,0x52,0xe9,0x57,0x15,0xe5,0x01, -0xe5,0xc8,0x54,0xe3,0x32,0x3f,0xc2,0xa2,0x82,0xc4,0xf6,0x34, -0x9e,0x5e,0x52,0x53,0x22,0x68,0x11,0x9e,0x9e,0x18,0x34,0x94, -0xb6,0x41,0x05,0xe6,0xdb,0x8b,0x04,0x51,0x01,0x76,0x89,0x13, -0x93,0x75,0x74,0x3f,0xe2,0x2f,0xb9,0x7f,0x86,0xe8,0xb6,0xb4, -0x43,0x02,0x32,0x62,0x4d,0x98,0x88,0xd7,0x0f,0xec,0xe8,0x68, -0x9b,0xa2,0x6c,0xcd,0x18,0x34,0xcb,0x0b,0xae,0x1f,0xa2,0x99, -0x08,0x41,0xe1,0xf9,0xb4,0xb0,0xf3,0xca,0x74,0x60,0x4c,0x3d, -0x21,0xd0,0x98,0xb8,0x36,0x45,0x40,0xd2,0x8b,0x4c,0x4b,0x0a, -0x3b,0x12,0x38,0x02,0xf9,0x93,0xd9,0x26,0x5d,0x3f,0x19,0xc5, -0x4f,0x96,0x74,0x4f,0xbf,0xa2,0x1e,0x04,0x74,0x16,0xa7,0x8e, -0xbd,0x50,0x44,0x18,0xb5,0x61,0x71,0xe9,0x7d,0xc0,0x15,0xc5, -0xcb,0x81,0xf4,0x04,0xe1,0x10,0xad,0x28,0xa0,0xc5,0x3f,0x7a, -0x4f,0x0b,0xf2,0xf1,0x6c,0x61,0xcd,0xac,0x49,0xd2,0x41,0x51, -0x91,0xdd,0x44,0xab,0x0a,0xe8,0x0f,0xf1,0x41,0x2d,0x75,0x73, -0x12,0xd8,0x95,0xfc,0x61,0x6e,0x85,0x16,0xfd,0x66,0x71,0x6d, -0x8a,0xa1,0x0a,0xfc,0x15,0x8c,0xd4,0xa6,0x11,0x2e,0xd3,0x90, -0x53,0x06,0x2d,0xd2,0xe0,0x17,0x2f,0x21,0xa9,0xaf,0xa8,0xa2, -0x95,0xd4,0x85,0x6d,0x68,0xb9,0xe1,0x5b,0xfa,0x85,0x4a,0xdc, -0x0d,0x93,0xf5,0xc3,0x5b,0x7d,0x92,0x08,0x71,0xdc,0x0e,0x66, -0x2d,0xee,0x4e,0x19,0xa1,0x1b,0xb2,0x79,0xc7,0x8e,0x4b,0xe0, -0xe2,0xc1,0x6e,0x2c,0x8a,0xa4,0x9e,0xec,0x20,0x71,0x22,0xfb, -0x85,0xdf,0x27,0xe2,0xc2,0xc1,0x2e,0x19,0x43,0xad,0x42,0x88, -0x51,0x30,0xd6,0xeb,0x69,0xb1,0x7d,0xb0,0x60,0x7f,0xb1,0xc4, -0x88,0x54,0xee,0x2c,0xe8,0x4e,0x2f,0xcc,0x3b,0x7f,0xcd,0xf5, -0xe1,0x4b,0xbd,0xa5,0xeb,0x32,0xd8,0x77,0x22,0x23,0xb4,0xa1, -0xf8,0x82,0xe6,0x15,0x4e,0x3d,0xc3,0x75,0x6c,0x29,0x59,0x95, -0x4b,0x40,0x82,0xc0,0x0e,0xe9,0x29,0xc4,0x08,0x26,0x46,0xbd, -0xa0,0x23,0x0f,0x03,0xdb,0x65,0x95,0xb5,0x81,0x49,0x9e,0xb7, -0xad,0xe8,0x8c,0xf4,0x7a,0xe5,0xe2,0x4e,0x9d,0x2e,0xee,0xca, -0x92,0xef,0xef,0x0b,0x45,0x1f,0x60,0xd1,0x40,0xbb,0x22,0x0d, -0xb2,0xc9,0x5d,0x78,0xbc,0x8c,0x16,0xa9,0x43,0xfe,0x34,0x5d, -0x48,0x68,0x4a,0x28,0xd5,0xce,0x72,0xee,0xa7,0xfa,0xba,0xf1, -0x38,0x68,0x70,0x1e,0x29,0x1c,0x3d,0x97,0x2e,0xcd,0x95,0x32, -0x21,0xea,0xe7,0xd5,0x30,0x15,0x8a,0x7a,0xa8,0xd6,0xf8,0x6a, -0x24,0x65,0xd5,0x35,0x5a,0xaf,0x86,0xc9,0x54,0x72,0x13,0xaa, -0x24,0x71,0x1b,0xb2,0x48,0x2d,0xe2,0x2b,0xaa,0x3c,0xbd,0x52, -0x4b,0x62,0x87,0x83,0xc0,0xb8,0x52,0x3a,0x92,0xd9,0x2c,0x9e, -0x24,0xd6,0x89,0x56,0xc0,0xc7,0xf2,0x21,0xe9,0xc2,0x48,0x18, -0x4f,0x66,0x4b,0xdf,0x76,0x36,0x32,0xda,0x65,0x17,0xbb,0x54, -0x76,0xdd,0x6a,0x95,0x42,0xc4,0x91,0x30,0x77,0x92,0x60,0x4d, -0x95,0xb3,0x8b,0xb4,0x9e,0x63,0x62,0xaf,0xb6,0xf2,0xd4,0x98, -0x08,0xe2,0x75,0x5c,0x8b,0xfb,0xaa,0x8a,0xda,0xdb,0xa1,0x96, -0xf9,0xe8,0x3a,0x03,0x5d,0xec,0x5b,0x10,0x31,0x95,0xed,0xf0, -0xf0,0x40,0xe0,0x93,0x15,0x57,0x2d,0x54,0xdf,0x14,0xd4,0xe7, -0x3d,0x89,0x4c,0x6f,0xbc,0xd5,0x96,0x6c,0x36,0xd8,0xa2,0xd4, -0x58,0x5a,0xc7,0xa2,0xd5,0x64,0x78,0x90,0x94,0x95,0x10,0xa4, -0x93,0x6a,0xec,0xb8,0x04,0x19,0x90,0x1b,0xdb,0x0e,0xc5,0x48, -0xea,0x98,0x39,0x45,0x0e,0x7d,0x44,0x01,0xba,0x83,0x17,0xc5, -0x85,0x3c,0xc4,0x2b,0x36,0x37,0x0c,0xe5,0x49,0x4a,0x4b,0x4d, -0x97,0x8c,0x95,0x27,0x92,0x27,0xa3,0x4c,0x81,0xac,0x38,0xa0, -0x72,0xe1,0xb8,0x4c,0xb7,0x48,0xe3,0xa2,0x6c,0xf0,0x78,0xa9, -0x90,0x54,0x70,0x40,0x9a,0xa3,0x05,0x01,0x4c,0xac,0x51,0x14, -0x67,0x40,0x30,0x99,0x47,0x53,0xda,0xd3,0x44,0xe3,0x24,0xb0, -0x44,0xcd,0xc0,0xb4,0xc3,0x60,0x4e,0x48,0xd4,0x0d,0x2f,0x2d, -0xb9,0x25,0x19,0xf5,0x2d,0x55,0x5e,0x6a,0x93,0x26,0x4b,0xf2, -0xab,0xb0,0x57,0xdf,0x0c,0xd2,0xa1,0x06,0x8b,0x20,0x4f,0x48, -0x7d,0x0e,0xe0,0xfe,0xde,0x36,0x1e,0xd8,0xd1,0x1c,0x14,0x9c, -0x30,0x86,0x68,0x47,0xba,0xca,0xd1,0xd0,0xf9,0x6a,0x0e,0x33, -0x5c,0x40,0xcb,0x13,0x6d,0xf1,0xe8,0x27,0x61,0xef,0x0f,0xba, -0x05,0xff,0x0b,0xc4,0xc2,0x43,0xac,0x74,0x27,0x39,0x3c,0x24, -0xd7,0x4b,0x5a,0x00,0xc2,0xb0,0xe4,0x06,0xc6,0x22,0x99,0xb7, -0x85,0x57,0x90,0x89,0xb8,0x5a,0x07,0xf4,0xb2,0x37,0xa7,0x98, -0xea,0xef,0xb3,0xbc,0xa0,0x26,0xea,0x91,0xe9,0x8a,0xde,0x35, -0xdd,0x59,0x52,0x36,0xfb,0x92,0xd5,0x1f,0xa1,0x88,0xf3,0xef, -0x41,0x10,0x01,0x22,0x3d,0x7d,0xc3,0x1d,0x5e,0x11,0xa6,0xbb, -0x00,0x82,0x09,0x37,0xa3,0x78,0xb5,0xae,0xa1,0x4c,0xaf,0x65, -0x7c,0x94,0xdb,0xbe,0x58,0xbb,0xb0,0x08,0x88,0xc4,0x57,0x79, -0xe9,0x42,0x92,0x80,0xe5,0xcc,0xc4,0x29,0xca,0xb2,0x71,0x4a, -0x96,0x63,0xad,0x94,0x18,0x73,0x18,0x51,0x2a,0xde,0xe6,0x4f, -0x79,0xca,0xe2,0xc2,0xd1,0xb4,0xb5,0xcb,0xfe,0x88,0x78,0x59, -0xfb,0xa4,0x7a,0x93,0x86,0x2c,0xd4,0x1a,0x53,0x47,0xdb,0x8d, -0xe9,0x62,0xb3,0x1a,0xc7,0xd4,0x11,0xc4,0xa7,0xa4,0x12,0xe6, -0x1c,0x4b,0x24,0x53,0x44,0x23,0x71,0x18,0x05,0x53,0x47,0x8a, -0xe0,0x80,0x36,0x6b,0x97,0x49,0xc1,0x8d,0x4a,0x5a,0x49,0xcb, -0x5c,0xb5,0xc6,0xc7,0xae,0xe4,0x51,0x55,0xe8,0xe8,0x8e,0x2a, -0x80,0x76,0x1b,0x35,0x23,0x69,0x89,0x79,0x99,0xb3,0xe5,0xaa, -0xf8,0x08,0xd4,0x61,0x2c,0x8d,0x03,0xb3,0xa4,0x28,0x32,0x7f, -0x9c,0xbc,0xca,0x06,0x55,0xcb,0x89,0x44,0x30,0xd2,0x89,0xcf, -0x37,0x96,0x89,0xd3,0xe6,0x11,0x53,0x38,0xe1,0xc7,0x27,0x8b, -0x12,0xf4,0x7f,0xa6,0x35,0xd5,0x86,0xbd,0x6a,0x56,0x76,0x8c, -0xaf,0x68,0x62,0x63,0xea,0x32,0x9e,0xe2,0x0d,0x1d,0x95,0xf4, -0xa1,0x20,0x1b,0x3a,0xb4,0xb8,0x3d,0x49,0xae,0x96,0xfc,0x0e, -0x55,0x47,0x8a,0xad,0x0b,0x7c,0x97,0x7b,0x95,0x64,0x79,0x9b, -0xe3,0x24,0xed,0xfd,0xe2,0x92,0xf6,0x38,0x5e,0xae,0x0d,0x3f, -0x00,0x23,0x8e,0x7b,0x35,0x66,0xe6,0x63,0x4c,0xbf,0x1a,0x4b, -0x58,0xfe,0xc8,0x1f,0xbe,0x59,0x0f,0xc7,0x37,0xfe,0x88,0x7e, -0xd8,0x8d,0xc1,0x70,0x50,0x78,0x12,0x80,0xd2,0x1f,0xde,0xdf, -0x87,0xc7,0x10,0x1b,0x1e,0x8f,0xf4,0x21,0x01,0x88,0xe4,0x0f, -0x05,0x4a,0xf9,0xba,0x43,0xeb,0xb0,0x17,0xb0,0x6b,0x18,0x0d, -0xd5,0x61,0x2f,0xcf,0x1f,0x15,0x93,0x47,0xd1,0x48,0x27,0xf3, -0x7a,0x0a,0xbe,0xfd,0xf6,0x2d,0x74,0xe1,0x48,0x9e,0x03,0x14, -0x6d,0x49,0x53,0x54,0x95,0x7a,0x50,0x48,0x1e,0x79,0xda,0x5f, -0x2f,0xbd,0x4b,0xf0,0x81,0x2b,0x05,0x0b,0xe6,0xe2,0x30,0x8d, -0x9d,0x3e,0x8f,0x4a,0x4f,0x38,0xdb,0xc0,0xe9,0x68,0xca,0x39, -0x0a,0x90,0x74,0x2d,0x48,0xbe,0x54,0xca,0x80,0x3c,0x88,0x1a, -0xc3,0xde,0x08,0xfe,0x8e,0x3a,0x20,0x17,0xc3,0x9e,0x64,0x4c, -0x07,0x7e,0x1a,0x23,0x3b,0x33,0x5a,0x94,0x90,0xd5,0x8a,0xfb, -0x6c,0xb1,0x00,0xeb,0x7d,0xae,0x30,0x36,0x2a,0xf2,0xfd,0x10, -0x5f,0xfd,0x25,0x5b,0x1a,0x85,0xb6,0x58,0xd0,0x80,0xbd,0xe2, -0xeb,0xf0,0x70,0xd8,0xba,0x9a,0x2e,0x46,0x43,0xe4,0xbf,0xf8, -0xc2,0xb8,0x19,0xf4,0x0c,0x34,0xdc,0x09,0x54,0x05,0x30,0x25, -0xb9,0x9a,0x2f,0x56,0x31,0xde,0x83,0x8b,0x49,0x3a,0x84,0x2b, -0xad,0x8e,0x18,0xac,0xd5,0x82,0x2f,0xb4,0x6d,0x29,0x6e,0xa4, -0xb9,0x5b,0x58,0x83,0x63,0x09,0xb2,0xc7,0x09,0xb9,0x68,0x48, -0x49,0x7d,0x71,0x14,0xc2,0xf0,0x89,0x19,0x15,0x12,0x46,0xec, -0x01,0xfd,0x1a,0x97,0xd2,0x86,0xa8,0x1b,0xd2,0xf5,0x6a,0x83, -0x6b,0x3a,0xfe,0x88,0xe2,0x46,0x66,0x1c,0x79,0xdd,0x60,0x34, -0x7a,0xde,0xbc,0x26,0x67,0xb9,0x03,0xd7,0x1a,0x22,0x28,0x15, -0xc6,0x07,0xfa,0x35,0x25,0x6e,0xf8,0x5a,0xf9,0xd6,0x29,0xe0, -0x91,0x00,0x1e,0x15,0x81,0x39,0x41,0xd7,0x5b,0x54,0x94,0xb7, -0xa7,0x3c,0xd1,0x65,0x38,0x38,0x62,0xd1,0x25,0x5f,0x80,0xc0, -0xd7,0x0e,0xeb,0xc8,0x5d,0xd9,0xac,0xd4,0xcf,0x07,0x4e,0x14, -0x19,0x62,0x89,0x1a,0x08,0xf3,0x0c,0xa5,0x39,0x20,0xb2,0x52, -0x24,0xd4,0x5a,0x58,0x03,0xb4,0x74,0x8d,0x71,0xcd,0xa6,0x36, -0xec,0xe3,0x5f,0xdd,0x61,0x1f,0x23,0x07,0xfe,0x48,0xfc,0xda, -0x9d,0xd9,0xeb,0x7a,0x5d,0x3a,0xa3,0x57,0x72,0x7c,0xc5,0xb6, -0x61,0xcb,0x6b,0x28,0xec,0x2e,0x22,0xa3,0xd1,0xf0,0x0f,0x24, -0x76,0x4e,0x1e,0x89,0x64,0x2a,0x0b,0xf5,0x31,0x14,0x45,0x3f, -0x85,0x92,0x3c,0x8f,0xbc,0xb4,0xbb,0xfc,0x1e,0x20,0x2e,0x94, -0x51,0x99,0xa2,0xbc,0x91,0x2e,0x4f,0x22,0x3c,0xe0,0xda,0x88, -0x6c,0x92,0x61,0x18,0xb9,0x95,0x2e,0x1e,0x92,0xdd,0x8b,0xa5, -0xab,0x99,0x4d,0x01,0x61,0x5a,0x74,0xc9,0x45,0xf2,0xd7,0x8d, -0xe5,0xa5,0xe0,0x8f,0xf4,0x60,0xf0,0x2b,0x69,0xb8,0xfe,0x40, -0xee,0xfe,0x0a,0x87,0xca,0xa8,0x64,0xdc,0x2a,0x4f,0x0d,0xd9, -0xd8,0xd4,0x72,0xb6,0x72,0xe6,0x4d,0x6f,0xa1,0x27,0x4c,0x97, -0x4c,0x31,0x3e,0xc8,0x0e,0x50,0x31,0xda,0xa9,0xda,0x4b,0x3b, -0xb7,0xc2,0xb3,0x98,0xc9,0x9b,0xc6,0x33,0xeb,0x8d,0x28,0x63, -0xec,0x38,0x70,0x0f,0x20,0x04,0xaa,0xf8,0x80,0x8b,0x9b,0x2f, -0x26,0x31,0x1a,0xde,0xa2,0x66,0xec,0x07,0x63,0x1d,0xf3,0x2d, -0x5a,0xf1,0x8c,0xa7,0x2c,0x8f,0xb6,0x1e,0xa5,0x65,0x38,0x5f, -0x0f,0x54,0xb5,0xbe,0x35,0x91,0x35,0x1d,0xd8,0xfa,0x34,0xf6, -0xf6,0x1d,0xe5,0x16,0x80,0x57,0xdd,0x8b,0xac,0x78,0x97,0x3d, -0x15,0x81,0x17,0xd9,0x6f,0xe4,0xad,0xf9,0xa0,0x1e,0xe1,0x87, -0xb5,0x9d,0x33,0xd0,0xc3,0xf3,0x9c,0xba,0xc6,0x45,0xdf,0x49, -0x52,0xa7,0x81,0x81,0x41,0x35,0x0d,0x65,0xba,0xa3,0x48,0xd5, -0x8e,0x73,0x36,0x9c,0x81,0xb3,0x65,0xa7,0x2e,0xc3,0x5f,0x41, -0xcf,0x81,0xc0,0xe6,0xb0,0x92,0x76,0x72,0xfe,0x40,0x37,0xa2, -0x83,0x45,0xc6,0xb1,0x43,0x3b,0xd3,0x8e,0xcc,0xa4,0xb4,0xe5, -0x71,0xeb,0x18,0xba,0x94,0xa9,0x65,0x2a,0x11,0xeb,0x8c,0x91, -0x54,0xb3,0x38,0x4e,0x33,0xb9,0x5f,0xe0,0x26,0x7c,0x8d,0x19, -0x21,0x53,0x85,0xef,0xf1,0x01,0x5f,0x71,0xc7,0x57,0x14,0x4e, -0x17,0x43,0x31,0x41,0xe3,0x56,0x84,0x94,0x7a,0x5b,0x84,0x1b, -0x43,0xbb,0x30,0x1f,0x90,0x6f,0x12,0x67,0x14,0x43,0xd5,0xae, -0xf6,0x3f,0x08,0xee,0xef,0xc5,0x57,0x08,0x63,0x46,0x69,0x90, -0xab,0x97,0xad,0x6f,0x37,0xa5,0x15,0x6a,0xcb,0xe9,0x27,0x52, -0x64,0xfc,0x55,0x9e,0xf4,0xae,0x06,0x37,0x4d,0x34,0xcc,0x00, -0x2d,0x54,0x7f,0x4c,0xc1,0xbe,0x37,0x40,0xd1,0x44,0x30,0x98, -0x79,0xbe,0x78,0xbd,0x98,0x5f,0x82,0x01,0xb6,0xae,0xf0,0x2e, -0x10,0xe7,0xc3,0xcb,0x57,0x2d,0xf8,0x38,0xb1,0xde,0x1a,0xde, -0x3b,0x75,0x7e,0xb7,0xca,0xb3,0x99,0x26,0xe7,0x1a,0x6a,0x5e, -0xe1,0x23,0xa2,0x56,0x16,0xc6,0xe3,0xcd,0x8c,0x0f,0xec,0xbb, -0xb6,0xc7,0x2f,0x4d,0x11,0xe7,0x62,0x46,0x46,0x60,0x38,0x3b, -0xac,0x76,0xad,0xd3,0x9b,0xe1,0x04,0x28,0xf6,0xea,0x26,0x8b, -0x99,0x26,0x42,0x3c,0xe6,0x5d,0x68,0x07,0x88,0x01,0x49,0x36, -0x9f,0xa1,0x0e,0xe8,0x8d,0xea,0x86,0x71,0xd3,0x1b,0x7d,0x0a, -0xd7,0x65,0xef,0x08,0x81,0xf1,0x31,0xeb,0x46,0x28,0x9d,0x92, -0x99,0x29,0x78,0xfb,0xc2,0x87,0x38,0x1d,0x0f,0x97,0x71,0xe7, -0x83,0x73,0xe8,0x74,0x9c,0x43,0x5c,0x67,0x03,0x0d,0xf0,0x0a, -0xbf,0xa7,0x6b,0xfc,0x3c,0xc7,0xcf,0x2b,0xfc,0xfc,0xc4,0xf9, -0x04,0x3e,0x7f,0xdd,0x2c,0x28,0xfe,0x13,0x8c,0xff,0x1f,0x59, -0xfb,0x79,0x17,0xba,0x73,0x57,0x21,0x6c,0x6d,0xe6,0x8c,0x31, -0x92,0xf3,0x5d,0x57,0xa7,0x71,0x8a,0x67,0x90,0x00,0xaa,0x26, -0xce,0x80,0xf1,0x92,0x0c,0x9c,0x75,0xb3,0xfa,0x81,0x51,0xda, -0x69,0x88,0x59,0x70,0x09,0x43,0xeb,0xfd,0x22,0x99,0xbb,0x8e, -0xe3,0xa1,0x36,0xf1,0x9d,0x2b,0xc7,0xf3,0x65,0xb9,0x16,0x0a, -0xb7,0x02,0x85,0x04,0x94,0x48,0xee,0x11,0x8b,0xc7,0x58,0x70, -0x00,0x13,0x4a,0x93,0x81,0x20,0x5a,0xc2,0x9b,0xfa,0x50,0x1e, -0x0e,0xba,0x90,0xa7,0x74,0xca,0x97,0x60,0x5a,0x6d,0xc6,0x91, -0x3d,0xc7,0xe9,0x00,0xd1,0x0d,0x01,0xd2,0x5a,0xc5,0xcb,0xe9, -0x10,0x66,0x16,0x16,0x2b,0x24,0x46,0xa3,0x30,0x7c,0x1a,0x56, -0x0f,0xa1,0xb2,0x22,0x12,0xb0,0x4f,0xe9,0x03,0x7a,0x2b,0x93, -0x3d,0xe7,0xe4,0x1a,0xa3,0xec,0x70,0xa0,0x00,0xfc,0xa5,0xe8, -0x61,0xf6,0xe0,0x0a,0x29,0x9e,0xd5,0x6b,0xbb,0xda,0xc9,0x94, -0x93,0xfb,0x32,0xe3,0xa0,0xfb,0x90,0xab,0xba,0xea,0xbc,0x88, -0xb5,0x63,0x9c,0xe3,0xc8,0x92,0xa2,0xca,0x22,0x1e,0x1b,0xcb, -0x10,0x3c,0x23,0x2a,0x0c,0x37,0x72,0xbd,0x2f,0xba,0xe8,0xf3, -0x90,0x83,0xd3,0x3c,0xfa,0x82,0x56,0x52,0x1b,0x8b,0xfd,0xc2, -0x70,0x64,0xae,0x44,0x5a,0xd7,0x5a,0xec,0xb3,0x24,0x29,0xcf, -0x4a,0x60,0x45,0x68,0x6d,0xce,0x58,0xab,0xbb,0x10,0x8b,0xb4, -0xc0,0x66,0xee,0x44,0xc9,0xe4,0x35,0xfa,0x15,0xc7,0x2b,0xe8, -0xf4,0xd2,0xe5,0xec,0x2b,0x43,0x27,0x82,0x5a,0xba,0x4c,0x84, -0xef,0x75,0x32,0x89,0x1a,0x0d,0x95,0xa1,0xe1,0x38,0x8a,0x18, -0x02,0xea,0xf1,0x0f,0x40,0x74,0x12,0x3a,0xba,0xb0,0x8e,0x67, -0x4b,0x5c,0x44,0x7e,0x13,0xaf,0xd7,0x20,0x2b,0xd8,0x41,0x90, -0xa1,0xf8,0x3e,0xd6,0xf1,0xab,0xa7,0x6e,0xff,0x97,0xf4,0x97, -0x37,0x83,0x46,0xcf,0x7b,0x7a,0x7e,0x7c,0xe5,0x1b,0x0f,0xba, -0x62,0x6a,0x54,0x48,0x16,0xfd,0x02,0x52,0x9a,0x76,0x0a,0x57, -0x63,0xbe,0xf8,0x06,0x05,0x28,0x3a,0x76,0x5b,0xde,0xdf,0x8f, -0x05,0x34,0x5e,0x5e,0x8f,0x3d,0xfc,0x13,0xe8,0x00,0xbf,0xfc, -0x02,0x1f,0xf0,0x07,0xbe,0x56,0xf0,0x85,0x86,0xc2,0x2f,0x73, -0xf8,0x40,0x1b,0xe2,0xff,0x81,0x5f,0x7c,0x0b,0xe7,0x97,0x4d, -0x3b,0x68,0xbf,0x80,0x00,0xff,0x8a,0x88,0x97,0x22,0xe2,0xa5, -0xb3,0x15,0x78,0x57,0xd1,0xf1,0x2f,0xbf,0xdc,0x7f,0x72,0xff, -0xcb,0xea,0xfe,0x97,0xf9,0xfd,0x2f,0xeb,0x7b,0xce,0xc8,0x3f, -0x2f,0x8f,0xaf,0xba,0xba,0xea,0xc6,0xf2,0x2a,0xea,0xcb,0xc9, -0x70,0x3d,0xc4,0x57,0x9b,0x88,0x1d,0xd2,0xc9,0x0a,0x15,0x7e, -0x57,0x46,0x46,0x7a,0x29,0x02,0x67,0xe8,0x32,0xba,0x82,0x9b, -0xac,0x81,0xa8,0xdf,0x00,0x45,0x86,0xc2,0xe8,0xbb,0x32,0x97, -0xd0,0x34,0x30,0xa5,0x66,0xf6,0x78,0x62,0x9e,0xe7,0x6b,0x08, -0x83,0xed,0xbb,0xc0,0x64,0xc3,0x95,0x60,0x06,0xa6,0x02,0xba, -0xff,0x67,0xa1,0xc7,0xd4,0x19,0x1a,0x9e,0x60,0x3a,0x17,0x17, -0xcb,0x46,0xf4,0x89,0xd3,0x45,0x26,0x28,0x8d,0x21,0x48,0x2f, -0xe8,0x08,0xc1,0x63,0x53,0x1e,0x7c,0x59,0xba,0xba,0x10,0x53, -0xee,0x61,0x12,0xfa,0x46,0x44,0x68,0x79,0x81,0x43,0xf8,0xff, -0x33,0x9c,0x56,0x4e,0xdc,0x6e,0x35,0xea,0x08,0xe5,0xa2,0x21, -0x44,0x46,0x29,0x22,0x41,0x07,0x4e,0x61,0x44,0x29,0xce,0x27, -0x8d,0x5f,0xe6,0xae,0x7b,0x71,0xb1,0x8e,0x5c,0x09,0x0f,0x4a, -0x17,0xac,0x43,0x54,0x45,0xbd,0x4f,0x3e,0xc1,0xb5,0x15,0x8a, -0x45,0x18,0xcf,0x03,0xe8,0x4f,0x88,0x17,0xaa,0x22,0xb5,0xc8, -0x0c,0x98,0x02,0x46,0x40,0x24,0xf0,0x48,0x26,0x58,0x48,0xba, -0x20,0xc4,0x0d,0x99,0xd2,0x00,0x91,0x16,0x9c,0x96,0x4d,0xc0, -0x8c,0x68,0x50,0xa5,0xe4,0x5c,0x90,0x02,0x78,0xca,0xc8,0xc2, -0xa2,0x44,0xad,0x05,0x62,0x95,0x0c,0x47,0x53,0xdc,0x65,0x92, -0xed,0x87,0xfe,0xad,0x2e,0x19,0xab,0x1f,0xb6,0xde,0x07,0x2c, -0x53,0x64,0x76,0xb6,0x10,0xf0,0x54,0x3b,0xa3,0x44,0x02,0xc9, -0x3e,0x50,0x11,0x7d,0xf2,0x09,0xfc,0xbc,0x2f,0x5e,0x09,0x45, -0xe2,0x02,0x8a,0x3c,0x99,0x5b,0xb6,0x10,0x91,0x0d,0xe0,0xc5, -0xe5,0xb5,0x4f,0x3e,0x81,0xb9,0x66,0xd7,0x2c,0x50,0x2a,0xf0, -0x8b,0x25,0x46,0x77,0xd7,0xab,0xfc,0x03,0x77,0x21,0xea,0x02, -0x4a,0xa7,0x57,0xd4,0x06,0xd7,0x1e,0x40,0x3e,0x2f,0x1c,0x41, -0xae,0xb7,0x1d,0xd3,0x7b,0xe5,0xa0,0xa5,0x79,0xcf,0x21,0x96, -0x0b,0x22,0x42,0xfc,0x69,0x15,0x03,0xbb,0xac,0xa7,0xd6,0x9e, -0xb1,0x20,0x8a,0xf2,0x2f,0xb8,0xff,0x95,0x3b,0x3a,0x65,0xf8, -0x60,0xe5,0x30,0x74,0xb1,0xc8,0xab,0xb6,0x84,0x65,0x7e,0x59, -0xb4,0xb2,0xdd,0x41,0x2a,0x6a,0x2b,0x81,0x63,0xbd,0xdd,0x0a, -0x8e,0x2f,0x11,0xd1,0x4a,0xe1,0xf5,0xb0,0x34,0x5a,0x49,0xbe, -0xf1,0x39,0x00,0x82,0x70,0xd5,0x6d,0x2f,0xa5,0xd1,0xb6,0xf0, -0xc4,0x21,0x81,0xf7,0xec,0xbc,0xb4,0x70,0xd8,0x15,0x43,0xa3, -0x8b,0xb7,0x1e,0xb1,0xcd,0x01,0x73,0x69,0xe0,0xf2,0x92,0xdf, -0x7b,0x5f,0xa1,0x93,0x2a,0x3f,0xf3,0x79,0x9d,0x5c,0xa2,0x92, -0xc5,0xe5,0x73,0xfc,0xc1,0xdb,0xd2,0xd8,0x32,0xe1,0x94,0xd2, -0x44,0x8d,0xf7,0x09,0xd1,0x3a,0x30,0x2e,0x9f,0xda,0x77,0xe8, -0xc4,0xd9,0x83,0x35,0x70,0xea,0xeb,0x29,0x10,0xa3,0x74,0x23, -0xc7,0x43,0xb4,0x7a,0xe0,0x14,0x34,0x1e,0x44,0x88,0xf2,0xf0, -0x50,0xd2,0xc8,0xe1,0xfb,0xfb,0xe0,0xc0,0x9c,0x90,0xdf,0xdf, -0x4f,0xe2,0x69,0xbc,0x8e,0xe9,0x18,0x5b,0x30,0x28,0x8f,0xba, -0x34,0x29,0xd9,0x2a,0xb6,0xf0,0xf2,0x36,0x54,0x18,0x3b,0x00, -0x32,0x80,0x70,0xff,0x79,0xb5,0xb6,0xb6,0x47,0x0c,0x32,0xac, -0x0a,0x5b,0x2c,0x31,0xaa,0x2e,0xcc,0x2e,0xb1,0x32,0x6c,0x94, -0xe3,0x7f,0xa0,0xd6,0xae,0x7d,0xfb,0x92,0x25,0xed,0x20,0x10, -0x73,0x95,0xc2,0x69,0xbf,0x02,0xb0,0x28,0x17,0x6d,0x0f,0x4d, -0xa0,0xd7,0xfd,0xdf,0x01,0x00,0x00,0xff,0xff,0x26,0x9f,0xa4, -0xcb,0x31,0xa6,0x08,0x00, +0xd4,0xfd,0x6b,0x77,0x1b,0x47,0x92,0x20,0x0c,0x7f,0x9f,0x5f, +0x21,0xe2,0x6d,0xf3,0xad,0x22,0x0a,0x60,0x15,0xa8,0x8b,0x05, +0xb8,0x88,0x63,0xcb,0xf6,0x58,0xbb,0xbe,0xe8,0xb1,0xd4,0x6e, +0xa9,0xd1,0x68,0x1e,0x10,0x28,0x8a,0x25,0x81,0x00,0x8c,0x02, +0x28,0x94,0x45,0x9c,0x33,0x7b,0x76,0xff,0x52,0x7f,0x7a,0xbe, +0xed,0x0f,0x98,0xfe,0x4b,0x4f,0xdc,0xf2,0x56,0x17,0x00,0x94, +0x3d,0xb3,0xb3,0xa7,0xdb,0x22,0x2a,0x33,0x32,0x32,0x32,0x32, +0x32,0x32,0x32,0x33,0x32,0x92,0x5f,0xfd,0xb4,0x7d,0xc1,0x43, +0x72,0x41,0x72,0xd3,0x22,0x72,0xb2,0xb2,0xd3,0xb2,0x74,0x46, +0x68,0xd8,0x13,0x24,0x9d,0x79,0xf8,0x85,0xfa,0x21,0x33,0xc9, +0xf0,0x21,0xc9,0x00,0x80,0x88,0x35,0x30,0x7c,0x10,0xac,0x4e, +0x44,0x50,0x4a,0x24,0xc8,0xc8,0x86,0x8c,0x04,0x32,0xb2,0x21, +0x4d,0x24,0x31,0xcb,0x5f,0xc5,0x78,0xa5,0x78,0x78,0xa1,0x08, +0xcb,0x9c,0x08,0x9d,0xfe,0xc9,0xaa,0xa9,0x12,0xc3,0x13,0xae, +0xa4,0xc5,0x54,0x9d,0x08,0xa4,0x90,0x0e,0x90,0x8a,0x5e,0x81, +0x6b,0x4a,0xa9,0x02,0x9c,0xe6,0xf5,0x5b,0x7c,0x2d,0x7e,0xbc, +0xae,0xd1,0x46,0x3a,0x57,0x4f,0xb4,0x32,0x0c,0x0b,0x97,0x8b, +0xdc,0xe1,0x88,0x1b,0x00,0x99,0x67,0x47,0x09,0x94,0x14,0xeb, +0x74,0x9b,0xdf,0x08,0x37,0x2f,0xe6,0x81,0x0d,0xff,0xf5,0x6b, +0xff,0xe4,0xeb,0xd7,0x01,0x58,0xa6,0xf0,0x8b,0x2e,0x27,0xbf, +0xf6,0x55,0xec,0xa9,0x8a,0x12,0x60,0xd1,0x7f,0xfd,0x06,0x4a, +0xbc,0x09,0xde,0x40,0x89,0x37,0x5c,0xe2,0x8d,0xbf,0xab,0xc8, +0x06,0x16,0x01,0x38,0x84,0x36,0xe8,0x52,0x01,0xbf,0xda,0x57, +0xe9,0x14,0x0f,0x60,0x2b,0x06,0xaa,0xde,0x5e,0xdc,0x7c,0x06, +0xd4,0x9c,0xff,0xfb,0x3f,0xd4,0xc3,0xe7,0x3b,0x2b,0xc8,0xa1, +0x82,0x1c,0x2a,0xc8,0xd1,0x61,0x03,0x7e,0x95,0x2a,0xc8,0xcb, +0x15,0xe4,0x9f,0x01,0xf1,0xa6,0x02,0x50,0x88,0xa4,0xc7,0x80, +0xc2,0x4d,0x88,0xcc,0x78,0x4d,0x8f,0xf9,0xe5,0x21,0x36,0xf3, +0x4d,0x18,0x80,0xca,0x0c,0x5e,0x07,0x6f,0xf0,0xd9,0x5f,0x8c, +0xf1,0x9d,0xc7,0xd0,0x9a,0xaf,0x5f,0xc7,0x4f,0x43,0xe0,0x41, +0x0c,0x13,0xb5,0xa5,0xe1,0x3a,0x66,0xfd,0xad,0x3b,0x92,0x76, +0x67,0xaa,0x3c,0x0e,0xa4,0x8b,0xdc,0x00,0x5e,0x15,0x93,0xb2, +0x08,0x40,0x5d,0xdf,0xdb,0xb3,0x25,0x46,0x0a,0x34,0x15,0xcf, +0xd7,0x2b,0xac,0xa3,0x5c,0xf5,0x4e,0xcd,0xfe,0x1a,0x24,0x43, +0x73,0xfc,0x8d,0xf7,0x26,0xf2,0xdb,0x19,0xc6,0xc9,0xf7,0x60, +0x64,0x41,0x5e,0xe4,0x5b,0xb7,0x78,0x75,0x06,0xc0,0x85,0x55, +0x19,0x3e,0xcc,0x11,0x16,0x45,0xc9,0xe1,0x0f,0x97,0x99,0x42, +0x37,0xa3,0x77,0xf3,0xa5,0x1c,0x4a,0x5c,0xf8,0x78,0x43,0xdb, +0xfa,0xec,0x5a,0x70,0x56,0x86,0xc3,0x07,0x0b,0xc1,0x61,0xba, +0xf8,0xb5,0xe5,0x38,0xf3,0xda,0x72,0x96,0x79,0x63,0x39,0xd1, +0xbc,0x31,0x8e,0x33,0x20,0x30,0xe7,0xaf,0xd1,0xd3,0xee,0x22, +0x06,0xd1,0x81,0xc2,0x28,0x43,0x11,0x2c,0x77,0xa0,0xc0,0xf9, +0x1b,0xce,0x00,0x39,0x82,0xd2,0x28,0x50,0x94,0x61,0x88,0x33, +0x27,0x7a,0x76,0x78,0x92,0xc1,0xe0,0x35,0x16,0x80,0x75,0xe4, +0x6b,0x2c,0x82,0xab,0xc4,0xca,0x86,0x1e,0xd6,0x1e,0xdb,0x11, +0xc8,0x71,0xfe,0xb1,0xda,0x63,0x39,0x02,0xc1,0x30,0x38,0xdf, +0x30,0xd9,0xf8,0xba,0x65,0x18,0xe3,0xc8,0x20,0xb2,0xf3,0xf0, +0x3c,0xe7,0x0c,0x7c,0xf0,0x32,0x8c,0x71,0x98,0x1c,0xd8,0x9e, +0xa2,0x9f,0x90,0x55,0x04,0xc3,0x70,0x7d,0x8a,0x4c,0xbc,0x84, +0x72,0x5a,0x22,0xe4,0xa3,0x28,0x0f,0x94,0x5c,0x96,0x86,0x97, +0x87,0x56,0xe9,0xc1,0x48,0xe7,0xa3,0x45,0x18,0xec,0xfc,0x30, +0xa0,0x46,0x05,0xcd,0x02,0xed,0xf9,0xf5,0x9b,0x72,0xe7,0x1c, +0x8e,0x7e,0xa2,0x4e,0x2e,0x27,0x79,0x05,0x7a,0xd0,0x34,0x93, +0xdc,0x41,0x7f,0x4f,0xab,0xc2,0x36,0x0f,0x02,0xed,0x08,0xaa, +0xd1,0xbd,0xf6,0xf8,0xe1,0x52,0xf4,0x25,0xcf,0x4b,0xb9,0x6f, +0x3c,0x7e,0xdc,0xd4,0x74,0x64,0xf0,0xba,0x02,0xc5,0x1b,0x52, +0x94,0x88,0xe2,0x4d,0x05,0x8a,0xd7,0xa4,0x52,0x2d,0x14,0x56, +0xfb,0x2c,0xd3,0xa4,0x7a,0xb8,0x0f,0x06,0x74,0x75,0xa9,0xf5, +0x34,0xe4,0x73,0x47,0xfc,0x78,0x1a,0xca,0x49,0xa3,0x3d,0xde, +0x15,0xe0,0xe7,0x94,0xc9,0x80,0x9f,0x53,0xa1,0xa1,0x3d,0xf3, +0x26,0xa3,0xd5,0x97,0xcb,0x71,0xcd,0xc4,0xcb,0x99,0x45,0x25, +0x59,0xa7,0x76,0x07,0xd9,0x7c,0xbd,0x1c,0x27,0x17,0x77,0x77, +0xfc,0xa3,0x7a,0x49,0x20,0xaf,0x5a,0x03,0x14,0xff,0xa8,0x86, +0x12,0x43,0x5a,0x30,0xaa,0x32,0x01,0x7f,0x23,0x4f,0xf9,0x97, +0x7a,0x22,0x1b,0x3d,0x26,0xe9,0x97,0x99,0x6b,0x98,0xf6,0x0a, +0x3b,0xae,0xe8,0xdf,0xa6,0x20,0xbc,0x3f,0x90,0x7c,0x1a,0x5e, +0x42,0x81,0xd0,0x7c,0x90,0x70,0x0a,0xac,0x6a,0xe8,0x45,0xf5, +0x96,0x0a,0x2e,0x70,0xba,0x17,0xba,0x06,0xbf,0xcb,0xd0,0x56, +0x9d,0xc2,0x96,0x83,0xea,0x14,0x58,0xcd,0xe4,0x03,0xeb,0x64, +0x68,0xab,0xce,0x8a,0x71,0xb8,0x4b,0x6f,0x71,0xa9,0x6e,0x68, +0x30,0x68,0xa9,0xb4,0x0d,0x7f,0x73,0x04,0x66,0x77,0x78,0x71, +0xcb,0xc5,0x5e,0x03,0x32,0x60,0x61,0x4d,0x29,0x89,0xae,0x09, +0xce,0xb8,0x0a,0x90,0x92,0xe8,0x2e,0xdb,0x35,0x69,0x12,0x68, +0x68,0xcf,0xe6,0x2c,0x43,0xed,0xbf,0x98,0xac,0xa2,0x1b,0x15, +0x8b,0x39,0xe7,0xfe,0x3a,0xd5,0x05,0xab,0xf7,0xa6,0x2e,0x9f, +0xcd,0x9b,0x52,0x95,0x07,0xb8,0x9f,0x74,0xa8,0x3c,0xfa,0x2d, +0xbd,0x59,0xaf,0xae,0x47,0x53,0xe3,0xa2,0x5a,0xcc,0xb1,0x6d, +0xb8,0xec,0xdf,0xff,0x5f,0x32,0xff,0x5d,0xdb,0x93,0xd6,0x1a, +0x9d,0x53,0x8f,0x96,0x08,0x0a,0x02,0xe3,0xdd,0xa8,0x82,0xff, +0xfc,0x1f,0xba,0x40,0x87,0x43,0x11,0x92,0x03,0xfd,0x3f,0xff, +0x07,0x9e,0xad,0xfb,0x3d,0xbd,0x2b,0x58,0x26,0xe6,0x1e,0xdb, +0x73,0xe5,0xd2,0x85,0x1d,0xba,0x32,0x80,0x73,0x7b,0xdc,0xca, +0x4d,0x59,0x9b,0xac,0x0e,0xe4,0x05,0x47,0x54,0xe6,0x76,0xe1, +0xba,0xcc,0xca,0xd3,0x3b,0x03,0xc7,0xc7,0xe3,0x53,0xbd,0x9a, +0x1b,0xf3,0x23,0x51,0x3a,0x66,0x7d,0x25,0x07,0x34,0x11,0x9f, +0xc8,0x03,0x55,0x7e,0x07,0x17,0x14,0x88,0xeb,0xad,0xfc,0x6c, +0x3e,0xbb,0xe2,0xfb,0xce,0xf7,0xf0,0x56,0xd6,0x85,0xaa,0xbc, +0x95,0x75,0x66,0xc9,0x2d,0x7a,0x7f,0x23,0x2b,0x1d,0xa3,0x6b, +0xda,0x56,0xcc,0xb6,0xfb,0xf7,0xed,0x6c,0x7e,0x83,0xd9,0xf7, +0x93,0xef,0xe8,0xd4,0x24,0x6e,0x03,0xbd,0xc8,0x36,0x3d,0xa6, +0xd1,0xde,0xa3,0x97,0x54,0x99,0x02,0xf5,0x2a,0x59,0x8d,0xcd, +0x9b,0x64,0x39,0xc6,0xbd,0xeb,0xd2,0x06,0x27,0xef,0x81,0x51, +0x25,0x83,0x4d,0xa0,0x46,0x14,0x90,0xc5,0xeb,0xc7,0x64,0x43, +0x6b,0xbf,0x16,0xb4,0xed,0xea,0x9f,0xff,0x36,0x34,0xbe,0xe8, +0x0a,0x61,0x3d,0xad,0x0a,0xe2,0x45,0x89,0x66,0x95,0x53,0xa0, +0x59,0x25,0xdb,0x9c,0x9e,0x2f,0x57,0xd7,0x73,0xb0,0x7b,0x16, +0xd7,0x3b,0xb9,0x6d,0x78,0xac,0x18,0x0b,0x43,0xc3,0x30,0xd6, +0xc1,0x72,0x0f,0xe6,0xda,0xe5,0x0a,0xc4,0xda,0x59,0x6e,0xa4, +0x1e,0xd0,0xc0,0x07,0x50,0x5c,0x29,0x1f,0xae,0xd6,0xdb,0xad, +0xf4,0xb0,0x8b,0x20,0xd9,0x52,0x79,0x6e,0xd5,0xf7,0x68,0xa6, +0x53,0xb0,0xd0,0x4e,0x27,0xcf,0xf1,0x00,0xa1,0x65,0xec,0x0f, +0x87,0xc8,0x95,0xb5,0x9f,0x84,0x76,0x5a,0x3a,0xbb,0xf6,0x36, +0x72,0xd5,0x18,0x55,0x5c,0xce,0xc7,0xc0,0xa4,0xc6,0xb5,0x5a, +0xcb,0x7d,0x3c,0x19,0x85,0x6c,0x84,0xf5,0x2d,0xc1,0x2b,0xd7, +0xfd,0x29,0x22,0x58,0xc6,0x52,0x68,0x77,0x19,0x40,0xe6,0xed, +0x1b,0x8e,0xb5,0xd5,0xce,0x6e,0xdf,0x9a,0x5f,0xed,0xea,0x3d, +0x04,0xa4,0x03,0x72,0x2f,0x30,0xd7,0x79,0x58,0x44,0x4f,0xe7, +0x2a,0xd7,0x6c,0x60,0xce,0x97,0x26,0x0a,0x1e,0xe6,0x8c,0x28, +0x48,0xb4,0x82,0xfb,0x9e,0x52,0x82,0x06,0xe7,0xb4,0xc6,0xd3, +0x79,0x96,0x4c,0x1a,0x65,0x88,0x67,0x94,0x81,0xef,0xc1,0x2c, +0xec,0x4c,0x5c,0x73,0x05,0x0d,0x4c,0x6d,0x71,0xb0,0xc2,0x46, +0x31,0xf7,0x2b,0x4a,0x16,0x18,0x8a,0x64,0x58,0x02,0xf9,0x92, +0xe2,0x1b,0x5e,0x42,0x7f,0x65,0x76,0xd6,0x57,0x98,0x10,0x34, +0x28,0xbd,0x35,0x5f,0x24,0xb3,0x46,0x29,0xf7,0x27,0x48,0x55, +0x10,0x15,0xa4,0x13,0x8c,0x50,0x7e,0xb9,0x9e,0x4d,0xa6,0x89, +0x93,0x4b,0x29,0xc1,0x78,0x44,0xcb,0x8b,0xa9,0x9d,0xf5,0x4c, +0xd2,0x82,0x86,0xca,0x2d,0x13,0xa0,0x60,0x98,0x06,0x0d,0x57, +0x41,0x86,0x82,0x14,0x4a,0x40,0x8f,0xce,0x57,0xf3,0x99,0x43, +0xcb,0x0f,0x92,0xc6,0x8f,0xd5,0x54,0x76,0xe2,0xce,0x87,0x6b, +0x38,0x7c,0x1c,0x24,0xc4,0x3a,0xb1,0xcd,0x94,0xc4,0xa7,0x42, +0xd2,0x9f,0x4e,0xf9,0x7d,0x11,0x00,0xc0,0x41,0x5e,0x90,0x17, +0x62,0xd5,0x57,0xc9,0x6f,0x69,0xb2,0x8c,0xf0,0x0c,0xa1,0x73, +0x7a,0x16,0x44,0xf0,0x1f,0xbf,0x65,0x5e,0x01,0xd5,0xe1,0xb7, +0xce,0xcf,0x08,0xb2,0x16,0xea,0x8c,0xa1,0x1e,0x0b,0xbe,0xc7, +0x43,0xf1,0x50,0xbb,0x69,0x5f,0xe3,0x31,0xb5,0x6e,0x0a,0x8e, +0xf5,0x74,0x8c,0xdb,0x6f,0x7a,0x6d,0x88,0x00,0xde,0x04,0x46, +0x39,0xb9,0x2b,0xe3,0x0f,0xfd,0xe0,0x8d,0xb8,0x2a,0x0f,0x86, +0xd4,0x0a,0x55,0x36,0x98,0x04,0x69,0xf0,0x4e,0xae,0x91,0x6d, +0x3a,0x41,0xde,0x09,0xd6,0xc1,0x6d,0x30,0x0a,0xb2,0x45,0x70, +0xb5,0xb1,0x1e,0xe9,0x41,0x65,0x71,0x95,0xdb,0x09,0xb9,0x1f, +0xcc,0x62,0xab,0x8a,0x60,0x01,0x7f,0xe3,0x59,0x2b,0x62,0x93, +0x97,0x2e,0x68,0xc3,0xa4,0x3d,0x7e,0x8f,0x3f,0xae,0x63,0x0a, +0x28,0x02,0x28,0xe3,0xd8,0x6a,0xf5,0xeb,0xe3,0xe3,0xdc,0x4d, +0x79,0xe3,0x2b,0xd2,0x08,0x37,0xbf,0x39,0x81,0x4e,0x73,0x78, +0x39,0x59,0x67,0x0d,0xcc,0x25,0x6c,0x95,0xc6,0xc7,0x47,0x83, +0xe6,0xd5,0xa6,0x8d,0xe1,0xe1,0x79,0xe9,0x37,0x21,0x24,0x78, +0x65,0x39,0xf5,0x83,0xe6,0x55,0x6e,0x67,0x41,0xd2,0x90,0xfd, +0xf1,0xd2,0x38,0xd2,0xe8,0x34,0x57,0xa1,0x10,0x46,0x1e,0xd0, +0x9f,0xd7,0x43,0x8a,0x4b,0xed,0x66,0xc7,0xb1,0x9b,0x7f,0x7c, +0x6c,0xe7,0x87,0x6e,0xf1,0x90,0xee,0x6a,0x5f,0xc7,0xa9,0xaa, +0x35,0xd4,0xb5,0xa6,0x47,0x71,0x7c,0x0d,0x99,0x79,0x14,0xbb, +0x15,0xb4,0x5c,0xfc,0xb8,0xbb,0xe5,0xd6,0xd0,0x72,0x6b,0x10, +0xde,0x33,0x33,0x3e,0xd2,0xf9,0x49,0xd7,0xbe,0xf9,0x8a,0xfb, +0x63,0x18,0x5c,0x66,0x92,0x6c,0xba,0xe9,0xd6,0x67,0x4a,0xa4, +0x0c,0xdd,0x87,0x77,0xcf,0x55,0xd4,0x3a,0x91,0x4f,0x62,0x5a, +0x97,0xfc,0x77,0xeb,0x07,0x23,0x0e,0x31,0x80,0x87,0xa5,0x9c, +0x16,0xdc,0x5a,0x29,0x84,0x3f,0x58,0x43,0x8f,0x21,0x6b,0x51, +0x2e,0xa4,0x9d,0x28,0x95,0xef,0x14,0x60,0xaa,0x00,0x47,0xb1, +0x95,0x44,0xd8,0x08,0xd0,0x6e,0xeb,0x6d,0x55,0x5b,0x73,0x17, +0xa0,0x82,0x5b,0x1d,0x03,0xf0,0xae,0x12,0x83,0x0b,0x50,0xc5, +0xef,0x93,0x4d,0xd4,0xcc,0xa3,0x93,0x3c,0x3a,0x8f,0x37,0x9d, +0x93,0x4d,0xa7,0x99,0x77,0x4e,0xf2,0x8e,0xdc,0xad,0x32,0x8d, +0x88,0x5b,0x51,0x4f,0xbd,0x82,0xb2,0x95,0xbc,0xb5,0xc9,0xdb, +0x8e,0x8a,0x4d,0x04,0xf6,0xa4,0xc0,0xb4,0x77,0xf4,0xf8,0x09, +0x8d,0x13,0xee,0xb4,0x6b,0x9f,0xee,0xe1,0xbf,0x03,0xe9,0xe8, +0x10,0xd7,0xde,0xf9,0x52,0xf2,0x9d,0xa0,0x3b,0x6f,0x45,0x74, +0xf3,0x51,0x17,0x29,0xe4,0x03,0x82,0x66,0x93,0x7b,0x36,0x5b, +0xc4,0x0c,0x27,0xf1,0x0f,0xa8,0x2b,0xde,0x11,0x52,0xbc,0xce, +0x50,0x2c,0xf9,0x45,0x28,0x0f,0xe8,0xf4,0x8e,0x78,0x06,0xbe, +0xb9,0x40,0x8d,0xf2,0xec,0xd9,0x5f,0xb8,0xba,0x41,0xb6,0x68, +0x75,0x64,0x54,0xe3,0xef,0x48,0xc9,0x9b,0xc6,0xa0,0x87,0xa8, +0xdf,0xf3,0x5b,0xad,0x6c,0xd1,0x53,0xb0,0xcd,0xe6,0x30,0x2e, +0xc0,0xca,0x6d,0xbd,0x69,0x8e,0xc3,0x99,0x47,0x04,0xe2,0xec, +0xa5,0xe7,0xd0,0xf6,0x56,0x2b,0xf5,0x31,0x8f,0x5b,0x48,0xa3, +0x98,0x51,0xa5,0xc3,0xa1,0x6f,0xee,0x10,0x4e,0xf3,0xad,0xbe, +0xe5,0xae,0x95,0x0a,0xef,0x10,0x6a,0x8d,0xd2,0xab,0xdb,0xed, +0x20,0x6d,0xa9,0x09,0xee,0x7a,0xf8,0xdd,0xde,0x1c,0xb8,0x53, +0x1d,0x5f,0x04,0x08,0xef,0x77,0x37,0x5b,0xfa,0xd1,0xce,0x0f, +0x2b,0x98,0xeb,0x82,0x39,0x17,0xd4,0xbb,0x18,0xda,0x7b,0x75, +0xb7,0xe7,0x03,0x36,0x6d,0x7d,0x39,0x9e,0x8e,0xb2,0xcc,0xce, +0x0f,0x54,0x8f,0x09,0x12,0xfd,0x5a,0xb1,0xe3,0xdc,0x60,0x6f, +0x61,0x94,0x41,0xe3,0x02,0x21,0x96,0x6f,0xf1,0x60,0xd8,0xab, +0x2b,0x55,0x70,0xde,0xb4,0x1e,0xe0,0x51,0x31,0xe8,0xf8,0x95, +0x6f,0x9e,0x25,0xf8,0xcd,0xea,0xc1,0x0c,0x85,0x87,0x0a,0x86, +0x2a,0x20,0x1d,0x46,0x9a,0x90,0xdc,0x94,0xf3,0x9a,0x7c,0x64, +0xab,0x4f,0x74,0x4f,0x2e,0xcd,0x95,0x99,0xf6,0xa3,0x13,0x04, +0xd9,0xd6,0xb6,0xbb,0xc2,0xa7,0xf3,0xbd,0x44,0x54,0x0b,0xc6, +0x55,0x94,0x6d,0x62,0x0c,0x37,0x10,0xda,0x14,0x92,0x58,0x16, +0xfb,0xf0,0xee,0xce,0x7b,0x8f,0x21,0x34,0xbc,0xc7,0xfc,0xd4, +0xf4,0x88,0x6e,0x70,0x82,0x2e,0xad,0x6c,0xc7,0x38,0xd6,0x94, +0x53,0xc4,0x9c,0x13,0x8a,0x98,0xb1,0x69,0xc6,0x14,0xb5,0xa3, +0x49,0xa1,0x33,0x4e,0xc6,0x41,0x4e,0x09,0x11,0x26,0x44,0x98, +0xd0,0x53,0xcb,0x41,0xbc,0xf6,0x72,0xf2,0x7e,0xb8,0xab,0x9d, +0x78,0xa9,0xda,0x6c,0xc8,0xad,0x65,0xef,0x4b,0x87,0xd9,0x9d, +0x2d,0xd6,0x2b,0x98,0xe3,0x6f,0x82,0x4b,0x68,0xf7,0x24,0x10, +0x13,0xa7,0x80,0x8e,0xed,0x2c,0x5d,0xba,0x82,0x3f,0xad,0xea, +0x12,0x1c,0x80,0x76,0x64,0xf1,0x4c,0xd8,0x40,0x04,0x50,0xe5, +0xb1,0x60,0x95,0xc3,0x36,0x0c,0xba,0xce,0xdf,0xb2,0x65,0x16, +0x3a,0xec,0xa2,0x22,0x03,0xef,0x86,0x7f,0xa8,0xca,0x99,0x6a, +0x1f,0xa5,0xe6,0x1d,0xea,0x5a,0x50,0x61,0x5f,0xdc,0xf4,0xfc, +0x89,0x80,0xbf,0x1b,0x16,0xf9,0xf3,0x7c,0x96,0x81,0xa5,0xef, +0x4d,0x02,0x9c,0xc8,0xfa,0x5e,0x75,0xee,0x98,0x72,0xef,0xee, +0x14,0x41,0xac,0x6c,0x8a,0xa0,0x2b,0x5c,0x81,0x8c,0x57,0x1e, +0x72,0x0f,0xe1,0x4d,0x03,0x18,0xde,0x57,0xe1,0x64,0xab,0xd1, +0x1f,0x1f,0xdf,0x17,0x3d,0xac,0x59,0x7b,0xff,0xc2,0x2d,0x2e, +0x14,0x96,0x0f,0x14,0x1a,0xe0,0xf9,0xa5,0x72,0xd3,0x92,0x64, +0xa3,0x49,0xf0,0xdd,0x9c,0xf5,0x6c,0x94,0x57,0x59,0x8b,0x28, +0x14,0xc9,0x04,0xe3,0xaf,0x68,0xdb,0xc9,0x39,0xd9,0xd5,0x4b, +0xc6,0x21,0x4c,0xf4,0xab,0x65,0x4a,0xd3,0x95,0xfd,0x0c,0x95, +0x6a,0xc1,0xed,0x7c,0x09,0x26,0x78,0xfa,0x2a,0xc9,0xb2,0x64, +0x4a,0x9b,0xbc,0xda,0xac,0xd4,0xc8,0x60,0x2a,0xa7,0xba,0x06, +0x49,0x7b,0x99,0xbc,0x45,0x37,0xa8,0x29,0x6b,0xff,0xa1,0xc4, +0x70,0x56,0xf3,0xad,0xce,0x5f,0x4a,0x3e,0x3e,0x44,0x45,0x45, +0xcb,0x46,0x3c,0x26,0xe3,0x23,0x74,0x64,0xc9,0xda,0x06,0x51, +0x30,0xde,0xc4,0xb7,0x38,0xb1,0x8f,0x73,0xf8,0x0b,0xb2,0x88, +0x90,0xe5,0xf2,0xb7,0x50,0x54,0x1c,0x71,0x2c,0xe3,0x08,0x0b, +0xb6,0xc6,0x30,0x85,0xe1,0x50,0x1d,0xe7,0xaa,0xfe,0xfb,0xd8, +0x45,0xfa,0x7a,0x07,0xc6,0xb1,0xbe,0x89,0xa9,0xb8,0x0a,0xaf, +0xd3,0xbb,0x81,0x99,0xf7,0x1d,0xcc,0xcc,0x9a,0xa5,0x62,0xb2, +0xde,0x52,0x35,0x28,0xc2,0xfc,0xb7,0x19,0x0d,0xa9,0xf1,0x1a, +0xce,0x74,0xab,0x70,0xdc,0xf4,0x2a,0x4f,0xa9,0xd6,0x0a,0x40, +0x20,0x64,0x11,0xa0,0xef,0xc3,0xb3,0xa1,0x2f,0x22,0xc7,0xa6, +0xf5,0x8e,0x4e,0xbf,0xaf,0xe5,0xff,0xd7,0x38,0x4a,0x1e,0xd7, +0xd8,0xf7,0x57,0x25,0x03,0x5f,0x96,0x07,0xae,0x79,0x2f,0x89, +0xe6,0xed,0x80,0x19,0xd9,0x41,0xda,0x42,0xd6,0xd6,0x53,0x7c, +0x4f,0x13,0x5f,0x82,0x9b,0xd7,0x08,0xac,0x30,0xc7,0x16,0x57, +0x3a,0xe0,0x8a,0x82,0xac,0x83,0xab,0x22,0x5c,0x12,0xc1,0xda, +0xa8,0xd3,0x8b,0xa0,0x15,0x30,0xd3,0x1d,0x1f,0x27,0xed,0x4b, +0xb0,0x4f,0xfa,0x5e,0x16,0xc1,0x77,0xb2,0x68,0x2f,0x01,0x92, +0x7f,0xc1,0x94,0xae,0x53,0xa7,0x3a,0x75,0xe9,0x07,0x52,0xb6, +0x8f,0xe6,0x7d,0x16,0xf5,0xb3,0xa8,0x9d,0x77,0x5b,0x7f,0x45, +0x5b,0x1e,0x94,0x77,0x0b,0x10,0x9e,0x50,0x1d,0x71,0xd6,0xe9, +0x67,0x1d,0xc8,0xfb,0x2b,0x1a,0xae,0x3a,0xab,0x03,0x68,0x37, +0xaa,0xe0,0x06,0x0b,0xe6,0xaa,0xe0,0xe8,0x84,0x48,0x94,0x82, +0x1b,0x28,0x98,0x77,0x4c,0x56,0x87,0x17,0xac,0xb7,0xb0,0x36, +0xe5,0xb3,0xec,0xe0,0x16,0x16,0xa0,0xb4,0xc8,0x1b,0xf6,0x94, +0x28,0xd4,0x8e,0xcb,0x08,0xa0,0xfd,0xa0,0x0c,0xb6,0xac,0x00, +0xdb,0x1a,0xc0,0x58,0xfd,0x70,0xc5,0x4b,0x52,0xd5,0xa0,0x85, +0x61,0xaa,0x3b,0x54,0x86,0xab,0xf9,0x46,0xc3,0x80,0xc6,0xa6, +0xb2,0x42,0x1c,0x4c,0xb7,0x05,0x37,0x99,0xba,0xa1,0x0b,0x26, +0x4f,0x42,0x8f,0xb8,0xb2,0x23,0x8e,0xc2,0x25,0x67,0x3b,0xbb, +0x06,0x35,0xd6,0x3d,0x18,0x81,0xc5,0x41,0x3f,0x2e,0x61,0x40, +0x28,0xc5,0x47,0x38,0x4b,0x8e,0x3b,0x13,0xf3,0x1e,0xe6,0x51, +0x7a,0x77,0xc7,0xc5,0x26,0xaa,0x3c,0x95,0x19,0xa4,0x30,0x71, +0x0d,0x8d,0x27,0x8f,0x29,0xab,0xab,0x15,0x02,0xa1,0xe0,0xd6, +0xe6,0x67,0x59,0x79,0x15,0x58,0x39,0x8b,0xdd,0xa6,0xa1,0xbc, +0x1f,0xcd,0x7c,0x17,0xab,0x68,0x1a,0x90,0x9d,0xd6,0x5f,0x87, +0x01,0xfe,0xc5,0x3f,0xea,0x5f,0x48,0xa3,0xe0,0xa8,0x47,0xde, +0xec,0xbc,0x23,0xc1,0x35,0x31,0x1c,0xa4,0xea,0x91,0x60,0x11, +0xa9,0x4a,0x68,0x89,0xd9,0xd1,0x5f,0xe4,0x82,0x11,0x2f,0x42, +0xf1,0xd1,0x80,0x1f,0xbc,0x46,0x5d,0x44,0xb2,0x3e,0x83,0x1f, +0xbc,0x0e,0x5b,0x74,0x64,0xbd,0x05,0x3f,0x20,0x65,0xb2,0x31, +0xb7,0x34,0x37,0x9d,0x16,0x2e,0x4a,0x27,0x79,0x9c,0x77,0x5a, +0x39,0x3d,0x58,0x69,0x6e,0x70,0xe6,0xfe,0x17,0xff,0xfe,0x0f, +0x26,0x09,0xf2,0xa3,0xf3,0x3c,0xec,0xb7,0xfe,0xda,0xfd,0x6b, +0xaf,0xd4,0xb0,0x9c,0x9a,0x92,0x0f,0xad,0xe8,0xa2,0xff,0x38, +0x9f,0x6c,0x54,0x54,0x92,0x4d,0x74,0xbe,0xa9,0x2c,0xba,0x61, +0x96,0x6c,0x80,0x17,0x5c,0x54,0xea,0x82,0x31,0xd7,0xda,0x84, +0xfe,0x89,0x47,0x44,0xf9,0xe7,0x42,0x26,0x7c,0x47,0xad,0x3c, +0xf4,0xfb,0x7f,0xc5,0x91,0xf8,0x5b,0x6c,0x53,0xda,0x9a,0x6c, +0x7a,0xfa,0xfb,0x37,0x24,0xbc,0xef,0xd6,0x15,0x9e,0x4f,0xf2, +0x3e,0x8c,0xfd,0x1c,0xe9,0xec,0x7a,0xbf,0xe1,0x83,0x06,0x5e, +0x2e,0xaf,0x8c,0xd5,0x36,0x08,0x2f,0xf6,0x73,0x30,0x1b,0xf1, +0x83,0xf2,0x8b,0xdb,0x07,0x56,0x1e,0x59,0x83,0x9e,0x1e,0xb6, +0xe9,0xb0,0xbc,0xd9,0x60,0x65,0xaa,0xeb,0x98,0xac,0x41,0x7b, +0xae,0xd0,0x64,0xfb,0x97,0x56,0x36,0x55,0x74,0x89,0xa2,0x76, +0xad,0xa5,0xe6,0x25,0x99,0xb0,0xba,0x9e,0x24,0xdc,0x67,0xbd, +0x25,0x45,0x68,0xc9,0xa5,0x8a,0xdf,0x63,0xd5,0xa5,0x8b,0xe7, +0xa6,0x78,0x55,0x7c,0xa5,0x3d,0x77,0xfd,0xad,0x26,0x1f,0x1f, +0x0f,0xac,0x2f,0x52,0x64,0xd6,0x67,0x67,0x48,0x93,0x0f,0xdf, +0x2d,0xb9,0xf0,0x4b,0xac,0xd2,0xc2,0xc6,0x9e,0x4e,0xa1,0x15, +0xe6,0x28,0x54,0xdb,0x17,0xc6,0x03,0xaa,0xa3,0xbd,0x9e,0x7a, +0x36,0xa6,0xc2,0x8a,0x0d,0x2f,0xc3,0xb3,0xae,0xa7,0xbf,0x1d, +0xfc,0xdb,0x31,0x7f,0xc9,0xb2,0x50,0x6f,0xde,0x31,0x0b,0x0c, +0x2f,0xb2,0xf4,0xb7,0xc3,0xfc,0x21,0xca,0xcc,0xf3,0x2e,0x80, +0x15,0xec,0x45,0x7d,0x01,0x92,0xed,0xb0,0xc8,0x65,0x89,0xa9, +0x0e,0x04,0xe8,0xbd,0xe5,0x6d,0x58,0x69,0xb0,0xd0,0x91,0xc7, +0x3e,0x6b,0x85,0x11,0x0d,0x86,0xf7,0x35,0x5b,0xfe,0xab,0x1a, +0x2c,0x7b,0xad,0xec,0x1a,0xa3,0x65,0x1a,0x17,0xe7,0xf2,0x60, +0x19,0x17,0xe7,0xed,0x1e,0xb1,0x74,0x30,0x1d,0x0e,0x96,0x43, +0x58,0xc8,0x5a,0x5f,0x31,0xff,0x5e,0x0e,0xe1,0x13,0xe3,0x7b, +0x11,0x5b,0x65,0x23,0x91,0xfd,0x34,0xba,0x44,0xf8,0x74,0x28, +0xce,0x18,0xfc,0xb9,0x84,0x2e,0xf0,0x55,0x2f,0x98,0xde,0x35, +0xeb,0x86,0x42,0x0f,0xe3,0xc6,0xde,0xfe,0x4d,0x60,0xc7,0x17, +0xc8,0x2c,0x67,0x18,0x87,0x09,0x69,0x19,0x54,0xf5,0xc2,0x7e, +0x31,0x90,0xc5,0xad,0x2d,0x0b,0xb2,0x68,0xf5,0x3e,0xb9,0xdb, +0xfc,0x36,0xe6,0xc3,0xc2,0xad,0x8e,0x74,0xd1,0x7c,0xae,0x20, +0x2b,0x36,0x99,0x87,0x55,0x24,0xa1,0x68,0x3e,0x61,0xd8,0xab, +0x8f,0x76,0xe8,0x2c,0xaa,0x0e,0x4d,0x05,0xcd,0x73,0xdf,0xdd, +0xdc,0x91,0xd4,0x9f,0x16,0x8b,0x79,0x96,0xae,0x92,0xf8,0xe3, +0xb4,0xdb,0x58,0x36,0x82,0x65,0xb7,0x31,0x6d,0x6c,0xf5,0x91, +0xc2,0xaf,0xeb,0xd1,0x64,0xb5,0x4c,0x92,0xe2,0x92,0xc2,0x3e, +0x12,0xb0,0x96,0x17,0x0a,0x5c,0x3a,0x53,0xa7,0xa7,0xb3,0x0c, +0xef,0xde,0xcc,0x80,0x17,0xe8,0x85,0xec,0x14,0x46,0x65,0x9a, +0x66,0x3f,0x8e,0x7e,0x84,0xbe,0x38,0x3e,0x96,0x9f,0xb9,0x8f, +0x3b,0x90,0x68,0xb4,0x8c,0xe4,0x75,0xf4,0xd9,0x26,0x9e,0xb5, +0x37,0xc1,0x2c,0x87,0x3f,0xb9,0x52,0x9c,0x47,0xf1,0x6c,0xe3, +0xdb,0x06,0xc1,0x6c,0xd3,0xda,0xf8,0x4d,0xf3,0x99,0xb7,0xc0, +0x42,0x68,0x87,0x91,0xcf,0x14,0x3c,0xbb,0x4e,0xa7,0x93,0x4a, +0x32,0x8c,0xba,0x9d,0x71,0x9c,0x91,0x19,0x4f,0x7e,0x3d,0xa8, +0x14,0x6b,0x54,0xdf,0x7c,0xdf,0xd9,0xc5,0xc6,0x25,0x82,0x19, +0x52,0xe7,0x20,0x0d,0xf6,0xd5,0xca,0xc6,0x08,0x56,0x01,0x65, +0xa1,0x92,0x3c,0x50,0xd5,0x4c,0xc4,0x4e,0xd9,0x87,0xa0,0xc0, +0xe2,0x7a,0x48,0x5e,0xb8,0x6c,0xe2,0xf6,0xa3,0x13,0xb0,0x5d, +0x9a,0x60,0xfe,0x07,0x59,0x4e,0x5f,0x79,0xd4,0x44,0x5a,0x97, +0xe9,0xdb,0xeb,0x55,0xbc,0x39,0x8f,0xb3,0x4d,0x70,0x39,0x5f, +0xad,0xe6,0x37,0x71,0x0e,0x1f,0x39,0x0c,0x07,0x8f,0xbf,0xbf, +0xf8,0x22,0xf2,0x9b,0x04,0xd6,0xe3,0x9e,0xc1,0x90,0x7a,0x30, +0x5b,0xb5,0x67,0xf3,0x09,0xad,0xaf,0x41,0x63,0x98,0x0f,0xbd, +0x85,0xa4,0x84,0xe2,0x47,0x7c,0xe7,0xcb,0x97,0x8a,0xfa,0xb8, +0x66,0xd9,0x74,0x71,0x75,0xa2,0xaa,0xeb,0xe3,0xfa,0x27,0xef, +0xe2,0x4a,0x27,0x0f,0x76,0x48,0xcc,0x96,0xc7,0xf6,0x26,0x0b, +0xf2,0x0c,0x26,0x81,0x19,0x64,0x5e,0x40,0xee,0x05,0x64,0xc3, +0x5f,0xf8,0x6f,0xef,0x08,0x37,0xe2,0x03,0x26,0x1b,0x94,0x8e, +0x09,0xfd,0x45,0x4c,0x55,0xc0,0x17,0x56,0x03,0x5f,0x1d,0xfd, +0xa4,0xf9,0x86,0x3e,0x2f,0xe2,0x96,0xb7,0x21,0xb0,0x0b,0xbe, +0x7e,0xbb,0xa1,0x79,0x25,0xa7,0x7f,0xdd,0x25,0x2f,0xc5,0x0d, +0x5e,0x95,0x2c,0x31,0xa3,0x25,0x26,0xed,0xcd,0x17,0x80,0x0b, +0xc3,0x9a,0x02,0x36,0xf8,0x02,0x0b,0xb7,0x9d,0x7f,0x91,0x53, +0x52,0x4e,0x49,0x39,0x26,0x6d,0xce,0xa1,0x6e,0x84,0xea,0x18, +0xa8,0xf3,0x9c,0x92,0xf2,0x8e,0x40,0x6d,0x44,0x0f,0x53,0x7e, +0xae,0x3f,0x72,0xdf,0x39,0x1e,0xd3,0x44,0xb0,0x51,0x71,0x11, +0x83,0x02,0xf3,0x1c,0xb5,0x95,0x63,0x5a,0x4e,0x0b,0x96,0x1e, +0x50,0x75,0xbe,0x51,0xe4,0x6d,0xd0,0x65,0x1a,0x12,0x72,0x45, +0x5c,0x0e,0x09,0x9b,0x0b,0x8b,0x34,0x82,0xb8,0xb0,0x08,0x23, +0x08,0x21,0x85,0x32,0xe5,0x37,0xa4,0x73,0xff,0x81,0xc8,0x77, +0x2e,0x5a,0xd8,0x75,0x64,0xd9,0x5f,0x80,0x15,0x7d,0xd1,0x9b, +0x6c,0xc8,0x04,0xee,0x10,0x8f,0x9b,0x13,0x94,0x0f,0xec,0x1c, +0xf8,0x99,0x93,0xf2,0x5a,0xce,0xe7,0xab,0x1a,0xb9,0xc2,0x3e, +0xc5,0x6c,0x7c,0xab,0x3c,0xb6,0x17,0x50,0x22,0x49,0x98,0x07, +0xc2,0x44,0x8d,0xa6,0x88,0xc5,0x81,0x6a,0x6b,0x51,0x7e,0xf0, +0x51,0x36,0x44,0x74,0x9b,0x82,0x62,0x34,0xa8,0xae,0x54,0xc4, +0x12,0x53,0xf3,0x2f,0x08,0xe1,0x5d,0x11,0x78,0x05,0x16,0x9e, +0x47,0xc8,0xbe,0x03,0x39,0xe3,0x93,0x13,0x99,0x48,0x1c,0xa2, +0xa4,0x07,0x36,0xb4,0x98,0xca,0xe9,0xdf,0x22,0x32,0x3c,0xf0, +0x60,0x8d,0x40,0x52,0xa6,0x56,0x7d,0xaa,0xc1,0x7a,0x09,0x0a, +0x12,0x99,0xb3,0x29,0x12,0x4f,0x58,0x55,0x21,0xc8,0x96,0x5f, +0xac,0x41,0x91,0x0c,0x0e,0x39,0x0b,0xe1,0xb0,0xd7,0xab,0xd2, +0x1b,0xdd,0x7e,0x5f,0x07,0xe1,0x35,0x4c,0x78,0x46,0xb0,0xaf, +0xb5,0xdf,0x75,0x31,0xe7,0x4d,0x70,0x06,0xd3,0x37,0x63,0x24, +0xe1,0xe0,0x61,0xc6,0x63,0x0e,0xfe,0xc5,0xc7,0xbe,0xf5,0xbc, +0x21,0xb3,0x20,0x2c,0x00,0x54,0xd2,0x7d,0x56,0x00,0xaa,0x0c, +0x2d,0x01,0x34,0x82,0x7b,0xac,0x01,0x0c,0x82,0xdc,0x42,0x70, +0x8f,0xdb,0x1e,0xca,0x9c,0x47,0xf5,0x86,0xad,0x83,0xc6,0x76, +0xa8,0x1f,0x68,0x93,0xe6,0x3e,0x66,0xbc,0x6f,0x11,0xa3,0x64, +0x88,0xfd,0x79,0x2d,0x1b,0x9e,0x6c,0xf7,0xa1,0x45,0xea,0xc1, +0x46,0xfa,0x6e,0x42,0x21,0x25,0x14,0xc2,0x2c,0xaa,0x76,0x90, +0x44,0xab,0xdf,0x80,0xd6,0xc2,0x16,0x39,0x7c,0x76,0x65,0xb9, +0xf9,0xfe,0xfc,0xf6,0x92,0x5e,0x16,0x74,0x52,0x0a,0x30,0xfc, +0x82,0x5a,0x01,0xec,0x27,0xe5,0x81,0x6b,0x43,0xfe,0xb8,0xbe, +0xb9,0xa4,0x4d,0x9c,0x8b,0x52,0x62,0x01,0x92,0x1d,0xdf,0x0b, +0x90,0x9c,0xa8,0x8c,0x22,0x2b,0xe3,0x62,0xc6,0x88,0x4f,0x07, +0xad,0xe6,0xb0,0xef,0xf5,0xbb,0x7f,0x9b,0x34,0xff,0xd6,0xee, +0xff,0x6d,0x72,0x72,0x47,0x7f,0x9a,0x3e,0xa4,0x0d,0x92,0x6f, +0x86,0x94,0x4f,0xef,0x65,0x9e,0xbe,0xed,0xb9,0x35,0x16,0xea, +0x72,0xe9,0x41,0xff,0xa1,0x41,0xc5,0x15,0x45,0xe5,0xb8,0x7d, +0xa9,0xc6,0x61,0x83,0x9f,0x2e,0xc4,0xe7,0x45,0x57,0xf8,0x92, +0xdd,0xf2,0xed,0xe5,0xc5,0x6c,0x74,0x93,0x64,0xf8,0xe4,0x9d, +0x87,0x47,0x13,0xa7,0x7f,0xf7,0xfe,0x7f,0x77,0x90,0xfc,0x37, +0xef,0xee,0x3a,0x9b,0xfe,0xcd,0xf3,0xc5,0x25,0xe5,0x92,0x5e, +0xbe,0x73,0x19,0xdd,0xad,0x6a,0x7f,0xf7,0x12,0x4d,0x07,0xf2, +0xa5,0x87,0x9a,0x01,0xe2,0xd9,0x7c,0x3a,0x5f,0x56,0x14,0x6e, +0xb0,0x17,0x34,0xd3,0x42,0xd6,0x74,0x3b,0xcd,0xd8,0xaa,0x2e, +0x55,0x46,0xc9,0xdd,0xaa,0x2e,0xec,0x56,0xf5,0x96,0x4f,0x3c, +0xd8,0x0e,0x0b,0xbd,0x46,0x58,0xe2,0x0a,0xcc,0xaa,0xcf,0x92, +0x51,0x96,0x5c,0x4c,0xf8,0x2d,0xe2,0x6a,0xcf,0x2e,0xe5,0xcd, +0x45,0xe7,0x63,0x08,0x5e,0xe1,0xb7,0x65,0x63,0xa1,0x7d,0x16, +0x9d,0x88,0x1f,0x24,0xcc,0x15,0x4f,0x1c,0x2a,0x18,0xcc,0xde, +0x06,0xe3,0xf5,0x65,0x3a,0xde,0x01,0x45,0xf9,0x5b,0xbc,0x60, +0xb9,0x03,0x28,0xc3,0x7b,0xc0,0xc9,0x66,0xb1,0x03,0x04,0x72, +0xd5,0xf5,0xd7,0x5d,0xb5,0x11,0x00,0xe0,0xc2,0xf7,0x86,0x81, +0x2e,0x5d,0x9a,0xbf,0x83,0xcb,0xd1,0xf8,0xbd,0x4e,0xc4,0x0f, +0x7a,0x14,0x61,0xbc,0x0b,0x25,0x03,0xc8,0x4b,0xa1,0x94,0x72, +0x03,0xd3,0xae,0xe6,0x66,0x23,0x65,0x4f,0x2e,0x1d,0x87,0x63, +0xbe,0x5e,0xe9,0x1a,0xe4,0x3a,0x5d,0x00,0x40,0x2d,0x48,0x6f, +0x58,0x19,0xf4,0x3a,0x4f,0xd0,0x80,0xd4,0x16,0x62,0xb0,0x27, +0xdb,0x02,0x01,0x02,0xeb,0x15,0x90,0x02,0x24,0xbe,0x5f,0x8a, +0xa3,0x8f,0xba,0x57,0x63,0xc0,0x61,0xa2,0x1e,0xf2,0xc7,0xdf, +0xbc,0xbe,0xc5,0x77,0x97,0x5b,0x0d,0x0c,0x54,0x8b,0x6e,0x04, +0x7d,0xca,0x70,0x1f,0x5b,0xee,0x62,0x5a,0x70,0x53,0x99,0x8f, +0x6f,0x2a,0x77,0xb1,0xa9,0x3a,0xf4,0x56,0x2c,0xe4,0xf0,0x53, +0xd9,0xfc,0x4e,0xb6,0x23,0x4f,0x37,0xb1,0xcd,0x2f,0x02,0xbb, +0x61,0x30,0xcd,0x2a,0xdd,0x6b,0xd3,0xd1,0xcd,0xc2,0xbb,0xf1, +0xd4,0xed,0x14,0x9a,0xbe,0x79,0xa0,0xe9,0x83,0x76,0x3e,0x23, +0xe5,0xc5,0xa5,0xd6,0xe9,0x41,0x44,0xaf,0xb8,0x16,0x46,0xcf, +0x77,0xe3,0x69,0x61,0xec,0x40,0x4a,0x11,0x26,0x2b,0xc1,0x64, +0x45,0x98,0xef,0x47,0x45,0x9d,0x0d,0x29,0x45,0xbd,0x4e,0xaf, +0xa8,0x14,0x74,0x06,0xa6,0x21,0x5c,0x45,0xe4,0x34,0xe6,0x27, +0x77,0x0f,0xa9,0xe5,0xc9,0x7c,0x4c,0x6d,0x69,0x8f,0xf1,0x5a, +0x49,0xf2,0xcd,0x34,0xc1,0xaf,0x1f,0x5f,0xa2,0x47,0xe9,0x2c, +0x6b,0xab,0xf7,0xb7,0x6f,0xdf,0x06,0xf8,0x7c,0xba,0x52,0x8f, +0xbb,0x91,0x6b,0x23,0x5f,0x25,0xbc,0x6d,0x67,0xc9,0xea,0xcb, +0x15,0x7c,0x5d,0xae,0x57,0x89,0xd7,0xd0,0x65,0x1b,0x81,0x80, +0xf4,0x58,0x0d,0xbf,0x35,0x68,0xdb,0x97,0xd0,0x35,0xbf,0x8c, +0xa6,0xe8,0xc1,0x9e,0xcd,0xa7,0xe9,0x84,0xa2,0x77,0xaa,0x4d, +0x32,0x09,0xf8,0x62,0xe2,0xb8,0xad,0xfa,0x2b,0x18,0x10,0x80, +0x6c,0xd3,0xb5,0xd3,0x9f,0x6b,0xa7,0x52,0x5f,0x91,0x47,0x1a, +0xa9,0x22,0x4c,0x9b,0x7e,0xa5,0xb8,0xac,0xd0,0x1a,0x26,0xe8, +0x65,0xa3,0xc9,0xcf,0xb0,0xaa,0x84,0x66,0xc3,0x97,0xc0,0x08, +0x92,0xc3,0x5f,0x90,0x9c,0xbd,0x4f,0x3e,0xbc,0x56,0xa9,0xf8, +0x81,0x69,0x14,0xac,0x52,0xa5,0xe1,0x07,0x24,0x36,0xf4,0xfe, +0x40,0x89,0xec,0xf8,0xe3,0xa8,0x1b,0x05,0x97,0xdd,0x30,0x18, +0xc3,0x7f,0x13,0xf8,0x9d,0xc0,0xdf,0xab,0x6e,0xb8,0x2d,0xcc, +0x7a,0xaf,0x74,0x6f,0xb8,0xb2,0xa0,0xd3,0x51,0x1e,0xa6,0xa3, +0x1c,0xc6,0xbb,0x38,0xe6,0xf2,0x47,0x9b,0x5d,0x49,0x2b,0x74, +0xb8,0x4e,0xa1,0x0d,0x24,0xe3,0xb6,0x80,0x81,0x66,0x68,0xdd, +0x25,0x9b,0x35,0xbc,0x1d,0xe5,0xee,0xd6,0x10,0x8c,0x3e,0x55, +0xe7,0x9a,0x2e,0xb8,0xa6,0x17,0x18,0xd9,0x8d,0xca,0xe0,0x1e, +0xb8,0x6f,0x07,0x1b,0xcb,0xb6,0x36,0x61,0xe3,0xeb,0xf9,0x72, +0x52,0x79,0xd1,0x6e,0x99,0x30,0x88,0x44,0xdc,0x78,0x0f,0xeb, +0xd4,0x4d,0x48,0xee,0x92,0xa0,0x30,0xe8,0xe5,0xe0,0x0c,0xdb, +0x48,0xbf,0x5e,0xae,0x6f,0x88,0x58,0xfa,0xa0,0x07,0xe0,0xcd, +0xd1,0xd3,0xcc,0xd7,0x05,0x38,0x47,0xdc,0x9d,0xa8,0x66,0x53, +0x8a,0x7e,0xbd,0x27,0x9f,0x39,0xf2,0x6f,0x30,0x4e,0x14,0x1b, +0xf2,0x07,0x13,0xa7,0x07,0x48,0xdc,0x34,0x63,0x16,0x41,0x3c, +0x2d,0x7b,0x37,0xec,0x69,0x0a,0x64,0x35,0x56,0xa8,0x4f,0x31, +0x48,0x93,0xe3,0x07,0xef,0x9b,0xf1,0x86,0x1d,0xcf,0xe6,0xcb, +0xd5,0xbf,0x52,0xed,0xc7,0xc7,0x56,0x89,0xfa,0xd3,0x31,0x53, +0xc0,0xd3,0xd5,0x0e,0x46,0x43,0xc3,0x85,0xc1,0x25,0x1d,0x5e, +0x23,0xdc,0x4b,0xc5,0x26,0x72,0x62,0xb0,0xd0,0x97,0xce,0xb4, +0xe8,0x08,0x6d,0xb2,0xa7,0x5a,0x8d,0xce,0x33,0xad,0x87,0x9a, +0xcd,0x07,0xd5,0x0c,0x75,0xbf,0x8f,0xbd,0x7f,0xfe,0xaf,0x96, +0x3c,0xa4,0x7e,0x32,0xc3,0x38,0xe4,0x9b,0x4a,0xbe,0x86,0xf1, +0xc6,0x61,0x2c,0xbf,0x32,0x96,0xc6,0x86,0x54,0x5a,0xc9,0xbf, +0x8b,0x1d,0xf2,0x07,0x13,0x62,0x7b,0x70,0xab,0x7a,0x01,0xbf, +0x27,0x90,0x30,0x42,0x7c,0x23,0xb0,0xb5,0x9b,0xf1,0xed,0xc9, +0xfb,0x9e,0x96,0x12,0x00,0x68,0xc2,0xa4,0xd4,0x04,0x98,0xf8, +0x23,0xfb,0x5e,0x4e,0x52,0xec,0x23,0xf9,0xfd,0x0e,0x3d,0xea, +0x96,0xab,0x2f,0xc9,0x5b,0x73,0x14,0x06,0xc9,0x6c,0x22,0xbf, +0x23,0x76,0x4b,0xee,0xde,0x6e,0xb7,0x1a,0x95,0x83,0xc3,0x94, +0xdb,0x58,0xe5,0x36,0x52,0xcc,0xdb,0xe0,0x11,0xd6,0xe9,0xfb, +0x2d,0xfa,0x1b,0x09,0x3f,0xb6,0xbc,0x5b,0x60,0x58,0x81,0xdf, +0xef,0xe2,0xb4,0xc0,0x05,0xb9,0xf2,0x68,0xda,0xc0,0x4d,0x78, +0xa7,0x36,0x7f,0xad,0x9c,0x77,0x94,0x93,0x0e,0x7b,0x72,0xc7, +0xaf,0x4d,0x95,0xeb,0x9b,0x98,0xec,0x6d,0x7d,0x7c,0xcc,0xe2, +0x2e,0x6e,0x2c,0x16,0xe0,0x17,0x36,0x5c,0x5f,0x6d,0x35,0x73, +0xa2,0xda,0x69,0x96,0x9b,0x94,0x5d,0x95,0xeb,0xdc,0x3a,0x54, +0x77,0x1e,0xfd,0x2d,0x4a,0xc9,0x33,0xaa,0xe6,0xf8,0x78,0x99, +0x90,0x34,0xb9,0x31,0x0e,0x29,0xe5,0xa3,0x50,0xb2,0x5b,0xda, +0x18,0x8f,0xe7,0x8d,0xda,0x36,0xb1,0xcd,0x51,0xdb,0x69,0xd6, +0x69,0x27,0xf0,0x2e,0x5d,0x88,0xcb,0x22,0x04,0x0a,0x25,0xad, +0xcf,0x09,0xa3,0x0c,0x76,0x11,0xdb,0x60,0x16,0x98,0x11,0xe5, +0x0e,0x9a,0xc0,0x50,0x11,0xb0,0x92,0xfa,0x88,0xe1,0xb7,0xa8, +0x13,0xe3,0x50,0xdf,0x43,0xc3,0x1c,0x99,0x91,0x0e,0x8b,0x2a, +0x33,0x8b,0x65,0xfc,0xc4,0xb8,0x1d,0xcb,0x3f,0xf5,0xae,0x16, +0xab,0x24,0xd1,0x47,0x64,0x9f,0x50,0x92,0xdf,0x65,0xb8,0x2d, +0x7f,0xb6,0x15,0x19,0x07,0x55,0xa8,0x80,0x37,0x3b,0xd0,0x2b, +0xe1,0x14,0xfc,0x86,0x27,0x87,0x55,0x61,0xc1,0xef,0xaa,0xc5, +0x80,0xd9,0x15,0x69,0x8e,0x1f,0x5e,0x97,0x29,0xa2,0xab,0x2b, +0xd6,0xa3,0x41,0xec,0xaa,0xb8,0x3b,0x0f,0xaf,0x47,0xe0,0x55, +0x25,0x46,0xaa,0xed,0x9a,0x18,0x48,0x55,0x33,0x2e,0x54,0x61, +0x22,0x66,0x51,0xc6,0xdd,0x9d,0x99,0xd6,0x04,0xab,0x2a,0x59, +0x64,0x82,0x2e,0xc9,0x19,0x4e,0x49,0xa7,0x69,0xf6,0x74,0x0a, +0xda,0x64,0x5c,0x1d,0x30,0x66,0x99,0x2c,0xd6,0x53,0xb0,0xe7, +0x71,0xfb,0xb8,0x3c,0xfb,0xe3,0x2a,0x0b,0x37,0x81,0x71,0x47, +0x8c,0x2c,0x3b,0x4c,0xe0,0xad,0xf2,0xa3,0x38,0xe6,0x32,0xb2, +0xad,0x48,0x39,0xe3,0x4d,0x0b,0x13,0xdb,0x18,0x77,0x40,0x52, +0x72,0x4e,0x01,0x7b,0x7b,0x16,0x47,0xd6,0x53,0x4e,0x93,0xcd, +0xc9,0x64,0xd3,0x9c,0xe4,0x27,0x13,0xde,0x18,0x5e,0x5d,0x27, +0xab,0x91,0x3e,0xcf,0x9f,0xcc,0xd4,0x5b,0x17,0x8c,0xe5,0x1a, +0x07,0x2f,0xa4,0xc2,0xff,0xd5,0x18,0x23,0xb4,0x8b,0x4d,0x2b, +0x9e,0xa0,0x37,0x26,0x7f,0xe5,0xf0,0x95,0xc3,0xd7,0x51,0xb8, +0x75,0x88,0x3d,0x3e,0x4e,0xb3,0x6f,0xd3,0x59,0x0a,0x86,0x1a, +0x60,0x76,0x50,0x53,0xfe,0x33,0x1b,0xff,0x0e,0xc4,0x5b,0xb1, +0x40,0x8f,0x2c,0xaa,0xac,0xa8,0x39,0x93,0xe5,0xe8,0xed,0xcd, +0xfc,0x36,0xc1,0xfd,0x4f,0xc0,0x4c,0xd6,0x06,0xac,0x97,0xc0, +0xbe,0x06,0x86,0x00,0x12,0xf3,0x9d,0x07,0xd4,0x25,0x14,0xf3, +0xf5,0x06,0x2d,0x5b,0xe2,0x22,0x14,0x0f,0x46,0xd3,0xc5,0xf5, +0x28,0x50,0xf7,0xec,0x33,0x7a,0x3b,0x07,0x45,0x0f,0xd5,0x0d, +0x56,0x97,0x71,0x49,0x54,0x3b,0x84,0x0a,0x71,0x02,0x34,0x98, +0x50,0x30,0x75,0x37,0x68,0xea,0x69,0x04,0x0d,0x58,0x6e,0xbe, +0x87,0x3f,0x30,0xf7,0x34,0x30,0xbe,0xd0,0x6f,0x49,0x3c,0x88, +0x82,0x68,0x18,0x5c,0x2d,0x53,0x22,0x35,0x6e,0x3f,0xa5,0xb3, +0xc1,0xaf,0xd5,0x8d,0x7f,0x63,0xa9,0x11,0xfa,0xef,0xad,0x3c, +0x02,0x7c,0x29,0x64,0x54,0x01,0xaa,0x3c,0xa1,0x30,0x6e,0x9d, +0x85,0x78,0x46,0x7c,0x8b,0x26,0x6c,0x3b,0x0a,0xa8,0x67,0xe3, +0xf6,0xe7,0xc4,0x4a,0xb2,0xa7,0xd4,0xd9,0xb0,0xea,0x4a,0xe6, +0x05,0xd2,0x6c,0xcb,0x28,0xf4,0xa0,0x47,0xdc,0x38,0x01,0x6a, +0x9f,0xe2,0x79,0x52,0xf8,0x48,0x1d,0x41,0x32,0x17,0xa1,0x79, +0x9e,0xc4,0x59,0xc0,0x96,0x32,0xef,0xba,0xf4,0x6f,0x1c,0x82, +0xc9,0x71,0x14,0x92,0x6d,0xfd,0x2b,0x98,0x87,0xf3,0x20,0x0b, +0x56,0xc1,0x34,0x78,0x4f,0xe7,0x1a,0x33,0x92,0x5d,0xed,0x63, +0x7c,0xe3,0xda,0xb1,0x6a,0xe7,0xfe,0x46,0x76,0xee,0xe7,0xb1, +0x32,0x59,0x83,0x2c,0x9e,0xb7,0xd5,0x34,0x07,0x3f,0x65,0x3e, +0xdc,0xc4,0xd0,0xc3,0xad,0x0c,0x7a,0x39,0x87,0x5f,0x39,0xfc, +0xca,0x03,0x6f,0xaa,0x9e,0x3e,0xc3,0xfb,0x25,0xd3,0x98,0x5b, +0xa2,0x7b,0x13,0x90,0x9d,0x78,0x90,0x6c,0x06,0xc4,0xd4,0xf7, +0x5b,0xba,0xdb,0xd1,0x3c,0x3e,0x9d,0x06,0x9b,0x93,0x78,0x1a, +0xe4,0xf8,0x0f,0x56,0x00,0x08,0xbd,0xf7,0x71,0xd6,0xfe,0x90, +0xe0,0x51,0xce,0x29,0xac,0x58,0xf9,0x57,0x53,0x25,0x81,0x0d, +0x89,0xd5,0xc7,0x38,0x04,0x80,0x9a,0x26,0x17,0x88,0x5a,0xef, +0x41,0x06,0xda,0x79,0x13,0xd3,0x69,0xac,0x41,0x22,0x93,0x23, +0xbd,0xe4,0xd3,0x33,0x6f,0x28,0x26,0x83,0x70,0x78,0x8a,0x4f, +0xd1,0xd1,0xef,0x08,0x7f,0x93,0xa1,0xff,0xde,0xf7,0xed,0x6d, +0xf4,0x79,0xac,0x4e,0x9b,0x82,0x39,0x56,0x03,0x16,0x0d,0xfc, +0xf5,0xa1,0xd6,0x39,0x56,0xe3,0xe5,0xf0,0x99,0xc3,0x27,0xd6, +0xc5,0x32,0x41,0xc5,0x0b,0xa2,0xf3,0xe5,0x18,0x16,0x9f,0x6b, +0x5a,0x57,0xfd,0x1a,0x17,0x4f,0x3e,0x49,0x21,0xa1,0x0b,0x33, +0x0d,0x07,0x11,0x7d,0xdf,0xb1,0x16,0x3d,0x43,0x06,0xc6,0x6d, +0xda,0x24,0x93,0xbb,0xbb,0x5f,0xf9,0xf8,0xc0,0x53,0x6a,0x6d, +0xee,0x2b,0x4f,0x1b,0x53,0xce,0xa1,0x9e,0xca,0xf5,0x3d,0x20, +0x1f,0xba,0x73,0xb1,0xc1,0x06,0xe0,0x8f,0xdc,0xef,0x62,0x5a, +0x2b,0xf6,0x30,0xb5,0x45,0xff,0xc6,0xd8,0x46,0xff,0x44,0x0d, +0x21,0x04,0xe5,0xfc,0x9c,0xf2,0xb1,0x5c,0x6e,0xe5,0xfb,0x3d, +0x16,0x52,0x14,0x6b,0x25,0xa5,0x3c,0x2c,0x2d,0x31,0xc5,0x83, +0x62,0x96,0x7e,0x1e,0x1c,0x87,0xd9,0x0a,0x04,0xba,0xe1,0x82, +0x7e,0x97,0x3e,0x15,0x9a,0x82,0xff,0xc6,0x2e,0x34,0x0c,0xaa, +0xd1,0x88,0xab,0x00,0xa3,0x71,0x37,0xb4,0x77,0x4e,0x88,0x08, +0xa9,0x91,0xe0,0x97,0x4d,0xca,0xd7,0xa5,0x88,0x22,0xfb,0x28, +0xd2,0x25,0x2a,0x22,0x6b,0x6c,0xfa,0x9b,0x6e,0xd3,0x21,0x58, +0x41,0xab,0x3a,0x4d,0x04,0x93,0x12,0x09,0x16,0x55,0x5a,0x99, +0x1d,0x4c,0x95,0x2e,0x71,0x10,0x55,0x0a,0x5a,0x51,0xa5,0xb5, +0xee,0x41,0xf5,0x69,0x68,0x83,0x54,0x25,0x29,0x84,0xa2,0x68, +0x0f,0x42,0x27,0xb0,0x7b,0x09,0x97,0xd9,0x4c,0x6a,0x50,0xfa, +0xfb,0xa0,0x2a,0x14,0xb0,0x41,0x26,0x29,0x0a,0x1b,0x4f,0x01, +0x07,0xe1,0x62,0x50,0x83,0x89,0xbe,0x15,0x1e,0x56,0xed,0x07, +0xe1,0xd9,0x20,0x0e,0x82,0xef,0x73,0xa9,0xcd,0x79,0x08,0xed, +0x0d,0xbb,0x1b,0xf2,0xef,0xe3,0xb1,0x49,0xd3,0xa5,0x1a,0x9c, +0x32,0x77,0xda,0x93,0xc8,0x86,0x36,0x5e,0xdb,0xf4,0x64,0xa0, +0x67,0x66,0x29,0xdf,0x57,0xd4,0xf1,0x20,0x56,0x83,0x86,0x9e, +0x43,0xa8,0xb2,0xb2,0xc8,0xd9,0x83,0x96,0xd4,0x80,0x69,0x86, +0x71,0x5f,0x68,0x52,0x36,0xfb,0x2a,0xb0,0xe4,0x40,0x0d,0x7e, +0x89,0x47,0x14,0xea,0x97,0x97,0xfa,0xb4,0x0e,0x0e,0x6e,0x74, +0x92,0xb5,0xd5,0x42,0x17,0x2d,0x2c,0x27,0x0e,0x03,0x83,0x77, +0xe6,0x74,0x3d,0x43,0x5f,0x4d,0x99,0x1b,0x27,0x86,0xe1,0x72, +0x34,0x9b,0xcc,0x6f,0x3c,0xff,0x84,0x46,0xac,0x26,0x54,0xd7, +0xcd,0x0e,0x22,0x1a,0x27,0x53,0x6a,0x68,0x1c,0x0c,0xe9,0x2e, +0x9c,0xba,0xb6,0x66,0xd7,0xad,0x36,0x52,0xde,0xd1,0x0c,0xca, +0xf9,0x64,0x70,0xa9,0x69,0xf4,0xdd,0xb0,0x67,0xe0,0xd5,0x74, +0xea,0xf8,0x2f,0xab,0x89,0xd5,0x0f,0x6c,0x40,0x59,0xc0,0xb9, +0x80,0x5c,0xda,0xdf,0x9a,0xad,0x41,0x55,0x20,0x1d,0x92,0x51, +0x85,0x3b,0x43,0x3a,0x11,0xcc,0x80,0x9d,0x53,0x7f,0xf0,0x41, +0xcd,0x82,0xc1,0xb5,0x9a,0x03,0x4b,0x9e,0x9a,0xce,0xbc,0xc3, +0x57,0x09,0xc1,0xbe,0x90,0x29,0x18,0xe0,0x76,0x98,0x0f,0x0d, +0x3e,0xe2,0xb2,0xc2,0xc2,0x4a,0x03,0x40,0x22,0xd5,0x4f,0x41, +0xae,0x3e,0x87,0x7e,0x55,0x29,0xe6,0x05,0x95,0x92,0x95,0xbf, +0x2a,0xc5,0x9f,0x50,0xaa,0xd9,0xd4,0xbc,0x65,0xd2,0x28,0x45, +0x98,0xc8,0x29,0xa5,0xa6,0x59,0x53,0x23,0x8b,0x15,0x4e,0x79, +0x54,0x0b,0xba,0x86,0x8b,0x0c,0x37,0x36,0x8d,0xe0,0x03,0x0c, +0x01,0x05,0x91,0x33,0x44,0x6e,0x41,0xe4,0x8d,0xe0,0xda,0x82, +0x58,0x08,0x12,0x99,0x43,0x4d,0xba,0x14,0x95,0xb9,0x93,0x2e, +0x05,0x28,0x0b,0x08,0x45,0xac,0x42,0x67,0xd9,0x1a,0xdd,0x2f, +0x72,0xda,0x36,0x9f,0xe2,0xa6,0x0d,0x6a,0xf9,0x85,0xe9,0xce, +0x48,0x0b,0x8e,0x1a,0x95,0x58,0x6c,0x24,0x48,0x9f,0x36,0xe0, +0x76,0xd0,0xa7,0xd4,0x7f,0x89,0x3e,0xdb,0xfa,0x63,0xfa,0x14, +0xe8,0x7d,0xe8,0x73,0xb0,0xd8,0x48,0x8c,0xb9,0x55,0x47,0x9d, +0x65,0x8c,0x39,0x9e,0xca,0x5c,0x88,0xa8,0xe2,0xdf,0x16,0x3d, +0x46,0x22,0x6a,0x1c,0x5b,0xac,0xd2,0xfc,0xd3,0x35,0xeb,0xf5, +0x12,0xc7,0x59,0xf2,0x54,0xed,0x39,0x1b,0x15,0xef,0xb5,0x23, +0xdf,0x68,0xd5,0xf9,0x62,0x0f,0x74,0xa8,0x81,0x71,0x09,0x55, +0x75,0x24,0x09,0xc9,0x77,0x77,0x1e,0xe5,0x82,0x3e,0xbf,0x4c, +0xae,0x61,0x7a,0x9a,0x2f,0x09,0xdc,0xf3,0x25,0x92,0xb0,0x13, +0x86,0xa2,0x8d,0x62,0x8c,0xd9,0xa4,0xd4,0x79,0x01,0xdd,0x08, +0x0a,0x16,0xec,0xd7,0x2a,0xdf,0x80,0x6b,0x48,0x59,0x0a,0x9a, +0x1c,0x58,0xa1,0xec,0x40,0x03,0xb9,0x30,0x60,0xaa,0xa6,0xc3, +0x34,0x23,0x14,0x37,0xf3,0x75,0x96,0x00,0xc2,0x65,0x0d,0x92, +0x1f,0x54,0xbe,0x6f,0x81,0xaf,0xeb,0x28,0xff,0x41,0xb2,0x7d, +0xee,0x68,0xa4,0xcf,0x0f,0x6e,0xe7,0xe9,0xe4,0x41,0xe8,0x77, +0xf1,0x8b,0x76,0x0f,0x96,0x09,0x6e,0x86,0xf2,0xc4,0xc7,0xab, +0xcd,0xa0,0x01,0xe2,0x64,0x5c,0x1a,0xeb,0x96,0x8c,0x71,0x27, +0x2c,0xd6,0x68,0x2f,0x14,0xe3,0xa8,0x67,0xf6,0x26,0xae,0xd3, +0x64,0x39,0x5a,0x8e,0xaf,0xf3,0x9a,0xfd,0x89,0xf1,0x7a,0x29, +0xfb,0x13,0xc1,0x24,0x59,0x80,0x8e,0xe6,0x95,0x01,0xdf,0x58, +0x41,0xbf,0xbb,0x2c,0xa6,0x3f,0x80,0x9a,0x1b,0xa3,0x11,0x06, +0xa6,0x10,0xfb,0x9e,0xe1,0x52,0x9e,0x3e,0x63,0x0b,0x13,0xcf, +0x24,0xb4,0x97,0x11,0x30,0x3e,0xd0,0x49,0x33,0xc6,0xa9,0x3d, +0x70,0xcc,0x4c,0x8d,0xae,0x71,0xb4,0x26,0x1a,0x93,0xae,0x6c, +0xab,0xba,0x71,0xc4,0xdd,0xd2,0x91,0x00,0xe1,0x6e,0xea,0xe5, +0xc6,0x24,0x56,0x6d,0x60,0x94,0x38,0x94,0xde,0x49,0x23,0x70, +0x57,0x60,0xb4,0xc4,0x55,0x3c,0x91,0x3a,0x56,0x57,0x17,0x83, +0xdb,0x66,0x3c,0xe1,0xad,0xca,0x1e,0x6e,0x26,0x1d,0x1f,0x8f, +0x79,0x63,0x14,0xff,0xf1,0x79,0x1f,0x19,0xa9,0x44,0x02,0xe8, +0x23,0xbe,0x15,0xa7,0xc6,0x8a,0xac,0xa6,0x04,0xd1,0xa8,0xe5, +0xcd,0xdd,0x5d,0xe8,0xdb,0xbb,0x2a,0xf6,0xe6,0x2c,0x95,0xb5, +0xb8,0x6f,0xf1,0x1d,0x1b,0xed,0xb0,0x00,0xdb,0xcf,0x3a,0x88, +0xbf,0x0d,0x1f,0xb1,0x6f,0x14,0x27,0x0d,0x23,0x89,0x8d,0x25, +0x7e,0x41,0xd3,0x55,0xb5,0xaa,0x2c,0xb1,0x4c,0xb4,0x8f,0x6a, +0xe0,0xed,0xfd,0xda,0x55,0xc9,0xb2,0xe0,0xd6,0x34,0x55,0x63, +0xc0,0xcd,0x1b,0xa2,0x4f,0x76,0x2b,0x14,0x06,0xd5,0x8b,0x93, +0x20,0x54,0xbd,0xc7,0x2b,0x2f,0xde,0x98,0xe7,0x18,0xf2,0x22, +0xf1,0x1a,0xd9,0x4b,0x48,0x0f,0x34,0xb7,0x2a,0x00,0x9e,0x69, +0xde,0x11,0x4d,0x15,0x10,0xbf,0x90,0x14,0x08,0x11,0x3a,0x95, +0xc4,0xe1,0xf0,0x2d,0x4b,0x58,0xa1,0xe9,0xa2,0xbc,0x3f,0xb9, +0x35,0x09,0x46,0x86,0x0f,0x5c,0x50,0x08,0xb4,0x83,0x53,0xa5, +0xda,0x78,0xb9,0x4d,0x07,0x21,0x65,0x50,0x07,0x23,0x25,0xd9, +0xe8,0x44,0x2c,0xec,0x57,0x1c,0xe6,0xc6,0xff,0x5a,0x09,0x0d, +0x39,0xf7,0x85,0x3e,0x7b,0xe0,0x99,0xd2,0xf6,0x76,0x28,0x8c, +0xb9,0x55,0xea,0x2e,0xc8,0xaa,0x8c,0x75,0x12,0xa5,0x4d,0x40, +0x51,0x54,0x77,0xc9,0xbd,0xd6,0x2d,0xe4,0x4f,0x4c,0xbb,0x9d, +0xb1,0xd1,0x35,0x18,0x5c,0x9a,0xbf,0x36,0x18,0xfa,0x99,0x7f, +0xe6,0x31,0xa4,0xee,0x1e,0x27,0x2c,0x84,0x74,0x39,0x9c,0x76, +0x2f,0x68,0x07,0x65,0x13,0x1b,0x01,0xee,0x4f,0x36,0xa7,0xe6, +0xab,0xab,0x6f,0xec,0x6b,0xfa,0xc7,0xb1,0x3d,0x80,0xa0,0x15, +0xf1,0x98,0x61,0x4f,0xb8,0x49,0x78,0xfe,0x34,0xb1,0xb7,0x2f, +0x91,0x5c,0xcf,0xec,0xea,0xd6,0x8c,0xf2,0xc9,0xfd,0x47,0xb9, +0xd1,0x85,0xb4,0xf8,0xb8,0x19,0xa1,0x37,0x28,0x57,0x67,0x91, +0x68,0x0e,0x84,0xa3,0xe6,0xc4,0x90,0xa5,0x7b,0x8b,0x0f,0x24, +0xcd,0xd8,0xb4,0x04,0xd8,0xf1,0xfa,0x37,0x37,0xa6,0xac,0x8e, +0xa4,0x67,0xaf,0xc3,0x40,0x99,0xf7,0x6a,0x83,0xcb,0xb4,0x19, +0x01,0x7c,0x7b,0x4c,0x9b,0x89,0xa9,0x62,0xb2,0xf2,0xec,0x6d, +0x55,0x73,0x8e,0x2d,0x94,0xde,0x7f,0xff,0x44,0x17,0x55,0x7b, +0x28,0x15,0x9a,0xe0,0x67,0x9e,0x8e,0x35,0xa8,0x35,0x58,0x1c, +0xe9,0x4e,0xab,0xb7,0xfa,0x21,0xdd,0xba,0xcf,0x43,0x82,0x50, +0x75,0xe7,0xd8,0xba,0x38,0x69,0x6b,0x58,0x28,0x4d,0xcc,0xa5, +0x30,0x33,0xcd,0xaa,0x67,0xd8,0xcc,0xb1,0x67,0xdf,0xfc,0xac, +0x79,0xea,0xc2,0x00,0xd0,0xc1,0x70,0x15,0x3a,0x75,0x6e,0xda, +0x57,0x3f,0x6a,0x50,0xa9,0x6c,0xbf,0x35,0xc2,0xc0,0x68,0x6d, +0xb0,0x2e,0x59,0x9b,0x64,0x12,0x52,0xc7,0x9c,0xf9,0x5b,0xae, +0xe2,0x7e,0x4f,0x9c,0x54,0x68,0x76,0x4d,0xcd,0xf9,0x3a,0x69, +0xcb,0xd8,0x52,0xc4,0xd0,0x70,0xd4,0xe2,0x5f,0xb1,0x22,0xee, +0x6b,0x3e,0xc1,0xc2,0x52,0x77,0x2b,0xd7,0x07,0x2b,0xdb,0x96, +0xfc,0x82,0xc5,0x67,0xb7,0x12,0x92,0xea,0xd0,0xde,0xe8,0xf8, +0xf7,0xdd,0x50,0x05,0xb1,0x82,0xae,0xcc,0x9c,0xb7,0x3a,0x2a, +0x4f,0xe5,0x65,0x08,0x4c,0x7a,0x08,0x8e,0x46,0xf7,0x47,0x44, +0xd3,0x55,0x38,0x59,0x19,0x4c,0x62,0x4d,0x88,0x73,0x8c,0x6d, +0x9d,0x62,0xc3,0xd8,0xc7,0x03,0x0d,0x34,0x3a,0xc7,0x2c,0xf0, +0xac,0x5d,0xc5,0x93,0xb3,0x30,0xa3,0xb9,0x5c,0xb0,0x70,0xc6, +0xe6,0x84,0x3b,0xfe,0xe7,0xff,0xd2,0x23,0x21,0x4d,0x3e,0x45, +0xfb,0x43,0x31,0xad,0xf7,0x11,0xc5,0xbd,0x27,0x39,0x42,0xc0, +0xd3,0x1b,0x95,0x37,0x64,0x1e,0x86,0xc5,0xc0,0x2b,0x5c,0x3a, +0x85,0x31,0xea,0xb6,0x1e,0x84,0x4f,0x43,0x0b,0x36,0xf5,0x4d, +0xb8,0x8a,0x56,0xb4,0xcb,0xe2,0xf8,0xe3,0xd6,0x32,0x92,0x39, +0x64,0x57,0xd5,0xa8,0xa6,0x1c,0x92,0x28,0x96,0x76,0x71,0x14, +0x48,0x96,0xe9,0xbe,0xd1,0x2d,0x52,0xcb,0xc3,0x9b,0xd0,0xa8, +0x01,0x2e,0xf7,0xb7,0x18,0x49,0xdd,0xd5,0xe8,0x49,0xe1,0x06, +0xb8,0x41,0x3c,0xd8,0xd8,0x38,0x6f,0xe9,0x8e,0x43,0x31,0x85, +0xef,0x53,0xd3,0x35,0xec,0x2c,0xe6,0x1b,0xdc,0x16,0x84,0xdc, +0x76,0xe2,0x06,0xf5,0x54,0x63,0xce,0xda,0x8b,0x64,0x79,0x83, +0x3e,0x64,0x9c,0x22,0xc5,0x35,0xbd,0x16,0x80,0x20,0x10,0x80, +0x9e,0xda,0x08,0x9a,0x07,0xf3,0xab,0xab,0x2c,0x01,0x58,0xfe, +0x5b,0x5b,0x67,0x30,0x53,0xad,0xd7,0xfb,0x44,0xfc,0x8d,0x81, +0xb4,0xac,0x63,0x22,0x7b,0x93,0x0b,0xbf,0xc9,0x77,0xc9,0xe0, +0xd4,0x65,0xd0,0x33,0x65,0x2e,0x95,0xa2,0x9e,0x08,0x74,0x58, +0x2e,0x8e,0x70,0xe5,0x07,0x76,0xb8,0xb3,0x6a,0x2c,0xec,0xdf, +0x32,0x6f,0xea,0x2b,0xe0,0xad,0x48,0x4a,0x07,0xe6,0x9a,0x3e, +0x63,0xd3,0xd7,0xdd,0xf0,0x36,0x9a,0xad,0xed,0x6d,0x57,0x51, +0xb9,0x8a,0xaf,0xc4,0x8f,0xea,0xfa,0x09,0x13,0xbf,0x16,0x27, +0x4a,0x26,0xb7,0x04,0x42,0xa9,0x7f,0x4d,0x96,0x73,0xf4,0x36, +0x2d,0xe5,0xae,0xe5,0xae,0x83,0x9d,0x28,0xb7,0x14,0xec,0x24, +0x1d,0xff,0x89,0x23,0x61,0x09,0x7d,0x87,0x2b,0x0b,0x3c,0xb8, +0xa0,0xa2,0xa2,0x2f,0xb2,0x2d,0x7f,0xb6,0xb9,0x59,0x07,0x21, +0x62,0xd0,0x9a,0x8d,0xf1,0x2a,0xc6,0x64,0xe4,0x48,0xba,0x61, +0x47,0xd2,0x5a,0xb6,0x09,0x59,0x84,0x5d,0x53,0xc5,0xac,0x3c, +0x8c,0x2c,0x86,0x3d,0x90,0x2e,0x16,0xa9,0x5a,0xc2,0x4c,0x67, +0x29,0xb2,0x28,0x45,0xd1,0x65,0x79,0xa7,0xfc,0xb6,0x73,0x33, +0xfd,0x37,0x55,0x7e,0xa3,0x8a,0xe6,0x87,0x15,0xcd,0x4d,0xd1, +0x5c,0x73,0x63,0xbd,0x3a,0xac,0x30,0x02,0xea,0xe2,0xf0,0x21, +0x08,0x8a,0x9a,0xd3,0xea,0x21,0xdb,0x35,0x1a,0xa3,0xfd,0xb0, +0xe7,0x73,0xc5,0x7d,0x66,0xda,0x17,0x76,0x2e,0x8f,0x81,0x55, +0x6a,0x14,0x66,0x01,0xf9,0x0f,0xa3,0xcd,0x73,0x56,0x0c,0x19, +0xfa,0x14,0xd6,0x81,0xfd,0x9c,0x4c,0xd6,0xe3,0xe4,0xe5,0xfa, +0xa6,0x64,0x7a,0xcc,0x76,0x05,0xb7,0x80,0x9a,0x31,0xb4,0x05, +0xfe,0xb9,0xa4,0xf0,0x3e,0xf3,0x05,0x86,0x5f,0xe2,0xdb,0x80, +0x21,0x7e,0xd2,0x86,0x1e,0x27,0x64,0x26,0x30,0x9b,0xd9,0x83, +0x7b,0x17,0xa7,0xca,0x31,0x8e,0xa1,0xce,0xa1,0x50,0xdf,0x83, +0x7f,0x9a,0x31,0x52,0x8c,0x9a,0x03,0xd1,0xf0,0x06,0xc3,0x3b, +0xbc,0xe9,0xc3,0x70,0x26,0x5b,0xb0,0x6b,0x08,0x35,0x40,0x55, +0xba,0x79,0x9d,0x45,0x1e,0x77,0x41,0x7c,0x78,0x69,0x8b,0xd3, +0x8b,0x3c,0x36,0x17,0x6e,0xcb,0xb6,0x97,0xc1,0xb5,0x0d,0x1a, +0xe2,0xb2,0xdd,0xa8,0x1c,0x71,0x32,0xa6,0xd8,0xff,0xbd,0x42, +0xee,0x75,0x77,0x67,0xe9,0xf4,0x7a,0xbe,0x4e,0x56,0xab,0x12, +0x21,0x46,0xf7,0x17,0xba,0x9b,0xef,0xe0,0x69,0x7d,0xce,0x3d, +0x3b,0x18,0x92,0x1c,0xd0,0xf3,0x24,0x55,0x07,0x19,0x2a,0xe8, +0xe5,0x5c,0x98,0x8f,0x41,0xfd,0x40,0x25,0x8b,0xf5,0xc5,0xea, +0xb7,0x37,0x3f,0x07,0x1c,0xb0,0x2e,0x42,0x4c,0x73,0x16,0x19, +0x39,0xaf,0xf0,0xb7,0x45,0x8c,0x79,0x88,0x67,0x26,0x08,0xda, +0x92,0x9e,0xf0,0x4f,0x3b,0x8a,0xf7,0x79,0xb8,0x0d,0x3e,0xa4, +0x6f,0xdf,0x4e,0x6b,0x5b,0x05,0xb3,0x03,0x45,0xc5,0xc9,0xce, +0x70,0x9d,0x0a,0x2a,0x39,0x2c,0x34,0x73,0xa3,0x9a,0x09,0x0d, +0xd6,0x1e,0x65,0xa6,0x71,0x39,0x46,0x09,0x89,0x61,0x6a,0x0a, +0x69,0xef,0x2a,0x2a,0xb7,0x34,0x8b,0x2c,0x39,0xcb,0xa2,0x62, +0x5b,0x35,0x18,0xbd,0x75,0x0b,0x2b,0x60,0x89,0xe9,0x08,0x7f, +0x5b,0x74,0xbb,0xca,0x5c,0x80,0x44,0x50,0x74,0xa5,0xcd,0xce, +0x62,0xcf,0x41,0xd2,0xd2,0x5f,0x2d,0xbe,0x84,0x75,0xea,0x75, +0x60,0x8d,0xea,0xf7,0xd2,0xf3,0xf7,0x50,0xf4,0xbd,0x9f,0x9d, +0x35,0xa5,0xc8,0x7b,0xa7,0xc8,0x7b,0xab,0xc8,0x64,0xd3,0xcb, +0x3a,0x20,0xce,0x67,0x27,0x0e,0xee,0x2d,0x33,0x78,0xde,0xa2, +0x58,0x3d,0x9d,0xd3,0x2c,0x02,0xcc,0x5d,0xe8,0xc0,0xf0,0x7c, +0x8e,0x87,0x0f,0x61,0x5c,0xd7,0x29,0x2d,0x60,0x8a,0xdd,0x11, +0xc9,0x66,0x31,0xb2,0xdf,0xb8,0xbe,0xa7,0x78,0xbd,0x8f,0xa3, +0xd3,0xd9,0xef,0x12,0x2b,0x58,0x74,0xcf,0x9d,0xbd,0x7b,0x84, +0x72,0x40,0x4e,0xe3,0x79,0x79,0x8f,0xbe,0x04,0x15,0xbf,0xaf, +0x91,0x42,0xa7,0xb9,0xbf,0xc1,0xa4,0x51,0x3d,0xdb,0xe0,0x74, +0xc2,0x77,0x3f,0xf4,0xaa,0x38,0x5b,0x61,0x20,0xec,0x9b,0x4a, +0x0b,0x55,0xe7,0x8a,0x95,0x6a,0xf6,0x52,0x61,0x39,0x1b,0x6c, +0x02,0xf8,0x97,0xc6,0x5d,0x71,0x45,0x4a,0xdf,0xcb,0x80,0x43, +0xd9,0x91,0x12,0x89,0xe9,0xdf,0xa5,0xb5,0xdc,0xe7,0x32,0x68, +0x61,0xae,0xae,0x97,0x49,0x76,0x3d,0xc7,0xed,0x60,0x40,0x38, +0x73,0xa0,0xa8,0x98,0x05,0x2b,0xce,0xeb,0x62,0xff,0xea,0x0e, +0x33,0x28,0x74,0x78,0x30,0xe8,0xb5,0xab,0x65,0xf2,0xeb,0x3a, +0x99,0x8d,0xf3,0x7e,0xd4,0x85,0x1e,0xa4,0xad,0x8c,0x9b,0x9e, +0x0f,0x95,0x60,0x45,0x1c,0x94,0x60,0x88,0x8d,0xc0,0x0d,0x1e, +0x83,0x62,0x90,0x36,0x41,0x4a,0x3d,0x4c,0x77,0x93,0xc1,0xd6, +0xc3,0xc4,0x9c,0x77,0x51,0x6e,0xce,0x43,0xbf,0xe8,0xa0,0xb2, +0xb1,0x16,0x6f,0x9b,0x73,0x6e,0x34,0x08,0xd2,0xf1,0xf1,0xe6, +0x0b,0xf9,0xc0,0x98,0xb6,0x9e,0xa6,0x00,0x4f,0x34,0x52,0x0a, +0x56,0x67,0xea,0x01,0xbe,0x46,0xc1,0x0d,0x45,0xe6,0xa3,0xda, +0x9a,0xf1,0x7b,0xfa,0x61,0x02,0x77,0xda,0xfb,0x2d,0x88,0x86, +0xec,0x44,0xdd,0x58,0x8c,0x35,0xc1,0x1d,0xa0,0x16,0x84,0xcc, +0x7b,0x67,0x87,0x52,0xfa,0xf5,0x67,0x62,0x2f,0x73,0xbd,0x2a, +0xff,0xab,0x74,0xf6,0x12,0xaa,0x79,0x9b,0x64,0x66,0x1b,0x53, +0xb2,0xee,0xbd,0x48,0x5c,0xd2,0x1e,0xa1,0x94,0x16,0xdb,0x6f, +0xb9,0x35,0x49,0x3c,0xdb,0x1c,0x86,0xd0,0xb4,0xc8,0xba,0xb4, +0x6e,0x21,0xe7,0x7c,0x1b,0x39,0x89,0xea,0x41,0xb8,0x85,0x1b, +0xa5,0xa3,0xdb,0x8d,0xd9,0x40,0x20,0xf4,0xf6,0x85,0xab,0x0a, +0xbe,0x7d,0x8b,0xee,0x49,0x0c,0x19,0x6c,0xfc,0x6d,0xb7,0x96, +0x56,0xae,0xcf,0xa6,0xd5,0x74,0xe5,0x81,0x5e,0x27,0xba,0xe7, +0x8f,0x1c,0x16,0xeb,0x0c,0x0b,0xb9,0xbd,0xdd,0xe4,0xc6,0xa9, +0xb0,0xc6,0x3d,0x47,0xa4,0xa0,0xf1,0xae,0x92,0xae,0xd2,0x65, +0xb6,0xfa,0xcb,0x68,0xfa,0x9e,0xf7,0x56,0x17,0x60,0x07,0xa4, +0xf3,0x75,0xf6,0x32,0xbd,0x9c,0xea,0x8b,0x49,0x35,0xbb,0x8e, +0x72,0x7d,0x85,0x12,0x2f,0x10,0xf5,0xfe,0x3d,0x48,0xeb,0xcc, +0x46,0xd5,0x44,0xdb,0xed,0xbc,0xf1,0x1a,0x10,0x31,0x94,0x60, +0xf6,0x49,0x43,0x8c,0x72,0x36,0x4e,0x32,0x8a,0x9d,0xaf,0xf3, +0x9d,0x3d,0xcc,0xb1,0x5b,0x02,0xc6,0xa8,0x69,0x15,0x23,0x76, +0x2a,0xf3,0x0d,0xc2,0xd1,0x62,0x01,0x26,0x20,0xed,0xcc,0x96, +0xe1,0x34,0x98,0xef,0xa6,0x73,0x4d,0x3d,0x23,0x1e,0xd8,0xf6, +0x97,0xd7,0xe9,0xd5,0x8a,0x77,0x6a,0xc9,0x14,0xbe,0x49,0x27, +0x1c,0xbf,0x02,0x03,0x4b,0x18,0xba,0x99,0x51,0x78,0x8d,0x6b, +0x9a,0xde,0x34,0xc7,0xa5,0x24,0xbf,0x57,0xe8,0x81,0xbe,0xa7, +0xf6,0x10,0x29,0x3f,0x2e,0x64,0xbb,0xf8,0xb2,0x64,0x31,0x5a, +0x8e,0xcc,0x4e,0x79,0xb1,0x37,0xa5,0xcb,0xda,0x37,0xf3,0x49, +0xec,0xa0,0x6d,0x29,0x72,0xfd,0xae,0x5b,0x9d,0x4a,0xe7,0xa3, +0xac,0x02,0x3e,0xf4,0xa2,0xfc,0xe3,0x88,0xb3,0x6e,0x20,0x80, +0xe6,0x9c,0xcf,0x26,0x46,0x2a,0x61,0x94,0xc8,0x6e,0xbe,0x11, +0x36,0x85,0x76,0xd3,0xdb,0x7d,0x0c,0x70,0xd0,0x96,0xbe,0xde, +0xce,0xdf,0x34,0xed,0x2a,0x80,0x4f,0x4a,0xc6,0x2c,0x92,0x9c, +0x2d,0x7c,0xfb,0x89,0x4e,0x23,0x4d,0x55,0x0d,0x34,0xe2,0x84, +0x8e,0x3c,0xa5,0x81,0xa6,0x46,0x46,0x86,0x82,0x14,0xdc,0xa6, +0x0b,0x3e,0x80,0xbc,0x9d,0xab,0x1f,0x65,0xfe,0x42,0xe6,0x0d, +0x93,0xcb,0x47,0x96,0x6d,0x7b,0xd0,0x64,0x80,0x01,0xb0,0x98, +0x96,0x04,0x19,0xa0,0x02,0x74,0x76,0x0a,0xe0,0x04,0xbc,0x0e, +0x0c,0xa4,0xcc,0xad,0x94,0x1e,0xd6,0xeb,0x8a,0xfa,0xcf,0xe8, +0xa8,0xe2,0x41,0xba,0x4f,0x64,0xba,0x99,0xdf,0x27,0x57,0x98, +0xb7,0xc0,0xbc,0x9b,0xe3,0x63,0xf8,0xd5,0xf3,0x91,0xcc,0x2a, +0xa8,0x39,0x62,0x98,0x17,0x31,0x08,0xfa,0xf9,0x82,0x32,0x85, +0x12,0x3d,0x62,0x89,0x17,0xc4,0x23,0x8b,0x72,0x25,0x61,0x20, +0xc8,0xa6,0xc9,0x22,0xd9,0xc0,0x07,0x5b,0xf4,0xa0,0x50,0x40, +0xe4,0x11,0x0e,0x72,0x3c,0x73,0xab,0xff,0x81,0xbc,0xc5,0x9d, +0xa4,0x2f,0xa5,0x72,0x2a,0x4c,0x04,0x18,0xcd,0x60,0xe8,0xc1, +0xe3,0x07,0x5c,0xdb,0x51,0xff,0x65,0x73,0xf5,0x13,0x93,0x61, +0x4d,0x57,0x60,0x33,0x42,0x16,0x7b,0x07,0xa1,0xe6,0x6e,0x67, +0x20,0x94,0xdd,0x63,0x3d,0x62,0xea,0x51,0x1d,0xc3,0xf0,0x10, +0x56,0x83,0xa3,0xe5,0x31,0x9a,0xc4,0xd4,0x60,0x1b,0x07,0xd0, +0x85,0x6c,0x21,0xfe,0xa6,0x8b,0x12,0x36,0xdd,0x35,0x84,0xec, +0xa6,0x88,0x6c,0x11,0x38,0x34,0x22,0xb2,0x05,0x20,0xbb,0x09, +0x9c,0x1e,0xd2,0xf7,0x42,0x55,0xea,0x76,0xff,0x19,0x11,0x1d, +0x0f,0xc6,0xea,0xdc,0xa7,0xa0,0x5e,0x29,0xc0,0x08,0xbd,0xe5, +0xc1,0xa7,0x89,0xe6,0x76,0x73,0xe5,0xac,0x65,0xc6,0x70,0xfc, +0x51,0x91,0xd0,0x55,0xa0,0x20,0x15,0xb0,0xda,0x00,0xe2,0xf1, +0x1a,0xe7,0x35,0x4e,0xe4,0xf0,0x83,0xfa,0x0a,0xfe,0xb2,0x75, +0xd0,0x2d,0x6a,0xe1,0x6a,0xcd,0xc6,0xc0,0xcd,0xa8,0x1b,0xe2, +0xae,0xbd,0x99,0x74,0xe8,0x14,0x34,0xb0,0x54,0x06,0xd1,0xdc, +0xa2,0x20,0x25,0xae,0xb6,0xa7,0x78,0x64,0xc0,0xf1,0xc2,0x18, +0x78,0x99,0x20,0x87,0x24,0x2a,0x4a,0xa9,0x7b,0x6e,0xe6,0xd9, +0x4a,0x85,0x29,0x3a,0xb4,0x1c,0x09,0x09,0x17,0x9c,0x24,0x49, +0x71,0xcc,0xd5,0x97,0xfb,0x1a,0x80,0x13,0x2c,0xb5,0x09,0x63, +0x24,0x14,0xbd,0xea,0xcd,0x68,0xc2,0x14,0x26,0x04,0x6f,0x78, +0x6d,0xa2,0x98,0x7e,0xb7,0x37,0xf6,0x88,0xa3,0xa4,0x00,0x21, +0x11,0x26,0x8f,0x62,0xac,0x9f,0x0f,0x62,0xef,0xee,0xa2,0x5e, +0xc9,0xd8,0xaa,0xea,0x6e,0xec,0xba,0x97,0xe9,0x6f,0xd6,0x59, +0x0f,0x9f,0x89,0xf2,0xac,0x70,0xa2,0x5d,0x04,0xcb,0x47,0xbd, +0x72,0xaa,0x08,0x8d,0x9e,0x26,0xab,0xe4,0x81,0x11,0x0c,0xeb, +0x34,0xc8,0xc6,0x15,0xcb,0xb9,0x31,0x5d,0x08,0x54,0xb1,0x2d, +0x6b,0xd1,0x9f,0xe6,0xd1,0x8e,0x1a,0xee,0x71,0x74,0xc9,0x5b, +0x51,0x78,0x00,0xe6,0xb7,0xc5,0xc1,0x03,0x7f,0x07,0x86,0x8f, +0xa5,0x1e,0x53,0x19,0xf6,0x65,0x12,0xc5,0xa7,0xf8,0x48,0x33, +0x96,0xa3,0xa1,0x18,0x34,0x87,0x9d,0xba,0x18,0xf8,0x4d,0xc0, +0x71,0x4e,0x4c,0xd2,0x36,0xa8,0x88,0xb0,0xb2,0xcf,0x3b,0x9e, +0xa8,0xe2,0x58,0x29,0x72,0xc0,0xea,0x0b,0x62,0xdd,0xb7,0x14, +0x40,0x85,0x8f,0x5a,0x79,0x74,0xa9,0x62,0xf7,0xaf,0xe4,0xa8, +0xb6,0x12,0x4c,0xa6,0x98,0x2d,0xbb,0xce,0x73,0xb1,0x4c,0xdd, +0x51,0x6e,0xdd,0xa9,0xcf,0x82,0x0e,0x7d,0x0e,0x3b,0x0e,0x77, +0x55,0x5d,0x95,0x8b,0x6b,0xb0,0x14,0x66,0xe1,0x13,0x8e,0xeb, +0xac,0xaf,0x2f,0xa5,0x74,0x2b,0x36,0xa1,0x05,0x86,0xff,0x54, +0x84,0xae,0xe0,0x8c,0xad,0x0e,0x0c,0xb5,0x89,0xe9,0x4f,0x1e, +0x87,0xc1,0x81,0x5a,0x96,0x6e,0x4f,0x2d,0xe3,0xe6,0xd2,0x13, +0x1f,0x28,0xdf,0xd9,0xff,0xab,0x2a,0x6b,0x9d,0xa2,0x01,0x67, +0x44,0x6f,0xe2,0xc9,0x10,0xdf,0x60,0x94,0x23,0xd3,0xa5,0xba, +0x6e,0x7b,0xe2,0x49,0x2b,0xa2,0xae,0x76,0x48,0xe8,0x9c,0x10, +0x9d,0xcb,0xd3,0x0f,0x81,0xfe,0x09,0x36,0xdb,0x69,0xe7,0xd0, +0xc9,0x81,0xc9,0x6e,0xc6,0x93,0xe5,0x27,0x93,0x7b,0xaf,0x9a, +0x5a,0x54,0xd3,0xb6,0xa4,0xd2,0x10,0xa5,0xbe,0xf9,0xcf,0x25, +0x3f,0x80,0x2e,0xbc,0x86,0xff,0x74,0xab,0xe5,0x26,0xde,0x8e, +0x76,0x5b,0xda,0x84,0x4b,0x05,0x07,0x29,0x95,0x42,0xcb,0xc8, +0x5d,0x4d,0x5f,0x8f,0xad,0x74,0x99,0x18,0xbf,0xbf,0x47,0xf8, +0x24,0x82,0xbc,0x08,0x16,0x74,0x20,0xc0,0xa3,0x97,0x30,0x30, +0x89,0x87,0xe1,0x10,0x58,0x89,0xc4,0x74,0x77,0x57,0xf9,0x84, +0xeb,0x45,0xb7,0xa9,0xaa,0x11,0x81,0xe6,0x8a,0x4a,0x77,0x6c, +0x77,0x3e,0x9f,0x2c,0xc0,0x1a,0x95,0xbe,0x50,0xbb,0xcb,0xbb, +0x63,0xfc,0xbe,0x46,0x1b,0x8c,0xa7,0x6b,0x7c,0x73,0xae,0x52, +0x21,0x48,0x9e,0xd1,0x09,0xca,0x86,0xc0,0x30,0x6d,0xc1,0xbd, +0x14,0xc4,0x26,0x0e,0xef,0x63,0x0f,0xed,0xf4,0x8c,0x32,0xeb, +0xa1,0xc2,0x62,0xa8,0xaf,0x3c,0xa6,0x4c,0x4d,0xd2,0x84,0xd7, +0x7a,0xc9,0xe3,0xab,0x29,0xb0,0x04,0xf3,0xc6,0xc0,0xf8,0x5d, +0x85,0xc9,0x6e,0x71,0x1f,0x96,0x56,0x75,0x4b,0x40,0xcc,0xf7, +0xbb,0xa1,0x42,0x1e,0x3a,0x19,0x62,0x50,0x56,0xda,0x4a,0x52, +0x39,0xd9,0xac,0x6c,0x74,0x15,0xad,0x22,0x81,0x60,0x1b,0x99, +0x41,0xfe,0x18,0x4b,0xe6,0x8f,0x30,0x5c,0xb4,0xb1,0xc1,0x6a, +0xb9,0x64,0x68,0xb0,0xba,0x56,0xf7,0x75,0x7d,0x65,0x6a,0xfc, +0x0e,0xd3,0xc5,0x8b,0x5a,0x82,0xb4,0xcf,0x29,0xa7,0xfc,0xd5, +0x8d,0x7c,0x83,0xfe,0xbf,0x84,0xdd,0x22,0x1d,0xf7,0xfb,0x4c, +0x17,0x41,0xe2,0x5a,0x2f,0x1a,0xf3,0xef,0x36,0x60,0x34,0xfa, +0x2a,0x1b,0x46,0x55,0xf3,0xc7,0x98,0x31,0xe5,0xaa,0x0e,0xb2, +0x64,0xa4,0x58,0x8d,0xfa,0xc2,0x0e,0xb9,0x19,0x2d,0xaa,0xbd, +0x58,0x28,0x34,0x8e,0xde,0xec,0x7b,0x6f,0x76,0x28,0x78,0xab, +0x0c,0x5f,0xf3,0x51,0xdb,0xf6,0x05,0x4d,0xa2,0x1f,0xf8,0xc1, +0x67,0x8b,0xbc,0xd2,0xee,0x9c,0x48,0xca,0x89,0x17,0x9e,0xbf, +0xef,0x87,0xdd,0xf7,0xe2,0xe4,0xcd,0xcf,0x23,0xf1,0x45,0x14, +0xfc,0x89,0xce,0xc2,0xe7,0x31,0xfe,0x02,0x20,0x7a,0xc2,0xc8, +0xd0,0xf6,0xeb,0x7a,0xb4,0x4c,0xaf,0xf2,0xfd,0xda,0xce,0xd9, +0x00,0x2a,0xee,0xfe,0x98,0x82,0x41,0x36,0xc6,0x87,0x24,0x67, +0xc1,0x12,0x23,0xee,0xc1,0x9c,0x20,0xde,0xe7,0xcb,0xf9,0x07, +0x3c,0x4a,0x58,0x02,0x97,0xd2,0x19,0xce,0x1b,0x1a,0x85,0x7a, +0xa2,0xe7,0x12,0x16,0x47,0x18,0x03,0x35,0x58,0xc7,0x0d,0x4a, +0xc3,0xb0,0x70,0x18,0xe1,0xaa,0x8f,0xa8,0xda,0x93,0x4d,0xb7, +0x31,0x29,0xa7,0xe6,0x5d,0x06,0x6e,0x39,0x79,0xd1,0xb1,0x59, +0x59,0x68,0x40,0x85,0x86,0xad,0x83,0x14,0xb4,0x10,0x27,0x04, +0x02,0x20,0x0f,0x98,0x51,0x6f,0x69,0x3a,0x25,0x73,0x73,0x22, +0x40,0x96,0x57,0x2a,0x35,0xaa,0x2d,0x2f,0x4a,0x79,0xb3,0x58, +0x97,0x51,0x5c,0x39,0x0f,0x7b,0x3e,0x82,0xd0,0xb9,0x04,0x77, +0x9e,0x86,0xa1,0x87,0x8b,0x0c,0x86,0x66,0x6c,0x7a,0x2e,0x68, +0xa8,0x5e,0x69,0x1c,0x71,0x7b,0xee,0xee,0x3c,0x62,0x6b,0xfc, +0x61,0x0e,0x0b,0x63,0x50,0x3b,0x1f,0x82,0xb5,0xef,0x7f,0x11, +0x23,0xcb,0xfa,0x86,0xd6,0xf6,0x62,0xbe,0x50,0x8c,0x24,0x78, +0x98,0x3f,0x54,0x05,0xad,0x98,0x28,0x41,0x00,0xae,0x44,0x3b, +0x90,0x12,0x36,0x6a,0x66,0x70,0x14,0xf9,0xc0,0xfd,0x5a,0xfe, +0x10,0xb5,0xf2,0x82,0x92,0x69,0xba,0xee,0x38,0xbf,0x67,0xf2, +0x8f,0x8f,0xbd,0xca,0x0a,0xc2,0x1a,0x24,0xbe,0x76,0x18,0xd6, +0x4e,0x81,0x8a,0x09,0xf6,0xbe,0x20,0x86,0x9a,0x7b,0xff,0x87, +0x8a,0x6b,0x51,0x4c,0xeb,0xe5,0x93,0x25,0xf8,0xf7,0xc8,0x48, +0x41,0x04,0xb8,0x37,0x8a,0x22,0x52,0x23,0x13,0xac,0xc4,0x38, +0xe1,0xb7,0x22,0x77,0x25,0x59,0x0f,0x14,0xa1,0x43,0x78,0x5e, +0x92,0xcb,0x9a,0x2e,0xe8,0x95,0xbb,0x40,0xd8,0x6d,0x77,0x81, +0x2d,0x82,0x5a,0x91,0x2d,0x83,0x4c,0x63,0x0a,0x96,0xec,0x64, +0xb0,0x04,0x11,0xa2,0xf1,0x2c,0xaa,0xcd,0x54,0xaa,0x2f,0xb9, +0x2f,0x31,0x91,0xde,0x27,0x44,0x2d,0x05,0xad,0xc2,0x32,0xe7, +0x4b,0xf9,0x11,0x2f,0x81,0xd2,0xf3,0x25,0xbb,0x1b,0x10,0xd2, +0xa5,0x39,0xd5,0xcb,0x4e,0xe2,0x2c,0x58,0x9f,0xc4,0xeb,0x40, +0x96,0x74,0x68,0xee,0xaf,0x4f,0x10,0xec,0x84,0xa6,0xa7,0xd3, +0x2c,0xc8,0x4e,0x29,0x25,0x9d,0x71,0x0a,0x98,0x53,0x40,0xce, +0xb6,0xec,0x0b,0x6f,0x09,0xe8,0x15,0x68,0x79,0x91,0x8f,0x79, +0x99,0x70,0x30,0x1f,0x89,0xb5,0x18,0x0b,0x81,0x7e,0xe4,0xc1, +0x6d,0xbc,0xee,0x2f,0x31,0x2c,0x9d,0x1e,0x69,0xa7,0x6b,0x30, +0xc0,0x50,0xfe,0xd6,0x71,0x2c,0x1d,0xc2,0x8c,0xf2,0x08,0xf9, +0xdd,0xdd,0xed,0xb9,0x1a,0x50,0x74,0xd5,0x43,0x6b,0x1f,0x7d, +0x85,0x9f,0x99,0x82,0xe1,0x07,0x62,0xbe,0xb4,0x9f,0xc3,0xbf, +0x93,0x3c,0xbe,0x45,0xa7,0xf6,0x79,0x5b,0xbd,0x64,0xa2,0x87, +0xe8,0xa6,0x29,0xf5,0xb4,0x36,0xc1,0xad,0x50,0x33,0x67,0x5a, +0x6e,0x81,0x16,0xbf,0x37,0x6f,0xff,0x86,0x67,0x9e,0x58,0xb4, +0x19,0x97,0x8a,0x70,0x4b,0x9a,0x80,0x5e,0x48,0x69,0xc5,0xb7, +0xfc,0x2e,0x49,0x15,0xd5,0x1b,0x9b,0xea,0xcd,0x5e,0xaa,0x37, +0x80,0x36,0x27,0xaa,0xf3,0x02,0xd5,0x79,0x53,0x55,0x97,0xef, +0xa0,0x3a,0xa2,0xa6,0x0b,0xd5,0x76,0x11,0x69,0x86,0xa6,0x7a, +0x83,0x54,0x6f,0x9d,0x03,0x3a,0x72,0x62,0xb2,0x77,0x11,0x58, +0x9e,0x93,0xec,0xee,0xce,0xbe,0x11,0x53,0xd8,0x2e,0x55,0x4b, +0x7d,0x5e,0xdf,0x87,0x81,0x5e,0xe1,0xd3,0x8f,0x89,0x8e,0x25, +0x21,0xdf,0x3a,0x9e,0x44,0xa0,0xb0,0x1f,0x1f,0x97,0xae,0x75, +0xa8,0x7d,0x4c,0xd2,0x1b,0x03,0xfc,0x50,0xc5,0x37,0x27,0x82, +0x86,0xcd,0x47,0x51,0x1a,0x9e,0xc2,0xd5,0x57,0x43,0xb0,0xab, +0xd5,0xa1,0xc2,0x85,0x19,0x39,0x3d,0xcb,0xc9,0xa0,0x71,0xf9, +0xe2,0x8e,0xe4,0xec,0x5b,0xdd,0x06,0xc4,0x7c,0xee,0x1e,0xfa, +0x69,0x1b,0x94,0x6a,0x9d,0x47,0xd1,0x86,0xe0,0xa3,0x60,0x85, +0x02,0x8f,0x5f,0x8c,0x26,0x3f,0x62,0x26,0x53,0x84,0x5d,0x46, +0x71,0x3c,0xcd,0x2c,0x16,0xd0,0xf0,0xa3,0xf3,0xbb,0xa8,0x69, +0x82,0x82,0x3c,0x32,0xa3,0x59,0x30,0xdd,0xff,0x6a,0x81,0x14, +0xf4,0xad,0xdd,0x2e,0xc4,0x53,0x15,0x2d,0xca,0xda,0x5f,0x9a, +0x7c,0x5b,0x5e,0xe4,0x2d,0x62,0xf5,0xf6,0x02,0xe3,0xe0,0xf3, +0x08,0x63,0x46,0x98,0x5b,0x64,0x64,0xb9,0x2e,0xfa,0x75,0x7c, +0x60,0x9c,0xdd,0xaa,0x6c,0x5e,0xab,0x95,0xce,0xcd,0x17,0xfd, +0xc1,0x22,0xa0,0xff,0x0d,0xbb,0x0b,0xdf,0xbe,0x1c,0x32,0x79, +0x36,0xa7,0x40,0xba,0x2b,0x37,0x86,0x51,0x3d,0xee,0x8d,0xbf, +0xdd,0xf5,0x92,0x8c,0xf0,0x85,0x63,0x52,0x42,0xe5,0x66,0xa7, +0x62,0xa2,0x2c,0x72,0x1d,0x3b,0xcb,0xaf,0x6d,0xa1,0xb3,0x73, +0xe6,0xd1,0x63,0x9d,0xca,0x03,0xc0,0xef,0x5b,0xdc,0xed,0x9a, +0x96,0x12,0x00,0xfa,0x5a,0x0e,0x36,0x01,0xfd,0x8f,0x04,0x4b, +0xb5,0x8e,0x36,0x0f,0xd4,0x87,0xea,0x44,0xd3,0x9b,0x2c,0x9e, +0x87,0xb9,0x3d,0x10,0xe8,0xa6,0x6f,0x64,0xb9,0x2b,0x2e,0x1e, +0x5a,0x52,0x28,0xf5,0x48,0x3c,0x3f,0x4c,0x25,0x22,0xbd,0x07, +0xba,0xd2,0x13,0xec,0x26,0x30,0x63,0x0f,0x07,0x80,0x11,0x46, +0xca,0xb7,0x1a,0x40,0xf2,0x7f,0xa0,0xdf,0x06,0x82,0x5a,0x82, +0x4d,0x09,0x06,0x15,0x0d,0xad,0x83,0x30,0x11,0xe4,0xa6,0xd9, +0x68,0x18,0x5c,0x98,0xb4,0x6f,0xb7,0x16,0x45,0xbf,0xb0,0xc6, +0xe1,0x78,0x0a,0x31,0x2c,0x8e,0x61,0x7a,0x9d,0x9a,0x25,0xf3, +0xff,0xfe,0x47,0xf0,0xcf,0xff,0xa9,0xde,0x04,0x2b,0x12,0xa0, +0x44,0xab,0x73,0x8e,0x07,0xc6,0xff,0xfc,0x9f,0x31,0xd8,0x96, +0x11,0xfd,0xfe,0xdf,0xff,0x40,0x7b,0xcf,0x5a,0x29,0x71,0x90, +0x08,0x50,0xe9,0xbd,0xc9,0xfc,0xc1,0x26,0xee,0x9c,0x38,0x51, +0x1c,0x60,0x0e,0xce,0x2b,0xd2,0x96,0x2a,0x04,0x51,0xef,0x03, +0x98,0x2d,0x89,0x77,0xb4,0xbc,0xbb,0x5b,0x9e,0x47,0x7a,0x98, +0xfe,0xef,0x7f,0x34,0xff,0xf9,0x3f,0x4f,0x36,0x27,0x46,0xdf, +0xb4,0x04,0xc9,0x74,0xfe,0xd6,0x5b,0xfa,0xa7,0x4b,0xb0,0x6a, +0x02,0xf8,0xfd,0x63,0xa1,0x55,0x4c,0x8f,0xb4,0x59,0xb7,0xbe, +0xcd,0x8d,0x97,0x7b,0x3a,0x93,0x33,0xeb,0x96,0x4e,0xaf,0x18, +0x59,0xcc,0x7d,0x2f,0x2e,0xd9,0x2c,0x3c,0x45,0x37,0x56,0x99, +0x2e,0x3f,0xa4,0xb3,0xef,0x40,0xd3,0x98,0x2a,0x6f,0xca,0xd1, +0xc9,0xac,0x63,0x6c,0x79,0x9b,0x59,0x3d,0xe6,0x98,0x35,0x63, +0x87,0x17,0xda,0x22,0x3a,0xbd,0xd9,0x72,0x9c,0x51,0x9a,0x66, +0xf0,0x8e,0x85,0x7a,0x3d,0x1e,0xbf,0x2f,0x66,0x60,0xc6,0x9a, +0xe8,0xab,0x57,0xd3,0xf9,0x7c,0xe9,0x84,0x57,0x1e,0x27,0xe9, +0xd4,0x4e,0xa0,0x2b,0x1a,0x54,0xb4,0xcd,0x81,0xad,0xab,0xc3, +0x61,0x33,0x72,0x86,0xf0,0xf0,0x19,0xf9,0x61,0xc0,0xff,0xba, +0xe1,0x5a,0x71,0x6d,0x61,0x88,0xc3,0x3e,0xd8,0x89,0x0e,0xba, +0xa8,0x50,0x3b,0xac,0x5b,0xa0,0xb9,0x60,0xc8,0x72,0x25,0x20, +0x4c,0x21,0x2c,0x27,0x02,0x9c,0xa4,0xc2,0xa1,0x5f,0x68,0x2b, +0x94,0xff,0x16,0xfb,0x8b,0x22,0x88,0x5d,0xd1,0x2f,0xaf,0xd1, +0x0e,0x93,0x06,0x6d,0x43,0x6b,0x98,0x1f,0x81,0x25,0x3f,0x26, +0x6f,0x61,0x84,0xdd,0x26,0x8a,0x25,0xe5,0xb1,0xd5,0x22,0x76, +0x23,0x7b,0xbc,0x16,0xe8,0x57,0x66,0x54,0x1d,0x18,0x21,0x21, +0x38,0x8b,0x7f,0x0b,0x58,0x3a,0xec,0x68,0x2d,0x64,0x97,0x5b, +0x1b,0x44,0xcc,0x47,0x9b,0x6b,0x28,0xc6,0x95,0x88,0x4c,0x3d, +0xc0,0x26,0x10,0xb8,0xf9,0x0c,0x9f,0xc8,0x6a,0x3f,0xb2,0x0b, +0xf3,0x8b,0xcc,0xd3,0x5d,0x84,0x08,0x88,0x07,0x0b,0xf5,0x8f, +0xab,0x6e,0x83,0x5c,0xb3,0x1a,0xc1,0xa8,0x3b,0x18,0x0c,0xf1, +0x4e,0x8b,0xc1,0x35,0x86,0x1e,0x7d,0x3b,0x5f,0xe6,0x51,0xb8, +0x8b,0x1c,0x85,0xce,0x57,0x6e,0xd2,0x67,0x17,0xa6,0x60,0x15, +0xba,0xce,0xa7,0xa2,0xeb,0xd4,0xa0,0xbb,0xfc,0x64,0x7c,0x97, +0xd5,0x08,0xc7,0x9f,0x8c,0x70,0x6c,0x64,0xd4,0xe2,0xde,0xa0, +0x13,0x3e,0xee,0x74,0x1e,0x43,0x47,0x3f,0x7e,0xf2,0xf0,0x21, +0x7c,0x04,0x9d,0xa7,0x9d,0x87,0x8f,0x3e,0xff,0x3c,0x88,0x1e, +0x86,0x67,0x0f,0x9f,0x74,0x3e,0x0f,0x9e,0x3e,0xe9,0xc0,0xf7, +0xa3,0xe0,0x69,0xf4,0xf4,0x49,0x74,0x06,0xe2,0xfe,0xf0,0x69, +0xf8,0xe4,0xec,0x2c,0x0c,0x3e,0x3f,0x7b,0xf4,0xe8,0x49,0x04, +0x09,0x9d,0xb3,0xc7,0x4f,0xa3,0xcf,0x1f,0x07,0xd1,0xa3,0x47, +0x8f,0xa3,0x27,0x8f,0x86,0xca,0x7b,0x7f,0xf9,0xf6,0x92,0xe3, +0x3e,0x93,0xd4,0x5b,0x4c,0x36,0xb5,0x46,0x0f,0x1f,0x3d,0x7c, +0xf8,0xd0,0xaa,0x1e,0x7e,0x3c,0x7a,0xfa,0xf8,0x49,0xc7,0xd0, +0x11,0x86,0x80,0xfb,0xa9,0x45,0x10,0x82,0x84,0x4f,0x1e,0x76, +0x34,0x65,0x51,0xe7,0xe9,0xa3,0x47,0x9f,0x3f,0x8e,0x0c,0x89, +0x1d,0x48,0x8e,0x10,0xad,0xa2,0x35,0x7a,0xdc,0x39,0x7b,0x18, +0x3d,0x7d,0x68,0x88,0x3e,0x0b,0x9f,0x76,0x3e,0x0f,0x9f,0x58, +0xd4,0x03,0x1d,0x9f,0x3f,0xfa,0xfc,0xa9,0x6a,0x06,0xd4,0xfc, +0x30,0x0a,0xa1,0x8a,0xbd,0xed,0xb9,0x8c,0x07,0x67,0x48,0xd2, +0x93,0x27,0xc1,0xa3,0xb3,0xa7,0x50,0xf8,0x69,0xf0,0x24,0x7c, +0x18,0x3e,0x81,0xbf,0x51,0xd8,0x79,0xfc,0xb0,0x03,0xd8,0x1f, +0x3f,0x8a,0x9e,0x86,0x4f,0x9f,0x04,0x4f,0x3b,0xd1,0xe3,0x47, +0x40,0x47,0x14,0x3d,0x8d,0x1e,0x45,0x11,0xd4,0x02,0x04,0x3d, +0x7e,0x7c,0xf6,0x18,0x72,0xc2,0xce,0xd3,0xa7,0x67,0x40,0xd0, +0xc3,0xce,0xe3,0xcf,0x43,0xa4,0x03,0xc8,0x82,0x66,0xe0,0x0f, +0x28,0x7b,0xd6,0x09,0x3e,0x7f,0xfc,0xf8,0x71,0xf4,0x18,0x72, +0xa0,0xd0,0x63,0x68,0x1a,0xf2,0xe4,0xe1,0xd3,0xc7,0x0f,0xcf, +0x10,0xe4,0xc9,0x93,0x33,0x60,0x1b,0x34,0xe9,0xc9,0xe3,0xcf, +0x21,0x21,0xfc,0xfc,0xec,0xe1,0x59,0xe7,0x21,0xe2,0xef,0x7c, +0xfe,0x08,0xd1,0x01,0x3b,0x9f,0x3e,0x7e,0xf4,0x70,0x6f,0x73, +0xc6,0xd0,0x9c,0xce,0xc3,0x87,0xc0,0x37,0x68,0x06,0x32,0x2b, +0x44,0x56,0x84,0x8f,0x1f,0x77,0x90,0xda,0x10,0xf2,0x90,0xc5, +0x80,0xf2,0x51,0xf8,0x08,0x2a,0x42,0x92,0x9e,0x3c,0x7e,0x88, +0x3f,0x3a,0x8f,0x3a,0x8f,0x9e,0xe2,0x8f,0xb3,0x87,0xd0,0x69, +0xc1,0x59,0xe7,0xd1,0x59,0xf8,0xe4,0x71,0xf0,0xe4,0xf1,0xa3, +0xce,0xc3,0x27,0x88,0xe5,0x71,0xf8,0x24,0x0c,0xa1,0xcc,0x59, +0x14,0x02,0x82,0x87,0x90,0xf3,0x14,0xa8,0x43,0x15,0x7a,0xf6, +0x14,0xb0,0x76,0xb8,0x37,0xb0,0x1d,0xd1,0xc3,0x33,0xe0,0xdb, +0xd3,0x08,0x19,0x77,0x06,0xbc,0x0d,0x9e,0x7e,0xfe,0xf8,0x73, +0xa8,0x11,0xd9,0x73,0xf6,0xf0,0x73,0x60,0x75,0xf4,0xb0,0xf3, +0xe4,0x49,0xf8,0x79,0x54,0xd1,0x1e,0xa3,0xf9,0x60,0x09,0x00, +0x93,0x49,0x65,0x18,0x6e,0xad,0x75,0x14,0x0c,0xaa,0x9d,0x81, +0xa3,0xef,0x38,0xe7,0xb7,0x03,0x4a,0xff,0x96,0x78,0x61,0x95, +0xc6,0xd4,0x4e,0xc3,0xbb,0x50,0x68,0x20,0x6f,0xd0,0x7e,0x34, +0x2c,0x23,0x51,0x33,0xe2,0x2e,0x1c,0x0a,0xc6,0xb3,0x4b,0xdf, +0xbe,0x85,0xf5,0xec,0xb8,0x72,0x53,0x18,0xd2,0x95,0xa5,0x11, +0xc6,0xe4,0xdf,0xfa,0x33,0x1d,0xc5,0x55,0x5f,0x04,0x0e,0x96, +0x51,0x0c,0x06,0xdb,0x1e,0xa0,0x51,0x18,0xef,0xbb,0x9c,0xdc, +0x9c,0xf0,0xd3,0x3c,0x50,0x3d,0xfb,0xbb,0x63,0xf4,0xe8,0xdd, +0x97,0x90,0xcb,0x45,0x26,0xa3,0xd8,0x1b,0x85,0xe7,0xa3,0x08, +0x5d,0xbe,0x46,0xf1,0x28,0xc4,0x9a,0x47,0x11,0xa2,0x9a,0x8c, +0x80,0x8c,0xa8,0x35,0x02,0x03,0x6f,0x72,0x15,0xff,0xf3,0xdf, +0xce,0x27,0xa3,0x7e,0x23,0x6c,0x74,0x1b,0x51,0x23,0x18,0x87, +0x6c,0xb8,0x8c,0xe7,0x99,0x87,0x00,0x99,0x7c,0x67,0x30,0xa7, +0xe3,0xf7,0x38,0xb2,0xf2,0xc1,0x5a,0xcc,0x22,0x2b,0x3f,0xb2, +0xee,0xfd,0x9d,0xc7,0x86,0xa4,0x1f,0x46,0x9b,0xfe,0x32,0xec, +0x37,0x7e,0x08,0x83,0x46,0x73,0x19,0x35,0x1b,0x5f,0xf2,0x1f, +0xf9,0x7a,0x10,0x3e,0x00,0xe9,0x7e,0x80,0x99,0xad,0x9d,0xb9, +0xf8,0xc9,0x38,0x42,0x86,0x0a,0x19,0x2a,0x14,0xa8,0x50,0x70, +0xec,0xca,0xc5,0xcf,0xbf,0x42,0x5b,0xff,0x08,0x5a,0x00,0x0f, +0x35,0x0b,0xbf,0x4e,0xc6,0x52,0x5d,0x74,0x92,0x85,0x15,0xe5, +0x1a,0xcd,0xc9,0x15,0x7c,0x47,0x0f,0x18,0x38,0xd2,0xc0,0xf0, +0xeb,0x7b,0x24,0x4b,0xa7,0x85,0x94,0x56,0x6a,0x80,0x20,0xc0, +0x1f,0x08,0x1c,0x6a,0x60,0xd5,0xa0,0xdf,0x4d,0x45,0x18,0x84, +0x0d,0xc4,0xf5,0x2f,0x1c,0xb7,0xcc,0xc8,0xbb,0xd5,0x95,0xcf, +0x4d,0x6a,0x60,0x09,0xbb,0x05,0xf1,0x93,0x49,0xb5,0x6f,0x69, +0x1b,0x80,0x97,0x3a,0xd1,0x5c,0xdc,0x36,0xb9,0xdf,0x48,0x52, +0x4f,0xaf,0x99,0xc6,0x6d,0x9b,0x14,0x73,0xdf,0x77,0xc7,0xb2, +0xaa,0x40,0xbb,0xf2,0x58,0xbf,0xa5,0xbb,0xe6,0x7e,0xd7,0xca, +0xde,0x62,0x4a,0xdb,0x6e,0xc9,0x41,0x15,0x14,0x9a,0x5e,0xac, +0xc0,0xca,0xe6,0x0a,0xaa,0xee,0x81,0xef,0xc2,0xef,0x32,0xae, +0x88,0xde,0xbe,0x13,0x8e,0xd8,0xcb,0x77,0xc2,0x77,0xe1,0xb6, +0x99,0x5e,0xc4,0x6c,0xee,0x87,0x23,0xde,0x31,0x94,0x5b,0xce, +0x53,0x47,0x2f,0x93,0x36,0x8c,0xbd,0xbd,0xda,0xb0,0xb9,0x57, +0x15,0xe2,0xc9,0x33,0x28,0xa9,0xbd,0xda,0x70,0xb7,0xe6,0x03, +0x2c,0x25,0xe5,0x27,0xb2,0x33,0x30,0x9a,0xca,0x3f,0x59,0x06, +0x46,0x4f,0xc1,0xd7,0x90,0x5a,0x68,0x56,0x24,0x76,0xf1,0xb8, +0x75,0x3d,0x9a,0x5e,0xfd,0xf3,0xdf,0x02,0x47,0x83,0xc5,0xff, +0xfc,0x5f,0xad,0x7f,0xff,0x47,0x4f,0xa6,0x0c,0x5c,0x01,0x90, +0xd3,0x87,0x6b,0xac,0xd3,0xa1,0x3d,0x64,0xd9,0xaf,0xab,0x79, +0xd6,0xef,0x9f,0xa9,0x84,0x56,0x93,0x1a,0x0b,0x88,0x11,0xfd, +0xde,0x28,0xbf,0x3a,0xf9,0xa2,0x3f,0xfc,0xec,0x39,0xfd,0xcc, +0x9d,0xfc,0x9c,0xf2,0xb7,0x81,0x55,0xc1,0xcb,0x55,0xb2,0xf8, +0x2a,0x81,0xd5,0x54,0xa2,0xee,0x38,0xc6,0x85,0x5c,0x3a,0xae, +0x2f,0x16,0xa1,0xc4,0xba,0x12,0x8c,0xcf,0x4c,0x96,0xc9,0xa8, +0x66,0x92,0x25,0x5e,0x25,0x23,0x27,0x8e,0xd9,0xd6,0x2e,0x57, +0xc3,0x31,0x42,0x59,0x40,0x50,0xc3,0x31,0x8d,0x05,0x38,0x46, +0xbf,0x35,0xc7,0xe4,0x8b,0xfe,0xd8,0xe3,0x9f,0xd3,0x43,0x17, +0x2c,0x64,0x38,0x7b,0x18,0x73,0x46,0xe4,0xc2,0x45,0x0c,0xc7, +0x3d,0x40,0x3f,0x73,0x07,0x20,0xe7,0x7c,0x6b,0xb8,0x72,0xb2, +0x5b,0x5d,0x2e,0xd5,0xe9,0x91,0xc7,0xa9,0x6e,0x5d,0x39,0xd7, +0xa5,0xf9,0x55,0xff,0x02,0x09,0xe5,0x18,0x47,0x9b,0x4c,0x3f, +0xba,0xc0,0x63,0x43,0xc2,0x43,0x93,0x63,0xcd,0xaa,0x90,0x27, +0xf1,0xa2,0xad,0x20,0x35,0x38,0x6f,0x64,0xed,0x45,0xd8,0x44, +0x63,0x27,0x6b,0x2f,0x03,0xf8,0x88,0xe0,0x1f,0x30,0x10,0xe0, +0x9f,0xd0,0x6f,0x7a,0x09,0x58,0x6f,0xd3,0xcc,0x83,0xc2,0x7e, +0x7f,0xbc,0x5e,0xde,0x26,0x36,0x18,0xfd,0x08,0xfd,0x6e,0x21, +0x63,0x05,0x3f,0x56,0x98,0x41,0x68,0xe5,0x0b,0x93,0x01,0xed, +0x8a,0xd0,0x72,0x01,0x9d,0xa3,0x31,0xf9,0x38,0x03,0x59,0x87, +0x89,0x8a,0xfc,0x2c,0x99,0x5e,0x05,0x57,0x81,0x69,0xb5,0x64, +0xc4,0x57,0x12,0xa9,0x00,0xf3,0xd9,0x99,0x48,0xbc,0x08,0xad, +0x74,0x05,0x8c,0xd9,0xae,0x01,0x56,0x09,0xb3,0xdb,0x00,0x3b, +0xac,0x88,0xb0,0xf7,0xe3,0xb2,0xbb,0x84,0x2a,0xf1,0xfd,0x8e, +0x51,0x84,0x2f,0x77,0x2c,0xc2,0xee,0x60,0x79,0xe2,0xd8,0x58, +0xf2,0x29,0x26,0x16,0xbe,0xab,0xef,0x82,0x44,0x2e,0x48,0xe4, +0x0f,0xad,0x33,0x14,0xe9,0x1d,0xfb,0x36,0x37,0xc8,0x6b,0x18, +0xc7,0x97,0xf0,0xef,0xf1,0x31,0xfc,0x8e,0xe8,0x77,0xb4,0x75, +0xcc,0xda,0x65,0xb0,0x08,0xf4,0x25,0x69,0xb2,0x13,0xd8,0x14, +0x10,0x2b,0xe1,0x41,0xd3,0x1b,0x9d,0xff,0xf3,0xdf,0x7c,0xb1, +0x15,0x16,0xa6,0x30,0xf7,0xdb,0x32,0x84,0x86,0x80,0xc1,0x0b, +0xb4,0x6a,0x24,0xff,0x0f,0xd8,0x45,0x58,0x76,0x11,0x6d,0xad, +0xa7,0x42,0x90,0x2f,0xf6,0xb3,0x1c,0x98,0x20,0x52,0xb8,0x74, +0x4c,0x07,0x92,0xea,0x3f,0xce,0x6e,0xe0,0xc7,0x0b,0x8a,0x6e, +0x79,0xbb,0xe6,0xc5,0x65,0xd5,0x74,0x2e,0x0f,0x2a,0x28,0x17, +0x3c,0xf5,0x66,0xc3,0xda,0x79,0xce,0x60,0xe7,0x4c,0xae,0xd9, +0x50,0xc6,0x2a,0xef,0x97,0x08,0x56,0xe1,0xcf,0x41,0x58,0x0d, +0x2f,0xcb,0x58,0xe5,0xdd,0x13,0x45,0xeb,0x1f,0x65,0x79,0x28, +0x9a,0x2d,0xdb,0x83,0x6b,0xf8,0x43,0xac,0x0f,0xc1,0x6e,0xec, +0x0f,0xf3,0x84,0x04,0x2a,0x43,0x98,0x03,0xde,0xce,0x0b,0x1b, +0x63,0x5a,0x24,0x55,0xa6,0xe5,0x7b,0x08,0x63,0x9c,0x23,0xcf, +0x16,0x1c,0x0d,0x17,0x67,0xb1,0x04,0xa0,0x2d,0x64,0xdc,0xc4, +0xde,0x22,0x6c,0xe7,0xcd,0xc5,0x59,0x3b,0x47,0xc3,0x64,0x11, +0x0f,0x40,0xc0,0x3f,0x6e,0xba,0x90,0xba,0x09,0xf2,0xee,0xcd, +0x96,0x3e,0xce,0xd4,0xc7,0xe2,0xcc,0x78,0x90,0xc6,0x0b,0x5a, +0x5d,0x2f,0x96,0x73,0x7c,0xe9,0x10,0x28,0xf2,0x03,0xd4,0xaa, +0x8b,0x41,0x38,0x6c,0x36,0x9e,0xe1,0x8f,0x08,0x7e,0xe0,0xc8, +0x18,0x74,0xd4,0x8f,0xb3,0xe1,0x81,0x83,0xc4,0x60,0x55,0xc2, +0xae,0x9a,0xfb,0x42,0xe7,0xe8,0x55,0x97,0xe4,0x94,0x04,0x74, +0x77,0xe0,0xa2,0xa2,0x80,0xe2,0x6b,0xc3,0x82,0xc9,0xc8,0xa8, +0xc6,0x5d,0x14,0xd3,0x9d,0xb1,0xb4,0x4b,0x62,0xea,0xe0,0x56, +0x92,0xaa,0x71,0x5b,0x8d,0x3d,0x08,0xbf,0x05,0xbf,0xb1,0xf0, +0x9a,0x64,0x83,0xbb,0x24,0x4a,0x35,0xb6,0x88,0x96,0xb4,0x02, +0xb8,0xe7,0x1f,0xd4,0x15,0x16,0xd0,0x45,0x5c,0xd1,0xae,0x52, +0x57,0xdd,0xb3,0xc9,0x16,0x7a,0xaf,0x40,0x04,0x5b,0x4a,0x86, +0x14,0x87,0x3b,0xfe,0x1e,0xae,0x64,0xf9,0xcd,0xe5,0xbc,0x7a, +0x78,0x71,0x96,0x1d,0xb8,0x49,0xd5,0xcc,0x39,0x1c,0xfb,0x85, +0xde,0xb0,0x73,0x07,0x15,0x47,0x83,0x31,0x80,0xcf,0xe8,0x79, +0x4a,0x9f,0x8e,0x85,0x8b,0xa0,0x5b,0x75,0xda,0x19,0x3b,0x45, +0x5e,0x41,0x0a,0x1f,0x74,0x3b,0xc9,0xe8,0x11,0xa8,0x8f,0x54, +0x28,0xa5,0x4d,0x65,0x0f,0x13,0x4a,0xa9,0xc5,0x12,0x49,0xc6, +0x01,0x02,0x09,0x59,0x5b,0xf9,0xba,0xef,0x39,0x77,0x35,0x46, +0x3e,0xf2,0xe6,0x2f,0x67,0xb5,0x21,0xbc,0xd3,0xf1,0x43,0xe4, +0x71,0xcf,0x0a,0x86,0x05,0xe3,0xe5,0x3c,0xb3,0x2e,0x3a,0x70, +0xe0,0x76,0x5e,0x87,0x99,0x83,0x33,0x4c,0x3d,0x7d,0x64,0x3c, +0x75,0x51,0x03,0xb5,0xce,0x4e,0x78,0x5a,0x6f,0xc1,0x9f,0xef, +0xf8,0xcf,0x2f,0x2a,0xf9,0x3b,0x9a,0xed,0x7f,0xd1,0x99,0x94, +0xf8,0x0b,0x25,0xea,0x2c,0x05,0xd8,0xb2,0xb3,0xb8,0x38,0x18, +0x69,0x28,0x45,0x37,0x73,0x3b,0x22,0x85,0x45,0x5a,0x5e,0xa4, +0x0d,0x23,0x6a,0x38,0x5d,0x3b,0x9a,0x9d,0x85,0xf8,0x2c,0xfc, +0x26,0x5e,0xe6,0x15,0x59,0xba,0x1d,0xbc,0x17,0x93,0xf3,0x36, +0xc9,0x06,0xb7,0x40,0x1a,0x68,0xa5,0xa0,0xb5,0x92,0x93,0x36, +0x6d,0xe9,0x54,0x24,0x8a,0x3c,0x1a,0x92,0x43,0xd8,0x55,0x60, +0x96,0x61,0xd5,0xf7,0xca,0x1c,0xc2,0x8f,0x07,0xb6,0x6d,0xa4, +0xc1,0x84,0x03,0x8d,0xd5,0x32,0xa5,0xc5,0x41,0x6b,0x32,0xff, +0x60,0x3f,0x64,0x6a,0xd5,0xba,0x29,0x72,0xc2,0x15,0x65,0x48, +0x3e,0x03,0x2e,0xe4,0xf1,0x72,0x73,0x52,0xce,0xb1,0x48,0x54, +0x2d,0x56,0x6c,0x10,0xae,0x68,0x06,0xc8,0xb7,0x4b,0xd6,0x7a, +0xf1,0x1f,0x4f,0x54,0xcb,0xa5,0xaa,0x40,0x94,0xd0,0xc4,0xf1, +0x3a,0x8c,0xaa,0xc1,0x91,0x9d,0xc5,0x05,0x45,0xf2,0x3e,0xc9, +0x33,0xcf,0x77,0xde,0xa3,0x24,0xef,0xaf,0xe7,0xb3,0xeb,0x64, +0x99,0xae,0x9e,0x4f,0x82,0xaa,0xe4,0xa0,0x4c,0xa1,0xd5,0xba, +0x33,0x3f,0x28,0x0b,0x17,0xe7,0xaf,0x46,0x33,0xef,0x2c,0xc4, +0x06,0xd2,0x5c,0x30,0xe3,0x0b,0x30,0x06,0xff,0x0b,0xf5,0x38, +0x27,0x7a,0xc5,0xba,0x35,0x4f,0xf8,0x8e,0x42,0x05,0x2c,0x69, +0x36,0x6a,0x58,0x32,0x65,0x75,0xeb,0x66,0xd5,0x54,0xd1,0x4e, +0x6e,0x16,0xab,0xbc,0xa6,0x1c,0xe5,0xd5,0x15,0x44,0x9f,0x92, +0x9a,0x72,0x1c,0x8b,0xb8,0xba,0x98,0xd6,0xaa,0xe5,0x62,0x98, +0xa5,0x5f,0x70,0x2d,0xc4,0xa9,0xd5,0xe0,0xf5,0xfa,0xb0,0xba, +0xeb,0xfa,0xba,0xa4,0x85,0xd7,0x43,0x5f,0x75,0x49,0xee,0xda, +0xc4,0xfc,0x8c,0x8e,0x56,0x36,0xdc,0xd6,0x25,0xc8,0x3c,0x9c, +0x1a,0x57,0xb7,0xaf,0xb6,0xd9,0x54,0x43,0xb1,0x3d,0x78,0x4d, +0xdd,0x5e,0x55,0xe2,0xd2,0x4e,0xfb,0x16,0x05,0xe9,0x24,0xa6, +0x30,0xe8,0xe9,0xc4,0x7a,0xe3,0x73,0x30,0xec,0xa9,0xb2,0x0e, +0x1b,0x2f,0x54,0xaa,0x41,0xdd,0x53,0x6e,0x02,0xf2,0x9a,0x06, +0x21,0x73,0x1f,0xd2,0xf8,0xa8,0x11,0xcb,0x53,0x88,0x6a,0x79, +0x3b,0x18,0x9a,0xe2,0x9c,0x82,0xa5,0x31,0x5e,0x96,0x78,0x3e, +0x52,0x62,0xd1,0x69,0x93,0x64,0x82,0x72,0xd0,0x03,0x1d,0x1d, +0xd0,0xb8,0x3d,0xb1,0xa2,0x89,0xe7,0x5f,0xe3,0x3e,0x75,0x71, +0x81,0x31,0x59,0x2e,0x2e,0xf0,0x21,0x53,0xdf,0xef,0x7b,0x0d, +0x95,0xd0,0x48,0x39,0x26,0xb7,0xc1,0xa1,0x61,0x63,0xe7,0xab, +0x30,0x76,0xf0,0x2e,0x8b,0x2a,0x01,0x48,0x81,0x75,0x02,0x6d, +0x40,0x2e,0x2e,0x06,0xe9,0x64,0x68,0x9e,0x25,0xd5,0x2d,0x27, +0x3f,0x2a,0x90,0x0c,0x27,0x9d,0x6e,0x5a,0x58,0x97,0xcd,0x2c, +0xe1,0xd0,0xbc,0x83,0x6a,0xac,0x07,0x76,0x2b,0x7b,0xfe,0xcb, +0xe9,0xf4,0xd0,0xce,0xcf,0xe4,0xba,0xbb,0x34,0xc2,0xe0,0xfc, +0x44,0x81,0xf8,0x92,0xf7,0x0d,0x0e,0x96,0x89,0x7b,0x77,0xbb, +0x84,0x81,0x36,0x1d,0xff,0xd1,0x1a,0xc1,0x35,0xec,0xd7,0x8d, +0x3d,0x4c,0x34,0x82,0x43,0xe4,0xf4,0x3d,0x92,0x38,0x8f,0x15, +0x66,0x43,0xe5,0xfb,0x2f,0xe6,0x20,0x9c,0x5a,0x16,0x25,0x7f, +0xf0,0x1e,0x45,0xb4,0x5e,0x7a,0xde,0xa3,0xf4,0x98,0xbc,0x3a, +0x81,0xf9,0x3d,0xb2,0x71,0x95,0x4e,0x9d,0x8b,0x6d,0xfc,0x5d, +0x10,0x0b,0xfe,0x57,0xc9,0x84,0xe9,0x75,0xe3,0x69,0x77,0xa4, +0x7c,0xec,0xb8,0x3c,0x8c,0x19,0x41,0xec,0xc8,0x03,0xa7,0xa9, +0x3a,0x6c,0x49,0x08,0x0b,0x82,0xa0,0xdc,0x89,0x3e,0x49,0x3b, +0x84,0x45,0x29,0x51,0xb1,0xb8,0x4a,0xda,0x81,0x29,0xa9,0xef, +0x77,0xdf,0xbc,0xd4,0x6b,0x3d,0x56,0xb0,0x8f,0xdf,0x32,0x42, +0xea,0x99,0xbe,0xfa,0x90,0xd8,0x21,0xdf,0xe9,0x01,0x76,0x4a, +0x93,0xa0,0x66,0x7a,0x90,0xf5,0x6a,0x66,0x9a,0x2f,0x3a,0x7d, +0x82,0x40,0x72,0x3c,0xbf,0x42,0xb8,0xb9,0x0a,0x5a,0x9f,0xd0, +0xbb,0xf0,0xce,0x14,0x73,0x91,0xa0,0xfb,0x3c,0x3f,0x31,0x42, +0x6e,0x95,0x04,0x5c,0x79,0x75,0xad,0x16,0xf3,0x32,0xa1,0xe7, +0x16,0x09,0x79,0xf5,0x3d,0xb5,0xda,0xa2,0x99,0x10,0x25,0x4d, +0xde,0xd6,0xf3,0x69,0xb4,0x5a,0x2d,0x5d,0x36,0xfd,0xf8,0x92, +0x63,0x63,0xd9,0x67,0xd5,0x00,0x44,0x9e,0xad,0x30,0xe8,0xe9, +0x21,0x70,0xa2,0xcc,0xbc,0x7a,0xce,0x24,0x96,0xc0,0x7f,0x7c, +0x59,0x53,0x00,0x32,0xf8,0x15,0xfc,0xc5,0x68,0x0c,0x02,0x81, +0x3f,0xa7,0x73,0x90,0x90,0x02,0x8e,0x57,0x48,0xbc,0x67,0xf6, +0x1e,0x99,0x93,0x97,0x7d,0x85,0xbf,0xeb,0x5d,0x36,0xe3,0x46, +0xa3,0xe8,0x49,0x98,0x06,0x23,0xee,0xdc,0xb7,0xf6,0xcb,0xec, +0x44,0xa3,0xee,0xec,0x23,0xc0,0x03,0x23,0x28,0x8d,0x2d,0x0f, +0x35,0xda,0xe7,0x34,0xc8,0x56,0x42,0x7b,0x56,0xc4,0x12,0xa4, +0x90,0x47,0xcf,0x3e,0x57,0x90,0x0b,0x4d,0xab,0x27,0xf8,0xc7, +0x97,0xf7,0x21,0xb9,0x96,0x4b,0xbf,0xb7,0x15,0x75,0x88,0xad, +0x86,0x81,0xb6,0x2f,0x0f,0x08,0xe5,0x92,0x08,0xc2,0xf1,0x00, +0x67,0x6e,0x12,0x16,0x9f,0xd0,0x63,0x0b,0x39,0x27,0xe0,0xe4, +0x01,0x7d,0x98,0x60,0xbd,0x08,0x25,0x47,0xbf,0x9a,0xd4,0xd8, +0x7a,0x2d,0x3f,0x8e,0xb9,0x5c,0x7f,0x52,0xfd,0xc8,0x7b,0xb7, +0xe0,0x4d,0x88,0xd0,0x31,0x3f,0xe4,0x8f,0x9b,0xd4,0x74,0x25, +0x88,0xe9,0xe9,0x55,0xa9,0x8e,0x0b,0x1a,0x08,0x3c,0x20,0x1b, +0x48,0x6c,0xbb,0xd1,0xb4,0x85,0xdd,0xe2,0x42,0xdf,0xea,0xcb, +0xae,0xfe,0xbd,0x7b,0x04,0xbd,0x2a,0x6b,0x1b,0xc0,0x2c,0xfa, +0xa6,0x42,0xa6,0xf5,0x66,0xe2,0x15,0x2b,0x06,0x77,0xc7,0x22, +0xa8,0x91,0x5e,0xe3,0x68,0x0a,0x6a,0x75,0xbf,0x98,0x5e,0x51, +0x6f,0x56,0xcb,0xe8,0x7d,0x09,0xa8,0x95,0xc5,0x83,0x69,0xaa, +0x15,0x3a,0x21,0x93,0x9c,0x88,0xf7,0x76,0x2a,0xa1,0xe5,0xbe, +0x2c,0x74,0xe3,0xa7,0x77,0x60,0xb6,0xca,0xed,0x0d,0x66,0xe2, +0x1d,0x0b,0xc5,0x62,0x99,0xce,0x97,0x78,0xd0,0x68,0x47,0x24, +0x07,0x60,0x5b,0x17,0x52,0x82,0x28,0x38,0xc0,0xb9,0x48,0x96, +0xab,0xbc,0xa8,0x10,0x09,0x86,0x5d,0xa3,0x2a,0xb4,0x83,0x46, +0xb9,0x43,0x39,0x00,0xe9,0x1f,0xd2,0xd9,0x64,0xfe,0x01,0x7b, +0xe5,0xd9,0xfc,0x66,0x01,0x1c,0x9d,0xbc,0xc4,0x82,0x66,0x92, +0xf1,0x31,0x4f,0x91,0xf0,0x0b,0xdf,0x20,0xae,0x56,0x7a,0xee, +0x58,0xaa,0xd3,0x18,0xd4,0xb0,0xcc,0xa0,0x34,0xca,0xcf,0x30, +0x66,0xab,0x5e,0x1a,0xaf,0xf0,0xff,0x06,0x1d,0x72,0x76,0x3e, +0xa3,0x10,0x5d,0x8d,0x8c,0x1a,0x6f,0x6c,0x18,0x22,0xec,0x23, +0xbb,0x86,0x73,0x94,0xc4,0x46,0x83,0xad,0x0d,0x85,0x5a,0x69, +0x18,0xdf,0x10,0xa3,0xf3,0xa8,0xb3,0x07,0xea,0x4b,0x82,0x6b, +0xba,0xaa,0x46,0x65,0x02,0xde,0x4a,0x5b,0xc2,0x51,0x08,0xdc, +0x54,0x16,0x25,0xe9,0x7a,0xab,0xcb,0xf6,0x48,0x4e,0xc5,0xd8, +0xe7,0x91,0x5f,0x2b,0x3e,0x07,0xe9,0x80,0xdf,0xd3,0xe3,0xbb, +0x07,0x65,0x75,0xbf,0x5e,0xb9,0xfd,0xba,0xad,0x33,0x8b,0xce, +0xf0,0xee,0xa1,0x61,0xae,0x1f,0xd8,0x43,0xd2,0x61,0xa4,0x69, +0xeb,0x0e,0x33,0x2d,0xd9,0xd8,0xa7,0x52,0x6c,0x77,0xec,0xed, +0x2f,0x2c,0xd5,0x90,0x8e,0x2a,0x40,0x41,0x4e,0x7d,0x6d,0x3c, +0x4a,0x2b,0x1c,0x0c,0xa8,0x0d,0x64,0xb2,0xe1,0x5b,0xcb,0xd6, +0x56,0x40,0x69,0x30,0x2e,0x7a,0x04,0xeb,0x5a,0x5e,0xed,0xf1, +0x7c,0x3d,0x5b,0x7d,0xd1,0x41,0xd6,0xf0,0x3c,0xce,0x51,0xe4, +0x28,0x18,0xc3,0xf1,0xf1,0x42,0x2a,0xa6,0xb0,0x85,0xd4,0x84, +0x5d,0x16,0x59,0x32,0xca,0x92,0x12,0x4b,0x0e,0xb4,0x59,0xa3, +0x7d,0x36,0x2b,0x22,0xef,0x7a,0x15,0x8b,0x0a,0xfd,0xf2,0x94, +0xbc,0xd4,0x44,0x90,0x55,0x97,0x1d,0x78,0xc7,0xab,0xc2,0xce, +0x3d,0xd0,0x40,0xa5,0xe6,0xf1,0x2b,0x18,0x7e,0x3d,0x13,0x26, +0xf8,0xee,0xfb,0x81,0x5c,0xa8,0xa1,0xa7,0x22,0xf2,0xc9,0xad, +0xfb,0xc8,0x89,0x6c,0x1f,0xbc,0xab,0x27,0x96,0xc9,0xb0,0x5f, +0x8a,0xa9,0x59,0xba,0x6e,0xbb,0x9e,0xfd,0x1e,0xd9,0xa1,0xcc, +0x60,0xfc,0xfb,0xb9,0xb1,0x2e,0x86,0x6d,0xf8,0x3f,0xc6,0x10, +0x45,0x89,0xbe,0x72,0x1b,0x05,0xfb,0xb9,0x63,0xd8,0x53,0x2c, +0xe6,0x1f,0xcc,0x29,0x55,0xf1,0x5e,0x66,0x61,0x83,0x0d,0xa3, +0x68,0xbb,0x6e,0x9a,0x66,0xab,0x64,0xa6,0xd6,0xdc,0x16,0xc3, +0x6a,0xcc,0x5c,0xb6,0x53,0x69,0x6f,0x31,0xae,0xdc,0x14,0x4e, +0xd5,0xc6,0x63,0x65,0xf6,0xf3,0x49,0xaf,0x3a,0x39,0x4e,0x27, +0x07,0x8d,0x1e,0xee,0x83,0x2a,0x1c,0xb5,0x3b,0x2d,0xe6,0x88, +0xac,0x4e,0x42,0xfd,0xca,0xfd,0xed,0x38,0x35,0xfb,0xdc,0x95, +0x14,0xab,0x5f,0x1c,0xb6,0xf4,0xb0,0xb1,0x4f,0xe7,0x6d,0x7b, +0x77,0x87,0x7a,0x9e,0xb5,0xe5,0x4a,0x0f,0x30,0xde,0xdd,0x95, +0x92,0x50,0x15,0x4d,0x52,0x30,0x1a,0x57,0xa8,0x9c,0xe5,0x15, +0x63,0x7e,0xfa,0xdf,0xa7,0xc7,0x20,0xdd,0x0e,0xde,0xba,0x56, +0x40,0xed,0xb4,0x53,0xb1,0xf9,0x6c,0x5d,0xb8,0xaa,0xd8,0x94, +0x71,0x36,0xe8,0x42,0xbd,0x43,0x97,0x4e,0xa2,0xb8,0xd9,0x2c, +0x6c,0xdf,0x3b,0x3b,0x38,0xc1,0xff,0xb1,0x9d,0x17,0x8b,0x97, +0xf1,0x4f,0x97,0x78,0x3c,0xdb,0x1e,0x2f,0x13,0x34,0xf8,0xaa, +0x3b,0x24,0x1c,0xfa,0x56,0x33,0x59,0x37,0x35,0x63,0x3b,0x45, +0xc6,0x60,0xc5,0x66,0xac,0xde,0x89,0x8d,0xec,0xcd,0xb4,0xe2, +0x6e,0xda,0x81,0x5b,0x69,0x91,0xe5,0xd8,0xb7,0x49,0xb3,0x6a, +0xf7,0x79,0xc8,0xf0,0xde,0xfa,0x1f,0xdf,0xf2,0xb4,0x5d,0x98, +0xa5,0xf1,0xfa,0xa5,0x5e,0x3a,0x06,0xf4,0x2c,0x2a,0x8b,0x2d, +0xcf,0xbe,0x7c,0x0f,0x5b,0x7a,0xf1,0xe2,0x02,0x1f,0x76,0x5d, +0x5d,0x5c,0xdc,0xdd,0x51,0x32,0x67,0x46,0x85,0xcc,0x58,0x2e, +0x02,0xcd,0x17,0x78,0x5f,0x1a,0xf1,0xab,0x20,0x5b,0xf8,0x9b, +0x6c,0xaf,0xac,0xcf,0x25,0xe9,0x15,0x6d,0xf7,0x43,0x26,0x52, +0x4e,0xa2,0xd2,0x5f,0x2a,0xb5,0x73,0xe1,0x77,0x05,0x52,0x6e, +0xb7,0xf9,0x5d,0x83,0x91,0x40,0xe5,0x22,0x9b,0xa9,0x8d,0x13, +0x2e,0xec,0x1a,0x38,0xa9,0x9c,0xb2,0xbb,0x62,0xcb,0x97,0xb2, +0x6b,0x21,0x26,0xb8,0xf8,0xad,0xd9,0xd4,0xf6,0x1a,0x84,0xb2, +0xe1,0xb7,0x51,0xa1,0x78,0xd4,0x24,0x61,0x13,0xf3,0xe2,0x1b, +0x5c,0x4d,0x10,0x6d,0xed,0x04,0x7f,0x7a,0xf8,0x76,0x74,0x06, +0xc6,0xa5,0xd7,0x78,0x0b,0xc3,0x55,0x9a,0x06,0xe5,0x69,0x9f, +0xa0,0x31,0x9e,0x8e,0xb2,0x0c,0xd2,0x05,0x29,0x1b,0xf7,0x8d, +0x39,0xac,0x0c,0x81,0x92,0x46,0xf0,0xef,0xff,0x10,0xac,0x1b, +0xd2,0xbf,0xf6,0x51,0x0d,0x57,0x01,0xe9,0x9e,0x5f,0x59,0x4e, +0x6d,0x9a,0x31,0x82,0x3f,0x2f,0x26,0xb8,0xcd,0x50,0x46,0x51, +0x2e,0x1b,0xa9,0x28,0xfc,0xea,0x42,0x07,0xc5,0x7d,0xf7,0x54, +0x23,0x41,0xff,0x5c,0x17,0x38,0xa2,0xdb,0x44,0x3c,0xc1,0xf7, +0xf5,0x08,0x4a,0xea,0xf4,0x16,0x74,0xf5,0x53,0x98,0x01,0xbd, +0x00,0x2a,0xcb,0x6b,0x60,0x62,0x89,0x0b,0x0a,0x91,0x7b,0x08, +0x45,0x08,0xc0,0x92,0xd7,0xec,0xd5,0x48,0xd0,0x73,0xb5,0x61, +0xf1,0x5d,0x67,0x90,0x59,0x2c,0xc1,0xbd,0x00,0xc6,0x74,0x0a, +0x83,0xc9,0x08,0x50,0xe5,0xf1,0x8f,0x10,0x6b,0x78,0x55,0x04, +0x22,0xfb,0x9c,0x98,0xae,0x32,0xb8,0x12,0x32,0xdc,0x3d,0x23, +0x33,0x0c,0x59,0x57,0x23,0x97,0x21,0x90,0xfa,0x1a,0x85,0xfa, +0xec,0x43,0x8a,0xaa,0x1e,0xd6,0x17,0xc0,0x3c,0xff,0xe3,0x18, +0xcc,0xc5,0x06,0xbf,0xd2,0xd2,0xe8,0x3a,0xc3,0x5a,0xfb,0xe7, +0x81,0x26,0x78,0x1d,0xe8,0x06,0x0b,0x73,0xf3,0x4e,0x23,0x20, +0x4f,0xdd,0x57,0x29,0xc6,0x20,0xfc,0x2d,0xb1,0x28,0x54,0x20, +0x8d,0x40,0x9b,0x23,0x0e,0x68,0x10,0xfa,0x4d,0xac,0xea,0x85, +0xc4,0x57,0xb4,0x58,0x25,0x45,0x37,0x80,0x3d,0xf4,0x77,0x57, +0xe5,0x16,0xb0,0xe1,0x0f,0xae,0x17,0xd1,0x48,0xa1,0x49,0x8e, +0xe3,0xe8,0x49,0x94,0xdc,0x98,0x01,0x83,0xd9,0xad,0xd1,0x0c, +0xbd,0xd9,0x20,0xf3,0x26,0x9d,0x4c,0xa6,0xd8,0x67,0x46,0x0a, +0x55,0xd9,0x06,0xf9,0x8c,0xa9,0xa7,0x16,0xe8,0xe4,0x9c,0xbc, +0x93,0x55,0xc5,0xcd,0xc6,0x2f,0xe1,0x77,0x0a,0x00,0xfd,0xc9, +0x7e,0x29,0x00,0xf8,0xbd,0x4b,0x98,0x39,0xde,0xf7,0xa8,0x37, +0x56,0xf3,0xc5,0xbd,0xbb,0xa2,0xb5,0xbf,0x2f,0x5a,0xde,0x61, +0x5c,0x39,0xa4,0x3b,0xaa,0xaa,0xdb,0xd5,0x1f,0x87,0xd7,0x5d, +0xec,0x92,0xf0,0x8f,0xea,0x90,0xd6,0xde,0x1e,0x69,0xd5,0x77, +0x09,0x06,0xd2,0xdb,0xd1,0x27,0x6f,0x4a,0x7d,0xb2,0x39,0xa0, +0x4f,0x36,0xbf,0xbb,0x4f,0x0a,0x35,0x58,0x1d,0x14,0x56,0xf7, +0xc9,0xc1,0xf5,0x99,0xae,0x0b,0x2b,0x86,0xc9,0x59,0xa7,0xbe, +0x57,0xc8,0x74,0xac,0xed,0x92,0x62,0x2f,0x04,0x76,0x2f,0x7d, +0x17,0xfe,0x62,0x77,0xc9,0x77,0x3b,0xbb,0x84,0x22,0x1c,0xde, +0xbb,0x4f,0xf6,0x77,0xc9,0xef,0x51,0x59,0xf7,0xed,0x8e,0xc3, +0xea,0xfa,0xf4,0xbe,0x60,0x73,0xbe,0xbe,0x37,0xee,0xd7,0x19, +0x6e,0x5f,0xe0,0x11,0x86,0x98,0x43,0x04,0xf5,0xd5,0x68,0x26, +0xa1,0x80,0x30,0x82,0x4f,0x21,0xc3,0xa3,0x48,0x95,0xc6,0xd0, +0x34,0x0f,0x41,0x32,0x24,0x24,0x34,0x27,0x9b,0xad,0x35,0x19, +0xf3,0x66,0x9d,0x63,0x66,0x6e,0x6c,0xa3,0xa3,0x1a,0x80,0x17, +0x51,0x3b,0xb1,0xb0,0x69,0xba,0x0f,0x95,0x63,0x7c,0x81,0x39, +0xb4,0x03,0x46,0x36,0x68,0x6d,0xfb,0x8e,0xe3,0x43,0x94,0x23, +0x0e,0xf0,0xa4,0x6b,0x4b,0xa9,0x3c,0x51,0xf6,0x13,0x65,0xb8, +0xf2,0x13,0x3f,0x0e,0x1c,0x8e,0xc3,0xb7,0x25,0x15,0xf1,0x59, +0xc1,0xda,0x8c,0x07,0x51,0x38,0x0c,0x8c,0x69,0x4b,0xf6,0xac, +0xde,0xc8,0xda,0xe0,0xc6,0x24,0x51,0x75,0x98,0x53,0x24,0x81, +0x6e,0x02,0x2c,0x27,0xf6,0xf3,0x96,0x3e,0xda,0xd2,0x84,0x83, +0xb0,0x08,0xec,0x06,0xb7,0x9a,0xad,0x36,0x73,0x63,0xb3,0x3e, +0x86,0x4e,0xe9,0xd6,0xf2,0x82,0xab,0x66,0x14,0x52,0x37,0xaf, +0x0a,0xca,0xbb,0x8b,0x15,0x6e,0xa2,0x2e,0x6b,0x34,0x80,0x60, +0x75,0xb3,0x2d,0xec,0xbf,0xdc,0xe3,0x25,0x49,0x0b,0x7e,0x63, +0xe1,0xfd,0x45,0x1e,0x94,0xd4,0x38,0x65,0x79,0x71,0x30,0x4e, +0x11,0x22,0x07,0xa9,0xa4,0x59,0x58,0x4b,0x61,0x3a,0x77,0x07, +0x89,0x99,0xf5,0x5d,0x1d,0x13,0x37,0x37,0x05,0xe9,0x6a,0xea, +0xa2,0x14,0x39,0x51,0x2a,0x77,0x0a,0x49,0xf5,0x2e,0xa2,0x83, +0xda,0x55,0xaa,0xbb,0x1e,0xbb,0x4b,0xd5,0x61,0x72,0xe6,0x36, +0x44,0x61,0x77,0x92,0x2d,0xd6,0xbd,0xa8,0x88,0x5d,0xb5,0xb3, +0x47,0x54,0x81,0xa6,0xdd,0x25,0x2f,0x54,0x34,0x66,0xd3,0x25, +0xeb,0xcb,0x49,0x7a,0x9b,0x4e,0xaa,0x36,0xc0,0x8b,0x88,0x8f, +0x8f,0xb1,0x18,0x17,0x76,0x2f,0x32,0x16,0x07,0x42,0xac,0xac, +0xf3,0xa0,0x3c,0x84,0xe2,0x8f,0x60,0x2b,0x76,0x23,0x0e,0x06, +0x0c,0x7f,0x19,0x12,0x7e,0xa0,0xbd,0xd2,0x8d,0xb6,0xca,0x6b, +0xf3,0x72,0xb9,0xce,0xae,0x2b,0x97,0xf9,0x94,0x53,0xbf,0xce, +0x2f,0xad,0xec,0xd5,0x3c,0x43,0x2f,0xed,0x24,0xcb,0x16,0xed, +0x21,0xe1,0x22,0x0b,0xd4,0xa3,0x99,0x85,0x5a,0x1f,0x92,0xcb, +0xf7,0xe9,0xaa,0xb5,0x1a,0x2d,0x5a,0xd7,0x40,0xdb,0x14,0xe9, +0x6b,0x8d,0xe7,0x53,0x9a,0x95,0x96,0x6f,0x2f,0x47,0x5e,0x18, +0xd0,0xff,0xfc,0x06,0x6d,0x32,0x35,0x6e,0xe6,0xeb,0x2c,0x41, +0x37,0x5c,0xa6,0xb5,0x11,0xd0,0x1f,0x9a,0xbf,0x18,0x60,0x35, +0x5f,0x8f,0xf9,0xbb,0x02,0x22,0xb8,0x1c,0x8d,0xdf,0xbf,0xe5, +0xd0,0x54,0xee,0x22,0xd2,0x64,0xd8,0x0b,0xc9,0x9e,0x49,0x2e, +0x2d,0x21,0x31,0xd8,0x5d,0x69,0x09,0xe9,0xa0,0x91,0x46,0xde, +0xa6,0x59,0x7a,0x99,0x4e,0x69,0x75,0xdb,0xb8,0x06,0x5b,0x34, +0x99,0x99,0xcc,0xf1,0x7a,0x99,0x51,0x63,0xc9,0xff,0xfb,0x7a, +0x94,0x2e,0x61,0x1a,0x76,0x49,0x83,0x99,0x1a,0xea,0xb6,0xc9, +0x3a,0x8c,0x16,0x5d,0xae,0x58,0x15,0x2e,0xcc,0x65,0x79,0xba, +0x4c,0xc8,0x43,0xd4,0xad,0x90,0x13,0x55,0x85,0xfc,0x95,0x05, +0xf6,0x7d,0xcf,0x1e,0x27,0xca,0x06,0x80,0x59,0xeb,0xab,0xe4, +0x02,0x79,0x6f,0x8b,0xb4,0x95,0xe7,0xf7,0x06,0x17,0xc5,0xbb, +0xeb,0xdb,0x12,0xc5,0x15,0xe6,0x80,0xc8,0x38,0xf5,0xed,0x33, +0x02,0x1b,0x4c,0x86,0xdb,0x1a,0x8e,0x6c,0xaa,0x50,0x9c,0x0e, +0x92,0x0f,0xc3,0x3f,0x9d,0xc2,0x02,0x3a,0x5b,0x41,0x6a,0xbf, +0x75,0xc6,0x91,0x87,0x2d,0x33,0xaa,0xa2,0xd0,0xdf,0x07,0xb3, +0x6c,0x58,0x5f,0xe8,0x43,0x3a,0x59,0x81,0xc8,0x3d,0x56,0xdf, +0xd7,0x09,0x59,0x9f,0x98,0xb0,0x4b,0x1a,0x14,0xe3,0x04,0x06, +0xf7,0x59,0xa7,0xa3,0x5c,0x44,0x97,0x1d,0x83,0x3d,0xbf,0xdf, +0x98,0xcd,0x67,0x49,0x83,0x2a,0x94,0xde,0xa3,0x17,0xdb,0xde, +0x56,0xee,0xae,0xbc,0xb5,0x85,0xbd,0x12,0xc2,0x64,0xfb,0x3d, +0x0a,0xec,0x59,0xb1,0xed,0xb2,0x29,0x63,0x31,0x3c,0x55,0x36, +0x60,0xa1,0xf1,0xca,0x16,0x6c,0xe9,0x7c,0x68,0xde,0x64,0x39, +0xfa,0xf0,0xda,0x13,0x52,0x61,0x99,0x92,0xd7,0x54,0x98,0xd7, +0x56,0x98,0x97,0x2b,0x54,0xdc,0xad,0xad,0xf1,0x8d,0x55,0x23, +0xa7,0xe8,0x04,0xdb,0x69,0x48,0x65,0xa1,0x76,0xab,0x1c,0x08, +0x5c,0x9d,0x71,0x92,0xa9,0x12,0x60,0xca,0xa5,0x63,0xfc,0x46, +0x73,0xf3,0x0d,0x8d,0xbd,0x41,0xf3,0x34,0x31,0x12,0xc6,0xcb, +0xcb,0x5c,0x67,0xfd,0x3d,0xb3,0xb3,0xfc,0x46,0x99,0xa2,0xd7, +0x36,0x49,0xb6,0x26,0xd0,0x5d,0xa0,0x2a,0xc2,0x26,0x57,0x69, +0x8d,0xa0,0x3d,0xa3,0x68,0xa1,0x41,0x3b,0x3b,0x77,0x86,0x84, +0xf4,0x95,0x2a,0x0f,0xbc,0xb3,0x50,0x15,0xe9,0x78,0xb3,0x9b, +0x0e,0xe8,0x99,0x7c,0x2f,0x1d,0x89,0xd0,0xf1,0xc1,0xa5,0x43, +0x75,0x61,0x6e,0x08,0xc9,0xab,0x08,0x31,0x4a,0xdc,0x9e,0x95, +0xde,0x27,0x39,0xce,0x05,0x90,0x74,0xd6,0x89,0xe9,0xb0,0x14, +0xa7,0x19,0xbc,0x61,0xf0,0x8c,0xbd,0x99,0x81,0xfa,0xb7,0x6f, +0x61,0xf6,0xbd,0xbb,0xf3,0xc6,0xa4,0x95,0x38,0x3e,0x20,0x58, +0x8b,0x90,0x8a,0x0f,0xc8,0xc6,0x86,0x03,0x2a,0x15,0x48,0x88, +0x0d,0x39,0x81,0x42,0x11,0x77,0xe8,0xe0,0x86,0x6a,0x78,0xb1, +0xa4,0x3f,0x32,0xff,0x7a,0xf6,0x3b,0x6f,0x50,0xf7,0x7a,0x51, +0x4b,0x10,0x26,0x0a,0x3e,0x7c,0x16,0x56,0x91,0xd1,0xac,0x24, +0xa3,0x59,0x49,0x46,0x78,0x08,0x15,0xc4,0x2e,0x56,0xca,0x7c, +0x5a,0x4e,0xef,0xf6,0xe1,0x35,0x24,0x9c,0x3f,0xe5,0xca,0x9e, +0x1f,0x20,0xc4,0x04,0x03,0xd4,0xf3,0x7b,0xdd,0x14,0xcd,0x38, +0x25,0xd6,0x37,0xe5,0x19,0x79,0x0c,0xaf,0xca,0x69,0x91,0x49, +0xc3,0xa0,0x6f,0x16,0x67,0x75,0x33,0x47,0xd3,0xd5,0x7f,0x4f, +0xf2,0xbe,0xb0,0xda,0xf0,0x7c,0xe0,0x19,0xe1,0x6a,0x9a,0x96, +0xe2,0x5a,0xcf,0x33,0xbd,0xdd,0xcc,0xed,0x9c,0xa1,0x6f,0x7a, +0x49,0x73,0xa7,0xa9,0xe9,0xfb,0x82,0x51,0xa3,0x90,0x58,0x1c, +0xd3,0x0c,0x53,0x80,0x91,0x06,0x04,0xa4,0x43,0xbf,0x6b,0x09, +0x81,0x28,0x5d,0x68,0xc2,0xeb,0xe3,0x63,0x64,0x44,0xc4,0x65, +0x82,0x0d,0x58,0x1b,0xa8,0x9d,0xf4,0x10,0x54,0x6c,0x0a,0x4d, +0x91,0x37,0x6e,0x91,0x3c,0x88,0x4c,0x91,0x37,0x6e,0x11,0xfa, +0xa5,0x33,0x31,0x8f,0x98,0x75,0xe1,0x7d,0xc4,0x13,0xb0,0x6e, +0x43,0xac,0x14,0x8c,0x08,0xd9,0x55,0x4c,0xed,0xf3,0x1c,0xdd, +0x95,0x59,0xb1,0xb1,0xb5,0xfb,0xd6,0xae,0x98,0x0f,0x4a,0xc4, +0x7f,0xe5,0x26,0x9d,0xe1,0xb3,0xd2,0xf5,0xfb,0xe7,0x40,0x7f, +0xa8,0x1f,0x7a,0xc5,0x48,0x46,0x4a,0x71,0xea,0x18,0xe4,0xb1, +0xb0,0x32,0x1d,0x06,0x3c,0x70,0xe3,0xb4,0x2f,0x3c,0xed,0x4a, +0x27,0xf0,0x4d,0xbd,0x44,0x0f,0xd7,0x44,0xf5,0x9f,0x3e,0xf5, +0x36,0x02,0xbe,0x0c,0x5b,0xb1,0x42,0x0d,0xf5,0xb5,0x28,0x42, +0x6f,0x53,0xa5,0x00,0x6f,0xd2,0x59,0xec,0x41,0x0d,0xcf,0xa6, +0xa3,0x9b,0x45,0x77,0x43,0x7f,0x7c,0x13,0x45,0x7a,0x19,0x06, +0x26,0x52,0x71,0x24,0x72,0x88,0x2f,0xca,0x76,0xd5,0x4f,0x2d, +0x86,0x7d,0x8c,0x4a,0xed,0x01,0x60,0x53,0x57,0xe8,0x37,0xe9, +0x39,0x01,0x11,0x42,0x2b,0x52,0x77,0x5c,0x57,0x41,0xe7,0x44, +0x84,0x25,0x1d,0xb6,0x20,0xc9,0xf7,0x89,0xc2,0x73,0x55,0xae, +0x4f,0x4f,0x6d,0x13,0x97,0x81,0x6e,0x5d,0x4d,0x17,0x53,0x4d, +0x9b,0x34,0x3f,0x8e,0x10,0xf4,0xee,0x4e,0x73,0x0a,0xbe,0x47, +0x9b,0xbe,0xa7,0x19,0xfa,0x35,0x1d,0x37,0x90,0x2c,0x2a,0xde, +0x5a,0x49,0x06,0x0f,0xd5,0xa8,0xb1,0x20,0x12,0x8c,0xde,0xde, +0xbd,0x9d,0xa7,0x93,0x07,0x61,0x61,0xc8,0xa3,0xed,0xe3,0x7f, +0xb4,0x46,0x3f,0x6a,0xe3,0xdd,0x96,0x78,0xd5,0x7c,0x77,0xb8, +0x1d,0x12,0x18,0xbb,0x1f,0xd6,0x1f,0x93,0xbc,0x6c,0x6e,0x32, +0xd8,0x07,0x63,0xbd,0x23,0x61,0xca,0x36,0x67,0xe7,0x29,0x9d, +0xb5,0x5e,0x94,0x33,0xc8,0xa0,0xaf,0x2e,0x43,0x59,0xe8,0x27, +0x54,0xca,0x91,0x89,0xa1,0x32,0xa3,0x50,0x09,0x89,0xd0,0xcf, +0xf4,0x9c,0x1e,0xf2,0xab,0x62,0x70,0xe2,0x4e,0xa5,0x6c,0xde, +0xb0,0x7c,0x04,0xac,0x07,0xd5,0x3d,0x6e,0x3a,0x79,0xa7,0x72, +0xaf,0xd4,0x55,0x68,0xc5,0x13,0xad,0x1a,0x95,0xc6,0x65,0xf4, +0x31,0x27,0xce,0xaf,0x44,0x13,0xdb,0xb1,0xc3,0x9c,0xa5,0x94, +0x94,0x52,0x6a,0x27,0xb6,0x6a,0x41,0xeb,0x7c,0x7d,0xe3,0x59, +0x6a,0x2c,0x3e,0x3a,0xfd,0xbb,0x37,0xbb,0xcb,0x7c,0x65,0x73, +0xa8,0x1c,0xd0,0x4d,0x1b,0xa3,0xba,0x08,0x2c,0xb9,0xfb,0x50, +0x05,0x96,0x9b,0x79,0xc6,0xae,0x8a,0xcc,0xf6,0x04,0xcc,0x6a, +0x35,0xf3,0xdb,0x4c,0x8b,0xd5,0x7c,0x74,0x81,0x89,0x2f,0xd7, +0x8b,0x05,0x20,0xcc,0x3c,0xa5,0xbf,0xcb,0xb3,0xce,0x07,0x87, +0x41,0xe2,0x5d,0x57,0xd9,0x6f,0xf2,0x59,0xd1,0x75,0xf4,0xe1, +0xa3,0xdf,0x88,0x66,0x31,0xbf,0x7b,0x38,0x79,0x45,0x42,0x91, +0xf5,0x3f,0x54,0x4b,0x8f,0x1e,0x1b,0xd5,0x22,0xa4,0x3a,0xdc, +0xef,0xd6,0x48,0x6c,0xa1,0x7c,0x41,0x6c,0x75,0x71,0x18,0x77, +0x34,0xdc,0x96,0xeb,0x05,0xae,0x90,0xac,0x51,0x76,0xd2,0xf0, +0xed,0x2c,0xcd,0x70,0xbf,0x3c,0xdd,0x81,0x7d,0xa2,0x13,0x2b, +0x26,0x39,0x93,0x1d,0x0d,0xf9,0x41,0x76,0x7c,0x0d,0x4d,0x75, +0x26,0x4d,0x09,0xc9,0x26,0x6e,0x9e,0x7e,0x28,0x75,0x74,0x90, +0xe4,0x90,0xfe,0xf7,0x59,0x31,0x5d,0xec,0x80,0x78,0xa0,0x27, +0x69,0x50,0xef,0x36,0x11,0xb9,0x49,0xcf,0xad,0xda,0x87,0x15, +0x73,0x35,0x14,0xac,0xa0,0x19,0x8a,0x29,0x07,0x16,0xc7,0x68, +0x00,0xf5,0x2c,0x73,0x33,0x97,0x51,0x8f,0x3d,0xf8,0xbd,0x7a, +0x05,0x46,0x6a,0xe8,0x40,0x0d,0x76,0xa0,0xb2,0xb2,0x85,0xde, +0xcd,0xf2,0x2b,0x55,0x03,0x6f,0x9c,0x6f,0xfd,0xc0,0x52,0xb9, +0xa4,0x28,0x1c,0x85,0x1e,0x38,0x1a,0x3f,0x50,0xfe,0x34,0x3c, +0x6a,0xbe,0x56,0x4e,0x35,0x84,0x22,0xb0,0xf1,0x06,0xca,0x38, +0x30,0x7a,0xc8,0x0f,0x36,0x3c,0x3d,0xe4,0xfc,0x47,0xea,0x89, +0x07,0x61,0x60,0xba,0x47,0xbe,0x78,0x3e,0xc5,0xf7,0x06,0x72, +0xfd,0x4b,0x56,0xf3,0xb1,0xbd,0x7e,0xfe,0x99,0xd3,0xac,0x29, +0x5c,0xf4,0xfd,0xad,0xb3,0x83,0x5b,0xbb,0xf9,0x53,0xd2,0x69, +0x85,0x68,0x88,0xac,0x34,0xa2,0xf8,0xe3,0x46,0xdb,0x10,0x79, +0x57,0x48,0x0d,0x52,0x77,0xee,0x0b,0xde,0x75,0x1d,0x66,0x6d, +0xa5,0x74,0xd9,0x47,0x44,0xb0,0xf6,0x0a,0xee,0x21,0x92,0x5c, +0xe3,0x46,0xd5,0x2f,0x6e,0x53,0xd9,0xb7,0x4f,0xc5,0xf1,0xd4, +0xd9,0x3c,0xb2,0x9a,0xea,0xce,0xd1,0x42,0x56,0x3b,0x75,0x3b, +0x57,0xa7,0xbf,0xd3,0x5d,0xa3,0x52,0x36,0xba,0x7b,0x54,0x4a, +0xbe,0x4b,0xa4,0xb6,0xbe,0x72,0xe6,0xa5,0xe4,0x6e,0x99,0x20, +0x92,0xb3,0xa2,0x13,0xf9,0x97,0xcb,0xe5,0x28,0x57,0xf6,0xb6, +0x62,0x7d,0x1b,0xd6,0xf1,0x79,0x35,0x64,0x5e,0x80,0xcc,0xb5, +0xe3,0x96,0xdb,0xde,0xb2,0xd1,0xe2,0x78,0xaa,0xbb,0x8d,0x8d, +0xf0,0xe9,0x09,0x72,0x50,0xcf,0x0b,0xe9,0x79,0x9c,0x53,0x7a, +0xbd,0x0d,0x6c,0x6c,0xde,0xed,0xd6,0xb7,0x7c,0x81,0x0f,0xeb, +0x90,0xa8,0xa6,0x43,0x22,0xe8,0x90,0x43,0xea,0xdc,0x31,0xff, +0xfb,0x5d,0x6f,0x97,0x06,0xf8,0x7d,0xd8,0xc9,0x11,0x99,0x87, +0x9d,0x75,0xfc,0xf5,0xdb,0x8e,0x0d,0xe7,0x4d,0xfc,0xdb,0xce, +0xd1,0x7c,0xb4,0xf9,0xe2,0x8b,0xe8,0xee,0x28,0x1f,0x32,0x5a, +0xbf,0xbb,0x51,0x15,0xe4,0x87,0x55,0x90,0xdf,0xb7,0x82,0x5c, +0x55,0x30,0x26,0x6d,0x73,0x58,0x2b,0xc0,0xe6,0x80,0x7f,0x45, +0x3f,0x1d,0xfd,0x46,0x93,0x8c,0xf9,0x82,0x55,0x5b,0x77,0xd3, +0x37,0xd9,0x5d,0x9c,0x28,0x4c,0xbe,0xaf,0xdb,0x86,0x68,0x06, +0x0c,0x27,0xe5,0x87,0xba,0x60,0x37,0x57,0x2b,0x0c,0x7e,0x5a, +0x4c,0xb4,0x1b,0x0b,0xa6,0x4d,0x25,0x0d,0xa8,0x30,0xd8,0x44, +0x41,0x0e,0x3a,0x33,0x0a,0x56,0x75,0xfe,0xdd,0x44,0xb7,0xb7, +0x09,0x63,0xa2,0x77,0x13,0xc5,0x48,0x29,0xed,0x69,0x41,0xda, +0x26,0x94,0xc4,0x4d,0x44,0xfb,0x21,0xae,0x98,0x0e,0xa8,0x02, +0xc8,0x07,0x23,0xe0,0x36,0x59,0xae,0xa4,0x0c,0xfc,0xe3,0x53, +0x19,0x6f,0x13,0xf9,0xf8,0x9e,0xdf,0xf9,0x06,0x83,0xba,0xae, +0x62,0x04,0x87,0xfc,0x08,0x33,0x61,0xdc,0x00,0xdc,0x91,0x65, +0x1f,0xdc,0xdd,0x6d,0xa2,0x23,0x6b,0xf3,0x00,0x57,0xa1,0x7a, +0x4a,0xe0,0x9a,0x64,0xb3,0x2d,0x17,0x62,0x73,0x21,0x76,0xc3, +0x69,0x79,0x48,0x94,0x47,0x71,0x1e,0xd1,0xb2,0xde,0x1d,0x3b, +0x03,0xe2,0x03,0xe4,0x1b,0x62,0xb1,0x0c,0xfc,0xe3,0x53,0x19, +0x2f,0x47,0x62,0xf3,0xf0,0x3c,0x67,0x62,0x11,0x1c,0xf2,0x23, +0xcc,0x44,0x62,0x73,0x20,0x36,0xb7,0x88,0xcd,0xa3,0x23,0x6b, +0x4f,0x03,0x89,0xd5,0x33,0x16,0xd7,0xe4,0xeb,0x1e,0x65,0x0e, +0xdb,0xd4,0xf4,0x89,0x51,0x76,0x8a,0xe2,0xb3,0x93,0x86,0x22, +0x63,0x41,0xba,0x30,0x83,0x32,0xe7,0xe5,0x4b,0x77,0x80,0xfe, +0xde,0xd5,0x0f,0x8a,0xa9,0x2e,0x7d,0xd8,0xf6,0x22,0x7d,0xc8, +0xa6,0x32,0x7d,0x06,0xd2,0x85,0x19,0x94,0x99,0xad,0xe8,0x51, +0x3c,0xd7,0xdf,0xbb,0x58,0x8f,0xa4,0xd3,0x90,0x40,0x21,0xc8, +0xa1,0x8a,0x01,0x0a,0x35,0x18,0x6a,0x30,0x28,0x44,0x2e,0x70, +0x28,0x29,0xae,0x9b,0x81,0x5b,0xfd,0xba,0x80,0xb3,0xfe,0xbb, +0xbb,0xf3,0x76,0x5a,0x1d,0x7b,0xa6,0x0d,0xe9,0xe1,0xad,0xbd, +0xa8,0x2c,0x57,0x79,0x74,0x04,0xf4,0x9b,0x2e,0x8c,0xad,0x0e, +0xbc,0xbb,0x3b,0x3a,0x02,0xda,0x73,0x2b,0xd3,0x70,0x8f,0x9f, +0xc9,0xe0,0x47,0x34,0xd8,0xa0,0x22,0x95,0x1b,0x34,0xe6,0xb3, +0x86,0xef,0x1c,0xa0,0x59,0x47,0x07,0xf1,0x47,0x7c,0x21,0x25, +0x6b,0x89,0x9e,0x0e,0x40,0x35,0x27,0x1f,0xf4,0x57,0xe6,0xe4, +0x7d,0x70,0xf2,0x66,0xf0,0x39,0xfb,0x90,0x25,0x26,0x01,0xca, +0xce,0x92,0xcc,0x2a,0x9d,0x14,0x20,0xb2,0x0f,0x2e,0x84,0x8e, +0xec,0x69,0x2b,0xd7,0x78,0x30,0x68,0xcc,0xf0,0x0c,0x07,0x9d, +0x33,0xe0,0xbf,0x0f,0x68,0xee,0xd2,0x3f,0x94,0x42,0xff,0x7c, +0x68,0x40,0xb7,0x12,0x04,0xff,0x9a,0x11,0x2c,0x46,0xee,0xe6, +0x38,0x26,0x29,0xdf,0xe1,0xa3,0xbf,0x1f,0xa9,0x16,0x3a,0x04, +0xf8,0x1a,0x6f,0x6d,0x4a,0x3e,0xa4,0xe4,0x2f,0x25,0x46,0xd1, +0xa0,0xf1,0x72,0x3d,0x9b,0xa0,0xad,0xdc,0xf8,0x61,0x2e,0x3f, +0x5e,0xad,0x93,0x8c,0x7f,0xfd,0x25,0x99,0xcc,0xd4,0xef,0x57, +0xd7,0xc0,0x35,0xfe,0xf9,0xed,0x32,0xe5,0x1f,0x2f,0x61,0x31, +0xba,0xc4,0x9f,0xf4,0xca,0x3d,0x56,0x74,0xb1,0x5e,0x8d,0xad, +0x00,0x1e,0x1f,0xc1,0xa4,0xc6,0xaa,0x2b,0xde,0x72,0x66,0x0b, +0x0e,0xaf,0x50,0xfd,0xf9,0xd5,0x33,0x84,0xc1,0x40,0x20,0x04, +0x9e,0xef,0x87,0xce,0x05,0xf8,0x5b,0x10,0xad,0x37,0x20,0xbd, +0xfb,0x4a,0x28,0x38,0x29,0xf6,0xdd,0x1c,0xda,0xb2,0xaf,0x0c, +0x01,0x49,0x81,0x1f,0xd2,0xe9,0x34,0xe5,0x67,0xf0,0xf7,0x96, +0xb3,0x61,0x75,0xf1,0xd9,0x1a,0xd6,0x5c,0xfb,0x4b,0x12,0x98, +0x2a,0x34,0x9f,0xad,0xae,0xf7,0x16,0x41,0x20,0x29,0xf0,0xf2, +0x30,0xfa,0x5e,0x3a,0xa4,0xbd,0x02,0x81,0xd8,0x59,0x02,0x01, +0x2c,0xd8,0xdf,0x60,0xf9,0xc5,0x61,0x30,0x2b,0x4a,0x85,0x5b, +0xbe,0x57,0xf2,0xd3,0x55,0x3d,0x4a,0x01,0x40,0x94,0x59,0x59, +0x38,0x94,0x88,0x2e,0xac,0xd8,0x1e,0x4a,0x40,0xac,0x28,0xc1, +0xed,0x0b,0x6b,0xa9,0x21,0x88,0xf2,0xc3,0xf0,0xe4,0x3b,0xd1, +0x54,0x09,0x54,0x1d,0x2e,0x05,0xbb,0x13,0x61,0x49,0xd4,0xea, +0xb0,0x11,0xe0,0x4e,0x54,0x75,0x42,0x58,0x87,0xd1,0x86,0xdf, +0x83,0xb8,0x24,0x9e,0xf5,0x38,0x09,0x74,0x37,0xba,0xa2,0xe0, +0xd6,0x22,0x43,0xc0,0x9d,0xa8,0x2a,0x44,0xba,0x0e,0xd9,0xcb, +0x03,0x1a,0x5a,0x14,0xf6,0x4a,0x5c,0x08,0x54,0x8b,0x45,0x4f, +0x26,0x6e,0x39,0xd2,0xaf,0x46,0xef,0x69,0x4d,0xcb,0x6f,0xdf, +0x60,0x26,0x62,0x8d,0x1b,0x9f,0x8d,0x1e,0x7c,0x76,0xf9,0xe0, +0xb3,0xe4,0xc1,0x67,0xaf,0x1f,0x7c,0xf6,0xa6,0x51,0x01,0x07, +0x30,0x37,0xa7,0x9f,0x4d,0x4e,0xcb,0xb9,0x82,0xe1,0xbb,0xee, +0x67,0x3f,0x74,0x3f,0x7b,0xd9,0xb0,0xb5,0xf9,0x1f,0xa7,0xc7, +0x6d,0xac,0x5f,0x5e,0x5e,0xe2,0xeb,0xd5,0x74,0xe9,0x44,0x6a, +0x60,0xf4,0x8c,0x9b,0xf1,0x32,0x46,0x46,0xc6,0x98,0x2c,0x24, +0x37,0xd8,0xc1,0x58,0xf4,0xbf,0x8d,0x66,0xeb,0xd1,0x92,0xea, +0x4c,0x60,0xc6,0xe3,0x9f,0x3f,0xe0,0x1b,0x57,0xf0,0xf7,0xcb, +0xc5,0x32,0x9d,0xd2,0x37,0xa6,0xfe,0xb7,0x35,0xcd,0x76,0xff, +0x6d,0x3d,0xc5,0xaf,0x2f,0xd7,0x6f,0xd7,0x19,0xee,0x89,0xbc, +0x4c,0x16,0xab,0x84,0x5e,0x33,0x0b,0x1a,0x3f,0x8d,0x57,0x73, +0xfe,0xf5,0xe3,0xfc,0x56,0x25,0x7e,0x9d,0x8c,0xf9,0x67,0xa1, +0xfa,0x62,0x2b,0x80,0x14,0x26,0x83,0x29,0xe0,0xfa,0xed,0xda, +0xb9,0x72,0xae,0x9b,0x2b,0xe6,0x2a,0xb9,0x3a,0xae,0x89,0x27, +0x3c,0xac,0xa4,0x9d,0xa3,0xf1,0xa4,0x6a,0xa4,0x85,0x36,0x68, +0x38,0xb3,0x59,0x42,0xc7,0xce,0xda,0x7f,0x81,0xcf,0xe3,0xb9, +0x20,0x70,0x98,0x73,0x03,0x71,0xc4,0xe7,0x81,0x83,0x0f,0x69, +0x70,0xd2,0x36,0x70,0xb0,0xc8,0xfe,0x33,0x48,0xad,0x80,0xeb, +0x49,0x8d,0x12,0xac,0xd9,0xd0,0xf3,0x9b,0x02,0x5b,0x40,0xe1, +0x10,0xe2,0x96,0x90,0x0b,0x6a,0xaa,0x45,0x59,0x6c,0x7f,0xb1, +0xab,0xa6,0x0b,0xd0,0x86,0x99,0xde,0x05,0xc2,0xa9,0xdf,0x05, +0x84,0x26,0xee,0x65,0x0d,0x8d,0x27,0x80,0x9b,0x25,0x1f,0x1e, +0x88,0x11,0xe1,0x75,0x92,0xb3,0x20,0xb4,0x5e,0x9c,0xc8,0xf7, +0x34,0x37,0x50,0x49,0x32,0x1d,0xea,0x6f,0xb6,0x2b,0xf0,0x3b, +0xdf,0xc7,0x4d,0x02,0x75,0xcb,0x1d,0xca,0x45,0x86,0x6e,0x45, +0x16,0x0f,0x69,0x50,0x5a,0x1f,0x65,0xc6,0xb8,0x0c,0x44,0x90, +0x4a,0xfe,0xfd,0x74,0xf5,0xc6,0xb1,0xce,0x0d,0xd3,0x6c,0xc1, +0xa3,0x0e,0xe0,0xbc,0x9e,0xfd,0xf8,0x19,0x3f,0x87,0x45,0x19, +0x2d,0x04,0x69,0x3d,0x4e,0x1e,0x9e,0xe8,0x56,0xba,0xf3,0x39, +0xb4,0x80,0xba,0xb1,0x22,0xc3,0xf7,0x4f,0x3f,0x7f,0xfc,0x30, +0x79,0x24,0xd7,0x3e,0x5d,0x1b,0x52,0x3f,0x57,0x6b,0xd1,0x98, +0xe3,0xe9,0x24,0xf5,0x3e,0xb4,0x6b,0x35,0xff,0x7e,0xfe,0x21, +0x59,0x3e,0x1b,0x65,0x78,0xd4,0x92,0xc6,0x4f,0x5a,0x69,0x4f, +0xc7,0x22,0xb9,0xa5,0x78,0xac,0x84,0x73,0x00,0xc0,0xc3,0x03, +0x87,0x92,0x57,0x3d,0x94,0xfc,0x9a,0x9e,0x6c,0x59,0xdf,0x60, +0x3b,0x36,0x53,0xff,0xb3,0x27,0x87,0x0d,0xb2,0x2a,0xb1,0x78, +0x72,0x62,0x31,0x57,0x0a,0x94,0x85,0x44,0x09,0x76,0xb9,0x8b, +0x34,0x1d,0x95,0x7d,0x55,0xec,0x7b,0x2e,0xd3,0x44,0xae,0x32, +0xdd,0xa7,0x4f,0xa8,0x3d,0xf9,0x51,0x1c,0xa7,0x3e,0x05,0x5f, +0xd4,0xec,0x6b,0xe2,0x5a,0x20,0x56,0xac,0x73,0xa5,0x49,0xe7, +0x93,0xe0,0x69,0x98,0x92,0xd4,0x31,0x1c,0xd7,0x0d,0xc8,0xfe, +0x13,0x5b,0xb5,0xb5,0x06,0xd0,0x87,0x24,0x79,0xaf,0x2b,0xc9, +0x68,0x6e,0x73,0xf2,0xb2,0x42,0x66,0x61,0xe4,0x10,0x88,0x33, +0xc2,0x04,0xac,0x3c,0xc8,0x10,0x54,0x46,0x99,0x0b,0xcb,0x89, +0x1a,0xee,0x4a,0xbd,0x68,0x67,0x4f,0xca,0x8e,0x39,0x00,0x65, +0x7e,0x4e,0x0a,0x00,0x3f,0x27,0x9e,0x3d,0x51,0xfb,0xf6,0x00, +0xfa,0x7e,0x3e,0x7f,0xbf,0x5e,0x14,0x0a,0x70,0x62,0x7d,0x21, +0x9e,0xcf,0xf6,0xd4,0xe3,0x4c,0x7a,0x15,0xc5,0x0f,0xac,0xb9, +0x06,0x0d,0xcd,0xab,0xbb,0x28,0xe0,0x79,0xbf,0x50,0xe0,0x90, +0x3a,0x2b,0x0b,0xee,0x6f,0x71,0x79,0xa2,0xaf,0x44,0x71,0x30, +0x05,0x35,0xa8,0x16,0xc9,0x12,0x8f,0xab,0x80,0x92,0xd3,0xbf, +0x7f,0x76,0x5a,0x30,0xcf,0x5e,0x8c,0x26,0x59,0xfc,0xb1,0xd1, +0x6a,0x74,0x1b,0x8d,0xe0,0xa2,0xdb,0x78,0xd0,0x08,0xc2,0x6e, +0x23,0x6c,0x6c,0x0b,0x70,0x00,0x34,0xea,0x56,0xfb,0x36,0x56, +0xb2,0x7d,0x30,0xd1,0x03,0x6b,0xb8,0x0d,0xbe,0xdc,0x53,0xb4, +0x00,0x7e,0xb9,0x0b,0xbc,0xdc,0x54,0x2e,0x2c,0x93,0x28,0x14, +0xff,0x6a,0x6f,0xf1,0x52,0x91,0x71,0xd7,0x6d,0xae,0x57,0x6d, +0x0a,0x03,0x53,0x2d,0xdc,0xc1,0xa2,0x84,0x5d,0x33,0xd5,0x9b, +0x18,0xcd,0x1b,0x2c,0x82,0x0e,0x28,0xda,0xe4,0xd3,0x8b,0x7e, +0x77,0xaf,0xa2,0xb2,0x05,0x20,0x65,0x9f,0x7f,0x4a,0xd9,0xcf, +0xa2,0xce,0xdd,0x5d,0xd4,0x11,0x14,0xef,0x0e,0x45,0x11,0x35, +0x2b,0x34,0x26,0x12,0x72,0x06,0x58,0xbe,0xbf,0x17,0x21,0xee, +0x7e,0x84,0xa0,0xb8,0xb9,0x1f,0x0a,0xee,0xde,0x66,0x24,0xcd, +0xf8,0xe1,0x9e,0x04,0xc8,0xb6,0x86,0x94,0x5e,0x54,0x4a,0x95, +0xc5,0xb3,0xf3,0x38,0xea,0xf4,0x1b,0x2f,0x7e,0x80,0xa1,0xf4, +0xe5,0x0f,0x30,0x80,0x5e,0xde,0xab,0xba,0x97,0x56,0x53,0xb1, +0xba,0x3f,0x1f,0x5c,0xba,0x42,0xfb,0x33,0xd3,0x11,0xcd,0x87, +0x5a,0xaa,0x65,0x3b,0xea,0x2f,0xf7,0xad,0xe7,0x66,0x5e,0x55, +0xcf,0x66,0xff,0x00,0xf2,0x83,0xd7,0xbb,0x81,0x78,0x84,0xe5, +0xf7,0x62,0x9b,0xb1,0xa6,0x3f,0x8b,0xc2,0x50,0x88,0x79,0xf3, +0xc9,0x28,0x92,0x87,0x80,0xe2,0x21,0xa0,0xf8,0xab,0x26,0x15, +0x2d,0xca,0xa0,0xf1,0x59,0xa3,0xbc,0x37,0x04,0x89,0x5b,0xa3, +0x27,0x17,0xb0,0xba,0x48,0x96,0xa4,0x27,0x9d,0xa4,0xbf,0xc0, +0x44,0xad,0xb5,0x23,0x68,0xc2,0xaa,0x4c,0xd0,0x78,0x4e,0xf2, +0x0f,0x46,0xc7,0x81,0x36,0x2b,0x67,0x59,0xfa,0x8a,0x12,0xbf, +0xc7,0x10,0x66,0x09,0x36,0x04,0x14,0x94,0x93,0x03,0xbd,0x1c, +0x24,0xe5,0xa4,0xef,0xdc,0x24,0x94,0xe0,0xce,0x43,0x50,0x15, +0x55,0xa9,0xef,0x4a,0xc5,0xc5,0xc4,0xf8,0xbe,0x40,0x99,0x35, +0x62,0x83,0x9b,0x0a,0xaa,0xe5,0x4d,0xed,0x1f,0x8a,0xc5,0x68, +0x9c,0xc1,0xf8,0x72,0x92,0xbf,0xbc,0x79,0x71,0x03,0x43,0xc8, +0x49,0x93,0x31,0x02,0x63,0xa3,0xc4,0x44,0xc6,0xcd,0xbb,0x09, +0x20,0xf4,0x55,0x4c,0x96,0xea,0xff,0x52,0x57,0x98,0x77,0x20, +0x2c,0x49,0xb6,0x58,0x4b,0x1b,0xd1,0xaf,0xab,0x72,0x50,0x68, +0x41,0x66,0x9d,0x1c,0xe2,0xce,0x1b,0x37,0x4d,0x49,0x99,0x25, +0x59,0x94,0xfe,0x57,0x25,0x5e,0x2e,0xee,0x14,0xec,0xdc,0xd1, +0xf4,0x05,0x4f,0xdd,0x46,0xc8,0xf8,0x69,0x74,0x9a,0xca,0xff, +0x96,0x9d,0xfc,0x6d,0xd2,0x34,0xf3,0xf9,0x08,0x18,0xa6,0x4d, +0x05,0x0e,0xe1,0x3f,0xba,0xe9,0xc2,0x90,0xb8,0xe9,0x46,0xc6, +0xdc,0x16,0xe1,0xb7,0x6d,0x4c,0x49,0xfa,0xf3,0x6a,0xec,0x98, +0x85,0x9c,0xfa,0x3c,0x9b,0x97,0xe1,0xbc,0xc6,0x67,0x6f,0x5a, +0x9f,0xdd,0xb4,0x3e,0x9b,0xbc,0x52,0x1b,0x3b,0xed,0xcf,0xbe, +0xff,0x6b,0xa3,0x54,0x4b,0x0a,0xa5,0xdd,0x3d,0x26,0x58,0x56, +0x3d,0x7f,0xf9,0x13,0xc7,0x48,0x3b,0x3e,0x6e,0xe2,0xf2,0x99, +0x66,0xba,0x46,0x27,0x0c,0xc3,0x56,0x18,0xc1,0xff,0x5f,0x85, +0x61,0x97,0xfe,0xdf,0x86,0x24,0x40,0xda,0x2f,0x51,0xf4,0x23, +0xbd,0xbd,0xdc,0x2d,0xa5,0x07,0x35,0x90,0x6d,0xe2,0xa9,0x15, +0x65,0x99,0x43,0xb4,0xc9,0xaa,0x00,0x56,0x64,0x9a,0x0c,0xc9, +0x51,0x6b,0x81,0x34,0xfb,0x71,0xf4,0x23,0xdb,0xfc,0x7d,0xf2, +0x7d,0xe4,0xb5,0x57,0x5d,0x35,0xab,0x39,0x37,0x2c,0x2e,0x01, +0xe8,0x2c,0x6d,0x97,0xf3,0x40,0x39,0x74,0x0b,0xc6,0xde,0x66, +0x88,0x92,0x33,0x7b,0x25,0x87,0x69,0xa7,0x90,0x56,0x5a,0xcb, +0x95,0x16,0x85,0x28,0xaa,0xce,0x2a,0x1a,0xe7,0x46,0x17,0x59, +0xed,0xb2,0xb0,0xb0,0x77,0x60,0x76,0xe3,0xfd,0x42,0x8b,0xac, +0x15,0x0e,0x7d,0x17,0x96,0x2d,0x6a,0x63,0xd7,0x59,0xe3,0x30, +0x60,0x79,0x8d,0x73,0x43,0xaa,0xe1,0x53,0x78,0x84,0x1b,0x06, +0x45,0x1e,0x41,0xda,0xa7,0xf1,0xa8,0x80,0xec,0x50,0x1e,0x99, +0x73,0x11,0xbf,0xd0,0x22,0xc3,0x23,0xfe,0xae,0x6c,0xb6,0xcb, +0x23,0x01,0x2c,0xf3,0xe8,0x1a,0x94,0xf4,0x41,0xbb,0x55,0x08, +0x80,0x73,0x59,0x5c,0xb7,0x8f,0x72,0xfa,0x38,0xec,0x55,0xb0, +0xf2,0xec,0x71,0xf2,0xe8,0xc4,0x2b,0x32,0x13,0x53,0x5b,0x0a, +0x25,0x5e,0xed,0x96,0x5f,0x9f,0xc4,0x60,0xaa,0xe2,0x13,0x38, +0xac,0xce,0xba,0x7c,0x87,0x1b,0x86,0xbb,0xf8,0x55,0xc1,0x2e, +0x97,0xb3,0x04,0x54,0x21,0x7b,0x38,0x63,0xfd,0x41,0x3b,0xa4, +0xa4,0x5a,0x0e,0xdc,0x1e,0x65,0xa3,0xb5,0xb0,0x33,0x78,0xe8, +0x96,0x9e,0x3a,0x57,0xf3,0xdd,0x66,0x58,0xf2,0x46,0x67,0x17, +0x15,0x2d,0x2d,0x48,0x1b,0x81,0x69,0xa6,0x38,0xf3,0x02,0x79, +0xe7,0xe3,0x2b,0x8c,0x59,0x3c,0x00,0xed,0x11,0x3c,0x4a,0xf0, +0x21,0x6b,0xf8,0xe7,0x0c,0x8c,0x27,0x18,0x2a,0xf0,0xf7,0x51, +0xf0,0x14,0xfe,0x8b,0x3e,0x87,0x7f,0xb0,0x6b,0xf1,0x59,0x6b, +0xf8,0xb7,0x13,0xe1,0xef,0x87,0x67,0x1d,0xf8,0x97,0xf6,0xe5, +0x82,0xe8,0x49,0x07,0x33,0x1e,0x87,0x0f,0x29,0xff,0xd1,0xd3, +0x4e,0xf2,0x38,0x78,0xf2,0xe4,0xc9,0x63,0xf8,0x73,0x16,0x3d, +0x82,0xb2,0x8f,0xcd,0xb6,0x38,0xd5,0x4b,0x73,0xee,0x0f,0xc9, +0xea,0x7a,0x3e,0xc1,0x73,0x67,0x57,0x83,0x04,0xd1,0x30,0x28, +0x26,0x3d,0x2a,0x27,0x45,0x15,0x69,0x67,0xa1,0x95,0xc6,0x63, +0xcd,0xc1,0x26,0x49,0x8f,0x2a,0xa0,0x2a,0xd2,0x1c,0x6c,0x28, +0x5f,0x0e,0x2e,0x4a,0x38,0x2b,0x26,0x3c,0x2e,0x15,0xe9,0x58, +0x29,0x68,0x98,0x44,0x85,0x6f,0x3b,0x1f,0x77,0x83,0x5c,0x82, +0xc9,0x4c,0x2c,0xa7,0xd8,0xf5,0xe2,0x26,0x58,0x10,0x0d,0xab, +0x58,0xfc,0xad,0x2c,0xfa,0x35,0x8b,0x95,0xc5,0x0e,0x33,0x7f, +0x43,0x82,0xc2,0xad,0x13,0x83,0xcb,0x64,0x7f,0xd5,0xf0,0x2b, +0xef,0x40,0xda,0xe2,0x59,0x55,0xee,0xf2,0xc1,0x67,0x93,0xea, +0xa2,0xd1,0x51,0x6c,0xad,0x88,0x2b,0x0b,0x8f,0x6a,0x0b,0x9b, +0xa5,0xce,0xf1,0xf1,0x01,0x88,0x9e,0x3f,0xf8,0x6c,0xb1,0x03, +0x91,0x52,0x38,0x95,0x45,0xc1,0x10,0xda,0xd5,0x76,0x3d,0x1b, +0x54,0x14,0xc6,0x93,0xb1,0xfa,0xa2,0x66,0xb2,0xad,0x28,0x0a, +0x86,0xd7,0xce,0x5a,0x9d,0x53,0xfd,0x1d,0x7d,0x1d,0x3b,0x19, +0xdf,0xba,0xeb,0xb3,0x92,0x60,0x18,0x4b,0xaf,0x34,0x2c,0xdb, +0xa5,0x1d,0x7e,0x33,0xf9,0xbb,0xc1,0x1a,0x4a,0x0b,0x33,0xca, +0xf7,0xca,0x21,0x26,0x6a,0xeb,0xaa,0x6d,0x8d,0xbf,0x2d,0xeb, +0x2c,0x9b,0x17,0xf1,0x47,0x52,0x6c,0xd6,0x4b,0x44,0xe8,0x1a, +0x19,0x64,0xab,0xf9,0x02,0xfe,0x49,0xec,0x35,0x23,0xab,0x40, +0xaf,0xc9,0x10,0x4d,0x03,0xa2,0x5e,0xb0,0x37,0x35,0xd0,0x2a, +0xd7,0x5a,0x12,0x52,0x22,0x9e,0x18,0x8b,0xc2,0xaa,0xe7,0x19, +0xa2,0xd2,0xb4,0xdc,0xa8,0xda,0x07,0x37,0x83,0x90,0xf6,0xbd, +0x83,0x1b,0x74,0x5c,0xdf,0xfa,0x25,0xcc,0x75,0xe3,0x54,0x8c, +0xf4,0x1d,0x43,0x95,0x21,0x76,0x8d,0x56,0xcb,0x4f,0xa3,0xa6, +0xf4,0xfe,0x31,0x6b,0x9c,0x64,0x6a,0x50,0xec,0x19,0xb9,0xca, +0x6d,0xc6,0x0c,0xde,0xbd,0x18,0xf7,0x0c,0x61,0xcb,0x45,0xa6, +0x0e,0xc1,0xee,0x81,0xec,0x78,0xbc,0x54,0xa3,0xd8,0x3d,0x9c, +0x1d,0x67,0x96,0x6a,0x04,0xbb,0x07,0x75,0xc9,0x5b,0xe7,0x5f, +0x4a,0x03,0x5b,0x8b,0xc6,0xfe,0x61,0x6d,0xa4,0xc8,0x89,0x82, +0x5f,0x21,0xbf,0xe5,0x91,0x4d,0xb2,0xe9,0x8c,0x6e,0x32,0x23, +0x7e,0xe7,0x08,0x37,0x35,0xd6,0x34,0x4a,0xde,0x5e,0xc2,0x30, +0x73,0x00,0xb0,0xb9,0x5e,0xe2,0xdb,0x5d,0x66,0xfc,0x2c,0x93, +0x5f,0xd7,0x49,0xb6,0xd2,0x95,0xcb,0x37,0x5e,0xb8,0x58,0xcc, +0x67,0x59,0xf2,0x0a,0x0a,0xd2,0x58,0x6a,0xbf,0xcb,0xec,0x60, +0x9e,0xeb,0xe5,0x34,0xc0,0x90,0x38,0x78,0xa9,0xdb,0xa6,0x1c, +0x2a,0xa0,0xbc,0x06,0xba,0x57,0xa4,0x63,0xda,0x74,0x3e,0xc5, +0xa2,0xe4,0xcd,0x80,0x3f,0x4c,0x31,0xa2,0xec,0x7a,0x75,0x33, +0xbd,0x07,0x5a,0x6c,0xc8,0x29,0x96,0x21,0x7c,0xf8,0xa3,0x80, +0x6f,0x73,0x33,0xfd,0xa4,0x86,0xbe,0xfe,0xe1,0x7b,0x6a,0xe7, +0xd6,0xf3,0x8b,0xbe,0xf8,0x37,0xf3,0xc9,0x1a,0xdf,0x7d,0x9e, +0xbd,0x5d,0x4f,0xa1,0x1f,0xf9,0x13,0x43,0x11,0x80,0x9a,0xe3, +0xb4,0x46,0x80,0x11,0x45,0x39,0x03,0xd7,0xef,0x18,0x04,0x64, +0xe9,0x35,0x7e,0xb6,0x21,0x0a,0x48,0x9f,0xcd,0x67,0x57,0xe9, +0xdb,0xf5,0x32,0x59,0xc6,0x1f,0xb7,0x3d,0xf3,0xd5,0x4e,0x67, +0xa9,0xe5,0xbd,0x3c,0xa7,0xb8,0xa2,0xc1,0x98,0xf2,0xad,0x1b, +0xd2,0x16,0x6a,0xf8,0x49,0xcf,0x7c,0x7a,0x0c,0x14,0xfc,0xe9, +0x7a,0xb5,0x5a,0x04,0xc4,0x49,0x8d,0x55,0x1e,0x8e,0x13,0x48, +0xac,0x51,0xd8,0x70,0xc1,0x9e,0xf8,0x17,0xfc,0x7a,0x9b,0x55, +0xc0,0xd0,0x0b,0x39,0x5c,0x9c,0x23,0x0c,0x1b,0x98,0x01,0xe4, +0x0c,0x7b,0x1c,0x9d,0x78,0x31,0x5a,0x8e,0x6e,0xb2,0xf8,0x82, +0xdd,0xaf,0x27,0xde,0x47,0xf1,0x13,0x93,0x0c,0x21,0xa5,0x3d, +0xe1,0x3b,0xcd,0x3f,0x33,0xf3,0x5f,0x50,0x1e,0xbf,0x64,0xd1, +0xbe,0x21,0x31,0x76,0x8f,0x92,0x87,0x7e,0x70,0xd1,0x4e,0xb3, +0x6f,0xc8,0x41,0xd5,0xc6,0x87,0x4f,0x0e,0xf1,0xf3,0xdb,0x4e, +0x2d,0xaa,0x81,0x44,0x99,0x50,0x0a,0xe5,0x5f,0x8e,0xae,0x12, +0xcf,0xae,0xc5,0xef,0x97,0x87,0x1d,0xb1,0xcd,0xd3,0x0d,0xe0, +0x30,0xd3,0x1f,0x81,0x8d,0x5d,0xf8,0x0f,0x1d,0xf9,0xbb,0xf6, +0x7a,0x62,0x74,0x58,0x39,0x5c,0xc6,0x8c,0x70,0x27,0x64,0x84, +0x18,0xb6,0xbe,0xa6,0x90,0x1f,0x90,0x05,0xba,0xb4,0x81,0xde, +0x00,0x6d,0x85,0xa1,0x23,0x92,0x11,0x7a,0xdc,0xcc,0x17,0x74, +0x5c,0x83,0x31,0x3c,0x97,0xa3,0x71,0xd2,0x18,0xf6,0x9c,0xd6, +0x58,0x02,0xb2,0x48,0x38,0x68,0xac,0x26,0x08,0xa3,0xa5,0xcf, +0x56,0xa3,0x74,0x96,0x79,0x56,0x05,0x81,0x06,0x74,0x59,0x2c, +0xb3,0xff,0xe8,0x32,0x9b,0x4f,0x41,0x51,0xbf,0x18,0xad,0x60, +0xf5,0x36,0x8b,0x4f,0xff,0x8e,0xcd,0xca,0xfa,0xdd,0xbf,0x9d, +0xfe,0xed,0xf4,0x34,0x35,0xb5,0x7f,0x29,0x80,0x7f,0x5e,0x4e, +0xcb,0x9b,0x44,0x2a,0xa6,0x97,0x6c,0x59,0x15,0x90,0xf2,0xa5, +0xb4,0x4c,0xc5,0xfc,0x17,0x94,0x97,0x40,0x06,0x22,0xc3,0x8e, +0xfe,0xf3,0x0c,0x04,0x04,0xf4,0xdc,0xc4,0x73,0x33,0xfd,0x7e, +0xa3,0xd1,0x75,0x93,0x02,0x1e,0x19,0xb8,0x0c,0xfc,0x4a,0x30, +0x68,0x72,0x60,0x49,0x2e,0x69,0x9a,0xa4,0x42,0x65,0xd0,0x28, +0x75,0x79,0xce,0x02,0xee,0x9b,0xdf,0xed,0x6c,0x7d,0xc9,0x94, +0x7a,0x61,0x60,0x25,0xf3,0x3d,0x81,0x56,0xe4,0x77,0x4d,0x62, +0xc0,0xa1,0x90,0xa5,0x0a,0x90,0x83,0xe5,0xe8,0xdb,0x34,0x99, +0x42,0xa7,0x96,0x93,0xee,0xee,0x06,0x43,0x8b,0xf4,0x6f,0x2c, +0x60,0x9b,0x7c,0x2b,0xbd,0xd8,0x04,0x1b,0xbf,0x0b,0xe9,0xd2, +0x21,0x43,0xed,0x3b,0xe8,0x46,0x97,0x9a,0x52,0xc6,0xdd,0x1d, +0x8c,0x56,0x43,0xd3,0xd7,0xa5,0x82,0x9a,0x32,0x12,0xee,0x12, +0x45,0xe5,0x9a,0x6e,0x25,0xb8,0xae,0x4d,0xcf,0x87,0x74,0x75, +0x8d,0x20,0x82,0x3e,0x73,0x74,0x9c,0x25,0xbc,0x82,0x2d,0xc3, +0xe4,0xda,0xb6,0x18,0xf1,0x49,0x66,0xe3,0xf9,0x24,0x79,0x3e, +0xc9,0xaa,0x05,0x48,0x67,0xfb,0xfd,0xa3,0xb0,0x5b,0x4c,0xb4, +0x7b,0x42,0xe3,0xd1,0x74,0x31,0x98,0xff,0xb1,0x54,0x13,0xff, +0x2a,0x72,0xda,0x51,0x6a,0xf1,0x8e,0x3c,0xe0,0x37,0x8c,0xf5, +0x2e,0x30,0x7d,0x31,0xcf,0xf8,0xef,0x9a,0xfe,0x70,0xbc,0x1d, +0xfc,0x35,0x9e,0xdf,0xc0,0x1a,0x14,0x7e,0x55,0x74,0x8c,0x5b, +0x8f,0xa6,0x96,0x54,0x60,0x14,0xd0,0x1f,0x09,0xb1,0x7e,0xa3, +0x74,0xcb,0x30,0x10,0xfd,0xcc,0xb9,0x77,0x77,0x0c,0x6c,0x26, +0x01,0x9b,0x73,0x82,0xa1,0x2f,0xa5,0x39,0xb2,0x74,0x83,0x29, +0x6a,0xf8,0x5d,0x04,0xe6,0x2b,0x66,0x8c,0x44,0x03,0x32,0xf2, +0xa8,0xeb,0x94,0x13,0x98,0x40,0xe6,0x19,0xc9,0x33,0x93,0x8b, +0xe8,0xe3,0x8f,0xbb,0xe6,0x07,0x86,0x19,0x32,0xfe,0x6c,0xeb, +0x8b,0x5c,0x09,0x63,0x96,0x87,0xb2,0xbd,0x28,0x4c,0xa0,0x68, +0xf1,0x0c,0xaa,0x32,0xb5,0x66,0x48,0x48,0x11,0x4d,0xfd,0x35, +0x27,0xd4,0x0d,0x08,0x01,0x17,0x28,0x85,0xef,0x90,0xfa,0xdd, +0x26,0x1e,0x54,0x42,0x52,0x99,0x59,0x3f,0xdd,0x26,0xcb,0x65, +0x6a,0x43,0x17,0xd3,0x0b,0x8a,0xe8,0x87,0x62,0xb1,0xd2,0x98, +0x47,0x81,0x9a,0x9b,0x7c,0x3d,0xe1,0x0d,0xe4,0x79,0x18,0x63, +0x12,0xeb,0xb9,0x42,0xa1,0x9b,0x31,0x76,0xaf,0xc1,0x73,0x76, +0x23,0x30,0x78,0x60,0x22,0x37,0x1f,0x22,0x6a,0x3c,0x0c,0xc0, +0xa8,0xaf,0x6b,0x92,0x29,0xe1,0xaa,0x98,0x52,0x8d,0x71,0x41, +0xce,0x02,0xbb,0x31,0x59,0x82,0xde,0xa9,0xa2,0xab,0xee,0xee, +0x6a,0x2a,0x93,0x46,0x1d,0xb9,0x63,0xe4,0xa2,0x7d,0x85,0x37, +0x51,0x18,0x85,0x91,0x66,0xdc,0xd5,0x55,0xb2,0x00,0xbf,0xdd, +0x99,0x36,0x8e,0xe3,0x2a,0x1b,0x87,0x9e,0x5a,0x90,0xca,0xc1, +0x62,0x78,0x86,0x11,0xe4,0xe7,0xcb,0xb8,0x94,0x72,0x77,0xc7, +0xd1,0xad,0xad,0x4e,0xfb,0xb3,0x01,0x37,0x93,0x07,0x3d,0xfc, +0x93,0x5e,0x79,0x40,0xf1,0xf5,0x48,0x99,0x74,0x16,0x9e,0x6f, +0x47,0xf8,0x4e,0x24,0x3f,0xf1,0xe3,0xfb,0xab,0xeb,0xe5,0xfc, +0x03,0xed,0x61,0x7f,0xb3,0x5c,0xce,0xc1,0x66,0xfd,0xf3,0xcf, +0xdf,0x3f,0x80,0xf9,0xfa,0xfa,0x01,0x5f,0xa7,0x4d,0x26,0x0f, +0xd2,0x6c,0xf6,0xff,0x5f,0xa1,0x9d,0x95,0xc2,0xea,0xb3,0xd0, +0xc7,0x16,0xc1,0xfc,0x2e,0x8f,0xdd,0x1d,0x96,0x81,0xec,0xce, +0x40,0xa5,0x0c,0x18,0x6e,0xe9,0xa4,0xdb,0x80,0x0a,0x82,0x25, +0x46,0xed,0xeb,0x36,0xe8,0x4f,0x23,0xe0,0x67,0x5e,0x94,0x85, +0xdb,0x6d,0xb8,0xdf,0x8d,0xc0,0xc2,0xf4,0x6c,0x3e,0x55,0x6f, +0xd1,0x36,0x2a,0x93,0x1b,0xb0,0x52,0x98,0xcd,0xe6,0x33,0x58, +0x95,0x4c,0x9f,0x43,0x6d,0x17,0x17,0xf6,0x77,0x23,0x48,0x56, +0xa3,0xb7,0x4e,0xd1,0x6f,0x20,0x01,0x2f,0xfa,0x4c,0xaf,0xbe, +0x4f,0x67,0xef,0xbb,0x8d,0xeb,0x65,0x72,0xd5,0xc0,0x5b,0x12, +0x5d,0xb6,0xd5,0xe0,0x9f,0xef,0xd3,0x8c,0xbf,0xf0,0x47,0x83, +0xf4,0x78,0x03,0xfe,0x69,0xb0,0x66,0x6f,0xe0,0xbf,0x0d,0xa5, +0xd6,0x95,0x5c,0x07,0xa8,0x0f,0xba,0x62,0xe8,0x91,0x75,0xd7, +0x15,0x23,0x2f,0x10,0xab,0xaf,0x6b,0xcc,0x3f,0xba,0x42,0x8e, +0xed,0x5e,0xa1,0x23,0x35,0x54,0x64,0x19,0xfe,0xd0,0xfd,0x54, +0xb7,0x9b,0x44,0x54,0x7c,0x33,0x4d,0xd0,0x9d,0x9d,0x88,0x91, +0xdf,0x8d,0x60,0x34,0x99,0x58,0xa0,0x3c,0x42,0xba,0x8d,0xaa, +0x54,0xaa,0xe9,0x05,0xf1,0x5a,0xb7,0xd0,0x7c,0x02,0x23,0xa7, +0x73,0xbc,0x10,0x45,0x7f,0x1a,0x41,0x3a,0xc9,0xb0,0xef,0xb2, +0x86,0xad,0x32,0x7f,0x2e,0x75,0xbe,0xb5,0x14,0xcb,0xaa,0x4d, +0x9b,0xb2,0xc0,0x68,0x2d,0x53,0x07,0x11,0x18,0x64,0xae,0xf0, +0xa1,0x3b,0x31,0xa6,0xbf,0x9a,0x63,0xfb,0xed,0xc7,0x41,0x21, +0x31,0x00,0xd9,0xbe,0x09,0xac,0x37,0x5e,0x16,0xfc,0x54,0x53, +0x8a,0xb1,0x3f,0x11,0xa0,0x9d,0xc1,0xea,0x15,0x77,0x0c,0x41, +0x0b,0xa5,0x13,0x0a,0xe9,0x19,0x63,0x99,0xd2,0xb2,0x09,0x57, +0x6b,0xe9,0x68,0xea,0x19,0x04,0xd6,0xca,0x09,0x13,0x61,0x2c, +0x72,0xf9,0x01,0x7e,0x0d,0xf1,0xc6,0x96,0x42,0xe8,0x64,0x6c, +0x75,0x45,0x83,0x8b,0xf6,0x74,0x04,0x86,0xa9,0x85,0x72,0xc8, +0x5a,0xca,0x6d,0xe1,0x5b,0x69,0xe1,0xb7,0xcb,0xf9,0x4d,0x5d, +0x1b,0x0f,0x6f,0x9d,0x5a,0xdd,0xd2,0x03,0x0c,0x54,0xb4,0xd0, +0x56,0x83,0xa4,0xa6,0x81,0xf8,0xf2,0x59,0x55,0xcb,0x7c,0xd3, +0x34,0xbb,0x77,0x9e,0x97,0xba,0x86,0x3a,0x25,0x9d,0x14,0xa5, +0xc2,0xed,0xc9,0x5a,0x49,0xc0,0xd7,0x42,0x14,0x86,0x12,0xa3, +0x9e,0x57,0x70,0x89,0xf9,0xe3,0xd6,0x55,0xe4,0xe9,0xde,0xda, +0x7c,0x6b,0xf2,0xf9,0x05,0x55,0xe4,0xf3,0x89,0x5b,0xc3,0x73, +0x13,0x49,0xae,0x71,0x14,0xc7,0x9c,0x74,0x7c,0x5c,0x98,0x4f, +0x04,0x52,0xd2,0xe9,0xf9,0x3b,0x49,0xb2,0x59,0x06,0xa3,0xbb, +0x92,0x67,0xeb,0xf2,0x42,0xe7,0x50,0xa6,0x29,0xed,0x16,0x68, +0x44,0x25,0xde,0x41,0xad,0x7f,0x34,0xf3,0x9c,0x5a,0xad,0xe9, +0x2f,0x4b,0x9e,0x59,0x2a,0xb9,0xda,0xa2,0x2f,0x00,0x81,0x5d, +0x1f,0x75,0xab,0xb3,0xec,0x99,0xb2,0x80,0xd9,0xb5,0x6e,0x8a, +0x0d,0x29,0xd2,0x51,0x3d,0xf8,0x6c,0x90,0x3a,0x06,0x51,0x6c, +0x21,0x1b,0x15,0x26,0x0f,0x6a,0xf9,0x62,0x83,0x0e,0x03,0x98, +0xa8,0xd7,0x54,0xa8,0x28,0x61,0xde,0xd8,0x61,0x80,0x33,0xad, +0x55,0xc9,0xbc,0x3b,0x9e,0x15,0x5a,0x7b,0x9e,0xa6,0x2d,0x30, +0x5a,0x4a,0x8e,0x2d,0xe3,0xa3,0x94,0x71,0x77,0x57,0xbd,0x07, +0x42,0x1b,0x1c,0xae,0xf2,0x2f,0x60,0x34,0x7c,0x51,0x49,0x15, +0xca,0xbf,0x50,0x46,0x83,0xba,0xd6,0xb0,0x85,0xff,0x39,0x1e, +0x35,0x8f,0x93,0x05,0x42,0xef,0xa8,0x5d,0x35,0xf4,0x0a,0xc6, +0x95,0x2c,0x09,0xec,0x92,0x3b,0x73,0xad,0x26,0x27,0x3c,0x91, +0x9a,0x9d,0x14,0x8a,0xab,0x4e,0x3b,0x6d,0xca,0xc4,0x97,0xdd, +0x29,0x69,0xda,0x47,0x29,0xd1,0x55,0x25,0x05,0xac,0xeb,0x82, +0x77,0x65,0x61,0xe3,0xb6,0xb0,0x44,0xa6,0xa6,0x23,0x35,0x89, +0x45,0x26,0xd6,0xb4,0xd0,0x55,0x17,0x87,0xd3,0xbf,0x93,0xde, +0x40,0xb5,0xce,0xa2,0xa7,0xae,0x02,0x3c,0xe9,0x29,0x76,0xe2, +0xb7,0x7b,0x68,0xbd,0x7f,0x33,0xad,0x12,0x85,0x3d,0x19,0xb4, +0x6d,0x2b,0xba,0xbc,0x98,0x6e,0x75,0xb6,0xef,0x2c,0x00,0xbf, +0x29,0x22,0x38,0x84,0xca,0x52,0xad,0xb5,0xf4,0xcd,0x67,0x5f, +0x25,0x57,0xf3,0x65,0x82,0x83,0xd5,0x32,0x9c,0xd2,0xdf,0x12, +0x3d,0xf8,0x77,0x80,0x14,0x44,0x54,0x93,0x81,0x1f,0x76,0x23, +0x7e,0xda,0x51,0x8b,0x99,0xd2,0xe7,0xd6,0x26,0xf9,0x01,0xe4, +0x21,0x7c,0xb1,0x31,0x3b,0x9b,0xf1,0xbb,0x1a,0xf0,0x09,0xa4, +0xef,0x21,0xda,0x60,0x47,0xd3,0x16,0xcf,0x0c,0x9e,0xf3,0xf6, +0x46,0xec,0x88,0x82,0x9a,0x56,0xaf,0xe7,0xeb,0xe9,0xe4,0xe5, +0xe8,0x36,0x61,0x73,0x38,0xae,0x49,0x77,0x04,0x49,0x56,0x90, +0xa1,0x5d,0x19,0x83,0x4d,0x93,0xac,0x7e,0x77,0xcd,0x6c,0xb4, +0x48,0x46,0xbf,0x8e,0x08,0x63,0x59,0xe3,0xaa,0x49,0x57,0x68, +0x6d,0x04,0xcb,0x86,0x1c,0xe7,0x6f,0x69,0x0f,0xe7,0xab,0xf9, +0x7c,0x9a,0x8c,0x74,0xad,0x18,0xa8,0x6b,0x1f,0x7a,0x8d,0x99, +0xcb,0x6c,0x8b,0x26,0xf7,0xfa,0xea,0x2a,0xdd,0x54,0xcf,0xda, +0x9c,0x27,0x9e,0x81,0x4e,0x5a,0x59,0xe9,0xbd,0x64,0x3c,0xf6, +0x7e,0x28,0x27,0x95,0x8c,0x1c,0x06,0xd4,0xf9,0x2e,0x39,0x3a, +0xac,0xaf,0xb5,0x0b,0x62,0xa7,0xd9,0x7b,0x3c,0xb0,0x08,0x92, +0x65,0xd2,0x2b,0x03,0x51,0x78,0xe4,0x90,0x8f,0x10,0xbf,0x5c, +0xbe,0xc5,0x6a,0x96,0xf8,0x43,0x1e,0x32,0xcc,0xcc,0x1a,0x93, +0x03,0x28,0x59,0xb5,0xf0,0x2b,0x0c,0x1d,0x58,0xfb,0x97,0x62, +0xcd,0xd8,0x50,0x1a,0x77,0xd7,0xb3,0x93,0x55,0x45,0x81,0x53, +0x87,0x06,0xe6,0x68,0xd1,0x48,0xdc,0xab,0x3d,0x6d,0x1d,0x20, +0xd0,0xb0,0x57,0x04,0xc5,0xf7,0x00,0x0f,0x2e,0x8d,0xaf,0xe6, +0x05,0x45,0x70,0xde,0xb2,0xd1,0x8c,0x1a,0xcf,0x31,0xc2,0xa5, +0x3d,0x86,0xb5,0x3d,0x6b,0x37,0xc1,0xc7,0xed,0x16,0x7c,0x6d, +0xcd,0x4e,0xb4,0x19,0xc2,0x7a,0x80,0xe6,0x4d,0x0c,0xa0,0x24, +0xbd,0xc4,0x6b,0x41,0x8b,0x13,0xae,0xe0,0x07,0x57,0xe6,0x10, +0x64,0x57,0xbf,0x0a,0xf4,0x51,0x88,0x05,0x0a,0xc8,0x7f,0x98, +0x4f,0x92,0xe9,0xef,0xc4,0x1b,0x31,0xde,0x22,0x23,0xab,0x16, +0x3b,0x56,0xf3,0x79,0x70,0x06,0x96,0xaa,0x12,0x5f,0xc4,0x43, +0x3a,0x88,0xca,0x0e,0x03,0x09,0x95,0x48,0x35,0xd9,0xcb,0xd5, +0x22,0x8e,0xe3,0x63,0x59,0xd4,0x15,0x33,0xac,0xd8,0x5f,0x26, +0xd5,0xff,0x68,0xe3,0xb5,0x7b,0xc9,0xa1,0xdf,0x02,0xc2,0x25, +0xdf,0x2e,0x05,0xec,0x59,0xb0,0xfb,0x98,0xb0,0x75,0xad,0x37, +0x36,0xed,0xaa,0x95,0x8c,0x0d,0x61,0xaf,0x0b,0xec,0xf4,0x92, +0x01,0x22,0x08,0xcd,0xe2,0x19,0x63,0xd9,0x54,0xda,0x1b,0x1a, +0x72,0x5a,0x38,0xf2,0x29,0x6d,0xae,0xe1,0x01,0x2b,0xf6,0x1d, +0x1e,0x10,0x95,0x76,0xe8,0x60,0x3a,0xe9,0x59,0x19,0x6d,0x27, +0x34,0x01,0x1f,0x04,0xc7,0xd6,0x80,0xe2,0x83,0x5f,0x3b,0xc2, +0x06,0xa7,0x89,0x1c,0x08,0x25,0xd5,0x08,0x79,0xbb,0xac,0x38, +0x9f,0x8d,0xd7,0x4b,0x4c,0x36,0xef,0x71,0x0a,0x18,0x0c,0xf0, +0x9e,0xe4,0xf5,0x7c,0x49,0xe3,0xf1,0xad,0x4a,0x04,0xf2,0x23, +0x96,0xbf,0x03,0x8b,0x9e,0x8a,0xe5,0x8c,0xbb,0x5b,0xa7,0xa3, +0x10,0x2a,0xd4,0x18,0xe6,0x7a,0x49,0x5b,0xa0,0x35,0xe4,0xeb, +0xf3,0xea,0x22,0xe9,0x72,0xc6,0x8d,0x27,0xff,0x7a,0xfb,0x1b, +0x7e,0xcb,0x3e,0xff,0x87,0xeb,0xd1,0x8a,0x76,0xf3,0x02,0xeb, +0x70,0x54,0xda,0xc9,0x87,0xd4,0xfa,0x8c,0xc9,0x94,0xa2,0xd9, +0xc0,0x6e,0x4e,0xd5,0x11,0x42,0x9b,0x4f,0x41,0x7c,0x65,0x2f, +0x17,0x51,0xd9,0xa7,0x07,0x15,0xb8,0x24,0xdb,0xef,0x21,0x71, +0x66,0xc2,0x95,0x03,0x6a,0x43,0x6c,0x5f,0xd0,0x0f,0x1a,0xcf, +0xaf,0x5a,0x3f,0xce,0x67,0x49,0xeb,0x07,0xda,0xff,0x1b,0xc6, +0xb4,0x47,0x69,0xe7,0xda,0x19,0x3c,0x1d,0x80,0x30,0x72,0x4c, +0x45,0x3c,0xeb,0xd4,0x3d,0xa7,0x78,0x0f,0xb9,0xcd,0x18,0x19, +0xd4,0x6f,0x9c,0x36,0x9a,0xf8,0x03,0x2f,0x33,0x52,0xaa,0x4d, +0x30,0xcf,0xac,0x77,0x77,0x90,0x57,0xe5,0x69,0x60,0x81,0x5a, +0xee,0x06,0x1f,0xd5,0x86,0xa8,0x8d,0xa9,0x78,0xec,0xe7,0x7d, +0xbc,0x91,0x8d,0xc7,0x7f,0xfd,0xe6,0x55,0xa3,0xb0,0xa2,0x28, +0xac,0x37,0x40,0x8b,0xe0,0x6e,0xeb,0x1f,0x88,0x0e,0x37,0x68, +0x0f,0x43,0xf7,0xe2,0xcf,0x87,0xa0,0x9b,0x1f,0xdc,0xda,0x17, +0x3f,0xbd,0x3c,0x00,0xa1,0x6c,0x15,0x1f,0x86,0xf2,0xeb,0x6f, +0xbe,0xff,0xe6,0xd5,0x37,0xfb,0x91,0xd2,0x7e,0xf3,0x61,0x28, +0xbf,0xfb,0xe6,0xcb,0xaf,0xf7,0x23,0xe4,0x4d,0xeb,0xc3,0x30, +0xbe,0xfa,0xf9,0xcb,0x67,0x07,0xd0,0xa8,0x76,0xbe,0x0f,0x43, +0xfa,0xd3,0x8b,0x57,0xcf,0x7f,0xfa,0xf1,0xe5,0x01,0x1d,0x44, +0xdb,0xe7,0x07,0xf6,0xd0,0x97,0xaf,0x9e,0x7d,0xb7,0x17,0xe5, +0x56,0x1c,0x26,0xf0,0x8c,0xc4,0xd5,0xe9,0x98,0x62,0x05,0xc6, +0xc2,0x33,0x15,0x4b,0xad,0x05,0x6e,0x36,0x8d,0xce,0x0a,0x95, +0x8c,0xa8,0x2f,0x2e,0x90,0x60,0x1a,0x8f,0x66,0x0f,0x76,0x99, +0x4c,0xd6,0x6a,0xe4,0xd9,0x6a,0xdd,0xe8,0x66,0x8d,0x6c,0x34, +0x5e,0xdf,0x58,0xbb,0xbf,0xf8,0x0b,0xbd,0x14,0xf0,0xef,0x4b, +0xd9,0x78,0x8b,0xb9,0x8e,0x76,0xe5,0x2e,0x9f,0x6c,0x17,0xa5, +0x57,0x9e,0x5d,0x84,0x0e,0x95,0xdc,0x62,0x8e,0x37,0x88,0x0b, +0xec,0x5b,0x4b,0x38,0x95,0xd8,0x13,0x4a,0x62,0x3b,0x71,0xab, +0x02,0x1c,0xdb,0x99,0x03,0xb7,0x9e,0xf2,0xcc,0x22,0xf6,0xce, +0x61,0xb0,0x55,0x87,0x41,0x43,0xf5,0x54,0x79,0x76,0x58,0x85, +0x00,0x38,0xec,0xc1,0x3f,0xa0,0xb5,0x85,0xd0,0x66,0x8c,0x1a, +0x14,0x92,0xda,0xef,0xe6,0xe9,0xcc,0x6b,0x04,0x0d,0x9f,0x9f, +0x9f,0xd3,0x4c,0x7f,0x3e,0xe9,0xf1,0x9f,0x02,0xb7,0x0b,0xdb, +0x8b,0xfd,0x52,0x5f,0x54,0xed,0x2c,0x8a,0x45,0x5c,0x82,0x2d, +0x42,0x94,0xb6,0xa3,0xcd,0xde,0xb2,0x4b,0x79,0xa1,0x2f,0xb5, +0x77,0x43,0x9f,0x7f,0xfd,0xf9,0xe7,0xe7,0xcf,0xe6,0x37,0x60, +0xf1,0x80,0x6c,0x29,0x14,0x5d,0xf9,0xab,0x9f,0x98,0x46,0x49, +0x6b,0x22,0x32,0xc1,0xec,0xce,0x7a,0xca,0x79,0x67,0x5b,0x94, +0xfd,0xab,0x04,0x46,0xa5,0xe3,0xb1,0xa3,0xe6,0x75,0x9c,0x90, +0xb8,0x67,0x94,0xa7,0x4e,0xfd,0x54,0x86,0xb0,0xd0,0x28,0x01, +0xe4,0x46,0x51,0xf9,0x40,0x92,0xea,0xed,0xb3,0x36,0x3d,0x37, +0x8c,0x54,0xf1,0x60,0x7e,0x3b,0x9d,0x5f,0x8e,0xa6,0x6c,0x7b, +0xa9,0x77,0xf0,0xcb,0x5e,0x79,0x1c,0x00,0xb9,0x02,0x96,0xd7, +0xc1,0xed,0x3f,0x61,0x8c,0xf8,0x41,0x83,0x26,0x44,0x10,0x87, +0x3f,0xfd,0x6a,0x39,0xfd,0xf1,0x2c,0xf9,0xa7,0x5f,0x2d,0x17, +0x3e,0x7e,0x2b,0xfc,0x65,0xb2,0xbc,0x4d,0xc7,0xe8,0x6d,0xea, +0xa0,0xd4,0x86,0x9f,0x06,0x5f,0x3a,0x26,0x34,0x6a,0x16,0x8f, +0xf5,0x00,0x6f,0xdd,0xcb,0xd2,0x5f,0x06,0x52,0xfd,0x16,0x33, +0x8f,0x9c,0x98,0xed,0xec,0xdd,0xa0,0xa5,0xa3,0xc5,0x21,0xd8, +0x3b,0x14,0x81,0x11,0x18,0xfa,0xdd,0x68,0x36,0x99,0x02,0x63, +0x54,0x5f,0x06,0x26,0x8d,0x75,0xcf,0x1e,0xe4,0x55,0xc7,0x8e, +0x1a,0x7f,0x55,0xe6,0xb7,0xc2,0x88,0x83,0xb0,0xd3,0xc9,0xa4, +0x46,0x87,0x67,0x5a,0x85,0x15,0x88,0xac,0xde,0x98,0x77,0x06, +0x63,0x1b,0x2f,0x7d,0x49,0x29,0xbc,0x6c,0x5a,0xcc,0x06,0x1b, +0x4f,0xd3,0x28,0x2b,0x5d,0x3b,0x3b,0x43,0x8b,0x76,0xa4,0x41, +0xe4,0xb3,0xaa,0x12,0xf6,0x78,0x53,0xf5,0x28,0xf5,0x5c,0xa8, +0xca,0x02,0xe2,0x2f,0x1b,0x88,0x3b,0xff,0xf8,0xb8,0x66,0xd7, +0x46,0xf6,0x82,0xb4,0xe1,0x0b,0x49,0xe6,0x24,0xc1,0xd5,0x1b, +0x9c,0xab,0x30,0x62,0xa5,0xd5,0xd3,0x82,0x82,0x2b,0xb1,0x1b, +0x84,0x57,0xb6,0x8a,0x2e,0xda,0xeb,0x19,0x4a,0xaf,0x04,0x01, +0xcc,0xe0,0xc7,0x22,0x1d,0xbf,0xaf,0x3f,0xbe,0x1d,0xa8,0x93, +0xfe,0xe2,0xd1,0x3e,0x86,0x8c,0x55,0xaa,0xc9,0x72,0x86,0x2b, +0x78,0x04,0xc4,0x82,0x5f,0x46,0x42,0x3d,0x65,0x7e,0xaf,0xa4, +0x17,0x15,0x53,0x7c,0xc3,0x43,0x73,0x2a,0xe9,0xb9,0xf5,0x04, +0x1a,0xb8,0xac,0x5f,0x35,0xdb,0x1c,0x44,0xfa,0xac,0xae,0x1a, +0x13,0x82,0xef,0x91,0xe1,0xc2,0xea,0x29,0x76,0xbf,0x79,0xde, +0xbc,0x1f,0x06,0xfb,0x45,0x50,0xda,0x5a,0xd1,0xca,0x05,0x64, +0x50,0xf3,0x90,0x34,0x43,0x3a,0x31,0xc6,0x83,0xe5,0x2e,0x5c, +0xc1,0x28,0x7d,0xe6,0xea,0x2a,0x28,0xab,0xe9,0xb6,0x82,0x32, +0x55,0xe2,0x7b,0xaa,0x76,0x95,0x96,0x67,0xb6,0x8d,0xc7,0x4c, +0xd9,0x0a,0x7a,0x30,0x0c,0xd4,0x06,0x8e,0x85,0x4e,0xc6,0x5a, +0x01,0x25,0x9d,0xbf,0x99,0xfd,0xa2,0xc1,0xd0,0x34,0x44,0x25, +0xd6,0xb3,0x0f,0xe7,0xfc,0x98,0x8c,0x2c,0x7b,0x8d,0x8e,0x6e, +0xed,0xf2,0x64,0xac,0x79,0x56,0xb4,0x53,0x6c,0x7e,0x99,0xec, +0x71,0x71,0x93,0xc3,0x21,0x9f,0x95,0x80,0xdb,0x07,0x74,0xac, +0xbb,0xb3,0x13,0x8c,0x90,0x99,0xe3,0xdb,0xfb,0x77,0xc3,0x7f, +0x48,0xc5,0x65,0x06,0x98,0xea,0xdd,0xa6,0xbb,0xe5,0x5e,0x80, +0xb6,0x49,0x71,0x6e,0xe3,0xbf,0xce,0xee,0x90,0x16,0x11,0xc9, +0xa4,0x5e,0x50,0x1a,0x52,0xd2,0x9e,0xa1,0x5e,0x96,0xdf,0xbe, +0xfa,0x81,0x9a,0xac,0x00,0xf7,0xaf,0xc9,0xaa,0x04,0xb6,0x63, +0xd6,0xac,0xb4,0x21,0x9d,0xed,0x4b,0x87,0x52,0x7c,0x95,0x48, +0xaa,0xc6,0x0d,0x94,0x1d,0x34,0x82,0xe2,0xc3,0x88,0xf2,0xbe, +0x26,0xc2,0x30,0xc6,0x82,0xd6,0x7e,0x92,0x74,0x53,0x2b,0xb9, +0x4a,0xc0,0x14,0x9a,0xc4,0x7f,0xfa,0xb5,0x4d,0xbf,0x3d,0x9f, +0xb6,0x3f,0xbe,0x5c,0xbe,0xcd,0xcc,0x36,0x73,0xcf,0xde,0x32, +0x5a,0x5d,0x27,0x33,0xcf,0x3e,0x68,0x70,0x36,0x44,0x4a,0x32, +0x6e,0x44,0x5c,0xe1,0xc5,0x88,0x85,0x58,0x1e,0xcf,0xc2,0x95, +0x43,0x66,0x0f,0x13,0x24,0x8e,0x26,0x24,0xab,0xc3,0xc3,0x40, +0xd1,0x47,0xfb,0x37,0xd3,0x5b,0x72,0xbd,0xf7,0xb7,0x45,0x01, +0x51,0x1d,0xad,0xa1,0x15,0x43,0x90,0xe0,0xfb,0xf6,0x84,0x5f, +0xe2,0x1a,0xf4,0xaf,0x67,0xac,0xc9,0x0a,0x96,0xed,0xe3,0x4f, +0x55,0x2b,0x06,0x88,0x71,0xf8,0x9f,0xd7,0x16,0xa9,0xba,0x88, +0x3e,0x50,0x67,0xe5,0x81,0x73,0x0a,0x5f,0xb1,0x59,0xd9,0x2f, +0xb5,0x42,0x7b,0x54,0x69,0x1c,0x1f,0xed,0x3b,0x0a,0x5d,0xaf, +0x54,0x82,0xea,0x08,0xf8,0x59,0x2a,0x5b,0xe1,0xae,0x96,0xe9, +0xc2,0xb2,0xaa,0xbc,0xc2,0xb6,0x3f,0x90,0xcc,0x0b,0x87,0xc0, +0x32,0x06,0x28,0xb1,0xd6,0x18,0x40,0x07,0x40,0xdf,0x7e,0x12, +0x0d,0xec,0xc0,0x67,0xeb,0x6c,0x35,0xbf,0xf9,0x49,0xed,0x8d, +0x49,0x2d,0x64,0x21,0x8d,0xdd,0x2c,0x6d,0xed,0x51,0x72,0xc1, +0x5c,0x94,0xfd,0xee,0x01,0x7b,0xe7,0x89,0x63,0x9e,0x5c,0xb9, +0x10,0xaf,0xd4,0x61,0xe0,0x5c,0xaa,0xf0,0x3f,0xea,0x32,0x93, +0x39,0x3e,0xe4,0x4a,0x68,0x6d,0x28,0x98,0xff,0x47,0x99,0xf2, +0xf1,0x98,0x4e,0x75,0x8d,0xf8,0x61,0xa8,0x52,0xe8,0xe3,0x38, +0x46,0xb4,0x7d,0xe5,0x06,0xd8,0xc5,0x2f,0xf2,0xc2,0x8c,0x09, +0x51,0x13,0xbf,0xdb,0xab,0xf9,0x9f,0x17,0x0b,0xe5,0x1a,0xda, +0xb3,0xd1,0xc6,0x44,0xfa,0x51,0x1c,0x3b,0xe8,0x8f,0x8f,0xb9, +0x2d,0xc5,0xf4,0xbe,0xcb,0x85,0x6e,0xf9,0xbe,0x08,0xeb,0x63, +0x3a,0xa0,0x77,0xf7,0x3a,0xac,0x3e,0xac,0xe2,0x27,0xbd,0xaf, +0xe2,0x15,0x8e,0xf8,0x5d,0x0c,0xca,0xad,0x87,0xac,0x13,0x6c, +0xa1,0xb1,0xc4,0x6d,0x3e,0x11,0x01,0x0e,0xd5,0xb8,0xbb,0x22, +0xe6,0x2f,0xd7,0xfa,0xaf,0xdf,0xbc,0xfa,0xfe,0xf9,0xcb,0x57, +0xaa,0x38,0x2d,0x36,0x2a,0x16,0x02,0xed,0xc9,0x5c,0x41,0x96, +0x0b,0x1b,0x71,0xaa,0x5f,0x07,0x78,0x57,0x62,0xe4,0xea,0x4d, +0x18,0x80,0x4d,0x75,0xb6,0xeb,0x1d,0x67,0xc3,0xf6,0x2a,0x8d, +0x96,0x6f,0xd8,0xfb,0xc9,0xc2,0x70,0xb0,0x91,0x16,0x38,0xc5, +0x82,0x03,0x91,0xf0,0xaa,0xae,0x76,0x52,0x2d,0x5a,0x01,0x66, +0xaa,0x1f,0x5b,0x93,0xab,0x9e,0xf1,0xf7,0x1f,0xfb,0x6b,0x5f, +0x98,0xa3,0x48,0x2c,0x8a,0x60,0x8a,0x77,0x74,0xe9,0x84,0xe8, +0xa0,0x95,0x6a,0xd1,0x91,0xd8,0xdd,0x13,0x81,0xf9,0x53,0xe3, +0x3b,0xd0,0x5d,0xaa,0x7a,0x6d,0xa3,0xb1,0xd0,0x72,0xc2,0x9a, +0x9e,0x0f,0x40,0xef,0x38,0xbd,0x0e,0x2b,0x6e,0x6a,0x1a,0x84, +0x5b,0xab,0xfd,0xf7,0xb6,0x22,0x80,0x87,0x87,0x51,0xa3,0x87, +0x11,0x45,0xe2,0x92,0x51,0x60,0x5a,0x78,0x20,0x16,0x6e,0x4d, +0xe5,0x88,0xba,0x1f,0x2e,0xd0,0x47,0x1a,0x0f,0xfc,0xfe,0x54, +0x2c,0x73,0x8b,0x1c,0xfc,0xf8,0x44,0x3c,0xac,0x56,0x35,0x26, +0xd6,0xba,0x9f,0x88,0x0b,0xf5,0x98,0xc6,0x84,0x1f,0x9f,0x88, +0x87,0x36,0xca,0x35,0x22,0xfa,0xfa,0x44,0x4c,0xb2,0x3f,0xae, +0x71,0xc9,0xf7,0xa7,0xf2,0x1c,0xb7,0xc5,0x0d,0xd3,0x47,0x35, +0x32,0x50,0x31,0xfd,0x5a,0xb9,0x55,0xe7,0xdb,0x26,0x5f,0x6b, +0x86,0x20,0xe3,0x1d,0xae,0x5a,0xe5,0x54,0xbd,0x52,0x28,0x2c, +0xe2,0xee,0xad,0x97,0xc2,0xdf,0xa9,0x97,0x7e,0xc7,0x70,0x0e, +0xff,0x20,0xb1,0x97,0xc7,0xed,0xfe,0xaf,0x94,0x57,0x73,0xef, +0xc0,0x56,0x11,0x92,0xf4,0x7f,0xd3,0x18,0xb8,0xbf,0x66,0xfe, +0x2a,0x7f,0x3e,0xf9,0x8f,0xd1,0xca,0x22,0x10,0xbf,0x6f,0x58, +0x86,0xb5,0xc3,0x52,0x68,0xf7,0x52,0x5c,0x60,0xfc,0xfa,0xa2, +0xda,0x28,0xe4,0x0d,0x7e,0x65,0x5f,0x01,0xac,0x0e,0xd7,0x46, +0x4f,0x96,0x16,0x8b,0x59,0xeb,0xb2,0x52,0xf7,0x43,0xe1,0x4d, +0xc9,0xf6,0x2c,0x1c,0x80,0xd1,0xd0,0x7c,0x35,0x7f,0xb1,0xe6, +0xe7,0x58,0x07,0x50,0x64,0x18,0xec,0x58,0xcd,0x69,0x70,0x94, +0x40,0xaf,0x80,0xbc,0xb0,0xcc,0x43,0x0e,0x24,0x4b,0xe2,0x1a, +0x55,0x3b,0x4b,0x3e,0xb0,0xa7,0xc4,0x0e,0x3b,0x91,0x49,0xf3, +0x7b,0x0a,0x96,0x08,0x8a,0x0d,0xa6,0xf2,0xe2,0x57,0x41,0xda, +0x01,0xa9,0xd4,0xda,0xcb,0x59,0x65,0xa2,0x5b,0x8a,0xc9,0x39, +0x7c,0x8d,0xe9,0x6c,0xa6,0x50,0xf4,0x3e,0xb5,0xfa,0x43,0x6c, +0x5f,0xc3,0xea,0xcd,0x72,0xc8,0x15,0x87,0x26,0xb5,0x69,0x6f, +0x16,0x92,0x82,0x55,0x07,0xfa,0x1b,0xd5,0x3a,0x81,0x7f,0x0a, +0x56,0x72,0xc7,0x88,0x55,0x7a,0x5b,0x3a,0xc4,0x6b,0x7c,0xf3, +0x6a,0xf4,0xb6,0x61,0x3d,0x03,0xb0,0x1a,0x1d,0x1f,0x8b,0x73, +0x04,0x7e,0xd4,0x8f,0x18,0x04,0x12,0xaf,0x07,0x5a,0x04,0x1b, +0x0e,0x38,0x63,0x87,0xb6,0x00,0xaa,0xa4,0xb9,0x28,0x66,0x55, +0x7b,0x2a,0x73,0xb3,0x86,0xd3,0x37,0xbc,0xd0,0xbb,0xa2,0xe2, +0xf4,0x83,0x8f,0x86,0xf8,0xf4,0x09,0xff,0xfd,0x16,0xd3,0x01, +0x1b,0xfe,0xbe,0xbb,0xe3,0x8a,0xf6,0x59,0xef,0x81,0xdc,0x64, +0xdd,0xed,0x8f,0xee,0xf1,0x43,0xf7,0x9a,0xf5,0x56,0x6d,0xb6, +0x27,0x37,0xb9,0xa0,0xe8,0x76,0xe3,0x97,0xed,0x02,0xa2,0xc8, +0x5f,0xda,0x0e,0x1d,0xe2,0xc9,0xa1,0x02,0x63,0xa8,0xb5,0x9c, +0xfa,0x5e,0x58,0xce,0x35,0xbb,0x9b,0x43,0x7d,0x63,0xf9,0xde, +0x28,0xf5,0xe6,0x15,0x07,0xa0,0x19,0x06,0x12,0x9d,0x02,0xc5, +0xca,0x48,0x09,0xf6,0x2b,0x75,0x6e,0xbc,0x4f,0xaa,0x8b,0x4c, +0xa8,0x10,0x40,0x18,0x2d,0xe3,0x04,0x1f,0x3b,0xa6,0x3a,0x2e, +0x38,0x20,0x11,0x62,0xa9,0x76,0x3d,0xde,0xdb,0x65,0x95,0x73, +0x7f,0xbf,0x62,0xd9,0xb5,0x0f,0x53,0x71,0xfd,0xc7,0x5b,0x26, +0x87,0x49,0x0c,0x6d,0xcd,0x74,0x6b,0x2b,0x65,0x5c,0xd4,0x63, +0x04,0x09,0x32,0x50,0x64,0x83,0x6c,0x06,0x11,0x27,0x9c,0x3c, +0x7f,0x3f,0x0d,0xb5,0x2c,0xd8,0xbd,0x67,0x55,0x6b,0xfd,0x91, +0x64,0x3b,0x44,0x1c,0xc8,0x07,0x9f,0x78,0xf0,0x89,0xb5,0x0a, +0xab,0xdc,0x7a,0x69,0x24,0x57,0x6b,0xec,0x71,0xb5,0x73,0xbf, +0x81,0x38,0x8a,0x61,0x45,0x87,0xf1,0x49,0xfe,0x58,0xcd,0x8e, +0x7d,0xf9,0x6d,0x79,0x17,0x87,0x7a,0x57,0xc6,0x26,0x46,0x4c, +0xb8,0x97,0x86,0x03,0x3a,0x5e,0x58,0x81,0x01,0x44,0x6b,0x20, +0x4f,0x45,0x73,0x1d,0x24,0x85,0xfa,0x38,0x7f,0x8f,0x62,0xa4, +0xad,0x9e,0xcb,0x77,0x78,0x53,0x87,0x31,0xf3,0xac,0x20,0xc9, +0x7b,0xf4,0x49,0x0f,0x3d,0x3e,0x7f,0x22,0xdf,0x4d,0x4f,0x4a, +0x90,0xcf,0xba,0xe0,0x2c,0xed,0x46,0x2a,0x18,0x76,0x8a,0x3b, +0x4c,0xb5,0x4a,0x99,0xfa,0x89,0xcd,0x55,0xae,0x99,0xa5,0x5c, +0x83,0xf9,0xfb,0x67,0x12,0x87,0x28,0xbe,0x97,0x7e,0xa3,0x85, +0xcd,0xef,0x9e,0xb5,0xc9,0xd1,0xa4,0xaf,0xb7,0x02,0x75,0xc9, +0x03,0xe6,0x9e,0x3f,0x64,0x14,0xff,0x47,0x6b,0xba,0xfb,0x8f, +0xf0,0x4a,0x35,0x78,0x08,0x1a,0xb5,0xd1,0x1d,0xd0,0x08,0xdf, +0xd5,0xa9,0xbf,0x5f,0x13,0x14,0xb6,0x8a,0x4d,0x87,0x5b,0xce, +0xa4,0x4e,0x4c,0xa5,0xc2,0xe4,0xec,0x07,0x26,0xe4,0x81,0x41, +0xa3,0x4f,0xa8,0x8b,0xf1,0x17,0xcc,0x94,0x6c,0x82,0x36,0x97, +0x8a,0xab,0x41,0x65,0x8c,0x1f,0xde,0x2c,0xaf,0x24,0xc9,0xf6, +0xc0,0xfd,0xd8,0x78,0xdd,0xfa,0xee,0xd5,0xab,0x17,0x2d,0xae, +0xad,0xa5,0x30,0x37,0xf4,0xcd,0xf7,0xd8,0x92,0xcc,0xbe,0x72, +0x5f,0xec,0xea,0xa4,0xad,0x6f,0x9d,0xae,0x17,0x5d,0x62,0x2b, +0x28,0xed,0xef,0xb1,0x64,0x6c,0xe2,0xaa,0xac,0x18,0x72,0x11, +0x76,0xf7,0x9f,0x07,0xce,0xe7,0x10,0x58,0x2e,0x36,0x8b,0x19, +0xde,0xae,0x5c,0xf8,0xdd,0xff,0x78,0x22,0xfe,0x0b,0x90,0xa0, +0x8a,0xc9,0xf6,0xca,0x3e,0x8a,0x0e,0x9c,0xe0,0xdc,0xb5,0xe7, +0xb7,0x4e,0x24,0x9b,0xda,0xe3,0x08,0x7b,0x1e,0x94,0xc3,0x08, +0x3e,0xc7,0xe1,0x81,0xab,0xe6,0x42,0xf9,0x2a,0xaf,0x43,0xdd, +0x8d,0xc1,0x4f,0xa8,0x4c,0x85,0x71,0x38,0xb0,0x3e,0x6b,0x67, +0xf4,0x13,0x2a,0xa3,0xe3,0xaa,0x43,0x6b,0xb2,0x76,0x91,0x78, +0xe1,0x23,0x47,0x3c,0xf7,0xae,0x94,0x06,0xbc,0x6d,0x59,0x10, +0xa2,0x72,0x8d,0xf6,0x56,0xd3,0x27,0x34,0x8e,0x43,0x60,0x1c, +0xd8,0x3a,0x67,0x37,0xea,0x13,0x2a,0x93,0x10,0x1b,0x07,0xd6, +0x56,0xd8,0x5b,0xfa,0x84,0xfa,0x74,0x08,0x8f,0x43,0x7b,0xcf, +0xde,0x81,0xf2,0x3e,0xb5,0xe7,0x38,0x4e,0x88,0x5b,0x65,0x4d, +0xe7,0xb9,0xe7,0x7a,0x87,0x9c,0xe8,0xed,0x27,0xa0,0x1a,0x49, +0x0d,0x01,0xbb,0xbc,0xfc,0x28,0x8e,0x4d,0xf1,0x22,0xb1,0x5c, +0xa3,0x90,0xb5,0x7c,0x21,0xfe,0x91,0x7c,0x9a,0xed,0x1c,0xa4, +0x31,0xd1,0x08,0x7b,0xee,0xa7,0x59,0xd2,0x3b,0x93,0x52,0xe5, +0xfe,0x1b,0xaf,0x0f,0x80,0x00,0x0c,0x7e,0x55,0x77,0x24,0x5a, +0xa0,0x95,0xcd,0x4e,0x76,0xf1,0x34,0x95,0x5e,0x55,0x8b,0x93, +0xe5,0xac,0x6d,0x2e,0xa4,0xd8,0x77,0x4b,0x3e,0xee,0x74,0x4a, +0xa7,0xe2,0x7c,0x57,0x2e,0x50,0x90,0x2e,0xab,0x54,0x81,0x02, +0xcb,0xa8,0x9c,0xc5,0x37,0xb3,0x47,0xe0,0x32,0xcb,0xba,0x26, +0xa3,0xe3,0x39,0x9a,0x94,0xeb,0xd2,0x0d,0x9c,0x36,0x9f,0xff, +0xf6,0xc8,0xb2,0xe3,0xf3,0xdf,0xda,0x79,0xbd,0xc0,0xa2,0xee, +0xd5,0x21,0x23,0xa0,0x50,0xa8,0x92,0x9d,0x5b,0x23,0x68,0xe8, +0xf3,0xef,0x78,0xd7,0x7e,0xeb,0x5e,0xaf,0x32,0x11,0x32,0x67, +0xc9,0x07,0xb9,0x81,0xe5,0x9c,0xf8,0x56,0xf9,0xfc,0x2a,0x56, +0x15,0x9d,0x84,0x35,0x8a,0x40,0xff,0xf2,0xad,0x48,0x9c,0x9e, +0x9d,0xba,0xd3,0x03,0xd8,0x00,0x72,0xb0,0x48,0x06,0xc3,0x00, +0x32,0x66,0xbe,0xa7,0x5b,0x06,0x75,0x5e,0xce,0x83,0x52,0xc6, +0xb0,0x62,0x0f,0x48,0x5f,0x3a,0x53,0x2e,0xc7,0x41,0xb1,0x45, +0x52,0xb3,0x0a,0x42,0xaa,0x76,0x8e,0x89,0x35,0x07,0xb8,0xd8, +0xaa,0x8d,0x66,0x5d,0xce,0x74,0x87,0x2a,0x5d,0xdb,0x41,0xe5, +0xc2,0x05,0x07,0x5d,0x45,0x1b,0xef,0x8c,0x2b,0xa0,0x82,0x9b, +0x6e,0x35,0x50,0x8d,0xb3,0x6e,0x35,0x70,0xa5,0xcb,0x6e,0x6d, +0xe5,0x65,0xc7,0xdd,0x6a,0xd0,0xf2,0x92,0x85,0x9d,0x68,0xa9, +0x64,0x39,0xb3,0xcc,0x8d,0xba,0x5d,0x8d,0x6a,0x1c,0x96,0xb3, +0x5a,0x11,0xd3,0xd6,0x19,0x59,0x75,0x22,0x59,0x35,0x0e,0xb6, +0xc3,0xad,0x5f,0x11,0x34,0x77,0x39,0x9f,0xf3,0x96,0x7d,0xb0, +0xc0,0xd7,0xe7,0xe6,0x6b,0xba,0xae,0xb9,0xcc,0xc6,0xf3,0x65, +0x12,0x63,0x66,0xfb,0x22,0xb8,0x84,0xca,0xde,0x53,0x00,0xdc, +0x80,0xb6,0xc8,0x5f,0xa0,0x07,0x5a,0xd1,0x1b,0x2d,0x80,0x15, +0x39,0x67,0xf0,0xea,0xdf,0xca,0x41,0x29,0xe1,0x2c,0x25,0x2f, +0x56,0x26,0xf9,0xdd,0x19,0xb4,0xe4,0x87,0x17,0x90,0x67,0x9b, +0x9d,0x4a,0x09,0x28,0xd9,0xe3,0xd1,0xca,0x4e,0xe7,0x94,0x40, +0xbf,0x52,0xa3,0x88,0x30,0x8f,0xd3,0x5c,0x8f,0xb2,0x9f,0x3e, +0xcc,0x5e,0x70,0x54,0xa4,0xdc,0x00,0xb8,0xe9,0xc1,0x8c,0x9e, +0xbb,0xf9,0x96,0x1f,0x50,0xb5,0x6b,0x90,0x37,0x55,0x05,0xe0, +0x87,0xd1,0xc2,0xce,0xbc,0x19,0x2d,0x24,0xe3,0x67,0xba,0xf7, +0x63,0xe7,0xf1,0x4d,0x20,0x27,0xfb,0xe7,0xf4,0xed,0xf5,0xaa, +0x0c,0x43,0xc9,0x8a,0x82,0x74,0x0a,0x2b,0x54,0x87,0x00,0x4a, +0x91,0xec,0x6f,0x40,0xf8,0x73,0x3b,0x17,0x47,0x83,0xa2,0xfe, +0xe5,0xfc,0xc6,0x65,0x1a,0x7c,0x4b,0xd6,0x73,0xe8,0xd4,0xcd, +0x4f,0x57,0x76,0x6e,0xca,0x49,0x02,0xf0,0xfd,0x08,0xf7,0x58, +0x4a,0x40,0x53,0x93,0xac,0x30,0xc9,0x35,0x55,0xee,0x7e,0x09, +0x7d,0x20,0x79,0xff,0x3d,0xc9,0x33,0xd5,0xfd,0x18,0xa8,0x58, +0x92,0xbf,0x82,0xaa,0x62,0x2d,0x04,0x24,0xf1,0xc1,0x45,0x75, +0x5c,0x51,0xf8,0xfd,0x20,0x9d,0xe1,0x60,0x18,0x27,0xf3,0xab, +0x07,0x17,0x7d,0x48,0xa0,0x7b,0x61,0x6e,0xaa,0xbc,0x07,0xfe, +0x61,0x39,0x5a,0x2c,0x92,0x09,0xee,0x54,0xa9,0xfd,0x00,0x8c, +0xf7,0xfa,0xe0,0x82,0x70,0x6e,0x7b,0x8d,0xb5,0xba,0x7a,0xdc, +0x38,0x8a,0x51,0xda,0xa0,0x70,0xb2,0x59,0xcc,0x97,0xab,0xac, +0xef,0x55,0x65,0x72,0x68,0xe8,0xe3,0x63,0x09,0x11,0x2d,0xb0, +0x78,0x25,0x87,0x7f,0xc5,0x6e,0x46,0x7c,0xe1,0x07,0xf2,0xb3, +0x7d,0x01,0x1f,0x5d,0x1e,0x30,0xf1,0x45,0x70,0xd1,0xfe,0xe5, +0x9b,0x9f,0x5f,0x3e,0xff,0xe9,0xc7,0xb8,0x11,0xb5,0x1f,0xb5, +0x3b,0x0d,0xb2,0x33,0xd0,0x4f,0x2d,0x66,0x77,0x35,0xf8,0x23, +0x92,0xe5,0x30,0x22,0xa0,0x47,0xb2,0x24,0x16,0x0e,0x06,0xd1, +0xa6,0xab,0x22,0xa8,0x8d,0x8e,0xb0,0x95,0x3e,0x7e,0xd8,0x92, +0x7a,0x7c,0x0c,0xa9,0x1a,0x53,0x1c,0x3b,0x99,0xbe,0x95,0xe7, +0x95,0x10,0xf7,0xd4,0x8d,0x2e,0x84,0xe2,0x58,0x07,0x80,0xa0, +0x69,0xbe,0xcc,0xc5,0xe3,0x34,0x0e,0x03,0x81,0x30,0xd9,0x3d, +0xfe,0x73,0x9e,0xf6,0xd2,0x66,0x13,0x09,0x53,0x35,0x88,0x47, +0x2a,0x57,0x83,0x9b,0x9d,0x83,0x74,0x18,0xa4,0xf8,0x03,0x63, +0x2b,0x8a,0x32,0x91,0x8b,0x67,0xec,0x1e,0xaf,0xea,0x41,0x99, +0x89,0x25,0xc6,0x35,0x82,0x07,0x56,0xc5,0x98,0x78,0xff,0x9a, +0xb1,0x14,0x54,0x3f,0x0c,0xe4,0x47,0x0d,0x15,0x3d,0x3a,0x68, +0x88,0x31,0x12,0x07,0xa9,0xde,0x7d,0x7d,0x22,0xfb,0x84,0x14, +0x1f,0xd7,0x38,0xab,0x63,0x37,0xc5,0xc8,0xa1,0xbe,0xe4,0x75, +0xb5,0xd2,0xe0,0x7e,0xc2,0x3a,0x54,0x1f,0x41,0x62,0x5f,0xd2, +0xca,0x7d,0xd3,0xf5,0xc8,0xa5,0x11,0x2b,0xb7,0xbd,0x5d,0xd7, +0x49,0x40,0xa3,0x36,0x98,0xa6,0x1c,0x42,0x85,0x6a,0xe1,0xbb, +0xdf,0xd5,0x3c,0x28,0x15,0x92,0x6d,0x6d,0x2c,0x27,0x37,0x24, +0x59,0x07,0x51,0xe4,0x9c,0xb8,0xc1,0x7a,0xea,0x01,0x8e,0x13, +0x0c,0xe6,0xfd,0x60,0x84,0xa3,0x9b,0x0c,0xb3,0x07,0xb3,0xf9, +0x03,0x89,0xa2,0xc7,0xc1,0xbc,0x1b,0x3d,0x75,0xdb,0x91,0x64, +0x79,0x3a,0xa1,0xd8,0xd0,0xb3,0x77,0xf5,0xec,0xbb,0x81,0x15, +0xb9,0xcb,0x43,0x41,0x58,0x0a,0xba,0x71,0xde,0xe9,0x89,0xd8, +0x13,0x3f,0x61,0x00,0xc2,0xbf,0x14,0xd8,0xc2,0x56,0xa7,0xcc, +0x54,0xa1,0x41,0xf1,0x95,0xb3,0x7c,0xe3,0xb6,0x86,0xb5,0x61, +0xcc,0x3b,0xa1,0x42,0x4d,0xb9,0x25,0x9e,0x83,0xb4,0x31,0x35, +0x7d,0x83,0xd5,0x73,0x68,0xf7,0xbb,0x15,0x39,0x7c,0x01,0xf3, +0x80,0xde,0x52,0xd8,0x11,0x53,0x5c,0xdd,0x5b,0xc4,0xa0,0x52, +0xc9,0xae,0x47,0x45,0x54,0x3a,0x73,0x0c,0xcf,0x1c,0xfc,0xe0, +0x48,0x3e,0xad,0xf8,0xa0,0x18,0x1a,0x87,0x63,0x84,0x5a,0x1d, +0xab,0xad,0x60,0xc4,0xb4,0x0d,0x2e,0xec,0x89,0x47,0x7a,0x6f, +0xf9,0x9f,0xd4,0x69,0x54,0xa7,0xdd,0x73,0x4c,0x84,0xdb,0x7d, +0x94,0xf6,0x07,0xf5,0x21,0xe1,0xaa,0xef,0x48,0x37,0x9b,0xd7, +0x83,0x65,0x5d,0x07,0x0d,0xe3,0x5f,0x47,0xa0,0x22,0x95,0x7a, +0xac,0x52,0x59,0xbd,0xb2,0xba,0xda,0x1e,0x2c,0x20,0xf0,0x9b, +0x0a,0xf6,0x49,0x5b,0xb5,0x5a,0x5c,0x7e,0xd8,0x55,0xbf,0x82, +0x43,0x65,0x88,0x94,0x2e,0xa2,0x1b,0x56,0x08,0x92,0x93,0xf9, +0x87,0x48,0x13,0x06,0xd9,0xa5,0x75,0xf0,0xea,0x5e,0xba,0x53, +0x47,0xba,0x9b,0xe5,0xfb,0x55,0x1d,0xef,0x83,0x1f,0xa6,0xe4, +0xfa,0x1e,0x57,0x20,0x63,0x06,0x5a,0xd7,0x65,0xf3,0x40,0x2b, +0x3f,0xa6,0x9a,0x4c,0xad,0x8b,0x36,0x87,0xd0,0xfd,0x63,0x75, +0x3e,0xdb,0x71,0x32,0x3d,0x73,0x45,0x7a,0x76,0xa6,0xcf,0xbe, +0xc9,0xf9,0x54,0xfd,0x7f,0x20,0x37,0x8e,0x8f,0x9d,0x89,0x82, +0x43,0x29,0xda,0xd3,0x00,0xe9,0x83,0x1d,0x7a,0x5b,0xb3,0x40, +0xef,0x2b,0x09,0xdd,0x07,0x75,0xda,0xd1,0x81,0x64,0x6e,0x75, +0x3d,0x48,0x10,0xd9,0xb5,0xd0,0x37,0xb8,0x44,0xdc,0x67,0x20, +0x49,0xca,0xdd,0x9d,0xad,0x20,0xf0,0x3c,0x65,0x95,0xae,0x72, +0x75,0xb0,0x48,0xf2,0x70,0x14,0xd6,0xf5,0x5b,0xd7,0xb2,0xaf, +0xb9,0xd7,0x98,0x02,0xd5,0x69,0x94,0xd1,0xd7,0xe9,0x9f,0x3e, +0x65,0x63,0xed,0x4a,0x3c,0xf9,0xcf,0xf1,0xf1,0xa1,0x73,0x77, +0x9f,0xa5,0xb8,0x2b,0xd6,0x0b,0x0e,0xd9,0x23,0xc6,0xa1,0x9e, +0x87,0x98,0x21,0xcf,0xd0,0xee,0x47,0xd6,0xcd,0xf2,0x3f,0x92, +0x75,0xd1,0x1e,0xd6,0xe1,0xea,0x83,0x39,0x47,0xf5,0x2b,0xc6, +0x61,0x72,0x5f,0xa5,0xfe,0x3e,0xb6,0x09,0x29,0x40,0xac,0xd0, +0x74,0x30,0xdf,0x84,0x61,0x46,0x0b,0x58,0x7c,0x33,0xe1,0xd7, +0xc8,0x7e,0x19,0x4f,0xd7,0x93,0xc4,0xe5,0xdb,0x0a,0x66,0xba, +0xc4,0x50,0x61,0xda,0x7f,0x14,0x75,0x9d,0xe5,0x15,0x37,0x5f, +0x16,0x56,0x9a,0x03,0x92,0xd9,0x6f,0x45,0x64,0xc3,0xab,0x7c, +0x4f,0xd0,0x76,0xab,0xd5,0x9f,0xae,0x8e,0x9f,0x73,0x89,0x63, +0x06,0xdf,0xd2,0xe8,0x48,0x67,0xb7,0xf3,0xf7,0x05,0x2a,0xed, +0xab,0x67,0x23,0xbc,0x62,0x96,0xd5,0xdc,0x80,0x4c,0x69,0xbb, +0x9c,0x42,0x33,0xe9,0x0d,0x36,0x29,0x6c,0xe2,0x58,0xa0,0x59, +0x5a,0x4f,0x94,0xc7,0x45,0xe5,0x49,0x81,0x2e,0xe5,0xa9,0xeb, +0x66,0xbe,0xb9,0x6a,0x06,0xdd,0x80,0x94,0xf8,0x4c,0xf4,0x62, +0xba,0xb6,0x0f,0x68,0x11,0x3b,0xbd,0x59,0x73,0x60,0x9d,0xcc, +0x08,0x7a,0x32,0x86,0xf1,0x7d,0xb8,0x4e,0x96,0x05,0x1e,0x8c, +0x56,0x2b,0x0c,0x8e,0x95,0x2e,0xb3,0x95,0x13,0xd9,0x8c,0x5f, +0xa6,0xa1,0x5c,0xbf,0x4f,0xd9,0x6a,0x28,0x0d,0x86,0xdd,0x8b, +0x01,0xa7,0x34,0x70,0x1a,0x6b,0x74,0x1b,0xac,0xdc,0x1a,0xc3, +0x4a,0x5a,0xac,0xa5,0x0a,0x18,0x42,0x0f,0x18,0x25,0x4c,0xf0, +0xf4,0x83,0x88,0x03,0x0b,0xc1,0x50,0x2a,0x36,0x8c,0x1e,0x3c, +0x47,0x21,0xd3,0x8e,0x55,0xfd,0xa5,0x86,0x7e,0x8b,0x72,0x6a, +0xa2,0xd5,0x32,0x9c,0xa7,0x03,0x64,0xd4,0xe6,0x80,0x55,0xa3, +0x56,0xbc,0x18,0x46,0x4c,0xc5,0x28,0x9c,0x93,0x3f,0x06,0x9a, +0x00,0xe1,0x50,0x96,0x7b,0xf0,0x8b,0x05,0x97,0x8d,0x8c,0x2f, +0x1e,0x3f,0x7a,0x74,0xf6,0x48,0x19,0x5f,0xf4,0x84,0x29,0xd4, +0x27,0x9d,0x8a,0x9f,0xb4,0x86,0xea,0x95,0x2b,0x60,0x26,0x63, +0xa6,0x14,0x6e,0x45,0xa7,0xa1,0xad,0x42,0x3e,0x8e,0xe7,0x37, +0x8b,0xf5,0x2a,0x99,0x74,0x31,0x87,0xc7,0x29,0xfd,0xd4,0x37, +0x57,0x0f,0xd1,0x05,0x7c,0xeb,0x87,0x11,0x69,0x0d,0xd0,0x3f, +0x50,0x15,0xb0,0xa8,0xf6,0x54,0xf9,0x73,0xa6,0xac,0xad,0xbe, +0xc1,0xc6,0x54,0xb4,0x32,0x75,0x8c,0x40,0xd3,0xad,0x7e,0x6c, +0xcd,0xfc,0xc9,0x97,0xe6,0xa8,0x57,0xd2,0xd9,0x7f,0x6e,0xaf, +0xa4,0xb3,0x4f,0xe9,0x95,0x07,0xb5,0xbd,0x62,0x3a,0xe5,0xff, +0x64,0x9f,0x7c,0xf1,0x47,0xf6,0x49,0x76,0xbd,0xbe,0xba,0x9a, +0x26,0x85,0xcd,0x26,0x6a,0xfd,0x68,0x36,0x61,0x2a,0xe2,0x30, +0x10,0xb0,0x89,0x65,0xd8,0xd5,0x34,0xdb,0xc7,0x57,0x16,0xd1, +0xd8,0xc5,0x3f,0xf3,0x1b,0x8f,0x30,0x34,0x9b,0xbe,0x46,0xc1, +0xa6,0x75,0x2b,0x1a,0xc6,0x3a,0x05,0x41,0x87,0x81,0xfb,0xc9, +0x3a,0x62,0x6b,0xca,0x11,0xb5,0xa3,0x9b,0x45,0x81,0xd8,0x60, +0x16,0xbc,0x5d,0x8f,0x96,0x26,0xda,0x7a,0x71,0xd5,0xf5,0x45, +0xe7,0xee,0x8e,0x20,0x70,0x92,0x1d,0x68,0xba,0x8c,0xe4,0xb4, +0x22,0x1f,0x74,0x9c,0xe2,0x04,0x31,0x80,0xb7,0x53,0xbd,0x30, +0x50,0xa3,0x1b,0x1f,0x4f,0x52,0x4f,0x4c,0x4d,0xe9,0xc1,0xf5, +0xe7,0xca,0x1c,0xa8,0x51,0xc6,0xce,0xec,0xc1,0x59,0x7d,0xee, +0x9e,0xba,0x7d,0x3d,0x7e,0x3f,0x6c,0xb8,0xc5,0x19,0x37,0x9b, +0x2f,0x57,0x5f,0x15,0xac,0x13,0xd5,0xad,0xf6,0x4a,0x53,0x11, +0xe1,0xd2,0x24,0xf5,0x99,0xc9,0x8a,0x66,0x16,0xaf,0x6e,0x02, +0xa9,0xb0,0x24,0x1c,0x49,0xa2,0xec,0x2e,0x03,0x8d,0x97,0x58, +0x67,0x3a,0xea,0x1e,0x6a,0xb4,0x6e,0x7d,0x6a,0x8c,0x71,0x96, +0x9d,0x26,0x57,0xab,0x60,0x49,0xeb,0x57,0x9e,0x8a,0x63,0x4c, +0x69,0x2b,0xc4,0xc1,0x65,0x4c,0x99,0x3a,0x01,0x87,0xec,0x08, +0xe6,0x8c,0x4b,0xd2,0x0e,0xa3,0xf3,0xcb,0xbb,0x3b,0x9e,0x9a, +0x30,0x16,0xa9,0x1e,0xb2,0x08,0x75,0x79,0x3e,0xb2,0xf2,0x2e, +0xb5,0x92,0x55,0x67,0x02,0x54,0x0f,0x4b,0x1f,0x57,0x41,0xbf, +0x41,0xc4,0x1a,0xbc,0x5f,0x23,0xfd,0xfb,0x76,0x39,0x5f,0x2f, +0x0c,0xeb,0x2f,0x93,0xeb,0xd1,0x6d,0x6a,0x45,0x45,0xde,0xd7, +0x29,0x6a,0x30,0x6e,0xb5,0x7e,0x8b,0xd9,0x22,0x22,0xd8,0xbe, +0x31,0x1f,0xbb,0x3b,0x7b,0x6d,0xa7,0x5a,0xd1,0x6b,0xeb,0x03, +0x4c,0x3c,0xd2,0x7b,0xaa,0x15,0xa2,0x2a,0xd0,0xb0,0x08,0x0a, +0x0b,0x1d,0x12,0x3c,0x6a,0x3c,0x48,0x1e,0xfd,0x75,0x3c,0x9c, +0xdd,0x52,0x1f,0x3d,0x7e,0x96,0xc5,0x64,0xf8,0x62,0xed,0xd2, +0xb4,0xde,0xb5,0x7e,0xe3,0x0e,0x47,0x61,0x65,0x75,0xc1,0xac, +0x3f,0xa4,0x1e,0x1b,0x91,0x52,0x09,0x68,0x91,0xae,0x67,0xab, +0x5d,0xc5,0xf1,0xd6,0xf3,0x0e,0xfa,0x9a,0x4d,0x87,0xc2,0x88, +0x90,0xa2,0x9c,0x26,0x13,0xb2,0x46,0x4d,0xf7,0xd3,0x06,0x5f, +0xb0,0x7b,0x75,0x20,0xfd,0xab,0xf5,0x7a,0x7d,0x17,0x9b,0x6d, +0x14,0x65,0x27,0xb1,0xf5,0x5a,0xbb,0x41,0x8b,0xd7,0x6e,0x3e, +0x80,0x02,0xbe,0x06,0x81,0x8d,0x89,0x16,0xb5,0xe3,0x82,0x29, +0xe7,0x90,0xd9,0x93,0x37,0xb2,0xd2,0x09,0x68,0x81,0x0f,0x4d, +0x4a,0x3e,0x3f,0x87,0x01,0x51,0x89,0x92,0x50,0x0c,0x00,0x78, +0xe8,0x7f,0xc1,0xf2,0x88,0xf8,0xe1,0xbb,0x19,0x75,0xa9,0x0e, +0xf8,0xa9,0xc7,0xcb,0xfc,0x03,0x2a,0xdd,0xd5,0xbc,0x10,0x77, +0xb3,0xa0,0xb7,0xfa,0xee,0x34,0xdd,0xb7,0xcc,0x6a,0xfc,0xee, +0xd6,0x6d,0x9f,0xf7,0x8d,0x3e,0xb2,0x96,0x54,0x5d,0x7d,0xcb, +0x9e,0x0a,0x0f,0x86,0xa4,0xf7,0xd3,0xdf,0x8a,0x53,0x54,0x69, +0xad,0x11,0xd6,0x57,0x64,0x7e,0x76,0xad,0x9d,0x28,0xb5,0xfb, +0x44,0xd6,0x26,0xd8,0xb7,0x30,0x67,0xe1,0xc1,0x3b,0xfc,0x59, +0x8d,0xec,0xb5,0x03,0x4b,0x40,0x71,0x9a,0xe1,0x8a,0x29,0x4f, +0x99,0xca,0x9c,0x34,0x53,0xd3,0x0d,0xb3,0x3a,0x1c,0x76,0x9d, +0x75,0x06,0xe2,0xc2,0xc9,0x84,0x57,0x29,0xbc,0xbb,0xb4,0xa7, +0xaa,0x8a,0xf2,0xb6,0x24,0xb4,0xbc,0x42,0xc5,0xb0,0xe8,0xf2, +0x09,0x3f,0x1e,0x4e,0xfd,0x51,0xed,0x70,0x6a,0x8c,0x2a,0x1a, +0xa5,0xa7,0x4a,0x07,0x72,0x16,0x84,0xbe,0x6c,0xa0,0x10,0x87, +0x61,0x1d,0x89,0xdb,0xe0,0x93,0xe5,0x7c,0x71,0xef,0x66,0x57, +0x34,0x73,0x4b,0xba,0xe0,0x66,0x31,0x1a,0x17,0x1b,0x5a,0xde, +0x8c,0x61,0x24,0x96,0xa8,0xb1,0xba,0xbf,0x9a,0xe2,0xab,0x8e, +0x33,0x3b,0x12,0x3e,0x98,0x4b,0x60,0x76,0x40,0xc5,0xf3,0x0f, +0xc1,0x7c,0xbd,0x82,0x4f,0x43,0x14,0x27,0x53,0xe0,0x61,0xde, +0xe7,0x20,0x68,0x3d,0x04,0x28,0xb8,0xf9,0x78,0xb4,0x12,0xc3, +0x93,0x4b,0x07,0x04,0xa4,0x96,0xf2,0x5c,0xa2,0x68,0x36,0x14, +0xa2,0xa4,0xfb,0x77,0x77,0x9c,0x22,0xc6,0x8c,0x32,0x21,0xa4, +0xfe,0x3e,0xea,0x53,0xb7,0x0e,0x06,0xe8,0x4a,0x73,0x64,0xb2, +0x28,0xb4,0xa2,0xcb,0x7f,0x0b,0xda,0x58,0x32,0x51,0xfd,0x97, +0x98,0xc1,0x4c,0x13,0x34,0x66,0x0a,0x94,0x4a,0x9c,0x5c,0x7c, +0x3b,0x96,0x96,0x9d,0x29,0x2c,0x73,0xd7,0x3b,0xfa,0x63,0x92, +0x5e,0x5d,0xc1,0xb2,0x6d,0x06,0x16,0x96,0x20,0xa8,0x5a,0x88, +0x47,0x2c,0x37,0xeb,0x59,0xfa,0x2b,0x87,0xe3,0xfa,0x75,0x5d, +0x1a,0x94,0x69,0xf6,0x92,0x74,0x76,0x85,0x6e,0x76,0xec,0x2f, +0x05,0x27,0xd1,0xe1,0xe9,0x15,0x61,0x5d,0x44,0x6b,0x71,0x83, +0x4d,0x7e,0xc4,0x47,0x11,0x6f,0xf1,0xa8,0x81,0x6a,0x69,0x78, +0x54,0x5d,0x42,0x44,0x69,0xcb,0x86,0xd3,0xcd,0x5e,0x68,0x90, +0x25,0xc9,0xac,0x68,0x3a,0x0b,0xce,0x9a,0xe9,0x5d,0x93,0xdc, +0xa7,0x84,0xe3,0x63,0x44,0x31,0xc0,0x7f,0xcc,0x18,0x8c,0xc5, +0xa4,0xe8,0xda,0x2f,0xa5,0x02,0x84,0x88,0x02,0x88,0x10,0x7d, +0xda,0xbd,0x1d,0x38,0x3b,0x9d,0x3c,0xb2,0x79,0xab,0xdb,0x3e, +0xf8,0x12,0xbe,0xdb,0x8e,0x5e,0x56,0xf7,0x61,0x57,0x78,0x5a, +0x54,0xac,0x1e,0x3b,0x92,0xa1,0x4e,0xcf,0x43,0x64,0xc5,0xd0, +0xe6,0x22,0x05,0x62,0x21,0xad,0xaa,0x77,0x5f,0x22,0xcb,0x68, +0x95,0x61,0x2b,0x15,0x72,0x71,0xc3,0x2d,0xe0,0xba,0xed,0xc8, +0xc7,0x83,0x11,0x11,0x5b,0xd1,0x4f,0x56,0xd7,0xc9,0xd2,0x36, +0xc8,0x65,0x5f,0x89,0xd2,0x03,0x42,0x70,0x1e,0x87,0x5b,0xd9, +0x7d,0x31,0x52,0x59,0x4f,0xb3,0x33,0xb6,0x8d,0x17,0x40,0xad, +0x00,0x97,0x1a,0xc3,0x92,0x51,0xbd,0x64,0xb0,0xdf,0x39,0xa0, +0x86,0xa8,0x01,0x8a,0xd4,0xfd,0x96,0x2e,0xec,0xce,0x50,0xd6, +0x83,0x4c,0x77,0xb4,0xe3,0xe1,0x29,0x3b,0xdf,0x90,0xd0,0xe0, +0xfc,0x86,0x2f,0x7e,0x20,0x1e,0xf4,0x90,0x96,0x4b,0x3c,0xbe, +0x60,0x8d,0x23,0xe7,0x35,0x78,0x9a,0xec,0x1e,0x1d,0x0b,0xe8, +0x20,0xc5,0x1b,0xbd,0x25,0xe4,0x8d,0x66,0x6a,0xc7,0x0d,0x51, +0x92,0x33,0xbf,0x74,0xb7,0xca,0x71,0x1d,0xa0,0x1f,0x2c,0xd4, +0x07,0x60,0xb4,0x3a,0x90,0xc5,0xc6,0x56,0x5b,0x43,0x96,0xed, +0x6c,0x4e,0xb6,0x11,0xb4,0xf2,0x64,0x9b,0x91,0x2a,0xbb,0x0e, +0xc1,0xf0,0x28,0x5b,0x5e,0x42,0x84,0x9f,0x5d,0x37,0x07,0xe6, +0xe2,0x61,0xac,0x7e,0x47,0xc3,0xde,0xbf,0x38,0xb4,0x6f,0x03, +0x2d,0x21,0x25,0x35,0xb3,0xa2,0xf0,0xee,0xa2,0x43,0x4c,0x1b, +0x58,0x3c,0xd4,0x32,0xa3,0x57,0xf0,0x04,0x70,0xac,0x35,0x3c, +0x84,0xb7,0x11,0x34,0x66,0xeb,0x9b,0xcb,0x64,0x69,0x3c,0x2b, +0x74,0xae,0x3a,0xc9,0x89,0x1d,0x83,0xd4,0xa2,0xc4,0x17,0x23, +0x2e,0x45,0x05,0x80,0x09,0xfd,0xb4,0x0b,0xb5,0x43,0xcd,0xe7, +0x5a,0x69,0x58,0xeb,0x56,0x26,0xa0,0xa9,0xf1,0x77,0xd5,0xaf, +0xad,0xf6,0x95,0xd0,0xbb,0xb3,0x4c,0x72,0xcd,0xfe,0xac,0xaf, +0x17,0xd8,0x16,0x90,0xe7,0xf2,0x86,0x3a,0xb2,0xe4,0x7e,0x50, +0xa0,0x57,0x37,0xb1,0xa7,0x57,0x68,0x62,0xac,0x3c,0xdf,0xc1, +0x7f,0x8c,0x56,0xb3,0x9b,0xf7,0x60,0xef,0xb3,0xf1,0xce,0xee, +0x20,0x58,0xa0,0xa7,0xdb,0x68,0xf9,0xef,0xa8,0x76,0xda,0x35, +0xaa,0xb6,0x5a,0x60,0x8a,0x4c,0x85,0xb6,0x5f,0x2a,0xe5,0x19, +0xc2,0xba,0xd5,0x99,0xc6,0xce,0x4f,0x63,0x8d,0x07,0x0b,0x74, +0x5d,0xe9,0x68,0xb5,0x7a,0x07,0x33,0x6a,0x89,0x4f,0x0e,0xd8, +0x6f,0x56,0x8f,0x96,0x60,0xae,0xac,0xe6,0x0b,0xf8,0x27,0x59, +0xf8,0x1f,0x4b,0x1b,0x20,0x71,0x04,0xd3,0x1e,0x02,0xc4,0x04, +0x7b,0x77,0x17,0x06,0xf4,0x23,0x0e,0x7d,0x2a,0x62,0x0e,0xaa, +0xff,0x3f,0xde,0x8e,0x65,0xb9,0x71,0xdc,0x78,0xce,0x5f,0x58, +0x4c,0x56,0x22,0x43,0xda,0xb2,0xe7,0x92,0x8c,0x38,0x1c,0x55, +0x6a,0xb2,0x5b,0xb5,0x97,0x39,0x64,0x73,0x93,0x14,0x15,0x29, +0x53,0x1e,0x6e,0xe9,0x65,0x49,0xce,0x78,0xca,0xd4,0xbf,0xa7, +0x9f,0x78,0xf0,0xe1,0x71,0xed,0xa6,0xf6,0x22,0x42,0x40,0xa3, +0xd1,0x00,0x1a,0x8d,0x46,0xa3,0x01,0xcc,0xde,0x2d,0xea,0xfa, +0x2e,0x6d,0x48,0x17,0xc3,0x4c,0x14,0x58,0x95,0xd5,0x26,0x24, +0x74,0xd7,0x84,0x25,0x1a,0x53,0xb1,0xa0,0xe7,0x25,0xd5,0x3d, +0xda,0x8a,0x98,0xbe,0x96,0x74,0x31,0x5c,0x71,0xff,0x9c,0x46, +0x04,0x83,0x67,0xce,0xe3,0x78,0x91,0x69,0x0d,0xe0,0x37,0xce, +0x10,0x99,0x11,0x2a,0x08,0xc5,0x5a,0xda,0xaa,0xf5,0xf4,0x00, +0x6f,0x71,0xbb,0xef,0x1d,0xec,0x56,0xfe,0x42,0x1c,0x4d,0xeb, +0x49,0x01,0x2b,0xc5,0x7b,0xcb,0x06,0xe8,0x6a,0x35,0x1c,0x22, +0x2c,0x67,0xd7,0x7e,0xc7,0x78,0x6d,0x6e,0x1b,0x23,0x92,0x9e, +0x50,0xf7,0xca,0x78,0x7e,0xa3,0xd4,0xd1,0x36,0x10,0x3c,0xea, +0xda,0x23,0x4e,0xad,0x89,0xaa,0x7f,0xf7,0x81,0x08,0x76,0xab, +0x8a,0xf8,0xc3,0xa6,0x8b,0x17,0x41,0x19,0x8b,0x25,0xd5,0x87, +0x69,0xb5,0x0b,0xbd,0x87,0x93,0x4a,0xfd,0xae,0xa2,0x22,0xa0, +0x7c,0xe5,0x3d,0x53,0x40,0x65,0xda,0xbf,0x29,0x3b,0xeb,0x6e, +0xd6,0xec,0x9b,0x0b,0xa0,0x4d,0x78,0xba,0x86,0xd3,0x91,0xda, +0x0e,0x15,0x98,0xef,0x4d,0x24,0x48,0x05,0xe4,0x40,0xa4,0xec, +0x39,0x41,0xaf,0x70,0x48,0x37,0xd0,0x10,0xdd,0x85,0xb6,0x48, +0x80,0x47,0xbc,0x15,0x13,0xb5,0xf5,0x77,0xb6,0x74,0xac,0x52, +0xd1,0x56,0x67,0x1c,0x9a,0xc9,0xc9,0xf3,0x0d,0x34,0x13,0x21, +0xc8,0x3c,0xff,0x68,0x6c,0xbd,0x32,0x1d,0x18,0xd3,0x4f,0x08, +0x74,0x26,0x1a,0xa7,0x08,0x48,0xdd,0xc8,0x5a,0x6f,0xd7,0x0a, +0xf2,0xab,0xed,0xd3,0xe9,0x7c,0x55,0x94,0x57,0x87,0x1c,0x0f, +0xf4,0x1a,0xea,0xaf,0xd0,0x15,0xfd,0x14,0xf8,0x96,0x22,0xc2, +0x68,0x35,0x8b,0x75,0xf4,0x82,0x26,0xc5,0xb5,0xbd,0x64,0x84, +0xfe,0x91,0x49,0x01,0x55,0xfe,0xe2,0x57,0xb2,0xc8,0x97,0xdb, +0xbd,0xb7,0xb4,0x26,0x4e,0x07,0x41,0xf5,0x45,0x1d,0xc9,0xc9, +0x21,0xc2,0xde,0x9c,0xc9,0x49,0xa0,0x58,0x72,0xc0,0xdd,0x0b, +0x6d,0x3a,0xce,0xa2,0x71,0x8a,0xa1,0x1a,0xed,0x2b,0x0d,0x69, +0x75,0x23,0xb4,0xd3,0x90,0x57,0x06,0x59,0x69,0x30,0xc4,0x36, +0x24,0x13,0xca,0x3a,0x7a,0x49,0xb0,0x50,0x67,0xdc,0x97,0x1b, +0xd7,0x3e,0x41,0x95,0xf8,0x9a,0x57,0xe7,0xef,0xef,0xf5,0x29, +0x11,0xa7,0xf2,0xfc,0xef,0x6a,0x5b,0xc2,0xb2,0x25,0x7c,0x95, +0x47,0xe8,0x68,0x37,0x6f,0xd9,0x71,0x09,0x5c,0xfc,0xda,0x7d, +0x64,0x89,0x59,0xd2,0xae,0x76,0x90,0x38,0xc9,0xbe,0x4c,0x66, +0x44,0xdc,0xdd,0xe2,0x35,0x1e,0x43,0xa9,0x42,0x88,0x91,0x31, +0xce,0xe7,0x4d,0xb3,0x7f,0xb0,0x60,0x7d,0x68,0x40,0xb7,0x16, +0xec,0xa0,0x17,0xfd,0x2e,0x39,0x73,0x7d,0x32,0x39,0x8d,0xce, +0xbe,0xcb,0xa0,0xe0,0x49,0x46,0xe8,0x43,0x09,0x65,0x78,0xa7, +0x02,0x89,0xfb,0xfc,0x5c,0x7a,0x42,0xd6,0xe4,0x12,0x48,0x60, +0xd8,0xfc,0x1e,0xbd,0x88,0xf1,0x3c,0xe8,0xf4,0x96,0x5c,0x49, +0xff,0x09,0x99,0xfc,0xb2,0xda,0xd2,0xc0,0x25,0x2f,0xba,0x74, +0x0c,0x46,0x3a,0xb2,0xb0,0xff,0x9a,0x29,0xc2,0x54,0x4b,0xae, +0xeb,0x46,0xd1,0x74,0x14,0x15,0x68,0x37,0xa4,0x41,0x36,0xdd, +0x86,0xdf,0x82,0xea,0x8c,0xd4,0x61,0xfb,0x5c,0x87,0x90,0x70, +0xad,0x50,0xee,0x2d,0x6d,0xb4,0xf8,0x33,0x63,0xdd,0xce,0x7d, +0xc9,0xed,0xc7,0xcc,0xe0,0x98,0x86,0x2b,0x28,0xf0,0xa8,0x3c, +0x21,0xf5,0x8b,0x7a,0x1a,0x15,0x8a,0xfa,0x5e,0xad,0xa3,0x89, +0x64,0xb5,0x35,0x3a,0x1f,0xf3,0x6a,0xa3,0xad,0x89,0x2f,0x58, +0x09,0x6e,0x87,0x17,0xa9,0x47,0x12,0x43,0x55,0x64,0x4d,0xb5, +0xc4,0x76,0x38,0x09,0xac,0x3a,0xb9,0xa3,0xda,0x6e,0xcb,0xfb, +0x2a,0xd7,0x5b,0xa7,0x04,0x35,0xf3,0x87,0xd2,0x85,0x91,0x30, +0x9f,0x6c,0x0f,0x89,0xef,0x6d,0xe4,0xf4,0xcb,0x6b,0xcd,0x65, +0xb2,0xdb,0x5e,0xeb,0x64,0x22,0x8e,0x84,0xc5,0x93,0x82,0x5d, +0x9b,0x9c,0x29,0xd2,0xfa,0x11,0x13,0xa7,0xbd,0x95,0xa7,0xce, +0x44,0x90,0x68,0x12,0x7a,0xad,0x6f,0xaa,0x68,0xdd,0x1d,0x7a, +0x1b,0x5f,0x4e,0x28,0x7f,0x06,0x16,0x33,0xd9,0x86,0xc3,0x81, +0xe0,0x33,0x37,0xa8,0x6a,0x0f,0xf5,0x77,0x05,0x8d,0xf9,0x48, +0x91,0xd9,0x9d,0xb7,0xde,0x92,0xdd,0x0e,0xdb,0xb7,0x3a,0xcb, +0xca,0x58,0xd4,0x9a,0x1c,0x17,0x92,0xb6,0x10,0x82,0x74,0x12, +0x8d,0x93,0x90,0x20,0x6f,0xc9,0x8f,0xed,0x15,0xc1,0xc8,0xb7, +0xdc,0x52,0x4b,0x91,0x47,0x1f,0x51,0x80,0xfe,0xe0,0x4d,0x76, +0x21,0x17,0xf1,0x8e,0xdd,0x0d,0x47,0x78,0x92,0xd0,0xb2,0xcf, +0x1e,0x59,0xd3,0x13,0xf1,0x93,0x53,0xa6,0x20,0x6b,0x4e,0xa8, +0x5c,0x38,0xda,0xe9,0xf6,0xee,0x63,0x21,0xee,0x7c,0xd9,0xba, +0xee,0xb7,0x63,0xb1,0xdb,0x60,0xc0,0xca,0x9b,0x45,0x71,0x05, +0x04,0xab,0x79,0x54,0xa5,0x23,0x4b,0x34,0xae,0x02,0x5b,0xd4, +0x2c,0x5c,0x3d,0x0c,0x16,0x85,0x44,0x5d,0xbe,0xf6,0xf8,0x96, +0x78,0x34,0xf1,0x44,0x79,0xab,0x4f,0xae,0x99,0x93,0x3f,0xdc, +0x4d,0xfb,0xbb,0x41,0x3d,0x6a,0xb0,0x08,0x72,0x85,0xb4,0x07, +0x01,0x9c,0xb7,0x15,0x49,0x79,0x60,0x4f,0x73,0x10,0x70,0xa2, +0x0c,0xd1,0x96,0x74,0x97,0xa7,0x61,0xf0,0xf3,0x8e,0x5e,0xb1, +0x97,0x47,0xdc,0x02,0x96,0x7f,0x84,0x7d,0xb6,0x48,0x1b,0x0e, +0x18,0x88,0x85,0xa7,0x58,0xf5,0x27,0x19,0x0e,0xc9,0xf7,0x92, +0x2c,0x40,0xf8,0x5f,0x5b,0x03,0x63,0x2f,0xe6,0x92,0xdb,0x86, +0x66,0xd3,0xeb,0x81,0xde,0x76,0xe7,0x94,0xb5,0xfe,0x5b,0xec, +0x0b,0x66,0xa5,0x9e,0xb9,0xbe,0xe8,0xa9,0xeb,0xcf,0x72,0x62, +0xb5,0xaf,0x3a,0xfe,0x1e,0x8a,0x38,0xff,0x1b,0x08,0x22,0x40, +0xa4,0x67,0xe6,0xf8,0xc3,0x1b,0xc2,0x9c,0x97,0xbf,0x00,0x4c, +0xfc,0x8c,0xca,0xe3,0xb9,0x87,0x32,0x6b,0xcc,0xf8,0x4d,0x7e, +0xfb,0x62,0xbc,0xf0,0x08,0xc8,0x24,0x94,0xb6,0x4c,0x17,0x4a, +0x02,0x96,0xc3,0x8e,0x46,0xcd,0x06,0xa3,0x49,0x16,0x35,0xc7, +0x5e,0x2e,0x71,0xd6,0x30,0x52,0xea,0x02,0xb8,0x85,0xf2,0xb4, +0xd9,0x85,0xa3,0x69,0x6f,0x97,0x1d,0x12,0xe9,0x26,0x85,0xee, +0x5d,0x1a,0xd2,0x50,0x7b,0x54,0x1d,0xab,0x37,0xf2,0x89,0x7f, +0x1a,0x08,0x12,0x34,0x0f,0xbb,0x1d,0xf7,0x07,0x24,0x53,0xa2, +0x91,0x38,0x7e,0x85,0x9e,0x23,0xcc,0xcb,0xf3,0xa2,0xd6,0xe2, +0x53,0x0a,0x1d,0x95,0xa7,0x73,0x7e,0xda,0x1b,0xbf,0xd5,0x94, +0x47,0x55,0xa1,0xb3,0x3b,0xa6,0x00,0xda,0x6e,0xb4,0x0d,0x49, +0x36,0xe6,0xc3,0x37,0xd6,0x5c,0x4d,0x3b,0xd2,0x4b,0x82,0x87, +0x6f,0x34,0x0f,0x6c,0xab,0x26,0xcb,0xfc,0x7e,0xf2,0x3a,0x3b, +0xd4,0xd8,0x13,0x89,0x60,0xa4,0x13,0x26,0xb7,0x0e,0xe2,0xac, +0x7a,0xc4,0x14,0xea,0x81,0xdd,0x3f,0xa4,0x37,0xcd,0x8e,0xbd, +0xe9,0x56,0xf6,0x8c,0xef,0xe8,0x62,0x67,0xe9,0x42,0x2f,0xa3, +0x74,0xd3,0xe7,0xdc,0xb0,0x43,0x1b,0x92,0x8d,0xfd,0x49,0xf2, +0xb5,0x24,0xff,0x0e,0xdc,0x6c,0xb4,0x77,0x92,0xe8,0x66,0x25, +0x69,0xde,0xee,0x3c,0x49,0x9b,0xbf,0x1d,0xcf,0x23,0xbb,0xef, +0x45,0xd3,0xa8,0xc6,0xcc,0x7c,0x8e,0xe9,0xd1,0x31,0x61,0x25, +0x45,0x92,0xff,0x72,0xc6,0xcb,0x25,0x0a,0xfa,0xb0,0x1f,0x83, +0xe3,0xa1,0x70,0x75,0x0b,0x42,0x3f,0xaf,0xeb,0xbb,0x31,0xc4, +0xde,0x8d,0x0b,0x7b,0x4a,0x00,0x22,0x39,0x60,0x40,0x29,0x5f, +0x9a,0x7b,0xa7,0xbd,0xa0,0xb9,0xf2,0x2c,0x37,0xa7,0xbd,0xa2, +0xa4,0x68,0x26,0x17,0x59,0x61,0x93,0xd9,0x9e,0x02,0x2a,0xd5, +0xe9,0x33,0x5e,0xda,0xad,0x07,0x01,0xa5,0x2f,0x69,0x89,0x6a, +0x52,0x07,0x8d,0xe4,0x22,0xb2,0x0e,0x7b,0xa7,0xaf,0xd5,0x19, +0x18,0xc1,0xc0,0x82,0xba,0x98,0x9f,0xca,0x60,0xc6,0xb3,0xd2, +0x15,0x67,0x5b,0x04,0x13,0x4b,0xb9,0xdc,0xaf,0x58,0xe0,0x85, +0xe0,0x0e,0xe4,0x67,0x32,0x85,0x3a,0x90,0x83,0x2c,0xce,0xa7, +0x05,0xfc,0x16,0x13,0xe0,0x8b,0x7c,0xaa,0x0d,0x33,0x81,0x4f, +0x5c,0xf8,0x99,0x51,0xa3,0x84,0xac,0x5e,0x9c,0x3c,0x09,0x6c, +0x30,0xc6,0x1d,0xf9,0xfe,0x55,0x3e,0xfc,0xf8,0x7c,0x70,0x0a, +0xbd,0x91,0x97,0x58,0xa0,0xa9,0x38,0x34,0x1c,0xe6,0x37,0x7c, +0x50,0x14,0xe3,0x38,0x84,0x71,0x5b,0x18,0x19,0xa8,0xb8,0x13, +0xa8,0xf9,0x83,0x29,0xd5,0xc3,0x6e,0x7f,0x2c,0xf1,0xce,0x73, +0x4c,0xb2,0xff,0xd0,0xd2,0x1a,0xc8,0x64,0x6d,0x0c,0xbe,0xd0, +0xb7,0xad,0xb8,0xc2,0xb6,0x6e,0xc3,0x06,0xc7,0x1c,0xe4,0xcf, +0x13,0x6a,0x34,0xa4,0xa4,0x99,0x9c,0x85,0x70,0x9c,0x62,0x8a, +0x46,0x42,0xc1,0x2e,0xd0,0x9f,0xd0,0x94,0x96,0xa3,0x6c,0x38, +0x9d,0x8f,0x4f,0xf4,0xae,0x7c,0x41,0x71,0x85,0x1b,0x47,0x6e, +0x37,0x18,0x8d,0xae,0x37,0x9f,0xc8,0x5b,0x6e,0x10,0x7a,0x53, +0x04,0xa5,0xc2,0xfc,0x40,0x5f,0x97,0xe3,0xf2,0x4f,0xc6,0xb9, +0xce,0x00,0x17,0x02,0x5c,0x34,0x81,0x39,0xc1,0xd6,0x5b,0x2a, +0xca,0xfb,0x53,0x91,0x0c,0x19,0xfe,0x5b,0x30,0xeb,0x92,0x33, +0xc0,0x6d,0x62,0x3d,0xd6,0xb1,0x75,0xb5,0x5b,0x69,0x9c,0x2f, +0x82,0x2c,0x73,0xd8,0x12,0x25,0x10,0xe6,0xc9,0x55,0x1d,0x90, +0xac,0x14,0x09,0xb5,0x16,0x6d,0x80,0x4c,0xd7,0x18,0x77,0x7d, +0x6d,0x15,0xfb,0xf2,0x31,0xcc,0x67,0x18,0xb9,0x48,0x0a,0xf9, +0xfa,0x83,0x39,0x4a,0xa3,0x94,0x9f,0x4b,0x6b,0x7a,0xbe,0x62, +0xdf,0xb0,0xe6,0x95,0x8b,0xde,0x45,0x64,0xc4,0x71,0x32,0x50, +0xec,0x9c,0x5c,0x48,0x32,0x95,0x85,0xf2,0x18,0x8a,0xa2,0x4f, +0xa3,0xa4,0x28,0x22,0x37,0x6d,0xac,0xaf,0x18,0xca,0xa8,0x4c, +0x29,0xaf,0xb0,0xe5,0x29,0xc2,0x01,0xd7,0x46,0xb2,0x69,0x83, +0x61,0xe4,0xc5,0xbc,0x7c,0x26,0xed,0xbb,0x3f,0x84,0xb6,0xb1, +0xe9,0x8f,0xa8,0x16,0x74,0x5f,0xd9,0x8f,0x8f,0x4f,0x9e,0x9b, +0x42,0x52,0xd8,0xc9,0xe0,0x91,0x24,0xdc,0x6c,0xa1,0xdb,0xbf, +0xe2,0x51,0x99,0xb5,0x94,0x5b,0xe3,0xaa,0xa1,0x9d,0x4d,0x3d, +0xe7,0x0b,0x67,0xde,0xf5,0x16,0x39,0xe1,0xfa,0x64,0xca,0xfc, +0xa0,0x03,0xa0,0x63,0xb6,0x33,0xb5,0x57,0x3d,0xb7,0xc3,0xb5, +0x98,0xc9,0x93,0xe3,0xe9,0x5d,0x73,0xc7,0x20,0x1c,0xec,0xf1, +0x56,0xb7,0xbb,0x01,0x17,0xb7,0xdb,0xdf,0x97,0xa8,0x78,0x4b, +0xcd,0xd8,0x11,0xc6,0x3b,0xe7,0xdb,0xd4,0xe2,0x19,0x4f,0x9b, +0x1f,0x7d,0x39,0x4a,0x66,0xb8,0xc4,0x4e,0x54,0xbd,0xce,0x35, +0x99,0xb7,0x1c,0xb8,0x24,0xf2,0x92,0x83,0xf1,0x0b,0x08,0x92, +0x40,0xc7,0x18,0x04,0xb9,0x08,0x08,0xb0,0x48,0x85,0x00,0x8a, +0x47,0xf8,0xb0,0xb4,0x73,0x1f,0x7d,0xd8,0xd1,0xd0,0x58,0xce, +0x82,0xea,0x14,0xc4,0x7c,0xb7,0x45,0x27,0x0d,0x6d,0xba,0xb3, +0xcc,0xd4,0x8e,0x73,0xc6,0xc1,0x22,0xb8,0xb0,0x57,0x97,0xe3, +0xb0,0x60,0xd7,0x40,0xa0,0x73,0x78,0x49,0xaf,0xb6,0xfc,0xc0, +0x76,0x62,0x80,0x45,0x96,0x65,0x40,0x5b,0xd3,0x81,0x66,0x32, +0xd2,0x72,0x7c,0x33,0x86,0x21,0xe5,0x4a,0x99,0x4e,0xc4,0x36, +0x63,0xa6,0x62,0x16,0xe7,0x69,0x26,0xf7,0x27,0xdc,0x85,0xef, +0x51,0x23,0x34,0x55,0x9c,0x8f,0x07,0x15,0x88,0x93,0xcf,0x21, +0x5d,0x71,0xf7,0xd3,0x66,0x9f,0xcb,0x02,0x8d,0x7b,0x11,0x52, +0xfa,0x75,0x11,0xee,0x0c,0xeb,0xc3,0x3c,0x20,0xe7,0x24,0xce, +0x28,0x53,0xd5,0x6b,0xfd,0x3f,0xb8,0xad,0x6b,0x09,0xdd,0xc1, +0x9c,0xd1,0x9a,0xe4,0xfa,0x79,0x0b,0xdf,0x2b,0x7f,0xcd,0x6d, +0x2a,0x33,0x64,0x98,0x27,0xa8,0xbb,0xc1,0x5d,0x15,0x0d,0x33, +0x40,0x0f,0xf5,0x9f,0x53,0xf0,0x2f,0x0e,0x30,0x34,0x11,0x0c, +0x66,0xde,0xed,0xf1,0xe2,0x05,0x50,0xc0,0x3a,0x5e,0xe4,0xbf, +0x92,0x03,0xe2,0xed,0xbb,0x16,0xe4,0x7d,0x68,0x6b,0x70,0xee, +0x73,0xbc,0x35,0xae,0xcd,0xb4,0x38,0xb7,0x50,0xbb,0x0e,0x27, +0x11,0x63,0x59,0x58,0xad,0x9e,0xb6,0x7c,0x62,0x3f,0xf4,0x5d, +0x7e,0x69,0x89,0xb8,0x93,0x15,0x19,0x81,0xe1,0xea,0xb0,0xdb, +0xb7,0xce,0xee,0x86,0x13,0xa0,0xec,0xd5,0xdd,0xef,0x9d,0xd7, +0xd2,0xb7,0xd5,0x2e,0x01,0xdc,0x8d,0x7e,0x80,0x18,0xe0,0x64, +0x3c,0xb4,0x40,0xe9,0xd5,0x0e,0x77,0xe6,0xe0,0x13,0x13,0x2d, +0xeb,0xcd,0x1e,0xc8,0xa4,0xa0,0xf8,0x2e,0x47,0x7f,0x45,0xe0, +0x6b,0x84,0xb8,0x53,0xaf,0x64,0x6e,0x14,0xbc,0x7e,0xe1,0xa5, +0x3c,0xad,0xf2,0x43,0x39,0x79,0x09,0x86,0xc1,0x24,0x18,0xa2, +0x9d,0x0d,0x24,0xc0,0x07,0x0c,0x6f,0xce,0x18,0xfc,0x88,0xc1, +0x07,0x0c,0x8e,0x82,0x11,0x04,0x1f,0x9f,0xf6,0x14,0x3f,0xc2, +0xf8,0x3f,0x3f,0xbf,0xfb,0x5b,0x0a,0xc3,0x39,0x35,0x08,0x6f, +0x9e,0x76,0x8c,0x31,0xd3,0xf5,0x6e,0x68,0xd3,0x38,0x25,0x72, +0x48,0x00,0x51,0x53,0x3e,0x43,0xc3,0x2b,0x19,0xb8,0xea,0x66, +0xf1,0x03,0xb3,0x74,0x10,0xcb,0x2a,0xb8,0x85,0x41,0x5e,0x1f, +0x0d,0x22,0x94,0x26,0x49,0xf0,0x10,0x44,0x89,0x96,0xeb,0xa1, +0x08,0x3b,0x50,0x28,0xa0,0x22,0xa9,0x11,0x4b,0xc4,0x58,0x70, +0x02,0x13,0xa1,0xc9,0x40,0x10,0xad,0xf0,0xae,0x3c,0xd4,0xd3, +0x41,0x4b,0x3d,0xa6,0xe3,0xee,0xb0,0xe2,0xe0,0x6a,0xf4,0x19, +0x47,0x4e,0x83,0x60,0x02,0x44,0xc7,0x02,0x72,0x73,0x2c,0x0f, +0x9b,0x1c,0x56,0x16,0x5e,0x53,0x28,0x46,0xa7,0x30,0xbc,0xde, +0xca,0x4e,0xa1,0x5a,0x11,0x05,0x9c,0x51,0x3a,0x9e,0xe2,0x21, +0x31,0xe5,0xd8,0x18,0x75,0xc0,0x81,0x00,0xc0,0x9b,0x5e,0x69, +0x84,0xf9,0x93,0x2b,0xa4,0x44,0xde,0xa8,0x4d,0xad,0x97,0x29, +0x27,0xcf,0x34,0xe3,0x22,0xfd,0x9e,0xaf,0xba,0x19,0xbc,0x88, +0x75,0xe2,0x1c,0xe4,0x78,0xae,0x9a,0x22,0x8b,0xda,0xd8,0x31, +0x43,0xf0,0x8a,0xa8,0x31,0xdd,0xa8,0xbd,0x2f,0x5b,0xca,0x75, +0x4a,0xb8,0xcc,0xa3,0x10,0xf4,0x92,0xd9,0x58,0x9c,0x35,0xa6, +0x23,0xd7,0x12,0xe9,0xdd,0x6b,0xf1,0x16,0x93,0xa4,0x1e,0x96, +0xc0,0x8a,0x90,0x6d,0xce,0xb1,0xd5,0x2d,0xc5,0x48,0x7b,0xd1, +0xc7,0x90,0xab,0xfb,0x4f,0xe8,0x58,0x5c,0x1e,0x61,0xd0,0xab, +0xcf,0xd9,0xcf,0x8e,0x4c,0x04,0xb1,0xb4,0xae,0x9e,0xf5,0xc9, +0xdd,0x2c,0x8e,0x4d,0x86,0x38,0x08,0x0c,0x31,0x04,0x34,0xe5, +0x0f,0x40,0x4c,0x2a,0x3a,0xbb,0x70,0x2e,0xb7,0x87,0x0d,0xdd, +0x49,0x73,0x3e,0x03,0xaf,0xe0,0x00,0xc1,0x06,0x85,0x98,0xc9, +0xf8,0xc3,0x0f,0xe1,0x6c,0x7e,0x9a,0xff,0xb2,0x88,0xa7,0xd1, +0x0f,0x1f,0xc7,0x0f,0xbc,0xdc,0x3c,0xec,0x37,0x92,0x9a,0x35, +0x92,0x65,0x5c,0x40,0xca,0xb5,0x9f,0xc2,0xd5,0xd8,0xed,0xe9, +0x39,0xf5,0x6c,0x1c,0xde,0x44,0xff,0x19,0x0b,0x34,0x14,0x48, +0x23,0x7c,0x04,0x03,0x60,0x3e,0x87,0x00,0xfc,0x40,0xe8,0x88, +0xd7,0x5a,0x62,0x60,0x07,0x01,0xd4,0x21,0xfe,0x04,0x5f,0x7c, +0x4d,0x6a,0xfe,0xf4,0xee,0xf6,0xdd,0xdf,0xe1,0x0f,0x7f,0x25, +0xe2,0xbd,0x44,0xbc,0x0f,0x2e,0x82,0xf7,0x98,0x8d,0xe7,0xf3, +0x7a,0x54,0xcf,0x8f,0xf5,0x7c,0x57,0xcf,0xcf,0x35,0x67,0xe4, +0xcf,0xfb,0xf1,0x43,0x6a,0xab,0xee,0x98,0x57,0x51,0x5e,0xd2, +0xbd,0xb1,0x27,0x69,0x0e,0xf5,0xb2,0x42,0x81,0x9f,0x6a,0xa4, +0x77,0x77,0xd8,0xc5,0xc0,0x76,0xb4,0x26,0x4b,0x20,0x1a,0x37, +0x72,0x9b,0x94,0x08,0x8c,0x59,0xa8,0xb9,0x44,0xd2,0xc0,0x92, +0x9a,0x9b,0x27,0x92,0x75,0x5e,0x62,0x21,0x9c,0x66,0x7f,0x0d, +0x4c,0x3b,0xae,0x05,0xb3,0x70,0x05,0x50,0xfd,0x17,0x91,0x63, +0xe6,0x10,0x0d,0x2f,0x30,0x83,0xe5,0xf2,0x10,0x67,0xa3,0x20, +0xc5,0x46,0x30,0x12,0x43,0x48,0x6f,0xc8,0x08,0x69,0x63,0x97, +0x1f,0x12,0x2d,0x3d,0xd9,0xaf,0xd7,0x27,0xe7,0xd8,0x25,0xa3, +0x8f,0x33,0x42,0xcb,0x06,0x0e,0x39,0x00,0xc0,0x70,0x56,0x38, +0x71,0xbf,0xf5,0x88,0x23,0xe4,0x8b,0x58,0x58,0xc6,0x08,0x22, +0xa1,0x03,0x97,0x30,0x52,0x4a,0x30,0x8a,0xe7,0xbb,0x10,0x6f, +0xa6,0xcd,0x42,0x85,0x07,0xa1,0x0b,0xda,0x21,0x8a,0xa2,0xe9, +0x68,0x84,0xb6,0x15,0x8a,0x45,0x98,0x28,0x02,0xe8,0x11,0xb5, +0x85,0xa9,0x48,0x2f,0x32,0x07,0xa6,0x81,0x11,0x10,0x09,0x1e, +0x6d,0x04,0x0f,0x49,0x0a,0x4c,0x1c,0x6b,0x4a,0x0c,0x2c,0x2d, +0x2d,0xad,0x5d,0xc0,0x0d,0x11,0x53,0xa5,0x74,0x2d,0x48,0x7f, +0xf0,0x98,0x91,0x87,0xc5,0xb0,0xda,0x0d,0xb0,0x55,0x95,0x17, +0x1b,0xdc,0x65,0xd2,0xfe,0x43,0x07,0xd7,0x90,0x94,0xd5,0x97, +0x4b,0xf4,0x82,0x65,0x4a,0xe6,0xe0,0x02,0x7f,0x22,0xd3,0xcf, +0x72,0x73,0x75,0x02,0x54,0x64,0xa3,0x11,0x7c,0x7e,0x6d,0xbd, +0x50,0x88,0xec,0x02,0x82,0xbc,0xda,0x79,0xba,0x10,0x91,0x0d, +0xe0,0x4d,0xf3,0xda,0x68,0x04,0x6b,0xcd,0xd4,0x2d,0x50,0x05, +0xf8,0xf2,0x80,0xd1,0xe9,0xf9,0xf8,0xed,0x85,0x87,0x10,0x0d, +0x01,0x23,0xd3,0x3b,0x6a,0x83,0xb6,0x07,0xe0,0xcf,0x65,0x20, +0xe4,0x46,0x97,0x15,0xb6,0x44,0x08,0x52,0x9a,0xf7,0x1c,0x4a, +0x35,0x88,0x08,0xfb,0x93,0x15,0x83,0xae,0x5f,0x37,0xb6,0x67, +0x2c,0x88,0xaf,0x5c,0x5f,0xf2,0xf8,0x6b,0x0f,0x74,0xef,0x61, +0x3f,0xce,0xe1,0xc8,0x62,0xc9,0x6b,0xb6,0x84,0x35,0xbf,0x16, +0x6d,0x74,0x77,0xe0,0x8a,0xde,0x4a,0xe0,0x5c,0xef,0xf7,0x42, +0x90,0x28,0x22,0xb2,0x14,0x7e,0xc9,0x5b,0xb3,0x95,0xb6,0x1b, +0x1f,0x04,0x20,0x88,0xd0,0x5c,0xf7,0xd2,0x9a,0x6d,0xfd,0x47, +0x43,0x96,0x04,0x3e,0xf5,0xf3,0x92,0xe1,0x30,0x95,0xa9,0x31, +0xc4,0x6b,0x8f,0xe4,0x99,0xbe,0x3d,0x3e,0xa1,0x4d,0x4f,0x63, +0x26,0xd0,0x5b,0xff,0x45,0xc7,0x58,0x08,0x9d,0xbe,0x54,0x6b, +0x14,0xb2,0x68,0x3e,0xc7,0x0f,0x3d,0xc6,0x4a,0x9a,0x09,0xa7, +0xb4,0x16,0x6a,0xbc,0x4f,0x88,0xda,0x81,0x73,0xfb,0xd4,0x5b, +0xa7,0x4e,0x5c,0x3d,0x78,0x13,0xa7,0xbd,0x9f,0x02,0x31,0xaa, +0x1f,0x39,0x9e,0xa2,0xb5,0x13,0xa7,0xd0,0x38,0xc8,0x10,0xe5, +0x70,0xa8,0x34,0xf2,0xff,0xba,0xbe,0x1d,0xb8,0x0b,0xf2,0xba, +0xe6,0xeb,0x68,0xe9,0x1c,0xdb,0xed,0xa2,0x3d,0xeb,0xd2,0xa2, +0xe4,0x62,0x9a,0x85,0xcd,0xdb,0x50,0x61,0x1c,0x00,0xd8,0x00, +0x84,0xfb,0xff,0x57,0x6b,0x6f,0x7b,0xc4,0x21,0xc3,0xab,0xb0, +0xd7,0x24,0x4e,0xd5,0x45,0xed,0x12,0xcb,0xb0,0x53,0x4e,0xf2, +0x42,0xbd,0x3d,0x69,0x17,0xe4,0x30,0x06,0xee,0xcd,0xf2,0x5a, +0xa5,0x71,0xdc,0xaf,0x01,0x2c,0xe5,0xa2,0xee,0x61,0x09,0x8c, +0xd2,0xff,0x05,0x00,0x00,0xff,0xff,0x09,0x6f,0xc0,0x81,0x67, +0xfd,0x08,0x00, })) if err != nil { diff --git a/mod/dashboard/resources/views-browser.html.go b/mod/dashboard/resources/views-browser.html.go index 42b8095f340..a246ebf1a6f 100644 --- a/mod/dashboard/resources/views-browser.html.go +++ b/mod/dashboard/resources/views-browser.html.go @@ -10,147 +10,147 @@ import ( func views_browser_html() []byte { gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ 0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0xc4,0x58, -0x7b,0x6f,0xa3,0x48,0x12,0xff,0x7b,0xee,0x53,0xf4,0x70,0xd2, -0x24,0x91,0x00,0xf3,0x6a,0xc0,0x19,0xdb,0xa3,0x9b,0xdc,0x48, -0x39,0x29,0x73,0x73,0xba,0x1d,0x45,0x5a,0xad,0x46,0x51,0x1b, -0x3a,0x86,0x0d,0x06,0x44,0xb7,0xed,0x78,0xb3,0xf9,0xee,0x5b, -0x45,0x63,0x07,0xf0,0x23,0xc9,0xee,0xec,0x6e,0x22,0x9b,0x7e, -0x54,0x57,0xfd,0xaa,0xba,0xba,0xeb,0x67,0x46,0x71,0xba,0x24, -0x51,0xc6,0x84,0x18,0x6b,0x5c,0x46,0xb1,0x11,0x15,0xb9,0x64, -0x69,0xce,0x2b,0x52,0x77,0xa7,0x55,0xb1,0x12,0xd0,0x79,0x78, -0x88,0x8a,0x6c,0x31,0xcf,0xc5,0xe3,0x23,0xb4,0xcb,0x8a,0x2f, -0x53,0xbe,0xaa,0xdb,0x82,0x2d,0xf9,0xe3,0xa3,0x36,0xf9,0x07, -0x81,0xbf,0xd1,0x5b,0xc3,0x50,0x8d,0xbe,0xde,0xb2,0x28,0x8b, -0xe5,0x46,0x6b,0xd3,0x31,0x78,0x55,0x15,0x55,0xb3,0xf4,0xd8, -0x2a,0x23,0x2f,0x64,0x94,0x68,0x93,0xd1,0x00,0x04,0x5e,0x20, -0x8e,0x4e,0xf0,0x5c,0xb6,0x34,0xe3,0xdf,0x17,0x98,0x5a,0x55, -0xa9,0xe4,0x44,0x26,0xa9,0x20,0x4b,0x96,0x2d,0xf8,0x87,0x27, -0x65,0x2f,0xd7,0x7d,0x9b,0x56,0xf3,0x9e,0xee,0xd1,0x74,0x21, -0x65,0x91,0x77,0x56,0x34,0x43,0xad,0xb6,0x21,0xe6,0x2c,0xcb, -0x3a,0x23,0x65,0x95,0xce,0x59,0xb5,0x26,0x9b,0xe0,0x2b,0xdd, -0x5b,0xa8,0xa3,0x81,0x92,0x9b,0xec,0xc3,0xd9,0x6a,0x1a,0xc6, -0xe4,0xd5,0x71,0x27,0x69,0xdc,0x08,0xe1,0x1e,0xfe,0xc5,0x9b, -0xf1,0x09,0xad,0x9d,0x3f,0xe3,0xd5,0xef,0xf3,0x46,0xe5,0xec, -0xdf,0xe2,0x0f,0x79,0xad,0x43,0x09,0x67,0x31,0xf8,0x23,0x8a, -0x2c,0x8d,0xdb,0x58,0x59,0x37,0x93,0x58,0x74,0xa7,0x91,0x7c, -0x66,0x44,0x59,0x1a,0xdd,0x8d,0x35,0xec,0x9f,0x9e,0x35,0x23, -0xb5,0xd8,0xc3,0x2d,0xcb,0x04,0x3f,0x3f,0xa9,0xc5,0xe3,0x54, -0xb0,0x69,0xc6,0xe3,0x93,0xc7,0x9f,0x78,0x8e,0xad,0x1b,0x5c, -0xf0,0xad,0x9f,0xb2,0x62,0x39,0x23,0xe0,0x90,0x48,0x8b,0x7c, -0xac,0xd9,0xa6,0xad,0x91,0xfb,0x79,0x96,0x83,0xb2,0x44,0xca, -0xf2,0x7c,0x30,0x58,0xad,0x56,0xe6,0xca,0x35,0x8b,0x6a,0x36, -0x70,0x2c,0xcb,0x1a,0x80,0x7c,0x23,0x72,0x7e,0x9f,0xa5,0xf9, -0xdd,0x3e,0x41,0x7b,0x38,0x1c,0x0e,0xea,0x59,0x10,0x1d,0x6b, -0x56,0x79,0xaf,0x91,0xb5,0x7a,0x76,0x8c,0xc3,0x0d,0x02,0xd7, -0xca,0x92,0xff,0x4b,0x94,0x3c,0x92,0xff,0x67,0x32,0x2d,0xc6, -0xda,0xfd,0xe7,0x34,0xff,0x11,0x3e,0x1a,0xc1,0xdb,0xe5,0x63, -0x81,0x0a,0x88,0x45,0x02,0xd7,0x74,0xa9,0x4f,0x7c,0x00,0xa8, -0xdc,0xa9,0xe3,0x31,0xab,0x8a,0x45,0x0e,0x5b,0x9e,0xf3,0x15, -0xe9,0x49,0x01,0xc6,0x73,0x51,0xb2,0x88,0x8f,0xb5,0x8d,0x9d, -0xbe,0xef,0x25,0x93,0x09,0x81,0xd5,0x9f,0xa9,0xe9,0x04,0xba, -0xeb,0x9a,0x8e,0xe3,0x67,0x8e,0x63,0x7a,0x4e,0xa8,0xab,0x47, -0x64,0x9b,0xd4,0x77,0x74,0xf5,0xed,0x99,0xd6,0x90,0x36,0x6d, -0x6a,0xfa,0x34,0xd0,0x2d,0x35,0x6f,0xb4,0xa4,0x0c,0x25,0x65, -0x19,0xb5,0xc4,0x95,0x1d,0x98,0x01,0xa8,0xf6,0x4c,0xcf,0x76, -0x13,0x2f,0x34,0xa9,0xed,0x75,0x30,0x44,0x8e,0xe9,0x58,0x43, -0xdd,0xd2,0x3d,0x50,0x12,0x0c,0x6d,0x78,0x7a,0x42,0x35,0x0d, -0x0f,0xff,0x2f,0x51,0x83,0x37,0xcc,0x6c,0xd0,0x67,0x81,0x10, -0x35,0x69,0xe8,0xd4,0x66,0xdd,0xda,0xac,0x5d,0x9b,0xa5,0xb5, -0x59,0x4f,0xb7,0x4c,0x0b,0x84,0x6a,0xd3,0x17,0xae,0x63,0xd2, -0xc0,0x07,0xd4,0x2e,0xa8,0x75,0x41,0x8a,0x82,0x0b,0xba,0x6b, -0x99,0x14,0x9c,0xed,0xa1,0x00,0x55,0x96,0xe3,0x02,0x6a,0xc7, -0xb4,0x3c,0x1f,0xd4,0xb8,0x43,0x68,0x86,0x20,0x08,0x29,0xe1, -0x0f,0xaf,0x30,0x40,0x8e,0xee,0x04,0x60,0x2a,0x88,0x0c,0xcb, -0x0c,0xa8,0xad,0xe3,0x37,0xac,0xb3,0x03,0x17,0x84,0x02,0x3f, -0x6c,0xda,0xb8,0x6c,0x78,0x81,0x70,0x42,0xb4,0xea,0xd1,0x10, -0x20,0x80,0x69,0x80,0xe3,0x01,0x9c,0x56,0xa8,0x7f,0xd1,0x06, -0xbd,0x0d,0xc1,0xec,0x6a,0x5f,0x72,0xec,0xe0,0x61,0x60,0x31, -0x9e,0x94,0xe7,0x52,0xf9,0xe9,0xb0,0x80,0x3c,0x9e,0x95,0x3f, -0x37,0xb7,0xdf,0xbc,0x79,0xf3,0xaa,0x94,0x86,0x0d,0x7a,0x41, -0x4a,0x3f,0x49,0xed,0x4f,0x69,0xb0,0xfa,0x94,0xc9,0x2e,0xc6, -0xda,0xd6,0xc3,0xa5,0x0d,0x3b,0x1a,0x86,0x4b,0xc7,0x5d,0xda, -0xb0,0x51,0x6e,0x10,0x59,0xba,0x4a,0xb5,0x26,0xcd,0xf0,0x7f, -0x5f,0xf2,0x5d,0x7b,0x90,0x02,0x7e,0x70,0x6d,0x0f,0x4d,0xdb, -0x1a,0x5e,0x87,0x11,0xe6,0x04,0x48,0xb5,0x73,0x12,0x4d,0x5e, -0xb8,0xbe,0xe9,0xda,0x75,0x46,0x29,0x93,0xb4,0xd6,0xb0,0xb1, -0x5f,0xef,0x6d,0x07,0x99,0xed,0xa3,0x42,0x75,0x14,0x9c,0x44, -0xc1,0x4b,0x1c,0x37,0x69,0xe0,0xed,0x81,0xb2,0xcf,0xf4,0x25, -0xb5,0x20,0xcb,0xe8,0x25,0xa4,0xa2,0x63,0x07,0x97,0x4a,0x69, -0x64,0x34,0xab,0x0d,0x4f,0x6f,0x64,0x75,0x05,0xd2,0x76,0x94, -0x51,0xc7,0xf4,0x01,0x95,0xed,0x9a,0x1b,0x00,0x7a,0x07,0xce, -0x16,0xed,0xb1,0x04,0xec,0xdf,0xda,0x0d,0x2b,0x32,0xd0,0xc3, -0xfe,0xcd,0x92,0xe6,0xe5,0x42,0x12,0xb9,0x2e,0x61,0xab,0x24, -0xbf,0x97,0x75,0x2a,0xce,0x8b,0x98,0x67,0x6a,0xf1,0xff,0x70, -0x0d,0x8e,0x41,0x0d,0xe1,0xd5,0x58,0x13,0xeb,0x3c,0xba,0x2a, -0x22,0x4c,0x97,0x1c,0xf3,0x54,0xb2,0x69,0x9a,0xc7,0x1c,0x12, -0x25,0x0c,0x43,0x8d,0x0c,0xf6,0x56,0xfe,0xba,0xfb,0x5d,0x28, -0x07,0x96,0xfe,0xf6,0x69,0xc1,0xfe,0xbf,0x99,0x64,0x00,0x65, -0xf2,0x03,0xb4,0xbb,0x04,0xe4,0x58,0x29,0x9b,0x16,0xf1,0xfa, -0x58,0xb9,0xcd,0x52,0xd1,0x2f,0x9a,0x23,0x89,0xb9,0x4f,0x22, -0x9e,0x65,0x25,0x1c,0xd3,0x34,0x9f,0xc1,0xf1,0xd0,0xea,0x3e, -0x26,0x7b,0xd3,0x57,0xfb,0x23,0xb1,0x54,0xd6,0x4d,0xb5,0x32, -0xee,0x28,0xcf,0xd9,0x9c,0x37,0xc5,0x54,0x9b,0xfc,0x17,0x3a, -0xa3,0x81,0x3c,0x2c,0x2d,0x65,0xb6,0x15,0xfe,0xfa,0xf5,0xea, -0xa8,0x2c,0x8b,0x70,0x67,0xc4,0x56,0xfe,0x5d,0x3e,0x15,0xe5, -0x76,0x05,0x34,0xb6,0xb8,0x46,0x12,0x43,0xd0,0x52,0x54,0x61, -0x5c,0x2b,0x5e,0x72,0x26,0xc7,0xda,0x1d,0x5f,0x93,0x34,0x27, -0x18,0x05,0xf2,0x2b,0x29,0x2a,0x50,0xf6,0x71,0x7d,0x7e,0x02, -0xc3,0x27,0xda,0x76,0x4d,0x03,0x60,0x02,0x57,0x5e,0xab,0xb6, -0xcb,0x6a,0x01,0xa5,0x3d,0x4e,0x2b,0xb8,0x57,0x8a,0x6a,0x0d, -0x65,0x1d,0x56,0x99,0xd0,0xff,0xa6,0x91,0xa4,0xe2,0xb7,0x63, -0xed,0x9f,0x83,0xa5,0x3d,0x80,0x41,0xf1,0xf0,0x80,0x53,0xf0, -0x01,0x4a,0x4e,0x92,0x74,0x96,0x64,0xf0,0x91,0x93,0xd6,0x30, -0x66,0x76,0xc7,0xe1,0x8d,0xd3,0x60,0x50,0xac,0x52,0xa0,0x42, -0x04,0x2f,0xd1,0xb7,0x6f,0x71,0x05,0xbf,0x2f,0xd3,0xaa,0x4e, -0x4d,0xad,0x1f,0x40,0x05,0xba,0x71,0x15,0xf7,0x7a,0xbb,0xde, -0x58,0x25,0x1c,0x34,0x20,0x6a,0x60,0x55,0x32,0x9d,0x73,0x52, -0xf1,0x0c,0xb4,0x2c,0x39,0x89,0x99,0xe4,0x38,0x02,0x5e,0x3d, -0x74,0x0d,0x98,0x62,0x31,0x15,0xb2,0x82,0x3d,0x3f,0xb5,0x74, -0xd2,0x9b,0x03,0xd3,0xf2,0x3f,0x78,0x2a,0xbe,0xdc,0x9e,0x9e, -0x18,0x27,0x67,0x67,0xf8,0x8b,0x03,0xbc,0x00,0x4d,0x1b,0xde, -0x76,0x08,0x4c,0xcc,0x6f,0xd9,0x22,0x93,0x3b,0xfb,0x9f,0x17, -0x39,0xc0,0x7b,0x37,0x8f,0x99,0x48,0xde,0xb7,0x75,0xec,0x0b, -0x4e,0xa7,0xbf,0x37,0xbb,0x9b,0x34,0xd1,0x76,0x24,0xf7,0x4a, -0xc3,0x6d,0xc0,0xa5,0x3a,0x77,0x3b,0x31,0x87,0x7d,0xdd,0xaf, -0x65,0x5b,0xe9,0xfa,0x91,0xae,0xb9,0x5f,0xfb,0x10,0x2b,0xf5, -0x37,0xa0,0x0d,0x6f,0x94,0x6e,0x61,0x44,0x9a,0x7c,0xc5,0xd6, -0xbc,0xba,0xf9,0xde,0x84,0xef,0x10,0xe6,0x5e,0xad,0x7c,0x1d, -0xf7,0x83,0xbb,0xda,0x73,0x9f,0x2d,0x94,0x4f,0x52,0x07,0x0b, -0xe5,0xde,0x68,0x6e,0x4b,0x94,0x37,0x84,0x62,0x80,0x55,0x82, -0x5a,0x34,0x33,0x7c,0xd3,0x72,0xa9,0x5e,0x7f,0x5f,0x41,0xb9, -0xb1,0xa1,0x68,0xb8,0xd4,0x0c,0x2d,0xb7,0x33,0x15,0x75,0xc8, -0x5f,0x9b,0x28,0x5a,0x8a,0x24,0x3e,0x47,0x21,0x0f,0xc1,0xca, -0x40,0x3d,0x45,0x4b,0x34,0x83,0x5a,0xe5,0xb8,0x50,0x18,0xeb, -0xc7,0xd3,0xf8,0x73,0xe4,0xf3,0x82,0xd6,0x9e,0xe8,0x50,0x6b, -0x87,0x9e,0xab,0x43,0xe5,0x1c,0x02,0x89,0x6b,0x7a,0x6d,0x5f, -0x37,0xe5,0xef,0x78,0x74,0x1c,0x28,0xd9,0xc0,0x65,0x91,0x18, -0xf8,0x4e,0xd6,0x8e,0x8e,0x07,0xac,0x3b,0xa0,0xba,0x07,0x6c, -0xcf,0x6f,0x4f,0xfc,0x51,0xf6,0x7c,0x30,0x34,0xc6,0x36,0x06, -0x57,0x50,0xda,0x83,0xd0,0x45,0x54,0x94,0xd2,0xd6,0x44,0xb3, -0x2f,0x46,0x4b,0x6d,0xd3,0x6e,0x8c,0x5f,0xe0,0x12,0x1b,0x10, -0x51,0xe4,0x3b,0xfa,0xa6,0x17,0x98,0xbe,0xde,0xf1,0xf4,0x68, -0x70,0x14,0x5f,0x38,0x34,0x7b,0xe4,0x42,0xdc,0x5e,0x35,0xbb, -0x2a,0xf7,0xcd,0xec,0x8e,0x76,0xcb,0xd5,0x40,0x56,0x9b,0x4a, -0xb4,0x2d,0x3f,0xd0,0xc6,0xb3,0x72,0x98,0x3a,0xec,0xfc,0xb6, -0x55,0x2f,0x74,0x76,0xca,0x33,0xf0,0x17,0x56,0x71,0x46,0xca, -0x0c,0x4e,0x54,0x52,0x64,0x31,0x32,0x96,0x4f,0x48,0x5c,0x08, -0xc3,0x3b,0x9a,0x60,0xdd,0x25,0x6c,0x0a,0x3f,0x8c,0x09,0xcb, -0x63,0x02,0xc5,0x50,0xbd,0x57,0x21,0x09,0xaf,0x78,0x9b,0xfa, -0x08,0xb8,0xd8,0x33,0x7e,0x8d,0x73,0x3b,0x1c,0x07,0x2f,0xae, -0x84,0xe5,0x33,0x38,0xb3,0x22,0x29,0x56,0x48,0x3b,0x90,0x7e, -0xf4,0x7e,0x20,0x6c,0xc0,0xf4,0xc6,0xfb,0xae,0xf0,0x79,0x29, -0xd7,0xbd,0xc5,0x87,0x05,0x8d,0x39,0x17,0x82,0xcd,0x60,0x5f, -0xb6,0x6f,0xb5,0x6e,0x9a,0x21,0x2c,0x95,0x9d,0xa8,0xed,0x09, -0xe4,0x5e,0xc5,0x71,0xca,0xb2,0x62,0xf6,0x12,0x08,0x4a,0xf2, -0x09,0xc3,0xce,0x0a,0xfc,0xc3,0x70,0xd4,0xc1,0x05,0x26,0x01, -0xd7,0x3b,0xdb,0xbc,0xbf,0x52,0x21,0xfb,0xb0,0x6b,0x65,0x17, -0xe2,0x31,0xe3,0x8a,0xde,0x89,0x03,0xc6,0x5f,0x48,0x32,0x1b, -0x4a,0xa9,0x28,0x23,0xb9,0xa8,0x91,0x89,0xdd,0x77,0x57,0x1d, -0xcd,0xac,0xe1,0x2e,0x3f,0xb3,0x25,0x13,0x51,0x95,0x96,0xf2, -0x7c,0x59,0xa4,0xf1,0xa9,0x75,0xf6,0x5e,0x9b,0x5c,0xb0,0x1c, -0x58,0x60,0x87,0x85,0x1f,0xdb,0x82,0x5e,0x7a,0xef,0xbc,0x7b, -0x69,0x1e,0xbf,0x05,0x00,0x00,0xff,0xff,0xd1,0x7f,0x20,0xf2, -0xee,0x14,0x00,0x00, +0x7b,0x6f,0xdb,0x46,0x12,0xff,0x3b,0xf7,0x29,0x36,0x3c,0x20, +0xb6,0x01,0x92,0xe2,0x6b,0x49,0xca,0x91,0x14,0x5c,0x7c,0x01, +0x7c,0x38,0xa7,0x29,0xda,0xc0,0x40,0x51,0x04,0xc1,0x8a,0x5c, +0x8b,0xac,0x29,0x92,0xe0,0xae,0x24,0xab,0xae,0xbf,0x7b,0x67, +0xb8,0x94,0x4c,0x52,0x0f,0xdb,0x6d,0xda,0xda,0x90,0xb8,0x8f, +0xd9,0x99,0xdf,0xcc,0xce,0xee,0xfc,0xc4,0x51,0x9c,0x2e,0x49, +0x94,0x31,0x21,0xc6,0x1a,0x97,0x51,0x6c,0x44,0x45,0x2e,0x59, +0x9a,0xf3,0x8a,0xd4,0xdd,0x69,0x55,0xac,0x04,0x74,0xee,0xef, +0xa3,0x22,0x5b,0xcc,0x73,0xf1,0xf0,0x00,0xed,0xb2,0xe2,0xcb, +0x94,0xaf,0xea,0xb6,0x60,0x4b,0xfe,0xf0,0xa0,0x4d,0xfe,0x45, +0xe0,0x6f,0xf4,0xda,0x30,0x54,0xa3,0xaf,0xb7,0x2c,0xca,0x62, +0xb9,0xd1,0xda,0x74,0x0c,0x5e,0x55,0x45,0xd5,0x2c,0x3d,0xb6, +0xca,0xc8,0x0b,0x19,0x25,0xda,0x64,0x34,0x00,0x81,0x67,0x88, +0xa3,0x13,0x3c,0x97,0x2d,0xcd,0xf8,0xf7,0x09,0xa6,0x56,0x55, +0x2a,0x39,0x91,0x49,0x2a,0xc8,0x92,0x65,0x0b,0xfe,0xee,0x51, +0xd9,0xf3,0x75,0xdf,0xa4,0xd5,0xbc,0xa7,0x7b,0x34,0x5d,0x48, +0x59,0xe4,0x9d,0x15,0xcd,0x50,0xab,0x6d,0x88,0x39,0xcb,0xb2, +0xce,0x48,0x59,0xa5,0x73,0x56,0xad,0xc9,0x26,0xf8,0x4a,0xf7, +0x16,0xea,0x68,0xa0,0xe4,0x26,0xfb,0x70,0xb6,0x9a,0x86,0x31, +0x79,0x71,0xdc,0x49,0x1a,0x37,0x42,0xb8,0x87,0x7f,0xf3,0x66, +0x7c,0x40,0x6b,0xe7,0x4f,0x78,0xf5,0xc7,0xbc,0x51,0x39,0xfb, +0x8f,0xf8,0x43,0x5e,0xea,0x50,0xc2,0x59,0x0c,0xfe,0x88,0x22, +0x4b,0xe3,0x36,0x56,0xd6,0xcd,0x24,0x16,0xdd,0x6a,0x24,0x9f, +0x19,0x51,0x96,0x46,0xb7,0x63,0x0d,0xfb,0xa7,0x67,0xcd,0x48, +0x2d,0x76,0x7f,0xc3,0x32,0xc1,0xcf,0x4f,0x6a,0xf1,0x38,0x15, +0x6c,0x9a,0xf1,0xf8,0xe4,0xe1,0x67,0x9e,0x63,0xeb,0x2b,0x2e, +0xf8,0xd2,0x4f,0x59,0xb1,0x9c,0x11,0x70,0x48,0xa4,0x45,0x3e, +0xd6,0x6c,0xd3,0xd6,0xc8,0xdd,0x3c,0xcb,0x41,0x59,0x22,0x65, +0x79,0x3e,0x18,0xac,0x56,0x2b,0x73,0xe5,0x9a,0x45,0x35,0x1b, +0x38,0x96,0x65,0x0d,0x40,0xbe,0x11,0x39,0xbf,0xcb,0xd2,0xfc, +0x76,0x9f,0xa0,0x3d,0x1c,0x0e,0x07,0xf5,0x2c,0x88,0x8e,0x35, +0xab,0xbc,0xd3,0xc8,0x5a,0x3d,0x3b,0xc6,0xe1,0x06,0x81,0x6b, +0x65,0xc9,0xff,0x23,0x4a,0x1e,0xc9,0x1f,0x98,0x4c,0x8b,0xb1, +0x76,0xf7,0x31,0xcd,0x7f,0x82,0x8f,0x46,0xf0,0x76,0x79,0x5f, +0xa0,0x02,0x62,0x91,0xc0,0x35,0x5d,0xea,0x13,0x1f,0x00,0x2a, +0x77,0xea,0x78,0xcc,0xaa,0x62,0x91,0xc3,0x96,0xe7,0x7c,0x45, +0x7a,0x52,0x80,0xf1,0x5c,0x94,0x2c,0xe2,0x63,0x6d,0x63,0xa7, +0xef,0x7b,0xc9,0x64,0x42,0x60,0xf5,0x47,0x6a,0x3a,0x81,0xee, +0xba,0xa6,0xe3,0xf8,0x99,0xe3,0x98,0x9e,0x13,0xea,0xea,0x11, +0xd9,0x26,0xf5,0x1d,0x5d,0x7d,0x7b,0xa6,0x35,0xa4,0x4d,0x9b, +0x9a,0x3e,0x0d,0x74,0x4b,0xcd,0x1b,0x2d,0x29,0x43,0x49,0x59, +0x46,0x2d,0x71,0x65,0x07,0x66,0x00,0xaa,0x3d,0xd3,0xb3,0xdd, +0xc4,0x0b,0x4d,0x6a,0x7b,0x1d,0x0c,0x91,0x63,0x3a,0xd6,0x50, +0xb7,0x74,0x0f,0x94,0x04,0x43,0x1b,0x9e,0x9e,0x50,0x4d,0xc3, +0xc3,0xff,0x4b,0xd4,0xe0,0x0d,0x33,0x1b,0xf4,0x59,0x20,0x44, +0x4d,0x1a,0x3a,0xb5,0x59,0xb7,0x36,0x6b,0xd7,0x66,0x69,0x6d, +0xd6,0xd3,0x2d,0xd3,0x02,0xa1,0xda,0xf4,0x85,0xeb,0x98,0x34, +0xf0,0x01,0xb5,0x0b,0x6a,0x5d,0x90,0xa2,0xe0,0x82,0xee,0x5a, +0x26,0x05,0x67,0x7b,0x28,0x40,0x95,0xe5,0xb8,0x80,0xda,0x31, +0x2d,0xcf,0x07,0x35,0xee,0x10,0x9a,0x21,0x08,0x42,0x4a,0xf8, +0xc3,0x2b,0x0c,0x90,0xa3,0x3b,0x01,0x98,0x0a,0x22,0xc3,0x32, +0x03,0x6a,0xeb,0xf8,0x0d,0xeb,0xec,0xc0,0x05,0xa1,0xc0,0x0f, +0x9b,0x36,0x2e,0x1b,0x5e,0x20,0x9c,0x10,0xad,0x7a,0x34,0x04, +0x08,0x60,0x1a,0xe0,0x78,0x00,0xa7,0x15,0xea,0x5f,0xb5,0x41, +0x6f,0x43,0x30,0xbb,0xda,0x97,0x1c,0x3b,0x78,0x18,0x58,0x8c, +0x27,0xe5,0xa9,0x54,0x7e,0x3c,0x2c,0x20,0x8f,0x67,0xe5,0xaf, +0xcd,0xed,0x57,0xaf,0x5e,0xbd,0x28,0xa5,0x61,0x83,0x9e,0x91, +0xd2,0x8f,0x52,0xfb,0x53,0x1a,0xac,0x3e,0x66,0xb2,0x8b,0xb1, +0xb6,0xf5,0x70,0x69,0xc3,0x8e,0x86,0xe1,0xd2,0x71,0x97,0x36, +0x6c,0x94,0x1b,0x44,0x96,0xae,0x52,0xad,0x49,0x33,0xfc,0xdf, +0x97,0x7c,0xd7,0x1e,0xa4,0x80,0x1f,0x5c,0xdb,0x43,0xd3,0xb6, +0x86,0xd7,0x61,0x84,0x39,0x01,0x52,0xed,0x9c,0x44,0x93,0x17, +0xae,0x6f,0xba,0x76,0x9d,0x51,0xca,0x24,0xad,0x35,0x6c,0xec, +0xd7,0x7b,0xdb,0x41,0x66,0xfb,0xa8,0x50,0x1d,0x05,0x27,0x51, +0xf0,0x12,0xc7,0x4d,0x1a,0x78,0x7b,0xa0,0xec,0x33,0x7d,0x49, +0x2d,0xc8,0x32,0x7a,0x09,0xa9,0xe8,0xd8,0xc1,0xa5,0x52,0x1a, +0x19,0xcd,0x6a,0xc3,0xd3,0x1b,0x59,0x5d,0x81,0xb4,0x1d,0x65, +0xd4,0x31,0x7d,0x40,0x65,0xbb,0xe6,0x06,0x80,0xde,0x81,0xb3, +0x45,0x7b,0x2c,0x01,0xfb,0xb7,0x76,0xc3,0x8a,0x0c,0xf4,0xb0, +0x7f,0xb3,0xa4,0x79,0xb9,0x90,0x44,0xae,0x4b,0xd8,0x2a,0xc9, +0xef,0x64,0x9d,0x8a,0xf3,0x22,0xe6,0x99,0x5a,0xfc,0x3d,0xae, +0xc1,0x31,0xa8,0x21,0xbc,0x1a,0x6b,0x62,0x9d,0x47,0x57,0x45, +0x84,0xe9,0x92,0x63,0x9e,0x4a,0x36,0x4d,0xf3,0x98,0x43,0xa2, +0x84,0x61,0xa8,0x91,0xc1,0xde,0xca,0x5f,0x77,0xbf,0x09,0xe5, +0xc0,0xd2,0xdf,0x3e,0x2d,0xd8,0xff,0x2f,0x93,0x0c,0xa0,0x4c, +0x7e,0x84,0x76,0x97,0x80,0x1c,0x2b,0x65,0xd3,0x22,0x5e,0x1f, +0x2b,0xb7,0x59,0x2a,0xfa,0x45,0x73,0x24,0x31,0xf7,0x49,0xc4, +0xb3,0xac,0x84,0x63,0x9a,0xe6,0x33,0x38,0x1e,0x5a,0xdd,0xc7, +0x64,0x6f,0xfa,0x6a,0x7f,0x24,0x96,0xca,0xba,0xa9,0x56,0xc6, +0x1d,0xe5,0x39,0x9b,0xf3,0xa6,0x98,0x6a,0x93,0xef,0xa0,0x33, +0x1a,0xc8,0xc3,0xd2,0x52,0x66,0x5b,0xe1,0xcf,0x9f,0xaf,0x8e, +0xca,0xb2,0x08,0x77,0x46,0x6c,0xe5,0xdf,0xe4,0x53,0x51,0x6e, +0x57,0x40,0x63,0x8b,0x6b,0x24,0x31,0x04,0x2d,0x45,0x15,0xc6, +0xb5,0xe2,0x25,0x67,0x72,0xac,0xdd,0xf2,0x35,0x49,0x73,0x82, +0x51,0x20,0xbf,0x91,0xa2,0x02,0x65,0xef,0xd7,0xe7,0x27,0x30, +0x7c,0xa2,0x6d,0xd7,0x34,0x00,0x26,0x70,0xe5,0xb5,0x6a,0xbb, +0xac,0x16,0x50,0xda,0xe3,0xb4,0x82,0x7b,0xa5,0xa8,0xd6,0x50, +0xd6,0x61,0x95,0x09,0xfd,0x2f,0x1a,0x49,0x2a,0x7e,0x33,0xd6, +0xfe,0x3d,0x58,0xda,0x03,0x18,0x14,0xf7,0xf7,0x38,0x05,0x1f, +0xa0,0xe4,0x24,0x49,0x67,0x49,0x06,0x1f,0x39,0x69,0x0d,0x63, +0x66,0x77,0x1c,0xde,0x38,0x0d,0x06,0xc5,0x2a,0x05,0x2a,0x44, +0xf0,0x12,0x7d,0xfd,0x1a,0x57,0xf0,0xbb,0x32,0xad,0xea,0xd4, +0xd4,0xfa,0x01,0x54,0xa0,0x1b,0x57,0x71,0xaf,0xb7,0xeb,0x8d, +0x55,0xc2,0x41,0x03,0xa2,0x06,0x56,0x25,0xd3,0x39,0x27,0x15, +0xcf,0x40,0xcb,0x92,0x93,0x98,0x49,0x8e,0x23,0xe0,0xd5,0x7d, +0xd7,0x80,0x29,0x16,0x53,0x21,0x2b,0xd8,0xf3,0x53,0x4b,0x27, +0xbd,0x39,0x30,0x2d,0xff,0x87,0xa7,0xe2,0xd3,0xcd,0xe9,0x89, +0x71,0x72,0x76,0x86,0xbf,0x38,0xc0,0x0b,0xd0,0xb4,0xe1,0x6d, +0x87,0xc0,0xc4,0xfc,0x86,0x2d,0x32,0xb9,0xb3,0xff,0x79,0x91, +0x03,0xbc,0x37,0xf3,0x98,0x89,0xe4,0x6d,0x5b,0xc7,0xbe,0xe0, +0x74,0xfa,0x7b,0xb3,0xbb,0x49,0x13,0x6d,0x47,0x72,0xaf,0x34, +0xdc,0x06,0x5c,0xaa,0x73,0xb7,0x13,0x73,0xd8,0xd7,0xfd,0x5a, +0xb6,0x95,0xae,0x1f,0xe9,0x9a,0xfb,0xb5,0x0f,0xb1,0x52,0xff, +0x7f,0xbe,0xc6,0x0b,0xa5,0x5b,0x17,0x91,0x25,0x5f,0xb1,0x35, +0xaf,0xbe,0x7e,0x6b,0xbe,0x77,0x08,0x72,0xaf,0x54,0xbe,0x8c, +0xfa,0xc1,0x55,0xed,0xb9,0x4f,0xd6,0xc9,0x47,0xa9,0x83,0x75, +0x72,0x6f,0x30,0xb7,0x15,0xca,0x1b,0x42,0x2d,0xc0,0x22,0x41, +0x2d,0x9a,0x19,0xbe,0x69,0xb9,0x54,0xaf,0xbf,0xaf,0xa0,0xda, +0xd8,0x50,0x33,0x5c,0x6a,0x86,0x96,0xdb,0x99,0x8a,0x3a,0xdc, +0xaf,0xcd,0x13,0x2d,0xc5,0x11,0x9f,0x62,0x90,0x87,0x60,0x65, +0xa0,0x9e,0xa2,0x25,0x9a,0x41,0xa9,0x72,0x5c,0xa8,0x8b,0xf5, +0xe3,0x71,0xfc,0x29,0xee,0x79,0x41,0x6b,0x4f,0x74,0x28,0xb5, +0x43,0xcf,0xd5,0xa1,0x70,0x0e,0x81,0xc3,0x35,0xbd,0xb6,0xaf, +0x9b,0xea,0x77,0x3c,0x3a,0x0e,0x54,0x6c,0xa0,0xb2,0xc8,0x0b, +0x7c,0x27,0x6b,0x47,0xc7,0x03,0xd2,0x1d,0x50,0xdd,0x03,0xb2, +0xe7,0xb7,0x27,0xfe,0x2c,0x79,0x3e,0x18,0x1a,0x63,0x1b,0x83, +0x2b,0xa8,0xec,0x41,0xe8,0x22,0x2a,0x4a,0x69,0x6b,0xa2,0xd9, +0x17,0xa3,0xa5,0xb6,0x69,0x37,0xc6,0x2f,0x70,0x89,0x0d,0x88, +0x28,0xd2,0x1d,0x7d,0xd3,0x0b,0x4c,0x5f,0xef,0x78,0x7a,0x34, +0x38,0x8a,0x2e,0x1c,0x9a,0x3d,0x72,0x1f,0x6e,0x6f,0x9a,0x5d, +0x95,0xfb,0x66,0x76,0x47,0xbb,0xd5,0x6a,0x20,0xab,0x4d,0x21, +0xda,0x56,0x1f,0x68,0xe3,0x59,0x39,0xcc,0x1c,0x76,0x7e,0xda, +0xaa,0xf7,0x39,0x3b,0xd5,0x19,0xe8,0x0b,0xab,0x38,0x23,0x65, +0x06,0x27,0x2a,0x29,0xb2,0x18,0x09,0xcb,0x07,0xe4,0x2d,0x84, +0xe1,0x15,0x4d,0xb0,0xec,0x12,0x36,0x85,0xdf,0xc5,0x84,0xe5, +0x31,0x81,0x5a,0xa8,0x5e,0xab,0x90,0x84,0x57,0xbc,0xcd,0x7c, +0x04,0xdc,0xeb,0x19,0xbf,0xc6,0xb9,0x1d,0x8a,0x83,0xf7,0x56, +0xc2,0xf2,0x19,0x9c,0x59,0x91,0x14,0x2b,0x64,0x1d,0xc8,0x3e, +0x7a,0xbf,0x0f,0x36,0x60,0x7a,0xe3,0x7d,0x57,0xf8,0xbc,0x94, +0xeb,0xde,0xe2,0xc3,0x82,0xc6,0x9c,0x0b,0xc1,0x66,0xb0,0x2f, +0xdb,0x97,0x5a,0x5f,0x9b,0x21,0xac,0x94,0x9d,0xa8,0xed,0x09, +0xe4,0x5e,0xc5,0x71,0xca,0xb2,0x62,0xf6,0x1c,0x08,0x4a,0xf2, +0x11,0xc3,0xce,0x0a,0xfc,0xc3,0x70,0xd4,0xc1,0x05,0x22,0x01, +0xb7,0x3b,0xdb,0xbc,0xbe,0x52,0x21,0x7b,0xb7,0x6b,0x65,0x17, +0xe2,0x31,0xe3,0x8a,0xdd,0x89,0x03,0xc6,0x9f,0xc9,0x31,0x1b, +0x46,0xa9,0x18,0x23,0xb9,0xa8,0x91,0x89,0xdd,0x57,0x57,0x1d, +0xcd,0xac,0xa1,0x2e,0xbf,0xb0,0x25,0x13,0x51,0x95,0x96,0xf2, +0x7c,0x59,0xa4,0xf1,0xa9,0x75,0xf6,0x56,0x9b,0x5c,0xb0,0x1c, +0x48,0x60,0x87,0x84,0x1f,0xdb,0x82,0x5e,0x7a,0xef,0xbc,0x7a, +0x69,0x1e,0xbf,0x07,0x00,0x00,0xff,0xff,0x7c,0x65,0x3f,0x3c, +0xed,0x14,0x00,0x00, })) if err != nil { From f62095428f1169c9d589abd50b96902d7cea2e87 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Thu, 10 Oct 2013 23:15:48 -0700 Subject: [PATCH 211/247] Update README.md --- README.md | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/README.md b/README.md index 247537a298e..71a81034449 100644 --- a/README.md +++ b/README.md @@ -593,6 +593,26 @@ Odd numbers are good because if you have 8 machines the majority will be 5 and i The result is that an 8 machine cluster can tolerate 3 machine failures and a 9 machine cluster can tolerate 4 nodes failures. And in the best case when all 9 machines are responding the cluster will perform at the speed of the fastest 5 nodes. +### Why SSLv3 alert handshake failure when using SSL client auth? +The `TLS` pacakge of `golang` checkes the key usage of certificate public key before using it. To use the certificate public key to do client auth, we need to add `clientAuth` to `Extended Key Usage` when creates the certificate public key. + +Here is how to do it: + +Add the following section to my openssl.cnf: + +``` +[ ssl_client ] +... + extendedKeyUsage = clientAuth +... +``` + +When create the cert be sure to reference it in the -extensions flag: + +``` +openssl ca -config openssl.cnf -policy policy_anything -extensions ssl_client -out certs/node.crt -infiles node.csr +``` + ## Project Details ### Versioning From b54a6a2ac81deb2759a01c0d585feb438089f8ad Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Thu, 10 Oct 2013 23:43:47 -0700 Subject: [PATCH 212/247] Update README.md --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 71a81034449..de8276dacd6 100644 --- a/README.md +++ b/README.md @@ -598,7 +598,7 @@ The `TLS` pacakge of `golang` checkes the key usage of certificate public key be Here is how to do it: -Add the following section to my openssl.cnf: +Add the following section to your openssl.cnf: ``` [ ssl_client ] @@ -607,7 +607,7 @@ Add the following section to my openssl.cnf: ... ``` -When create the cert be sure to reference it in the -extensions flag: +When creating the cert be sure to reference it in the -extensions flag: ``` openssl ca -config openssl.cnf -policy policy_anything -extensions ssl_client -out certs/node.crt -infiles node.csr From 8540249c4e45c3147f5d0843dc012fe83e6f5d5e Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Thu, 10 Oct 2013 23:44:59 -0700 Subject: [PATCH 213/247] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index de8276dacd6..fe8560db32d 100644 --- a/README.md +++ b/README.md @@ -594,7 +594,7 @@ The result is that an 8 machine cluster can tolerate 3 machine failures and a 9 And in the best case when all 9 machines are responding the cluster will perform at the speed of the fastest 5 nodes. ### Why SSLv3 alert handshake failure when using SSL client auth? -The `TLS` pacakge of `golang` checkes the key usage of certificate public key before using it. To use the certificate public key to do client auth, we need to add `clientAuth` to `Extended Key Usage` when creates the certificate public key. +The `TLS` pacakge of `golang` checks the key usage of certificate public key before using it. To use the certificate public key to do client auth, we need to add `clientAuth` to `Extended Key Usage` when creating the certificate public key. Here is how to do it: From b7084fda45a3acf9a8d631a6bc9e911fac74ae50 Mon Sep 17 00:00:00 2001 From: Rob Szumski Date: Fri, 11 Oct 2013 15:37:30 -0700 Subject: [PATCH 214/247] docs(dashboard): improve instructions --- mod/dashboard/README.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/mod/dashboard/README.md b/mod/dashboard/README.md index f360837861d..1b9da7f86f0 100644 --- a/mod/dashboard/README.md +++ b/mod/dashboard/README.md @@ -2,6 +2,8 @@ ## Developing +If you'd like to contribute to the etcd dashboard mod, follow these instructions. For contributing to the rest of etcd, see the contributing document in the root of the repository. + ### Install yeoman http://yeoman.io/ @@ -12,7 +14,7 @@ http://yeoman.io/ npm install ``` -### Install Bower +### Install Bower Components ``` bower install @@ -20,4 +22,6 @@ bower install ### View in Browser +Run etcd like you normally would and afterward browse to: + http://localhost:4001/etcd/mod/dashboard/ From cc9b5fca90d90a0658ad6e53f462cfd14e7d4df7 Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Sat, 12 Oct 2013 21:27:00 -0700 Subject: [PATCH 215/247] feat(README): add go-discover --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index fe8560db32d..377bb403dc9 100644 --- a/README.md +++ b/README.md @@ -574,6 +574,7 @@ See [CONTRIBUTING](https://github.com/coreos/etcd/blob/master/CONTRIBUTING.md) f - [binocarlos/yoda](https://github.com/binocarlos/yoda) - etcd + ZeroMQ - [calavera/active-proxy](https://github.com/calavera/active-proxy) - HTTP Proxy configured with etcd - [derekchiang/etcdplus](https://github.com/derekchiang/etcdplus) - A set of distributed synchronization primitives built upon etcd +- [go-discover](https://github.com/flynn/go-discover) - service discovery in Go - [gleicon/goreman](https://github.com/gleicon/goreman/tree/etcd) - Branch of the Go Foreman clone with etcd support - [garethr/hiera-etcd](https://github.com/garethr/hiera-etcd) - Puppet hiera backend using etcd - [mattn/etcd-vim](https://github.com/mattn/etcd-vim) - SET and GET keys from inside vim From 478118497cfcb9ca4cde2e837a0194d085d01002 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Adri=C3=A1n?= Date: Mon, 14 Oct 2013 19:14:14 +0200 Subject: [PATCH 216/247] A little lit bit easy to read I have only reordered to parameters to be according to the first instance executed. --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 377bb403dc9..4f8b521856c 100644 --- a/README.md +++ b/README.md @@ -416,8 +416,8 @@ A similar argument `-sl` is used to setup the listening address for the server p Let the join two more nodes to this cluster using the -C argument: ```sh -./etcd -c 127.0.0.1:4002 -s 127.0.0.1:7002 -C 127.0.0.1:7001 -d nodes/node2 -n node2 -./etcd -c 127.0.0.1:4003 -s 127.0.0.1:7003 -C 127.0.0.1:7001 -d nodes/node3 -n node3 +./etcd -s 127.0.0.1:7002 -c 127.0.0.1:4002 -C 127.0.0.1:7001 -d nodes/node2 -n node2 +./etcd -s 127.0.0.1:7003 -c 127.0.0.1:4003 -C 127.0.0.1:7001 -d nodes/node3 -n node3 ``` Get the machines in the cluster: From 40142cbacc668572c415416f37f4545d19eb0247 Mon Sep 17 00:00:00 2001 From: Rob Szumski Date: Mon, 14 Oct 2013 22:42:12 -0700 Subject: [PATCH 217/247] feat(dashboard): Notate leader in machine list --- mod/dashboard/app/scripts/controllers/stats.js | 18 +++++++++++++++--- mod/dashboard/app/styles/etcd-widgets.css | 6 ++++++ mod/dashboard/app/views/stats.html | 15 +++++++++------ 3 files changed, 30 insertions(+), 9 deletions(-) diff --git a/mod/dashboard/app/scripts/controllers/stats.js b/mod/dashboard/app/scripts/controllers/stats.js index e689147b093..9ce6ea159a2 100644 --- a/mod/dashboard/app/scripts/controllers/stats.js +++ b/mod/dashboard/app/scripts/controllers/stats.js @@ -23,10 +23,22 @@ angular.module('etcdStats', ['ngRoute', 'etcd']) function readStats() { EtcdV1.getStat('leader').get().success(function(data) { $scope.leaderStats = data; - $scope.followers = []; + $scope.leaderName = data.leader; + $scope.machines = []; + //hardcode leader stats + $scope.machines.push({ + latency: { + average: 0, + current: 0, + minimum: 0, + maximum: 0, + standardDeviation: 0 + }, + name: data.leader + }); $.each(data.followers, function(index, value) { value.name = index; - $scope.followers.push(value); + $scope.machines.push(value); }); drawGraph(); }); @@ -46,7 +58,7 @@ angular.module('etcdStats', ['ngRoute', 'etcd']) chart({ el: $scope.graphContainer, data: { - 'stats': $scope.followers + 'stats': $scope.machines } }).width(width).height(height).update(); }); diff --git a/mod/dashboard/app/styles/etcd-widgets.css b/mod/dashboard/app/styles/etcd-widgets.css index 1cccf39f338..fef7e82783e 100644 --- a/mod/dashboard/app/styles/etcd-widgets.css +++ b/mod/dashboard/app/styles/etcd-widgets.css @@ -624,6 +624,12 @@ body { background-color: #00DB24; } + .etcd-container.etcd-stats .etcd-list .etcd-machine-type { + color: #999; + padding-left: 3px; + font-size: 13px; + } + .etcd-container.etcd-stats .etcd-list .etcd-latency-value { display: inline-block; } diff --git a/mod/dashboard/app/views/stats.html b/mod/dashboard/app/views/stats.html index eb1a615477a..e99076681c0 100644 --- a/mod/dashboard/app/views/stats.html +++ b/mod/dashboard/app/views/stats.html @@ -20,23 +20,26 @@
-

Follower Latency

+

Machine Latency

-

Follower List

+

Machine List

- - + + From 713d50951b68057ef8d0d8f203e3784769d9eeb7 Mon Sep 17 00:00:00 2001 From: Rob Szumski Date: Tue, 15 Oct 2013 10:39:50 -0700 Subject: [PATCH 218/247] fix(dashboard): fix bug in toggling views --- mod/dashboard/app/views/stats.html | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mod/dashboard/app/views/stats.html b/mod/dashboard/app/views/stats.html index e99076681c0..63e584305fd 100644 --- a/mod/dashboard/app/views/stats.html +++ b/mod/dashboard/app/views/stats.html @@ -1,7 +1,7 @@
-
+
+
From d12f97c8cf74dfdb9eb3997b62c24cd2823c0f03 Mon Sep 17 00:00:00 2001 From: Rob Szumski Date: Tue, 15 Oct 2013 10:36:59 -0700 Subject: [PATCH 219/247] chore(dashboard): Remove restangular --- mod/dashboard/app/browser.html | 1 - mod/dashboard/app/stats.html | 1 - 2 files changed, 2 deletions(-) diff --git a/mod/dashboard/app/browser.html b/mod/dashboard/app/browser.html index 27127833eb6..dcb769bd3cc 100644 --- a/mod/dashboard/app/browser.html +++ b/mod/dashboard/app/browser.html @@ -37,7 +37,6 @@ - diff --git a/mod/dashboard/app/stats.html b/mod/dashboard/app/stats.html index eb324f1b22f..6056a75125a 100644 --- a/mod/dashboard/app/stats.html +++ b/mod/dashboard/app/stats.html @@ -38,7 +38,6 @@ - From 98d1209ca249562c35a090aa149b6bf2a9b87a2c Mon Sep 17 00:00:00 2001 From: Rob Szumski Date: Tue, 15 Oct 2013 15:34:33 -0700 Subject: [PATCH 220/247] chore(dashboard) update color ranges --- mod/dashboard/app/views/stats.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mod/dashboard/app/views/stats.html b/mod/dashboard/app/views/stats.html index 63e584305fd..e8a85ceac4e 100644 --- a/mod/dashboard/app/views/stats.html +++ b/mod/dashboard/app/views/stats.html @@ -38,7 +38,7 @@

Machine List

{{machine.name}}
From 59b2eb46185f30ab160005aaa97bcb26fed34a18 Mon Sep 17 00:00:00 2001 From: Rob Szumski Date: Tue, 15 Oct 2013 22:01:45 -0700 Subject: [PATCH 221/247] feature(selection): prevent selection of non-text elements with CSS --- mod/dashboard/app/styles/etcd-widgets.css | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/mod/dashboard/app/styles/etcd-widgets.css b/mod/dashboard/app/styles/etcd-widgets.css index fef7e82783e..03f0c594c88 100644 --- a/mod/dashboard/app/styles/etcd-widgets.css +++ b/mod/dashboard/app/styles/etcd-widgets.css @@ -11,6 +11,10 @@ body { box-sizing: border-box; -moz-box-sizing: border-box; position: relative; + user-select: none; + -webkit-user-select: none; + -moz-user-select: none; + -ms-user-select: none; } a { @@ -99,6 +103,10 @@ body { .etcd-popover-error .etcd-popover-content { color: #FF3C43; font-weight: bold; + user-select: text; + -webkit-user-select: text; + -moz-user-select: text; + -ms-user-select: text; } .etcd-popover-notch { @@ -194,6 +202,10 @@ body { padding-bottom: 6px; padding-left: 0px; vertical-align: text-top; + user-select: text; + -webkit-user-select: text; + -moz-user-select: text; + -ms-user-select: text; } .etcd-body table .etcd-ttl-header { From 56bbab74ca8b76de126ff6ce15378ee1d626a288 Mon Sep 17 00:00:00 2001 From: Rob Szumski Date: Tue, 15 Oct 2013 23:11:34 -0700 Subject: [PATCH 222/247] bug(back button): back button now is disabled when invalid --- mod/dashboard/app/scripts/controllers/browser.js | 2 +- mod/dashboard/app/views/browser.html | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/mod/dashboard/app/scripts/controllers/browser.js b/mod/dashboard/app/scripts/controllers/browser.js index 5032ecc7ec6..42018f2ffd7 100644 --- a/mod/dashboard/app/scripts/controllers/browser.js +++ b/mod/dashboard/app/scripts/controllers/browser.js @@ -44,7 +44,7 @@ angular.module('etcdBrowser', ['ngRoute', 'etcd', 'timeRelative']) localStorage.setItem('etcdPath', $scope.etcdPath); $scope.enableBack = true; //disable back button if at root (/v1/keys/) - if($scope.etcdPath === '') { + if($scope.etcdPath === '/v1/keys') { $scope.enableBack = false; } diff --git a/mod/dashboard/app/views/browser.html b/mod/dashboard/app/views/browser.html index beb081f4fb6..b59f946364a 100644 --- a/mod/dashboard/app/views/browser.html +++ b/mod/dashboard/app/views/browser.html @@ -23,7 +23,7 @@
- + = (uint64(eh.Queue.Size) - eh.DupCnt) { // future index + if index > eh.LastIndex { // future index return nil, nil } - i := int((start + uint64(eh.Queue.Front)) % uint64(eh.Queue.Capacity)) + i := eh.Queue.Front events := make([]*Event, 0) var eventIndex uint64 @@ -85,11 +83,10 @@ func (eh *EventHistory) scan(prefix string, index uint64) ([]*Event, *etcdErr.Er i = (i + 1) % eh.Queue.Capacity - if i == eh.Queue.back() { + if i > eh.Queue.back() { if eventIndex == 0 { // find nothing, return and watch from current index return nil, nil } - return events, nil } } diff --git a/store/store.go b/store/store.go index debf1192eaa..5e374f1ff12 100644 --- a/store/store.go +++ b/store/store.go @@ -32,6 +32,7 @@ type Store interface { Recovery(state []byte) error TotalTransactions() uint64 JsonStats() []byte + DeleteExpiredKeys(cutoff time.Time, index uint64, term uint64) } type store struct { @@ -435,7 +436,7 @@ func (s *store) internalGet(nodePath string, index uint64, term uint64) (*Node, } // deleteExpiredKyes will delete all -func (s *store) deleteExpiredKeys(cutoff time.Time, index uint64, term uint64) { +func (s *store) DeleteExpiredKeys(cutoff time.Time, index uint64, term uint64) { s.worldLock.Lock() defer s.worldLock.Unlock() diff --git a/store/v2/command_factory.go b/store/v2/command_factory.go index 4f0e7260c81..8332891d66d 100644 --- a/store/v2/command_factory.go +++ b/store/v2/command_factory.go @@ -2,7 +2,7 @@ package v2 import ( "time" - + "github.com/coreos/etcd/store" "github.com/coreos/go-raft" ) @@ -28,8 +28,8 @@ func (f *CommandFactory) CreateUpgradeCommand() raft.Command { // CreateSetCommand creates a version 2 command to set a key to a given value in the store. func (f *CommandFactory) CreateSetCommand(key string, value string, expireTime time.Time) raft.Command { return &SetCommand{ - Key: key, - Value: value, + Key: key, + Value: value, ExpireTime: expireTime, } } @@ -37,18 +37,18 @@ func (f *CommandFactory) CreateSetCommand(key string, value string, expireTime t // CreateCreateCommand creates a version 2 command to create a new key in the store. func (f *CommandFactory) CreateCreateCommand(key string, value string, expireTime time.Time, unique bool) raft.Command { return &CreateCommand{ - Key: key, - Value: value, + Key: key, + Value: value, ExpireTime: expireTime, - Unique: unique, + Unique: unique, } } // CreateUpdateCommand creates a version 2 command to update a key to a given value in the store. func (f *CommandFactory) CreateUpdateCommand(key string, value string, expireTime time.Time) raft.Command { return &UpdateCommand{ - Key: key, - Value: value, + Key: key, + Value: value, ExpireTime: expireTime, } } @@ -56,7 +56,7 @@ func (f *CommandFactory) CreateUpdateCommand(key string, value string, expireTim // CreateDeleteCommand creates a version 2 command to delete a key from the store. func (f *CommandFactory) CreateDeleteCommand(key string, recursive bool) raft.Command { return &DeleteCommand{ - Key: key, + Key: key, Recursive: recursive, } } @@ -71,3 +71,7 @@ func (f *CommandFactory) CreateCompareAndSwapCommand(key string, value string, p ExpireTime: expireTime, } } + +func (f *CommandFactory) CreateSyncCommand(now time.Time) raft.Command { + return &SyncCommand{time.Now()} +} diff --git a/store/v2/delete_command.go b/store/v2/delete_command.go index 6bd48368f44..3e8bac81c37 100644 --- a/store/v2/delete_command.go +++ b/store/v2/delete_command.go @@ -1,8 +1,8 @@ package v2 import ( - "github.com/coreos/etcd/store" "github.com/coreos/etcd/log" + "github.com/coreos/etcd/store" "github.com/coreos/go-raft" ) diff --git a/store/watcher_hub.go b/store/watcher_hub.go index 33eda248ef6..52af68c2148 100644 --- a/store/watcher_hub.go +++ b/store/watcher_hub.go @@ -54,7 +54,7 @@ func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (<-chan eventChan <- e } - if len(events) > 1 { + if events[0].Action == Expire { eventChan <- nil } From 5620f88635d815a20a8cfbfb07f7c5a61d8fadb6 Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Wed, 16 Oct 2013 18:59:25 -0700 Subject: [PATCH 224/247] feat(server): insert the mod path --- mod/dashboard/dashboard.go | 3 +++ mod/mod.go | 11 +++++++---- server/server.go | 7 +++++++ 3 files changed, 17 insertions(+), 4 deletions(-) diff --git a/mod/dashboard/dashboard.go b/mod/dashboard/dashboard.go index 7941fc7a3e4..0a74252a0f1 100644 --- a/mod/dashboard/dashboard.go +++ b/mod/dashboard/dashboard.go @@ -7,10 +7,12 @@ import ( "path" "time" + "github.com/coreos/etcd/log" "github.com/coreos/etcd/mod/dashboard/resources" ) func memoryFileServer(w http.ResponseWriter, req *http.Request) { + log.Debugf("[recv] %s %s [%s]", req.Method, req.URL.Path, req.RemoteAddr) upath := req.URL.Path if len(upath) == 0 { upath = "index.html" @@ -42,6 +44,7 @@ func HttpHandler() (handler http.Handler) { // Serve the dashboard from a filesystem if the magic env variable is enabled dashDir := os.Getenv("ETCD_DASHBOARD_DIR") if len(dashDir) != 0 { + log.Debugf("Using dashboard directory %s", dashDir) handler = http.FileServer(http.Dir(dashDir)) } diff --git a/mod/mod.go b/mod/mod.go index f764d6c2f23..741b19002a0 100644 --- a/mod/mod.go +++ b/mod/mod.go @@ -3,13 +3,16 @@ package mod import ( "net/http" + "github.com/coreos/etcd/mod/dashboard" + "github.com/gorilla/mux" ) var ServeMux *http.Handler -func init() { - // TODO: Use a Gorilla mux to handle this in 0.2 and remove the strip - handler := http.StripPrefix("/etcd/mod/dashboard/", dashboard.HttpHandler()) - ServeMux = &handler +func HttpHandler() (handler http.Handler) { + modMux := mux.NewRouter() + modMux.PathPrefix("/dashboard/"). + Handler(http.StripPrefix("/dashboard/", dashboard.HttpHandler())) + return modMux } diff --git a/server/server.go b/server/server.go index 48757ad2ba5..f7d37a73b07 100644 --- a/server/server.go +++ b/server/server.go @@ -12,6 +12,7 @@ import ( etcdErr "github.com/coreos/etcd/error" "github.com/coreos/etcd/log" + "github.com/coreos/etcd/mod" "github.com/coreos/etcd/server/v1" "github.com/coreos/etcd/server/v2" "github.com/coreos/etcd/store" @@ -55,6 +56,7 @@ func New(name string, urlStr string, listenHost string, tlsConf *TLSConfig, tlsI s.handleFunc("/version", s.GetVersionHandler).Methods("GET") s.installV1() s.installV2() + s.installMod() return s } @@ -119,6 +121,11 @@ func (s *Server) installV2() { s.handleFunc("/v2/speedTest", s.SpeedTestHandler).Methods("GET") } +func (s *Server) installMod() { + r := s.Handler.(*mux.Router) + r.PathPrefix("/etcd/mod").Handler(http.StripPrefix("/etcd/mod", mod.HttpHandler())) +} + // Adds a v1 server handler to the router. func (s *Server) handleFuncV1(path string, f func(http.ResponseWriter, *http.Request, v1.Server) error) *mux.Route { return s.handleFunc(path, func(w http.ResponseWriter, req *http.Request) error { From 4fab919fcb0a5e2356030989ba458a28031f1245 Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Fri, 8 Nov 2013 16:23:52 -0800 Subject: [PATCH 225/247] fix(dashboard): use a more memory friendly bindata --- mod/dashboard/build | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mod/dashboard/build b/mod/dashboard/build index 37d90464110..b675f422a63 100755 --- a/mod/dashboard/build +++ b/mod/dashboard/build @@ -13,5 +13,5 @@ export GOPATH="${DIR}/../../" for i in `find dist -type f`; do file=$(echo $i | sed 's#dist/##g' | sed 's#/#-#g') go build github.com/jteeuwen/go-bindata - ./go-bindata -pkg "resources" -toc -out resources/$file.go -prefix dist $i + ./go-bindata -nomemcopy -pkg "resources" -toc -out resources/$file.go -prefix dist $i done From 7ca09579c2fc69dfdb20e0b2ee18fbb99e7dc7d8 Mon Sep 17 00:00:00 2001 From: Rob Szumski Date: Wed, 16 Oct 2013 14:47:10 -0700 Subject: [PATCH 226/247] feat(dashboard): Sort machine list to prevent jumping --- mod/dashboard/app/scripts/controllers/stats.js | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/mod/dashboard/app/scripts/controllers/stats.js b/mod/dashboard/app/scripts/controllers/stats.js index 9ce6ea159a2..14dfdafb661 100644 --- a/mod/dashboard/app/scripts/controllers/stats.js +++ b/mod/dashboard/app/scripts/controllers/stats.js @@ -40,6 +40,12 @@ angular.module('etcdStats', ['ngRoute', 'etcd']) value.name = index; $scope.machines.push(value); }); + //sort array so machines don't jump when output + $scope.machines.sort(function(a, b){ + if(a.name < b.name) return -1; + if(a.name > b.name) return 1; + return 0; + }); drawGraph(); }); } From d045f43cda00c1d1cc4506eae7916b77cebcf94a Mon Sep 17 00:00:00 2001 From: Rob Szumski Date: Wed, 16 Oct 2013 14:48:26 -0700 Subject: [PATCH 227/247] bug(css): Reset line-height to prevent jumping --- mod/dashboard/app/styles/etcd-widgets.css | 1 + 1 file changed, 1 insertion(+) diff --git a/mod/dashboard/app/styles/etcd-widgets.css b/mod/dashboard/app/styles/etcd-widgets.css index 03f0c594c88..5abfdc25f4e 100644 --- a/mod/dashboard/app/styles/etcd-widgets.css +++ b/mod/dashboard/app/styles/etcd-widgets.css @@ -640,6 +640,7 @@ body { color: #999; padding-left: 3px; font-size: 13px; + line-height: 100%; } .etcd-container.etcd-stats .etcd-list .etcd-latency-value { From c91cb72ee2a20e36967796652b1cf8485ac22ee2 Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Fri, 8 Nov 2013 17:41:04 -0800 Subject: [PATCH 228/247] feat(dashboard): have browser use the v2 api --- .../app/scripts/common/services/etcd.js | 19 ++++++++++++------- .../app/scripts/controllers/browser.js | 16 ++++++++-------- mod/dashboard/app/views/browser.html | 2 +- 3 files changed, 21 insertions(+), 16 deletions(-) diff --git a/mod/dashboard/app/scripts/common/services/etcd.js b/mod/dashboard/app/scripts/common/services/etcd.js index 5a303bac8bb..aaad9ed4196 100644 --- a/mod/dashboard/app/scripts/common/services/etcd.js +++ b/mod/dashboard/app/scripts/common/services/etcd.js @@ -2,10 +2,10 @@ angular.module('etcd', []) -.factory('EtcdV1', ['$http', function($http) { - var keyPrefix = '/v1/keys/' - var statsPrefix = '/v1/stats/' - var baseURL = '/v1/' +.factory('EtcdV2', ['$http', function($http) { + var keyPrefix = '/v2/keys/' + var statsPrefix = '/v2/stats/' + var baseURL = '/v2/' delete $http.defaults.headers.common['X-Requested-With']; @@ -15,7 +15,8 @@ angular.module('etcd', []) return ''; } parts = parts.filter(function(v){return v!=='';}); - return parts.join('/'); + parts = parts.join('/'); + return parts } function newKey(keyName) { @@ -32,7 +33,11 @@ angular.module('etcd', []) }; self.path = function() { - return '/' + cleanupPath(keyPrefix + self.name); + var path = '/' + cleanupPath(keyPrefix + self.name); + if (path === keyPrefix.substring(0, keyPrefix.length - 1)) { + return keyPrefix + } + return path }; self.get = function() { @@ -43,7 +48,7 @@ angular.module('etcd', []) return $http({ url: self.path(), data: $.param({value: keyValue}), - method: 'POST', + method: 'PUT', headers: {'Content-Type': 'application/x-www-form-urlencoded'} }); }; diff --git a/mod/dashboard/app/scripts/controllers/browser.js b/mod/dashboard/app/scripts/controllers/browser.js index 42018f2ffd7..72ca269ea08 100644 --- a/mod/dashboard/app/scripts/controllers/browser.js +++ b/mod/dashboard/app/scripts/controllers/browser.js @@ -2,7 +2,7 @@ angular.module('etcdBrowser', ['ngRoute', 'etcd', 'timeRelative']) -.constant('keyPrefix', '/v1/keys') +.constant('keyPrefix', '/v2/keys/') .config(['$routeProvider', 'keyPrefix', function ($routeProvider, keyPrefix) { //read localstorage @@ -18,7 +18,7 @@ angular.module('etcdBrowser', ['ngRoute', 'etcd', 'timeRelative']) }); }]) -.controller('MainCtrl', ['$scope', '$location', 'EtcdV1', 'keyPrefix', function ($scope, $location, EtcdV1, keyPrefix) { +.controller('MainCtrl', ['$scope', '$location', 'EtcdV2', 'keyPrefix', function ($scope, $location, EtcdV2, keyPrefix) { $scope.save = 'etcd-save-hide'; $scope.preview = 'etcd-preview-hide'; $scope.enableBack = true; @@ -43,12 +43,12 @@ angular.module('etcdBrowser', ['ngRoute', 'etcd', 'timeRelative']) // Notify everyone of the update localStorage.setItem('etcdPath', $scope.etcdPath); $scope.enableBack = true; - //disable back button if at root (/v1/keys/) - if($scope.etcdPath === '/v1/keys') { + //disable back button if at root (/v2/keys/) + if($scope.etcdPath === keyPrefix) { $scope.enableBack = false; } - $scope.key = EtcdV1.getKey(etcdPathKey($scope.etcdPath)); + $scope.key = EtcdV2.getKey(etcdPathKey($scope.etcdPath)); }); $scope.$watch('key', function() { @@ -59,14 +59,14 @@ angular.module('etcdBrowser', ['ngRoute', 'etcd', 'timeRelative']) //hide any errors $('#etcd-browse-error').hide(); // Looking at a directory if we got an array - if (data.length) { - $scope.list = data; + if (data.dir === true) { + $scope.list = data.kvs; $scope.preview = 'etcd-preview-hide'; } else { $scope.singleValue = data.value; $scope.preview = 'etcd-preview-reveal'; $scope.key.getParent().get().success(function(data) { - $scope.list = data; + $scope.list = data.kvs; }); } $scope.previewMessage = 'No key selected.'; diff --git a/mod/dashboard/app/views/browser.html b/mod/dashboard/app/views/browser.html index b59f946364a..b40dc38dc9f 100644 --- a/mod/dashboard/app/views/browser.html +++ b/mod/dashboard/app/views/browser.html @@ -56,7 +56,7 @@
- {{key.key}} + {{key.key}}
From bfe33bd3c48e604b402ce15a4cdaf392383ec258 Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Fri, 8 Nov 2013 17:45:55 -0800 Subject: [PATCH 229/247] feat(dashboard): have stats use the v2 API --- mod/dashboard/app/scripts/controllers/stats.js | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mod/dashboard/app/scripts/controllers/stats.js b/mod/dashboard/app/scripts/controllers/stats.js index 14dfdafb661..5e1216ace1e 100644 --- a/mod/dashboard/app/scripts/controllers/stats.js +++ b/mod/dashboard/app/scripts/controllers/stats.js @@ -14,14 +14,14 @@ angular.module('etcdStats', ['ngRoute', 'etcd']) }); }]) -.controller('StatsCtrl', ['$scope', 'EtcdV1', 'statsVega', function ($scope, EtcdV1, statsVega) { +.controller('StatsCtrl', ['$scope', 'EtcdV2', 'statsVega', function ($scope, EtcdV2, statsVega) { $scope.graphContainer = '#latency'; $scope.graphVisibility = 'etcd-graph-show'; $scope.tableVisibility = 'etcd-table-hide'; //make requests function readStats() { - EtcdV1.getStat('leader').get().success(function(data) { + EtcdV2.getStat('leader').get().success(function(data) { $scope.leaderStats = data; $scope.leaderName = data.leader; $scope.machines = []; From 84be60c9205a2a5da3cf3484f17c4b2f35ba1d5c Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Fri, 8 Nov 2013 17:47:49 -0800 Subject: [PATCH 230/247] feat(dashboard): use all of the latest js/css rebuild all of the js and css as go binaries --- mod/dashboard/resources/browser.html.go | 71 +- mod/dashboard/resources/index.html.go | 256 +- .../resources/scripts-browser-modules.js.go | 9503 +---------- .../resources/scripts-browser-scripts.js.go | 227 +- .../resources/scripts-stats-modules.js.go | 13763 +--------------- .../resources/scripts-stats-scripts.js.go | 2743 +-- mod/dashboard/resources/stats.html.go | 71 +- mod/dashboard/resources/styles-main.css.go | 211 +- mod/dashboard/resources/views-browser.html.go | 164 +- mod/dashboard/resources/views-stats.html.go | 90 +- 10 files changed, 170 insertions(+), 26929 deletions(-) diff --git a/mod/dashboard/resources/browser.html.go b/mod/dashboard/resources/browser.html.go index 9b1a8dacb5f..3e0c851eb1a 100644 --- a/mod/dashboard/resources/browser.html.go +++ b/mod/dashboard/resources/browser.html.go @@ -4,73 +4,36 @@ import ( "bytes" "compress/gzip" "io" + "reflect" + "unsafe" ) +var _browser_html = "\x1f\x8b\x08\x00\x00\x09\x6e\x88\x00\xff\x8c\x54\x4f\x6f\xdb\x3e\x0c\xbd\xf7\x53\xb0\xbe\xcb\x3e\xfc\xf0\x43\xdb\x41\x36\xd0\x0d\x3d\xf4\xb6\xcb\x80\x0d\x45\x51\x28\x12\x13\xab\xb3\x45\x4d\xa2\x93\xe5\xdb\x4f\xf2\x9f\xd6\x6d\xb3\x21\x39\xc4\xb4\xc8\xf7\x44\xf2\xbd\x44\x5e\x1a\xd2\x7c\xf4\x08\x2d\xf7\x5d\x73\x21\x2f\x85\x78\xb0\x5b\xe8\x18\xee\xef\xe0\xea\xb1\x81\xf1\x23\x73\x16\x74\xa7\x62\xac\x0b\x47\xe2\x39\xa6\x0a\x61\xf1\x66\x7a\x5c\x4f\x8f\xab\xa2\x01\x79\xf9\x80\xce\xd8\xed\xa3\x10\xaf\x6c\x6b\xaa\x33\xd8\xfe\x41\x73\x7d\x0e\xcd\xdf\xf0\x3b\x9e\x29\xf2\x41\x73\x02\x3f\x02\x85\x78\x03\xce\xf7\xa0\x32\x39\x48\x61\x8f\xac\x40\xb7\x2a\x44\xe4\xba\x18\x78\x2b\x52\xb7\xab\x54\xcb\xec\x05\xfe\x1a\xec\xbe\x2e\xbe\x8b\x6f\xb7\xe2\x0b\xf5\x5e\xb1\xdd\x74\x58\x80\x26\xc7\xe8\x12\xee\xfe\xae\x46\xb3\xc3\x05\xc9\x96\x3b\x6c\x90\xb5\x81\xcf\x81\x0e\x11\x83\xac\xa6\xb3\x15\xb3\x53\x3d\xd6\x85\xc1\xa8\x83\xf5\x6c\xc9\xad\xf8\x8a\x8f\x85\x7b\x8b\x07\x4f\x81\x57\x55\x07\x6b\xb8\xad\x0d\xee\xad\x46\x31\xbe\x2c\xb8\x34\x33\x7c\xed\x94\x46\xd8\xaa\x94\x25\x57\xa6\x2f\x50\xce\x80\xf2\xbe\x43\xc1\x34\xe8\x56\x8c\x09\xef\x76\x60\x1d\x70\x8b\x10\x88\x18\x8c\x0d\xa8\x99\xc2\x11\xf2\xb2\x2e\x5e\xb4\xe9\xac\xfb\x09\x01\xbb\xba\x88\x7c\xec\x30\xb6\x88\xa9\x97\x36\xe0\x76\x39\xa9\x7a\x65\x5d\xa9\x63\xda\xfa\x85\xac\x96\x1d\xcb\x0d\x99\x23\xb8\x9d\x48\x37\xd7\x45\xde\xc9\xbc\x92\x55\xaf\x6f\x0c\x3a\x5f\x29\xfd\xa2\xe4\x66\xac\x6f\x13\xfe\x58\x34\x3f\x68\x00\x15\x10\x86\x68\x53\xe3\xca\x81\x8c\x1c\xc8\xed\x1a\x1a\xd8\x28\x46\x23\xab\xf9\x00\x26\x5c\x28\xd3\x26\x50\x45\x04\xa9\xe6\x76\xb3\xa6\x9f\xaa\x6a\xc5\x5b\x6a\xea\xab\xa2\x19\xfc\x2e\x28\x83\x70\xa4\x21\x2c\x70\x59\xa9\x06\x98\xc0\xf6\x3e\xd0\x7e\xce\xe1\x6f\x8f\xc1\xa2\xd3\x58\xca\xca\x2f\x83\xac\x4c\x76\x62\xb4\x9b\xd7\xd1\x26\xc9\x21\x06\x9d\xa6\xa3\x03\x86\xa7\x74\xbf\x27\x97\x54\x8d\x15\xc6\xff\x45\x6c\x6d\xff\x12\x94\xd9\xc7\x69\xac\x11\x74\x1e\xc7\x73\x24\xf7\x5f\xd5\xd9\xcd\x14\x95\x7d\x12\xe6\x23\xcb\xe9\x8e\xe1\xd6\x98\x69\xca\x68\x19\x81\xc2\xe8\x19\xab\x55\xf6\xe8\x62\x3e\x68\x31\x89\x30\xfd\x9c\x12\xcc\xd8\x3d\x58\x93\xe5\x35\x4f\xf3\xde\x8a\xac\x79\x36\xed\x24\xfa\xa2\x76\x95\x4a\x9b\x57\x57\xad\xa7\x98\xe2\x38\xeb\x12\x44\x4f\x66\x48\xb6\x7a\xd7\xf8\x79\xd8\xf9\xfd\x1d\x56\x56\xd9\x8c\xa3\x3b\xf3\x9f\xe3\x9f\x00\x00\x00\xff\xff\x4a\x5c\x90\x9e\x2c\x05\x00\x00" + // browser_html returns raw, uncompressed file data. func browser_html() []byte { - gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ -0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0x8c,0x54, -0x4f,0x6f,0xdb,0x3e,0x0c,0xbd,0xf7,0x53,0xb0,0xbe,0xcb,0x3e, -0xfc,0xf0,0x43,0xdb,0x41,0x36,0xd0,0x0d,0x3d,0xf4,0xb6,0xcb, -0x80,0x0d,0x45,0x51,0x28,0x12,0x13,0xab,0xb3,0x45,0x4d,0xa2, -0x93,0xe5,0xdb,0x4f,0xf2,0x9f,0xd6,0x6d,0xb3,0x21,0x39,0xc4, -0xb4,0xc8,0xf7,0x44,0xf2,0xbd,0x44,0x5e,0x1a,0xd2,0x7c,0xf4, -0x08,0x2d,0xf7,0x5d,0x73,0x21,0x2f,0x85,0x78,0xb0,0x5b,0xe8, -0x18,0xee,0xef,0xe0,0xea,0xb1,0x81,0xf1,0x23,0x73,0x16,0x74, -0xa7,0x62,0xac,0x0b,0x47,0xe2,0x39,0xa6,0x0a,0x61,0xf1,0x66, -0x7a,0x5c,0x4f,0x8f,0xab,0xa2,0x01,0x79,0xf9,0x80,0xce,0xd8, -0xed,0xa3,0x10,0xaf,0x6c,0x6b,0xaa,0x33,0xd8,0xfe,0x41,0x73, -0x7d,0x0e,0xcd,0xdf,0xf0,0x3b,0x9e,0x29,0xf2,0x41,0x73,0x02, -0x3f,0x02,0x85,0x78,0x03,0xce,0xf7,0xa0,0x32,0x39,0x48,0x61, -0x8f,0xac,0x40,0xb7,0x2a,0x44,0xe4,0xba,0x18,0x78,0x2b,0x52, -0xb7,0xab,0x54,0xcb,0xec,0x05,0xfe,0x1a,0xec,0xbe,0x2e,0xbe, -0x8b,0x6f,0xb7,0xe2,0x0b,0xf5,0x5e,0xb1,0xdd,0x74,0x58,0x80, -0x26,0xc7,0xe8,0x12,0xee,0xfe,0xae,0x46,0xb3,0xc3,0x05,0xc9, -0x96,0x3b,0x6c,0x90,0xb5,0x81,0xcf,0x81,0x0e,0x11,0x83,0xac, -0xa6,0xb3,0x15,0xb3,0x53,0x3d,0xd6,0x85,0xc1,0xa8,0x83,0xf5, -0x6c,0xc9,0xad,0xf8,0x8a,0x8f,0x85,0x7b,0x8b,0x07,0x4f,0x81, -0x57,0x55,0x07,0x6b,0xb8,0xad,0x0d,0xee,0xad,0x46,0x31,0xbe, -0x2c,0xb8,0x34,0x33,0x7c,0xed,0x94,0x46,0xd8,0xaa,0x94,0x25, -0x57,0xa6,0x2f,0x50,0xce,0x80,0xf2,0xbe,0x43,0xc1,0x34,0xe8, -0x56,0x8c,0x09,0xef,0x76,0x60,0x1d,0x70,0x8b,0x10,0x88,0x18, -0x8c,0x0d,0xa8,0x99,0xc2,0x11,0xf2,0xb2,0x2e,0x5e,0xb4,0xe9, -0xac,0xfb,0x09,0x01,0xbb,0xba,0x88,0x7c,0xec,0x30,0xb6,0x88, -0xa9,0x97,0x36,0xe0,0x76,0x39,0xa9,0x7a,0x65,0x5d,0xa9,0x63, -0xda,0xfa,0x85,0xac,0x96,0x1d,0xcb,0x0d,0x99,0x23,0xb8,0x9d, -0x48,0x37,0xd7,0x45,0xde,0xc9,0xbc,0x92,0x55,0xaf,0x6f,0x0c, -0x3a,0x5f,0x29,0xfd,0xa2,0xe4,0x66,0xac,0x6f,0x13,0xfe,0x58, -0x34,0x3f,0x68,0x00,0x15,0x10,0x86,0x68,0x53,0xe3,0xca,0x81, -0x8c,0x1c,0xc8,0xed,0x1a,0x1a,0xd8,0x28,0x46,0x23,0xab,0xf9, -0x00,0x26,0x5c,0x28,0xd3,0x26,0x50,0x45,0x04,0xa9,0xe6,0x76, -0xb3,0xa6,0x9f,0xaa,0x6a,0xc5,0x5b,0x6a,0xea,0xab,0xa2,0x19, -0xfc,0x2e,0x28,0x83,0x70,0xa4,0x21,0x2c,0x70,0x59,0xa9,0x06, -0x98,0xc0,0xf6,0x3e,0xd0,0x7e,0xce,0xe1,0x6f,0x8f,0xc1,0xa2, -0xd3,0x58,0xca,0xca,0x2f,0x83,0xac,0x4c,0x76,0x62,0xb4,0x9b, -0xd7,0xd1,0x26,0xc9,0x21,0x06,0x9d,0xa6,0xa3,0x03,0x86,0xa7, -0x74,0xbf,0x27,0x97,0x54,0x8d,0x15,0xc6,0xff,0x45,0x6c,0x6d, -0xff,0x12,0x94,0xd9,0xc7,0x69,0xac,0x11,0x74,0x1e,0xc7,0x73, -0x24,0xf7,0x5f,0xd5,0xd9,0xcd,0x14,0x95,0x7d,0x12,0xe6,0x23, -0xcb,0xe9,0x8e,0xe1,0xd6,0x98,0x69,0xca,0x68,0x19,0x81,0xc2, -0xe8,0x19,0xab,0x55,0xf6,0xe8,0x62,0x3e,0x68,0x31,0x89,0x30, -0xfd,0x9c,0x12,0xcc,0xd8,0x3d,0x58,0x93,0xe5,0x35,0x4f,0xf3, -0xde,0x8a,0xac,0x79,0x36,0xed,0x24,0xfa,0xa2,0x76,0x95,0x4a, -0x9b,0x57,0x57,0xad,0xa7,0x98,0xe2,0x38,0xeb,0x12,0x44,0x4f, -0x66,0x48,0xb6,0x7a,0xd7,0xf8,0x79,0xd8,0xf9,0xfd,0x1d,0x56, -0x56,0xd9,0x8c,0xa3,0x3b,0xf3,0x9f,0xe3,0x9f,0x00,0x00,0x00, -0xff,0xff,0x4a,0x5c,0x90,0x9e,0x2c,0x05,0x00,0x00, - })) + var empty [0]byte + sx := (*reflect.StringHeader)(unsafe.Pointer(&_browser_html)) + b := empty[:] + bx := (*reflect.SliceHeader)(unsafe.Pointer(&b)) + bx.Data = sx.Data + bx.Len = len(_browser_html) + bx.Cap = bx.Len + + gz, err := gzip.NewReader(bytes.NewBuffer(b)) if err != nil { panic("Decompression failed: " + err.Error()) } - var b bytes.Buffer - io.Copy(&b, gz) + var buf bytes.Buffer + io.Copy(&buf, gz) gz.Close() - return b.Bytes() + return buf.Bytes() } + func init() { go_bindata["/browser.html"] = browser_html } diff --git a/mod/dashboard/resources/index.html.go b/mod/dashboard/resources/index.html.go index aa411167fa7..4118b77e36d 100644 --- a/mod/dashboard/resources/index.html.go +++ b/mod/dashboard/resources/index.html.go @@ -4,258 +4,36 @@ import ( "bytes" "compress/gzip" "io" + "reflect" + "unsafe" ) +var _index_html = "\x1f\x8b\x08\x00\x00\x09\x6e\x88\x00\xff\xb4\x59\x6d\x6f\xe3\xc6\x11\xfe\xde\x5f\xc1\x30\xe8\xa7\x90\xd4\xbe\xbf\xf8\x24\x17\x57\x5f\x82\x04\x48\x8a\xa0\x41\x8b\x16\x41\x50\xd0\x14\x2d\xb1\x47\x8b\xaa\x48\xdb\x77\x57\xf4\xbf\xf7\x99\x5d\x52\xb6\x65\x59\x76\x2e\xad\x00\x9d\x38\xe4\xee\xec\xbc\x3c\xf3\xcc\xd0\x37\xff\x62\xd9\x55\xc3\xc7\x6d\x9d\xac\x87\xeb\xf6\xfc\x77\xf3\x2f\xf2\xfc\xe7\xe6\x2a\x69\x87\xe4\xbb\xaf\x13\xfb\xcb\x79\x12\x3e\x73\x7a\x9a\x54\x6d\xd9\xf7\x8b\x74\xd3\xe5\xff\xec\xb1\x22\x6f\x6a\x1f\x7f\x5c\xfc\xb1\xe9\x79\x32\xff\xe2\xe7\x7a\xb3\x6c\xae\x7e\xc9\xf3\x7b\x6d\x0f\x55\xbd\x42\xdb\x09\x35\xee\x35\x6a\x9e\xdb\xbf\x1a\x46\x15\x74\xe3\xfc\xc8\xfe\xb0\x31\xcf\x1f\x6d\xa6\x73\xea\x72\x49\x17\xb8\xbc\xae\x87\x32\xa9\xd6\xe5\xae\xaf\x87\x45\x7a\x33\x5c\xe5\xb0\xf6\xc1\xa3\xf5\x30\x6c\xf3\xfa\x5f\x37\xcd\xed\x22\xfd\x5b\xfe\x97\xb7\xf9\x45\x77\xbd\x2d\x87\xe6\xb2\xad\xd3\xa4\xea\x36\x43\xbd\xc1\xbe\xef\xbe\x5e\xd4\xcb\x55\x3d\xed\x1c\x9a\xa1\xad\xcf\xeb\xa1\x5a\x26\xcb\xb2\x5f\x5f\x76\xe5\x6e\x39\x9f\xc5\xbb\x0f\x74\x6f\xca\xeb\x7a\x91\x2e\xeb\xbe\xda\x35\xdb\xa1\xe9\x36\x0f\x34\xa6\x4f\x17\xde\x36\xf5\xdd\xb6\xdb\x0d\x0f\x56\xdd\x35\xcb\x61\xbd\x58\xd6\xb7\x4d\x55\xe7\x41\x98\xf6\xb5\xcd\xe6\x7d\xb2\xde\xd5\x57\x8b\x94\x5c\x38\x9b\xcd\xae\xb0\xa9\x2f\x56\x5d\xb7\x6a\xeb\x72\xdb\xf4\x45\xd5\x5d\xcf\xaa\xbe\xff\xc3\x55\x79\xdd\xb4\x1f\x17\x3f\x75\x37\xbb\xaa\xfe\xea\xa7\x72\xd3\x7f\xf5\xe3\xae\x3b\x13\x8c\x65\x12\x5f\x15\xbf\xcd\x50\xb6\x4d\x95\x19\x48\x16\x5f\xcf\x58\x9a\xec\xea\x76\x91\xf6\xc3\xc7\xb6\xee\xd7\x75\x0d\xc3\x08\x78\x8b\x74\xa8\x3f\x0c\xa4\xf9\x37\xdb\x72\xd1\x2d\xeb\x60\x0b\xd9\xa0\xf1\x1d\x4f\x7f\xf5\xc9\x61\x45\xbc\xa6\xcf\x65\xb7\xfc\x98\xfc\x7b\x2f\xd2\x67\x5b\x2e\x97\xcd\x66\x75\x96\x48\xb6\xfd\xf0\xe6\xd1\xa3\xeb\x72\xb7\x6a\x36\x67\xc9\xa3\x07\xff\xd9\x5f\xad\xf9\x81\x2a\x72\x2a\x8f\x0e\x9c\x25\x69\x74\x21\xa1\x70\x26\x70\x21\xcd\x92\xf4\xdb\xba\xbd\xad\x87\xa6\x2a\x93\x3f\xd5\x37\x35\xee\xec\x6f\x64\xc9\xdb\x5d\x53\xb6\x59\xd2\x63\x79\xde\xd7\xbb\xe6\xea\xcd\x53\xdd\x77\x75\xb3\x5a\x0f\x67\x09\xa2\xf1\xac\xa5\xe1\x2b\xc6\x8b\x37\xc7\x5d\x7d\xc6\xa1\xe6\x6a\x07\x9c\x1d\x38\x75\xd9\xed\x96\xf5\xee\x2c\xd9\x74\x9b\xfa\xe1\xa6\xfd\x65\x79\xb0\xa1\xea\xda\x0e\xeb\xbf\xe4\xb5\xa9\x2b\xfe\xd8\x02\x4a\x4f\xbe\xac\xab\x6e\x57\x12\xdc\x4f\x69\x3d\x5b\x77\xb7\xf5\xee\x40\xf7\x93\xfd\x37\x1b\x18\x07\x78\x1d\x57\x72\xd4\xa1\x18\xab\xfc\xb2\x1b\x86\xee\xfa\x30\xed\x4f\x37\x7f\xf5\x2a\x2d\xcf\x29\xf9\xf2\xaa\xeb\x86\x27\x6e\x84\x42\x3d\x4b\x38\x63\xbf\x3f\x92\xe5\xff\x13\x82\x46\x8b\x87\x6e\x7b\x16\xf0\x71\xd4\xde\x60\x33\xc2\x5b\x77\x7d\xde\x76\xab\xee\xc0\xf0\x7b\x3d\x64\x3d\x30\x56\xde\x0c\x5d\xc2\xc2\xcf\x9b\x27\x2b\xd7\x23\x5c\x9f\x56\xd6\x83\x20\xb8\x27\x0f\x0f\xcc\x79\x62\x52\x7f\xbb\x3a\x62\x56\x08\x5f\xd3\xb6\x40\x9e\xf7\xfe\xe9\x71\xd1\xf4\x0f\xf9\x7d\xec\x8f\x19\x45\x9f\x65\xd3\x6f\xdb\x12\x09\x68\x36\x84\xac\xfc\xb2\xed\xaa\xf7\xc7\x97\x02\xa1\x14\xfb\x36\x07\x35\xae\x10\x93\xeb\x66\xb9\x6c\xeb\xa7\x6b\x0f\x23\xbc\xed\xee\xea\x5d\xbd\xcc\x2f\x0f\xd9\x28\x78\x41\x20\xe8\x9b\x4f\x35\xac\x14\xc7\x8c\x9c\x2a\x4c\x4a\xf9\x6c\x58\x9f\x62\xeb\x57\xb8\xf6\x5a\xb7\x82\x8e\x29\xcb\xdc\x1f\x3b\x31\x54\xec\xa8\xa6\x42\xcb\xaa\x77\x87\xc9\xa6\x7f\xe7\xb3\x91\xa8\xe7\xb3\xd8\x9a\xe7\xc4\xd4\x23\x87\xaf\x79\xec\xa4\xef\xee\x3b\x29\x6e\xc5\x67\x63\x6d\xf6\xbb\x8a\xba\x41\x89\x9e\x42\x23\x40\x9a\x04\x75\x63\x7b\x1c\x83\xb1\x87\xa3\x0a\xa9\x4f\xcf\xe7\xb3\xb8\xfb\x88\xaa\xcb\x5d\x77\x87\x12\xfa\x7c\x65\xcb\xe6\x36\x69\x96\x8b\x34\x96\x7f\x7a\xdf\x81\xf6\x4f\xee\x21\x90\x9e\xff\x18\xaf\x13\xc0\x61\x5e\x3e\x68\x94\x3d\x3a\xe5\xaa\x19\xd6\x37\x97\xb1\x3f\x86\x22\x98\x51\x30\xd2\x10\x92\xf9\xac\xc4\xc1\xd0\x78\x44\xff\x83\x8a\x79\x70\x7c\x58\x52\x3e\xee\xc5\x71\x25\x9d\x70\xb0\x30\x2c\xa6\x6a\x03\x1e\x7a\xd0\xed\x22\xe5\x05\x4f\x83\xf6\xef\xcb\x8f\xf5\xee\x1f\x10\x3e\x5c\xb7\x9b\x7e\xaf\xea\xee\xee\xae\xb8\x93\x45\xb7\x5b\xcd\x30\x3d\xb0\x19\x36\x8f\x4b\xce\x3e\xd0\x10\x70\x6c\x21\x47\xb9\xce\xc2\x53\x2c\x5d\xa4\x08\x67\x9a\x7c\x8c\xbf\x47\x6b\x2e\xd9\xee\x6a\xe4\xe6\xb6\x7e\xdb\x6f\xeb\x6a\xf8\x33\x75\x82\x45\xfa\xe1\x87\x66\xf3\x77\x7c\xd3\x84\xa6\xa4\x3f\x76\xa4\x09\xc4\x64\xbd\x00\xfd\x98\x34\xa9\x37\x25\x26\xb6\xfc\xb2\xac\xde\xaf\x76\x1d\xda\x06\x06\xc4\xfa\x2e\x79\xb4\x06\x86\x9e\xf5\xdb\xb2\x42\xae\xa7\x33\x8e\x45\x64\xf5\xf4\xde\xa9\xfb\xe1\x19\x26\xc6\x75\xa0\xa7\x45\xfa\xa5\x96\x6f\xe5\xbb\xb7\x69\x02\x13\x7e\xe0\xd2\x14\xdc\xb8\x4c\xe9\x42\x0b\x7b\x61\x4d\xe1\xfc\x24\x65\xc2\x15\xc6\xf9\xcc\xcb\xc2\x4a\x3f\x49\x5c\xcb\x0a\xb3\x90\x2f\x84\xd1\x99\x72\x85\x60\xb8\xc7\x6c\xa1\xac\x1a\x7f\xf7\xf2\xb3\xe6\xd0\xa7\x22\x15\x5a\x64\x2c\xae\x36\x3a\x7f\xa8\x0c\xe2\xa8\xe4\x42\x28\x59\x18\x29\x27\x33\xb8\xd7\x85\x12\x93\x89\x8f\xed\xff\x94\xce\x5e\x19\x83\x6f\xb8\x61\xe6\xdd\xe3\x18\x68\x5d\x48\xe7\xab\x9c\xdb\x42\x38\x99\xb1\x5c\xf2\xc2\x2b\x9e\x09\x5b\x18\xa5\x73\x89\xdb\x52\x67\xc6\x14\xcc\x60\x15\xc3\x1e\x58\x83\x7b\x06\xd7\xa0\xc2\x4c\x14\xca\x59\x5c\x2b\xeb\x32\xd8\xaa\xcc\x69\xff\xb1\xd2\x0a\x99\x41\x9d\xb2\x39\xf4\x70\x99\x71\x01\x45\xf0\xbc\x50\xda\x65\xdc\x15\x8e\xb9\x70\x10\x47\x2e\x0a\x6f\x04\x5d\x5b\x9d\xe9\xc2\x6b\x33\x5e\xbb\xc2\x3b\x87\x84\xc8\x82\x09\x9f\xb1\x82\x61\xa7\x19\xaf\xa7\xe7\xfa\xb4\x21\xac\x10\x0a\x7e\xc1\x6e\x64\x03\x31\xd6\x64\x87\x35\x48\xe7\xde\x0e\xce\x2b\x57\x70\x46\xb7\x98\x82\xd7\x06\xcf\xc9\x79\x6b\x10\x1e\x0d\x3b\xc9\x40\xe6\x7d\x25\x0b\x8f\x6b\x1c\xad\x4c\x66\x0b\x17\x34\x0a\xc4\x86\xf3\xf1\x01\x84\xd3\xd6\x28\x18\x8f\xe0\x67\x0e\x0e\x32\xe7\x68\xa7\xf3\x3a\x0f\x3b\x2b\x4f\xc7\x87\x70\xe3\x81\x2d\x2c\x73\x08\x16\xd3\xc1\x06\xee\xf3\x68\x03\xc3\x86\x10\x2a\x4e\xf6\x5a\x1b\xf6\x02\xbf\x85\xd1\x61\xaf\x71\xa7\x2d\x90\x48\x9e\x56\xb9\x42\x0a\x33\x1d\x92\x9e\x73\x8d\x1f\x31\x49\x00\x84\x12\xee\x42\xd0\x42\x6b\x32\xef\xa3\xa1\x1e\x06\x5b\x3e\xc2\x28\x7b\x8c\xaa\x5f\x81\xcc\xf0\x19\x91\x89\x7a\xd4\x40\x20\x87\x7f\x3a\x82\xce\x5b\x4f\xf8\x10\x8e\x9c\x15\xe4\xb2\xf5\x26\x97\x08\xb8\x83\xc5\xc8\x53\xee\x02\x3a\x38\x2b\xb4\xb6\xb9\x10\x85\xf0\x36\x40\xda\x44\x08\x0b\x35\x4a\x2f\xa0\x53\x15\x0e\x41\x63\xb9\x27\xd5\xc8\xa5\x62\x50\x13\xb3\x08\x44\x78\x46\x85\x82\x70\x9b\x4c\x15\xda\xe1\x3c\x94\x8a\xcd\xfc\x98\x06\xe5\x01\x22\xd4\xa9\x0f\xf8\x55\x08\x1d\x56\x69\x8e\x6b\x03\xd6\xf0\x85\xf4\xd3\x35\x57\x10\xfc\x49\x5b\xfa\x08\x21\xe8\x56\x84\xea\xfd\x2e\x53\x91\x52\x95\xc5\x03\xa0\xd4\xd9\xf0\x58\xd1\x63\xc5\x44\x14\x2a\x5d\x30\xb0\x0a\x0b\xfb\x45\xc0\x82\x18\xf7\x07\x13\x5e\x60\x29\xc4\x91\x71\x32\x96\x3c\x02\x03\x6a\x09\xf7\x1c\x0a\x4b\x00\x1f\x56\xe7\x70\x12\x41\xa7\x22\x36\x78\xa0\x19\xc7\x39\x06\x31\xd6\x85\xe1\x7a\xbc\x0e\x35\x78\xc1\x9d\x42\x0d\xa3\x14\x14\x14\x90\x0d\x0e\xc8\xa6\xfa\x02\x8c\x24\xe1\xe7\x71\xb2\x3f\x3d\xd3\x7d\xc6\xcf\x33\x70\x9a\xcf\x7e\x6b\x77\x10\x92\x7f\x23\x58\xc4\x9f\x54\x40\x81\x87\x49\x8c\x88\x45\x56\xe0\x05\xed\x34\x62\x09\x58\x39\x6e\xc0\x19\x1c\x85\x28\xa8\x17\xc0\x1f\x50\x90\x11\x2d\x21\x8f\x51\x0e\x78\x21\xc5\x4b\x14\x88\xda\x40\x40\x75\xc1\x51\xb0\xd8\x00\xc2\x42\xb0\x00\x9e\x9c\x53\xa5\xf3\x28\x00\x68\x66\x04\xa3\x20\x70\x53\x08\x51\x5a\x93\x40\xa0\xf6\x1c\x14\x28\x64\xa1\x54\x20\x0c\x62\x66\xdc\x36\xcc\x51\xd7\x42\xac\xa3\x70\xda\x18\x00\x5d\x38\x6a\x47\x3a\x10\x0e\xd5\x36\xb2\x0c\x02\xe1\x96\x7c\x12\xc2\xb5\xc1\x35\x3b\x9e\x40\x95\x06\xb6\x46\x8e\xb5\x77\xc1\x60\x22\x25\xe4\x18\x7b\x03\x11\x45\xa1\xca\x05\xc2\xc4\xc9\x7a\x05\xda\x94\x3c\x27\x42\x75\x93\xa0\x89\xb8\x4f\xd7\xe2\x05\x2a\xb8\x90\x38\x56\xe0\x78\x69\x32\x09\x05\x4a\xab\x29\x2b\xd9\x41\x96\x5e\x4f\x35\x0f\x53\xad\x10\x63\x6f\x22\xfa\x2c\xe3\x15\xfc\x91\xa1\x3b\x83\xc6\x9c\x08\x15\x63\xbd\x9c\x24\x05\x76\xf1\x96\x62\x0e\x9c\xab\xc0\x2f\x0a\xd4\xa7\xa8\x14\x7d\x7e\xbf\x08\xd2\x0b\x08\xc0\x39\xdc\x5b\x6a\xb6\x06\xc5\x98\x87\x63\xc4\x28\x44\x05\x17\xd2\x81\xb4\x51\x58\x9c\x64\xaf\x32\x89\x70\x84\xf6\x14\x4d\xcd\x0e\x4c\xff\x94\xdc\x3b\xe3\x89\x96\x5e\x60\x3a\x02\x2a\x35\x1c\x44\x55\xe8\x31\xd3\xa3\x10\xbd\xab\x18\xf1\xbf\x46\x23\x81\x56\x91\x87\x7e\xa7\xf2\xfd\x0a\x08\x3d\x49\xcc\x53\x4a\xb0\xdb\x4e\x52\x7c\x78\x3a\xb7\xd2\x03\x0d\x84\x29\x17\x90\x86\xa0\xe1\x4c\xb9\x37\x3d\x3b\x70\xe5\x33\x93\x8b\x59\xcb\x5a\x8a\x90\x45\x01\xfb\x35\x01\x53\xc9\x16\x43\x87\xa1\xb2\x35\xce\xad\x41\x85\xc2\x83\x2b\x35\x0b\xbd\x83\x58\x1e\x0f\x40\xd3\xd1\x75\x2a\x04\x67\x27\xe9\x85\xfe\x59\x78\x49\x85\x84\x89\x42\x6b\x62\x67\x4c\x01\x61\x86\x40\x37\xd6\x56\xb5\xb1\x5b\x51\x9d\x59\xa9\x50\x1e\x85\xa1\x16\x6d\x5d\x68\xb9\x3c\xcc\x42\xe8\x5c\x08\x21\x77\xf1\xba\xca\x89\x68\xa8\x82\x60\x93\xc6\x85\xa2\x89\x25\x66\xc0\xc4\x7a\x95\xb7\x0a\x34\xc0\x4e\xcf\x16\xeb\x98\x18\xf1\xd7\x31\x0e\x9f\x17\x4c\x70\x34\x98\xe2\xbe\x52\x50\x95\x3e\x10\xc7\x38\x2e\xc2\x1e\x8c\xae\x93\x14\x90\x6c\xc3\x80\x86\x61\x8a\xc6\x0f\x4c\x25\xb8\x07\x04\x91\xcf\x36\x0b\x14\xb7\xce\x15\xf8\xe3\x85\x5a\xa1\xe8\x51\x1d\x32\x26\x09\x68\x12\x83\x36\x65\x05\x5c\x17\xe8\x98\x8f\x12\x9a\x91\xe1\x01\xd0\xe8\xfd\x96\xda\x15\x23\xab\x2c\x99\x43\x6d\x09\x1c\x4d\xe1\xd4\xaf\x63\x68\x30\x0f\x3a\x1f\x08\xc6\x3a\xa2\x61\x83\x79\x07\xb4\x07\x4d\x60\x35\x6a\xbb\x41\x40\x12\x31\x93\x60\x3c\x44\x19\x23\xc2\x2e\x1f\x69\x30\x5c\x8f\x55\xac\x19\xb5\x35\xaa\x62\x0a\x5f\xa6\x85\x0c\x83\xde\x54\xc4\x07\x51\xfd\x74\xd2\xac\xe4\x07\xcc\x36\x64\x3c\x82\x0d\x9e\xb2\x86\x4a\x94\xbc\xb5\xd4\x4b\x9c\x27\x56\x31\x11\xad\x56\x8a\x51\xaa\x68\x94\x11\x9a\x50\x84\x92\xf2\xd4\xbd\xac\x71\x04\x09\x0a\x6b\x58\xf2\xed\xa8\xf6\xf3\xde\x23\x0c\xf8\xd7\x60\xec\xe0\x1a\x90\x66\x44\x1b\xe8\x03\x16\x93\x38\x4d\xaa\x4e\x13\xd5\x83\xaa\x33\x45\x23\x9b\x1f\xa5\x0a\x63\x84\x33\x94\x2d\xdc\x36\xf4\xb2\xc1\xc2\xe0\x30\x4a\x71\xd1\x0b\xe3\x7b\x86\x63\x1c\x39\x8d\x63\x50\xa9\x1a\xcd\x51\xe6\x93\x02\x12\x2e\x0c\x9c\xa4\xe9\x48\x30\x6a\x3a\x36\xdb\x5b\xea\x68\x46\x33\xd9\x81\xe5\x60\x50\x63\x09\x97\xd3\x8d\x17\x0c\xa0\x4e\x6c\xc0\xfb\xd4\x96\xf5\xbe\xb3\x49\x0c\x5e\x7e\x14\x02\x40\x98\x0b\x00\xc1\x28\xad\x02\xc1\x32\x2f\x27\x69\x5c\xc5\xa8\x5d\x63\x9e\x8d\x0d\x1e\xa1\x42\x99\x02\x13\xd3\xa2\x20\x9d\xa6\x53\xcc\x73\x60\x73\x01\x4f\x59\x68\xe7\x93\x1f\x8e\xf0\x29\xb3\xc7\x6e\xbd\x3e\xcd\x78\x65\x56\xef\xfc\x98\x66\xcc\xfb\x96\x5a\xbd\x33\x84\xa7\xca\xd2\xa0\x1d\xcb\x98\xbc\x52\x5c\x4c\x45\x2a\xd1\x25\x85\x1f\xa5\x2a\x82\x31\x63\xe3\x0b\x53\x98\x6d\xfc\x24\x20\xed\xd6\xbc\x18\x67\x60\xdc\x33\x15\x76\x86\x29\x9e\xeb\xd0\x87\xf0\x1a\x99\xc7\x41\xa5\x8d\x90\x37\x28\x5b\xe6\x44\x78\x17\x30\xe8\xc7\x20\x64\x43\x47\x78\xa9\x88\xdc\x25\xc0\x38\x4a\xe1\xf8\x17\x06\x23\x16\xca\x59\xd1\x00\x12\x8e\x17\x04\x1a\x41\xde\x31\xbc\x50\x45\x29\x8e\x86\x2e\x8e\x86\x06\x53\xad\x0e\x73\x3a\x8d\x68\x14\x6f\xbc\x11\x48\xdd\xe6\xc4\xdb\x2a\x92\x09\xcd\x7e\xde\x89\x9c\x9a\x84\xa0\x03\xa8\x4e\x11\x84\x9c\xe6\x6a\x16\xfa\xf0\x0b\xe1\xc8\x63\x83\xa2\xa9\x30\x4e\xa3\xf4\x72\xe6\x27\x81\xca\xda\x68\x20\x8a\xde\x59\x68\x3a\x0d\x6e\xd3\x9b\x2b\x21\x8a\xd8\x10\x93\x14\xaa\x1e\x93\x79\x1b\x62\x46\xaf\xcd\xfe\x05\x78\xd1\xac\xa0\x9c\xc9\x68\x60\x1d\x67\xc3\xf0\x6a\x8b\x21\x61\x92\xa8\xb3\x48\xdf\xe7\x61\x1a\x0e\x7f\x3b\x70\x78\xef\xc1\x88\x2f\xd8\x24\x55\xf9\xd8\x22\x18\x91\x28\x46\x5e\x8a\x01\xd6\x20\xc8\x14\x04\xea\xb0\xe6\xfb\x03\x94\xfd\x0f\xdf\x06\x9e\xb9\xd9\xdf\x1e\xdc\xa6\x3f\xee\xdd\xff\x59\xef\xfe\xaf\x7c\xe3\xe5\x7c\x16\xff\x44\x3a\x9f\xc5\xff\xe7\xfd\x6f\x00\x00\x00\xff\xff\x81\x9d\x08\x43\xf8\x1d\x00\x00" + // index_html returns raw, uncompressed file data. func index_html() []byte { - gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ -0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0xb4,0x59, -0x6d,0x6f,0xe3,0xc6,0x11,0xfe,0xde,0x5f,0xc1,0x30,0xe8,0xa7, -0x90,0xd4,0xbe,0xbf,0xf8,0x24,0x17,0x57,0x5f,0x82,0x04,0x48, -0x8a,0xa0,0x41,0x8b,0x16,0x41,0x50,0xd0,0x14,0x2d,0xb1,0x47, -0x8b,0xaa,0x48,0xdb,0x77,0x57,0xf4,0xbf,0xf7,0x99,0x5d,0x52, -0xb6,0x65,0x59,0x76,0x2e,0xad,0x00,0x9d,0x38,0xe4,0xee,0xec, -0xbc,0x3c,0xf3,0xcc,0xd0,0x37,0xff,0x62,0xd9,0x55,0xc3,0xc7, -0x6d,0x9d,0xac,0x87,0xeb,0xf6,0xfc,0x77,0xf3,0x2f,0xf2,0xfc, -0xe7,0xe6,0x2a,0x69,0x87,0xe4,0xbb,0xaf,0x13,0xfb,0xcb,0x79, -0x12,0x3e,0x73,0x7a,0x9a,0x54,0x6d,0xd9,0xf7,0x8b,0x74,0xd3, -0xe5,0xff,0xec,0xb1,0x22,0x6f,0x6a,0x1f,0x7f,0x5c,0xfc,0xb1, -0xe9,0x79,0x32,0xff,0xe2,0xe7,0x7a,0xb3,0x6c,0xae,0x7e,0xc9, -0xf3,0x7b,0x6d,0x0f,0x55,0xbd,0x42,0xdb,0x09,0x35,0xee,0x35, -0x6a,0x9e,0xdb,0xbf,0x1a,0x46,0x15,0x74,0xe3,0xfc,0xc8,0xfe, -0xb0,0x31,0xcf,0x1f,0x6d,0xa6,0x73,0xea,0x72,0x49,0x17,0xb8, -0xbc,0xae,0x87,0x32,0xa9,0xd6,0xe5,0xae,0xaf,0x87,0x45,0x7a, -0x33,0x5c,0xe5,0xb0,0xf6,0xc1,0xa3,0xf5,0x30,0x6c,0xf3,0xfa, -0x5f,0x37,0xcd,0xed,0x22,0xfd,0x5b,0xfe,0x97,0xb7,0xf9,0x45, -0x77,0xbd,0x2d,0x87,0xe6,0xb2,0xad,0xd3,0xa4,0xea,0x36,0x43, -0xbd,0xc1,0xbe,0xef,0xbe,0x5e,0xd4,0xcb,0x55,0x3d,0xed,0x1c, -0x9a,0xa1,0xad,0xcf,0xeb,0xa1,0x5a,0x26,0xcb,0xb2,0x5f,0x5f, -0x76,0xe5,0x6e,0x39,0x9f,0xc5,0xbb,0x0f,0x74,0x6f,0xca,0xeb, -0x7a,0x91,0x2e,0xeb,0xbe,0xda,0x35,0xdb,0xa1,0xe9,0x36,0x0f, -0x34,0xa6,0x4f,0x17,0xde,0x36,0xf5,0xdd,0xb6,0xdb,0x0d,0x0f, -0x56,0xdd,0x35,0xcb,0x61,0xbd,0x58,0xd6,0xb7,0x4d,0x55,0xe7, -0x41,0x98,0xf6,0xb5,0xcd,0xe6,0x7d,0xb2,0xde,0xd5,0x57,0x8b, -0x94,0x5c,0x38,0x9b,0xcd,0xae,0xb0,0xa9,0x2f,0x56,0x5d,0xb7, -0x6a,0xeb,0x72,0xdb,0xf4,0x45,0xd5,0x5d,0xcf,0xaa,0xbe,0xff, -0xc3,0x55,0x79,0xdd,0xb4,0x1f,0x17,0x3f,0x75,0x37,0xbb,0xaa, -0xfe,0xea,0xa7,0x72,0xd3,0x7f,0xf5,0xe3,0xae,0x3b,0x13,0x8c, -0x65,0x12,0x5f,0x15,0xbf,0xcd,0x50,0xb6,0x4d,0x95,0x19,0x48, -0x16,0x5f,0xcf,0x58,0x9a,0xec,0xea,0x76,0x91,0xf6,0xc3,0xc7, -0xb6,0xee,0xd7,0x75,0x0d,0xc3,0x08,0x78,0x8b,0x74,0xa8,0x3f, -0x0c,0xa4,0xf9,0x37,0xdb,0x72,0xd1,0x2d,0xeb,0x60,0x0b,0xd9, -0xa0,0xf1,0x1d,0x4f,0x7f,0xf5,0xc9,0x61,0x45,0xbc,0xa6,0xcf, -0x65,0xb7,0xfc,0x98,0xfc,0x7b,0x2f,0xd2,0x67,0x5b,0x2e,0x97, -0xcd,0x66,0x75,0x96,0x48,0xb6,0xfd,0xf0,0xe6,0xd1,0xa3,0xeb, -0x72,0xb7,0x6a,0x36,0x67,0xc9,0xa3,0x07,0xff,0xd9,0x5f,0xad, -0xf9,0x81,0x2a,0x72,0x2a,0x8f,0x0e,0x9c,0x25,0x69,0x74,0x21, -0xa1,0x70,0x26,0x70,0x21,0xcd,0x92,0xf4,0xdb,0xba,0xbd,0xad, -0x87,0xa6,0x2a,0x93,0x3f,0xd5,0x37,0x35,0xee,0xec,0x6f,0x64, -0xc9,0xdb,0x5d,0x53,0xb6,0x59,0xd2,0x63,0x79,0xde,0xd7,0xbb, -0xe6,0xea,0xcd,0x53,0xdd,0x77,0x75,0xb3,0x5a,0x0f,0x67,0x09, -0xa2,0xf1,0xac,0xa5,0xe1,0x2b,0xc6,0x8b,0x37,0xc7,0x5d,0x7d, -0xc6,0xa1,0xe6,0x6a,0x07,0x9c,0x1d,0x38,0x75,0xd9,0xed,0x96, -0xf5,0xee,0x2c,0xd9,0x74,0x9b,0xfa,0xe1,0xa6,0xfd,0x65,0x79, -0xb0,0xa1,0xea,0xda,0x0e,0xeb,0xbf,0xe4,0xb5,0xa9,0x2b,0xfe, -0xd8,0x02,0x4a,0x4f,0xbe,0xac,0xab,0x6e,0x57,0x12,0xdc,0x4f, -0x69,0x3d,0x5b,0x77,0xb7,0xf5,0xee,0x40,0xf7,0x93,0xfd,0x37, -0x1b,0x18,0x07,0x78,0x1d,0x57,0x72,0xd4,0xa1,0x18,0xab,0xfc, -0xb2,0x1b,0x86,0xee,0xfa,0x30,0xed,0x4f,0x37,0x7f,0xf5,0x2a, -0x2d,0xcf,0x29,0xf9,0xf2,0xaa,0xeb,0x86,0x27,0x6e,0x84,0x42, -0x3d,0x4b,0x38,0x63,0xbf,0x3f,0x92,0xe5,0xff,0x13,0x82,0x46, -0x8b,0x87,0x6e,0x7b,0x16,0xf0,0x71,0xd4,0xde,0x60,0x33,0xc2, -0x5b,0x77,0x7d,0xde,0x76,0xab,0xee,0xc0,0xf0,0x7b,0x3d,0x64, -0x3d,0x30,0x56,0xde,0x0c,0x5d,0xc2,0xc2,0xcf,0x9b,0x27,0x2b, -0xd7,0x23,0x5c,0x9f,0x56,0xd6,0x83,0x20,0xb8,0x27,0x0f,0x0f, -0xcc,0x79,0x62,0x52,0x7f,0xbb,0x3a,0x62,0x56,0x08,0x5f,0xd3, -0xb6,0x40,0x9e,0xf7,0xfe,0xe9,0x71,0xd1,0xf4,0x0f,0xf9,0x7d, -0xec,0x8f,0x19,0x45,0x9f,0x65,0xd3,0x6f,0xdb,0x12,0x09,0x68, -0x36,0x84,0xac,0xfc,0xb2,0xed,0xaa,0xf7,0xc7,0x97,0x02,0xa1, -0x14,0xfb,0x36,0x07,0x35,0xae,0x10,0x93,0xeb,0x66,0xb9,0x6c, -0xeb,0xa7,0x6b,0x0f,0x23,0xbc,0xed,0xee,0xea,0x5d,0xbd,0xcc, -0x2f,0x0f,0xd9,0x28,0x78,0x41,0x20,0xe8,0x9b,0x4f,0x35,0xac, -0x14,0xc7,0x8c,0x9c,0x2a,0x4c,0x4a,0xf9,0x6c,0x58,0x9f,0x62, -0xeb,0x57,0xb8,0xf6,0x5a,0xb7,0x82,0x8e,0x29,0xcb,0xdc,0x1f, -0x3b,0x31,0x54,0xec,0xa8,0xa6,0x42,0xcb,0xaa,0x77,0x87,0xc9, -0xa6,0x7f,0xe7,0xb3,0x91,0xa8,0xe7,0xb3,0xd8,0x9a,0xe7,0xc4, -0xd4,0x23,0x87,0xaf,0x79,0xec,0xa4,0xef,0xee,0x3b,0x29,0x6e, -0xc5,0x67,0x63,0x6d,0xf6,0xbb,0x8a,0xba,0x41,0x89,0x9e,0x42, -0x23,0x40,0x9a,0x04,0x75,0x63,0x7b,0x1c,0x83,0xb1,0x87,0xa3, -0x0a,0xa9,0x4f,0xcf,0xe7,0xb3,0xb8,0xfb,0x88,0xaa,0xcb,0x5d, -0x77,0x87,0x12,0xfa,0x7c,0x65,0xcb,0xe6,0x36,0x69,0x96,0x8b, -0x34,0x96,0x7f,0x7a,0xdf,0x81,0xf6,0x4f,0xee,0x21,0x90,0x9e, -0xff,0x18,0xaf,0x13,0xc0,0x61,0x5e,0x3e,0x68,0x94,0x3d,0x3a, -0xe5,0xaa,0x19,0xd6,0x37,0x97,0xb1,0x3f,0x86,0x22,0x98,0x51, -0x30,0xd2,0x10,0x92,0xf9,0xac,0xc4,0xc1,0xd0,0x78,0x44,0xff, -0x83,0x8a,0x79,0x70,0x7c,0x58,0x52,0x3e,0xee,0xc5,0x71,0x25, -0x9d,0x70,0xb0,0x30,0x2c,0xa6,0x6a,0x03,0x1e,0x7a,0xd0,0xed, -0x22,0xe5,0x05,0x4f,0x83,0xf6,0xef,0xcb,0x8f,0xf5,0xee,0x1f, -0x10,0x3e,0x5c,0xb7,0x9b,0x7e,0xaf,0xea,0xee,0xee,0xae,0xb8, -0x93,0x45,0xb7,0x5b,0xcd,0x30,0x3d,0xb0,0x19,0x36,0x8f,0x4b, -0xce,0x3e,0xd0,0x10,0x70,0x6c,0x21,0x47,0xb9,0xce,0xc2,0x53, -0x2c,0x5d,0xa4,0x08,0x67,0x9a,0x7c,0x8c,0xbf,0x47,0x6b,0x2e, -0xd9,0xee,0x6a,0xe4,0xe6,0xb6,0x7e,0xdb,0x6f,0xeb,0x6a,0xf8, -0x33,0x75,0x82,0x45,0xfa,0xe1,0x87,0x66,0xf3,0x77,0x7c,0xd3, -0x84,0xa6,0xa4,0x3f,0x76,0xa4,0x09,0xc4,0x64,0xbd,0x00,0xfd, -0x98,0x34,0xa9,0x37,0x25,0x26,0xb6,0xfc,0xb2,0xac,0xde,0xaf, -0x76,0x1d,0xda,0x06,0x06,0xc4,0xfa,0x2e,0x79,0xb4,0x06,0x86, -0x9e,0xf5,0xdb,0xb2,0x42,0xae,0xa7,0x33,0x8e,0x45,0x64,0xf5, -0xf4,0xde,0xa9,0xfb,0xe1,0x19,0x26,0xc6,0x75,0xa0,0xa7,0x45, -0xfa,0xa5,0x96,0x6f,0xe5,0xbb,0xb7,0x69,0x02,0x13,0x7e,0xe0, -0xd2,0x14,0xdc,0xb8,0x4c,0xe9,0x42,0x0b,0x7b,0x61,0x4d,0xe1, -0xfc,0x24,0x65,0xc2,0x15,0xc6,0xf9,0xcc,0xcb,0xc2,0x4a,0x3f, -0x49,0x5c,0xcb,0x0a,0xb3,0x90,0x2f,0x84,0xd1,0x99,0x72,0x85, -0x60,0xb8,0xc7,0x6c,0xa1,0xac,0x1a,0x7f,0xf7,0xf2,0xb3,0xe6, -0xd0,0xa7,0x22,0x15,0x5a,0x64,0x2c,0xae,0x36,0x3a,0x7f,0xa8, -0x0c,0xe2,0xa8,0xe4,0x42,0x28,0x59,0x18,0x29,0x27,0x33,0xb8, -0xd7,0x85,0x12,0x93,0x89,0x8f,0xed,0xff,0x94,0xce,0x5e,0x19, -0x83,0x6f,0xb8,0x61,0xe6,0xdd,0xe3,0x18,0x68,0x5d,0x48,0xe7, -0xab,0x9c,0xdb,0x42,0x38,0x99,0xb1,0x5c,0xf2,0xc2,0x2b,0x9e, -0x09,0x5b,0x18,0xa5,0x73,0x89,0xdb,0x52,0x67,0xc6,0x14,0xcc, -0x60,0x15,0xc3,0x1e,0x58,0x83,0x7b,0x06,0xd7,0xa0,0xc2,0x4c, -0x14,0xca,0x59,0x5c,0x2b,0xeb,0x32,0xd8,0xaa,0xcc,0x69,0xff, -0xb1,0xd2,0x0a,0x99,0x41,0x9d,0xb2,0x39,0xf4,0x70,0x99,0x71, -0x01,0x45,0xf0,0xbc,0x50,0xda,0x65,0xdc,0x15,0x8e,0xb9,0x70, -0x10,0x47,0x2e,0x0a,0x6f,0x04,0x5d,0x5b,0x9d,0xe9,0xc2,0x6b, -0x33,0x5e,0xbb,0xc2,0x3b,0x87,0x84,0xc8,0x82,0x09,0x9f,0xb1, -0x82,0x61,0xa7,0x19,0xaf,0xa7,0xe7,0xfa,0xb4,0x21,0xac,0x10, -0x0a,0x7e,0xc1,0x6e,0x64,0x03,0x31,0xd6,0x64,0x87,0x35,0x48, -0xe7,0xde,0x0e,0xce,0x2b,0x57,0x70,0x46,0xb7,0x98,0x82,0xd7, -0x06,0xcf,0xc9,0x79,0x6b,0x10,0x1e,0x0d,0x3b,0xc9,0x40,0xe6, -0x7d,0x25,0x0b,0x8f,0x6b,0x1c,0xad,0x4c,0x66,0x0b,0x17,0x34, -0x0a,0xc4,0x86,0xf3,0xf1,0x01,0x84,0xd3,0xd6,0x28,0x18,0x8f, -0xe0,0x67,0x0e,0x0e,0x32,0xe7,0x68,0xa7,0xf3,0x3a,0x0f,0x3b, -0x2b,0x4f,0xc7,0x87,0x70,0xe3,0x81,0x2d,0x2c,0x73,0x08,0x16, -0xd3,0xc1,0x06,0xee,0xf3,0x68,0x03,0xc3,0x86,0x10,0x2a,0x4e, -0xf6,0x5a,0x1b,0xf6,0x02,0xbf,0x85,0xd1,0x61,0xaf,0x71,0xa7, -0x2d,0x90,0x48,0x9e,0x56,0xb9,0x42,0x0a,0x33,0x1d,0x92,0x9e, -0x73,0x8d,0x1f,0x31,0x49,0x00,0x84,0x12,0xee,0x42,0xd0,0x42, -0x6b,0x32,0xef,0xa3,0xa1,0x1e,0x06,0x5b,0x3e,0xc2,0x28,0x7b, -0x8c,0xaa,0x5f,0x81,0xcc,0xf0,0x19,0x91,0x89,0x7a,0xd4,0x40, -0x20,0x87,0x7f,0x3a,0x82,0xce,0x5b,0x4f,0xf8,0x10,0x8e,0x9c, -0x15,0xe4,0xb2,0xf5,0x26,0x97,0x08,0xb8,0x83,0xc5,0xc8,0x53, -0xee,0x02,0x3a,0x38,0x2b,0xb4,0xb6,0xb9,0x10,0x85,0xf0,0x36, -0x40,0xda,0x44,0x08,0x0b,0x35,0x4a,0x2f,0xa0,0x53,0x15,0x0e, -0x41,0x63,0xb9,0x27,0xd5,0xc8,0xa5,0x62,0x50,0x13,0xb3,0x08, -0x44,0x78,0x46,0x85,0x82,0x70,0x9b,0x4c,0x15,0xda,0xe1,0x3c, -0x94,0x8a,0xcd,0xfc,0x98,0x06,0xe5,0x01,0x22,0xd4,0xa9,0x0f, -0xf8,0x55,0x08,0x1d,0x56,0x69,0x8e,0x6b,0x03,0xd6,0xf0,0x85, -0xf4,0xd3,0x35,0x57,0x10,0xfc,0x49,0x5b,0xfa,0x08,0x21,0xe8, -0x56,0x84,0xea,0xfd,0x2e,0x53,0x91,0x52,0x95,0xc5,0x03,0xa0, -0xd4,0xd9,0xf0,0x58,0xd1,0x63,0xc5,0x44,0x14,0x2a,0x5d,0x30, -0xb0,0x0a,0x0b,0xfb,0x45,0xc0,0x82,0x18,0xf7,0x07,0x13,0x5e, -0x60,0x29,0xc4,0x91,0x71,0x32,0x96,0x3c,0x02,0x03,0x6a,0x09, -0xf7,0x1c,0x0a,0x4b,0x00,0x1f,0x56,0xe7,0x70,0x12,0x41,0xa7, -0x22,0x36,0x78,0xa0,0x19,0xc7,0x39,0x06,0x31,0xd6,0x85,0xe1, -0x7a,0xbc,0x0e,0x35,0x78,0xc1,0x9d,0x42,0x0d,0xa3,0x14,0x14, -0x14,0x90,0x0d,0x0e,0xc8,0xa6,0xfa,0x02,0x8c,0x24,0xe1,0xe7, -0x71,0xb2,0x3f,0x3d,0xd3,0x7d,0xc6,0xcf,0x33,0x70,0x9a,0xcf, -0x7e,0x6b,0x77,0x10,0x92,0x7f,0x23,0x58,0xc4,0x9f,0x54,0x40, -0x81,0x87,0x49,0x8c,0x88,0x45,0x56,0xe0,0x05,0xed,0x34,0x62, -0x09,0x58,0x39,0x6e,0xc0,0x19,0x1c,0x85,0x28,0xa8,0x17,0xc0, -0x1f,0x50,0x90,0x11,0x2d,0x21,0x8f,0x51,0x0e,0x78,0x21,0xc5, -0x4b,0x14,0x88,0xda,0x40,0x40,0x75,0xc1,0x51,0xb0,0xd8,0x00, -0xc2,0x42,0xb0,0x00,0x9e,0x9c,0x53,0xa5,0xf3,0x28,0x00,0x68, -0x66,0x04,0xa3,0x20,0x70,0x53,0x08,0x51,0x5a,0x93,0x40,0xa0, -0xf6,0x1c,0x14,0x28,0x64,0xa1,0x54,0x20,0x0c,0x62,0x66,0xdc, -0x36,0xcc,0x51,0xd7,0x42,0xac,0xa3,0x70,0xda,0x18,0x00,0x5d, -0x38,0x6a,0x47,0x3a,0x10,0x0e,0xd5,0x36,0xb2,0x0c,0x02,0xe1, -0x96,0x7c,0x12,0xc2,0xb5,0xc1,0x35,0x3b,0x9e,0x40,0x95,0x06, -0xb6,0x46,0x8e,0xb5,0x77,0xc1,0x60,0x22,0x25,0xe4,0x18,0x7b, -0x03,0x11,0x45,0xa1,0xca,0x05,0xc2,0xc4,0xc9,0x7a,0x05,0xda, -0x94,0x3c,0x27,0x42,0x75,0x93,0xa0,0x89,0xb8,0x4f,0xd7,0xe2, -0x05,0x2a,0xb8,0x90,0x38,0x56,0xe0,0x78,0x69,0x32,0x09,0x05, -0x4a,0xab,0x29,0x2b,0xd9,0x41,0x96,0x5e,0x4f,0x35,0x0f,0x53, -0xad,0x10,0x63,0x6f,0x22,0xfa,0x2c,0xe3,0x15,0xfc,0x91,0xa1, -0x3b,0x83,0xc6,0x9c,0x08,0x15,0x63,0xbd,0x9c,0x24,0x05,0x76, -0xf1,0x96,0x62,0x0e,0x9c,0xab,0xc0,0x2f,0x0a,0xd4,0xa7,0xa8, -0x14,0x7d,0x7e,0xbf,0x08,0xd2,0x0b,0x08,0xc0,0x39,0xdc,0x5b, -0x6a,0xb6,0x06,0xc5,0x98,0x87,0x63,0xc4,0x28,0x44,0x05,0x17, -0xd2,0x81,0xb4,0x51,0x58,0x9c,0x64,0xaf,0x32,0x89,0x70,0x84, -0xf6,0x14,0x4d,0xcd,0x0e,0x4c,0xff,0x94,0xdc,0x3b,0xe3,0x89, -0x96,0x5e,0x60,0x3a,0x02,0x2a,0x35,0x1c,0x44,0x55,0xe8,0x31, -0xd3,0xa3,0x10,0xbd,0xab,0x18,0xf1,0xbf,0x46,0x23,0x81,0x56, -0x91,0x87,0x7e,0xa7,0xf2,0xfd,0x0a,0x08,0x3d,0x49,0xcc,0x53, -0x4a,0xb0,0xdb,0x4e,0x52,0x7c,0x78,0x3a,0xb7,0xd2,0x03,0x0d, -0x84,0x29,0x17,0x90,0x86,0xa0,0xe1,0x4c,0xb9,0x37,0x3d,0x3b, -0x70,0xe5,0x33,0x93,0x8b,0x59,0xcb,0x5a,0x8a,0x90,0x45,0x01, -0xfb,0x35,0x01,0x53,0xc9,0x16,0x43,0x87,0xa1,0xb2,0x35,0xce, -0xad,0x41,0x85,0xc2,0x83,0x2b,0x35,0x0b,0xbd,0x83,0x58,0x1e, -0x0f,0x40,0xd3,0xd1,0x75,0x2a,0x04,0x67,0x27,0xe9,0x85,0xfe, -0x59,0x78,0x49,0x85,0x84,0x89,0x42,0x6b,0x62,0x67,0x4c,0x01, -0x61,0x86,0x40,0x37,0xd6,0x56,0xb5,0xb1,0x5b,0x51,0x9d,0x59, -0xa9,0x50,0x1e,0x85,0xa1,0x16,0x6d,0x5d,0x68,0xb9,0x3c,0xcc, -0x42,0xe8,0x5c,0x08,0x21,0x77,0xf1,0xba,0xca,0x89,0x68,0xa8, -0x82,0x60,0x93,0xc6,0x85,0xa2,0x89,0x25,0x66,0xc0,0xc4,0x7a, -0x95,0xb7,0x0a,0x34,0xc0,0x4e,0xcf,0x16,0xeb,0x98,0x18,0xf1, -0xd7,0x31,0x0e,0x9f,0x17,0x4c,0x70,0x34,0x98,0xe2,0xbe,0x52, -0x50,0x95,0x3e,0x10,0xc7,0x38,0x2e,0xc2,0x1e,0x8c,0xae,0x93, -0x14,0x90,0x6c,0xc3,0x80,0x86,0x61,0x8a,0xc6,0x0f,0x4c,0x25, -0xb8,0x07,0x04,0x91,0xcf,0x36,0x0b,0x14,0xb7,0xce,0x15,0xf8, -0xe3,0x85,0x5a,0xa1,0xe8,0x51,0x1d,0x32,0x26,0x09,0x68,0x12, -0x83,0x36,0x65,0x05,0x5c,0x17,0xe8,0x98,0x8f,0x12,0x9a,0x91, -0xe1,0x01,0xd0,0xe8,0xfd,0x96,0xda,0x15,0x23,0xab,0x2c,0x99, -0x43,0x6d,0x09,0x1c,0x4d,0xe1,0xd4,0xaf,0x63,0x68,0x30,0x0f, -0x3a,0x1f,0x08,0xc6,0x3a,0xa2,0x61,0x83,0x79,0x07,0xb4,0x07, -0x4d,0x60,0x35,0x6a,0xbb,0x41,0x40,0x12,0x31,0x93,0x60,0x3c, -0x44,0x19,0x23,0xc2,0x2e,0x1f,0x69,0x30,0x5c,0x8f,0x55,0xac, -0x19,0xb5,0x35,0xaa,0x62,0x0a,0x5f,0xa6,0x85,0x0c,0x83,0xde, -0x54,0xc4,0x07,0x51,0xfd,0x74,0xd2,0xac,0xe4,0x07,0xcc,0x36, -0x64,0x3c,0x82,0x0d,0x9e,0xb2,0x86,0x4a,0x94,0xbc,0xb5,0xd4, -0x4b,0x9c,0x27,0x56,0x31,0x11,0xad,0x56,0x8a,0x51,0xaa,0x68, -0x94,0x11,0x9a,0x50,0x84,0x92,0xf2,0xd4,0xbd,0xac,0x71,0x04, -0x09,0x0a,0x6b,0x58,0xf2,0xed,0xa8,0xf6,0xf3,0xde,0x23,0x0c, -0xf8,0xd7,0x60,0xec,0xe0,0x1a,0x90,0x66,0x44,0x1b,0xe8,0x03, -0x16,0x93,0x38,0x4d,0xaa,0x4e,0x13,0xd5,0x83,0xaa,0x33,0x45, -0x23,0x9b,0x1f,0xa5,0x0a,0x63,0x84,0x33,0x94,0x2d,0xdc,0x36, -0xf4,0xb2,0xc1,0xc2,0xe0,0x30,0x4a,0x71,0xd1,0x0b,0xe3,0x7b, -0x86,0x63,0x1c,0x39,0x8d,0x63,0x50,0xa9,0x1a,0xcd,0x51,0xe6, -0x93,0x02,0x12,0x2e,0x0c,0x9c,0xa4,0xe9,0x48,0x30,0x6a,0x3a, -0x36,0xdb,0x5b,0xea,0x68,0x46,0x33,0xd9,0x81,0xe5,0x60,0x50, -0x63,0x09,0x97,0xd3,0x8d,0x17,0x0c,0xa0,0x4e,0x6c,0xc0,0xfb, -0xd4,0x96,0xf5,0xbe,0xb3,0x49,0x0c,0x5e,0x7e,0x14,0x02,0x40, -0x98,0x0b,0x00,0xc1,0x28,0xad,0x02,0xc1,0x32,0x2f,0x27,0x69, -0x5c,0xc5,0xa8,0x5d,0x63,0x9e,0x8d,0x0d,0x1e,0xa1,0x42,0x99, -0x02,0x13,0xd3,0xa2,0x20,0x9d,0xa6,0x53,0xcc,0x73,0x60,0x73, -0x01,0x4f,0x59,0x68,0xe7,0x93,0x1f,0x8e,0xf0,0x29,0xb3,0xc7, -0x6e,0xbd,0x3e,0xcd,0x78,0x65,0x56,0xef,0xfc,0x98,0x66,0xcc, -0xfb,0x96,0x5a,0xbd,0x33,0x84,0xa7,0xca,0xd2,0xa0,0x1d,0xcb, -0x98,0xbc,0x52,0x5c,0x4c,0x45,0x2a,0xd1,0x25,0x85,0x1f,0xa5, -0x2a,0x82,0x31,0x63,0xe3,0x0b,0x53,0x98,0x6d,0xfc,0x24,0x20, -0xed,0xd6,0xbc,0x18,0x67,0x60,0xdc,0x33,0x15,0x76,0x86,0x29, -0x9e,0xeb,0xd0,0x87,0xf0,0x1a,0x99,0xc7,0x41,0xa5,0x8d,0x90, -0x37,0x28,0x5b,0xe6,0x44,0x78,0x17,0x30,0xe8,0xc7,0x20,0x64, -0x43,0x47,0x78,0xa9,0x88,0xdc,0x25,0xc0,0x38,0x4a,0xe1,0xf8, -0x17,0x06,0x23,0x16,0xca,0x59,0xd1,0x00,0x12,0x8e,0x17,0x04, -0x1a,0x41,0xde,0x31,0xbc,0x50,0x45,0x29,0x8e,0x86,0x2e,0x8e, -0x86,0x06,0x53,0xad,0x0e,0x73,0x3a,0x8d,0x68,0x14,0x6f,0xbc, -0x11,0x48,0xdd,0xe6,0xc4,0xdb,0x2a,0x92,0x09,0xcd,0x7e,0xde, -0x89,0x9c,0x9a,0x84,0xa0,0x03,0xa8,0x4e,0x11,0x84,0x9c,0xe6, -0x6a,0x16,0xfa,0xf0,0x0b,0xe1,0xc8,0x63,0x83,0xa2,0xa9,0x30, -0x4e,0xa3,0xf4,0x72,0xe6,0x27,0x81,0xca,0xda,0x68,0x20,0x8a, -0xde,0x59,0x68,0x3a,0x0d,0x6e,0xd3,0x9b,0x2b,0x21,0x8a,0xd8, -0x10,0x93,0x14,0xaa,0x1e,0x93,0x79,0x1b,0x62,0x46,0xaf,0xcd, -0xfe,0x05,0x78,0xd1,0xac,0xa0,0x9c,0xc9,0x68,0x60,0x1d,0x67, -0xc3,0xf0,0x6a,0x8b,0x21,0x61,0x92,0xa8,0xb3,0x48,0xdf,0xe7, -0x61,0x1a,0x0e,0x7f,0x3b,0x70,0x78,0xef,0xc1,0x88,0x2f,0xd8, -0x24,0x55,0xf9,0xd8,0x22,0x18,0x91,0x28,0x46,0x5e,0x8a,0x01, -0xd6,0x20,0xc8,0x14,0x04,0xea,0xb0,0xe6,0xfb,0x03,0x94,0xfd, -0x0f,0xdf,0x06,0x9e,0xb9,0xd9,0xdf,0x1e,0xdc,0xa6,0x3f,0xee, -0xdd,0xff,0x59,0xef,0xfe,0xaf,0x7c,0xe3,0xe5,0x7c,0x16,0xff, -0x44,0x3a,0x9f,0xc5,0xff,0xe7,0xfd,0x6f,0x00,0x00,0x00,0xff, -0xff,0x81,0x9d,0x08,0x43,0xf8,0x1d,0x00,0x00, - })) + var empty [0]byte + sx := (*reflect.StringHeader)(unsafe.Pointer(&_index_html)) + b := empty[:] + bx := (*reflect.SliceHeader)(unsafe.Pointer(&b)) + bx.Data = sx.Data + bx.Len = len(_index_html) + bx.Cap = bx.Len + + gz, err := gzip.NewReader(bytes.NewBuffer(b)) if err != nil { panic("Decompression failed: " + err.Error()) } - var b bytes.Buffer - io.Copy(&b, gz) + var buf bytes.Buffer + io.Copy(&buf, gz) gz.Close() - return b.Bytes() + return buf.Bytes() } + func init() { go_bindata["/index.html"] = index_html } diff --git a/mod/dashboard/resources/scripts-browser-modules.js.go b/mod/dashboard/resources/scripts-browser-modules.js.go index d48ef578ca1..dd5b097d539 100644 --- a/mod/dashboard/resources/scripts-browser-modules.js.go +++ b/mod/dashboard/resources/scripts-browser-modules.js.go @@ -4,9505 +4,36 @@ import ( "bytes" "compress/gzip" "io" + "reflect" + "unsafe" ) +var _scripts_browser_modules_js = "\x1f\x8b\x08\x00\x00\x09\x6e\x88\x00\xff\xdc\xbd\xeb\x76\x1b\x47\xd2\x20\xf8\x7f\x9e\x82\xa8\xf6\xc0\x55\x42\x12\x22\x64\xbb\xe7\xeb\x82\x8b\x18\x59\x92\x3f\x6b\xc7\xb7\xcf\xd2\xd7\x97\x05\x21\x1d\x90\x28\x8a\x25\x81\x00\x8c\x02\x45\xc9\x04\xfa\x59\xf6\x3d\xf6\xc7\x9e\xb3\x2f\xb4\xaf\xb0\x71\xcd\x5b\x15\x20\xc9\xee\x99\xb3\x67\xbb\x8f\x45\x54\x5e\x23\x23\x23\x23\x23\x23\x23\x22\x3b\x97\x37\x8b\x8b\x4d\xb5\x5c\xa4\xb7\xd5\x62\xb6\xbc\x35\x37\x8b\x59\x79\x59\x2d\xca\x59\x76\xa7\x59\x47\x55\xfd\x70\xbd\x9e\xbe\x9f\x57\x6f\xca\x74\x79\xfe\x3a\xbb\x7b\x3b\x5d\x1f\xcd\xcb\xc5\xab\xcd\x55\x01\xdf\x7d\xfe\x69\x36\xef\x57\x65\xf1\xfa\x3f\x6e\xca\xf5\xfb\x3e\xfe\xa6\xb2\xc3\x75\xb9\xb9\x59\x2f\x8e\x24\xbd\xaa\xff\x46\xfd\x50\xde\xa8\x33\xc8\x07\x45\x41\x6d\x2c\x96\xb3\xf2\x39\x54\xea\x76\xb9\xb5\x51\xe7\x24\x4f\xa6\xd8\x6d\x02\x25\xb0\xb9\xed\x36\x51\x88\x92\x0e\x27\x75\xbb\xe9\x09\xe4\x72\x0d\xc8\x5f\xdc\x5c\x9f\x97\xeb\x84\x33\x97\x97\x02\xa3\x36\x79\x7a\xa2\xbf\x8e\x07\x47\xd5\xe2\x08\x41\xd8\xd9\x41\x5e\xac\xcb\xe9\xa6\xfc\x69\x85\x1f\x75\xba\xe4\xbf\x3c\x54\x28\x58\x5e\x6c\x0a\x49\x7b\x34\xbd\xb8\x2a\xc7\xf2\x31\x29\xee\x76\xd1\x10\x4b\xc8\xd7\xfa\xfd\xeb\xe9\x06\xbe\x2e\x96\xeb\xf2\xe5\x7a\xb1\xdc\xdc\x5e\x55\x9b\x32\xdb\x6e\xc7\x13\x63\x11\xff\xd2\x5c\xce\xa7\xaf\xb2\x3b\xee\x65\x8c\x1f\x93\xa2\x73\xb2\xcb\x0c\xa7\x38\x10\xab\xc5\xa6\x5c\x2f\xa6\xf3\xc7\xd3\xcd\x34\x2d\xe7\xe5\xb5\x59\x4c\xaf\x4b\x33\x83\x4f\xb3\x7a\xbb\xc9\xee\xaa\xcb\x54\x80\x98\x5e\x5c\x94\xab\x8d\x2d\x98\xf1\x40\x36\x57\x15\x83\x6f\x00\x64\xa3\xcd\xfd\x8f\xf2\xbd\x4e\x5b\xf9\x6e\x35\x85\xe9\x31\xaf\xca\xcd\x37\xef\x7f\x84\xc6\x8b\xa4\xde\xac\xab\xc5\x2b\x87\x53\xea\xb2\xaa\x7f\x84\xf9\x2a\xb0\x69\x3b\x73\xe6\x02\x5b\x2e\x38\x6b\x24\x0d\x52\x5a\x4e\xb0\x56\x33\xcd\xc3\xcf\xb1\xd7\xfb\x24\x6f\xa4\x74\xbb\xde\xd7\x10\xc6\x55\xcd\xba\x5d\x6a\x6c\x5c\xcd\x20\x37\x85\xf1\x6e\xb7\x36\xa1\x8f\x38\x00\xbc\x76\x2c\xe0\xdb\x2d\x26\x01\x9d\x38\x82\x96\x69\xaa\x66\xdb\x6d\xba\x0f\x92\x02\xa0\xa4\xd9\x9a\x41\xd6\xa6\x9c\x3d\x9d\xd5\xfd\xd5\x72\x95\x42\xdb\x32\xa2\x57\x37\xd5\xac\xd7\xcb\x71\x34\xae\x5e\x66\x2c\x28\xd0\xb8\xfd\x0d\xc4\x21\xa8\xf2\x53\xfb\x9b\xe5\xff\xf6\xec\xa7\x1f\x15\xe7\x8b\xe5\x72\x95\x65\x26\x4d\x78\xba\x43\x4c\xfb\x44\x1f\x64\x64\x8c\x83\x91\xeb\xcb\x4e\xe1\xa6\x5c\xcc\x5c\x6f\x44\x23\x59\x1e\xa2\x6a\x5f\x61\xca\xe4\x1a\x99\xb1\xd4\x52\xb8\xc6\x08\xed\xa9\xcd\xa1\xf2\x8d\x04\x18\xb6\x5f\x3b\xcc\xcc\x4c\x3c\x31\x30\x12\x5b\x64\x6c\xe9\xe6\xba\x9c\x3f\x9a\xd6\x65\x4a\xc0\x4c\x0a\xae\x6a\xa7\x77\x94\xc2\x6c\xba\x96\xc7\x58\x0a\x86\x7a\x33\x9f\x17\x05\xe4\x40\x93\x61\xfe\x9e\x66\xb3\x2c\x0f\xca\xe1\xba\xd8\x35\x17\xdc\x2f\xe5\xf5\xf2\x6d\x19\x2d\xbb\x8f\x5a\x71\x95\x99\x3b\x4c\xfc\x2b\x96\x4d\xb8\x50\x27\x79\xf8\x8d\x2b\xc5\x4e\x16\x41\x87\xa0\xfa\x08\x2e\x02\xa2\x89\xc8\x02\x80\xb6\x7c\x9a\x58\x3e\xa3\x69\x84\xff\x16\xf8\x4f\xff\x62\xb9\xb8\x98\x6e\x74\xcc\xd7\xd3\x15\x95\x30\x31\x7a\x01\xaf\x98\x8e\x5c\xd6\xf6\xcc\xad\xf0\x4c\xe5\x54\xad\x68\x9f\x15\x63\xbb\x0b\xeb\x4b\x55\x82\xa3\x5e\xcd\xab\x4d\x9a\x1c\x25\x59\x36\xbc\x5c\xae\xd3\xaa\x38\x31\x73\x06\x91\x59\xfc\x70\x7e\x5a\x0d\xab\x5e\x2f\xe3\x85\x7c\x14\x92\xca\xb8\x9a\x4c\x10\x57\x1d\x5a\x43\x55\xfd\xe4\x7a\xb5\x79\x8f\x73\xf7\x13\xad\xc1\xdc\x62\x81\x32\x38\x31\x73\x48\xcc\x84\x97\xec\x98\x0d\xa5\xd2\x47\xb4\x8c\x1a\xcd\x7a\x53\x93\xe1\xfa\x8d\xe6\x7c\x5e\x4e\x17\x44\x3f\x63\x9c\xe9\x89\xe9\x9c\x64\x0a\x48\x7d\xb3\x5a\x2d\xd7\x9b\x3e\x77\xf4\x84\x27\x5b\xf8\x5f\x87\x97\x67\x9f\x37\xf0\x51\x0c\x8b\x9b\xe2\x02\x97\x47\xb6\xf3\xe8\x1b\xc1\x7c\xb8\xd9\xac\x99\xaa\xdf\x94\xef\x69\x6d\x12\xdd\xd0\x3a\x0e\x16\x29\xee\xd4\x01\xe5\x32\x85\xd3\x64\x25\x58\xfc\x38\xe9\x41\x13\xfd\x75\xb9\x9a\x4f\x2f\xca\x74\x7d\x7d\x33\xdf\x54\x8f\xa7\xf5\x95\x49\x8e\x3f\x1b\x24\x19\xf0\xbd\xef\x97\xb7\xe5\x9a\x66\x3a\x1b\x6a\x27\xd4\x26\x2c\x6c\x04\xa4\x3a\xbf\xd9\x28\x15\x34\x76\x1e\x86\x6d\xb3\x7e\x7f\x47\xf5\x92\xcd\xfa\xa6\x44\xe1\x00\xbf\x48\x5c\xb8\x9c\xce\x6b\x2f\x65\x90\x27\x38\x62\x9b\x80\x1f\x79\x0f\x7f\xf6\x12\x9b\x48\xdf\xf9\xfa\x7c\x0d\x20\xf7\x37\x65\xbd\x21\xa0\x32\x9d\x94\xd5\x74\x5d\x97\xc8\xad\x39\x39\xc7\x7f\x77\x17\xb4\xa9\xc3\xf8\x77\x52\x6a\x66\x39\x83\xc5\xe1\xae\x04\x58\x08\x64\x87\xc3\x9d\xec\x40\xd4\x88\x27\x5d\xc5\x64\x62\x65\x2c\x44\x04\x91\xb7\x2e\x06\xcc\x01\xbc\xa5\x84\x6f\x94\x82\x78\x8f\xe8\xb4\xd1\x2b\x36\xc3\x4b\x06\x89\x2d\xe1\x4d\x47\xeb\x08\x01\x77\x06\x22\xbc\x80\xbc\x61\x01\xe2\x94\xe7\x80\xdd\x34\xbb\xdb\x97\xfd\x2d\xe2\xda\xe5\x0f\x5c\x7e\x5d\x9d\xcf\x61\xde\xd2\x8b\x9b\xb5\x99\x55\xeb\xec\x6e\xb6\x3c\x82\xdf\x05\xfc\x37\x86\xef\xc9\x10\x84\xa0\x79\x89\xd9\x40\x53\x00\x0e\xfc\x70\x24\xa5\xb2\x14\x24\xba\x16\x81\xb2\x17\x20\x32\x22\x82\xcb\xc5\xa6\x36\xbf\xde\x4c\xe7\xd5\x65\x55\xae\x01\xdd\xe5\x8a\xc8\xd5\x26\x15\xf6\xd7\x76\x7b\x62\x2c\x5e\xbe\x55\x69\xcb\x66\xeb\x1a\x56\xb1\xed\x15\xd0\xad\xeb\xc2\x4a\x67\xcc\x83\x79\x3a\xa0\xc2\x5f\xa7\xf3\xa2\xd3\xb1\x8d\x00\xeb\x9a\xcf\xa5\x8c\x21\xb6\xaf\x03\x90\xb2\x45\x81\x20\xee\x88\xda\x5d\x2d\x3b\xdc\x8f\x06\x41\x11\x7d\x84\x1f\xd0\xaa\x1b\xaf\xdf\x43\x63\xc9\xb8\xd1\xd2\x00\x2e\xab\x39\x6c\x68\xe5\xac\xf8\x84\x0e\x1b\xcb\x9e\xbb\xaa\xea\x67\xd5\xf5\x6a\x2e\x6b\x66\x2f\x56\xb9\x47\x97\x6f\x14\x04\xd3\xa1\xb9\x1b\xba\x81\x7c\xa8\x46\xb6\xfb\x74\x6c\xe9\xfc\x2f\x78\x37\xa3\x89\x72\xb0\x9e\x16\x27\x16\x7d\xf1\x41\xe0\xd9\xf4\xb2\xfc\x76\x3d\x7d\x85\xad\xa7\xb3\xe5\xc5\x0d\xfe\x90\x93\x4f\x55\x6f\x0a\xc4\x06\x0a\x22\xb5\x6e\x45\xdb\x24\x33\xb5\x54\x2a\xb4\x42\x9f\x1b\x7b\x2c\x9f\xb6\x41\x42\xa1\x96\x96\x42\x4f\x78\x28\x19\xae\xf7\x21\xf6\xa1\x3b\x59\xd6\x5e\x30\xa5\x32\x24\x9c\x5a\xaa\xd3\x92\x6e\x34\xc0\x78\x66\x3f\xad\x1f\xae\x56\x28\xe7\xd1\xf8\x37\x78\xd8\xf0\xa8\x09\x79\xaf\x34\x59\x7f\xf3\xfe\xf9\xf4\x15\x0e\x2b\xc5\x52\xe3\x13\x90\x66\xa9\xc8\x94\xea\x3f\x82\x85\xcb\x8d\xf4\x97\xb7\x8b\x72\xfd\x38\x1c\xa5\xc2\x85\x55\x3d\x7c\xce\xaa\x7a\x7a\x3e\x2f\x9f\x5d\xac\xab\xd5\x46\xa6\x07\xd1\x38\x05\x7e\xdf\x64\xfe\xb8\x23\xa6\x09\x92\x6f\x62\x47\x45\x85\xe8\x74\x99\x62\xa5\x6e\x17\xff\x05\xbc\x97\x17\x38\x8f\xb3\xac\x97\xdc\x4f\x7a\xb6\x10\xad\x43\x9f\x5d\xd5\x1b\x10\xe8\x1b\xdd\xd3\xc9\xac\x58\xd7\x94\x8e\x74\xfd\xc3\xb4\x7e\x53\xce\x40\x92\x2a\x2f\x52\xdb\x98\x85\x81\x8a\x8f\x1c\x24\xf4\x3d\x1e\xf0\xd1\x05\xf6\x3c\x14\x10\xdd\x0e\x26\x03\x88\x40\xa9\xcb\xcd\xbf\xcf\x97\xe7\xd3\xf9\x93\xb7\x53\xe6\x1a\x35\x08\x9c\x97\x8a\x7c\x38\x71\xc3\xd4\x23\x6c\xcc\x50\x8a\x93\x21\x6e\x59\x9d\x82\x8a\x12\xaa\x6a\x10\x5c\x32\x12\x6b\x84\xb2\x5f\xba\xad\x27\x79\x65\xdb\x4e\x4c\xc7\x6b\xd7\x1e\x5d\xb8\xb0\x97\x03\xad\x05\xd5\xfc\x69\xbb\x98\x2f\x17\xe5\xa3\xe5\xea\xfd\x93\xb7\x38\xa9\xf5\xfa\xc2\xcc\x00\x95\xc4\x6d\x91\x27\xe0\x87\x77\x6a\x97\x2e\xae\xa6\x35\x89\x30\x50\x5c\xcf\x9c\x38\x25\x28\x06\x2f\xe7\xb3\xc7\xde\xd1\x83\x81\xc1\x72\x06\x98\x7d\x33\x07\xdb\xd7\x3a\x80\x48\x04\xa2\x2e\xe4\xbb\xcf\x9f\xb8\x88\xf8\x17\x6c\x32\x22\xf6\x70\x53\x00\xc7\x62\x36\x2f\xb5\x65\x29\x8f\x67\x75\x44\x31\x82\x84\xbb\xa9\xd4\x75\xf2\x23\x27\x8c\x31\x7f\x12\xcb\x91\x2a\x68\x63\x91\xfe\x74\x36\x63\x00\x99\xe0\xbc\x6a\x38\x41\x3b\xed\x16\x07\x02\x72\x9e\xff\x19\x9d\xbc\xe0\x80\xe8\xe7\xc2\x04\xf8\x4b\xf7\xdd\x23\x9c\x04\x5c\x12\x4f\xeb\xfa\xa6\xac\xbd\x59\x20\xf1\x40\x78\x90\x61\x1d\xc0\xb0\x6d\x66\xf8\x10\x20\x05\x5d\x16\x7e\x85\xf2\x98\xe9\x44\xb2\xe6\x62\x49\xbd\xd3\xf4\x77\xbb\x58\x33\x3e\x7c\x64\x77\xb3\xd6\x69\x63\x91\x9c\x50\x3c\x73\xd8\x57\x0c\xf2\x5a\x61\xaa\x22\x1c\x32\xf4\x32\x65\xd9\x90\x60\x8c\x17\x54\xd8\x73\xb6\x4b\x78\xdd\xa2\x18\xa7\xc3\x61\x18\x61\x3f\x7a\xb7\x01\xa9\x02\x30\x45\x3f\x47\x69\xc8\x7f\x08\x3e\xca\xb1\x45\x4c\xc8\x22\xa8\x44\x96\xbb\xe3\xb8\xed\x61\x44\x79\x28\x13\x02\xec\x38\x2b\x30\xb7\x94\xb2\x04\x10\xd7\xdf\x3d\xff\xe1\x7b\x6a\xd3\x7e\x65\x26\x42\xe9\xd5\xe6\x7a\xfe\x15\x61\xb5\xdb\xc5\x92\x20\xd7\x70\xc9\x6e\x57\xb1\x0f\x03\xbe\xe6\x46\x6d\x66\xa6\xdd\xd8\x94\x22\xa8\x8c\xc0\x56\x8b\xd5\x4d\x84\x8d\xeb\xe9\xa2\x5a\xdd\xcc\xa7\x48\x4b\x2f\xd7\x70\x06\xb8\x78\x83\x78\x40\xa2\xe0\x4d\x9b\xc6\x8f\x24\x22\xe3\x02\x29\x75\x0a\x22\xfb\x23\x2c\x09\x32\x02\xa5\x5d\xc8\x07\x96\x95\xdf\x44\x80\x7d\x60\x17\x37\xa5\xe0\x99\x7e\x2b\x90\xf4\xe1\x92\x09\x93\x2b\xd1\x5f\x38\x4c\xfa\x1d\x3e\x83\xa5\x7b\xb1\xd1\x1e\x6b\xfd\xc2\x26\xa2\x12\x79\xda\x1c\xe8\x76\x9b\xe0\x24\xc2\x9c\x4c\xfd\x64\x8b\x34\x69\xe2\xaf\x16\x2c\x3f\xc1\x5f\x6e\xb8\x0b\x81\x30\x07\x27\x5c\x22\x0a\xda\x26\x95\x19\xd7\x46\x59\xb2\xb9\x5c\x82\x3c\xaf\xf2\x95\x14\x6e\xdd\x42\x51\xb4\x45\x4d\x12\xc8\x64\xf6\x04\x30\x3a\x54\x01\x37\x4e\x18\xcd\xbd\x24\xcb\xa3\xe6\x7f\x45\xd2\x60\x24\x2c\xd7\x0f\x71\x4b\x38\xd0\x74\x5c\xd8\x6b\xd6\x16\xa6\xb3\x2f\x8d\x84\xb8\x1f\x8f\x69\x3c\xa1\x51\xd6\x85\xb6\x74\x81\xfb\x3d\xd2\x39\x6c\x21\x92\x76\x60\x47\xea\x50\x3f\x56\xa9\xc5\xd3\xe0\x44\x8e\x11\x75\xd2\x5f\xdd\xd4\x57\xbc\x0b\xeb\xd9\xf6\xba\x5c\xbf\x2a\x19\x04\x23\x73\x60\x2b\xd9\xed\x17\x3e\xfc\xd3\xe8\x76\x0b\x09\x76\xd3\xb1\x9d\xf9\x73\x37\x0a\x9a\x1f\x4b\xd6\x84\xe7\x2f\xcb\xe9\x4f\xc0\x6b\x1f\x07\xe4\x2f\x92\xc2\x07\x17\x91\x93\x14\x80\xe0\xe8\x23\x5a\x46\x94\x26\x4b\xc7\xdb\x5e\x81\xfd\xcd\x96\xeb\x9f\xd7\xcb\x15\x41\x5e\x6f\xde\xc3\x76\x45\x27\x33\xd5\xda\x20\xfb\xa4\x64\x15\xa9\xed\x69\x10\xa9\xf2\x62\x4a\x15\x45\x2b\x73\x35\x5d\x3f\xdc\xa4\x27\x78\xc4\xfe\x4f\x10\xd5\x84\xa5\xf7\x58\x55\x32\xaf\xe0\x34\x3e\xc8\xcc\x72\x5d\xbd\xb2\x75\x80\x96\x2f\xea\xfa\x67\x10\x06\xaa\x77\x20\xc4\xca\x66\x57\x1d\x1f\x0f\x33\xe9\xdf\xcf\x87\x49\xee\x49\x8f\x66\x3f\x6c\xf2\x5b\xfb\xf1\x8f\xb8\xdf\x55\xb3\x59\x29\xa7\xa9\x72\x1e\x9e\x66\xca\x39\x4b\x99\x26\x59\x00\x67\xc4\x45\xac\x1a\x91\xba\x16\xd1\x06\x78\xf8\x6a\x3e\x7d\x9f\x64\xee\xac\x8b\x13\x3a\xad\x16\x75\x8b\x3c\xca\x27\x31\x4f\xee\xba\x5a\xde\x02\x00\xa5\x3b\x2d\x60\x8a\x13\xb7\xa4\x75\x5e\xe3\x57\x04\xa9\x21\xe6\x55\xe3\x9a\x00\x01\xba\x7c\x87\xe2\x01\x5f\x78\x68\x1b\x56\x3e\xe0\xbb\x05\x2a\x35\xa4\x7f\x61\x2d\xd8\xe5\xc1\x52\x16\xa6\xf2\xe2\xea\x13\xd6\x80\x54\xb8\x7d\xc9\x2a\x5a\xa4\x39\x10\x75\xec\xa8\x8d\xfc\x2a\x5c\x13\x7d\x05\x1a\x87\x32\x0a\x9b\xc3\x9b\x10\xc4\x24\xb0\x09\x29\xb5\xdd\xa6\xcd\xaa\x45\x02\x2d\x93\x26\xa4\x99\xd7\xed\x06\x73\x96\x7d\x22\xc4\x06\xa6\xee\xa5\x2c\x84\xc7\x9c\x96\xda\x43\x24\xb1\x67\xf8\x5f\x1e\x01\x9d\x32\xea\x8b\xb0\x67\x93\x5a\x90\x9a\xa3\xea\x70\x95\xcc\xb2\x82\x7d\xf0\x70\xb9\x91\x7c\xe7\xfb\x09\x2c\x53\x85\x22\xcf\xfa\xef\x9d\x5e\x9c\x15\x0f\xe0\x36\x04\x27\xed\x19\xed\x53\x45\xb3\x1c\x4f\x72\x92\x73\x07\x59\x70\x52\x42\x98\x82\x33\xc7\xcf\xcb\xba\xda\x54\x6f\xcb\x1f\xe9\x76\x8c\x07\x4c\x4d\x99\xfa\xe6\x7c\xb3\x9e\x5e\x6c\xbc\x13\x11\xd0\xfc\x7a\x71\x73\x4d\x87\x5a\x3e\x0b\xf1\x36\x1e\x9c\x83\xca\x7a\xf4\xc3\x74\x73\xd5\xbf\x9e\xbe\x4b\x4f\x8c\x24\xc1\x61\xe8\x38\xd5\x16\xb7\xdb\x93\x2c\xeb\xa5\x9a\xf5\x00\xc1\x5d\xbd\x4b\x64\xce\x1d\x78\xd3\x1b\x3a\x10\xff\xad\x9a\x6d\xae\x7e\x5a\x7f\x57\x56\xaf\xae\x36\x9e\x8e\x1d\x38\xf6\x1a\x15\xa9\xdf\x2c\xd7\xb3\x72\xfd\xcd\xf2\x9d\x21\xb4\x78\x47\xa5\xaa\xa0\x32\x40\xc4\xa9\x57\x6c\x94\x9c\xd3\x4f\xc0\x10\x31\xfe\xc5\x26\xc9\x46\x5f\xe6\xc9\x2d\xf6\x43\x62\x02\x0a\x21\x83\xfc\x04\x11\x01\xb3\xfc\x25\x09\xf9\xc5\x83\x2c\xb9\x9e\xae\x5f\x55\x24\xab\x50\xbb\x4c\xf7\xbd\x06\x47\xa2\xcc\x1e\x7c\xb2\x2e\x16\x0f\x52\x9d\x13\x05\x2e\xf3\x21\x1e\xa5\x16\x84\xb0\xd1\xe3\x26\x9b\x5b\xc1\xb9\x02\xf5\x37\xfb\x1b\x56\xf8\x3a\x1f\x6a\x4a\xc6\x1f\xb4\xd4\x4b\x08\xcf\x89\xdf\x62\x96\xb7\x0f\xf0\x83\xb0\xb8\x12\x9d\x0f\xe0\xea\xe3\x61\x51\x22\x83\x66\x02\xb9\xec\x20\x79\x30\xf1\x12\x5d\x3d\x75\x68\x2f\x3a\x3c\xb7\xd1\x94\xf3\x66\x71\x79\x59\x4b\xa3\xb9\x97\xc0\xcd\x0b\x34\x05\x74\xfb\x8c\x7e\x09\x1b\xf2\xa6\xb4\x88\xc4\xfa\xf3\xe5\xbb\x67\xd5\x6f\x80\x0a\x58\xd4\x3c\xd6\x63\x48\x6a\xdd\xc8\x6c\x51\x18\xf8\x40\x07\x8e\xd2\xd8\xc9\x69\x01\xe0\x6e\xb7\x7c\x85\x05\x3f\x49\x0e\xc0\x11\xc0\xf9\xf0\xd1\xb3\x67\xde\xa8\x75\x02\xa0\x4e\x54\x85\x67\xc0\x63\x2a\xa2\xff\x35\xb8\xa4\x81\x55\xac\xde\xb1\xd0\x82\x65\x33\x87\xeb\x61\x03\x79\xde\x60\xa1\xcd\x7d\xc3\xfd\xa5\x9c\x57\x28\x0d\x6d\xb7\x6f\x49\xd5\xd9\xec\x17\xd3\x49\x85\xfe\xed\x7c\x39\xe5\x7e\x81\x33\xec\x5c\xd7\xbd\x8f\x61\x00\x74\x5f\x7b\x70\x6d\x9b\x78\x04\x8a\xa4\x1e\x72\x1d\x4f\xa9\xd1\xdc\x96\xec\x8e\x44\x64\x34\x5b\x5e\x58\xe5\x5d\xb0\xeb\xca\xef\xb1\x16\x9f\x28\xad\x3a\xb6\xad\xc5\x81\xfd\xdd\x4c\xe7\x71\xfb\x06\x9a\xcd\x4c\xbc\x85\xc1\xa1\xd5\xd6\xaf\x2e\xd7\x28\x76\xc9\x5f\x15\xa5\xd3\xe4\x6b\x4e\x38\xa2\x7f\x79\xec\xc5\xe7\x27\x9f\x1f\x11\x65\xd3\xaf\x2b\x42\x1b\xfe\xbc\x7f\x9a\x64\x44\x6e\x09\xfc\xf3\x1c\xb0\x97\xd8\xbd\x2d\x3f\x9f\x2f\x2f\xde\x1c\x75\xaa\x6b\x9c\xc5\x29\xa2\x2d\x13\xc5\xde\xf3\x25\xaa\x38\xfb\x3a\x70\xd5\x44\x22\xc8\x0a\xce\xf8\x64\xd2\x17\x74\xb3\x05\xc7\x76\xdb\xc8\x50\xf9\x2b\xeb\x3b\x14\x42\xb3\xb7\xeb\x0a\x55\x62\x5f\x77\xe0\x83\x54\x2f\x78\x0c\x3e\xfd\x9a\xff\x3d\x5f\xce\xde\x9f\x26\xd4\x55\xff\x62\xbe\x24\x4d\xc4\xc7\x20\x92\x3b\x07\x61\x7b\x83\xe6\x16\x19\xab\xda\x1a\x93\xa4\x68\xb6\x6d\x7a\x3b\x4f\xd8\xb8\x36\x6c\x8f\x7c\xb2\x76\x09\x31\xa1\x7e\x93\xaf\xc7\x43\xd4\xe1\x30\x1c\xe0\xd1\xb2\x07\x14\x79\x12\x46\xa0\xd5\x64\x3d\x47\xda\x02\xdf\xf9\x0d\x1c\xbe\x7e\x9e\xc2\x28\xeb\x74\x45\x62\x38\xda\x83\x18\x58\x0e\xb3\x0a\x0b\x4c\xe7\x06\xf8\xaf\x77\x4f\xe4\xae\xa2\xf5\xf2\x16\x6f\x8b\xb2\xc0\x2a\x05\x1a\xb0\x8a\xf2\xca\xbc\xc5\xdb\x34\xdb\xdc\x76\x4b\xb7\x60\x6f\xca\x0d\x33\x09\xee\x34\x1b\xa1\xae\x4b\x00\x78\x9b\xe5\x4d\xb0\x7a\xc9\x38\xe9\x35\xad\x17\xde\x8e\xaa\x1c\x04\xcc\x5e\x32\x49\xcc\xdb\x06\xd8\xbb\x6c\x48\x17\x64\x00\x74\x00\x82\x36\xd3\x29\x1a\x96\x44\x99\x07\x08\x59\x16\x51\x03\xf1\xed\xd8\x1e\xf8\xb0\x08\x81\x62\x2f\xc5\x9a\x20\x39\xd2\x98\xc1\xb4\xd2\xd1\x07\x6f\x1e\xea\x9f\xd6\xcf\xd7\xd3\x45\x8d\xab\xa6\x4e\xf1\xc8\x0d\x94\xb3\x2e\xed\x19\xc6\x22\x14\x45\x4f\x3c\x18\xc2\x1e\xb7\x2e\xeb\x1a\x92\x08\xd9\xd9\x9d\x5e\xcb\x74\xfc\x9b\xcc\xb0\x24\x70\x59\x2c\x5b\xb4\xb4\xd1\x4c\x2a\xf0\x30\x3f\x24\x76\x25\x79\xa4\x99\xd0\x8f\xba\xa5\x95\x50\xd7\xb7\xdf\x2c\x29\xa0\x22\x7b\x69\x46\xc3\xe0\x8b\x09\x6d\xda\xf6\x01\xf2\x68\xaf\x37\x19\x66\x49\x4f\xef\x54\x31\x15\x48\x7e\x94\x36\xca\xda\xb3\x28\xe9\x23\x8c\x43\xe6\x58\x4b\x4c\x8a\x96\x34\x04\x2c\xeb\xdf\x2c\xea\xab\xea\x72\x23\xc0\xe4\x9f\x56\x99\xd4\x0e\x54\x33\x30\xef\xc0\x3b\x84\xcd\x87\x66\xda\x88\x31\x17\x1d\xa0\x2b\x20\x17\x31\x12\x33\xaf\x7f\xfd\xfb\x77\xbf\xf8\x76\x72\xdc\x9e\x1d\x37\x2f\x4e\x55\x68\xe9\xba\x97\x52\xe5\xcc\x83\xbb\x63\xaf\x2a\x69\x9d\xee\x19\x86\x6f\x33\xb6\x52\xa0\x7f\x5a\x7f\x3b\x45\x4d\xcf\x7b\xd9\xc0\xa4\x82\x37\x94\xa2\x59\x36\x3d\x3c\x24\x01\x75\x2f\xdd\x7a\x8d\x6f\xb7\x75\x59\xbe\x81\x52\x5e\x4a\x73\x84\x5e\x85\xc9\x28\xae\x30\xea\xa4\x56\xe9\xd7\x52\x01\xce\x0c\xcb\x6a\x76\x74\x92\x5b\xa3\x3a\x4b\x7a\x96\x26\xda\xaa\x99\x78\x3a\x82\xcc\xce\x00\x78\x90\xd1\x7e\x77\x74\x94\x50\xb0\xd1\x6c\x2b\x06\xd2\x91\x16\x9a\x23\x5a\x2a\x89\x26\xb5\x09\x22\x2c\x04\x54\x5a\x38\x94\x00\xe1\x93\x75\x1b\x97\xc7\x95\xec\x31\x9e\xd7\xd3\x77\x4f\xf8\x62\x60\x03\x72\x7e\x09\xc2\xe8\x5a\xb6\xa4\x59\x59\xae\xc8\xba\xe0\x72\x3e\xdd\xc8\x6c\x29\x8b\xc4\x6a\xcf\xca\xcd\x06\xe0\xad\xfb\x5e\xfe\x76\x2b\x17\x1e\x50\x8f\x34\x36\xd0\x18\xfc\x37\x86\xcf\x49\x64\x80\x95\x7a\xd5\x28\x7f\xc4\x00\xe4\xd8\x2f\x59\xb8\x94\x2b\x34\xeb\xca\x28\xb3\xd0\x56\xec\x66\x86\xf9\xf6\x10\x2e\x77\x1b\x40\xd4\x32\x0a\xcc\xcd\xe4\x23\x1c\xed\x77\xa4\xf4\xff\xa5\x04\x6c\x2e\x6a\xbc\xe0\x60\x0a\x44\x95\x3c\xa7\xe8\xc5\xf5\xba\x26\xdb\x2a\xb6\x98\xda\x98\xcb\x4a\xec\x20\x29\x81\x6e\x61\x44\x08\xa9\x8b\x5a\xe5\x91\xda\x63\x89\xde\x94\xd8\xc6\xbf\xad\xca\xf9\x0c\xb3\xc2\x84\xe0\x8e\x28\xcc\xca\xe2\xe4\x1a\xc5\x63\x84\x78\x1c\x16\xe4\xcb\xa0\x49\x61\xcb\x71\x02\xeb\x16\x86\x30\xeb\x1e\xaf\x44\x1a\x19\x66\x8e\xa8\x99\xa4\x33\x18\x66\x68\x80\x93\x42\x42\xdd\xbf\xae\xae\x49\xfd\x08\xf2\x21\xf6\x8b\x9a\x64\x45\xdf\x77\xe5\x14\x64\xc3\x34\x79\xc4\xc3\x3f\x7e\x4e\x97\x90\x74\xbc\xb8\xe0\x7b\x65\x05\x5f\x11\x84\xba\x3e\xfd\xcd\x10\x76\xbb\xe1\x37\xcb\x02\x50\x9d\x2f\x7b\xc2\x65\xc7\x37\xa4\xe7\x20\x1d\xbd\xd9\x89\x39\x8f\x0e\xc8\xc7\x51\x16\xcc\x56\x38\x6e\xdc\xc4\xef\xda\x10\x5e\xd3\x01\xa8\xe3\x17\x06\x5e\x83\x53\xfb\xb6\x5c\x23\xb3\x26\xf8\x7a\xc9\x51\xd2\xf3\xcb\x4c\x80\x1f\x07\xbd\x61\x29\x01\x31\xa0\x22\x20\xea\xe0\x9b\x0a\xc2\x5a\x0c\x2a\x07\x5f\xdb\x6d\x50\x41\x4f\x31\x41\x99\x51\x1a\x7c\x76\xc2\x59\x06\x69\xbf\x81\xc3\xa0\x7c\xe6\x08\x7f\x1c\x64\x4c\x94\x0b\x86\xcb\xe7\x11\x63\x23\x75\x34\xcd\xeb\x05\xb1\xf4\x00\x6f\x16\xf1\xa6\x0a\x57\x06\x48\x60\xd7\x2b\xe3\x90\x47\x96\xa9\x81\xcc\xe0\xb3\x55\xde\xa6\x33\xdc\x64\xde\x86\xd3\x85\x66\x0b\x54\xf0\x5b\xda\x50\xc8\xce\x92\x3b\x2e\xfc\x74\x9b\x6a\x5c\xb3\x59\xe6\x7a\x1b\x0f\x26\x44\x8f\x08\x10\x71\x26\x6f\x5e\x33\x6f\x8a\xf1\x67\x28\xba\x4c\x8a\x80\x06\xf0\xe7\x84\x17\x95\x8c\xd6\x83\xb7\xd7\xab\x26\xa4\xce\x86\xf5\xc6\x86\x46\x6b\x32\xe2\xb0\x29\x38\xbe\x6e\x17\xff\x8d\xb2\xb1\xd9\xc2\xeb\x06\x8b\x10\xa9\x49\xa1\x09\x5d\x88\x68\x6e\x72\xcf\xcb\x31\x1d\xcc\xb1\xa3\x7b\x20\xcb\x4d\x07\x87\x02\xef\xf5\x8a\xda\x7e\xe0\x59\x31\xe2\xf4\x00\x52\x0a\x0b\x06\x2e\xf7\x7d\x40\x60\x59\xa0\xf4\x26\x0c\x92\x01\x6b\x95\xea\x16\x20\x57\x8c\xe2\x56\xa8\xe7\x49\xde\x48\xe9\x60\x69\xbe\xac\x26\x34\x72\x5b\xc6\x23\x8a\x15\x51\x45\x75\x7c\x6c\x4e\x94\xb4\x32\x6f\xf5\x63\x33\xd4\x88\x30\x15\xc0\x2c\x80\xb5\xb9\x5a\x2f\x6f\xeb\x64\x92\x59\x3a\xc1\x2c\x4b\x1f\x22\xc5\xa3\x51\xdf\x9e\x02\xce\xda\x8e\xd7\xdb\x5d\xbd\x81\xb3\x58\x9e\x90\x5a\x61\x5d\xae\xd7\xcb\x75\x62\xe8\x0f\x8d\x69\x04\x59\x3f\x2e\x05\xe3\x35\x5d\xed\xac\x97\xd7\x80\x1b\xc1\xde\x66\xe9\xe6\x6a\xb7\xdb\x11\x85\xeb\x67\xd8\x41\x7d\x73\x71\x01\xc2\x73\x62\xd8\x42\x50\xe0\xd9\x35\x0c\x85\xe0\x28\x3d\x9b\xae\x67\xc0\x8b\x53\xb6\x4e\xd4\x2b\x91\xf2\xf6\x88\x6d\x32\xfb\x7f\xff\xe1\xfb\xef\x36\x9b\xd5\x2f\xe5\xaf\x37\xc0\x4d\x3d\xfb\xc1\xb8\xb1\x87\x17\xa8\xa8\x3d\xd4\x94\x94\x10\x13\xbf\xe4\x87\xea\x62\xbd\xac\x97\x97\x1b\xea\xe3\xf9\xf3\x9f\x93\xec\x40\xf3\xdf\xbe\xfb\x71\x79\x6b\x8d\xf6\x50\x37\xfc\x1c\x76\x93\xe5\x0d\x0b\xd5\x24\x59\x02\xfb\xc4\x42\x9e\xd1\x62\x66\x38\xc5\x5e\xb6\x41\x0b\x71\xc3\xcf\x6f\xcb\x72\x51\xa7\xd3\x45\x75\x4d\x57\x65\xc0\x39\x96\xab\xda\x5a\x13\x93\x50\x4b\x49\x4e\x84\xc5\x4f\x56\xdb\x38\x65\xee\xc5\x72\x8e\xf2\x18\x1e\x71\xd2\x0d\x36\xc9\x64\xbf\x5c\x89\x14\x2f\xd6\xc7\x36\x0b\x65\xa9\x2c\xbe\xa0\x71\x8d\x1c\xbc\xa2\x21\x22\xd5\x92\xa2\x53\x67\x73\xbe\x70\x14\x02\xa3\x5a\xfc\x7a\x43\x7f\xa8\xe5\x58\x5f\x85\x85\x61\x45\x55\x20\x60\x04\xde\x23\x40\x39\x37\x73\x54\x2b\x2e\x57\xab\x72\x16\x43\x6b\x3b\x73\xa7\x10\xf5\xa7\x81\x09\x00\xba\x76\xd6\x7a\x8f\xe5\x1b\xce\x6f\xd3\xf9\xed\xf4\x7d\xed\xcf\x9a\x9a\x39\x57\x17\x6f\xfa\x64\x75\x04\x2c\x05\x7e\x17\x5e\x11\xe4\xdd\x0c\x84\xb3\xfe\xb4\x98\xe7\x25\x80\xe4\x50\xd0\x74\x03\x73\xf1\x49\x06\xf6\x97\xeb\x69\xb5\x00\x09\xb3\xf0\x34\xff\x16\xf8\x3e\xac\x99\x35\xd5\xee\xb9\xb4\xd9\xcd\x9a\x7e\x1c\x7b\x8d\x03\x54\x25\xb0\x3f\xdb\xda\xfd\x66\x71\xb4\xdb\x04\x44\x5e\x20\x13\x1a\x1c\x63\xf1\xbd\x38\xeb\x13\x25\x1c\xbe\x8a\x8b\x0b\xeb\x54\xaf\x6f\x80\x06\xb9\x1b\x4f\x8a\x65\x0c\x03\x91\x6f\xaa\xcb\xf7\x7f\xab\x36\x7c\x57\x6d\xc6\x1e\x4d\x70\x1d\x87\x10\xa0\xc0\xc1\xa9\xa4\x5a\xef\x25\x9b\x9b\xa7\xb6\x51\x20\x84\xe5\xfc\x6d\xd9\xd6\xea\x84\x0f\x24\x0e\xa1\x85\xad\x05\x64\x75\x5d\xc1\xce\x77\x87\x55\x72\x4b\x69\x75\xde\x30\x24\x72\x04\x98\x21\x05\xc6\x25\x40\x1c\xbf\x23\x9b\xb9\xe9\xfc\xc9\xb4\x46\xd0\x80\x41\x58\x52\xb5\x27\xc1\x9f\x6d\x2b\xb9\x4f\xd1\xe1\x39\x31\xd7\xf3\xa3\x9d\xf7\xbc\x95\x6a\x74\x4e\x73\x7b\x2e\x92\x04\xc3\x93\x91\xc3\xa1\xd6\xe3\x20\x79\xc8\x1d\x00\x6a\xb1\x21\xc3\x4c\x5d\x06\x54\x92\x11\xe8\xe6\x16\xc7\x6b\xb4\x52\x94\xde\x0f\x46\xad\xfc\x24\x2a\x53\x52\xa6\xa5\x84\x06\x89\x91\x02\x81\x7e\x67\x0c\xfb\x8e\x56\xb4\x83\xf8\xd5\x72\xb3\x7c\xa2\x00\x47\x04\x2b\x79\xa3\x3d\x84\x9b\x9f\x0c\x1b\x6b\x93\x1c\x16\x68\x79\x4a\x3a\x6c\xac\x9f\x4e\xe0\x03\x3b\x20\x05\xe1\x63\xa8\xd1\x48\x61\x10\x3a\xbd\xe2\xb8\xe1\x1c\x2e\x4d\x9e\x2e\xbb\x5d\xc6\x04\xea\x2d\x53\xfa\xa6\xc1\xe0\x2f\x11\x11\x79\x3f\x38\x34\x55\xd9\x5e\xce\xcd\x3c\xb5\x8d\x79\xee\xe1\xe4\x6e\xdd\x44\x3d\xda\x7b\x10\x6e\xd2\xda\x9f\x1f\xda\xd6\x5a\xcc\xca\xe3\x61\xe0\xb2\xc8\xba\xdd\xb6\x64\xcf\x66\xdc\x66\xdb\x26\x2f\xdf\xf5\x37\xb0\x9a\xd6\x69\xb8\x76\x91\x95\x1b\x8f\x03\x60\xc5\xdc\x81\x05\x62\xc5\x4d\x99\x47\x9d\x51\x22\x9c\xd8\x4d\x30\x0f\xaf\x50\x21\x18\x83\xab\xe9\xa8\xb8\x5f\x94\x71\x2e\xa6\xc5\xf3\x74\xb1\x44\xe3\xef\x4d\x99\xf5\x2f\xa7\xd5\x3c\xae\x81\x69\x76\x93\x8a\xf2\x38\xd5\xdb\x46\x1b\x24\x11\xd2\x80\xbb\xdd\xe3\x1b\x21\x9a\x5f\xc3\xcb\xd5\x5c\x2d\x97\x6f\x6a\x77\x5f\x8f\xb2\x36\x4f\x91\xda\xb0\x34\x3c\x8c\xa8\x5c\x26\xf5\x8b\x90\x33\xb0\x5e\x98\x0d\xd8\xa8\x19\xbd\xda\x8f\xf4\xea\x2c\x11\xa0\xbd\x0f\xb7\x42\xdf\x20\xbd\xb7\x54\x95\xbc\x13\x15\x53\xc4\xfd\x02\x9d\xf7\xa8\x14\x75\xc9\x85\x8c\x6c\xe1\x7e\xf5\x8c\x47\xe8\x5d\x28\x7c\x87\xdf\x02\x29\xe5\x75\xbb\x09\x5b\x45\x26\x30\x78\x4a\x41\x94\x21\x1c\xf4\x21\x26\x93\x02\x73\xd8\x07\xb5\x12\x62\x8f\x8b\x85\xc8\x84\x93\x72\xcb\x90\x04\x35\x21\x06\xa5\x84\x30\x53\xf6\x81\x69\xc1\xb1\x14\xf0\x6c\xc4\xf9\x4e\xce\x2e\xe3\xd4\x5b\xaf\xb4\x4a\x89\x0a\x88\xbf\xf3\xfc\x8b\x88\x24\x88\x03\x01\xfd\xd9\xd5\xf2\xd6\x6c\x96\xaf\x5e\xcd\x4b\xe6\xcf\x8c\x1d\xb4\x20\x86\x53\x3b\xd3\x2f\xf9\xf9\xf1\xcd\xa0\x77\x57\x49\x7b\x1b\x9e\x87\xd9\x00\x95\xec\xe0\xc4\x12\x25\xf0\x7b\x88\x4d\x62\x86\x6e\x95\xa1\xed\x8a\x3f\x4f\x2f\x29\x95\xa6\x4a\xee\x7d\x2f\xdf\xc1\x19\x8f\x2f\x6a\x79\x5a\x6e\x16\x54\x06\xd5\x3a\x61\x02\xec\x1a\x02\xb3\x4e\x20\xba\xf4\xf4\x69\x10\x71\x82\x2f\xdd\x85\xad\x6c\xb7\xd2\x48\x8a\x6a\xce\x30\xaf\xd7\x23\x6c\xb4\x08\x91\x7b\x92\xc3\xea\x70\xfe\x93\x61\xd2\xb7\x37\xc0\xbe\x3a\x62\xc7\x70\x22\x10\xc8\x89\x1a\xbe\x24\x30\xfa\x84\x2f\x2f\x13\x47\x6d\x72\x65\x6f\xd7\x32\x14\x22\x54\x2f\xe1\x44\x77\x39\x87\x93\xc8\x98\x6d\x63\xf4\xdb\x84\x9f\x7f\x8f\xbe\xff\x31\x31\xd0\xd6\xbc\xfa\x90\x69\x99\x9a\xeb\xb4\x15\xba\xc4\xeb\x6b\x2c\x12\xdf\x85\x73\xc3\xdf\xe0\xcd\xea\x8f\x65\x39\xab\xbf\x9f\xbe\x87\xe3\x14\x34\x25\x3d\xa2\x5e\xe1\x43\xa6\x50\x23\x86\xf7\xb7\xe5\xf2\xba\x18\xe4\x91\x89\x16\xb7\x73\x4c\x77\xb7\x49\xc6\x82\x9d\x1d\x1a\x1a\x1a\x05\x63\x2d\x12\xa6\xdc\x24\xb6\x3c\xae\xaf\xd6\xd5\xe2\xcd\xdf\xd6\xd3\x15\xc1\x5a\xb3\xfc\xd3\x32\xd7\x51\x7b\x41\x77\xa8\x0f\x88\x70\x1d\x15\x18\xc4\x05\xfe\x11\x15\x78\x30\xd9\xf9\x06\x56\x01\xc3\x6e\x61\xbc\xeb\xcb\x77\x1b\x52\x3e\x78\x96\x48\xec\x9e\xd8\xe4\x95\xb2\xfa\x0b\xfe\x83\x66\xc1\xf4\x03\xe7\x93\xd9\x04\xb7\x5f\x14\x62\x67\x36\x42\x5c\x95\x09\x1c\xf4\x81\x75\x00\x6a\x51\xef\x59\x2d\x6e\xca\xa1\xf0\x01\x16\xf9\x78\xbf\x40\xfd\x86\xc8\x72\x9a\xcb\x9f\xac\x15\x45\xe6\xd3\x66\x11\x77\xf9\x8e\xdb\x8e\x3c\x30\x82\x4c\x83\xce\xd2\x3a\x6d\x62\x31\xff\x8c\xac\xc7\xd4\x1e\x4e\x53\xfa\x62\xec\x26\x03\x45\x03\xe7\x30\xa7\x50\x73\x38\x35\x77\x93\x2b\x6c\x62\x57\x7d\xec\x2d\xcd\x48\x4a\xe0\x6d\xde\x9b\xf5\xa0\x20\xa2\x85\xd8\x46\x6b\x51\x65\xc4\x43\x1d\xd2\x3a\x72\x8e\xb6\xa3\xb6\x02\x1f\xdd\xcd\x02\x93\xd5\xbb\x68\x22\x11\xc7\xe2\x89\x01\xb3\x58\x8e\xf7\xc2\x1f\xb4\xbe\xc3\x92\x3a\x0d\x76\xee\xe9\x84\x40\x00\x7b\xa2\x1b\x1f\x23\xd4\xf6\x4f\x90\xc5\x3d\xe5\x27\x99\xd7\x6f\x11\x66\xda\xf9\x8a\x20\x65\xc9\xf6\xc8\x9b\x25\xbc\x1a\x09\xaa\x16\x04\x09\x4b\x7a\xd2\x33\xba\x5f\x53\x62\x89\x86\xe3\x5e\xb6\xf7\xdb\xb3\x53\xc2\x66\x80\x7a\x85\x2d\x4a\x02\xda\xaa\x05\xce\x21\xde\x29\x48\xcf\x62\xf6\xf8\x23\xdb\xaf\xaf\x3d\xa2\xe2\x28\xe7\x6d\x96\xb8\xa0\x80\x6f\x55\x9b\xc3\xb5\x7d\x03\xac\xc5\xb7\xef\x52\xf6\xe3\x59\x5c\xcc\x6f\x66\x25\x59\xeb\x30\x29\xdc\x5e\x55\x17\x57\x06\x1d\xb2\xea\xe2\x8e\x61\x26\xeb\x75\x52\x2f\xcb\x5c\xba\x3a\x85\xff\x81\x63\xb2\x96\x77\xc7\x41\xd3\xd4\x6a\x11\x58\x9e\x51\x17\x63\xb5\x7f\xeb\x51\x89\x49\x21\xa9\xd6\x58\x4d\x92\x49\xe9\x6f\xaf\xe6\x5c\xc3\x30\x15\x54\x03\x44\xd1\xe9\x45\xb5\x79\xcf\xf5\xfb\x6c\x49\x43\x17\x00\xdc\x4f\x64\x7d\x46\x11\x4b\xca\x36\xc7\xc2\xda\xcf\x24\xeb\xb2\xfc\x2f\xf1\x26\x37\xf2\xcd\xc2\xff\x5a\x95\xb7\xe2\x54\xc7\x7e\x24\xdc\x40\xde\x19\xec\x58\x59\x34\x9d\xbd\xff\xbe\xaa\x37\x66\xbd\x5c\x6e\xb8\x17\xd3\xb0\xf5\xd7\xfb\x58\x9b\x60\xd4\xe0\xa6\x10\x35\xa1\x35\xc0\x01\x6e\xcf\x0a\x05\xc9\xd0\x6f\xf3\x92\x1b\xd7\x74\xe9\xea\xe5\x67\x9a\xf0\x99\xb9\x98\x4f\xeb\xfa\x01\x79\xbb\xa1\x9b\x52\x18\x15\x03\xa5\x24\x4a\x12\x35\x6b\x91\x0c\xfa\x7f\xe9\x0f\x12\x4e\x64\x45\x5d\x23\x92\x06\x27\x73\x11\xe4\xae\xcd\x50\x1b\x90\x28\xc3\x45\x65\x73\x23\x9f\x52\xb9\x00\x2d\xfc\x9f\x2e\x1b\x45\x24\x9d\x0b\x6d\x96\xcf\xe8\x0a\xbb\x70\x63\xe9\x6b\x1a\x97\xb8\x9a\xd6\x3f\xdd\x2e\xfc\x7c\x4e\x11\x35\x88\x20\x1f\xbd\x72\x0a\x7f\xb8\xe4\xa7\x23\x9b\xac\x93\xc2\x6a\xf1\xba\x30\xe2\x6d\x10\x2c\x41\x3d\xe1\x2d\x78\xf1\xc5\x65\xbd\x09\xcf\x04\xdb\xab\xc5\xcd\x75\x71\x7f\xdc\x3b\x9e\x8c\xd2\x51\x7e\x36\xbb\x77\xd6\xdf\x66\x67\xb3\x1e\x7c\x8c\xcb\x27\x13\xca\x80\xcf\x6d\x76\xbf\x5f\x2f\x6f\xd6\x8a\x19\x6b\xd1\x51\xdc\x3f\x7b\xd6\xbb\xff\xca\xac\x09\xfc\xfb\x2f\xc6\x67\xf5\xd9\xcd\xb7\x4f\xbe\xfd\xf6\xec\xdd\xc3\x93\x49\x6f\x1b\x7d\x7f\x86\x45\x7f\xbd\x81\xa3\x26\xda\x8d\x40\x79\xe8\x27\xfd\x7a\x7c\x76\x7b\xf6\xb7\x49\xef\x34\x1b\xbf\x38\x9d\xdc\xdb\xfe\x29\x85\x84\xe3\xc9\xbd\x2c\xfb\xec\xbe\x59\x23\xa7\x98\x97\xcf\xa7\xaf\xa0\xf4\xd7\xe9\xd9\x6d\x2f\x3b\xab\xef\x9d\xdd\x1f\x9d\x42\xd5\xaf\xcf\xee\x9f\x0d\x4e\xb7\x54\x0e\xb6\xde\x6a\x86\x5e\x0a\x35\x81\x31\x31\xf9\xdd\xee\xac\x9e\xdc\xb3\x79\xe4\x7f\x0f\x99\x50\xef\xc5\x36\xdf\x9a\x0c\x07\x0c\x4d\x8d\x33\x1a\x00\x95\x29\xeb\x8b\xe9\x0a\x07\x75\x86\x08\x48\xce\xce\xce\xee\x9f\x5f\x2e\xd6\x9b\xc9\xf6\x66\x7c\x36\x9b\x1e\x5f\x3e\x3c\xfe\x76\x72\xf7\xe5\x2e\xb3\x35\x36\xcb\x37\x70\x7e\x2f\xee\x27\xe3\x17\x58\x7c\x7d\xb6\x98\xdc\x4b\xb6\x18\x37\x60\x4b\xb1\x02\xc8\x74\x72\x7b\xcc\xd8\xed\xed\xc5\x2e\x34\x77\x2d\xae\x10\x00\xfe\xf1\x75\x7d\x0c\x60\xcf\xa6\xf5\xd5\xc3\xf9\xea\x6a\x5a\xdc\x3f\x4e\xa9\xff\xdf\x26\x50\xb4\x32\x97\xf6\xb0\xe9\x08\x03\x4e\xfc\x70\x64\xd9\x6c\xd0\x11\x5b\x48\x82\x3f\x43\x87\x0d\x9c\x77\x3e\x57\xcf\x5c\x5d\xf2\x9e\xcb\xee\xac\x13\x32\x3a\x1f\x92\xf7\x1c\xb2\x09\xd4\x83\xc3\x66\x01\xe2\xe9\x8c\xcc\x8b\xc9\x3b\x51\xa2\x25\x69\x5b\x74\xd7\xac\x95\x89\xc1\x3c\xc3\x1b\x0e\xf6\x90\x62\xf3\x39\x63\x7d\xf3\x20\x37\x85\x8d\xc6\x70\x8e\x7f\xc0\xd8\xdb\xff\x28\xf5\x5a\xb7\xbe\x7d\x9a\x9b\x26\x8f\x7f\xfa\x41\x6e\xa4\xbf\x07\x30\xcb\x59\xe2\x46\x89\x5a\x4f\x23\x0c\xa7\xb5\x2e\x0d\x2c\x2c\x9f\xe5\xae\x3f\x86\x92\x7d\x09\x93\xe5\x82\xc0\xa7\xeb\x1b\xa0\xb5\xc5\xab\xd2\xab\x69\xbb\x89\xaa\x44\x1d\xc0\xd0\x87\x76\xa1\xaa\x48\x67\xb7\xcd\xe2\xee\x35\x79\x57\xe5\x3e\x2b\xc0\x05\xcc\xc6\x21\xcb\xb5\xa8\x5c\x0d\xae\xf0\x7c\x2f\x5b\xf0\x97\xba\xb3\xbc\x27\x1d\x15\x39\x66\x69\xf9\x40\x15\xd8\xe6\xde\x6f\x0b\xa2\x48\xcc\x0e\xcd\xc9\xd7\x38\xdd\x9a\xe1\x3c\x83\xe0\x48\x72\xda\x96\x63\xbf\x35\x32\x16\x94\x8c\xd2\x4e\x8b\x2f\x46\x63\x5c\x29\xc6\x0e\x04\xbf\x26\xb9\x63\x16\x2c\x9f\x5b\x70\x4c\x87\x80\xd9\x6e\x3b\xea\x26\x2d\x66\x05\xc0\x12\xe5\xea\x41\x3e\xad\x63\x59\x9f\x31\x3b\x4a\x6d\x86\x87\xa4\x3e\x7a\xb1\xbb\xe6\x73\x44\x48\xdf\x43\xbb\x56\x8a\x0b\x0e\x15\x2d\x78\xdf\x7c\xa7\xb6\x0e\xef\x36\xea\xe1\x86\x66\x3b\x9b\xe9\xe2\x02\x71\xc9\x5d\xa9\x17\x1d\x1c\x75\x72\x9d\xad\xc0\x91\x8c\x54\x08\x7e\x70\x10\xf4\xc0\xb4\x9d\xe8\x0c\xdb\xe1\x3a\x21\x40\x13\x02\x4f\x25\x3b\xfc\xdc\x6e\xd8\x1d\x90\xf3\x3c\xc6\xca\xf6\x17\x76\x10\xd6\xc6\xa6\xaa\x7f\x9e\x4f\xab\x85\x06\x96\x91\xf1\xd3\xb5\x33\x15\xb6\x56\x1e\x90\xda\x54\x57\xe2\x28\xc6\x54\x6e\x92\x8d\xbc\x0f\x6d\x48\xf3\x18\xd3\x28\x12\x71\xab\x26\xca\x1f\x7a\x14\x8a\x67\x24\x32\x9c\xb5\xab\xd3\x39\x39\x7e\xf3\xfe\xe9\x4c\x06\xf1\x60\xc2\xb6\xba\xdd\xae\x27\x04\xa1\x93\x21\x4d\x10\xa5\x55\x33\x38\x36\xdb\xd2\xaa\x9f\xb5\xe4\x10\x4f\x32\xc1\x28\xa7\xb3\x01\xe9\xa0\x61\xf6\x48\x04\xdb\x79\xe0\xf5\x75\xee\x2d\xa6\x29\x55\x9b\xb1\x0b\x83\x95\xd8\xf6\x86\x54\x56\x82\x9d\xc6\x34\x68\x4b\x3b\x0b\x2a\x87\xb0\x64\x79\x13\xfd\x16\xf6\x91\x37\x2a\xe6\xbd\x8e\xc8\xdd\xd2\xd4\x1f\x2d\x41\xb3\x9a\xf0\xf7\x43\x58\x14\x52\xb7\xee\x85\x28\x2c\x59\x4f\xdf\x94\xac\xd6\x0c\x2a\xe2\x1e\xa0\x09\x79\x92\x18\xbd\xa9\x30\x75\xf5\x5b\x99\x7b\xdb\x82\x8f\x62\x2e\xb4\x83\x03\x2a\xb5\xd8\x52\xcc\xc9\x76\xac\x06\xa7\xae\x76\xe8\xf7\xe9\x0a\x83\xd4\xe3\x04\x27\x52\x99\x41\x0a\x51\x69\x5f\x1a\x86\x63\xec\xc9\xa9\x26\x8e\xbd\xae\x7b\x90\x38\x21\xa2\x1d\xe3\xaf\x9d\x41\x81\x12\x76\xbb\x8b\x37\x79\x10\x20\xc4\x5e\xcd\x6e\x8a\xc6\xea\x0e\x18\x0b\xd3\x6a\xed\x87\x75\xe9\xa3\xcd\x00\xaf\x3b\x56\x28\x62\x9a\x4f\x10\x7d\xcb\xe6\xcb\xcd\xce\xe0\xdd\xb7\xeb\x1c\x07\x7d\x0e\xe0\x18\x38\xc6\xd4\xf1\x69\x82\xae\xc9\xa9\xc9\xb0\xd8\xce\x10\x6d\xb8\x56\x2e\x17\x71\x55\x2a\x60\xef\x0a\x45\x8f\x7f\xb9\x90\x2b\x19\x43\x18\xdf\x37\x6b\x16\x47\xa9\x37\x3b\xd3\xd5\x6a\xfe\x9e\x81\x01\x18\x6e\x38\x40\x05\x40\x42\x26\x50\xfb\x5a\x2a\x7f\x85\x9d\x66\x67\x40\x96\x3e\x54\xe4\x18\x6f\x3a\xcb\x5f\x5d\x89\xca\x46\x94\x2c\xbc\xb9\x34\xaf\x8b\x5e\xd5\x4b\x4f\x4e\xab\x11\x24\xc0\xf1\x77\xd8\x0e\xf4\xeb\xd3\x82\x03\x3b\x9e\xbe\x1e\x11\x2d\x8c\x5f\x4f\x26\xf9\x78\x02\x9d\x5c\x4f\x57\x4d\xd4\xef\x1b\xbc\x17\xd3\x8c\x86\xdd\x08\x01\xa4\x24\x2c\xed\x34\x63\xff\xec\x10\x41\x70\x84\xde\x8b\x68\x4b\x38\xdb\x6d\x83\xd2\x38\x38\x17\x11\x6c\x6e\x0f\x48\xa6\x5e\xae\x37\x30\x96\x3e\xfe\x35\x6c\x81\x43\x9f\xf4\x6b\x67\xc2\x33\x85\x27\xa8\xd8\x0c\x13\x5c\x24\xb9\x74\x4d\x88\x34\x3b\x18\x60\xe2\x62\xb9\x7a\xff\x94\xaf\x3a\xe8\x37\xdf\xbb\xa8\x9e\x80\xe2\x82\x88\x2d\x67\x61\x49\x83\xac\xf3\xc8\xa6\x6c\x60\x6f\xe9\x35\xcf\xd9\x33\x94\xab\x42\x6c\x0e\x92\xf3\xe5\x12\xe3\x9d\x39\x59\x86\x5b\x24\x99\x14\x8a\x89\xe1\x68\xa3\x9b\x81\x76\xf3\x20\x33\x0d\x97\x03\x2e\x6d\x95\x37\xfe\x7e\x47\x39\x19\x46\x2a\xe4\x16\x51\xe9\x26\x70\x16\x45\x85\xcc\x94\xd3\x69\xe6\x8f\x8f\x2b\x77\xf5\x48\xbe\xed\x78\x9d\xc4\x8e\xef\x82\x06\x0f\xa4\x6a\xc2\xdb\xae\xf5\x42\x90\x1b\x75\x40\xa5\x0c\x43\xae\x6b\x10\x97\x1a\x44\x54\x7d\x10\x28\x9f\xdc\xf9\x57\xef\x65\xec\x28\x3b\xd0\xc7\x9e\x9d\x7e\xf5\x1e\xc7\xe1\x4d\x52\x11\x5d\x4f\x51\x91\x2c\x1b\xf9\x65\x82\x0f\x98\x04\x9e\x46\xb4\xe3\xf5\x24\x0a\xd9\x06\xd6\x17\xd9\x08\xfe\xc1\x25\x94\xb7\x15\xf3\xc1\xb1\x85\x61\x52\xfc\xc1\x46\x61\x4e\xc8\x7a\x99\x29\x87\xa1\xcb\xf1\x4f\xbc\x9f\xf9\xf5\xb9\x98\x73\xff\x27\xd3\x61\x13\xd9\x3c\x2c\x96\x70\xb0\xb8\x84\xa5\xe0\xb1\x1b\xb2\x35\xd6\x65\xa7\xba\x0c\xab\xd3\x87\x6e\x6c\xda\xcb\xcf\x32\xc3\x08\x0f\x74\x20\x2d\x65\x25\x43\x54\x26\xba\x75\x62\x6c\xcf\x5f\x88\x31\x03\x3e\x89\x01\xff\x6d\x0a\x62\xff\xc0\x5c\x2d\xe7\xb3\x5f\x42\x8e\x8d\x49\x78\x91\x32\x9f\x8d\x7c\x8e\x8d\x15\x7a\xbd\x3c\x90\x00\x3a\x27\x4d\x86\x7f\x0b\xe5\x48\x3e\xc2\x1f\x6c\xd7\xd7\x39\x3e\x8e\x5b\xca\x5d\x1c\x38\xea\x5e\x2c\x68\x55\x1c\x23\x1f\xa5\xa6\xe9\x97\xdf\x4a\x36\x0c\x5b\x40\xc7\x04\xec\x52\x8c\x03\x9b\x5d\x9e\x9e\x00\x2d\x5a\xa5\x55\x60\x54\x60\xe5\x2c\x09\x1b\x33\xc9\x3c\x76\x05\x07\x99\x57\xaf\xd0\x7e\xd4\x79\x59\x89\xeb\x98\xe4\xa4\x09\xb5\x9a\x64\xe8\x9c\x69\x3f\xb2\x1d\x22\x5d\x97\xb5\xc3\x0f\x45\xcc\x13\xa7\x05\x2f\x5e\x6a\xc3\x87\x08\x6b\xb3\x68\x42\xff\x2a\xcd\x6f\xb7\xad\x2d\xd9\xf0\xc3\x6d\xcd\x88\xde\xae\xad\xe2\x11\xf3\x0a\x48\xe9\x76\xe1\x1f\x0e\x70\xcc\xa4\x44\xf1\x60\x61\xa4\xeb\xea\xa2\xb5\x6a\x07\xb2\xa7\x3f\xa6\x9e\xcf\x24\xb9\x3e\xe1\xdd\xe3\xb7\xc8\xe2\xb5\x7f\x04\x65\x7f\xdf\xd4\xe3\x88\xd5\x5c\x94\x99\x37\x78\x25\x7c\xb7\xc6\x96\xc5\x8a\x4e\x19\x36\x0e\xb4\x68\xbc\xe3\x61\x7b\x13\xe7\xa9\x95\xbb\x8a\x38\x38\x8f\x39\x44\xf0\x21\x2d\x72\xaf\xfb\x7d\xbc\xb6\x5b\x3f\x16\xb4\xc7\xc9\xbd\xa8\xd1\xce\x78\x0d\x2d\x24\xa1\x55\xac\xe2\xed\xa4\xdd\x6e\xc7\xd3\xec\x59\x98\x4d\xe2\x95\xc1\xab\xbc\xd6\x52\x7e\x43\x6e\x3b\x35\x09\x0c\x4c\x3f\x7e\xba\x4c\x1c\x0c\xb1\x75\xaa\xa8\x71\xdf\x94\xef\x7d\xc7\x0b\x3f\x10\x36\x24\x85\x91\x4c\x5a\x81\x85\x52\x44\x65\x5e\x3c\xc7\x08\x9d\x7b\xe3\x43\x36\x23\x3b\x8a\x79\xac\xad\x7f\x5d\xbf\xca\xee\xc8\x30\x97\x14\x93\x4f\x30\x97\x12\x41\xfc\xd0\x23\x6d\x1e\xb8\xb5\x59\xcd\x05\x06\xcb\xe3\x38\x49\x9e\x7d\x3e\x4c\x69\x9b\xa7\x50\xe6\x51\xe4\xb0\xb9\xe1\xdb\xa3\x72\xea\x35\xaa\x27\x73\xed\xb0\x40\x05\x51\x74\x6a\xdf\x6e\x95\x5b\x90\x0f\x1c\x81\x3c\x2b\xbc\x23\x33\x29\x23\x08\x02\x53\x4b\xb3\x1d\xaf\x8f\x6e\x77\x6c\xbd\x77\xb9\xf2\x68\x6c\x43\xde\x04\x9e\x9e\x9c\x8b\xe7\xee\x49\x2e\x1f\x4a\xb2\xe4\x67\x68\xe3\xf9\x91\xa3\x93\x3d\xfd\x6b\xaf\xb6\x7b\xcb\xe6\x34\xc3\xf9\x7d\x86\x51\x6a\x26\x3c\x01\x33\x2f\xf4\x4e\xa6\xb3\x82\xe1\x3b\xc3\x59\x89\xb7\x39\x2c\x61\x37\x33\xfc\x60\x0d\xc5\xa8\x91\x22\xc1\x4c\x99\x55\x70\x1c\x54\x32\x72\x6e\x0d\xb9\x2a\xf7\x8c\x45\x10\xa0\x6a\xaa\x81\x9c\xbb\x5d\x4f\xb9\xeb\x02\xa8\xba\x28\xb0\x9e\xea\xd6\x24\xff\x3d\xc9\xa2\x1c\x56\xd1\x9a\x64\xd2\xc8\x61\x95\xb0\x49\x30\x26\xc7\x08\xe9\xd4\x4a\x73\x89\x0c\x9b\x3d\x3f\x32\x38\x0d\x5a\x73\x29\x22\xe5\xe4\xe9\x82\x5a\x38\x22\x94\x69\x31\xc3\x0e\x14\x8a\xce\xbf\xc7\x34\xce\x4b\xea\xdd\xf5\x1c\x6d\xf1\x87\x9f\x42\xdb\xc8\x87\x04\xc7\x8f\x7f\xfa\xe1\x67\x32\x4c\x1f\x91\x99\x3f\x82\x6d\x93\x0c\xb4\x8d\x66\xf5\x3c\x07\xdf\xc2\x59\x4d\x38\x34\xad\x30\x8a\x62\x75\x1f\x8a\x24\xa8\xe1\xc4\xa2\x58\xf9\x90\xb9\x37\x34\x9c\x64\xd8\x68\x7f\x5a\xbf\x5f\x5c\x14\x12\xff\x96\x52\x50\xad\x09\x45\x52\x09\x2a\x67\xb9\x14\xb6\xdb\x88\x47\x0b\x89\xdd\x2e\xd6\x8a\x1c\xc0\x81\x47\x62\x6a\x6b\xac\xaa\xc0\xfe\xde\x59\x7e\xb4\xcf\x03\xe2\xda\x4e\x03\x34\xb9\x33\x18\xf4\xdc\x3f\x26\xed\x8c\x0b\x43\x18\x4f\x0b\x93\x59\x83\xfc\x9c\x78\x86\xeb\x9d\xd7\xb6\xb7\x91\x73\x5d\x2d\x01\xcd\x32\x73\x95\x67\x0e\x28\x77\x97\x71\xa9\x9d\xb1\x2a\x7b\x4f\x69\x4b\x73\xe3\x6c\xe5\x79\x07\x74\xc1\x8d\xe5\x42\xc0\x24\xd7\xf5\xb1\x4f\xbb\xf6\x5e\xc0\xbb\x08\xc8\x70\xc0\x6c\xed\x11\xea\x21\x24\xe2\x93\xef\x11\xee\x02\xb8\x05\x9f\xa1\x17\x8c\x84\xb3\x08\x13\x63\x5d\x03\xee\x23\x91\xbe\xc1\xd9\xd2\x55\xce\x3a\xd5\x7b\xc2\x41\xe4\xa1\x22\x7e\xf1\x01\x97\x03\xb7\x40\xa1\x5f\x29\x53\x02\x95\xca\x31\x49\x03\x04\x15\xf6\x78\xcc\x1a\x04\xf4\xb7\xc6\xdb\x5b\xac\x6c\x34\x68\x1c\x30\x75\x3a\x56\x39\x17\xee\xca\x8b\x6e\xfc\x91\xed\x14\xd4\x0e\xbb\x9f\xa8\x2f\xf9\x47\x82\xa6\xdb\x2c\xb6\x58\x19\xfe\xf1\x3b\xa1\x3b\xd0\x94\x07\xa0\x0d\x8d\x85\x42\x12\x92\x70\x6e\xaf\x15\x55\x10\xc1\xdf\xdc\x5a\xf2\xff\xfc\x9f\xff\xd7\xff\xfd\x7f\x24\xd9\xc8\xce\x64\x78\x97\x48\x5c\x9b\x62\x1b\x26\x49\x1e\xd5\xa5\x92\xbb\xfc\x63\x6a\x52\x5e\x2f\xf1\x69\x97\x2e\x35\x21\x05\x35\x27\xa2\x15\x74\x4d\x81\x34\x6c\xd8\x48\xd6\xd3\x9e\x49\x02\xb9\x8e\x07\xc2\x2f\x94\xa6\x08\xe7\x8e\x8e\x84\x87\x41\x46\x16\x05\x67\xc3\xc7\x28\x1a\x5b\x0f\x94\x1b\x8d\xe1\x9f\x49\x8e\x35\x9c\x4a\x84\xc7\x89\x55\xa8\x25\x56\xb2\x49\x80\xdf\x68\x71\x21\xc4\x4e\xb7\xc4\x34\xbc\x96\x7b\x00\x77\x83\x9c\xf9\xaa\x49\x49\x13\xf3\x61\xa8\x2f\xca\x1f\x92\xb2\x50\x41\x05\x69\x76\xb1\x14\xd5\x08\xb5\x53\x9e\x43\x04\xe4\xf4\xaa\x2c\xaf\x72\x32\x5d\x71\xca\x03\x22\x20\x42\x09\xfc\x03\x14\x22\x26\x02\xf6\x49\x0a\xe1\xc4\xc7\x03\xc0\x3c\xa2\xc4\x53\xf5\xa1\xbe\xcd\xd4\x25\x88\x16\x62\x5b\x3e\x2f\xf8\xcb\x81\x41\x65\x9c\xd6\x8c\xac\xc9\x9b\x6f\xa1\xc8\xa2\x38\x7d\x3d\x7c\x0d\x40\x51\x1d\x72\xdd\x97\xe6\xc6\xaf\x27\x8e\xe0\x87\x36\xcd\x3f\xa3\x0f\xdb\x6a\xa1\xf3\xbf\xf5\x84\x74\x70\x14\x15\xab\x0a\x81\xb1\x03\x79\x45\xda\x57\xc7\x96\xaa\xc5\x5b\x3f\xb6\x36\xce\x26\xc5\x5c\x0b\xe3\xad\x59\x0f\x0f\xb6\x27\x79\x5b\x74\x3a\xf0\x6f\xa8\xa2\xb1\xc1\xb9\xb5\x6d\xee\x8a\xd6\x25\x1a\xdf\xa2\x6f\x1a\x97\x01\x91\x05\xb5\xb8\x1a\x88\x90\x23\x18\x3a\xfd\x6e\xa4\x36\x8c\x00\x06\x06\xb4\x87\x8d\xfa\x80\xb6\x32\x52\xd6\x24\xcb\x10\x87\xfb\xb8\x15\x0d\x26\x64\x32\xde\x48\xa8\x7f\xc3\x4b\x4c\xc3\x6e\x42\x83\x63\x1c\x11\xb7\x20\xf6\xb9\x31\xf7\xe2\xce\xff\x15\xed\xfa\xcb\x85\x0d\x3a\x84\x45\xc3\xbc\x41\x1e\x6a\xf3\x6f\xaa\x59\x3e\x40\x8b\xa2\x77\x81\xd2\xda\x99\x45\x50\xfc\x67\x60\xe5\x5c\x86\x04\xae\x30\xe4\x40\xcb\x29\x01\xe5\xa9\xcb\x85\x0b\xe4\xa8\xa7\x02\x68\xf7\x72\x81\x72\x55\x9b\x5d\xfe\xe5\x22\x1b\xd1\xce\x55\xc4\x37\x0f\x56\x81\x67\x1e\x64\x0c\x46\xd1\x54\xdb\x5e\x2e\x64\x6c\xf6\xe8\xa1\x1a\x71\x35\x66\x49\xf7\xb6\x9b\x91\xe4\x8e\x0d\xd3\x9b\x31\x00\x67\xf0\x37\x7a\x50\x86\x4b\x7a\x21\x3a\x77\x66\x4a\xfe\x87\x31\x25\xc2\x70\x31\x0c\x00\x93\x1f\x48\xdd\xd5\x02\x43\x30\x19\x32\xb6\xfd\x77\xe0\x8a\xeb\xe9\xad\xf8\xa0\xec\x21\xcd\xf3\x9b\xf9\x9b\x82\xb7\x82\x37\xfc\xb6\x8e\x17\xea\xd6\x3f\x90\xe3\x19\x34\xbb\xb3\x5d\xa0\x03\x8a\x25\x27\xcc\xd3\xe8\x03\x04\xa6\x83\xae\x42\xf8\x24\x56\x58\x00\x95\xdd\xa5\x75\xa3\x0d\x7c\xdb\xbd\x6e\x5a\xe6\x91\x69\x0f\x75\x4d\xd3\x5b\x74\x31\xa5\x51\x20\x8d\x4e\x6f\x47\x30\xc9\x4e\x17\x5f\x8b\xbf\x1e\xd2\x04\xa9\xd4\xf3\x94\x06\xcc\x54\x12\xee\x0f\x16\x8f\x76\xbe\xb1\x28\xb7\xe5\x5b\x44\x4a\x21\xb4\x60\x06\x7a\x6a\x59\xae\x97\x0b\xb7\x94\xb0\x51\x84\x8a\xea\x70\xb8\x3b\x0f\x3a\x5e\x6c\x51\x79\x2f\xf2\x99\x45\x03\x19\x8d\xd5\x39\x02\x34\x0a\xc6\x97\xe5\xe2\x62\x66\x1b\x39\xe1\x46\x72\x45\x36\x0a\x9b\xb7\xcd\x3b\x88\x94\x0e\x23\x68\xf8\x81\x12\x3d\x6a\xff\x40\x64\xdc\x85\xa6\x1f\x7a\x83\x54\x34\x55\x37\x56\xc9\xc7\xfe\x37\xf2\xd1\xf4\x4d\x34\x1f\xb4\x3b\xd9\xab\x7b\x54\xfa\xd8\x6b\x69\x92\xed\xcd\xf9\x68\x2b\x93\x66\xc5\x16\x13\x13\x8e\x4a\xe0\x3a\xdb\x7c\xb2\x95\x49\x54\x25\xb6\x32\x21\xc5\xc5\x66\x49\xd7\x21\x78\x7a\xc4\x9f\xbc\x24\xa5\x3e\x45\xb7\xb2\xc7\x30\x67\xef\x12\x1e\xd0\x3c\x57\x5f\x68\xa1\xdb\x85\x7f\xa0\x08\x1c\x85\x96\x73\xd8\xe5\x3a\xce\xbd\x42\xd2\x28\x26\x2d\x7b\x82\x36\x54\xc5\x02\x86\xad\x0f\x88\x29\x2f\x37\x49\xc3\xff\xda\x57\x1d\x07\xed\x9a\xaf\xe0\x6c\xbd\xc7\xa0\x68\xb7\x4b\xed\xf3\x0d\x4e\x5b\xac\xd7\x95\xac\xd2\xf4\x2f\x41\x93\x6f\x58\x5b\x74\xc4\x31\x2b\x8f\xf8\xa0\x6c\x35\x00\x47\xb4\x6f\x12\x31\x1f\xfd\x52\xbe\x7a\xf2\x6e\x75\xc4\xd2\x25\xeb\xb2\x12\xdf\xb1\xde\x8b\x77\xc5\x87\x2d\x4f\xbb\x99\x8c\x99\xf3\x1d\xb9\x00\x51\x93\xb6\x83\x55\xe6\x59\xec\x14\xb1\xc6\x9a\xe6\xd2\x7f\x1b\x0e\xc3\xcc\xcb\x60\x1e\xc9\x4e\x5b\x7b\x0b\x4a\x7d\x83\xf5\x26\xa9\xb1\xe5\x49\xc6\xa8\xf5\xbd\x39\xf5\xb0\x8c\x1f\xaa\x6b\xba\x83\x6a\xce\x50\x02\xa8\xa0\xc9\xe3\x75\x79\xbd\x5c\xbf\x47\xc9\x0c\x08\x9d\xd3\xbe\xe7\x5d\x81\x3f\x9e\x92\x73\x0d\xff\x7e\x46\x36\xca\xf4\xd8\xc5\x18\xcd\xfa\x61\x18\x45\x47\xfd\x38\x61\xef\x2b\x51\x8f\x66\x42\x37\x14\x3e\x74\x93\x11\x0a\xf5\x54\xd8\x77\xf0\xe8\x93\xe3\x5f\x70\xf7\xc8\xe7\xbd\x4e\x0b\xaf\x53\xf4\x00\xf6\x3e\x8b\x93\x00\xd4\xc2\x7b\x1b\x43\x32\xc8\x37\x12\x52\xbb\x5d\xbf\xe0\xa9\xd7\xfc\xd0\xfb\xcd\x22\x39\x96\x1f\x7b\xa9\x13\xd9\xe9\x11\x42\x0d\x7c\x80\x9a\x3f\x3a\xc5\x75\xbb\x3a\x10\xf4\xc6\xfc\x89\x5f\xbf\xc9\xee\x64\x90\xb0\x88\x6d\x80\x11\x82\x66\x60\x18\x9a\x94\xb0\x36\xa2\x7f\xfb\xfa\x18\x21\x39\xc4\x70\x92\x44\x7a\xc9\x72\x6e\x68\x24\xc8\xce\xeb\x72\x7e\xd9\x97\xb8\xf5\xa9\x58\x62\x5c\x16\x77\xc0\xbb\xf2\xd0\x9f\x1b\x2b\xc8\xc5\x2d\x61\xca\x7f\x36\xa4\xe3\xc7\x33\x93\xd3\xbb\xbf\xc6\x48\xfe\xf2\x22\x4a\x59\xc9\xd6\xbf\x40\x26\x41\x00\x73\x86\xc1\xbd\x0a\x26\x2b\x7d\xf6\x6f\x16\xd5\xaf\x28\x30\x12\xd4\x57\xd3\x9a\xca\x6f\xb7\xfc\x3c\x09\x4a\xd9\xf8\x0d\xa7\xb8\x57\x78\xfa\x79\x65\xf1\xe0\xf4\x6a\xf2\x96\xa3\x80\x89\xbe\x42\x9e\x14\x25\x53\x3c\xda\x47\x02\xb9\x92\x56\xea\x93\x0c\x1b\xd8\x13\xaa\x39\x3f\xcb\x7c\x63\x20\xbc\x60\x43\x05\x6c\xcb\x4d\x3d\xcf\x5c\x84\xa8\x9b\xe8\x91\x19\xc1\x96\x0d\x42\x4b\x24\x26\x5e\x0b\xca\x58\xe5\xcd\x19\x28\x48\xd4\xc0\x8e\x6b\x59\x76\x7a\x3c\x18\x66\x04\xbf\x06\xd4\xa0\x35\x37\xd0\x81\xda\x81\x08\x11\x17\x94\x1f\x52\xf6\xf1\xb1\xbf\x74\xa2\x22\x94\x76\x7c\x9c\xed\xd4\xf8\x03\xa6\xa4\xd5\x74\xe4\x52\x1d\x44\x2c\xac\x20\x24\x11\x45\x01\x8c\x79\x27\xed\xe0\xef\xed\xb6\xe3\x21\x1b\x75\x4c\x28\x65\xec\xc1\x1b\x32\x0a\xee\x53\x88\x77\x5f\x39\xe6\x26\xb2\x7c\x9c\x15\x7b\x50\xb7\xc5\x8c\x82\x40\xd9\xa1\x4d\xfb\x9b\x96\x96\xb9\x51\xd7\x1a\x37\x8f\x01\xca\xfc\xd5\x24\x9d\x60\x1b\xad\x5d\x50\x2b\x64\xe7\x42\xd7\x98\x9e\xfd\x88\x28\xf7\x03\xcb\x1d\x3a\x5a\xe0\x3f\x14\x9f\x8d\xbe\xc6\x7e\x41\xb6\x58\x1f\xb9\x9f\x29\xad\x84\x1a\xa4\x63\xc6\x2e\xf1\x42\xd8\xab\xa9\x5b\x8e\x49\x84\xd4\xce\xfc\x41\x57\x0f\xb0\x77\x22\x65\xfa\xa9\x23\xc0\x94\x36\x2c\xe0\x68\x15\xfa\xd8\x80\xc7\xab\xda\x36\xf6\x0e\x65\xec\xec\x8b\xa6\xd8\x56\xe3\xd2\x5d\x45\x3d\x8f\xa6\x28\xca\x21\xb1\x8e\x9b\x15\xc6\xce\x1d\x8f\x13\xb9\x09\xc6\xf0\xa3\xe8\x54\x67\xe2\x0d\x11\xa5\xa2\x8b\xf2\x88\xe7\x08\xe3\x90\x4b\x85\x59\x32\x31\x58\x9b\x0e\x24\x26\x41\x4f\xe2\x8f\xaa\xfc\x9a\xa2\xab\x51\x65\x0e\xde\x00\x95\xd5\xad\xb9\xad\x01\xad\x3b\xa1\xad\x6d\x53\x16\x09\x46\xf3\xa4\xc0\xbc\x2a\x03\x4b\x20\x98\x56\x42\xdb\x94\x70\x3e\x23\xe7\xb9\x96\x7c\xeb\x34\xcf\xde\xd4\x16\xfb\xe2\x2b\x1d\xcf\xc6\xe6\xca\x8f\x7c\x20\x01\xcf\x7c\x13\x93\xe8\x81\x59\x2b\x6b\x3b\x8b\xb9\xf2\x96\x12\x43\x06\xcf\x93\xe1\x0b\x40\x94\x22\xa7\x6f\x0e\xb3\x42\x29\xb8\xdd\x39\x6b\xe7\xe0\xf4\x4c\xca\x11\xe0\x2c\xf4\x63\xa8\x03\x1b\x73\xb5\xc1\x64\x12\x7b\x86\x31\xa4\x68\xbe\xbe\xc0\x5a\xed\x66\x64\x43\x2d\xe5\x19\x95\xd8\x3e\x35\x4f\xa5\xc3\x6c\x14\xa7\xa8\x79\x9b\x8e\x5a\xad\x0e\x04\xbf\x5e\x32\x3d\xdb\xe8\xbc\xde\x6d\x0e\x53\x48\x96\x6b\xc2\x98\xb1\x81\x31\xa9\x37\x57\xc9\x84\xe0\x65\xc7\x2b\xec\x6f\x64\xeb\x59\x00\x72\xb6\x10\x5b\x8c\xc6\x0a\x1c\xaa\x29\x75\x7c\xe8\xec\x6a\x70\x06\x51\xa4\xdf\x65\xae\xda\x4e\x69\xeb\x03\x66\x03\xa3\x70\xc9\xa1\x22\x5f\xb1\x91\xcb\x8f\xdd\xce\xc5\x87\x71\x4f\x10\x4b\x66\x7f\x55\xad\x4a\x05\xbf\x8f\xf4\x65\x3e\x9a\x30\x88\x3f\xf3\xfc\x3e\x90\xb5\x21\x3e\x34\x9c\xf8\xc5\x64\x28\x0d\x3b\x2a\xe0\xfd\x18\xf6\x6f\xbf\x7c\xb7\xab\xa9\xa1\x7f\x29\x2e\x36\xaf\xd8\x8e\x3b\xaf\xc7\x83\x17\xd5\x04\xba\x54\x4e\xad\xc9\x0f\x28\x11\xd9\x75\x66\x22\xfa\x3b\x81\x9e\xf7\xaf\x3e\x5b\x28\x9e\x56\x2d\x30\xd2\xc9\x68\x67\x92\x7b\xdb\xe1\xd1\x2a\x8b\xe5\x70\x17\x8c\x75\x99\x67\xad\xc9\x87\x0f\x58\x07\xf0\x2f\x9f\xe0\x35\xc7\xb8\x22\xfa\x6b\x67\x6e\x03\x3e\x50\xdf\x60\xa8\xe7\x6a\x81\x27\x66\x75\xba\x24\x32\xa6\x87\x6b\x48\x73\x46\x9f\x8f\x78\xb7\xa1\x08\x70\xb8\x0a\xec\x37\xea\x7f\x24\x8d\xab\xec\x57\x81\x59\x5b\xb8\xa0\xbc\xca\xd8\x2e\xf4\x0f\x3e\xd7\xa8\x37\x7d\x1e\x7c\x6d\xcb\xd8\xcb\x76\x2b\xd9\x1a\x11\x5b\xd2\x45\x8f\x70\xdb\xfe\xc8\xab\x94\x37\x15\x0b\x37\xab\x19\x86\x96\x81\x1e\x0a\x8f\x74\x2f\x74\xbc\xfc\x3a\x42\x33\x2c\xaf\x68\x78\xb4\x1c\x6a\xdf\x69\xc2\xf5\x35\x85\x49\xc3\x52\xf1\x74\x30\xda\x8f\xaa\xdc\x73\x26\x16\x26\xe1\x4d\xcb\xa8\x2d\x84\x50\x0c\x62\x7e\x7c\x6c\xc7\xbc\xdd\xb6\x46\x64\x89\xab\xfc\x17\xd8\x94\xad\x0f\xf2\xe9\x20\x13\xd7\x5a\xaf\x67\xbe\x2e\x26\x31\x4e\x64\xb5\x06\x89\xb4\x14\x89\x88\xa6\x59\x22\x56\xb2\x7b\xf4\x01\xa8\x6b\x67\xe0\x61\x19\x9f\x91\xb7\xe7\x28\x43\x77\x13\x0c\xf3\x1a\x93\x73\x50\x39\x13\x5e\x1f\x85\xa7\xf1\x78\x7d\xd0\x56\x63\xad\x84\xa8\xcb\x82\x19\x71\x57\x01\x87\xa7\xe8\x30\x80\xa6\x11\xc0\xc9\x57\xb6\x89\x5b\x7e\xc3\xfa\x96\x93\x0d\xba\x8b\x4d\x0d\x3d\x75\x25\x46\xf8\xe6\x52\x4c\x4d\xf0\x58\xcf\x0f\xce\xfd\xc8\x0f\xb1\x3f\xe3\x3a\x18\x59\x0c\x24\xe7\xb7\xf1\xd3\x93\x6a\xca\x92\x40\x5e\xc2\x8f\xee\x56\x6f\xfb\xb5\xff\xdc\x6e\x42\x1a\x11\x6c\x0e\x64\xa6\x0d\xbd\x36\xf3\xd6\x7b\x6b\x2c\x39\x3a\xfa\x7a\x5e\x2d\xde\xdc\x3f\xfd\x7a\x83\x8c\xf9\xf4\xeb\xfb\xf2\x77\x7a\x74\xb5\x2e\x2f\x8b\xcf\xef\x4f\x3f\x3f\x9d\x7e\x7d\x7f\x7a\xfa\x35\x81\xcc\x07\xc8\xcf\xe9\xcd\xa3\xf3\xe5\x3b\x8c\x49\x8f\x23\x2a\xb0\xd9\x76\x5b\x82\x7b\xd0\xe9\xf4\x40\xfe\x34\xc1\xb7\x28\x4d\x07\x5a\x81\xf3\xc8\x14\xff\x9b\xcf\xf5\x48\x22\x51\xfb\x76\x43\xc6\xd4\x5e\x04\x70\x76\x42\x21\x14\xc4\x1d\x22\x78\xd7\x72\x5f\x3d\x79\xbd\x0c\x5f\xf5\xc0\xd1\x1d\x00\x93\xdf\x26\x23\x50\xa7\xf2\x7e\x8b\x44\xe1\x2f\x12\x8c\x15\x35\x58\xbd\x1b\x52\x24\x89\x1c\x55\x6b\x43\xf1\x41\xce\xfb\x5f\x25\x46\x09\xe2\x0e\x9f\x9d\xf0\x26\x27\x4f\x36\xfc\x5a\xc0\xdb\xbe\x9d\x25\x60\xd6\x53\x94\x54\xff\x86\x3e\x9e\x35\x34\x52\xe6\x5f\x14\x5c\x86\xae\xdc\x68\x38\xee\x7d\xf1\x0d\xda\x7d\xe6\x9d\xfd\x60\x53\x01\x6b\xc3\x61\x30\x2e\xff\xb3\x72\x5d\x4d\xe7\xe8\xe9\xd1\x39\x50\x11\xa9\xc2\xd5\xa3\x01\xe7\xf7\x61\xa0\xf7\xd9\x0e\x68\x1a\xbe\xea\x9c\x50\x01\xc4\x23\x52\xcd\x8f\xcb\xf5\x35\xf5\x30\xcb\x93\xfb\xf4\x72\x5b\x5c\x1c\x4b\xd1\x6c\x31\x96\xee\xbf\x38\xe9\x7f\x65\x5b\x96\x28\x13\x9c\x97\xe1\x73\x43\x64\x2d\x09\xe0\x7a\x98\xa7\x24\x43\x74\xf8\xd3\x22\xc7\x3b\x43\x98\x1f\x7e\x96\x0e\x89\xc0\xbe\x2a\x07\xbf\xed\xb3\x73\xa6\x5c\xd0\x23\x05\x38\xf0\x3d\x04\x01\xbc\xf7\x1a\x86\x2d\x05\x8d\x7b\xd4\x2f\x4f\xbe\xce\x17\xd3\xb7\xb0\x40\xe8\x0f\x4d\xdc\x9e\x36\x20\x1f\x1f\x6a\xd0\xe7\x1d\xd1\xe8\xd7\xbd\x1b\x68\x60\xd5\xfc\x00\xa9\xf3\x3c\x79\xf4\xec\xd9\xe0\xd1\xf2\x7a\x35\xdd\x04\xda\xf4\x0b\x4a\xc2\x32\x26\x78\x12\x3c\xef\x9c\x18\xff\xdd\x46\xfc\x6e\x0f\x51\x82\x36\xcb\x71\x50\x10\xb6\x63\xe6\x37\x3d\x7e\x20\x3f\xfb\x5f\xc8\x9f\x1f\x1a\x69\xbc\xf9\x81\x89\xab\xea\x5d\x39\xe7\xd7\x85\x96\x0b\xf4\x60\xe7\x15\x62\x1f\x0d\xc4\xa7\x5d\xc2\xc7\x24\xf5\x51\x34\x29\x17\x0c\x5f\x9f\x17\x94\xa5\xa9\x5a\x00\xbf\x15\x98\xa8\xc7\x36\x19\x67\x4d\x0b\x91\x12\x5d\x62\x82\x20\xbd\x6e\x82\x30\x9b\xad\xaf\xa7\x17\x00\xaf\x2c\xe8\x3d\x73\x24\xcb\x59\x06\x15\xb2\x4c\x22\xa2\x04\x6d\x25\x8c\x0b\x05\x83\x8d\xd1\x7b\x5a\x5c\xe3\x55\x4b\x0d\x6d\x8d\x2f\x62\x61\x71\xb7\xb6\x8e\x74\x05\x8d\xe3\x03\x05\x4b\xaf\x07\xfa\xfe\xab\xd6\xb4\xfd\xc8\x4d\x74\x4b\x3b\x82\x52\x61\xeb\x6d\x25\x16\x8e\xeb\xeb\x0e\xf3\xe1\xc7\x8b\x6d\xd1\x80\x7f\x52\xf3\x0e\x58\xcd\x0a\x26\x5c\xa7\x58\xca\xd0\x37\x11\x46\x61\x9b\x8c\x88\x22\xf8\x42\xc7\x22\xe6\x6e\xf6\x2d\x4a\x98\x6c\xef\xae\x05\x6d\x15\xc3\x14\x52\x15\x80\x1c\xf7\x26\x31\xfe\x29\xa4\xe5\x8d\x53\x24\x08\xde\xfb\x62\x10\x2a\xbc\x34\xc9\xec\xe5\x27\xd4\x3e\xbf\xae\x68\x59\xf0\x05\x10\xfe\xba\x04\x2c\xd5\x15\xac\x82\x93\x5d\xd6\xd8\x61\xed\x7e\x5d\x00\x34\x49\xaf\x62\x84\x0b\x10\xe3\xaa\x97\x7c\x73\x73\x0e\x64\x5c\xc3\xd9\xc2\x16\xc5\x5b\xd6\x5b\x79\x69\x45\xc6\xc4\xad\xd5\x63\x5b\x66\xa2\x0f\xa2\x92\x92\xda\xbd\x48\xf3\x56\x78\x20\xea\x3a\x5e\xad\xf1\xa5\xc3\x47\xf3\x6a\x55\xe8\x7b\x39\xf4\x3a\x51\xcb\x48\xdb\x2b\x25\x6e\xba\x60\xe3\x59\x13\xc2\xe8\x65\xa4\xb0\x3d\xd9\x81\xda\xda\x10\xa9\x27\xd6\x15\xc8\x0b\x7e\xe5\xda\x70\x50\x8f\xc7\xd5\x5b\xb3\x99\xd5\x08\xd8\x2f\x25\xe0\xaf\xd0\xb0\x1e\xf9\xc9\x90\x4b\xc0\x0f\x7e\xff\x06\x7e\x04\xaf\xda\x40\xf2\xbb\xe3\x9a\x18\x54\xee\x01\x35\x3c\xbe\x5e\xfe\x76\xbc\x2f\xef\xb6\x3c\x7f\x53\x6d\xf6\x65\x27\x06\xb7\xc5\x36\x67\x56\x7f\x0b\xe4\xad\x13\x23\x63\xe3\x2f\x0e\xda\xcc\x83\x3a\x2c\x88\x19\x5b\x2e\x16\x14\xec\x00\x29\x28\x09\xfc\x95\xa0\x2a\x27\xc3\x95\x72\xda\xe9\x39\x48\x9a\x40\x09\x43\x14\x2a\xd0\xd2\xe8\x72\x93\x1f\xff\x05\xfe\x07\xf2\x05\x23\xea\x58\xc4\x0d\x1e\x44\xb0\x4c\x6d\xc7\x59\x28\xfd\x54\x6f\x1b\x82\x9f\xca\x7b\x9b\x35\xfc\x37\x43\xb9\x6f\x46\x3f\x36\xfc\xeb\x3e\xa6\x8b\x2c\x98\xe0\xcc\x1d\x10\x8d\x36\x33\x8c\x71\x3d\xc3\xcb\xea\x78\xc0\x1f\x33\xc9\x14\x98\xcb\x17\x76\xe9\xe5\x76\x69\x2f\x78\xbd\x2b\xe8\xc3\x3d\x74\x48\xe9\x83\x46\x3a\xb7\x6b\xf9\xab\xec\x6d\x1c\xe5\xed\x27\x6a\xb6\x2e\xbc\x5e\xbb\xdd\x7d\xdd\x46\x98\xe3\xd5\xd5\x98\x59\x4b\x68\xee\x95\xb0\x06\x89\xfa\x59\x4d\x0a\xf5\x72\x15\x6f\x28\x55\x0a\xc2\xf0\x67\xb8\x2e\x98\x88\xbe\x8c\x08\xe3\xbf\xee\xa1\x25\xc8\x70\xd8\xb0\x5b\x7e\xf1\xa5\xac\x6d\xef\xdd\x34\x5b\x6a\xb6\x2c\xeb\x1f\x97\x9b\xa7\x1c\x64\x87\xa5\x86\xa7\x8b\x6f\x80\xea\x18\x7f\xa4\x33\x20\x22\xe4\xda\xcf\x97\x2b\xbd\x23\x07\x4a\x41\xd1\x06\x3a\x9f\x3d\xd3\x67\x13\xa5\xd5\x40\xb2\x28\x92\xc1\x7f\x45\x59\x2a\xdd\x53\x0f\x69\x97\x4c\x97\x32\xf4\x0d\xc4\x37\x50\x57\xcd\x41\xa8\xdc\x52\x24\x80\x0c\x64\x56\x1f\xd5\x1a\xa3\x8f\xaa\x40\xc3\xf4\xe5\x98\x15\xe1\xe4\x63\x4e\x10\xb4\xe2\x33\x57\x31\x22\x8c\x26\xa9\x28\x07\x6c\x54\xb9\x76\x52\x59\x11\xe7\x71\x10\xa3\xe4\xc4\xa7\x3d\x49\x23\x4e\x10\x53\xb9\x27\xe1\x15\x1d\xcf\xe1\x68\x2f\x62\x1c\x97\xf6\x90\xed\x41\x84\x37\x13\x62\x21\x6f\x01\xc0\xa0\x76\x6d\xaf\x15\xcb\x46\xfd\x81\x25\xa3\x78\xe8\x71\x74\x2c\xa2\x6f\x9f\xec\x95\xd6\x59\xc4\x1d\x62\x67\xf9\x20\x31\x87\x83\xf3\x15\x5f\xb4\x90\xb3\xeb\xdb\xf2\x06\x0e\xb9\xd7\xe0\x89\xf0\x0d\x5c\x0f\xff\xe5\x3c\xef\xc8\x15\xe0\xfc\x2b\x1f\xe7\xb1\xa4\x5d\x7c\xd1\xd9\xbf\xa4\xf6\xc5\x14\x4c\x69\x19\xf9\xc1\x02\x01\xe3\x94\xc6\x97\x99\x31\x7f\x37\xde\x5e\x04\xd4\x8a\x0c\xda\xd1\x2e\xbb\x1b\x67\x74\x30\x97\x83\xb3\x15\x00\xf1\x80\x3c\x65\x91\xad\xe0\xb8\x1f\x0c\xd9\x2e\x65\x63\x02\x7a\x27\xac\xa4\x60\x3d\x67\x77\x18\x44\xe8\xd9\xe4\xde\xd9\x6e\x7b\x36\xd6\xdf\x13\x0a\xfa\x73\x7d\x33\xdf\x54\x8f\xa7\xf5\x15\x94\x1c\x3f\x3c\xfe\xdf\x31\x3c\xce\x30\xba\x4c\xba\x40\xe3\x06\x74\x1e\x15\x41\x26\x4f\xb8\x40\xd2\x4b\xfd\x18\x2b\x3d\xb2\xb1\x5d\x63\x89\x6b\x10\xc5\xac\xc5\xf2\xfd\xb3\xc7\xf7\x5f\x91\x14\xbe\x58\x62\x04\xba\xfc\xae\xbc\x3e\x87\x13\x1d\x08\x64\x6c\xd9\x91\x27\x17\xf3\xba\x9a\xe5\x8f\x1f\xfc\xb7\x47\x8f\xbf\xf9\xf3\x93\xe3\x87\x4f\xfe\xfc\xf8\x78\x30\xb8\xb8\x3c\xfe\xcb\x9f\xbf\xf9\xb7\xe3\x2f\xbf\xfc\xf2\xab\xaf\xbe\xf8\xea\xcb\x13\xf8\x5f\x62\xf0\xb6\xa2\x44\xc1\x8e\x2e\x4d\xa9\xc1\xc0\x62\x2c\x7e\xb7\xd8\x0f\xf2\x65\x23\xcd\xa2\xb5\x06\x66\xc5\xaf\xe5\x4f\xf2\xb6\x54\xd3\xe9\x70\x34\x8c\x8e\x38\x7b\x61\xaf\x62\x2a\xcd\x4e\xe5\xfc\xfe\x40\xd3\x6d\xc0\x04\x2e\x40\x15\x48\x2e\x6b\x79\x30\x24\x2e\xa3\xb7\xdd\xcd\x01\x85\xfe\x07\xda\xc6\x2f\x51\x44\x3f\x2a\xb4\x33\x2f\xff\x28\x20\x86\x5c\x5b\x5f\xfe\x71\x60\xb8\x21\x34\x3c\x5c\x6d\xda\x66\x49\x03\x89\xb8\x30\xa3\x83\x4e\x23\xf2\xe8\x5f\xe2\x24\xe7\x39\x47\xbe\x75\x44\x50\x45\xe4\x93\x21\xb3\xc7\x99\xe3\x03\x1e\x1a\x6a\x14\xdd\xe1\xa2\xdb\x2d\xff\x15\x47\x5a\xaa\xf8\xaa\xa9\x84\xab\x40\x4c\xa2\x47\xec\xb1\xac\xa7\x2f\xb4\xfe\xfa\xe9\x5d\x38\x09\x9e\x09\x23\xdd\xea\x61\xf0\x3a\x79\x39\x13\x49\x54\xe2\x93\xd8\xa7\x5e\x68\x39\xa3\x12\x30\xf2\x42\x24\x94\x79\xb1\x17\x22\x9f\x2f\x1b\xb5\xb2\x35\x84\x6b\xa7\x25\xb8\x25\xbb\xb1\xe1\xf0\x6a\xf4\xdf\x22\x4b\x09\x0e\x12\x48\x95\xdd\x61\x66\x58\x7d\xcd\x31\xf8\xf4\x05\xf2\x5e\x2f\xa3\xc8\xcd\x1c\xd9\xaf\x9a\xf4\x17\xfa\xe8\xb8\x86\x76\x83\x0d\x15\x3a\x3a\xc6\xb0\x9a\x7b\x82\x3c\x7b\xef\x9e\x7f\x25\xcf\xce\x20\x28\x11\x39\xca\x23\xa5\xd6\xe7\x79\x1f\xfc\x44\x6d\x7a\xb6\xc2\x79\x11\x53\xe5\xd8\xa5\x16\x70\xca\x51\x4a\xe8\x92\xad\x11\x5d\x93\xb1\x48\x57\x0f\xe4\xdd\x69\x4d\xb6\xc4\x80\x36\x0c\x3a\x11\xda\xa5\xaa\x3f\x89\x7b\xed\x1f\xc1\x1c\x85\x03\x43\x5a\x88\x27\x8c\xcd\x4b\xc9\xd3\x2f\x4f\x3f\x1a\x38\x6b\xf0\xea\x7c\xe6\x68\x1f\x60\x4d\x43\xf3\xba\x84\x73\xc5\x71\xbd\xb9\xb8\x11\x86\x30\x08\x48\x3b\x0c\x5e\x24\x51\x0f\x4d\xde\x32\xd0\x35\xf0\x2b\xc5\x3b\x0f\x99\x07\xe9\xe2\x9c\x1f\x1f\x15\xf1\x9f\xda\x1c\x51\x04\xcb\x22\x95\xe0\x68\x18\xaf\xb8\x97\x50\xa9\x84\xe3\xa7\xb7\x04\x70\xe5\x88\x90\xe2\x04\xd9\xf9\x95\x43\x3d\x47\xe1\x12\xa8\xcb\xd1\xa1\x16\x9a\x91\x76\xd8\x2b\x2f\xa7\x4a\x6c\xe4\xc1\x29\x46\xba\x18\xbb\xe7\x9e\xcc\xac\xdc\x37\xda\xec\x6e\xa3\xef\x56\xf1\x80\x86\x76\xe0\x0a\xc9\xaf\x2e\x42\x33\x8f\x85\xec\xa2\xc4\x88\x8a\xbb\x57\x43\xba\x85\x7c\xeb\xeb\x63\x87\x03\x5a\x73\x6b\x0b\xb2\xbf\x6f\xd2\x52\x19\xf7\xbb\x1b\xa2\x36\x4d\xcd\x31\x80\x90\x2f\xe9\xc5\xcb\xb8\x4f\x0f\xba\xe3\x63\x0d\x5b\x7d\x71\xb3\x66\xeb\x6d\x0c\x16\x7d\xf9\x4e\xcd\xcf\xba\x5d\xae\xab\xef\x68\xf9\x1d\xd8\x38\xe7\xdc\x02\xda\xec\x19\xdf\x88\x9a\x00\xe7\xe6\x01\xe9\x1d\xaf\xdb\x6e\x57\x82\xaa\xb7\x84\xb0\x36\x1e\x16\x5a\xa7\x43\x3c\xbf\x0b\x7e\x98\xcc\x15\x4e\x22\x03\x8e\x97\xe1\xf2\x6c\x8b\x0e\x8c\x8b\xf0\x8e\x4d\xad\x3e\x60\xfc\x12\x5f\xb0\x6b\x5b\x71\x58\x5e\x0f\xa8\xc4\x2e\xa8\x46\x29\x59\x72\x59\xfb\xde\x13\x51\x62\xb4\xe4\x6a\x8a\x68\x58\x3c\xd8\xf7\x7a\x24\xcf\x1a\x6f\x2c\x1b\x7d\xb6\xaf\xc0\x29\x35\x5c\x15\xe7\x3c\x66\x2e\x5f\x73\xd6\x28\x20\x68\xdd\xd6\x08\xed\x39\x35\xe8\xf3\x46\xcc\xce\x5b\x19\xcd\x9e\x05\x42\xdc\xc6\x8d\x66\xd8\x42\xf5\xb6\x48\x66\x02\x22\xf4\xc9\x0e\x36\x77\xaa\x41\x2f\xbd\x45\x6b\xc1\xd5\xdf\x65\x2d\x8b\x9a\x09\xe8\x23\x78\xe4\xbe\xf6\xe6\xbe\xf7\x1b\x3e\x21\x61\xc2\x26\x2b\xb7\x4f\x5e\xbe\x1b\xd9\x5f\xf8\xe0\x46\x89\xcf\x15\x42\xfe\x64\xbb\xc5\x3f\xb9\xad\xed\xb1\x16\x0e\x26\x26\x5d\x23\xa2\x3c\xdb\x23\xbb\x94\xd8\x04\x8c\x6d\xc3\x0b\xcf\x4c\x9c\x8a\x60\x7a\x36\x74\x0b\xd2\xe7\x1b\xa4\x58\xd4\xd2\xd2\x00\x92\xe0\xce\x50\xce\x7f\x7c\x08\x57\x0e\x2e\x81\x96\xa2\x3f\x35\x6c\x6d\x08\x70\x1b\x3c\x81\x1f\xc4\xbb\xc1\x57\x86\xf8\xb2\xb6\xc5\x97\xa1\x14\x4d\x16\x5b\x0c\x54\x41\x78\x2a\xb9\xef\xf5\xc7\x71\x7c\x4c\x0d\xca\x85\x71\xe3\x89\x15\xb2\x21\x1d\xeb\x2f\x00\x91\x43\x21\xed\x59\x27\x18\x39\xc4\x2d\x13\x27\xaa\xc5\x53\x33\xac\x8e\x8f\x87\x19\x7a\x47\x35\xd8\x88\xc4\x27\x32\x0d\x86\x44\xef\xcd\x75\xbb\xe8\x8c\x4e\x4c\x86\x14\x97\x00\x79\xaf\x67\x6c\x1a\x31\x16\x35\xad\xf2\x1c\xe4\x28\x21\x95\x0b\xee\xd0\x86\x7f\x97\x89\xe8\x3c\xa3\x7e\x0c\xc6\x7c\xa0\x1f\x6b\x92\x6e\x8b\xfb\xe3\xb3\x0d\x45\x61\xc5\x90\xaa\xdc\x60\x71\xff\x6c\x8d\x5f\xa4\x3f\x27\x05\x0d\xc5\x9d\xa5\xc3\xe7\x96\x0f\xa6\x5b\xbc\x5f\x9f\xae\xcb\xe9\x16\x24\xc6\xcd\x72\xb1\x65\xc1\x0b\xce\x98\x15\x36\x5c\x5d\xbc\x71\xf5\xa6\x5b\x2c\xc8\x59\x12\x71\x82\x33\xe4\xc2\x60\xab\xf7\x7c\xdb\xe9\xcd\x66\x49\x9d\xd2\x2f\x7a\x82\x9d\xdc\xe8\x29\x22\xe3\x7a\x39\xaf\x79\x12\xb7\x7a\xc5\xb4\xe5\x60\xde\xdb\xf9\x72\xb9\xda\xd2\xd1\x16\xce\x8a\xdb\xe5\x0a\x92\xd0\xa3\x61\xb9\x98\xbf\x87\x1f\xbf\xde\xa0\x05\xe5\xb6\xbe\x58\xe2\x53\x3e\x04\xc6\x4d\x5d\x3e\xe6\xd8\xcf\xed\x90\x50\xa9\xe8\xda\xb7\x08\xed\x04\xfa\x51\xb6\x14\x7f\x4a\x27\xf4\xa8\x28\x61\x6e\xd8\xe4\xdd\x28\x4f\x7b\x31\xf6\x50\x08\x0e\x65\xcc\x16\x69\x54\x93\xa0\xaa\x71\x35\x5a\x04\x40\x2b\xf8\x3d\x6c\xf4\xf2\x09\x92\x1f\x89\xb2\xd4\x31\x9f\x38\x79\x0d\xaf\x7e\x27\xd4\x14\xd0\xfc\xe3\xa0\xfe\xb9\xd1\x8b\xb3\xc9\xf3\x0e\x18\xfc\x52\xcd\x3b\x3e\x37\xc0\x89\x0e\xdb\x6e\x1d\x15\xf9\xd2\x50\x10\x42\x8a\x65\xe5\x85\xe9\x66\xb1\xc4\xe5\xf9\x8f\xf5\xe1\x91\x76\x36\x7b\x84\x2b\x25\x8f\xcf\x01\x74\x5f\x82\x39\x65\xcd\x2e\xc9\x20\x15\x61\x80\xee\xdf\x7e\x33\xaf\xd5\x09\x35\xe0\x4e\x00\xeb\x05\x70\xf6\xa6\x77\x89\x78\x77\xd2\x9f\xf6\xd7\xbe\x83\x07\xef\xda\xa6\xed\xb5\x7d\x30\x80\xc2\x37\xf6\x15\xea\xd4\xf3\x76\xe3\x99\x90\x18\x94\x6a\xb9\x90\x61\x7c\x7f\xe8\x53\x80\xe3\x47\x32\x79\x54\x05\x57\x06\x96\x96\x1c\x7a\x9e\x3c\xf0\xaa\x76\x47\x5c\xe0\x71\x28\x26\xb6\xbd\x2f\x42\xdf\x16\x82\x11\x3a\x05\xf5\xc2\x34\x7c\x53\xd3\x73\x82\xa7\x74\x83\x69\x39\xfe\xc3\x67\x57\xf4\xaa\x26\x74\x17\x02\x2f\x3b\x3e\x67\xd0\xab\x3b\x96\xe2\xcb\x92\x58\x86\x1a\xfc\x5a\x1e\xd3\xec\x15\x2e\x6d\x18\x76\x1c\x84\x54\x81\xa2\xad\x2e\x0a\xff\x62\x7a\xc0\x0b\x8c\x78\x25\xb8\x18\x27\xff\x33\xa9\xc4\x1b\xce\xff\xff\x08\xe5\x43\x74\xa2\x2f\xd4\xee\x21\x96\x53\xa8\x86\xb4\x84\x4f\x91\xda\x0e\x82\x22\xa6\x8d\x7e\x08\x13\xa3\x98\x8a\x00\x9a\x90\x90\xf8\xbd\x90\x36\x42\x62\x7b\xdd\xbf\x4e\xe7\x9e\xe7\x8f\x4f\x01\xa6\xaa\xd1\x13\xaf\x68\x46\x6f\xd2\x8a\xd1\xd1\xa6\x41\x20\x6d\x1a\x91\x2a\xa2\x0c\x0f\xbe\x06\x65\x54\x11\x65\x38\x88\xdd\xaf\x5d\xd6\x2e\xed\x07\x21\xb3\xf9\x79\x07\xfb\xde\xa5\x6d\x0f\x17\x0b\xb9\x57\xf9\x20\x19\xcf\x4c\x19\xc3\x12\xd8\xe2\x35\xe3\xfc\x00\xe1\xb9\xd9\x71\x95\x28\x62\xc2\x30\xe3\x46\x19\xa5\x23\x76\x2e\xe8\xa8\xd3\x14\x8f\xde\x2d\x00\x82\x69\x4c\x85\x46\x89\x32\xd8\x24\x4f\xbc\x55\x94\x4c\xbc\xf2\xe4\xb2\xca\x2a\x8e\x96\xeb\x16\x58\x2d\x6e\x06\x19\x13\x1a\x40\xd8\x36\x61\x8f\x2c\x2f\x9d\x2e\x28\x79\xf9\xd2\xe6\xbf\x7c\x99\xc4\xab\x34\xfa\x2e\xc2\xcf\xed\xd6\x85\x41\xc1\x80\x23\x1f\x6c\x3e\xa3\x45\x8d\xfb\xa0\x22\xa4\x19\x43\xdd\x7b\x2c\xd6\x76\x8b\x0b\x45\xf3\x69\xad\x10\x03\xf4\xd9\xdf\x70\xee\x98\x01\xae\x7d\xe1\x05\xc1\x3b\x54\xf8\x7e\xb2\x24\x7f\x78\xc5\xdb\x75\xec\x90\x01\x6b\x58\x35\xc8\x27\x43\x1b\x0d\xce\x04\x01\x8d\x3c\xee\x8d\x11\x4c\xf8\x89\x32\xb7\x66\x02\xa5\xed\xf0\x8e\x83\xee\x04\xac\xda\x46\x0c\xb1\x75\x5a\xdc\x36\xc4\x03\xbe\x7d\xe5\x49\xec\x8a\x26\xcd\x0f\x15\x31\x3e\x56\xa0\x64\xe1\x5a\x1e\x35\x57\x27\xd1\x2f\x24\xa3\x17\xa3\xbc\xd2\x47\x7e\xcc\xf0\x1b\x4b\x17\x30\xef\x8d\x50\x24\x92\xd7\x2b\x1c\x51\x78\x4f\xeb\x65\xd2\xad\xd5\x9a\xad\xf0\x73\x9f\x66\xd4\xf6\x06\x8c\x10\x9a\xa3\x1f\x3d\x60\x80\x59\xa4\xc1\x82\x0c\x7e\x3a\x8f\xe3\x48\xe3\x93\xea\x56\xef\x12\xe6\xed\xd1\xe7\xdb\xd7\xf6\xe0\x60\x6b\x9f\xda\x53\x3d\x11\xa4\xa5\x6a\x54\x6e\xd4\xd4\xab\x13\x84\xf9\xe3\xd5\xc6\x2b\x02\x47\x29\x3b\x9c\x1f\x06\xa6\x1d\x60\xe2\xf9\x7b\x00\x3e\x74\x01\x61\x01\x7e\x15\x00\x8c\x0e\x36\xf2\xb2\x1c\x64\x88\xba\x5b\x20\x0e\x40\x46\xbf\x9b\x9c\x4a\x53\x2f\x3c\xb9\x0d\x41\x01\xf2\xb1\xa0\x5b\x23\x3c\x16\xbc\x93\x93\xf8\x72\x58\x02\x66\x86\x42\x83\xb4\xe8\x72\x75\x2c\xb9\x38\x44\xe7\x77\x41\x30\x70\xbe\xd0\x11\xa2\x8d\x2f\x0f\x18\x28\x5d\x6c\xf0\x41\x2c\x87\x1f\x11\xbd\xac\x44\x4b\xce\x85\x72\x46\x23\xf4\xba\xd3\xc0\xe6\xad\x3d\x29\x77\xb1\x56\xa2\xb8\x2c\xd5\x55\x9b\xda\xd0\x40\xc8\xec\xed\x49\x49\x7a\x94\x63\xf7\x69\x34\x67\x13\x6b\xe3\x63\x79\x50\xce\xce\xe1\x76\x7b\xc2\xaf\x59\xa9\x03\x01\x14\x18\x91\x76\x7e\x3c\x31\xd7\xd3\x77\xf4\xcd\x0f\x5d\x0d\xec\xa3\xb9\x36\xd0\x8f\xd4\x1d\x41\xc1\xdc\x16\x24\xc3\x99\x77\x8e\xbf\x71\x2d\x1b\x68\x18\x03\x71\xa4\xe2\xb7\x6d\x01\xed\x76\xab\x4e\xc1\x6e\xa3\x40\x99\xd1\x09\xd2\xb3\xb2\x14\x1f\x5f\x6b\x67\x99\xb3\x8b\x92\x34\x1e\xdd\x60\x69\x21\xa0\xa3\xed\x56\x4a\xb8\xf0\xfe\xb6\x0d\xef\x12\x8d\xe9\x36\x6d\x94\x35\x68\x74\x8d\xf6\x63\x2b\xba\x3c\xb2\x51\xb7\x84\x5d\x71\xf9\x8c\xd9\x0e\xa2\x3b\xf3\xaf\x49\x86\x8c\x59\xd6\xab\xcb\x45\x86\x9f\x8f\xe1\xd0\xe3\x79\xf7\xef\xd0\x64\x62\x1a\x4a\xfb\x30\xfc\x8d\x1f\x1f\x84\x1f\x24\xb0\x86\xe2\xcd\xe3\xa0\x0b\xd4\xef\x02\x48\xaa\xcf\x6d\x20\x09\xf1\x88\xc4\xe7\x03\x36\x94\x9d\x7e\x58\x49\x3d\x6d\x12\x5c\x71\x3c\xd0\x62\xb0\xc0\x4c\x78\xd0\x77\x57\x5e\xde\x10\x79\xdf\x01\xb9\x05\xa3\x1d\xe2\x46\xb4\x40\x76\x1f\x4a\xc4\xca\x9c\xba\x5d\xb4\x55\x58\xf0\x7e\xf0\x6f\xee\xe7\x03\xfd\x69\x8f\x00\xcc\x10\x1a\x57\x9b\x6d\xf2\xc8\xc8\x3b\x51\x37\x40\xcc\x53\x86\x8c\x4c\x87\x16\x1c\x08\xd6\x85\xab\xc0\xc0\x87\xcb\x0b\xb9\x8d\xe4\x82\xc0\xd4\xe8\x98\xde\x8c\xdc\x10\xee\x01\x88\x19\xef\xfd\x54\x8c\x26\x23\x52\x11\x9b\x95\xd3\xc1\x7f\xa4\x4a\xab\x5c\xd5\x58\x99\x8b\xe7\xe6\x06\x20\xdc\x54\x01\x88\x98\xac\x2c\x93\x36\x5e\x4b\x9d\x64\xcc\x5c\xf7\xa1\x6c\x8f\xc5\x8c\xe5\xc5\xc1\xba\xe3\x06\x8d\x63\xb6\xb6\x0a\xf1\x5c\x5d\xb2\xbc\x45\xc6\x60\xd7\xfb\xf6\x86\x5a\xe1\x7d\x6b\x9f\xde\x6d\xdb\x1e\x84\x1a\x23\x70\x8c\x6e\xc3\x99\x9d\xd2\xa6\xa2\xc7\x61\x43\x6f\x19\xf7\x68\x91\xe2\x05\x4a\x3d\x20\xe5\x58\xf9\x1d\x27\xd6\x93\xce\xe5\xdc\xda\x2e\xa4\x53\x18\x38\x2d\xdf\xed\x36\x4e\x82\x7c\x54\xb3\x57\xd0\xbe\xfc\xae\x7d\x1e\xd2\x07\xb5\x51\x54\x27\x52\xdf\x75\xbb\x9e\x52\xd0\x2f\xe8\x1b\x88\xb8\xdb\xec\x44\x9e\x8e\x3b\xe6\x28\x25\x19\xbf\x6d\x32\x56\x68\x26\x20\x5e\xe7\xcd\x14\x8f\xe6\xbd\x15\x86\x53\x42\xc3\x75\x88\xe6\x69\x8b\x40\x1c\x61\xe9\x5c\x1b\xcc\x98\xab\xc8\x3e\x4d\xee\x11\x77\x7b\x79\xa8\x17\x5c\xa6\x69\xb8\x0e\x24\xc7\x56\xed\xfa\x2a\x67\x73\x37\x60\x89\x44\x8c\xef\xa3\xad\xdf\xdf\xef\x8f\x9a\xb4\x27\x86\xf3\x22\xff\xbe\xc5\x50\x6f\xa9\xab\x47\x62\x17\x67\xee\x90\x51\xca\xf4\xc1\x80\xa6\xe7\xbc\x93\x26\xf0\x8b\x58\x6a\x62\x54\xa7\x9b\x53\xe8\xf1\x9f\xe0\x17\x7a\xa3\x2f\xd7\x20\xce\xa2\xc3\xc7\xb7\xf0\xcb\xb0\xcd\x46\x92\xfb\x0e\x54\xb0\x15\x8b\x97\x63\x02\x3f\xf9\x52\x31\x31\x17\xe5\x7c\x8e\x9e\x3a\x68\xb8\x96\xe0\xc7\x33\xfe\xe0\x1c\x35\x69\xa3\x9c\x9f\xe5\x29\x43\xb3\x5e\xde\x42\x95\x05\x00\xb0\xbc\x85\xe2\x0b\x8c\x1f\x34\xe7\x14\xf8\xc1\x29\x40\x46\x18\xac\x2e\x81\xbf\x3f\x4c\x57\x89\xa1\xb0\x41\x62\x0c\x9a\xd0\xc7\x37\xf4\x91\x18\x31\x6d\x2e\x67\x15\x19\xec\xe6\x6a\xbb\xfd\x44\x12\x92\x58\xd7\xda\xba\x71\xb8\x43\x8b\x6c\x1e\xff\x8a\x8d\xe3\x77\x72\xfb\x03\x8b\x30\xe0\xfa\x58\xc0\xe3\xfa\x2e\xce\x67\x83\x99\xff\x51\x6e\x48\x8b\x90\x55\xbe\x2c\x73\xb6\x4b\xe0\x1f\xb7\x39\xb8\xc6\x78\x66\x74\xf1\x09\x85\xee\x15\x93\xad\x74\x8c\x82\x54\x73\xbb\x20\x63\xff\x44\x29\x3e\xb1\xf2\x4c\x50\xad\xdb\x0d\x3e\x3d\x81\x9a\xec\x62\x9e\x2e\x36\x69\x58\x80\xd1\x32\x38\xc9\x72\x77\x91\xc3\x8c\x2d\x38\xa1\x80\x5c\xe8\x2e\x6c\xda\xf2\xc5\x32\x0a\x5d\xbf\x46\x27\x5e\x2c\x3c\x3a\x38\xe8\xc6\x5c\x34\x87\x2e\xa6\x63\xfa\x36\x6d\xd1\x2a\x5e\xf0\xdb\x9e\x2d\xba\x2c\x2c\xd6\x66\x93\xc5\x95\x30\xa4\x55\xd5\xa6\x02\xa3\x77\x59\xbd\x9b\x98\x6e\xb7\xc1\x44\x39\x14\x2a\xee\x66\x7f\x8c\xe3\xa3\x5f\x1b\xb6\xd2\x3e\x9d\x54\xc6\x7a\x7f\x10\xb8\xdd\x2e\xff\xed\xbf\xb5\x41\x6d\x47\x4d\xf1\xc8\x8f\x36\xb8\x87\x9f\x87\x46\x79\x9e\x42\xe7\xd0\x86\x9e\x1f\xc2\x0a\x2c\xef\x03\xc8\x08\x19\x7a\x73\xe7\x3c\xb0\xea\xb9\xeb\x4f\xda\x43\x79\xb1\xa2\xd3\x1c\xfc\xd8\xed\xcc\x61\xb8\xd3\x58\x8e\x94\x7d\xe5\x20\x3d\xb6\x0a\x6d\xcd\x69\x3b\xb8\x0b\x86\x2f\xc8\x12\x6b\x59\xe3\x5b\x39\x78\xf0\x76\x6b\x13\xbf\x98\xef\x7c\xea\xb4\x1e\xee\x3c\x6d\xf4\xae\x0f\x6a\xb3\xd8\x66\x05\x65\x94\x2b\xf9\x57\x2b\xc3\xc4\x35\xdc\xc4\xa9\x56\x89\xf5\x1e\x7d\xbe\xec\xfd\x97\x20\x97\x90\x95\x26\x15\xbd\x91\xb9\x20\xd5\x24\x7b\xbe\xb9\xcf\x8b\x25\x6c\x94\xb5\x26\x8c\x92\x84\xc3\xc5\x0a\x42\x03\x44\x67\xbf\x07\xd3\x9f\x06\xad\xbc\x10\x5c\xc7\xe5\x6c\x0b\xc1\x2b\x82\xe2\x6f\xd0\x38\x12\x58\x30\x79\xbe\x50\xdf\xa6\xaa\x1e\x3b\x70\x5d\xd1\x7b\xb8\xdf\x28\x1e\xe6\xce\x34\x16\x41\x24\x5a\xf0\x8c\x59\x4a\xc0\x67\x9c\x0e\xa3\xa6\x49\x34\x89\x15\x0f\x47\x20\xc9\x06\x24\x14\x04\x32\x19\xcb\xab\xd9\x46\x5f\xcb\x9e\x34\xc3\xfd\xb5\x9f\xfd\xa2\xf7\x89\xda\x0b\x99\xfd\x42\xae\x98\x38\x39\x38\xf7\x9d\x84\x12\xb4\x2e\x70\xe7\x20\xb1\xea\xdb\x51\x30\xd1\x48\x4a\x0e\xdd\xa2\x1d\xbb\x91\xa1\x92\x3f\xb4\x49\xea\xf5\x05\xfc\xfb\x3f\x79\xdc\x7b\x64\x8b\xfd\x07\x50\xf3\x20\x1b\x06\x1a\xd9\xc6\x41\x94\xc6\x6c\xf6\x0e\x69\xff\x08\x22\xf9\xad\x85\x21\x84\x01\xf4\x9b\xb0\x7d\x49\x8a\xc7\x06\xbe\xc9\x93\xa2\x8d\xab\x53\xc6\x07\xfb\x09\x5c\x55\xb6\xdb\x0f\x73\x82\x03\xd5\x0b\xab\xb7\x6e\xc4\x88\xe8\x7b\x9e\xea\x0e\x58\x8b\x94\x86\x5e\x29\x9c\xd9\x66\xb9\xbd\x93\xcb\xea\x37\x9e\x5f\xa7\x8a\xf3\x1e\x4a\x61\xff\x5a\xfe\x11\xa9\x3c\x25\xd1\x55\xeb\x76\x1b\x49\x61\x15\x56\x58\xb4\x4d\x8b\xf8\xd5\x87\x63\x85\xbd\x5e\x33\x8a\x04\x7e\x2c\xe9\x8c\xd4\xa8\x2b\xbe\xfe\xee\x1d\x0a\xa6\x33\x3e\x72\x9a\x44\x43\x52\xf8\xd4\x66\xe9\x2c\xb8\x05\x38\x48\x66\xf2\x6e\x4a\x93\xda\x54\x8f\x3e\x42\x37\xdf\xdc\x9d\x3c\x03\xbd\xf7\xef\x07\xa9\x75\x86\xc3\x32\x1f\x64\x59\x47\xcd\x0b\x97\x1b\x15\xbd\xd4\x6f\xbf\x5d\x43\xc9\xaa\x4e\x4f\x43\x89\x0a\x4a\x9f\xa1\xb1\x6f\x10\x46\x47\x78\x02\x45\xeb\x03\x86\x5c\x6c\x18\xf5\xa6\x7c\xcf\x8e\xd7\xf7\x5f\xc0\x4f\x74\x18\x5a\x82\x5c\xa8\x49\x50\x97\xbe\xf5\x85\x5c\xd8\xe8\x6e\xb2\x2d\x9d\x1e\xee\x8b\xb9\xd8\x0f\xcb\xf5\xea\x8a\x8b\x8a\xfb\x35\x1b\x74\xd1\xbf\xcb\x9b\xcd\xf9\xfc\x66\x4d\x9e\x48\x48\x36\xc8\x09\x28\x58\x06\x56\x18\xbf\xe8\x4f\xee\xd1\x1b\xe3\xfd\xb4\xdf\xcb\xf0\x91\x72\xeb\x99\x44\xfd\xdf\xf1\x73\x24\xe8\x99\x14\x44\xdd\x2c\xd5\x94\xb6\x36\x57\xd3\xc5\x6c\x5e\xae\xd9\xb3\xc5\x5d\x4f\xaa\x69\x21\xb5\x53\x9b\x8d\x94\xfb\xe9\xfc\xf5\xd3\x85\x21\x19\x62\x3a\xe7\xcc\xef\x28\xc3\xe5\x6b\x8b\x62\x8a\x6a\x01\xae\xcd\x72\x5d\xbd\xa2\x58\x1e\xd8\xfd\x63\xcf\x31\xc3\x99\xfa\xd9\xcb\xa4\xc7\x36\x04\xab\x34\xd7\x97\xbf\xb0\x76\x3d\x50\x0a\x05\x5f\xfe\x16\x5e\x9e\xd6\xb0\xa3\x0a\x32\xdd\x2b\xcf\xda\x01\x87\x33\x4f\xfd\x4f\x05\x90\x83\x9b\x67\x86\x5d\xe0\xf9\x06\x05\x21\x64\x34\xd7\xe8\xc2\xd1\x9e\x43\x2f\x34\xa6\x1e\x9e\x5c\x01\xee\xc7\x56\x6d\xcf\xf6\x42\x7d\x3b\x0b\x34\x3e\xb7\x31\x68\xed\xd7\xe4\xee\x84\x21\x4f\xa9\xf3\x6b\x74\xb0\x28\x60\xbd\xd3\x0d\x8e\xb7\xab\x05\x25\xd1\x6f\x10\xf5\x8d\x12\xf2\xce\x03\xad\x4f\x44\xe3\x85\x87\x33\x71\x26\x3f\x06\xc1\x74\xc5\x1e\x0a\xf5\x61\xfb\x96\x04\xd8\xc5\x86\xce\xa1\xf6\x86\x7b\xa3\x56\xa0\x21\xb5\xf3\x63\x58\x54\x72\xbc\x99\x90\x89\x02\x1b\x90\x2a\x4d\x61\xe4\x7c\x7c\xb9\xda\x91\x5b\x81\xa1\xf6\xc7\x0f\x26\x0c\x82\x84\x52\xee\xe3\x4f\x60\xb1\x68\xa5\xcf\x44\x5c\x04\xc3\x97\xc4\xb1\x5c\x54\xde\xf1\x13\x75\x85\xbd\xb7\x1f\x49\x01\x0a\xc2\xf1\x0a\xa4\x55\xec\x3b\xd7\x44\xd4\x46\x90\x4a\x68\xbb\x25\xda\xff\xc8\x2e\x2c\x59\x46\xac\x91\x55\x94\x6c\xee\x2b\xc3\xcc\xed\x1a\x22\x07\x2a\x5a\xb4\x42\xb0\xb9\x92\x3b\xbd\x5b\xe0\x53\xb1\x7b\x01\xd9\xbd\x7c\x8e\xe6\xd2\x12\xf3\xc9\xa6\x3a\xa2\xc1\xb7\xd0\x69\xd2\xfa\x7e\x41\x3e\xe4\xba\x85\x63\x71\x9d\x3b\xac\xf7\x5f\x2f\xab\x05\xe1\x79\xe7\xb3\x0c\x58\x04\xca\x16\x38\x36\x44\xcd\xe3\xcf\xdc\x82\x0b\x33\x30\x06\xab\x66\x58\x64\x3f\x22\x3b\xe7\x13\xc5\x2b\xca\xa9\x37\xab\x6e\x37\xf8\xf4\xdc\x23\x08\x3b\x1e\xff\xf1\xe8\x35\x23\x5d\x82\xde\x43\xc5\xd1\xd4\x47\xad\xa9\x6c\x78\xed\xf3\xbc\xce\x80\x0f\xea\x61\xec\x8e\x38\x25\xa5\x68\x19\x71\xe5\x2c\xb3\x24\x88\x3d\xa1\xd7\xb8\xfb\xf2\x06\x61\x91\x98\xb9\x9f\xfd\x56\x96\x15\x67\x14\xfe\x47\x96\x59\x2a\x18\x59\xbc\x4a\xe8\xde\x76\x3c\xf7\x7a\xe6\xc4\xeb\x3d\xb7\xa5\xe8\x46\xd2\x03\x2b\xa0\x6e\xde\x70\x64\x0e\x3b\x27\x64\xea\xc5\x01\x24\x9b\xc1\x8a\x5b\x76\x21\x4b\xa0\xd7\xe8\x91\x4e\x8b\x49\x0c\xf2\x5f\x7b\x3b\x0c\x6e\x4c\xb8\x10\x08\x4e\xb3\xd1\x4d\x4a\xb7\xa5\x4f\xdf\x80\xc4\x4d\x96\xb7\xa0\xc8\x82\xa8\xb1\x2d\xa1\x36\xbe\x7d\x1b\xe0\x0d\xeb\x8f\xf3\x3f\xf4\x62\xfc\x5f\xc1\x02\xd9\x05\x01\x61\xfe\x5f\xc0\x0c\x5b\x97\x39\x8f\x02\xc6\xca\x80\x76\xbb\x18\xb9\x8f\x63\xe1\xa7\x49\xfa\x62\x7b\x76\xd6\xcf\x58\xd7\x15\xb0\x17\x48\x06\xa9\xa7\x7f\x0f\xfe\x6e\x33\xf4\x80\x4b\xf1\xd7\x67\x19\x06\xda\x52\xba\x28\x5e\x17\x96\x64\x05\xb9\xaf\x11\xb9\x8e\xdb\x6a\xf6\xf8\xf5\xc4\x74\x3c\x82\xeb\x76\x15\xa5\xc0\x25\xdc\xda\xd2\xc4\xed\xd6\x8a\x22\xfe\x0a\x0b\xca\xf2\xc2\x24\xef\x84\x0e\xba\x22\x10\xe7\x74\xd9\x76\x40\xd9\x76\xeb\xb8\xaf\xf7\x40\xbe\x2b\xea\xb2\xd3\xe4\xde\x3d\xd4\xe7\x68\xca\x76\xdb\x69\x16\xf3\x19\xaa\xae\xef\xd7\x18\x96\xbb\xad\xc5\xf6\x85\x7f\x7c\x6c\x19\x13\xaf\x58\xc7\x61\xf9\xbb\x95\x3b\x65\x43\x8b\x79\x18\x73\x84\x79\x8f\xbb\x6d\xca\xe9\x7a\xb6\xbc\x5d\xb8\x46\x35\xc5\x77\x6c\xf3\x98\x76\x24\x2f\x09\xe3\x0e\xf4\xb6\xcc\x66\x2d\x3b\x69\xd4\x51\x83\xf5\x60\xe7\xd9\xd9\x97\x85\xf0\x9b\x1e\x17\xe2\x55\x1c\xac\x01\x79\xe1\xd2\xb6\xdd\xd3\x05\xd8\x64\x59\x9d\x13\xef\xdd\x61\xef\xc5\x53\xe1\x16\x19\x3d\x8c\xcd\x70\x84\xf0\xed\x75\x63\x4b\xb8\x26\x3f\x18\x2c\x32\x9c\xc7\x3e\x31\x93\x37\x39\x2a\x8d\xf7\x7c\xdf\x09\xde\xc5\x40\x82\x9b\x87\xdd\x1b\xb1\x82\xb6\xd5\xe7\x14\xa1\xe9\xb9\x27\x9c\x10\x33\xad\x98\x85\xfe\x3c\xdd\x5c\x15\x64\x8d\xe5\x1e\x27\x15\x26\xd3\x78\xde\x95\x7b\xe7\xab\x4a\x38\x78\xb1\x98\x89\xeb\x9f\x33\x3c\x26\xb4\xaf\xaa\x2d\x62\xeb\x3b\x5e\xe7\x78\x55\xce\x8f\x4c\xa1\x55\x31\xf2\x09\x2b\x67\x3a\x08\xcd\x17\x4d\xaf\xf8\x7f\x6b\x26\x75\xbc\xd3\x16\xaf\x47\x9a\xcf\x76\x39\x99\xec\x4a\x21\xdb\x19\x3a\xf7\xc9\xbc\x59\x6e\xed\x64\x64\x54\xc2\x81\xca\x98\xf2\x98\x95\x3a\x84\xfa\x49\xc4\x79\x33\x99\x93\x22\xec\x23\x67\x7b\xfb\x48\x5e\x60\x86\x19\x47\x5c\x60\x94\x61\xec\x67\x3d\x0b\xf3\x2a\xa0\x7a\x0d\xa7\xee\x52\x44\x13\xfc\x93\x71\xb3\x40\xa4\xcf\x79\xb8\x78\x73\x10\x4d\x40\xd1\x26\xd1\xc5\x85\x80\x5e\x8b\x28\x69\xf4\x47\xf9\x37\xd9\x79\x18\x5d\x7b\xf8\x2a\x9f\xe7\x77\x22\x93\x24\xef\xe3\xa7\xfe\xa7\x9c\x3d\x6c\x74\x00\x79\x0f\x96\x83\x8c\x4d\xf2\x56\xdf\x65\x23\xb9\x1f\x7d\x16\xf0\x57\x18\x70\x5f\xcb\xc0\x18\x8d\xc0\xca\xc3\x04\x3d\x43\xa9\x2c\xca\xbc\x2b\x93\x37\xa4\xbd\xb6\x30\x70\xbe\x54\x5d\x2c\x39\x88\x9a\x8b\x47\x60\x5f\x8d\xa6\x11\xf2\x76\xed\x96\x71\xd1\xb6\x07\xcb\x8e\xdb\x20\x78\x57\x8d\xc8\x8b\x1e\xe0\x17\x6b\x7d\xa7\xe0\xca\xd0\xc0\x7f\xd8\x92\x6e\x59\x04\x4b\x80\x68\xa3\x4f\x5b\x37\x16\xc7\xbf\x45\x91\x36\x95\xfa\x6e\x99\x66\x42\x7e\xae\x05\xdc\x18\xf5\x3e\xa6\x2a\x6f\x69\xbf\x94\x0e\xff\x26\x21\xe3\x38\x88\x4f\xb6\xa3\xa0\x75\xc5\xc9\x90\xe0\xb5\xcd\x90\x41\x0b\xbe\x81\xad\x04\x8d\x1e\x50\x53\xc0\x02\xb0\xc7\x67\x9b\x25\x6e\xe8\x69\x36\xcc\x1c\x73\x2a\xaa\xd3\xc1\xc8\x61\xa1\x21\xab\x04\xa2\x4a\x91\x06\x72\x20\xb2\x4f\xcb\x90\x29\x6a\xd0\xd8\x35\x3c\x89\x84\x46\x2a\xcc\xcd\x24\xba\xf7\xea\x7e\xab\x6f\xb1\xdd\xb0\xd2\x45\xb3\x0b\x66\x09\xdd\x2e\x64\x8c\xf9\xf7\xc4\xd6\xf4\xfc\xc3\x38\x56\x08\xa1\x3f\x6c\x71\x5f\xfb\xf2\xf4\x0b\x03\xbb\x5a\xd3\x5f\xb9\xb6\x4c\x59\xb0\x75\xf8\xa1\xe1\x77\xd2\x90\xd0\x15\xbb\x52\xe9\x67\x6e\x02\x71\xeb\x48\xfe\xa5\x4c\xa4\xdb\xd6\x35\xc5\x5b\x05\x21\x69\x78\x6b\x02\x6f\xae\x28\xba\xa1\x75\x48\x6e\xbf\xcb\x9b\x22\xe6\x3b\x4d\x64\xd0\xd2\x80\x1c\x46\x1b\xfc\xa0\x6b\xd1\xd0\xda\x38\x5a\x47\xd9\x9d\x6e\x27\x16\xd9\x24\xae\xa5\x7e\x12\xc7\x17\x32\x7b\x74\x29\x58\x84\x42\x76\xba\xde\x52\xef\xa1\xa9\xdd\x9e\x6a\xde\x43\x21\xcd\x0e\xf1\x01\x40\xb5\xf3\xf4\xb9\xe0\x8e\x1e\x14\x21\xd5\x4c\x24\x01\x64\x77\xbc\x4b\x04\xbd\x5d\x56\xef\x24\x97\xd4\x99\x15\x99\x47\xba\x03\x14\x1d\x48\x00\x00\xab\xac\x23\xdf\xe4\x42\xdf\xfe\x38\x10\x61\xdd\x4a\xf1\x69\x8b\x1f\xc4\xde\xd5\x41\x62\xfe\x41\x26\x1b\x14\xbe\xdb\xe9\x4b\xbd\xe3\x13\x89\x25\x29\xbc\xdf\x72\x39\x66\xfa\xd4\xad\xe5\x9d\x40\xdb\x8f\x05\x47\x8e\x30\xbd\x44\xcf\xe2\x9f\x71\xc3\xfc\xd8\xd7\x32\xfe\x87\xef\x55\xcf\x5d\x5a\x51\x36\xaa\x6d\x51\x91\x19\xe2\x4c\x82\xd3\xc2\x6f\xe9\x23\x19\x14\x02\xc0\x25\x60\xd1\x22\xfb\x93\xe1\x49\x93\xac\x79\x43\x9f\x2c\x77\x84\xb0\x79\xee\x10\x13\x75\xf5\xf4\xfa\xba\x9c\x55\x80\xab\xf6\x3e\xd3\x4e\x73\x37\xd7\xa5\xee\xa7\xed\x3f\xba\x64\x7a\x10\xee\xc7\x47\x2a\x24\x31\x99\x2f\xdc\x8f\x5d\x65\xda\x75\xf1\xfa\x2d\x4d\x5b\xa9\xa0\x79\xd8\x9a\x70\x7c\x36\xce\xd0\x93\x97\xa7\xe9\xc8\x84\xbd\x04\xa8\xe2\xe7\x95\xa1\x9f\xe8\x75\xc6\x40\xac\x40\xab\x52\x61\x8d\x69\x3b\x6f\x94\x31\xa0\x9b\xbf\x87\xc3\xd4\x7b\xe7\xd0\x12\xd9\xb2\xde\x28\x95\x39\x0e\xe8\xa7\x36\x68\xcf\x44\xab\x5b\xe7\x31\x96\xef\xaf\x02\x89\x1e\x0e\x1b\x8d\x65\x8c\xfe\x2b\xf1\x32\x0e\x35\x66\xed\xe7\x3c\x63\xf7\x52\x91\xa1\x28\xac\xba\x5f\x82\xf6\x22\xdf\x8f\x46\x68\x46\x1c\xd8\x95\x69\xa3\xac\x6d\x97\x6f\x66\x9d\x04\x3b\xb4\x3c\x5b\x24\x09\x7e\x7e\x54\x9d\x98\xa2\x2e\xf0\x53\x6d\xec\x29\xcc\xd4\xbe\x6e\xf8\x7d\x36\xc6\x80\x3c\xb6\x3b\x0c\xa0\x17\x5f\x82\x96\xf3\x7e\x35\xc1\x53\x5b\xcb\x21\x9b\xdc\xae\xa4\xcd\x31\x24\x4e\x8a\x90\x80\x82\x2c\x6f\x4d\x78\x4a\x52\xb1\x28\x42\x25\x89\x61\x6b\x78\x12\xef\x69\xbf\xc6\x9b\x26\x75\x69\x47\x5b\x7b\x2d\xc3\x81\x87\xc6\x50\x64\xa2\x71\xc8\xb3\x00\x8c\x6e\x57\xbe\x62\xe5\xdb\x50\xd3\xf5\x9c\xed\x13\x02\x17\xa6\xdd\x29\x47\x69\xc0\x92\x98\x54\xda\xb9\x78\x50\x3e\xde\xbe\x6e\x1c\xde\xf7\x35\x4a\xb0\xdb\x56\x9d\xda\x81\xb4\x0e\x41\x9b\x78\x69\xee\xb7\x82\x8f\x37\xbd\x6b\x6c\x64\x2a\x8b\xc4\x67\x9d\xcc\xdf\x10\x65\x43\x23\xbf\xf8\x8b\xe5\xea\x3d\x9f\xba\x1c\x65\x90\x06\xa8\x5a\xe0\x23\xfc\xf6\xf0\x84\xbd\x91\x75\x0e\xf9\x30\xc8\x87\x28\x02\x28\x7a\x85\x24\xa9\x7f\x53\x58\xa2\xd0\xda\xde\xd5\x9e\x3b\x41\x8a\x6f\x28\x65\xb1\x75\xa3\xbd\x12\x6c\x14\x82\x0c\x31\x80\xdc\x65\x04\xbc\x36\x4d\xb7\xc3\x35\x17\xa2\x9f\xfa\x2e\x6f\x90\x2f\x0e\xa2\xf4\x5b\xce\x86\x8d\xf3\x5f\x30\x7a\xdc\x9a\xb0\x1b\x1b\x96\x0c\x15\x60\x64\x5a\x88\xa9\xb8\x0e\x18\xdf\x98\x34\x29\x82\xaa\x9c\x36\x0c\x44\x91\xf6\x93\x57\x50\xad\x5f\xaf\x2f\xf0\xde\x34\x14\xfd\x0d\x06\xab\xf4\x2b\x05\x1e\xcb\xc1\x31\xce\x2f\xe4\x1d\x3e\x84\x5d\x5e\x97\x9b\xe9\xff\x28\xdf\x17\x9d\x4e\xf0\xad\xf3\x0b\x13\x37\xc7\x30\x3b\xe1\xa7\x70\xd2\x10\x33\xac\xa7\x60\xa3\xd4\x3a\x4f\xa6\xf3\x0d\xb4\x73\xc4\x67\x83\xfa\xe8\x62\xba\xb8\x28\xe7\xc8\x84\x8e\x2e\x36\xeb\x39\x66\x05\x5b\x33\x23\xe4\xe7\xab\x69\x8d\x21\x8c\x08\x86\xa3\x35\x54\x00\xa9\x58\x0a\xd0\xd9\x1f\x93\x79\x2c\x14\x50\xe6\xd9\x66\x7a\xbd\x3a\x7a\x0b\x67\x9d\xa3\xdb\xab\xea\xe2\x2a\x7c\xe1\x53\x68\x0e\x2f\x6b\x1d\x9d\x08\x78\x17\x57\xe8\x2a\x0a\xff\x3c\x02\x64\x60\x74\x26\xfc\x0f\x7f\x47\x4d\xe0\x70\xe3\x1d\x44\xc7\x1d\x5d\xfc\x33\x02\x09\x0e\x3b\x09\xf4\x55\xc8\x03\x4e\x52\xad\xaf\xdd\x8e\x1a\x29\xb9\x4d\x11\x68\x64\x9a\x60\x3b\xf3\x96\x83\x8e\x81\x77\x8d\x23\xfe\x03\x28\x9e\x57\x50\xf4\xef\xf2\xf7\x1f\x47\x97\xeb\xe5\xb5\x50\xce\x11\xc7\x13\xfd\xbb\xfc\xfd\xc7\x11\xec\xbd\xe5\xdf\xe9\xdf\x7f\x1c\xd5\x17\xeb\xb2\x5c\xfc\x5d\xfe\xfe\xe3\x68\xb3\x94\x5a\x9f\x84\x0a\x64\x22\x18\x74\x94\x01\x86\x23\x89\x01\x62\x35\x62\xac\x67\x87\xc5\xdf\xc6\x03\xcd\xe5\x79\x89\xc3\x16\xc4\x12\xc8\x6a\x25\xed\x30\xc7\x83\x56\x84\x43\xbf\x21\xc5\xef\x39\x3b\x23\x70\x85\xd6\x88\x9f\xcf\xe5\x08\xdf\x36\xd7\x8d\x8a\x61\x68\x74\xde\xc3\x08\xbf\xf4\x1a\x6f\xbf\xa6\x77\x6f\xbf\x2f\x2f\xa1\x2f\x8e\xfc\xcd\xc1\x59\xbd\xe4\x93\xec\xd8\x95\xe7\x16\x1a\xe5\xfd\xe4\x93\xcc\xeb\xfc\x1f\x71\xe7\xff\x68\x74\xfe\x7c\xb9\x6a\xe9\x9b\x52\x5b\xba\x8e\x4b\x7b\xa9\x27\x18\x03\x4d\x85\x2a\x6f\x25\x76\xbb\xde\x4c\x79\x42\xa0\x57\xa4\xf0\x27\x38\x64\x54\x8e\xe8\x2d\x9d\xe5\x5e\x69\x1d\x2c\x2d\x9c\xad\x44\xf6\x29\x42\x8f\x59\x7f\x69\x0d\xba\x5c\x64\x34\xc8\x1f\xe8\xcf\x2f\xf2\x2f\xf5\xe7\x83\xfc\xc4\x2d\x21\x11\x23\xf3\x3b\x7c\x76\x39\xbf\x53\xcd\x10\xc5\x75\x25\x89\x28\xbf\x6b\xe8\x84\xf7\xda\xc2\xf2\x01\x4d\x6c\x61\xbb\x5d\x67\xb0\xa3\xde\xd2\x7c\xec\x16\xd7\x77\x68\x7c\x94\xba\xdf\x20\x0a\xe3\x55\xa7\x1a\xc9\xf0\x2b\x01\xad\xbd\x4b\xfc\x4d\xff\xc9\x10\x7c\x94\xee\xad\x7b\x04\x9a\xf7\x57\x6c\x20\xc3\xa3\xb3\x1f\x78\x83\x52\xa9\x2f\x3f\x9e\x8c\x09\x6e\x97\x12\x31\x91\x49\x76\x06\xed\x62\x0e\xa1\x40\xdf\x49\x3b\x04\x08\xb6\x21\x23\xc5\x9f\xe1\x40\xdb\x7a\x5e\x02\xfe\xa0\xeb\x12\x84\x86\xf2\x66\xc1\x33\xe3\xcb\xf9\xed\x87\x73\x11\xf3\x5b\x8f\x23\xe1\xf6\xc9\xb0\xb3\x79\xb2\x5f\x15\x6d\xff\x4c\x5d\x5d\xdf\xcc\x83\xc7\x1c\xed\x75\x87\x9c\x49\x79\xff\x62\xf6\x56\x46\xe6\x04\xb1\xa8\x20\x55\x3c\x2b\x83\xaa\x7e\x26\x3d\x70\xec\x61\x1f\xb2\x1c\x43\xfb\x0c\xb9\xfd\x51\x9b\x66\x23\x65\x9f\x79\x82\x26\xdb\x63\x54\xe2\x6e\x78\x70\xab\x68\x55\x29\xed\x53\x54\x39\x9b\x59\xef\xce\xc7\x7f\x98\xdd\x26\xda\x5b\xfc\x96\xc8\x9a\x72\x21\x44\xe2\x6b\x5b\x1d\xb9\xbd\x6f\xc9\x49\xbd\xfa\x48\x24\xbb\xb6\xc0\x9d\xda\xbc\xfa\xf7\x15\x56\x95\xcf\x31\x4f\xf8\x5d\xf1\x27\xac\x8c\xf7\xfd\x26\xc5\x28\xb4\xdd\x61\xd2\x2b\xc0\xfa\xa8\xb8\x2d\xb6\x0f\x95\xce\xdd\x1b\xe4\x8c\x22\x17\x7b\x62\x7d\xc1\xd2\x4e\xb0\x42\x8e\xaa\x45\xbd\x41\x69\xc7\x9a\x14\x71\xbd\x11\x96\x87\xb3\xec\xfa\x82\xed\x85\x78\x91\x84\xb2\x35\xb6\x68\x99\x47\xa1\x45\x39\xa9\x39\xb7\x54\x60\x16\x25\x6e\xb7\x98\xea\x53\x3d\xeb\x03\x31\x15\xd8\xef\xcf\x01\x15\x30\x38\x8d\xe4\x54\x1f\xd7\x7c\xbe\x66\x23\x77\xf8\xc9\x4f\x5b\xe7\x01\x74\x3c\x7a\xcf\xca\x85\x96\x05\x71\x46\xc6\x8b\x0c\x46\x25\xb5\xc2\x61\x40\x93\xac\x36\x71\xb1\xbc\x95\x47\x78\xe3\xc3\x0a\xde\xa6\x58\x97\x82\x58\x3a\x77\x73\x10\xce\x12\xca\xf4\x9b\x25\x01\x7a\xd7\xc4\x9c\x3f\x26\xd3\xa6\x44\x8a\x0a\x1c\x50\xfd\x04\x25\xc3\x35\x16\x3f\xde\x2a\x81\x50\x82\x29\x1f\xee\x9b\x5b\x37\x01\x86\x64\xfa\x68\xfd\x8e\xe2\x84\x14\x04\xf0\x60\xda\x71\x49\x99\x48\xd5\xf2\x29\x10\x35\x87\xda\x84\x29\x6a\x1e\x16\x7b\x53\xb9\x63\x50\xe7\x89\xc2\x3f\xef\xb6\x05\x85\x03\xc6\x52\x6d\x48\xcd\x63\xf7\xf8\x83\xb8\xf7\x01\x62\x67\xfa\xb8\xf3\xc8\x31\xe0\x8e\xa4\x68\x44\xf0\x3a\x4f\xe8\x37\x70\xa5\x75\xc2\xc2\xf5\xbc\x9c\xbe\x2d\x35\x99\x76\x25\xf7\x1a\x2b\x20\x08\xcf\x13\xee\x21\xdd\x40\xcb\x86\xd9\x93\xe2\x2e\xd8\xdd\xa0\xb4\xd1\x5b\x10\xfa\x60\x6e\xd2\xd8\xce\xd4\x7d\x72\xe3\x29\x60\x45\x96\x2a\x5a\x24\x2b\xcf\xb6\x2d\x52\x15\x8a\xcc\x9c\x76\xa4\xfc\x76\x2b\x3f\xf0\x21\x75\x11\xdc\x54\xd3\x2f\x2f\x00\xd4\x29\x67\x68\x87\x9e\x06\x92\x56\x4e\x53\x7b\x48\xba\xc6\xa6\x75\x54\xeb\x73\xbe\xc6\x6b\x09\x71\x67\xc8\xda\xbf\x69\xc8\xce\xcf\x26\xc9\x53\x47\x9e\x7b\x83\x8f\x62\x29\x54\xdc\x91\x49\xda\x47\x8b\x67\xfc\x24\x1b\x3a\x8b\x84\xad\x62\x08\x4b\x2e\x41\xe2\x58\xff\x25\x37\x8f\x67\xc0\x15\x86\x6c\xd5\x04\xef\x71\x28\x95\x01\xe8\xba\x5c\x64\x59\x83\xed\x17\x07\xbd\xa4\x3c\xfe\x16\x64\xb3\x7c\xaa\x2e\x5c\xd8\x8e\x73\xa6\x41\x05\xca\x75\x1c\x3f\x1d\xd3\x4c\x12\xe0\x0a\xe5\xce\xe6\xb8\xfc\x82\x6d\xe3\xf0\xa5\x28\xc9\x7e\x79\xae\x4b\xd3\xcd\xce\xfe\x5e\xd1\x2e\x23\x88\x01\xfe\x11\xe2\x5a\xd8\x91\x67\xb8\xd1\x92\xcb\x4b\xd9\xf7\x19\xb0\x2a\x78\xb9\x5f\x8f\x8c\x71\x55\x8a\x4b\x13\x1d\x6d\xd4\x82\x08\x66\x04\xb7\x51\xeb\x98\x3f\x4c\x44\x62\xc9\xc2\x85\x2c\xa6\x1d\x5e\x9a\x94\xce\xcf\x82\x7d\x80\xd2\xb9\xd0\x7e\x4a\x77\x96\xf4\xa2\x0d\xf3\x83\x13\x65\xa3\x34\x6d\x3f\x8a\x6c\xb7\xce\xc3\xde\x26\xb6\x12\x10\x0f\x08\x37\xd5\x72\xbd\x79\xcf\xe0\xf4\x5f\xf2\xdf\x26\x21\xd9\xc7\xe4\xec\x39\x2f\x94\xbf\xb5\x1d\x84\x4e\x63\x8b\xbd\x7c\x7d\x53\x6f\xa4\xc5\x19\x6d\x08\xd1\x95\x60\x63\x7d\xee\xeb\xbd\xd9\x5c\x1b\xb5\xb4\xf6\x3a\x88\xfa\x74\x54\xa4\x9d\x79\x57\x53\x04\x23\x9f\x65\xf6\x21\x8c\x8f\x30\x74\x26\xc2\xc7\x8d\x5b\x40\x6e\xe1\x20\xc3\x78\x36\xad\x8d\x8d\xf8\x3d\xb7\x3d\xa2\x10\x90\x51\x3b\x13\xf0\x0b\xee\xc7\x5e\x27\x5a\x27\xee\x66\xda\x1e\x5b\x5c\xd2\x73\x35\xca\xf8\x08\xac\xb5\xae\xbd\x98\xb7\xb4\x0d\xa6\xc1\x5b\x0e\xec\x9a\x8c\x48\xff\xba\xc5\x93\xc2\xc5\x5c\xe9\xe3\x46\x24\x2b\xa3\xe3\x05\x64\xf2\x4f\xf4\x7e\xfa\x28\xda\x75\x0f\x6f\x81\xf6\xe0\xfb\x61\xa6\xd3\xca\x53\x04\xa7\x40\x78\x87\x57\x7d\x0b\xaf\x91\x63\xbd\x65\x36\x81\x18\xc4\xfa\x06\x7b\xf4\xe7\x93\xbf\x7f\x1e\x6f\x91\x7c\x24\x88\xc0\x94\xae\x87\x4e\xac\x7f\x48\x3c\x33\xed\xd4\x81\x02\x90\x3f\x55\x66\xcf\xe5\x3a\xd1\xc9\xb0\x55\xc8\x82\x42\x93\x16\xae\x48\x81\x4b\x05\xac\x5e\x8f\x54\x5b\xa2\x9d\x88\xad\xcd\x69\x28\x6a\x64\x48\x52\x68\xdb\xac\x60\x7b\xc7\xc7\xda\xa2\xd7\x5e\xdb\x11\xb6\xd1\x64\x7b\x7c\x7c\x5f\xae\x65\xf3\x6c\x6b\xe3\x48\x17\xb8\x97\x0b\x8a\x56\x65\x63\x6d\x92\xa2\xe0\xdb\x05\x5e\x1f\x36\x0c\xcf\x36\x6c\xc1\xdd\x88\x0e\xee\x99\xb5\xd2\x1d\xf1\x8c\x5e\x97\xb1\xfd\x58\x97\x1d\x17\x30\x7c\xe8\x9b\x6a\x72\xb3\x7c\x1a\x10\x2d\x48\x08\xa3\x98\x69\x92\x5d\x07\x02\xeb\x2f\xb6\x1d\x00\xea\x6c\xc4\x58\x47\x8b\x6f\x39\x8c\xf0\x29\x87\xa0\x99\xa2\x05\x8e\x5c\x8b\xa3\x45\x6e\x1c\x47\xce\x9a\x46\x63\x53\x04\x08\x35\xe3\xd5\x0e\x33\x0e\x0d\x38\xc3\xd7\x2c\xc8\x42\x01\xfe\x7a\xc7\xb6\xe1\x7f\x21\xcb\x55\x34\x22\xbb\x5c\xf8\x91\x6b\x75\x8c\x78\xab\x0a\x63\xc6\xe0\xeb\x34\x31\xfc\xee\x44\x83\xf4\xc3\x10\x60\x19\x3e\xe2\xa5\x64\xcd\xf5\xda\x99\xc4\x0e\xdf\xa0\x20\xff\x02\x29\x25\x4e\x08\xde\x57\xe4\x36\xd5\x1a\xc3\x31\x5a\x7a\x76\x63\x62\x82\x03\x88\x43\x87\x34\x0c\xa7\x89\x6f\x13\x7f\x80\x30\xc3\x30\xdd\xfb\xe9\x17\x0f\x9b\x30\xde\xbd\xcd\x61\x4b\xce\x82\x96\x7c\x0e\xde\x73\x4c\x98\x0d\x5b\x88\xb3\xd1\xfe\x2a\x52\x4f\x70\xaa\xbb\x9e\xd5\x88\x88\xf6\x10\x14\x15\xd0\x27\x45\x39\x39\x34\x6e\xe1\x09\x69\x31\xbc\x18\x35\xcf\x39\xbd\xa4\x9f\xf4\x5a\x8a\xe6\x2d\x47\xa2\xe6\xad\x77\xd3\xb1\x84\x67\xec\xd0\x82\xde\xb7\x18\x2f\x2f\xa3\xd5\xe8\x2d\xc4\x70\x11\xca\xf9\xcf\x12\xbe\xd8\x81\xe9\x8c\x34\xd7\x54\xc6\x6f\xad\x1c\x5a\x32\xba\x62\xb8\xa4\xaf\x09\xfa\x08\x12\xf4\x37\x33\x4b\x85\x01\x01\xe2\x01\x39\x26\x99\xc3\x84\x47\x1a\x51\x2d\xb2\x33\x37\x8b\xb6\x26\x1a\xb5\x05\x89\x52\x9d\x6a\x2a\x6d\x34\x43\xc9\x9a\x8f\x81\xa3\xb1\x4e\x10\x96\x0f\xb6\xe9\x35\x37\x68\x09\xb8\x3d\xb2\xd0\xda\x7a\xe8\xa2\x20\x5a\x37\x37\x0a\xe7\xb0\x80\xd9\xdc\x7b\x43\x6d\x4f\x54\x13\x3c\xc1\xf6\xe1\x19\x53\xcd\xb3\xad\xcb\x16\x17\x3b\xd7\xbe\x18\x31\xee\x7d\xec\x85\x84\x32\x8d\x54\xeb\xf9\xb5\xb7\x6b\xb8\x5d\x3f\x24\x0c\xc2\x0e\xea\x1c\x87\x9d\x0c\xc2\x36\xab\xc6\x7b\x95\x4c\xb3\x8e\xaa\xfa\xc7\x29\x5e\x42\xa4\x1e\x66\xd7\x0b\x4a\x62\x39\xe9\x72\x81\xe1\xe6\x76\xb6\x02\x87\xc4\x78\x84\x9b\xbb\x3e\x37\x8c\xbf\xf1\x82\x18\x8d\x5e\x14\x64\x4a\x2c\xfc\x97\xaa\x22\x77\x69\x2c\xcf\x96\x1b\xf0\xab\x57\x50\xb0\xee\x27\xe8\xe5\x47\x35\xf5\xd1\x20\x39\xe5\xf2\xe3\x7f\x54\x45\x2c\xd8\x27\xc6\xa6\x49\x50\xa8\x9d\x83\xf1\x7a\xba\x7e\x63\x83\xf6\x7a\x03\xbb\x5c\x8c\x7d\x9d\xe8\xe5\xc2\x55\xc1\x98\xe2\xeb\x4d\xaa\x5c\x16\xdf\x75\x3c\xf8\xea\xe8\xd0\xbb\x2a\xba\x5c\xd0\x2b\xbf\xee\x8a\xc8\x06\x29\xbe\xc4\x63\xdc\xfc\xfd\xdd\x4c\x1e\x86\xf4\x60\x7c\x56\xfd\xf6\xdb\xbc\x74\x74\x2a\xde\xd8\x86\x2f\x57\x90\x44\x71\x9e\x10\x16\xb2\x8d\xe1\xf9\xbd\x36\x6a\xa5\x60\x2a\x43\x71\x3c\x6b\xb3\x9c\xcf\xcc\xa2\x82\xff\xca\x5b\x31\xf8\xc1\x9f\xcf\xa4\x5d\xe4\x98\xa9\xb4\x3d\x92\xbf\xf1\x9d\xae\x24\xe7\xb0\x77\xf0\x03\x2d\x90\x95\x79\x17\x66\xe8\x07\xb4\xd1\xe2\xda\x58\xa6\x10\x17\xf2\xd7\xbb\x1b\x96\x31\x14\xf2\x97\x6c\x2d\x3b\xde\x9a\xdb\x27\x7a\x65\x36\x50\x0b\xd5\x43\xd8\x31\x78\x59\xaa\x83\xd6\xae\x5c\x4c\x41\x7e\x6d\x30\x7a\x68\x90\xfd\x33\x3a\x0a\xce\x53\x0c\x76\x86\x06\xed\xa5\xbc\xc7\x47\x08\x2d\xd6\xbf\xde\xc0\xc9\x98\x48\x8e\xfc\xd8\x2c\x10\x68\x06\x76\xcd\xd6\x8c\xe3\xc1\x84\x6a\xfc\xa5\xf0\x3a\xd1\x57\x10\x2d\x34\xee\x65\xe6\x6f\xde\x3f\x9d\xa5\xd7\x78\xeb\x4b\x47\x27\xfa\xe3\xdb\x8c\x34\xc7\x47\x25\x2a\xf4\x82\xbe\x8e\x72\x79\x71\xb0\x13\x81\x24\xed\x54\xcc\x6a\x9d\x48\xb9\x97\x29\x5a\x33\x9b\x40\x02\xee\xac\x16\x53\x89\xaf\x64\x6f\xc3\x4f\x07\xca\xa2\x75\xfc\xc0\xda\x46\x61\x51\x11\xd8\x2c\x51\x3b\x0b\xde\x26\xf2\xbc\x67\xad\x9d\x47\xef\x49\x66\x3b\x22\x52\xd6\x89\xf9\x02\x2d\x4b\xbd\x57\x5c\xbe\x79\xff\xc8\xc5\x5b\x6f\x6d\xdb\xe6\xff\x01\xf8\x6c\x1b\x38\xc9\x1e\x68\x28\xbf\x2b\x30\xbf\xd6\x53\x74\xed\x39\xbf\x79\xf5\xea\xfd\x7f\x3c\x7b\x18\xb9\x28\x13\xf5\xc0\x92\xa5\xd8\x58\x80\x62\xe1\x47\xde\xea\xd5\xf9\xf3\x57\x71\xe1\x13\xa0\x73\xcb\xa3\x47\x23\x5d\xb2\x4a\x46\x14\xe5\xd4\x2d\x94\x1f\x1b\xe1\xc9\x58\x58\x62\xfe\x51\x6c\x96\x6f\xca\x45\xf5\x9b\x8f\x74\x02\xd0\x43\x80\x17\x88\x03\x1f\xd2\xcc\x46\x08\x39\x94\x71\xc1\xa9\xcb\xfa\x62\x8a\x5e\x3e\x67\x67\x9f\x75\x61\xbb\xd5\xba\x75\x5c\x17\xc7\x9c\xd1\xc0\x93\x31\xfc\xf3\x79\xd2\x83\xdf\xbd\xe4\xf3\x09\x05\x97\x67\x88\x02\xd3\x30\x4e\x1a\x57\x93\x02\x0b\x6e\x96\x8a\x91\xd4\x66\x64\x43\x0f\x77\xeb\xba\x3a\x9f\x03\x6f\x89\xb0\xee\x48\xc2\xd7\xd5\xb4\x21\x5a\x40\x60\xa7\x20\x03\x1b\x1f\x9e\xcc\x5c\xbe\x6f\x21\x70\x98\x7e\x1c\x50\xf8\x8a\xd6\xfa\xbd\xb6\xf0\x90\xf3\x9e\xb5\x51\xb8\xec\x20\x40\x41\x4f\xd6\x6b\x8c\x79\x61\x37\x10\x40\xb6\x85\xb7\x11\x62\x94\x26\x05\x4e\xce\x6a\x02\x4c\x4d\xdb\xc1\xbb\x49\xc2\x87\x2c\x4c\xf2\xd9\x20\xc9\xda\x77\x1b\xb7\x39\x09\x16\x1f\xa1\x02\x27\x9d\x9a\x73\xd9\xe8\x6f\xd6\xc5\x79\xb7\x0b\x07\xc5\xea\xf2\x12\xad\x64\x81\xe3\xfc\xf3\xbc\x5f\x2f\x6f\xd6\x17\xe5\x53\x8e\x79\xfd\xc3\xc3\xbf\xbf\xfc\xf1\xc9\xbf\x3f\x7c\xfe\xf4\xaf\x4f\xb2\xe3\xf4\x9f\xd3\x03\xd9\x64\xdd\x0b\x4d\xe9\x82\xc4\xdf\xe2\x59\x67\x4d\x76\xd9\x18\x17\x40\x7d\xc6\x20\x91\x2b\x32\xa6\x17\xc5\xb9\xd4\x3b\x1e\xd8\x58\x8b\xa3\x41\x7e\x3c\x88\x65\x15\x8a\x3a\xf7\x73\x5d\xde\xcc\x96\xe1\x13\x67\x2d\xb1\x7a\xe8\x8a\x3c\x50\x21\x86\x2b\x47\x9f\xe0\xe3\xcb\x09\x09\xf6\x25\x0c\x93\xee\x69\xf8\x8c\xb0\x8b\x61\xf8\x86\x6e\x2b\xfe\xb5\x40\xa4\x21\x14\xf8\x14\x05\xdf\x89\x48\x42\xf6\x11\x70\xe9\x53\xf0\xd3\xb9\xc0\xe6\x49\x4b\xa1\x14\xa5\x3f\x54\xe4\xb6\xc5\x34\xa1\xe8\xe9\x2f\xd3\x5e\x13\x69\x4c\x6d\x88\x5d\x3c\x78\xb1\x19\x27\xfa\x28\xeb\x02\x64\x29\x74\xcc\x80\xa2\x1a\x9d\xdd\x76\x08\x0c\xc2\x2f\x1a\xb0\x09\xac\x30\x7e\x1d\xe4\x03\x6b\x98\xa0\x97\x2f\xe5\x80\xbc\x67\x4d\xa5\xe1\x43\x12\x33\x7e\x92\xd1\xe2\xa4\xc1\x0a\x0d\x85\xe5\xc4\x20\xb5\x9e\x28\xa6\xa3\x30\x54\x5c\x7c\xfa\xeb\xe5\xb7\xd3\xb5\x8a\x64\x20\x48\x7d\x4b\xd6\x7e\x35\x4b\xa9\x33\x66\xb2\x24\x34\x8f\x7d\xcb\x6e\x76\x24\xa5\x22\x99\x8b\x07\xc5\x3d\x8e\x4e\x72\xce\x11\xeb\xe5\x13\xd6\x37\x51\x47\xee\xc8\x29\x4c\x1c\x70\xe6\x3a\x2d\x48\xa8\xb1\xdf\x43\xaa\x32\xcc\xee\xd2\x8e\xc0\xbf\xdd\xaa\x0c\x74\xb1\xbc\xbe\x9e\x8a\xfc\x83\xa5\x32\xba\xc1\xbc\x66\xaf\x01\xed\x0b\xff\x15\x20\x78\xf3\x3d\x99\xa8\x4d\xf8\x76\xcb\xd5\x04\x0e\xf1\xa2\x23\xbc\x00\x4c\x99\xda\x8c\xe3\x6d\x81\xf1\xfa\x84\x33\xe7\x14\xa0\xaf\x83\x9e\xad\x35\xfb\x8c\xe7\xd1\x3a\xab\x72\x7b\x62\xe6\xcd\x71\xf3\x74\x22\xc8\x52\xc8\x02\x15\x71\xba\x23\x7c\xf3\xc4\xec\x19\x84\x25\xb0\x2c\xd4\xe3\x11\xee\xd8\x56\x33\x13\x9a\x61\x70\x30\x83\xb5\x06\x01\xd4\xdb\xad\x43\xfc\x58\x1c\xef\xb4\x5e\x9c\xc3\xc9\x58\xe7\x77\x8e\x94\x48\x4d\xe8\x98\x73\x3e\x6e\x80\x28\x0e\x4b\x62\x76\x0e\x0b\xff\xcd\xae\x41\x6d\x5c\x5f\x42\x2c\xd3\xc7\x88\x8f\x28\xfd\x12\xf7\x22\xb7\x97\xe6\x8e\x96\xd3\x88\x0c\x33\x4b\xaa\xfe\x9a\xb2\x5b\x36\x8f\xce\x2d\x7d\xfb\x9c\x16\x8f\x5a\xd6\xbb\x55\xa2\x24\x89\xf7\xd8\x94\x5d\x35\x52\x1a\x5f\x93\x09\x82\x56\x6b\x01\xef\x48\x37\x9b\x3d\xb2\xa4\x26\x38\xc1\x73\x96\x26\x99\x73\xe0\xa6\x7a\xdc\x83\x8d\xc6\x66\xf4\xe1\xd3\xd0\xe5\xc5\x8f\xcb\x85\x0a\x7f\x05\x96\x06\x19\xcb\x09\x10\xc8\x67\xb1\xe4\x6c\xb9\x20\x26\x5d\xe0\x8f\x5e\xcf\x9e\x80\x5d\x7b\x97\xd5\xba\xde\x44\xc6\x5f\xba\x03\xbf\xbb\x9e\x33\x46\x38\x30\x09\x59\x56\x41\xab\x93\xa1\x7a\xa3\x94\xbe\xd3\x38\x08\x01\x11\x5c\x99\xe5\xd6\x34\xbe\x66\xe3\xb1\x59\x58\xd0\x31\x0d\x1e\xd5\x07\x7c\x88\x5f\x82\x38\xb1\xa6\xa9\x85\x3d\x1e\x9d\x06\x70\x80\xeb\x9b\x45\x8d\xcc\xaa\xa7\x03\x45\x72\x3a\x08\xf6\x47\xc1\xad\x1e\x24\x2d\x20\xdb\xe7\x7e\x5c\x44\x84\xdf\x8b\x1d\x94\xb6\xed\xa0\xb8\xbe\xaa\x00\x24\xd6\x8e\x53\x09\x50\x64\x2c\x56\x5c\xb8\x3a\xd4\x1d\x0a\x90\xf4\x71\x32\xe1\x49\xa7\x97\xb7\x71\xa4\xa4\x33\xe7\x3c\x38\x2b\x16\xec\x1e\x24\x0f\xe7\x32\xfb\x46\xf4\x59\x09\x87\x33\x74\x60\xca\xfc\xe3\xee\x8a\x31\x77\xa1\x4a\x8e\xc1\xa4\xd8\x8b\x2b\x18\xb3\xed\xc6\x15\xc7\x3e\x1c\x16\xdd\x9a\x28\x19\x2f\x3f\x48\x6b\xd2\x6a\xed\xed\xf9\x9c\x60\x9f\x72\xfc\x10\xd5\xf2\x3a\x8e\xaa\xf1\x9e\xec\x98\x0e\x39\x38\xed\x9f\x66\x95\xdf\x00\xd2\xdc\x56\x38\x99\x78\xb2\xca\x12\xb6\xf4\x05\x88\x3e\x37\x0b\xb7\x03\xaf\xc4\xa2\xbe\x1d\x26\xea\x0c\xe4\xec\xff\xd4\x1a\xe2\x90\x45\x0c\xc7\x36\xa3\x3c\x87\x43\xad\x88\x9b\x01\x7c\x78\x8c\x87\xcf\xd4\xb6\x06\x1e\x38\xd0\xed\x8c\xfb\xde\x6e\xd5\x9f\xa3\x31\x36\x8c\xcb\xe0\x75\xef\x1f\x9c\xb9\x37\xa4\xff\x15\x27\x57\xce\x7f\xcf\xaf\xe3\xc9\xe3\xa5\x9d\x32\xbb\xa7\xf8\x91\x89\x98\xb5\xa1\x19\x89\xe4\xf1\x4f\xc0\x1a\xff\xb4\xa7\x0d\x7b\x76\xb1\x45\x61\xb7\x72\x1f\x76\x2d\x60\x3c\x4a\x9b\x5a\xf8\xdd\xdb\x54\xd8\xdc\x5d\x2f\xb6\x15\xfc\x68\xb6\x82\xa9\xcd\x56\x9a\xe0\x65\x87\x44\x47\x3d\xab\x34\x18\xd8\xa6\xe4\x80\x21\x38\x07\x80\x9e\x1f\xa6\x2b\x92\x87\xa0\x61\xfd\xb9\x2e\xcb\x77\x55\xbd\x81\xf3\x83\xea\xa4\x74\xe6\xb1\x52\x4d\xf2\xe0\xd6\x3e\x3e\x2b\xe7\xb6\x3a\xf5\xd5\xc4\x89\x89\x95\x4f\xa3\xb1\xa4\x4c\xf4\xbc\x8b\xcf\x23\xeb\x6c\x3c\x5d\x14\x1d\x3b\x57\x18\x9a\xa1\xc4\x29\xb7\xb7\x71\xd4\x6f\x6e\x09\x9b\x3e\x05\x78\x27\xc9\x05\x43\xd5\x86\x7f\xba\x51\x37\xdc\xf5\xc8\xe1\x11\x98\x19\xf6\x30\xb2\x75\x73\x6f\xd0\x20\xa3\xb8\x69\x1b\x8d\x27\xb9\x20\x21\xb7\xb0\x0e\x55\x7f\x83\x53\x79\x1d\x72\x87\xa7\x0b\xaf\xeb\x10\x24\xaf\x59\x92\xa1\x4a\x0c\xbe\xa0\x63\xf2\x2a\xc9\x5c\xf8\x15\xa8\x30\x20\x2c\x6c\xb0\x2a\x30\x39\x60\x23\xf2\xc8\x24\x24\xcb\xda\x73\xed\x8e\xa5\x5d\x14\xf5\xad\x80\x0f\xf0\x06\xe9\xb4\xe6\x48\x04\xb2\x0a\x40\x1f\x6d\x16\x61\x51\x8e\x5c\x28\xe1\x88\x88\x73\xb8\x5e\x5b\xa0\xf3\x40\x22\x18\x69\x10\xb4\xb4\x1d\x4c\x95\x80\x32\x74\x7d\xd0\x9d\xaf\x3f\xaf\x18\x7a\x0a\xd1\x42\xfb\x36\x47\x98\xf8\xa4\x7e\x19\x5e\xd7\xc1\x48\x02\xb9\xb0\x4a\x82\x56\x11\xc1\x90\x33\xa5\x61\xa5\xd3\xe3\x81\x9e\x92\xb0\x32\xe2\x51\xa8\x43\xbe\x19\x7f\x3b\xde\xb1\x3c\x58\x5b\xe6\xb9\x28\x74\x79\x8d\x3c\xb0\x25\xe6\x93\x47\x8f\xa6\x31\xa8\x2c\x77\x49\x3e\x5f\x19\xc5\xc8\x52\x26\xe0\x11\x17\x22\x2b\x6f\x51\xc6\x78\x2d\xfa\x87\x3c\x49\xfe\x76\xbd\xbc\x7e\x4e\xa2\x64\x43\x2e\x25\x21\x42\x75\xe9\xca\x5c\x5f\xb7\x88\xaa\xf3\x72\x8a\xe1\x72\x7f\x41\xe3\xcf\xea\x6d\xc9\xe7\xae\xb5\x7c\x8d\x45\x50\x85\x73\x09\x5f\xfa\x57\xd7\x88\x88\x6a\x63\x8b\x47\xd5\xb7\xdb\xb0\x3e\x9e\x0f\x81\xf0\xa2\x52\xa3\x41\x7e\xc2\x40\xa9\xce\x2b\x94\x73\xf7\x07\x56\x46\x89\xc4\x1b\xd9\xae\x01\x10\x5e\x1f\x71\xd3\x0f\x17\xef\x3f\xa5\xf5\x80\xca\x02\xec\x51\x31\x20\xb1\x43\x9d\xe1\x89\x75\x7c\x40\xcc\x10\xe1\x20\xc2\x03\x10\x2d\xe4\x5a\xa5\x18\xbe\x9d\x86\x42\xd4\x35\xd0\x8b\x74\x8e\xc7\xac\x00\x1a\x55\x69\x66\x8e\x85\xfb\x88\x6c\x76\x9d\x47\xc8\x68\xd9\xe6\x77\xd1\x8b\xb8\x32\xa2\x76\x52\xa8\x84\x14\x32\x37\xee\x10\xbd\xfb\xc4\x33\x45\x54\x36\x19\x06\xda\x76\xed\x88\xa5\x90\xb8\x1b\x59\x0f\xb4\x6e\x5c\x96\x6a\x60\xb4\x4d\xbb\x5b\xeb\x43\xbb\xbd\x5e\x45\x63\x7a\x0d\x9b\x7a\xeb\x30\x5e\x4b\xfb\xc3\xd7\x30\x66\x77\xfe\xb2\x60\x57\xa7\x03\x56\x3d\xb5\x0e\x85\x72\x1b\xa7\x42\x3d\x39\x9a\xea\x78\x90\x65\xad\x3a\x4b\xbb\x12\x4f\x2b\xbb\x47\x35\x56\xb1\xb4\x53\x99\xd7\x20\x4c\xc8\x28\xf6\x95\x2d\x82\x2a\x5e\x85\xe6\x99\x35\xdb\x59\x11\xd7\x67\xeb\x2e\x30\xca\x9e\xd1\xb6\xb2\x9d\x7f\xc7\xc3\xb2\x14\xad\xa3\x39\x47\x55\xac\xfd\xed\x2b\x9d\xf8\x78\x30\xfb\x05\xc4\x7c\x90\x62\xcf\xdf\x3f\x2b\x37\x9e\x44\x65\x05\x76\xcc\x92\xc3\x4f\x11\xb5\xec\x8a\xd4\x37\xab\x72\x2d\xc9\x45\x28\x64\x79\xc4\x6d\x79\x2d\x13\x88\xae\x2a\x7b\x4b\x6d\x5e\xdb\x29\xb1\x80\x90\x98\x2d\xe2\xab\xc6\x46\xad\x8a\xe4\x24\x31\x56\x82\x2e\x58\x12\x82\x72\x76\xc1\x8a\xe0\x1d\xf4\xa3\x90\x7c\x33\xbd\x78\x73\xb3\x6a\x2c\xee\x40\x70\xb3\x43\xee\x76\x65\x39\x2c\x66\xfd\xe7\x0f\xff\x3d\x45\xb1\x2d\x68\xf5\xe0\x45\x40\x66\xe4\xb4\xfb\x9f\x8b\xea\xd7\x9b\xd2\x9e\x7d\x25\x7c\x57\x00\x10\xb0\xe1\x1f\x30\x6c\xd4\x1a\x97\xce\x35\xc6\x1d\xea\x0f\x48\x8f\x64\x01\x45\x63\xa7\x08\xe8\xc2\x31\x2b\x77\xb7\xaa\x18\x77\xc7\xb8\xa2\x31\xe3\xd9\x90\x31\x94\xda\x53\x30\xdf\x7c\x20\xd3\x41\x66\xe0\x21\x80\x19\xb3\x3e\x99\xad\x5c\x22\x22\x05\x79\x3c\xdb\xb1\x91\x16\xce\x76\xd7\xbc\xf2\x1b\xb2\xee\xc8\x1f\xa2\xe0\xa8\x08\x30\xe7\x8f\xa5\xd7\x6b\x8e\x66\x47\xe4\x0b\xb5\x79\x3c\x1d\x2b\x79\xca\xfd\xa3\x4f\x40\x18\xfb\x91\x08\xe6\xa6\xe5\x28\x45\xd7\x33\x7e\xe9\x5e\x51\x19\x69\x1c\x1f\xe2\xf4\xb3\x9a\x48\xf1\x96\x8f\x20\x44\xb1\xe1\xce\x99\x8e\xb2\x03\xec\x04\xe2\xa4\xdf\xcb\xe9\x09\x5f\x5d\x90\x90\xe6\x9f\x1a\xd1\x94\x50\x9b\xa2\xcf\x34\xf8\x06\xa1\x4d\x02\x05\x0b\xda\x41\x64\xf5\x56\x95\x15\xb6\x5c\x5a\xb6\x6b\xbb\x80\x72\xd9\xc6\x9b\x26\x7b\xfa\xd0\x5c\xcb\x0b\x42\x64\xf7\x5a\x38\x0a\x70\x6b\x51\x04\xde\xd0\xe4\x3e\xc3\x78\x81\x0a\xa4\xf2\xbf\x0f\x93\xc4\xbe\xb5\xc0\xcb\x29\x73\xfc\x61\xa7\x3b\x0a\xcd\xe3\x28\x38\x10\xfa\x8c\x2b\xcb\xfd\x2f\x8f\xd1\xee\x3b\xc5\xe9\x04\xd6\xca\xd8\x9a\x9a\x48\x2d\x61\x5f\xb7\xb6\x5b\xfb\x1e\x73\x0d\x8a\xdc\xa2\xcd\x0d\xc3\xbb\x72\xff\xf8\x1e\xdc\xc8\x1d\xb0\xf5\xa8\xec\xa5\x02\xfe\xe1\xab\x05\xe4\x66\xcc\x53\x5b\xee\x6a\x49\xb5\xcb\xb3\x8b\xea\x30\x56\x27\x8b\x4c\x4d\xbe\xcf\xbc\xd3\xa9\x8a\xdc\xfe\xb0\x9a\x5a\x13\x88\xb4\xa7\x0f\xba\xdd\xe4\xe9\x63\x54\x6d\x72\xd5\xc2\x4a\xb1\x99\x98\x75\xff\xa5\x28\x1a\x76\x18\x40\x61\x91\xb5\x45\xab\xe0\x30\x50\xf9\xe7\xce\xd9\x30\x14\x8e\x63\x3f\x7d\xcc\x7d\xaa\x88\x12\xa8\xf4\x6f\x16\x72\xc7\x7c\xa9\xbf\xec\x2d\x66\x06\x05\x4d\x47\x3f\x22\x1b\x0b\xab\x4e\xb6\xf7\xa1\x3c\x72\xdd\xfb\x59\xf3\xce\xba\x64\x45\x9c\x7f\xf4\x22\xf0\x0e\xc4\x1c\x1f\x9d\xe4\x01\x06\x71\xe5\xe3\x41\xcc\x47\x1e\x3e\x15\x1c\x21\x84\x42\xdb\xd1\x60\xc5\x10\x91\xf4\xa6\x88\x06\x87\x10\xb1\x51\x94\xe3\x59\x41\x01\x7a\x3e\x0d\x41\xe1\x75\x78\x78\x20\x39\x7c\x29\x2e\xcf\x03\x2b\x96\x56\x22\x55\x0d\x5c\xd0\xee\xc2\xbb\xa4\x6b\x13\x99\x9c\xc5\xcf\xc7\x19\x5c\x90\xf8\xe1\x9b\x7b\xf0\x66\xb3\x73\xea\xf4\x55\xe5\xaf\x40\xbe\x4b\x09\xa5\x96\x80\x08\xed\xf2\xda\x63\x14\xe0\xae\xdd\x7d\x3d\x9b\x5c\xd8\xa4\xd9\xdd\x8e\x17\xa4\xa7\x5e\xc5\x99\xc1\x97\xb6\xf0\x65\x19\x43\x6f\x28\x1a\x01\x0b\x1f\xd6\x7f\x7c\x83\x58\x9a\x6e\x9a\xbc\xce\x78\xf6\x53\xc6\x8f\xa0\x81\xbb\x76\x0c\xb4\x1a\x8e\xc8\x2f\xe6\x6f\xb5\x8d\xac\xa5\x36\x3b\x06\xe3\xb6\xfe\x84\x6f\x50\x8a\x84\xb3\x2c\x92\x9a\x98\x54\xd2\x3b\x46\x2f\xdf\xc7\x08\x89\x6f\xd2\x55\xb0\x19\xa0\x0d\xd6\x61\xc4\x62\xa5\xb8\xdb\xa9\xe0\x03\x6c\x10\x95\xfc\xf0\x87\x5e\xde\x64\xa5\x79\x60\xf1\x67\xdc\xed\x4f\x94\x21\x98\x58\xb7\xe5\xf9\xee\xe3\x6a\xf4\xe5\x82\x0e\xfa\x46\x01\xc5\xe0\xeb\xaf\xbf\x18\x98\xe9\x7a\xcd\x3a\xbc\x55\x01\x3f\xfb\xf0\xd7\x20\xf5\xf0\x07\xfc\x60\xc2\xa1\x4f\xfa\x65\xe4\x08\x4a\x29\xf2\x7b\xbb\xdd\xf3\x04\xb9\xbd\x77\x42\x73\xce\x06\xa7\x97\x80\x11\xb8\x27\xf3\xf5\x82\x92\x6f\x35\x54\x7b\x83\x9d\xa1\x80\xe9\x1c\x97\x36\x19\x9f\x9d\xbd\x7b\x70\x72\x76\xb6\x39\x3b\x5b\x9f\x9d\x2d\xce\xce\x2e\x27\x89\xc1\x80\x34\xd3\x0b\x20\x83\x27\xf2\xa6\x4e\x91\xe0\xab\x28\x67\x18\x55\x16\x2a\xdc\x1e\x4f\xb6\xe3\x17\x50\xf1\xe4\xe4\x18\xfe\x9d\x02\x7b\xed\x25\xa6\x82\x3d\x7e\x83\x4f\xa1\xad\x8b\x46\x7d\xbb\xd0\x93\x5b\x7c\xa6\xea\x4f\x18\x58\x1c\xb6\x3f\xba\x46\x85\xb6\xc7\xf7\x5e\x7c\xb6\xed\xfc\x73\x32\x2a\xb2\xc4\xb8\x87\xdc\x8b\xe4\xec\x6c\x9c\xf4\x1c\xb8\xbd\xe4\x5e\x9a\xf4\x1a\xad\xf7\x92\x2c\x2e\x35\xca\x93\x9e\xed\xa1\x91\x97\x8e\x3f\x3f\x4b\x26\x59\xea\x06\xf5\x02\xff\x4e\xb2\x7b\xa3\xec\xec\xec\x8b\x2d\x74\xe2\x46\x03\xad\x6f\xe1\xff\x51\x0f\x50\x3a\x31\x2b\x32\x3b\x00\x38\xf3\x3d\x60\x51\x07\x69\xfa\xc1\xfe\xa2\x8c\x34\x03\x2c\x4f\x26\xdb\xa4\xe7\xbd\x6a\xaf\x28\xfc\xc2\xfc\x1b\xa0\x3b\xbb\x97\x6d\xfb\xf7\xa0\x36\x82\x66\xe8\xa4\x59\xf8\xc1\x82\x5f\x84\xf0\xf6\xa8\x8f\x17\xb6\x63\xed\x10\x9a\x89\x0a\x7e\x06\x13\x84\x6f\x28\xf1\xed\xfa\xa1\x36\xef\x99\xe8\x5b\x6a\xe9\xfd\xf8\xc1\xba\x69\x93\xf0\x4e\x7b\xff\x9c\xc4\x68\xc6\x26\x19\xcb\x7e\x6b\x82\x77\xc8\xf3\x88\x2e\xea\xcd\x9f\xc0\xcf\xf4\x00\x8e\x3c\xb0\xb8\x7b\xfa\x38\x0f\x0a\xff\x69\x1f\x51\x65\xe6\xd1\xf7\x0f\x9f\x3d\x0b\x4b\x63\xb0\xe5\xbd\xe5\x7f\x7c\xf8\xc3\x93\xb8\x38\xc5\x9a\x28\x88\x04\x46\x7b\x6a\x72\x26\xd2\x54\x66\xe0\xe8\x17\xb6\xd0\x56\x27\x5a\x50\xf7\x30\xe8\x33\xf6\xff\xf0\xf9\xf3\x5f\xf2\x08\x13\x8e\x86\x32\xf3\xf3\xb3\x27\xff\xf9\xf8\xa7\xb8\x84\xc5\xe7\xa3\xef\x9e\x7e\x1f\x21\x27\xa7\xe8\xb5\x5b\xba\x05\xde\x02\x5f\xdd\x6c\x17\x9b\x2b\xfc\xef\x18\x3f\xb2\xe3\xf4\xe2\xaa\x9a\xcf\xb6\xcb\xcb\x63\xda\x97\x99\xe2\xe3\xc9\x46\xbb\xef\xed\x72\x36\x03\x2a\x1c\xf7\x80\x7b\x64\xe9\xd9\xd9\xec\x5e\xb6\x68\x2c\x2b\x5a\x9c\x54\x22\xce\x80\x0a\xf8\x24\x53\x73\x1d\xd2\x02\x48\x2a\x18\x7c\xf0\x0a\xcb\x21\xe2\x1b\x23\xa9\x6d\x73\x07\x56\xf9\xeb\xf6\x15\x8c\x8e\xc7\xe6\x86\xba\x67\x34\x90\x0a\x7c\x6f\x96\x8d\x68\x10\x6d\x00\xa5\xa3\x62\xfc\x02\x46\xf1\x99\x80\xb6\xb3\xdb\x78\x71\x7f\x4c\x74\xbf\x01\xaa\x5f\x00\xb3\xbd\x37\x06\x50\xee\x1b\xb1\x5a\x2f\xee\xbf\x18\xbf\xb8\x9b\xf4\xce\xee\xce\xea\x7b\x48\x38\x98\x08\xd2\xc3\xac\x84\x22\xd6\xe4\x97\x1f\xb7\xfa\x13\x2c\x21\xe0\xc4\x80\x94\xf4\xec\x16\xfe\x05\xb2\x94\x04\x7a\xdd\x8a\xcc\xab\x0e\x3d\xb7\x25\xd1\x95\xf8\xd5\xad\xab\x72\x8a\xb7\x5c\xf7\x5f\x5c\x9d\xcd\x28\x81\x85\xb0\xe2\xfe\xe7\xdb\xb3\xb3\xfb\xaf\xcc\xda\x12\xd1\x7f\xdc\x2c\x91\x35\xdf\x3f\x2b\xa2\x81\xa4\xe3\x17\x9f\x27\xc0\xb5\xee\x65\x51\xc6\xd9\x04\x1b\x50\xb9\x0e\x6a\x9e\x01\xa0\xb3\xe9\xf1\xe5\xc3\xe3\x6f\x27\x77\x03\xf3\xe7\x5d\x58\x61\xb4\xed\x67\x50\xc3\x4a\x82\x4e\xc9\xf3\xd2\x70\x8a\x9c\x37\xae\xaa\x57\x57\x45\x72\xf2\x2e\xe9\x49\xf2\xf1\x9f\xbf\xfa\xea\x8b\x3f\xeb\x49\x06\xb3\xf1\xed\x07\xf8\x33\x92\x02\xf9\xc9\x29\x7d\x3e\x23\x1b\x6e\x8a\x39\xf6\x48\x82\xaf\xa5\x98\xd1\xa3\x06\xb2\x7c\x5f\xfe\xe9\xe9\xe0\x64\xfb\xd5\x57\x0f\xfe\xf2\x67\x33\x38\x79\xf0\x45\x17\xd3\xb6\x5f\xfd\xf9\x8b\x07\xe8\xb7\x88\xd6\x95\x9e\x48\xe8\x4b\x2f\x71\x8c\xb1\x3e\x2d\x1a\x94\x59\x6b\x90\x1a\x51\xa8\xd5\xc3\x88\xb3\xc3\x67\x21\xc1\xbd\x7f\x18\x5d\x0f\xab\x99\xfa\x78\xc2\xd6\x05\xbc\xed\x93\x22\xa0\x45\xf7\x11\x1d\xee\x76\x24\xff\x15\x72\x34\xe6\x8f\x16\x5b\xbe\x08\x6a\x12\x28\xf4\xf1\xf4\xc8\x0c\x9f\x2a\xc5\xc3\xb4\xaf\x0f\x87\xc9\xa3\xe4\xbb\xe7\x3f\x7c\x9f\x78\x8a\x24\x45\x8b\x9a\x0c\xe6\x9d\xc1\xce\x97\x3b\x15\x52\x3f\xc9\xc2\x8b\x95\x2c\xbc\x64\x51\x3c\xc2\x7f\x62\x08\x31\x2d\xf0\x12\xf0\xa0\x0b\x74\x5a\x7f\xe1\x28\x57\xde\xf9\x70\xd6\x9c\xc1\x51\xaa\x09\x58\x56\xc5\xe1\xa2\xa5\x64\x28\x22\x17\x84\x6c\xac\x9c\xa9\x08\xdb\xdf\xb0\xe1\xf8\x8f\xcb\x47\xb0\xe5\x92\xb9\x8e\x3a\x74\xe8\x10\xd1\x43\xe3\xce\x1a\x9a\xbe\xc5\x73\x48\xb9\x98\x3d\x42\x22\x4a\x29\x9a\x1b\x0b\xac\x5c\x3d\x4d\x12\x74\x56\xc7\x72\xad\xf6\xe9\xb8\xab\x8a\xe4\xb8\x73\x40\x78\x62\x57\x6b\xef\xd8\x5c\xb5\x00\x8c\xe2\xdc\x15\xc9\xd7\xcc\x4d\x4e\xbf\xbe\x2f\x3f\x92\xa1\xfa\xaf\xaa\x8c\x8c\x35\x90\x8b\x12\x98\x91\x25\xb6\xea\x39\xec\x53\xe3\xf6\x21\xed\x8e\x86\x2c\x77\x8e\x16\x6c\xfd\xe9\x40\x0d\x2d\xe6\x3f\x88\x2c\x1f\x6a\xf8\x3e\xa2\xd3\x41\xf1\xf9\x55\x35\x03\x11\xe1\xa8\xfc\x1c\xc6\x00\xc9\xa7\xcd\x3c\xcd\x49\x4c\x03\x97\x9e\xbd\xfe\xfe\xbc\x34\x29\x2d\xa6\x47\x69\x88\x8e\x0b\x0b\x3e\x14\x32\x0f\xc8\x30\xe7\x63\xda\xc9\x70\x69\x44\xa8\xd8\x8f\x05\x1a\xbe\xb5\xd3\xef\x9d\x98\x18\x1f\x53\xb6\xd7\xfd\x1c\x58\xa8\x94\x49\x70\xd4\x53\xc6\x46\x6b\x9e\x60\x84\x09\x1e\x1a\xc3\x6e\xbf\xa1\xe8\x07\xd8\xa7\xcd\xa0\x5d\x94\xc6\xca\x31\xd3\x57\x88\x58\xa4\xd5\x60\x8c\x82\xc2\xb6\xe4\x54\x7a\xd5\x91\x03\x8a\x1e\xf4\x0e\x96\xec\x9d\x68\x59\x7b\xaf\xe3\xd0\xf4\x14\xd8\xed\x86\x30\xd5\x09\x1b\x21\x8f\x12\xed\xcb\x82\xcf\x66\xea\xb2\xc2\x00\x95\x06\xe1\x07\xc4\x93\x7e\x83\x1e\x85\xe5\x57\x06\x3e\x85\xf8\xa6\x47\xf8\xb4\x6d\xf1\xf9\x9f\x18\xc5\x4c\x56\x0e\x4f\xec\xef\x2a\x0b\xc7\x25\x07\x2b\x07\xdf\x2b\x0a\xa2\x99\x25\x7f\x4a\x84\x76\xf6\xd4\x48\xf9\x3d\xdd\x6c\x97\x8d\xee\x76\xf9\x1d\x7e\x7c\xf4\x3b\xb9\xf2\x14\xef\x83\x8c\x9f\xaa\xfa\xf8\x7a\x58\x1a\xad\xfa\x4d\xdb\x04\x8c\x52\x5f\x6b\xe6\xed\x70\x56\x15\xc2\xda\x1b\x46\x46\xbb\x9f\x52\x03\x0f\xa1\x0a\x4f\xee\x9e\xf6\x39\x39\x55\x33\xbb\x23\xc2\x7e\xe6\x3b\x39\x8d\xc6\xd7\x93\x7c\x3c\x01\xc8\xbd\xab\xc9\x08\x4a\x1b\x0c\x61\xfd\x74\x56\x54\xb3\x83\x4a\xac\x61\xbb\xa1\xfc\x7e\xcc\xa1\x37\x04\xc7\x34\x80\xd6\x77\xbb\x2c\xff\xff\x14\xb2\x46\xe2\x59\x45\xaf\x71\x71\x97\xcd\x77\xcb\x1b\xdd\xb5\x3c\x6d\x4e\xe3\xec\xeb\x33\xe3\xd5\x8c\xf0\xee\xde\xd0\xfc\x9f\x3f\x03\xe4\xaa\x00\x90\x14\x5e\xd4\xc0\x0f\x8f\xa4\xb5\x14\x0f\xc6\xda\xc0\x51\xcc\x22\x12\x40\x74\x78\x76\x32\x4d\x70\xc1\x57\xec\xdd\xff\x9d\xdd\x22\x64\xb9\x79\x0e\x1f\x2c\x3d\xe4\x83\x16\x01\x3e\x3a\xe8\xaf\x06\x7d\x58\xc7\xdb\xbc\xbd\x67\x2b\x75\x6e\xd4\xa2\xe8\xc4\x0a\xa0\x1f\x6c\x9b\xfc\xde\xef\x51\xe4\x21\xf8\xf2\xec\x6e\xd5\x54\x87\xa5\xd6\x86\x41\x2c\xf0\x43\x35\x44\x0a\x84\x58\x48\xdd\x45\xf2\xac\x87\x58\x3c\x81\x17\x8d\xed\xb1\xdb\xfd\xdd\x28\xfd\x68\x7c\xd2\x80\xc9\x77\xc5\x06\x7e\x71\x60\x91\x22\xa1\xd8\xe7\xf3\x67\x81\xb3\x62\xc1\x27\x81\xe8\x64\xa1\x10\x4e\xe7\xd5\x4a\x0b\x7d\xc4\x60\xe5\x1f\xf0\x0e\xb4\x30\xe0\xb1\xd5\x57\xe3\xe2\xcc\x5b\x0d\x6f\x31\x4e\xf8\x7d\xe5\x64\x62\x7c\xf7\xc1\xc2\x3a\x65\xe3\x36\x1b\x7b\x8f\x91\x69\xe9\x27\xc9\x97\xcb\x95\x77\x0b\x55\xce\x8a\xcf\x71\xf3\xe4\x44\x4f\xf8\xa4\xad\xb4\xe1\xaa\x96\x8c\xb5\xda\xc4\x0a\x50\xdb\xad\xf3\x6e\x24\xda\x6a\x53\x32\x8e\xf2\x0b\x8e\x6b\xb5\xd5\xb7\x13\xb6\x55\x7d\x3d\x5d\x59\x2b\xcb\x2d\x08\xdc\x33\x52\x8c\x68\x0f\x5b\xf6\x10\x4f\xb2\x3d\xa0\x68\x8b\x07\x00\x71\x45\x80\x55\x7c\x14\x92\xe8\x60\xcf\x42\xb7\xca\xac\x47\x15\xa0\xe8\xfe\x7e\x8c\x54\x2f\x20\xdf\xa1\xa3\xdb\x8d\xa1\x40\x65\xec\xa4\x68\x62\xe4\x2c\x39\x4b\xb6\x9f\x7f\x7e\x60\x84\xe5\x82\x50\x75\x68\x84\x5a\xc4\x24\xb9\x62\x76\x6f\x7b\xf7\x4c\xfe\x0e\xf5\x7f\x51\x1b\xa6\x7f\x2f\x4f\x28\x1e\x97\x25\x3b\xb9\x5e\xb0\x7e\x90\x96\x04\x25\xa3\x50\xa9\x2e\x2a\x48\xeb\x83\x33\x96\xbf\xfd\xb0\x2f\xef\xb6\x3c\x7f\x53\x6d\xf6\x66\x2f\xf7\xe6\x5c\xd7\x51\x16\x52\x7f\xeb\xbc\xea\x40\x00\x27\xb0\x38\x17\x44\x50\x54\x57\x8c\x00\xe4\xb5\x1c\x94\xaf\xd9\x83\xdf\x34\xd3\xc7\x75\x07\x67\xd6\xc3\x99\xf4\x2e\x78\xeb\x14\x56\x61\x8d\xf4\xe5\x16\xb2\xa7\x29\x73\xb8\x66\x1f\xd2\x2d\x9e\x24\x43\x1e\xd0\x28\xad\xbd\x78\x35\xf4\x9a\x27\xe0\x05\x84\x12\xcd\xc9\x1c\x96\xf0\xea\x05\xe4\x2f\x3d\xa8\xab\x53\x9f\xdb\x03\xad\x0b\xe7\x14\xa3\x35\x91\x37\xf1\xd4\x19\xc1\x4d\xe3\xf3\x76\x0e\xe5\x6f\x56\xe8\xeb\x79\xee\x09\x76\xd6\xcb\x12\x7d\x2f\x6f\x56\xdb\x6d\x27\xed\xd0\x5f\xf4\xd1\x87\x1f\x9e\x1b\x46\x27\x9d\xf2\xfb\x99\x02\xeb\x28\xfc\x4c\xa1\x74\x96\x4f\xf7\xc1\xdd\xed\x0e\xfe\xdc\xdd\x9b\x4b\x95\x33\xdf\xad\x85\x46\x87\x56\x30\x6c\x7a\x71\x5e\x04\x50\xe3\x85\xce\x39\x0e\xd8\xfa\x43\x0c\x6d\x7c\x06\x77\x89\x56\x7c\x3a\x2e\xa5\x64\x80\x96\x51\xea\x5f\x04\xa2\x93\xf7\x09\xc8\x9f\x52\x12\xe0\xda\x3b\xe2\x03\xc8\x38\x80\x89\x2c\x1b\x0d\xba\x92\xbb\xdd\x4e\x83\x98\x8f\x03\x0a\x90\xe2\x25\x79\x13\xce\xea\x19\xbe\xec\xa5\x09\xf1\x95\x3a\x66\x9a\x8d\x8e\x07\xf9\xf9\xe1\x42\xe7\x19\x5a\xa2\xe6\x5f\x6a\xf7\x58\x65\xb0\x7f\x4e\x29\x3b\x9e\x33\x71\x2a\x26\x59\x68\x0a\x04\xe7\x43\xcb\x14\xe8\x27\xa0\x2d\xff\x74\x62\xce\xe1\xcf\x39\x39\x4e\x4e\x3d\x27\xe0\xa3\x06\xde\xc9\x32\x62\x4a\x74\x8a\x14\xe3\xcd\x12\x8c\xca\x1b\x20\x0c\x03\x4a\x51\x02\x99\x75\x51\x45\x84\x86\x88\x5a\xeb\x35\x3c\xa2\xc9\xd8\x0b\x1d\x91\xa7\x2d\xef\x09\x0d\xb3\xe9\xaa\x7f\xb3\x60\xe3\x02\x74\x67\xb6\xa5\xcf\x5b\x4b\x9f\xb7\x94\x1e\x92\x7d\x36\x42\x81\x7f\x81\x8c\x9d\x6b\x59\x35\x0a\xc1\xc1\x02\x86\x8a\xc1\xaa\x92\x5a\xfe\x64\xd1\xe0\xda\xd2\x61\xb4\x3b\x13\x62\x6e\x60\xc6\x80\xbc\x09\x3f\x0b\x6a\x97\x87\x53\x97\xcc\xca\x0d\xf0\x56\x5b\xa1\x2e\x82\x9b\x6f\xfb\xbe\x4a\xae\xbf\x76\x46\x14\x8f\xba\x91\xb8\x53\x04\xde\xa6\x97\xea\xa8\xa5\x42\x9a\x18\xda\x50\x26\x59\xae\xda\x10\xee\x12\xeb\x2b\x6c\xce\x6e\x58\xa1\x01\x31\x56\x67\x07\xad\xbd\x9a\xd6\xbd\x01\x4e\x28\x97\x5a\x28\x4a\x36\xda\x90\xa3\x51\xa4\xc0\x37\x49\xf1\xf9\x67\x03\x14\x04\x0c\x70\xc2\x3d\x1b\x69\x24\x3f\xea\x7e\x2e\x1c\x5f\x45\x07\xdd\x00\x38\x82\x25\xc2\xee\x6f\xfc\x2e\x35\xa3\xf8\x03\x12\xf1\x38\xdc\xd8\xdc\xb0\x91\x7e\xd7\xf8\x28\xce\xde\x2d\x91\x31\xd0\x77\xc3\x1f\x0c\x34\x44\xa3\x0d\x7a\x13\xc5\x54\xc1\x0e\xbd\x37\x10\x5a\x26\xcb\x9a\x15\xf0\xd3\x51\xd8\x9a\x7d\x3b\xea\xf4\xc4\xce\x9b\xdd\xd8\x9c\xc0\xee\xdb\x8a\x0b\x19\xb4\x47\x38\x71\xd6\x29\x1f\x17\x9d\xa6\x19\xdb\xc4\x82\x81\x73\x19\xd1\x0c\x9d\x3b\x08\xb9\x20\x87\xaa\x8f\xfd\xef\x25\x9f\x68\xd6\xe9\x50\x43\x5e\xf9\x51\x24\x0e\x93\x42\x67\x45\xa4\x12\xe3\xd0\xfa\xd9\x08\xb2\xb8\xb9\x3c\x6a\xae\xa9\x71\x1e\x35\x15\x21\x7c\x8e\x4a\xa9\x83\xf6\x53\x37\x95\xc8\x84\x1a\x9a\x75\x25\x80\x80\x7d\x09\xa0\x73\x32\xc2\x9f\x39\x34\xd8\xed\xbe\xc5\x98\xd2\x18\x59\xf2\xb2\x82\xd3\x1c\x7e\xb1\xbf\x32\x85\x39\x32\xbe\xeb\xb0\x43\xf3\x75\xfd\x0a\x23\xce\xae\x97\xb7\xe8\xe7\x76\x44\x41\x2e\xd2\xe4\xd9\x7b\x98\xa9\x77\x47\x54\xd6\x1c\xdd\x2c\xd6\xe5\xc5\xf2\x15\x5a\xb9\xcd\x8e\x90\xb0\xca\xba\xc6\xb8\xe6\x47\x49\x0f\xab\xdb\xa6\x9d\x31\xa2\x6b\xdf\xda\xf5\xd9\xc3\xf6\xcc\xf1\x29\x3a\x73\x0f\xe8\xd1\xc2\xea\x32\xda\xa6\xf7\x31\x37\x3d\xa1\x37\xb8\xa1\x5f\xbd\xf5\x3c\x3e\x21\x5b\x92\x92\xdd\x2e\x6c\xea\xf1\x00\x7d\xe2\x5e\x17\x0e\x2c\xe7\x00\x48\x8d\xd0\x7b\xf0\xb6\x53\xb6\xba\x72\x85\xf1\x59\xf8\x41\xd6\x38\xb9\x8b\x45\x92\xde\xf0\xe8\xe7\x1e\x75\x0d\x45\x20\x4f\x12\xda\x74\x6d\x60\xa5\x40\x6b\x80\xf8\x09\xa2\x1d\xf9\x61\x66\xb6\xdb\xbf\x04\x5f\x83\x41\x1c\x1b\xa9\x11\x82\x92\x1a\xc7\x35\x48\x17\xcb\x0b\x6b\xa2\x17\xa7\x13\x0a\x4a\xb5\x70\xf6\xb4\xb1\x84\x5b\xe7\x01\xec\xc7\x03\xc1\xb6\x7a\x85\x0c\x99\x47\x6a\x7d\x6b\xbf\x08\x00\xfd\xb2\x68\x04\x8a\x3a\xb2\xc3\xa6\x88\xff\x9e\xb7\x31\xe9\xb5\x88\xf8\x65\x26\x83\x6e\xe8\xba\x6c\xe8\x31\x46\xd2\x52\xb8\x1b\x36\x66\xfd\x75\x71\xe7\xc5\x32\xcb\xbf\x3a\x31\x12\x73\x83\x4e\x10\xb9\x6f\xd8\xca\x67\x91\xdc\x1a\x58\x90\xcd\x27\x3e\xbb\xa5\x06\x83\xf9\x5d\x72\x9a\xe4\x70\x78\x5d\xe7\xbe\x07\xba\x21\x1c\xd1\xfb\x3b\xc9\x51\x4b\x3e\x24\xf7\x6c\xf2\xba\x7c\x5b\x2d\x6f\x6a\x41\x5c\x50\xf7\x9f\xfb\x0a\xed\x76\xce\x1b\x31\xbf\x23\xd3\x08\xb7\x9a\xc9\x00\x2f\xf0\x1c\xb6\x9e\xca\x68\xea\x79\xd0\x32\x91\x4b\x7d\x31\x29\x24\xa4\xc4\x97\x93\xed\x96\x7f\x7d\x05\xbf\x92\x24\x3b\x5c\x3d\xf9\x67\x91\xa8\xb9\xeb\xf8\xc1\x44\xbd\x02\xb1\x41\xf4\x54\xd7\x8f\x1e\xbd\xbc\xc5\x46\xb1\xea\x60\xf2\x25\xf0\x10\x36\xd2\xf8\xe8\x91\x04\x2c\xdb\x24\x8b\xcd\x95\xeb\x7c\x60\x4d\x69\xcd\x17\xd9\xc8\x82\xb1\xdd\x06\xd1\x13\xd4\xec\x56\x07\xfe\xe5\xa4\xe8\xd9\x91\x8f\x74\xe0\x9a\xf4\x67\xa8\x3e\xc8\xf2\x07\xf7\x52\x7a\xc0\xd6\x75\x86\xed\x26\xcb\xd9\xcc\x4f\xd1\x20\x1b\x50\xdf\xb6\xf9\xdf\x26\x82\x82\x7f\x6b\xad\x90\xeb\x4f\x6b\xda\xdd\x0a\xe5\x4e\x6d\x5d\x62\x44\x11\x7b\x7d\x77\x01\x9c\xd9\xdc\x2c\x7e\x45\x61\x68\x26\x46\xfc\x00\x84\x58\x94\xc3\xac\x0c\x7d\x84\xd2\x16\x47\x78\x67\x79\xc6\xf6\x35\xc2\x1d\x23\x8f\x91\xf1\x40\xd1\xff\x25\x2a\xb5\xb9\x0f\x90\x96\xf8\xf4\xcd\x2d\x68\x32\xbd\xb9\x40\xd0\x38\xfb\x68\xcd\xa3\x70\xd8\x2e\x5f\x93\xdd\x03\xf7\x59\x62\x87\x20\xb2\xca\x31\x97\xcd\x8e\xa3\x74\xeb\x7a\xda\x66\x46\x6d\xa4\x92\xb1\xd0\xdb\xda\x51\x89\x98\x1c\xbf\x40\xdf\x4a\x7d\x1c\xee\x0e\x2d\x97\x82\x5b\x79\x52\x22\x0a\x65\xb2\x06\x58\x53\x47\x8d\x48\xd8\xe8\x3f\x6f\x2b\xd9\x72\x07\x17\x52\x44\xda\x07\xee\x58\xb4\x39\xd1\xe0\xb7\x47\x26\xf2\xc0\xc3\x90\x97\x6c\xfa\xd5\xd4\xcc\x32\x05\xad\x40\x7a\x29\xd7\x8b\xc2\x99\x9a\x8e\x6d\x09\x0e\x8e\x63\x83\x94\x50\x41\x90\xa2\xb4\x4a\xf4\xa6\x7f\xa8\x69\xcb\x92\x9e\xd7\x4e\x64\x7a\x44\x4f\xfa\xa3\xe9\xb3\xed\xd4\x53\x18\xb7\x0f\x5f\x3a\xf5\xc2\xcb\xdb\x1a\xf6\x0a\xa7\x21\x41\x7d\xf8\xde\x23\x4d\xa8\x19\x7c\xe1\x02\x03\x6c\x02\xba\x42\xf6\x4a\x2f\x28\xa9\x01\x26\x87\x23\x39\x14\x34\x4a\xde\x10\xf6\xc4\x5b\x4f\xaa\x0d\x9f\x03\xe4\xa2\x23\x54\x67\x61\x5c\x66\xe9\x22\xd7\x1f\x23\x91\xa3\x7a\x28\x25\x24\x41\x99\x91\xf4\x22\x9e\xa4\x79\xd4\xc4\xc8\x3e\x2a\x26\xd9\x2f\xc2\x6c\x4a\xed\x76\x4f\xac\x48\x64\xd7\x21\x0f\x2f\x4f\xee\xb5\x56\x68\x2d\x7c\x0a\x87\xda\xe4\xb3\x43\xe5\x79\x89\x1d\x53\xa2\x2e\x62\x07\xfa\x3f\xc3\xaa\xf8\x48\x63\x4f\x06\x8e\xbb\x46\x5b\x6f\xdb\x43\xc8\xc0\xe7\x62\xbc\x6e\x4f\x8c\xdf\x6f\x6f\x60\x7b\xee\x25\xc7\x49\x8e\xef\x20\x60\x60\x8e\x78\x27\x22\x87\x8f\xdb\xab\xe9\xc6\x86\xbd\xe2\x6d\xda\x90\x2d\x1d\x3f\xb8\x8c\x9e\xb4\x65\x41\xbb\x90\x18\x76\x78\xcc\x04\x5f\x54\xb9\x9d\xae\x67\x45\x82\x35\xa2\x02\xc7\x5f\x66\x66\x79\x49\x02\x5f\x22\xd6\x86\x38\x20\xec\xcf\x8f\xd4\x4d\x3d\xf2\x34\x61\x23\xa3\xd6\x65\xd1\x89\x83\x5b\x7e\x30\xce\x4c\x23\xc4\x0c\x49\xa3\x18\xf4\x8d\x7e\x51\xd8\x2e\x53\x6f\xa6\xeb\x0d\x9a\xa3\x17\x3c\x4e\x18\x80\x8c\x68\x94\x78\x62\x5f\xd2\x22\xcf\x30\x28\x45\x04\x16\x2d\x80\x82\x47\x7d\x90\x77\x99\x9b\x9a\xed\xd5\x8b\x0e\x80\xdc\xed\x76\xb8\x0e\x0a\xc3\xdc\x1a\x09\xb7\x0c\x95\x88\xfc\x00\xe6\x90\x7f\x5a\x79\x91\x25\x47\xfc\xc7\xc5\xa9\xe1\x86\xd8\xa2\x6a\x3f\xdf\xc4\x93\x95\xca\xd2\xf1\xe9\xbb\x33\x18\x12\x62\xd0\xa7\x0a\x5f\x70\x9b\xbf\x4f\x0a\x35\xf2\xe9\x50\x4e\xb7\x1b\xa0\x67\x67\x77\x04\x84\x99\xaa\x8e\x15\x8f\x3c\x1a\x4f\xbc\xce\x25\xc5\x9a\xd6\x4c\x94\x8a\xba\x5d\x45\x0a\x0f\xd3\x0b\xa5\xc3\x75\xfc\x60\x3a\x51\x0a\x85\xd3\x69\x84\xb7\x21\x97\x16\x0a\x08\x6b\xe7\xbc\x08\x43\xeb\xa0\x1f\x82\x86\xdb\x19\x4c\x24\xd6\xe0\xde\x12\x0f\xb8\xa1\xc2\xb6\xd6\xed\xca\x68\x9c\x7d\xe0\xd8\x66\x4e\x78\x7a\x7a\x3d\xaf\xb8\xbb\x6d\xa6\x19\x83\x81\x50\x97\x0e\xbc\x13\x8c\x72\x86\x28\x44\x37\x84\x34\xb3\xa1\x87\x82\x89\xea\x76\x7b\x3d\xac\xc7\x2d\xa9\xe7\xc0\x5d\x3c\x72\x8a\xec\x43\xbe\x24\x8e\xe4\xb1\xda\x44\xbd\x5d\xf4\xec\xa2\x78\xc7\xf7\xc7\x09\x87\x61\xac\xa2\xb6\xd8\x45\x99\xf5\x17\x6a\xa2\x2b\xf3\xd0\x38\x60\x27\x72\x3e\xe8\xa4\x7f\x1c\x21\x28\x18\x7d\x04\x8d\x77\x94\xc6\x3b\x31\x8d\x6f\xb7\x1d\x46\x1e\x74\xe5\x8d\x9b\x40\xf3\x47\x1c\x7c\x7b\x23\x76\x48\x25\x54\x66\x84\x5b\x51\x68\xc1\xc1\xda\xee\x7e\x98\x7b\x4c\x2c\x8d\xa9\x4a\x38\xdd\x76\x8b\x5f\xff\x95\x7e\x43\xda\x09\xda\xa3\x5d\x5e\xde\xa7\xef\xd3\xe2\x04\x9f\xb9\x8c\xe5\x60\x16\x43\x5d\x68\x42\xbe\x5b\x59\xbf\xc2\x00\xe5\x12\x7d\x8f\xaf\x89\xc6\xfc\x77\x22\xb1\x1d\x9c\x7f\xd1\x58\x25\x59\x1f\x45\xf1\xa9\x21\x81\x31\xb1\x66\xa2\x9c\x1d\x71\x05\xd4\x83\xf0\x2f\x67\x96\xe1\xa2\x5a\x8f\x2e\xbd\x00\x8d\xf9\xe5\xc2\x85\x71\xc2\xe4\xba\x90\x5e\xe5\x12\xcb\xc0\xfe\xae\xa5\x27\x26\x02\xb0\x7f\x35\xad\x7f\xba\x5d\xfc\x2c\xaf\x0f\xa5\x6d\xf0\x66\xa3\x4f\x89\xf4\x58\xcd\xde\xd9\x70\x80\x97\x52\xa8\x11\xdc\xd1\x8a\xdb\x12\x43\x6a\xf6\xae\x68\xc6\x30\x71\x5e\xb5\x19\xf9\x4e\x8e\xa1\x98\x1f\xe1\x91\x3e\xbd\x42\xbb\x6c\x8f\xdd\xd5\xa5\xec\x55\x27\x08\x48\x4d\xe5\x16\x78\xe4\xe5\xc9\xc3\x17\x6e\x37\xf9\xbe\x21\x6a\x44\x25\x1a\x1a\x5e\x22\xd3\xdd\xbe\x35\x29\x56\xdf\xf8\x22\x76\x5c\x6b\x8b\x6d\x90\x0d\xc3\xf8\x5b\x6e\x42\x3f\x8c\xe0\x03\xf1\xaf\x9c\xcf\xbd\x3a\x36\x53\x45\x52\xd0\xa2\x83\x3f\x86\x2b\xaa\x7c\x3f\x3e\x3f\xb6\x4c\x1c\xf2\x8a\x11\x1d\xa0\xd9\x06\x87\xc9\x0e\xed\xfd\x36\x48\x08\xe0\x08\x99\x12\x95\x50\x80\x28\xd5\x41\xa4\x1a\x3c\xd3\xb1\x16\xd8\xc8\x62\x76\xf4\x58\x7f\xfd\xc1\xb9\x38\x68\x2a\x16\xbb\xf0\x52\xa6\x53\x50\xef\x9c\xce\x78\x4f\x3f\x81\x31\x49\x6b\x1f\x69\xac\xd2\x12\xdd\x2a\x19\x1c\x3c\xa7\x20\xe9\x81\xaa\xca\x49\x99\xd4\x36\x06\x4a\x01\x30\xe6\xd3\xc5\xab\x3d\x20\x60\x96\x8b\xd3\xef\x1c\x74\xf8\x70\x82\xb9\x74\x9a\x38\xc4\x47\xa8\x75\xe0\x22\xd4\x14\xf5\x55\xe0\x3f\x7f\xe0\x98\xa8\xf4\xf6\x3d\x34\x33\x9c\x2d\x8f\x24\xac\x39\x7e\x16\xa1\xf5\x4c\xcb\x21\x08\x66\x3d\xc7\xfe\x93\x56\x3d\x74\xc2\x59\x39\xe5\x10\xc4\xde\x99\x94\x3a\xd0\x1f\x11\x88\x36\xbf\x28\x18\x2b\x27\xb2\x27\x53\x59\xc5\x3a\x66\xa1\x3c\x9e\x0d\xe1\x8c\x08\xb4\xe1\x69\x1f\xbd\xa8\xed\xec\x5a\x1d\xa8\x48\x33\x77\x87\x0c\x13\xc6\x4f\x32\xc5\x3c\x86\x5f\x6b\xa9\xaf\xd4\xef\x72\xbe\xbc\x90\x47\x3e\xa3\x04\x64\xb5\xd6\xde\x16\x7f\x77\xbb\xf8\xaf\x08\xed\x74\x72\x60\x2a\x9a\xed\xcc\x7a\xb9\x6c\x74\x14\xc6\xf8\x91\x8b\x6c\x7d\x13\xfb\x43\x65\x5b\x5f\xa2\x76\xc1\xf3\x11\xb6\x6f\xb1\x21\x77\x91\x65\x93\x68\xeb\xef\x74\x84\xe8\x49\xc9\x4a\x3f\xd1\xe4\x75\xbb\xfd\x27\x27\x4f\xcf\x49\x68\x80\xe3\xad\x18\xae\x1c\x32\x81\x55\x83\x16\x7a\x32\x05\xbd\x52\xeb\x4f\xaa\x82\xcf\x90\xb3\xfd\x4f\xdb\x5c\x58\xcd\xc8\x81\x43\x40\x33\xbe\xb3\x55\x7b\xc8\x81\x47\xad\x9a\xb6\x09\x5b\x50\xb5\x16\x52\x8b\xa6\x9d\xd1\x5f\x87\x86\xe0\xdf\xda\x47\x09\xb6\x25\x16\x15\x83\xc6\x65\xc4\xe5\xf5\x6a\xf3\x3e\x6e\xfe\xf7\x68\xd2\xf5\x35\x02\x1d\xcd\x69\xf2\xdf\x93\xed\xf6\x8b\x66\x90\xcd\x2f\x1b\xab\xa1\x19\xc8\x51\x4e\x63\xed\xc3\xee\xf8\x22\x52\x9f\x06\x20\xaa\x7b\xc3\x5e\x54\x7b\xb0\x25\x3e\x56\xfd\x96\x87\xfe\x76\x86\xa6\x6c\x5f\x45\x76\xe5\x6a\xaf\xc8\x4e\x5c\xad\x14\xf3\xfb\xa2\x81\x7b\x71\xb8\xbd\xa5\x11\x45\xe7\xc6\xc7\xd3\xde\xb5\xf2\x0c\xd4\xdd\x34\x5a\x3e\x00\x05\x74\x88\x4d\x05\xdd\x61\xf4\x49\xd2\xf5\xa4\x74\xd1\xd9\xc2\x58\xd9\xbe\x1c\x98\x2e\x16\x68\x9c\x46\x6d\x43\xf8\xb8\x16\xdd\x13\xec\x8b\x17\x1e\x2b\x21\x31\x70\x67\x46\xea\x8a\x0f\x56\x21\x19\x40\xa2\x75\x1b\x8d\x59\x21\xad\x88\x2e\x76\x80\x8d\x95\xbf\xfe\x9e\xa6\x4c\x1c\xa9\x7c\x7c\x72\xaa\x49\x23\xfd\xd1\x93\x80\xc7\x56\x0c\xde\xf1\xab\xbb\xbf\x0f\x76\xcf\xc1\x7c\x28\x72\x05\xfa\x94\x17\x0f\x32\xbf\xb8\x5e\xf9\x05\x32\x9f\xe4\x41\xf7\xcb\xd9\xec\x8f\xf6\x3e\xf8\xfd\xbd\xcf\x7f\xd7\xb4\x79\xb8\xf6\x70\xf0\x61\x74\x0f\x8f\x8f\x2b\x38\x62\x0d\x3f\x05\xc2\x57\xff\x6b\x21\xec\xf5\xaa\xaf\x45\x38\xfe\x04\x28\x77\x3b\xba\xcc\xac\x40\xea\xbd\xa3\xb7\x31\xf3\x8e\x28\x04\xcf\x97\xef\xf0\xf7\x25\x88\x1b\xf8\x17\x7d\x62\x6e\x97\xeb\x19\xfe\xae\xae\xa7\xaf\x30\x71\x97\x05\x47\x48\x10\xb3\x9b\x0f\x19\x54\x99\xeb\x80\x1f\xce\xc5\x16\x40\x68\x2e\x37\x07\x5a\x08\x9e\x21\x80\x26\xe4\x6c\x52\x58\xdb\x0c\xfe\x6c\x48\xd6\x1c\xdf\x5b\x43\xd4\x78\x41\x82\xf0\xa0\x53\xce\x83\xd8\x52\x98\x24\x51\xef\x83\x90\x10\xcd\xc0\xf7\x1d\x89\x7c\xef\x5e\x23\x81\x43\x3e\xfd\x6d\x09\x74\x93\xe9\xfb\x20\xc1\x51\x45\x3a\x92\xd3\x84\x17\xe1\x8d\x8a\xd2\x01\x91\x8b\xb8\x13\x95\x1f\xe2\x88\xe6\x50\xa0\xc8\xa3\x71\x04\x99\xc3\xb6\x11\xa5\x71\x04\xde\x4f\x0b\xee\x66\x2f\xf2\xb9\x6d\x31\x75\xd7\x5d\x71\xb1\xb9\x2a\xc2\x6d\xf2\x57\xdf\x9d\x03\x43\x9f\xd9\xa3\xba\x7b\xaa\xdf\xaf\x11\x9f\xe8\xe9\xd6\xc4\x7d\xfa\xce\xa1\x20\x30\xfb\x76\x31\xf2\xd4\x18\xfe\xb6\x8f\x71\x62\x64\x1a\x2e\x63\x9f\x1e\x6d\xbb\xf3\xf6\x33\xc7\x49\x9e\x4c\x0a\x2f\xc1\x42\x26\x69\x6c\xc3\x51\x34\x2c\x3a\x34\x7f\xd3\xb4\x6a\xd0\x2c\xf2\x01\xc5\xa0\x26\xbe\x03\xae\x89\x5e\x67\x2f\x22\x9b\xa3\x9d\xbc\x84\xc6\xee\x74\x70\xd0\xd9\x54\xf3\xe2\xfe\x7f\xe2\x1f\xf4\xf7\x66\x91\xa5\x46\xc5\x32\x3b\x7d\x4b\x02\xc6\x88\x7d\x0b\x9f\x54\x70\x4b\x26\xf8\xf7\x0d\x3d\x88\x8b\x8a\xf8\xfb\x2f\xfa\xe3\x17\xf9\x9f\xce\xc6\x67\x7d\x33\xb9\x87\xed\xf8\x4e\xf4\xc1\xf0\xf9\xe6\xcf\xcf\x36\xaf\x6e\xa6\xeb\x29\xfc\x2c\x67\x12\x74\xee\x8e\xf4\x96\xd0\x2f\x71\x0c\x3e\x45\xd6\xf8\x1b\x45\x35\xe2\x18\x00\x0c\x2e\xf1\x61\xf3\x99\xd4\xcb\xe0\x29\xbf\x48\x45\x81\x76\x1f\x28\x8e\x5e\x36\x62\x9f\x78\x36\x1b\x7b\x9f\xe4\xc2\x8a\xfc\xb6\x3e\x3f\x4f\x0c\x6b\xe3\xd9\x66\x7a\xf1\x46\x5e\x4e\x76\x9d\x09\x7d\xfa\x82\x01\x47\x53\x3a\x09\x63\xac\xc4\x0f\xe9\x63\x07\x68\xc6\x48\xef\xe5\xb9\xd0\xe5\x62\x18\x83\x36\x2b\xec\x34\xe3\xb5\xe2\x91\xa6\xf7\x4e\x20\xc7\x6e\xc1\xd1\xfa\x26\x1a\x45\x04\x36\xb6\x32\x18\x05\x74\x88\x9d\x64\x39\xfe\x83\x95\x2d\x49\x17\xfc\x54\xb0\x8d\xdc\x1c\x7c\x21\x2f\xcb\xe1\xc0\xdd\xb3\xfd\x93\x39\x08\xea\x2a\x3c\x6f\x19\x7a\x3c\x53\xe3\x7b\xd1\x57\x5d\xe8\x3b\x9b\xfc\x9a\x2f\x8d\x9a\x27\x86\xf3\x23\x2f\x4c\xea\xf4\x77\x62\x96\x67\x8d\x9b\x45\x8e\xe8\x21\x77\x67\x16\xfe\x71\xb2\xa1\x48\x89\x90\x06\xab\x67\xb1\xbc\xe5\x06\xed\x80\x3b\x83\x2c\xb3\x17\xd8\x7f\xb0\x29\x7a\x5f\xbe\xaa\xf7\x37\xd3\xe9\x78\xcf\xf4\xee\x7f\xeb\x76\x7d\x28\x60\x57\x44\xb1\x4c\xd0\x36\x34\x2c\xa9\x06\x52\x79\x80\x31\x03\xe1\x24\xb7\x64\x46\xd8\x0f\xaa\x39\xf5\x51\xee\xcf\x90\x5b\x0c\x9a\x0d\xc3\xba\x98\x2f\x6b\x00\xa4\x39\xb6\xda\x39\x2a\xd9\x08\xc4\x62\x14\x3d\xf7\xd7\xa9\x91\x45\xb0\x5a\xd6\xc5\x81\x01\xa2\x63\xc0\xde\xe5\x5c\xc7\xa3\xb7\x9d\x6f\xb7\x01\x1e\x72\x20\x2a\x21\x29\x35\x5d\x96\x95\x35\xa4\x87\x9f\xd0\x82\x39\x7a\x7d\x0e\x92\xdc\xcb\x63\x62\x53\x4a\xaf\x37\xa9\x59\x99\x46\xd8\xc6\xa0\xce\x82\x67\xb4\x76\xc6\xab\x55\x6f\x29\xc7\x16\xb4\x58\xc6\xb8\xc1\x11\x25\xc8\xde\x8c\xa6\xd2\x7c\x6f\xd2\x34\xaa\xde\xb5\xd3\x1d\x56\xb6\x4a\xf1\x3d\xcb\x7f\xc7\xd1\xa4\x0e\x1c\xd9\x47\xad\x36\x6e\xda\x5e\xf5\xff\x52\xf7\xee\xfd\x6d\x1c\x47\xa2\xe8\xff\xe7\x53\x80\xb3\x5c\x64\xc6\x18\x82\xa4\xec\x64\x93\x81\x47\x38\xb4\x1e\xb1\x76\x65\x49\x2b\xd2\x9b\xec\x05\x61\x1d\x90\x00\x49\xc8\x20\x80\x60\x40\x3d\x42\xe0\xbb\xdf\x7a\x76\x57\xf7\x0c\x28\x39\x27\xbf\x73\xee\xfd\xd9\x22\x7a\xfa\xfd\xac\xae\xaa\xae\xc7\xfc\x64\xb5\x1a\x7d\xd6\x4d\xa4\x2e\x75\x99\xe3\x57\x44\x99\x88\xd2\x7a\x4f\xaa\x3b\xc4\x1e\x83\x02\x84\x89\xf0\xfe\x2a\xa4\x8e\x76\x5b\x02\x56\xd1\x56\xb6\x1c\x50\x67\x69\x46\x8e\x7f\x51\xe7\x47\xf7\x1c\x3a\xc2\xca\x47\xe3\x86\xfb\x20\xd4\x49\xac\x50\x1a\x70\xe7\x41\x8a\xcf\x8a\xdb\x1e\x32\x8a\xdb\xd1\xaf\x13\x1e\x87\x3f\x97\x8e\xd1\xed\x0d\xe5\x6b\xd4\xb0\xf0\x4e\xe6\x46\x40\x9e\x6a\x2d\x13\x00\x4e\x0c\x66\x01\x48\xa1\xbd\x30\x03\xbc\x1b\xef\x1b\x5d\xb3\x11\x5e\xc6\x34\x4e\xb4\x23\xf9\x25\xf0\x83\x2e\x9d\x99\x2c\x0e\xa1\x39\xce\xdd\x6b\xf2\x9d\x57\x44\xdf\xb5\x13\x9d\x85\x0e\xca\x01\xa5\x3c\xc5\x9b\xd1\xc4\x70\x57\xf2\xc0\x5d\x7c\x33\x33\x84\x85\x60\x9a\x1e\xad\xbd\xe4\xcb\xca\xcb\x67\xcb\xbb\xa6\x9b\x55\xa9\x94\x05\x6e\x05\x55\xd9\xb1\x67\xa5\x33\xe3\xa9\x08\x85\x58\x19\xc1\xcc\x15\x26\xfc\x26\xe2\xe4\x4f\x73\xc2\x92\xbe\xaa\x26\xc9\xba\x65\x54\xa5\xb9\x27\xa2\xb8\x20\x85\xed\x93\x75\xb6\x65\xd4\xe6\x6b\xca\xc5\x6f\xff\xd2\x26\xec\xff\xaf\x9d\x80\x86\x96\x7f\x43\xe9\x1d\xed\xff\x43\x13\x68\x7b\x22\x79\xff\x07\x77\xe8\x1f\x5b\x8f\x58\x2c\x42\x17\x45\x66\xf0\x0b\x5b\x44\xe7\x39\x8d\x76\xe4\x66\x73\xbf\xcd\x0c\x23\x51\xc5\xe9\x1d\xb6\xfa\x55\xb5\x46\xdc\x48\x2c\xaf\x18\xee\x83\xe5\x95\xf1\x25\x63\x9c\x5e\xad\xe0\x23\xc9\xf8\x3d\x41\xaa\x08\xe5\xf3\x35\xf6\x2f\xa1\xcd\xc3\x22\x80\x38\x42\xbc\x1a\x51\x01\x92\xc5\x0b\x24\xb0\xd0\x21\x9e\x3b\xdd\x22\x0a\xef\x72\xd0\xdc\xe6\x21\xa2\x8d\xb7\xb4\x83\x8e\x4b\xc6\x70\xae\xe6\xb2\x0c\x0e\x23\xa5\x4f\xbe\xb8\x49\x42\x8b\xec\x01\x3b\xef\xe1\x98\x35\xf7\x10\x75\x27\x7c\x06\x24\xc8\x34\x17\x63\x28\x78\x97\x11\x2e\x6b\xb1\x08\xb4\xe7\xbb\x17\x53\x1d\x3c\xae\x1d\xf7\x61\x1e\x95\xb6\x74\x92\x19\x81\xf4\x05\x2f\x58\xf4\x41\xbf\x62\xf5\x86\xfa\xf5\x6b\x21\xa8\x27\x5c\x42\xec\xd1\xa9\x06\xa1\x9c\x84\x7f\x73\x83\x30\xc9\x72\x02\xdd\x99\x14\xf0\x91\x92\x5d\x95\x15\xdb\x6d\xd3\x37\x1a\xe7\x0b\xfc\x21\x9c\x82\x32\xe2\xdd\x4c\x1a\x33\xfd\x81\x7e\xa3\xe5\x8a\x26\x64\xc4\xf4\x28\x23\x6b\x99\xd1\xb1\x44\x3f\x60\x72\x30\x15\x87\xd3\xa7\x57\x64\x87\xdc\xad\x8c\xeb\x2a\xc2\x9f\xfe\x14\x21\x47\x30\x2e\x26\x47\xcb\xd2\xe8\x9e\xc7\x28\xd4\x66\xb3\x27\x37\x32\xe2\x3e\x40\xf4\x72\x21\x14\x72\xc0\xe1\x87\x15\xea\x7b\xba\xc3\x95\x72\xc1\x92\xb8\x1b\xc1\x8b\xf3\xd8\xc1\x07\x23\x80\x98\xfb\xb7\x04\xda\xd8\x68\x28\x6b\xde\x9b\x97\xf3\xe0\xd9\x80\x44\x61\x4c\xb3\x73\x91\xbc\x80\x8d\xc2\x4d\xcf\xfd\xae\x87\xa5\xef\xd9\xa7\x98\xaa\x4c\x46\x17\x17\xab\xcd\x68\xb5\x9e\x5e\xce\x26\x9b\x51\x35\x05\x50\x33\xba\x1b\x4f\x17\x9b\x8b\xf1\x74\x73\x39\x9a\x7f\x18\x55\xe4\x4f\x8b\xfe\xcc\xa6\xd5\x7a\x33\x9e\x00\x5d\x33\xab\x36\x57\xd3\xeb\xcb\x11\xbd\xc1\x60\xf0\x6e\x35\xd9\x5c\x2d\x16\xe8\x6c\x87\x5f\x09\x36\x37\xcc\x52\xc2\x87\xd4\xcd\xed\x04\x13\xe6\xa3\x0f\x9b\xc5\xdd\x1a\x2d\xb9\x2d\x57\x8b\x6b\x54\x2d\xd9\x54\x13\x1a\xed\xa6\xba\xbb\x85\x9c\x9f\x37\xeb\xe9\xed\x64\xf3\x01\xba\xb1\x48\xd0\x02\x1c\x0c\x71\xc2\x53\x5e\x1e\x0a\x34\x3a\x1f\x77\xc8\x98\x27\x5e\xbc\x1b\x34\xa9\x97\xa0\x71\xb6\xf9\xa2\xba\x99\xde\xb2\xac\x8e\x15\x4f\xfd\x9e\x0c\x69\xba\x01\xa3\xa5\xc2\xf3\xf3\xea\xf0\xf1\x50\x2c\xee\xad\xc4\xaf\xc5\x5f\xbc\x45\xd1\xc3\x5f\xce\xab\xce\x61\xbe\xfa\x74\xb3\xbe\x9d\x9d\x8d\xae\xcb\x43\xa8\x64\x8f\x0d\xce\xad\x36\x97\x8b\xd9\x66\x72\x7b\x01\xdb\xe3\x66\xb5\x99\xde\x5e\x6f\xd8\x34\x1d\x74\x94\x86\x39\xda\xc0\xf1\x1c\xdd\x66\x29\x9a\xb2\x2b\x86\x9d\x6c\xf0\xcb\xe3\xe1\x37\xd9\xf9\xe1\xe3\xc3\xeb\x69\xbe\x12\x43\x19\x58\xa5\xa4\x43\x43\xeb\x8b\xc5\x18\xc6\xf7\x3d\xfd\x92\x2d\x3b\x68\x18\xbe\x37\xed\x7f\xe9\x9f\x7f\xec\xf4\x90\xa7\xb1\x78\x81\x6f\xd8\x92\x00\x63\xaa\x2e\x57\xd3\xe5\x7a\x53\xad\x3f\xc3\xba\x61\xe3\x19\x14\xbc\x1d\xcd\xa7\xcb\xbb\x19\xbd\x6a\xbe\x5b\x11\xbf\x13\x1f\xe7\x48\xe2\x90\x38\x2a\xca\x02\xdd\x10\x53\x94\x0d\xe7\xc9\x8b\x5a\x79\x28\x81\xf3\x0a\x75\xe5\x07\xbf\x94\xc3\x4d\x09\x61\x7d\x71\xeb\x62\x03\x2b\x6e\x56\x2a\xdc\xdf\x9c\x1f\x42\xd6\xf7\xa3\x0f\xa3\xcd\xe4\xf2\x76\x94\x71\x6a\x98\xef\xa7\x51\x45\xb5\xff\xb2\x5e\xdd\x4d\x20\x7f\xf7\x1b\x1c\xf2\x25\x5a\xca\x3a\xa5\x4c\x34\xdf\xdf\x7c\xbf\x87\x0a\xfa\x83\x27\x4f\x4f\xce\x4e\xce\x07\x9b\x83\x83\x6c\x83\x11\xc3\xf3\x21\x86\x1f\x43\x8e\x7d\x58\xe6\x8f\xab\xd1\x12\x9d\x4a\xdd\xf3\xe3\x5f\x31\x38\xce\xd5\xce\x82\x33\x3c\x5e\xfe\x4e\x43\xbf\x7b\x0c\x8b\xec\x2d\x2c\x0c\x81\xc0\xbf\x06\x78\xc7\xa5\xae\xa6\x93\xd9\x18\x10\x5d\xce\xe3\xbf\x86\x39\xae\x34\xe7\x81\x0b\x84\x93\x29\x30\xcc\x69\x69\x39\x89\x7d\x3d\x73\xaa\x86\x91\x7b\x02\xbb\x89\x33\xac\x71\xe6\x39\x5d\x82\x90\xbc\x2a\x06\x8f\x5c\x1a\xaf\xb7\x64\xa1\xa0\xc9\x0a\xdb\xac\x21\xaf\xcb\x08\xc9\x74\xbc\xb8\xb4\xfb\xb2\x6d\x41\x3f\xbe\xad\x95\x5f\xaf\xa4\xbd\xd5\xe3\x86\x46\xdf\x01\xe4\x1b\xc1\xe4\x29\x08\x56\xbd\x2c\xdc\x76\xa7\x93\xd5\x14\x00\xc0\xdf\x81\x9c\x38\x42\xc9\xa5\x24\x19\xd2\x40\xff\xfa\x3d\x9b\xd3\x4a\xc4\xae\xd6\x10\x40\xd9\xe8\x6a\xf2\x7c\x35\xba\x26\xdb\x72\xcc\xf9\x3e\x35\x51\xce\xee\x5c\x96\x5f\x49\xd4\xd3\xe9\x87\xd2\x96\x8a\xcd\xc3\xf1\xc3\x37\x57\xf5\x8c\x79\xba\x29\x9b\x11\xc8\x7a\xb2\x27\xba\xb0\x25\x98\x79\x6d\x22\x50\xb7\x47\x3f\xf9\x94\xb9\x2f\x04\x56\xee\x4b\x27\xd0\x47\x30\x64\xf3\xd9\x71\x61\x7d\x55\x37\x3e\x61\x9c\xd7\x19\x81\xe1\xd3\x22\x9b\xd5\x88\x50\xaa\xd1\x25\xaa\x23\x08\x7a\xd2\x9c\x53\x8d\xef\x78\xa3\x2d\x86\x2f\x9b\x7a\xfa\x54\x5e\x70\x33\x99\xb4\x34\xad\x11\xad\x91\x02\xa5\x9b\x7f\x99\x51\x64\xec\x92\x42\x22\xb7\x8f\xd8\x34\x0a\x04\xd1\x97\x7b\xb8\xa9\x9c\x75\x76\x9a\x86\x00\x5b\xc7\xfd\x41\xdc\x05\xc7\x1d\x7e\x1e\x24\x65\x96\x0a\x24\x92\xcc\x1a\x85\x54\x56\x1c\x31\x74\xa4\x6e\x2a\xc7\x22\x67\x34\x45\x53\x94\x72\xea\xa9\x6d\xe8\xa3\x21\xe3\x7a\x98\x59\x39\x79\x58\x20\x6f\x1c\x70\xd6\x9d\xfc\x2d\x3d\x82\xc1\xce\x16\xf3\x49\xba\x77\x94\xf5\xea\xc4\x7c\xbb\x8d\x75\x85\xb6\xe0\xb4\x29\x1a\x2f\x21\x93\x11\xeb\x4f\x05\x6f\x08\xbf\xeb\x45\xc8\xb5\x91\x5a\x31\x76\xcd\x3c\x33\xa6\x51\x3c\xc0\xf0\x38\xb6\x6e\x3d\x69\x62\x2c\x3f\x85\x57\x80\x2e\x84\x78\x0d\xc2\x4d\x16\xaf\x43\xff\x2b\x17\x80\xaa\x6e\x5c\x82\xa2\xa1\x06\x65\x64\x38\x3a\x9c\x6b\x72\xb4\x05\x52\xfb\x57\x4a\x10\x54\x29\x3e\x7b\x71\x50\x31\x45\xf7\x6d\x17\x3f\x2b\xa8\x98\xcc\x01\xc5\x6c\x65\xef\xc5\x83\x26\xce\xae\x1b\x6a\xb9\x63\xf0\xbd\xdf\xbe\x09\x7d\x15\xfd\xda\x64\x14\xae\x4b\x77\xf3\xb0\x53\x11\xcf\x95\xf6\x18\x9c\xce\x78\xd2\x62\xda\x8a\xea\x4d\x10\x48\xa1\x2c\x56\xd0\x1f\x91\x0d\xfb\xcb\x74\x7d\xc3\x6c\x1a\x4b\x31\x41\xcd\x30\x41\xc8\x85\xa1\xa9\xda\xd5\x8f\xf1\xe2\xf6\x27\x44\x0f\x9c\x80\x6a\x95\xe3\x3b\x68\x48\xf6\x91\x60\x35\x65\x8f\x34\x4c\xa3\xb8\x3f\xc5\x51\x19\x03\x9b\x00\x6e\x33\x89\xca\xd4\xfd\xff\xfd\x9e\x05\x47\xdb\x71\xf8\x02\x6a\x18\x85\x51\x28\xfd\xb7\xf4\xf4\x38\xee\xa9\x59\x75\x86\x2c\x51\xc4\x8e\x9e\x50\xf3\xa3\xab\x80\x0e\xfb\x3f\xd7\x7a\x40\x52\x60\x4f\xd8\x7a\x64\x03\xf7\xf2\xd7\xc9\x64\xf9\x14\x68\x81\x48\xd6\x15\x9f\x41\xac\x93\x24\xe1\x56\xb3\x8f\xa4\x74\xcf\xbb\xf2\xdc\x41\x2c\xc7\x06\x0b\x90\x9e\xd5\xa6\x98\x10\x9b\x84\x52\x50\xfa\xcc\x82\x88\x24\x66\x4a\x51\xbe\x46\x0c\x31\x64\x2c\x80\x18\xcc\x83\xab\xad\xdd\x8e\x5f\x68\x28\x6f\x70\x65\x30\xe9\x45\x86\x05\xfe\x3c\x5b\x5c\x8c\x66\xcf\xd0\x16\x80\x69\x21\x4f\x18\xc3\x4d\xea\x4d\x05\x86\x37\xb9\x37\x01\xec\xa9\x89\x8c\xfd\x96\x99\xa4\xac\x0d\x76\xe6\x1e\x9a\x0d\x7a\x22\x8a\xaf\x27\xbe\x80\x6a\x7d\x0d\x8c\x9e\xf2\xbc\x10\x22\x54\xb9\x26\x22\x5e\x10\x2f\x60\x92\x39\xa3\xca\x9c\x5d\x2d\x9e\x1e\x45\x17\x17\xdd\xc1\x7e\xe8\x48\x54\x9e\xcc\xc7\xcf\x3e\xd0\x6e\x1e\xcb\x0a\xb9\x18\x6f\x87\xd4\xc6\x8a\x3f\xb0\x20\xae\xbf\x77\x5c\x7c\xa1\x32\x2d\x16\xc7\xf7\x83\x72\x45\x2d\x9d\xcf\x47\x74\xfd\x87\xf7\x2c\x63\x16\x04\xc0\xbf\x34\x22\x3c\x5b\x78\x73\xfc\x6f\xa0\x87\x01\xda\x01\x08\x0a\xf2\x06\xeb\xcf\x55\x88\x2c\xd5\x30\xc8\xcc\xa8\x99\x05\x3b\x88\x59\x79\xce\x30\x9b\x97\x46\xb6\x24\x38\xe0\xfc\x99\x37\x71\x49\x22\x24\xf5\x97\x6b\x6a\x73\xb3\xb1\x14\x2b\x73\xa9\xb8\xf7\x8e\x85\xd2\x4c\x5c\xb4\xdb\x96\x92\x7f\xb8\x60\x8d\x72\x87\xc2\xb5\xb8\xb0\x0a\xc1\xda\x07\xa9\x12\xe2\x80\xb2\x4f\x2e\x5d\xf2\x80\xe9\x9a\xac\xa6\x74\x3e\xcc\x70\xda\x71\x32\xd9\x51\x91\x9b\x1e\xe5\x11\x00\xf9\xb3\x7f\x0c\xa4\xd4\xfe\xa3\xc7\x49\x46\x76\xd4\x49\xbd\x47\xdf\xf0\xec\x41\xa6\xe5\x6a\x38\xc2\xe9\x17\xcf\x70\x1e\x2e\x12\xf7\x85\x8f\x69\x79\x64\xac\xc3\x6c\x99\xff\xd3\x44\x1b\x70\x91\x2f\x21\xf8\x06\xdf\xa8\xed\x53\x8f\x6c\x35\x20\x5a\xd2\x23\x55\x39\xa0\x04\xff\x1c\x5a\x46\x7b\x44\xf6\xa7\xa0\x3a\x5c\xb6\x8b\x7c\x45\xc6\x7a\x90\xd1\x04\x23\x52\x2e\xa6\xbb\xfe\x06\x94\x73\xd8\x80\x24\x10\x6f\x0b\xa5\x3e\xe2\x2b\x4d\x95\x13\xa3\xcb\xb0\xa7\x0f\x3c\x69\x84\x6e\x21\x50\x4c\x33\x29\xd5\x70\x65\xce\xc9\x45\x13\x9e\x65\xee\xe4\x97\x5e\xbe\xa4\x46\xfb\xd2\x0e\x45\x65\x40\xc6\x2a\x17\xaa\x30\x11\x35\x9e\x23\xaa\x79\x31\x42\x9d\x63\xd2\x1c\xba\x84\xa6\xdf\xc1\x85\x05\x8b\x2c\x2e\x9c\x06\x43\x56\x9a\x21\x86\x1e\x2b\xa9\x92\x42\xe7\xcd\xa8\x62\xbe\x4a\x95\x57\xf2\x8b\xe6\xdd\x95\xd2\x6e\x78\xd7\xae\x84\x43\x9d\x4f\x5f\x2d\x9e\x20\x24\x2b\x67\x07\xc7\xbc\x39\x4a\x6c\x02\xf9\xb4\x0f\xa2\xd7\xb2\xea\x00\x0f\x7c\x24\x1a\xa1\x3b\x7e\x5c\xce\x1a\x1e\xc3\x65\xf5\xa3\xd3\x4c\x6c\x26\x6a\x1e\xee\x12\x65\x4a\xd9\xd3\xfb\x10\x11\x49\x22\xee\x9e\x0c\x81\x11\x21\xcd\xc7\xd1\x3d\xdf\x29\xb4\xdc\xc9\x03\x92\x73\x6c\x10\x7a\x12\xf0\xa6\xb9\xef\x13\xe1\x81\x07\x3b\x35\xe0\x2e\xa3\x17\x81\xab\x00\x0b\xab\x2d\x16\xd3\xa8\x33\x68\x47\xe7\x5b\xa7\xeb\xe2\x0e\xae\x54\xc7\x01\xe1\xb2\x4d\xc4\x2a\xa2\x74\x4c\x3e\xb1\x42\x9c\xd6\x63\xdf\x7e\x48\x49\x58\xe3\x3d\x29\xe0\xec\x61\xfa\xc6\xa9\x90\x54\x95\x89\x4b\x6d\xec\x71\x49\x7f\xeb\x77\x3a\xef\xa3\x64\xbd\x4a\x32\xdd\x3d\xf6\x31\x45\x00\x92\xdb\x4b\x0e\x09\x52\x55\x01\xde\x79\x99\xd9\x84\xa5\x54\xe3\x1c\x3d\x09\x44\x24\xbd\x47\xbf\x29\x01\xbf\xd3\xdd\x87\xb2\xd3\x98\x18\xdc\xad\xb4\xb5\xe1\xc0\xa3\xd7\x62\x5f\xb9\x1b\x00\xbf\x27\xe9\x7e\x97\x6e\x52\x19\x8f\xa7\x65\x6e\x99\x64\xd9\x9b\xa7\x40\x05\x98\x12\x4a\x4f\xb2\x3e\xbe\x43\xbc\x5e\x9d\x78\x5a\x9c\x53\x99\x50\x2b\x34\x82\x1a\x9b\x66\x62\xdb\x48\xfa\x47\xb2\x1b\xe8\xd2\x41\x7a\x36\x08\x27\xe3\xe0\x38\x5e\x7c\x33\xd7\x3a\x98\x15\x9c\x00\x38\xfc\x3a\xb3\x88\x1d\xfa\x06\xec\x64\x6a\x1b\x28\x7c\xe5\xd9\xaf\xf2\x42\x84\x78\xa9\xc8\xc1\x23\xae\xa6\x57\xe9\x3b\x44\x57\x64\xa2\xae\x1d\xa2\x8b\x39\x62\x1c\x19\x61\xc8\x9c\xb5\x71\xb8\xb6\x6a\x75\xa9\xfc\xa8\xd1\xfb\xd1\xa7\xf4\xfe\x6e\x35\x2b\x34\x85\x8d\xad\x27\x7f\x7e\x76\x96\x10\x46\x74\x46\x9f\xb2\x14\xf9\xa8\xfa\x3c\xbf\x2c\x60\xa3\x73\x93\x18\x4a\xc8\x2c\x55\x95\x90\xf0\xac\xb2\x1f\x7d\x8f\x52\x19\x01\x09\xca\xb8\xa0\x53\xf2\x9a\x33\x61\x23\xf7\x62\x64\xbf\xc6\xb0\x9a\x11\x81\x41\x89\xb6\xe0\x78\x10\x5e\x18\x60\xa9\xe6\x81\x8c\xc4\x07\xf8\xee\x3c\x5b\x14\x09\x87\x12\xa5\x68\x31\x4a\x82\x49\x6e\x6f\x89\x22\x61\x42\x52\x63\x4f\x88\xae\x4b\x88\xbc\x4b\xf4\x7a\x45\x16\x5a\x62\xae\xda\x24\x7e\xfb\x5c\xac\xa6\xd7\xd3\xf9\x68\xf6\xc0\x0b\xa8\xbf\x6e\x2c\x0d\x51\x89\x79\x6b\x16\xda\xa3\x1e\x94\xb1\xa0\x20\xe9\x08\x94\x9c\xe8\xf6\x63\x0f\x23\x1f\x97\x1e\x69\xa9\xca\xa9\x9a\x21\xc0\xa9\x61\xf6\x8f\x63\xa4\xa9\x68\x0d\xd7\x82\x44\xca\x40\xfb\x3c\xe4\x37\x3e\xe4\x02\xc1\xcd\x15\xb8\x1e\x14\xe7\xc1\x2c\x6b\xb2\x4b\xce\x64\xc7\x4b\x65\x44\x3f\xf0\xe3\xdf\x97\x88\x08\x72\xac\x02\xc7\x40\x38\xc7\xac\xfc\xcd\xa4\x48\x3e\xcd\x61\xbb\xba\x84\xe9\xfc\xcd\xe8\xda\x83\x9e\x2f\x11\x88\x3d\xcf\xf2\xb4\xd8\xec\xef\x09\x8e\xb9\x1b\x4e\x65\x65\x19\x49\x81\x8b\xb1\x8e\xe2\x26\xdf\x27\x9d\x89\x55\x31\xe9\x24\x80\x47\xf6\xa9\x8b\x8c\x27\x52\x90\x98\xb4\x7b\x68\xd2\xd4\x30\xce\x0d\x42\xc8\xfd\x44\x9d\x77\xfc\xb4\xdc\xf5\x80\xd0\xe3\x6a\x6d\xaa\xa1\xfa\xd0\x9a\x62\x34\xa6\x39\x03\x66\x9a\x4e\x07\x19\xa2\xc4\x27\xaa\x97\x55\x23\xd5\xad\x79\xc3\x1a\xf9\x1e\xce\x4d\xc6\x00\xd3\x2c\x55\x29\xb0\x9c\xba\x9c\xd9\xb5\x2a\x2d\xbd\x6f\x89\x66\x06\x86\x3e\x23\xd1\xce\x9d\xce\x34\xb3\xf5\x42\x64\xbb\x7d\x35\xfd\xf4\x44\xa7\xf5\x45\x55\xdd\x4d\x2a\x06\x85\x51\x46\x5a\xe6\x60\x93\xa1\x98\x67\x7d\x9f\x61\xdf\x6d\x07\x4d\x98\xf4\x4e\x7d\x6f\x83\x11\xda\x0f\x97\x4f\xc6\xfb\xa5\x61\xe1\x31\xa5\xac\x4f\x16\xcb\xcf\xd4\x8d\x1d\x23\x20\x8b\x03\x51\x4e\x3a\x3c\xdc\x90\x53\xd1\xdf\x35\xf5\xf6\xaa\x37\x79\x8c\x1f\xe3\x90\x55\x12\x1c\xb7\x3d\x3e\x56\xed\xf6\x0e\x06\x4a\xd0\xaa\x9d\x41\x36\x2d\x81\xa4\x0a\x75\x02\x75\xc5\x0c\x16\x15\x82\x01\xaf\x88\xad\x37\x27\x83\x3a\x48\xf7\xc0\xf1\x7d\xce\x83\x56\x27\x9e\xeb\xdb\x65\x8e\xd6\xf4\xe9\x36\x27\xde\x72\x3e\x0b\xa4\x18\xe8\x2d\xab\xe9\x0d\xcb\x19\xba\xc4\x3e\x56\x4e\x38\xda\x8b\x01\x3b\xfe\x3e\xdd\xc8\x18\xf0\x4a\xb0\x98\x9e\xf0\xab\x21\xaa\x8d\xe9\xb3\x0e\x1c\x0c\x39\x07\x01\x5a\x43\x2d\xe4\x21\xb9\xce\xec\xb2\x82\xa1\x90\x1a\xad\x30\xc4\x36\x57\xc3\x08\xdf\xed\xb2\x84\x7f\x9b\x0d\x0e\x25\x60\xd0\xaa\x91\xcf\xc6\x77\x35\x9c\x97\x32\xa2\x93\x05\x80\xed\x24\x93\x69\x06\xf5\x69\x6c\x00\x45\x87\x8e\xe4\xee\xea\xdb\x22\xce\xb9\x01\x59\x98\x0c\xf5\x74\x84\x1b\xf5\x10\x59\xdd\xa1\xbc\x8f\x86\xf9\x7b\x2e\x75\x34\x64\x7b\x8d\x32\x40\x6f\x3f\x85\x38\x13\xff\x38\xb3\x40\x78\x7f\x34\xed\x22\xbd\x11\x4c\x94\x7b\x2e\x6b\xa8\xc0\x4f\x13\xbe\x1c\x01\x2c\x8d\xba\x41\x1b\x2d\x73\xea\xa1\x82\x64\xee\x91\x63\x89\xcd\x86\x25\x01\x4c\x2f\xfa\xee\x1d\x19\xb2\xc8\x54\x35\x65\x3b\x2a\x60\x02\xf0\x9f\x25\x13\xde\x8b\x1c\xc8\x24\x94\xb1\x52\x4c\x9c\xe6\xae\x86\xfd\xd2\x23\x69\x54\x02\xed\x5e\x2a\xc2\x0b\xb8\x23\xb7\xde\x40\x77\xd4\x38\x8a\x3c\x5a\xd2\x20\x68\xd8\xd1\xd8\x5d\xf3\x90\x41\xdf\x06\xad\x2b\x93\xa4\x17\x8e\x88\x96\x3a\x6c\x20\x48\xcf\x30\x3f\xbd\x23\xfb\xbd\xc0\x16\x5d\xbe\xbc\x96\x7c\x60\x7a\x72\x60\x00\x9c\x61\x2d\x51\x53\x0e\x1b\x71\xf6\x5f\xe5\x30\xc9\xd5\xa7\x68\x2b\x6c\xe3\xd4\xd0\x20\x55\x2e\x7a\xa4\x48\x88\x7d\x7a\xca\xc7\x40\x0a\x31\x88\xa7\xbd\x40\x59\xc5\x93\x08\x5a\x4e\xde\x73\xf0\x6b\xb3\x39\x38\xf6\x60\xc2\x4a\x2e\x1b\x18\x87\x88\xb9\x53\xbc\xf9\x5a\x04\x86\xbc\xa1\x48\x57\x6b\xe0\x81\x73\x78\xc0\xaf\xb5\xc5\xb0\x9e\x66\x46\x80\x2e\xdf\xe0\xef\x75\x4c\x90\x24\xae\xfd\x6b\x24\x09\x75\xc9\x90\x24\x4a\x1a\x35\x7a\x4b\xe1\x4b\x00\x7b\x88\x2c\x64\xe1\x8f\xc5\xf0\x1f\x79\xa6\xcb\x75\xc3\xdb\x04\x59\xf5\x9a\x8e\x09\x51\x24\xcc\x78\x3a\x47\x8b\x72\xa3\xd9\x7f\x4c\x3e\x1b\x05\x21\xd4\x4f\x13\x8b\x49\x3a\x7f\x64\x27\x6b\x0c\x95\x00\x78\x14\x2f\xcf\xd1\x16\x08\x12\x73\xb2\xc8\x3b\x72\x82\xd5\xa8\xca\xba\xee\x4a\x64\xc0\xd4\xd7\x8b\x21\xd3\xeb\x22\xf5\xbd\x77\x1b\xc9\x47\xc9\xf6\x84\x25\x46\x5f\x68\x28\xca\x66\x86\x30\xa4\x81\x95\xd3\xb1\x5a\x1e\x9a\x8e\x87\xec\x3e\x1c\xe3\xb9\x13\x8c\x9d\xe0\x44\xb4\xa6\xcc\x4e\xd7\x78\xe9\x1d\x9b\xf2\xe9\x07\x1d\x17\xae\x95\x9b\x43\x47\x98\x71\x82\xc1\x22\x68\x86\xa9\xd6\x1b\xb2\xa0\x2c\x0a\x7b\xd8\x13\xe8\x33\xcf\x51\xcb\x45\x85\x33\xda\x97\xe4\xda\xb0\x0a\x6b\xdc\x8f\x9b\xb4\xf6\xfd\x88\xb0\x08\x7c\xda\x34\x65\x4c\x4d\x8d\xac\x63\x18\xb4\x21\xd8\x05\x56\xc5\xdd\x18\xbf\x18\xab\x6e\xe9\x18\xa5\xcd\x45\x62\x8e\x45\x5e\x91\xb3\x70\x8a\x12\x57\x15\xca\xf2\x3d\x39\x3d\xcd\x57\xa3\xd9\xf2\x66\x54\x1e\xd2\xcf\x79\x3a\xf8\x25\x1b\x7e\x73\x4e\x92\x52\x0b\xb8\x0f\xa6\xeb\xcf\xe5\xa1\x04\xce\xab\x6f\x50\x9c\x8a\xb3\xa0\x08\xd4\x52\xf4\x67\x51\x32\x6b\xbd\x58\x6e\x80\x70\xba\x59\x6f\x2e\x16\xeb\xf5\xe2\x76\x33\x9b\x5c\xad\xc9\x09\xea\x78\x5a\xc1\x9d\xf8\xb9\xfa\x08\x17\x2b\x64\x9c\x23\x61\x41\x57\x42\xda\xdf\x3b\xb8\x1c\x4c\x46\xc3\xac\x4b\x42\x64\xb7\xa3\x15\x10\x5e\x90\x87\x03\x28\x33\x76\x77\x8b\x26\x8c\xd7\x65\xcd\x17\x2f\x91\x64\x90\x8c\xfe\x9d\xbb\xdf\x64\xfb\x2a\x0b\x07\x51\xd0\xc0\xf2\xd3\xc3\x25\xfa\x7b\xcb\x4f\xd9\x60\x74\xf0\xf7\x7f\x1d\x76\x5c\xd1\xd5\x64\xf6\xea\x2e\x72\xe0\x4c\x7e\x70\xb3\x32\x2a\x2f\x25\x70\x29\x64\x70\xe5\xfd\x0f\xaf\x9f\xfe\x37\xd0\xcc\xb3\xc5\xe5\xaf\x40\x05\x5f\x56\xd5\xe9\xcd\xe2\x63\x79\xaf\x53\x04\x94\xf3\x45\xb5\x98\xc1\x72\x26\xf9\x87\x69\x35\xbd\x98\xc2\xb0\x3e\x17\x09\x3b\x8c\x41\x47\x31\x54\x4f\x50\xc3\xab\x05\x4c\xc9\xec\x6c\x35\x9a\x57\xb0\xf5\x6f\xcb\x7b\x58\x5d\x58\xf7\x53\x5c\x8d\xf9\x75\x71\x94\x5f\x01\x38\xfb\xcb\x04\x27\xbd\xf8\xee\xe8\x88\xca\xf0\x8e\x2c\x07\xc9\xd9\x62\x09\xdd\x7c\x8b\x89\xf0\xfb\x03\xad\x09\x04\x5e\xc2\xaa\xa0\x6c\x56\x55\xbd\x59\xc1\x9e\xfb\x84\x88\x5f\xf2\x17\x72\xae\x02\xa9\xaf\xe1\xdf\x4f\x8b\xbf\xc3\xdf\xdb\x2a\x19\x36\xe8\x07\x42\xb9\xc8\xbe\xe4\xd7\xbf\xfd\x30\xdb\xd9\x94\xc1\x5d\x09\x17\x6f\x5e\xf1\x7e\xbc\x45\xa9\x38\x7a\x06\xea\x59\x39\x1c\xbe\x2a\xd8\x9c\x38\x01\x45\xce\x5e\xba\x8d\x2c\x50\x1e\x55\xdf\xc8\x3c\x7a\xcd\x1f\x3b\x54\x4c\x4c\x07\x00\x56\x4e\x99\x14\x06\xe2\x3b\x84\xe8\xed\xde\xb1\xf4\xc3\xe8\x63\x3b\x17\x57\xd4\xe1\xbd\x06\x39\x26\x2a\x52\x1b\x59\xd1\xd4\x08\x3e\x59\xb8\x2c\xb5\x27\x8b\xc7\xc7\x28\x60\x0f\x5b\xa6\xe1\xe9\x1c\xa3\x7f\x9c\x8e\xe5\x79\x8e\x18\xef\xb0\x6d\x9a\x9e\xf8\x83\x9c\xe8\x30\x70\x71\x7d\x3d\xb3\x4f\xe0\x6b\xb2\x36\x8e\x13\x8f\xae\x3e\x4b\xe7\xf0\xd3\xcb\x85\x63\x8e\x07\xe4\x4c\xb2\xfb\x14\xcb\xf4\x29\x5f\x31\xad\x7e\xa4\xfd\xcb\xed\x39\xc5\x38\x7e\x83\xc0\xce\xa4\x3a\x17\x12\x87\xfd\x4e\xf1\xd5\xa1\x81\x2f\x52\x55\x3f\x2e\x16\xbf\x56\xc5\xbd\x80\x9b\xe2\xbe\x66\x12\x27\x47\xed\xe9\x3b\xb4\xd8\x8b\x77\x83\xff\x50\xa9\x79\x06\x69\x42\x9d\x49\x35\xde\xbb\x69\x42\x56\x43\xd7\xfd\xe4\x38\x41\xa9\xf4\x2d\xda\x65\xc3\x63\x76\x77\x7b\x81\x96\x73\x2f\xe1\x78\xde\xce\x9f\x2c\xee\xe6\x6b\xd1\xaf\x9f\xbd\x96\x9e\xec\x05\x67\x0d\xbe\xf0\x45\xf1\x47\xf7\xb5\xf0\xd9\x16\x2b\x80\xa2\x73\x52\xb7\xfd\x38\x1d\x2f\x3e\x52\xe8\xef\xa4\xd5\x4f\xa1\xc5\xe2\x96\x0c\x66\xd3\x01\x5c\x2c\x61\xb4\xc9\xd5\x6c\x31\x5a\x27\xb1\xc0\x22\x64\x78\x8e\x09\xfd\x44\x43\x49\x91\xd0\x76\xe3\x8f\x2d\x6f\xd7\x62\xd7\xf1\xca\x61\x5e\x57\x23\x9a\x27\xc6\xa1\xbf\xad\x73\x35\xfe\x58\x8f\xa2\x4f\xaa\xd9\xcd\x2a\x5f\x8d\x37\xb8\x36\xc4\xcf\x23\xe9\x5f\x87\x5e\xdc\x4e\x66\x44\x39\xd1\x16\xe7\x3e\x95\xbe\x12\x32\x04\x6f\xf3\xcb\xb0\x07\x5a\x11\x5a\xe5\xdb\x99\x56\xc2\xf5\x3c\x5e\xac\x30\x96\x50\x7b\xaa\xd2\xf5\x01\x68\x13\xea\x94\xa9\x9b\x36\x10\x33\x18\xbd\x78\x85\x46\xbb\x6a\xf3\x9d\x8f\xca\x54\x5f\xbb\x9d\xc0\xc6\x03\x84\x57\x3f\x49\x2b\x81\xc2\xc4\xf0\x93\xc7\x4c\x99\xe0\x2c\x00\x0b\x90\xb3\xa0\x5e\x72\x27\x7a\xe2\x15\x32\x78\x37\xca\xfd\x63\xa2\x18\xe0\x64\xb5\x07\xbe\x7f\xec\x73\x2e\x62\x4b\xdc\x57\xcc\x81\x34\xe6\x71\xf6\x0d\x86\x1e\x0d\x3b\xcb\xd1\xaa\xe2\xad\x90\x36\x42\x9b\x8c\x96\xad\x4c\xe6\xb4\xbb\xc9\xc3\x08\x4b\x0f\xc8\xdb\x95\x26\xb8\x4e\xa0\x9b\xa8\x57\xee\x1d\x39\xd5\x74\xb1\xff\x6a\xa7\x93\x0f\x4c\xb0\x84\x54\xac\x53\x26\x4b\x74\x7c\x15\xef\x64\x80\x30\x2b\xe2\x52\x11\xbc\x46\x64\x79\xcf\x75\xe3\x48\x8c\x3a\x7a\x7b\xda\xf8\xbc\x81\x92\xac\xf3\x71\x42\x3a\x2d\x7e\x36\x4b\x58\x93\x9b\xc9\x6a\x8a\x98\xbc\xae\x54\x15\xae\x14\xcf\x16\xaf\x55\xa5\x6b\x65\xcf\x43\x16\x2c\x3b\x7b\x43\xb1\x4d\x50\x5e\xfb\x10\x4d\x47\xb5\xe9\x94\x51\x7d\x7a\x67\xf0\xbb\xed\x1d\xb5\xf5\x75\x47\xc5\x19\x54\xfe\xa7\x9d\x0e\x7f\xea\xfe\x09\x47\x64\xc7\x49\x40\x4f\x20\xf1\x49\x38\xd2\xa9\xc5\xc1\xdb\xe9\x95\xfc\x16\x28\xd3\xcc\xc9\x9c\xa1\x1d\x7c\xc2\x70\x70\x0f\x92\x47\x10\x4c\x45\xe4\xbe\x8e\xfb\x68\x55\xb5\x04\x71\x74\x92\x13\x70\xa7\x7e\x6c\x36\xf4\xd3\x87\xed\x7e\x5b\x9a\x53\x02\xe5\x33\xee\x29\x99\xbd\x32\xfc\x5b\xd8\xcf\xb0\xab\x2e\xb1\x00\xda\x92\x47\x2e\x17\x7a\x28\xc9\xf0\x0f\x80\xd9\x8f\x56\x32\x92\x46\x21\x12\x42\xee\x35\x8e\x9f\xb1\x1d\xbc\xe4\x97\x8f\xd9\x18\xb0\x19\xa2\xc6\x75\x58\x52\x2c\x83\x24\xd9\x6d\x7e\xc9\x38\x22\x8f\x23\x4a\x29\x23\xe0\x84\xee\x38\x7d\x02\xe4\xc7\x08\xea\x10\xb6\x8f\x26\x71\xb3\xc6\xf6\xea\x95\x6a\x07\xac\x07\x0a\xb8\x91\xc5\xaa\x1c\x2c\xed\x13\xb9\x5a\xe9\xc4\xf6\x53\x87\x6b\xc5\x2e\x41\xa4\xfc\x8e\x82\xb2\xe6\x00\x76\x50\x4f\x8f\xb6\x41\x83\xc7\x9c\xfc\x5d\x78\x91\xc3\xbd\xb9\xbe\xc9\x6f\xa7\xf3\xbf\x70\x60\xf4\x89\x03\x9a\xab\x74\xf9\x89\xd7\x1c\x20\x81\x34\x41\x92\xda\xd7\x00\xf6\x4b\xad\x90\x92\x9b\x0e\x55\x98\xd3\x0c\x3c\x19\xfe\xa1\xbc\x7e\x8d\xa9\x9d\x15\x29\x80\xa6\x03\x10\x88\xad\xd0\x49\xd2\xd7\x72\x2a\xe0\x2c\x1b\x55\xbb\x08\x73\xcc\x0c\xed\xc2\xdc\x05\x7c\x3e\x6a\xb7\x85\x28\x0a\xb5\xe4\x68\x86\x4a\xaa\xa1\x1b\xce\x96\x44\xd6\x26\x4f\xe3\x75\x2e\xc3\x6c\x51\xaa\xad\x19\x87\x18\xcc\xaa\x34\x68\xb3\x7c\x6c\xaa\xd2\x75\x21\xaa\x5b\x03\xac\x57\xb8\xf5\x8e\x89\x62\x17\x82\x5d\xd8\x30\x28\xbb\x42\xeb\x0b\x83\xfe\xd2\x1e\x64\x8e\x9f\x29\xf3\x1b\xb6\x1c\xd2\xaa\xf9\xaa\x82\xff\x91\x3e\xfa\xc7\x77\xda\x57\x6f\x24\x35\xa3\x8f\x9e\x9f\x78\x7c\xe6\x7c\x0a\x42\x60\x62\x76\x6c\x8e\x3d\x47\x81\x87\xfb\x03\x87\x23\x8b\x28\x23\xe3\x2e\x90\xce\xe8\x2d\x5f\xc3\x32\xd6\x72\x85\x2e\xc4\x2a\xcd\x87\x51\xd8\x3c\x47\x94\xb5\x39\xa5\x68\x41\xf3\x38\x4b\x82\x88\xf1\xe9\xf4\xef\x13\x35\x4b\x07\xe8\xf5\xe4\x96\x10\xec\xdc\x8d\xa2\xbb\x04\x4a\x73\x86\x95\x77\x10\x45\xb0\xe5\x9b\x1b\xe6\x88\x4c\xe0\x3a\x21\xed\xa3\xbb\xf5\x82\xf1\xf6\xf0\x6d\x7b\x90\xdc\x4c\x84\xca\xa5\x8d\x08\xd4\xad\x97\xaa\x11\x97\x5c\x8d\xd7\x5f\xf9\x00\x7d\xa1\xd8\x73\x74\xee\xfb\xfa\xfc\xd1\x5d\x5c\x5d\x01\x7e\x41\x3b\x19\xe6\xcf\xb0\x38\x78\x25\x62\x94\x2c\x91\x1c\x89\x23\x92\xba\x98\x59\x1a\x65\x66\x41\x5e\xa7\xe7\xae\xa5\x89\xd7\x2b\x26\x36\x62\xf4\x03\xd5\x19\xbe\x98\xc7\x39\x42\xae\x6a\xc3\x0d\x28\x05\x92\x3d\xe2\x13\x46\x31\xf4\xd8\x65\xf7\xbc\x73\xe8\x3f\x11\xa7\x8c\x1f\x26\x8c\x0a\xd6\x2a\xeb\x8f\xee\xe8\x8d\xe9\xc1\x8e\xc5\x78\xe2\xc5\xe2\x13\xec\x23\x40\x8c\xd1\xfe\xe1\x02\xbd\x64\x1d\x40\x94\x79\x5d\xf2\xb3\xe9\xb2\x26\x86\x6e\xd7\xdf\xe2\x88\x98\x5e\x71\xf5\x42\xa4\x05\x18\x15\xed\x06\x4d\x79\x68\x3f\x78\x3b\x90\x92\x99\x97\x39\xf5\x17\x42\xed\xa8\xf4\xeb\x87\x47\x94\x94\x3d\x38\x90\x18\x76\xe0\xd1\xef\x1e\x1d\x7f\x63\x10\x16\x66\x42\x75\xf7\x8f\xb3\x4e\x92\x14\x6e\x0b\x22\xf5\x9a\x24\x3b\x57\xd3\xac\xa3\x81\x3b\xb9\xed\x47\xfd\x58\x2b\x05\x5b\xd6\x70\x22\xae\x13\x4e\x1f\x72\x07\x53\xcd\x96\x74\x8e\x8f\x8e\xbe\x61\xa4\x3f\xc9\xb0\x3f\x62\xbe\xa7\x0c\x81\x76\xc3\xe8\x37\x9b\x2a\xf8\x4a\x92\x1e\x47\x20\x69\x5c\x1e\xe7\xdc\xe2\xe3\xf2\x18\x93\x4a\xa1\x13\xd0\x42\xa5\x7d\x66\x5c\x4d\x6f\x53\x71\x4a\xef\x7d\x2b\x62\x0f\x49\x3c\x46\x80\x69\xcc\x47\x45\x0b\xd4\x4d\xf1\x69\xc2\x55\x25\x02\x6d\x84\x32\x09\x47\xb2\xd7\x30\x14\x47\x9e\xc8\x77\xc9\x7d\xe0\xad\x21\x59\xfa\xcd\xdd\x94\x89\xcc\x0a\x4e\x46\x23\x50\x1d\x8d\xf3\x7b\xb7\x41\xd7\x46\x77\x33\x10\x8b\x53\xe4\xa1\xfe\x44\xe8\x01\xb1\xf9\x1a\x76\xf6\xad\x4f\xfd\xaa\xdd\xed\x36\x59\x0d\x46\xdd\x3b\x06\x25\x8b\x53\x1f\x38\x3e\x25\x73\x91\x07\x7c\x30\x4d\x83\xc9\xd0\x41\x9e\x6d\xfd\x15\x91\x2e\x04\xef\xa1\xd6\x73\x1a\x97\x71\x9c\x00\xf9\x35\xf1\x33\x67\xcc\xbe\x34\x00\x7e\x09\x27\xb2\x0e\xe0\x31\xf6\x41\x00\x5f\x1f\xb2\x3b\xcb\x01\xc1\x42\xd5\xc7\xb7\xaf\xab\x44\x19\x5e\x04\x1e\x5d\xd7\xd3\x8c\xdb\xa7\xbb\xce\x1d\x5c\x3f\x1b\xdb\xcc\x72\xbf\x96\x2b\x3f\x54\x63\x9d\xcf\x4b\x53\xdb\xd8\x2e\xf3\x8b\x1f\x42\x81\xcc\xcd\xf4\x3d\xba\x22\x30\x91\x0c\x87\x21\xb6\x2e\x89\xae\xfb\x89\xf9\x79\xaf\x29\x3b\xd2\x7f\xc8\xad\xc7\x43\x61\xa8\x4b\xcb\x24\xea\xca\xb6\xb0\xa4\x64\xed\xc6\xdb\xe6\x4d\x23\x21\x2e\xb8\x35\x15\x69\x87\xb2\xb7\x7b\xec\xaa\x94\x15\x0a\xb8\xf1\xc6\x43\x40\xb4\x1c\x8d\xf1\x45\x1b\x83\x7c\x8b\x14\x09\x4d\x86\x95\x4d\x5b\x12\xef\x3b\xaf\xee\xae\xe0\xa7\x69\xf7\x60\x72\x87\x93\x61\x1b\xf1\xcb\x5a\x4d\x8e\xdc\x98\x01\xcd\x39\xcb\x04\x69\x3e\x94\xb9\x5e\x57\x75\x93\x48\x54\xaa\xcf\x52\xbb\xf4\xaa\x81\x0e\x83\xb2\x42\x64\xc1\x7b\xdf\xa9\xa0\xbd\xd4\xa4\xbd\x70\x4c\xfd\xc1\x74\xe8\xba\x44\x4d\x90\x24\xbe\x84\x0e\x1e\xb9\xf0\x91\x23\xeb\xb4\xaa\xed\x36\x0f\xe8\x08\xae\x39\xab\x03\x8c\x68\xe4\xc8\x42\x29\x6b\xb7\x7d\x26\x0f\x4a\xab\x47\x47\xe5\xe1\xbf\x3e\x3a\x42\xdb\x05\x17\x2b\xa0\x48\x21\xf3\xe1\xf9\xe0\x7c\x88\x8f\x3e\x4f\xde\xbe\x7c\x0e\x5f\xab\xfe\xf9\x1c\xd3\xd9\xc0\x28\xac\x22\x72\x19\x2b\xd6\xe4\xe7\xb8\x0d\x1b\x35\xde\x90\xe9\xd2\x0d\x99\x1f\xdd\xa0\x69\x53\xd6\xec\x97\x72\x24\xf0\x4b\x85\xd8\x54\x01\xbf\x19\x6f\xf0\x15\x9c\x8c\x1a\xfc\x3a\xf9\x7c\x3d\x99\x67\x87\xd3\x86\x27\x8b\x4a\x15\x33\x1a\xf8\xe5\x92\x9b\x94\xe2\xd5\x48\x9f\xe4\xe6\x67\x07\xdc\xbc\x61\xd4\x2e\xd5\xb6\x48\xbb\x46\x5f\x6e\x49\x08\x48\x9b\x01\xa0\x20\x3a\x92\x9a\xe4\x98\xd3\x2e\x73\xdf\x09\xd0\xaa\x5d\x2c\x4d\x61\x49\xdd\x6d\x93\x7d\x44\x6c\x8d\xf9\x8c\xd8\x15\x0c\x05\x2c\x7c\x36\x2a\xbd\x17\x70\xe2\xa7\x55\x6a\xdd\xff\x23\x11\xe0\xe7\x9a\xb7\x89\x53\x42\x7c\xc5\x74\xc5\x5e\xb4\x8c\x92\x8b\x35\x14\x55\xab\x53\x04\x09\xf6\x1e\x30\xe4\x60\xca\xa1\xee\x67\x30\x73\x53\x31\x19\x22\x4e\x74\x03\x75\x5c\xf4\x0f\x9b\x46\x9e\xc8\x20\x8a\xdd\xff\x45\xef\x45\xc8\xbf\xe9\x1b\x59\x64\xe4\xcb\xd9\xf3\xab\x8b\x77\x4f\x1e\x65\x98\x09\xee\x78\xe7\xc8\xf8\xf1\x37\x36\x6e\xe5\x3c\x39\x5f\x9d\xcf\x13\xbc\x9b\x8b\xdf\x5e\x84\xe5\x44\x0d\x4e\x4a\x1b\xce\x43\xbe\x51\xbe\x46\x6b\x16\x18\x1e\x89\x32\xb0\x42\x28\x14\xcb\x02\x80\xe6\xf3\xc2\x56\xf7\x88\x9e\xd7\x3d\xa9\x2b\x33\x30\xa4\x49\xb3\xc2\xb0\x7d\xfb\x80\x48\x32\x9a\x5e\x0d\xf4\xe5\x69\x58\x4e\xe6\x97\xb0\xc8\x3f\xbf\x7d\x81\xcc\x1b\x80\xf7\xf3\x35\xb6\x02\x58\x27\xe0\x79\x0d\x69\xa2\x80\x43\xbc\x6d\xdf\xed\x88\xf1\x67\x53\x8c\x8c\xf5\xe9\x64\xbd\x46\x13\x52\xee\xca\xb3\x91\x5d\x3b\x0b\x79\xb4\xa0\x23\x34\x34\x2e\x66\xee\xbc\xf4\xf7\xb4\x7a\x33\x1b\x4d\xe7\x6c\x6a\x0d\xf2\x64\xf6\x52\x18\x59\x8a\x0a\x2d\xb7\xb9\xc3\xc0\xea\x38\x32\x90\xcc\x3b\x33\xe2\x59\x47\xde\xd9\x28\x23\x41\xbd\x37\xb8\x50\x95\xde\x17\x23\x81\x8f\x43\xbb\x5e\xb8\x3c\x9e\x34\xea\xbe\x5f\x4c\xe7\x69\xd2\xb6\xe2\xdb\x8f\x8e\xd0\x89\xaa\xb9\x06\xb1\x73\xc9\xc5\xec\x6e\xd5\x22\x0f\x0b\xfc\x17\x1a\xa5\xdf\xc5\xdd\xba\x05\xf8\xff\x18\xfd\xf1\x01\xd0\x69\x55\x97\xab\xc5\x6c\xd6\xba\x9b\x53\xe4\xe5\x6c\x7a\xf9\x6b\x6b\x7c\x31\xe3\xc0\xed\xe2\xae\x9a\x8c\x17\x1f\xe7\x1c\xba\x5b\xf2\x2f\xe2\xb7\x1c\x82\xc0\x4a\x42\x50\x2f\x05\x26\x28\x3c\xc0\xc1\xd9\x64\x04\x19\x2f\x6f\x46\xf3\xeb\x89\x98\x84\x6a\xf1\x19\x6f\xc1\x0e\xa0\x7a\xe1\x97\x3c\x28\x63\x00\xaa\x27\x77\x20\x2d\x91\x40\x02\xb0\x74\x97\x98\xbb\x6c\x27\xe1\x5d\x97\xfa\x26\x41\x16\xb4\x8c\x25\x13\x57\x7b\x0f\x3d\x62\x4d\x7f\x7d\x66\x26\x21\x07\x2d\xc4\x32\xdc\x70\xbd\x5e\x5f\x4f\x98\xdd\x19\x19\xd5\xbb\xc1\x61\xfb\xc6\xae\xe6\xaf\xe1\x1b\x4a\xbe\xbe\x5b\x47\x50\xdb\x4d\x88\x64\xca\xba\x7e\x62\x52\x2a\xb0\xd9\x48\xca\x96\x6e\x3d\xdc\xb0\x2f\x17\x97\x6f\xf0\xaa\xcd\xe5\x83\x80\x1c\x7d\xbc\x03\x7c\xe9\xd2\x9d\x49\xb4\x4f\x9a\x71\xfc\x8a\x36\x2e\xdc\x88\x7d\xb8\x1c\xc9\xa7\xc7\xe1\xbf\x74\xc9\xd6\x2f\xdc\x0f\x87\xe9\xa0\xdf\x1e\x66\xef\xca\xc1\x2f\xed\xe1\x37\x98\x81\xec\x0f\xd1\x3d\xd9\xfd\xa6\x0f\x38\x42\xeb\x7c\x3d\x44\xc1\x0b\x04\x29\x68\x8b\x67\xd5\xdf\x3f\xbc\xbe\xcd\x57\xe8\x00\x64\xf6\x06\x8d\x37\x5f\x2e\x66\x7c\x43\x8e\x2e\x60\xa5\x37\xa3\xe5\x12\xff\x1d\xa0\xe6\x05\x5e\xac\xdd\xce\x01\x5d\x88\x15\x1b\x3b\x9a\xd1\x55\xbb\xf9\x38\x1d\xa3\x61\xd9\x82\x6c\x0e\x2f\x54\x6a\x8d\xaa\xf9\xf3\xb3\xb3\xcd\x8f\xcf\x4e\x9e\x92\x08\xc7\xd2\xb7\x70\x7e\x78\x7e\x08\x31\x77\x2b\x6a\x6e\x70\xfe\x11\x6a\x1e\x76\x8a\x0c\x4d\xdd\x40\x12\xf6\xf1\xb0\xff\x2f\x05\x74\x12\x62\x8a\x14\xad\x1b\x6d\xe0\xbf\xc3\xfc\x1d\xee\x5f\x63\xe6\x10\x3f\x73\x3a\x4f\x6c\x5b\x1a\x10\xa7\x35\x72\xf5\x11\x1d\xa5\xaf\xd1\x6c\xc6\xd8\x42\xf2\xcd\x61\xd2\x65\xcd\xb1\x34\xf9\x46\x94\x13\xed\xd4\x97\xde\x0f\x0a\xd4\xe7\x5f\x6f\x82\x3c\xbb\x6c\xb0\x8c\x92\x2c\x58\x47\xaa\x03\x1d\x57\x06\xa5\x6b\x39\xb6\x76\x23\x94\x38\x1f\xfc\x72\x17\x64\x0c\x7d\x5d\x91\xd3\xba\x70\x02\x8c\x21\xb8\xd5\x8c\x8d\xf3\x54\x46\x71\xae\xc9\xae\x32\x64\x6c\xb7\x69\x32\xf5\xd5\x92\x3e\xe4\x39\x80\xd0\x0b\x77\x98\x18\x51\x33\xe6\xdc\x60\x76\xe7\xd5\x84\x5f\x74\xbd\x41\x66\xa0\x0e\x4a\x1c\x80\x7b\x83\x6b\x79\xbc\x04\xd2\x1e\x23\x19\x61\x8c\xcb\x41\x4e\x76\x2c\x03\x69\x39\x7e\x89\xd6\x25\x86\x4d\xea\x11\xd6\x9b\x19\x40\xee\x2e\x28\x1e\x67\x06\x24\x97\x8c\xb4\x94\x91\xf3\x8f\x79\xa0\x2b\x38\x06\x08\x11\x27\x6b\x2c\xd3\xa0\x09\xfc\xbe\x9a\xbc\x79\x7d\x7a\x96\x88\x7e\x9b\x97\xe8\xae\x69\xf7\xe0\x2c\x93\x62\x8f\x13\xf7\x62\xbd\x1e\x14\xd6\x65\x3d\x1f\x69\x11\x2e\xed\x31\xaa\x88\xb8\xf5\xd1\xb9\x3b\x13\x17\x4e\xfc\x55\x7a\x43\x06\x5e\xe5\x2e\xb6\x7d\x9a\xb0\x7d\x21\xa7\xc1\xea\x91\x80\x6a\x82\x12\xc4\x61\xdd\x59\x68\x8e\x1a\x0d\xe8\x99\x54\xe8\x17\x12\x94\x28\xe7\xe5\x66\xaf\xdd\x76\xbd\x7c\xff\xb7\xbf\xfe\xf8\x16\x5d\x60\xae\xef\xaa\xec\x9e\xba\x44\xb7\x8d\x7b\xa9\xd2\xca\x60\x2f\x52\xde\xae\xad\x5d\x0a\xe6\x94\x82\x4e\xcf\x72\x56\x79\x0f\x29\x71\xba\xb2\xd1\x79\x1f\xd0\xe3\x1c\x26\xd2\x1c\x83\x4f\xa4\x6b\xf2\x49\xee\xf9\x35\xd7\x1d\xc9\x04\xe9\x17\xea\x1b\x05\x74\x3c\xa1\x83\xe6\xbe\x60\xbf\x7c\x06\x84\x87\x80\x7b\x21\x9e\x4e\x21\x3e\x56\x76\x1a\xd0\xa3\x65\x9e\x00\x39\x1e\xf1\x0a\x6e\x27\xeb\x9b\xc5\x58\x5b\x19\xf0\xe7\x30\x3c\x83\x74\xc7\xb8\xe9\xe2\x5e\xed\x34\x49\x83\x99\x33\xdd\x83\xfc\x4c\xee\x8a\xba\xbd\x4d\x35\x92\x7c\xa3\xdf\xd7\xf9\xee\x9d\xc9\xbd\xf2\x7b\xd3\xed\x54\xb2\x0e\x00\x34\x2b\xcd\x63\xa1\xd5\x37\xca\xd2\xb0\x33\xa6\xe2\x88\x34\xa4\x7e\x5a\x8c\xd1\xc3\x17\x79\x83\x9f\xac\x47\xd7\xf8\x6b\xd1\xae\x82\xda\x0f\xae\x32\xa3\xfb\x36\xad\x30\x76\x56\x84\x57\x0d\xa3\xf0\x16\x04\x0e\x8e\x87\x99\xd3\x86\x43\x1b\xf3\x0b\xec\x26\x09\xb9\xc2\xa7\xe8\xcb\x39\x7b\xf4\x7c\xea\x10\x64\x4d\xb9\xc9\xc3\x4f\x07\x1f\x3f\x7e\x3c\xc0\x27\xdd\x03\xe8\x0e\xa3\x9d\xe3\x1e\xe2\x26\x2b\xa4\x42\x7f\x3e\x7b\x7e\xf0\xc7\x44\x04\x64\x51\x6a\xe6\x9b\xa4\xd0\x0b\x82\xfd\xe0\x90\x0b\x9b\xc3\x25\xa2\x83\x09\xdb\x27\xe0\x18\x3e\xda\xe8\x24\x2c\x6c\xef\x76\x96\xb7\x28\xc3\x27\x4c\x7f\x5f\x91\x8c\x9e\xc9\x80\x31\x92\x03\xcd\xc3\x89\x0c\xb1\xb1\x37\x7a\x8f\x75\x62\xe9\x43\x6e\x8e\x5a\x3a\xe4\x9a\xa8\xf4\xe1\xd6\x1d\xb7\xe7\x64\x9c\x8d\x8b\x24\x1a\xf9\xd7\x9f\x5e\x26\xd2\x77\x7b\x0a\xb9\x0d\xc0\x38\xf0\x5a\xc4\x91\x52\x27\x92\x42\x1e\x59\x4f\xe9\x3e\xc8\x69\x70\x2d\x1a\x1c\xce\x2b\x7f\x62\xab\x4e\x92\x88\xe0\xcb\xbf\x9f\xbe\x7e\x25\x89\x38\xce\x20\x0d\xda\xdf\xe6\x57\x40\xa2\xbd\xe6\x87\x62\xde\x08\xba\x48\xe4\x2b\x60\xeb\xb6\xca\xdd\x32\xb6\x47\x8f\x1c\x68\xda\x40\x5e\xfc\x4c\x22\xfa\x74\xf8\x79\x2f\x9a\xa0\x14\x6b\xc0\xfc\x33\x5f\x57\x61\x0a\x34\xe4\x14\x33\xf4\x19\xf7\xeb\x8d\xa2\x0f\x05\x20\xe1\x67\x0b\xf7\x59\xbd\x5e\x9d\x39\x5c\x22\xf5\x48\x06\xdf\xf4\x2e\xe9\xc1\x52\x1e\x19\xe1\x52\x45\x00\x26\xf4\x6d\xfd\x5e\x63\x5b\x74\x5d\x08\x00\x9d\x8f\xf0\xfc\x9d\xd2\x07\x01\x56\x5d\xdf\x4a\xdc\x53\x55\x6a\x85\x40\xc0\xa2\x1e\xeb\x9c\xf0\x6b\x7f\x51\xdf\xca\xd9\x15\xc8\x8c\x75\x95\x71\xe5\xbd\x47\xe4\x2a\x7c\x24\x1c\x6e\xf8\x2d\x1f\xe5\xf8\xce\x07\x78\xe0\x19\xfc\xa0\xad\x75\x14\xc1\x39\xe3\xa8\xd4\x26\x65\x1e\xe7\xf2\xd0\xc9\x35\xff\x23\xf7\x95\x77\x5c\x29\x3d\x47\xbe\x7c\xae\x77\xc7\x68\xfc\xf9\x94\x5a\xe4\xfe\x01\xd6\xfe\x5d\x71\xe4\x47\x4b\x2f\x99\x7a\x53\xc2\x1c\xfe\x48\xa2\xd5\x6f\x35\x39\x95\xab\xc6\x17\x40\x15\x75\xae\xa9\x7c\x74\x04\x17\xf8\xb7\x47\x47\x8f\x39\x62\xb3\xf9\xf6\x08\xdd\x86\xf1\x57\x3f\xad\xba\xd3\x2b\x85\x6c\xe8\x57\x5f\x82\x25\xf7\x0c\xb6\xc8\xdb\x60\x10\x69\xf2\x12\x40\xe1\x81\x96\x00\x54\xe1\xd6\x17\x96\x7d\x66\x81\xe5\x80\x44\xbd\x7f\x7e\xfb\x72\x58\x6a\x46\x5f\x64\x77\x23\x08\x5f\x1b\xeb\xc6\x84\xa6\x3a\xb3\xfc\x51\x30\x2c\xb7\x23\x4a\x38\x87\x66\xd1\x93\xf9\x42\x00\x4f\x92\x15\xdf\x7e\x6d\x91\xf5\xad\x1b\x6f\x61\xf2\xf1\x55\x4d\x20\x26\xad\xdc\xe4\x67\xb6\xb0\xcb\xdc\xa5\x1d\xa5\xfb\xd3\xc4\xd3\x6d\x44\xdb\xd5\x44\xf2\xf6\x35\x3d\xa2\x88\x0c\x5a\xe7\x9c\xbe\x85\x3c\xd5\x75\xdd\xf3\x91\x99\xec\x60\x1d\xc1\x84\x11\x08\xdd\x03\x2e\xb5\x3c\x42\x65\x78\x5e\x05\x89\xb1\xa8\x4b\xd7\xd4\x91\xc6\xc7\x85\x3c\x0f\x6b\x7b\x9d\x24\xf1\xbd\xed\xc3\xfe\x87\x16\x27\x63\x44\x8a\x16\xb3\x0f\x6c\xad\x4c\x2f\x5a\xf5\xb7\x32\xd0\xa3\x6a\xc6\x22\x08\x53\x61\x2a\x40\x6c\xb5\xb9\xbc\xc5\xd1\xa8\x34\x8d\x72\x18\x8e\xe7\x09\x6a\xf4\xf8\xa0\xae\x0d\x86\xcd\x41\xbd\x9a\xae\x26\xac\xc6\x02\x73\xa3\xba\xe6\xb0\x47\x9c\x5b\x07\x21\x91\xfd\x10\x03\x5c\xac\xb0\x58\x9a\xf6\xcb\xcc\x87\xe2\x1a\xda\x41\xc5\x3c\x9f\xea\x30\xb1\xfd\xaf\x1b\xe4\x30\x0b\x3b\x9b\x3e\xd0\xdb\x08\x95\xd4\xba\xa0\x8a\x83\x03\x27\x60\x8e\x8b\xea\x5e\x1d\x58\xd9\x23\x28\x4f\x58\x29\xec\x92\x6d\x8d\x72\x20\x02\x2a\x15\x00\x8e\xf4\x0a\xd3\x2d\x1e\x37\xd3\x24\xf9\x45\x6b\x37\x3d\xb1\xb4\x8f\x0a\xd6\xb9\x1e\xe3\x66\x38\x19\x80\x5e\x3b\x68\x0f\x6a\xe3\x82\x79\x15\xf1\xc8\xee\x96\x29\x20\x69\x7a\xc7\xe4\xd1\xec\x96\xce\xc9\x06\xec\xe5\xbc\x3e\x91\x3e\x9d\x7c\x7f\x07\x65\x8d\xfe\x72\x9c\xc2\xbc\x35\xf7\xa6\x15\x25\x3b\xa1\x76\x9a\xeb\x5c\x77\xba\x76\x5c\xb7\x44\x5a\xdf\x25\x9a\xe5\x89\x54\x58\xa5\x09\x32\x49\x5a\xb7\x93\xdb\xc5\xea\x73\x12\x6c\xed\xca\x6c\x7f\xb2\x32\xb4\x9a\x40\xaf\xaa\xb5\x4c\x54\x59\x8b\x61\x1b\xd0\xf7\xdb\x9c\x2f\x3e\x04\x80\xab\x93\x0b\xbc\xce\x92\xcb\x11\xb4\x82\xcc\x6d\x3e\x58\xe5\xbd\xbf\xae\xe0\x8e\xaa\x01\xef\xc2\xb2\x58\xf9\x7a\x26\xcb\xd6\xc8\xe7\x7c\x24\xe0\x76\x42\x24\xf9\x5e\xb4\x7c\x19\xfb\x17\x0a\xe2\x50\xf6\x8f\x8a\x97\xca\xcc\x61\xf6\x40\xe3\x96\xc9\x7a\x59\x14\x3f\xa0\xb2\x75\x23\x4e\xec\xa7\x6c\xf0\x68\xa8\x95\x47\xc5\xa0\xeb\x51\x89\x6d\xc0\x2e\xa7\x52\xcc\x30\xa7\xe0\x56\x2c\x8d\x84\x53\x51\x35\x3c\x72\xb8\x29\xe8\x37\x0e\x41\x9c\x48\x54\x38\xab\x66\x7d\x9a\x95\x45\x48\xe4\x8a\x04\x5f\xe7\xbb\x3c\x62\xb2\x22\x02\xba\x6c\xc7\x6c\x0d\x6b\x3e\x98\x31\x47\x71\x77\xd2\x66\x43\x6d\x86\x19\xac\x98\xaf\x92\xb7\xc8\x2c\x44\x4f\xc8\x3f\xc1\x89\x25\xc2\xc4\x23\xba\x96\xfc\xd3\x1e\x55\xdd\x5b\xc9\x48\x10\x45\x6b\xf1\x1b\xd7\x97\xbf\x1d\x89\xcb\x3b\xa4\x66\x70\x23\x61\x04\xee\xa7\xc7\xbc\x9b\xc8\x31\x0e\x24\x21\xc7\x19\x93\x7c\x1d\x03\x8c\x1e\x96\x83\x38\x06\xb5\x64\x38\x24\x3e\xf8\xf9\xd2\x18\xcd\x3e\x8e\x3e\x57\x58\xfd\xc0\xde\x22\xc3\xc8\x6c\xdf\x08\x4f\x46\xa8\x0e\x22\x17\xe1\x3d\xdb\x60\x9a\x8f\x66\x74\x6b\x56\xc1\x75\xc9\x27\xca\x55\xa5\x60\xac\xdd\x76\xc1\x2e\xd5\x9c\xba\x0a\xb2\x9c\xd0\x61\xd4\xa2\x70\x31\x6c\x98\x84\x2c\x11\xe8\xfd\x01\x54\xe2\xed\x14\x56\x9d\xfa\xec\x39\x1b\x65\xed\xa2\x01\x44\x5d\xaf\x47\xc1\x2c\xf8\x0b\x5b\x91\x04\x46\x30\x38\x7c\x35\x42\xa7\x0a\x5d\x84\xeb\x29\xa2\xeb\x30\x88\x2e\xfd\x58\x2a\x17\x2f\x7f\xc3\xa5\x47\xa6\x6c\x1e\xc4\x28\xd7\x33\x8f\xe8\xdc\x4e\x72\x78\x88\x60\x8b\x1e\xda\xf4\x96\xe8\x32\xdd\xbe\xd9\xe8\x37\xb3\x03\x7c\x3c\xc7\xe4\x8c\x38\x31\x63\xd3\x8a\xb3\xf8\x78\x40\xb0\xbf\x49\x22\x87\xd9\xec\x45\x20\x25\x25\xb4\x15\xe0\x75\x1f\x51\x5f\x9b\xf5\xd8\x87\xb9\x38\xa4\xe9\x5e\xae\x16\x40\x71\x2f\x6e\x81\x02\x06\xc0\xbf\x8c\x18\x93\x34\x03\x11\x47\x32\x0f\x0a\x95\x7b\xe9\x1e\x95\xd2\x97\xe4\xe3\x21\x1c\xfa\x68\xf0\xed\x36\xa7\x3d\xaa\xa5\x3d\x1a\x6a\xab\x83\x6f\x51\x70\x3d\xb9\x59\xaf\x97\x05\x0a\x0f\x68\x6d\xfd\x3f\x1e\x15\xdf\x7d\xf7\x2d\x0a\xe1\x87\xbc\x83\x28\x7f\xd4\xa4\x2b\x46\x1d\x1e\x93\x01\x4c\x72\x59\xa8\x1c\x06\xef\xdf\xc2\xbb\xab\x92\x7c\x32\xaf\x65\xf0\xd2\x2b\xc8\x6b\xf8\xe0\x94\xa1\x99\x99\x25\x60\x0a\x5f\xa4\x22\x01\x47\x52\x09\x70\xde\xb8\xb9\xbf\x1f\x94\x55\x84\xf1\x3d\x73\xf9\xc3\xf2\xf0\x45\x1d\xa2\x41\x47\x5e\xa8\x89\x51\x9a\x4e\xc7\x4b\x83\x34\x21\x35\xc8\x76\x73\x5b\x8f\x7f\x60\x4d\x7f\x5e\x2e\x9d\x01\x83\x0a\xbd\x6b\x2b\x2b\x7f\xcf\xb3\xf5\xc5\xc7\x17\xfb\xe4\x75\x38\x4c\x49\xdb\x22\x28\x44\x20\x4e\x66\x2f\xcc\xd6\xe1\x65\x93\xc7\x0c\x91\x85\x91\x1c\x59\x3f\x69\x03\x42\xd9\x47\x4f\x76\x3c\xbd\xa2\x14\xcb\x5f\xb4\xd5\x48\x31\x19\x1d\x73\xd3\xba\xe0\xe9\x5c\xa9\x4b\x65\x5f\x8d\x86\xfc\x39\x04\xa4\x2b\xd9\x3f\x7e\x57\x26\x1d\xff\xc4\xd2\xe9\x64\x85\xe6\xec\x7c\x55\xaf\x92\x5a\x05\xd8\xa7\x80\x88\x7c\x90\x0e\x84\x75\x61\x00\x14\x5d\x71\x69\xf2\xe2\xca\xd1\x95\x07\xa7\x53\xa8\x3b\xc9\x1f\xac\xc9\xb3\xeb\x02\xa2\xf0\xc1\x06\x5e\x01\xa0\x3b\x20\xc7\x9f\x49\x73\x69\x18\x4d\xea\x0f\x87\x5f\x4d\xfc\x32\x4c\xb7\x3d\x9c\x7e\x0f\xa4\x4c\x4a\xb6\xbb\x7d\xa9\xea\x00\xb3\x25\x79\x58\x2a\xdf\x55\xe8\x84\x58\x75\x89\x05\x78\x64\xd1\xbd\x12\xa5\x70\x7c\x81\xb6\x29\xc3\xfe\xce\x94\x0e\x3e\xfb\x20\x93\x23\x88\xee\x27\x79\x0b\x56\x54\x98\x80\x9d\xa4\xd7\xfa\x5b\x79\xd4\x3d\x22\xf1\xc8\xac\xf0\x95\x41\xd9\xa1\xf7\x75\x0b\xfb\x51\x59\x30\x3b\x7a\x3e\xcd\x5d\x16\x35\x9d\x53\x75\xd9\x18\x16\xb2\xad\x69\xef\xfa\x4f\xb6\xc4\x16\xd3\x3f\x42\xb2\x64\x25\x4f\xb7\x87\x0f\x01\x80\x90\x5b\x33\xeb\x79\xa4\x95\x62\x12\xeb\x99\x97\x49\xb0\x63\xa6\x12\xe1\x57\xef\xc6\xe2\x78\xcb\x23\x80\x4e\xa6\xae\xa7\x9e\x9d\xf3\x30\x34\xf3\xdc\xad\x1a\x34\xcb\xee\x6b\xdc\x9d\xe3\xaf\xa6\x34\x3d\x77\xdf\xd2\x6d\xb0\x18\xc8\x00\xc6\xc5\x17\xea\x08\x5f\x67\x02\x4e\x14\xca\x19\x29\x87\xca\x8a\x3d\x9a\x79\x4a\x24\x3f\x3e\xa5\xbb\x8a\x32\x7e\x0e\xac\xa4\x9f\x1e\x23\xa9\x90\x83\x18\x22\x7f\x84\x94\x64\xfe\x7d\x90\xcc\xbf\x2a\x2e\x96\x91\xfd\xb6\xd6\xa4\x47\x98\xcb\x01\x4c\x78\xb6\x65\xd3\x1b\x1a\x91\xbc\x5a\xb4\xdc\x14\xfa\x17\x32\xea\x22\x21\xd2\x6c\xa2\x2d\x64\x14\xfa\xc7\xbc\xf0\x15\x01\x45\x42\x88\xf6\x74\xc4\xbc\x7b\x39\x70\xf6\x2a\xa8\x52\xe4\xe0\x16\xbb\x9f\x28\x76\xd5\x1b\xbe\x63\x24\xc4\x19\xb6\x4f\x05\x86\xd2\x74\x4c\x75\x6e\x57\xf8\xe6\x9e\xe9\x9d\xb7\x02\xbe\xf8\x8e\x78\x74\x9f\xd2\x14\xff\xe9\xc0\xa7\x04\xec\x73\x69\xad\xd1\xff\x4a\xc4\x02\x27\xfe\xb5\x54\x61\xb0\x73\x79\x8a\x0b\x86\x6f\x8c\x7a\x30\xd6\x09\x7f\xb7\xd1\xb8\xdd\x99\x48\x9d\x41\x3f\x8f\x13\x67\xf7\xfe\xa6\xb2\x12\x2d\x1a\xbb\x77\x1c\xa1\x4d\x94\xc6\xaf\x90\xf4\x62\xa2\x37\x3d\xe6\x0c\xdb\x75\x9b\xa7\xb9\x5d\x02\x34\xa6\x62\x11\xe5\xe6\x39\x45\x88\xe4\xdf\xb0\xf1\xcb\x59\xd8\x4f\xf0\x2b\xc9\xc8\x8e\xf2\x2e\x6d\x1a\xd9\xac\xf7\x55\x60\xc5\xfe\x9d\xd9\x46\xdc\xce\xce\x67\x72\x67\x43\x85\x03\x7c\xa0\x89\xe3\xd8\xe5\x98\x27\xfc\x52\x83\xb3\xc1\x39\xf4\xe9\x26\xca\xe0\x6a\xa8\x56\x97\x8a\x7d\x70\xc4\x82\x44\x5c\x4a\xf7\x45\x00\x88\x0e\x27\x4b\xa8\x94\xa6\xdf\xd3\x8a\x00\x66\x76\x9f\x4a\x08\xf9\x88\x5c\xd0\xc3\xad\xcd\xe6\x10\x6b\x9c\x8c\x37\x0a\x34\x0f\x05\x0b\x8a\x73\x92\xaa\xed\xd7\x76\x83\x2d\xb9\x70\xb2\xb5\xcf\x5e\x8b\x0a\xac\xef\x54\x62\xfe\x52\xe6\x99\x2a\x71\x7d\xd7\x65\x48\x1f\x1d\x1d\xc1\xf9\x17\xfe\x1c\x4a\x1f\xe2\xda\x86\xe6\x7b\xcd\x86\x08\x3d\xe1\x45\xd4\x9d\x2e\xa9\xeb\x18\x0f\x2c\xf5\x10\x67\x8f\x34\x25\x44\x9c\x73\x31\x1b\x3b\x0e\x0d\x4a\x9d\xad\x10\x64\x2c\xcb\xc3\xb4\xcc\xce\xfb\x69\xbf\x6c\x6f\xf6\xb3\xcd\x79\xff\xbc\x7f\xd8\xab\x83\x10\xca\x5b\x24\x3a\x0c\x7e\x4f\x5b\x6a\x7d\x45\x24\xaa\xe8\x9e\x46\x6d\x9b\xdd\xe5\x62\x99\x66\x9e\xa5\xc7\xa6\x55\x00\x6d\x0b\xb1\x36\x4b\xd0\x0e\xb4\xa2\x21\x6e\x44\xf7\x22\xba\xf3\xb0\x63\xa7\xe8\x8d\x6c\x69\x0f\x9e\x33\x52\xe9\xde\x98\xf8\x1e\x0c\xba\xfa\x8a\x74\x3a\x01\x22\x7d\x5c\xa1\x00\xe4\xdc\xb1\xf0\x9e\xb0\xda\xdf\x64\x45\x43\x46\x35\x43\xd8\xd4\xd4\xc6\x9e\x60\xba\x3c\x91\x8a\x7e\xc3\x7e\x07\xa4\x14\xfe\x92\x3a\x7f\xe4\x94\x4f\xf0\xb7\xbd\x34\xc0\xb1\xd8\x1a\xa8\x13\xce\xf8\xe2\xab\x29\x49\x73\x06\x8d\xca\x3b\x75\x82\xbf\xe6\xe2\x92\x0e\x43\xfd\x3c\x29\x65\x0d\xc5\x4a\xed\xf8\x75\x60\xba\x64\x0d\xa2\x87\x51\x8e\xac\x1f\x45\xa4\x88\x98\x05\x31\x6e\xda\xfa\xd5\x40\x83\xc3\xd2\x84\x3d\x1d\x40\xe5\x90\x14\x48\x3a\xb6\x5b\xae\xca\x3d\x4b\x5a\x34\xd1\x2a\x3a\xfb\x9e\x50\xa1\x72\x24\xe8\x18\xd4\xc8\x38\xae\xdc\x3f\x03\x81\x7d\xfc\xba\x6a\xc4\x05\xbc\x52\x50\xbc\x17\xfc\x2e\x46\xa4\x2d\x98\xc4\x4e\xd2\xfa\x38\xaa\xd0\x11\x21\xed\x2d\x7a\x8a\xaa\x95\x87\xa9\xdf\x46\x88\x73\xc9\x57\xb8\xdd\x83\x25\xbf\x08\x0f\x6c\xf5\xc3\xbc\x29\x32\xec\x73\xd4\x98\x97\x61\xd9\xe6\x01\x37\xc9\x14\x6a\xac\xd4\x1e\x87\x2a\x4c\xa3\x55\x08\x37\x4b\x7c\xca\xa2\x6d\x10\x82\x02\x32\x73\x66\x57\xa4\x61\x92\x1c\xa1\x6c\xf6\x9f\xe9\x12\xec\x77\xf3\x95\x36\xcd\x71\x43\xa5\x76\x50\xfe\xfa\xdf\x1a\x23\x62\xaa\xc6\xc1\x50\xf3\xd3\xcd\xca\x75\x3b\x87\x8f\x53\xd6\xa7\x00\xe0\x0a\x1f\x2f\xc6\xe5\x11\xfe\xbe\x9e\xff\x4c\xa0\x97\x71\x7c\x79\xc7\x3f\x21\x7a\xff\xaf\x2c\xba\x6a\x44\x78\x18\xee\xfc\x3a\xf9\x4c\x44\x00\xfc\x22\xd1\x62\x9b\xc9\xec\x07\x72\x7c\x87\x11\x48\xef\x35\xc9\xd6\x42\xa1\xc6\x96\xfb\xb5\xed\xbc\xc7\xce\x66\x58\xee\xa3\xdd\x16\xcb\x91\x6b\x80\xc6\xa3\xd5\x18\xb6\x07\x02\x69\x8e\x94\x8a\x30\x6a\x5b\xd4\xf2\x05\xd3\xd1\x24\x04\x8c\x7d\x4a\xeb\xe6\x22\x16\xab\xaa\xdc\xdb\xb3\x85\x01\x72\x7d\x9c\xae\x6f\x9e\xac\x26\x63\xd8\xa6\x53\xa0\x41\x12\x9e\x93\x60\xb6\x6b\x6d\x39\x1b\x77\xd0\x66\x54\x63\x1e\x56\xdf\x88\xa3\x45\xa8\xd9\x5e\x80\x9b\xd5\x8d\xbf\x2f\xf4\x51\x5f\xf7\x6d\x33\xca\x25\x94\xa5\x23\xe3\xb8\x0c\xdb\x1c\xcb\xa7\xd8\xaf\x52\x26\x86\x09\xcf\xbb\x0a\x2d\x7d\xdd\x4e\xfa\x10\xd7\x5d\x2c\x27\xf3\xd4\x31\x12\x99\x71\x43\x88\x58\xee\x33\xe6\xe8\x1e\xa0\xaa\x3e\x2e\x56\xe3\xac\x78\xb0\x10\x82\x39\xc8\xc0\xb2\x29\x99\x21\x90\x7d\x24\x84\x80\xb2\x2c\x4d\x14\xfa\x6f\xf7\x1c\xf0\x76\x9b\x5a\x88\x18\xe8\xcd\xb1\x86\x71\x1e\x61\xd0\x9b\x8d\xd2\xdb\xc9\x5f\x0f\x84\x18\x9f\x8c\x0f\xc8\xa2\x34\x72\x06\x1f\x48\x2d\x93\xbf\xfe\xf4\xf2\xc7\xf5\x7a\x29\x09\xc6\x75\x05\x0d\x46\x69\x7d\xec\x50\x03\xa5\x6f\xe8\x7c\x25\x0b\x57\xab\xec\x7e\xcb\xd9\x51\x0c\x2f\x62\xa5\x90\x23\xd1\x0d\x99\x34\xf0\x12\x5e\x4d\x78\x29\x6b\x61\xd2\x9b\x74\xed\xb5\xaf\x41\x2e\x84\x7a\x8d\x36\x8d\x9c\x4c\x9f\x47\x6c\xf1\xb1\x1f\x3b\x64\xd1\x55\x93\xd5\xbc\x07\xf3\x36\x82\xb2\x34\xff\x75\xcc\xd5\xc9\x25\x2f\x96\x35\xb8\xd4\x6e\x0b\xc3\x2e\x00\x2f\x5c\x23\x40\x4a\x1d\xd8\x77\x7b\x71\x67\x78\xb5\x95\x95\x81\x14\xb3\xbb\x63\xdc\xcb\xdc\x5d\x45\x85\x9a\x27\x84\x92\x1a\xdf\xa1\xd2\x2c\xaf\x21\x49\xdc\xb8\x17\x97\x32\x32\x26\x15\x19\xdf\x2c\xe3\x1c\x9e\x43\x20\x32\x01\xbe\x2b\xf8\xe9\xf9\x01\x56\xf4\x20\xd9\x7a\xf9\x04\x05\x86\xc8\xc4\x37\xdb\xb6\x7f\xfc\xe8\xd1\xb7\x4e\x10\xc3\x0b\x27\x3c\x3a\xfa\x0e\x70\x12\x0e\x87\x5d\xeb\x03\x7a\x5f\x7c\x77\xf4\xdd\x56\xda\x44\xae\xca\xd5\xe2\xd3\x09\xa1\xfb\xcf\x3e\x21\x19\x8e\x0f\x13\xf7\x9e\x2e\x50\x59\xcf\x03\x66\xc1\x34\x64\xde\x1a\x69\x1b\x97\x5d\x26\xba\x69\xa7\xc5\xb3\x4f\xcc\x14\x82\x07\xfd\xfa\x46\xeb\x1b\xae\x8c\x23\x15\x8b\x94\x77\x45\xd9\xe9\xd0\x25\xd7\xb0\x95\x52\xbb\x89\xe0\x04\xdb\x4f\xdc\x13\x42\xb9\xf2\x85\x94\x75\x59\xb9\x21\x8d\xeb\x81\x5b\xbf\x69\x33\x3a\x13\x28\x94\xdc\xb4\xcf\x7d\x57\x1d\x61\x85\x2e\xd1\x6b\x04\x92\x3f\x6d\x2e\xdf\x0e\xea\xe8\xea\xd3\xab\xc5\x47\x7a\xda\xc7\x01\xaf\xae\x3e\xad\xbd\x22\x1b\x5b\x59\xdb\xa0\xbd\xb3\x0d\x1a\x38\x23\x91\xf9\xab\x4f\xf3\x9a\x31\xc1\x7e\x21\xf6\x04\x37\x59\x6c\x91\x90\x8d\x11\x1a\x43\x86\xab\xbb\x79\x79\x08\xc0\xea\x6e\x42\x9a\x79\x50\xe5\x68\x3e\xbd\x25\xac\xdf\xb3\xf4\xca\x81\x98\x24\x76\x51\xc3\x7c\xfd\x71\x32\x99\xd3\x0b\x34\xca\x48\x0e\x8c\xce\x23\x1c\x7a\xeb\x7f\x69\x3e\xce\xef\xe6\xd3\x35\x17\x60\xcd\x31\xb1\x28\x8b\x11\x36\xbf\x68\x34\xf2\x98\xac\x81\x2a\x91\xc6\x2b\xa9\x0a\xd4\x28\x4f\xe9\x35\x1f\x90\x9c\x0e\xa7\x6c\x36\x47\x40\xf0\x8e\x66\xc8\x9a\xbb\x1d\x7d\x7a\x01\x3d\x1c\xb1\x70\xc5\x23\xb2\xf2\x47\x15\xb3\x91\xb2\xf9\xb8\xec\xe8\xd3\x12\x75\xac\x34\x6f\x49\x35\xe3\x53\xa4\x60\x8b\xea\x4c\x6c\x72\x0a\xff\x92\x21\xae\x29\x59\xd4\x18\x21\x04\xe6\x8e\x18\xbd\x54\xee\xa5\x53\xeb\xc5\x05\xde\x6c\xa0\xdd\xcd\xe6\xb8\x37\x5e\xb4\xb8\x9f\xf4\x17\x28\xa2\xee\xef\x13\x1e\xca\x21\x47\xe5\x81\xdd\x96\xb8\x2a\xca\xd9\xc1\xe6\xb3\xde\x47\x20\xc7\x91\x4a\x87\x42\xd0\x25\x0e\xd8\x19\x3a\x14\x11\xc6\x76\x1b\x8d\xca\x53\x72\xbb\x7d\x70\x10\x4c\x4f\xb6\xe5\x02\x34\x0f\x7e\x95\xba\x3c\x26\xfa\x2b\x31\x38\x6f\xee\x7d\x8d\xbb\x91\xea\x37\x9a\x0b\x43\xe4\x03\xfe\x39\xef\x10\x58\x68\x3b\x74\x38\xe2\x89\x6e\xaa\x32\x14\x21\x76\xf1\xf9\xbd\x6c\xa8\x22\xd8\x49\x56\x4b\xa1\x41\xc0\x1f\x73\x04\xf2\xfd\x54\x84\xfe\x96\xcc\x90\x2f\xf8\x83\xfe\x1a\x15\xa2\x9e\x2a\xd5\xd2\xb4\x12\xcd\x8b\xde\x7e\xd8\x15\x1b\x67\xf6\xf6\x1d\x51\xca\x9f\xb2\xf4\xe8\x6f\xa7\x93\x61\x0e\xce\x36\xa0\x28\x7f\x1c\x44\x25\x3c\xfc\x24\x8d\x8c\x30\x0a\xa6\xbc\xba\x99\x5e\x19\x70\xb7\xf5\x5a\x2a\x7e\x0e\x1c\x67\x55\x9c\x67\x64\xf7\x12\xe8\x37\x1c\xd3\x7a\x9d\x45\x53\xae\x80\xea\xb1\xbc\x5a\x3a\x90\x25\xfd\xcd\xe9\x6f\x97\x9e\xa6\x89\xe7\x78\x0f\x84\x2a\x9a\x90\x45\x5d\x81\x82\xb3\x4c\x61\xbf\xec\xb0\x58\x45\xb3\x8a\x27\x7f\x32\xaa\x50\x56\x89\x76\x2c\x7b\x5c\x74\x46\x7d\xc5\xe3\xbe\xce\x64\x2e\x56\x20\x31\x7f\xc9\x3f\xe8\x11\xe9\x23\x59\xdf\x60\x59\xfc\x50\x5e\x8a\x23\x79\xa7\x8a\x0a\xe9\x47\x01\x2f\x04\x20\xb8\x3e\xd8\xb6\xd8\x0f\xfa\x70\xbb\xfc\xcb\x27\x9d\x0c\x0e\xc4\xfc\x24\xb6\x71\xe6\x66\x66\xc9\xba\xcc\x6e\x18\x4e\x1f\x5a\xac\x98\x39\x03\x66\x7d\x6f\xca\x8c\x7d\x11\x47\x55\x38\x8b\xef\x3e\xcb\x36\x07\xc0\x6c\x8e\xc2\x64\x75\x09\x98\xa1\x40\xd4\x51\x85\x38\xd8\x57\x76\x86\x63\x16\x55\x49\xc5\x4a\x3b\x93\xdd\xf1\x1d\xc3\x01\x67\xcc\x98\xa6\x7d\x60\x56\x62\xa8\x4d\xe7\x8d\x05\xbf\xd1\xd4\xa3\xfc\xb8\x39\x07\x1c\x40\x5b\x01\xae\x51\xaa\x2b\x73\xe0\x57\x10\x00\x08\x76\xaf\xe3\x63\xc2\xea\x00\x15\x5f\x8a\x9f\x36\x1b\xe5\xdd\x52\x75\xfd\x96\x9a\xe3\x0d\x4a\x8e\xa1\x82\x85\xa8\xdc\x42\x54\x5f\x5e\x08\x97\x45\xc4\x49\xe2\xf3\xd0\xc5\xcd\x6f\x8e\x47\x94\x9c\x47\x15\x97\xf7\xce\x61\x78\x68\x2c\x82\x80\x82\x9a\x74\xab\x20\x3d\x54\x3b\xf6\xf0\x7f\xb0\x76\x35\x02\x38\xf1\xf1\x6a\x2b\x81\x4d\x66\xc7\xf1\xb6\x54\x3f\xe5\x16\x76\x5c\x55\x3e\x27\x4a\xa7\xe4\x9c\x17\x28\x62\xb2\x87\x44\x36\xc8\x30\xa2\xcf\x3f\xc5\x51\x56\x34\xf7\x2d\x32\x18\x23\xe3\x53\x7d\x9b\x4f\xb4\x6a\x41\xaf\x76\x27\x49\xe1\xa2\x3e\xd8\x74\xc7\x68\x63\x1b\x86\xa6\xb2\x06\x2b\x84\x26\x35\xeb\xef\xba\x76\xcd\xb4\x70\x10\x76\x57\xc7\x5f\x9a\xbb\xa6\xa1\x74\x99\xb7\x76\xf3\xc8\x36\x63\x5d\xdb\x33\x80\x7c\xcd\x29\x64\x31\xeb\xbe\x69\x22\xcd\x90\xbd\x31\x55\x13\x19\xf8\x71\xfd\x52\xd7\x00\xf6\xe7\xb1\xf5\x13\x44\x31\x01\x37\x44\x24\x13\x7e\x10\xcd\x6c\x32\x74\x45\xfc\x87\xaa\x7a\x3e\x2f\x23\xcd\xdb\xde\x4e\x4d\xdc\x6a\x39\x99\xb8\xfb\xa0\xf6\xaa\x28\xb2\x45\x98\x07\xa0\x7e\xdd\x5c\x30\x26\xf4\xa9\xc5\x66\x05\x44\x56\xd5\xe5\xdb\x6e\x92\x5e\x4f\xe6\xcf\x3f\xb1\x28\x20\xfa\x5e\x6a\x6e\x3a\x54\xe5\x55\x95\xa6\x2b\x20\x56\xce\x16\x45\xd4\xed\xf5\x62\x67\xcf\xc5\x37\x32\x71\xfe\xd5\x4e\x71\x46\x87\xcb\x59\x06\xce\x8f\xd4\x4a\x31\x3b\xa0\x76\xfd\x74\x26\x88\xd7\x8b\xed\xae\x5e\x0a\x3e\x3e\x09\xf1\xa2\x1d\xb9\xf9\x7e\x40\x47\x96\x9e\x69\xfe\x0c\x3f\x45\x73\x9d\xcd\xcc\x00\x04\x85\x02\x8e\x7d\x85\x35\xed\x58\x9e\x7c\xbc\xf0\x98\x5b\x74\x1b\x62\xbf\x4a\x97\xca\x0b\x12\xe9\x87\x6d\x73\xdb\x60\xd6\x33\xb5\xa1\xf6\xe1\xb4\xba\xb1\x95\x62\x85\x70\x00\x17\xcb\x94\x6c\x4f\xa7\x34\x0e\x77\x6c\xd9\x01\x1b\x9b\xb5\xe0\xb2\x64\x82\x29\x2c\xe4\xbc\xf3\xd4\x1b\x32\x51\xb9\xd4\xcc\xdd\xea\x12\xf5\x43\x12\x14\xc6\xeb\xbb\xc9\x2e\x9b\x8b\xb2\xa5\xb6\x8c\x9d\x1d\x24\x6f\xa1\x17\xa1\x4c\x27\x79\x60\x58\xfd\x27\xe5\xbd\x86\xab\xe1\xd9\x7c\xac\x1c\x9b\xc5\x92\xa2\xfd\xf8\xe9\x6a\xf2\xa9\x6a\x5c\x16\x82\x3d\x61\x98\xf8\x18\x6a\x2a\xd5\x1a\x75\xd4\x35\xe1\x34\xb1\xfa\x2b\xf9\x4a\xd3\x19\x1f\xa4\xbc\x6c\xc3\xd7\x88\xa6\xfb\x74\x80\x2e\x2a\xce\x23\xf7\x2f\xcf\x83\xf8\xa3\xb8\xfa\x94\xe4\x83\xa1\x22\x5b\x0d\xce\xee\xc7\x13\x9e\xde\xbd\x23\xc1\xb2\x05\x74\x33\xe0\x82\xbf\x9d\xc4\x53\x9f\x09\x53\xbe\x5e\x62\x91\xbe\x72\x2b\x59\xe7\xb7\x01\xbb\xd1\x24\xc7\x95\x18\x27\x88\x78\xbb\x6d\x3e\x68\xb2\x90\x50\x93\xd9\x4e\x4d\x9a\x31\xc7\x40\x11\xea\xed\xe1\x4b\x95\x21\xc5\xcc\xcf\x3c\xdc\xf6\xce\xea\x5d\xcd\x25\x0f\xc3\x79\x1b\xc6\x38\xf2\xc6\x43\xd1\x5a\x3b\x82\xe9\x3d\xf6\xc8\xce\x8c\x3f\x37\x49\x41\x36\x9a\x2c\x67\x2c\x39\x0d\x13\xfd\x61\xd0\xbd\x91\xbb\xf9\x3f\x96\xb9\x25\x22\xe8\x72\xc2\x5d\xcb\xd1\xdf\x75\x2a\x79\x36\x9b\x3d\x2d\xe6\x58\xd5\x92\xc4\xe7\x8e\x44\xfa\xd0\x95\x2b\x1f\xa9\x1d\xf2\xcb\x66\xbf\x58\x85\x55\xdc\x2a\x0f\xec\x13\xee\xce\x8e\x95\xce\x79\x0c\x34\xbd\x66\xcf\x24\x62\x3f\xb8\x96\xc0\x9b\x69\xd8\xbc\x9b\x84\xc8\xa3\x8c\x7d\xfa\x2b\x2b\x53\x1c\xd1\x9a\x91\x73\x0e\x01\x1a\xb0\x6b\xa5\x70\x34\x0d\xb4\xe9\x03\xf4\x12\xe8\x05\x13\x94\x0a\xea\x31\x1e\x63\xcd\xf2\xdf\xb8\x3b\xc4\x5f\x7f\xe3\x8e\xf0\x36\xbe\x77\xed\x08\xbc\x08\x77\xed\x00\xbf\x55\x8f\x9a\xe9\x5d\x6a\xc3\x1d\x0a\xfb\xe5\x06\xda\x10\x69\xc6\xaa\x30\xcc\x4c\xee\xb6\xa6\x51\x7d\x5f\xcd\x00\xdf\x78\xba\xf8\x38\x2f\xf8\x06\x67\x34\x04\x2e\x70\x8c\xff\x79\xa9\xb1\x84\x95\x48\xec\x19\xbb\x1e\x90\x14\xc1\x5f\xb2\x1c\xaf\xf1\x17\x73\x6f\xe3\x9f\x6b\xda\x52\xfc\xeb\xbb\xb5\x49\xa0\xca\x38\x41\xea\xf2\x69\x52\xdd\x36\x76\x37\xc9\x7c\x87\x07\x7c\x4d\x3e\x88\xeb\x04\xa8\x0a\x73\x2c\xbe\x88\xa1\x50\x86\x1d\x0d\x5c\x09\x01\x01\x57\x13\x63\x50\x0d\x66\x0c\x18\x81\xaa\x5d\xcf\x14\x9d\xa1\x4f\x02\x11\xf6\xbb\x42\x77\xc4\x57\xf3\x76\x5b\x29\xf0\x86\xb7\x77\x2a\x03\x60\x8f\xfa\xa0\x74\x5e\x61\xbb\x54\xd8\x1a\xf8\xd7\xda\xd2\x71\x55\x71\x52\xa6\x79\xdd\xf5\x0d\x23\x71\x04\x64\xe9\x09\x84\xc5\xd5\x55\xff\xa8\xf0\x16\xe5\x65\x6c\x36\x77\xdf\x7e\x14\xf6\x03\xe1\xbb\xa1\x35\xb0\xb7\x55\x3f\x8e\x18\xd8\x12\xc3\x22\x4e\xf6\xae\xd9\xc4\xc8\x3d\x66\x17\xd8\xe9\x82\x64\xf6\x3b\x63\xd5\x2f\x89\x61\xd8\x87\xdf\x68\xae\x1b\x7f\x75\xc2\x73\xfb\x51\xd6\xed\x30\xda\x37\x67\x2e\x8e\xef\xcd\x1c\xb2\xa0\xc4\xb5\xd5\x0c\xb9\x5d\x72\x46\xea\x5e\x06\xed\x27\x76\xcb\x3d\x9a\x5a\x1c\x59\xd6\x9b\xdb\xac\x4b\xb4\x4c\x4e\x4b\x5a\x4b\xeb\xfe\x1e\x25\x85\xe1\x94\x2f\xaa\x74\xf9\x0d\x05\xdf\xbc\xc8\x0e\x1f\x79\xaa\x42\xe0\xaf\x31\x47\xf2\x29\xa6\x94\x89\x90\xf6\xc9\x50\xc4\x3e\x74\x89\xc9\x31\x52\x71\x6b\x04\xe6\xe8\xc5\x05\xa1\x17\xf1\xcd\x03\x7b\x38\xbd\xe9\xf7\x11\x64\x05\x38\x46\x31\xa5\x82\x48\xb9\x20\xf0\xa1\xd9\x45\xe1\xd5\x8a\x61\x8d\x72\xc0\xf2\xe0\x00\x40\x65\x2f\xa8\xd2\x9d\x0e\x22\x5e\x51\xac\x28\xe7\x7e\xf8\x07\xfd\x60\x64\xb7\xd6\x58\x10\x7d\x02\x41\xc7\x3d\x70\xcb\x26\x0d\x10\x8f\x8e\xb3\x78\xab\x95\x9f\x98\x33\x92\x66\xb6\x5a\xf2\xd1\x84\x26\xcc\x8e\xbf\xcd\xa3\x8c\x76\x1e\xe5\x49\x41\x51\x86\x17\x63\x14\x84\x7d\x21\x65\xd3\x70\xfe\x1b\x6a\xcf\x82\x36\x09\x41\xb5\xef\x1b\x88\x2e\xba\xca\xa4\x81\x4c\x9f\x31\xd8\x6d\x70\x1e\x1f\x26\x20\x70\x67\x8b\x8f\xc5\x1f\x8e\x8e\x00\xf8\x56\xeb\x02\x45\xd2\x1c\x97\x84\xfc\x0c\x85\xac\x01\xff\x9e\xf3\x9b\xad\x59\x0a\xb0\x1d\xef\xb0\x67\xa9\x32\x9d\xe8\x8d\x2e\xdc\x5e\x4d\x76\x41\xe4\x2a\xbe\x62\xca\x7a\x9b\xc9\x66\x08\xa8\x49\xb6\x83\xe9\x9b\x73\x6a\xfc\x80\xb2\xc6\x26\xa8\x32\x0f\xf4\x88\xbd\x69\xfd\x7c\x78\x4f\xc2\x91\x1b\x77\x07\x20\xb8\x25\xe4\x57\xb1\x31\x4d\x77\xde\x89\x45\x3a\x55\x23\x8a\xe3\x05\xf9\x0a\x45\xa9\x9d\xfc\x62\xf1\xa9\xbc\x47\x5a\x05\x19\xde\x57\xeb\x02\x66\x9a\xc6\x24\x5e\xd6\xd1\x09\xbe\xf5\x87\x18\x98\x98\x27\xb5\xaa\xc5\x65\xe6\xe8\x2c\xaa\x16\xa5\x39\x63\x49\x50\x9d\x0d\xeb\x17\x9b\xba\x40\x53\xdf\x4f\xad\xcb\xb2\xeb\xc9\xfa\x07\xf4\xc4\x01\x70\xe6\xc9\x6c\x0a\x85\xdf\xc2\xe5\xd5\xe4\xb8\x0c\xd6\x17\x7b\xbf\xbb\x10\x2a\x1b\xc1\x18\x4b\xb2\x56\x8d\x8f\x7e\xd4\xdb\x9c\x86\x0b\x25\xbb\xf0\xdb\xc1\xe7\xc0\xee\x72\x74\x3d\xf9\x6f\x9e\x32\x92\x66\x7d\x46\x5c\x24\xe5\xcb\x64\x07\xda\xdd\xee\x25\xd5\x7d\x86\xe2\x6b\x47\x19\x4f\x18\x56\x84\x01\x5f\xd3\x5f\x9b\x6b\x22\xe3\xe2\x71\x55\x18\x89\x75\x6d\x33\xac\xc8\xed\x69\xd9\x32\xf7\x6e\x25\x77\xb0\xd6\xa7\x62\xb8\x4f\x7d\xa2\x19\x6e\x1e\x1b\x44\xd5\x94\x24\xeb\x25\xf8\x58\x39\xbd\x24\xdd\x28\x67\xef\xd6\x98\x58\x75\xa6\x64\xcb\x64\x35\x99\x91\x02\x7b\xc2\xfb\x05\x10\x56\x18\x32\x32\xc1\xd9\x52\xfd\xdd\x8a\xd6\x59\xde\x53\xd9\x34\x3d\x44\x72\x4f\x4b\x49\x96\x31\xa4\x99\x58\xeb\x45\x06\x57\xad\x7b\xa4\x2f\x2d\x19\x88\xcf\x55\xcb\x41\xc6\x77\x49\xec\xe1\x12\x0d\x3b\x4e\xd4\x7c\x6f\x99\x7a\xe7\x66\x66\x44\x48\x5a\x4c\x3f\x4d\xc6\x36\xce\x03\x4d\xf5\x01\xc9\x7c\xcc\x7c\xe0\x7a\x66\xba\x30\xcc\x1e\x1f\x1c\xcb\xab\xd1\x3d\x71\xfe\x5d\x93\xf7\xdb\x5e\xad\x1f\xfd\xd4\xe6\xd0\xb1\x7b\xab\xbc\x39\x4f\x5e\x69\x72\xe1\xbe\xd3\xc9\x0c\xe2\xc9\xce\x7d\x91\x4a\x09\x63\x95\xdb\xf5\x33\xc3\xb7\x4d\x2d\x5a\xcb\x40\x3b\x8c\x76\x66\x23\xc2\x40\x40\xc7\x68\xa1\x3b\x7d\x9c\x91\x3a\x17\x9e\xfa\x65\x84\xa3\xc3\x74\xa7\x53\x2d\x44\x7a\x97\x31\x54\x0c\xdb\xf8\x03\x57\x8a\x8e\x14\xf7\x3f\x2e\x3f\x63\x63\xfb\xfa\x9c\x06\xfd\xb7\x29\xa6\x0a\x3a\x4c\x32\xc6\x2c\x4f\xd0\x48\xe1\x75\xe2\x3d\x8b\xf4\xb5\x18\x25\x98\xce\x33\x1e\x5e\xe8\x12\xe0\x16\xe2\x28\x8f\x82\x18\xd6\x9e\xf3\x91\x67\x6e\x2e\x54\x65\x61\xb0\x27\x38\x34\x75\xe8\x0d\x71\x51\x73\xfe\xc8\x99\xa7\x2a\x5b\xfd\x01\xc0\xa9\x6c\x18\xb7\x1b\x1f\x38\x9a\x7d\x39\xee\x0f\x41\xb2\x22\xb5\xbd\x91\x97\x1b\x13\x03\x3b\x4d\x6a\x31\x49\x5e\xd0\xcd\x79\xaa\xb5\x65\x10\xbe\xb3\x15\x32\x34\xcb\x1b\x8c\xcc\x66\x73\x95\x65\xc1\xe8\x69\xad\xed\xb0\x6a\x55\xb3\x29\x64\xd8\x0b\x6c\x0b\x99\xd8\xaf\x41\x0d\xb4\xd4\x5f\x51\x05\x6e\x05\x5f\x87\x40\xf0\x85\xeb\xc5\x41\xdc\xad\x83\xda\x64\xb3\x25\x62\xf2\x4f\xb8\xa7\x90\x7b\xe1\x3b\x71\x50\xeb\xd6\xae\x2a\xc8\x97\xa1\x88\x69\xe4\xb6\xbb\x5f\x69\xa3\x57\x9f\x9a\x1f\x5e\xcd\x07\x74\x2a\x6c\x36\x4f\x47\x35\xae\xaf\x2e\x2e\xa9\xaa\x2a\xec\xdf\x31\xdb\xc1\x55\x91\x05\xbd\x6b\xd8\x0b\xfc\x61\xcc\xf5\x7d\x45\xcf\x9b\x48\x7c\x77\x31\x16\x89\xb9\x39\x93\xdc\xdd\xbd\x1c\x2f\x77\xb3\xb5\xa6\x2d\x16\xcb\xd8\x12\x3b\xd1\x05\x00\x93\x0e\xff\xfb\x50\x8d\x4d\x43\xb4\x79\x7e\xa8\x59\x5d\x33\xd6\x78\xbf\xec\x3f\x52\xda\xfa\xa0\x76\x72\x43\x9c\x22\x70\x53\x11\x79\xa3\xea\x43\xd6\x3e\x76\x06\x09\x4e\x0c\xc3\x3f\x7e\x5c\x46\xcb\x5a\xdd\x5d\x53\xa5\xfd\x65\xc7\xab\xfa\x91\x4a\x05\x0e\x33\x41\xff\xa7\x7d\x2f\xd5\x94\x19\x44\x23\x25\x47\x52\x39\x66\x80\xdf\xa2\x9e\xe9\x0c\x29\x94\x2c\x68\x00\x0d\x14\xe4\x2c\x68\x0c\x08\xbe\x1f\x74\xcd\x6f\xa4\x78\x57\x8a\x57\x53\x7c\x13\x3a\xd7\x28\x74\x62\x0b\x71\x90\x12\x73\x4d\x02\xd3\x7b\x5c\xde\x19\x51\x27\x37\xe7\x49\x87\xf2\x89\x4a\x02\x5b\xa4\xc3\x07\xf9\xc5\xdd\x5a\x13\x4d\x9d\x42\x2a\x3c\x23\x1f\x1f\x18\xfb\x2a\x32\x05\xab\x71\x66\x0f\xf0\x81\xb6\xe2\x49\x97\x37\x80\x9c\x92\xdd\xef\x78\xc8\xe4\x8e\xd7\x37\x61\x9e\xa5\x1c\x83\x9d\xab\x53\x77\x5f\x61\x6e\x69\x4b\x9c\x80\xa9\x2b\xc0\xbd\xa3\xbe\x40\x15\x18\x17\x43\x3a\xa3\x28\xf1\xa5\x5d\x49\x33\x62\x3a\x0f\x7b\xa8\x17\x1b\x0d\xb4\x5b\x94\x7d\x83\xec\xdc\x72\x09\x23\xa4\x3c\xb3\xc3\xe2\x4f\x65\xe4\xa6\xb1\x9f\x2a\x29\x50\x43\xee\x89\xda\xbf\x1d\x7d\x62\x64\x12\x9d\x97\x93\xfa\x02\xec\x32\xa9\x0d\xa9\x88\x38\xca\xe4\x65\xf8\x11\xe4\x6d\x88\x0a\xfa\x97\x65\x45\xcd\xa1\x62\x3f\x06\xd9\x34\x43\xe2\x8f\x26\x78\xd2\x8d\xe6\x4f\xfd\xda\x70\x94\xdb\x04\x7d\xb1\xef\x6f\xb4\x25\x35\xc9\x9d\x00\xef\xc0\x8c\x1b\x50\x71\xf7\x7d\x81\xb3\x79\xa2\x8b\xe6\x59\x54\x5c\x1b\x8a\xbe\xe2\x6f\x77\x74\x3b\xb6\x61\xa9\x48\xa3\xd2\x84\x4d\xe4\xe0\xcb\x4a\x83\xb3\x1e\xce\x0c\xfd\x50\xb1\x46\x9f\x87\x23\x72\x5d\x2c\xaf\xf4\x99\xdd\x03\x52\x35\x69\xe1\x03\xd1\x25\xaa\xfd\xaa\x31\xb9\xdb\xe9\xfc\xd9\x6a\x85\x56\xcd\xee\x66\x9e\x83\x1f\xb1\x5f\x6e\x61\x37\x02\x44\x46\x54\x11\x6d\xe7\xb8\x73\x82\xb7\x35\x1b\xaa\x2e\x93\x41\xd2\x91\x5a\xfa\xfc\xd3\x81\xfd\x8d\x0a\xd2\x1d\x2c\xd3\x49\x86\x2d\x34\x41\x78\xbb\x44\x3c\xda\xd4\x70\x3c\x74\xb1\x27\xab\xeb\xca\x9a\x39\xa5\xb7\xac\xe9\xd5\x67\x43\x44\x5f\xbc\x77\x5d\xb4\x68\x2e\x44\xf7\xe1\x0f\x60\x01\x6c\x23\x26\xf5\xb6\x35\x0e\x5b\xe7\xf7\x83\xf3\xea\xfc\x74\x88\x36\x90\x51\x5f\x7b\x5a\xfd\xac\x93\xc2\x25\x13\x37\x49\x09\x24\x4a\x0d\x5a\x67\x81\x5a\xb1\x5d\xd7\x17\x8a\x67\x67\x7b\x32\x29\xa5\x78\x32\xd0\x51\xf8\x96\xcf\xef\xcf\xc7\x9d\xf3\xed\xe1\xb5\xb9\xc0\x50\x18\x57\x1e\x4f\x57\xd7\xc2\xfe\xef\x50\xac\xd8\xb2\x3d\xce\x0f\x8e\x1d\x38\x60\xe6\xfb\xa3\xef\xed\x0c\x09\x6c\xea\x23\x17\xa1\xb4\x09\x03\xc9\x3d\xcc\xcd\xd4\x40\xa6\xac\x0f\x7f\x9a\xa7\xa6\xff\xd0\xdc\x50\xd1\xc6\xb9\xd1\x4a\x8b\xf5\xe2\xdf\x2b\x69\x24\x13\x33\x3f\x59\xae\xf3\x22\xbf\x9d\xe4\x7c\x4e\xf6\x36\x0e\x0f\x49\x47\x09\x79\x31\xd7\x40\x4d\xad\xde\x03\xf6\xb3\xba\x3e\x4c\x3a\x1f\x26\xab\x8a\x9e\x69\xe1\x78\x75\x92\xc3\xfa\x3e\x3a\xf4\xfb\x28\x9f\x96\x8f\x7a\xd3\xef\x63\x48\xcd\xde\x97\xa3\x86\xd1\x98\xd2\xb4\x9f\xf4\xa1\x74\x1b\xcd\x32\x2c\xa1\xe6\xe9\xc1\xa3\x9d\xe6\xe6\xfd\x22\xfb\xed\x39\x05\xa0\xe3\x04\x72\x26\x1f\x5b\x64\xc2\x4c\x97\x1e\x60\x81\x3b\x47\x62\x3d\xfe\xe5\xf4\xd7\x09\x6f\x53\xf4\x47\xcb\x1c\xe2\x8b\xf7\x9b\x8d\x83\xca\x98\x26\xbc\x93\xbd\xe3\x9e\x38\xa6\xc6\xd7\x28\xdc\xbf\x32\x18\x69\xef\xb8\xa4\xc2\x46\xcc\x43\x96\x7e\xef\x28\xdc\xa7\x58\x3d\xd3\xb8\xfc\x85\xb6\x3e\x94\x3f\x59\xe3\x95\xeb\xd5\xe6\x6d\x17\x73\xe8\xe0\x98\x3c\x39\x5e\xbc\xf7\x63\x82\x5d\xfe\x0c\x2f\x69\x88\xcc\xa7\x24\x31\xba\x58\xa9\xc9\x4e\xaf\x3c\x04\x03\xc5\x66\xf1\x29\x36\x3c\x92\x99\x51\x2b\xc2\xef\xc4\xb1\x7c\xe1\xfe\x84\x78\xc0\x50\xb9\x69\xf7\x89\x50\x5e\x3f\x70\xe4\x37\xa3\xea\xf5\xc7\xb9\x7a\x54\x24\x73\x59\xed\xb6\xf6\x44\xec\x21\x88\x1d\x04\xc8\x4e\x1a\x4a\x39\x66\xe2\x87\x5d\xee\x58\x57\x46\xc1\x35\xca\xc7\x5e\x59\x4a\x28\x33\xb1\x69\x6d\x90\xae\xa2\x78\x79\x75\x6c\xe5\x51\x0f\xfe\x7e\x6f\x16\x0f\x3e\x61\x2f\x7e\x75\x27\xa3\x39\xfa\xdf\x1a\xb5\x62\xe6\x76\x0d\x2b\x52\x3c\xfa\x8f\xc9\xe7\x8a\xb7\xa5\xac\x1a\xb2\xe4\x9d\x04\xec\xd7\xb4\x8f\x65\x98\x27\x6d\x9b\xa2\xd8\x8a\xf4\x31\x6a\xdb\xe6\x94\x5a\xde\xb1\x79\x4c\xd3\x55\x19\xf5\x91\x1d\xb3\x7f\x4f\x75\x9b\xe3\xfd\xe0\x0c\xe0\x49\xa5\x59\x60\xfb\x14\xa6\x7b\xbe\x5f\xab\x09\x42\x9a\x89\x78\x5a\xd0\xea\x9e\xcf\xeb\xb7\x1e\x23\x08\x64\x9d\xcd\x67\x33\x0e\x31\xcc\xb9\x9f\x43\x27\x7e\x9e\x8e\x0d\x91\x37\x9e\x5e\x4f\xd7\x02\xda\xef\xa6\xe3\xe0\x59\xb7\x47\x70\x41\x5e\x78\x73\xca\x89\x79\xf4\xd1\x14\x15\xd9\xd1\x02\xd7\xc9\x3a\x05\x7a\xf5\xf7\xff\x56\x96\x94\x45\x79\xad\x3e\x67\x99\x9c\x24\x39\x56\xce\x4e\x20\x12\x92\x84\xf8\xd3\xd1\xde\xce\xfc\x0c\x2d\xba\x57\xab\xc5\xed\x13\x69\x24\xa5\xbc\x9d\xe3\x2c\xac\xc8\x36\x72\x94\x6c\x7d\x55\x4e\xb6\x18\xa2\xc3\x32\x66\xb7\x4d\xd6\x3f\x8e\xaa\x1b\x58\x4a\x5a\x77\xb8\xea\x6e\xe8\x62\xde\xdf\xbf\xe1\xe8\xf2\xa6\x90\x77\xe0\x20\xda\xd7\x20\xac\x9a\x71\x25\x10\xe6\xa6\x84\xa0\xcf\xa8\x2b\xab\x27\xd6\xa3\x0b\x21\x92\x00\x7f\xe0\x8c\x43\xd1\x76\xdb\x42\xb1\xa6\x15\x86\x4c\x74\x86\xc4\x23\x33\x62\x77\x66\x18\x90\x0a\xc3\xc8\xe1\xc7\xc0\x7a\xbe\x2c\xfc\x0b\x19\xf2\xe4\x5e\x70\x64\x7e\x7c\x94\xd9\x9c\xe4\x46\x5a\xd8\x2c\x91\xaf\x47\x19\x2a\x5c\x2b\xa9\x04\x0d\xda\xb5\xcd\xef\x1d\xb8\x2c\xb8\xf8\x36\xcb\x14\x69\xf5\xdb\x6f\x81\x24\xdd\xbd\x69\x91\x14\x17\xe1\xd8\xee\xbb\x76\xf6\x7d\x2a\x0d\xf1\xb9\xf3\x2b\x55\xc7\xf6\x3c\x45\x0b\x53\x61\x6f\x37\x8f\x17\x04\x65\x0d\x7e\x10\xba\x9f\xb2\x65\x9f\x7e\xa1\xe4\xde\xce\x92\x22\xb4\x16\x76\x97\x59\x8c\x14\xd7\xf0\xce\x5d\xab\x43\x2e\xca\xb0\xf1\x46\x97\x59\xb6\x94\xf8\x9b\x0c\x4b\xc5\x77\x69\x7d\xa4\xf8\xa0\x1f\x96\x19\x70\x07\x5b\x98\x34\xc4\xa2\x82\x88\x89\x14\xa3\x00\x94\x18\x8f\xd8\x51\x09\xa5\x7d\x55\x2d\xcf\x83\x9d\xee\x2a\xaa\x13\x26\xb5\x31\x88\x16\x51\x73\x07\x38\xf1\xab\x7a\x60\x10\x9e\x7b\x7f\x33\xf1\x35\xac\xf4\x09\x7f\xa9\x53\x0d\xfe\x1a\xcd\x26\x2b\x49\x30\x0f\x94\xc1\x8a\x5e\xc2\xd5\xd4\x5c\xf1\xfe\x04\xf2\x9e\xb0\x09\x20\xfa\xfe\x48\x58\xa9\x9d\x18\xd4\x99\x31\x65\xdd\xd0\x30\x61\x48\x5e\xea\xc3\x91\x11\xc6\x6f\x2a\x50\x43\x25\x88\x96\xf9\x3d\xc9\xf2\xb6\xf8\xe3\xd8\x72\x9b\x0d\x7d\xe2\xb8\x28\x80\xde\x16\x4c\x4d\xc8\x23\xdc\x8d\x59\xb1\x94\x35\x5d\xd3\x11\xc8\x6b\x80\x63\x2c\xc7\x33\x9b\x56\xec\x30\x02\x0b\xf2\x5d\xdd\x7c\x67\xd6\x0a\x21\xcc\x93\x72\x16\x76\x5d\xce\xee\xc6\x93\x0a\x60\x2c\x6c\xba\xdc\x4c\xd9\xc1\xf1\x5e\xa9\x46\x22\x7c\xa2\x2d\x19\xa7\xf1\xab\x27\x7c\xa9\x71\x89\xcc\x79\xe3\x31\x91\x34\xd5\x3d\xe3\xec\x8e\x30\x7d\xcc\x60\xd1\x00\xca\x86\xb6\xfb\x30\x05\x6f\x7b\x25\x96\x7a\xda\x3b\xdf\x13\xca\xf3\x96\x6c\xa3\x48\x6f\x0c\xef\x84\xaf\xe8\xb0\xb3\x9c\x1c\xd0\x5f\xe4\x94\x84\xbb\x11\x89\x4d\xe5\xd1\xd9\x81\x6d\xf9\x39\xad\x16\x77\xab\xcb\x49\x0e\x13\xb7\x9e\xce\x59\x30\x94\xee\x7c\x3d\x0f\x9c\x4e\x38\x3b\xef\x63\x89\x10\xab\x50\xf3\xeb\x9f\x98\x22\x4f\x2e\x97\x1f\xd1\x89\xc5\x93\xd1\xfc\x77\x6b\xaa\x7a\xaf\xf5\xd3\xe8\x57\xd8\x97\xf8\x31\x9d\x54\x2d\x38\xbc\x5c\x67\x6b\xb1\x6a\x51\x5d\xd0\xe1\x6a\x8d\xc6\x68\x2b\xd8\xa7\x64\xf9\xa1\x52\xd5\xf3\x2e\xe3\x08\x71\xb7\xb8\x71\x98\x4d\x9b\x50\xef\xc9\x34\xea\xc8\x29\x15\x6b\x8d\xe6\xe3\x96\x29\x08\xd3\x3d\x91\x0b\x08\xf6\x9b\xb4\xa8\x00\x4d\x87\x79\x6f\x0a\xc8\xba\x8a\x04\x87\x5f\x74\xce\x6b\x57\xdd\x16\x62\xed\x25\x3f\xd7\x44\xa4\x91\x01\x2d\xc5\x18\x4c\x66\x8f\x39\xe8\xf9\x31\xa9\xcd\xf8\x80\xc8\xa9\xf9\x6c\x84\x1e\x6c\xb3\x18\x5f\x96\x11\xc5\x19\x4b\xdb\x35\x8c\xc8\x7a\x16\x9f\x30\xad\xdf\x38\xbb\x5f\x3e\xb2\x94\xfd\xc3\x3f\xac\x9c\x6d\xe7\xaf\x6f\xf3\xda\x0d\x37\x18\x22\xf9\x4e\x17\x50\x53\x56\x24\x5e\x4d\x22\xbe\x38\xa1\xba\x2d\xb2\x83\x1d\xc8\xdf\x55\x2e\x48\xee\x4a\xae\xc2\xdd\xce\x12\x41\xdc\xd2\xe6\xbe\xdd\x03\xea\xe2\x0e\x95\xc9\x64\x10\xc7\x1b\x00\x50\x8b\x8f\x4f\xa8\xd0\xea\x32\x27\xfc\x0f\xfe\x20\x0a\x47\x66\xb1\xe3\xb9\x5f\x5d\x66\xf0\xaf\x99\x56\x49\xf6\xf7\x51\x5f\x05\x4d\x14\x57\x77\x17\x70\xd5\xa7\x47\xf9\x23\xea\x9f\xa2\x7a\x50\x54\x96\x46\x3b\x65\xf1\xbb\xc9\xdf\xee\x46\x33\xa0\x45\x8e\xf3\xc5\x23\x3a\x25\x0b\x22\xc5\x1f\x29\xdd\x4e\x0a\xa5\x4c\xdc\x97\x8b\x63\x96\xd5\x0d\x32\x1c\x13\x51\x8c\x4a\x97\x8b\x63\xb8\x87\xd0\x6d\x82\x2d\xed\xa9\x7e\xdc\x71\xf8\xef\x74\xb2\xce\xd7\xc7\x4e\x9c\xed\x38\x5f\x3f\x72\x1f\x8f\xc8\xda\x1e\xf4\x60\xfd\x28\x40\x77\x8e\xd9\x58\x84\x23\xfd\x8f\x33\x01\x34\xb4\xcc\xf8\xe9\x38\x65\x1c\x83\x53\xb0\x38\xf6\x2b\x8f\x9d\xf2\x5f\x7c\x56\x65\xa9\xa1\x34\x90\x9c\xee\xeb\x91\xab\x0b\xca\x7b\x4e\x12\x55\xe0\x3f\xb9\x06\xb9\x9c\x8f\x0d\x84\x83\xf2\x96\x03\x72\x1c\x7c\x3d\xb2\xec\x8b\x47\x86\x37\xc2\xd3\x52\xca\xda\x2b\x8d\x7a\x8c\x17\x40\xe2\xd6\x17\x29\x26\xa2\x96\xda\xed\x3d\xcb\x7e\x38\xe6\xe5\xe5\x29\x72\xeb\xc9\xb4\xf2\xe2\x91\x24\x46\x4d\xf1\xd6\xd8\x3b\xda\xda\xf6\x1e\x61\x7b\x7b\x9c\x61\xd7\x6e\x6b\xe8\x8c\xb4\xb1\x17\x58\x76\x0b\x7a\x58\xeb\x84\x6e\x8f\x6d\xb0\xaa\x76\x46\x20\x21\x55\x6e\xd1\xb1\xca\x0e\xd1\x22\x48\xf8\xde\x73\x25\x84\xbd\x83\x0c\x1a\xe1\x48\x7c\xcd\x44\xb8\x3e\x6c\x35\xca\xde\x6e\xe4\x17\x8d\x6e\xc2\xe3\x9c\x7e\x1e\x31\x12\x61\x7c\xeb\x61\x9a\x7a\x50\x23\x86\x26\x63\x1e\x41\x6e\x4b\x2f\x62\x16\x64\x5e\x22\x21\x57\xb1\x72\xf3\x8b\xa0\xca\xa0\x92\x20\x0b\x49\xd0\xb8\x9a\x2e\xc4\x95\xd5\x95\x13\x86\x45\x3f\xda\x9f\x43\x9e\xb2\x3a\xec\x7a\xd4\x0f\x1a\x16\x0a\xf2\x51\x56\x38\x6c\xcb\x2e\x15\x54\x88\xae\xf9\xdc\xe5\x0a\x87\x92\x0f\x46\xff\x6a\x5e\xb8\x56\x94\x2f\x5b\x27\xa6\xe2\xe6\xa1\x98\xa0\x96\xd4\x5f\x5f\x43\xd3\xbc\x69\xe7\xd0\x0f\x45\xb1\xa3\x60\xb6\x6d\x78\x87\xfe\x42\xab\x46\x71\x0a\x12\xa8\x31\x8c\xb7\xac\x0d\xe6\xee\xbe\x65\x6e\xf1\x2a\xf4\x07\xca\x3e\x53\xe9\xb3\xb7\x83\xa4\x62\x96\xde\x3e\x62\xd4\xc1\xe1\xc0\xb7\x49\x7f\x2a\x0a\x07\x09\xbc\x33\xd1\x32\xd9\xff\xcb\x8b\x57\x4f\x5f\xff\x45\x7c\x88\xb6\xdb\x4a\x2f\xa8\xd3\x74\xc9\xf5\xf4\xf5\x93\x9f\x7f\x7a\xf6\xea\x8c\x78\xd3\x04\x6a\xd4\x89\x7b\xca\x19\x4e\x9f\xbc\x7e\xf3\x2c\x21\x44\x2d\x1e\x17\xe1\xd1\x4b\xe8\xfb\xfa\xf3\x43\xc4\x2c\xe4\xea\xfb\xce\xd6\x1f\x03\xf2\x70\x96\xa4\xc6\x7e\xd2\x6a\x25\x05\xbf\x17\x79\xbe\xd8\x6a\x71\x4b\x2d\xa3\x3d\x2a\xf3\x8e\x21\xf0\x93\x62\xfb\xd4\x02\x31\x14\x38\xa2\xc0\xbf\xb6\xef\x3f\xf0\x13\xa4\xd2\x67\x70\xac\x65\x8e\x8e\xf6\x64\x72\x1c\x34\xa0\x65\x2a\x67\x68\x86\xfb\x12\x4d\x36\x27\x49\x47\xf0\x5b\x7e\x46\xdb\x4b\x93\x2b\x18\xe9\x87\xcd\x26\x39\x92\xdf\x2b\x80\x0f\x13\x09\xcf\x17\x1a\x90\xdf\x01\x92\x47\x1f\x18\xd5\x6a\x49\x15\xc7\xbd\x80\x65\xe0\x4f\x36\x9e\x53\x18\xd6\xd9\xe8\x5a\xfd\x0a\x67\xf7\x12\x28\xdf\xff\xed\x25\x50\x26\x2e\xbe\x7b\x39\x43\xc3\xa7\x6c\x3d\x45\x22\xd9\xc1\x1d\x32\x97\x9c\xd1\x94\x2d\x8e\xe7\xec\xd9\x5f\xcf\xde\xbd\x7a\xfd\xf4\x59\xf9\x2d\xbd\x26\xfe\x08\xd9\xb4\xbe\xd8\xfd\x9d\xab\x9f\xea\xe2\xea\x43\x27\xc8\x83\xa3\xa1\x63\xaf\xc3\xde\x72\x95\xf7\xfd\xa4\x69\x23\xfc\x6c\x8e\x21\x31\x91\x7e\xf8\x4b\xfa\xfd\xe0\x97\xc7\xc3\xce\xe3\xec\x30\x43\x07\x0a\xee\x5d\xe5\x97\xef\xd1\x5d\xe5\xf9\xc1\xb0\x93\x1d\x46\x6f\x3f\xb9\xf3\x72\xac\x7b\xee\xfb\xa4\xe3\x1b\x73\xa9\x5b\x33\x6e\xe9\x72\x43\x97\xec\x61\x5d\x7d\x7e\x3a\xd9\xe1\x41\xf7\xde\x0c\x7c\xbc\x2b\x93\x99\x67\x5f\x2b\x6d\x44\x40\x5b\xff\x8b\x3c\xfc\xfe\x2a\x01\xc7\x45\x7e\xe7\xd0\x66\xa4\xf2\xf0\xaa\x8e\x68\x55\x57\x84\x2d\x0b\x8a\xf5\x85\xb6\x75\xe0\xea\x2b\x85\xad\xac\x1f\x70\x80\x5d\xf5\xa5\x46\x6a\xf1\x12\x8a\xe3\x25\xd7\x3c\x64\x57\x8e\xcc\xae\x79\xe6\x13\x5e\xd6\x99\x07\x5d\x41\x82\x14\x38\x1e\x66\xfd\x2f\x55\x0a\x79\x0a\x40\xe1\x94\xf1\xde\x37\x2f\x2f\x7c\x99\xf6\x35\xc4\x74\x0a\xca\x8f\x14\x1a\x55\x0e\x1c\xc3\x1e\xe2\x87\x3e\x1e\xe1\x13\xd0\xe0\x01\xe7\x7e\xbd\x70\x13\xef\x38\xf7\x6c\x1e\xe5\xab\x78\x02\xcc\xbd\x0e\xb8\x49\x7d\xcd\xce\x39\xae\xfc\x23\x21\xe4\x91\x45\xa0\x16\xb8\xeb\xf2\x44\xb6\x9a\xb2\x1c\x09\xae\xf2\xde\x51\xd6\x31\xd9\x45\x9c\x21\x29\xcc\x8b\x9a\x66\xf7\xb9\x48\x7c\x0b\x85\x5d\xbf\xa2\xee\x0f\x5f\xaa\xf6\x83\xad\x91\xcd\xc5\xe8\x05\xc7\x1f\xce\x2d\x72\x91\x24\x06\xa5\xd7\x5a\x4e\x27\xd7\xb7\xb2\xe9\x3d\x23\xb6\xd6\x04\x36\xe0\xcf\xf3\xbf\x3e\xfa\xc3\xe1\xf5\x34\x0f\x7c\x2d\x1f\xfe\xeb\xb7\x4f\x29\xb2\x0c\x22\x1f\xfd\x40\x91\x1d\xcb\x14\x6f\xa8\x7e\x79\xb9\x7e\x46\xb1\xa7\x4b\x28\x57\xc5\x3d\x89\x4e\xa7\x6d\xe0\xbb\x23\x6a\xe0\x7f\x86\x5d\x39\xa1\xc8\x22\xec\xca\x77\x87\xd7\x39\xdc\xbf\x41\xdc\x13\xca\x98\x87\x91\xe8\x58\x3f\xea\x51\x3f\x81\x68\xa8\x30\x18\x88\x3c\xdc\xbe\x98\xc3\x29\x14\x00\x9a\x5f\x2c\x16\x6b\xb8\x10\xd1\xfd\x88\xcf\x17\x82\x5f\x05\xe9\x2c\x4d\x4e\x28\x09\xef\x01\x49\x27\xb8\x0e\x65\x54\xb2\x84\x9f\x7d\x73\xe7\xd4\x7e\x20\xa1\x21\xa9\x9e\xa3\x59\x97\xf9\x75\x01\xf9\x61\x18\xf3\xeb\x03\x0e\x7c\x3a\x70\x41\x54\x30\xd3\xaf\x61\xfe\xea\xcf\xef\x4e\xde\xbc\x79\xf7\xe4\xe5\xc9\xe9\xe9\xbb\xb7\xcf\xfe\xfc\xec\xaf\x6f\xca\xc3\xf3\x6a\x7e\x3d\x28\x00\x40\x43\x9e\xb4\x38\xaf\xbe\x21\x78\x3d\x7e\x07\x00\xbb\xd7\xcf\xfa\xe7\xd5\xa1\xe3\x16\x50\x93\xa1\x18\x53\x46\x7e\xd9\xd5\xc1\x0b\xfa\x9c\xe4\xf1\x3a\x29\x1b\xa0\xa9\x64\x2c\x3f\x7c\x7e\x31\xe6\x32\x59\xee\xdd\xd0\xe8\xd4\xc3\xfc\xe6\xc9\xf9\x39\x2c\x9b\x0e\xb6\x4b\x62\x1f\xa7\xe2\x91\xf5\x64\x86\xfe\xa3\xb4\x27\xbb\xb2\xa4\x49\x97\xe5\x63\x32\xe9\x08\x00\xd7\xaf\x2d\xd2\x91\xe6\xbf\xbe\xe0\x40\x0b\x0e\x7d\x31\x3c\x89\x51\xc9\x48\x6e\x89\xb6\x01\x52\x20\x7e\x99\x05\x49\x9f\x8d\x2a\xf2\xa5\x53\x26\x2d\x38\xeb\xd2\xac\x8b\xed\x40\x6c\xce\x0e\x88\x1a\x16\x92\x0d\x44\xb9\xcc\x19\xbb\x2a\xea\xa7\xbe\x91\x72\x12\xec\xa9\x32\x55\xaf\x46\x7c\x1f\x79\x51\x8d\xaa\x83\x87\x05\xce\x05\xa0\xc3\xd1\x1c\x8c\xd6\x80\x9f\x5d\xdc\xad\xed\x36\xc0\xb8\xec\xde\x8c\xa6\xdd\xe6\x3d\x81\x09\xe4\xf8\x1d\x0d\x93\xee\xee\x07\x65\x53\xaf\xf0\x08\xf9\x6d\x4d\xee\x48\xa5\xb5\x43\xd1\x1f\xf0\xef\x10\x28\x15\x4b\xfc\xb8\x12\x61\x3b\x95\x0a\x89\xfd\xa0\x19\xca\x50\xf2\x79\x07\x42\xe6\xb6\xe3\x74\xfe\x9e\x96\x3e\x95\x9b\x73\x3d\xba\x2e\x3d\xea\x84\x6c\x43\xd9\xf3\xfd\x44\x43\x49\xd1\x84\xfa\xf5\x62\x9e\xe2\x05\xf6\x67\x39\x86\x29\x3f\x59\x2e\x5b\x27\x33\xb2\xac\xd6\x72\xfd\x5c\x4e\xc6\x2d\xb4\xc4\x49\xe2\xb4\x2d\x99\x84\xd6\xef\xee\x8f\xb6\xbf\x4b\x72\xe8\x45\xb6\x95\x11\x96\xf2\x4b\x06\x96\x24\xec\x5e\x2a\x07\xc9\xfe\x72\xb5\xf8\x80\xfa\x9d\x7e\xed\x34\x2a\xbb\xd7\x10\xaf\x44\x9a\xec\xaf\xa0\x79\x69\x2b\xc9\x1d\x78\x1a\x66\xb5\x8a\x01\xf0\x88\x9a\x84\xce\x50\xc9\x16\xcd\x5e\xe8\x84\xe9\x02\x78\x56\x32\x27\xc0\x94\x7e\x58\xfc\x3a\xc1\xae\x61\x6b\x44\xb5\xc0\x24\x84\x4d\x27\xfb\xa8\x10\x38\x25\x3b\x1c\xfb\x5a\x12\xc3\xa2\xc6\x64\x8d\x4a\x63\x05\xda\xd7\x5c\x8a\xe5\x5a\x46\x4d\x47\xa0\x85\x6e\xc8\xd7\xdd\x67\x0a\xd0\x6c\x02\x5d\x69\x52\xb5\xb6\x8d\x69\x28\xd3\x4a\xdd\x5a\x72\x9b\x78\xe8\xa5\xf6\xee\x84\xe4\xe3\xc6\x64\x80\x01\x91\x2e\x29\xba\x45\xe8\xfb\xf4\xd9\xf3\x67\x6f\xdf\xfd\xf0\xfa\xf5\xd9\xe9\xd9\xdb\x13\x00\xbe\xbf\xd4\x23\xf7\x0e\x75\x9a\x58\x7e\xad\xdd\xde\xab\x67\x62\x21\x5f\x11\xb5\x23\x78\xd7\x37\x1b\x1b\x85\xe5\x4d\x5a\x69\xc2\xee\x98\xd7\xeb\x24\x43\x37\x72\xad\xa1\x25\x47\xd8\xc1\x0d\x47\x85\x1e\x5d\x7f\xd2\x13\xe5\xe0\x83\x89\x35\x58\xbe\x08\xd2\xe9\x8e\xa1\x9b\x4e\x22\xb7\x68\x4d\xc3\xf4\x78\xeb\x04\x6f\x3d\xd9\x34\x1f\xfd\x3a\x79\xc7\xb4\x00\x99\x58\x9d\xa0\x23\x21\x5c\x06\xef\xfa\x56\x62\x4a\x17\x02\x60\xf6\x2e\xc9\x83\xa1\x9e\xbe\x3a\xf9\x8f\x67\xef\x9e\x9c\x9c\x3e\x13\x58\xe9\x7b\x38\x03\x02\x15\x09\xd5\x85\x43\x3b\x52\x08\xf7\x5d\x6d\x24\x6b\xc5\x99\x42\x27\x4e\xdb\x88\xdd\xf2\xef\x8c\xcf\xa8\xc0\x6d\x19\xc8\x44\x8a\x94\x70\x3f\x65\xd0\xa2\x92\x91\x93\xc0\xdf\xee\xd5\x3c\xe7\x8d\x59\xfc\xfb\x7f\x62\xae\x37\x4e\x27\x94\xb7\x35\x3e\x58\xa1\xf8\xe8\x64\x55\xcb\xe0\x93\xdc\x76\xab\xe5\x71\xa7\x40\x9e\xe1\x27\x63\xf2\x12\x5d\xcf\x66\x52\x61\x8d\x24\x1d\x6f\x0a\x1e\xa2\x3c\x20\x25\x6c\x64\x1f\x15\x01\xe8\xff\xe3\x07\xb2\x92\x7d\x11\xc8\x79\xcc\xff\x3f\x90\x93\xbd\x46\x4b\x46\x40\x6b\x0b\x99\x30\x2e\xe0\x36\x67\x08\xa9\x0d\x4e\x56\xa1\xc1\xfe\x93\x15\x49\xe5\xe5\xe2\x87\x6e\x54\xc9\xf3\xce\x1e\x8a\xe6\xc5\xa0\x77\xb4\x9a\xfc\x0d\xe1\xae\x70\x80\x18\xaa\xe2\x83\xd1\xfd\xf1\x96\xb7\x11\x6c\xa8\x7e\x22\xf5\x40\x18\x7d\x9b\x4c\x57\x68\x40\xbc\xe7\x99\x4b\x0d\x5d\x78\x3e\xf7\x9d\x60\x57\x1f\x44\x7d\x9c\xcc\xe7\x8b\xb5\x3c\x39\x69\xf9\xa6\x54\xe4\x38\x33\xb9\x82\xbd\xc6\xfb\x73\x75\x8d\xbc\xbb\x01\xca\x2f\xaa\x90\x1a\x80\x17\x3f\xe4\x48\xd0\x91\x5b\x46\x6f\xb4\xad\x91\x13\x6a\xc8\x5b\xd7\xf0\x9d\x20\xd9\x72\xdd\x20\x2e\x00\xb1\x24\x1f\x69\x8c\xdb\x75\x65\x0a\xf8\x79\x23\x29\x7c\x4e\xc0\xe2\x1a\x46\xfe\x6a\x81\xaf\x3b\x96\x31\xec\x85\xd8\x3f\x31\xde\x93\x84\x9c\x63\xe4\x89\x11\x04\xab\xdd\x8a\xa3\x31\xc9\xc0\xe5\x51\x01\x7d\xce\x1b\x21\xed\x0a\x20\x03\x56\xac\xc5\x19\xb5\x15\xdf\xa9\x6b\x3a\xba\xcc\xde\x1a\xad\x6f\x72\x3c\xaa\xcf\xe7\x67\x0b\xba\x6a\x78\x57\x60\x7c\xe6\x9f\xce\xed\xe3\x0a\xc9\x50\x95\x98\x41\xc9\xed\x2e\x80\x48\x74\x2f\xf5\x42\xd8\x9f\x28\x93\x88\x26\x34\x4a\x23\xa0\x45\x22\x5b\x10\x7c\x3c\xa5\xd7\x3a\xa4\xce\x45\x16\x2b\xa7\xa7\x79\x7a\xb2\x4d\xe3\x5a\xb2\xe0\x0d\x66\x2f\xe8\x28\x6e\x86\x50\xc0\x97\x38\xbc\xb6\x0a\x7a\x59\x2e\x42\x69\x37\x74\x12\xc1\x30\xf9\xe5\x82\x2c\x2f\x8b\x84\xf4\xbd\xa1\xc6\xaa\xbb\x15\x91\xd4\xbc\x5d\xae\x46\xb8\xe6\x9f\xad\x28\xc1\x40\xbc\x3a\xa6\x2e\x5c\x4a\x2e\xc0\x83\x88\x5a\x71\xf7\x24\x2f\x5c\x29\x52\xd4\xe6\xfe\x74\x47\x45\xda\x93\x1f\x91\xcf\x4e\xe4\x58\x27\x39\x6f\xb2\x2c\x4f\xf8\x8e\x48\xf2\x58\xf4\x5a\xd0\x1c\xc3\x5b\x09\x54\x2a\xe4\x6c\xa2\x99\xef\xf9\xd5\xf4\xda\x48\xb3\x3d\xb8\x39\xb5\xb9\xcc\x55\xd0\x6e\xeb\x95\x75\x53\xcf\x9f\xb1\xe3\x6b\x4c\x96\x09\x61\xc3\xd4\x32\x2a\x49\x92\x09\x35\x7a\x4f\x5e\x22\x00\x91\x9e\x97\x23\xdc\x98\x82\x74\xad\x54\xd9\x84\x5d\x8e\xfc\x44\x1f\x0f\x88\x24\x71\x15\xff\x29\x36\x46\x7c\x11\x38\xa8\x78\xc7\x26\xc3\x74\x10\xd7\xec\x78\xdc\x0e\x83\xd3\xad\xb3\xdd\xb2\x77\x55\x1e\xba\x1c\xc3\x68\x51\x01\xc9\x5b\x40\x29\x38\x8b\xbc\xa1\x3c\x9c\xa4\x93\xf8\x61\x34\x9d\x21\xd6\xb3\xd7\xfa\xef\xc5\x5d\x6b\x02\xb8\xea\x04\x56\x6b\x5a\x55\xcb\x09\x3a\x5b\x00\xc4\x75\x22\x8b\x47\x27\x15\xdf\xe8\xe1\x98\x21\x2c\x5a\x2f\x5a\x68\x78\xb8\x35\x5d\x77\x5b\x2f\xae\x5a\xab\xc9\xf5\xb4\x82\x89\xc1\x97\xfd\x91\x16\xe1\x89\x85\x4a\x46\xeb\xd6\x67\xa8\x1f\x3d\x6c\x4d\xaf\x3e\x53\xad\x63\x32\xff\x39\x99\x5f\xa2\x10\xc0\xa8\xa2\xb8\x6a\x02\xeb\x3f\x76\x4c\xff\x2e\xc3\x71\x45\x51\xdd\xcc\x91\xbb\x95\xbb\xf9\x0f\xb3\x85\xf8\x5e\xe1\x5d\x53\xda\xf5\x09\x30\x4e\x4e\x48\xe2\xf9\x2b\xef\xdf\x99\x5a\x0b\x13\xce\xdf\xb9\xfa\x0b\x17\x72\x9b\xac\x70\xdb\x15\xbb\x57\xa8\x29\x15\x5a\xb5\x22\xec\x84\x43\xdc\x13\xcd\x40\x46\x5d\xe8\x18\xee\xcc\x2a\xe9\x68\x1a\x66\xb2\xfa\x30\xbd\x9c\xec\xcc\x29\xe9\x89\x08\x71\xec\xcc\x47\xa9\xa8\x8b\xbc\xa0\xa1\xaf\x77\x66\xd4\x0c\x10\x14\x2a\x21\xc9\xbc\x25\xe5\xa8\x98\x20\xce\x6f\x74\x64\x79\xa2\xbb\x00\x07\xc9\x16\x60\xc3\x12\x1c\xd9\x5c\xc0\xa0\x49\x61\x21\x9f\xd0\x5c\x70\x0c\x4b\x71\x89\x3a\xdf\xb5\x72\x84\xf1\x9b\x42\x2e\x27\x37\x07\x7b\xa6\xe0\x9f\xd0\x4a\xe9\x05\x2e\xb7\xf7\x56\xa2\xeb\xcf\x58\x30\x27\xaa\x77\x56\xc9\xa3\x60\x0b\x4d\x8a\x63\x28\x75\x70\x2c\x3e\xb2\xa4\x79\xe3\x39\xd3\x77\x17\xb3\x69\x75\xf3\x0c\xf0\xc9\xd5\x7c\x34\x3b\x79\xf3\x22\x15\x98\x0a\x34\x0d\x23\x99\xf2\x9d\xdf\x3b\xf2\xb8\x70\xa1\x1c\xc5\x13\x0a\xfc\x23\x28\x69\xc1\x3f\x39\x3f\x91\x16\xfc\xa3\x74\x95\xe0\x64\xca\xe4\x50\x16\x81\xc7\x3c\x43\xe2\x2f\x47\xa9\xcc\x82\x6c\xdf\xe3\xa5\x55\xe0\x1f\x79\x27\x12\x95\x89\x5c\xdf\x82\x0a\x0d\xe4\x2a\xb8\x59\x68\x20\x37\x42\x97\x56\x31\xc3\xf3\xb4\x0b\x17\xca\xf5\x15\xc9\x29\x06\x18\x25\x90\xc2\x07\x73\x15\xda\x70\xd2\x1b\xb9\x8a\x3a\x16\x1a\xc8\x9d\x88\x5b\xe1\x42\xb9\x60\x66\x85\xfc\xe6\xfb\xfb\x7c\xe5\x15\xfc\x93\x8b\x0e\x47\x21\xbf\x39\xcb\x1c\x88\x40\x4a\xee\x9e\x2e\x0a\x17\xca\xef\xd0\x93\x28\xc5\xb9\x90\xb3\x7b\x54\xa1\x3d\xa2\x3b\x94\xfc\x2b\x8e\x88\x3b\xc2\x2b\xc9\x70\xb8\xdc\x79\xc9\xd3\x03\x4f\x90\x17\x09\x75\x32\xa8\x3f\xb1\x0f\x49\xbb\xb2\xa0\x0c\x4d\x57\xa1\x0d\x1c\x83\x99\x44\xef\x73\xba\x1e\x87\x6c\x5b\xab\x00\x8a\xfa\x6a\x86\xf9\xd7\xf2\x1d\x4c\x5b\x8e\xe0\xdf\x7f\x12\x1e\xbe\xac\xeb\xce\x5e\x7a\x3f\x2a\x90\x5c\x38\x99\x5f\xde\x2c\x56\x4f\x35\x1a\xb6\xe1\xf2\x0e\x01\x13\xfc\xf5\x91\x88\x15\x02\xa6\x3f\x8a\xe3\xd1\x85\x13\x6e\xe0\x5b\x1f\x25\xfe\xf1\xf8\xc7\x44\x13\xdf\xaf\xe0\x1f\x13\x8d\xda\x74\x05\xfd\xf5\x91\xac\x9f\x5f\xf0\x8f\x8f\x9e\x5f\xff\x80\x07\x80\x7f\xe2\x68\x7c\xc2\x2a\x7c\x30\x4e\x3e\x13\xdd\xa6\x22\xfc\xb4\xd9\x9e\x20\xeb\xaf\x90\xdf\x5a\x02\xba\x8d\x2c\x4c\xb8\x96\xe1\xf5\x78\x5c\xf8\x60\x90\x5c\x2d\x0b\xfa\x1b\x96\x59\x8c\x7e\x2d\xe4\x37\x48\xf0\xf0\xd7\x7e\xd8\x2c\xcf\x71\xd6\xf9\xc7\x46\xff\x08\x2b\x5c\xf0\x8f\x8d\x7e\x71\x55\xe0\x9f\x20\x8a\xa5\x31\x0b\x17\x0a\x13\xa7\xeb\x82\x7f\x6c\xf4\xab\xc5\x1c\x67\x0e\xb1\x94\x22\xf8\xb2\x99\xde\xcc\xee\x56\x40\x53\xfc\x1d\xb3\xb8\xb0\xcd\xf0\x16\xd0\x8d\x11\x56\xcf\x01\x9b\x74\x7a\xb3\xf8\x58\xf0\x4f\x10\x4d\x3b\x44\x7e\x83\x84\x8f\x53\x38\x82\x85\x06\xea\x49\x7f\xb9\xa1\x25\xf3\x1f\xf5\x2c\x4f\xc5\x3e\x50\xf4\x6d\x33\xbe\x66\x63\x11\x85\x0b\xd9\x44\x72\xbb\xa3\x73\xe9\x3f\x6c\x16\x38\xd7\x13\xdc\x99\xf4\x6b\x13\x5e\xc2\xc5\x59\xf0\x4f\xb0\xfe\xe4\x0b\xa2\xd0\x80\x4f\x52\xc2\x59\xd1\x9e\x71\x38\xaf\x0f\x24\xd2\xc3\x59\x21\xbf\x2e\x61\x6b\x81\xc1\xfc\xfa\x44\x39\xd3\x27\xb3\xe9\xc8\x0f\xb2\x0a\x73\x91\xf7\x54\x93\x98\xd7\xc1\xcf\x3d\x20\x24\x08\x52\x4e\x49\x59\xb7\xd8\x3f\x31\x5f\x0a\x87\x72\x45\x5a\x30\x39\xc0\x5e\xf2\xfd\x0b\x40\x9f\x01\xa1\x2a\xf6\x7f\xe0\x80\x4f\x21\x4f\x96\xcf\x05\x6b\xdb\x7f\x62\xbe\x4c\x1e\x7f\x7c\xf6\x9f\xd4\x30\x96\x7c\x5f\x39\xcb\xc5\xbe\xda\x20\xf2\x69\x13\xf5\x4e\xf2\x23\x79\xec\x80\x1a\x9e\x45\x31\x3e\xaf\xe0\x55\xfb\xcf\x03\x54\x2a\xdf\x27\x0b\x57\xcb\x05\xc1\x9a\xfd\x17\xfe\x23\xca\x81\x2a\xf5\xfb\x2a\x8e\xee\xd3\x48\xb7\x71\x1f\x1d\x03\x85\x71\x3f\xc0\x3d\x86\x78\x05\x25\xc9\x87\xcf\xa1\x22\xef\xc5\xbe\x3a\x89\xb7\x69\xd7\x18\x7d\xed\x63\xe8\x19\xbe\xd8\x7f\x83\x3f\x3e\xd6\xf1\x8f\x8b\xfd\xb7\x1a\xf4\xa9\x7f\x2b\xf6\xff\xd3\x7f\x55\x80\x0a\xef\x9f\x5e\x4e\x82\x98\xa7\x00\xd9\xaf\x69\xd4\xa7\xfe\xc3\xe4\x98\x4f\xaf\xae\x70\xc2\x4e\x39\xe0\x53\x54\xed\x94\xd6\xb3\xd8\x3f\xb3\x9f\x26\x17\xbb\x73\x81\x74\x0e\xf8\x14\xbe\xaf\x8b\x7d\x96\xfc\xd1\xf8\x2d\x32\x8f\x3d\x86\xf7\xfe\x6f\xaf\xe0\x0a\x7b\x31\x76\x24\x60\xa7\xf3\xfe\x6f\x2f\xc6\x46\x20\x0d\xa8\x87\xd9\x13\x65\x95\x7a\x51\xfc\x80\x0f\xfa\xe6\xd9\x93\x17\x27\x2f\xdf\x3d\xf9\xf1\xe4\xed\x69\x8d\x15\xfa\xce\xf3\x57\x73\x61\x8b\xb2\x0e\xba\xe7\x06\xd0\x67\xdf\xb1\x43\x8d\xff\xf3\x82\x23\xb7\xfe\x19\xe9\xa7\xd7\xff\xcf\xbb\x1f\x4f\x9e\xfc\x87\xb6\x03\x84\xe3\xdf\xf7\x8f\xed\x0b\xea\x2e\x2e\x20\x11\x42\xe3\x69\xb5\xc4\x94\x33\x52\xfb\xa7\x4d\x8a\xe8\x57\x95\x33\x47\xe7\xc5\xd5\xab\xc5\x89\x93\xda\xba\x37\x6a\x6d\x58\x05\xd5\x99\x92\xd7\x79\x31\x21\x3c\x59\xe7\xa4\x3f\x81\xb2\xea\x10\x78\xc9\x8c\x1a\xf7\x52\x80\xae\x3c\x39\x91\x82\x92\x4c\xe1\xd5\x64\x4e\x7a\x01\xa5\xe9\x04\x79\xe3\x1f\xdd\xf6\x07\xd6\xda\x35\x37\x37\x2c\x28\x72\x48\x7e\x83\x08\xda\x94\x76\x28\x28\xb6\xb8\xd7\x30\x04\xb5\x71\xcb\x95\x20\x0b\xaa\x87\x8d\xaa\xd4\x37\xc5\xa0\x55\x1a\x8a\xe4\x67\x97\xcc\x8d\xa8\x3c\xf2\x63\x2a\xd9\x7c\x0b\x95\x72\x91\x8f\x35\x67\x4f\x03\x9d\x0e\x55\x19\x3d\x79\x41\xea\x40\x73\x0c\x33\x3f\x84\xbe\x3e\x8c\x88\xbf\x69\x81\x26\x80\xa9\xa1\xfc\xf2\x6a\x01\x34\x66\xe1\xc7\xbb\xe7\x82\x66\x62\xa1\x8f\x66\x6a\xcb\x54\xe7\xb6\x74\xcf\x9c\x12\x91\x66\x6a\x42\xae\x67\x0a\x3c\xf6\x35\xf5\x7c\x10\x46\x41\x13\x42\x14\x95\x58\xf8\xd0\x7a\x06\x3e\x9b\x51\x5b\x56\x77\x88\xff\xfe\xb7\x5d\xa6\xd3\x89\xa5\x65\xb3\xd5\x6c\xba\x07\x89\x41\x85\xfb\xfa\xb5\xd9\xd8\xf8\xdc\xec\x4a\x9f\x27\x28\x9a\xc7\xc6\x53\x4d\x91\xf8\xc4\x04\x8f\xc8\x12\xb6\xf2\x98\x9c\x2b\x0b\x85\xbb\x68\xe3\xd1\x58\x1b\xb2\x8a\xac\xb4\xc8\xde\x69\xf5\xed\x76\xf2\x7d\xb2\x67\x16\x48\x64\x16\x85\x43\xc4\x7b\xc6\xb3\x87\x00\x35\x06\xda\xf8\xe5\x62\x41\x6a\x19\x77\x4b\x6d\xb9\x6a\x7d\x98\x8e\x5a\x95\xbc\x9b\xd7\x95\x31\x5a\x17\x9f\xa5\xae\xbd\xd6\xe9\x64\x52\xb4\x44\x41\x1e\x6e\xba\x58\x3d\x7e\xb4\x9c\x1e\x46\x6f\x00\x49\xa0\x91\x1e\x4d\xd0\xb6\x69\x58\xf2\x0c\x3c\xfd\xb0\xd3\xe9\x32\xa4\x41\xb5\xf0\xb7\x4b\x76\x58\x7e\x3c\xfb\xe9\x65\xc9\xe2\x74\xed\x7f\x39\xfe\xc3\x51\xef\xfb\x43\x12\xad\xd3\x17\xfa\x1c\x73\x5a\x9f\xc3\xf8\x6d\x5c\x04\xcb\x73\xc7\xc9\x78\x8c\x8e\x08\xc4\x92\x09\xe6\xa1\x1d\x4a\x71\xe2\xed\x6a\x35\xba\xb6\xc7\x31\xea\x9f\x5e\xf8\xcf\x25\x5b\x8a\xa6\x8a\x25\xac\x52\x7e\xec\x2d\x85\x64\x12\x9b\x5a\x75\x13\x13\x6d\xa9\x27\x24\x73\xe8\xf6\x55\xb8\x75\x58\x22\x11\x6b\xa1\x37\xcd\xa8\xe8\xd3\xc9\x08\xee\x6f\x5f\x96\x63\x19\x94\xe3\x1b\x92\x7f\xf5\x36\x6a\x2d\x79\xf3\xf1\xa7\x9e\xe2\xe3\x75\x6f\xfa\xbd\xe6\xb0\x8a\x2f\x41\x8b\xee\xa0\x4f\x87\xb5\x4e\xbd\xbe\x72\x27\x83\x4d\x9a\x5c\xcd\xf3\xbb\xb9\xdb\x72\xaa\x1b\x20\x42\x6f\x36\xa5\x71\x73\xc3\xc5\x87\xf2\xd6\xb0\xbd\x39\xfe\x5f\x20\x22\xcd\x5a\xe3\xc5\x24\xd8\xcb\xc4\x7c\xfc\x5f\xba\xd3\xff\x97\x63\x40\xca\xeb\xf8\xe4\x03\x49\xf0\x70\x0f\x9f\xb1\xa3\xe5\xd3\x35\xfa\x96\xd6\xae\x26\x9c\x25\xc9\xc4\xa5\xe0\x83\x79\x39\x0b\xd4\xed\xdc\x0f\x5a\xc5\x54\xb2\x2e\xe4\x64\xdc\xb8\x5e\x23\x86\x82\xdf\x02\xc5\x73\x35\x57\x8e\x2b\x46\x60\x1b\x51\x7a\xf4\xcd\xf4\x7c\x1e\x4e\xa2\x2d\x85\xa6\xd4\x49\x35\x88\xab\x26\x73\xe3\xc3\xad\x17\x18\xe1\x87\x4b\xe7\x60\xca\x37\xcd\xad\xd9\xae\x5e\xcd\x33\x74\xc6\xf2\x75\xcd\x4b\x4b\x8d\xcd\x67\x85\x55\x71\xb3\x29\x24\x10\x01\xed\x6c\xb3\xda\x56\xa9\xef\x54\xe3\xd4\x43\xdc\x61\xbf\x18\x3b\x41\x0f\x40\xc4\xd8\x78\x8f\x59\x13\x38\xaf\x84\xed\x0d\x5c\xf6\x21\x82\x5b\x9b\x85\x4d\x60\x60\xbd\x4e\xf4\x94\xba\x5f\x2f\x49\xc2\x07\x7c\x09\xc8\x8b\x78\xcf\x56\xd4\x75\xab\x1d\xc4\xf2\x60\xbb\x7a\x1f\xb7\xdb\x0d\x65\xd0\xc2\xb5\xb9\xb1\xf3\xda\x51\xc9\xdc\xac\xd6\xbb\x95\x47\x13\x60\x8c\xfa\xc6\x33\xda\xb8\x5d\x23\x81\xf9\x7f\x74\x62\x37\x9b\x83\x63\x27\xf0\x19\x2b\x53\xf7\x83\x39\xd9\x6c\xd2\xb8\xcf\xbe\x55\x8f\x4f\x7f\x69\x25\xcb\xfb\x6d\x98\xc7\xe8\xc7\xab\xc8\x42\x61\xd3\xc3\xb9\x67\x75\xb9\x18\x66\xda\xdd\x16\xcd\x0c\xd9\xfc\x7f\xe8\xdc\xb3\x37\x71\x64\x72\x12\x1a\x59\xc6\xb3\x80\x15\x4a\x4c\xb9\xa7\xb9\xf0\x05\xd1\x0a\xff\x62\xf1\x29\x9a\xd9\xfe\x33\x57\xe2\xcb\x90\x66\x8e\x28\xb3\x91\x75\x0b\x54\x94\x24\xf7\xab\x61\x91\xcd\xe6\x8b\x9d\x64\xff\x05\x38\x7f\xda\x0d\xf6\x26\xe1\x27\x90\xfd\x96\x8a\x52\x91\x74\xc0\x9d\x10\xc8\x2a\x5a\x54\xa6\x59\x9b\x2a\xfe\x28\xb0\xba\x9e\x9a\x4a\x80\x08\x92\xf6\xad\xed\xca\x1f\x47\x15\x31\xb7\x5c\x1f\x15\x4e\xd7\xae\x3b\xf4\x89\xaa\xcc\x84\x3e\x42\xaf\x4e\xda\x94\x94\x26\x24\x6c\x87\x96\x23\x51\x48\x04\xa1\x9c\x97\xa1\x1b\x9c\xcf\xcf\xd7\x43\x14\xa3\x6b\x59\x27\xf2\x50\x97\x36\x4b\x05\x1e\x1f\x1c\x17\x56\xad\xc8\x82\xa4\xb0\xb7\x97\x15\x77\x1f\x85\x6e\x7c\xd8\x49\x91\xa2\x76\xb8\xeb\x99\x37\xf3\xe0\x73\x36\xc2\x62\x4d\xf6\x92\x4f\x55\xc3\x00\x73\xf8\xbe\x4d\xff\x29\x13\xe1\x84\x3d\xa1\x2e\xaa\xd5\xf5\x80\x84\x1b\x31\x4f\x16\x88\xd6\x38\x24\x66\xf7\x6c\xa0\x4f\xde\x2f\x4c\x88\xc2\x9c\x29\x2a\x5b\x5e\x4b\xd6\xf2\x9f\x31\xa2\xde\x6f\x9e\x69\x0d\x95\xe1\xf8\xf3\x03\x54\xab\x8c\xba\x18\x6c\x1c\xcd\x4a\xdd\x21\xf8\x1f\xe4\xed\x94\x41\x86\xad\x97\x5b\xdc\xbd\xa6\x51\x15\x59\xfd\xd4\x38\xf4\x11\xf9\x2b\x4e\x0a\xd9\x52\x1e\x7e\xf3\x54\xa5\x13\x69\xf6\x5a\xff\x7b\x1e\xee\xb8\x54\x51\xc9\x31\x8a\x8f\xae\xae\xbe\xca\x37\x57\x6c\xfa\x11\x43\x91\x3e\x74\x54\x0d\x21\x86\xd8\xbb\x40\x8e\x9a\x7d\x4b\xd7\xd0\x5c\xc7\x50\x8b\xee\x7a\x81\x00\x9c\xeb\x85\x15\x81\xf2\x20\x6d\x1f\x36\x8b\x08\xc3\x58\xc6\x36\xee\x0f\xfb\x15\xb7\xd9\x5c\x1b\xf1\x38\x04\xea\x37\x10\xe2\x4e\xf6\xf4\x4f\x65\xd9\xa0\x9d\x23\x56\xab\x8d\x2c\x2d\xd9\x53\x10\xf9\xa9\x0c\xad\x29\x71\xac\xb2\x0f\x70\xbd\x44\x9d\x20\x90\x71\x64\x69\x6c\xab\x98\x99\x59\x0d\xa6\x5e\xdc\xca\x52\x2c\xed\x6e\x03\x09\x1b\xd1\xc2\xc2\x4d\x46\x16\x58\xeb\x78\xd4\x85\xcf\x51\xfe\xf0\xfa\xf5\xcb\x67\x27\xaf\xde\x9d\x9c\x9d\xbd\x25\x3c\x27\x14\xac\x73\xb7\xbb\x29\xd3\x6e\x6b\xa1\x67\x2f\x9f\xa1\x8e\xdb\xa9\x6e\x13\xb7\xd3\x86\x24\x2a\xac\x05\x0c\x5f\x8c\x49\x3b\x83\xb9\xba\xee\x31\xce\x24\xb0\xc1\x64\x28\x43\x34\x59\xf0\x63\xdc\xf1\x1f\x68\x0e\x56\xf4\x2b\xac\x76\xc4\x32\x1a\xa2\xad\x68\x31\xa7\xf3\xa8\xfe\x4b\x14\xc5\xf0\x78\x52\x34\xfa\x0e\x40\xe9\x92\xd1\x35\x31\x3e\x5d\x75\x51\x7c\xbd\xbe\x4b\x24\xe1\x67\x3f\xdc\x5d\xa0\xb1\xce\xbd\x23\x57\xa1\x7a\x00\x4e\xed\x67\x29\x95\xae\x2e\x85\xc8\xf5\x46\x6a\x33\xfb\xb4\x2a\x85\xbc\x7b\x63\xfc\x44\xaa\x88\x15\x6e\xf8\xbb\x6c\x1a\x70\xef\xeb\x66\x21\xae\x19\x35\x06\xa4\x8c\x18\xb2\xfe\x40\xf4\x69\xbe\x2b\xfb\xf1\x96\x53\x08\xac\x84\x69\x75\x31\x99\xe6\x4a\xc8\x26\x5c\x6d\x51\xb6\x79\x48\x2c\x0d\xd8\x4f\x93\xcc\x22\x12\x11\xa1\xfb\x03\x55\x14\x0d\x76\x13\x82\xdd\x3f\x3e\x2e\x6f\xab\x29\x60\x0e\x8d\x33\x82\x6c\xbe\xe6\xa5\xb7\x49\x0d\xa3\x23\x19\xa3\x22\xb5\x64\x4f\x54\x77\x40\x12\xc5\xb5\x87\x89\xf5\xfa\xbd\xb3\x38\x7b\x14\xbc\x7b\x5a\x37\x44\x49\xf1\x27\xec\xc6\xdb\xb8\x72\xba\x70\x28\x8d\x76\xa6\xce\xae\x58\x71\x54\x35\x63\x9d\xcc\xa1\x64\x51\x5f\x9d\xf8\xdd\xaf\x9b\xe4\x21\x0d\xee\xc0\x3c\x56\xd6\xaf\x45\xa5\x59\x81\x71\x81\x7e\x79\xbd\x60\xe9\xcc\x93\x71\x76\x94\x7a\x93\x4e\xa0\x79\xce\xce\xaf\xd6\xf4\x16\x9a\xa8\xf8\x09\xb5\x0e\x90\xa2\xf4\x72\xce\x6c\x18\x85\x9d\xc8\xde\xb1\x53\xd6\x40\x27\x88\xc4\x37\x27\xa9\xaa\x5d\x8b\xf4\x10\x90\x9e\xe4\x0d\x1e\xce\xe2\xd3\xc9\xe5\x8c\x24\x08\x7b\xff\x63\xa7\x15\xa2\xab\x79\x3f\x95\x92\xe8\x57\x43\x82\x68\xa1\x5c\x63\x89\x9a\xf5\x86\x79\xb9\x7a\xcc\xdb\x60\xd9\xf2\xf4\xec\xed\x8b\x37\xef\x9e\xbc\xfe\x89\x00\x27\x0b\x79\x73\x3f\x4a\x2e\x27\x2a\x9c\xcf\x01\x24\xbf\xfd\xf3\xa9\xd7\x5e\x91\x5c\xa8\xcd\xc9\x28\x0d\xe7\x78\x77\xfa\xe6\xe5\x8b\xb3\xcc\x6a\xcc\x5d\x67\xf7\x28\x36\xaa\x2d\x72\x3e\x93\x01\xf6\x35\xae\xcc\xaa\x42\x0f\x16\x72\x2d\xc8\x50\xf8\xd6\x9e\x8b\x9a\x27\x50\xa0\x7e\xc0\xa5\x0e\x5c\x05\x2d\x98\x53\x80\x93\x57\xba\xd1\x1f\x1c\xe7\x56\xe4\xf6\x6a\x3e\xc0\xf4\x61\x9e\x5c\xcd\x61\xa0\x66\x16\xd9\xa2\xe8\x11\xcd\x3d\x6c\x80\xb0\x10\x65\x27\xbb\xe8\x52\x22\xbe\x40\x62\x55\x86\xb3\x05\x4a\x5b\x98\x87\x07\x61\x01\x09\xe1\x34\x96\xf7\xa3\xba\x08\x9f\xa1\xf6\x1c\x15\x6b\xa8\xad\xbe\xd3\x70\xc2\x8c\xa1\x91\x3e\x57\x69\xe6\xe8\x4e\x8d\x6a\x36\xc7\xe7\x9e\x34\x03\xd1\xb2\x77\x6c\x80\xe8\x41\x81\x48\x2f\x10\x96\x86\xfe\xc5\xa5\x06\x6f\xa5\xc2\xc7\x65\xec\xdb\x80\xbf\x4a\x0d\xbd\xf0\xaa\x1e\x24\x11\x36\x15\x2f\x5f\x5a\x28\xdf\x73\x1f\xdd\x7d\xf4\xcf\xbe\x43\xfc\x70\x79\x8d\x56\xd1\x13\x7d\x1e\x13\xf9\xc3\xdb\xbb\x6a\x2d\xd6\x84\x5b\x58\xba\x25\x32\x6f\x2d\x16\x7a\x74\x92\x7f\x6a\x9b\x4e\x4a\x33\x8d\x4f\xaa\x5b\x1a\x75\x7a\x77\x75\x35\xfd\x34\x74\xfd\x7e\x67\x54\xd5\x45\xdc\xd5\x4a\xc8\x1a\xd1\xd2\x70\x96\xef\xf7\xc9\xa1\xb1\xe6\xda\x06\xf6\x00\x69\x52\x9d\x40\xb4\x8a\x58\xfb\x4d\x62\x1b\x1a\x58\xd1\x43\x2f\x37\xe3\x34\x51\x9c\x41\xbb\x69\xd3\x0c\xdb\xea\x83\x87\x44\xd6\xee\xb1\xe8\x67\x53\xe3\xa1\x69\xbc\x2c\xb0\x7b\x41\x8d\x04\x35\x3c\xbc\x97\x9c\x2c\x60\x96\xd7\xe7\x5f\x19\x29\xfa\x26\x51\x4b\xf1\x4d\xa3\xb2\x36\xbd\x4e\xa6\x32\x91\xaf\xe8\x9d\x10\x63\x9f\xab\xcb\xb7\xd5\xd4\x3d\x14\xd7\x77\x20\x7a\x13\x37\x45\xa3\xa5\xcf\x72\x2c\x8d\xab\x57\xda\x6a\x68\x53\xf6\x6a\x31\xb1\x4f\x2e\xcc\xe0\xe4\x41\x75\x30\x2f\x22\x35\x27\x6d\x20\xb7\xd5\x19\xa5\xa8\xe6\x52\x32\x42\x32\xb8\x0d\xdb\x4b\x8f\x7c\x61\x9b\xdc\xda\x63\x8f\xf2\xb4\xa2\x8a\x13\x83\x2b\x32\xa6\x66\x44\x5e\x63\xd5\xe9\x20\x7b\x5d\x8f\x07\x19\x35\x58\xfb\x44\xeb\xa7\x39\x95\xd4\xec\x3e\x4c\x82\xfb\x51\x92\x10\xb0\x92\x38\x9a\x7d\x8e\x71\xa5\xbc\x60\xf7\xf3\x79\x19\x0a\x93\x49\x9e\x9e\xef\xb2\x97\xa2\x14\xfb\x20\xf5\xc1\x3e\x9f\x77\x9d\x2c\x73\xa6\xd9\x5c\x92\x97\xc8\x35\x0f\x12\x46\x26\xd8\x67\xb4\x72\xbc\xf8\x64\x31\x9d\x96\x26\xca\x51\x9f\x53\x11\xf3\xbf\xf7\x35\x91\x9d\x95\x40\x44\x7b\xe8\xb6\x7e\xf3\xb6\xf4\xe5\x50\x51\xbf\xa7\x79\x06\x26\xfe\x78\x38\x94\x07\x4b\x27\xd9\x6d\x52\x1f\x0d\xd5\x7c\x96\x81\xd7\x32\x7d\xfd\x48\xf4\xb4\x01\x34\xd3\x46\xd3\x15\x71\xd7\xed\x3f\x5a\xde\xde\xad\xb2\x03\x9c\x88\xbd\x17\x3c\x64\x40\x1f\xb5\xe5\x44\xeb\x65\x21\x44\xc5\xd3\x6a\xbc\x4c\xba\x62\x71\xba\xdd\x9e\xa0\x63\xb7\xcb\x5f\xdb\x6d\x64\x9e\xc8\x87\xe3\x99\xb8\x7c\xc4\x2c\x29\xdd\x27\xda\xe0\x4e\x3a\x92\xdb\x5d\xf8\xfe\xa6\xe1\x16\x57\x28\xe8\xfb\x7c\x34\x25\xe9\xf5\x45\xcb\xc0\x56\x95\x4a\x47\xb5\x93\xf1\xdd\x04\x52\x8b\xf3\x39\x69\x0b\xa9\xae\x36\x57\x0d\xb4\x83\x36\x09\x41\xf4\x96\x92\x79\x71\xf3\x3a\x7e\xc1\xa2\xbb\x0e\xcf\x20\x59\x21\xaf\x8f\x61\xc9\xed\x53\xb9\x47\x0c\x2c\x63\x2e\x18\x16\x89\x15\x16\x6c\x26\x9f\x6b\x60\xa2\x51\x73\xf5\xc5\xab\xd3\xb3\x93\x57\x67\x2f\x4e\xce\x5e\xbc\xfa\xf3\xae\x2b\xf8\x72\x3c\x41\x95\xf2\x27\xd3\x15\x3a\x80\x5a\x79\xd1\xfb\xcf\x00\x3f\x00\xbf\x2b\x70\x4a\x12\xd2\xb3\x11\xc3\x03\xad\xef\x0f\x90\xa3\xe7\xc4\x9b\x6b\x4d\x6f\x8d\x31\x0e\x2a\xa7\x3a\xa5\xb6\xdb\x79\x43\x97\x83\x0e\x37\x65\xd0\xeb\xcc\x56\xb4\xbd\xc2\xe7\xf6\xd9\xe7\x7b\xd6\xe8\x61\x29\x0a\x6b\xd8\x95\xf7\x31\xa0\x83\x64\x15\x88\x24\x5e\x85\x3f\xa0\x9a\x3d\xc4\xad\xc7\xb7\x40\x44\xc1\x15\xbb\xb4\x68\x3f\x3b\x8f\x2d\x8f\xd4\xaf\xae\xa2\xb9\x02\x2e\xd4\x9d\x2c\x83\x0c\x36\xf2\xa1\x79\x10\x4c\xd4\x9c\x56\x23\x52\xb8\x63\x3d\xa6\xeb\x5f\x01\x69\x4d\x5e\xcc\xe1\x7e\x40\x81\x36\x51\xab\x65\x73\x19\xbf\x4e\xe6\x7b\xad\x67\x9f\x96\x10\x01\x3b\x58\x66\x81\x95\x2f\x46\x55\x8b\x5b\x61\x6d\x30\x5a\x34\xe2\xf4\xe3\xb8\xf8\x88\xf3\xd0\xdb\x6d\xfe\x6d\x32\xf5\xd5\xe7\x24\x62\xbe\x17\x66\x53\x92\x65\x91\x6d\x45\xd2\x88\xa9\xc7\xe5\x81\x76\xb9\x9a\x97\x88\x9d\xd3\x04\x0c\x33\x9a\xe2\xfe\xc1\x71\x31\xf2\xe6\xa2\xb2\x7b\x14\x7c\x6e\x1d\x15\x7a\x2b\xc1\xed\xda\xa3\xa8\x63\x13\x35\x52\x28\x49\x29\x8f\xea\x29\xb4\x3e\x68\x9b\x84\x73\x7c\xbb\x3b\x07\xff\x3e\xd2\x9c\xdf\x7d\x39\x27\xff\x7e\xab\x25\x7e\xff\xf5\x25\xf8\xf7\x3b\x2d\xf9\x87\xdf\x5e\x92\x7f\x7f\xaf\x35\xfc\xdb\x3f\x5e\x03\xff\xfe\x41\x6b\xfa\xe3\xff\x7e\x4d\xfc\xfb\x6f\x5a\xe3\x9f\xfe\x79\x35\xf2\xef\x1f\xb5\xe6\xe3\xa3\x7f\x7e\xd5\xfc\xfb\xa7\x21\xba\x65\x66\x79\x5a\xd7\x44\x64\x25\xac\x0a\xc1\x85\xc7\xb7\x91\x89\x10\x00\x0c\xc5\xe6\x72\xae\x6a\x32\x66\xeb\x2f\x4f\x3c\x5e\x6e\x51\x48\xc7\x7c\x31\xe9\xde\x29\x6c\xe9\xec\x5e\x9e\xd1\xab\x3e\xfe\x1d\xe0\x1f\x7f\x2d\x16\x94\xe2\x8b\x38\x84\x9a\x6c\x56\x9a\x5a\xc3\xfe\x08\x96\xc2\xdd\x77\x5d\x96\x71\x38\xfd\x88\x34\x28\x03\xf8\x80\xfd\x2c\xb4\x98\x58\xc9\xbb\xe7\x2a\x45\x89\x26\x37\x93\x54\x98\x30\x8a\xf1\x19\xb0\x91\x2b\x08\x2d\x34\x90\x03\xcc\x29\x22\x4b\x23\x4d\x54\x5c\x93\x82\x5e\x8c\xd9\x6f\x36\x8d\x57\x23\x33\x1d\xb6\x24\xf0\x15\xdc\x27\xe8\xf5\x30\xac\xa2\x4c\xbc\xfa\x0f\x5e\x1e\x08\xfd\x03\x6c\x97\xa6\x59\x78\x46\x21\x99\x53\x3a\x0d\x87\xe2\xde\x69\x77\x85\xdc\x02\x8d\xf6\x2a\x5d\x61\xba\x62\x01\x4e\x8f\x2b\x4c\x96\x58\x55\xde\x0a\x13\xe5\x4d\xd8\x69\x6c\x85\xa9\x1a\x9d\xe5\x8e\xba\x2a\x5c\x68\xeb\x67\x41\xd1\x92\x32\x9c\xfb\xfd\x9a\xc1\x87\x08\x8b\x09\xb2\x5b\x65\xc8\x1d\x57\xda\xdd\x7c\x89\x78\xd7\xcf\xf3\x5f\xe7\x8b\x8f\x7e\xa5\x77\x62\x16\xf8\xb4\x6c\x29\x47\x5c\xb9\x98\x8e\x2a\x83\x1c\x5f\xee\x73\x90\xdd\xd8\x96\xe0\x59\xf6\x6f\x1a\x0f\x63\xf4\x26\x7b\xbc\x1d\xbf\x44\xf0\x2d\x2d\xa1\xe9\x96\x00\x5d\xd7\x46\xd4\xda\x6e\x42\x2f\x64\x55\xef\x6a\x08\xbd\x9b\xa3\x62\x96\x9d\x46\xcd\xe3\xc1\x5c\xa3\x00\xbc\x90\xbd\xe8\x45\x94\x13\x00\x9d\x78\xc6\x86\x2f\xca\xbd\xa3\x1e\x31\x48\xc7\xd3\x0a\x23\x4e\x6c\x1e\x0b\xf4\x9a\x0b\x1f\x6f\x99\xbd\x4a\x54\xf6\x20\x11\xf1\x68\xb4\xf7\xa1\x62\xe2\x6a\x21\x44\xcc\x85\x78\xde\x88\xe8\x13\xbb\x8c\x46\x2a\x3c\xc4\xa0\x9f\xa3\x70\x1e\x0f\x2b\x65\x43\xe1\xde\xca\x38\x31\xda\x6b\x33\x0d\x99\x1a\x8c\xf8\x70\x89\xcd\x26\x19\xa1\xd5\xd3\xd0\x26\x9d\x7d\x9c\x42\xd6\xad\xd4\xee\x24\xf0\xd4\xc8\xb8\xe1\x0f\xd1\x5c\xc8\xcc\x4e\x66\xb7\x08\x02\x01\x81\xd4\xd1\x20\xf8\xba\x49\x51\xdc\xab\xba\xe9\x43\x13\xb7\xe5\x2e\x1b\x4b\x98\x23\x43\x6f\x67\xea\x78\x17\xf6\xf8\xe2\xbf\xa6\x93\x8f\x68\x17\x04\x0b\x46\x53\xd0\x50\x4f\xf5\xc3\x67\x7a\xca\xe3\xaa\x76\xd4\x45\xce\x6b\x81\x7a\xc0\x4c\xed\xb6\x4c\xbf\xf7\xcd\x77\x94\x1f\x65\x45\x63\xec\x56\xbc\xb5\xb1\x19\x4b\xcd\xa2\x11\xce\xae\x42\xc3\xfe\x80\x66\xdc\x9a\x8a\x3d\xfb\xb4\xfe\xee\x13\xcf\xd9\xd6\x82\x1e\x5b\x81\x33\x90\x9f\x72\xff\xc8\xdf\x05\x85\xb6\x46\x68\x47\x14\x39\x6a\xfe\xc4\x50\x39\xc1\x69\x06\x98\x1d\xa6\x0e\xec\x5f\xdf\xad\xf1\x50\xa1\x4b\x3f\xd4\x6f\x41\x8b\x2d\x78\x1c\x91\xe6\x71\x98\x05\x71\x75\x9a\x0c\xb0\x1e\x67\x9e\x5e\x41\x03\xca\xb5\xba\x9e\xa0\x52\xe0\xc1\x41\x8e\x0e\x8e\x76\xa4\xb2\x20\x79\x43\xa2\xaa\x16\xba\x07\x62\xec\xd3\x43\xf9\x96\xe8\x59\x23\x35\xd4\x3b\x8e\xbd\x4b\x3e\xac\x52\xba\x44\x43\x5b\x97\x6f\xf8\x91\x5d\x55\x45\x50\x3c\x5d\xf4\x1e\xb2\xfb\x3d\x3f\x4f\x37\x93\xcb\x5f\x53\xff\xb8\xb2\x84\x52\xcf\xe7\x46\x4c\x91\x23\xb2\x7b\xfe\x4d\xc9\xc8\x1d\xee\x22\xae\xaa\xf4\xb5\xa6\x54\x55\xee\xdc\xb3\x6f\x03\x17\x3c\xd3\xd5\xe4\xe7\xd5\x8c\xf5\x91\xa0\x39\xb8\xaa\x55\xcf\x84\xdf\xdc\xf0\x49\x40\xd6\x18\xf2\xed\x95\x88\xed\x75\xef\x56\x33\xf4\x43\x9f\x86\x89\x26\xcd\xbd\x93\xdc\x69\xdd\x2a\xb8\x68\x46\x30\x93\xa8\xec\x5e\x43\xa9\xaf\x81\xc4\x11\x59\x3d\x61\x76\x45\x2e\x50\xf3\xd5\xe8\xa3\x8a\xf9\xba\xe3\x8d\x18\xae\x6e\x68\xb5\x2b\xe3\x40\x1c\x14\x42\xdc\x40\xe3\xe5\xd3\x4c\xb8\xa6\xf8\x98\x9c\xfc\xd3\x47\xc9\x36\x2e\x47\xba\x1e\x76\xc1\xd3\xc9\x15\x3a\xac\x27\x7b\x0b\xd4\xeb\x69\xf5\xd3\xe2\xf2\xd7\x52\x7c\x75\xed\xd8\x74\x40\xfb\x3e\xb0\x93\x90\xe9\x48\x75\xed\xef\xef\x3e\x26\xe5\xee\xa4\x5c\x0a\x4f\xe7\x97\xf5\x44\x6e\xdf\x9c\xf5\x1d\x5d\xec\x74\xb6\x5c\x0f\xe0\x9a\xd3\xab\xcf\xa8\x5b\xf7\x6a\x51\xaf\xae\xf2\x55\xa9\x32\xee\x6f\xdb\xac\x0f\x1c\xcd\xbe\xd6\x08\x70\xf4\xc1\x83\x47\xf6\xfd\xb5\xf5\x6d\x8f\x9d\x9d\xbb\x33\x90\x4b\x37\xdc\xb4\x8e\xc6\xe3\x37\x14\x55\x06\x28\x80\x7b\x6b\xf2\x4f\xfd\xa6\x16\xd8\xe7\xf6\xd8\x1e\x1f\x1d\xd9\x13\xab\x6d\x70\x5f\xa0\xb6\xfc\x6a\xce\x1d\x89\xce\x86\x87\xba\xab\xc9\x55\x7e\x01\x37\xa1\x5a\x7c\x73\x37\x0c\x8b\xa9\x60\x52\x92\xe5\xf1\x41\xec\xe9\xd1\xf0\x7d\x87\x8f\x5c\x9e\x15\x89\x5d\xa2\x4d\xc0\x75\x1b\x9d\x04\x3c\xaa\x3b\x0e\x49\x96\x43\x35\x5c\x3c\xec\x70\x89\xf1\x3c\x35\x7a\xe9\x44\x39\xb0\x7d\x85\xf0\x7a\xb8\xfa\xd2\xa1\xbe\x7c\xeb\xbb\xe7\x29\xf1\x80\x08\xa2\x24\x09\x35\x59\xa4\x9a\x05\xa7\xae\x9e\x6e\xe7\x88\x4c\xeb\xa5\x09\x4e\x5d\xb2\x2b\x1e\xcd\x5e\xa7\x76\xd6\xcc\xf4\xf4\xdd\xec\xeb\x33\x2c\xf5\x20\x58\x13\x1a\x2f\xed\xfc\xad\x57\x9a\xd0\xca\x36\x9b\x20\xaf\xb5\x89\xf9\x6f\x28\xed\xf6\xbb\x44\xb6\x5f\x1d\xde\x21\x39\xe4\x62\x51\xf1\x16\x41\x04\x2d\xe6\x62\xee\x20\x6f\xd3\x61\x92\x5e\x04\x65\xf1\x99\x3b\x9a\xf1\x76\x5b\xe4\xa2\x44\xb7\x1d\xaa\x4d\x13\xb8\x92\xd0\xa7\x33\x62\x80\x16\xc0\x67\x66\xbd\xe0\xf2\xbf\xa4\xc8\x7e\x43\x79\x9f\x1a\xd7\x50\x84\x27\x29\x8d\xea\x8f\x46\x7a\x64\x62\xdc\x8c\x44\xe7\xd6\x29\xf3\x0b\xdc\xc1\xf5\xfd\x11\x17\x24\x7a\x46\xa2\x45\xda\xb5\xf8\xba\x45\xf1\xab\x1f\x2e\xd1\x2f\xa8\x33\x53\xf5\xcf\x8b\xf3\xc3\xf3\xc3\xc1\x2f\xe7\x87\xc3\x6f\xd8\x87\x65\x92\xf8\x93\xfa\x04\x55\x73\xc8\x7a\x4e\xee\x3f\xf9\x35\xa6\x4c\xd0\x78\x12\x7e\xbf\x19\x91\x0e\x99\xe9\x63\x9a\xf1\x52\x5e\x4a\xf1\xd0\xe4\x8e\xf5\x1d\x4c\x19\x54\x77\x8e\x3e\xd8\x3e\x02\x87\xf3\x29\x9b\xbc\x27\x4d\xa4\xb9\x32\xad\xcd\x85\x27\x2d\x20\x26\x1d\x75\x8f\xf0\x98\x5a\x9f\xeb\x45\x6d\xab\xb5\x4a\x54\x2e\xb3\xd7\x12\x1b\x4e\x66\x3e\x58\xb2\xd0\x94\x8e\x9c\xec\x70\x4a\x69\x32\x20\xb3\x96\x7d\xe6\x70\xa4\x17\xd5\x44\x8b\xcc\xec\x2b\xe7\x88\xe7\x85\x4c\xf0\xdd\xcd\x27\xd5\xe5\x68\x39\x91\xaa\xc4\x17\x07\x3e\x84\x1d\xa9\x27\x80\x9e\xe9\x94\x3c\x81\x86\x12\x2d\xf5\xf4\xdd\xb5\xb2\xff\xd3\x63\x14\xa7\x35\x90\x4d\x4a\x6f\xeb\x9e\x83\xeb\x93\x59\x4a\xd5\xb4\x54\x9d\xa4\xec\x11\xe3\x23\xe9\xf8\x7d\xd2\x49\x50\x35\x01\x1f\xda\xca\xb3\x9b\xbb\xbc\x75\x74\xdc\xfa\xf7\xd1\xbc\x75\xfc\xa7\x7f\x3b\x6a\x1d\x1d\x15\xf4\x7f\xeb\xcf\x3f\x9d\x19\x6f\xa9\xce\x24\xbd\xdd\x2c\x65\xc3\x2e\x88\x5b\x4f\x3a\x12\xc1\x35\x40\xd3\xb5\xee\xa8\xde\x60\xe7\x38\xb7\xb5\x3f\xfe\xee\xe8\x4f\x7f\x00\xba\x01\x11\xd6\x8f\xa3\x15\x9c\x7c\x9e\x85\xd6\xef\xd4\x7a\xeb\xef\xe0\x46\xad\xaa\xe9\xc5\xec\x33\x6b\xf3\x4c\xd6\x68\xa0\x68\xf1\x61\xb2\xba\x42\x9a\x6e\xdc\xba\x98\x5c\x8e\xd0\x61\xf0\x74\xdd\xfa\x88\x36\x86\x16\x0b\x98\xcd\xd5\xf5\xa4\x95\x6a\xfb\xdc\x54\x27\x69\x3d\x6e\x61\x73\xad\x8b\xcf\xeb\x49\x95\xed\x21\x8b\x82\x4f\xfc\x18\x71\x29\x7b\x27\xa3\x40\x16\x4a\x16\x91\x25\x51\xbe\x69\x5f\x8c\x9d\x9a\xe2\x2e\xbc\x25\x77\x59\x2d\xf6\x6b\xa0\x88\x88\x79\x45\x18\xdc\xc0\x15\x1b\xe6\x0f\x53\x27\x5b\xee\x17\xfa\x93\x88\xd1\x40\x53\x09\x8a\xed\xb9\x2f\x3b\x42\x11\x4b\xf4\x03\x1d\x73\x61\xcf\xbc\x8b\xea\x94\xf4\x61\x3f\xdd\xd1\x73\xcd\x10\x60\xad\xae\xda\x87\x46\x43\xac\x0d\x7c\x7b\x46\x11\x7b\xc3\xcf\x88\xf4\xf2\x89\x1b\xb5\x83\xe1\x70\x8d\x3f\x72\x9d\x60\xd0\xd9\x75\x6d\xe4\x3a\x78\x62\xd0\xeb\xe9\x7b\x4d\xec\xc9\x47\x47\x48\xba\x32\x3b\x48\x49\x4b\x7c\x36\xda\x09\x08\xfa\xdc\x64\x63\xcc\x9a\x1a\xe0\x27\xbf\x17\x63\xb1\x4b\x12\xaa\x4b\x5f\xc1\xe9\x45\x0b\xe4\x6b\x7c\x65\xa4\x9f\xbd\x92\xe2\x9e\xcd\x11\xe4\xc0\x9c\xce\x26\x10\xec\x6b\xa0\x2c\x29\x93\x49\xe2\x88\xee\x3c\x2b\xc2\x98\x1c\xc8\xf6\x5f\x53\x49\xcc\xf9\x17\xd6\xc3\xc7\xe6\xda\x4e\xe6\x42\x65\x98\xd0\x65\xbc\xd0\xd0\x72\x54\x1a\xe5\xf8\x9e\x51\x46\x14\x80\x7c\xc6\x7d\x77\x91\x7b\xa5\x8b\x45\x27\x74\x1a\x6d\xc3\xdd\xa5\xcf\x93\xe5\x36\xbb\x0b\x63\xd3\x9a\x3d\x23\x45\x52\x99\x45\x74\x84\x43\x41\xb5\x94\xe6\xcc\xdd\xd9\x39\x4f\xb2\x34\x99\x4e\xc7\xe4\x29\x8c\x8b\x39\x83\x69\x23\x31\xef\xbb\x1e\xe1\x1b\x5d\x92\x4b\xbd\xac\x23\x58\x4d\xff\x3e\x41\x75\x6e\x40\x6c\xaa\x52\x74\x5a\xe0\x7e\x92\x95\xcb\xef\xa7\xe3\x42\xf2\xa3\xfd\x54\x56\xaf\x81\x1a\x96\xa3\xcb\xe9\xfa\x73\x29\xd9\xda\x6d\x09\x74\x35\x65\xb3\x61\x73\x46\xdd\x9f\x4e\xfe\xfa\xee\xbf\x4e\x5e\xfe\xfc\x2c\x9f\xad\xee\x90\x5f\x8d\xe5\xdd\xec\x33\x33\x42\x57\xd1\x32\xc2\x78\xc4\x03\x69\x7b\x58\xde\xa3\x91\x9d\x26\xa1\x36\x42\x36\x56\x77\x34\x6f\xa5\xb4\x41\x8f\x84\x80\xd6\xd9\xcf\xf2\x1e\xfe\xa2\x18\xa6\x67\x6b\xea\x66\xd4\xe2\xa1\x64\xb2\x7a\x44\x21\xc9\xb7\x42\x5d\x12\xb1\x66\x12\xce\x1a\xc0\xc6\x48\xb9\x28\xc7\xe8\xc7\x3a\x05\xed\x36\x9d\x18\xd6\x33\x74\x9b\xb7\x4b\xca\x50\x22\x39\x47\x8f\x12\x76\x50\x0f\x0c\x07\xb1\x18\xd7\xd1\x9d\x03\x70\x3d\xda\x8a\x0e\xf8\x57\x57\xaf\xb1\x78\xed\x6b\x06\x7b\x30\xdd\x9a\x69\x2a\x9d\x2d\x97\x53\xc7\x67\xcf\xa9\xcb\x39\x97\x53\xe8\x23\x72\x5b\x8b\x00\x62\xdb\x1d\x8d\x73\xe3\xa1\xb9\x3d\x38\xc8\x74\x5c\x27\xb3\x99\x75\xbb\xa3\x3b\x53\xb6\x73\xd3\x5e\x0b\xb6\xd9\x36\x17\xb5\xc4\x5a\x25\xba\x25\xe1\x38\x30\xaf\x46\xaa\xa2\x0f\xe9\x55\xb4\x39\xf1\x09\xe0\x6a\xd1\xe0\x05\xc8\x9f\x27\xaa\x30\xbf\xc7\xee\x15\xf8\x67\xab\xef\x40\x5c\x95\x31\x30\x69\x0f\x75\x17\xeb\x8d\x51\x76\x8a\xab\x3b\xfb\xe0\x8a\x8c\x02\x10\xbd\x26\xe8\x59\xbf\xc7\x52\xfe\x34\xf1\xf3\x14\xc6\xa5\xcc\x8b\xbf\x5a\x6c\xf3\xa0\xe5\x00\xce\xbb\x5a\x9a\x0f\x67\x58\xd4\xde\x7c\x8d\xb6\x49\xe2\xfb\x2f\x00\x63\xe6\x9e\xb3\xf1\x9e\xd3\x1a\x5c\x34\x89\xda\x42\xa9\x92\xe8\x0e\x0b\xcd\x78\x19\xf3\x5f\x44\xf5\x58\x7b\x3d\xb8\x49\xf4\xd1\xcd\xc5\x26\xb9\xc8\x2f\xbf\x7b\xfa\xe2\xed\xb3\x27\x67\x2f\xfe\xeb\x99\xf3\x5f\xf0\xcb\x79\xf5\x8d\x33\xf1\x73\x2e\x0e\x0c\xc6\xe8\x72\x06\x5d\x18\x9c\x57\x9d\xb4\xfb\x4d\xb6\x7f\x98\xb3\xb1\xfc\x7a\xf9\xd4\x66\x4f\xfb\xc5\x79\x91\x0e\x7e\xe9\x41\x38\xeb\xf7\xfa\xd9\x61\x3e\x42\x22\xe8\x74\x04\xb4\xde\xf4\xef\x44\x1e\xff\xe5\x06\x48\x49\x32\x09\x42\x6d\xa7\x4c\x7b\x6d\xae\xd6\xcb\xcd\xed\x68\x3a\x5b\x2f\x36\x90\xbc\xb9\x82\xf1\x66\xc5\x61\x3e\xbd\xbd\x3e\x5d\x5d\x7e\x65\x79\x2e\xb4\xc1\x9d\x5f\x4c\x6f\x47\xd7\x93\xf3\xc3\xc3\xfc\xd9\x7f\xe1\xc0\x7f\x3c\x79\xf5\xf4\xe5\xb3\xb7\xa4\x36\xe3\xc7\x9e\x2e\xe6\x83\xd1\xc1\xdf\x87\x9d\x0d\x5a\x31\x1b\xd1\x74\xc3\x60\xf5\x01\x46\x66\xa5\x34\x17\x3e\xdb\x3b\x7c\xea\x6d\x22\xb1\xb5\x15\xf9\x8c\xd7\xf7\x61\xd1\x4e\x6b\x08\xd1\xa1\xf8\x6c\x8d\x3c\xf5\x86\x89\xe3\xca\x4d\x39\x77\x5b\xe6\xc1\x1e\x68\x7c\xa9\x85\x0b\x24\xc8\x24\x24\x10\xca\xe1\xa8\x0d\x27\x2b\xc3\xda\xe1\x4d\x14\xca\xd1\x26\x35\x1b\x49\x4d\xb2\xb5\x75\x4b\x4a\x6a\x33\xc2\x6d\xd2\xba\xe8\x64\x43\xdf\x7c\xd5\xb5\x29\x10\x57\x71\xc8\x76\x0f\x6a\x2e\xf7\x63\xdb\xf8\xb5\xa5\xe9\x19\x09\x3f\x97\x98\xf5\x7d\x15\xf7\x62\x3e\xaf\x50\x29\x5e\x9f\x6b\x5b\xec\xb9\x8f\xae\x64\x6b\xb7\x7d\x14\x5e\x0b\xe8\x1a\x32\xce\x53\xd6\xaa\xa2\xac\x99\xb1\x9a\xd9\x5d\xae\xa6\x8b\x15\x62\x21\xf5\x28\xa0\x26\x4c\x46\xe3\x6a\xd6\xc4\x12\x95\x1c\x7e\x6e\x36\xe1\xe6\x54\xb9\xce\xb2\x16\xb3\xd9\xd8\x2e\xab\x32\x9e\x1d\x59\xad\x26\xa4\x93\x2f\xd7\x65\x3d\x6a\xb3\x41\x7f\xf2\x7e\xad\x99\x95\x63\xe6\xd0\xbf\x81\xc4\xdb\x84\x9e\x42\xcc\x9c\x54\xdb\x61\x16\x6d\x6d\xde\x19\x51\x9d\xba\xb4\xde\x7c\x82\x98\x33\xb6\xe2\xff\xb5\xa3\x9b\x89\xf9\x50\x7e\x2f\x7d\x00\x4c\xb9\x6d\x08\x55\x00\xc1\x6e\xb8\xc1\xaa\x9a\x29\x09\x70\x6a\x77\xd7\xc2\x79\x58\xe5\xa5\xd8\x9d\x4f\xba\xf3\x10\xd8\xfb\x0d\xfd\x79\xa8\x9a\xa0\x43\x0f\x64\x8c\x1e\x94\x2d\x34\x30\xd6\xce\x9a\x81\x43\x42\xe6\xcb\xf0\x37\x30\xf3\x85\x11\x64\x87\x8c\xdd\x53\x38\xed\xcf\x3c\x72\x64\xe1\xe9\xc5\x04\x6d\x8c\x35\x7b\xad\x30\x70\xc7\xf4\xa7\x0e\x84\xd8\x94\x5a\x64\x70\x4c\xec\xa1\x59\xc3\x71\xe6\xd9\xdb\x52\x98\xd0\xbe\x33\x5b\x17\x3d\x56\xa2\x43\x0b\x35\xba\x49\xea\xbe\xf9\xda\xd9\x02\x7c\x3e\xcf\x6f\x47\x9f\xde\xc8\x49\xce\xa7\xd7\xf3\xc5\xca\x58\xf5\x43\x82\x69\xba\xb8\xab\xe4\x86\x7f\xa2\x76\xde\x83\xea\xac\xc9\x21\xe6\xc4\x22\x93\xd7\xe6\x50\xc5\xd7\x20\x32\xf3\x6f\x6c\x61\xef\x3c\x0b\x12\x3e\x15\x96\x7e\x5b\x96\xce\xe3\xb8\x28\x96\xe9\x27\x7b\x4c\x13\x4f\x75\xe7\xa7\x9d\x43\xe4\x40\x05\x55\x0e\xa8\x92\x21\xd5\xa0\x7d\x21\x9f\xe6\xdd\x8f\xe8\x9d\x22\xf9\xbe\x5a\x8e\xe6\x8f\xbf\x3f\xa4\x9f\x24\x73\x8a\xb0\x28\x88\xb8\x65\xd2\x0d\xab\x5b\x54\x50\xf0\x25\x40\xc6\xe7\xf3\xd2\x56\xff\xd0\xec\x86\x49\xff\xc0\x5c\xd7\x6c\xa1\x8b\xbf\x08\xb4\xda\x03\x79\xe6\x50\x1a\xdf\xa5\xfc\x85\xcc\xe9\x09\xfd\x24\x5e\x0c\x7e\x1f\x41\x3a\xf6\xa2\x0c\x8b\xf6\x6b\xee\x26\x30\x99\xb5\x1f\xc3\xf5\x2a\xc2\xb1\x88\xd4\xbc\xab\xb7\x59\x66\x9e\xa6\xdc\xe5\x19\x4c\x87\xbd\xf4\x38\x5a\xca\xcd\xe6\x4f\x51\x4c\x86\x0c\x3d\xad\x76\xf2\xb7\x74\x9a\xa9\xd3\x16\x1e\x55\x2e\x6e\x59\xd4\x4f\xc8\xe8\xca\x1b\x11\x70\x05\xc9\x65\x96\xcc\x42\x34\x5d\xed\x76\xf8\x6d\x0a\x71\xcd\x79\xb4\xdc\x68\xb2\x39\x88\x90\x59\xf6\xe5\x5c\x28\xf3\x41\xfb\x6e\x1e\x74\xd1\x19\x3a\x70\xfe\x9c\x08\x57\xd0\x78\x7c\x9c\xe0\x9c\x3e\x43\xa3\xcf\xc2\x60\x13\xe2\xe4\xe1\x0b\x45\xb4\xff\x8c\xb7\x9d\x7f\xe8\xa8\x07\x8d\xd5\x67\xc0\xb5\x1a\x34\x74\x81\x12\xe3\x67\xa6\x1f\x7e\x33\x70\x69\x31\x17\xc7\x61\x3a\xe7\x14\xc1\x60\x8d\x82\xb6\x34\x3e\x2e\x4c\xf3\x39\x12\x76\x17\x3c\x5c\x6c\x12\xd1\x34\x95\xc7\x86\x8d\xa8\x3d\xa9\xed\xc3\xb0\x94\xa8\x24\xca\x17\x1a\x12\x70\x95\xcc\xb9\xa2\x19\xf5\xaa\x0a\xea\x99\x43\x3d\xb4\x95\xc3\xca\x06\xf3\x61\xee\x07\xa5\x25\x07\xd0\xea\xd0\x8e\x30\x4c\xf0\x05\xfa\xa9\x0f\xb3\x1f\x98\x7e\xea\xe7\xa1\x14\x47\x46\xf3\xc9\xc7\xd4\x09\x6b\xc6\x05\x00\x8a\x06\x00\x2d\x3c\x27\xbe\x32\xc0\x3e\xe2\x9a\xeb\xf3\x5c\x9a\xda\xfd\x2e\xaa\xe7\xdb\x6c\xf6\x6a\x2b\xdc\x6e\xdb\x7d\xd7\xf7\x35\xa5\x76\xa9\xcd\x2a\xd3\xaa\x07\xbb\xc6\x5b\x91\x0a\x76\x4e\x0c\x00\xe9\xec\xea\x96\xf2\x59\x6b\x73\xe6\x40\x67\x94\xa7\xbb\xbf\xef\x63\x48\x47\xdd\xb6\x97\x46\xb9\x73\x6d\x8e\x9e\x15\x9d\x81\x25\x32\x77\x5e\xcb\x5b\x6b\x49\xb2\xa3\xed\x8d\xb4\x61\x1a\x83\x81\x66\xc5\x57\x4e\x9a\x7b\xd7\x69\x38\x67\xb2\xcc\x0e\x6c\xf9\x0f\x73\x60\x8d\x3d\xb6\x07\x2b\xdb\xea\xe5\xb1\xe3\xe0\x7a\x54\x38\xc7\xa7\xcc\x2a\xe7\x7d\xfe\x1c\x2b\x92\x30\xbd\x1e\xf3\x03\x5c\xed\x80\xc2\x99\xa2\x62\x24\x5e\x7b\xe2\x7d\xcf\x19\x4a\xec\x12\x71\x1e\x63\x92\xbb\xb2\x27\x37\x87\xaa\xb9\xdd\xa3\xb2\x9c\xf6\x0d\x74\x2b\xfc\xa8\x22\x38\x97\xd9\xf3\x6a\xc8\x00\x71\xac\x49\x62\x5b\xbe\xb5\xb3\x05\x19\xc6\x33\xe3\xb4\xcd\x73\xdb\xbb\xc1\x2d\x71\xb1\x06\xd4\xcd\x1d\xd0\x95\xfc\x26\x07\x90\xc2\xf7\x8e\x31\x1c\x3d\x8d\x93\xd5\x2d\x9b\x9e\xdc\x33\x3d\x08\xec\xea\xed\x48\xd0\x91\x61\x4b\x45\xe3\x85\x11\xe6\xb7\xe0\xa9\x11\x1c\x14\xc1\xa6\xd5\x65\xb6\x60\x15\x23\xa2\x04\x33\xc4\xcc\xee\x92\xd2\x84\x51\x5c\x55\x8b\x6f\x36\x76\x9f\x35\xcf\x5e\xc0\xc2\x36\xf5\xf4\xa3\x9b\x8a\x26\xd9\x5e\x9a\x4d\x5b\xaa\xbe\x97\x1f\xb8\x2c\x45\xad\x91\xbc\x1c\xbb\x2b\x3a\x77\xde\x95\x03\x5c\x86\x6b\xfb\x69\xb4\x24\x4f\x88\x40\xa8\xe0\x4f\x4f\x54\x68\x1c\xc2\x73\x2f\x8a\x30\x23\x63\x9c\xdd\x6e\x3b\x17\x7c\x85\x8c\x20\xb4\x5b\xee\x1c\x28\xbf\x13\xc8\x1f\xd8\x59\xc9\xf2\xe4\x59\xf2\xe0\x10\x1c\x62\x88\xfd\x61\x8b\x35\x73\x1e\xc6\xb5\x9a\x7a\x61\xce\x79\x4e\x36\x57\x2a\x1e\x95\xf1\x10\xf9\x1e\x6e\xcb\xf7\xef\x4b\x4e\x85\xcd\x4a\xbf\x7a\xb6\xdf\xbf\x7f\xfc\xbe\xf7\x5e\x91\x40\x2c\x75\x8a\x22\x48\xe4\xfe\x72\xef\x98\xe6\xe4\xd9\x7c\x2c\x9f\xc8\x6d\xc7\x18\xa9\x6b\xf0\x7e\x98\xef\xa1\x91\x8d\xcd\x06\xff\x3e\x2e\xff\xb8\xd9\x90\x17\x49\xf6\x0c\x33\x45\x0b\x8f\xc4\x4f\x70\x1e\x28\x4d\x9f\xcb\xa6\x99\x22\x3d\x33\xf4\xaa\x89\xdc\xb5\x1f\x5e\xbc\x7a\xfa\xe2\xd5\x9f\xd9\xc7\x9e\x2f\x88\x44\x03\xd5\x6a\xfd\xd2\xb9\x54\x79\x7b\x4f\xff\x00\x13\x7b\x80\x4a\x6f\x01\x5f\xe7\x15\xb5\x6c\x72\x3b\xd1\x8d\x94\x86\xbd\xc1\x67\xb0\x7d\x92\xd9\xe8\x99\xae\x96\x65\x58\x43\x27\xa1\xdc\x09\xfa\x03\x0b\x66\x8c\xdd\x8c\x99\x39\x9b\x9b\x2e\x1d\xb1\x3b\x3c\x51\x16\xf9\x7d\xd6\x49\x26\xf3\x71\x62\x7c\xb0\x36\xe6\xfb\x03\xba\x69\x7d\x68\xc2\xe2\xfd\xa4\xdb\x78\x30\x67\x15\x3c\xd7\xa7\x4a\x63\x58\xec\x80\x4c\x4f\xe1\xba\xb5\xdb\x2c\xd9\xc2\xfe\xbf\xfa\x0d\xde\xb8\x69\x47\x05\x96\xb9\xa8\xd2\x47\xa8\x64\xea\xbd\x86\xe6\x0d\x36\x88\xe8\xc4\xce\x75\xd9\x82\x5e\xa0\xc4\x0e\x9c\x22\xcc\x6a\x0c\x9c\x1b\x8e\x68\x74\xd8\x65\x8f\xb3\x2e\xe2\xae\xe3\xc7\x27\x03\x39\x47\x0f\xa1\xd0\xc1\xa2\xd9\xf5\xca\xc8\x09\x93\x77\x00\xcb\x77\xb0\x03\x1c\x8e\xb9\xea\xd1\x7d\x98\x3c\x14\x3a\xf7\x11\x24\xeb\xcb\xee\x61\x9b\x59\xdd\x35\x0f\xb1\xd9\xce\xe5\x55\xcf\xb0\x5f\x1c\xef\x93\x87\x41\x48\x3c\xf5\xbc\xf4\x54\xf9\xb7\xc8\x1a\xf3\x03\x76\x21\xdd\x8c\x94\x8b\x79\x85\x1d\x2e\x71\x34\x54\x3d\xc1\xde\xc5\x6a\x32\xfa\x55\xd5\xfb\xa0\x8b\x68\xc6\xa4\x71\x2d\xa3\x7b\xd9\x73\x03\x82\x3a\xfe\x58\x20\x71\x25\x73\xb7\xe3\xa1\x81\x67\x2f\xaa\x84\xdd\xf1\x22\x5c\x78\x78\x26\x8f\xbf\x3c\x93\x3f\xfd\xc3\x33\x09\xcb\x94\x05\xd4\x9f\x9a\x48\xf4\xb8\x4b\xb5\x58\xad\xd3\x8b\xcf\x5a\x7b\xc0\xa2\xf4\x8a\x11\xab\xc5\xdd\xf2\xf4\x72\x24\x6c\x15\xb7\x57\x75\x9f\x7a\x0a\x8d\x50\xb6\xf1\x64\xb9\xbe\x41\xb4\xed\xca\xc3\x22\xe1\xb9\xdc\x8c\x2a\x63\x8c\xb0\x16\xe5\xf3\x67\xd9\xfd\x78\x41\x66\x05\xe8\x86\x12\xad\x20\xc1\x41\x9c\x52\x10\xac\x2b\xf4\x97\xf4\x82\x04\xcb\x41\x9d\x56\x77\xd1\xe4\xac\x7e\x2c\x2f\xec\x10\x03\x7d\xe4\xab\x17\x0d\x51\xff\xee\xfe\x18\x62\x29\x47\x37\x69\x18\x54\x2f\x66\x36\xe0\x6a\x3e\xd0\xe3\x76\x9b\x06\xde\xe9\xe4\xcd\xb9\xb0\x4e\xc9\x73\x70\xc0\xb8\xa4\x41\x7d\x38\x42\x1a\x84\x4d\x7b\x3a\xbd\x40\xed\x86\xed\xc7\x1b\xe4\xc1\x51\xa9\xc7\x47\x62\xc0\x39\x2e\xaa\x88\x8c\xa1\xe6\xac\xf1\x21\x5a\x3e\x55\xdf\x60\xbc\xe6\x2f\xe4\x3c\x18\x75\x5c\x08\x49\x6a\x58\xd1\x66\xd6\x91\x72\x1f\x18\xcd\xf1\x9c\xc5\x2a\xb6\x95\x59\xfa\x3d\xe3\xcd\xd4\x35\xb4\x23\x08\xdd\x17\xab\x37\x9c\x8b\x2f\xe2\xd9\x06\x53\xcd\x95\x25\x7a\x52\xc7\xb7\xdf\xff\xed\x09\xa3\x73\x28\x2d\xaf\x26\xd7\xdf\xf2\xed\x1b\xf0\x36\x78\xca\x2a\xf4\x05\xbb\x76\xe1\x2f\xb1\x3c\xe0\x50\x4b\x5e\x14\x1d\xa1\xb2\x4d\xd3\xbc\x44\xb3\xb0\xf6\x98\x60\xee\xf2\x81\x15\xc3\xca\xea\xf5\x90\xac\xca\xee\x57\x10\x33\x0a\xb5\xd4\x80\xd4\x3f\x76\x2a\x6e\x1d\xa2\x1e\x6c\xbe\x79\x1c\x5c\xd7\x43\x1d\xf0\x73\x27\x3d\x58\xa0\xb5\xa5\xc0\x5a\xa0\x37\x96\x88\x6f\x1a\x5c\x6e\x7f\x62\xfd\xa0\xf3\x85\x0b\x3b\x6d\x35\x45\x05\x1d\x76\x6d\x58\x8a\x4d\x59\x16\x6d\x19\xcd\x04\x2b\x74\x37\xa3\x54\x95\xb1\x3d\xc5\x5e\xf2\x0b\xa0\x15\x62\xf2\x50\x92\x8c\x05\x7b\xf2\xe9\x5a\x8a\x0d\xda\x4c\xc7\xa3\xf9\xe4\x06\x3a\x06\x4c\xee\x17\xcd\x05\x93\x56\xeb\x50\xe0\x2b\x37\xc9\x7c\xd7\x34\x10\x34\x03\x9d\xe5\xee\xe8\x60\x07\x51\x85\xc3\x14\x6d\x4c\x4a\x27\x42\xa3\x92\xf9\x1f\x4b\x5f\x2c\x30\x04\x69\x63\xf7\xf7\xed\xdb\x97\xb4\x46\x7f\x37\x9b\x5d\xf9\xf2\x5d\x09\xe2\xd4\x72\x4f\x46\xbf\xa7\x63\xda\x01\xa4\x2f\xd7\xec\x48\xd7\xf7\x9a\xe1\x71\xde\x52\x8f\x3e\x68\xe7\xdf\xed\x18\x06\xcb\x79\x0b\x20\xc7\xef\xd6\xad\x8b\x09\x03\xe8\xbd\x44\xbd\x04\x19\x72\xc6\x1a\x70\x62\x93\x40\xee\xb9\x88\x95\xa2\x75\xe5\xdd\x90\xd1\x3a\x9a\xbc\x1b\x68\x75\xe6\xc9\x89\x33\xdf\x53\x56\xde\xa4\x5f\xda\x94\x64\x97\x2c\xb2\x46\xb4\x83\x11\xc3\x00\xce\xf3\x7c\xbf\x82\xe5\xc9\x50\xd0\x71\x7c\x89\x91\x29\x00\xdb\xaf\x86\xde\xb2\x95\x7d\x56\x00\xf2\x40\x5b\xea\x07\x10\xb0\xa8\x6e\x46\xb3\xd9\xe2\xe3\x93\xc5\xf2\x73\x1a\xc2\xc6\x90\x91\x22\x0e\xa9\x53\xcf\x9a\x0e\x72\x33\x29\x0a\x83\xd7\xde\x29\x85\xea\x7a\x0b\xd5\xbd\xa8\x08\xe7\x22\xee\x53\x44\xff\xbe\x7c\xfd\xe4\xe4\x65\x20\xa8\x91\x0e\xfe\x67\xd9\x1e\x66\xe9\x79\xbf\x9f\xe1\xe7\xf9\xc7\x6f\xf0\x17\x28\x1f\x7e\x51\x09\x58\x74\x1c\xb5\xd9\xd0\xa7\xb3\xb4\xbb\xab\x49\x71\x93\x6d\x25\x4f\xa7\xf8\x1a\xb8\x90\x55\x79\xe5\x34\x77\x51\x12\x9a\xd5\xf1\xb9\x85\x3f\xa3\x9a\x2d\x37\x3f\x59\x33\x4a\x57\xfa\xd2\xf2\x6e\x64\xc7\x92\x91\x3d\xf9\x91\x52\x69\x8a\xd0\x4a\x4f\x09\x9f\x73\xb0\x80\x20\x80\x22\x6a\x68\x22\x46\xf3\x1f\x0f\x95\xc6\x97\xe1\xee\x4f\x79\x58\xe8\x98\x0c\x20\x5a\x35\x70\xb5\x0d\xd1\x1a\xce\xa4\xa3\x0d\x52\x2d\xcc\x10\x48\xfe\x67\x52\xc8\x9a\x2c\x2e\x50\xe7\x98\x91\x10\xca\xe5\x66\x42\x44\xf3\xa8\x3a\x5b\x29\x6f\x69\x21\xdc\xa0\x7d\xa9\x01\x90\x4c\xad\x03\x41\x02\x15\x28\xcd\xf2\x08\x4d\xe7\xf2\xa0\x8c\x59\x5c\xb5\x7d\xc9\x4c\xa3\xfc\x59\x6a\x2a\xcb\x2c\x22\x9e\x94\x49\x81\xba\x8e\x32\x77\x00\x75\xe2\xa2\xa2\xac\xe3\xd6\xad\xe4\xe7\xcf\x5a\x13\x7e\x3d\x4b\x97\xb7\x0b\x14\x06\xa0\xd5\x9b\x4d\x4d\x0f\xdd\x6d\x88\xb2\x36\x10\x57\x3a\xe8\x74\x1e\x83\xbf\x39\xf4\x97\x6a\x07\x10\xf8\xec\x13\xdc\xbb\x15\x7a\x7e\x14\x94\xf4\xae\x02\xe8\x07\x4b\x7d\x13\xc3\x3f\xf6\x85\x32\x3f\xe0\xa2\xe4\x40\x37\x89\x67\x77\xe7\x11\x63\xb7\xff\xac\xcb\xf1\xdb\x3a\x5f\xed\xd0\x9e\x25\xdd\x32\xca\xc8\xf5\x35\x17\x77\xc6\xf4\x7c\x4e\xa0\x43\xe3\xb6\x11\xc9\x08\x32\xb8\x6e\xf6\xbf\xd8\x61\xb6\x2b\xe1\x56\xd0\xb6\x9e\xdb\xfe\xed\xae\x28\xcb\x6c\xc6\x6d\xb0\xc7\xda\x49\xf1\xe5\xed\x53\xeb\x99\xd7\xe8\x14\x23\x1f\xc1\x2c\x44\x73\xa4\x16\x34\xb6\xd2\xae\x9a\x15\x69\xbe\x3a\xa7\xd5\xe5\x92\xec\xf8\xb0\x3b\x73\x81\x02\x2d\x6a\xbc\xe5\xe1\x10\x4a\x1a\x05\x1b\x08\xb6\x56\xb7\xf5\xd4\xa5\x17\xad\xfb\x6e\xb7\x8b\x7e\xeb\x0b\x48\x7c\x04\xbb\x0b\x3e\xd1\x87\xfd\x43\xfb\xc7\x83\xc6\xdc\xb7\xc4\xce\x65\xd1\x08\x55\xdd\x7d\x63\x65\x8c\xcb\x37\xa4\x36\x88\x3c\xa9\x72\x90\x66\x7e\x11\x9a\x19\x29\xef\x19\xc2\x14\xf2\x5e\x2a\x37\x4b\xe1\xae\x18\xba\x83\xaa\x42\xae\x49\xc3\x65\xae\xdd\xa7\xdb\x9e\x41\x5f\x9a\x04\x82\x72\x00\x97\x65\x19\x20\x49\xa6\x04\x6f\x82\x50\x64\x68\x68\x1d\x0a\x3b\x03\x7f\x06\x4d\x32\x15\x38\xbb\x29\x3b\x90\xb5\xfe\x4e\x54\xab\xde\x84\x1b\xbe\x3e\xd7\x25\x9d\xb0\x63\x01\x7e\xd8\xd0\x47\x2b\x99\xe5\x53\x4f\x2a\x51\x93\x9c\x01\xbc\xe7\x3d\xde\x9c\x6d\xd8\xd0\x27\x12\x4d\xe5\x07\x50\x43\x68\xc4\x6f\xa9\xc8\x49\x61\xf4\xc5\xe4\xc2\xb7\x90\x80\x04\xdc\x0f\x69\x40\x4e\x53\x32\xa2\xdd\x8e\x30\xb2\x30\xd9\x22\x66\x0f\xc9\x62\xe5\xa4\xd3\x0a\x37\xe9\xd9\xe8\x3a\x35\x65\x68\x67\x3f\xfc\x04\xe6\x44\x0b\xbe\xf2\x19\x0c\xe6\xc5\x92\x3e\x6a\x96\xa7\x37\x7d\x8c\x0c\x92\x83\x83\x60\x56\x7c\xc6\xff\x4f\x4d\xcb\x76\xc7\x1b\x4a\x73\xf4\x66\x73\xbf\x75\x0f\x04\x00\x62\x42\xd8\xe2\x37\x5f\x88\xc8\x7b\xe1\xa5\xbc\x11\x7a\xcc\xa2\x57\x43\xc6\xd2\xf4\x69\x4b\xd9\x57\xe5\x41\x4d\x9b\x61\x17\x90\xdb\xd1\xfb\xee\xae\xfc\x0e\xff\xfd\x62\x0d\xb5\x8c\x81\x20\x4f\x19\xe1\xd1\xfb\x91\x81\x7f\x93\x35\x33\xfc\x8a\x2f\xb7\xda\xe0\xa3\x75\x15\x31\x33\xca\x9d\x5c\x8e\xfa\x73\x7e\xc0\x2a\x91\xc3\x5d\x7b\xf3\x34\x02\x3b\x5e\xca\xd4\xe7\x42\x91\x9d\xe0\x0d\xc7\xc0\x5e\x40\x1f\x2d\x07\x21\x48\x99\xcc\xc7\x11\xf3\x30\x10\x27\x32\xbc\x25\xcb\xea\x69\xe2\x4a\xb8\x5d\xe5\x15\x1a\x6b\x9b\xe6\x71\x5d\x2c\x35\xe3\xdb\x19\x7d\x23\x84\x3b\xce\x74\x93\x45\x28\x48\x79\x28\xda\xe3\x65\x2d\xc6\x48\xa1\x1a\xe8\xab\x7d\xfb\x79\x35\xdb\x6c\x9c\xdd\xe5\xa7\x77\xcb\xd9\xf4\x12\xb1\xe3\x04\xea\x39\x94\x4b\x7f\xdc\x12\xb1\x8c\x9d\xfb\x73\xdc\xb8\xdd\x8c\xad\xae\x70\x24\xd8\xf3\x50\xb2\xc8\xce\x25\xca\x3f\x49\xcb\x4e\x0e\x6a\xe7\x49\xf2\x17\x7a\xfe\x70\x8d\x52\x93\x15\x0e\x7e\x55\x17\xee\xcd\xf7\x1a\xa7\xc8\xca\xec\x06\x17\xf5\xce\x15\x32\x17\x70\x13\x4c\x29\x9b\x22\x11\x7a\xe5\x0d\x4b\xf1\x3b\x73\xcd\xbe\x12\xdd\xd0\xcb\xc6\x7b\xd6\x48\xf4\x06\x25\x86\x3b\x57\xe8\x2b\xca\xda\x39\xde\x3d\x60\x7f\x66\x71\x75\x13\xf5\x45\x8d\xaf\x41\x41\x75\xc8\xf3\xab\x0f\x51\x4b\x57\x64\x49\xa9\x09\x9e\xec\x1a\x40\x13\x98\x32\x1d\x4e\x04\xc4\x25\xa6\x1b\x8c\xf0\xa7\x35\x00\x5e\x3f\x8b\xe6\x0c\x7f\xed\xb9\xff\x4d\x00\x37\xf2\xa5\x08\x60\x15\xbf\xc8\xb1\x4e\xb4\xe2\x05\xfa\x26\xb2\x95\x45\x8b\x44\xee\x75\x32\xcb\x7e\x2e\x6d\x47\x90\x01\x2e\x10\xf9\x2f\x40\xf2\xa5\x01\xff\x59\xfd\xac\x3a\x7b\x3f\x6e\xd4\x61\x8d\x59\x03\xa8\x76\x82\xba\xee\x0a\x0d\x80\x77\x3c\xc7\xb5\x6b\xa1\xdd\x8e\x63\xf8\x1c\xde\xef\x3a\xf1\xc5\xee\x4b\xb2\xbe\x13\x8a\xa6\x9d\x54\xbb\x23\x8b\x5a\xcc\x16\x4d\x5a\xf8\xb5\x97\x09\xb2\x5e\x29\xed\xb4\x90\x3d\xe7\x35\xf2\xab\xd3\x70\xf9\xbb\xe8\xf9\x26\x45\xdf\x06\xbf\x75\xe2\x02\x38\xe5\x20\x51\x66\x4c\xe4\xdb\xc3\x23\xc9\x49\x7d\x68\xbb\x8f\x4d\x0d\xa3\x68\xb8\x25\xc4\x1a\x63\x83\x66\x87\xef\x52\xbf\x1e\x17\x02\xdf\x60\xd7\x66\x45\x3d\x7b\xdc\xde\x78\x32\xd7\x87\x4a\x55\x0d\x8b\xef\x8f\x40\x67\xc2\xdb\x61\xa9\xe1\x1c\x66\xf4\xf1\x62\xb2\xeb\x54\x71\xf8\x15\x55\xdf\x49\xc4\x99\x6a\xb0\xb2\xc1\xd9\xd2\xea\xf0\x60\x1d\xef\xf9\x6f\x41\x4e\x36\x9b\x63\x7c\x06\x37\x5b\xc1\x89\xcc\x34\x93\xdb\xeb\xe5\x6c\x85\x02\xf9\x3a\xe2\x26\xba\x9a\x1d\x0b\xdc\x8c\xe0\x7b\xf2\x69\x74\xb9\x9e\x7d\x6e\xc1\x56\x6c\x21\x37\xd7\xb9\x89\x6b\x91\xcd\xe7\x10\xbd\x4d\xc8\x4a\xc7\x8e\xc3\x1f\xac\x95\xdd\x22\x3d\xc1\xa1\xcf\xec\xfa\x01\x5d\x8c\x80\xae\xb8\xdf\x6e\x7b\x1e\xd7\xb2\xc8\x99\x98\x36\xaf\x4b\x2b\xd9\x76\xec\x23\xf0\x92\x5c\x60\x4c\x3b\xc7\x79\x0d\xc5\x3b\xc0\xe8\x8c\x6e\xff\xa0\x17\x10\x75\x3b\x59\x5d\x4f\x6c\xa4\xb0\x8e\x6b\x7c\xe5\xb0\x60\x23\x26\xc9\x0f\x9b\xf5\x83\x1b\xed\x0a\xf2\x39\xd8\x84\x1d\x64\xff\x27\xcf\xa4\x6e\x78\x7a\xfa\xd9\xb5\xdd\x03\x69\x41\x69\xe3\xcc\x77\x38\x6d\x98\xff\x86\xd9\x9f\x86\xd0\x2c\x7a\xcf\x0c\x76\x51\x5d\xf4\x79\xc7\xdb\xe5\xff\x6d\xb8\xde\x4c\x4a\xb0\x8d\xfa\xab\xba\x8a\x99\x25\x91\x6b\x89\x0f\xc0\xb9\xfa\x84\x20\x32\xfc\xdc\x1b\x60\x23\x89\xc2\xbe\x79\xa8\x65\x3d\xe2\x5d\x4f\xe2\xc5\x4c\xb8\x02\xa6\x84\x10\xda\xf8\x28\xab\xc1\xe6\x07\xd2\xdf\x40\x78\x07\xd7\xda\xd6\xee\x76\xbe\xc7\x45\x0c\x21\x92\xee\x24\x59\xe4\x18\x9b\xfa\x09\xed\xa2\xde\x8e\x3e\xd5\xf0\xac\x06\xed\x3c\x6f\x0f\x26\x96\x13\xaf\xd3\x33\xed\x76\x2d\xaa\x6b\x65\xfa\x23\xb1\xcf\xb2\x61\xab\x08\x73\x25\xd8\xaf\xbe\xf0\xd6\xbe\xa3\xbb\x32\xa9\x81\x65\x2c\x76\xe8\xac\xdd\x3d\x24\x5b\x45\xb3\xeb\x04\x13\x81\xb8\x74\x72\x7b\xea\x7b\xb7\x2c\xcb\x58\xac\x46\xe7\x02\xe5\x44\x9d\xc4\x26\x8b\x8d\x42\xa1\x2f\x2b\xa5\x66\xca\x02\x69\x80\x1f\x95\x51\xe6\x44\x93\xb0\x46\x2b\x35\xfb\x0a\x5a\x1b\xcf\xc3\x0e\x72\x3b\x4f\xcd\x44\x58\x33\x3e\x9b\x8d\x49\x68\xa2\x78\xd1\x55\xc2\x5e\x83\xd2\xa3\x33\x5f\xe4\x6c\xa6\xb1\xe5\x00\x3f\xa3\x96\x04\x2b\xe5\xe5\xdb\xc7\xe4\xf7\x42\xe9\x17\xe1\x32\x10\x95\x5f\x98\xc5\xc0\xa7\x54\xb3\xbe\xb1\x72\xa5\xbe\xbe\xb9\x88\x2f\x68\x5b\xca\xfa\xdd\x86\x7e\xf2\x77\x5d\x59\x63\xb4\x1f\xb8\xba\x64\x8e\x34\x04\x30\xa9\x84\x5f\x7e\xe7\xcc\x21\x99\x62\xe0\x57\x62\x1c\x15\x41\x51\xfa\xe5\x9e\x22\xc7\xd6\x2a\x2c\x4b\x2d\x90\x59\x87\x64\x1f\x88\x31\x08\x79\x89\x03\x9c\xcd\xd5\x25\x99\x4e\xd0\x67\xea\x4e\x99\x26\xd5\xfa\xf3\x6c\x82\x26\x54\x21\xa1\x9f\xf4\x92\x02\x69\x8b\x8e\xe6\xcc\x72\x6a\xb6\x9a\xac\xbd\x85\x8d\x1c\x00\x80\xf4\x9c\xf3\x20\x57\x56\xfb\x03\x09\xcd\xfd\x61\xa7\x9f\xdc\x4c\xda\xac\x69\x24\xa2\x94\xd0\xe2\x40\xb2\x0f\xcb\xd4\x7e\xf5\xed\x07\x12\x41\x05\xfa\x46\xe5\x62\x85\x1b\x09\xb6\xe0\xb5\x94\xd6\x68\xbd\x8c\x93\xf2\xc6\x58\x34\xe4\x94\xb8\x4a\xf6\xb9\x02\x3f\x6d\x9b\x0d\xce\x40\x83\xfb\x81\xcd\x86\xfa\x66\x8c\x7d\xc8\xe2\x71\x54\x30\x43\xdb\xac\xa6\x0f\xd5\x7c\x43\x47\x77\x30\x5f\x2f\xc1\x13\xfe\x57\x5f\xbe\xbb\x04\x87\x88\x4f\x76\x05\x70\x5a\x7b\xf0\xca\xeb\x4d\x04\xf1\x4f\x8c\x50\x01\xde\x39\xec\x9d\x66\x30\xcc\x2f\x26\xb0\xda\x6e\x00\x4f\x1e\xa0\x42\x91\x0d\x48\xc6\x6d\x1b\xd0\x96\x4a\xdd\x5f\xe4\x63\x38\xc8\x1f\x26\xe3\xd3\x20\x9f\xb1\x3c\x53\xab\x24\xbf\x37\x08\x19\xab\x27\x98\x47\x19\xfa\x16\x74\x89\xc5\xea\x3d\xb6\x85\x06\x15\xcd\xd5\x5c\xaf\x3a\x40\xf5\xfa\x0f\xa7\xa7\x0d\x8b\x95\x15\x0f\x97\xe9\x39\xab\x15\x4d\x94\x23\xa9\xd7\x12\xa4\x46\x5d\x59\x0c\x9c\xad\x80\x0c\x98\x8c\xdf\x4e\xaa\xc5\xdd\xea\x92\x5a\xb5\x3d\xcc\xef\xc9\xfa\x45\x11\x6a\xe4\x6e\xb3\x6e\x75\x77\x79\x39\x81\xc3\xe5\x15\x92\x98\xb6\xd1\xa7\xb0\x10\x95\x09\xf0\x66\x4f\x3e\xf5\xc8\xee\x1c\x95\x6b\x24\xd4\xb4\xce\x86\x25\x0a\x48\xb5\x87\xe9\x31\xad\xe5\xff\xf7\x84\x58\xc3\x2c\x10\xee\x60\x17\xac\x57\x9b\x6e\x43\x65\x05\x5c\x9b\xe0\xd4\xef\x24\xdc\xf2\xaf\xa5\xbd\xea\xeb\xfc\x00\x5d\x25\x42\x8c\xb5\x12\x4c\x3c\xd9\x25\xd9\x09\x0b\x1a\x78\x23\xba\xd0\xf4\x68\x63\xe0\x80\x7a\xdc\x69\x82\x03\x59\xbe\x03\x56\x95\xbf\x4d\x3a\x73\x07\xa2\x1e\x4e\x6c\x03\xa4\xf9\x6d\xc0\xd5\x2b\x8b\x37\xaa\x0c\x92\x50\xf3\x34\xbb\x27\x01\x5c\xbb\x4f\xf1\xdd\xc1\x94\x00\xfc\x2a\x06\xa5\x78\xc7\xee\x82\xce\xbb\xf5\xbc\xf1\x99\xd7\xbc\x1f\xd6\xe9\x94\x9e\x03\xec\xde\x51\x33\x61\x26\x84\x8e\xfb\x44\x05\xd5\xe1\x72\xbf\x54\x6d\xed\x7a\x46\x8c\x79\xeb\x47\xd4\x90\xc3\x3c\x3e\x37\x17\x6f\xba\x51\x7a\xcd\x1d\x80\x53\xbf\x73\x23\xe2\x73\xb3\xd6\xb6\x8b\xa9\x17\x1c\xbc\xa8\xeb\xca\x31\x6d\x6e\xd9\x77\x35\xdb\xb9\x55\xd3\x9d\xd7\xea\x43\xb2\x7a\x7e\x7e\xb2\xad\xbf\x7e\xf9\x3e\x13\xb3\xe7\x46\xa4\xb0\x5a\x2e\xe6\x15\x82\x06\xa8\xe6\x66\x32\x42\x4f\xa7\x58\xc3\xd5\xf4\xda\x79\xba\xa8\xc3\xbe\xc5\x68\x1c\xb8\x18\xc3\x88\x96\x42\x2b\xf1\x78\xc1\x95\xa0\x49\x64\x42\xf3\xb4\x45\xa6\x62\xea\x63\xa1\x2d\xbe\x63\x18\xf7\x6e\x18\xfd\xd4\x2f\x39\xa1\xdf\xa2\xd1\x1e\xc5\xce\xdd\x04\x9b\x48\x53\x79\x2d\xcd\x34\x96\x15\xff\xe0\x6a\x3c\x34\x02\x23\x1c\xee\x55\x19\xd2\x51\x7e\xa1\xd6\x79\xae\xae\xca\x0b\x47\xea\x1c\x8c\x5c\x50\xaf\xfd\x23\x7a\x21\xb9\xba\xea\xe3\x9f\x62\x44\xf7\x03\x6e\x5e\x0a\xf4\xf9\xfb\x7b\xf9\x3a\x38\x2e\x8e\x21\x0b\x91\x45\x07\x17\xfc\x6b\xc8\xd5\x1a\x1b\xea\xe3\xcd\x68\xed\x00\x53\x13\x03\xca\x49\x38\xc3\x95\x5c\xcb\xb7\xe3\x86\xbc\xfd\x7f\xb9\x7b\xd7\x85\x36\x92\xab\x51\xf4\xf7\xde\xfb\x25\x44\x87\xc8\xdd\x56\x21\x10\x9e\x64\x27\xad\x69\x6b\x33\x18\xc7\x4e\x8c\xe1\x03\x26\xc9\x04\x34\x8e\x40\x0d\xc8\x16\x12\x23\x09\x6c\x82\xf4\x2e\xe7\xc7\x79\x92\xfd\x64\x67\xdd\xaa\x6a\x55\x77\x0b\xb0\x27\x67\x7f\xe7\x3b\x93\x18\x75\xdd\x56\xad\xba\xad\x5a\x55\xb5\x2e\x37\xc3\xd9\xa0\x4f\xaa\x0f\xbb\xf8\x79\x3d\xcc\x95\x56\x47\xed\x18\xe6\x88\xc1\xdd\xaf\x0b\x08\x7d\x42\x65\x07\xdc\x41\xef\x37\x17\x35\x92\x06\x7a\x81\x2e\x53\x8a\x35\x15\x0d\xe1\x50\x90\x90\xd7\xb7\x13\x4a\x58\x42\x9d\xd0\x9f\xa2\x65\xe3\xd9\x5c\x25\x01\x08\x54\x32\x10\x08\xc4\x4c\xe4\x14\x32\xc8\xa3\x9e\xfe\xe4\x5c\x78\x6f\x47\x30\xdd\x30\x45\xa3\x46\x05\x95\x05\x9a\xad\x4a\x82\x8d\x55\x2c\xac\x99\x0e\xd2\xc5\x46\x99\x4a\x91\x30\xb3\x52\x35\x12\x1d\x91\x40\x67\xdb\x66\xe2\xca\x03\xdc\x0a\x6f\x9e\xd5\x50\x5c\x2d\x09\xbd\x83\x3a\xe0\xa1\xc0\x5d\x00\xb7\x24\xa4\x39\x92\x8d\xc3\xe9\x18\x59\x29\xcd\x45\x70\x8e\xf1\xcc\xa9\xec\x7e\x5e\x67\xc7\xe9\x1e\xb1\x62\x99\x08\xaa\x45\x5f\x70\xb5\xbe\x61\x75\xbb\x72\xb6\xf9\x3c\x7a\xbb\xfb\x27\x38\xb5\x16\x74\x46\xf1\x81\x11\x0e\x53\xcb\xca\x8c\xd0\xe8\x4e\x65\x62\xd2\x21\x26\xfa\x60\xe7\x70\xef\xc7\x83\xed\x9d\x0f\x3f\x1e\xbc\x4b\xd9\x76\x63\x30\xa5\xbe\x56\x09\xcf\xc9\xf0\x3e\x30\xbb\xec\x39\x99\xae\x91\xc3\x51\xc4\xc5\xc7\xab\xea\x9a\xcf\xdd\x23\xba\xd9\x88\x0e\x77\xde\xed\x6c\x1f\x51\x44\xd8\xfe\x25\x0b\x73\x9a\x0f\x09\x0a\x2c\x4c\x91\xd6\x45\x0a\x3e\xbb\x04\xae\xd5\x42\x7f\xe6\x15\x94\x58\xcc\x73\x66\x7d\x44\xe7\xfd\x66\x4d\xc8\x9b\x90\x79\xbd\xf4\xb8\xda\xf6\xf2\xa5\xb0\x86\xe6\xf9\x1f\x59\x0d\x5a\x8c\x48\x3c\x91\x7b\xe9\x5e\xd6\x8a\x55\x11\x28\x96\x50\x88\xca\xee\x17\xd4\x7d\x4b\x6d\xcf\x89\x72\x2c\xdd\x81\x55\xf7\x11\x34\x64\x7c\xc5\xbe\xf4\x49\xba\xd1\x8e\x03\x1a\x26\x25\x42\xf5\xe6\x68\xf7\x5d\xed\xd5\xde\x2e\x7b\x9c\xf7\xdd\x35\xad\xc1\xea\xaa\xa1\xf7\xa0\x21\x99\x5f\x6e\xd6\x6a\xfb\xc3\x1c\xf5\xf4\xd0\x06\x33\x76\x32\x2c\xae\x1a\x62\x49\xa0\x62\x38\x6d\x5d\xa2\x53\x43\x88\x3d\x03\x6a\xfc\x89\xac\x0d\xc1\x2e\x5c\x43\xc7\xf3\x23\x8a\x4a\x6c\x5c\x33\x4a\xda\xff\xfd\x81\xb9\x83\x48\x88\xf1\xb6\x95\x0d\xb3\x64\x95\x71\xab\x4d\x81\x78\xa9\xb2\x59\x90\x64\xf7\xd7\x58\x4b\x58\x53\x3e\xb4\x82\x54\x8c\x83\x33\x7e\x92\x90\x83\x0b\x00\x81\xd7\xbd\xa5\xb1\xa9\xd7\x8b\x31\x62\x54\x4c\x64\xb5\x45\x0c\x3d\x79\x1a\xc1\x61\x58\xd3\x3c\x70\xe9\x8c\x8e\x49\x92\xc0\xa9\xf0\xd2\xc3\x91\x4c\x35\x38\x02\x1c\x90\x25\x50\x7c\x09\x7a\xef\x88\x31\x69\x33\x9c\xa3\xaf\x21\xc9\x6f\xb3\x65\xc5\x72\xfc\x6e\x8e\x5f\xec\xc5\xa3\x98\x2e\x2c\xb2\xc8\xf5\x66\x55\x30\x85\xdc\x63\xed\x38\x77\x35\x9a\x89\x32\x03\xa3\xe3\x4b\xb7\xae\x85\x72\x78\x18\xa8\xac\x4b\xfc\x08\xb9\x6c\x8d\x46\x37\x93\x96\x3b\xb1\xb8\x8f\xd9\xc0\x7c\xdc\xcc\x3e\x36\x54\xbb\xd6\x5a\xa8\x12\x5f\x85\x82\x55\x88\x87\x22\x80\x07\x86\x36\x3b\x41\x1d\x1f\xbb\x59\x18\xde\xec\xa6\x62\x01\xb5\x90\xaf\x5d\x01\x7f\x2d\x0b\xb0\x60\xc9\xe4\x05\xf7\x58\xbd\x2e\x1b\x9a\x8c\x32\xf1\x65\xb1\x34\xa7\x72\xf8\xf8\xcd\xf2\x7c\xd2\xbb\x08\xfd\x7b\xb0\x74\x85\xb5\xe5\xfe\x5a\xd2\x63\x38\x75\xca\x27\xba\x3d\xca\x47\x7d\xae\xa1\x12\xb2\x9d\x40\xc7\xcc\xfa\x37\xf3\x2f\xd7\xbd\x51\x7f\xdc\xad\x1c\x86\x62\x26\xd7\xf7\x9f\xb2\x96\xf9\xf4\x69\xd9\x2c\x6a\x7f\xfa\xf4\xf2\x53\xfb\x93\xb5\x3e\x60\x6f\x79\x4b\x73\xf2\x53\xb7\x2d\x07\x10\xcb\x11\x59\x03\xc2\x68\xb3\xba\xa2\x45\x36\x9b\x35\x4d\x5b\x01\x71\x51\x31\xf3\xed\xd4\x29\xad\x26\xc1\x37\x6b\x91\x75\x5a\x7f\x4b\x90\x15\x9d\x91\x09\xa9\x67\x4b\x77\xab\x85\x06\x89\x01\x3c\xcc\x42\xc4\x1f\xc5\x9f\x16\x6d\x0f\x4d\xb9\x9a\xbc\x5f\x75\x37\x4d\x5e\x80\xc0\x6d\xed\x66\xd5\x1a\x9e\x4a\x95\xb1\x1d\x08\x02\xab\x92\xdc\xdb\x2f\x34\x9e\xc5\x5f\x82\xfe\xcb\x8d\x7a\xdd\x9a\x9e\xf3\xa6\xab\x42\x5c\x8d\x83\xb3\x80\xa3\x18\x4f\xd5\x5f\x57\x91\x02\xf2\x50\x5d\xd3\xbc\xca\x92\xb5\xf9\x8c\x9a\x7d\xd8\x45\x4e\xcf\x47\xa9\x7d\x92\x7f\xdd\x57\x64\xac\x3d\x1f\xa1\x2b\xe6\xd9\xa4\x05\xbb\xf8\x64\x53\x29\x30\x92\x92\x34\x65\x9c\xb6\x32\xcc\x20\x0e\x28\xd6\x4f\xa6\x8d\xf5\x44\x52\x36\x31\x65\x33\x48\x41\x7f\x5d\xf9\x24\x75\xce\xc9\xc9\x32\x8e\xc0\xd1\x86\x71\xee\x6d\x0e\x4a\xcb\x24\x07\xbe\x0e\x7d\x84\x22\x1f\xa5\xc8\xa6\x23\x2f\x4c\xdc\x38\xb3\xe4\xde\x04\x5a\x91\xe0\x09\x6b\x30\xba\xc9\x6b\x54\x6f\x9b\x71\x67\xa6\x83\x72\xb9\x47\x16\x4e\x41\x3b\x05\xfa\x3d\x41\x34\xe9\xac\x9e\x61\x14\xb1\x31\xbc\x9b\x09\x91\xe9\xb0\xdf\xe5\xea\x9f\x4b\x27\x94\x99\x33\x04\x43\x55\xe8\x5b\x01\x25\x3b\x93\xf5\xff\x18\x89\xf5\xcb\xe6\xaa\x9f\x4f\x85\x82\x3c\x8e\x52\x5c\x17\xa4\x8b\x34\x51\x66\xb7\x3c\x2b\x4c\x06\x63\x79\x3f\x73\xca\xc6\x24\xfe\x92\xdf\x65\x15\xa6\x25\xc2\x36\xe1\xbe\x45\xde\x93\x7d\x21\xe0\x05\x0a\xed\x86\xf5\x75\xad\xac\xa4\xbb\x29\x95\xf9\x42\xdc\x1c\xfd\xa2\x61\x33\x75\xfc\xea\xe5\x64\x9b\x90\x3a\x5d\xaf\xac\x90\xc5\x95\x05\x0e\x63\x59\x69\x6d\xc6\x04\x51\x23\xab\x25\x89\xeb\x04\xc5\x09\x87\x8d\x01\x46\x26\xda\xd2\xac\xb2\x37\xe3\x81\xf3\x41\x4e\x12\x61\x3a\x1f\x21\x68\xba\x40\xe7\x84\xa6\x5b\x48\xb8\x56\x8f\x03\x7a\x29\xc2\x2b\xf7\xe1\xad\xf5\xf6\xc1\x0e\xa5\xc8\xde\x45\x90\x13\x4f\x27\xaa\xea\x7a\x7d\x25\x48\x16\xcd\xbd\xe5\xb6\x4a\x71\x0a\x7a\xd4\x96\x14\x7f\xc0\xb2\x68\x92\xc8\x50\xab\x61\xcb\xa2\x9b\x11\x9e\x15\xd3\xa8\x11\x40\x83\xae\x75\x24\x05\xda\xb1\xd2\xc2\x66\xdf\x0c\x87\x59\x66\x97\x4e\xd9\x29\x4b\x61\x16\x89\xa9\x75\x5c\x43\x8e\x26\xa5\x55\x2b\xcc\xa9\x00\x72\xf7\xb1\x93\x39\xcd\xdd\x4b\x21\x17\xd1\x0e\xd8\x4b\x77\x8f\xaa\x38\x4c\x9a\x55\x81\x4b\x31\x76\x63\x28\x23\xf4\xa8\x51\x5c\x20\xb4\x02\x2c\x24\xb6\xe7\x5a\x0b\x96\x7d\xe0\x15\x8f\x2a\xd3\x8a\xb3\xca\xb4\x78\x58\x31\x43\xe7\x8c\xaa\x88\x76\x81\xcb\xa6\xa1\x02\x26\xdb\xdb\x85\x0a\x7c\x36\xa1\x7f\xb3\x6a\x17\x91\x4a\x53\xce\x97\x11\x46\x7d\x3e\x47\x77\xd4\x24\x2d\x2a\x6f\x8e\xe8\x24\x8d\x7d\x2f\xd1\xe9\xee\xf0\xee\xea\x74\x3c\x0c\x0e\x1b\x4d\x95\x00\xfc\x04\xf0\x10\x55\x99\x5c\x34\xbd\xd5\x95\x9e\x81\xb2\xe8\xfe\x3e\x22\x7b\xff\x16\x14\xcc\xe9\xc5\x22\x42\x3f\x1d\x52\xb0\x33\x80\x72\x33\x3c\x3a\x7a\xf3\x79\x56\xa6\xd0\xaa\xb1\xb9\xb7\x1c\x67\x97\xe8\xe4\xfe\xe4\x7e\xfd\xc2\x28\xc0\x89\x4f\x5c\x2c\x20\xc9\x55\x00\x83\x5b\x30\x9d\x94\xad\xff\xcc\x56\x8c\x8e\xb7\xd6\xfe\xd1\x5d\x77\xc6\xf4\xf9\x84\xa8\x4d\xc3\x97\x79\x8c\xd0\xd9\xf3\x19\x04\x86\x64\x61\x68\xa4\xcd\x26\xed\x1f\xec\xbc\x7e\xfb\x77\x39\x8b\xa2\xd8\x5d\x12\x58\x9b\xb7\x77\x86\x05\x77\xb5\xca\xd4\x04\x9a\x99\xdf\x7e\x7f\x74\x40\x2a\xbd\x68\x48\xfd\xe4\xb0\x91\xc4\xb0\x01\xf7\xa6\x68\x32\xfe\xe4\x33\xda\x7f\xb7\x86\xd4\xad\xe9\xe5\x82\xef\xab\x33\xef\x56\xdb\x9a\x57\x5d\x66\x29\xfd\x4c\x2b\xf3\x7b\x53\x8d\x74\x51\x22\x8f\xad\x0a\x39\x3e\x61\xa6\x2a\x46\xce\x86\xaa\xc6\x07\xac\x1a\x5b\x0f\x36\x55\x16\x86\xc5\x03\x5a\xc9\xf0\x47\xee\x94\x56\xab\xdd\x98\x8b\x29\x33\xe5\x48\x9c\x67\xd5\xf9\x80\x95\xd3\x9d\x1d\x06\x0f\x08\x89\x23\x8b\x73\xf8\x38\xa1\xa9\xae\xe7\x13\x0d\xd2\x29\x46\x2b\xd8\x4e\xb9\xda\x78\x20\x4a\xa4\xbf\x24\x95\xa3\xc7\xa4\xa3\x3b\x50\x25\x74\xd1\xf3\xf8\x0c\xed\xa2\x68\x4d\xb8\xa0\x71\x2b\x28\x7b\x20\xb9\xac\xb7\x9e\x42\xba\x71\x06\x75\x5f\x07\xdd\x57\xc8\xe6\x5d\x2b\x07\xb6\xdd\xbd\xeb\xf8\x72\xcf\xab\xe6\xd3\x8d\xd5\x0a\xc7\xc3\xc2\x1e\xd3\xbc\x81\x7d\x10\x79\xf5\xf1\x79\x2d\x68\x40\xc9\x1f\x8d\x9a\x73\x78\x1d\xc3\x6a\xa6\xdb\xe4\x61\xbd\x06\xb5\x8e\x27\x33\xb5\x22\xe4\xbd\xb5\x37\x65\xd5\xe4\x95\xda\xfb\x71\x8d\xe1\xd6\xb8\x56\xeb\xf0\xba\x5f\xbb\x1d\xf4\x6a\xff\xe4\xaa\xff\xd9\x8c\x74\x83\xe7\x73\x35\xd4\x34\xf3\x55\x53\xda\xa1\xe2\xa1\x4f\xe9\x66\x05\x1f\xf2\x6e\xda\x2d\x02\x2f\x12\xf6\x58\xf9\x98\xe7\x26\x37\xa9\x0b\x93\x3d\x70\x02\xe8\xdc\x0d\x87\x9e\x2a\x76\x0a\xbb\xd7\xb2\xba\xc8\x37\x94\x5f\x60\x10\x2c\xaf\x29\xed\x25\x57\xdc\xfd\x92\x9f\x27\xe7\xe4\x37\x09\xda\x47\xaa\xc8\x6f\xf8\xf9\x28\x96\x67\x24\x5e\x85\xc2\x39\x9a\x81\xa1\x3c\x7d\xe5\x6f\x44\xf2\x75\x62\xe7\x87\x80\x23\xac\xb7\xbb\x93\x51\x94\x68\x77\xb4\x23\x14\x0a\xc0\x37\x3f\xe5\xb1\x2e\x85\x2c\x50\x85\x72\x60\x4d\xa2\x01\x94\xc9\x99\x86\x1b\x24\x6c\x04\x23\x2b\xa5\xb1\xe4\x30\x31\x50\x31\xe3\x47\x7b\x60\x47\x7d\x37\xb2\xc8\xd4\x48\xce\x27\x55\xb1\x19\xb9\xea\x4d\xa4\x51\x89\x24\xf9\x0e\x91\xa6\xfc\x89\x97\x61\xd0\x23\x12\x00\x3a\x9a\x39\x6a\x6a\x33\x74\xe4\xc3\x9b\xf8\x2c\x59\xc8\x0e\xb6\x18\x0f\x0b\x5d\x4c\x78\xc0\x95\xe3\x91\x10\x6d\xee\xf8\x6c\xc7\xbe\xd7\x08\x2a\xb0\x1b\x64\x48\xd3\xe7\x50\xb7\x68\x24\x28\x83\x9e\x3a\xd0\x62\x4c\x8c\xcf\x07\xee\xbd\xf0\x7c\x34\x55\x86\xf0\x5f\x7b\x4e\x0b\x1a\x04\x7f\x82\xcc\x49\xea\x86\xfb\x5c\xbb\xa0\x45\x66\x8a\xbc\xe6\x00\x33\x12\xe4\x17\x67\x51\x6a\x93\x04\x7a\x2d\x42\x2a\xe8\x0b\xe7\xc6\xd7\xcd\xc1\x97\xd9\xe6\x06\x9c\xa5\x5f\x6c\x6c\xbc\xe4\x08\xb5\x44\xde\xcc\x66\xd7\x85\x8d\xf5\xcf\x87\x7b\xef\x3f\x1c\x1e\x6d\x1d\x1c\x89\x99\x8f\x93\xe3\xf9\xc9\xfd\xf1\xcf\x27\xf7\xdd\x64\xdd\x50\xea\xce\xfb\x57\xd9\xfa\xf1\xc9\xe2\xa4\xdb\x65\x83\x1f\xfb\x07\x7b\x47\x68\x61\x0d\xd2\x78\x37\xc7\xa2\xc9\x49\xf7\x64\xf1\xcc\x74\x4e\x46\xeb\x66\x7b\xef\xfd\x11\xde\x3e\x1f\xfd\xb4\xbf\xf3\x61\x6b\x7f\xff\xdd\xdb\xed\x2d\xca\x8e\xf0\xb2\x7b\x52\x80\x86\x55\xb4\x86\x22\x26\x30\x87\x71\x81\x0d\x58\x78\x71\xfd\xe3\x74\x3c\x6a\xa3\x14\x19\x1c\xee\xb3\x9b\xd9\xf9\xda\x1f\x22\x74\x31\x44\x5a\xf8\xc2\xfd\xba\xd0\xbd\x1b\x94\x03\x79\xc8\x4d\x8f\xbd\x16\x3b\xf6\x9a\x1f\x16\xd9\xe3\x28\x16\x05\x22\xb1\xaf\xf1\x8f\xe2\x4b\x0a\x8d\x42\xde\xc4\xf8\xee\xe1\x6b\x73\x29\x6f\xfb\x25\x88\x64\xa0\xe7\x93\xf1\xd5\x9f\xa7\x16\x81\x84\x87\x6f\xd1\x35\x0a\x57\x72\x72\xa7\x51\x55\x78\x5a\x5d\x40\x80\xb7\x02\x93\x89\x6c\x8b\x25\x9d\xd9\x98\x41\x26\x69\x1f\x40\xd9\x55\x82\xee\x42\xae\xc6\xa3\xf4\x7e\xeb\x0c\x49\x5f\xb9\x23\x4d\x0d\xaf\xbe\xd7\xa1\x81\x03\xf8\x7e\xbe\xfe\x1c\xfa\x12\xc5\x31\xd2\x07\x07\xc8\xa0\x87\xb0\x47\x72\xe0\xd6\xfe\x70\x9e\x85\xf9\x32\x9d\x9c\xb3\xc3\x46\x3a\xf2\x46\x7f\x3f\x3c\x78\xbd\x76\xb4\xf7\x97\x9d\xf7\x11\xa5\xf1\x02\x95\xb4\x35\x95\xba\xe0\x0b\x7a\x6c\xd3\x78\xc2\x1e\x36\xd0\xdb\x27\x7b\xa7\xf0\x29\x74\x4d\x63\x9f\xf0\xdf\x2e\x2d\x51\x91\x83\xbc\xc0\x04\x9b\x02\x0a\x90\xfd\xd0\x3b\xfb\x44\xe6\x35\xa3\xd5\x53\x76\xf7\x47\x3e\x22\x02\xa7\x4a\x05\x2f\x11\xbf\xb0\x2b\x0a\xcb\xc0\xf9\x5d\x45\xc1\x33\x16\x9a\x09\x60\x05\xbe\x1e\x7e\x31\x0e\x8a\xba\xab\x22\x28\x64\xd4\x88\xdc\x48\xb2\x54\x42\x99\x22\xd9\xc9\xef\xc4\x19\x78\x65\x63\x48\x89\x03\x3a\x59\x07\xa2\x33\x69\x48\xcf\x6c\x22\xbd\x8b\xba\xac\xcd\x50\x22\xa2\x59\xaa\x31\xf1\x2e\xe0\x3c\x55\x72\xa5\x85\x3c\x75\x30\x22\x5d\xfd\x05\x06\x82\xa6\x36\x06\x93\x82\xc4\xaf\xa5\xd5\x67\xc5\x46\xa2\xe9\xc6\xca\x8d\x95\x03\x42\x48\xda\x85\x2d\xd4\x0f\x04\x47\xbc\x1e\xc9\x9a\x81\x1d\xd4\x13\x68\x9b\x86\x4b\x37\x00\x97\xd9\x14\x38\xb8\xe1\x8e\xb0\x92\x05\xc9\x76\x0b\x39\xe6\xdf\x6e\x98\x6a\x2f\xf6\x0b\x99\xf0\x4d\x86\x64\x1d\x72\x35\xef\x8d\xdb\x81\x5e\x05\xd1\x30\xe4\x2a\xe4\x8a\x4c\x33\x4b\xfa\xdc\xc8\xb8\x9c\xda\xe7\xa0\x0c\x97\xed\xaf\xb6\x02\xe0\x33\xf9\xc8\x26\x93\x11\x15\xa3\x76\x46\x81\x75\x45\x76\xce\x92\x2e\x1c\x76\xd5\x88\xf8\x12\x61\xbc\xc7\x2a\xb1\x16\x54\x24\xfc\x16\xd6\x20\xd1\x27\xba\x25\x0d\x3a\x83\x9c\x02\x7a\x80\xf7\xd5\x7d\xa3\x58\x9d\xa0\x34\x4b\xc7\x05\x1d\x87\x00\x15\x26\xe4\x83\xdc\x96\x0d\x32\x26\x68\x8e\xab\xb2\x36\x7f\xc5\xba\xa4\x19\x6d\x5f\xc1\x71\x80\x4f\x37\x53\xdd\x19\xa6\x2c\x9c\xd3\x41\x5b\x74\x21\x47\x5c\xe8\xe9\x60\x53\xe3\x8d\xc2\x0d\x7a\x31\xc5\x94\xf7\xbf\xaa\xbc\x9c\xb4\xb0\xdb\x46\x16\xac\xb8\x90\xba\xb4\xfd\x89\x16\x82\x26\x4c\x2c\xcc\xab\xac\x40\x1d\x78\x8a\x64\x37\x68\x86\xb0\x62\xea\xd0\x85\x0a\x92\x7d\x96\x8d\xb8\x99\x0c\xdf\x4e\x0f\xa1\x3b\xf6\xc8\xe4\x41\xac\xe4\xa5\x3a\x96\x60\x5a\xc7\xd5\x71\x72\x2c\xc9\xe1\x96\x32\x9f\xbb\xe6\x86\x09\x5d\xc5\x40\xba\x2a\xdd\x2a\x9f\x6a\x68\x7e\x5c\x0a\xd0\xf4\x50\x3a\x18\xe2\xea\x93\xee\xa3\x64\x10\x32\x2d\x2a\x4c\xc4\xcb\xf6\x4f\xd8\x5d\x6d\xa6\xc9\xbf\x20\xbd\xcd\x42\xea\x2b\x19\x35\xdf\x57\xe0\x9c\x2b\x28\x30\x55\xae\xc8\xaf\xf7\xb7\xa9\xc0\x25\xfe\x4a\xb0\x44\x1a\xf9\xaa\xda\xd2\xc5\x48\xa4\x4c\xd7\xf0\x7c\x1a\xe9\x05\x21\x25\xc9\x34\x6a\x15\x65\x5b\x84\xde\x3e\xa5\x76\x34\x9a\xb5\x3d\xc9\xe9\xa4\x88\x27\x63\x62\x67\x7c\x2e\xd7\xd5\xe5\x7c\x4b\x20\x64\x4b\x8b\x98\x29\x4c\x59\xe8\x10\x35\x6b\x5f\x69\x16\xba\x39\xbb\xcc\xc5\xf5\x83\x5e\x10\xe5\xd5\x93\x00\xe9\xbc\x04\x56\x29\x3b\x0e\x46\xd8\xdb\x8f\x41\x5b\xff\xe3\xab\xc1\x34\xcf\x60\x3b\xfb\x8c\x50\x65\xcc\x89\xf8\x78\xcb\x85\xe4\xfc\xab\xaf\x19\x0e\x25\x01\xe8\x63\x93\x7b\x1d\x6a\xca\x6a\x9b\xcf\x2b\x22\x77\xf0\x24\x9a\xa0\x4e\x16\xe0\xe7\x24\x7f\x2b\x32\x9a\xa5\x85\x4d\x21\x3b\xb7\xb9\x58\x1b\xc7\x86\xd5\x79\xa1\xaa\x42\x36\xb3\xbc\x2c\x70\x06\x5c\x38\x90\x8f\xc5\x91\x40\xc1\x5b\x4a\xb1\x42\xab\xcc\x17\x14\xa3\xdb\xb6\xab\xe5\x57\x46\x91\x00\xb8\x22\xee\x85\xcb\x66\x12\xf9\xfd\x2c\x38\x58\x15\xf2\x10\xa0\x92\x0c\x68\x82\x57\xe2\x4b\xf8\x20\xe6\x64\x96\xf1\x45\x74\x10\x66\xd8\x0b\xfb\xc1\x57\x07\x8f\xa3\x41\x4a\x17\xff\x2f\xe3\xa2\x74\x68\xe8\x8d\xff\xf0\x72\x3c\x99\xb1\xe1\xd2\x69\x4c\x8f\x91\x34\x6d\xdd\xdd\x86\x29\x1c\xb6\x89\x0d\x95\x6b\x4c\x97\x04\x84\xda\x09\xca\x5a\xdd\x0c\x62\x57\x83\x1d\x84\xac\x75\xdc\xf3\x0e\x90\xd2\xad\x12\x94\x4b\xe1\x1f\x8a\xa9\x04\xda\x3d\x25\xcc\x50\x5e\x85\xa8\xe3\xaf\xc1\x90\xba\xee\x57\xa0\x49\x00\x52\x3a\xc1\x15\x10\x5e\x42\x73\x14\xbf\xe1\x99\xd8\x3e\xca\x4d\x17\xc6\xcd\x52\x27\x3e\x97\x26\xac\x8f\x02\xc4\xaf\x74\xba\xef\xb0\x8b\x59\x38\x93\x51\x8b\x8e\x8b\x70\xaa\xee\x3a\x04\x68\x37\x61\x97\xd3\x56\xf2\x01\x37\x57\x5c\x6e\xf4\x40\xb7\xcf\x93\xc3\xcb\x40\x0b\xe0\x10\x44\xf0\xa0\xb2\x7a\x7d\xd9\x43\x8a\xa1\xe3\xf8\x72\x2c\xd1\xe2\x46\x4f\x01\x9f\xdc\x73\xd8\x6b\xc0\x4a\xfa\x06\xd2\xa9\x52\x1f\x90\xd3\x78\x20\xf9\x4d\x81\x9e\xaa\x08\xa4\x04\x49\xcc\xa7\x9a\x42\x6d\x69\x58\x69\xfa\xe0\xe6\x9a\xf2\xcf\x22\x68\x0b\xf6\xdb\x7e\x6e\x9d\xd3\x5b\xc7\xc1\x7d\x71\x82\xb9\x77\xce\x27\xbd\xe6\x75\xae\xfd\xd7\xbb\xc5\xd8\x46\x95\xcd\x0c\xb2\xd7\xeb\x95\xf9\x9c\x62\x79\xff\x8b\x69\x25\xde\x8d\x31\xbb\x01\x46\xed\xa5\x6b\x63\x5b\x8a\x5b\x0e\x7d\xc7\x6e\x85\x67\xae\x17\x24\x02\xa7\x6d\x76\x7a\x33\x18\xf6\x51\xe1\xc9\xb3\x54\x96\x7d\xb8\x26\x7f\x94\x24\x1e\x58\x8d\x8f\x95\x06\x27\x76\x2f\x20\x54\xc5\xae\x30\xc5\x08\x18\x37\xa9\x85\x90\x57\xdc\x14\x85\x71\x33\x51\xc9\xf2\x8e\x1a\xfd\x69\xe7\x88\xed\xfa\x79\x4e\x11\x39\x00\xcc\xe2\xaf\x07\x75\x41\x7a\x1a\x70\xa1\xd4\x5f\x98\x84\xb5\x75\xc2\xb0\xe5\x8b\x49\xcd\x2b\xe1\xee\x4d\xac\x67\x78\xea\x67\xf1\xe4\x8c\x3a\x64\xb8\x4c\x8c\xf7\x68\xe9\xf3\x24\x74\xa7\xef\xc3\xd4\x33\x89\x76\xe5\xec\xa3\x9f\xd2\x61\xbe\x48\xdb\x5a\x44\x56\x95\x75\x0a\x2b\xc9\x27\xe1\x3b\x8b\x0a\x6d\x40\x08\xcd\x05\xab\x28\x74\x2c\x90\x2e\x2d\x6f\x36\x37\x36\xcc\xfd\xc2\xaa\x25\x05\xf4\x45\x06\xbd\x9a\xb1\xf4\xc8\xc9\x84\x96\x2b\x8e\x90\xd5\xc7\x59\xe8\x08\x22\xd2\x3e\x45\x15\x1d\x23\x3b\xb8\xca\xc7\x37\x33\x53\xcd\xeb\xd9\x68\xbb\x9c\x49\x15\xad\xbc\x9d\xb9\x99\x4e\x88\xf3\xdc\xa6\x57\x17\xf9\x96\x26\x40\x6a\x5b\xc4\xcb\x67\x15\xfe\x7f\x0f\x49\xb0\x37\xe6\x32\x95\x4a\xb8\x7c\x07\x20\xe6\xb4\x6f\xd9\x2a\xaa\xe3\x09\x89\x66\xf3\xe0\xf1\x83\xfa\x7c\x6e\x8d\x59\xd3\x4f\xd7\x6b\x37\x31\x48\x5f\x03\xde\x46\xc8\xf4\xbd\x25\x1b\xd8\x99\xdc\xf3\xdd\xb2\x15\x54\xbb\x20\xf3\x11\xb9\x57\x99\x0c\xfe\xe3\x26\x9f\xb0\x16\x6d\x23\xca\xa2\x46\x21\xfe\x36\x61\xe9\x4f\xec\xff\x46\xbc\xd6\xca\xf0\x90\xe5\xdf\x0a\x3a\x51\xd2\x81\x3f\x69\x54\x8f\x92\x06\x43\x67\xc1\x1a\x08\xdb\xab\x09\xb7\x46\xb2\x82\x7b\x70\x76\xa6\x9a\x98\x2a\xde\xb6\xc2\xa5\x72\xd5\xfd\x5d\x25\x0b\xac\xbc\x56\x97\x01\x7b\x3e\xd7\x5e\xdf\x56\x94\xec\x84\xaa\xfa\x15\x39\xd2\x92\x67\xe6\x8a\x4c\x5a\x19\xfb\xa1\x3b\xc5\x87\x5a\x61\xc4\x9f\xa9\xbd\x7d\x43\x18\xc0\xd6\xfe\x1f\xc0\xbe\x5d\xd9\x7d\x76\x77\x21\x7f\xcd\xb0\xe2\x2d\x4e\x69\x05\xc3\xe9\xee\x24\x2c\xda\xb1\x3d\xe2\xb8\x3c\x89\xbf\x44\x24\x1e\xff\xc9\x60\xd4\xc5\x9f\x00\x42\x11\x65\x53\xb9\x05\xe1\x9d\x6e\x05\x9b\x1a\x41\xef\x44\x26\xe2\x13\x28\x7c\xe0\xde\x0d\x3f\x78\xc9\x8d\xd3\xf2\x01\xf6\x31\xc2\x0b\x6f\xc8\x0a\x04\xce\x69\x0a\xbb\x17\x04\xfb\xc1\xf1\xc1\x73\xe2\x1b\x4f\xde\x96\xbd\x24\xaa\x5b\x62\xfb\x80\xa9\x1d\x07\xfb\xdb\x60\x77\x01\x2c\xef\xd1\x2e\x93\x17\x96\xa0\x06\xa8\x3a\x7d\xa1\xbf\xbf\x39\x70\x57\xc8\xcc\x01\x58\x38\xcd\xde\xe8\xe2\x66\xd8\x9b\x90\x43\xd7\x53\x28\x37\xf5\x90\x71\x57\xb0\xd9\xac\x65\x08\x16\x16\x3d\x1b\x0f\xdd\xa3\x07\x50\x03\x16\xc1\xe8\x16\x59\xf1\x47\x51\x79\x45\x98\xf8\x9a\x27\xbd\xcf\xf6\x95\xd7\x59\xfd\xd8\x97\xfa\x14\x2f\x4c\x23\x86\xac\x14\x71\xd7\xe3\x51\x6e\x15\x27\x27\x83\xeb\x59\xa6\x80\x88\x40\xb2\x48\x0d\xc7\x11\xe7\x80\x21\xc4\x42\xe2\xed\x22\x53\x0f\xb6\xba\xe8\xe9\xb8\x7f\x67\x85\x48\x49\xb8\x97\x0b\x73\xd9\x7a\x9d\xf8\xf1\xc4\x3d\xc4\x72\x62\x93\x84\x68\x23\x7a\x30\xf9\xd8\xbb\xed\x49\x7d\x46\x52\xa7\x93\x33\x24\xa7\x06\x45\xdf\x3a\x12\x37\x1e\x01\x8a\xfd\x3b\x64\x32\x73\x38\xbb\x8e\x2e\x72\x8d\xd0\x3a\xaa\x05\xe6\xfd\x39\x8a\xce\xe0\xac\x5d\xe7\xc7\x22\x29\x4b\x25\x0f\xb1\x64\xc2\x08\x59\x07\x1e\xc9\x22\x75\xe0\x11\x42\xe6\x42\x7c\xa4\x54\x79\x4d\xa9\xcd\x5a\xa0\x59\xb7\x59\x0a\x2c\x8a\xaf\xa9\x6a\xc3\x26\xbb\x38\x76\x38\xdd\xdb\xa6\xdd\xa7\x8b\x1b\x74\xb0\x33\xab\x37\x09\xce\x2e\x8b\x39\x76\x5c\x3e\xca\xce\xe3\xc0\xbf\xa2\xfe\x77\x9f\xc0\xca\x7e\xb9\x9c\xd4\xeb\xf0\xa7\xd9\x3b\x45\x4f\x47\x05\x83\x0f\xd8\x6f\x16\x98\xc3\xed\x91\x93\x14\x6d\xf5\x32\xf1\xb2\xe2\x4c\x9c\xcf\x95\x88\x22\x32\x7a\x6e\x4d\xb4\x05\xf7\xb7\xb0\xa3\x07\x73\x1c\x56\xd7\xe8\x2c\x1f\xc6\x2e\x3d\xf1\x6d\xc9\x00\x73\xd2\x2d\x15\xa4\xb2\xe8\x7c\x40\xea\x49\x16\x6a\xc7\x62\xd9\x01\x9e\x2b\xfd\x6e\xe3\x3b\x7b\x24\x91\xfc\xad\xcd\xcd\x17\x24\x03\x06\x01\xc8\xe2\x92\x6d\x63\x1f\x39\x36\x7a\xc2\xb0\xba\x6a\xfb\x6b\xef\x66\x86\x22\x18\x8a\xa8\xc6\xa3\xf1\xf8\x9a\xf7\x78\x06\x47\x07\x00\x5f\x72\x30\x3a\x2b\x17\x22\x6d\x83\x98\x98\x09\x9c\xf8\x70\xea\xb3\x05\x20\x04\xf1\x42\x7b\xf5\x45\xa1\xdc\xf4\x8a\xc4\x96\xb4\xe0\x6d\x3f\x8b\x3e\x44\x0d\x37\x7c\xf8\xdc\x70\x83\xbb\x54\xa3\x81\x7e\x0d\x65\x6b\x7c\xf1\xfb\xa4\xed\x72\x1c\xfb\xb2\xea\x3c\xcf\x0f\xba\x95\x99\x9a\xee\x39\x97\xe5\xf6\xfc\xf8\x68\x72\xe3\xa9\x1e\x3d\xdf\x6e\x6f\xbd\x7b\xf7\xc3\xd6\xf6\x5f\x80\x02\x96\xe8\x68\x33\x6a\x78\xf0\x4a\x0a\xe3\xa1\xfa\x3b\x4b\x27\x2c\xb2\xdb\xcb\xcb\x25\xe9\x23\x33\x7d\x3e\x5f\xdb\x74\x6a\x09\x95\x70\x16\x4a\x30\x9a\x66\x64\xfe\xb9\x06\xb4\xba\x8d\xeb\x0a\x4e\xe9\xc1\x3a\x47\xf9\xa7\x47\xee\x84\x89\xe9\xf5\x07\xa0\x5b\x31\x47\x8b\xd0\xa6\xb9\x5d\xdc\xcc\xd0\x2b\xd1\xe8\x05\x2d\xe6\x47\x88\x22\x4c\xbc\xef\x32\x5c\x35\x9a\x00\x06\x0c\x93\x7d\xa9\xc2\x3c\xe8\x3f\x70\x38\x3c\x08\x53\x62\x98\x29\x8f\x75\x18\xa1\xca\x2b\x87\xeb\xf2\xd4\xaa\xa3\x23\xd2\x20\x15\x75\x5b\x0b\x68\x00\xab\x52\xa4\x7d\xc0\xa8\x63\xa9\xe2\xf5\x34\xf6\xab\xae\x47\xb2\xe9\xa8\x2c\x3c\xcf\x70\x7f\xc2\xfe\x8a\xa4\x97\x65\x4f\x12\x94\xd1\x17\x42\xf3\x72\x23\xa1\x2b\x53\x4b\x76\xb2\x80\x2a\xc5\x21\xa9\xb5\x84\x3a\x61\x33\x6d\x12\xaa\xd7\xe5\x83\x8e\xa4\x61\xa8\x00\x40\x6b\x8c\xad\x2a\xbd\xce\x82\xb8\x88\x96\x88\x45\xf9\x55\x25\xfb\x8a\xb2\xab\xfc\xa8\xae\x33\x15\x95\xd6\xb4\x46\x42\x27\xd6\x39\x79\xee\x21\x80\x24\x55\xf1\x22\x2d\xe9\xeb\x79\x18\xa2\xcf\xa7\xe1\xb9\xd8\x82\xec\x25\x5d\x9e\x21\xd7\x56\x14\x7d\xc6\xb8\xe9\x59\xae\x99\x38\xca\x6b\x4a\x39\x0d\xe6\xd3\x8f\xf6\x25\x75\x69\x34\x84\xf2\xa6\x77\x9b\x7b\xef\x22\x66\x16\xe8\x26\x7a\x85\x10\x6a\xf7\x5b\xe2\xd7\x01\x65\xfe\x38\x1f\xa1\x00\x25\x9f\x2d\xb2\x0d\x63\x4f\xb1\x46\xf4\x8b\xc8\xb8\xb8\x68\xd8\x5d\xf6\xa6\x81\xb6\x26\xba\x63\x65\x5b\x93\x78\x46\x13\x45\x1b\x02\xd4\x4e\xf0\x6a\x2a\xf6\xf5\x31\x1c\x7b\x58\x54\xfd\x2f\x87\x1a\xb1\x3f\x16\x5b\xb4\xc2\xfc\xae\x7f\x8d\x87\xd8\x50\x40\xde\xb1\xf7\xc7\xa4\xc3\x87\x91\x95\xcc\x67\x23\x0d\x36\x7b\xd2\x25\xa0\x2c\x9c\xc6\x07\x26\xec\x00\x9f\x37\x3c\x16\x9f\x8f\xac\x13\x10\xe8\x9f\xac\x50\xf4\x21\x3c\x5c\xdf\xa2\x78\xce\xf9\x08\x55\xd0\x32\xdf\xc5\x36\xb1\xe1\x1a\xf5\x6e\x59\xef\x6e\x24\xa9\x6d\x10\x77\xee\xa3\x8d\x21\xa1\x52\xac\xc5\xfa\xc3\x44\x13\xe6\x32\x92\x5c\x54\x12\xe6\xf3\x58\x81\x42\x51\x24\xab\x4f\x96\x14\x66\x8f\xad\x53\x86\xb7\x65\x95\x7a\xd5\x3c\xf4\x8a\xbd\x3c\x1b\xd0\xd5\x0d\x32\x93\x35\x72\x5e\xa8\x34\xb2\x01\x4f\x52\x6c\x3e\x19\x1d\x92\x71\xb9\x9a\xd4\x71\xd3\x1b\xd6\x76\xa6\x67\xbd\x6b\xd4\x97\xb6\xfa\xbd\x53\x5d\x10\xf5\x79\x67\x97\x3d\x92\x4b\x45\xb3\x0a\xe8\xe9\xb1\x66\x35\xaa\x6b\x5e\xb6\x74\x5a\xc3\x43\x65\xad\x57\x93\x46\xf0\xba\x45\x3d\x6b\xeb\x6d\xac\x59\xab\x1d\xe6\x79\x0d\x0f\x64\xe9\xfa\x3a\x9c\x67\xa6\xf6\x9c\xf3\x71\xda\x1c\x4f\x2e\xd6\x01\x8b\xf5\xd1\x45\x93\xd7\x27\xad\x1f\xe1\xe4\x57\xca\x2b\x6d\x3e\x2f\x8e\x5a\x27\x66\x0c\xad\x7e\x9e\x2c\x1d\x98\x4c\xa1\xcd\x27\x5c\x96\xa4\x28\x21\x4b\x13\x3b\xd9\xda\xfe\x2b\xaa\x9e\x46\xb6\x28\xb0\x43\x2c\xdf\x4b\x83\xc7\x23\x7a\x3c\xe8\x26\x2c\x5f\xc9\x11\x0e\x3c\xcd\xe5\x2c\x1c\xcc\x4e\xc1\x80\x55\x1c\x26\x53\x91\x24\xa5\x4c\xd4\x71\x7b\x5c\x13\xcb\xa9\x64\x04\x7f\x3e\x27\xb8\x4a\x19\x85\xc2\x51\x84\xa6\xde\x70\x26\x7a\x21\xe4\x6b\xf1\x29\x48\x78\xf0\xd5\x13\x41\x23\x93\xd6\xd8\x49\xa8\x44\x8b\x31\x5e\x0d\x80\xba\x8e\xef\x8d\x22\xa7\x3b\x32\x11\x55\x75\xe0\x3b\x60\x5e\x65\x55\x53\x8f\xa2\x26\x2c\xc5\xfc\x6c\xa6\x8d\x00\xc8\x84\x23\x53\x50\xd4\x46\xc8\xe7\x19\xc3\x24\x69\x97\xf5\x52\xb8\x1e\xdc\x9b\x65\xf5\x52\x39\xf8\x66\xe2\x48\x7f\x51\x3f\xc5\xda\x2d\x28\xec\x5c\xbc\x9e\xb5\xe6\x85\x25\xa0\x85\x25\xef\x35\x31\xec\x33\xa6\x7d\x43\x5a\xa2\x0c\x12\x1c\x48\x9d\x54\xa6\xdf\xca\xaa\xf5\x43\x2a\x4a\xa9\x0d\x4b\x97\x09\x6e\x28\x68\x5a\xdf\xa2\xcd\xd0\xea\xeb\x89\x40\x52\xcd\x5f\x50\xfc\xa2\x37\x35\x25\x83\x66\xef\x26\x7e\x51\x9b\xd9\x40\xaa\x00\x5a\x8b\x0c\x68\xef\xce\x10\xf3\x6e\xf8\x26\x6a\x0b\xdf\x7e\x84\x2f\xc8\x67\x16\x9d\xcc\x5e\x3b\xa8\x38\x73\x36\xcc\x7b\x93\x52\x8e\x20\xf6\xeb\x5e\x3a\x08\x8f\x4c\xdd\xd2\x63\x18\x1f\x07\xe0\x27\x85\x45\x6a\x05\x74\x60\xc1\x4e\x3f\x0d\xae\x09\x55\x95\x5d\x37\x00\x45\x13\x7c\xb0\xbd\xf4\x99\x96\xdf\x6a\x47\xfe\x61\x44\x34\x88\x00\x77\xe0\xce\x54\x6b\xb5\xbe\x91\x43\x7c\x34\x9e\x0d\xce\xef\x62\x87\x18\x90\x0d\x6e\x04\x6a\xbe\xba\xd8\x97\x19\xc5\xa1\x30\x69\xe1\xc9\xcb\x97\x4c\xc2\xde\x8c\xab\xd0\x71\xe7\x05\x1b\x35\x3d\xae\xca\x86\xfe\xb2\x6c\xef\x54\xbf\xeb\xf1\xb0\x8b\xc9\x84\xe5\x70\xdc\x00\xb9\x5b\x7a\x67\xea\xe3\x96\x5c\x45\xb9\xdb\x16\x1b\x27\xc7\x6a\x3f\xf7\xed\xcb\x43\xe1\x9d\x1c\xb6\xb7\x8a\x0a\x51\xb6\xcb\x41\xef\xc4\x8f\x60\x67\x6f\x20\x23\xae\x33\xef\x47\xff\xce\x3e\xc4\x23\x55\xba\xd2\x5a\xb8\x3e\x0a\x56\xe9\x3b\x54\x9d\xc8\x2b\xd7\x68\x69\xd9\xdf\x0f\xfa\x69\x94\x8f\xd6\x6e\xa6\x91\x79\xff\xe3\xee\x0f\x3b\x07\x1f\x5e\xef\x1d\xec\x6e\x1d\x1d\xa6\xf7\xaf\x76\xb6\xdf\xee\x6e\xbd\xfb\x70\xb8\xb3\x9f\x46\xcd\xc8\xfc\xe9\x60\xef\xc7\x7d\x0e\x99\xc8\xec\x6f\x1d\x1d\xed\x1c\xbc\x3f\x4c\x8f\xef\xaf\x06\x23\x68\x56\xda\x32\xf0\xf1\x7a\xd2\x3b\x83\xd5\x70\xd5\xfb\x42\x5f\x2f\xf0\x96\x67\x7f\x92\xa7\x51\x84\x5f\x87\x37\xe7\xf8\x35\xca\x2f\x28\x6e\x8d\x3e\x25\xf2\xe2\x10\xf5\xc8\x5f\x98\xa1\x7c\xe0\xfb\x78\x11\xf0\xa6\x03\xbc\xe9\x00\xff\xef\xff\xbb\x0a\x74\x8c\xd1\x16\x78\x52\x01\xbd\x6b\xb6\x7f\x3c\x38\xd8\x79\xbf\xfd\xd3\x87\xc3\x9f\x76\xd1\x9e\xe8\xc2\xbc\xda\x3a\xda\x39\x7a\xbb\xbb\xe3\xfb\x60\x77\xef\xfd\xd1\x9b\x34\xfa\x73\x6f\x74\xd3\x9b\xdc\x99\xd7\xf9\xe9\x84\x3e\x76\x7b\x93\xb3\x4b\xb3\x75\x3d\x19\x0c\xe1\xfb\xce\xfc\xf9\x66\x94\xc3\x9f\xe1\x9d\xd9\xba\xb9\x80\xfd\xd3\x1c\xc2\xfe\x91\xa3\xe3\x21\xb3\x77\x36\x1b\xe3\xef\xfb\xf1\x2d\x47\xbc\xca\xcf\xe8\x23\xb2\x7a\x17\x06\x26\xc7\xe1\x9b\xbd\x83\x23\x5f\x1b\xd6\x84\x95\x60\x15\xb6\x02\x84\x8f\xe0\x11\x36\x42\x45\x88\x08\x2c\x80\xf3\x6a\xeb\xa7\x34\x3a\x84\x9d\x18\xca\xec\x8e\xe9\xe7\x08\x4e\x5b\xf8\xfb\xb7\xbc\x3f\xe2\xaf\xa3\xcb\x9b\x09\x7d\xbc\x9e\x0c\xf0\xe7\x10\xce\xad\x13\xf8\x28\x63\x64\xc1\x21\x2c\x04\x84\x40\xb0\x38\x96\xc4\x62\x41\x89\xad\xdd\xfd\x5d\x98\x0f\xd1\xd6\x2e\x04\xf7\x77\xa3\xae\xb9\xca\xfb\x83\x9b\xab\x34\xda\xdd\xdd\xad\xf5\x4d\xed\xae\x76\x99\x5e\x5d\xa5\xd3\x69\xad\x07\x39\xa6\x78\x79\x0e\x3c\xc2\xee\x7a\x7f\xfd\x8e\x93\x30\xfe\x1c\x88\xde\x2b\xdc\xa5\xa3\x1d\xf8\xcf\xd4\x76\x6d\xe1\xc8\x0c\xc7\xa3\x0b\x4e\x52\x91\x5c\x87\x8b\x96\x58\x02\x2e\x91\x04\xdf\x66\x3c\x1a\xa0\xf4\xb7\xc2\x83\x72\xfa\x58\x88\x5a\x98\xeb\xe1\xcd\xa4\x37\xdc\xee\x29\x6d\xff\xd1\xcd\x95\x23\x14\x2d\xd4\x58\xbe\xb9\xea\x44\xe3\x11\x6a\x13\x8c\x81\x62\x4f\xe0\x30\xaa\x0e\xb4\xfc\x50\xb6\xdf\x9b\x5d\x02\x7f\x03\xcc\xb5\x3f\x6f\xe5\x64\x08\x02\x59\x86\xd9\xa5\xed\xbc\x75\x54\xf4\xcb\x6c\x92\x53\xdc\x5f\x5b\x6b\x27\x36\x12\xf9\x22\xf7\xfa\x76\xc8\x91\xb1\x4a\x74\xcf\xa7\x0e\x0a\xb3\x4c\x00\xba\xa0\x35\xb4\x75\x0a\xd4\xfd\x86\xed\xa2\xf6\xfc\xb7\xbb\x42\xdf\x3b\xfd\xe8\x8c\x63\x4d\x73\x7c\xed\xd4\x4a\xd6\xaa\x04\x2b\x69\x49\x11\x94\xd5\xb0\x57\xa0\xae\xa0\xbb\xea\x34\x61\xce\x4b\x74\x29\xed\x73\x61\x30\xb0\xdd\x2d\xf0\x60\x5c\xd0\x3a\x4e\xac\xe0\x41\x14\x1c\x54\x5e\xed\xbc\xde\xfa\xf1\xdd\xd1\x87\x7d\x98\xa1\x87\xc7\xe5\xea\x44\xaf\xa6\xd8\xf0\xeb\x6b\x6c\xf3\x24\x47\xb6\xfc\x76\x49\x9b\x27\xb0\x5d\x7f\x01\x8e\x00\x3a\x6e\x25\xcb\x54\x5e\x6f\x68\xb8\x6d\x33\x91\xb1\x7f\x97\x01\x8b\x34\x54\x38\x51\x46\xc2\x55\x07\x06\x39\x0a\x0d\x86\x29\x01\xdb\x1a\x0d\xf2\xc1\x5b\xb4\x8c\x08\xf3\x0b\xdb\xef\xea\x83\x2a\x32\xf1\x10\xdb\xc4\xdc\xa4\xe7\xea\x10\xeb\x14\x12\xfc\x49\xb0\x95\xa4\x61\x5a\x52\xe8\xeb\x69\x8e\x04\x8d\xc7\xe4\x2f\xf9\x1d\x89\x9e\xb2\x42\x66\x93\x93\x8a\x05\xe0\x90\x53\x89\x2b\x97\xc1\xd4\x62\x09\xac\x9a\x5a\xb0\x92\x95\x13\x02\xeb\xd7\x15\xbd\x82\x7d\x5b\x8e\x56\x53\xfb\x34\xbf\x18\x8c\xe8\x0d\x2e\xa6\x4f\xf3\xf9\x72\x3c\xf4\x5b\xfb\x46\x96\x51\x84\xbb\x3f\xa0\x4c\x49\x87\x23\x45\x39\x8d\xe2\xec\x71\xb8\x64\x81\x0c\xbb\xf2\xfa\x0d\x34\x8c\x6e\xf1\x45\xe3\x15\x0f\xd6\xc1\x9b\xf7\x6f\x22\xbb\x12\xf1\x39\x9c\xa2\x3b\x22\xda\xa5\xf4\xe3\xe8\x60\x5e\x00\x4d\x9a\x37\x04\xda\x4b\x11\xf8\x22\x61\xed\x4d\x74\x67\xfa\xd6\xd6\x09\x6b\xbc\xd1\x0a\x24\xc5\x50\x96\xf4\x07\xbc\x20\xaf\x86\x87\x53\x62\xc3\x04\x78\xaf\x47\x85\x30\x02\xdd\xd4\xaa\xd2\xef\xa4\xfb\xdf\xcc\xae\x86\xbf\x23\xe2\x71\x7d\x8d\x75\x98\x53\xf8\xb3\x4f\x53\xd4\xd9\x94\x41\x2b\xb2\xbf\x43\x83\x54\x3e\x2d\xf3\x9f\xce\xf2\x87\x40\x78\x3f\xc6\xb6\x67\xbe\x17\x24\x3e\x69\x97\x09\x96\xd4\x49\x97\x6c\x72\xb3\xc6\x57\x32\x81\x18\x9e\x25\x60\xb3\x4b\x5c\x98\x6a\x6e\x04\x35\x62\x8b\xe9\x3e\x64\xc5\x3d\x98\x4b\x11\x67\xa7\xcc\xce\x39\x77\xa8\xbc\x9e\x5d\x5e\x4f\xce\xbf\x44\xe6\x99\xf5\xc1\x0a\x40\x6a\x11\x99\x66\xab\x01\xa7\x36\xc5\xfb\x0a\x84\x22\xa4\x04\x92\xe0\x98\xd9\x7c\x46\x37\xe1\x41\xed\xb6\x75\x4c\x95\xa4\xe2\x62\xc3\xd0\x54\x72\xac\x42\x19\xed\x15\x12\x81\x97\xd2\xe3\x29\x3e\x2c\x16\x62\xb2\x82\xa3\x5e\x59\xdd\xb3\xb1\x5b\xda\x92\xdf\xae\x6d\x5a\xcd\x76\xec\xe0\xbb\x03\xf3\xb8\x51\xda\x6e\x54\x86\x24\x75\xf6\x58\x56\xf1\xbd\x46\xed\x77\x0a\xdb\xa4\x11\x73\x0d\x28\x06\xd2\xe0\x4f\xb2\x04\x8f\x20\x2c\xce\xb0\xa3\xe0\x20\x05\x7d\xd3\xf0\xa0\xbd\x87\x7d\xd7\xca\x49\x4e\xd6\x31\x2a\x46\xbc\x47\x9d\x49\x96\x2c\xb9\x5f\x65\x2d\xc6\x9c\x50\x31\x13\x68\x0e\x24\x2b\xfa\xf2\x22\xf6\xc5\x33\x01\x58\x51\xde\x4f\x67\xc3\x89\x05\x28\x61\x7b\x62\x55\x12\xd7\x9a\x14\x99\xcf\xe5\x23\x95\xec\x0d\x5f\x77\x92\x2e\xc7\x5a\xcd\xdf\x87\x6a\xe5\xf2\x69\xb8\xcc\x90\x5e\x35\x00\x87\x30\xaf\x25\x77\x15\xeb\x1d\xc6\xea\xb4\x37\xba\xd0\xcb\x0f\xc7\xcf\x2e\xf9\xff\x03\x0b\x19\x5f\x3e\xc6\x37\x33\x2c\xf0\xc0\x28\xce\xe7\x0f\x76\x94\x11\x28\xd8\x1e\xda\xae\x7f\x03\x7b\x69\x08\x59\x6d\xa5\x0a\x94\x6f\xac\x09\xb3\x3b\x83\x2a\x44\xee\x3a\x61\x22\x2e\x90\x80\xb4\x84\xd5\x2f\xa7\x30\x97\xd3\x27\x50\x18\xc4\xa9\x8a\xc2\xa8\x81\x09\xc8\x4b\x58\x79\xd8\xeb\xff\x7f\x24\x23\x8d\xd8\x43\xed\xf8\x4e\x51\x74\x05\x01\x3c\x42\x51\xdc\x7d\x9a\xdd\x7a\xed\x74\xce\xb2\x70\x3b\xa6\x1d\xfe\xd1\x05\xf4\xb6\xbc\x75\xea\x75\x54\xdc\x3a\x2b\xd6\x9f\x18\x06\x20\xcb\x37\xde\x30\xc0\xd3\xb6\xd2\x27\xd2\x4e\x7b\x8c\x90\x72\x95\x2d\x7d\x1a\x5d\xb2\xf4\xa5\xa1\xba\xbf\x9a\x20\x3d\x95\x22\xd9\xa5\x22\xd2\xd4\xd7\x62\xc7\xa3\xca\x98\x82\x35\x55\x83\xf6\x96\x6c\xbe\xee\x52\x50\x87\x21\x40\xdc\x3f\xe0\x1b\xe5\xc1\xcb\xb0\xc3\x37\x40\x2d\xac\xca\x29\x9d\xb0\xce\x34\x0e\xc3\x99\x87\x2d\x05\x4a\xcb\x90\x57\xe7\xa2\x70\xa3\x23\x32\x25\xfa\x69\xd4\x77\x6c\x16\x45\x86\x26\xce\x2e\x1a\x4f\x5f\x69\xf1\x68\xab\x74\x75\xe7\xc5\xb3\xcd\xe1\x6f\xef\x27\x25\xa1\xa3\xc8\x5d\xc6\x71\xf2\xa2\xe9\xe3\x65\xd9\xf8\x0a\xbd\x74\x0f\x19\xf3\x2c\xc1\xa6\x68\x80\xec\x0a\x5c\x91\xa9\x7f\x06\x6b\x23\x0b\x0f\xa5\xc1\x5d\xb2\x92\x7b\x9b\x8e\xc8\x6a\x9b\x55\x8c\x16\x69\xad\x25\x37\xcc\x4e\x0c\x4e\x4a\x05\x86\xdb\xd5\xad\x33\x59\x1a\xb7\xbd\xbc\x4d\x32\x04\xf1\x98\x24\x7d\xc5\x76\xa7\xdc\x51\x02\xbc\x5e\xff\x0c\x78\xef\x38\x72\x84\x9b\xb3\x8b\xfe\x40\x64\x5c\x7c\x93\x29\x12\x8c\xa7\x40\x62\xcb\x5a\xce\x3f\x95\xad\x6e\x97\x8c\x31\x2b\x5e\x1a\x4d\x90\xb9\xa7\xf7\xa6\x8d\x89\xf1\x46\x74\x80\x2f\xff\xb8\xf4\x42\xd1\x98\xb6\xeb\xc4\x8e\x5d\x89\x99\x3a\x04\xf8\x72\x40\x49\x2d\x3c\x60\xc2\x91\x95\xd4\x58\x64\xb6\x9b\x9a\x30\x10\x28\xe5\xd9\x29\xf2\xfc\xe9\x72\x9a\xc6\xec\x0a\x57\xed\x48\x86\xaa\x39\xac\xa9\x82\xb8\x25\xbe\xeb\x48\x9c\x44\xe7\x77\x04\x13\x37\x14\x45\x34\x55\x6f\x0b\xf7\x10\xeb\x18\x21\x76\x1a\x8d\x24\x98\x02\x4d\x98\x2f\x11\x59\xfd\x55\x13\x88\x4c\x0c\xb3\x58\x37\x7d\x36\xcf\x66\x93\x21\x99\xea\x93\xf0\x55\x3e\xeb\x51\x78\x73\x25\xe3\x98\xcf\x97\x83\x33\x75\xc9\x93\x0f\xaf\xac\x03\x15\x4e\x9f\x01\xb5\xce\x67\x49\x3b\xea\xe1\xad\x82\x97\x5f\x82\x8c\xd6\x80\x37\xab\x14\xa3\x8a\x00\x47\x02\x59\x0c\x2c\xb5\x6e\x74\xe7\xf3\x15\x4c\xcb\xe0\x9f\x33\x56\x9e\xa0\xc7\x09\x5e\x6f\xbc\x0d\x9c\x4e\x69\xfa\x50\x1e\x34\x24\xc8\x36\xef\x50\x50\x04\xbb\x62\x96\x8f\x68\xf6\x54\xf4\x91\x94\x4c\xda\xf2\x81\xad\x05\x20\x6c\x85\x91\xf1\x8f\x12\xf4\x0b\xeb\xe1\xb8\x0e\xa1\x95\x8e\x62\xad\xfb\x13\x8a\x80\x25\x8f\xe7\x78\x4e\xbc\xe6\x38\xc9\x11\x87\xa8\xac\x14\x66\x32\xfa\xd7\x28\x0d\xa9\x2e\x10\xea\x07\xc9\x9b\x81\x09\xa5\x53\x8f\xa3\xf3\xf3\xb5\xdf\xff\xe1\xbb\xcd\x8d\x3f\xac\x85\xb5\x47\x5d\x7c\x42\x47\x69\xe0\x8a\x75\xba\x92\xf9\x79\xe2\x25\xf2\x08\xaf\x8a\x45\x8d\xd2\x37\x2e\xcf\x18\x51\x13\xba\xe1\x6f\x0c\xf3\xcf\x44\x40\x50\xf4\xad\xa2\x32\x9d\xfc\x34\x12\x83\xef\x7a\xe4\xee\x1b\x59\xb8\x32\xc8\xc4\x4a\x19\xbb\x51\xb0\xfa\x08\x8f\xb4\x25\x31\xbc\xd5\xb6\x1f\xb5\x65\x87\x53\x8c\xa9\x59\x56\x06\xd3\x2e\x8f\x9c\xb4\xd1\x3c\x40\x5f\x17\x8f\x6b\x7c\xf5\x07\x17\x24\xdb\x09\xc3\xc6\x06\x43\x09\xc0\x36\x0b\xf5\x65\x1b\xee\x89\x54\x15\x61\x6b\xd6\x4b\x30\x0f\xc5\x0a\xc5\xe8\xa7\x38\xc0\x2f\xac\x0d\x8a\x73\x6f\xd1\xba\xde\x7a\x5d\xc0\x55\xf4\x04\x9c\xd9\x83\xbc\x8d\xc6\xe3\x66\x02\xbf\x6a\x0e\x2c\xdf\x64\x4a\x73\xa0\x53\x1e\x15\xc9\x9a\xc6\x8f\xcd\xf1\xb0\x6b\x1e\x1c\x46\x54\x04\x31\x15\x7d\x47\x02\x41\xba\x2f\xf4\xd2\x2b\x3c\x5b\x5d\x14\xf8\x9b\x7e\x7e\x7a\x73\x81\xac\xf0\x34\x1f\x9e\xd3\xb9\xa2\x2d\xe6\x78\x20\x7e\x67\xd4\x3b\x1d\xe6\x7d\xa5\xf0\x3a\xec\x5d\x54\xf0\x1e\x14\xdd\x89\x19\x16\x06\x84\xef\xa0\x88\x02\xcf\x51\x36\x86\x67\xad\x82\x39\x34\x51\x7b\xbb\xc7\xfa\xcd\xa8\x1d\xea\x2a\x84\x6f\x67\x89\x6c\x7c\x5e\xa3\x0c\x68\xe7\x7d\x72\x81\xaf\xf5\x67\x9f\x3a\xf0\x95\x61\xe8\x0a\xb8\x84\xde\x45\x8e\xc2\x4d\x40\xe2\x5d\xba\xbb\x79\x53\x79\x92\x0e\x0b\xcd\xa0\x97\x71\x15\xdd\x40\x43\x5d\x0d\x57\x32\x2d\x7c\xb1\xfb\xb1\x83\x77\x5c\x7b\x56\x5d\xd0\x66\x6a\x44\x29\xc7\x90\xb5\xaf\x04\x4f\x15\x5a\x1e\x7a\x04\x07\xf6\x1c\x06\x26\x9e\x91\xd4\xb5\x18\x8d\xc0\x48\xff\xa2\xce\x61\x52\x68\x1d\x8e\x2f\xc8\xc9\x11\xc5\x1c\x63\x19\xd8\xb4\x24\xd8\x84\xc4\xf9\x1c\xe5\x82\x9d\x31\x4d\xcc\xcd\x47\x9a\x4e\x61\xa5\x02\x1a\x55\x4a\x3e\x15\xea\xb8\x34\x06\x98\x5d\x24\xb4\x0a\xc3\x83\xe4\x45\xd5\x13\x0b\x32\xd8\x4e\x60\xb1\x53\x0d\xa6\x85\x91\x9b\x68\x05\x04\x8d\xf4\x51\x04\x8b\xb8\x60\x74\x27\x8a\x52\x4a\x5e\x88\x44\x3b\x66\x4b\x55\x07\x45\x68\xe4\x0d\x59\xb4\xf3\x71\x10\x8d\x11\x10\xff\xb9\x37\x19\x05\xf1\x18\x01\xf1\x24\x61\x1f\x24\x50\x0c\x2a\x1d\xe0\xfc\x4c\x0b\xfd\x72\xee\x7a\x97\xf2\x52\x1e\x77\xe1\x5c\x0b\x84\x00\x20\xa5\x5e\x3f\xb7\xed\xc6\x15\x14\xd8\x92\x8b\x43\x7b\x72\xf9\x68\x7a\x33\xc9\x0f\x7b\xe7\xf9\x2e\xbd\x4d\x92\xf1\x62\x7a\x93\xc1\xe7\x38\x2f\xdd\xc4\x5e\x31\x94\x15\x3f\xeb\x18\xc3\x5e\x67\x10\x85\x71\x77\x19\xd3\xfc\xec\x7c\xd8\x8f\xcc\xb3\x83\x9c\x2d\x2d\xe3\xdd\x45\x50\xbc\x76\x3e\xc8\x87\xf4\xb8\xbe\xc5\x9b\x77\x20\xb9\x35\x98\x2a\xdf\x0e\x2b\x35\x8f\x07\x89\xf0\x3c\x2b\x22\x67\x3b\x02\x11\xaa\x6a\x9b\xa8\xbe\x8d\x4f\x3f\x56\x35\x0b\xa2\x81\xa8\x9f\x7e\x6c\x6a\x33\x93\x59\x06\x31\xcb\x1b\x37\x82\x73\x88\x6e\xdb\x6b\x2f\xc7\xf2\x0d\x0d\x8a\x4a\x0d\xd2\x68\x59\x45\x1b\x0e\x59\x1a\xca\xa1\xde\x30\x9f\x48\x82\x12\x0b\x59\x8a\xb7\x93\xcd\xd1\xb8\xa3\xb7\x8c\xbf\x51\xc2\xbf\x1b\x7b\xfc\xeb\x98\xdc\xf9\x1c\x43\x16\x71\x20\xd2\xfd\x64\x29\x9e\xfd\xf1\x55\x01\x49\xf4\x01\x82\x90\xa6\xff\x2e\x1c\x65\xca\x00\x2e\xfa\x01\x85\xee\x03\x70\x9e\xe0\xfd\x13\xec\x3f\xb3\xbf\x8a\x26\x25\x15\x36\x63\x12\x0e\x9b\x26\xf7\xf2\x91\xc9\x2f\x52\x41\xef\x56\x20\xbf\xb3\x36\xfa\x83\xd7\xde\x26\xbd\xf6\x6e\xb4\xf3\xc0\xe7\xc2\xcb\x16\x5b\x6a\x47\xe3\x8a\x95\xab\xd1\x66\xb7\x46\x2c\x04\x17\x66\x89\xec\xfd\x02\xda\x22\x04\xb4\xc9\x64\x62\x5b\x45\xa2\x70\xa7\xca\x82\xce\x3a\x25\x57\xa6\xe2\x13\x8c\xd5\x11\x18\x16\x49\x6e\x69\x61\xf3\x66\xf4\x79\xd2\xbb\x16\x15\x5f\x14\x4d\x17\x09\x95\xbf\x01\x41\xc3\x6b\x4e\x8b\x16\x9c\xcf\x57\x6f\x81\xfc\x61\xcf\xc2\xc9\xa5\x2c\x74\x53\x6d\x1c\x03\x67\xed\xbd\x17\x7a\xb9\x45\xa1\xea\x45\xb2\xc0\xb1\x20\xec\x28\x2e\x0b\x94\x61\x6d\x2c\x5a\x76\x1e\x73\xfb\x31\xec\x2c\x75\x7c\x75\x8f\xfa\xbe\x71\x03\x6f\x3f\xd4\xe6\x38\xbd\x46\x70\x7c\x81\xf4\x9a\x0c\x54\x6f\xa0\x1e\x43\x0b\xff\x6c\xe2\x9f\x17\xf8\xe7\xbb\xf2\x9c\x71\x02\x77\x15\x28\x11\x14\x87\xc7\xb2\x2c\xad\xc7\xb3\x6c\x3e\x9e\xe5\xc5\xe3\x59\xbe\x53\x7d\x52\x39\xfe\x7e\xe3\x16\x0f\x79\xca\x1e\xb0\x15\x9c\xc3\xb9\x8f\xa6\xda\x39\xad\x5e\x17\xd3\xae\x65\x4f\xbd\x1b\x49\x87\xd3\x52\x02\xd6\x56\x0e\xb7\xb3\x2c\x13\x30\x28\x6c\xca\xf0\x42\x81\x53\x8a\x4b\x63\x9b\x28\xbf\x38\x8c\x1b\x5d\x8b\x02\x0a\x2e\xd3\x87\xcc\xe9\x47\xe7\xae\xab\x33\x76\xb6\x61\x38\xc6\xe8\x29\xea\xd0\x30\x5f\x33\xa7\x13\x53\xc0\x95\x26\x2d\x4d\xa1\x7a\x9d\x15\xc0\xb3\x22\xde\x85\x77\xaa\x72\x5b\x5b\xff\xd5\xda\xba\xf9\xed\x6d\xdd\xfc\xaf\xd6\xd6\x17\xdf\xde\xd6\x17\xff\xd5\xda\xfa\xdd\xb7\xb7\xf5\xbb\xff\x52\x6d\x95\x00\x5a\x24\x7e\xf4\x43\x1d\x38\x2a\xc8\xe5\x7f\x3e\x99\xfc\xd5\xb4\xe7\x57\x2f\xe8\x5f\xbd\x4a\x9e\x3e\xf5\x2a\x67\xde\x93\x46\x31\x70\xad\xc8\xbb\x3f\xb1\x89\xb2\x45\xba\x3d\x13\x0f\x15\x36\x07\xd9\xb8\x28\x0e\x26\x3d\x6e\xda\x4b\xba\x20\xe7\x31\x26\x75\xd9\x71\xd6\x88\xe6\xd8\x5f\xf2\xbb\x69\x89\x8d\xb4\x09\xef\xac\x3e\x0f\x07\x9d\x04\x21\xf0\xdf\xb2\x6d\x03\xbb\x92\xc0\xe9\xf1\xf7\x2f\xc3\x22\x9d\x22\x1b\x63\x93\x8f\x65\xd7\xa4\xef\x96\xfa\xde\x54\xdf\x2f\xd4\xf7\x77\xdd\x12\xa7\xf3\xd0\x6c\xbf\x45\x13\xe5\xc0\x04\xf7\xc7\xf8\x99\x2d\x45\x03\x9d\xa5\x99\x6f\x0d\x15\x11\x0a\xf1\x90\x5f\x45\x0e\xd8\x99\x32\xe0\xd3\x26\x35\xa5\x38\xec\xab\x97\x03\x77\x5a\x40\x82\xd0\x76\x0a\xb0\xf8\x66\x9d\x45\xf8\x35\x34\x38\x5c\xb5\xeb\xf6\xc9\x28\x72\x26\x62\x2d\x14\x7f\x69\x81\x27\x03\xb1\xdc\xb1\x8c\xf3\xf2\x8c\x17\x42\x6f\x64\x11\x8c\xe5\xb4\x06\x8b\x99\xd6\x79\x6d\x3e\xaf\x71\xc8\x21\x9f\xd4\xec\x73\x35\x54\x3e\xcc\xf0\xef\x34\x8b\x1a\xac\x04\xd6\x89\xa6\x51\xfa\x2c\x8e\x3f\xd5\xeb\x9f\x8a\x93\x30\x7a\xd6\x80\xfa\x1a\xcf\xa2\x24\xe9\x7c\x4a\xa7\xc9\xb3\xa4\xf1\xec\xd8\x45\x76\x01\xce\xb3\x46\xbc\x84\xfb\x7b\x36\x38\xaf\x01\x5a\xf5\x7a\x6d\xca\xa6\x89\x6a\xf7\x27\x40\x85\x3f\x23\x50\x69\x80\xf2\x12\x12\xad\x5f\x98\x67\x27\x27\x11\xd6\x10\x25\x00\xb8\x86\xc5\x57\x62\xa2\xde\x78\x9e\x9b\x26\x02\x80\xd0\xaf\x5d\x23\x85\xad\xa9\x36\x72\x64\x81\x58\x43\x91\x6b\xa6\xd2\xed\x05\x02\x5d\x40\xa6\x69\x86\x2f\xaa\xb7\x14\x78\x46\x2f\xfd\xae\x1f\x5d\x2f\xb1\x08\x1a\xde\xc5\xe6\xfd\xd7\xf2\x10\x9c\x39\x53\xbd\x11\x3a\xa2\xfc\x84\x56\x3c\x3e\x47\x54\x34\x69\x87\x59\x9d\xc2\x50\x85\x2e\xcd\x19\x3d\x67\x6a\x05\xaf\x70\x01\xd8\x13\x40\x00\x30\xc8\x63\xc2\x9d\xce\x5d\x3b\x45\xe1\xe0\x45\x42\xe4\x60\x6b\xac\x20\x1f\x19\xaa\x8e\x9c\x8f\xd4\x1d\xeb\x3e\x1e\xb0\x8b\x8e\x4e\xc8\xfa\x0e\x9c\x08\xf9\xf8\xbd\x27\x27\xda\x7b\x58\x92\xe9\x4a\xcb\x84\x23\x8e\x31\x43\xba\xa8\x55\xd8\x41\xec\xc6\x82\xaf\x66\xc3\xdc\xcb\x34\x67\x8b\x9a\xde\x9d\x38\xa8\xbb\x08\x65\x65\xe5\x56\x69\xd6\x3e\x94\x55\xee\x73\xcb\x18\x7e\x2b\x26\x15\x90\x96\xe3\x52\xce\x5c\xb8\x5f\x3e\x1f\x0c\x67\xa5\x57\xec\x82\x77\x08\xce\xa3\x5e\xac\xb5\xbb\x88\xb0\x3e\x18\x21\xff\x64\x0b\x81\xc2\xa4\x51\x57\xe3\x76\x43\x7a\x0c\xdf\x7a\x7d\x25\x84\x51\xb9\x6d\x59\x70\x25\x86\x8c\xe7\x9e\x24\xe3\x53\x1b\x61\xdf\xc4\xbb\xce\x38\x3a\xe6\xca\xbb\x35\xa9\xb3\x76\x3e\x86\x85\x8d\xcb\x1e\x2f\x9e\xfc\x15\x4e\xed\x9f\x91\xa5\x1d\x8d\xe8\x9f\xcd\xda\xd6\xcd\x6c\x7c\xd5\x9b\x0d\xce\x6a\x3c\xd6\xa4\x11\x8a\xaa\x84\x32\xe6\x0f\x5d\x04\xe5\x10\x44\xdd\x50\xf4\xe5\x9a\x2c\xd4\x13\x2f\x76\x86\x97\x71\xf7\xf7\x41\xed\xe9\xe7\x01\xbe\x1c\x89\xba\x22\xe5\xc3\x97\x5a\xab\xc8\x98\x5a\x73\x6c\xc5\x3e\xc1\x9c\x81\x61\xb6\x63\x88\xe1\x0d\x7c\x98\x7f\xc9\xd9\x42\xc2\x3b\xfc\x0a\x67\x98\x78\xb4\xe0\x74\x5a\x9e\x93\x98\x0a\x18\x37\x13\x82\xec\x4e\x59\xac\x80\x38\x8b\x6e\x4f\x9a\x4e\x3d\xd9\xac\xb4\x80\x2f\x2e\x93\x0b\x48\xb2\xf6\xee\x08\xc5\xac\x08\xc9\xfa\xd8\x50\x9d\x42\x3d\xe0\x34\x4e\x53\x4b\xbe\xbe\x5c\x5b\x5b\xdb\x36\x0a\xaf\xf7\x17\xa1\x23\x96\xff\x78\x92\xb2\x60\x59\x1b\xbe\x52\x9e\xa3\x98\xcd\x2d\x8c\x5f\xac\xdd\x30\xaf\x52\x2b\x16\x1a\xc2\x97\x35\xff\xf4\xe6\xd2\x17\x15\x60\x75\x0b\x1c\xe8\x51\xfe\x65\x76\x34\x38\xfb\x64\xca\x58\x78\xc3\x9b\xd6\xa2\x99\x18\x53\xb9\x2d\x5b\x0c\x58\x14\x33\xef\x4c\x26\x94\x77\x92\xf7\xa6\xe3\x91\xb2\x68\x25\x36\xac\x28\x56\x79\xc2\x1e\x3a\xe5\xb1\xa9\x7d\x27\x13\x35\x46\xab\xc3\x78\x66\x9f\x46\xd1\x3e\x04\x99\x9d\xb2\x06\xe3\x5c\xc1\xce\x71\x37\xf5\x2a\x72\x8e\x55\x91\x64\x53\xbc\xae\x63\x0b\x1d\xce\x8e\x8a\x41\x81\x14\x7b\x93\x57\x3e\x3d\x71\x93\xa7\x6c\x37\xbe\x74\x7e\xc1\xe7\x51\x49\xd5\x2e\x0c\xd7\xd6\x04\x3c\x19\x72\x0c\x35\x11\x25\x7b\xb0\x86\x7d\x77\x3d\x50\x51\xc1\x6a\xa7\xeb\x4d\x64\x08\x36\xb2\x2c\x3b\xb3\x8f\xb9\x4b\xab\x34\x45\x45\xd0\x85\xeb\xf2\xa2\x7c\x24\xb7\xc3\xeb\x25\xb2\x9d\x31\xf5\x8e\xe5\x06\xea\xde\x5a\x15\x0d\xcf\x9c\xe8\xee\x9f\x0b\x85\x16\x6d\x80\xf1\xe7\xe8\xb6\x85\xe9\x79\x56\xaa\x34\xc3\xf1\x40\x10\xde\xea\x4b\xd3\xda\x07\xb0\xd3\x56\x3f\x36\xdb\x2b\x6a\x67\xc6\x44\xd4\xcc\x8b\xa5\xbd\xc2\xb9\x4d\xf1\x59\xd0\x7b\x28\x55\xce\xe3\x6f\xe3\xc9\x38\xa4\xfd\x6e\xa9\xef\x4d\xf2\x48\xb0\x10\x4b\xca\x69\x69\x1c\x2b\x46\x40\x0f\x19\x0c\x3d\x2b\xb4\xa6\x7a\x6e\x5e\x4c\x48\x48\xef\xc9\x1d\xf7\x7f\xa4\x57\x54\x93\x3d\x8e\xd4\x76\x99\x42\xe9\x3d\x76\x59\x5a\xa2\x55\xf8\x1c\x48\xbf\xb6\x14\x13\x30\x31\x56\x03\xd3\xd1\xad\x70\xda\x08\xf3\xbe\xa5\x33\x25\xf6\x06\xed\x08\x70\x11\xd7\x9d\xb1\x72\x44\xe1\x88\x5f\xc7\x7e\xa5\x05\xca\x95\x08\x24\xe5\xad\xd1\xc1\xa3\x61\xc9\x93\x12\x1d\x84\xb8\xff\x8e\xd6\x6b\x18\x37\x21\x6b\x59\x69\xa0\x1f\xc1\x4d\xfa\x20\xe9\xc8\x47\x1a\x92\xc9\xc4\x4f\x88\xaf\xc2\xcc\x23\xb6\xaf\x3a\x37\xab\x98\x4c\x0a\x3f\x51\xa1\xd6\xe8\x05\x43\xd3\xd1\xa1\x72\x17\x3a\x98\x0a\xd7\x4a\xd4\xdc\xae\xce\xb3\xb4\x63\x67\x2b\xbd\x69\x1f\x17\x06\xbb\xd0\xc1\x55\xcd\xea\x26\xa9\x5a\x9a\x5f\x5f\x3e\x91\x8d\xc3\xd1\x3c\x13\x51\x03\xa2\xf2\x9c\x0d\x64\x6d\x95\xee\xba\xda\x5f\x23\xa0\x54\x10\xba\xab\x2c\xed\xf6\xb5\xab\xde\x27\x67\x39\x96\x29\xa9\xcc\x8e\x7e\xd5\x0a\x68\x7b\xbb\x8f\x94\xa7\x53\x98\x51\x3c\x7b\xd3\x70\x66\x70\x64\xb1\x6d\x0e\x83\x4b\x1a\x91\x70\xef\x36\x83\xe9\x41\x80\x86\xc5\x7c\xef\x66\x76\x7d\x33\x23\x53\x6c\x6d\x9c\x31\x85\x78\xd7\x42\x67\x96\xd8\x4f\x8b\x60\xe6\x52\x2b\x74\xdb\x73\xe4\xdc\x16\x8e\x95\xd4\x50\xeb\xf5\x8a\x35\xcc\x49\x7c\xfc\xee\x54\x44\xc6\xe5\xf3\x69\xb9\xab\x55\x2b\x35\x9b\x4c\x86\xf5\x2b\x91\xc4\x14\x42\x14\x35\x0b\x1f\x82\x56\x9a\x1d\x4b\x4e\x62\x95\x9d\xbf\xb2\xb1\x1c\x9d\x42\x01\x85\x11\x6a\xe2\x22\x87\xf2\xa0\xe1\xa6\xa0\x2f\xfd\x62\x49\x3a\xf4\xbd\x8c\x42\x3f\x34\x15\xab\xb6\x94\xc2\xac\x3c\x0b\xf9\x42\xe4\x42\x0a\x93\xd1\x6d\x90\x65\xba\x29\x82\x27\x0f\x6f\x1d\x5f\x8b\xe0\x7f\x16\x31\xae\x6a\x38\xda\xe9\x29\x8c\x99\x79\xc2\xd6\x28\x06\x9f\xbf\x61\x7f\x24\xd5\xb0\x6f\xda\x16\x4b\x2b\xb8\xa2\x91\xc6\x75\xc1\x93\x37\xc5\x22\x42\x4f\xed\xfe\x6f\x45\xe7\x69\x5b\x61\x11\xab\x6f\xda\x02\x9f\xb8\x03\x56\x2f\xa2\x73\xeb\xdc\xbb\x92\x82\xe0\x04\x80\x33\x05\xfe\xe0\x9d\x43\x61\x3a\x17\x4b\x7f\xc3\x6e\x58\x79\xee\x78\xb8\xd6\x10\xaa\x5b\x26\x1a\x90\xef\x66\x06\x15\xf2\x1c\x15\x78\x68\x7e\xa2\x82\x74\x48\x17\x32\x23\xcd\xfe\x0a\x2c\xb3\xcd\x3f\xb4\xc0\x52\xfc\x63\xa0\xf1\x29\xfc\xd3\x4a\x26\x07\xf6\x94\x5c\xb8\x1f\x3c\x3a\x7a\x97\xb5\x36\x94\x00\x2b\x0b\xe8\x64\xce\x77\xac\x3f\xc8\x8b\x96\x11\xcb\xeb\x1e\xcd\x96\x5a\xce\x73\x82\x68\x8e\x1d\x8f\xb1\x16\xd9\x9e\xe1\xf3\x01\x87\xc5\x55\x76\xf3\x9c\x51\x3d\xa7\x27\xe2\x6f\x0f\xbc\x3b\xe3\xb2\x1d\x3d\x31\xb0\x27\xa5\x14\x2f\x42\xcd\x71\x17\x16\x83\x7e\x86\xd3\xf2\xc7\x41\x3f\xf6\xaa\x8a\x28\xa6\x2c\xba\x6e\x2c\x8c\x6f\x15\xdf\x48\xf4\x58\x39\x4e\xc7\xa2\x87\x83\xd3\x21\x1e\xdb\x6c\xe1\x49\x7e\x5b\x88\x3a\x43\x0b\xba\x68\x02\x32\x88\x38\xea\x0d\x86\x6c\xed\x95\xb4\x88\x22\xfc\x1b\x75\x25\x0b\xf6\x3b\x7b\x41\x97\x22\x7d\xe8\xf5\xc9\xf8\x0e\xce\x97\x2b\x2d\xa7\x16\x87\x97\x1d\xff\x71\x93\xa3\x75\xf8\xae\x43\x7e\x3c\x9d\xbd\xa2\x41\x2a\xa6\x78\xcf\xe8\xf7\x4e\x39\x6e\x30\x25\x4b\x50\x3f\x0c\x88\x27\xc5\x94\x82\xda\xfd\x3e\xf6\x45\x4c\x3d\x52\x12\x67\xe7\x8e\xb2\x02\x5e\x85\x39\x84\xe2\x91\x38\xa7\x61\xe8\xee\x37\x16\xb5\xde\x90\x4c\x74\xe2\x85\x9e\x9d\xe9\x51\x85\x70\x78\xd2\x2e\xc7\x65\xf4\x57\x49\xe7\xa0\xad\x1d\xc6\x2b\xbc\x00\x92\xec\xa1\x79\x08\xf1\x35\x7e\x34\x66\xef\xcc\xec\x52\xdb\x0a\x5c\x7a\x8b\x7f\x6e\x23\xd5\xde\x9c\xcf\x47\x9c\x3d\xb8\x68\x47\x45\x82\xbf\xe1\x3c\xf8\x6b\x6f\x08\x08\x2c\xb8\x72\x32\x4f\x41\x46\x9f\xef\x95\x88\x61\x2a\x77\x5b\xa3\xfc\xb3\xdf\xd6\xa5\xcf\x19\x09\xb2\xae\x6c\x68\x42\x78\x27\x09\x94\xde\x89\x29\x96\x2e\x0f\x19\x0c\x85\xd5\xdc\xe0\x4f\x1b\xad\xa7\x43\x69\x82\xb8\x4c\xc5\xe9\x51\x3d\x6b\x92\x34\x26\xc4\xf4\x2d\xc8\xc2\x50\x94\x6a\x29\xcd\x4f\x8f\xa4\x6a\x4a\xb8\xae\x12\x8e\x74\x53\xfc\x4c\x67\x0c\xa7\xa5\x44\xaa\x55\xe7\xf2\xb9\xb5\x67\x23\xf4\xe2\xb3\x71\x7e\xa9\x05\x31\x7e\xad\xb9\x7e\x58\xb2\x4c\x31\xa7\x29\xae\xdb\x4e\x5c\xcc\x52\x55\xbd\x29\xad\x6e\xfa\x72\x9a\xc4\x0f\x90\x01\xd5\x23\x0b\xc3\x2d\xf5\xf3\x85\x82\xf8\xd4\x69\x7b\xca\xb0\x23\xee\x9d\x5f\x6e\x7a\xc3\xc1\xcc\xda\x4c\xa3\xf7\x4d\xea\x31\xa4\xac\x7a\xda\x3b\x00\x11\x7d\x45\x28\x0a\x3e\xe9\xdd\x65\x53\x59\x36\xb6\x67\x8d\x7c\x64\xf7\xe7\xa3\xd4\x55\x86\xe6\x28\x52\x3d\xe9\x11\x3e\x7a\x4e\x47\x43\xa2\xa9\x83\x9d\xff\x92\xae\xac\x84\x88\x2d\x44\x49\xda\x31\x17\x16\xa6\x18\x75\xe6\x20\x49\x95\x7b\x64\x6d\x1e\x96\x27\x37\x91\x0d\xc3\x06\x24\xe8\x35\xf5\xe3\x1b\x4c\x3c\x44\x69\x3c\xec\xe3\x0f\xfb\x3f\xbf\xb7\x80\xf9\xd9\x0d\x38\x10\x14\x6c\x96\x5b\x7e\x6b\x53\xb1\x66\x31\xaf\xd7\xa1\x2d\x2c\x17\xdc\x43\xdd\x30\xd2\x26\x21\x37\x82\xbd\x21\xe0\xe6\x2b\x7d\x7a\xfd\xbe\x38\x19\x81\x66\xa5\xde\xaa\x9c\x3c\x10\x07\xec\x50\x89\xbe\xed\x18\xb8\x57\xc1\x1a\x45\x03\x33\x52\x3d\x66\x40\xdf\x05\x8a\xf3\x6f\x61\x21\x68\x43\xd3\x0f\xd5\x23\x13\x6e\x1b\xdd\xbf\x53\x7e\x3f\xf5\x50\x4a\xd6\x0d\x9a\xb2\x4a\x5b\x28\x40\x33\x03\x6a\xe1\x26\x02\x55\x81\x72\xf6\xe1\x33\x1f\x9e\xb3\x18\x2b\x24\x8a\xe9\xd3\x4f\xf9\x1d\xce\x0c\xe7\xae\xc4\x16\x4b\x12\x8a\xa5\xdb\xec\x77\x83\x4f\xb9\x4a\xb8\xe7\x6e\x43\x5f\x29\x19\x19\x46\x83\xde\xa5\x7c\x28\x23\x2a\x49\x61\x02\x76\xb4\x88\x4e\xd8\x0c\xd6\x48\xe7\x86\x68\xad\xbc\xca\x67\x50\x7d\xde\x47\xe3\x79\x0e\xbb\xcc\x56\x6a\xad\x38\x3a\x38\x50\xb5\xcb\x85\xcf\x2c\x05\x18\xa6\x58\x8d\x47\xc0\x15\x4b\x9c\x00\x31\xca\x47\xb8\x68\xb9\x71\xb4\x00\x8e\x07\x5d\xae\xcb\x86\x1e\xaa\x0d\x8d\x47\xa9\xac\x62\xbb\xbb\xa2\xbb\xb8\xb3\xab\xfa\x8b\x53\x48\x70\xd8\xa1\xfc\x70\x1f\x6d\x50\x3b\x60\x1c\x71\x3f\x77\xa3\xe4\x7a\xae\xe2\xba\x1e\xea\x75\xc5\x75\x5f\x55\x64\xed\xb8\xa6\xe1\x0b\x82\xee\x09\x0c\x3f\xd8\x17\xf4\xe4\x10\x64\x87\x9d\xcc\x35\xea\xe1\x9c\xe5\x06\xb3\x90\xbb\x2d\xfd\xd2\x8f\xe2\x7d\x31\xab\xee\x0e\x5b\x45\xf2\x50\x1b\xeb\xf5\x95\x47\x7a\xcb\xd5\xbb\xb6\x66\x2d\x07\x06\xc8\xc3\xc2\x25\xe3\xdc\x6a\xa4\x2d\x44\x3d\xc6\x36\xae\xdc\xba\x50\xf3\xcd\x26\x2c\x96\x2e\xf2\x2d\x4b\x4b\x2c\x41\x70\xcb\xd3\xf5\x2a\x29\x73\xb1\x2e\xb4\x8b\x72\x99\x9c\xa2\x57\x01\x15\x98\x6a\x8e\x5c\x58\x86\x8c\x24\xc2\x83\xe5\x8c\x7c\xec\x83\xf3\xb5\xad\x2f\xa3\x44\x47\xb0\x92\x4e\x95\xc8\x1d\x37\x0c\xdf\x0a\xf9\x70\x53\x54\xcc\xa1\xec\xfc\x1c\x42\x9b\xa1\x71\xfb\xa5\x50\x88\xfe\x00\x0d\x0b\x20\x3f\x60\xb5\xea\x50\x8e\xe2\x6d\xff\x0b\xfe\xec\x4e\x2f\x0c\xf1\x61\x47\xbd\xe9\x27\xf3\x20\x8f\xf6\x34\xde\xcc\xcc\xe0\xf0\x05\xe7\x28\xc3\x8a\xbb\xdc\xa5\x84\xd2\xbb\x31\x3d\x48\x29\xbe\x3d\x92\x26\xc1\xe6\xd9\x1f\xd3\x1b\x08\xe1\x4a\x8a\x7b\x8c\x69\xc6\x50\xda\x1e\x0d\xe7\x63\x13\x2f\x08\x1c\xe6\x99\xca\x60\x65\xea\x5d\x62\x73\xea\x5f\x60\x63\x1f\xeb\x5f\xec\xd5\x0d\x41\xd9\xfe\x2e\x6c\xd0\x80\x1c\x2c\x35\xcf\xe1\x31\x6e\x6a\x9b\x4b\x10\x77\xa1\xab\x36\xce\x22\x3a\x94\x65\xc2\x18\x53\xaa\xcb\x73\xcc\x89\x5d\xee\x20\xf4\x8a\x45\xab\x82\x42\xe4\x23\x49\xaa\x22\x53\x3b\x31\x8e\xae\xa4\xe1\x27\xae\x52\x46\xaa\x99\xff\xd2\xc9\x91\xab\x11\xc3\x0e\x34\x0f\x92\x34\x1a\xdd\x90\xb1\x47\xc7\x53\xc8\xed\x63\x29\x1e\xb3\xe3\xa5\xe4\xfb\xde\x7b\xe7\x00\x81\x43\x04\x88\xdc\xfd\xf3\xc0\x6c\x18\x5f\x7f\xe6\xea\x26\x8f\x6c\x72\xed\xcd\x08\x70\xd2\xf9\x48\xe1\x93\x65\x99\x66\xd3\xe4\xb2\x93\x66\xac\x6d\xa5\xf9\xdd\x4b\x98\x3d\x64\x86\x0d\xe7\x67\xf6\xdd\x1a\x04\xdd\xe4\x39\xe6\xd8\x2e\x30\x1b\xc5\x28\x62\x33\x78\x36\x67\x8a\xa1\x13\x04\xe1\xf8\xd4\x89\x80\x67\xac\x45\x0d\x1f\xd5\x1c\x91\x42\x90\x0f\x2b\x2b\xcc\xa9\x8b\x15\xa0\x8d\x2c\x6a\xcb\x46\x82\x50\xac\xe7\x32\x6a\x71\x03\x92\x98\x9c\xa8\x24\xea\xb7\x12\xe6\xfc\xa8\xc2\x20\x93\xe4\xb1\x59\x87\xdc\x29\xbd\xfc\xab\x09\xe7\x98\xf4\xf9\x5c\x22\x61\x62\xf0\x1a\xa9\xd7\x7d\x36\xc5\xfc\x27\x34\x37\xdb\xe5\xdc\x25\xd8\xba\x50\x3b\xb1\x0b\xd0\xa5\xf3\xa1\x67\xc1\xe2\x89\x36\x75\x44\x16\xc9\x01\x53\x9a\x1f\x00\x14\x06\x6c\x6d\x4d\xce\xdc\xfa\x24\x5c\xba\xc4\x21\xfd\x44\x58\xfe\x72\x00\x77\x9a\xd6\x35\x67\xfa\x17\xad\xa4\x93\x13\xbe\x66\x6d\x0b\xfd\xe5\x00\x5e\x2b\x27\x4c\x24\x61\xe9\xd4\xce\xd1\x7e\xba\x95\xe1\xc1\xfe\xae\xfd\x4e\x95\x4d\x6b\x64\x63\x9b\x08\x11\x0f\x89\x1d\x0c\xe8\x77\x6e\x03\xa1\x3c\x9f\x97\x48\x0b\x73\x41\x1a\xf9\x76\x81\xc6\x05\x34\xa8\x98\x26\xe4\x27\x7e\x6c\x78\x91\xa0\xf3\xad\x89\xa6\xe8\xc1\x1d\xc6\x0a\x51\x4f\xec\xd5\xc2\x2d\x8b\x93\x20\xf2\x97\x3f\x1c\x10\x5b\x3b\x5a\x8b\x5b\x0a\x79\xdb\x6d\xea\xae\x86\x3c\x2c\x14\x66\x56\x26\x95\xc6\x15\x49\xe5\x7b\xa5\xa4\x00\x81\xce\x8b\xd5\x10\x28\xa9\x7c\x0d\xe5\xef\xb5\x7c\x1c\x94\x2d\x47\x56\x5f\x68\x05\xb8\x94\xe3\x3c\x24\x15\x59\x7d\xc0\xae\xc0\x29\xbc\x5e\xfb\xb7\x5c\xa7\xe1\xb8\xe3\x46\x94\x6a\x89\xb0\x49\x51\x3e\xd3\xdf\xfc\x4c\x12\x3e\x9f\x49\x06\x29\x4d\x82\x44\x21\x88\xaa\xcb\xa6\xd0\x98\xc1\x6a\x69\xaa\x2b\x47\x47\xfc\x7c\xb1\xc4\x2a\x40\xb9\x64\xbd\x5e\x69\x26\x21\x29\x5f\xfe\x31\xd1\xbb\xa7\x3d\x98\xee\x1a\x8c\xdf\x78\x53\xfc\x5c\x60\xa3\x14\x1b\xe1\x9b\x85\x8e\xa4\xab\xb9\x0c\xeb\x14\x03\x8b\x92\x9a\x71\xb1\x2f\xfc\xf3\x41\x2d\x60\x38\x28\xb3\x5b\x0b\xc4\x11\x50\x81\x47\x16\xab\x3c\x23\xdf\x07\x54\x01\xeb\xa8\xec\x05\x07\x4b\xee\x1e\xab\x9e\x44\xe8\xd9\x69\x55\x9f\x66\xd9\x0e\xad\x3b\xce\xd2\xa1\x14\xa2\xfa\xef\xdc\x0d\x54\xf1\xa6\x94\xbd\x40\x6b\xd5\x63\x9f\xd9\x1b\x6f\x2c\x64\xcf\x0a\x40\x71\xeb\x0c\xa3\x64\x9b\xb2\x98\xe8\x83\x7a\x98\xf1\xd8\x5a\x0e\x08\xa3\x7d\x23\xba\x7e\xca\x5f\x0d\x66\x61\x53\xab\x5a\x68\x06\x96\x69\xcd\xaf\xae\x67\xc4\x55\xab\x8b\xa3\xe9\x6c\x8c\xe2\xb6\xd7\xbd\x0b\xe7\xf0\x85\x2c\x4d\x64\x84\x16\xfb\xb2\xe6\xdd\x8d\x06\x84\x75\x54\x8a\xa5\x34\xb1\x2d\x01\xdc\x58\x98\xd0\x6a\x8b\xce\xcd\x36\x65\x8a\x46\x2e\xa8\x50\x31\x92\x2c\xa4\xdb\x5e\xd8\x42\x8b\x02\xec\x4c\x22\x3e\xa6\x0c\x5d\xaf\x0f\x6f\x5a\x8e\xf7\x2d\x8c\x8b\xbd\x4a\x29\x0c\xdf\x7c\x4e\x5d\x63\xc4\x3e\x10\xef\xc3\xd4\x5e\x2e\x41\x92\x43\xc2\x8a\x16\x86\x35\x60\x48\xe5\x50\x3f\x28\x0e\x1e\x1e\xd2\x71\x62\x97\x62\x45\x9d\x9f\xae\x29\x75\xd3\x1e\x5b\x3a\x74\x0c\x2c\x60\x62\x9d\x75\x42\xeb\xcd\x00\x4e\x90\x8e\x47\x46\x55\x83\x70\x50\x12\x27\xb1\x8e\xdb\xdb\xd4\x37\xb4\xc0\x91\xf0\x8d\x55\x5b\xe7\x5e\x18\xbf\x11\x56\xcd\xbd\x61\x79\xda\xe9\x43\x4b\x78\x04\xa1\x03\x94\xfd\x7e\x70\xda\x49\x9e\xff\xbc\x89\xa4\x99\xb3\xaa\x79\xe2\x4e\x81\x6e\xae\x79\x26\xb0\x34\xdb\xf0\x60\xeb\x67\xd4\xf0\xb1\xc9\x34\x2c\xce\xa3\xe1\xbf\x6b\x0a\x0d\x9f\x34\x7b\xfe\x8b\x70\xcc\xc1\x34\x65\x3f\x7e\x7e\x27\xf1\x6f\x2a\x65\xfb\x42\x81\xe5\x0c\x74\x52\x94\xbf\x1e\x4f\x0e\xf2\x0b\xd8\xc4\x62\xcf\x3e\x4c\xbd\xea\x4b\x7c\xbc\x16\x27\x27\xc7\x27\xdd\xfb\x45\xa3\xf3\xbc\xb9\x7a\xf2\xf3\xdc\xa4\xbf\xf9\x7e\xe5\xe4\xa4\x9b\xac\x5f\x98\xe8\xe4\x64\xb5\x15\x25\x4a\x55\xe6\xcb\xc6\x1f\x38\x1e\x3e\xb4\x05\xf7\x5e\xff\xe3\xcd\x74\xb6\xcb\xec\x37\x1b\xdb\xc6\x6d\x8a\xee\xae\xf3\xe1\xb9\xb3\x0e\x0e\x91\x4e\x6a\x87\x82\x7c\x99\x2a\x87\x2b\x9b\x85\x0a\x4a\xc0\x9b\x7e\x7e\xfe\xfc\x79\x94\xbc\x5c\x73\x6e\x9d\xa6\x67\xfe\xb4\xf0\xf9\xac\x37\x41\x37\xb8\x6f\x87\xc3\xfc\xa2\x37\xac\x4d\xd1\x97\xdb\xe8\x2c\xaf\x41\x21\xd2\xe0\xa1\x0a\x6c\xa5\xe8\x58\x89\x22\xc4\xf6\x82\xad\xb7\x1d\xe2\x96\x15\xfb\xd0\xe6\xf3\xfe\x13\x4f\x4e\x9e\xc3\xff\xa1\xe2\xe6\xf3\x28\x8c\x86\xb8\xe3\x9f\xd3\xf5\x66\xa7\xde\xee\x42\x1a\xde\x25\xd5\x00\xcc\x0e\x80\x89\x7e\x8e\x1a\x02\xaa\x11\xad\x46\x74\x98\x43\xd1\x24\x4a\x74\x7d\xe0\x04\x0f\xaa\xca\x89\xa5\x1c\x2a\xde\xae\xe8\x0f\xc9\x16\x99\x67\xbb\xf6\x48\x74\xd5\xbb\xab\x8d\x47\xc3\x3b\x60\x79\x6a\x3c\x28\xc6\x76\xcb\x75\x0f\x2f\xb0\xe0\x4c\x35\x9e\x48\x5d\x35\x7e\x0d\x99\x3e\x5b\x36\xc6\x53\x8b\xa8\xc8\x7a\x73\x62\xde\xdf\xf5\x77\xfb\xed\xb2\xbd\x45\x5b\xa4\x5e\xb7\x92\xdd\x36\xca\xf3\x11\x6e\xf2\x14\x41\x32\xe7\x51\x3d\xd3\x90\xb9\x2c\xe6\x57\xd7\x81\x87\x67\xf9\xab\x1c\x67\xc6\xac\xe4\x7f\xe4\x70\x7b\xe7\xc3\xf6\xde\xfb\xa3\x9d\xbf\x1f\x1d\x66\x3a\xd0\x16\x81\x25\x36\x49\x34\x19\xfe\xed\x12\x0e\x90\x48\x67\xb2\x63\xee\xbe\xae\x51\xa9\x3f\xc0\xc0\x7f\xe2\xd4\x2e\x9f\xb3\x2a\x8b\x3e\x5d\xf0\xa0\xb2\x78\x61\x08\x44\x5a\xcb\x54\xe5\x95\x67\xf2\x4a\x14\xbf\x09\x0b\x5f\xfc\x71\x2c\x5c\xde\x82\xc0\x04\xe9\x14\x69\xcf\xd0\x5a\x86\xc2\xcb\x46\x40\x36\xef\xbf\xd9\xb8\x2c\x81\x50\x26\x54\x8e\xc6\xc0\x64\x02\x18\xe7\x6c\xc1\xb6\xa6\x48\x77\xd0\x42\xec\xdb\xe9\x21\xec\x5b\x7b\xf4\xc2\xe5\xbd\x37\x24\xe9\xca\x8a\x5d\x56\xf9\x97\xfc\x4c\xf9\x75\x40\x23\x85\x6a\x05\xb0\xfc\xa0\xb4\x71\x8b\x4d\xb9\xfc\x70\xb7\x3f\x86\x1d\xe7\x4e\x59\xdf\x37\x23\x53\xe9\xa3\x02\xad\x86\xab\x8b\x24\x23\xc6\x60\xd0\x2e\x2a\x6e\x2a\xb8\x93\x8e\xb2\xaa\xb1\xb4\xfb\xe9\xc8\x6f\xa5\xae\x03\xaa\xf2\xa3\xb0\xb7\x6f\x1e\xac\x24\x5b\xd1\x46\xfb\x74\x92\xf7\x3e\x21\xc1\x91\xb8\xa4\xb2\x66\x37\x7e\x4f\xac\xd9\xe5\x5f\x5a\x73\x4b\x6a\xb6\x53\x8d\xe3\xb5\xc6\xf2\x08\x6f\x67\xf2\x37\xe3\x21\x2c\x4e\x74\x0b\x4a\x16\x41\xc5\xa0\xac\x8b\xf4\x53\x57\xdc\xc1\xfd\x55\xe4\xe6\xc4\x7e\x32\x69\x62\x1d\xa9\xa4\x0a\x2d\x48\x5d\xd2\x0b\x78\x61\x6d\x30\xe1\x7d\x55\x4a\x58\x00\x69\x30\x61\x63\xaa\x92\xad\xff\xb9\xaa\xaa\x96\xa1\x85\x46\xad\x2b\x61\x3d\xa0\xbd\xf9\x00\x30\x35\xad\x34\x5c\xdf\xbd\xd4\xe6\xad\x29\xe9\x8f\x99\xb0\xeb\xac\xf1\x34\x6b\x59\xea\xf4\x8e\x50\x2a\xbc\xec\x90\xa1\xb5\x0e\xa7\xb1\x05\xb5\x94\x44\x8c\x91\xab\x52\xa5\xab\xf6\x66\xf1\xf2\x07\x8b\x7d\x0b\xb6\x19\x38\x9f\xe4\xfd\xda\x6c\xcc\x28\xd5\x7a\xd6\xed\x21\x0a\xa7\xdc\x92\x59\x71\xc9\xdf\xb4\x2e\x17\x69\x93\x6e\xd7\x08\x5d\xb9\xb9\x2b\x37\x03\x11\x11\x4b\x03\x3a\x7e\x3e\xd7\x21\x6d\x73\x00\x9d\x3d\x14\x32\x27\x15\x13\xa4\xad\x9e\xdf\x9d\xb7\xc0\xa0\x50\x45\x8b\x31\xdb\xb2\xe6\x8e\xc6\xa3\x35\xd9\x76\x5d\xcb\x7b\xdc\xe6\xd1\x4c\x5c\x3f\x62\x62\x4f\x36\xe7\x34\xe8\x06\x6e\xba\x92\x9c\xfd\xac\xc7\x2e\x5c\x14\x7e\xf4\x65\x82\xc2\xd2\xbd\x3b\xcd\x65\xe6\x28\xf1\x69\x1f\xa9\x4d\x05\x6a\x58\xbc\x2a\xd1\xd2\x6d\x47\x67\x5f\x32\x1b\x53\x9d\x27\x30\x4a\xe0\x7c\x39\xe2\xf8\x85\xd8\xf8\xf1\xd3\xf1\xf3\xb9\x0e\x55\x8c\x5f\x00\xa4\xa2\x45\xed\x5f\x3d\xbd\x55\xe1\x7a\x7d\x59\x67\xe9\x25\x50\x81\xc5\x92\x8e\x42\xf0\x44\x60\xb2\x80\xf3\x68\x1e\xec\x1c\xee\xfd\x78\x00\x31\x3f\x1e\xbc\xa3\x9e\x79\x70\xe3\x09\xba\xa0\xb2\x0f\x2a\xa6\xe8\x68\x9a\xc3\xa1\x63\x08\xb3\xf4\x87\xe1\xf8\xec\x13\xb4\x06\x5d\xec\xe3\xc4\xb3\x64\xbd\x76\x3e\x19\x5f\x91\x8d\xff\xd1\x78\x66\x09\x76\xed\xf4\x8e\xe0\x58\x6e\xaa\x76\x4d\x38\x00\x43\x0d\xa8\x3a\x6e\x5a\x35\x5c\xed\x77\x8b\x25\xad\x7d\x73\xb4\xfb\xce\xa2\x8d\x06\xa3\x0f\x7b\x68\xee\xf6\x5f\xc4\xdd\xa9\x96\x55\xb4\xe2\x66\x34\xed\x9d\xab\x95\x46\xd6\xe2\xc6\xb5\x1b\x38\x10\xf6\x46\x35\x4e\xd5\xab\x8c\xc2\x96\xbc\x44\xfc\xd8\x1b\xd4\xa8\xa9\xee\xbf\xbf\xba\xb6\xe3\x64\x70\xfa\xc5\xd1\xea\x54\xea\x45\x03\xc6\x71\x88\x88\xcf\x8a\x2f\x7d\x3a\x2b\x4b\xa4\x54\xaf\xce\xac\x62\x1b\x4d\x0c\x4f\x6b\xe5\x79\x50\xe6\x79\x69\x14\xba\x55\xe5\xab\x6b\xb2\x50\x43\x20\xdb\x87\x87\xbf\x1a\x06\x4c\xa4\x5f\x0d\xe3\xcf\xbf\x1e\x0d\xbd\x06\x2b\x81\x2d\xc3\x3d\x31\xf7\xb2\xdb\xa6\xf2\x6b\x3c\xe5\x4b\xfd\xa7\x11\xaa\x9c\xca\x6f\xa8\x9e\x0c\x27\x96\x82\xbc\x73\x2e\x86\x65\x81\x89\x23\x4e\x20\x2f\x1a\x9a\x7d\x94\x99\x77\x10\x56\x44\xa2\x59\x22\x96\xb9\x02\xd7\x6c\xba\x5a\xf3\x15\xae\xc0\x3d\xaf\xae\xf2\x11\xe5\x92\x2a\x80\x72\x4e\x07\xb9\xd5\x8b\xe0\xcc\x6c\xda\xdd\x86\xd0\xde\xb8\x4e\xa1\x77\x43\x15\x5e\x09\x8c\xeb\xfd\xe1\xa5\x4e\xab\xda\x83\x69\x23\xfd\x84\xd6\x32\x1e\x72\xe0\x3c\xce\xa7\x44\xde\xa6\x37\xd7\xe8\xc3\xad\x46\x16\x2a\x47\xf9\x0c\xed\x33\x0e\xc7\x93\x7c\x52\xbb\x85\x23\x0e\x8e\xc8\xf7\xb5\x3f\xe2\xaa\x66\xa8\x35\x74\x14\x00\x44\xef\xa7\xf1\x4d\xed\x0c\xd6\x3e\xba\x3b\xc1\x3e\x44\xf2\xd8\xeb\xf7\xad\xd5\x4a\xac\xaf\xf6\xfd\x0a\xa0\x8a\x74\x8f\x48\xcd\x4b\xe2\x05\x30\x6d\x7c\x8d\xd6\x02\xee\x80\xd6\xd6\x70\xed\xb9\x6e\xf9\x1a\xaf\xcf\xa8\x20\x0d\xb8\x4c\x90\xe0\xb0\x89\x59\x40\xd5\x52\x36\x48\xcf\xe8\xf1\x5d\x4f\xd1\xa4\x8d\x1e\x93\x07\xd3\x92\x99\x62\x65\xff\x4f\x26\x05\x66\x94\x19\x9c\xe9\x71\xb5\x91\x26\x74\xd0\x1c\xe6\x51\xf3\x5c\xb9\x68\x0e\xf3\x48\xa4\x9d\x86\xf3\x79\xac\xab\x2c\x40\xf7\x8c\x3f\xf2\x0e\x15\x8a\x53\x8b\xa0\x22\x58\x38\xa3\x19\x0a\x98\x52\x2c\xbb\x9c\x99\x16\x80\xf0\x63\x90\x37\xb7\x90\xe9\x87\xb5\xd0\xa0\x41\x73\x48\xaf\xc7\x64\xb2\x89\xc2\x56\xe6\xb2\xc3\x61\x65\x52\x08\x2d\xda\x16\x1e\xec\x8a\x9e\xac\xb1\x72\x2e\x17\x64\x4f\xe4\xb2\x8f\xdd\x0d\x29\xbc\x15\x01\x29\x74\x8b\xd1\xbd\x25\xdf\x45\xe5\x79\x3d\xfc\x4a\x69\x6d\x74\x73\x25\x52\x4e\xfe\xce\x1b\xad\xfc\x28\x37\x00\x94\x82\x13\xe6\xf8\x0c\xbe\x86\xdb\xf4\x4a\x46\x38\x7d\xe8\x4d\x3f\x44\x0d\xca\x9f\x74\xb3\xc2\xf3\x9a\xee\x39\x55\x0f\x3f\xa5\x99\x02\x38\x68\xcb\x07\xa1\xcb\x55\x10\xc3\x81\x56\x9d\xe8\xe1\x8a\xe6\x53\x11\x30\x01\x5d\x82\x67\x08\xd5\x1e\x8e\x4a\x20\x17\x34\x7b\x42\xba\xcc\x66\x4d\x96\x19\x8f\xf0\xce\xa5\x3d\xf1\xf4\xd4\xd2\x3a\x96\xb6\x69\xa2\x9e\x9f\x43\xec\x44\x3c\x39\xd1\xe9\x9a\xef\xe5\x0f\x99\x26\xa1\xac\x58\x6f\xd4\x9f\x8c\x07\x7d\xf2\x3a\x19\xaf\x4b\xa8\x16\x9f\xf4\x1b\xc9\x3a\xdf\x59\xf8\x41\xb3\xd5\x34\x47\xbd\xdb\x01\xac\xb3\xf1\x04\xad\xbf\x26\x4d\x60\x53\x26\x5b\x17\x24\x2f\x84\x17\xf7\xc9\x71\x0b\xb6\xaa\xd3\xf1\x97\x3c\xcf\xd6\x7f\xc0\x9f\xf5\x41\x73\x86\x4f\x94\x8f\x03\x30\xb6\x01\x01\xf9\x26\x53\xdb\xdc\x18\xba\x2f\xcd\xd6\x7f\x8e\x77\xc7\xff\x9a\x7f\xce\x4f\x3f\x0d\x66\xf3\xbd\xf9\xd5\x34\x89\x3b\xd9\xf1\xd6\xda\x3f\xba\xc9\x3a\x54\xdd\xbf\x3b\x9c\xdd\x0d\xf3\xcc\x51\x3e\x8c\xaa\xd7\x83\x60\x73\x8a\x59\x60\x9a\xf7\x46\xd3\x01\x09\x6f\xe0\xcb\x1e\x30\x43\x4c\xeb\x30\x84\x3b\x85\x03\xe6\xb5\xe5\xd1\x5c\x2c\x52\x6c\x9f\x64\x2f\x2f\x32\x85\xa4\xdc\xf8\x40\x5e\x14\xec\x56\x23\xc1\xf7\x46\x68\x35\x20\x88\xd5\x01\xef\x05\xb1\x85\x27\xf0\x1f\xaf\xe1\x2c\x41\x73\x2f\x69\x54\x65\x6b\x25\x72\x07\xa2\x13\x81\xe8\x05\xf0\xa3\xbf\x51\x6f\xed\x5d\xf7\xce\x80\x7a\x45\x1a\xff\x7a\x3d\xe2\xae\xc4\x17\x6a\xdd\x1f\x2b\x38\xd9\x6d\x38\x28\x32\x9f\x6b\xe0\x8d\xe8\xa8\x3a\x5b\x12\x74\x28\x80\x73\xc1\x07\xa1\x6d\x55\xe6\x4a\xcc\x8a\x4c\x50\x3c\x7c\x3b\x34\xeb\x75\x5f\x07\x3e\x3f\xab\x06\xb8\xfb\xff\x8a\x91\x6f\x72\x93\x3d\xe2\x01\xae\x4f\x28\xe9\x90\x4c\xac\xce\xef\xbd\x78\x91\x49\x57\xe2\x15\x3b\xd5\x25\x6a\x3e\x2f\xc6\xd0\xa5\xf3\xe1\x0c\xb6\xab\xf9\xfc\xbb\x97\xae\x61\xb4\x6e\xd8\xb1\x18\x4b\x84\xa6\xd1\x78\xe4\x43\xd8\x21\x02\x08\xf8\xae\x15\x87\x9d\x66\x5a\xe6\xf3\xca\xe8\x97\xff\x93\xc0\xee\x20\x05\x50\x72\x0b\xce\xf7\x0b\xaa\x26\xdd\xcc\xe0\xe8\x4b\x51\xf5\xfa\x1f\xe1\x0c\x8c\xac\x15\xb7\x8d\x57\x83\xf6\xfe\xa4\x49\x89\xbc\x0b\x8a\x0a\x43\x7f\x70\xbb\x33\xbc\xf2\xab\xef\x0c\x26\xe7\x2c\x17\xbf\x2e\x71\x04\xc9\x11\x1a\x17\x2b\x15\xcf\xa0\xa9\x51\x83\xbe\x71\x7d\x31\x98\x85\x7e\xb4\x0a\xf3\x2f\x0c\x9a\xec\x72\xd5\xd0\xc1\x13\xe6\x36\x9f\x5e\x3b\x4b\xe2\x81\x45\x41\xd9\xd6\xdb\x1c\x2d\x7b\xe9\x69\x97\x06\xe4\x52\x4d\xa3\x54\x7d\xab\x39\x92\xfa\xcf\x90\xc3\x46\xd7\xca\xe3\x9b\xd9\x93\xcc\x00\x29\x3f\x4f\xbf\x7c\x85\x59\x20\xe7\xe6\xe9\x97\x2a\x13\x41\xfe\x72\x8e\x31\x41\x95\x2d\xf2\x31\x6f\xb4\x4f\x7e\x1a\x2a\xc9\xf1\xb6\xef\xac\xb8\x64\xab\xbf\x34\xad\x52\xb1\xb5\x29\x5a\xb4\x08\xf3\x8d\xfe\xfe\x5d\x65\xd9\x72\xf9\x1f\x7c\xbc\x2d\x19\x46\x39\x1f\xc5\x5a\x6e\xb1\x68\xe3\x06\xd6\xcb\x83\xe2\x33\x22\x29\x6e\x8b\x69\x1f\xf3\x0e\xa5\xee\xe2\x89\x5e\xfa\x2b\xca\x66\xe5\x5e\x9c\x1e\x7b\xc8\xd5\x8e\xfb\x5d\x4e\x75\x7c\x96\x32\xdf\xe2\xb7\xdf\x55\x47\x2b\xc7\xc2\x46\x37\x20\x0f\x35\xba\xca\x6b\xff\x93\x7a\xcb\x84\x23\x28\x18\xc7\x15\x59\x13\x76\xdc\x2f\x61\xbd\x50\xc2\x07\x0c\xb9\x90\x47\x2b\x06\x37\xde\x2b\x1f\x52\x20\x20\x74\x10\x83\xa6\xc2\x80\x20\xbf\xc7\xf3\xd1\x34\x9f\x6d\xcd\x80\x3e\x9f\xde\xcc\x72\x71\xf8\x64\xe8\x45\xc5\xd8\xf2\x3a\x37\xa7\x3c\x19\xc4\x3d\xfe\xa4\x15\x30\xcc\xb5\x78\xdb\x2e\x26\xda\xf8\xce\x92\x78\xff\xcc\x9d\xae\xae\x9b\x28\x42\x27\x90\x06\x7d\x80\x97\x6a\x81\x38\x23\xee\x1e\x4b\xf8\x92\x3f\xc8\xca\x58\x0f\xff\xe7\x93\x8e\xaf\x00\x36\x8d\x52\x05\xe8\xa3\xb2\x22\x4e\x41\xf8\x4d\x80\x21\xc9\x9a\x54\x60\x49\xe2\x27\x48\x8a\x4b\x7d\x01\x71\xc6\x3a\xdd\x2e\x25\x4a\xbc\xf5\xeb\xbd\x24\x59\x79\x25\x5d\x92\x23\x45\xf7\xd8\x4b\xd2\x16\xc1\x0c\x0b\x5e\xe6\xf0\x12\x1c\x18\xd1\x1f\xbd\xbf\x64\x3a\x9d\xb9\xcd\x5e\xa5\x77\x6e\xb4\x07\x73\x17\x9d\xfa\xef\xc2\x39\xce\x0e\x36\x3a\xd0\xa0\xda\x7e\x54\xfe\xd9\xdd\xe9\x8e\x1c\xc1\xeb\x1c\x18\xa1\xf6\x0e\x76\x47\x51\xb9\x75\xd0\xd1\x01\x15\x17\xd9\x43\x8f\x2a\xf4\x9a\xec\xe6\xb9\x42\xe8\x97\x1e\xbf\xd4\x46\x30\xc9\x2f\x50\x8e\x65\x22\x1e\x4e\xd8\xbe\x9b\x5c\x00\x5b\x7d\x33\x3c\xc9\x88\x0e\x30\xc1\xd3\x84\xc9\x9e\xf8\xc4\x41\x8a\x15\x94\x64\x7b\x7d\x64\x29\x4d\xca\xb0\x0a\x91\xab\x8e\xec\xba\x52\x0a\xbe\xa3\x4b\x9e\x85\x93\x36\x65\x3c\x9b\xe7\xd6\xde\x1c\x40\x6f\x4c\x6f\xce\x71\xff\xb5\x38\xf2\x85\x03\xc5\x65\x11\x37\x34\xb2\xcf\xe0\x5c\x89\xab\x6d\x99\x0a\x7d\x59\x35\xbe\xec\xe4\x92\x8f\xaa\x16\xb1\xe0\x8a\x54\x61\x85\x2e\x64\x8c\x6b\x5c\xc4\xc2\x35\x67\x77\x91\xb1\x5f\x8c\x60\xa2\xf2\xf4\xe9\x5e\x0b\xff\x96\xd3\xac\x75\x4c\xfe\x2d\xa7\x7f\x9c\x02\x3f\x64\xf0\x6f\x39\x6d\x38\xb8\x02\xe6\x75\x1c\x19\xf9\xa8\xc8\x61\x0f\x6e\x90\xc7\x7e\x96\x73\x89\x52\x86\xe1\xdf\x72\xfa\x78\x82\x97\x99\xd0\x44\xf9\x28\xe7\xb8\xc1\xd3\x09\xd7\xe3\x3e\x25\x97\x9f\xa3\xba\x89\x71\xb9\xfb\x59\xfb\xd2\x4b\xe8\x1a\x54\x7e\xc5\x7b\xd1\x31\xcb\xfa\xac\x58\x73\x7e\x94\xd1\xbd\x45\x50\x48\xdc\x81\xe4\xfd\x01\x1a\xbe\x24\xf9\x10\xb1\x69\xe9\x23\x81\xa2\xe4\x15\x46\x51\xec\x89\xee\x63\xb6\xd1\xfe\xf8\xbd\xca\x2e\x8f\xd3\x1f\xf9\x61\x7a\xc5\xa7\x1c\x7f\xec\x5a\xb1\x04\xc7\x17\xcb\xc7\xc6\xc2\xc8\x2b\x9e\x46\xbe\x60\x49\x92\xce\x69\x6c\x5e\xf2\x74\x3c\x1e\xe6\xbd\x11\x59\xd8\x6c\x65\x59\x50\xc8\x7d\x67\x81\xd6\x29\x5e\xfb\xf9\x7b\x58\xbe\xbb\x6b\x8a\x0a\x8e\x4b\x5f\xc8\x61\xd0\x5a\xaa\x7c\x12\x30\x0c\x64\x71\x14\x35\x28\x16\xce\x9c\xef\x70\xca\xf0\x99\xd3\x60\x3c\x2a\x9f\x85\xd1\x4e\x7e\x8a\x8a\xbc\x5c\x6b\x2d\x16\xca\x67\x72\x45\x55\x55\xfa\xc6\x98\x02\xbb\xc1\x0a\xbd\x98\xe2\x6b\x86\x23\xfd\xb6\x7f\x19\x9e\x03\xe3\xcf\xbc\x49\xc1\x74\x29\x62\x78\x1f\xf6\x2c\x85\x64\x86\xa7\x81\x55\x53\x67\xb7\xd8\xf6\x8d\xc7\x93\x47\x87\xc5\x94\xa2\x34\xac\x83\x1b\x12\x64\x78\x08\x92\x1d\x02\x3b\xcf\x20\xe5\x2f\xa2\x0f\x09\xc8\x62\x7f\xac\xa2\x85\x52\x8e\xf6\x4d\xaf\xd7\x7d\xa3\x8f\x39\xb1\xcb\x10\x13\x37\xd7\xdc\xec\x23\x64\x68\x21\x44\xa9\xd6\xa3\x1d\x7c\x4f\x4e\x93\xc4\x7e\x1f\x4f\x64\x05\x76\x50\x84\xd8\x76\x10\x43\x13\xa7\xc0\xbc\x2d\xca\x46\x62\x9d\x1b\xa9\xa7\x76\xb8\x2f\x93\xdd\xaf\xaa\xd0\x22\xec\x6e\xe5\x4a\x08\xbb\x49\xd5\x04\x5d\x95\x65\x10\xdd\x59\x09\xf5\x56\x7c\x16\xd6\x00\x75\xae\x05\x30\x73\x5b\xad\x69\x96\xdd\xaf\xbe\xa8\x93\x8e\xb9\x75\xd7\x89\x02\x91\xac\x4a\x93\x81\xad\x18\xd8\xd8\xb0\xe2\xc8\xbd\x1a\xf8\xf7\xfb\x7f\x27\x32\x6c\xdf\x5a\x70\x22\xc3\xf9\xcb\x30\x6b\x2b\xaa\xe2\x49\x4d\xb1\x36\xd5\x97\x52\xa0\x60\xcb\x96\x26\xd1\xc2\x8e\x00\x53\x6c\xe0\x8e\x8e\xbb\x86\xe9\x23\x2b\xd4\x2b\xd2\xe8\x6d\x80\x66\x94\x06\xc4\xb1\x5d\xa4\xb9\x4e\xb3\xcf\xc2\x63\x64\xe4\x1e\xd4\xee\x03\x92\xa6\x58\xb7\x70\x47\x65\x5f\x8d\x43\x6b\xa3\x84\x1e\x24\xa6\x99\x44\x36\xdf\xff\xb8\xfb\xc3\xce\xc1\x87\xd7\x7b\x07\xbb\x5b\x47\x87\x25\xe7\x70\xbd\x2b\xb4\x79\xea\xf6\xe8\xc3\xbb\xab\xd3\xf1\xb0\xd2\xa1\x75\x21\x0b\x6a\x55\x07\x31\x99\xd4\xdc\xdc\xfe\xf1\xe0\x60\xe7\xfd\xf6\x4f\x1f\x0e\x7f\xda\x05\xbe\x86\x62\xdf\xd3\xb4\xb7\xb5\xd9\x9c\xfb\x5b\x47\x47\x3b\x07\xef\x0f\xd1\x32\xa8\x8d\xfb\xd3\xc1\xde\x8f\xfb\x1f\x0e\x77\xf6\x5d\xcc\xab\x9d\xed\xb7\xbb\x5b\xef\x28\x6e\x53\x8b\xc5\xde\x6c\x6c\x6c\x7d\xb7\x7e\x51\x44\x5e\x75\x94\xde\xb8\x7f\x55\x37\x31\x20\x73\x3e\xe9\x51\xf8\x70\xf0\x2f\x3f\x25\x83\x16\xda\x8c\xc5\x16\x6e\x3c\xb5\x85\x41\x15\xaa\x29\x55\xb5\x88\x04\xa9\xb9\x98\x8c\x6f\xae\x0f\xf3\x6b\x38\x9e\x9e\x0d\xae\x7a\x43\xfc\x0c\x51\x25\x66\x16\xb5\x47\xb9\x64\x32\x9f\xa3\x95\x0d\x54\xfe\xcc\x6d\x94\x10\xbb\x88\x2d\xe1\x43\x6e\x7c\x2e\x1a\xdc\xe6\xd9\xc6\x4b\xce\xd1\xe6\x9f\x6c\x17\xdd\x4e\xf4\x4e\xa7\xb6\x20\xdb\x6a\xb8\xb9\x82\x73\x42\xc6\x51\x0d\x38\x20\x31\xbe\x79\xff\x08\x37\x4f\x08\x03\x8f\x3f\x43\xfe\x83\xae\xd9\xbe\x5c\x8f\x47\x78\x91\xcd\x17\x67\x6b\x2d\x54\x45\x27\x00\x5e\xee\x18\xdf\xdc\x69\xa4\xf8\xf2\x58\x92\x29\x80\x12\xd4\x27\xfd\x93\x66\xb7\x91\xe4\xf1\x5a\x27\xe1\x1b\xf9\xa4\x7d\xc5\x1a\xbb\xd1\x5a\x24\xf2\x87\xc7\x9b\x5d\x14\x1e\xc1\xaf\x17\xdd\x97\xba\x47\x1a\xad\x8e\x60\x1c\x6d\x44\x69\x1c\x20\xcb\x09\x21\x9e\x1b\x24\x47\xa1\xa2\x12\x0d\xed\xe5\x06\x3a\x19\xc1\x96\xc3\x4e\x5f\xaf\xb7\xa4\xc3\x60\x89\x14\x01\x9f\x92\xab\x80\xd7\x83\x2f\xe8\x8b\x54\x8f\x50\xe2\x1d\x48\xd8\xf8\x77\xf9\x28\x8b\xa5\xdd\xec\xe5\x43\xcd\x13\x7c\x48\x40\x61\x9c\xc4\xed\x62\x6a\xa9\x06\x90\x11\x0b\x15\xe6\xe1\xbb\x82\x03\x1f\x7f\xf4\xbe\xc4\x32\x8d\x30\xf2\x35\xe4\x34\x0a\x81\xc4\xb8\xc4\xde\x17\x4c\x14\x41\x88\xeb\xf1\x67\x06\x04\x1f\x71\x6b\x23\x9c\x6d\xc1\x44\x99\xa0\x5d\x77\x99\x2a\xcf\x21\x77\xb2\x0e\x7f\xda\xba\x9d\xc4\x5a\xc9\x5c\xaa\x68\xa8\xf9\x7c\x89\x1e\x46\x6d\x6e\x58\x49\x6d\x57\xd2\x45\x72\x67\x08\x6a\x53\x54\x8e\xa0\x35\x91\x59\xec\x87\x17\x88\x99\x09\x23\x29\x0e\x27\x20\xd5\x60\x3d\xef\x65\x5c\xb4\x41\x7f\x13\xd4\x25\xcd\x74\xfa\x1a\x27\x07\x66\x39\x20\x8f\x48\x6a\xc6\xf0\xb9\x36\x48\x7e\xcb\x15\x65\x19\x4c\x8c\x0d\xb4\xa8\x51\x98\x0a\x8d\xcc\x2e\xd9\xc4\x84\xf1\x5c\x95\x70\x3b\x83\x64\xc1\x82\xa2\x00\x15\x78\x16\x8d\x06\xd7\x16\x20\x06\xd5\x0e\xff\x5d\xf5\x52\xf3\x5c\x3f\x4b\x0d\xdf\xeb\x51\x6e\xbb\x15\xd0\xc0\x35\xd4\xd6\x69\xb0\x06\x37\x90\x77\x0b\xe3\x0a\x98\x78\x5a\xd5\x70\xf5\xb8\xa7\x9d\x70\x6d\x2c\xfc\x6d\xc2\x4c\x36\x6c\x4f\x9f\x3a\x76\x38\x47\xf9\xc5\xfe\x24\x4f\x6d\x10\xfa\x0c\x82\x89\x51\x85\x34\x02\x41\x42\x35\xb4\xc3\x9b\x73\x0d\x0d\x82\xb6\xd0\xc7\x31\xac\x9f\x48\xab\x62\x5c\xf7\xfa\x9e\x36\x9b\xfe\xe0\x62\x30\x9b\x1a\x60\xef\xae\x44\xa7\x2e\xbf\x00\x22\x48\xbd\x4a\xe4\x94\x8c\x9b\x40\xd4\x1a\x1d\x2e\xb3\x35\xf8\x93\xd0\x17\x2f\x05\x5c\x41\xb6\xd3\x19\x56\x3b\xa1\xd4\x0d\x4e\x76\x4f\xa8\x03\x82\x04\x29\x58\x40\x7a\xcf\x97\x5d\xe3\xb2\x09\xea\x41\x5c\x60\x41\x65\x6d\x1e\xfa\x41\x0c\xee\xd0\xbd\xc5\x14\x97\xc7\xf8\xfc\x7c\x9a\xcf\x04\x6d\xeb\x0a\x93\xe2\x32\xfe\x99\xcf\x37\xfc\x8d\x41\xdf\x99\x24\x63\x36\x07\xc3\xc7\xf8\xb2\x0c\x38\xa2\xa2\x92\x33\x35\x1a\x73\xe1\x97\x1b\xf3\x39\xe5\x7c\xb9\xc6\x11\x89\xb5\xae\xd0\x10\xf0\x6c\x85\x5d\x4c\x22\xac\xb5\x36\x33\x89\x77\x56\x18\x5a\x9b\x38\x02\xb6\xa7\x99\xfb\x23\xcc\x09\xe5\x45\xd8\x3c\xa0\x9b\x41\x0b\x2f\xc7\x93\xd9\x6b\x98\x01\xe5\x13\x36\xe6\xb6\x9b\xf1\xc3\x2d\x22\x4b\x59\xee\x24\x1f\x3b\x98\x9d\xe8\xf0\xcd\xde\xc1\x11\xe7\x4b\xf9\x05\x3f\x60\x0f\xa6\xc7\x50\xb2\x7b\x4c\x70\xbb\x0a\x51\xbc\xfa\xfd\x07\xec\x28\x82\xa9\xef\xd3\x7f\xa1\x25\xc9\xb5\xd6\xf3\xbe\x95\xeb\x80\x8c\x18\xb7\x47\x3d\x12\x53\x3f\xf4\xf3\x3e\x96\xcd\x30\xfe\x65\xb6\xd1\x89\xd0\x81\x73\xe4\xef\xde\x6c\x86\x46\xe6\x3b\x0d\x89\xf2\x31\x15\x80\xfc\xe7\xc3\xf1\x18\x0e\x23\xd1\x59\x3e\x18\x46\xdd\x18\xa3\xd7\x7f\xbf\x91\x00\x9b\xd5\x08\x4b\xd0\x7e\x8f\xc9\xbf\xe5\x64\xa5\x9f\x72\x75\x7d\xa5\x90\xf7\xdd\x68\xad\xdc\x0b\xfe\x6f\xc6\x37\x93\x69\x9c\x7c\xdf\xda\xec\x58\xb6\x67\x6b\x77\x7f\x17\x39\xa3\x34\x8c\x68\x75\xc3\x71\x2c\xb2\x6f\x2e\x11\x2f\x7f\xf8\x72\xf2\x68\xfc\x0a\x32\xc6\x7c\x9e\x52\x6c\x43\xdb\x3d\x3e\x73\x92\xf0\x0f\x07\x1f\xde\x1e\xee\xfd\xe1\xf7\x1b\xad\x0f\x87\x47\x07\xf6\x5d\x0e\x00\x90\xc8\x39\x41\x82\x46\xce\xfe\x85\x28\xc3\x56\x32\xfb\xd7\xee\x60\x04\xbf\x34\xa7\xd8\xe8\x0c\xf3\x14\x7f\xe8\x76\xa8\x75\x30\x1e\x3f\x1e\x6d\xbf\xbe\x19\x0e\x7f\x82\xd3\x49\x6a\xe3\x6c\x04\x5d\xef\x3f\x50\x90\x7a\xc6\x95\xa2\x10\xb3\x32\xc7\x7f\x44\xdb\x45\x82\x07\x0a\x21\xd8\x58\xd6\x47\x3a\x6e\x6d\x74\x13\xc1\xae\x2a\xb5\xd5\x85\x85\xef\x91\x66\xc3\x62\x34\x46\x3e\x37\x4a\x23\xf8\xd0\x66\x37\x59\x6b\xa9\xf0\x8b\xae\x6c\xf7\x97\xaa\x82\xef\x60\xab\xdd\x48\xd6\x18\x2d\x73\xa5\x52\x7e\x67\x53\x00\x23\x33\x55\x09\xbf\xa7\x04\x73\x35\xd5\x2c\x41\x2b\x7f\xf1\x9c\x2e\x7e\x5f\x0f\xc7\xbd\x59\x1c\x6d\x34\xa3\x86\xe4\xff\x9f\x94\x3f\x49\xf4\xd3\x4f\xa9\x11\x97\xe6\xca\x4c\x01\x26\xb7\xd1\x6e\x11\x3c\xce\x74\xd9\x12\x8c\x32\x8a\x45\x9c\xf4\xef\xbf\x5b\x24\x6b\x1d\xf8\x38\xe9\xbb\xdf\xb8\x93\x1e\xb9\xaf\xb4\x53\xf5\x79\xd2\x24\x16\x33\xe9\xe0\xff\xe2\x7f\xcc\xe3\xe3\xc6\x5a\x37\xe1\x64\x9b\x0d\x93\x56\xd7\x4b\x07\x07\xb5\x26\xac\x0d\x49\x91\x36\x99\x15\x18\x63\x9c\xab\x9c\x51\x4e\x52\xc0\xd1\x5c\xc1\x79\xf1\xe6\xea\x15\x8b\xff\x71\x9a\x3d\xb0\xbc\xda\x3a\xda\x39\x7a\xbb\xbb\x63\x8f\x2c\xc7\x9c\x0e\x3d\xc7\x1f\xc6\xbf\xd2\x23\x4d\x41\x7b\x31\x38\xc3\xe5\x9c\x03\x3d\xf2\xf6\xfd\x9f\x58\xf2\x84\xd2\x3b\x38\x5a\xf4\x95\x96\x96\x15\x45\xc3\x3c\x99\x0a\x41\x08\x00\xba\x25\x23\xb9\xe0\x5c\xa1\x82\x8a\x08\x08\x4b\x41\xc8\xb5\x13\x5e\x95\xd8\x08\xdb\x80\x0f\x87\xfb\xef\xde\x1e\xb1\x64\x88\x74\x18\xf7\x14\x3a\xf7\xc3\x2e\x12\x35\x5b\x0a\x48\x1f\xb6\x2c\x1b\x93\xc9\x66\x3e\xbe\x46\x5b\x31\x71\x69\xcf\x77\xf9\x50\xbe\x3c\x29\x39\x3a\x21\x90\xa5\x0b\xd0\x51\x80\x9f\xd0\x6e\x1a\xb7\x46\x76\x3e\xea\x9c\xcb\xe8\x2e\x1b\x11\x6b\x75\xdf\xeb\x7d\xfe\xfc\x6c\xfe\x6c\x95\x54\x3c\xb5\x7a\xe7\xb3\x67\x18\xf3\x8c\xdc\x92\x21\x70\xb5\x3b\xf8\xeb\xed\x8a\x7b\x61\xbe\x16\xf2\xb7\x94\x6c\x53\x85\x63\xc9\x56\xba\x87\x13\xdc\x83\x57\x80\x22\xe9\x06\xbe\x2d\x0f\xaf\x94\x29\x81\x5e\xac\xed\x6c\xe2\x18\x3b\xb0\x14\x6a\x93\x92\x31\x94\xa5\x25\xcf\x50\xcc\x92\x02\x94\xda\xa1\xbf\xb8\x63\x51\x62\x73\x4a\xca\xc3\x1b\x82\x40\xaa\x23\x29\xca\x70\x8c\xd8\x82\x49\xed\x11\xd5\x8c\xcc\xf8\x86\x54\xff\xc8\xd6\x13\xc1\xd4\x39\x3b\x16\x29\x1f\x95\x52\xd4\xf7\x6b\x3a\x0e\x8d\x1b\x51\xc6\x20\x36\x61\x6c\x60\x97\x14\x45\x2d\x41\x0e\x42\x3a\x5b\x83\x11\x1c\x05\x91\x89\xd3\xdc\xc2\x07\x6b\xe6\x30\x31\x19\x15\xb0\x9d\x8b\xf1\x1b\x3d\xd0\xc1\x03\x82\x08\xf8\x2e\x7b\x09\x98\x02\xcf\xb1\x6f\x6f\x92\x0c\xaa\xa5\xa3\x47\x28\x04\xa0\xf6\x46\xbc\x72\xed\xc9\x95\x6b\xcb\x8c\x37\xfd\x7d\x3e\xdf\x7f\x06\x40\xf4\x99\x42\x34\x93\xae\xae\xb3\x20\x0b\xa0\x2e\x80\x70\xb4\xf1\x6c\x81\x79\x12\x75\xc1\x6b\x29\xf0\xc6\x42\xbd\xac\x11\x6e\xdb\x1e\x17\xcc\x68\xfa\xf9\xf4\xcc\xfa\x67\x71\x93\xf7\x07\xbe\x26\x8d\x55\xa2\x77\x27\xdd\x33\xa7\xca\x9d\xde\xd5\x75\x7c\x6a\x7a\xc9\x82\x6e\xef\x17\x85\x26\xe7\xf1\x6d\xcb\xdc\x6e\x8a\x04\x47\xcb\xd9\xd9\x6a\x99\xd9\xa6\x0b\x6c\xba\x9d\xa5\x95\x65\xb3\xcd\x8e\xbe\x7b\x6f\x21\xb7\xd9\xca\x6e\x5b\x85\x8b\xfe\xdb\xcd\xec\x76\x33\x8c\x83\xc8\x16\x32\xac\x9b\x9d\x8d\xf4\x76\xf3\xe5\x6d\xab\xb3\xd6\x4a\x5b\x49\x3a\xdb\x7c\x39\xe3\xef\xc5\x63\xef\x33\x98\x1e\x74\x74\x98\x1c\x24\x39\xbf\x4d\x61\x81\x4e\x10\x4c\x8f\x83\x60\x37\x9c\x2e\xc0\x80\x5c\x87\xc5\xb5\x05\x72\x0b\x51\xfc\x48\xd9\x81\x40\xd9\x3f\xe4\x7b\x5d\x86\xf9\xdc\x0a\xfa\x7a\x6d\x63\x59\xed\x1e\x08\x74\x63\x0c\x0c\x69\xe6\x8b\xf9\x4b\x7c\xd8\xda\xd6\xaa\x53\x68\x5f\xf1\x35\x2f\xcb\x66\x5c\x9c\x4a\xe5\xf3\x0f\x62\xd1\x4a\x98\x53\x17\xd1\x62\x8f\x54\x62\xca\x73\x72\xe9\x2c\xc3\xc9\x84\x8f\x9c\x3d\x02\x16\x9f\x26\x24\xe8\xe2\xa6\xe7\xc2\xdb\xe6\xa4\xc1\xda\x1e\x5f\xb3\xd1\x41\x5a\x60\xc1\x85\x30\x2e\x2c\x97\x47\x34\xaa\xe9\x52\x58\x11\x04\x9f\x8e\xe3\x13\x57\xaf\x1d\xfe\x0c\x17\x7d\x70\xe1\x79\xf1\x6a\x30\xc9\x49\x84\x0b\x8d\x69\xf1\x97\xba\xce\x75\x76\xd8\x7c\x22\x5b\x92\xe3\x40\x76\x3f\x1c\x8c\x3e\xa5\x2e\x0c\x3b\x92\xfb\x46\x81\x23\x92\xe9\xcf\xca\x51\x30\x9e\x5b\xdb\x91\xb1\xaf\xf0\x1e\xb8\xc7\x0c\x4f\x49\xa8\x0c\x30\x41\x2b\x8a\x13\xeb\x6d\xdf\xc0\xf1\x1a\x95\xd9\xfd\x79\x68\x7c\x71\x31\xcc\xff\x8a\x2a\x89\xdb\xd3\x29\x1c\xcc\xe9\xd3\x90\x8e\x22\xc9\x93\xed\xa0\x67\x8f\xbf\xe4\x24\xa2\x55\x8c\xcb\xca\x51\x1d\x98\x41\x8d\xe9\xa8\xf7\x29\xff\x40\x27\xb6\x72\x0e\x03\x39\x48\xa4\x43\x50\x82\x46\xa1\xd1\xdb\xed\x61\x0f\xea\xb7\x08\x74\xde\xbe\xff\xeb\xd6\xbb\xb7\xaf\x3e\x6c\xbf\xdb\x3a\x3c\x4c\xd5\x77\xd2\xa8\xc0\xad\x09\x47\xaf\x02\x00\x5d\x3c\x00\x56\x09\x60\x61\xef\xa8\xd9\xee\x09\x9b\xad\xc0\x2e\xcc\x2c\x96\x1c\x15\x27\x28\x13\x6f\x3b\x35\xc2\x12\x11\xac\x2e\xe4\x72\xa8\xc3\x67\x93\xa1\x11\x0d\xcf\x6d\xbc\x7a\x87\x9d\x8c\x5c\xa3\x4c\x89\xc7\x6c\xae\x52\x80\xcc\x68\x33\x94\xa9\xec\xa6\x90\x84\x87\xd8\x8c\x86\x47\xec\xf2\xc9\xf7\x05\x02\x36\x9c\xc7\xd9\x86\xe4\xe0\xf5\x64\x30\x9d\x0d\xd8\x03\x02\xc7\x50\xcd\x3e\x28\xa8\x60\x09\xdf\xa4\xe6\x2a\xf6\x16\xd7\x4f\xec\x5a\xe2\x86\xc2\x75\xe3\xfe\xc1\xdb\xc3\xa3\xb7\xef\x77\xa4\xcb\x4c\x61\x9a\x00\xb7\x23\x35\x78\x50\xfe\x95\x55\xda\x96\xdc\xb3\xa5\xf6\xf7\xe3\xd9\x9b\x50\x0f\x51\x32\x70\x9b\x8d\xc8\x7a\x26\xae\x4f\x78\xc1\x5a\x28\x26\xc8\x2d\xd7\x5a\xc7\x41\x64\x37\xb3\x99\x17\x82\x96\xcc\xa9\xa5\x98\x15\x60\x56\x81\xcc\x2c\x50\x74\x4f\x47\x52\x68\x15\xb9\x8c\x65\x69\x79\x94\x3d\x71\xff\x85\x0c\x81\xfa\x99\x76\x34\xfe\x94\x8f\x88\x29\x00\xec\xe0\xd8\x49\x5d\x09\x04\x3d\x2e\x64\x01\x46\xd2\xb8\xc6\x84\xd6\xa7\x6d\xf7\x98\x62\x63\x15\xb8\x40\xaa\x20\x80\x6b\x57\xb6\xeb\x02\x9c\xf1\x84\x65\xc6\xb8\x1f\x17\x4a\x74\x59\xc6\x96\x4a\x25\x94\x11\xfa\x5e\xe3\xc3\x4d\x74\xc3\xf4\x4b\x60\x54\x2d\xd6\xab\x60\x6d\x2d\x58\x14\x28\x18\x5d\x49\x76\x92\xc7\x66\x71\x62\x96\xe0\x4a\x3e\x17\x8a\x73\xb4\xd4\xfd\xc1\x22\x78\x64\x10\x68\x61\xd8\x67\x02\xec\x89\xa0\x01\xcb\xf0\xa7\x4e\xe0\xe7\x9f\xd1\xd9\xf0\x06\x76\xaf\x42\x37\x09\xbb\xc1\xf6\x7c\x96\x35\xe6\x17\xeb\x17\x42\x57\xda\x68\x94\x5a\xd8\xfa\x15\x2d\x64\x1a\x92\xb4\x7f\xf1\xb6\xe4\xdc\x50\x06\xa3\xd0\x2a\x8e\xc2\xc6\x42\x4d\xbc\x57\x44\x91\xb4\x65\xa7\x0a\xaa\x5e\xa0\x26\x9e\xcc\xbc\x7a\x7b\x70\xf4\x93\xa5\x31\x01\x89\xdb\x28\x91\xb8\x56\xb1\x6d\x54\x75\xac\x57\xc1\xbe\xcd\xfc\x08\x3e\xba\xda\xe5\x34\xef\x29\x24\x97\xd6\xbe\x5b\x98\x0f\xd0\x19\x85\x5d\x4c\x2f\xe5\x7e\x07\x86\x23\xe8\x5b\xa4\x80\xa4\x6f\xc9\x06\xcc\xf4\x6e\x6d\xce\x70\x53\xf1\x1e\x89\x9d\xbf\x14\x6d\xbf\xab\xd2\x19\xa6\xdb\xbb\xd0\xc4\x5f\xd2\xf6\x3c\x3f\x82\x85\x8d\xe5\x68\x32\xb8\x02\x4e\xe2\x28\x72\x17\xc3\x19\xde\xeb\x3a\x53\x27\x58\x31\x4c\x83\x81\xd8\x87\x5e\x71\x57\xc5\x53\x2d\x8d\xac\x6a\xe6\x02\x38\xed\x6c\x19\xfb\xb6\x9e\x90\xa7\x01\xe7\x2e\xd9\x8a\xfe\x5b\x39\xff\x24\xb1\xa8\x02\x1c\x89\xf3\x26\xf4\xfc\x53\x9d\x08\xee\xb2\x64\xf3\xbb\x8a\xb6\x4b\x06\xa4\x30\xfc\xf5\x80\x74\xb7\xb3\x92\x9d\x58\x81\x60\x36\xd5\x07\x0c\x5e\x5b\xa3\xf3\x29\xbf\xeb\x8f\x3f\x8f\x94\xec\x9e\xe8\x29\x88\x6c\x08\xeb\x28\x34\xe1\x6b\x1b\x35\x3a\xff\x88\x47\x14\x08\xcc\xe7\xf0\xe7\x65\xeb\x77\xf5\x7a\xeb\x8f\x2f\x5d\x38\x7b\xf1\x3f\xeb\xf5\xef\x36\x5e\x72\x8e\xa0\x1d\x64\x3b\x52\x57\x2c\x5a\x16\xbe\x23\x4c\x45\x07\x5e\xf7\x20\x11\x47\x50\x97\xa4\xc8\xda\x19\xf6\x62\x50\x45\xb2\xe0\x21\x82\x75\xd4\x0f\x3b\x4e\x4f\x15\xce\x33\x98\xee\xa0\x95\xbd\xb8\x30\x0b\x92\x4e\x14\xa5\xc5\x38\xab\xd1\x47\x6f\x37\x7f\x65\x52\x03\x6c\x33\x5f\x1a\x49\x74\x26\xb3\x6e\x5f\x1e\xd1\x85\x22\xe5\xda\x1b\x17\xda\x9f\x2a\x68\x3d\x86\xc8\x70\x1a\x7a\x6b\xba\x20\x83\xc5\x78\x9f\xc6\x71\x9d\xd8\xcf\x3e\x4b\xf4\x22\xa9\x3a\xc2\xeb\x19\x81\x9b\x3e\x9c\x0f\xb6\x18\xef\xd4\x9f\x26\xad\xa4\xc1\x12\xe1\x3b\x34\xff\x40\x4b\x4f\xe3\x3f\x9f\xac\xc7\xcd\xe7\x09\xfc\x3d\xbe\x18\x5c\x75\x9f\x27\xab\xeb\xea\x1e\x8d\x5b\xae\xac\x5b\xd9\x6b\x60\xe3\x6e\x80\x4d\xb1\xdb\x1e\x70\x9c\x47\x1d\x66\xc1\x5a\xfd\xbd\x24\x7d\x14\x82\x1a\x9d\xbd\xd3\x8f\x99\x36\x06\x2e\xd1\x74\xcd\xb0\xe2\xf3\xcc\xe7\x2a\x40\xfd\x2c\xea\xc7\xd6\xf1\x94\x1b\xc8\x28\x81\xc0\x98\x07\x24\x32\x11\x34\x92\x8c\xc6\xa3\x95\x04\xd4\x02\x3e\xcd\x6b\x3d\x6b\x6e\xeb\xf4\x66\x56\xfb\xdc\x9b\xa2\x75\x93\x66\x4d\x54\x70\xd2\xda\xfd\xe6\x22\x72\xa2\x15\xbe\x52\x33\x9d\xf5\xc8\xe2\xf1\x51\xef\xc2\x1e\x5f\xfc\x15\x75\xb1\x33\xb0\x80\x55\x91\xe4\x01\xe6\x0b\xc8\x99\x33\xa9\x55\xec\x24\x4b\xdf\xd8\x75\xf7\xb2\x4c\x89\x91\x69\xbb\x3b\x18\xc9\x65\x13\x3f\x77\xd8\x20\xdd\xc1\x95\xf2\x18\x48\x67\xbb\xf7\x8f\x0d\xeb\x4a\xd5\x04\xaf\xd7\xf9\x2a\x53\x9e\x24\x5d\x65\x95\x93\xdc\xa5\x16\xa7\x6f\xe5\x54\xd7\xb9\xdd\xa2\x58\xb4\x2b\xfa\xa2\xdc\x84\x64\x49\xd7\x56\xe4\x24\xe3\x4b\xd2\x2b\xbd\x2f\x41\xcf\xd9\x60\xd0\x73\x2e\x0f\x2c\x8c\x2f\xff\xb6\x9e\x7b\xe9\x2a\xab\xee\x39\x9b\xfa\xb4\x9e\x53\xb9\x1f\xe9\xb9\x52\x13\x96\xf6\x5c\x39\x67\x49\x9c\xea\x9b\x98\x83\x01\x8b\xc2\x7e\x4b\xd9\xaa\x75\x51\x45\x50\xd8\x3e\x6d\xd5\x20\x38\xdb\x8e\x18\x37\x9f\xcb\x1b\xc8\xc1\xce\x9f\x76\xfe\xbe\xff\x28\xcd\xb6\x42\xe0\xd8\xc7\x64\xfc\x86\xf2\x76\x70\x6b\x4e\x09\xa0\x98\xcd\x57\x6f\x59\xc2\xb0\x54\x8e\x9a\x03\x17\x52\xf6\x65\x03\xb2\x84\xf6\x56\xb5\x12\x37\xc2\x28\x6a\x50\x60\x21\x74\x02\x56\x82\xa3\x0f\x99\xc2\xd0\x25\x22\x61\xf8\xc6\x89\x0d\x25\x5f\x72\x5f\x2c\x59\xd7\x4f\x5d\xff\x4f\x59\xf9\x4f\x59\xf3\x15\xab\x1d\xe6\xb3\x5b\xe5\xe5\xf6\x43\x22\x2e\xef\x5f\xb3\xb0\x71\x45\x2f\x5b\xcb\x4f\x5d\xc5\x4f\x59\xbf\x4f\x59\xb9\xaa\xfd\xbf\x7a\x32\xcd\xe7\xee\xb9\xef\x89\x6b\xe3\x01\x76\x66\xd9\x9c\x0f\x95\x91\xbe\x85\xac\x7c\x2b\x4d\x21\xfe\x10\x2a\x7d\x94\xc9\xa9\xee\x9b\x1f\x0f\xde\x3d\x95\x7c\x90\xb9\xa7\x47\xfa\x87\xf3\x04\x9d\xf3\xdf\xed\x44\x28\x8e\xa1\x46\xba\x92\x34\x06\x19\x94\x65\xda\xab\xde\xe0\x3f\xa1\x8f\xa9\xda\x6f\xec\xe5\x9d\xdd\xad\xb7\x4f\xee\x67\xaa\xe8\xd1\x9e\xb6\xb9\x0a\x6c\x75\x75\x57\x87\xb8\x57\x76\x76\x21\x8b\x7a\xfa\xea\xf5\x07\xe3\x47\x7b\x0d\xf6\x45\x25\xb6\xc9\xbc\x07\xca\x0e\xf9\xd3\x13\xc6\xc1\x02\x82\xc8\xc8\x28\xf7\x88\xc1\xb1\x6c\x38\x38\xfb\x14\x99\xf2\xe1\x09\x8d\xfb\x90\x9c\x39\xda\xee\xf9\x8a\x33\x32\xe1\xe1\x0e\xca\xb6\xdd\xe5\x33\x9a\x92\x70\x77\x25\xda\xe5\xba\x79\xc3\xcc\xb2\xc2\x21\x6d\xc1\x3b\xd4\xea\xf8\x14\xba\xf3\x36\x8f\x23\xca\x16\x05\xb5\xa9\x0e\x25\x60\xa7\xe3\x2f\x4f\xe8\x53\xb1\xd4\x95\xff\xd5\xe3\x86\x97\x0b\x12\x61\xce\x7b\x70\x7c\x0f\xd2\x5e\xbb\x98\xb6\x7b\xc8\x72\x00\x12\xb2\x9b\x60\xa1\xe1\xfc\x72\x79\x3c\x24\xcc\xa4\xe0\xd2\xcd\xe0\x83\x43\xf4\x15\xc3\x51\xee\xd1\x87\x87\xa5\x62\x04\x4a\x5d\x1f\x2c\xb8\x07\x3d\xa3\xaf\x90\xb5\xc6\x3c\x28\xf7\xc4\x7d\x45\xc6\x5d\x95\x7f\x12\x2f\xa7\x4b\x77\x5c\xd9\xd4\x77\xaf\xde\x38\xce\xf0\xaa\xcc\xbf\x79\xb1\xa4\x5f\x3e\x0c\x35\x19\xe9\x29\x03\x4e\x88\x74\xaf\xc6\x32\x7a\xa6\x64\x82\xe9\xde\x3e\x66\xa5\xf4\x96\x45\x0f\x61\xde\xbe\x50\x69\xae\xa9\x37\x2b\x01\x4c\x6e\x70\xc4\xe7\x19\x3b\x26\x4a\xee\x63\x8b\x0b\x74\xc3\xca\xc6\x7c\x2e\xc3\x4e\x52\xee\xbf\xdd\x84\x48\x87\xab\x73\xc1\x56\xaf\xaf\x88\xd6\x5a\xe0\xff\x11\x32\xe8\x2b\x43\x89\x34\xee\xb6\x50\x6a\x4c\x24\x3b\xdb\xa0\x92\x48\xfd\x22\xef\x41\x50\xd7\x11\x96\xbc\x10\x75\xda\xf9\x10\xc7\x77\x44\x81\x5c\x65\x0d\x0c\x53\xf7\xab\xe1\xb8\x84\x27\x00\x59\x96\xe5\xde\xfb\x75\x54\xe5\xec\x6e\x21\x11\x22\x86\x5b\x8b\x92\xc0\x39\xa4\xcf\xcf\xb2\x0c\x0c\x5b\x99\xc6\x76\xf7\xa4\x92\x51\x49\xfe\x18\xf4\x22\x5f\xaf\x4b\x19\x9e\x9f\x9f\x68\xb1\x49\x8c\xab\x72\x51\x40\x46\xf9\xb0\xf3\x26\xd3\xda\x32\xde\xec\x5c\x0e\xbb\x87\x6d\xf8\x9b\xf2\x94\xa1\x9d\xab\x40\x11\x09\x76\x40\x36\x2a\xa6\x5a\xe0\x49\xcd\xd5\x40\xb6\xb9\xdd\x94\x47\xfd\x0a\x7a\x58\x0a\xf0\x89\x78\x1e\x86\x4a\xc0\x10\x81\x53\x88\xbf\x84\x77\x1e\xf7\xb3\x56\x9d\x63\xda\x10\x40\xad\x3c\x9b\xa3\x8e\xe2\x12\x98\xc1\x4f\xe5\x8e\x9b\x00\xcb\x50\x4b\xf5\x54\x5b\x96\x89\x6e\xa5\x59\x6d\xd2\xa9\xe8\x7a\x4a\x65\x45\x3d\x8b\xf2\x07\x12\xdf\x11\x79\xcf\x40\x5c\x23\x15\xe1\x40\xaf\x8a\xfb\x0d\xf0\x94\xbd\x21\x07\xef\xaa\x37\x82\x05\xfb\xae\x02\xcb\x0a\x80\x00\xcb\xcb\x7e\x91\x59\xa6\xf5\x0b\x75\x53\x7f\xe9\x8a\x70\x81\x26\x1a\x04\xdd\xbe\xec\x4d\xf0\x32\x37\x7e\xb1\x39\x3f\xbb\x24\x31\x08\x0c\xb2\xc4\xc4\x02\x10\xb1\x38\xfc\x58\xd1\xb2\x47\x71\xe8\xad\xfd\xeb\x6b\x70\x58\x7b\xf1\xa2\x5e\x8d\x44\x3b\x1a\xe0\x54\x8b\xde\x46\x61\xc7\x93\xfb\x38\xdb\x39\x85\xce\x52\x83\x51\x68\x02\x33\x90\x68\xe4\xc2\x7c\xfc\xe5\xdd\x60\x06\x3f\xff\x71\x93\x4f\xee\x8c\x28\xf7\xee\x8e\xfb\x37\xc3\xdc\x8c\x00\x07\xb4\x34\xf6\xc1\x90\xc4\x18\xac\x73\x16\x1d\x33\xb8\x78\x31\x84\xbf\xc6\x59\x78\x66\x22\x51\x61\xfb\xd9\xd0\x7d\x99\xf6\x65\x3f\xba\x40\xc7\xc7\x5c\x59\x26\xa6\x05\xac\x55\x40\x13\x06\xd1\x01\x5e\x10\x91\xdd\x2f\x80\x16\xdf\x0c\xfa\xd9\x71\xb4\x11\x19\xf9\xd7\x6d\x63\x73\xc4\xa0\x18\x7e\x2e\xb1\x26\xe6\x6c\x80\x55\xdb\x0f\x63\x55\x2e\x32\x40\x84\xab\xcf\x83\x84\x86\xa0\x60\xce\xc9\x7a\xf3\x79\xbb\x36\xb9\x4d\xbf\x05\x7a\x62\xd0\x8d\x72\x53\xf4\xfc\xe9\xd5\x4f\x84\x7d\x54\x9c\x58\x43\x1d\x5c\x55\x98\x32\x94\x39\xa3\x7a\x18\xf2\x75\x96\xec\xf2\x6e\x4e\x0a\x6b\xcf\xb7\x65\xf4\xec\x23\xf2\x93\x8b\xf4\xab\x8a\x2a\x73\x1f\xd3\xe7\xeb\xa1\xa4\x25\xc4\x88\x8d\x11\x86\x8c\xea\xa1\x6e\xfa\x64\x7f\x7c\x89\x3d\xe9\x11\xb5\x77\xbc\xce\x44\x23\x87\xdd\x2b\x96\x2d\xd9\x91\x88\xd4\x33\x5e\x8e\xf9\x23\xe2\xf6\x9e\xad\x7a\xa0\xbd\x49\x58\x1e\xa5\xa4\x8e\xd7\x27\x28\xa5\xa1\x40\x7f\xa3\x58\x61\x92\x16\x63\x54\x1f\x2d\xc1\xba\x84\xad\x8b\x50\x68\x7b\x78\x34\xbe\x87\xef\xb7\xfe\xb2\xf3\x61\x7b\xeb\x70\x47\x0e\x61\x99\xa3\x57\x62\x9f\x33\x03\x26\x68\x38\x4c\xa3\x56\x73\xb3\xb9\xb1\x36\x39\x6b\xbe\x40\x43\xc4\x1f\xc7\x93\xb4\x85\x17\x4b\xf0\xbb\x69\xfa\xe3\x59\x8a\x32\x04\x52\x5b\x04\x67\xc4\xf1\xd9\x60\x7c\x33\x5d\x9b\x91\xe2\x74\xb4\x80\xe5\xbd\x8d\x5e\x05\xb3\x3f\xff\x07\x2e\xf3\xe6\x19\x05\xee\x31\x9e\x4c\x23\x4a\x74\xfe\xe5\xba\x07\x4b\x0c\x19\xb9\xb5\xa8\x11\x5b\xc1\xe3\xc4\x6a\x47\xc4\x09\x14\x78\x0b\x5b\x15\x32\x45\xf4\x1a\x65\x9f\x99\xd0\x29\x37\xaf\x4e\x67\xf7\xaa\x98\xa5\x34\xec\x6c\xed\x1c\xdd\x9d\x29\x61\x94\xa0\x48\x2c\x39\xf0\x34\x59\x1e\x80\x8a\xf2\xb3\x19\x34\x4c\x9e\xc9\xd0\x98\x97\xcd\xb1\x30\xbc\x1f\x3e\x86\x73\x45\xae\x27\xa0\x5d\x51\xea\x6b\x31\xef\xe7\x4b\x31\x3f\xdc\xdf\xd9\x7e\xbb\xf5\xee\xc3\xf6\x9b\xad\x83\x43\x37\x4d\xe2\xe3\x93\xf4\x64\xed\xe4\x43\xb7\x11\x37\x13\x14\x78\xde\xdd\xfb\xc7\x87\x37\x5b\xdb\x7f\x71\x39\x7e\xbe\x1a\xff\x2b\xb6\x46\x09\x99\xbc\x17\xa8\x2f\x47\x02\x05\xe6\xe1\xdf\x77\x7e\x01\x64\x3a\x78\x47\x01\x30\xd5\x7b\xfd\xbb\xd0\x47\x9d\x7f\xce\x9e\x0c\x2e\x2e\x48\xe8\x99\xbc\x53\xe2\x39\x0d\x7f\xe9\xbd\x9c\x2c\x67\xb1\x3e\xf7\x84\xdd\x25\x44\x28\x64\x87\xc2\x35\x78\xd1\xab\x3a\xbe\x87\x66\xf6\x60\xb2\x75\xa6\x3c\xd7\xd0\x70\x98\x40\x4e\x52\xf6\xea\x86\xe7\xbc\x57\x7b\xbb\xdb\x6c\x53\xfe\xdd\xb8\xd7\xcf\xfb\x91\xb1\x99\xa4\x15\xd6\x40\x0d\xe5\x46\xdb\xdf\x3e\x07\x74\xa6\xdd\x8a\x8a\xfe\x84\xf9\x20\x55\x66\x60\x49\x36\xcc\x0f\x1f\x89\xc4\x59\xe1\x89\x08\xe8\x06\xf1\x80\xc7\x72\x25\x2c\xac\xab\x01\xde\xb5\x11\x75\x61\xd5\xe5\xbf\xa4\x4a\x00\x9c\xb8\x3d\x27\xd5\x0d\x21\x14\xcf\xe6\x31\xa0\x7a\xd8\xb1\x5c\x37\x49\x75\x1c\x35\x5c\x04\xa1\x25\x7d\x21\xbe\x9f\x60\xcd\x23\x1e\x29\xed\xbe\x28\xd1\x98\xe2\xc6\x8c\x06\x90\xd8\x57\x14\x05\xe9\x6b\x61\x7e\xd8\xdb\x7b\xb7\xb3\xf5\xfe\xc3\xd6\xd1\xd1\x01\x1a\xd6\xb1\xed\x8b\xae\x6e\x86\xb3\x01\x8c\x87\x1c\xe9\xf2\xbe\x91\x13\xad\xe9\x0f\xa6\x64\xbe\xd6\xe0\xd8\xec\x8d\x86\x77\x86\x8d\xf8\x43\x0c\x90\xce\x51\x24\x8a\xa4\xb0\xeb\x26\x25\xc1\x7e\x5d\xdd\xb1\xdf\x16\x39\xb5\x9b\xc9\x05\x3a\x91\x40\x9b\x75\xe7\xdd\xce\xee\xce\xfb\xa3\xc3\x00\x3b\x96\x9a\x67\xd4\xa0\x56\xe2\xe2\xf1\xb2\xac\x07\x38\x99\xd3\x9b\xd9\x0c\xc2\x78\x54\x36\xb8\x7e\x06\xc3\xe9\x93\x90\xb2\x35\x1d\xfb\x5d\xc1\x22\xb6\xb2\xb1\x48\x9c\x90\xc7\x7d\xbf\x37\xeb\xa5\x3c\xab\x80\x0c\xf6\xcc\x60\x74\x99\x4f\x20\xd0\x7f\xe5\x13\xde\xea\x38\xf6\xdb\xb7\x7c\xa7\xa8\x28\xe2\x48\x02\x9a\xa2\x46\xab\x7c\x89\x93\x14\x1c\xe6\x13\xa9\xc4\x4b\x75\x1a\x6b\x1b\xe8\x5b\x2b\x71\x76\x89\x1c\x4d\x44\x3b\x64\x65\x0a\xc5\x16\x89\x42\x0a\xe7\x2d\x96\x51\xea\x02\x15\xb8\xe9\x84\x21\x68\xbe\x91\xa0\x39\x83\xa8\x4a\x88\xf6\xa5\x5f\x1c\xf0\x65\xde\xac\x2d\x41\x24\xd5\x16\xb9\xaa\x93\x17\x1e\xbb\x3a\xed\xc2\xfc\xc3\xcb\x4c\x8c\xb2\x41\xa4\xdb\xe9\xad\xa7\x37\x36\x27\x5a\x15\x2b\xa7\x42\xfb\xbc\x24\xe5\xa3\xde\xcd\x6c\x1c\x25\x74\xa1\x89\xd1\xf3\xb9\xe3\x0f\x54\xa1\xb0\x4e\x0b\xa2\xe3\xce\xa0\xc8\xeb\x10\x88\x45\x45\x69\x99\xec\x74\xf0\x7c\xb8\x53\x82\x73\x58\xbf\xd2\x64\x31\xf4\x5b\xf5\xd2\xa2\xec\x5d\xab\x59\x52\xe8\xc3\x90\x4f\xb1\x3e\xee\x62\x7d\x01\x4a\xc3\x3a\xc5\x7d\x1e\x01\xf5\xdf\xce\xf2\x2b\xae\x72\x3e\x47\x5e\x15\xb5\xc6\xf3\xb3\xc1\xf9\x20\xef\x77\xc2\x3a\x53\x7f\x12\x97\x97\xaa\xa0\x1a\xdc\x04\x5c\xa7\x68\x9b\x77\xec\xf7\x33\x00\x85\xcf\x79\x85\xc2\x2d\xb3\x6c\x06\x16\x8b\xb2\x78\x1d\x5d\x52\x14\x5a\xbf\xbc\x7a\xce\xd0\xb6\x25\x6d\xc6\x0b\x95\x91\x47\x05\x3a\x30\xab\x4a\x65\x30\x9b\xde\x6d\x09\x3b\xfa\x80\x90\x9a\x1c\x10\x5a\xa0\x87\xcd\xf1\xf5\xc3\xc3\xef\x78\xee\x10\xff\x62\x87\x52\xac\xb9\x45\x3b\xc9\x1b\x8e\x4d\xe5\xb4\x05\xd1\x46\xbd\xb5\xb9\x3d\x1a\x19\x38\x48\x73\x15\xab\x29\x87\x65\x50\x4e\x37\x7b\xbf\xf7\x6a\xe7\xc3\xd1\x4f\xfb\xf0\x67\xe7\xef\x47\x1f\xf6\x0f\xf6\xf6\x77\x0e\x8e\x7e\x3a\xd6\xec\x2c\x5e\x0b\x2b\x07\x68\xfe\x66\x5d\x90\xb5\xc0\x2c\x0f\x7c\x6c\x23\xba\x69\x14\xf9\xe1\x75\xb1\x61\x73\x88\x53\x58\x82\x86\xda\x9c\xe5\x18\x11\x2f\x43\xb8\xd5\xcd\x60\xe7\x00\x2e\xec\x88\xdc\xf3\x2c\xcb\xf6\x02\xb2\x61\x9b\xe8\xc6\x13\x4e\x2c\x0f\x80\x7b\x08\x04\xb6\x45\x58\x92\xc8\x48\x3f\x37\x57\xfb\xb7\xa8\x79\x28\xc1\x45\x4c\xe4\xa1\x3c\xfe\x9e\x1c\x96\xfb\x92\xad\xfc\x1c\xc2\x6e\xb5\x7d\x84\x24\xc7\x1d\xa3\x1c\xc5\xf7\x94\xce\xee\xe2\x76\xba\x4e\x21\x5c\xc1\xcd\xb8\x0b\x73\xda\x49\x15\x63\xc3\x11\xc9\x3d\xff\x36\x2d\x2f\x80\x17\xa2\x08\x8a\xf9\x1d\x49\x24\xec\xe6\x73\x09\xb1\xcd\x2f\x56\x2b\x97\xcc\x22\x7e\x41\xcf\xf7\x1c\xb5\x28\x9c\x94\x98\x24\x06\x21\x4b\x26\xd1\x97\xc0\x57\xf6\x53\x01\x38\x8d\x3c\x9e\x04\xb5\x01\x09\x43\x1e\x31\xd1\x50\xe3\xd4\x6f\x1a\x2e\xaa\x3d\xf8\xde\x07\xb4\x46\x89\x6c\xfe\x79\x6f\x38\x3e\x8b\x7d\x16\x52\x2a\x29\xd5\x27\x4d\x58\x28\x0b\x85\x23\xd9\x47\x8b\x0c\xb5\xac\x65\xa5\x76\x76\xd1\x42\xaf\xa6\x9b\xd6\xd3\x1a\xda\x6b\x82\xd6\xc6\x9b\x59\x76\x3e\x72\x5a\x74\xe7\xa3\x95\x2c\x0b\xf7\xda\x20\xd2\xf3\x09\x1d\x84\x98\x12\x44\xed\x5c\x28\xb4\xb9\x88\x79\x12\x56\x5b\x63\x8d\x1a\xc5\x6c\x5a\x9b\x5d\xe7\xa3\xcc\x42\x47\x4e\x22\x01\x8e\x9e\x79\x55\xf2\xc2\xda\x12\xea\x89\x20\xc4\x66\x16\xc5\xaa\x5c\x68\x83\x11\xe3\xd8\x22\x95\xd3\x0a\x83\x54\x67\xeb\x89\x27\x00\xb4\x13\x56\x8d\xf9\xf8\xd1\xbe\x66\x39\xac\x3b\xce\xa0\x8a\x42\xd0\xa0\x16\x98\x0a\x92\x91\xa8\x8f\x2f\x3f\x7a\xd3\x50\x6e\x6d\x67\x16\x9d\x8f\x8c\x34\xf7\x74\x5b\xcd\x3b\xbe\xd9\x68\xb8\x12\xa9\xfb\x5a\x04\xae\x18\x96\x77\x55\xa1\x5f\xa4\x0a\xdd\x5a\xcd\x59\xf2\x6e\xa6\xd8\xc8\x03\x17\x01\xac\x2c\xce\xb6\x54\x4f\x3d\xa3\x3c\x38\xd7\xc6\xa3\xd7\xa5\x83\xa4\xb9\x19\x89\xa7\x0f\x37\xc6\x76\x17\xd1\x29\x22\xd0\xa7\x0f\x83\x78\xda\x04\x5c\xd1\x9f\x08\x47\xff\x06\x37\x8e\xb2\x03\x11\x74\xed\xf1\x4f\x7b\xff\xfc\x4f\x74\xa8\xf9\x4f\xf6\xf4\x0b\x28\xff\x13\xcd\x86\xc0\x84\xc6\xe7\xaa\x48\x1e\xa5\x31\x6d\x2a\x13\x67\x87\xaf\x15\x0e\xa1\x64\xee\x59\x51\xce\x12\xa1\x55\x70\x34\x99\xfc\x60\x5e\xce\x62\xcd\x76\x4f\xe7\xf3\x27\x00\x36\x82\xc3\xfd\xc2\x56\xf1\x70\x31\xa9\xc3\xa2\x23\x56\xc3\x11\x86\xb3\xe9\x2c\x59\x19\x70\xe2\xc7\x93\xee\xe0\xe4\xe4\x51\xd3\x27\x0f\x72\xbf\x75\xef\x3a\xe4\xf5\x68\xca\xa2\xc3\x53\xf6\xc5\x45\x12\x98\x36\x85\xe9\xfd\xd5\xf8\x66\x9a\xe7\xe8\xad\x45\xac\x14\xa2\x8e\x3a\xc6\x0d\xf3\xde\x6d\x2e\x71\xce\xb9\x1d\x1c\x7a\x00\x64\x68\x24\xde\x46\x2b\x3b\xec\x12\x4f\xfa\x76\xaf\x24\x72\x7f\xcc\x36\xc5\x0b\x2a\xa0\xec\xf3\x74\xfc\x79\x94\xfd\x11\x16\x7e\xcf\x6d\xfa\x9d\x9e\xbb\x25\xb1\x62\x9c\x78\x02\xbb\xce\x4e\xeb\xf5\x53\x51\x95\x42\xda\xe8\x74\xed\xa0\x34\x24\xcf\xe7\x2b\xf1\x0a\xfd\xa2\x05\x2a\xf8\x70\x00\x31\x85\x2a\x72\x08\x77\xc2\x60\x0c\xb9\x93\xb4\xb7\x0c\xef\x7a\xbd\xf5\xfb\xfa\xd2\x54\x2a\x9c\xe8\xab\x17\x6a\x1d\x7a\x54\x60\x47\xc7\xa7\x59\x80\x35\x92\xba\x53\x6c\x70\xd9\x20\xe1\xa2\xad\x87\xcc\xde\xcc\x52\xdc\x55\xef\x3a\xbb\xf7\xa3\x93\xf2\x48\x8d\x51\x08\xdb\x8f\xa3\x8d\xbd\x85\x21\x5d\xb4\x83\xd5\x6b\x81\x30\xe8\x4a\x69\x73\xed\x9b\x7b\x92\x0f\xd1\xb6\x8e\x48\x9f\x4b\xe8\x88\x32\xb4\xe3\x15\x09\xa3\xa4\x34\x7d\x68\x1f\xce\xae\x4f\xc5\x4d\xb7\x64\x41\x5d\x50\x9e\xee\x5c\x25\x3b\xe1\x5b\x08\x33\x5d\xbe\xe3\x0b\xc9\x0e\x97\x4c\x4c\xb1\x7b\x2a\xa7\xfa\xc2\xc6\xca\x8b\xf4\x08\xab\x41\x73\x38\x42\xe8\xf6\xce\xcf\x8d\xdc\x22\xff\x6d\x30\xbb\x2c\xb3\x00\x92\x88\xa3\x25\x9b\x24\x3d\xa4\xf1\x18\x16\x74\xf6\x68\x48\x83\xbd\xdb\xb2\x4c\x6e\xc9\xe2\xdd\xa6\x5c\x18\x69\xc8\x7e\x0c\x46\x54\x11\x55\xd2\x61\xa8\x4d\x74\x2e\x37\x99\xfd\x90\x9f\x23\xdd\xc0\x74\x43\xc9\x4d\xed\x11\x3b\x95\xbc\x02\x74\x1b\x79\x06\xce\xeb\x50\xa0\x42\xc4\xce\x61\x17\x10\x57\x01\x45\x2a\x0e\xf4\xb4\xc8\x25\xf9\x01\x6e\xce\xf3\x25\xa6\x0c\x03\x55\x13\x8a\x0c\x7c\xbd\x6e\xa1\x8a\x88\x8d\x64\xc6\x47\x58\x49\xe0\x4b\x08\x1c\xbe\xc7\xef\xbd\x7d\xfd\xf8\xc2\x01\x27\xdd\xeb\xeb\x7c\xd4\xaf\x38\xec\x50\x8f\x56\x0c\xc0\x28\xec\x79\x82\x97\xdc\xc7\x55\xa8\x03\x1d\xa9\x8a\xd6\xe2\x3c\x54\x3b\xf7\x3b\x43\xc2\x5e\xbe\x86\xf1\x78\x00\x29\xb6\x81\x5b\x06\xeb\x27\x9a\x4b\x3b\x1f\x4c\xa6\x33\x02\xdf\x7e\x72\x53\x3c\xab\xa8\x26\x10\xa5\xf1\x64\xa0\x87\x58\x83\x7e\x14\xcb\x08\x62\x2c\x4f\x7a\xfb\x95\xc9\xfd\xa0\x4b\x41\x83\x6d\x62\x4f\x7c\xc9\x5a\xe0\x4f\x32\x03\x5e\x9a\x9c\x16\x8c\x9f\xa0\x36\x26\xe8\x4b\x37\x49\xe4\xda\xa8\x62\x5e\x54\xae\xb8\xaf\xc5\x8c\x9e\xad\x0b\x35\xf6\xce\x91\x8e\x96\xc7\x2e\xff\xbc\xa3\xd7\x4a\x30\x50\x0f\x50\x86\xaa\x71\xf3\xa0\x8a\x24\xe0\x2b\x16\x7f\x61\x69\xdb\x57\x7a\x21\x71\x5b\x12\x34\xea\x6d\x3e\x60\xff\x38\x95\x55\xfe\x38\xb5\xd4\x64\xcb\x81\xe1\xf2\xec\xb3\xdf\x98\x40\xc0\xcd\x47\xa3\xf5\x50\x1b\xc8\x56\xc2\x43\x43\x09\x1c\xf0\x32\x3e\x77\x27\x44\xb7\x8c\x62\x52\x2e\xbf\x90\xee\x5e\x42\xc5\x96\x8f\x85\xb7\x3b\x47\xd3\xa0\x85\x6c\x82\xf4\xb8\x63\x3d\x04\x34\xa9\x6a\x91\x88\x5e\x45\x2d\xea\xfe\x06\x73\xc8\x60\xda\x81\x29\xbe\xd4\x95\x72\xb8\xb3\x62\x3e\xf4\x0a\xe0\x6a\x68\xdb\x58\x39\xbe\x2c\x5e\x01\x8e\x2b\x48\x29\xae\x1c\x7e\xed\x84\x0b\x5d\x05\x05\x5c\x95\x57\x70\x20\x1c\x54\x91\x9e\x92\x10\x95\xba\x62\x12\xec\xa6\x3f\xdc\x1d\xf5\x2e\xf0\xd4\x1f\xab\xae\x3e\x1b\x8e\x47\xb9\xbd\x1a\xc6\x6f\xfb\xc2\x21\x8c\x6a\xb9\x2a\xda\x7c\x11\x8c\x71\x4c\x7b\x81\x2d\x8d\x9f\xc2\xaa\x93\xbf\xac\x63\x07\x4c\xb6\x7a\x84\x96\xb9\x2f\xdc\x03\x3c\x87\x94\x1d\xdf\x5f\x4f\x38\x4d\xac\xfd\xe2\x75\xa2\x99\xce\xc6\xd7\x78\xff\xd3\xbb\x20\x05\x55\x8a\x5c\x74\xdd\xc2\xb4\x58\x99\xf0\xc1\x69\xd4\x64\x61\xbe\xa0\x55\x4d\x31\x5a\xe4\x1b\xc6\xd4\xf4\x57\x1c\xc2\xf1\xcf\x0b\x6f\xce\x85\xaf\xa8\xaa\x8f\x7d\xa5\xa3\x6a\x7c\x1b\x9c\x38\xf5\x71\x90\xc1\x1a\x2b\xf3\xa7\xbc\x3e\x4a\x21\xa1\xeb\x72\xad\x91\x78\xb2\x41\x9b\xab\x18\xe3\x5b\x0e\x59\x6b\x4a\x85\x48\x61\x01\x79\x90\x37\xc5\x5e\x68\x9e\xc2\xe4\x2c\xbd\xf8\x35\xc7\xa3\x72\xce\x9b\xd1\x92\xbc\xe7\xe7\xc0\x3a\x00\x71\xb9\xdc\xed\x5d\xeb\x57\xc3\xeb\x1b\xb5\x54\xf1\x22\x40\xee\x71\xa8\x01\xe8\xf8\xe3\x2f\xf9\x1d\xc6\xbb\x27\x21\xbc\x30\x0b\x4a\x04\x4e\xc2\xc3\x22\xe5\x8d\xe8\x53\x7e\xa7\x5f\xf3\xa8\x08\x6a\x6b\x06\xc5\x6c\x27\x89\x86\xbf\xcd\xd4\x35\x72\x7b\x43\x73\xf7\xf5\xfb\x0f\x5b\x07\x7f\x3a\xcc\xd6\x7f\xb6\xc0\x4f\xa6\xcf\x8f\x7f\x3e\x89\xbb\xcf\x4f\x62\xf8\x8c\xe1\x3b\xe9\x3e\x4f\x4e\x92\xf5\x2b\xc3\x99\xd9\xb8\x56\xb6\x6e\xd6\x25\x22\x23\xf9\x88\xf8\x03\x5a\xcf\x3d\x6c\x74\x92\x93\x16\x0b\x47\xa0\x61\xb0\xfd\x0f\xdb\x7b\xbb\xfc\xdc\xb5\x1e\xc7\x27\xeb\x28\x4e\xb2\x9a\xcc\xf1\xeb\xf9\xf1\xc9\xf4\xe4\xb0\xfb\xbc\x73\xf2\xfc\x64\x1d\x5f\x76\xaf\x8c\x7b\xb4\x29\xbc\xdf\xaa\xc7\x1c\xb3\xca\xee\xa0\x8a\x4f\xbc\x36\x1a\x73\xb0\xc7\x30\xe7\x7e\x95\x5c\xa2\xf2\x77\xe0\xf9\xd4\x7a\xef\x10\xb7\xec\x96\xee\xe0\x41\xda\x84\x9e\x2f\xfc\x46\x19\xb8\xf5\xb0\x4a\xb2\xe4\xb9\x22\x5a\xf3\x3e\xd7\xda\xf2\xec\x53\xaf\x47\xcd\x68\x25\x0b\x7d\xad\x58\x5f\xa7\x41\x3b\x50\xcf\x70\x76\x06\x28\x38\x35\x43\x74\x41\x4a\x42\x7b\x35\x8b\x58\xcd\x6a\x0d\xd6\x3e\xc3\xc1\xa1\xf6\xac\xf9\xac\x76\x31\x9e\xd5\x9e\xdd\x6f\x2c\x9e\x35\x23\x5e\xfb\xed\x62\x63\x68\xed\x7b\x63\xfe\x5d\x54\xce\x35\x25\x8f\x20\xe4\x3f\xc4\x7a\x02\x09\xfd\x7b\x88\xfe\xb0\xee\x39\x89\x72\xd2\xae\x7c\xfa\x2b\xd1\x62\xde\xcd\x98\xa7\x31\x7d\xa0\xde\x72\xe8\xc6\x30\xb1\x77\xf4\x55\xaf\xd3\xcf\x31\xfd\xb5\xc6\x46\x5b\x5d\xe3\xb7\xce\x2c\x64\x9d\xc8\x51\xa1\x03\x22\xc5\xc5\x33\x8d\x2d\xc2\x95\xbe\xf7\xbb\x54\x56\x59\x40\x6d\x63\x6c\xe8\xa4\x5d\x38\x72\x54\xb3\x48\x54\xb6\xcc\x26\x15\xeb\xc4\x63\x06\x36\xbb\x5e\xb7\x3d\x16\x63\xd0\x6c\x90\xac\x84\xe1\x93\x74\xa9\xd7\xb8\xa3\xc2\xd7\xa0\xf8\x41\x40\x95\x4c\x6a\x55\xef\x8b\xd7\xe1\x99\xba\xe1\x29\xe7\x52\xdc\x5c\x09\x26\xcd\x47\xda\x5d\x19\xa0\x0b\x7b\xd7\x3e\x2e\x2a\xe9\xb8\xcf\x34\x10\xbf\x2d\x24\x7a\x31\xd8\x94\x4d\x7d\x57\xf7\x7f\x81\xf7\xb6\x3c\x5b\x19\xb5\x47\x7a\x5d\xb3\xa4\xff\x1f\x6e\x9f\x62\x42\xbf\xba\x89\xe2\x8d\x97\x79\x8d\x45\x77\xd1\x35\xab\x78\x79\x34\x18\x96\x68\xa6\x44\x47\x49\x7b\x75\x9b\x3f\x2d\xcd\xf4\xf2\x79\x9e\x76\x32\xb7\xb3\x7f\xb0\xf3\xfa\xed\xdf\xbd\xf0\x4d\xfc\x85\x04\x74\x3e\x74\xe7\x28\x45\x20\xdf\xc9\xfa\xc0\xfc\xfd\xcd\x81\x15\x3b\xfa\xfb\xee\xbb\x37\xb3\xd9\xf5\x01\xfb\x74\x9a\xcf\xb5\x55\x83\xc9\x9d\x13\xad\x87\x53\x0a\xbb\x0c\xfc\xbb\xdc\xdb\x47\xbb\xd3\x2f\x57\xc3\x4d\x2a\x7f\x74\xb4\xdf\xfc\x7d\x73\x23\x72\xfe\xe9\x5a\xc9\xfd\xe2\xab\x4a\xbf\xd0\xa5\x37\xbf\xb2\xb4\x2f\xf9\x02\x4b\x06\xfa\xe2\xab\xe8\xea\xf9\x87\xde\xd9\x27\x38\x43\xb2\xca\xf8\x97\x4b\x74\x35\x78\x44\x0e\xa5\x59\x85\xa9\x7c\xd1\x1c\x76\x0a\xfa\x7c\xc6\xfd\x0f\xd6\xe1\xf5\x78\x58\xb1\xbf\xa9\x24\xd8\xe3\xf6\xb7\x8e\xde\x7c\xd8\xdd\x3a\xda\x7e\x83\x43\x00\x9b\x74\xe7\x37\xb0\x4b\xc7\x27\x1d\xf8\xc6\xaf\xa4\x13\xff\x06\x75\xf7\xd1\x90\xa8\x79\xb5\xf3\x7a\xeb\xc7\x77\x47\x1f\xf6\xf7\x0e\x50\xec\x84\x1c\x53\xff\x61\xc3\xe0\xef\x34\xfd\xee\xbb\x17\xe6\x1c\x22\x36\x5b\x0b\x36\x3f\x89\x03\x53\xac\xdc\xc6\xc3\x4c\x79\x27\x9f\xc8\x05\x9d\xf6\x46\x17\x6f\x47\x6f\x66\x57\xc3\xdf\x39\x2f\x5c\xc4\x10\x15\x33\x2d\x49\x2d\x17\xbc\x5f\x5d\xc5\x87\xb1\xdf\xa1\xd3\xc8\xd5\x55\x39\xbe\x63\xa0\x77\x3a\x85\x9c\xa9\x45\x44\xcc\x04\x47\xab\xab\x9c\x10\x91\xe3\x39\xbf\xa2\x21\x60\x2f\xd2\x4a\xaf\x69\xe8\x04\x2f\x51\xdc\x16\x6c\x97\xe8\x09\xcf\x3b\x08\xf0\x9d\xcb\xe2\xae\x58\xc0\x39\xca\x13\x23\x07\xf5\x3a\x15\x45\xf7\x20\x71\x3f\x47\x91\xc4\x1f\x0f\xde\xe2\x22\x22\x93\xfe\xde\x24\x2e\x9c\x37\xad\x39\x84\xf9\xdc\xc5\x4a\x69\xf1\x40\x62\x2d\xe4\x92\x11\x7e\xde\x7d\x91\x93\x53\xa6\x70\x81\x72\xd8\xd6\x50\xfa\xc2\xfb\xc8\x2b\xf7\x88\x4d\xc2\x17\x07\x74\x7e\x57\xce\x81\xd1\x90\x4a\x7e\xe5\x2a\xca\x43\x74\x44\xb6\x1b\x2e\x0b\xa9\x87\x3e\x4f\x0f\x75\xc5\x5d\x77\x93\xdb\x14\x59\x4b\xe4\x72\x0e\x23\x94\x7b\x39\x82\x85\xbe\xe4\xf0\x03\x9d\x32\xb3\xef\x3d\x2f\x58\x4e\x61\x43\xaf\x2b\xac\x04\x75\x2f\xae\x70\x8a\x5e\xf0\xd9\x13\x4e\x6d\x23\x0d\x07\x90\x01\x90\x6b\x96\x5a\x2b\xa5\x01\xb7\x02\xea\x94\x82\x5c\x97\xca\xc8\xfa\xbb\xc0\x2a\xb3\x5a\x94\xe4\x71\x06\x91\x56\xdc\xcb\xa1\x2f\x4d\x3c\x5b\xb8\x3e\xe2\x68\x30\x9d\x60\x23\x2f\x68\xb1\xe7\x35\xba\x32\x73\x8e\xfb\xd1\x3b\x3c\xbd\x26\xb9\x62\xbf\x91\x01\x4f\xfe\x59\x43\xe3\xc3\xb5\xab\x1b\xc8\x4e\x56\x24\x58\xfc\x1f\x5f\x9b\x7a\xe4\xb4\x08\xe5\xca\x23\xc5\xcd\x11\xd2\xfc\xb3\x08\x9d\xca\xf8\x3e\x53\x07\x21\xe6\x66\xe8\x82\x41\x12\x3b\xea\x18\xe0\x20\x1e\xf3\x4f\x37\xad\x8c\x55\xa5\x17\x61\x6f\xe3\xbe\x31\x26\x4b\x96\x3c\x19\xc9\x05\xe2\xb2\xa9\x82\x89\x91\xf1\xee\xdc\xed\xa2\x2e\x4b\x7b\x0b\x74\xc9\x80\x12\x35\xfc\x9e\x28\x9e\xc7\xc8\xe5\xe5\xdf\x7a\x93\x11\x4a\xd8\xb3\xde\x57\x91\x4e\x51\x64\xe4\x9c\x87\x4a\xe6\x6d\x2b\x04\x8c\xc2\x0c\x5b\x47\x7b\x07\x40\x04\x23\xec\xa0\xa8\x02\x09\xbe\x7c\x41\xe7\xe3\x79\x45\x32\x7a\x1f\x8b\x48\x7d\xac\x2a\x11\x48\xa8\x17\x87\xa1\xf3\x3f\x9a\x51\x57\xb3\xdc\xb9\x8a\x37\xda\x40\x64\x2f\xeb\x05\x6e\xe4\xcd\x69\x76\x1a\x46\xf8\x57\xce\x5e\xd2\xf1\x81\xd3\xa4\xd3\x6b\x9c\xa6\xbd\x34\x88\x3a\xf5\xa2\x4a\x0b\xb4\x48\xf8\xef\x40\x20\x0e\x30\xe8\xa5\x1b\xc9\x5a\x5c\xa8\x74\x03\x36\xb0\xe8\xf9\x53\x6a\x0b\x40\x3f\x0f\x6b\x02\x18\xeb\x5f\x0d\x63\xbd\x04\xe3\xb7\x5f\x0d\xe3\xb7\x25\x18\x3f\x7f\x35\x8c\x9f\x4b\x30\xb2\x48\xe6\x41\x96\x65\x5f\x0d\x0e\x5f\x16\x8b\x00\x57\xbe\x01\xce\x4a\x05\x9c\x6f\x42\xa7\x02\x9b\x6f\x40\xa6\x04\xe5\xfb\xaf\x06\xf2\x7d\x09\xc6\xcb\xaf\x86\xf1\xb2\x8c\xc7\xd7\xb7\xe6\xfb\x72\x6b\x5e\x7e\x3d\x94\x97\x65\x28\xf5\xfa\x57\x43\xa9\xd7\x4b\x50\x80\x73\xf8\x5a\x28\xf3\x79\x19\x97\xaf\x47\xa5\x8c\xc9\x13\x60\x84\x85\xc2\x6c\x41\x12\x4d\xbd\x65\x00\x1d\x35\x0e\xcb\x2c\x16\x66\xe7\x70\x7b\x6b\x7f\x27\xbb\x1f\xa5\xd1\x09\xda\x3c\x83\x9f\x73\x60\xac\xe0\x07\xb8\xf5\x59\x1a\xfd\xb7\xc8\xdc\xa6\xd1\xff\x88\xd0\x82\x7a\x0a\xff\xcc\xb3\xe8\x59\x0a\xff\x16\xe6\x1d\xf0\x81\xda\xad\x23\x8b\x95\xc9\x89\x5a\x42\x99\xfc\x2e\xda\x94\x5b\x33\xb5\x67\x10\x0d\x1b\x0a\x89\x33\x53\x22\x1c\xfe\xbf\x78\xf4\xd9\xdb\x21\xc1\x22\x4b\xfe\xf8\x87\xd9\x40\x7e\x66\xe1\x7d\x10\xd8\x29\xbf\xc1\x73\x0c\x1c\x08\x67\xdb\x97\x30\xdf\x22\x0e\xcf\xd0\xfe\xa1\xb5\xbb\x4a\x57\xba\x14\x43\xae\x4d\x31\xd6\xc3\xfc\xde\xd5\x66\x2f\x79\xd9\xae\x8e\xd4\xe3\x53\x85\x87\xf3\x25\x85\x41\x1d\x4c\xe3\xe8\x04\x2d\xcd\x27\x72\x55\xd6\xeb\x5b\x6d\x77\x86\xe1\xc5\x4d\x25\xbf\x98\x00\xb1\xc9\xf3\xb9\x83\xd3\x8c\x2c\x43\x1c\xe6\xba\xce\xf3\x4f\x71\xa2\x6a\x90\xc4\x12\xe8\xb7\xc8\x5c\x38\xc8\x3e\x3f\xc7\x0b\xc6\x9f\x7b\x50\xd5\xbd\xc1\xba\xa2\x7b\xa4\xc7\xd8\x2b\xc7\x1b\xe4\x17\x03\x7b\x29\x53\x5d\x78\xac\xbe\xfd\x9d\x54\x62\xb3\x36\xa9\x43\xd7\xf0\x25\x95\xc3\xd4\x55\xce\xdf\x18\xb4\xa7\x84\x62\x1c\x01\x9b\xb0\x38\xee\x36\x4d\x3b\xed\xd8\x5e\x93\x0a\xe8\x6d\x9a\x5f\xe0\x53\xdf\xd1\x2c\xe3\x2a\x8d\xa2\x21\x4c\x7d\x3b\xd2\x63\xa3\x3a\x0d\xda\x75\x1c\xe9\x1e\x5d\x74\x53\xc3\x4e\x00\x74\x86\x7a\x1d\xa1\x34\x6f\x46\xd3\xcb\xc1\xb9\xef\x2f\xa3\x8a\xb9\x4c\x9c\x25\x51\xd3\xb0\xd1\x70\xfc\xb1\x14\xf8\xdb\xe5\x60\x96\x4f\xaf\x51\x1f\xce\xf5\xfd\x7d\x50\x00\xdf\xd6\x07\xa3\x1b\xf6\xc4\x7e\x76\xb9\x99\x49\xbe\x86\x1a\x5f\x73\x76\xf9\x22\x83\x34\x15\xb7\x99\x98\xf3\x51\xe6\xb8\xb5\x63\x29\xd5\x85\xd8\x4d\x15\x0d\x85\x30\xea\x45\x10\xf5\x02\x26\xff\xe8\x45\x27\x7e\x6a\x07\x43\x09\x80\x91\x42\x99\x45\xd0\xda\xec\x45\x02\x91\x9b\x5f\x03\x68\x93\x01\x6d\x16\x00\x6d\x22\xa0\xa7\xc3\xb1\x23\x4e\xb0\x8a\xe3\x7e\x6c\xd2\x60\xdc\x1b\x6b\x6a\x98\xb9\x3a\x2b\x2e\x48\x47\x16\xb2\x02\x1d\x47\x3f\x8e\x72\x6b\xbb\x0e\xef\x3c\x6b\xb8\xaa\x7b\x10\x9c\xd4\x22\xa3\xeb\xf7\x60\x5a\xc9\x42\x93\x18\x41\x2a\x38\x07\x70\x5b\x16\xc0\x9c\xa6\x4a\x0e\xa0\x37\x71\x5a\xc2\xe4\x8c\x8f\x62\xbc\x0f\x5b\x99\x27\x0b\x03\xcd\xf9\xca\x62\x8c\x0a\xbe\xc0\xc2\x1c\x51\xfa\x46\x22\x04\x79\x73\x95\x0d\xe6\xf3\x96\x16\x45\x94\xb6\x40\x52\x89\xde\x75\x1e\x20\x71\x58\x80\xbd\xe9\x5b\x7a\x94\x56\x29\x34\x9f\x5d\xbe\x44\x2f\x5a\x40\x4e\xfe\x08\x9b\x3e\x74\x8e\xd1\x4b\xa2\xaa\x48\x54\x43\xba\x73\x76\x09\xdb\xf2\xc9\xc4\x7d\xfe\x37\x1f\x39\x72\x9f\xff\xc3\x7d\xfd\xef\xff\x8b\x3f\x11\x3c\x11\xb4\xe5\xc8\xf4\x10\x99\x7f\x11\x32\xf3\x39\xc5\x6c\x61\xcc\x3f\x24\x26\xfa\xe0\x80\xae\x7a\x98\x3b\x5f\xae\xf7\xc4\xa5\x6e\x25\xce\x6b\xae\x50\xc3\x7e\x85\xb4\x9a\x86\xd3\xcf\x36\x75\xad\x8a\x41\xb6\x75\x68\xf2\x11\x8a\x87\x8c\xfa\x19\xfc\xb3\x00\xc8\x08\x00\x0b\xf8\x0d\xd1\x7f\xa2\x3f\x50\x50\x99\xa4\x13\x4d\x6b\x51\x83\xbe\x1b\x68\x7b\x5d\x8d\x50\x54\x3b\x96\xb0\x72\x56\xcc\x4a\xf6\xb6\x36\xd4\x4c\x4b\xa1\xc7\x1b\x10\x68\xcb\xf9\x5d\x1d\x1a\xe3\x68\x88\x3b\x31\xde\xd3\xd1\x96\x5c\x63\xe4\xc9\xb6\x63\x6f\x86\x28\xdd\x5c\x8d\xee\x5b\x8b\xd0\x3d\x6e\xed\xf8\x7e\x73\xd1\x6d\x46\x6c\xac\xd9\x30\xe2\xc6\x21\x42\x17\xcc\x76\x9f\x0a\xf4\x13\x64\x3f\x16\x77\x86\x51\xc4\xdd\x92\xa9\x8e\x78\x78\x73\x66\x02\xaa\xf4\x64\x1e\xda\xa2\x49\x7f\x06\xb6\xa1\x25\xa3\x95\x88\x6b\x4d\x48\xf5\xf6\x67\x71\x51\xd9\x85\xce\x84\x99\x80\xe4\x04\xc4\x51\x1c\x35\x59\x62\x2e\x51\x82\xa6\x36\x3d\x9d\x1b\xea\xad\xd7\xb9\x48\x71\xb3\x17\x40\x30\x53\xb1\x36\x71\x6d\x29\xad\x92\x90\xdd\x83\xcb\xb8\xe3\x75\x4d\x75\x75\xf3\xf9\x83\xd5\xc1\x84\xce\xf1\x71\xee\xe1\xea\xf8\xb2\xa5\x44\x50\xdf\xb2\xc5\x68\x9c\x19\x74\xe1\x07\x54\x78\xa1\x77\xbe\x85\x0c\x74\xeb\xb9\xf8\x57\x5d\x46\xfd\x79\xbe\x12\xe1\x97\x9c\x44\xf1\x49\xa5\xb4\xf2\x76\x02\xf3\x2c\x16\x32\xd1\x0a\xf4\xc0\xcf\x33\xa4\x95\xaf\xc6\x33\x22\x95\x6f\x89\xb6\x5f\xe5\xb3\xcb\x31\xe9\x10\x19\xbe\x69\x9b\x02\x7a\x24\x6d\x49\x57\x33\x5f\x3b\x25\x51\xea\xe6\x11\x3e\x91\x26\xa0\x9e\x81\xcc\x8e\x69\xde\x4f\xcd\xc9\x76\x62\xf3\xa3\x89\x07\x46\x3f\xd3\xe0\x08\x4d\x1c\xf7\x90\x27\x41\x87\x41\x9c\x9b\xe4\x5d\x5d\x83\x75\x3b\x5c\xe4\x12\xb6\xb7\xaa\x25\xae\x0c\x34\x24\x66\xc4\x92\x7b\xdf\x89\x19\xa1\x63\x1f\x56\x05\x83\x35\xa6\x55\x2d\x41\x36\xcc\xb3\x61\x82\x5c\x7a\xd3\xce\x5c\x65\xe2\x53\x5e\x4d\x6a\xc5\x64\x61\x37\xf1\x7c\x74\xf9\x61\xa6\x39\x36\x3f\x2b\x4f\x29\x42\x80\x4c\xfd\x3a\x16\x09\xaf\x96\xb5\xc3\x02\xca\x02\xfc\x28\xf1\xb1\x01\xd7\x45\x29\x5d\xa3\x38\xde\x62\x9a\xa7\x1e\xec\x44\x3b\xe3\x9f\xd7\x23\x86\x6a\xf4\xf1\x48\x11\xc9\xb6\xab\x0c\x42\x40\x9d\xe3\xaa\x83\x9c\x9b\xf2\xe2\x9f\x3b\x3c\x4f\xa2\xff\x85\xc1\xc5\xa8\x70\x04\x0c\x15\xcc\xa6\xaa\x20\xe3\xc3\x82\x25\x3c\xf7\x09\x15\xfb\x6d\x0f\x74\xb8\xb0\x4a\x2b\x75\xc6\xe6\xe7\xfd\xd8\xe3\x31\x60\xc9\x72\xb6\xeb\x8e\x7a\x1f\x26\xb4\xac\xe6\x96\xe5\xd1\x96\x17\x69\xb4\xb8\x90\x5a\xa7\xae\xac\xac\xf6\xa2\x46\xf7\x2f\x37\x63\xab\xb8\xf7\xc0\xd2\x85\x35\x62\xa9\x02\xef\x91\xb8\xd4\x27\xbd\xcf\x62\xc4\x84\x80\x98\x7c\x7a\xd6\xbb\x46\x63\x63\x4f\xdb\x86\x1e\x5a\xf7\x38\xdb\x1c\x78\x5a\xaf\x0c\x5c\x9c\xb9\xdb\xa5\x44\xbe\x00\xed\x32\x2d\x6c\xe2\x9a\x0f\xd5\xab\xfd\x77\x49\x1b\x8a\x58\xf3\xd4\xc7\x27\xfd\xde\xda\x79\xf7\xfe\xbb\xc5\xfa\xc0\x92\x94\x2a\x1a\x7d\x33\x1a\xe0\xfb\x4c\x8d\xd1\xa8\x1d\x9f\x9c\xdc\x44\x8d\x4b\x64\x22\xba\x51\xc8\x3a\x7f\x67\xa6\x82\x75\x95\xb1\x1a\x9a\x2b\x6f\x81\x82\x41\x59\xd3\xfa\xbd\xa8\x62\x8a\x22\xda\x75\xc6\xb7\x0b\x70\x02\xe9\xb6\x2d\x14\x88\xee\xc0\x3f\x38\x1a\x2c\x5c\x07\x3b\xfd\xcd\xe8\xe4\x44\xfa\xc2\xa6\x6d\xb8\x1d\x0d\xba\x38\xe3\x91\x49\xca\xec\x6c\xe3\x29\x9b\x89\x1b\x01\x69\x92\x18\x18\x7a\x78\x77\x11\x23\x44\x30\x59\x59\x4f\xd1\x35\x04\x1a\x10\xec\x6e\x15\x07\x0c\x58\x6a\x57\x83\x11\x0a\xdb\xd7\x08\x71\xd9\x4e\x12\xb9\xbe\xdf\xe7\x0d\xc7\xd5\x49\xcc\x97\x59\x65\x8f\xf4\x26\xbc\x50\xa1\xb4\x8c\x73\xf0\x9b\x00\x67\xcb\x6c\xf6\xea\x6b\x17\xae\xa2\xf9\x8f\x9d\x83\xbd\x0a\x53\x32\x1b\x0b\x23\x19\x96\x5c\xcc\x70\x2a\x53\x84\xf0\x6e\x86\x3a\xad\xfa\x82\x86\xc8\x22\xfe\x09\x2e\x60\x7c\x2b\xf0\x2f\xdf\xef\x10\x10\x4b\x39\x98\x7a\xe1\x23\x91\xe4\x1d\x5f\x0c\x80\xb2\xed\x1d\x30\x18\x5b\xfb\x76\x6f\x38\xe4\x0c\xe7\x83\x7c\xd8\xdf\x3a\x3b\x03\x2e\x94\x23\xf8\x8d\x48\xed\x74\xdc\x35\x30\x59\x07\xa3\xc0\x6b\x40\x71\xa4\x06\xfc\x0a\xcd\x4b\x03\x71\x8a\xcc\x3d\x9f\x3b\xb1\x51\x3c\x8f\x36\xd0\x28\x60\xdb\x0b\x98\x61\x36\x3e\x30\x5d\x4f\x06\x57\xbd\xc9\x5d\x2c\x67\xcc\x29\x1a\xb2\xa0\xe7\x39\xe7\xa1\xb7\x86\x0e\xdd\xca\xd7\x27\xc2\x84\x15\x30\xe9\x8d\x60\x75\xba\xa3\x29\x15\x08\xae\xb2\x8e\xd1\x2d\x29\x9b\x9c\x18\xc2\x36\x38\xe9\x0d\xb3\x95\x95\x20\x2c\xa9\x34\x90\x3d\x74\xaf\xbe\x12\x46\x88\xb4\x9b\x11\xc4\x8b\xe6\x30\x24\xba\x6d\x79\x56\xc6\x25\x86\x1d\x3f\x49\x24\xad\xd4\xbd\xf6\x56\x04\xab\xb8\xb9\xca\x21\x6b\x54\xb8\xee\xb1\x50\x8e\x8b\x50\xc8\xc7\xcc\xab\xfc\x6c\x88\xce\xb7\x08\x8b\xea\x82\xf7\xc5\x82\x3c\xdc\xb1\xf6\x17\xe1\xef\xac\xa4\x54\xd2\x76\x45\x64\x7f\xb5\xad\xae\x20\x8e\x38\x07\x7a\xb6\xf9\xea\x80\x13\x19\xd9\xee\xfc\xb6\x0f\x93\x56\xf2\xa9\x6e\xde\xb0\xb0\xfd\xc0\x6c\x24\x4e\xd1\x11\x2f\x19\x58\xb1\xe2\xcb\xac\x3d\xa2\x25\x13\xf6\x2e\xda\x12\x41\x06\x11\xd9\x3e\xe2\xae\x30\x13\x2d\xae\x4e\x1c\xf6\xbb\x5a\x0a\x36\xc5\x42\x4e\xec\x07\xbb\xe2\x4c\x01\x66\x08\xc9\x26\xdb\x72\x15\x5d\x4a\x2b\xc8\x02\x4e\x92\x94\x99\xd0\xa7\xc2\x50\xcb\x52\xc1\x28\xef\x44\xbb\xfb\x7b\x87\x87\x6f\x7f\x78\xb7\x13\xb9\x65\x22\xd9\xab\x4f\xce\x57\xd3\x0b\x19\x87\xfb\xaa\x93\xeb\xf4\x6e\x34\x43\x5b\xcf\xd1\x21\x7d\xd8\xa3\x2b\xb9\xbf\x61\x79\x40\xf4\x43\xe0\x8f\xb1\xe8\x87\xc0\xbe\x46\x07\x67\xd9\x17\x8b\xae\x97\x2a\x84\xec\xc7\xb0\x99\xe2\xe9\xd6\xdf\x6a\x1a\x87\x4a\xb8\x21\x7b\x12\x58\xdc\xbb\x7d\xde\x44\xee\x6a\x08\x2f\xbd\xf2\xd0\xed\x63\x56\x45\x25\x92\xaa\xd6\xde\xe4\x63\xd8\x4d\xf4\x1d\x16\xf0\x8c\xd8\x1c\xdf\x14\x3a\xb6\x47\x9a\xc5\x2c\x5f\x54\x01\x2d\x29\x5e\x1d\xe5\x2d\x93\x6f\x9a\xfc\x85\xc9\xbf\xf3\xd7\xdf\x01\x42\x2f\x37\x92\xe2\x6a\x73\xd0\xcc\x4c\xdd\xfe\x12\x05\x41\xcd\x9a\x16\x2c\x36\xfc\xdd\x94\xdf\x17\xf2\xfb\xdd\x7c\xbe\x92\xb7\xd0\x22\xe7\x26\xfe\x79\x81\x7f\xbe\x73\xbb\x3b\xc2\x59\xf8\x57\x64\x6e\xe9\x12\x44\x0b\xf8\xd0\x61\x5d\x67\x68\x6b\xa0\x72\xef\xc8\xeb\x78\x45\x2f\xea\xa7\xec\x0c\x96\x16\xa8\x7e\x71\x17\xc3\x94\x44\xb7\x64\x42\x08\x35\xba\x56\xd8\x90\x17\x3c\x84\x2b\x48\x10\xd4\xe7\x89\xbf\xa9\xe5\x4e\xec\xf5\x38\x6a\xe4\x2d\xe4\xce\x8c\xba\x8d\x48\xf0\x6d\x1d\x16\xcc\xeb\x91\xb6\xce\x64\x26\x83\x8b\x4b\xa5\xa9\xf5\x84\xf3\xc4\x79\xf8\x5e\xc4\x00\xe0\x44\x61\x69\x5b\x4a\x31\x8e\xd4\xe1\x51\x1b\xfd\x66\x14\xaa\x1e\xe6\xe7\xb0\x36\x06\xfd\xfe\x30\xff\x06\x1c\xb0\x74\x90\xd0\x61\x50\x41\x1c\x23\x52\x3c\xf8\x38\x34\x11\x86\xc3\x12\x4d\xe8\x23\x00\x15\x51\x6e\xc6\xe9\xa0\xb2\x15\xff\x8e\x5e\x24\x40\xe5\xae\x2c\xe0\x58\x46\xc9\x33\x10\x55\xb7\x18\x3e\x15\x5f\xaf\x48\x77\xb5\x6a\x89\xe2\xc4\x76\x13\x25\x5a\xa0\x4d\x24\xf8\xd7\x86\x7f\xf4\x9e\xe1\xa1\xc8\x79\xae\xb4\x9f\x27\x66\x25\xd8\x9f\xda\x91\x13\x10\xc3\x37\x1e\x05\x40\x2e\x8f\x7d\x0c\xf9\x9c\xaf\xec\xa8\x2a\xad\x8a\x12\x24\xef\x27\xd7\x25\xa9\xea\x8e\x07\x78\x96\x90\x90\xd3\x9e\x70\x31\x85\x37\xd1\x40\x81\x1f\x75\x71\x5c\x0b\xab\xfa\x96\x5f\x06\x71\x80\xdc\xde\x2c\xc4\x14\x18\x09\xbe\x16\x59\x89\xcb\x4c\x46\x34\x87\xbe\x49\xd4\x34\x6e\x7b\x10\x76\x7e\xf1\xb4\x72\x4c\x88\xea\x6f\x5c\xc6\x16\xb3\xa5\x48\x85\x4c\x0d\xbe\x04\xe9\x43\x40\xec\x69\x2e\xfa\x10\xbc\x98\xbe\x1e\xf1\xe4\x58\x8a\x71\x8a\x18\x8b\xc3\x74\x38\x13\x42\x86\xac\xf2\xe5\x98\xbd\x4b\xdf\x7b\x47\xac\x17\x30\xed\x28\xd2\x3b\x62\xc5\xea\x42\x4f\xac\x17\x53\xeb\x84\x15\xd3\xd0\xb7\x71\x30\x2a\x4e\x97\xd5\xea\xf4\x50\x2a\x66\x4e\x16\x25\x0f\xef\x6a\x75\x31\xa2\x8b\x85\x54\xe9\x27\xae\x1e\x27\xec\x4c\xb5\x09\x2e\x91\xa8\xf2\x87\x0d\xf4\x28\xe7\x43\x45\x5e\x98\x56\xa7\x29\xce\x0b\xa1\x7d\x8e\xbf\xaf\xea\xe1\x2c\x42\x81\x53\x5a\xec\x0c\xbe\x8a\xda\x5f\x5d\x0f\x07\x39\xb0\xfc\xae\x7e\xf2\x99\xb4\xe4\x1a\x7f\x43\xf3\x1c\x78\x95\x5f\x3b\x83\xb3\x02\x6e\x4f\x28\x2e\x47\x20\xe8\xc0\xe0\xb6\x28\x42\xbe\x80\xb1\x97\x4f\x64\x9b\xdb\x15\x44\x58\xf0\x95\x0c\x42\x70\x75\x66\x47\x79\x13\x22\x66\x6e\x2f\x28\x76\x9e\x6c\x05\xc5\xde\x73\x07\xbb\x07\xfb\xaf\x43\xfd\xc7\x20\x8a\x6d\x58\x36\xa3\x3b\x3a\xdf\xeb\x70\x43\x0a\x41\x58\xab\x82\x7a\x34\x1c\x2b\x95\xba\x1e\x14\xd3\x44\xb6\x9d\x0e\xf3\xa7\x10\x0f\xc9\xbc\xf5\xfe\xd5\x23\xc4\xe3\x1b\xa9\x87\x86\x8f\x93\xde\x87\x8b\xe3\x50\x22\x6b\x68\xff\x1c\xfd\x35\x3c\xd4\xff\xf5\x7a\x44\xde\xa2\xbf\x01\x99\xc4\x70\x77\xd9\x7a\x1e\xc3\x87\x0c\x12\x40\x32\xb9\x2d\x7c\x60\x45\x65\x11\x49\x57\xb1\xe0\x18\x8b\x7d\x7d\x05\x8a\xbe\xd5\x0e\x41\x5f\xf1\x63\x28\xf6\xfa\xa8\x1e\x8b\x5a\x27\x0f\x20\xf8\x7d\x84\x32\x57\x24\x33\x45\x22\x4f\x5f\x81\x9c\xee\x02\x87\x9e\xad\xf4\x29\xb3\x4d\x8c\x40\xa1\x1b\x6e\x46\xb3\x02\xbf\x46\x44\xce\x98\xdb\xc9\xd3\x70\x2a\x82\x74\x74\x9b\xb1\x0b\x93\x9f\x82\xe3\x8d\x10\xcd\x0a\xd4\x9e\x47\x28\xe3\x88\x32\x8a\x4f\x46\x4f\xa0\x15\xb0\xba\xa9\x22\x44\x54\x30\x38\xfe\xf8\x2e\x49\x8a\x97\x38\x55\x23\xbb\xe6\x88\x8b\xc3\x48\xdd\xac\x2d\x41\xac\x12\xd2\x8a\x83\x24\x9c\x7b\xbc\xa4\x70\x88\x15\x32\x08\xee\x58\xed\x9b\xc7\xe7\x75\xa7\x66\xed\x9e\xaf\x28\x73\xc8\x33\xf0\xe9\xb4\xf8\x26\x41\x19\x97\xbe\x49\x2c\xe3\x7d\xd5\x66\x80\x8a\xcc\xe7\x55\xef\x12\xf3\xb9\xdc\xcf\x54\x6f\x1d\x15\xaf\x15\x94\x8f\x39\xc3\xc2\xae\x24\xaf\x16\x95\x00\xb9\x05\x4f\x7a\xc5\x30\xea\x7a\x23\xe8\xc3\x72\x07\xd2\x1e\xfb\x7a\x54\xc1\x06\xea\x59\xe4\x6e\xbb\xf0\xd2\xfc\xc1\x13\x02\x31\xbd\xe6\xda\x8c\x33\xa8\xae\x20\xc7\x9c\x49\x65\x41\xb4\xad\x67\xdc\x89\x6f\x33\x60\xec\x6f\x26\xf9\x61\xef\x3c\x17\x11\xfc\x31\x6a\x00\xab\xd6\xc2\x4e\x55\xaf\xdf\xc2\x6e\x96\x8f\x48\x25\x50\xb5\x1c\x66\x14\x9a\x74\xd8\x67\xe9\xef\x29\x5e\x5c\x65\xb7\x26\x5a\x5d\xbd\x8d\x06\xc0\x18\xcf\xe7\xf1\x75\x13\x02\x4a\x8c\xee\x9a\xe0\x04\x7e\x43\x92\x7b\xce\x84\x66\x2e\xd1\x60\x66\x76\x8b\x41\xf8\x48\xb4\x5c\xf5\x03\x0f\x50\x41\x47\xa0\x7e\x68\x55\x9b\xcd\x14\x5a\x58\xd1\xd8\x62\x8f\xe9\x86\xdb\x7e\xc2\xa2\xa4\xca\x6b\x0d\xb1\x25\x5e\x0b\x1c\xef\xc9\x82\x03\xb2\xdc\x5f\xb1\x54\xbe\x68\x60\x3a\xae\x19\x9f\x21\x92\xc8\x5e\xd8\xba\xbb\x1a\x59\x41\x49\x7f\x5c\x7b\x98\xff\x6c\x7f\xbe\x1c\x0c\xf3\xf0\xee\xd2\xa0\xf4\x5b\xf9\x82\xb4\x30\xeb\x4a\xbc\x6f\xc8\xa2\x85\x4c\x78\xd7\x5d\xf4\x05\xad\xe9\x04\xa1\x10\x42\xca\xa1\xaf\xe2\xdb\x75\x79\x46\xf8\x7c\xb4\x3f\x9b\xa0\x8e\x7b\x40\x06\xec\xf5\x23\x9b\x0c\x6d\x97\x06\x91\x4a\x85\x23\x47\x6b\x22\xa3\x04\x3e\x07\x74\xd4\xb7\xbd\x63\xe4\x63\x41\x4a\x29\x84\x17\x5e\x32\xd1\x6f\x4b\x7e\x37\xe5\xf7\x85\xfc\x7e\xe7\x0d\xcd\x95\xb0\xb8\x0d\x30\x00\xa6\xa9\x78\xf9\x5c\xdc\x39\x44\xcf\xf0\x35\x49\x90\x1a\x98\x47\xdb\xfe\xb6\x97\x66\x4a\xf7\xa1\x99\x12\x82\xa8\x20\x26\x1e\x7c\x60\x8c\xe7\xf5\xc8\xf9\x37\x7a\x3d\x72\x77\x03\xb0\x54\x03\x04\x5a\xc9\x62\xd9\x54\xfb\x36\x22\xe5\x27\x18\x74\x0a\x79\x26\xa0\x99\xa2\x90\x0c\x66\x0b\x64\x2a\x60\x3d\x5d\x7a\xd4\xa3\xdc\x40\x21\xe4\x62\x3c\x65\xb7\xf7\x7c\x21\xa2\x1a\xe5\x89\x74\x71\x20\x3e\x89\x5e\xd2\xb2\x71\x58\x3c\x3a\x0e\x55\xe7\x68\xa4\x44\xbc\xff\xf2\x19\x0b\x3d\xbd\xbb\xdb\xcb\xb0\xf7\xd2\x48\xbf\xfe\x94\x87\xd2\x21\x28\xaf\x8f\x10\x4e\x9d\xd5\x03\xb1\xbf\x90\x14\x5e\x66\x7e\xed\x88\x2e\x9e\x3a\xa2\xdc\xa7\xa8\xee\xc3\x43\xea\x91\x2d\x5e\xb9\xd8\x94\xcc\x65\xc1\x0b\x17\x2e\x7f\x6c\xe3\x9a\x44\x68\x5d\xe8\x56\x14\xc6\xd4\x9d\x9c\xdd\xbd\xa8\xdc\x93\xc6\x9d\x9f\x44\x2d\x63\xe2\xd4\x93\x56\xa7\x67\x25\x8d\x26\x88\x82\x86\x1f\x6e\xef\x7c\xd8\xde\x7b\x8f\x36\x59\x0f\xb3\x7b\xb4\xc1\x99\x46\xa8\x27\x19\x99\xed\xc3\xc3\x34\x3a\x43\xd7\x3e\x3f\x1e\x40\x24\xb9\xff\x3b\xd8\x39\xdc\xfb\xf1\x00\x8a\x50\x14\x0c\xdb\xf8\x66\x72\x96\xa3\x96\xa4\xf9\x33\xe4\xfe\x38\x8d\x16\xa8\x2d\x89\x5c\x1d\xcc\x03\x52\xb1\x77\x16\xf4\xc4\x1a\x20\xcf\xf2\x38\xea\x41\xe5\x63\x38\x16\x0f\x46\x50\x3c\x83\x42\x07\x00\x6d\x78\x6b\x8d\xcf\x37\xad\xd2\x57\xf3\x72\x92\x9f\xa3\xdf\xa1\xf6\xea\x6b\xba\xa6\x79\x50\x91\xd8\xb0\xc1\xea\xb3\x3b\xce\xab\xf3\x50\xa7\x62\x16\x96\x74\x5a\x9e\x81\xba\xf0\xd5\xce\xf6\xdb\xdd\xad\x77\x1f\x0e\x77\xf6\x33\x94\xb6\x78\xb5\x75\xb4\xf3\xe1\xf5\xde\xc1\xee\x16\xf6\xd3\x1d\xfc\x97\xa2\x27\x65\xab\xdb\xf8\xfa\x66\x38\xfc\x29\xef\x4d\x22\xf3\x5d\x62\x96\xa6\x6d\xa2\x22\xf5\x06\x64\x58\x92\xde\x4a\xcc\x2e\xfc\x47\xa9\x87\xb3\x89\xcd\xb0\x0b\x44\xfc\x32\xa2\xb4\xea\x24\x02\x2a\x89\x85\x94\x4d\x02\x5a\x95\xd2\xc2\x94\x7e\x3f\x48\x42\x57\x15\x50\x06\xe2\x2b\xa2\x21\xfb\x9b\x37\x41\xfc\x1b\x18\xfe\x29\xe5\xaf\x8c\x87\x02\x97\x97\x95\x05\xcc\x5a\x0b\x0a\x55\xa6\xb5\x38\xed\xea\x2a\xc4\x79\x30\x42\xeb\xda\x54\xd7\x92\x14\xa8\x0d\x98\x7a\x9d\x74\x98\xa3\x41\x26\x2e\xb4\x24\x85\x0a\x4d\x0b\x00\x87\x43\xe0\xf1\x24\xc3\x8b\xc4\xec\xc0\x7f\xc5\x7e\x7f\xd5\xbb\x8b\x28\xa5\x2a\x81\xbd\x64\xa5\xbd\xab\xeb\x2b\x8e\x36\xff\x48\x51\x9d\xfe\x1f\xe3\x91\xd4\xb2\x08\xe6\x93\xb5\x7b\x12\xc7\x9d\xf4\xf8\xe7\xbb\xdd\xfe\x9b\xcb\xab\x69\xef\x1f\x3b\xcf\xba\x8d\x64\x0e\x71\xcf\x28\xfe\x59\x77\xfe\xec\x59\xf2\xfc\x19\x45\xed\x34\xe6\x77\x8d\xf9\x6e\x63\xde\x6f\xcc\xdf\x34\xe6\x97\x8d\xf9\x55\x63\x3e\x6d\xcc\x7b\xf3\x7f\x24\x09\xaa\x67\xaf\x1b\x71\x20\x8c\xb6\x52\xde\x93\x25\x95\xb5\xce\x49\xbf\xb1\xba\xde\x46\x8c\x1f\x99\xfd\x4e\x18\x94\xf3\x31\x37\x88\x07\x48\x1b\x9f\x78\x8f\x48\x85\x2c\x37\xde\x33\xd2\x78\x02\xcb\xf4\x87\x8a\xa5\xc8\xca\x94\x5c\x15\x12\x9b\xad\xd1\xd9\xe5\x78\xe2\x1c\xe4\x39\x58\xca\xdb\xdd\x4e\x64\xc4\xa6\x40\xd9\xb8\x02\x7b\xba\x2b\x59\xb7\x27\x3f\x65\x48\x3d\xe6\x73\xe7\xb4\x52\x3e\xd1\x77\x61\x1c\x61\x5a\x84\x9e\x76\x4c\x68\xf9\x2e\x2e\xd0\xac\xed\xf1\x15\xd3\xac\xb7\x3b\xb5\xf3\xc1\x17\xbc\x68\x2a\x5e\x00\x3a\x23\x0b\xce\x3e\x74\x85\x43\x45\xb1\x49\xa9\x8d\xc6\x0b\x12\xc0\xf0\xb1\x25\xa6\xd0\xd2\x53\xcc\xee\xc5\x12\x33\xba\x70\x06\xd3\xb7\x86\x83\x9e\xf7\x25\x38\xd5\xbe\x1e\xb4\x5d\x7b\xa5\x41\x3d\x19\x5f\x93\xd0\x17\xd6\x47\x46\x1e\xd8\x4a\xaa\xd8\xbb\x86\x1d\xdf\xe6\xb0\x36\x7f\x27\x57\xbd\xe1\xe0\x5f\x79\x3f\xeb\xdb\x6a\xde\xdb\xb8\x98\xfd\xda\x38\x50\xed\xe5\x98\x1d\x7b\x40\xdd\xb2\xd4\xcc\x3d\x1c\xca\x81\xfc\xcf\xee\xd2\xd6\xc6\x46\x79\x68\xfd\x7d\x75\x65\x3f\xcb\x90\x57\x38\xae\xf3\x75\x96\x9c\x57\xf8\x91\xb7\xe8\x1b\x91\xe9\x10\x2f\x6e\xda\xbe\xf0\x71\x34\x9d\x9c\xc1\xe4\x80\xbf\x50\x02\x3e\x68\x9c\x14\x50\xdf\x9b\xff\x19\x9d\xf6\xc7\x3f\x3e\xc1\xf7\x63\xe8\xa8\xcf\x43\x2e\xf5\x0c\xfd\xd8\x15\x13\xf6\x10\xe7\x30\xbc\xa4\x9c\xf1\x39\x98\x30\x3e\x0b\x75\xbc\x0d\x75\xad\x4b\x3c\xe6\xf3\x50\x7c\xe2\x35\x54\xbc\x3d\x83\x4d\xfe\x9e\xbc\x2e\xb2\x81\x6d\x56\x5e\xb5\xee\x1c\x83\x38\xe5\x0b\xd7\xc7\x40\x17\xe9\xe0\xfe\x64\x30\x9d\xc1\x79\x99\x4d\x91\xb4\xa9\x0a\x67\xb9\x5b\x93\x1a\x41\x19\x06\x70\x15\x51\x44\x23\xcd\xe2\x25\x84\xc9\x0f\xba\x3e\x71\xdd\xff\x9a\x0d\x15\xf9\x8e\x1f\x4c\xdf\x5f\x20\x6c\x3b\x00\x8f\x58\x2d\x2a\x01\xcc\xee\x47\xec\x4e\x0a\x22\xd1\xd0\x82\x50\x33\x0b\xb6\x13\xed\x6c\x6d\x47\x42\xdc\x9c\xc3\x92\xb0\x31\xcb\xd7\x06\x9a\x85\x2b\x4e\x00\xac\x68\x47\x4d\x02\x05\x97\xef\xce\x69\x88\x7b\x54\xc4\xca\x29\x69\x8a\x63\x0d\xd5\x66\x25\xa2\x55\x41\x9e\x3a\xd5\x34\x2b\xb5\xc6\x75\x11\xc9\xbf\x8a\xab\xd9\x45\xbb\xc2\x16\xae\xc2\x16\x0f\xa4\xd1\xf4\xe6\xf4\x6a\x00\x5d\x5b\x8d\x53\xa2\x5b\x47\x04\x76\xb5\x8f\x3d\x3a\xbe\x0b\xbc\x4f\x3a\xcb\x35\x41\x87\x55\xb8\xae\xfa\xda\xea\x17\x62\x06\x87\x7d\x2c\xb0\x7d\x23\x37\xb7\x35\x6a\x9c\x04\x07\x27\xdf\xf9\x31\x4f\x01\x60\x0b\x70\xbd\x67\xc5\xfd\x68\x44\xd3\xa1\x4d\x89\xf5\xfa\x54\xdf\x3a\x50\x9c\x1a\x46\x8e\x48\x4c\x58\x7f\xbd\xfe\xa4\xbe\x09\x0b\x35\xc3\xe5\x17\xab\xb9\x62\x96\xa3\xe2\xaf\xb8\x04\x13\x39\x3b\x29\x14\xf5\xaa\x17\x72\xa0\x2c\xef\xfa\xe5\xb1\xe8\x2e\x4c\xb8\x5e\xaa\x96\x63\x8c\xfb\xdf\xeb\xc7\xb3\x21\xd7\xe5\x7d\x9e\xa3\xf5\x9a\xf3\xd9\xf5\x1c\x6d\xd1\xd0\x9f\x69\x92\xa2\x95\xb8\xf8\xe4\x73\x23\xbd\xdf\x30\xad\xc5\xc9\xe7\xe7\xff\x2b\xe9\xa0\x65\xb9\x24\x4e\x8f\x37\xd6\xfe\x08\xec\x16\x04\xd7\xe7\x27\xe8\x1a\xec\xf3\x6f\x56\xd2\x66\xa7\x91\xd5\x7f\xfb\xbf\x56\x4e\xd6\x4e\xd6\xbb\x6c\xf1\x46\xbb\xfb\x86\x2a\xd0\x39\x58\x6f\xed\x5f\x50\xb8\xf9\xe1\xb7\x8d\xb5\x6e\xe3\x7f\xa9\x18\x08\x9e\x34\x25\xdc\xbd\xdf\x34\xbf\x5f\xac\x3a\xb6\xcc\x41\x40\x03\x77\x27\x6b\xf3\x13\xaa\x1b\xb8\x39\xf8\x03\x11\x4d\xf8\x0b\x6c\x06\x9b\xbb\x1b\x58\x6f\xd2\xff\x0f\x75\x7f\xdf\xd0\xc6\x91\x2c\x0a\xe3\x7f\xff\xce\xa7\x80\x39\x5e\x3c\x63\x35\x42\xc2\x8e\xb3\x19\x79\xd0\x21\xc6\x4e\x7c\xae\xb1\x73\x0d\xd9\x9c\x8d\x50\xb8\x02\x09\x50\x0c\x12\xd1\x88\x38\x0e\xd2\xef\xb3\x3f\xf5\xd6\xdd\xd5\x3d\x23\xc0\xbb\x7b\xce\x7d\x9e\xcd\x1a\xcd\x74\x57\x57\xd7\xf4\x4b\x75\x75\x75\x75\x55\xe1\xed\x29\x5d\x84\x69\xd9\xce\xc8\x1d\x0f\x9f\x8c\x4e\x6b\x74\xc0\x7a\x43\x51\xc0\xf3\x30\xc2\xba\xa1\x08\xe0\x79\x18\x07\xdc\xd8\x28\xd6\x79\x25\x9c\xb5\xb9\x18\x0f\x87\x23\x76\x36\x29\xc1\xa3\xc4\x1b\x25\x4d\x1d\x7e\x06\x36\x37\xe2\xc7\x25\x93\xee\xfb\x0d\xf8\xa7\x78\x2d\x22\x3e\xcc\xc1\xd8\x35\x2b\x95\x5c\x17\xa8\xbd\x26\xee\xf1\xab\xc4\x46\xd1\xca\x93\xee\xe4\x7c\x1f\x36\x99\x97\xf7\x47\x42\x96\xa8\xdb\xa7\x34\x57\x52\xd7\xa2\x2a\xb2\x16\x4d\x44\xf2\xde\xdd\x5f\x2c\x1c\x00\x2b\x3f\xbe\x20\xa0\xfc\xbf\xc1\x60\xef\x9b\xbf\xed\xbe\x7d\xb3\x77\xfc\xf2\xed\xee\xc1\x01\x85\x02\x24\xd3\xa7\xc4\xbc\x79\x17\x67\x8c\x27\x92\xf5\xc3\x87\x37\x07\x87\x6f\xde\xbd\x52\x79\xd7\xb2\xc8\xc1\xee\xf3\xcd\x87\xc3\xbf\xab\x9c\x21\xae\x86\x89\x79\xc7\x5f\xef\x17\x0b\x6c\x60\x5e\xdd\xa0\x79\x47\x7f\x9c\x8e\x48\x7b\x2d\x3e\x4d\x83\x35\x50\xad\x8b\x2c\x8d\xab\x4e\xe0\x8f\xad\x94\x37\x54\xda\xd8\xa2\xe2\x28\x45\x47\xb1\x23\xb7\xbb\xb4\x76\xbf\x2c\x4b\x58\x3c\xe9\xd1\xfc\xce\x91\xe7\x01\x82\x8e\x88\xff\x17\xbb\x98\x8c\xd3\x8a\x6a\x52\x17\xaf\xe6\x95\x93\xc1\xc7\xd1\xb1\x0d\x31\x16\x41\xd0\x49\x1c\x7a\x3f\x7b\x14\xba\xb5\x63\xf7\x66\x96\x82\x6e\xd0\xec\xb9\x7a\xce\x1a\x35\xc4\x35\x5d\x68\x60\x87\x40\x17\x0f\x90\xd5\x22\x60\x23\x76\x1f\xc9\xbc\xe0\xcb\x48\xcd\x77\x83\x77\x62\x75\x7e\x85\xdd\xb6\x22\x4f\xc2\x8e\xa3\x86\x4e\xac\x53\x5c\x20\x73\x9f\x86\xb8\x5f\x5e\x0c\x26\xe7\x23\xbb\x3c\xa9\x4c\x3b\x6e\xac\x9f\x9b\xe6\x23\x1a\x2f\x18\x0b\x4a\x4a\x23\xcd\x3e\x57\xc6\xa0\xcf\xa7\xc0\x62\x8f\xdc\x22\xc5\x92\x1c\x8f\x35\xd8\xbf\x16\x4c\x62\x2a\x00\x9c\x8e\x3c\x9a\x1e\x0e\x00\xc0\xc3\x8a\x15\x04\x45\x69\xf0\x00\x59\xe8\xe5\xcc\x4e\x62\x72\x70\x36\xe1\x12\xec\xad\xd2\xda\x93\xb2\xe5\x29\x99\x11\x4e\x36\x19\x00\x3d\xd1\x35\xd7\x6c\x20\x05\xb4\x4a\x4d\x4c\x40\x91\xb1\xe6\xa7\x87\x83\xf3\xd4\x8e\x0f\x7b\x80\x60\x23\xd2\x13\xc3\x92\x56\xb8\x3b\xce\x7c\x35\x6a\x0e\x3a\xcf\xe2\x38\x66\x18\xcc\x47\x02\x57\xc9\x9b\xf5\x50\xcb\x1a\xcb\x4e\x28\x30\x14\x8f\xbc\x37\x73\x1d\x62\x2e\xa1\xae\xf6\xb3\x19\xb7\x85\x7a\x29\x35\xd2\x51\x2f\xa7\x37\x93\x79\xd1\x82\x41\x8f\x23\x4e\x6c\x98\xf8\x19\x76\x83\x0e\xb9\x1b\xc6\x21\x6b\xc9\x4c\x34\x4f\x71\xdd\x64\x1c\x4a\xf2\x0e\x9a\x21\x9e\x75\x32\x2d\x40\xd6\xa2\x5a\x7b\x55\x90\x3e\x7c\xfb\xba\x80\x21\xbf\x95\x79\x94\xae\x2e\xb0\xb1\xa1\xbf\x6e\x73\x33\xf8\xd8\xc5\x22\x5d\x49\xf4\xca\xc1\xcc\x67\xc7\x61\xa9\x76\x16\x83\xb5\x42\x34\xed\xa0\xde\x46\x23\x33\xab\x49\x76\x5f\x18\xb7\xe8\x1d\x9c\x4f\xc9\x6d\x41\x7b\xdf\xd1\xcc\x44\x60\xe6\xbc\xa7\xaa\xdd\x4f\xe5\x3a\x46\x3c\xd1\x35\x27\xa8\x65\x91\x6a\x61\xc9\x56\x0e\x18\x55\xf3\xdf\x1c\x4f\x8b\xa7\x49\xcc\xf3\xf8\xc0\x32\x24\xc3\x5e\x52\xb1\x64\xb6\x2a\x64\xb6\xeb\xc9\x8c\x08\xf2\x94\x6a\xfa\xe3\xa6\xa5\x2d\x63\xaa\x63\xcc\x68\xfe\x1a\x3a\xc6\x76\x2e\xa7\xad\x3e\xa0\xc2\xa8\xed\x64\x76\xdf\xa0\x78\x38\x7f\xab\xe7\x6f\x76\x09\x95\xcd\x73\x58\x7f\x0d\xeb\xf6\xb4\x5c\xda\xcd\x06\xb9\xaf\xb4\x6f\xa9\x73\x11\x89\x53\x2e\x5a\x96\x21\x71\x89\x17\xf9\xf8\xbe\x3b\xee\x45\xe8\x14\xf4\x51\xa0\x1c\x89\xce\x81\xf8\x73\x3d\x9b\x16\x82\xe9\x86\xdd\x29\xed\x0c\xaa\x1f\xee\xb7\xb7\xb2\x18\x31\xa0\x4f\x30\xe3\xe1\x1f\x85\x7f\xb5\x87\x22\x78\x7a\x12\xe3\x64\x8c\x1d\x28\xb0\xb9\xd9\x11\x0f\xe0\xbe\x64\x0f\xd2\xfb\xd2\x13\x1d\x2e\xea\xc6\x95\xea\x86\x28\x47\x7a\xe1\x54\x36\x36\xc8\xde\xc9\xa4\x2b\x03\x71\x4c\xbe\x54\x6d\x20\x2a\x9b\x69\x2b\x53\xf6\xdc\x72\x64\x92\x5f\xba\xb4\x6b\xeb\xeb\xdd\x79\x45\xe6\x7a\x98\xe8\x29\x87\xf6\xd4\x04\xb4\x61\xa4\x44\xdc\x76\x9e\xd9\x2d\x24\xa7\xb4\xfb\x21\xd3\xef\x9c\xb9\xdd\x9a\xd7\x9e\xa4\x0e\x8f\xd7\x5d\xae\xdc\xf4\x9d\xad\xd8\xee\x79\x1c\xb4\x45\x83\x46\xe2\x61\x59\xab\x87\x15\x81\xfb\x1f\x91\xb7\x6b\x87\xbc\xc4\xb0\xf1\x54\xca\x70\x1d\x41\x9d\xa9\x2c\xe3\x5c\x84\xa8\xcb\x5c\xe4\xdc\x87\xf4\xe1\xbd\xfb\x82\x2b\x4d\xa3\x8c\x79\xd1\x97\xd9\x7a\xf0\x48\x54\xa6\x0a\xf2\xe5\xe9\x6c\x95\x60\x10\x94\xda\xd8\xe0\x2f\x16\x61\xc2\x05\xa0\xe4\x54\xcd\xef\x13\x5b\x24\x41\xa5\x82\x8b\x46\x79\x0f\x64\x4b\x0e\x3f\x61\xba\xc7\xd3\x8f\x5b\xd4\x91\x9b\xc9\x4c\x10\x76\xe7\x7c\xf3\x28\x80\x50\x3f\xa8\xaa\x09\xb8\x85\x80\xc7\xd3\x4d\xf6\xf5\x30\x6a\xb0\x4f\x1f\xd4\x2b\x5f\x3a\x78\xbc\xdf\x54\xdc\xb7\x37\x9f\x64\x47\x5b\x5b\xec\x37\x55\x06\x08\xd6\x8c\x2e\x3f\xd1\x9d\x24\xf6\x10\x01\x6f\x6c\xa0\xcb\xdd\x0f\xa3\x73\x90\x1f\xbd\xb7\x54\xa7\x68\xc1\x32\x20\xb9\x99\x84\x8d\x18\x54\xaf\xba\x2f\x13\x47\x9d\x4a\xde\x73\x59\x12\x62\x16\x70\xa8\x68\x42\x2e\x9b\xb4\x30\xc4\xea\x5d\x92\xc4\x34\x73\x34\x56\x63\x2a\x0b\x3b\x43\x9c\x62\x0c\x33\x1b\x5f\xd9\x08\x06\x30\xf0\x31\x63\x69\xbb\xdb\xf6\x26\x01\xf2\xfe\xcb\xd4\x0f\x84\x95\x62\x2c\x3b\x94\x96\x81\xc9\x27\xe9\x3e\xe8\xb6\x32\x44\xb2\x88\xef\x96\x8b\xd7\x45\xe0\xb5\x77\x27\x89\xe7\xd3\xc0\x78\xf9\xfe\xdd\xc1\xe1\xee\xbb\xc3\x63\xd8\x2e\xfd\xf8\x4a\x69\x65\xd0\x01\xe6\x82\x3c\x5d\x2e\x8e\x86\x8d\xec\xd1\x16\x8c\x21\x6a\xab\xbb\x06\xd1\xdd\xc7\x05\xf3\xeb\x4b\x03\xff\x76\xf5\x41\x50\x6d\xfd\xb0\x9d\x2f\xe7\xa9\x80\x36\xa5\xde\xac\xbb\x82\x41\xe8\x33\x83\x84\x3e\x30\x31\x35\x7c\x4a\x86\xc6\x72\x15\x9f\xa9\x39\xa8\xb0\xa5\xee\x38\xa4\xb0\x15\xfe\x1e\xce\xb6\x6f\xc7\x13\xc5\x03\x27\xe7\xee\x39\xb6\x2c\x0c\xcf\x02\x2a\x9b\x02\x54\x23\x60\x2c\x0a\xd8\x20\x25\x59\x73\xc8\xfb\x0f\x9b\x60\x84\x46\xac\x2c\x33\x75\xd4\x63\x4e\x85\x78\x5b\x09\xea\x4c\xe4\x4a\x89\x0a\xa1\x91\x24\xb9\xfb\x96\x4c\xbe\xe4\x70\x74\x75\x8d\x8e\xa7\x03\x15\x91\x76\x48\xad\xd4\x12\x2a\xf9\x61\xc7\x43\x1c\xd0\xc8\x15\x7a\x3d\x29\x34\x8e\x20\x78\xb3\x1c\x7d\xa8\x8f\xb3\x94\x65\x3e\x4c\xe9\x03\xdb\x2e\xa8\xb2\xc2\x65\x43\xec\xc9\x03\xda\x90\xfa\xbe\x2f\x0d\x86\x9e\xad\x83\xc6\x42\x33\x8e\x7a\x25\xce\xc8\x69\x68\x1e\xd2\x58\x4e\x8b\x73\x3e\x9a\xf3\xfd\x7e\x1a\xa0\x6e\x06\x32\xb7\x19\x72\x59\xda\xfb\x66\xcd\xf9\x54\x3c\xf3\x65\xcb\x7f\x7c\x7c\xe1\x17\x29\x23\xbe\xa1\x55\x2f\x54\x40\x82\x61\x18\x52\xa9\x57\x2c\x4b\x09\x1e\x2c\x53\x33\x60\x30\xa0\xc3\xd9\x0d\x08\x1e\x84\x28\xfc\x10\xfe\x12\xdb\xc2\x44\xba\x6f\xde\xd3\xe0\x35\x4d\x78\x01\x16\xb0\xf7\xc3\xe1\x6a\x48\xc8\x4c\x8c\x87\xc5\x93\x87\xd5\xc0\x98\x4b\xa6\x08\x08\x3d\x1d\x7c\xac\x9f\xde\xb7\xf7\x9c\x7f\x2b\xce\x21\x78\x12\xaf\xab\xcf\x4c\xdd\xb6\x0a\xfb\xe8\x94\x20\x33\x3e\x65\xf6\x52\xad\x1a\x63\x55\x66\xcc\x31\xfa\xd9\x0a\xc9\xca\xc5\xc9\x7f\x24\xd2\x86\xe5\x75\x38\x40\xab\x5a\xde\x58\xb9\xeb\x99\xfb\xe8\x69\xdd\x79\x97\x2d\xd0\x3c\x2d\xaf\x41\x2e\x5b\x72\x45\x74\x9c\x53\x7f\x12\xce\x47\xef\x6b\xc3\x93\x4b\x7e\xa0\xf0\x91\x18\x1a\x93\x9f\x6e\xae\xd7\x5c\x40\xc9\x35\x1b\x70\x92\x1f\xb0\x71\xd6\x7c\xe4\xc9\x35\x1f\x9a\x12\x8d\xba\x08\x05\xfc\x02\x02\xf8\x4b\xea\xa9\x35\x56\x7d\xc3\xc6\xe8\xf4\xa6\x5c\x3b\xb9\xbc\x41\xa7\x63\xd7\x9f\xd7\x4e\x01\xe3\xf5\x00\x06\x5d\x52\x1b\xd6\x74\xe2\x8e\x8e\xfd\x79\x31\x6a\xdd\x56\x9e\x1e\x4f\xe4\xe4\x38\xfa\xea\x5e\x50\xbc\xef\xad\x2b\x74\x3c\x9a\x87\x73\x01\x36\x4d\xd5\x53\x30\xaa\xc0\x73\xc3\xa9\x32\x07\xe1\xcf\xa9\x58\x3a\xc8\x8c\x65\x53\x54\xbd\x23\xb1\x95\xc3\x6e\x16\x21\xf9\xc4\x70\x49\x0b\x03\xc6\x95\x90\xb3\xe3\xf3\x37\x67\xc1\x48\xb2\x81\x78\xd4\xa7\x49\x92\x1b\x49\xf3\xd9\x60\x52\x9e\x5e\xde\x0c\x41\xdc\x74\x2a\x6e\x37\xbc\x9e\x83\xec\x20\xde\x44\xc8\x88\xce\x1f\x2a\xec\xde\x63\x5e\x62\x3c\xe2\x6a\x43\x3a\x85\xb9\x55\x8c\xfb\xb6\xa4\x08\x85\xf6\x40\x96\x5e\x0e\x10\x36\xda\x9d\xdb\x75\xe7\xcd\x59\x65\x39\xd0\x08\x60\xcf\x6b\x3f\xb8\x49\x43\x32\xd5\xb9\x12\x79\x52\xde\x54\xc0\x6a\x55\x2f\xf9\xd6\xb2\x2f\x4d\xbb\x5b\x4c\x35\x88\x2e\x38\x9f\x7e\x3b\x9d\x42\x4d\x96\x9a\xa0\x78\x61\xbf\x01\xc4\x6d\xbc\xb5\xed\x1a\x48\xc1\xa8\x30\x8e\x97\x1c\xf7\x44\x93\x48\x69\x2e\xec\x92\xc4\x8f\x91\xc4\x51\x74\x96\xea\x52\xb2\xa5\xb5\x30\x40\x1e\xf0\x66\x42\x55\x06\xc3\x04\x0f\xfb\x70\x49\x9c\xcb\x1a\x4b\x56\x90\x74\xf8\x41\x66\x46\x07\xa7\xc8\xb1\xf0\xdd\x06\x29\x31\x6a\x68\xc9\x82\xea\xfb\x16\xb1\x99\x10\x97\x09\x30\xb9\x18\x28\xee\x4b\xc8\xdc\xb2\xee\xe4\xea\xe5\xae\x1a\x8c\xcf\xa2\xc1\x58\x3b\x74\x1f\x36\x28\x4b\x77\x9a\x5f\xce\x4e\x61\xdb\x53\xd8\xf1\xc4\xad\x23\x7b\x1f\xc8\x33\xd3\x09\xb0\xfb\xa1\x03\xe1\x57\x8a\x02\x31\xb8\x81\xd5\x9c\x3e\xc8\xe5\x62\x52\x49\x49\x2b\x6c\xea\x1f\x05\x11\x0a\xd9\xf4\x72\xce\x1d\x2f\x2f\x7e\xe8\xe3\x56\x9e\x74\xaa\xa3\x19\xc6\x92\xc7\x51\x75\x73\xfd\x16\x18\xa4\x10\x29\xf1\xff\xb5\xe8\xaf\x11\xe2\xd0\x53\xaf\xc1\xd8\x55\xe9\xec\xaf\x23\xaa\xbd\x66\xde\x04\xf9\x31\x3c\x23\x59\x86\x02\x07\x49\x12\xc4\x16\x77\xcb\x0f\xde\xf0\x35\xe5\x16\xd7\x06\x61\xb3\x53\xb9\x2f\x75\x31\x2e\x59\x85\xf1\x66\x58\x34\x1a\x41\x13\x74\x00\xaa\xcb\x83\x0b\xc5\x13\x2c\x64\x6e\x4f\x71\x70\xe5\xe1\x58\x5b\x66\xcd\xf2\x86\xdd\x80\xb8\x1a\x60\x44\x5d\x4f\x27\xe5\xc8\x79\x96\x70\xb5\x90\x77\x53\x5f\x8b\xd8\x26\x8d\x3e\x71\xdb\xe8\xf9\xda\xd1\x63\x27\xb5\x20\xd5\xf9\xba\xaa\x9f\x2a\xed\x6e\x31\x44\x4d\xc9\xd3\x39\x4c\x64\xd1\xcc\x7d\x45\x85\x03\x84\x43\x07\x3b\xc3\x8f\x34\x37\xdb\x22\xb0\xa6\x0d\xb6\x9b\x66\x59\x30\x52\x32\xb3\xae\x22\x0c\xe8\x41\x8e\x3a\x01\xfd\xbe\xb1\xb1\x1e\x6c\xee\x22\xd8\x60\xd2\x47\x9f\x0f\x45\xae\x70\x89\x87\xfd\x85\x6e\xa4\xb7\x30\xb9\x46\xc3\x24\x0b\x36\x8d\x6e\x0a\xb2\x62\xab\x49\xc7\x0b\x69\xa4\xc9\x5f\xd5\xa3\x1b\x1b\x77\xf4\xc8\xd2\x57\x54\x47\x8d\xc4\x03\x42\x82\xb2\xfc\x4e\x34\x9e\xbb\x8e\xe7\x0f\x15\x46\xef\xb2\x4a\xd2\x4c\xab\xac\xaa\xfa\xca\x26\x57\xc5\x76\x63\x06\x0d\xc7\x27\x28\xf6\xe3\x11\xdf\x8a\xea\x35\xe3\xd4\xd2\x23\x15\xff\x01\x84\x30\x12\x9d\x82\x65\x41\x0c\x5c\xcd\xca\xfd\x25\x03\x98\x70\x9f\x89\x13\xe8\xdb\x0f\xbb\x2f\x5f\x15\x5b\xb7\xcb\x2d\x1b\x93\x54\x33\xf5\xdd\xfb\xf5\x59\xce\x8f\xf0\xc9\x68\xe6\xb8\xeb\x29\xf6\xa8\xf9\x74\x31\x9a\xb8\x24\x96\x03\x3e\xd1\xfd\xb3\x55\xbb\x53\xcc\xcd\x30\xea\x37\x48\xfa\xc2\xc4\xe9\x79\xb1\x68\x11\xb2\xb2\xd0\xcd\x2b\xe8\x29\xec\x28\x67\xc3\x1b\xde\xc3\xb9\x5d\xf2\xf9\xe8\xc1\xe7\xab\x93\xe9\x65\xb0\x31\x6e\xaa\x0c\x18\xe7\xa3\xc9\xb0\x0e\xc8\x25\xa7\x19\x79\x27\x1e\x4d\x8a\xad\x5f\xb0\x86\x14\x2d\xb2\xcb\xac\x9b\x36\x51\xb7\xe3\x44\x72\x5a\xb0\xfc\x32\xe6\xce\x76\xa9\x81\xc8\x14\x52\x6c\x53\x09\x17\xeb\x69\xc2\x2c\x60\xe3\xf4\x05\x91\xcd\x86\x03\xb0\x31\x99\xd3\x04\xc1\xc8\xb0\xd7\x27\x11\x51\x7c\x73\x37\xeb\x17\x2b\x1a\xb7\x17\xe0\x23\x53\x46\x77\x78\x43\x55\xd7\x7e\x00\xc5\xcc\x54\x6d\xcb\x17\xe8\x42\x55\x83\x83\x76\x14\xd1\xa0\xd2\x7d\xd0\x70\xe3\x83\x5c\x12\x73\xaf\x36\x5c\x33\x67\x6a\x7a\xdf\x71\xa6\x43\x2b\xd4\x6b\xe0\x30\x72\xc3\x53\xc6\x81\xc3\xed\x2f\xc1\x8c\xcb\x77\x83\x77\x56\x0f\x98\x24\x39\x3f\xa2\x53\x62\xfa\x96\xc5\x42\x34\x38\x32\x2f\x9a\xd7\x34\xab\x5e\x0e\x44\x2b\xb1\xc9\x04\x66\x99\x1e\x56\x3d\xca\xea\x47\x53\x00\x36\xcd\xea\x1e\x21\x2e\x36\x7f\xc3\xcb\x90\x81\xa6\x43\x12\x89\xf7\x20\xeb\xf9\x30\xba\x1e\x0d\x42\xcb\x21\xd9\xd7\xd4\xee\x03\x28\xcf\xf8\xed\x80\xd6\x66\x7c\x0b\x1f\xf0\xd1\x46\x03\x4e\x4f\xf0\x8d\x56\x4e\x7a\xe2\xc1\x4e\x37\x55\x8a\x82\x53\xa0\xc9\x29\x26\xb8\xb4\x93\x84\x94\x8d\xa0\x79\x8b\x22\x9f\x50\x44\x99\xf6\xcb\xcb\xa2\x27\x4f\xfd\xce\x70\xaa\x02\x2c\xd7\xc5\x47\x36\xeb\x76\x8d\x63\x4f\xac\x16\x47\x14\x60\x9e\xaf\x36\xc9\xeb\x7a\x4c\x73\x27\x24\xda\xe2\x60\x93\xc5\x77\xdf\x1d\x7f\x78\xb5\xff\xfe\x6f\xaf\xf6\x8a\xe4\xd1\x23\xff\x96\xb8\xf6\x8e\xee\x08\xd9\x64\xe7\x5b\xec\x9e\xfd\x15\x6e\xdf\x35\x67\xbe\x5b\x7a\x45\xae\xe9\x95\x00\x0f\xda\x4e\x41\xf5\xa7\x1f\xbf\xfd\x0c\x83\xcd\xf8\x47\xb9\x63\x21\x09\x6f\x86\x34\x14\xfd\x2b\xe9\xbb\x75\xc2\xfb\x93\x5f\xe1\xf5\xf2\xa2\x34\x33\xf8\x47\x23\x96\xfc\x07\x8f\xcf\xc6\x80\x06\x26\xaf\x7a\xf3\x13\xd7\xda\xc3\x48\x93\x18\x3e\x98\x50\x13\x5b\x7c\x87\x92\x41\x1f\xb0\xbe\xa3\xb2\x31\x9e\xc0\x9f\xb4\xf9\xa4\x9b\x91\x91\x5f\xd9\x20\x12\xe0\xf7\xe4\x33\x65\x20\xd0\x13\xb4\x2a\xcc\x28\xce\xd5\xeb\xc9\x5b\xba\xf7\x55\xdc\x3e\x1a\x0f\x73\x89\xb8\x4b\x2e\x76\xd7\x09\xb7\x58\xcd\x84\x7d\x95\x26\x63\xa0\xc1\xc5\x76\x1d\x0d\xb5\x1b\xb6\x31\xdb\x5b\xa2\x47\xb3\xc7\xc7\x30\x1e\xae\x8e\x31\xe9\xf8\x14\x35\x36\xd4\xd6\xc7\x3d\x6e\xd2\xb5\x93\xcf\x6b\xc7\xe3\xe1\x71\xff\x31\xb9\x4e\xd1\xc1\x5f\x3d\x3a\x3a\x1d\x86\x66\x2b\xec\xa1\x0a\xb6\x5f\x61\x63\xd0\xa9\x0e\x91\xb4\x67\x3a\xad\x9b\xc6\x1d\x66\x95\x0c\x3e\x3d\x8b\xfa\xd0\xef\x05\xdc\x15\x40\x09\xf6\x6f\xc7\x4c\xd0\x59\xb0\x3e\xe8\x66\xec\x05\x99\x74\xd1\x2e\x6c\xe7\x5e\xd4\xf5\x72\xe9\x39\x80\x41\x43\x11\x74\x84\x29\x51\x0c\xa2\x4f\xb0\x43\x55\x97\x40\x7f\x2e\x69\x3c\xf8\x6a\xbe\xc4\x7d\x83\x0d\xad\x2c\xba\xde\x68\x9c\x06\x25\xf5\x77\xc3\x6a\xc0\x43\x10\x7a\xc4\x8d\xbd\xb4\x9b\xa7\xbd\xa3\x47\x47\x9f\xf0\xe6\xd0\x51\xea\x9e\x61\x9c\x19\x8a\xb9\x6c\xdf\xb3\x0c\x47\xdd\xdd\xe3\x6a\x3c\xe4\x91\x25\x43\xe7\x31\x8e\x9d\x15\xc3\xe8\xf1\x5a\x79\x31\xbd\xb9\x1c\x92\x0f\x1d\x18\x77\xae\x4d\x31\xee\xdc\xe3\xf4\x18\xe8\x3d\x36\x6b\xc7\xf4\x89\xc7\xd9\x63\x35\x46\x4d\x65\xc0\xc1\xf7\x20\x73\x0d\xba\xa6\xf0\x81\x0c\xdd\xe0\x0b\xba\xd7\x0d\xc3\x8e\xf5\x51\x4e\x5c\x7f\x7f\x70\x4d\x26\x51\x7a\xdd\x7c\xe9\xc8\x4e\x71\xfe\xfb\x1d\x8f\x4b\xb7\x27\x07\xd8\xe5\x7c\xa0\x45\x21\xf3\x39\xfc\x2f\x76\xe8\x5b\x4e\x54\x7a\x0e\x3f\x40\x5d\xf7\xf9\xa7\xd7\x78\x57\xde\x22\x87\xae\x2e\x0d\x31\x6d\xb7\x4a\x1c\x4e\x39\x28\x2b\x19\x9e\x8f\xa7\x37\x25\x2d\x47\x96\xff\xa1\x8d\x00\x56\xaf\x3e\xc8\xbf\xbf\x9f\x51\x18\x6a\xba\x6f\x2f\x47\x7b\x6f\xc7\x1f\x47\xfa\x63\xb2\xb0\xee\xc2\xbf\x6a\x0a\x8b\x70\xea\x2d\x16\xf1\x10\x66\xff\xa0\x0f\x2a\xc1\x0c\x36\xaa\x56\x42\x07\x41\x43\xe1\xe0\x51\xf4\xf9\xc7\xd8\x81\x38\x8e\xf8\x8d\x8d\xe4\x51\xb2\x8e\x93\xd7\x07\x7d\x84\x4d\x50\x80\x9b\x57\x48\x84\xee\x44\x19\xe5\x74\x86\x1b\x1a\xac\x58\x75\x5c\x11\x41\x49\x1f\xf3\x4f\x11\xb6\xad\x64\xae\x28\x22\x51\x94\x3a\xe2\x2f\x8e\x1d\x64\x8b\x2b\x63\xf4\x29\x87\x37\xa3\x7d\x49\xdc\xcc\x05\x68\xba\xec\x14\x37\x4c\xec\x51\xa2\x04\x53\x57\x05\x38\xc4\xba\x6b\xe4\x42\x75\x45\x85\x3f\xa2\x2f\xae\x11\xca\x22\xf3\xef\xc3\x26\xf5\xa3\x33\xf9\x3f\x96\xf3\xff\x1f\x98\xaf\xb0\x47\xd5\x93\x26\xee\x09\x57\x2c\xcb\x68\xe8\x16\x1a\xb8\xe7\x72\xfb\x46\x62\x41\xae\xca\xd6\xe3\x58\xa5\xb3\x10\x13\x8d\x6a\x69\x07\xce\x73\xfe\xab\x35\x86\x3b\xa8\x64\x96\x66\x65\xf7\x10\xb1\x9f\xf2\x22\x0b\xd2\xcf\xc6\x46\x24\xc4\x49\xb0\x00\x47\x2e\x67\x8f\x2d\xb5\x7c\x86\x19\xb2\xcc\xe1\xcd\x35\xde\x44\x4d\xf6\x6e\xd8\xa3\xcf\xa8\xc4\xb1\x3e\x40\x27\xde\x78\x8f\x10\x7d\x02\x8c\xc8\xd5\xd8\xe0\x12\xf7\x30\xc3\xe6\xda\x8f\xe5\x68\xed\xb1\xed\x08\xcd\x17\xd7\xe6\xd3\xb5\x12\x56\xf4\xf1\xd9\x67\x74\x33\x0e\x3b\x77\x64\xfa\x65\x73\xed\x83\xe0\x22\xc7\xa4\x66\xcd\x55\x85\xd9\x62\x06\xab\xb8\xab\x6f\x92\x4e\x7d\xeb\xde\x82\xa8\xe1\x80\x96\x2b\x3b\x68\xbd\xbd\x54\xb3\x57\xb7\x4b\x76\xd7\xb8\xe1\x19\x9c\xd6\x8c\x19\x1a\xcc\x31\x03\x2c\x56\x48\xeb\x26\x52\xdf\xc5\xe5\xb2\x38\xfa\xb5\x67\xa9\xb1\xe8\xe9\xb6\x1d\x3d\x2f\xff\xf6\xf1\x4c\x29\x33\xd2\xff\x91\x7a\x31\x23\xb6\x65\xc3\xa5\xdd\xc5\x0d\xea\xb9\xc0\xed\x7f\x0b\x1b\xe0\x16\xad\xed\xd2\xda\x69\xb4\x89\x71\x85\xd3\x60\x7d\x59\x01\xe6\xb6\x12\x26\xde\xf1\xdc\x2a\xdd\xbf\x6a\x2b\xb7\x36\x16\x1a\x3d\xec\x77\xd6\x5c\x86\x7d\xd0\xdb\x1d\x71\xf0\x62\xb3\xd0\x6a\x87\x9f\x74\xc7\x64\x9d\xca\x26\xcd\x82\x91\x42\x8e\x07\x05\x05\xb2\x5f\x35\x74\x48\x7d\x28\xdb\x21\x4d\x60\x96\x85\xcb\x6d\xb0\x8f\x62\x57\xfa\xab\xce\x3a\x3a\x3e\x63\x95\x0c\x19\x0b\xa6\xaa\x44\x9d\x58\xaa\x0f\x65\xb4\xc0\xa9\xd3\x29\x04\x70\x81\xde\x8d\xab\x79\x38\xb5\x38\x19\xb2\xd5\x4a\xb7\xd9\x0e\xc0\xc4\x0f\xe0\x7a\x5a\xc1\x8b\x41\x9d\x42\x74\x21\x51\xd3\xe1\x30\x2a\x86\x47\x76\x5c\xe5\x5f\xb6\x8b\xa2\x55\x19\x2e\x8b\x05\xef\xef\xee\x3c\x0d\xc2\x9f\x1e\xfd\x95\x39\xd4\x68\xf4\x63\x8f\x11\xee\xf6\x35\x39\x67\xb6\x2c\x37\xc7\xf0\x4b\x8e\xcf\x35\xe8\x70\xb5\xf6\x24\xe9\xa1\xfd\xcf\xd0\x6a\x5c\x17\x8a\xf0\x78\x14\xea\x92\x1b\x1b\xfa\xcd\x8e\xa0\x6e\x5d\x62\xce\x1f\xdc\x92\xf9\x6b\x93\x8b\x6a\x3b\xc0\x44\x0c\x79\x71\xb4\xfa\x2c\xb3\x65\x20\xe4\x6a\x58\xaf\xbf\x3d\xb8\x98\x7e\xfa\xc2\x13\xd4\x2f\xbc\x70\xdd\xe4\x4a\x2a\xc7\x96\x16\x6d\x2f\x3e\x42\xec\x26\xca\x00\x21\xc9\x13\x6b\x2f\x92\xf4\x9d\x72\x00\xcf\xbb\x2f\xd0\x9b\x87\x33\xcb\xf8\x7e\x1c\x1d\xf1\xfd\x77\x7c\x07\x56\xf2\x25\xdf\xe1\x28\xcf\x83\x4f\xba\xeb\x3b\x0e\xe6\x9f\x57\x69\xb4\xbf\xbc\xd9\x11\x57\xa0\x54\xa3\x94\xd2\x4c\x61\xd0\xd2\x53\x76\xeb\x1e\xc9\x26\x92\x1f\xd7\x8b\x42\x25\xdb\x75\xd3\x25\x05\x2d\x60\x4a\x4c\xf3\x9a\xba\xd3\xb2\x4c\x29\xc9\x90\xcd\x8c\x71\x48\xbd\xba\x1a\x41\x5c\x32\xec\x66\xd1\x77\x3f\x0a\x4a\x07\x14\x74\xfd\x4b\x8f\xf3\x43\x35\xbb\x32\x24\x65\x74\xc1\xb5\x69\x7f\xf5\x2d\xbe\x15\x81\xaa\x62\xd4\x78\x43\x1f\xdc\x6b\x79\x6a\x51\xbf\x54\xf7\xa6\xe9\x60\x75\x74\x49\xea\x95\x43\xa7\x00\x23\xa7\x7d\x94\x66\x97\x1c\x43\x1d\x84\xd7\x96\xec\x01\x2c\xe3\x92\xf3\x57\x10\xc7\x6c\x09\xe2\x2a\xb4\x27\x0a\xba\xd6\x95\xaf\x8c\x43\xeb\x82\x68\x0c\x32\xc8\x78\x5c\x84\x78\x9c\xfb\xa1\xf1\xce\x98\x7c\x10\x85\xd9\xbd\x71\x5f\x1f\x97\x46\xb2\x54\xfc\x11\x00\x9d\x75\xe2\x44\xbc\x5b\x56\xa1\xdd\xa4\x35\x8d\x52\x54\x1b\x90\xdb\xbf\x97\xac\x27\x0d\x56\x0f\x2f\x16\xab\x81\xba\x09\x05\x43\xad\x58\x59\xf2\x29\x98\x17\xf4\xea\xfa\x43\x7b\x6f\x8a\x32\xc3\x4e\xac\x39\x0e\x8d\x5a\x94\x76\x94\x41\x5a\x66\xaa\x58\x9b\xca\xe8\x21\x00\x56\x2b\x1d\x7c\xd6\x2b\x7d\x4a\xce\xa7\x88\x45\x0d\x32\x19\x87\x95\xc6\x67\x62\x34\xa2\xca\x4a\xe7\xf3\x0c\xe3\xf7\x76\x13\xfc\x4e\xa6\x35\x5e\xbb\xce\xad\x8f\x07\x2d\xab\xce\xd7\xee\xd4\xf3\xfe\xb5\xd5\xf2\x93\xf1\x17\x3d\x1b\xef\xd0\xf7\x96\x77\xda\xd0\xdc\x7d\x83\xc0\x8f\x20\x7b\x5e\xe8\xbf\xa0\x5f\xdc\x0f\xb2\x58\xa0\xa7\xbf\x7b\xc1\xc4\xe9\x98\xfa\x78\xff\x68\x69\xcb\xe5\xd7\x79\x66\x11\xce\xc6\xce\x02\xfe\x3f\xd5\x9c\x30\xd7\x8a\xe8\xbd\xd2\x50\x90\xf6\x0f\xb5\x8a\x1f\xd7\x2b\x0f\x90\x35\xe7\x56\xd7\x91\x3d\x72\xbd\x32\xb8\xc3\x00\xfd\xd5\xa8\x1a\xd7\x09\xf1\xb5\x52\x4f\x04\xde\x2d\x9d\xce\xae\x81\x89\x40\x15\x6f\xa0\xd6\xf3\xc1\xe5\xda\x0d\xc8\xf8\xd3\xb3\x35\x0d\xe7\xad\xf1\x70\x9f\x8b\x51\x4d\xac\x35\xc6\xfa\xda\xbb\xa9\xdc\xe1\x54\x40\xf3\x8b\xc1\x7c\x4d\xfa\xae\x84\x9d\xbe\xb6\xaa\x58\x3b\x9b\xde\x4c\x86\xfa\x8e\x2a\x46\x12\xb9\xf3\x56\xaa\xff\x96\x42\x3d\x57\xd6\xad\xda\x13\x92\x32\xf0\xf5\x71\xaa\xbc\xa1\x28\x2e\x15\x4b\xe0\x8f\x02\x6b\x59\x74\x85\xe4\xaf\x8f\xb2\x2f\x24\x06\xe4\x3b\x38\xe5\xe9\x6c\x7c\x1d\x1e\xca\x45\x46\x56\xca\x29\x8a\xce\xa8\xbf\xc6\xff\xe0\xa3\x22\xf6\x58\x84\xe7\x84\x5b\x20\x53\x59\xcc\x49\x51\xf8\x3b\xfb\x7c\x3c\x24\x39\xe8\x51\x8e\xb2\xf0\xf2\x24\x3a\xf7\xf4\xaa\x54\x76\x4a\x18\x52\x07\x03\x7c\x9e\xaa\xb2\x46\xfc\x5a\x32\xaf\x7c\xcf\x1e\x60\x2b\x47\x63\x92\x9e\x64\x1e\xa6\xe6\xe6\x94\xfa\xc6\xa5\x5d\x42\x82\x06\x54\x46\x68\x15\xc3\x4d\x67\x58\x26\x16\x9c\x72\x88\xf7\xfe\x87\xc3\x37\xef\xdf\x1d\x84\x3e\x24\xf0\x98\x29\xed\xe6\x47\x65\x63\x50\xda\x63\xa7\xac\xcb\x29\xe7\xb3\xe9\xcd\xb5\x3f\x77\x7a\x02\x19\xf0\x00\x4b\x29\xbe\xba\x23\x03\xfe\xfb\x84\xfc\x8b\x1d\xa5\xfe\xb8\xc0\xa6\x87\x87\x08\x2a\xf5\x88\x3c\x55\xa8\xe3\x2e\xae\x35\x3a\xed\x22\x7a\xf0\xb2\x07\x6c\xc8\xf6\xdd\x35\xb8\xdb\xe0\x56\xa9\xb8\xf3\xb9\xcb\xe7\x43\x2f\xe1\x46\x84\x06\x73\xb7\xbc\xfa\x66\x25\x37\x71\xde\x10\xc4\xf3\x41\x0d\x47\x91\xc9\x24\xc6\x29\xd6\x6b\x11\x77\xa9\xb9\x99\x7c\x9c\x4c\x3f\x4d\xe4\x0d\xbd\x33\xb2\x6b\x63\xf1\x0b\x6c\xf5\xf1\xe7\xfe\x8b\x82\xef\xc3\xc5\xfc\x8c\xac\xdf\x4f\x90\x19\xf0\x21\x62\xe9\xaf\xa8\x63\xee\x78\x32\x56\x66\x6f\x0a\xd5\xb1\xf1\x84\x1c\x87\x94\x1c\x67\xb7\x41\x9d\xf5\x85\x8a\x95\xe5\x8b\x10\xdb\x92\x29\x81\x2d\x8d\x64\x57\xae\xa5\xac\xd0\x27\xb3\xb6\xe3\x71\xc2\xcd\xc1\x36\x08\xc9\xe3\x4c\x35\x8f\x98\x04\xe0\x65\x5f\x45\xa5\xbe\xb7\xe9\x2e\x74\x3a\xd6\x83\x52\x9f\xdc\x9d\x0d\xe8\x74\x52\xcd\xc6\x46\x98\xce\x1b\x30\x0a\xe5\x43\x1f\xc2\xef\x2b\xbe\x85\x58\x2d\x2a\x2a\xaf\x55\x2a\x54\x2f\xba\xeb\x0a\xe9\xf7\xd1\x2d\xae\xe7\xf1\xc2\xe9\x8f\x9a\x2a\x77\x87\xcb\xd2\x54\x81\x08\x48\xe3\xa1\x27\xdf\x05\x35\x14\x49\x77\x2d\x69\xa8\xd3\xc1\xac\x91\xac\x75\x93\x4e\xf8\xe5\xd8\x54\xbe\x8c\x62\xdf\xd7\xa8\x74\x06\xfe\x1d\x80\xab\xfc\xb8\x60\xd4\xd0\xe8\xe1\x2b\xb1\x92\x28\xdd\x7b\x90\xcf\x70\x0d\xb7\xea\x3e\xa4\x6f\xbf\x58\x1b\x6c\x4f\x38\xad\x32\x6d\xe5\x8d\xd5\x95\xed\x45\xec\x21\x7b\xc8\x3e\x4e\xee\xdc\x3a\x3b\x90\x03\x58\x72\x31\x0a\x12\x41\xf2\x77\x39\x03\x43\xdf\xbd\x92\x43\x97\x62\x6f\x83\x6e\x17\x7f\x11\xb1\xcd\x8d\x1b\x0c\xf5\x63\xa4\xe3\xf1\xe9\x11\xe7\xae\x13\x76\xd3\x2f\x1b\xdf\x21\xe5\x3c\x4f\x1c\x32\xc3\xee\x28\xfc\x65\xc4\x11\xde\xdb\x5b\xdd\xa1\x59\x5e\x7f\xc7\x31\x28\xd8\xad\xd6\x08\x52\x42\xae\xbe\xab\x76\xe4\xfb\x5b\xa2\x11\xc9\xd8\xe7\xbc\x97\x0b\x7b\x7c\xc5\x4d\x81\x2f\x6c\x9e\xa0\x17\xf4\xc2\x92\x56\x3f\x23\xe3\xeb\x06\x6e\x84\xec\x8b\x43\xc4\xda\x31\xe2\x2f\xa4\xa2\xda\x0d\xd1\x76\x4e\xef\x18\x17\x78\x48\x5e\xa2\x4d\xec\x1a\xf0\xcb\x0b\x98\x0a\x95\xdb\xb3\x9d\x70\x23\x6b\x09\x83\xbe\x18\xa6\xc2\x4d\xf5\x75\x11\x4e\xc9\x6e\xf9\xb7\x69\xfb\x51\x05\xb6\xa7\x2a\xc9\x96\x78\xea\x38\x03\xdf\x20\x5d\xae\x34\x1b\xa3\x60\x1e\x65\x6a\x3f\xc9\xc4\x44\x2e\x16\x2e\xaf\xc0\x4b\x2d\x95\xaf\xa8\xdc\xb6\x8f\x47\xe8\x17\x76\xb7\xf6\x9b\xfd\xaf\x69\xa1\x8d\x0d\xe5\x62\x3b\x68\x19\x77\xc1\x35\x18\x27\x04\x1d\x8d\x0c\x91\xee\xee\x18\x18\x0e\xd4\xb6\x04\x7d\x09\x57\xf7\x1d\x0a\x5f\xe4\x28\x51\xbd\xcb\xb3\x19\xfd\x41\x9e\x2f\xce\x7f\xa0\x91\xed\x5e\xa5\xc6\xe8\x9d\xac\x0c\x48\x96\x7b\xab\x4f\xac\x39\x89\x63\xb1\xf3\xd1\x80\xfd\x78\x3a\xd3\xb5\x94\x5a\xf6\x78\x39\x38\x01\xc9\x43\xd1\x52\x16\xb7\x49\x92\xf7\xfa\x4b\x13\x11\x5c\x82\x7c\x0a\xb2\x95\x72\xdb\x10\xfb\x71\xe0\xd3\xa9\x92\x57\xc2\xf2\xf5\xc4\xdd\x0e\xec\x91\x7d\x46\x89\xa7\x1b\x88\xa9\x8b\x07\xf1\xa3\x21\x9e\x6b\xf1\x2a\x50\x66\x7c\x17\xb4\x14\x5f\xd7\x64\x8f\x6a\x35\x28\x23\x74\xe2\x8d\x16\x0d\xd6\x47\x29\x85\x67\x43\x69\xf2\xf5\x64\x63\xc3\xde\x60\xf6\x54\x64\xb4\x62\xb8\xb2\x7a\xda\xf5\xfa\x99\x8b\x51\x4d\x18\xde\xc8\x89\xbd\x7f\x79\xe1\x11\x59\x45\x9a\xcf\x6c\x34\xb2\x4b\x65\x2c\xc4\x77\xac\x7c\x81\x9e\x87\xec\x6b\xfa\x79\x47\xc1\x14\x47\x91\x36\xea\x0b\x73\xc4\xe7\xb5\x55\x9f\xa1\xbe\x55\x1f\x47\xca\x89\x63\xf1\xb1\x62\xc5\xb0\xe2\x10\x92\x87\x87\x74\x8a\x4b\xfd\xa8\x8e\x1b\x93\x47\xb8\x88\x04\xe6\x16\x19\x0a\xd7\xe3\x09\xac\x66\x97\xce\xb0\x8a\x1b\x02\xcd\x90\xd0\x26\xac\xd2\x42\xdc\xb7\x7c\x54\x19\x0d\xaa\x82\x46\xeb\xb7\x9f\xe3\x86\xa1\x4b\x28\xa9\x02\x2e\xf4\x10\xed\x45\x58\xf0\x42\xff\xc3\x81\x51\x59\x19\x8f\x6d\xcd\x11\x5c\x6a\x96\x19\x37\xe6\x1c\x8f\xd5\xfd\x2a\x2b\x8d\x74\x6d\xb7\xb6\x8b\x73\xbb\xf9\x0b\xc3\x3e\xac\xab\xbb\xd0\xce\x9c\x41\x10\x68\xe7\x20\x78\x94\x77\xbb\xec\xb8\x97\xfa\x91\xe7\x46\x5b\x11\xd2\xe0\x8a\x65\x18\x06\xb5\x8e\xbc\x70\xa4\x29\x94\xd6\x7b\x55\x5c\xa0\xa3\x47\xa5\x7a\x5e\x2c\xec\xcb\x92\x78\x4a\x31\x1c\x97\xb0\x85\xae\xf4\x2c\x73\x9c\x82\xff\x46\x17\xc2\x2b\xcc\xc8\x46\x1d\x17\xab\x84\x95\x6d\x6c\x19\x8b\x1a\xbb\xb9\x58\x6c\x21\x4a\x41\x6c\x09\xcc\x1d\xa5\x6c\x0c\x64\x3b\x19\x06\x91\xdf\x22\x39\x1f\x5e\xbe\x49\xba\xc1\xb0\x02\x66\xe8\xdc\x68\x20\x85\x30\x60\xb9\x32\x78\x70\x35\xad\xab\x16\x5a\x66\x79\xd0\x5e\x77\x63\xeb\xd6\xa2\x6b\x61\xa8\x18\xcf\xe1\x8b\x96\x5e\x01\x8a\xca\xa8\x16\x0e\xa6\x60\x76\x7c\xe1\x8e\x7f\x44\xa6\x80\x2d\x11\xcf\xce\x18\x61\xcf\x17\x09\xe6\xd0\xdd\x13\x2e\x58\x5e\x58\xc7\xc2\x84\xbd\x28\x3c\xbe\x6e\x1a\xae\x7e\x85\x3d\x67\xca\xa1\x34\x15\xb5\xb7\xe4\x9b\xa4\x84\x4a\x33\x71\x99\x4d\xad\x94\xc4\x33\x5a\xc6\x59\xae\x07\x13\xa5\x2c\xe3\x65\xb5\xe8\x85\x15\xd7\x91\xcb\x46\xd7\x61\xb9\x58\xb8\x11\x0d\x59\x88\xac\xe9\x74\x7a\x79\x5c\xbe\xa8\x54\x13\xb4\x6e\xd4\x1a\x51\x61\x3c\x35\x8e\x6a\xa2\xaf\x5b\x8f\xbb\x0c\xc4\xf7\x5a\x82\xc2\xc6\xab\x43\x15\x63\xca\x32\x2d\x40\x90\x32\xc1\x44\x86\x30\x41\x63\xdf\x61\x05\xc3\x70\x1a\xdc\x2e\x38\x51\x23\x55\x3e\x5b\x62\x43\xf7\x31\x0c\xa4\x22\x26\x04\xb3\xdf\x18\xf5\xb4\x6d\x21\xa9\x97\x5f\x37\x36\x14\x1e\xbe\x7c\x50\x57\x2a\x28\x93\xc5\x88\xc7\x43\x8f\x15\x3d\xdc\x69\x94\xb8\xb9\xa8\x80\x7b\xe0\xa0\x51\x51\x23\x69\xe7\xba\xc7\xe8\x85\x57\x8d\x37\xde\xc2\x45\x75\x38\x8e\x1e\x21\xc1\x81\x48\x3b\x43\x45\xae\x67\x7a\xdd\x91\xea\x5d\x4e\xca\xdd\x2d\x05\x2e\x11\xcf\xc2\x8c\xbe\xd0\x7f\x8f\x0c\x2c\x4f\x58\x4c\x00\x37\xf2\x5d\xed\x59\xd6\xcd\x37\xcf\x0f\x02\xd1\x35\xd7\x88\x0c\x30\x4e\x47\x89\xe7\x9a\x11\x05\xcb\xa0\xcd\xbb\xba\x51\x07\x67\x78\x80\x66\xe7\x6c\xbe\x6a\xea\xc8\x4c\xb7\xe7\x6f\xc1\x48\x94\x44\x2f\x98\x35\x1a\x9d\xf8\xf3\x82\x49\x91\x55\x3e\x1e\x3a\x36\x73\x95\xd9\xdd\x2c\xad\x54\x9d\x55\x9c\x54\xb3\xf5\xac\x86\x7f\x21\x4a\x1c\x5d\x15\xac\xce\xab\x12\xdd\x2c\x60\x0f\x49\xd2\xd3\x78\x8b\x47\x6c\xc9\x2b\x6a\xe6\x2c\x73\x76\xe2\x81\x46\xfc\xe1\x17\x10\xb8\x37\x8e\xd7\xd2\x41\xb9\x76\x4c\xbd\x77\x9c\x75\x31\x7f\x4d\x0c\xc4\xb3\xae\x58\x88\x57\xcd\xcb\x43\x23\xf1\xc8\xed\xa7\x27\x3e\x38\x5d\x09\x98\xb5\x84\xdd\x72\x22\x8a\xbd\x7d\x60\xed\xc6\xbd\x61\xb9\x78\xd8\xa2\xd5\xd0\x5e\x64\xb0\xb9\xcf\xfb\x56\x78\x96\xac\xaf\xfa\xc6\xc9\xb3\x21\x4a\xb4\x55\xc7\xf3\x14\x11\xa9\xe2\xfa\xba\xb6\xba\xdc\xd5\x26\xb0\x65\x0c\xfc\x75\x5f\x2e\x46\x48\x9d\x7f\x15\x2b\x60\x6b\x82\xdd\x0d\xa0\xff\xda\xcf\x72\xe2\xd4\x95\x31\x51\xf4\x7a\x6e\x46\x85\x9b\x58\x2b\x76\x2c\xfb\xfd\x8e\xe7\x03\xa8\x91\xb5\x97\x6d\x7d\xaa\xf8\x42\xce\x94\x9a\xb9\xe2\x53\x84\x95\xee\x35\x20\xe8\x8d\x31\x5c\x43\xdd\xb9\xd3\x3f\xad\x37\xd0\xbb\x6b\x6f\x0f\xcd\xa9\xf6\x43\x59\x42\x54\xfb\x65\xbd\x83\x96\x4d\xb5\xdf\x95\x29\x13\xf6\xda\x1d\xae\xdb\xba\x06\x7b\x55\xb1\xd5\x40\x81\xba\xd7\x7f\x90\xe4\x16\xcc\xeb\x07\x89\x6e\x91\xe4\x76\x8f\x60\xc1\x6b\x76\xfb\x4b\xd7\x6c\xf8\xa8\x34\x68\xbe\x9a\xe5\xdb\x49\x3b\x7a\x21\x73\x1b\xcb\xa0\x34\x2b\xdd\xec\xfc\x41\xeb\xea\xea\x6e\x32\xb3\x23\x9b\xbe\x70\xd5\x86\x5d\xdd\x18\x60\xc2\x3d\x32\xb5\x57\x52\x26\xb3\x7a\x97\x5e\xbb\x9d\xa0\x0b\x06\x59\xb8\xef\xa7\x3d\xda\xda\x9d\x78\x71\x7f\xcb\x57\x55\x9c\xf3\xbd\x9a\x8d\xdf\x92\x77\x5c\xd2\x26\x35\x8a\x48\x03\xa2\x3f\xed\xb9\xc5\x13\x66\xb8\x4d\xc4\x82\x49\x90\x8f\xd3\xca\x65\xb9\x1d\xe4\x17\xb5\x58\xf8\xa9\xb5\x7b\xf8\x2f\x69\x3f\xa6\x4e\xbc\xa8\xd6\x6f\x22\xe9\x52\xa3\x34\xc5\xbd\x8d\x7a\xdf\x28\xb9\xab\xa2\x1a\xa5\xde\xef\xde\xed\x58\xa0\xb5\xe5\x9f\x50\x3d\xca\x69\xd9\x52\xbc\x42\xa2\x43\x4e\x6f\x7e\xa5\x14\xe3\xea\xa8\xc0\xfb\xf2\xd4\x67\x72\xb6\xb4\xd3\x28\xf0\x91\xb4\x7d\x53\xab\x96\xb5\x16\xb3\x5a\x3f\x75\x80\xb7\xde\x36\xa1\x10\x56\x88\x29\xeb\xaa\xb0\x6a\x56\x31\x4a\x47\x70\xc1\x26\xea\x01\x25\x89\x69\x60\xd9\xf0\xac\xb0\x5e\x0c\xc4\x20\x78\x6c\x23\x0c\x4d\xe6\x2e\xb4\xda\x04\xcc\x1f\x17\xf6\xad\x62\xa1\x26\x63\xda\xe6\xa3\x89\x9a\x9c\x22\xa9\x26\x56\x32\xa9\xc7\x34\xfa\x2d\x1d\xdb\xd1\x04\x58\xd4\x41\x04\x9e\xa3\xea\xe3\x53\xb3\xea\x18\xd7\x2b\x79\x6c\xcc\x1b\xeb\xfa\xd2\x79\x89\xfc\x20\x09\xe2\xf0\xd1\xe6\xff\xed\x3e\x67\xa0\x95\x03\x88\x3a\x27\x9e\x51\x95\x72\x8c\xf8\xbb\xd2\x40\xca\x08\x5f\x76\x02\x7c\x81\xf3\xc7\x0a\x49\xd1\xe9\x87\xf2\x07\x69\xb5\x0c\x35\x45\x1e\xe2\x02\xb4\x52\x2c\xad\x3f\xec\xc2\x09\xe6\x07\x75\x37\xd4\x9a\x8f\xaa\x87\x6d\x59\x6e\xbb\xa1\x1b\x9d\xbd\xd4\x02\x07\x27\x78\x35\x10\xfa\xec\x8e\x6d\x37\x98\x98\x2f\x76\x2d\x68\x2d\x00\x0e\xfc\x04\xbf\x75\x27\xe3\x36\xc4\x85\x3f\x5f\xbe\xc3\x66\xe1\x6e\xa7\x95\x15\x0b\x17\x7d\x1c\x47\x1d\x23\x47\x38\x0f\x75\x64\x48\xdb\xac\x8c\x62\x25\x06\xd0\x41\xd0\x33\x3e\xa2\x37\x61\x19\x17\x6a\xf2\x5e\x27\x16\xbe\x91\x49\x0a\x4e\x1e\x49\x82\x77\x5a\x2f\x6e\xb8\x1d\x47\x70\x96\x88\x8a\x61\x72\x1a\x7b\x05\x0c\x31\x82\x64\x25\x99\x2e\x8a\x4e\x1d\x94\x3a\x58\xc5\x20\x35\x8e\x07\x38\x4b\x8b\xee\x68\xc5\xae\x79\xbd\xad\x4f\x2f\x8b\xb0\xa3\x43\xdf\x8d\xdd\x60\x61\x08\xb2\x62\x6f\x06\x68\x75\x4d\x4e\x0d\xaa\x2d\x2d\x9e\x0d\xe4\x97\xcd\xaf\xe1\x21\x20\x5b\x8f\xa7\x54\x50\xa9\xf6\xf2\x86\x19\xde\x4f\x42\x5e\x9b\xad\x86\xcd\xfd\x8e\xa1\x57\x11\xa0\x90\xd8\xbb\x04\x65\x68\xb9\xee\xcc\x9b\x56\xd9\x73\x2d\x81\x45\x83\x84\xf8\x9f\xff\xfb\x66\x44\x01\x7c\xae\x6f\x4e\x2e\xc7\xe5\xc5\xab\x3f\x00\x06\x20\x76\x7f\x78\x93\x82\x70\x7f\x73\x39\x00\x1e\x14\xad\x4b\xe8\xb4\x63\x30\x0c\xc4\x7b\x01\x45\x1f\x31\x0e\xcc\x9c\x4c\xa7\x73\xa8\x7c\x40\x7e\x74\x24\x6a\xa8\x71\x48\x8c\x94\xb1\x02\xaa\x42\xcf\x27\x88\x17\x50\x49\x92\x39\x33\x89\xc7\x2f\xe8\x13\xd7\xd0\x98\xac\x60\x53\x33\x8c\x7d\xba\xf3\x1f\x78\x0c\x02\xdd\xb9\x96\xfc\x78\xf8\x7a\xf3\xaf\x49\xa7\x37\x39\x3f\x3a\xca\xc9\x6b\x62\xdf\xf4\xac\x03\x45\x78\xc4\xa1\xb7\xa9\xde\xff\x50\x2f\x4d\xfb\x68\x9a\x3e\x99\x52\xf1\x82\xc0\xad\xec\x4a\x81\x93\x4c\x46\x6b\xeb\xe3\xab\xeb\xe9\x0c\xe3\xb9\x76\x96\x54\x17\x72\x72\x07\xc2\xb7\x21\x97\x2f\xb6\x88\xdc\x9d\xc7\xea\x04\x54\x9a\x40\x3e\x5c\xf9\x7e\xbc\x4d\xd0\xd2\x91\x3b\x2a\xa1\x6d\xf0\x23\x1b\xba\x55\xac\xda\x6c\x63\x3d\x7a\xe4\xc2\xc3\x5a\x88\x24\xeb\xd8\xdc\xab\xe9\xf0\xe6\x72\xc4\x6e\x21\x24\xd3\xa0\xbb\xf4\xa4\x0f\x8d\x2a\x41\x98\x54\x41\xa3\x7c\xb1\x39\xdf\x1d\xbf\x55\x7d\xab\x89\xef\x8e\xdf\xd4\x31\xea\x68\x72\x0a\x6c\xfd\xc7\xd9\xf8\x60\x74\x4e\xbd\x47\xf6\x31\x2e\x1e\xb5\x64\xf2\xe0\xc2\x7b\x0a\xc0\xf2\x9c\x63\x95\xad\xbf\x6c\x3f\xdf\x3a\x1f\x9b\x64\x23\xd1\x89\x4f\xf7\x28\xb1\x08\x12\xb7\xbf\xa5\xc4\x46\xa2\x4e\x7b\x6b\xd0\x5f\x83\xc0\x4e\xa9\x07\xd7\x50\xae\x8c\x29\xf9\xf0\xe6\x25\xf0\x76\xe8\x3b\x21\x54\x55\xf0\xac\x45\x15\xfc\x47\x48\xca\x2e\x25\xe6\x21\x29\xcf\xb6\xce\xf1\xf4\x2d\x48\x7b\x49\x80\x26\x4c\x04\x94\x31\x45\xdd\x04\x92\x01\x61\xf0\x21\x1f\xa6\x37\x30\xad\xac\x5d\xa3\x19\xb2\x85\x72\x29\x06\x50\x36\xbd\x70\x00\x94\x6c\xa1\x0a\xfb\x40\x9e\x83\x28\x8b\x03\xfb\x0e\xae\xe8\x2a\x85\xaf\x45\xba\xdb\x46\xe1\x02\x28\x5c\x01\x06\x57\x62\x11\x5d\x1a\x8e\x36\xa7\x8d\x71\x60\x76\xc1\x2e\x62\xce\xd8\x52\x9c\x38\x02\xc4\x29\xb2\xe0\x0d\x69\x6b\x6d\xbd\xac\xab\xfc\x42\x22\x8e\x01\x61\x75\x35\x09\x12\x77\x55\x40\xa7\x46\x17\x2b\xd8\x77\xcf\xb8\x7c\x1d\x4a\x73\x20\x16\xaa\x1d\x06\x2a\x2d\x80\x18\x15\x17\xdf\x0a\x6b\x7c\x44\xba\xb1\x01\xfd\x2b\x67\x67\xfe\xd4\xb4\xcb\x41\x98\xf9\xeb\x38\xaf\xbc\x39\x81\x49\x98\x62\x60\x12\x16\xef\x08\xaf\x6f\x4a\x69\xf2\x0f\xe2\x18\xed\x0d\x2e\x38\x18\xeb\x01\x24\x2e\xef\xf2\x4d\xda\xc3\x26\x34\xed\x74\xab\xf6\x88\x73\x21\x89\x96\x1b\xf4\xec\xfa\x92\xb5\x80\x33\x1c\x20\x74\xce\xcc\x43\x05\xba\x2e\x0b\xdb\x5b\x37\xf5\xde\xab\xd7\xbb\x3f\xbe\x3d\x3c\xde\x7d\x49\x6a\x41\xd7\xad\x51\x3b\xab\x26\xe6\x04\xe3\x5d\x9d\x5e\x0c\xca\x6f\xa7\xc3\xcf\xe8\x04\xfc\x87\xf7\x07\x87\x8b\x1f\x7e\x84\x7f\xbb\x87\x2f\xbf\xcf\x1e\x6d\x8d\xc5\xe5\x13\xef\x52\xaf\x46\xf3\x8b\xe9\x30\xeb\xd8\x6f\xe9\x4d\x68\xf3\xe7\x51\xb7\xcd\x60\xdb\x0c\x9e\x9a\xc1\x33\xf1\xa2\x8a\xcd\x2c\x9e\xf2\x0c\xb9\x54\xe3\xb1\x41\x63\xa8\x24\x83\x7a\x0c\x7c\x7f\xc3\x37\x2a\x44\xca\xbe\x45\x23\xf7\xb5\x67\x39\xc7\xce\x19\x3c\xb3\x08\x8a\xc1\xd3\x0e\x65\x3d\xcd\xe9\x67\x3b\x67\x8f\xf0\x6e\x98\x0c\xb6\x33\x0a\xa6\x87\xf8\x81\x14\xac\xbc\x00\x7a\x54\x71\xb7\x41\xd1\xa5\xda\x68\xa9\x60\x61\xda\x46\xaa\xdd\x16\x60\x97\xb3\x6d\x73\x84\x8a\x76\x1e\x22\xe9\x7a\x1c\xb9\x34\x69\x97\x49\x68\xe7\x8e\x28\x46\xca\x08\x5a\x39\xbf\xc8\x00\xcb\x59\x91\x1b\x2d\x02\x69\x72\x32\x18\x62\xec\x7f\xad\xcb\xbd\xb9\xc6\x8b\xe8\xcf\xd6\x5c\xd3\xad\xf5\xb8\x06\xc3\x2d\xbe\x66\x9b\x7c\x8d\x48\xee\x1b\xd2\xd2\xde\xb6\x96\xbe\x44\x62\x2a\xed\x4e\xa3\x6f\x5c\xbe\xa1\xb0\xe5\xa7\xa3\x97\x83\xcb\xcb\x02\xd1\x81\x90\xcb\x29\xd3\x33\x37\x8a\x65\xaf\x1f\x42\xd3\xe7\xe6\x32\x56\xc4\x20\xa4\xdb\xeb\xe7\xec\xf9\x5f\x86\x3f\xc2\x64\x06\xd7\x18\x90\x4e\xcf\xc6\xe7\x68\x63\x32\xab\xce\xad\xc2\xa2\xf1\x49\x1b\x1b\xd5\xb4\xa6\x2d\xba\x58\xac\x9e\xa7\x0e\x3f\xc5\xf0\xf9\x27\x2a\xa1\xf2\x8b\x85\xd7\x04\x85\x93\xac\x96\x8d\x25\xdc\x33\xec\x2d\x03\x98\x92\xb4\x8b\x7f\xf7\xf5\xd8\xb4\xd4\xb7\x0e\xb3\x37\xcf\x2b\xc8\x98\x4b\x46\x57\x00\x48\x32\x76\xc1\x8d\x4b\x1c\xa4\x09\xe2\xd1\x8f\x76\x6d\x50\x80\xc6\xf3\x8e\x95\x0c\xbf\xc9\x34\x23\x63\xca\x64\xc2\x66\x36\x8b\xd8\x11\x87\x28\x9d\x5e\x8d\xcb\x51\x41\xf2\x82\xaa\x20\x6b\xce\xd1\xdf\x5c\x8d\x5f\x4c\xcb\x11\x0a\xc7\x26\xa9\x4e\x8b\x88\xd9\xf1\x23\x79\x45\xed\x2d\x7d\x0e\xee\xc1\xac\xac\x63\xad\x8c\x28\x63\xbd\x58\x5f\x0f\x07\x5b\xb6\x7a\x0a\x9d\x9e\x9d\xe3\x0c\xc2\x0e\xc4\x73\x0c\x0b\xb1\x76\x4a\x34\xdf\xcc\x38\xaa\xfd\x9a\x9b\x62\x96\x44\x9c\x68\x68\x69\x33\x40\x97\x11\x13\x9a\x43\xf6\xd4\x03\x5f\xf1\xa0\x23\x1a\xf0\x09\x19\xab\xc1\xb2\x3f\x3d\xf9\x95\x2c\xe2\x6b\x89\xaf\x80\x81\x40\x17\xe2\x49\xb5\xea\xa1\x68\x39\x86\x4e\x6d\xe6\x5a\x17\x0d\x0c\x45\xc5\xc7\x1a\x88\x60\xb2\x51\x2e\xde\x31\xcb\x53\x1a\x43\x7e\xd9\x13\x5d\x86\x6b\xee\x42\x7e\xdd\x9e\x53\xb2\x67\x1c\xff\x9f\x42\x72\x55\x56\x37\xb9\x56\x6f\xd3\x95\x37\xb9\xca\xca\x58\x83\x2e\x1d\xf1\x74\xc2\x9d\x7a\xa6\x1d\x8f\xfe\x06\x15\x60\x9b\xf8\xb4\xa5\x5b\x01\x23\x72\xef\x1c\x6b\xcc\xa3\x6a\x98\x8b\x87\x13\xb4\xa9\x30\x4c\x4b\x4c\xf8\x5d\x4d\xdc\x95\x8c\x66\xa5\xd5\xff\xac\xe4\x27\xe8\x73\x31\x60\x88\x42\xa5\x78\xf1\xab\x34\xb6\xa9\xb4\x4a\xdb\x06\x96\x30\xb6\x0f\x71\xab\x3c\x9f\xe2\x06\x08\x04\x77\x76\x38\xae\x56\x5c\x61\xfb\xc1\x12\x1b\x88\x4e\x32\x7b\x81\x57\xf0\xea\x65\x21\xed\x62\x25\x08\xdc\xaa\xcc\x73\x1b\x08\x02\x56\x5a\x84\xeb\xbc\xad\x8c\x6e\x5a\x84\x35\x76\xbc\xfc\x03\xe5\xdc\x77\x2e\x16\x9c\x80\xb7\x94\xdc\xf7\xe0\x0e\x54\x89\x0c\xc3\xe1\x18\x1f\x06\xcc\xa9\xf6\x9c\x38\x2c\x18\xeb\x04\xd9\x95\x82\xe6\x0a\x5c\x4e\x28\xf2\xad\x4a\x6b\x5d\x24\x3b\x15\xb7\x20\x1c\xe6\xb7\x2c\xe4\xe4\xc9\x77\xaf\x0e\x93\xa5\x29\x07\xbf\x8f\x7c\x1a\xca\x47\x90\xf8\x1b\xee\x41\x42\x48\x23\xd3\x16\x77\xd9\xa2\x88\xf2\x00\x7b\xaf\xde\xbe\x3a\x7c\x05\x05\x13\xbe\xcd\x90\x54\xb3\x96\x06\x47\x9e\xdb\xf2\x91\x3e\x4b\x66\xbc\x4b\x94\x77\xf9\x7e\x97\xcc\xaf\x06\x27\xb8\x4b\xc3\x17\xe3\x87\x41\xe1\x79\x90\x4d\x32\x2c\x09\xfb\x9e\x00\x3e\x04\xed\x89\xf2\x97\x34\xbd\x1c\x60\x52\x1a\xe6\x66\x4e\xec\x27\xc1\xd4\x0f\xcc\xe2\x56\xaf\x35\xb9\xf2\xdd\x45\x0b\x8e\x0c\x1b\xee\x84\x1f\xad\xc5\x36\xee\xe0\x78\xa7\x86\x7a\x15\x75\x87\xe7\x06\xaf\x89\x59\x58\xb2\x66\x3b\x73\xdb\x22\xe3\x77\x3b\x94\xae\x37\x3f\x6e\x31\x86\x44\x71\xee\xbf\x75\xf4\xd3\x96\xda\x8a\x13\x21\x7c\x6b\x2d\xbc\x03\x81\x76\x18\x9c\xb9\x7a\x01\xc1\x29\x80\x01\xe7\x83\x82\x1c\xfc\x12\x96\x02\x0e\x48\xb5\x46\x48\x46\xe8\x53\x08\xc1\x9b\xc0\xd4\xd7\x55\xd8\xa3\xe4\x97\xa3\xa3\x61\x03\x37\x93\x24\x54\x73\x8d\x1b\x1b\xf4\x1b\xc4\x47\x4a\xd2\x5f\x16\x3d\x00\x3e\x3a\xea\x67\xb0\x7d\x24\x80\x46\x92\x1e\x1d\xfd\xb4\x78\x94\xd9\xe2\xb8\x10\xc3\xb4\x76\x4d\xc0\x22\x20\x5e\xe9\x41\x19\x01\x5b\xf1\x86\x54\x48\xb2\x43\x3d\x3a\xca\x71\x2b\x0b\xdb\x5b\x3b\xd9\xfd\x22\xaf\xef\x6f\xab\x46\xf5\x2d\x77\xec\x1a\x9e\xd6\x1a\x29\x1a\xdf\x24\x71\x30\x5d\xce\xef\xd9\x84\x7e\xce\x17\xae\x64\x42\xfa\x74\xb5\x36\xee\xf9\x28\x9e\x19\x9b\xc2\x70\x50\xb9\x6e\xea\x47\x49\x51\xab\x81\xa8\x7c\xab\x6e\x49\x68\x3e\x5b\x9b\x6f\x41\x93\x9c\x27\x99\x1a\x7d\x8d\xe4\x51\x1b\xdd\x2f\xdf\x85\x28\xdd\xea\x66\x77\x20\x73\x4d\x45\xe7\xfd\xe6\x12\x56\x8c\xf1\xe4\xfc\xe0\x72\x50\x5e\x8c\x80\x65\x0e\xc6\x4e\x53\x92\x6c\xc1\x70\xc3\x04\xb5\x2b\xc5\xd7\x3c\x2c\xd3\xc0\xb4\x65\x7d\x57\x6e\x35\x1e\x6d\xa1\xb3\x87\xba\xac\xa3\x66\xda\x2d\x8e\x3e\x35\xd2\x47\x8b\xa3\x6e\x96\x01\x1c\x0c\x43\xc3\x53\xb1\x59\x07\x7f\x74\xd4\x04\xa0\x2d\x84\xb2\xc3\xe0\x7a\xf5\xae\x3c\x1c\x21\x24\x9e\x2e\x16\x32\xd3\x45\x6c\x2c\x82\x37\x1a\x5f\x41\x8a\xda\xb2\x93\x26\xd3\x44\x3c\x7e\xd9\xf7\x9a\x43\xfe\x69\x3a\xad\xe4\x0a\xb5\x5a\xa4\x4c\x73\x9b\xee\x47\xc4\xa9\x60\x80\xfe\x3e\xe6\x0b\x00\x2e\x47\x62\xbc\xa6\xd7\xd6\xc4\x1f\x44\x61\x1f\x3c\x2e\xe0\xac\x38\x0c\xd2\x28\x49\xa9\x41\x97\xe6\xd6\x71\xc2\x9c\xd1\xa1\x71\x28\x63\xf4\xfb\x7f\x64\xa3\x32\x96\xf0\x11\x8f\x41\xac\x4a\x05\x24\x89\x49\x39\x26\x15\x2e\xa6\xd2\x25\xef\x37\x3e\x71\x9f\x46\x14\x90\x56\xdc\x4e\x67\xe3\xf3\x31\xad\x6f\xf3\x8b\x9c\xd0\xcc\x46\xe7\xa3\x3f\xae\xe9\x79\xc9\xc6\xfd\x00\xd8\xfc\x28\xbe\xfd\xac\xd0\x04\xb9\x05\xfe\xf1\x1d\x9f\xf6\xd2\xac\xd9\xcf\x90\x25\x1c\x1d\xe1\xe8\x57\x59\x47\x5b\x59\x37\xc7\x88\xec\x59\xda\x3b\xea\x2e\x8e\x9e\xf4\xb3\x2e\x00\x2a\x6e\x50\xe2\x18\x25\x7b\x0e\x7b\xc1\xc3\xdb\x7a\xd0\x1d\x39\x6f\xd9\x26\xc6\xba\x6c\xfd\x82\x86\x40\x45\xf2\xa4\x3e\xb7\xe3\xdd\x68\xca\x69\xda\x2d\x72\xd1\xdc\x57\x83\x4a\xec\x75\xfb\x88\xb7\x5a\x90\x8c\x82\xfe\x92\xa1\x7a\x92\x34\x52\x9b\x8d\x76\xcc\x94\x93\xc1\x54\xed\xe6\x3a\x87\x92\x73\x98\x3f\x8d\x14\x09\x82\x4d\x58\xda\x6c\x74\xb3\x04\x70\xa4\xbd\x5f\xb6\xfa\x8d\x0c\xb3\x2c\x38\x19\x0c\x35\x92\x2c\x89\x92\x96\xba\xc9\x7a\x47\x5b\x8f\xb0\x9d\x7c\x7b\x62\x87\x35\xb9\x77\x8a\x60\x0d\x40\x7e\x3e\xbf\x00\x96\x83\x31\xa9\x5c\x27\x77\x93\x71\x82\x24\x61\x31\xaf\x06\xa2\x75\x8d\xd4\x7c\xe8\xce\x59\x8b\x7d\xd8\xf5\x08\x41\xcb\x19\xc3\xf6\x30\xb5\x1f\x89\x05\x78\x18\x80\x9e\xed\xdf\x4f\x0e\x46\x83\xd9\xe9\x05\xcb\x27\x08\x4f\x8b\x3d\xae\x3e\xe1\xb8\x64\xac\x99\x88\x02\x2c\x0c\xf2\xcd\x7a\x1c\x74\x05\x71\x2f\xcc\xa3\xda\xbc\x5f\xa4\x2e\xbd\x8a\x2e\xad\x65\x82\xcc\x2c\xe7\x2f\xde\x4a\x3a\xff\x26\xa4\x6a\x9c\x75\x24\x73\xee\xe1\x54\x06\xb6\x22\x52\x97\xb4\xc4\xda\x3d\x0b\xb6\x94\xa8\x45\xa7\xb0\x31\x98\x7d\x1a\xeb\x50\x82\x22\x0a\xdf\x2a\x60\x6a\x56\x32\xa3\x72\xdb\x5c\x85\xe3\x11\x88\x48\x78\x3c\x39\x9b\x4e\x39\xa0\x00\xaa\xca\xd1\xa0\x81\x0e\xf3\x0d\x66\x20\x77\xe1\xfd\x3d\x6b\xd1\xf1\x2c\x13\xb7\x2e\x53\x8a\xeb\xbe\x32\xe6\x49\x14\xcd\xc4\x55\x60\x1c\x7a\xa3\x91\xc3\x96\xc8\x38\xc4\xa6\x3e\xf8\x09\x05\x37\x71\xdc\x86\x75\x6b\xc4\xfd\x88\x7d\x00\xf3\x03\x9c\x32\x62\xb0\x53\x99\x55\x90\x1c\x87\x8f\x4a\x2d\x87\x4a\x35\xce\xe0\xd1\x6b\x3d\x6e\xd3\x14\x25\x33\xc5\x42\x67\x73\xe8\x47\xf1\x43\xbc\x7e\x15\x40\x7b\xff\x38\x64\xe6\x54\x5c\x29\xcb\xa6\x9d\x71\xa7\xd1\x18\x3b\x62\xe4\xba\x0b\x7a\xd8\x42\x01\x23\x29\x29\x8c\x18\xae\x94\xc0\x56\xa7\x67\x6b\x57\xbd\x71\xbf\x3b\x1c\x55\x34\xfb\x98\x9e\xe5\xf8\xb7\x43\xba\x93\xdf\xf1\x0c\x2f\xf5\x0b\x4d\x73\x62\xaf\xbe\xb8\x51\xc2\x99\x9e\x33\xdf\x5c\xc3\xb6\x78\xc4\xca\x56\x1b\x9b\xe3\x8f\x39\xfb\x6c\x97\x54\x32\x2d\x2d\xb8\x4b\x9a\x12\x62\x82\x5c\x37\xb2\x2d\x30\xbb\xc9\x6b\x3e\x62\x00\x60\x6e\x98\x68\x5f\x37\x36\xdc\xf0\xe6\x8b\x7d\x04\x8b\x23\x5a\x3a\x97\x80\xfd\x3b\x48\x3f\xeb\x04\x12\x4e\x5c\x48\x85\xe6\x3c\x1d\x7d\xa0\x54\x36\xbb\xb6\x0b\xae\x60\x64\x31\x5b\xed\x00\x34\x50\x30\xa0\x32\xe3\xc7\x5c\xf3\xd1\xc9\x0c\x30\xc2\xa2\x83\xc1\x29\x08\xe8\x47\x6a\x91\x84\x28\x43\xdd\x01\xe2\x5f\x2c\xe8\x0d\x1a\x57\x91\x41\xa1\xa1\xef\xc2\xc4\x11\x33\x0e\xd0\xfe\x33\x21\xd7\x69\x8c\xd3\x84\x4d\x49\x1f\x60\xb8\x3d\xe5\xd3\x2d\x03\x40\x5b\x10\x27\x23\x4a\x5c\xbb\x08\x24\xeb\xba\x59\x43\xcd\xa8\xcf\x84\x63\x58\xa3\x9a\x2a\xcb\x9a\x25\x35\x6d\xaa\x13\x1d\x53\xcf\x72\x8f\x16\x04\x9e\x18\x53\xa5\x1b\x23\x22\x32\x95\x20\xd5\x64\x99\xc2\x4e\x7a\x0e\x66\x3f\xe8\xd7\x22\x52\x64\x64\xd6\xab\x6a\xe8\x42\xc3\x68\x7f\x18\x62\xe8\x18\x33\x4f\x76\x17\xda\x14\x9d\x82\x3f\x3a\x08\x37\x91\x62\xc1\x55\x2b\xe3\x79\xe3\xae\x7e\x51\x69\x7c\x96\xdb\xba\x8e\x1d\xd1\xe5\x56\x4e\xcc\x7d\xe2\x78\xf2\xfb\xf4\xa3\x37\xf1\xaa\x91\xf3\xdd\x49\x17\x51\x6b\xdf\xb2\x6e\x75\xa7\xea\x40\x71\xf0\x55\x0e\xc8\x82\xce\xcb\xf2\xd5\x15\xa1\xfb\x91\xa0\xae\x1f\x79\x03\x75\x47\x85\x02\xa1\x31\xa8\xe7\xb0\x66\xdd\x35\x45\x14\xe0\x50\x47\x26\xd2\xb8\xef\x68\x16\xa9\x9a\xaa\xb8\xa4\x80\x35\x87\xf5\x54\xb8\x7a\x0b\x1f\xb7\x48\x67\xaf\x8a\x5f\xb4\x4a\x71\x16\x9f\x64\xa1\xca\x10\xe4\xd9\xbb\x68\xcd\x9c\x41\x60\xf3\x51\xa5\x7f\x68\x9c\x0f\x2e\x2f\x53\x7b\xd1\xad\x52\xb7\x64\xdc\x12\xd7\x8d\x98\xac\x34\x81\x6f\x09\x1a\xf2\x32\x76\x03\x56\xa7\x79\x60\xc0\xea\xee\xe1\x75\xc2\xa1\x58\x9d\xa5\x79\x94\x0e\x8e\x21\x7a\xb5\x7a\x0a\x1f\x80\x9e\x14\x85\x0a\xb9\xa8\xcd\xd0\xf9\x97\xda\x20\xf8\xf5\xe6\x56\x02\x6f\xe2\xc7\xf0\xad\x80\x15\xb3\x98\xc5\x29\xa5\x77\xe5\x15\x7e\x5d\x62\x2e\xcb\x42\x58\xd4\x88\x03\x15\x76\x25\xb2\x14\x14\xe2\xcb\x07\x76\x87\xc4\xc2\xa2\x9c\xab\xe5\x55\x6e\x53\x65\x73\x4e\x96\xf2\xdc\x51\x4e\xc0\xac\x7b\x7c\xb7\x54\x72\xa5\x36\x79\xb1\x10\xf1\x10\xa5\x87\xfe\xc6\x46\x40\x82\xa4\x3a\x4a\x6e\x97\x1a\xff\xed\x52\xdf\x0d\xd7\x2b\x00\x4b\x12\x96\xa6\x5b\xa5\xde\xf4\x1b\xa4\xb8\x59\xa5\x10\xc7\x53\x95\x40\x91\xb9\xde\xe5\x97\xac\x7f\x30\x63\x62\xd4\xe4\xfe\x35\x13\xe5\xa7\x38\xa9\x3b\xe7\xeb\x27\x74\xa5\x94\x4d\xb4\x28\x85\x9a\xbf\x90\x17\x1b\x21\x80\xf7\x5a\xcd\x27\xd9\x16\x59\x62\x17\x1a\xb6\xd7\x46\x2a\x3d\x66\xb5\x87\xc6\xcd\x42\xa5\xca\x7d\x77\x75\x02\x77\x13\xe2\x3d\x45\x15\xe2\xa8\xed\x54\x8c\x03\x4c\x27\x7c\xf0\x57\x5d\xd5\xb1\x7f\x6e\xb9\xe9\x73\x19\x67\x2c\x92\xe8\xf0\x52\x41\xb1\x56\x30\xd5\xd0\x4d\xe0\x6e\xf9\x79\x72\x9a\x2a\x09\x0b\xe6\xbf\x6d\x74\x0d\x4b\xd6\x50\x76\x20\x45\x53\x52\x97\x16\xba\x96\xfd\x65\xbc\xc7\xa7\x2f\x54\x3b\x7d\x2f\xc1\x07\x46\x93\x64\x0e\xb8\x54\x96\x12\x93\x73\xb4\x48\xb6\xea\x65\x46\xbf\x32\x94\xa0\x77\x68\x64\xee\xf2\x8a\x89\x11\x05\x6b\x63\x61\x51\x74\xc1\x2f\x8f\x28\xb8\x2a\x26\x4b\xe4\x70\x2c\x0a\x19\xac\x82\x89\xe1\x27\xfe\xdf\x0f\xdd\x17\x49\xdc\xc2\xe6\x84\xa7\xd7\x70\x55\xf5\x2e\x9c\xdf\x2f\x76\xfc\x0e\xe2\x77\xb4\xee\xe0\x2e\xc4\x2d\x3d\x77\x04\xda\x13\x37\xc8\x0f\x09\xb1\xc7\x8d\x67\xd8\x11\x30\xdd\x8c\x51\x6b\xdb\x4a\xe7\xc6\xce\x8b\x1c\xc7\xbe\x80\xea\x0a\x1f\x24\x8f\x50\xa9\xd8\x78\xb6\xbd\xe2\x2d\x06\x9a\xbb\xeb\xce\xb0\xad\x51\x3e\x3c\xba\x5f\x53\xfb\x9f\xb3\xad\x55\x6a\x6c\x84\x83\xa8\xf4\xa0\x85\x1a\xea\x69\x15\x93\x37\xeb\xaf\x64\xed\x96\xd1\xc8\xea\xd5\xc1\xe0\x99\xb6\x23\x4b\x9a\x56\xa2\x51\xeb\x78\xc7\xda\x80\xb5\x5a\xc0\x5b\xb5\x7f\x41\xd9\x25\x76\x45\x14\x8b\xb0\x36\x68\x20\x5a\x50\xc7\x11\x03\x23\xc0\xdf\x83\xc0\x81\x26\x8a\x45\xb8\x94\x4b\xfd\x95\xa1\xb4\xbc\x2f\x2e\xe7\xca\x78\xa0\x1d\xed\xa3\xa9\x4e\x7c\xe1\x99\x95\x19\x3b\xc3\xd0\x4e\x94\xa3\xf6\xf2\xe2\x4f\xb6\x82\x45\x8d\xe9\x20\xe6\x7a\xbb\xc1\x6b\xcb\x47\xa5\x96\xc4\x84\x9a\x54\x0c\x79\xac\x10\x56\xe0\xad\x4e\xa4\x8e\x37\x53\x80\x06\x55\x76\xe8\x43\x6e\x33\x2f\xc6\xc0\x57\x8a\x27\x67\xe1\x6b\x53\x36\x19\xac\x98\xb9\x99\xdf\x17\x43\xf6\x54\x0d\x03\xef\xdd\xb8\xff\xa5\x1a\xe6\x95\x86\x9b\xd5\xb6\x7c\x39\x9d\x7e\x1c\x63\xc4\x89\xaa\x15\xe6\xa9\xcf\x0a\xb5\x4a\x20\x37\x7e\x2a\xc3\x70\xdc\x4a\x29\x24\xb9\x8a\xb7\xd3\x8a\x2f\xfc\x0d\x9d\xd8\xf0\xee\x4d\xe0\x84\x00\x19\x04\x64\xa5\x42\xa1\xa0\x6d\x90\x08\xc9\xcf\xb4\x65\xbb\x50\xc6\x27\xbd\x20\x04\xda\x3a\x9b\x92\x41\x08\x54\x23\x04\x48\x05\x46\x2e\x42\x05\xa8\xcc\xaa\x0d\x24\xfd\xac\xb3\xaa\xe4\xa5\x2e\x81\x8b\x4b\x6d\xe5\x62\xb7\x20\x5f\x85\xe7\x5c\x35\xbb\xbd\x0a\xba\xac\x1b\xd0\x53\xad\x2f\x17\x19\x29\x00\x43\xce\x2b\x15\x65\xb7\xae\xc6\x76\xd4\xc2\x45\x85\xd0\xfa\x76\x09\x50\x63\x74\xb8\x00\x8b\xfb\x6c\xdd\x1f\x75\x20\x59\xb7\x8e\xd4\x3c\xfc\xbe\xba\x82\xba\xcd\x98\x1f\x90\x9b\xfa\x70\xb4\x08\x1a\xb4\xc7\x52\x6d\x44\xe6\x59\x37\x93\x57\xa8\x94\x83\xef\xaf\x3b\x64\x76\x44\x2b\x65\x81\x4b\x73\xc2\x9e\x6d\xa9\xc1\x70\xf8\x03\x4c\xc7\xd7\x93\xf8\xa6\xa8\xf0\xc0\x3b\x9a\xb6\x4a\xf4\x7a\x11\x96\xb2\xd1\x61\xc2\x5e\x0a\x61\x0c\xfb\xcc\x0a\xd3\xf4\xb4\xa8\x05\xb0\x5d\x69\x1b\x23\xdc\xf6\xf1\xd5\x57\xb4\xb4\x49\x33\xdf\x5c\xa1\x38\xcc\xf7\x15\x70\xe2\x66\x16\xdb\xb2\x86\x39\x1c\xc0\x1b\x5b\x69\x3b\x66\xa1\x5d\x8f\x32\x15\x56\xe6\x44\x83\x05\x97\x2b\x97\xfd\xac\xdd\x8b\x05\xe6\xfb\x90\xda\x0c\xc7\xe9\x58\xce\x66\xd3\xab\xff\x2c\x9d\x81\xad\xd8\xbf\x9a\xeb\x9b\x10\xab\x0d\x8d\x16\x60\x74\x9d\x3d\x9f\x2a\x14\xce\xf6\x21\xa4\x4a\xc6\x6d\x80\x00\x6f\x30\xfc\xcb\x0e\xfa\xae\x06\x1f\x47\xe8\x92\xa9\xb4\x37\x62\xc6\x66\x7a\xf2\x2b\x0e\x5e\x76\xfa\x56\x62\xa4\x6d\xde\xc7\x99\x44\x9c\x02\x14\xad\xce\xf8\x05\x3b\x68\xb3\x17\xe1\xd0\x27\xc6\xc9\xaf\x3d\x4a\xec\x8d\xfb\x78\x98\x6e\x5b\x0e\xd2\xbd\x00\x8b\x52\xe0\x0f\x74\xef\x2b\xc5\x47\x03\x8b\xf0\xf0\x32\x64\xd0\x98\x4b\x7a\x4f\xbc\xf4\x3e\x1f\x9c\x1b\xf8\x47\xfe\xc6\x70\x9f\x00\x8c\x74\x30\xfb\xcc\x91\xa2\x39\xd9\x35\xa7\x0f\xa3\x2a\x39\x12\xc3\xe3\x95\xf5\x04\x2f\xc9\x7d\xba\x8c\xdb\x29\xe7\x83\xd3\x8f\x4d\x1c\xbe\xe8\x80\x6f\x3c\x01\xe9\x66\xe4\x40\x75\x66\xbf\x93\x11\x4d\xaf\x26\x43\xa4\x28\x21\x9f\xc7\x2e\x37\xeb\xd8\xb3\x2d\xce\xaf\xd4\xb6\xb1\xa1\xc1\xf1\x00\xfb\x9c\xaf\x82\x46\x48\x1d\xd1\xf4\x85\xc5\xef\xd3\xf1\xb0\x82\x6b\xb1\x58\x5f\xa7\x6c\x06\x5a\x2c\x18\x35\xad\x69\xb6\x3c\xc9\xa3\xe4\x96\x95\xcd\xcd\xcb\xb9\x53\x9d\xee\x1e\x1e\x7e\x10\xf7\x07\xf1\xb1\x3b\xad\x13\xc3\xe9\xcd\xc9\xe5\xe8\x7f\xdf\x4c\xe7\x74\xb8\x8f\x5e\x99\xe8\x92\x9a\x4e\xb9\x99\xfc\xe6\xdf\xf4\xac\xa9\x14\x06\xea\xe2\xd2\x68\x01\xfa\x9b\x7e\x05\xc1\x8c\x48\x15\x16\xcc\xc7\x14\xaf\x26\xf3\xf1\x1c\x19\x97\x53\x82\xca\x28\xe1\x88\x25\x1b\x1b\xc1\xab\xfd\x70\xf1\x22\xce\xe3\x23\x52\x0e\x49\x2b\xab\xc1\x64\xc7\xfa\xf5\xb4\xc4\xd1\xfc\x90\xd1\x44\xc3\x06\xe1\xa5\x3f\xe5\x8c\xae\x03\x49\x3b\x45\x4b\xba\xb9\x07\x6f\xb0\x50\x49\x19\xcc\xdb\xdc\xa4\x03\x1e\x86\xe2\xab\xb7\xe3\x18\xc5\x78\xa7\x80\xfc\xce\x18\x60\xed\xb7\x8d\x26\x43\xff\xa1\xa8\x30\x62\xf4\x14\xcd\x40\x15\xc6\x72\xbc\x30\xd9\x28\x3a\xf0\xbd\xac\x13\xe2\x61\x8a\x97\xf6\xe9\x0c\x06\xa7\x1b\x4d\x5f\x3f\x1c\xab\x1b\xfc\x35\xae\x25\x24\xaf\xbf\xec\x50\x61\x9a\x75\x54\x01\x72\xe8\x70\xfe\x50\x6c\x2f\x98\x03\xb5\x13\x28\xc3\xe2\x44\x40\xbd\x81\x48\xf3\x49\xf6\xe2\x08\x1d\x2f\xa3\x51\x45\xf9\x24\x69\xe8\xd2\x8d\xa4\xf7\xcb\x4e\xff\xc9\x0e\xb0\x9f\xb1\x56\x24\x0d\x2e\xc5\xb1\xb5\x3b\x96\x44\x1d\xe3\x9c\x35\xf8\x5c\xc9\xcb\xf7\xfb\xfb\xaf\xde\x1d\xda\x21\x9f\x04\xa7\xf5\x2f\xf7\x76\x0f\x77\x83\x2c\x37\xcc\xe8\x1b\x7d\xeb\xd3\x6b\x1a\x8d\x4d\xaa\x1a\x7d\x94\xb2\xa1\xef\x6a\xf6\x60\x2f\xb6\xa3\xaa\x8b\x9a\x80\x3a\xea\xfd\x59\x9a\xbc\x58\xdf\xdc\x4c\xb2\xae\x78\xbf\x0b\xf3\x20\xc7\x3c\xcb\xd8\xd5\x01\x8d\x2e\xca\xbe\xa4\xf0\xe6\x0e\x64\x27\x91\xf8\x78\x36\x88\x12\x85\x0c\x15\xa2\x39\xcc\x90\xfa\x0a\x89\x3b\x44\x88\xf8\xb8\x19\x85\xca\x67\x82\x02\x2d\xcb\x6d\x27\xf9\x5c\xf6\x9d\xf0\x34\x33\xd2\xed\x78\xd7\x63\xef\xfd\xcb\xc3\xbf\xff\xf0\x4a\x9a\x8e\x4d\xaa\xb0\x18\x7c\x09\xeb\x3f\x09\x07\x2b\xe8\x42\x58\x51\x56\x92\xf9\x75\x3c\x20\xd8\xdb\x47\xab\x4f\xd6\x39\xaa\xb6\x6f\x5f\x7d\xf7\xe6\xdd\x77\xc7\xaf\xde\xed\x1d\x1f\xee\x7e\xf7\xb0\x5a\x05\xf8\x8e\x5a\xfd\x17\xda\x7a\xdd\x4d\x65\x97\x60\x29\x0b\xb1\xe9\xce\xae\x12\xaa\x00\x15\x89\x4e\x33\xac\x68\x3c\x38\xdc\xfd\x70\xf8\x2f\xa4\x32\xc6\x67\x82\x75\x54\x53\xca\x8f\x30\x99\x6b\x86\xdd\x8b\x84\xd7\x0f\x9a\x4b\x2d\x76\xa0\xd1\x45\x90\x3c\x22\xa8\x25\xc3\x86\x38\x83\x83\x4c\x92\x18\x4e\xe2\x37\x7e\xf9\xcc\xc2\xbb\x26\x84\x9b\xf6\x1d\xd6\xbc\xb0\x84\x6d\xe8\x7c\xfc\xa7\x36\x2f\xb4\x57\xf1\x0e\x2f\x40\xba\x91\xec\xd9\xda\xa7\x41\xb9\x06\x4b\xc1\xc9\x25\xd9\xa1\x13\x0c\x85\x3f\x38\x9b\x62\xa8\x42\xa0\x6c\x8d\x64\x03\xf4\xe4\x43\x9f\xc7\xc1\x0c\xa8\xb7\x3a\x8e\x5d\x2e\xf5\xc4\xce\xf4\xc5\xa6\x9a\x65\xca\xc5\xbf\x1d\x0f\x87\xa3\xc9\x0f\xb3\x11\xb4\xea\x64\x34\xfb\xfe\x70\xff\xad\x18\xe8\x3b\xa3\x97\x17\x68\xec\xb2\x71\x39\xef\x20\xcf\x09\xc1\x0f\xe9\xc0\xd8\x27\x62\x63\x88\x2e\x05\x57\xcb\xf8\x5a\xdf\x0a\x1a\xa2\xfa\x36\xa8\xbe\xc1\xd5\x75\x47\xf1\xbf\x77\xef\xdf\x1d\xef\xbe\xfd\xe1\xfb\xdd\x77\x3f\xee\xbf\xfa\xf0\xe6\x65\x45\x30\x08\x50\x26\x1b\xff\x2e\xe1\x6e\xa8\xd3\x5e\x42\xdd\x64\x80\xd7\x48\x3a\x89\xb6\xe7\x51\x9f\xe6\x13\x77\x28\xf1\x1c\x13\x43\x49\xf0\x40\xfa\xeb\xa7\xd9\x18\x95\x80\x27\x37\x67\xb2\x2e\x9f\x4f\x40\x96\x27\x6f\x11\x37\x73\xb7\x24\xa3\xfd\x33\xc2\x18\xf8\x47\x32\x8f\x0b\xe3\x4d\x82\x80\x97\x9c\x71\x9d\xd7\xd2\xc0\x2d\x24\xd4\x2f\xec\x99\x59\xe7\xba\xaa\x2b\x18\xe4\xd2\xd6\x92\x49\x21\x58\x7e\x5e\x2c\xda\xeb\x05\x59\xad\x06\xb0\xe8\xc0\xf3\x66\x4e\x53\x08\xa9\x66\xec\xf1\xf1\x08\x53\x55\x77\x74\x4c\x7b\x4b\x3c\xc2\xa8\xd2\x49\xbe\x5c\x6c\x9d\xbb\x24\x2e\x5d\xb2\xf9\xe0\xcd\x6c\xac\xde\x61\x15\x58\x87\x95\x62\x9d\xbb\x89\xd9\xcc\x8f\x1f\xde\x58\x06\x63\x09\x5c\x13\x02\xc9\xf3\x08\x3e\x3c\x2e\xc8\xc5\x3d\x3c\xd5\x0e\x2a\x01\x02\x18\x14\xc5\xf0\x99\x5a\xb5\x9b\x6c\xed\x24\x79\xb2\x93\xe0\x79\x1f\x08\x28\x41\xf3\xdf\xd9\xe4\x0f\x6b\xc6\x2d\xdd\x8e\x94\x84\x55\xa1\xfc\x06\x8b\x9d\xf4\x59\xea\x06\x0a\xd0\x40\xcc\x44\xd9\x50\x0b\x97\x93\xca\x6a\x3e\x8f\x21\x32\x51\xf4\x45\xbc\xa5\xee\x0e\xb1\x85\xc0\xb0\x22\xf6\xd9\xcb\x50\xc4\x3c\xa8\x1f\x61\x7c\xaa\x23\xb2\xca\x96\xa7\x7e\xe0\x67\x34\xac\xdd\xc9\x92\x89\x19\x7a\xb1\xf5\xcb\x0b\x0e\x57\xf1\x29\xdf\xec\x37\xb2\x94\xa3\x52\xc8\x2b\xbd\x3c\x29\x10\xa0\x9b\xa3\xd5\x5e\xef\x97\xa4\xff\x24\xa1\xe0\x17\x8f\x7b\xbf\x3c\xee\x3f\x79\x9c\x2d\x40\x90\x3a\x2a\xa1\x68\xd6\xcd\x9e\x70\x0c\xf7\x2d\x8a\xe5\xbe\xb3\x65\xc2\x45\x4e\xea\x02\x71\x4c\x55\xc8\x62\xd8\x96\x51\xbb\x88\x62\xcb\x93\x13\x13\x80\xf4\x21\x11\x50\x95\x90\xc1\x29\x8f\x31\x05\x88\x49\x1d\x35\x64\x23\x19\x2f\x9f\x48\xc2\x96\xa9\x5f\xfd\x1d\x79\x5b\x26\x14\xf1\x8a\x2d\x94\xa9\x28\x36\x07\x48\x47\x80\x35\x94\x3f\x30\x5b\x52\x52\xfa\x9a\x6e\xb6\xb3\x35\x36\x5a\x12\x44\x90\xa3\x1e\xa5\x1c\xf5\x08\x51\xbf\x8f\x88\xfc\x3c\xc2\xdb\x9e\xe9\xd9\xfc\x7a\x81\x46\x03\x65\x37\xcb\xa1\x95\xb6\x16\x57\x83\xf1\xe5\x7c\x9a\x2f\xe6\xa3\xcb\x7c\xf1\xef\x19\x60\x5d\xc1\x5f\xb1\xc9\x7e\x39\xfa\xf7\xcd\xff\xff\x62\x6d\xb1\x4e\x66\x8f\x7a\xa7\x57\xd8\xed\x78\x32\x98\x8d\x06\xe6\x64\x86\x3e\xbb\xcc\xc5\xcc\x8c\xaf\xce\xcd\xa7\x93\x59\x92\x99\x70\xbb\x19\xc4\x08\xf5\xa5\xa1\x14\x39\xbf\x31\xc3\xa1\x19\xce\xcd\x25\x6c\x74\xcc\xfc\x64\x3a\xfc\x6c\xe6\x43\x33\x3f\x9b\x4e\xe7\x06\x5d\x83\xe1\xf5\x1c\x33\xaf\x62\x7d\x13\xec\x86\x3d\xda\xd9\xb5\x99\xcd\x2b\xd0\x0e\xae\x7a\xc4\x7d\x17\xda\xbb\xbe\x24\xda\xba\xdf\x8b\x39\x28\x6c\x7c\x23\x0e\x87\xe8\xba\xcf\xa0\x13\xbd\xd3\x4b\xd8\x22\x96\x18\x14\x91\x50\xd3\x36\xd4\x9c\x0e\xd8\x9f\xce\x29\x9d\x42\xe1\xa1\xaf\x19\x8e\xe1\x77\xfc\xbb\x19\x5e\x1a\xba\xdf\x36\xc2\x1f\x0b\x87\x2d\x07\x70\x17\x6d\x73\xb1\x6d\x2e\x9e\x9a\x8b\x67\xe6\xe2\x2b\x73\xf1\xdc\xf0\x45\x27\x73\xc1\xad\x8e\x1d\x36\x41\x3a\xae\x0d\x50\x74\x03\xbb\xeb\xdf\x0d\xf4\xe3\x35\x20\xe3\x98\x43\xa6\x64\xef\x50\xc0\xce\x40\x44\x31\x37\x97\xc8\xda\x42\x1d\xc4\x97\xb6\xa6\xff\x68\x33\x38\x81\x81\x33\x38\x9d\x4d\x27\x9f\xaf\xcc\x89\x39\x19\x8e\xe1\xdf\xd4\x9c\x8c\xcf\x69\x44\x8d\xf1\xc3\x31\xf8\x3b\x7d\xee\xd9\xc4\x8c\xae\xe0\xc3\xf0\x4c\x9e\x86\x19\x52\xfe\xf1\x64\x28\x8e\xd5\xe9\x1b\x06\xb3\x8f\xe6\x37\x33\xbb\x39\xf9\x6c\x68\x08\x98\xd2\x94\x20\x4e\x98\xf2\x0a\x77\x61\xe5\xf5\x60\x62\x50\xde\xfb\x38\xc2\x9f\xe9\xe4\xdc\x80\x00\x08\xff\x60\xcc\x8d\x61\x63\x3e\x9f\x9b\x1b\x03\x7c\x11\xbf\x31\x5a\x65\xfd\xd0\xb2\xed\x82\x6e\x20\xd8\xcd\xa0\x9a\x15\xd5\xa6\xd0\x93\x26\x1c\x2b\x51\x33\xae\x18\xab\x78\x03\x80\xd7\x4e\x4f\xc2\x09\x6c\xdb\xb0\xff\xf0\x4a\x10\xb6\xd1\xc5\x6c\x74\x66\x2e\xe1\x73\x86\xa3\xf2\xd4\x94\xb3\x53\x73\x53\x8e\xa0\x3d\x2c\x79\x5c\xbc\x4a\x9b\xc5\xac\xba\x84\x3a\xe4\x12\xd6\x22\xf8\x3b\x37\x83\x3f\xc6\x40\xf4\x39\x4c\xd2\xe9\xcc\x9c\x4c\x29\x88\xf7\xe9\xe8\xf2\xf2\x1a\xef\x61\x41\xeb\xe1\x33\x34\xea\x29\x3d\xa3\x17\x42\x83\x27\x61\xc4\x0b\xa6\xf4\xb7\xa4\x3f\xd8\xec\x78\x5e\x33\x38\x9d\xa3\x5e\x74\x36\x2c\x69\xf0\x9e\x81\xcc\x25\xe3\xb1\x84\xdf\xf1\xf9\xc5\x9c\x3e\xe5\x12\x28\x35\x17\x88\x77\x64\xc6\x25\xf6\x2b\xa5\xe0\x9f\x9b\xc1\xf9\xc8\x4c\xa6\xf4\xc1\x93\xe9\xa7\xd9\x00\x5d\x02\x5d\xc2\xbf\xdf\x0d\x2a\x84\xe9\x0f\xd6\x36\xbb\xc1\x90\x9a\x12\x37\x80\xce\x8c\x90\xc4\xf2\x62\x80\xef\x3c\x0a\x06\x38\x3a\x6e\xae\xae\x50\x49\x05\x2f\xe7\x23\xe0\x34\xe3\x39\x8c\x71\x34\x47\xa5\x76\x83\x46\x60\xb9\xe7\x77\xa6\xe5\xd3\x78\x38\xbf\xc0\xb1\xe1\xa4\xde\x38\x64\xae\xf3\xe9\x05\x13\xa8\xd6\x7f\x87\x5d\x4e\x13\xd3\xeb\x67\xec\xea\x45\x2f\xd7\x7e\xb5\xf6\x12\x59\x4d\x61\x74\xa5\x72\x89\xcb\x71\x82\x27\x9d\xa1\x6f\x19\x5c\xd6\xdf\xbe\x79\xf7\xbf\xfe\x7e\xfc\xe3\x87\xb7\x9e\x95\x57\x17\x82\x54\x56\x82\xac\xdb\xdb\xdd\xfc\x79\xb0\xf9\x67\x6b\xf3\x9b\xe6\xf1\x5f\x1a\xb0\x46\xfe\x47\x76\x74\xf0\x04\x98\x7f\x79\xd4\x3c\xea\x1c\x99\xa3\xf4\x28\x3b\xba\x3d\x5a\x1e\xbd\x38\xda\xe9\x6f\x99\xfd\xdd\x37\x6f\x0f\xdf\xfb\x45\x46\xf0\x6c\x75\x62\xdb\x05\xdc\x12\x48\xd3\x72\xc8\x38\xda\x30\x29\x25\x48\xc7\xb9\x44\xc5\x6b\x2e\x30\xab\x67\x83\x4f\xa4\x18\xe1\x2d\x7e\xaf\x6f\x3e\x91\xd8\x51\xd4\x48\x22\x24\xc4\xe0\xc5\x62\xbc\x98\xc4\xa7\x18\xa4\x34\xaa\xb1\x61\x63\x0a\x28\x0c\xb7\x85\x6e\x72\x62\x21\x79\x1d\xde\xf5\x42\xf5\x22\x8d\xc6\x2d\x98\x75\x32\xbc\x58\xe3\xf6\xfe\xd6\x9b\x68\x51\x58\xa7\xa3\x7c\x4f\xab\x48\xa4\x35\xe8\x1a\x11\xba\x6a\x63\x63\x28\xd6\x79\xf1\xd7\xc8\x66\x12\x2b\x73\xc6\xf4\xe3\x4c\x7f\x4b\x13\xc7\x37\xde\xe2\xb1\x25\x58\x7f\x98\x0c\x12\x05\x94\x85\xe8\x2a\xfb\xec\xa0\x9b\x50\x59\xe1\x0a\xe0\xec\x07\x5c\x19\x35\xb7\x27\x83\x76\xc1\x8d\x78\x03\x6f\x7b\x35\xa6\x9d\xd5\x30\x5e\x0c\x5c\x3e\x40\xd1\x2f\xa6\x59\xb0\x34\xf1\x3d\x40\x1b\xaa\x19\x8f\x92\x66\x30\x59\x67\x68\x2d\x36\x9d\x37\x8f\x0d\xb9\x4d\x80\x8e\x07\x26\x45\x77\x2e\x7f\xc0\xeb\x37\xc5\x2e\x47\x14\xb1\x57\x71\xcc\xfb\x93\x5f\x39\xe3\x3d\x5d\xb0\x56\x39\x68\xe7\xc9\x59\xd6\xe2\x53\x65\xe2\x8e\xac\xf0\x68\x69\x87\x66\xca\xcb\xf1\xe9\x48\xa7\x52\x02\x9a\x15\x9c\x0e\xe6\x3a\x9d\x53\xcc\x7c\xca\x87\x9d\x85\x25\xa2\x69\x53\x4c\x78\x67\xce\x03\x84\xe9\xb0\xd6\xe2\x91\xf8\x6b\xb9\x01\xaa\x6a\xb0\x97\x40\x19\x00\x23\x9d\xa9\x4c\x64\x87\x9c\xf1\x61\x34\xbc\x09\x49\x9e\x51\x4a\x90\xfd\x01\xb9\x6a\x15\x86\x92\x2d\x05\xc4\x4b\x02\x02\x28\x45\xb2\x5f\xfd\x3e\x9a\x7d\xd6\xb9\x23\x4c\x90\xcc\x83\xe9\x55\xd8\x68\xf0\x2e\x59\xa2\xfe\xd3\xb9\xa2\xbd\x11\x80\xb7\x5e\x47\xa8\x81\x94\xea\xd0\x62\xe2\x7b\xb7\xd2\xfd\x72\x0b\x57\xf2\x30\xb2\x89\xed\x7e\xba\xe5\xc0\xc9\xdf\xe2\x0d\x64\x37\x08\x68\x43\x6e\x8e\x83\x8b\xb0\x3e\x6c\xd6\xc9\xaf\xda\xfd\xc5\x71\x17\x12\x72\x1c\x9e\x61\x6a\x4a\xd6\x66\xc7\xb8\xe4\x5c\xa3\xaf\xef\x93\x5f\x69\x69\x5f\x6b\x65\xe4\xfc\xe2\x58\xae\xcf\x26\xee\xfc\x3c\x59\xb7\x17\x1b\x46\x7f\xa0\x5f\xa9\xb2\x9b\xd6\x65\x32\x7b\xdf\xd8\xe0\xdf\xa6\xc0\xe2\x65\x6e\x7e\x2a\xc2\x8c\xe2\x18\xef\x9d\xd1\x63\xf3\x18\x5e\x72\x9e\x30\xc5\xb1\x39\x6e\xfe\xed\xd5\x87\x83\x37\xef\xdf\x15\x49\xbb\xf9\x55\x73\x9b\x1d\x4e\x8d\x70\x6c\x1d\x37\xe5\xc7\x5e\x37\x0e\x6e\x04\xe3\xbc\x46\xef\x82\x64\x42\x43\x7a\x6a\x34\x4c\xa7\xdb\x99\xf8\x4d\xf8\xa2\x47\xea\xc6\x06\xa4\x3a\x4c\x45\x11\x64\x66\x2a\x2f\xad\x20\x76\x2a\x66\x84\x92\x63\x81\xa2\x68\xf8\xb7\x30\xde\xb2\x75\x9b\xeb\xb2\x9d\xbb\x5c\xeb\xca\xd2\xd6\xd0\x3c\x45\xf3\x63\xa9\xc6\xd0\xf9\x5e\xdf\xd0\xf9\x20\x6a\x9c\x85\x99\xc8\xca\xc3\x16\x3b\xb6\x1e\xba\x24\x73\x4c\x43\x87\x7a\xd0\xa8\x8a\x55\x74\x97\x2f\xa8\x99\x6f\xb9\xf4\x39\x0e\x0f\xd2\x51\x4f\x45\xe7\x18\xa7\x32\x54\x2d\x1e\x58\xef\xeb\x13\x6f\xd4\xaa\xaf\xfd\x71\xe0\x0c\x28\xd1\x95\xbc\xdc\x31\x0d\xee\x27\xac\xc3\xf6\x11\x24\x76\x25\xad\xda\x37\x79\x3a\xa2\x78\xe4\x50\x79\xa4\xf5\x91\xe8\x1e\xe3\x92\x4e\x30\xa8\x16\x3e\xc8\xab\x6f\x83\x4a\xa1\xcc\x99\xa3\x96\x30\x41\xe4\xba\x19\xf0\x20\xb2\x98\x2e\x12\xe6\x53\xa8\xdf\x24\x0f\xa4\x1c\x86\x6a\xed\xd3\x78\x7e\xb1\x36\x99\xae\xa1\x6f\x51\x10\xc3\x25\xc0\x21\x34\x1a\x97\xa5\xb1\x7c\x39\xbc\x84\x5f\xb1\x15\xaa\x6f\xbe\xab\xd1\xd5\x34\x6c\x43\x41\x58\xc4\xbe\x71\x76\xb6\x3b\x32\xec\xa9\x3d\x61\x02\xe2\xe9\x32\x48\x69\x01\x3b\xe5\x46\x15\x1a\x6c\xbb\x72\x96\x15\x6c\xa4\x36\xd4\xf9\x08\x15\x40\x24\x69\x04\x2b\x6d\x8e\x3b\x29\xa2\xa6\xeb\xb1\xa6\x01\xed\x59\x5e\x93\x43\xc7\x74\x0f\xe9\x2d\x8b\x1d\x31\x15\xf5\xbd\x45\x0d\x54\x29\x99\xa7\x54\xc4\xa6\x73\x8b\xf1\xad\xf3\x75\x79\xb5\x4e\xe9\x81\xf7\x1d\x02\x27\xa3\xce\x4c\x55\xc7\x3a\xc1\x01\x31\x2d\xcd\xb1\x5e\x78\xa4\xf7\x66\xff\x43\x9d\x46\x75\xea\x9e\x63\x22\xc2\xee\xa3\xb4\x7f\x51\x1f\x12\xae\xd5\x1d\x19\x66\x8b\xf5\x68\x85\xd7\xa1\x6f\x68\x7a\x5a\x07\x16\x69\xd9\x63\x1d\xcb\xaa\x09\x46\xb5\x7c\xf0\x00\xc1\x93\x15\x2c\xc8\xf1\x7c\x36\x37\xb9\x7c\x3f\xb7\x4f\xe6\xa1\x63\x88\x98\xae\x72\x7b\x1e\x0c\xa4\x20\xf3\x5f\x32\x9a\xd0\x97\x24\x34\xc2\x70\x34\xff\x22\xde\xe9\x6f\x03\x7c\xbe\x9f\xd5\x11\xe4\x03\x99\x5c\x37\x95\x1b\x07\x9c\x83\x96\x6b\x2c\x1e\x38\xe6\xc7\x54\x93\xa8\x75\x2c\x1e\xdb\xff\xb5\x3c\x9f\xe5\x38\x59\x9e\xb9\x22\xb7\x3a\xd3\x6b\xd7\xe7\xfc\xa3\xfc\xff\x81\xad\xb1\xb1\x11\x2c\x14\xce\x3c\xc2\x2d\x03\xc4\x0f\xee\xe0\xdb\xae\x09\xe4\x9b\x6d\xd3\x3d\xac\xd3\xd6\x1f\x48\xe6\xd2\xd5\x83\x04\x91\x5c\x0b\x7d\x83\x4e\xd1\xee\x13\x90\x24\x65\xb1\xd0\x0c\x62\x3c\x1c\xa1\xc6\xfe\x73\xe0\x60\xc7\x9b\x17\xc5\xfd\x96\x2b\xf9\x9a\x7b\x8d\x29\xb0\x9d\x46\x19\x5d\x97\xfe\x8f\x2f\xd9\xe4\xfe\x48\xc8\xe1\x9f\x8d\x8d\x87\xae\xdd\x5d\x1e\xc5\xb9\x48\x2f\x38\x65\xd7\x19\x87\xac\xe5\x30\x93\x70\x3c\xe3\x3e\x00\x9a\x0e\x5e\xfe\x85\x4d\xd7\xbe\xa7\xe9\x70\xf7\xc1\x2d\x47\xf5\xdb\x86\xc3\xe4\xae\x4d\xfd\xe7\x9a\x4d\x48\x01\x62\x85\xa6\x07\xb7\x9b\x34\x98\xe7\x02\xaa\xdd\x8e\x9b\xe2\x6f\xac\x24\xf9\x85\xe3\xce\x07\xed\x66\xb5\x26\x95\xef\x5f\x6f\xe7\xc1\xf6\x8a\x3f\x5f\x36\x56\xae\x05\x24\xb3\xbb\xd9\x26\x19\xde\x1d\x9b\x0b\xda\xbc\x9e\xfd\x85\xa7\xb2\x18\xc6\x8e\xc0\x97\x34\x3b\xf8\xba\x67\x48\xa5\x38\x8b\x94\xe8\xa1\xe7\x65\x41\x7b\x66\x6e\x1a\xb7\x54\x9b\xed\x4c\xdc\x14\xe1\xc7\xfa\x1b\x98\xd6\xd3\xa4\x9b\xe1\x28\x96\xae\x26\x4a\xfc\x39\x76\xc5\xab\x12\xe5\xf5\xf8\xa5\x9f\x35\xd9\x3a\x93\xbb\x01\x29\xc9\x98\xe8\xeb\xcb\x9b\xd3\x8f\x21\xcd\x74\x86\xf9\xc0\x3a\xb9\x21\xd8\x94\x91\xf0\x7d\xba\x18\xcd\xa2\x36\x90\x83\xd2\xf1\xac\xd4\xdc\x6a\x5c\xbe\x42\x61\x96\x8f\x51\xb3\x2e\x65\xdb\xa9\xd4\xeb\xe7\xc7\x3d\x4e\x49\x70\x19\x4b\xf2\x84\x99\x5b\xd2\xaf\xa5\x45\x6d\x55\xd0\xe2\x98\x51\xa2\xc7\x3c\x3a\x50\x45\xe2\xd6\xc5\x09\x2a\x5f\x19\x13\x1e\x68\x27\x0f\xfa\x84\xb6\x2b\xe6\x4f\x2b\xe8\x57\x94\xd3\x27\xaa\x2f\xa3\x00\xd4\xd8\x50\x7f\x3c\x60\xd7\xe8\x18\xef\xc6\xc6\xb1\x73\x86\x87\x32\x0a\x0d\xd3\x5e\xab\x2f\xdb\x3d\x78\xe2\x81\x2b\xc1\xe9\x9e\x7f\xf5\xd5\xd3\xaf\xac\xf0\xb5\x8f\x8e\x25\xa0\x3e\xe9\xd4\x7d\x72\xa5\x82\x72\x4e\xb5\x02\x6e\x64\xcc\x94\xc2\x9b\xed\xad\x96\x66\x21\xb7\xa8\x76\xbe\xc1\xc8\x54\x98\xc3\xf3\x94\x1e\xdd\x15\xb5\x87\xf0\x02\xbe\x38\xc3\x88\x1c\x07\xe8\x3e\x90\x15\xf0\x50\xed\xd8\xf2\x3b\x72\x33\xcf\xbe\x83\x8c\x69\x69\x65\xea\x18\x81\xa3\xdb\x3e\x2c\xfd\xfa\xd9\x14\x23\x5e\xe8\x95\xf1\xe4\x7f\xb6\x57\xc6\x93\x7f\xa4\x57\xd6\x56\xf6\x8a\xef\x94\xff\x9b\x7d\xf2\xe2\x5f\xd9\x27\xe5\xc5\xcd\xd9\xd9\xe5\x28\x52\x36\xd1\xd7\x0f\x26\x43\x17\xc7\x4f\xc0\x86\x4a\xb0\x5b\xf1\xd9\x30\x39\x07\x24\xec\xe2\xcf\xf4\xca\x86\x3a\xcb\x1c\x0a\x16\xad\x37\xdb\xfd\xc2\xa5\x20\x68\xdf\x84\xaf\x85\xf5\x7d\x6c\x93\x89\xda\xc1\xd5\x75\x44\xac\x99\x98\xf3\x9b\xc1\x6c\xe8\x7d\x25\x45\xbb\xae\x17\xdb\x8b\x05\x41\xe0\x22\xdb\x73\x74\xf9\x91\xb3\xd9\xce\x80\xc7\xd9\x96\xa0\x06\x60\x75\x6a\xda\x32\x76\x76\xc3\xe3\x24\x13\x19\xe2\x72\x3a\xfd\x78\x73\xfd\xc6\x8a\x03\x2b\x98\x71\xb0\x7a\xe8\x2b\x26\xf9\x2a\xbd\x1e\x07\xba\xe9\x2f\x71\xc5\xc5\x40\xc8\xdf\x46\xd2\x89\xed\x56\xbd\xd3\xb4\x44\x84\x34\x49\x7d\x7e\xb1\xa2\x95\x25\x5d\xb5\x80\xd4\x48\x12\xc1\x48\xa2\x6c\x09\x9b\x7a\x4a\x8a\xf5\xf1\x20\x7f\xa8\xd0\xba\xcc\xe8\x63\xd4\xed\xfa\xd1\xd9\xdc\xcc\x68\xff\xca\x4b\x71\x81\x29\x4d\x8b\xd8\x9c\x14\x94\xe9\x12\x70\xca\x0e\xf0\x8e\x0a\x3b\x5e\xdd\x39\x59\x2c\x78\x69\x02\x3e\x30\x70\x53\x16\xa1\x4e\x76\x06\x2a\xef\xc4\x31\x59\xeb\x67\x85\xea\xe1\xd1\xc7\x55\xd0\x33\x0c\x31\x89\xc1\x20\xfd\x4b\x67\xcd\xbe\xe9\x4f\x46\x17\x83\xdf\xc7\xd3\x9a\xdb\xae\x2b\x3a\xc5\x4e\xc6\xa5\xe3\x6f\x05\x4b\x44\x7c\xd7\xc1\x8b\x8f\xf9\x9d\xbd\x76\x27\x5b\xf1\x6e\x6a\xee\xef\x07\xe2\x7b\xf6\x2b\x84\x55\xf8\xd0\x67\x9e\x29\xd0\xc0\x93\x30\x75\x1c\x7f\x35\x70\xc7\x10\x96\xba\x85\xd1\x74\x31\x28\x55\x46\x26\xd2\x2e\x2d\xeb\xb9\x7a\x46\x0d\x47\xb4\xb3\x3a\xe6\xa6\x7f\x48\x3d\x1a\x91\x65\x09\x28\x91\xde\x4c\xe6\x77\x15\xcf\x6e\xef\xa4\xaf\xd1\x08\x28\x6c\x13\x52\x8e\x3e\xce\x71\xb1\xbc\x3d\x34\xa9\xef\xef\xde\x1d\x48\xff\x3a\xbe\xbe\xba\x8b\xbd\x1a\xc5\xca\x49\x2c\xbd\xae\x54\xd0\x66\xe6\x72\xfa\x09\x18\xf0\x05\x0c\xd8\x82\x63\xd4\x8b\xc6\x05\x53\x76\x20\xb3\x23\x51\xa1\xc7\x43\xe0\x02\x9f\x1a\x94\xbc\xb3\x03\x13\xa2\x16\x25\xa1\xe8\x01\x70\x3f\x7b\xc1\xe3\x11\xf1\xc3\x7b\xa3\x9d\x53\x1d\xf0\xe8\xe6\xcb\xf4\x13\x32\xdd\xf9\x94\x0f\x34\x56\xf1\xad\x6e\xb8\x4c\x77\x95\x58\x8d\xef\xf9\x2a\xf5\x79\xd7\xf3\x23\xb5\xa5\x02\x4e\x4c\x74\xb1\xb6\x88\xe2\xcd\x43\xcf\x04\x16\x61\xba\x7e\xbf\xd7\x68\xad\xae\xc8\x3f\xe6\x4a\x13\x65\xb5\x4f\x24\x6d\x82\x7c\x0b\x6b\x16\x1e\xef\xc3\xcf\x7c\xa0\xf7\x0e\x3c\x02\xe2\x65\x86\x2b\xa6\x3c\x2b\x2a\x73\xd2\xc4\x2e\x37\xdc\xd4\xad\x7e\x1e\xec\x33\x10\x17\x2e\x26\xbc\x4b\x61\xed\xd2\x3d\x55\xd5\x94\xd7\x23\x61\x33\x8d\x2a\x86\x4d\x57\x46\xf8\xc3\xcb\x08\xff\xdc\x77\x04\x35\xb6\x6b\x3e\xca\x2d\x95\x01\xe4\xc4\xb4\x32\x51\xa0\x50\x0b\xa3\x13\x49\xd4\x84\xcd\xa6\xd7\x5f\xfc\xd9\x35\x9f\xb9\x24\x5e\x40\xb6\x1a\x11\xba\xaa\x32\x86\x91\xa8\xa1\xc6\xec\xfe\xec\x12\xe3\x5f\x29\xc9\x74\x3c\x01\x71\x09\xad\x2f\xd0\xa4\x19\x4d\x24\xe1\xd5\x13\xc5\xc9\x28\x3a\x8a\x9e\x83\xa0\xdd\x14\xc0\xdb\x9d\x78\xe2\x2a\x82\x27\x97\x36\x04\x64\xb7\xf2\x5c\x22\x16\x1b\xfc\x24\xe2\x84\xc5\x82\x53\x44\x98\xb1\x22\x84\xd4\xdf\x45\x7e\x1a\xd6\x21\x57\xe6\xe4\x73\x64\xb1\x88\xbe\x22\xe7\xdf\x88\x1b\x4b\x26\xb2\xff\x4a\x63\x70\xa3\x09\x1a\xbf\x04\x4a\x25\x41\x2e\x1a\x89\xd0\xb6\x73\x0c\xdb\xdc\x9b\x3b\xfa\x63\x38\x3e\x3b\x83\x6d\xdb\x04\x24\x2c\x41\x50\xb7\x11\x6f\xf3\xb8\xb9\x99\x8c\x7f\x2b\xf8\xe7\xa6\x32\x29\xc7\xe5\x01\xf1\xec\x1a\xde\x1c\xc8\x5f\x16\x0e\xcd\x2a\x04\xc2\xb1\x5c\xbf\x4a\x7b\x6c\xf2\x80\x06\xb0\x1d\xad\xd1\x57\x1c\x1e\x59\x97\x10\x51\x51\xd9\x70\xba\xd7\x85\x9a\x72\x04\x6d\x1a\x89\xce\x82\x73\xc5\xf2\xee\x48\xee\xca\x0d\x14\x44\xd1\xc3\x3f\x7e\x0e\xda\x48\xfc\xb9\xd7\xd0\xa4\x08\x21\x43\x01\x2d\x93\xa9\x80\xea\x6d\x13\x68\x3a\x79\x66\xb3\xaa\x5b\x1f\x7c\x49\xbb\x4f\x27\x35\x37\x99\xb8\x2b\x52\x37\x54\x54\x8f\xad\xcb\x54\x27\x67\x37\x62\x89\x57\x19\x05\x22\x21\xcd\xeb\xb5\x2f\x6d\x25\xb4\xca\xb4\x95\x0a\xb9\x78\xec\x67\xde\x01\xf3\x64\xa4\xab\x8c\x7e\x9d\x40\x97\x8b\x5a\x20\x17\xbd\x12\xa5\xd3\x3d\xcc\x6c\xa7\x40\x7d\x03\x51\xed\x47\xe5\x6a\x9a\x83\xb9\xed\xad\x00\x56\x0e\xe0\xca\xc7\xf0\xc8\xa8\xdf\x32\xac\xab\x6e\xa4\x0f\xb1\x13\x14\xa9\xfb\x73\x7c\x5d\x84\x5e\x6e\x52\x75\x10\x43\x1a\x8f\xd4\xca\xf9\x9e\x84\x84\xf3\x93\x4c\xec\x40\x52\xe8\x21\x37\x2e\xf1\xf8\x82\x39\x8e\xbd\xfd\x82\x37\x50\x83\xa3\x63\x01\xed\x8d\xfb\x45\x0d\xf2\xa4\x31\x8e\x1c\xad\xd3\xc8\xe1\xf8\x01\x9e\x5a\xdc\x07\x48\xcc\x62\x77\x58\x5f\x14\xb4\x3b\xb0\xee\x6f\x9c\x34\xa4\x64\x67\x7f\xb2\x8d\xa0\xb5\x27\xdb\x8c\xd4\xca\x75\x08\x46\x77\x64\x39\x19\x1e\xf3\x30\x07\xd6\xe2\x7e\x61\x9f\xdb\xfd\x90\xf4\xa5\x71\x03\xa4\xc2\x65\x60\xa4\x38\x9e\xa0\x3e\x81\x47\x87\xdd\x65\x74\x22\x43\x80\x40\x58\xa3\x08\x2f\x0a\x41\x32\xb9\xb9\x3a\x19\xcd\xbc\x61\x85\xcb\xb5\x07\x39\x45\x20\x8f\x2a\x4a\x32\x91\xe1\xc6\x38\xff\x31\xa1\x3b\xce\xf1\xae\x23\xde\x16\xb2\x3c\x43\x6d\x5b\x99\x80\x86\xc3\x9f\xdb\xa7\xa5\x33\x95\x70\xca\x59\x26\x79\x85\x7a\x36\x73\xfb\x6b\x05\x94\x86\x6d\x43\xfd\x58\xb1\x3e\x88\xe8\x75\x9f\xd8\x71\x1b\x34\x91\x55\xde\xdc\xd1\xfe\x78\x35\xfc\xee\xb6\x07\x71\x9f\x65\x77\xb6\x06\xc1\x02\x1d\xf7\x8d\xca\x7c\xc7\x7e\xa7\xae\xd1\x7e\xab\x02\xb3\x64\x5a\xb4\xdd\x4a\xa9\xd4\x13\x96\xd7\x67\x66\xca\xe3\xa8\xc3\x83\x05\xf2\x70\x74\x6c\x6e\x76\x1e\xdc\x50\x33\xf4\xaf\xe2\x9b\x48\x4c\x43\xe7\x53\x8c\x6a\x3e\xba\xce\x6e\x2b\xfa\x8f\xa2\x0d\xab\x1e\x02\x14\x04\xbb\x58\xb4\xd8\x9e\xb4\x68\x65\x54\xc4\x9f\x53\x93\x97\xae\x76\x27\x62\x2e\x6e\x30\xd1\xc3\xe9\x68\x7c\x99\x12\xba\x4d\xc2\x92\x6d\x51\xb5\x20\xe6\x99\xf1\x10\x55\x45\x4c\x5f\x85\xb9\xb8\x51\x31\xfc\xa3\x93\x11\x4c\x0f\x1e\x1b\x8d\x7e\x61\xbf\x00\xfe\x36\x0a\x44\xe6\x26\x26\x42\xb1\x90\x76\x1a\x68\x5a\xb2\x5b\x94\x55\xc2\xb8\x0d\xf8\x10\xee\xc3\x51\xb3\x6e\x28\xca\xa3\x1f\x06\x68\x69\xb5\xb1\x81\xb0\x5c\xdc\xf6\x3b\xa6\x3b\x4f\xb1\x2e\xa5\xe9\x23\x88\xaf\x66\xf1\xa8\x51\x0c\x84\x0d\x04\xcf\xea\x8e\x88\x3b\x5e\x43\xb5\xfa\xf0\x81\x08\xd6\x9f\x4a\xe1\xf7\x63\x0b\x2f\x82\x72\x0a\x4b\xfa\x1e\xa6\xd5\xef\xf3\xce\x4b\xcb\xf4\xeb\xaa\xca\x32\x0c\x81\x0c\xbb\x43\x1f\x34\x81\xb0\xb8\xd7\x8e\x84\xf1\x3c\xa3\xae\x44\xd0\x18\xde\xf9\xdf\x15\xa6\xad\xa8\xc0\x72\x0f\x22\x41\x3e\x80\x4d\xe2\x64\xb3\x8e\xe6\x47\xfc\x64\xcf\xcf\x10\xdd\x92\x4e\x48\xf0\xc6\x81\xde\x30\x51\x5b\xdf\x73\xa2\xe3\x65\x8a\xaa\x34\xa3\x68\x26\x1b\xcf\x07\xd0\x4c\x84\xe0\xe0\xd9\x8d\x4e\x5e\x99\x0e\x4c\x59\x4d\x88\x5c\x58\x26\x20\x6b\x45\xe6\x47\x0a\x1b\x12\x24\x82\x7c\xed\xea\xa6\x9c\xaf\x9d\xa0\xb3\xbc\xb2\x1c\x0d\x1d\xf5\xe4\x60\xa7\x4c\x42\x45\x11\x61\xf4\x82\xc5\x59\x76\x8b\x1a\xc5\xb3\xbe\xb5\x04\xe1\x37\xd2\x28\xa0\xc4\x7f\xf2\x2b\x29\xe4\x47\x57\xd3\x60\x67\x4d\x23\xfd\x02\xdd\xfd\x8b\xd7\x0a\xb2\x87\xf0\xe1\xf5\x38\x0b\xe4\x4a\x7e\xd0\x47\xa1\xb1\xdd\x2c\xea\xa6\x18\x2a\x6a\x5f\x69\x48\x2f\x1a\xa1\x9a\x86\x8c\x32\x48\x49\x83\x4f\xac\x42\x72\x4f\x45\x4d\x2f\x09\x16\xea\x8c\xe1\xe8\x52\xab\x27\xe8\x23\x3e\x0d\xc6\xf3\xfb\x8f\xfa\x2c\x11\xe5\x68\x7e\x38\xbe\x1a\xe1\x65\xb6\x3b\xc7\x08\x39\x52\xe3\x13\x3b\xae\x81\xab\x3f\xd3\x61\x4b\x78\x48\xfa\xcd\x0e\x12\x27\xc5\x8f\x4d\x8f\x88\x6b\xf7\xef\x1a\x63\xc8\x55\x08\x31\x0e\x8c\xf9\xfc\x32\xee\x1f\xac\xd8\x86\x05\xb7\x27\x0b\x7e\xd2\x8b\x78\x47\x57\x44\x70\x17\xc6\xbe\xc4\xc5\x74\x19\xe4\x3b\x29\x88\xf7\x01\xf9\xc9\xc5\xd8\x41\x07\x72\x01\x93\x75\xa5\x04\xb2\x29\x01\x21\xf0\x42\x64\xbb\xdb\x22\x4b\xd2\x3d\x72\x02\xac\xeb\xaa\x72\x03\x4d\x9e\x8f\xd9\x12\x8d\x97\xc9\xf4\x53\x61\x11\x76\x6c\xcd\x8b\x45\x54\xf5\x3a\x56\x0d\xb4\x3b\xd2\xa0\x98\x3d\x85\xbf\x02\xc9\x19\xa9\xc3\xf6\xd9\x4c\x21\x63\xd3\x42\xb9\x7e\xb6\x7b\x3f\x37\xd7\xfd\xda\x67\x5a\x3b\x85\xc3\xd1\x4d\xe9\x66\x88\x1d\x13\xf2\x7d\xd9\x8a\x46\x85\xaa\xee\xfb\xea\x2c\x97\xa2\xfe\x8b\xe6\xb3\xc1\xf8\xd2\xb6\x26\x7c\x92\xc5\xad\xc6\x22\xf5\x88\x71\x54\x65\x5e\x53\x4b\xc3\x0e\x17\x81\xd3\xda\xd1\x31\xbe\xba\x1a\x0d\xc7\xce\xe3\xa0\xa0\xe6\xf1\x61\xe9\xc2\x44\x58\x4f\xae\xae\x4d\x68\x6c\xa4\xfa\xe5\xae\xe6\x72\xc5\x7d\xaf\xd5\x0e\x22\xeb\xdc\xc9\x81\x6d\xba\x92\x1d\xa4\x75\x07\x33\xbb\x2b\x3f\x9e\x3a\x93\xae\xaa\xe7\x69\xd0\xfa\xee\x13\xbd\xb5\xc3\xca\xc6\x47\xd3\x19\x98\x62\xef\x60\x88\xb9\x62\x1b\x1b\xeb\x82\xcf\x7e\xb8\xeb\xa1\xd5\x5d\x41\x73\x3e\xb3\xc8\xfc\xc1\xdb\xca\x9a\x75\x87\x4d\x2b\x9d\xe5\x79\x2c\x4a\x4d\xca\x82\xa4\xca\x84\x20\x9f\x58\x63\x9e\x12\x64\x8b\xcc\xd8\xee\x60\x8c\xc4\x8e\xb9\xa5\xc8\xa0\x8f\x28\x40\x73\xf0\x78\xb8\x90\x85\x78\xcd\xe1\x86\x62\x9e\xc4\xb4\x7c\xec\x0e\xaf\x79\xa2\xf1\xa4\xea\x14\x64\xf1\x82\xca\x95\xa3\x9a\x6e\xaa\xc3\x1f\xe8\xf5\xd2\x21\xa9\x69\x01\x2b\x8e\x46\x03\x70\x1c\xac\xa2\xec\xed\xa5\x45\xbe\x5e\x3c\xd1\xb8\x09\xac\x50\xd3\xd7\x72\x18\xec\x09\x89\xba\xc1\x59\x30\x6e\x69\x8c\x9a\x80\x95\x57\xfa\x64\x93\x47\xf2\x8b\x76\x77\x75\x37\x58\x83\x1a\xac\x82\x2c\x21\xfd\x3d\x80\xc5\x22\x14\x1e\xd8\xd0\x1c\x18\x9c\x08\x43\x74\x22\x5d\x67\x68\x98\xbc\x99\xd0\x3d\xb9\xb5\xa9\x0d\xd4\xe7\xec\x2c\x7b\xfd\x4e\x64\x7f\x81\x58\x78\x89\xb5\xe6\x24\x1b\x1b\x3e\x9e\x09\x5d\x81\x9f\xf9\x28\x27\x48\x26\xef\x9e\x23\xc9\x66\xa5\x01\x7a\xd5\x9a\x53\xb6\xfa\x0f\x51\x2f\xb8\x8d\x7a\xa1\x4d\xd1\x03\xd7\x60\x25\x8b\x7d\xe3\xd9\x3f\x43\x11\x97\x7f\x00\x41\x04\x88\xf4\xf4\x94\x39\xbc\x23\x4c\x85\xaf\x01\x30\x31\x33\x1a\xcd\xe6\x2b\x28\xf3\xba\x8c\x7f\xc8\x6c\x5f\x74\x17\x01\x01\x12\x94\xa2\x46\x75\x61\x49\xc0\x7a\xd8\xce\x28\x6e\x30\xeb\x9a\x71\xf5\x28\x51\x7b\x18\xa9\x15\x1d\x2f\x52\x99\xea\x70\xe1\x64\x3a\xda\x65\x7b\x44\x8e\x01\x57\x7b\x48\x43\x12\xea\x0a\x51\x47\xb9\xb0\x26\x97\xf4\x34\x11\xe4\xd1\x52\x89\xd7\xc8\x90\x4c\x49\x46\xe2\x30\x09\xb6\x8e\x1c\x0d\x90\x5e\xbc\x58\x7b\x3d\x8e\xcc\xa8\xac\x94\x74\xfd\xd9\xf5\xc6\x3f\xaa\xc9\xa3\x4f\xa1\xab\x3b\xae\x02\x3a\x6d\xf4\x0d\x49\x2a\xe6\xeb\xcf\x2c\xb9\xba\x76\x5c\xb2\x17\x3f\x5a\x07\xae\xc6\xf1\x90\xf9\xe7\xc9\xab\xed\x50\xa7\x4e\x24\x82\x91\x4e\x0a\x8d\x55\x21\xce\x8b\x47\x4c\xa1\x8b\xee\xfd\x3f\xd1\x9b\xee\xc0\xde\x75\x2b\x1b\xc6\xd7\x74\xb1\xda\xba\x90\xc3\xdd\x7a\xfa\x70\x20\x2b\x1e\x1a\x1f\x4f\x92\xa9\x25\x99\x77\xe0\x59\xa3\xba\x05\x2e\x67\x95\x24\x79\xeb\x75\x92\xce\x7e\x55\x4c\x4d\x67\x6b\xad\xe2\x77\xd2\xac\xc6\xc2\x7c\x8d\xe9\x37\xa5\xc2\x32\x27\x66\x70\x80\xae\xab\xcc\x09\xfd\xb0\x19\x83\x32\x50\x58\x6b\x01\xd3\x1f\x2c\x16\xed\x2d\x48\x6d\x6f\x9d\xf8\x4b\x02\x90\xc8\x0f\x0e\x94\xca\x75\x82\x08\xc8\xc7\x18\x32\xa2\x18\xb8\xcb\x5e\x99\x39\x89\xb3\x4f\x8a\x13\x9f\xcd\xfa\x14\xbc\xfe\x4c\x8e\xd9\xec\x3d\x40\xe9\x4b\xda\xa2\xba\xdc\xf5\x28\xfb\x24\xf3\xf6\x7a\x12\x2d\xdb\xc1\x72\x98\xec\xa4\xc7\xab\xd2\x1a\x17\xeb\x27\xb9\xa7\x5c\xfc\xad\x9e\x64\x9d\x00\xf2\x1d\xa9\x42\x15\xe4\x7a\xd1\x18\x74\x4f\xe0\xef\x49\x0e\xe3\x62\xd0\xb5\x0d\x93\xc3\x4f\xe3\x24\x2c\x8c\x12\x25\x14\x0d\xd2\xbe\x9d\x4e\xd1\xc3\xb1\xc3\xd8\xa8\x29\xc7\x4e\xd2\x54\xa5\x4d\x89\x20\x0b\x4d\xc5\x4f\x1b\x1b\x83\xe6\xf9\xe5\xf4\x64\x80\xed\x2f\x4f\x98\x76\x05\x33\x03\x05\x77\x02\x75\x2f\x98\xc3\x3e\x51\x5e\x42\x35\x98\xe5\xdf\x50\xd3\x6a\xa3\xea\x3a\x85\x2f\xf4\x6d\x25\xed\xc4\xb7\x6e\xa4\x83\xe3\x11\x14\xae\x13\x56\x69\x48\x59\x3d\xb9\x0a\xa1\x6c\x62\x4e\xa2\x8c\x13\xb6\x80\x7e\x49\x71\xa6\x91\x37\x94\xf3\xd9\x0d\x05\x70\x3a\xa1\xb4\x13\x9d\x46\x56\x37\x98\x8c\x96\x37\x2f\xc9\x58\x6e\x3d\x0d\x96\x08\xca\x85\xf5\x81\x7e\xf5\x88\x1b\xbc\x74\xb6\x75\x0e\xf8\x44\x80\x4f\x62\x60\xce\xf0\xdf\x2d\x1f\xca\xc7\x53\x99\x4c\x19\x7e\x3d\xe1\xa1\x4b\xb6\x00\x2d\xe3\x0d\xd6\xb1\x75\x6d\xb7\xd2\x3c\xef\x27\x45\xa1\x86\x25\x72\x20\x2c\x33\xb0\xe2\x80\x14\xa5\x44\xf8\x6a\x91\x06\xd8\x7b\x25\xa4\x6d\x6e\x7a\xc1\x7e\xf4\x5b\x3a\xe8\x61\x62\xdf\x9c\xc8\x6f\x38\x99\xb3\x4e\xd6\xa1\x3b\x7a\x15\xc3\x57\xec\x1b\x96\xbc\x06\x22\x77\x11\x19\x8d\x86\x59\xb7\xd8\x39\xfb\x44\xb2\xa9\x2e\xe4\xc7\x50\x15\xfd\x44\x35\x65\x19\x87\x67\xc7\xf8\x6a\xac\x28\xa3\x3a\xa5\xbe\x13\x5f\x9f\x45\xb8\xce\x5f\x23\xc5\x6c\x83\x61\xe2\xd2\x9a\x78\xd8\xe6\x9e\x5e\xa7\xbe\xb1\xe9\x45\x44\x8b\x0e\x99\x48\x62\x9c\xa8\x80\xa5\xf9\xc5\xe0\x37\xe2\x70\xbd\xbe\x3d\xfd\x15\x83\xca\xa2\x22\xdc\x3a\x4b\x0d\xdb\xd9\xd4\x73\x21\x73\xe6\x43\x6f\xe1\x13\xda\x24\x53\xd6\x07\x3b\x01\x6a\x56\x3b\xf7\xf5\x56\xce\xad\xb1\x2c\x66\xf2\xc4\x29\x7d\xdd\xda\xb1\x9e\xae\xc3\x1b\xb9\x50\xa2\xea\x26\xd3\xe1\x08\x05\x6f\xf9\x32\xb6\x83\x09\xae\xf9\xc6\x52\x3c\xe3\xa9\x8e\xc7\x90\x8f\x92\x1a\xce\xf8\x85\x6a\xa5\x6d\x4d\x11\x6c\x07\x96\x86\xd6\xde\x5e\xb2\xeb\xa3\xe6\x27\x76\x8e\xc1\x23\x57\x01\x0f\xcc\x52\xe1\x61\x8f\xc2\x66\x25\xcc\xed\x92\xbe\x5f\x9e\x27\x34\x35\x8e\x7b\xc9\xb8\xac\xc4\x70\xd6\x34\x54\xe9\x2e\x0a\xf7\x75\x5c\xb2\x91\xf4\x93\x25\x1b\x75\x29\x7b\x05\xbf\x07\x02\x99\x23\xc8\xba\xb3\xe5\xd7\x7d\x27\x26\x58\xe5\x68\x94\xd0\xc9\x74\x62\x0b\x39\x6e\xb9\xd5\xdc\x82\x29\xa5\xb9\x4c\x2d\x62\x5f\xd0\x85\x6e\xc3\x75\x9a\xc9\x7d\x8d\x87\xf0\x2b\xc4\x08\x9b\x2b\xb6\xc7\xeb\x63\x60\x27\xef\x52\x72\x5d\xf7\xfa\x72\x3a\x90\x0d\x1a\xf7\x22\xe4\xac\x96\x45\xb8\x33\xbc\x09\xf3\x3a\xd9\x26\x71\x41\x59\xaa\xee\xea\xff\xf5\xd6\x62\x21\x4f\x6d\x58\x33\x2a\x8b\xdc\xea\xb1\xf5\xee\xa6\xa2\xa1\x0e\x8c\x7e\x0a\x47\x86\xf7\x98\x5d\x0b\xae\x45\x34\x2c\x00\x3d\xb4\xfa\x9a\x42\xe8\x37\xc0\xd1\x44\x30\x58\x78\x32\xc5\xb0\xfc\x20\x80\xd5\xf9\x49\x95\xfb\xe1\x55\x57\x0b\x12\x06\xd1\x2b\x9c\x57\xd9\xdd\x3a\xcb\x66\xda\x9c\x7b\xa8\x49\x8d\x8d\x88\xd3\x2c\x9c\x9e\xde\x5c\xf1\x85\xfd\x34\xb4\xf8\xa5\x2d\xe2\x44\x76\x64\x04\x86\xbb\xc3\x7a\xd3\x3a\x7f\x18\x4e\x80\x72\x56\x37\x9c\x5e\x79\x22\xae\xc6\x13\x03\xb8\xa3\x7e\x80\x14\x72\x65\xf9\x47\x41\xf9\xe3\x09\x9e\xcc\xc1\x4f\x83\x68\x39\xbb\x9c\x02\x99\xf4\x28\xa6\xcb\xd9\x13\x04\xde\x44\x88\xb6\x35\x4a\xe6\x46\x41\xef\x0b\xb7\xa3\xf2\x74\x70\x3d\xca\x6f\x93\x8d\x24\x17\xf7\x84\x26\x79\x81\xcf\xe8\x3f\xd0\x24\xe8\x5e\x8e\xbc\x06\x9a\xc7\xc9\x63\x78\x44\x97\x4d\x98\xfe\x18\xd3\xff\xfd\x8f\xed\xaf\x3b\x30\x9d\x3b\x0e\x61\xf3\x66\xc2\x18\x0b\xbb\xdf\x4d\x7d\x1e\xe7\x64\x8a\x84\x0f\x18\x84\x11\x1d\x9c\x08\x19\xda\x23\x6d\x2f\x69\xc8\x2e\xb8\x82\xc1\xb9\xe6\x40\x6e\xc2\x11\x8c\x6d\xbd\x01\x8a\xb4\x06\x85\x05\xb4\x48\x16\x1c\x1e\x95\xb0\xe0\x02\x26\x4c\x93\x81\x20\xd9\xc2\x6b\x7e\x68\x2f\x07\x1d\xdb\x5b\x3a\xfa\x84\x15\x27\x57\xd4\x67\x9c\x88\x9e\x39\x81\xe8\x86\x80\x38\x6f\x26\x41\x53\x58\x8c\x91\x6f\x68\xbf\x84\xda\x0f\xb1\x80\x3d\xca\xc7\x4b\x3c\xc4\xa6\x94\x8e\xd1\x4e\x38\x60\x00\xd6\xb7\xca\xe7\x70\x71\x85\x9c\x2c\x98\xb5\x1d\x6f\x64\xca\xd9\x3d\x5b\xb0\xdf\xb9\xcf\x54\xdd\x4d\x5e\xc4\x9a\xab\x7b\x1c\x7f\x8c\x63\x96\x45\x6d\xac\xd4\x10\xbc\x23\x8a\x96\x1b\xab\xef\x2b\x8e\xc5\x31\x35\x6e\xf3\x38\xbc\xc1\xb1\x3f\x58\xec\x45\xcb\x91\xd6\x44\x06\x6e\x2d\x1e\xa2\x92\xb4\x77\x25\xf0\x43\x48\x37\xa7\x74\x75\xc7\xa2\xa4\x45\x6f\x30\x6c\xbd\x31\x7c\x89\x76\xc5\xa3\x19\x4c\x7a\x6b\x72\xf6\x46\xf1\x44\x60\x4b\x67\x63\xb1\xbd\x1e\x0f\x8b\x46\xc3\x15\x68\x24\x89\x23\x86\x80\xba\xfc\x03\x10\xf9\x98\xae\x2e\xd8\x10\x3d\x07\xa3\xf9\x1c\xc6\x0a\x4e\x10\x6c\x50\x48\xc9\xb7\x5e\xfc\x25\xed\x1d\x95\x47\x07\xfd\x46\x37\xfb\x0b\xba\xc5\x53\xe1\xc2\x30\xb7\x88\xb2\x65\x5e\x40\xce\x66\x98\xc3\x9f\x31\x99\x72\x50\xaf\xad\xb4\x99\xfd\xb2\x25\xd0\x50\x21\xcd\xf0\xc7\x09\x06\x04\x86\x07\xf8\x03\x4f\x33\x78\x42\x41\xe1\x68\x02\x0f\x28\x43\xfc\xff\xe0\x77\x8e\x09\x37\xdb\xad\xed\xbf\xc2\x0b\xff\x4a\xc2\x37\x92\xf0\x4d\xb2\x14\xbc\xb3\x62\xeb\xe8\x68\xf1\x78\x71\x34\x5b\x1c\x4d\x16\x47\xf3\x05\x17\xe4\x9f\x6f\xb6\xce\x3b\xfe\xd3\x95\x7a\x15\xf9\x25\x86\xc8\x31\xa5\x34\x87\x35\xb2\x42\x86\xdf\xb1\x89\x85\x57\x45\xe0\x0e\xdd\x26\xd7\xb4\x66\xe6\x9d\x33\x01\x45\x8a\x61\xf4\x52\x5b\x4a\x38\x0d\x6c\xa9\xb9\x79\x32\xd9\xe7\x19\x0f\xa1\x9a\xfd\x2e\x30\xdb\x71\x15\x98\xbe\x66\x40\x8b\x47\xc2\xc7\xdc\x1d\x1a\xde\x60\x26\xc7\xc7\xd7\x8d\xe2\x71\xd2\x09\x7c\x6a\x0b\xe9\xb1\xff\x78\xa6\x59\x8f\x07\x63\x6b\x37\xd3\xb3\xb3\x52\xdd\xba\x64\xf4\x0d\x76\xd5\xcd\x0a\x0e\xb1\xff\x67\x38\xcf\x9c\xb8\xdf\x56\xb0\x23\x1c\x17\x0d\x19\x32\x8e\x11\x09\x1d\xb8\x85\x91\x5a\x92\xc7\x8d\xa3\x49\x9a\x1e\x1f\xcf\x8b\xd4\xc2\x03\xd3\x05\xe9\x10\x59\x51\xf7\xf1\x63\xd4\xad\x50\x2a\xc2\x64\x19\x40\x3f\xa6\xb6\x70\x1f\xb2\x12\x99\x82\x89\x30\x02\x22\xc1\x63\x1b\x21\x40\xd2\x81\x41\xdc\xb0\x39\x0d\x18\xd2\xd2\xd2\xb6\x0b\xb8\x21\xd8\x67\x94\xdd\x0b\xd2\x0b\xde\x32\x0a\xb0\xb8\xa1\xd6\x84\x61\x35\x46\x5f\x82\x68\x4b\x29\xfd\x87\xf6\xad\x29\x09\xab\xb7\xcb\xec\x16\xeb\x94\xc2\xc9\x12\x5e\x32\xd7\xcf\x38\x22\x81\x64\x03\x54\x14\x8f\x1f\xc3\xcf\xaf\xb1\x4b\x28\x1a\x2e\x18\x8c\x6d\x12\xc8\x42\x44\x36\x80\xc7\xea\xb5\xc7\x8f\x61\xaf\xd9\xd1\x15\x5a\x06\x7e\x7c\x8d\xc9\x9d\xf9\xec\xf3\x2d\x4f\x21\x9a\x02\x8e\xa7\xd7\x7c\x0d\xea\x1e\x60\x7c\x1e\x27\x42\x6e\xb6\x3c\x25\x57\x62\x23\x0c\x54\x87\x67\x0e\x23\xab\x10\x91\xe1\x4f\x5a\x0c\x9c\xb2\x99\xd3\x3d\x63\x45\x94\x64\x8e\xad\x7b\xeb\x78\xa2\x53\x81\xdb\xa0\x84\xe2\xc5\x52\xd6\x1d\x09\xdb\xf2\xb6\x6a\x27\xbb\xc3\xa8\x58\xf9\x11\xb8\xd6\x87\xbd\x90\xb8\xa8\x6c\xa4\x29\xbc\x18\x54\x56\x2b\xdb\x6e\x7c\x0f\x80\x20\x52\xe7\xed\xa5\xb2\xda\x86\x11\xba\x8f\x09\xbc\x1b\x96\x25\xc5\x61\x47\x96\xc6\x14\xbd\x1e\xb1\xcc\x01\x7b\x69\x68\x65\x5c\x9f\xe0\x67\x86\x46\xaa\xe4\x54\xbb\xbc\x18\x9f\x21\x93\x45\xf5\x39\xfe\x5c\xe3\x74\x25\xc9\x84\x73\x2a\x1b\x35\x3e\x27\x44\xe9\x40\x39\x9f\x7a\xe8\xd2\x89\xbb\x87\x60\xe1\xf4\xee\x29\x10\xa3\x35\x23\xc7\x4b\xb4\x7e\xe1\x14\x1a\xd7\x8b\x09\x45\xd5\xb0\x34\xf2\xfb\x62\xd1\x5a\xd7\x1b\xf2\xc5\x42\x02\xaa\x4c\xe9\x5e\x66\x75\xd5\xc5\xb6\xfa\x37\x14\x67\xa4\x5d\x58\xbf\x0d\x5f\x8c\x33\x00\x5b\x80\x90\xff\xeb\x3e\x3b\x38\x1f\x51\x74\x04\x5f\x1c\xb4\x89\xfa\x76\x91\xbb\x44\x35\xac\xea\x31\xb7\xd4\xdd\x79\xb5\x22\x35\x32\xf0\x70\x96\x37\x2b\xd1\x75\xbf\x08\x58\xea\x45\xe1\xc3\x13\xa8\x44\x26\x15\xdf\xcb\x59\xfe\x5c\x0f\x86\x87\xd3\x8f\x23\x67\xed\x06\xb2\x47\xf5\x7c\xd2\x4f\x38\xbc\xf0\xf6\xf3\x68\x36\x7d\x3d\x06\xf4\x24\xf8\xf8\x96\x18\x64\x52\x5e\x85\x9e\x9c\xce\x86\x18\x21\x72\xfc\xe7\x48\xd5\x02\x4b\xc5\x78\x3a\xbc\xab\x1a\xfa\x1e\xf4\x77\x99\x66\x4d\x41\x51\x53\x9b\xe0\x51\xd5\xbd\x15\x17\x99\x68\xcf\xe7\x12\xf7\xa7\xe4\x93\x12\x06\xc8\xd9\x18\x84\x6e\x58\x14\x4f\x3f\xbe\x87\x79\x03\xdb\xa0\x4f\x36\xd5\x48\xd7\x10\x6a\x49\xf3\x18\xf6\x6e\x66\x03\x66\x3e\xf2\x60\x0d\x4c\x66\x57\xf4\x6d\xc3\x37\x78\x01\xa3\x70\xef\xac\x60\xf9\x11\xf6\xfa\xa5\x2f\x62\x3e\x8f\xd0\xb1\x7f\x54\xa8\x89\xa9\x68\x40\x79\x05\x7b\xbe\x8b\x6a\x36\x25\x63\xfe\xa7\xd1\xe8\x63\x35\x1b\x53\x31\x77\x38\xf8\x5c\xcd\x1c\xa2\x32\xa9\x65\x2e\x80\x8b\x55\x33\x31\x95\xea\x1d\x4f\x30\xee\x69\xb5\x62\x4a\x27\xd3\xce\x11\x34\xc8\xb0\x0a\xc1\xe9\x8c\xe3\xf2\x72\xbc\x0a\x4c\x65\x02\x6c\x87\x87\x2a\xdd\x58\x29\x6c\xdb\x18\x4e\x0c\xd0\x34\xf4\x5b\xa3\x3d\x7a\xfa\xc4\x3e\x3f\x1f\x3d\x7b\x22\x54\x37\x9e\x3e\x1f\x7d\xf5\x84\x3e\x50\x70\x50\x43\x34\xf0\x6f\xe3\xeb\x27\xd4\x66\x16\x39\xb7\x6f\x83\x7f\x1b\xed\xed\x27\xd4\x1f\xae\xd8\x7c\x80\x47\x65\xfc\x76\x72\x73\x02\xab\x81\x0e\x24\x20\xc3\x83\xf4\x94\xce\x00\x97\x95\xa4\x27\x91\x5f\xc3\x74\x4c\x21\xb7\x71\x57\x7f\x42\x51\x60\xac\xf2\x3c\x86\x4b\xac\x96\x25\xa1\x02\xce\x7f\x62\x71\xe2\x1e\x33\x53\x2d\x45\x3c\xe0\xfd\x99\x14\x92\x37\x28\x23\x4f\x99\x19\x78\xb2\x07\x27\xe5\x07\xb4\xeb\x4c\xd9\x38\xdc\xcd\xaf\xd6\x0e\x27\x74\xbd\x01\xae\x40\xe4\x4a\x43\x20\x49\x1e\x5d\x30\xfb\x39\x57\x3c\x75\xbc\x8d\xdc\xd7\xf1\x05\xa0\x82\x61\x70\x2b\x23\xd7\x84\xc4\x80\x58\x17\xea\x64\x02\x9c\xb4\x92\x06\x3f\xda\x26\xe3\x37\xf5\x39\xc3\xe1\xfb\xd9\xc1\xcd\xc9\x7c\x36\x38\x9d\xdb\x49\xf9\x7a\x36\xbd\x92\x29\x7e\x35\xbd\x32\x6e\x4c\x8d\xcb\x5d\x76\xd2\xcb\xc7\x26\x1c\xf0\xff\xbd\x48\xb8\x7c\xbf\x91\xc6\x10\xcf\x8f\x70\x0c\x5b\x24\xe1\x98\xe4\x59\xe6\xf3\xf0\xd5\xce\x5b\x55\x82\x12\x3a\xba\x20\x7a\x37\xbc\x02\xf8\xa6\x98\xff\xa4\x0d\x7e\x0f\xc6\xf8\x13\x4b\x70\x66\x52\xc4\xbc\x58\x30\x26\x0a\x4a\x22\x93\x14\x8b\xf1\x73\x9a\xc9\xbc\xc6\x24\x7c\xc2\x10\xab\x58\x8c\xeb\xa2\xd0\x98\xee\x21\xa3\x09\xa1\x2a\x60\xcc\x0c\x4a\xcf\xa9\x7f\xca\x64\x8a\x68\xf0\xe0\x5b\xd6\xab\x0d\x4a\x98\xd0\x0d\xf0\x8d\x4a\x44\x9c\x75\xdf\xe2\xbf\x41\x3e\x0b\xe9\x91\x8f\xa0\x6f\xd2\x21\xc3\xad\x2e\x9f\xef\xbb\xad\x54\x86\x57\x3c\xa0\x36\x43\x15\x26\x17\xd7\x68\x51\x8f\xbd\x02\x2b\x9f\x00\x7e\x39\x52\xba\x36\x38\x1b\x11\x51\x25\x9b\xde\xb7\xf9\x6a\xc5\xb6\x19\x4e\x31\xd0\x1e\x29\xbb\x6c\x44\x2b\x98\x0b\x85\xf5\x2a\x21\xd0\x76\x0f\xc1\x85\x5c\x0c\x19\xfe\xdd\x1b\x9f\x9d\x71\x01\x98\xd6\x61\x81\xcd\xa8\x00\xde\x3c\x42\xd3\x4c\x1b\x04\x0e\xd2\x45\xe3\x98\x2a\x42\xe4\xda\x42\x1b\xb8\xd4\xba\xdc\x7f\xd8\x86\xe7\xc5\x62\x3d\x00\x9a\x4f\xdf\xc0\xdc\x72\xa0\xd9\x3a\xea\x87\x5d\x12\x96\xc0\x90\xea\x54\x65\xa3\x61\xa7\x2e\xbf\x7a\xc2\x55\x34\x6a\xbb\x3e\xf0\xfa\x78\x83\x7f\x49\xe3\x24\x4f\xec\x10\xe1\xd3\x68\x36\x1a\x16\x94\x04\xad\xfe\x16\xdf\xf1\xe0\x33\x55\xc1\x4a\xd2\x66\x56\x3e\xda\xe2\x88\x50\x1d\x82\x24\xf8\xdd\xcb\x31\x00\x96\x3d\x4a\x81\x8f\x39\x05\x49\xee\xd2\x6e\x5c\xca\x9e\xa0\x86\x0c\x79\xb2\x07\x57\x04\x5f\x43\xa6\x5e\xcc\xa9\xc7\x39\x21\x16\x01\x90\x41\x93\x06\xcd\xc4\x62\x81\x5c\x46\xb2\x46\x0a\x1a\x87\x7a\x8e\xf9\x3c\x82\xe3\x74\x09\x6b\x28\xa2\xd6\x23\x28\x13\xc2\x04\x85\x88\x84\x5e\x08\xd0\x2f\x54\xbd\x62\x07\xe1\xad\x70\xc2\xa2\xbe\x41\xd0\x47\xfa\xdb\x71\x39\x4f\xcf\xc6\xa3\xcb\xa1\xb5\x30\x01\x79\x8f\xb6\xb6\xa3\x99\xb3\x44\xc7\x6c\x1f\xa8\x08\xd7\x60\xd8\x73\x13\x64\xf1\xb5\xc0\x16\x09\x70\x85\x84\x23\xc7\x63\xb1\xf5\x4a\x31\xe2\x17\x89\x3d\x72\xeb\x70\xf1\xf6\xb6\x2b\xcf\xf9\x4b\x66\x3b\x3d\x2a\xac\xc4\xf5\x33\xfc\x88\xb9\x76\x6c\x30\x36\xe7\x54\x52\xe4\xf5\x42\xf8\xd5\x19\x30\x6c\x94\x33\x05\x83\xba\x61\xd9\x51\x77\xaa\xdc\x39\x0f\xe3\x45\xb7\x88\x7c\xa5\x9f\xab\xe1\x9f\xc2\x8b\xd7\x52\x97\xba\x00\x2c\x2b\x8e\x54\x0b\xa3\xf8\x66\x7e\x0a\x7f\x91\x4b\xf2\x17\x29\x5d\xbf\xec\x28\x88\xc5\x44\x64\x9a\x2b\xa9\x8c\x42\xdf\x2f\x0d\xdf\x4b\xe2\xcf\x94\xd2\x5c\xb5\x84\x73\x72\x0c\x80\xda\x2f\xbc\x82\xc7\x47\xe0\x16\x3c\xab\xdc\xbb\xf3\x3d\x6f\x67\x3b\xef\x66\x5e\x4f\x67\x2f\xa7\xa3\xd9\xa9\x13\x83\x4f\xf1\x6d\x34\xe4\x43\xa9\xa2\x51\x03\xc7\xfb\x96\xc2\x79\x82\xc3\xfe\x0e\x4a\x6d\x6c\xb8\xb3\xb1\x20\x1d\x27\x00\x97\x0d\x92\x77\x8a\x56\x57\xc9\x29\x61\x91\xdc\xcb\x34\x61\x46\xc6\x64\xf8\xef\xc2\x95\xe9\xcd\x64\x9f\x56\x3d\x14\x06\x79\x59\xf4\x6a\x79\x31\x2f\x4e\xf1\x4f\xf3\xc7\xc3\x97\x0a\xa8\xd1\xc6\x5b\xe8\x4d\x68\x3d\x48\xdf\xe3\x38\xd4\x11\xde\xbf\x03\x30\x95\x50\xe7\x7f\x6f\x47\x83\x6b\x9f\xde\x7d\xfa\xfc\x79\xfe\xf4\xf9\x57\x7a\x59\x0a\x21\x6c\x49\x7c\xf9\xcb\x33\x98\x5c\xad\x8d\x0d\x7a\x6e\xb7\xb0\x0d\x5b\x8b\x05\xe7\xb4\x70\xe2\xb5\xd4\x4a\x14\xec\x71\xae\x64\xd3\x2e\xef\x1d\x10\x3d\x06\x1b\x1b\x9b\xdb\x50\x06\x1e\xaf\xcf\x9a\x36\x07\xcd\xa4\xe4\x11\x73\x06\xbd\xfd\xf7\xef\x0e\xbf\xef\xbf\x80\x7a\xd4\xeb\x4e\xbb\xdd\xa5\xa7\x9c\x12\xf7\x76\x0f\x5f\xf5\x5f\xb4\x05\x84\xde\x76\x74\xcb\x52\xf2\xdf\x5f\xed\x7e\xe8\x1b\x85\x24\xeb\x22\x24\x63\xf8\xfe\xfd\x8f\x1f\x7c\x25\xf4\xb6\xb3\xfd\xb4\x8b\x0f\x0c\xb0\xff\xe6\xdd\x8f\x58\x89\xa3\x83\xdf\x77\xbe\xfa\xa6\xcb\x8f\x0c\x76\xf0\xea\xe5\xfb\x77\x7b\x1e\x4c\xde\x11\x8c\x1f\x2d\xb6\xb7\x6f\xdf\xc4\xb0\x3a\x71\xe7\x9b\x6f\x10\xb1\x4b\xc8\x37\xdb\x86\x1b\xea\xd8\x36\xcf\xde\xe0\xf3\xfb\x33\xec\x27\x68\x32\xfc\xb6\x1d\x9b\xb1\x58\xd8\xa7\x1d\xfc\xc0\x4c\x37\x29\x25\x98\xb0\xc9\x0b\xfb\xa0\x85\x13\xbe\xc7\x0d\x3c\x18\xa3\x11\x81\xb8\xf1\xfa\x72\x70\x5e\xfa\x0d\x2c\xfd\x22\x96\xe2\x96\x5c\x06\xe7\xeb\x6d\x73\x33\xb9\x29\x47\xbc\x95\x2f\xf3\x5e\x5f\xde\x89\x85\xe3\xab\xad\x25\xdf\xdc\xe6\xf0\x4b\x6f\x41\x94\xc7\x01\x92\xb7\x88\x89\x30\x20\xe0\x19\xb3\x25\x30\x75\x5d\xce\x66\xf1\x9c\xf2\x9a\xf8\x0e\x55\x55\x8e\x66\x62\x30\x8c\x11\x97\x21\x69\xa9\x47\xf0\xdf\x30\x23\xbd\x8a\x4f\x24\x61\xcf\xc7\x59\x28\x02\xba\x97\x42\x0e\xc1\x49\x3a\x3e\x17\xe9\x18\xc5\x89\xb0\x95\x5e\x60\xb4\x2c\x4e\xa2\x2f\x76\x6f\x9a\x5c\x97\xe8\x3e\x28\x06\x7b\x2d\xec\x5b\x52\xa3\x0f\xc1\x9e\xe1\xa8\xbb\x21\x89\x9a\xf4\x96\x9b\x33\x41\x2b\xea\x0c\xdd\x13\x56\x20\xa3\x5e\x17\x24\x35\x22\x86\x53\x5b\xe8\x53\x6f\x78\x46\xa7\xb4\x2b\xa4\x1f\xd4\xb3\x26\x9b\x49\x96\x03\x88\xda\xa0\x4d\x07\x43\x44\xe6\x43\x1a\x97\xe1\x31\x76\xd9\xc4\x08\x27\x68\xf4\xeb\xc2\x89\xb0\x0b\xbd\xc5\x22\x0d\x13\x48\xdf\x6b\x09\xcb\x22\x68\x5a\xb8\x04\x7f\x94\xe5\x89\xb9\x99\x68\x72\x5c\x84\xe4\x55\xe0\xb8\x21\x84\x2c\x8a\x5a\x31\xf6\x66\xb0\xc8\xb9\x7e\xe5\x00\x28\x13\x3c\xe3\xa1\x10\xc7\x64\xee\x8c\xf1\x2b\xbc\xc9\x2c\x5f\x6b\x54\xc8\xfb\x18\xbd\xb0\xe4\x08\xfc\x19\xeb\xb1\x7f\xbb\x19\xcf\xa0\xe9\x9a\x5b\x08\xb6\x95\x34\x3e\x2a\x1d\xb5\x73\x51\xa3\x30\x2c\xe9\x2a\x26\x92\x61\x97\xe6\x70\x39\x66\xdf\x6f\xb2\x43\x21\x0b\x21\x24\x02\x73\x0a\x20\x2e\x55\x36\x43\x84\xb6\x83\x37\xd8\xe4\xab\xd1\x34\x6c\xfc\x02\xfb\x57\x0c\x90\x78\xef\x4c\x5f\x57\xd4\x0e\x0c\x94\xb3\x6d\x80\x67\xe8\x78\xf3\x6b\x41\x2f\x76\xcb\x80\x8d\xb3\xaa\x60\xa3\x8d\x1e\xa1\x09\x00\xfe\x74\x27\x74\x9e\xe3\x30\xd1\x34\xef\xfc\xba\xd3\xea\x84\xe4\x33\x7a\xeb\xb7\xec\x57\x7b\x16\x0e\x25\xc2\xaf\xc2\xc3\x62\x72\x16\x4d\x78\xc5\xe2\xbc\xf8\x75\x63\x23\xdc\x0f\x71\xc7\x51\x1f\xae\xb7\x32\x00\xd8\x6c\x8b\x1d\xd9\xaf\x9b\x9b\x4b\x10\x4c\x96\xf5\x8d\xec\x47\x08\xc7\xdd\xe6\x49\x8c\x8a\x7b\x9e\x63\xe1\x26\x8e\xc5\x6b\x89\x62\xb2\x75\xd4\xe3\xc3\xcc\xad\xac\xcb\xe9\x6e\xf3\xf0\xcb\x51\x6f\x71\xd4\x7f\x84\xf1\x0e\xa1\x05\xa2\xcc\xa3\x23\x4e\x0f\x85\x5f\xae\xf8\x75\x28\x63\xaa\xcd\x9c\xdb\xb8\x89\x5c\x28\x44\x9c\x45\xf4\x3a\xa1\xac\xf6\x5a\x7f\x60\xb0\xef\x6e\x6f\x33\x0e\x2a\xef\x77\x32\x36\xb7\xdf\xbd\x3b\x3b\x5f\xd1\x6e\x16\xa0\x7a\x99\x15\x37\xee\xb7\x4a\x8d\x93\x24\x7a\x27\xe9\xe9\xe3\xec\x46\x61\x31\x29\x2b\x4c\x4b\x47\xd7\xe6\x39\x79\x56\x24\x58\xb9\xe0\xae\xae\x1e\x88\xb6\xc7\x37\x3a\x03\x5a\xdd\x8e\x2d\x67\x3b\xfa\xaa\x69\xd7\x9a\xac\x2b\x8d\x5c\x8c\xfe\xb8\x1e\x4c\x84\xcd\xdb\x4d\xc0\x95\x28\x92\x33\x41\xe0\x5b\x88\xdf\x91\xf5\xad\xc8\x29\x56\x77\xfb\x4a\x6c\xb0\xf2\x65\xb9\xad\xd3\x2e\x3c\xb1\x84\x58\x47\x27\x96\x50\x8a\x79\x19\x8c\x6f\xa7\xc0\x12\xa1\xf4\x6b\xdf\xc5\xf1\x88\xc7\x82\xcd\xcb\x00\x4e\x20\x16\x0b\xfa\xe5\xa8\xd3\xc5\x57\xd4\x89\x97\x53\xe8\x88\x78\x2c\x78\x37\x03\x78\xe1\x89\xc2\x18\xd7\xc3\x51\xa8\x5c\x69\x82\x4e\x26\xad\x2e\xc3\xdd\x4e\x9f\xda\x92\xe6\x8e\xef\x41\x8f\x64\xf7\x10\x65\xc6\x9b\x85\xbf\xcb\x46\x80\xc1\xca\x41\x61\x1b\xc9\x3a\x06\xb0\xf0\x51\xc3\x1c\xff\x5a\x5d\x3e\x1b\x7c\x58\x0b\x73\xca\x12\xeb\xf2\x3d\xf8\x9f\xb3\xd9\x26\x5b\x3c\x2a\x7e\x78\x31\x1b\x8d\xf6\xc6\xe7\xb0\xc9\x66\x33\xef\xbf\xc3\xff\xc4\x2a\xfc\x3b\xf8\x9f\x3c\x9e\xc3\xff\x6a\x4a\xbf\x9e\xde\xcc\xe2\xc2\xba\xb4\x2e\x5e\x57\xfe\x60\xfc\x87\x2e\x7e\x20\xe0\x07\xfe\xc1\x3e\xd5\x53\xff\x7e\x02\xbf\x95\x4f\xd8\xdf\xdf\x97\x52\xfb\xfe\x71\x38\x74\x0f\xea\x69\x58\x83\xf4\xa7\xe9\x6c\xc8\x88\x06\x02\xb8\xeb\xa0\xaa\x8b\xb7\x95\x52\x2f\xb3\xe6\xf1\xd5\x68\x36\x1e\x8e\x47\x57\xd4\x4b\x8c\xe2\xbf\xea\xda\xdc\x5e\xe3\xdc\x17\x82\x7f\x96\x7a\x7e\xfe\x79\x05\xf4\x9f\xd3\x89\xe0\x3b\xac\x83\xb0\x9f\xed\xda\x4a\x1e\x54\x5f\xb8\x8e\x90\x87\xef\xbf\x97\x87\x8b\x0b\x79\xb8\xba\x92\x87\xb2\xb4\xad\x67\xf1\xd9\xf6\xb2\x65\x6d\x51\x5b\xd2\x16\x94\xdf\x4f\xf6\xd7\x3e\xfc\x64\x7f\xed\xc3\x48\x7e\x5f\xd5\xf7\xe9\xfb\xd9\xe1\xa7\xa9\xf4\xa8\x98\xb1\xf8\xcb\x10\xca\x44\x65\x86\x13\xe1\xfa\x15\x1b\x4b\x58\xdb\x34\xe1\x0c\x34\xf6\xbd\x18\x49\x36\x3b\x18\x1b\x0b\x63\xb6\xeb\xa3\xb4\xb9\x34\xef\x3e\xeb\x68\x51\xcb\x2e\x56\xd4\xd6\x7a\x8d\xce\xe7\xff\x3c\xbd\xb8\x99\x7c\x2c\xd2\x6a\xbf\x34\x47\x7f\x8c\x4e\x2d\xf0\x62\xd1\xeb\x67\x78\x8e\x8b\xfe\x16\xca\x22\x95\x72\x0d\xa8\x5a\x96\x49\x46\x20\x65\x5f\x62\x26\x6c\xd9\xa1\x18\xc8\x1b\xa6\x65\x30\xe4\x98\xe8\xc0\x1b\xe9\xf3\xd6\x13\x42\xd3\x03\xb1\xa6\xc1\xca\x09\x7e\xdf\x76\x6b\x59\xd2\x40\xfd\x2d\xa7\xb6\xfa\xdd\x4d\x29\x9c\xcb\x6f\x70\xa4\x80\x95\x1e\xb2\x47\x42\xfc\x4a\xcd\x3d\xd8\xb1\x5a\xc0\x43\xd0\xc4\x00\xf7\x65\x73\x36\xdd\xb6\xc3\xbc\x96\xb7\xf8\xe9\x96\xe4\x56\x55\x43\xdb\x92\x34\x40\x22\x5b\x62\xd1\xaa\x32\xeb\x06\xe1\xa8\xc3\xd2\x51\xed\xd4\x1d\x14\x77\xce\x38\xd6\xb6\xd3\x74\x13\x7c\xa2\x2a\x1a\x74\x6b\x6b\x1e\xe4\x7e\x53\x19\xec\xa9\x98\x60\x4d\xc9\x9e\x9a\x4c\x77\x7d\x13\x29\x02\x82\x4f\x0a\x3e\x68\x6f\xcf\x23\xaa\xa2\xb2\xd4\x0c\xed\x36\x7b\x35\x22\x9c\xcb\x61\xcd\xa4\x6b\x08\x0a\x34\x52\xfd\xb6\xf3\xfc\xaf\xdd\xf6\x37\xad\x56\xbe\x3d\x7a\x1a\xa1\x72\x8c\x41\x18\xf6\x7d\x88\x75\x69\xc5\x14\x2d\xf9\xe3\xf2\x87\xab\xbb\x3b\x0a\x20\xf6\x6b\x70\x7d\x5f\xe1\x46\x8a\x2b\x85\x44\x91\xba\x64\x25\x51\x57\x8a\x89\x45\x3d\xcf\x3a\x94\x95\x25\x4b\xc5\xf5\xc2\x92\xa2\x25\x59\x59\xb2\x7e\xb5\x8a\x6b\xf7\xea\x16\x41\x84\x47\xbc\x69\xd2\x6a\xa2\xb9\x56\xa5\x97\xff\xcb\x37\xea\xd0\xdd\xfb\xa7\x22\xca\x7c\xbe\xa6\x9c\x5e\x41\x2c\x02\xd8\x9e\xff\x78\xf8\x12\x8d\x2a\x6c\xca\xfc\xcf\xab\x62\x35\xbf\xab\x7e\xe1\xfd\x8c\x7c\x78\xdf\xea\x1a\xb3\x7b\x62\x1b\x05\x33\x67\x17\x15\xb1\x2d\xf7\xda\xdc\xea\xc4\xe2\x42\xb8\x74\x29\x71\x42\x44\x8b\x5a\x5c\x18\x85\x20\x9a\x5d\x9f\x1c\xf7\xfa\x84\x96\x67\xae\x39\x3e\xf5\xa8\xac\x1c\x13\xe8\x45\x01\x3b\x11\x5b\xe6\x25\x41\xa6\x9a\x31\x8e\x89\x31\x9a\xd3\x9b\xd9\x0c\xc4\x75\x72\xe2\x81\xba\xc9\xc3\xe9\x8f\x25\x06\xe6\xfd\x03\x67\xb1\xf9\x44\x76\x54\xbc\x9b\xc7\x83\x01\x98\xdf\xf4\xcb\xa4\x89\xd6\x7d\xdd\xf5\x34\xef\x89\x15\xca\x42\x3d\xf3\xae\xdb\xda\x6f\x38\xda\x61\x1b\x4a\x0a\x27\xc7\x98\x99\x13\x55\x93\x99\xf5\x41\x73\x08\x71\x81\xf9\xbf\xd6\x9a\xa0\xe5\xb2\x3d\xb8\x7e\xda\xa5\xf1\x86\xe3\x15\x52\x4d\xbb\x45\xdc\x24\x69\x7f\x43\x31\xf7\xf3\x64\xbb\xc5\x0f\xf8\x2f\xae\x90\xd8\x47\xd7\x9d\x00\xe0\x77\x93\xa6\x37\xcb\x23\x98\xa5\x51\x3d\x23\x8c\xfb\x53\xf3\xbb\xef\xf8\xee\x26\x3e\xff\xe4\x1f\x5f\x75\xb1\x49\x0b\xc7\x29\xb1\x7b\x7e\x42\xbb\x07\xfb\x5d\x29\x16\xcd\x0c\x15\x6a\xc3\xcf\x2b\xf3\x0c\x06\x56\xee\xf6\xf9\x2b\x99\x93\xed\xa1\xc2\x56\x35\xe4\x8f\xff\x89\x93\x01\xef\x90\x37\x2d\xb9\x05\x18\xf9\xd6\x81\x17\x6c\x9d\x06\x6d\x4e\x8e\x11\x53\x73\x38\xfd\x94\xb3\x1d\x4b\xa1\xc0\x3e\x21\x18\x52\xe6\x6a\xd9\xd8\x90\x8a\x5f\x84\x85\x39\xbd\xd1\x30\xf7\x7f\xf0\xf9\x39\x53\xef\x06\x59\x80\x29\x7a\xfd\x94\xf9\xe1\x33\xb0\x4c\x1e\xaa\x20\xd3\x1c\x53\xb3\x14\x61\x9e\x7b\xcd\xaa\x10\x30\xa6\xdc\xd0\x2f\xea\x47\x81\x1a\xc7\x9c\x12\x8f\x81\x2a\xd6\x9d\xe8\x58\x81\xd0\x67\x6a\x3a\xd7\xea\xa7\x31\xce\x14\xcd\x4c\xda\xcb\xda\xc3\x0a\x3f\x33\x5b\xd5\x8a\x74\x6b\x88\x7c\x80\x08\xe4\xb0\x83\x95\xfa\x1a\x86\x57\x7a\x05\xc2\xbb\x5d\xbe\xe9\xf2\x74\x67\x5c\x9d\x7a\xb0\xe7\x6f\x34\xc6\x99\x4e\x60\x5e\x83\x0f\xba\x69\xc6\xec\x57\xa0\xf3\xb5\x28\x1c\xea\x4b\x54\xd1\x77\xf1\x6c\x63\xdc\x6d\xe7\xad\x3c\xa8\x95\x8b\xd0\x92\xd9\x90\x65\x27\x55\x8b\xc0\x62\xd1\xca\xb6\x9e\xb7\x84\x53\xda\x25\x72\x15\xe4\x5f\x10\xd2\xaf\x4a\x69\xb8\xbe\x74\x55\x7b\xe7\xf8\x8c\x0f\x99\xf6\xfa\xb4\x92\xbb\xca\xfd\x3c\xcd\x5d\xa3\x53\xd9\x8e\xab\x76\xb1\x88\x0f\x7b\xeb\xcd\xd0\x9c\x48\xa2\x7a\xae\xe8\xd5\x19\xa3\xc5\xe7\xd7\x2c\x4c\x56\x52\x71\x56\xd5\xd9\x94\x55\x8b\xd3\x82\x5a\x49\x66\x73\x92\x1a\x68\x67\x6a\xd2\x37\xf5\xcb\x8d\xb6\xb8\x58\xb1\x1c\x54\xfd\x3f\x79\x77\x4d\xba\x8f\xa0\x01\x3e\xc9\xa0\x7d\x0d\x5d\xc2\xec\xd8\xf8\x44\x3b\xd8\x7d\x0a\x8f\xed\x7e\x6e\x4b\x56\x8b\x45\x65\xa4\x40\xa8\x4a\xdc\x93\x0f\x63\x09\x63\xd7\xa9\x82\xe2\x23\x9e\x01\xba\xfa\x55\xf3\x9b\x0e\x3f\xf0\x86\x32\xaf\xb8\xc4\x46\xb9\xe1\xcc\x9c\xd5\x2c\xbc\x6f\x29\x3f\x8e\xc9\x02\xf5\x7e\x1e\xcf\xfb\xb3\x22\x49\x1a\x6e\x84\x48\x1a\x5b\x67\xc9\xb5\x24\xab\x65\x9e\x4f\xe7\x83\xcb\x1f\x7c\xb5\x02\xc4\x96\x26\x4c\x42\xa8\x84\xb3\x58\xcf\x78\xa9\x58\xa5\x13\xc5\xfd\x21\xf1\x8b\xf1\x8b\xb9\x3e\x39\xa1\x59\xaf\xe4\x19\xf2\xa0\xa2\x3e\xbb\x48\x1f\xa0\xfc\x59\xb9\x13\xb5\x58\xd0\xe6\x9f\x5b\xcc\x7e\xae\x13\x9b\xe4\xdd\xda\x25\xa8\x42\x18\x23\x9f\x0b\x59\x9d\x77\x0b\x55\xde\xae\xaf\xd4\x09\x1c\x1f\xb5\x0b\xb4\x6b\x73\x5b\x15\x29\xd8\xef\xa8\xa7\xa1\x5e\x9c\x85\x4f\x7d\x47\x34\x8a\x3a\xd8\x5a\xf5\x30\x8b\x78\x5d\x5d\x53\xb7\x3a\xd2\xda\x79\xe5\x8b\x44\x39\x47\x9f\xc4\x9b\x5d\x73\xf7\x26\x5a\x8f\x52\x3b\x85\x1d\x5a\x7b\xe0\xb6\x1e\x74\xc7\x03\x2a\xed\x28\x98\xe0\x2c\xae\xd0\xc3\x77\xb3\xbe\x9d\x4c\x30\xaa\xef\xed\x38\xb1\xc5\xd4\xdb\x3a\x5f\xc2\x9e\x5f\xb7\xb7\xd5\x5a\x3c\xb0\x8b\x4c\x7b\x3b\x2c\x47\x97\x6e\x37\x36\xda\xb8\x40\x45\xc0\xd5\xe2\x85\xac\xdd\x55\x1e\x68\x6a\xad\x98\xf5\xd9\x5b\xa0\xf4\xf1\xa7\x80\xa5\x3e\xe5\x48\x8f\x7a\xd9\x02\x7f\xfa\xf0\xd3\x4b\x7b\xbf\x1c\xf5\x8f\x7a\xfd\x27\xd9\x51\x1f\x53\x9b\xd9\xd6\x79\x74\xa5\x07\x58\xca\x75\xdb\x5c\x6f\x9b\xeb\xa7\xe6\xfa\x99\x43\x7a\xdd\x5e\x2c\xae\xb7\xe1\xdf\x53\xf8\xf7\x6c\x99\xe9\x23\x1b\xa5\x89\xaa\xa5\xa2\xb7\x79\x04\x94\xfc\xf2\xe8\x49\xa3\xdb\x4c\xb3\x45\xef\xa8\x7f\xbb\xec\xe3\xf1\xcb\xd1\xd1\xa3\x8d\xf8\x08\xa6\xc2\x37\xab\x5c\x1f\xa5\x33\x6e\x2b\x73\x32\x2a\xe5\xf4\xc0\xd0\x8d\xdf\xc3\xe9\xb7\x23\xb4\x0a\xb2\x9b\x94\x03\x4c\xb4\x66\x4b\x8e\x51\xd9\x69\x13\xad\x18\xf1\x29\xb2\xde\x3f\xaa\x2d\x69\x6b\x0b\xfa\x4c\xee\x49\xda\x33\x93\xf1\x8b\x18\x39\x0b\x33\x8a\x8a\xa2\x65\x3c\xe1\x85\x77\x21\x63\xbb\x7b\x85\x49\x80\x2f\x93\xa9\xf2\x50\x8f\xff\x1c\x64\x98\x77\x2d\x39\x1a\x85\x3d\x50\x51\x69\x38\x28\x15\x9d\x8d\x42\x57\x13\x4f\x3d\x13\x82\xb6\x5b\x4f\x22\xe8\x9a\xb3\x71\x13\x81\x70\x18\x7c\x8d\xc8\xc8\x5d\x54\xd5\x87\x8b\x85\x7a\xd9\xd1\xc0\xe4\x14\xc3\xe7\x85\x88\xfc\x80\x50\x9f\x91\x65\x1d\x69\xef\xd3\x78\xd8\x2c\x16\x0a\xec\xae\x91\x18\xed\x84\x85\xc1\xfb\x15\x95\x26\x4f\x31\x2e\xa7\xb4\x4a\x05\xeb\x11\x8e\x3f\xb9\x2e\xc7\xe3\xe5\x59\x67\xbc\xc3\x5e\xe7\x6c\x0e\x9e\xd1\x39\x91\xe0\x0c\xd1\x60\xed\xb0\x14\x6e\xb6\xfb\x0d\x01\x79\xde\x5f\x2c\x92\xb5\x44\xd4\x14\x4b\x3b\xf4\x9e\xa9\x88\xdd\xe5\x94\x14\xec\xec\xaa\x3c\xa0\x41\x61\x6f\x14\x1a\xae\xd5\x17\x7c\xf7\xa8\x77\x15\xeb\x02\x0c\xc9\xda\xcf\x49\x76\xe7\xa0\xb3\xfc\x8a\x42\x91\xd7\xcc\x22\x41\x5b\xdf\xe4\xc4\x99\xc3\x16\x27\x51\x20\x6a\x6e\x58\xcc\x07\xe5\xf5\xbb\xd1\xfc\x3f\xcb\xe9\x44\xb5\xbb\xa8\x73\xb8\x4c\xe1\xad\xf6\xba\x55\x3a\x72\x41\x54\x93\x95\x36\x24\x0f\x95\xd0\x79\xc2\x04\x7b\x43\x41\xc2\xde\x5d\x3d\x50\xf2\xd0\xae\xb9\xeb\x39\x3f\x6f\x6b\xec\xb1\xfb\x2a\xfe\x8f\x08\x94\x05\x73\x2d\x81\x12\x22\xc4\xfa\x04\x8a\x68\xab\xdf\x6c\xd4\x69\xd9\x62\x33\x67\xfb\x55\xe9\x67\x73\x65\x86\xe6\xc2\xec\x9b\xd2\x5c\x89\xe5\x2e\x6d\x35\x5d\xd1\x08\xc2\x4a\xe1\xed\x6f\xbe\x6e\xed\x7c\xde\xd8\xa0\x3d\x63\xa9\x84\xe8\xcf\xfc\xbd\x61\x5d\x6e\xd3\x5a\x57\x83\xb5\xcd\x8b\xdc\xe9\xda\x7b\x57\xab\x6a\xd4\x02\x7f\xa5\xd2\x40\xcb\xc1\x47\x00\xd6\x2a\x84\x5d\xed\xd7\xf6\x36\xcf\x31\x34\x66\x12\x3d\x24\x82\xf2\x28\xb3\xa5\x9b\xa2\x8e\x08\x35\xf3\x55\xcf\xfd\x94\xae\xcc\xa7\x78\x9e\x30\x2e\xa7\x3a\x61\xc2\xda\x2d\xf7\x8d\xe3\xd0\x5e\x97\x44\xd8\xf1\x1c\xf6\x5e\x38\x65\x77\xcf\xa7\x32\xab\x8c\x5c\xdc\x90\xa0\x2a\x07\x37\x67\x67\xe3\x3f\x28\x42\x29\x60\x19\xc9\x31\xb4\x3e\x5a\x9e\x8d\x2e\xc9\x3f\x0d\xd9\x66\x71\x61\xd4\xd7\xac\xaf\x87\x18\x04\xbb\x45\x14\x08\x01\x0a\x41\x70\xa1\x22\xc4\xc0\x55\xb3\x97\x70\xbe\x87\x31\xa3\xcb\x2b\xce\xe6\x5d\x97\xcd\xb6\xda\xa3\xa7\x99\x3b\x15\x62\x48\xc9\xa3\x3d\x3c\x5f\x8f\xe0\x74\x81\xa2\x74\xba\xc4\xc1\xc9\x04\xb2\xb5\xfd\xcc\xde\xcd\xe2\x54\x04\xd8\x7a\xfa\xfc\xab\x8c\x1d\xd4\x3e\xfb\x6a\xc7\x5d\x7d\xe8\x25\x65\x62\xef\x43\xa1\x63\x7b\x34\xfa\x62\xd4\x98\x77\x95\x40\x1a\x80\xeb\xa4\xab\xc4\x92\x28\xf0\x54\x27\x66\x5d\x20\xf4\xf6\xf6\x8e\x4f\xb8\x48\x98\x68\x81\xe4\xdb\x1c\xbd\x64\x48\x80\x5f\xed\xf8\x84\x61\x42\x5f\xc1\xb5\xb9\xe4\x7d\x84\x7b\x0a\x29\x2e\x61\x3f\x31\xfa\x93\x5a\x99\x60\xa6\xaf\x45\x88\xcf\x58\x04\x7e\x3e\x27\xdc\x02\xa1\x23\xd3\xed\x7e\x11\x76\x10\xa5\x3e\xed\x17\xba\x1f\x76\x5a\x9c\xfc\xac\x4f\xc3\xdc\x54\x86\x9d\xcc\x4c\x10\x66\xd8\x73\xab\x1b\x16\x38\x19\x58\xe3\xc4\x66\x21\x18\x31\x8b\xf4\x56\x38\xf5\xa2\x57\xd1\x4c\xf1\x11\xdc\xf0\xd7\x9b\x72\x3e\x1a\x8a\x68\x47\x3e\x22\x6b\x80\x37\x23\x84\xd8\x06\x87\x53\xf7\x5e\x5f\x86\xaf\xcc\x7c\x4e\xdd\xa4\x8a\x4a\xed\x8c\xd0\xdb\x7f\x1a\xa5\x6e\x16\x5f\x67\x48\xc7\xe6\xd7\x3b\x51\x4e\x15\xb6\x81\xb0\xe1\x27\x58\xab\x70\x34\xbb\x69\xc2\x5e\x2a\x4d\xa4\x83\x55\xb1\xcc\xdc\x62\x7b\x29\xe3\xe6\x10\x87\x57\x48\xa6\xd9\xd6\xd7\x3c\xa0\xf3\x08\xe4\x33\xe5\x06\x0a\xa7\x8a\x0e\x95\x34\x40\x81\xea\xb4\xae\x9d\xa2\x44\xcb\x98\x91\xe2\xdd\xe1\xd0\x38\xbc\x66\x58\xe5\xd4\x5c\x85\xb6\x9e\x56\xcd\x1d\xa9\x9c\xf9\xad\x2b\xbf\x79\x6d\x8f\x42\x85\x51\x5f\x6e\x0e\x1b\xe9\x70\xa7\x8e\xee\xee\xd7\x39\x73\x00\x51\x8e\x7e\xfd\x24\x45\xdc\x9b\xed\xac\x91\x4a\x25\xd1\xb0\xe1\xab\x53\x54\x4b\xa3\x6d\x6e\xa9\x59\xbd\x4a\xb6\xd5\xa5\x04\xfc\xb3\xd9\xf6\x78\x03\x88\x20\x55\x2b\x70\xb1\x56\x97\xb9\x0c\xd7\xbc\xe8\xce\x6a\x9d\x90\x23\xc6\x37\x4e\xee\xb2\x27\xdf\x4e\xa0\xf1\xeb\x93\xdf\xc3\x6c\x6c\xdc\x6d\x5c\x2c\x11\xd1\x0a\x91\x5f\x78\xdc\xc8\xbe\x27\xbd\x55\x96\xc2\xad\x65\x96\xaf\x58\x08\x95\x30\x38\x66\x44\x35\x2a\xa8\xac\x79\x0d\x5b\x40\xb5\x0c\xd2\x25\x30\xaa\xad\x94\x8f\x0f\x45\x23\xb5\x29\x92\x75\x73\xa5\xcc\x83\x27\x47\x59\xce\xed\xd3\xb5\x82\x96\x18\x2d\xd5\xc8\x64\xf1\x36\x32\x7f\x80\xe4\x9a\xaf\x14\x48\x0d\x12\x13\x76\x5f\x24\x3b\x7d\x47\x57\x2e\x00\xe5\x01\xdf\xbd\xc0\xab\xe6\xec\xef\xca\x59\x3e\xca\xf5\xf3\xf0\xbd\x01\x8b\x8f\xba\xe3\x22\x46\x60\x38\x34\x6e\xe6\xa7\x72\x1d\x7f\x5c\xa2\x5a\x33\x81\x3f\x49\xee\x1d\xc8\xa8\xc3\xf5\xae\xdc\x51\x1f\xc2\x52\x36\x9a\x27\x0d\x28\xda\x40\x8b\x54\x2b\x8c\xd4\xdd\xe3\xe3\x6b\xe4\xf4\x37\x77\xa5\xcf\x83\xd2\x59\x34\x7e\xed\xe5\xcc\xef\xfc\x27\xba\xcf\x73\x57\x25\xdd\x77\xae\xba\xd0\x8e\xb7\x72\x19\x64\x05\xfa\xdd\x52\x55\x60\xce\x06\xa7\xe4\x4d\xb5\xa6\x9e\x64\x50\x75\x4a\x67\x2b\x6b\x60\x65\x5b\x5c\x38\xaa\xe7\x3b\xf2\xd1\x09\x90\x75\xb3\x8a\x7c\x4f\xc0\x60\x27\x5a\xaf\x34\x27\xcf\x96\xf6\x1e\x2c\xbf\x9b\xb1\xf9\xdb\xab\x0f\x07\x6f\xde\xbf\x2b\x92\xed\xe6\xd3\x66\x5b\xd6\x65\xbe\xcd\x47\x8f\x06\x4f\x7d\x8a\x96\xa1\x03\x97\xa2\x6d\xf0\x50\xa5\xd8\x36\xa8\x06\x2a\x9e\x1a\x3e\x89\x28\x9e\x19\x3e\x32\x2f\xbe\x32\xea\x00\xbd\x78\xee\x4d\xb0\xf1\x0e\xb3\xb3\x7a\x2e\x14\xd5\x4e\xb8\xbc\xa2\x2c\xbc\xb0\x89\xbf\xb0\x25\xba\x9e\xce\xe6\xa5\x89\xb6\x4a\xc5\xd6\x2f\x47\x5b\x5d\x1c\xe0\x47\x69\x7a\xb4\xd9\x3d\x1a\x36\xb2\xad\xb1\x40\xe1\xc2\x7e\x70\x3d\x98\x28\x68\x80\x81\x99\xda\xcd\xd3\xa3\xe1\x93\xec\xa8\x09\xcf\x58\xe2\x28\xa7\x1f\x48\x97\xa7\xa3\x26\x66\xdc\x3e\x5d\x66\xdd\xac\xbb\x65\x70\x27\x2b\x9d\x64\x6b\x4d\x01\xcf\x0f\x88\x08\xfe\xdf\x6b\x6d\x7e\x63\x9a\xfd\x27\xd9\xdf\x19\xb7\x7b\xdf\x8f\xde\xf7\xe8\xfd\x30\x48\xfb\xfe\x9e\x32\x07\x48\xc2\xc2\xbf\xff\x94\x3d\xda\x32\xb1\x5e\x1a\x3f\xac\xd7\xfb\x05\x95\x62\xa8\x23\x4b\x8f\x8e\x00\xcb\xfe\x74\xb1\xbf\xdf\xc5\xff\x16\x7b\xd3\xc5\xde\x1e\xfd\xe9\xe2\x7f\x8b\xe1\x70\xd8\x1d\xc2\xcf\xb4\xbb\xf8\xd4\x9b\x2e\x3e\xf5\xbb\x8b\x9f\xe0\xf7\x27\xf8\x25\x8b\x92\x85\xfc\x59\x9c\x9f\xa7\xe7\xe7\xe7\x48\xc1\x77\xdf\xa5\xdf\x7d\xf7\x1d\x3e\x8d\x16\xaf\x16\x83\xc5\xee\xe2\xe2\xa2\xbb\xf8\xfe\xfb\xee\xe2\xea\xaa\xbb\x28\xcb\xee\xe2\xe0\xa0\x7b\xd0\x5d\xfc\xd7\xe2\xcf\x3f\xbb\x8b\x9f\x7f\xee\x2e\x48\x2f\x57\x6b\x31\x59\x4b\xee\xdb\xc3\xc5\xdb\xb7\x5d\xfc\x6f\x71\x79\xdb\x36\xcf\x96\x58\x7c\x95\x59\x59\xb1\x75\x34\x3c\x1a\x42\xd7\xac\xb6\x25\x44\x10\xc0\xf3\x74\xa9\x81\xe2\xfc\x30\xd7\x9b\x43\x4a\xe1\x67\x41\xb6\xb3\x76\x2c\xb6\x7a\x8d\xa3\xcd\x7e\x97\x60\x9e\x07\x30\x68\x7c\x08\xd9\xd0\x5b\xfd\x27\xbd\xc7\x83\xcd\x3f\x8f\x6e\x5a\xad\xdd\xd6\x26\xfc\x7c\xf5\xfa\x35\xfc\xfd\xba\x85\x2f\x7b\x5f\xe3\xcb\xeb\x6f\xe8\xe5\xf5\xde\x4b\x7c\xd9\x7b\x4d\x2f\xaf\x5f\xbd\xee\x37\x16\x3d\x00\x7d\x4e\xb9\xad\xe7\x00\xba\xd5\x6f\xa4\x47\xe5\x93\x6e\x98\xdc\x6f\x64\x40\xc0\xf6\x72\x6b\x6c\xaa\x8a\x8f\x62\xeb\x67\xc0\x82\x74\x62\x53\xe5\x5d\xfc\x1b\x02\x16\x5b\x87\x95\x92\x62\xd7\x08\x9f\x70\x24\x9f\x08\x15\x37\xa5\x25\x65\x2e\xb8\x99\x57\x3e\x81\x8c\x67\xcb\x4d\x9e\x50\x47\xc3\x4d\xfc\x03\xfd\xf9\x13\xfe\x3e\xb2\x0f\x9b\x94\x86\x4f\xf0\x90\xa5\xe9\xe1\x62\x2d\xa3\xd7\x34\x57\x7f\xb1\x12\xe8\x51\xf8\x7f\x26\xff\xa5\x31\xf5\x98\xf8\x88\x28\x10\xd5\x26\x99\x3f\x6d\xee\xef\x6f\xee\xed\x1d\x7e\xff\x7d\x7e\x75\x95\x97\xe5\xcf\x89\xb1\x8a\xa7\xa2\xa7\x00\x12\x43\x66\x27\x9b\xbd\x9f\xfa\x3f\xfd\x14\xbc\x6c\xbe\x82\x57\x02\x44\x0b\xaf\xbe\xb1\x8a\xa5\xa2\xd7\x4b\x2c\xd6\xe6\x41\x62\xb6\x88\x72\x22\xc7\xfd\x71\x4d\xb3\xd5\x37\x1e\xba\x1e\xd6\x83\x54\xf2\x25\x4b\xa7\x6f\xf5\xe5\xe8\x27\xb2\x35\x84\xb9\xc3\xed\xb2\xa0\x26\xd9\x3a\x1f\xe3\x6d\xde\x3f\x3e\xf3\x02\x53\xba\xe5\xba\x2c\xc8\x05\xea\xe2\x7b\xdc\xa9\x2d\xc4\x5a\x68\x71\xc0\x9b\x21\x78\xf7\x3b\xa3\xc4\xde\xb1\x58\x90\xeb\xc1\xf1\x5c\x65\xbe\xa6\xd5\x06\xd8\xb5\x2e\x90\xb7\xcd\x81\x7d\x1a\x01\xd3\x17\x9b\xc5\xe7\xa3\x67\x86\x6a\xcb\xd1\xdb\x85\x01\xd1\xb4\xcc\xff\xfa\xfc\x19\x3c\xd2\x51\x63\x99\x6f\x7f\xf5\xcd\xf6\xe8\xb9\x41\x31\x12\x60\xda\x5f\x01\xd8\xf3\xa5\x51\x77\xa1\x8b\xdb\xab\x32\x4f\xd4\xae\x0d\x36\xb1\x79\x62\x1f\xaf\x30\x0b\xab\x82\xed\x67\x9e\xe0\x0e\x14\x76\x1f\x39\xdd\xd0\x35\x7b\xf8\x8b\x39\x9f\x72\xbe\xc9\x6b\x7e\xca\x13\xe8\xc8\x9f\xe8\x79\x3f\x97\x7b\xb8\xe6\x73\x9e\xa0\x48\x0b\x05\xf6\xa8\x88\x48\xb5\x89\x19\x71\x39\x42\xf6\xca\x15\xa5\xd7\xf3\x73\xce\x63\xc0\xef\xbe\x73\xb9\x94\x80\x01\x56\xf4\xc5\xed\xe2\x16\x0a\x4d\xcf\x48\xde\xd6\x15\x40\x19\xbb\x3d\x08\xb0\x4b\xba\xa2\x16\x5f\xb9\xb8\xaf\x55\xa0\x38\x39\xac\x3e\xb2\xef\xc7\x95\x35\xf2\x66\x83\x43\x61\x6f\x6f\xed\xd3\xda\x4f\x6b\xfb\x6b\x7b\x6b\x43\xd7\xe1\x6b\xd0\xe1\xd7\xb0\x91\xb3\xaa\xee\x22\xc1\xfc\xef\xd7\x2e\xd6\xae\xd6\x4a\x84\x0f\x20\xeb\x8e\xe7\x60\x60\xac\x72\xf3\x63\xbd\x34\xb4\x97\x66\x7f\x5f\x41\x45\xd7\x23\xb4\x0b\x1d\x36\x07\x3d\xb8\xc0\xd0\x0a\xec\xb2\x90\x61\x09\xc3\x97\xa0\x88\x4a\xef\xad\xa2\x91\xbd\x4e\x2c\x69\x34\xac\x04\x71\xfb\xd5\x25\x0c\xb7\xd5\x50\x94\x3f\x7c\x10\x91\x56\xbd\x06\x33\x27\xa2\x74\xf8\x85\x18\xea\xda\x6a\xf8\xa5\x48\xa2\xf2\x9f\x56\x7d\x24\xc2\xe3\x57\xfe\xb4\x0a\x40\x06\x26\xc2\xfc\xfd\xef\x35\x40\x81\x5f\x16\x2a\xc1\x5b\x7d\xbc\x26\x6c\xb6\xa9\xd4\x17\x95\x33\xcf\xa4\xcc\x17\x15\xfa\x0a\x0a\xc1\x9c\x7e\x48\x09\x6f\xfb\xe6\x48\x44\xd3\xc6\xbb\x1a\xc8\x0e\x95\xf3\x15\x80\x77\x55\x42\xa4\x01\x83\x79\x48\x29\xc5\x04\x14\x75\xb8\xaa\xdd\xd3\x3b\x96\xc0\xef\x56\xc0\xde\x53\x15\xd1\xb8\xd2\xb7\x97\x0c\x29\xc4\xff\xea\x1e\x3a\x04\x6c\xb0\x0a\xcc\x4e\x68\xb9\x93\xc1\xa4\x90\xde\x31\xe5\x4d\x9e\x58\xec\xe0\xeb\x7a\x0b\x30\xed\xfe\x4b\x30\xb5\x01\xd3\xf7\xab\x30\x49\x21\xd8\xbb\xdc\x03\xf1\x97\xf6\xf6\x62\xd1\xde\x5e\xc2\xa2\xb5\x8a\x3d\xda\x2a\x97\xb0\xc6\xad\x80\x91\xf5\x16\x61\x0e\xea\x60\xc8\xdc\x4b\xb0\xf9\xe5\x39\xcd\xb6\x60\x38\x60\x99\xba\x42\x61\xf7\xde\x81\x21\xa3\x01\x75\xf0\x00\x24\xd5\xe2\xe6\x29\x14\xfd\x39\x8f\xbc\xf1\x0d\x8a\x4d\x82\x45\x69\x06\x9d\xee\x17\x49\xc3\x6d\xfd\x5b\x3b\x03\x8a\xe6\xb1\x39\xc0\x74\xbc\x37\x7a\xd2\xa8\x21\x75\x40\xaa\xf1\xed\xac\x91\xe4\x49\x6d\x3e\x5a\xbb\x11\xe1\x3f\xff\x77\x54\xdf\x6e\x3d\x01\x0a\x98\xef\xfc\xb9\xaa\xd7\x10\xff\xee\xc9\x09\x4d\xb2\x3f\xef\x84\xc2\x28\x0d\x08\xf5\x5f\xab\x80\x40\x36\xfa\x03\x75\x16\x06\x63\x37\xa3\x3c\xcb\xcb\x1b\xc8\xab\x6a\xa9\x84\x37\xcb\xc9\xd5\x63\x9c\x05\x6b\x4d\xd2\xef\xc4\x92\x81\xbb\xd5\x3b\x2e\x2a\x59\x1c\x0d\xa1\xd6\x28\x67\xdc\x48\xa6\x49\x3f\x2e\x92\xd6\xc3\xf6\xd1\x67\x08\x99\x47\x6a\x59\x43\xd5\x1c\x24\xdf\x59\xeb\xb8\x5f\x78\x47\x80\x2b\x2a\xdb\xe6\xca\xea\xb2\x9b\x78\x87\xe3\x6e\x0c\x08\x02\xa3\xd7\xfa\xd9\xb3\xf7\x93\xb5\x27\xc4\x72\x34\xf7\xfd\xa5\x35\x9f\xe4\x55\x67\xcc\xe6\x14\xe8\x41\x47\xf2\x30\x59\xd4\x9f\x48\x5f\x8d\x5b\x7e\x84\xe8\x62\x87\xa3\x95\x28\xbe\x90\xfa\xaa\x97\x1c\x27\x0d\x49\x58\x1a\xf1\xd7\x95\x27\xff\x39\x98\xdc\x0c\x66\x9f\x8f\x5f\x8f\x4e\x66\xf4\xb0\x3f\x98\x9d\x5e\x1c\xef\x5e\xcf\xc6\x97\xf0\xfc\xf9\xf8\x3f\x6f\x26\x23\xf8\x73\xf9\xf9\x78\xf7\xe6\xfc\xa6\x9c\x1f\x1f\x8c\xae\xe7\x23\x3c\x0e\x3b\x7e\x0f\xd2\x3c\xfe\xbe\x9b\xfe\xce\x09\x7b\xa3\x53\x7a\x70\x82\xde\x71\x62\xbd\x6e\xf9\x6f\xbc\x8a\xd4\x5d\x9c\xdf\x73\x8e\xb8\xfa\x8e\x3a\x1a\x74\x44\x22\x92\x87\x94\x21\x5d\x96\x2a\x24\x0a\x69\x42\x82\x90\x14\x24\x03\x29\xa8\xa9\x9c\x11\xdd\x43\x01\x01\x05\x64\xa8\xfb\x44\xaa\x30\x26\xbe\x73\x5e\x35\xc7\xa8\x3b\x34\x64\xf0\xc3\xd6\x80\x0a\x23\x95\xc4\x30\x88\x71\x5a\xd1\xeb\xb3\x9d\x70\x7b\xdb\x5b\x2a\x54\xa0\xc8\x29\x15\x1e\x8f\x58\x8d\x28\x7a\xcf\xe9\x6d\xc3\x1e\x69\xdc\xcf\xb8\xca\x22\xf9\x25\x69\x78\xf9\xb8\xa4\x23\x25\x72\xfe\xbe\x08\x33\x58\xb2\x93\x5c\x53\x57\x55\xe5\x26\x9d\xbf\x2d\xd7\xa4\xcb\x72\x7c\x57\xae\xb6\x2c\x5f\x90\xf5\x2d\xe3\x02\x6e\x43\x57\x5a\x8e\x91\x27\x07\x37\x13\x78\x38\x86\xed\x1b\xfe\x1c\xde\x8c\x4a\xfc\xfd\x69\x34\x9c\xf0\xd3\xe1\x05\x2c\x76\xf8\xf0\x1a\x56\x56\xf8\x39\x18\x00\x12\xdc\xd5\xe8\x0e\x75\xe8\x56\xf6\xa6\x85\xe8\xc9\x99\x56\x5f\x11\x21\x23\x0a\x28\x41\x32\x90\x06\xac\x1f\x6b\xc6\x4a\xb1\xc6\xda\xca\xee\x19\x3f\x01\x58\x5d\xb5\xc0\x2d\xb1\x52\xa8\x13\xaa\x84\x1a\xa1\x42\xa8\x0f\xaa\xab\xad\x6d\x5f\x7b\x46\x5d\x55\x17\x00\xa9\x9a\x82\x13\x76\x5f\x58\x92\xef\x1b\xad\x41\x69\x37\x5e\x83\x54\x37\x62\xbf\x8e\x07\x6c\x00\x16\x0f\x59\x1e\xae\xed\x7e\x46\x94\x8e\x2b\xa3\xd6\xed\x19\xaa\xe3\x36\xdc\x93\xac\xce\xc7\x5d\x4f\x38\xae\x63\x8a\xbe\x6c\x64\xc7\xa5\x79\x6c\xeb\x96\xd4\xa3\x3b\xbc\x80\x9e\xdf\xbe\x3d\xcc\x93\x8b\xfc\xea\x6a\x6d\x37\x31\x6f\xf3\x64\x7f\x7f\x6b\x6f\x6f\x8b\x6e\xd5\x99\xb7\xf4\xbe\x8f\x7b\x62\x9b\x10\xa6\xac\xbd\x3d\xa4\x44\x48\xc5\x4d\x97\x59\x8b\xf2\x60\xdd\x0e\x6b\xf3\x6e\x40\xac\xcb\x10\xf1\x5c\xc0\x5f\x12\x42\x93\x07\x0e\x1b\xc2\x87\xe1\x36\x36\x56\x01\x36\xe7\xd3\x1f\x31\x02\x29\xbb\x5f\xa1\x40\x72\x77\x63\x8e\x0a\x78\x43\x47\xfc\x86\x05\xfc\x7f\x6f\x0f\xd5\xc4\x43\x6d\x5b\x19\x5d\x38\x12\xe3\x9f\x36\x46\xa9\x59\xf9\x01\x05\xd3\x91\x19\x71\x9b\x60\xf0\x2a\x61\x1e\x9f\x0e\xc9\x99\xe0\x35\xde\x86\xe5\x34\xba\x72\x1b\xfa\x94\x41\x1b\xbe\xdd\x79\x8a\xa2\x6d\x78\x55\x3b\xdf\xea\x0d\xae\xfb\x47\xcd\xee\x55\x17\xfe\x6c\xc1\x9c\x91\x5c\x5f\x8f\x75\x9e\xc9\xae\x34\xc7\x25\xe1\xf5\x91\x5b\x30\x17\xfd\x45\x49\x46\x37\xb9\xbe\x02\x09\xf3\x87\xfd\x24\x77\x29\x03\x4c\xd9\xdd\x87\x4e\x3d\x3e\x1d\x80\xf0\x32\x1c\xcc\xf2\xdb\x12\x86\xd7\x1e\xaa\x74\x7a\x87\x53\x18\x6d\x6b\x83\x79\x9f\x06\x05\xba\x0f\xb1\xe9\x57\xd3\x19\xba\x15\xd7\x59\x3f\x91\xc6\x07\x9b\x77\xad\x67\xd3\xd1\x8b\x00\x17\xf9\x3b\x8c\xdf\xd1\x4c\xa3\xc3\x3c\x2e\xd3\x7b\x0b\x8f\xfd\xb5\xb0\x28\x52\xf1\xea\x12\xda\x21\x79\x4b\xfa\x28\xa1\x4e\x8f\x36\x13\xfb\xca\x10\xbf\xcd\x02\xab\xc7\x5a\x5d\xdc\x20\x2a\xd3\x15\xe7\xa8\x6c\x28\x81\x08\x73\xdb\xa9\xc7\xda\x76\x26\xbf\x3d\x23\xd3\x9a\x3c\x01\x21\xe8\x2f\x20\x94\x5e\x03\xcd\x79\xf2\x97\x72\x6d\x70\x3e\x25\x95\xde\x60\xed\x0c\x66\xb8\xd5\x3f\xa2\x66\x6f\xb0\x66\x75\x7b\x57\xf0\xf6\x97\xa1\xf5\x7a\x4a\xba\xbe\x01\xf7\x10\x3c\x5f\x50\x1e\x75\x17\x29\xff\x06\x6b\xa4\x44\x1b\x0e\x29\x9d\x45\xe0\x7d\xc2\xc6\xfa\xbe\xfd\x7d\x46\x26\x62\xf3\x67\xcc\x62\x05\xe0\xe7\xcf\x94\x43\xe6\x24\xd0\x68\x01\xfd\xae\xe1\x6a\x4d\x90\x62\x03\xa2\x6a\xab\x6a\x64\x3d\x06\x7f\x68\xe3\x3e\xac\x4a\x69\x77\x3f\x69\xff\x82\xaa\x7d\xeb\x07\x97\xda\x9b\x21\xfd\xa7\xa0\xa3\x4b\x99\x83\x12\x8e\x84\x75\xe8\x35\x14\x23\xe8\x4e\xab\x9b\x70\x2f\xc2\xa8\x47\x7c\xc9\x1d\x5f\x20\x47\xd8\xfc\x23\x8c\xc7\x1e\x6c\x2b\x1a\x4b\xa0\x51\x32\x9d\x8e\x32\x6e\xeb\x68\xf1\x14\x28\xcf\xff\xff\x32\x4c\xfc\x77\xda\xec\x9c\x92\xed\x41\xbd\x47\x19\xc5\xaf\xe1\x57\x68\x9e\x69\x39\x3f\x8b\x18\xf2\x0a\x50\x52\xcd\x86\xfe\x66\x94\x05\x88\xb2\x0a\xf2\x4b\x11\xde\xd4\x0b\x5f\x3f\xb3\xaa\x4d\xc4\x8b\xfc\x96\x2f\x18\x42\x7a\xfe\x1c\xd2\x94\xeb\x95\xdc\x05\x38\x66\x65\xb6\xce\x5a\x75\xf6\xac\x60\xd0\x79\xbb\x9c\xef\x86\xfc\xd5\x28\xd7\x2d\x86\x6f\x32\x38\x96\x7b\xc2\xd1\xb4\x7c\x67\x22\x10\x1a\x27\x13\x18\xdb\x4a\xd1\xf5\x1c\xe5\x36\x52\x59\x7e\xdc\x1e\x8f\xd9\x2b\x91\x39\x3e\x93\x1e\x87\x35\x36\xa7\x62\x72\x69\x22\xe7\x1f\xc3\x87\xfe\xe8\x86\x2d\x5b\x1a\x2f\x20\x3f\x84\x56\x3a\xa2\xee\xfc\x33\x14\x17\x01\xcd\x7c\xad\x2a\x5f\x6f\x39\xa2\x5a\x9e\xea\x9a\x0f\x0a\x3e\x64\x29\x5e\x31\xfd\x47\xc0\x2e\xbd\x62\xef\x10\x38\x89\xa2\x1b\xe0\x62\x9f\x6a\xa2\x63\xff\xf8\xfb\x9d\x68\x50\x8e\xcf\x27\xc0\x97\xe6\x7c\x0c\xf4\xa6\x9c\xc2\xfc\xb7\xc7\xd0\x85\x33\x42\x71\x3e\xdf\xc5\x30\xc2\x46\x5d\x2b\x2a\x4e\x41\x19\xbd\xab\xd6\x17\x15\x9f\x53\xec\xec\x3c\xb7\x08\xba\xb7\x4b\x69\x08\xb6\x0a\x27\xef\x5b\xce\x43\xa4\xc0\xa0\xf3\x37\x8b\x91\x97\x7b\x46\xe2\x6c\x1a\x02\x77\xd5\x9c\xc7\x96\xe0\xc5\x8a\x33\x7a\x6d\xfc\x9c\x75\x53\x6c\x04\x54\xc8\xa0\x05\x23\x19\x90\xb7\xfb\xdd\xcd\x76\xde\xf6\xdf\x71\xfb\x19\x27\x53\xce\x0a\x1a\x86\xa1\x9b\x9e\xd9\x13\x6a\xc0\x8b\x20\x87\xae\xab\x48\xce\x55\x90\x23\x97\x28\x25\xaf\x0c\xf2\xc4\x01\x80\x2d\x57\x46\x05\xbd\x8b\x00\x86\x40\x93\x23\x67\x49\x1f\x18\x0e\x04\x1f\x47\x61\x31\x57\x7c\x9d\xed\xf2\x60\x70\xd8\x48\x41\xd4\x90\x1b\x1b\xa1\x47\x01\xcf\x22\x0d\xfc\xd7\x4c\x9c\x4d\xb1\x18\xfb\x42\xb1\xac\xdb\xca\xf1\x87\x89\x0c\x5a\xd0\xd6\x27\xdf\xb4\x0d\x7b\x87\xfd\x38\xf1\x29\x24\x0e\xe3\xc4\x67\x90\x78\x11\x27\x7e\x05\x89\x57\x71\xe2\x73\x48\x2c\xe3\xc4\xaf\x21\xf1\x53\x9c\xf8\x57\x0a\x77\x0c\xd4\xb3\xc1\x54\x25\xa8\x81\x9a\x07\x68\x27\x46\x37\x3a\x23\xe7\xf3\xe4\x91\x8d\x5c\xcf\x03\x1a\xf6\xcf\x26\x06\xeb\xf4\x4c\xd8\xdd\x4c\xc4\xd8\x24\xd8\x0e\x62\xfb\xe2\x26\x28\x7b\xae\x91\xb3\x66\x77\xea\x5c\x67\x7a\xa4\x8d\x75\x1c\x5a\xaa\x34\x90\xc1\xac\xa7\x43\xea\xc6\x8e\xf2\x9f\x28\x3e\x0a\xbb\xce\x11\x61\xbd\xcb\x45\x8b\x40\xae\xec\x17\x52\x2a\x8d\x3c\x18\x22\x0b\x29\x92\xd1\x24\xc9\xf2\xd8\x79\x62\xbd\xf3\x42\x33\x2b\xaa\xb6\x48\xd2\x6a\x91\x97\xbb\x1a\x13\x39\xc6\xd0\x3c\x46\x8f\x8d\x59\x1e\xc1\x73\x72\xd0\x26\x7b\x18\xc3\x20\xdc\x09\xf9\xa6\xa0\x18\xfd\x5c\x54\x3d\x32\x16\xe8\x4d\xfc\xb2\x28\x95\xbc\x18\xd7\x90\xb4\x8c\x0d\xf5\x56\x86\x9a\xd4\x51\x79\x19\x54\x95\xdd\xab\xb0\xe9\x3b\x4a\x5b\x60\xd8\xe5\x14\xa4\xb2\xb5\x7e\xcf\xdb\xe4\xad\xac\xb3\xb9\x39\xce\x9c\x7f\x6a\x02\x20\xbf\x76\xb8\xb1\x97\x0a\x43\xbf\xd9\xbe\x52\x71\x3a\x1e\x3b\xbf\xd6\xbe\xc9\x3d\xd1\x2c\x12\x28\x37\xd3\x68\x3a\x19\xba\x76\xa6\x25\x0c\x2f\x81\x85\xb6\x77\x04\xd9\x15\xe5\x27\xf9\x09\x35\x5c\x38\xaf\x75\x45\x8a\x97\xcb\xae\x5c\xbd\x34\x8d\x7f\x0e\xa2\x93\xd7\x2e\x86\x9c\xe8\xc1\x71\x29\xb5\x55\xda\xd1\x53\x88\x2d\xb0\x8e\x40\x83\x81\xcf\x6b\xe4\x20\xc1\x4a\x76\x7f\x12\x7a\xe6\xfd\x59\x15\xae\x21\xb6\x7a\x14\x97\x43\x54\x21\x1c\x17\x0b\x2f\xc9\x93\x1d\xc2\x1f\x35\xd8\x95\xff\x66\x36\xc0\x43\x73\xfe\xa5\xb1\x1e\xf6\x57\xc9\x65\x44\x2c\xec\x59\xe9\xec\x89\xe6\x62\x53\xc4\x63\xdc\xf6\xad\x91\x9e\x40\x14\x05\xd6\x6a\x64\xad\xf7\xdd\xfe\x61\xff\xe7\x9f\x13\x42\x7f\xa7\xd0\xc7\x74\x77\xbd\x3d\x69\x60\xf4\x88\xc5\xdf\x1c\xbc\x5f\x49\x60\xe8\xaf\x50\xb5\x1e\xcb\x35\x46\x59\xcd\xf4\x0e\xfb\xde\x04\xe6\xe0\xa0\xf7\x73\x9f\xa9\x23\x43\xd4\xf8\x80\xe5\x8a\x36\x13\x32\xa0\x7a\x57\xf6\xbc\xd5\x29\x68\x8d\x0d\xf3\x60\xae\xdc\xf1\xdb\x95\x3a\x7b\xbb\xf2\x07\x5e\x94\xae\x4f\x94\xfa\x4b\x7b\xeb\xaf\xe6\x83\xdc\x7d\x40\x1e\x02\x30\x69\x0f\x0e\x0f\x30\x88\xd3\x68\xe5\x81\xfd\xf1\xa0\x2b\x87\x91\xe2\x9b\x31\x76\x03\x2a\x50\x26\xd5\x06\xab\x6a\xee\x48\xbe\x65\x78\xee\x1d\x95\x16\x6c\xa6\x9b\x65\x3b\x2d\x94\x76\x69\x31\xb7\x06\xcc\x35\xf4\x78\x83\x61\xc6\x71\x7d\x86\xdf\xc0\x93\x6c\x77\xbe\xf2\x03\x94\xdb\x61\x18\xbc\xf3\xd3\xbb\x0e\x9a\x48\x67\x42\xa6\x78\xf7\x40\x19\xf5\xb5\xc5\x7a\x5b\xc2\x39\xc5\x9b\x25\x9a\xc0\x07\xca\xbd\x9a\xec\x7e\x83\x81\x45\x46\x05\x0a\x12\x63\x6b\x54\xd7\x55\xc7\x7f\xf4\x49\xad\xdf\xa0\xa5\x6e\xbf\x38\xd0\x46\x0d\x2c\x97\x92\x3a\x8a\x2c\xfa\x6f\xdc\x6a\x0a\x8f\xc5\xaa\x5b\x6e\xe1\x12\x97\xa2\x03\x1c\xb1\xd1\xce\xe3\x3c\x5f\x81\xb9\x2f\x98\x0b\x7d\x27\x14\x34\xed\x4c\x9a\xab\x14\xe0\xff\x97\xd3\xbb\xe9\x08\xc6\x0d\x7f\x4c\x2c\xad\x29\x66\x50\x92\x80\x29\x64\x7b\x15\x02\x14\x74\xba\x03\x3d\x35\x2c\x6f\xa7\xe1\x14\xf3\xd8\x3c\x84\xa1\xf1\x08\x53\x1d\x61\x29\x66\x88\x67\xcc\x7c\x44\xbb\x89\x95\xc8\xb3\x1b\x27\x1c\x40\xa3\x76\x01\x34\x6c\xdb\x45\x97\x25\xe1\x7d\xb1\x10\xab\x2f\x9b\xd0\x25\x25\x48\xf1\xec\xe9\xf6\xe8\x2b\xa9\x48\x3b\x80\xcf\x1a\x54\x5f\x90\x64\xd5\x96\x45\x7b\xfb\x89\x36\x22\x61\xd2\xf8\x19\xdd\xb6\x29\x83\x27\xce\xba\x0a\xcb\x37\x0a\x82\xd9\x0c\xb8\x2d\x88\x0d\xb3\xb9\x0a\x22\xb1\x99\x62\x51\x0f\x03\xed\x5e\x81\xc9\xb2\x2d\xd5\x0d\x9b\xd5\x36\xbb\xb3\x06\x0b\x94\xaa\x96\xbd\xbb\x3e\xdb\xfc\x52\x6d\xd8\xc2\x4e\xcb\xbc\x85\xfe\x03\x60\xd7\x43\xed\x4b\x1f\x8a\xd8\x6c\xdb\x59\x0b\x3e\xd7\x0f\x08\x86\xeb\xa8\x33\xe8\x0b\x73\xe0\x9b\xc4\xbe\x2f\x4c\x47\xb3\x42\xb6\xf7\x0b\x7b\x74\xd3\x8e\xa0\x6c\x8b\xec\x0d\xc5\x04\x70\x25\xd0\xf3\xd6\xb3\xbf\x02\x14\xa6\x66\x76\x84\x8b\xfa\x2d\x77\x91\xa8\x1c\xe3\x39\x83\xa9\xe3\x27\x07\xba\x5f\x0b\x95\x73\x91\x1c\xe2\xe7\x24\x0f\x30\xa8\x84\x0a\x65\x22\x0b\x68\x3e\xc7\xf2\x6b\xf3\xe2\xe6\x6a\x30\x81\xb1\x9c\x86\x37\x15\xa5\xee\x77\x53\x65\x90\x55\x5f\x33\xe1\x44\x58\xbb\x9a\x67\x26\xc6\x54\x55\x0a\xfb\x39\x5d\x78\x4a\x9d\xd7\x2f\x3f\x81\x65\x00\xf8\x91\x81\x1d\x00\x73\x8d\x95\xad\xe8\xb6\x49\x34\x89\x9b\xcf\x77\x10\x47\x37\xb1\xea\xe8\x24\xdf\x6c\x4b\x92\x55\x60\x27\x79\x4b\xa5\xec\x01\xa6\xbc\xad\x4a\x51\xc2\xb6\x24\x88\x0e\x3d\xc9\xbf\x56\x09\x8c\xc4\x57\x11\x2c\x1f\xb2\x66\xe8\x26\xb6\xdf\x6d\xbd\x17\xd3\xbc\x20\x19\xc1\x46\xb0\xa8\x95\x27\x5c\x78\x0b\x35\xe7\xad\x64\x71\xe7\x12\x9a\xbd\x08\xc4\x3f\x66\x03\x2d\x3b\x8d\x16\x8b\x7b\x20\xbf\xb2\x90\x4b\x33\xd4\x02\x96\xba\x44\x83\xd7\xce\x34\xe3\x15\xa1\x4f\xdf\x52\xcb\x55\x5a\x70\x6d\x2d\xb8\x60\xa3\xee\xe9\x86\x97\x88\x55\xfb\x89\x40\x80\x97\xfd\x6e\x87\xac\x0e\xda\x04\xc0\x25\x4c\x76\xfc\xe1\x03\xf7\x55\x64\xbe\x3f\x23\x2e\x6a\xbe\xe4\xda\xcf\xca\xab\x5a\x4c\x6d\x60\xcf\x55\x75\x1e\xba\xe2\x8a\x32\xb5\x51\x9e\x7a\x92\x64\xc4\xb3\x30\xea\x9f\xdb\x56\xfe\xd1\xb7\x8e\x12\x2a\x91\xb8\x9b\x47\x1e\x7c\xdf\x06\xc5\x52\xdf\x5a\x5d\x55\xb2\x7f\xe4\xb2\x92\xab\x14\xcd\xc3\x78\xe2\xe5\xf1\x5e\x50\x1c\xb9\xde\xb1\x20\x8a\x7b\x57\xe2\xd9\xb9\x5a\xa3\x5a\xe2\xaf\x91\x79\x7c\xae\x1b\x40\x1c\x47\xd2\x2c\xa3\x47\x6b\x96\x2c\x4e\x21\x71\x3e\x2a\x43\x37\x8b\x89\xb8\x75\x1e\x18\xb6\xb9\x4a\x98\xc5\xe7\x81\x45\x9b\xcd\x94\x95\x21\xaf\xda\x90\x81\xc4\x2a\xfa\xde\x90\x99\x07\x66\x86\x20\x53\x38\x02\xfd\xa2\x14\x5b\x19\x3a\x39\x07\x38\x41\x4d\x3b\x3e\x40\xb0\x60\xda\x85\x01\x72\x1a\x5f\x80\x8d\x2b\xef\x4a\xd3\xb1\xf4\xd4\xce\x9a\x56\x16\x84\x05\x15\xb8\x65\x9b\x78\xc8\x2e\x6c\x4c\x66\xb5\xf2\x56\x44\x4c\xdd\xbd\x28\xae\x86\xfe\x46\xa6\xf2\x8d\x80\x9f\x84\xd4\xee\x34\x42\xd9\x2b\xcc\x45\xa2\xbe\x1d\x01\x87\x1c\xfd\xcf\x52\xf5\xe2\x5e\xaa\x0e\x06\x57\xff\xc3\x34\x41\x4f\xde\x43\xd5\x95\xb6\xf6\x98\xce\x2f\xd4\xa1\x15\xbd\x59\x0d\x4b\xad\xa7\x07\x1a\x4a\x3b\x04\xc7\x5c\x89\x1e\x01\xe9\xe0\x8f\x7f\x02\x29\x81\xec\x20\xbe\x00\xe9\x9f\x81\xc2\x44\x31\x67\x96\xc9\x8b\x48\x40\xd7\xfc\xd8\x5e\xad\xd5\x7c\x9b\x01\xf5\xe2\x62\xef\xa6\x08\x4f\xbb\xeb\xaa\xad\xf0\xef\x7b\xbc\xd7\x66\xc6\x39\x51\xe0\x84\x17\xed\xe7\xae\xf0\xf3\xd6\x13\xcd\x83\x85\xf0\x42\x2d\x5a\x76\xe3\xda\x92\xd0\xef\xeb\x85\x75\xd5\xfc\xa0\x5d\x51\x2c\xc1\x31\x92\x4d\xae\x20\xb9\x02\xa9\xa7\x4d\x02\x8f\xe8\x19\xac\xa1\xe7\xea\xb3\xb9\x60\xb9\xe3\x02\xef\x82\x11\x5d\x5f\xe0\xe5\xff\x43\xda\xf5\xf5\xb4\x0d\x03\xf1\xf7\x7d\x0a\x88\x94\xaa\x6e\x9c\x91\x02\xa2\x52\xbb\x28\x9a\xb4\xa7\x3d\x6c\x48\x3c\xed\x69\x2a\xa3\xdb\x2a\x31\x90\x08\x2f\xd5\xca\x77\x9f\xef\x9f\x7d\x4e\x6c\x06\xe5\xa5\x4d\x7c\xce\xf9\xe2\xf8\xcf\xd9\xe7\xfb\xdd\x3d\x19\x36\x1f\x37\x37\x47\x6e\x1c\x82\xfd\xe9\xf5\xed\x11\x54\x36\xf1\xf1\x3b\x67\x63\x46\xa3\xfa\x67\xc6\x3c\x2a\xaa\x75\xd9\x56\x46\xc5\xdf\xeb\xfe\xe3\xed\xf6\x97\xeb\x3b\xe0\xbd\x43\x9f\x32\x73\x40\x83\x9d\xbb\xf5\xfa\x34\x5a\xf1\x99\x65\x63\xa3\xd5\x08\x11\xcb\x0b\x8a\xfb\x65\xd5\xc4\x98\xa8\x03\x3d\x6d\x2a\xbd\xcb\xea\xd5\x15\x29\x46\xec\xa8\x9e\xd5\x3b\xd0\x69\x9e\x20\x25\xa6\xe9\x05\x11\x2a\xb1\x99\xc5\x12\xce\x97\x86\x17\x13\xa6\x9a\xa7\xfa\x45\x10\xc2\x2b\x48\x80\x86\xe0\x35\x24\x06\x82\x20\x0b\x72\x56\x58\x28\xa9\x55\x96\x64\x6c\x85\xd1\x0a\x21\xb6\x28\x0f\x52\x77\x06\xab\x28\x25\xdf\x2e\x16\x6d\x27\xa2\xed\x48\x2a\x75\xa4\xff\xe5\x82\xcd\xed\xf9\xc1\x05\xc6\x86\x74\x55\x12\xe2\xec\xea\x97\x43\x2f\x13\xfc\x22\xab\x77\x89\x72\x90\x51\x54\xe7\x8b\x19\xf1\xab\x81\x64\xc2\xcb\xe5\x8b\x4b\xbe\x18\x24\xa6\x5e\xec\x05\x05\x7a\xdc\x87\x4c\x81\xa0\xb2\xfb\xfd\x86\xa9\xa9\x16\x75\xf2\x1b\x9b\x72\x91\x13\x00\xb8\x27\x1a\x1a\x93\xc2\x3b\xa7\xa4\xc8\x8d\xea\x28\xcb\x7e\xbf\x58\xfa\xbb\x90\x5c\x2e\x3a\xb6\x06\x63\x39\x0b\x70\x68\xd1\xa3\xc2\xeb\x54\x27\x8e\x71\x8a\xca\xec\x88\x0b\xed\xd8\xbf\x88\x57\xe2\xe8\x89\x62\x4f\x43\x9c\x94\x45\x5c\x79\x88\x83\x9a\x5e\xe6\x4c\x4e\x11\xb8\x16\x8d\x8d\x98\x7f\x1a\xae\x73\x56\x2f\x9a\x08\x9e\x0c\x43\x82\xa6\x1d\x31\x35\x96\x5e\x0a\x57\x21\xfd\x14\x1e\x64\x4c\x07\x94\xc5\x68\xb2\xe0\xba\x97\x7b\xd0\xac\x52\xe5\xb0\xe7\x63\x21\x80\x52\x85\x5f\x98\xfc\xbc\xc3\x25\x4b\x1b\xdd\x2a\x22\xc7\x8e\x1e\x24\xa8\x0c\xd0\x0e\x35\x1d\x81\x59\xc2\x2d\x37\x4e\x9d\x83\x93\x54\xa6\xc7\xfb\xcf\x57\x5f\xbf\xb4\x3a\xc1\x1b\x31\x06\xe6\x21\x65\xa5\x6c\x65\x2e\xd7\x96\x22\x0e\x8c\x3e\xdc\xeb\x10\x34\xa7\x38\xa0\x36\x85\x57\x8f\xce\x29\xd0\x87\x1f\x87\xd4\x66\xcd\x47\x85\xd3\xe6\x9c\x78\x63\x31\x38\xbb\xe4\x79\x5c\xaf\xe0\x27\x3e\x00\xa1\x6f\x4a\xf0\x9d\x15\x82\xdf\x76\xd2\x39\x08\x39\x0a\xb9\x48\x3e\x79\xf6\x22\x04\xc4\xf7\x8f\x6a\x50\x29\x2e\x9a\xa3\x71\xd3\x3f\x3c\x44\x60\x53\xaa\x34\x8d\x37\xe5\x1e\x21\x3a\xfe\x96\xa7\xe7\xf8\xd6\x55\xc8\x1e\x70\xa8\x30\x33\xd6\x09\xfc\x94\x67\x0d\x57\x88\xca\x7c\xc3\x58\x4e\x2c\x8b\x34\x21\xf8\x2b\xe7\xa7\x8c\x64\x15\x24\x41\xc2\x09\xc0\x90\x62\x7e\x22\xe3\x2f\x8d\xaf\x29\xcb\x89\x7f\xda\xaf\xba\x11\x54\xe8\x19\x23\xe0\xd1\xf8\xd3\x56\x38\xcd\xcf\xc2\xb7\xad\xf4\x47\x75\xa2\xce\xc8\x6f\xb9\x62\x8f\xe5\x99\x72\x9a\xfa\x1e\xc4\x76\x7a\x14\x6f\xe3\xc5\xbb\x75\xb2\x55\x27\x9b\x6d\x9b\x87\xcd\xdd\x8f\x4d\x8b\x85\xc8\x06\x69\x04\x3c\x16\x32\xd9\xe3\xc0\x20\xda\x94\xf1\x28\x44\x9e\x3c\x38\xef\x2b\x76\x14\x7f\x0e\x50\x73\xe5\x9d\x4d\x63\xdf\x68\x72\x19\x9e\x19\x50\x59\x56\xf9\xda\x86\x08\xf4\x83\xbe\xa5\xaa\x9e\xa8\xd8\xc3\x74\x05\xcb\x53\xd4\xcf\x88\x42\x7d\x7c\xfa\x1a\xa3\xcb\x41\x12\xd7\xcf\x4a\x5c\x67\x24\xae\xff\x2f\xf1\x9b\x27\xd4\x78\x7e\x40\xd8\x1d\xf4\xa8\xec\x0f\x66\x8b\xf0\x33\xc4\xdb\x1f\xb6\x8e\x0f\x8a\x33\x55\x4e\x7e\xfb\x42\x71\xc6\x0c\x63\x37\x9e\xa4\x73\x1d\xa5\x4f\x98\xc9\x45\xbf\xec\x43\x90\x78\xaf\xed\xf7\x53\x23\x2e\x40\x03\x2a\xbb\xcb\x18\x06\xc6\x8b\x89\xd4\xf5\x05\x4b\x2f\xa6\xb1\xa5\x39\x40\xf0\x0d\xf8\x8a\xf9\xd9\xf8\xc1\x38\xce\xe0\x4d\xd0\xd5\x78\x93\x8a\xc0\xfd\xa2\xb6\xb3\xee\xaf\x84\xc8\x18\x46\x2a\xe5\x43\xd3\x15\x35\xac\xe1\x5c\xc5\x5d\x16\x15\xc2\x6b\xf5\xa8\x3a\xf7\x55\xf1\x8d\x08\x3c\x14\x76\xdc\xf2\x21\x38\x13\x26\xc3\x3b\x76\xd8\x49\x20\xb6\x11\x26\xe1\xab\xed\xf7\xfc\x0a\xfb\x3d\x8b\xd5\x41\x94\xb1\x90\xa1\xc3\xdf\x0a\x82\xf5\x10\x7f\xca\xde\xf1\x7f\x28\x41\x1e\x97\xae\x0a\x11\x72\x20\xea\x64\x71\xd9\x7c\x2a\x9c\x96\x13\x3c\xdb\xd2\x28\x11\x26\x9d\x3c\x3c\x9c\xb5\x85\x63\x59\x49\x74\xa4\x6d\x06\x7e\x02\x54\x1a\x9b\x80\x6b\xda\xc6\x3d\xc0\x90\xde\x33\x62\x5b\xa0\xf2\x51\x58\xb2\xfb\x98\xe1\x32\x1f\xda\x2b\x40\x11\x61\xa3\x1b\xcd\x18\x74\xb6\xa2\xf6\x83\xbf\x6e\xa9\x27\x3c\x33\xb8\x39\x42\xa7\x47\xa7\x9d\xf0\xf0\x87\xfd\x9b\x3d\xee\x0c\xbd\xe1\xba\xa5\xbb\x72\xde\x78\x7b\xa3\xd3\x4a\x69\xa2\xf1\xa4\x06\x9c\x72\xbb\x02\x76\x6c\x81\x7a\x0d\x7b\xe5\xc5\xf2\x94\x2e\x61\x13\xf5\x8c\x2e\x1f\xdc\x25\xe4\x0a\x6b\x09\x60\x50\x49\xd8\x4a\x13\x80\x97\xba\x69\x8c\xae\xc4\xe3\xa3\xd5\x90\x52\xee\xf3\x8f\xb5\x6e\xd2\x96\x27\x13\xfa\x7f\xbf\xfe\x73\xd3\xd1\x25\xd8\x0d\x81\x45\x11\x9c\x46\x38\x9c\xad\x15\x04\x27\x2a\x52\x59\xcd\x50\x02\xf2\x8f\x74\x13\xd1\xdd\x3d\x95\x7c\xdc\xb6\xc7\xcd\x64\xa2\x45\xe1\x6a\x7d\x22\x54\x33\x49\x7d\xa2\x88\x9d\xb4\x62\xfc\x17\x00\x00\xff\xff\xb7\xab\xf3\xee\xbf\xef\x05\x00" + // scripts_browser_modules_js returns raw, uncompressed file data. func scripts_browser_modules_js() []byte { - gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ -0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0xdc,0xbd, -0xeb,0x76,0x1b,0x47,0xd2,0x20,0xf8,0x7f,0x9e,0x82,0xa8,0xf6, -0xc0,0x55,0x42,0x12,0x22,0x64,0xbb,0xe7,0xeb,0x82,0x8b,0x18, -0x59,0x92,0x3f,0x6b,0xc7,0xb7,0xcf,0xd2,0xd7,0x97,0x05,0x21, -0x1d,0x90,0x28,0x8a,0x25,0x81,0x00,0x8c,0x02,0x45,0xc9,0x04, -0xfa,0x59,0xf6,0x3d,0xf6,0xc7,0x9e,0xb3,0x2f,0xb4,0xaf,0xb0, -0x71,0xcd,0x5b,0x15,0x20,0xc9,0xee,0x99,0xb3,0x67,0xbb,0x8f, -0x45,0x54,0x5e,0x23,0x23,0x23,0x23,0x23,0x23,0x23,0x22,0x3b, -0x97,0x37,0x8b,0x8b,0x4d,0xb5,0x5c,0xa4,0xb7,0xd5,0x62,0xb6, -0xbc,0x35,0x37,0x8b,0x59,0x79,0x59,0x2d,0xca,0x59,0x76,0xa7, -0x59,0x47,0x55,0xfd,0x70,0xbd,0x9e,0xbe,0x9f,0x57,0x6f,0xca, -0x74,0x79,0xfe,0x3a,0xbb,0x7b,0x3b,0x5d,0x1f,0xcd,0xcb,0xc5, -0xab,0xcd,0x55,0x01,0xdf,0x7d,0xfe,0x69,0x36,0xef,0x57,0x65, -0xf1,0xfa,0x3f,0x6e,0xca,0xf5,0xfb,0x3e,0xfe,0xa6,0xb2,0xc3, -0x75,0xb9,0xb9,0x59,0x2f,0x8e,0x24,0xbd,0xaa,0xff,0x46,0xfd, -0x50,0xde,0xa8,0x33,0xc8,0x07,0x45,0x41,0x6d,0x2c,0x96,0xb3, -0xf2,0x39,0x54,0xea,0x76,0xb9,0xb5,0x51,0xe7,0x24,0x4f,0xa6, -0xd8,0x6d,0x02,0x25,0xb0,0xb9,0xed,0x36,0x51,0x88,0x92,0x0e, -0x27,0x75,0xbb,0xe9,0x09,0xe4,0x72,0x0d,0xc8,0x5f,0xdc,0x5c, -0x9f,0x97,0xeb,0x84,0x33,0x97,0x97,0x02,0xa3,0x36,0x79,0x7a, -0xa2,0xbf,0x8e,0x07,0x47,0xd5,0xe2,0x08,0x41,0xd8,0xd9,0x41, -0x5e,0xac,0xcb,0xe9,0xa6,0xfc,0x69,0x85,0x1f,0x75,0xba,0xe4, -0xbf,0x3c,0x54,0x28,0x58,0x5e,0x6c,0x0a,0x49,0x7b,0x34,0xbd, -0xb8,0x2a,0xc7,0xf2,0x31,0x29,0xee,0x76,0xd1,0x10,0x4b,0xc8, -0xd7,0xfa,0xfd,0xeb,0xe9,0x06,0xbe,0x2e,0x96,0xeb,0xf2,0xe5, -0x7a,0xb1,0xdc,0xdc,0x5e,0x55,0x9b,0x32,0xdb,0x6e,0xc7,0x13, -0x63,0x11,0xff,0xd2,0x5c,0xce,0xa7,0xaf,0xb2,0x3b,0xee,0x65, -0x8c,0x1f,0x93,0xa2,0x73,0xb2,0xcb,0x0c,0xa7,0x38,0x10,0xab, -0xc5,0xa6,0x5c,0x2f,0xa6,0xf3,0xc7,0xd3,0xcd,0x34,0x2d,0xe7, -0xe5,0xb5,0x59,0x4c,0xaf,0x4b,0x33,0x83,0x4f,0xb3,0x7a,0xbb, -0xc9,0xee,0xaa,0xcb,0x54,0x80,0x98,0x5e,0x5c,0x94,0xab,0x8d, -0x2d,0x98,0xf1,0x40,0x36,0x57,0x15,0x83,0x6f,0x00,0x64,0xa3, -0xcd,0xfd,0x8f,0xf2,0xbd,0x4e,0x5b,0xf9,0x6e,0x35,0x85,0xe9, -0x31,0xaf,0xca,0xcd,0x37,0xef,0x7f,0x84,0xc6,0x8b,0xa4,0xde, -0xac,0xab,0xc5,0x2b,0x87,0x53,0xea,0xb2,0xaa,0x7f,0x84,0xf9, -0x2a,0xb0,0x69,0x3b,0x73,0xe6,0x02,0x5b,0x2e,0x38,0x6b,0x24, -0x0d,0x52,0x5a,0x4e,0xb0,0x56,0x33,0xcd,0xc3,0xcf,0xb1,0xd7, -0xfb,0x24,0x6f,0xa4,0x74,0xbb,0xde,0xd7,0x10,0xc6,0x55,0xcd, -0xba,0x5d,0x6a,0x6c,0x5c,0xcd,0x20,0x37,0x85,0xf1,0x6e,0xb7, -0x36,0xa1,0x8f,0x38,0x00,0xbc,0x76,0x2c,0xe0,0xdb,0x2d,0x26, -0x01,0x9d,0x38,0x82,0x96,0x69,0xaa,0x66,0xdb,0x6d,0xba,0x0f, -0x92,0x02,0xa0,0xa4,0xd9,0x9a,0x41,0xd6,0xa6,0x9c,0x3d,0x9d, -0xd5,0xfd,0xd5,0x72,0x95,0x42,0xdb,0x32,0xa2,0x57,0x37,0xd5, -0xac,0xd7,0xcb,0x71,0x34,0xae,0x5e,0x66,0x2c,0x28,0xd0,0xb8, -0xfd,0x0d,0xc4,0x21,0xa8,0xf2,0x53,0xfb,0x9b,0xe5,0xff,0xf6, -0xec,0xa7,0x1f,0x15,0xe7,0x8b,0xe5,0x72,0x95,0x65,0x26,0x4d, -0x78,0xba,0x43,0x4c,0xfb,0x44,0x1f,0x64,0x64,0x8c,0x83,0x91, -0xeb,0xcb,0x4e,0xe1,0xa6,0x5c,0xcc,0x5c,0x6f,0x44,0x23,0x59, -0x1e,0xa2,0x6a,0x5f,0x61,0xca,0xe4,0x1a,0x99,0xb1,0xd4,0x52, -0xb8,0xc6,0x08,0xed,0xa9,0xcd,0xa1,0xf2,0x8d,0x04,0x18,0xb6, -0x5f,0x3b,0xcc,0xcc,0x4c,0x3c,0x31,0x30,0x12,0x5b,0x64,0x6c, -0xe9,0xe6,0xba,0x9c,0x3f,0x9a,0xd6,0x65,0x4a,0xc0,0x4c,0x0a, -0xae,0x6a,0xa7,0x77,0x94,0xc2,0x6c,0xba,0x96,0xc7,0x58,0x0a, -0x86,0x7a,0x33,0x9f,0x17,0x05,0xe4,0x40,0x93,0x61,0xfe,0x9e, -0x66,0xb3,0x2c,0x0f,0xca,0xe1,0xba,0xd8,0x35,0x17,0xdc,0x2f, -0xe5,0xf5,0xf2,0x6d,0x19,0x2d,0xbb,0x8f,0x5a,0x71,0x95,0x99, -0x3b,0x4c,0xfc,0x2b,0x96,0x4d,0xb8,0x50,0x27,0x79,0xf8,0x8d, -0x2b,0xc5,0x4e,0x16,0x41,0x87,0xa0,0xfa,0x08,0x2e,0x02,0xa2, -0x89,0xc8,0x02,0x80,0xb6,0x7c,0x9a,0x58,0x3e,0xa3,0x69,0x84, -0xff,0x16,0xf8,0x4f,0xff,0x62,0xb9,0xb8,0x98,0x6e,0x74,0xcc, -0xd7,0xd3,0x15,0x95,0x30,0x31,0x7a,0x01,0xaf,0x98,0x8e,0x5c, -0xd6,0xf6,0xcc,0xad,0xf0,0x4c,0xe5,0x54,0xad,0x68,0x9f,0x15, -0x63,0xbb,0x0b,0xeb,0x4b,0x55,0x82,0xa3,0x5e,0xcd,0xab,0x4d, -0x9a,0x1c,0x25,0x59,0x36,0xbc,0x5c,0xae,0xd3,0xaa,0x38,0x31, -0x73,0x06,0x91,0x59,0xfc,0x70,0x7e,0x5a,0x0d,0xab,0x5e,0x2f, -0xe3,0x85,0x7c,0x14,0x92,0xca,0xb8,0x9a,0x4c,0x10,0x57,0x1d, -0x5a,0x43,0x55,0xfd,0xe4,0x7a,0xb5,0x79,0x8f,0x73,0xf7,0x13, -0xad,0xc1,0xdc,0x62,0x81,0x32,0x38,0x31,0x73,0x48,0xcc,0x84, -0x97,0xec,0x98,0x0d,0xa5,0xd2,0x47,0xb4,0x8c,0x1a,0xcd,0x7a, -0x53,0x93,0xe1,0xfa,0x8d,0xe6,0x7c,0x5e,0x4e,0x17,0x44,0x3f, -0x63,0x9c,0xe9,0x89,0xe9,0x9c,0x64,0x0a,0x48,0x7d,0xb3,0x5a, -0x2d,0xd7,0x9b,0x3e,0x77,0xf4,0x84,0x27,0x5b,0xf8,0x5f,0x87, -0x97,0x67,0x9f,0x37,0xf0,0x51,0x0c,0x8b,0x9b,0xe2,0x02,0x97, -0x47,0xb6,0xf3,0xe8,0x1b,0xc1,0x7c,0xb8,0xd9,0xac,0x99,0xaa, -0xdf,0x94,0xef,0x69,0x6d,0x12,0xdd,0xd0,0x3a,0x0e,0x16,0x29, -0xee,0xd4,0x01,0xe5,0x32,0x85,0xd3,0x64,0x25,0x58,0xfc,0x38, -0xe9,0x41,0x13,0xfd,0x75,0xb9,0x9a,0x4f,0x2f,0xca,0x74,0x7d, -0x7d,0x33,0xdf,0x54,0x8f,0xa7,0xf5,0x95,0x49,0x8e,0x3f,0x1b, -0x24,0x19,0xf0,0xbd,0xef,0x97,0xb7,0xe5,0x9a,0x66,0x3a,0x1b, -0x6a,0x27,0xd4,0x26,0x2c,0x6c,0x04,0xa4,0x3a,0xbf,0xd9,0x28, -0x15,0x34,0x76,0x1e,0x86,0x6d,0xb3,0x7e,0x7f,0x47,0xf5,0x92, -0xcd,0xfa,0xa6,0x44,0xe1,0x00,0xbf,0x48,0x5c,0xb8,0x9c,0xce, -0x6b,0x2f,0x65,0x90,0x27,0x38,0x62,0x9b,0x80,0x1f,0x79,0x0f, -0x7f,0xf6,0x12,0x9b,0x48,0xdf,0xf9,0xfa,0x7c,0x0d,0x20,0xf7, -0x37,0x65,0xbd,0x21,0xa0,0x32,0x9d,0x94,0xd5,0x74,0x5d,0x97, -0xc8,0xad,0x39,0x39,0xc7,0x7f,0x77,0x17,0xb4,0xa9,0xc3,0xf8, -0x77,0x52,0x6a,0x66,0x39,0x83,0xc5,0xe1,0xae,0x04,0x58,0x08, -0x64,0x87,0xc3,0x9d,0xec,0x40,0xd4,0x88,0x27,0x5d,0xc5,0x64, -0x62,0x65,0x2c,0x44,0x04,0x91,0xb7,0x2e,0x06,0xcc,0x01,0xbc, -0xa5,0x84,0x6f,0x94,0x82,0x78,0x8f,0xe8,0xb4,0xd1,0x2b,0x36, -0xc3,0x4b,0x06,0x89,0x2d,0xe1,0x4d,0x47,0xeb,0x08,0x01,0x77, -0x06,0x22,0xbc,0x80,0xbc,0x61,0x01,0xe2,0x94,0xe7,0x80,0xdd, -0x34,0xbb,0xdb,0x97,0xfd,0x2d,0xe2,0xda,0xe5,0x0f,0x5c,0x7e, -0x5d,0x9d,0xcf,0x61,0xde,0xd2,0x8b,0x9b,0xb5,0x99,0x55,0xeb, -0xec,0x6e,0xb6,0x3c,0x82,0xdf,0x05,0xfc,0x37,0x86,0xef,0xc9, -0x10,0x84,0xa0,0x79,0x89,0xd9,0x40,0x53,0x00,0x0e,0xfc,0x70, -0x24,0xa5,0xb2,0x14,0x24,0xba,0x16,0x81,0xb2,0x17,0x20,0x32, -0x22,0x82,0xcb,0xc5,0xa6,0x36,0xbf,0xde,0x4c,0xe7,0xd5,0x65, -0x55,0xae,0x01,0xdd,0xe5,0x8a,0xc8,0xd5,0x26,0x15,0xf6,0xd7, -0x76,0x7b,0x62,0x2c,0x5e,0xbe,0x55,0x69,0xcb,0x66,0xeb,0x1a, -0x56,0xb1,0xed,0x15,0xd0,0xad,0xeb,0xc2,0x4a,0x67,0xcc,0x83, -0x79,0x3a,0xa0,0xc2,0x5f,0xa7,0xf3,0xa2,0xd3,0xb1,0x8d,0x00, -0xeb,0x9a,0xcf,0xa5,0x8c,0x21,0xb6,0xaf,0x03,0x90,0xb2,0x45, -0x81,0x20,0xee,0x88,0xda,0x5d,0x2d,0x3b,0xdc,0x8f,0x06,0x41, -0x11,0x7d,0x84,0x1f,0xd0,0xaa,0x1b,0xaf,0xdf,0x43,0x63,0xc9, -0xb8,0xd1,0xd2,0x00,0x2e,0xab,0x39,0x6c,0x68,0xe5,0xac,0xf8, -0x84,0x0e,0x1b,0xcb,0x9e,0xbb,0xaa,0xea,0x67,0xd5,0xf5,0x6a, -0x2e,0x6b,0x66,0x2f,0x56,0xb9,0x47,0x97,0x6f,0x14,0x04,0xd3, -0xa1,0xb9,0x1b,0xba,0x81,0x7c,0xa8,0x46,0xb6,0xfb,0x74,0x6c, -0xe9,0xfc,0x2f,0x78,0x37,0xa3,0x89,0x72,0xb0,0x9e,0x16,0x27, -0x16,0x7d,0xf1,0x41,0xe0,0xd9,0xf4,0xb2,0xfc,0x76,0x3d,0x7d, -0x85,0xad,0xa7,0xb3,0xe5,0xc5,0x0d,0xfe,0x90,0x93,0x4f,0x55, -0x6f,0x0a,0xc4,0x06,0x0a,0x22,0xb5,0x6e,0x45,0xdb,0x24,0x33, -0xb5,0x54,0x2a,0xb4,0x42,0x9f,0x1b,0x7b,0x2c,0x9f,0xb6,0x41, -0x42,0xa1,0x96,0x96,0x42,0x4f,0x78,0x28,0x19,0xae,0xf7,0x21, -0xf6,0xa1,0x3b,0x59,0xd6,0x5e,0x30,0xa5,0x32,0x24,0x9c,0x5a, -0xaa,0xd3,0x92,0x6e,0x34,0xc0,0x78,0x66,0x3f,0xad,0x1f,0xae, -0x56,0x28,0xe7,0xd1,0xf8,0x37,0x78,0xd8,0xf0,0xa8,0x09,0x79, -0xaf,0x34,0x59,0x7f,0xf3,0xfe,0xf9,0xf4,0x15,0x0e,0x2b,0xc5, -0x52,0xe3,0x13,0x90,0x66,0xa9,0xc8,0x94,0xea,0x3f,0x82,0x85, -0xcb,0x8d,0xf4,0x97,0xb7,0x8b,0x72,0xfd,0x38,0x1c,0xa5,0xc2, -0x85,0x55,0x3d,0x7c,0xce,0xaa,0x7a,0x7a,0x3e,0x2f,0x9f,0x5d, -0xac,0xab,0xd5,0x46,0xa6,0x07,0xd1,0x38,0x05,0x7e,0xdf,0x64, -0xfe,0xb8,0x23,0xa6,0x09,0x92,0x6f,0x62,0x47,0x45,0x85,0xe8, -0x74,0x99,0x62,0xa5,0x6e,0x17,0xff,0x05,0xbc,0x97,0x17,0x38, -0x8f,0xb3,0xac,0x97,0xdc,0x4f,0x7a,0xb6,0x10,0xad,0x43,0x9f, -0x5d,0xd5,0x1b,0x10,0xe8,0x1b,0xdd,0xd3,0xc9,0xac,0x58,0xd7, -0x94,0x8e,0x74,0xfd,0xc3,0xb4,0x7e,0x53,0xce,0x40,0x92,0x2a, -0x2f,0x52,0xdb,0x98,0x85,0x81,0x8a,0x8f,0x1c,0x24,0xf4,0x3d, -0x1e,0xf0,0xd1,0x05,0xf6,0x3c,0x14,0x10,0xdd,0x0e,0x26,0x03, -0x88,0x40,0xa9,0xcb,0xcd,0xbf,0xcf,0x97,0xe7,0xd3,0xf9,0x93, -0xb7,0x53,0xe6,0x1a,0x35,0x08,0x9c,0x97,0x8a,0x7c,0x38,0x71, -0xc3,0xd4,0x23,0x6c,0xcc,0x50,0x8a,0x93,0x21,0x6e,0x59,0x9d, -0x82,0x8a,0x12,0xaa,0x6a,0x10,0x5c,0x32,0x12,0x6b,0x84,0xb2, -0x5f,0xba,0xad,0x27,0x79,0x65,0xdb,0x4e,0x4c,0xc7,0x6b,0xd7, -0x1e,0x5d,0xb8,0xb0,0x97,0x03,0xad,0x05,0xd5,0xfc,0x69,0xbb, -0x98,0x2f,0x17,0xe5,0xa3,0xe5,0xea,0xfd,0x93,0xb7,0x38,0xa9, -0xf5,0xfa,0xc2,0xcc,0x00,0x95,0xc4,0x6d,0x91,0x27,0xe0,0x87, -0x77,0x6a,0x97,0x2e,0xae,0xa6,0x35,0x89,0x30,0x50,0x5c,0xcf, -0x9c,0x38,0x25,0x28,0x06,0x2f,0xe7,0xb3,0xc7,0xde,0xd1,0x83, -0x81,0xc1,0x72,0x06,0x98,0x7d,0x33,0x07,0xdb,0xd7,0x3a,0x80, -0x48,0x04,0xa2,0x2e,0xe4,0xbb,0xcf,0x9f,0xb8,0x88,0xf8,0x17, -0x6c,0x32,0x22,0xf6,0x70,0x53,0x00,0xc7,0x62,0x36,0x2f,0xb5, -0x65,0x29,0x8f,0x67,0x75,0x44,0x31,0x82,0x84,0xbb,0xa9,0xd4, -0x75,0xf2,0x23,0x27,0x8c,0x31,0x7f,0x12,0xcb,0x91,0x2a,0x68, -0x63,0x91,0xfe,0x74,0x36,0x63,0x00,0x99,0xe0,0xbc,0x6a,0x38, -0x41,0x3b,0xed,0x16,0x07,0x02,0x72,0x9e,0xff,0x19,0x9d,0xbc, -0xe0,0x80,0xe8,0xe7,0xc2,0x04,0xf8,0x4b,0xf7,0xdd,0x23,0x9c, -0x04,0x5c,0x12,0x4f,0xeb,0xfa,0xa6,0xac,0xbd,0x59,0x20,0xf1, -0x40,0x78,0x90,0x61,0x1d,0xc0,0xb0,0x6d,0x66,0xf8,0x10,0x20, -0x05,0x5d,0x16,0x7e,0x85,0xf2,0x98,0xe9,0x44,0xb2,0xe6,0x62, -0x49,0xbd,0xd3,0xf4,0x77,0xbb,0x58,0x33,0x3e,0x7c,0x64,0x77, -0xb3,0xd6,0x69,0x63,0x91,0x9c,0x50,0x3c,0x73,0xd8,0x57,0x0c, -0xf2,0x5a,0x61,0xaa,0x22,0x1c,0x32,0xf4,0x32,0x65,0xd9,0x90, -0x60,0x8c,0x17,0x54,0xd8,0x73,0xb6,0x4b,0x78,0xdd,0xa2,0x18, -0xa7,0xc3,0x61,0x18,0x61,0x3f,0x7a,0xb7,0x01,0xa9,0x02,0x30, -0x45,0x3f,0x47,0x69,0xc8,0x7f,0x08,0x3e,0xca,0xb1,0x45,0x4c, -0xc8,0x22,0xa8,0x44,0x96,0xbb,0xe3,0xb8,0xed,0x61,0x44,0x79, -0x28,0x13,0x02,0xec,0x38,0x2b,0x30,0xb7,0x94,0xb2,0x04,0x10, -0xd7,0xdf,0x3d,0xff,0xe1,0x7b,0x6a,0xd3,0x7e,0x65,0x26,0x42, -0xe9,0xd5,0xe6,0x7a,0xfe,0x15,0x61,0xb5,0xdb,0xc5,0x92,0x20, -0xd7,0x70,0xc9,0x6e,0x57,0xb1,0x0f,0x03,0xbe,0xe6,0x46,0x6d, -0x66,0xa6,0xdd,0xd8,0x94,0x22,0xa8,0x8c,0xc0,0x56,0x8b,0xd5, -0x4d,0x84,0x8d,0xeb,0xe9,0xa2,0x5a,0xdd,0xcc,0xa7,0x48,0x4b, -0x2f,0xd7,0x70,0x06,0xb8,0x78,0x83,0x78,0x40,0xa2,0xe0,0x4d, -0x9b,0xc6,0x8f,0x24,0x22,0xe3,0x02,0x29,0x75,0x0a,0x22,0xfb, -0x23,0x2c,0x09,0x32,0x02,0xa5,0x5d,0xc8,0x07,0x96,0x95,0xdf, -0x44,0x80,0x7d,0x60,0x17,0x37,0xa5,0xe0,0x99,0x7e,0x2b,0x90, -0xf4,0xe1,0x92,0x09,0x93,0x2b,0xd1,0x5f,0x38,0x4c,0xfa,0x1d, -0x3e,0x83,0xa5,0x7b,0xb1,0xd1,0x1e,0x6b,0xfd,0xc2,0x26,0xa2, -0x12,0x79,0xda,0x1c,0xe8,0x76,0x9b,0xe0,0x24,0xc2,0x9c,0x4c, -0xfd,0x64,0x8b,0x34,0x69,0xe2,0xaf,0x16,0x2c,0x3f,0xc1,0x5f, -0x6e,0xb8,0x0b,0x81,0x30,0x07,0x27,0x5c,0x22,0x0a,0xda,0x26, -0x95,0x19,0xd7,0x46,0x59,0xb2,0xb9,0x5c,0x82,0x3c,0xaf,0xf2, -0x95,0x14,0x6e,0xdd,0x42,0x51,0xb4,0x45,0x4d,0x12,0xc8,0x64, -0xf6,0x04,0x30,0x3a,0x54,0x01,0x37,0x4e,0x18,0xcd,0xbd,0x24, -0xcb,0xa3,0xe6,0x7f,0x45,0xd2,0x60,0x24,0x2c,0xd7,0x0f,0x71, -0x4b,0x38,0xd0,0x74,0x5c,0xd8,0x6b,0xd6,0x16,0xa6,0xb3,0x2f, -0x8d,0x84,0xb8,0x1f,0x8f,0x69,0x3c,0xa1,0x51,0xd6,0x85,0xb6, -0x74,0x81,0xfb,0x3d,0xd2,0x39,0x6c,0x21,0x92,0x76,0x60,0x47, -0xea,0x50,0x3f,0x56,0xa9,0xc5,0xd3,0xe0,0x44,0x8e,0x11,0x75, -0xd2,0x5f,0xdd,0xd4,0x57,0xbc,0x0b,0xeb,0xd9,0xf6,0xba,0x5c, -0xbf,0x2a,0x19,0x04,0x23,0x73,0x60,0x2b,0xd9,0xed,0x17,0x3e, -0xfc,0xd3,0xe8,0x76,0x0b,0x09,0x76,0xd3,0xb1,0x9d,0xf9,0x73, -0x37,0x0a,0x9a,0x1f,0x4b,0xd6,0x84,0xe7,0x2f,0xcb,0xe9,0x4f, -0xc0,0x6b,0x1f,0x07,0xe4,0x2f,0x92,0xc2,0x07,0x17,0x91,0x93, -0x14,0x80,0xe0,0xe8,0x23,0x5a,0x46,0x94,0x26,0x4b,0xc7,0xdb, -0x5e,0x81,0xfd,0xcd,0x96,0xeb,0x9f,0xd7,0xcb,0x15,0x41,0x5e, -0x6f,0xde,0xc3,0x76,0x45,0x27,0x33,0xd5,0xda,0x20,0xfb,0xa4, -0x64,0x15,0xa9,0xed,0x69,0x10,0xa9,0xf2,0x62,0x4a,0x15,0x45, -0x2b,0x73,0x35,0x5d,0x3f,0xdc,0xa4,0x27,0x78,0xc4,0xfe,0x4f, -0x10,0xd5,0x84,0xa5,0xf7,0x58,0x55,0x32,0xaf,0xe0,0x34,0x3e, -0xc8,0xcc,0x72,0x5d,0xbd,0xb2,0x75,0x80,0x96,0x2f,0xea,0xfa, -0x67,0x10,0x06,0xaa,0x77,0x20,0xc4,0xca,0x66,0x57,0x1d,0x1f, -0x0f,0x33,0xe9,0xdf,0xcf,0x87,0x49,0xee,0x49,0x8f,0x66,0x3f, -0x6c,0xf2,0x5b,0xfb,0xf1,0x8f,0xb8,0xdf,0x55,0xb3,0x59,0x29, -0xa7,0xa9,0x72,0x1e,0x9e,0x66,0xca,0x39,0x4b,0x99,0x26,0x59, -0x00,0x67,0xc4,0x45,0xac,0x1a,0x91,0xba,0x16,0xd1,0x06,0x78, -0xf8,0x6a,0x3e,0x7d,0x9f,0x64,0xee,0xac,0x8b,0x13,0x3a,0xad, -0x16,0x75,0x8b,0x3c,0xca,0x27,0x31,0x4f,0xee,0xba,0x5a,0xde, -0x02,0x00,0xa5,0x3b,0x2d,0x60,0x8a,0x13,0xb7,0xa4,0x75,0x5e, -0xe3,0x57,0x04,0xa9,0x21,0xe6,0x55,0xe3,0x9a,0x00,0x01,0xba, -0x7c,0x87,0xe2,0x01,0x5f,0x78,0x68,0x1b,0x56,0x3e,0xe0,0xbb, -0x05,0x2a,0x35,0xa4,0x7f,0x61,0x2d,0xd8,0xe5,0xc1,0x52,0x16, -0xa6,0xf2,0xe2,0xea,0x13,0xd6,0x80,0x54,0xb8,0x7d,0xc9,0x2a, -0x5a,0xa4,0x39,0x10,0x75,0xec,0xa8,0x8d,0xfc,0x2a,0x5c,0x13, -0x7d,0x05,0x1a,0x87,0x32,0x0a,0x9b,0xc3,0x9b,0x10,0xc4,0x24, -0xb0,0x09,0x29,0xb5,0xdd,0xa6,0xcd,0xaa,0x45,0x02,0x2d,0x93, -0x26,0xa4,0x99,0xd7,0xed,0x06,0x73,0x96,0x7d,0x22,0xc4,0x06, -0xa6,0xee,0xa5,0x2c,0x84,0xc7,0x9c,0x96,0xda,0x43,0x24,0xb1, -0x67,0xf8,0x5f,0x1e,0x01,0x9d,0x32,0xea,0x8b,0xb0,0x67,0x93, -0x5a,0x90,0x9a,0xa3,0xea,0x70,0x95,0xcc,0xb2,0x82,0x7d,0xf0, -0x70,0xb9,0x91,0x7c,0xe7,0xfb,0x09,0x2c,0x53,0x85,0x22,0xcf, -0xfa,0xef,0x9d,0x5e,0x9c,0x15,0x0f,0xe0,0x36,0x04,0x27,0xed, -0x19,0xed,0x53,0x45,0xb3,0x1c,0x4f,0x72,0x92,0x73,0x07,0x59, -0x70,0x52,0x42,0x98,0x82,0x33,0xc7,0xcf,0xcb,0xba,0xda,0x54, -0x6f,0xcb,0x1f,0xe9,0x76,0x8c,0x07,0x4c,0x4d,0x99,0xfa,0xe6, -0x7c,0xb3,0x9e,0x5e,0x6c,0xbc,0x13,0x11,0xd0,0xfc,0x7a,0x71, -0x73,0x4d,0x87,0x5a,0x3e,0x0b,0xf1,0x36,0x1e,0x9c,0x83,0xca, -0x7a,0xf4,0xc3,0x74,0x73,0xd5,0xbf,0x9e,0xbe,0x4b,0x4f,0x8c, -0x24,0xc1,0x61,0xe8,0x38,0xd5,0x16,0xb7,0xdb,0x93,0x2c,0xeb, -0xa5,0x9a,0xf5,0x00,0xc1,0x5d,0xbd,0x4b,0x64,0xce,0x1d,0x78, -0xd3,0x1b,0x3a,0x10,0xff,0xad,0x9a,0x6d,0xae,0x7e,0x5a,0x7f, -0x57,0x56,0xaf,0xae,0x36,0x9e,0x8e,0x1d,0x38,0xf6,0x1a,0x15, -0xa9,0xdf,0x2c,0xd7,0xb3,0x72,0xfd,0xcd,0xf2,0x9d,0x21,0xb4, -0x78,0x47,0xa5,0xaa,0xa0,0x32,0x40,0xc4,0xa9,0x57,0x6c,0x94, -0x9c,0xd3,0x4f,0xc0,0x10,0x31,0xfe,0xc5,0x26,0xc9,0x46,0x5f, -0xe6,0xc9,0x2d,0xf6,0x43,0x62,0x02,0x0a,0x21,0x83,0xfc,0x04, -0x11,0x01,0xb3,0xfc,0x25,0x09,0xf9,0xc5,0x83,0x2c,0xb9,0x9e, -0xae,0x5f,0x55,0x24,0xab,0x50,0xbb,0x4c,0xf7,0xbd,0x06,0x47, -0xa2,0xcc,0x1e,0x7c,0xb2,0x2e,0x16,0x0f,0x52,0x9d,0x13,0x05, -0x2e,0xf3,0x21,0x1e,0xa5,0x16,0x84,0xb0,0xd1,0xe3,0x26,0x9b, -0x5b,0xc1,0xb9,0x02,0xf5,0x37,0xfb,0x1b,0x56,0xf8,0x3a,0x1f, -0x6a,0x4a,0xc6,0x1f,0xb4,0xd4,0x4b,0x08,0xcf,0x89,0xdf,0x62, -0x96,0xb7,0x0f,0xf0,0x83,0xb0,0xb8,0x12,0x9d,0x0f,0xe0,0xea, -0xe3,0x61,0x51,0x22,0x83,0x66,0x02,0xb9,0xec,0x20,0x79,0x30, -0xf1,0x12,0x5d,0x3d,0x75,0x68,0x2f,0x3a,0x3c,0xb7,0xd1,0x94, -0xf3,0x66,0x71,0x79,0x59,0x4b,0xa3,0xb9,0x97,0xc0,0xcd,0x0b, -0x34,0x05,0x74,0xfb,0x8c,0x7e,0x09,0x1b,0xf2,0xa6,0xb4,0x88, -0xc4,0xfa,0xf3,0xe5,0xbb,0x67,0xd5,0x6f,0x80,0x0a,0x58,0xd4, -0x3c,0xd6,0x63,0x48,0x6a,0xdd,0xc8,0x6c,0x51,0x18,0xf8,0x40, -0x07,0x8e,0xd2,0xd8,0xc9,0x69,0x01,0xe0,0x6e,0xb7,0x7c,0x85, -0x05,0x3f,0x49,0x0e,0xc0,0x11,0xc0,0xf9,0xf0,0xd1,0xb3,0x67, -0xde,0xa8,0x75,0x02,0xa0,0x4e,0x54,0x85,0x67,0xc0,0x63,0x2a, -0xa2,0xff,0x35,0xb8,0xa4,0x81,0x55,0xac,0xde,0xb1,0xd0,0x82, -0x65,0x33,0x87,0xeb,0x61,0x03,0x79,0xde,0x60,0xa1,0xcd,0x7d, -0xc3,0xfd,0xa5,0x9c,0x57,0x28,0x0d,0x6d,0xb7,0x6f,0x49,0xd5, -0xd9,0xec,0x17,0xd3,0x49,0x85,0xfe,0xed,0x7c,0x39,0xe5,0x7e, -0x81,0x33,0xec,0x5c,0xd7,0xbd,0x8f,0x61,0x00,0x74,0x5f,0x7b, -0x70,0x6d,0x9b,0x78,0x04,0x8a,0xa4,0x1e,0x72,0x1d,0x4f,0xa9, -0xd1,0xdc,0x96,0xec,0x8e,0x44,0x64,0x34,0x5b,0x5e,0x58,0xe5, -0x5d,0xb0,0xeb,0xca,0xef,0xb1,0x16,0x9f,0x28,0xad,0x3a,0xb6, -0xad,0xc5,0x81,0xfd,0xdd,0x4c,0xe7,0x71,0xfb,0x06,0x9a,0xcd, -0x4c,0xbc,0x85,0xc1,0xa1,0xd5,0xd6,0xaf,0x2e,0xd7,0x28,0x76, -0xc9,0x5f,0x15,0xa5,0xd3,0xe4,0x6b,0x4e,0x38,0xa2,0x7f,0x79, -0xec,0xc5,0xe7,0x27,0x9f,0x1f,0x11,0x65,0xd3,0xaf,0x2b,0x42, -0x1b,0xfe,0xbc,0x7f,0x9a,0x64,0x44,0x6e,0x09,0xfc,0xf3,0x1c, -0xb0,0x97,0xd8,0xbd,0x2d,0x3f,0x9f,0x2f,0x2f,0xde,0x1c,0x75, -0xaa,0x6b,0x9c,0xc5,0x29,0xa2,0x2d,0x13,0xc5,0xde,0xf3,0x25, -0xaa,0x38,0xfb,0x3a,0x70,0xd5,0x44,0x22,0xc8,0x0a,0xce,0xf8, -0x64,0xd2,0x17,0x74,0xb3,0x05,0xc7,0x76,0xdb,0xc8,0x50,0xf9, -0x2b,0xeb,0x3b,0x14,0x42,0xb3,0xb7,0xeb,0x0a,0x55,0x62,0x5f, -0x77,0xe0,0x83,0x54,0x2f,0x78,0x0c,0x3e,0xfd,0x9a,0xff,0x3d, -0x5f,0xce,0xde,0x9f,0x26,0xd4,0x55,0xff,0x62,0xbe,0x24,0x4d, -0xc4,0xc7,0x20,0x92,0x3b,0x07,0x61,0x7b,0x83,0xe6,0x16,0x19, -0xab,0xda,0x1a,0x93,0xa4,0x68,0xb6,0x6d,0x7a,0x3b,0x4f,0xd8, -0xb8,0x36,0x6c,0x8f,0x7c,0xb2,0x76,0x09,0x31,0xa1,0x7e,0x93, -0xaf,0xc7,0x43,0xd4,0xe1,0x30,0x1c,0xe0,0xd1,0xb2,0x07,0x14, -0x79,0x12,0x46,0xa0,0xd5,0x64,0x3d,0x47,0xda,0x02,0xdf,0xf9, -0x0d,0x1c,0xbe,0x7e,0x9e,0xc2,0x28,0xeb,0x74,0x45,0x62,0x38, -0xda,0x83,0x18,0x58,0x0e,0xb3,0x0a,0x0b,0x4c,0xe7,0x06,0xf8, -0xaf,0x77,0x4f,0xe4,0xae,0xa2,0xf5,0xf2,0x16,0x6f,0x8b,0xb2, -0xc0,0x2a,0x05,0x1a,0xb0,0x8a,0xf2,0xca,0xbc,0xc5,0xdb,0x34, -0xdb,0xdc,0x76,0x4b,0xb7,0x60,0x6f,0xca,0x0d,0x33,0x09,0xee, -0x34,0x1b,0xa1,0xae,0x4b,0x00,0x78,0x9b,0xe5,0x4d,0xb0,0x7a, -0xc9,0x38,0xe9,0x35,0xad,0x17,0xde,0x8e,0xaa,0x1c,0x04,0xcc, -0x5e,0x32,0x49,0xcc,0xdb,0x06,0xd8,0xbb,0x6c,0x48,0x17,0x64, -0x00,0x74,0x00,0x82,0x36,0xd3,0x29,0x1a,0x96,0x44,0x99,0x07, -0x08,0x59,0x16,0x51,0x03,0xf1,0xed,0xd8,0x1e,0xf8,0xb0,0x08, -0x81,0x62,0x2f,0xc5,0x9a,0x20,0x39,0xd2,0x98,0xc1,0xb4,0xd2, -0xd1,0x07,0x6f,0x1e,0xea,0x9f,0xd6,0xcf,0xd7,0xd3,0x45,0x8d, -0xab,0xa6,0x4e,0xf1,0xc8,0x0d,0x94,0xb3,0x2e,0xed,0x19,0xc6, -0x22,0x14,0x45,0x4f,0x3c,0x18,0xc2,0x1e,0xb7,0x2e,0xeb,0x1a, -0x92,0x08,0xd9,0xd9,0x9d,0x5e,0xcb,0x74,0xfc,0x9b,0xcc,0xb0, -0x24,0x70,0x59,0x2c,0x5b,0xb4,0xb4,0xd1,0x4c,0x2a,0xf0,0x30, -0x3f,0x24,0x76,0x25,0x79,0xa4,0x99,0xd0,0x8f,0xba,0xa5,0x95, -0x50,0xd7,0xb7,0xdf,0x2c,0x29,0xa0,0x22,0x7b,0x69,0x46,0xc3, -0xe0,0x8b,0x09,0x6d,0xda,0xf6,0x01,0xf2,0x68,0xaf,0x37,0x19, -0x66,0x49,0x4f,0xef,0x54,0x31,0x15,0x48,0x7e,0x94,0x36,0xca, -0xda,0xb3,0x28,0xe9,0x23,0x8c,0x43,0xe6,0x58,0x4b,0x4c,0x8a, -0x96,0x34,0x04,0x2c,0xeb,0xdf,0x2c,0xea,0xab,0xea,0x72,0x23, -0xc0,0xe4,0x9f,0x56,0x99,0xd4,0x0e,0x54,0x33,0x30,0xef,0xc0, -0x3b,0x84,0xcd,0x87,0x66,0xda,0x88,0x31,0x17,0x1d,0xa0,0x2b, -0x20,0x17,0x31,0x12,0x33,0xaf,0x7f,0xfd,0xfb,0x77,0xbf,0xf8, -0x76,0x72,0xdc,0x9e,0x1d,0x37,0x2f,0x4e,0x55,0x68,0xe9,0xba, -0x97,0x52,0xe5,0xcc,0x83,0xbb,0x63,0xaf,0x2a,0x69,0x9d,0xee, -0x19,0x86,0x6f,0x33,0xb6,0x52,0xa0,0x7f,0x5a,0x7f,0x3b,0x45, -0x4d,0xcf,0x7b,0xd9,0xc0,0xa4,0x82,0x37,0x94,0xa2,0x59,0x36, -0x3d,0x3c,0x24,0x01,0x75,0x2f,0xdd,0x7a,0x8d,0x6f,0xb7,0x75, -0x59,0xbe,0x81,0x52,0x5e,0x4a,0x73,0x84,0x5e,0x85,0xc9,0x28, -0xae,0x30,0xea,0xa4,0x56,0xe9,0xd7,0x52,0x01,0xce,0x0c,0xcb, -0x6a,0x76,0x74,0x92,0x5b,0xa3,0x3a,0x4b,0x7a,0x96,0x26,0xda, -0xaa,0x99,0x78,0x3a,0x82,0xcc,0xce,0x00,0x78,0x90,0xd1,0x7e, -0x77,0x74,0x94,0x50,0xb0,0xd1,0x6c,0x2b,0x06,0xd2,0x91,0x16, -0x9a,0x23,0x5a,0x2a,0x89,0x26,0xb5,0x09,0x22,0x2c,0x04,0x54, -0x5a,0x38,0x94,0x00,0xe1,0x93,0x75,0x1b,0x97,0xc7,0x95,0xec, -0x31,0x9e,0xd7,0xd3,0x77,0x4f,0xf8,0x62,0x60,0x03,0x72,0x7e, -0x09,0xc2,0xe8,0x5a,0xb6,0xa4,0x59,0x59,0xae,0xc8,0xba,0xe0, -0x72,0x3e,0xdd,0xc8,0x6c,0x29,0x8b,0xc4,0x6a,0xcf,0xca,0xcd, -0x06,0xe0,0xad,0xfb,0x5e,0xfe,0x76,0x2b,0x17,0x1e,0x50,0x8f, -0x34,0x36,0xd0,0x18,0xfc,0x37,0x86,0xcf,0x49,0x64,0x80,0x95, -0x7a,0xd5,0x28,0x7f,0xc4,0x00,0xe4,0xd8,0x2f,0x59,0xb8,0x94, -0x2b,0x34,0xeb,0xca,0x28,0xb3,0xd0,0x56,0xec,0x66,0x86,0xf9, -0xf6,0x10,0x2e,0x77,0x1b,0x40,0xd4,0x32,0x0a,0xcc,0xcd,0xe4, -0x23,0x1c,0xed,0x77,0xa4,0xf4,0xff,0xa5,0x04,0x6c,0x2e,0x6a, -0xbc,0xe0,0x60,0x0a,0x44,0x95,0x3c,0xa7,0xe8,0xc5,0xf5,0xba, -0x26,0xdb,0x2a,0xb6,0x98,0xda,0x98,0xcb,0x4a,0xec,0x20,0x29, -0x81,0x6e,0x61,0x44,0x08,0xa9,0x8b,0x5a,0xe5,0x91,0xda,0x63, -0x89,0xde,0x94,0xd8,0xc6,0xbf,0xad,0xca,0xf9,0x0c,0xb3,0xc2, -0x84,0xe0,0x8e,0x28,0xcc,0xca,0xe2,0xe4,0x1a,0xc5,0x63,0x84, -0x78,0x1c,0x16,0xe4,0xcb,0xa0,0x49,0x61,0xcb,0x71,0x02,0xeb, -0x16,0x86,0x30,0xeb,0x1e,0xaf,0x44,0x1a,0x19,0x66,0x8e,0xa8, -0x99,0xa4,0x33,0x18,0x66,0x68,0x80,0x93,0x42,0x42,0xdd,0xbf, -0xae,0xae,0x49,0xfd,0x08,0xf2,0x21,0xf6,0x8b,0x9a,0x64,0x45, -0xdf,0x77,0xe5,0x14,0x64,0xc3,0x34,0x79,0xc4,0xc3,0x3f,0x7e, -0x4e,0x97,0x90,0x74,0xbc,0xb8,0xe0,0x7b,0x65,0x05,0x5f,0x11, -0x84,0xba,0x3e,0xfd,0xcd,0x10,0x76,0xbb,0xe1,0x37,0xcb,0x02, -0x50,0x9d,0x2f,0x7b,0xc2,0x65,0xc7,0x37,0xa4,0xe7,0x20,0x1d, -0xbd,0xd9,0x89,0x39,0x8f,0x0e,0xc8,0xc7,0x51,0x16,0xcc,0x56, -0x38,0x6e,0xdc,0xc4,0xef,0xda,0x10,0x5e,0xd3,0x01,0xa8,0xe3, -0x17,0x06,0x5e,0x83,0x53,0xfb,0xb6,0x5c,0x23,0xb3,0x26,0xf8, -0x7a,0xc9,0x51,0xd2,0xf3,0xcb,0x4c,0x80,0x1f,0x07,0xbd,0x61, -0x29,0x01,0x31,0xa0,0x22,0x20,0xea,0xe0,0x9b,0x0a,0xc2,0x5a, -0x0c,0x2a,0x07,0x5f,0xdb,0x6d,0x50,0x41,0x4f,0x31,0x41,0x99, -0x51,0x1a,0x7c,0x76,0xc2,0x59,0x06,0x69,0xbf,0x81,0xc3,0xa0, -0x7c,0xe6,0x08,0x7f,0x1c,0x64,0x4c,0x94,0x0b,0x86,0xcb,0xe7, -0x11,0x63,0x23,0x75,0x34,0xcd,0xeb,0x05,0xb1,0xf4,0x00,0x6f, -0x16,0xf1,0xa6,0x0a,0x57,0x06,0x48,0x60,0xd7,0x2b,0xe3,0x90, -0x47,0x96,0xa9,0x81,0xcc,0xe0,0xb3,0x55,0xde,0xa6,0x33,0xdc, -0x64,0xde,0x86,0xd3,0x85,0x66,0x0b,0x54,0xf0,0x5b,0xda,0x50, -0xc8,0xce,0x92,0x3b,0x2e,0xfc,0x74,0x9b,0x6a,0x5c,0xb3,0x59, -0xe6,0x7a,0x1b,0x0f,0x26,0x44,0x8f,0x08,0x10,0x71,0x26,0x6f, -0x5e,0x33,0x6f,0x8a,0xf1,0x67,0x28,0xba,0x4c,0x8a,0x80,0x06, -0xf0,0xe7,0x84,0x17,0x95,0x8c,0xd6,0x83,0xb7,0xd7,0xab,0x26, -0xa4,0xce,0x86,0xf5,0xc6,0x86,0x46,0x6b,0x32,0xe2,0xb0,0x29, -0x38,0xbe,0x6e,0x17,0xff,0x8d,0xb2,0xb1,0xd9,0xc2,0xeb,0x06, -0x8b,0x10,0xa9,0x49,0xa1,0x09,0x5d,0x88,0x68,0x6e,0x72,0xcf, -0xcb,0x31,0x1d,0xcc,0xb1,0xa3,0x7b,0x20,0xcb,0x4d,0x07,0x87, -0x02,0xef,0xf5,0x8a,0xda,0x7e,0xe0,0x59,0x31,0xe2,0xf4,0x00, -0x52,0x0a,0x0b,0x06,0x2e,0xf7,0x7d,0x40,0x60,0x59,0xa0,0xf4, -0x26,0x0c,0x92,0x01,0x6b,0x95,0xea,0x16,0x20,0x57,0x8c,0xe2, -0x56,0xa8,0xe7,0x49,0xde,0x48,0xe9,0x60,0x69,0xbe,0xac,0x26, -0x34,0x72,0x5b,0xc6,0x23,0x8a,0x15,0x51,0x45,0x75,0x7c,0x6c, -0x4e,0x94,0xb4,0x32,0x6f,0xf5,0x63,0x33,0xd4,0x88,0x30,0x15, -0xc0,0x2c,0x80,0xb5,0xb9,0x5a,0x2f,0x6f,0xeb,0x64,0x92,0x59, -0x3a,0xc1,0x2c,0x4b,0x1f,0x22,0xc5,0xa3,0x51,0xdf,0x9e,0x02, -0xce,0xda,0x8e,0xd7,0xdb,0x5d,0xbd,0x81,0xb3,0x58,0x9e,0x90, -0x5a,0x61,0x5d,0xae,0xd7,0xcb,0x75,0x62,0xe8,0x0f,0x8d,0x69, -0x04,0x59,0x3f,0x2e,0x05,0xe3,0x35,0x5d,0xed,0xac,0x97,0xd7, -0x80,0x1b,0xc1,0xde,0x66,0xe9,0xe6,0x6a,0xb7,0xdb,0x11,0x85, -0xeb,0x67,0xd8,0x41,0x7d,0x73,0x71,0x01,0xc2,0x73,0x62,0xd8, -0x42,0x50,0xe0,0xd9,0x35,0x0c,0x85,0xe0,0x28,0x3d,0x9b,0xae, -0x67,0xc0,0x8b,0x53,0xb6,0x4e,0xd4,0x2b,0x91,0xf2,0xf6,0x88, -0x6d,0x32,0xfb,0x7f,0xff,0xe1,0xfb,0xef,0x36,0x9b,0xd5,0x2f, -0xe5,0xaf,0x37,0xc0,0x4d,0x3d,0xfb,0xc1,0xb8,0xb1,0x87,0x17, -0xa8,0xa8,0x3d,0xd4,0x94,0x94,0x10,0x13,0xbf,0xe4,0x87,0xea, -0x62,0xbd,0xac,0x97,0x97,0x1b,0xea,0xe3,0xf9,0xf3,0x9f,0x93, -0xec,0x40,0xf3,0xdf,0xbe,0xfb,0x71,0x79,0x6b,0x8d,0xf6,0x50, -0x37,0xfc,0x1c,0x76,0x93,0xe5,0x0d,0x0b,0xd5,0x24,0x59,0x02, -0xfb,0xc4,0x42,0x9e,0xd1,0x62,0x66,0x38,0xc5,0x5e,0xb6,0x41, -0x0b,0x71,0xc3,0xcf,0x6f,0xcb,0x72,0x51,0xa7,0xd3,0x45,0x75, -0x4d,0x57,0x65,0xc0,0x39,0x96,0xab,0xda,0x5a,0x13,0x93,0x50, -0x4b,0x49,0x4e,0x84,0xc5,0x4f,0x56,0xdb,0x38,0x65,0xee,0xc5, -0x72,0x8e,0xf2,0x18,0x1e,0x71,0xd2,0x0d,0x36,0xc9,0x64,0xbf, -0x5c,0x89,0x14,0x2f,0xd6,0xc7,0x36,0x0b,0x65,0xa9,0x2c,0xbe, -0xa0,0x71,0x8d,0x1c,0xbc,0xa2,0x21,0x22,0xd5,0x92,0xa2,0x53, -0x67,0x73,0xbe,0x70,0x14,0x02,0xa3,0x5a,0xfc,0x7a,0x43,0x7f, -0xa8,0xe5,0x58,0x5f,0x85,0x85,0x61,0x45,0x55,0x20,0x60,0x04, -0xde,0x23,0x40,0x39,0x37,0x73,0x54,0x2b,0x2e,0x57,0xab,0x72, -0x16,0x43,0x6b,0x3b,0x73,0xa7,0x10,0xf5,0xa7,0x81,0x09,0x00, -0xba,0x76,0xd6,0x7a,0x8f,0xe5,0x1b,0xce,0x6f,0xd3,0xf9,0xed, -0xf4,0x7d,0xed,0xcf,0x9a,0x9a,0x39,0x57,0x17,0x6f,0xfa,0x64, -0x75,0x04,0x2c,0x05,0x7e,0x17,0x5e,0x11,0xe4,0xdd,0x0c,0x84, -0xb3,0xfe,0xb4,0x98,0xe7,0x25,0x80,0xe4,0x50,0xd0,0x74,0x03, -0x73,0xf1,0x49,0x06,0xf6,0x97,0xeb,0x69,0xb5,0x00,0x09,0xb3, -0xf0,0x34,0xff,0x16,0xf8,0x3e,0xac,0x99,0x35,0xd5,0xee,0xb9, -0xb4,0xd9,0xcd,0x9a,0x7e,0x1c,0x7b,0x8d,0x03,0x54,0x25,0xb0, -0x3f,0xdb,0xda,0xfd,0x66,0x71,0xb4,0xdb,0x04,0x44,0x5e,0x20, -0x13,0x1a,0x1c,0x63,0xf1,0xbd,0x38,0xeb,0x13,0x25,0x1c,0xbe, -0x8a,0x8b,0x0b,0xeb,0x54,0xaf,0x6f,0x80,0x06,0xb9,0x1b,0x4f, -0x8a,0x65,0x0c,0x03,0x91,0x6f,0xaa,0xcb,0xf7,0x7f,0xab,0x36, -0x7c,0x57,0x6d,0xc6,0x1e,0x4d,0x70,0x1d,0x87,0x10,0xa0,0xc0, -0xc1,0xa9,0xa4,0x5a,0xef,0x25,0x9b,0x9b,0xa7,0xb6,0x51,0x20, -0x84,0xe5,0xfc,0x6d,0xd9,0xd6,0xea,0x84,0x0f,0x24,0x0e,0xa1, -0x85,0xad,0x05,0x64,0x75,0x5d,0xc1,0xce,0x77,0x87,0x55,0x72, -0x4b,0x69,0x75,0xde,0x30,0x24,0x72,0x04,0x98,0x21,0x05,0xc6, -0x25,0x40,0x1c,0xbf,0x23,0x9b,0xb9,0xe9,0xfc,0xc9,0xb4,0x46, -0xd0,0x80,0x41,0x58,0x52,0xb5,0x27,0xc1,0x9f,0x6d,0x2b,0xb9, -0x4f,0xd1,0xe1,0x39,0x31,0xd7,0xf3,0xa3,0x9d,0xf7,0xbc,0x95, -0x6a,0x74,0x4e,0x73,0x7b,0x2e,0x92,0x04,0xc3,0x93,0x91,0xc3, -0xa1,0xd6,0xe3,0x20,0x79,0xc8,0x1d,0x00,0x6a,0xb1,0x21,0xc3, -0x4c,0x5d,0x06,0x54,0x92,0x11,0xe8,0xe6,0x16,0xc7,0x6b,0xb4, -0x52,0x94,0xde,0x0f,0x46,0xad,0xfc,0x24,0x2a,0x53,0x52,0xa6, -0xa5,0x84,0x06,0x89,0x91,0x02,0x81,0x7e,0x67,0x0c,0xfb,0x8e, -0x56,0xb4,0x83,0xf8,0xd5,0x72,0xb3,0x7c,0xa2,0x00,0x47,0x04, -0x2b,0x79,0xa3,0x3d,0x84,0x9b,0x9f,0x0c,0x1b,0x6b,0x93,0x1c, -0x16,0x68,0x79,0x4a,0x3a,0x6c,0xac,0x9f,0x4e,0xe0,0x03,0x3b, -0x20,0x05,0xe1,0x63,0xa8,0xd1,0x48,0x61,0x10,0x3a,0xbd,0xe2, -0xb8,0xe1,0x1c,0x2e,0x4d,0x9e,0x2e,0xbb,0x5d,0xc6,0x04,0xea, -0x2d,0x53,0xfa,0xa6,0xc1,0xe0,0x2f,0x11,0x11,0x79,0x3f,0x38, -0x34,0x55,0xd9,0x5e,0xce,0xcd,0x3c,0xb5,0x8d,0x79,0xee,0xe1, -0xe4,0x6e,0xdd,0x44,0x3d,0xda,0x7b,0x10,0x6e,0xd2,0xda,0x9f, -0x1f,0xda,0xd6,0x5a,0xcc,0xca,0xe3,0x61,0xe0,0xb2,0xc8,0xba, -0xdd,0xb6,0x64,0xcf,0x66,0xdc,0x66,0xdb,0x26,0x2f,0xdf,0xf5, -0x37,0xb0,0x9a,0xd6,0x69,0xb8,0x76,0x91,0x95,0x1b,0x8f,0x03, -0x60,0xc5,0xdc,0x81,0x05,0x62,0xc5,0x4d,0x99,0x47,0x9d,0x51, -0x22,0x9c,0xd8,0x4d,0x30,0x0f,0xaf,0x50,0x21,0x18,0x83,0xab, -0xe9,0xa8,0xb8,0x5f,0x94,0x71,0x2e,0xa6,0xc5,0xf3,0x74,0xb1, -0x44,0xe3,0xef,0x4d,0x99,0xf5,0x2f,0xa7,0xd5,0x3c,0xae,0x81, -0x69,0x76,0x93,0x8a,0xf2,0x38,0xd5,0xdb,0x46,0x1b,0x24,0x11, -0xd2,0x80,0xbb,0xdd,0xe3,0x1b,0x21,0x9a,0x5f,0xc3,0xcb,0xd5, -0x5c,0x2d,0x97,0x6f,0x6a,0x77,0x5f,0x8f,0xb2,0x36,0x4f,0x91, -0xda,0xb0,0x34,0x3c,0x8c,0xa8,0x5c,0x26,0xf5,0x8b,0x90,0x33, -0xb0,0x5e,0x98,0x0d,0xd8,0xa8,0x19,0xbd,0xda,0x8f,0xf4,0xea, -0x2c,0x11,0xa0,0xbd,0x0f,0xb7,0x42,0xdf,0x20,0xbd,0xb7,0x54, -0x95,0xbc,0x13,0x15,0x53,0xc4,0xfd,0x02,0x9d,0xf7,0xa8,0x14, -0x75,0xc9,0x85,0x8c,0x6c,0xe1,0x7e,0xf5,0x8c,0x47,0xe8,0x5d, -0x28,0x7c,0x87,0xdf,0x02,0x29,0xe5,0x75,0xbb,0x09,0x5b,0x45, -0x26,0x30,0x78,0x4a,0x41,0x94,0x21,0x1c,0xf4,0x21,0x26,0x93, -0x02,0x73,0xd8,0x07,0xb5,0x12,0x62,0x8f,0x8b,0x85,0xc8,0x84, -0x93,0x72,0xcb,0x90,0x04,0x35,0x21,0x06,0xa5,0x84,0x30,0x53, -0xf6,0x81,0x69,0xc1,0xb1,0x14,0xf0,0x6c,0xc4,0xf9,0x4e,0xce, -0x2e,0xe3,0xd4,0x5b,0xaf,0xb4,0x4a,0x89,0x0a,0x88,0xbf,0xf3, -0xfc,0x8b,0x88,0x24,0x88,0x03,0x01,0xfd,0xd9,0xd5,0xf2,0xd6, -0x6c,0x96,0xaf,0x5e,0xcd,0x4b,0xe6,0xcf,0x8c,0x1d,0xb4,0x20, -0x86,0x53,0x3b,0xd3,0x2f,0xf9,0xf9,0xf1,0xcd,0xa0,0x77,0x57, -0x49,0x7b,0x1b,0x9e,0x87,0xd9,0x00,0x95,0xec,0xe0,0xc4,0x12, -0x25,0xf0,0x7b,0x88,0x4d,0x62,0x86,0x6e,0x95,0xa1,0xed,0x8a, -0x3f,0x4f,0x2f,0x29,0x95,0xa6,0x4a,0xee,0x7d,0x2f,0xdf,0xc1, -0x19,0x8f,0x2f,0x6a,0x79,0x5a,0x6e,0x16,0x54,0x06,0xd5,0x3a, -0x61,0x02,0xec,0x1a,0x02,0xb3,0x4e,0x20,0xba,0xf4,0xf4,0x69, -0x10,0x71,0x82,0x2f,0xdd,0x85,0xad,0x6c,0xb7,0xd2,0x48,0x8a, -0x6a,0xce,0x30,0xaf,0xd7,0x23,0x6c,0xb4,0x08,0x91,0x7b,0x92, -0xc3,0xea,0x70,0xfe,0x93,0x61,0xd2,0xb7,0x37,0xc0,0xbe,0x3a, -0x62,0xc7,0x70,0x22,0x10,0xc8,0x89,0x1a,0xbe,0x24,0x30,0xfa, -0x84,0x2f,0x2f,0x13,0x47,0x6d,0x72,0x65,0x6f,0xd7,0x32,0x14, -0x22,0x54,0x2f,0xe1,0x44,0x77,0x39,0x87,0x93,0xc8,0x98,0x6d, -0x63,0xf4,0xdb,0x84,0x9f,0x7f,0x8f,0xbe,0xff,0x31,0x31,0xd0, -0xd6,0xbc,0xfa,0x90,0x69,0x99,0x9a,0xeb,0xb4,0x15,0xba,0xc4, -0xeb,0x6b,0x2c,0x12,0xdf,0x85,0x73,0xc3,0xdf,0xe0,0xcd,0xea, -0x8f,0x65,0x39,0xab,0xbf,0x9f,0xbe,0x87,0xe3,0x14,0x34,0x25, -0x3d,0xa2,0x5e,0xe1,0x43,0xa6,0x50,0x23,0x86,0xf7,0xb7,0xe5, -0xf2,0xba,0x18,0xe4,0x91,0x89,0x16,0xb7,0x73,0x4c,0x77,0xb7, -0x49,0xc6,0x82,0x9d,0x1d,0x1a,0x1a,0x1a,0x05,0x63,0x2d,0x12, -0xa6,0xdc,0x24,0xb6,0x3c,0xae,0xaf,0xd6,0xd5,0xe2,0xcd,0xdf, -0xd6,0xd3,0x15,0xc1,0x5a,0xb3,0xfc,0xd3,0x32,0xd7,0x51,0x7b, -0x41,0x77,0xa8,0x0f,0x88,0x70,0x1d,0x15,0x18,0xc4,0x05,0xfe, -0x11,0x15,0x78,0x30,0xd9,0xf9,0x06,0x56,0x01,0xc3,0x6e,0x61, -0xbc,0xeb,0xcb,0x77,0x1b,0x52,0x3e,0x78,0x96,0x48,0xec,0x9e, -0xd8,0xe4,0x95,0xb2,0xfa,0x0b,0xfe,0x83,0x66,0xc1,0xf4,0x03, -0xe7,0x93,0xd9,0x04,0xb7,0x5f,0x14,0x62,0x67,0x36,0x42,0x5c, -0x95,0x09,0x1c,0xf4,0x81,0x75,0x00,0x6a,0x51,0xef,0x59,0x2d, -0x6e,0xca,0xa1,0xf0,0x01,0x16,0xf9,0x78,0xbf,0x40,0xfd,0x86, -0xc8,0x72,0x9a,0xcb,0x9f,0xac,0x15,0x45,0xe6,0xd3,0x66,0x11, -0x77,0xf9,0x8e,0xdb,0x8e,0x3c,0x30,0x82,0x4c,0x83,0xce,0xd2, -0x3a,0x6d,0x62,0x31,0xff,0x8c,0xac,0xc7,0xd4,0x1e,0x4e,0x53, -0xfa,0x62,0xec,0x26,0x03,0x45,0x03,0xe7,0x30,0xa7,0x50,0x73, -0x38,0x35,0x77,0x93,0x2b,0x6c,0x62,0x57,0x7d,0xec,0x2d,0xcd, -0x48,0x4a,0xe0,0x6d,0xde,0x9b,0xf5,0xa0,0x20,0xa2,0x85,0xd8, -0x46,0x6b,0x51,0x65,0xc4,0x43,0x1d,0xd2,0x3a,0x72,0x8e,0xb6, -0xa3,0xb6,0x02,0x1f,0xdd,0xcd,0x02,0x93,0xd5,0xbb,0x68,0x22, -0x11,0xc7,0xe2,0x89,0x01,0xb3,0x58,0x8e,0xf7,0xc2,0x1f,0xb4, -0xbe,0xc3,0x92,0x3a,0x0d,0x76,0xee,0xe9,0x84,0x40,0x00,0x7b, -0xa2,0x1b,0x1f,0x23,0xd4,0xf6,0x4f,0x90,0xc5,0x3d,0xe5,0x27, -0x99,0xd7,0x6f,0x11,0x66,0xda,0xf9,0x8a,0x20,0x65,0xc9,0xf6, -0xc8,0x9b,0x25,0xbc,0x1a,0x09,0xaa,0x16,0x04,0x09,0x4b,0x7a, -0xd2,0x33,0xba,0x5f,0x53,0x62,0x89,0x86,0xe3,0x5e,0xb6,0xf7, -0xdb,0xb3,0x53,0xc2,0x66,0x80,0x7a,0x85,0x2d,0x4a,0x02,0xda, -0xaa,0x05,0xce,0x21,0xde,0x29,0x48,0xcf,0x62,0xf6,0xf8,0x23, -0xdb,0xaf,0xaf,0x3d,0xa2,0xe2,0x28,0xe7,0x6d,0x96,0xb8,0xa0, -0x80,0x6f,0x55,0x9b,0xc3,0xb5,0x7d,0x03,0xac,0xc5,0xb7,0xef, -0x52,0xf6,0xe3,0x59,0x5c,0xcc,0x6f,0x66,0x25,0x59,0xeb,0x30, -0x29,0xdc,0x5e,0x55,0x17,0x57,0x06,0x1d,0xb2,0xea,0xe2,0x8e, -0x61,0x26,0xeb,0x75,0x52,0x2f,0xcb,0x5c,0xba,0x3a,0x85,0xff, -0x81,0x63,0xb2,0x96,0x77,0xc7,0x41,0xd3,0xd4,0x6a,0x11,0x58, -0x9e,0x51,0x17,0x63,0xb5,0x7f,0xeb,0x51,0x89,0x49,0x21,0xa9, -0xd6,0x58,0x4d,0x92,0x49,0xe9,0x6f,0xaf,0xe6,0x5c,0xc3,0x30, -0x15,0x54,0x03,0x44,0xd1,0xe9,0x45,0xb5,0x79,0xcf,0xf5,0xfb, -0x6c,0x49,0x43,0x17,0x00,0xdc,0x4f,0x64,0x7d,0x46,0x11,0x4b, -0xca,0x36,0xc7,0xc2,0xda,0xcf,0x24,0xeb,0xb2,0xfc,0x2f,0xf1, -0x26,0x37,0xf2,0xcd,0xc2,0xff,0x5a,0x95,0xb7,0xe2,0x54,0xc7, -0x7e,0x24,0xdc,0x40,0xde,0x19,0xec,0x58,0x59,0x34,0x9d,0xbd, -0xff,0xbe,0xaa,0x37,0x66,0xbd,0x5c,0x6e,0xb8,0x17,0xd3,0xb0, -0xf5,0xd7,0xfb,0x58,0x9b,0x60,0xd4,0xe0,0xa6,0x10,0x35,0xa1, -0x35,0xc0,0x01,0x6e,0xcf,0x0a,0x05,0xc9,0xd0,0x6f,0xf3,0x92, -0x1b,0xd7,0x74,0xe9,0xea,0xe5,0x67,0x9a,0xf0,0x99,0xb9,0x98, -0x4f,0xeb,0xfa,0x01,0x79,0xbb,0xa1,0x9b,0x52,0x18,0x15,0x03, -0xa5,0x24,0x4a,0x12,0x35,0x6b,0x91,0x0c,0xfa,0x7f,0xe9,0x0f, -0x12,0x4e,0x64,0x45,0x5d,0x23,0x92,0x06,0x27,0x73,0x11,0xe4, -0xae,0xcd,0x50,0x1b,0x90,0x28,0xc3,0x45,0x65,0x73,0x23,0x9f, -0x52,0xb9,0x00,0x2d,0xfc,0x9f,0x2e,0x1b,0x45,0x24,0x9d,0x0b, -0x6d,0x96,0xcf,0xe8,0x0a,0xbb,0x70,0x63,0xe9,0x6b,0x1a,0x97, -0xb8,0x9a,0xd6,0x3f,0xdd,0x2e,0xfc,0x7c,0x4e,0x11,0x35,0x88, -0x20,0x1f,0xbd,0x72,0x0a,0x7f,0xb8,0xe4,0xa7,0x23,0x9b,0xac, -0x93,0xc2,0x6a,0xf1,0xba,0x30,0xe2,0x6d,0x10,0x2c,0x41,0x3d, -0xe1,0x2d,0x78,0xf1,0xc5,0x65,0xbd,0x09,0xcf,0x04,0xdb,0xab, -0xc5,0xcd,0x75,0x71,0x7f,0xdc,0x3b,0x9e,0x8c,0xd2,0x51,0x7e, -0x36,0xbb,0x77,0xd6,0xdf,0x66,0x67,0xb3,0x1e,0x7c,0x8c,0xcb, -0x27,0x13,0xca,0x80,0xcf,0x6d,0x76,0xbf,0x5f,0x2f,0x6f,0xd6, -0x8a,0x19,0x6b,0xd1,0x51,0xdc,0x3f,0x7b,0xd6,0xbb,0xff,0xca, -0xac,0x09,0xfc,0xfb,0x2f,0xc6,0x67,0xf5,0xd9,0xcd,0xb7,0x4f, -0xbe,0xfd,0xf6,0xec,0xdd,0xc3,0x93,0x49,0x6f,0x1b,0x7d,0x7f, -0x86,0x45,0x7f,0xbd,0x81,0xa3,0x26,0xda,0x8d,0x40,0x79,0xe8, -0x27,0xfd,0x7a,0x7c,0x76,0x7b,0xf6,0xb7,0x49,0xef,0x34,0x1b, -0xbf,0x38,0x9d,0xdc,0xdb,0xfe,0x29,0x85,0x84,0xe3,0xc9,0xbd, -0x2c,0xfb,0xec,0xbe,0x59,0x23,0xa7,0x98,0x97,0xcf,0xa7,0xaf, -0xa0,0xf4,0xd7,0xe9,0xd9,0x6d,0x2f,0x3b,0xab,0xef,0x9d,0xdd, -0x1f,0x9d,0x42,0xd5,0xaf,0xcf,0xee,0x9f,0x0d,0x4e,0xb7,0x54, -0x0e,0xb6,0xde,0x6a,0x86,0x5e,0x0a,0x35,0x81,0x31,0x31,0xf9, -0xdd,0xee,0xac,0x9e,0xdc,0xb3,0x79,0xe4,0x7f,0x0f,0x99,0x50, -0xef,0xc5,0x36,0xdf,0x9a,0x0c,0x07,0x0c,0x4d,0x8d,0x33,0x1a, -0x00,0x95,0x29,0xeb,0x8b,0xe9,0x0a,0x07,0x75,0x86,0x08,0x48, -0xce,0xce,0xce,0xee,0x9f,0x5f,0x2e,0xd6,0x9b,0xc9,0xf6,0x66, -0x7c,0x36,0x9b,0x1e,0x5f,0x3e,0x3c,0xfe,0x76,0x72,0xf7,0xe5, -0x2e,0xb3,0x35,0x36,0xcb,0x37,0x70,0x7e,0x2f,0xee,0x27,0xe3, -0x17,0x58,0x7c,0x7d,0xb6,0x98,0xdc,0x4b,0xb6,0x18,0x37,0x60, -0x4b,0xb1,0x02,0xc8,0x74,0x72,0x7b,0xcc,0xd8,0xed,0xed,0xc5, -0x2e,0x34,0x77,0x2d,0xae,0x10,0x00,0xfe,0xf1,0x75,0x7d,0x0c, -0x60,0xcf,0xa6,0xf5,0xd5,0xc3,0xf9,0xea,0x6a,0x5a,0xdc,0x3f, -0x4e,0xa9,0xff,0xdf,0x26,0x50,0xb4,0x32,0x97,0xf6,0xb0,0xe9, -0x08,0x03,0x4e,0xfc,0x70,0x64,0xd9,0x6c,0xd0,0x11,0x5b,0x48, -0x82,0x3f,0x43,0x87,0x0d,0x9c,0x77,0x3e,0x57,0xcf,0x5c,0x5d, -0xf2,0x9e,0xcb,0xee,0xac,0x13,0x32,0x3a,0x1f,0x92,0xf7,0x1c, -0xb2,0x09,0xd4,0x83,0xc3,0x66,0x01,0xe2,0xe9,0x8c,0xcc,0x8b, -0xc9,0x3b,0x51,0xa2,0x25,0x69,0x5b,0x74,0xd7,0xac,0x95,0x89, -0xc1,0x3c,0xc3,0x1b,0x0e,0xf6,0x90,0x62,0xf3,0x39,0x63,0x7d, -0xf3,0x20,0x37,0x85,0x8d,0xc6,0x70,0x8e,0x7f,0xc0,0xd8,0xdb, -0xff,0x28,0xf5,0x5a,0xb7,0xbe,0x7d,0x9a,0x9b,0x26,0x8f,0x7f, -0xfa,0x41,0x6e,0xa4,0xbf,0x07,0x30,0xcb,0x59,0xe2,0x46,0x89, -0x5a,0x4f,0x23,0x0c,0xa7,0xb5,0x2e,0x0d,0x2c,0x2c,0x9f,0xe5, -0xae,0x3f,0x86,0x92,0x7d,0x09,0x93,0xe5,0x82,0xc0,0xa7,0xeb, -0x1b,0xa0,0xb5,0xc5,0xab,0xd2,0xab,0x69,0xbb,0x89,0xaa,0x44, -0x1d,0xc0,0xd0,0x87,0x76,0xa1,0xaa,0x48,0x67,0xb7,0xcd,0xe2, -0xee,0x35,0x79,0x57,0xe5,0x3e,0x2b,0xc0,0x05,0xcc,0xc6,0x21, -0xcb,0xb5,0xa8,0x5c,0x0d,0xae,0xf0,0x7c,0x2f,0x5b,0xf0,0x97, -0xba,0xb3,0xbc,0x27,0x1d,0x15,0x39,0x66,0x69,0xf9,0x40,0x15, -0xd8,0xe6,0xde,0x6f,0x0b,0xa2,0x48,0xcc,0x0e,0xcd,0xc9,0xd7, -0x38,0xdd,0x9a,0xe1,0x3c,0x83,0xe0,0x48,0x72,0xda,0x96,0x63, -0xbf,0x35,0x32,0x16,0x94,0x8c,0xd2,0x4e,0x8b,0x2f,0x46,0x63, -0x5c,0x29,0xc6,0x0e,0x04,0xbf,0x26,0xb9,0x63,0x16,0x2c,0x9f, -0x5b,0x70,0x4c,0x87,0x80,0xd9,0x6e,0x3b,0xea,0x26,0x2d,0x66, -0x05,0xc0,0x12,0xe5,0xea,0x41,0x3e,0xad,0x63,0x59,0x9f,0x31, -0x3b,0x4a,0x6d,0x86,0x87,0xa4,0x3e,0x7a,0xb1,0xbb,0xe6,0x73, -0x44,0x48,0xdf,0x43,0xbb,0x56,0x8a,0x0b,0x0e,0x15,0x2d,0x78, -0xdf,0x7c,0xa7,0xb6,0x0e,0xef,0x36,0xea,0xe1,0x86,0x66,0x3b, -0x9b,0xe9,0xe2,0x02,0x71,0xc9,0x5d,0xa9,0x17,0x1d,0x1c,0x75, -0x72,0x9d,0xad,0xc0,0x91,0x8c,0x54,0x08,0x7e,0x70,0x10,0xf4, -0xc0,0xb4,0x9d,0xe8,0x0c,0xdb,0xe1,0x3a,0x21,0x40,0x13,0x02, -0x4f,0x25,0x3b,0xfc,0xdc,0x6e,0xd8,0x1d,0x90,0xf3,0x3c,0xc6, -0xca,0xf6,0x17,0x76,0x10,0xd6,0xc6,0xa6,0xaa,0x7f,0x9e,0x4f, -0xab,0x85,0x06,0x96,0x91,0xf1,0xd3,0xb5,0x33,0x15,0xb6,0x56, -0x1e,0x90,0xda,0x54,0x57,0xe2,0x28,0xc6,0x54,0x6e,0x92,0x8d, -0xbc,0x0f,0x6d,0x48,0xf3,0x18,0xd3,0x28,0x12,0x71,0xab,0x26, -0xca,0x1f,0x7a,0x14,0x8a,0x67,0x24,0x32,0x9c,0xb5,0xab,0xd3, -0x39,0x39,0x7e,0xf3,0xfe,0xe9,0x4c,0x06,0xf1,0x60,0xc2,0xb6, -0xba,0xdd,0xae,0x27,0x04,0xa1,0x93,0x21,0x4d,0x10,0xa5,0x55, -0x33,0x38,0x36,0xdb,0xd2,0xaa,0x9f,0xb5,0xe4,0x10,0x4f,0x32, -0xc1,0x28,0xa7,0xb3,0x01,0xe9,0xa0,0x61,0xf6,0x48,0x04,0xdb, -0x79,0xe0,0xf5,0x75,0xee,0x2d,0xa6,0x29,0x55,0x9b,0xb1,0x0b, -0x83,0x95,0xd8,0xf6,0x86,0x54,0x56,0x82,0x9d,0xc6,0x34,0x68, -0x4b,0x3b,0x0b,0x2a,0x87,0xb0,0x64,0x79,0x13,0xfd,0x16,0xf6, -0x91,0x37,0x2a,0xe6,0xbd,0x8e,0xc8,0xdd,0xd2,0xd4,0x1f,0x2d, -0x41,0xb3,0x9a,0xf0,0xf7,0x43,0x58,0x14,0x52,0xb7,0xee,0x85, -0x28,0x2c,0x59,0x4f,0xdf,0x94,0xac,0xd6,0x0c,0x2a,0xe2,0x1e, -0xa0,0x09,0x79,0x92,0x18,0xbd,0xa9,0x30,0x75,0xf5,0x5b,0x99, -0x7b,0xdb,0x82,0x8f,0x62,0x2e,0xb4,0x83,0x03,0x2a,0xb5,0xd8, -0x52,0xcc,0xc9,0x76,0xac,0x06,0xa7,0xae,0x76,0xe8,0xf7,0xe9, -0x0a,0x83,0xd4,0xe3,0x04,0x27,0x52,0x99,0x41,0x0a,0x51,0x69, -0x5f,0x1a,0x86,0x63,0xec,0xc9,0xa9,0x26,0x8e,0xbd,0xae,0x7b, -0x90,0x38,0x21,0xa2,0x1d,0xe3,0xaf,0x9d,0x41,0x81,0x12,0x76, -0xbb,0x8b,0x37,0x79,0x10,0x20,0xc4,0x5e,0xcd,0x6e,0x8a,0xc6, -0xea,0x0e,0x18,0x0b,0xd3,0x6a,0xed,0x87,0x75,0xe9,0xa3,0xcd, -0x00,0xaf,0x3b,0x56,0x28,0x62,0x9a,0x4f,0x10,0x7d,0xcb,0xe6, -0xcb,0xcd,0xce,0xe0,0xdd,0xb7,0xeb,0x1c,0x07,0x7d,0x0e,0xe0, -0x18,0x38,0xc6,0xd4,0xf1,0x69,0x82,0xae,0xc9,0xa9,0xc9,0xb0, -0xd8,0xce,0x10,0x6d,0xb8,0x56,0x2e,0x17,0x71,0x55,0x2a,0x60, -0xef,0x0a,0x45,0x8f,0x7f,0xb9,0x90,0x2b,0x19,0x43,0x18,0xdf, -0x37,0x6b,0x16,0x47,0xa9,0x37,0x3b,0xd3,0xd5,0x6a,0xfe,0x9e, -0x81,0x01,0x18,0x6e,0x38,0x40,0x05,0x40,0x42,0x26,0x50,0xfb, -0x5a,0x2a,0x7f,0x85,0x9d,0x66,0x67,0x40,0x96,0x3e,0x54,0xe4, -0x18,0x6f,0x3a,0xcb,0x5f,0x5d,0x89,0xca,0x46,0x94,0x2c,0xbc, -0xb9,0x34,0xaf,0x8b,0x5e,0xd5,0x4b,0x4f,0x4e,0xab,0x11,0x24, -0xc0,0xf1,0x77,0xd8,0x0e,0xf4,0xeb,0xd3,0x82,0x03,0x3b,0x9e, -0xbe,0x1e,0x11,0x2d,0x8c,0x5f,0x4f,0x26,0xf9,0x78,0x02,0x9d, -0x5c,0x4f,0x57,0x4d,0xd4,0xef,0x1b,0xbc,0x17,0xd3,0x8c,0x86, -0xdd,0x08,0x01,0xa4,0x24,0x2c,0xed,0x34,0x63,0xff,0xec,0x10, -0x41,0x70,0x84,0xde,0x8b,0x68,0x4b,0x38,0xdb,0x6d,0x83,0xd2, -0x38,0x38,0x17,0x11,0x6c,0x6e,0x0f,0x48,0xa6,0x5e,0xae,0x37, -0x30,0x96,0x3e,0xfe,0x35,0x6c,0x81,0x43,0x9f,0xf4,0x6b,0x67, -0xc2,0x33,0x85,0x27,0xa8,0xd8,0x0c,0x13,0x5c,0x24,0xb9,0x74, -0x4d,0x88,0x34,0x3b,0x18,0x60,0xe2,0x62,0xb9,0x7a,0xff,0x94, -0xaf,0x3a,0xe8,0x37,0xdf,0xbb,0xa8,0x9e,0x80,0xe2,0x82,0x88, -0x2d,0x67,0x61,0x49,0x83,0xac,0xf3,0xc8,0xa6,0x6c,0x60,0x6f, -0xe9,0x35,0xcf,0xd9,0x33,0x94,0xab,0x42,0x6c,0x0e,0x92,0xf3, -0xe5,0x12,0xe3,0x9d,0x39,0x59,0x86,0x5b,0x24,0x99,0x14,0x8a, -0x89,0xe1,0x68,0xa3,0x9b,0x81,0x76,0xf3,0x20,0x33,0x0d,0x97, -0x03,0x2e,0x6d,0x95,0x37,0xfe,0x7e,0x47,0x39,0x19,0x46,0x2a, -0xe4,0x16,0x51,0xe9,0x26,0x70,0x16,0x45,0x85,0xcc,0x94,0xd3, -0x69,0xe6,0x8f,0x8f,0x2b,0x77,0xf5,0x48,0xbe,0xed,0x78,0x9d, -0xc4,0x8e,0xef,0x82,0x06,0x0f,0xa4,0x6a,0xc2,0xdb,0xae,0xf5, -0x42,0x90,0x1b,0x75,0x40,0xa5,0x0c,0x43,0xae,0x6b,0x10,0x97, -0x1a,0x44,0x54,0x7d,0x10,0x28,0x9f,0xdc,0xf9,0x57,0xef,0x65, -0xec,0x28,0x3b,0xd0,0xc7,0x9e,0x9d,0x7e,0xf5,0x1e,0xc7,0xe1, -0x4d,0x52,0x11,0x5d,0x4f,0x51,0x91,0x2c,0x1b,0xf9,0x65,0x82, -0x0f,0x98,0x04,0x9e,0x46,0xb4,0xe3,0xf5,0x24,0x0a,0xd9,0x06, -0xd6,0x17,0xd9,0x08,0xfe,0xc1,0x25,0x94,0xb7,0x15,0xf3,0xc1, -0xb1,0x85,0x61,0x52,0xfc,0xc1,0x46,0x61,0x4e,0xc8,0x7a,0x99, -0x29,0x87,0xa1,0xcb,0xf1,0x4f,0xbc,0x9f,0xf9,0xf5,0xb9,0x98, -0x73,0xff,0x27,0xd3,0x61,0x13,0xd9,0x3c,0x2c,0x96,0x70,0xb0, -0xb8,0x84,0xa5,0xe0,0xb1,0x1b,0xb2,0x35,0xd6,0x65,0xa7,0xba, -0x0c,0xab,0xd3,0x87,0x6e,0x6c,0xda,0xcb,0xcf,0x32,0xc3,0x08, -0x0f,0x74,0x20,0x2d,0x65,0x25,0x43,0x54,0x26,0xba,0x75,0x62, -0x6c,0xcf,0x5f,0x88,0x31,0x03,0x3e,0x89,0x01,0xff,0x6d,0x0a, -0x62,0xff,0xc0,0x5c,0x2d,0xe7,0xb3,0x5f,0x42,0x8e,0x8d,0x49, -0x78,0x91,0x32,0x9f,0x8d,0x7c,0x8e,0x8d,0x15,0x7a,0xbd,0x3c, -0x90,0x00,0x3a,0x27,0x4d,0x86,0x7f,0x0b,0xe5,0x48,0x3e,0xc2, -0x1f,0x6c,0xd7,0xd7,0x39,0x3e,0x8e,0x5b,0xca,0x5d,0x1c,0x38, -0xea,0x5e,0x2c,0x68,0x55,0x1c,0x23,0x1f,0xa5,0xa6,0xe9,0x97, -0xdf,0x4a,0x36,0x0c,0x5b,0x40,0xc7,0x04,0xec,0x52,0x8c,0x03, -0x9b,0x5d,0x9e,0x9e,0x00,0x2d,0x5a,0xa5,0x55,0x60,0x54,0x60, -0xe5,0x2c,0x09,0x1b,0x33,0xc9,0x3c,0x76,0x05,0x07,0x99,0x57, -0xaf,0xd0,0x7e,0xd4,0x79,0x59,0x89,0xeb,0x98,0xe4,0xa4,0x09, -0xb5,0x9a,0x64,0xe8,0x9c,0x69,0x3f,0xb2,0x1d,0x22,0x5d,0x97, -0xb5,0xc3,0x0f,0x45,0xcc,0x13,0xa7,0x05,0x2f,0x5e,0x6a,0xc3, -0x87,0x08,0x6b,0xb3,0x68,0x42,0xff,0x2a,0xcd,0x6f,0xb7,0xad, -0x2d,0xd9,0xf0,0xc3,0x6d,0xcd,0x88,0xde,0xae,0xad,0xe2,0x11, -0xf3,0x0a,0x48,0xe9,0x76,0xe1,0x1f,0x0e,0x70,0xcc,0xa4,0x44, -0xf1,0x60,0x61,0xa4,0xeb,0xea,0xa2,0xb5,0x6a,0x07,0xb2,0xa7, -0x3f,0xa6,0x9e,0xcf,0x24,0xb9,0x3e,0xe1,0xdd,0xe3,0xb7,0xc8, -0xe2,0xb5,0x7f,0x04,0x65,0x7f,0xdf,0xd4,0xe3,0x88,0xd5,0x5c, -0x94,0x99,0x37,0x78,0x25,0x7c,0xb7,0xc6,0x96,0xc5,0x8a,0x4e, -0x19,0x36,0x0e,0xb4,0x68,0xbc,0xe3,0x61,0x7b,0x13,0xe7,0xa9, -0x95,0xbb,0x8a,0x38,0x38,0x8f,0x39,0x44,0xf0,0x21,0x2d,0x72, -0xaf,0xfb,0x7d,0xbc,0xb6,0x5b,0x3f,0x16,0xb4,0xc7,0xc9,0xbd, -0xa8,0xd1,0xce,0x78,0x0d,0x2d,0x24,0xa1,0x55,0xac,0xe2,0xed, -0xa4,0xdd,0x6e,0xc7,0xd3,0xec,0x59,0x98,0x4d,0xe2,0x95,0xc1, -0xab,0xbc,0xd6,0x52,0x7e,0x43,0x6e,0x3b,0x35,0x09,0x0c,0x4c, -0x3f,0x7e,0xba,0x4c,0x1c,0x0c,0xb1,0x75,0xaa,0xa8,0x71,0xdf, -0x94,0xef,0x7d,0xc7,0x0b,0x3f,0x10,0x36,0x24,0x85,0x91,0x4c, -0x5a,0x81,0x85,0x52,0x44,0x65,0x5e,0x3c,0xc7,0x08,0x9d,0x7b, -0xe3,0x43,0x36,0x23,0x3b,0x8a,0x79,0xac,0xad,0x7f,0x5d,0xbf, -0xca,0xee,0xc8,0x30,0x97,0x14,0x93,0x4f,0x30,0x97,0x12,0x41, -0xfc,0xd0,0x23,0x6d,0x1e,0xb8,0xb5,0x59,0xcd,0x05,0x06,0xcb, -0xe3,0x38,0x49,0x9e,0x7d,0x3e,0x4c,0x69,0x9b,0xa7,0x50,0xe6, -0x51,0xe4,0xb0,0xb9,0xe1,0xdb,0xa3,0x72,0xea,0x35,0xaa,0x27, -0x73,0xed,0xb0,0x40,0x05,0x51,0x74,0x6a,0xdf,0x6e,0x95,0x5b, -0x90,0x0f,0x1c,0x81,0x3c,0x2b,0xbc,0x23,0x33,0x29,0x23,0x08, -0x02,0x53,0x4b,0xb3,0x1d,0xaf,0x8f,0x6e,0x77,0x6c,0xbd,0x77, -0xb9,0xf2,0x68,0x6c,0x43,0xde,0x04,0x9e,0x9e,0x9c,0x8b,0xe7, -0xee,0x49,0x2e,0x1f,0x4a,0xb2,0xe4,0x67,0x68,0xe3,0xf9,0x91, -0xa3,0x93,0x3d,0xfd,0x6b,0xaf,0xb6,0x7b,0xcb,0xe6,0x34,0xc3, -0xf9,0x7d,0x86,0x51,0x6a,0x26,0x3c,0x01,0x33,0x2f,0xf4,0x4e, -0xa6,0xb3,0x82,0xe1,0x3b,0xc3,0x59,0x89,0xb7,0x39,0x2c,0x61, -0x37,0x33,0xfc,0x60,0x0d,0xc5,0xa8,0x91,0x22,0xc1,0x4c,0x99, -0x55,0x70,0x1c,0x54,0x32,0x72,0x6e,0x0d,0xb9,0x2a,0xf7,0x8c, -0x45,0x10,0xa0,0x6a,0xaa,0x81,0x9c,0xbb,0x5d,0x4f,0xb9,0xeb, -0x02,0xa8,0xba,0x28,0xb0,0x9e,0xea,0xd6,0x24,0xff,0x3d,0xc9, -0xa2,0x1c,0x56,0xd1,0x9a,0x64,0xd2,0xc8,0x61,0x95,0xb0,0x49, -0x30,0x26,0xc7,0x08,0xe9,0xd4,0x4a,0x73,0x89,0x0c,0x9b,0x3d, -0x3f,0x32,0x38,0x0d,0x5a,0x73,0x29,0x22,0xe5,0xe4,0xe9,0x82, -0x5a,0x38,0x22,0x94,0x69,0x31,0xc3,0x0e,0x14,0x8a,0xce,0xbf, -0xc7,0x34,0xce,0x4b,0xea,0xdd,0xf5,0x1c,0x6d,0xf1,0x87,0x9f, -0x42,0xdb,0xc8,0x87,0x04,0xc7,0x8f,0x7f,0xfa,0xe1,0x67,0x32, -0x4c,0x1f,0x91,0x99,0x3f,0x82,0x6d,0x93,0x0c,0xb4,0x8d,0x66, -0xf5,0x3c,0x07,0xdf,0xc2,0x59,0x4d,0x38,0x34,0xad,0x30,0x8a, -0x62,0x75,0x1f,0x8a,0x24,0xa8,0xe1,0xc4,0xa2,0x58,0xf9,0x90, -0xb9,0x37,0x34,0x9c,0x64,0xd8,0x68,0x7f,0x5a,0xbf,0x5f,0x5c, -0x14,0x12,0xff,0x96,0x52,0x50,0xad,0x09,0x45,0x52,0x09,0x2a, -0x67,0xb9,0x14,0xb6,0xdb,0x88,0x47,0x0b,0x89,0xdd,0x2e,0xd6, -0x8a,0x1c,0xc0,0x81,0x47,0x62,0x6a,0x6b,0xac,0xaa,0xc0,0xfe, -0xde,0x59,0x7e,0xb4,0xcf,0x03,0xe2,0xda,0x4e,0x03,0x34,0xb9, -0x33,0x18,0xf4,0xdc,0x3f,0x26,0xed,0x8c,0x0b,0x43,0x18,0x4f, -0x0b,0x93,0x59,0x83,0xfc,0x9c,0x78,0x86,0xeb,0x9d,0xd7,0xb6, -0xb7,0x91,0x73,0x5d,0x2d,0x01,0xcd,0x32,0x73,0x95,0x67,0x0e, -0x28,0x77,0x97,0x71,0xa9,0x9d,0xb1,0x2a,0x7b,0x4f,0x69,0x4b, -0x73,0xe3,0x6c,0xe5,0x79,0x07,0x74,0xc1,0x8d,0xe5,0x42,0xc0, -0x24,0xd7,0xf5,0xb1,0x4f,0xbb,0xf6,0x5e,0xc0,0xbb,0x08,0xc8, -0x70,0xc0,0x6c,0xed,0x11,0xea,0x21,0x24,0xe2,0x93,0xef,0x11, -0xee,0x02,0xb8,0x05,0x9f,0xa1,0x17,0x8c,0x84,0xb3,0x08,0x13, -0x63,0x5d,0x03,0xee,0x23,0x91,0xbe,0xc1,0xd9,0xd2,0x55,0xce, -0x3a,0xd5,0x7b,0xc2,0x41,0xe4,0xa1,0x22,0x7e,0xf1,0x01,0x97, -0x03,0xb7,0x40,0xa1,0x5f,0x29,0x53,0x02,0x95,0xca,0x31,0x49, -0x03,0x04,0x15,0xf6,0x78,0xcc,0x1a,0x04,0xf4,0xb7,0xc6,0xdb, -0x5b,0xac,0x6c,0x34,0x68,0x1c,0x30,0x75,0x3a,0x56,0x39,0x17, -0xee,0xca,0x8b,0x6e,0xfc,0x91,0xed,0x14,0xd4,0x0e,0xbb,0x9f, -0xa8,0x2f,0xf9,0x47,0x82,0xa6,0xdb,0x2c,0xb6,0x58,0x19,0xfe, -0xf1,0x3b,0xa1,0x3b,0xd0,0x94,0x07,0xa0,0x0d,0x8d,0x85,0x42, -0x12,0x92,0x70,0x6e,0xaf,0x15,0x55,0x10,0xc1,0xdf,0xdc,0x5a, -0xf2,0xff,0xfc,0x9f,0xff,0xd7,0xff,0xfd,0x7f,0x24,0xd9,0xc8, -0xce,0x64,0x78,0x97,0x48,0x5c,0x9b,0x62,0x1b,0x26,0x49,0x1e, -0xd5,0xa5,0x92,0xbb,0xfc,0x63,0x6a,0x52,0x5e,0x2f,0xf1,0x69, -0x97,0x2e,0x35,0x21,0x05,0x35,0x27,0xa2,0x15,0x74,0x4d,0x81, -0x34,0x6c,0xd8,0x48,0xd6,0xd3,0x9e,0x49,0x02,0xb9,0x8e,0x07, -0xc2,0x2f,0x94,0xa6,0x08,0xe7,0x8e,0x8e,0x84,0x87,0x41,0x46, -0x16,0x05,0x67,0xc3,0xc7,0x28,0x1a,0x5b,0x0f,0x94,0x1b,0x8d, -0xe1,0x9f,0x49,0x8e,0x35,0x9c,0x4a,0x84,0xc7,0x89,0x55,0xa8, -0x25,0x56,0xb2,0x49,0x80,0xdf,0x68,0x71,0x21,0xc4,0x4e,0xb7, -0xc4,0x34,0xbc,0x96,0x7b,0x00,0x77,0x83,0x9c,0xf9,0xaa,0x49, -0x49,0x13,0xf3,0x61,0xa8,0x2f,0xca,0x1f,0x92,0xb2,0x50,0x41, -0x05,0x69,0x76,0xb1,0x14,0xd5,0x08,0xb5,0x53,0x9e,0x43,0x04, -0xe4,0xf4,0xaa,0x2c,0xaf,0x72,0x32,0x5d,0x71,0xca,0x03,0x22, -0x20,0x42,0x09,0xfc,0x03,0x14,0x22,0x26,0x02,0xf6,0x49,0x0a, -0xe1,0xc4,0xc7,0x03,0xc0,0x3c,0xa2,0xc4,0x53,0xf5,0xa1,0xbe, -0xcd,0xd4,0x25,0x88,0x16,0x62,0x5b,0x3e,0x2f,0xf8,0xcb,0x81, -0x41,0x65,0x9c,0xd6,0x8c,0xac,0xc9,0x9b,0x6f,0xa1,0xc8,0xa2, -0x38,0x7d,0x3d,0x7c,0x0d,0x40,0x51,0x1d,0x72,0xdd,0x97,0xe6, -0xc6,0xaf,0x27,0x8e,0xe0,0x87,0x36,0xcd,0x3f,0xa3,0x0f,0xdb, -0x6a,0xa1,0xf3,0xbf,0xf5,0x84,0x74,0x70,0x14,0x15,0xab,0x0a, -0x81,0xb1,0x03,0x79,0x45,0xda,0x57,0xc7,0x96,0xaa,0xc5,0x5b, -0x3f,0xb6,0x36,0xce,0x26,0xc5,0x5c,0x0b,0xe3,0xad,0x59,0x0f, -0x0f,0xb6,0x27,0x79,0x5b,0x74,0x3a,0xf0,0x6f,0xa8,0xa2,0xb1, -0xc1,0xb9,0xb5,0x6d,0xee,0x8a,0xd6,0x25,0x1a,0xdf,0xa2,0x6f, -0x1a,0x97,0x01,0x91,0x05,0xb5,0xb8,0x1a,0x88,0x90,0x23,0x18, -0x3a,0xfd,0x6e,0xa4,0x36,0x8c,0x00,0x06,0x06,0xb4,0x87,0x8d, -0xfa,0x80,0xb6,0x32,0x52,0xd6,0x24,0xcb,0x10,0x87,0xfb,0xb8, -0x15,0x0d,0x26,0x64,0x32,0xde,0x48,0xa8,0x7f,0xc3,0x4b,0x4c, -0xc3,0x6e,0x42,0x83,0x63,0x1c,0x11,0xb7,0x20,0xf6,0xb9,0x31, -0xf7,0xe2,0xce,0xff,0x15,0xed,0xfa,0xcb,0x85,0x0d,0x3a,0x84, -0x45,0xc3,0xbc,0x41,0x1e,0x6a,0xf3,0x6f,0xaa,0x59,0x3e,0x40, -0x8b,0xa2,0x77,0x81,0xd2,0xda,0x99,0x45,0x50,0xfc,0x67,0x60, -0xe5,0x5c,0x86,0x04,0xae,0x30,0xe4,0x40,0xcb,0x29,0x01,0xe5, -0xa9,0xcb,0x85,0x0b,0xe4,0xa8,0xa7,0x02,0x68,0xf7,0x72,0x81, -0x72,0x55,0x9b,0x5d,0xfe,0xe5,0x22,0x1b,0xd1,0xce,0x55,0xc4, -0x37,0x0f,0x56,0x81,0x67,0x1e,0x64,0x0c,0x46,0xd1,0x54,0xdb, -0x5e,0x2e,0x64,0x6c,0xf6,0xe8,0xa1,0x1a,0x71,0x35,0x66,0x49, -0xf7,0xb6,0x9b,0x91,0xe4,0x8e,0x0d,0xd3,0x9b,0x31,0x00,0x67, -0xf0,0x37,0x7a,0x50,0x86,0x4b,0x7a,0x21,0x3a,0x77,0x66,0x4a, -0xfe,0x87,0x31,0x25,0xc2,0x70,0x31,0x0c,0x00,0x93,0x1f,0x48, -0xdd,0xd5,0x02,0x43,0x30,0x19,0x32,0xb6,0xfd,0x77,0xe0,0x8a, -0xeb,0xe9,0xad,0xf8,0xa0,0xec,0x21,0xcd,0xf3,0x9b,0xf9,0x9b, -0x82,0xb7,0x82,0x37,0xfc,0xb6,0x8e,0x17,0xea,0xd6,0x3f,0x90, -0xe3,0x19,0x34,0xbb,0xb3,0x5d,0xa0,0x03,0x8a,0x25,0x27,0xcc, -0xd3,0xe8,0x03,0x04,0xa6,0x83,0xae,0x42,0xf8,0x24,0x56,0x58, -0x00,0x95,0xdd,0xa5,0x75,0xa3,0x0d,0x7c,0xdb,0xbd,0x6e,0x5a, -0xe6,0x91,0x69,0x0f,0x75,0x4d,0xd3,0x5b,0x74,0x31,0xa5,0x51, -0x20,0x8d,0x4e,0x6f,0x47,0x30,0xc9,0x4e,0x17,0x5f,0x8b,0xbf, -0x1e,0xd2,0x04,0xa9,0xd4,0xf3,0x94,0x06,0xcc,0x54,0x12,0xee, -0x0f,0x16,0x8f,0x76,0xbe,0xb1,0x28,0xb7,0xe5,0x5b,0x44,0x4a, -0x21,0xb4,0x60,0x06,0x7a,0x6a,0x59,0xae,0x97,0x0b,0xb7,0x94, -0xb0,0x51,0x84,0x8a,0xea,0x70,0xb8,0x3b,0x0f,0x3a,0x5e,0x6c, -0x51,0x79,0x2f,0xf2,0x99,0x45,0x03,0x19,0x8d,0xd5,0x39,0x02, -0x34,0x0a,0xc6,0x97,0xe5,0xe2,0x62,0x66,0x1b,0x39,0xe1,0x46, -0x72,0x45,0x36,0x0a,0x9b,0xb7,0xcd,0x3b,0x88,0x94,0x0e,0x23, -0x68,0xf8,0x81,0x12,0x3d,0x6a,0xff,0x40,0x64,0xdc,0x85,0xa6, -0x1f,0x7a,0x83,0x54,0x34,0x55,0x37,0x56,0xc9,0xc7,0xfe,0x37, -0xf2,0xd1,0xf4,0x4d,0x34,0x1f,0xb4,0x3b,0xd9,0xab,0x7b,0x54, -0xfa,0xd8,0x6b,0x69,0x92,0xed,0xcd,0xf9,0x68,0x2b,0x93,0x66, -0xc5,0x16,0x13,0x13,0x8e,0x4a,0xe0,0x3a,0xdb,0x7c,0xb2,0x95, -0x49,0x54,0x25,0xb6,0x32,0x21,0xc5,0xc5,0x66,0x49,0xd7,0x21, -0x78,0x7a,0xc4,0x9f,0xbc,0x24,0xa5,0x3e,0x45,0xb7,0xb2,0xc7, -0x30,0x67,0xef,0x12,0x1e,0xd0,0x3c,0x57,0x5f,0x68,0xa1,0xdb, -0x85,0x7f,0xa0,0x08,0x1c,0x85,0x96,0x73,0xd8,0xe5,0x3a,0xce, -0xbd,0x42,0xd2,0x28,0x26,0x2d,0x7b,0x82,0x36,0x54,0xc5,0x02, -0x86,0xad,0x0f,0x88,0x29,0x2f,0x37,0x49,0xc3,0xff,0xda,0x57, -0x1d,0x07,0xed,0x9a,0xaf,0xe0,0x6c,0xbd,0xc7,0xa0,0x68,0xb7, -0x4b,0xed,0xf3,0x0d,0x4e,0x5b,0xac,0xd7,0x95,0xac,0xd2,0xf4, -0x2f,0x41,0x93,0x6f,0x58,0x5b,0x74,0xc4,0x31,0x2b,0x8f,0xf8, -0xa0,0x6c,0x35,0x00,0x47,0xb4,0x6f,0x12,0x31,0x1f,0xfd,0x52, -0xbe,0x7a,0xf2,0x6e,0x75,0xc4,0xd2,0x25,0xeb,0xb2,0x12,0xdf, -0xb1,0xde,0x8b,0x77,0xc5,0x87,0x2d,0x4f,0xbb,0x99,0x8c,0x99, -0xf3,0x1d,0xb9,0x00,0x51,0x93,0xb6,0x83,0x55,0xe6,0x59,0xec, -0x14,0xb1,0xc6,0x9a,0xe6,0xd2,0x7f,0x1b,0x0e,0xc3,0xcc,0xcb, -0x60,0x1e,0xc9,0x4e,0x5b,0x7b,0x0b,0x4a,0x7d,0x83,0xf5,0x26, -0xa9,0xb1,0xe5,0x49,0xc6,0xa8,0xf5,0xbd,0x39,0xf5,0xb0,0x8c, -0x1f,0xaa,0x6b,0xba,0x83,0x6a,0xce,0x50,0x02,0xa8,0xa0,0xc9, -0xe3,0x75,0x79,0xbd,0x5c,0xbf,0x47,0xc9,0x0c,0x08,0x9d,0xd3, -0xbe,0xe7,0x5d,0x81,0x3f,0x9e,0x92,0x73,0x0d,0xff,0x7e,0x46, -0x36,0xca,0xf4,0xd8,0xc5,0x18,0xcd,0xfa,0x61,0x18,0x45,0x47, -0xfd,0x38,0x61,0xef,0x2b,0x51,0x8f,0x66,0x42,0x37,0x14,0x3e, -0x74,0x93,0x11,0x0a,0xf5,0x54,0xd8,0x77,0xf0,0xe8,0x93,0xe3, -0x5f,0x70,0xf7,0xc8,0xe7,0xbd,0x4e,0x0b,0xaf,0x53,0xf4,0x00, -0xf6,0x3e,0x8b,0x93,0x00,0xd4,0xc2,0x7b,0x1b,0x43,0x32,0xc8, -0x37,0x12,0x52,0xbb,0x5d,0xbf,0xe0,0xa9,0xd7,0xfc,0xd0,0xfb, -0xcd,0x22,0x39,0x96,0x1f,0x7b,0xa9,0x13,0xd9,0xe9,0x11,0x42, -0x0d,0x7c,0x80,0x9a,0x3f,0x3a,0xc5,0x75,0xbb,0x3a,0x10,0xf4, -0xc6,0xfc,0x89,0x5f,0xbf,0xc9,0xee,0x64,0x90,0xb0,0x88,0x6d, -0x80,0x11,0x82,0x66,0x60,0x18,0x9a,0x94,0xb0,0x36,0xa2,0x7f, -0xfb,0xfa,0x18,0x21,0x39,0xc4,0x70,0x92,0x44,0x7a,0xc9,0x72, -0x6e,0x68,0x24,0xc8,0xce,0xeb,0x72,0x7e,0xd9,0x97,0xb8,0xf5, -0xa9,0x58,0x62,0x5c,0x16,0x77,0xc0,0xbb,0xf2,0xd0,0x9f,0x1b, -0x2b,0xc8,0xc5,0x2d,0x61,0xca,0x7f,0x36,0xa4,0xe3,0xc7,0x33, -0x93,0xd3,0xbb,0xbf,0xc6,0x48,0xfe,0xf2,0x22,0x4a,0x59,0xc9, -0xd6,0xbf,0x40,0x26,0x41,0x00,0x73,0x86,0xc1,0xbd,0x0a,0x26, -0x2b,0x7d,0xf6,0x6f,0x16,0xd5,0xaf,0x28,0x30,0x12,0xd4,0x57, -0xd3,0x9a,0xca,0x6f,0xb7,0xfc,0x3c,0x09,0x4a,0xd9,0xf8,0x0d, -0xa7,0xb8,0x57,0x78,0xfa,0x79,0x65,0xf1,0xe0,0xf4,0x6a,0xf2, -0x96,0xa3,0x80,0x89,0xbe,0x42,0x9e,0x14,0x25,0x53,0x3c,0xda, -0x47,0x02,0xb9,0x92,0x56,0xea,0x93,0x0c,0x1b,0xd8,0x13,0xaa, -0x39,0x3f,0xcb,0x7c,0x63,0x20,0xbc,0x60,0x43,0x05,0x6c,0xcb, -0x4d,0x3d,0xcf,0x5c,0x84,0xa8,0x9b,0xe8,0x91,0x19,0xc1,0x96, -0x0d,0x42,0x4b,0x24,0x26,0x5e,0x0b,0xca,0x58,0xe5,0xcd,0x19, -0x28,0x48,0xd4,0xc0,0x8e,0x6b,0x59,0x76,0x7a,0x3c,0x18,0x66, -0x04,0xbf,0x06,0xd4,0xa0,0x35,0x37,0xd0,0x81,0xda,0x81,0x08, -0x11,0x17,0x94,0x1f,0x52,0xf6,0xf1,0xb1,0xbf,0x74,0xa2,0x22, -0x94,0x76,0x7c,0x9c,0xed,0xd4,0xf8,0x03,0xa6,0xa4,0xd5,0x74, -0xe4,0x52,0x1d,0x44,0x2c,0xac,0x20,0x24,0x11,0x45,0x01,0x8c, -0x79,0x27,0xed,0xe0,0xef,0xed,0xb6,0xe3,0x21,0x1b,0x75,0x4c, -0x28,0x65,0xec,0xc1,0x1b,0x32,0x0a,0xee,0x53,0x88,0x77,0x5f, -0x39,0xe6,0x26,0xb2,0x7c,0x9c,0x15,0x7b,0x50,0xb7,0xc5,0x8c, -0x82,0x40,0xd9,0xa1,0x4d,0xfb,0x9b,0x96,0x96,0xb9,0x51,0xd7, -0x1a,0x37,0x8f,0x01,0xca,0xfc,0xd5,0x24,0x9d,0x60,0x1b,0xad, -0x5d,0x50,0x2b,0x64,0xe7,0x42,0xd7,0x98,0x9e,0xfd,0x88,0x28, -0xf7,0x03,0xcb,0x1d,0x3a,0x5a,0xe0,0x3f,0x14,0x9f,0x8d,0xbe, -0xc6,0x7e,0x41,0xb6,0x58,0x1f,0xb9,0x9f,0x29,0xad,0x84,0x1a, -0xa4,0x63,0xc6,0x2e,0xf1,0x42,0xd8,0xab,0xa9,0x5b,0x8e,0x49, -0x84,0xd4,0xce,0xfc,0x41,0x57,0x0f,0xb0,0x77,0x22,0x65,0xfa, -0xa9,0x23,0xc0,0x94,0x36,0x2c,0xe0,0x68,0x15,0xfa,0xd8,0x80, -0xc7,0xab,0xda,0x36,0xf6,0x0e,0x65,0xec,0xec,0x8b,0xa6,0xd8, -0x56,0xe3,0xd2,0x5d,0x45,0x3d,0x8f,0xa6,0x28,0xca,0x21,0xb1, -0x8e,0x9b,0x15,0xc6,0xce,0x1d,0x8f,0x13,0xb9,0x09,0xc6,0xf0, -0xa3,0xe8,0x54,0x67,0xe2,0x0d,0x11,0xa5,0xa2,0x8b,0xf2,0x88, -0xe7,0x08,0xe3,0x90,0x4b,0x85,0x59,0x32,0x31,0x58,0x9b,0x0e, -0x24,0x26,0x41,0x4f,0xe2,0x8f,0xaa,0xfc,0x9a,0xa2,0xab,0x51, -0x65,0x0e,0xde,0x00,0x95,0xd5,0xad,0xb9,0xad,0x01,0xad,0x3b, -0xa1,0xad,0x6d,0x53,0x16,0x09,0x46,0xf3,0xa4,0xc0,0xbc,0x2a, -0x03,0x4b,0x20,0x98,0x56,0x42,0xdb,0x94,0x70,0x3e,0x23,0xe7, -0xb9,0x96,0x7c,0xeb,0x34,0xcf,0xde,0xd4,0x16,0xfb,0xe2,0x2b, -0x1d,0xcf,0xc6,0xe6,0xca,0x8f,0x7c,0x20,0x01,0xcf,0x7c,0x13, -0x93,0xe8,0x81,0x59,0x2b,0x6b,0x3b,0x8b,0xb9,0xf2,0x96,0x12, -0x43,0x06,0xcf,0x93,0xe1,0x0b,0x40,0x94,0x22,0xa7,0x6f,0x0e, -0xb3,0x42,0x29,0xb8,0xdd,0x39,0x6b,0xe7,0xe0,0xf4,0x4c,0xca, -0x11,0xe0,0x2c,0xf4,0x63,0xa8,0x03,0x1b,0x73,0xb5,0xc1,0x64, -0x12,0x7b,0x86,0x31,0xa4,0x68,0xbe,0xbe,0xc0,0x5a,0xed,0x66, -0x64,0x43,0x2d,0xe5,0x19,0x95,0xd8,0x3e,0x35,0x4f,0xa5,0xc3, -0x6c,0x14,0xa7,0xa8,0x79,0x9b,0x8e,0x5a,0xad,0x0e,0x04,0xbf, -0x5e,0x32,0x3d,0xdb,0xe8,0xbc,0xde,0x6d,0x0e,0x53,0x48,0x96, -0x6b,0xc2,0x98,0xb1,0x81,0x31,0xa9,0x37,0x57,0xc9,0x84,0xe0, -0x65,0xc7,0x2b,0xec,0x6f,0x64,0xeb,0x59,0x00,0x72,0xb6,0x10, -0x5b,0x8c,0xc6,0x0a,0x1c,0xaa,0x29,0x75,0x7c,0xe8,0xec,0x6a, -0x70,0x06,0x51,0xa4,0xdf,0x65,0xae,0xda,0x4e,0x69,0xeb,0x03, -0x66,0x03,0xa3,0x70,0xc9,0xa1,0x22,0x5f,0xb1,0x91,0xcb,0x8f, -0xdd,0xce,0xc5,0x87,0x71,0x4f,0x10,0x4b,0x66,0x7f,0x55,0xad, -0x4a,0x05,0xbf,0x8f,0xf4,0x65,0x3e,0x9a,0x30,0x88,0x3f,0xf3, -0xfc,0x3e,0x90,0xb5,0x21,0x3e,0x34,0x9c,0xf8,0xc5,0x64,0x28, -0x0d,0x3b,0x2a,0xe0,0xfd,0x18,0xf6,0x6f,0xbf,0x7c,0xb7,0xab, -0xa9,0xa1,0x7f,0x29,0x2e,0x36,0xaf,0xd8,0x8e,0x3b,0xaf,0xc7, -0x83,0x17,0xd5,0x04,0xba,0x54,0x4e,0xad,0xc9,0x0f,0x28,0x11, -0xd9,0x75,0x66,0x22,0xfa,0x3b,0x81,0x9e,0xf7,0xaf,0x3e,0x5b, -0x28,0x9e,0x56,0x2d,0x30,0xd2,0xc9,0x68,0x67,0x92,0x7b,0xdb, -0xe1,0xd1,0x2a,0x8b,0xe5,0x70,0x17,0x8c,0x75,0x99,0x67,0xad, -0xc9,0x87,0x0f,0x58,0x07,0xf0,0x2f,0x9f,0xe0,0x35,0xc7,0xb8, -0x22,0xfa,0x6b,0x67,0x6e,0x03,0x3e,0x50,0xdf,0x60,0xa8,0xe7, -0x6a,0x81,0x27,0x66,0x75,0xba,0x24,0x32,0xa6,0x87,0x6b,0x48, -0x73,0x46,0x9f,0x8f,0x78,0xb7,0xa1,0x08,0x70,0xb8,0x0a,0xec, -0x37,0xea,0x7f,0x24,0x8d,0xab,0xec,0x57,0x81,0x59,0x5b,0xb8, -0xa0,0xbc,0xca,0xd8,0x2e,0xf4,0x0f,0x3e,0xd7,0xa8,0x37,0x7d, -0x1e,0x7c,0x6d,0xcb,0xd8,0xcb,0x76,0x2b,0xd9,0x1a,0x11,0x5b, -0xd2,0x45,0x8f,0x70,0xdb,0xfe,0xc8,0xab,0x94,0x37,0x15,0x0b, -0x37,0xab,0x19,0x86,0x96,0x81,0x1e,0x0a,0x8f,0x74,0x2f,0x74, -0xbc,0xfc,0x3a,0x42,0x33,0x2c,0xaf,0x68,0x78,0xb4,0x1c,0x6a, -0xdf,0x69,0xc2,0xf5,0x35,0x85,0x49,0xc3,0x52,0xf1,0x74,0x30, -0xda,0x8f,0xaa,0xdc,0x73,0x26,0x16,0x26,0xe1,0x4d,0xcb,0xa8, -0x2d,0x84,0x50,0x0c,0x62,0x7e,0x7c,0x6c,0xc7,0xbc,0xdd,0xb6, -0x46,0x64,0x89,0xab,0xfc,0x17,0xd8,0x94,0xad,0x0f,0xf2,0xe9, -0x20,0x13,0xd7,0x5a,0xaf,0x67,0xbe,0x2e,0x26,0x31,0x4e,0x64, -0xb5,0x06,0x89,0xb4,0x14,0x89,0x88,0xa6,0x59,0x22,0x56,0xb2, -0x7b,0xf4,0x01,0xa8,0x6b,0x67,0xe0,0x61,0x19,0x9f,0x91,0xb7, -0xe7,0x28,0x43,0x77,0x13,0x0c,0xf3,0x1a,0x93,0x73,0x50,0x39, -0x13,0x5e,0x1f,0x85,0xa7,0xf1,0x78,0x7d,0xd0,0x56,0x63,0xad, -0x84,0xa8,0xcb,0x82,0x19,0x71,0x57,0x01,0x87,0xa7,0xe8,0x30, -0x80,0xa6,0x11,0xc0,0xc9,0x57,0xb6,0x89,0x5b,0x7e,0xc3,0xfa, -0x96,0x93,0x0d,0xba,0x8b,0x4d,0x0d,0x3d,0x75,0x25,0x46,0xf8, -0xe6,0x52,0x4c,0x4d,0xf0,0x58,0xcf,0x0f,0xce,0xfd,0xc8,0x0f, -0xb1,0x3f,0xe3,0x3a,0x18,0x59,0x0c,0x24,0xe7,0xb7,0xf1,0xd3, -0x93,0x6a,0xca,0x92,0x40,0x5e,0xc2,0x8f,0xee,0x56,0x6f,0xfb, -0xb5,0xff,0xdc,0x6e,0x42,0x1a,0x11,0x6c,0x0e,0x64,0xa6,0x0d, -0xbd,0x36,0xf3,0xd6,0x7b,0x6b,0x2c,0x39,0x3a,0xfa,0x7a,0x5e, -0x2d,0xde,0xdc,0x3f,0xfd,0x7a,0x83,0x8c,0xf9,0xf4,0xeb,0xfb, -0xf2,0x77,0x7a,0x74,0xb5,0x2e,0x2f,0x8b,0xcf,0xef,0x4f,0x3f, -0x3f,0x9d,0x7e,0x7d,0x7f,0x7a,0xfa,0x35,0x81,0xcc,0x07,0xc8, -0xcf,0xe9,0xcd,0xa3,0xf3,0xe5,0x3b,0x8c,0x49,0x8f,0x23,0x2a, -0xb0,0xd9,0x76,0x5b,0x82,0x7b,0xd0,0xe9,0xf4,0x40,0xfe,0x34, -0xc1,0xb7,0x28,0x4d,0x07,0x5a,0x81,0xf3,0xc8,0x14,0xff,0x9b, -0xcf,0xf5,0x48,0x22,0x51,0xfb,0x76,0x43,0xc6,0xd4,0x5e,0x04, -0x70,0x76,0x42,0x21,0x14,0xc4,0x1d,0x22,0x78,0xd7,0x72,0x5f, -0x3d,0x79,0xbd,0x0c,0x5f,0xf5,0xc0,0xd1,0x1d,0x00,0x93,0xdf, -0x26,0x23,0x50,0xa7,0xf2,0x7e,0x8b,0x44,0xe1,0x2f,0x12,0x8c, -0x15,0x35,0x58,0xbd,0x1b,0x52,0x24,0x89,0x1c,0x55,0x6b,0x43, -0xf1,0x41,0xce,0xfb,0x5f,0x25,0x46,0x09,0xe2,0x0e,0x9f,0x9d, -0xf0,0x26,0x27,0x4f,0x36,0xfc,0x5a,0xc0,0xdb,0xbe,0x9d,0x25, -0x60,0xd6,0x53,0x94,0x54,0xff,0x86,0x3e,0x9e,0x35,0x34,0x52, -0xe6,0x5f,0x14,0x5c,0x86,0xae,0xdc,0x68,0x38,0xee,0x7d,0xf1, -0x0d,0xda,0x7d,0xe6,0x9d,0xfd,0x60,0x53,0x01,0x6b,0xc3,0x61, -0x30,0x2e,0xff,0xb3,0x72,0x5d,0x4d,0xe7,0xe8,0xe9,0xd1,0x39, -0x50,0x11,0xa9,0xc2,0xd5,0xa3,0x01,0xe7,0xf7,0x61,0xa0,0xf7, -0xd9,0x0e,0x68,0x1a,0xbe,0xea,0x9c,0x50,0x01,0xc4,0x23,0x52, -0xcd,0x8f,0xcb,0xf5,0x35,0xf5,0x30,0xcb,0x93,0xfb,0xf4,0x72, -0x5b,0x5c,0x1c,0x4b,0xd1,0x6c,0x31,0x96,0xee,0xbf,0x38,0xe9, -0x7f,0x65,0x5b,0x96,0x28,0x13,0x9c,0x97,0xe1,0x73,0x43,0x64, -0x2d,0x09,0xe0,0x7a,0x98,0xa7,0x24,0x43,0x74,0xf8,0xd3,0x22, -0xc7,0x3b,0x43,0x98,0x1f,0x7e,0x96,0x0e,0x89,0xc0,0xbe,0x2a, -0x07,0xbf,0xed,0xb3,0x73,0xa6,0x5c,0xd0,0x23,0x05,0x38,0xf0, -0x3d,0x04,0x01,0xbc,0xf7,0x1a,0x86,0x2d,0x05,0x8d,0x7b,0xd4, -0x2f,0x4f,0xbe,0xce,0x17,0xd3,0xb7,0xb0,0x40,0xe8,0x0f,0x4d, -0xdc,0x9e,0x36,0x20,0x1f,0x1f,0x6a,0xd0,0xe7,0x1d,0xd1,0xe8, -0xd7,0xbd,0x1b,0x68,0x60,0xd5,0xfc,0x00,0xa9,0xf3,0x3c,0x79, -0xf4,0xec,0xd9,0xe0,0xd1,0xf2,0x7a,0x35,0xdd,0x04,0xda,0xf4, -0x0b,0x4a,0xc2,0x32,0x26,0x78,0x12,0x3c,0xef,0x9c,0x18,0xff, -0xdd,0x46,0xfc,0x6e,0x0f,0x51,0x82,0x36,0xcb,0x71,0x50,0x10, -0xb6,0x63,0xe6,0x37,0x3d,0x7e,0x20,0x3f,0xfb,0x5f,0xc8,0x9f, -0x1f,0x1a,0x69,0xbc,0xf9,0x81,0x89,0xab,0xea,0x5d,0x39,0xe7, -0xd7,0x85,0x96,0x0b,0xf4,0x60,0xe7,0x15,0x62,0x1f,0x0d,0xc4, -0xa7,0x5d,0xc2,0xc7,0x24,0xf5,0x51,0x34,0x29,0x17,0x0c,0x5f, -0x9f,0x17,0x94,0xa5,0xa9,0x5a,0x00,0xbf,0x15,0x98,0xa8,0xc7, -0x36,0x19,0x67,0x4d,0x0b,0x91,0x12,0x5d,0x62,0x82,0x20,0xbd, -0x6e,0x82,0x30,0x9b,0xad,0xaf,0xa7,0x17,0x00,0xaf,0x2c,0xe8, -0x3d,0x73,0x24,0xcb,0x59,0x06,0x15,0xb2,0x4c,0x22,0xa2,0x04, -0x6d,0x25,0x8c,0x0b,0x05,0x83,0x8d,0xd1,0x7b,0x5a,0x5c,0xe3, -0x55,0x4b,0x0d,0x6d,0x8d,0x2f,0x62,0x61,0x71,0xb7,0xb6,0x8e, -0x74,0x05,0x8d,0xe3,0x03,0x05,0x4b,0xaf,0x07,0xfa,0xfe,0xab, -0xd6,0xb4,0xfd,0xc8,0x4d,0x74,0x4b,0x3b,0x82,0x52,0x61,0xeb, -0x6d,0x25,0x16,0x8e,0xeb,0xeb,0x0e,0xf3,0xe1,0xc7,0x8b,0x6d, -0xd1,0x80,0x7f,0x52,0xf3,0x0e,0x58,0xcd,0x0a,0x26,0x5c,0xa7, -0x58,0xca,0xd0,0x37,0x11,0x46,0x61,0x9b,0x8c,0x88,0x22,0xf8, -0x42,0xc7,0x22,0xe6,0x6e,0xf6,0x2d,0x4a,0x98,0x6c,0xef,0xae, -0x05,0x6d,0x15,0xc3,0x14,0x52,0x15,0x80,0x1c,0xf7,0x26,0x31, -0xfe,0x29,0xa4,0xe5,0x8d,0x53,0x24,0x08,0xde,0xfb,0x62,0x10, -0x2a,0xbc,0x34,0xc9,0xec,0xe5,0x27,0xd4,0x3e,0xbf,0xae,0x68, -0x59,0xf0,0x05,0x10,0xfe,0xba,0x04,0x2c,0xd5,0x15,0xac,0x82, -0x93,0x5d,0xd6,0xd8,0x61,0xed,0x7e,0x5d,0x00,0x34,0x49,0xaf, -0x62,0x84,0x0b,0x10,0xe3,0xaa,0x97,0x7c,0x73,0x73,0x0e,0x64, -0x5c,0xc3,0xd9,0xc2,0x16,0xc5,0x5b,0xd6,0x5b,0x79,0x69,0x45, -0xc6,0xc4,0xad,0xd5,0x63,0x5b,0x66,0xa2,0x0f,0xa2,0x92,0x92, -0xda,0xbd,0x48,0xf3,0x56,0x78,0x20,0xea,0x3a,0x5e,0xad,0xf1, -0xa5,0xc3,0x47,0xf3,0x6a,0x55,0xe8,0x7b,0x39,0xf4,0x3a,0x51, -0xcb,0x48,0xdb,0x2b,0x25,0x6e,0xba,0x60,0xe3,0x59,0x13,0xc2, -0xe8,0x65,0xa4,0xb0,0x3d,0xd9,0x81,0xda,0xda,0x10,0xa9,0x27, -0xd6,0x15,0xc8,0x0b,0x7e,0xe5,0xda,0x70,0x50,0x8f,0xc7,0xd5, -0x5b,0xb3,0x99,0xd5,0x08,0xd8,0x2f,0x25,0xe0,0xaf,0xd0,0xb0, -0x1e,0xf9,0xc9,0x90,0x4b,0xc0,0x0f,0x7e,0xff,0x06,0x7e,0x04, -0xaf,0xda,0x40,0xf2,0xbb,0xe3,0x9a,0x18,0x54,0xee,0x01,0x35, -0x3c,0xbe,0x5e,0xfe,0x76,0xbc,0x2f,0xef,0xb6,0x3c,0x7f,0x53, -0x6d,0xf6,0x65,0x27,0x06,0xb7,0xc5,0x36,0x67,0x56,0x7f,0x0b, -0xe4,0xad,0x13,0x23,0x63,0xe3,0x2f,0x0e,0xda,0xcc,0x83,0x3a, -0x2c,0x88,0x19,0x5b,0x2e,0x16,0x14,0xec,0x00,0x29,0x28,0x09, -0xfc,0x95,0xa0,0x2a,0x27,0xc3,0x95,0x72,0xda,0xe9,0x39,0x48, -0x9a,0x40,0x09,0x43,0x14,0x2a,0xd0,0xd2,0xe8,0x72,0x93,0x1f, -0xff,0x05,0xfe,0x07,0xf2,0x05,0x23,0xea,0x58,0xc4,0x0d,0x1e, -0x44,0xb0,0x4c,0x6d,0xc7,0x59,0x28,0xfd,0x54,0x6f,0x1b,0x82, -0x9f,0xca,0x7b,0x9b,0x35,0xfc,0x37,0x43,0xb9,0x6f,0x46,0x3f, -0x36,0xfc,0xeb,0x3e,0xa6,0x8b,0x2c,0x98,0xe0,0xcc,0x1d,0x10, -0x8d,0x36,0x33,0x8c,0x71,0x3d,0xc3,0xcb,0xea,0x78,0xc0,0x1f, -0x33,0xc9,0x14,0x98,0xcb,0x17,0x76,0xe9,0xe5,0x76,0x69,0x2f, -0x78,0xbd,0x2b,0xe8,0xc3,0x3d,0x74,0x48,0xe9,0x83,0x46,0x3a, -0xb7,0x6b,0xf9,0xab,0xec,0x6d,0x1c,0xe5,0xed,0x27,0x6a,0xb6, -0x2e,0xbc,0x5e,0xbb,0xdd,0x7d,0xdd,0x46,0x98,0xe3,0xd5,0xd5, -0x98,0x59,0x4b,0x68,0xee,0x95,0xb0,0x06,0x89,0xfa,0x59,0x4d, -0x0a,0xf5,0x72,0x15,0x6f,0x28,0x55,0x0a,0xc2,0xf0,0x67,0xb8, -0x2e,0x98,0x88,0xbe,0x8c,0x08,0xe3,0xbf,0xee,0xa1,0x25,0xc8, -0x70,0xd8,0xb0,0x5b,0x7e,0xf1,0xa5,0xac,0x6d,0xef,0xdd,0x34, -0x5b,0x6a,0xb6,0x2c,0xeb,0x1f,0x97,0x9b,0xa7,0x1c,0x64,0x87, -0xa5,0x86,0xa7,0x8b,0x6f,0x80,0xea,0x18,0x7f,0xa4,0x33,0x20, -0x22,0xe4,0xda,0xcf,0x97,0x2b,0xbd,0x23,0x07,0x4a,0x41,0xd1, -0x06,0x3a,0x9f,0x3d,0xd3,0x67,0x13,0xa5,0xd5,0x40,0xb2,0x28, -0x92,0xc1,0x7f,0x45,0x59,0x2a,0xdd,0x53,0x0f,0x69,0x97,0x4c, -0x97,0x32,0xf4,0x0d,0xc4,0x37,0x50,0x57,0xcd,0x41,0xa8,0xdc, -0x52,0x24,0x80,0x0c,0x64,0x56,0x1f,0xd5,0x1a,0xa3,0x8f,0xaa, -0x40,0xc3,0xf4,0xe5,0x98,0x15,0xe1,0xe4,0x63,0x4e,0x10,0xb4, -0xe2,0x33,0x57,0x31,0x22,0x8c,0x26,0xa9,0x28,0x07,0x6c,0x54, -0xb9,0x76,0x52,0x59,0x11,0xe7,0x71,0x10,0xa3,0xe4,0xc4,0xa7, -0x3d,0x49,0x23,0x4e,0x10,0x53,0xb9,0x27,0xe1,0x15,0x1d,0xcf, -0xe1,0x68,0x2f,0x62,0x1c,0x97,0xf6,0x90,0xed,0x41,0x84,0x37, -0x13,0x62,0x21,0x6f,0x01,0xc0,0xa0,0x76,0x6d,0xaf,0x15,0xcb, -0x46,0xfd,0x81,0x25,0xa3,0x78,0xe8,0x71,0x74,0x2c,0xa2,0x6f, -0x9f,0xec,0x95,0xd6,0x59,0xc4,0x1d,0x62,0x67,0xf9,0x20,0x31, -0x87,0x83,0xf3,0x15,0x5f,0xb4,0x90,0xb3,0xeb,0xdb,0xf2,0x06, -0x0e,0xb9,0xd7,0xe0,0x89,0xf0,0x0d,0x5c,0x0f,0xff,0xe5,0x3c, -0xef,0xc8,0x15,0xe0,0xfc,0x2b,0x1f,0xe7,0xb1,0xa4,0x5d,0x7c, -0xd1,0xd9,0xbf,0xa4,0xf6,0xc5,0x14,0x4c,0x69,0x19,0xf9,0xc1, -0x02,0x01,0xe3,0x94,0xc6,0x97,0x99,0x31,0x7f,0x37,0xde,0x5e, -0x04,0xd4,0x8a,0x0c,0xda,0xd1,0x2e,0xbb,0x1b,0x67,0x74,0x30, -0x97,0x83,0xb3,0x15,0x00,0xf1,0x80,0x3c,0x65,0x91,0xad,0xe0, -0xb8,0x1f,0x0c,0xd9,0x2e,0x65,0x63,0x02,0x7a,0x27,0xac,0xa4, -0x60,0x3d,0x67,0x77,0x18,0x44,0xe8,0xd9,0xe4,0xde,0xd9,0x6e, -0x7b,0x36,0xd6,0xdf,0x13,0x0a,0xfa,0x73,0x7d,0x33,0xdf,0x54, -0x8f,0xa7,0xf5,0x15,0x94,0x1c,0x3f,0x3c,0xfe,0xdf,0x31,0x3c, -0xce,0x30,0xba,0x4c,0xba,0x40,0xe3,0x06,0x74,0x1e,0x15,0x41, -0x26,0x4f,0xb8,0x40,0xd2,0x4b,0xfd,0x18,0x2b,0x3d,0xb2,0xb1, -0x5d,0x63,0x89,0x6b,0x10,0xc5,0xac,0xc5,0xf2,0xfd,0xb3,0xc7, -0xf7,0x5f,0x91,0x14,0xbe,0x58,0x62,0x04,0xba,0xfc,0xae,0xbc, -0x3e,0x87,0x13,0x1d,0x08,0x64,0x6c,0xd9,0x91,0x27,0x17,0xf3, -0xba,0x9a,0xe5,0x8f,0x1f,0xfc,0xb7,0x47,0x8f,0xbf,0xf9,0xf3, -0x93,0xe3,0x87,0x4f,0xfe,0xfc,0xf8,0x78,0x30,0xb8,0xb8,0x3c, -0xfe,0xcb,0x9f,0xbf,0xf9,0xb7,0xe3,0x2f,0xbf,0xfc,0xf2,0xab, -0xaf,0xbe,0xf8,0xea,0xcb,0x13,0xf8,0x5f,0x62,0xf0,0xb6,0xa2, -0x44,0xc1,0x8e,0x2e,0x4d,0xa9,0xc1,0xc0,0x62,0x2c,0x7e,0xb7, -0xd8,0x0f,0xf2,0x65,0x23,0xcd,0xa2,0xb5,0x06,0x66,0xc5,0xaf, -0xe5,0x4f,0xf2,0xb6,0x54,0xd3,0xe9,0x70,0x34,0x8c,0x8e,0x38, -0x7b,0x61,0xaf,0x62,0x2a,0xcd,0x4e,0xe5,0xfc,0xfe,0x40,0xd3, -0x6d,0xc0,0x04,0x2e,0x40,0x15,0x48,0x2e,0x6b,0x79,0x30,0x24, -0x2e,0xa3,0xb7,0xdd,0xcd,0x01,0x85,0xfe,0x07,0xda,0xc6,0x2f, -0x51,0x44,0x3f,0x2a,0xb4,0x33,0x2f,0xff,0x28,0x20,0x86,0x5c, -0x5b,0x5f,0xfe,0x71,0x60,0xb8,0x21,0x34,0x3c,0x5c,0x6d,0xda, -0x66,0x49,0x03,0x89,0xb8,0x30,0xa3,0x83,0x4e,0x23,0xf2,0xe8, -0x5f,0xe2,0x24,0xe7,0x39,0x47,0xbe,0x75,0x44,0x50,0x45,0xe4, -0x93,0x21,0xb3,0xc7,0x99,0xe3,0x03,0x1e,0x1a,0x6a,0x14,0xdd, -0xe1,0xa2,0xdb,0x2d,0xff,0x15,0x47,0x5a,0xaa,0xf8,0xaa,0xa9, -0x84,0xab,0x40,0x4c,0xa2,0x47,0xec,0xb1,0xac,0xa7,0x2f,0xb4, -0xfe,0xfa,0xe9,0x5d,0x38,0x09,0x9e,0x09,0x23,0xdd,0xea,0x61, -0xf0,0x3a,0x79,0x39,0x13,0x49,0x54,0xe2,0x93,0xd8,0xa7,0x5e, -0x68,0x39,0xa3,0x12,0x30,0xf2,0x42,0x24,0x94,0x79,0xb1,0x17, -0x22,0x9f,0x2f,0x1b,0xb5,0xb2,0x35,0x84,0x6b,0xa7,0x25,0xb8, -0x25,0xbb,0xb1,0xe1,0xf0,0x6a,0xf4,0xdf,0x22,0x4b,0x09,0x0e, -0x12,0x48,0x95,0xdd,0x61,0x66,0x58,0x7d,0xcd,0x31,0xf8,0xf4, -0x05,0xf2,0x5e,0x2f,0xa3,0xc8,0xcd,0x1c,0xd9,0xaf,0x9a,0xf4, -0x17,0xfa,0xe8,0xb8,0x86,0x76,0x83,0x0d,0x15,0x3a,0x3a,0xc6, -0xb0,0x9a,0x7b,0x82,0x3c,0x7b,0xef,0x9e,0x7f,0x25,0xcf,0xce, -0x20,0x28,0x11,0x39,0xca,0x23,0xa5,0xd6,0xe7,0x79,0x1f,0xfc, -0x44,0x6d,0x7a,0xb6,0xc2,0x79,0x11,0x53,0xe5,0xd8,0xa5,0x16, -0x70,0xca,0x51,0x4a,0xe8,0x92,0xad,0x11,0x5d,0x93,0xb1,0x48, -0x57,0x0f,0xe4,0xdd,0x69,0x4d,0xb6,0xc4,0x80,0x36,0x0c,0x3a, -0x11,0xda,0xa5,0xaa,0x3f,0x89,0x7b,0xed,0x1f,0xc1,0x1c,0x85, -0x03,0x43,0x5a,0x88,0x27,0x8c,0xcd,0x4b,0xc9,0xd3,0x2f,0x4f, -0x3f,0x1a,0x38,0x6b,0xf0,0xea,0x7c,0xe6,0x68,0x1f,0x60,0x4d, -0x43,0xf3,0xba,0x84,0x73,0xc5,0x71,0xbd,0xb9,0xb8,0x11,0x86, -0x30,0x08,0x48,0x3b,0x0c,0x5e,0x24,0x51,0x0f,0x4d,0xde,0x32, -0xd0,0x35,0xf0,0x2b,0xc5,0x3b,0x0f,0x99,0x07,0xe9,0xe2,0x9c, -0x1f,0x1f,0x15,0xf1,0x9f,0xda,0x1c,0x51,0x04,0xcb,0x22,0x95, -0xe0,0x68,0x18,0xaf,0xb8,0x97,0x50,0xa9,0x84,0xe3,0xa7,0xb7, -0x04,0x70,0xe5,0x88,0x90,0xe2,0x04,0xd9,0xf9,0x95,0x43,0x3d, -0x47,0xe1,0x12,0xa8,0xcb,0xd1,0xa1,0x16,0x9a,0x91,0x76,0xd8, -0x2b,0x2f,0xa7,0x4a,0x6c,0xe4,0xc1,0x29,0x46,0xba,0x18,0xbb, -0xe7,0x9e,0xcc,0xac,0xdc,0x37,0xda,0xec,0x6e,0xa3,0xef,0x56, -0xf1,0x80,0x86,0x76,0xe0,0x0a,0xc9,0xaf,0x2e,0x42,0x33,0x8f, -0x85,0xec,0xa2,0xc4,0x88,0x8a,0xbb,0x57,0x43,0xba,0x85,0x7c, -0xeb,0xeb,0x63,0x87,0x03,0x5a,0x73,0x6b,0x0b,0xb2,0xbf,0x6f, -0xd2,0x52,0x19,0xf7,0xbb,0x1b,0xa2,0x36,0x4d,0xcd,0x31,0x80, -0x90,0x2f,0xe9,0xc5,0xcb,0xb8,0x4f,0x0f,0xba,0xe3,0x63,0x0d, -0x5b,0x7d,0x71,0xb3,0x66,0xeb,0x6d,0x0c,0x16,0x7d,0xf9,0x4e, -0xcd,0xcf,0xba,0x5d,0xae,0xab,0xef,0x68,0xf9,0x1d,0xd8,0x38, -0xe7,0xdc,0x02,0xda,0xec,0x19,0xdf,0x88,0x9a,0x00,0xe7,0xe6, -0x01,0xe9,0x1d,0xaf,0xdb,0x6e,0x57,0x82,0xaa,0xb7,0x84,0xb0, -0x36,0x1e,0x16,0x5a,0xa7,0x43,0x3c,0xbf,0x0b,0x7e,0x98,0xcc, -0x15,0x4e,0x22,0x03,0x8e,0x97,0xe1,0xf2,0x6c,0x8b,0x0e,0x8c, -0x8b,0xf0,0x8e,0x4d,0xad,0x3e,0x60,0xfc,0x12,0x5f,0xb0,0x6b, -0x5b,0x71,0x58,0x5e,0x0f,0xa8,0xc4,0x2e,0xa8,0x46,0x29,0x59, -0x72,0x59,0xfb,0xde,0x13,0x51,0x62,0xb4,0xe4,0x6a,0x8a,0x68, -0x58,0x3c,0xd8,0xf7,0x7a,0x24,0xcf,0x1a,0x6f,0x2c,0x1b,0x7d, -0xb6,0xaf,0xc0,0x29,0x35,0x5c,0x15,0xe7,0x3c,0x66,0x2e,0x5f, -0x73,0xd6,0x28,0x20,0x68,0xdd,0xd6,0x08,0xed,0x39,0x35,0xe8, -0xf3,0x46,0xcc,0xce,0x5b,0x19,0xcd,0x9e,0x05,0x42,0xdc,0xc6, -0x8d,0x66,0xd8,0x42,0xf5,0xb6,0x48,0x66,0x02,0x22,0xf4,0xc9, -0x0e,0x36,0x77,0xaa,0x41,0x2f,0xbd,0x45,0x6b,0xc1,0xd5,0xdf, -0x65,0x2d,0x8b,0x9a,0x09,0xe8,0x23,0x78,0xe4,0xbe,0xf6,0xe6, -0xbe,0xf7,0x1b,0x3e,0x21,0x61,0xc2,0x26,0x2b,0xb7,0x4f,0x5e, -0xbe,0x1b,0xd9,0x5f,0xf8,0xe0,0x46,0x89,0xcf,0x15,0x42,0xfe, -0x64,0xbb,0xc5,0x3f,0xb9,0xad,0xed,0xb1,0x16,0x0e,0x26,0x26, -0x5d,0x23,0xa2,0x3c,0xdb,0x23,0xbb,0x94,0xd8,0x04,0x8c,0x6d, -0xc3,0x0b,0xcf,0x4c,0x9c,0x8a,0x60,0x7a,0x36,0x74,0x0b,0xd2, -0xe7,0x1b,0xa4,0x58,0xd4,0xd2,0xd2,0x00,0x92,0xe0,0xce,0x50, -0xce,0x7f,0x7c,0x08,0x57,0x0e,0x2e,0x81,0x96,0xa2,0x3f,0x35, -0x6c,0x6d,0x08,0x70,0x1b,0x3c,0x81,0x1f,0xc4,0xbb,0xc1,0x57, -0x86,0xf8,0xb2,0xb6,0xc5,0x97,0xa1,0x14,0x4d,0x16,0x5b,0x0c, -0x54,0x41,0x78,0x2a,0xb9,0xef,0xf5,0xc7,0x71,0x7c,0x4c,0x0d, -0xca,0x85,0x71,0xe3,0x89,0x15,0xb2,0x21,0x1d,0xeb,0x2f,0x00, -0x91,0x43,0x21,0xed,0x59,0x27,0x18,0x39,0xc4,0x2d,0x13,0x27, -0xaa,0xc5,0x53,0x33,0xac,0x8e,0x8f,0x87,0x19,0x7a,0x47,0x35, -0xd8,0x88,0xc4,0x27,0x32,0x0d,0x86,0x44,0xef,0xcd,0x75,0xbb, -0xe8,0x8c,0x4e,0x4c,0x86,0x14,0x97,0x00,0x79,0xaf,0x67,0x6c, -0x1a,0x31,0x16,0x35,0xad,0xf2,0x1c,0xe4,0x28,0x21,0x95,0x0b, -0xee,0xd0,0x86,0x7f,0x97,0x89,0xe8,0x3c,0xa3,0x7e,0x0c,0xc6, -0x7c,0xa0,0x1f,0x6b,0x92,0x6e,0x8b,0xfb,0xe3,0xb3,0x0d,0x45, -0x61,0xc5,0x90,0xaa,0xdc,0x60,0x71,0xff,0x6c,0x8d,0x5f,0xa4, -0x3f,0x27,0x05,0x0d,0xc5,0x9d,0xa5,0xc3,0xe7,0x96,0x0f,0xa6, -0x5b,0xbc,0x5f,0x9f,0xae,0xcb,0xe9,0x16,0x24,0xc6,0xcd,0x72, -0xb1,0x65,0xc1,0x0b,0xce,0x98,0x15,0x36,0x5c,0x5d,0xbc,0x71, -0xf5,0xa6,0x5b,0x2c,0xc8,0x59,0x12,0x71,0x82,0x33,0xe4,0xc2, -0x60,0xab,0xf7,0x7c,0xdb,0xe9,0xcd,0x66,0x49,0x9d,0xd2,0x2f, -0x7a,0x82,0x9d,0xdc,0xe8,0x29,0x22,0xe3,0x7a,0x39,0xaf,0x79, -0x12,0xb7,0x7a,0xc5,0xb4,0xe5,0x60,0xde,0xdb,0xf9,0x72,0xb9, -0xda,0xd2,0xd1,0x16,0xce,0x8a,0xdb,0xe5,0x0a,0x92,0xd0,0xa3, -0x61,0xb9,0x98,0xbf,0x87,0x1f,0xbf,0xde,0xa0,0x05,0xe5,0xb6, -0xbe,0x58,0xe2,0x53,0x3e,0x04,0xc6,0x4d,0x5d,0x3e,0xe6,0xd8, -0xcf,0xed,0x90,0x50,0xa9,0xe8,0xda,0xb7,0x08,0xed,0x04,0xfa, -0x51,0xb6,0x14,0x7f,0x4a,0x27,0xf4,0xa8,0x28,0x61,0x6e,0xd8, -0xe4,0xdd,0x28,0x4f,0x7b,0x31,0xf6,0x50,0x08,0x0e,0x65,0xcc, -0x16,0x69,0x54,0x93,0xa0,0xaa,0x71,0x35,0x5a,0x04,0x40,0x2b, -0xf8,0x3d,0x6c,0xf4,0xf2,0x09,0x92,0x1f,0x89,0xb2,0xd4,0x31, -0x9f,0x38,0x79,0x0d,0xaf,0x7e,0x27,0xd4,0x14,0xd0,0xfc,0xe3, -0xa0,0xfe,0xb9,0xd1,0x8b,0xb3,0xc9,0xf3,0x0e,0x18,0xfc,0x52, -0xcd,0x3b,0x3e,0x37,0xc0,0x89,0x0e,0xdb,0x6e,0x1d,0x15,0xf9, -0xd2,0x50,0x10,0x42,0x8a,0x65,0xe5,0x85,0xe9,0x66,0xb1,0xc4, -0xe5,0xf9,0x8f,0xf5,0xe1,0x91,0x76,0x36,0x7b,0x84,0x2b,0x25, -0x8f,0xcf,0x01,0x74,0x5f,0x82,0x39,0x65,0xcd,0x2e,0xc9,0x20, -0x15,0x61,0x80,0xee,0xdf,0x7e,0x33,0xaf,0xd5,0x09,0x35,0xe0, -0x4e,0x00,0xeb,0x05,0x70,0xf6,0xa6,0x77,0x89,0x78,0x77,0xd2, -0x9f,0xf6,0xd7,0xbe,0x83,0x07,0xef,0xda,0xa6,0xed,0xb5,0x7d, -0x30,0x80,0xc2,0x37,0xf6,0x15,0xea,0xd4,0xf3,0x76,0xe3,0x99, -0x90,0x18,0x94,0x6a,0xb9,0x90,0x61,0x7c,0x7f,0xe8,0x53,0x80, -0xe3,0x47,0x32,0x79,0x54,0x05,0x57,0x06,0x96,0x96,0x1c,0x7a, -0x9e,0x3c,0xf0,0xaa,0x76,0x47,0x5c,0xe0,0x71,0x28,0x26,0xb6, -0xbd,0x2f,0x42,0xdf,0x16,0x82,0x11,0x3a,0x05,0xf5,0xc2,0x34, -0x7c,0x53,0xd3,0x73,0x82,0xa7,0x74,0x83,0x69,0x39,0xfe,0xc3, -0x67,0x57,0xf4,0xaa,0x26,0x74,0x17,0x02,0x2f,0x3b,0x3e,0x67, -0xd0,0xab,0x3b,0x96,0xe2,0xcb,0x92,0x58,0x86,0x1a,0xfc,0x5a, -0x1e,0xd3,0xec,0x15,0x2e,0x6d,0x18,0x76,0x1c,0x84,0x54,0x81, -0xa2,0xad,0x2e,0x0a,0xff,0x62,0x7a,0xc0,0x0b,0x8c,0x78,0x25, -0xb8,0x18,0x27,0xff,0x33,0xa9,0xc4,0x1b,0xce,0xff,0xff,0x08, -0xe5,0x43,0x74,0xa2,0x2f,0xd4,0xee,0x21,0x96,0x53,0xa8,0x86, -0xb4,0x84,0x4f,0x91,0xda,0x0e,0x82,0x22,0xa6,0x8d,0x7e,0x08, -0x13,0xa3,0x98,0x8a,0x00,0x9a,0x90,0x90,0xf8,0xbd,0x90,0x36, -0x42,0x62,0x7b,0xdd,0xbf,0x4e,0xe7,0x9e,0xe7,0x8f,0x4f,0x01, -0xa6,0xaa,0xd1,0x13,0xaf,0x68,0x46,0x6f,0xd2,0x8a,0xd1,0xd1, -0xa6,0x41,0x20,0x6d,0x1a,0x91,0x2a,0xa2,0x0c,0x0f,0xbe,0x06, -0x65,0x54,0x11,0x65,0x38,0x88,0xdd,0xaf,0x5d,0xd6,0x2e,0xed, -0x07,0x21,0xb3,0xf9,0x79,0x07,0xfb,0xde,0xa5,0x6d,0x0f,0x17, -0x0b,0xb9,0x57,0xf9,0x20,0x19,0xcf,0x4c,0x19,0xc3,0x12,0xd8, -0xe2,0x35,0xe3,0xfc,0x00,0xe1,0xb9,0xd9,0x71,0x95,0x28,0x62, -0xc2,0x30,0xe3,0x46,0x19,0xa5,0x23,0x76,0x2e,0xe8,0xa8,0xd3, -0x14,0x8f,0xde,0x2d,0x00,0x82,0x69,0x4c,0x85,0x46,0x89,0x32, -0xd8,0x24,0x4f,0xbc,0x55,0x94,0x4c,0xbc,0xf2,0xe4,0xb2,0xca, -0x2a,0x8e,0x96,0xeb,0x16,0x58,0x2d,0x6e,0x06,0x19,0x13,0x1a, -0x40,0xd8,0x36,0x61,0x8f,0x2c,0x2f,0x9d,0x2e,0x28,0x79,0xf9, -0xd2,0xe6,0xbf,0x7c,0x99,0xc4,0xab,0x34,0xfa,0x2e,0xc2,0xcf, -0xed,0xd6,0x85,0x41,0xc1,0x80,0x23,0x1f,0x6c,0x3e,0xa3,0x45, -0x8d,0xfb,0xa0,0x22,0xa4,0x19,0x43,0xdd,0x7b,0x2c,0xd6,0x76, -0x8b,0x0b,0x45,0xf3,0x69,0xad,0x10,0x03,0xf4,0xd9,0xdf,0x70, -0xee,0x98,0x01,0xae,0x7d,0xe1,0x05,0xc1,0x3b,0x54,0xf8,0x7e, -0xb2,0x24,0x7f,0x78,0xc5,0xdb,0x75,0xec,0x90,0x01,0x6b,0x58, -0x35,0xc8,0x27,0x43,0x1b,0x0d,0xce,0x04,0x01,0x8d,0x3c,0xee, -0x8d,0x11,0x4c,0xf8,0x89,0x32,0xb7,0x66,0x02,0xa5,0xed,0xf0, -0x8e,0x83,0xee,0x04,0xac,0xda,0x46,0x0c,0xb1,0x75,0x5a,0xdc, -0x36,0xc4,0x03,0xbe,0x7d,0xe5,0x49,0xec,0x8a,0x26,0xcd,0x0f, -0x15,0x31,0x3e,0x56,0xa0,0x64,0xe1,0x5a,0x1e,0x35,0x57,0x27, -0xd1,0x2f,0x24,0xa3,0x17,0xa3,0xbc,0xd2,0x47,0x7e,0xcc,0xf0, -0x1b,0x4b,0x17,0x30,0xef,0x8d,0x50,0x24,0x92,0xd7,0x2b,0x1c, -0x51,0x78,0x4f,0xeb,0x65,0xd2,0xad,0xd5,0x9a,0xad,0xf0,0x73, -0x9f,0x66,0xd4,0xf6,0x06,0x8c,0x10,0x9a,0xa3,0x1f,0x3d,0x60, -0x80,0x59,0xa4,0xc1,0x82,0x0c,0x7e,0x3a,0x8f,0xe3,0x48,0xe3, -0x93,0xea,0x56,0xef,0x12,0xe6,0xed,0xd1,0xe7,0xdb,0xd7,0xf6, -0xe0,0x60,0x6b,0x9f,0xda,0x53,0x3d,0x11,0xa4,0xa5,0x6a,0x54, -0x6e,0xd4,0xd4,0xab,0x13,0x84,0xf9,0xe3,0xd5,0xc6,0x2b,0x02, -0x47,0x29,0x3b,0x9c,0x1f,0x06,0xa6,0x1d,0x60,0xe2,0xf9,0x7b, -0x00,0x3e,0x74,0x01,0x61,0x01,0x7e,0x15,0x00,0x8c,0x0e,0x36, -0xf2,0xb2,0x1c,0x64,0x88,0xba,0x5b,0x20,0x0e,0x40,0x46,0xbf, -0x9b,0x9c,0x4a,0x53,0x2f,0x3c,0xb9,0x0d,0x41,0x01,0xf2,0xb1, -0xa0,0x5b,0x23,0x3c,0x16,0xbc,0x93,0x93,0xf8,0x72,0x58,0x02, -0x66,0x86,0x42,0x83,0xb4,0xe8,0x72,0x75,0x2c,0xb9,0x38,0x44, -0xe7,0x77,0x41,0x30,0x70,0xbe,0xd0,0x11,0xa2,0x8d,0x2f,0x0f, -0x18,0x28,0x5d,0x6c,0xf0,0x41,0x2c,0x87,0x1f,0x11,0xbd,0xac, -0x44,0x4b,0xce,0x85,0x72,0x46,0x23,0xf4,0xba,0xd3,0xc0,0xe6, -0xad,0x3d,0x29,0x77,0xb1,0x56,0xa2,0xb8,0x2c,0xd5,0x55,0x9b, -0xda,0xd0,0x40,0xc8,0xec,0xed,0x49,0x49,0x7a,0x94,0x63,0xf7, -0x69,0x34,0x67,0x13,0x6b,0xe3,0x63,0x79,0x50,0xce,0xce,0xe1, -0x76,0x7b,0xc2,0xaf,0x59,0xa9,0x03,0x01,0x14,0x18,0x91,0x76, -0x7e,0x3c,0x31,0xd7,0xd3,0x77,0xf4,0xcd,0x0f,0x5d,0x0d,0xec, -0xa3,0xb9,0x36,0xd0,0x8f,0xd4,0x1d,0x41,0xc1,0xdc,0x16,0x24, -0xc3,0x99,0x77,0x8e,0xbf,0x71,0x2d,0x1b,0x68,0x18,0x03,0x71, -0xa4,0xe2,0xb7,0x6d,0x01,0xed,0x76,0xab,0x4e,0xc1,0x6e,0xa3, -0x40,0x99,0xd1,0x09,0xd2,0xb3,0xb2,0x14,0x1f,0x5f,0x6b,0x67, -0x99,0xb3,0x8b,0x92,0x34,0x1e,0xdd,0x60,0x69,0x21,0xa0,0xa3, -0xed,0x56,0x4a,0xb8,0xf0,0xfe,0xb6,0x0d,0xef,0x12,0x8d,0xe9, -0x36,0x6d,0x94,0x35,0x68,0x74,0x8d,0xf6,0x63,0x2b,0xba,0x3c, -0xb2,0x51,0xb7,0x84,0x5d,0x71,0xf9,0x8c,0xd9,0x0e,0xa2,0x3b, -0xf3,0xaf,0x49,0x86,0x8c,0x59,0xd6,0xab,0xcb,0x45,0x86,0x9f, -0x8f,0xe1,0xd0,0xe3,0x79,0xf7,0xef,0xd0,0x64,0x62,0x1a,0x4a, -0xfb,0x30,0xfc,0x8d,0x1f,0x1f,0x84,0x1f,0x24,0xb0,0x86,0xe2, -0xcd,0xe3,0xa0,0x0b,0xd4,0xef,0x02,0x48,0xaa,0xcf,0x6d,0x20, -0x09,0xf1,0x88,0xc4,0xe7,0x03,0x36,0x94,0x9d,0x7e,0x58,0x49, -0x3d,0x6d,0x12,0x5c,0x71,0x3c,0xd0,0x62,0xb0,0xc0,0x4c,0x78, -0xd0,0x77,0x57,0x5e,0xde,0x10,0x79,0xdf,0x01,0xb9,0x05,0xa3, -0x1d,0xe2,0x46,0xb4,0x40,0x76,0x1f,0x4a,0xc4,0xca,0x9c,0xba, -0x5d,0xb4,0x55,0x58,0xf0,0x7e,0xf0,0x6f,0xee,0xe7,0x03,0xfd, -0x69,0x8f,0x00,0xcc,0x10,0x1a,0x57,0x9b,0x6d,0xf2,0xc8,0xc8, -0x3b,0x51,0x37,0x40,0xcc,0x53,0x86,0x8c,0x4c,0x87,0x16,0x1c, -0x08,0xd6,0x85,0xab,0xc0,0xc0,0x87,0xcb,0x0b,0xb9,0x8d,0xe4, -0x82,0xc0,0xd4,0xe8,0x98,0xde,0x8c,0xdc,0x10,0xee,0x01,0x88, -0x19,0xef,0xfd,0x54,0x8c,0x26,0x23,0x52,0x11,0x9b,0x95,0xd3, -0xc1,0x7f,0xa4,0x4a,0xab,0x5c,0xd5,0x58,0x99,0x8b,0xe7,0xe6, -0x06,0x20,0xdc,0x54,0x01,0x88,0x98,0xac,0x2c,0x93,0x36,0x5e, -0x4b,0x9d,0x64,0xcc,0x5c,0xf7,0xa1,0x6c,0x8f,0xc5,0x8c,0xe5, -0xc5,0xc1,0xba,0xe3,0x06,0x8d,0x63,0xb6,0xb6,0x0a,0xf1,0x5c, -0x5d,0xb2,0xbc,0x45,0xc6,0x60,0xd7,0xfb,0xf6,0x86,0x5a,0xe1, -0x7d,0x6b,0x9f,0xde,0x6d,0xdb,0x1e,0x84,0x1a,0x23,0x70,0x8c, -0x6e,0xc3,0x99,0x9d,0xd2,0xa6,0xa2,0xc7,0x61,0x43,0x6f,0x19, -0xf7,0x68,0x91,0xe2,0x05,0x4a,0x3d,0x20,0xe5,0x58,0xf9,0x1d, -0x27,0xd6,0x93,0xce,0xe5,0xdc,0xda,0x2e,0xa4,0x53,0x18,0x38, -0x2d,0xdf,0xed,0x36,0x4e,0x82,0x7c,0x54,0xb3,0x57,0xd0,0xbe, -0xfc,0xae,0x7d,0x1e,0xd2,0x07,0xb5,0x51,0x54,0x27,0x52,0xdf, -0x75,0xbb,0x9e,0x52,0xd0,0x2f,0xe8,0x1b,0x88,0xb8,0xdb,0xec, -0x44,0x9e,0x8e,0x3b,0xe6,0x28,0x25,0x19,0xbf,0x6d,0x32,0x56, -0x68,0x26,0x20,0x5e,0xe7,0xcd,0x14,0x8f,0xe6,0xbd,0x15,0x86, -0x53,0x42,0xc3,0x75,0x88,0xe6,0x69,0x8b,0x40,0x1c,0x61,0xe9, -0x5c,0x1b,0xcc,0x98,0xab,0xc8,0x3e,0x4d,0xee,0x11,0x77,0x7b, -0x79,0xa8,0x17,0x5c,0xa6,0x69,0xb8,0x0e,0x24,0xc7,0x56,0xed, -0xfa,0x2a,0x67,0x73,0x37,0x60,0x89,0x44,0x8c,0xef,0xa3,0xad, -0xdf,0xdf,0xef,0x8f,0x9a,0xb4,0x27,0x86,0xf3,0x22,0xff,0xbe, -0xc5,0x50,0x6f,0xa9,0xab,0x47,0x62,0x17,0x67,0xee,0x90,0x51, -0xca,0xf4,0xc1,0x80,0xa6,0xe7,0xbc,0x93,0x26,0xf0,0x8b,0x58, -0x6a,0x62,0x54,0xa7,0x9b,0x53,0xe8,0xf1,0x9f,0xe0,0x17,0x7a, -0xa3,0x2f,0xd7,0x20,0xce,0xa2,0xc3,0xc7,0xb7,0xf0,0xcb,0xb0, -0xcd,0x46,0x92,0xfb,0x0e,0x54,0xb0,0x15,0x8b,0x97,0x63,0x02, -0x3f,0xf9,0x52,0x31,0x31,0x17,0xe5,0x7c,0x8e,0x9e,0x3a,0x68, -0xb8,0x96,0xe0,0xc7,0x33,0xfe,0xe0,0x1c,0x35,0x69,0xa3,0x9c, -0x9f,0xe5,0x29,0x43,0xb3,0x5e,0xde,0x42,0x95,0x05,0x00,0xb0, -0xbc,0x85,0xe2,0x0b,0x8c,0x1f,0x34,0xe7,0x14,0xf8,0xc1,0x29, -0x40,0x46,0x18,0xac,0x2e,0x81,0xbf,0x3f,0x4c,0x57,0x89,0xa1, -0xb0,0x41,0x62,0x0c,0x9a,0xd0,0xc7,0x37,0xf4,0x91,0x18,0x31, -0x6d,0x2e,0x67,0x15,0x19,0xec,0xe6,0x6a,0xbb,0xfd,0x44,0x12, -0x92,0x58,0xd7,0xda,0xba,0x71,0xb8,0x43,0x8b,0x6c,0x1e,0xff, -0x8a,0x8d,0xe3,0x77,0x72,0xfb,0x03,0x8b,0x30,0xe0,0xfa,0x58, -0xc0,0xe3,0xfa,0x2e,0xce,0x67,0x83,0x99,0xff,0x51,0x6e,0x48, -0x8b,0x90,0x55,0xbe,0x2c,0x73,0xb6,0x4b,0xe0,0x1f,0xb7,0x39, -0xb8,0xc6,0x78,0x66,0x74,0xf1,0x09,0x85,0xee,0x15,0x93,0xad, -0x74,0x8c,0x82,0x54,0x73,0xbb,0x20,0x63,0xff,0x44,0x29,0x3e, -0xb1,0xf2,0x4c,0x50,0xad,0xdb,0x0d,0x3e,0x3d,0x81,0x9a,0xec, -0x62,0x9e,0x2e,0x36,0x69,0x58,0x80,0xd1,0x32,0x38,0xc9,0x72, -0x77,0x91,0xc3,0x8c,0x2d,0x38,0xa1,0x80,0x5c,0xe8,0x2e,0x6c, -0xda,0xf2,0xc5,0x32,0x0a,0x5d,0xbf,0x46,0x27,0x5e,0x2c,0x3c, -0x3a,0x38,0xe8,0xc6,0x5c,0x34,0x87,0x2e,0xa6,0x63,0xfa,0x36, -0x6d,0xd1,0x2a,0x5e,0xf0,0xdb,0x9e,0x2d,0xba,0x2c,0x2c,0xd6, -0x66,0x93,0xc5,0x95,0x30,0xa4,0x55,0xd5,0xa6,0x02,0xa3,0x77, -0x59,0xbd,0x9b,0x98,0x6e,0xb7,0xc1,0x44,0x39,0x14,0x2a,0xee, -0x66,0x7f,0x8c,0xe3,0xa3,0x5f,0x1b,0xb6,0xd2,0x3e,0x9d,0x54, -0xc6,0x7a,0x7f,0x10,0xb8,0xdd,0x2e,0xff,0xed,0xbf,0xb5,0x41, -0x6d,0x47,0x4d,0xf1,0xc8,0x8f,0x36,0xb8,0x87,0x9f,0x87,0x46, -0x79,0x9e,0x42,0xe7,0xd0,0x86,0x9e,0x1f,0xc2,0x0a,0x2c,0xef, -0x03,0xc8,0x08,0x19,0x7a,0x73,0xe7,0x3c,0xb0,0xea,0xb9,0xeb, -0x4f,0xda,0x43,0x79,0xb1,0xa2,0xd3,0x1c,0xfc,0xd8,0xed,0xcc, -0x61,0xb8,0xd3,0x58,0x8e,0x94,0x7d,0xe5,0x20,0x3d,0xb6,0x0a, -0x6d,0xcd,0x69,0x3b,0xb8,0x0b,0x86,0x2f,0xc8,0x12,0x6b,0x59, -0xe3,0x5b,0x39,0x78,0xf0,0x76,0x6b,0x13,0xbf,0x98,0xef,0x7c, -0xea,0xb4,0x1e,0xee,0x3c,0x6d,0xf4,0xae,0x0f,0x6a,0xb3,0xd8, -0x66,0x05,0x65,0x94,0x2b,0xf9,0x57,0x2b,0xc3,0xc4,0x35,0xdc, -0xc4,0xa9,0x56,0x89,0xf5,0x1e,0x7d,0xbe,0xec,0xfd,0x97,0x20, -0x97,0x90,0x95,0x26,0x15,0xbd,0x91,0xb9,0x20,0xd5,0x24,0x7b, -0xbe,0xb9,0xcf,0x8b,0x25,0x6c,0x94,0xb5,0x26,0x8c,0x92,0x84, -0xc3,0xc5,0x0a,0x42,0x03,0x44,0x67,0xbf,0x07,0xd3,0x9f,0x06, -0xad,0xbc,0x10,0x5c,0xc7,0xe5,0x6c,0x0b,0xc1,0x2b,0x82,0xe2, -0x6f,0xd0,0x38,0x12,0x58,0x30,0x79,0xbe,0x50,0xdf,0xa6,0xaa, -0x1e,0x3b,0x70,0x5d,0xd1,0x7b,0xb8,0xdf,0x28,0x1e,0xe6,0xce, -0x34,0x16,0x41,0x24,0x5a,0xf0,0x8c,0x59,0x4a,0xc0,0x67,0x9c, -0x0e,0xa3,0xa6,0x49,0x34,0x89,0x15,0x0f,0x47,0x20,0xc9,0x06, -0x24,0x14,0x04,0x32,0x19,0xcb,0xab,0xd9,0x46,0x5f,0xcb,0x9e, -0x34,0xc3,0xfd,0xb5,0x9f,0xfd,0xa2,0xf7,0x89,0xda,0x0b,0x99, -0xfd,0x42,0xae,0x98,0x38,0x39,0x38,0xf7,0x9d,0x84,0x12,0xb4, -0x2e,0x70,0xe7,0x20,0xb1,0xea,0xdb,0x51,0x30,0xd1,0x48,0x4a, -0x0e,0xdd,0xa2,0x1d,0xbb,0x91,0xa1,0x92,0x3f,0xb4,0x49,0xea, -0xf5,0x05,0xfc,0xfb,0x3f,0x79,0xdc,0x7b,0x64,0x8b,0xfd,0x07, -0x50,0xf3,0x20,0x1b,0x06,0x1a,0xd9,0xc6,0x41,0x94,0xc6,0x6c, -0xf6,0x0e,0x69,0xff,0x08,0x22,0xf9,0xad,0x85,0x21,0x84,0x01, -0xf4,0x9b,0xb0,0x7d,0x49,0x8a,0xc7,0x06,0xbe,0xc9,0x93,0xa2, -0x8d,0xab,0x53,0xc6,0x07,0xfb,0x09,0x5c,0x55,0xb6,0xdb,0x0f, -0x73,0x82,0x03,0xd5,0x0b,0xab,0xb7,0x6e,0xc4,0x88,0xe8,0x7b, -0x9e,0xea,0x0e,0x58,0x8b,0x94,0x86,0x5e,0x29,0x9c,0xd9,0x66, -0xb9,0xbd,0x93,0xcb,0xea,0x37,0x9e,0x5f,0xa7,0x8a,0xf3,0x1e, -0x4a,0x61,0xff,0x5a,0xfe,0x11,0xa9,0x3c,0x25,0xd1,0x55,0xeb, -0x76,0x1b,0x49,0x61,0x15,0x56,0x58,0xb4,0x4d,0x8b,0xf8,0xd5, -0x87,0x63,0x85,0xbd,0x5e,0x33,0x8a,0x04,0x7e,0x2c,0xe9,0x8c, -0xd4,0xa8,0x2b,0xbe,0xfe,0xee,0x1d,0x0a,0xa6,0x33,0x3e,0x72, -0x9a,0x44,0x43,0x52,0xf8,0xd4,0x66,0xe9,0x2c,0xb8,0x05,0x38, -0x48,0x66,0xf2,0x6e,0x4a,0x93,0xda,0x54,0x8f,0x3e,0x42,0x37, -0xdf,0xdc,0x9d,0x3c,0x03,0xbd,0xf7,0xef,0x07,0xa9,0x75,0x86, -0xc3,0x32,0x1f,0x64,0x59,0x47,0xcd,0x0b,0x97,0x1b,0x15,0xbd, -0xd4,0x6f,0xbf,0x5d,0x43,0xc9,0xaa,0x4e,0x4f,0x43,0x89,0x0a, -0x4a,0x9f,0xa1,0xb1,0x6f,0x10,0x46,0x47,0x78,0x02,0x45,0xeb, -0x03,0x86,0x5c,0x6c,0x18,0xf5,0xa6,0x7c,0xcf,0x8e,0xd7,0xf7, -0x5f,0xc0,0x4f,0x74,0x18,0x5a,0x82,0x5c,0xa8,0x49,0x50,0x97, -0xbe,0xf5,0x85,0x5c,0xd8,0xe8,0x6e,0xb2,0x2d,0x9d,0x1e,0xee, -0x8b,0xb9,0xd8,0x0f,0xcb,0xf5,0xea,0x8a,0x8b,0x8a,0xfb,0x35, -0x1b,0x74,0xd1,0xbf,0xcb,0x9b,0xcd,0xf9,0xfc,0x66,0x4d,0x9e, -0x48,0x48,0x36,0xc8,0x09,0x28,0x58,0x06,0x56,0x18,0xbf,0xe8, -0x4f,0xee,0xd1,0x1b,0xe3,0xfd,0xb4,0xdf,0xcb,0xf0,0x91,0x72, -0xeb,0x99,0x44,0xfd,0xdf,0xf1,0x73,0x24,0xe8,0x99,0x14,0x44, -0xdd,0x2c,0xd5,0x94,0xb6,0x36,0x57,0xd3,0xc5,0x6c,0x5e,0xae, -0xd9,0xb3,0xc5,0x5d,0x4f,0xaa,0x69,0x21,0xb5,0x53,0x9b,0x8d, -0x94,0xfb,0xe9,0xfc,0xf5,0xd3,0x85,0x21,0x19,0x62,0x3a,0xe7, -0xcc,0xef,0x28,0xc3,0xe5,0x6b,0x8b,0x62,0x8a,0x6a,0x01,0xae, -0xcd,0x72,0x5d,0xbd,0xa2,0x58,0x1e,0xd8,0xfd,0x63,0xcf,0x31, -0xc3,0x99,0xfa,0xd9,0xcb,0xa4,0xc7,0x36,0x04,0xab,0x34,0xd7, -0x97,0xbf,0xb0,0x76,0x3d,0x50,0x0a,0x05,0x5f,0xfe,0x16,0x5e, -0x9e,0xd6,0xb0,0xa3,0x0a,0x32,0xdd,0x2b,0xcf,0xda,0x01,0x87, -0x33,0x4f,0xfd,0x4f,0x05,0x90,0x83,0x9b,0x67,0x86,0x5d,0xe0, -0xf9,0x06,0x05,0x21,0x64,0x34,0xd7,0xe8,0xc2,0xd1,0x9e,0x43, -0x2f,0x34,0xa6,0x1e,0x9e,0x5c,0x01,0xee,0xc7,0x56,0x6d,0xcf, -0xf6,0x42,0x7d,0x3b,0x0b,0x34,0x3e,0xb7,0x31,0x68,0xed,0xd7, -0xe4,0xee,0x84,0x21,0x4f,0xa9,0xf3,0x6b,0x74,0xb0,0x28,0x60, -0xbd,0xd3,0x0d,0x8e,0xb7,0xab,0x05,0x25,0xd1,0x6f,0x10,0xf5, -0x8d,0x12,0xf2,0xce,0x03,0xad,0x4f,0x44,0xe3,0x85,0x87,0x33, -0x71,0x26,0x3f,0x06,0xc1,0x74,0xc5,0x1e,0x0a,0xf5,0x61,0xfb, -0x96,0x04,0xd8,0xc5,0x86,0xce,0xa1,0xf6,0x86,0x7b,0xa3,0x56, -0xa0,0x21,0xb5,0xf3,0x63,0x58,0x54,0x72,0xbc,0x99,0x90,0x89, -0x02,0x1b,0x90,0x2a,0x4d,0x61,0xe4,0x7c,0x7c,0xb9,0xda,0x91, -0x5b,0x81,0xa1,0xf6,0xc7,0x0f,0x26,0x0c,0x82,0x84,0x52,0xee, -0xe3,0x4f,0x60,0xb1,0x68,0xa5,0xcf,0x44,0x5c,0x04,0xc3,0x97, -0xc4,0xb1,0x5c,0x54,0xde,0xf1,0x13,0x75,0x85,0xbd,0xb7,0x1f, -0x49,0x01,0x0a,0xc2,0xf1,0x0a,0xa4,0x55,0xec,0x3b,0xd7,0x44, -0xd4,0x46,0x90,0x4a,0x68,0xbb,0x25,0xda,0xff,0xc8,0x2e,0x2c, -0x59,0x46,0xac,0x91,0x55,0x94,0x6c,0xee,0x2b,0xc3,0xcc,0xed, -0x1a,0x22,0x07,0x2a,0x5a,0xb4,0x42,0xb0,0xb9,0x92,0x3b,0xbd, -0x5b,0xe0,0x53,0xb1,0x7b,0x01,0xd9,0xbd,0x7c,0x8e,0xe6,0xd2, -0x12,0xf3,0xc9,0xa6,0x3a,0xa2,0xc1,0xb7,0xd0,0x69,0xd2,0xfa, -0x7e,0x41,0x3e,0xe4,0xba,0x85,0x63,0x71,0x9d,0x3b,0xac,0xf7, -0x5f,0x2f,0xab,0x05,0xe1,0x79,0xe7,0xb3,0x0c,0x58,0x04,0xca, -0x16,0x38,0x36,0x44,0xcd,0xe3,0xcf,0xdc,0x82,0x0b,0x33,0x30, -0x06,0xab,0x66,0x58,0x64,0x3f,0x22,0x3b,0xe7,0x13,0xc5,0x2b, -0xca,0xa9,0x37,0xab,0x6e,0x37,0xf8,0xf4,0xdc,0x23,0x08,0x3b, -0x1e,0xff,0xf1,0xe8,0x35,0x23,0x5d,0x82,0xde,0x43,0xc5,0xd1, -0xd4,0x47,0xad,0xa9,0x6c,0x78,0xed,0xf3,0xbc,0xce,0x80,0x0f, -0xea,0x61,0xec,0x8e,0x38,0x25,0xa5,0x68,0x19,0x71,0xe5,0x2c, -0xb3,0x24,0x88,0x3d,0xa1,0xd7,0xb8,0xfb,0xf2,0x06,0x61,0x91, -0x98,0xb9,0x9f,0xfd,0x56,0x96,0x15,0x67,0x14,0xfe,0x47,0x96, -0x59,0x2a,0x18,0x59,0xbc,0x4a,0xe8,0xde,0x76,0x3c,0xf7,0x7a, -0xe6,0xc4,0xeb,0x3d,0xb7,0xa5,0xe8,0x46,0xd2,0x03,0x2b,0xa0, -0x6e,0xde,0x70,0x64,0x0e,0x3b,0x27,0x64,0xea,0xc5,0x01,0x24, -0x9b,0xc1,0x8a,0x5b,0x76,0x21,0x4b,0xa0,0xd7,0xe8,0x91,0x4e, -0x8b,0x49,0x0c,0xf2,0x5f,0x7b,0x3b,0x0c,0x6e,0x4c,0xb8,0x10, -0x08,0x4e,0xb3,0xd1,0x4d,0x4a,0xb7,0xa5,0x4f,0xdf,0x80,0xc4, -0x4d,0x96,0xb7,0xa0,0xc8,0x82,0xa8,0xb1,0x2d,0xa1,0x36,0xbe, -0x7d,0x1b,0xe0,0x0d,0xeb,0x8f,0xf3,0x3f,0xf4,0x62,0xfc,0x5f, -0xc1,0x02,0xd9,0x05,0x01,0x61,0xfe,0x5f,0xc0,0x0c,0x5b,0x97, -0x39,0x8f,0x02,0xc6,0xca,0x80,0x76,0xbb,0x18,0xb9,0x8f,0x63, -0xe1,0xa7,0x49,0xfa,0x62,0x7b,0x76,0xd6,0xcf,0x58,0xd7,0x15, -0xb0,0x17,0x48,0x06,0xa9,0xa7,0x7f,0x0f,0xfe,0x6e,0x33,0xf4, -0x80,0x4b,0xf1,0xd7,0x67,0x19,0x06,0xda,0x52,0xba,0x28,0x5e, -0x17,0x96,0x64,0x05,0xb9,0xaf,0x11,0xb9,0x8e,0xdb,0x6a,0xf6, -0xf8,0xf5,0xc4,0x74,0x3c,0x82,0xeb,0x76,0x15,0xa5,0xc0,0x25, -0xdc,0xda,0xd2,0xc4,0xed,0xd6,0x8a,0x22,0xfe,0x0a,0x0b,0xca, -0xf2,0xc2,0x24,0xef,0x84,0x0e,0xba,0x22,0x10,0xe7,0x74,0xd9, -0x76,0x40,0xd9,0x76,0xeb,0xb8,0xaf,0xf7,0x40,0xbe,0x2b,0xea, -0xb2,0xd3,0xe4,0xde,0x3d,0xd4,0xe7,0x68,0xca,0x76,0xdb,0x69, -0x16,0xf3,0x19,0xaa,0xae,0xef,0xd7,0x18,0x96,0xbb,0xad,0xc5, -0xf6,0x85,0x7f,0x7c,0x6c,0x19,0x13,0xaf,0x58,0xc7,0x61,0xf9, -0xbb,0x95,0x3b,0x65,0x43,0x8b,0x79,0x18,0x73,0x84,0x79,0x8f, -0xbb,0x6d,0xca,0xe9,0x7a,0xb6,0xbc,0x5d,0xb8,0x46,0x35,0xc5, -0x77,0x6c,0xf3,0x98,0x76,0x24,0x2f,0x09,0xe3,0x0e,0xf4,0xb6, -0xcc,0x66,0x2d,0x3b,0x69,0xd4,0x51,0x83,0xf5,0x60,0xe7,0xd9, -0xd9,0x97,0x85,0xf0,0x9b,0x1e,0x17,0xe2,0x55,0x1c,0xac,0x01, -0x79,0xe1,0xd2,0xb6,0xdd,0xd3,0x05,0xd8,0x64,0x59,0x9d,0x13, -0xef,0xdd,0x61,0xef,0xc5,0x53,0xe1,0x16,0x19,0x3d,0x8c,0xcd, -0x70,0x84,0xf0,0xed,0x75,0x63,0x4b,0xb8,0x26,0x3f,0x18,0x2c, -0x32,0x9c,0xc7,0x3e,0x31,0x93,0x37,0x39,0x2a,0x8d,0xf7,0x7c, -0xdf,0x09,0xde,0xc5,0x40,0x82,0x9b,0x87,0xdd,0x1b,0xb1,0x82, -0xb6,0xd5,0xe7,0x14,0xa1,0xe9,0xb9,0x27,0x9c,0x10,0x33,0xad, -0x98,0x85,0xfe,0x3c,0xdd,0x5c,0x15,0x64,0x8d,0xe5,0x1e,0x27, -0x15,0x26,0xd3,0x78,0xde,0x95,0x7b,0xe7,0xab,0x4a,0x38,0x78, -0xb1,0x98,0x89,0xeb,0x9f,0x33,0x3c,0x26,0xb4,0xaf,0xaa,0x2d, -0x62,0xeb,0x3b,0x5e,0xe7,0x78,0x55,0xce,0x8f,0x4c,0xa1,0x55, -0x31,0xf2,0x09,0x2b,0x67,0x3a,0x08,0xcd,0x17,0x4d,0xaf,0xf8, -0x7f,0x6b,0x26,0x75,0xbc,0xd3,0x16,0xaf,0x47,0x9a,0xcf,0x76, -0x39,0x99,0xec,0x4a,0x21,0xdb,0x19,0x3a,0xf7,0xc9,0xbc,0x59, -0x6e,0xed,0x64,0x64,0x54,0xc2,0x81,0xca,0x98,0xf2,0x98,0x95, -0x3a,0x84,0xfa,0x49,0xc4,0x79,0x33,0x99,0x93,0x22,0xec,0x23, -0x67,0x7b,0xfb,0x48,0x5e,0x60,0x86,0x19,0x47,0x5c,0x60,0x94, -0x61,0xec,0x67,0x3d,0x0b,0xf3,0x2a,0xa0,0x7a,0x0d,0xa7,0xee, -0x52,0x44,0x13,0xfc,0x93,0x71,0xb3,0x40,0xa4,0xcf,0x79,0xb8, -0x78,0x73,0x10,0x4d,0x40,0xd1,0x26,0xd1,0xc5,0x85,0x80,0x5e, -0x8b,0x28,0x69,0xf4,0x47,0xf9,0x37,0xd9,0x79,0x18,0x5d,0x7b, -0xf8,0x2a,0x9f,0xe7,0x77,0x22,0x93,0x24,0xef,0xe3,0xa7,0xfe, -0xa7,0x9c,0x3d,0x6c,0x74,0x00,0x79,0x0f,0x96,0x83,0x8c,0x4d, -0xf2,0x56,0xdf,0x65,0x23,0xb9,0x1f,0x7d,0x16,0xf0,0x57,0x18, -0x70,0x5f,0xcb,0xc0,0x18,0x8d,0xc0,0xca,0xc3,0x04,0x3d,0x43, -0xa9,0x2c,0xca,0xbc,0x2b,0x93,0x37,0xa4,0xbd,0xb6,0x30,0x70, -0xbe,0x54,0x5d,0x2c,0x39,0x88,0x9a,0x8b,0x47,0x60,0x5f,0x8d, -0xa6,0x11,0xf2,0x76,0xed,0x96,0x71,0xd1,0xb6,0x07,0xcb,0x8e, -0xdb,0x20,0x78,0x57,0x8d,0xc8,0x8b,0x1e,0xe0,0x17,0x6b,0x7d, -0xa7,0xe0,0xca,0xd0,0xc0,0x7f,0xd8,0x92,0x6e,0x59,0x04,0x4b, -0x80,0x68,0xa3,0x4f,0x5b,0x37,0x16,0xc7,0xbf,0x45,0x91,0x36, -0x95,0xfa,0x6e,0x99,0x66,0x42,0x7e,0xae,0x05,0xdc,0x18,0xf5, -0x3e,0xa6,0x2a,0x6f,0x69,0xbf,0x94,0x0e,0xff,0x26,0x21,0xe3, -0x38,0x88,0x4f,0xb6,0xa3,0xa0,0x75,0xc5,0xc9,0x90,0xe0,0xb5, -0xcd,0x90,0x41,0x0b,0xbe,0x81,0xad,0x04,0x8d,0x1e,0x50,0x53, -0xc0,0x02,0xb0,0xc7,0x67,0x9b,0x25,0x6e,0xe8,0x69,0x36,0xcc, -0x1c,0x73,0x2a,0xaa,0xd3,0xc1,0xc8,0x61,0xa1,0x21,0xab,0x04, -0xa2,0x4a,0x91,0x06,0x72,0x20,0xb2,0x4f,0xcb,0x90,0x29,0x6a, -0xd0,0xd8,0x35,0x3c,0x89,0x84,0x46,0x2a,0xcc,0xcd,0x24,0xba, -0xf7,0xea,0x7e,0xab,0x6f,0xb1,0xdd,0xb0,0xd2,0x45,0xb3,0x0b, -0x66,0x09,0xdd,0x2e,0x64,0x8c,0xf9,0xf7,0xc4,0xd6,0xf4,0xfc, -0xc3,0x38,0x56,0x08,0xa1,0x3f,0x6c,0x71,0x5f,0xfb,0xf2,0xf4, -0x0b,0x03,0xbb,0x5a,0xd3,0x5f,0xb9,0xb6,0x4c,0x59,0xb0,0x75, -0xf8,0xa1,0xe1,0x77,0xd2,0x90,0xd0,0x15,0xbb,0x52,0xe9,0x67, -0x6e,0x02,0x71,0xeb,0x48,0xfe,0xa5,0x4c,0xa4,0xdb,0xd6,0x35, -0xc5,0x5b,0x05,0x21,0x69,0x78,0x6b,0x02,0x6f,0xae,0x28,0xba, -0xa1,0x75,0x48,0x6e,0xbf,0xcb,0x9b,0x22,0xe6,0x3b,0x4d,0x64, -0xd0,0xd2,0x80,0x1c,0x46,0x1b,0xfc,0xa0,0x6b,0xd1,0xd0,0xda, -0x38,0x5a,0x47,0xd9,0x9d,0x6e,0x27,0x16,0xd9,0x24,0xae,0xa5, -0x7e,0x12,0xc7,0x17,0x32,0x7b,0x74,0x29,0x58,0x84,0x42,0x76, -0xba,0xde,0x52,0xef,0xa1,0xa9,0xdd,0x9e,0x6a,0xde,0x43,0x21, -0xcd,0x0e,0xf1,0x01,0x40,0xb5,0xf3,0xf4,0xb9,0xe0,0x8e,0x1e, -0x14,0x21,0xd5,0x4c,0x24,0x01,0x64,0x77,0xbc,0x4b,0x04,0xbd, -0x5d,0x56,0xef,0x24,0x97,0xd4,0x99,0x15,0x99,0x47,0xba,0x03, -0x14,0x1d,0x48,0x00,0x00,0xab,0xac,0x23,0xdf,0xe4,0x42,0xdf, -0xfe,0x38,0x10,0x61,0xdd,0x4a,0xf1,0x69,0x8b,0x1f,0xc4,0xde, -0xd5,0x41,0x62,0xfe,0x41,0x26,0x1b,0x14,0xbe,0xdb,0xe9,0x4b, -0xbd,0xe3,0x13,0x89,0x25,0x29,0xbc,0xdf,0x72,0x39,0x66,0xfa, -0xd4,0xad,0xe5,0x9d,0x40,0xdb,0x8f,0x05,0x47,0x8e,0x30,0xbd, -0x44,0xcf,0xe2,0x9f,0x71,0xc3,0xfc,0xd8,0xd7,0x32,0xfe,0x87, -0xef,0x55,0xcf,0x5d,0x5a,0x51,0x36,0xaa,0x6d,0x51,0x91,0x19, -0xe2,0x4c,0x82,0xd3,0xc2,0x6f,0xe9,0x23,0x19,0x14,0x02,0xc0, -0x25,0x60,0xd1,0x22,0xfb,0x93,0xe1,0x49,0x93,0xac,0x79,0x43, -0x9f,0x2c,0x77,0x84,0xb0,0x79,0xee,0x10,0x13,0x75,0xf5,0xf4, -0xfa,0xba,0x9c,0x55,0x80,0xab,0xf6,0x3e,0xd3,0x4e,0x73,0x37, -0xd7,0xa5,0xee,0xa7,0xed,0x3f,0xba,0x64,0x7a,0x10,0xee,0xc7, -0x47,0x2a,0x24,0x31,0x99,0x2f,0xdc,0x8f,0x5d,0x65,0xda,0x75, -0xf1,0xfa,0x2d,0x4d,0x5b,0xa9,0xa0,0x79,0xd8,0x9a,0x70,0x7c, -0x36,0xce,0xd0,0x93,0x97,0xa7,0xe9,0xc8,0x84,0xbd,0x04,0xa8, -0xe2,0xe7,0x95,0xa1,0x9f,0xe8,0x75,0xc6,0x40,0xac,0x40,0xab, -0x52,0x61,0x8d,0x69,0x3b,0x6f,0x94,0x31,0xa0,0x9b,0xbf,0x87, -0xc3,0xd4,0x7b,0xe7,0xd0,0x12,0xd9,0xb2,0xde,0x28,0x95,0x39, -0x0e,0xe8,0xa7,0x36,0x68,0xcf,0x44,0xab,0x5b,0xe7,0x31,0x96, -0xef,0xaf,0x02,0x89,0x1e,0x0e,0x1b,0x8d,0x65,0x8c,0xfe,0x2b, -0xf1,0x32,0x0e,0x35,0x66,0xed,0xe7,0x3c,0x63,0xf7,0x52,0x91, -0xa1,0x28,0xac,0xba,0x5f,0x82,0xf6,0x22,0xdf,0x8f,0x46,0x68, -0x46,0x1c,0xd8,0x95,0x69,0xa3,0xac,0x6d,0x97,0x6f,0x66,0x9d, -0x04,0x3b,0xb4,0x3c,0x5b,0x24,0x09,0x7e,0x7e,0x54,0x9d,0x98, -0xa2,0x2e,0xf0,0x53,0x6d,0xec,0x29,0xcc,0xd4,0xbe,0x6e,0xf8, -0x7d,0x36,0xc6,0x80,0x3c,0xb6,0x3b,0x0c,0xa0,0x17,0x5f,0x82, -0x96,0xf3,0x7e,0x35,0xc1,0x53,0x5b,0xcb,0x21,0x9b,0xdc,0xae, -0xa4,0xcd,0x31,0x24,0x4e,0x8a,0x90,0x80,0x82,0x2c,0x6f,0x4d, -0x78,0x4a,0x52,0xb1,0x28,0x42,0x25,0x89,0x61,0x6b,0x78,0x12, -0xef,0x69,0xbf,0xc6,0x9b,0x26,0x75,0x69,0x47,0x5b,0x7b,0x2d, -0xc3,0x81,0x87,0xc6,0x50,0x64,0xa2,0x71,0xc8,0xb3,0x00,0x8c, -0x6e,0x57,0xbe,0x62,0xe5,0xdb,0x50,0xd3,0xf5,0x9c,0xed,0x13, -0x02,0x17,0xa6,0xdd,0x29,0x47,0x69,0xc0,0x92,0x98,0x54,0xda, -0xb9,0x78,0x50,0x3e,0xde,0xbe,0x6e,0x1c,0xde,0xf7,0x35,0x4a, -0xb0,0xdb,0x56,0x9d,0xda,0x81,0xb4,0x0e,0x41,0x9b,0x78,0x69, -0xee,0xb7,0x82,0x8f,0x37,0xbd,0x6b,0x6c,0x64,0x2a,0x8b,0xc4, -0x67,0x9d,0xcc,0xdf,0x10,0x65,0x43,0x23,0xbf,0xf8,0x8b,0xe5, -0xea,0x3d,0x9f,0xba,0x1c,0x65,0x90,0x06,0xa8,0x5a,0xe0,0x23, -0xfc,0xf6,0xf0,0x84,0xbd,0x91,0x75,0x0e,0xf9,0x30,0xc8,0x87, -0x28,0x02,0x28,0x7a,0x85,0x24,0xa9,0x7f,0x53,0x58,0xa2,0xd0, -0xda,0xde,0xd5,0x9e,0x3b,0x41,0x8a,0x6f,0x28,0x65,0xb1,0x75, -0xa3,0xbd,0x12,0x6c,0x14,0x82,0x0c,0x31,0x80,0xdc,0x65,0x04, -0xbc,0x36,0x4d,0xb7,0xc3,0x35,0x17,0xa2,0x9f,0xfa,0x2e,0x6f, -0x90,0x2f,0x0e,0xa2,0xf4,0x5b,0xce,0x86,0x8d,0xf3,0x5f,0x30, -0x7a,0xdc,0x9a,0xb0,0x1b,0x1b,0x96,0x0c,0x15,0x60,0x64,0x5a, -0x88,0xa9,0xb8,0x0e,0x18,0xdf,0x98,0x34,0x29,0x82,0xaa,0x9c, -0x36,0x0c,0x44,0x91,0xf6,0x93,0x57,0x50,0xad,0x5f,0xaf,0x2f, -0xf0,0xde,0x34,0x14,0xfd,0x0d,0x06,0xab,0xf4,0x2b,0x05,0x1e, -0xcb,0xc1,0x31,0xce,0x2f,0xe4,0x1d,0x3e,0x84,0x5d,0x5e,0x97, -0x9b,0xe9,0xff,0x28,0xdf,0x17,0x9d,0x4e,0xf0,0xad,0xf3,0x0b, -0x13,0x37,0xc7,0x30,0x3b,0xe1,0xa7,0x70,0xd2,0x10,0x33,0xac, -0xa7,0x60,0xa3,0xd4,0x3a,0x4f,0xa6,0xf3,0x0d,0xb4,0x73,0xc4, -0x67,0x83,0xfa,0xe8,0x62,0xba,0xb8,0x28,0xe7,0xc8,0x84,0x8e, -0x2e,0x36,0xeb,0x39,0x66,0x05,0x5b,0x33,0x23,0xe4,0xe7,0xab, -0x69,0x8d,0x21,0x8c,0x08,0x86,0xa3,0x35,0x54,0x00,0xa9,0x58, -0x0a,0xd0,0xd9,0x1f,0x93,0x79,0x2c,0x14,0x50,0xe6,0xd9,0x66, -0x7a,0xbd,0x3a,0x7a,0x0b,0x67,0x9d,0xa3,0xdb,0xab,0xea,0xe2, -0x2a,0x7c,0xe1,0x53,0x68,0x0e,0x2f,0x6b,0x1d,0x9d,0x08,0x78, -0x17,0x57,0xe8,0x2a,0x0a,0xff,0x3c,0x02,0x64,0x60,0x74,0x26, -0xfc,0x0f,0x7f,0x47,0x4d,0xe0,0x70,0xe3,0x1d,0x44,0xc7,0x1d, -0x5d,0xfc,0x33,0x02,0x09,0x0e,0x3b,0x09,0xf4,0x55,0xc8,0x03, -0x4e,0x52,0xad,0xaf,0xdd,0x8e,0x1a,0x29,0xb9,0x4d,0x11,0x68, -0x64,0x9a,0x60,0x3b,0xf3,0x96,0x83,0x8e,0x81,0x77,0x8d,0x23, -0xfe,0x03,0x28,0x9e,0x57,0x50,0xf4,0xef,0xf2,0xf7,0x1f,0x47, -0x97,0xeb,0xe5,0xb5,0x50,0xce,0x11,0xc7,0x13,0xfd,0xbb,0xfc, -0xfd,0xc7,0x11,0xec,0xbd,0xe5,0xdf,0xe9,0xdf,0x7f,0x1c,0xd5, -0x17,0xeb,0xb2,0x5c,0xfc,0x5d,0xfe,0xfe,0xe3,0x68,0xb3,0x94, -0x5a,0x9f,0x84,0x0a,0x64,0x22,0x18,0x74,0x94,0x01,0x86,0x23, -0x89,0x01,0x62,0x35,0x62,0xac,0x67,0x87,0xc5,0xdf,0xc6,0x03, -0xcd,0xe5,0x79,0x89,0xc3,0x16,0xc4,0x12,0xc8,0x6a,0x25,0xed, -0x30,0xc7,0x83,0x56,0x84,0x43,0xbf,0x21,0xc5,0xef,0x39,0x3b, -0x23,0x70,0x85,0xd6,0x88,0x9f,0xcf,0xe5,0x08,0xdf,0x36,0xd7, -0x8d,0x8a,0x61,0x68,0x74,0xde,0xc3,0x08,0xbf,0xf4,0x1a,0x6f, -0xbf,0xa6,0x77,0x6f,0xbf,0x2f,0x2f,0xa1,0x2f,0x8e,0xfc,0xcd, -0xc1,0x59,0xbd,0xe4,0x93,0xec,0xd8,0x95,0xe7,0x16,0x1a,0xe5, -0xfd,0xe4,0x93,0xcc,0xeb,0xfc,0x1f,0x71,0xe7,0xff,0x68,0x74, -0xfe,0x7c,0xb9,0x6a,0xe9,0x9b,0x52,0x5b,0xba,0x8e,0x4b,0x7b, -0xa9,0x27,0x18,0x03,0x4d,0x85,0x2a,0x6f,0x25,0x76,0xbb,0xde, -0x4c,0x79,0x42,0xa0,0x57,0xa4,0xf0,0x27,0x38,0x64,0x54,0x8e, -0xe8,0x2d,0x9d,0xe5,0x5e,0x69,0x1d,0x2c,0x2d,0x9c,0xad,0x44, -0xf6,0x29,0x42,0x8f,0x59,0x7f,0x69,0x0d,0xba,0x5c,0x64,0x34, -0xc8,0x1f,0xe8,0xcf,0x2f,0xf2,0x2f,0xf5,0xe7,0x83,0xfc,0xc4, -0x2d,0x21,0x11,0x23,0xf3,0x3b,0x7c,0x76,0x39,0xbf,0x53,0xcd, -0x10,0xc5,0x75,0x25,0x89,0x28,0xbf,0x6b,0xe8,0x84,0xf7,0xda, -0xc2,0xf2,0x01,0x4d,0x6c,0x61,0xbb,0x5d,0x67,0xb0,0xa3,0xde, -0xd2,0x7c,0xec,0x16,0xd7,0x77,0x68,0x7c,0x94,0xba,0xdf,0x20, -0x0a,0xe3,0x55,0xa7,0x1a,0xc9,0xf0,0x2b,0x01,0xad,0xbd,0x4b, -0xfc,0x4d,0xff,0xc9,0x10,0x7c,0x94,0xee,0xad,0x7b,0x04,0x9a, -0xf7,0x57,0x6c,0x20,0xc3,0xa3,0xb3,0x1f,0x78,0x83,0x52,0xa9, -0x2f,0x3f,0x9e,0x8c,0x09,0x6e,0x97,0x12,0x31,0x91,0x49,0x76, -0x06,0xed,0x62,0x0e,0xa1,0x40,0xdf,0x49,0x3b,0x04,0x08,0xb6, -0x21,0x23,0xc5,0x9f,0xe1,0x40,0xdb,0x7a,0x5e,0x02,0xfe,0xa0, -0xeb,0x12,0x84,0x86,0xf2,0x66,0xc1,0x33,0xe3,0xcb,0xf9,0xed, -0x87,0x73,0x11,0xf3,0x5b,0x8f,0x23,0xe1,0xf6,0xc9,0xb0,0xb3, -0x79,0xb2,0x5f,0x15,0x6d,0xff,0x4c,0x5d,0x5d,0xdf,0xcc,0x83, -0xc7,0x1c,0xed,0x75,0x87,0x9c,0x49,0x79,0xff,0x62,0xf6,0x56, -0x46,0xe6,0x04,0xb1,0xa8,0x20,0x55,0x3c,0x2b,0x83,0xaa,0x7e, -0x26,0x3d,0x70,0xec,0x61,0x1f,0xb2,0x1c,0x43,0xfb,0x0c,0xb9, -0xfd,0x51,0x9b,0x66,0x23,0x65,0x9f,0x79,0x82,0x26,0xdb,0x63, -0x54,0xe2,0x6e,0x78,0x70,0xab,0x68,0x55,0x29,0xed,0x53,0x54, -0x39,0x9b,0x59,0xef,0xce,0xc7,0x7f,0x98,0xdd,0x26,0xda,0x5b, -0xfc,0x96,0xc8,0x9a,0x72,0x21,0x44,0xe2,0x6b,0x5b,0x1d,0xb9, -0xbd,0x6f,0xc9,0x49,0xbd,0xfa,0x48,0x24,0xbb,0xb6,0xc0,0x9d, -0xda,0xbc,0xfa,0xf7,0x15,0x56,0x95,0xcf,0x31,0x4f,0xf8,0x5d, -0xf1,0x27,0xac,0x8c,0xf7,0xfd,0x26,0xc5,0x28,0xb4,0xdd,0x61, -0xd2,0x2b,0xc0,0xfa,0xa8,0xb8,0x2d,0xb6,0x0f,0x95,0xce,0xdd, -0x1b,0xe4,0x8c,0x22,0x17,0x7b,0x62,0x7d,0xc1,0xd2,0x4e,0xb0, -0x42,0x8e,0xaa,0x45,0xbd,0x41,0x69,0xc7,0x9a,0x14,0x71,0xbd, -0x11,0x96,0x87,0xb3,0xec,0xfa,0x82,0xed,0x85,0x78,0x91,0x84, -0xb2,0x35,0xb6,0x68,0x99,0x47,0xa1,0x45,0x39,0xa9,0x39,0xb7, -0x54,0x60,0x16,0x25,0x6e,0xb7,0x98,0xea,0x53,0x3d,0xeb,0x03, -0x31,0x15,0xd8,0xef,0xcf,0x01,0x15,0x30,0x38,0x8d,0xe4,0x54, -0x1f,0xd7,0x7c,0xbe,0x66,0x23,0x77,0xf8,0xc9,0x4f,0x5b,0xe7, -0x01,0x74,0x3c,0x7a,0xcf,0xca,0x85,0x96,0x05,0x71,0x46,0xc6, -0x8b,0x0c,0x46,0x25,0xb5,0xc2,0x61,0x40,0x93,0xac,0x36,0x71, -0xb1,0xbc,0x95,0x47,0x78,0xe3,0xc3,0x0a,0xde,0xa6,0x58,0x97, -0x82,0x58,0x3a,0x77,0x73,0x10,0xce,0x12,0xca,0xf4,0x9b,0x25, -0x01,0x7a,0xd7,0xc4,0x9c,0x3f,0x26,0xd3,0xa6,0x44,0x8a,0x0a, -0x1c,0x50,0xfd,0x04,0x25,0xc3,0x35,0x16,0x3f,0xde,0x2a,0x81, -0x50,0x82,0x29,0x1f,0xee,0x9b,0x5b,0x37,0x01,0x86,0x64,0xfa, -0x68,0xfd,0x8e,0xe2,0x84,0x14,0x04,0xf0,0x60,0xda,0x71,0x49, -0x99,0x48,0xd5,0xf2,0x29,0x10,0x35,0x87,0xda,0x84,0x29,0x6a, -0x1e,0x16,0x7b,0x53,0xb9,0x63,0x50,0xe7,0x89,0xc2,0x3f,0xef, -0xb6,0x05,0x85,0x03,0xc6,0x52,0x6d,0x48,0xcd,0x63,0xf7,0xf8, -0x83,0xb8,0xf7,0x01,0x62,0x67,0xfa,0xb8,0xf3,0xc8,0x31,0xe0, -0x8e,0xa4,0x68,0x44,0xf0,0x3a,0x4f,0xe8,0x37,0x70,0xa5,0x75, -0xc2,0xc2,0xf5,0xbc,0x9c,0xbe,0x2d,0x35,0x99,0x76,0x25,0xf7, -0x1a,0x2b,0x20,0x08,0xcf,0x13,0xee,0x21,0xdd,0x40,0xcb,0x86, -0xd9,0x93,0xe2,0x2e,0xd8,0xdd,0xa0,0xb4,0xd1,0x5b,0x10,0xfa, -0x60,0x6e,0xd2,0xd8,0xce,0xd4,0x7d,0x72,0xe3,0x29,0x60,0x45, -0x96,0x2a,0x5a,0x24,0x2b,0xcf,0xb6,0x2d,0x52,0x15,0x8a,0xcc, -0x9c,0x76,0xa4,0xfc,0x76,0x2b,0x3f,0xf0,0x21,0x75,0x11,0xdc, -0x54,0xd3,0x2f,0x2f,0x00,0xd4,0x29,0x67,0x68,0x87,0x9e,0x06, -0x92,0x56,0x4e,0x53,0x7b,0x48,0xba,0xc6,0xa6,0x75,0x54,0xeb, -0x73,0xbe,0xc6,0x6b,0x09,0x71,0x67,0xc8,0xda,0xbf,0x69,0xc8, -0xce,0xcf,0x26,0xc9,0x53,0x47,0x9e,0x7b,0x83,0x8f,0x62,0x29, -0x54,0xdc,0x91,0x49,0xda,0x47,0x8b,0x67,0xfc,0x24,0x1b,0x3a, -0x8b,0x84,0xad,0x62,0x08,0x4b,0x2e,0x41,0xe2,0x58,0xff,0x25, -0x37,0x8f,0x67,0xc0,0x15,0x86,0x6c,0xd5,0x04,0xef,0x71,0x28, -0x95,0x01,0xe8,0xba,0x5c,0x64,0x59,0x83,0xed,0x17,0x07,0xbd, -0xa4,0x3c,0xfe,0x16,0x64,0xb3,0x7c,0xaa,0x2e,0x5c,0xd8,0x8e, -0x73,0xa6,0x41,0x05,0xca,0x75,0x1c,0x3f,0x1d,0xd3,0x4c,0x12, -0xe0,0x0a,0xe5,0xce,0xe6,0xb8,0xfc,0x82,0x6d,0xe3,0xf0,0xa5, -0x28,0xc9,0x7e,0x79,0xae,0x4b,0xd3,0xcd,0xce,0xfe,0x5e,0xd1, -0x2e,0x23,0x88,0x01,0xfe,0x11,0xe2,0x5a,0xd8,0x91,0x67,0xb8, -0xd1,0x92,0xcb,0x4b,0xd9,0xf7,0x19,0xb0,0x2a,0x78,0xb9,0x5f, -0x8f,0x8c,0x71,0x55,0x8a,0x4b,0x13,0x1d,0x6d,0xd4,0x82,0x08, -0x66,0x04,0xb7,0x51,0xeb,0x98,0x3f,0x4c,0x44,0x62,0xc9,0xc2, -0x85,0x2c,0xa6,0x1d,0x5e,0x9a,0x94,0xce,0xcf,0x82,0x7d,0x80, -0xd2,0xb9,0xd0,0x7e,0x4a,0x77,0x96,0xf4,0xa2,0x0d,0xf3,0x83, -0x13,0x65,0xa3,0x34,0x6d,0x3f,0x8a,0x6c,0xb7,0xce,0xc3,0xde, -0x26,0xb6,0x12,0x10,0x0f,0x08,0x37,0xd5,0x72,0xbd,0x79,0xcf, -0xe0,0xf4,0x5f,0xf2,0xdf,0x26,0x21,0xd9,0xc7,0xe4,0xec,0x39, -0x2f,0x94,0xbf,0xb5,0x1d,0x84,0x4e,0x63,0x8b,0xbd,0x7c,0x7d, -0x53,0x6f,0xa4,0xc5,0x19,0x6d,0x08,0xd1,0x95,0x60,0x63,0x7d, -0xee,0xeb,0xbd,0xd9,0x5c,0x1b,0xb5,0xb4,0xf6,0x3a,0x88,0xfa, -0x74,0x54,0xa4,0x9d,0x79,0x57,0x53,0x04,0x23,0x9f,0x65,0xf6, -0x21,0x8c,0x8f,0x30,0x74,0x26,0xc2,0xc7,0x8d,0x5b,0x40,0x6e, -0xe1,0x20,0xc3,0x78,0x36,0xad,0x8d,0x8d,0xf8,0x3d,0xb7,0x3d, -0xa2,0x10,0x90,0x51,0x3b,0x13,0xf0,0x0b,0xee,0xc7,0x5e,0x27, -0x5a,0x27,0xee,0x66,0xda,0x1e,0x5b,0x5c,0xd2,0x73,0x35,0xca, -0xf8,0x08,0xac,0xb5,0xae,0xbd,0x98,0xb7,0xb4,0x0d,0xa6,0xc1, -0x5b,0x0e,0xec,0x9a,0x8c,0x48,0xff,0xba,0xc5,0x93,0xc2,0xc5, -0x5c,0xe9,0xe3,0x46,0x24,0x2b,0xa3,0xe3,0x05,0x64,0xf2,0x4f, -0xf4,0x7e,0xfa,0x28,0xda,0x75,0x0f,0x6f,0x81,0xf6,0xe0,0xfb, -0x61,0xa6,0xd3,0xca,0x53,0x04,0xa7,0x40,0x78,0x87,0x57,0x7d, -0x0b,0xaf,0x91,0x63,0xbd,0x65,0x36,0x81,0x18,0xc4,0xfa,0x06, -0x7b,0xf4,0xe7,0x93,0xbf,0x7f,0x1e,0x6f,0x91,0x7c,0x24,0x88, -0xc0,0x94,0xae,0x87,0x4e,0xac,0x7f,0x48,0x3c,0x33,0xed,0xd4, -0x81,0x02,0x90,0x3f,0x55,0x66,0xcf,0xe5,0x3a,0xd1,0xc9,0xb0, -0x55,0xc8,0x82,0x42,0x93,0x16,0xae,0x48,0x81,0x4b,0x05,0xac, -0x5e,0x8f,0x54,0x5b,0xa2,0x9d,0x88,0xad,0xcd,0x69,0x28,0x6a, -0x64,0x48,0x52,0x68,0xdb,0xac,0x60,0x7b,0xc7,0xc7,0xda,0xa2, -0xd7,0x5e,0xdb,0x11,0xb6,0xd1,0x64,0x7b,0x7c,0x7c,0x5f,0xae, -0x65,0xf3,0x6c,0x6b,0xe3,0x48,0x17,0xb8,0x97,0x0b,0x8a,0x56, -0x65,0x63,0x6d,0x92,0xa2,0xe0,0xdb,0x05,0x5e,0x1f,0x36,0x0c, -0xcf,0x36,0x6c,0xc1,0xdd,0x88,0x0e,0xee,0x99,0xb5,0xd2,0x1d, -0xf1,0x8c,0x5e,0x97,0xb1,0xfd,0x58,0x97,0x1d,0x17,0x30,0x7c, -0xe8,0x9b,0x6a,0x72,0xb3,0x7c,0x1a,0x10,0x2d,0x48,0x08,0xa3, -0x98,0x69,0x92,0x5d,0x07,0x02,0xeb,0x2f,0xb6,0x1d,0x00,0xea, -0x6c,0xc4,0x58,0x47,0x8b,0x6f,0x39,0x8c,0xf0,0x29,0x87,0xa0, -0x99,0xa2,0x05,0x8e,0x5c,0x8b,0xa3,0x45,0x6e,0x1c,0x47,0xce, -0x9a,0x46,0x63,0x53,0x04,0x08,0x35,0xe3,0xd5,0x0e,0x33,0x0e, -0x0d,0x38,0xc3,0xd7,0x2c,0xc8,0x42,0x01,0xfe,0x7a,0xc7,0xb6, -0xe1,0x7f,0x21,0xcb,0x55,0x34,0x22,0xbb,0x5c,0xf8,0x91,0x6b, -0x75,0x8c,0x78,0xab,0x0a,0x63,0xc6,0xe0,0xeb,0x34,0x31,0xfc, -0xee,0x44,0x83,0xf4,0xc3,0x10,0x60,0x19,0x3e,0xe2,0xa5,0x64, -0xcd,0xf5,0xda,0x99,0xc4,0x0e,0xdf,0xa0,0x20,0xff,0x02,0x29, -0x25,0x4e,0x08,0xde,0x57,0xe4,0x36,0xd5,0x1a,0xc3,0x31,0x5a, -0x7a,0x76,0x63,0x62,0x82,0x03,0x88,0x43,0x87,0x34,0x0c,0xa7, -0x89,0x6f,0x13,0x7f,0x80,0x30,0xc3,0x30,0xdd,0xfb,0xe9,0x17, -0x0f,0x9b,0x30,0xde,0xbd,0xcd,0x61,0x4b,0xce,0x82,0x96,0x7c, -0x0e,0xde,0x73,0x4c,0x98,0x0d,0x5b,0x88,0xb3,0xd1,0xfe,0x2a, -0x52,0x4f,0x70,0xaa,0xbb,0x9e,0xd5,0x88,0x88,0xf6,0x10,0x14, -0x15,0xd0,0x27,0x45,0x39,0x39,0x34,0x6e,0xe1,0x09,0x69,0x31, -0xbc,0x18,0x35,0xcf,0x39,0xbd,0xa4,0x9f,0xf4,0x5a,0x8a,0xe6, -0x2d,0x47,0xa2,0xe6,0xad,0x77,0xd3,0xb1,0x84,0x67,0xec,0xd0, -0x82,0xde,0xb7,0x18,0x2f,0x2f,0xa3,0xd5,0xe8,0x2d,0xc4,0x70, -0x11,0xca,0xf9,0xcf,0x12,0xbe,0xd8,0x81,0xe9,0x8c,0x34,0xd7, -0x54,0xc6,0x6f,0xad,0x1c,0x5a,0x32,0xba,0x62,0xb8,0xa4,0xaf, -0x09,0xfa,0x08,0x12,0xf4,0x37,0x33,0x4b,0x85,0x01,0x01,0xe2, -0x01,0x39,0x26,0x99,0xc3,0x84,0x47,0x1a,0x51,0x2d,0xb2,0x33, -0x37,0x8b,0xb6,0x26,0x1a,0xb5,0x05,0x89,0x52,0x9d,0x6a,0x2a, -0x6d,0x34,0x43,0xc9,0x9a,0x8f,0x81,0xa3,0xb1,0x4e,0x10,0x96, -0x0f,0xb6,0xe9,0x35,0x37,0x68,0x09,0xb8,0x3d,0xb2,0xd0,0xda, -0x7a,0xe8,0xa2,0x20,0x5a,0x37,0x37,0x0a,0xe7,0xb0,0x80,0xd9, -0xdc,0x7b,0x43,0x6d,0x4f,0x54,0x13,0x3c,0xc1,0xf6,0xe1,0x19, -0x53,0xcd,0xb3,0xad,0xcb,0x16,0x17,0x3b,0xd7,0xbe,0x18,0x31, -0xee,0x7d,0xec,0x85,0x84,0x32,0x8d,0x54,0xeb,0xf9,0xb5,0xb7, -0x6b,0xb8,0x5d,0x3f,0x24,0x0c,0xc2,0x0e,0xea,0x1c,0x87,0x9d, -0x0c,0xc2,0x36,0xab,0xc6,0x7b,0x95,0x4c,0xb3,0x8e,0xaa,0xfa, -0xc7,0x29,0x5e,0x42,0xa4,0x1e,0x66,0xd7,0x0b,0x4a,0x62,0x39, -0xe9,0x72,0x81,0xe1,0xe6,0x76,0xb6,0x02,0x87,0xc4,0x78,0x84, -0x9b,0xbb,0x3e,0x37,0x8c,0xbf,0xf1,0x82,0x18,0x8d,0x5e,0x14, -0x64,0x4a,0x2c,0xfc,0x97,0xaa,0x22,0x77,0x69,0x2c,0xcf,0x96, -0x1b,0xf0,0xab,0x57,0x50,0xb0,0xee,0x27,0xe8,0xe5,0x47,0x35, -0xf5,0xd1,0x20,0x39,0xe5,0xf2,0xe3,0x7f,0x54,0x45,0x2c,0xd8, -0x27,0xc6,0xa6,0x49,0x50,0xa8,0x9d,0x83,0xf1,0x7a,0xba,0x7e, -0x63,0x83,0xf6,0x7a,0x03,0xbb,0x5c,0x8c,0x7d,0x9d,0xe8,0xe5, -0xc2,0x55,0xc1,0x98,0xe2,0xeb,0x4d,0xaa,0x5c,0x16,0xdf,0x75, -0x3c,0xf8,0xea,0xe8,0xd0,0xbb,0x2a,0xba,0x5c,0xd0,0x2b,0xbf, -0xee,0x8a,0xc8,0x06,0x29,0xbe,0xc4,0x63,0xdc,0xfc,0xfd,0xdd, -0x4c,0x1e,0x86,0xf4,0x60,0x7c,0x56,0xfd,0xf6,0xdb,0xbc,0x74, -0x74,0x2a,0xde,0xd8,0x86,0x2f,0x57,0x90,0x44,0x71,0x9e,0x10, -0x16,0xb2,0x8d,0xe1,0xf9,0xbd,0x36,0x6a,0xa5,0x60,0x2a,0x43, -0x71,0x3c,0x6b,0xb3,0x9c,0xcf,0xcc,0xa2,0x82,0xff,0xca,0x5b, -0x31,0xf8,0xc1,0x9f,0xcf,0xa4,0x5d,0xe4,0x98,0xa9,0xb4,0x3d, -0x92,0xbf,0xf1,0x9d,0xae,0x24,0xe7,0xb0,0x77,0xf0,0x03,0x2d, -0x90,0x95,0x79,0x17,0x66,0xe8,0x07,0xb4,0xd1,0xe2,0xda,0x58, -0xa6,0x10,0x17,0xf2,0xd7,0xbb,0x1b,0x96,0x31,0x14,0xf2,0x97, -0x6c,0x2d,0x3b,0xde,0x9a,0xdb,0x27,0x7a,0x65,0x36,0x50,0x0b, -0xd5,0x43,0xd8,0x31,0x78,0x59,0xaa,0x83,0xd6,0xae,0x5c,0x4c, -0x41,0x7e,0x6d,0x30,0x7a,0x68,0x90,0xfd,0x33,0x3a,0x0a,0xce, -0x53,0x0c,0x76,0x86,0x06,0xed,0xa5,0xbc,0xc7,0x47,0x08,0x2d, -0xd6,0xbf,0xde,0xc0,0xc9,0x98,0x48,0x8e,0xfc,0xd8,0x2c,0x10, -0x68,0x06,0x76,0xcd,0xd6,0x8c,0xe3,0xc1,0x84,0x6a,0xfc,0xa5, -0xf0,0x3a,0xd1,0x57,0x10,0x2d,0x34,0xee,0x65,0xe6,0x6f,0xde, -0x3f,0x9d,0xa5,0xd7,0x78,0xeb,0x4b,0x47,0x27,0xfa,0xe3,0xdb, -0x8c,0x34,0xc7,0x47,0x25,0x2a,0xf4,0x82,0xbe,0x8e,0x72,0x79, -0x71,0xb0,0x13,0x81,0x24,0xed,0x54,0xcc,0x6a,0x9d,0x48,0xb9, -0x97,0x29,0x5a,0x33,0x9b,0x40,0x02,0xee,0xac,0x16,0x53,0x89, -0xaf,0x64,0x6f,0xc3,0x4f,0x07,0xca,0xa2,0x75,0xfc,0xc0,0xda, -0x46,0x61,0x51,0x11,0xd8,0x2c,0x51,0x3b,0x0b,0xde,0x26,0xf2, -0xbc,0x67,0xad,0x9d,0x47,0xef,0x49,0x66,0x3b,0x22,0x52,0xd6, -0x89,0xf9,0x02,0x2d,0x4b,0xbd,0x57,0x5c,0xbe,0x79,0xff,0xc8, -0xc5,0x5b,0x6f,0x6d,0xdb,0xe6,0xff,0x01,0xf8,0x6c,0x1b,0x38, -0xc9,0x1e,0x68,0x28,0xbf,0x2b,0x30,0xbf,0xd6,0x53,0x74,0xed, -0x39,0xbf,0x79,0xf5,0xea,0xfd,0x7f,0x3c,0x7b,0x18,0xb9,0x28, -0x13,0xf5,0xc0,0x92,0xa5,0xd8,0x58,0x80,0x62,0xe1,0x47,0xde, -0xea,0xd5,0xf9,0xf3,0x57,0x71,0xe1,0x13,0xa0,0x73,0xcb,0xa3, -0x47,0x23,0x5d,0xb2,0x4a,0x46,0x14,0xe5,0xd4,0x2d,0x94,0x1f, -0x1b,0xe1,0xc9,0x58,0x58,0x62,0xfe,0x51,0x6c,0x96,0x6f,0xca, -0x45,0xf5,0x9b,0x8f,0x74,0x02,0xd0,0x43,0x80,0x17,0x88,0x03, -0x1f,0xd2,0xcc,0x46,0x08,0x39,0x94,0x71,0xc1,0xa9,0xcb,0xfa, -0x62,0x8a,0x5e,0x3e,0x67,0x67,0x9f,0x75,0x61,0xbb,0xd5,0xba, -0x75,0x5c,0x17,0xc7,0x9c,0xd1,0xc0,0x93,0x31,0xfc,0xf3,0x79, -0xd2,0x83,0xdf,0xbd,0xe4,0xf3,0x09,0x05,0x97,0x67,0x88,0x02, -0xd3,0x30,0x4e,0x1a,0x57,0x93,0x02,0x0b,0x6e,0x96,0x8a,0x91, -0xd4,0x66,0x64,0x43,0x0f,0x77,0xeb,0xba,0x3a,0x9f,0x03,0x6f, -0x89,0xb0,0xee,0x48,0xc2,0xd7,0xd5,0xb4,0x21,0x5a,0x40,0x60, -0xa7,0x20,0x03,0x1b,0x1f,0x9e,0xcc,0x5c,0xbe,0x6f,0x21,0x70, -0x98,0x7e,0x1c,0x50,0xf8,0x8a,0xd6,0xfa,0xbd,0xb6,0xf0,0x90, -0xf3,0x9e,0xb5,0x51,0xb8,0xec,0x20,0x40,0x41,0x4f,0xd6,0x6b, -0x8c,0x79,0x61,0x37,0x10,0x40,0xb6,0x85,0xb7,0x11,0x62,0x94, -0x26,0x05,0x4e,0xce,0x6a,0x02,0x4c,0x4d,0xdb,0xc1,0xbb,0x49, -0xc2,0x87,0x2c,0x4c,0xf2,0xd9,0x20,0xc9,0xda,0x77,0x1b,0xb7, -0x39,0x09,0x16,0x1f,0xa1,0x02,0x27,0x9d,0x9a,0x73,0xd9,0xe8, -0x6f,0xd6,0xc5,0x79,0xb7,0x0b,0x07,0xc5,0xea,0xf2,0x12,0xad, -0x64,0x81,0xe3,0xfc,0xf3,0xbc,0x5f,0x2f,0x6f,0xd6,0x17,0xe5, -0x53,0x8e,0x79,0xfd,0xc3,0xc3,0xbf,0xbf,0xfc,0xf1,0xc9,0xbf, -0x3f,0x7c,0xfe,0xf4,0xaf,0x4f,0xb2,0xe3,0xf4,0x9f,0xd3,0x03, -0xd9,0x64,0xdd,0x0b,0x4d,0xe9,0x82,0xc4,0xdf,0xe2,0x59,0x67, -0x4d,0x76,0xd9,0x18,0x17,0x40,0x7d,0xc6,0x20,0x91,0x2b,0x32, -0xa6,0x17,0xc5,0xb9,0xd4,0x3b,0x1e,0xd8,0x58,0x8b,0xa3,0x41, -0x7e,0x3c,0x88,0x65,0x15,0x8a,0x3a,0xf7,0x73,0x5d,0xde,0xcc, -0x96,0xe1,0x13,0x67,0x2d,0xb1,0x7a,0xe8,0x8a,0x3c,0x50,0x21, -0x86,0x2b,0x47,0x9f,0xe0,0xe3,0xcb,0x09,0x09,0xf6,0x25,0x0c, -0x93,0xee,0x69,0xf8,0x8c,0xb0,0x8b,0x61,0xf8,0x86,0x6e,0x2b, -0xfe,0xb5,0x40,0xa4,0x21,0x14,0xf8,0x14,0x05,0xdf,0x89,0x48, -0x42,0xf6,0x11,0x70,0xe9,0x53,0xf0,0xd3,0xb9,0xc0,0xe6,0x49, -0x4b,0xa1,0x14,0xa5,0x3f,0x54,0xe4,0xb6,0xc5,0x34,0xa1,0xe8, -0xe9,0x2f,0xd3,0x5e,0x13,0x69,0x4c,0x6d,0x88,0x5d,0x3c,0x78, -0xb1,0x19,0x27,0xfa,0x28,0xeb,0x02,0x64,0x29,0x74,0xcc,0x80, -0xa2,0x1a,0x9d,0xdd,0x76,0x08,0x0c,0xc2,0x2f,0x1a,0xb0,0x09, -0xac,0x30,0x7e,0x1d,0xe4,0x03,0x6b,0x98,0xa0,0x97,0x2f,0xe5, -0x80,0xbc,0x67,0x4d,0xa5,0xe1,0x43,0x12,0x33,0x7e,0x92,0xd1, -0xe2,0xa4,0xc1,0x0a,0x0d,0x85,0xe5,0xc4,0x20,0xb5,0x9e,0x28, -0xa6,0xa3,0x30,0x54,0x5c,0x7c,0xfa,0xeb,0xe5,0xb7,0xd3,0xb5, -0x8a,0x64,0x20,0x48,0x7d,0x4b,0xd6,0x7e,0x35,0x4b,0xa9,0x33, -0x66,0xb2,0x24,0x34,0x8f,0x7d,0xcb,0x6e,0x76,0x24,0xa5,0x22, -0x99,0x8b,0x07,0xc5,0x3d,0x8e,0x4e,0x72,0xce,0x11,0xeb,0xe5, -0x13,0xd6,0x37,0x51,0x47,0xee,0xc8,0x29,0x4c,0x1c,0x70,0xe6, -0x3a,0x2d,0x48,0xa8,0xb1,0xdf,0x43,0xaa,0x32,0xcc,0xee,0xd2, -0x8e,0xc0,0xbf,0xdd,0xaa,0x0c,0x74,0xb1,0xbc,0xbe,0x9e,0x8a, -0xfc,0x83,0xa5,0x32,0xba,0xc1,0xbc,0x66,0xaf,0x01,0xed,0x0b, -0xff,0x15,0x20,0x78,0xf3,0x3d,0x99,0xa8,0x4d,0xf8,0x76,0xcb, -0xd5,0x04,0x0e,0xf1,0xa2,0x23,0xbc,0x00,0x4c,0x99,0xda,0x8c, -0xe3,0x6d,0x81,0xf1,0xfa,0x84,0x33,0xe7,0x14,0xa0,0xaf,0x83, -0x9e,0xad,0x35,0xfb,0x8c,0xe7,0xd1,0x3a,0xab,0x72,0x7b,0x62, -0xe6,0xcd,0x71,0xf3,0x74,0x22,0xc8,0x52,0xc8,0x02,0x15,0x71, -0xba,0x23,0x7c,0xf3,0xc4,0xec,0x19,0x84,0x25,0xb0,0x2c,0xd4, -0xe3,0x11,0xee,0xd8,0x56,0x33,0x13,0x9a,0x61,0x70,0x30,0x83, -0xb5,0x06,0x01,0xd4,0xdb,0xad,0x43,0xfc,0x58,0x1c,0xef,0xb4, -0x5e,0x9c,0xc3,0xc9,0x58,0xe7,0x77,0x8e,0x94,0x48,0x4d,0xe8, -0x98,0x73,0x3e,0x6e,0x80,0x28,0x0e,0x4b,0x62,0x76,0x0e,0x0b, -0xff,0xcd,0xae,0x41,0x6d,0x5c,0x5f,0x42,0x2c,0xd3,0xc7,0x88, -0x8f,0x28,0xfd,0x12,0xf7,0x22,0xb7,0x97,0xe6,0x8e,0x96,0xd3, -0x88,0x0c,0x33,0x4b,0xaa,0xfe,0x9a,0xb2,0x5b,0x36,0x8f,0xce, -0x2d,0x7d,0xfb,0x9c,0x16,0x8f,0x5a,0xd6,0xbb,0x55,0xa2,0x24, -0x89,0xf7,0xd8,0x94,0x5d,0x35,0x52,0x1a,0x5f,0x93,0x09,0x82, -0x56,0x6b,0x01,0xef,0x48,0x37,0x9b,0x3d,0xb2,0xa4,0x26,0x38, -0xc1,0x73,0x96,0x26,0x99,0x73,0xe0,0xa6,0x7a,0xdc,0x83,0x8d, -0xc6,0x66,0xf4,0xe1,0xd3,0xd0,0xe5,0xc5,0x8f,0xcb,0x85,0x0a, -0x7f,0x05,0x96,0x06,0x19,0xcb,0x09,0x10,0xc8,0x67,0xb1,0xe4, -0x6c,0xb9,0x20,0x26,0x5d,0xe0,0x8f,0x5e,0xcf,0x9e,0x80,0x5d, -0x7b,0x97,0xd5,0xba,0xde,0x44,0xc6,0x5f,0xba,0x03,0xbf,0xbb, -0x9e,0x33,0x46,0x38,0x30,0x09,0x59,0x56,0x41,0xab,0x93,0xa1, -0x7a,0xa3,0x94,0xbe,0xd3,0x38,0x08,0x01,0x11,0x5c,0x99,0xe5, -0xd6,0x34,0xbe,0x66,0xe3,0xb1,0x59,0x58,0xd0,0x31,0x0d,0x1e, -0xd5,0x07,0x7c,0x88,0x5f,0x82,0x38,0xb1,0xa6,0xa9,0x85,0x3d, -0x1e,0x9d,0x06,0x70,0x80,0xeb,0x9b,0x45,0x8d,0xcc,0xaa,0xa7, -0x03,0x45,0x72,0x3a,0x08,0xf6,0x47,0xc1,0xad,0x1e,0x24,0x2d, -0x20,0xdb,0xe7,0x7e,0x5c,0x44,0x84,0xdf,0x8b,0x1d,0x94,0xb6, -0xed,0xa0,0xb8,0xbe,0xaa,0x00,0x24,0xd6,0x8e,0x53,0x09,0x50, -0x64,0x2c,0x56,0x5c,0xb8,0x3a,0xd4,0x1d,0x0a,0x90,0xf4,0x71, -0x32,0xe1,0x49,0xa7,0x97,0xb7,0x71,0xa4,0xa4,0x33,0xe7,0x3c, -0x38,0x2b,0x16,0xec,0x1e,0x24,0x0f,0xe7,0x32,0xfb,0x46,0xf4, -0x59,0x09,0x87,0x33,0x74,0x60,0xca,0xfc,0xe3,0xee,0x8a,0x31, -0x77,0xa1,0x4a,0x8e,0xc1,0xa4,0xd8,0x8b,0x2b,0x18,0xb3,0xed, -0xc6,0x15,0xc7,0x3e,0x1c,0x16,0xdd,0x9a,0x28,0x19,0x2f,0x3f, -0x48,0x6b,0xd2,0x6a,0xed,0xed,0xf9,0x9c,0x60,0x9f,0x72,0xfc, -0x10,0xd5,0xf2,0x3a,0x8e,0xaa,0xf1,0x9e,0xec,0x98,0x0e,0x39, -0x38,0xed,0x9f,0x66,0x95,0xdf,0x00,0xd2,0xdc,0x56,0x38,0x99, -0x78,0xb2,0xca,0x12,0xb6,0xf4,0x05,0x88,0x3e,0x37,0x0b,0xb7, -0x03,0xaf,0xc4,0xa2,0xbe,0x1d,0x26,0xea,0x0c,0xe4,0xec,0xff, -0xd4,0x1a,0xe2,0x90,0x45,0x0c,0xc7,0x36,0xa3,0x3c,0x87,0x43, -0xad,0x88,0x9b,0x01,0x7c,0x78,0x8c,0x87,0xcf,0xd4,0xb6,0x06, -0x1e,0x38,0xd0,0xed,0x8c,0xfb,0xde,0x6e,0xd5,0x9f,0xa3,0x31, -0x36,0x8c,0xcb,0xe0,0x75,0xef,0x1f,0x9c,0xb9,0x37,0xa4,0xff, -0x15,0x27,0x57,0xce,0x7f,0xcf,0xaf,0xe3,0xc9,0xe3,0xa5,0x9d, -0x32,0xbb,0xa7,0xf8,0x91,0x89,0x98,0xb5,0xa1,0x19,0x89,0xe4, -0xf1,0x4f,0xc0,0x1a,0xff,0xb4,0xa7,0x0d,0x7b,0x76,0xb1,0x45, -0x61,0xb7,0x72,0x1f,0x76,0x2d,0x60,0x3c,0x4a,0x9b,0x5a,0xf8, -0xdd,0xdb,0x54,0xd8,0xdc,0x5d,0x2f,0xb6,0x15,0xfc,0x68,0xb6, -0x82,0xa9,0xcd,0x56,0x9a,0xe0,0x65,0x87,0x44,0x47,0x3d,0xab, -0x34,0x18,0xd8,0xa6,0xe4,0x80,0x21,0x38,0x07,0x80,0x9e,0x1f, -0xa6,0x2b,0x92,0x87,0xa0,0x61,0xfd,0xb9,0x2e,0xcb,0x77,0x55, -0xbd,0x81,0xf3,0x83,0xea,0xa4,0x74,0xe6,0xb1,0x52,0x4d,0xf2, -0xe0,0xd6,0x3e,0x3e,0x2b,0xe7,0xb6,0x3a,0xf5,0xd5,0xc4,0x89, -0x89,0x95,0x4f,0xa3,0xb1,0xa4,0x4c,0xf4,0xbc,0x8b,0xcf,0x23, -0xeb,0x6c,0x3c,0x5d,0x14,0x1d,0x3b,0x57,0x18,0x9a,0xa1,0xc4, -0x29,0xb7,0xb7,0x71,0xd4,0x6f,0x6e,0x09,0x9b,0x3e,0x05,0x78, -0x27,0xc9,0x05,0x43,0xd5,0x86,0x7f,0xba,0x51,0x37,0xdc,0xf5, -0xc8,0xe1,0x11,0x98,0x19,0xf6,0x30,0xb2,0x75,0x73,0x6f,0xd0, -0x20,0xa3,0xb8,0x69,0x1b,0x8d,0x27,0xb9,0x20,0x21,0xb7,0xb0, -0x0e,0x55,0x7f,0x83,0x53,0x79,0x1d,0x72,0x87,0xa7,0x0b,0xaf, -0xeb,0x10,0x24,0xaf,0x59,0x92,0xa1,0x4a,0x0c,0xbe,0xa0,0x63, -0xf2,0x2a,0xc9,0x5c,0xf8,0x15,0xa8,0x30,0x20,0x2c,0x6c,0xb0, -0x2a,0x30,0x39,0x60,0x23,0xf2,0xc8,0x24,0x24,0xcb,0xda,0x73, -0xed,0x8e,0xa5,0x5d,0x14,0xf5,0xad,0x80,0x0f,0xf0,0x06,0xe9, -0xb4,0xe6,0x48,0x04,0xb2,0x0a,0x40,0x1f,0x6d,0x16,0x61,0x51, -0x8e,0x5c,0x28,0xe1,0x88,0x88,0x73,0xb8,0x5e,0x5b,0xa0,0xf3, -0x40,0x22,0x18,0x69,0x10,0xb4,0xb4,0x1d,0x4c,0x95,0x80,0x32, -0x74,0x7d,0xd0,0x9d,0xaf,0x3f,0xaf,0x18,0x7a,0x0a,0xd1,0x42, -0xfb,0x36,0x47,0x98,0xf8,0xa4,0x7e,0x19,0x5e,0xd7,0xc1,0x48, -0x02,0xb9,0xb0,0x4a,0x82,0x56,0x11,0xc1,0x90,0x33,0xa5,0x61, -0xa5,0xd3,0xe3,0x81,0x9e,0x92,0xb0,0x32,0xe2,0x51,0xa8,0x43, -0xbe,0x19,0x7f,0x3b,0xde,0xb1,0x3c,0x58,0x5b,0xe6,0xb9,0x28, -0x74,0x79,0x8d,0x3c,0xb0,0x25,0xe6,0x93,0x47,0x8f,0xa6,0x31, -0xa8,0x2c,0x77,0x49,0x3e,0x5f,0x19,0xc5,0xc8,0x52,0x26,0xe0, -0x11,0x17,0x22,0x2b,0x6f,0x51,0xc6,0x78,0x2d,0xfa,0x87,0x3c, -0x49,0xfe,0x76,0xbd,0xbc,0x7e,0x4e,0xa2,0x64,0x43,0x2e,0x25, -0x21,0x42,0x75,0xe9,0xca,0x5c,0x5f,0xb7,0x88,0xaa,0xf3,0x72, -0x8a,0xe1,0x72,0x7f,0x41,0xe3,0xcf,0xea,0x6d,0xc9,0xe7,0xae, -0xb5,0x7c,0x8d,0x45,0x50,0x85,0x73,0x09,0x5f,0xfa,0x57,0xd7, -0x88,0x88,0x6a,0x63,0x8b,0x47,0xd5,0xb7,0xdb,0xb0,0x3e,0x9e, -0x0f,0x81,0xf0,0xa2,0x52,0xa3,0x41,0x7e,0xc2,0x40,0xa9,0xce, -0x2b,0x94,0x73,0xf7,0x07,0x56,0x46,0x89,0xc4,0x1b,0xd9,0xae, -0x01,0x10,0x5e,0x1f,0x71,0xd3,0x0f,0x17,0xef,0x3f,0xa5,0xf5, -0x80,0xca,0x02,0xec,0x51,0x31,0x20,0xb1,0x43,0x9d,0xe1,0x89, -0x75,0x7c,0x40,0xcc,0x10,0xe1,0x20,0xc2,0x03,0x10,0x2d,0xe4, -0x5a,0xa5,0x18,0xbe,0x9d,0x86,0x42,0xd4,0x35,0xd0,0x8b,0x74, -0x8e,0xc7,0xac,0x00,0x1a,0x55,0x69,0x66,0x8e,0x85,0xfb,0x88, -0x6c,0x76,0x9d,0x47,0xc8,0x68,0xd9,0xe6,0x77,0xd1,0x8b,0xb8, -0x32,0xa2,0x76,0x52,0xa8,0x84,0x14,0x32,0x37,0xee,0x10,0xbd, -0xfb,0xc4,0x33,0x45,0x54,0x36,0x19,0x06,0xda,0x76,0xed,0x88, -0xa5,0x90,0xb8,0x1b,0x59,0x0f,0xb4,0x6e,0x5c,0x96,0x6a,0x60, -0xb4,0x4d,0xbb,0x5b,0xeb,0x43,0xbb,0xbd,0x5e,0x45,0x63,0x7a, -0x0d,0x9b,0x7a,0xeb,0x30,0x5e,0x4b,0xfb,0xc3,0xd7,0x30,0x66, -0x77,0xfe,0xb2,0x60,0x57,0xa7,0x03,0x56,0x3d,0xb5,0x0e,0x85, -0x72,0x1b,0xa7,0x42,0x3d,0x39,0x9a,0xea,0x78,0x90,0x65,0xad, -0x3a,0x4b,0xbb,0x12,0x4f,0x2b,0xbb,0x47,0x35,0x56,0xb1,0xb4, -0x53,0x99,0xd7,0x20,0x4c,0xc8,0x28,0xf6,0x95,0x2d,0x82,0x2a, -0x5e,0x85,0xe6,0x99,0x35,0xdb,0x59,0x11,0xd7,0x67,0xeb,0x2e, -0x30,0xca,0x9e,0xd1,0xb6,0xb2,0x9d,0x7f,0xc7,0xc3,0xb2,0x14, -0xad,0xa3,0x39,0x47,0x55,0xac,0xfd,0xed,0x2b,0x9d,0xf8,0x78, -0x30,0xfb,0x05,0xc4,0x7c,0x90,0x62,0xcf,0xdf,0x3f,0x2b,0x37, -0x9e,0x44,0x65,0x05,0x76,0xcc,0x92,0xc3,0x4f,0x11,0xb5,0xec, -0x8a,0xd4,0x37,0xab,0x72,0x2d,0xc9,0x45,0x28,0x64,0x79,0xc4, -0x6d,0x79,0x2d,0x13,0x88,0xae,0x2a,0x7b,0x4b,0x6d,0x5e,0xdb, -0x29,0xb1,0x80,0x90,0x98,0x2d,0xe2,0xab,0xc6,0x46,0xad,0x8a, -0xe4,0x24,0x31,0x56,0x82,0x2e,0x58,0x12,0x82,0x72,0x76,0xc1, -0x8a,0xe0,0x1d,0xf4,0xa3,0x90,0x7c,0x33,0xbd,0x78,0x73,0xb3, -0x6a,0x2c,0xee,0x40,0x70,0xb3,0x43,0xee,0x76,0x65,0x39,0x2c, -0x66,0xfd,0xe7,0x0f,0xff,0x3d,0x45,0xb1,0x2d,0x68,0xf5,0xe0, -0x45,0x40,0x66,0xe4,0xb4,0xfb,0x9f,0x8b,0xea,0xd7,0x9b,0xd2, -0x9e,0x7d,0x25,0x7c,0x57,0x00,0x10,0xb0,0xe1,0x1f,0x30,0x6c, -0xd4,0x1a,0x97,0xce,0x35,0xc6,0x1d,0xea,0x0f,0x48,0x8f,0x64, -0x01,0x45,0x63,0xa7,0x08,0xe8,0xc2,0x31,0x2b,0x77,0xb7,0xaa, -0x18,0x77,0xc7,0xb8,0xa2,0x31,0xe3,0xd9,0x90,0x31,0x94,0xda, -0x53,0x30,0xdf,0x7c,0x20,0xd3,0x41,0x66,0xe0,0x21,0x80,0x19, -0xb3,0x3e,0x99,0xad,0x5c,0x22,0x22,0x05,0x79,0x3c,0xdb,0xb1, -0x91,0x16,0xce,0x76,0xd7,0xbc,0xf2,0x1b,0xb2,0xee,0xc8,0x1f, -0xa2,0xe0,0xa8,0x08,0x30,0xe7,0x8f,0xa5,0xd7,0x6b,0x8e,0x66, -0x47,0xe4,0x0b,0xb5,0x79,0x3c,0x1d,0x2b,0x79,0xca,0xfd,0xa3, -0x4f,0x40,0x18,0xfb,0x91,0x08,0xe6,0xa6,0xe5,0x28,0x45,0xd7, -0x33,0x7e,0xe9,0x5e,0x51,0x19,0x69,0x1c,0x1f,0xe2,0xf4,0xb3, -0x9a,0x48,0xf1,0x96,0x8f,0x20,0x44,0xb1,0xe1,0xce,0x99,0x8e, -0xb2,0x03,0xec,0x04,0xe2,0xa4,0xdf,0xcb,0xe9,0x09,0x5f,0x5d, -0x90,0x90,0xe6,0x9f,0x1a,0xd1,0x94,0x50,0x9b,0xa2,0xcf,0x34, -0xf8,0x06,0xa1,0x4d,0x02,0x05,0x0b,0xda,0x41,0x64,0xf5,0x56, -0x95,0x15,0xb6,0x5c,0x5a,0xb6,0x6b,0xbb,0x80,0x72,0xd9,0xc6, -0x9b,0x26,0x7b,0xfa,0xd0,0x5c,0xcb,0x0b,0x42,0x64,0xf7,0x5a, -0x38,0x0a,0x70,0x6b,0x51,0x04,0xde,0xd0,0xe4,0x3e,0xc3,0x78, -0x81,0x0a,0xa4,0xf2,0xbf,0x0f,0x93,0xc4,0xbe,0xb5,0xc0,0xcb, -0x29,0x73,0xfc,0x61,0xa7,0x3b,0x0a,0xcd,0xe3,0x28,0x38,0x10, -0xfa,0x8c,0x2b,0xcb,0xfd,0x2f,0x8f,0xd1,0xee,0x3b,0xc5,0xe9, -0x04,0xd6,0xca,0xd8,0x9a,0x9a,0x48,0x2d,0x61,0x5f,0xb7,0xb6, -0x5b,0xfb,0x1e,0x73,0x0d,0x8a,0xdc,0xa2,0xcd,0x0d,0xc3,0xbb, -0x72,0xff,0xf8,0x1e,0xdc,0xc8,0x1d,0xb0,0xf5,0xa8,0xec,0xa5, -0x02,0xfe,0xe1,0xab,0x05,0xe4,0x66,0xcc,0x53,0x5b,0xee,0x6a, -0x49,0xb5,0xcb,0xb3,0x8b,0xea,0x30,0x56,0x27,0x8b,0x4c,0x4d, -0xbe,0xcf,0xbc,0xd3,0xa9,0x8a,0xdc,0xfe,0xb0,0x9a,0x5a,0x13, -0x88,0xb4,0xa7,0x0f,0xba,0xdd,0xe4,0xe9,0x63,0x54,0x6d,0x72, -0xd5,0xc2,0x4a,0xb1,0x99,0x98,0x75,0xff,0xa5,0x28,0x1a,0x76, -0x18,0x40,0x61,0x91,0xb5,0x45,0xab,0xe0,0x30,0x50,0xf9,0xe7, -0xce,0xd9,0x30,0x14,0x8e,0x63,0x3f,0x7d,0xcc,0x7d,0xaa,0x88, -0x12,0xa8,0xf4,0x6f,0x16,0x72,0xc7,0x7c,0xa9,0xbf,0xec,0x2d, -0x66,0x06,0x05,0x4d,0x47,0x3f,0x22,0x1b,0x0b,0xab,0x4e,0xb6, -0xf7,0xa1,0x3c,0x72,0xdd,0xfb,0x59,0xf3,0xce,0xba,0x64,0x45, -0x9c,0x7f,0xf4,0x22,0xf0,0x0e,0xc4,0x1c,0x1f,0x9d,0xe4,0x01, -0x06,0x71,0xe5,0xe3,0x41,0xcc,0x47,0x1e,0x3e,0x15,0x1c,0x21, -0x84,0x42,0xdb,0xd1,0x60,0xc5,0x10,0x91,0xf4,0xa6,0x88,0x06, -0x87,0x10,0xb1,0x51,0x94,0xe3,0x59,0x41,0x01,0x7a,0x3e,0x0d, -0x41,0xe1,0x75,0x78,0x78,0x20,0x39,0x7c,0x29,0x2e,0xcf,0x03, -0x2b,0x96,0x56,0x22,0x55,0x0d,0x5c,0xd0,0xee,0xc2,0xbb,0xa4, -0x6b,0x13,0x99,0x9c,0xc5,0xcf,0xc7,0x19,0x5c,0x90,0xf8,0xe1, -0x9b,0x7b,0xf0,0x66,0xb3,0x73,0xea,0xf4,0x55,0xe5,0xaf,0x40, -0xbe,0x4b,0x09,0xa5,0x96,0x80,0x08,0xed,0xf2,0xda,0x63,0x14, -0xe0,0xae,0xdd,0x7d,0x3d,0x9b,0x5c,0xd8,0xa4,0xd9,0xdd,0x8e, -0x17,0xa4,0xa7,0x5e,0xc5,0x99,0xc1,0x97,0xb6,0xf0,0x65,0x19, -0x43,0x6f,0x28,0x1a,0x01,0x0b,0x1f,0xd6,0x7f,0x7c,0x83,0x58, -0x9a,0x6e,0x9a,0xbc,0xce,0x78,0xf6,0x53,0xc6,0x8f,0xa0,0x81, -0xbb,0x76,0x0c,0xb4,0x1a,0x8e,0xc8,0x2f,0xe6,0x6f,0xb5,0x8d, -0xac,0xa5,0x36,0x3b,0x06,0xe3,0xb6,0xfe,0x84,0x6f,0x50,0x8a, -0x84,0xb3,0x2c,0x92,0x9a,0x98,0x54,0xd2,0x3b,0x46,0x2f,0xdf, -0xc7,0x08,0x89,0x6f,0xd2,0x55,0xb0,0x19,0xa0,0x0d,0xd6,0x61, -0xc4,0x62,0xa5,0xb8,0xdb,0xa9,0xe0,0x03,0x6c,0x10,0x95,0xfc, -0xf0,0x87,0x5e,0xde,0x64,0xa5,0x79,0x60,0xf1,0x67,0xdc,0xed, -0x4f,0x94,0x21,0x98,0x58,0xb7,0xe5,0xf9,0xee,0xe3,0x6a,0xf4, -0xe5,0x82,0x0e,0xfa,0x46,0x01,0xc5,0xe0,0xeb,0xaf,0xbf,0x18, -0x98,0xe9,0x7a,0xcd,0x3a,0xbc,0x55,0x01,0x3f,0xfb,0xf0,0xd7, -0x20,0xf5,0xf0,0x07,0xfc,0x60,0xc2,0xa1,0x4f,0xfa,0x65,0xe4, -0x08,0x4a,0x29,0xf2,0x7b,0xbb,0xdd,0xf3,0x04,0xb9,0xbd,0x77, -0x42,0x73,0xce,0x06,0xa7,0x97,0x80,0x11,0xb8,0x27,0xf3,0xf5, -0x82,0x92,0x6f,0x35,0x54,0x7b,0x83,0x9d,0xa1,0x80,0xe9,0x1c, -0x97,0x36,0x19,0x9f,0x9d,0xbd,0x7b,0x70,0x72,0x76,0xb6,0x39, -0x3b,0x5b,0x9f,0x9d,0x2d,0xce,0xce,0x2e,0x27,0x89,0xc1,0x80, -0x34,0xd3,0x0b,0x20,0x83,0x27,0xf2,0xa6,0x4e,0x91,0xe0,0xab, -0x28,0x67,0x18,0x55,0x16,0x2a,0xdc,0x1e,0x4f,0xb6,0xe3,0x17, -0x50,0xf1,0xe4,0xe4,0x18,0xfe,0x9d,0x02,0x7b,0xed,0x25,0xa6, -0x82,0x3d,0x7e,0x83,0x4f,0xa1,0xad,0x8b,0x46,0x7d,0xbb,0xd0, -0x93,0x5b,0x7c,0xa6,0xea,0x4f,0x18,0x58,0x1c,0xb6,0x3f,0xba, -0x46,0x85,0xb6,0xc7,0xf7,0x5e,0x7c,0xb6,0xed,0xfc,0x73,0x32, -0x2a,0xb2,0xc4,0xb8,0x87,0xdc,0x8b,0xe4,0xec,0x6c,0x9c,0xf4, -0x1c,0xb8,0xbd,0xe4,0x5e,0x9a,0xf4,0x1a,0xad,0xf7,0x92,0x2c, -0x2e,0x35,0xca,0x93,0x9e,0xed,0xa1,0x91,0x97,0x8e,0x3f,0x3f, -0x4b,0x26,0x59,0xea,0x06,0xf5,0x02,0xff,0x4e,0xb2,0x7b,0xa3, -0xec,0xec,0xec,0x8b,0x2d,0x74,0xe2,0x46,0x03,0xad,0x6f,0xe1, -0xff,0x51,0x0f,0x50,0x3a,0x31,0x2b,0x32,0x3b,0x00,0x38,0xf3, -0x3d,0x60,0x51,0x07,0x69,0xfa,0xc1,0xfe,0xa2,0x8c,0x34,0x03, -0x2c,0x4f,0x26,0xdb,0xa4,0xe7,0xbd,0x6a,0xaf,0x28,0xfc,0xc2, -0xfc,0x1b,0xa0,0x3b,0xbb,0x97,0x6d,0xfb,0xf7,0xa0,0x36,0x82, -0x66,0xe8,0xa4,0x59,0xf8,0xc1,0x82,0x5f,0x84,0xf0,0xf6,0xa8, -0x8f,0x17,0xb6,0x63,0xed,0x10,0x9a,0x89,0x0a,0x7e,0x06,0x13, -0x84,0x6f,0x28,0xf1,0xed,0xfa,0xa1,0x36,0xef,0x99,0xe8,0x5b, -0x6a,0xe9,0xfd,0xf8,0xc1,0xba,0x69,0x93,0xf0,0x4e,0x7b,0xff, -0x9c,0xc4,0x68,0xc6,0x26,0x19,0xcb,0x7e,0x6b,0x82,0x77,0xc8, -0xf3,0x88,0x2e,0xea,0xcd,0x9f,0xc0,0xcf,0xf4,0x00,0x8e,0x3c, -0xb0,0xb8,0x7b,0xfa,0x38,0x0f,0x0a,0xff,0x69,0x1f,0x51,0x65, -0xe6,0xd1,0xf7,0x0f,0x9f,0x3d,0x0b,0x4b,0x63,0xb0,0xe5,0xbd, -0xe5,0x7f,0x7c,0xf8,0xc3,0x93,0xb8,0x38,0xc5,0x9a,0x28,0x88, -0x04,0x46,0x7b,0x6a,0x72,0x26,0xd2,0x54,0x66,0xe0,0xe8,0x17, -0xb6,0xd0,0x56,0x27,0x5a,0x50,0xf7,0x30,0xe8,0x33,0xf6,0xff, -0xf0,0xf9,0xf3,0x5f,0xf2,0x08,0x13,0x8e,0x86,0x32,0xf3,0xf3, -0xb3,0x27,0xff,0xf9,0xf8,0xa7,0xb8,0x84,0xc5,0xe7,0xa3,0xef, -0x9e,0x7e,0x1f,0x21,0x27,0xa7,0xe8,0xb5,0x5b,0xba,0x05,0xde, -0x02,0x5f,0xdd,0x6c,0x17,0x9b,0x2b,0xfc,0xef,0x18,0x3f,0xb2, -0xe3,0xf4,0xe2,0xaa,0x9a,0xcf,0xb6,0xcb,0xcb,0x63,0xda,0x97, -0x99,0xe2,0xe3,0xc9,0x46,0xbb,0xef,0xed,0x72,0x36,0x03,0x2a, -0x1c,0xf7,0x80,0x7b,0x64,0xe9,0xd9,0xd9,0xec,0x5e,0xb6,0x68, -0x2c,0x2b,0x5a,0x9c,0x54,0x22,0xce,0x80,0x0a,0xf8,0x24,0x53, -0x73,0x1d,0xd2,0x02,0x48,0x2a,0x18,0x7c,0xf0,0x0a,0xcb,0x21, -0xe2,0x1b,0x23,0xa9,0x6d,0x73,0x07,0x56,0xf9,0xeb,0xf6,0x15, -0x8c,0x8e,0xc7,0xe6,0x86,0xba,0x67,0x34,0x90,0x0a,0x7c,0x6f, -0x96,0x8d,0x68,0x10,0x6d,0x00,0xa5,0xa3,0x62,0xfc,0x02,0x46, -0xf1,0x99,0x80,0xb6,0xb3,0xdb,0x78,0x71,0x7f,0x4c,0x74,0xbf, -0x01,0xaa,0x5f,0x00,0xb3,0xbd,0x37,0x06,0x50,0xee,0x1b,0xb1, -0x5a,0x2f,0xee,0xbf,0x18,0xbf,0xb8,0x9b,0xf4,0xce,0xee,0xce, -0xea,0x7b,0x48,0x38,0x98,0x08,0xd2,0xc3,0xac,0x84,0x22,0xd6, -0xe4,0x97,0x1f,0xb7,0xfa,0x13,0x2c,0x21,0xe0,0xc4,0x80,0x94, -0xf4,0xec,0x16,0xfe,0x05,0xb2,0x94,0x04,0x7a,0xdd,0x8a,0xcc, -0xab,0x0e,0x3d,0xb7,0x25,0xd1,0x95,0xf8,0xd5,0xad,0xab,0x72, -0x8a,0xb7,0x5c,0xf7,0x5f,0x5c,0x9d,0xcd,0x28,0x81,0x85,0xb0, -0xe2,0xfe,0xe7,0xdb,0xb3,0xb3,0xfb,0xaf,0xcc,0xda,0x12,0xd1, -0x7f,0xdc,0x2c,0x91,0x35,0xdf,0x3f,0x2b,0xa2,0x81,0xa4,0xe3, -0x17,0x9f,0x27,0xc0,0xb5,0xee,0x65,0x51,0xc6,0xd9,0x04,0x1b, -0x50,0xb9,0x0e,0x6a,0x9e,0x01,0xa0,0xb3,0xe9,0xf1,0xe5,0xc3, -0xe3,0x6f,0x27,0x77,0x03,0xf3,0xe7,0x5d,0x58,0x61,0xb4,0xed, -0x67,0x50,0xc3,0x4a,0x82,0x4e,0xc9,0xf3,0xd2,0x70,0x8a,0x9c, -0x37,0xae,0xaa,0x57,0x57,0x45,0x72,0xf2,0x2e,0xe9,0x49,0xf2, -0xf1,0x9f,0xbf,0xfa,0xea,0x8b,0x3f,0xeb,0x49,0x06,0xb3,0xf1, -0xed,0x07,0xf8,0x33,0x92,0x02,0xf9,0xc9,0x29,0x7d,0x3e,0x23, -0x1b,0x6e,0x8a,0x39,0xf6,0x48,0x82,0xaf,0xa5,0x98,0xd1,0xa3, -0x06,0xb2,0xbc,0x2d,0xff,0xab,0xaf,0x1e,0xfc,0xe5,0xcf,0x5b, -0x2c,0x75,0x7a,0x3a,0x38,0x31,0x5f,0xfd,0xf9,0x8b,0x07,0x27, -0xdb,0xc1,0xc9,0x83,0x2f,0xba,0x98,0x96,0xed,0xc8,0xa8,0xde, -0x13,0x09,0x7d,0xe9,0x25,0x8e,0x31,0xd6,0xa7,0x45,0x83,0x32, -0x6b,0x0d,0x52,0x23,0x0a,0xb5,0x7a,0x18,0x71,0x76,0xf8,0x2c, -0x24,0xb8,0xf7,0x0f,0xa3,0xeb,0x61,0x35,0x53,0x1f,0x4f,0xd8, -0xba,0x80,0xb7,0x7d,0x52,0x04,0xb4,0xe8,0x3e,0xa2,0xc3,0xdd, -0x8e,0xe4,0xbf,0x42,0x8e,0xc6,0xfc,0xd1,0x62,0xcb,0x17,0x41, -0x4d,0x02,0x85,0x3e,0x9e,0x1e,0x99,0xe1,0x53,0xa5,0x78,0x98, -0xf6,0xf5,0xe1,0x30,0x79,0x94,0x7c,0xf7,0xfc,0x87,0xef,0x13, -0x4f,0x91,0xa4,0x68,0x51,0x93,0xc1,0xbc,0x33,0xd8,0xf9,0x72, -0xa7,0x42,0xea,0x27,0x59,0x78,0xb1,0x92,0x85,0x97,0x2c,0x8a, -0x47,0xf8,0x4f,0x0c,0x21,0xa6,0x05,0x5e,0x02,0x1e,0x74,0x81, -0x4e,0xeb,0x2f,0x1c,0xe5,0xca,0x3b,0x1f,0xce,0x9a,0x33,0x38, -0x4a,0x35,0x01,0xcb,0xaa,0x38,0x5c,0xb4,0x94,0x0c,0x45,0xe4, -0x82,0x90,0x8d,0x95,0x33,0x15,0x61,0xfb,0x1b,0x36,0x1c,0xff, -0x71,0xf9,0x08,0xb6,0x5c,0x32,0xd7,0x51,0x87,0x0e,0x1d,0x22, -0x7a,0x68,0xdc,0x59,0x43,0xd3,0xb7,0x78,0x0e,0x29,0x17,0xb3, -0x47,0x48,0x44,0x29,0x45,0x73,0x63,0x81,0x95,0xab,0xa7,0x49, -0x82,0xce,0xea,0x58,0xae,0xd5,0x3e,0x1d,0x77,0x55,0x91,0x1c, -0x77,0x0e,0x08,0x4f,0xec,0x6a,0xed,0x1d,0x9b,0xab,0x16,0x80, -0x51,0x9c,0xbb,0x22,0xf9,0x9a,0xb9,0xc9,0xe9,0xd7,0xf7,0xe5, -0x47,0x32,0x54,0xff,0x55,0x95,0x91,0xb1,0x06,0x72,0x51,0x02, -0x33,0xb2,0xc4,0x56,0x3d,0x87,0x7d,0x6a,0xdc,0x3e,0xa4,0xdd, -0xd1,0x90,0xe5,0xce,0xd1,0x82,0xad,0x3f,0x1d,0xa8,0xa1,0xc5, -0xfc,0x07,0x91,0xe5,0x43,0x0d,0xdf,0x47,0x74,0x3a,0x28,0x3e, -0xbf,0xaa,0x66,0x20,0x22,0x1c,0x95,0x9f,0xc3,0x18,0x20,0xf9, -0xb4,0x99,0xa7,0x39,0x89,0x69,0xe0,0xd2,0xb3,0xd7,0xdf,0x9f, -0x97,0x26,0xa5,0xc5,0xf4,0x28,0x0d,0xd1,0x71,0x61,0xc1,0x87, -0x42,0xe6,0x01,0x19,0xe6,0x7c,0x4c,0x3b,0x19,0x2e,0x8d,0x08, -0x15,0xfb,0xb1,0x40,0xc3,0xb7,0x76,0xfa,0xbd,0x13,0x13,0xe3, -0x63,0xca,0xf6,0xba,0x9f,0x03,0x0b,0x95,0x32,0x09,0x8e,0x7a, -0xca,0xd8,0x68,0xcd,0x13,0x8c,0x30,0xc1,0x43,0x63,0xd8,0xed, -0x37,0x14,0xfd,0x00,0xfb,0xb4,0x19,0xb4,0x8b,0xd2,0x58,0x39, -0x66,0xfa,0x0a,0x11,0x8b,0xb4,0x1a,0x8c,0x51,0x50,0xd8,0x96, -0x9c,0x4a,0xaf,0x3a,0x72,0x40,0xd1,0x83,0xde,0xc1,0x92,0xbd, -0x13,0x2d,0x6b,0xef,0x75,0x1c,0x9a,0x9e,0x02,0xbb,0xdd,0x10, -0xa6,0x3a,0x61,0x23,0xe4,0x51,0xa2,0x7d,0x59,0xf0,0xd9,0x4c, -0x5d,0x56,0x18,0xa0,0xd2,0x20,0xfc,0x80,0x78,0xd2,0x6f,0xd0, -0xa3,0xb0,0xfc,0xca,0xc0,0xa7,0x10,0xdf,0xf4,0x08,0x9f,0xb6, -0x2d,0x3e,0xff,0x13,0xa3,0x98,0xc9,0xca,0xe1,0x89,0xfd,0x5d, -0x65,0xe1,0xb8,0xe4,0x60,0xe5,0xe0,0x7b,0x45,0x41,0x34,0xb3, -0xe4,0x4f,0x89,0xd0,0xce,0x9e,0x1a,0x29,0xbf,0xa7,0x9b,0xed, -0xb2,0xd1,0xdd,0x2e,0xbf,0xc3,0x8f,0x8f,0x7e,0x27,0x57,0x9e, -0xe2,0x7d,0x90,0xf1,0x53,0x55,0x1f,0x5f,0x0f,0x4b,0xa3,0x55, -0xbf,0x69,0x9b,0x80,0x51,0xea,0x6b,0xcd,0xbc,0x1d,0xce,0xaa, -0x42,0x58,0x7b,0xc3,0xc8,0x68,0xf7,0x53,0x6a,0xe0,0x21,0x54, -0xe1,0xc9,0xdd,0xd3,0x3e,0x27,0xa7,0x6a,0x66,0x77,0x44,0xd8, -0xcf,0x7c,0x27,0xa7,0xd1,0xf8,0x7a,0x92,0x8f,0x27,0x00,0xb9, -0x77,0x35,0x19,0x41,0x69,0x83,0x21,0xac,0x9f,0xce,0x8a,0x6a, -0x76,0x50,0x89,0x35,0x6c,0x37,0x94,0xdf,0x8f,0x39,0xf4,0x86, -0xe0,0x98,0x06,0xd0,0xfa,0x6e,0x97,0xe5,0xff,0x9f,0x42,0xd6, -0x48,0x3c,0xab,0xe8,0x35,0x2e,0xee,0xb2,0xf9,0x6e,0x79,0xa3, -0xbb,0x96,0xa7,0xcd,0x69,0x9c,0x7d,0x7d,0x66,0xbc,0x9a,0x11, -0xde,0xdd,0x1b,0x9a,0xff,0xf3,0x67,0x80,0x5c,0x15,0x00,0x92, -0xc2,0x8b,0x1a,0xf8,0xe1,0x91,0xb4,0x96,0xe2,0xc1,0x58,0x1b, -0x38,0x8a,0x59,0x44,0x02,0x88,0x0e,0xcf,0x4e,0xa6,0x09,0x2e, -0xf8,0x8a,0xbd,0xfb,0xbf,0xb3,0x5b,0x84,0x2c,0x37,0xcf,0xe1, -0x83,0xa5,0x87,0x7c,0xd0,0x22,0xc0,0x47,0x07,0xfd,0xd5,0xa0, -0x0f,0xeb,0x78,0x9b,0xb7,0xf7,0x6c,0xa5,0xce,0x8d,0x5a,0x14, -0x9d,0x58,0x01,0xf4,0x83,0x6d,0x93,0xdf,0xfb,0x3d,0x8a,0x3c, -0x04,0x5f,0x9e,0xdd,0xad,0x9a,0xea,0xb0,0xd4,0xda,0x30,0x88, -0x05,0x7e,0xa8,0x86,0x48,0x81,0x10,0x0b,0xa9,0xbb,0x48,0x9e, -0xf5,0x10,0x8b,0x27,0xf0,0xa2,0xb1,0x3d,0x76,0xbb,0xbf,0x1b, -0xa5,0x1f,0x8d,0x4f,0x1a,0x30,0xf9,0xae,0xd8,0xc0,0x2f,0x0e, -0x2c,0x52,0x24,0x14,0xfb,0x7c,0xfe,0x2c,0x70,0x56,0x2c,0xf8, -0x24,0x10,0x9d,0x2c,0x14,0xc2,0xe9,0xbc,0x5a,0x69,0xa1,0x8f, -0x18,0xac,0xfc,0x03,0xde,0x81,0x16,0x06,0x3c,0xb6,0xfa,0x6a, -0x5c,0x9c,0x79,0xab,0xe1,0x2d,0xc6,0x09,0xbf,0xaf,0x9c,0x4c, -0x8c,0xef,0x3e,0x58,0x58,0xa7,0x6c,0xdc,0x66,0x63,0xef,0x31, -0x32,0x2d,0xfd,0x24,0xf9,0x72,0xb9,0xf2,0x6e,0xa1,0xca,0x59, -0xf1,0x39,0x6e,0x9e,0x9c,0xe8,0x09,0x9f,0xb4,0x95,0x36,0x5c, -0xd5,0x92,0xb1,0x56,0x9b,0x58,0x01,0x6a,0xbb,0x75,0xde,0x8d, -0x44,0x5b,0x6d,0x4a,0xc6,0x51,0x7e,0xc1,0x71,0xad,0xb6,0xfa, -0x76,0xc2,0xb6,0xaa,0xaf,0xa7,0x2b,0x6b,0x65,0xb9,0x05,0x81, -0x7b,0x46,0x8a,0x11,0xed,0x61,0xcb,0x1e,0xe2,0x49,0xb6,0x07, -0x14,0x6d,0xf1,0x00,0x20,0xae,0x08,0xb0,0x8a,0x8f,0x42,0x12, -0x1d,0xec,0x59,0xe8,0x56,0x99,0xf5,0xa8,0x02,0x14,0xdd,0xdf, -0x8f,0x91,0xea,0x05,0xe4,0x3b,0x74,0x74,0xbb,0x31,0x14,0xa8, -0x8c,0x9d,0x14,0x4d,0x8c,0x9c,0x25,0x67,0xc9,0xf6,0xf3,0xcf, -0x0f,0x8c,0xb0,0x5c,0x10,0xaa,0x0e,0x8d,0x50,0x8b,0x98,0x24, -0x57,0xcc,0xee,0x6d,0xef,0x9e,0xc9,0xdf,0xa1,0xfe,0x2f,0x6a, -0xc3,0xf4,0xef,0xe5,0x09,0xc5,0xe3,0xb2,0x64,0x27,0xd7,0x0b, -0xd6,0x0f,0xd2,0x92,0xa0,0x64,0x14,0x2a,0xd5,0x45,0x05,0x69, -0x7d,0x70,0xc6,0xf2,0xb7,0x1f,0xf6,0xe5,0xdd,0x96,0xe7,0x6f, -0xaa,0xcd,0xde,0xec,0xe5,0xde,0x9c,0xeb,0x3a,0xca,0x42,0xea, -0x6f,0x9d,0x57,0x1d,0x08,0xe0,0x04,0x16,0xe7,0x82,0x08,0x8a, -0xea,0x8a,0x11,0x80,0xbc,0x96,0x83,0xf2,0x35,0x7b,0xf0,0x9b, -0x66,0xfa,0xb8,0xee,0xe0,0xcc,0x7a,0x38,0x93,0xde,0x05,0x6f, -0x9d,0xc2,0x2a,0xac,0x91,0xbe,0xdc,0x42,0xf6,0x34,0x65,0x0e, -0xd7,0xec,0x43,0xba,0xc5,0x93,0x64,0xc8,0x03,0x1a,0xa5,0xb5, -0x17,0xaf,0x86,0x5e,0xf3,0x04,0xbc,0x80,0x50,0xa2,0x39,0x99, -0xc3,0x12,0x5e,0xbd,0x80,0xfc,0xa5,0x07,0x75,0x75,0xea,0x73, -0x7b,0xa0,0x75,0xe1,0x9c,0x62,0xb4,0x26,0xf2,0x26,0x9e,0x3a, -0x23,0xb8,0x69,0x7c,0xde,0xce,0xa1,0xfc,0xcd,0x0a,0x7d,0x3d, -0xcf,0x3d,0xc1,0xce,0x7a,0x59,0xa2,0xef,0xe5,0xcd,0x6a,0xbb, -0xed,0xa4,0x1d,0xfa,0x8b,0x3e,0xfa,0xf0,0xc3,0x73,0xc3,0xe8, -0xa4,0x53,0x7e,0x3f,0x53,0x60,0x1d,0x85,0x9f,0x29,0x94,0xce, -0xf2,0xe9,0x3e,0xb8,0xbb,0xdd,0xc1,0x9f,0xbb,0x7b,0x73,0xa9, -0x72,0xe6,0xbb,0xb5,0xd0,0xe8,0xd0,0x0a,0x86,0x4d,0x2f,0xce, -0x8b,0x00,0x6a,0xbc,0xd0,0x39,0xc7,0x01,0x5b,0x7f,0x88,0xa1, -0x8d,0xcf,0xe0,0x2e,0xd1,0x8a,0x4f,0xc7,0xa5,0x94,0x0c,0xd0, -0x32,0x4a,0xfd,0x8b,0x40,0x74,0xf2,0x3e,0x01,0xf9,0x53,0x4a, -0x02,0x5c,0x7b,0x47,0x7c,0x00,0x19,0x07,0x30,0x91,0x65,0xa3, -0x41,0x57,0x72,0xb7,0xdb,0x69,0x10,0xf3,0x71,0x40,0x01,0x52, -0xbc,0x24,0x6f,0xc2,0x59,0x3d,0xc3,0x97,0xbd,0x34,0x21,0xbe, -0x52,0xc7,0x4c,0xb3,0xd1,0xf1,0x20,0x3f,0x3f,0x5c,0xe8,0x3c, -0x43,0x4b,0xd4,0xfc,0x4b,0xed,0x1e,0xab,0x0c,0xf6,0xcf,0x29, -0x65,0xc7,0x73,0x26,0x4e,0xc5,0x24,0x0b,0x4d,0x81,0xe0,0x7c, -0x68,0x99,0x02,0xfd,0x04,0xb4,0xe5,0x9f,0x4e,0xcc,0x39,0xfc, -0x39,0x27,0xc7,0xc9,0xa9,0xe7,0x04,0x7c,0xd4,0xc0,0x3b,0x59, -0x46,0x4c,0x89,0x4e,0x91,0x62,0xbc,0x59,0x82,0x51,0x79,0x03, -0x84,0x61,0x40,0x29,0x4a,0x20,0xb3,0x2e,0xaa,0x88,0xd0,0x10, -0x51,0x6b,0xbd,0x86,0x47,0x34,0x19,0x7b,0xa1,0x23,0xf2,0xb4, -0xe5,0x3d,0xa1,0x61,0x36,0x5d,0xf5,0x6f,0x16,0x6c,0x5c,0x80, -0xee,0xcc,0xb6,0xf4,0x79,0x6b,0xe9,0xf3,0x96,0xd2,0x43,0xb2, -0xcf,0x46,0x28,0xf0,0x2f,0x90,0xb1,0x73,0x2d,0xab,0x46,0x21, -0x38,0x58,0xc0,0x50,0x31,0x58,0x55,0x52,0xcb,0x9f,0x2c,0x1a, -0x5c,0x5b,0x3a,0x8c,0x76,0x67,0x42,0xcc,0x0d,0xcc,0x18,0x90, -0x37,0xe1,0x67,0x41,0xed,0xf2,0x70,0xea,0x92,0x59,0xb9,0x01, -0xde,0x6a,0x2b,0xd4,0x45,0x70,0xf3,0x6d,0xdf,0x57,0xc9,0xf5, -0xd7,0xce,0x88,0xe2,0x51,0x37,0x12,0x77,0x8a,0xc0,0xdb,0xf4, -0x52,0x1d,0xb5,0x54,0x48,0x13,0x43,0x1b,0xca,0x24,0xcb,0x55, -0x1b,0xc2,0x5d,0x62,0x7d,0x85,0xcd,0xd9,0x0d,0x2b,0x34,0x20, -0xc6,0xea,0xec,0xa0,0xb5,0x57,0xd3,0xba,0x37,0xc0,0x09,0xe5, -0x52,0x0b,0x45,0xc9,0x46,0x1b,0x72,0x34,0x8a,0x14,0xf8,0x26, -0x29,0x3e,0xff,0x6c,0x80,0x82,0x80,0x01,0x4e,0xb8,0x67,0x23, -0x8d,0xe4,0x47,0xdd,0xcf,0x85,0xe3,0xab,0xe8,0xa0,0x1b,0x00, -0x47,0xb0,0x44,0xd8,0xfd,0x8d,0xdf,0xa5,0x66,0x14,0x7f,0x40, -0x22,0x1e,0x87,0x1b,0x9b,0x1b,0x36,0xd2,0xef,0x1a,0x1f,0xc5, -0xd9,0xbb,0x25,0x32,0x06,0xfa,0x6e,0xf8,0x83,0x81,0x86,0x68, -0xb4,0x41,0x6f,0xa2,0x98,0x2a,0xd8,0xa1,0xf7,0x06,0x42,0xcb, -0x64,0x59,0xb3,0x02,0x7e,0x3a,0x0a,0x5b,0xb3,0x6f,0x47,0x9d, -0x9e,0xd8,0x79,0xb3,0x1b,0x9b,0x13,0xd8,0x7d,0x5b,0x71,0x21, -0x83,0xf6,0x08,0x27,0xce,0x3a,0xe5,0xe3,0xa2,0xd3,0x34,0x63, -0x9b,0x58,0x30,0x70,0x2e,0x23,0x9a,0xa1,0x73,0x07,0x21,0x17, -0xe4,0x50,0xf5,0xb1,0xff,0xbd,0xe4,0x13,0xcd,0x3a,0x1d,0x6a, -0xc8,0x2b,0x3f,0x8a,0xc4,0x61,0x52,0xe8,0xac,0x88,0x54,0x62, -0x1c,0x5a,0x3f,0x1b,0x41,0x16,0x37,0x97,0x47,0xcd,0x35,0x35, -0xce,0xa3,0xa6,0x22,0x84,0xcf,0x51,0x29,0x75,0xd0,0x7e,0xea, -0xa6,0x12,0x99,0x50,0x43,0xb3,0xae,0x04,0x10,0xb0,0x2f,0x01, -0x74,0x4e,0x46,0xf8,0x33,0x87,0x06,0xbb,0xdd,0xb7,0x18,0x53, -0x1a,0x23,0x4b,0x5e,0x56,0x70,0x9a,0xc3,0x2f,0xf6,0x57,0xa6, -0x30,0x47,0xc6,0x77,0x1d,0x76,0x68,0xbe,0xae,0x5f,0x61,0xc4, -0xd9,0xf5,0xf2,0x16,0xfd,0xdc,0x8e,0x28,0xc8,0x45,0x9a,0x3c, -0x7b,0x0f,0x33,0xf5,0xee,0x88,0xca,0x9a,0xa3,0x9b,0xc5,0xba, -0xbc,0x58,0xbe,0x42,0x2b,0xb7,0xd9,0x11,0x12,0x56,0x59,0xd7, -0x18,0xd7,0xfc,0x28,0xe9,0x61,0x75,0xdb,0xb4,0x33,0x46,0x74, -0xed,0x5b,0xbb,0x3e,0x7b,0xd8,0x9e,0x39,0x3e,0x45,0x67,0xee, -0x01,0x3d,0x5a,0x58,0x5d,0x46,0xdb,0xf4,0x3e,0xe6,0xa6,0x27, -0xf4,0x06,0x37,0xf4,0xab,0xb7,0x9e,0xc7,0x27,0x64,0x4b,0x52, -0xb2,0xdb,0x85,0x4d,0x3d,0x1e,0xa0,0x4f,0xdc,0xeb,0xc2,0x81, -0xe5,0x1c,0x00,0xa9,0x11,0x7a,0x0f,0xde,0x76,0xca,0x56,0x57, -0xae,0x30,0x3e,0x0b,0x3f,0xc8,0x1a,0x27,0x77,0xb1,0x48,0xd2, -0x1b,0x1e,0xfd,0xdc,0xa3,0xae,0xa1,0x08,0xe4,0x49,0x42,0x9b, -0xae,0x0d,0xac,0x14,0x68,0x0d,0x10,0x3f,0x41,0xb4,0x23,0x3f, -0xcc,0xcc,0x76,0xfb,0x97,0xe0,0x6b,0x30,0x88,0x63,0x23,0x35, -0x42,0x50,0x52,0xe3,0xb8,0x06,0xe9,0x62,0x79,0x61,0x4d,0xf4, -0xe2,0x74,0x42,0x41,0xa9,0x16,0xce,0x9e,0x36,0x96,0x70,0xeb, -0x3c,0x80,0xfd,0x78,0x20,0xd8,0x56,0xaf,0x90,0x21,0xf3,0x48, -0xad,0x6f,0xed,0x17,0x01,0xa0,0x5f,0x16,0x8d,0x40,0x51,0x47, -0x76,0xd8,0x14,0xf1,0xdf,0xf3,0x36,0x26,0xbd,0x16,0x11,0xbf, -0xcc,0x64,0xd0,0x0d,0x5d,0x97,0x0d,0x3d,0xc6,0x48,0x5a,0x0a, -0x77,0xc3,0xc6,0xac,0xbf,0x2e,0xee,0xbc,0x58,0x66,0xf9,0x57, -0x27,0x46,0x62,0x6e,0xd0,0x09,0x22,0xf7,0x0d,0x5b,0xf9,0x2c, -0x92,0x5b,0x03,0x0b,0xb2,0xf9,0xc4,0x67,0xb7,0xd4,0x60,0x30, -0xbf,0x4b,0x4e,0x93,0x1c,0x0e,0xaf,0xeb,0xdc,0xf7,0x40,0x37, -0x84,0x23,0x7a,0x7f,0x27,0x39,0x6a,0xc9,0x87,0xe4,0x9e,0x4d, -0x5e,0x97,0x6f,0xab,0xe5,0x4d,0x2d,0x88,0x0b,0xea,0xfe,0x73, -0x5f,0xa1,0xdd,0xce,0x79,0x23,0xe6,0x77,0x64,0x1a,0xe1,0x56, -0x33,0x19,0xe0,0x05,0x9e,0xc3,0xd6,0x53,0x19,0x4d,0x3d,0x0f, -0x5a,0x26,0x72,0xa9,0x2f,0x26,0x85,0x84,0x94,0xf8,0x72,0xb2, -0xdd,0xf2,0xaf,0xaf,0xe0,0x57,0x92,0x64,0x87,0xab,0x27,0xff, -0x2c,0x12,0x35,0x77,0x1d,0x3f,0x98,0xa8,0x57,0x20,0x36,0x88, -0x9e,0xea,0xfa,0xd1,0xa3,0x97,0xb7,0xd8,0x28,0x56,0x1d,0x4c, -0xbe,0x04,0x1e,0xc2,0x46,0x1a,0x1f,0x3d,0x92,0x80,0x65,0x9b, -0x64,0xb1,0xb9,0x72,0x9d,0x0f,0xac,0x29,0xad,0xf9,0x22,0x1b, -0x59,0x30,0xb6,0xdb,0x20,0x7a,0x82,0x9a,0xdd,0xea,0xc0,0xbf, -0x9c,0x14,0x3d,0x3b,0xf2,0x91,0x0e,0x5c,0x93,0xfe,0x0c,0xd5, -0x07,0x59,0xfe,0xe0,0x5e,0x4a,0x0f,0xd8,0xba,0xce,0xb0,0xdd, -0x64,0x39,0x9b,0xf9,0x29,0x1a,0x64,0x03,0xea,0xdb,0x36,0xff, -0xdb,0x44,0x50,0xf0,0x6f,0xad,0x15,0x72,0xfd,0x69,0x4d,0xbb, -0x5b,0xa1,0xdc,0xa9,0xad,0x4b,0x8c,0x28,0x62,0xaf,0xef,0x2e, -0x80,0x33,0x9b,0x9b,0xc5,0xaf,0x28,0x0c,0xcd,0xc4,0x88,0x1f, -0x80,0x10,0x8b,0x72,0x98,0x95,0xa1,0x8f,0x50,0xda,0xe2,0x08, -0xef,0x2c,0xcf,0xd8,0xbe,0x46,0xb8,0x63,0xe4,0x31,0x32,0x1e, -0x28,0xfa,0xbf,0x44,0xa5,0x36,0xf7,0x01,0xd2,0x12,0x9f,0xbe, -0xb9,0x05,0x4d,0xa6,0x37,0x17,0x08,0x1a,0x67,0x1f,0xad,0x79, -0x14,0x0e,0xdb,0xe5,0x6b,0xb2,0x7b,0xe0,0x3e,0x4b,0xec,0x10, -0x44,0x56,0x39,0xe6,0xb2,0xd9,0x71,0x94,0x6e,0x5d,0x4f,0xdb, -0xcc,0xa8,0x8d,0x54,0x32,0x16,0x7a,0x5b,0x3b,0x2a,0x11,0x93, -0xe3,0x17,0xe8,0x5b,0xa9,0x8f,0xc3,0xdd,0xa1,0xe5,0x52,0x70, -0x2b,0x4f,0x4a,0x44,0xa1,0x4c,0xd6,0x00,0x6b,0xea,0xa8,0x11, -0x09,0x1b,0xfd,0xe7,0x6d,0x25,0x5b,0xee,0xe0,0x42,0x8a,0x48, -0xfb,0xc0,0x1d,0x8b,0x36,0x27,0x1a,0xfc,0xf6,0xc8,0x44,0x1e, -0x78,0x18,0xf2,0x92,0x4d,0xbf,0x9a,0x9a,0x59,0xa6,0xa0,0x15, -0x48,0x2f,0xe5,0x7a,0x51,0x38,0x53,0xd3,0xb1,0x2d,0xc1,0xc1, -0x71,0x6c,0x90,0x12,0x2a,0x08,0x52,0x94,0x56,0x89,0xde,0xf4, -0x0f,0x35,0x6d,0x59,0xd2,0xf3,0xda,0x89,0x4c,0x8f,0xe8,0x49, -0x7f,0x34,0x7d,0xb6,0x9d,0x7a,0x0a,0xe3,0xf6,0xe1,0x4b,0xa7, -0x5e,0x78,0x79,0x5b,0xc3,0x5e,0xe1,0x34,0x24,0xa8,0x0f,0xdf, -0x7b,0xa4,0x09,0x35,0x83,0x2f,0x5c,0x60,0x80,0x4d,0x40,0x57, -0xc8,0x5e,0xe9,0x05,0x25,0x35,0xc0,0xe4,0x70,0x24,0x87,0x82, -0x46,0xc9,0x1b,0xc2,0x9e,0x78,0xeb,0x49,0xb5,0xe1,0x73,0x80, -0x5c,0x74,0x84,0xea,0x2c,0x8c,0xcb,0x2c,0x5d,0xe4,0xfa,0x63, -0x24,0x72,0x54,0x0f,0xa5,0x84,0x24,0x28,0x33,0x92,0x5e,0xc4, -0x93,0x34,0x8f,0x9a,0x18,0xd9,0x47,0xc5,0x24,0xfb,0x45,0x98, -0x4d,0xa9,0xdd,0xee,0x89,0x15,0x89,0xec,0x3a,0xe4,0xe1,0xe5, -0xc9,0xbd,0xd6,0x0a,0xad,0x85,0x4f,0xe1,0x50,0x9b,0x7c,0x76, -0xa8,0x3c,0x2f,0xb1,0x63,0x4a,0xd4,0x45,0xec,0x40,0xff,0x67, -0x58,0x15,0x1f,0x69,0xec,0xc9,0xc0,0x71,0xd7,0x68,0xeb,0x6d, -0x7b,0x08,0x19,0xf8,0x5c,0x8c,0xd7,0xed,0x89,0xf1,0xfb,0xed, -0x0d,0x6c,0xcf,0xbd,0xe4,0x38,0xc9,0xf1,0x1d,0x04,0x0c,0xcc, -0x11,0xef,0x44,0xe4,0xf0,0x71,0x7b,0x35,0xdd,0xd8,0xb0,0x57, -0xbc,0x4d,0x1b,0xb2,0xa5,0xe3,0x07,0x97,0xd1,0x93,0xb6,0x2c, -0x68,0x17,0x12,0xc3,0x0e,0x8f,0x99,0xe0,0x8b,0x2a,0xb7,0xd3, -0xf5,0xac,0x48,0xb0,0x46,0x54,0xe0,0xf8,0xcb,0xcc,0x2c,0x2f, -0x49,0xe0,0x4b,0xc4,0xda,0x10,0x07,0x84,0xfd,0xf9,0x91,0xba, -0xa9,0x47,0x9e,0x26,0x6c,0x64,0xd4,0xba,0x2c,0x3a,0x71,0x70, -0xcb,0x0f,0xc6,0x99,0x69,0x84,0x98,0x21,0x69,0x14,0x83,0xbe, -0xd1,0x2f,0x0a,0xdb,0x65,0xea,0xcd,0x74,0xbd,0x41,0x73,0xf4, -0x82,0xc7,0x09,0x03,0x90,0x11,0x8d,0x12,0x4f,0xec,0x4b,0x5a, -0xe4,0x19,0x06,0xa5,0x88,0xc0,0xa2,0x05,0x50,0xf0,0xa8,0x0f, -0xf2,0x2e,0x73,0x53,0xb3,0xbd,0x7a,0xd1,0x01,0x90,0xbb,0xdd, -0x0e,0xd7,0x41,0x61,0x98,0x5b,0x23,0xe1,0x96,0xa1,0x12,0x91, -0x1f,0xc0,0x1c,0xf2,0x4f,0x2b,0x2f,0xb2,0xe4,0x88,0xff,0xb8, -0x38,0x35,0xdc,0x10,0x5b,0x54,0xed,0xe7,0x9b,0x78,0xb2,0x52, -0x59,0x3a,0x3e,0x7d,0x77,0x06,0x43,0x42,0x0c,0xfa,0x54,0xe1, -0x0b,0x6e,0xf3,0xf7,0x49,0xa1,0x46,0x3e,0x1d,0xca,0xe9,0x76, -0x03,0xf4,0xec,0xec,0x8e,0x80,0x30,0x53,0xd5,0xb1,0xe2,0x91, -0x47,0xe3,0x89,0xd7,0xb9,0xa4,0x58,0xd3,0x9a,0x89,0x52,0x51, -0xb7,0xab,0x48,0xe1,0x61,0x7a,0xa1,0x74,0xb8,0x8e,0x1f,0x4c, -0x27,0x4a,0xa1,0x70,0x3a,0x8d,0xf0,0x36,0xe4,0xd2,0x42,0x01, -0x61,0xed,0x9c,0x17,0x61,0x68,0x1d,0xf4,0x43,0xd0,0x70,0x3b, -0x83,0x89,0xc4,0x1a,0xdc,0x5b,0xe2,0x01,0x37,0x54,0xd8,0xd6, -0xba,0x5d,0x19,0x8d,0xb3,0x0f,0x1c,0xdb,0xcc,0x09,0x4f,0x4f, -0xaf,0xe7,0x15,0x77,0xb7,0xcd,0x34,0x63,0x30,0x10,0xea,0xd2, -0x81,0x77,0x82,0x51,0xce,0x10,0x85,0xe8,0x86,0x90,0x66,0x36, -0xf4,0x50,0x30,0x51,0xdd,0x6e,0xaf,0x87,0xf5,0xb8,0x25,0xf5, -0x1c,0xb8,0x8b,0x47,0x4e,0x91,0x7d,0xc8,0x97,0xc4,0x91,0x3c, -0x56,0x9b,0xa8,0xb7,0x8b,0x9e,0x5d,0x14,0xef,0xf8,0xfe,0x38, -0xe1,0x30,0x8c,0x55,0xd4,0x16,0xbb,0x28,0xb3,0xfe,0x42,0x4d, -0x74,0x65,0x1e,0x1a,0x07,0xec,0x44,0xce,0x07,0x9d,0xf4,0x8f, -0x23,0x04,0x05,0xa3,0x8f,0xa0,0xf1,0x8e,0xd2,0x78,0x27,0xa6, -0xf1,0xed,0xb6,0xc3,0xc8,0x83,0xae,0xbc,0x71,0x13,0x68,0xfe, -0x88,0x83,0x6f,0x6f,0xc4,0x0e,0xa9,0x84,0xca,0x8c,0x70,0x2b, -0x0a,0x2d,0x38,0x58,0xdb,0xdd,0x0f,0x73,0x8f,0x89,0xa5,0x31, -0x55,0x09,0xa7,0xdb,0x6e,0x4f,0x08,0x51,0x97,0x97,0xff,0x55, -0x38,0x1f,0xfe,0xbe,0x4f,0xbf,0x4f,0x8b,0x13,0x7c,0xe6,0x32, -0x96,0x83,0x59,0x0c,0x75,0xa1,0x09,0xf9,0x6e,0x65,0xfd,0x0a, -0x03,0x94,0x4b,0xf4,0x3d,0xbe,0x26,0x1a,0xf3,0xdf,0x89,0xc4, -0x76,0x70,0xfe,0x45,0x63,0x95,0x64,0x7d,0x14,0xc5,0xa7,0x86, -0x04,0xc6,0xc4,0x9a,0x89,0x72,0x76,0xc4,0x15,0x50,0x0f,0xc2, -0xbf,0x9c,0x59,0x86,0x8b,0x6a,0x3d,0xba,0xf4,0x02,0x34,0xe6, -0x97,0x0b,0x17,0xc6,0x09,0x93,0xeb,0x42,0x7a,0x95,0x4b,0x2c, -0x03,0xfb,0xbb,0x96,0x9e,0x98,0x08,0xc0,0xfe,0xd5,0xb4,0xfe, -0xe9,0x76,0xf1,0xb3,0xbc,0x3e,0x94,0xb6,0xc1,0x9b,0x8d,0x3e, -0x25,0xd2,0x63,0x35,0x7b,0x67,0xc3,0x01,0x5e,0x4a,0xa1,0x46, -0x70,0x47,0x2b,0x6e,0x4b,0x0c,0xa9,0xd9,0xbb,0xa2,0x19,0xc3, -0xc4,0x79,0xd5,0x66,0xe4,0x3b,0x39,0x86,0x62,0x7e,0x84,0x47, -0xfa,0xf4,0x0a,0xed,0xb2,0x3d,0x76,0x57,0x97,0xb2,0x57,0x9d, -0x20,0x20,0x35,0x95,0x5b,0xe0,0x91,0x97,0x27,0x0f,0x5f,0xb8, -0xdd,0xe4,0xfb,0x86,0xa8,0x11,0x95,0x68,0x68,0x78,0x89,0x4c, -0x77,0xfb,0xd6,0xa4,0x58,0x7d,0xe3,0x8b,0xd8,0x71,0xad,0x2d, -0xb6,0x41,0x36,0x0c,0xe3,0x6f,0xb9,0x09,0xfd,0x30,0x82,0x0f, -0xc4,0xbf,0x72,0x3e,0xf7,0xea,0xd8,0x4c,0x15,0x49,0x41,0x8b, -0x0e,0xfe,0x18,0xae,0xa8,0xf2,0xfd,0xf8,0xfc,0xd8,0x32,0x71, -0xc8,0x2b,0x46,0x74,0x80,0x66,0x1b,0x1c,0x26,0x3b,0xb4,0xf7, -0xdb,0x20,0x21,0x80,0x23,0x64,0x4a,0x54,0x42,0x01,0xa2,0x54, -0x07,0x91,0x6a,0xf0,0x4c,0xc7,0x5a,0x60,0x23,0x8b,0xd9,0xd1, -0x63,0xfd,0xf5,0x07,0xe7,0xe2,0xa0,0xa9,0x58,0xec,0xc2,0x4b, -0x99,0x4e,0x41,0xbd,0x73,0x3a,0xe3,0x3d,0xfd,0x04,0xc6,0x24, -0xad,0x7d,0xa4,0xb1,0x4a,0x4b,0x74,0xab,0x64,0x70,0xf0,0x9c, -0x82,0xa4,0x07,0xaa,0x2a,0x27,0x65,0x52,0xdb,0x18,0x28,0x05, -0xc0,0x98,0x4f,0x17,0xaf,0xf6,0x80,0x80,0x59,0x2e,0x4e,0xbf, -0x73,0xd0,0xe1,0xc3,0x09,0xe6,0xd2,0x69,0xe2,0x10,0x1f,0xa1, -0xd6,0x81,0x8b,0x50,0x53,0xd4,0x57,0x81,0xff,0xfc,0x81,0x63, -0xa2,0xd2,0xdb,0xf7,0xd0,0xcc,0x70,0xb6,0x3c,0x92,0xb0,0xe6, -0xf8,0x59,0x84,0xd6,0x33,0x2d,0x87,0x20,0x98,0xf5,0x1c,0xfb, -0x4f,0x5a,0xf5,0xd0,0x09,0x67,0xe5,0x94,0x43,0x10,0x7b,0x67, -0x52,0xea,0x40,0x7f,0x44,0x20,0xda,0xfc,0xa2,0x60,0xac,0x9c, -0xc8,0x9e,0x4c,0x65,0x15,0xeb,0x98,0x85,0xf2,0x78,0x36,0x84, -0x33,0x22,0xd0,0x86,0xa7,0x7d,0xf4,0xa2,0xb6,0xb3,0x6b,0x75, -0xa0,0x22,0xcd,0xdc,0x1d,0x32,0x4c,0x18,0x3f,0xc9,0x14,0xf3, -0x18,0x7e,0xad,0xa5,0xbe,0x52,0xbf,0xcb,0xf9,0xf2,0x42,0x1e, -0xf9,0x8c,0x12,0x90,0xd5,0x5a,0x7b,0x5b,0xfc,0xdd,0xed,0xe2, -0xbf,0x22,0xb4,0xd3,0xc9,0x81,0xa9,0x68,0xb6,0x33,0xeb,0xe5, -0xb2,0xd1,0x51,0x18,0xe3,0x47,0x2e,0xb2,0xf5,0x4d,0xec,0x0f, -0x95,0x6d,0x7d,0x89,0xda,0x05,0xcf,0x47,0xd8,0xbe,0xc5,0x86, -0xdc,0x45,0x96,0x4d,0xa2,0xad,0xbf,0xd3,0x11,0xa2,0x27,0x25, -0x2b,0xfd,0x44,0x93,0xd7,0xed,0xf6,0x9f,0x9c,0x3c,0x3d,0x27, -0xa1,0x01,0x8e,0xb7,0x62,0xb8,0x72,0xc8,0x04,0x56,0x0d,0x5a, -0xe8,0xc9,0x14,0xf4,0x4a,0xad,0x3f,0xa9,0x0a,0x3e,0x43,0xce, -0xf6,0x3f,0x6d,0x73,0x61,0x35,0x23,0x07,0x0e,0x01,0xcd,0xf8, -0xce,0x56,0xed,0x21,0x07,0x1e,0xb5,0x6a,0xda,0x26,0x6c,0x41, -0xd5,0x5a,0x48,0x2d,0x9a,0x76,0x46,0x7f,0x1d,0x1a,0x82,0x7f, -0x6b,0x1f,0x25,0xd8,0x96,0x58,0x54,0x0c,0x1a,0x97,0x11,0x97, -0xd7,0xab,0xcd,0xfb,0xb8,0xf9,0xdf,0xa3,0x49,0xd7,0xd7,0x08, -0x74,0x34,0xa7,0xc9,0x7f,0x4f,0xb6,0xdb,0x2f,0x9a,0x41,0x36, -0xbf,0x6c,0xac,0x86,0x66,0x20,0x47,0x39,0x8d,0xb5,0x0f,0xbb, -0xe3,0x8b,0x48,0x7d,0x1a,0x80,0xa8,0xee,0x0d,0x7b,0x51,0xed, -0xc1,0x96,0xf8,0x58,0xf5,0x5b,0x1e,0xfa,0xdb,0x19,0x9a,0xb2, -0x7d,0x15,0xd9,0x95,0xab,0xbd,0x22,0x3b,0x71,0xb5,0x52,0xcc, -0xef,0x8b,0x06,0xee,0xc5,0xe1,0xf6,0x96,0x46,0x14,0x9d,0x1b, -0x1f,0x4f,0x7b,0xd7,0xca,0x33,0x50,0x77,0xd3,0x68,0xf9,0x00, -0x14,0xd0,0x21,0x36,0x15,0x74,0x87,0xd1,0x27,0x49,0xd7,0x93, -0xd2,0x45,0x67,0x0b,0x63,0x65,0xfb,0x72,0x60,0xba,0x58,0xa0, -0x71,0x1a,0xb5,0x0d,0xe1,0xe3,0x5a,0x74,0x4f,0xb0,0x2f,0x5e, -0x78,0xac,0x84,0xc4,0xc0,0x9d,0x19,0xa9,0x2b,0x3e,0x58,0x85, -0x64,0x00,0x89,0xd6,0x6d,0x34,0x66,0x85,0xb4,0x22,0xba,0xd8, -0x01,0x36,0x56,0xfe,0xfa,0x7b,0x9a,0x32,0x71,0xa4,0xf2,0xf1, -0xc9,0xa9,0x26,0x8d,0xf4,0x47,0x4f,0x02,0x1e,0x5b,0x31,0x78, -0xc7,0xaf,0xee,0xfe,0x3e,0xd8,0x3d,0x07,0xf3,0xa1,0xc8,0x15, -0xe8,0x53,0x5e,0x3c,0xc8,0xfc,0xe2,0x7a,0xe5,0x17,0xc8,0x7c, -0x92,0x07,0xdd,0x2f,0x67,0xb3,0x3f,0xda,0xfb,0xe0,0xf7,0xf7, -0x3e,0xff,0x5d,0xd3,0xe6,0xe1,0xda,0xc3,0xc1,0x87,0xd1,0x3d, -0x3c,0x3e,0xae,0xe0,0x88,0x35,0xfc,0x14,0x08,0x5f,0xfd,0xaf, -0x85,0xb0,0xd7,0xab,0xbe,0x16,0xe1,0xf8,0x13,0xa0,0xdc,0xed, -0xe8,0x32,0xb3,0x02,0xa9,0xf7,0x8e,0xde,0xc6,0xcc,0x3b,0xa2, -0x10,0x3c,0x5f,0xbe,0xc3,0xdf,0x97,0x20,0x6e,0xe0,0x5f,0xf4, -0x89,0xb9,0x5d,0xae,0x67,0xf8,0xbb,0xba,0x9e,0xbe,0xc2,0xc4, -0x5d,0x16,0x1c,0x21,0x41,0xcc,0x6e,0x3e,0x64,0x50,0x65,0xae, -0x03,0x7e,0x38,0x17,0x5b,0x00,0xa1,0xb9,0xdc,0x1c,0x68,0x21, -0x78,0x86,0x00,0x9a,0x90,0xb3,0x49,0x61,0x6d,0x33,0xf8,0xb3, -0x21,0x59,0x73,0x7c,0x6f,0x0d,0x51,0xe3,0x05,0x09,0xc2,0x83, -0x4e,0x39,0x0f,0x62,0x4b,0x61,0x92,0x44,0xbd,0x0f,0x42,0x42, -0x34,0x03,0xdf,0x77,0x24,0xf2,0xbd,0x7b,0x8d,0x04,0x0e,0xf9, -0xf4,0xb7,0x25,0xd0,0x4d,0xa6,0xef,0x83,0x04,0x47,0x15,0xe9, -0x48,0x4e,0x13,0x5e,0x84,0x37,0x2a,0x4a,0x07,0x44,0x2e,0xe2, -0x4e,0x54,0x7e,0x88,0x23,0x9a,0x43,0x81,0x22,0x8f,0xc6,0x11, -0x64,0x0e,0xdb,0x46,0x94,0xc6,0x11,0x78,0x3f,0x2d,0xb8,0x9b, -0xbd,0xc8,0xe7,0xb6,0xc5,0xd4,0x5d,0x77,0xc5,0xc5,0xe6,0xaa, -0x08,0xb7,0xc9,0x5f,0x7d,0x77,0x0e,0x0c,0x7d,0x66,0x8f,0xea, -0xee,0xa9,0x7e,0xbf,0x46,0x7c,0xa2,0xa7,0x5b,0x13,0xf7,0xe9, -0x3b,0x87,0x82,0xc0,0xec,0xdb,0xc5,0xc8,0x53,0x63,0xf8,0xdb, -0x3e,0xc6,0x89,0x91,0x69,0xb8,0x8c,0x7d,0x7a,0xb4,0xed,0xce, -0xdb,0xcf,0x1c,0x27,0x79,0x32,0x29,0xbc,0x04,0x0b,0x99,0xa4, -0xb1,0x0d,0x47,0xd1,0xb0,0xe8,0xd0,0xfc,0x4d,0xd3,0xaa,0x41, -0xb3,0xc8,0x07,0x14,0x83,0x9a,0xf8,0x0e,0xb8,0x26,0x7a,0x9d, -0xbd,0x88,0x6c,0x8e,0x76,0xf2,0x12,0x1a,0xbb,0xd3,0xc1,0x41, -0x67,0x53,0xcd,0x8b,0xfb,0xff,0x89,0x7f,0xd0,0xdf,0x9b,0x45, -0x96,0x1a,0x15,0xcb,0xec,0xf4,0x2d,0x09,0x18,0x23,0xf6,0x2d, -0x7c,0x52,0xc1,0x2d,0x99,0xe0,0xdf,0x37,0xf4,0x20,0x2e,0x2a, -0xe2,0xef,0xbf,0xe8,0x8f,0x5f,0xe4,0x7f,0x3a,0x1b,0x9f,0xf5, -0xcd,0xe4,0x1e,0xb6,0xe3,0x3b,0xd1,0x07,0xc3,0xe7,0x9b,0x3f, -0x3f,0xdb,0xbc,0xba,0x99,0xae,0xa7,0xf0,0xb3,0x9c,0x49,0xd0, -0xb9,0x3b,0xd2,0x5b,0x42,0xbf,0xc4,0x31,0xf8,0x14,0x59,0xe3, -0x6f,0x14,0xd5,0x88,0x63,0x00,0x30,0xb8,0xc4,0x87,0xcd,0x67, -0x52,0x2f,0x83,0xa7,0xfc,0x22,0x15,0x05,0xda,0x7d,0xa0,0x38, -0x7a,0xd9,0x88,0x7d,0xe2,0xd9,0x6c,0xec,0x7d,0x92,0x0b,0x2b, -0xf2,0xdb,0xfa,0xfc,0x3c,0x31,0xac,0x8d,0x67,0x9b,0xe9,0xc5, -0x1b,0x79,0x39,0xd9,0x75,0x26,0xf4,0xe9,0x0b,0x06,0x1c,0x4d, -0xe9,0x24,0x8c,0xb1,0x12,0x3f,0xa4,0x8f,0x1d,0xa0,0x19,0x23, -0xbd,0x97,0xe7,0x42,0x97,0x8b,0x61,0x0c,0xda,0xac,0xb0,0xd3, -0x8c,0xd7,0x8a,0x47,0x9a,0xde,0x3b,0x81,0x1c,0xbb,0x05,0x47, -0xeb,0x9b,0x68,0x14,0x11,0xd8,0xd8,0xca,0x60,0x14,0xd0,0x21, -0x76,0x92,0xe5,0xf8,0x0f,0x56,0xb6,0x24,0x5d,0xf0,0x53,0xc1, -0x36,0x72,0x73,0xf0,0x85,0xbc,0x2c,0x87,0x03,0x77,0xcf,0xf6, -0x4f,0xe6,0x20,0xa8,0xab,0xf0,0xbc,0x65,0xe8,0xf1,0x4c,0x8d, -0xef,0x45,0x5f,0x75,0xa1,0xef,0x6c,0xf2,0x6b,0xbe,0x34,0x6a, -0x9e,0x18,0xce,0x8f,0xbc,0x30,0xa9,0xd3,0xdf,0x89,0x59,0x9e, -0x35,0x6e,0x16,0x39,0xa2,0x87,0xdc,0x9d,0x59,0xf8,0xc7,0xc9, -0x86,0x22,0x25,0x42,0x1a,0xac,0x9e,0xc5,0xf2,0x96,0x1b,0xb4, -0x03,0xee,0x0c,0xb2,0xcc,0x5e,0x60,0xff,0xc1,0xa6,0xe8,0x7d, -0xf9,0xaa,0xde,0xdf,0x4c,0xa7,0xe3,0x3d,0xd3,0xbb,0xff,0xad, -0xdb,0xf5,0xa1,0x80,0x5d,0x11,0xc5,0x32,0x41,0xdb,0xd0,0xb0, -0xa4,0x1a,0x48,0xe5,0x01,0xc6,0x0c,0x84,0x93,0xdc,0x92,0x19, -0x61,0x3f,0xa8,0xe6,0xd4,0x47,0xb9,0x3f,0x43,0x6e,0x31,0x68, -0x36,0x0c,0xeb,0x62,0xbe,0xac,0x01,0x90,0xe6,0xd8,0x6a,0xe7, -0xa8,0x64,0x23,0x10,0x8b,0x51,0xf4,0xdc,0x5f,0xa7,0x46,0x16, -0xc1,0x6a,0x59,0x17,0x07,0x06,0x88,0x8e,0x01,0x7b,0x97,0x73, -0x1d,0x8f,0xde,0x76,0xbe,0xdd,0x06,0x78,0xc8,0x81,0xa8,0x84, -0xa4,0xd4,0x74,0x59,0x56,0xd6,0x90,0x1e,0x7e,0x42,0x0b,0xe6, -0xe8,0xf5,0x39,0x48,0x72,0x2f,0x8f,0x89,0x4d,0x29,0xbd,0xde, -0xa4,0x66,0x65,0x1a,0x61,0x1b,0x83,0x3a,0x0b,0x9e,0xd1,0xda, -0x19,0xaf,0x56,0xbd,0xa5,0x1c,0x5b,0xd0,0x62,0x19,0xe3,0x06, -0x47,0x94,0x20,0x7b,0x33,0x9a,0x4a,0xf3,0xbd,0x49,0xd3,0xa8, -0x7a,0xd7,0x4e,0x77,0x58,0xd9,0x2a,0xc5,0xf7,0x2c,0xff,0x1d, -0x47,0x93,0x3a,0x70,0x64,0x1f,0xb5,0xda,0xb8,0xfd,0xbf,0xd4, -0xbd,0x79,0x7f,0x1b,0x47,0x92,0x28,0xf8,0xff,0xfb,0x14,0x60, -0x0d,0x07,0x5d,0x65,0x14,0x41,0x52,0x76,0xf7,0x74,0x17,0x5d, -0xc2,0xa3,0x29,0xa9,0xad,0x19,0x5d,0x23,0xd2,0xed,0x9e,0x05, -0x61,0x3d,0x90,0x00,0x49,0xc8,0x20,0x00,0xa3,0x40,0x1d,0x4d, -0xe0,0xbb,0x6f,0x9c,0x99,0x91,0x59,0x05,0x4a,0xee,0x37,0xfb, -0x76,0xf7,0x67,0x8b,0xc8,0xca,0x23,0xf2,0xce,0x8c,0x88,0x8c, -0x43,0xe1,0x4d,0x66,0xc7,0xcb,0xe5,0xf0,0xb3,0x2e,0x22,0x75, -0xa9,0xcb,0x1c,0xbf,0x22,0xca,0x44,0x94,0xd6,0x7b,0x52,0xdd, -0x21,0xf6,0x18,0x14,0x20,0x4c,0x84,0xd7,0x57,0x21,0x30,0xda, -0x6d,0x09,0x58,0x45,0x5b,0x59,0x72,0x40,0x9d,0xa5,0x19,0x39, -0xfe,0x45,0x9d,0x1f,0x5d,0x73,0xe8,0x08,0x2b,0x1f,0x8e,0x1a, -0xee,0x83,0x50,0x27,0xb1,0x42,0x69,0xc0,0xad,0x1b,0x29,0xde, -0x2b,0x6e,0x79,0x48,0x2f,0x6e,0x87,0xbf,0x8e,0xb9,0x1f,0x7e, -0x5f,0x3a,0x46,0xb7,0x37,0x94,0xaf,0x51,0x83,0xc2,0x3b,0x99, -0x1b,0x02,0x79,0xaa,0x50,0xc6,0x70,0x38,0xf1,0x31,0x0b,0x87, -0x14,0xda,0x0b,0x33,0x87,0x77,0xe3,0x7d,0xa3,0x73,0x36,0xc4, -0xcb,0x98,0xfa,0x89,0x76,0x24,0xbf,0x74,0xfc,0xa0,0x4b,0x67, -0x26,0x8b,0xc3,0xd3,0x1c,0xc7,0xee,0x35,0xf9,0xce,0x2b,0xa2, -0xef,0xda,0x8e,0xce,0x42,0x07,0xe5,0x80,0x52,0x9e,0xe2,0xcd, -0x68,0x62,0xb8,0x29,0x79,0xe0,0x2e,0xbe,0x99,0x19,0xc2,0x42, -0x30,0x4d,0x8f,0xd6,0x5e,0xf2,0x65,0xe9,0xe5,0xb3,0xe5,0x5d, -0xd3,0x8d,0xaa,0x00,0x65,0x81,0x5b,0x41,0x55,0xb6,0xac,0x59, -0x69,0xcc,0x68,0x22,0x42,0x21,0x56,0x46,0x30,0x73,0x85,0x09, -0xbf,0x89,0x38,0xf9,0x93,0x9c,0xb0,0xa4,0xaf,0x82,0x24,0x59, -0x37,0x8c,0xaa,0x34,0xb7,0x44,0x14,0x17,0xa4,0xb0,0x7d,0xb2, -0xce,0x36,0x8c,0xda,0x7c,0x4d,0xb9,0xf8,0xed,0x5f,0xea,0x84, -0xf5,0xff,0xb5,0x03,0xd0,0x50,0xf3,0xef,0x28,0xbd,0xa5,0xfe, -0x7f,0x6a,0x00,0x6d,0x4b,0x24,0xef,0xff,0xe0,0x06,0xfd,0x73, -0xf3,0x11,0x8b,0x45,0xe8,0xa4,0xc8,0x08,0x7e,0x61,0x89,0xe8, -0x38,0xa7,0xd1,0x8a,0x5c,0xaf,0xef,0x37,0x99,0x61,0x24,0xaa, -0x38,0xbd,0xc3,0x56,0xbf,0x0a,0x6a,0xc4,0x8d,0xc4,0xf2,0x8a, -0xe1,0x3e,0x58,0x5e,0x19,0x5f,0xd2,0xc7,0xc9,0xd5,0x12,0x3e, -0x92,0x8c,0xdf,0x13,0x04,0x44,0x28,0x9f,0xaf,0xb1,0x3f,0x87, -0x36,0x0f,0x8b,0xe0,0xc4,0x11,0xe2,0xd5,0x88,0x0a,0x90,0x2c, -0x5e,0x20,0x81,0x85,0x0e,0xf1,0xdc,0xee,0x16,0x51,0x78,0x97, -0x83,0xc6,0x36,0x0f,0x11,0x6d,0xbc,0xa5,0xdd,0xe9,0xb8,0x60, -0x0c,0xe7,0x6a,0x26,0xd3,0xe0,0x30,0x52,0xfa,0xe4,0x8b,0x9b, -0x24,0xb4,0xc8,0x1e,0xb0,0xf3,0x1e,0x8e,0x59,0x73,0x7f,0xa2, -0x6e,0x3d,0x9f,0x01,0x09,0x32,0xd5,0xc5,0x18,0x0a,0xde,0x65, -0x84,0xcb,0x5a,0x2c,0x02,0xed,0xf9,0xee,0xc4,0x54,0x07,0xf7, -0x6b,0xcb,0x7d,0x98,0x47,0xa5,0x2d,0x9d,0x64,0x7a,0x20,0x6d, -0xc1,0x0b,0x16,0x7d,0xd0,0x2f,0x59,0xbd,0xa1,0x7e,0xfd,0xda, -0x13,0xd4,0x13,0x2e,0x21,0xf6,0xe8,0x54,0x83,0x50,0x4e,0xc2, -0xbf,0xb9,0x41,0x98,0x64,0x39,0x81,0xee,0x4c,0x0a,0xf8,0x48, -0xc9,0xae,0xca,0x92,0xed,0xb6,0xe9,0x1b,0x8d,0xf3,0x05,0xfe, -0x10,0x4e,0x41,0x19,0xf1,0x6e,0x26,0x8d,0x99,0x5e,0x5f,0xbf, -0xd1,0x72,0x45,0x13,0x32,0x62,0x5a,0x94,0x91,0xb5,0xcc,0x68, -0x5b,0xa2,0x1f,0x30,0xd9,0x98,0x8a,0xc3,0xe9,0xd3,0x2b,0xb2, -0x43,0xee,0x96,0xc6,0x75,0x15,0xe1,0x4f,0x7f,0x89,0x90,0x23, -0xe8,0x17,0x93,0xa3,0x65,0x69,0x74,0xcf,0x63,0x14,0x6a,0xbd, -0xde,0x91,0x1b,0x19,0x71,0x1f,0x20,0x7a,0xb9,0x10,0x0a,0x39, -0x60,0xf7,0x43,0x80,0xfa,0x9e,0xee,0x70,0xa5,0x5c,0xb0,0x24, -0x6e,0x46,0xf0,0xe2,0x3c,0x72,0xe7,0x83,0x11,0x40,0xcc,0xfd, -0x5b,0x02,0x2d,0x6c,0x34,0x94,0x35,0x3b,0x9a,0x95,0xb3,0xe0, -0xd9,0x80,0x44,0x61,0x4c,0xb5,0x33,0x91,0xbc,0x80,0x85,0xc2, -0x55,0xcf,0xfc,0xaa,0x87,0xa9,0x3f,0xb2,0x4f,0x31,0x55,0x99, -0x0c,0x2f,0x2e,0x96,0xeb,0xe1,0x72,0x35,0xb9,0x9c,0x8e,0xd7, -0xc3,0x6a,0x02,0x47,0xcd,0xf0,0x6e,0x34,0x99,0xaf,0x2f,0x46, -0x93,0xf5,0xe5,0x70,0xf6,0x61,0x58,0x91,0x3f,0x2d,0xfa,0x33, -0x9d,0x54,0xab,0xf5,0x68,0x0c,0x74,0xcd,0xb4,0x5a,0x5f,0x4d, -0xae,0x2f,0x87,0xf4,0x06,0x83,0xc1,0xbb,0xe5,0x78,0x7d,0x35, -0x9f,0xa3,0xb3,0x1d,0x7e,0x25,0x58,0xdf,0x30,0x4b,0x09,0x1f, -0x52,0xd7,0xb7,0x63,0x4c,0x98,0x0d,0x3f,0xac,0xe7,0x77,0x2b, -0xb4,0xe4,0xb6,0x58,0xce,0xaf,0x51,0xb5,0x64,0x5d,0x8d,0xa9, -0xb7,0xeb,0xea,0xee,0x16,0x72,0x7e,0x5e,0xaf,0x26,0xb7,0xe3, -0xf5,0x07,0x68,0xc6,0x3c,0x41,0x0b,0x70,0xd0,0xc5,0x31,0x0f, -0x79,0xb9,0x2f,0xa7,0xd1,0xf9,0xa8,0x43,0xc6,0x3c,0xf1,0xe2, -0x5d,0xa3,0x49,0xbd,0x04,0x8d,0xb3,0xcd,0xe6,0xd5,0xcd,0xe4, -0x96,0x65,0x75,0xac,0x78,0xea,0xf7,0x64,0x48,0xd3,0x75,0x18, -0x2d,0x15,0x9e,0x9f,0x57,0xfb,0x8f,0x07,0x62,0x71,0x6f,0x29, -0x7e,0x2d,0x7e,0xf6,0x16,0x45,0xf7,0x7f,0x39,0xaf,0x3a,0xfb, -0xf9,0xf2,0xd3,0xcd,0xea,0x76,0x7a,0x36,0xbc,0x2e,0xf7,0x01, -0xc8,0x0e,0x1b,0x9c,0x5b,0xae,0x2f,0xe7,0xd3,0xf5,0xf8,0xf6, -0x02,0x96,0xc7,0xcd,0x72,0x3d,0xb9,0xbd,0x5e,0xb3,0x69,0x3a, -0x68,0x28,0x75,0x73,0xb8,0x86,0xed,0x39,0xbc,0xcd,0x52,0x34, -0x65,0x57,0x0c,0x3a,0x59,0xff,0x97,0xc7,0x83,0x6f,0xb2,0xf3, -0xfd,0xc7,0xfb,0xd7,0x93,0x7c,0x29,0x86,0x32,0x10,0xa4,0xa4, -0x43,0x45,0xab,0x8b,0xf9,0x08,0xfa,0xf7,0x3d,0xfd,0x92,0x2d, -0x3b,0xa8,0x18,0xbe,0xd7,0xed,0x7f,0xe9,0x9d,0x7f,0xec,0x1c, -0x21,0x4f,0x63,0xfe,0x1c,0xdf,0xb0,0x25,0x01,0xfa,0x54,0x5d, -0x2e,0x27,0x8b,0xd5,0xba,0x5a,0x7d,0x86,0x79,0xc3,0xca,0x33, -0x28,0x78,0x3b,0x9c,0x4d,0x16,0x77,0x53,0x7a,0xd5,0x7c,0xb7, -0x24,0x7e,0x27,0x3e,0xce,0x91,0xc4,0x21,0x71,0x54,0x94,0x05, -0xba,0x26,0xa6,0x28,0x1b,0xce,0x93,0x17,0xb5,0x72,0x5f,0x02, -0xe7,0x15,0xea,0xca,0xf7,0x7f,0x29,0x07,0xeb,0x12,0xc2,0xfa, -0xe2,0xd6,0xc5,0x0a,0x96,0x5c,0xad,0x00,0xdc,0x5d,0x9f,0xef, -0x43,0xd6,0xf7,0xc3,0x0f,0xc3,0xf5,0xf8,0xf2,0x76,0x98,0x71, -0x6a,0x98,0xef,0xe5,0xb0,0x22,0xe8,0xbf,0xac,0x96,0x77,0x63, -0xc8,0xdf,0xfd,0x06,0xbb,0x7c,0x89,0x96,0xb2,0x4e,0x29,0x13, -0x8d,0xf7,0x37,0xdf,0xef,0xa0,0x82,0x7e,0xff,0xe4,0xc9,0xf1, -0xd9,0xf1,0x79,0x7f,0xbd,0xb7,0x97,0xad,0x31,0x62,0x70,0x3e, -0xc0,0xf0,0x63,0xc8,0xb1,0x0b,0xd3,0xfc,0x71,0x39,0x5c,0xa0, -0x53,0xa9,0x7b,0x7e,0xfc,0x2b,0xfa,0x87,0xb9,0xda,0x59,0x70, -0x86,0xc7,0xcb,0x3f,0x68,0xe8,0x0f,0x8f,0x61,0x92,0xbd,0x85, -0x85,0x01,0x10,0xf8,0xd7,0x70,0xde,0x71,0xa9,0xab,0xc9,0x78, -0x3a,0x02,0x44,0x97,0xf3,0xf8,0xaf,0x41,0x8e,0x33,0xcd,0x79, -0xe0,0x02,0xe1,0x64,0x0a,0x0c,0x72,0x9a,0x5a,0x4e,0x62,0x5f, -0xcf,0x9c,0xaa,0x61,0xe4,0x9e,0xc0,0x6a,0xe2,0x0c,0x2b,0x1c, -0x79,0x4e,0x97,0x20,0x24,0x2f,0x8b,0xfe,0x23,0x97,0xc6,0xf3, -0x2d,0x59,0x28,0x68,0xb2,0xc2,0x32,0x6b,0xc8,0xeb,0x32,0x42, -0x32,0x6d,0x2f,0x2e,0xed,0xbe,0x6c,0x5d,0xd0,0x8e,0x6f,0x6b, -0xe5,0x57,0x4b,0xa9,0x6f,0xf9,0xb8,0xa1,0xd2,0x77,0x70,0xf2, -0x0d,0x61,0xf0,0xf4,0x08,0x56,0xbd,0x2c,0x5c,0x76,0xa7,0xe3, -0xe5,0x04,0x0e,0x80,0x7f,0x00,0x39,0x71,0x80,0x92,0x4b,0x49, -0x32,0xa0,0x8e,0xfe,0xfd,0x7b,0x36,0xa7,0x95,0x88,0x5d,0xad, -0x01,0x1c,0x65,0xc3,0xab,0xf1,0xb3,0xe5,0xf0,0x9a,0x6c,0xcb, -0x31,0xe7,0xfb,0xd4,0x44,0x39,0xbb,0x73,0x59,0x7e,0x25,0x51, -0x4f,0x26,0x1f,0x4a,0x5b,0x2a,0x36,0x0f,0xc7,0x0f,0xdf,0x0c, -0xea,0x29,0xf3,0x74,0x53,0x36,0x23,0x90,0x1d,0xc9,0x9a,0xe8, -0xc2,0x92,0x60,0xe6,0xb5,0x89,0x40,0xdd,0x1e,0xfd,0xe4,0x5d, -0xe6,0xbe,0xf0,0xb0,0x72,0x5f,0x3a,0x80,0x3e,0x82,0x4f,0x36, -0x9f,0x1d,0x27,0xd6,0x83,0xba,0xf1,0x09,0xa3,0xbc,0xce,0x08, -0x0c,0x9f,0x16,0xd9,0xac,0x46,0x84,0x52,0x0d,0x2f,0x51,0x1d, -0x41,0xd0,0x93,0xe6,0x9c,0x6a,0x7c,0xc7,0x1b,0x6d,0x31,0x7c, -0xd9,0xd4,0xd3,0xa7,0xf2,0x82,0x9b,0xc9,0xa0,0xa5,0x69,0x8d, -0x68,0x8d,0x14,0x28,0xdd,0xf8,0xcb,0x88,0x22,0x63,0x97,0x14, -0x12,0xb9,0x7e,0xc4,0xa6,0x51,0x20,0x88,0xbe,0xdc,0xc3,0x4d, -0xe5,0xac,0xb3,0xd3,0x30,0x04,0xd8,0x3a,0xae,0x0f,0xe2,0x2e, -0x38,0xee,0xf0,0xb3,0x20,0x29,0xb3,0x54,0x20,0x91,0x64,0xd6, -0x28,0xa4,0xb2,0xe2,0x88,0xa1,0x23,0xb0,0xa9,0x1c,0x8b,0x9c, -0xd1,0x10,0x4d,0x50,0xca,0xe9,0x48,0x6d,0x43,0x1f,0x0c,0x18, -0xd7,0xc3,0xcc,0xca,0xc9,0xc3,0x02,0x79,0x63,0x87,0xb3,0xee, -0xf8,0xb7,0xf4,0x00,0x3a,0x3b,0x9d,0xcf,0xc6,0xe9,0xce,0x41, -0x76,0x54,0x27,0xe6,0xdb,0x6d,0x84,0x15,0xda,0x82,0xd3,0xaa, -0xa8,0xbf,0x84,0x4c,0x46,0xac,0x3f,0x15,0xbc,0x21,0xfc,0xee, -0x28,0x42,0xae,0x8d,0xd4,0x8a,0xb1,0x6b,0xe6,0x99,0x31,0x8d, -0xe2,0x01,0x86,0xc7,0xb1,0x71,0xf3,0x49,0x03,0x63,0xf9,0x29, -0x3c,0x03,0x74,0x21,0xc4,0x73,0x10,0x2e,0xb2,0x78,0x1e,0x7a, -0x5f,0x39,0x01,0x04,0xba,0x71,0x0a,0x8a,0x06,0x08,0xca,0xc8, -0x70,0x74,0x38,0x43,0x72,0xb4,0x05,0x52,0xfb,0x57,0x4a,0x10, -0x54,0x29,0x3e,0x7b,0x71,0x50,0x31,0x45,0xf7,0x6d,0x27,0x3f, -0x2b,0xa8,0x98,0x8c,0x01,0xc5,0x6c,0x64,0xed,0xc5,0x9d,0x26, -0xce,0xae,0xeb,0x6a,0xb9,0xa5,0xf3,0x47,0xbf,0x7f,0x11,0x7a, -0x10,0xbd,0xda,0x60,0x14,0xae,0x49,0x77,0xb3,0xb0,0x51,0x11, -0xcf,0x95,0xd6,0x18,0xec,0xce,0x78,0xd0,0x62,0xda,0x8a,0xe0, -0x26,0x78,0x48,0xa1,0x2c,0x56,0xd0,0x1e,0x91,0x0d,0xfb,0x79, -0xb2,0xba,0x61,0x36,0x8d,0xa5,0x98,0x00,0x32,0x0c,0x10,0x72, -0x61,0x68,0xa8,0xb6,0xb5,0x63,0x34,0xbf,0x7d,0x89,0xe8,0x81, -0x13,0x50,0xad,0x72,0x7c,0x07,0x0d,0xc9,0x3e,0x12,0xac,0xa6, -0xec,0x91,0x86,0x69,0x14,0xf7,0x97,0x38,0x2a,0xe3,0xc3,0x26, -0x38,0xb7,0x99,0x44,0x65,0xea,0xfe,0xff,0xfd,0x96,0x05,0x5b, -0xdb,0x71,0xf8,0x02,0x6a,0x18,0x85,0x51,0x28,0xfd,0xf7,0xb4, -0xf4,0x30,0x6e,0xa9,0x99,0x75,0x3e,0x59,0xa2,0x88,0x2d,0x2d, -0xa1,0xea,0x87,0x57,0x01,0x1d,0xf6,0x7f,0xae,0xf6,0x80,0xa4, -0xc0,0x96,0xb0,0xf5,0xc8,0x06,0xee,0xe5,0xaf,0xe3,0xf1,0xe2, -0x09,0xd0,0x02,0x91,0xac,0x2b,0x3e,0x83,0x58,0x27,0x49,0xc2, -0xad,0x66,0x1f,0x49,0xe9,0x8e,0x77,0xe5,0xb9,0x85,0x58,0x8e, -0x0d,0x16,0x20,0x3d,0xab,0x55,0x31,0x21,0x36,0x0e,0xa5,0xa0, -0xf4,0x99,0x05,0x11,0x49,0xcc,0x94,0xa2,0x7c,0x8d,0x18,0x62, -0xc8,0x58,0x00,0x31,0x18,0x07,0x07,0xad,0xdd,0x8e,0x5f,0x68, -0x28,0x6f,0x70,0x65,0x30,0xe9,0x45,0x86,0x05,0xfe,0x3a,0x9d, -0x5f,0x0c,0xa7,0x4f,0xd1,0x16,0x80,0xa9,0x21,0x4f,0x18,0xc3, -0x4d,0xea,0x55,0x05,0x86,0x37,0xb9,0x35,0xc1,0xd9,0x53,0x13, -0x19,0xfb,0x3d,0x23,0x49,0x59,0x1b,0xec,0xcc,0x3d,0x34,0x1a, -0xf4,0x44,0x14,0x5f,0x4f,0x7c,0x01,0xd5,0xda,0x1a,0x18,0x3d, -0xe5,0x71,0x21,0x44,0xa8,0x72,0x55,0x44,0xbc,0x20,0x9e,0xc0, -0x24,0x73,0x46,0x95,0x39,0xbb,0x5a,0x3c,0x3d,0x88,0x2e,0x2e, -0xba,0x83,0x7d,0xd7,0x91,0xa8,0x3c,0x9e,0x8d,0x9e,0x7e,0xa0, -0xd5,0x3c,0x92,0x19,0x72,0x31,0xde,0x0e,0xa9,0x8d,0x15,0x7f, -0x60,0x41,0x5c,0x6f,0xe7,0xb0,0xf8,0x02,0x30,0x2d,0x16,0xc7, -0xf7,0x82,0x72,0x45,0x2d,0x9d,0xf7,0x47,0x74,0xfd,0x87,0xf7, -0x2c,0x63,0x16,0x74,0x80,0x7f,0xa9,0x47,0xb8,0xb7,0xf0,0xe6, -0xf8,0xdf,0x40,0x0f,0x03,0xb4,0x03,0x10,0x14,0xe4,0x0d,0xd6, -0x9f,0xab,0x10,0x59,0xaa,0x61,0x90,0x99,0x51,0x33,0x0b,0x56, -0x10,0xb3,0xf2,0x9c,0x61,0x36,0x2f,0x8d,0x6c,0x49,0x70,0xc0, -0xf9,0x33,0x6f,0xe2,0x92,0x44,0x48,0xea,0x2f,0xd7,0x54,0xe7, -0x7a,0x6d,0x29,0x56,0xe6,0x52,0x71,0xeb,0x1d,0x0b,0xa5,0x99, -0xb8,0x68,0xb7,0x2d,0x25,0xff,0x70,0xc1,0x1a,0xe5,0x0e,0x85, -0x6b,0x71,0x21,0x08,0xc1,0xda,0xfb,0xa9,0x12,0xe2,0x80,0xb2, -0x8f,0x2f,0x5d,0x72,0x9f,0xe9,0x9a,0xac,0xa6,0x74,0x3e,0xc8, -0x70,0xd8,0x71,0x30,0xd9,0x51,0x91,0x1b,0x1e,0xe5,0x11,0x00, -0xf9,0xb3,0x7b,0x08,0xa4,0xd4,0xee,0xa3,0xc7,0x49,0x46,0x76, -0xd4,0x49,0xbd,0x47,0xdf,0xf0,0xec,0x46,0xa6,0xe9,0x6a,0xd8, -0xc2,0xe9,0x17,0xf7,0x70,0x1e,0x4e,0x12,0xb7,0x85,0xb7,0x69, -0x79,0x60,0xac,0xc3,0x6c,0x98,0xff,0xd3,0x44,0x1b,0x70,0x91, -0x2f,0x21,0xf8,0x06,0xdf,0xa8,0xad,0x53,0x8f,0x6c,0x35,0x20, -0x5a,0xd2,0x22,0x55,0x39,0xa0,0x04,0xff,0x1c,0x5a,0x46,0x6b, -0x44,0xd6,0xa7,0xa0,0x3a,0x5c,0xb6,0x8b,0x7c,0x45,0xc6,0x7a, -0x90,0xd1,0x04,0x3d,0x52,0x2e,0xa6,0xbb,0xfe,0xfa,0x94,0x73, -0xd0,0x80,0x24,0x10,0x6f,0x0b,0xa5,0x3e,0xe2,0x2b,0x4d,0x95, -0x13,0xa3,0xcb,0xf0,0x48,0x1f,0x78,0xd2,0x08,0xdd,0xc2,0x43, -0x31,0xcd,0xa4,0x54,0xc3,0x95,0x39,0x23,0x17,0x4d,0xb8,0x97, -0xb9,0x91,0x5f,0x7a,0xf9,0x12,0x88,0xf6,0xa5,0x1d,0x8a,0x4a, -0x87,0x8c,0x55,0x2e,0x54,0x61,0x22,0x6a,0x3c,0x47,0x54,0xf3, -0x62,0x88,0x3a,0xc7,0xa4,0x39,0x74,0x09,0x55,0xbf,0x83,0x0b, -0x0b,0x26,0x59,0x5c,0x38,0xf5,0x07,0xac,0x34,0x43,0x0c,0x3d, -0x56,0x52,0x25,0x85,0xce,0x9b,0x61,0xc5,0x7c,0x95,0x2a,0xaf, -0xe4,0x17,0xcd,0xbb,0x2b,0xa5,0xdd,0xf0,0xae,0x5d,0x09,0x87, -0x3a,0x9f,0xbc,0x9a,0x9f,0xe0,0x49,0x56,0x4e,0xf7,0x0e,0x79, -0x71,0x94,0x58,0x05,0xf2,0x69,0x1f,0x44,0xaf,0x65,0xd6,0xe1, -0x3c,0xf0,0x91,0x68,0x84,0xee,0xf0,0x71,0x39,0x6d,0x78,0x0c, -0x97,0xd9,0x8f,0x76,0x33,0xb1,0x99,0xa8,0x7a,0xb8,0x4b,0x94, -0x29,0x65,0x77,0xef,0x43,0x44,0x24,0x89,0xb8,0x7b,0x32,0x04, -0x7a,0x84,0x34,0x1f,0x47,0x1f,0xf9,0x46,0xa1,0xe5,0x4e,0xee, -0x90,0xec,0x63,0x83,0xd0,0x93,0x80,0x37,0x8d,0x7d,0x8f,0x08, -0x0f,0xdc,0xd8,0xa9,0x39,0xee,0x32,0x7a,0x11,0xb8,0x0a,0xb0, -0xb0,0xda,0x64,0x31,0x8d,0x3a,0x85,0x7a,0x74,0xbc,0x75,0xb8, -0x2e,0xee,0xe0,0x4a,0x75,0x1c,0x10,0x2e,0xdb,0x44,0xac,0x22, -0x4a,0xc7,0xe4,0x13,0xcd,0x69,0xa9,0x70,0xec,0xdb,0x0f,0x29, -0x09,0x6b,0xbc,0x27,0x05,0x9c,0x3d,0x4c,0x5f,0x39,0x15,0x12, -0x50,0x99,0xb8,0xd4,0xc6,0x16,0x97,0xf4,0xb7,0x7e,0xa7,0xf3, -0x3a,0x4a,0x56,0xcb,0x24,0xd3,0xd5,0x63,0x1f,0x53,0xe4,0x40, -0x72,0x6b,0xc9,0x21,0x41,0xaa,0x2a,0xc0,0x2b,0x2f,0x33,0x8b, -0xb0,0x14,0x30,0xce,0xd1,0x93,0x9c,0x88,0xa4,0xf7,0xe8,0x17, -0x25,0xe0,0x77,0xba,0xfa,0x50,0x76,0x1a,0x13,0x83,0xbb,0x95, -0x96,0x36,0x6c,0x78,0xf4,0x5a,0xec,0x81,0xbb,0x0e,0xf0,0x7b, -0x92,0xae,0x77,0x69,0x26,0x95,0xf1,0x78,0x5a,0xe6,0xa6,0x49, -0xa6,0xbd,0x79,0x08,0x54,0x80,0x29,0xa1,0xf4,0x24,0xeb,0xe1, -0x3b,0xc4,0xeb,0xe5,0xb1,0xa7,0xc5,0x39,0x95,0x09,0xb5,0x42, -0x23,0xa8,0xb2,0x49,0x26,0xb6,0x8d,0xa4,0x7d,0x24,0xbb,0x81, -0x2e,0x1d,0xa4,0x65,0xfd,0x70,0x30,0xf6,0x0e,0xe3,0xc9,0x37, -0x63,0xad,0x9d,0x59,0xc2,0x0e,0x80,0xcd,0xaf,0x23,0x8b,0xd8, -0xa1,0xaf,0xc0,0x0e,0xa6,0xd6,0x81,0xc2,0x57,0x9e,0xfd,0x2a, -0x2f,0x44,0x88,0x97,0x8a,0x1c,0x3c,0xe2,0x6a,0x7a,0x95,0xbe, -0x43,0x74,0x45,0x06,0xea,0xda,0x21,0xba,0x98,0x23,0xc6,0x91, -0xf1,0x0c,0x99,0xb1,0x36,0x0e,0x43,0xab,0x96,0x97,0xca,0x8f, -0x1a,0xbe,0x1f,0x7e,0x4a,0xef,0xef,0x96,0xd3,0x42,0x53,0xd8, -0xd8,0x7a,0xf2,0xd7,0xa7,0x67,0x09,0x61,0x44,0x67,0xf4,0x29, -0x53,0x91,0x0f,0xab,0xcf,0xb3,0xcb,0x02,0x16,0x3a,0x57,0x89, -0xa1,0x84,0xcc,0x52,0x55,0x09,0x09,0xcf,0x2a,0xfb,0xd1,0xb7, -0x28,0x95,0x1e,0x90,0xa0,0x8c,0x0b,0x3a,0x25,0xaf,0x19,0x13, -0x36,0x72,0x2f,0x46,0xf6,0x6b,0x0c,0xab,0x19,0x11,0x18,0x94, -0x68,0x0b,0xb6,0x07,0xe1,0x85,0x01,0x96,0x6a,0x1e,0xc8,0x48, -0x7c,0x80,0xef,0xce,0xb3,0x79,0x91,0x70,0x28,0x51,0x8a,0x16, -0xa3,0x24,0x98,0xe4,0xf6,0x96,0x28,0x12,0x26,0x24,0x35,0xf6, -0x98,0xe8,0xba,0x84,0xc8,0xbb,0x44,0xaf,0x57,0x64,0xa1,0x25, -0xe6,0xaa,0x4d,0xe2,0xb7,0xcf,0xf9,0x72,0x72,0x3d,0x99,0x0d, -0xa7,0x0f,0xbc,0x80,0xfa,0xeb,0xc6,0xd2,0x10,0x95,0x98,0xb7, -0x66,0xa1,0x3d,0x6a,0x41,0x19,0x0b,0x0a,0x92,0x8e,0x40,0xc9, -0x89,0x6e,0x3d,0x1e,0x61,0xe4,0xe3,0xd2,0x23,0x2d,0x55,0x39, -0x51,0x33,0x04,0x38,0x34,0xcc,0xfe,0x71,0x8c,0x34,0x15,0xad, -0x61,0x28,0x48,0xa4,0xf4,0xb5,0xcd,0x03,0x7e,0xe3,0x43,0x2e, -0x10,0xdc,0x5c,0x81,0xeb,0x41,0x71,0x1e,0xcc,0xb2,0x26,0xdb, -0xe4,0x4c,0xb6,0xbc,0x54,0x46,0xf4,0x03,0x3f,0xfe,0x7d,0x89, -0x88,0x20,0xc7,0x2a,0xb0,0x0d,0x84,0x73,0xcc,0xca,0xdf,0x4c, -0x8a,0xe4,0x93,0x1c,0x96,0xab,0x4b,0x98,0xcc,0xde,0x0c,0xaf, -0xfd,0xd1,0xf3,0x25,0x02,0xf1,0xc8,0xb3,0x3c,0x2d,0x36,0xfb, -0x47,0x3a,0xc7,0xdc,0x0d,0xa7,0xb2,0xb2,0x8c,0xa4,0xc0,0xc5, -0x58,0x47,0x71,0x93,0xef,0x93,0xce,0xd8,0xaa,0x98,0x74,0x12, -0xc0,0x23,0x7b,0xd4,0x44,0xc6,0x13,0x29,0x48,0x4c,0xda,0x1d, -0x34,0x69,0x6a,0x18,0xe7,0x06,0x21,0xe4,0x76,0xa2,0xce,0x3b, -0x7e,0x5a,0xee,0x7a,0x40,0xe8,0x31,0x58,0x9b,0x6a,0xa8,0x3e, -0xb4,0xa6,0x18,0xf5,0x69,0xc6,0x07,0x33,0x0d,0xa7,0x3b,0x19, -0xa2,0xc4,0x13,0xd5,0xcb,0xaa,0x91,0xea,0xd6,0xbc,0x61,0x8d, -0x7c,0x0f,0xc7,0x26,0xe3,0x03,0xd3,0x4c,0x55,0x29,0x67,0x39, -0x35,0x39,0xb3,0x73,0x55,0x5a,0x7a,0xdf,0x12,0xcd,0x7c,0x18, -0xfa,0x8c,0x44,0x3b,0x77,0x3a,0x93,0xcc,0xc2,0x85,0xc8,0x76, -0xfb,0x6a,0xf2,0xe9,0x44,0x87,0xf5,0x79,0x55,0xdd,0x8d,0x2b, -0x3e,0x0a,0xa3,0x8c,0x34,0xcd,0xc1,0x22,0x43,0x31,0xcf,0xfa, -0x3a,0xc3,0xb6,0xdb,0x06,0x9a,0x30,0xe9,0x9d,0xfa,0xd6,0x06, -0x3d,0xb4,0x1f,0x2e,0x9f,0xf4,0xf7,0x4b,0xdd,0xc2,0x6d,0x4a, -0x59,0x4f,0xe6,0x8b,0xcf,0xd4,0x8c,0x2d,0x3d,0x20,0x8b,0x03, -0x51,0x4e,0xda,0x3c,0x5c,0x91,0x53,0xd1,0xdf,0x36,0xf4,0xf6, -0xaa,0x37,0x79,0x8c,0x1f,0xe3,0x90,0x55,0x12,0x6c,0xb7,0x1d, -0xde,0x56,0xed,0xf6,0x16,0x06,0x4a,0x50,0xab,0x1d,0x41,0x36, -0x2d,0x81,0xa4,0x0a,0x35,0x02,0x75,0xc5,0x0c,0x16,0x15,0x1e, -0x03,0x5e,0x11,0x5b,0x6f,0x4e,0x3e,0xea,0x20,0xdd,0x1f,0x8e, -0xef,0x73,0xee,0xb4,0x3a,0xf1,0x5c,0xdd,0x2e,0x72,0xb4,0xa6, -0x4f,0xb7,0x39,0xf1,0x96,0xf3,0x69,0x20,0xc5,0x40,0x6f,0x59, -0x4d,0x6f,0x58,0xce,0xd0,0x25,0xb6,0xb1,0x72,0xc2,0xd1,0x5e, -0x0c,0xd8,0xf1,0xf7,0xe9,0x46,0xc6,0x80,0x57,0x82,0xc5,0xf4, -0x84,0x5f,0x0d,0x51,0x6d,0x4c,0x9f,0x75,0x60,0x63,0xc8,0x3e, -0x08,0xd0,0x1a,0xaa,0x21,0x0f,0xc9,0x75,0x66,0x97,0x15,0x7c, -0x0a,0xa9,0xd1,0x0a,0x43,0x6c,0x33,0x18,0x46,0xf8,0x6e,0x17, -0x25,0xfc,0x5b,0xaf,0xb1,0x2b,0x01,0x83,0x56,0x8d,0x7c,0x36, -0xbe,0xab,0xe1,0xb8,0x94,0x11,0x9d,0x2c,0x07,0xd8,0x56,0x32, -0x99,0x46,0x50,0x9f,0xc6,0xfa,0x50,0x74,0xe0,0x48,0xee,0xae, -0xbe,0x2d,0xe2,0x98,0x9b,0x23,0x0b,0x93,0x01,0x4e,0x47,0xb8, -0x51,0x0f,0x91,0xd5,0x1d,0xca,0xfb,0x68,0x90,0xbf,0xe7,0x52, -0x07,0x03,0xb6,0xd7,0x28,0x1d,0xf4,0xf6,0x53,0x88,0x33,0xf1, -0xcf,0x33,0x0b,0x84,0xf7,0x47,0xc3,0x2e,0xd2,0x1b,0xc1,0x40, -0xb9,0xe7,0xb2,0x06,0x00,0x7e,0x98,0xf0,0xe5,0x08,0xce,0xd2, -0xa8,0x19,0xb4,0xd0,0x32,0xa7,0x1e,0x2a,0x48,0xe6,0x0e,0x39, -0x96,0x58,0xaf,0x59,0x12,0xc0,0xb4,0xa2,0xe7,0xde,0x91,0x21, -0x8b,0x0c,0x55,0x53,0xb6,0x83,0x02,0x06,0x00,0xff,0x59,0x32, -0xe1,0xbd,0xc8,0x81,0x8c,0x43,0x19,0x2b,0xc5,0xc4,0x69,0xec, -0x6a,0xd8,0x2f,0x3d,0x92,0x46,0x25,0xd0,0xee,0xa5,0x22,0xbc, -0x80,0x3b,0x72,0xed,0x0d,0x74,0x47,0x8d,0xa3,0xc8,0xbd,0x25, -0x0d,0x82,0x86,0x15,0x8d,0xcd,0x35,0x0f,0x19,0xf4,0x6d,0xd0, -0xba,0x32,0x49,0x8e,0xc2,0x1e,0xd1,0x54,0x87,0x15,0x04,0xe9, -0x19,0xe6,0xa7,0x77,0x64,0xbf,0x16,0xd8,0xa2,0xcb,0x97,0xe7, -0x92,0x37,0xcc,0x91,0x6c,0x18,0x38,0xce,0x10,0x4a,0x54,0x95, -0xc3,0x46,0x9c,0xfd,0x57,0xd9,0x4c,0x72,0xf5,0x29,0xda,0x0a, -0xcb,0x38,0x35,0x34,0x48,0x95,0x8b,0x1e,0x29,0x12,0x62,0x9f, -0x9e,0xf0,0x36,0x90,0x42,0x7c,0xc4,0xd3,0x5a,0xa0,0xac,0xe2, -0x49,0x04,0x2d,0x27,0xef,0xb8,0xf3,0x6b,0xbd,0xde,0x3b,0xf4, -0xc7,0x84,0x95,0x5c,0x36,0x67,0x1c,0x22,0xe6,0x4e,0xf1,0xe6, -0x6b,0x11,0x18,0xf2,0x86,0x22,0x4d,0xad,0x1d,0x0f,0x9c,0xc3, -0x1f,0xfc,0x0a,0x2d,0x3e,0xeb,0x69,0x64,0xe4,0xd0,0xe5,0x1b, -0xfc,0xbd,0xf6,0x09,0x92,0xc4,0xb5,0x7f,0x8d,0x24,0xa1,0x26, -0x19,0x92,0x44,0x49,0xa3,0x46,0x6f,0x29,0x7c,0x09,0x60,0x0b, -0x91,0x85,0x2c,0xfc,0xb1,0xf8,0xfc,0x47,0x9e,0xe9,0x62,0xd5, -0xf0,0x36,0x41,0x56,0xbd,0x26,0x23,0x42,0x14,0x09,0x33,0x9e, -0xcc,0xd0,0xa2,0xdc,0x70,0xfa,0x1f,0xe3,0xcf,0x46,0x41,0x08, -0xf5,0xd3,0xc4,0x62,0x92,0x8e,0x1f,0xd9,0xc9,0x1a,0x01,0x10, -0x38,0x1e,0xc5,0xcb,0x73,0xb4,0x04,0x82,0xc4,0x9c,0x2c,0xf2, -0x0e,0x9d,0x60,0x35,0xaa,0xb2,0xae,0xba,0x12,0x19,0x30,0xf5, -0xf5,0x62,0xc8,0xf4,0xba,0x48,0x7d,0xeb,0xdd,0x42,0xf2,0x51, -0xb2,0x3c,0x61,0x8a,0xd1,0x17,0x1a,0x8a,0xb2,0x99,0x2e,0x0c, -0xa8,0x63,0xe5,0x64,0xa4,0x96,0x87,0x26,0xa3,0x01,0xbb,0x0f, -0xc7,0x78,0x6e,0x04,0x63,0x27,0x38,0x10,0xad,0x09,0xb3,0xd3, -0x35,0x5e,0x5a,0xc7,0xa6,0x7c,0x7a,0x41,0xc3,0x85,0x6b,0xe5, -0xc6,0xd0,0x11,0x66,0x9c,0x60,0xb0,0x08,0x1a,0x61,0x82,0x7a, -0x43,0x16,0x94,0x45,0x61,0x0f,0x5b,0x02,0x6d,0xe6,0x31,0x6a, -0xb9,0xa8,0x70,0x44,0x7b,0x92,0x5c,0xeb,0x56,0x61,0x8d,0xfb, -0x71,0x95,0xd6,0xbe,0x1f,0x11,0x16,0x81,0x4f,0x9b,0xa6,0x8c, -0xa9,0x81,0xc8,0x3a,0x86,0x41,0x1d,0x82,0x5d,0x20,0x28,0x6e, -0xc6,0xe8,0xf9,0x48,0x75,0x4b,0x47,0x28,0x6d,0x2e,0x12,0x73, -0x2c,0xf2,0x8a,0x9c,0x85,0x53,0x94,0xb8,0xaa,0x50,0x96,0xef, -0xe4,0xf4,0x34,0x5f,0x0e,0xa7,0x8b,0x9b,0x61,0xb9,0x4f,0x3f, -0xe7,0x69,0xff,0x97,0x6c,0xf0,0xcd,0x39,0x49,0x4a,0xcd,0xe1, -0x3e,0x98,0xac,0x3e,0x97,0xfb,0x12,0x38,0xaf,0xbe,0x41,0x71, -0x2a,0xce,0x82,0x22,0x50,0x0b,0xd1,0x9f,0x45,0xc9,0xac,0xd5, -0x7c,0xb1,0x06,0xc2,0xe9,0x66,0xb5,0xbe,0x98,0xaf,0x56,0xf3, -0xdb,0xf5,0x74,0x7c,0xb5,0x22,0x27,0xa8,0xa3,0x49,0x05,0x77, -0xe2,0xe7,0xea,0x23,0x5c,0xac,0x90,0x71,0x86,0x84,0x05,0x5d, -0x09,0x69,0x6f,0x67,0xef,0xb2,0x3f,0x1e,0x0e,0xb2,0x2e,0x09, -0x91,0xdd,0x0e,0x97,0x40,0x78,0x41,0x1e,0x0e,0xa0,0xcc,0xd8, -0xdd,0x2d,0x9a,0x30,0x5e,0x95,0x35,0x5f,0xbc,0x44,0x92,0x41, -0x32,0xfa,0x77,0xee,0x7e,0x93,0xed,0xaa,0x2c,0x1c,0x44,0x41, -0x05,0x8b,0x4f,0x0f,0x97,0xe8,0xed,0x2c,0x3e,0x65,0xfd,0xe1, -0xde,0x3f,0xfe,0x75,0xd0,0x71,0x45,0x97,0xe3,0xe9,0xab,0xbb, -0xc8,0x81,0x33,0xf9,0xc1,0xcd,0xca,0xa8,0xbc,0x94,0xc0,0xa9, -0x90,0xce,0x95,0xf7,0x3f,0xbc,0x7e,0xf2,0x5f,0x40,0x33,0x4f, -0xe7,0x97,0xbf,0x02,0x15,0x7c,0x59,0x55,0xa7,0x37,0xf3,0x8f, -0xe5,0xbd,0x0e,0x11,0x50,0xce,0x17,0xd5,0x7c,0x0a,0xd3,0x99, -0xe4,0x1f,0x26,0xd5,0xe4,0x62,0x02,0xdd,0xfa,0x5c,0x24,0xec, -0x30,0x06,0x1d,0xc5,0x10,0x9c,0x00,0xc2,0xab,0x39,0x0c,0xc9, -0xf4,0x6c,0x39,0x9c,0x55,0xb0,0xf4,0x6f,0xcb,0x7b,0x98,0x5d, -0x98,0xf7,0x53,0x9c,0x8d,0xd9,0x75,0x71,0x90,0x5f,0xc1,0x71, -0xf6,0xf3,0x18,0x07,0xbd,0xf8,0xee,0xe0,0x80,0xca,0xf0,0x8a, -0x2c,0xfb,0xc9,0xd9,0x7c,0x01,0xcd,0x7c,0x8b,0x89,0xf0,0xfb, -0x03,0xcd,0x09,0x04,0x5e,0xc0,0xac,0xa0,0x6c,0x56,0x55,0xbd, -0x59,0xc2,0x9a,0xfb,0x84,0x88,0x5f,0xf2,0x33,0x39,0x57,0x81, -0xd4,0xd7,0xf0,0xef,0xe5,0xfc,0x1f,0xf0,0xf7,0xb6,0x4a,0x06, -0x0d,0xfa,0x81,0x50,0x2e,0xb2,0x2f,0xf9,0xf5,0x6f,0x3f,0xcc, -0x76,0x36,0x65,0x70,0x55,0xc2,0xc5,0x9b,0x57,0xbc,0x1e,0x6f, -0x51,0x2a,0x8e,0x9e,0x81,0x8e,0xac,0x1c,0x0e,0x5f,0x15,0x6c, -0x4e,0x9c,0x0e,0x45,0xce,0x5e,0xba,0x85,0x2c,0xa7,0x3c,0xaa, -0xbe,0x91,0x79,0xf4,0x9a,0x3f,0x76,0x00,0x4c,0x4c,0x07,0x38, -0xac,0x9c,0x32,0x29,0x74,0xc4,0x37,0x08,0xd1,0xdb,0x9d,0x43, -0x69,0x87,0xd1,0xc7,0x76,0x2e,0xae,0xa8,0xc1,0x3b,0x0d,0x72, -0x4c,0x54,0xa4,0xd6,0xb3,0xa2,0xa9,0x12,0x7c,0xb2,0x70,0x59, -0x6a,0x4f,0x16,0x8f,0x0f,0x51,0xc0,0x1e,0x96,0x4c,0xc3,0xd3, -0x39,0x46,0xff,0x38,0x19,0xc9,0xf3,0x1c,0x31,0xde,0x61,0xd9, -0x34,0x3d,0xf1,0x07,0x39,0xd1,0x61,0xe0,0xfc,0xfa,0x7a,0x6a, -0x9f,0xc0,0x57,0x64,0x6d,0x1c,0x07,0x1e,0x5d,0x7d,0x96,0xce, -0xe1,0xa7,0x97,0x0b,0xc7,0x1c,0x0f,0xc8,0x99,0x64,0xf7,0x29, -0x96,0xe9,0x51,0xbe,0x62,0x52,0xfd,0x48,0xeb,0x97,0xeb,0x73, -0x8a,0x71,0xfc,0x06,0x81,0x8d,0x49,0x75,0x2c,0x24,0x0e,0xdb, -0x9d,0xe2,0xab,0x43,0x03,0x5f,0xa4,0xaa,0x7e,0x9c,0xcf,0x7f, -0xad,0x8a,0x7b,0x39,0x6e,0x8a,0xfb,0x9a,0x49,0x9c,0x1c,0xb5, -0xa7,0xef,0xd0,0x62,0x2f,0xde,0x0d,0xfe,0x43,0xa5,0xe6,0xf9, -0x48,0x13,0xea,0x4c,0xc0,0x78,0xef,0xa6,0x09,0x59,0x0d,0x5d, -0xf5,0x92,0xc3,0x04,0xa5,0xd2,0x37,0x68,0x97,0x0d,0xb7,0xd9, -0xdd,0xed,0x05,0x5a,0xce,0xbd,0x84,0xed,0x79,0x3b,0x3b,0x99, -0xdf,0xcd,0x56,0xa2,0x5f,0x3f,0x7d,0x2d,0x2d,0xd9,0x09,0xf6, -0x1a,0x7c,0xe1,0x8b,0xe2,0x8f,0xee,0x6b,0xee,0xb3,0xcd,0x97, -0x70,0x8a,0xce,0x48,0xdd,0xf6,0xe3,0x64,0x34,0xff,0x48,0xa1, -0x7f,0x90,0x56,0x3f,0x85,0xe6,0xf3,0x5b,0x32,0x98,0x4d,0x1b, -0x70,0xbe,0x80,0xde,0x26,0x57,0xd3,0xf9,0x70,0x95,0xc4,0x02, -0x8b,0x90,0xe1,0x19,0x26,0xf4,0x12,0x0d,0x25,0x45,0x42,0xcb, -0x8d,0x3f,0x36,0xbc,0x5c,0x8b,0x6d,0xdb,0x2b,0x87,0x71,0x5d, -0x0e,0x69,0x9c,0x18,0x87,0xfe,0xb6,0xce,0xd5,0xf8,0x73,0x3d, -0x8a,0x3e,0x09,0xb2,0x1b,0x55,0xbe,0x1a,0x6f,0x70,0x6e,0x88, -0x9f,0x47,0xd2,0xbf,0x0e,0xbd,0xb8,0x1d,0x4f,0x89,0x72,0xa2, -0x25,0xce,0x6d,0x2a,0x3d,0x10,0x32,0x04,0x6f,0xf3,0x4b,0xb7, -0xfb,0x0a,0x08,0xad,0xf2,0x6d,0x4d,0x2b,0xe1,0x7a,0x1e,0xcd, -0x97,0x18,0x4b,0xa8,0x3d,0x81,0x74,0x6d,0x00,0xda,0x84,0x1a, -0x65,0x60,0xd3,0x02,0x62,0x06,0xa3,0x17,0xaf,0xd0,0x68,0x07, -0x36,0xdf,0xfa,0xa8,0x4c,0xf0,0xda,0xed,0x04,0x16,0x1e,0x20, -0xbc,0xfa,0x49,0x5a,0x09,0x14,0x26,0x86,0x9f,0x3c,0x66,0xca, -0x00,0x67,0xc1,0xb1,0x00,0x39,0x0b,0x6a,0x25,0x37,0xe2,0x48, -0xbc,0x42,0x06,0xef,0x46,0xb9,0x7f,0x4c,0x14,0x03,0x9c,0xac, -0xf6,0xc0,0xf7,0x8f,0x7d,0xce,0x45,0x6c,0x89,0xdb,0x8a,0x39, -0x90,0xc6,0x3c,0xcc,0xbe,0xc1,0xd0,0xa3,0x41,0x67,0x31,0x5c, -0x56,0xbc,0x14,0xd2,0xc6,0xd3,0x26,0xa3,0x69,0x2b,0x93,0x19, -0xad,0x6e,0xf2,0x30,0xc2,0xd2,0x03,0xf2,0x76,0xa5,0x09,0xae, -0x11,0xe8,0x26,0xea,0x95,0x7b,0x47,0x4e,0x35,0x5d,0xec,0xbf, -0xda,0xe1,0xe4,0x0d,0x13,0x4c,0x21,0x15,0xeb,0x94,0xc9,0x02, -0x1d,0x5f,0xc5,0x2b,0x19,0x4e,0x98,0x25,0x71,0xa9,0xe8,0xbc, -0x46,0x64,0x79,0xc7,0x35,0xe3,0x40,0x8c,0x3a,0x7a,0x7b,0xda, -0xf8,0xbc,0x81,0x92,0xac,0xb3,0x51,0x42,0x3a,0x2d,0x7e,0x34, -0x4b,0x98,0x93,0x9b,0xf1,0x72,0x82,0x98,0xbc,0xce,0x54,0x15, -0xce,0x14,0x8f,0x16,0xcf,0x55,0xa5,0x73,0x65,0xf7,0x43,0x16, -0x4c,0x3b,0x7b,0x43,0xb1,0x55,0x50,0x5e,0xfb,0x10,0x4d,0x5b, -0xb5,0x69,0x97,0x11,0x3c,0xbd,0x33,0xf8,0xdd,0xf6,0x8e,0xea, -0xfa,0xba,0xad,0xe2,0x0c,0x2a,0xff,0xb7,0xed,0x0e,0xbf,0xeb, -0xfe,0x1b,0xb6,0xc8,0x96,0x9d,0x80,0x9e,0x40,0xe2,0x9d,0x70, -0xa0,0x43,0x8b,0x9d,0xb7,0xc3,0x2b,0xf9,0xed,0xa1,0x4c,0x23, -0x27,0x63,0x86,0x76,0xf0,0x09,0xc3,0xc1,0x35,0x48,0x1e,0x41, -0x30,0x15,0x91,0xfb,0x3a,0xee,0xa3,0xa0,0x6a,0x09,0xe2,0xe8, -0x24,0xa7,0xc3,0x9d,0xda,0xb1,0x5e,0xd3,0x4f,0x0f,0x96,0xfb, -0x6d,0x69,0x76,0x09,0x94,0xcf,0xb8,0xa5,0x64,0xf6,0xca,0xf0, -0x6f,0x61,0x3d,0xc3,0xaa,0xba,0xc4,0x02,0x68,0x4b,0x1e,0xb9, -0x5c,0xe8,0xa1,0x24,0xc3,0x3f,0x70,0xcc,0x7e,0xb4,0x92,0x91, -0xd4,0x0b,0x91,0x10,0x72,0xaf,0x71,0xfc,0x8c,0xed,0xce,0x4b, -0x7e,0xf9,0x98,0x8e,0x00,0x9b,0x21,0x6a,0x5c,0xbb,0x25,0xc5, -0x32,0x48,0x92,0xd5,0xe6,0xa7,0x8c,0x23,0xf2,0x38,0xa2,0x94, -0x32,0x72,0x9c,0xd0,0x1d,0xa7,0x4f,0x80,0xfc,0x18,0x41,0x0d, -0xc2,0xfa,0xd1,0x24,0x6e,0xd6,0x58,0x5f,0x1d,0xa8,0x36,0xc0, -0x7a,0xa0,0x80,0x1b,0x59,0xac,0xca,0xc1,0xd4,0x9e,0xc8,0xd5, -0x4a,0x3b,0xb6,0x97,0x3a,0x5c,0x2b,0x76,0x09,0x22,0xe5,0xb7, -0x14,0x94,0x39,0x87,0x63,0x07,0xf5,0xf4,0x68,0x19,0x34,0x78, -0xcc,0xc9,0xdf,0x85,0x17,0x39,0xdc,0x9b,0xab,0x9b,0xfc,0x76, -0x32,0xfb,0x99,0x03,0xc3,0x4f,0x1c,0xd0,0x5c,0xa5,0xcb,0x4f, -0xbc,0xe6,0x00,0x09,0xa4,0x01,0x92,0xd4,0x9e,0x06,0xb0,0x5d, -0x6a,0x85,0x94,0xdc,0x74,0xa8,0xc2,0x9c,0x66,0xe0,0xc1,0xf0, -0x0f,0xe5,0xf5,0x6b,0x4c,0xed,0xac,0x48,0x01,0x34,0x1d,0x80, -0x87,0xd8,0x12,0x9d,0x24,0x7d,0x2d,0xa7,0x02,0xf6,0xb2,0x51, -0xb5,0x8b,0x30,0xc7,0xcc,0xd0,0x2e,0xcc,0x5d,0xc0,0xe7,0xa3, -0x76,0x5b,0x88,0xa2,0x50,0x4b,0x8e,0x46,0xa8,0x24,0x08,0xdd, -0x70,0xb4,0x24,0xb2,0x36,0x78,0x1a,0xaf,0x63,0x19,0x66,0x8b, -0x52,0x2d,0x64,0xec,0x62,0x30,0xaa,0x52,0xa1,0xcd,0xf2,0xb1, -0x09,0xa4,0x6b,0x42,0x04,0x5b,0x03,0xac,0x57,0xb8,0xf1,0x8e, -0x89,0x62,0x17,0x82,0x5d,0x58,0x30,0x28,0xbb,0x42,0xf3,0x0b, -0x9d,0xfe,0xd2,0x1a,0x64,0x8e,0x9f,0x29,0xf3,0x3b,0x96,0x1c, -0xd2,0xaa,0xf9,0xb2,0x82,0xff,0x91,0x3e,0xfa,0xe7,0x57,0xda, -0x57,0x2f,0x24,0x35,0xa3,0x8f,0x9e,0x9f,0xb8,0x7f,0x66,0x7f, -0x0a,0x42,0x60,0x62,0xb6,0x2c,0x8e,0x1d,0x47,0x81,0x87,0xeb, -0x03,0xbb,0x23,0x93,0x28,0x3d,0xe3,0x26,0x90,0xce,0xe8,0x2d, -0x5f,0xc3,0xd2,0xd7,0x72,0x89,0x2e,0xc4,0x2a,0xcd,0x87,0x51, -0x58,0x3d,0x47,0x94,0xb5,0x31,0xa5,0x68,0x41,0xf3,0x38,0x4b, -0x82,0x88,0xf1,0xe9,0xe4,0x1f,0x63,0x35,0x4b,0x07,0xe8,0xf5, -0xf8,0x96,0x10,0xec,0xdc,0xf5,0xa2,0xbb,0x00,0x4a,0x73,0x8a, -0xc0,0x3b,0x88,0x22,0xd8,0xf2,0xcd,0x15,0x73,0x44,0x26,0xe7, -0x3a,0x21,0xed,0xc3,0xbb,0xd5,0x9c,0xf1,0xf6,0xf0,0x6d,0xbb, -0x9f,0xdc,0x8c,0x85,0xca,0xa5,0x85,0x08,0xd4,0xad,0x97,0xaa, -0x11,0x97,0x5c,0x8d,0xd7,0x5f,0xf9,0x00,0x7d,0xa1,0xd8,0x73, -0xb4,0xef,0x7b,0xfa,0xfc,0xd1,0x9d,0x5f,0x5d,0x01,0x7e,0x41, -0x2b,0x19,0xc6,0xcf,0xb0,0x38,0x78,0x26,0x62,0x94,0x2c,0x91, -0x1c,0x89,0x23,0x92,0xba,0x98,0x59,0x2a,0x65,0x66,0x41,0x5e, -0xa7,0xe7,0xae,0xa5,0x8a,0xd7,0x4b,0x26,0x36,0x62,0xf4,0x03, -0xd5,0x19,0xbe,0x98,0xc7,0x39,0x42,0xae,0x6a,0xdd,0x0d,0x28, -0x05,0x92,0x3d,0xe2,0x1d,0x46,0x31,0xf4,0xd8,0x65,0xd7,0xbc, -0x73,0xe8,0x3f,0x16,0xa7,0x8c,0x1f,0xc6,0x8c,0x0a,0xd6,0x80, -0xf5,0x86,0x77,0xf4,0xc6,0xf4,0x60,0xc3,0x62,0x3c,0xf1,0x62, -0xfe,0x09,0xd6,0x11,0x20,0xc6,0x68,0xff,0x70,0x8e,0x5e,0xb2, -0xf6,0x20,0xca,0xbc,0x2e,0xf9,0xd1,0x74,0x59,0x13,0x43,0xb7, -0xeb,0x6f,0x71,0x40,0x4c,0xaf,0x18,0xbc,0x10,0x69,0x01,0x46, -0x45,0xab,0x41,0x53,0x1e,0x5a,0x0f,0xde,0x0e,0xa4,0x64,0xe6, -0x69,0x4e,0xfd,0x85,0x50,0xdb,0x2a,0xbd,0xfa,0xe6,0x11,0x25, -0x65,0x7f,0x1c,0x48,0x0c,0x3b,0xf0,0xe8,0x75,0x0f,0x0e,0xbf, -0x31,0x08,0x0b,0x33,0xa1,0xba,0xbb,0x87,0x59,0x27,0x49,0x0a, -0xb7,0x04,0x91,0x7a,0x4d,0x92,0xad,0xb3,0x69,0xe6,0xd1,0x9c, -0x3b,0xb9,0x6d,0x47,0x7d,0x5b,0x2b,0x05,0x5b,0xd6,0x70,0x22, -0x86,0x09,0xbb,0x0f,0xb9,0x83,0xa9,0x66,0x4b,0x3a,0x87,0x07, -0x07,0xdf,0x30,0xd2,0x9f,0x64,0xd8,0x1e,0x31,0xdf,0x53,0x86, -0x87,0x76,0x43,0xef,0xd7,0xeb,0x2a,0xf8,0x4a,0x92,0x23,0x8e, -0x40,0xd2,0xb8,0x3c,0xcc,0xb9,0xc6,0xc7,0xe5,0x21,0x26,0x95, -0x42,0x27,0xa0,0x85,0x4a,0xfb,0xcc,0xb8,0x9c,0xdc,0xa6,0xe2, -0x94,0xde,0xfb,0x56,0xc4,0x16,0x92,0x78,0x8c,0x1c,0xa6,0x31, -0x1f,0x15,0x2d,0x50,0x37,0xc5,0xa7,0x09,0x83,0x4a,0xe4,0xb4, -0x11,0xca,0x24,0xec,0xc9,0x4e,0x43,0x57,0x1c,0x79,0x22,0xdf, -0x25,0xb7,0x81,0x97,0x86,0x64,0xe9,0x35,0x37,0x53,0x06,0x32, -0x2b,0x38,0x19,0x8d,0x40,0x75,0x34,0xce,0xaf,0xdd,0x06,0x5d, -0x1b,0x5d,0xcd,0x40,0x2c,0x4e,0x90,0x87,0xfa,0x92,0xd0,0x03, -0x62,0xf3,0x35,0xac,0xec,0x5b,0x9f,0xfa,0x55,0xab,0xdb,0x2d, -0xb2,0xda,0x19,0x75,0xef,0x18,0x94,0x2c,0x4e,0xbd,0xe7,0xf8, -0x94,0xcc,0x45,0xee,0xf3,0xc6,0x34,0x15,0x26,0x03,0x77,0xf2, -0x6c,0xea,0xaf,0x88,0x74,0x21,0x78,0x0f,0xb5,0x9e,0xd3,0xb8, -0x88,0xe3,0xe4,0x90,0x5f,0x11,0x3f,0x73,0xca,0xec,0x4b,0x73, -0xc0,0x2f,0x60,0x47,0xd6,0x0f,0x78,0x8c,0x7d,0xf0,0x80,0xaf, -0x77,0xd9,0xed,0xe5,0x80,0x60,0x21,0xf0,0xf1,0xed,0xeb,0x80, -0x28,0xc3,0x8b,0x8e,0x47,0xd7,0xf4,0x34,0xe3,0xfa,0xe9,0xae, -0x73,0x1b,0xd7,0x8f,0xc6,0x26,0xb3,0xdc,0xaf,0xc5,0xd2,0x77, -0xd5,0x58,0xe7,0xf3,0xd2,0xd4,0x36,0xb6,0xcb,0xfc,0xe2,0x87, -0x50,0x20,0x73,0x33,0x7d,0x5f,0x1e,0xc8,0xc1,0xc4,0x91,0x7c, -0x0e,0x43,0x6c,0x5d,0x12,0x5d,0xd7,0x13,0xf3,0xf3,0x5e,0x53, -0x76,0xa4,0xff,0x90,0x5b,0x8f,0x9b,0xc2,0x50,0x97,0x96,0x49, -0xd4,0x95,0x65,0x61,0x49,0xc9,0xda,0x8d,0xb7,0xc9,0x9b,0x7a, -0x42,0x5c,0x70,0x6b,0x2a,0xd2,0x76,0x65,0x67,0x7b,0xdf,0x55, -0x29,0x2b,0x14,0x70,0xe3,0x85,0x87,0x07,0xd1,0x62,0x38,0xc2, -0x17,0x6d,0x0c,0xf2,0x2d,0x52,0x24,0x34,0x18,0x56,0x36,0x6d, -0x41,0xbc,0xef,0xbc,0xba,0xbb,0x82,0x9f,0xa6,0xd5,0x83,0xc9, -0x1d,0x4e,0x86,0x65,0xc4,0x2f,0x6b,0x35,0x39,0x72,0x63,0x06, -0x34,0xe7,0x2c,0x63,0xa4,0xf9,0x50,0xe6,0x7a,0x55,0xd5,0x4d, -0x22,0x51,0xa9,0x1e,0x4b,0xed,0xd2,0xab,0x06,0x3a,0x0c,0xca, -0x0a,0x91,0x05,0x3f,0xfa,0x4e,0x05,0xed,0x05,0x92,0xb6,0xc2, -0x31,0xf5,0xfb,0x93,0x81,0x6b,0x12,0x55,0x41,0x92,0xf8,0x12, -0xda,0x7b,0xe4,0xc2,0x07,0x8e,0xac,0x53,0x50,0x9b,0x4d,0x1e, -0xd0,0x11,0x0c,0x39,0xab,0x1f,0x18,0x51,0xcf,0x91,0x85,0x52, -0xd6,0x6e,0xfb,0x4c,0x1e,0x94,0x96,0x8f,0x0e,0xca,0xfd,0x7f, -0x7d,0x74,0x80,0xb6,0x0b,0x2e,0x96,0x40,0x91,0x42,0xe6,0xfd, -0xf3,0xfe,0xf9,0x00,0x1f,0x7d,0x4e,0xde,0xbe,0x78,0x06,0x5f, -0xcb,0xde,0xf9,0x0c,0xd3,0xd9,0xc0,0x28,0xcc,0x22,0x72,0x19, -0x2b,0xd6,0xe4,0xe7,0xb8,0x35,0x1b,0x35,0x5e,0x93,0xe9,0xd2, -0x35,0x99,0x1f,0x5d,0xa3,0x69,0x53,0xd6,0xec,0x97,0x72,0x24, -0xf0,0x4b,0x85,0xd8,0x54,0x01,0xbf,0x19,0xaf,0xf1,0x15,0x9c, -0x8c,0x1a,0xfc,0x3a,0xfe,0x7c,0x3d,0x9e,0x65,0xfb,0x93,0x86, -0x27,0x8b,0x4a,0x15,0x33,0x1a,0xf8,0xe5,0x92,0x9b,0x94,0xe2, -0xd5,0x48,0x9f,0xe4,0xe6,0x67,0x07,0x5c,0xbc,0x61,0xd4,0x36, -0xd5,0xb6,0x48,0xbb,0x46,0x5f,0x6e,0x49,0x08,0x48,0xab,0x81, -0x43,0x41,0x74,0x24,0x35,0xc9,0x31,0xa7,0x5d,0xe6,0x9e,0x13, -0xa0,0x55,0xbb,0x58,0x9a,0xc2,0x92,0xba,0x9b,0x26,0xfb,0x88, -0x58,0x1b,0xf3,0x19,0xb1,0x29,0x18,0x0a,0x58,0xf8,0x6c,0x54, -0x7a,0x27,0xe0,0xc4,0x4f,0xaa,0xd4,0xba,0xff,0x47,0x22,0xc0, -0x8f,0x35,0x2f,0x13,0xa7,0x84,0xf8,0x8a,0xe9,0x8a,0x9d,0x68, -0x1a,0x25,0x17,0x6b,0x28,0xaa,0x56,0xa7,0x08,0x12,0xec,0x3c, -0x60,0xc8,0xc1,0x94,0x43,0xdd,0xcf,0x60,0xe4,0x26,0x62,0x32, -0x44,0x9c,0xe8,0x06,0xea,0xb8,0xe8,0x1f,0x36,0x8d,0x3c,0x91, -0x41,0x14,0xbb,0xff,0x8b,0xde,0x8b,0x90,0x7f,0xd3,0x33,0xb2, -0xc8,0xc8,0x97,0xb3,0xfb,0x57,0x27,0xef,0x9e,0x3c,0xca,0x30, -0x13,0xdc,0xf1,0xce,0x91,0xf1,0xe3,0x6f,0x6c,0x5c,0xca,0x79, -0x72,0xbe,0x3c,0x9f,0x25,0x78,0x37,0x17,0xbf,0xbf,0x08,0xcb, -0x89,0x1a,0x9c,0x94,0x16,0x9c,0x3f,0xf9,0x86,0xf9,0x0a,0xad, -0x59,0x60,0x78,0x28,0xca,0xc0,0x7a,0x42,0xa1,0x58,0x16,0x1c, -0x68,0x3e,0x2f,0x2c,0x75,0x8f,0xe8,0x79,0xdd,0x93,0xba,0x32, -0x03,0x9f,0x34,0x69,0x56,0x18,0xb6,0x6f,0x0f,0x10,0x49,0x46, -0xd3,0xab,0xbe,0xbe,0x3c,0x0d,0xca,0xf1,0xec,0x12,0x26,0xf9, -0xa7,0xb7,0xcf,0x91,0x79,0x03,0xe7,0xfd,0x6c,0x85,0xb5,0x00, -0xd6,0x09,0x78,0x5e,0x43,0x9a,0x28,0xe0,0x10,0x6f,0xdb,0x37, -0x3b,0x62,0xfc,0xd9,0x14,0x23,0x63,0x7d,0x3a,0x5e,0xad,0xd0, -0x84,0x94,0xbb,0xf2,0x6c,0x64,0xd7,0x8e,0x42,0x1e,0x4d,0xe8, -0x10,0x0d,0x8d,0x8b,0x99,0x3b,0x2f,0xfd,0x3d,0xa9,0xde,0x4c, -0x87,0x93,0x19,0x9b,0x5a,0x83,0x3c,0x99,0xbd,0x14,0x86,0x96, -0xa2,0x42,0xcb,0x6d,0x6e,0x33,0xb0,0x3a,0x8e,0x74,0x24,0xf3, -0xce,0x8c,0x78,0xd4,0x91,0x77,0x36,0xcc,0x48,0x50,0xef,0x0d, -0x4e,0x54,0xa5,0xf7,0xc5,0x50,0xce,0xc7,0x81,0x9d,0x2f,0x9c, -0x1e,0x4f,0x1a,0x75,0xdf,0xcf,0x27,0xb3,0x34,0x69,0x5b,0xf1, -0xed,0x47,0x07,0xe8,0x44,0xd5,0x5c,0x83,0xd8,0xb8,0xe4,0x62, -0x7a,0xb7,0x6c,0x91,0x87,0x05,0xfe,0x0b,0x95,0xd2,0xef,0xfc, -0x6e,0xd5,0x02,0xfc,0x7f,0x84,0xfe,0xf8,0xe0,0xd0,0x69,0x55, -0x97,0xcb,0xf9,0x74,0xda,0xba,0x9b,0x51,0xe4,0xe5,0x74,0x72, -0xf9,0x6b,0x6b,0x74,0x31,0xe5,0xc0,0xed,0xfc,0xae,0x1a,0x8f, -0xe6,0x1f,0x67,0x1c,0xba,0x5b,0xf0,0x2f,0xe2,0xb7,0x1c,0x82, -0xc0,0x52,0x42,0x00,0x97,0x02,0x63,0x14,0x1e,0xe0,0xe0,0x74, -0x3c,0x84,0x8c,0x97,0x37,0xc3,0xd9,0xf5,0x58,0x4c,0x42,0xb5, -0x78,0x8f,0xb7,0x60,0x05,0x10,0x5c,0xf8,0x25,0x0f,0xca,0x18, -0x00,0xf0,0xe4,0x0e,0xa4,0x25,0x12,0x48,0x70,0x2c,0xdd,0x25, -0xe6,0x2e,0xdb,0x4a,0x78,0xd7,0xa5,0xbe,0x49,0x90,0x05,0x2d, -0x63,0xc9,0xc0,0xd5,0xde,0x43,0x0f,0x58,0xd3,0x5f,0x9f,0x99, -0x49,0xc8,0x41,0x0b,0xb1,0x0c,0x37,0x5c,0xaf,0xd7,0xd7,0x63, -0x66,0x77,0x46,0x46,0xf5,0x6e,0xb0,0xdb,0xbe,0xb2,0xab,0xd9, -0x6b,0xf8,0x86,0x92,0xaf,0xef,0x56,0xd1,0xa9,0xed,0x06,0x44, -0x32,0x65,0x5d,0x3f,0x30,0x29,0x15,0x58,0xaf,0x25,0x65,0x43, -0xb7,0x1e,0x2e,0xd8,0x17,0xf3,0xcb,0x37,0x78,0xd5,0xe6,0xf2, -0x41,0x87,0x1c,0x7d,0xbc,0x03,0x7c,0xe9,0xd2,0xed,0x49,0xb4, -0x4f,0x9a,0x71,0xfc,0x92,0x16,0x2e,0xdc,0x88,0x3d,0xb8,0x1c, -0xc9,0xa7,0xc7,0xfe,0xbf,0x74,0xc9,0xd6,0x2f,0xdc,0x0f,0xfb, -0x69,0xbf,0xd7,0x1e,0x64,0xef,0xca,0xfe,0x2f,0xed,0xc1,0x37, -0x98,0x81,0xec,0x0f,0xd1,0x3d,0xd9,0xfd,0xa6,0x07,0x38,0x42, -0xeb,0x7c,0x35,0x40,0xc1,0x0b,0x3c,0x52,0xd0,0x16,0xcf,0xb2, -0xb7,0xbb,0x7f,0x7d,0x9b,0x2f,0xd1,0x01,0xc8,0xf4,0x0d,0x1a, -0x6f,0xbe,0x9c,0x4f,0xf9,0x86,0x1c,0x5e,0xc0,0x4c,0xaf,0x87, -0x8b,0x05,0xfe,0xdb,0x43,0xcd,0x0b,0xbc,0x58,0xbb,0x9d,0x3d, -0xba,0x10,0x2b,0x36,0x76,0x34,0xa5,0xab,0x76,0xfd,0x71,0x32, -0x42,0xc3,0xb2,0x05,0xd9,0x1c,0x9e,0xab,0xd4,0x1a,0x81,0xf9, -0xeb,0xd3,0xb3,0xf5,0x8f,0x4f,0x8f,0x9f,0x90,0x08,0xc7,0xc2, -0xd7,0x70,0xbe,0x7f,0xbe,0x0f,0x31,0x77,0x4b,0xaa,0xae,0x7f, -0xfe,0x11,0x20,0x0f,0x3a,0x45,0x86,0xa6,0x6e,0x20,0x09,0xdb, -0xb8,0xdf,0xfb,0x97,0x02,0x1a,0x09,0x31,0x45,0x8a,0xd6,0x8d, -0xd6,0xf0,0xdf,0x7e,0xfe,0x0e,0xd7,0xaf,0x31,0x73,0x88,0x9f, -0x39,0xed,0x27,0xb6,0x2d,0x0d,0x88,0xd3,0x0a,0xb9,0xfa,0x88, -0x8e,0xd2,0xd7,0x70,0x3a,0x65,0x6c,0x21,0xf9,0x66,0x3f,0xe9, -0xb2,0xe6,0x58,0x9a,0x7c,0x23,0xca,0x89,0x76,0xe8,0x4b,0xef, -0x07,0x05,0xe0,0xf9,0xd7,0x9b,0x20,0xcf,0x36,0x1b,0x2c,0xc3, -0x24,0x0b,0xe6,0x91,0x60,0xa0,0xe3,0xca,0xa0,0x74,0x2d,0xc7, -0xc6,0x2e,0x84,0x12,0xc7,0x83,0x5f,0xee,0x82,0x8c,0xa1,0xaf, -0x2b,0x72,0x5a,0x17,0x0e,0x80,0x31,0x04,0xb7,0x9c,0xb2,0x71, -0x9e,0xca,0x28,0xce,0x35,0xd9,0x55,0x86,0x8c,0xed,0x36,0x0d, -0xa6,0xbe,0x5a,0xd2,0x87,0x3c,0x07,0x10,0x7a,0xe1,0x36,0x13, -0x23,0x6a,0xc6,0x9c,0x1b,0x8c,0xee,0xac,0x1a,0xf3,0x8b,0xae, -0x37,0xc8,0x0c,0xd4,0x41,0x89,0x1d,0x70,0x6f,0x70,0x2d,0x8f, -0x97,0x40,0xda,0x63,0x24,0x23,0x8c,0x71,0x39,0xc8,0xc9,0x8e, -0x65,0x20,0x2d,0xc7,0x2f,0xd1,0xba,0xc4,0xb0,0x49,0x3d,0x40, -0xb8,0x99,0x39,0xc8,0xdd,0x05,0xc5,0xfd,0xcc,0x80,0xe4,0x92, -0x9e,0x96,0xd2,0x73,0xfe,0x31,0x0f,0x74,0x05,0xc7,0x00,0x21, -0xe2,0x64,0x8d,0x65,0x18,0x34,0x81,0xdf,0x57,0x93,0x37,0xaf, -0x4f,0xcf,0x12,0xd1,0x6f,0xf3,0x12,0xdd,0x35,0xed,0x1e,0x1c, -0x65,0x52,0xec,0x71,0xe2,0x5e,0xac,0xd7,0x83,0xc2,0xba,0xac, -0xe7,0x23,0x35,0xc2,0xa5,0x3d,0x42,0x15,0x11,0x37,0x3f,0x3a, -0x76,0x67,0xe2,0xc2,0x89,0xbf,0x4a,0x6f,0xc8,0xc0,0xab,0xdc, -0xc5,0xb6,0x4f,0x13,0xb6,0x2f,0xe4,0x34,0x58,0x3d,0x12,0x50, -0x8d,0x51,0x82,0x38,0x84,0x9d,0x85,0xe6,0xa8,0xd1,0x80,0x9e, -0x49,0x85,0x76,0x21,0x41,0x89,0x72,0x5e,0x6e,0xf4,0xda,0x6d, -0xd7,0xca,0xf7,0xbf,0xfd,0xfd,0xc7,0xb7,0xe8,0x02,0x73,0x75, -0x57,0x65,0xf7,0xd4,0x24,0xba,0x6d,0xdc,0x4b,0x95,0x02,0x83, -0xb5,0x48,0x79,0xbb,0x16,0xba,0x14,0xcc,0x29,0x05,0x9d,0x9e, -0xe5,0xac,0xf2,0x1e,0x52,0xe2,0x74,0x65,0xa3,0xf3,0x3e,0xa0, -0xc7,0x39,0x4c,0xa4,0x39,0x06,0x4f,0xa4,0x69,0xf2,0x49,0xee, -0xf9,0x35,0xd7,0x1d,0xc9,0x04,0xe9,0x17,0xea,0x1b,0x05,0x74, -0x3c,0xa1,0x83,0xe6,0xbe,0x60,0xbf,0x7c,0xe6,0x08,0x0f,0x0f, -0xee,0xb9,0x78,0x3a,0x85,0xf8,0x58,0xd9,0xa9,0x4f,0x8f,0x96, -0x79,0x02,0xe4,0x78,0xc4,0x2b,0xb8,0x1d,0xaf,0x6e,0xe6,0x23, -0xad,0xa5,0xcf,0x9f,0x83,0x70,0x0f,0xd2,0x1d,0xe3,0x86,0x8b, -0x5b,0xb5,0xd5,0x24,0x0d,0x66,0xce,0x74,0x0d,0xf2,0x33,0xb9, -0x2b,0xea,0xd6,0x36,0x41,0x24,0xf9,0x46,0xbf,0xae,0xf3,0xed, -0x2b,0x93,0x5b,0xe5,0xd7,0xa6,0x5b,0xa9,0x64,0x1d,0x00,0x68, -0x56,0x1a,0xc7,0x42,0xc1,0x37,0xca,0xd2,0xb0,0x33,0xa6,0xe2, -0x80,0x34,0xa4,0x5e,0xce,0x47,0xe8,0xe1,0x8b,0xbc,0xc1,0x8f, -0x57,0xc3,0x6b,0xfc,0xb5,0x68,0x57,0x41,0xf5,0x07,0x57,0x99, -0xd1,0x7d,0x9b,0x54,0x18,0x3b,0x2d,0xc2,0xab,0x86,0x51,0x78, -0x7b,0x04,0xf6,0x0f,0x07,0x99,0xd3,0x86,0x43,0x1b,0xf3,0x73, -0x6c,0x26,0x09,0xb9,0xc2,0xa7,0xe8,0xcb,0x39,0x7b,0xf4,0xbc, -0xeb,0xf0,0xc8,0x9a,0x70,0x95,0xfb,0x9f,0xf6,0x3e,0x7e,0xfc, -0xb8,0x87,0x4f,0xba,0x7b,0xd0,0x1c,0x46,0x3b,0x47,0x47,0x88, -0x9b,0x2c,0x91,0x0a,0xfd,0xe9,0xec,0xd9,0xde,0x9f,0x13,0x11, -0x90,0x45,0xa9,0x99,0x6f,0x92,0x42,0x2f,0x08,0xf6,0x83,0x43, -0x2e,0x6c,0xf6,0x17,0x88,0x0e,0x26,0x6c,0x9f,0x80,0x63,0x78, -0x6b,0xa3,0x93,0xb0,0xb0,0xbe,0xdb,0x69,0xde,0xa2,0x0c,0x9f, -0x30,0xfd,0x7d,0x45,0x32,0x7a,0x26,0x03,0xc6,0x48,0x0e,0x34, -0x0f,0x27,0x32,0xc4,0xc6,0xde,0xe8,0x3d,0xc2,0xc4,0xd2,0xfb, -0x5c,0x1d,0xd5,0xb4,0xcf,0x90,0xa8,0xf4,0xfe,0xc6,0x6d,0xb7, -0x67,0x64,0x9c,0x8d,0x8b,0x24,0x1a,0xf9,0xf7,0x97,0x2f,0x12, -0x69,0xbb,0xdd,0x85,0x5c,0x07,0x60,0x1c,0x78,0x2d,0x62,0x4f, -0xa9,0x11,0x49,0x21,0x8f,0xac,0xa7,0x74,0x1f,0xe4,0xd4,0xb9, -0x16,0x75,0x0e,0xc7,0x95,0x3f,0xb1,0x56,0x27,0x49,0x44,0xe7, -0xcb,0xbf,0x9f,0xbe,0x7e,0x25,0x89,0xd8,0xcf,0x20,0x0d,0xea, -0xdf,0xe4,0x57,0x40,0xa2,0xbd,0xe6,0x87,0x62,0x5e,0x08,0x3a, -0x49,0xe4,0x2b,0x60,0xe3,0x96,0xca,0xdd,0x22,0xb6,0x47,0x8f, -0x1c,0x68,0x5a,0x40,0x5e,0xfc,0x4c,0x22,0x7a,0xb4,0xf9,0x79, -0x2d,0x9a,0xa0,0x14,0x6b,0xc0,0xfc,0x33,0x0f,0xab,0x30,0x05, -0x1a,0x72,0x8a,0x19,0xfa,0x8c,0xdb,0xf5,0x46,0xd1,0x87,0x02, -0x90,0xf0,0xb3,0xb9,0xfb,0xac,0x5e,0x2f,0xcf,0x1c,0x2e,0x91, -0x7a,0x24,0x83,0x6f,0x7a,0x97,0xf4,0x60,0x29,0x8f,0x8c,0x70, -0xa9,0x22,0x38,0x26,0xf4,0x6d,0xfd,0x5e,0x63,0x5b,0x74,0x5d, -0xc8,0x01,0x3a,0x1b,0xe2,0xfe,0x3b,0xa5,0x0f,0x3a,0x58,0x75, -0x7e,0x2b,0x71,0x4f,0x55,0xa9,0x15,0x02,0x39,0x16,0x75,0x5b, -0xe7,0x84,0x5f,0xfb,0x8b,0xfa,0x56,0xf6,0xae,0x9c,0xcc,0x08, -0xab,0x8c,0x81,0x1f,0x3d,0x22,0x57,0xe1,0x43,0xe1,0x70,0xc3, -0x6f,0xf9,0x28,0xc7,0x77,0x3e,0xc0,0x03,0xcf,0xe0,0x07,0x6d, -0xad,0xa3,0x08,0xce,0x19,0x47,0xa5,0x36,0x29,0xf3,0x38,0x97, -0x3f,0x9d,0x5c,0xf5,0x3f,0x72,0x5b,0x79,0xc5,0x95,0xd2,0x72, -0xe4,0xcb,0xe7,0x7a,0x77,0x0c,0x47,0x9f,0x4f,0xa9,0x46,0x6e, -0x1f,0x60,0xed,0xdf,0x15,0x07,0xbe,0xb7,0xf4,0x92,0xa9,0x37, -0x25,0x8c,0xe1,0x8f,0x24,0x5a,0xfd,0x56,0x93,0x53,0xb9,0x6a, -0x7c,0x01,0x54,0x51,0x67,0x48,0xe5,0xa3,0x03,0xb8,0xc0,0xbf, -0x3d,0x38,0x78,0xcc,0x11,0xeb,0xf5,0xb7,0x07,0xe8,0x36,0x8c, -0xbf,0x7a,0x69,0xd5,0x9d,0x5c,0xe9,0xc9,0x86,0x7e,0xf5,0x25, -0x58,0x72,0xcb,0x60,0x89,0xbc,0x0d,0x3a,0x91,0x26,0x2f,0xe0, -0x28,0xdc,0xd3,0x12,0x80,0x2a,0xdc,0xfa,0xc2,0xb2,0xce,0xec, -0x61,0xd9,0x27,0x51,0xef,0x9f,0xde,0xbe,0x18,0x94,0x9a,0xd1, -0x17,0xd9,0x5e,0x09,0x9e,0xaf,0x8d,0xb0,0x31,0xa1,0x09,0x66, -0x96,0x3f,0x0a,0xba,0xe5,0x56,0x44,0x09,0xfb,0xd0,0x4c,0x7a, -0x32,0x9b,0xcb,0xc1,0x93,0x64,0xc5,0xb7,0x5f,0x5b,0x64,0x75, -0xeb,0xfa,0x5b,0x98,0x7c,0x7c,0x55,0xd3,0x11,0x93,0x56,0x6e, -0xf0,0x33,0x5b,0xd8,0x65,0xee,0xd2,0x8a,0xd2,0xf5,0x69,0xe2, -0xe9,0x36,0xa2,0xe5,0x6a,0x22,0x79,0xf9,0x9a,0x16,0x51,0x44, -0x06,0xb5,0x73,0x4e,0x5f,0x43,0x9e,0xea,0xbc,0xee,0xf8,0xc8, -0x4c,0x56,0xb0,0xf6,0x60,0xcc,0x08,0x84,0xae,0x01,0x97,0x5a, -0x1e,0xa0,0x32,0x3c,0xcf,0x82,0xc4,0x58,0xd4,0xa5,0x6b,0x60, -0xa4,0xf1,0x76,0x21,0xcf,0xc3,0x5a,0x5f,0x27,0x49,0x7c,0x6b, -0x7b,0xb0,0xfe,0xa1,0xc6,0xf1,0x08,0x91,0xa2,0xf9,0xf4,0x03, -0x5b,0x2b,0xd3,0x8b,0x56,0xfd,0xad,0xf4,0x75,0xab,0x9a,0xbe, -0x08,0xc2,0x54,0x18,0x00,0x88,0xad,0x36,0x97,0xb7,0x38,0x1a, -0x95,0xa6,0x5e,0x0e,0xc2,0xfe,0x9c,0xa0,0x46,0x8f,0x0f,0xea, -0xdc,0x60,0xd8,0x6c,0xd4,0xab,0xc9,0x72,0xcc,0x6a,0x2c,0x30, -0x36,0xaa,0x6b,0x0e,0x6b,0xc4,0xb9,0x75,0x10,0x12,0xd9,0x77, -0x31,0xc0,0xc5,0x0a,0x8b,0xa5,0x69,0xbb,0xcc,0x78,0x28,0xae, -0xa1,0x0d,0x54,0xcc,0xf3,0x89,0x76,0x13,0xeb,0xff,0xba,0x4e, -0x0e,0xb2,0xb0,0xb1,0xe9,0x03,0xad,0x8d,0x50,0x49,0x85,0x05, -0x20,0xf6,0xf6,0x9c,0x80,0x39,0x4e,0xaa,0x7b,0x75,0x60,0x65, -0x8f,0xa0,0x3c,0x61,0xa5,0xb0,0x4a,0x36,0x35,0xca,0x81,0x08, -0xa8,0x54,0x0e,0x70,0xa4,0x57,0x98,0x6e,0xf1,0xb8,0x99,0x26, -0xc9,0x2f,0x5a,0xbb,0x39,0x12,0x4b,0xfb,0xa8,0x60,0x9d,0xeb, -0x36,0x6e,0x3e,0x27,0x83,0xa3,0xd7,0x76,0xda,0x1f,0xb5,0x71, -0xc1,0xbc,0x8a,0x78,0x64,0x77,0x8b,0x14,0x90,0x34,0xbd,0x63, -0xf2,0x68,0x74,0x4b,0xe7,0x64,0x03,0xd6,0x72,0x5e,0x1f,0x48, -0x9f,0x4e,0xbe,0xbf,0x83,0xb2,0x46,0x7f,0x39,0x4e,0x61,0xde, -0x9a,0x7b,0xd3,0x8a,0x92,0x9d,0x50,0x3b,0x8d,0x75,0xae,0x2b, -0x5d,0x1b,0xae,0x4b,0x22,0xad,0xaf,0x12,0xcd,0x72,0x22,0x00, -0xab,0x34,0x41,0x26,0x49,0xeb,0x76,0x7c,0x3b,0x5f,0x7e,0x4e, -0x82,0xa5,0x5d,0x99,0xe5,0x4f,0x56,0x86,0x96,0x63,0x68,0x55, -0xb5,0x92,0x81,0x2a,0x6b,0x31,0x6c,0x03,0xfa,0x7e,0x93,0xf3, -0xc5,0x87,0x07,0xe0,0xf2,0xf8,0x02,0xaf,0xb3,0xe4,0x72,0x08, -0xb5,0x20,0x73,0x9b,0x37,0x56,0x79,0xef,0xaf,0x2b,0xb8,0xa3, -0x6a,0x87,0x77,0x61,0x59,0xac,0x7c,0x3d,0x93,0x65,0x6b,0xe4, -0x73,0x3e,0x92,0xe3,0x76,0x4c,0x24,0xf9,0x4e,0x34,0x7d,0x19, -0xfb,0x17,0x0a,0xe2,0x50,0xf6,0x8f,0x8a,0x97,0xca,0xcc,0x61, -0xf6,0x40,0xe3,0x92,0xc9,0x8e,0xb2,0x28,0xbe,0x4f,0x65,0xeb, -0x46,0x9c,0xd8,0x4f,0x59,0xff,0xd1,0x40,0x81,0x47,0xc5,0xa0, -0xe9,0x51,0x89,0x4d,0xc0,0x2e,0xa7,0x52,0xcc,0x30,0xa7,0xe0, -0x46,0x2c,0x8d,0x84,0x43,0x51,0x35,0x3c,0x72,0xb8,0x21,0xe8, -0x35,0x76,0x41,0x9c,0x48,0x54,0x38,0xaa,0x66,0x7e,0x9a,0x95, -0x45,0x48,0xe4,0x8a,0x04,0x5f,0x67,0xdb,0x3c,0x62,0xb2,0x22, -0x02,0xba,0x6c,0xc7,0x6c,0x0d,0x73,0xde,0x9f,0x32,0x47,0x71, -0x7b,0xd2,0x7a,0x4d,0x75,0x86,0x19,0xac,0x98,0xaf,0x92,0xb7, -0xc8,0x2c,0x44,0x4f,0xc8,0x2f,0x61,0xc7,0x12,0x61,0xe2,0x11, -0x5d,0x4b,0xfe,0x69,0x8b,0xaa,0xee,0xad,0x64,0xa4,0x13,0x45, -0xa1,0xf8,0x85,0xeb,0xcb,0xdf,0x0e,0xc5,0xe5,0x1d,0x52,0x33, -0xb8,0x90,0x30,0x02,0xd7,0xd3,0x63,0x5e,0x4d,0xe4,0x18,0x07, -0x92,0x90,0xe3,0x8c,0x49,0x1e,0x46,0x1f,0xa3,0x07,0x65,0x3f, -0x8e,0x41,0x2d,0x19,0x0e,0x89,0x0f,0x7e,0xbe,0x34,0x86,0xd3, -0x8f,0xc3,0xcf,0x15,0x82,0xef,0xdb,0x5b,0x64,0x10,0x99,0xed, -0x1b,0xe2,0xce,0x08,0xd5,0x41,0xe4,0x22,0xbc,0x67,0x1b,0x4c, -0xb3,0xe1,0x94,0x6e,0xcd,0x2a,0xb8,0x2e,0x79,0x47,0x39,0x50, -0x7a,0x8c,0xb5,0xdb,0x2e,0xd8,0x25,0xc8,0xa9,0x03,0x90,0xe5, -0x84,0x0e,0xa3,0x16,0x85,0x8b,0x61,0xc3,0x24,0x64,0x89,0x40, -0xef,0x0f,0xa0,0x12,0x6f,0x27,0x30,0xeb,0xd4,0x66,0xcf,0xd9, -0x28,0x6b,0x17,0x0d,0x20,0xea,0x7a,0x3d,0x0a,0x66,0xc1,0x5f, -0x58,0x8b,0x24,0x30,0x82,0xc1,0xe1,0xab,0x21,0x3a,0x55,0xe8, -0xe2,0xb9,0x9e,0x22,0xba,0x0e,0x9d,0xe8,0xd2,0x8f,0xa5,0x72, -0xf1,0xf2,0x37,0x5c,0x7a,0x64,0xca,0xe6,0x41,0x8c,0x72,0x3d, -0xf3,0x88,0xce,0xed,0x24,0xfb,0xfb,0x78,0x6c,0xd1,0x43,0x9b, -0xde,0x12,0x5d,0xa6,0xdb,0xd7,0x6b,0xfd,0x66,0x76,0x80,0x8f, -0xe7,0x98,0x9c,0x11,0x27,0x66,0x6c,0x5a,0x71,0x16,0x1f,0x0f, -0x08,0xf6,0x37,0x49,0xe4,0x30,0x9b,0xbd,0x08,0xa4,0xa4,0x84, -0xb6,0x04,0xbc,0xee,0x23,0xea,0x6b,0xb3,0x1e,0xfb,0x20,0x17, -0x87,0x34,0xdd,0xcb,0xe5,0x1c,0x28,0xee,0xf9,0x2d,0x50,0xc0, -0x70,0xf0,0x2f,0x22,0xc6,0x24,0x8d,0x40,0xc4,0x91,0xcc,0x83, -0x42,0xe5,0x4e,0xba,0x43,0xa5,0xf4,0x25,0xf9,0x70,0x00,0x9b, -0x3e,0xea,0x7c,0xbb,0xcd,0x69,0x8f,0x6a,0x69,0x8f,0x06,0x5a, -0x6b,0xff,0x5b,0x14,0x5c,0x4f,0x6e,0x56,0xab,0x45,0x81,0xc2, -0x03,0x0a,0xad,0xf7,0xe7,0x83,0xe2,0xbb,0xef,0xbe,0x45,0x21, -0xfc,0x90,0x77,0x10,0xe5,0x8f,0xaa,0x74,0xc5,0xa8,0xc1,0x23, -0x32,0x80,0x49,0x2e,0x0b,0x95,0xc3,0xe0,0xfd,0x5b,0x78,0x77, -0x55,0x92,0x4f,0xc6,0xb5,0x0c,0x5e,0x7a,0x05,0x79,0x0d,0x1f, -0x9c,0x32,0x34,0x33,0xb3,0x00,0x4c,0xe1,0x8b,0x54,0x24,0xe0, -0x48,0x2a,0x01,0xce,0x0b,0x37,0xf7,0xf7,0x83,0xb2,0x8a,0x30, -0xfe,0xc8,0x5c,0xfe,0x30,0x3d,0x7c,0x51,0x87,0x68,0xd0,0x81, -0x17,0x6a,0x62,0x94,0xa6,0xd3,0xf1,0xd2,0x20,0x4d,0x48,0x0d, -0xb2,0xdd,0xdc,0xd2,0xe3,0x1f,0x98,0xd3,0x9f,0x16,0x0b,0x67, -0xc0,0xa0,0x42,0xef,0xda,0xca,0xca,0xdf,0xf1,0x6c,0x7d,0xf1, -0xf1,0xc5,0x3e,0x79,0x1d,0x0e,0x53,0xd2,0xb2,0x08,0x0a,0xd1, -0x11,0x27,0xa3,0x17,0x66,0xeb,0xf0,0xb4,0xc9,0x63,0x86,0xc8, -0xc2,0x48,0x8e,0xac,0x97,0xb4,0x01,0xa1,0xec,0xa1,0x27,0x3b, -0x1e,0x5e,0x51,0x8a,0xe5,0x2f,0x5a,0x6a,0xa4,0x98,0x8c,0x8e, -0xb9,0x69,0x5e,0x70,0x77,0x2e,0xd5,0xa5,0xb2,0x07,0xa3,0x21, -0xbf,0x0f,0x01,0xe9,0x4a,0x76,0x0f,0xdf,0x95,0x49,0xc7,0x3f, -0xb1,0x74,0x3a,0x59,0xa1,0x39,0x3b,0x5f,0xd5,0xaa,0xa4,0x06, -0x00,0xdb,0x14,0x10,0x91,0x0f,0xd2,0x81,0x30,0x2f,0x7c,0x00, -0x45,0x57,0x5c,0x9a,0x3c,0xbf,0x72,0x74,0xe5,0xde,0xe9,0x04, -0x60,0x27,0xf9,0x83,0x90,0x3c,0xbb,0x2e,0x20,0x0a,0x1f,0xac, -0xe0,0x15,0x1c,0x74,0x7b,0xe4,0xf8,0x33,0x69,0x2e,0x0d,0xbd, -0x49,0xfd,0xe6,0xf0,0xb3,0x89,0x5f,0x86,0xe9,0xb6,0x83,0xc3, -0xef,0x0f,0x29,0x93,0x92,0x6d,0xaf,0x5f,0x40,0xed,0x61,0xb6, -0x24,0x0f,0x4b,0xe5,0xdb,0x0a,0x1d,0x13,0xab,0x2e,0xb1,0x07, -0x1e,0x59,0x74,0xaf,0x44,0x29,0x1c,0x5f,0xa0,0x6d,0xca,0xa0, -0xb7,0x35,0xa5,0x83,0xcf,0x3e,0xc8,0xe4,0x08,0xa2,0x7b,0x49, -0xde,0x82,0x19,0x15,0x26,0x60,0x27,0x39,0x6a,0xfd,0x56,0x1e, -0x74,0x0f,0x48,0x3c,0x32,0x2b,0x3c,0x30,0x28,0x3b,0xf0,0xbe, -0x6e,0x61,0x3d,0x2a,0x0b,0x66,0x4b,0xcb,0x27,0xb9,0xcb,0xa2, -0xa6,0x73,0xaa,0x2e,0x1b,0xc3,0x42,0xb6,0x35,0xad,0x5d,0xff, -0xc9,0x96,0xd8,0x62,0xfa,0x47,0x48,0x96,0xac,0xe4,0xe1,0xf6, -0xe7,0x43,0x70,0x40,0xc8,0xad,0x99,0x1d,0x79,0xa4,0x95,0x62, -0x12,0xeb,0x99,0x97,0x49,0xb0,0x43,0xa6,0x12,0xe1,0x57,0xef, -0xc6,0xe2,0x70,0xc3,0x3d,0x80,0x46,0xa6,0xae,0xa5,0x9e,0x9d, -0xf3,0xf0,0x69,0xe6,0xb9,0x5b,0xb5,0xd3,0x2c,0xbb,0xaf,0x71, -0x77,0x0e,0xbf,0x9a,0xd2,0xf4,0xdc,0x7d,0x4b,0xb7,0xc1,0x64, -0x20,0x03,0x18,0x27,0x5f,0xa8,0x23,0x7c,0x9d,0x09,0x38,0x51, -0x28,0x67,0xa4,0x1c,0x2a,0x2b,0xf6,0x68,0xc6,0x29,0x91,0xfc, -0xf8,0x94,0xee,0x00,0x65,0xfc,0x1c,0x58,0x49,0x3b,0x3d,0x46, -0x52,0x21,0x07,0x31,0x44,0xfe,0x08,0x29,0xc9,0xfc,0xfb,0x20, -0x99,0x7f,0x55,0x5c,0x2c,0x23,0xfb,0x6d,0xad,0xf1,0x11,0x61, -0x2e,0x7b,0x30,0xe0,0xd9,0x86,0x4d,0x6f,0x68,0x44,0xf2,0x6a, -0xde,0x72,0x43,0xe8,0x5f,0xc8,0xa8,0x89,0x84,0x48,0xb3,0x89, -0xb6,0x90,0x51,0xe8,0x1f,0xf3,0xc2,0x57,0x04,0x14,0x09,0x21, -0xda,0xd3,0x11,0xf3,0xee,0xe5,0xc0,0xd9,0xab,0x20,0xa0,0xc8, -0xc1,0x2d,0xb6,0x3f,0x51,0x6c,0x83,0x1b,0xbe,0x63,0x24,0xc4, -0x19,0xb6,0x4f,0x05,0x86,0xd2,0x74,0x4c,0x75,0xae,0x57,0xf8, -0xe6,0x9e,0xe9,0x9d,0xb7,0x02,0xbe,0xf8,0x96,0x78,0x74,0x9f, -0xd2,0x14,0xff,0x69,0xcf,0xa7,0x04,0xec,0x73,0xa9,0xad,0xd1, -0xff,0x4a,0xc4,0x02,0x27,0xfe,0xb5,0x80,0x30,0xd8,0xb9,0x3c, -0xc5,0x05,0xdd,0x37,0x46,0x3d,0x18,0xeb,0x84,0xbf,0x9b,0xa8, -0xdf,0x6e,0x4f,0xa4,0xce,0xa0,0x9f,0xc7,0x89,0xb3,0x7b,0x7f, -0x53,0x59,0x89,0x16,0x8d,0xdd,0x39,0x8c,0xd0,0x26,0x4a,0xe3, -0x57,0x48,0x7a,0x31,0xd1,0x9b,0x1e,0x73,0x86,0xf5,0xba,0xc5, -0xd3,0x5c,0x2f,0x1d,0x34,0x06,0xb0,0x88,0x72,0xf3,0x98,0xe2, -0x89,0xe4,0xdf,0xb0,0xf1,0xcb,0x59,0xd8,0x4f,0xf0,0x2b,0xc9, -0xc8,0x8e,0xf2,0x36,0x6d,0x1a,0x59,0xac,0xf7,0x55,0x60,0xc5, -0xfe,0x9d,0x59,0x46,0x5c,0xcf,0xd6,0x67,0x72,0x67,0x43,0x85, -0x03,0xbc,0xa1,0x89,0xe3,0xd8,0xe5,0x98,0x13,0x7e,0xa9,0xc1, -0xd1,0xe0,0x1c,0xfa,0x74,0x13,0x65,0x70,0x10,0xaa,0xe5,0xa5, -0x62,0x1f,0x1c,0x31,0x27,0x11,0x97,0xd2,0x7d,0xd1,0x01,0x44, -0x9b,0x93,0x25,0x54,0x4a,0xd3,0xee,0x49,0x45,0x07,0x66,0x76, -0x9f,0x4a,0x08,0xf9,0x88,0x5c,0xd0,0x9f,0x5b,0xeb,0xf5,0x3e, -0x42,0x1c,0x8f,0xd6,0x7a,0x68,0xee,0x0b,0x16,0x14,0xe7,0x24, -0x55,0xdb,0xaf,0x6d,0x06,0x5b,0x72,0xe1,0x64,0x6b,0x9f,0xbd, -0x16,0x15,0x58,0xdf,0xa9,0xc4,0xfc,0xa5,0x8c,0x33,0x01,0x71, -0x6d,0xd7,0x69,0x48,0x1f,0x1d,0x1c,0xc0,0xfe,0x17,0xfe,0x1c, -0x4a,0x1f,0xe2,0xdc,0x86,0xe6,0x7b,0xcd,0x82,0x08,0x3d,0xe1, -0x45,0xd4,0x9d,0x4e,0xa9,0x6b,0x18,0x77,0x2c,0xf5,0x27,0xce, -0x0e,0x69,0x4a,0x88,0x38,0xe7,0x7c,0x3a,0x72,0x1c,0x1a,0x94, -0x3a,0x5b,0xe2,0x91,0xb1,0x28,0xf7,0xd3,0x32,0x3b,0xef,0xa5, -0xbd,0xb2,0xbd,0xde,0xcd,0xd6,0xe7,0xbd,0xf3,0xde,0xfe,0x51, -0xfd,0x08,0xa1,0xbc,0x45,0xa2,0xdd,0xe0,0xf7,0xb4,0x85,0xc2, -0x2b,0x22,0x51,0x45,0xf7,0x34,0x6a,0xeb,0xec,0x2e,0xe6,0x8b, -0x34,0xf3,0x2c,0x3d,0x36,0xad,0x02,0x68,0x5b,0x88,0xb5,0x59, -0x82,0xb6,0xaf,0x80,0x06,0xb8,0x10,0xdd,0x8b,0xe8,0xd6,0xcd, -0x8e,0x8d,0xa2,0x37,0xb2,0x85,0xdd,0x78,0xce,0x48,0xa5,0x7b, -0x63,0xe2,0x7b,0x30,0x68,0xea,0x2b,0xd2,0xe9,0x84,0x13,0xe9, -0xe3,0x12,0x05,0x20,0x67,0x8e,0x85,0x77,0xc2,0x6a,0x7f,0xe3, -0x25,0x75,0x19,0xd5,0x0c,0x61,0x51,0x53,0x1d,0x3b,0x82,0xe9, -0xf2,0x40,0x2a,0xfa,0x0d,0xeb,0x1d,0x90,0x52,0xf8,0x4b,0xea, -0xfc,0x91,0x53,0x3e,0xc1,0xdf,0x76,0xd2,0x00,0xc7,0x62,0x6b, -0xa0,0x4e,0x38,0xe3,0x8b,0xaf,0xa6,0x24,0xcd,0x19,0x54,0x2a, -0xef,0xd4,0x09,0xfe,0x9a,0x8b,0x4b,0x1a,0x0c,0xf0,0x79,0x50, -0xca,0x1a,0x8a,0x95,0xda,0xfe,0x6b,0xc7,0x74,0xca,0x1a,0x44, -0x0f,0xa3,0x1c,0x59,0x2f,0x8a,0x48,0x11,0x31,0x0b,0x62,0xdc, -0xb0,0xf5,0xaa,0xbe,0x06,0x07,0xa5,0x09,0x7b,0x3a,0x80,0xca, -0x21,0x29,0x90,0x74,0x6c,0xb3,0x1c,0xc8,0x1d,0x4b,0x5a,0x34, -0xd1,0x2a,0x3a,0xfa,0x9e,0x50,0xa1,0x72,0x24,0xe8,0x18,0x40, -0x64,0x1c,0x57,0xee,0x9f,0xbe,0x9c,0x7d,0xfc,0xba,0x6a,0xc4, -0x05,0xbc,0x52,0x50,0xbc,0x16,0xfc,0x2a,0x46,0xa4,0x2d,0x18, -0xc4,0x4e,0xd2,0xfa,0x38,0xac,0xd0,0x11,0x21,0xad,0x2d,0x7a, -0x8a,0xaa,0x95,0x87,0xa1,0xdf,0x44,0x88,0x73,0xc9,0x57,0xb8, -0x5d,0x83,0x25,0xbf,0x08,0xf7,0x2d,0xf8,0x41,0xde,0x14,0x19, -0xb6,0x39,0xaa,0xcc,0xcb,0xb0,0x6c,0xf2,0x80,0x9b,0x64,0x0a, -0x35,0x02,0xb5,0xdb,0xa1,0x0a,0xd3,0x68,0x16,0xc2,0xc5,0x12, -0xef,0xb2,0x68,0x19,0x84,0x47,0x01,0x99,0x39,0xb3,0x33,0xd2, -0x30,0x48,0x8e,0x50,0x36,0xeb,0xcf,0x34,0x09,0xd6,0xbb,0xf9, -0x4a,0x9b,0xc6,0xb8,0x01,0xa8,0xed,0x94,0xbf,0xfe,0x37,0xc6, -0x88,0x98,0xaa,0x71,0xf0,0xa9,0xf9,0xe9,0x66,0xe9,0x9a,0x9d, -0xc3,0xc7,0x29,0xeb,0x53,0xc0,0xe1,0x0a,0x1f,0xcf,0x47,0xe5, -0x01,0xfe,0xbe,0x9e,0xfd,0x44,0x47,0x2f,0xe3,0xf8,0xf2,0x8e, -0x7f,0x4c,0xf4,0xfe,0xdf,0x59,0x74,0xd5,0x88,0xf0,0xf0,0xb9, -0xf3,0xeb,0xf8,0x33,0x11,0x01,0xf0,0x8b,0x44,0x8b,0xad,0x26, -0xb3,0x1f,0xc8,0xf1,0x1d,0x44,0x47,0xfa,0x51,0x93,0x6c,0x2d, -0x14,0x6a,0xac,0xb9,0x57,0x5b,0xce,0x3b,0xec,0x6c,0x86,0xe5, -0x3e,0xda,0x6d,0xb1,0x1c,0xb9,0x82,0xd3,0x78,0xb8,0x1c,0xc1, -0xf2,0xc0,0x43,0x9a,0x23,0x05,0x10,0x46,0x6d,0x8a,0x5a,0xbe, -0x60,0x38,0x9a,0x84,0x80,0xb1,0x4d,0x69,0xdd,0x5c,0xc4,0x7c, -0x59,0x95,0x3b,0x3b,0xb6,0x30,0x9c,0x5c,0x1f,0x27,0xab,0x9b, -0x93,0xe5,0x78,0x04,0xcb,0x74,0x02,0x34,0x48,0xc2,0x63,0x12, -0x8c,0x76,0xad,0x2e,0x67,0xe3,0x0e,0xea,0x8c,0x20,0xe6,0x21, -0xf8,0x46,0x1c,0x2d,0x42,0xcd,0x76,0x02,0xdc,0xac,0x6e,0xfc, -0x7d,0xae,0x8f,0xfa,0xba,0x6e,0x9b,0x51,0x2e,0xa1,0x2c,0x1d, -0x19,0xc7,0x65,0xd8,0xe6,0x58,0x3e,0xc1,0x76,0x95,0x32,0x30, -0x4c,0x78,0xde,0x55,0x68,0xe9,0xeb,0x76,0xdc,0x83,0xb8,0xee, -0x7c,0x31,0x9e,0xa5,0x8e,0x91,0xc8,0x8c,0x1b,0x42,0xc4,0x72, -0x9f,0x31,0x47,0xf7,0x00,0x55,0xf5,0x71,0xbe,0x1c,0x65,0xc5, -0x83,0x85,0xf0,0x98,0x83,0x0c,0x2c,0x9b,0x92,0x19,0x02,0xd9, -0x47,0x42,0x08,0x28,0xcb,0xd2,0x44,0xa1,0xff,0x76,0xcf,0x01, -0x6f,0xb7,0xa9,0x86,0x88,0x81,0xde,0x1c,0x6b,0x18,0xe7,0x11, -0x06,0xbd,0x5e,0x2b,0xbd,0x9d,0xfc,0x7d,0x4f,0x88,0xf1,0xf1, -0x68,0x8f,0x2c,0x4a,0x23,0x67,0xf0,0x81,0xd4,0x32,0xf9,0xfb, -0xcb,0x17,0x3f,0xae,0x56,0x0b,0x49,0x30,0xae,0x2b,0xa8,0x33, -0x4a,0xeb,0x63,0x83,0x1a,0x28,0x7d,0x43,0xe7,0x2b,0x59,0xb8, -0x5c,0x66,0xf7,0x1b,0xce,0x8e,0x62,0x78,0x11,0x2b,0x85,0x1c, -0x89,0xae,0xc9,0xa4,0x81,0x97,0xf0,0x6a,0xc2,0x4b,0x59,0x0b, -0x93,0xde,0xa4,0x6b,0xaf,0x7d,0x0d,0x72,0x21,0xd4,0x6a,0xb4, -0x69,0xe4,0x64,0xfa,0x3c,0x62,0x8b,0x8f,0xfd,0xd8,0x20,0x8b, -0xae,0x9a,0xac,0xe6,0x3d,0x98,0x97,0x11,0x94,0xa5,0xf1,0xaf, -0x63,0xae,0x4e,0x2e,0x79,0xbe,0xa8,0x9d,0x4b,0xed,0xb6,0x30, -0xec,0x82,0xe3,0x85,0x21,0xc2,0x49,0xa9,0x1d,0xfb,0x6e,0x27, -0x6e,0x0c,0xcf,0xb6,0xb2,0x32,0x90,0x62,0x76,0x77,0x8c,0x7b, -0x99,0xbb,0xab,0xa8,0x50,0xf3,0x80,0x50,0x52,0xe3,0x3b,0x54, -0x9a,0xe5,0x35,0x24,0x89,0x2b,0xf7,0xe2,0x52,0x46,0xc6,0xa4, -0x22,0xe3,0x9b,0x65,0x9c,0xc3,0x73,0x08,0x44,0x26,0xc0,0x37, -0x05,0x3f,0x3d,0x3f,0xc0,0x8a,0x1e,0x24,0x1b,0x2f,0x9f,0xa0, -0x87,0x21,0x32,0xf1,0xcd,0xb2,0xed,0x1d,0x3e,0x7a,0xf4,0xad, -0x13,0xc4,0xf0,0xc2,0x09,0x8f,0x0e,0xbe,0x03,0x9c,0x84,0xc3, -0x61,0xd3,0x7a,0x80,0xde,0x17,0xdf,0x1d,0x7c,0xb7,0x91,0x3a, -0x91,0xab,0x72,0x35,0xff,0x74,0x4c,0xe8,0xfe,0xd3,0x4f,0x48, -0x86,0xe3,0xc3,0xc4,0xbd,0xa7,0x0b,0x54,0xd6,0x73,0x8f,0x59, -0x30,0x0d,0x99,0x37,0x46,0xda,0xc6,0x65,0x97,0x81,0x6e,0x5a, -0x69,0xf1,0xe8,0x13,0x33,0x85,0xce,0x83,0x5e,0x7d,0xa1,0xf5, -0x0c,0x57,0xc6,0x91,0x8a,0x45,0xca,0xab,0xa2,0xec,0x74,0xe8, -0x92,0x6b,0x58,0x4a,0xa9,0x5d,0x44,0xb0,0x83,0xed,0x27,0xae, -0x09,0xa1,0x5c,0xf9,0x42,0xca,0xba,0xac,0xdc,0x90,0xc6,0x70, -0xe0,0xd6,0x6f,0x5a,0x8c,0xce,0x04,0x0a,0x25,0x37,0xad,0x73, -0xdf,0x54,0x47,0x58,0xa1,0x4b,0xf4,0x1a,0x81,0xe4,0x77,0x9b, -0xcb,0xb7,0x85,0x3a,0xba,0xfa,0xf4,0x6a,0xfe,0x91,0x9e,0xf6, -0xb1,0xc3,0xcb,0xab,0x4f,0x2b,0xaf,0xc8,0xc6,0x56,0xd6,0xd6, -0x68,0xef,0x6c,0x8d,0x06,0xce,0x48,0x64,0xfe,0xea,0xd3,0xac, -0x66,0x4c,0xb0,0x57,0x88,0x3d,0xc1,0x75,0x16,0x5b,0x24,0x64, -0x63,0x84,0xc6,0x90,0xe1,0xf2,0x6e,0x56,0xee,0xc3,0x61,0x75, -0x37,0x26,0xcd,0x3c,0x00,0x39,0x9c,0x4d,0x6e,0x09,0xeb,0xf7, -0x2c,0xbd,0xb2,0x2f,0x26,0x89,0x5d,0xd4,0x20,0x5f,0x7d,0x1c, -0x8f,0x67,0xf4,0x02,0x8d,0x32,0x92,0x7d,0xa3,0xf3,0x08,0x9b, -0xde,0xfa,0x5f,0x9a,0x8d,0xf2,0xbb,0xd9,0x64,0xc5,0x05,0x58, -0x73,0x4c,0x2c,0xca,0x62,0x84,0xcd,0x2f,0x1a,0x8d,0xdc,0x27, -0x6b,0xa0,0x4a,0xa4,0xf1,0x4a,0x02,0x81,0x1a,0xe5,0x29,0xbd, -0xe6,0x03,0x92,0xd3,0xe1,0x94,0xf5,0xfa,0x00,0x08,0xde,0xe1, -0x14,0x59,0x73,0xb7,0xc3,0x4f,0xcf,0xa1,0x85,0x43,0x16,0xae, -0x78,0x44,0x56,0xfe,0x08,0x30,0x1b,0x29,0x9b,0x8d,0xca,0x8e, -0x3e,0x2d,0x51,0xc3,0x4a,0xf3,0x96,0x54,0x33,0x3e,0x45,0x0a, -0xb6,0xa8,0xce,0xc4,0x26,0xa7,0xf0,0x2f,0x19,0xe2,0x9a,0x90, -0x45,0x8d,0x21,0x9e,0xc0,0xdc,0x10,0xa3,0x97,0xca,0xad,0x74, -0x6a,0xbd,0x38,0xc1,0xeb,0x35,0xd4,0xbb,0x5e,0x1f,0x1e,0x8d, -0xe6,0x2d,0x6e,0x27,0xfd,0x05,0x8a,0xa8,0xfb,0xc7,0x84,0xbb, -0xb2,0xcf,0x51,0x79,0x60,0xb7,0x25,0x06,0x45,0x39,0x3b,0x58, -0x7d,0x76,0xf4,0x11,0xc8,0x71,0xa4,0xd2,0xa1,0x10,0x34,0x89, -0x03,0x76,0x84,0xf6,0x45,0x84,0xb1,0xdd,0x46,0xa3,0xf2,0x94, -0xdc,0x6e,0xef,0xed,0x05,0xc3,0x93,0x6d,0xb8,0x00,0x8d,0x83, -0x9f,0xa5,0x2e,0xf7,0x89,0xfe,0x4a,0x0c,0x8e,0x9b,0x7b,0x5f, -0xe3,0x66,0xa4,0xfa,0x8d,0xe6,0xc2,0x10,0xf9,0x80,0x7f,0xce, -0x3b,0x04,0x16,0xda,0x0c,0x1c,0x8e,0x78,0xac,0x8b,0xaa,0x0c, -0x45,0x88,0x5d,0x7c,0x7e,0x2f,0x0b,0xaa,0x08,0x56,0x92,0xd5, -0x52,0x68,0x10,0xf0,0xc7,0x1c,0x81,0x7c,0x3f,0x15,0xa1,0xbf, -0x25,0x33,0xe4,0x0b,0xfe,0xa0,0xbf,0x46,0x85,0xe8,0x48,0x95, -0x6a,0x69,0x58,0x89,0xe6,0x45,0x6f,0x3f,0xec,0x8a,0x8d,0x33, -0x7b,0xfb,0x8e,0x28,0xe5,0x4f,0x59,0x8e,0xe8,0x6f,0xa7,0x93, -0x61,0x0e,0xce,0xd6,0xa7,0x28,0xbf,0x1d,0x44,0x25,0x3c,0xfc, -0x24,0x8d,0x8c,0x30,0x0a,0x86,0xbc,0xba,0x99,0x5c,0x99,0xe3, -0x6e,0xe3,0xb5,0x54,0xfc,0x18,0x38,0xce,0xaa,0x38,0xcf,0xc8, -0xee,0x25,0xd0,0x6b,0xd8,0xa6,0x75,0x98,0x45,0x53,0xae,0x80, -0xea,0xb1,0xbc,0x5a,0xda,0x90,0x25,0xfd,0xcd,0xe9,0x6f,0x97, -0x9e,0xa6,0x89,0xe7,0x78,0x0f,0x84,0x2a,0x9a,0x90,0x45,0x5d, -0x81,0x82,0xb3,0x4c,0x60,0xbd,0x6c,0xb1,0x58,0x45,0xa3,0x8a, -0x3b,0x7f,0x3c,0xac,0x50,0x56,0x89,0x56,0x2c,0x7b,0x5c,0x74, -0x46,0x7d,0xc5,0xe3,0xbe,0x8e,0x64,0x2e,0x56,0x20,0x31,0x7f, -0xc9,0x3f,0xe8,0x11,0xe9,0x23,0x59,0xdf,0x60,0x59,0xfc,0x50, -0x5e,0x8a,0x23,0x79,0xa5,0x8a,0x0a,0xe9,0x47,0x39,0x5e,0xe8, -0x80,0x60,0x78,0xb0,0x6c,0xb1,0x1d,0xf4,0xe1,0x56,0xf9,0x97, -0x77,0x3a,0x19,0x1c,0x88,0xf9,0x49,0x6c,0xe3,0xcc,0x8d,0xcc, -0x82,0x75,0x99,0x5d,0x37,0x9c,0x3e,0xb4,0x58,0x31,0x73,0x06, -0xcc,0x7a,0xde,0x94,0x19,0xfb,0x22,0x8e,0x40,0x38,0x8b,0xef, -0x3e,0xcb,0x26,0x87,0x83,0xd9,0x6c,0x85,0xf1,0xf2,0x12,0x30, -0x43,0x39,0x51,0x87,0x15,0xe2,0x60,0x5f,0xd9,0x18,0x8e,0x99, -0x57,0x25,0x15,0x2b,0xed,0x48,0x76,0x47,0x77,0x7c,0x0e,0x38, -0x63,0xc6,0x34,0xec,0x7d,0x33,0x13,0x03,0xad,0x3a,0x6f,0x2c, -0xf8,0x8d,0xa6,0x1e,0xe4,0x87,0xcd,0x39,0x60,0x03,0x5a,0x00, -0x38,0x47,0xa9,0xce,0xcc,0x9e,0x9f,0x41,0x38,0x40,0xb0,0x79, -0x1d,0x1f,0x13,0x82,0x03,0x54,0x7c,0x21,0x7e,0xda,0x6c,0x94, -0x77,0x4b,0xd5,0xf5,0x4b,0x6a,0x86,0x37,0x28,0x39,0x86,0x0a, -0x26,0xa2,0x72,0x13,0x51,0x7d,0x79,0x22,0x5c,0x16,0x11,0x27, -0x89,0xf7,0x43,0x17,0x17,0xbf,0xd9,0x1e,0x51,0x72,0x1e,0x01, -0x2e,0xef,0x9d,0xc3,0xf0,0xd0,0x58,0x04,0x1d,0x0a,0x6a,0xd2, -0xad,0x82,0xf4,0x50,0xed,0xd8,0x9f,0xff,0xfd,0x95,0x83,0x08, -0xc7,0x89,0x8f,0x57,0x5b,0x09,0x6c,0x32,0x3b,0x8e,0xb7,0xa5, -0x7a,0x29,0xd7,0xb0,0xe5,0xaa,0xf2,0x39,0x51,0x3a,0x25,0xe7, -0xbc,0x40,0x11,0x93,0x3d,0x24,0xb2,0x41,0x86,0x11,0x3d,0xfe, -0x29,0x0e,0xb2,0xa2,0xb9,0x6d,0x91,0xc1,0x18,0xe9,0x9f,0xea, -0xdb,0x7c,0xa2,0x59,0x0b,0x5a,0xb5,0x3d,0x49,0x0a,0x17,0xf5, -0xce,0xa6,0x5b,0x7a,0x1b,0xdb,0x30,0x34,0xc0,0x1a,0xac,0x10, -0x9a,0xd4,0xac,0xb7,0xed,0xda,0x35,0xc3,0xc2,0x41,0x58,0x5d, -0x1d,0x7f,0x69,0x6e,0x1b,0x86,0xd2,0x65,0xde,0xd8,0xc5,0x23, -0xcb,0x8c,0x75,0x6d,0xcf,0xe0,0xe4,0x6b,0x4e,0x21,0x8b,0x59, -0xf7,0x4d,0x03,0x69,0xba,0xec,0x8d,0xa9,0x9a,0xc8,0xc0,0x8f, -0xeb,0x97,0x9a,0x06,0x67,0x7f,0x1e,0x5b,0x3f,0x41,0x14,0x13, -0x70,0x43,0x44,0x32,0xe1,0x07,0xd1,0xcc,0x26,0x43,0x57,0xc4, -0x7f,0xa8,0xaa,0x67,0xb3,0x32,0xd2,0xbc,0x3d,0xda,0xaa,0x89, -0x5b,0x2d,0xc6,0x63,0x77,0x1f,0xd4,0x5e,0x15,0x45,0xb6,0x08, -0xf3,0xc0,0xa9,0x5f,0x37,0x17,0x8c,0x09,0x3d,0xaa,0xb1,0x59, -0x01,0x91,0x55,0x75,0xf9,0xb6,0x1b,0xa7,0xd7,0xe3,0xd9,0xb3, -0x4f,0x2c,0x0a,0x88,0xbe,0x97,0x9a,0xab,0x0e,0x55,0x79,0x55, -0xa5,0xe9,0x0a,0x88,0x95,0xb3,0x79,0x11,0x35,0x7b,0x35,0xdf, -0xda,0x72,0xf1,0x8d,0x4c,0x9c,0x7f,0xb5,0x53,0x9c,0xd1,0xe6, -0x72,0x96,0x81,0xf3,0x03,0xb5,0x52,0xcc,0x0e,0xa8,0x5d,0x3b, -0x9d,0x09,0xe2,0xd5,0x7c,0xb3,0xad,0x95,0x82,0x8f,0x8f,0x43, -0xbc,0x68,0x4b,0x6e,0xbe,0x1f,0xd0,0x91,0xa5,0x67,0x9a,0x3f, -0xc5,0x4f,0xd1,0x5c,0x67,0x33,0x33,0x70,0x82,0x42,0x01,0xc7, -0xbe,0x42,0x48,0x5b,0xa6,0x27,0x1f,0xcd,0x3d,0xe6,0x16,0xdd, -0x86,0xd8,0xae,0xd2,0xa5,0xf2,0x84,0x44,0xfa,0x61,0x9b,0xdc, -0x56,0x98,0x1d,0x19,0x68,0xa8,0x7d,0x38,0xa9,0x6e,0x2c,0x50, -0x04,0x08,0x1b,0x70,0xbe,0x48,0xc9,0xf6,0x74,0x4a,0xfd,0x70, -0xdb,0x96,0x1d,0xb0,0xb1,0x59,0x0b,0x2e,0x4b,0x26,0x98,0xc2, -0x42,0xce,0x3b,0x4f,0xbd,0x22,0x13,0x95,0x0b,0x64,0x6e,0x56, -0x97,0xa8,0x1f,0x92,0xa0,0x30,0x5e,0xdf,0x4d,0x76,0x59,0x5c, -0x94,0x2d,0xb5,0x65,0xec,0xe8,0x20,0x79,0x0b,0xad,0x08,0x65, -0x3a,0xc9,0x03,0xc3,0xf2,0x3f,0x29,0xef,0x35,0x5c,0x0d,0x4f, -0x67,0x23,0xe5,0xd8,0xcc,0x17,0x14,0xed,0xfb,0x4f,0x57,0x93, -0x4f,0x55,0xe3,0xb2,0x10,0x3c,0x12,0x86,0x89,0x8f,0xa1,0xaa, -0x52,0x85,0xa8,0xbd,0xae,0x09,0xa7,0x89,0xd5,0x5f,0xc9,0x57, -0x9a,0xc6,0xf8,0x20,0xe5,0x65,0x1b,0xbe,0x46,0x34,0xdd,0xa7, -0xc3,0xe9,0xa2,0xe2,0x3c,0x72,0xff,0xf2,0x38,0x88,0x3f,0x8a, -0xab,0x4f,0x49,0xde,0x1f,0x28,0xb2,0xd5,0xe0,0xec,0x7e,0x34, -0xe6,0xe1,0xdd,0x39,0x10,0x2c,0x5b,0x8e,0x6e,0x3e,0xb8,0xe0, -0x6f,0x27,0xf1,0xd4,0x67,0xc2,0x94,0xaf,0x97,0x58,0xa4,0xaf, -0xdc,0x4a,0xd6,0xf9,0x65,0xc0,0x6e,0x34,0xc9,0x71,0x25,0xc6, -0x09,0x22,0xde,0x6e,0x9b,0x0f,0x1a,0x2c,0x24,0xd4,0x64,0xb4, -0x53,0x93,0x66,0xcc,0x31,0x50,0x84,0x7a,0x7b,0xf8,0x12,0x30, -0xa4,0x98,0xf9,0x99,0x87,0xeb,0xde,0x0a,0xde,0x41,0x2e,0xb9, -0x1b,0xce,0xdb,0x30,0xc6,0x91,0x37,0x1e,0x8a,0x56,0xe8,0x78, -0x4c,0xef,0xb0,0x47,0x76,0x66,0xfc,0xb9,0x41,0x0a,0xb2,0xd1, -0x60,0x39,0x63,0xc9,0x69,0x98,0xe8,0x37,0x83,0xae,0x8d,0xdc, -0x8d,0xff,0xa1,0x8c,0x2d,0x11,0x41,0x97,0x63,0x6e,0x5a,0x8e, -0xfe,0xae,0x53,0xc9,0xb3,0x5e,0xef,0x68,0x31,0xc7,0xaa,0x96, -0x24,0xde,0x77,0x24,0xd2,0x87,0xae,0x5c,0x79,0x4b,0x6d,0x91, -0x5f,0x36,0xeb,0xc5,0x2a,0xac,0xe2,0x52,0x79,0x60,0x9d,0x70, -0x73,0xb6,0xcc,0x74,0xce,0x7d,0xa0,0xe1,0x35,0x6b,0x26,0x11, -0xfb,0xc1,0xb5,0x04,0x5e,0x4c,0x83,0xe6,0xd5,0x24,0x44,0x1e, -0x65,0xec,0xd1,0x5f,0x99,0x99,0xe2,0x80,0xe6,0x8c,0x9c,0x73, -0xc8,0xa1,0x01,0xab,0x56,0x0a,0x47,0xc3,0x40,0x8b,0x3e,0x40, -0x2f,0x81,0x5e,0x30,0x41,0x01,0x50,0x8f,0xf1,0x18,0x6b,0x96, -0xff,0xce,0xd5,0x21,0xfe,0xfa,0x1b,0x57,0x84,0xb7,0xf1,0xbd, -0x6d,0x45,0xe0,0x45,0xb8,0x6d,0x05,0xf8,0xa5,0x7a,0xd0,0x4c, -0xef,0x52,0x1d,0x6e,0x53,0xd8,0x2f,0xd7,0xd1,0x86,0x48,0xd3, -0x57,0x3d,0xc3,0xcc,0xe0,0x6e,0x6a,0x1a,0xd5,0xf7,0xd5,0x14, -0xf0,0x8d,0x27,0xf3,0x8f,0xb3,0x82,0x6f,0x70,0x46,0x43,0xe0, -0x02,0xc7,0xf8,0x9f,0x16,0x1a,0x4b,0x58,0x89,0xc4,0x9e,0xb1, -0xeb,0x01,0x49,0x11,0xfc,0x25,0xcb,0xf1,0x1a,0x7f,0x3e,0xf3, -0x36,0xfe,0x19,0xd2,0x86,0xe2,0x5f,0xdf,0xad,0x4c,0x02,0x01, -0xe3,0x04,0x81,0xe5,0xd3,0x04,0xdc,0x26,0x76,0x37,0xc9,0x7c, -0x87,0x07,0x7c,0x4d,0x3e,0x88,0xeb,0x04,0xa8,0x0a,0x73,0x2c, -0xbe,0x88,0xa1,0x50,0x86,0x2d,0x15,0x5c,0x09,0x01,0x01,0x57, -0x13,0x63,0x50,0x0d,0x66,0x0c,0x18,0x81,0xaa,0x5d,0xcf,0x14, -0x9d,0xa1,0x4f,0x02,0x11,0xf6,0xbb,0x42,0x77,0xc4,0x57,0xb3, -0x76,0x5b,0x29,0xf0,0x86,0xb7,0x77,0x2a,0x03,0xc7,0x1e,0xb5, -0x41,0xe9,0xbc,0xc2,0x36,0xa9,0xb0,0x10,0xf8,0xd7,0xda,0xd2, -0x71,0xa0,0x38,0x29,0xd3,0xbc,0xee,0xfa,0x86,0x9e,0x38,0x02, -0xb2,0xf4,0x04,0xc2,0xfc,0xea,0xaa,0x77,0x50,0x78,0x8b,0xf2, -0xd2,0x37,0x9b,0xbb,0x67,0x3f,0x0a,0xfb,0x81,0xe7,0xbb,0xa1, -0x35,0xb0,0xb5,0x55,0x2f,0x8e,0xe8,0xdb,0x12,0x83,0x22,0x4e, -0xf6,0xae,0xd9,0xc4,0xc8,0x3d,0x66,0x97,0xb3,0xd3,0x05,0xc9, -0xec,0x77,0xc6,0xaa,0x5f,0x12,0xc3,0x67,0x1f,0x7e,0xa3,0xb9, -0x6e,0xfc,0xd5,0x01,0xcf,0xed,0x47,0x59,0xb7,0xc3,0x68,0xdf, -0x9c,0xb9,0x38,0xbe,0x37,0x73,0xc8,0x1e,0x25,0xae,0xae,0xe6, -0x93,0xdb,0x25,0x67,0xa4,0xee,0x65,0xd0,0x7e,0x62,0xb7,0xdc, -0xa3,0xa9,0xc5,0xa1,0x65,0xbd,0xb9,0xc5,0xba,0x40,0xcb,0xe4, -0x34,0xa5,0xb5,0xb4,0xee,0x1f,0x51,0x52,0x18,0x76,0xf9,0xbc, -0x4a,0x17,0xdf,0x50,0xf0,0xcd,0xf3,0x6c,0xff,0x91,0xa7,0x2a, -0xe4,0xfc,0x35,0xe6,0x48,0x3e,0xc5,0x94,0x32,0x11,0xd2,0x3e, -0x19,0x8a,0xd8,0x87,0x2e,0x31,0x39,0x46,0x2a,0x6e,0x8d,0x87, -0x39,0x7a,0x71,0xc1,0xd3,0x8b,0xf8,0xe6,0x81,0x3d,0x9c,0xa3, -0xc9,0xf7,0xd1,0xc9,0x0a,0xe7,0x18,0xc5,0x94,0x7a,0x44,0xca, -0x05,0x81,0x0f,0xcd,0x2e,0x0a,0xaf,0x56,0x0c,0x6b,0x94,0x3b, -0x2c,0xf7,0xf6,0xe0,0xa8,0x3c,0x0a,0x40,0xba,0xdd,0x41,0xc4, -0x2b,0x8a,0x15,0xe5,0xdc,0x0e,0xff,0xa0,0x1f,0xf4,0xec,0xd6, -0x1a,0x0b,0xa2,0x4f,0x20,0xe8,0xb8,0x05,0x6e,0xda,0xa4,0x02, -0xe2,0xd1,0x71,0x16,0x6f,0xb5,0xf2,0x13,0x73,0x46,0xd2,0xcc, -0x82,0x25,0x1f,0x4d,0x68,0xc2,0xec,0xf0,0xdb,0x3c,0xca,0x68, -0xc7,0x51,0x9e,0x14,0x14,0x65,0x78,0x3e,0x42,0x41,0xd8,0xe7, -0x52,0x36,0x0d,0xc7,0xbf,0x01,0x7a,0x16,0xd4,0x49,0x08,0xaa, -0x7d,0xdf,0x40,0x74,0xd1,0x01,0x93,0x0a,0x32,0x7d,0xc6,0x60, -0xb7,0xc1,0x79,0xbc,0x99,0x80,0xc0,0x9d,0xce,0x3f,0x16,0x7f, -0x3a,0x38,0x80,0xc3,0xb7,0x5a,0x15,0x28,0x92,0xe6,0xb8,0x24, -0xe4,0x67,0x28,0x64,0x0d,0xf8,0xf7,0x9c,0xdf,0x6d,0xcd,0x52, -0x0e,0xdb,0xd1,0x16,0x7b,0x96,0x2a,0xd3,0x89,0xde,0xe8,0xc2, -0xe5,0xd5,0x64,0x17,0x44,0xae,0xe2,0x2b,0xa6,0xac,0x37,0x99, -0x2c,0x86,0x80,0x9a,0x64,0x3b,0x98,0xbe,0x3a,0xa7,0xc6,0x0f, -0x28,0x6b,0x6c,0x82,0x2a,0xf3,0x87,0x1e,0xb1,0x37,0xad,0x9f, -0x0f,0xef,0x49,0x38,0x72,0xe3,0xee,0x0e,0x08,0xae,0x09,0xf9, -0x55,0x6c,0x4c,0xd3,0xed,0x77,0x62,0x91,0x4e,0xd4,0x88,0xe2, -0x68,0x4e,0xbe,0x42,0x51,0x6a,0x27,0xbf,0x98,0x7f,0x2a,0xef, -0x91,0x56,0x41,0x86,0xf7,0xd5,0xaa,0x80,0x91,0xa6,0x3e,0x89, -0x97,0x75,0x74,0x82,0x6f,0xfd,0x21,0x06,0x26,0xe6,0x49,0xad, -0x6a,0x7e,0x99,0x39,0x3a,0x8b,0xc0,0xa2,0x34,0x67,0x2c,0x09, -0xaa,0xa3,0x61,0xfd,0x62,0x53,0x13,0x68,0xe8,0x7b,0xa9,0x75, -0x59,0x76,0x3d,0x5e,0xfd,0x80,0x9e,0x38,0xe0,0x9c,0x39,0x99, -0x4e,0xa0,0xf0,0x5b,0xb8,0xbc,0x9a,0x1c,0x97,0xc1,0xfc,0x62, -0xeb,0xb7,0x17,0x42,0x65,0x23,0xe8,0x63,0x49,0xd6,0xaa,0xf1, -0xd1,0x8f,0x5a,0x9b,0x53,0x77,0xa1,0x64,0x17,0x7e,0x3b,0xf8, -0x1c,0xd8,0x5d,0x0c,0xaf,0xc7,0xff,0xc5,0x43,0x46,0xd2,0xac, -0x4f,0x89,0x8b,0xa4,0x7c,0x99,0x6c,0x4f,0x9b,0xdb,0xbd,0x24, -0xd8,0x67,0x28,0xbe,0x76,0x90,0xf1,0x80,0x21,0x20,0x0c,0x78, -0x48,0x7f,0x6f,0x86,0x44,0xc6,0xc5,0x63,0x50,0x18,0x89,0xb0, -0x36,0x19,0x02,0x72,0x6b,0x5a,0x96,0xcc,0xbd,0x9b,0xc9,0x2d, -0xac,0xf5,0x89,0x18,0xee,0x53,0x9f,0x68,0x86,0x9b,0xc7,0x06, -0x51,0x35,0x25,0xc9,0x8e,0x12,0x7c,0xac,0x9c,0x5c,0x92,0x6e, -0x94,0xb3,0x77,0x6b,0x4c,0xac,0x3a,0x53,0xb2,0x65,0xb2,0x1c, -0x4f,0x49,0x81,0x3d,0xe1,0xf5,0x02,0x08,0x2b,0x74,0x19,0x99, -0xe0,0x6c,0xa9,0xfe,0x6e,0x49,0xf3,0x2c,0xef,0xa9,0x6c,0x9a, -0x1e,0x22,0xb9,0xa5,0xa5,0x24,0x4b,0x1f,0xd2,0x4c,0xac,0xf5, -0x22,0x83,0xab,0xd6,0x3c,0xd2,0x97,0x96,0x0c,0xc4,0xe7,0xaa, -0xe5,0x20,0xe3,0xbb,0x24,0xf6,0x70,0x89,0x86,0x1d,0xc7,0x6a, -0xbe,0xb7,0x4c,0xbd,0x73,0x33,0xd3,0x23,0x24,0x2d,0x26,0x9f, -0xc6,0x23,0x1b,0xe7,0x0f,0x4d,0xf5,0x01,0xc9,0x7c,0xcc,0xbc, -0xef,0x5a,0x66,0x9a,0x30,0xc8,0x1e,0xef,0x1d,0xca,0xab,0xd1, -0x3d,0x71,0xfe,0x5d,0x95,0xf7,0x9b,0xa3,0x5a,0x3b,0x7a,0xa9, -0xcd,0xa1,0x7d,0xf7,0x56,0x79,0x73,0x1e,0xbc,0xd2,0xe4,0xc2, -0x75,0xa7,0x83,0x19,0xc4,0x93,0x9d,0xfb,0x22,0x95,0x12,0xc6, -0x2a,0xb7,0x6b,0x67,0x86,0x6f,0x9b,0x5a,0xb4,0x96,0x81,0x56, -0x18,0xad,0xcc,0x46,0x84,0x81,0x0e,0x1d,0xa3,0x85,0xee,0xf4, -0x71,0x86,0xea,0x5c,0x78,0xe2,0xa7,0x11,0xb6,0x0e,0xd3,0x9d, -0x4e,0xb5,0x10,0xe9,0x5d,0xc6,0x50,0x31,0x6c,0xe3,0xf7,0x5c, -0x29,0xda,0x52,0xdc,0xfe,0xb8,0xfc,0x94,0x8d,0xed,0xeb,0x73, -0x1a,0xb4,0xdf,0xa6,0x18,0x10,0xb4,0x99,0xa4,0x8f,0x59,0x9e, -0xa0,0x91,0xc2,0xeb,0xc4,0x7b,0x16,0xe9,0x69,0x31,0x4a,0x30, -0x8d,0x67,0x3c,0xbc,0xd0,0x29,0xc0,0x25,0xc4,0x51,0x1e,0x05, -0x31,0xac,0x3d,0xe7,0x23,0xcf,0xdc,0x5c,0xa8,0xca,0xc2,0xc7, -0x9e,0xe0,0xd0,0xd4,0xa0,0x37,0xc4,0x45,0xcd,0xf9,0x23,0x67, -0x9e,0xaa,0x2c,0xf5,0x07,0x0e,0x4e,0x65,0xc3,0xb8,0xd5,0xf8, -0xc0,0xd6,0xec,0xc9,0x76,0x7f,0xe8,0x24,0x2b,0x52,0xdb,0x1a, -0x79,0xb9,0x31,0x31,0xb0,0xd2,0x04,0x8a,0x49,0xf2,0x82,0x6e, -0xce,0x53,0xad,0x2d,0x83,0xe7,0x3b,0x5b,0x21,0x43,0xb3,0xbc, -0x41,0xcf,0x6c,0x36,0x07,0x2c,0x0b,0x7a,0x4f,0x73,0x6d,0xbb, -0x55,0x03,0xcd,0xa6,0x90,0x61,0x2d,0xb0,0x2d,0x64,0x62,0xbf, -0x06,0x10,0x68,0xaa,0xbf,0x02,0x04,0x2e,0x05,0x0f,0x43,0x4e, -0xf0,0xb9,0x6b,0xc5,0x5e,0xdc,0xac,0xbd,0xda,0x60,0xb3,0x25, -0x62,0xf2,0x4f,0xb8,0xa3,0x27,0xf7,0xdc,0x37,0x62,0xaf,0xd6, -0xac,0x6d,0x20,0xc8,0x97,0xa1,0x88,0x69,0xe4,0xb6,0xb9,0x5f, -0x69,0xa3,0x57,0x9f,0x9a,0x1f,0x9e,0xcd,0x07,0x74,0x2a,0x6c, -0x36,0x4f,0x47,0x35,0xce,0xaf,0x4e,0x2e,0xa9,0xaa,0xea,0xd9, -0xbf,0x65,0xb4,0x83,0xab,0x22,0x0b,0x5a,0xd7,0xb0,0x16,0xf8, -0xc3,0x98,0xeb,0xfb,0x8a,0x96,0x37,0x91,0xf8,0xee,0x62,0x2c, -0x12,0x73,0x73,0x26,0xb9,0xbb,0x7b,0x39,0x5e,0xee,0x66,0x6b, -0x4d,0x5b,0x2c,0x96,0xb1,0x25,0x76,0xa2,0x0b,0xe0,0x4c,0xda, -0xff,0xaf,0x7d,0x35,0x36,0x0d,0xd1,0xe6,0xf9,0xa1,0x66,0x75, -0xcd,0x58,0xe3,0xfd,0xb2,0xff,0x48,0xa9,0xeb,0x83,0xda,0xc9, -0x0d,0x71,0x8a,0xc0,0x4d,0x45,0xe4,0x8d,0xaa,0x07,0x59,0x7b, -0xd8,0x18,0x24,0x38,0x31,0x0c,0xff,0xf8,0x71,0x19,0x2d,0x6b, -0x75,0xb7,0x0d,0x95,0xb6,0x97,0x1d,0xaf,0xea,0x47,0x2a,0x00, -0x1c,0x66,0x82,0xfe,0x4f,0x7b,0x5e,0xaa,0x29,0x33,0x88,0x46, -0x4a,0x8e,0xa4,0x72,0xcc,0x00,0xbf,0x45,0x3d,0xd3,0x19,0x52, -0x28,0x59,0x50,0x01,0x1a,0x28,0xc8,0x59,0xd0,0x18,0x10,0x7c, -0xdf,0xe9,0x9a,0xdf,0x48,0xf1,0xae,0x14,0xcf,0xa6,0xf8,0x26, -0x74,0xae,0x51,0x68,0xc7,0x16,0xe2,0x20,0x25,0xe6,0x9a,0x04, -0xa6,0xf7,0xb8,0xbc,0x33,0xa2,0x4e,0x6e,0xce,0x93,0x0e,0xe5, -0x13,0x95,0x04,0xb6,0x48,0x87,0x0f,0xf2,0xf3,0xbb,0x95,0x26, -0x1a,0x98,0x42,0x2a,0x3c,0x25,0x1f,0x1f,0x18,0xfb,0x2a,0x32, -0x05,0xab,0x71,0x66,0x0d,0xf0,0x86,0xb6,0xe2,0x49,0x97,0x37, -0x80,0x9c,0x92,0xdd,0xef,0xb8,0xcb,0xe4,0x8e,0xd7,0x57,0x61, -0x9e,0xa5,0x1c,0x83,0x9d,0xc1,0xa9,0xbb,0xaf,0x30,0xb7,0xd4, -0x25,0x4e,0xc0,0xd4,0x15,0xe0,0xce,0x41,0x4f,0x4e,0x15,0xe8, -0x17,0x9f,0x74,0x46,0x51,0xe2,0x4b,0xab,0x92,0x46,0xc4,0x34, -0x1e,0xd6,0xd0,0x51,0x6c,0x34,0xd0,0x2e,0x51,0xf6,0x0d,0xb2, -0x75,0xc9,0x25,0x8c,0x90,0xf2,0xc8,0x0e,0x8a,0xbf,0x94,0x91, -0x9b,0xc6,0x5e,0xaa,0xa4,0x40,0x0d,0xb9,0x27,0x6a,0xff,0x76, -0xf8,0x89,0x91,0x49,0x74,0x5e,0x4e,0xea,0x0b,0xb0,0xca,0x04, -0x1a,0x52,0x11,0x71,0x94,0xc9,0xcb,0xe7,0x47,0x90,0xb7,0x21, -0x2a,0x68,0x5f,0x96,0x15,0x35,0x87,0x8a,0xbd,0xf8,0xc8,0xa6, -0x11,0x12,0x7f,0x34,0xc1,0x93,0x6e,0x34,0x7e,0xea,0xd7,0x86, -0xa3,0xdc,0x22,0xe8,0x89,0x7d,0x7f,0xa3,0x2d,0xa9,0x49,0x6e, -0x07,0x78,0x07,0x66,0x5c,0x81,0x8a,0xbb,0xef,0xca,0x39,0x9b, -0x27,0x3a,0x69,0x9e,0x45,0xc5,0xd0,0x50,0xf4,0x15,0x7f,0xbb, -0xc3,0xdb,0x91,0x0d,0x0b,0x20,0x8d,0x4a,0x13,0x36,0x91,0x83, -0x2f,0x2b,0x0d,0xce,0x7a,0x38,0x33,0xb4,0x43,0xc5,0x1a,0x7d, -0x1e,0x8e,0xc8,0x75,0xb2,0xbc,0xd2,0x67,0x76,0x0f,0x48,0xd5, -0xb8,0x85,0x0f,0x44,0x97,0xa8,0xf6,0xab,0xc6,0xe4,0x6e,0x27, -0xb3,0xa7,0xcb,0x25,0x5a,0x35,0xbb,0x9b,0x7a,0x0e,0x7e,0xc4, -0x7e,0xb9,0x85,0xd5,0x08,0x27,0x32,0xa2,0x8a,0x68,0x3b,0xc7, -0xed,0x13,0xbc,0xad,0xd9,0x50,0x75,0x99,0xf4,0x93,0x8e,0x40, -0xe9,0xf1,0x4f,0x07,0xd6,0x37,0x2a,0x48,0x77,0xb0,0x4c,0x27, -0x19,0xb4,0xd0,0x04,0xe1,0xed,0x02,0xf1,0x68,0x03,0xe1,0x70, -0xe0,0x62,0x8f,0x97,0xd7,0x95,0x35,0x73,0x4a,0x6f,0x59,0x93, -0xab,0xcf,0x86,0x88,0xbe,0x78,0xef,0x9a,0x68,0xd1,0x5c,0x88, -0xee,0xc1,0x1f,0xc0,0x02,0xd8,0x46,0x4c,0xea,0x6d,0x6b,0xec, -0xb7,0xce,0xef,0xfb,0xe7,0xd5,0xf9,0xe9,0x00,0x6d,0x20,0xa3, -0xbe,0xf6,0xa4,0xfa,0x49,0x07,0x85,0x4b,0x26,0x6e,0x90,0x12, -0x48,0x14,0x08,0x0a,0xb3,0x40,0xad,0xd8,0xae,0x6b,0x0b,0xc5, -0xb3,0xb3,0x3d,0x19,0x94,0x52,0x3c,0x19,0x68,0x2f,0x7c,0xcd, -0xe7,0xf7,0xe7,0xa3,0xce,0xf9,0x66,0xff,0xda,0x5c,0x60,0x28, -0x8c,0x2b,0x8f,0xa7,0xcb,0x6b,0x61,0xff,0x77,0x28,0x56,0x6c, -0xd9,0x1e,0xe6,0x7b,0x87,0xee,0x38,0x60,0xe6,0xfb,0xa3,0xef, -0xed,0x08,0xc9,0xd9,0xd4,0x43,0x2e,0x42,0x69,0x13,0xfa,0x92, -0x7b,0x90,0x9b,0xa1,0x81,0x4c,0x59,0x0f,0xfe,0x34,0x0f,0x4d, -0xef,0xa1,0xb1,0xa1,0xa2,0x8d,0x63,0xa3,0x40,0x8b,0xd5,0xfc, -0xdf,0x2b,0xa9,0x24,0x13,0x33,0x3f,0x59,0xae,0xe3,0x22,0xbf, -0x9d,0xe4,0x7c,0x46,0xf6,0x36,0xf6,0xf7,0x49,0x47,0x09,0x79, -0x31,0xd7,0x40,0x4d,0x2d,0xdf,0x03,0xf6,0xb3,0xbc,0xde,0x4f, -0x3a,0x1f,0xc6,0xcb,0x8a,0x9e,0x69,0x61,0x7b,0x75,0x92,0xfd, -0xfa,0x3a,0xda,0xf7,0xeb,0x28,0x9f,0x94,0x8f,0x8e,0x26,0xdf, -0xc7,0x27,0x35,0x7b,0x5f,0x8e,0x2a,0x46,0x63,0x4a,0x93,0x5e, -0xd2,0x83,0xd2,0x6d,0x34,0xcb,0xb0,0x00,0xc8,0x93,0xbd,0x47, -0x5b,0xcd,0xcd,0xfb,0x49,0xf6,0xcb,0x73,0x02,0x87,0x8e,0x13, -0xc8,0x19,0x7f,0x6c,0x91,0x09,0x33,0x9d,0x7a,0x38,0x0b,0xdc, -0x3e,0x12,0xeb,0xf1,0x2f,0x26,0xbf,0x8e,0x79,0x99,0xa2,0x3f, -0x5a,0xe6,0x10,0x5f,0xbc,0x5f,0xaf,0xdd,0xa9,0x8c,0x69,0xc2, -0x3b,0xd9,0x39,0x3c,0x12,0xc7,0xd4,0xf8,0x1a,0x85,0xeb,0x57, -0x3a,0x23,0xf5,0x1d,0x96,0x54,0xd8,0x88,0x79,0xc8,0xd4,0xef, -0x1c,0x14,0x6a,0xac,0x1e,0xc1,0xad,0xd7,0x3b,0xd1,0x66,0x80, -0x8b,0x0b,0xed,0x7d,0x28,0x8f,0xb2,0xc6,0x2f,0xd7,0xeb,0xcd, -0xdb,0x2f,0xe6,0xd0,0xde,0x21,0x79,0x73,0xc4,0x25,0xee,0x3a, -0x06,0x4b,0xfd,0x29,0xde,0xd4,0x10,0x9b,0x4f,0x48,0x6c,0x74, -0xbe,0x54,0xbb,0x9d,0x5e,0x83,0x08,0x7a,0x8b,0xc5,0xf0,0x3d, -0x36,0x6c,0x4a,0x66,0x74,0x8b,0xf0,0x3b,0x71,0x7c,0x5f,0xb8, -0x44,0x21,0x1e,0xd0,0x54,0xae,0xdb,0x7d,0xe2,0x51,0xaf,0x1f, -0xd8,0xfd,0x9b,0x61,0xf5,0xfa,0xe3,0x4c,0xdd,0x2a,0x92,0xcd, -0xac,0x76,0x5b,0x5b,0x22,0x46,0x11,0xc4,0x18,0x02,0x64,0x27, -0x35,0xa5,0x1c,0x33,0xf1,0xeb,0x2e,0x37,0xac,0x2b,0xbd,0x60, -0x88,0xf2,0xb1,0x53,0x96,0x12,0xca,0x4c,0x6c,0x5a,0xeb,0xa4, -0x03,0x14,0xcf,0xb1,0xf6,0xad,0x3c,0x38,0x82,0xbf,0xdf,0x9b, -0x19,0x84,0x4f,0x58,0x90,0x5f,0xdd,0xc8,0x68,0x8c,0xfe,0xb7, -0x7a,0xad,0xe8,0xf9,0xc5,0x7b,0x3f,0x87,0x15,0x69,0x1f,0xfd, -0xc7,0xf8,0x73,0xc5,0x6b,0x53,0x66,0x0d,0xf9,0xf2,0x4e,0x0c, -0xf6,0x6b,0xea,0xc7,0x32,0xcc,0x98,0xb6,0x55,0x51,0x6c,0x45, -0x4a,0x19,0xb5,0x65,0x73,0x4a,0x35,0x6f,0x59,0x3c,0xa6,0xea, -0xaa,0x8c,0xda,0xc8,0xde,0xd9,0xbf,0x27,0xd8,0x66,0x8f,0x3f, -0x38,0x02,0xb8,0x5d,0x69,0x14,0xd8,0x48,0x85,0x69,0x9e,0x6f, -0xd7,0x72,0x8c,0xc7,0xcd,0x58,0xdc,0x2d,0x28,0xb8,0x67,0xb3, -0xfa,0xd5,0xc7,0x58,0x02,0x99,0x68,0xf3,0xd9,0x8c,0x57,0x0c, -0xb3,0xf9,0x67,0xd0,0x88,0x9f,0x26,0x23,0x43,0xe9,0x8d,0x26, -0xd7,0x93,0x95,0x9c,0xef,0x77,0x93,0x51,0xf0,0xb6,0x7b,0x44, -0x87,0x83,0x3c,0xf3,0xe6,0x94,0x13,0xf3,0xe8,0xcb,0x29,0x6a, -0xb3,0xa3,0x19,0xae,0xe3,0x55,0x0a,0x44,0xeb,0x1f,0xff,0xad, -0x2c,0x29,0x8b,0x32,0x5c,0x7d,0xce,0x32,0x39,0x4e,0x72,0x04, -0xce,0x9e,0x20,0x12,0x12,0x87,0xf8,0xcb,0xc1,0xce,0xd6,0xfc, -0x7c,0x7c,0x77,0xaf,0x96,0xf3,0xdb,0x13,0xa9,0x24,0xa5,0xbc, -0x9d,0xc3,0x2c,0x04,0x64,0x2b,0x39,0x48,0x36,0x1e,0x94,0x13, -0x30,0x86,0xe8,0xb0,0x8c,0x59,0x6d,0xe3,0xd5,0x8f,0xc3,0xea, -0x06,0xa6,0x92,0xe6,0x1d,0xee,0xbb,0x1b,0xba,0x9d,0x77,0x77, -0x6f,0x38,0xba,0xbc,0x29,0xe4,0x31,0x38,0x88,0xf6,0x10,0x84, -0x5f,0x33,0xaa,0xe4,0x84,0xb9,0x29,0x21,0xe8,0x33,0xea,0xcc, -0xea,0x8e,0xf5,0x38,0x43,0x88,0x29,0xc0,0x1f,0xd8,0xe3,0x50, -0xb4,0xdd,0xb6,0xa7,0x58,0xd3,0x0c,0x43,0x26,0xda,0x43,0xe2, -0x96,0x19,0x51,0x3c,0xd3,0x0d,0x48,0x85,0x6e,0xe4,0xf0,0x63, -0x0e,0x7c,0xbe,0x31,0xfc,0x33,0x19,0x32,0xe6,0x9e,0x73,0x64, -0x7e,0x78,0x90,0xd9,0x9c,0xe4,0x4b,0x5a,0x78,0x2d,0x91,0xc3, -0x47,0xe9,0x2a,0xdc,0x2d,0xa9,0x04,0x0d,0xee,0xb5,0xc9,0xef, -0xdd,0x71,0x59,0x70,0xf1,0x4d,0x96,0x29,0xe6,0xea,0x97,0xdf, -0x1c,0xe9,0xba,0x7b,0x53,0x23,0x69,0x2f,0xc2,0xb6,0xdd,0x75, -0xf5,0xec,0xfa,0x54,0xea,0xe2,0x33,0xe7,0x5c,0xaa,0x8e,0xf2, -0x79,0xb2,0x16,0x86,0xc2,0x5e,0x71,0x1e,0x39,0x08,0xca,0x1a, -0x24,0x21,0xf4,0x41,0x65,0xcb,0x3e,0xf9,0x42,0xc9,0x9d,0xad, -0x25,0x45,0x72,0x2d,0x6c,0x2e,0xf3,0x19,0x29,0xae,0xe1,0xb1, -0xbb,0x06,0x43,0x30,0x97,0xb0,0xf2,0x46,0xbf,0x59,0xb6,0x94, -0x38,0x9d,0x0c,0x4b,0xc5,0x97,0x69,0xbd,0xa7,0xf8,0xaa,0x1f, -0x96,0xe9,0x73,0x03,0x5b,0x98,0x34,0xc0,0xa2,0x82,0x8d,0x89, -0x28,0xa3,0x1c,0x28,0x31,0x32,0xb1,0x05,0x08,0xa5,0x7d,0x15, -0x94,0x67,0xc1,0x4a,0x77,0x80,0xea,0xd4,0x49,0xad,0x0f,0xa2, -0x4a,0xd4,0xdc,0x00,0x4e,0xfc,0xaa,0x16,0x18,0xac,0xe7,0xde, -0xdf,0x4c,0x7c,0x0d,0x2b,0x91,0xc2,0x5f,0xea,0x59,0x83,0xbf, -0x86,0xd3,0xf1,0x52,0x12,0xcc,0x2b,0x65,0x30,0xa3,0x97,0x70, -0x35,0x35,0x03,0xde,0x1d,0x43,0xde,0x63,0xb6,0x03,0x44,0xdf, -0x1f,0x09,0x35,0xb5,0x03,0x83,0x8a,0x33,0xa6,0xac,0xeb,0x1a, -0x26,0x0c,0xc8,0x55,0x7d,0xd8,0xb3,0x10,0x27,0x9a,0x54,0x6a, -0xad,0x04,0x71,0x33,0xbf,0x26,0x59,0xe8,0x16,0x7f,0x1c,0x6f, -0x6e,0xbd,0xa6,0x4f,0xec,0x17,0x05,0xd0,0xe5,0x82,0x81,0x84, -0x8c,0xc2,0xed,0x98,0x15,0x8b,0x5a,0xd3,0x35,0x1d,0x1d,0x79, -0x0d,0xe7,0x18,0x0b,0xf3,0x4c,0x27,0x15,0x7b,0x8d,0xc0,0x82, -0x7c,0x57,0x37,0xdf,0x99,0xb5,0x42,0x78,0xe6,0x49,0x39,0x7b, -0x76,0x5d,0x4e,0xef,0x46,0xe3,0x0a,0xce,0x58,0x58,0x74,0xb9, -0x19,0xb2,0xbd,0xc3,0x9d,0x52,0x2d,0x45,0xf8,0x44,0x5b,0x32, -0x4e,0xe3,0xa7,0x4f,0xf8,0x52,0x0b,0x13,0x99,0x73,0xc9,0x63, -0x22,0x69,0xa8,0x8f,0x8c,0xc7,0x3b,0x42,0xf7,0x31,0x83,0x45, -0x03,0x28,0x1b,0x1a,0xf0,0xc3,0x14,0xbc,0xed,0x95,0x62,0x3a, -0xd2,0xd6,0xf9,0x96,0x50,0x9e,0xb7,0x64,0x20,0x45,0x5a,0x63, -0x18,0x28,0x7c,0x45,0x87,0x8d,0xe5,0xe4,0x80,0x08,0x23,0xcf, -0x24,0xdc,0x8c,0x48,0x76,0x2a,0x8f,0xf6,0x0e,0x2c,0xcb,0xcf, -0x69,0x35,0xbf,0x5b,0x5e,0x8e,0x73,0x18,0xb8,0xd5,0x64,0xc6, -0xd2,0xa1,0x74,0xe7,0xeb,0x7e,0xe0,0xf4,0x0c,0xe9,0x02,0x5e, -0xc7,0x12,0x21,0xa6,0xa1,0x66,0xd7,0x2f,0x99,0x2c,0x4f,0x2e, -0x17,0x1f,0xd1,0x93,0xc5,0xc9,0x70,0xf6,0x87,0x15,0x81,0xde, -0x69,0xbd,0x1c,0xfe,0x0a,0xeb,0x12,0x3f,0x26,0xe3,0xaa,0x05, -0x9b,0x97,0x61,0xb6,0xe6,0xcb,0x16,0xc1,0x82,0x06,0x57,0x2b, -0xb4,0x48,0x5b,0xc1,0x3a,0x25,0xf3,0x0f,0x95,0xea,0x9f,0x77, -0x19,0x47,0x88,0x9b,0xc5,0x95,0xc3,0x68,0xda,0x84,0x7a,0x4b, -0x26,0x51,0x43,0x4e,0xa9,0x58,0x6b,0x38,0x1b,0xb5,0x4c,0x41, -0x18,0xee,0xb1,0x5c,0x40,0xb0,0xde,0xa4,0x46,0x3d,0xd0,0xb4, -0x9b,0xf7,0xa6,0x80,0xcc,0xab,0x88,0x71,0xf8,0x49,0xe7,0xbc, -0x76,0xd6,0x6d,0x21,0x56,0x61,0xf2,0x63,0x4d,0x94,0x1a,0x59, -0xd1,0x52,0x8c,0xc1,0x64,0xf6,0x98,0x83,0xee,0x1f,0x93,0xda, -0x8c,0x0f,0x88,0xb0,0x9a,0xcf,0x46,0xe8,0xc1,0x26,0x8b,0xf1, -0x65,0xe9,0x51,0x9c,0xb1,0xb4,0x4d,0xc3,0x88,0xec,0xc8,0xe2, -0x13,0xa6,0xf6,0x1b,0x67,0xfc,0xcb,0x47,0x96,0xb2,0x7e,0xf8, -0x87,0x35,0xb4,0xed,0xf8,0xf5,0x6c,0x5e,0xbb,0xe0,0xfa,0x03, -0xa4,0xe1,0xe9,0x02,0x6a,0xca,0x8a,0x14,0xac,0x49,0xc4,0x67, -0x27,0xd4,0xb9,0x45,0x9e,0xb0,0x3b,0xf2,0xb7,0x95,0x0b,0x92, -0xbb,0x92,0xab,0x70,0xb7,0xb3,0x44,0x10,0xcb,0xb4,0xb9,0x6d, -0xf7,0x80,0xba,0xb8,0x4d,0x65,0x32,0x19,0xc4,0xf1,0x06,0x0e, -0xa8,0xf9,0xc7,0x13,0x2a,0xb4,0xbc,0xcc,0x09,0xff,0x83,0x3f, -0x88,0xc2,0x91,0x6d,0xec,0x78,0xec,0x97,0x97,0x19,0xfc,0x6b, -0xa6,0x55,0x92,0xdd,0x5d,0x54,0x5a,0x41,0x3b,0xc5,0xd5,0xdd, -0x05,0x5c,0xf5,0xe9,0x41,0xfe,0x88,0xda,0xa7,0xa8,0x1e,0x14, -0x95,0xa9,0xd1,0x46,0x59,0xfc,0x6e,0xfc,0xdb,0xdd,0x70,0x0a, -0xb4,0xc8,0x61,0x3e,0x7f,0x44,0xbb,0x64,0x4e,0xf4,0xf8,0x23, -0x25,0xde,0x49,0xab,0x94,0x29,0xfc,0x72,0x7e,0xc8,0x02,0xbb, -0x41,0x86,0x43,0x22,0x8a,0x51,0xf3,0x72,0x7e,0x08,0xf7,0x10, -0xfa,0x4e,0xb0,0xa5,0x3d,0xe9,0x8f,0x2b,0x0e,0xff,0x9d,0x8e, -0x57,0xf9,0xea,0xd0,0xc9,0xb4,0x1d,0xe6,0xab,0x47,0xee,0xe3, -0x11,0x99,0xdc,0x83,0x16,0xac,0x1e,0x05,0xe8,0xce,0x21,0x5b, -0x8c,0x70,0xec,0x80,0xc3,0x4c,0x0e,0x1a,0x9a,0x66,0xfc,0x74, -0xec,0x32,0x8e,0xc1,0x21,0x98,0x1f,0xfa,0x99,0xc7,0x46,0xf9, -0x2f,0xde,0xab,0x32,0xd5,0x50,0x1a,0x48,0x4e,0xf7,0xf5,0xc8, -0xc1,0x82,0xf2,0x9e,0x9d,0x44,0x00,0xfc,0x27,0x43,0x90,0xcb, -0xf9,0xd0,0x9c,0x70,0x50,0xde,0xb2,0x41,0x0e,0x83,0x2f,0x4e, -0x93,0x3e,0x3c,0x32,0x0c,0x12,0x1e,0x96,0x52,0xe6,0x5e,0x69, -0xd4,0x43,0xbc,0x00,0x12,0x37,0xbf,0x48,0x31,0x11,0xb5,0xd4, -0x6e,0x07,0x9c,0x90,0x43,0x9e,0x5e,0x1e,0x22,0x37,0x9f,0x4c, -0x2b,0xcf,0x1f,0x49,0x62,0x54,0x15,0x2f,0x8d,0x9d,0x83,0x8d, -0xad,0xef,0x11,0xd6,0xb7,0xc3,0x19,0xb6,0xad,0xb6,0x86,0xc6, -0x48,0x1d,0x3b,0x81,0x79,0xb7,0xa0,0x85,0xb5,0x46,0xe8,0xf2, -0xd8,0x04,0xb3,0x6a,0x47,0x04,0x12,0x52,0x65,0x19,0x1d,0xaa, -0x00,0x11,0x4d,0x82,0x84,0xef,0x3d,0x57,0x42,0xf8,0x3b,0xc8, -0xa0,0x11,0x8e,0xc4,0xd7,0x0c,0x84,0x6b,0xc3,0x46,0xa3,0xec, -0xed,0x46,0xce,0xd1,0xe8,0x26,0x3c,0xcc,0xe9,0xe7,0x11,0x23, -0x11,0xc6,0xc1,0x1e,0xa6,0xa9,0x1b,0x35,0xe2,0x6a,0x32,0xe6, -0x11,0xe4,0xb6,0xf4,0x22,0x66,0x41,0x0e,0x26,0x12,0x72,0x15, -0x6b,0x38,0x3f,0x0f,0x40,0x06,0x40,0x82,0x2c,0x24,0x46,0xe3, -0x20,0x5d,0x88,0x3f,0xab,0x2b,0x27,0x11,0x8b,0xce,0xb4,0x3f, -0x87,0x8c,0x65,0xf5,0xda,0xf5,0xa8,0x17,0x54,0x2c,0x14,0xe4, -0xa3,0xac,0x70,0xd8,0x96,0x9d,0x2a,0x00,0x88,0xfe,0xf9,0xdc, -0xe5,0x0a,0x9b,0x92,0x37,0x46,0xef,0x6a,0x56,0xb8,0x5a,0x94, -0x39,0x5b,0x27,0xa6,0xe2,0xea,0xa1,0x98,0xa0,0x96,0xd4,0x5e, -0x0f,0xa1,0x69,0xdc,0xb4,0x71,0xe8,0x8c,0xa2,0xd8,0x52,0x30, -0xdb,0x34,0x3c,0x46,0x7f,0xa1,0x56,0xa3,0x3d,0x05,0x09,0x54, -0x19,0xc6,0x5b,0xd6,0x06,0xb3,0x78,0xdf,0x32,0xcb,0x78,0x19, -0x3a,0x05,0x65,0xc7,0xa9,0xf4,0x29,0x23,0xb6,0xff,0xcb,0xf9, -0x6e,0x47,0x9e,0x63,0x71,0x63,0xe0,0x3b,0xa4,0x5f,0xfc,0x85, -0xdb,0xf0,0xde,0x71,0x68,0x99,0xec,0xfe,0xfc,0xfc,0xd5,0x93, -0xd7,0x3f,0x8b,0xbf,0xd0,0x76,0x5b,0xc9,0x02,0x75,0x90,0x2e, -0xb9,0x9e,0xbc,0x3e,0xf9,0xe9,0xe5,0xd3,0x57,0x67,0xc4,0x87, -0xa6,0x13,0x45,0x1d,0xb6,0xa7,0x9c,0xe1,0xf4,0xe4,0xf5,0x9b, -0xa7,0x09,0xe1,0x63,0x71,0xf3,0x09,0x5d,0x5e,0x40,0x13,0x57, -0x9f,0x1f,0xa2,0x59,0x21,0x57,0xcf,0x37,0xb6,0xce,0xf8,0xcf, -0xc3,0xc1,0x10,0x88,0xbd,0xa4,0xd5,0x4a,0x0a,0x7e,0x1b,0xf2, -0xec,0xaf,0xe5,0xfc,0x96,0x6a,0x46,0xdb,0x53,0xe6,0xcd,0x42, -0x8e,0x49,0x8a,0xed,0x51,0x0d,0xc4,0x37,0xe0,0x88,0x02,0xff, -0xda,0xb6,0xff,0xc0,0xcf,0x8d,0x4a,0x86,0xc1,0xee,0x95,0x31, -0x3a,0xd8,0x91,0xc1,0x71,0x9b,0x9e,0x66,0xa3,0x9c,0xa2,0xc9, -0xed,0x4b,0x34,0xcf,0x9c,0x24,0x1d,0x41,0x63,0xf9,0xc9,0x6c, -0x27,0x4d,0xae,0xa0,0xa7,0x1f,0xd6,0xeb,0xe4,0x40,0x7e,0xaf, -0xe0,0x18,0x18,0x4b,0x78,0x36,0xd7,0x80,0xfc,0xf6,0x91,0x0a, -0xfa,0xc0,0x18,0x55,0x4b,0x40,0x1c,0x1e,0x05,0x9c,0x01,0xbf, -0x81,0x71,0x3b,0x42,0xb7,0xce,0x86,0xd7,0xea,0x43,0x38,0xbb, -0x97,0x40,0xf9,0xfe,0xb7,0x17,0x40,0x80,0xb8,0xf8,0xee,0xe5, -0x14,0x8d,0x9c,0xb2,0xa5,0x14,0x89,0x64,0x67,0x76,0xc8,0x43, -0x72,0x06,0x52,0x36,0xd8,0x9f,0xb3,0xa7,0x7f,0x3f,0x7b,0xf7, -0xea,0xf5,0x93,0xa7,0xe5,0xb7,0xf4,0x72,0xf8,0x23,0x64,0x53, -0x78,0xb1,0xab,0x3b,0x07,0x9f,0x60,0x31,0xf8,0xd0,0xe1,0x71, -0xff,0x60,0xe0,0x58,0xe9,0xb0,0xb6,0x1c,0xf0,0x9e,0x1f,0x34, -0xad,0x84,0x9f,0xc8,0x31,0x24,0xe6,0xd0,0xf7,0x7f,0x49,0xbf, -0xef,0xff,0xf2,0x78,0xd0,0x79,0x9c,0xed,0x67,0xe8,0x2c,0xc1, -0xbd,0xa1,0xfc,0xf2,0x3d,0xba,0xa6,0x3c,0xdf,0x1b,0x74,0xb2, -0xfd,0xe8,0x9d,0x27,0x77,0x1e,0x8d,0x75,0xcd,0x7d,0x9f,0x74, -0x7c,0x65,0x2e,0x75,0x63,0xfa,0x2d,0x4d,0x6e,0x68,0x92,0xdd, -0x93,0xcb,0xcf,0x4f,0xc6,0x5b,0xbc,0xe5,0xde,0x9b,0x8e,0x8f, -0xb6,0x65,0x32,0xe3,0xec,0xa1,0xd2,0x42,0x04,0xec,0xf4,0x6f, -0xe4,0xcd,0xf7,0x57,0x09,0x38,0x66,0xf1,0x3b,0x87,0x1d,0x23, -0x31,0x87,0x37,0x72,0x44,0x92,0xba,0x22,0x6c,0x45,0x50,0x2c, -0x2d,0xb4,0xad,0xb3,0x56,0x0f,0x14,0x96,0xb2,0x7e,0xc0,0x06, -0x76,0xe0,0x4b,0x8d,0xd4,0xe2,0x25,0x14,0xc7,0xbb,0xac,0xb9, -0xcb,0xae,0x1c,0x99,0x58,0xf3,0x3c,0x26,0x3c,0x7a,0x32,0x7f, -0x42,0x05,0x09,0x52,0xe0,0x70,0x90,0xf5,0xbe,0x04,0x14,0xf2, -0x14,0x80,0xa9,0x29,0x7f,0xbd,0x67,0x1e,0x5d,0xf8,0xce,0xec, -0x69,0x88,0xc9,0x11,0x94,0x15,0x29,0x34,0xaa,0xec,0x3b,0xbe, -0x3c,0xc4,0x0f,0x7c,0x3c,0x9e,0x4f,0x40,0x6a,0x07,0x0c,0xfa, -0xd5,0xdc,0x0d,0xbc,0x63,0xd0,0xb3,0x29,0x94,0xaf,0x22,0xfd, -0x99,0x49,0x1d,0x30,0x8d,0x7a,0x9a,0x9d,0x73,0x5c,0xf9,0x07, -0x41,0xc8,0x23,0x93,0x40,0x35,0x70,0xd3,0xe5,0x39,0x6c,0x39, -0x61,0x99,0x11,0x9c,0xe5,0x9d,0x83,0xac,0x63,0xb2,0x8b,0xe8, -0x42,0x52,0x98,0xd7,0x33,0xcd,0xee,0x73,0x91,0xa8,0x16,0x0a, -0xb6,0x7e,0x05,0xec,0x0f,0x5f,0x02,0xfb,0xc1,0x42,0x64,0xd3, -0x30,0x7a,0x8f,0xf1,0x87,0x73,0x81,0x5c,0x24,0x89,0xc1,0xdc, -0x15,0xca,0xe9,0xf8,0xfa,0x56,0x16,0xbd,0xe7,0xb7,0xd6,0xaa, -0xc0,0x0a,0xfc,0x7e,0xfe,0xd7,0x47,0x7f,0xda,0xbf,0x9e,0xe4, -0x81,0x5f,0xe5,0xfd,0x7f,0xfd,0xf6,0x09,0x45,0x96,0x41,0xe4, -0xa3,0x1f,0x28,0xb2,0x63,0x79,0xdf,0x0d,0xe0,0x17,0x97,0xab, -0xa7,0x14,0x7b,0xba,0x80,0x72,0x55,0xdc,0x92,0x68,0x77,0xda, -0x0a,0xbe,0x3b,0xa0,0x0a,0xfe,0x67,0xd8,0x94,0x63,0x8a,0x2c, -0xc2,0xa6,0x7c,0xb7,0x7f,0x9d,0x03,0x0e,0x1a,0xc4,0x9d,0x50, -0xc6,0x3c,0x8c,0x44,0x27,0xfa,0x51,0x8b,0x7a,0x09,0x44,0x03, -0xc0,0xa0,0x23,0xf2,0x48,0xfb,0x7c,0x06,0xbb,0x50,0x0e,0xd0, -0xfc,0x62,0x3e,0x5f,0xc1,0x85,0x88,0xae,0x46,0x7c,0xbe,0xf0, -0xf8,0xd5,0x23,0x9d,0x25,0xc7,0x09,0xf3,0xe0,0x35,0x20,0xe9, -0x74,0xae,0x43,0x19,0x95,0x22,0xe1,0x27,0xde,0xdc,0x39,0xb0, -0xef,0x4b,0x68,0x40,0x6a,0xe6,0x68,0xc2,0x65,0x76,0x5d,0x40, -0x7e,0xe8,0xc6,0xec,0x7a,0x8f,0x03,0x9f,0xf6,0x5c,0x10,0x95, -0xc9,0xf4,0x6b,0x90,0xbf,0xfa,0xeb,0xbb,0xe3,0x37,0x6f,0xde, -0x9d,0xbc,0x38,0x3e,0x3d,0x7d,0xf7,0xf6,0xe9,0x5f,0x9f,0xfe, -0xfd,0x4d,0xb9,0x7f,0x5e,0xcd,0xae,0xfb,0x05,0x1c,0xd0,0x90, -0x27,0x2d,0xce,0xab,0x6f,0xe8,0xbc,0x1e,0xbd,0x83,0x03,0xfb, -0xa8,0x97,0xf5,0xce,0xab,0x7d,0xc7,0x14,0xa0,0x2a,0x43,0x91, -0xa5,0x8c,0x7c,0xb0,0xab,0x33,0x17,0xf4,0x2f,0xc9,0xfd,0x75, -0x12,0x35,0x40,0x3a,0x49,0x5f,0x7e,0xf8,0xfc,0x7c,0xc4,0x65, -0xb2,0xdc,0xbb,0x9c,0xd1,0xa1,0x87,0xf1,0xcd,0x93,0xf3,0x73, -0x98,0x36,0xed,0x6c,0x97,0x44,0x3c,0x4e,0xc5,0xfb,0xea,0xf1, -0x14,0x7d,0x45,0x69,0x4b,0xb6,0x65,0x49,0x93,0x2e,0xcb,0xc2, -0x64,0xd2,0x10,0x38,0x5c,0xbf,0xb6,0x48,0x47,0xaa,0xff,0xfa, -0x82,0x7d,0x2d,0x38,0xf0,0xc5,0x70,0x27,0x46,0x25,0x23,0x19, -0x25,0x5a,0x06,0x48,0x68,0xf8,0x69,0x16,0x5c,0x7c,0x3a,0xac, -0xc8,0x6f,0x4e,0x99,0xb4,0x60,0xaf,0x4b,0xb5,0x2e,0xb6,0x03, -0xb1,0x39,0x3b,0x1b,0x6a,0x98,0x48,0x36,0x06,0xe5,0x32,0x67, -0xec,0x96,0xa8,0x97,0xfa,0x4a,0xca,0x71,0xb0,0xa6,0xca,0x54, -0x3d,0x18,0xf1,0x7d,0xe4,0xc5,0x32,0xaa,0x0e,0x6e,0x16,0xd8, -0x17,0x80,0xf5,0x46,0x63,0x30,0x5c,0x01,0x7e,0x76,0x71,0xb7, -0xb2,0xcb,0x00,0xe3,0xb2,0x7b,0xd3,0x9b,0x76,0x9b,0xd7,0x04, -0x26,0x90,0x93,0x77,0x34,0x42,0xba,0xbd,0x1d,0x94,0x4d,0x3d, -0xc0,0xe3,0xc9,0x6f,0x21,0xb9,0x2d,0x95,0xd6,0x36,0x45,0xaf, -0xcf,0xbf,0x03,0x20,0x48,0x2c,0x8d,0xe3,0x4a,0x84,0xf5,0x54, -0x2a,0x10,0xf6,0x83,0x66,0x28,0x43,0x29,0xe7,0x2d,0x08,0x99, -0x5b,0x8e,0x93,0xd9,0x7b,0x9a,0xfa,0x54,0x6e,0xce,0xd5,0xf0, -0xba,0xf4,0xa8,0x13,0x72,0x07,0x65,0xcd,0xf7,0x12,0x0d,0x25, -0x45,0x13,0xea,0x77,0x14,0xb3,0x0e,0x2f,0xb0,0x3d,0x8b,0x11, -0x0c,0xf9,0xf1,0x62,0xd1,0x3a,0x9e,0x92,0x15,0xb5,0x96,0x6b, -0xe7,0x62,0x3c,0x6a,0xa1,0xd5,0x4d,0x12,0x9d,0x6d,0xc9,0x20, -0xb4,0xfe,0x70,0x7f,0xb0,0xf9,0x43,0x92,0x43,0x2b,0xb2,0x8d, -0xf4,0xb0,0x94,0x5f,0x32,0xa6,0x24,0x61,0xf7,0x20,0xd9,0x4f, -0x76,0x17,0xcb,0xf9,0x07,0xd4,0xe5,0xf4,0x73,0xa7,0x51,0xd9, -0xbd,0x86,0x78,0x26,0xd2,0x64,0x77,0x09,0xd5,0x4b,0x5d,0x49, -0xee,0x8e,0xa7,0x41,0x56,0x03,0x0c,0x07,0x8f,0xa8,0x44,0xe8, -0x08,0x95,0x6c,0xbd,0xec,0xb9,0x0e,0x98,0x4e,0x80,0xe7,0x18, -0x73,0x02,0x0c,0xe9,0x87,0xf9,0xaf,0x63,0x6c,0x1a,0xd6,0x46, -0x54,0x0b,0x0c,0x42,0x58,0x75,0xb2,0x8b,0xca,0x7f,0x13,0xb2, -0xb9,0xb1,0xab,0x25,0x31,0x2c,0x2a,0x4b,0xd6,0x80,0x34,0x02, -0xd0,0xb6,0xe6,0x52,0x2c,0xd7,0x32,0x6a,0x26,0x02,0xad,0x71, -0x43,0xbe,0xee,0x2e,0x13,0x7a,0x66,0x11,0xe8,0x4c,0x93,0x5a, -0xb5,0xad,0x4c,0x43,0x99,0x02,0x75,0x73,0xc9,0x75,0xe2,0xa6, -0x17,0xe8,0xdd,0x31,0xc9,0xc2,0x8d,0xc8,0xd8,0x02,0x22,0x5d, -0x52,0x74,0x83,0xa7,0xef,0x93,0xa7,0xcf,0x9e,0xbe,0x7d,0xf7, -0xc3,0xeb,0xd7,0x67,0xa7,0x67,0x6f,0x8f,0xe1,0xf0,0xfd,0xa5, -0x1e,0xb9,0xb3,0xaf,0xc3,0xc4,0xb2,0x6a,0xed,0xf6,0x4e,0x3d, -0x13,0x53,0x90,0x22,0x56,0x47,0xe7,0x5d,0xcf,0x2c,0x6c,0x14, -0x8c,0x37,0x69,0xa5,0x09,0xbb,0x6d,0x5e,0x87,0x49,0x46,0x6d, -0xe4,0x5a,0x43,0xab,0x8d,0xb0,0x82,0x1b,0xb6,0x0a,0xbd,0xad, -0xbe,0xd4,0x1d,0xe5,0xce,0x07,0x13,0x6b,0xb0,0x7c,0x11,0x9a, -0xd3,0x15,0x43,0x37,0x9d,0x44,0x6e,0xd0,0x72,0x86,0x69,0xf1, -0xc6,0x09,0xd9,0x7a,0xb2,0x69,0x36,0xfc,0x75,0xfc,0x8e,0x69, -0x01,0x32,0xa7,0x3a,0x46,0xa7,0x41,0x38,0x0d,0xde,0xcd,0xad, -0xc4,0x94,0x2e,0x04,0x87,0xd9,0xbb,0x24,0x0f,0xba,0x7a,0xfa, -0xea,0xf8,0x3f,0x9e,0xbe,0x3b,0x39,0x3e,0x7d,0x2a,0x67,0xa5, -0x6f,0xe1,0x14,0x08,0x54,0x24,0x54,0xe7,0x0e,0xed,0x48,0x21, -0xdc,0x73,0xd0,0x48,0xae,0x8a,0x33,0x85,0x0e,0x9b,0x36,0x11, -0x57,0xe5,0xdf,0x19,0x9f,0x51,0xe1,0xda,0x32,0x90,0x7f,0x14, -0x89,0xe0,0x5e,0xca,0x47,0x8b,0x4a,0x41,0x8e,0x03,0xdf,0xba, -0x57,0xb3,0x9c,0x17,0x66,0xf1,0xef,0xff,0x89,0xb9,0xde,0x38, -0xfd,0x4f,0x5e,0xd6,0xf8,0x2e,0x85,0xa2,0xa2,0xe3,0x65,0x2d, -0x83,0x4f,0x72,0xcb,0xad,0x96,0xc7,0xed,0x02,0x79,0x6d,0x1f, -0x8f,0xc8,0x23,0x74,0x3d,0x9b,0x49,0x85,0x39,0x92,0x74,0xbc, -0x29,0xb8,0x8b,0xf2,0x4e,0x94,0xb0,0x41,0x7d,0x14,0xfa,0xa7, -0xff,0x0f,0x1f,0xc8,0x4a,0xb6,0x44,0x20,0xe7,0x21,0xff,0xff, -0x40,0x4e,0xf6,0x10,0x2d,0x19,0x01,0xad,0x2d,0x64,0xc0,0xb8, -0x80,0x5b,0x9c,0xe1,0x49,0x6d,0x70,0xb2,0x0a,0x8d,0xf3,0x1f, -0x2f,0x49,0x02,0x2f,0x17,0x9f,0x73,0xc3,0x4a,0x5e,0x71,0x76, -0x50,0x0c,0x2f,0x3e,0x7a,0x87,0xcb,0xf1,0x6f,0x78,0xee,0x0a, -0xa3,0x87,0x4f,0x55,0x7c,0x17,0xba,0x3f,0xdc,0xf0,0x32,0x82, -0x05,0xd5,0x4b,0x04,0x0e,0x84,0xd1,0x8f,0xc9,0x64,0x89,0xc6, -0xc2,0x8f,0x3c,0x0f,0xa9,0xa1,0x09,0xcf,0x66,0xbe,0x11,0xec, -0xd6,0x83,0xa8,0x8f,0xe3,0xd9,0x6c,0xbe,0x92,0x97,0x25,0x2d, -0xdf,0x94,0x8a,0x8c,0x65,0x26,0x57,0xb0,0xd5,0x78,0x7f,0x2e, -0xaf,0x91,0x45,0xd7,0x47,0x59,0x45,0x15,0x46,0x83,0xe3,0xc5, -0x77,0x39,0x12,0x6a,0xe4,0x9a,0xd1,0xf3,0x6c,0x6b,0xe8,0x64, -0x17,0xf2,0xd6,0x35,0x7c,0x27,0x48,0xb6,0x5c,0x37,0x48,0x05, -0x40,0x2c,0xc9,0x42,0x1a,0x43,0x76,0x5d,0x19,0x02,0x7e,0xc5, -0x48,0x0a,0x9f,0x13,0xb0,0xb8,0xa0,0xe7,0xd7,0xb4,0x4b,0x98, -0x93,0x34,0x5c,0xdd,0xe4,0xb8,0x2b,0x9e,0xcd,0xce,0xe6,0x74, -0xaa,0xf3,0x04,0x60,0x7c,0xe6,0x1f,0xa3,0xed,0x73,0x05,0x49, -0x25,0x95,0x98,0x41,0x29,0xdb,0x2e,0x9c,0x46,0xe8,0xb5,0xe9, -0xb9,0x30,0x14,0x51,0xd4,0x0f,0x2d,0x53,0x94,0x46,0xe4,0x89, -0x84,0xa0,0x20,0xf8,0x78,0x42,0xef,0x5f,0x48,0x08,0x8b,0x74, -0x53,0x4e,0x8f,0xdd,0xf4,0x08,0x9a,0xc6,0x50,0xb2,0xe0,0x55, -0x63,0x27,0x68,0x28,0x8e,0x7b,0x28,0x37,0x4b,0x3c,0x53,0x0b, -0x82,0xde,0x6a,0x8b,0x50,0x7e,0x0c,0x7d,0x2f,0xf0,0xf1,0xf7, -0x62,0x4e,0x06,0x8d,0x45,0xf0,0xf8,0xde,0x10,0x3e,0xd5,0xdd, -0x92,0xa8,0x57,0x9e,0x99,0xab,0x21,0x0e,0xef,0x67,0xfb,0x38, -0xdf,0x17,0x67,0x89,0xa9,0x0b,0x97,0x92,0x0b,0x50,0x8e,0x8d, -0xa3,0x8a,0x08,0x8e,0xfc,0x88,0x38,0x73,0x22,0x3b,0x23,0xc9, -0x79,0x9e,0xb2,0x3c,0xe1,0x63,0x36,0xc9,0x63,0x49,0x65,0xc1, -0x14,0x0c,0x7b,0x22,0xd0,0x40,0x90,0xe5,0x8d,0x56,0xb1,0x67, -0x57,0x93,0x6b,0x23,0xf7,0xa5,0x29,0xed,0xb6,0x1e,0xe7,0x11, -0xa3,0x9f,0xae,0x21,0x76,0x00,0x8d,0xc9,0xd2,0x03,0x36,0xd0, -0x2c,0xcd,0x95,0x24,0x19,0x01,0xa3,0xff,0xe3,0x1f,0xc5,0x11, -0x21,0x78,0x31,0xc4,0x95,0x24,0x08,0xc9,0x52,0x95,0x2e,0xd8, -0xf5,0xc6,0x4b,0xfa,0x78,0x40,0x2a,0x87,0x41,0xfc,0xa7,0xd8, -0xda,0xf0,0x45,0x60,0x11,0xe3,0xfd,0x93,0x0c,0xd2,0x7e,0x0c, -0xd9,0xb1,0x79,0x1d,0x76,0xa3,0x73,0xbd,0xd9,0xb0,0x97,0x51, -0xee,0xba,0x9c,0x1e,0x22,0x12,0x6e,0x10,0x84,0x0c,0x90,0xa0, -0x39,0x94,0x84,0x93,0x84,0x57,0x81,0x3f,0x47,0x68,0x03,0x7e, -0x18,0x4e,0xa6,0x88,0x15,0xec,0xb4,0xfe,0x6b,0x7e,0xd7,0x1a, -0x03,0x2e,0x37,0x86,0xa9,0x98,0x54,0xd5,0x62,0x8c,0x8e,0x07, -0x00,0xb1,0x1b,0xcb,0xcc,0xb4,0x70,0x68,0xf0,0xa9,0x1a,0xf6, -0x06,0xee,0xd5,0xd5,0xbc,0x85,0x46,0x78,0x5b,0x93,0x55,0xb7, -0xf5,0xfc,0x0a,0x26,0xe1,0x7a,0x52,0xc1,0xe0,0xe0,0x03,0xf7, -0x50,0x8b,0xf0,0xe0,0x02,0x90,0xe1,0xaa,0xf5,0x19,0xe0,0xa3, -0xb7,0xa9,0xc9,0xd5,0x67,0x82,0x3a,0x22,0x53,0x98,0x40,0x4a, -0xe3,0x5b,0xf8,0xb0,0xa2,0xb8,0x6a,0x0c,0x93,0x3b,0x72,0xbc, -0xef,0x2e,0x9f,0x73,0x8a,0xc2,0xb9,0xd1,0x23,0xd7,0x23,0x77, -0xb3,0x1f,0xa6,0x73,0xf1,0x43,0xc2,0x4b,0xa2,0xb4,0x73,0x14, -0x60,0x64,0x9c,0x90,0xc4,0x63,0x58,0xde,0xbf,0x33,0x50,0x0b, -0x13,0xce,0xdf,0x39,0xf8,0x85,0x0b,0xb9,0x25,0x58,0xb8,0xb5, -0x88,0xcd,0x2b,0xd4,0xac,0x08,0xcd,0x5c,0x11,0x36,0xc2,0x21, -0xb6,0x89,0x66,0x20,0x03,0x27,0xb4,0x77,0xb6,0x66,0x95,0x74, -0x34,0x93,0x32,0x5e,0x7e,0x98,0x5c,0x8e,0xb7,0xe6,0x94,0xf4, -0x44,0x64,0x19,0xb6,0xe6,0xa3,0x54,0xd4,0xcb,0x9d,0x53,0xd7, -0x57,0x5b,0x33,0x6a,0x06,0x08,0x0a,0x16,0x9d,0x64,0xde,0xaa, -0x70,0x54,0x4c,0x10,0xcb,0x37,0xda,0xb3,0x3c,0xd1,0x55,0x80, -0x9d,0x64,0x6b,0xa8,0x61,0x09,0x8e,0x6c,0x2e,0x60,0xd0,0x88, -0xb0,0x90,0x4f,0x68,0x2e,0x38,0x82,0xa9,0xb8,0x44,0xfd,0xe7, -0x5a,0x39,0xc2,0x88,0x4d,0x21,0x97,0x93,0xab,0x83,0x35,0x53, -0xf0,0x4f,0x68,0xb1,0xf3,0x02,0xa7,0xdb,0x1f,0x2f,0x3a,0xff, -0x8c,0x25,0x72,0xa2,0x7a,0x2a,0x95,0x3c,0x7a,0x26,0xa1,0x79, -0x6d,0x0c,0xa5,0xee,0x90,0x8a,0xb7,0x2d,0x69,0xa1,0x78,0xce, -0xed,0xdd,0xc5,0x74,0x52,0xdd,0x3c,0x05,0x7c,0x6b,0x39,0x1b, -0x4e,0x8f,0xdf,0x3c,0x4f,0xe5,0xc0,0x04,0x9c,0x9f,0x91,0x30, -0xf9,0xce,0xef,0x1d,0xf9,0x58,0xb8,0x50,0x8e,0xaf,0xf4,0x05, -0xfe,0x11,0x94,0xad,0xe0,0x9f,0x9c,0x5f,0x0a,0x0b,0xfe,0x51, -0xba,0x43,0x70,0x16,0x65,0x02,0x28,0x09,0xed,0x31,0xb3,0x90, -0x38,0xca,0x51,0x38,0xb1,0x20,0x3b,0xf0,0x78,0xd3,0x14,0xf8, -0x47,0xde,0x51,0x44,0x7d,0x20,0xd7,0xb7,0x92,0x42,0x03,0xb9, -0xca,0x2f,0x16,0x1a,0xc8,0x8d,0xec,0xa1,0x55,0x52,0xf0,0x3c, -0xdf,0xc2,0x85,0x72,0x7d,0x65,0x71,0x0a,0x0b,0x46,0x21,0xa2, -0xf0,0xc1,0x5c,0x65,0x17,0x9c,0x10,0x43,0xae,0x12,0x7f,0x85, -0x06,0x72,0x27,0xe9,0x55,0xb8,0x50,0x2e,0x98,0x8b,0xca,0xe0, -0xe7,0xbb,0xbb,0x7c,0x62,0x16,0xfc,0x93,0x8b,0x3e,0x43,0x21, -0xbf,0x39,0x3f,0xbd,0x8b,0x5c,0x46,0xee,0x58,0xfb,0x85,0x0b, -0xe5,0x77,0xe8,0x55,0x93,0xe2,0x5c,0xc8,0xd9,0x00,0xaa,0xd0, -0x36,0xcf,0x1d,0x0a,0xc0,0x15,0x07,0xc4,0x3d,0xe0,0x99,0xe4, -0x73,0xb8,0xdc,0x7a,0x33,0xd3,0x03,0x48,0x90,0x17,0x09,0x59, -0x32,0x2e,0x3f,0xb6,0x0f,0x2d,0xdb,0xb2,0xa0,0x28,0x49,0x57, -0x4f,0x1b,0xd8,0x06,0x53,0x89,0xde,0xe5,0x74,0xdd,0x0e,0xd9, -0xa6,0x06,0x00,0x8a,0x7a,0x30,0x83,0xfc,0x6b,0xe9,0x72,0x53, -0x97,0x23,0x88,0x77,0x4f,0xc2,0xcd,0x97,0x75,0xdd,0xde,0x4b, -0xef,0x87,0x05,0xa2,0xd3,0xc7,0xb3,0xcb,0x9b,0xf9,0xf2,0x89, -0x46,0xc3,0x32,0x5c,0xdc,0xe1,0xc1,0x04,0x7f,0x7d,0x24,0x0a, -0xbc,0x01,0x26,0x3c,0x8c,0xe3,0xd1,0x9d,0x11,0x2e,0xe0,0x5b, -0x1f,0x25,0xbe,0xe2,0xf8,0xc7,0x44,0x13,0x5f,0xac,0xe0,0x1f, -0x13,0x8d,0x9a,0x65,0x05,0xfd,0xf5,0x91,0xac,0xab,0x5e,0xf0, -0x8f,0x8f,0x9e,0x5d,0xff,0x80,0x1b,0x80,0x7f,0xe2,0x68,0x7c, -0xe2,0x29,0x7c,0x30,0x4e,0x3e,0x13,0x3d,0x9f,0x22,0xfc,0xb4, -0xd9,0x4e,0x90,0x35,0x56,0xc8,0x6f,0x2d,0x01,0x5d,0x28,0x16, -0x26,0x5c,0xcb,0xf0,0x7a,0x34,0x2a,0x7c,0x30,0x48,0xae,0x16, -0x05,0xfd,0x0d,0xcb,0xcc,0x87,0xbf,0x16,0xf2,0x1b,0x24,0xf8, -0xf3,0xd7,0x7e,0xd8,0x2c,0xcf,0x70,0xd4,0xf9,0xc7,0x46,0xff, -0x08,0x33,0x5c,0xf0,0x8f,0x8d,0x7e,0x7e,0x55,0xe0,0x9f,0x20, -0x8a,0x85,0x12,0x0b,0x17,0x0a,0x13,0x27,0xab,0x82,0x7f,0x6c, -0xf4,0xab,0xf9,0x0c,0x47,0x0e,0xb1,0x94,0x22,0xf8,0xb2,0x99, -0xde,0x4c,0xef,0x96,0xc3,0xe9,0xe4,0x1f,0x98,0xc5,0x85,0x6d, -0x86,0xb7,0x80,0x6e,0x0c,0x11,0x3c,0x07,0x6c,0xd2,0xe9,0xcd, -0xfc,0x63,0xc1,0x3f,0x41,0x34,0xad,0x10,0xf9,0x0d,0x12,0x3e, -0x4e,0x60,0x0b,0x16,0x1a,0xa8,0x27,0xfd,0x7c,0x43,0x53,0xe6, -0x3f,0xea,0x59,0x9e,0x88,0xad,0x9c,0xe8,0xdb,0x66,0x7c,0xcd, -0x86,0x13,0x0a,0x17,0xb2,0x89,0xe4,0x82,0x46,0xc7,0xd2,0x7f, -0xd8,0x2c,0xb0,0xaf,0xc7,0xb8,0x32,0xe9,0xd7,0x26,0xbc,0x80, -0x8b,0xb3,0xe0,0x9f,0x60,0xfe,0xc9,0x2f,0x42,0xa1,0x01,0x9f, -0xa4,0x84,0xa5,0xa2,0x3d,0xa3,0x70,0x5c,0x1f,0x48,0xa4,0x87, -0xa5,0x42,0x7e,0x5d,0xc2,0xc6,0x1e,0x06,0xb3,0xeb,0x63,0xe5, -0xdc,0x1e,0x4f,0x27,0x43,0xdf,0xc9,0x2a,0xcc,0x45,0x9e,0x44, -0x4d,0x62,0x5e,0x3f,0x7e,0xee,0x01,0x21,0xc1,0x23,0xe5,0x94, -0x14,0x57,0x8b,0xdd,0x63,0xf3,0xa5,0xe7,0x50,0xae,0x48,0x0b, -0x26,0x07,0xd8,0x4b,0xbe,0x7b,0x01,0x28,0x34,0x20,0x54,0xc5, -0xee,0x0f,0x1c,0xf0,0x29,0xe4,0xd5,0xf1,0x99,0x60,0x6d,0xbb, -0x27,0xe6,0xcb,0xe4,0xf1,0xdb,0x67,0xf7,0xa4,0x86,0xb1,0xe4, -0xbb,0xca,0x79,0x2d,0x76,0xd5,0x1e,0x8f,0x4f,0x1b,0xab,0xa7, -0x8e,0x1f,0xc9,0x7b,0x05,0x40,0x78,0x1a,0xc5,0xf8,0xbc,0x82, -0x57,0xed,0x3e,0x0b,0x50,0xa9,0x7c,0x97,0xac,0x3d,0x2d,0xe6, -0x74,0xd6,0xec,0x3e,0xf7,0x1f,0x3e,0x07,0xe9,0xf2,0xed,0xa2, -0x23,0x9c,0x30,0xee,0x07,0xb8,0xab,0x10,0x77,0xa0,0x24,0xf9, -0xf0,0x39,0x54,0xba,0xbb,0xd8,0x55,0xa7,0xe8,0x36,0xed,0x1a, -0xa3,0xaf,0x7d,0x0c,0x3d,0x45,0x17,0xbb,0x6f,0xf0,0xc7,0xc7, -0x3a,0x1e,0x6a,0xb1,0xfb,0x56,0x83,0x3e,0xf5,0xb7,0x62,0xf7, -0x3f,0xfd,0x57,0x05,0xe8,0xee,0xee,0xe9,0xe5,0x38,0x88,0x79, -0x02,0xa7,0xf7,0x35,0xf5,0xec,0xd4,0x7f,0x98,0x1c,0xb3,0xc9, -0xd5,0x15,0x0e,0xca,0x29,0x07,0x7c,0x8a,0xaa,0x59,0xd2,0x9c, -0x15,0xbb,0x67,0xf6,0xd3,0xe4,0x62,0xf7,0x25,0x90,0xce,0x01, -0x9f,0xc2,0x77,0x72,0xb1,0xcb,0xd2,0x2f,0x3e,0x7e,0xf7,0x6e, -0x39,0xfd,0x69,0x35,0x01,0x1c,0x6b,0x77,0xf7,0x27,0x09,0x6a, -0xea,0x06,0xd9,0xab,0x1e,0xc7,0x7b,0xff,0xdb,0x2b,0xb8,0xc4, -0x9e,0x8f,0x1c,0x21,0xd8,0xe9,0xbc,0xff,0xed,0xf9,0xc8,0x48, -0x66,0x01,0xfd,0x30,0x3d,0x51,0x66,0xa2,0x97,0x49,0x0f,0x38, -0x85,0x6f,0x9e,0x9e,0x3c,0x3f,0x7e,0xf1,0xee,0xe4,0xc7,0xe3, -0xb7,0xa7,0x35,0x66,0xe1,0x3b,0xcf,0x81,0xcc,0x85,0x71,0xc8, -0x1a,0xd9,0x9e,0x88,0xa7,0xcf,0x9e,0x63,0x18,0x1a,0x6f,0xe0, -0x05,0x47,0x6e,0xfc,0x43,0xcb,0xcb,0xd7,0xff,0xd7,0xbb,0x1f, -0x8f,0x4f,0xfe,0x43,0xeb,0x01,0xd2,0xf1,0x1f,0xbb,0x87,0xf6, -0x8d,0x71,0x1b,0x9f,0x8c,0x48,0xa1,0xd1,0xa4,0x5a,0x60,0xca, -0x19,0x29,0xc1,0xd3,0x32,0x45,0x04,0xac,0xca,0x99,0x11,0xf3, -0xfc,0xea,0xd5,0xfc,0xd8,0x89,0x2f,0xdd,0x1b,0xfd,0x2e,0x04, -0x41,0x30,0x53,0xf2,0xc1,0x2e,0x06,0x75,0xc7,0xab,0x9c,0x14, -0x09,0x50,0x68,0x1b,0x02,0x2f,0x98,0xbf,0xe2,0x78,0xe9,0xe8, -0xd8,0x92,0x13,0x29,0x28,0xc9,0x14,0x5e,0x8e,0x67,0x24,0x20, -0x5f,0x9a,0x46,0x90,0x6f,0xfa,0xe1,0x6d,0xaf,0x6f,0x6d,0x3f, -0x73,0x75,0x83,0x82,0x22,0x07,0xe4,0x45,0x87,0xce,0x9b,0xd2, -0x76,0x05,0xe5,0xf7,0x76,0x1a,0xba,0xa0,0x16,0x5f,0x19,0x08, -0x72,0x8e,0x8e,0xb0,0x52,0x15,0x7f,0xa6,0x18,0xb4,0xd1,0x42, -0x91,0xfc,0x30,0x91,0xb9,0x1e,0x95,0x07,0xbe,0x4f,0x25,0x1b, -0x33,0xa1,0x52,0x2e,0xf2,0xb1,0xe6,0x3c,0xd2,0x40,0xa7,0x43, -0x20,0xa3,0x47,0x21,0x48,0xed,0x6b,0x8e,0x41,0xe6,0xbb,0xd0, -0xd3,0xa7,0x03,0xf1,0xbe,0x2c,0xe7,0x09,0xe0,0x6a,0x28,0xc8, -0xbb,0x9c,0x03,0x95,0x59,0xf8,0xfe,0xee,0xb8,0xa0,0x19,0x58, -0x68,0xa3,0x19,0xda,0x32,0xd5,0xb1,0x2d,0xdd,0x43,0xa0,0x44, -0xa4,0x99,0x1a,0x54,0x3b,0x32,0x05,0x1e,0x7b,0x48,0x47,0x3e, -0x08,0xbd,0xa0,0x01,0x21,0x9a,0x4a,0xec,0x5d,0x28,0x9c,0xbe, -0xcf,0x66,0x94,0x78,0xd5,0x39,0xe0,0xbf,0xff,0xb6,0xcd,0x90, -0x38,0x3d,0x51,0xdb,0x6c,0x35,0x0b,0xe7,0x41,0x62,0x00,0x70, -0x57,0xbf,0xd6,0x6b,0x1b,0x9f,0x9b,0x55,0xe9,0xf3,0x04,0x45, -0xf3,0xd8,0x94,0xa8,0x29,0x12,0xef,0x98,0xe0,0x99,0x55,0xc2, -0x56,0x30,0x91,0x73,0x65,0xa1,0xf8,0x13,0x2d,0x3c,0xea,0x6b, -0x43,0x56,0x11,0x1a,0x16,0xe9,0x34,0x05,0xdf,0x6e,0x27,0xdf, -0x27,0x3b,0x66,0x82,0x44,0xb2,0x55,0xf8,0x44,0xbc,0x66,0x94, -0xd3,0x3c,0x9b,0x03,0x72,0x0c,0xd4,0xf1,0x8b,0xf9,0x9c,0xf4, -0x13,0xee,0x16,0x5a,0x73,0xd5,0xfa,0x30,0x19,0xb6,0x2a,0x79, -0x59,0xae,0x6b,0x25,0xb4,0x2e,0x3e,0x0b,0xac,0x9d,0xd6,0xe9, -0x78,0x5c,0xb4,0x44,0x5d,0x1c,0xee,0xba,0x58,0x59,0x7c,0xb8, -0x98,0xec,0x47,0x5c,0xf2,0x24,0xd0,0xcf,0x8e,0x06,0x68,0xd3, -0xd4,0x2d,0x79,0x28,0x9d,0x7c,0xd8,0xea,0x82,0x18,0xd2,0x00, -0x2c,0xfc,0xed,0x92,0x55,0x92,0x1f,0xcf,0x5e,0xbe,0x28,0x59, -0xe0,0xac,0xfd,0x2f,0x87,0x7f,0x3a,0x38,0xfa,0x7e,0x9f,0x84, -0xcf,0xf4,0x0d,0x3b,0xc7,0x9c,0xd6,0x03,0x2f,0x7e,0x1b,0x87, -0xb9,0xf2,0x20,0x70,0x3c,0x1a,0xa1,0x59,0x7e,0xb1,0xeb,0x81, -0x79,0x68,0x85,0x52,0x9c,0xf8,0x7e,0x5a,0x0e,0xaf,0xed,0x76, -0x8c,0xda,0xa7,0x57,0xfe,0x33,0xc9,0x96,0xa2,0xe1,0x5e,0x09, -0xab,0x1c,0x1c,0xfb,0x0e,0x21,0xa9,0xbd,0xa6,0x5a,0xdd,0xc0, -0x44,0x4b,0xea,0x84,0xa4,0xf2,0xdc,0xba,0x0a,0x97,0x0e,0xcb, -0xec,0x21,0x14,0x7a,0xf5,0x8b,0x8a,0x3e,0x19,0x0f,0xe1,0x76, -0xf7,0x65,0x39,0x96,0x8f,0x72,0x7c,0x65,0xf1,0xef,0xc2,0x46, -0xbf,0x23,0x6f,0xde,0xfe,0xd4,0x52,0x7c,0xde,0x3d,0x9a,0x7c, -0xaf,0x39,0xac,0x06,0x48,0x50,0xa3,0xdb,0xe8,0x93,0x41,0xad, -0x51,0xaf,0xaf,0xdc,0xce,0x60,0x03,0x1f,0x57,0xb3,0xfc,0x6e, -0xe6,0x96,0x9c,0x0a,0xc9,0x8b,0x58,0x98,0x4d,0x69,0x5c,0xdc, -0x70,0xf1,0xa1,0xe0,0x31,0x2c,0x6f,0x8e,0xff,0x17,0x88,0x48, -0xb3,0xd6,0x68,0x3e,0x0e,0xd6,0x32,0xb1,0x1f,0xff,0x97,0xae, -0xf4,0xff,0xe5,0x58,0x90,0xf2,0x7e,0x3c,0xfe,0x40,0x32,0x2e, -0xdc,0xc2,0xa7,0xec,0x76,0xf8,0x74,0x85,0x9e,0x96,0xb5,0xa9, -0x09,0x67,0x49,0x32,0x71,0xb0,0xf7,0x60,0x5e,0xce,0x02,0xb0, -0x9d,0x33,0x3e,0xab,0xa1,0x49,0xb6,0x76,0x9c,0x14,0x18,0xc3, -0x35,0x82,0x1a,0xf8,0x2d,0xa7,0x78,0xae,0xc6,0xbb,0x71,0xc6, -0xe8,0xd8,0x46,0xa4,0x1e,0x3d,0x15,0x3d,0x9b,0x85,0x83,0x68, -0x4b,0xa1,0x61,0x71,0xd2,0x91,0x61,0xd0,0x64,0x7c,0x7b,0xb0, -0xf1,0x22,0x15,0xfc,0xb4,0xe7,0xdc,0x2d,0xf9,0xaa,0xb9,0x36, -0xdb,0xd4,0xab,0x59,0x86,0xae,0x49,0xbe,0xae,0x7a,0xa9,0xa9, -0xb1,0xfa,0xac,0xb0,0xba,0x5e,0x36,0x85,0x44,0x06,0xa0,0x9e, -0x4d,0x56,0x5b,0x2a,0xf5,0x95,0x6a,0x5c,0x5c,0x88,0x73,0xe8, -0xe7,0x23,0x27,0x0a,0x01,0x88,0x18,0x9b,0xb2,0x31,0x73,0x02, -0xfb,0x95,0x70,0xc1,0xbe,0xcb,0x3e,0xc0,0xe3,0xd6,0x66,0x61, -0x83,0x10,0x08,0xd7,0x09,0x67,0x52,0xf3,0xeb,0x25,0xe9,0x79, -0x9e,0x2f,0x01,0x79,0x33,0x3e,0xb2,0x80,0xba,0x6e,0xb6,0x83, -0x58,0xee,0x6c,0x57,0xef,0xe3,0x76,0xbb,0xa1,0x0c,0xda,0x7b, -0x36,0x37,0x76,0x5e,0xdb,0x2a,0x99,0x1b,0xd5,0x7a,0xb3,0xf2, -0x68,0x00,0x8c,0x89,0xdb,0x78,0x44,0x1b,0x97,0x6b,0x24,0x39, -0xfe,0xcf,0x0e,0xec,0x7a,0xbd,0x77,0xe8,0x44,0x22,0x63,0xad, -0xe2,0x5e,0x30,0x26,0xeb,0x75,0x1a,0xb7,0xd9,0xd7,0xea,0xf1, -0xe9,0x2f,0xcd,0x64,0x79,0xbf,0x09,0xf3,0x18,0x45,0x71,0x7d, -0xd4,0x2f,0x6c,0x7a,0x38,0xf6,0xac,0x37,0x16,0x9f,0x99,0x76, -0xb5,0x45,0x23,0x43,0x16,0xf0,0x1f,0xda,0xf7,0xec,0x5b,0x1b, -0xd9,0x9c,0x84,0x46,0x96,0xf1,0x28,0x20,0x40,0x89,0x29,0x77, -0x34,0x17,0x3e,0xfc,0x59,0xf1,0x58,0x2c,0x3e,0x41,0xa3,0xd3, -0x7f,0x65,0x20,0xbe,0x0c,0xa9,0xa8,0x88,0x56,0x17,0x99,0x79, -0x40,0x8d,0x41,0x72,0x46,0x1a,0x16,0x59,0xaf,0xbf,0xd8,0x48, -0xb6,0xe6,0x8f,0xe3,0xa7,0xcd,0x60,0xdf,0x0a,0x7e,0x00,0xd9, -0x8b,0xa7,0x68,0xd7,0x48,0x03,0xdc,0x0e,0x81,0xac,0xa2,0x4e, -0x64,0xaa,0xb5,0xa9,0xe2,0x9d,0x01,0xc1,0x1d,0xa9,0xcd,0x00, -0x88,0x20,0x79,0xd8,0xda,0xaa,0xfc,0x71,0x58,0x11,0x7b,0xcb, -0xb5,0x51,0xcf,0x69,0x27,0x1a,0xb1,0x55,0x2c,0xcd,0x48,0x8f, -0xf5,0xcf,0x67,0xe7,0xab,0x01,0x0a,0x90,0xb5,0xac,0xab,0x74, -0x28,0xa9,0xe0,0xa8,0xc0,0x63,0xab,0x80,0x6a,0xcf,0x99,0xb0, -0x09,0x97,0x15,0xb7,0x09,0x65,0x4d,0x7c,0xd8,0x9b,0x2b,0xf0, -0x71,0x8d,0x47,0xa9,0x26,0x7b,0xd1,0x1e,0x2f,0x63,0x07,0x38, -0xcf,0x6d,0xfa,0x4f,0x74,0xc9,0x09,0x2c,0x42,0x49,0x82,0xe1, -0x2a,0x21,0x01,0x3d,0xcc,0x13,0x48,0x87,0x38,0x2c,0xe3,0xff, -0xa1,0x9e,0x6d,0x99,0x3c,0x97,0xc1,0xef,0xf2,0xb8,0xf3,0x8d, -0xfd,0x8e,0xfa,0xb4,0xa5,0x2f,0x8c,0x31,0x21,0xc3,0xc1,0x89, -0xa6,0x5a,0x64,0xdb,0x0f,0x78,0x55,0x3a,0x39,0x57,0xaf,0xf1, -0xbd,0xe3,0xb7,0x9a,0x4b,0x15,0x3d,0x0d,0xa3,0xf4,0xe6,0x60, -0xf5,0x54,0xe8,0xb5,0x62,0xdb,0x7f,0x18,0x8a,0x74,0x61,0x23, -0x30,0x84,0x0b,0x61,0xeb,0x02,0xe1,0x5a,0x76,0x2e,0x5c,0xc3, -0xec,0x1c,0x17,0x29,0xba,0xde,0x64,0x1f,0x71,0xae,0xe7,0x56, -0x2e,0xc6,0xef,0xe2,0xdd,0xa4,0x93,0x8a,0x84,0x84,0xe5,0xe6, -0xa2,0x35,0x25,0xfb,0x15,0xd7,0xd9,0x0c,0x8d,0xc8,0x7a,0x39, -0xe8,0x1a,0x68,0x4f,0x27,0x90,0xf8,0x97,0xb2,0x6c,0x50,0xd9, -0x10,0xb3,0xc5,0x46,0xc0,0x92,0x74,0xe9,0x45,0xa8,0x26,0x43, -0x4b,0x3a,0x1c,0xab,0x14,0x33,0xce,0x97,0xc8,0x98,0x07,0x82, -0x6f,0x2c,0xa2,0x6b,0x95,0xf2,0x32,0xab,0xd6,0x72,0x14,0xd7, -0xb2,0x10,0x53,0xab,0x9b,0x40,0x16,0x44,0x54,0x73,0x90,0xed, -0x48,0x26,0x38,0xeb,0xa8,0xc3,0x85,0xcf,0x51,0xfe,0xf0,0xfa, -0xf5,0x8b,0xa7,0xc7,0xaf,0xde,0x1d,0x9f,0x9d,0xbd,0xa5,0xab, -0x3d,0x94,0xb6,0x72,0x17,0x9a,0x29,0xd3,0x6e,0x6b,0xa1,0xa7, -0x2f,0x9e,0xa2,0xe2,0xd3,0xa9,0x2e,0x13,0xb7,0xd2,0x06,0x24, -0x3f,0xaa,0x05,0x0c,0x2b,0x88,0xa9,0x19,0x83,0xac,0xb9,0xe6, -0x31,0x9a,0x20,0x57,0xb0,0xc9,0x50,0x86,0x98,0xa1,0xa0,0x84, -0xb8,0xe2,0x3f,0xd0,0x18,0x2c,0xe9,0x57,0xf8,0xcb,0xb8,0xe5, -0x1a,0xa2,0xad,0xbc,0x29,0xa7,0x73,0xaf,0xfe,0x26,0xda,0x43, -0x78,0x8d,0x52,0x34,0x1a,0x8f,0x47,0xb1,0x8a,0xe1,0x35,0x71, -0x02,0x1d,0xb8,0x28,0xbe,0x0e,0xef,0x12,0xa9,0xd6,0xe9,0x0f, -0x77,0x17,0x68,0xad,0x71,0xe7,0xc0,0x01,0x54,0x17,0xb0,0xa9, -0xfd,0x2c,0x05,0xe8,0xf2,0x52,0xe8,0x3a,0x6f,0xa5,0x34,0xb3, -0xef,0x89,0x52,0xc8,0xfb,0xb7,0xc5,0x4f,0x24,0x04,0x58,0x0b, -0x83,0xbf,0xcb,0xa6,0x0e,0x1f,0x7d,0xdd,0x28,0xc4,0x90,0x51, -0x8c,0x5c,0xca,0x88,0x25,0xe3,0x0f,0x44,0x92,0xe5,0xdb,0xb2, -0x1f,0x6e,0x38,0x85,0x8e,0x95,0x30,0xad,0x2e,0x1f,0xd2,0x0c, -0x64,0xbd,0x3e,0x28,0xcb,0xda,0xa4,0x6c,0xf2,0x90,0x3e,0xe8, -0xb3,0xa3,0x1e,0x19,0x45,0xc4,0x9b,0x43,0xfb,0xf7,0xaa,0x24, -0x18,0xac,0x26,0x14,0x75,0xfc,0xf3,0xe3,0xf2,0xb6,0x9a,0x8c, -0x7b,0xcd,0xeb,0x02,0x39,0x5b,0xcd,0x53,0x6f,0x93,0x1a,0x7a, -0x47,0xc2,0x35,0x45,0x6a,0x31,0xfd,0x08,0x76,0x40,0x05,0xc4, -0xd0,0xc3,0xc4,0x3a,0x7c,0xef,0x2d,0xcc,0x6e,0x05,0xef,0x9f, -0xd4,0x75,0x51,0x52,0xfc,0x0e,0xbb,0xf1,0xf6,0x8d,0x9c,0x82, -0x14,0xca,0x4d,0x9d,0xa9,0xb7,0x23,0xd6,0x26,0x54,0x7b,0xd0, -0x4e,0x10,0x4d,0xb2,0xa8,0xb3,0x46,0xfc,0xee,0xd5,0xcd,0xb1, -0x90,0xf6,0x6e,0x60,0x1a,0x29,0xeb,0xd5,0xa2,0xd2,0xac,0xc0, -0xb8,0x40,0xb7,0xb8,0x5e,0xb0,0x74,0xa6,0xa9,0x38,0x3b,0xca, -0x67,0x49,0x23,0xd0,0x3e,0x63,0xe7,0x57,0x6b,0x76,0x09,0xcd, -0x13,0xbc,0x44,0x51,0x74,0x24,0xa2,0xbc,0xf0,0x2b,0x1b,0xc5, -0x60,0x2f,0xa2,0x77,0xec,0x95,0x33,0x50,0x14,0x21,0x99,0xbe, -0x71,0xaa,0x2a,0xb7,0x22,0x32,0x03,0xd4,0x16,0xb9,0x03,0x87, -0xbd,0xf8,0x64,0x7c,0x39,0x25,0x59,0xb7,0xa3,0xad,0x06,0x68, -0xae,0x66,0xbd,0x54,0x0a,0xa2,0x5f,0x05,0x09,0xe2,0xe5,0xae, -0xb1,0x44,0xbf,0x79,0xc3,0xac,0x0c,0x1d,0xf3,0x36,0x58,0x36, -0x3c,0x3d,0x7b,0xfb,0xfc,0xcd,0xbb,0x93,0xd7,0x2f,0xe9,0xdc, -0x64,0xc1,0x5f,0x6e,0x46,0xc9,0xe5,0x44,0xad,0xef,0x19,0x9c, -0xc8,0x6f,0xff,0x7a,0xea,0x35,0x1a,0x24,0x17,0x6a,0xf8,0x31, -0x56,0xc2,0x39,0xde,0x9d,0xbe,0x79,0xf1,0xfc,0x2c,0xb3,0x5a, -0x54,0xd7,0xd9,0x3d,0x8a,0x12,0x6a,0x8d,0x9c,0xcf,0x64,0x80, -0x65,0x8d,0x13,0xb3,0xac,0xd0,0x83,0x81,0xdc,0x0a,0xd2,0x15, -0xbe,0xb4,0x67,0xa2,0xfa,0x07,0x34,0x97,0xef,0x70,0xa9,0x1d, -0x77,0x06,0xfe,0x88,0x36,0xc6,0xb1,0x2b,0x5d,0xef,0xf7,0x0e, -0x73,0x2b,0x86,0x79,0x35,0xeb,0x63,0xfa,0x20,0x4f,0xae,0x66, -0xd0,0x51,0x33,0x8a,0x6c,0x51,0xf2,0x80,0x86,0x1e,0xe6,0x3f, -0x2c,0x44,0xd9,0xc9,0x2e,0xb6,0x94,0x88,0xef,0x8f,0x58,0xbc, -0xfd,0x6c,0x8e,0x12,0x06,0x86,0xd5,0x2e,0x4c,0x0f,0x21,0x15, -0x46,0xf2,0x9e,0x52,0x17,0x5d,0x33,0xf4,0x8d,0xa3,0xdb,0x0c, -0x7d,0xd1,0x73,0x5a,0x2f,0x98,0x31,0xb4,0xcf,0xe6,0x80,0x66, -0x8e,0xd2,0xd2,0x28,0x03,0xf6,0x7f,0x58,0x2d,0x46,0x7d,0xc7, -0x0b,0xe4,0xa9,0xde,0x31,0x36,0x10,0x3a,0x8e,0x96,0x14,0x6f, -0x79,0xc0,0xc7,0x65,0x6c,0xb4,0x9e,0xbf,0x4a,0x0d,0x3d,0xf7, -0x72,0xfd,0x24,0xde,0x34,0x11,0xf7,0x4d,0x5a,0x28,0xdf,0x71, -0x1f,0xdd,0x5d,0x74,0xbc,0xcd,0xac,0x24,0x27,0x43,0xa6,0xdc, -0xa4,0xc5,0x35,0x9a,0xbb,0x4e,0xf4,0xa5,0x47,0x84,0xe9,0x6e, -0xef,0xaa,0x95,0x98,0x89,0x6d,0x61,0xe9,0x96,0x08,0x70,0xb5, -0x58,0x8a,0xcf,0x89,0xb1,0xa9,0xbd,0x31,0x29,0xcd,0xe4,0x2a, -0xe9,0xe9,0x68,0xd4,0xe9,0xdd,0xd5,0xd5,0xe4,0xd3,0xc0,0xb5, -0xfb,0x9d,0xd1,0x4b,0x16,0x81,0x4b,0x2b,0xa3,0x69,0x84,0x20, -0xc3,0xd1,0xbb,0xdf,0x25,0x4f,0xb5,0x9a,0x6b,0x13,0xd8,0x78, -0x23,0x91,0xb1,0x54,0xcd,0x35,0xab,0x3c,0xad,0x9f,0x7d,0x5b, -0x51,0xdf,0xca,0xd1,0x79,0x21,0x10,0xa7,0x76,0xe0,0x8c,0x94, -0x4d,0x9a,0x46,0xd8,0x82,0x0f,0xde,0xc4,0x58,0x95,0xc3,0xa2, -0x95,0x4d,0x95,0x87,0xe6,0xce,0xb2,0xc0,0x96,0x01,0x55,0x12, -0x40,0xa8,0x0f,0xab,0x92,0xfa,0xca,0x35,0xaf,0xa5,0x78,0x88, -0xa8,0x70,0x4b,0xef,0x67,0xa9,0x8c,0xcf,0x2b,0x7a,0xc9,0xc2, -0xd8,0x67,0xea,0xa2,0x6b,0x39,0x71,0x0f,0x9d,0xf5,0x85,0x85, -0xde,0x9f,0x4d,0xd1,0x68,0x46,0xb3,0x1c,0x4b,0xe3,0xa4,0x94, -0x16,0x0c,0xad,0xb5,0xa3,0x5a,0x4c,0xec,0x43,0x09,0x33,0x38, -0x99,0x45,0xed,0xcc,0xf3,0x48,0x55,0x45,0x2b,0xc8,0x2d,0x38, -0xa3,0xd8,0xd2,0x5c,0x4a,0x7a,0x48,0x06,0x92,0x61,0xd5,0xe8, -0x16,0x2d,0x6c,0x95,0x1b,0x8b,0x40,0xa3,0xcc,0xa7,0xa8,0x53, -0xc4,0xc7,0x0b,0xd9,0xbd,0x32,0x62,0x99,0xb1,0xfa,0x6b,0x90, -0xbd,0xae,0x8b,0x81,0xac,0x04,0x84,0x3e,0x56,0xf8,0x34,0xa6, -0x92,0x9a,0xdd,0x87,0x49,0x70,0x9d,0x49,0x12,0x1e,0x84,0x24, -0x32,0x65,0x1f,0x0c,0x5c,0x29,0x2f,0x59,0xfc,0x6c,0x56,0x86, -0x02,0x4f,0x92,0xe7,0xc8,0x37,0xd9,0x4b,0xfa,0x89,0x29,0x87, -0x7a,0x67,0x9f,0xcd,0xba,0x4e,0xe6,0x36,0xd3,0x6c,0x2e,0xc9, -0x4b,0x8d,0x1a,0x96,0xb9,0x91,0x5b,0xf5,0x19,0xad,0xac,0x29, -0x32,0xd5,0x27,0x93,0xd2,0x44,0x39,0x62,0x71,0x22,0xf2,0xe3, -0xf7,0x1e,0x12,0x99,0xc4,0x08,0x44,0x89,0x07,0xee,0xb0,0x6c, -0x5e,0x96,0xbe,0x1c,0x2a,0x5b,0x1f,0x69,0x9e,0xbe,0x89,0x3f, -0x1c,0x0c,0xe4,0x49,0xcd,0x49,0x20,0x9b,0xd4,0x47,0x03,0xb5, -0x74,0x64,0x8e,0x61,0x19,0xbe,0x5e,0x24,0x1e,0xd9,0x70,0xe2, -0xd2,0x42,0xd3,0x19,0x71,0xd7,0xe3,0x3f,0x5b,0xde,0xde,0x85, -0xb2,0x02,0x54,0xa4,0xdc,0x08,0xc7,0xf1,0xf9,0x1d,0xd5,0xe5, -0x44,0xc0,0x65,0x22,0x44,0x4d,0xcf,0x6a,0x2d,0x8c,0xbb,0x62, -0x21,0xb8,0xdd,0x1e,0xa3,0x23,0xae,0xcb,0x5f,0xdb,0xed,0xbd, -0x43,0x40,0xc0,0xf9,0xc3,0xb1,0x8d,0x5c,0x3e,0x04,0x0a,0x64, -0x6a,0xd7,0xd8,0x4c,0x4e,0x3a,0x92,0xdb,0x5d,0xd0,0xfe,0x02, -0xe1,0x1a,0x97,0x28,0x8c,0xfa,0x6c,0x38,0x21,0x09,0xeb,0x79, -0xcb,0x1c,0x99,0x2a,0x39,0x0d,0xf7,0x4a,0x6b,0x74,0x37,0x86, -0xd4,0xe2,0x7c,0x46,0x1a,0x1f,0xaa,0x6f,0xcb,0xa0,0x01,0xd5, -0xd7,0x2a,0x21,0x88,0xde,0x2d,0x32,0x2f,0x12,0x5d,0xc7,0x07, -0x58,0xbc,0xd4,0xe1,0x05,0x24,0xcf,0xe2,0x05,0xfd,0x2d,0x75, -0x7c,0x2a,0xd7,0x83,0x39,0xcb,0x68,0x6b,0x52,0x91,0x58,0xb0, -0xde,0x66,0xf2,0xb9,0xfa,0x26,0x1a,0xb5,0x0f,0x9f,0xbf,0x3a, -0x3d,0x3b,0x7e,0x75,0xf6,0xfc,0xf8,0xec,0xf9,0xab,0xbf,0x6e, -0xbb,0x59,0x2f,0x47,0x63,0x54,0x0b,0x3e,0x99,0x2c,0xd1,0x61, -0xcf,0xd2,0x8b,0x87,0x7f,0x86,0xf3,0x03,0xf0,0xb1,0x02,0x87, -0x24,0x21,0x05,0x0e,0x51,0x1e,0x6f,0x7d,0xbf,0x87,0x1c,0x2d, -0x27,0x82,0x5b,0xab,0x7a,0x63,0x0c,0x2a,0x50,0x39,0xd5,0x0b, -0xb4,0xcd,0xce,0x1b,0x9a,0x1c,0x34,0xb8,0x29,0x83,0xde,0x52, -0x16,0xd0,0xe6,0x0a,0x1f,0x84,0xa7,0x9f,0xef,0x59,0x55,0x84, -0xdf,0xf9,0xad,0x0d,0x4e,0x5e,0xc7,0x80,0xbe,0x91,0x01,0x17, -0x92,0xca,0x14,0x72,0x5e,0x55,0x46,0x88,0x9f,0x8c,0xaf,0x55, -0x88,0x32,0x2b,0x36,0x68,0xb1,0x74,0x76,0xf6,0x59,0x1e,0xa8, -0x1f,0x54,0x45,0x4b,0xe5,0xb8,0x50,0xf7,0x9f,0x7c,0x64,0xb0, -0xa1,0x06,0xcd,0x83,0xc7,0x44,0xcd,0xc9,0x30,0x22,0x71,0x5b, -0xe6,0x63,0xb2,0xfa,0x15,0x90,0xcc,0xe4,0xf9,0x0c,0xee,0x07, -0x14,0xba,0x12,0xd5,0x48,0x36,0x79,0xf0,0xeb,0x78,0xb6,0xd3, -0x7a,0xfa,0x69,0x01,0x11,0xb0,0x82,0x65,0x14,0x58,0x41,0x60, -0x58,0xb5,0xb8,0x16,0xd6,0xe8,0xa1,0x49,0x23,0x5e,0x34,0xf6, -0x8b,0xb7,0x38,0x77,0xbd,0xdd,0xe6,0xdf,0x26,0xab,0x4c,0x3d, -0x4e,0x22,0xf6,0x70,0x61,0x16,0x25,0x59,0x87,0xd8,0x54,0x24, -0x31,0x97,0x7a,0xdc,0x1b,0x68,0x8d,0xab,0x59,0x89,0xd8,0x34, -0x0d,0xc0,0x20,0xa3,0x21,0xee,0xed,0x1d,0x16,0x43,0x6f,0xd9, -0x27,0xbb,0x47,0xe1,0xdc,0xd6,0x41,0xa1,0xb7,0x12,0xdc,0xae, -0x47,0x14,0x75,0x68,0xa2,0x86,0x7a,0x4a,0x52,0xca,0xa3,0x7a, -0x0a,0xcd,0x0f,0xda,0x97,0xe0,0x1c,0xdf,0x6e,0xcf,0xc1,0xbf, -0x8f,0x34,0xe7,0x77,0x5f,0xce,0xc9,0xbf,0xdf,0x6a,0x89,0x3f, -0x7e,0x7d,0x09,0xfe,0xfd,0x4e,0x4b,0xfe,0xe9,0xf7,0x97,0xe4, -0xdf,0x3f,0x2a,0x84,0x7f,0xfb,0xe7,0x21,0xf0,0xef,0x9f,0x14, -0xd2,0x9f,0xff,0xf7,0x21,0xf1,0xef,0xbf,0x29,0xc4,0xbf,0xfc, -0xf7,0x41,0xe4,0xdf,0x3f,0x2b,0xe4,0xc3,0x83,0xff,0x7e,0xd0, -0xfc,0xfb,0x97,0x01,0xba,0xd1,0x65,0x99,0x4f,0x57,0x45,0x64, -0xd0,0xa9,0x0a,0x8f,0x0b,0x8f,0x46,0x23,0xcd,0x1f,0x1c,0x18, -0x8a,0xcd,0xe5,0x0c,0x6a,0x3c,0x62,0x0b,0x1e,0x27,0x1e,0xdd, -0xb6,0x28,0xa4,0xe3,0x95,0x98,0x74,0xef,0xc4,0xb3,0x74,0x26, -0x0a,0xcf,0xe8,0xdd,0x19,0xff,0xf6,0xf1,0x8f,0xbf,0x16,0x0b, -0x4a,0xf1,0x45,0x1c,0x42,0x4d,0xe6,0x05,0x0d,0xd4,0xb0,0x3d, -0x82,0xa5,0x70,0xf3,0x5d,0x93,0xa5,0x1f,0x4e,0x86,0x3f,0x0d, -0xca,0x00,0x3e,0x60,0x3f,0x0b,0x2d,0x26,0x4a,0x66,0xf7,0x0c, -0x52,0x14,0x3d,0x72,0x33,0x48,0x85,0x09,0xa3,0xa0,0x99,0x39, -0x36,0x72,0x3d,0x42,0x0b,0x0d,0xe4,0x70,0xe6,0x14,0x91,0xb5, -0x88,0x26,0xe2,0xac,0x49,0x91,0x2c,0xc6,0xec,0xd7,0xeb,0xc6, -0xab,0x91,0x99,0x04,0x1b,0x12,0x49,0x0a,0xee,0x93,0x92,0x3d, -0xeb,0x1b,0x10,0x65,0xe2,0x55,0x54,0xf0,0xf2,0xc0,0xd3,0x3f, -0xc0,0x76,0x69,0x98,0x85,0xc5,0x93,0x07,0x0d,0x2c,0x9d,0x14, -0x7e,0xe1,0xe8,0x9f,0x22,0xa4,0xee,0x35,0xda,0xab,0x1d,0x85, -0xe9,0x8a,0x05,0x38,0x5d,0xa3,0x30,0x59,0x62,0x55,0xc1,0x28, -0x4c,0x94,0x57,0x4b,0xa7,0x55,0x14,0xa6,0x6a,0x74,0x96,0x3b, -0xea,0xaa,0x70,0xa1,0x8d,0x1f,0x05,0x45,0x4b,0xca,0x70,0xec, -0x77,0x6b,0x4a,0xfb,0x11,0x16,0x13,0x64,0xb7,0x4a,0x7b,0x5b, -0xae,0xb4,0xbb,0xd9,0x02,0xf1,0xae,0x9f,0x66,0xbf,0xce,0xe6, -0x1f,0xfd,0x4c,0x6f,0xc5,0x2c,0xf0,0xf1,0xd3,0x52,0x8e,0x38, -0x73,0x31,0x1d,0x55,0x06,0x39,0xbe,0xdc,0xe6,0x20,0xbb,0xb1, -0x0f,0xc0,0xa3,0xec,0x9f,0x20,0x1e,0xc6,0xe8,0x4d,0xf6,0x78, -0x39,0x7e,0x89,0xe0,0x5b,0x58,0x42,0xd3,0x4d,0x01,0xba,0x1a, -0x8d,0xa8,0xb5,0xed,0x84,0x5e,0xc8,0x59,0xde,0x56,0x11,0x7a, -0xa3,0x46,0xe5,0x21,0x3b,0x8c,0x9a,0xc7,0x1f,0x73,0x8d,0x42, -0xda,0x42,0xf6,0xa2,0xd7,0x47,0x4e,0x00,0x74,0xe2,0x29,0x1b, -0x2f,0x28,0x77,0x0e,0x8e,0x88,0x9f,0x39,0x9a,0x54,0x18,0x71, -0x6c,0xf3,0xd8,0x43,0xaf,0xb9,0xf0,0xe1,0x86,0xb9,0xa1,0x44, -0x65,0xf7,0x13,0x11,0xef,0x45,0x9b,0x0d,0x2a,0xe6,0xac,0x56, -0x1e,0xc4,0xe4,0x83,0x67,0x79,0x88,0x42,0xab,0xcb,0x68,0xa4, -0x9a,0x43,0x0c,0xfa,0x19,0x8a,0x8f,0x71,0xb7,0x52,0xb6,0xe9, -0xec,0x0d,0x42,0x13,0x5f,0xbc,0x36,0xd2,0x90,0xa9,0xc1,0x10, -0x0b,0x97,0x58,0xaf,0x93,0x21,0x1a,0xa8,0x0c,0xed,0x8a,0xd9, -0xb7,0x24,0x64,0xb5,0x0a,0x74,0x27,0x23,0xa6,0xf6,0xa0,0x0d, -0xdb,0x87,0xc6,0x42,0x46,0x76,0x3c,0xbd,0xc5,0x23,0x10,0x10, -0x48,0xed,0x0d,0x1e,0x5f,0x37,0x29,0x0a,0x24,0x55,0x37,0x3d, -0xa8,0xe2,0xb6,0xdc,0x66,0x27,0x07,0x73,0x64,0xe8,0x9d,0x4a, -0x1d,0xa5,0xc2,0x1a,0x9f,0xff,0x6d,0x32,0xfe,0x88,0xb6,0x1d, -0xb0,0x60,0x34,0x04,0x0d,0x70,0xaa,0x1f,0x3e,0xd3,0xcb,0x1b, -0x83,0xda,0x02,0x8b,0x9c,0x8d,0x02,0xf5,0x80,0x99,0xda,0x6d, -0x19,0x7e,0xef,0x4b,0xed,0x20,0x3f,0xc8,0x8a,0xc6,0xd8,0x8d, -0x78,0xd7,0x62,0x53,0x84,0x9a,0x45,0x23,0x9c,0x6e,0x7c,0xc3, -0xfa,0x80,0x6a,0xdc,0x9c,0x8a,0xe9,0xf1,0xb4,0xfe,0x4c,0x13, -0x8f,0xd9,0xc6,0x1e,0x3d,0x16,0x80,0xb3,0x65,0x9e,0x72,0xfb, -0xc8,0x35,0x01,0x85,0x36,0x46,0xac,0x44,0x94,0x0d,0x6a,0xfe, -0x9f,0x50,0xb8,0xde,0x49,0xb6,0x9b,0x15,0xa6,0x0e,0xc7,0x5f, -0xdf,0xad,0x70,0x53,0xa1,0x0b,0x36,0xd4,0xc1,0x40,0xab,0x1b, -0xb8,0x1d,0x91,0xe6,0x71,0x98,0x05,0x71,0x75,0x9a,0x6c,0x65, -0x1e,0x66,0x9e,0x5e,0x41,0x5b,0xb7,0x35,0x58,0x27,0xa8,0xb8, -0xb6,0xb7,0x97,0xa3,0x33,0x9a,0x2d,0xa9,0x2c,0xea,0xdc,0x90, -0xa8,0xea,0x6f,0xee,0x3d,0x17,0xdb,0xf4,0x50,0xbe,0x05,0x3a, -0x41,0x48,0x0d,0xf5,0x8e,0x7d,0xef,0x92,0xcf,0xa1,0x94,0x2e, -0xd1,0xd0,0x5e,0xe1,0x1b,0x7e,0x13,0x57,0xf7,0xd6,0x28,0x40, -0x2d,0x72,0xfb,0xd9,0xfd,0x8e,0x1f,0xa7,0x9b,0xf1,0xe5,0xaf, -0xa9,0x7f,0x0b,0x59,0x40,0xa9,0x67,0x33,0x23,0x48,0xc7,0x11, -0xd9,0x3d,0xff,0xa6,0x64,0xa8,0x0c,0x57,0x11,0x83,0x2a,0x3d, -0xd4,0x94,0x40,0xe5,0xce,0x9d,0xf6,0x26,0xf0,0x96,0x32,0x59, -0x8e,0x7f,0x5a,0x4e,0x59,0x67,0x06,0xaa,0x43,0x8e,0xbd,0x4c, -0x29,0x44,0xef,0x94,0x88,0xdc,0x75,0xef,0x96,0x53,0x74,0x13, -0x9e,0x86,0x89,0x26,0xcd,0x3d,0x63,0xdc,0x29,0x28,0x95,0xa4, -0x33,0x0d,0x9e,0x4a,0x14,0x54,0x22,0xa1,0xd4,0x43,0x20,0xf9, -0x38,0x96,0x97,0x9f,0x5e,0x91,0x87,0xca,0x7c,0x39,0xfc,0xa8, -0x72,0xa7,0x6e,0x37,0x23,0x42,0xab,0xeb,0x57,0x4d,0x81,0xb8, -0x13,0x0d,0x0a,0x21,0x2a,0xa0,0xf1,0xf2,0x69,0xc6,0x57,0x53, -0x7c,0x4c,0x4e,0xee,0xc3,0xa3,0x64,0x1b,0x97,0x23,0x19,0x0f, -0x93,0xfe,0x64,0x7c,0x85,0xfe,0xc4,0x49,0xbf,0x9f,0x5a,0x3d, -0xa9,0x5e,0xce,0x2f,0x7f,0x2d,0xc5,0x95,0xd2,0x96,0x35,0x06, -0xa4,0xee,0x03,0x0b,0x07,0x79,0x8c,0x04,0x6b,0x77,0x77,0xfb, -0xae,0x28,0xb7,0x27,0xe5,0x52,0x78,0x32,0xbb,0xac,0x27,0x72, -0xfd,0x66,0x6b,0x6f,0x69,0x62,0xa7,0xb3,0x61,0x38,0x80,0x5a, -0x4e,0xae,0x3e,0xa3,0xba,0xd7,0xab,0x79,0x1d,0x5c,0xe5,0x41, -0xa9,0x7e,0xe8,0xef,0x5b,0x9b,0x0f,0xec,0xc4,0x9e,0x42,0x84, -0x63,0xf3,0xc1,0x7d,0x46,0x96,0xd7,0xb5,0xf6,0xcd,0x11,0xfb, -0xa2,0x76,0x4b,0x3e,0x97,0x66,0xb8,0x61,0x1d,0x8e,0x46,0x6f, -0x28,0xaa,0x0c,0x6e,0x7c,0xf7,0x14,0xe4,0x1f,0xe2,0x0d,0x14, -0x58,0xe7,0x76,0x97,0x1e,0x1e,0x1c,0xd8,0x0d,0xaa,0x75,0x70, -0x5b,0x00,0x5a,0x7e,0x35,0xe3,0x86,0x44,0x7b,0xc3,0x1f,0xb2, -0xcb,0xf1,0x55,0x7e,0x01,0x17,0x9f,0x1a,0xe9,0x72,0x17,0x0a, -0x0b,0x91,0x60,0x52,0x82,0xf7,0x1d,0x3d,0xe7,0x8d,0xb0,0x2c, -0x5d,0xb2,0xba,0x35,0x7c,0xdb,0xe1,0x43,0xb3,0x11,0x77,0x04, -0xbe,0xe9,0x37,0xaa,0xb9,0xc4,0x78,0xee,0xa3,0x5e,0x16,0x51, -0x0e,0x04,0xa4,0x27,0xb3,0xee,0x92,0x9e,0x40,0xee,0xc9,0xb7, -0xbe,0x2f,0x9e,0x12,0xef,0x86,0x0e,0xe2,0x24,0xc9,0x11,0x74, -0x91,0x6a,0x16,0x1c,0x83,0x7a,0xba,0xed,0x2c,0x99,0x35,0x4b, -0x13,0x1c,0x83,0x64,0x5b,0x3c,0x5a,0x16,0xd6,0xca,0x53,0x37, -0x6c,0xfa,0xbc,0x49,0x10,0xed,0xe0,0x70,0x13,0x82,0xe1,0x2d, -0xcd,0xc8,0xb8,0x01,0x64,0x26,0xca,0xc6,0xd9,0xd8,0x70,0xa9, -0xeb,0x75,0x50,0xd8,0x9a,0x28,0xfc,0x37,0x14,0x45,0xfb,0x43, -0x22,0x4b,0xab,0x7e,0x96,0x21,0x65,0xe3,0x62,0x51,0xcf,0x13, -0xb7,0x3f,0x4d,0xd4,0x7c,0xe6,0x0e,0xd1,0xa6,0x8d,0x22,0xad, -0x08,0xca,0xe2,0x0b,0x73,0x34,0x09,0xed,0xb6,0x48,0x24,0x89, -0x2a,0x35,0x80,0x4d,0x13,0xb8,0x5d,0xd0,0x9d,0x2e,0x22,0x73, -0xf6,0xac,0xce,0xcc,0x14,0xc2,0x3d,0x7e,0x49,0x91,0xbd,0x86, -0xf2,0x3e,0x35,0x86,0x50,0x84,0xbb,0x24,0x8d,0xe0,0x47,0x3d, -0x3d,0x30,0x31,0x6e,0x44,0xa2,0x3d,0xe9,0x74,0xc7,0xe5,0x4c, -0xc1,0x29,0xff,0x11,0x67,0x28,0x7a,0x11,0xa2,0x59,0xdb,0xb6, -0x1e,0x74,0xd5,0xe2,0x57,0x2f,0x9c,0xa2,0x5f,0x50,0x41,0xa3, -0xea,0x9d,0x17,0xe7,0xfb,0xe7,0xfb,0xfd,0x5f,0xce,0xf7,0x07, -0xdf,0xb0,0xfb,0xc0,0x24,0xf1,0xbb,0xf0,0x04,0xf5,0x40,0xc8, -0x12,0x4b,0xee,0x3f,0xf9,0x61,0xa5,0x84,0x35,0x7a,0x49,0xdf, -0x6f,0x86,0xa4,0xb0,0x64,0xda,0x98,0x66,0x3c,0x95,0x97,0x52, -0x3c,0x34,0xdf,0x62,0xdd,0xb6,0x52,0x06,0x55,0xd4,0xa2,0x0f, -0x56,0xc7,0xe7,0x70,0x3e,0x61,0x43,0xe3,0xa4,0xf6,0x32,0x53, -0xfe,0xb3,0xb9,0xcc,0xa4,0x06,0x44,0x8a,0xa3,0xe6,0x11,0x4a, -0x52,0x6b,0x73,0xbd,0xa8,0xad,0xb5,0x06,0x44,0xa5,0x1a,0x8f, -0x5a,0x62,0xe7,0xc7,0x8c,0x07,0xcb,0xf4,0x99,0xd2,0x91,0x6b, -0x13,0x4e,0x29,0x4d,0x06,0xe4,0xbb,0xb2,0xa7,0x12,0x8e,0xf4, -0x82,0xa7,0x68,0x20,0x97,0x3d,0x94,0x1c,0xf0,0xb8,0x90,0x45, -0xb4,0xbb,0xd9,0xb8,0xba,0x1c,0x2e,0xc6,0x02,0x4a,0x3c,0x20, -0xe0,0x9b,0xd6,0x81,0xda,0x5f,0x3f,0x32,0x8d,0x92,0xd7,0xcc, -0x50,0x96,0xa4,0x9e,0xbe,0x1d,0x2a,0xbb,0x9e,0x3c,0xcc,0xbc, -0x1d,0x1f,0x53,0x7a,0x53,0x77,0xda,0x5a,0x1f,0xcc,0x52,0x40, -0xd3,0x54,0x75,0x92,0xf2,0x88,0x78,0x18,0x49,0xc7,0xaf,0x93, -0x4e,0x82,0x72,0xf0,0xf8,0x66,0x56,0x9e,0xdd,0xdc,0xe5,0xad, -0x83,0xc3,0xd6,0xbf,0x0f,0x67,0xad,0xc3,0xbf,0xfc,0xdb,0x41, -0xeb,0xe0,0xa0,0xa0,0xff,0x5b,0x7f,0x7d,0x79,0x66,0x1c,0x55, -0x3a,0x0b,0xe1,0x76,0xb1,0x94,0x0d,0xab,0x20,0xae,0x3d,0xe9, -0x48,0x04,0x43,0x80,0xaa,0x6b,0xcd,0x51,0x25,0xb5,0xce,0x61, -0x6e,0xa1,0x3f,0xfe,0xee,0xe0,0x2f,0x7f,0x02,0x12,0x00,0x71, -0xcf,0x8f,0x43,0x14,0x2a,0xe6,0x51,0x68,0xfd,0x41,0x8d,0x69, -0xfe,0x01,0x6e,0xcb,0xaa,0x9a,0x5c,0x4c,0x3f,0xb3,0xea,0xc8, -0x78,0x85,0xf6,0x70,0xe6,0x1f,0xc6,0xcb,0x2b,0x24,0xcf,0x46, -0xad,0x8b,0xf1,0xe5,0x10,0x7d,0xb5,0x4e,0x56,0xad,0x8f,0x68, -0xd2,0x66,0x3e,0x87,0xd1,0x5c,0x5e,0x8f,0x5b,0xa9,0xd6,0xcf, -0x55,0x75,0x92,0xd6,0xe3,0x16,0x56,0xd7,0xba,0xf8,0xbc,0x1a, -0x57,0xd9,0x0e,0x72,0x1b,0x78,0xc7,0x8f,0x10,0x4f,0xb2,0xf7, -0x2d,0x8a,0x42,0xa1,0x4c,0x0f,0x19,0x76,0xe4,0x5b,0xf4,0xf9, -0xc8,0xe9,0xc4,0x6d,0xc3,0x49,0x72,0x97,0xd5,0x22,0xb2,0xe6, -0x14,0x11,0x01,0xab,0x08,0x3b,0xeb,0xbb,0x62,0x83,0xfc,0x61, -0x42,0x63,0xc3,0xed,0x42,0x2b,0xfe,0x31,0x8a,0x67,0x80,0xa0, -0xc0,0x9c,0xfb,0xb2,0x3d,0x14,0x81,0x40,0xdf,0xd1,0x11,0x17, -0xf6,0x7c,0xb8,0x08,0xa6,0xa4,0x0f,0x7a,0xe9,0x96,0x96,0x6b, -0x86,0x00,0x23,0x75,0x60,0x1f,0xea,0x0d,0x71,0x29,0xf0,0x19, -0xb9,0xd8,0x39,0x34,0xb4,0x46,0xac,0x06,0x4e,0x8c,0xa5,0x2d, -0xbc,0x83,0x6b,0xfc,0x91,0xeb,0x04,0x83,0xce,0xcc,0x66,0x23, -0x03,0xc1,0xd3,0x75,0x5e,0x2d,0xdc,0xab,0xfd,0x8e,0x3f,0x3a, -0x9a,0xd0,0x95,0xd9,0x42,0x15,0x5a,0x3a,0xb2,0x51,0x2d,0x3d, -0x68,0x73,0x93,0x59,0x2b,0xab,0xd9,0xce,0xaf,0x77,0xcf,0x47, -0x62,0x06,0x23,0xd4,0xcd,0xbd,0x82,0xdd,0x8b,0x06,0xa1,0x57, -0xf8,0x60,0x48,0x3f,0x3b,0x25,0xc5,0x3d,0x9d,0xe1,0x91,0x03, -0x63,0x3a,0x1d,0x43,0xb0,0xa7,0x81,0xb2,0xa4,0x4c,0x26,0x89, -0x23,0xba,0xb3,0xac,0x08,0x63,0x72,0xa0,0xc0,0x7f,0x4d,0x25, -0x31,0xe7,0x5f,0x98,0x0f,0x1f,0x9b,0x6b,0x3d,0x99,0x0b,0x95, -0x61,0x42,0x97,0xc5,0x11,0x0d,0x59,0x46,0xa5,0x51,0x82,0xee, -0x29,0x65,0x44,0xd1,0xc3,0xa7,0xdc,0x76,0x17,0xb9,0x53,0xba, -0x58,0x74,0xfd,0xa5,0xd1,0x36,0xdc,0x5d,0xf8,0x3c,0x59,0x6e, -0xb3,0xbb,0x30,0x56,0xad,0xd9,0x33,0xd2,0x5a,0x94,0x51,0x44, -0xf7,0x23,0x14,0xac,0x19,0xe7,0xb2,0x63,0x8e,0x06,0xba,0x26, -0x93,0x11,0xf9,0x67,0xe2,0x62,0xce,0x3e,0xd7,0x50,0xac,0xad, -0xae,0x86,0xf8,0xdc,0x96,0xe4,0x02,0x97,0x15,0xd2,0xaa,0xc9, -0x3f,0xc6,0xa8,0x3b,0x0c,0x88,0x4d,0x55,0x8a,0x02,0x05,0xdc, -0x4f,0x32,0x73,0xf9,0xfd,0x64,0x54,0x48,0x7e,0x34,0x67,0xc9, -0xba,0x1c,0x00,0x61,0x31,0xbc,0x9c,0xac,0x3e,0x97,0x92,0xad, -0xdd,0x96,0x40,0x57,0x53,0xd6,0x6b,0xb6,0x9e,0xd3,0x7d,0x79, -0xfc,0xf7,0x77,0x7f,0x3b,0x7e,0xf1,0xd3,0xd3,0x7c,0xba,0xbc, -0x43,0xd6,0x33,0x96,0x77,0xa3,0xcf,0x7c,0x05,0x9d,0x45,0xcb, -0xd3,0xe2,0x1e,0xf7,0xa5,0xee,0x41,0x79,0x8f,0x36,0x5d,0x9a, -0xe4,0xc9,0x08,0xd9,0x58,0xde,0xd1,0xb8,0x95,0x52,0x07,0xbd, -0xf7,0x01,0x5a,0x67,0x3f,0xcb,0x7b,0xf8,0x8b,0x02,0x90,0x9e, -0x43,0xa9,0x8b,0x51,0x8b,0x87,0x32,0xc1,0xea,0xa0,0x82,0x84, -0xce,0x0a,0x75,0x04,0xc3,0x6a,0x30,0x38,0x6a,0x70,0x36,0x46, -0x9a,0x2c,0x39,0x46,0x3f,0xd6,0x21,0x68,0xb7,0x69,0xc7,0xb0, -0x52,0x9b,0x5b,0xbc,0x5d,0xd2,0xbc,0x11,0x5b,0xc0,0xf4,0xbe, -0x60,0x3b,0xf5,0x40,0x77,0x10,0x8b,0x71,0x0d,0xdd,0xda,0x01, -0xd7,0xa2,0x8d,0x28,0x1c,0x7f,0x35,0x78,0x8d,0xc5,0x6b,0x5f, -0x33,0xd8,0x8d,0xe9,0xe6,0x4c,0x53,0x69,0x6f,0xb9,0x9c,0xda, -0x3f,0xbb,0x4f,0x5d,0xce,0x99,0xec,0x42,0x1f,0x91,0x5b,0x28, -0x72,0x10,0xdb,0xe6,0x68,0x9c,0xeb,0x0f,0x8d,0xed,0xde,0x5e, -0xa6,0xfd,0x3a,0x9e,0x4e,0xad,0xb3,0x13,0x5d,0x99,0xb2,0x9c, -0x9b,0xd6,0x5a,0xb0,0xcc,0x36,0xb9,0xe8,0xc0,0xd5,0x80,0xe8, -0x92,0x84,0xed,0x40,0x41,0x05,0x45,0x1f,0xd2,0xaa,0x68,0x71, -0x22,0x37,0xff,0x6a,0xde,0xe0,0x7b,0xc5,0xef,0x27,0x02,0x98, -0xdf,0x63,0xf3,0x0a,0xfc,0xb3,0xd1,0x27,0x1d,0x06,0x65,0x8c, -0x15,0xda,0x4d,0xdd,0x45,0xb8,0x31,0xca,0x4e,0x71,0x75,0xdf, -0x0b,0x0c,0xc8,0xa8,0xcf,0xd0,0xc3,0x80,0xee,0xf5,0x7b,0x2c, -0xe5,0x77,0x13,0xbf,0x34,0x61,0x5c,0xca,0x6c,0xf5,0xab,0xf9, -0x26,0x0f,0x6a,0x0e,0xce,0x79,0x07,0xa5,0x79,0x73,0x86,0x45, -0xed,0xcd,0xd7,0x68,0x26,0x23,0xbe,0xff,0x82,0x63,0xcc,0xdc, -0x73,0x36,0xde,0x33,0x4d,0x83,0x8b,0x26,0x51,0xb3,0x1c,0x55, -0x12,0xdd,0x61,0xa1,0xd5,0x28,0x63,0x6d,0x8a,0xa8,0x1e,0x6b, -0x1e,0x06,0x17,0x89,0xbe,0x9f,0xb9,0xd8,0x24,0x17,0xd1,0xe1, -0x77,0x4f,0x9e,0xbf,0x7d,0x7a,0x72,0xf6,0xfc,0x6f,0x4f,0x9d, -0x39,0xf9,0x5f,0xce,0xab,0x6f,0x9c,0x45,0x99,0x73,0xb1,0x27, -0x3f,0x42,0x0f,0x20,0x68,0x51,0xfe,0xbc,0xea,0xa4,0xdd,0x6f, -0xb2,0xdd,0xfd,0x9c,0x6d,0x97,0xd7,0xcb,0xa7,0x36,0x7b,0xda, -0x2b,0xce,0x8b,0xb4,0xff,0xcb,0x11,0x84,0xb3,0xde,0x51,0x2f, -0xdb,0xcf,0x87,0x48,0x04,0x9d,0x0e,0x81,0xd6,0x9b,0xfc,0x83, -0xc8,0xe3,0x9f,0x6f,0x80,0x94,0x24,0xfb,0x13,0x54,0x77,0xca, -0xb4,0xd7,0xfa,0x6a,0xb5,0x58,0xdf,0x0e,0x27,0xd3,0xd5,0x7c, -0x7d,0x05,0x7d,0xcd,0x8a,0xfd,0x7c,0x72,0x7b,0x7d,0xba,0xbc, -0xfc,0xca,0xb2,0x5c,0x68,0x8d,0xab,0xbe,0x98,0xdc,0x0e,0xaf, -0xc7,0xe7,0xfb,0xfb,0xf9,0xd3,0xbf,0x61,0xa7,0x7f,0x3c,0x7e, -0xf5,0xe4,0xc5,0xd3,0xb7,0xa4,0xac,0xe2,0xfb,0x9d,0xce,0x67, -0xfd,0xe1,0xde,0x3f,0x06,0xdf,0xac,0xd1,0x60,0xd6,0x90,0x86, -0x1a,0x3a,0xaa,0xef,0x28,0x32,0x22,0xa5,0xb9,0xec,0xd9,0xb4, -0xde,0x13,0x6f,0x7e,0x87,0xcd,0x7a,0xc8,0x67,0x3c,0xb7,0x0e, -0x69,0x67,0x73,0xcf,0xa9,0xb7,0xfc,0x1a,0x17,0x31,0x16,0xf8, -0xdc,0xfd,0x97,0x07,0xb3,0xda,0xf8,0x8c,0x0a,0x57,0x42,0x90, -0x49,0x88,0x1a,0x14,0x92,0x51,0x23,0x40,0x56,0x6e,0xb4,0xc3, -0xcb,0x22,0x94,0x5d,0x4d,0x6a,0x46,0x76,0x9a,0xe4,0x59,0xeb, -0xa6,0x78,0xd4,0xe4,0x80,0x5b,0x76,0x75,0xb9,0xc6,0x86,0xb6, -0x79,0xd0,0xf5,0x51,0x43,0x6e,0x78,0x00,0xb3,0xdc,0x8d,0xcd, -0x8e,0xd7,0x0a,0x1d,0x19,0xc1,0x3b,0x97,0x98,0xf5,0x3c,0x88, -0x7b,0xb1,0xbc,0x56,0xa8,0xcc,0xac,0xcf,0xb5,0x29,0x76,0xdc, -0x47,0x57,0xb2,0xb5,0xdb,0x3e,0x0a,0x8f,0x78,0x74,0xae,0x17, -0xe7,0x29,0x6b,0xa0,0x28,0x6b,0x66,0x0c,0x2e,0x76,0x17,0xcb, -0xc9,0x7c,0x89,0x18,0x45,0x3d,0x0a,0x28,0x03,0x93,0x91,0x68, -0xdb,0xf0,0x73,0xbd,0x0e,0x97,0x95,0x0a,0x56,0x96,0xb5,0x98, -0xf5,0xda,0x36,0x4e,0x95,0xd7,0x6c,0x1f,0x6a,0x90,0x90,0xba, -0xbd,0x5c,0x95,0xf5,0xa8,0xf5,0x1a,0x7d,0x6f,0xfb,0xf9,0x64, -0x06,0x8c,0x19,0x2d,0xff,0x08,0x11,0x2f,0x05,0x7a,0x8b,0x30, -0xbd,0xaf,0x36,0x83,0x2c,0x5a,0xbe,0x3c,0xfb,0x11,0x4c,0x9d, -0x44,0xaf,0x61,0x2f,0x06,0x6d,0xad,0xbc,0x7c,0x6d,0xd3,0x65, -0x62,0x63,0x92,0x1f,0x2c,0x1f,0x38,0x5c,0xdc,0x52,0x03,0x10, -0x40,0x66,0x9b,0x6d,0xa9,0xaa,0x8c,0x92,0x00,0x3b,0x73,0x3b, -0x14,0xce,0xc3,0x2a,0x22,0xc5,0xf6,0x7c,0xd2,0x9c,0x87,0x0e, -0xac,0xdf,0xd1,0x9e,0x87,0xc0,0x04,0x0d,0x7a,0x20,0x63,0xf4, -0xa2,0x6b,0x77,0xbc,0x31,0x89,0xd5,0x7c,0x00,0x24,0x64,0xff, -0x0a,0x7f,0x03,0x3b,0x51,0x18,0x41,0x86,0xac,0xd8,0xc6,0xbf, -0xd3,0x96,0xcc,0x23,0x6f,0x00,0x9e,0xca,0x4b,0xd0,0x48,0x15, -0xfe,0x38,0xbb,0x50,0xcd,0x7e,0x00,0xcc,0x41,0x63,0x1a,0x57, -0x3f,0x75,0xd8,0x30,0x57,0x64,0xbe,0x4a,0xac,0x6b,0x59,0x53, -0x63,0xe6,0x11,0xda,0x12,0x89,0xd0,0x18,0x63,0xa2,0xca,0xd9, -0x3c,0x8b,0x5e,0x11,0xd1,0x5b,0x80,0x5a,0x6c,0x24,0xb5,0xd9, -0x7c,0xe5,0x0c,0xc9,0x3d,0x9b,0xe5,0xb7,0xc3,0x4f,0x6f,0x64, -0x2f,0xe7,0x93,0xeb,0xd9,0x7c,0xe9,0x2d,0xb8,0xc1,0xe6,0xb0, -0xe5,0xac,0x59,0x1a,0x66,0xa0,0x22,0x6f,0xd6,0xe6,0x50,0x4d, -0xd1,0x20,0x32,0xf3,0xcf,0x5e,0x61,0x33,0x3c,0xe7,0x10,0x3e, -0xd5,0x47,0xe1,0xb7,0x65,0xe9,0xdc,0x33,0x8b,0x26,0x96,0x7e, -0xb2,0xdf,0x29,0xf1,0xf7,0x75,0x7e,0xda,0xd9,0x47,0xc6,0x51, -0x00,0x52,0x5d,0xe2,0x63,0x6e,0x6d,0x0b,0x39,0x80,0xee,0x7e, -0x44,0x1b,0xff,0xc9,0xf7,0xd5,0x62,0x38,0x7b,0xfc,0xfd,0x3e, -0xfd,0x24,0x99,0xd3,0x1c,0x45,0x51,0xc0,0x0d,0x53,0x5c,0x08, -0x6e,0x5e,0x41,0xc1,0x17,0x70,0x08,0x3e,0x9b,0x95,0x16,0xfc, -0x43,0xc3,0x18,0x26,0x3d,0x34,0xa8,0x35,0xbb,0xd7,0x62,0x5e, -0x1f,0x4d,0xb8,0x9c,0xcc,0x67,0x33,0xc8,0x86,0x6f,0x42,0xfe, -0x7a,0xe5,0xf4,0x84,0x7e,0x12,0x2f,0x71,0xbe,0x8b,0xc7,0x34, -0x56,0x57,0x86,0x45,0x7b,0x35,0xeb,0xfc,0x98,0xcc,0x7a,0x81, -0xe1,0xc4,0x14,0x61,0xa3,0x45,0x40,0xdd,0xc1,0x6d,0x16,0x4f, -0xa7,0xb1,0x75,0x79,0xfa,0x93,0xc1,0x51,0x7a,0x18,0xcd,0xd9, -0x7a,0xfd,0x97,0x28,0x26,0x43,0x86,0x9b,0x82,0x1d,0xff,0x96, -0x4e,0x32,0xf5,0x71,0xc1,0xbd,0xca,0xc5,0x8b,0x85,0xba,0x55, -0x18,0x5e,0x79,0x85,0x75,0x57,0x90,0x3c,0x0c,0xc9,0x28,0x44, -0xc3,0xd5,0x6e,0x87,0xdf,0xa6,0x10,0x43,0xce,0xa3,0x79,0x45, -0x0b,0xbe,0x41,0x84,0x8c,0xb2,0x2f,0xe7,0x42,0x99,0x0f,0xda, -0x27,0xea,0xa0,0x89,0x4e,0xe9,0xdd,0xb9,0xbf,0xa1,0xfb,0x5f, -0xe3,0xf1,0xf1,0x80,0x73,0xfa,0x0c,0x8d,0x2e,0xde,0x82,0xd5, -0x86,0x83,0x87,0x2f,0x08,0xd1,0x42,0x33,0xce,0x49,0x1e,0xde, -0xbc,0x01,0xd4,0x7a,0x57,0x1d,0xf8,0x00,0xe2,0x05,0x4a,0x61, -0x9f,0x99,0x0a,0xfd,0xac,0x73,0x69,0x31,0x12,0xc6,0x61,0xda, -0xb9,0x14,0xc1,0x87,0x13,0x05,0x6d,0x69,0xe4,0xf2,0x4f,0xf2, -0x19,0x52,0x58,0x17,0xdc,0x2f,0xac,0x12,0xb1,0x2b,0x95,0x71, -0x86,0x15,0xa7,0x2d,0xa9,0x2d,0xb8,0xb0,0x94,0xa8,0xe5,0xc9, -0x17,0xea,0xd2,0x3b,0x20,0x33,0x06,0x34,0xa5,0x56,0x55,0x01, -0x9c,0x19,0xc0,0xa1,0x35,0x1b,0x02,0xeb,0xcf,0x06,0xb9,0xef, -0x94,0x96,0xec,0x43,0xad,0x03,0xdb,0xc3,0x30,0xc1,0x17,0xe8, -0xa5,0x3e,0xcc,0xfe,0x31,0x7a,0xa9,0x1f,0x87,0x52,0x1c,0xbc, -0xcc,0xc6,0x1f,0x53,0x27,0x00,0x19,0x17,0x80,0x73,0x31,0x38, -0xa2,0xc2,0x0d,0xe1,0x81,0x01,0x42,0x11,0x43,0xae,0x8f,0x73, -0x69,0xa0,0xfb,0xe5,0x52,0xcf,0xb7,0x5e,0xef,0xd4,0x66,0xb8, -0xdd,0xb6,0x0b,0xac,0xe7,0x21,0xa5,0x76,0xaa,0xcd,0x2c,0xd3, -0xac,0x07,0xab,0xc6,0xdb,0x0e,0x0a,0x56,0x4e,0x7c,0xd2,0xd1, -0x26,0xd5,0x25,0xe5,0xb3,0xd6,0xc6,0xcc,0x9d,0x91,0x51,0x9e, -0xee,0xee,0xae,0x8f,0x21,0x35,0x6d,0x5b,0x5f,0x1a,0xe5,0xce, -0xb5,0x3a,0x7a,0xdf,0x73,0x66,0x75,0xc8,0xcc,0x75,0x2d,0x6f, -0xad,0x26,0xc9,0x9e,0x01,0x32,0x98,0x36,0x0c,0x63,0xd0,0xd1, -0xac,0xf8,0xca,0x41,0x73,0x0f,0x2c,0x0d,0xfb,0x4c,0xa6,0xd9, -0x9d,0x4f,0xfe,0xc3,0x6c,0x58,0x63,0x85,0xeb,0x41,0x60,0x1b, -0xbd,0x25,0xb6,0x6c,0x5c,0x8f,0xdd,0xe6,0xf8,0xa6,0x58,0xe5, -0xbc,0xce,0x9f,0x21,0x20,0x09,0xd3,0x33,0x2e,0xbf,0x84,0xd5, -0x36,0x28,0xec,0x29,0x2a,0x46,0x22,0xab,0xc7,0xde,0x27,0x97, -0x21,0xa0,0x2e,0x11,0x73,0x31,0xa6,0x98,0x2b,0xbb,0x73,0x73, -0x00,0xcd,0xf5,0x1e,0x94,0xe5,0xa4,0x67,0x8e,0xb1,0xc2,0xf7, -0x2a,0x3e,0xd0,0xec,0x7e,0x35,0x98,0xbd,0x38,0x1c,0x24,0x51, -0x28,0x5f,0xdb,0xd9,0x9c,0xcc,0xa1,0x99,0x7e,0xda,0xea,0xb9, -0xee,0xad,0xe7,0x6a,0x46,0xde,0x62,0x83,0x73,0xc0,0xd7,0xcd, -0x18,0x89,0xee,0xb5,0xf1,0xf2,0x96,0xcd,0x09,0xee,0x18,0xf8, -0x81,0xad,0xb4,0x2d,0x09,0xda,0x6e,0xac,0xa9,0x68,0x3c,0xf7, -0xc3,0xfc,0xf6,0xf0,0x69,0xdc,0xec,0x45,0xb0,0x24,0x75,0x12, -0xed,0xa1,0x89,0x11,0x51,0x82,0xe9,0x62,0x66,0xd7,0x40,0x69, -0xc2,0x28,0xe0,0xa9,0xc5,0xd7,0x6b,0x53,0xc2,0xc9,0x4d,0xf8, -0xbc,0xbd,0xe8,0xae,0xa1,0x81,0xb4,0xf7,0x5b,0xd3,0xa2,0xa8, -0xaf,0xc6,0x07,0xef,0x35,0x52,0xf6,0x23,0xff,0xad,0xee,0x36, -0xcd,0x9d,0xdf,0xd8,0x00,0xed,0x60,0x68,0x2f,0x87,0x0b,0xf2, -0xf1,0x06,0xd4,0x03,0xfe,0x1c,0x89,0x62,0x89,0xc3,0x4d,0xee, -0x45,0x3d,0x64,0x68,0xcc,0x6a,0xdb,0x85,0xe3,0x82,0xaf,0x90, -0xaf,0x82,0x16,0xa7,0x9d,0x6b,0xd8,0x77,0x72,0x76,0x07,0xc6, -0x42,0xb2,0x3c,0x79,0x9a,0x3c,0x8c,0x02,0xea,0xee,0xc4,0xf6, -0xb0,0xd9,0x95,0x19,0x77,0xe3,0x5a,0xed,0x95,0x30,0x13,0x3a, -0x27,0xc3,0x21,0x15,0xf7,0xca,0xf8,0xbe,0x7b,0x0f,0xf7,0xdd, -0xfb,0xf7,0x25,0xa7,0xc2,0x82,0xa4,0x5f,0xdd,0x9d,0xef,0xdf, -0x3f,0x7e,0x7f,0xf4,0x5e,0xf1,0x35,0x2c,0x75,0x8a,0x92,0x3a, -0x04,0x15,0xbf,0x9e,0xce,0x46,0x14,0xe6,0xe7,0x77,0x8c,0x11, -0x40,0xfd,0xf7,0x83,0x3c,0xdd,0x41,0x3b,0x11,0xeb,0x35,0xfe, -0x7d,0x5c,0xfe,0x79,0xbd,0x26,0xef,0x78,0xec,0xd1,0x63,0x32, -0x1e,0x21,0xca,0x4d,0x44,0xbe,0xf3,0xad,0x67,0xda,0x5c,0x36, -0x8d,0x14,0x69,0x5f,0xa1,0xbf,0x40,0x64,0x56,0xfd,0xf0,0xfc, -0xd5,0x93,0xe7,0xaf,0xfe,0xca,0xde,0xc3,0x7c,0x41,0x07,0xd5, -0x47,0xc9,0xdb,0x75,0xfa,0xa7,0xda,0xe0,0xee,0x1d,0xee,0x94, -0xfc,0x9e,0x6d,0xb3,0xb3,0x83,0x1c,0x79,0x73,0xa7,0xfe,0x26, -0xa8,0x24,0xce,0xf9,0x82,0x8c,0xac,0xee,0xf0,0x47,0x74,0x69, -0x64,0x87,0xa6,0x9c,0x45,0xe3,0xc3,0xee,0x21,0xa5,0x19,0x07, -0xec,0xd1,0x4b,0x0b,0x67,0x9d,0x64,0x3c,0x1b,0x25,0xc6,0x8d, -0x64,0x63,0xbe,0x3f,0xa1,0xa7,0xc9,0x87,0x46,0x26,0xee,0x9b, -0xae,0xd7,0xfe,0x8c,0x35,0xd0,0x5c,0x9b,0x2a,0x8d,0xe1,0xa7, -0x7a,0xb2,0x91,0x84,0x53,0xd4,0x6e,0xb3,0x34,0x48,0x49,0x59, -0x7b,0x0d,0x0e,0x85,0x69,0xe9,0x00,0xd5,0xec,0x4e,0x69,0x66, -0x4b,0x3c,0x42,0x1d,0x4b,0xef,0xf8,0x30,0x6f,0xb0,0x98,0x43, -0x5b,0x73,0xa6,0xf3,0x13,0xb4,0x02,0xa5,0x5c,0x60,0xbb,0x60, -0x56,0x63,0x83,0xda,0x70,0x12,0xa3,0x5d,0x2d,0x8b,0x99,0x55, -0xf1,0xb6,0xed,0x33,0xde,0x02,0xc8,0xb7,0x79,0x60,0xef,0xe4, -0x5b,0xd7,0xb3,0x3c,0xcb,0x39,0x13,0x53,0x7c,0x5d,0xba,0x13, -0xc2,0xb1,0x2f,0x3d,0x0a,0x0e,0x83,0x87,0x32,0xd7,0x3e,0x82, -0x44,0x5d,0xd9,0xc3,0x65,0x33,0x7b,0xb8,0xe6,0xe4,0x32,0xdb, -0x3a,0xbd,0xea,0xdc,0xf2,0x8b,0xfd,0x3d,0x79,0xf8,0xac,0x88, -0x87,0x9e,0xa7,0x9e,0x80,0x7f,0x8b,0x8c,0x29,0xdf,0x61,0x17, -0xd2,0xc5,0x48,0xb9,0x58,0x12,0xa5,0xc3,0x25,0x0e,0x06,0xaa, -0x26,0x77,0x74,0xb1,0x1c,0x0f,0x7f,0x55,0xed,0x36,0x68,0x22, -0x5a,0xdd,0x68,0x9c,0xcb,0xe8,0x0a,0xf5,0xa4,0x78,0x00,0xe3, -0xcf,0x05,0x12,0x3c,0x32,0x76,0x5b,0x98,0xf3,0x3c,0x7a,0x11, -0x10,0xf6,0x28,0x8a,0x07,0xc0,0xc3,0x23,0x79,0xf8,0xe5,0x91, -0x7c,0xf9,0x4f,0x8f,0x24,0x4c,0x53,0x16,0x50,0x64,0x6a,0xc3, -0xce,0xa3,0x19,0xd5,0x7c,0xb9,0x4a,0x2f,0x3e,0x2b,0xf4,0x80, -0x41,0xe8,0xf5,0x02,0x96,0xf3,0xbb,0xc5,0xe9,0xe5,0x50,0x78, -0x1a,0x6e,0xad,0xea,0x3a,0xf5,0xc4,0x14,0x61,0x57,0xa3,0xf1, -0x62,0x75,0x83,0x18,0xd6,0x95,0x3f,0x8b,0x84,0xe1,0x71,0x33, -0xac,0xdc,0x8e,0x6d,0x88,0xf2,0xf9,0xb3,0xec,0x7e,0x34,0x27, -0xad,0x7a,0xba,0x8a,0x44,0x29,0x46,0x10,0x0a,0xa7,0x13,0x03, -0xf3,0x0a,0xed,0x25,0xb5,0x18,0x41,0x59,0x50,0xa5,0xd3,0xdd, -0x28,0x39,0x6b,0xdf,0xca,0xab,0x34,0xc4,0x40,0x1b,0xf9,0x8e, -0x45,0x4b,0xc1,0x7f,0xb8,0x3f,0x84,0x58,0xca,0xd1,0x4d,0x1a, -0x3a,0x75,0x14,0x33,0x00,0x70,0x36,0x1f,0x68,0x71,0xbb,0x4d, -0x1d,0xef,0x74,0xf2,0xe6,0x5c,0x08,0x53,0xf2,0xec,0xed,0x31, -0xda,0x67,0xf0,0x18,0x8e,0x90,0x0a,0x61,0xd1,0x9e,0x4e,0x2e, -0x50,0xb8,0x7f,0xf3,0xf1,0x06,0x39,0x5d,0x54,0xea,0xf1,0x81, -0x58,0xd8,0x8d,0x8b,0x2a,0xc6,0x62,0x08,0x2f,0x6b,0x2a,0x87, -0xa6,0x4f,0xb5,0x17,0x18,0x81,0xf9,0x99,0xfc,0x9f,0xa2,0x8a, -0x07,0xe1,0xcd,0x0d,0x33,0xda,0xcc,0xce,0x51,0x8e,0x00,0xe3, -0x33,0x9e,0x95,0x57,0xc5,0xb6,0x7b,0x4b,0xbf,0x66,0xbc,0x51, -0xb5,0x86,0x7a,0x04,0x3b,0xfb,0x22,0x78,0xc3,0x4d,0xf8,0x22, -0x4a,0x6c,0xd0,0xce,0x5c,0x79,0x90,0xc7,0x75,0xd4,0xf8,0xfd, -0x6f,0x27,0x8c,0xb7,0xa1,0xf4,0xb8,0xda,0xc4,0x7e,0xcb,0xc2, -0x86,0x4d,0xfc,0x06,0xd8,0xa6,0x3c,0x78,0x15,0x0a,0x50,0xa0, -0x73,0xcb,0x55,0xd3,0xc0,0x2d,0xd0,0x12,0xa7,0x5d,0xf8,0x98, -0xbb,0x7c,0x60,0x0e,0x10,0x58,0x1d,0x0e,0x49,0x6c,0x6c,0x7f, -0x55,0xc0,0xd4,0x17,0x16,0xe5,0x85,0x6f,0x2c,0x03,0x8d,0x8a, -0x6b,0x87,0xa8,0x07,0xab,0x6f,0xee,0x07,0xc3,0x7a,0xa8,0x01, -0x90,0x1c,0xb6,0x60,0x8e,0xe6,0x7e,0x02,0x6b,0x75,0xde,0x58, -0x1f,0xbe,0x11,0x70,0xb9,0xdd,0xb1,0x75,0xce,0xcc,0x57,0x28, -0xac,0x9d,0xe5,0x04,0x35,0x4e,0xd8,0xa7,0x5c,0x29,0x66,0x3c, -0x59,0xc0,0x63,0x38,0x45,0x41,0x5b,0x6b,0x20,0x43,0x40,0x65, -0x6c,0xcf,0xef,0x28,0xf9,0x05,0x10,0x05,0x31,0xb9,0x27,0x49, -0xc6,0x68,0x38,0x39,0x9a,0x2c,0xc5,0xec,0x67,0xa6,0xfd,0xd1, -0x7c,0x72,0xa7,0x1c,0x02,0x76,0xfb,0x8b,0xe6,0x82,0x41,0xab, -0x35,0x28,0x70,0xe0,0x99,0x64,0xbe,0x69,0x1a,0x08,0xaa,0x81, -0xc6,0x72,0x73,0xb4,0xb3,0xfd,0x08,0xe0,0x20,0x45,0x1b,0x87, -0xd2,0x88,0xd0,0xa8,0x61,0xbe,0x23,0x8d,0xd8,0x51,0xd0,0x5b, -0x4e,0xbf,0xcb,0x15,0x3b,0xd9,0xf4,0x85,0xf9,0xa0,0xcb,0xd5, -0xa7,0x20,0x59,0x38,0x77,0x13,0xc7,0xe7,0x5d,0xde,0x82,0x2d, -0xf9,0x87,0x55,0xeb,0x62,0xcc,0x27,0xdf,0x4e,0xa2,0x1e,0x52, -0x0c,0x41,0x60,0xed,0xfd,0xb0,0xa9,0x19,0xf7,0x0a,0xc2,0xca, -0xb6,0x3a,0x01,0x30,0x52,0xdc,0x4f,0xb4,0x92,0x25,0xdc,0x70, -0x05,0x67,0x5e,0x52,0x38,0xf3,0x3d,0x65,0xe5,0xb5,0xf2,0xa5, -0xb5,0x41,0xf6,0xa9,0x22,0x2b,0x37,0x5b,0x98,0x11,0x7c,0x72, -0x78,0x06,0xe7,0x57,0xb0,0xfd,0xf8,0x78,0x71,0xec,0x4d,0x62, -0xe6,0xc9,0x49,0xe8,0xcf,0x1c,0xbd,0xbe,0x2a,0xcb,0x2c,0x2f, -0xcb,0x52,0x6b,0xea,0x05,0x47,0x4b,0x51,0xdd,0x0c,0xa7,0xd3, -0xf9,0xc7,0x93,0xf9,0xe2,0x73,0x1a,0x1e,0x3a,0x21,0x33,0x41, -0x9c,0xd5,0xa6,0x9e,0x0f,0x1b,0xe4,0x66,0x62,0x0e,0x3a,0xaf, -0xad,0x53,0x1a,0xcf,0xb5,0x16,0xc0,0x3d,0xaf,0x08,0x99,0x21, -0x0e,0x4c,0x44,0x41,0xbe,0x78,0x7d,0x72,0xfc,0x22,0x90,0x1a, -0x48,0xfb,0xff,0xb3,0x6c,0x0f,0xb2,0xf4,0xbc,0xd7,0xcb,0xf0, -0xf3,0xfc,0xe3,0x37,0xf8,0xbb,0xbb,0x9f,0xf3,0x3b,0x41,0xc0, -0xa6,0xe2,0xa8,0xf5,0x9a,0x3e,0xbd,0x83,0xfa,0x2d,0x55,0x8a, -0x0b,0x5d,0x2b,0x06,0x39,0xc1,0x47,0xae,0xb9,0xcc,0xca,0x2b, -0xa7,0x11,0x8a,0x74,0x0c,0xab,0x79,0x73,0x0d,0x7f,0x45,0xf5, -0x4d,0xae,0x7e,0xbc,0x12,0xef,0xeb,0xbe,0xb4,0xbc,0x86,0xd8, -0xbe,0x64,0x64,0x49,0x7b,0xa8,0x24,0x99,0x62,0x8a,0xd2,0x52, -0x42,0x94,0xdc,0x96,0xa4,0x8d,0xa8,0x18,0x10,0x9a,0x1e,0xd1, -0xfc,0x87,0x03,0xa5,0x92,0xa5,0xbb,0xbb,0x13,0xee,0x16,0x3a, -0x65,0x82,0x83,0xa5,0xea,0x3b,0x68,0x03,0xb4,0xb2,0x32,0xee, -0x68,0x85,0x04,0x85,0x49,0xea,0xe4,0x7f,0x26,0x85,0xcc,0xc9, -0xfc,0x02,0x75,0x59,0xf9,0x76,0xa7,0x5c,0x6e,0x24,0x44,0x4e, -0x8c,0xc0,0x59,0xa0,0xbc,0xa4,0x85,0x22,0x82,0xfa,0x05,0xc2, -0x92,0xbd,0xba,0x53,0x1e,0x88,0xa5,0x02,0xa5,0x99,0x1e,0x21, -0x96,0x5c,0x1e,0x14,0x78,0x8a,0x41,0xdb,0x37,0xb9,0x34,0xca, -0x9f,0xa5,0x06,0x58,0x66,0x31,0xdc,0xa4,0x4c,0x0a,0xd4,0xa1, -0x93,0xb1,0x83,0x53,0x27,0x2e,0x2a,0xca,0x24,0x6e,0xde,0x4a, -0x7e,0xc8,0xab,0x55,0xe1,0xe7,0xb3,0x74,0x79,0xbb,0x80,0xba, -0x03,0xbe,0xba,0x5e,0xd7,0xf4,0x9b,0xdd,0x82,0x28,0x6b,0x1d, -0x71,0xa5,0x83,0x46,0xe7,0xf1,0xf1,0x37,0x83,0xf6,0x12,0x74, -0x38,0x02,0x9f,0x7e,0x82,0xeb,0xaf,0x42,0xaf,0x77,0x82,0xeb, -0xdd,0x55,0xea,0xdc,0x3d,0x3a,0xff,0xd8,0x0b,0xc4,0x6c,0x8f, -0x8b,0x92,0xf3,0xd0,0x24,0x1e,0xdd,0xad,0x5b,0x8c,0x5d,0x82, -0xb3,0x62,0xc1,0xef,0x6b,0x7c,0xb5,0x45,0x2b,0x93,0x94,0x98, -0x28,0x23,0xc3,0x6b,0x2e,0xee,0x6c,0xaf,0xf9,0x9c,0x40,0xe0, -0xc5,0x75,0xe3,0x5d,0x1f,0x64,0x70,0xcd,0xec,0x7d,0xb1,0xc1, -0x6c,0xaf,0xc0,0xcd,0xa0,0xad,0x3d,0xb7,0xed,0xdb,0x0e,0x28, -0xcb,0x6c,0xc6,0x4d,0xb0,0xc6,0xda,0x49,0xf1,0xe5,0xe5,0x53, -0x6b,0x99,0x57,0x1d,0x14,0xe3,0x11,0xc1,0x28,0x44,0x63,0xa4, -0x96,0x19,0x36,0x52,0xaf,0x9a,0xab,0x68,0xbe,0x3a,0x27,0xd5, -0xe5,0x82,0xec,0xc3,0xc0,0x7e,0x9c,0x8c,0x5a,0x72,0x0a,0xb4, -0xa8,0xf2,0x96,0x3f,0x87,0x50,0x48,0x26,0x58,0x40,0xb0,0xb4, -0xba,0xad,0x27,0x2e,0xbd,0x68,0xdd,0x77,0xbb,0x5d,0xf4,0x69, -0x5d,0x40,0xe2,0x23,0x58,0x5d,0xf0,0x89,0xfe,0xad,0x1f,0x5a, -0x3f,0xfe,0x68,0xcc,0x7d,0x4d,0xec,0x58,0x13,0x8d,0x1b,0xd5, -0x5d,0xd7,0x59,0xeb,0xd5,0x0d,0xa9,0x0d,0xd2,0x3a,0xaa,0xa9, -0xa2,0x99,0x9f,0x87,0xe6,0x2b,0xca,0x7b,0x3e,0x61,0x0a,0x79, -0x1c,0x94,0x9b,0xa5,0x70,0x57,0x0c,0xdd,0x41,0x55,0x21,0xd7, -0xa4,0xe1,0xc5,0xd6,0xee,0xd3,0xcd,0x91,0xaf,0xa5,0x6c,0x92, -0x73,0xc9,0xe1,0xb8,0x2c,0x4b,0x2b,0xf7,0x62,0x3a,0xc1,0x77, -0x5b,0x3f,0x94,0x84,0x19,0x58,0x67,0xaa,0xce,0x70,0x9c,0x11, -0x1b,0x30,0x00,0x9c,0x3d,0x8e,0xdd,0xc0,0xd4,0x32,0xe2,0x56, -0x21,0xd0,0x00,0xc5,0x6a,0x80,0x6f,0xc5,0x82,0x7c,0xea,0x71, -0x85,0x82,0xa9,0x6c,0xe8,0x87,0x87,0xac,0xdf,0x9c,0x6d,0x50, -0x36,0x80,0x44,0x19,0x47,0x7e,0xc0,0x33,0xb8,0x7a,0xfc,0x16, -0x88,0xec,0x05,0x46,0x3d,0x4c,0x2e,0xe4,0xe5,0x07,0x74,0xd1, -0x6e,0x48,0x18,0x71,0x9a,0x62,0xe2,0xed,0x76,0x84,0x4d,0x85, -0xc9,0x16,0xa9,0x7a,0x48,0x3c,0x28,0x27,0xc5,0x47,0xb8,0x05, -0xcf,0x86,0xd7,0xa9,0x29,0x43,0xab,0xf2,0xe1,0x27,0x1c,0xf7, -0x06,0xfe,0x95,0xcf,0x38,0x6e,0x5c,0x0c,0x09,0xf1,0xd0,0xc0, -0xf8,0x6c,0xff,0x9f,0x1a,0x19,0xf7,0x1a,0xe5,0x16,0x45,0x88, -0x43,0x7b,0x09,0x18,0x43,0x78,0x7a,0x96,0x5f,0xe3,0x66,0x9e, -0x46,0x0f,0x59,0x8c,0x34,0xe9,0x7b,0x8c,0xb2,0x69,0xca,0xbd, -0x9a,0xa4,0xbb,0x5f,0x3d,0x88,0x8c,0x9b,0x21,0xc3,0x4f,0x38, -0x90,0xc2,0x93,0x88,0x25,0x8b,0xb7,0x9d,0x53,0x9c,0xba,0x8a, -0xbd,0x7d,0x72,0xb4,0x95,0xae,0x28,0x23,0xdc,0x75,0x37,0xb2, -0xad,0x6e,0xb2,0x3a,0x3d,0x4e,0x0f,0x6e,0x1b,0xcd,0xdd,0xf0, -0x0e,0x1c,0x10,0xee,0xb2,0x7a,0x6a,0x8f,0x65,0x46,0xa4,0xc3, -0xcb,0x16,0xfa,0x5c,0x28,0xd4,0x11,0x3c,0x1d,0x98,0x03,0x0b, -0x70,0x2e,0x4b,0xfd,0x06,0x29,0xe3,0xd9,0x28,0x62,0x65,0x05, -0x02,0x27,0x86,0xd3,0x61,0x19,0x0f,0x4d,0x14,0xb5,0x5b,0x0f, -0x5e,0x25,0xad,0x36,0xb5,0x8f,0xeb,0xc2,0x88,0x19,0x5f,0x69, -0x68,0x49,0x36,0x5c,0x17,0xa6,0x99,0xfc,0xf6,0x8e,0xa4,0x3f, -0x09,0xd8,0xbc,0x9a,0x3f,0xb9,0x5b,0x4c,0x27,0x97,0x88,0x0b, -0x26,0x72,0xbd,0x8d,0x5a,0xf2,0x08,0xbf,0x6d,0xda,0xfd,0xaa, -0x0b,0x26,0xd9,0x58,0x3b,0x0a,0xab,0xc7,0xea,0x42,0x81,0x11, -0x3b,0x00,0x28,0xd6,0x22,0x35,0x3b,0xf1,0x96,0xad,0x0b,0xce, -0x5f,0x5d,0xf9,0xc3,0x10,0x0d,0xa4,0x78,0xcd,0xc6,0x31,0x46, -0xf0,0x32,0x0b,0x37,0x66,0x24,0xcb,0x99,0x6f,0x1f,0x56,0xbf, -0x47,0xb3,0xe0,0xe6,0x7a,0x62,0x5f,0x84,0xeb,0x91,0xeb,0xf5, -0xfd,0x26,0x6f,0x98,0x89,0x3f,0x98,0x7b,0xe9,0x95,0x68,0xe5, -0x5d,0x36,0x5e,0x4c,0x46,0x2a,0x33,0x28,0x31,0xd8,0x3a,0x49, -0x5f,0x51,0xd6,0x0e,0xf3,0xf6,0x3e,0xfb,0xbd,0xb6,0x65,0x3d, -0x69,0x86,0x8a,0x2c,0xce,0x34,0x1d,0x6d,0xdb,0xda,0xd8,0x90, -0xd7,0x57,0x5c,0x3f,0xe5,0xea,0x5b,0x21,0x4f,0xd4,0xfd,0x5a, -0x59,0x86,0x1d,0xe8,0xa5,0x7e,0x77,0x7d,0xed,0x8e,0xfc,0x5d, -0x67,0x59,0xe4,0x14,0xed,0x64,0x7e,0xcb,0x4e,0xdb,0x5a,0xb5, -0x39,0x2d,0xd0,0xdd,0x87,0x05,0x16,0x4d,0x03,0xf9,0x41,0xc9, -0x2c,0x9b,0xb2,0xb4,0x0d,0x41,0x46,0xa9,0x9c,0x95,0x3f,0x03, -0x05,0x93,0x06,0x7c,0x4a,0x75,0x98,0xe8,0xcc,0xa2,0xb8,0x5e, -0x87,0x10,0xb3,0x86,0x43,0xd4,0x89,0x4d,0x36,0x5c,0x4b,0x70, -0xac,0xc6,0xc3,0x5f,0x3b,0xb0,0xdb,0xed,0x38,0x86,0x09,0xa2, -0xac,0x30,0x63,0x2f,0x0d,0xb4,0xee,0xdd,0x6c,0xb3,0xc8,0xec, -0xec,0x0a,0xb9,0x90,0x69,0x38,0xfc,0x5d,0xf4,0xa7,0x91,0xa2, -0xc9,0xf4,0xdf,0xdb,0xf0,0x2c,0x10,0xea,0x76,0xc3,0x81,0x47, -0x76,0xc3,0xda,0x95,0xe4,0xa4,0x7e,0xbd,0x6d,0x5f,0xb5,0xb5, -0x8b,0xb0,0xe1,0xc2,0x17,0xa3,0x71,0x0d,0x92,0xee,0xbe,0x49, -0xbd,0x7a,0x5c,0x78,0xc2,0x05,0xab,0x26,0x2b,0xea,0xd9,0xe3, -0xfa,0x46,0xe3,0x99,0x3e,0x28,0xa9,0xda,0x4b,0x7c,0x48,0x07, -0x92,0xe5,0xde,0x7e,0x44,0xed,0x36,0x36,0xbd,0x8f,0x27,0x93, -0x7d,0x10,0x8a,0x0f,0x9b,0x08,0x7c,0x27,0x11,0xaf,0x84,0xc1, -0xcc,0x06,0x6b,0x5b,0xc1,0xe1,0xc2,0x3e,0xdc,0xf1,0xdf,0x72, -0x6d,0xaf,0xd7,0x87,0xf8,0x5c,0x69,0x96,0x82,0x93,0x61,0x68, -0xa6,0xde,0x56,0x8b,0xe9,0x12,0xc5,0x96,0xb5,0xc7,0x4d,0x64, -0x1a,0x9b,0x35,0xbf,0x19,0xc2,0xf7,0xf8,0xd3,0xf0,0x72,0x35, -0xfd,0xdc,0x82,0xa5,0xd8,0x42,0xe6,0xa0,0x73,0x2f,0xd8,0x22, -0xd3,0xb4,0x21,0xca,0x96,0x90,0x05,0x82,0x2d,0x9b,0x2f,0x98, -0x2b,0xbb,0x44,0x08,0xaf,0x80,0x0b,0xe8,0xcc,0xce,0x1f,0x90, -0x59,0x78,0xd0,0x14,0xf7,0x9b,0xcd,0x91,0x91,0x20,0x32,0x68, -0x8b,0x58,0x60,0xae,0x8b,0x8f,0xd8,0x7a,0xec,0x63,0xdd,0x82, -0x2c,0xeb,0x4f,0x3a,0x87,0x79,0x0d,0xf9,0xd9,0xc3,0xe8,0x8c, -0x2e,0xc6,0xa0,0x15,0x10,0x75,0x3b,0x5e,0x5e,0x8f,0x6d,0xa4, -0x70,0x22,0x6b,0x6c,0xca,0xb0,0x60,0x23,0x8e,0xc5,0x0f,0x50, -0xf5,0x8d,0x1b,0xad,0x0a,0x72,0xde,0x55,0x5b,0xbd,0x3f,0x2d, -0xa7,0xd9,0xff,0xc9,0x3d,0xa9,0x0b,0x9e,0x18,0xfa,0xdb,0x96, -0x7b,0x20,0x80,0x25,0x75,0x9c,0xf9,0x06,0xa7,0x0d,0xe3,0xdf, -0x30,0xfa,0x13,0x0b,0x26,0xdf,0xbe,0xa9,0xc3,0x57,0xa7,0xda, -0x51,0xd7,0x3c,0xe8,0x47,0x6c,0xbe,0xfa,0xaa,0xae,0xe6,0x62, -0xc9,0xa4,0x5a,0xe2,0x03,0x67,0x4b,0x63,0xcd,0xfe,0xec,0x62, -0x88,0x59,0xcf,0x3c,0x79,0xb1,0x5e,0xe2,0xb6,0xe7,0xc2,0x62, -0x2a,0xc4,0xa1,0x29,0x21,0xc4,0x16,0x3e,0x6f,0x69,0xb0,0xf9, -0xa9,0xe9,0x77,0x10,0x5f,0xc1,0x55,0xb2,0xb1,0x2b,0x8c,0xef, -0x2e,0x79,0xa2,0x8d,0xc4,0xd8,0x48,0xa4,0x32,0x46,0x2e,0x5e, -0xa2,0xc9,0xc4,0xdb,0xe1,0xa7,0xb4,0x76,0xef,0x35,0x60,0xe0, -0x4e,0x86,0x3b,0x16,0x77,0xad,0xa3,0x9d,0xed,0x76,0x2d,0xaa, -0x6b,0x45,0x93,0x23,0xf9,0xb6,0xb2,0x01,0x27,0x12,0x1a,0x3b, -0x90,0x38,0xf6,0x85,0x37,0xf6,0x45,0xd2,0x0b,0x0e,0x98,0xf3, -0x83,0x65,0xaf,0x9c,0x65,0xac,0x87,0xe4,0x4e,0x68,0x74,0x9d, -0x74,0x16,0x90,0x3a,0x4e,0x78,0x49,0x1d,0x47,0x96,0x65,0x19, -0x8b,0x1c,0xe8,0x58,0xa0,0xea,0xb4,0x13,0x5b,0x63,0x4d,0x6a, -0x28,0xf4,0x65,0x95,0xb8,0xec,0x01,0xc2,0xb9,0x32,0x0a,0x65, -0x68,0x2d,0xd2,0xe8,0xc4,0x65,0x5f,0x41,0xf9,0xe1,0x7e,0xd8, -0x42,0xfc,0xe5,0xa9,0x19,0x08,0x6b,0x16,0x64,0xbd,0x36,0x09, -0x4d,0xf4,0x17,0x5a,0x51,0xdf,0x69,0x50,0xc7,0x72,0xe6,0x50, -0x74,0xa8,0x33,0xd6,0x44,0xf6,0x23,0x6a,0x55,0xd3,0x4a,0x79, -0x43,0xf4,0x31,0xf9,0xbd,0xd0,0x9d,0x45,0x38,0x0d,0x44,0x73, -0x16,0x66,0x32,0xf0,0x35,0xcc,0xcc,0x6f,0xac,0xf6,0xa5,0x0f, -0x28,0x2e,0xe2,0x0b,0x7a,0x60,0x32,0x7f,0xb7,0xa1,0x93,0xe7, -0x6d,0xd7,0xc4,0x08,0x6d,0x8d,0x2d,0x2f,0xc5,0xa1,0xf9,0xf2, -0x12,0x93,0x4a,0xf8,0xe5,0xa7,0xaa,0x1c,0x92,0x29,0x06,0x7e, -0x25,0xc6,0x61,0xce,0x14,0xa5,0x5f,0xee,0x35,0x69,0x64,0x0d, -0x46,0xf2,0xfb,0x2f,0xa9,0x89,0x27,0xbb,0xc9,0x0e,0xfa,0x83, -0xf4,0x6f,0xb7,0x38,0x9a,0xcb,0x4b,0x52,0xc5,0xd6,0x97,0xc6, -0x4e,0x99,0x26,0xd5,0xea,0xf3,0x74,0x8c,0xd6,0x15,0x21,0xa1, -0x97,0x1c,0x25,0x05,0xe2,0xd3,0x1d,0xcd,0x99,0xe5,0x54,0x6d, -0x35,0x5e,0x79,0x8d,0xfd,0x1c,0x0e,0x00,0x69,0x39,0xe7,0x41, -0xe6,0x9c,0xb6,0x07,0x12,0x9a,0xdb,0x43,0x92,0x47,0x09,0x57, -0x93,0x36,0x6b,0x46,0x88,0x98,0x19,0xd4,0xd8,0x97,0xec,0x83, -0x32,0xb5,0x5f,0x3d,0xfb,0x81,0x88,0x7f,0x01,0x18,0x46,0x87, -0x8b,0x15,0xae,0x27,0x58,0x83,0xd7,0xaa,0x58,0xa1,0x35,0x24, -0x4e,0xca,0x1b,0x63,0xd1,0x30,0x4c,0xe2,0x80,0xec,0x32,0x00, -0x3f,0x6c,0x40,0xf8,0xc2,0x08,0x34,0x58,0x26,0x5f,0xaf,0xa9, -0x6d,0xc6,0x78,0x80,0x4c,0x1e,0x47,0x05,0x23,0xb4,0xc9,0x6a, -0xfa,0x1b,0xcd,0xb7,0x62,0x7e,0x31,0x86,0x91,0x74,0x89,0xaf, -0xb6,0xdd,0x82,0x7c,0xef,0x04,0xcf,0xb3,0xf5,0x4b,0x88,0x9f, -0x67,0xaf,0xe0,0x48,0x6e,0x80,0x17,0xc4,0x9f,0x98,0x27,0x60, -0xbc,0x5e,0xd8,0x47,0x45,0x7f,0x10,0x35,0xe7,0xe4,0x01,0x22, -0x0b,0xf9,0x4f,0x64,0xe2,0xb2,0x01,0x2b,0xa8,0x9c,0xb3,0xfb, -0x11,0xec,0xd9,0x0f,0xe3,0xd1,0x69,0x90,0xcf,0x18,0xad,0xa8, -0x01,0xc9,0xef,0x3d,0x29,0x5e,0x04,0x9c,0x34,0x18,0x38,0x89, -0xf0,0x2c,0x75,0x56,0xfd,0x27,0x76,0x3c,0x05,0x05,0x51,0x61, -0x09,0xe3,0xcc,0x16,0xb5,0xc4,0x45,0xbd,0xd6,0x00,0xc9,0xea, -0x3d,0x9c,0x9e,0x36,0xcc,0x4c,0x56,0x3c,0x5c,0xe6,0xc8,0xe9, -0xc2,0x37,0xd1,0x6c,0xa4,0xf1,0x47,0xe7,0x35,0xaa,0xef,0x61, -0xe0,0x6c,0x09,0x08,0xf8,0x78,0xf4,0x76,0x5c,0xcd,0xef,0x96, -0x97,0x54,0xab,0x6d,0x21,0x0c,0x12,0x6a,0x05,0x16,0xa1,0x92, -0xe0,0x26,0xeb,0x56,0x77,0x97,0x97,0x63,0xd8,0x62,0x5e,0xbb, -0x82,0xa9,0x0a,0x7d,0xd3,0x08,0x11,0x9a,0x00,0x63,0xf5,0x84, -0xcb,0x11,0x59,0xb3,0xa2,0x72,0x8d,0x24,0x92,0xc2,0x6c,0x98, -0xbd,0x80,0x48,0x7a,0x98,0x12,0x52,0x28,0xff,0xbf,0x27,0x81, -0x1a,0x46,0x81,0x30,0x08,0x3b,0x61,0x47,0xb5,0xe1,0x36,0xf4, -0x4d,0xc0,0xaf,0x08,0xb6,0xf8,0x56,0x92,0x29,0xff,0x5a,0xaa, -0xa7,0x3e,0xcf,0x0f,0x50,0x34,0x22,0xe6,0x55,0x2b,0xc1,0x64, -0x8b,0x9d,0x92,0xad,0xc7,0x44,0x03,0x57,0x42,0x27,0x9a,0x04, -0xdc,0xcd,0x11,0xa1,0x2e,0x39,0x9a,0x8e,0x88,0x2c,0xdf,0x72, -0x8c,0x95,0xbf,0x4f,0x7e,0x6d,0x0b,0xba,0x1e,0x0e,0x6c,0x7d, -0x02,0x8d,0x76,0x6a,0xa3,0x46,0x13,0x09,0x72,0x4e,0xb2,0x7b, -0x12,0x3a,0xb4,0x2b,0x0f,0xb9,0xdb,0xa6,0x04,0xe0,0x4d,0xf1, -0xb9,0x89,0x77,0xe7,0xb6,0xa3,0x78,0xbb,0x62,0x29,0x0a,0xe9, -0x9a,0xe7,0xa1,0xfa,0xd1,0x7f,0xe4,0x4e,0x71,0xef,0x4a,0x95, -0x30,0x0e,0x42,0xb3,0x7d,0xa2,0x9e,0xcb,0xe1,0x04,0xbe,0x50, -0xad,0xd1,0x7a,0x46,0x8c,0x79,0xeb,0x7b,0xd4,0x90,0xc3,0xbc, -0x48,0x36,0x17,0x6f,0xba,0x3e,0x8e,0x9a,0x1b,0x00,0xfb,0x78, -0xeb,0xd2,0xc2,0xd7,0x44,0x85,0xb6,0x8d,0x41,0x16,0x6c,0xa5, -0xa8,0xe9,0xca,0xfd,0x6b,0xae,0xd9,0x37,0x35,0xdb,0xba,0xf8, -0xd2,0x6d,0x97,0x75,0xba,0xf5,0x72,0x7d,0x48,0xbe,0xca,0xb0, -0xc7,0xbf,0x36,0xdf,0xc6,0x5f,0xd6,0x7c,0xc5,0x89,0xa9,0x64, -0x23,0x2e,0x56,0x2d,0xe6,0xb3,0x0a,0x4f,0x0b,0x00,0x73,0x33, -0x1e,0xa2,0x37,0x43,0x84,0x70,0x35,0xb9,0x76,0xd6,0xf1,0xeb, -0xc7,0xe1,0x7c,0x38,0x0a,0xdc,0x12,0x61,0x44,0x4b,0x0f,0x30, -0xb1,0x92,0xcf,0x40,0xd0,0xae,0x2a,0xe1,0x7f,0x5a,0x23,0x93, -0x37,0xf5,0x3e,0xd3,0x1e,0xd9,0xd2,0x8d,0x7b,0xd7,0x8d,0x5e, -0xea,0xd7,0x0c,0xe1,0xe5,0xa2,0x9a,0x1b,0xc5,0xce,0xdc,0x0c, -0x99,0x48,0xab,0x9c,0x15,0xa7,0x99,0xca,0xb2,0x62,0xdb,0x74, -0x1a,0x51,0x92,0x7f,0x76,0x66,0x1f,0xea,0xe5,0xe6,0x6b,0xf2, -0x78,0x3c,0xd1,0xcb,0x91,0xa7,0xc3,0xfc,0xc2,0x09,0x6c,0x5c, -0x38,0x4a,0x6a,0x6f,0xe8,0x82,0x86,0x8a,0xad,0x71,0x84,0x3e, -0xde,0x0c,0x57,0x64,0xb0,0x6c,0x32,0xbf,0xab,0x9a,0x78,0x41, -0x4e,0x84,0x14,0xee,0xe8,0x5a,0xbe,0x2d,0x57,0xe6,0xed,0xdd, -0x74,0x35,0x19,0x91,0xb4,0xf8,0x4b,0x0c,0x2e,0xa6,0x63,0x23, -0x08,0xdf,0xea,0xc3,0x0a,0xc9,0xf1,0x3a,0x1c,0x40,0x83,0x7e, -0x45,0xf9,0x70,0xbc,0x52,0xef,0x1f,0x6d,0x5a,0x24,0xe7,0xf1, -0x2d,0x3a,0x59,0x88,0x6b,0x8a,0x2d,0x77,0xd0,0x27,0x35,0xde, -0x32,0x2d,0xcc,0x53,0xba,0x21,0xdc,0xbf,0x46,0x31,0x61,0x05, -0x79,0xd4,0xfe,0x92,0xcb,0x09,0x87,0x6c,0x20,0xea,0x85,0x99, -0xc8,0x8d,0x5c,0x90,0xc7,0xd8,0x17,0x11,0x72,0xf1,0x5e,0x47, -0xbe,0x38,0xc8,0x63,0x7b,0x2b,0x91,0x94,0x37,0xad,0x55,0x23, -0x9b,0xc4,0x52,0xe9,0x6a,0x56,0x80,0x34,0x4d,0x51,0x5a,0x4e, -0x64,0x87,0x54,0xe6,0x42,0xa2,0x13,0x12,0xd5,0x3b,0xd2,0x4c, -0x5c,0x79,0xd0,0xb6,0xe8,0x8d,0xaf,0x19,0x8a,0xab,0x25,0xa3, -0x77,0x3f,0x07,0x3c,0x14,0xa5,0x0a,0xe0,0xd6,0xc4,0xef,0x66, -0x72,0xef,0x38,0xb5,0x0c,0x95,0xbf,0xdb,0x04,0xe4,0x8d,0xc7, -0x56,0x51,0x56,0x00,0x86,0xd3,0xab,0x39,0x38,0x75,0x0d,0xd6, -0xc5,0x91,0x15,0x9d,0x7c,0xc2,0xbd,0xfa,0x23,0x6b,0x28,0xd5, -0xb3,0xad,0xd7,0xc9,0xf3,0x97,0x7f,0x05,0x62,0x36,0xd2,0xa7, -0x83,0xe3,0x3f,0x01,0x1a,0x6b,0x5b,0x99,0x19,0x5a,0x09,0x69, -0x4c,0xcc,0x7a,0x84,0x55,0xbf,0x7d,0x7a,0xfa,0xfa,0xa7,0xb7, -0x27,0x4f,0xdf,0xfd,0xf4,0xf6,0x45,0xc1,0x26,0xe2,0x82,0x25, -0xf5,0x7b,0xf5,0x96,0x9c,0x74,0xe6,0x03,0xab,0x4b,0xc9,0x67, -0xe2,0xe8,0x86,0xb3,0x88,0x9b,0x8f,0x77,0xd5,0x82,0xc9,0xf1, -0x19,0x31,0x3c,0xfe,0x6f,0xee,0xde,0x7e,0xbf,0x8d,0xdc,0x58, -0x14,0xfc,0xfb,0xde,0xfb,0x12,0x54,0x1f,0x85,0xee,0x1e,0x42, -0x94,0x28,0x4f,0x72,0x92,0xe6,0xb4,0x79,0x34,0xb2,0x1c,0x3b, -0xb1,0x2c,0x1d,0x49,0x93,0x93,0x44,0xe2,0x38,0x94,0xd8,0x92, -0x68,0x53,0xa4,0x86,0x1f,0xb2,0x1d,0x91,0xef,0xb2,0x7f,0xec, -0x93,0xdc,0x27,0xdb,0xfa,0x02,0x50,0xe8,0x6e,0x4a,0xb2,0x93, -0xbd,0x39,0xbb,0x33,0x3f,0x8b,0x8d,0xaf,0x42,0x01,0x28,0x14, -0x0a,0x40,0xa1,0x2a,0x3a,0xde,0x7b,0xbb,0xb7,0x7b,0x42,0x11, -0x61,0xfb,0x57,0x4c,0xcc,0x69,0x3e,0x24,0x28,0x30,0x31,0x45, -0x0f,0x13,0xf9,0xf7,0xec,0x1a,0xc4,0x58,0x0b,0xfd,0x99,0x7f, -0xd3,0xc1,0x0a,0x7c,0x33,0xeb,0x05,0x36,0xef,0x37,0x6b,0xc2, -0x94,0x84,0xc9,0xeb,0xa9,0xc7,0xd5,0xb6,0x57,0x4f,0x05,0x34, -0xf0,0xfd,0xc8,0x64,0xd0,0x0a,0x26,0xe2,0x69,0xd8,0xab,0x6d, -0xf2,0x83,0x41,0x15,0x01,0x5b,0xec,0x62,0x54,0x76,0xbf,0xa4, -0xde,0x5b,0x69,0xe5,0x4a,0xde,0x0d,0xd2,0xc9,0x58,0x75,0x17, -0x41,0x3b,0xc6,0x37,0xec,0x2b,0x9b,0xd4,0xd6,0xec,0x30,0xa0, -0xf9,0x43,0xe2,0x53,0xaf,0x4f,0xf6,0xdf,0xd6,0x5e,0x1e,0xec, -0xb3,0x47,0x69,0xdf,0x5b,0xd3,0x1a,0x4c,0xae,0x1a,0xba,0x1b, -0x19,0x92,0x91,0xd7,0x66,0xad,0x76,0x38,0xcc,0xf1,0x65,0x13, -0x5a,0x7a,0xc5,0x3e,0x86,0xb9,0x55,0x43,0x2c,0x09,0x54,0x0c, -0xbb,0xaf,0x6b,0xf4,0x82,0x06,0xb1,0x17,0xc0,0x8c,0x3f,0x92, -0x71,0x14,0x58,0x82,0x6b,0xe8,0x58,0x7a,0x44,0x51,0x89,0x8d, -0x6b,0x46,0x05,0x9e,0x53,0x56,0x41,0x15,0x7b,0x52,0x6b,0x5b, -0x66,0xc5,0x1c,0xe3,0x46,0x9b,0x02,0xeb,0x52,0x65,0xb3,0x20, -0xc9,0xae,0xad,0xb1,0xd6,0x9c,0xa5,0x7c,0x68,0xb3,0xa5,0x18, -0x97,0x9d,0x76,0x93,0x84,0x2c,0xe4,0x03,0x08,0x3c,0x03,0x2e, -0x0d,0x4d,0xbd,0x5e,0x8c,0x11,0x1b,0x48,0xa2,0x83,0x2b,0xea, -0xc5,0xc9,0xd3,0xd8,0x0d,0xc3,0x9a,0xe6,0x81,0x67,0x57,0xf4, -0x6c,0x90,0x04,0x4e,0x48,0x57,0xee,0x95,0x84,0xd2,0x60,0x47, -0x70,0x44,0xe6,0x06,0xf1,0x4a,0xe6,0x9d,0x63,0xc5,0xa4,0xa5, -0x7e,0x89,0xbe,0x49,0x24,0xbf,0xcd,0x96,0x15,0xcb,0xf1,0x05, -0x32,0x7e,0xb1,0x1b,0x80,0x62,0xba,0xc8,0xd7,0xa2,0xaf,0x99, -0x55,0xc1,0x14,0x66,0x8f,0xb5,0x23,0xe9,0x06,0xaf,0xc8,0x95, -0x89,0x0b,0x1d,0x5f,0x3a,0x8a,0x2d,0x94,0xc3,0x9d,0x44,0x65, -0x5d,0xe2,0x77,0xc4,0x65,0x6b,0x34,0xba,0x99,0xb4,0xdc,0xbd, -0x28,0xfe,0x90,0x0d,0xcc,0x87,0xed,0xec,0x43,0x43,0xb5,0x6b, -0xa3,0x85,0x8f,0x85,0xab,0x50,0xb0,0x4f,0x85,0xa1,0x08,0xe0, -0x81,0xa1,0xed,0x4e,0x50,0xc7,0x87,0x6e,0x16,0x86,0xb7,0xbb, -0xa9,0x98,0x59,0x2c,0xe4,0x6b,0x57,0xc0,0xdf,0xc8,0x02,0x2c, -0x58,0xe3,0x74,0xc9,0x3d,0x56,0xaf,0xcb,0x72,0x26,0xa3,0x4c, -0xf2,0x56,0x2c,0xcd,0xa9,0x1c,0x3e,0xbe,0x3c,0xbc,0x9c,0xf4, -0xae,0x42,0x07,0x01,0xac,0x66,0x60,0x0d,0x46,0xbf,0x92,0xf4, -0x18,0x36,0xa1,0xf2,0x89,0x6e,0x52,0xf2,0x51,0x9f,0x6b,0xa8, -0x84,0x6c,0x09,0xe8,0x94,0xf7,0x0d,0xcd,0xfc,0xf3,0x6d,0x6f, -0xd4,0x1f,0x77,0x2b,0x87,0xa1,0x98,0xc9,0xf5,0xfd,0xc7,0xac, -0x65,0x3e,0x7e,0x5c,0x45,0x45,0xed,0x8f,0x1f,0x5f,0x7c,0x6c, -0x7f,0xb4,0xef,0xb2,0xed,0xd1,0x6f,0x89,0x26,0x3f,0x76,0xdb, -0xb2,0x7b,0xb1,0xf2,0x90,0xb5,0x52,0x8a,0x86,0x71,0x2b,0x5a, -0x64,0xb3,0x39,0x6f,0xfa,0x65,0x88,0xcb,0x0a,0xca,0xb7,0xa4, -0x53,0x9a,0x4d,0x82,0x6f,0xd6,0x22,0x13,0x98,0xfe,0xd0,0x20, -0x2b,0x3a,0x2f,0x12,0x4e,0xcf,0x86,0xb9,0xd6,0x0b,0x0d,0x12, -0x7b,0x5d,0x98,0x85,0x78,0x3f,0x6a,0xfa,0x2c,0xdb,0x1e,0x9a, -0x72,0x4d,0x77,0xbf,0xee,0x0e,0x9e,0xfc,0x4d,0xbe,0x5b,0xd8, -0xcd,0xba,0xb5,0x9e,0x93,0x2a,0x43,0x22,0x10,0x04,0x41,0x25, -0xb9,0xb7,0x5f,0x68,0x01,0x88,0xbf,0x04,0xfd,0x17,0x5b,0xf5, -0xba,0xb5,0x88,0xe5,0xed,0xef,0x84,0xb8,0x1a,0x07,0x67,0x09, -0xdb,0x30,0x26,0xd5,0x7f,0xac,0x22,0x05,0x64,0x65,0x5d,0xff, -0x13,0x2a,0x9b,0xe6,0x55,0xf6,0x72,0xcd,0x27,0x7c,0x39,0x85, -0x7d,0xe4,0x1e,0x70,0xa8,0x67,0x75,0xe4,0x90,0xf3,0x25,0x99, -0x84,0xce,0x47,0xe8,0xbb,0x75,0x36,0x69,0xc1,0x22,0x3e,0xd9, -0x56,0x0f,0xc4,0x48,0x89,0x91,0x32,0x4e,0x5b,0x19,0x66,0x10, -0x33,0xf7,0x9b,0x67,0xd3,0xc6,0x66,0x22,0x29,0xdb,0x98,0xb2, -0x1d,0xa4,0xa0,0x83,0x9f,0x7c,0x92,0x3a,0x6f,0xc6,0x64,0xf6, -0x43,0xe0,0x68,0xab,0x1f,0xf7,0x36,0x07,0xa5,0x65,0x92,0x03, -0xef,0x8c,0x3e,0x40,0x91,0x0f,0x52,0x64,0xdb,0xf1,0x17,0xe6, -0x6e,0x9c,0x59,0x72,0x6f,0x03,0xb3,0x48,0x70,0x5b,0x34,0x18, -0xcd,0xf3,0x1a,0xd5,0xdb,0x66,0xdc,0x59,0xe6,0xa0,0x5c,0xee, -0xea,0x85,0x53,0xf0,0x65,0xb7,0xbe,0x65,0x90,0x27,0x52,0x2c, -0xa5,0x82,0x44,0x18,0xb1,0xed,0xae,0xf9,0x84,0xf8,0x74,0xd8, -0xf1,0x72,0x21,0xc0,0xa5,0x13,0xca,0xcc,0x19,0x82,0xb1,0x2a, -0xf4,0xad,0x80,0x92,0xa5,0xc9,0x3a,0x8c,0x8b,0xc4,0x5a,0x5f, -0x73,0xdd,0x13,0x54,0xa1,0x20,0x8f,0xa3,0x14,0xd7,0x05,0xe9, -0x60,0x4d,0x9e,0xff,0x5a,0x91,0x15,0xa8,0xc1,0x58,0xd1,0xcf, -0x9c,0xf3,0xf3,0xfb,0x3f,0xe6,0x5f,0xb2,0x8a,0xc7,0xf8,0x61, -0x9b,0x70,0xe1,0x22,0x77,0xab,0xbe,0x10,0x08,0x03,0x85,0x76, -0xc3,0x04,0xbb,0x55,0xb6,0x98,0xfd,0x9b,0x20,0x5f,0x88,0x9b, -0xa3,0xef,0x39,0x6c,0xa6,0x8e,0x9f,0xbe,0x9c,0x6c,0x13,0x52, -0xf7,0x88,0x27,0x2b,0x64,0x71,0x65,0x41,0xc4,0x58,0x55,0x3a, -0x9b,0x8e,0x7a,0x1f,0xf3,0xf7,0xe4,0xe1,0x0c,0x51,0x8b,0x36, -0xd0,0xb1,0x89,0xeb,0x04,0x25,0x08,0x87,0x8d,0x01,0x49,0x26, -0xda,0xd1,0x92,0xb2,0x37,0x7c,0x80,0xf4,0x20,0x1b,0x89,0x30, -0x9d,0x77,0x10,0x44,0x2e,0xd0,0x39,0xa1,0x5d,0x0b,0xb2,0x38, -0xa1,0xc7,0x21,0xf3,0x86,0xf4,0xf0,0xd6,0x72,0x3c,0xbc,0xb3, -0xbe,0x05,0x0c,0x19,0x09,0x08,0x32,0xe3,0xfe,0x44,0xd5,0x5e, -0xaf,0xaf,0x05,0xc9,0xf2,0x2a,0x6b,0xb5,0x79,0x45,0xa4,0x42, -0x8f,0xdd,0x8a,0xe2,0x0f,0x18,0x43,0x44,0xcb,0x16,0x71,0x61, -0xe4,0xb2,0x68,0x3e,0xc2,0xdd,0x62,0x1a,0x35,0x02,0x68,0xd0, -0xbb,0x8e,0xab,0x40,0x3b,0xd6,0x5a,0xd8,0xf2,0xf9,0x70,0x98, -0x65,0x76,0xf6,0x94,0xbd,0x3f,0x14,0x08,0x49,0x6c,0x3a,0xe3, -0x34,0x72,0x6c,0x29,0xad,0x9a,0x64,0xee,0x79,0x97,0x75,0xe5, -0x5f,0xdc,0x04,0x48,0x21,0x17,0xd1,0x0e,0x44,0x4c,0x77,0x10, -0xab,0xa4,0x4c,0x22,0xac,0xc0,0x2f,0x11,0xbb,0x3e,0x93,0xc1, -0x79,0xd4,0x8e,0x27,0xf0,0x5a,0x01,0x16,0xf2,0xdb,0x4b,0xfd, -0xc2,0x91,0x1d,0x69,0x15,0x77,0x2b,0xd3,0x8a,0xed,0xca,0xb4, -0xb8,0x5f,0x31,0x43,0xe7,0xf5,0xa6,0x88,0x76,0x41,0xd2,0xa6, -0xa1,0x02,0x41,0xdb,0x5b,0xcd,0x09,0x9c,0xc3,0xa0,0x93,0xa4, -0x6a,0xb7,0x72,0xea,0xe8,0xca,0x97,0x11,0x61,0x7d,0xb1,0x40, -0x17,0xb6,0xa4,0x3a,0x29,0x97,0x91,0xe8,0x69,0x89,0x9d,0xbc, -0xd0,0xfe,0xee,0xf8,0xcb,0xcd,0xf9,0x78,0x98,0xc5,0xeb,0xca, -0x35,0x98,0x48,0x4c,0x22,0xd6,0xc4,0x11,0x3e,0xd3,0xd5,0x3b, -0x92,0xa6,0x2a,0x89,0x56,0x49,0x40,0xd2,0x10,0x30,0x41,0x2e, -0x17,0x4d,0x77,0x7f,0xa5,0xbb,0xa3,0x2c,0xba,0xbf,0x8f,0xc8, -0xf4,0xb8,0x85,0x05,0x54,0xbf,0x5c,0x46,0xe8,0x32,0x40,0x0a, -0x76,0x06,0x50,0x6e,0x86,0xdb,0x4b,0x6f,0x40,0xcc,0xaa,0x00, -0xda,0x33,0x31,0x77,0x01,0xe4,0xbc,0xdb,0x9c,0xdd,0x9f,0xdd, -0x6f,0x5e,0x19,0x05,0x38,0xf1,0x89,0xcb,0x25,0x24,0xb9,0x0a, -0x60,0xf8,0x0b,0xa6,0x67,0xb2,0xcd,0x9f,0xd9,0x18,0xcc,0xe9, -0xce,0xc6,0x5f,0xbb,0x9b,0xce,0xae,0x37,0x6f,0x23,0xb5,0x95, -0xea,0xb2,0x24,0x12,0xba,0x90,0xbd,0x80,0xc0,0x90,0x0c,0xb7, -0x90,0x25,0x17,0x8b,0xc1,0xe1,0xd1,0xde,0xab,0x37,0x7f,0x96, -0x0d,0x2b,0x6a,0xc9,0x25,0x81,0xe1,0x6b,0x7b,0x1a,0x58,0x70, -0x82,0xa9,0x5e,0xf0,0xa3,0xc5,0xeb,0xdd,0x77,0x27,0x47,0xf4, -0xa0,0x13,0xed,0x3a,0x9f,0x1d,0x37,0x92,0x18,0x56,0xe9,0xde, -0x14,0xad,0x57,0x9f,0x7d,0x42,0x53,0xd4,0xd6,0xae,0xb3,0xb5, -0x27,0x5b,0x70,0xc3,0x73,0xe1,0x9d,0xf5,0x26,0xf7,0xde,0x02, -0x1d,0x9d,0x90,0xc8,0x8d,0xac,0xaa,0x91,0x37,0x97,0xa9,0x8a, -0x91,0x6d,0xa1,0x02,0xf3,0x80,0xfd,0x55,0xeb,0x21,0xa3,0xca, -0xfc,0xa9,0x78,0x58,0x2a,0x19,0x49,0xc8,0xdd,0x3b,0xc4,0x6a, -0x8f,0xc7,0x62,0xdf,0x49,0xf9,0x1c,0x66,0x52,0xb9,0x1c,0xf0, -0x7b,0x63,0xf7,0xc2,0xdd,0x03,0x22,0xbf,0xff,0xa4,0xde,0xe1, -0xe3,0x84,0x95,0xba,0xee,0x4c,0x34,0x48,0xf7,0xd6,0x55,0xc1, -0x76,0xef,0x65,0x8d,0x07,0xa2,0x14,0xd7,0x4b,0x5a,0x3a,0xba, -0xa3,0x3b,0xba,0x03,0x55,0x02,0xf9,0x36,0x9f,0xa1,0x0d,0x09, -0xfd,0x40,0x2a,0x68,0xdc,0x1a,0xea,0x22,0x48,0x2e,0xeb,0x0d, -0xa4,0x90,0x6e,0x9c,0x41,0xd0,0x57,0x41,0xf7,0x15,0xb2,0x79, -0x2f,0xac,0x81,0xbd,0x69,0xef,0x65,0xba,0xdc,0xf3,0xaa,0xf9, -0x74,0x54,0xb5,0xc6,0xf1,0x30,0x5b,0xc7,0x44,0x37,0xde,0x91, -0x7c,0xd0,0x80,0x92,0xbf,0x0b,0xf5,0x34,0x1f,0x0f,0x62,0xf8, -0xe5,0xe0,0x2e,0x39,0x63,0xae,0x41,0xad,0xe3,0xc9,0x4c,0x91, -0xb9,0xdc,0xbc,0xf6,0xa6,0xfc,0xda,0x74,0xad,0xf6,0x6e,0x5c, -0x63,0xb8,0x35,0xae,0xd5,0xfa,0xc6,0xed,0xd7,0xee,0x06,0xbd, -0xda,0xdf,0xb8,0xea,0xbf,0x35,0x23,0xdd,0xe0,0xc5,0x42,0x0d, -0x35,0x11,0xbe,0x6a,0x4a,0x3b,0x7c,0x8f,0xe6,0x53,0xba,0x59, -0xc1,0xdd,0xb4,0x23,0xbb,0x65,0x60,0xa5,0xde,0xee,0x28,0x1f, -0xf3,0x0c,0xe3,0x88,0xba,0x40,0xec,0x81,0x93,0x31,0xe7,0x99, -0x34,0xb4,0x84,0xbf,0x57,0x58,0xb4,0x56,0xd5,0x45,0xbe,0x67, -0xfc,0x04,0x83,0x60,0x79,0x4e,0x69,0x87,0x9a,0xe2,0x19,0x94, -0xfc,0xc8,0x38,0x7f,0xa0,0x49,0xd0,0x3e,0x7a,0x5d,0xfa,0x9a, -0x6f,0x8d,0x62,0xb9,0x3d,0xe2,0x59,0x28,0x32,0xa3,0x19,0x18, -0xca,0xd3,0x57,0xfe,0x0c,0x24,0x5f,0x27,0x76,0x56,0xd1,0x39, -0xc2,0x7a,0xd3,0x3a,0x1b,0x45,0x89,0x76,0x65,0x39,0x42,0xf5, -0x00,0xbc,0x2b,0x54,0x1e,0xb1,0x52,0xc8,0x02,0x55,0x28,0x5f, -0xb7,0xa4,0x24,0x40,0x99,0x9c,0x19,0xad,0x41,0xc2,0x76,0x0d, -0xb2,0x52,0x1a,0x6b,0xef,0x92,0xdc,0x14,0x33,0x7e,0xb4,0xf4, -0x75,0xd4,0x77,0x23,0x8b,0x4c,0x8d,0xf4,0x7e,0x52,0x15,0x9b, -0x91,0x57,0xcf,0x44,0x1a,0x95,0x48,0x92,0xef,0x10,0x69,0xca, -0xef,0x79,0x1a,0x06,0x3d,0x22,0x01,0xe0,0xa3,0x99,0xe3,0xa6, -0x36,0x43,0x47,0x3e,0xbc,0xe5,0xc2,0x92,0x85,0xdf,0x60,0xdd, -0xf0,0xb0,0xd0,0xe0,0xbd,0x07,0x5c,0x39,0x1e,0x09,0xf1,0xe6, -0x8e,0xcf,0x76,0xea,0x7b,0x8d,0xa0,0x82,0x94,0x41,0x16,0x04, -0x7d,0x0e,0x75,0x80,0x46,0xda,0x33,0xe8,0x0d,0x00,0x6d,0x71, -0xc4,0x78,0x6f,0xe0,0xae,0x09,0x2f,0x47,0x53,0x65,0xb2,0xfb, -0x95,0x17,0xb0,0xa0,0x41,0xf0,0x27,0xc8,0x9c,0xa4,0x6e,0xb8, -0x2f,0xb5,0xfb,0x4a,0x94,0xa1,0xc8,0x2b,0x07,0xc8,0x20,0x41, -0x7e,0x71,0x46,0xa3,0x56,0x3e,0xe0,0xd7,0xa2,0xae,0x82,0xbe, -0x36,0xe6,0xbe,0x6e,0x0e,0xbe,0xc8,0xb6,0xb7,0x60,0x1b,0xfd, -0x7c,0x6b,0xeb,0x05,0x47,0xa8,0x29,0xf2,0x7a,0x36,0xbb,0x2d, -0xac,0x96,0x7f,0x38,0x3e,0x78,0xf7,0xfe,0xf8,0x64,0xe7,0xe8, -0x44,0x2c,0x37,0x9c,0x9d,0x2e,0xce,0xee,0x4f,0x7f,0x3e,0xbb, -0xef,0x26,0x9b,0x86,0x52,0xf7,0xde,0xbd,0xcc,0x36,0x4f,0xcf, -0x96,0x67,0xdd,0x2e,0xdb,0x70,0x38,0x3c,0x3a,0x38,0x41,0x6b, -0x54,0x90,0xc6,0x4b,0x34,0x16,0x4d,0xce,0xba,0x67,0xcb,0x67, -0xa6,0x73,0x36,0xda,0x34,0xbb,0x07,0xef,0x4e,0xf0,0xdc,0xf9, -0xe4,0x2f,0x87,0x7b,0xef,0x77,0x0e,0x0f,0xdf,0xbe,0xd9,0xdd, -0xa1,0xec,0x08,0x2f,0xbb,0xa7,0x77,0xb1,0x30,0x8b,0x36,0x50, -0xd9,0x04,0x68,0x18,0x27,0xd8,0x80,0x95,0x19,0x37,0x3f,0x4c, -0xc7,0xa3,0x36,0x6a,0x95,0xc1,0xb6,0x3e,0x9b,0xcf,0x2e,0x37, -0x7e,0x1b,0xa1,0x0b,0x13,0x7a,0x58,0x2d,0x42,0xaf,0x0b,0xdd, -0xbb,0x41,0x39,0x92,0xfb,0xdb,0xf4,0xd4,0x3f,0x4c,0xc6,0x5e, -0x2b,0x39,0x5c,0xa0,0x58,0x54,0x90,0xc4,0xbe,0xc6,0x3f,0x4a, -0xd8,0x28,0x34,0x0a,0x05,0x0e,0xe3,0xbb,0x87,0x0f,0xcc,0xa5, -0xbc,0xed,0x97,0x20,0x92,0x81,0x5e,0x4e,0xc6,0x37,0x7f,0x98, -0x5a,0x04,0x12,0x1e,0xbe,0x65,0xd7,0x28,0x5c,0xc9,0x89,0x96, -0x46,0x55,0xe1,0x69,0x1f,0xbd,0x01,0xbc,0x35,0x20,0x26,0xb2, -0xc3,0x94,0x74,0x66,0x63,0x06,0x99,0xa4,0x7d,0x00,0x65,0x67, -0x09,0xba,0x30,0xb8,0x19,0x8f,0xd2,0xfb,0x9d,0x0b,0x64,0x7d, -0xe5,0x8e,0x34,0x35,0x3c,0xf5,0xde,0x84,0x06,0x0e,0xe0,0xfb, -0xbb,0xcd,0xef,0xa0,0x2f,0x51,0x1f,0x3a,0x7d,0x70,0x80,0x0c, -0x7a,0x20,0x7a,0x24,0x07,0x2e,0xed,0x0f,0xe7,0x59,0x9a,0xcf, -0xd3,0xc9,0x25,0x3b,0x84,0xa3,0xcd,0x6e,0xf4,0xe7,0xe3,0xa3, -0x57,0x1b,0x27,0x07,0x7f,0xdc,0x7b,0x17,0x51,0x1a,0x4f,0x50, -0x49,0xdb,0x50,0xa9,0x4b,0x3e,0x9b,0xc7,0x36,0x8d,0x27,0xec, -0x0b,0x00,0xbd,0x09,0xb2,0x1d,0x7d,0x9f,0x42,0x07,0x34,0xf6, -0xe6,0xfe,0xcd,0xca,0x12,0x15,0x39,0xc8,0x27,0x45,0xb0,0x28, -0xa0,0x2a,0xd9,0x8f,0xbd,0x8b,0x8f,0x64,0x8a,0x30,0x5a,0x3f, -0x67,0x77,0x62,0x64,0xcd,0x3e,0x70,0xda,0x52,0xb0,0x67,0xff, -0x0b,0x1b,0xcd,0xf7,0x02,0x9c,0xb2,0x66,0xef,0x97,0x18,0x05, -0xdc,0x58,0xd0,0x26,0x00,0x1c,0x58,0xa5,0xff,0xc5,0x28,0xa9, -0xcf,0x01,0x54,0xa7,0x57,0x04,0x90,0xec,0xd7,0x90,0xfb,0x3a, -0x56,0x52,0x28,0x73,0x2a,0x3b,0x29,0x9c,0x76,0x83,0xf3,0xab, -0x7e,0xab,0x74,0x09,0x9d,0xea,0x03,0xf1,0x9f,0x34,0xe4,0x73, -0x36,0x91,0x2e,0x4a,0x5d,0xd6,0x66,0xa8,0x20,0xd1,0x2c,0xd5, -0xa8,0x9c,0xe3,0x7b,0x6e,0xe5,0x4a,0x0b,0xdb,0xea,0x60,0x44, -0xba,0xfe,0x0b,0x0c,0x10,0x91,0x3c,0x06,0x93,0x82,0x66,0xb0, -0xe5,0xe1,0x17,0xc5,0x46,0xa2,0xf9,0xbb,0xca,0x05,0x97,0x03, -0xc2,0x60,0xda,0x85,0xa5,0xd5,0x8f,0x09,0x47,0xbc,0x1a,0xc9, -0x5c,0x42,0xc9,0xfe,0x55,0x31,0x0d,0xa7,0x74,0x00,0x2e,0xb3, -0x29,0xb0,0x4b,0xc3,0x95,0x62,0x2d,0x0b,0x92,0xed,0xd2,0x72, -0xca,0xbf,0xdd,0x30,0xd5,0x9e,0xf5,0x17,0x32,0xe1,0x35,0x0d, -0x69,0xa5,0xe7,0x6a,0x3e,0x18,0xb7,0x32,0xbd,0x0c,0xa2,0x61, -0xc8,0x55,0xc8,0x15,0x99,0x66,0x96,0x25,0xba,0x91,0x71,0x39, -0xb5,0xaf,0x33,0x19,0x2e,0xdb,0x5f,0x6d,0x05,0xc0,0x67,0xf2, -0x91,0x4d,0x66,0x2f,0x2a,0x46,0xad,0x98,0x02,0xeb,0x86,0x2c, -0x4b,0x25,0x5d,0xd8,0xd9,0xaa,0x11,0xf1,0x25,0xc2,0x78,0x8f, -0x55,0x62,0x8d,0x65,0x48,0xf8,0x0d,0xcc,0x4d,0xe2,0x5b,0x74, -0x6e,0x1a,0x74,0x06,0x39,0x23,0xf3,0x00,0xef,0xab,0xfb,0x46, -0x89,0x40,0x41,0x69,0xd6,0x9f,0x0b,0x3a,0x0e,0x01,0x2a,0x4c, -0xd0,0xdd,0x98,0x2b,0x1b,0x64,0x4c,0xd0,0xf2,0x52,0x65,0x6d, -0xfe,0xd0,0x75,0x45,0x33,0xda,0xbe,0x82,0xd3,0x00,0x9f,0x6e, -0xa6,0xba,0x33,0x4c,0xf1,0x9e,0x83,0x6d,0xd1,0xa5,0xec,0x67, -0xa1,0xa7,0x83,0xc5,0x8e,0x17,0x10,0x37,0xe8,0xc5,0x14,0x53, -0x5e,0x17,0xab,0xf2,0x72,0xd2,0xd2,0x2e,0x27,0x59,0x30,0xe3, -0x42,0xee,0xd2,0xf6,0x3b,0x5d,0x08,0x9a,0x30,0xb1,0x40,0x57, -0x59,0x81,0x3b,0x30,0x89,0x64,0x73,0x34,0xfc,0x56,0x41,0x3a, -0x74,0xbe,0x82,0xcb,0x01,0x2b,0x4b,0xa8,0xb3,0x43,0x60,0x1d, -0xd0,0x2d,0x07,0xf4,0x60,0x3f,0x56,0x6a,0x54,0x1d,0xcb,0x43, -0xad,0xe3,0xdc,0x38,0x39,0x95,0xe4,0x70,0xc9,0x59,0x2c,0x5c, -0xb3,0xc3,0x84,0xae,0x12,0x30,0x5d,0xd5,0x6e,0xb6,0x4f,0x35, -0x34,0x3f,0x3e,0x05,0x68,0x7a,0x48,0x1d,0x0c,0x71,0x35,0x48, -0xc7,0x54,0xd6,0x9b,0xba,0x56,0x2a,0x26,0x26,0x66,0xfb,0x29, -0xec,0xb6,0x36,0xf3,0xe6,0x5f,0x90,0xef,0x66,0x21,0x17,0x96, -0x8c,0x5a,0x2e,0x2c,0x48,0xd6,0x15,0x9c,0x98,0x2a,0x57,0x6c, -0xd8,0xfb,0xfb,0x53,0xe0,0x12,0x7f,0x52,0x58,0x62,0x91,0x7c, -0x88,0x6d,0xf9,0x63,0x24,0xfa,0xa8,0x1b,0xb8,0x7f,0x8d,0xf4, -0xc4,0x90,0x92,0x64,0x66,0xb2,0x8a,0xc3,0x2d,0x43,0x6f,0x83, -0x52,0x3b,0xda,0x49,0xda,0x9d,0xe4,0xb4,0x93,0xc4,0x9d,0x33, -0x89,0x3b,0x3e,0x97,0xeb,0xea,0x72,0xbe,0x15,0x10,0xb2,0x95, -0x45,0xcc,0x14,0x48,0x17,0x3a,0x44,0x51,0xef,0x4b,0x2d,0x62, -0x37,0x67,0xd7,0xb9,0x58,0xbc,0xd7,0x13,0xa3,0x3c,0x8b,0x12, -0x60,0xa1,0xd7,0x20,0x4a,0x65,0xa7,0xc1,0x08,0x7b,0xb3,0x23, -0x5d,0x03,0x3b,0xed,0x9b,0xc1,0x14,0xc8,0xf8,0x97,0xe6,0x27, -0x84,0x2a,0x63,0x4e,0x4c,0xc8,0x1b,0xab,0x23,0x37,0x46,0x7d, -0x2d,0x90,0x28,0xc5,0x40,0x1f,0x9b,0xdc,0xeb,0x50,0x53,0x66, -0xdd,0x62,0x51,0x11,0xb9,0x87,0x3b,0xd5,0x04,0xdf,0x70,0x01, -0x7e,0x4e,0x47,0xb8,0x22,0xa3,0x59,0x59,0xd8,0x14,0xb2,0x73, -0x9b,0x8b,0xb5,0x71,0x6c,0x58,0x9d,0xd7,0xb6,0x2a,0x64,0x33, -0xab,0xcb,0x82,0x84,0xc0,0x85,0x03,0xbd,0x5b,0x1c,0x09,0x54, -0xe8,0xa5,0x14,0xab,0x0c,0xcb,0xf2,0x41,0x31,0xba,0x6d,0xbb, -0x5a,0x7e,0x65,0x14,0x09,0x80,0x2b,0xe2,0xee,0xbe,0x6c,0x26, -0xd1,0xf4,0xaf,0x74,0xcf,0x1f,0x00,0x2a,0xa9,0x86,0x26,0x78, -0x52,0xbe,0x42,0x1e,0x62,0x89,0x66,0x95,0x7c,0x44,0x1b,0x65, -0x86,0xbd,0xb4,0x1f,0x7c,0xb4,0xf0,0x38,0x1a,0xf4,0x3c,0xe3, -0xff,0x65,0x5c,0xd4,0x9b,0x1b,0x3a,0xcc,0x3e,0xbe,0x1e,0x4f, -0x66,0x6c,0x32,0x72,0x1a,0xd3,0x35,0x25,0x91,0xad,0x3b,0xfb, -0x30,0x85,0xcd,0x38,0x89,0xa3,0x72,0xcc,0xe9,0x92,0xd0,0x41, -0xbe,0x65,0x78,0xf6,0x15,0x07,0x89,0xad,0xc1,0x4a,0x42,0x36, -0x2b,0xee,0x79,0x25,0x48,0xe9,0xd4,0x09,0xca,0xa5,0xf0,0x0f, -0x35,0x58,0x82,0xd7,0x40,0x25,0xcc,0x50,0x95,0x85,0xb8,0xe3, -0x3f,0x82,0x21,0x75,0xdd,0x3f,0x80,0x26,0x01,0x48,0x69,0x87, -0x57,0x40,0x78,0x05,0xcf,0x51,0x72,0x87,0x17,0x66,0xfb,0xa8, -0x8f,0x5d,0x18,0x37,0xcb,0x9d,0xc4,0x4d,0x3b,0xbf,0x5c,0x01, -0xe6,0x57,0xda,0xfd,0x77,0xd8,0xc5,0x25,0xec,0xd9,0xa8,0x45, -0xa7,0x45,0x38,0x55,0x67,0x21,0x02,0xb4,0x9b,0xb0,0xcb,0x5b, -0xab,0x14,0x81,0x8b,0x2b,0x4e,0x37,0xba,0xb2,0x3b,0x64,0xe2, -0xf0,0xaa,0xd1,0x02,0x38,0x04,0x11,0xdc,0xb3,0xac,0xdf,0x5e, -0xf7,0x90,0x63,0xe8,0x38,0x3e,0x3c,0x4b,0xb4,0x26,0xd2,0x53, -0xc0,0x27,0xf7,0x1c,0xf6,0x2f,0x66,0x25,0x7d,0x0b,0xf9,0x54, -0xa9,0x0f,0xc8,0x69,0x35,0xb0,0x7c,0x7b,0xdf,0x98,0xaa,0x08, -0xe4,0x04,0x49,0xcc,0xbb,0x9b,0x42,0x6d,0x69,0x58,0x69,0xfa, -0xe0,0xe2,0x9a,0xf2,0xcf,0x32,0x68,0x0b,0xf6,0xdb,0x61,0x6e, -0x9d,0x63,0x5b,0xc7,0xa5,0xfd,0xcf,0x99,0x3d,0xa2,0xe3,0x47, -0x4a,0xb7,0xb9,0xf6,0x9f,0xed,0x26,0x63,0x1b,0x9f,0x78,0x66, -0x90,0xbd,0x5e,0xaf,0xcc,0xe7,0x1e,0x7f,0xf7,0x3f,0x9b,0x56, -0xe2,0xdd,0xa8,0xb2,0x1b,0x52,0x7c,0xe7,0x74,0x6b,0x6c,0x4b, -0x71,0xc9,0xa1,0xef,0xd8,0xcd,0xf0,0xcc,0xf5,0x82,0x44,0x20, -0xd9,0x66,0xe7,0xf3,0xc1,0xb0,0x8f,0x4f,0xa3,0xbc,0x48,0x65, -0xc5,0x87,0x5b,0xf2,0xac,0x47,0x8a,0x83,0xd5,0xf8,0x58,0x25, -0x71,0x12,0xfb,0x02,0x46,0x55,0xec,0x0a,0x53,0x8c,0x80,0x71, -0x93,0x5a,0x08,0x79,0x25,0x4d,0x51,0x18,0x17,0x13,0x95,0x2c, -0xd7,0xab,0xd1,0xef,0xf7,0x4e,0xd8,0x94,0x9b,0x97,0x18,0x51, -0x02,0xc0,0x2c,0xfe,0xf8,0x50,0x17,0xa4,0xab,0x03,0x17,0x4a, -0xfd,0x81,0x4a,0x58,0x5b,0x27,0x0c,0x5b,0xf9,0x98,0x1e,0x84, -0x25,0xdc,0xbd,0x89,0xf5,0x4c,0x4d,0xfd,0x2c,0x9e,0x64,0xf1, -0xb5,0x19,0x4e,0x13,0xe3,0x7d,0xf3,0xf9,0x3c,0x09,0x9d,0xf9, -0xfb,0x30,0xf5,0x4c,0xa2,0x5d,0xc9,0xfa,0xe8,0xa7,0x74,0x98, -0x2f,0xd2,0xb6,0x46,0x70,0x55,0x65,0x9d,0xc2,0x4c,0xf2,0x49, -0x78,0x0f,0xa3,0x42,0x5b,0xe8,0x99,0xfe,0x56,0x97,0x25,0x23, -0xed,0xe9,0xca,0xf2,0x66,0x7b,0x6b,0xcb,0xdc,0x2f,0xed,0x03, -0xa6,0x80,0xbf,0xc8,0xa0,0x57,0x0b,0x96,0x1e,0x39,0x21,0x68, -0x39,0xf5,0x08,0x45,0x7e,0xa4,0x42,0xc7,0x10,0x91,0xf7,0x29, -0xae,0xe8,0x04,0x59,0xf6,0x46,0x6f,0xaa,0x65,0x3d,0x1b,0x6d, -0xa7,0x33,0x3d,0x5a,0x2b,0x2f,0x67,0x8e,0xd2,0x09,0x71,0xa6, -0x6d,0xba,0x95,0x91,0x6f,0x69,0x02,0xa4,0xb6,0x45,0xef,0x7c, -0x56,0xe1,0xad,0xf4,0x98,0x34,0x7e,0x63,0x2e,0x53,0xf9,0x68, -0x97,0xcf,0x02,0xc4,0x82,0xf2,0x1d,0x1b,0xc2,0x74,0x32,0x21, -0xf1,0x6c,0x1e,0x3c,0xbe,0x67,0x5f,0x2c,0xac,0xfd,0x62,0xfa, -0xe9,0xfa,0x57,0x53,0x0c,0xd2,0xd7,0xa0,0xee,0x1b,0xef,0xc8, -0xec,0x71,0x26,0xe7,0x80,0x77,0x6c,0xf8,0xd2,0x4e,0xc8,0x7c, -0x44,0xae,0x2a,0x26,0x83,0xff,0x9c,0xe7,0x13,0x7e,0x75,0xdb, -0x88,0xb2,0xa8,0x51,0x88,0xbf,0x4b,0x58,0x31,0x14,0xfb,0xbf, -0x11,0x6f,0xb4,0xb2,0x0c,0x3e,0xfc,0x5d,0x42,0x27,0x4a,0x3a, -0xf0,0x27,0x8d,0xea,0x51,0xd2,0x60,0xe8,0xac,0x72,0x03,0x61, -0x7b,0x44,0xe1,0xe6,0x48,0x56,0x70,0x4f,0xcc,0x6e,0x21,0x13, -0x53,0x25,0xdb,0x56,0x38,0x80,0xad,0x3a,0xdf,0xab,0x14,0x81, -0x95,0xe7,0xdc,0x32,0x60,0x2f,0xe7,0xda,0xe3,0xdd,0x8a,0x92, -0x9d,0xf0,0x69,0x7f,0x45,0x8e,0xb4,0xe4,0x4d,0xb6,0x22,0x93, -0x7e,0xbc,0xfd,0xd0,0x99,0xe3,0x43,0xad,0x30,0xe2,0x98,0xd1, -0x9e,0xc2,0x21,0x0c,0x10,0x6b,0xff,0x0f,0x60,0xdf,0xae,0xec, -0x3e,0xbb,0xba,0x20,0x56,0x06,0x66,0xbc,0xc5,0x29,0xad,0x10, -0x38,0xdd,0xd9,0x84,0x45,0x3b,0xb6,0x5b,0x1c,0x97,0x27,0xf1, -0x87,0x89,0x24,0xe3,0x3f,0x19,0x8c,0x3a,0x00,0x14,0x40,0xa8, -0xbd,0x6c,0x2a,0x97,0x20,0x3c,0xf3,0xad,0x10,0x53,0x23,0xe8, -0x9d,0xc8,0x44,0xbc,0x03,0x85,0x0f,0x5c,0xbb,0xe1,0x07,0x0f, -0xc1,0x91,0x2c,0x1f,0x10,0x1f,0x23,0x3c,0x10,0x87,0xac,0xc0, -0xe0,0xdc,0x9b,0x62,0x77,0xc3,0x60,0x3f,0x38,0x3e,0xb8,0x6e, -0x7c,0xed,0xd9,0xdb,0xaa,0x9b,0x46,0x75,0x8a,0x6c,0x2f,0x38, -0xb5,0x0b,0x54,0x7f,0x40,0xec,0xce,0x84,0xe5,0xbe,0xda,0x65, -0xf2,0x1a,0x12,0xd4,0x00,0x55,0xa7,0x2f,0xf4,0xe7,0xd7,0x47, -0xee,0x54,0x99,0x25,0x00,0x0b,0xa7,0xd9,0x1b,0x5d,0xcd,0x87, -0xbd,0x09,0x39,0xac,0x3c,0x87,0x72,0x53,0xa3,0x75,0x57,0x5c, -0x36,0x6b,0x49,0x82,0xf5,0x48,0x2f,0xc6,0x43,0x77,0x29,0x02, -0xdc,0x80,0xf5,0x2e,0xba,0x45,0x51,0xfc,0x51,0x54,0x5e,0x12, -0x26,0xbe,0xe6,0x49,0xef,0x93,0xbd,0x05,0x76,0x56,0x42,0x0e, -0xa5,0x3e,0x25,0x0b,0xd3,0x88,0xa1,0x28,0x45,0xd2,0xf5,0x78, -0x94,0xdb,0x07,0x99,0x93,0xc1,0xed,0x2c,0x53,0x40,0x8a,0x9a, -0x37,0x9c,0x03,0x86,0x10,0x0b,0x89,0x9f,0x01,0xed,0x51,0x5e, -0x17,0x3d,0x1f,0xf7,0xbf,0x58,0xfd,0x52,0xd2,0xfb,0xe5,0xc2, -0x5c,0xb6,0x5e,0x27,0x79,0x3c,0x71,0x17,0xb5,0x9c,0xd8,0x24, -0xfd,0xda,0x88,0x2e,0x54,0x3e,0xf4,0xee,0x7a,0x52,0x9f,0x91, -0xd4,0xe9,0xe4,0x02,0xd9,0xa9,0x41,0xa5,0xb8,0x8e,0xc4,0x8d, -0x47,0x80,0x62,0xff,0x0b,0x0a,0x99,0x39,0xec,0x5d,0x47,0x57, -0xb9,0x46,0x68,0x13,0x5f,0x0b,0xe6,0xfd,0x05,0xea,0xcb,0x20, -0xd5,0x6e,0xf2,0x65,0x92,0x94,0xa5,0x92,0xc7,0x58,0x32,0x61, -0x84,0xac,0xeb,0x84,0x64,0x99,0x3a,0xf0,0x08,0x21,0x73,0x21, -0xde,0x52,0xaa,0xbc,0xa6,0xd4,0x66,0xad,0xeb,0xac,0xdb,0x2c, -0x05,0x96,0xc5,0xdb,0x56,0xb5,0x60,0x93,0x1d,0x1d,0x3b,0x9c, -0xee,0xee,0xd3,0xae,0xd3,0xc5,0x05,0x3a,0x58,0x99,0xd5,0xdd, -0x04,0x67,0x97,0xc9,0x1c,0x3b,0x29,0x1f,0xd5,0xea,0x71,0xe0, -0x5f,0x52,0xff,0xbb,0x4f,0x10,0x65,0x3f,0x5f,0x4f,0xea,0x75, -0xf8,0xd3,0xec,0x9d,0xa3,0xd7,0x98,0x82,0x81,0x08,0xec,0x37, -0x0b,0xcc,0xe1,0xf6,0xc8,0x4e,0x8a,0x96,0x7a,0x21,0xbc,0x0c, -0xc9,0x4c,0x34,0x5b,0x8e,0xf7,0x8e,0xfe,0xb4,0x77,0xf4,0x7e, -0x7f,0xe7,0x64,0xf7,0x35,0x3e,0x27,0x8b,0xa2,0x12,0x9d,0x76, -0x13,0x34,0xe7,0x2e,0x8d,0x78,0x03,0x4b,0x7b,0x40,0xec,0x30, -0xcd,0x46,0x17,0xf9,0x30,0x76,0xe9,0x89,0x6f,0x54,0x06,0x4d, -0x60,0x53,0xa6,0xd2,0xe2,0x08,0x9d,0xd2,0x83,0x68,0x6b,0x31, -0xe9,0x58,0x74,0x3b,0x20,0x7c,0xa5,0xdf,0x6f,0x7d,0x6f,0xf7, -0x26,0x92,0xbf,0xb5,0xbd,0xfd,0x9c,0x34,0xc0,0x20,0x00,0x59, -0x5c,0xb2,0x6d,0xf5,0x23,0xfb,0x47,0xcf,0x21,0xd6,0xd7,0x6d, -0xc7,0x1d,0xcc,0x67,0xa8,0xab,0xa1,0xb8,0x6b,0x3c,0x1a,0x8f, -0x6f,0x79,0xb1,0x67,0x70,0xb4,0x13,0xf0,0x25,0x07,0xa3,0x8b, -0x72,0x21,0x7a,0x91,0x10,0x93,0x54,0x81,0x33,0x00,0xb6,0x7f, -0xb6,0x00,0x84,0x20,0x5e,0x98,0xb0,0x3e,0x31,0x94,0xa3,0x5f, -0xd1,0xd7,0x92,0x16,0xbc,0xe9,0x67,0xd1,0xfb,0xa8,0xe1,0xc6, -0x11,0xef,0x1f,0xe6,0xb8,0x5c,0x35,0x1a,0xe8,0xb8,0x4c,0xd6, -0xc8,0xe7,0xbf,0x49,0xda,0x2e,0xc7,0xa9,0x2f,0xab,0x36,0xf6, -0x7c,0xf3,0x5b,0x99,0xa9,0xe9,0xee,0x7d,0x59,0xaf,0xcf,0x8f, -0x8f,0xe6,0x3b,0x9e,0xfd,0xd1,0x3d,0xef,0xee,0xce,0xdb,0xb7, -0x3f,0xee,0xec,0xfe,0x11,0x58,0x61,0x89,0xa1,0x36,0xa3,0x86, -0x07,0xaf,0xd4,0x35,0x1e,0xaa,0xbf,0xb3,0x92,0x72,0x51,0xee, -0x5e,0x5d,0x2e,0x49,0x1f,0x21,0xf9,0xc5,0x62,0x63,0xdb,0x3d, -0x5d,0xa8,0x84,0xb3,0x54,0xba,0xd3,0x44,0x91,0xf9,0xa7,0x1a, -0x30,0xed,0x36,0x4e,0x30,0xd8,0xae,0x07,0x13,0x1e,0x15,0xa5, -0x1e,0x39,0x1c,0xf6,0xea,0xe3,0x3c,0x47,0xa7,0xb9,0x9d,0xda, -0x2c,0xce,0x2b,0x95,0xe9,0x25,0x4d,0xe5,0x47,0x58,0x22,0x50, -0xdb,0xf7,0x19,0x4e,0x15,0xcd,0xfe,0x02,0x71,0xc9,0xde,0x57, -0x61,0x1e,0xf4,0xc4,0x36,0x1c,0x1e,0x85,0x29,0x40,0x70,0xd3, -0x01,0xf6,0x92,0xcd,0x79,0x1a,0x91,0xc0,0xba,0x21,0xfa,0x82, -0xe2,0xbb,0x04,0x4f,0xb2,0xdf,0xe2,0x68,0xf6,0xae,0x72,0x15, -0x45,0x0a,0x09,0x64,0xda,0x7f,0x00,0x35,0xc2,0xd7,0xdb,0xde, -0x74,0xb6,0xb1,0x3f,0xee,0x93,0xc7,0x3e,0x08,0x1f,0xe2,0xab, -0x91,0x5e,0x84,0x02,0x6d,0x50,0x2d,0x08,0xf5,0x45,0x14,0x81, -0x7f,0x38,0x6b,0x3f,0x1a,0xa3,0xe2,0x25,0xa4,0x7a,0x72,0x60, -0x9b,0x15,0xb6,0xc9,0xe6,0x6b,0x7b,0x47,0x31,0x41,0x4d,0x0d, -0xb9,0x71,0x61,0x23,0xa0,0x6c,0xb1,0x88,0xb4,0x87,0x96,0x62, -0x9e,0xf3,0x41,0x92,0xa1,0x71,0x63,0xde,0xc1,0x1d,0xef,0x19, -0x77,0x47,0x47,0xa4,0x41,0x2a,0xbe,0xff,0x2d,0xa0,0x01,0x52, -0x5b,0x71,0x19,0x00,0x5c,0xb1,0x54,0xf1,0xa4,0x1e,0x29,0x4b, -0xd7,0x23,0xd9,0x74,0x54,0x16,0x6e,0xed,0x98,0xb8,0x40,0xd4, -0xc0,0x55,0x08,0xdf,0x1d,0x90,0x83,0x3a,0x61,0xb4,0x2f,0xb6, -0x12,0x3a,0x3b,0xb6,0x6c,0x37,0x0b,0xb8,0x72,0x1c,0xae,0x39, -0x76,0xc5,0x4a,0xd8,0xbe,0x9d,0x84,0xea,0x75,0xf9,0xa0,0xbd, -0x79,0x18,0x2a,0x00,0xd0,0xaf,0xea,0xd6,0xd5,0xcb,0xd7,0x82, -0x5e,0x8d,0xd6,0x18,0x46,0xed,0x5d,0xa5,0xf9,0x8b,0x9a,0xbb, -0xac,0x7d,0xa0,0x33,0x15,0x1f,0xf6,0xe9,0x47,0x1b,0x9d,0x58, -0xe7,0xe4,0xb9,0xc7,0x6e,0xfe,0x55,0xbc,0xa8,0x95,0xfa,0x7a, -0x1e,0x86,0xe8,0xf3,0x69,0x78,0x2e,0xb6,0xa0,0xa4,0xea,0x9c, -0xfb,0x17,0x55,0xc3,0x9d,0x27,0x7f,0x2f,0xcd,0x8a,0xcf,0xfd, -0xb2,0x87,0xfe,0xe9,0x85,0x96,0x05,0xca,0x0f,0xca,0xd1,0x76, -0xcc,0xeb,0xde,0x5d,0xee,0x3d,0x6b,0x98,0x59,0xf0,0x7e,0xd3, -0xbf,0x99,0xa1,0x76,0x93,0x93,0x4a,0xec,0x58,0xfe,0xb8,0x1c, -0xa1,0xa6,0x29,0x6f,0xb2,0xb2,0x2d,0x63,0xb7,0xf3,0x46,0xde, -0x60,0x21,0x00,0xfb,0x0a,0xf1,0xba,0x37,0x0d,0x1e,0xb4,0x66, -0x6b,0x2d,0xc3,0x86,0x31,0x71,0xb3,0x2a,0x8f,0x91,0xd8,0xa9, -0x67,0x42,0x4e,0x32,0x7d,0x7d,0x0c,0xc7,0xee,0x9a,0x55,0xff, -0xcb,0xee,0x4e,0x0c,0xb7,0xc5,0x16,0xad,0x30,0xbf,0xeb,0x5f, -0xe3,0x21,0x36,0x14,0x90,0xb7,0xec,0x52,0x30,0xe9,0xf0,0xae, -0x6c,0x2d,0xf3,0xd9,0xe8,0x95,0x9f,0xdd,0xf2,0x13,0x50,0xd6, -0xe2,0xe3,0x9d,0x23,0x76,0x80,0xcf,0x1b,0x9e,0x0f,0x5c,0x8e, -0xac,0x03,0x0c,0xe8,0x9f,0xac,0x50,0xf4,0x21,0x3c,0x5c,0xdf, -0xa2,0x1e,0xd3,0xe5,0x08,0x9f,0xe9,0x65,0xbe,0x8b,0x6d,0x62, -0xc3,0x35,0xea,0xed,0xaa,0xde,0xdd,0x4a,0x52,0xdb,0x20,0xee, -0xdc,0x47,0x1b,0x43,0xda,0xb7,0x58,0x8b,0x75,0xb2,0x88,0x96, -0xc8,0x65,0x24,0xb9,0xa8,0x24,0x00,0xe7,0x55,0xa0,0x50,0x67, -0xcb,0xbe,0xb9,0x4b,0x0a,0xd4,0x63,0xeb,0x94,0xe1,0x6d,0xd9, -0x77,0xcf,0x8a,0x0e,0xfd,0xdb,0x67,0xa6,0x06,0x5c,0x0b,0x50, -0xaa,0xae,0x91,0x47,0x3c,0xf5,0x66,0x1d,0xf0,0xa4,0xa7,0xdf, -0x67,0xa3,0x63,0xb2,0xca,0x57,0x93,0x3a,0xe6,0xbd,0x61,0x6d, -0x6f,0x7a,0xd1,0xbb,0xc5,0x17,0xe5,0xf6,0x09,0xf4,0x54,0x17, -0xc4,0x27,0xcf,0xb3,0xeb,0x1e,0x29,0xf0,0xa2,0xe1,0x09,0x74, -0x1f,0x58,0xb3,0x6f,0xce,0x6b,0x5e,0x09,0x77,0x5a,0xc3,0xdd, -0x75,0xad,0x57,0x93,0x46,0xf0,0xbc,0xc5,0x97,0xe8,0xd6,0xd3, -0x56,0xb3,0x56,0x3b,0xce,0xf3,0x1a,0xee,0x4c,0xd3,0xcd,0x4d, -0xd8,0xd8,0x4d,0xed,0x86,0xef,0xc3,0xb4,0x39,0x9e,0x5c,0x6d, -0x02,0x16,0x9b,0xa3,0xab,0x26,0xcf,0x4f,0x9a,0x3f,0xb2,0xa5, -0x59,0x2b,0xcf,0xb4,0xc5,0xa2,0x38,0x6a,0x9d,0x98,0x31,0xb4, -0x6f,0x18,0x65,0xea,0x00,0x31,0x85,0x66,0xb2,0x70,0x5a,0xd2, -0x43,0x12,0x99,0x9a,0xd8,0xc9,0xd6,0x68,0x62,0xf1,0x79,0x6e, -0x64,0x8b,0x82,0x38,0xc8,0x8a,0xd0,0x34,0x78,0x3c,0xa2,0xe8, -0x86,0x9e,0x15,0x51,0x39,0xc2,0x81,0x27,0x5a,0xce,0xc2,0xc1, -0xec,0x14,0x6c,0x7e,0xc5,0x61,0x32,0x15,0x49,0x52,0xca,0x44, -0x1d,0x77,0xc0,0x35,0xb1,0xe2,0x4e,0x46,0xf0,0x17,0x0b,0x82, -0xab,0x1e,0xeb,0x50,0x38,0x8a,0xd0,0x46,0x1e,0x52,0xa2,0xd7, -0xd6,0xbe,0x15,0xb7,0x76,0x84,0x07,0x9f,0xc1,0x11,0x34,0x5a, -0x60,0xb1,0x93,0xf0,0xa1,0x31,0xc6,0xf8,0x47,0x10,0xd4,0x75, -0x7c,0x80,0x16,0xb9,0xb7,0x35,0x13,0x59,0xed,0x41,0xee,0x02, -0xba,0xca,0xaa,0x48,0x8f,0xa2,0x26,0xac,0xee,0xfd,0x6c,0xa6, -0xcd,0x24,0x08,0xc1,0x91,0xf5,0x2c,0x6a,0x23,0xe4,0xf3,0x82, -0x71,0x92,0xb4,0xcb,0xef,0x76,0xb8,0x1e,0x5c,0x99,0x65,0xf6, -0x52,0x39,0xf8,0x66,0xe6,0x48,0x7f,0xf1,0xfd,0x8e,0xb5,0xec, -0x50,0x58,0xb9,0x78,0x3e,0xeb,0x77,0x27,0x96,0x81,0x16,0xa6, -0xbc,0x7f,0x87,0x62,0xef,0x73,0xed,0x65,0xda,0x8a,0xb7,0x30, -0xc1,0xce,0xdc,0xa9,0xaf,0xfa,0xa5,0xac,0xfa,0x75,0x4c,0x45, -0x29,0xb5,0x60,0xe9,0x32,0xc1,0x51,0xcd,0x5b,0xd4,0x5e,0xcf, -0x2b,0x4f,0x69,0x4a,0x00,0xef,0x07,0xfd,0x34,0xca,0x47,0x1b, -0x73,0x90,0xfe,0xde,0xfd,0xb4,0xff,0x23,0xec,0x08,0x5f,0x1d, -0x1c,0xc1,0xa6,0xf0,0x38,0xbd,0x7f,0xb9,0xb7,0xfb,0x66,0x7f, -0xe7,0xed,0xfb,0xe3,0xbd,0xc3,0x34,0x6a,0x46,0xe6,0xf7,0x47, -0x07,0x3f,0x1d,0x72,0xc8,0x44,0xe6,0x70,0xe7,0xe4,0x64,0xef, -0xe8,0xdd,0x71,0x7a,0x7a,0x7f,0x33,0x18,0xc1,0x54,0x4a,0x5b, -0x06,0x3e,0x5e,0x4d,0x7a,0x17,0xe9,0x16,0xda,0x43,0xa5,0xaf, -0xe7,0xb8,0x91,0x3e,0x9c,0xe4,0x29,0x88,0x87,0xf0,0x75,0x3c, -0xbf,0xc4,0xaf,0x51,0x7e,0x45,0x71,0x1b,0xf4,0x29,0x91,0x57, -0xc7,0xf8,0x8a,0xf7,0xb9,0x19,0xca,0x07,0x5e,0x41,0x16,0x01, -0x6f,0x3b,0xc0,0xdb,0x0e,0xf0,0xff,0xfe,0xbf,0xab,0x40,0xc7, -0x18,0x6d,0x81,0x27,0x15,0xd0,0xbb,0x66,0xf7,0xa7,0xa3,0xa3, -0xbd,0x77,0xbb,0x7f,0x79,0x7f,0xfc,0x97,0x7d,0x34,0xf1,0xb8, -0x34,0x2f,0x77,0x4e,0xf6,0x4e,0xde,0xec,0xef,0xf9,0x3e,0xd8, -0x3f,0x78,0x77,0xf2,0x3a,0x8d,0xfe,0xd0,0x1b,0xcd,0x7b,0x93, -0x2f,0xe6,0x55,0x7e,0x3e,0xa1,0x8f,0xfd,0xde,0xe4,0xe2,0xda, -0xec,0xdc,0x4e,0x06,0x43,0xf8,0xfe,0x62,0xfe,0x30,0x1f,0xe5, -0xf0,0x67,0xf8,0xc5,0xec,0xcc,0xaf,0x60,0x66,0x9a,0x63,0xa0, -0xcc,0x1c,0xfd,0x87,0x98,0x83,0x8b,0xd9,0x18,0x7f,0xdf,0x8d, -0xef,0x38,0xe2,0x65,0x7e,0x41,0x1f,0x91,0x55,0x7d,0x37,0xc0, -0xc5,0x8f,0x5f,0x1f,0x1c,0x9d,0xf8,0xda,0xb0,0x26,0xac,0x04, -0xab,0xb0,0x15,0x20,0x7c,0x04,0x8f,0xb0,0x11,0x2a,0x42,0x44, -0x60,0x01,0x9c,0x97,0x3b,0x7f,0x49,0xa3,0x63,0x98,0xe3,0x50, -0x66,0x7f,0x4c,0x3f,0x27,0x20,0xc7,0xe1,0xef,0x7f,0xe5,0xfd, -0x11,0x7f,0x9d,0x5c,0xcf,0x27,0xf4,0xf1,0x6a,0x32,0xc0,0x9f, -0x63,0x90,0x87,0x27,0xf0,0x51,0xc6,0xc8,0x82,0x43,0x58,0x08, -0x08,0x81,0x60,0x71,0x2c,0x89,0xc5,0x82,0x12,0x3b,0xfb,0x87, -0xfb,0x40,0x0f,0xd1,0xce,0x3e,0x6e,0x1d,0xf6,0xa3,0xae,0xb9, -0xc9,0xfb,0x83,0xf9,0x4d,0x1a,0xed,0xef,0xef,0xd7,0xfa,0xa6, -0xf6,0xa5,0x76,0x9d,0xde,0xdc,0xa4,0xd3,0x69,0xad,0x07,0x39, -0xa6,0x78,0x3e,0x09,0xdc,0x67,0x7f,0xb3,0xbf,0xf9,0x85,0x93, -0x30,0xfe,0x12,0x38,0xd6,0x4b,0x9c,0xff,0xd1,0x1e,0xfc,0x67, -0x6a,0xfb,0xb6,0x30,0x1e,0x50,0x8c,0xae,0x38,0x49,0x45,0x72, -0x1d,0x2e,0x5a,0x62,0x09,0xb8,0x44,0x12,0x7c,0x9b,0xf1,0x64, -0x80,0x0a,0xb8,0x0a,0x0f,0xca,0xe9,0x63,0x21,0x6a,0x69,0x6e, -0x87,0xf3,0x49,0x6f,0xb8,0xdb,0x53,0x4f,0xad,0x47,0xf3,0x1b, -0x37,0x0b,0x5b,0xf8,0x5c,0x74,0x7e,0xd3,0x89,0x60,0x2b,0x0d, -0x0d,0x18,0x83,0xdc,0x3c,0x01,0x31,0x57,0x89,0xca,0x7c,0x17, -0x71,0xd8,0x9b,0x5d,0x03,0xe7,0x84,0x65,0xdb,0x4b,0x72,0x39, -0x3d,0xc3,0x47,0x66,0x34,0xbb,0xb6,0x9d,0xb7,0x19,0xa1,0xdd, -0x5d,0x9b,0xe4,0x5e,0x4d,0x6f,0x6c,0xb4,0x13,0x1b,0x89,0x1c, -0xd7,0x5d,0x70,0x1c,0x73,0x64,0xac,0x12,0xdd,0x0d,0x95,0x83, -0xc2,0xcc,0x18,0x40,0x2b,0x25,0x53,0x64,0xcb,0xf6,0x86,0x68, -0x7c,0xfe,0x41,0x79,0xc4,0xcf,0xf4,0x79,0x10,0xbb,0x5b,0xc6, -0x8b,0xbf,0x36,0xe4,0xc2,0x9b,0x6e,0x7b,0x80,0xe4,0x1e,0x3e, -0x71,0xfc,0x35,0x3a,0x3e,0x75,0x0f,0x9f,0x24,0x2f,0x74,0x27, -0x9a,0x14,0x11,0x07,0xc9,0xbf,0xee,0x82,0xcc,0xf2,0x72,0xef, -0xd5,0xce,0x4f,0x6f,0x4f,0xde,0x1f,0x02,0x49,0x1d,0x9f,0x5a, -0x20,0xf2,0x02,0xa1,0x80,0xdf,0xce,0xed,0xed,0x0a,0x14,0x81, -0xe5,0xbc,0x5e,0x85,0x20,0xf4,0x66,0x56,0xe1,0xc3,0xdc,0x7b, -0x80,0xe6,0x7c,0xd3,0x1c,0x27,0x2e,0xbf,0x98,0xf8,0x63,0xfe, -0x85,0xb4,0xd8,0xbc,0x4f,0x6c,0xdb,0xaa,0xde,0xf4,0x01,0x60, -0xbf,0xee,0xd2,0x8e,0xcc,0xf8,0x8a,0xeb,0x75,0xe8,0xe6,0xb5, -0xcc,0x47,0x04,0xe6,0x75,0x15,0x82,0x90,0xad,0xe1,0x83,0x6a, -0x60,0xce,0xf3,0xab,0xc1,0x88,0x0e,0xe9,0x63,0xfa,0x34,0x9f, -0xae,0xc7,0xc3,0xdc,0x10,0x61,0x7d,0x1a,0xa8,0x9b,0x84,0xad, -0x2c,0xa3,0x24,0x27,0x61,0x53,0xf6,0xa4,0xc3,0x91,0xf2,0xce, -0x85,0xe2,0xac,0xc0,0x98,0x3a,0x20,0x4a,0x5b,0x03,0xd6,0xd0, -0xdb,0xd7,0xd0,0x4a,0xea,0x42,0x79,0x3f,0x87,0xd2,0x67,0x70, -0x43,0xf6,0x6f,0x91,0x25,0x2a,0xbc,0x3c,0xa3,0xe8,0x8e,0x28, -0x82,0xa8,0xd7,0x36,0x24,0xbd,0x16,0x40,0x93,0x1e,0x3f,0x81, -0xf6,0x77,0x8e,0xbe,0x48,0x58,0x7b,0xe8,0x7f,0x1f,0xc8,0xb5, -0xd1,0x0a,0xf4,0x4a,0x50,0xf3,0xec,0x47,0x3c,0x45,0xab,0x86, -0x87,0xb2,0xc0,0x96,0x09,0xf0,0xde,0x8c,0x0a,0x61,0x04,0xba, -0xad,0x5f,0x53,0xbe,0x95,0x83,0xce,0xd7,0xb3,0x9b,0xe1,0xaf, -0x91,0xd8,0x7a,0xb7,0xb7,0x58,0x87,0x39,0x87,0x3f,0xb0,0x7c, -0xa0,0x21,0x6c,0x6b,0x9c,0x02,0xad,0x53,0xfe,0x1a,0x2d,0xdb, -0xf8,0xb4,0xcc,0x7f,0x3a,0x0b,0x02,0x02,0xe1,0xdd,0x18,0xdb, -0x9e,0xf9,0x5e,0x90,0xf8,0x44,0xf4,0xef,0x79,0x63,0x12,0x68, -0xe5,0x38,0x2b,0x60,0xf2,0x5e,0x2a,0x98,0xa4,0xf2,0xe2,0x48, -0x2e,0x6c,0x67,0x18,0x9f,0x29,0x62,0x09,0x6a,0x35,0x34,0x1f, -0xf0,0xc2,0xd7,0x5d,0xb1,0x49,0x11,0x67,0xf3,0xc8,0x9e,0xf0, -0x3a,0xe9,0xeb,0x76,0x76,0x7d,0x3b,0xb9,0xfc,0x1c,0x99,0x67, -0xd6,0x51,0x1f,0x00,0xa9,0x45,0x64,0xe5,0xa9,0x76,0x33,0x00, -0x59,0x19,0x04,0x7b,0x84,0x52,0xbb,0xa5,0xf6,0x42,0x12,0xc8, -0x63,0xcd,0x67,0x74,0x64,0x16,0xd4,0x9e,0xb4,0xf5,0xec,0x95, -0x8a,0x2d,0xfe,0x46,0xf4,0x8b,0xe8,0xb5,0xb4,0x8d,0x73,0x3d, -0x82,0xc6,0x59,0x63,0x15,0xca,0x88,0x21,0x4a,0x04,0x9e,0xe8, -0x8c,0xa7,0x78,0x41,0x51,0x88,0xc9,0x0a,0x3e,0x1e,0x65,0x6a, -0xcf,0xc6,0x6e,0x5e,0x4b,0x7e,0x4e,0x48,0x0c,0x4d,0x6b,0x3b, -0xaa,0xf0,0xdd,0x01,0x0a,0x6f,0x94,0x78,0xaa,0xca,0x90,0xa4, -0xce,0xe2,0x03,0xaa,0xe9,0x66,0x8a,0xa9,0x2b,0x6c,0x93,0x46, -0xcc,0x35,0xe0,0x75,0x72,0x83,0x3f,0xc9,0x02,0x35,0x82,0xb0, -0x38,0xf7,0xce,0xa7,0x38,0x74,0x41,0x8f,0x35,0x3c,0x68,0xef, -0x23,0xd9,0xb5,0x72,0x92,0xd3,0xe3,0xfb,0x0a,0x52,0xe9,0x51, -0x17,0x93,0xa9,0x3c,0xee,0x6d,0x99,0xa5,0x31,0x27,0x54,0xd0, -0x07,0x51,0x46,0xb2,0xa6,0x65,0xff,0xd8,0x17,0xcf,0x04,0x60, -0x45,0x79,0x4f,0xe8,0x86,0x13,0x0b,0x50,0xc2,0xf6,0xc4,0xaa, -0x24,0xce,0x42,0x29,0xb2,0x58,0xc8,0x47,0x2a,0xd9,0x1b,0xbe, -0x6e,0xd8,0x28,0xaf,0xc4,0x5a,0x51,0xf5,0x43,0xb5,0x72,0xf9, -0x34,0x9c,0x80,0xc8,0xc9,0x1a,0x80,0x43,0x98,0xd7,0xca,0xfd, -0x15,0x9c,0x00,0xc6,0xea,0x1c,0xb6,0x93,0x9a,0x19,0xe0,0xf8, -0x59,0x66,0xf0,0xa4,0x29,0xee,0x66,0xae,0x85,0x40,0x87,0x4c, -0xe6,0x91,0x99,0x8f,0xc7,0x85,0xe3,0xf9,0x0c,0x0b,0x3c,0x30, -0x7a,0x8b,0xc5,0x83,0x1d,0x64,0x04,0x0a,0xb6,0x03,0xa1,0x00, -0x69,0xc3,0x4a,0x11,0x40,0xf6,0x8b,0x52,0x47,0x81,0xf2,0x8d, -0x34,0x61,0x76,0x67,0xa7,0x81,0x18,0x60,0x27,0x4c,0xc4,0x89, -0x11,0x30,0x9a,0xb0,0xfa,0xd5,0xfc,0xe6,0x7a,0xfa,0x04,0x7e, -0x83,0x38,0x55,0xf1,0x1b,0x35,0x20,0x01,0xb3,0x09,0x2b,0x0f, -0x7b,0xfd,0xff,0x8f,0xec,0xa3,0x11,0x7b,0xa8,0x1d,0xdf,0x29, -0x8a,0x9f,0x20,0x80,0x47,0x38,0x89,0xdb,0x86,0xda,0xc5,0xd8, -0x92,0x71,0x96,0x85,0x0b,0x34,0xad,0xf9,0x8f,0x4e,0x9c,0x37, -0xe5,0xc5,0x54,0xcf,0x9f,0xe2,0x62,0x5a,0x31,0xef,0xe4,0xe1, -0x31,0x2d,0x11,0xfe,0xe1,0xf1,0x57,0x2d,0xae,0x8f,0xf1,0x4c, -0x2b,0x23,0x4b,0xb9,0xca,0x96,0x3e,0x8d,0x1f,0x59,0xbe,0xd2, -0x50,0xdd,0x5f,0xcd,0x88,0x9e,0xca,0x89,0xec,0x54,0x11,0x6d, -0xcc,0x5b,0xb1,0x13,0x50,0xf5,0x58,0xdb,0xda,0xb7,0x40,0x33, -0x2e,0x36,0x5f,0x77,0x25,0xa8,0xe3,0x10,0x20,0xae,0x1b,0xf0, -0x8d,0xfa,0xa4,0x65,0xd8,0xe1,0xd1,0xb9,0x56,0x76,0xe3,0x94, -0x4e,0x58,0x67,0x1a,0x87,0xe1,0xcc,0xc3,0xb6,0x26,0x70,0x8a, -0xd3,0x90,0x67,0xe7,0xb2,0x70,0x5c,0x21,0xb7,0xce,0xfa,0x46, -0xc1,0x77,0x2c,0x5e,0x2d,0x11,0xe1,0xec,0xa3,0x51,0xe7,0xb5, -0x16,0x8f,0xb6,0x4a,0x77,0xe8,0xdf,0x0a,0xb5,0x39,0xfc,0xad, -0xd2,0xa2,0x24,0x74,0x14,0xbb,0xcb,0x38,0x4e,0x2e,0x02,0x7c, -0xbc,0x4c,0x1b,0x5f,0xa1,0xd7,0x0e,0x20,0x3b,0x81,0x25,0xd8, -0x14,0x0d,0x90,0x5d,0x81,0x1b,0x32,0x2a,0xce,0x60,0x6d,0x64, -0xe1,0x7e,0x21,0x78,0x55,0xa9,0xf4,0x66,0xa6,0x23,0xb2,0x07, -0x65,0x1f,0x5e,0x8a,0xb6,0x87,0xbe,0x6d,0x50,0x0f,0x27,0x9d, -0x1a,0x8d,0x94,0x0a,0xec,0x41,0xab,0x9b,0x07,0x32,0x4e,0x6c, -0x7b,0x79,0x97,0x6e,0x21,0xe3,0x31,0x69,0x0a,0x8a,0x59,0x40, -0xd1,0x5d,0x06,0x78,0xbd,0xfe,0x05,0x48,0xe3,0x71,0xe4,0x18, -0x37,0x67,0x17,0xfd,0xe3,0xc8,0xb8,0xf8,0x26,0x73,0x24,0x18, -0x4f,0x81,0xc4,0x06,0x7b,0x9c,0x3f,0x1c,0x5b,0xdd,0x3e,0x59, -0x79,0x55,0xd2,0x35,0x5a,0x36,0x72,0x37,0x56,0x4d,0x1b,0x13, -0xe3,0xf1,0xf7,0x00,0xaf,0xcb,0x70,0xea,0x85,0x37,0xea,0x6d, -0xd7,0x89,0x1d,0x3b,0x13,0x33,0xb5,0x2d,0xf0,0xe5,0x80,0x93, -0x5a,0x78,0x20,0x96,0xa3,0x08,0xa9,0xb1,0xc8,0x6c,0x37,0x35, -0x61,0x20,0x50,0x4b,0xac,0x53,0xdc,0x05,0xa4,0xab,0x79,0x1a, -0x8b,0x29,0x5c,0xb5,0x63,0x19,0xaa,0xe6,0xb0,0xa6,0x0a,0xe6, -0x96,0xf8,0xae,0xa3,0x5b,0x68,0x9d,0xdf,0x31,0x4c,0x5c,0x50, -0x14,0xd3,0x54,0xbd,0x2d,0xd2,0x43,0xac,0x63,0x84,0xd9,0x69, -0x34,0x92,0x80,0x04,0x9a,0x40,0x2f,0x11,0xd9,0x13,0x55,0x04, -0x44,0xc6,0x4b,0x59,0x2d,0x94,0x3e,0x9b,0x17,0xb3,0xc9,0x90, -0x8c,0x80,0x49,0xf8,0x26,0x9f,0xf5,0x28,0xbc,0xbd,0x96,0x71, -0xcc,0xa7,0xeb,0xc1,0x85,0x3a,0xc1,0xc8,0x87,0x37,0xd6,0x55, -0x03,0xa7,0xcf,0x80,0x5b,0xe7,0xb3,0xa4,0x1d,0xf5,0xd0,0xdc, -0x95,0x57,0x7b,0x80,0x8c,0xd6,0x32,0x30,0x3f,0x4d,0x44,0x15, -0x63,0x8e,0x04,0xb6,0x18,0x18,0x81,0xdc,0x82,0x3d,0xf5,0x1a, -0xa6,0x65,0xf0,0xcf,0x59,0x41,0x4e,0xd0,0x12,0xbe,0xf8,0xc6, -0x27,0x26,0x7e,0x3e,0x25,0xf2,0xa1,0x3c,0x68,0xa2,0x8c,0x4d, -0x69,0xe1,0xed,0x2a,0x76,0x05,0xec,0x30,0x88,0x7a,0x2a,0xfa, -0x48,0x4a,0x26,0x6d,0xf9,0xc0,0xd6,0x02,0x10,0xb6,0xef,0xc6, -0xf8,0x47,0x09,0xfa,0x5e,0xf4,0x70,0x5c,0x87,0xd0,0x4c,0x47, -0xb5,0xb8,0xc3,0x09,0x45,0xc0,0x94,0xc7,0x6d,0x3d,0x27,0xde, -0x72,0x9c,0xe4,0x88,0x43,0x54,0xd6,0x0a,0x94,0x8c,0x76,0xff, -0x4b,0x43,0xaa,0x0b,0x84,0xef,0x0b,0xe4,0x2d,0x81,0x09,0xb5, -0xdb,0x4e,0xa3,0xcb,0xcb,0x8d,0xdf,0xfc,0xf6,0xfb,0xed,0xad, -0xdf,0x6e,0x84,0xb5,0x47,0x5d,0xbc,0x79,0x42,0x6d,0xc2,0x8a, -0x79,0xba,0x96,0x79,0x3a,0xf1,0x8a,0x3c,0x84,0x57,0xc5,0xa4, -0xc6,0x2b,0x6b,0x97,0x67,0x8c,0xa8,0x09,0xdf,0xf0,0xc7,0x61, -0xf9,0x27,0x62,0x20,0xa8,0x31,0x53,0x51,0x99,0x4e,0x7e,0x1a, -0x8b,0xc1,0xe3,0x70,0xf2,0x9b,0x8b,0x22,0x5c,0x19,0x64,0x62, -0xb5,0x14,0xdd,0x28,0x58,0x7d,0xe6,0x47,0xda,0x92,0x18,0x5e, -0x6a,0xdb,0x8f,0x9a,0xc8,0x42,0x12,0x63,0x6e,0x96,0x95,0xc1, -0xb4,0xcb,0x23,0x27,0x6d,0x34,0x0f,0xf0,0xd7,0xe5,0xe3,0x2f, -0x46,0xfa,0x83,0x2b,0xd2,0x0d,0x83,0x61,0x63,0x53,0x84,0x04, -0x60,0x97,0x75,0x81,0xb2,0x2d,0x77,0xb3,0xa0,0x8a,0xb0,0xa1, -0xdc,0x15,0x98,0x87,0xda,0x48,0x62,0x4e,0x50,0xdc,0x3f,0x17, -0xe6,0x06,0xc5,0xb9,0x2b,0x1c,0x5d,0x6f,0xbd,0x2e,0xe0,0x2a, -0x7a,0x02,0xf6,0xea,0x41,0xde,0x46,0xe3,0x71,0xeb,0x63,0x5f, -0x45,0x03,0xab,0x17,0x99,0x12,0x0d,0x74,0xca,0xa3,0x22,0x59, -0xd3,0xf8,0x31,0x1a,0x0f,0xbb,0xe6,0xc1,0x61,0x24,0xd5,0x92, -0x8a,0xbe,0xa3,0x7b,0x74,0xdd,0x17,0x7a,0xea,0x15,0xee,0x64, -0xae,0x0a,0xf2,0x4d,0x3f,0x3f,0x9f,0x5f,0xa1,0x28,0x3c,0xcd, -0x87,0x97,0xb4,0xaf,0x68,0x8b,0xb9,0x0f,0x88,0xdf,0x1b,0xf5, -0xce,0x87,0x79,0x5f,0x3d,0x98,0x1b,0xf6,0xae,0x2a,0x64,0x0f, -0x8a,0xee,0xc4,0x0c,0x0b,0x03,0x22,0x77,0x50,0x44,0x41,0xe6, -0x28,0x1b,0xdb,0xb2,0x56,0x87,0x1c,0x9a,0xf8,0xfa,0xb3,0xc7, -0xef,0x23,0xf1,0x75,0x99,0xab,0x10,0xbe,0x9d,0xa5,0xa3,0xf1, -0x65,0x8d,0x32,0xa0,0x09,0xe9,0xc9,0x15,0x5e,0x72,0x5d,0x7c, -0xec,0xc0,0x57,0x86,0xa1,0x1b,0x90,0x12,0x7a,0x57,0x39,0xea, -0x04,0x00,0x8b,0x77,0xe9,0xee,0x2c,0x4e,0xe5,0x49,0x3a,0x7c, -0xd7,0x8c,0x4a,0x3c,0x2a,0x9a,0x54,0x79,0x1a,0xae,0x64,0x5a, -0xf8,0x62,0x47,0x47,0x47,0x6f,0xb9,0xf6,0xac,0xba,0xa0,0xcd, -0xd4,0x88,0x52,0x8e,0x21,0x6b,0x42,0x09,0xee,0x2a,0xb4,0x3e, -0xe5,0x68,0x3a,0x1e,0xe6,0x30,0x30,0xf1,0x8c,0xb4,0x36,0xe5, -0xf1,0x39,0x46,0x66,0x56,0x55,0x58,0xc2,0xf4,0x20,0x6e,0x38, -0xbe,0x22,0xe7,0x2b,0x14,0x73,0x8a,0x65,0x60,0xd1,0x92,0x60, -0x13,0x12,0x17,0x0b,0x54,0x27,0x74,0x36,0xfa,0x30,0x37,0x6f, -0x69,0x3a,0x85,0x99,0x0a,0x68,0x54,0x3d,0x12,0xa8,0x78,0xce, -0x47,0x63,0x80,0xd9,0x45,0xb1,0xa1,0x30,0x3c,0xc8,0x5e,0x54, -0x3d,0xb1,0x20,0x83,0xed,0x04,0x11,0x3b,0xd5,0x60,0x5a,0x18, -0xb9,0x8d,0xd6,0x04,0xd0,0x08,0x98,0x8f,0x58,0x8a,0x0e,0x2c, -0x26,0xa4,0xaa,0x4b,0x22,0x34,0x1b,0x85,0x42,0xd9,0xe5,0x38, -0x88,0xc6,0x08,0x88,0xff,0xd4,0x9b,0x8c,0x82,0x78,0x8c,0x80, -0x78,0xd2,0xc9,0x0d,0x12,0x28,0x06,0xd5,0x94,0x91,0x22,0xd3, -0x42,0x4f,0x5c,0xba,0xfe,0xa4,0xbc,0x94,0xc7,0x1d,0x3a,0xeb, -0x1d,0x0f,0xa5,0xd4,0xeb,0x97,0xb6,0xa5,0x38,0x67,0x02,0xeb, -0x54,0x71,0x68,0xa1,0x2a,0x1f,0x4d,0xe7,0x93,0xfc,0xb8,0x77, -0x99,0xef,0xd3,0x55,0x1b,0x19,0x42,0xa5,0x97,0x8c,0x78,0xbb, -0xe4,0xd5,0x00,0xd8,0xc0,0xbe,0xb2,0x0b,0x66,0x6d,0xec,0xdb, -0x03,0x0c,0xe2,0x29,0xee,0xf4,0x62,0x9a,0x5f,0x5c,0x0e,0xfb, -0x91,0x79,0x76,0x94,0xb3,0xd5,0x56,0x3c,0xad,0x08,0x8a,0xd7, -0x2e,0x07,0xf9,0xb0,0x8f,0x26,0x17,0x76,0x78,0xb9,0x0e,0x54, -0x1c,0x06,0x53,0x65,0x27,0x7e,0xad,0xe6,0xf1,0xa0,0xbb,0xee, -0x67,0x45,0xe4,0x6c,0x47,0x20,0x42,0x55,0x6d,0x93,0xc7,0x32, -0xe3,0xf3,0x0f,0x55,0xcd,0x82,0x68,0x60,0xe3,0xe7,0x1f,0x9a, -0xda,0x70,0x5d,0x86,0xb7,0x14,0xab,0x1b,0x37,0x82,0x9d,0x87, -0x6e,0xdb,0x2b,0x67,0x85,0xe9,0x5b,0x1a,0x14,0xad,0x6a,0x10, -0xe0,0xa0,0x76,0xab,0xf9,0x67,0xd6,0x87,0xba,0x98,0xde,0x2a, -0x5e,0x34,0x98,0xc6,0x64,0x17,0xc9,0x32,0x21,0xd4,0x32,0xa2, -0x08,0xc7,0x4c,0x2e,0xf4,0x0d,0xca,0xa7,0xde,0xa3,0xf9,0xf1, -0xa2,0x61,0x57,0x97,0xb9,0xcd,0xf3,0x8f,0xf1,0x40,0x54,0x14, -0xe6,0x37,0xd9,0x60,0xb1,0x68,0xb9,0x87,0x5e,0xa4,0xea,0x03, -0xb1,0x3f,0x28,0x45,0xaa,0x0e,0x7d,0xcb,0x39,0x9a,0xcb,0x91, -0xa4,0x6b,0x2d,0x6d,0xaf,0xea,0xdd,0x1c,0x49,0x0e,0xd1,0x73, -0xcf,0x19,0xae,0x5f,0x64,0xd1,0x56,0x54,0xaf,0x47,0xbf,0x8b, -0x5e,0x64,0xda,0x21,0xe4,0x60,0x4a,0xa6,0x59,0xa7,0xb7,0xa8, -0x7f,0xeb,0x4b,0x44,0x35,0x7c,0x06,0x78,0x0d,0xdb,0x98,0xb3, -0x89,0xfd,0xfa,0x1f,0x2e,0x6a,0x64,0xbf,0xfe,0x97,0xfd,0xf8, -0xdf,0xff,0x17,0x7d,0x69,0xb8,0x6f,0x50,0xff,0xb0,0x84,0x44, -0x0f,0x91,0xf8,0x3b,0x21,0x01,0xcc,0x0b,0x63,0x76,0x30,0xe6, -0xaf,0x12,0x13,0xbd,0xb7,0x10,0xd7,0x4b,0x00,0x61,0x1c,0x0f, -0x6e,0xd1,0xde,0xc7,0x58,0xb7,0x2d,0xda,0xb0,0x25,0x1a,0xf2, -0xa1,0x3b,0x40,0x19,0x14,0x43,0x92,0x63,0xd6,0x45,0x2c,0x81, -0xb5,0xb7,0x50,0xb1,0x22,0xb9,0x87,0x3f,0xa8,0x51,0x81,0x6f, -0xef,0x51,0x05,0x8d,0x19,0xf1,0xf0,0x78,0x36,0xc9,0xfc,0x4a, -0x47,0xd9,0x61,0xcd,0x98,0xc2,0x72,0x41,0xdf,0x0d,0xa8,0xb9, -0xc1,0x63,0x10,0xd5,0x4e,0xd1,0x1b,0x7c,0x59,0xcf,0x8b,0xc0, -0x37,0xa2,0x2e,0xb9,0xaf,0x44,0xbd,0xad,0x76,0x15,0xe5,0x03, -0x01,0xb2,0x0a,0xca,0x5b,0xfc,0xa8,0xc9,0xaa,0x04,0xf4,0x5b, -0x23,0xb5,0xa5,0xe1,0xfc,0x06,0x95,0x50,0x70,0x0a,0x78,0xd2, -0xaf,0x9d,0xde,0x6f,0x2f,0xbb,0xcd,0x88,0x39,0x9e,0x61,0x74, -0x59,0xe9,0x48,0xbf,0x97,0xed,0xf5,0xa5,0x2f,0xfc,0xf6,0x69, -0x44,0x11,0x78,0xa6,0x41,0x28,0xf2,0xbd,0x5a,0x9b,0xfe,0x6a, -0x52,0x13,0x0b,0x01,0x17,0xd7,0xda,0xd8,0x5d,0x91,0xfa,0x12, -0xba,0xee,0x89,0x9a,0x15,0x3d,0x9f,0x70,0x3d,0x0d,0x48,0x68, -0x3b,0x1d,0x68,0x24,0xf8,0xdd,0xeb,0x8c,0xe8,0x9e,0x8b,0xe6, -0x54,0xb4,0x5e,0x0f,0x87,0x97,0xf3,0x95,0x60,0xd4,0xc8,0x7c, -0x66,0x81,0x0e,0xea,0x75,0xce,0x8d,0x30,0xa4,0x7a,0x29,0x0e, -0x74,0x85,0xe0,0x19,0x88,0xc5,0x5b,0x42,0x62,0xe5,0xbf,0x55, -0xc6,0x93,0x8f,0x95,0x0b,0x95,0x2c,0x16,0xab,0x2a,0x01,0xc2, -0xcb,0xd1,0x0d,0xcc,0xc3,0x95,0x90,0xfb,0x80,0xb6,0xa2,0xc1, -0xc8,0x1e,0x3f,0xa3,0x79,0x49,0xbc,0xdc,0x8d,0x60,0x01,0x61, -0x6a,0x6a,0x2c,0x65,0x88,0x5a,0xdf,0xf1,0x87,0xd8,0x3e,0x17, -0xbf,0x26,0x94,0x89,0x15,0x51,0x69,0xbc,0x53,0xc9,0x84,0x1a, -0xf5,0x29,0x88,0x73,0x97,0xa3,0xb4,0x7c,0x54,0xc7,0x79,0x96, -0xcb,0x02,0x71,0xf0,0x1c,0xf5,0xb4,0x81,0x1c,0xea,0xe5,0x78, -0x66,0xb0,0x6d,0xac,0xe4,0xc9,0x0a,0xea,0x64,0xff,0xe8,0xe2, -0x9a,0x6d,0x54,0x3e,0x4a,0x3a,0xf8,0x62,0x58,0xf4,0x3f,0x03, -0x62,0x31,0x8a,0x50,0x1c,0x77,0x28,0x50,0x4d,0x3b,0x91,0x4c, -0xe8,0x47,0x8d,0xb1,0xc9,0xa4,0x34,0x55,0x8e,0xe3,0x64,0xa4, -0x9f,0x90,0x80,0x24,0x0f,0x39,0xac,0x70,0x58,0x0b,0x62,0x2e, -0x5c,0xa0,0x6b,0x7c,0x34,0x1c,0xe2,0xe7,0x72,0x02,0x8e,0x31, -0x55,0x9f,0xdc,0xfb,0x96,0x67,0x54,0xb3,0xbd,0x22,0x93,0x1a, -0x37,0x98,0x09,0xb4,0x04,0xaf,0x30,0xcf,0x96,0x09,0x72,0x59, -0xdd,0x4a,0x57,0x8d,0xb8,0x93,0x60,0x52,0x0b,0x19,0xb0,0x90, -0x8a,0xcb,0x0a,0xe4,0xe0,0x8d,0xd6,0x97,0x07,0x9f,0xaa,0x5d, -0x62,0x4f,0x1c,0x1c,0xee,0x1d,0xed,0x9c,0x1c,0x1c,0x1d,0x17, -0x0d,0xbd,0x52,0x96,0x24,0x21,0x08,0xcd,0x4b,0xb1,0x7c,0xdf, -0x44,0x72,0xc9,0x5c,0x19,0xb6,0x31,0xda,0xf5,0x33,0x95,0x2d, -0xba,0x66,0xfc,0x03,0x12,0x1b,0xa5,0xe3,0x0a,0x29,0x8a,0x8d, -0x0e,0x9a,0x5c,0xac,0x3a,0x92,0x23,0xc9,0xc8,0x1a,0xc6,0x15, -0xea,0x13,0xf3,0xb0,0x3a,0x69,0x69,0xee,0x7b,0xc0,0xc8,0xae, -0x14,0xb1,0x52,0x72,0x78,0x00,0x3c,0x55,0x05,0x19,0x03,0x51, -0x9e,0x26,0x3e,0x07,0xd4,0xac,0x67,0x06,0x5b,0xdf,0x57,0x14, -0x8b,0x96,0xc6,0xcb,0x33,0xc7,0xd2,0x38,0x75,0x1f,0xaa,0x95, -0xf1,0xc4,0x69,0x2d,0x93,0xaa,0x79,0x26,0xb9,0x1b,0x2d,0xce, -0xaf,0xa6,0x83,0x2b,0x56,0x98,0x53,0x72,0x21,0xf5,0xcb,0x7c, -0x6c,0x1f,0x53,0x94,0x26,0x0b,0x90,0xae,0x57,0xac,0xc6,0xdc, -0x38,0xa5,0x26,0xbd,0x4f,0x5c,0x34,0xa3,0xa2,0x26,0x47,0xb5, -0x56,0x3a,0x6e,0x7e,0x80,0x37,0x97,0x27,0x19,0x92,0x82,0x03, -0x45,0xb3,0x85,0x01,0xb1,0xa4,0x39,0x17,0xe2,0x66,0xd3,0x6d, -0x9f,0x8b,0xca,0xc9,0x8c,0x5d,0x4b,0x26,0xd8,0xaf,0x93,0x36, -0xe4,0x91,0xe7,0x5a,0x9b,0xa7,0x67,0xfd,0xde,0xc6,0x65,0xf7, -0xfe,0xfb,0xe5,0xe6,0x00,0x66,0x6d,0x15,0x2f,0x9b,0x8f,0x06, -0x78,0xc3,0x54,0xe3,0x1a,0x6b,0xa7,0x67,0x67,0xf3,0xa8,0x71, -0x8d,0x0b,0x64,0x37,0x92,0x39,0xd0,0xc8,0xbe,0x37,0x53,0xc1, -0x8d,0x71,0x6c,0xa2,0x05,0x47,0xd8,0xee,0x4e,0x76,0xf1,0x28, -0x92,0x16,0xc6,0x37,0x23,0xb4,0x51,0xfa,0xd9,0xb4,0x7e,0xa3, -0x9d,0x0f,0xc0,0x9e,0x37,0xdb,0x3b,0xde,0xdd,0x39,0xdc,0x3b, -0xbd,0xb8,0xee,0xb6,0x2d,0x14,0x88,0xee,0xc0,0xbf,0x14,0x84, -0x05,0xd7,0x65,0x4b,0xbb,0x5a,0x44,0x67,0x67,0xdc,0x62,0x9b, -0xb4,0xe5,0x98,0x3c,0xf4,0x1e,0xf7,0x74,0x12,0x48,0x5c,0x8d, -0x47,0xb8,0xad,0xeb,0x5b,0x69,0x46,0xca,0x3f,0x0f,0xb3,0x5f, -0xce,0x83,0x07,0x63,0x72,0x16,0x34,0x75,0xc3,0xe3,0x98,0xbe, -0xee,0xd0,0x9f,0x46,0xe2,0xb0,0x1e,0x15,0x91,0x09,0x49,0x61, -0xb9,0x40,0x6b,0xda,0x8b,0x05,0xf2,0x64,0x46,0x17,0x15,0xef, -0xad,0x26,0x3e,0x4d,0x6d,0x54,0xc4,0x27,0x79,0x33,0x03,0xe1, -0xa3,0x92,0x84,0xca,0x6c,0xd0,0x32,0xda,0x29,0xf4,0x5b,0xf4, -0x2c,0x4a,0x12,0x45,0xcf,0xd0,0x87,0x6a,0x0d,0xf6,0x1c,0x1b, -0xf9,0x37,0x4a,0x00,0x49,0x61,0x69,0x8c,0x13,0x2e,0x6e,0xe5, -0x0f,0x55,0xd8,0xf1,0xfe,0x44,0xad,0x41,0x06,0x05,0xe8,0xe8, -0xde,0x20,0xa0,0x08,0xad,0xaf,0x63,0x2b,0x4e,0xb7,0xba,0x76, -0x12,0x8b,0x0b,0x8e,0x53,0x19,0x1d,0xbb,0xb4,0x92,0x0a,0xb3, -0xe2,0x68,0xb8,0xef,0xe7,0x60,0xf0,0x0e,0x00,0x11,0xd4,0x28, -0xc4,0x11,0x0c,0xce,0xf2,0xb4,0xdb,0x34,0xed,0xb4,0x13,0x09, -0x73,0x0c,0xc7,0x9c,0x75,0xfc,0x69,0xcc,0xa1,0x97,0x69,0x78, -0x09,0xc5,0xf4,0xd4,0x70,0x63,0x01,0xdb,0xd3,0x48,0xda,0xbf, -0xec,0xa6,0x06,0x1f,0x00,0x19,0x1b,0xcd,0xaf,0x2a,0xdd,0x8b, -0x79,0xbc,0xbf,0xe5,0x7c,0x2e,0xc9,0x9a,0x5b,0xb2,0x1c,0xc1, -0xd2,0x65,0x69,0x41,0xb8,0xb7,0x39,0xac,0x11,0x3c,0x36,0x4c, -0x72,0xbd,0x0d,0xd4,0xd3,0xe0,0xae,0x06,0x3a,0x78,0x0e,0xa1, -0x6d,0x0e,0x6e,0xe3,0xe3,0x01,0xc5,0xdb,0x61,0xb6,0x40,0xc4, -0x76,0x10,0xb3,0x8d,0x51,0xcf,0x83,0xa8,0xe7,0xdd,0x36,0x44, -0x75,0xaa,0x78,0x66,0xd0,0x17,0xcf,0x89,0xc6,0x47,0xcf,0x97, -0x6e,0x4a,0x3f,0x4f,0x20,0xbc,0xfd,0x78,0xc9,0x6d,0x2e,0xb9, -0xed,0x4b,0x6e,0x63,0xc9,0xc7,0x0b,0x72,0x39,0x35,0x08,0xa7, -0x26,0xb5,0x83,0xd0,0xd8,0xe0,0x8e,0x67,0x80,0x2d,0x54,0x1b, -0xd0,0x13,0x09,0xe4,0x2b,0xd8,0xa3,0xc2,0x34,0x82,0xaf,0x19, -0x1e,0x32,0x4e,0x7a,0x10,0x9c,0xd4,0x22,0x2e,0x21,0xe5,0x5a, -0xc9,0x52,0x66,0x0b,0xcc,0x48,0x7b,0xbd,0x49,0x48,0x15,0xac, -0x4e,0x4f,0x78,0xab,0x48,0xd6,0x5e,0xd7,0x2f,0x07,0xc3,0x19, -0x3e,0xba,0x0e,0xf6,0x8d,0xaa,0xfa,0x9b,0xe9,0x15,0x4f,0x4f, -0xe7,0xd3,0x34,0x90,0xfb,0xa7,0x5f,0x46,0xb3,0xde,0x67,0x90, -0xfb,0x8f,0xe9,0xc3,0x0a,0xfe,0x27,0x58,0x42,0x8c,0x8c,0xa3, -0xdc,0xef,0x36,0x01,0xec,0xa9,0xf2,0x92,0x5c,0xba,0x05,0x3b, -0x81,0xe7,0xcb,0x6e,0xcd,0xba,0xc4,0xc3,0xec,0xa7,0xc0,0xa4, -0x71,0x6f,0xe0,0x27,0x82,0x71,0xa8,0x34,0x2d,0x8f,0xa7,0xe8, -0x02,0xff,0x57,0x39,0x92,0xc2,0x3e,0x95,0xb0,0xe2,0x37,0x83, -0x5b,0x99,0x75,0x88,0x63,0x75,0xfc,0xaa,0x5a,0x37,0xcf,0xc7, -0x78,0xbb,0xa8,0x46,0x00,0xe4,0x04,0x44,0x3f,0x2f,0x6e,0xd2, -0xf5,0x53,0x09,0xe9,0x75,0x98,0xf8,0x85,0x6d,0x72,0xde,0x32, -0xf9,0xb6,0xc9,0x9f,0x9b,0xfc,0x7b,0x42,0x22,0x40,0xe0,0xc5, -0x96,0xd8,0x33,0xd3,0x9c,0x62,0xab,0x6b,0x66,0x8a,0x17,0xe0, -0xc2,0x38,0xcb,0xb2,0xbc,0x05,0x6b,0x17,0xfc,0x6c,0xf3,0xcf, -0x73,0xfe,0xf9,0x7e,0xb1,0x58,0xcb,0x5b,0x78,0xe5,0xb2,0x8d, -0x7f,0x9e,0xe3,0x9f,0xef,0x13,0x8d,0x92,0x50,0x85,0xde,0x69, -0x73,0xc3,0x42,0xcc,0x3c,0x16,0x25,0xac,0x83,0x16,0x76,0x62, -0x24,0x21,0xa8,0xc6,0xf3,0xae,0x7a,0x5d,0x53,0xae,0xb8,0x40, -0xe4,0x85,0x15,0x93,0x65,0x38,0x9d,0x94,0x62,0xf9,0x07,0xc7, -0x06,0x47,0x00,0x78,0xe0,0x32,0xbf,0xc9,0xa1,0x76,0xd8,0xdb, -0x5a,0x24,0x0b,0x6b,0x36,0xc0,0x9f,0xbb,0xa1,0x31,0xd2,0x16, -0xa4,0x1f,0xd8,0xca,0xe6,0x2d,0x5c,0xb1,0x8d,0xb0,0x71,0x0f, -0x77,0x3e,0xea,0x4d,0xbe,0xa0,0xf7,0xc4,0x91,0x99,0x0c,0xae, -0xae,0xbd,0xb9,0x84,0xa7,0x08,0x81,0x97,0x41,0xb4,0x00,0x58, -0x92,0x23,0x7d,0x32,0xf1,0x9f,0x52,0x4c,0xd3,0x06,0xf5,0x56, -0x05,0x26,0x99,0xd4,0x3c,0xcc,0x2f,0x81,0x96,0x07,0xfd,0xfe, -0x30,0xff,0x06,0x14,0xb0,0x74,0x90,0xd0,0x61,0x50,0x41,0x1c, -0xe3,0x51,0x14,0x56,0x1d,0x96,0x08,0xc3,0x21,0x59,0xaf,0x33, -0x00,0x15,0xb1,0xba,0x15,0xe7,0x03,0xdd,0x88,0x7f,0x46,0x1f, -0x12,0xa0,0x72,0x47,0x16,0x50,0x5c,0x8d,0x11,0x3d,0x0b,0xa6, -0x13,0xcc,0x38,0x78,0xea,0x27,0x91,0x78,0x46,0xdc,0x4e,0xca, -0x73,0x0d,0xa8,0x96,0x48,0x23,0x5a,0xa2,0xd6,0x3d,0xfc,0x6b, -0xc3,0x3f,0x5a,0xd8,0x7c,0x59,0x39,0x30,0x26,0x06,0xb9,0x8b, -0x76,0x04,0xf1,0x22,0x6c,0x4d,0x88,0x11,0x0a,0x24,0x89,0x57, -0x65,0x57,0xa5,0xe4,0x98,0xcb,0xc7,0xc0,0x2c,0x4e,0xab,0xbb, -0xc4,0xe2,0xcb,0x3b,0x02,0x76,0xfb,0x55,0x82,0xc4,0x9e,0xbf, -0x82,0x66,0xa9,0xea,0x4e,0x07,0x28,0x46,0x4a,0x08,0x0d,0xfe, -0xd0,0xa3,0x60,0x17,0x13,0xd4,0x17,0xba,0xf5,0x52,0xfa,0x23, -0xef,0x83,0x46,0x86,0x5e,0xc6,0x68,0x84,0x94,0x53,0x0f,0x10, -0x83,0x58,0xf0,0x5a,0x13,0xa1,0xc6,0x76,0xc8,0x22,0x4a,0x5c, -0x8f,0x60,0x99,0x36,0x15,0x0c,0x29,0xc6,0x6e,0xb9,0x0c,0x57, -0x88,0x33,0xd3,0x63,0x61,0xe3,0x0a,0x5e,0xce,0x04,0x3e,0x09, -0x04,0xb2,0x5c,0xc5,0x9e,0x25,0xd2,0x8d,0xc4,0xf4,0xd5,0x88, -0x07,0xba,0x0a,0xaf,0x14,0xf1,0x92,0xa3,0xf2,0x37,0x64,0xee, -0x26,0xab,0x1a,0x0d,0x58,0x45,0x6f,0xe7,0xea,0xb9,0x28,0xdf, -0x30,0x50,0x64,0x57,0x86,0x86,0x6b,0xd2,0xc3,0xe2,0xef,0x15, -0x38,0x0d,0x46,0x23,0xec,0x71,0x77,0x0e,0x5f,0x3c,0x75,0x9f, -0x7a,0x63,0x21,0x65,0x59,0xdb,0x22,0xba,0x5c,0x4a,0x95,0x6c, -0xb3,0x49,0x8d,0x81,0xee,0x37,0x1d,0xef,0x6e,0x9b,0x68,0x63, -0xca,0x9e,0x18,0xd5,0x38,0xeb,0x68,0xde,0x90,0xe0,0xc4,0x32, -0x6a,0xa0,0x85,0x4f,0xc5,0x16,0xf3,0x42,0x67,0x66,0x11,0x3e, -0x34,0xa5,0xd9,0xc9,0xb0,0x0a,0x1c,0xf9,0xe6,0x76,0x38,0x40, -0x6f,0xae,0xae,0x9e,0xda,0xf9,0x7c,0x56,0x71,0xb4,0xb8,0xa5, -0x57,0x72,0x3c,0x5e,0xac,0x5d,0xf4,0x46,0xb4,0x5a,0x9c,0xe7, -0x52,0x9a,0xfc,0xa3,0xbb,0x15,0x83,0x10,0xf5,0xd8,0xf9,0xfb, -0x1d,0xf1,0xe7,0x5e,0xe6,0x92,0x82,0xa0,0x64,0x10,0x8e,0xa8, -0x33,0x3b,0xd6,0x98,0x10,0xbb,0x29,0xf1,0x6a,0xe9,0x22,0x61, -0xd5,0xaa,0x8f,0x86,0xe3,0xab,0x01,0x94,0x3c,0x38,0x5a,0xd5, -0x4b,0x1d,0xea,0x25,0x2e,0xa8,0x70,0xae,0x20,0xcc,0x4e,0xf5, -0xba,0xe0,0xca,0x24,0xa8,0x1e,0xe6,0xfb,0xd7,0xc9,0x21,0xa9, -0xeb,0x18,0xde,0x8d,0x25,0xc5,0x36,0x28,0x1c,0xab,0xe6,0xb3, -0x24,0xef,0xbc,0x7b,0xb9,0x7a,0x3e,0x7f,0xd5,0x84,0xd6,0x00, -0x93,0x65,0x09,0x0f,0x8a,0xbf,0x2f,0x32,0x95,0x5f,0xe6,0x20, -0x1a,0xcc,0x56,0x12,0x1b,0xec,0xa2,0xc8,0xd7,0xda,0x93,0xeb, -0x4d,0x4c,0xd8,0x09,0xbe,0x82,0x62,0xd5,0x93,0x9c,0x5f,0xac, -0xf6,0x86,0x2b,0x29,0x3d,0x83,0x15,0x61,0x0d,0xff,0x64,0xf2, -0x99,0x3d,0x8e,0x8d,0xaf,0xb0,0x84,0x8b,0xae,0xb1,0x88,0x4d, -0xaf,0xdf,0x1f,0x90,0x07,0xec,0x15,0xb8,0xfc,0x00,0xf5,0xbf, -0x80,0x7f,0x3f,0x20,0x22,0x2f,0x9e,0x80,0x87,0xae,0xac,0x84, -0x89,0xaf,0xad,0x8a,0x32,0xe4,0x5d,0x31,0xf9,0x84,0x20,0x8c, -0x42,0x54,0x1a,0x11,0x79,0x0d,0x6c,0x27,0x0f,0x55,0x5f,0x84, -0xe1,0xbd,0xae,0x05,0x88,0x14,0xb3,0x55,0xa1,0x33,0x17,0x6e, -0x14,0x62,0xf1,0x1d,0x60,0xb1,0x09,0xff,0x7e,0xf5,0x18,0x26, -0x52,0x7c,0x05,0x02,0x73,0x35,0xcd,0xa9,0x48,0xdb,0x09,0x32, -0xb6,0xb5,0x49,0xe7,0x76,0x32,0xb8,0xa1,0x6c,0x69,0x61,0x54, -0xd0,0x79,0x62,0xc7,0xd5,0xfb,0xd7,0xbd,0xa3,0x83,0x72,0xbd, -0xc5,0x32,0x6b,0x58,0xc6,0xca,0xa0,0xe5,0xdc,0xae,0x2e,0xb5, -0x73,0xb0,0x51,0x62,0x85,0x88,0x42,0xb8,0x11,0xb0,0x20,0x63, -0x00,0x29,0xd1,0x59,0xb0,0x8e,0x1b,0x2b,0x43,0x83,0x80,0xe3, -0x0f,0x0e,0x6c,0xb1,0x53,0x55,0xac,0x87,0xb6,0x06,0x5f,0xe6, -0x17,0xc3,0x1e,0x9b,0xa2,0x8f,0xcb,0xd9,0xef,0x55,0x76,0x76, -0x11,0x25,0x99,0xee,0xcb,0x2b,0x75,0xdb,0x66,0x74,0x0d,0x94, -0x88,0x70,0xb1,0x40,0x6e,0xdf,0xb3,0x2d,0x52,0xab,0x58,0xb8, -0x4b,0x90,0x7d,0x45,0x2c,0xf9,0x9c,0x10,0x98,0xd9,0x08,0x98, -0x6f,0xf9,0xa4,0x37,0x44,0xbd,0x2d,0x77,0xb2,0x34,0xa2,0xbb, -0x50,0x7e,0xd7,0xdd,0xc6,0x40,0xe6,0x7b,0xcb,0x40,0xd3,0xf1, -0x18,0x1f,0xcf,0xe9,0x63,0xba,0xab,0xc6,0xa5,0x89,0x1e,0x88, -0xc7,0xae,0x1f,0xa5,0xf7,0x77,0x7b,0xc3,0xa1,0x8d,0x34,0xee, -0x6d,0xb9,0x7c,0x90,0x71,0xa6,0x24,0x05,0x70,0x21,0x10,0x9b, -0x6c,0xcb,0x85,0xdd,0x46,0x27,0xe2,0x16,0x26,0x0c,0x39,0xdd, -0x15,0x3c,0xa5,0x38,0xdd,0x8a,0xef,0xb0,0x11,0x5b,0x5f,0x3c, -0x38,0xc3,0xdc,0x3f,0x3c,0x38,0x3e,0x7e,0xf3,0xe3,0xdb,0x3d, -0xaf,0x00,0x20,0x39,0x03,0x59,0xcf,0xc3,0x61,0x94,0x44,0x3a, -0xc2,0x78,0x37,0x86,0xbc,0xe5,0x2e,0x1e,0xa1,0x53,0x1e,0x75, -0x84,0xbe,0x4a,0xee,0x57,0xcb,0x2c,0xe9,0xe6,0x54,0x1d,0xa3, -0x2f,0x16,0x42,0x47,0xd5,0x8b,0x72,0xc5,0xe1,0x3a,0xe5,0x63, -0x59,0xb9,0xb0,0xde,0xcb,0x21,0x7b,0x25,0x40,0xc6,0xb9,0x70, -0xe8,0xee,0xbb,0x43,0x8f,0x8a,0x7b,0x2b,0x49,0x62,0xc9,0xab, -0x51,0x28,0xf6,0xfa,0x77,0xfa,0x3c,0xab,0xf0,0x40,0xf8,0xc1, -0x1d,0x0f,0x89,0xf6,0xe6,0xd6,0x8c,0x71,0xe8,0x83,0x24,0x10, -0x28,0xa5,0x8a,0x20,0xda,0xdd,0xed,0x77,0xe2,0xbb,0xac,0xac, -0xa3,0x80,0x8e,0x7e,0x99,0x02,0xef,0xea,0xf5,0x3b,0xb1,0x3b, -0x13,0xdf,0x66,0x77,0xe8,0x45,0xe6,0x2e,0x1a,0x80,0x60,0x8f, -0xa6,0x2d,0x9a,0x10,0xf0,0x2f,0x7d,0xcc,0x6d,0xc1,0x3a,0x39, -0xfa,0xe1,0xba,0xe7,0x4c,0xf0,0x89,0x5a,0x59,0xf0,0x81,0x41, -0xf8,0x48,0xbc,0x97,0x83,0x07,0x6f,0x37,0x82,0x26,0xa2,0x37, -0xb1,0x87,0x5a,0x53,0xa5,0x6c,0x11,0xf6,0x06,0x35,0x4a,0x79, -0x53,0x0d,0x47,0x28,0x98,0x8c,0x97,0x23,0x3b,0xfd,0x58,0x33, -0xdf,0xab,0x03,0xf1,0x1e,0x00,0x0f,0xcc,0x13,0x58,0xac,0xd5, -0x79,0x0f,0xef,0x13,0xfa,0xe3,0xda,0x4a,0x61,0xba,0x4d,0xb6, -0x3a,0x1c,0xaf,0x33,0x51,0x52,0x1e,0xed,0x24,0x5a,0x29,0x74, -0x86,0x5b,0x86,0xae,0xe3,0x0f,0x01,0xa2,0x9d,0x20,0x14,0x42, -0x48,0x39,0xf4,0x55,0xbb,0x0c,0x5d,0x5e,0x7c,0xb3,0x8f,0x0e, -0x67,0x13,0xf4,0x06,0x16,0x4c,0x3f,0xcb,0xb5,0x58,0xb5,0x0a, -0xc6,0x9a,0xb2,0x59,0xad,0x2a,0xff,0x6d,0xf9,0x0e,0x6f,0x4e, -0x52,0x4a,0xa1,0xfa,0xf0,0x10,0x8a,0x7e,0x5b,0xf2,0xbb,0x2d, -0xbf,0xcf,0xe5,0xf7,0xfb,0x2e,0xdd,0xd8,0x38,0x9a,0x94,0x95, -0x2a,0xbb,0x6b,0xff,0x53,0xc8,0xd2,0x6d,0x5a,0x95,0x54,0x59, -0x5e,0xb0,0xb4,0x57,0xf7,0x57,0x7c,0x97,0x00,0xe4,0xb2,0x6b, -0x17,0x8b,0xb5,0x2d,0xa2,0x8c,0x6e,0x35,0x65,0x84,0x85,0xc3, -0x59,0xef,0x61,0x0a,0xe1,0xd8,0x70,0x62,0xdc,0xa7,0x5b,0x94, -0xd0,0x4f,0xad,0xae,0xb5,0x95,0x2c,0x9f,0x40,0x5a,0x8f,0x32, -0x12,0x4f,0x60,0xd0,0x6c,0xba,0x26,0x21,0x4a,0x51,0xa8,0x05, -0xd4,0x02,0x99,0x0a,0xb8,0x4e,0xcb,0x1b,0xd3,0xff,0xe9,0xf4, -0x12,0x21,0x3b,0x4c,0x76,0x59,0x46,0xf1,0xfe,0xc7,0x9d,0xc2, -0xa8,0xb6,0xe8,0x19,0x69,0xd7,0x7f,0x3b,0xfb,0xff,0xe4,0x7c, -0x9f,0x57,0x74,0xf9,0xf2,0x81,0x2e,0x2f,0x6c,0xf0,0x91,0x91, -0xf0,0x9a,0xcf,0xbb,0x44,0x10,0x17,0xfc,0xd1,0xa7,0xeb,0x2d, -0xd8,0x36,0xb5,0xbd,0xf9,0xb3,0x60,0xac,0x1c,0x2e,0x72,0x0c, -0x0f,0xe1,0xd4,0xd9,0x93,0x4b,0x99,0xb5,0x24,0x1c,0xfa,0x07, -0x87,0x6c,0xf9,0xd4,0x21,0xe3,0xbe,0x42,0xaf,0xb2,0x3c,0x66, -0x1e,0xc5,0xe2,0xe9,0x8e,0x4d,0xc9,0x5c,0x16,0x3c,0xdb,0xe1, -0xf2,0xa7,0x36,0xae,0x49,0x4c,0xd2,0x85,0xa8,0x29,0xe1,0x39, -0x9f,0x57,0x0f,0x83,0x72,0x4f,0x1a,0x57,0x7f,0x02,0x85,0x32, -0x2c,0x73,0x60,0x60,0xe5,0x5b,0xf6,0x98,0x36,0x0b,0xb4,0xcb, -0x8c,0xdf,0xdb,0x67,0xea,0x3c,0xc1,0x68,0x2e,0x9d,0x05,0x3c, -0xdb,0x28,0x89,0x23,0xd3,0xe2,0x87,0x51,0x6b,0x6f,0xa6,0x17, -0x62,0xa3,0xe4,0xd8,0x4c,0x1f,0x4e,0xd9,0x71,0x40,0x99,0xb0, -0x13,0x2b,0x4c,0xdc,0x76,0x37,0xc4,0x43,0xd7,0xac,0x2b,0xd3, -0xf0,0xd5,0xf1,0xcb,0x23,0x87,0xd7,0xf7,0x74,0x95,0x43,0x3d, -0xc1,0x17,0x3c,0x5b,0xe8,0xe1,0x9a,0xe9,0xd0,0x89,0xe9,0x49, -0x5a,0x38,0x82,0x43,0xfb,0x82,0x5b,0x6b,0x85,0xeb,0x86,0xd2, -0x41,0x79,0x6f,0xa4,0xce,0xb2,0x79,0x62,0x88,0x1c,0x8c,0x0c, -0xd8,0x12,0xad,0x93,0x75,0xd7,0x82,0x70,0x81,0xa4,0x5d,0xb2, -0x8d,0xe0,0x74,0x6d,0x11,0xc0,0x8a,0x4b,0xe8,0x96,0xef,0x1a, -0x44,0xb4,0xd9,0x9f,0xc4,0x90,0x38,0xe9,0x1a,0x7a,0xf2,0xc5, -0xd9,0x23,0x5c,0x24,0xb0,0xb6,0xd1,0x24,0x6b,0x1b,0x5b,0x96, -0x3b,0x3a,0x03,0x59,0x4c,0xce,0x38,0x91,0x2b,0xd5,0x47,0x6d, -0x76,0x7b,0xf0,0x6f,0xeb,0x6b,0x8b,0x85,0x55,0xd2,0xf6,0x40, -0x77,0x9c,0x80,0x1a,0xc9,0x03,0x6d,0x15,0x89,0x8b,0x88,0xca, -0x02,0x53,0xca,0xe6,0xca,0x54,0x3c,0x19,0x95,0xd0,0x11,0x64, -0x64,0x42,0x64,0x25,0xbb,0x20,0x8d,0x11,0xda,0x9a,0x7a,0xf3, -0xc6,0x16,0xea,0xef,0xab,0xfd,0xb9,0xf0,0xd2,0x24,0x49,0x6e, -0x81,0x5a,0x92,0xb8,0x28,0x26,0x2c,0xee,0xb4,0x07,0x77,0x67, -0xb0,0xe2,0x8e,0x7c,0x94,0x8f,0xb9,0x3d,0x24,0x62,0xd9,0xe9, -0xf9,0xd5,0x3d,0xe4,0xdb,0xea,0x06,0xcb,0x7e,0xa8,0x4b,0x93, -0xe9,0x2d,0x82,0xfb,0xbd,0x15,0xd7,0x21,0xff,0x16,0x72,0xd5, -0x16,0xfe,0xd9,0xc6,0x3f,0xcf,0xf1,0xcf,0xf7,0x7e,0x9c,0x4b, -0x22,0x9a,0x42,0x85,0x4a,0xbb,0xfa,0x57,0x65,0x69,0x3d,0x9e, -0x65,0xfb,0xf1,0x2c,0xcf,0x1f,0xcf,0xe2,0xb1,0x5e,0x25,0x87, -0x09,0x11,0x91,0xdb,0x0a,0x24,0x56,0xe8,0x9d,0x8c,0xd3,0xea, -0x75,0x71,0x47,0x5c,0xd0,0x2b,0xc2,0x26,0x26,0x1d,0x4e,0x63, -0x21,0xcc,0xe9,0x0b,0xb3,0x93,0x7e,0x01,0x83,0x76,0xbf,0x18, -0x5e,0x68,0xfb,0x8b,0xe2,0xd2,0xd8,0x26,0xca,0x2f,0x8e,0x13, -0x08,0x4d,0x12,0x42,0x1b,0x72,0xf4,0x51,0x24,0x42,0x07,0x3b, -0x76,0x7e,0x89,0x38,0xc6,0x68,0x5a,0x53,0x12,0xd3,0x57,0x10, -0x67,0x62,0x0a,0x38,0xb1,0x80,0x8f,0xe3,0x55,0xaf,0xb3,0xf3, -0x81,0xac,0x88,0x5f,0xc1,0xb6,0x41,0xb9,0x4d,0xad,0xff,0xae, -0x6d,0xda,0xfe,0xf6,0x36,0x6d,0xff,0x77,0x6d,0xd3,0xf3,0x6f, -0x6f,0xd3,0xf3,0xff,0xae,0x6d,0xfa,0xfe,0xdb,0xdb,0xf4,0xfd, -0x7f,0xcb,0x36,0x49,0x00,0xbd,0x59,0x3f,0xfa,0xa1,0xb6,0x2f, -0xee,0x3c,0x85,0x56,0x5f,0x3c,0x4e,0x71,0x0c,0x19,0x44,0x66, -0x9b,0x4a,0x46,0x88,0x8b,0x1c,0x8b,0xcc,0x1d,0x24,0xc1,0x69, -0x8a,0xe4,0x3c,0xc5,0xa4,0xae,0xec,0x04,0x09,0xb3,0x3f,0xe6, -0x5f,0xa6,0xa5,0x55,0xdb,0x26,0xbc,0xb5,0x86,0x31,0x39,0xe8, -0xc4,0xd1,0xcb,0x18,0xa5,0xbc,0xcb,0x51,0xf6,0x9b,0x17,0x61, -0xd6,0x4e,0x71,0x75,0xb1,0xc9,0xa7,0xc2,0xeb,0xe8,0xbb,0xa5, -0xbe,0xb7,0xd5,0xf7,0x73,0xf5,0x0d,0x63,0x69,0xdb,0x5b,0x3c, -0xdd,0x09,0x8e,0x4e,0xd0,0x09,0x3a,0xc8,0x18,0xb0,0x73,0x87, -0xcf,0x6c,0x65,0xf5,0x20,0x72,0x28,0xe0,0x5f,0x19,0xb2,0x88, -0x54,0x9e,0x42,0x29,0x62,0xa1,0x64,0xa4,0x05,0x39,0x2b,0x08, -0xfb,0xe6,0xc5,0xc0,0x6d,0x11,0x90,0x5c,0xfc,0xc1,0x28,0xea, -0x12,0x66,0x11,0xe9,0x25,0x1b,0x1c,0x96,0xda,0x6d,0xfb,0x6c, -0x14,0x39,0x27,0xb3,0x16,0x8a,0x5f,0xd1,0x50,0xe0,0x12,0x9f, -0x1f,0xab,0x16,0x42,0xbf,0x0e,0x22,0xf4,0x46,0x16,0xc1,0x98, -0x4d,0x6b,0xb0,0x32,0xd1,0xa2,0x55,0x5b,0x2c,0x6a,0x1c,0x72, -0xc8,0x27,0x35,0x7b,0x4c,0x06,0x95,0x0f,0x33,0xfc,0x3b,0xcd, -0xa2,0x06,0xeb,0xca,0x75,0xa2,0x69,0x94,0x3e,0x8b,0xe3,0x8f, -0xf5,0xfa,0xc7,0x22,0xb1,0x45,0xcf,0x1a,0x50,0x5f,0x03,0xd5, -0xe8,0x3a,0x1f,0xd3,0x69,0xf2,0x2c,0x69,0x3c,0x3b,0x75,0x91, -0xdd,0x67,0x8d,0x08,0x5b,0xd3,0x00,0x04,0x6a,0x80,0x41,0xbd, -0x5e,0x9b,0xf2,0x01,0x40,0xed,0x1e,0xa3,0x9f,0xd5,0x30,0x7e, -0x8d,0xe7,0x27,0xaa,0xfe,0xc3,0xb6,0x13,0x53,0xa0,0x58,0xed, -0x96,0xb0,0x88,0xf0,0x0b,0x67,0x52,0x4d,0x61,0xeb,0xe2,0x0b, -0xf3,0x12,0xca,0xca,0xf1,0x40,0x7b,0x99,0x70,0xa6,0x25,0x67, -0x9d,0x66,0x68,0x1e,0xe1,0xce,0x45,0x2d,0x5d,0xcf,0xb8,0x76, -0x47,0x78,0x7b,0x6c,0x5f,0xe2,0xc4,0x11,0xda,0xe9,0xfe,0x18, -0x51,0x2e,0xb2,0x52,0x6b,0x4d,0x62,0x56,0x58,0x8b,0xc4,0x3c, -0xf6,0x85,0x57,0xd5,0x84,0xcb,0x2e,0x47,0xea,0x8d,0xe2,0x21, -0xaa,0x0f,0x15,0x5e,0x29,0xb2,0xeb,0xa9,0xfb,0x65,0xe8,0x3e, -0x9a,0xb7,0x20,0x81,0xc1,0x02,0xff,0xaa,0x50,0xb4,0xb2,0x6c, -0x52,0xd9,0xc8,0x44,0x8e,0xbc,0x62,0xfa,0x69,0x80,0x6a,0xad, -0x62,0x70,0x94,0xa2,0xf0,0x8d,0x83,0x35,0x45,0x9a,0x6a,0x37, -0x52,0xc5,0xa1,0xc5,0xdc,0xec,0x07,0xee,0x14,0x3e,0xbb,0xa9, -0xff,0xcc,0x44,0x49,0x0c,0xad,0xf6,0xae,0xb5,0x4c,0x11,0x97, -0x26,0xb2,0x87,0x36,0xd5,0x63,0xb1,0x71,0x35,0x41,0x11,0xeb, -0xa4,0xd7,0x46,0xe1,0xa1,0xd3,0x32,0x78,0x57,0xb6,0xfe,0x9f, -0xab,0x9c,0x9f,0x04,0x16,0x1d,0xca,0x56,0xa3,0x2b,0x0d,0x38, -0x14,0xb3,0xb9,0xae,0xfa,0xc5,0x3a,0x1a,0x72,0xc5,0xac,0x1d, -0xf3,0xf0,0x29,0xad,0x7f,0x6b,0xeb,0xd2,0x97,0x15,0x60,0x75, -0x0b,0x1c,0x68,0x3c,0xb5,0x3f,0x19,0x5c,0x7c,0x34,0x65,0x2c, -0x5c,0x66,0xe7,0x02,0x49,0x9c,0x2e,0xdc,0x95,0x2d,0x6b,0x2f, -0x8b,0x99,0x61,0x5b,0x48,0x79,0x27,0x79,0x6f,0x3a,0x1e,0x29, -0x17,0x38,0xe2,0xf4,0x86,0x62,0xd5,0x69,0x18,0x5d,0x52,0xd0, -0x8a,0x35,0xb5,0x0f,0x63,0xc5,0xc3,0x9b,0x75,0xef,0x76,0x61, -0xdf,0x42,0xa3,0x15,0x75,0xf2,0x53,0x63,0x3d,0x4c,0xb9,0x82, -0x9d,0xd3,0x6e,0x7a,0xbf,0x2c,0xbe,0xa8,0xb4,0xc9,0xa6,0xb8, -0x59,0x62,0x4b,0xfe,0xce,0xdf,0x82,0x41,0x0b,0x14,0x76,0x1f, -0x55,0x5e,0x52,0xb9,0xc9,0x53,0x76,0x38,0x5d,0x92,0xc1,0x13, -0xb6,0x85,0x8f,0xa9,0xea,0x14,0xd8,0x6c,0x6c,0x08,0x78,0xf2, -0xfc,0x16,0x78,0xeb,0xb3,0xd9,0xd1,0x83,0x90,0xab,0xc7,0x77, -0xd7,0x03,0x15,0x15,0xdc,0xfc,0xb9,0xde,0x44,0x05,0x4c,0xd4, -0x0a,0xbc,0xb0,0xaf,0xb7,0x57,0x56,0x69,0x8a,0x3e,0xf2,0x96, -0xae,0xcb,0x8b,0x06,0x91,0xb8,0x1d,0x36,0xbf,0x11,0xc7,0x44, -0xea,0xe1,0xaa,0x1b,0xa8,0x7b,0xeb,0x86,0x30,0x14,0x44,0x80, -0xb5,0x4b,0xa1,0xd0,0xf3,0x05,0xac,0xeb,0x1c,0xdd,0xb6,0x30, -0xfd,0x52,0xc5,0xe7,0x0f,0x38,0x1e,0x08,0xc2,0x7b,0x87,0x70, -0x47,0x0f,0x96,0x6c,0xf5,0xeb,0x72,0xbb,0xe1,0x77,0xc6,0xfe, -0xc5,0x1c,0x73,0xb1,0xb4,0x37,0xcc,0x6c,0x53,0x7c,0x16,0xbc, -0x9f,0xe0,0x83,0x07,0x1a,0x7f,0x1b,0x4f,0xde,0xe4,0xec,0x77, -0x4b,0x7d,0x6f,0x93,0x2b,0xf3,0xa5,0xb8,0x5e,0x4d,0x4b,0xe3, -0x58,0x31,0x02,0x7a,0xc8,0x60,0xe8,0x47,0xe3,0xd9,0xe0,0xf2, -0x4b,0xaa,0x69,0xf3,0x6a,0x42,0x56,0x79,0x9e,0xdc,0x71,0xff, -0x47,0x7a,0x45,0x35,0xd9,0xe3,0x48,0x6d,0x17,0x12,0x4a,0xef, -0xb1,0xcb,0xd2,0x12,0xaf,0xc2,0xb7,0x71,0xf4,0x6b,0x4b,0x31, -0x03,0x13,0xff,0x16,0x40,0x8e,0x6e,0x86,0x7f,0x22,0x7f,0x3c, -0x7d,0xcb,0x67,0x4a,0xa6,0xfc,0xd1,0xde,0x36,0x17,0x71,0xdd, -0x19,0x2b,0x0f,0xf6,0x8e,0xf9,0x75,0xec,0x57,0x5a,0xe0,0x5c, -0x89,0x40,0x72,0x96,0xa9,0x13,0x0f,0x8f,0x86,0x25,0x4f,0x4a, -0x7c,0x10,0xe2,0xd0,0xc5,0x03,0xa3,0x26,0x5c,0x2d,0x2b,0x8d, -0xf3,0x23,0xa8,0x49,0x17,0x24,0x1d,0xf9,0x48,0x43,0x2e,0x99, -0x78,0x7a,0xf8,0x46,0xc4,0x0e,0x55,0xdf,0x66,0x15,0xb4,0xa4, -0xf0,0x63,0x82,0x0b,0xd0,0x0b,0x46,0xa6,0xa3,0x43,0xe5,0x1e, -0x74,0x30,0x15,0xae,0x95,0xa8,0xb9,0xeb,0x5b,0x26,0xd2,0x8e, -0x25,0x56,0x3a,0xd7,0x3e,0x2d,0x8c,0x75,0xa1,0x83,0xab,0x9a, -0xd5,0x95,0x63,0x49,0x9e,0x99,0x5f,0x5f,0x3e,0x91,0x75,0xc3, -0xb1,0x3c,0x13,0x51,0x03,0xa2,0x32,0xc9,0x06,0xb6,0xb5,0x94, -0x4b,0x62,0xb5,0xbc,0x46,0xc0,0xa8,0x20,0xf4,0xa5,0xb2,0xb4, -0x57,0x0f,0xe9,0x7d,0x74,0x9e,0x26,0x99,0x91,0x0a,0x75,0xf4, -0xab,0x26,0x40,0xdb,0xfb,0x89,0xa3,0x3c,0x9d,0x02,0x45,0x31, -0xf1,0xa6,0x21,0x65,0x70,0x64,0xb1,0x6d,0x0e,0x83,0x6b,0x1a, -0x91,0x70,0xe9,0x36,0x83,0xe9,0x51,0x80,0x86,0xc5,0xfc,0x60, -0x3e,0xbb,0x9d,0xb3,0x52,0x40,0x1b,0x29,0xa6,0x10,0xef,0x5a, -0xe8,0xdc,0x98,0x7a,0xb2,0x08,0x28,0x97,0x5a,0xa1,0xdb,0x9e, -0x1b,0xbc,0x94,0x70,0xc2,0x9c,0x86,0x8a,0xcf,0x07,0x4a,0x53, -0x98,0x93,0x58,0x12,0xef,0x54,0x44,0xc6,0x65,0x21,0xb7,0xdc, -0xd5,0xaa,0x95,0x6a,0x85,0xa5,0x97,0xba,0xd5,0x48,0xd2,0x1b, -0x5e,0x44,0x34,0x49,0x1f,0x86,0x56,0xa2,0x8e,0xb2,0x9c,0x40, -0x19,0x2a,0x3b,0x1f,0x95,0x3c,0x56,0xa1,0x53,0x28,0xa0,0x30, -0x42,0xb3,0xd2,0x28,0xa0,0x3c,0xe8,0xdf,0x24,0xe8,0x4b,0x3f, -0x59,0x92,0x0e,0xdf,0x1e,0xad,0x60,0xd0,0x0f,0x91,0x62,0xd5, -0x8a,0x52,0xa0,0xca,0x8b,0x50,0x2c,0x44,0x21,0xa4,0x40,0x8c, -0x6e,0x7d,0x2c,0xf3,0x4d,0x31,0x3b,0xf1,0xf0,0xca,0xf1,0xb5, -0x08,0xfe,0xab,0x98,0x71,0x55,0xc3,0xd1,0x9d,0x45,0x61,0xcc, -0xcc,0x13,0x56,0x46,0x71,0x10,0xfb,0x0d,0xcb,0x23,0xe9,0xde, -0x7d,0xd3,0xaa,0x58,0x9a,0xc1,0x15,0x8d,0x34,0xae,0x0b,0x9e, -0xbc,0x28,0x16,0x11,0x7a,0x6a,0xf7,0x7f,0x2b,0x3a,0x4f,0x5b, -0x0a,0x8b,0x58,0x7d,0xd3,0x12,0xf8,0xc4,0x15,0xb0,0x7a,0x12, -0x5d,0x4a,0xb7,0x57,0x73,0x10,0x24,0x00,0xd8,0x52,0xe0,0x0f, -0x5a,0x07,0x2a,0x90,0x73,0xb1,0xf4,0x37,0xac,0x86,0x95,0xdb, -0x8e,0x87,0x6b,0x0d,0xa1,0xba,0x69,0xa2,0x01,0xf9,0x6e,0x66, -0x50,0xa1,0xcc,0x51,0x81,0x87,0x96,0x27,0x2a,0x58,0x87,0x74, -0x21,0xcb,0xd1,0xec,0xdf,0xdc,0xca,0xda,0xfc,0x43,0x13,0x2c, -0xc5,0x3f,0x78,0x73,0x8f,0xd7,0xc2,0xda,0xa8,0xe4,0x91,0xdd, -0x24,0x17,0xce,0x33,0x4e,0x4e,0xde,0x66,0xad,0x2d,0x65,0xb0, -0x8a,0x9f,0x4b,0x65,0x37,0xf2,0x6a,0x4a,0xed,0xe3,0xc5,0xaa, -0x28,0xdb,0xe7,0x3a,0x99,0xad,0x74,0x30,0xe5,0xcc,0xd0,0xf8, -0xe7,0xf7,0x58,0x8b,0x2c,0xcf,0xf0,0x59,0xb0,0xc3,0x64,0xfd, -0xd7,0xae,0x72,0x2f,0xe5,0x7c,0x4f,0x39,0xbb,0x90,0xfe,0xf0, -0xc0,0x96,0xad,0x72,0x37,0x25,0x7e,0xa8,0xa4,0x94,0x92,0x45, -0xa8,0x39,0xee,0xbc,0x62,0xd0,0x27,0x85,0xbd,0x9f,0x06,0xfd, -0xd8,0x9b,0x26,0x46,0xb3,0x64,0x62,0xdb,0x96,0x8d,0xef,0x59, -0x43,0xb7,0x64,0x6a,0x0c,0xdd,0xbb,0x49,0x18,0x8b,0x1e,0x0f, -0xce,0x87,0xb8,0x6b,0xb3,0x85,0x27,0xf9,0x5d,0x21,0xea,0x02, -0x3d,0x6e,0xa2,0x9f,0xb4,0x20,0xe2,0xa4,0x37,0x18,0xb2,0x53, -0x48,0xb2,0x1a,0x1a,0xe1,0xdf,0xa8,0x2b,0x59,0xb0,0xdf,0xe9, -0xd3,0xe2,0xd4,0x87,0x5e,0x9f,0x8c,0xbf,0xc0,0xf6,0x72,0xad, -0xe5,0xcc,0xe0,0xe2,0x59,0xc7,0x7f,0xce,0x73,0xf4,0x26,0xdd, -0x75,0xc8,0x8f,0xa7,0xb3,0x97,0x34,0x48,0xc5,0x94,0xe1,0x60, -0x3a,0xcb,0x47,0x88,0xfd,0xbd,0x33,0x86,0x3b,0x98,0x92,0xc3, -0x94,0x1f,0x07,0x24,0x93,0x62,0x4a,0xc1,0x0a,0xff,0x21,0xf6, -0x45,0x4c,0x3d,0x52,0x32,0x5f,0xc7,0x1d,0x65,0x9f,0xdb,0x15, -0x68,0x08,0x8d,0x23,0x21,0x4d,0xc3,0xd0,0x91,0xed,0x90,0x21, -0x39,0xf5,0xc3,0xb3,0x43,0x4b,0xe9,0x51,0x85,0x31,0xb8,0xa4, -0x5d,0x8e,0xcb,0xe8,0xaf,0xba,0x1a,0x1d,0xe6,0xbd,0x09,0xe3, -0x15,0x9e,0xff,0x48,0xf6,0xd0,0x89,0x02,0x9a,0x5a,0x1d,0x0c, -0xf3,0x93,0xf1,0x2b,0x3a,0x6a,0x33,0x64,0xce,0xc8,0x9a,0x5b, -0xf2,0x8e,0xb1,0xdc,0x42,0xda,0x9b,0x02,0xb9,0xcc,0x76,0x26, -0x57,0xfc,0xb2,0x8c,0xb2,0x1b,0x7d,0x30,0x88,0x86,0x03,0xff, -0x0b,0xe9,0xe0,0x4f,0xa4,0xff,0xbd,0xe4,0xca,0xc9,0x25,0x04, -0x39,0x89,0xbd,0x57,0x06,0x86,0x52,0x39,0xda,0x1a,0xe5,0x9f, -0xfc,0xb2,0x2e,0x7d,0xce,0x48,0x90,0x37,0x56,0x43,0x04,0xe1, -0x6c,0xed,0x70,0x7a,0x27,0xa6,0x58,0xb2,0x8f,0xc9,0x60,0x28, -0xac,0x68,0x83,0x3f,0x6d,0xb4,0x26,0x87,0x12,0x81,0xb8,0x4c, -0x45,0xf2,0xa8,0xa6,0x9a,0x24,0x8d,0x09,0x31,0x7d,0x08,0xb2, -0x34,0x14,0xa5,0x5a,0x4a,0xf4,0xe9,0x91,0x54,0x4d,0x09,0xe7, -0x55,0xc2,0x91,0x8e,0xc4,0x2f,0x74,0xc6,0x90,0x2c,0x25,0x52, -0xcd,0x3a,0x97,0xcf,0xcd,0x3d,0x1b,0xa1,0x27,0x9f,0x8d,0xf3, -0x53,0x2d,0x88,0xf1,0x73,0xcd,0xf5,0xc3,0x8a,0x69,0x8a,0x39, -0x4d,0x71,0xde,0x76,0xe2,0x62,0x96,0xaa,0xea,0x4d,0x69,0x76, -0xd3,0x97,0xb3,0x1c,0xfe,0x00,0x1b,0x50,0x3d,0xb2,0x34,0xdc, -0x52,0x4f,0x2f,0x14,0xdc,0x03,0xd2,0xb5,0x3d,0x25,0xca,0x36, -0x7b,0xf2,0x42,0xc1,0xfa,0x48,0x1e,0xdb,0x21,0x41,0xce,0xaa, -0xc9,0xde,0x01,0x88,0xe8,0x2b,0xc2,0x87,0x56,0xa8,0x7d,0x36, -0x95,0x69,0x63,0x7b,0xd6,0xc8,0x47,0x76,0x7f,0x39,0x4a,0x5d, -0x65,0xf8,0x12,0x39,0xd5,0x44,0x8f,0xf0,0x53,0xf8,0x87,0xfe, -0xf6,0x52,0x07,0x3b,0xff,0x25,0x5d,0x5b,0x0b,0x11,0x5b,0x8a, -0x51,0x74,0x27,0x5c,0x58,0x98,0xf2,0x88,0x8b,0x83,0x64,0x45, -0xce,0x23,0x6b,0xf3,0x88,0x92,0x63,0x64,0xc3,0xb0,0x00,0x09, -0x7a,0x4d,0xed,0x84,0x0c,0x08,0x0f,0x51,0x1a,0x0f,0xfb,0xf8, -0x43,0x2d,0x4a,0xee,0x2d,0x60,0xbe,0x09,0x42,0x53,0x36,0x97, -0xb1,0x3d,0x4a,0xb7,0xae,0xc7,0x6a,0x16,0xf3,0x7a,0x1d,0xda, -0xe2,0xf4,0x6b,0xc4,0x7a,0xe4,0x64,0x70,0x85,0xfb,0x5a,0xc0, -0xcd,0x57,0xfa,0xf4,0xfa,0x7d,0x71,0xf2,0x15,0xcb,0x46,0xbc, -0xab,0x72,0xf2,0x40,0x1c,0x91,0x2f,0xea,0x98,0xbe,0xed,0x18, -0x38,0x53,0x74,0xac,0xd5,0x87,0xaa,0x6d,0x95,0x63,0x06,0xfc, -0x5d,0xa0,0xb8,0xd7,0xfd,0x16,0x82,0xf6,0x47,0xfb,0x50,0x3d, -0x42,0x70,0xbb,0xe3,0xe1,0x30,0xa7,0xfc,0x9e,0xf4,0x50,0xad, -0xc8,0x0d,0x9a,0x72,0xde,0x58,0x28,0x40,0x94,0x01,0xb5,0x70, -0x13,0x81,0xab,0x40,0xb9,0xdf,0x7b,0xf5,0x70,0xd6,0x09,0x82, -0x44,0xf1,0x10,0xf8,0x31,0xa7,0x57,0x0f,0x83,0xa9,0x28,0x11, -0xdb,0x62,0x49,0x42,0xb1,0x74,0x98,0xfd,0x76,0xf0,0x31,0x57, -0x09,0xf7,0xdc,0x6d,0xf3,0x7c,0x0d,0x5d,0xaf,0xd0,0x9b,0xaa, -0x21,0xe5,0x43,0x05,0x1d,0x49,0x0a,0x13,0xb0,0xa3,0xe5,0x62, -0xd4,0x66,0xb0,0xbe,0xec,0xb6,0xc4,0x4a,0xe5,0xcb,0x7c,0x46, -0x3a,0x5a,0x8d,0x46,0x62,0x1c,0x76,0x99,0xad,0xd4,0x3a,0x3b, -0x73,0x70,0xd6,0x50,0xb7,0x5f,0x72,0x91,0x51,0xa1,0x10,0x86, -0x29,0x56,0xe3,0x11,0x70,0xc5,0x12,0x67,0x69,0x05,0x6f,0x43, -0x5d,0xb4,0x1c,0x38,0x5a,0x00,0xa7,0x83,0x2e,0xd7,0x65,0x43, -0x0f,0xd5,0x86,0x9e,0x90,0x54,0x56,0xb1,0x50,0x52,0xd1,0x5d, -0xdc,0xd9,0x55,0xfd,0x75,0xe0,0x14,0x1b,0x3d,0xca,0x0f,0xf7, -0xd1,0x16,0xb5,0x03,0xc6,0x11,0xd7,0x73,0x37,0x4a,0xae,0xe7, -0x2a,0x4e,0xeb,0xa1,0x5e,0x57,0x5c,0xf7,0x55,0x45,0xd6,0x8e, -0x6b,0x1a,0x5e,0x20,0xe8,0x9e,0xc0,0xf0,0x83,0x7d,0x41,0x37, -0x0e,0x41,0x76,0x58,0xc9,0x5c,0xa3,0x1e,0xce,0x59,0x6e,0x30, -0xe9,0x42,0xbb,0xd2,0x2f,0xfc,0x28,0xde,0x17,0xb3,0xea,0xee, -0xb0,0x55,0x24,0x0f,0xb5,0xb1,0x5e,0x5f,0x7b,0xa4,0xb7,0x5c, -0xbd,0x1b,0x1b,0xd6,0xa9,0x73,0x80,0x3c,0x4c,0x5c,0x7a,0xd8, -0xa3,0x46,0xda,0x42,0xd4,0x63,0x6c,0xe3,0xca,0xad,0x0b,0x2d, -0xdd,0xda,0x84,0xe5,0xca,0x49,0xbe,0x63,0x79,0x89,0x65,0x08, -0x6e,0x7a,0xba,0x5e,0xe5,0x07,0x22,0x4b,0xb1,0xbf,0xcb,0x51, -0x2e,0x93,0x33,0xec,0x5a,0x40,0x05,0x48,0xcd,0xb1,0x0b,0x2b, -0x90,0x91,0x3a,0x5e,0x30,0x9d,0xd9,0xa2,0xcc,0x03,0xf4,0xda, -0xd6,0x87,0x51,0x62,0x13,0xb8,0x92,0x4f,0x95,0xd8,0x1d,0x37, -0x0c,0xaf,0x0a,0x79,0x73,0x53,0x34,0xcb,0x49,0xd9,0xed,0x63, -0x09,0x58,0x0c,0x8d,0x5b,0x2f,0x85,0x43,0xf4,0x07,0xe8,0x48, -0x80,0x1f,0x2a,0xb1,0x15,0x5d,0x7c,0xcb,0xf8,0xa6,0xff,0x19, -0x7f,0xf6,0xa7,0x57,0xe6,0x41,0xc9,0xec,0x69,0x12,0x99,0x99, -0xc1,0x96,0x0b,0x76,0x4f,0x86,0xcd,0x73,0x73,0x47,0x12,0x22, -0x6f,0xc7,0x74,0x0b,0xa5,0xa4,0xf5,0x48,0x1a,0x02,0x4b,0x66, -0x7f,0x4c,0x17,0x1f,0x84,0x21,0x99,0xe7,0x65,0xfc,0x32,0x86, -0xd2,0xf6,0x68,0x38,0x43,0x3c,0x74,0xde,0xc9,0xb9,0xf8,0x46, -0x35,0x56,0x99,0x44,0x95,0x51,0x6d,0xf8,0xcb,0x5e,0x27,0x61, -0xbd,0x85,0x5a,0x61,0xe6,0x78,0x81,0xcd,0x82,0x73,0xab,0x16, -0x19,0x48,0x13,0x36,0x69,0xe3,0x2c,0x06,0x62,0x4b,0x67,0x83, -0x51,0xa1,0x54,0x97,0xe7,0x94,0x13,0xbb,0xdc,0x72,0xf7,0x46, -0x9e,0x42,0xe8,0x0e,0x34,0x96,0xaa,0xc8,0x43,0x0e,0xd9,0x46, -0x93,0x34,0xfc,0xc4,0x49,0xc7,0x48,0x35,0xf3,0x5f,0x3a,0xf4, -0xbe,0x53,0xfc,0x32,0xd0,0xb0,0x26,0x69,0xc4,0x76,0xe9,0xbc, -0x88,0x20,0x87,0x89,0xa5,0x78,0xcc,0x4e,0x06,0x86,0x7a,0xef, -0x64,0xf7,0x6b,0x43,0x04,0x08,0xa7,0xaf,0xf4,0xf8,0x96,0xf1, -0xf5,0x67,0xae,0x6e,0x58,0xc3,0xbf,0xd8,0x53,0x6c,0x46,0x80, -0x93,0x2e,0x47,0x0a,0x9f,0x2c,0xcb,0xb4,0xd4,0x25,0x67,0x97, -0x44,0x80,0xb6,0x95,0xe6,0xd7,0x2f,0x80,0x2c,0xf0,0x01,0x29, -0x91,0x5b,0xf6,0xfd,0x06,0x04,0x1d,0x55,0x9c,0x72,0x6c,0x17, -0x64,0x87,0x62,0x14,0x49,0x0d,0x4c,0x9c,0x99,0x92,0xcf,0x04, -0x41,0xd4,0x27,0x88,0x40,0x04,0xac,0x45,0x0d,0x1f,0xd5,0xc4, -0x9d,0xd0,0x62,0xe1,0xc3,0xca,0xf7,0x68,0xea,0x62,0x05,0x68, -0x23,0x8b,0xda,0xb2,0x2e,0x20,0x14,0x71,0x9a,0xca,0x2d,0x6e, -0x40,0x12,0x73,0x07,0x95,0x44,0xfd,0x56,0xc2,0x9c,0xef,0x48, -0x18,0x64,0xf2,0x28,0xd5,0xd1,0xc3,0x67,0x7a,0x21,0xe8,0x09, -0xce,0xc9,0xdc,0x8b,0x85,0x44,0xa2,0xfa,0x36,0x11,0x7f,0xbd, -0xee,0xb3,0x29,0x59,0x3e,0x21,0xda,0x6c,0x97,0x73,0x97,0x60, -0xeb,0x42,0xed,0xc4,0xce,0x2c,0x97,0xce,0x7b,0x18,0x79,0x81, -0x60,0x53,0x47,0xf4,0xd6,0x0e,0x30,0x25,0xfa,0x40,0x83,0x2b, -0xb3,0xe1,0xc6,0x86,0x6c,0xa1,0xf5,0xc6,0xb6,0x74,0x26,0x43, -0xc6,0x86,0x61,0x5e,0xcb,0x7e,0xda,0x19,0x4a,0xaf,0x91,0xde, -0x38,0x76,0x05,0xfa,0x06,0x46,0xd5,0x8f,0x7e,0xb3,0xb6,0x73, -0x3e,0x26,0xf3,0x3b,0x6b,0x67,0xcc,0xf3,0x60,0xea,0xd4,0x2e, -0xd1,0x6b,0x30,0x2e,0x50,0x68,0xb0,0x07,0xfb,0xbb,0xf6,0x6b, -0x55,0x36,0xad,0x91,0x67,0x59,0xe2,0x30,0x3c,0x24,0x76,0x30, -0x12,0xfb,0x8a,0x82,0x50,0x5e,0x2c,0x4a,0x3c,0x83,0x85,0x1a, -0x8d,0x7c,0xbb,0xc0,0xbc,0x02,0xe6,0x52,0x4c,0x13,0x9e,0x12, -0x3f,0x36,0xbc,0xc8,0x9f,0xf9,0x10,0x44,0x33,0xe8,0xe0,0x48, -0x62,0x8d,0xd8,0x22,0xf6,0x6a,0xe1,0xd0,0xc4,0x39,0x95,0xf3, -0x67,0x39,0x1c,0x10,0x45,0x1e,0x6d,0x84,0x5d,0x0a,0x79,0x97, -0x6b,0xea,0xe8,0x85,0xfc,0x8a,0x17,0x28,0x2b,0x93,0x4a,0xe3, -0x8a,0xa4,0xf2,0x31,0x51,0x52,0x80,0x40,0xdb,0xbf,0x6a,0x08, -0x94,0x54,0x3e,0x55,0xf2,0xc7,0x54,0x3e,0x0e,0xad,0x8e,0x95, -0x22,0xab,0xcf,0xa7,0x02,0x5c,0xca,0x71,0x1e,0x92,0x8a,0xac, -0xde,0x2f,0x57,0xe0,0x14,0x9e,0x96,0xfd,0x53,0x4e,0xc7,0x70, -0xdc,0x71,0x19,0x4a,0xb5,0xc2,0xd4,0xa4,0xf8,0x6a,0xd4,0x1f, -0xe4,0x4c,0x12,0xde,0x6e,0xb9,0xc7,0xa7,0x5e,0x2d,0x28,0x04, -0x51,0x75,0x76,0x14,0xfa,0x22,0x58,0x2f,0x91,0x3a,0xa4,0x5b, -0x33,0xfa,0x7c,0x1b,0xa1,0x28,0xf1,0xe1,0x92,0xf5,0x7a,0xa5, -0x97,0x83,0xa4,0x7c,0x96,0xe7,0x1f,0x35,0x22,0xf2,0x4a,0x0e, -0x28,0xa2,0x5f,0x2d,0x28,0x84,0xc5,0xe9,0x3d,0x60,0xa9,0xa0, -0x3b,0xf9,0xaf,0x05,0x52,0x03,0x65,0x76,0x74,0x4f,0x6b,0x3f, -0xc3,0x79,0x78,0x62,0xca,0x0d,0xf0,0x7d,0xc0,0x01,0xb0,0x8e, -0xca,0x16,0x3b,0x58,0x72,0x6c,0x58,0x75,0x9b,0x41,0x37,0x46, -0xeb,0x7a,0x23,0x4a,0xc6,0xca,0xfd,0x4e,0x94,0xf6,0x93,0x10, -0xd5,0x7f,0xeb,0x0e,0x8f,0x8a,0x87,0x9c,0xa7,0x98,0xde,0xd5, -0x36,0xc3,0x7d,0x66,0xef,0x5f,0xb1,0x90,0x3d,0x2b,0x00,0xc5, -0x65,0x32,0x8c,0x92,0x25,0xc9,0x62,0xa2,0xf7,0xd8,0x61,0xc6, -0x53,0x6b,0xec,0x2f,0x8c,0xf6,0x8d,0xe8,0x7a,0xf2,0xbe,0x19, -0xcc,0xc2,0xa6,0x56,0xb5,0xd0,0x0c,0xac,0xbc,0x99,0xdf,0xdc, -0xce,0x48,0x20,0x56,0x67,0x3e,0xd3,0xd9,0xf8,0x16,0xb7,0x10, -0xbd,0x2b,0x71,0xba,0xdf,0x32,0xe4,0x14,0x22,0x23,0xb4,0x52, -0xea,0x3f,0x5e,0xc9,0x68,0x40,0xe4,0x99,0x6a,0xa1,0x94,0x66, -0xac,0x25,0x80,0x5b,0x4b,0x13,0x3a,0x58,0xd1,0xb9,0xd9,0xfd, -0x4b,0xd1,0x1f,0x05,0x15,0x2a,0x46,0xa2,0x6d,0x30,0xd7,0x0b, -0x3b,0xf8,0xce,0x96,0xdd,0xa5,0xc7,0xa7,0x94,0xa1,0xeb,0x0d, -0xd9,0x9b,0x96,0x13,0x60,0x0b,0xe3,0x62,0x4f,0x41,0x0a,0xc3, -0xb7,0x58,0x50,0xd7,0x18,0x71,0xe5,0xc3,0x6b,0x2e,0xb5,0x37, -0x73,0xcf,0x72,0xa5,0x13,0x0b,0x23,0x1d,0x0a,0x9f,0xb2,0x1f, -0x1f,0x14,0x07,0x0f,0xf7,0xd7,0x48,0xd8,0x55,0x09,0xf2,0xf4, -0x96,0x0e,0x19,0x75,0xeb,0x92,0x62,0x27,0x5b,0x8d,0x6f,0x42, -0xf2,0x91,0xa9,0x45,0x3b,0xbc,0x02,0xa6,0xa8,0x00,0x7e,0x91, -0xc7,0x03,0x83,0xc6,0x7e,0x61,0x73,0xe8,0xe4,0xe5,0xa9,0x6f, -0x68,0x41,0xfa,0xe0,0xc3,0xa6,0x76,0x50,0xb1,0xf1,0x8b,0x5e, -0x15,0xed,0x0d,0xcb,0x64,0xa7,0x77,0x1e,0xe1,0x3e,0x82,0xf6, -0x3e,0xf6,0xfb,0x41,0xb2,0x93,0x3c,0xff,0x3a,0x42,0xd2,0x82, -0x58,0x15,0x9d,0xb8,0x0d,0x9c,0xa3,0x35,0x2f,0xf0,0x95,0xa8, -0x8d,0x9f,0xef,0x5a,0x8a,0x1a,0x3e,0x46,0x4c,0xc3,0x22,0x1d, -0x0d,0x9f,0x42,0x3f,0x4f,0x21,0x91,0xe1,0xe3,0xd4,0xf1,0xff, -0x1d,0xe9,0x38,0x20,0xd3,0x25,0x9d,0x1d,0xfa,0x95,0xc4,0x5f, -0x87,0x94,0x5d,0x01,0x05,0xaa,0xc9,0xc7,0x17,0xf9,0xcb,0x7c, -0x98,0xc3,0x9c,0x2b,0xf9,0xfe,0x3f,0xde,0xdd,0x7b,0xbf,0x7b, -0xf0,0xee,0x64,0xef,0xcf,0x27,0xc7,0x99,0x0e,0xb4,0x45,0xbf, -0x82,0x3d,0xa6,0x4c,0x86,0x64,0xfa,0x14,0xfb,0x36,0x3b,0x8d, -0xf0,0xf8,0x22,0xea,0x1a,0x95,0xfa,0xe3,0xb0,0x77,0xf1,0x91, -0x53,0xbb,0x2c,0x47,0x56,0x16,0x7d,0xfa,0x3d,0x69,0x65,0x71, -0xaf,0xd7,0x54,0x4a,0x93,0x5b,0xbc,0x4a,0x94,0xbe,0xa9,0x56, -0x5f,0xbc,0x5c,0xab,0x4b,0x2b,0xdc,0xdf,0x0e,0xe9,0x8a,0x6f, -0xbd,0x3f,0xbe,0x98,0xb3,0x73,0xbe,0xf0,0x4a,0x17,0xdd,0x6f, -0xfe,0x34,0x1b,0x0c,0xa7,0xfa,0xde,0x16,0xca,0x18,0x57,0xc2, -0xa8,0x6b,0x5c,0x97,0x3b,0xd0,0x1d,0x13,0x37,0xb2,0x37,0xbc, -0x93,0x11,0x87,0xc9,0x3f,0x79,0xaf,0x9d,0x3c,0x36,0xb0,0x5f, -0x96,0x1c,0x1d,0x0f,0xa7,0x39,0x98,0x1e,0xc3,0x5c,0x3d,0xa0, -0x03,0xf9,0xd8,0x97,0x4c,0xd7,0xd6,0x5c,0xa0,0x89,0x6e,0xd3, -0xc4,0xae,0xf4,0x8d,0x3d,0x15,0xf7,0xd7,0x7d,0xa4,0xef,0x24, -0x9d,0xb0,0xc3,0xae,0x3d,0x7e,0xfc,0x72,0x38,0x86,0x69,0xf6, -0x45,0xf9,0x0b,0x37,0x23,0x8f,0x57,0xa6,0xea,0xb7,0x3a,0x03, -0xe8,0xd5,0xd8,0xef,0x98,0xc9,0xa3,0x98,0xf8,0x09,0x41,0x9b, -0xda,0x38,0xa9,0x90,0x93,0x8c,0xb2,0xaa,0x71,0xb6,0xfc,0x64, -0xe4,0x59,0x89,0xeb,0x94,0xaa,0xfc,0xa8,0xa6,0xea,0x9b,0x9a, -0xdc,0xbb,0x8a,0xb6,0xbc,0xa5,0x77,0x89,0x4b,0x2a,0x6b,0x76, -0x63,0xfd,0xc4,0x9a,0x5d,0xfe,0x95,0x35,0xb7,0xa4,0x66,0x4b, -0x86,0x1c,0xaf,0x9f,0x51,0x8d,0x70,0x27,0x9a,0xbf,0x1e,0x0f, -0x61,0xa2,0x9e,0x7c,0xb9,0xcd,0xc9,0x79,0xa1,0x98,0x06,0x77, -0x91,0x9e,0xac,0x67,0x93,0x39,0xb0,0x3c,0x39,0x69,0xb5,0xa2, -0xf0,0x7c,0x84,0x9a,0x15,0x27,0x2a,0xa9,0xe2,0x15,0x88,0x2e, -0xe9,0x75,0x53,0xb0,0x36,0x98,0x0c,0xbe,0x2a,0x75,0xcf,0x89, -0x2c,0x87,0xb0,0x31,0x55,0xc9,0xfc,0x72,0xff,0xe0,0xb2,0xaa, -0xaa,0x55,0x68,0xa1,0xff,0xdd,0x4a,0x58,0x0f,0xbc,0x5e,0x79, -0x00,0x98,0xa2,0x2c,0x0d,0x57,0x99,0xc1,0xc3,0xec,0x3b,0x53, -0x7a,0x5f,0x62,0xc2,0xae,0xb3,0x7e,0x9e,0xac,0x4b,0x9c,0xf3, -0x2f,0x84,0x52,0xe1,0x50,0x9a,0x7c,0x42,0x75,0x38,0x8d,0x9d, -0x3d,0xa5,0xa4,0x1d,0x89,0xab,0x8a,0x2a,0x6d,0x75,0x01,0xa6, -0x17,0xfe,0xd4,0x42,0x8c,0x91,0x00,0x33,0xd8,0x99,0xcd,0x50, -0x3e,0xa3,0x47,0xf1,0x8c,0x52,0xad,0xc7,0x67,0x6c,0x78,0x2c, -0x31,0x10,0xb3,0xed,0x92,0xbf,0x59,0xdb,0xe5,0x0f,0x32,0xcb, -0xdb,0xae,0x11,0xba,0x72,0x4a,0x51,0x6e,0x06,0x22,0x22,0xef, -0x7c,0x75,0xfc,0x62,0xa1,0x43,0xfa,0xc5,0x2f,0x7a,0xaa,0x2f, -0x64,0x4e,0x2a,0x08,0xa4,0xad,0x6e,0x0e,0xd7,0xec,0xf1,0x5f, -0x50,0xa8,0xa2,0xc5,0x98,0x6d,0x55,0x73,0x47,0xe3,0xd1,0x06, -0xc3,0xf3,0x2d,0xef,0x71,0x9b,0x47,0x33,0x58,0x7b,0x7e,0x99, -0x0f,0x28,0xb1,0x27,0x56,0xda,0xd3,0xa0,0x1b,0xb8,0xe9,0x4a, -0xe9,0xef,0x93,0x1e,0xbb,0x70,0x52,0xf8,0xd1,0x17,0x02,0x85, -0xa9,0xfb,0xe5,0x3c,0x17,0xca,0x51,0x9a,0x9f,0x3e,0x52,0x7b, -0x35,0xd3,0xb0,0x78,0x56,0xa2,0x53,0xce,0x8e,0xce,0xbe,0x82, -0x1a,0x53,0x9d,0x27,0x78,0x29,0x29,0x71,0x4c,0x86,0x21,0x36, -0x7e,0xfc,0x74,0xfc,0x62,0xa1,0x43,0x15,0xe3,0x17,0x00,0xa9, -0x68,0x51,0xfb,0x1f,0x26,0x6f,0x55,0xb8,0x5e,0x5f,0xd5,0x59, -0x7a,0x0a,0x54,0x60,0xb1,0xa2,0xa3,0xc8,0x06,0x34,0x32,0x98, -0x2c,0x90,0x42,0x9a,0x47,0x7b,0xc7,0x07,0x3f,0x1d,0x41,0xcc, -0x4f,0x47,0x6f,0x13,0x36,0xc2,0xfe,0xc0,0x1a,0x14,0x74,0x41, -0x65,0x1f,0x54,0x90,0xe8,0x68,0x9a,0x83,0xd0,0x35,0x04,0x2a, -0xfd,0x71,0x38,0xbe,0xf8,0x08,0xad,0x19,0xc2,0x66,0x00,0x09, -0xcf,0xb2,0xf5,0x1a,0x3a,0x42,0x20,0x77,0xe4,0x64,0xdc,0x8d, -0x2b,0xad,0x9d,0x7f,0x21,0x38,0x56,0xb2,0xaa,0xdd,0x12,0x0e, -0xcd,0x5a,0x0d,0x50,0x15,0x0a,0x0d,0x1a,0xae,0x0e,0x89,0x97, -0x2b,0x5a,0xfb,0xfa,0x64,0xff,0xad,0x45,0x1b,0x7d,0xdb,0x1e, -0xf7,0xd0,0x33,0xe7,0xdf,0x41,0x5c,0x0b,0x5a,0x56,0xd1,0x8a, -0xf9,0x68,0xda,0xbb,0x54,0x33,0x0d,0xd1,0x87,0xa9,0x36,0x07, -0x81,0x98,0x6c,0x70,0x60,0xaa,0x9e,0x65,0x14,0xb6,0xec,0x25, -0xe2,0x7b,0xaa,0xa0,0xc6,0x92,0x01,0x91,0x7f,0x6a,0x75,0x6d, -0x27,0xe8,0x20,0xf9,0xc5,0xd1,0xfa,0x54,0xea,0x45,0x5b,0xc5, -0x71,0x88,0x88,0xcf,0x8a,0xb7,0x1a,0x3a,0x2b,0x5f,0xa6,0x57, -0xcf,0xce,0xac,0x62,0x19,0x4d,0x0c,0x93,0x75,0xe6,0x1f,0x84, -0x09,0x9d,0x97,0x46,0xa1,0x5b,0x55,0xbe,0xba,0x26,0x0b,0x35, -0x04,0xb2,0x7b,0x7c,0xfc,0x0f,0xc3,0x00,0x42,0xfa,0x87,0x61, -0xfc,0xe1,0x1f,0x47,0x43,0xcf,0xc1,0x4a,0x60,0xab,0x70,0x4f, -0xcc,0xbd,0xac,0xb6,0xa9,0xfc,0x1a,0xcf,0xf9,0x52,0xff,0x69, -0x84,0x2b,0xa7,0xf2,0xbb,0x2c,0xee,0x5e,0x0a,0xaa,0x9a,0xb9, -0xf8,0xc0,0x04,0x21,0x8e,0x24,0x81,0xbc,0xe8,0x13,0xf3,0x51, -0x41,0xdf,0x41,0x58,0x13,0xe9,0x5e,0x22,0x0a,0x02,0xbd,0x53, -0xb8,0xd4,0x22,0xbd,0x9a,0xf3,0x5a,0x90,0x17,0x25,0x4b,0x2f, -0xca,0xab,0x7c,0xc4,0xb9,0xa4,0x0a,0xe0,0x9c,0xd3,0x41,0x6e, -0x55,0xba,0x39,0x33,0x7b,0xa1,0xb6,0x21,0x74,0x8d,0xac,0x53, -0xe8,0x8e,0x44,0x85,0xd7,0x02,0xa3,0x2c,0xbf,0x7d,0xa1,0xd3, -0xaa,0xd6,0x60,0x5a,0x48,0x3f,0xe2,0xe3,0x60,0x64,0x40,0x17, -0x33,0xbb,0x8c,0xce,0x7b,0xc3,0xda,0x1e,0x7a,0x5f,0xc1,0xd9, -0xdb,0x1f,0xe7,0x6c,0x1a,0x68,0x3a,0xbf,0xbd,0x1d,0x4f,0x66, -0xb5,0x37,0x74,0xc3,0x9c,0xcf,0xd0,0x75,0xdf,0x70,0x3c,0xc9, -0x27,0xb5,0x3b,0xd8,0x54,0xe3,0x88,0xfc,0x50,0xfb,0x1d,0xce, -0x6a,0x86,0x5a,0x43,0x9f,0xe6,0xc0,0xf4,0xfe,0x32,0x9e,0x93, -0xb1,0xe3,0xcb,0xc1,0x67,0x12,0xcf,0x90,0x3d,0xa2,0x49,0x56, -0x64,0x0c,0xd7,0x79,0x0d,0xeb,0xab,0xfd,0xb0,0x06,0xa8,0x22, -0xdf,0x23,0x56,0xf3,0x82,0x64,0x01,0x4c,0x1b,0xdf,0xa2,0xf7, -0x81,0x2f,0xc0,0x6b,0x6b,0x38,0xf7,0x5c,0xb7,0x00,0xd8,0xe3, -0x1c,0x33,0xcf,0x6e,0xd3,0xcd,0x4d,0x88,0x9d,0x5a,0x57,0xc7, -0x1f,0xa6,0xcd,0xf1,0xe4,0x6a,0x13,0x50,0xdf,0xc4,0x23,0xf9, -0x29,0xf2,0xe8,0xf1,0x04,0x70,0x99,0x20,0xc3,0x61,0x6f,0x98, -0x80,0xaa,0xe5,0x6c,0x90,0x9e,0xd1,0x45,0xa3,0x26,0xd1,0xa4, -0x0d,0xd1,0xb0,0x29,0x2a,0x79,0x54,0x55,0xf6,0x63,0x84,0x28, -0x30,0xa3,0x50,0x70,0xa6,0xc7,0xd5,0x46,0x52,0x06,0x4f,0xcf, -0x61,0x1e,0x45,0xe7,0x98,0xcd,0x8a,0xc8,0x41,0x1e,0x89,0xb4, -0x64,0xb8,0x58,0xc4,0xba,0xca,0x02,0x74,0x2f,0xf8,0x7f,0xb1, -0x96,0x23,0x0b,0x2f,0x6f,0x83,0x8a,0xc8,0x3d,0x13,0xea,0xc6, -0x51,0x2c,0x91,0x29,0xc0,0x0c,0x81,0xf0,0x61,0xb8,0x5c,0x0c, -0x4d,0xb1,0x05,0xea,0x12,0xc1,0xbd,0xda,0xa2,0x24,0x6b,0x9d, -0x09,0x0d,0x56,0x50,0xd8,0xaa,0x8b,0x75,0x38,0xbc,0xc2,0x28, -0xbc,0x7d,0xb3,0x1e,0x34,0x85,0x2b,0xe7,0x72,0x05,0x9b,0xee, -0x7c,0xd8,0x41,0x49,0x99,0xc2,0x5b,0x31,0x90,0x42,0xb7,0x18, -0xdd,0x5b,0xf2,0xed,0x14,0x59,0x99,0x6a,0x9a,0xf6,0xf9,0xaf, -0x26,0x03,0xf5,0xee,0x66,0x34,0xbf,0x11,0x45,0x0d,0x7f,0xf6, -0x47,0x6e,0xc6,0xbc,0xa3,0x3d,0x4a,0x41,0xc2,0x39,0xbd,0x80, -0xaf,0xe1,0x2e,0xdd,0x16,0x10,0x6e,0xef,0x7b,0xd3,0xf7,0x51, -0x83,0xf2,0x27,0xdd,0xac,0x70,0xcd,0xa0,0x7b,0x50,0xd5,0x23, -0x57,0x13,0x05,0x70,0xd0,0xa6,0xf7,0xc2,0x9f,0xab,0x20,0x86, -0x03,0xae,0x3a,0xd3,0xc3,0x95,0xc7,0x1b,0x45,0xc0,0x04,0x74, -0x05,0x9e,0x21,0x54,0xbb,0x49,0x2a,0x81,0x5c,0x12,0x15,0x85, -0xfc,0x99,0x5f,0xd0,0xaf,0x7a,0xfe,0x6e,0x9d,0x0b,0x6b,0x26, -0xea,0xb9,0x26,0xa7,0x7a,0xb6,0x29,0x0f,0x8c,0x73,0x88,0x9d, -0xb0,0xb3,0x7b,0x43,0xbb,0x6c,0x3e,0x9f,0x3c,0x66,0xde,0x84, -0xea,0x2e,0xbd,0x51,0x7f,0x32,0x1e,0xf4,0x51,0x71,0x2b,0x8e, -0x37,0x25,0x54,0x8b,0xcf,0xfa,0x8d,0x64,0xb3,0x99,0x7f,0xce, -0x2f,0x62,0x3f,0x68,0xb6,0x9a,0xe6,0xa8,0x77,0x37,0xb8,0x42, -0xcf,0x81,0xe8,0xb6,0x37,0x69,0x82,0xb8,0x32,0xd9,0xb9,0x22, -0x1d,0x09,0x3c,0xc0,0x4c,0x4e,0x5b,0xb0,0x64,0x9d,0x8f,0x3f, -0xe7,0x79,0xb6,0xf9,0x23,0xfe,0x6c,0x0e,0x9a,0x33,0xbc,0xaa, -0x79,0x1c,0x80,0xb1,0x0d,0x08,0xd8,0x38,0x79,0x07,0xe6,0xc6, -0x1c,0xe5,0x57,0xf9,0xe7,0x6c,0xf3,0xe7,0x78,0x7f,0xfc,0xf7, -0xc5,0xa7,0xfc,0xfc,0xe3,0x60,0xb6,0x38,0x58,0xdc,0x4c,0x93, -0xb8,0x93,0x9d,0xee,0x6c,0xfc,0xb5,0x9b,0x6c,0x42,0xd5,0xfd, -0x2f,0xc7,0xb3,0x2f,0xc3,0x3c,0x73,0x1c,0x10,0xa3,0xea,0xf5, -0x20,0xd8,0x9c,0x62,0x16,0x20,0xf7,0xde,0x68,0x3a,0xa0,0x0b, -0x6b,0xbc,0xe1,0x00,0xa1,0x88,0x79,0xde,0x94,0x1c,0x93,0x5d, -0xc6,0x0e,0x98,0x7f,0xef,0x8b,0xe6,0xc3,0x90,0x73,0xfb,0x24, -0x7b,0x88,0x91,0x29,0x24,0xb9,0xf7,0x30,0x2f,0xea,0xa6,0xaa, -0x91,0xe0,0x33,0x25,0x7c,0xf7,0x1c,0xc4,0xea,0x80,0x77,0xb3, -0xd7,0xc2,0x9d,0xf8,0x4f,0xb7,0xb0,0xa7,0x20,0xda,0x4b,0x1a, -0x55,0xd9,0x5a,0x89,0x9c,0x85,0xe8,0x44,0x60,0x7e,0x01,0xfc, -0xe8,0xbf,0xa8,0xb7,0x0e,0x6e,0x7b,0x17,0xc0,0xc5,0x22,0x8d, -0x7f,0xbd,0x1e,0x71,0x57,0xe2,0x4d,0x9d,0xee,0x8f,0x35,0x24, -0x76,0x1b,0x0e,0x8a,0x2c,0x16,0x1a,0x78,0x23,0x3a,0xa9,0xce, -0x96,0x04,0x1d,0x0a,0xe0,0x5c,0xf0,0x41,0x68,0x3b,0x95,0xb9, -0x12,0xb3,0x26,0x04,0x8a,0x9b,0x70,0x87,0x66,0xbd,0xee,0xeb, -0xc0,0x6b,0x38,0xd5,0x80,0xc1,0x54,0xf6,0x0a,0x15,0x23,0xdf, -0xe4,0x26,0x7b,0xc4,0x03,0x5c,0x9f,0x50,0xd2,0x21,0xe9,0x5c, -0x67,0xdc,0xc3,0x84,0x45,0xd3,0x0f,0xe9,0x5a,0xbc,0x66,0x49, -0x5d,0xa2,0x16,0x8b,0x62,0x0c,0x5d,0xfb,0x1d,0xa3,0x1d,0xc0, -0xc5,0xe2,0xfb,0x17,0xae,0x61,0x34,0x6f,0x12,0x03,0xe2,0xfc, -0x35,0x2b,0xb5,0xa5,0xd1,0x78,0xe4,0x43,0xd8,0x21,0x02,0x08, -0xe4,0xaf,0x35,0x87,0x9d,0x16,0x5e,0x16,0x8b,0xca,0xe8,0x17, -0xff,0x4e,0x60,0xf7,0x90,0x03,0xa8,0xfb,0xdb,0x3b,0x62,0x17, -0x78,0x28,0x41,0xee,0x2c,0x60,0x0b,0x4c,0x51,0xf5,0xfa,0xef, -0x60,0x2f,0x8c,0x22,0x96,0xf5,0x0b,0xc4,0x0a,0xb6,0x3f,0x59, -0x91,0x29,0xd6,0xac,0x44,0xee,0x47,0x44,0x0b,0xbb,0x3f,0xb8, -0xdb,0x1b,0xde,0xf8,0xd9,0x77,0x01,0xc4,0x39,0xcb,0xf7,0xd8, -0x78,0x5d,0x1c,0x41,0x32,0x9a,0x1c,0x2f,0x17,0xcf,0xa0,0xa9, -0x51,0x83,0xbe,0x71,0x7e,0x31,0x98,0xa5,0x3e,0xbc,0x0f,0xf3, -0x2f,0xd1,0x3a,0x52,0xea,0xaa,0xa1,0x0d,0x28,0xd0,0x36,0xef, -0x62,0x3b,0x2b,0xe2,0x41,0x54,0x41,0xf5,0xbc,0xbb,0x3c,0x85, -0xe9,0xae,0xc9,0x2e,0x0d,0xd8,0xa5,0x22,0xa3,0x54,0x7d,0x2b, -0x1a,0x49,0xfd,0x67,0x28,0x69,0x9f,0x0c,0x6e,0xf2,0xf1,0x7c, -0xf6,0x24,0x43,0x26,0xee,0x09,0x52,0xb4,0xfe,0xcb,0x57,0x18, -0x36,0x91,0x62,0x66,0xfd,0x97,0x2a,0x23,0x27,0xfe,0x90,0x8e, -0x31,0xc1,0x57,0x27,0xfd,0x7c,0xd8,0x43,0xe3,0x41,0xe8,0x39, -0x64,0x07,0xef,0x85,0xc4,0xb0,0x3d,0xe7,0x78,0xd3,0x77,0x76, -0x28,0xb2,0xf5,0x5f,0x9a,0xf6,0x5d,0xa4,0x35,0x95,0x55,0xb4, -0x69,0x61,0xa6,0x1f,0x07,0xb7,0x04,0x45,0x39,0x10,0xd6,0xb0, -0xeb,0xf5,0x35,0x15,0x74,0x8a,0x94,0xb6,0xb2,0x6c,0xb5,0xce, -0x03,0x5e,0x62,0x95,0x4c,0x3b,0x5c,0x8e,0x02,0x0d,0xc1,0xa2, -0x95,0x0e,0x98,0x2f,0x0f,0xaa,0x11,0x88,0xb2,0xab,0x2d,0x36, -0x3d,0xf5,0x16,0xbc,0x1c,0x4a,0xdd,0xa5,0x6b,0x53,0xa8,0xb2, -0xc1,0x97,0x68,0xc9,0x92,0x7b,0x30,0x31,0x15,0x65,0xb3,0x72, -0x2f,0x4e,0x4f,0x3d,0xe4,0xcc,0x9b,0xf8,0x28,0x9a,0x04,0xc1, -0x9c,0x6a,0x1b,0x2d,0x65,0x9a,0x17,0x78,0x08,0x34,0xcc,0x8a, -0x56,0x55,0xbc,0x00,0xc4,0x61,0x90,0x1d,0xcb,0xc8,0xd0,0xcc, -0xb1,0xb0,0x3b,0xf1,0xc3,0x8d,0xb6,0x3d,0x18,0x71,0x8d,0x79, -0x9f,0x1c,0xc9,0x3f,0xa1,0xb7,0x4c,0x38,0x82,0x82,0x71,0x5c, -0x91,0x35,0x21,0xd7,0x5d,0x46,0xc2,0xc1,0x44,0x59,0xff,0x49, -0xee,0x34,0xaa,0xa7,0x8a,0x36,0xb5,0x61,0xcb,0xa8,0xcb,0x0f, -0x16,0x78,0xe5,0x3c,0x1f,0xdf,0x6f,0x43,0x9c,0xed,0xc8,0x56, -0xeb,0x05,0x31,0x2f,0xe0,0x91,0x10,0x8b,0x76,0x97,0x80,0x97, -0xbf,0xc3,0x2d,0xd6,0x34,0x9f,0xed,0xcc,0x80,0xb5,0x9f,0xcf, -0x67,0x20,0xca,0x61,0xb9,0xc8,0xe0,0xdf,0xc4,0x58,0x18,0x3a, -0x37,0xa7,0x3c,0x19,0x04,0x61,0xd4,0xb9,0xc7,0x40,0x5a,0x01, -0xc9,0xd0,0xa9,0xfc,0xc5,0x78,0x58,0x4c,0xb4,0xf1,0xe6,0x7a, -0x3c,0x9d,0x95,0x4a,0x42,0xdc,0xb2,0x0a,0x1c,0x91,0x51,0x18, -0xbf,0x92,0xe7,0xf6,0x60,0x60,0xf9,0x0d,0x09,0xde,0x26,0xb9, -0x67,0xa7,0xbc,0x22,0xe1,0x6e,0x81,0xf6,0x79,0x84,0xe4,0xda, -0x96,0xdd,0xaa,0x38,0xb3,0x33,0xf2,0x6b,0xf4,0xd5,0x97,0xb6, -0xc8,0xf2,0xcb,0x1c,0xc4,0xbb,0x9f,0xec,0xcd,0x95,0xec,0x7d, -0x4a,0xcf,0x63,0x7c,0xbe,0x8e,0x7f,0x6d,0x6b,0xa3,0xb0,0xda, -0xd4,0x07,0x0b,0x9b,0x25,0xdb,0x41,0xe8,0x5e,0xdf,0xb6,0xc2, -0x45,0xba,0x2d,0x14,0xf6,0x54,0x90,0x83,0xba,0x2e,0x64,0xce, -0xff,0x45,0x4d,0xae,0x24,0x38,0x92,0xcd,0xf1,0x71,0x13,0x65, -0x51,0xa7,0xdd,0xeb,0xaf,0xc8,0xca,0x95,0x2b,0xb4,0x7e,0xcb, -0x5f,0x01,0x5d,0x5e,0xe1,0x85,0x39,0xab,0x95,0x98,0x4b,0x36, -0x01,0xa5,0x74,0xc9,0xb8,0x40,0xf3,0xd2,0xda,0x86,0x82,0x5c, -0x8d,0xe9,0xfc,0x12,0x57,0x1a,0x9b,0x99,0xb7,0xd8,0x14,0x97, -0x45,0x5c,0x63,0x64,0x2f,0x81,0x19,0x78,0x66,0x3f,0x56,0xbd, -0x77,0x2d,0xbf,0x63,0x2d,0x1b,0x07,0xe3,0x4d,0x99,0x45,0x2c, -0x38,0x14,0x54,0x58,0xc1,0x16,0x05,0x6f,0xa6,0xa5,0x51,0x11, -0x5f,0xa7,0x5f,0x7c,0x89,0x8c,0xfd,0x62,0x04,0x13,0x95,0xa7, -0x4f,0x27,0x39,0xf8,0xb7,0x9c,0x66,0xed,0x99,0xf1,0x6f,0x39, -0x9d,0x4d,0x28,0xe3,0xdf,0x72,0xda,0x70,0x70,0x03,0x62,0xda, -0x38,0x32,0xf2,0x51,0x91,0xc3,0x6e,0x51,0x20,0x8f,0xfd,0x2c, -0xe7,0x12,0x95,0x6b,0xc3,0xbf,0xe5,0xf4,0xf1,0x04,0x8f,0xef, -0xa0,0x89,0xf2,0x51,0xce,0x31,0x47,0x39,0x9c,0xeb,0x71,0x9f, -0x92,0xab,0xe8,0xa8,0xec,0x95,0x75,0x57,0x56,0xec,0x7e,0x7e, -0x2a,0xe5,0x0d,0x93,0x1b,0x7c,0xa9,0xc6,0x7e,0xd8,0x13,0x71, -0xcd,0xce,0xa6,0xb7,0x28,0xa3,0x3b,0x7d,0xa7,0x90,0x18,0x42, -0xce,0xfb,0xe8,0x14,0x87,0x6c,0xaa,0xb7,0xc5,0xca,0x9b,0x8f, -0x6c,0x5e,0x5c,0xe7,0x15,0x16,0x0c,0xec,0xde,0xe5,0x43,0xb6, -0xd5,0xfe,0xf0,0x83,0xca,0x2e,0xd7,0xb1,0x1f,0xf8,0x2a,0x76, -0xcd,0xa7,0x9c,0x7e,0xe8,0x5a,0xbb,0x13,0x4e,0x02,0x94,0x8f, -0xad,0xa5,0x91,0x29,0xad,0x91,0x2f,0x58,0x7d,0x63,0x37,0xdf, -0x14,0x79,0x3e,0x1e,0x0f,0xf3,0x1e,0xc4,0x41,0x0d,0xad,0x2c, -0x0b,0x0a,0xb9,0xef,0x2c,0x78,0x22,0x46,0x86,0xe0,0xef,0x0b, -0xe7,0x0e,0xa2,0x60,0xef,0xd2,0x97,0xb2,0xed,0xb1,0x56,0xe5, -0x9e,0x04,0x8c,0x9c,0x2f,0xc4,0x11,0x3b,0x1d,0xc3,0xdd,0xd5, -0x5b,0x24,0x19,0xde,0x5d,0x19,0x8c,0xc7,0x97,0x22,0x61,0xb4, -0xf3,0xb2,0x4b,0x45,0x5e,0x6c,0xb4,0x96,0x3c,0x57,0xf3,0xde, -0x04,0xf6,0x7c,0x15,0x55,0x55,0x3d,0x0e,0xc4,0x14,0xd8,0x68, -0xad,0xd1,0x1d,0xa1,0xf2,0x49,0xbc,0xe5,0xfa,0x97,0xe1,0x39, -0x30,0x7e,0x77,0x97,0xb4,0x43,0x63,0x7e,0xe4,0x89,0x24,0xec, -0x59,0x0a,0x09,0x85,0xa7,0x95,0x76,0xfe,0x5c,0xdf,0x78,0x3c, -0x79,0x74,0xf8,0x85,0x65,0x94,0x86,0x75,0x70,0x43,0x82,0x0c, -0x0f,0x41,0xb2,0x43,0xa0,0x2c,0xdb,0xff,0x51,0x1e,0x2f,0x01, -0xb2,0xd8,0x1f,0xeb,0xe8,0x24,0x8b,0xa3,0x7d,0xd3,0xeb,0x75, -0xdf,0xe8,0x53,0x4e,0x14,0x2f,0x26,0x89,0xa3,0x35,0x47,0x7d, -0x84,0x0c,0x4d,0x84,0x28,0xd5,0x8f,0xde,0x06,0x3f,0xa0,0x01, -0x6d,0x65,0x31,0x1f,0x4d,0x5e,0x7a,0xb0,0x83,0x22,0xc4,0xb6, -0x83,0x18,0x9a,0x23,0x04,0x31,0x65,0xd9,0x2e,0x99,0x4d,0x94, -0x79,0xfa,0xf4,0x0e,0xf7,0x65,0xb2,0xfb,0x75,0x15,0x5a,0x86, -0xdd,0xad,0x8c,0xa8,0x63,0x37,0xa9,0x9a,0xa0,0xab,0x32,0xb4, -0xe2,0xdf,0x59,0x0b,0xb5,0xd2,0x7d,0x16,0x7e,0xae,0x25,0x8b, -0xed,0xec,0x1a,0x33,0xb7,0xd5,0x9c,0x66,0xd7,0x94,0xd5,0x47, -0x52,0xd2,0x31,0x77,0xee,0xe0,0x4c,0x20,0x92,0xb9,0x4a,0xb2, -0x86,0x13,0x83,0xc0,0x16,0x56,0x1c,0xb9,0x73,0x72,0x7f,0x63, -0xfd,0xcf,0x44,0x46,0xfc,0xfe,0x30,0x4e,0x64,0xc3,0x76,0x15, -0x66,0x6d,0xc5,0x55,0x3c,0xab,0x29,0xd6,0xa6,0xfa,0x52,0x0a, -0x14,0xec,0x4e,0xb2,0x5b,0x0b,0x3b,0x02,0xcc,0xb1,0x41,0x62, -0x39,0xed,0x1a,0xe6,0x8f,0xfc,0xfa,0x55,0xb1,0x46,0x6f,0xaf, -0x8f,0xdd,0x61,0x01,0x73,0x6c,0x17,0x79,0xae,0x7b,0xb7,0x63, -0xe1,0x31,0x32,0x72,0xe2,0x67,0xd7,0x01,0x49,0x53,0x46,0x11, -0xc2,0x15,0x15,0x55,0x99,0x2e,0x7a,0x43,0x6b,0x50,0x80,0x8e, -0xe0,0xa7,0x99,0x44,0x36,0xdf,0xfd,0xb4,0xff,0xe3,0xde,0xd1, -0xfb,0x57,0x07,0x47,0xfb,0x3b,0x27,0xc7,0x25,0x1f,0x91,0xbd, -0x1b,0xb4,0x4f,0xe8,0xd6,0xe8,0xe3,0x2f,0x37,0xe7,0x63,0xa7, -0xd3,0x54,0xd3,0x9b,0xf7,0x42,0x16,0x7c,0x02,0x19,0xc4,0x64, -0x52,0x73,0x73,0xf7,0xa7,0xa3,0xa3,0xbd,0x77,0xbb,0x7f,0x79, -0x7f,0xfc,0x97,0xfd,0xc4,0x70,0xac,0xb8,0x3c,0x97,0xda,0x6c, -0xce,0xc3,0x9d,0x93,0x93,0xbd,0xa3,0x77,0xc7,0x68,0xc5,0xcf, -0xc6,0xfd,0xfe,0xe8,0xe0,0xa7,0xc3,0xf7,0xc7,0x7b,0x87,0x2e, -0xe6,0xe5,0xde,0xee,0x9b,0xfd,0x9d,0xb7,0x14,0xb7,0x9d,0x80, -0x68,0x73,0x3b,0x44,0x3f,0xe0,0x9b,0x67,0xf3,0xad,0xad,0x9d, -0xef,0x37,0xaf,0x8a,0xc8,0xab,0x8e,0xd2,0x0b,0xf7,0x3f,0xd4, -0x4d,0x0c,0xc8,0x5c,0xa2,0xa7,0x6c,0x08,0x1f,0x0f,0xfe,0xee, -0x49,0x32,0x68,0xa1,0xcd,0x58,0x6c,0xe1,0xd6,0x53,0x5b,0x18, -0x54,0xa1,0xbd,0x99,0x56,0xd4,0x02,0x74,0x8e,0x57,0x45,0xe6, -0x6a,0x32,0x9e,0xdf,0x1e,0xe7,0xb7,0xb0,0x11,0xbb,0x18,0xdc, -0xf4,0x86,0xf8,0x19,0xa2,0xca,0x1e,0xea,0x7b,0xef,0xa4,0x64, -0xb2,0x58,0xe0,0x93,0x78,0x7c,0xda,0x95,0xdb,0x28,0x61,0x76, -0x51,0x44,0x32,0x03,0xe4,0xc6,0x0b,0x92,0xc1,0x5d,0x9e,0x6d, -0xbd,0xe0,0x1c,0x6d,0xfe,0xc9,0xf6,0xd1,0x02,0x74,0xef,0x7c, -0x6a,0x0b,0xf2,0xc3,0xea,0xf9,0xcd,0xf1,0x6c,0x92,0x71,0x54, -0x23,0x8a,0xa4,0x6d,0x79,0xff,0x04,0x17,0x4f,0x08,0x83,0xc0, -0x4d,0xee,0x73,0xe9,0x40,0xe9,0xf3,0xed,0x78,0x94,0x93,0x77, -0x14,0x3c,0x22,0xda,0x68,0xe1,0xbb,0x51,0x02,0xe0,0xbd,0xd3, -0xe3,0x2d,0x33,0xfb,0xcb,0xa4,0x63,0x52,0x49,0x66,0x05,0xb9, -0xcd,0xf8,0xf4,0xac,0x7f,0xd6,0xec,0x36,0x92,0x3c,0xde,0xe8, -0x24,0x7c,0xf6,0x9c,0xb4,0x6f,0xf8,0x3d,0x5e,0xb4,0x11,0x89, -0x16,0xde,0xe9,0x76,0x17,0xd5,0x25,0xf0,0xeb,0x79,0xf7,0x85, -0xee,0x91,0x46,0xab,0x23,0x18,0x47,0x5b,0x51,0x1a,0x07,0xc8, -0x72,0x42,0x88,0xe7,0x16,0x69,0x0e,0xa8,0xa8,0x44,0x43,0x43, -0xff,0xbf,0xdc,0x72,0x58,0xe9,0xeb,0xf5,0x96,0x74,0x18,0x4c, -0x91,0x22,0x60,0x88,0x05,0x41,0xe1,0xd5,0xe0,0x33,0x4c,0xa8, -0x60,0x84,0x94,0xf3,0x3b,0x1b,0xff,0x36,0x1f,0x65,0xb1,0xb4, -0x9b,0x0d,0x6e,0x2b,0x3a,0xc1,0x23,0x73,0x54,0x3f,0x49,0xdc, -0x2a,0xa6,0xa6,0x6a,0x00,0x19,0xb1,0x50,0x61,0x1e,0xbe,0x9b, -0xc1,0x28,0xe6,0x8f,0xde,0xe7,0x58,0xc8,0x08,0x23,0x5f,0x41, -0x4e,0xa3,0x10,0x48,0x8c,0x4b,0xec,0x7d,0xc6,0x44,0xb9,0xfa, -0xbf,0x1d,0x7f,0x62,0x40,0xf0,0x11,0xb7,0xb6,0x42,0x6a,0x0b, -0x08,0x05,0x48,0x73,0xd4,0x17,0x52,0xf9,0x0e,0x72,0x27,0x9b, -0xf0,0xa7,0xad,0xdb,0x49,0xa2,0x95,0xd0,0x52,0x45,0x43,0xcd, -0xa7,0x6b,0x58,0x40,0x33,0x9b,0x1b,0x66,0x52,0xdb,0x95,0x74, -0x91,0xdc,0x19,0x82,0xda,0x14,0xd5,0xa1,0x69,0x4e,0x64,0x16, -0xfb,0xe1,0x15,0x62,0x66,0xc2,0x48,0x8a,0x43,0x02,0xa4,0x1a, -0xac,0xcf,0x91,0x8c,0x8b,0x36,0xe8,0x6f,0x82,0x2f,0xc5,0x32, -0x9d,0xbe,0xc1,0xc9,0xc1,0x1b,0x7a,0xc8,0x23,0xba,0x89,0x68, -0xf1,0x35,0x86,0xe0,0xc6,0x20,0xf9,0x15,0xe5,0xab,0xd7,0xd1, -0x57,0xcb,0xa0,0x40,0x0a,0x8d,0xcc,0x4e,0xd9,0xc4,0x84,0xf1, -0x5c,0x95,0x48,0x3b,0x03,0xf6,0x40,0x38,0xc8,0x00,0x22,0xc8, -0x2c,0x1a,0x0d,0x5f,0x5b,0x80,0x1c,0x54,0x3b,0xfc,0x67,0xd4, -0x4b,0xcd,0x73,0xfd,0x2c,0xd0,0x7f,0xd0,0xa3,0xdc,0x76,0x33, -0xa0,0x81,0x73,0xa8,0xad,0xd3,0x60,0x0e,0x6e,0xa1,0xec,0x16, -0xc6,0x15,0x30,0xf1,0xbc,0xaa,0xe1,0xea,0x71,0x97,0x18,0xe1, -0xdc,0x58,0xfa,0xad,0xbd,0x75,0xa3,0xed,0xf9,0x53,0xc7,0x0e, -0xe7,0x28,0xbf,0x3a,0x9c,0xe4,0xa9,0x0d,0x42,0x9f,0x41,0x30, -0x31,0xaa,0x90,0x46,0x20,0x48,0xa8,0x86,0x76,0x3c,0xbf,0xd4, -0xd0,0x20,0x68,0x0b,0x7d,0x18,0xc3,0xfc,0x81,0xc9,0xa7,0xdc, -0x6a,0xf6,0xfa,0x9e,0x37,0x9b,0xfe,0xe0,0x6a,0x30,0x9b,0x1a, -0x10,0xef,0x6e,0xe4,0x15,0x4d,0x7e,0x05,0x4c,0x90,0x7a,0x95, -0xd8,0x29,0x59,0x22,0x80,0xa8,0x0d,0xda,0x5c,0x66,0x1b,0xf0, -0x27,0xa1,0x2f,0x9e,0x0a,0x38,0x83,0x6c,0xa7,0x33,0xac,0x76, -0x42,0xa9,0x5b,0x9c,0xec,0x2e,0x0b,0x07,0x04,0x09,0x52,0xb0, -0x80,0xf4,0x9e,0x2f,0xbb,0xc1,0x65,0x13,0xb4,0x9a,0x70,0x85, -0x05,0x95,0x65,0x68,0xe8,0x07,0xb1,0x8e,0x41,0xe7,0x0f,0x53, -0x9c,0x1e,0xe3,0xcb,0xcb,0x69,0x3e,0x13,0xb4,0xad,0xe3,0x27, -0x8a,0xcb,0xf8,0x67,0xb1,0xd8,0xf2,0x27,0x06,0x7d,0x67,0x3f, -0x88,0xc5,0x1c,0x0c,0x9f,0xe2,0x1d,0x2a,0xe0,0x88,0x4f,0x13, -0xbc,0xf7,0x59,0x2e,0xfc,0x62,0x6b,0xb1,0xa0,0x9c,0x2f,0x36, -0x38,0x22,0xb1,0x6f,0xa7,0x1b,0x02,0x9e,0x2d,0x26,0xcb,0x83, -0xe7,0x8d,0xd6,0x76,0x26,0xf1,0xee,0x8d,0x75,0x6b,0x1b,0x47, -0xc0,0xf6,0x34,0x4b,0x7f,0x84,0x39,0xa1,0xbc,0x0c,0x9b,0x07, -0x7c,0x33,0x68,0xe1,0xf5,0x78,0x32,0x7b,0x05,0x14,0x50,0xde, -0x61,0x63,0x6e,0xbb,0x18,0x3f,0xdc,0x22,0x32,0x6b,0xe3,0x76, -0xf2,0xb1,0x83,0xd9,0x89,0x8e,0x5f,0x1f,0x1c,0x9d,0x70,0xbe, -0x94,0xef,0xaa,0x03,0xf1,0x60,0x7a,0x0a,0x25,0xbb,0xa7,0x04, -0xb7,0xab,0x10,0xc5,0x43,0xce,0xbf,0xc2,0x8a,0x22,0x98,0xfa, -0x3e,0xfd,0x3b,0x9a,0x7d,0xdb,0x68,0x7d,0xd7,0xb7,0x9a,0x0c, -0x90,0x11,0xe3,0x0e,0xa8,0x47,0x62,0xea,0x87,0x7e,0xde,0xc7, -0xb2,0x19,0xc6,0xbf,0xc8,0xb6,0x3a,0x51,0x23,0x4a,0x81,0xca, -0xdc,0x6c,0xb1,0x19,0x1a,0x99,0xef,0x34,0x64,0xca,0xa7,0x54, -0x00,0xf2,0x5f,0x0e,0xc7,0x63,0xd8,0x8c,0x44,0x17,0xf9,0x60, -0x18,0x75,0x63,0x8c,0xde,0xfc,0xcd,0x56,0x02,0x62,0x56,0x23, -0x2c,0x41,0xeb,0x3d,0x26,0xff,0x8a,0x93,0x95,0xb1,0xf0,0x9b, -0xdb,0x1b,0x85,0xbc,0xef,0x46,0x6b,0x91,0x5a,0xf0,0x7f,0x3d, -0x9e,0x4f,0xa6,0x71,0xf2,0x43,0x6b,0xbb,0x63,0xc5,0x9e,0x9d, -0xfd,0xc3,0xfd,0x63,0xf2,0x66,0x1f,0x44,0xb4,0xba,0xe1,0x38, -0x16,0xc5,0x37,0x97,0x88,0x87,0x3f,0x7c,0xe7,0x76,0x32,0x7e, -0x09,0x19,0x63,0xde,0x4f,0x29,0xb1,0xa1,0xed,0xae,0x59,0x39, -0x49,0xe4,0x87,0xa3,0xf7,0x6f,0x8e,0x0f,0x7e,0xfb,0x9b,0xad, -0xd6,0xfb,0xe3,0x93,0x23,0x7b,0x03,0x05,0x00,0x48,0xc9,0x9a, -0x20,0xa1,0xb3,0xb2,0xbf,0x23,0xca,0xb0,0x94,0xcc,0xfe,0xbe, -0x3f,0x18,0xc1,0x2f,0xd1,0x14,0x5b,0x88,0x60,0x99,0xe2,0xb7, -0xdd,0x0e,0xb5,0x0e,0xc6,0xe3,0xa7,0x93,0xdd,0x57,0xf3,0xe1, -0xf0,0x2f,0xb0,0x3b,0x49,0x6d,0x9c,0x8d,0xa0,0x83,0xec,0x07, -0x0a,0x52,0xcf,0xb8,0x52,0x14,0x62,0x51,0xe6,0xf4,0x77,0x68, -0x68,0x44,0xf0,0xc0,0xeb,0x76,0x1b,0xdb,0xe0,0x8f,0x16,0x3a, -0xf2,0x62,0xec,0xaa,0x52,0x5b,0x5d,0x98,0xf8,0x1e,0x69,0xb6, -0x02,0x44,0x63,0xe4,0x73,0xe3,0xbd,0xbb,0x0f,0x6d,0x77,0x93, -0x8d,0x96,0x0a,0x3f,0xef,0xca,0x72,0x7f,0xad,0x2a,0xf8,0x1e, -0x96,0xda,0xad,0x64,0x83,0xd1,0x32,0x37,0x2a,0xe5,0xd7,0x36, -0x05,0x30,0x32,0x53,0x95,0xf0,0x1b,0x4a,0x30,0x37,0x53,0x2d, -0x12,0xb4,0xf2,0xe7,0xdf,0xd1,0x29,0xec,0xab,0xe1,0xb8,0x37, -0x8b,0xa3,0xad,0x66,0xd4,0x90,0xfc,0xff,0x4e,0xf9,0xbd,0x9b, -0x3c,0xdf,0x81,0xaa,0x11,0xd7,0xe6,0xc6,0x4c,0x01,0x26,0xb7, -0xd1,0x2e,0x11,0x3c,0xce,0x74,0xd8,0x12,0x8c,0x32,0x2a,0x00, -0x9c,0xf5,0xef,0xbf,0x5f,0x26,0x1b,0x1d,0xf8,0x38,0xeb,0xbb, -0xdf,0xb8,0x93,0x9e,0xb8,0xaf,0xb4,0x53,0xf5,0x79,0xd6,0x24, -0x11,0x33,0xe9,0xe0,0xff,0xf1,0x5f,0x17,0xf1,0x69,0x63,0xa3, -0x9b,0x70,0xb2,0xcd,0x86,0x49,0xeb,0x9b,0xa5,0x8d,0x83,0x9a, -0x13,0xd6,0xe0,0x9b,0xe8,0x55,0xcc,0x0a,0x82,0x31,0xd2,0x2a, -0x67,0x94,0x9d,0x14,0x48,0x34,0x37,0xb0,0x5f,0x9c,0xdf,0xbc, -0x64,0x85,0x37,0x4e,0xb3,0x1b,0x96,0x97,0x3b,0x27,0x7b,0x27, -0x6f,0xf6,0xf7,0xec,0x96,0xe5,0x94,0xd3,0xa1,0xe7,0xf8,0xc3, -0xf8,0xfb,0x68,0xe4,0x29,0x68,0x0d,0x02,0x29,0x5c,0xf6,0x39, -0xd0,0x23,0x6f,0xde,0xfd,0x9e,0x75,0x2c,0x28,0xbd,0x83,0xa3, -0x45,0x5f,0x69,0x69,0x5a,0x51,0x34,0xd0,0xc9,0x54,0x18,0x42, -0x00,0xd0,0x4d,0x19,0xc9,0x05,0xfb,0x0a,0x15,0x54,0x4c,0x40, -0x44,0x0a,0x42,0xae,0x9d,0xf0,0xac,0xc4,0x46,0xd8,0x06,0xbc, -0x3f,0x3e,0x7c,0xfb,0xe6,0x84,0x75,0x20,0xa4,0xc3,0xb8,0xa7, -0xd0,0x7f,0x0f,0x76,0x91,0x3c,0xac,0xa3,0x80,0xf4,0x61,0xcb, -0x8a,0x31,0x99,0x2c,0xe6,0xe3,0x5b,0x72,0x92,0x5d,0x5a,0xf3, -0x5d,0x3e,0xf2,0xb1,0x5c,0x72,0x4a,0x40,0x20,0x4b,0x07,0xa0, -0xa3,0x00,0x3f,0xe1,0xdd,0x34,0x6e,0x8d,0xec,0x72,0xd4,0xb9, -0x94,0xd1,0x5d,0x35,0x22,0xd6,0x44,0xb6,0xdb,0xb0,0xc6,0x3f, -0x3f,0x5b,0x3c,0x5b,0x4f,0x60,0xc7,0x8a,0x72,0xbb,0x8b,0x7e, -0xf6,0x0c,0x63,0x9e,0x45,0xf4,0x10,0x1b,0x80,0xab,0xd5,0xc1, -0x1f,0x6f,0x57,0x9c,0x0b,0x5b,0x67,0xcb,0x76,0x92,0xb0,0xc5, -0x04,0x8e,0x25,0xc3,0xc6,0xca,0x6b,0xbe,0x3e,0x07,0xaf,0x00, -0x45,0xf7,0xf8,0x7c,0x5a,0x1e,0x1e,0x29,0x53,0x02,0xdd,0xcd, -0x5a,0x6a,0xe2,0x18,0x3b,0xb0,0x14,0x6a,0xd3,0xb3,0x42,0x28, -0x4b,0x53,0x9e,0xa1,0x98,0x15,0x05,0x28,0xb5,0x43,0x7f,0x71, -0xc5,0xa2,0xc4,0xe6,0x94,0x9e,0x0b,0x6e,0x09,0x02,0xa9,0x8e, -0xa4,0x28,0xc3,0x31,0x62,0xe9,0x21,0xb5,0x5b,0x54,0x33,0x32, -0xe3,0x39,0x3d,0x7c,0x23,0x4b,0x2e,0x04,0x53,0xe7,0xec,0x58, -0xa4,0x7c,0x54,0x4a,0x51,0x3f,0x6c,0xe8,0x38,0x34,0x5d,0x42, -0x19,0x83,0xd8,0x84,0xb1,0x81,0x55,0x52,0x9e,0x26,0x09,0x72, -0xf1,0x20,0x00,0xd8,0x60,0x04,0x47,0x41,0x64,0xe2,0xde,0x2a, -0xe1,0xd5,0x2c,0x4b,0x98,0x98,0x8c,0x4f,0x2e,0x9d,0x4f,0xe6, -0xb9,0x1e,0xe8,0xe0,0x02,0x41,0x54,0x5a,0x57,0xdd,0x04,0x4c, -0x41,0xe6,0x38,0xb4,0x27,0x49,0x06,0x1f,0xa2,0x42,0xe6,0x03, -0x04,0xa0,0xd6,0x46,0x3c,0x72,0xed,0xc9,0x91,0x6b,0xcb,0x8c, -0xb7,0xfd,0x79,0x3e,0x9f,0x7f,0x06,0x40,0x4a,0xbe,0x43,0xb1, -0x74,0x16,0x64,0x41,0x87,0xaf,0x0c,0x08,0x47,0x1b,0xf7,0x16, -0x98,0x27,0x51,0x07,0xbc,0x96,0x03,0x6f,0x2d,0xd5,0x15,0x17, -0xe1,0xb6,0xeb,0x71,0xc1,0x8c,0xa6,0x9f,0x4f,0x2f,0xac,0x2f, -0x05,0x47,0xbc,0x3f,0xf2,0x31,0x69,0xac,0x12,0x3b,0xbe,0xe1, -0xe6,0x5c,0x39,0xd0,0xb9,0xb9,0x8d,0xcf,0x4d,0x2f,0x59,0xd2, -0xe9,0xfd,0xb2,0xd0,0xe4,0x3c,0xbe,0x6b,0x99,0xbb,0x6d,0xd1, -0x55,0x68,0x39,0x2b,0x3a,0x2d,0x33,0xdb,0x76,0x81,0x6d,0xb7, -0xb2,0xb4,0xb2,0x6c,0xb6,0xdd,0xd1,0x67,0xef,0x2d,0x94,0x36, -0x5b,0xd9,0x5d,0xab,0x70,0xd0,0x7f,0xb7,0x9d,0xdd,0x6d,0x87, -0x71,0x10,0xd9,0x42,0x81,0x75,0xbb,0xb3,0x95,0xde,0x6d,0xbf, -0xb8,0x6b,0x75,0x36,0x5a,0x69,0x2b,0x49,0x67,0xdb,0x2f,0x66, -0xfc,0xbd,0x7c,0xec,0x7e,0x06,0xd3,0x83,0x8e,0x0e,0x93,0x83, -0x24,0xe7,0x63,0x25,0x2c,0xd0,0x09,0x82,0xe9,0x69,0x10,0xec, -0x86,0xe4,0x02,0x02,0xc8,0x6d,0x58,0x5c,0x9b,0x0b,0xb6,0x10, -0xc5,0xe7,0x8b,0x1d,0x08,0xd4,0x72,0x43,0xb9,0xd7,0x65,0x58, -0x2c,0xac,0x6a,0xab,0x37,0x17,0x2a,0xb3,0xdd,0x03,0x81,0x6e, -0x8c,0x41,0x20,0xcd,0x7c,0x31,0x7f,0x88,0x0f,0x4b,0xdb,0x46, -0x75,0x0a,0xad,0x2b,0xbe,0xe6,0x55,0xd9,0x8c,0x8b,0x53,0xa9, -0xbc,0xff,0x41,0x2c,0x5a,0x09,0x4b,0xea,0xa2,0x4c,0xeb,0x91, -0x4a,0x4c,0x99,0x26,0x57,0x52,0x19,0x12,0x13,0x5e,0x72,0xf6, -0x08,0x58,0x7c,0x9e,0x90,0x4a,0x87,0x23,0xcf,0xa5,0x37,0xa4, -0x47,0x83,0xb5,0x3b,0xbe,0x55,0xce,0x94,0x83,0x03,0x61,0xe7, -0x47,0x19,0xf3,0x58,0xd7,0xdb,0x78,0x28,0xac,0x18,0x82,0x4f, -0xc7,0xf1,0x89,0xab,0xe7,0x0e,0x7f,0x86,0x93,0x3e,0x38,0xf0, -0xbc,0x7a,0x39,0x98,0xe4,0xa4,0xac,0x84,0xa6,0x72,0xf8,0x4b, -0x1d,0xe7,0x3a,0x2b,0x4b,0x3e,0x91,0xed,0x44,0x71,0x20,0xbb, -0x1f,0x0e,0x46,0x1f,0x53,0x17,0x86,0x15,0xc9,0x7d,0xa3,0x6a, -0x0d,0x69,0xb1,0x67,0xe5,0x28,0x18,0xcf,0x9d,0xdd,0xc8,0xd8, -0xeb,0x70,0x0f,0xdc,0x63,0x86,0xbb,0x24,0x54,0x7f,0x9f,0xa0, -0xc9,0xb3,0x89,0xf5,0x4b,0x6a,0x60,0x7b,0x3d,0xd1,0x0f,0x6d, -0x67,0xe3,0xab,0xab,0x61,0xfe,0x27,0x7c,0x84,0xb7,0x3b,0x9d, -0xc2,0xc6,0x9c,0x3e,0x0d,0xbd,0xca,0x23,0x8d,0x03,0x72,0x71, -0xfb,0xc7,0x9c,0x94,0x91,0x8a,0x71,0x59,0x39,0xaa,0x03,0x14, -0xd4,0x98,0x8e,0x7a,0x1f,0xf3,0xf7,0xb4,0x63,0x2b,0xe7,0x30, -0x90,0x03,0xd7,0x14,0xeb,0x97,0x16,0x1a,0x85,0x16,0x2a,0x77, -0x87,0x3d,0xa8,0xdf,0x22,0xd0,0x79,0xf3,0xee,0x4f,0x3b,0x6f, -0xdf,0xbc,0x7c,0xbf,0xfb,0x76,0xe7,0xf8,0x38,0x55,0xdf,0x49, -0xa3,0x02,0xb7,0x26,0x6c,0xbd,0x0a,0x00,0x74,0xf1,0x00,0x58, -0x25,0x80,0xa5,0x3d,0xa3,0x66,0x4b,0x07,0xfc,0x50,0x1d,0xbb, -0x30,0xb3,0x58,0x72,0x54,0x9c,0xa0,0x16,0xb8,0xed,0xd4,0x08, -0x4b,0x44,0xe8,0x9a,0x1d,0xa4,0x1c,0xea,0xf0,0xd9,0x64,0x68, -0xe4,0x4d,0xe3,0x2e,0x1e,0xbd,0xc3,0x4a,0x46,0x7e,0x0c,0xa6, -0x24,0x63,0x36,0xd7,0x29,0x40,0x36,0x6f,0x19,0xca,0x54,0x56, -0x53,0x48,0xc2,0x4d,0x6c,0x46,0xc3,0x23,0x56,0xb7,0xe4,0xfb, -0x0a,0x01,0x1b,0xce,0xe3,0x4c,0xba,0x71,0xf0,0x76,0x32,0x98, -0xce,0x06,0x6c,0xae,0x9c,0x63,0xa8,0x66,0x1f,0x14,0x54,0xb0, -0x84,0x6f,0x52,0x73,0x1d,0x7b,0x8b,0xeb,0x27,0x71,0xcd,0x79, -0x42,0xf7,0xdd,0x78,0x78,0xf4,0xe6,0xf8,0xe4,0xcd,0xbb,0x3d, -0xe9,0x32,0x53,0x20,0x13,0x7c,0xc9,0x7c,0x59,0x00,0xe5,0x6f, -0x59,0xa5,0x6d,0x78,0x75,0xcc,0x8d,0xe4,0x19,0x68,0xa3,0x6d, -0xdb,0xb9,0xc9,0x20,0xeb,0x10,0xa4,0xc2,0xe3,0xbc,0x20,0x4f, -0x22,0x87,0x59,0xa7,0x41,0x6c,0x37,0xb3,0x10,0xff,0xe7,0x52, -0xb0,0x11,0x52,0x7a,0x0c,0x21,0x5b,0x73,0x15,0xcc,0xcc,0x42, -0x45,0x0f,0x52,0xa4,0x66,0x55,0x91,0xcb,0x58,0x49,0x96,0x07, -0xd7,0xf3,0xf4,0x5f,0xc8,0x6c,0x9f,0x27,0x30,0xf2,0x9a,0x4e, -0xb2,0x00,0x60,0xc7,0x0e,0x7f,0x07,0x7d,0xe0,0xe3,0x71,0x21, -0x8b,0x61,0xf7,0xde,0x84,0xe5,0x32,0xb4,0x10,0x6b,0x3b,0xd1, -0xa7,0x9b,0x12,0xb8,0x40,0x99,0x20,0x80,0x6b,0x27,0xb4,0xeb, -0x02,0x24,0x74,0xc2,0x32,0x63,0xdc,0x4f,0x0b,0x25,0xba,0xac, -0x44,0x4a,0xa5,0x12,0xca,0x08,0x9d,0xaf,0xf1,0xe1,0x26,0xba, -0xc1,0xfc,0x25,0xb0,0x94,0x14,0x6b,0xe2,0xdf,0xd8,0x08,0xe6, -0x02,0x6a,0xfe,0x56,0x72,0x9b,0xe4,0x31,0xe2,0x4d,0xcc,0x0a, -0x5c,0xc9,0x2e,0x7a,0x91,0x34,0x4b,0xdd,0x1f,0xd0,0xfe,0x23, -0x83,0x40,0xf3,0xc1,0xde,0x0e,0x60,0x4f,0x04,0x0d,0x58,0x85, -0x3f,0x75,0x02,0xdf,0xfa,0x8c,0x2e,0x86,0x73,0x58,0xb4,0x0a, -0xdd,0x24,0x52,0x06,0x1b,0xee,0x58,0xd5,0x98,0x5f,0xac,0xed, -0x76,0x5d,0x69,0xa3,0x51,0x6a,0x61,0xeb,0x1f,0x68,0x21,0xb3, -0x8e,0xa4,0xfd,0x8b,0x37,0x1b,0xe5,0x86,0x32,0x18,0x85,0x56, -0x71,0x14,0xb6,0x96,0x8a,0xf0,0x5e,0x12,0x23,0xd2,0x26,0x5c, -0x2a,0x98,0x79,0x81,0x89,0x78,0xee,0xf2,0xf2,0xcd,0xd1,0xc9, -0x5f,0x2c,0x6b,0x09,0x38,0xdb,0x56,0x89,0xb3,0xb5,0x8a,0x6d, -0xa3,0xaa,0x63,0x3d,0x0b,0x0e,0x6d,0xe6,0x47,0xf0,0xd1,0xd5, -0xae,0x66,0x75,0x4f,0xe1,0xb4,0x34,0xf7,0xdd,0xc4,0x7c,0x80, -0xcf,0x28,0xec,0x62,0xba,0x20,0x57,0x26,0x26,0xf6,0xd0,0x6c, -0x51,0xe1,0xba,0x5d,0xdf,0x2a,0x71,0x8a,0xbc,0x79,0xbe,0xe3, -0x97,0xed,0xf2,0x5c,0x5a,0x42,0x77,0x62,0x03,0x96,0xdf,0x42, -0xf9,0x35,0x1d,0x36,0xb5,0x6f,0x70,0xff,0x43,0x6f,0x16,0xd9, -0x08,0x92,0x5e,0xff,0xcd,0x05,0x2e,0x53,0xd6,0x21,0xa4,0x72, -0x97,0xa0,0x6d,0x00,0x55,0xba,0xc2,0x73,0xab,0x21,0x9a,0x09, -0x4b,0xda,0x7e,0x17,0x81,0x60,0x61,0xa9,0x3a,0x99,0x0c,0x6e, -0x00,0xe1,0x93,0xc8,0x1d,0x35,0x67,0x78,0x52,0x6c,0xb5,0x94, -0xa8,0x62,0xa0,0xb0,0x81,0x98,0x87,0x5d,0x73,0x87,0xcf,0x53, -0xad,0xc9,0xab,0x6a,0xe6,0x02,0x48,0xd1,0xb6,0x8c,0xbd,0xad, -0x4f,0xc8,0xd0,0xb8,0xf3,0x6a,0x69,0xd5,0xe6,0xad,0x8e,0x7c, -0x92,0x58,0x54,0x01,0x8e,0xc4,0x79,0x33,0x5c,0xfe,0xf2,0x4f, -0x94,0x5e,0x59,0x2b,0xf8,0x6d,0x45,0xdb,0x25,0x03,0x32,0x2f, -0xfe,0x7a,0x40,0x33,0xda,0x19,0xc9,0x4d,0xac,0x32,0x2d,0x9b, -0xfb,0x02,0x91,0xb1,0xad,0xd1,0xf9,0x98,0x7f,0xe9,0x8f,0x3f, -0x8d,0x94,0x36,0xa0,0xe8,0xf8,0x8b,0xb6,0x09,0xeb,0xf7,0x37, -0xe1,0x6b,0x17,0x5f,0x45,0xfe,0x0e,0x37,0x3d,0x10,0x58,0x2c, -0xe0,0xcf,0x8b,0xd6,0xaf,0xeb,0xf5,0xd6,0xef,0x5e,0xb8,0x70, -0xf6,0xfc,0xdf,0xeb,0xf5,0xef,0xb7,0x5e,0x70,0x8e,0xa0,0x1d, -0x64,0x6b,0x4e,0x57,0x2c,0x2f,0x14,0x7c,0x47,0x98,0x8a,0x0e, -0xbc,0xed,0x41,0x22,0x8e,0xa0,0x2e,0x49,0x91,0xb5,0x0b,0xec, -0xc5,0xa0,0x8a,0x64,0xc9,0x43,0x04,0x53,0xb4,0x1f,0x76,0x9c, -0x26,0x15,0x4b,0xec,0x85,0xf1,0x4f,0x3a,0x51,0x94,0x16,0xe3, -0xec,0x7b,0x38,0xba,0x07,0xfa,0x13,0xf3,0x2f,0x10,0xc1,0xf9, -0x00,0x4a,0xa2,0x33,0xa1,0xb7,0x43,0xb9,0x90,0x17,0x36,0x97, -0x6b,0x37,0x3c,0x57,0x68,0x99,0xb4,0x38,0xbb,0xf4,0x9c,0x43, -0x07,0x2d,0x57,0x64,0xd4,0x14,0x4f,0xe5,0x38,0xae,0x13,0x7b, -0x8a,0xb3,0x3c,0x34,0x92,0x4a,0x23,0x32,0xe0,0xc2,0xf9,0xd2, -0x87,0xf3,0xc1,0x8a,0xe5,0xbd,0xf7,0x12,0xa1,0x4a,0x1a,0x4c, -0x0b,0x3e,0x89,0xf3,0xd7,0xbc,0x74,0xc1,0xfe,0xf3,0xd9,0x66, -0xdc,0xfc,0x2e,0x81,0xbf,0xa7,0x57,0x83,0x9b,0xee,0x77,0xc9, -0xfa,0xa6,0x3a,0x8d,0xe3,0x36,0xe3,0xc9,0xdf,0x51,0x7e,0xb5, -0xf7,0xf9,0xd6,0x1d,0x26,0x1b,0x77,0x8e,0x6c,0x8a,0x1d,0xf6, -0x80,0xaf,0x2c,0xea,0x2a,0x0b,0xd6,0xbe,0x77,0x4b,0xd2,0x47, -0x21,0xa8,0x71,0x39,0x38,0xff,0x60,0x8d,0x8e,0x91,0xc9,0x40, -0x89,0xa6,0xc3,0x8a,0x35,0x9f,0x67,0xb1,0x50,0x01,0xea,0x67, -0x79,0xb6,0x6b,0x7d,0xcd,0xb8,0x21,0x8c,0x12,0x08,0x8c,0x79, -0x40,0x22,0x13,0x41,0x23,0xc9,0x4e,0x34,0x5a,0x17,0xc0,0xd7, -0xb3,0xe7,0x79,0xad,0x27,0x8d,0xaf,0x9d,0xcf,0x67,0xb5,0x4f, -0xbd,0x29,0x5a,0x05,0x69,0xd6,0x44,0x7d,0x3a,0xad,0xdd,0x6f, -0x2f,0x23,0xa7,0xa0,0xe1,0x2b,0x35,0xd3,0x59,0x8f,0xac,0xa2, -0x9e,0xf4,0xae,0xec,0x26,0x28,0xd1,0xce,0x9e,0x83,0xce,0xc0, -0x02,0xf6,0x49,0x21,0x0f,0x30,0x1f,0x63,0xce,0x9c,0x11,0xc0, -0x62,0x27,0x59,0x9e,0xc6,0x4e,0x77,0x57,0x65,0x4a,0x8c,0x10, -0xec,0xfe,0x60,0x24,0x47,0x56,0x7c,0x69,0x62,0x83,0x74,0x92, -0x57,0xca,0x63,0x20,0x9d,0x4d,0x5d,0x3f,0x36,0xac,0x6b,0x21, -0x69,0xd7,0xeb,0x7c,0x14,0x2a,0x57,0x9a,0xae,0x9a,0x4a,0xf2, -0x76,0xa9,0x45,0xc2,0xad,0x24,0x72,0x9d,0xdb,0x4d,0x87,0x65, -0xbb,0xa2,0x17,0xca,0xc8,0x27,0x2b,0x3a,0xb5,0x22,0x27,0x99, -0x2b,0x92,0xfe,0xe8,0x7d,0x0e,0xfa,0xcc,0x06,0x83,0x3e,0x73, -0x79,0x60,0x4a,0x7c,0xfe,0x27,0xf4,0xd9,0x0b,0x57,0x4d,0x75, -0x9f,0xd9,0xd4,0xa7,0xf5,0x99,0xca,0xfd,0x48,0x9f,0x95,0x90, -0x5f,0xd9,0x67,0xe5,0x9c,0x25,0x45,0xac,0x6f,0x12,0x02,0x06, -0xac,0x44,0xfb,0x2d,0x65,0xab,0xe6,0x42,0x15,0x13,0x61,0x5b, -0x96,0x61,0xf7,0x3b,0x0b,0x70,0x18,0xb7,0x58,0xc8,0xbd,0xc9, -0xd1,0xde,0xef,0xf7,0xfe,0x7c,0xf8,0x28,0x87,0xb6,0x8a,0xe3, -0xd8,0xbb,0x5e,0x5c,0xea,0xe0,0xe2,0x9b,0x12,0x40,0x31,0xa4, -0xad,0xee,0xbf,0x44,0x24,0xa9,0x1c,0x2f,0x07,0x2e,0xe4,0xe3, -0xab,0x86,0x62,0x05,0xa7,0x0d,0xdb,0x87,0x4b,0x5d,0x14,0x35, -0x58,0x54,0x13,0x7e,0x00,0x74,0xef,0xf8,0x40,0xa6,0x70,0x73, -0x89,0xc8,0x00,0xbe,0x9a,0x8c,0xa1,0xcc,0x0b,0x6e,0xff,0x8a, -0xf9,0xfb,0xd4,0x79,0xfe,0x94,0x19,0xfe,0x94,0xb9,0x5d,0x31, -0xab,0x81,0x7a,0xdd,0x6c,0x2e,0xb7,0x1c,0x12,0x71,0x1a,0x7f, -0xdb,0x04,0xc6,0x99,0xbb,0x6a,0xce,0x3e,0x75,0xb6,0x3e,0x65, -0x9e,0x3e,0x65,0x86,0xaa,0x96,0xff,0x03,0xa4,0xb3,0x58,0xb8, -0xab,0xc0,0x27,0xce,0x81,0x07,0x84,0x94,0x55,0xb4,0xed,0x79, -0xc7,0x7c,0x32,0xfc,0x26,0xc6,0xf1,0xad,0x5c,0xa3,0x2d,0x8f, -0xa4,0x1e,0x15,0x5d,0x8a,0xbd,0xf2,0xd3,0xd1,0xdb,0xa7,0x32, -0x08,0x32,0x7b,0xf4,0x48,0xcf,0x70,0x9e,0x82,0xe8,0x56,0x3d, -0x6c,0x1a,0xdb,0x4a,0xae,0x17,0x64,0xf0,0x5d,0x9b,0xdf,0xf4, -0x06,0xff,0x82,0xce,0xa5,0x6a,0xbf,0xba,0x7b,0xf7,0xf6,0x77, -0xde,0x3c,0xb9,0x83,0xa9,0x8a,0x47,0xbb,0xd8,0xe6,0x7a,0x52, -0x27,0x87,0x58,0x57,0x76,0x73,0x21,0x8b,0xba,0x09,0xeb,0xf5, -0x07,0xe3,0x47,0xfb,0x0b,0x16,0x3b,0xb5,0xdf,0x66,0x51,0x82, -0x8f,0x1b,0xdd,0x81,0x28,0xc4,0xc1,0x9c,0x81,0xc8,0xc8,0x28, -0xd7,0x66,0xc1,0x9e,0x6a,0x38,0xb8,0xf8,0x18,0x99,0xf2,0xce, -0x07,0xad,0xdb,0x90,0xda,0x39,0x1a,0xaf,0xf9,0x8a,0x0d,0x2e, -0xe1,0xe1,0x76,0xb9,0xb6,0xdd,0xe5,0x0d,0x96,0x52,0x78,0x77, -0x25,0xda,0xe5,0xba,0x79,0x2d,0xcc,0xb2,0xc2,0x3e,0x6b,0xc9, -0x4b,0xd0,0xfa,0xf8,0x1c,0xba,0xf3,0x2e,0x8f,0x23,0xca,0x16, -0x05,0xb5,0xa9,0x0e,0x25,0x60,0xe7,0xe3,0xcf,0x4f,0xe8,0x53, -0x31,0x55,0x95,0xff,0xc9,0xe3,0x86,0x27,0x03,0x12,0x61,0x2e, -0x7b,0xb0,0xf7,0x0e,0xd2,0x5e,0xb9,0x98,0xb6,0xbb,0xd7,0x72, -0x00,0x12,0x32,0x18,0x60,0xa1,0x21,0x7d,0xb9,0x3c,0x1e,0x12, -0x66,0x52,0x70,0xe9,0xc4,0xf0,0xc1,0x21,0xfa,0x8a,0xe1,0x28, -0xf7,0xe8,0xc3,0xc3,0x52,0x31,0x02,0xa5,0xae,0xff,0xaa,0xc5, -0x40,0xc6,0x2f,0x73,0xbd,0xb0,0x7c,0x92,0xa0,0xa5,0x4b,0x77, -0x5c,0xd9,0xd4,0x77,0x93,0xe6,0xf9,0x17,0x78,0x14,0xe6,0xaf, -0xb2,0x58,0x81,0x2f,0x1f,0x86,0x0f,0x14,0xe9,0x86,0x02,0xb6, -0x6c,0x74,0x6e,0xc6,0xaa,0x77,0xa6,0x64,0x4b,0xe8,0xde,0xde, -0x51,0xa5,0x74,0x45,0x45,0xf7,0x5b,0xde,0x50,0x4e,0x89,0x66, -0xd4,0x55,0x94,0x00,0x26,0xdf,0x15,0xe2,0x77,0x88,0xbd,0x89, -0x24,0xf7,0xb1,0xc5,0x05,0xba,0x61,0x6d,0x6b,0xb1,0x90,0xe1, -0x23,0xe5,0xf5,0x5f,0x6d,0x43,0xa4,0xc3,0xd5,0xb9,0x41,0xaa, -0xd7,0xd7,0xe4,0x31,0x5a,0xe0,0x83,0x0d,0x32,0xe8,0x23,0x41, -0x89,0x34,0xee,0x34,0x50,0x6a,0x4c,0x24,0x3b,0x1b,0x53,0x92, -0x48,0x7d,0xd1,0xee,0x41,0x50,0xd7,0x11,0x96,0x3c,0xa1,0x74, -0xda,0xe5,0x10,0xc7,0x77,0x44,0x81,0x5c,0x65,0xd5,0xfa,0x7c, -0xb6,0xe6,0x22,0x1c,0x97,0xf0,0x04,0x20,0xab,0xb2,0xdc,0x7b, -0xdf,0x6a,0xaa,0x9c,0xbd,0x72,0x95,0x08,0xd1,0xae,0xad,0x45, -0x49,0xe0,0xa0,0xcd,0xe7,0x67,0x15,0x05,0x86,0x8d,0xf7,0x55, -0x05,0x6d,0x1e,0x9b,0x51,0x29,0xf4,0x18,0xf4,0xe4,0x5c,0xaf, -0x4b,0x19,0xa6,0xcf,0x8f,0x34,0x69,0x24,0xc6,0x55,0xb9,0x2c, -0x20,0xa3,0xfc,0x48,0x79,0xdb,0x5f,0x6d,0x19,0x6f,0x76,0xf0, -0x84,0xdd,0xc3,0xc6,0xb8,0x4d,0x99,0x64,0xd8,0xc8,0x6e,0xc8, -0xd9,0x08,0x76,0x30,0xfd,0x2b,0x48,0x4d,0x9f,0x66,0xf8,0x1a, -0xc8,0xa3,0xb7,0x23,0x79,0x7c,0x36,0x41,0x17,0x47,0x01,0x3e, -0x11,0xd3,0x61,0xf8,0xb6,0x17,0x22,0x90,0x84,0xf8,0x4b,0x44, -0xdd,0x71,0x3f,0x6b,0xd5,0x39,0xa6,0xdd,0xaa,0x43,0x10,0x9f, -0xdb,0xd9,0x3c,0x78,0x32,0x34,0xee,0x2b,0x52,0xee,0x38,0x02, -0x58,0x85,0x5a,0xaa,0x49,0x6d,0x55,0x26,0x71,0x9f,0xce,0x83, -0xaa,0x8c,0xce,0x86,0xf7,0x6d,0x24,0x26,0xb8,0x97,0xb9,0x9e, -0x9d,0x59,0x0d,0xcf,0xa2,0xda,0x81,0xc4,0x77,0x44,0xcd,0x33, -0xd0,0xd2,0x48,0x45,0x27,0xd0,0xbf,0xc0,0xfd,0x06,0x78,0xca, -0xa0,0x8e,0x83,0x77,0xd3,0x1b,0xc1,0x84,0x7e,0x5b,0x81,0x65, -0x05,0x40,0x80,0xe5,0x55,0xbe,0xc8,0xee,0xd0,0xe6,0x95,0x3a, -0xa9,0xbf,0x76,0x45,0xb8,0x40,0x13,0x2d,0x5f,0xee,0x5e,0xf7, -0x26,0x78,0xe2,0x1a,0x3f,0xdf,0x5e,0x5c,0x5c,0x93,0xf6,0x03, -0x06,0x59,0x51,0x62,0x09,0x88,0x58,0x1c,0x7e,0xaa,0x68,0xd9, -0xa3,0x38,0xf4,0x36,0xfe,0xfe,0x35,0x38,0x6c,0x3c,0x7f,0x5e, -0xaf,0x46,0xa2,0x1d,0x0d,0x90,0x14,0xa3,0x37,0x51,0xd8,0xf1, -0xe4,0x13,0xca,0x76,0x4e,0xa1,0xb3,0xd4,0x60,0x14,0x9a,0xc0, -0x22,0x22,0x9a,0x62,0x30,0x1f,0x7e,0x79,0x3b,0x98,0xc1,0xcf, -0x7f,0xce,0xf3,0xc9,0x17,0x23,0x6f,0x7a,0xf7,0xc7,0xfd,0xf9, -0x30,0x37,0x23,0xc0,0x01,0x4d,0x69,0xbd,0x37,0xa4,0x28,0x06, -0x7c,0x80,0x35,0xc6,0x0c,0x4e,0x6e,0x0c,0xe1,0xaf,0x71,0xa6, -0x8c,0x4b,0xf4,0x66,0x53,0x0c,0x1d,0x70,0x69,0x7f,0xd3,0xa3, -0x2b,0x74,0x4e,0xca,0x95,0x65,0xd6,0xe0,0x81,0x84,0x4d,0x18, -0x44,0xaf,0x56,0x41,0x44,0x76,0xbf,0x04,0x5e,0x3d,0x1f,0xf4, -0xb3,0xd3,0x68,0x2b,0x32,0xf2,0xaf,0xdb,0xc6,0xe6,0x88,0xc5, -0x2c,0xfc,0x5c,0x61,0x2e,0xcb,0x19,0xb9,0xaa,0x36,0x90,0xc5, -0x2f,0xb8,0xc8,0xc2,0x0e,0xce,0x4e,0x0f,0x12,0x1a,0x82,0xfa, -0x38,0x67,0x9b,0xcd,0xef,0xda,0xb5,0xc9,0x5d,0xfa,0x2d,0xd0, -0x13,0x83,0xae,0x4e,0x9b,0xf2,0xbc,0x9f,0x6e,0xfd,0x44,0xc7, -0x47,0xc5,0x89,0xd9,0xcf,0xc1,0x4d,0x85,0xcd,0x3e,0xa1,0x19, -0xd5,0xc3,0x90,0xaf,0xb3,0x52,0xb6,0x17,0x9a,0x14,0x11,0x9e, -0x8f,0xba,0xe8,0x6e,0x46,0xd4,0x26,0x97,0xe9,0x57,0x15,0x75, -0x64,0xfd,0xf3,0xd9,0xf4,0xbb,0xcd,0x50,0xc1,0x12,0x62,0xd6, -0x29,0x4a,0x20,0xe3,0xab,0x50,0x47,0x3e,0xd9,0xef,0x5e,0x60, -0x4f,0x7a,0x44,0xed,0xa1,0xac,0xb3,0x45,0xc8,0x61,0x77,0xd5, -0x64,0x4b,0x76,0x24,0x22,0xf5,0x02,0x96,0x13,0xf2,0x88,0xf9, -0xbd,0x23,0xd6,0x1c,0xa1,0x61,0x45,0x98,0x1e,0xa5,0xa4,0x8e, -0x7f,0x46,0x50,0x4a,0x43,0x3d,0xfe,0x46,0xb1,0xc2,0x24,0x2d, -0xc6,0xa8,0x3e,0x5a,0x81,0x75,0x09,0x5b,0x17,0xa1,0xd0,0xf6, -0xf0,0x68,0x7c,0x8f,0xdf,0xed,0xfc,0x71,0xef,0xfd,0xee,0xce, -0xf1,0x9e,0x6c,0xb6,0x32,0xc7,0xaf,0xc4,0x10,0x65,0x06,0x42, -0xd2,0x70,0x98,0x46,0xad,0xe6,0x76,0x73,0x6b,0x63,0x72,0xd1, -0xdc,0x46,0x8b,0xbb,0x1f,0xc6,0x93,0xb4,0x85,0x27,0x44,0xf0, -0xbb,0x6d,0xfa,0xe3,0x59,0x8a,0x3a,0x04,0x52,0x5b,0x74,0xde, -0x9b,0xe4,0xd7,0xbd,0x51,0x7f,0x03,0x28,0xf0,0x86,0x5e,0x79, -0xe1,0x29,0x78,0xb4,0x84,0x49,0xbe,0x8b,0x0e,0xc3,0xb2,0x3f, -0xfc,0x27,0x4e,0xf6,0xe6,0x05,0x05,0xee,0x31,0x9e,0x2c,0x00, -0x4a,0x74,0xfe,0xf9,0x16,0x0a,0x8f,0x51,0xdc,0xdb,0x88,0x1a, -0xb1,0xd5,0x3a,0x4e,0xec,0xd3,0x88,0x38,0x81,0x02,0x6f,0x60, -0x41,0x43,0xd1,0x89,0x2e,0x8e,0xec,0x8d,0x10,0xba,0xcf,0xe5, -0x39,0xea,0x2c,0x9d,0x14,0xb3,0x94,0x06,0x9f,0x8d,0x7b,0x5f, -0x8e,0xfc,0xad,0x51,0xb1,0x48,0x2c,0x39,0x70,0xef,0x58,0x1e, -0x86,0x8a,0xf2,0xb3,0x19,0x34,0x4c,0x6e,0xb4,0xd0,0x66,0x95, -0xcd,0xb1,0x34,0xbc,0x6a,0x3e,0x86,0x73,0x45,0xae,0x27,0xa0, -0x5d,0x51,0xea,0x6b,0x31,0xef,0xe7,0x2b,0x31,0x3f,0x3e,0xdc, -0xdb,0x7d,0xb3,0xf3,0xf6,0xfd,0xee,0xeb,0x9d,0xa3,0x63,0x47, -0x2c,0xf1,0xe9,0x59,0x7a,0xb6,0x71,0xf6,0xbe,0xdb,0x88,0x9b, -0x09,0x6a,0x3b,0xef,0x1f,0xfc,0xf5,0xfd,0xeb,0x9d,0xdd,0x3f, -0xba,0x1c,0x3f,0xdf,0x8c,0xff,0x1e,0x5b,0xdb,0x7b,0xcc,0xe4, -0x0b,0x3c,0x98,0x23,0x81,0x0f,0xf3,0xf0,0x1f,0x3a,0x33,0xf8, -0x42,0x0e,0xca,0xd3,0x39,0xb9,0x74,0xf7,0xed,0x40,0xd4,0xfd, -0xcd,0xf3,0x64,0x70,0x75,0x45,0x1a,0xcf,0xe4,0x78,0x0e,0x77, -0x65,0xf8,0x4b,0xb7,0xe6,0x64,0x20,0x8a,0x1f,0x73,0x4f,0xd8, -0x3b,0x40,0x84,0x1a,0x76,0xa8,0x62,0x83,0x27,0xb6,0xaa,0xe3, -0x7b,0x68,0x4d,0x0e,0x9d,0xa1,0x4f,0x99,0xd6,0xd0,0x3e,0x96, -0x40,0x4e,0x52,0x76,0xe2,0x84,0xbb,0xba,0x97,0x07,0xfb,0xbb, -0x6c,0x42,0xfd,0xed,0xb8,0xd7,0xcf,0xfb,0x91,0xb1,0x99,0xa4, -0x15,0xd6,0x4c,0x0c,0xe5,0x46,0x53,0xd7,0x3e,0x07,0x74,0xa6, -0x5d,0x90,0x8a,0x9e,0x3f,0x79,0xbb,0x55,0x16,0x73,0x49,0x31, -0xcc,0x0f,0x1f,0xe9,0xc3,0x59,0x15,0x8a,0x08,0xb8,0x07,0x49, -0x8a,0xa7,0x72,0xc2,0x2b,0x02,0xae,0x01,0x09,0xb7,0x11,0x75, -0x61,0xd6,0xe5,0xbf,0x28,0x5f,0xf0,0x2c,0x13,0x3a,0x95,0x6e, -0x08,0xa1,0x6e,0x36,0x8f,0x01,0xd5,0xc3,0x7e,0xa4,0xba,0x49, -0xaa,0xe3,0xa8,0xe1,0xa2,0x05,0x2d,0xe9,0x4b,0x71,0xf5,0x02, -0x33,0x1f,0xf1,0x48,0x69,0x0d,0x46,0x75,0xc6,0x14,0x97,0x67, -0xf8,0x35,0xec,0x1a,0x86,0x82,0xf4,0xb5,0x34,0x3f,0x1e,0x1c, -0xbc,0xdd,0xdb,0x79,0xf7,0x7e,0xe7,0xe4,0xe4,0x08,0x0d,0x5b, -0xd9,0xf6,0x45,0x37,0xf3,0xe1,0x6c,0x00,0xe3,0x21,0x1b,0xbf, -0x1c,0x7d,0x93,0xd3,0xfe,0xd5,0xf4,0x07,0x53,0xb2,0xd6,0x6a, -0x70,0x6c,0x0e,0x46,0xc3,0x2f,0x86,0x6d,0xd6,0x43,0x0c,0x30, -0xd0,0x51,0x24,0xaf,0x48,0x61,0xed,0x4d,0x4a,0x5a,0xfd,0xba, -0xba,0x53,0xbf,0x38,0x72,0x6a,0x57,0x54,0x17,0x58,0x06,0xb1, -0x59,0xf7,0xde,0xee,0xed,0xef,0xbd,0x3b,0x39,0x0e,0xb0,0x63, -0x95,0x79,0x46,0x0d,0x6a,0x25,0x59,0x1f,0x0f,0xc5,0x80,0xcd, -0xf5,0xcc,0xf9,0x7c,0x36,0x83,0x30,0x6e,0xa8,0x0d,0xce,0x1f, -0x74,0x3d,0xf2,0x14,0xa4,0x6c,0x4d,0xa7,0x7e,0x6d,0xb0,0x88, -0xad,0x6d,0x2d,0x13,0xa7,0xea,0x71,0xdf,0xef,0xcd,0x7a,0x29, -0x53,0x15,0xb0,0xc1,0x9e,0x19,0x8c,0xae,0xf3,0x09,0x04,0xfa, -0x2f,0x7d,0xc2,0x1b,0x1d,0xc7,0x6e,0xba,0x56,0xaf,0x17,0x15, -0x45,0x1c,0x4b,0x40,0xcb,0xcb,0x68,0x7c,0x2e,0x71,0x6a,0x82, -0xc3,0x7c,0x22,0x95,0x78,0x95,0x4e,0x63,0x0d,0x03,0x7d,0x6b, -0x25,0xce,0x28,0x91,0xe3,0x89,0x68,0x33,0xab,0xcc,0xa1,0xd8, -0x1c,0x51,0xc8,0xe1,0xbc,0x75,0x2d,0x4a,0x5d,0xe2,0xeb,0x6d, -0xda,0x87,0x08,0x9a,0xaf,0x25,0x68,0x2e,0x20,0xaa,0x12,0xa2, -0xbd,0x9a,0x17,0xb7,0x5a,0x99,0xb7,0xde,0x4a,0x10,0xe9,0x5d, -0x4b,0xa0,0x08,0xe3,0x66,0xa7,0x9d,0x98,0xbf,0x75,0x06,0xc4, -0x20,0xd2,0xad,0xf7,0xd6,0xb1,0x13,0x5b,0xcd,0xac,0x8a,0x95, -0xbd,0xa3,0xbd,0x27,0x92,0xf2,0x51,0x6f,0x3e,0x1b,0x47,0x09, -0x1d,0x5f,0x62,0xf4,0x62,0xe1,0xa4,0x04,0x55,0x28,0xac,0xd3, -0x82,0xe8,0xb8,0x9d,0x2a,0x4a,0x3c,0x04,0x62,0x59,0x51,0x5a, -0x88,0x9d,0xb6,0xa7,0x0f,0x77,0x4a,0xb0,0x1b,0xeb,0x57,0x5a, -0xe6,0x85,0x7e,0xab,0x9e,0x5a,0x94,0xbd,0x6b,0x9f,0x95,0x14, -0xfa,0x30,0x94,0x56,0xac,0x4b,0xab,0x58,0x1f,0x77,0xd2,0xb0, -0x4e,0x71,0x9d,0x47,0x40,0xfd,0x37,0xb3,0xfc,0x86,0xab,0x64, -0xe7,0xfc,0xf8,0x64,0x3c,0xbf,0x18,0x5c,0x0e,0xf2,0x7e,0x27, -0xac,0x33,0xf5,0xfb,0x75,0xb9,0x7e,0x0a,0xaa,0xc1,0x45,0xc0, -0x75,0x8a,0xb6,0xcf,0xc6,0x6e,0xfe,0x02,0x50,0x78,0x2f,0x57, -0x28,0xdc,0x32,0xab,0x28,0xb0,0x58,0x94,0x95,0xec,0xe8,0x28, -0xa3,0xd0,0xfa,0xd5,0xd5,0x73,0x86,0xb6,0x2d,0x69,0x33,0x5e, -0xa9,0x8c,0x3c,0x2a,0xd0,0x81,0x59,0x55,0x2a,0x83,0xd9,0xf6, -0x5e,0x3a,0x58,0x51,0x0b,0x42,0x8a,0x38,0x20,0xb4,0x44,0x87, -0x7a,0xe3,0xdb,0x87,0x87,0xdf,0x49,0xde,0x21,0xfe,0xc5,0x0e, -0xa5,0x58,0x73,0x87,0xe6,0x80,0xb7,0x9c,0xb0,0xca,0x69,0x4b, -0xe2,0x8d,0x7a,0x69,0x73,0x6b,0x34,0x0a,0x70,0x90,0xe6,0x2a, -0x56,0x24,0x87,0x65,0xf0,0x44,0x20,0x7b,0x77,0xf0,0x72,0xef, -0xfd,0xc9,0x5f,0x0e,0xe1,0xcf,0xde,0x9f,0x4f,0xde,0x1f,0x1e, -0x1d,0x1c,0xee,0x1d,0x9d,0xfc,0xe5,0x54,0x0b,0xb5,0x78,0x08, -0xdc,0x6d,0xaf,0xd4,0x5b,0xeb,0x58,0x60,0x56,0x12,0x3e,0xb5, -0x11,0xdd,0x34,0x8a,0xfc,0xf0,0xba,0xd8,0xb0,0x39,0x24,0x29, -0xac,0x40,0x43,0x2d,0xce,0xb2,0x99,0x88,0x57,0x21,0xdc,0xea, -0x66,0xb0,0x72,0x80,0x14,0x76,0x42,0xde,0x68,0x56,0x65,0x7b, -0x0e,0xd9,0xb0,0x4d,0x74,0x2e,0x0a,0xfb,0x96,0x07,0xc0,0x3d, -0x04,0x02,0xdb,0x22,0x22,0x49,0x64,0xa4,0x9f,0x9b,0xeb,0xfd, -0x3b,0x7c,0x76,0x28,0xc1,0x65,0x4c,0xec,0xa1,0x3c,0xfe,0x9e, -0x1d,0x96,0xfb,0x92,0x4d,0xfc,0x1c,0xc3,0x6a,0xb5,0x7b,0x82, -0x2c,0xc7,0x6d,0xa6,0x1c,0xc7,0xf7,0x9c,0xce,0xae,0xe2,0x96, -0x5c,0xa7,0x10,0xae,0x90,0x66,0xdc,0xf1,0x38,0xad,0xa4,0x4a, -0xb0,0xe1,0x88,0xe4,0x9e,0x7f,0x9b,0x56,0x16,0xc0,0x63,0x53, -0x04,0xc5,0xf2,0x8e,0x24,0xde,0xb1,0x1a,0xa2,0x84,0xd8,0xe0, -0x17,0xbf,0x29,0x97,0xcc,0xa2,0x41,0x41,0xf7,0xf0,0x1c,0xb5, -0x2c,0xec,0x97,0x98,0x25,0x06,0x21,0xcb,0x26,0xd1,0x74,0xfe, -0x57,0xf6,0x53,0x01,0x38,0x8d,0x3c,0xee,0x07,0xb5,0xf5,0x08, -0x43,0x0e,0xf0,0xd0,0x14,0xe3,0xd4,0x2f,0x1a,0x2e,0xaa,0x3d, -0xf8,0xc1,0x07,0xf4,0x73,0x12,0x59,0xfc,0xf3,0xde,0x70,0x7c, -0x11,0xfb,0x2c,0xf4,0xa2,0xa4,0x54,0x9f,0x34,0x61,0x69,0x94, -0x98,0x2c,0xeb,0x68,0x51,0xa0,0x96,0xb9,0xac,0xde,0x9c,0x5d, -0xb5,0xd0,0x89,0xe1,0xb6,0xf5,0x31,0x86,0xc6,0x9a,0xa0,0xb5, -0xf1,0x76,0x96,0x5d,0x8e,0xdc,0x13,0xba,0xcb,0xd1,0x5a,0x96, -0x85,0x6b,0x6d,0x10,0xe9,0xe5,0x84,0x0e,0x42,0x4c,0x09,0xa2, -0xf6,0xa5,0x23,0x5d,0x28,0x87,0xbd,0x98,0x27,0xe1,0x37,0x6b, -0xfc,0x9c,0x46,0x09,0x9b,0xd6,0x60,0xd7,0xe5,0x28,0xb3,0xd0, -0x51,0x92,0x48,0x40,0xa2,0x67,0x59,0x95,0x9c,0x2e,0xb6,0x84, -0x7b,0x22,0x08,0x31,0x98,0x45,0xb1,0x2a,0x17,0x44,0x53,0x4e, -0x36,0x47,0xa5,0x7d,0xf7,0x3b,0x43,0x4f,0x4c,0x00,0xd0,0x4e, -0x98,0x35,0xe6,0xc3,0x07,0x7b,0x77,0xe5,0xb0,0xee,0x38,0x6b, -0x2a,0x0a,0x41,0x83,0x4f,0xc0,0x54,0x90,0x2c,0x44,0x7d,0x78, -0xf1,0xc1,0xdb,0x85,0x72,0x73,0x3b,0xb3,0xe8,0x7c,0x60,0xa4, -0xb9,0xa7,0xdb,0x8a,0xee,0xf8,0x7c,0xa3,0xe1,0x4a,0xa4,0xee, -0x6b,0x19,0x78,0x1e,0x58,0xdd,0x55,0x85,0x7e,0x91,0x2a,0x74, -0x6b,0xb5,0x64,0xc9,0xab,0x99,0x12,0x23,0x8f,0x5c,0x04,0x88, -0xb2,0x48,0x6d,0xa9,0x26,0x3d,0xa3,0x1c,0xb6,0xd6,0xc6,0xa3, -0x57,0xa5,0x8d,0xa4,0x99,0x8f,0xc4,0xb1,0x85,0x1b,0x63,0xbb, -0x8a,0xe8,0x14,0xd1,0xc3,0xd3,0x9b,0x41,0xdc,0x6d,0x02,0xae, -0xe8,0x3e,0x83,0xa3,0xff,0x0d,0x17,0x8e,0xb2,0xbf,0x0c,0xf4, -0x64,0xf1,0x37,0x7b,0x4a,0xfd,0xb7,0xda,0x78,0x52,0xfb,0x1b, -0x3b,0xf6,0x04,0x94,0xff,0x86,0x36,0x43,0x80,0xa0,0xf1,0x52, -0x2b,0x92,0xcb,0x67,0x4c,0x9b,0x0a,0xe1,0xec,0xf1,0xb1,0xc2, -0x31,0x94,0xcc,0xbd,0x28,0xca,0x59,0x22,0x34,0x7e,0x8d,0x96, -0x81,0x1f,0xcc,0xcb,0x59,0xac,0x75,0xea,0xe9,0x62,0xf1,0x04, -0xc0,0x46,0x70,0xb8,0x5f,0xda,0x2a,0x1e,0x2e,0x26,0x75,0x58, -0x74,0xc4,0x50,0x2b,0xc2,0x70,0xa6,0x8b,0x25,0x2b,0x03,0x4e, -0xfc,0x78,0xd2,0x49,0x9c,0xec,0x3c,0x6a,0x7a,0xe7,0x41,0xde, -0xa6,0xee,0x5d,0x87,0xbc,0x1a,0x4d,0x59,0xcb,0x77,0xca,0xae, -0xa7,0x48,0x71,0xd2,0xa6,0x30,0xbf,0xbf,0x19,0xcf,0xa7,0x79, -0x8e,0xce,0x49,0xc4,0x44,0x21,0x3e,0x50,0xc7,0xb8,0x61,0xde, -0xbb,0xcb,0x25,0xce,0xf9,0x72,0x83,0x4d,0x0f,0x80,0x0c,0x6d, -0xa1,0xdb,0x68,0x65,0x6e,0x5c,0xe2,0xe9,0xb1,0xdd,0x4b,0x89, -0x3c,0x1c,0xb3,0xe9,0xec,0xc2,0xfb,0x4f,0x7a,0x68,0x87,0x7a, -0xca,0xd9,0xef,0x60,0xe2,0xf7,0xdc,0xa2,0xdf,0xe9,0xb9,0x53, -0x12,0xab,0x7d,0x89,0x3b,0xb0,0xdb,0xec,0xbc,0x5e,0x3f,0x97, -0x77,0x52,0xc8,0x1b,0xdd,0x43,0x3b,0x28,0x0d,0xc9,0x8b,0xc5, -0x5a,0xbc,0x46,0xbf,0x68,0x7e,0x0a,0x3e,0x1c,0x40,0x4c,0xa1, -0x8a,0x1c,0xc2,0x9d,0x30,0x18,0x43,0xee,0x24,0xed,0xad,0xc2, -0xbb,0x5e,0x6f,0xfd,0xa6,0xbe,0x32,0x95,0x0a,0x27,0xfa,0xe8, -0x85,0x5a,0x87,0x8e,0x03,0xd8,0xaf,0xe9,0x79,0x16,0x60,0x8d, -0xac,0xee,0x1c,0x1b,0x5c,0xb6,0x46,0xb8,0x6c,0xeb,0x21,0xb3, -0xe7,0xb3,0x14,0x77,0xd3,0xbb,0xcd,0xee,0xfd,0xe8,0xa4,0x3c, -0x52,0x63,0xd4,0x97,0xf6,0xe3,0x68,0x63,0xef,0x60,0x48,0x97, -0xed,0x60,0xf6,0x5a,0x20,0x0c,0xba,0x52,0x31,0x5c,0xbb,0xe2, -0x9d,0xe4,0x43,0x34,0xac,0x23,0x8a,0xe2,0x12,0x3a,0xa1,0x0c, -0xed,0x78,0x4d,0xc2,0xa8,0xe0,0x4c,0x1f,0xda,0x65,0xab,0xeb, -0x53,0xf1,0xca,0x2b,0x59,0xf0,0x21,0x28,0x93,0x3b,0x57,0xc9, -0x3e,0xe7,0x96,0x22,0x4c,0x97,0xcf,0xf8,0x42,0xb6,0xc3,0x25, -0x13,0x53,0xec,0x9e,0x4a,0x52,0x5f,0xda,0x58,0xb9,0xb7,0x1e, -0x61,0x35,0x68,0x0b,0x47,0x18,0xdd,0xc1,0xe5,0xa5,0x91,0xb3, -0xe4,0xff,0x1a,0xcc,0xae,0xcb,0x22,0x80,0x24,0xe2,0x68,0xc9, -0x22,0x49,0xd7,0x6d,0xe2,0x87,0x3e,0x7c,0xb0,0x47,0x43,0x1a, -0xac,0xdd,0x56,0x64,0x72,0x53,0x16,0xcf,0x36,0xe5,0xc0,0x48, -0x43,0xf6,0x63,0x30,0xa2,0x8a,0xa8,0x92,0x8e,0x78,0xeb,0x47, -0x5f,0x6a,0x93,0xd9,0x8f,0xf9,0x25,0xf2,0x0d,0x4c,0x37,0x94, -0xdc,0xd4,0x0e,0x70,0x53,0xc9,0x2b,0x40,0x77,0x51,0x66,0xe0, -0xbc,0x0e,0x05,0x2a,0x44,0xe2,0x1c,0x76,0x01,0x49,0x15,0x50, -0xa4,0x62,0x43,0x4f,0x93,0x5c,0x92,0x1f,0x90,0xe6,0xbc,0x5c, -0x62,0xca,0x30,0xf0,0x15,0x41,0x51,0x80,0xaf,0xd7,0x2d,0x54, -0x51,0xa8,0x91,0xcc,0x78,0x55,0x2b,0x09,0x7c,0x08,0x81,0xc3, -0xf7,0xf8,0xe9,0xb7,0xaf,0x1f,0xef,0x39,0x60,0xa7,0x7b,0x7b, -0x9b,0x8f,0xfa,0x15,0x9b,0x1d,0xea,0xd1,0x8a,0x01,0x18,0x85, -0x3d,0x4f,0xf0,0x92,0xfb,0xb8,0x0a,0x75,0xe0,0x23,0x55,0xd1, -0x5a,0x79,0x87,0x6a,0xe7,0x7e,0x67,0x48,0xd8,0xcb,0xb7,0x30, -0x1e,0x0f,0x20,0xc5,0x06,0x70,0xcb,0x60,0x3d,0xa1,0xb9,0xb4, -0xcb,0xc1,0x64,0x3a,0x23,0xf0,0xed,0x27,0x37,0xc5,0x8b,0x8a, -0x8a,0x80,0x28,0x8d,0x89,0x81,0xae,0x6b,0x0d,0xba,0x0d,0x2c, -0x23,0x88,0xb1,0x4c,0xf4,0xf6,0x2b,0x93,0xf3,0x41,0x97,0x82, -0xd6,0xda,0xc4,0xc0,0xf7,0x8a,0xb9,0xc0,0x9f,0x64,0x90,0xbb, -0x44,0x9c,0x16,0x8c,0x27,0x50,0x1b,0x13,0xf4,0xa5,0x23,0x12, -0x39,0x36,0xaa,0xa0,0x8b,0xca,0x19,0xf7,0xb5,0x98,0xd1,0xe5, -0x76,0xa1,0xc6,0xde,0x25,0xf2,0xd1,0xf2,0xd8,0xe5,0x9f,0xf6, -0xf4,0x5c,0x09,0x06,0xea,0x01,0xce,0x50,0x35,0x6e,0x1e,0x54, -0x91,0x05,0x7c,0xc5,0xe4,0x2f,0x4c,0x6d,0x7b,0x97,0x2f,0x2c, -0x6e,0x47,0x82,0x46,0xdd,0xe0,0x07,0xe2,0x1f,0xa7,0xf2,0xc3, -0x3f,0x4e,0x2d,0x35,0xd9,0x4a,0x60,0x38,0x3d,0xfb,0xec,0x1e, -0x25,0x50,0x67,0xf3,0xd1,0x68,0x3a,0xd4,0x06,0xb2,0xb5,0x70, -0xd3,0x50,0x02,0x07,0xb2,0x8c,0xcf,0xdd,0x09,0xd1,0x2d,0xa3, -0x98,0x94,0xcb,0x2f,0xa5,0xbb,0x57,0x70,0xb1,0xd5,0x63,0xe1, -0x8d,0xce,0x11,0x19,0xb4,0x50,0x4c,0x90,0x1e,0x77,0xa2,0x87, -0x80,0xa6,0x57,0x55,0xa4,0x90,0x57,0x51,0x8b,0x3a,0xbf,0xc1, -0x1c,0x32,0x98,0x76,0x60,0x8a,0xf7,0x75,0xa5,0x1c,0x6e,0xaf, -0x98,0x0f,0xfd,0xeb,0x6f,0x35,0xb4,0x6d,0xac,0x1c,0xef,0x17, -0x6f,0x00,0xc7,0x35,0xe4,0x14,0x37,0x0e,0xbf,0x76,0xc2,0x85, -0x6e,0x82,0x02,0xae,0xca,0x1b,0xd8,0x10,0x0e,0xaa,0x58,0x4f, -0x49,0xd5,0x4a,0x1d,0x31,0x09,0x76,0xd3,0x1f,0xbf,0x9c,0xf4, -0xae,0x70,0xd7,0x1f,0xab,0xae,0xbe,0x18,0x8e,0x47,0xb9,0x3d, -0x1a,0xc6,0x6f,0x7b,0xc3,0x21,0x82,0x6a,0xb9,0x2a,0x5a,0x7c, -0x11,0x8c,0x71,0x42,0x7b,0x41,0x2c,0x8d,0x9f,0x22,0xaa,0x93, -0x5b,0xa8,0x53,0x07,0x4c,0x96,0x7a,0x84,0x96,0xb9,0x2f,0xc8, -0x53,0xf0,0xf9,0x8f,0x07,0x88,0x66,0x3a,0x1b,0xdf,0xe2,0x89, -0x4f,0xef,0x8a,0x1e,0xa6,0x52,0xe4,0xd2,0xbf,0xa8,0x16,0x34, -0x4c,0x78,0xc3,0x34,0x62,0x4b,0x54,0x41,0x2b,0x08,0x77,0xe2, -0x97,0xff,0xc0,0x36,0x1b,0xff,0x3c,0xf7,0xd6,0x5a,0xf8,0x10, -0xaa,0x7a,0x63,0x57,0xda,0x8c,0xc6,0x77,0xc1,0x9e,0x52,0x6f, -0xf8,0x18,0x2c,0x9f,0x56,0x84,0x6e,0x0c,0xa5,0x90,0x70,0x6e, -0x39,0xb8,0x48,0x3c,0x63,0xa0,0xe5,0x53,0x6c,0xed,0xad,0x86, -0xac,0x9f,0x30,0x85,0x48,0x61,0x01,0xb9,0x78,0x37,0xc5,0x5e, -0x68,0x9e,0x03,0xf9,0x95,0xee,0xf4,0x9a,0xe3,0x51,0x39,0xe7, -0x7c,0xb4,0x22,0xef,0xe5,0x25,0x08,0x07,0xc0,0x3e,0xae,0xf7, -0x7b,0xb7,0xfa,0x5e,0xf0,0x76,0xae,0x26,0x23,0x6e,0xf5,0xe5, -0xa4,0x86,0x1a,0x80,0xee,0x91,0xfe,0x98,0x7f,0xc1,0x78,0x77, -0xe9,0x83,0x47,0x62,0x41,0x89,0xc0,0xeb,0x75,0x58,0xa4,0xbc, -0xd4,0x7c,0xcc,0xbf,0xe8,0xfb,0x3a,0x2a,0x82,0x4f,0x27,0x83, -0x62,0xb6,0x93,0xe4,0x25,0xbf,0xcd,0xd4,0x35,0x72,0x3e,0x43, -0x2b,0xd2,0xab,0x77,0xef,0x77,0x8e,0x7e,0x7f,0x9c,0x6d,0xfe, -0x6c,0x81,0x9f,0x4d,0xbf,0x3b,0xfd,0xf9,0x2c,0xee,0x7e,0x77, -0x16,0xc3,0x67,0x0c,0xdf,0x49,0xf7,0xbb,0xe4,0x2c,0xd9,0xbc, -0x31,0x9c,0x99,0x6d,0x67,0x65,0x9b,0x66,0x53,0x22,0x32,0xd2, -0x83,0x88,0xdf,0xa3,0x71,0xdc,0xe3,0x46,0x27,0x39,0x6b,0xb1, -0x12,0x04,0xda,0xfd,0x3a,0x7c,0xbf,0x7b,0xb0,0xcf,0x17,0x5a, -0x9b,0x71,0x7c,0xb6,0x89,0x6a,0x23,0xeb,0xc9,0x02,0xbf,0xbe, -0x3b,0x3d,0x9b,0x9e,0x1d,0x77,0xbf,0xeb,0x9c,0x7d,0x77,0xb6, -0x89,0x77,0xb7,0x37,0xde,0x31,0x7e,0xe1,0x86,0x56,0x5d,0xd7, -0x98,0x75,0xf6,0x6b,0x54,0xbc,0xc4,0xb5,0xd1,0x98,0x83,0x5d, -0x5f,0x39,0x7f,0xa2,0xe4,0xe3,0x93,0xbf,0x03,0x57,0x9e,0xd6, -0x4b,0x86,0xf8,0x19,0xb7,0x9c,0x05,0xb7,0xca,0x26,0x74,0x6c, -0xe1,0x97,0xc2,0xc0,0x7d,0x86,0x7d,0xb1,0x4a,0x8e,0x29,0xa2, -0x0d,0xef,0x3c,0xac,0x2d,0x17,0x3b,0xf5,0x7a,0xd4,0x8c,0xd6, -0x28,0x5d,0x59,0x9d,0x11,0xe7,0x9d,0x41,0x3b,0xf0,0x01,0xe0, -0xec,0x02,0x50,0x70,0xef,0xff,0xd0,0xa7,0x26,0x29,0xef,0xd5, -0x2c,0x62,0x35,0xfb,0x9c,0xaf,0xf6,0x09,0xb6,0x06,0xb5,0x67, -0xcd,0x67,0xb5,0xab,0xf1,0xac,0xf6,0xec,0x7e,0x6b,0xf9,0xac, -0x19,0xf1,0xdc,0x6f,0x17,0x1b,0x43,0x73,0xdf,0xdb,0xea,0xef, -0xe2,0x4b,0x59,0x53,0x72,0xf8,0x81,0x91,0xce,0xd1,0x47,0xe8, -0xbe,0x43,0x1e,0xf3,0xea,0x9e,0x93,0x28,0xa7,0xf5,0xca,0xfb, -0xbb,0x12,0xb7,0xe5,0xf5,0x8a,0xa5,0x16,0xd3,0x07,0xfe,0x2c, -0xdb,0x6a,0x0c,0x93,0x00,0x47,0x5f,0xf5,0x3a,0xfd,0x9c,0xd2, -0x5f,0x6b,0x4b,0xb4,0xd5,0x35,0x7e,0x71,0xcc,0x42,0xe1,0x88, -0x3c,0xee,0x39,0x20,0x52,0x9c,0xfd,0xd4,0xb8,0x22,0x5c,0xe9, -0x3b,0xbf,0x0e,0x65,0x95,0x05,0xd4,0x42,0xc5,0xcf,0xcb,0xdb, -0x85,0x4d,0x45,0xb5,0x10,0x44,0x65,0xcb,0x82,0x50,0xb1,0x4e, -0xdc,0x48,0x60,0xb3,0xeb,0x75,0xdb,0x63,0x31,0x06,0xcd,0x16, -0x69,0x43,0x18,0xde,0x2b,0x97,0x7a,0x8d,0x3b,0x2a,0xbc,0xef, -0x89,0x1f,0x04,0x54,0x29,0x86,0x56,0xf5,0xbe,0xb8,0xd1,0x9d, -0xa9,0x33,0x9c,0x72,0x2e,0x25,0xaf,0x95,0x60,0x12,0x3d,0xd2, -0xfa,0xc9,0x00,0x5d,0xd8,0x3b,0xa4,0x73,0x51,0x49,0xc7,0x7d, -0xa6,0x81,0x1a,0x6e,0x21,0xd1,0xab,0xc3,0x6a,0xbb,0x38,0xa1, -0x66,0x30,0x15,0x79,0xa8,0x0b,0xb4,0x28,0xf9,0x2f,0xc4,0xba, -0xa4,0x1a,0xfd,0x28,0xe2,0xe2,0x0a,0x96,0xe5,0x80,0x65,0x77, -0xd9,0x35,0xeb,0x78,0x94,0x33,0x18,0x96,0xf8,0x9b,0x44,0x47, -0x49,0x7b,0x7d,0x97,0x3f,0x2d,0x7f,0xf3,0x3a,0x73,0x9e,0xcf, -0xf1,0x4e,0xe8,0xf0,0x68,0xef,0xd5,0x9b,0x3f,0x7b,0x55,0x98, -0xf8,0x33,0xa9,0xcb,0xbc,0xef,0x2e,0xf0,0x4e,0x5f,0xbe,0x93, -0xcd,0x81,0xf9,0xf3,0xeb,0x23,0xab,0x04,0xf4,0xe7,0xfd,0xb7, -0xaf,0x67,0xb3,0xdb,0x23,0xf6,0x75,0xb4,0x58,0x68,0x73,0x00, -0x93,0x2f,0x4e,0x1d,0x1e,0xf6,0x0c,0xec,0xa7,0xee,0xcf,0x72, -0x8a,0x1e,0xed,0x4f,0x3f,0xdf,0x0c,0xb7,0xa9,0xfc,0xc9,0xc9, -0x61,0xf3,0x37,0xcd,0xad,0xc8,0x39,0x45,0x6b,0x25,0xf7,0xcb, -0xaf,0x2a,0xfd,0x5c,0x97,0xde,0xfe,0xca,0xd2,0xbe,0xe4,0x73, -0x2c,0x19,0x3c,0xba,0x5e,0x47,0x3f,0xc3,0x3f,0xf6,0x2e,0x3e, -0xc2,0x8e,0x8e,0xdf,0x5d,0x7f,0xbe,0x46,0xff,0x76,0x27,0xe4, -0xcd,0x98,0x1f,0x0e,0x95,0x8f,0x7d,0xc3,0x4e,0x41,0x87,0xc3, -0xb8,0x56,0xc1,0x9c,0xb9,0x1d,0x0f,0x2b,0xd6,0x22,0x95,0x04, -0xeb,0xd1,0xf1,0xde,0xd1,0x9f,0xf6,0x8e,0xde,0xef,0xef,0x9c, -0xec,0xbe,0xc6,0x41,0x38,0xfd,0x39,0xed,0x36,0x92,0x14,0x17, -0xc3,0xf8,0xec,0x53,0x23,0xbd,0xdf,0x32,0xad,0xe5,0xd9,0xa7, -0xef,0xfe,0x23,0xe9,0xc4,0x67,0xf7,0x9d,0xd3,0xb3,0x4f,0x67, -0xcd,0x0d,0x58,0x7f,0x97,0xb0,0xa0,0xa6,0xf1,0xe9,0xd6,0xc6, -0xef,0xba,0x68,0x0f,0x14,0xd6,0x4c,0x58,0x8c,0x3b,0xff,0x06, -0xab,0x31,0x7c,0x77,0x00,0x0c,0x7e,0xc2,0xf7,0xbf,0xe1,0xeb, -0x79,0x34,0x08,0x6a,0x0e,0x77,0x4e,0x5e,0xeb,0x7a,0x38,0xf7, -0x8a,0xcc,0x2f,0xf7,0x5e,0xed,0xfc,0xf4,0xf6,0xe4,0xfd,0xe1, -0xc1,0x11,0xaa,0x9b,0x90,0xff,0xe5,0xdf,0x6e,0x19,0xfc,0x9d, -0xa6,0xdf,0x7f,0xff,0xdc,0x5c,0x42,0xc4,0x76,0x6b,0xc9,0x36, -0x27,0x91,0x04,0x8a,0xcd,0xb4,0xf1,0x40,0x93,0x6f,0xe5,0x13, -0x65,0xa3,0xf3,0xde,0xe8,0xea,0xcd,0xe8,0xf5,0xec,0x66,0xf8, -0x6b,0xe7,0x07,0x8b,0xc4,0xa4,0x62,0xa6,0x15,0xa9,0xe5,0x82, -0xf7,0xeb,0xeb,0x78,0x21,0xf6,0x6b,0xf4,0x89,0xb8,0xbe,0x2e, -0xdb,0x76,0x0c,0xf4,0xce,0xa7,0x90,0x33,0xb5,0x88,0x88,0x6d, -0xe0,0x68,0x7d,0x9d,0x13,0x22,0x72,0x8e,0xe6,0x39,0x02,0xba, -0x65,0x93,0xd2,0xa5,0x5b,0x34,0x48,0x73,0x77,0x68,0xb2,0x88, -0xa2,0xc7,0x36,0xef,0x15,0xc0,0x77,0x2e,0x2b,0xbb,0x62,0x01, -0x2b,0x63,0x59,0x9b,0x04,0xf5,0x3a,0x15,0x45,0x9f,0x20,0x71, -0x3f,0x47,0x85,0xc4,0x9f,0x8e,0xde,0xe0,0x74,0x25,0x3b,0xfe, -0xde,0x0e,0x2e,0xec,0x33,0xad,0xf5,0x82,0xc5,0xc2,0xc5,0x4a, -0x69,0x71,0x3b,0x62,0xcd,0xe2,0x92,0xe5,0x7d,0x5e,0x93,0x51, -0xbe,0x53,0xf6,0x6f,0x81,0xf3,0xd8,0xd6,0x50,0xfa,0xd2,0x7b, -0x70,0x2b,0xf7,0x88,0x4d,0xc2,0x9b,0x06,0x74,0xe3,0x56,0xce, -0x81,0xd1,0x90,0x8a,0x04,0x5f,0x55,0x1e,0xa2,0x23,0x32,0xb5, -0x70,0x5d,0x48,0x3d,0xf6,0x79,0x7a,0xf8,0xcc,0xdb,0x75,0x37, -0xf9,0x4a,0x91,0x59,0x1b,0x6d,0xa2,0x71,0x3d,0xbc,0xc0,0x72, -0x82,0x50,0x87,0x60,0x41,0x42,0x03,0x3f,0xd0,0xe7,0x30,0x35, -0x5c,0xbb,0x96,0xc6,0xb0,0xa1,0x5b,0x15,0x7e,0xea,0x74,0x2f, -0xfe,0x6f,0x8a,0xce,0xde,0xd9,0xfd,0x4d,0x6d,0x2b,0x0d,0x07, -0x90,0x01,0x90,0x3f,0x96,0x5a,0x2b,0xa5,0x01,0xb7,0xea,0xe9, -0x94,0x82,0xb2,0x98,0xca,0xc8,0x8f,0x71,0x41,0x80,0xe6,0xc7, -0x4f,0x92,0xc7,0x99,0x43,0x5a,0x73,0x37,0x86,0xbe,0x34,0x49, -0x72,0xe1,0xfc,0x88,0xa3,0xc1,0x74,0x82,0x8d,0xbc,0x22,0xb6, -0x92,0xd7,0xe8,0xa8,0xcc,0xf9,0xa7,0x47,0x27,0xe8,0x74,0x8b, -0xe4,0x8a,0xfd,0x9b,0x0c,0x78,0xf2,0xb7,0x1a,0xee,0xf3,0x6a, -0x37,0x73,0xc8,0x4e,0x46,0x1f,0x58,0xf9,0x1f,0x6f,0x99,0x7a, -0xe4,0xa9,0x08,0xb5,0xca,0x23,0x25,0xe3,0x11,0xd2,0xfc,0xb3, -0x0c,0x3d,0xc9,0xf8,0x3e,0x53,0xdb,0x23,0x6b,0x42,0xc7,0x27, -0x76,0xd4,0xe6,0xc0,0x41,0x3c,0xe5,0x9f,0x6e,0x5a,0x19,0xab, -0x4a,0x2f,0xc3,0xde,0xc6,0x15,0x6a,0x4c,0xe6,0x2b,0x99,0x18, -0x91,0x5a,0x57,0x92,0x0a,0x26,0x46,0xc6,0x7b,0x2d,0xb7,0x93, -0xba,0xac,0xeb,0x2d,0xd0,0x25,0x03,0x6a,0xd2,0xf0,0x3d,0x22, -0x4d,0x4d,0x7e,0x06,0x56,0x64,0x4c,0xec,0xd1,0x3f,0x31,0xa8, -0x9e,0xb0,0x73,0x72,0x70,0x04,0xec,0x2d,0xc2,0xa6,0x47,0x15, -0xe0,0xf9,0x38,0x05,0xbd,0x66,0xe7,0x15,0xc9,0xe8,0x4c,0x2c, -0xa2,0x67,0x63,0x55,0x89,0xc0,0x1c,0xbd,0x82,0x0b,0xed,0xef, -0xd1,0x2a,0xba,0xa2,0x5f,0xe7,0xeb,0xdc,0x68,0x7b,0x8f,0xbd, -0xac,0x17,0xf8,0x41,0x37,0xe7,0xd9,0x79,0x18,0xe1,0xef,0x2d, -0x7b,0x49,0xc7,0x07,0xce,0x93,0x4e,0xaf,0x71,0x9e,0xf6,0xd2, -0x20,0xea,0xdc,0x2b,0x1f,0x2d,0xd1,0xc0,0xe0,0x3f,0x03,0x81, -0x38,0xc0,0xa0,0x97,0x6e,0x25,0x1b,0x71,0xa1,0xd2,0x2d,0x58, -0x04,0xa3,0xef,0x9e,0x52,0x5b,0x00,0xfa,0xbb,0xb0,0x26,0x80, -0xb1,0xf9,0xd5,0x30,0x36,0x4b,0x30,0x7e,0xf5,0xd5,0x30,0x7e, -0x55,0x82,0xf1,0xf3,0x57,0xc3,0xf8,0xb9,0x04,0x23,0x8b,0x84, -0x0e,0xb2,0x2c,0xfb,0x6a,0x70,0x78,0x57,0x58,0x04,0xb8,0xf6, -0x0d,0x70,0xd6,0x2a,0xe0,0x7c,0x13,0x3a,0x15,0xd8,0x7c,0x03, -0x32,0x25,0x28,0x3f,0x7c,0x35,0x90,0x1f,0x4a,0x30,0x5e,0x7c, -0x35,0x8c,0x17,0x65,0x3c,0xbe,0xbe,0x35,0x3f,0x94,0x5b,0xf3, -0xe2,0xeb,0xa1,0xbc,0x28,0x43,0xa9,0xd7,0xbf,0x1a,0x4a,0xbd, -0x5e,0x82,0x02,0x32,0xc1,0xd7,0x42,0x59,0x2c,0xca,0xb8,0x7c, -0x3d,0x2a,0x65,0x4c,0x9e,0x00,0x23,0x2c,0x14,0x66,0x0b,0x92, -0x88,0xf4,0x56,0x01,0x74,0xdc,0x38,0x2c,0xb3,0x5c,0x9a,0xbd, -0xe3,0xdd,0x9d,0xc3,0xbd,0xec,0x7e,0x94,0x46,0x67,0x68,0x70, -0x0c,0x7e,0x2e,0x41,0x64,0x82,0x1f,0x90,0xf8,0x67,0x69,0xf4, -0x3f,0x22,0x73,0x97,0x46,0xff,0x2b,0x42,0x83,0xe8,0x29,0xfc, -0x33,0xcf,0xa2,0x67,0x29,0xfc,0xa3,0xd3,0x39,0xf4,0x92,0x39, -0xda,0xb5,0x0f,0x44,0xd6,0xa7,0x17,0xa5,0x55,0x06,0xa2,0x50, -0xba,0xdf,0xdd,0x7b,0xbf,0x7b,0xf0,0x0e,0x15,0xe2,0x60,0x99, -0x41,0x05,0xa8,0x34,0x42,0x61,0x35,0x32,0xbb,0xc7,0xc7,0x69, -0x74,0x81,0xaf,0x2f,0x7f,0x3a,0x82,0x48,0x32,0xb1,0x70,0xb4, -0x77,0x7c,0xf0,0xd3,0x11,0x14,0xa1,0x28,0xf4,0x33,0x3b,0x9f, -0x5c,0xe4,0x28,0xaa,0x9a,0x3f,0x40,0xee,0x0f,0xd3,0x68,0xd9, -0x2e,0xf8,0x73,0xad,0xdc,0xde,0x15,0x1c,0x9d,0xea,0x3c,0x6c, -0x21,0x1e,0xb2,0x68,0x87,0x64,0x55,0x19,0x68,0x01,0x55,0xde, -0x8f,0xb2,0xa8,0x19,0x19,0x6d,0x85,0x3e,0xbb,0xff,0x02,0xff, -0xa5,0xca,0x7d,0x4b,0x64,0xfd,0x4c,0x44,0xe6,0xfb,0xc4,0xac, -0x4c,0xdb,0xc6,0xed,0xed,0x16,0x64,0x58,0x91,0xde,0x4a,0xcc, -0x3e,0xfc,0x97,0x86,0x9e,0x53,0xa2,0xfd,0xf1,0x08,0xe4,0x47, -0x4a,0xab,0x4e,0x22,0xa0,0x92,0x58,0x48,0xd9,0x26,0xa0,0x55, -0x29,0x2d,0x4c,0xe9,0xf7,0x83,0x24,0xf6,0x72,0xb0,0x0d,0xf1, -0x15,0xd1,0x90,0xfd,0xf5,0xeb,0x20,0x9e,0xfc,0x64,0x50,0xfe, -0xca,0x78,0x28,0x70,0x7d,0x5d,0x59,0xc0,0x6c,0xa0,0x17,0x99, -0xca,0xb4,0x16,0xa7,0xdd,0xdc,0x84,0x38,0x0f,0x46,0xa8,0x81, -0x4c,0x75,0xad,0x48,0x81,0xda,0xa6,0xd3,0x20,0xe9,0x38,0xc7, -0x4b,0x2b,0x2e,0xb4,0x22,0x85,0x0a,0x4d,0x0b,0x00,0x87,0xc3, -0xc1,0xd4,0x66,0x78,0x9e,0x98,0x3d,0xf8,0xaf,0xd8,0xef,0x2f, -0x7b,0x5f,0x22,0x4a,0xa9,0x4a,0xe0,0xf7,0xc6,0xa9,0xf7,0xca, -0x62,0xfe,0x9a,0x86,0x3e,0x66,0x96,0xa6,0xec,0x75,0x01,0x0f, -0x82,0x3b,0xe9,0xe9,0xcf,0x5f,0xf6,0xfb,0xaf,0xaf,0x6f,0xa6, -0xbd,0xbf,0xee,0x3d,0x83,0xfd,0xed,0x02,0xe2,0x9e,0x51,0xfc, -0xb3,0xee,0xe2,0xd9,0xb3,0xe4,0xbb,0x67,0x14,0xb5,0xd7,0x58, -0x7c,0x69,0x2c,0xf6,0x1b,0x8b,0x7e,0x63,0xf1,0xba,0xb1,0xb8, -0x6e,0x2c,0x6e,0x1a,0x8b,0x69,0x63,0xd1,0x5b,0xfc,0x35,0x49, -0x70,0x2b,0xbb,0x69,0x02,0x2f,0x13,0x78,0x16,0xdd,0x6f,0xac, -0x6f,0xb6,0xbd,0x33,0x97,0x95,0xa4,0x5f,0x70,0xc6,0xeb,0x1c, -0x2c,0xbb,0xf8,0xa4,0xe8,0x47,0xd7,0x65,0x99,0xfb,0x07,0xa4, -0x81,0x35,0xfd,0x60,0xae,0x92,0x10,0xca,0x55,0x21,0x43,0xd8, -0x19,0x5d,0x5c,0x8f,0x27,0xce,0xce,0x80,0x83,0xa5,0x8c,0x06, -0xec,0x45,0x46,0x8e,0x79,0xca,0xa7,0x58,0x6c,0x30,0xa0,0xa4, -0xfe,0x4f,0xcf,0xbd,0xd1,0x31,0x36,0xdb,0x4c,0x56,0xe6,0x93, -0xc9,0x94,0x83,0x75,0x00,0x8e,0x5b,0xc8,0x50,0x35,0x20,0x2e, -0xf8,0xe2,0x86,0x5d,0x2a,0x86,0xe2,0xe8,0xcd,0x1e,0xec,0x58, -0x3e,0x47,0x49,0xa2,0xae,0x80,0x03,0xdb,0x05,0xfe,0x74,0xb2, -0xca,0xbe,0x84,0x28,0xed,0x68,0xad,0x7a,0x41,0x02,0x96,0x1a, -0x56,0xd9,0x09,0x2f,0xc6,0xd8,0x36,0x28,0x6c,0xff,0x46,0x57, -0x4e,0xa3,0x7c,0x67,0x38,0xe8,0x79,0x93,0x0c,0x53,0xfd,0x18, -0x46,0x2b,0xfe,0x6b,0x7b,0xf2,0xe3,0x5b,0x3a,0xe2,0xc3,0xfa, -0xe8,0xd4,0x8d,0xd5,0xc8,0x44,0x21,0x18,0x04,0x17,0x9b,0xc3, -0x2a,0x45,0x4e,0x6e,0x7a,0xc3,0xc1,0xdf,0xf3,0xbe,0xb7,0x30, -0xfe,0xce,0xc6,0xc5,0xfc,0xf0,0xcf,0x81,0x6a,0xaf,0xc6,0xec, -0xd4,0x03,0xea,0x96,0xdf,0xa9,0xde,0xdf,0x4e,0x06,0xe3,0x09, -0xec,0x6f,0xd2,0xd6,0xd6,0x56,0x79,0x68,0xcb,0x4e,0x17,0xaa, -0x6c,0x44,0x54,0xbc,0xff,0xf7,0x75,0x96,0x5e,0xf7,0xf8,0x91, -0xb7,0xe8,0x9b,0xb5,0x35,0x67,0x50,0x85,0xfe,0xf3,0xda,0x3f, -0xa7,0x11,0xec,0x54,0x81,0x38,0xe0,0x2f,0x94,0x80,0x0f,0x1a, -0x27,0x05,0xd4,0xf7,0xe6,0xbf,0xa2,0xd3,0x7e,0xf7,0xbb,0x27, -0x98,0xd0,0x08,0xed,0x1d,0x78,0xc8,0xa5,0x9e,0xa1,0x1f,0x3b, -0x63,0xc2,0x1e,0xe2,0x1c,0x86,0xa7,0x94,0xbb,0x9d,0x07,0x82, -0xf1,0x59,0xa8,0xe3,0x6d,0xa8,0x6b,0x2d,0x0b,0x58,0x7f,0x94, -0xde,0x02,0x7a,0x76,0xaf,0xcc,0x82,0xf3,0x5e,0x20,0xb4,0xcc, -0x2d,0x71,0xca,0x34,0x90,0x8f,0x21,0xf3,0xbe,0x3e,0x68,0xad, -0xe7,0xf2,0xe9,0x70,0x3b,0xb4,0x6a,0xaf,0x59,0x8d,0xa0,0x0c, -0x03,0xb8,0x4e,0x86,0xd3,0x23,0xf7,0x8c,0x8a,0xd9,0x0f,0xbe, -0x0d,0x73,0xdd,0xff,0x8a,0xef,0x79,0x7c,0xc7,0x0f,0xa6,0xef, -0xae,0xb4,0xaf,0xb2,0x47,0x2e,0x7d,0x4a,0x00,0x41,0xc0,0xa2, -0x57,0xb7,0x64,0x10,0xde,0x14,0xb8,0x99,0x7b,0xc2,0x15,0x62, -0xbf,0x7a,0x32,0xe0,0xb5,0x79,0x71,0xc4,0x11,0xf2,0x9e,0x1a, -0x75,0x05,0x97,0xdf,0xfa,0xd0,0x98,0xb2,0x17,0x3f,0xd1,0xb4, -0x08,0x58,0x4c,0xd9,0xb6,0xad,0xe5,0x52,0x15,0xfc,0xa8,0x53, -0xcd,0xa4,0x52,0xab,0x6e,0x88,0x48,0x5a,0x53,0x3b,0xcb,0x76, -0x85,0x76,0xa0,0xc2,0x16,0xdf,0x69,0x47,0xd3,0xf9,0xf9,0xcd, -0x00,0xfa,0xb2,0x1a,0xa7,0x44,0xb7,0x8e,0x38,0xea,0x7a,0x1f, -0xbb,0x70,0xfc,0x25,0xb0,0xda,0xe1,0x6e,0x0f,0x82,0x0e,0xab, -0x78,0xcc,0xfb,0xb5,0xd5,0x2f,0xe5,0x2a,0x82,0x5f,0x9d,0x78, -0x7b,0xd3,0x44,0xcc,0x1a,0xb5,0x07,0x9c,0x00,0x98,0x1e,0x4e, -0xf0,0xac,0xb8,0x00,0xb1,0xf9,0xfe,0x36,0x25,0xa2,0x2b,0x6d, -0x92,0x1c,0x78,0x44,0x29,0x4e,0x0d,0x23,0x47,0x68,0x5b,0xde, -0x58,0x3f,0x5b,0xa9,0x7f,0xb4,0x6f,0xc2,0x42,0x05,0x4b,0xf8, -0xb1,0xa2,0x15,0xb3,0x1a,0x15,0x77,0x84,0x62,0x31,0xc9,0x3f, -0xcf,0x70,0x95,0x54,0x28,0xea,0x69,0x2e,0xf3,0xdf,0x3f,0x01, -0xe2,0x29,0xd4,0x71,0xa5,0x6e,0xbf,0xc4,0xc1,0x24,0x49,0x8c, -0x5e,0xe7,0x77,0x76,0xa3,0x65,0x92,0x06,0x19,0x96,0x5d,0xb6, -0xea,0xed,0xa7,0x55,0xd5,0xac,0x8d,0x71,0x99,0x7c,0xf5,0x78, -0x36,0x94,0xcc,0xbc,0x89,0x38,0xbc,0x0d,0xb8,0x9c,0xdd,0x2e, -0xf0,0x6c,0x9f,0xfe,0x4c,0x57,0x5d,0x3f,0x1c,0x37,0x92,0x38, -0x95,0x2b,0x07,0xbc,0x71,0x58,0xa0,0x55,0xde,0xb3,0x4f,0xff, -0xb6,0x96,0x36,0x3b,0x8d,0xac,0xfe,0xab,0xff,0x58,0x3b,0xdb, -0x38,0xdb,0xec,0xf2,0x0d,0x82,0x36,0x92,0x06,0x55,0xe0,0x23, -0xeb,0xde,0xc6,0xdf,0xa1,0x70,0xf3,0xfd,0xaf,0x1a,0x1b,0xdd, -0xc6,0x7f,0xa8,0x18,0x08,0x9e,0x35,0x25,0xdc,0xbd,0xdf,0x36, -0xbf,0x59,0xae,0x3b,0xd1,0xcd,0x41,0x40,0x35,0x82,0xb3,0x8d, -0xc5,0x19,0xd5,0x0d,0x12,0x1f,0xfc,0x81,0x88,0x26,0xfc,0x05, -0x69,0x84,0x95,0x0a,0x06,0xd6,0x06,0x57,0x46,0x66,0xe1,0xd2, -0xc0,0x36,0x9c,0x6c,0x79,0xd2,0x82,0x05,0x50,0xba,0x04,0xd0, -0x96,0xfd,0x0c,0xd9,0x4e,0x4b,0x43,0x8b,0x74,0x86,0xec,0xa6, -0xa5,0xa1,0xf5,0x34,0x63,0x6d,0x7f,0xa5,0x25,0x23,0x60,0xe6, -0x7a,0xd0,0xef,0xe7,0xac,0xc2,0x23,0x8f,0x70,0x45,0xc7,0x87, -0x26,0x1c,0x7f,0xc3,0x98,0xe7,0x33,0xd1,0xf2,0x21,0xd4,0xfd, -0xb8,0x01,0x9b,0x95,0xfb,0x26,0x62,0xd7,0x6c,0xbc,0x4e,0x73, -0x5c,0x49,0x75,0x86,0xed,0x2a,0xac,0x4c,0xed,0x45,0xf6,0x35, -0x72,0x1a,0x75,0x46,0x57,0xfb,0xe3,0x3e,0x6a,0x10,0x3c,0xb6, -0x68,0x8a,0xad,0xb2,0x0b,0x9a,0x61,0xb1,0xeb,0x51,0xf5,0x42, -0x99,0xa6,0x2f,0x69,0x41,0x77,0x17,0x0b,0x97,0x81,0x5f,0x39, -0x7d,0x85,0x01,0x3e,0x98,0x21,0x5d,0xa3,0xdc,0x8f,0x90,0x45, -0x85,0x3b,0x5c,0xf6,0x22,0x13,0xf8,0x25,0xa1,0x04,0xb1,0xb6, -0x1f,0x99,0xd0,0x2a,0x3d,0xa5,0x59,0x1b,0xf4,0xb0,0x41,0xf5, -0x86,0xec,0x29,0x85,0x8c,0xd5,0x47,0xe6,0x1d,0x37,0xde,0xaf, -0x30,0xd8,0xbf,0xbc,0x06,0x42,0xef,0xe6,0x9f,0x2f,0xf2,0x5b, -0xbe,0x7a,0x22,0xd5,0xb0,0x60,0xa5,0x54,0xab,0x27,0xcb,0xec, -0x6a,0x0c,0xb8,0xad,0xa5,0xf2,0x86,0x4a,0x1b,0x5b,0xd4,0x58, -0x27,0x5a,0xde,0x18,0xc0,0xbf,0xce,0xb5,0xcc,0x7a,0xd5,0x6d, -0xf4,0xbf,0xcc,0xb7,0x0c,0x9f,0xde,0x3b,0xf3,0x6f,0x19,0xdb, -0xf2,0x6c,0xbe,0xeb,0xbd,0x13,0x35,0x93,0x1b,0x1c,0xb6,0x15, -0x69,0x62,0xe3,0x0d,0x0d,0xc4,0x70,0x84,0xb7,0x1a,0xe7,0xe3, -0x10,0xf6,0x2e,0xd9,0x76,0xb7,0x6b,0x9a,0x4a,0xd4,0xbe,0x0b, -0x38,0xc6,0x39,0x37,0x90,0xd2,0xd6,0xef,0x06,0x07,0x95,0xd3, -0x18,0x8e,0xa0,0xf7,0xd9,0xeb,0x6e,0x65,0x63,0x79,0x8f,0x69, -0xed,0xf7,0xde,0xe9,0xd3,0xba,0x35,0xc5,0x8c,0xf1,0xc8,0xa2, -0xe9,0xe3,0x18,0x32,0xf8,0xbc,0x4d,0xe8,0xc6,0xc1,0xd5,0x88, -0x1e,0xbb,0xf8,0x0c,0x25,0x9b,0xe0,0x3c,0x87,0xe9,0x66,0x7a, -0xc4,0x25,0x58,0x25,0x08,0x26,0x3d,0x9a,0x43,0x61,0x75,0x1f, -0x58,0x70,0x40,0x1a,0x1f,0x6d,0x70,0x06,0x54,0x21,0xd0,0xd6, -0xc0,0x5b,0xcb,0xc8,0x04,0x18,0x05,0x86,0xc0,0x2d,0x7d,0x24, -0xf6,0x22,0x49,0xcc,0xf8,0x21,0x93,0x6a,0x87,0x32,0x40,0xb6, -0xee,0x55,0xb6,0xf5,0x3b,0xfa,0x88,0x06,0xc2,0xcf,0xb5,0xc7, -0xdc,0x00,0x89,0xeb,0x1f,0xae,0xce,0xba,0x01,0x6a,0xaf,0x97, -0x94,0x3d,0x9e,0xe0,0x79,0x87,0x61,0x3c,0xe2,0xfe,0xc5,0xcd, -0x09,0xeb,0xa4,0xe4,0x9e,0x6b,0x3d,0x2d,0x67,0xe9,0xae,0x65, -0xd9,0x9a,0x64,0x43,0x66,0x28,0x54,0x1e,0xaf,0x2e,0x50,0xaf, -0x7f,0x95,0x9f,0x17,0x8f,0xf4,0x4a,0x52,0xc3,0x17,0xdf,0x25, -0x3f,0x27,0x49,0x31,0xdb,0x56,0x08,0xa6,0x55,0xf0,0x94,0x92, -0x98,0xd5,0x28,0xbb,0x16,0x16,0x7b,0xf4,0x01,0xbe,0xf4,0x04, -0xb7,0x2a,0xc5,0x6e,0x26,0x04,0x13,0xa7,0x40,0xb6,0xc2,0x3b, -0x49,0xe5,0x34,0xd4,0xf3,0xb4,0x92,0x81,0x3d,0xc9,0x7f,0x89, -0xaa,0xd9,0xd9,0xb0,0x2c,0x59,0x7c,0x2d,0x72,0x24,0xd6,0x36, -0x0d,0xd1,0x40,0x57,0xb8,0x1a,0xcd,0xad,0x12,0x9a,0xad,0x6a, -0x34,0x9f,0xe6,0xf6,0xa5,0xda,0xab,0x8b,0x7a,0x48,0xa7,0xb9, -0x5f,0xa8,0x0c,0xec,0xb4,0x6e,0xed,0x9e,0xbe,0xc4,0x46,0xbd, -0x63,0x91,0xb8,0xc4,0x61,0xb9,0xad,0x9e,0xfb,0xd8,0x05,0x4e, -0x36,0xc0,0x61,0xfd,0x15,0x8c,0xd5,0xe3,0xe2,0xbc,0x68,0x90, -0x56,0x90,0x77,0xfe,0x61,0x35,0x6f,0x70,0xca,0x15,0x16,0x4d, -0x88,0x5c,0x92,0x3b,0x10,0x7a,0xd0,0x83,0xdb,0x0b,0xac,0xa7, -0xbd,0x1e,0x1c,0x70,0x54,0x1a,0xa4,0xf1,0x4c,0x54,0x10,0x26, -0x13,0x10,0x6c,0x07,0xb4,0xdc,0x70,0xbf,0x45,0x95,0xa5,0xa2, -0x68,0x71,0xd4,0x0c,0xfa,0x9f,0x33,0x65,0x80,0x54,0xd4,0x9c, -0x51,0x0d,0xba,0x08,0x93,0x21,0xb6,0xa1,0xc0,0xc6,0x46,0x5b, -0x94,0xa0,0x7d,0xc9,0x53,0x88,0xef,0x5a,0x1b,0xee,0x2b,0xfd, -0xbb,0x14,0x3d,0x7f,0xc8,0x28,0x68,0x1b,0xaa,0x68,0x1e,0x08, -0x76,0x14,0x5d,0x3b,0x34,0x4a,0xba,0x2f,0xed,0x8f,0xad,0xc0, -0x77,0xea,0x16,0x0b,0x13,0xfd,0x4c,0x7e,0xae,0xf1,0x5a,0xc0, -0x6f,0xb8,0x4b,0x12,0xd1,0xd3,0xe4,0xc2,0xa9,0x33,0xd8,0x08, -0x65,0x71,0x90,0x28,0x12,0x77,0x92,0x97,0x76,0x57,0xc8,0x31, -0xad,0x6e,0xc8,0xf4,0xdb,0x97,0x6e,0x03,0xa6,0x7c,0xac,0x39, -0x38,0xa1,0x55,0xda,0xca,0x7d,0xdc,0xe5,0x8a,0x1d,0x9c,0x87, -0x41,0xbb,0x2e,0xb4,0x7c,0x49,0x64,0x59,0x79,0x96,0x2a,0xd2, -0xf0,0xb7,0x08,0xc3,0x95,0x24,0x5f,0x30,0x30,0xeb,0xce,0xe3, -0x9c,0xb1,0xc9,0xa6,0x45,0x87,0xb0,0x4b,0x9c,0x79,0xa0,0xa7, -0x8c,0xe1,0xa3,0x42,0xfb,0x4d,0x60,0xb0,0x99,0x69,0x5e,0xce, -0xbc,0x6c,0x3d,0xc0,0x9d,0xda,0x32,0x55,0x1e,0xb6,0x71,0x1d, -0x94,0xa2,0xb5,0x2e,0xb4,0x79,0x2d,0xda,0xf4,0xb8,0x20,0x55, -0x9a,0xb9,0xb6,0x45,0xd9,0x86,0xb5,0x35,0xbd,0xf1,0x48,0xce, -0x92,0x29,0xf9,0xa2,0xe1,0xdf,0xbb,0x55,0x46,0xae,0xe7,0xa3, -0xe9,0xf5,0xe0,0x72,0xa6,0x33,0x14,0x6c,0x9b,0xfa,0x6a,0x02, -0xae,0x21,0xd9,0x4b,0x0e,0x77,0x78,0xcb,0x0e,0xd4,0x83,0x63, -0xfb,0xa4,0xd1,0xf9,0x5a,0x22,0xf2,0xca,0x62,0x9b,0xd6,0xa7, -0x8d,0x58,0x46,0x14,0x42,0xc1,0x9a,0x51,0xcf,0x49,0x7c,0x6b, -0xb2,0x8b,0xf6,0x7a,0xbd,0xc2,0xbb,0x4d,0xe2,0xce,0x50,0xb0, -0xcc,0x62,0x01,0x93,0xdc,0x50,0xcf,0xa8,0xd1,0x75,0x2d,0x73, -0xce,0xab,0xd4,0xf3,0x48,0x97,0x4a,0x87,0x28,0xc4,0xd6,0x5d, -0x94,0x3c,0xd2,0x76,0x78,0x94,0x8d,0x44,0x09,0xeb,0x42,0x98, -0x3c,0x4c,0xda,0x95,0xd5,0x32,0x21,0x27,0x4a,0x95,0xce,0x01, -0x78,0x23,0xf4,0xb5,0xce,0x22,0x58,0xd1,0xf6,0x4e,0x1b,0x59, -0xf4,0x46,0xc4,0x94,0x7a,0x0b,0x4d,0xff,0xdd,0x83,0x77,0xc7, -0x27,0x3b,0xef,0x4e,0xde,0xc3,0xe6,0xe3,0xa7,0x3d,0x75,0xc4, -0x81,0x3a,0x3c,0x0b,0x52,0xd6,0x59,0xa0,0x51,0xca,0xf5,0x4d, -0x18,0x6b,0x6a,0xef,0x43,0x83,0xfd,0xf0,0x11,0xfd,0xec,0x76, -0x68,0xe0,0xdf,0x8e,0xbe,0x7c,0xa9,0xac,0x9f,0x8d,0xc4,0x4b, -0xd6,0xa6,0xd4,0xab,0xbc,0x0a,0x87,0x13,0x5a,0x9f,0xd3,0x5b, -0xe3,0xe3,0x15,0x7c,0xe5,0x4f,0xd2,0xdf,0xab,0xf8,0x42,0xc5, -0xe5,0x80,0x2d,0xf5,0xc0,0xc5,0x80,0xad,0xf0,0x2e,0x9c,0x15, -0x3f,0x0e,0x46,0x8a,0x67,0x69,0xe7,0xae,0x0f,0x9e,0xbf,0x97, -0x84,0x78,0xdc,0x94,0xe3,0xf3,0x19,0xb4,0xb8,0x98,0x34,0xfb, -0xbc,0x5f,0xb0,0x11,0xd6,0x17,0x10,0x56,0x96,0x98,0x2a,0xec, -0x31,0xa5,0x84,0xbc,0xad,0x04,0x0f,0x20,0xe2,0xd2,0xab,0x9f, -0x28,0x4a,0x5d,0x5b,0x12,0x69,0xc9,0x49,0x7e,0x73,0x8b,0xfa, -0xb7,0xc1,0x79,0x8b,0xd6,0xcb,0x0d,0x2c,0x16,0xbb,0xe8,0xa7, -0x5d,0xc9,0xf0,0x2b,0x4b,0x57,0xe8,0xd5,0x28,0xd3,0x30,0x02, -0x8b,0x52,0x72,0xdd,0xa0,0x1a,0x67,0x31,0x4b,0xbc,0xed,0x94, -0x27,0xf6,0x5d,0x50,0x65,0x89,0x1b,0x86,0xd0,0xa3,0x27,0xf4, -0x21,0x8d,0x7d,0x57,0x3a,0x0c,0xd5,0x6e,0x83,0xce,0x42,0xf5, -0x86,0xe0,0x24,0xe4,0x89,0x9d,0xf3,0xed,0x04,0x81,0x28,0xac, -0x26,0x0a,0x4c,0x5d,0xd9,0x28,0xbc,0x60,0x25,0x1c,0xc9,0x46, -0xe7,0x64,0x0e,0x8b,0x37,0x15,0x50,0x8e,0x04,0x5d,0x6b,0x77, -0x03,0x7b,0xef,0x59,0xc1,0xfc,0x7b,0xc4,0x8b,0x96,0x64,0x3b, -0xe8,0xf7,0x57,0xe7,0x84,0xc4,0xc8,0xf8,0xbc,0x78,0x10,0xbf, -0x3a,0x33,0xa6,0xd2,0x3d,0x3c,0xe6,0x1e,0xf7,0x3e,0x56,0x4f, -0xb5,0xfb,0x47,0xee,0x7f,0xd5,0x2c,0x16,0x38,0x91,0xf2,0x26, -0x51,0xf9,0x10,0x01,0xbb,0xff,0x82,0x72,0x26,0x7c,0xcb,0xea, -0x25,0x42,0x35,0xde,0x65,0xc6,0xc8,0x46,0xfc,0xbc,0x6f,0x56, -0x94,0x29,0xa3,0xff,0x88,0xa4,0x0f,0xa7,0xb7,0x21,0xb1,0x94, -0x8f,0x2f,0x8b,0xa7,0x96,0x9e,0xd1,0xe6,0xcf,0xab,0xae,0x7f, -0x9c,0x37,0xbe,0x8b,0xe9,0x2d,0x79,0xdc,0xa4,0x8a,0xe8,0x76, -0xa3,0xfa,0x26,0x98,0xaf,0x9e,0x6b,0xfd,0xf3,0x21,0x7f,0x90, -0x7d,0x09,0xb4,0x9d,0xc1,0x5f,0xf3,0xdb,0x9a,0xb3,0x38,0x51, -0xb3,0x16,0x29,0xf8,0x03,0x3b,0xa7,0xe6,0x4d,0x53,0xd4,0xbc, -0xed,0x8a,0x9a,0xb8,0x27,0xc4,0x5f,0x00,0x00,0x7f,0xe9,0xe0, -0xa5,0xc6,0x67,0xba,0xb0,0xa9,0xb8,0x98,0x4f,0x6b,0xe7,0xc3, -0xf9,0x24,0xaa,0x34,0x74,0x32,0x72,0x77,0xa5,0xfe,0x82,0x14, -0x0f,0x90,0x56,0x5e,0x97,0x8e,0xe4,0xaa,0xb4,0xd0,0xcc,0xd3, -0xa0,0x78,0xd7,0xab,0x13,0xe8,0xf7,0x6b,0xd5,0x7e,0xfb,0x57, -0xf1,0xab,0xcb,0x91,0x3d,0xac,0xa2,0xab,0xcc,0xb0,0x82,0x44, -0xbb,0x68,0x2c,0x58,0xdb,0x2b,0x5d,0xed,0xaf,0xf2,0x1f,0x71, -0x69,0x2b,0x87,0xad,0x1f,0xe6,0xe4,0x1b,0xb3,0x25,0x71,0x65, -0x7c,0xdb,0x22,0x97,0xa5,0x57,0x6f,0x2e,0x03,0xd2,0xb1,0x0f, -0xf7,0x54,0xd3,0x24,0xca,0x91,0xce,0x6c,0xd2,0x1b,0x4d,0xc9, -0xb7,0x6c,0x1a,0xb9,0xd3,0xda,0x80,0x9e,0x60,0x14,0x6f,0x06, -0xa3,0xde,0x10,0xe9,0x55,0x39,0x61,0x78,0x44,0x9f,0xc2,0x78, -0xc0,0xe5,0x8e,0x74,0x67,0xbf,0xf6,0x8c,0xd7,0xf7,0x25,0xd9, -0x2c,0xb0,0x17,0x92,0x14,0x38,0xc6,0xbc,0x85,0xad,0xac,0x65, -0xfa,0x6f,0x2e,0x4b,0x6c,0x4b,0x03,0x00,0x49,0xdc,0x36,0xb8, -0x49,0x34,0x18,0xeb,0x54,0xb1,0x45,0x21,0x21,0x65,0xc2,0x4a, -0xd5,0x8b,0x5b,0x4c,0x17,0x68,0xda,0xad,0x55,0xac,0xb3,0xe8, -0x82,0xde,0x73,0x29,0x63,0x13,0x14,0xcf,0x6c,0x1b,0x40,0x26, -0x45,0x4d,0x6e,0xd7,0x41,0x2a,0x8f,0x32,0xec,0x30,0xe4,0x17, -0x55,0x1a,0x45,0x8a,0x73,0xcf,0x34,0xe5,0xbd,0x99,0x44,0xe6, -0x85,0xbb,0x44,0x17,0x93,0x2c,0xed,0x95,0x3a,0x4e,0xfa,0x37, -0xec,0x48,0x38,0x20,0x13,0xbc,0xb6,0xc2,0x23,0xfa,0x99,0x2c, -0x70,0xa4,0x9a,0x47,0xe7,0xf8,0xa4,0x57,0x73,0xfc,0xff,0x50, -0xf7,0xf6,0xfd,0x6d,0xdb,0xc8,0xc2,0xe8,0xdf,0xf7,0x7c,0x0a, -0x9b,0x27,0xeb,0x90,0x11,0x2c,0x5b,0x4e,0xda,0x6e,0xa9,0xd0, -0x7a,0xd2,0xbc,0xb4,0x39,0x4f,0x9c,0xe4,0x89,0x9d,0xed,0x6e, -0x65,0xd5,0x3f,0xd9,0xa2,0x6d,0x36,0xb6,0xe4,0x8a,0x52,0x93, -0xd4,0xd2,0xfd,0xec,0x17,0xf3,0x02,0x60,0x00,0x92,0xb2,0xd2, -0x76,0xcf,0x73,0xcf,0x6e,0x63,0x91,0xc0,0x60,0x30,0xc4,0xcb, -0x60,0x30,0x18,0xcc,0x9c,0x01,0x8b,0x82,0x77,0x73,0x51,0x4a, -0x89,0xa1,0x55,0x59,0xcd,0x00,0x9b,0xf2,0x71,0x29,0x0f,0x93, -0xbd,0x87,0x65,0xbf,0x44,0xc9,0x25,0x50,0x9e,0xc1,0x3c,0xd5, -0xc3,0x4c,0x8e,0xbf,0xda,0xd1,0xba,0xde,0x38,0x2c,0xed,0x01, -0x76,0x39,0x3d,0xd3,0xdb,0x01,0x13,0x25,0x86,0x1b,0x84,0xf7, -0x04,0x3a,0x4f,0x4d,0xc6,0xe0,0xd4,0xd7,0x82,0xd0,0x2b,0x5e, -0x09,0x01,0x5f,0x9e,0xf4,0x0d,0x36,0x17,0x92,0x4a,0x4c,0xea, -0xd6,0xb3,0x8b,0x7b,0xbe,0xb3,0x15,0x76,0x18,0x8a,0x59,0xfc, -0xe2,0x46,0x3b,0x6c,0x75,0x51,0xe7,0x98,0x4f,0xc1,0xa1,0x1c, -0x0c,0xa4,0xf9,0xcd,0xab,0x61,0x39,0x63,0x22,0xd9,0x09,0xa0, -0x14,0xb5,0x25,0x42,0x18,0x6d,0xe2,0xd5,0x1b,0xae,0x22,0x1d, -0x83,0xc7,0x26,0x41,0xed,0x35,0x53,0xc5,0xcb,0x0f,0xe1,0x09, -0xc9,0x7f,0x2c,0xfd,0x18,0x1a,0x28,0x26,0x20,0x2b,0x7c,0x52, -0xbe,0x73,0x16,0x98,0x31,0x35,0xb9,0xb4,0x7a,0x9a,0x9e,0xb1, -0xf7,0xa1,0xcb,0xa2,0xa4,0x3d,0xfe,0xcb,0x51,0xd6,0x6a,0x79, -0x6d,0xd0,0xd5,0x50,0x3d,0x1a,0x50,0x20,0x7b,0x40,0x21,0x75, -0x8b,0x8e,0xc5,0x53,0x7f,0x7c,0x2d,0x93,0x76,0x39,0x3f,0x3b, -0xcb,0x21,0xaa,0x89,0xa9,0x41,0x8f,0xa2,0x9b,0xc9,0xb8,0x24, -0x6b,0x25,0xaf,0x96,0x2c,0xf3,0x6a,0x61,0x03,0x9c,0xfc,0x23, -0x35,0x8e,0x9c,0xa3,0x5d,0x39,0x78,0x62,0x03,0x52,0x9d,0xa3, -0x4d,0x1d,0x55,0x69,0x78,0x83,0x21,0x68,0x4b,0x9a,0xc2,0x7e, -0x22,0xc9,0x5d,0xf6,0x2b,0x2a,0xb3,0xde,0x1f,0x3b,0xd0,0x1b, -0x6e,0xa8,0xd9,0x19,0x16,0x80,0xb5,0x8d,0xcb,0x9d,0x38,0x49, -0xbc,0xa1,0xe2,0xf9,0xc5,0xf5,0x46,0x39,0x6c,0x96,0xe5,0xfb, -0xd6,0xd6,0xa6,0xb7,0x9b,0x0a,0x60,0xbd,0x89,0x1e,0x7c,0xbe, -0x2e,0x72,0x0d,0xcb,0xfa,0xbd,0xc2,0x6b,0x24,0xf6,0xb2,0x9d, -0x78,0xbb,0x34,0x3b,0x07,0x49,0xf3,0xd3,0x46,0xfd,0x7b,0x1c, -0xa8,0xba,0x9b,0x7a,0x14,0x22,0xc4,0x34,0xf6,0xc8,0xd2,0x55, -0x54,0x47,0x0d,0xdf,0x43,0x04,0x82,0x92,0x74,0x25,0x1a,0xc7, -0x51,0x8b,0xd9,0xba,0x12,0xe7,0x2a,0x4b,0x1c,0xc9,0xb5,0xca, -0xaa,0x2e,0xac,0x6c,0x53,0x55,0x64,0x1c,0xa5,0xeb,0x7d,0x3d, -0x19,0x83,0xec,0x0e,0x27,0x54,0x0d,0xd5,0x4b,0xce,0x29,0x97, -0x74,0x2c,0xfe,0x56,0x4b,0x5a,0x28,0x2e,0x79,0x4b,0x01,0x5b, -0x71,0xaa,0xc6,0x0d,0x1d,0x01,0x28,0x7f,0x63,0x87,0x5e,0xb6, -0xdf,0x3d,0x79,0xfa,0x3c,0xdb,0xb9,0x5d,0xee,0x18,0xcf,0x24, -0x9e,0x55,0xc6,0x3a,0x21,0x9b,0xc8,0xe0,0x0b,0x0e,0x2a,0x2d, -0x7b,0x3d,0x83,0x1e,0x55,0x1f,0x2f,0xf3,0xb1,0x4d,0xa2,0xb5, -0x1f,0x92,0x82,0x78,0x6a,0x41,0x6e,0x02,0xbe,0xbf,0xb4,0x38, -0xcf,0x5c,0x1c,0x9f,0x17,0x8b,0x5d,0x44,0x56,0x7a,0x51,0x86, -0x19,0x3d,0x3a,0x1f,0xa1,0x6c,0xfd,0x06,0x2e,0x4b,0xf4,0x2b, -0x1e,0xef,0x1d,0x7e,0xbe,0x3e,0x9d,0x5c,0x79,0x3b,0xd1,0xb6, -0xc8,0xd0,0xe3,0x3c,0x1f,0x8f,0xea,0x80,0x6c,0x72,0x0c,0x17, -0x8f,0x7e,0xd4,0xa8,0xb3,0x9d,0x9f,0xa1,0x86,0x18,0x6c,0x8e, -0xcb,0xa4,0x17,0xb7,0x41,0x99,0x62,0xe5,0x6e,0x5c,0xb1,0xdc, -0x3a,0x66,0x8f,0x26,0x95,0xf5,0xa0,0xcc,0x06,0x98,0x88,0x8b, -0x14,0x23,0x7e,0x96,0xe6,0xe3,0xf8,0x05,0x81,0xc5,0x81,0x05, -0xb0,0xe1,0x34,0x22,0x00,0x8b,0xbc,0x08,0x1b,0x68,0x08,0x0d, -0xe7,0xf6,0xdb,0x7a,0xf0,0x0f,0xb2,0x86,0xc6,0xed,0x7b,0xf8, -0x28,0x94,0x91,0xa1,0x1f,0xab,0xae,0xfd,0x00,0xf4,0xab,0x21, -0xda,0x16,0xfd,0x65,0x04,0x7b,0x7b,0x0b,0x6d,0x29,0xc2,0x41, -0x25,0xfb,0xa0,0x65,0xc7,0x47,0x0b,0x8e,0xed,0xa9,0x57,0x5b, -0xb6,0x99,0x13,0x31,0xbd,0x57,0x1c,0x7a,0x88,0xc8,0xa0,0x72, -0x1c,0x58,0xdc,0xce,0x0d,0x0a,0xc5,0x6a,0x61,0xed,0x19,0xf8, -0x17,0xc6,0x47,0xf0,0x42,0x8a,0xdf,0xb2,0x58,0xb0,0xca,0x84, -0xe7,0x45,0xfb,0x06,0x67,0xd5,0x53,0x13,0x08,0x76,0x9b,0x08, -0x4c,0x12,0x39,0xac,0xfa,0x98,0x35,0x08,0xa6,0xc0,0x26,0x84, -0xe1,0x11,0xdb,0x1d,0x0e,0x51,0xe6,0xa9,0x16,0x4c,0xc0,0xb0, -0x84,0x59,0xcf,0xbb,0xfc,0x26,0x1f,0xfa,0x76,0x2f,0xbc,0x97, -0xa9,0x95,0xfd,0x31,0x4f,0xb9,0x2d,0x00,0xfa,0x42,0xfe,0xfe, -0xe4,0xdd,0xf3,0x83,0x37,0xff,0x78,0xfe,0x2c,0x8b,0xee,0xdd, -0x73,0x6f,0x91,0x45,0x5f,0x09,0xab,0x43,0xc9,0x91,0x69,0xa3, -0x2f,0xdc,0x42,0xac,0x96,0xd6,0x80,0x49,0xb8,0x8d,0xed,0x5a, -0x3b,0x06,0x5d,0xfd,0xd9,0x87,0xef,0x3e,0xeb,0xb6,0x55,0xee, -0x91,0x8d,0xe6,0x39,0xe1,0xe5,0xe8,0xc5,0xd8,0x3d,0xa3,0x46, -0x54,0x26,0xbc,0x39,0xfd,0x45,0xbf,0x5e,0x5d,0x96,0x6a,0xaa, -0xff,0x61,0xef,0xbc,0xc4,0xdb,0x93,0xe7,0x85,0xc6,0xa1,0x07, -0xaa,0x78,0x73,0x83,0xd4,0x98,0x2e,0x70,0x7b,0x50,0xcc,0xf4, -0x4c,0x0c,0x62,0x1b,0x83,0xbd,0x7c,0x00,0xd3,0xfc,0xb8,0x6c, -0x15,0x63,0xfd,0x27,0x6e,0x3f,0xe8,0x25,0x68,0x8e,0x55,0xb6, -0x90,0x04,0xfd,0x7b,0xfa,0x19,0x33,0x00,0xe8,0x01,0xd8,0x7f, -0x25,0x78,0xc3,0xf3,0xc5,0xf8,0x15,0x5e,0xb8,0xc9,0x6e,0xef, -0x15,0xa3,0x94,0x3d,0xd0,0x60,0xd8,0xf7,0x4d,0xc4,0xcd,0x06, -0x0e,0x7e,0x47,0xc5,0x51,0x11,0xc4,0x3a,0x77,0x24,0xc1,0xb8, -0x05,0xc5,0x31,0x5c,0x94,0xbd,0x7f,0x52,0x68,0x19,0xea,0x04, -0x92,0x4e,0xce,0x40,0x05,0x81,0x0d,0x7d,0xd2,0xa7,0xf6,0xdc, -0x38,0xfd,0xbc,0x71,0x52,0x8c,0x4e,0x06,0xf7,0x31,0x2e,0xba, -0x74,0x86,0xe2,0xd0,0xe1,0x51,0xa1,0x6e,0xb6,0xcc,0xc6,0x8d, -0x9f,0xda,0x97,0xbd,0x81,0xe8,0x0d,0x4e,0x7b,0x24,0xd3,0x7a, -0x71,0xd8,0x5b,0x66,0x13,0xed,0xd2,0x13,0xd9,0x81,0x99,0xf4, -0xd1,0x43,0xbd,0xa4,0xfc,0x90,0x17,0x5e,0x4f,0x69,0x46,0x28, -0xdb,0xb0,0xef,0x65,0xa2,0xaf,0x16,0xbf,0x91,0xfb,0x41,0xbf, -0x1b,0xc7,0xe1,0x12,0x86,0x43,0x03,0x66,0xe4,0x18,0x32,0xa4, -0xdf,0x0c,0x52,0x59,0x02,0x22,0x6b,0xc5,0xe1,0xc8,0xab,0xf9, -0x12,0xfb,0x0d,0xc6,0xcf,0x10,0x6b,0x11,0x83,0x41,0xea,0x95, -0x94,0xdf,0xbd,0xe4,0xb0,0xfe,0x99,0xee,0x0e,0x3b,0xf0,0xe2, -0x5e,0x1a,0xf7,0x8f,0xef,0x1d,0x7f,0x84,0x4b,0x20,0xc7,0xb1, -0x7d,0xd6,0x83,0x4c,0xa1,0x03,0x22,0xf3,0x9e,0x24,0x30,0xe4, -0x56,0x0f,0xaa,0x62,0x44,0xc3,0x8a,0xc7,0xcd,0x7d,0x18,0x38, -0x0d,0x63,0xe8,0xfe,0x46,0x79,0x39,0x99,0x5f,0x8d,0xf0,0xa6, -0xb5,0x1e,0x74,0xb6,0x4d,0xe1,0xba,0xf5,0xfd,0xf8,0x44,0xd3, -0x7b,0xa2,0x36,0x4e,0xf0,0x13,0x4f,0x92,0xfb,0x62,0x80,0xaa, -0xca,0x68,0xd3,0xdf,0xc3,0x7e,0x94,0x5d,0xd7,0x64,0xee,0xfe, -0xbe,0x1d,0x79,0x5e,0xf7,0xda,0x31,0x48,0x97,0x51,0xb4,0xf8, -0xf6,0x9d,0xe6,0xcf,0x1f,0x0e,0xc0,0x99,0xea,0xd2,0x57,0x25, -0x3c,0xb5,0x64,0xc7,0x30,0xf9,0x9d,0x68,0x6f,0xd3,0xa5,0x2f, -0x50,0xf6,0x04,0x0d,0xce,0x6d,0xc8,0x17,0x0e,0x74,0x28,0xc5, -0x4b,0x17,0x1a,0x01,0xe5,0x06,0xa8,0xed,0x3e,0xe5,0x10,0xea, -0xee,0x2d,0xd5,0x29,0x10,0x84,0x86,0xc3,0xc5,0x64,0x5e,0xa2, -0x2f,0x1e,0xc3,0xdc,0xe0,0x40,0x18,0x6a,0x10,0x34,0xbb,0xf7, -0x37,0x53,0x74,0xbb,0x34,0xe8,0xba,0x10,0x8c,0xaf,0x8a,0x0f, -0xb9,0xa4,0x37,0xf1,0xab,0xca,0xdc,0xab,0x37,0x9d,0xc4,0xf3, -0x62,0x11,0x0e,0x51,0xba,0x99,0x7f,0x37,0x38,0xb1,0xce,0xa0, -0x42,0x4d,0x9d,0x70,0xd7,0x2d,0x28,0x73,0x8f,0x41,0x9c,0x40, -0x1c,0xce,0x5b,0x5b,0xd1,0xbd,0x68,0x13,0x66,0xa6,0x73,0x64, -0xa0,0x45,0x79,0x0f,0x37,0xc7,0x7c,0xd4,0xd0,0xdd,0x20,0x03, -0xc2,0xd2,0x68,0xb1,0x1c,0x2a,0x16,0xbd,0x92,0x05,0x50,0xdc, -0x81,0x1c,0xe9,0xdf,0x6f,0x55,0xce,0x6c,0x28,0x42,0x33,0x7f, -0xb7,0xcb,0x71,0xfc,0x29,0xdc,0x22,0xfe,0x25,0x37,0xe6,0xe0, -0xad,0xca,0x95,0x84,0x2d,0x89,0x87,0xa6,0x87,0xa0,0xa9,0x9f, -0xc8,0xf1,0x78,0x68,0xac,0x88,0x02,0xe4,0x4c,0xcc,0x36,0xb2, -0x6c,0xfb,0x0a,0xf3,0x53,0x72,0x78,0x87,0xcd,0x6a,0x0b,0x26, -0x09,0x0e,0xb8,0x4c,0x02,0xf7,0x6d,0xee,0x40,0xb1,0xb3,0x82, -0xa6,0x6c,0x39,0x1c,0x45,0x7a,0x46,0xa3,0xd8,0x6f,0x46,0xfe, -0x28,0xca,0xb3,0x1e,0x1e,0x24,0x86,0x15,0x54,0x12,0xf3,0x71, -0xee,0x33,0x25,0x62,0x37,0x39,0x11,0x75,0x72,0x8b,0x3f,0x5b, -0x5b,0xf8,0x43,0xf2,0x38,0x79,0xa1,0x8a,0xbd,0xcf,0xa0,0xec, -0xc2,0x50,0x4b,0xe7,0x58,0x3e,0x73,0x1b,0xcd,0x6f,0xe0,0xfa, -0x5f,0xf4,0x6c,0x0e,0xc1,0x8c,0xb4,0x6c,0x54,0xa2,0x9f,0xf9, -0x8d,0x29,0x42,0x69,0xbe,0x35,0x9c,0xe6,0xe8,0xb4,0x66,0x78, -0x05,0x62,0xf5,0xa8,0xbd,0xf1,0xbe,0xcc,0x37,0xee,0x9b,0x95, -0x52,0x72,0xb0,0x8d,0xd9,0x64,0x83,0x22,0x99,0x7c,0xde,0x98, -0x8f,0x0b,0xbd,0x99,0x04,0xf6,0x5c,0xb6,0x37,0xde,0x31,0xae, -0x74,0x43,0x73,0x36,0xb5,0x61,0xab,0x82,0x6c,0x36,0x2c,0x14, -0x7c,0xd0,0x35,0x49,0xb7,0xbe,0x75,0x6f,0xb5,0x44,0x60,0x81, -0x96,0x8d,0x1d,0xb4,0xd9,0x59,0x8a,0xa9,0x28,0xdb,0x25,0x59, -0x35,0x6e,0x68,0x3a,0xc6,0x35,0x63,0x06,0x47,0x66,0xa0,0x29, -0xa2,0x16,0x66,0xee,0x55,0xba,0xdd,0x80,0x9f,0x5e,0xe3,0x3a, -0xd8,0x30,0x3c,0x27,0x76,0x72,0x80,0x24,0xee,0xd2,0x40,0x8b, -0x95,0x20,0x5b,0xe1,0x89,0xa8,0x56,0xcd,0xd6,0xfa,0x59,0x7a, -0xfb,0x6f,0x99,0xa6,0xd4,0x48,0xb5,0xbd,0xa4,0x82,0xa1,0xc9, -0xea,0x5c,0x52,0x06,0x89,0x6f,0xb4,0x0b,0x4a,0x26,0x17,0x84, -0xee,0x68,0xb2,0x61,0x33,0xcc,0x83,0xe7,0x01,0xf4,0xe3,0x25, -0xe8,0x78,0x4c,0x16,0x18,0x3b,0xd0,0x93,0x6c,0xd0,0xa4,0x1b, -0x10,0x91,0x59,0x5c,0xa8,0xae,0xa1,0x8e,0x44,0x57,0x68,0x41, -0x7f,0xa1,0x4e,0x89,0xfd,0x49,0x4a,0xba,0xf4,0x6e,0xc6,0x5b, -0xb7,0xb8,0xd8,0x18,0x5d,0x4c,0x92,0x63,0xef,0x26,0xa5,0x77, -0xd7,0x65,0x34,0xc9,0x5b,0xa1,0x10,0x27,0x4a,0xd4,0x89,0x70, -0x52,0x3b,0x2f,0x85,0x33,0x99,0x8e,0x5e,0x62,0x32,0x88,0x1f, -0x52,0xcd,0x83,0xc1,0x4d,0xc9,0xe0,0x9b,0xdd,0x2d,0x1c,0xdb, -0x1d,0x0f,0xec,0xba,0x18,0x81,0xcb,0xfe,0xcd,0xb8,0x82,0x77, -0xb1,0x08,0xd1,0xf9,0x44,0x4d,0x46,0xa3,0xa0,0x18,0x9c,0xdd, -0x00,0x35,0x1c,0x66,0x3a,0x09,0x47,0xc9,0x62,0x41,0xbb,0xa0, -0x55,0xc7,0x02,0xb5,0x67,0x00,0xeb,0x76,0x18,0x41,0x8b,0xf1, -0x97,0x89,0x9a,0xc2,0xd1,0x82,0xc0,0x20,0x92,0xf8,0xa3,0xc3, -0x43,0xc2,0x7d,0xcf,0xb0,0xf8,0x57,0xb8,0x1b,0xf4,0x78,0x53, -0xc0,0x8d,0xf5,0x56,0xd6,0x13,0xcf,0x24,0xac,0x53,0xb1,0x1d, -0x5e,0x4e,0x3e,0x7e,0xe1,0xc1,0xd6,0x17,0x5e,0xfc,0x6c,0x53, -0x25,0x95,0xd3,0x24,0x83,0xb6,0x1f,0x9e,0xec,0xf4,0x22,0x71, -0x10,0x1c,0xa5,0x91,0x39,0x92,0x8f,0x06,0xce,0xa9,0xed,0xf8, -0x62,0xfb,0x12,0x5c,0x0a,0xd8,0xe3,0xf1,0x1f,0x8a,0xe0,0xe4, -0xe5,0xdf,0xf1,0x1d,0x50,0xc9,0x97,0x7c,0x87,0xa5,0x3c,0xf5, -0x3e,0x69,0xd5,0x77,0x60,0xbc,0xb3,0x3f,0x60,0xd0,0x52,0xdb, -0xec,0x80,0xcb,0xd3,0x7b,0x60,0x4a,0x09,0x61,0xae,0xe9,0x29, -0xb9,0xb5,0x8f,0x68,0xcf,0x45,0x8f,0x14,0xe4,0xda,0x24,0x9b, -0x25,0xc7,0x26,0x79,0x2d,0xa0,0x30,0x5a,0x9a,0x53,0xa6,0x9c, -0x41,0x60,0x6b,0xac,0x17,0x6d,0x17,0x94,0x45,0x2a,0x22,0xbb, -0x51,0xd0,0x76,0xa6,0x60,0x09,0xea,0x19,0x14,0x1c,0x0e,0xd1, -0x4b,0xd6,0x97,0x9e,0xb2,0xfa,0x9a,0x50,0x61,0x04,0x47,0xe8, -0xbc,0xdb,0x9c,0xee,0x72,0x4d,0x68,0xd9,0x0d,0xda,0x3c,0x50, -0x4a,0xea,0x3e,0xb8,0xd3,0x6a,0xce,0xa0,0x7e,0x2a,0xae,0x73, -0xe2,0xe1,0x57,0x4e,0x81,0x97,0x8e,0xac,0xd2,0xa6,0xb4,0x81, -0x19,0x8d,0x4b,0x68,0x85,0x1d,0x04,0x17,0x23,0xcc,0x21,0x19, -0xe1,0xe2,0x33,0x32,0x2d,0x9e,0x98,0x12,0xc8,0x32,0x50,0xe0, -0xf7,0xba,0xd6,0x96,0xaf,0x8c,0x43,0x19,0x30,0xa9,0x28,0x32, -0x1f,0x8f,0x75,0x8a,0x5c,0xec,0x17,0xe8,0x19,0xd9,0xcf,0xee, -0x17,0x03,0x79,0xa4,0x15,0x08,0x21,0xe1,0x47,0x60,0x04,0xa5, -0x30,0x11,0x6e,0xaf,0x54,0x68,0x57,0x71,0x4d,0xa3,0x64,0xd5, -0x06,0xa4,0xf6,0xef,0x47,0x9b,0x1c,0x8c,0x13,0x8c,0x74,0x1b, -0x81,0x7a,0x11,0x38,0x9d,0xab,0x84,0x4f,0x6f,0xd3,0x41,0x85, -0x93,0x91,0xea,0xfa,0x43,0x3a,0x63,0x09,0x32,0xfd,0x4e,0xac, -0x39,0xb1,0x0a,0x5a,0x14,0xb7,0x4b,0x5e,0x5a,0xa2,0xaa,0x58, -0xdb,0xe2,0x2c,0xda,0x03,0x16,0xeb,0x8e,0xfe,0xac,0xe7,0xf2, -0x24,0x93,0x0e,0x7a,0xb2,0x1a,0x64,0x3c,0x0e,0x2b,0x8d,0x4f, -0xc4,0x48,0x44,0x95,0x43,0x2d,0x97,0xa7,0x08,0xbf,0x3b,0xce, -0xa6,0x77,0xb4,0x78,0x70,0x0a,0x50,0x6a,0x7d,0xd0,0x85,0x37, -0x1d,0x81,0xac,0xd4,0x4d,0x7e,0xb5,0xbb,0xeb,0x26,0xe3,0xcf, -0x72,0x36,0xae,0xd0,0x51,0x96,0x2b,0x4d,0x1b,0x56,0x5b,0x41, -0xbb,0x11,0x64,0x8e,0x74,0xdc,0x17,0x0c,0xb2,0xbb,0x41,0x20, -0xfc,0x85,0xba,0x1b,0x8c,0x5a,0x5a,0x7e,0xbc,0x7b,0x34,0xb4, -0x99,0xc0,0x7e,0xd6,0x43,0x04,0x73,0x36,0xba,0xc3,0xfc,0x3f, -0xaa,0x39,0xf5,0x5c,0xcb,0x82,0xf7,0x4a,0x43,0xe9,0xb4,0x3f, -0xd4,0x2a,0x6e,0x5c,0x37,0x9e,0xf1,0x49,0xce,0xed,0x30,0x7a, -0x17,0xfc,0xc5,0x17,0xd2,0x05,0x10,0x97,0x92,0x89,0xe7,0x0a, -0x63,0xaf,0x55,0x7b,0x97,0xde,0x1d,0xfd,0x33,0xe1,0xb6,0x40, -0x4c,0xe3,0x8a,0xc0,0xe8,0x99,0xf3,0x81,0xcf,0x12,0x77,0x47, -0x8c,0x9c,0x96,0x10,0x20,0x19,0xda,0x97,0x67,0xd3,0xe2,0xc6, -0x3f,0x58,0x08,0x8c,0x43,0xc4,0xc7,0xc9,0x8c,0xfa,0x8b,0xb4, -0x6b,0xeb,0xff,0xc9,0xb5,0x08,0x9c,0x75,0xec,0x68,0xa1,0xc3, -0x60,0x8e,0xb2,0xcc,0xdd,0x9a,0xe5,0x88,0x92,0x94,0xf3,0x7e, -0x7a,0x45,0x59,0x70,0x43,0x4a,0x97,0xca,0x9c,0x0a,0x0d,0x4f, -0x4c,0xba,0x3e,0x75,0x7a,0x04,0xcc,0x62,0x51,0x56,0x51,0x6c, -0x41,0x66,0x26,0x6f,0x28,0x66,0x61,0xcd,0xd5,0x07,0x41,0xff, -0xd2,0xf0,0x4f,0xaf,0x71,0x84,0x61,0x4c,0xc5,0x98,0xcc,0x1a, -0xbb,0xb0,0x55,0x19,0x9f,0xba,0xbc,0x79,0x7b,0xf4,0xf2,0xcd, -0xeb,0x43,0xff,0x86,0x36,0x1c,0x0d,0xc4,0xbd,0xf4,0xb8,0x6c, -0x0d,0x4b,0x73,0x54,0x90,0xf4,0x28,0xe5,0x62,0x3a,0x99,0xdf, -0xb8,0xb3,0x02,0x70,0x35,0xab,0x1f,0xf4,0x3a,0x02,0xaf,0x56, -0xd3,0x8b,0x7f,0x8f,0x47,0x83,0x07,0xe8,0xe3,0x87,0x1d,0xcd, -0x07,0x39,0xbe,0xfe,0xd7,0x4b,0x3f,0xc6,0xdb,0xe0,0xe2,0xa0, -0x82,0xea,0x0e,0xce,0x29,0x90,0x2a,0xb0,0x01,0xd7,0x1b,0x8e, -0x03,0x7b,0x9b,0xe5,0xd6,0xbb,0x1c,0xc6,0x9e,0x35,0x56,0xdd, -0xab,0xee,0x47,0xd4,0x94,0xba,0xd9,0xec,0x65,0x8d,0x41,0x20, -0x0a,0x89,0x1b,0xc4,0xf6,0xca,0x31,0x5f,0x2f,0x76,0x4d,0x6c, -0xa7,0x07,0x4f,0x17,0x3e,0x42,0x37,0x0e,0x44,0xa8,0x63,0xd5, -0x7c,0xfc,0x61,0x3c,0xf9,0x38,0xe6,0x37,0x70,0x68,0x46,0x41, -0xaf,0x38,0x56,0xa5,0xd1,0xb4,0x5e,0xb8,0x2f,0xf2,0xbe,0x0f, -0xd6,0xb3,0x73,0xb4,0xb1,0x3d,0x9d,0xcc,0xc7,0x23,0x3a,0xfe, -0x29,0xdd,0x3d,0x50,0xc8,0x2d,0xc6,0x85,0xb0,0xce,0x11,0xa8, -0x4e,0x94,0x23,0xe4,0xc4,0xa7,0xe4,0x24,0xb9,0xf5,0xea,0xac, -0x2f,0x94,0x35,0x96,0xcf,0x7c,0x6c,0x4b,0xa2,0x44,0x4b,0xf5, -0x9c,0x1d,0x8a,0x60,0xee,0x73,0xf9,0x20,0x12,0xee,0xe0,0x89, -0x5a,0xe5,0x75,0x2a,0x7b,0xcf,0xca,0x32,0x0b,0x10,0x64,0xf8, -0x4a,0x9b,0x57,0xaf,0x5d,0xa8,0xb7,0xb6,0xfc,0x74,0xe3,0x66, -0x3d,0x61,0xc2,0xe8,0xbd,0x81,0x36,0xe3,0x8a,0x97,0xb2,0x9d, -0x11,0x1d,0xab,0x27,0x2b,0xa4,0xdf,0x45,0x37,0xc7,0x1a,0x80, -0x7b,0x60,0xef,0x25,0x55,0xf6,0xba,0x85,0xa1,0xa9,0x02,0xe1, -0x91,0x46,0x43,0x89,0xbf,0xeb,0x1f,0x10,0xa1,0xb9,0xb7,0x11, -0xb5,0xc4,0x51,0x4d,0xd2,0x8a,0x36,0x7a,0x51,0xd7,0xff,0x72, -0x68,0x2a,0x57,0x46,0x30,0x5c,0x0e,0x15,0x15,0x7b,0xe0,0x22, -0x3f,0x2c,0x18,0x34,0x34,0x38,0xcf,0x89,0x8c,0x70,0x85,0x26, -0xd5,0xfc,0x19,0xb6,0xe1,0x9a,0xae,0x29,0xb9,0xf6,0x0b,0x15, -0x7e,0xe6,0xb8,0xc9,0x68,0x6b,0x1a,0x2f,0x92,0x35,0xb6,0x17, -0x4e,0xf7,0x64,0x9d,0xad,0x09,0x5f,0x85,0xb3,0x17,0xfa,0x0f, -0x8b,0xf1,0xc5,0x55,0xce,0x90,0xf4,0x5d,0xd6,0xac,0xc9,0x75, -0x2f,0xe7,0xe0,0x5d,0xb5,0x5b,0xaf,0xdb,0xf9,0x92,0x75,0x78, -0xd2,0x6f,0x07,0x43,0xfd,0x18,0xe9,0x3a,0x7c,0x72,0xc4,0xd9, -0x8b,0x3f,0xbd,0xf8,0xcb,0xc6,0xb7,0x4f,0x39,0xcd,0x13,0x8b, -0x8c,0xa3,0x82,0xbb,0x7b,0x43,0x39,0x5c,0x0f,0x6b,0xee,0xd0, -0x24,0xf5,0xa2,0xd0,0x5a,0x3c,0x5e,0xc1,0x5e,0xb5,0x46,0xbd, -0xae,0xa7,0xe2,0xbb,0x6a,0x47,0xbe,0xbb,0xb4,0x15,0x90,0x0c, -0x7d,0x4e,0xdb,0x13,0xbf,0xc7,0x1b,0x6c,0x92,0xbf,0xb0,0x79, -0xbc,0x5e,0x90,0x0b,0x45,0x5c,0xfd,0x8c,0x84,0x0c,0x9b,0xed, -0x08,0x39,0x60,0x5f,0x63,0xb5,0x63,0xc4,0xdd,0x0f,0x03,0x4d, -0x12,0xa0,0xed,0x9e,0xad,0x18,0x17,0x70,0x62,0x59,0x82,0x25, -0xde,0x06,0x07,0x9b,0xa9,0x5c,0x66,0xeb,0xfa,0x7b,0x33,0x43, -0x18,0x84,0x58,0x8a,0x23,0x9a,0x42,0xd2,0x30,0xbd,0x21,0xb2, -0x71,0xe6,0x0c,0xea,0xb0,0x4a,0xb4,0x60,0x94,0x01,0x8e,0xf1, -0x23,0x1b,0x8d,0x55,0xf4,0x12,0x39,0xbc,0x2a,0x63,0xf3,0x49, -0x2a,0x24,0x72,0xb1,0xb0,0x79,0x19,0xba,0xc4,0x09,0x01,0x2a, -0x97,0x60,0xc3,0x11,0xfa,0x85,0xdd,0x8d,0x7b,0x3e,0xd0,0x8f, -0xf2,0xf1,0xde,0x9f,0x6f,0xa1,0xad,0x2d,0xc4,0x57,0x0d,0xfd, -0x8c,0x61,0x02,0xab,0xe3,0x04,0xa1,0x83,0x91,0xc1,0x32,0xdb, -0x8a,0x81,0x61,0x41,0x4d,0x4b,0xe0,0x97,0x50,0x75,0xdf,0x83, -0x48,0x85,0x3e,0xc8,0xc4,0x3b,0x3f,0xab,0xfc,0x13,0x5e,0x48, -0xbf,0x78,0x4b,0x71,0x3c,0xcc,0x2b,0xd7,0x18,0xbc,0x2b,0x94, -0xce,0x5e,0xc9,0xd3,0x45,0x4a,0x7a,0x89,0x1a,0x67,0xd2,0x3b, -0x9b,0x0f,0xc7,0x23,0x3b,0x43,0xa5,0x61,0x8d,0x57,0xc3,0x53, -0x2d,0x45,0x08,0x3a,0xca,0xec,0x36,0x8a,0x52,0x08,0x77,0x18, -0x10,0x5b,0x6a,0x89,0x53,0xcb,0x49,0xe2,0x26,0x75,0x78,0xb5, -0x9a,0x4e,0x2a,0x4a,0x5a,0x05,0xcb,0x17,0xcc,0x85,0x13,0xdc, -0x14,0xc1,0x69,0x14,0xa8,0xce,0x01,0x53,0xaf,0xc4,0xa8,0xbd, -0x70,0xc6,0x41,0x2b,0x40,0x99,0xd0,0x75,0xaf,0x92,0x5d,0xbd, -0xa2,0x05,0x9c,0x51,0x08,0xe4,0xb3,0x6c,0xb3,0x03,0xe7,0xce, -0x36,0x16,0x38,0x58,0xd6,0x82,0x64,0xf8,0x62,0xbc,0xb5,0x65, -0x6e,0x1a,0x3a,0x2a,0x12,0x5c,0x2d,0x6c,0x59,0x39,0xe5,0xfa, -0x83,0xc4,0xc6,0x43,0x43,0x0c,0x2f,0xf9,0x74,0xd5,0xbd,0x3c, -0x76,0x88,0x8c,0x5e,0xc8,0x65,0xb6,0x5a,0xc9,0x95,0xb0,0xda, -0xa0,0x9b,0x1c,0xae,0x40,0xdf,0x41,0x0e,0x24,0xfd,0x24,0xff, -0x13,0xc5,0x3c,0x64,0x8c,0x57,0xee,0xfa,0xc2,0x1c,0xa7,0xb4, -0xe9,0x33,0xc4,0xb7,0xca,0xa3,0x29,0x3e,0x7d,0xca,0x3e,0x54, -0x4e,0x9c,0x6b,0x0e,0xa4,0xaa,0xdf,0x41,0x3d,0xd0,0x37,0x7d, -0x74,0x65,0xed,0x57,0x28,0xfb,0x83,0x3b,0x91,0x4a,0xf9,0x98, -0x29,0x18,0x20,0x19,0x8e,0xbc,0xef,0x3e,0x87,0x1f,0x89,0x36, -0xec,0xb1,0x00,0xce,0xe4,0x70,0xeb,0x07,0x58,0xe0,0x9e,0xec, -0xfa,0xc0,0xa0,0x47,0x0b,0xc7,0xa9,0x9c,0xd9,0x36,0x35,0xd1, -0x6d,0x6d,0xc6,0x8f,0xe5,0x95,0xb2,0x8f,0x78,0xc5,0xe0,0x6e, -0xea,0xd5,0x76,0x57,0x6a,0xb6,0x66,0x5e,0x6a,0xb2,0xe9,0xee, -0x62,0xd8,0x83,0x67,0x2e,0x2f,0xaf,0xde,0xc3,0x91,0xcf,0xed, -0xb2,0x6b,0x5f,0xea,0x07,0x91,0x1d,0x38,0x99,0x4f,0x82,0x2d, -0x06,0x4e,0xc5,0x6b,0xa9,0xf3,0x07,0x8d,0x40,0x99,0x65,0x59, -0x2d,0xe1,0x5d,0x39,0xc0,0xc4,0xf3,0x62,0x61,0x5e,0x96,0xc8, -0x1e,0xb2,0x51,0x51,0xea,0xbd,0x6b,0xa5,0x63,0x89,0x79,0x64, -0xf4,0x37,0xb8,0xbe,0x59,0xe1,0x2b,0xac,0x06,0x31,0xe7,0xc7, -0x8d,0x4d,0x6c,0xc6,0x5f,0x65,0xbc,0x51,0x6d,0x8c,0xd8,0x10, -0x98,0x5a,0x4a,0xc9,0x06,0xc3,0xf4,0xb1,0x1e,0x43,0x6e,0xe7, -0x62,0xa2,0x24,0x88,0x26,0xe9,0x79,0xa3,0x4a,0xf3,0x35,0x7b, -0x39,0x1d,0x28,0xd4,0xe3,0x95,0x2a,0xd3,0x0f,0xb6,0xa6,0x4d, -0xd1,0x42,0xcb,0x24,0xf5,0xda,0x6b,0x35,0xb6,0x5e,0x2d,0x3a, -0xbd,0xa7,0x4f,0x04,0xb3,0xce,0x76,0x25,0x33,0xcf,0x2a,0x83, -0x9a,0x99,0x91,0x80,0xd9,0x77,0x85,0xbb,0xee,0x11,0xe6,0x37, -0xb4,0x44,0x38,0x39,0x43,0x84,0x7d,0x57,0xc4,0x9b,0x42,0xab, -0xe7,0x9b,0xb7,0x52,0x90,0x72,0x83,0x08,0x7b,0x9c,0x39,0x7c, -0xbd,0xd8,0x5f,0xc4,0x32,0x73,0x02,0x92,0xea,0xd2,0x58,0xd4, -0xdc,0x69,0x6d,0xa3,0xf6,0x27,0x4e,0xd8,0xa9,0x2c,0xb6,0x52, -0x14,0x4e,0x68,0x1e,0x67,0xa9,0x1c,0x4c,0x98,0xb2,0x0c,0x57, -0xc7,0xac,0xef,0x57,0x5c,0x47,0x2e,0x05,0x06,0xf6,0xcb,0x85, -0x32,0x0a,0xab,0xa6,0x7c,0x64,0x46,0xc1,0x0c,0xbe,0x7d,0xc2, -0x7a,0x2b,0xd5,0x78,0xad,0x1b,0xb4,0x46,0x50,0x18,0x0e,0x39, -0x83,0x9a,0xf0,0xeb,0x36,0xc3,0x2e,0xd3,0x52,0x78,0x2d,0x41, -0x7e,0xe3,0xd5,0xa1,0x0a,0x31,0x25,0x89,0x94,0x05,0x70,0x8f, -0xaf,0x02,0xfb,0x06,0xaf,0xb1,0x57,0x18,0x37,0x10,0x9c,0x04, -0x37,0x46,0x08,0x41,0x23,0x55,0x3e,0x9b,0x30,0x74,0x06,0x7a, -0xcb,0x23,0x89,0xf1,0xc1,0xcc,0x37,0x06,0x3d,0x6d,0x5a,0x88, -0xeb,0xa5,0xd7,0xad,0x2d,0x81,0x87,0x2c,0x97,0xeb,0x4a,0x79, -0x65,0x92,0x10,0x71,0x31,0x72,0x58,0xc1,0x7f,0x94,0x44,0x09, -0x7b,0x84,0x0a,0xb8,0x03,0xf6,0x1a,0x15,0x54,0x81,0x66,0xae, -0x3b,0x8c,0x4e,0x06,0x95,0x78,0xc3,0x9d,0x58,0x50,0x87,0xe5, -0xe8,0x01,0x12,0x18,0x88,0xb8,0xc1,0x13,0xe4,0x3a,0xa6,0xd7, -0xcb,0x45,0xef,0x52,0x52,0x6a,0x94,0x9e,0x5c,0x22,0x9c,0x85, -0x09,0x7e,0xa1,0xfb,0x1e,0x1e,0x58,0x8e,0xb0,0x90,0x00,0x6a, -0xe4,0x55,0xed,0x59,0xd6,0xcd,0x37,0xc7,0x0f,0x3c,0x29,0x34, -0x95,0x88,0x94,0x66,0x9c,0x96,0x12,0xc7,0x35,0x03,0x0a,0x96, -0x5e,0x9b,0xf7,0x64,0xa3,0x62,0x00,0x3c,0x6b,0xf3,0x93,0x36, -0x4d,0x1d,0x9e,0xe9,0xe6,0x64,0xc8,0x1b,0x89,0x9c,0xe8,0x64, -0xac,0x56,0xab,0x1b,0x7e,0x9e,0x37,0x29,0x92,0xca,0xc7,0xeb, -0x8e,0x4d,0x6c,0x65,0x66,0x53,0x8a,0x2b,0x55,0xb7,0x89,0x93, -0x4a,0xb6,0x9e,0xd4,0xf0,0x2f,0x40,0x09,0xa3,0xab,0x82,0xd5, -0xfa,0x2a,0x41,0x53,0x6d,0xf2,0x3b,0xc2,0x3d,0x0d,0x57,0x00, -0xd8,0x3e,0xb7,0xa2,0x03,0x4e,0x92,0x8a,0xc7,0x3a,0x26,0x1f, -0x7c,0xd6,0xad,0x6b,0xd5,0x4d,0x3d,0x72,0xb2,0x11,0x0f,0xcb, -0x8d,0x13,0xec,0xc1,0x93,0xa4,0x07,0xf9,0x1b,0x6c,0x78,0x9b, -0xf4,0xd8,0xf2,0xb6,0x6a,0xb6,0xeb,0x1b,0xdf,0x06,0x6e,0xef, -0xdc,0x07,0x78,0x3e,0xef,0x3c,0x86,0x9d,0xd0,0x45,0x64,0x2b, -0xa6,0x18,0x93,0xee,0x9a,0x88,0x5c,0xca,0x0a,0x5d,0xd6,0x3a, -0xdc,0xe4,0x7e,0x8d,0xbb,0x14,0x91,0xf5,0xd5,0x40,0x59,0x91, -0xd6,0x47,0x69,0x62,0x78,0xb1,0x58,0x15,0xd6,0xd7,0x33,0xd5, -0xa5,0xb6,0x36,0x86,0x2d,0x43,0xe0,0x6f,0x06,0x6c,0x6d,0xce, -0x75,0xfe,0x9d,0x0d,0x30,0x8d,0xe9,0x6b,0xcf,0x83,0xfe,0xfb, -0x20,0xc1,0x68,0xd1,0xd5,0x75,0x2d,0xeb,0xf7,0xed,0xac,0xf2, -0xf7,0xa3,0x46,0xf4,0x58,0x0e,0x06,0x5d,0xc7,0x0b,0x40,0xb9, -0x6a,0x6e,0xeb,0xb9,0x54,0x76,0x05,0x9a,0x08,0x0d,0x70,0xc5, -0xf3,0x00,0xe9,0xc3,0x6b,0x40,0xc0,0xdf,0x99,0xbf,0x8e,0xda, -0x43,0x9f,0x3f,0xad,0x02,0x90,0x1b,0x65,0x67,0x8a,0x4a,0xa9, -0xe6,0x43,0x49,0x4a,0x14,0xdb,0x5f,0xb9,0x21,0x36,0xd6,0xd2, -0xce,0x06,0xaf,0x6e,0x97,0x6a,0xb7,0x9f,0xde,0x7e,0x93,0xcd, -0x07,0x40,0x92,0xee,0x0f,0xd6,0x12,0xd9,0xbc,0x09,0xbd,0x96, -0xcc,0x16,0x88,0x6c,0x77,0x48,0x14,0xb4,0x58,0x77,0xbe,0x74, -0xb1,0xd6,0x1f,0x15,0x7b,0x6d,0x56,0xb3,0x6e,0x5b,0x31,0x47, -0xae,0x60,0xd6,0x86,0xd1,0x2b,0x4d,0x4a,0x33,0x33,0x69,0xc0, -0x00,0xb6,0xba,0x67,0x4c,0xcc,0x70,0x4e,0xee,0xda,0x78,0x0b, -0x2b,0x6d,0x22,0xde,0x21,0x14,0x1b,0x25,0x61,0x06,0x29,0x77, -0xdb,0xb5,0x7b,0x09,0x34,0xea,0x4e,0xfc,0xfd,0x3b,0x6e,0xd0, -0x36,0x56,0xe2,0x05,0xf3,0x4a,0xb2,0xfd,0xb7,0xfe,0xac,0x6a, -0x36,0x7d,0x4b,0xda,0x6e,0x71,0xbb,0xd4,0x28,0x13,0x95,0x96, -0xfb,0x33,0xa4,0x80,0x86,0x8e,0xbf,0x47,0x84,0x82,0x91,0x97, -0x8f,0xd1,0x6a,0x4d,0x96,0xdd,0x3e,0x7e,0x71,0xab,0xf9,0x9f, -0x5b,0xbb,0xd3,0xff,0x92,0x36,0x24,0x0a,0xd9,0x49,0x61,0xfd, -0x2e,0x12,0xe3,0xd4,0x71,0x73,0xdc,0xd9,0xb0,0x77,0x8d,0x96, -0x55,0x15,0xd5,0x28,0xe7,0x84,0x97,0x20,0x4f,0xfb,0x4a,0x3f, -0xbe,0x9a,0x93,0xd2,0x92,0x25,0x3b,0x5d,0x03,0x7f,0x77,0xae, -0x7d,0x85,0x82,0x5b,0xa8,0xfc,0x9d,0xab,0x3c,0x79,0x56,0x66, -0x4a,0x5b,0x8d,0x02,0x1d,0x06,0x9b,0x37,0xb1,0x64,0x19,0x43, -0x26,0xa3,0xbd,0x13,0x07,0x6b,0x9b,0x1d,0xe5,0x4b,0x61,0x26, -0x86,0x7a,0x10,0xc5,0x83,0xc7,0x94,0x53,0x70,0x26,0x2a,0xdc, -0x45,0xad,0x51,0x12,0x99,0x07,0x94,0xf5,0xcf,0xf0,0xea,0xe5, -0x40,0x08,0x18,0x4f,0xc6,0xa4,0xba,0xc9,0xec,0x7d,0x4c,0x93, -0x00,0xf9,0x45,0x66,0xde,0x2a,0xc6,0x53,0x3c,0xae,0x4d,0x3e, -0x58,0x4f,0x19,0xcf,0x3e,0xae,0x89,0x85,0x50,0xea,0x30,0xe5, -0xbf,0xc6,0x85,0x19,0x4d,0x1a,0x8b,0x38,0x50,0x80,0xf3,0x4d, -0x79,0xac,0xa9,0x9a,0x8e,0x57,0x9d,0x92,0xc7,0x84,0x85,0x30, -0x1e,0xe5,0xac,0xf3,0xb5,0x77,0x9c,0x90,0xd0,0xc2,0x62,0xf2, -0xff,0x71,0x97,0xaf,0xbd,0xca,0x41,0x42,0x9d,0x6f,0xbc,0xa0, -0x4a,0x3e,0x0e,0xfc,0x4d,0x68,0x13,0x79,0x84,0x2f,0xbb,0x1e, -0x3e,0xcf,0xe1,0x5a,0x85,0xa4,0xe0,0x14,0x43,0xf8,0x60,0x33, -0x6a,0x86,0x9a,0x22,0xeb,0x78,0xd6,0xab,0x14,0x8b,0xeb,0x0f, -0xad,0x60,0x82,0xb9,0x41,0xdd,0xf3,0xb5,0xdf,0x79,0xf5,0xd0, -0x2c,0x49,0x4d,0x37,0xf4,0x82,0x33,0x94,0x5a,0x60,0xef,0x24, -0xae,0x06,0x42,0x9e,0xc1,0x91,0xd5,0x04,0x11,0xf3,0xc5,0x9e, -0xc0,0xcc,0xc9,0xfc,0xa1,0x9b,0xe0,0xb7,0xf6,0xc4,0xda,0xb8, -0x77,0x77,0xe7,0xc4,0x2b,0x6c,0x09,0x56,0xfb,0x98,0xab,0xd8, -0x96,0xc8,0x63,0x35,0xec,0x18,0x3e,0x8a,0x59,0xd7,0xef,0x18, -0xee,0xb3,0x12,0x38,0xa2,0xeb,0x7a,0xd0,0x5e,0x5c,0xa0,0xdf, -0xc8,0x09,0x9c,0x5f,0x26,0x59,0xae,0xeb,0x4e,0xc8,0x35,0x32, -0x8a,0xc0,0x11,0x47,0x9c,0x17,0x3e,0xa1,0xd9,0xcb,0xad,0xe5, -0x08,0xd6,0x48,0x4e,0x30,0x4c,0x4a,0x23,0x9f,0x60,0x3e,0x46, -0x2d,0x61,0x71,0xa6,0x8d,0x3b,0x51,0x07,0x25,0x0e,0x48,0x21, -0xac,0x83,0xe5,0x01,0xd6,0x02,0xa2,0x97,0x37,0x6c,0x9b,0x37, -0x3b,0xf2,0x14,0x32,0xf3,0x3b,0xda,0x77,0xb5,0xd6,0xf3,0x16, -0x06,0x2f,0x2b,0xbc,0x0b,0x0d,0x06,0xc1,0x78,0x25,0xba,0xda, -0xd2,0x7c,0x2f,0x9a,0x7f,0xc9,0x32,0x58,0x3f,0x78,0x64,0xcb, -0xf1,0x14,0x33,0x2a,0xd1,0x5e,0xce,0x60,0xc2,0xdd,0xb2,0x4e, -0x6b,0xb3,0xc5,0xb0,0xb9,0xdb,0xef,0x6a,0x13,0x01,0x02,0x89, -0x31,0x73,0x2f,0x7d,0xa3,0x6a,0x6b,0x7c,0xd4,0x64,0x49,0xb5, -0xd4,0x2c,0x5a,0x4b,0x8a,0xff,0xf5,0x7f,0xe6,0x39,0x06,0xaf, -0xb8,0x99,0x9f,0x5e,0x15,0xe5,0xe5,0xf3,0x4f,0x1a,0x46,0x43, -0x3c,0x79,0xfb,0x32,0xd6,0x92,0xfd,0xfc,0x6a,0xa8,0x79,0x50, -0xb0,0x2e,0xc1,0x95,0xff,0xe1,0xc8,0x93,0xed,0x19,0x14,0x3c, -0x4c,0x58,0x30,0x75,0x3a,0x99,0xcc,0x74,0xe5,0x43,0xf4,0xc2, -0x11,0x53,0x14,0x73,0x65,0x91,0x28,0x2e,0x63,0x04,0x55,0x81, -0x9e,0x4e,0x02,0x2f,0x75,0x25,0x51,0x62,0xcd,0x1d,0xee,0x3f, -0xc6,0x4f,0xdc,0xc0,0xb8,0xd3,0x64,0xe4,0x05,0x21,0xfc,0xf6, -0xff,0x17,0xdc,0xf6,0xd3,0xdd,0xb9,0x11,0xbd,0x3f,0x7a,0xb1, -0xfd,0xf7,0xa8,0xdb,0x1f,0x5f,0x1c,0x1f,0xa7,0xe8,0x58,0x6d, -0xa0,0xfa,0xc6,0xc7,0x9a,0x7e,0x84,0xa1,0xb7,0x2d,0xde,0x3f, -0x89,0x97,0xb6,0x79,0x54,0x6d,0x97,0x8c,0xa9,0x60,0xbb,0x7e, -0xcb,0x5b,0x52,0xcd,0x49,0xc6,0xf9,0xc6,0x66,0x71,0x0d,0x31, -0x97,0x87,0xe3,0x59,0x77,0x89,0x75,0x01,0x27,0xb7,0x20,0x74, -0x71,0x6d,0xf9,0x78,0x07,0xc9,0xdd,0xbf,0x2f,0x4e,0x32,0xb9, -0x09,0xf8,0xc3,0x85,0x7b,0xb8,0xdb,0x68,0x0e,0x2a,0x7f,0xec, -0xa8,0x88,0xc2,0xe7,0x9a,0x08,0x84,0x7c,0x7f,0xde,0x34,0xd6, -0xbd,0x7b,0x36,0xca,0xa1,0x81,0x88,0x92,0xae,0xc9,0xbd,0x9e, -0x8c,0xe6,0x57,0x39,0xdd,0xb2,0xe7,0x4c,0x05,0xde,0x88,0xa3, -0x81,0x6e,0x54,0x0e,0x40,0x22,0x0a,0x2a,0xe1,0xbd,0xc9,0xde, -0xfc,0xff,0xb5,0xea,0x8d,0x89,0x6f,0xfe,0xff,0x2a,0x8e,0x43, -0xf3,0x31,0xc6,0xd0,0x9e,0x16,0x87,0xf9,0x05,0xf6,0x1e,0xda, -0xb9,0x30,0x8f,0xb2,0x99,0x34,0xb8,0xc0,0x84,0x5e,0xb3,0x3c, -0xeb,0x96,0x61,0xe7,0x6f,0x7b,0x5f,0xef,0x5c,0x14,0x10,0xce, -0x52,0x26,0x3e,0x7c,0x86,0x89,0x99,0x97,0xb8,0xf7,0x1d,0x26, -0xb6,0x22,0x71,0x6a,0x5b,0x83,0xfe,0x46,0x0b,0xed,0x98,0x7a, -0x78,0xa3,0xcb,0x95,0x21,0x25,0x32,0xd4,0x37,0x10,0x2a,0x2a, -0x78,0xb4,0x8b,0x15,0xfc,0x2f,0x9f,0x94,0x27,0x98,0x98,0xfa, -0xa4,0x3c,0xda,0xb9,0xd0,0xed,0xe3,0xa7,0x3d,0x45,0x40,0xe5, -0x27,0x6a,0x94,0x21,0x45,0xbd,0x48,0x27,0x6b,0x84,0xde,0x87, -0xbc,0x9b,0xcc,0xf5,0xb4,0x32,0x16,0x85,0x8a,0x43,0x44,0x97, -0x6c,0xc8,0x64,0xd2,0x33,0x0b,0x80,0xc9,0x06,0x2a,0x33,0x0f, -0xe8,0x77,0x04,0xb3,0xe6,0xd3,0xab,0xb7,0x10,0x03,0x1a,0xad, -0xfc,0x5d,0x2d,0xdc,0xdd,0x26,0x02,0x0d,0xc4,0x5a,0xc7,0x50, -0xd1,0x6c,0xac,0x5b,0x2a,0x8a,0xcf,0x24,0x8d,0x6a,0xf4,0xec, -0xd2,0xbb,0x88,0x19,0x61,0x8b,0x61,0xe2,0x30,0x10,0xa5,0xf0, -0x82,0x37,0xc2,0x2d,0xb6,0x71,0x62,0x2c,0xf2,0x33,0x8e,0xb6, -0xa3,0x09,0xab,0xab,0x89,0x91,0x58,0x2b,0x76,0x99,0x1a,0xd8, -0xfc,0x93,0xe7,0x8f,0xa2,0x7c,0xe1,0x4b,0x73,0x5a,0x2c,0x14, -0x3b,0x0c,0xd0,0x58,0x68,0x62,0xc8,0x29,0x88,0x27,0xac,0xd1, -0x4d,0xe0,0xad,0x2d,0xdd,0xbf,0x7c,0x78,0x26,0xa2,0x9c,0x53, -0x2c,0x51,0xfa,0x3a,0xca,0x2b,0xe7,0xa7,0x7a,0x12,0xc6,0xe0, -0xf7,0x9f,0xc4,0x3b,0xc4,0xeb,0x9a,0x92,0x9b,0xfc,0x1d,0xbb, -0x55,0x7a,0x09,0x0b,0x0e,0xb8,0x52,0xd7,0x12,0x97,0x73,0x18, -0xc5,0xed,0x61,0x12,0xda,0x66,0xba,0x55,0x7b,0xc4,0x3a,0x9d, -0x03,0x0b,0x0c,0x7c,0xb6,0x7d,0x49,0x6a,0xc0,0x29,0x0c,0x10, -0x3c,0x33,0xa6,0xa1,0x22,0x23,0xdb,0x73,0xa7,0x89,0xa6,0x7e, -0xf6,0xfc,0xc5,0x93,0xf7,0xaf,0x8e,0x4e,0x9e,0x3c,0x45,0xbd, -0xa0,0xed,0xd6,0xa0,0x9d,0x45,0x13,0x53,0x82,0x72,0xce,0x11, -0x2f,0x87,0xe5,0x77,0x93,0xd1,0x67,0xf0,0xd9,0xfb,0xf6,0xcd, -0xe1,0xd1,0xe2,0xed,0x7b,0xfd,0x0f,0x22,0xeb,0x27,0xf7,0x76, -0x0a,0x76,0x18,0x43,0xbb,0xd4,0xeb,0x7c,0x76,0x39,0x19,0x25, -0x5d,0xf3,0x2d,0xfd,0x31,0x6e,0xfe,0x1c,0xea,0x8e,0x1a,0xee, -0xa9,0xe1,0x43,0x35,0x7c,0xc4,0x7e,0x17,0xa1,0x99,0xd9,0xcf, -0x96,0x42,0x87,0x4c,0x34,0x36,0x70,0x0c,0xad,0x8e,0x1e,0xff, -0x28,0xa5,0xd0,0x14,0xc3,0x47,0x06,0x41,0x36,0x7c,0x48,0xc1, -0xe3,0x1f,0xa6,0xf8,0xb3,0x97,0x52,0x18,0x78,0x3b,0x4c,0x86, -0x7b,0x09,0x86,0x9f,0x02,0xfc,0x9a,0x14,0xa8,0x3c,0xd3,0xf4, -0x88,0xe2,0x76,0x83,0x22,0x4b,0x75,0xc0,0xea,0xc0,0xc0,0x74, -0x14,0x57,0xbb,0xc7,0xc0,0x36,0x67,0xcf,0xe4,0x3c,0xb4,0x21, -0xec,0x3d,0x24,0x3d,0x87,0x23,0xe5,0x26,0xed,0x11,0x09,0x9d, -0xd4,0x12,0x45,0x48,0xbb,0x1c,0x1f,0xdf,0x8f,0x12,0xcf,0xf1, -0xeb,0xfd,0x45,0x20,0x8e,0x4e,0x87,0x23,0xdd,0x44,0xa5,0x54, -0xe4,0xce,0x6f,0xe0,0xce,0xf0,0x23,0x1b,0xc5,0xbe,0xdc,0xe8, -0x53,0x0d,0x8a,0x5a,0x7c,0xc3,0x34,0xf9,0x06,0x92,0x3c,0x50, -0xa8,0xa2,0xbd,0xdd,0x5d,0xba,0x12,0x91,0xaa,0xb4,0x3b,0x8e, -0xbe,0xa2,0x7c,0x39,0x2e,0xf5,0x32,0x77,0x96,0x3f,0x1d,0x5e, -0x5d,0x65,0x80,0x4e,0x0b,0xb9,0x94,0x32,0x39,0xb7,0xa3,0x98, -0xf7,0xfa,0x3e,0x34,0x7e,0x6e,0xca,0x63,0x85,0x8d,0x3b,0x7a, -0xfd,0x41,0x4a,0x0e,0xb5,0x79,0xf8,0x03,0x4c,0xa2,0x60,0x8d, -0xd1,0xd2,0xe9,0x79,0x71,0x01,0xf6,0x22,0xd3,0xea,0xdc,0xca, -0x0c,0x1a,0x97,0xb4,0xb5,0x55,0x4d,0x6b,0x9b,0xa2,0x8b,0x45, -0xf3,0x3c,0xb5,0xf8,0x31,0x44,0xc6,0x9f,0xa8,0x04,0xcb,0x2f, -0x16,0x4e,0x1b,0xe4,0x4f,0xb2,0x5a,0x36,0x16,0x51,0xcf,0x90, -0x97,0x02,0xcd,0x94,0xb8,0x5d,0xdc,0xbb,0xab,0xc7,0xa4,0xc5, -0xae,0x75,0x88,0xbd,0x39,0x5e,0x81,0x46,0x59,0x2e,0x1b,0x25, -0x6b,0xec,0x24,0x85,0x6c,0xa3,0xad,0x65,0xa2,0xf7,0x66,0x41, -0x10,0x68,0x94,0x63,0x18,0x8d,0x5c,0xbe,0x4d,0x84,0x02,0x37, -0x4a,0x78,0x96,0x26,0x26,0x0b,0x79,0x10,0x45,0xf2,0x9b,0x5c, -0x17,0x65,0x9e,0xa1,0x90,0x20,0x2a,0x48,0xda,0x33,0x70,0x51, -0x55,0xe3,0x4a,0xcf,0xb0,0x81,0xcc,0xf2,0x46,0xac,0xd3,0x20, -0x22,0x1e,0x7c,0x8f,0x5f,0x41,0x75,0x8b,0x03,0x04,0x36,0x5e, -0x46,0xc0,0x31,0x66,0x42,0x98,0xb1,0x99,0x6d,0x6e,0xfa,0x23, -0x2c,0x69,0x9e,0x37,0x67,0xe7,0x17,0x30,0x6d,0xa0,0xd7,0xe0, -0xe4,0xc2,0x40,0x6c,0x9c,0x21,0xcd,0xf3,0x29,0xc6,0x4d,0x69, -0x6f,0xd8,0x79,0x65,0x48,0x84,0xd9,0x05,0x36,0xe5,0x43,0xb8, -0xd2,0x3f,0xc6,0x89,0x63,0xce,0x39,0xe0,0x15,0x8e,0x36,0x82, -0x51,0x1e,0xa1,0xa5,0x99,0x5e,0xeb,0x27,0xa7,0xbf,0xa0,0x79, -0x7a,0x2d,0xf1,0x15,0x30,0x2d,0xc5,0xf9,0x78,0x62,0xa9,0x6f, -0xc8,0x76,0x2d,0x17,0xc7,0x36,0xb3,0xad,0x0b,0xd6,0x81,0xac, -0xd7,0x23,0xb5,0x83,0x37,0xc3,0x30,0x17,0xee,0x3c,0xa5,0x14, -0xf9,0xce,0xad,0x75,0xac,0xc0,0xb0,0xcd,0x9d,0xf1,0xaf,0xdd, -0x68,0x72,0x36,0x34,0xd4,0xd5,0x6f,0x18,0xcc,0x40,0xc5,0xcc, -0x4b,0x16,0x0b,0xd8,0x51,0x27,0x3c,0xb4,0x6d,0x67,0x82,0xc0, -0x9e,0x4f,0xcb,0x44,0x55,0x96,0xbe,0xcc,0x87,0x14,0x8e,0xaa, -0x2a,0xcb,0x66,0x5d,0xb5,0x39,0xcd,0x35,0xaa,0x54,0xf8,0x34, -0xfc,0x55,0x57,0x00,0x6d,0xe7,0xd2,0x96,0x3c,0xf8,0x6a,0xb8, -0x48,0xe3,0xcc,0x17,0xde,0xba,0x3c,0x0e,0xc6,0x0d,0xc2,0x4e, -0xbb,0x2a,0x0d,0xa5,0x2a,0x94,0x76,0x8c,0xe3,0x76,0x65,0xda, -0x1f,0xf6,0xb6,0xb3,0x09,0xec,0x58,0xb4,0xa4,0x4d,0x3e,0x85, -0xc5,0x12,0xc9,0x7c,0xda,0x5b,0x13,0x3d,0x59,0x87,0x67,0x9e, -0xe6,0x02,0xb4,0xdc,0x18,0x48,0xb3,0xba,0x30,0x02,0xbb,0x8c, -0xd2,0xbc,0xd4,0x04,0x69,0xde,0x97,0xf9,0x0b,0xb3,0xa9,0x0c, -0xaf,0x2c,0xf8,0x35,0x76,0x9d,0xc0,0xa2,0xcb,0xd9,0xef,0x5c, -0x2c,0x28,0x01,0x2e,0xf4,0xd8,0xef,0x81,0x2d,0xa3,0x58,0xe3, -0x47,0xa3,0x02,0x1e,0x86,0xc4,0x65,0x9e,0x59,0xf9,0x95,0x31, -0xd6,0x49,0x9e,0x8d,0x92,0x61,0x03,0x2e,0x2b,0xc5,0xb8,0x56, -0x5d,0x52,0x78,0x74,0x4f,0xd8,0xc9,0x6e,0xb5,0x34,0x97,0xde, -0x92,0x54,0x92,0x46,0xdf,0x3f,0x3f,0x8a,0x96,0xaa,0x1c,0xfe, -0x96,0xbb,0x34,0x10,0x68,0x74,0xe2,0xaf,0xb0,0x69,0xf0,0x21, -0x15,0x4f,0x39,0xd8,0x16,0xb3,0xe6,0xc8,0x01,0x3c,0x7b,0xfe, -0xea,0xf9,0xd1,0x73,0x5d,0x30,0xa2,0x6b,0x04,0x51,0x35,0x6b, -0xa9,0x60,0x68,0xda,0x3d,0x1a,0x2a,0xa0,0x78,0xb6,0xda,0x44, -0x7e,0xe7,0xef,0xb7,0xc9,0xf4,0xaa,0x60,0x72,0xda,0x34,0x78, -0x51,0x6e,0x18,0x64,0x8e,0x7f,0x98,0x24,0x0e,0x83,0xef,0x7a, -0x42,0xf3,0x10,0xdd,0x9e,0x20,0x30,0x71,0xd3,0xf3,0x71,0x23, -0xa6,0x41,0x6e,0x62,0xe5,0x74,0x94,0x24,0xdd,0xc0,0xcc,0x6e, -0xe5,0x3a,0x91,0x0a,0x0f,0x46,0xb8,0x58,0xf0,0xb0,0xa1,0x4e, -0x78,0x6f,0x4c,0xa5,0x61,0xcb,0x45,0x5b,0x2b,0x50,0x84,0x88, -0xcb,0x30,0x73,0xb8,0x51,0x65,0x60,0xd1,0xfe,0xec,0xdc,0xee, -0x63,0x94,0xdb,0x9e,0x60,0xba,0xdc,0xad,0xd8,0xd5,0x53,0x27, -0xb2,0xff,0xee,0x9d,0xe3,0x1f,0x77,0xc4,0xde,0x19,0x09,0x49, -0x6e,0x37,0x45,0x44,0x8e,0xe8,0xe7,0x63,0x08,0x29,0x1e,0x25, -0x24,0x98,0x12,0xc4,0xd6,0x16,0xfe,0x7a,0xa1,0x3b,0xa2,0xf8, -0xe7,0x45,0x5f,0x03,0x1f,0x1f,0x0f,0x12,0xbd,0x05,0x43,0x80, -0x56,0x14,0x1f,0x1f,0xff,0xb8,0xb8,0x97,0x98,0xe2,0xb0,0xae, -0xe9,0x99,0x66,0xa9,0x22,0x31,0x0a,0xae,0xb7,0xc0,0x3a,0x0b, -0x1f,0x36,0x47,0x35,0x0c,0xef,0xf2,0xf4,0xb6,0x1f,0xb6,0x83, -0x7a,0x8b,0x68,0xe6,0x9f,0x5b,0x33,0xe5,0xf5,0x5c,0xf1,0x9d, -0xee,0x63,0x4e,0x6c,0x5b,0x20,0xeb,0xe6,0xa2,0xe1,0xad,0x0a, -0x0b,0xd3,0xa3,0xfc,0xbe,0x49,0x18,0xa4,0x74,0x99,0x88,0xe7, -0x88,0x4b,0x17,0x4b,0x8d,0xb1,0x1e,0x87,0xed,0x2e,0xd9,0x93, -0x50,0xdc,0xa3,0x5e,0xec,0x3a,0x2e,0xab,0xdd,0xc5,0x57,0xbe, -0x55,0xb6,0xa4,0x6e,0x3e,0x53,0x9b,0x6b,0x41,0x15,0x5d,0x40, -0x14,0x73,0x8b,0xb7,0x15,0xdd,0xeb,0x80,0x03,0xd4,0x55,0x88, -0xe2,0x9d,0x5e,0xb2,0x02,0x99,0x6d,0x2a,0x3c,0x30,0x57,0x57, -0x7a,0x69,0x29,0xc6,0x17,0x87,0x57,0xc3,0xf2,0x32,0xd7,0x5c, -0x6c,0x58,0x58,0x6d,0x43,0xb4,0xa3,0xb7,0x76,0x90,0x20,0x76, -0x76,0xf0,0x9a,0xfa,0x65,0x5a,0x90,0xb6,0xac,0xef,0xca,0x9d, -0xd6,0xbd,0x1d,0x0c,0xc4,0x5e,0x93,0x75,0xdc,0x8e,0x7b,0xd9, -0xf1,0xc7,0x56,0x7c,0x6f,0x71,0xdc,0x4b,0x12,0x0d,0xd7,0xd6, -0x80,0x34,0x3b,0xda,0x75,0xf0,0xc7,0xc7,0x6d,0x0d,0xb4,0x03, -0x50,0x66,0x18,0xdc,0x34,0xef,0x6c,0xfd,0x11,0x82,0x22,0xde, -0x62,0xc1,0x93,0x8f,0xa5,0xb0,0xcc,0x7b,0xc3,0xf1,0xe5,0xa5, -0x88,0x6d,0x2f,0x6a,0x03,0x55,0xc0,0x76,0x97,0x03,0xa7,0x7d, -0xa3,0x9f,0xb6,0xd5,0xec,0x35,0xa8,0xa6,0x02,0x85,0x94,0xdd, -0xb8,0x72,0xb0,0xc1,0xf8,0xc6,0x18,0xb5,0x6b,0xf9,0xd1,0xe9, -0x5a,0x88,0xb3,0xc3,0x45,0x72,0x7e,0x14,0x0a,0xc2,0xa5,0xba, -0xb5,0x2c,0x27,0xa5,0xe2,0x60,0x37,0x49,0x18,0xdc,0xce,0xf8, -0x1e,0x72,0x27,0x3d,0x03,0x7e,0x2b,0xc8,0xda,0xde,0xe6,0x00, -0x27,0xe3,0xb1,0x03,0x8f,0x70,0x74,0x60,0xd4,0x10,0x7a,0x4d, -0x1f,0x97,0x05,0xaa,0x3d,0x21,0x15,0xef,0xec,0xbe,0x74,0x89, -0x07,0x38,0x82,0x34,0x91,0xd9,0xed,0x64,0x5a,0x5c,0x14,0xb8, -0xc4,0xcc,0x2e,0x53,0x44,0x33,0xcd,0x2f,0xf2,0x4f,0x37,0xf8, -0xbc,0x24,0xe3,0x76,0x0d,0xd8,0xfe,0xc0,0x7e,0xc8,0xf8,0xcb, -0x20,0x37,0x83,0x3f,0xae,0xa3,0xe3,0x7e,0x9c,0xb4,0x07,0x09, -0xb0,0x80,0xe3,0x63,0x18,0xed,0x22,0xeb,0x78,0x27,0xe9,0xa5, -0x10,0xc1,0x37,0x89,0xfb,0xc7,0xbd,0xc5,0xf1,0x83,0x41,0xd2, -0xd3,0x80,0x62,0xf6,0x97,0x30,0x26,0xd1,0x00,0xc2,0x5c,0x6e, -0x70,0xc6,0x11,0x78,0x3f,0xcc,0x99,0x83,0xb1,0x85,0x2b,0x99, -0x8b,0x80,0xe5,0x4c,0x16,0x3d,0xa8,0xcf,0xed,0x3a,0x7f,0x7e, -0x7c,0x02,0x45,0xe1,0xbf,0x5d,0x35,0xa0,0xf8,0xdd,0x34,0x8f, -0x70,0xa3,0x03,0xc8,0xc8,0xf0,0x2f,0x1a,0x77,0x47,0x51,0x2b, -0x36,0xd9,0x60,0xfc,0x8b,0x39,0x89,0x9e,0x9a,0xbd,0x54,0xe6, -0x60,0x72,0xaa,0xe7,0x4b,0x2b,0x06,0x82,0xf4,0xc6,0x05,0x7c, -0x4f,0xf6,0x22,0x8d,0x23,0xee,0xff,0xbc,0x03,0x01,0x8a,0x75, -0x5e,0x94,0x88,0x32,0x1c,0xe6,0x43,0xb6,0xdf,0xf1,0xce,0x3d, -0x68,0x19,0xd7,0x82,0xd0,0x45,0x6d,0xea,0x8f,0xcc,0xe3,0xf2, -0xc0,0xb1,0x67,0x97,0x9a,0xa9,0x40,0xa0,0x15,0xdb,0xad,0xbd, -0xa8,0x88,0x80,0x08,0x28,0xe6,0x94,0x25,0xb8,0x98,0xa0,0x32, -0x0c,0x5c,0xa6,0x4a,0x59,0x0b,0x3a,0x1b,0x20,0x70,0x3f,0x41, -0xb0,0x7d,0x48,0x1d,0x58,0x0d,0xca,0x34,0x07,0xaf,0xd1,0x6f, -0xc6,0x87,0xf9,0x70,0x7a,0x76,0x49,0xc2,0x00,0xc0,0xe1,0xca, -0x0a,0xeb,0x8a,0x3f,0x02,0x09,0x5b,0xc2,0xeb,0x2e,0x49,0x5e, -0x14,0xbc,0x01,0x86,0x57,0x86,0x7c,0x09,0xf2,0xb0,0x16,0xe7, -0xd0,0xa6,0x87,0xaf,0xac,0x69,0xda,0x55,0x5e,0x66,0x92,0xd2, -0x97,0xee,0x44,0x5d,0xa6,0x50,0xa2,0x94,0x94,0x52,0xea,0xd1, -0x84,0x47,0xae,0xa0,0x4d,0x96,0x30,0x34,0x1a,0x99,0x1e,0x1a, -0x86,0x75,0x85,0x13,0x2d,0x27,0x4f,0x3f,0x16,0x32,0x6c,0x15, -0x8b,0x9b,0xb7,0x02,0x18,0x5b,0x11,0x0d,0x8b,0xec,0x36,0x50, -0xe0,0xb8,0xa7,0xc5,0x10,0x38,0xb3,0x9b,0x4e,0x26,0xe4,0xa3, -0x1b,0xf4,0xc7,0x70,0xca,0x8f,0x27,0xdc,0x0a,0x32,0x60,0x36, -0xd3,0xa6,0x97,0x54,0xcb,0x11,0x07,0xc7,0x9f,0x60,0xa4,0xdf, -0xc6,0xd0,0x01,0x41,0x50,0x00,0x5b,0x81,0xb2,0xe8,0x95,0x44, -0xae,0xb7,0x02,0xca,0x22,0x56,0xf5,0x31,0x04,0x30,0x46,0x80, -0x65,0x27,0xa4,0x70,0x42,0x6e,0x83,0xfc,0x41,0x33,0x1b,0x8d, -0x93,0x07,0x08,0xf4,0x25,0xf1,0x02,0x94,0x95,0xe0,0x51,0xe8, -0xaa,0x40,0xd3,0x44,0x19,0x34,0x58,0x13,0x73,0xb2,0x0c,0x73, -0x10,0x8d,0xf7,0x32,0x99,0x4d,0x61,0xc6,0xd8,0xdd,0xe9,0xe6, -0xb5,0x07,0xed,0xfc,0x99,0xa0,0x0d,0x50,0x76,0x2d,0xcc,0x7e, -0xf6,0x8b,0x6e,0xab,0x55,0x58,0x62,0xf8,0xe2,0x06,0x78,0x44, -0x02,0x89,0x21,0x02,0xde,0x3c,0xbe,0x80,0xa5,0x4f,0x73,0xd4, -0xc9,0xf9,0xc6,0x75,0xbf,0x18,0xf4,0x46,0x79,0x45,0xdd,0x0d, -0xe9,0x49,0x0a,0x7f,0xbb,0xa8,0x50,0xf8,0x0d,0x0e,0xb6,0x62, -0xb7,0x72,0xb4,0xc7,0xe6,0xd6,0x88,0x1d,0x25,0x94,0xe9,0x98, -0xf2,0xfc,0x46,0x6f,0x1b,0x73,0xd2,0x40,0x1a,0x77,0xf7,0x9f, -0x66,0xe4,0x06,0x99,0x53,0xd1,0xe0,0x32,0xa3,0x2e,0x69,0xb3, -0xd7,0x76,0x74,0x3d,0x47,0x16,0xb2,0xe4,0x5b,0xac,0x7d,0x8f, -0x00,0x34,0xf7,0x82,0x44,0xf3,0xba,0xb5,0xc5,0xb7,0xd5,0x10, -0x06,0x46,0x32,0x77,0x2a,0x02,0xb9,0x77,0x2d,0xc6,0x6c,0x22, -0x88,0x3f,0x4f,0x75,0xaa,0x6e,0xc6,0xb3,0xfc,0x1d,0xa6,0x92, -0x11,0xb2,0x59,0x39,0x19,0x23,0x62,0xc3,0xfd,0xaf,0xc8,0xf4, -0x06,0x50,0xa2,0xdc,0x18,0x6b,0x43,0x94,0xec,0xe1,0x48,0xaf, -0x22,0xe0,0xdf,0x1d,0x81,0xde,0x63,0x0b,0x44,0x48,0x11,0xec, -0xa5,0x01,0xef,0x62,0x81,0x6f,0xba,0x31,0x45,0xf5,0x18,0x7e, -0x74,0x15,0x26,0x72,0x3a,0x7f,0x08,0x16,0x90,0x11,0xba,0xb6, -0x22,0x9c,0xca,0x6f,0x3a,0x24,0x5c,0x51,0xfb,0xf1,0x27,0x9b, -0x09,0x8f,0xe1,0x01,0x0f,0xb1,0xf3,0xe3,0x20,0x2b,0xe9,0xd9, -0xd9,0x81,0xcd,0x26,0x0f,0x44,0x43,0x58,0x25,0x9a,0x26,0x49, -0xda,0x25,0x36,0x65,0x2c,0x13,0x2d,0xaf,0x4e,0x52,0x87,0x56, -0x4b,0x2a,0x21,0xa6,0x4a,0xb7,0x05,0x44,0x24,0x22,0x81,0xab, -0x49,0x12,0x81,0x1d,0xf7,0xf1,0xc4,0x66,0xc0,0x9d,0x42,0xa0, -0x3c,0x4a,0x8c,0xf7,0x47,0xdf,0x73,0x83,0x92,0x6e,0x18,0xd8, -0xda,0xcf,0xed,0x2c,0x99,0x42,0xdc,0x97,0xdb,0xed,0xad,0x11, -0xc6,0x08,0xba,0x56,0x18,0x73,0x96,0x4c,0x83,0xcc,0x36,0x32, -0x09,0x56,0x3d,0xcb,0x5e,0xf0,0x26,0x26,0x25,0xa6,0x2e,0xb1, -0x18,0xff,0x36,0xf9,0xe0,0xec,0x98,0x94,0x13,0xc0,0xed,0x31, -0x0e,0x52,0x65,0xde,0x92,0x9e,0xd8,0xe7,0xdb,0x44,0x88,0x0c, -0x1b,0x9e,0xfa,0x78,0x9d,0x02,0x37,0x7b,0x43,0xc4,0xe0,0xc5, -0xc2,0xc3,0xfd,0x9e,0x76,0x32,0x35,0x15,0x70,0x8e,0x2c,0x29, -0x9e,0xfd,0x9a,0x64,0x13,0x67,0x41,0x00,0x2f,0x19,0x9c,0x43, -0xe2,0xae,0xf9,0x6c,0xae,0x12,0x51,0x5f,0x61,0xac,0x86,0xa3, -0xfa,0xda,0x6d,0x7d,0x99,0x0b,0xd9,0x21,0xb3,0x9b,0x42,0x77, -0x34,0x29,0x1b,0xc3,0x63,0x18,0x50,0x7d,0x69,0x91,0xb3,0x8e, -0xc6,0xc4,0x5a,0xb1,0xb5,0xef,0x55,0xda,0x1f,0xc7,0xe7,0xf0, -0xea,0x2a,0x36,0xd7,0xb3,0x2a,0x75,0x72,0xc6,0x2d,0x72,0xc5, -0x80,0x09,0xf2,0xa7,0xbb,0x16,0xc0,0xa1,0xca,0x63,0x10,0x59, -0x92,0xe4,0x51,0x1e,0x4b,0xba,0x83,0x27,0x31,0x27,0x21,0x55, -0x8e,0xe4,0x25,0xd2,0x0f,0x3c,0xeb,0x94,0xea,0x29,0x5b,0x03, -0x3d,0xea,0xca,0x04,0x72,0x56,0x19,0x81,0x13,0x25,0x21,0x99, -0xbb,0x75,0xe0,0x96,0xc3,0x94,0xc3,0xc7,0x90,0x0d,0x7b,0x30, -0xfb,0x48,0xac,0x11,0xfa,0x3f,0x5a,0x71,0x37,0x39,0xde,0x26, -0x2f,0x4c,0x59,0xcd,0xf2,0x5c,0x61,0x2b,0x2c,0xdb,0xe8,0x42, -0x64,0x22,0x6f,0x36,0x27,0x24,0xb3,0xf1,0xe1,0x4f,0xea,0xb8, -0x42,0x95,0x0d,0x59,0x99,0xc6,0x71,0x2f,0x3e,0x9e,0x31,0x0e, -0xb1,0xed,0x92,0x45,0x95,0x99,0xe4,0xc5,0x82,0xc5,0x33,0x58, -0xc5,0x31,0xee,0xb8,0xa8,0x9a,0x53,0x2d,0x05,0xb7,0x4b,0x89, -0xff,0x76,0x29,0x2f,0x20,0x4b,0x0e,0x4d,0x2b,0xba,0xa1,0xe9, -0x56,0xa8,0xf2,0xdc,0x4e,0xc4,0x34,0x23,0x03,0xa3,0x58,0x6d, -0xe2,0x9d,0xa5,0x72,0xdb,0x5c,0xd2,0x86,0x5e,0x15,0xc8,0x40, -0xd1,0x79,0x65,0xc2,0x0a,0x3e,0x76,0xea,0x75,0x41,0x97,0x22, -0xf0,0xa2,0x23,0xd9,0x0d,0x61,0x0a,0x36,0x77,0xc6,0x2f,0xc6, -0x17,0x38,0x6d,0x66,0xda,0x0f,0x92,0x1d,0x34,0x13,0xce,0x24, -0x6c,0xbf,0x03,0xd4,0x39,0xcc,0x62,0x53,0x8a,0x9a,0xe0,0xb0, -0xca,0x03,0x6b,0xcc,0x0f,0xc2,0x3b,0xbb,0xe6,0x10,0x85,0x28, -0x52,0x2f,0x16,0xa3,0x48,0xa3,0x11,0x9d,0x46,0x55,0x57,0x59, -0xe8,0x97,0x5b,0x6a,0xf2,0x94,0xc7,0x15,0x89,0x06,0x32,0x62, -0x8a,0x57,0x6c,0xd7,0x9b,0x52,0xe0,0x56,0xed,0x49,0xf9,0x79, -0x7c,0x16,0x0b,0x09,0x47,0xcf,0x6f,0xd3,0xd8,0x12,0x16,0x4d, -0x74,0xcc,0x00,0x0a,0xa6,0x9e,0x2c,0xcd,0x74,0x2d,0x07,0x95, -0x3d,0x2d,0x7e,0xa1,0xd8,0xd9,0x3a,0x09,0xda,0xb3,0xe4,0x43, -0x1b,0xb5,0xa5,0x38,0xbe,0x1f,0x5f,0x80,0x99,0xac,0x51,0xa1, -0x12,0xfa,0xc6,0x88,0x58,0xce,0xfb,0x8d,0x5a,0xe5,0x45,0x30, -0x0c,0x8c,0xe5,0x87,0x5b,0xf8,0xf2,0x28,0x59,0x4d,0x71,0x17, -0x02,0xff,0x53,0x64,0xa4,0x66,0xbf,0x4c,0xc4,0xc7,0x81,0x4f, -0xfc,0xbf,0x1e,0x8e,0x2a,0x94,0x78,0x99,0x9d,0x31,0xcf,0xae, -0xe1,0x9e,0xe2,0x9d,0x39,0xbb,0x5b,0xc4,0xe8,0x5d,0x8b,0xbf, -0xc1,0xba,0x02,0xbb,0x00,0xbb,0xb4,0xac,0x88,0x1d,0xc5,0x4e, -0x5c,0xd7,0x89,0x1a,0x45,0x8d,0xa7,0xc8,0x71,0x2a,0xde,0xd5, -0x10,0x6b,0x57,0xa3,0xa7,0x57,0x1b,0xee,0x89,0xbc,0xdc,0xeb, -0xea,0x32,0x17,0xf7,0x09,0x51,0x89,0x70,0x4f,0xa6,0xbd,0x42, -0x11,0x1f,0x6c,0xb0,0x65,0x67,0x98,0xd6,0x28,0xd7,0x0f,0x58, -0xd5,0x96,0xee,0xc8,0x4c,0x6b,0x95,0x12,0x1b,0xe2,0xa0,0xd8, -0xf4,0x16,0x34,0x13,0x43,0x3d,0xae,0x62,0x72,0xb6,0xe6,0x95, -0xac,0x27,0x65,0x30,0xb2,0xfa,0x75,0x30,0x70,0xd0,0x6a,0xc9, -0xe2,0xa6,0xe5,0x00,0xa9,0x32,0x4e,0xa7,0xb4,0xaa,0xac,0x16, -0x70,0x0d,0xf8,0x05,0x65,0x97,0xd0,0x15,0x41,0x78,0xad,0xda, -0x38,0x58,0x18,0x12,0x3d,0x08,0x82,0x15,0x00,0xfe,0xe6,0xc5, -0xc2,0x52,0x41,0x78,0xad,0x25,0x5f,0x35,0xaf,0x0c,0xa5,0xe5, -0x5d,0xb1,0xe6,0x1a,0x63,0xdc,0x75,0xa5,0x03,0xa0,0x3a,0x31, -0x85,0x66,0x56,0xa2,0xcc,0x0c,0x03,0xe3,0x45,0xea,0xd9,0xf0, -0x00,0x84,0x36,0x85,0xee,0xa0,0x04,0x5f,0x1b,0x8e,0x4f,0x9a, -0x0e,0x5b,0xac,0xa8,0x97,0x55,0x14,0xe3,0x77,0x1c,0xb0,0x18, -0xf1,0x5f,0xe4,0x51,0x42,0x37,0x30,0x5b,0x54,0xde,0xa9,0x8f, -0x79,0x99,0x4d,0xfe,0xab,0x9c,0x8c,0x39,0xac,0x26,0x89,0x31, -0x68,0x9a,0x97,0xd5,0x58,0xea,0x41,0xae,0x33,0xd3,0xbb,0x31, -0x2b,0x04,0xb7,0x5f,0xa4,0x7c,0x9d,0x28,0xc4,0x24,0x15,0x08, -0x2b,0xf0,0x46,0xdb,0x52,0xb7,0xea,0xa0,0xeb,0x7a,0x51,0x76, -0xe4,0x82,0xe0,0xd2,0x2a,0x03,0x61,0x7b,0xc4,0x6a,0x93,0xf8, -0xaf,0x6d,0xde,0xee,0x90,0xca,0x67,0x3e,0xbb,0x2b,0xc8,0xe3, -0x99,0x18,0xe0,0xce,0xcf,0xed,0xe0,0x4b,0x95,0xd1,0x8d,0x76, -0x92,0xd5,0xb6,0x7c,0x3a,0x99,0x7c,0x28,0xc0,0x17,0x7f,0xd5, -0xe8,0xf1,0xcc,0x65,0xf9,0xfa,0x2a,0x2d,0xf9,0x7e,0x2c,0xfd, -0x00,0xb9,0x42,0xdd,0xc4,0xb9,0x52,0x11,0x0d,0xb2,0x0c,0x73, -0x6e,0xf0,0xfd,0x42,0xfb,0x47,0x86,0x63,0x02,0x78,0x78,0xa3, -0x51,0x08,0xc6,0x6a,0x35,0xee,0xf3,0x39,0x3f,0x91,0x86,0xe4, -0x4c,0x19,0x9d,0xd3,0x6a,0x71,0xd6,0xd4,0xd9,0xe6,0x0c,0x44, -0x20,0x1a,0xc1,0x43,0xca,0x30,0x7c,0xf7,0xc8,0x43,0xa5,0xc2, -0xa1,0xeb,0x85,0x71,0xdf,0x24,0x25,0xcc,0x53,0x59,0x02,0x96, -0xcd,0xda,0xca,0xd9,0x62,0x80,0xbf,0x0a,0x8e,0xc4,0xd2,0xea, -0x49,0x53,0x05,0x5d,0xd2,0xf3,0xe8,0xa9,0xd6,0x97,0xb2,0xf4, -0xe7,0x81,0xc1,0x9a,0xc2,0x15,0x25,0xb7,0xb6,0xc6,0x4e,0xd0, -0xc2,0x59,0x85,0xd0,0xfa,0x76,0xf1,0x50,0xeb,0x6f,0xf6,0xb1, -0xd8,0xcf,0x96,0xfd,0x51,0x07,0x92,0xf4,0xea,0x48,0x4d,0xfd, -0xef,0xab,0x2b,0x28,0xdb,0x8c,0x38,0x1d,0x3a,0x2c,0xf7,0x47, -0x0b,0xa3,0x01,0xf3,0x27,0xd1,0x46,0x68,0x0d,0x35,0x1f,0x3f, -0x07,0x75,0x9f,0xfe,0xfe,0xba,0x23,0x62,0x4b,0xb4,0x60,0x53, -0x36,0xcd,0x8a,0xb1,0xa6,0xa5,0x86,0xa3,0xd1,0x5b,0x3d,0x1d, -0x5f,0x8c,0xc3,0x5b,0x99,0xcc,0xdd,0x57,0x34,0x6d,0x95,0xe8, -0xcd,0xcc,0x2f,0x65,0xe2,0x66,0xf8,0xbd,0xe4,0xc3,0xd0,0x86, -0x37,0x48,0x93,0xd3,0xa2,0x16,0xc0,0x74,0xa5,0x69,0x0c,0x7f, -0xe3,0x4a,0xd7,0x4c,0xc1,0xc6,0x25,0x4e,0x5c,0x73,0xf9,0x82, -0x3e,0x5d,0x0f,0x80,0x89,0x9b,0x18,0x6c,0xcb,0x1a,0xe6,0x70, -0xa8,0xdf,0xc8,0x28,0xda,0x32,0x0b,0xe9,0x87,0x93,0xa8,0x30, -0xd2,0x34,0x98,0x1b,0xd8,0x5c,0xbe,0x5b,0x67,0x02,0xc4,0x19, -0x60,0xba,0x82,0x28,0x0d,0x5b,0x7a,0x76,0x85,0x9a,0x4e,0xae, -0x61,0x89,0x30,0x9a,0x24,0x32,0x37,0x55,0x37,0x73,0x1f,0xab, -0x09,0xef,0xe4,0x61,0xcc,0xfc,0x55,0xc6,0x7a,0xdf,0x63,0xcb, -0x05,0x9f,0x2a,0x1e,0xb7,0x1e,0x02,0xb8,0x30,0xf0,0x97,0x9d, -0x09,0x5e,0x0f,0x3f,0xe4,0xe0,0xcd,0xa8,0xb4,0xf1,0xf7,0xd5, -0xe4,0xf4,0x17,0x18,0xbc,0xe4,0x2b,0xad,0x84,0xb8,0xb8,0xb4, -0x43,0x55,0x11,0x5f,0xc2,0xcf,0x76,0xbb,0xc5,0x63,0xf2,0x6b, -0x66,0xee,0x9d,0x81,0x0f,0x8a,0xd3,0x5f,0xfa,0x98,0xd8,0x2f, -0x06,0x70,0xee,0x6e,0x5a,0x4e,0xa7,0x3b,0xd1,0x1c,0xe4,0xdb, -0xb7,0x78,0xcd,0x2a,0x86,0x47,0xa5,0xc5,0x8b,0xd1,0x95,0xcf, -0xa0,0x21,0x17,0x35,0xac,0x70,0xc1,0x7c,0x36,0xbc,0x50,0xfa, -0x1f,0xba,0xe9,0x82,0x1d,0x90,0x66,0xa4,0xc3,0xe9,0x67,0x0a, -0xeb,0x4a,0xc9,0xb6,0x39,0x5d,0xcc,0x43,0xce,0xe1,0xd8,0x0a, -0xcf,0x8d,0x4f,0x70,0x4e,0x1e,0xe0,0x1d,0xd8,0x6e,0x39,0x1b, -0x9e,0x7d,0x68,0xc3,0xf0,0x05,0xbf,0x75,0xc5,0x58,0xcb,0x6d, -0xb9,0x05,0x95,0x99,0x83,0x6e,0x82,0x34,0x3d,0x1f,0x8f,0x80, -0x22,0xf0,0x1e,0x23,0x72,0x93,0xae,0x39,0x24,0xa3,0xfc,0x4a, -0x6d,0x5b,0x5b,0x12,0x1c,0xce,0xba,0x2f,0xe8,0xe6,0x65,0x80, -0xd4,0x12,0x8d,0x5f,0x98,0xfd,0x36,0x29,0x46,0x15,0x5c,0x8b, -0xc5,0xe6,0x26,0x66,0x13,0xd0,0x62,0x41,0xa8,0x71,0x4d,0x33, -0xe5,0x51,0x6a,0x41,0xef,0xa4,0x64,0xdd,0x5d,0xce,0xac,0xb2, -0xf6,0xc9,0xd1,0xd1,0x3b,0x76,0x37,0x10,0x9e,0xd0,0xe3,0x3a, -0x31,0x9a,0xcc,0x4f,0xaf,0xf2,0xff,0x33,0x9f,0xcc,0xd0,0x0e, -0x00,0xbc,0x20,0xe1,0x9d,0x30,0x99,0x32,0x1f,0xff,0xea,0xde, -0xe4,0xac,0xa9,0x14,0xd6,0xd4,0x85,0xa5,0xc1,0xe0,0xf2,0x57, -0xf9,0xaa,0x45,0x4e,0x24,0x95,0x59,0x30,0x1d,0x80,0x3c,0x1f, -0xcf,0x8a,0x19,0x30,0x2e,0xab,0x8e,0xe5,0x51,0x42,0x81,0x29, -0xb6,0xb6,0xbc,0x57,0xf3,0xe1,0xec,0x4f,0x9a,0xc6,0x47,0xa0, -0xde,0xe2,0x56,0x16,0x83,0xc9,0x8c,0xf5,0x9b,0x49,0x09,0xa3, -0x79,0x9d,0xd1,0x84,0xc3,0x06,0xe0,0xb9,0x3f,0xf9,0xd0,0xaf, -0xab,0x93,0xf6,0xb3,0x5d,0xee,0xe6,0xbe,0x7e,0xd3,0x0b,0x15, -0x97,0x81,0xbc,0xed,0x6d,0x3c,0x3a,0x22,0x28,0xba,0xe9,0x5a, -0x84,0x28,0x8a,0xfd,0x4c,0xe7,0x77,0x0b,0x0d,0x6b,0xbe,0x4d, -0xcb,0xc5,0xee,0x43,0x41,0x05,0x46,0xe8,0xd1,0xaf,0xbd,0x28, -0x0c,0xe5,0x68,0x61,0x32,0xd1,0x4d,0xf4,0xf7,0x92,0x96,0x8b, -0x86,0x29,0xdc,0x95,0xc7,0xd3,0x1d,0x98,0x6e,0x38,0x7d,0xdd, -0x70,0xac,0xaa,0x2e,0x36,0xa8,0x16,0x9f,0xbc,0xc1,0xb2,0x8b, -0x85,0x71,0xd6,0x61,0x05,0xc0,0xa1,0xfd,0xf9,0x83,0x51,0x8f, -0xf4,0x1c,0xa8,0x9d,0x40,0x09,0x14,0x47,0x02,0xea,0x6d,0x49, -0xda,0x0f,0x92,0xc7,0xc7,0xe0,0x7f,0x18,0xec,0x2f,0xca,0x07, -0x51,0x4b,0x96,0x6e,0x45,0xfd,0x9f,0xf7,0x07,0x0f,0xf6,0x35, -0xfb,0x29,0xa4,0x8a,0x6c,0x78,0xc5,0x1e,0x9c,0xed,0x81,0x27, -0x68,0x49,0x67,0x74,0x76,0x40,0x95,0x3c,0x7d,0x73,0x70,0xf0, -0xfc,0xf5,0x91,0x19,0xf2,0x91,0x77,0xd0,0xff,0xf4,0xd9,0x93, -0xa3,0x27,0x5e,0x96,0x1d,0x66,0xf8,0x8d,0xae,0xf5,0xf1,0x35, -0x0e,0xc6,0x26,0x56,0x0d,0xae,0x3d,0xc9,0xc4,0xb6,0x99,0x3d, -0x98,0xbb,0xe4,0xa0,0xc4,0xc3,0x26,0xc0,0x8e,0x7a,0x73,0x1e, -0x47,0x8f,0x37,0xb7,0xb7,0xa3,0xa4,0xc7,0x8e,0xe3,0xfc,0xbc, -0xed,0xed,0x7d,0x4d,0x10,0xbe,0xe2,0xe0,0x8a,0x2d,0x31,0x93, -0x6b,0x52,0x91,0x04,0x09,0xc8,0x4e,0xf9,0x5c,0x1a,0x84,0xc5, -0x47,0x1c,0x21,0x0c,0x6c,0x8c,0x4d,0xe3,0xbb,0x5c,0x72,0x45, -0xf0,0x30,0x51,0xdc,0x9d,0x70,0x65,0xe2,0xbb,0xe7,0xdf,0xbf, -0x7c,0xfd,0xfd,0xc9,0xf3,0xd7,0xcf,0x4e,0x8e,0x9e,0x7c,0xff, -0x9c,0x9b,0x86,0xac,0xab,0xa0,0xb8,0xa6,0x94,0x34,0xb5,0x88, -0x8b,0x54,0x8b,0x0c,0x6c,0x7c,0x98,0x28,0xa3,0x07,0xae,0xab, -0x93,0x7c,0x67,0xec,0x0e,0xec,0x15,0x5c,0x9b,0x60,0xba,0xc4, -0xc7,0x26,0x9b,0xb5,0x4a,0xa8,0x00,0x14,0x24,0x5a,0x6d,0xb2, -0xa0,0xf1,0xf0,0xe8,0xc9,0xbb,0xa3,0xbf,0x90,0xca,0x10,0x9f, -0xf2,0x56,0x2c,0x49,0x29,0x3d,0xea,0x69,0x53,0xd3,0xc1,0x8f, -0xf5,0x62,0xfa,0x1f,0x74,0x26,0xa6,0x87,0xed,0x2e,0xb9,0x88, -0xe8,0x01,0x4c,0x1a,0x50,0xb4,0xcb,0x3d,0x89,0x93,0xd0,0x42, -0x46,0x51,0x08,0xc7,0x11,0xe1,0xbe,0x7c,0x10,0xc3,0x2d,0x0a, -0xc4,0x8d,0x22,0xbe,0x31,0xc2,0x2e,0xf5,0x8e,0x6f,0x56,0xfc, -0x2e,0x8d,0xb0,0xcd,0x25,0xb3,0xa3,0x4b,0x2d,0x48,0x70,0xf6, -0x74,0xe3,0xe3,0xb0,0xdc,0xd0,0x5c,0xf7,0xf4,0x0a,0x8d,0xad, -0x11,0x66,0x63,0xa6,0x21,0xce,0x27,0x10,0x2f,0x4d,0x53,0xb6, -0x81,0xcb,0x30,0x38,0xa8,0xc1,0xcf,0x03,0x23,0xec,0x08,0x47, -0x25,0x09,0xa6,0xd8,0x30,0x4b,0x39,0x87,0x12,0x79,0x65,0xa7, -0x66,0x45,0xb0,0xe1,0x32,0x8b,0xd1,0x28,0x1f,0xbf,0x9d,0xe6, -0xba,0x59,0xc7,0xf9,0xf4,0x87,0xa3,0x83,0x57,0x6c,0x85,0x6e, -0x0d,0x55,0x1e,0x83,0x81,0xca,0xd6,0xd5,0xac,0x0b,0xd3,0xdb, -0x07,0x3f,0xc2,0x53,0x60,0x97,0x08,0x8d,0xc1,0x0a,0x19,0x58, -0x98,0xc2,0x0b,0x6b,0x0d,0x34,0x04,0xf5,0x6d,0x61,0x7d,0xc3, -0xeb,0x9b,0xae,0x60,0x35,0xaf,0xdf,0xbc,0x3e,0x79,0xf2,0xea, -0xed,0x0f,0x4f,0x5e,0xbf,0x3f,0x78,0xfe,0xee,0xe5,0xd3,0xca, -0x1a,0xec,0xa1,0x8c,0xb6,0xfe,0x93,0x63,0x8c,0x60,0xa7,0x3d, -0xd5,0x75,0xa3,0x59,0x5c,0x2b,0xea,0x46,0xd2,0x06,0x47,0x7c, -0x9a,0x4b,0xdc,0xc7,0xc4,0x0b,0x48,0xf4,0x85,0xae,0x43,0xee, -0xaf,0x1f,0xa7,0x05,0x68,0x12,0x4f,0xe7,0xe7,0xbc,0x04,0x5e, -0x8c,0xb5,0xd8,0x8c,0x7e,0x10,0xe6,0x33,0xbb,0xfa,0x81,0xa1, -0x30,0xc0,0x28,0xfd,0x0f,0xc5,0x0b,0x1b,0xef,0x17,0xd7,0x5c, -0x27,0xa4,0xc2,0x92,0x2a,0x17,0xde,0x5b,0x9d,0x50,0xbf,0x86, -0x42,0x94,0x79,0xac,0xab,0xba,0x58,0xe8,0x5c,0xdc,0xc5,0x11, -0x29,0x08,0x4b,0xcf,0x8b,0x45,0x67,0x13,0x7a,0xd4,0x17,0x85, -0xc0,0xc4,0x4e,0x13,0x8b,0x73,0x08,0xa8,0x26,0xec,0x81,0xf2, -0x88,0xc2,0xa6,0xd7,0x9e,0x13,0xe3,0x36,0x0e,0xce,0x41,0xaa, -0x74,0xa2,0xa7,0x12,0x53,0xe7,0x13,0x94,0x4c,0xae,0xc8,0xa8, -0x6f,0x3e,0x2d,0xc4,0xbb,0xe6,0xe5,0x9b,0x9a,0x2b,0x6f,0x52, -0x37,0x11,0x9f,0x79,0xff,0xee,0xa5,0xe1,0x30,0x86,0xc0,0x0d, -0x26,0x10,0x7d,0x6a,0xc0,0xc3,0xfd,0x2c,0xba,0x4f,0x4f,0xb5, -0x83,0x8a,0x81,0x34,0x0c,0x48,0x3d,0xf0,0x8c,0xad,0xda,0x8b, -0x76,0xf6,0xa3,0x34,0xd2,0x2b,0x82,0xde,0x26,0x68,0x59,0xc0, -0x6b,0xfe,0x95,0x4d,0xbe,0x5e,0x33,0xee,0xc8,0x76,0xc4,0x24, -0xa8,0x0a,0x44,0xa5,0x2c,0x33,0x7d,0x16,0xdb,0x81,0xa2,0x69, -0x40,0x66,0x22,0x8c,0x8d,0x99,0xcd,0x71,0x65,0x35,0x9f,0x47, -0x10,0x09,0x6b,0x0b,0x03,0xde,0x52,0x77,0x3b,0xd6,0x40,0x40, -0xa8,0x0a,0xf3,0xec,0xc4,0x15,0x64,0x1e,0xd8,0x8f,0x7a,0x7c, -0x8a,0xf3,0xb5,0xca,0xee,0xa2,0x7e,0xe0,0x27,0x38,0xac,0xed, -0xf1,0x94,0x0a,0x39,0x7a,0xb6,0xf3,0xf3,0x63,0x0a,0x91,0xf0, -0x31,0xdd,0x1e,0xb4,0x92,0x98,0xe2,0x20,0xf0,0x2b,0xbe,0x3c, -0xc8,0x00,0xa0,0x97,0x82,0x6d,0x5d,0xff,0xe7,0x68,0xf0,0x20, -0xc2,0x90,0x0b,0xf7,0xfb,0x3f,0xdf,0x1f,0x3c,0xb8,0x9f,0x2c, -0xb4,0xcc,0x72,0x5c,0xea,0xa2,0x49,0x2f,0x79,0x40,0xf1,0x9e, -0x77,0x30,0xee,0xf3,0xfe,0x8e,0xf2,0x57,0x39,0xae,0x4b,0x4b, -0x3e,0xa2,0x42,0x92,0x78,0x76,0x94,0x10,0xd8,0xb3,0x1d,0x47, -0x4e,0x48,0x00,0xd0,0x07,0x44,0xe8,0xaa,0x98,0x0c,0x4a,0xb9, -0x0f,0x29,0x9a,0x98,0xd8,0x52,0x83,0x96,0x8c,0xe1,0xfa,0x09, -0x24,0xec,0xa8,0xfa,0xe5,0xdf,0x92,0xb7,0xa3,0x7c,0x69,0x2a, -0xdb,0x01,0xf1,0x05,0xa3,0x41,0x68,0x59,0x45,0x63,0x95,0xf2, -0x14,0x64,0x1e,0xf7,0x31,0xe5,0xb8,0x8f,0x30,0x83,0x01,0xc0, -0xb8,0x29,0x02,0x57,0x14,0xe3,0xf3,0xd9,0xcd,0x02,0x8c,0x05, -0xca,0x5e,0x92,0xea,0x06,0xd8,0x59,0x5c,0x0f,0x8b,0xab,0xd9, -0x24,0x5d,0xcc,0xf2,0xab,0x74,0xf1,0x9f,0xc9,0x4e,0xa1,0x1a, -0x58,0x27,0xb4,0xc6,0xcf,0xc7,0xff,0xb9,0xfd,0xff,0x2e,0x36, -0x16,0x9b,0x68,0x85,0x28,0xf7,0x4b,0x99,0xd9,0xd4,0x46,0xc3, -0x69,0x3e,0x54,0xa7,0x53,0x70,0x38,0xa5,0x2e,0xa7,0xaa,0xb8, -0xbe,0x50,0x1f,0x4f,0xa7,0x51,0xa2,0xfc,0x4d,0xdb,0x77,0x72, -0x9f,0xe8,0x4a,0xeb,0x52,0xe8,0xb1,0x45,0x8d,0x46,0x6a,0x34, -0x53,0x57,0x7a,0xbb,0xa0,0x66,0xa7,0x93,0xd1,0x67,0x35,0x1b, -0xa9,0xd9,0xf9,0x64,0x32,0x53,0xb3,0x4b,0xfd,0x5f,0x3e,0xd4, -0xaf,0x55,0xac,0x2f,0xbd,0x3d,0xa5,0x43,0x3b,0xbd,0x51,0xd3, -0x59,0x05,0xda,0xc2,0xf9,0xca,0x71,0x38,0xfa,0x5e,0x85,0x76, -0xd5,0x97,0x04,0x1b,0xe0,0x3b,0x31,0x7b,0x85,0x95,0x6b,0xc4, -0xd1,0x08,0x9c,0xcd,0x29,0x70,0xfb,0x76,0x76,0xa5,0x37,0x5a, -0x25,0x04,0x99,0x43,0xd4,0xb8,0x99,0x53,0x67,0x43,0x72,0x02, -0x73,0x86,0xa7,0x54,0x70,0x28,0xac,0x46,0x85,0xfe,0x2d,0x7e, -0x53,0xa3,0x2b,0x85,0xf7,0xb3,0x72,0xf8,0x31,0x70,0xd0,0x72, -0x1a,0xee,0xb2,0xa3,0x2e,0xf7,0xd4,0xe5,0x43,0x75,0xf9,0x48, -0x5d,0x7e,0xa5,0x2e,0xbf,0x56,0x74,0x05,0x49,0x5d,0x52,0xab, -0x43,0x87,0x8d,0x81,0x8e,0x1b,0xa5,0x29,0x9a,0xeb,0x3d,0xea, -0x6f,0x4a,0xf7,0xe3,0x8d,0x46,0x46,0x21,0x6a,0x54,0x49,0x2e, -0x8d,0x34,0xa7,0xd2,0xd2,0x87,0x9a,0x5f,0x01,0xd7,0xf2,0x77, -0xf2,0x5f,0xda,0x9a,0xee,0xa3,0xd5,0xf0,0x54,0x0f,0x9c,0xe1, -0xd9,0x74,0x32,0xfe,0x7c,0xad,0x4e,0xd5,0xe9,0xa8,0xd0,0xff, -0x26,0xea,0xb4,0xb8,0xc0,0x11,0x55,0xc0,0x87,0x43,0x18,0x68, -0xfc,0xdc,0xf3,0xb1,0xca,0xaf,0xf5,0x87,0xc1,0x99,0x3d,0x0e, -0x33,0xa0,0xfc,0xc3,0xe9,0x88,0x3d,0x7b,0xe3,0x37,0x0c,0xa7, -0x1f,0xd4,0xaf,0x6a,0x3a,0x3f,0xfd,0xac,0x70,0x08,0xa8,0x52, -0x95,0x5a,0x52,0x50,0xe5,0x35,0xec,0x65,0xca,0x9b,0xe1,0x58, -0x81,0x28,0xf7,0x21,0x87,0x9f,0xc9,0xf8,0x42,0x69,0xd9,0x4e, -0xff,0xd3,0x63,0xae,0xd0,0xdb,0xdb,0xd9,0x4c,0xcd,0x95,0x66, -0x79,0xf0,0x8d,0xc1,0x02,0xea,0x86,0x96,0x69,0x17,0xf0,0x5d, -0x40,0x8e,0xf1,0xc4,0xac,0xa8,0x36,0x85,0x9c,0x34,0xfe,0x58, -0x09,0x9a,0xb1,0x61,0xac,0x82,0xc9,0x3d,0x2d,0x8b,0x8e,0x84, -0x53,0xbd,0xf9,0x81,0xfe,0x83,0x6b,0x31,0xd0,0x46,0x97,0xd3, -0xfc,0x5c,0x5d,0xe9,0xcf,0x19,0xe5,0xe5,0x99,0x2a,0xa7,0x67, -0x6a,0x5e,0xe6,0xba,0x3d,0x0c,0x79,0x54,0xbc,0x4a,0x9b,0xc1, -0x2c,0xba,0x04,0x3b,0xe4,0x4a,0x2f,0x33,0xfa,0xef,0x4c,0x0d, -0x3f,0x15,0x9a,0xe8,0x0b,0x3d,0x49,0x27,0x53,0x75,0x3a,0xc1, -0x20,0xc1,0x67,0xf9,0xd5,0xd5,0x0d,0xdc,0x45,0xd2,0xad,0x07, -0xcf,0xba,0x51,0xcf,0xf0,0x19,0xfc,0xe6,0x29,0x38,0x29,0x43, -0x5e,0x30,0xc1,0xbf,0x25,0xfe,0x81,0x66,0x87,0x53,0x8f,0xe1, -0xd9,0x0c,0xb4,0x8b,0xd3,0x51,0x89,0x83,0xf7,0x5c,0x8b,0x53, -0x3c,0x1e,0x4b,0xfd,0x5b,0x5c,0x5c,0xce,0xf0,0x53,0xae,0x34, -0xa5,0xea,0x12,0xf0,0xe6,0xaa,0x28,0xa1,0x5f,0x31,0x05,0xfe, -0xcc,0x87,0x17,0xb9,0x1a,0x4f,0xf0,0x83,0xc7,0x93,0x8f,0xd3, -0x21,0xf8,0xb1,0xb9,0xd2,0xff,0x7e,0x53,0xa0,0x56,0xc5,0x3f, -0x50,0xdb,0x74,0x0e,0x21,0x0a,0xd9,0x69,0x3d,0x9e,0xbc,0x00, -0x89,0xe5,0xe5,0x10,0xde,0x69,0x14,0x0c,0x61,0x74,0xcc,0xaf, -0xaf,0x41,0xd5,0xa3,0x5f,0x2e,0x72,0xcd,0x69,0x8a,0x99,0x1e, -0xe3,0x60,0x2e,0x8a,0xed,0xa6,0x1b,0x81,0x44,0x9a,0xdf,0x88, -0x96,0x8f,0xc5,0x68,0x76,0x09,0x63,0xc3,0x0a,0xb4,0x59,0x93, -0x23,0x2a,0x3d,0x81,0x6a,0x9d,0x4e,0x98,0x95,0x32,0x52,0xfd, -0x41,0x42,0xfe,0x49,0xe4,0x4a,0xec,0x16,0x62,0x27,0x6c,0xd5, -0x14,0x06,0xff,0x1f,0x57,0xb0,0xd2,0x46,0x70,0x12,0xea,0x3b, -0x44,0x81,0x15,0xfb,0xd5,0xcb,0xd7,0xff,0xfb,0x5f,0x27,0xef, -0xdf,0xbd,0x72,0xac,0xbc,0xba,0x10,0xc4,0xbc,0x12,0x24,0xbd, -0xfe,0x93,0xed,0x9f,0x86,0xdb,0xbf,0xef,0x6e,0x7f,0xdb,0x3e, -0xf9,0x5b,0x4b,0x2f,0x7f,0xff,0x2b,0x39,0x3e,0x7c,0xa0,0x99, -0x7f,0x79,0xdc,0x3e,0xee,0x1e,0xab,0xe3,0xf8,0x38,0x39,0xbe, -0x3d,0x5e,0x1e,0x3f,0x3e,0xde,0x1f,0xec,0xa8,0x83,0x27,0x2f, -0x5f,0x1d,0xbd,0x71,0x8b,0x0c,0xe3,0xd9,0xe9,0x86,0xb6,0x0d, -0x20,0xed,0x73,0xd3,0xa2,0x5e,0x63,0x13,0xf7,0x42,0x42,0x95, -0xd0,0xb5,0x8e,0x3c,0xe1,0x5e,0x89,0x9e,0xd5,0xd3,0xe1,0x47, -0x54,0x2f,0xd0,0x86,0xba,0x3f,0x50,0x1f,0x51,0xa2,0xc8,0x6a, -0x84,0x0c,0x94,0x4f,0xe0,0x62,0x2c,0xdc,0x04,0xa2,0xb3,0x00, -0x54,0xbd,0x54,0x4f,0x5d,0x98,0x02,0x30,0x9d,0xb2,0xd0,0x6d, -0x4a,0xcc,0x38,0xaf,0x4b,0x3b,0x5a,0x5d,0x3d,0x0b,0x9a,0x61, -0x0b,0x26,0xdd,0x04,0x6e,0xb2,0x98,0x8d,0xaa,0x32,0xfe,0x2f, -0x33,0x1b,0x2a,0x9f,0x2e,0x46,0x65,0x11,0xb7,0x06,0xde,0xdb, -0x01,0xff,0x62,0x64,0x24,0x45,0x9a,0x23,0xfa,0x1a,0xde,0x27, -0x42,0x65,0xd6,0xc6,0xbd,0x48,0xe4,0xb7,0xb4,0x61,0x7c,0xc3, -0xb5,0x19,0x53,0x82,0xb4,0x70,0xd1,0x30,0x12,0x40,0x89,0x8f, -0xae,0xb2,0x87,0xf6,0xba,0x09,0xee,0xee,0xd8,0x02,0x30,0xfb, -0x35,0xae,0x04,0x9b,0xdb,0x91,0x81,0x1b,0xdc,0x56,0xb8,0x39, -0x37,0xbd,0x1a,0xd2,0x4e,0x4a,0x0f,0x27,0xe1,0x2d,0xd7,0x50, -0x97,0x5b,0x1f,0xf1,0x75,0xe7,0xbc,0xa0,0x46,0xe5,0x34,0x9c, -0x1d,0xdd,0xeb,0xf0,0xf8,0xf6,0x9d,0x84,0x08,0x90,0x3e,0xe5, -0x4b,0xca,0x7a,0xb0,0xe8,0x81,0xe0,0xde,0x82,0x88,0x51,0x74, -0x93,0x98,0x6f,0x07,0x09,0x8d,0xb8,0x40,0x6d,0x6f,0x06,0xf3, -0xfd,0x3e,0xb2,0x95,0x87,0xce,0x38,0xb3,0x58,0xad,0xed,0x19, -0xdd,0xdd,0x75,0x2e,0x3e,0x4e,0xda,0x39,0x6c,0x8c,0x4e,0xd0, -0x22,0x3e,0x16,0x05,0x54,0xd3,0x49,0x88,0x83,0xf1,0xdc,0x31, -0x92,0x91,0xdf,0x89,0x5c,0x48,0x44,0x86,0xb9,0xde,0x64,0x6f, -0xf1,0xff,0x3a,0xd7,0x1f,0xc0,0x37,0xa4,0x78,0xc3,0x84,0x37, -0x30,0xdb,0xb3,0xc9,0x2b,0xd8,0xae,0x3c,0x85,0xed,0x4a,0x32, -0x48,0xd4,0x89,0x9e,0x18,0xcf,0xc1,0x3d,0x5d,0x2c,0xf1,0xe9, -0xc9,0xc1,0xa7,0x20,0x5e,0x2d,0xe6,0x03,0xcd,0xbd,0x7a,0xac, -0xb3,0x28,0x0f,0x87,0xe7,0x6c,0xc7,0x6b,0x7c,0x5c,0xf4,0xaa, -0x5a,0x4e,0xba,0xf3,0x6e,0x3f,0x80,0x18,0xe9,0xad,0x6e,0x46, -0xb8,0xf3,0x06,0xd6,0x91,0x6e,0x6f,0x43,0x77,0xd8,0xd7,0x29, -0x87,0x5e,0x2a,0x52,0xb6,0x53,0x65,0x33,0x38,0x77,0xe1,0xb5, -0xd4,0x74,0x1d,0x20,0x41,0x10,0xc1,0x43,0xcf,0xeb,0x48,0x91, -0x8b,0x24,0xc5,0x2e,0x05,0xe1,0x8a,0x05,0xdc,0xe7,0xcf,0xa3, -0x41,0xd7,0xfb,0x1a,0x31,0x40,0xf4,0xec,0x1a,0xd2,0x85,0x23, -0xdb,0xa5,0x7c,0xb9,0xbd,0x8c,0x45,0x05,0xca,0x02,0xfa,0x4d, -0x9c,0x2c,0xe9,0x70,0xe0,0xb4,0x9c,0x5c,0xa1,0x75,0x01,0xdc, -0x42,0x1d,0x6b,0x2e,0x49,0xbc,0x17,0x59,0xef,0x4e,0xe1,0x6a, -0x7f,0xc2,0x80,0x60,0xea,0xeb,0x8c,0xc2,0x70,0x32,0x0a,0x85, -0x31,0xbc,0x6e,0x6d,0x05,0x48,0x49,0x27,0xc7,0xb0,0xf6,0xc6, -0xdb,0xa9,0x26,0x03,0x90,0x9d,0xb4,0xfd,0x13,0x74,0x99,0x99, -0x80,0x6a,0xcb,0x4f,0x52,0x34,0x33,0xc0,0x25,0xc2,0x77,0x8c, -0x41,0xba,0x07,0xe3,0x34,0x4b,0x52,0x50,0x99,0xfe,0xa8,0x7b, -0x3b,0x44,0x8f,0x00,0xee,0xb9,0x67,0x4f,0xd7,0x26,0x92,0xdd, -0xb5,0x1e,0x97,0xc8,0xf7,0x67,0xb8,0x0a,0xbc,0xff,0xf6,0xa2, -0xc8,0xaf,0x46,0xf6,0x9a,0x9f,0x48,0x42,0x5f,0xb6,0x8e,0xf4, -0xe7,0x02,0x58,0x92,0x2f,0xd2,0xc3,0x4f,0x90,0xf8,0x7d,0x48, -0x9f,0x0e,0x9e,0x6a,0x3f,0xe8,0x6e,0xf4,0xa9,0xa9,0x64,0xe0, -0xfd,0x43,0x47,0xd3,0xb3,0x4a,0x41,0x5f,0x61,0x52,0xa1,0xa8, -0x5a,0x13,0xc1,0xf9,0xf4,0x7c,0x2c,0x66,0x97,0x00,0x62,0xae, -0x81,0x7b,0x3c,0x4e,0x0c,0x5e,0x73,0x05,0x16,0x2f,0x40,0x37, -0xd5,0xe0,0x86,0x0f,0x29,0x15,0x5e,0x8e,0xca,0xfa,0x01,0x64, -0xb3,0x93,0xde,0xe6,0x6e,0x1a,0x26,0xca,0x9e,0xb0,0x78,0x9c, -0x3d,0x24,0x26,0x61,0x40,0x4d,0xbf,0x26,0x7a,0x0a,0x5b,0xda, -0x63,0x6a,0xd9,0x8a,0x3c,0xdd,0xde,0x78,0xb3,0x7d,0x09,0x27, -0x4d,0xf4,0x3b,0xc7,0x1f,0x73,0x4f,0x1d,0x30,0x5f,0x5f,0x4f, -0xc6,0x60,0x59,0x5c,0xed,0x18,0xbf,0x1e,0xff,0xaa,0x57,0x87, -0x6c,0x8d,0xf7,0x78,0xe1,0x32,0xbc,0x65,0xe0,0x5d,0x68,0xde, -0x43,0x77,0x7f,0x1a,0xd8,0x2d,0x02,0xb2,0xe5,0x18,0x43,0x8f, -0x4b,0xd3,0x49,0x62,0x44,0x14,0x45,0x49,0x7a,0x62,0x9d,0x5e, -0x10,0x12,0x0b,0x48,0xc8,0x3b,0xa9,0x57,0x8e,0x61,0x14,0xaf, -0x33,0x9c,0x27,0x4e,0x1a,0x89,0x1f,0xdf,0xae,0x5a,0x1f,0x08, -0x66,0x90,0x59,0x3b,0x6e,0x1a,0x57,0xdc,0x30,0xd3,0x75,0x9b, -0x3d,0x1c,0x4c,0x24,0xc7,0x87,0xb3,0x82,0x52,0x1b,0xa6,0x04, -0x17,0x71,0x3a,0x27,0x76,0x8f,0xd1,0x30,0x21,0x18,0xdc,0xec, -0x18,0x18,0xdf,0x3a,0xf5,0xfb,0x9f,0xb8,0x56,0x09,0x4e,0xa5, -0xc6,0x7a,0xf3,0x5b,0x3e,0x9d,0x16,0x12,0x3a,0x4c,0x0f,0x18, -0xd1,0x41,0x58,0xac,0x32,0xe7,0xf1,0x3a,0xab,0xcb,0xb7,0x0b, -0x5e,0x7f,0xc0,0xfe,0xd3,0xad,0xe4,0x65,0xd7,0x0a,0x83,0x6e, -0x4c,0xd8,0x63,0xe3,0x6e,0x41,0x39,0x3c,0x7a,0x21,0x77,0x2f, -0x3c,0xd4,0x68,0x1a,0xb8,0x2b,0xd9,0x95,0x4f,0x72,0x25,0x7c, -0x16,0x53,0xa9,0x31,0x0b,0xc6,0x99,0x92,0x1f,0x43,0x57,0x06, -0x98,0x57,0x2d,0x16,0x0d,0x95,0xf1,0x47,0x6d,0xfa,0x73,0xe4, -0x84,0x1c,0x1a,0x12,0x0a,0x11,0xd7,0x6c,0x2c,0x42,0x2b,0x8e, -0x73,0x7f,0xa5,0x85,0x28,0x37,0x35,0x32,0x0e,0x08,0x06,0xe2, -0xf2,0xf9,0x53,0xd8,0x97,0x4d,0xa6,0x59,0x25,0x65,0xb1,0x88, -0xe0,0xce,0x41,0x24,0x3a,0xed,0xbd,0x03,0x77,0x8b,0x07,0x1e, -0x64,0xc3,0xe6,0xe5,0x04,0x9c,0x0f,0xc4,0x15,0x3c,0x6c,0xf9, -0x47,0x1e,0xc2,0xf8,0x60,0x07,0x4e,0x5e,0xf1,0x5e,0x48,0x1c, -0xe9,0x2d,0xc4,0x06,0xdc,0x33,0xb5,0xd1,0x8a,0x36,0x8a,0x72, -0x7c,0x7f,0xb6,0x81,0xfb,0xf2,0x28,0xec,0x63,0x41,0x30,0x5a, -0x0b,0x78,0xdd,0x21,0x04,0x64,0x7f,0x05,0xaa,0x64,0xe8,0xe9, -0x06,0xd1,0x78,0x74,0x05,0x74,0x05,0x24,0x8d,0xd8,0x18,0x91, -0xae,0x95,0x1b,0x09,0x37,0x8d,0xfc,0xf7,0x48,0x09,0x4c,0x4f, -0xad,0x37,0xeb,0x34,0xaa,0x4d,0x8e,0xd4,0xd9,0x70,0x3c,0x9e, -0x8c,0x8b,0xb3,0xe1,0xd5,0x4b,0x5d,0xdb,0xc9,0x89,0x7c,0x8f, -0x54,0x3e,0x1b,0x5e,0x78,0x45,0x9f,0xeb,0x04,0x0c,0x0b,0x74, -0xfe,0x0a,0x22,0x58,0x46,0xb0,0xc9,0x89,0xc0,0x4f,0x47,0x4a, -0xb2,0x9a,0xfe,0xf3,0xaa,0x28,0xe9,0x0d,0x1e,0x22,0xe4,0xe3, -0x91,0xfe,0x13,0x11,0x67,0x8f,0xe0,0x6f,0x64,0xd8,0xba,0x19, -0xd7,0xa8,0x41,0x48,0x59,0xd0,0x43,0xe9,0x2e,0x65,0x21,0xcf, -0xb8,0xa7,0x4e,0x9d,0xf8,0x77,0x03,0xbb,0x1b,0xf8,0x6e,0x8c, -0x9b,0xad,0x2b,0x12,0x82,0xbf,0xee,0x7e,0xac,0xdb,0x4f,0x42, -0x2a,0x4c,0xa0,0x86,0xc8,0x3d,0x47,0xe0,0x8f,0x45,0x80,0x1e, -0xb0,0xc7,0x93,0xba,0x54,0xac,0xe9,0x2d,0xb6,0xb5,0xfd,0x42, -0xf7,0x1a,0x91,0x2d,0x74,0x1a,0xe1,0x4f,0x04,0xfe,0xf4,0xa0, -0xef,0xca,0x48,0xb2,0xcc,0x77,0x95,0xce,0x97,0x17,0xba,0xea, -0x45,0x9b,0xea,0x80,0xb1,0x5c,0xa6,0x09,0x42,0x39,0x64,0xfe, -0xe0,0xd3,0x24,0x60,0xfa,0xd1,0x04,0xbe,0xdf,0x55,0x7e,0x0e, -0x89,0xe8,0xa6,0xd5,0x18,0x62,0xb1,0xe3,0x2b,0xb3,0x2d,0x47, -0x00,0x63,0xde,0x04,0x2e,0x1f,0x0a,0x32,0x4c,0x41,0xdf,0xba, -0x95,0x6d,0x13,0xec,0xd6,0x8a,0xe1,0x55,0x2c,0xb7,0xb9,0x6e, -0x49,0xd6,0x89,0x7a,0x2e,0x52,0xf9,0x3e,0xbc,0x0d,0xd0,0x88, -0x8a,0x11,0x7a,0x19,0x4b,0x5b,0x51,0xff,0x84,0xec,0x04,0x04, -0x4a,0xf6,0x05,0xe1,0x7f,0xe1,0x05,0x7f,0xe1,0x8b,0xe9,0xe4, -0xba,0xe9,0x1b,0xd7,0xff,0x3a,0x69,0x86,0x88,0x37,0x41,0x92, -0xe0,0x5b,0x1d,0x92,0x86,0x0f,0x84,0xf3,0xa0,0xba,0x2f,0x4b, -0xdc,0xa7,0xc9,0xde,0x79,0x59,0xe9,0x1a,0xec,0x94,0x62,0x14, -0x8e,0x0a,0xbf,0x27,0x1b,0x47,0x02,0x04,0xa3,0x31,0x18,0x2a, -0x0d,0xf5,0xb2,0xa6,0x95,0xa8,0x7d,0xfc,0xba,0xc2,0x36,0xbd, -0xb3,0xb6,0x44,0x2c,0x3e,0xff,0x00,0x16,0xf9,0x72,0xe4,0xd7, -0xf0,0xd2,0x7e,0x4f,0x14,0x6d,0x66,0x19,0x25,0x6d,0x6d,0x05, -0xeb,0x09,0x43,0x72,0xfa,0xeb,0xf9,0xd5,0x95,0x49,0x92,0x4d, -0xa6,0x67,0x77,0x6d,0x9b,0xcd,0xab,0x1b,0x9d,0x75,0x1b,0xcd, -0x70,0x37,0x65,0x11,0x55,0xda,0x4e,0xd7,0xfa,0x57,0x37,0x9e, -0x57,0xab,0x58,0xfe,0xca,0xfc,0xa9,0x60,0xc9,0xf5,0x12,0x7d, -0x00,0xa4,0xe5,0xfa,0x4e,0x5a,0x9f,0x25,0x57,0xca,0x00,0x73, -0xfd,0x29,0xfb,0x46,0x03,0x1d,0xf5,0x93,0x4f,0x82,0x34,0x35, -0x10,0x9a,0xe2,0x4a,0x54,0x90,0xdc,0x6f,0x6c,0x17,0x09,0x3a, -0x00,0x0f,0x4d,0x73,0x2c,0x14,0x8e,0xb0,0xf8,0xcc,0x6b,0x00, -0x6f,0x59,0xab,0x1b,0xf3,0xfe,0x7c,0x36,0x68,0xe5,0x3a,0x4d, -0x5e,0xd4,0xc8,0x77,0xa0,0x13,0x3e,0x2a,0x19,0x8b,0x45,0xbd, -0x0e,0x04,0x15,0x1c,0x3e,0xf3,0x0f,0x30,0xba,0x76,0x31,0x49, -0x35,0xcc,0x3f,0x28,0x63,0x41,0x7d,0x69,0x58,0xe0,0x97,0xce, -0x1e,0x57,0xd4,0x6e,0x3e,0xf4,0x5c,0xcf,0x2b,0xde,0x12,0xc8, -0x92,0x2b,0x73,0xc5,0x27,0x9b,0xeb,0x21,0x56,0x93,0x82,0x97, -0x40,0x51,0xd3,0x66,0x44,0x7c,0xdf,0xdf,0x46,0x25,0x98,0x16, -0x83,0xa5,0x3e,0xb8,0xb9,0xa0,0xea,0x7f,0x61,0x85,0x4c,0xe7, -0x1f,0x50,0x38,0xba,0x0b,0x1a,0xb1,0xe1,0x0b,0x7d,0x76,0xb1, -0x3e,0xfd,0x2b,0xe9,0x35,0xbe,0xce,0x53,0x41,0x4f,0x53,0x05, -0xe0,0xc3,0x28,0xec,0xc4,0x17,0x77,0xd0,0xfa,0xe5,0x9f,0x29, -0x4a,0x04,0x3a,0x99,0xd0,0x33,0x68,0x43,0xba,0xe8,0xec,0xc4, -0xdb,0x00,0x56,0x5c,0x8b,0xae,0x43,0x65,0xa5,0xd6,0x46,0xfa, -0x26,0xe3,0xef,0xf2,0xf3,0xc9,0x14,0x8f,0x63,0x84,0xe0,0x54, -0xfc,0x9e,0xdb,0xc9,0xbf,0x02,0x24,0x18,0xa2,0xce,0x7b,0x93, -0x7e,0x91,0x1f,0xf1,0x66,0x45,0x2d,0x6e,0x49,0xd7,0x22,0x6b, -0xf8,0x21,0xab,0xc8,0x03,0xf8,0xf0,0x63,0x56,0x7e,0xc6,0x9f, -0xfa,0x80,0x3f,0x40,0xfa,0x1d,0x44,0x3b,0xec,0x20,0xda,0x1e, -0x7d,0xbe,0xc9,0x5f,0x92,0x7a,0x23,0xf3,0x86,0x82,0x59,0x56, -0x2f,0x27,0xf3,0xab,0xd1,0xe1,0xf0,0xb7,0x9c,0xc4,0xe1,0xac, -0x21,0xdd,0x1b,0x48,0xbc,0x83,0xdc,0x95,0x95,0x11,0xd8,0x15, -0xb8,0x7c,0x6c,0xd2,0xae,0x39,0x45,0x0b,0x67,0xf4,0x9a,0x88, -0x08,0xaf,0xde,0xdb,0x2d,0xab,0x55,0x04,0xb3,0x42,0x8e,0xaf, -0xbc,0xa3,0x0e,0xe7,0xbb,0xc9,0x04,0x2e,0x09,0x1a,0xe4,0x70, -0x83,0xf2,0x2e,0xf4,0x16,0x33,0x95,0x59,0x86,0x22,0xf7,0xfc, -0xfc,0xbc,0xf8,0x54,0xbf,0x6a,0x53,0x5e,0xd2,0x83,0x7b,0xaa, -0xa9,0x97,0x56,0x65,0x7a,0x87,0x84,0x47,0xea,0x43,0x29,0xa9, -0x22,0xe4,0x10,0xa0,0xcd,0xf7,0xc9,0xc1,0x4b,0xcf,0xe0,0xc7, -0x5f,0x68,0x41,0x64,0x9a,0xd4,0xf1,0xe8,0x4d,0x10,0x6f,0x93, -0x8e,0x1c,0x84,0xa3,0x00,0x4f,0x58,0xcb,0x5c,0x23,0x19,0x3d, -0x99,0x5e,0x40,0x35,0x53,0x78,0x60,0x6b,0xb9,0xd2,0xed,0x31, -0x29,0x60,0xa7,0xa8,0x85,0x62,0x47,0xed,0xe9,0xbd,0x7f,0xe8, -0x90,0xb9,0x27,0xa1,0x2c,0xee,0x34,0x96,0xc9,0xa6,0x22,0xe5, -0xd5,0x61,0x81,0xc9,0xae,0x1f,0x88,0x3b,0xba,0xe3,0x5b,0xfb, -0x00,0x34,0xe8,0x86,0xa0,0x8b,0x45,0xbc,0x7e,0xe9,0xac,0x0f, -0x01,0xf1,0x02,0x70,0x52,0xd9,0x08,0x17,0x93,0xfa,0xf3,0xbd, -0x39,0x6c,0xe5,0x59,0xf9,0x09,0x09,0xa8,0x5b,0x20,0xee,0xaf, -0x4c,0x94,0x0d,0x42,0x7c,0x00,0xd7,0x4d,0x88,0x89,0xcf,0xbd, -0x44,0x7b,0x41,0xd1,0x12,0xfe,0xc0,0x57,0xe7,0xee,0x10,0x64, -0x55,0xbf,0x32,0xf4,0xe6,0x2e,0x14,0x08,0x90,0x63,0x64,0x9b, -0x3f,0x89,0xb7,0x43,0x78,0xc3,0x86,0xac,0xdb,0xec,0x88,0xcf, -0x67,0xc7,0x64,0x82,0x55,0xb1,0x7f,0x9c,0x75,0x3a,0x08,0xcb, -0x0e,0x14,0x85,0xee,0x43,0xd2,0xbc,0xed,0x6a,0x88,0x63,0x6b, -0x8b,0x37,0x75,0x61,0x86,0xdb,0xda,0x09,0xf4,0xc9,0xad,0xc4, -0x2b,0x7b,0xc9,0xa3,0x5f,0x00,0x61,0xd0,0xaf,0x15,0x0c,0x38, -0x16,0xb0,0x77,0x35,0xc2,0xd2,0x97,0xde,0x48,0xb4,0xab,0x67, -0x32,0x12,0x42,0xee,0x0b,0x64,0x7a,0x45,0x00,0x61,0x84,0x6e, -0xf3,0x0c,0x7e,0x03,0x6a,0xe5,0x0d,0x0b,0x79,0x15,0x1c,0xf9, -0x54,0x94,0x6b,0x70,0xc0,0x0a,0x7d,0x07,0x07,0x44,0x15,0x0d, -0x9d,0x5e,0x4e,0xba,0x22,0xc3,0xf9,0x6d,0x05,0x9a,0xe8,0x20, -0x38,0x0b,0x7c,0xb6,0xfa,0x0e,0xdc,0x28,0x8d,0xc7,0x01,0x53, -0x52,0x8f,0x90,0xd4,0x65,0xe1,0x7a,0xc6,0x97,0xf1,0x5c,0x84, -0x35,0x06,0x03,0xa3,0x49,0xe3,0x05,0x20,0xe1,0x34,0x9a,0xdf, -0xa6,0x84,0xf5,0x19,0xc0,0xbf,0x7d,0x41,0x4f,0xcd,0x76,0xc6, -0xd7,0xd6,0x09,0xdf,0x8b,0x84,0x7a,0x9a,0xff,0x96,0x4f,0x51, -0x05,0xda,0x40,0xbe,0x3d,0xaf,0x0e,0x49,0xe7,0x33,0x6e,0xbd, -0xe3,0xb9,0xb2,0xea,0x6f,0xfd,0xcc,0x7a,0xfe,0x8f,0x97,0xc3, -0x19,0x6a,0xf3,0x94,0x38,0x1c,0x75,0x5e,0x6c,0x32,0x71,0xc6, -0xe4,0x4a,0xb9,0x90,0x1b,0x2b,0x8e,0x10,0xda,0x74,0x0a,0x92, -0x18,0x79,0x39,0x44,0x25,0x4f,0x0f,0x6a,0x70,0x71,0x76,0xd2, -0x05,0xe2,0xdc,0x82,0xcb,0x07,0xd4,0x8e,0xd8,0x1e,0xa3,0xef, -0x47,0x2f,0xcf,0xb7,0x5f,0x4f,0xc6,0xf9,0x36,0x7a,0x58,0x89, -0x06,0x19,0xea,0x28,0x65,0xae,0xcc,0xa0,0xe5,0x00,0x8c,0x3a, -0xb0,0x6e,0x38,0xeb,0xb4,0x3d,0x67,0xda,0x5e,0xe7,0xb6,0x32, -0x68,0xa0,0x5e,0xb4,0x13,0xb5,0xe0,0x01,0x42,0x9f,0x63,0xaa, -0x24,0x98,0x56,0x56,0xf4,0x20,0x59,0x67,0x69,0x20,0x40,0x85, -0xb9,0xc1,0xad,0x51,0x88,0x4a,0x4c,0xe1,0xb1,0x5f,0xec,0xbb, -0x69,0xf6,0x77,0x14,0xc1,0x7e,0x43,0x73,0x11,0xd0,0xb6,0xfe, -0x85,0xe8,0x40,0x41,0xbb,0x1e,0xba,0xb7,0xef,0xd7,0x41,0x37, -0x59,0xfb,0x6b,0xd1,0x55,0xf5,0x9d,0x08,0x59,0x55,0xbc,0x1e, -0x4a,0xf6,0x55,0x7d,0x27,0x52,0xd4,0x37,0xaf,0x87,0xf2,0x87, -0xe7,0x4f,0x9e,0xdd,0x8d,0x90,0x94,0xd6,0xeb,0x61,0x3c,0x7a, -0xf7,0xe4,0xe9,0x1a,0x34,0x1a,0xcd,0xf7,0x7a,0x48,0x39,0x4c, -0xf2,0x1a,0x1d,0x84,0xea,0xf3,0x35,0x7b,0x08,0x82,0xa2,0xdc, -0x89,0x72,0xc9,0x06,0x13,0xe8,0x10,0xd4,0xe3,0xe9,0x90,0x22, -0x9c,0x70,0xc3,0x99,0x8a,0x60,0x6b,0xca,0xcf,0xc6,0xd9,0x59, -0xc3,0x92,0x01,0xf5,0xc9,0x09,0x10,0x8c,0xf3,0xd1,0xe9,0x60, -0xa7,0xf9,0x68,0x6e,0x66,0x9e,0x64,0xeb,0x8e,0x37,0xbb,0x5b, -0x70,0x67,0xf3,0x6b,0xa1,0xfd,0x85,0x27,0xb0,0x52,0x80,0xdf, -0x43,0x56,0xbc,0x65,0x54,0x47,0xbb,0x56,0xcb,0xc7,0xea,0xa2, -0xe2,0x3c,0x96,0x45,0xf0,0x50,0xc9,0x2f,0xe6,0x59,0x83,0xf8, -0xc0,0x89,0xd8,0xc2,0x99,0xc4,0x2e,0x53,0x92,0xc9,0x44,0x1b, -0xab,0x55,0x66,0xf6,0xfd,0x7a,0xaa,0x2b,0x0b,0xcb,0x3b,0xeb, -0xc1,0xd6,0x1d,0x06,0x0d,0x5c,0xbc,0xa3,0xb5,0x90,0x68,0xc0, -0x41,0x57,0xff,0x01,0xc7,0xfd,0x44,0x68,0x0b,0xdc,0xc1,0xb6, -0x74,0x12,0x9b,0x94,0x29,0xb8,0x89,0x61,0x5d,0x6f,0x91,0xf6, -0xb6,0x4b,0x3f,0x41,0x6b,0x07,0xea,0xc5,0x5e,0xa5,0x2f,0xea, -0x34,0x8b,0x2c,0x11,0x57,0x60,0x43,0x88,0x8a,0x3a,0xda,0xe9, -0x96,0x7d,0xca,0x83,0xbe,0xb4,0xd6,0x0d,0xbd,0x9a,0x00,0x5b, -0x8c,0x22,0xe5,0x5f,0xeb,0x56,0x14,0x46,0x1a,0xb8,0xb6,0x6d, -0x31,0x66,0x7f,0xd5,0x33,0xc6,0x3b,0xcb,0x70,0xec,0x9f,0xe7, -0x7a,0x56,0x7a,0x16,0x3b,0x66,0x5d,0x87,0x05,0x89,0x2f,0x71, -0xb0,0xa5,0x4e,0xf3,0x52,0x06,0xb0,0xfa,0xa3,0x18,0x90,0x3e, -0x0a,0xcb,0x2b,0x4e,0x6a,0x96,0xcf,0xd0,0xdf,0x5c,0x06,0x54, -0xd1,0x64,0xbe,0xb8,0x9a,0x9c,0x0e,0xaf,0x48,0xf6,0xe2,0xb8, -0x21,0x35,0x56,0x79,0x38,0xf9,0x54,0x0d,0x6c,0xe2,0xfb,0xce, -0x35,0x2e,0x70,0x6b,0xc2,0xae,0xc9,0x78,0x6b,0x64,0x81,0x7b, -0x98,0x4f,0x7f,0x2b,0xce,0xf2,0x17,0x93,0xa9,0x87,0xd2,0x0a, -0x7e,0x16,0x7c,0xea,0x89,0xd0,0xc0,0x59,0xac,0xcf,0x6e,0x90, -0xa2,0x9d,0x23,0xe4,0xd5,0x2a,0x66,0x9a,0x39,0xe4,0xfc,0x4e, -0xad,0x06,0xad,0x1c,0x2d,0x0e,0xb4,0xbc,0x83,0xb7,0xc0,0x74, -0x83,0xfe,0xc0,0x77,0x08,0x4d,0x5f,0x2a,0x97,0x46,0xbc,0xe7, -0x0e,0xe4,0x75,0xc7,0x8e,0x16,0x7f,0x5d,0xa6,0xf5,0xe3,0xb3, -0x0e,0x76,0x3c,0x99,0xb4,0xe8,0xe0,0x4c,0x2b,0xd8,0x81,0xf0, -0xee,0x8d,0xda,0xce,0x61,0xd4,0xdb,0x95,0xdc,0x94,0x02,0x87, -0x56,0x61,0x36,0x04,0x4d,0x32,0x34,0xf2,0x4e,0x57,0x66,0x97, -0x20,0xd1,0x0e,0x2d,0x08,0xbf,0xd6,0x55,0x42,0x16,0x6f,0xa6, -0x1e,0xc3,0x9e,0x83,0xaa,0x04,0x10,0xbd,0x49,0x20,0xea,0xfc, -0xad,0xad,0x06,0xad,0x0d,0xeb,0x82,0xac,0xe0,0xab,0x93,0xdc, -0x49,0x82,0xcf,0x37,0x28,0xd7,0x60,0x84,0x4a,0xeb,0x97,0x05, -0x03,0x57,0x69,0x6e,0x3d,0x78,0x59,0x55,0x74,0xd2,0x9e,0x8f, -0x61,0xf4,0x9e,0x90,0xa9,0x78,0xa9,0x1f,0x6e,0x8a,0xb3,0x0f, -0xcd,0xc7,0xb7,0x7d,0x73,0xd2,0x1f,0x1e,0xed,0x0f,0x92,0xa4, -0xc6,0xa2,0x2e,0x09,0x2c,0x02,0x32,0xc6,0xcf,0x33,0xa1,0x99, -0xb2,0xa4,0x5b,0xe1,0x8b,0xa6,0x51,0x12,0xd7,0x86,0xee,0x54, -0x32,0xf6,0xeb,0x51,0x16,0xb8,0xca,0x5f,0x6d,0xb3,0x79,0x88, -0xec,0x59,0x5d,0x3d,0x26,0x74,0xad,0xba,0x7a,0x0c,0x07,0xbb, -0xa7,0xcc,0x7f,0xa7,0x75,0xf3,0xcb,0x30,0x48,0x97,0xf2,0xa8, -0x5a,0xb1,0xcc,0x05,0x82,0x4b,0x9b,0x36,0x44,0xce,0x00,0xc7, -0xb0,0x66,0x1d,0x13,0xe6,0xc2,0x35,0x0d,0x65,0xcf,0x5c,0x7d, -0x06,0x25,0x3e,0x5d,0x32,0x28,0x57,0x25,0xf8,0x5f,0x95,0x55, -0x4a,0xf7,0xae,0x02,0x8f,0x5b,0xb2,0x0d,0x74,0x7f,0xa0,0x8c, -0x02,0x47,0xa0,0xe3,0xb9,0x16,0xa0,0x24,0x3f,0x64,0x56,0x5f, -0xe4,0x6e,0x05,0x8a,0xd8,0xe7,0x2b,0xd7,0xfc,0x0c,0x85,0x2c, -0xb9,0x47,0xbf,0xb9,0xd2,0xec,0xba,0x0d,0xbb,0x3d,0x17,0xdb, -0x4e,0xed,0x85,0x9f,0x5f,0x25,0xfb,0x2c,0x54,0x72,0x78,0xe4, -0x13,0x13,0xf0,0xfb,0x60,0x6e,0x62,0xb3,0x34,0x76,0x82,0x1b, -0x64,0xee,0xf8,0xf6,0xcb,0xbb,0xe1,0xdf,0x52,0x71,0xb5,0x01, -0x5c,0xf5,0xfe,0xa7,0xfb,0xe5,0xde,0x52,0xa8,0xa2,0xd8,0x84, -0x64,0xf2,0xd4,0x86,0x66,0x88,0x70,0x26,0xf6,0x82,0xe1,0x90, -0x9c,0x06,0x71,0x9e,0x4c,0xf8,0xb1,0xc4,0x3c,0x00,0x27,0x0b, -0xe0,0xbe,0xcf,0x67,0x15,0xb0,0x15,0xab,0x66,0xad,0x0c,0xe9, -0xa9,0x2f,0x3d,0x4a,0xe9,0xee,0x06,0x56,0x0d,0x0a,0x94,0x15, -0x34,0x6a,0xc6,0xa7,0x01,0xa2,0xc4,0x12,0x21,0xdc,0x94,0x38, -0x68,0x6b,0x27,0x89,0xf1,0xd6,0xf2,0xf3,0x5c,0x8b,0x42,0xa3, -0xec,0xde,0xaf,0x6d,0x7c,0x06,0x4f,0x91,0x1a,0xe8,0xc9,0xf4, -0xa2,0x74,0x6a,0xe6,0xae,0x54,0x19,0xf9,0xde,0x8d,0xc1,0x33, -0xbc,0x54,0x88,0x54,0xc6,0xb8,0x1b,0xe2,0x06,0xaf,0xea,0xd0, -0x06,0x03,0xce,0xc2,0x8d,0x41,0x26,0x3a,0x0c,0x6a,0x93,0xb7, -0x26,0x08,0x1f,0x64,0x5c,0xef,0x1a,0xfa,0x8c,0xbf,0x6e,0xac, -0x6f,0x19,0x0e,0x10,0xd3,0xd1,0x16,0xda,0x34,0x08,0x10,0xfc, -0xa5,0x3d,0x91,0x54,0x5a,0x4d,0xf7,0x6f,0xec,0xa4,0xc9,0x9a, -0x26,0xbb,0xab,0x7d,0xea,0xbe,0xa2,0x0f,0x18,0x07,0xff,0x7d, -0xdf,0xc2,0x55,0x87,0xe8,0x6d,0xdc,0x33,0xe5,0x9d,0xc2,0xd7, -0x28,0x2b,0x7b,0x95,0xaf,0xb0,0x16,0x55,0x16,0xc7,0xad,0xbc, -0xa3,0x90,0xc6,0x95,0x12,0x14,0xd2,0x11,0xae,0x05,0x6e,0x78, -0x0c,0x77,0x36,0x2d,0x6e,0x84,0x54,0x15,0x07,0x6a,0xff,0x09, -0xb8,0xb4,0xc4,0x39,0x2f,0x84,0x01,0x4c,0x6c,0x14,0x06,0xc0, -0x00,0x30,0x91,0x7e,0xa8,0xb5,0x1c,0xf8,0x74,0x5e,0xce,0x26, -0xd7,0x6f,0x8c,0x6e,0x8c,0x6b,0x41,0x09,0xe9,0xcc,0xcf,0xb2, -0xd2,0x1e,0x26,0x07,0xe2,0x22,0xeb,0xbb,0xfb,0x64,0x9d,0xc7, -0x86,0x79,0x7c,0xe5,0x82,0xad,0x52,0x07,0xca,0xbb,0x54,0x91, -0xdc,0xda,0x32,0xa3,0x89,0x86,0x22,0xb4,0x12,0x4a,0xaf,0xff, -0xc3,0xd2,0xd8,0x78,0x5c,0x5d,0xd9,0x1a,0xe1,0xc5,0x51,0x65, -0xd0,0x63,0x80,0x97,0x7c,0xda,0x33,0x66,0x80,0x29,0xbc,0xa1, -0x15,0x66,0x86,0x88,0x5a,0xf0,0xde,0x9e,0x4d,0xde,0xdf,0xdc, -0x18,0xd3,0xd0,0xae,0x44,0x9b,0x21,0xe9,0x9b,0x59,0xe6,0xa1, -0xdf,0xda,0xa2,0x6f,0x09,0xd3,0x7b,0x7e,0x2b,0xa4,0xd5,0xfb, -0x22,0xc4,0x8f,0xf1,0x80,0xde,0xd7,0x75,0x88,0x3e,0xac,0x6b, -0x4f,0x0c,0x65,0x1b,0x07,0x47,0xfc,0x3e,0x06,0x63,0xd6,0x83, -0xd2,0x09,0x79,0x71,0x32,0xb8,0x64,0x3b,0x21,0x01,0x1e,0xd5, -0x09,0x46,0x73,0x12,0x9d,0xfb,0xfd,0xf3,0xa3,0x57,0x2f,0x0f, -0x8f,0x4c,0x71,0xdc,0x6c,0xd4,0x6c,0x04,0xda,0xa3,0x89,0x81, -0xac,0x16,0x76,0xc3,0xa9,0x79,0x1f,0x10,0x9f,0xb3,0x90,0x6b, -0x95,0x30,0x1a,0xb6,0xb0,0xd9,0xbe,0x75,0x9c,0x84,0xed,0xd6, -0x0a,0x2d,0xcf,0xc9,0xfa,0x49,0x60,0x58,0x5b,0x48,0x53,0x5e, -0x31,0xb5,0x26,0x12,0xda,0xd5,0x35,0x2e,0xaa,0xa1,0x14,0xe0, -0x96,0xfa,0x33,0xb1,0xb8,0xda,0x15,0xff,0xee,0x63,0x7f,0x6b, -0x0b,0xb3,0xd9,0x61,0x89,0x82,0xdc,0xf3,0xe2,0x09,0xd1,0x5a, -0x3b,0xd5,0xd0,0x90,0xd8,0xd7,0x89,0x18,0xef,0xfd,0xcf,0x57, -0xef,0xf2,0xa4,0xb9,0x54,0xfd,0xde,0xc6,0x62,0xc1,0xed,0x84, -0x58,0x9e,0xd7,0x40,0xef,0x19,0xbd,0x0e,0x6a,0x1c,0x63,0x39, -0x84,0x4b,0xf1,0xfd,0x5f,0x2c,0x45,0xe8,0x36,0x5c,0x8f,0x1a, -0x3b,0x8d,0xc0,0xd2,0xce,0xcc,0x02,0xf7,0x85,0x6b,0x62,0xa1, -0xaf,0xa9,0x9d,0x51,0x5f,0x86,0x4b,0xf3,0x23,0x8b,0x47,0x3f, -0xff,0x51,0x2c,0x13,0x41,0x0e,0xbc,0xfc,0x41,0x3c,0xc4,0x56, -0x2d,0x26,0xe2,0xba,0x7f,0x10,0x17,0xf0,0x31,0x8b,0x09,0x5e, -0xfe,0x20,0x1e,0x54,0x94,0x5b,0x44,0xf8,0xf6,0x07,0x31,0xb1, -0x7e,0xdc,0xe2,0xe2,0xf7,0x3f,0xda,0xe6,0xa0,0x16,0x77,0x8d, -0x3e,0x6c,0x18,0x03,0x35,0xcb,0xaf,0xc8,0xad,0x3b,0xdf,0x76, -0xf9,0x96,0x33,0xa8,0x92,0x34,0x5c,0x8d,0xcc,0xa9,0x7e,0xa7, -0x10,0x6c,0xe2,0xbe,0x98,0x2f,0xed,0xfe,0x49,0xbe,0xf4,0x27, -0xa6,0xf3,0xee,0x5f,0x34,0xec,0x15,0xf9,0xc1,0xff,0x1f,0x39, -0x5e,0xdd,0xbd,0x03,0xc9,0x22,0x38,0xe9,0x7f,0xd2,0x1c,0xf8, -0x72,0xce,0xfc,0xdd,0xe7,0x97,0xa3,0x7f,0x0f,0x57,0xe6,0x01, -0xf1,0xe7,0xa6,0xe5,0x6e,0xe3,0xb4,0x64,0xda,0xe3,0x02,0x36, -0x18,0xbf,0xbe,0xad,0x17,0x0a,0x49,0xc1,0x6f,0xe4,0x2b,0x0d, -0xab,0x45,0x56,0xf6,0x45,0x9d,0xd4,0x14,0x13,0xfb,0xb2,0x4a, -0xf7,0xeb,0xc2,0x9f,0x2a,0xb2,0x67,0x70,0x00,0x86,0x53,0xf3, -0x68,0xf2,0x76,0x3e,0xc3,0xd7,0xbe,0x2e,0x32,0x50,0x2b,0x76, -0x73,0x16,0x1c,0x46,0x60,0x1c,0x20,0x0f,0xb6,0x79,0xd0,0x02, -0xf9,0x14,0x5b,0xcd,0x44,0x6d,0x20,0x4b,0x89,0x15,0x72,0x22, -0x91,0x96,0x74,0x0d,0x2c,0x12,0x94,0x39,0x4c,0xd5,0xcd,0xaf, -0x81,0x4c,0x6a,0x43,0x64,0x0b,0xf0,0x30,0xec,0xf5,0xba,0x7b, -0x4c,0x4f,0x99,0x82,0x2e,0xf3,0xcc,0xee,0x0f,0xb0,0x3d,0x83, -0xf0,0xe0,0x4e,0x5a,0x67,0x83,0x26,0xa3,0xb4,0x77,0x1b,0x49, -0xc6,0x2a,0x22,0xba,0x37,0xd9,0x65,0xff,0x11,0xac,0x68,0x8e, -0xe1,0x62,0xc4,0x73,0x87,0xc4,0xd1,0xf3,0xa3,0xe1,0x85,0xbb, -0x54,0x06,0xd5,0x6e,0x6d,0xb1,0x71,0x04,0xbc,0x34,0xcf,0x18, -0x00,0x62,0xab,0x07,0xdc,0x04,0xbb,0x16,0xf0,0xe6,0x0e,0xaa, -0x00,0xea,0x46,0x73,0x38,0xcc,0xea,0x74,0x2a,0x13,0xb7,0x87, -0xb3,0x37,0xbc,0x38,0x3c,0x6c,0x78,0xfa,0x41,0x47,0x43,0x74, -0xfa,0x04,0x7f,0x5f,0x40,0xba,0xc6,0x06,0xcf,0x8b,0x05,0x55, -0x74,0x97,0xf4,0xae,0xf8,0x26,0xeb,0x6a,0x7b,0x74,0x8a,0xdc, -0xe8,0x9a,0x5e,0xd4,0x26,0x2d,0xb9,0xd1,0x04,0xc5,0x7e,0x37, -0x46,0xfd,0x17,0x26,0x20,0x86,0xfc,0xa9,0x34,0xe8,0x60,0x4b, -0x0e,0xa6,0xc2,0xf4,0x91,0x7d,0xbf,0x11,0xc6,0x35,0xab,0x3f, -0x07,0xfb,0x46,0xd8,0xde,0x18,0xf6,0x16,0x37,0x47,0xf6,0x62, -0xef,0x14,0x30,0xac,0xdc,0x28,0xc1,0xc8,0xf6,0x38,0x14,0xef, -0x1a,0xd5,0x61,0x23,0xd4,0x0c,0x40,0x3d,0x5b,0x20,0x5c,0x45, -0x3e,0xc2,0x3a,0x4e,0xda,0xd7,0xc3,0x9b,0x20,0xec,0xbe,0xaf, -0xbf,0xb8,0xab,0xcb,0x6a,0xd7,0xfe,0x5e,0xcd,0xb6,0xeb,0x2e, -0x4c,0xe1,0xfe,0x8f,0x54,0x26,0xeb,0x8d,0x18,0x54,0xcd,0xa4, -0x8d,0x95,0x12,0x2e,0xec,0x31,0x84,0xd4,0x63,0x20,0x6c,0x06, -0x56,0x06,0x61,0x4b,0x78,0x79,0xc9,0xdd,0x34,0x34,0x36,0xc1, -0x6a,0x9d,0x55,0xa3,0xf4,0x47,0x31,0x49,0x25,0x11,0x6b,0xb6, -0x43,0x82,0x6d,0xf0,0x07,0x6b,0xe5,0xa6,0xf2,0xeb,0xc5,0x99, -0x5c,0xcf,0xb1,0xcf,0xea,0x8d,0xfb,0x1d,0xc4,0x66,0xa6,0x77, -0x74,0xe0,0x9f,0xe4,0xaf,0xe5,0xec,0xd0,0x97,0x2f,0xaa,0x5a, -0x1c,0xec,0x5d,0x9e,0x9b,0xe0,0x31,0xe1,0x8b,0x38,0x9c,0xa6, -0xe3,0x6d,0x25,0xd2,0x39,0x45,0xe1,0x22,0xce,0xb5,0xd6,0x28, -0xb4,0xc7,0xf9,0x77,0x30,0x46,0x54,0xf5,0x9c,0xfe,0x02,0x37, -0x75,0x08,0x33,0xad,0x0a,0x9c,0x7c,0x07,0x3f,0xe9,0x82,0xc5, -0xe7,0x1b,0xb4,0xdd,0x8c,0xb9,0x04,0xda,0xac,0x33,0xce,0x8a, -0x36,0xd2,0xc0,0x90,0x51,0xdc,0x7a,0xac,0x95,0xcb,0x34,0x2f, -0x6c,0x3e,0x73,0x2d,0x05,0x73,0x55,0x93,0x0f,0xa0,0xa7,0x07, -0x47,0x67,0xd9,0x17,0xf1,0x37,0xdc,0xd8,0xfc,0xe9,0x55,0x1b, -0x0d,0x4d,0x7a,0x56,0x15,0x68,0x4b,0xae,0xb1,0xf6,0xfc,0x25, -0xb3,0xf8,0xdf,0xcd,0xe9,0xbe,0x7c,0x86,0xd7,0xb2,0xc1,0x75, -0xd0,0x18,0x45,0x37,0xc5,0x45,0x5c,0xd5,0xa9,0x7f,0x9e,0x13, -0x04,0xaa,0x62,0xd7,0xe1,0xc2,0x98,0xd4,0xf3,0xa9,0x14,0x2c, -0xce,0x10,0xf9,0xd2,0xb8,0x2a,0x70,0x68,0xec,0x09,0x75,0xe8, -0x7f,0xc1,0x2d,0xc9,0x46,0x16,0xa8,0x29,0x6e,0x26,0x95,0x13, -0x7e,0x48,0x59,0x5e,0x4b,0x92,0xb4,0xc0,0xbd,0x8d,0xfe,0xb9, -0xfd,0xc3,0xd1,0xd1,0xdb,0x6d,0xaa,0x6d,0xdb,0x60,0x8e,0xec, -0xcd,0xf7,0x4c,0x8c,0xcc,0x9e,0x31,0x5f,0x4c,0x6d,0xd2,0x32, -0x11,0xa7,0xeb,0xa1,0x49,0x6c,0x0d,0xa5,0xbd,0x3b,0x24,0x19, -0x49,0x5c,0x9d,0x14,0x83,0x26,0xc2,0xbe,0xfe,0xb9,0xef,0xbd, -0x0e,0x74,0x93,0xb3,0xcc,0xe2,0xa6,0xb7,0x3f,0x2e,0x92,0xf4, -0xdf,0x4f,0xc4,0xff,0x0f,0x48,0x30,0xc5,0x58,0xbd,0x72,0x17, -0x45,0x6b,0x2e,0x70,0xfe,0xde,0xf3,0x85,0xe7,0xc9,0xa6,0xf1, -0x38,0x42,0xae,0x83,0x7c,0x18,0x41,0xe7,0x38,0x34,0x71,0xcd, -0x5a,0xc8,0x6f,0xd5,0x7d,0xa8,0xaf,0x18,0xfc,0x03,0x95,0x19, -0x37,0x0e,0x6b,0xd6,0x27,0x34,0xa3,0x7f,0xa0,0x32,0x3c,0xae, -0x5a,0xb7,0x26,0xa1,0x45,0xa2,0x8d,0x0f,0x1f,0xf1,0x7c,0x71, -0xa5,0x38,0xe1,0xa5,0x64,0x81,0x88,0xaa,0x35,0x4a,0x55,0xd3, -0x1f,0xf8,0x38,0x72,0x81,0xb1,0xe6,0xd7,0x79,0xda,0xa8,0x3f, -0x50,0x19,0xbb,0xd8,0x58,0xb3,0xb6,0x40,0xb7,0xf4,0x07,0xea, -0xb3,0x2e,0x3c,0xd6,0xed,0x3d,0xa9,0x81,0x8a,0xff,0x68,0xcf, -0x91,0x9f,0x10,0xbf,0xca,0x86,0xce,0xf3,0xcf,0xf5,0xd6,0x39, -0xd1,0xbb,0x9b,0x80,0x7a,0x24,0x0d,0x04,0xac,0xb2,0xf2,0xa3, -0x20,0x65,0x01,0x3a,0xbe,0x46,0xc1,0x7b,0xf9,0xc0,0xff,0x11, -0xbf,0x3a,0x75,0x0e,0xd0,0x98,0x5b,0x84,0x5d,0xff,0xd5,0x6d, -0xe9,0xbd,0x45,0xa9,0x56,0xff,0x46,0xfb,0x03,0x4d,0x00,0x38, -0xbf,0x6a,0x3a,0x12,0x0d,0x68,0x25,0xb1,0x93,0x4c,0x3c,0x5d, -0xa5,0xe7,0xf5,0xc3,0x49,0x18,0x6b,0xbb,0x0b,0x29,0xf2,0x6e, -0xc9,0xed,0x4a,0xa3,0x74,0x2c,0x4e,0x77,0xe5,0x6c,0x84,0x62, -0xbf,0xa9,0x4c,0x81,0xa0,0xc9,0xb0,0x9c,0x68,0x37,0xa7,0x23, -0xf0,0x1b,0x4b,0x5c,0x93,0xb1,0xfe,0x1c,0x5d,0xca,0x65,0xe5, -0x06,0x4e,0x9b,0xce,0x7f,0xbb,0x28,0xd9,0xd1,0xf9,0x6f,0xe3, -0xba,0x1e,0x34,0x91,0x1f,0x8d,0xb6,0x69,0x06,0x04,0x85,0x6a, -0x9b,0x53,0xc4,0xdb,0x05,0x9b,0x7f,0xcf,0xba,0xf6,0x85,0x7f, -0xbd,0xca,0x79,0xc8,0x1c,0x63,0xfc,0x4d,0xb8,0x5d,0xe5,0x9d, -0xf8,0xd6,0xd9,0xfc,0x9a,0xa6,0x0a,0x8d,0x84,0x2d,0x0a,0x65, -0x9f,0x12,0xe1,0x89,0x33,0x96,0xa9,0x2b,0x2d,0x80,0x1d,0x20, -0x39,0x8b,0x24,0xb0,0x0c,0xdd,0x20,0x9b,0xf5,0x1e,0x6f,0x19, -0x34,0x59,0x39,0xf7,0x2b,0x19,0x83,0x1a,0x1d,0x90,0xbd,0x74, -0x66,0x4c,0x8e,0x55,0xf8,0x45,0x5c,0xb3,0x71,0x42,0x6a,0x34, -0xc7,0xd8,0x34,0x6b,0x98,0xd8,0x1a,0x45,0xb3,0x2d,0xe7,0xba, -0xc3,0x94,0x6e,0xec,0xa0,0x6a,0xe1,0xc0,0x40,0xd7,0xd0,0x46, -0x9a,0x71,0x03,0x14,0x98,0xe9,0xd6,0x03,0x35,0x18,0xeb,0xd6, -0x03,0xd7,0x9a,0xec,0x36,0x56,0x5e,0x35,0xdc,0xad,0x07,0xad, -0x6e,0x59,0xc8,0x88,0x16,0x4b,0x56,0x33,0xab,0xad,0xd1,0xa4, -0xd5,0xa8,0xc7,0x21,0x8c,0xd5,0x42,0x4c,0x4b,0x6f,0x66,0x35, -0x0d,0xc9,0xba,0x79,0xb0,0x1c,0x80,0x27,0xdd,0x8a,0xd3,0x5c, -0x88,0xd5,0x97,0xb1,0x66,0x25,0xff,0xad,0x98,0xcc,0xf1,0xba, -0xe6,0xb4,0x3c,0x83,0x80,0x11,0x90,0xd9,0x3e,0x51,0xa7,0xba, -0xb2,0x0f,0xe8,0x00,0x57,0xa1,0x8a,0xfc,0x2d,0x58,0xa0,0x85, -0xd6,0x68,0x4a,0xef,0xc8,0x29,0x83,0x76,0xff,0x22,0x07,0x46, -0x09,0x65,0x99,0xf1,0x22,0x32,0xd1,0xee,0xce,0xa1,0x45,0x3b, -0x3c,0x85,0x96,0x6d,0x32,0x15,0x13,0x60,0x64,0x9f,0x0d,0x67, -0x32,0x9d,0x52,0x94,0x39,0xdb,0xc8,0x0c,0x11,0xf6,0xb4,0x43, -0x5d,0x0e,0xcb,0x37,0x1f,0xc7,0x6f,0xc9,0x2b,0xd2,0x67,0x07, -0xe0,0xa7,0xab,0xf1,0x10,0xa2,0xb1,0xbe,0xe0,0x68,0xb6,0xa2, -0x06,0x13,0xd0,0x96,0x00,0x0e,0x86,0x37,0x32,0x13,0x7c,0x88, -0x53,0xc6,0x3b,0xbc,0xf7,0x23,0xf3,0xe8,0x26,0x90,0x97,0xfd, -0x0e,0x5c,0x91,0x57,0x61,0x30,0xd9,0x50,0x80,0x0e,0xb8,0x3d, -0x02,0x30,0x85,0xb3,0x9f,0xeb,0xc1,0xff,0x59,0xe6,0xc2,0x6c, -0x30,0xd4,0x1f,0x4e,0xae,0xfd,0x46,0xd3,0xef,0x9c,0xf5,0x92, -0x62,0x17,0xc9,0x5c,0x0e,0x67,0xc4,0x00,0x10,0x82,0xb8,0x06, -0xe8,0xca,0x25,0x1b,0x4c,0x7c,0x4d,0x95,0xba,0x9f,0x5d,0x1f, -0x70,0xde,0xff,0xce,0x3f,0x97,0xa6,0xfb,0xc1,0x51,0x31,0x27, -0x7f,0xa7,0xab,0xca,0xec,0x20,0xc0,0x11,0xaf,0x4e,0xea,0xfd, -0x8a,0xea,0xe7,0x8d,0x62,0x0c,0x93,0xe1,0x2c,0x9f,0x9c,0x6f, -0x9c,0xf4,0x74,0x02,0xde,0x0b,0xf3,0x53,0xe9,0x7e,0xd5,0x09, -0xf8,0x69,0xbf,0xc9,0x47,0xa0,0xa9,0x32,0xfa,0x00,0xf0,0xf7, -0xba,0x71,0x82,0x38,0x97,0xdd,0xc8,0x86,0x6e,0x8d,0x36,0x33, -0x18,0x6d,0xba,0x70,0xfe,0xe9,0x66,0x32,0x9d,0x95,0xbd,0xb8, -0x2e,0x93,0x5c,0x43,0x6f,0x6d,0xb1,0x8b,0x68,0x86,0x85,0x2b, -0x39,0xf4,0x94,0xf9,0x19,0xd9,0x49,0xa2,0xf8,0xb1,0x7d,0xa2, -0x5f,0x52,0x9a,0x30,0xd9,0x89,0x3a,0x69,0xff,0xe3,0xf9,0xbb, -0xc3,0x97,0x6f,0x5e,0x67,0x51,0xa7,0xfd,0x55,0x7b,0x2f,0x42, -0x39,0x03,0xec,0xd4,0x32,0x32,0x57,0xd3,0x3f,0x26,0x74,0xb2, -0x6c,0x08,0x88,0xdc,0x38,0x1d,0xb2,0x2f,0x1c,0x0a,0x91,0x56, -0x9c,0xa3,0xc6,0x75,0x13,0xbe,0x32,0x81,0x17,0x39,0x52,0xb7, -0xb6,0x74,0xaa,0xc5,0x94,0x65,0x5e,0x66,0x22,0xf2,0xe2,0x0a, -0x62,0x1b,0xdd,0x0c,0xa0,0x38,0x22,0x5d,0x96,0xb5,0xdc,0x9b, -0xbb,0x78,0x5c,0x64,0xbb,0x8a,0x21,0x5c,0x76,0x97,0x7e,0xf6, -0x0b,0x8c,0x20,0xa9,0xf1,0x98,0x1a,0xd8,0x22,0x95,0xaa,0x51, -0x18,0x5a,0x72,0xa0,0x30,0x34,0x25,0xf8,0x56,0x64,0x66,0xc2, -0x17,0xcf,0xc8,0x3c,0xde,0xd4,0x03,0x63,0x26,0x63,0x1f,0xd7, -0x00,0xae,0x44,0xc5,0x90,0xf8,0xe5,0x35,0x43,0x29,0x88,0x69, -0xa9,0xf8,0xa1,0x81,0x8a,0x2e,0x1e,0x34,0x64,0xe0,0x89,0x03, -0x59,0xef,0x5d,0x7d,0xc2,0x7a,0x42,0xf4,0x8f,0xeb,0x8c,0xd5, -0xa1,0x9b,0x32,0x68,0xa1,0x1e,0xe7,0xa5,0x96,0x69,0x50,0x3f, -0x41,0x1d,0xa6,0x8f,0x74,0x62,0x8f,0xd3,0xaa,0x7d,0x93,0xc6, -0x68,0xd2,0x08,0x95,0x4b,0x6b,0xd7,0x79,0x4e,0xf1,0xc4,0xd4, -0x55,0x41,0x2e,0x54,0xb0,0x16,0xba,0xfb,0x5d,0xdf,0x06,0x95, -0x42,0xac,0xd6,0x86,0x72,0x7c,0x43,0x92,0x78,0x10,0x7a,0xce, -0xc9,0x22,0xe2,0x53,0x10,0xef,0x2b,0x07,0x67,0xde,0x1b,0x43, -0x98,0xdd,0x28,0x98,0x6d,0x8c,0x27,0x1b,0xec,0x45,0x8f,0x9c, -0x79,0x47,0x5d,0x73,0xdb,0x11,0xc7,0xf2,0xd5,0x08,0x7d,0x43, -0x53,0x98,0xea,0xfa,0xe6,0xbb,0xd6,0x3b,0x72,0xbf,0x0d,0x19, -0x61,0xc5,0xe9,0xc6,0xfe,0x5e,0x97,0x87,0x3d,0xb6,0xa7,0x9e, -0x80,0x10,0xd8,0x14,0x1c,0x5b,0x48,0x76,0x4a,0x8d,0xca,0x34, -0x98,0x76,0xa5,0xac,0xc4,0x99,0xad,0x41,0x6d,0xe0,0xf3,0x8e, -0xa9,0x30,0x4b,0x6e,0xa5,0xcd,0x21,0xfc,0x08,0x52,0xd3,0x73, -0x58,0x63,0x8f,0xf6,0x24,0xad,0xc9,0xa1,0x0b,0x98,0x6b,0xf4, -0x96,0xc1,0x0e,0x98,0xb2,0xfa,0xde,0xc2,0x06,0xaa,0x94,0x4c, -0x63,0x2c,0x62,0xd2,0xa9,0xc5,0xe0,0xcc,0x01,0xc2,0x72,0xd1, -0xab,0xf0,0x0f,0x0a,0xae,0x71,0xc8,0x47,0xa8,0xe8,0x58,0x2b, -0x05,0x03,0xa6,0xa5,0x3a,0x91,0x0b,0x0f,0xf7,0xde,0xf4,0xbf, -0xa9,0xd3,0xb0,0x4e,0xd9,0x73,0x44,0x84,0xdf,0x7d,0x98,0xf6, -0x17,0xf5,0x21,0xe2,0x6a,0xee,0x48,0x3f,0x9b,0xf6,0x83,0x55, -0x5e,0xa7,0x3f,0x8c,0x9e,0x36,0x35,0x8b,0x34,0xec,0xb1,0x8e, -0x65,0x75,0xab,0xec,0x6a,0xb9,0xf6,0x00,0x81,0x50,0x83,0x50, -0xb0,0x87,0xdc,0x6a,0x7b,0x9b,0xca,0x0f,0x52,0xf3,0xa4,0xd6, -0x1d,0x43,0xc8,0x74,0x01,0xdd,0xa0,0x66,0x20,0x79,0x99,0x7f, -0xc9,0x68,0x02,0x27,0xbb,0xb8,0x0f,0x9e,0x7d,0x11,0xef,0xb4, -0x9e,0xee,0xc6,0x9f,0xef,0x66,0x75,0xa4,0x07,0x5f,0x8f,0xc9, -0xf5,0x62,0xaa,0x80,0xe7,0x0c,0x04,0x4d,0x27,0xf1,0xc0,0x32, -0x3f,0xa2,0x1a,0x45,0xad,0x93,0x36,0xb9,0xd0,0xfd,0x6b,0x79, -0x3e,0xc9,0x71,0xbc,0x3c,0x53,0x45,0x76,0x75,0xc6,0xd7,0x9e, -0xcb,0xf9,0xa3,0xfc,0x7f,0xcd,0xd6,0xd8,0xda,0xf2,0x16,0x0a, -0x1b,0x99,0xd7,0x2e,0x03,0xc8,0x0f,0x56,0xf0,0x6d,0xdb,0x04, -0x56,0xaf,0xc4,0x74,0xaf,0xd5,0x69,0x9b,0x6b,0x92,0xb9,0xb4, -0xf5,0x00,0x41,0x28,0xd7,0xea,0xbe,0x81,0x2d,0xe2,0x5d,0x02, -0x12,0xa7,0x2c,0x16,0x92,0x41,0xc0,0x79,0xca,0xac,0x98,0x7d, -0x36,0x07,0x8b,0x38,0x1e,0x5c,0x64,0xeb,0xb0,0xdf,0x52,0x21, -0x5f,0x53,0xaf,0x11,0x05,0xa6,0xd3,0x30,0xa3,0x67,0xd3,0xff, -0xf8,0x92,0x0d,0xb5,0x9b,0xe1,0x49,0x3f,0x5b,0x5b,0xeb,0xae, -0xdd,0x3d,0x1a,0xc5,0x29,0x4b,0x2f,0x30,0x65,0x37,0x09,0x87, -0x09,0x0f,0x31,0x86,0x36,0x03,0xb9,0x1f,0x9a,0x6e,0xfc,0xf9, -0xaf,0x6c,0xba,0xce,0x1d,0x4d,0x07,0xbb,0x0f,0x6a,0x39,0xac, -0xdf,0x34,0x1c,0x24,0xf7,0x4c,0xea,0x9f,0x6b,0x36,0x26,0x45, -0x13,0xcb,0x34,0xad,0xdd,0x6e,0xdc,0x60,0x8e,0x0b,0x88,0x76, -0x73,0xee,0xd7,0x50,0x7e,0x39,0xbb,0x9a,0x8f,0x72,0xbf,0xdd, -0x4c,0xa8,0xa1,0xca,0xf7,0x6f,0x76,0x52,0x6f,0x7b,0x45,0x9f, -0xcf,0x1b,0x2b,0xdb,0x02,0x9c,0xd9,0xdb,0xee,0xa0,0x0c,0x6f, -0xe3,0xc8,0x32,0xda,0xb4,0x9e,0xfd,0xf9,0x51,0x4a,0xb3,0x8c, -0x43,0x0a,0x2d,0x71,0x76,0x14,0xe3,0xdf,0x26,0x1f,0x02,0x2a, -0xe5,0xd5,0xb3,0x21,0x5c,0x31,0x2b,0x1b,0x6e,0x40,0x16,0xa8, -0x2e,0x47,0xd7,0x4c,0x56,0xc1,0xc6,0x85,0x9d,0x1f,0x0b,0x10, -0x4b,0x9b,0x89,0x8a,0xa9,0x28,0x87,0x14,0xa0,0x10,0xfa,0xe6, -0xba,0x59,0xe2,0xae,0x9a,0xe9,0x6e,0x00,0x4a,0x12,0x22,0xfa, -0xe6,0x6a,0x2e,0x0f,0x68,0x01,0x3b,0xc6,0xac,0x59,0xb3,0x4e, -0x6a,0x08,0x8a,0xa2,0x8f,0xf8,0x3e,0x5e,0xe6,0xd3,0xa0,0x0d, -0x38,0x70,0x68,0x31,0x2d,0x67,0x9e,0x67,0x33,0x8a,0x4c,0x83, -0xb9,0x49,0x0f,0xb3,0xcd,0x54,0xea,0x0f,0xd2,0x93,0x3e,0xa5, -0x44,0xb0,0x8c,0x45,0x69,0x44,0xcc,0x2d,0x1a,0xd4,0xd2,0x22, -0xb6,0x2a,0x5a,0x10,0xa2,0x60,0xed,0xb0,0x05,0xa1,0x50,0xe8, -0x40,0x9c,0x96,0x10,0x1c,0xa5,0x2c,0xc3,0xd8,0xc9,0xb3,0xb9, -0x4b,0xb4,0x43,0x55,0x3f,0x36,0xd0,0x2f,0x28,0xc7,0x4f,0x14, -0x5f,0x06,0xeb,0xb4,0x82,0x86,0xfa,0xb4,0xc6,0xae,0xd1,0x32, -0x5e,0x70,0x23,0x66,0x7c,0x14,0x4e,0xd0,0x1e,0x03,0x44,0x80, -0xdd,0x01,0x6f,0xf7,0xf4,0x13,0x0d,0x5c,0x12,0x32,0x1e,0x7f, -0xfd,0xd5,0x57,0x0f,0xbf,0x32,0xc2,0xd7,0x01,0xb8,0x84,0xd0, -0xf5,0x71,0xa7,0xc2,0x2b,0xee,0xa1,0xba,0xd5,0x0a,0xa8,0x91, -0x21,0x93,0x0b,0x6f,0x77,0x76,0x76,0x25,0x0b,0xb9,0x85,0x58, -0x6d,0xf3,0x59,0x3e,0x4a,0x21,0x87,0xe6,0x29,0x3e,0xda,0x9b, -0xab,0xeb,0xf0,0x02,0xba,0xf5,0x43,0x88,0x2c,0x07,0xe8,0xad, -0xc9,0x0a,0x68,0xa8,0x76,0x4d,0xf9,0x7d,0xa2,0xac,0x6d,0xde, -0xb5,0x8c,0x69,0x68,0x25,0xea,0x08,0x81,0xa5,0xdb,0x3c,0x2c, -0xdd,0xfa,0x49,0x97,0xe6,0xb0,0x57,0x8a,0xf1,0x7f,0x6f,0xaf, -0x14,0xe3,0x3f,0xd2,0x2b,0x1b,0x8d,0xbd,0xe2,0x3a,0xe5,0xff, -0x66,0x9f,0x3c,0xfe,0x2b,0xfb,0xa4,0xbc,0x9c,0x9f,0x9f,0x5f, -0xe5,0x81,0xb2,0x09,0xbf,0x7e,0x38,0x1e,0x11,0x15,0xd9,0xae, -0x62,0xb0,0x91,0x10,0xec,0x1a,0x3e,0x5b,0x4f,0xce,0x21,0x0a, -0xbb,0xf0,0x33,0xb9,0xe6,0x68,0xec,0xad,0xc4,0xa2,0x20,0xd1, -0x7a,0xbb,0x33,0xc8,0x6c,0x0a,0x80,0x0e,0x94,0xff,0x4a,0x3c, -0x62,0xe9,0xca,0x21,0xb5,0xc3,0xeb,0x9b,0x80,0x58,0x35,0x56, -0x17,0xf3,0xe1,0xd4,0x79,0x5b,0x0f,0x77,0x5d,0x8f,0xf7,0x16, -0x0b,0x84,0x80,0x45,0xb6,0x6f,0xe9,0x72,0x23,0x67,0xbb,0x93, -0x68,0x1e,0x67,0x5a,0x02,0x1b,0x80,0xd4,0xa9,0xf1,0xae,0x32, -0xb3,0x1b,0x82,0x27,0x99,0x10,0x53,0x57,0x93,0xc9,0x87,0xf9, -0xcd,0x4b,0x23,0x0e,0x34,0x30,0x63,0x6f,0xf5,0xa0,0xac,0x1e, -0x75,0x4f,0x93,0x5e,0x8f,0xe2,0x87,0x0d,0xfe,0x63,0x09,0x4b, -0x6e,0x39,0x99,0xce,0xbe,0x0b,0xc4,0x13,0xd3,0xaf,0x72,0xab, -0x69,0xa8,0xf0,0x89,0xe2,0x0a,0xdd,0x6a,0x85,0x4b,0x4b,0xdc, -0xb4,0x82,0xd4,0x88,0x12,0xde,0x50,0xc2,0xec,0x94,0x80,0xce, -0x30,0x1c,0x5d,0x31,0x4c,0xd7,0x95,0x5a,0x97,0x09,0x7e,0x8c, -0xb3,0x96,0xbd,0xca,0xcf,0x67,0x6a,0x8a,0x1b,0x58,0x5a,0x8b, -0x33,0x48,0x69,0x1b,0xc4,0xea,0x34,0xc3,0x4c,0x9b,0x00,0x73, -0x76,0xa8,0x17,0x8d,0x53,0x64,0x0f,0xc3,0xfd,0xd3,0xc5,0x82, -0xd6,0x26,0x70,0x46,0x6a,0xe7,0x2c,0x40,0x9d,0xee,0x0f,0x45, -0xde,0xa9,0xe5,0xb2,0xe6,0x50,0x00,0xeb,0xa1,0xe1,0x47,0x55, -0xe0,0xb3,0x1e,0x63,0x11,0x29,0x6c,0xb8,0x83,0x31,0x42,0xab, -0x6b,0xfa,0xd3,0xfc,0x72,0xf8,0x5b,0x21,0xdc,0x22,0xdf,0xd5, -0x29,0x66,0x36,0x2e,0x2d,0x83,0xcb,0x48,0x24,0x42,0xd8,0x9e, -0x93,0x1f,0xd3,0x95,0xbd,0xb6,0x92,0xaf,0xd8,0xcd,0xf5,0x1a, -0x32,0x1e,0x32,0x3e,0xf3,0x15,0xcc,0x2b,0x40,0xb2,0x50,0xc1, -0x4e,0x07,0x07,0x1e,0x7e,0xbc,0x1e,0x79,0xf8,0xeb,0x99,0x38, -0xfb,0xa5,0x6e,0x63,0x8a,0xcb,0xe2,0x32,0x12,0x16,0x77,0x71, -0x5d,0x4f,0xc5,0x33,0xa8,0x38,0x82,0xad,0xd5,0x09,0x35,0xfd, -0x3a,0xf5,0x48,0x44,0x86,0x27,0x80,0x48,0x3a,0x1f,0xcf,0x56, -0x15,0x87,0x6b,0xcf,0x2b,0xe8,0x6b,0xb5,0x3c,0x0a,0x3b,0x88, -0x14,0xc6,0x69,0x3e,0x42,0x71,0xd4,0x75,0x3f,0x6a,0xf8,0xd4, -0xea,0xed,0x01,0xf7,0xaf,0x65,0xec,0xcd,0x5d,0xec,0xf4,0x28, -0x46,0x50,0x22,0xf1,0xb5,0x51,0x43,0x0b,0xf7,0x6e,0x3e,0x6a, -0x0e,0x7c,0xa9,0x07,0x6c,0x86,0xb4,0x18,0x95,0x0b,0xa4,0xec, -0xeb,0xcc,0x2e,0x07,0xc9,0x2a,0x46,0x9a,0x0b,0x7c,0x6c,0x61, -0xf2,0xfe,0xbe,0x9e,0x10,0xb5,0x28,0x11,0x45,0x5f,0x03,0x0f, -0x92,0xc7,0x34,0x1e,0x01,0xbf,0x7e,0x6f,0x75,0x52,0xac,0x43, -0x3f,0xda,0xf9,0x32,0xf9,0x08,0x5c,0x77,0x36,0x09,0x1c,0x6f, -0x06,0x8c,0xab,0xe7,0xaf,0xd3,0x3d,0x21,0x57,0xc3,0x7b,0xda, -0xa4,0x3f,0xef,0x39,0x7e,0x24,0xf6,0x54,0xa9,0xbd,0x66,0x8f, -0x85,0xfb,0x03,0x64,0xfc,0x5e,0x88,0x74,0x59,0xbf,0xdb,0x6c, -0xec,0x36,0x57,0xe4,0x1e,0x53,0xa1,0x8a,0x32,0xea,0x27,0x14, -0x37,0xb5,0x80,0xab,0x17,0x2d,0x38,0x79,0xd7,0x3f,0xb3,0xa1, -0xdc,0x3c,0xd0,0x08,0x08,0xd7,0x19,0xaa,0x18,0xf3,0x8c,0xac, -0x4c,0x49,0x63,0xb3,0xde,0x50,0x53,0xef,0x0e,0x52,0x6f,0xa3, -0x01,0xb8,0x60,0x35,0xa1,0x6d,0x0a,0xa9,0x97,0xee,0xa8,0xaa, -0xa6,0xbc,0x1c,0x09,0xdb,0x71,0x50,0xb1,0xde,0x75,0x25,0x88, -0x1f,0x4e,0xa7,0xfe,0xaa,0xef,0xf0,0x6a,0xec,0xd4,0x7c,0x94, -0x5d,0x2b,0x3d,0xc8,0xb1,0xda,0x4d,0x58,0x83,0x82,0x2d,0xac, -0x37,0x92,0xa0,0x07,0x1f,0x4d,0x27,0x37,0x5f,0xfc,0xd9,0x35, -0x9f,0xb9,0x44,0x5e,0x80,0x11,0x8e,0x03,0x74,0x55,0x6d,0x0c, -0x21,0x11,0x43,0x8d,0xd8,0xfd,0xf9,0x15,0x84,0x75,0x1c,0x4b, -0x57,0xf8,0x5a,0x5e,0x82,0x98,0xc5,0x57,0x7a,0x02,0xa8,0xc9, -0x7c,0xa6,0x5f,0x1d,0x51,0x94,0x8c,0x9e,0x87,0x49,0xd1,0x81, -0xd0,0x76,0x0a,0xa0,0x77,0xf3,0xb3,0xe1,0x8c,0x25,0x4f,0x2a, -0xad,0x10,0xc8,0xec,0xe5,0xa9,0x44,0x28,0x37,0x04,0x6e,0xd2, -0x93,0xc5,0x82,0x52,0x58,0x9a,0x31,0x32,0x04,0xd7,0xdf,0x03, -0x7e,0xea,0xd7,0x41,0x00,0x29,0x7f,0x0e,0x2f,0x16,0xc1,0x57, -0xa4,0xf4,0x1b,0x70,0x63,0xce,0x04,0xf6,0x5f,0x69,0x0c,0x6a, -0x34,0x46,0xe3,0x96,0x40,0xae,0xc4,0xcb,0x85,0xe0,0xb1,0xb8, -0xef,0x2c,0xf4,0x3e,0x77,0xbe,0xa2,0x3f,0x46,0xc5,0xf9,0xb9, -0xde,0xb7,0x8d,0xb5,0x88,0xc5,0x08,0xea,0x76,0xe2,0x1d,0x1a, -0x37,0xf3,0x71,0xf1,0x2b,0xf9,0xe3,0xfa,0x75,0x5e,0x99,0x94, -0x45,0x79,0x88,0x3c,0xbb,0x86,0x37,0x7b,0x02,0x98,0x81,0x63, -0xf7,0xf0,0x00,0x61,0x59,0xae,0x5b,0xa5,0x1d,0x36,0x7e,0xc8, -0x36,0x3b,0xa4,0xe3,0x31,0x13,0x55,0x70,0x78,0x60,0x5d,0x4c, -0x44,0x45,0x67,0x43,0xe9,0x4e,0x19,0xaa,0xca,0x3c,0x1f,0x87, -0xb2,0x33,0xe3,0x6c,0x58,0xde,0x2d,0xc9,0x3d,0x4c,0xd8,0xda, -0x02,0x14,0x7d,0xf8,0xe3,0xe6,0x60,0xc6,0x22,0x45,0x2a,0x43, -0xa5,0x6a,0x08,0x1e,0x0a,0x7a,0x08,0xe1,0xab,0xec,0x6d,0xe5, -0xa9,0x3a,0x69,0x66,0x93,0xae,0x5b,0x9e,0x7c,0x71,0xbb,0x4b, -0x4b,0x2f,0xd1,0x7d,0xd0,0x15,0xb1,0x1d,0x2a,0xa2,0xc7,0x36, -0x79,0xaa,0x63,0x7c,0x88,0x32,0xf4,0x6d,0xce,0xa3,0x80,0x25, -0xa4,0x59,0xbd,0xfa,0xa5,0x23,0x84,0x56,0x9e,0xb6,0x5c,0x21, -0x15,0x77,0xad,0xa5,0x5b,0x5d,0x5a,0xf2,0xd1,0x64,0x04,0xc4, -0xc2,0xfd,0xc9,0xec,0x32,0x9f,0x4a,0x89,0x9c,0x15,0x4b,0x98, -0xae,0x10,0xc1,0x7e,0xb6,0xbb,0x64,0xf5,0x8b,0x1b,0x95,0xcd, -0x34,0x7b,0x73,0xdb,0x99,0x01,0x34,0x0e,0xe0,0xca,0xc7,0xd0, -0xc8,0xa8,0xdf,0x33,0xc8,0x40,0x07,0xf8,0x21,0x66,0x82,0x02, -0x75,0xbf,0x17,0x37,0xb2,0x33,0x8c,0xf4,0xc0,0xcb,0x1d,0xaa, -0x3c,0x62,0x23,0xe7,0x3b,0x12,0x22,0xca,0x8f,0x12,0x36,0x04, -0x89,0x75,0x0f,0xd9,0x71,0x09,0xe7,0x17,0xc4,0x71,0xf8,0xc0, -0x06,0x8e,0x93,0xfd,0xb3,0x63,0x06,0xed,0x17,0x70,0xa5,0xb7, -0x82,0x3c,0x6a,0x15,0xd2,0x71,0x88,0x19,0x39,0x93,0x53,0x5f, -0x57,0x0e,0xfb,0x00,0x1b,0xb1,0xd0,0x9e,0x80,0xe1,0xee,0x80, -0x37,0x1b,0x4b,0x2b,0x0d,0x09,0xd9,0xd9,0x1d,0x6d,0x03,0x68, -0xed,0xd1,0x36,0x21,0x35,0x72,0x1d,0x80,0xc1,0x59,0x36,0x87, -0x42,0xd4,0x8f,0xa9,0x9f,0xa3,0xd7,0xe2,0x41,0x66,0x9e,0x3b, -0x03,0x9f,0xf4,0xa5,0xb2,0x03,0xa4,0xc2,0x65,0x66,0xe8,0xde, -0x9d,0x59,0x88,0xfb,0x04,0x1a,0x1d,0x66,0x97,0xd1,0x0d,0x2c, -0x01,0x3c,0x61,0x0d,0x0e,0xe1,0x25,0x82,0x68,0x3c,0xbf,0x3e, -0xcd,0xa7,0xce,0xb2,0xc2,0xe6,0x9a,0x93,0x9c,0xcc,0x93,0x47, -0x05,0x25,0x09,0xcb,0x70,0x05,0xcc,0x7f,0x48,0xe8,0x15,0xa9, -0xae,0x5d,0xd7,0xbc,0x6f,0x79,0x86,0xd8,0xb7,0x12,0x01,0x2d, -0x8b,0x3f,0x35,0x4f,0x4b,0x6b,0x2b,0x61,0xb5,0xb3,0x44,0x72, -0x83,0x7e,0x36,0xb1,0x1b,0x6c,0x01,0x14,0xfb,0x6d,0x83,0xfd, -0x58,0x31,0x3f,0x08,0xe8,0xb5,0x9f,0xd8,0xb5,0x1b,0x34,0x96, -0x55,0x5e,0xae,0x68,0x7f,0xf0,0x56,0xb3,0xba,0xed,0xb5,0xb8, -0x4f,0xb2,0x3b,0x99,0x83,0x40,0x81,0xae,0xfd,0x46,0x61,0xbf, -0x63,0xbe,0x53,0xd6,0x68,0xbe,0x55,0x80,0x19,0x32,0x0d,0xda, -0x5e,0xa5,0x54,0xec,0x08,0x4b,0xeb,0x33,0x9d,0x98,0x5f,0x64, -0x16,0x0f,0x14,0x48,0xfd,0xd1,0xb1,0xbd,0xdd,0x5d,0xbb,0xa1, -0xa6,0x10,0x72,0x40,0xc6,0xac,0x1e,0x4e,0xb5,0xb4,0x32,0x9b, -0xdc,0xe8,0x3f,0xf9,0x4d,0x72,0x5b,0x51,0x80,0x64,0x1d,0xbd, -0xea,0x01,0x40,0x86,0xb0,0x8b,0xc5,0xae,0xc2,0x87,0x6c,0x37, -0xc1,0x22,0xee,0xa0,0xba,0xbf,0x37,0x58,0x2c,0x3a,0xdd,0x80, -0xb9,0xd8,0xc1,0x84,0x0f,0x67,0x79,0x71,0x15,0x23,0xba,0x6d, -0xc4,0x92,0xec,0x60,0xb5,0x5a,0xcc,0x53,0xc5,0x08,0x74,0x45, -0x44,0x5f,0x85,0xb9,0xd8,0x51,0x31,0xfa,0xd4,0x4d,0x10,0x06, -0xee,0x9c,0xb7,0x5a,0x83,0xcc,0x7c,0x81,0xfe,0xdb,0xca,0x00, -0x99,0x9d,0x98,0x00,0x45,0x42,0xda,0x59,0x25,0xf4,0x00,0x1d, -0x71,0xcb,0x78,0x07,0xe3,0x33,0x7f,0x1f,0x0e,0xaa,0x75,0x75, -0xaa,0x37,0x8a,0x23,0x37,0x0c,0xc0,0xd4,0x6a,0x6b,0x0b,0x60, -0xa9,0xb8,0xe9,0x77,0x48,0x37,0xcd,0xed,0x52,0x98,0xd1,0x23, -0xea,0x46,0x16,0x4f,0x31,0x4a,0x85,0xb0,0x01,0xe0,0x49,0xdd, -0x19,0x71,0xd7,0xa9,0xa8,0x9a,0x4f,0x1f,0x90,0x60,0xf9,0xa9, -0x80,0x3f,0x0e,0x4d,0xbc,0x10,0xca,0x6a,0x2c,0xf1,0x7b,0x88, -0x56,0xb7,0xcf,0xbb,0x28,0x0d,0xd3,0xaf,0xab,0x2a,0xd1,0x94, -0x9f,0x79,0x61,0x0a,0xb0,0x4e,0xf7,0xda,0x25,0x63,0xdd,0xab, -0x73,0xb2,0xcd,0xd5,0xa0,0x21,0x3c,0xba,0xe1,0x14,0x4c,0x5b, -0x50,0x01,0xe5,0xd6,0x22,0x81,0x3f,0x80,0x2f,0x44,0xf2,0x99, -0x93,0xee,0x15,0x7a,0x32,0x07,0x68,0x80,0x6e,0x89,0x47,0x24, -0x7a,0x8c,0x78,0x1b,0x26,0x6c,0xeb,0x3b,0x8e,0x74,0x9c,0x4c, -0x51,0x95,0x66,0x04,0xcd,0x68,0xe4,0xb9,0x06,0xcd,0x48,0x08, -0x0c,0x9e,0x27,0xc1,0xd1,0x2b,0xd1,0x01,0x29,0xcd,0x84,0xe8, -0xce,0x04,0xdd,0x14,0x02,0x19,0x33,0xb2,0x4a,0xec,0x5a,0x46, -0xbe,0x71,0x3d,0x2f,0x67,0x1b,0xa7,0xf9,0xc6,0xcd,0x10,0x2e, -0xf4,0x5a,0xea,0x37,0xc0,0x14,0xbd,0x8c,0x7c,0x45,0x11,0x62, -0x74,0x82,0xc5,0x79,0x72,0x0b,0x2a,0xc5,0x73,0xe7,0x64,0x04, -0xdf,0x50,0xa3,0x00,0x12,0xff,0xe9,0x2f,0xa8,0x91,0xcf,0xaf, -0x27,0xde,0xce,0x1a,0x47,0xba,0x66,0x54,0x97,0xc6,0x90,0x1c, -0x0d,0x22,0x9c,0xe7,0x4c,0xca,0xd2,0x72,0x25,0x3d,0xc8,0xb3, -0xd0,0xd0,0x70,0x16,0x74,0x53,0x04,0x15,0xb4,0x2f,0x37,0xa4, -0x13,0x8d,0x40,0x4d,0x83,0x56,0x19,0xa8,0xa4,0x81,0x27,0x52, -0x21,0xd9,0xa7,0xac,0xa6,0x97,0x18,0x0b,0x76,0xc6,0x28,0xbf, -0x92,0xea,0x09,0xfc,0x88,0x8f,0xc3,0x62,0x76,0xf7,0x59,0x9f, -0x21,0xa2,0xcc,0x67,0x47,0xc5,0x75,0xae,0x77,0x2d,0xf1,0xca, -0x31,0x82,0x57,0xbb,0xe9,0xc8,0x8e,0x6a,0xa0,0xea,0xcf,0x65, -0x90,0x25,0x1a,0x92,0x6e,0xb3,0x03,0xc4,0x71,0xf1,0x13,0xd5, -0x47,0xe2,0x3a,0x83,0x55,0x63,0x0c,0xb8,0x0a,0x22,0x86,0x81, -0x31,0x9b,0x5d,0x85,0xfd,0x03,0x15,0x9b,0x40,0x03,0xe6,0x68, -0xc1,0x4d,0x7a,0x16,0xef,0xd4,0x8c,0xbe,0x27,0xe3,0xdb,0xe8, -0x64,0xbb,0xac,0xe5,0x3b,0x2e,0xa8,0xfb,0x90,0x9f,0x32,0xf0, -0xa9,0x80,0xec,0x7e,0x38,0xcb,0x3d,0x26,0x6b,0x4b,0x31,0xa4, -0x1e,0xb0,0xc3,0x11,0x58,0x11,0xc3,0x7d,0xd0,0xde,0x2e,0x9a, -0x92,0x3e,0xd3,0x85,0xfc,0xba,0xaa,0xdc,0x40,0x92,0x97,0x2c, -0x6b,0x26,0x23,0x5e,0x59,0x98,0x7c,0xcc,0x0c,0xc2,0xae,0xa9, -0x79,0xb1,0x08,0xaa,0xc6,0xab,0xa8,0x9a,0x76,0x4b,0x9a,0x2e, -0x66,0x8e,0xe1,0xaf,0xb5,0xe4,0x0c,0xd4,0x41,0xfb,0x6c,0xc7, -0x3a,0x63,0xdb,0x40,0x49,0x2f,0x6d,0xb8,0xf7,0xb3,0x73,0xdd, -0xad,0x7d,0x6a,0x77,0x3f,0xb3,0x38,0x7a,0xf1,0x99,0xae,0x70, -0x6a,0xc6,0x04,0x7f,0x5f,0xd2,0xd0,0xa8,0xba,0xaa,0xbb,0xbe, -0x3a,0x49,0xb9,0xa8,0xfb,0xa2,0xd9,0x74,0x58,0x5c,0x99,0xd6, -0x84,0x08,0x56,0x8c,0x5b,0x8c,0x45,0xec,0x11,0x65,0xa9,0x4a, -0x9c,0xa6,0x16,0x87,0x1d,0x2c,0x02,0x67,0xb5,0xa3,0xa3,0xb8, -0xbe,0xce,0x47,0xc5,0xd0,0x78,0x9d,0x62,0xd4,0x34,0x3e,0x0c, -0x5d,0x90,0xa8,0xd7,0x93,0xeb,0x1b,0xe5,0x5b,0x1b,0x89,0x7e, -0x59,0xd5,0x5c,0xb6,0xb8,0xeb,0xb5,0xda,0x41,0x44,0x89,0x7a, -0xef,0x64,0xc0,0xb6,0x6d,0xc9,0x2e,0xd0,0xba,0x0f,0x99,0xbd, -0xc6,0x8f,0xc7,0xce,0x04,0x90,0x24,0x8d,0xbd,0xd6,0xb7,0x9f, -0xe8,0xcc,0x1d,0x1a,0x1b,0x9f,0x6f,0x28,0xbf,0xd6,0x43,0xcc, -0x16,0xdb,0xda,0xda,0x64,0x7c,0xd6,0x83,0xaa,0xe9,0xa1,0xe6, -0xae,0xc0,0x39,0x9f,0x18,0x64,0xee,0xe4,0xad,0xb1,0x66,0xd9, -0x61,0x93,0x4a,0x67,0x39,0x1e,0x0b,0x52,0x93,0x30,0x21,0xa9, -0x32,0x21,0x9d,0x8f,0xac,0x31,0x8d,0x11,0x72,0x17,0xed,0xd8, -0x56,0x30,0x46,0xf2,0x72,0x8b,0x2d,0x85,0x16,0x7d,0x48,0x01, -0xd8,0x83,0x87,0xc3,0x05,0x4d,0xc4,0x6b,0x0e,0x37,0x04,0xf3, -0x44,0xa6,0xe5,0xc2,0x1e,0x39,0xcd,0x13,0x8e,0x27,0x51,0x27, -0x23,0x0b,0x17,0x54,0xaa,0x1c,0xd4,0x74,0x13,0x19,0x2c,0x44, -0xae,0x97,0x15,0x77,0xbf,0x35,0x7b,0xdd,0x60,0x00,0x16,0xde, -0x2a,0x0a,0x3b,0x20,0xbd,0x99,0x07,0x51,0x3a,0x71,0x44,0xc3, -0x26,0xb0,0x42,0xcd,0x40,0xca,0x61,0x7a,0x4f,0x88,0xd4,0x0d, -0xcf,0xbd,0x71,0x8b,0x63,0x54,0x79,0xac,0xbc,0xd2,0x27,0xdb, -0x34,0x92,0x1f,0x77,0x7a,0xcd,0xdd,0x60,0x2c,0x6a,0xa0,0x0a, -0x34,0x85,0x74,0x17,0x01,0x44,0x6c,0x45,0x14,0x1e,0xc8,0xd2, -0x5c,0x33,0x38,0x16,0x86,0xf0,0x48,0xba,0xce,0xd2,0x30,0x7a, -0x39,0xc6,0x28,0xf6,0x1c,0xc4,0x2d,0x22,0xfe,0x87,0xd8,0xfb, -0x83,0x6e,0x60,0x80,0x01,0x58,0x68,0x89,0x35,0xf6,0x24,0x5b, -0x5b,0x68,0x7b,0x89,0x0a,0x20,0x78,0x37,0xad,0x01,0xa9,0x4b, -0xeb,0xe4,0x36,0x90,0x6c,0x1a,0x2d,0xd0,0xab,0xe6,0x9c,0xbc, -0xd5,0x5f,0x47,0xbd,0x60,0x37,0xea,0x99,0xb4,0x45,0xef,0x4a, -0x7b,0x96,0x92,0xc4,0xbe,0x62,0xfa,0x67,0x28,0xa2,0xf2,0x6b, -0x10,0x84,0x80,0x40,0x4f,0x5f,0xd8,0xc3,0x5b,0xc2,0x44,0xe4, -0x2f,0x0d,0xc6,0x76,0x46,0xf9,0x74,0xd6,0x40,0x99,0xd3,0x65, -0xfc,0x21,0xbb,0x7d,0xd6,0x5d,0x78,0x04,0x64,0xfc,0x54,0x55, -0x5d,0x18,0x12,0xa0,0x1e,0x32,0x34,0x0a,0x1b,0x0c,0x17,0x59, -0x90,0x1c,0x1b,0x47,0x89,0xd8,0xc3,0x70,0xad,0x03,0x3d,0x5a, -0xb0,0x4c,0x75,0xb8,0x50,0x32,0x1e,0xed,0x92,0x41,0x22,0x7a, -0x52,0xa8,0x3f,0xa4,0x41,0x09,0xb5,0x41,0xd4,0x71,0x72,0x23, -0xdd,0xf8,0xc7,0x89,0xc0,0x8f,0x36,0xb0,0xdb,0x74,0x72,0x03, -0x64,0x72,0x32,0x10,0x47,0x51,0xe8,0x29,0xc1,0x46,0x9e,0x67, -0xb1,0x16,0x42,0x29,0xd4,0x7c,0x3c,0xde,0xf3,0x33,0xbd,0xf1, -0x47,0x35,0x79,0xf8,0x29,0x78,0x77,0xc7,0x56,0x80,0xa7,0x8d, -0xae,0x21,0x51,0xc5,0x7c,0xf3,0x99,0x24,0x57,0xdb,0x8e,0x18, -0x49,0xf0,0xe6,0x33,0xae,0x03,0xd7,0x45,0x38,0x64,0xfe,0x3c, -0x79,0xb5,0x1d,0x6a,0xd5,0x89,0x48,0x30,0xd0,0xa9,0x17,0xb7, -0x1a,0xe2,0x9c,0x78,0x44,0x14,0x9a,0x0b,0xbb,0xff,0x2d,0xbd, -0x69,0x0f,0xec,0x6d,0xb7,0x92,0x65,0x7c,0x4d,0x17,0x8b,0xad, -0x0b,0x46,0x46,0xa9,0xa7,0x4f,0x78,0xd8,0xc1,0xf3,0xc8,0xe0, -0x78,0x12,0x6d,0x2d,0xd1,0xbe,0x03,0xce,0x1a,0x9d,0x4f,0x12, -0x73,0x56,0x89,0x92,0xb7,0x5c,0x27,0xf1,0xec,0xb7,0x26,0x3c, -0xb2,0x8c,0x17,0x8d,0xb3,0x1a,0x0a,0xd3,0x3d,0xa6,0x5f,0x85, -0x0a,0x4b,0x9d,0xaa,0xe1,0xe1,0x0c,0x9c,0x4b,0x9c,0xe2,0x0f, -0x99,0x31,0x08,0x03,0x85,0x8d,0x5d,0xcd,0xf4,0x87,0x8b,0x45, -0x67,0x47,0xa7,0x76,0x76,0x4e,0xdd,0x2d,0x01,0x9d,0x48,0x0f, -0x16,0x14,0xcb,0x75,0x87,0xde,0x6d,0x2f,0xdd,0x5c,0xc3,0x6c, -0x68,0x6f,0x7b,0x25,0xea,0x34,0xcc,0x3e,0xcd,0x4e,0x5d,0x36, -0xe9,0x53,0xb4,0x48,0x55,0xbe,0x06,0xa7,0xdd,0xe6,0x22,0x20, -0xf7,0x25,0x6e,0x51,0x6d,0xee,0x66,0x90,0x7d,0x9a,0x38,0x83, -0xbd,0xf2,0x63,0x31,0xd3,0x03,0xc1,0xc2,0x6a,0x71,0x71,0x58, -0xe6,0x51,0x9f,0x56,0xa5,0x0d,0x2a,0x36,0x88,0x52,0x47,0x39, -0xfb,0x57,0x3c,0x05,0x87,0xe0,0x02,0xf2,0x35,0xaa,0x42,0x05, -0xe4,0x66,0xd6,0x1a,0xf6,0x4e,0xf5,0xdf,0xd3,0x54,0x8f,0x8b, -0x61,0xcf,0x34,0x4c,0xaa,0x7f,0x5a,0xa7,0x7e,0x61,0x90,0x28, -0x75,0x51,0x2f,0x8d,0x43,0x02,0x5b,0x8c,0xad,0x9a,0x72,0xef, -0xf2,0x8b,0xe7,0x9f,0x6e,0x44,0xa5,0x6d,0x8e,0xc4,0xa2,0x9b, -0x8a,0x9e,0xb6,0xb6,0x86,0x6d,0xba,0x28,0x0a,0x69,0xf4,0x04, -0x69,0xd7,0x7a,0x66,0x80,0xe0,0x8e,0xa0,0xf6,0x05,0x72,0x8a, -0x8b,0xf1,0x64,0x9a,0x83,0xcf,0x73,0xc8,0x72,0x6f,0xa0,0x69, -0x8d,0x78,0xb1,0xb6,0x0a,0x5f,0xdd,0xb7,0x95,0xb4,0x53,0xd7, -0xba,0x81,0x0e,0x8e,0x46,0x90,0xbf,0x4e,0x18,0xa5,0x21,0x66, -0xf5,0xf9,0x2e,0x84,0xb0,0x89,0x39,0x0d,0x32,0x4e,0xc9,0x04, -0xfa,0x29,0xa8,0xd2,0x86,0xc0,0x1b,0xca,0xd9,0x74,0x8e,0x71, -0xe5,0x4f,0x31,0xed,0x54,0xa6,0xa1,0xd5,0x0d,0x24,0x83,0xe5, -0xcd,0x53,0xb4,0x96,0xdb,0x8c,0xbd,0x25,0x02,0x73,0xf5,0xfa, -0x80,0xbf,0x72,0xc4,0x0d,0x9f,0x5a,0xe3,0x3a,0x0b,0x7c,0xca, -0xc0,0xa7,0x21,0x30,0x65,0xb8,0xef,0xe6,0x0f,0xa5,0xe3,0xa9, -0x84,0xa7,0x0c,0xbd,0x9e,0xd2,0xd0,0x45,0x5b,0x80,0x5d,0xe5, -0x2c,0xd6,0xa1,0x75,0x4d,0xb7,0xe2,0x3c,0x1f,0x44,0x59,0x26, -0x86,0x25,0x70,0x20,0x28,0x33,0x34,0xe2,0x00,0x17,0xc5,0x44, -0xfd,0xd5,0x2c,0x0d,0xa0,0xea,0x1a,0xd2,0xb6,0xb7,0x9d,0x60, -0x9f,0xff,0x1a,0x0f,0xfb,0x90,0x38,0x50,0xa7,0xfc,0xeb,0x4f, -0xe6,0xa4,0x9b,0x74,0x29,0x5c,0x5a,0x68,0xf9,0x0a,0x7d,0x43, -0x92,0xd7,0x90,0xe5,0x2e,0x24,0xa3,0xd5,0x52,0x9b,0x06,0x3b, -0x65,0x9f,0x72,0x36,0xd6,0x05,0xfc,0x58,0x57,0x85,0x3f,0x41, -0x4d,0x49,0x82,0x66,0xda,0xf0,0xbd,0xac,0x28,0xc3,0x3a,0xb9, -0xbe,0x53,0x57,0x9f,0x41,0xb8,0x49,0x5f,0xc3,0xc5,0x4c,0x83, -0x41,0xe2,0xd2,0x46,0x3e,0xe3,0xf6,0x9d,0xdc,0xc4,0xae,0xb1, -0xf1,0x85,0x45,0x0b,0xf4,0x57,0xf6,0xfc,0xd7,0xb9,0x67,0xa5, -0xa0,0x4e,0xdd,0x62,0xf0,0x2b,0x72,0xb8,0xfe,0xc0,0x9c,0xfe, -0xb2,0x45,0x65,0x56,0x11,0x6e,0xad,0xa5,0x86,0xe9,0x6c,0xec, -0x39,0x9f,0x39,0xd3,0xa1,0x37,0xf3,0x09,0x69,0x93,0xc9,0xeb, -0x83,0x99,0x00,0x35,0xab,0x9d,0xfd,0x7a,0x23,0xe7,0xd6,0x98, -0x16,0x13,0x79,0x7c,0x3d,0xbd,0x6e,0xed,0xd8,0x8c,0x37,0x27, -0xe0,0xd5,0xad,0xb3,0x49,0xd5,0x8d,0x27,0xa3,0x1c,0x04,0x6f, -0xfe,0x32,0xb2,0x83,0xf1,0xee,0xf9,0x86,0x52,0x3c,0xe1,0xa9, -0x8e,0x47,0x9f,0x8f,0xa2,0x1a,0x4e,0xb9,0x85,0xaa,0xd1,0xb6, -0x26,0xf3,0xb6,0x03,0x4b,0xc5,0x91,0x1c,0xac,0x59,0x40,0xa4, -0x22,0x33,0xc7,0xf4,0x23,0x55,0xa1,0x1f,0x88,0xa5,0xea,0x07, -0x60,0x8f,0xfa,0x87,0xb8,0x9d,0x0c,0xfa,0x30,0xc6,0xa9,0x71, -0xd2,0x8f,0x8a,0x32,0x6a,0x91,0x6f,0x8b,0x5a,0x1a,0xaa,0x74, -0x67,0x99,0xfd,0x3a,0x2a,0xd9,0x8a,0x06,0xd1,0x92,0x8c,0xba, -0x84,0xbd,0x82,0xdb,0x03,0x69,0x99,0xc3,0xcb,0x5a,0xd9,0xf2, -0x9b,0xae,0x13,0x23,0xa8,0x32,0xcf,0x23,0x3c,0x99,0x8e,0x4c, -0x21,0xcb,0x2d,0x77,0xda,0x3b,0x7a,0x4a,0x49,0x2e,0x53,0x8b, -0xd8,0x15,0xcc,0x0c,0x9b,0x85,0x75,0x9a,0xc8,0x7d,0x01,0x87, -0xf0,0x0d,0x62,0x84,0xc9,0x65,0xe3,0xe3,0xcd,0x42,0xb3,0x93, -0xd7,0x31,0xba,0xb8,0x7b,0x71,0x35,0x19,0xf2,0x06,0x8d,0x7a, -0x51,0xe7,0x34,0xcb,0x22,0xd4,0x19,0xce,0x86,0x79,0x13,0x6d, -0x93,0xa8,0x20,0x2f,0x55,0xab,0xfa,0x7f,0x73,0x77,0xb1,0xe0, -0xa7,0x8e,0x5e,0x33,0x2a,0x8b,0x5c,0xf3,0xd8,0x82,0x78,0xe5, -0xab,0xac,0xa6,0x32,0x4b,0x86,0x0d,0x41,0x5d,0x0f,0x2e,0x45, -0x34,0x28,0xa0,0x7b,0xa8,0xf9,0x9e,0x82,0xef,0x38,0xc0,0xd2, -0x84,0x30,0x50,0x78,0x3c,0x01,0xc7,0x0b,0x5a,0x00,0xab,0x89, -0xc8,0xbf,0xc1,0x17,0xc4,0xab,0xbe,0x16,0x38,0x3e,0xb4,0x53, -0x38,0x37,0x19,0xde,0x5a,0xd3,0x66,0xdc,0x9c,0x3b,0xa8,0x71, -0x8d,0x8d,0x88,0xd5,0x2c,0x9c,0x9d,0xcd,0xaf,0xe9,0xc6,0x7e, -0xec,0x9b,0xfc,0xe2,0x16,0x71,0xcc,0x3b,0x32,0x04,0x83,0xdd, -0x61,0xbd,0x69,0x9d,0x3b,0x0c,0x47,0x40,0x3e,0xab,0x1b,0x4d, -0x44,0xb4,0xf4,0xeb,0x62,0xac,0x34,0xee,0xa0,0x1f,0x74,0x8a, -0x1e,0xc9,0x70,0x69,0x01,0xf3,0x8b,0x31,0x9c,0xcc,0xe9,0x9f, -0x16,0xd2,0x72,0x7e,0x35,0xd1,0x64,0xe2,0x23,0xdb,0x2e,0x27, -0x0f,0x00,0x78,0x1b,0x20,0x3a,0xc6,0x2a,0x99,0x1a,0x05,0xdc, -0x2f,0xdc,0xe6,0xe5,0xd9,0xf0,0x26,0x4f,0x6f,0xa3,0xad,0x28, -0x8d,0xb6,0x40,0xcf,0xa6,0x39,0xc0,0x63,0x78,0xbe,0x9a,0xc1, -0xe3,0x3e,0x3c,0x5e,0xc0,0xe3,0xfd,0xe8,0xbe,0x7e,0xfc,0x75, -0x3e,0xc1,0xf4,0xfb,0x90,0xfe,0x9f,0x9f,0xf6,0xbe,0xe9,0xea, -0xe9,0xdc,0xb5,0x08,0xdb,0xf3,0x31,0x61,0xcc,0xcc,0x7e,0x37, -0x76,0x79,0x94,0x93,0x08,0x12,0x34,0xab,0xc9,0x3f,0xe9,0x86, -0x37,0x64,0xc0,0xae,0x9b,0xd8,0x8f,0x5e,0xa5,0xa3,0x16,0xef, -0x82,0x2b,0x18,0x38,0xfa,0x68,0x94,0x00,0x37,0x51,0xd1,0x45, -0x94,0x28,0x53,0xaf,0x87,0x22,0xae,0x41,0x61,0x00,0x0d,0x92, -0x05,0x60,0x49,0x08,0x0b,0x2c,0x60,0xcc,0x34,0x09,0x48,0x27, -0x1b,0x78,0xc9,0x0f,0xcd,0xed,0xa0,0x13,0x73,0x4d,0x47,0x9e, -0xb0,0xc2,0xe4,0x0a,0xfa,0x8c,0x12,0x7b,0x51,0x94,0x6a,0xa2, -0x5b,0x0c,0xd2,0x9e,0xe6,0x37,0x57,0x43,0xbd,0xb3,0xf0,0x9a, -0xc2,0x60,0x14,0x95,0x81,0x7b,0x2b,0xb7,0x84,0x9a,0x0f,0x31, -0x80,0x7d,0xcc,0x87,0x5b,0x3c,0xc8,0xa6,0x84,0x8e,0xd1,0x4c, -0x38,0xcd,0x00,0xc0,0xd3,0x2b,0xce,0x30,0x7f,0x71,0xd5,0x39, -0x89,0x37,0x6b,0xbb,0xce,0xc8,0x94,0xb2,0xfb,0xa6,0xe0,0xa0, -0x7b,0x97,0xad,0xba,0x9d,0xbc,0x80,0x35,0x15,0x17,0x39,0x3e, -0x15,0x21,0xcb,0xc2,0x36,0x16,0x6a,0x08,0xda,0x11,0x05,0xcb, -0x8d,0xd1,0xf7,0x65,0x27,0xec,0x4e,0x09,0xb6,0x79,0xf8,0xa4, -0x7b,0xc9,0x1e,0x2c,0xf6,0x83,0xe5,0x48,0x6a,0x22,0x3d,0xbf, -0x16,0xeb,0xa8,0x24,0xcd,0x65,0x09,0xf8,0x10,0xd4,0xcd,0x09, -0x5d,0xdd,0x09,0x2b,0x69,0x97,0x26,0x18,0x72,0x31,0x7a,0x0a, -0x76,0xc5,0xf9,0x54,0x4f,0x7a,0x63,0x72,0xf6,0x52,0xf0,0x44, -0xcd,0x96,0xce,0x8b,0x4f,0x26,0xe4,0x6e,0xd6,0x6a,0xd9,0x02, -0xad,0x28,0xb2,0xc4,0x20,0x50,0x8f,0x7e,0x34,0x44,0x5a,0xe0, -0xdd,0x85,0x59,0x7e,0x7d,0x73,0x85,0x3e,0x69,0x66,0x33,0x3d, -0x56,0x60,0x82,0x40,0x83,0xea,0x94,0x74,0xe7,0xf1,0xdf,0xe2, -0xfe,0x71,0x79,0x7c,0x38,0x68,0xf5,0x92,0xbf,0xed,0xef,0x5c, -0xd0,0x76,0xf3,0x66,0x72,0xc5,0xb9,0x59,0x90,0xcd,0xf3,0x42, -0xe7,0x6c,0xfb,0x39,0xf4,0x19,0xe3,0x09,0x86,0x53,0xcf,0x76, -0xe2,0x76,0xf2,0xf3,0x0e,0x43,0xeb,0x0a,0x71,0x86,0xdf,0xd7, -0x13,0xe0,0xf8,0x58,0x3f,0xe8,0x3f,0xfa,0x69,0x0a,0x6e,0x2d, -0xe1,0x61,0xac,0x1f,0x40,0x86,0xf8,0x7f,0xf4,0x2f,0x44,0x93, -0x3a,0x9e,0xef,0xed,0xee,0xfd,0x5d,0xbf,0xd0,0x2f,0x27,0x7c, -0xcb,0x09,0xdf,0x46,0x4b,0xc6,0x3b,0xcd,0x76,0x8e,0x8f,0x17, -0xf7,0x17,0xc7,0xd3,0xc5,0xf1,0x78,0x71,0x3c,0x5b,0x50,0x41, -0xfa,0xf9,0x76,0xe7,0xa2,0xeb,0x3e,0x5d,0xa8,0x57,0x81,0x5f, -0xa2,0xdf,0xd8,0x92,0x9b,0xc3,0x18,0x59,0x01,0xc3,0xef,0x9a, -0x44,0xcf,0x77,0xd8,0xd2,0xc2,0xd6,0xb4,0x26,0x71,0x20,0x9c, -0x37,0xec,0x4d,0x8a,0x19,0x46,0x3f,0x36,0xa5,0x98,0xd3,0xe8, -0x2d,0x35,0x35,0x4f,0xc2,0xfb,0x3c,0xe5,0x20,0x44,0xb3,0xaf, -0x02,0x33,0x1d,0x57,0x81,0x19,0x48,0x06,0xb4,0xb8,0xc7,0x7c, -0xcc,0x5e,0xa2,0xa1,0x0d,0x66,0x74,0x72,0x72,0xd3,0xca,0xee, -0x47,0x5d,0x68,0x04,0xcb,0x31,0x98,0xf4,0x80,0x47,0x70,0x1b, -0xcb,0xf1,0xa0,0x4c,0xed,0x6a,0x72,0x7e,0x5e,0x8a,0x6b,0x97, -0x84,0xbe,0x95,0x21,0x5a,0x52,0x70,0xb0,0xfd,0x3f,0xc1,0x39, -0xe6,0x44,0xfd,0xd6,0xc0,0x8e,0x60,0x5c,0xb4,0x78,0xc8,0x58, -0x46,0xc4,0x74,0xc0,0x16,0x86,0x6b,0x89,0xee,0xb7,0x8e,0xc7, -0x31,0x78,0xa6,0xcd,0x62,0x03,0xaf,0x99,0xae,0x96,0x0e,0x81, -0x15,0xf5,0xee,0xdf,0x07,0xdd,0x0a,0xa6,0x02,0x4c,0x92,0x68, -0xe8,0xfb,0xd8,0x16,0xf6,0x43,0x1a,0x91,0x09,0x98,0x00,0xa3, -0x46,0xc4,0x78,0x4c,0x23,0x78,0x48,0xba,0x7a,0x10,0xb7,0x4c, -0x4e,0x4b,0x0f,0x69,0x6e,0x69,0xd3,0x05,0xd4,0x10,0x2d,0xfc, -0x28,0xb3,0x17,0xc4,0x17,0xb8,0x66,0xe4,0x61,0xb1,0x43,0xad, -0xad,0x87,0x55,0x31,0x3c,0xbd,0x82,0x53,0x26,0xd3,0x7f,0x60, -0xdf,0x1a,0xa3,0xb0,0x7a,0xbb,0x4c,0x6e,0xa1,0x4e,0x2e,0x1c, -0x2d,0xf5,0x4b,0x62,0xfb,0x99,0x3d,0x57,0x2b,0x4d,0x45,0x76, -0xff,0xbe,0xfe,0xf9,0xa5,0x12,0xa1,0x10,0x86,0x8b,0x66,0xe4, -0xc5,0xd8,0x93,0x85,0x90,0x6c,0x0d,0x1e,0xaa,0xd7,0xee,0xdf, -0xd7,0x7b,0xcd,0xae,0xac,0xd0,0x30,0xf0,0x93,0x1b,0x48,0xee, -0xce,0xa6,0x9f,0x6f,0x69,0x0a,0xe1,0x14,0xb0,0x3c,0xbd,0xe6, -0x6b,0x40,0xf7,0xa0,0xc7,0xe7,0x49,0xc4,0xe4,0x26,0xcb,0x33, -0x68,0x89,0x58,0x73,0x69,0x3a,0x73,0xc8,0x8d,0x42,0x84,0x87, -0x3f,0x6a,0x31,0xd0,0xfd,0xba,0xd5,0x3d,0x43,0x45,0xe4,0x72, -0xfd,0x84,0xe6,0x5f,0x75,0xa2,0x7b,0x81,0xfd,0xa8,0x84,0xe0, -0xc5,0x5c,0xd6,0x1e,0x09,0x9b,0xf2,0xa6,0x6a,0x2b,0xbb,0xeb, -0x51,0xd1,0xf8,0x11,0xb0,0xd6,0xfb,0xbd,0x10,0x29,0x83,0x08, -0x35,0x85,0x97,0xc3,0xca,0x6a,0x65,0xda,0x8d,0xee,0x01,0x20, -0x44,0x6c,0xdd,0xbd,0x54,0x56,0x5b,0x3f,0x68,0xc8,0x09,0x82, -0xf7,0xfc,0xb2,0xa8,0x38,0xec,0xf2,0xd2,0x18,0x83,0xdb,0x23, -0x0e,0xd3,0x37,0x81,0x10,0xda,0x18,0x1a,0x53,0xe9,0xde,0xfa, -0x0d,0xec,0x62,0xf5,0x53,0x79,0x59,0x9c,0x03,0x93,0x05,0xf5, -0x39,0xfc,0x60,0x30,0x56,0x94,0x4c,0x28,0xa7,0xb2,0x51,0xa3, -0x73,0x42,0x90,0x0e,0x84,0xf7,0xa9,0x75,0x97,0x4e,0xd8,0x3d, -0x78,0x0b,0xa7,0xf3,0x4f,0x01,0x18,0x8d,0x19,0x39,0xdc,0xa2, -0x75,0x0b,0x27,0xd3,0xb8,0x99,0x01,0xca,0xad,0x2d,0x43,0x23, -0xbd,0x2f,0x16,0xbb,0x9b,0x72,0x43,0xbe,0x58,0x90,0x3b,0x5a, -0xbc,0xc7,0xb6,0x3b,0xa8,0xae,0xba,0xb8,0x29,0x59,0xda,0x66, -0x21,0xf5,0xb6,0xfe,0x60,0x98,0x00,0xd0,0x00,0x88,0xfb,0xaf, -0xfb,0x6a,0xef,0x78,0x44,0x90,0xe1,0x7d,0xb0,0xd7,0x24,0xe2, -0xd3,0x59,0xec,0x62,0xcd,0xb0,0xa8,0x47,0xdd,0x62,0x6f,0xa7, -0xd5,0x8a,0xc4,0xc0,0x80,0xb3,0x59,0xda,0xab,0x04,0xd7,0xfd, -0x02,0x60,0xae,0x17,0x64,0x0f,0x47,0xa0,0x90,0x98,0xac,0xf7, -0x2d,0x11,0x3b,0xfd,0x66,0x38,0x3a,0x9a,0x7c,0xc8,0xad,0xb1, -0xdb,0x7c,0x3c,0xab,0x1e,0x4f,0xba,0xf9,0x06,0xf7,0xdd,0x7e, -0xca,0xa7,0x93,0x17,0x85,0x46,0x8f,0x72,0x8f,0x6b,0x89,0x61, -0xc2,0xe5,0x85,0x7f,0xc9,0xc9,0x74,0x54,0x8c,0x87,0x57,0xc5, -0xef,0xb9,0xa8,0x45,0xaf,0x14,0xc5,0x64,0xb4,0xaa,0x1a,0xfc, -0x9e,0xab,0x21,0x44,0xce,0x69,0x33,0x8a,0x9a,0xda,0x18,0x8f, -0xa8,0xee,0x15,0x78,0xf4,0x1b,0x5e,0xe4,0x60,0xce,0x67,0x13, -0x0f,0x26,0x18,0x9f,0x86,0x7d,0x34,0xea,0x06,0xcf,0xcf,0x3e, -0x80,0xc7,0x6c,0xbd,0x0b,0xfa,0x68,0x3d,0x3b,0x72,0xd7,0x20, -0x6a,0x4e,0x73,0x18,0x9e,0x19,0x2f,0x7f,0x23,0xe3,0xda,0x8f, -0xed,0x4b,0xa6,0xd7,0xf8,0x6d,0xa3,0x97,0x70,0xff,0x22,0xb3, -0xef,0xa4,0x5f,0x79,0xaf,0xb7,0xfa,0xa5,0x2b,0xa2,0x3e,0xe7, -0x43,0x38,0x32,0xf4,0x0b,0xb5,0x21,0x15,0xec,0x27,0xaf,0xf5, -0x96,0xef,0xb2,0x9a,0x8d,0xc9,0x90,0xff,0x31,0xcf,0x3f,0x54, -0xb3,0x21,0x15,0x72,0x47,0xc3,0xcf,0xd5,0xcc,0x11,0xe8,0x92, -0x76,0xd5,0xa5,0x66,0x62,0xd5,0x4c,0x48,0xc5,0x7a,0x8b,0xf1, -0x7c,0x96,0xd7,0x54,0x8c,0xe9,0x68,0xd9,0x99,0xeb,0x06,0x19, -0x55,0x21,0x28,0x9d,0x70,0x5c,0x5d,0x15,0x4d,0x60,0x22,0x53, -0xc3,0x76,0x69,0xa8,0xe2,0x85,0x95,0xcc,0xb4,0x8d,0xa2,0x44, -0x0f,0x4d,0x4b,0xbe,0xb5,0x3a,0xf9,0xc3,0x07,0xe6,0xf9,0xeb, -0xfc,0xd1,0x03,0xa6,0xba,0xf5,0xf0,0xeb,0xfc,0xab,0x07,0xf8, -0x81,0x8c,0x03,0x1b,0xa2,0x05,0x7f,0x5b,0xdf,0x3c,0xc0,0x36, -0x33,0xc8,0xa9,0x7d,0x5b,0xf4,0xdb,0xea,0xec,0x3d,0xc0,0xfe, -0xb0,0xc5,0x66,0x43,0x38,0x29,0xa3,0xb7,0xd3,0xf9,0xa9,0x5e, -0x0c,0x62,0x19,0x62,0x82,0x86,0x07,0xaa,0x29,0xad,0xfd,0x2d, -0xe9,0x48,0x4f,0x03,0xbf,0x86,0x71,0x01,0x7a,0xd9,0x21,0x6c, -0xea,0x4f,0xf5,0x1f,0xe7,0x38,0x36,0x84,0x8b,0x8c,0x92,0x25, -0xc2,0x02,0xd6,0x7f,0x62,0x76,0x6a,0x1f,0x13,0x55,0x2d,0x85, -0x3c,0xe0,0xcd,0x39,0x17,0xe2,0x37,0x5d,0x86,0x9f,0x12,0x25, -0x22,0xfe,0x0c,0x4f,0xcb,0x77,0x60,0xd6,0x19,0x93,0x6d,0xb8, -0x9d,0x5f,0xbb,0xfb,0x94,0xd0,0x73,0xf6,0xb7,0x0c,0x91,0x0a, -0x05,0x01,0x27,0x39,0x74,0xde,0xec,0xa7,0x5c,0xf6,0xd4,0xf1, -0x2a,0x70,0x5f,0x47,0xf7,0x7f,0x32,0x82,0x81,0x9d,0x0c,0xdf, -0x12,0x62,0xfb,0x61,0x59,0xa8,0x9b,0x30,0x70,0xb4,0x1b,0xb5, -0xe8,0xd1,0x34,0x19,0xbd,0x79,0x6e,0x91,0xdf,0x4c,0x0f,0xe7, -0xa7,0x18,0x6e,0xc9,0x4c,0x4a,0x08,0x68,0xc9,0x53,0xfc,0x7a, -0x72,0xad,0xec,0x98,0x2a,0xca,0x27,0x23,0xb0,0xef,0x52,0x74, -0x6a,0xf2,0xfe,0x46,0xf7,0xb1,0x6e,0x1f,0x12,0x70,0xe9,0x7a, -0x23,0x8e,0x21,0x9a,0x1f,0xfe,0x18,0x36,0x48,0xfc,0x31,0x49, -0xb3,0xcc,0xe5,0xc1,0xab,0x99,0xb7,0xa2,0x04,0x26,0x74,0x65, -0x41,0xf0,0x6e,0x78,0xad,0xe1,0xdb,0x6c,0xfd,0x13,0xb7,0xe8, -0xdd,0x1b,0xe3,0x0f,0x0c,0xc1,0x89,0x8a,0x01,0xf3,0x62,0x41, -0x98,0xa0,0x9f,0xcd,0x24,0x85,0x62,0xf4,0x1c,0x27,0x3c,0xaf, -0x21,0x09,0x9e,0xe2,0x24,0x41,0xfa,0xa8,0x2e,0xf8,0xd6,0xd8, -0x3e,0x24,0x38,0x21,0x44,0x05,0x84,0x99,0x40,0xf1,0x39,0x76, -0x4f,0x09,0x4f,0x11,0x09,0xee,0x7d,0xcb,0x66,0xb5,0x41,0x11, -0x93,0xee,0x81,0xf6,0x5c,0x24,0x02,0xce,0xba,0x6f,0x71,0xdf, -0xc0,0x9f,0x05,0xf4,0xf0,0x47,0xe0,0x37,0xc9,0x38,0xc7,0x46, -0x95,0x4f,0xd7,0xdd,0x1a,0x75,0xe1,0x15,0x0f,0xa8,0x6d,0x5f, -0x83,0x49,0xc5,0x25,0x5a,0x50,0x63,0x37,0x60,0xa5,0x03,0xc0, -0x2f,0x47,0x8a,0xb7,0x06,0xa7,0x39,0x12,0x55,0x92,0xe5,0x7d, -0x87,0x6e,0x56,0xec,0xa9,0x91,0xfe,0xfe,0xa7,0x13,0xd4,0x75, -0xf1,0x66,0x1f,0xac,0x35,0x32,0xe3,0x55,0x82,0xa1,0xcd,0x16, -0x82,0x0a,0x99,0x63,0x24,0xb6,0xeb,0x78,0x56,0x9c,0x9f,0x53, -0x01,0x3d,0xad,0xfd,0x02,0xdb,0x41,0x01,0xb8,0x78,0x04,0x96, -0x99,0x30,0x1d,0xd9,0x22,0x85,0x15,0x8e,0xb1,0x20,0x84,0x6f, -0x2d,0x74,0x34,0x97,0xda,0xe4,0xeb,0x0f,0x7b,0xfa,0x79,0xb1, -0xd8,0xf4,0x80,0x66,0x93,0x97,0x7a,0x6e,0x59,0x50,0x88,0xdd, -0x21,0x92,0xa0,0x44,0xa2,0xfb,0x15,0xab,0x6c,0xb5,0xcc,0xd4, -0xa5,0x57,0x47,0xb8,0x6b,0x25,0xbb,0x3e,0xd0,0xfa,0x38,0x87, -0xbf,0xa8,0x70,0xe2,0x27,0x72,0x88,0xf0,0x31,0x87,0x80,0x3c, -0x98,0xa4,0x5b,0xfd,0x15,0xbc,0x53,0xe4,0x67,0xbb,0xe7,0x04, -0x55,0x44,0x79,0x6f,0x47,0x45,0xf7,0x3a,0x51,0xd2,0x45,0x48, -0x84,0x7f,0x02,0x21,0xa3,0xf5,0x46,0x13,0x53,0xf4,0xc7,0x9c, -0x69,0x49,0xce,0x46,0x56,0x2e,0xfb,0x8c,0x5a,0x67,0xf0,0x93, -0x39,0xb7,0x42,0xf8,0x1a,0x32,0xe5,0x62,0x8e,0x3d,0x4e,0x09, -0xa1,0x08,0x00,0x0c,0x1a,0x15,0x68,0x2a,0x14,0x0b,0xf8,0x2e, -0x92,0xb1,0x51,0x90,0x38,0xc4,0x73,0xc8,0xe7,0x01,0x1c,0xa6, -0x8b,0x5f,0x43,0x16,0xb4,0x1e,0x42,0x29,0x1f,0xc6,0x2b,0x84, -0x24,0xf4,0x7d,0x80,0x41,0x26,0xea,0x65,0x33,0x08,0x67,0x84, -0xe3,0x17,0x75,0x0d,0x72,0x3d,0xfc,0x90,0x63,0xac,0xb2,0x73, -0x08,0x3f,0x63,0x0c,0x4c,0xe6,0xe8,0xc6,0x7a,0xa6,0xb7,0xd7, -0xd6,0x10,0x1d,0xb2,0xed,0x65,0x9e,0x08,0xd6,0x60,0xbd,0xe5, -0x46,0xc8,0xec,0x1b,0x86,0xcd,0x22,0xcd,0x15,0x22,0x74,0x87, -0x0a,0x3d,0x0f,0x92,0x7f,0x50,0x0c,0xf9,0x45,0x64,0x4e,0xdc, -0xba,0x54,0xbc,0xb3,0x67,0xcb,0x53,0xfe,0x92,0xd8,0x4e,0x1f, -0x0b,0x0b,0x71,0x1d,0xa2,0x3b,0x0e,0x67,0xd2,0xaf,0x41,0xa1, -0x2e,0xb0,0x24,0xcb,0xeb,0x19,0xf3,0xab,0x73,0xcd,0xb0,0x41, -0xce,0x64,0x0c,0xe2,0x82,0x65,0x57,0x5c,0xa9,0xb2,0xc7,0x3c, -0x84,0x17,0xdc,0x22,0xd2,0x8d,0x7e,0xaa,0x86,0x7e,0x32,0x27, -0x5e,0x73,0x5d,0xe2,0xfe,0x2f,0xaf,0x38,0x5c,0xad,0x1e,0xc5, -0xf3,0xd9,0x99,0xfe,0x0b,0x5c,0x92,0xbe,0x48,0xa8,0xfa,0x79, -0x47,0x81,0x2c,0x26,0x20,0x53,0x5d,0x73,0x65,0x7a,0xa3,0x1a, -0x25,0x4b,0x45,0xd7,0x92,0xe8,0x33,0xb9,0x34,0x55,0x4d,0x04, -0x26,0x96,0x01,0x60,0xfb,0xf9,0x37,0xf0,0xe8,0x04,0xdc,0x80, -0x27,0x95,0x6b,0x77,0xae,0xe7,0xcd,0x6c,0xa7,0xdd,0x0c,0xba, -0xbe,0xce,0xa7,0x67,0x56,0x0c,0x3e,0x83,0xb7,0x7c,0x44,0x67, -0x52,0x59,0xab,0x06,0x8e,0xf6,0x2d,0x99,0xf5,0x04,0x07,0xfd, -0xed,0x95,0xda,0xda,0xb2,0x47,0x63,0x5e,0x3a,0x4c,0x00,0x2a, -0xeb,0x25,0xef,0x67,0xbb,0x3d,0x21,0xa7,0xf8,0x45,0x52,0x27, -0xd3,0xf8,0x19,0x09,0x91,0xe1,0xbe,0x0b,0x56,0xa6,0x97,0xe3, -0x03,0x5c,0xf5,0x40,0x18,0xa4,0x65,0xd1,0x69,0xe5,0xd9,0xba, -0x38,0x86,0x3f,0xed,0xf7,0x47,0x4f,0x05,0x50,0xab,0x03,0x97, -0xd0,0x21,0x7c,0x9f,0x4e,0x47,0xa0,0x24,0xc4,0xfb,0x2f,0x0d, -0x8c,0x25,0xc4,0xf1,0xdf,0xab,0x7c,0x78,0xe3,0xd2,0x7b,0x0f, -0xbf,0xfe,0x3a,0x7d,0xf8,0xf5,0x57,0x72,0x59,0xf2,0x21,0x6e, -0xc5,0x59,0x35,0x24,0xfc,0xed,0xd1,0xd6,0x16,0x34,0x1e,0x3e, -0x77,0x76,0x77,0xb5,0x40,0x6d,0x73,0x76,0x77,0xc5,0x4a,0xe4, -0xed,0x71,0x38,0x80,0xc2,0x84,0xdf,0xbb,0x5a,0xf4,0x18,0x6e, -0x6d,0x6d,0xef,0xe9,0xa2,0xfa,0xf1,0xe6,0xbc,0x6d,0x72,0xc0, -0x4a,0x8a,0x1f,0x21,0x67,0xd8,0x3f,0x78,0xf3,0xfa,0xe8,0x87, -0xc1,0x63,0x5d,0x8f,0x78,0xdd,0xef,0x74,0x7a,0xf8,0x94,0x62, -0xe2,0xb3,0x27,0x47,0xcf,0x07,0x8f,0x3b,0x0c,0x82,0x6f,0xfb, -0xb2,0x65,0x31,0xf9,0x5f,0xcf,0x9f,0xbc,0x1b,0x28,0x81,0x24, -0xe9,0x01,0x24,0x61,0xf8,0xe1,0xcd,0xfb,0x77,0xae,0x12,0x7c, -0xdb,0xdf,0x7b,0xd8,0x83,0x07,0x02,0x38,0x78,0xf9,0xfa,0x3d, -0x54,0x62,0xe9,0xa0,0xf7,0xfd,0xaf,0xbe,0xed,0xd1,0x23,0x81, -0x1d,0x3e,0x7f,0xfa,0xe6,0xf5,0x33,0x07,0xc6,0xef,0x00,0x46, -0x8f,0x06,0xdb,0xab,0x57,0x2f,0x43,0x58,0x99,0xb8,0xff,0xed, -0xb7,0x80,0xd8,0x26,0xa4,0xdb,0x1d,0x45,0x0d,0x75,0x62,0x9a, -0xe7,0xd9,0xf0,0xf3,0x9b,0x73,0xe8,0x27,0xdd,0x64,0xf0,0x6d, -0xfb,0x26,0x63,0xb1,0x30,0x4f,0xfb,0xf0,0x81,0x89,0x6c,0x52, -0x4c,0x50,0x7e,0x93,0x67,0xe6,0x41,0x0a,0x27,0x74,0x8d,0x1b, -0xc2,0x03,0xe9,0x1d,0x8a,0x16,0x37,0x5e,0x5c,0x0d,0x2f,0x4a, -0xb7,0x81,0xa5,0xd8,0x05,0x1a,0x4b,0x76,0x8b,0x2e,0x83,0xd3, -0xcd,0x8e,0x9a,0x8f,0xe7,0x65,0x4e,0x5b,0xf9,0x32,0xed,0x0f, -0xf8,0x1d,0x59,0x38,0xbc,0x9a,0x5a,0xd2,0xed,0x3d,0x75,0x76, -0xa9,0xb1,0xbe,0xd2,0xa2,0x3c,0x0c,0x90,0x74,0x17,0x99,0x08, -0x01,0x6a,0x3c,0x05,0x19,0x02,0x63,0xd7,0xa5,0x64,0x15,0x4f, -0x29,0x2f,0x90,0xef,0x60,0x55,0x65,0x3e,0x65,0x7b,0x61,0x08, -0x35,0xa1,0x93,0x96,0x72,0x04,0xff,0x03,0x32,0xe2,0xeb,0xf0, -0x40,0x52,0xef,0xf9,0x28,0x0b,0x44,0x40,0xfb,0x92,0xf1,0x19, -0x38,0x4a,0xc7,0x17,0x2c,0x1d,0x83,0x38,0xe1,0xb7,0xd2,0xe3, -0x5d,0x2d,0x76,0x52,0x12,0x7e,0xb1,0x7d,0x93,0xe4,0xda,0x44, -0xfb,0x41,0x21,0xd8,0x0b,0x66,0xdf,0x9c,0x1a,0x7c,0x08,0xf4, -0x0c,0x9c,0xba,0x9d,0xcd,0x7c,0x12,0x25,0xe9,0xbb,0x76,0xce, -0x78,0xad,0x28,0x33,0x64,0x4f,0x18,0x81,0x0c,0x7b,0x9d,0x91, -0xd4,0x88,0x18,0x56,0x6d,0x21,0x0f,0xbd,0xf5,0x33,0x38,0xa5, -0x6d,0x90,0x7e,0x40,0xcd,0x1a,0x6d,0x47,0x49,0xaa,0x41,0xc4, -0x06,0x6d,0x32,0x1c,0x01,0xb2,0xd8,0x7a,0xbb,0x29,0xfd,0x53, -0xec,0x52,0xcb,0x8e,0xa7,0x53,0xb0,0xf9,0x55,0x57,0x5c,0x29, -0xb9,0xd0,0x5b,0x2c,0x62,0x3f,0x01,0xd5,0xbd,0x86,0xb0,0x24, -0x80,0xc6,0x85,0x8b,0xf1,0x07,0x59,0x8e,0x98,0xf9,0x58,0x92, -0x03,0x01,0x7d,0x51,0x9f,0xd7,0x04,0x0e,0x1b,0x42,0x9d,0xf5, -0x0c,0xd6,0xd0,0xc2,0x59,0xc1,0x02,0xe7,0xfa,0x05,0xeb,0x50, -0x63,0x38,0xe2,0x01,0xa5,0xe1,0x0c,0xad,0x9d,0x75,0x01,0xb7, -0xc2,0x92,0x0d,0xe4,0xa6,0x40,0x3e,0xd8,0xda,0xd2,0x22,0xd5, -0x01,0x7a,0x8f,0x4f,0x48,0x8d,0xfd,0xeb,0xbc,0x98,0xea,0xa6, -0x6b,0xef,0x00,0xd8,0x4e,0xd4,0xfa,0x20,0x54,0xd4,0xd6,0x43, -0x8d,0xc0,0xb0,0xc4,0x9b,0x98,0x40,0x86,0x59,0x9a,0xfd,0xe5, -0x98,0x7c,0xbf,0xf1,0x0e,0x05,0x0d,0x84,0x80,0x08,0xc8,0xc9, -0x34,0x71,0xb1,0x30,0x19,0x42,0xb4,0x5d,0xb8,0xc0,0xc6,0x5f, -0x0d,0x96,0x61,0xc5,0x63,0xe8,0x5f,0xb6,0x3f,0xa2,0xbd,0x33, -0x7e,0x5d,0x56,0x3b,0x30,0x40,0xce,0x6e,0x63,0x7e,0x0c,0x1d, -0xaf,0x7e,0xc9,0xf0,0xc5,0x6c,0x19,0xa0,0x71,0x9a,0x0a,0xb6, -0x3a,0xe0,0x11,0x1a,0x01,0xf4,0x9f,0xde,0x18,0x8f,0x73,0x2c, -0x26,0x9c,0xe6,0xdd,0x5f,0xf6,0x77,0xbb,0x3e,0xf9,0x84,0xde, -0xf8,0x2d,0xfb,0xc5,0x1c,0x85,0xeb,0x12,0xfe,0x57,0xc1,0x59, -0x31,0x3a,0x8b,0x46,0xbc,0x6c,0x70,0x9e,0xfd,0xb2,0xb5,0xe5, -0xef,0x87,0xa8,0xe3,0xb0,0x0f,0x37,0x77,0x13,0x0d,0xb0,0xdd, -0x61,0x33,0xb2,0x5f,0xb6,0xb7,0x97,0x5a,0x30,0x59,0xd6,0x37, -0xb2,0x1b,0x21,0x14,0x5e,0x8c,0x26,0x31,0xe8,0xed,0x69,0x8e, -0xf9,0x9b,0x38,0x12,0xaf,0xdb,0x78,0x06,0x13,0xef,0x1c,0xf7, -0xe9,0x2c,0x73,0x27,0xe9,0x51,0xba,0xdd,0x3c,0xfc,0x7c,0xdc, -0x5f,0x1c,0x0f,0xee,0xed,0x5c,0x28,0x2d,0x48,0xa5,0x41,0xe6, -0xf1,0x31,0xa5,0xfb,0xc2,0x2f,0x55,0xfc,0xc2,0x97,0x31,0xc5, -0x66,0xce,0x6e,0xdc,0x58,0x2e,0x64,0x22,0xce,0x03,0x7a,0xad, -0x50,0x56,0x7b,0xab,0xdf,0xb3,0xd7,0xb7,0x97,0xb7,0x09,0x07, -0x96,0x77,0x3b,0x19,0x93,0x3b,0xe8,0xad,0xce,0x4e,0x1b,0xda, -0xcd,0x00,0x54,0xef,0xb2,0xc2,0xc6,0xfd,0x56,0xa8,0x71,0xa2, -0x48,0xee,0x24,0x1d,0x7d,0x94,0xdd,0xca,0x0c,0x26,0x61,0x84, -0x69,0xe8,0xe8,0x99,0x3c,0x2b,0xcf,0xb2,0x04,0xcb,0xf7,0xdb, -0xc5,0xcd,0x03,0xd6,0xf6,0xb8,0x46,0x27,0x40,0xa3,0xdb,0x31, -0xe5,0x4c,0x47,0x5f,0xb7,0xcd,0x5a,0x93,0xf4,0xb8,0x91,0xb3, -0xfc,0xd3,0xcd,0x70,0xcc,0x6c,0xde,0x6c,0x02,0xae,0x59,0x91, -0x9c,0x30,0x02,0xd7,0x42,0xf4,0x0e,0xac,0xaf,0x21,0x27,0x6b, -0xee,0xf6,0x46,0x6c,0x7a,0xe5,0x4b,0x52,0x53,0xa7,0x59,0x78, -0x42,0x09,0xb1,0x8e,0x4e,0x28,0x21,0x14,0xf3,0x3c,0x18,0x5f, -0x4d,0x34,0x4b,0xd4,0xa5,0x5f,0xb8,0x2e,0x0e,0x47,0x3c,0x14, -0x6c,0x5f,0x79,0x70,0x0c,0xb1,0x58,0xe0,0xef,0x92,0xbc,0x05, -0x7c,0x85,0x9d,0x88,0xf1,0xee,0xc3,0xb1,0xe0,0xbc,0x0c,0xc0, -0x7d,0x27,0x2d,0x5e,0x6f,0x6d,0xd5,0xc3,0xcd,0xf2,0xd2,0x10, -0x9c,0x74,0x13,0x6e,0x75,0x1e,0xee,0x66,0xfa,0xd4,0x96,0x54, -0x2b,0xbe,0x27,0x51,0x77,0x12,0xa5,0x8a,0xed,0xcc,0x5d,0x65, -0x43,0x40,0x6f,0xe5,0x78,0x4b,0x61,0x62,0x2f,0xf2,0x4f,0x1a, -0x0b,0x1d,0x35,0xcc,0xe0,0xaf,0xd1,0xe5,0x93,0xbd,0x87,0x31, -0x30,0xc7,0x2c,0x36,0x2e,0x7f,0xa6,0xff,0x67,0x4d,0xb6,0xd1, -0x14,0x0f,0x8b,0x1f,0x5d,0x4e,0xf3,0xfc,0x59,0x71,0xa1,0x37, -0xd9,0x64,0xe5,0xfd,0x2f,0xfd,0x3f,0x36,0x0a,0xff,0x5e,0xff, -0x8f,0x1f,0x2f,0xf4,0xff,0x6a,0x4a,0xbf,0x98,0xcc,0xa7,0x61, -0x61,0x59,0x5a,0x16,0xaf,0x2b,0x7f,0x58,0x7c,0x92,0xc5,0x0f, -0x19,0xfc,0xd0,0x3d,0x98,0xa7,0x7a,0xea,0xdf,0x8c,0xf5,0x6f, -0xe5,0x13,0x0e,0x0e,0x0e,0xb8,0xd4,0x81,0x7b,0x1c,0x8d,0xec, -0x83,0x78,0x1a,0xd5,0x20,0xfd,0x71,0x32,0x1d,0x11,0xa2,0x21, -0x03,0x3e,0xb1,0x50,0xd5,0xc5,0xdb,0x48,0xa9,0x57,0x49,0xfb, -0xe4,0x3a,0x9f,0x16,0xa3,0x22,0xbf,0xc6,0x5e,0x22,0x14,0xff, -0xac,0x6b,0x73,0x73,0x8b,0xf3,0x80,0x09,0xfe,0x89,0xeb,0xf9, -0xe9,0xa7,0x06,0xe8,0xdf,0x27,0x63,0xc6,0x77,0x54,0x07,0x61, -0x3e,0xdb,0xb6,0x15,0x3f,0x88,0xbe,0xb0,0x1d,0xc1,0x0f,0x3f, -0xfc,0xc0,0x0f,0x97,0x97,0xfc,0x70,0x7d,0xcd,0x0f,0x65,0x69, -0x5a,0xcf,0xe0,0x33,0xed,0x65,0xca,0x9a,0xa2,0xa6,0xa4,0x29, -0xc8,0xbf,0x1f,0xcd,0xaf,0x79,0xf8,0xd1,0xfc,0x9a,0x87,0x9c, -0x7f,0x9f,0xd7,0xf7,0xe9,0x9b,0xe9,0xd1,0xc7,0x09,0xf7,0x28, -0x5b,0xb1,0xb8,0xbb,0x10,0xc2,0x42,0x65,0x0a,0x13,0xe1,0xe6, -0x39,0xd9,0x4a,0x18,0xd3,0x34,0xe6,0x0c,0x38,0xf6,0x9d,0x18, -0x89,0x26,0x3b,0x7a,0x45,0x57,0x51,0x01,0x56,0xb1,0x62,0xd3, -0xcf,0xcd,0x7b,0x40,0x3a,0x5a,0xd0,0xb2,0xb3,0x11,0xb5,0x31, -0x5e,0xc3,0xe3,0xf9,0xdf,0xcf,0x2e,0xe7,0xe3,0x0f,0x59,0x5c, -0xed,0x97,0x76,0xfe,0x29,0x3f,0x33,0xc0,0x8b,0x45,0x7f,0x90, -0xc0,0x31,0x2e,0xb8,0x5b,0x28,0xb3,0x98,0xcb,0xb5,0x74,0xd5, -0xbc,0x4c,0x12,0x02,0x2e,0xfb,0x14,0x32,0xf5,0x96,0x5d,0x17, -0xd3,0xf2,0x86,0xda,0x55,0xba,0xa4,0xd1,0x81,0xb7,0xe2,0xaf, -0x77,0x1f,0x20,0x9a,0xbe,0x16,0x6b,0x5a,0xa4,0x9c,0xa0,0xf7, -0x3d,0xbb,0x96,0x45,0x2d,0xd0,0xdf,0x52,0xea,0xee,0xa0,0xb7, -0xcd,0x85,0x53,0xfe,0xf5,0x8e,0x14,0xa0,0xd2,0x23,0x72,0x48, -0x08,0x5f,0x29,0xb9,0x07,0xf9,0x55,0xf3,0x78,0x08,0x58,0x18, -0xc0,0xbe,0x6c,0x46,0x96,0xdb,0x66,0x98,0xd7,0xf2,0x16,0x37, -0xdd,0xa2,0xd4,0xa8,0x6a,0x70,0x5b,0x12,0x7b,0x48,0x78,0x4b, -0xcc,0x5a,0x55,0x62,0xdd,0x5a,0x38,0xea,0x92,0x74,0x54,0x3b, -0x75,0x87,0xd9,0xca,0x19,0x47,0xda,0x76,0x9c,0x6e,0x8c,0x8f, -0x55,0x45,0xc3,0x5e,0x6d,0xcd,0xc3,0xd4,0x6d,0x2a,0xbd,0x3d, -0x15,0x11,0x2c,0x29,0x79,0x26,0x26,0xd3,0xaa,0x6f,0x42,0x45, -0x80,0xf7,0x49,0xde,0x07,0x3d,0x7b,0xe6,0x10,0x55,0x51,0x19, -0x6a,0x46,0x66,0x9b,0xdd,0x8c,0x08,0xe6,0xb2,0x5f,0x33,0xea, -0x1a,0xbc,0x02,0xad,0x58,0xbe,0xed,0x7f,0xfd,0xf7,0x5e,0xe7, -0xdb,0xdd,0xdd,0x74,0x2f,0x7f,0x18,0xa0,0xb2,0x8c,0x81,0x19, -0xf6,0x5d,0x88,0x65,0x69,0xc1,0x14,0x0d,0xf9,0x45,0xf9,0xf6, -0x7a,0x75,0x47,0x69,0x88,0x83,0x1a,0x5c,0x3f,0x54,0xb8,0x91, -0xe0,0x4a,0x3e,0x51,0xa8,0x2e,0x69,0x24,0xea,0x5a,0x30,0xb1, -0xa0,0xe7,0x49,0x87,0xd2,0x58,0xb2,0x14,0x5c,0xcf,0x2f,0xc9, -0x5a,0x92,0xc6,0x92,0xf5,0xab,0x55,0x58,0xbb,0x53,0xb7,0x30, -0x22,0x38,0xe2,0x8d,0xa3,0xdd,0x36,0x58,0x6b,0x55,0x7a,0xf9, -0x9f,0xae,0x51,0x47,0xf6,0xda,0x3f,0x16,0x11,0xd6,0xf3,0x35, -0xe5,0xe4,0x0a,0x62,0x10,0xe8,0xed,0xf9,0xfb,0xa3,0xa7,0x60, -0x54,0x61,0x52,0x66,0xbf,0x5f,0x67,0xcd,0xfc,0xae,0xfa,0x85, -0x77,0x33,0xf2,0xd1,0x5d,0xab,0x6b,0xc8,0xee,0x91,0x6d,0x64, -0xc4,0x9c,0xcb,0xf9,0xa9,0x66,0x9b,0x7a,0xcf,0xd5,0xe1,0x6b, -0x6d,0x76,0x75,0x22,0x71,0xc1,0x5f,0xba,0x84,0x38,0xc1,0xa2, -0x45,0x2d,0x2e,0x88,0x42,0x10,0xcc,0xae,0x8f,0x96,0x7b,0x7d, -0xc4,0xb8,0xfd,0xf6,0xad,0x8f,0x65,0xf9,0x98,0x40,0x2e,0x0a, -0xd0,0x89,0xd0,0x32,0x7e,0x58,0x4c,0xde,0x02,0x41,0xa6,0x3a, -0x9b,0x4f,0x21,0x94,0x3d,0xfa,0xf0,0x00,0xbd,0xe4,0xd1,0xe4, -0x7d,0x99,0xab,0xf3,0xe2,0x13,0xcc,0x62,0xf5,0x11,0xcd,0xa8, -0x68,0x37,0x0f,0x07,0x03,0x7a,0x7e,0xe3,0x2f,0x91,0xc6,0x5a, -0xf7,0x4d,0xdb,0xd3,0xb4,0x27,0x16,0x28,0x33,0xf1,0x4c,0xbb, -0xee,0x33,0x2f,0x8c,0xa8,0xa6,0x5d,0x6f,0x43,0x51,0xe1,0x64, -0x19,0x33,0x71,0xa2,0x6a,0x32,0xb1,0x3e,0xdd,0x1c,0x4c,0x9c, -0x67,0xfd,0x2f,0xb5,0x26,0x60,0xb8,0x6c,0x0e,0xae,0x1f,0xf6, -0x70,0xbc,0xc1,0x78,0xd5,0xa9,0xaa,0xb3,0x8b,0xdc,0x24,0xea, -0x7c,0x1b,0xb5,0xf4,0x7b,0x1a,0xed,0xed,0xd2,0x03,0xfc,0x0b, -0x2b,0x44,0xf6,0xd1,0xb3,0x27,0x00,0xf0,0xdd,0xa8,0xe9,0x4d, -0xd2,0x00,0x66,0xa9,0x44,0xcf,0x30,0xe3,0xfe,0xd8,0xfe,0xfe, -0x7b,0xba,0xba,0x09,0xcf,0x3f,0xba,0xc7,0xe7,0x3d,0x68,0xd2, -0xcc,0x72,0x4a,0xe8,0x9e,0x1f,0xc1,0xee,0xc1,0x7c,0x57,0x0c, -0x45,0x13,0x85,0x85,0x3a,0xfa,0xe7,0xb9,0x7a,0xa4,0x07,0x56, -0x6a,0xf7,0xf9,0x8d,0xcc,0xc9,0xf4,0x50,0x66,0xaa,0x1a,0xd1, -0xc7,0xff,0x48,0xc9,0x1a,0xef,0x88,0x36,0x2d,0xa9,0x01,0xc8, -0x5d,0xeb,0xe8,0x17,0x68,0x9d,0x16,0x6e,0x4e,0x4e,0x00,0x53, -0x7b,0x34,0xf9,0x98,0x92,0x1d,0x4b,0x26,0xc0,0x3e,0x02,0x18, -0x50,0x66,0x6b,0xd9,0xda,0xe2,0x8a,0x1f,0xfb,0x85,0x29,0xbd, -0xd5,0x52,0x77,0x7f,0xf0,0xc5,0x05,0x51,0x6f,0x07,0x99,0x87, -0x29,0x78,0xfd,0xe8,0x82,0xeb,0x9b,0x1e,0xc8,0xa0,0x0a,0x34, -0xcd,0x51,0x35,0x4b,0x11,0xe4,0xd9,0xd7,0xa4,0x0a,0xa1,0xc7, -0x94,0x1d,0xfa,0x59,0xfd,0x28,0x10,0xe3,0x98,0x52,0xc2,0x31, -0x50,0xc5,0xba,0x1f,0x1c,0x2b,0x20,0xfa,0x44,0x4c,0xe7,0x5a, -0xfd,0x34,0xc4,0x99,0xc2,0x99,0x89,0x7b,0x59,0x73,0x58,0xe1, -0x66,0xe6,0x6e,0xb5,0x22,0xd9,0x1a,0x2c,0x1f,0x00,0x02,0x3e, -0xec,0x20,0xa5,0xbe,0x84,0xa1,0x95,0x5e,0x80,0xd0,0x6e,0x97, -0x2e,0xba,0x3c,0xdc,0x2f,0xaa,0x53,0x4f,0xef,0xf9,0x5b,0xad, -0x22,0x91,0x09,0xc4,0x6b,0xe0,0x41,0x36,0x4d,0x41,0x6e,0x05, -0xba,0xdf,0xb0,0xc2,0xa1,0xbe,0x44,0x15,0x7d,0x0f,0xce,0x36, -0x8a,0x5e,0x27,0xdd,0x4d,0xbd,0x5a,0xa9,0x08,0x2e,0x99,0x2d, -0x5e,0x76,0x62,0xb1,0x08,0x2c,0x16,0xbb,0xc9,0xce,0xd7,0xbb, -0xcc,0x29,0xcd,0x12,0xd9,0x04,0xf9,0x37,0x80,0x74,0xab,0x52, -0xec,0xaf,0x2f,0x3d,0xd1,0xde,0x29,0x3c,0xc3,0x43,0x22,0x9d, -0x3e,0x35,0x72,0x57,0xbe,0x9e,0x27,0xb9,0x6b,0x70,0x2a,0xdb, -0xb5,0xd5,0x2e,0x16,0xe1,0x61,0x6f,0xbd,0x19,0x9a,0x15,0x49, -0x44,0xcf,0x65,0xfd,0x3a,0x63,0xb4,0xf0,0xfc,0x9a,0x84,0xc9, -0x4a,0x2a,0xcc,0xaa,0x3a,0x9b,0xb2,0x6a,0x71,0x5c,0x50,0x2b, -0xc9,0x64,0x4e,0x52,0x03,0x6d,0x4d,0x4d,0x06,0xaa,0x7e,0xb9, -0xf1,0x02,0xb5,0xd7,0x2f,0x07,0x55,0xf7,0x4f,0xce,0x5b,0x93, -0xec,0x23,0xdd,0x00,0x1f,0x79,0xd0,0xbe,0xd0,0x5d,0x42,0xec, -0x58,0xb9,0x44,0x33,0xd8,0x5d,0x0a,0x8d,0xed,0x41,0x6a,0x4a, -0x56,0x8b,0x05,0x65,0xb8,0x80,0xaf,0x4a,0x7c,0xc6,0x1f,0x46, -0x12,0xc6,0x13,0xab,0x0a,0x0a,0x8f,0x78,0x86,0xe0,0xe9,0x57, -0xcc,0x6f,0x3c,0xfc,0x80,0x0b,0xca,0xb4,0xe2,0x22,0x1b,0xa5, -0x86,0x53,0x33,0x52,0xb3,0xd0,0xbe,0xa5,0xfc,0x50,0xa0,0x05, -0xea,0xdd,0x3c,0x9e,0xf6,0x67,0x59,0x14,0xb5,0xec,0x08,0xe1, -0x34,0xb2,0xce,0xe2,0x5b,0x49,0x46,0xcb,0x3c,0x9b,0xcc,0x86, -0x57,0x6f,0x5d,0xb5,0x0c,0x44,0x96,0x26,0x44,0x82,0xaf,0x84, -0x33,0x58,0xcf,0x69,0xa9,0x68,0xd2,0x89,0xc2,0xfe,0x10,0xf9, -0x45,0xf1,0x78,0x26,0x4f,0x4e,0x70,0xd6,0x0b,0x79,0x06,0x1d, -0xa8,0x88,0xcf,0xce,0xe2,0x35,0x94,0x3f,0x8d,0x3b,0x51,0x83, -0x05,0x4c,0xfe,0xa9,0xc5,0xcc,0xe7,0x5a,0xb1,0x89,0xdf,0x8d, -0x5d,0x82,0x28,0xa4,0x19,0x1c,0x17,0x32,0x3a,0xef,0x5d,0x50, -0x79,0xdb,0xbe,0x12,0x27,0x70,0x74,0xd4,0xce,0xd0,0xb6,0xcd, -0x4d,0x55,0xa8,0x60,0x5f,0x51,0x4f,0x4b,0xbc,0x58,0x0b,0x9f, -0xfa,0x8e,0x68,0x65,0x75,0xb0,0xb5,0xea,0x61,0x12,0xf1,0x7a, -0xb2,0xa6,0x5e,0x75,0xa4,0x75,0xd2,0xca,0x17,0xb1,0x72,0x0e, -0x3f,0x89,0x36,0xbb,0x6a,0xf5,0x26,0x5a,0x8e,0x52,0x33,0x85, -0x2d,0x5a,0x73,0xe0,0xb6,0xe9,0x75,0xc7,0x1a,0x95,0x76,0x05, -0x8c,0x77,0x16,0x97,0xc9,0xe1,0xbb,0x5d,0xdf,0x4e,0xca,0x1b, -0xd5,0x77,0x76,0x1c,0xdb,0x62,0xca,0x6d,0x9d,0x2b,0x61,0xce, -0xaf,0x3b,0x7b,0x62,0x2d,0x1e,0x9a,0x45,0xa6,0xb3,0xe7,0x97, -0xc3,0x3b,0xb7,0x5b,0x5b,0x1d,0x58,0xa0,0x02,0xe0,0x6a,0xf1, -0x8c,0xd7,0xee,0x2a,0x0f,0x54,0xb5,0x56,0xcc,0xf2,0xec,0xcd, -0x53,0xfa,0xb8,0x53,0xc0,0x52,0x9e,0x72,0xc4,0xc7,0xfd,0x64, -0x01,0x3f,0x03,0xfd,0xd3,0x8f,0xfb,0x3f,0x1f,0x0f,0x8e,0xfb, -0x83,0x07,0xc9,0xf1,0x00,0x52,0xdb,0xc9,0xce,0x45,0x70,0xa3, -0x47,0xb3,0x94,0x9b,0x8e,0xba,0xd9,0x53,0x37,0x0f,0xd5,0xcd, -0x23,0x8b,0xf4,0xa6,0xb3,0x58,0xdc,0xec,0xe9,0x7f,0x0f,0xf5, -0xbf,0x47,0xcb,0x44,0x1e,0xd9,0x08,0x4d,0x54,0x2d,0x15,0xfd, -0xed,0x63,0x4d,0xc9,0xcf,0xf7,0x1e,0xb4,0x7a,0xed,0x38,0x59, -0xf4,0x8f,0x07,0xb7,0xcb,0x01,0x1c,0xbf,0x1c,0x1f,0xdf,0xdb, -0x0a,0x8f,0x60,0x2a,0x7c,0xb3,0xca,0xf5,0x41,0x3a,0xa3,0xb6, -0x52,0xa7,0x79,0xc9,0xa7,0x07,0x0a,0x2f,0xfc,0x1e,0x4d,0xbe, -0xcb,0xc1,0x2a,0xc8,0x6c,0x52,0x0e,0x21,0xd1,0x98,0x2d,0x59, -0x46,0x65,0xa6,0x4d,0xb0,0x62,0x84,0xa7,0xc8,0x72,0xff,0x28, -0xb6,0xa4,0xbb,0x3b,0xba,0xcf,0xf8,0x9a,0xa4,0x39,0x33,0x29, -0x1e,0x87,0xc8,0x49,0x98,0x11,0x54,0x64,0xbb,0xca,0x11,0x9e, -0x39,0x0f,0x32,0xa6,0xbb,0x1b,0x4c,0x02,0x5c,0x99,0x44,0x94, -0xd7,0xf5,0xb8,0xcf,0x01,0x86,0xb9,0x6a,0xc9,0x91,0x28,0xcc, -0x81,0x8a,0x48,0x83,0x41,0x29,0xe8,0x6c,0x65,0xb2,0x9a,0x70, -0xea,0x29,0x1f,0xb4,0xb3,0xfb,0x20,0x80,0xae,0x39,0x1b,0x57, -0x01,0x08,0x85,0xc1,0x97,0x88,0x14,0x5f,0x45,0x15,0x7d,0xb8, -0x58,0x88,0x97,0x7d,0x09,0x8c,0x3e,0x31,0x5c,0x9e,0x8f,0xc8, -0x0d,0x08,0xf1,0x19,0x49,0xd2,0xe5,0xf6,0x3e,0x0b,0x87,0xcd, -0x62,0x21,0xc0,0x56,0x8d,0xc4,0x60,0x27,0xcc,0x0c,0xde,0xad, -0xa8,0x38,0x79,0xb2,0xa2,0x9c,0xe0,0x2a,0xe5,0xad,0x47,0x30, -0xfe,0xf8,0xb6,0x1c,0x8d,0x97,0x47,0xdd,0x62,0x9f,0x9c,0xce, -0x99,0x1c,0x38,0xa3,0xb3,0x22,0xc1,0x39,0xa0,0x81,0xda,0xf5, -0x52,0xb8,0xdd,0x19,0xb4,0x18,0xe4,0xeb,0xc1,0x62,0x11,0x6d, -0x44,0xac,0xa6,0x58,0x9a,0xa1,0xf7,0x48,0x44,0xec,0x2e,0x27, -0xa8,0x60,0x27,0x4f,0xe5,0x1e,0x0d,0x02,0x7b,0x2b,0x93,0x70, -0xbb,0x03,0xc6,0x77,0x87,0x7a,0x57,0xb0,0x2e,0x8d,0x21,0xda, -0xf8,0x29,0x4a,0x56,0x0e,0x3a,0xc3,0xaf,0x30,0x14,0x79,0xcd, -0x2c,0x62,0xb4,0xf5,0x4d,0x8e,0x9c,0xd9,0x6f,0x71,0x14,0x05, -0x82,0xe6,0xd6,0x8b,0xf9,0xb0,0xbc,0x79,0x9d,0xcf,0xfe,0xab, -0x9c,0x8c,0x45,0xbb,0xb3,0x3a,0x87,0xca,0x64,0xce,0x6a,0xaf, -0x57,0xa5,0x23,0x65,0x44,0x35,0x59,0x71,0x8b,0xf3,0x40,0x09, -0x9d,0x46,0x44,0xb0,0x33,0x14,0x44,0xec,0xbd,0xe6,0x81,0x92, -0xfa,0x76,0xcd,0x3d,0xc7,0xf9,0x69,0x5b,0x63,0x8e,0xdd,0x9b, -0xf8,0x3f,0x20,0x10,0x16,0xcc,0xb5,0x04,0x72,0x84,0x10,0xe3, -0x12,0x28,0xa0,0xad,0x7e,0xb3,0x51,0xa7,0x65,0x0b,0xcd,0x9c, -0xcd,0x57,0xc5,0x9f,0xd5,0xb5,0x1a,0xa9,0x4b,0x75,0xa0,0x4a, -0x75,0xcd,0x96,0xbb,0xb8,0xd5,0xb4,0x45,0x03,0x08,0x23,0x85, -0x77,0xbe,0xfd,0x66,0x77,0xff,0xf3,0xd6,0x16,0xee,0x19,0x4b, -0x21,0x44,0x7f,0xa6,0xef,0xf5,0xeb,0xb2,0x9b,0xd6,0xba,0x1a, -0x8c,0x6d,0x5e,0xe0,0x4d,0xd7,0xdc,0xbb,0x6a,0xaa,0x51,0x0a, -0xfc,0x95,0x4a,0x3d,0x2d,0x07,0x1d,0x01,0x18,0xab,0x10,0xf2, -0xb4,0x5f,0xdb,0xdb,0x34,0xc7,0xc0,0x98,0x89,0xf5,0x90,0x00, -0x4a,0xa3,0xcc,0x94,0x6e,0xb3,0x3a,0xc2,0xd7,0xcc,0x57,0x1d, -0xf7,0x63,0xba,0x30,0x9f,0xa2,0x79,0x42,0xb8,0xac,0xea,0x84, -0x08,0xeb,0xec,0xda,0x6f,0x2c,0x7c,0x7b,0x5d,0x14,0x61,0x8b, -0x99,0xde,0x7b,0xc1,0x94,0x7d,0x72,0x31,0xe1,0x59,0xa5,0xf8, -0xe2,0x06,0xc7,0x54,0x39,0x9c,0x9f,0x9f,0x17,0x9f,0x30,0x42, -0xa9,0xc6,0x92,0xf3,0x31,0xb4,0x3c,0x5a,0x9e,0xe6,0x57,0xe8, -0x9e,0x06,0x6d,0xb3,0xa8,0x30,0xe8,0x6b,0x36,0x37,0x7d,0x0c, -0x8c,0xdd,0x20,0xf2,0x84,0x00,0x81,0xc0,0xbb,0x50,0xe1,0x63, -0xa0,0xaa,0xc9,0x49,0x38,0xdd,0xc3,0x98,0xe2,0xe5,0x15,0x6b, -0xf3,0x2e,0xcb,0x26,0x3b,0x9d,0xfc,0x61,0x62,0x4f,0x85,0x08, -0x92,0xf3,0x70,0x0f,0x4f,0xd7,0x23,0x28,0x9d,0xa1,0x30,0x1d, -0x2f,0x71,0x50,0x32,0x82,0xec,0xec,0x3d,0x32,0x77,0xb3,0x28, -0x15,0x00,0x76,0x1e,0x7e,0xfd,0x55,0x42,0xfe,0x69,0x1f,0x7d, -0xb5,0x6f,0xaf,0x3e,0xf4,0xa3,0x32,0x32,0xf7,0xa1,0xc0,0xaf, -0x3d,0x18,0x7d,0x11,0x6a,0xc8,0xbb,0x8e,0x74,0x9a,0x06,0x97, -0x49,0xd7,0x91,0x21,0x91,0xe1,0xb1,0x4e,0xc8,0xba,0x04,0xe8, -0xbd,0xbd,0x7d,0x97,0x70,0x19,0x11,0xd1,0x0c,0x49,0xb7,0x39, -0xfa,0xd1,0x08,0x01,0xbf,0xda,0x77,0x09,0xa3,0x08,0xbf,0x82, -0x6a,0xb3,0xc9,0x07,0x00,0xf7,0x50,0xa7,0xd8,0x84,0x83,0x48, -0xc9,0x4f,0xda,0x4d,0x18,0x33,0x7e,0x2d,0x40,0x7c,0x86,0x22, -0xfa,0xe7,0x73,0x44,0x2d,0xe0,0xfb,0x31,0xdd,0x1b,0x64,0x7e, -0x07,0x61,0xea,0xc3,0x41,0x26,0xfb,0x61,0x7f,0x97,0x92,0x1f, -0x0d,0x70,0x98,0xab,0xca,0xb0,0xe3,0x99,0xa9,0x85,0x19,0x72, -0xdc,0x6a,0x87,0x05,0x4c,0x06,0xd2,0x38,0x91,0x59,0x08,0x04, -0xcc,0x42,0xbd,0x15,0x4c,0xbd,0xe0,0x95,0x35,0x53,0x74,0x04, -0x37,0xfa,0x65,0x5e,0xce,0xf2,0x11,0x8b,0x76,0xe8,0x22,0xb2, -0x06,0x78,0x3b,0x40,0x08,0x6d,0x70,0x34,0xb1,0xef,0xf5,0x65, -0xe8,0xca,0xcc,0xe7,0xd8,0x4e,0xaa,0xa0,0xd4,0x7e,0x0e,0xce, -0xfe,0xe3,0x20,0x75,0x3b,0xfb,0x26,0x01,0x3a,0xb6,0xbf,0xd9, -0x0f,0x72,0xaa,0xb0,0x2d,0x80,0xf5,0x3f,0xc1,0x58,0x85,0x83, -0xd9,0x4d,0x5b,0xef,0xa5,0xe2,0x88,0x3b,0x58,0x14,0x4b,0xd4, -0x2d,0xb4,0x97,0x30,0x6e,0xf6,0x71,0x38,0x85,0x64,0x9c,0xec, -0x7c,0x43,0x03,0x3a,0x0d,0x40,0x3e,0x63,0xae,0xa7,0x70,0xaa, -0xe8,0x50,0x51,0x03,0xe4,0xa9,0x4e,0xeb,0xda,0x29,0x48,0x34, -0x8c,0x19,0x28,0x7e,0x32,0x1a,0x29,0x8b,0x57,0x8d,0xaa,0x9c, -0x9a,0xaa,0x90,0xd6,0xd3,0xa2,0xb9,0x03,0x95,0x33,0xbd,0xf5, -0xf8,0x37,0xad,0xed,0x51,0x5d,0x61,0xd0,0x97,0xdb,0xa3,0x56, -0x3c,0xda,0xaf,0xa3,0xbb,0xf7,0x4d,0x4a,0x1c,0x80,0x95,0xa3, -0xdf,0x3c,0x88,0x01,0xf7,0x76,0x27,0x69,0xc5,0x5c,0x49,0x30, -0x6c,0xe8,0xea,0x14,0xd6,0xd2,0xea,0xa8,0x5b,0x6c,0x56,0xa7, -0x92,0xdd,0xed,0x61,0x02,0xfc,0xd9,0xee,0x38,0xbc,0x1e,0x84, -0x97,0x2a,0x15,0xb8,0x50,0xab,0xcd,0x5c,0xfa,0x6b,0x5e,0x70, -0x67,0xb5,0x4e,0xc8,0x61,0xe3,0x1b,0x2b,0x77,0x99,0x93,0x6f, -0x2b,0xd0,0xb8,0xf5,0xc9,0xed,0x61,0xb6,0xb6,0x56,0x1b,0x17, -0x73,0x40,0xb4,0x8c,0xe5,0x17,0x1a,0x37,0xbc,0xef,0x89,0x6f, -0x85,0xa5,0xf0,0xee,0x32,0x49,0x1b,0x16,0x42,0x21,0x0c,0x16, -0x84,0xa8,0x46,0x05,0x95,0xb4,0x6f,0xf4,0x16,0x50,0x2c,0x83, -0x78,0x09,0x0c,0x6b,0x2b,0xf9,0xe3,0x7d,0xd1,0x48,0x6c,0x8a, -0x78,0xdd,0x6c,0x94,0x79,0xe0,0xe4,0x28,0x49,0xa9,0x7d,0x7a, -0x46,0xd0,0x62,0xa3,0xa5,0x1a,0x99,0x2c,0xdc,0x46,0xa6,0x6b, -0x48,0xae,0x69,0xa3,0x40,0xaa,0x80,0x18,0xbf,0xfb,0x02,0xd9, -0xe9,0x7b,0xbc,0x72,0xa1,0x51,0x1e,0xd2,0xdd,0x0b,0xb8,0x6a, -0x4e,0xee,0xae,0xac,0xe5,0x23,0x5f,0x3f,0xf7,0xdf,0x5b,0x7a, -0xf1,0x11,0x77,0x5c,0xd8,0x08,0x0c,0x86,0xc6,0x7c,0x76,0xc6, -0xb7,0xf1,0x8b,0x12,0xd4,0x9a,0x91,0xfe,0x13,0xa5,0xce,0x7f, -0x8c,0x38,0x5c,0xef,0xf1,0x1d,0xf5,0x91,0x5e,0xca,0xf2,0x59, -0xd4,0xd2,0x45,0x5b,0x60,0x91,0x6a,0x84,0x91,0xba,0x7b,0x7c, -0x74,0x8d,0x1c,0xff,0xa6,0xb6,0xf4,0x85,0x57,0x3a,0x09,0xc6, -0xaf,0xb9,0x9c,0xf9,0xbd,0xfb,0x44,0xfb,0x79,0xf6,0xaa,0xa4, -0xfd,0xce,0xa6,0x0b,0xed,0x70,0x2b,0x97,0x40,0x1a,0xd0,0x3f, -0x29,0x45,0x05,0xea,0x7c,0x78,0x86,0xce,0x54,0x6b,0xea,0x89, -0x86,0x55,0x9f,0x74,0xa6,0xb2,0x16,0x54,0xb6,0x43,0x85,0x83, -0x7a,0xbe,0x47,0x17,0x9d,0x1a,0xb2,0x6e,0x56,0xa1,0xeb,0x09, -0x3d,0xd8,0x91,0xd6,0x6b,0xc9,0xc9,0x93,0xa5,0xb9,0x07,0x4b, -0xef,0xaa,0x50,0xff,0x78,0xfe,0xee,0xf0,0xe5,0x9b,0xd7,0x59, -0xb4,0xd7,0x7e,0xd8,0xee,0xf0,0xba,0x4c,0xb7,0xf9,0xf0,0x51, -0xc1,0xa9,0x4f,0xb6,0xab,0xf0,0xc0,0x25,0xeb,0x28,0x38,0x54, -0xc9,0xf6,0x14,0xa8,0x81,0xb2,0x87,0x8a,0x4e,0x22,0xb2,0x47, -0x8a,0x8e,0xcc,0xb3,0xaf,0x94,0x38,0x40,0xcf,0xbe,0x76,0x26, -0xd8,0x70,0x87,0xd9,0x5a,0x3d,0x67,0x82,0x6a,0x2b,0x5c,0x5e, -0x63,0x16,0x5c,0xd8,0x84,0x5f,0xbd,0x25,0xba,0x99,0x4c,0x67, -0xa5,0x0a,0xb6,0x4a,0xd9,0xce,0xcf,0xc7,0x3b,0x3d,0x18,0xe0, -0xc7,0x71,0x7c,0xbc,0xdd,0x3b,0x1e,0xb5,0x92,0x9d,0x82,0xa1, -0x60,0x61,0x3f,0xbc,0x19,0x8e,0x05,0xb4,0x86,0xd1,0x33,0xb5, -0x97,0xc6,0xc7,0xa3,0x07,0xc9,0x71,0x5b,0x3f,0x43,0x89,0xe3, -0x14,0x7f,0x74,0x3a,0x3f,0x1d,0xb7,0x21,0xe3,0xf6,0xe1,0x32, -0xe9,0x25,0xbd,0x1d,0x05,0x3b,0x59,0xee,0x24,0x53,0x6b,0xac, -0xf1,0xbc,0x05,0x44,0xfa,0xbf,0xfe,0xee,0xf6,0xb7,0xaa,0x3d, -0x78,0x90,0xfc,0x8b,0x70,0xdb,0xf7,0x83,0xe0,0xfd,0x19,0xbe, -0x1f,0x79,0x69,0x3f,0xdc,0x51,0xe6,0x10,0x48,0x58,0xb8,0xf7, -0x1f,0x93,0x7b,0x3b,0x2a,0xd4,0x4b,0xc3,0x87,0xf5,0xfb,0x3f, -0x83,0x52,0x0c,0x74,0x64,0xf1,0xf1,0xb1,0xc6,0x72,0x30,0x59, -0x1c,0x1c,0xf4,0xe0,0xff,0x8b,0x67,0x93,0xc5,0xb3,0x67,0xf8, -0xa7,0x07,0xff,0x5f,0x8c,0x46,0xa3,0xde,0x48,0xff,0x4c,0x7a, -0x8b,0x8f,0xfd,0xc9,0xe2,0xe3,0xa0,0xb7,0xf8,0x51,0xff,0xfe, -0xa8,0x7f,0xd1,0xa2,0x64,0xc1,0x7f,0x16,0x17,0x17,0xf1,0xc5, -0xc5,0x05,0x50,0xf0,0xfd,0xf7,0xf1,0xf7,0xdf,0x7f,0x0f,0x4f, -0xf9,0xe2,0xf9,0x62,0xb8,0x78,0xb2,0xb8,0xbc,0xec,0x2d,0x7e, -0xf8,0xa1,0xb7,0xb8,0xbe,0xee,0x2d,0xca,0xb2,0xb7,0x38,0x3c, -0xec,0x1d,0xf6,0x16,0xff,0x5c,0xfc,0xfe,0x7b,0x6f,0xf1,0xd3, -0x4f,0xbd,0x05,0xea,0xe5,0x6a,0x2d,0x26,0x6b,0xc9,0x7d,0x75, -0xb4,0x78,0xf5,0xaa,0x07,0xff,0x5f,0x5c,0xdd,0x76,0xd4,0xa3, -0x25,0x14,0x6f,0x32,0x2b,0xcb,0x76,0x8e,0x47,0xc7,0x23,0xdd, -0x35,0xcd,0xb6,0x84,0x00,0xa2,0xf1,0x3c,0x5c,0x4a,0xa0,0x30, -0xdf,0xcf,0x75,0xe6,0x90,0x5c,0xf8,0x91,0x97,0x6d,0xad,0x1d, -0xb3,0x9d,0x7e,0xeb,0x78,0x7b,0xd0,0x43,0x98,0xaf,0x3d,0x18, -0x30,0x3e,0xd4,0xd9,0xba,0xb7,0x06,0x0f,0xfa,0xf7,0x87,0xdb, -0xbf,0x1f,0xcf,0x77,0x77,0x9f,0xec,0x6e,0xeb,0x9f,0xaf,0x5e, -0xbc,0xd0,0x7f,0xbf,0xd9,0x85,0x97,0x67,0xdf,0xc0,0xcb,0x8b, -0x6f,0xf1,0xe5,0xc5,0xb3,0xa7,0xf0,0xf2,0xec,0x05,0xbe,0xbc, -0x78,0xfe,0x62,0xd0,0x5a,0xf4,0x35,0xe8,0xd7,0x98,0xbb,0xfb, -0xb5,0x06,0xdd,0x19,0xb4,0xe2,0xe3,0xf2,0x41,0xcf,0x4f,0x1e, -0xb4,0x12,0x4d,0xc0,0xde,0x72,0xa7,0x50,0x55,0xc5,0x47,0xb6, -0xf3,0x93,0xc6,0x02,0x74,0x42,0x53,0xa5,0x3d,0xf8,0xeb,0x03, -0x66,0x3b,0x47,0x95,0x92,0x6c,0xd7,0xa8,0x3f,0xe1,0x98,0x3f, -0x51,0x57,0xdc,0xe6,0x96,0xe4,0xb9,0x60,0x67,0x5e,0xf9,0x40, -0x67,0x3c,0x5a,0x6e,0xd3,0x84,0x3a,0x1e,0x6d,0xc3,0x1f,0xdd, -0x9f,0x3f,0xc2,0xef,0x3d,0xf3,0xb0,0x8d,0x69,0xf0,0xa4,0x1f, -0x92,0x38,0x3e,0x5a,0x6c,0x24,0xf8,0x1a,0xa7,0xe2,0x2f,0x54, -0xa2,0x7b,0x54,0xff,0x97,0xf0,0xff,0xe3,0x90,0x7a,0x48,0xbc, -0x87,0x14,0xb0,0x6a,0x13,0xcd,0x9f,0xb6,0x0f,0x0e,0xb6,0x9f, -0x3d,0x3b,0xfa,0xe1,0x87,0xf4,0xfa,0x3a,0x2d,0xcb,0x9f,0x22, -0x65,0x14,0x4f,0x59,0x5f,0x00,0x44,0x0a,0xcd,0x4e,0xb6,0xfb, -0x3f,0x0e,0x7e,0xfc,0xd1,0x7b,0xd9,0x7e,0xae,0x5f,0x11,0x10, -0x2c,0xbc,0x06,0xca,0x28,0x96,0xb2,0x7e,0x3f,0x32,0x58,0xdb, -0x87,0x91,0xda,0x41,0xca,0x91,0x1c,0xfb,0xc7,0x36,0xcd,0xce, -0x40,0x39,0xe8,0x7a,0x58,0x07,0x52,0xc9,0xe7,0x2c,0x99,0xbe, -0x33,0xe0,0xa3,0x9f,0xc0,0xd6,0x50,0xcf,0x1d,0x6a,0x97,0x05, -0x36,0xc9,0xce,0x45,0x01,0xb7,0x79,0x3f,0x7d,0xa6,0x05,0xa6, -0xb4,0xcb,0x75,0x99,0xa1,0x07,0xd4,0xc5,0x0f,0xb0,0x53,0x5b, -0xb0,0xb5,0xd0,0xe2,0x90,0x36,0x43,0xfa,0xdd,0xed,0x8c,0x22, -0x73,0xc7,0x62,0x81,0x9e,0x07,0x8b,0x99,0xc8,0x7c,0x81,0xab, -0x8d,0x66,0xd7,0xb2,0x40,0xda,0x51,0x87,0xe6,0x29,0xd7,0x4c, -0x9f,0x6d,0x16,0xbf,0xce,0x1f,0x29,0xac,0x2d,0x05,0x6f,0x17, -0x4a,0x8b,0xa6,0x65,0xfa,0xf7,0xaf,0x1f,0xe9,0x47,0x3c,0x6a, -0x2c,0xd3,0xbd,0xaf,0xbe,0xdd,0xcb,0xbf,0x56,0x20,0x46,0x6a, -0x98,0xce,0x57,0x1a,0xec,0xeb,0xa5,0x12,0x77,0xa1,0xb3,0xdb, -0xeb,0x32,0x8d,0xc4,0xae,0x4d,0x6f,0x62,0xd3,0xc8,0x3c,0x5e, -0x43,0x16,0x54,0xa5,0xb7,0x9f,0x69,0x04,0x3b,0x50,0xbd,0xfb, -0x48,0xf1,0x86,0xae,0x7a,0x06,0xbf,0x90,0xf3,0x31,0xa5,0x9b, -0xbc,0xea,0xc7,0x34,0xd2,0x1d,0xf9,0x23,0x3e,0x1f,0xa4,0x7c, -0x0f,0x57,0x7d,0x4e,0x23,0x10,0x69,0x75,0x81,0x67,0x58,0x84, -0xa5,0xda,0x48,0xe5,0x54,0x0e,0x91,0x3d,0xb7,0x45,0xf1,0xf5, -0xe2,0x82,0xf2,0x08,0xf0,0xfb,0xef,0x6d,0x2e,0x26,0x40,0x7c, -0x15,0x79,0x71,0x3b,0xbb,0xd5,0x85,0x26,0xe7,0x28,0x6f,0xcb, -0x0a,0x74,0x19,0xb3,0x3d,0xf0,0xb0,0x73,0xba,0xa0,0x16,0x5e, -0xa9,0xb8,0xab,0x95,0xa1,0x28,0xd9,0xaf,0x3e,0xb0,0xef,0x87, -0x95,0x35,0xf0,0x66,0x03,0x43,0xe1,0xd9,0xb3,0x8d,0x8f,0x1b, -0x3f,0x6e,0x1c,0x6c,0x3c,0xdb,0x18,0xd9,0x0e,0xdf,0xd0,0x1d, -0x7e,0xa3,0x37,0x72,0x46,0xd5,0x9d,0x45,0x90,0xff,0xc3,0xc6, -0xe5,0xc6,0xf5,0x46,0x09,0xf0,0x1e,0x64,0xdd,0xf1,0x9c,0x1e, -0x18,0x4d,0x6e,0x7e,0x8c,0x97,0x86,0xce,0x7f,0x2c,0xd5,0xc1, -0x81,0x00,0x0b,0xee,0x47,0x48,0x1f,0x3a,0x64,0x0f,0x7a,0x78, -0x09,0xa1,0x15,0xc8,0x65,0x21,0xc1,0x22,0x86,0x2f,0x41,0x11, -0x94,0x7e,0xd6,0x44,0x24,0xb9,0x9d,0x58,0xe2,0x70,0x68,0x04, -0xb1,0x1b,0xd6,0xa5,0x1e,0x6f,0xcd,0x50,0x98,0x3f,0x5a,0x8b, -0x48,0xa3,0x5f,0xd3,0x53,0x27,0xa0,0x74,0xf4,0x85,0x18,0xea, -0xda,0x6a,0xf4,0xa5,0x48,0x82,0xf2,0x1f,0x9b,0x3e,0x12,0xe0, -0xe1,0x2b,0x7f,0x6c,0x02,0xe0,0x91,0x09,0x30,0xff,0xfa,0x57, -0x0d,0x90,0xe7,0x98,0x05,0x4b,0xd0,0x5e,0x1f,0xee,0x08,0xab, -0x3d,0x2c,0xf5,0x45,0xe5,0xd4,0x23,0x2e,0xf3,0x45,0x85,0xbe, -0xd2,0x85,0xf4,0xa4,0x5e,0xa7,0x84,0x33,0x7e,0xb3,0x24,0x82, -0x6d,0xe3,0xaa,0x06,0x32,0x43,0xe5,0xa2,0x01,0x70,0x55,0x25, -0x48,0x9a,0xe6,0x30,0xeb,0x94,0x12,0x5c,0x40,0x50,0x07,0xcb, -0xda,0x1d,0xbd,0x63,0x08,0xfc,0xbe,0x01,0xf6,0x8e,0xaa,0x90, -0xc6,0x46,0xe7,0x5e,0x3c,0xa4,0x00,0xff,0xf3,0x3b,0xe8,0x60, -0xb0,0x61,0x13,0x98,0x99,0xd0,0x7c,0x29,0x83,0x48,0x41,0xc5, -0x63,0x4c,0xbb,0x3c,0x36,0xd9,0x81,0xd7,0xcd,0x5d,0x8d,0xe9, -0xc9,0x5f,0x82,0xa9,0xa3,0x31,0xfd,0xd0,0x84,0x89,0x0b,0xe9, -0xcd,0xcb,0x1d,0x10,0x7f,0xeb,0xec,0x2d,0x16,0x9d,0xbd,0xa5, -0x5e,0xb5,0x9a,0xf8,0xa3,0xa9,0x72,0xa9,0x17,0xb9,0x06,0x18, -0x5e,0x70,0x01,0xe6,0xb0,0x0e,0x06,0xed,0xbd,0x18,0x9b,0x5b, -0x9f,0xe3,0x64,0x47,0x0f,0x07,0x28,0x53,0x57,0xc8,0xef,0xde, -0x15,0x18,0x12,0x1c,0x50,0x87,0x6b,0x20,0xa9,0x16,0x57,0x0f, -0x75,0xd1,0x9f,0xd2,0xc0,0x1b,0xdf,0x30,0xdb,0x46,0x58,0x10, -0x67,0xc0,0xe9,0x7e,0x16,0xb5,0xec,0xde,0x7f,0x77,0x7f,0x88, -0xd1,0x3c,0xb6,0x87,0x90,0x0e,0x17,0x47,0x4f,0x5b,0x35,0xa4, -0x0e,0x51,0x37,0xbe,0x97,0xb4,0xa2,0x34,0xaa,0xcd,0x07,0x73, -0x37,0x24,0xfc,0xa7,0x7f,0x47,0xf5,0x9d,0xdd,0x07,0x9a,0x02, -0xe2,0x3b,0xbf,0x37,0xf5,0x1a,0xe0,0x7f,0x72,0x7a,0x8a,0x93, -0xec,0xf7,0x95,0x50,0x10,0xa5,0x01,0xa0,0xfe,0xd9,0x04,0xa4, -0x85,0xa3,0x4f,0xa0,0xb4,0x50,0x10,0xbb,0x19,0x04,0x5a,0x5a, -0xde,0xb4,0xc0,0x2a,0x96,0x4a,0xfd,0x66,0x38,0xb9,0x78,0x0c, -0xb3,0xf4,0x5a,0x13,0x0d,0xba,0xa1,0x68,0x60,0xaf,0xf5,0x16, -0x59,0x25,0x8b,0xa2,0x21,0xd4,0x5a,0xe5,0x14,0xad,0x68,0x12, -0x0d,0xc2,0x22,0x71,0x3d,0xec,0x00,0x9c,0x86,0xa0,0x7d,0xa4, -0x14,0x36,0x44,0xcd,0x5e,0xf2,0xca,0x5a,0x8b,0x41,0xe6,0x3c, -0x01,0x36,0x54,0xb6,0x47,0x95,0xd5,0x65,0xb7,0xe1,0x12,0xc7, -0x6a,0x0c,0x00,0xa2,0x47,0xaf,0x71,0xb4,0x67,0x2e,0x28,0x4b, -0x57,0x88,0x65,0x3e,0x73,0xfd,0x25,0x55,0x9f,0xe8,0x56,0xa7, -0x20,0x7b,0x0a,0x70,0xa1,0xc3,0x79,0x90,0xcc,0xfa,0x4f,0xa0, -0xaf,0xc6,0x2d,0x3f,0x40,0xf4,0xa0,0xc3,0xc1,0x4c,0x14,0x5e, -0x50,0x7f,0xd5,0x8f,0x4e,0xa2,0x16,0x27,0x2c,0x15,0x3b,0xec, -0x4a,0xa3,0xff,0x1a,0x8e,0xe7,0xc3,0xe9,0xe7,0x93,0x17,0xf9, -0xe9,0x14,0x1f,0x0e,0x86,0xd3,0xb3,0xcb,0x93,0x27,0x37,0xd3, -0xe2,0x4a,0x3f,0x7f,0x3e,0xf9,0xaf,0xf9,0x38,0xd7,0x7f,0xae, -0x3e,0x9f,0x3c,0x99,0x5f,0xcc,0xcb,0xd9,0xc9,0x61,0x7e,0x33, -0xcb,0xe1,0x3c,0xec,0xe4,0x8d,0x16,0xe7,0xe1,0xf7,0xf5,0xe4, -0x37,0x4a,0x78,0x96,0x9f,0xe1,0x83,0x95,0xf4,0x4e,0x22,0xe3, -0x76,0xcb,0x7d,0xe3,0x75,0xa0,0xef,0xa2,0xfc,0xbe,0xf5,0xc4, -0x35,0xb0,0xd4,0xe1,0xa0,0x43,0x12,0x81,0x3c,0xa0,0x0c,0xe8, -0x32,0x54,0x01,0x51,0x40,0x13,0x10,0x04,0xa4,0x00,0x19,0x40, -0x41,0x4d,0xe5,0x84,0xe8,0x0e,0x0a,0x10,0xc8,0x23,0x43,0x5c, -0x28,0x12,0x85,0x21,0xf1,0xb5,0x75,0xab,0x59,0x80,0xf2,0x50, -0xa1,0xc5,0x0f,0x99,0x03,0x0a,0x8c,0x58,0x12,0xc2,0x20,0x86, -0x69,0x59,0x7f,0x40,0x86,0xc2,0x9d,0x3d,0x67,0xaa,0x50,0x81, -0x42,0xaf,0x54,0x70,0x3e,0x62,0x54,0xa2,0xe0,0x3e,0xa7,0xbf, -0xa7,0x37,0x49,0xc5,0x20,0xa1,0x2a,0xb3,0xe8,0xe7,0xa8,0xe5, -0x04,0xe4,0x12,0xcf,0x94,0xd0,0xf9,0xfb,0xc2,0xcf,0x20,0xc9, -0x8e,0x73,0x55,0x5d,0x55,0x95,0xab,0x74,0xee,0xba,0x5c,0x1b, -0x6f,0xcb,0xd1,0x65,0xb9,0xda,0xb2,0x74,0x43,0xd6,0xb5,0x8c, -0x0d,0xb8,0xad,0xbb,0xd2,0x70,0x8c,0x34,0x3a,0x9c,0x8f,0xf5, -0xc3,0x89,0xde,0xbf,0xc1,0xcf,0xd1,0x3c,0x2f,0xe1,0xf7,0xc7, -0x7c,0x34,0xa6,0xa7,0xa3,0x4b,0xbd,0xd8,0xc1,0xc3,0x0b,0xbd, -0xb2,0xea,0x9f,0xc3,0xa1,0x46,0x02,0xdb,0x1a,0xd9,0xa1,0x16, -0x5d,0x63,0x6f,0x1a,0x88,0x3e,0x1f,0x6a,0x0d,0x04,0x11,0x3c, -0xa2,0x34,0x25,0x40,0x06,0xd0,0x00,0xf5,0x43,0xcd,0x50,0x29, -0xd4,0x58,0x5b,0xd9,0x1d,0xe3,0xc7,0x03,0xab,0xab,0x56,0x73, -0x4b,0xa8,0x54,0xd7,0xa9,0xab,0xd4,0x35,0xea,0x0a,0x75,0x7d, -0xba,0xba,0xda,0xda,0x0e,0xa4,0x6b,0xd4,0xa6,0xba,0x34,0x90, -0xa8,0xc9,0x3b,0x62,0x77,0x85,0x39,0xf9,0xae,0xd1,0xea,0x95, -0xb6,0xe3,0xd5,0x4b,0xb5,0x23,0xf6,0x9b,0x70,0xc0,0x7a,0x60, -0xe1,0x90,0xa5,0xe1,0xda,0x19,0x24,0x48,0x69,0x51,0x19,0xb5, -0x76,0xcf,0x50,0x1d,0xb7,0xfe,0x9e,0xa4,0x39,0x1f,0x76,0x3d, -0xfe,0xb8,0x0e,0x29,0xfa,0xb2,0x91,0x1d,0x96,0xa6,0xb1,0x2d, -0x5b,0x52,0x8e,0x6e,0xff,0x06,0x7a,0x7a,0xfb,0xea,0x28,0x8d, -0x2e,0xd3,0xeb,0xeb,0x8d,0x27,0x91,0x7a,0x95,0x46,0x07,0x07, -0x3b,0xcf,0x9e,0xed,0xe0,0xb5,0x3a,0xf5,0x0a,0xdf,0x0f,0x60, -0x53,0x6c,0x12,0xfc,0x94,0x8d,0x57,0x47,0x98,0xa8,0x53,0x61, -0xd3,0xa5,0x36,0x82,0x3c,0xbd,0x6e,0xfb,0xb5,0x39,0x3f,0x20, -0xc6,0x67,0x08,0xbb,0x2e,0xa0,0x2f,0xf1,0xa1,0xd1,0x05,0x87, -0x09,0xe1,0x43,0x70,0x5b,0x5b,0x4d,0x80,0xed,0xd9,0xe4,0x3d, -0x44,0x20,0x25,0xff,0x2b,0x18,0x48,0x6e,0x35,0xe6,0xa0,0x80, -0xb3,0x74,0x84,0x6f,0x58,0xe8,0xff,0x9e,0x3d,0x03,0x3d,0xf1, -0x48,0x1a,0x57,0x06,0x37,0x8e,0xd8,0xfa,0xa7,0x03,0x51,0x6a, -0x1a,0x3f,0x20,0x23,0x3a,0x12,0xc5,0x7e,0x13,0x14,0xdc,0x25, -0x4c,0xc3,0xe3,0x21,0x3e,0x14,0xbc,0x81,0xeb,0xb0,0x94,0x86, -0x77,0x6e,0x7d,0xa7,0x32,0x60,0xc4,0xf7,0x64,0x16,0x83,0x68, -0xeb,0xdf,0xd5,0x4e,0x77,0xfa,0xc3,0x9b,0xc1,0x71,0xbb,0x77, -0xdd,0xd3,0x7f,0x76,0xf4,0x9c,0xe1,0x5c,0x57,0x8f,0xf1,0x9e, -0x49,0xbe,0x34,0x8b,0x12,0xf1,0xba,0xc8,0x2d,0x90,0x0b,0x0e, -0xa3,0x38,0xa3,0x17,0xdd,0x5c,0x6b,0x09,0xf3,0xed,0x41,0x94, -0xda,0x94,0x21,0xa4,0x3c,0x39,0xd0,0x9d,0x7a,0x72,0x36,0xd4, -0xc2,0xcb,0x68,0x38,0x4d,0x6f,0x4b,0x3d,0xbc,0x9e,0x81,0x4e, -0xa7,0x7f,0xf4,0xff,0x31,0x77,0x35,0x3a,0x6d,0x24,0x49,0xf8, -0x55,0xcc,0x48,0x46,0x1e,0x3c,0x43,0x6c,0xc2,0xc2,0x6a,0xc8, -0x30,0xca,0x5d,0x4e,0xa7,0x3d,0x2d,0xc9,0x4a,0x44,0x3a,0x6d, -0x10,0x42,0xce,0xda,0x04,0xa4,0x18,0xaf,0xb0,0x57,0xb7,0x1c, -0xf6,0xbb,0x5f,0xd7,0x5f,0x77,0x55,0x4f,0x8f,0x21,0x44,0x5a, -0x9d,0x22,0x85,0x71,0x77,0x75,0x77,0x4d,0x4f,0xff,0x54,0x57, -0x57,0x7d,0xb5,0x70,0xa3,0xad,0x37,0x59,0x5d,0xe2,0xa0,0x00, -0xfc,0x10,0x49,0x9f,0x2f,0xee,0x01,0x56,0x5c,0x67,0xfd,0x1b, -0x55,0x3e,0xd0,0xbd,0xbd,0x0b,0x49,0x07,0x18,0x01,0x2a,0xf2, -0xab,0x1b,0xbf,0xb3,0x7b,0x5d,0x1d,0xe4,0x51,0x99,0x8b,0x9f, -0xdd,0xe3,0x65,0xcf,0x16,0x05,0x2e,0xfe,0xf1,0xd5,0xf5,0x43, -0xf6,0x33,0x2a,0xa4,0x98,0x3b,0x3d,0xda,0x8a,0x18,0x2c,0x83, -0x81,0x9b,0x99,0x56,0x8f,0xb5,0x54,0xdc,0x20,0x2c,0xd3,0x30, -0x3a,0x2a,0x59,0x4a,0x40,0x85,0x95,0x7c,0xd4,0x2b,0x6d,0x3c, -0x53,0x3d,0x5e,0xa3,0x6d,0x4d,0x95,0x39,0x21,0xa8,0xef,0x84, -0xd2,0xdf,0x1d,0xcf,0x55,0xd6,0x5f,0xf6,0x26,0x5f,0x16,0xa8, -0xd3,0x9b,0xf4,0xae,0xdd,0x0c,0x17,0x05,0x24,0xa8,0xf6,0x26, -0x3d,0x51,0xee,0xcd,0xdd,0xaf,0xfe,0x54,0x60,0x4f,0x51,0xd9, -0x37,0xa1,0x2f,0xe4,0x9e,0x6f,0x30,0x0f,0x3f,0x17,0x6a,0xff, -0x26,0x3d,0xd4,0xa2,0x4d,0xa7,0x98,0x4e,0x22,0xf0,0x19,0xd6, -0x46,0x0a,0xbf,0xb3,0x33,0xaa,0x8c,0xc5,0xe6,0x07,0xc8,0x22, -0x0d,0xe0,0xc3,0x03,0xe6,0xa0,0x3d,0x89,0xeb,0x34,0xc3,0xbf, -0xef,0xb8,0xa4,0x0d,0x52,0x6c,0x41,0xd4,0xee,0x55,0x5d,0xd9, -0x05,0x91,0x3f,0xb7,0x73,0x9f,0xd7,0x24,0xf7,0x7b,0x98,0xb4, -0x7d,0xd0,0xed,0x0b,0x10,0x2e,0xf6,0x37,0x51,0x86,0x57,0x01, -0xa4,0x4b,0x9e,0x83,0x1c,0x8e,0x84,0x94,0xe8,0x09,0x8e,0x81, -0xf4,0x74,0xd4,0x64,0xf4,0x15,0xdd,0xa8,0x87,0xfa,0xb2,0x2d, -0x6f,0xc0,0x77,0xd8,0xf4,0x87,0x17,0x1e,0xb9,0xd9,0x56,0x3c, -0x2e,0x1d,0x8f,0x9c,0xe9,0x95,0x94,0x71,0x5f,0x47,0x9b,0x27, -0x53,0x85,0xf5,0xbf,0x3f,0xcd,0xc2,0x7b,0x4a,0x76,0x85,0xc9, -0x72,0x53,0x1f,0xaa,0x8c,0xe2,0xd7,0xd0,0x4f,0xd7,0x3d,0x8b, -0xe5,0xea,0x3a,0x5a,0x90,0x3b,0x48,0x51,0x37,0x6b,0x01,0x67, -0x94,0x09,0x88,0x32,0x0b,0x0a,0x5b,0x11,0xb8,0xea,0xd9,0x9f, -0x0f,0xa4,0x6a,0x63,0xf1,0xa2,0x7a,0x24,0x0f,0x43,0x97,0x5e, -0x1d,0xb9,0x34,0x85,0xbd,0x52,0xf9,0x00,0xc7,0xa4,0xcd,0xd6, -0x59,0x5d,0x97,0xcf,0x8a,0x06,0xd0,0xdb,0xf9,0x82,0xd7,0xae, -0xaf,0x85,0xc2,0x6e,0x29,0xc8,0x95,0xc1,0x2f,0xb9,0x9f,0x29, -0x9a,0x56,0xf8,0x98,0x40,0x04,0xd6,0xc9,0x48,0x46,0xc6,0x52, -0xe8,0x9f,0xa3,0x70,0x23,0x95,0xe9,0xc7,0xe3,0xd5,0x2d,0xc1, -0x12,0x15,0x57,0xd7,0xfc,0xc5,0xdd,0x1e,0x5b,0x61,0x31,0xf6, -0x9a,0xa8,0xe8,0x4f,0x41,0xb7,0xfe,0x80,0xc3,0x96,0x6f,0x8a, -0x20,0x20,0x3f,0x87,0x57,0xbc,0xa3,0x3e,0xf9,0x1e,0x8e,0x6b, -0xc3,0x33,0xf9,0x55,0x55,0x3b,0x23,0xcf,0xd4,0x28,0x70,0x9d, -0x78,0x21,0xf3,0x22,0x1b,0x86,0xc5,0x0c,0x2f,0xe1,0x4e,0xe9, -0x2d,0x83,0x07,0x83,0x12,0x85,0x2e,0xe0,0x6c,0xa0,0x5a,0x44, -0xf7,0xfe,0xf1,0xfb,0x7b,0xd1,0x60,0x79,0xfb,0xe5,0xce,0xad, -0x4b,0x2b,0xba,0x07,0xfa,0x69,0xb9,0x70,0xf3,0x5f,0xee,0xa1, -0x6b,0x6f,0x85,0xe2,0x41,0xdf,0xd9,0x32,0x42,0xa2,0xae,0xd5, -0x2d,0x54,0x50,0xaa,0xde,0x37,0x1b,0x8a,0x32,0xe8,0x14,0xa1, -0x9d,0x57,0x52,0x41,0xf3,0xb8,0xe1,0x8e,0x20,0xb3,0x70,0x84, -0xdf,0xf2,0x10,0x91,0x4c,0x03,0xe8,0x6f,0x52,0x23,0x6d,0xf7, -0x54,0x89,0x37,0x6a,0x30,0x78,0xd5,0x94,0x47,0xa6,0xe0,0x75, -0xc7,0x25,0xbd,0xb6,0x7e,0xce,0x9b,0x01,0x74,0x02,0x28,0x64, -0xc0,0x84,0x11,0x2d,0xc8,0xc7,0x97,0x4d,0x39,0xae,0xc6,0xe1, -0x3d,0x1e,0x1f,0x60,0x32,0x55,0xa4,0xa0,0x21,0x1a,0x74,0xf5, -0xcc,0xf7,0xb0,0x03,0x6f,0x4c,0x0e,0xfa,0xab,0x70,0xce,0xdc, -0xe4,0xb0,0x17,0x25,0xe7,0x2d,0x4d,0x1e,0x23,0x00,0x48,0xb9, -0x65,0x54,0x30,0x60,0x04,0x10,0x05,0xd8,0x1c,0x79,0x53,0x7a, -0x63,0x39,0x60,0x5e,0x0e,0xc3,0x62,0x76,0xbc,0x9d,0x7c,0x72, -0x33,0x38,0x24,0x52,0x10,0x76,0xe4,0xee,0xae,0x85,0x14,0x08, -0x4b,0x64,0xe1,0xfe,0xed,0x67,0xde,0xa8,0x98,0xad,0x7d,0x5d, -0xb1,0xbc,0x19,0x55,0xf0,0x87,0x98,0x34,0x3d,0x28,0xed,0xf1, -0x3b,0x1d,0xb8,0xb3,0xc3,0x59,0x9c,0xf8,0xda,0x25,0x4e,0xe3, -0xc4,0x43,0x97,0x78,0x13,0x27,0xfe,0xe0,0x12,0xe7,0x71,0xe2, -0x91,0x4b,0x5c,0xc6,0x89,0xc7,0x2e,0xf1,0x3f,0x71,0xe2,0x8f, -0x18,0xee,0xd8,0x71,0x4f,0x16,0x53,0xad,0xa8,0x06,0x6a,0x1e, -0x80,0xa1,0x18,0xba,0x74,0x46,0xe8,0xf3,0x08,0xc9,0x86,0xd8, -0xf3,0xae,0x1a,0x02,0x68,0x63,0x8b,0x75,0x7c,0xc6,0xda,0xfd, -0x4c,0x84,0xd8,0x24,0xd0,0x0f,0x6c,0xfc,0xe2,0x27,0x28,0x41, -0xd7,0xf0,0x65,0xb3,0xbf,0x76,0x4e,0xd9,0x1e,0x69,0x6b,0x1d, -0x5f,0x2d,0x36,0x6a,0x64,0x30,0x81,0x3a,0xc4,0xcf,0x78,0xa2, -0x00,0x14,0x19,0xa4,0xb0,0xf1,0x48,0x84,0x69,0xcc,0x45,0xa9, -0x80,0x7d,0xf6,0x6b,0x2e,0x35,0x88,0x20,0x0c,0x61,0x09,0xa9, -0xb3,0xd9,0x5d,0x96,0x57,0x31,0x7a,0x62,0x1a,0xbd,0xb0,0xb8, -0xaf,0xdb,0xc6,0x48,0xdc,0x6b,0x11,0xcc,0x5d,0xc2,0x46,0x8e, -0x6a,0xd8,0xbf,0x02,0xc8,0xc6,0xbc,0x8a,0xe8,0x29,0xd9,0xf4, -0xc9,0x3b,0x08,0x62,0x60,0x4f,0x42,0xa1,0x2b,0x30,0x46,0x3f, -0x15,0x55,0x8f,0x54,0x8b,0xfb,0x9a,0xf0,0x66,0x51,0x2a,0xc2, -0x18,0x27,0x58,0xda,0xc4,0x96,0x7a,0x9d,0xa1,0x26,0x75,0x54, -0x5e,0x22,0x55,0x65,0xdf,0xb5,0x96,0xe9,0x2d,0xa5,0x85,0xd8, -0x9d,0x72,0x6a,0x54,0xd9,0x0a,0xf0,0xf9,0x18,0xe1,0xca,0x4e, -0xca,0xf2,0x36,0xf7,0x00,0xd5,0x48,0x80,0xc0,0x76,0x70,0xb0, -0xe7,0x06,0x2d,0x70,0x76,0x68,0x94,0x51,0xc7,0x63,0xf4,0x6b, -0x0d,0x4e,0x1e,0x98,0x26,0x91,0x40,0xe1,0x4c,0x83,0xed,0xa4, -0xc5,0x76,0xc6,0x2d,0x0c,0xbc,0xc0,0xac,0xf1,0x1d,0x52,0x36, -0xac,0xfc,0x44,0xa0,0xd0,0x82,0x0a,0x57,0x49,0x2c,0x52,0xf0, -0x2e,0x9b,0xfb,0x76,0x71,0x1a,0x7f,0x32,0xd1,0xc9,0x93,0x9b, -0x21,0x25,0x06,0x72,0xd8,0x4a,0xa5,0x49,0x19,0x3d,0x35,0x1b, -0x03,0xeb,0x10,0x34,0x10,0xf8,0x3c,0x21,0x07,0x71,0xad,0x68, -0xf8,0xc7,0xb1,0x67,0x3e,0x5c,0xb7,0xe9,0x86,0x6c,0xac,0x87, -0x81,0x39,0x58,0x15,0x42,0x71,0xb1,0xc0,0x4b,0x1e,0x0d,0x11, -0xfe,0x4c,0xd4,0xae,0x00,0x9c,0xc9,0x02,0x0f,0xec,0xf9,0x37, -0x85,0x40,0xec,0x77,0xc9,0x65,0xc8,0xac,0x3b,0xb3,0xe2,0xdd, -0x13,0xce,0xc5,0x7d,0x16,0x8f,0xe1,0xd8,0xd7,0x43,0x3d,0x01, -0x2b,0x0a,0xc4,0x6c,0xa4,0x77,0xf1,0xcf,0xb3,0x8f,0x97,0x9f, -0x3e,0x65,0x58,0xfd,0x56,0xa1,0x8f,0xf8,0x6e,0x82,0x41,0xa9, -0xb1,0x7a,0x84,0xe2,0x3f,0x9d,0x7f,0xe8,0x64,0xd0,0x02,0x16, -0xaa,0xde,0x23,0xb9,0xa6,0x50,0x66,0x33,0x17,0x1f,0x2f,0x83, -0x0d,0xcc,0xf9,0xf9,0xc5,0xa7,0x4b,0xe2,0x0e,0x2d,0x51,0xe3, -0x0b,0x96,0x39,0x1e,0x26,0x78,0x40,0x5d,0xcc,0xe5,0xbe,0xd5, -0x2b,0x68,0x0b,0x89,0xf3,0x50,0xcc,0xfd,0xf5,0xdb,0x5c,0xdd, -0xbd,0xcd,0xc3,0x85,0x17,0xa6,0xeb,0x1b,0xa5,0xcb,0x8d,0xb8, -0xfd,0x25,0x5e,0xc8,0x3b,0x04,0xd2,0x10,0x70,0x93,0xf6,0xfc, -0xe3,0x39,0x04,0x71,0x9a,0x75,0x5e,0xd8,0x5f,0x4d,0x1a,0xbe, -0x8c,0x64,0x70,0xc6,0x18,0x07,0x94,0xa9,0x8a,0x81,0xb6,0x58, -0x55,0x73,0x87,0xf3,0x65,0xc1,0xf3,0xbf,0x41,0x69,0x41,0x76, -0xba,0x79,0x7e,0x3a,0x02,0x69,0x17,0x37,0x73,0xb1,0x60,0x4e, -0xf0,0x13,0x2c,0x86,0xa9,0x8e,0xdf,0xaf,0xe1,0x1d,0x68,0x92, -0xbd,0x5d,0x75,0xbe,0x80,0xc2,0x1d,0x76,0x83,0x77,0xf5,0xdb, -0xb6,0x8b,0x26,0xd4,0x99,0xa0,0x2d,0xde,0x13,0x54,0x85,0x7a, -0xdb,0x7a,0x67,0xcc,0xf1,0x9c,0xe2,0xc3,0x12,0x4e,0xe0,0x73, -0x85,0xaf,0xc6,0xa7,0x5f,0x33,0xb0,0xd0,0xa8,0x40,0x51,0x42, -0x70,0x8d,0xf6,0xbe,0xea,0xd7,0x1f,0x7d,0x53,0x1b,0x0e,0x68, -0x03,0x7f,0x5e,0x9c,0x68,0xa3,0x06,0x92,0x4b,0x51,0x1d,0x85, -0x26,0xfd,0x7f,0xf8,0xdd,0xd4,0x3d,0xd6,0x5d,0x6e,0x6e,0x76, -0x8b,0x1b,0x00,0x02,0x0e,0x1b,0x69,0x57,0x71,0x5e,0x68,0xa0, -0x78,0x2a,0x9a,0x0b,0xbe,0xa7,0x2b,0x58,0x8c,0x73,0xee,0xae, -0x25,0x13,0xff,0x9f,0xf3,0x5b,0x7a,0x86,0xe1,0xc0,0x1f,0x33, -0x8b,0x7b,0x4a,0x31,0x59,0xa2,0x80,0xc9,0x6c,0x07,0x15,0x82, -0x2b,0xe8,0x75,0x07,0x7a,0x6a,0xc8,0xda,0x8e,0xc3,0x29,0x5e, -0x63,0x2b,0x4b,0x83,0xe3,0xd1,0x4d,0x75,0xa0,0xc5,0xa0,0x21, -0x61,0x61,0xa6,0x2b,0xda,0x12,0x1a,0xe1,0x67,0x3f,0x4e,0x28, -0x82,0x46,0x72,0x03,0x2c,0xc8,0xb8,0x0b,0xbd,0x25,0xdd,0xef, -0xf5,0x9a,0xcd,0xbe,0x24,0xa1,0x41,0x25,0x48,0x7d,0xf8,0xfa, -0x60,0xf6,0x03,0x37,0xa4,0x11,0xe0,0xf3,0x21,0xb6,0x67,0x92, -0x44,0x6d,0x59,0x8f,0x0f,0xf6,0xb4,0x11,0x09,0xb1,0x46,0xcf, -0x80,0xdb,0xa6,0x2c,0x9e,0x28,0x6b,0x6e,0xcb,0x0f,0x6b,0xa4, -0x29,0xcd,0x6a,0xeb,0xc4,0x86,0xfb,0x95,0x8a,0x22,0x51,0x0e, -0xa0,0x68,0xa0,0x71,0xfd,0xde,0xa2,0xc9,0xf3,0x57,0xea,0x33, -0x94,0xed,0x3e,0xdb,0xda,0x82,0x10,0x0d,0x54,0xcf,0x6e,0x6f, -0x4f,0xba,0x9f,0x9b,0xb5,0x3d,0xec,0xb5,0xcc,0xaf,0x00,0x40, -0xc0,0x9d,0x7a,0xb0,0x7f,0xf1,0x45,0xa1,0x36,0xe9,0x3b,0x31, -0xe1,0xf3,0xdf,0x01,0xc8,0x60,0x1f,0xf5,0x16,0x7d,0x36,0xc7, -0xbd,0x13,0x1b,0xf8,0xd9,0x74,0xb0,0x2b,0x24,0x83,0x3f,0xfb, -0x45,0x4b,0x19,0x41,0xf9,0x2b,0x34,0x38,0x64,0x1b,0xc0,0x4e, -0xa2,0xa3,0xd1,0xe1,0x8f,0x8e,0x0a,0x52,0x73,0x19,0xe1,0xac, -0x7e,0xab,0x7c,0x28,0x2a,0xbf,0xf0,0x5c,0xbb,0xa9,0x13,0x26, -0x07,0xe0,0xaf,0x59,0xe5,0x5c,0x24,0x87,0x84,0x39,0x49,0x03, -0xcc,0x35,0x82,0x85,0x72,0x96,0x05,0xf4,0x3a,0x47,0xf2,0xeb, -0xfe,0xcd,0x1f,0xf3,0xc9,0x9d,0x1b,0xcb,0x03,0xeb,0xaa,0xc8, -0x6d,0xbf,0x5f,0x28,0x83,0xac,0x74,0xcb,0x58,0x27,0xd0,0xca, -0x6e,0x9e,0x17,0x71,0x4d,0x6d,0xa5,0x70,0x98,0xd3,0x75,0xe0, -0xd4,0xc3,0x7e,0x85,0x09,0xcc,0x03,0x20,0x8c,0x0c,0xf8,0x00, -0x6e,0xae,0x91,0xb2,0x15,0x70,0x9b,0x58,0x93,0x58,0x1e,0x9d, -0x42,0x1d,0x4d,0x26,0xea,0xe8,0xac,0x2a,0xc7,0x9c,0x24,0x0a, -0xec,0xac,0x1a,0xa9,0x94,0x77,0xae,0xa6,0x6a,0xac,0x4a,0x61, -0xc2,0x01,0x27,0xb0,0x0e,0x3d,0xab,0x8e,0x55,0x02,0x55,0x12, -0x9a,0x30,0xdb,0x07,0xef,0x19,0xba,0x8b,0xe5,0xbd,0x05,0xbe, -0x18,0xe7,0x05,0xca,0x08,0x12,0xc2,0x22,0x29,0x4f,0xf8,0xf8, -0x16,0x6a,0xce,0x8b,0x64,0xb1,0x75,0x0b,0xcd,0xdf,0x18,0xf1, -0x8f,0x96,0x81,0x91,0x4c,0xa3,0xf5,0xfa,0x09,0xca,0x1f,0x84, -0x72,0x53,0x4c,0xb5,0x80,0xa5,0xbc,0x68,0xc0,0xef,0x4c,0x2f, -0xbc,0x2c,0xf4,0x69,0x37,0xb5,0x4a,0xa5,0x19,0xbf,0x35,0xe3, -0x61,0xa3,0x1c,0x75,0xad,0x17,0xb1,0xea,0x3f,0x16,0x08,0xc0, -0xdb,0xef,0x71,0x4a,0xea,0xa0,0xd2,0x11,0x6e,0xdc,0x64,0x87, -0x3f,0x74,0xe1,0xde,0xc5,0xe6,0x87,0x6b,0x5c,0x45,0x8b,0x6f, -0xf1,0xfb,0xe9,0xf4,0xd5,0x22,0x6e,0x8d,0x3d,0x57,0x1b,0x3d, -0xb4,0xc3,0x47,0x19,0xfb,0xa8,0x1a,0x04,0x96,0x78,0xc4,0x93, -0x30,0x1a,0x9e,0xc7,0x22,0xff,0x68,0xb7,0xa3,0x0c,0x4b,0x64, -0xde,0xf5,0x28,0x90,0x9f,0x49,0x54,0x2c,0xf5,0xae,0xed,0x5d, -0x25,0x7f,0x89,0xb7,0x92,0x6f,0x14,0xcc,0xc3,0x68,0xe2,0x55, -0xf1,0x59,0x90,0x91,0x5c,0xb7,0x6c,0x88,0x8c,0xef,0x8a,0x6b, -0x76,0xa5,0xf6,0xa8,0x11,0x03,0x36,0xd2,0x1a,0x5f,0xe9,0x0e, -0x60,0xe4,0x48,0x9c,0x65,0xf8,0x28,0x76,0xc9,0x8c,0x0a,0x09, -0xf3,0x51,0x19,0xba,0x49,0x4d,0xb8,0x5a,0x57,0xc6,0xb0,0xcd, -0x37,0x42,0x4b,0x7c,0x65,0x2c,0xda,0x24,0x93,0x77,0x86,0xaa, -0x6d,0x43,0xe6,0x24,0x56,0xd6,0xf7,0xda,0xc5,0xdc,0x98,0x19, -0x3a,0x99,0xc2,0x33,0x18,0x36,0xa5,0xd8,0xca,0xd0,0xcb,0x39, -0x6e,0x25,0x48,0xf4,0xe3,0x33,0x04,0x0b,0xe2,0x9d,0x17,0x40, -0x4a,0x23,0x0f,0xd8,0xb8,0xf1,0x86,0xbb,0x8e,0xa4,0xa7,0x71, -0xbe,0x2f,0xb2,0xa0,0xdb,0x50,0xdd,0x6a,0x39,0xc6,0x35,0xe4, -0xad,0x3b,0x98,0xdc,0x27,0xe5,0xad,0x88,0x99,0x94,0x63,0x14, -0x35,0x83,0xff,0x47,0xb6,0xf2,0x43,0xb3,0x9e,0x58,0x6e,0x4f, -0x87,0x56,0xf6,0xb2,0xb9,0xc0,0xd4,0xdf,0x66,0x6e,0x85,0x9c, -0xfd,0xb5,0x5c,0xbd,0x79,0x92,0xab,0xf3,0xc9,0xfc,0x2f,0xe6, -0xc9,0x7d,0xc9,0x27,0xb8,0x9a,0x6b,0x6b,0x8f,0xc5,0xea,0x46, -0x5d,0x5a,0xe1,0x2f,0xd1,0xb0,0x24,0xa1,0x1e,0x70,0x28,0x9d, -0x22,0x1d,0xad,0x4a,0xf8,0xe8,0x2a,0x9d,0xfc,0xf9,0x1d,0x95, -0x22,0xc9,0x29,0xd4,0x67,0x2a,0xfd,0xaf,0x51,0x98,0xa8,0xc5, -0x99,0x64,0xf2,0x3a,0x12,0xd0,0xf5,0x7a,0x2c,0xbe,0xb5,0x7a, -0xdd,0x26,0x42,0xbd,0xb9,0x88,0x73,0x0a,0xaf,0x69,0xdb,0x7c, -0x6d,0x79,0xfd,0x7e,0x02,0xbe,0x36,0x2f,0x3c,0x8a,0x02,0x25, -0xbc,0x19,0x1f,0xf9,0xc2,0x47,0xa3,0x3d,0xbd,0x06,0x33,0xe3, -0xb5,0xda,0xb4,0xe4,0xe0,0x3a,0xe2,0xd0,0xef,0x3b,0xb5,0x60, -0x35,0x3f,0xeb,0x54,0x14,0x4b,0x70,0x54,0x49,0x49,0x0d,0x64, -0x73,0x27,0xf5,0x8c,0x51,0xe0,0x61,0x3d,0x83,0x18,0x7a,0x76, -0xdf,0xcd,0x99,0xed,0x8e,0x0a,0xbc,0x37,0x23,0x3a,0x5d,0xe0, -0xef,0x0b,0xba,0xd8,0x5c,0xcd,0xa6,0x3d,0xb7,0x0e,0x81,0x7e, -0x7a,0xf2,0xb5,0x07,0x9d,0x4d,0xf5,0x78,0xcd,0x59,0xbb,0xa2, -0x56,0xff,0x73,0xc5,0xbc,0x2a,0xaa,0x73,0xd9,0xad,0xac,0x8a, -0x37,0x93,0xe5,0xdb,0xaf,0xb7,0x5f,0xdc,0xdc,0x01,0xf7,0x1d, -0xfa,0x94,0x1d,0x06,0x1a,0xec,0xdd,0xad,0xcf,0xa7,0xe6,0xc4, -0x97,0x57,0xa3,0x02,0x50,0xaa,0xcc,0x89,0x84,0x08,0xfa,0x47, -0x23,0x94,0x69,0x64,0x63,0x4c,0xf4,0x81,0xde,0x36,0x95,0xdc, -0x55,0xe8,0xd3,0x15,0x09,0x46,0xec,0xa9,0xde,0x29,0x77,0xa0, -0xd7,0x3c,0x61,0x4a,0x0c,0xd2,0x07,0x22,0x14,0x62,0x3b,0x0e, -0x4b,0xb8,0x5f,0xe6,0x7c,0x98,0xc8,0x87,0xe3,0xd4,0xbc,0x08, -0x4c,0x78,0x01,0x09,0xe0,0x10,0xbc,0x84,0xc4,0x48,0x10,0x74, -0x83,0xdc,0xc9,0x2c,0xb4,0x54,0xab,0x9b,0x64,0x1c,0x85,0xe6, -0x84,0x60,0x6f,0x94,0xa3,0xd4,0x87,0x1c,0xbb,0x28,0xc5,0xdf, -0x83,0x65,0xed,0x41,0x58,0x7b,0x20,0xae,0x94,0x49,0xff,0xf3, -0x19,0x1b,0x17,0x87,0x2f,0x6e,0xd0,0x5e,0xa4,0xab,0x96,0x10, -0x68,0x57,0xbf,0x1c,0x7a,0x99,0xe0,0x17,0x49,0x35,0x83,0xf5, -0x98,0x2e,0x3f,0xde,0xa3,0xea,0x4a,0xc8,0xca,0xc3,0xbb,0x75, -0xb7,0x96,0x7c,0x2f,0x48,0x7c,0x61,0x83,0x1e,0xf7,0xa1,0xa3, -0x41,0x90,0xd8,0xbd,0xba,0x61,0x90,0x0f,0x8f,0xcb,0xe4,0x27, -0xce,0xfb,0xc7,0x5d,0x0c,0x40,0xed,0x89,0x71,0xc6,0x59,0xe1, -0x9d,0x53,0x5c,0x74,0x2d,0xea,0xc8,0xcb,0x7a,0x7d,0x5c,0xf9, -0x5f,0x21,0xb9,0x7f,0xdc,0xf0,0x65,0x30,0xb6,0x73,0x0c,0xfe, -0x2c,0x7a,0x51,0xf8,0x36,0xc9,0x89,0x63,0x9c,0xa2,0x2c,0xdb, -0xaa,0x85,0x14,0xf6,0xcf,0xaa,0x2b,0x61,0x79,0xa2,0xaa,0xa7, -0x15,0x4e,0xda,0xa2,0x5a,0x79,0x85,0x83,0x9e,0xae,0xba,0x6e, -0x9c,0x0c,0xb8,0x16,0x2d,0x8d,0x48,0x3f,0x08,0xcf,0x5d,0x97, -0x5e,0xb4,0x0f,0x6c,0x72,0x86,0x04,0x4d,0x3b,0x62,0x6a,0x2c, -0xbd,0x14,0xae,0x42,0xba,0x14,0xda,0x31,0xa6,0x03,0xca,0x62, -0x34,0x59,0x70,0xdd,0xeb,0x2a,0x98,0x9f,0xa4,0xda,0x61,0xcf, -0xc7,0x4c,0x00,0xa5,0x32,0x7f,0x2e,0xb9,0xbe,0xc3,0x13,0x4b, -0x6d,0x7e,0xaa,0x4c,0x8e,0x1d,0x1d,0x25,0x28,0x02,0x18,0x87, -0x3a,0x1f,0x81,0x59,0xc2,0x4f,0x1e,0x9c,0x9a,0x82,0x93,0x14, -0xd1,0x6a,0xf1,0xaf,0xf3,0x0f,0xef,0x6b,0x9d,0xe0,0xef,0x30, -0xa2,0xdb,0x21,0x75,0x49,0x59,0xcb,0x56,0xae,0x2f,0x8a,0x38, -0x30,0x7a,0xac,0xea,0x10,0x34,0x27,0x1b,0x50,0x9b,0xc2,0xab, -0x1b,0x33,0x05,0xfa,0xf0,0xed,0x90,0xda,0x2c,0xf8,0xa8,0x70, -0xda,0x4c,0x89,0x3f,0x0a,0x0c,0xce,0x2e,0x34,0xab,0xc9,0x09, -0xfc,0x67,0xed,0x1f,0xf4,0x8f,0x3e,0xf8,0xce,0x4a,0x86,0xd7, -0x3a,0x69,0x0a,0x42,0x8e,0xc2,0x5a,0x84,0x4e,0xca,0x1e,0x85, -0x80,0xf8,0xbe,0xa8,0x06,0x95,0xe2,0xa6,0x39,0x1a,0x37,0xfd, -0x85,0x42,0x04,0x36,0xa5,0x5a,0xd3,0x78,0x53,0xae,0x08,0xe5, -0xe3,0xff,0xfd,0x83,0x43,0x7c,0xeb,0x61,0x20,0x0f,0x38,0x54, -0x48,0x8c,0x7d,0x02,0xff,0xf5,0x5f,0x8f,0xb8,0x43,0x14,0xf1, -0x94,0xb1,0x9c,0x98,0x17,0x19,0x42,0xf0,0xa7,0x3f,0x3e,0x60, -0x24,0xab,0xc0,0x09,0x66,0xbc,0x02,0x18,0x52,0xa4,0xa7,0x6c, -0xfc,0x9f,0xd6,0xd7,0xd4,0xc5,0x89,0x2f,0xed,0x0f,0xdd,0x08, -0x2a,0xb4,0xe5,0x0e,0xb0,0xd7,0xfe,0xb4,0x43,0xdc,0xe5,0xf7, -0xc2,0xb7,0x1d,0x92,0x9f,0xf2,0x9e,0x31,0xef,0x77,0x2c,0xe7, -0x43,0xf6,0x59,0xde,0x53,0x5e,0x53,0x57,0x81,0x71,0x27,0x48, -0xb1,0x1e,0xcf,0xaa,0xeb,0x44,0x57,0x27,0xda,0xb6,0xd9,0xfd, -0xec,0xee,0xb7,0x59,0x8d,0x67,0x10,0xd1,0x90,0x1a,0xe8,0xb1, -0x40,0x54,0xec,0x84,0x0a,0x8c,0x56,0xc6,0xe3,0x10,0xf9,0xec, -0xc8,0xe0,0x57,0x2e,0x52,0xbc,0x21,0xa0,0xae,0x95,0x55,0x9b, -0x79,0xf1,0x9d,0x77,0x2e,0xb1,0xd1,0x80,0x22,0x39,0xe9,0xee, -0x6f,0x88,0x41,0x1f,0xcd,0x2e,0xd5,0xf9,0x94,0x8b,0x73,0x4c, -0x77,0xb0,0x94,0xa2,0x99,0x46,0x39,0x34,0xcb,0x07,0xdf,0x72, -0xeb,0xf2,0x22,0x8e,0xcb,0xad,0x1c,0x97,0x1d,0x1c,0x97,0x4f, -0x73,0xfc,0xdd,0x5b,0xaa,0xdd,0x21,0x10,0x78,0x07,0x5d,0x2a, -0x97,0x2f,0xae,0x16,0x01,0x68,0xa8,0x6e,0x6f,0x6d,0x6d,0x2d, -0xc5,0x39,0x57,0x4c,0xbf,0x7d,0xa3,0xb8,0x67,0x86,0xd5,0x1b, -0x4d,0xe9,0xdc,0x44,0x59,0x26,0xee,0xc9,0x45,0xc0,0x5c,0x86, -0x30,0xf1,0x5e,0xdc,0x5f,0x0e,0x72,0xf1,0x01,0x8a,0x72,0xd9, -0x5f,0x26,0x67,0x68,0x3c,0x9b,0x49,0x93,0x5f,0xd0,0xf4,0x6c, -0x1e,0x5f,0x35,0x07,0x10,0xbe,0xa8,0x5e,0xb9,0x7f,0xce,0xfd, -0x72,0x6c,0x09,0xfc,0x1d,0xf4,0xb0,0xad,0xa5,0x22,0x78,0x3f, -0x33,0x76,0x26,0xcb,0x73,0xc9,0x64,0x14,0x23,0x95,0xf2,0x66, -0xd4,0x64,0x25,0x1c,0xe2,0x5c,0xc7,0xfd,0x92,0x0d,0x11,0x60, -0x6b,0x89,0xb2,0xf3,0x72,0x98,0xfd,0x4a,0x19,0xbc,0x18,0x36, -0x3c,0xf2,0x21,0x3c,0x13,0x26,0xc3,0x3b,0x36,0x38,0x49,0x20, -0xba,0x11,0x26,0xe1,0xab,0xad,0xd7,0xfc,0x0a,0xeb,0x35,0xb3, -0xd5,0x40,0x9c,0xb1,0x40,0xd0,0xe0,0xff,0x43,0x08,0xd7,0x43, -0xf5,0x13,0x79,0xc3,0x7f,0x43,0x0b,0x52,0x5c,0xa6,0x2a,0xc4, -0xc8,0x81,0xb8,0x93,0xd9,0x2f,0xa3,0x77,0x99,0x93,0x73,0x82, -0x6b,0x5b,0x1a,0x27,0x22,0x4f,0x27,0xc7,0xd6,0x59,0xb7,0x60, -0x97,0x95,0xc4,0x47,0xba,0xed,0x00,0xa0,0x00,0xa1,0xa6,0x48, -0x00,0x36,0xdd,0xda,0x19,0x90,0x93,0xe4,0xd3,0xaa,0x36,0x43, -0xf1,0x23,0x2b,0xe8,0xe2,0x27,0x8f,0xcf,0xf9,0x30,0x5e,0x01, -0x8c,0x08,0x07,0x5d,0x6b,0xcf,0x20,0xe3,0x8a,0xd2,0x2f,0xfe, -0x7a,0xa4,0xbe,0xa2,0xbd,0x62,0x38,0x3e,0x30,0xe9,0xc6,0xdc, -0x09,0xad,0x3f,0x8a,0xc7,0x4e,0x7b,0x67,0x98,0x0d,0x9f,0x6b, -0xfa,0xd5,0x1f,0x8f,0xfc,0x85,0xa3,0x93,0x4b,0x69,0xa3,0xf1, -0x59,0x23,0xf0,0xca,0x6d,0x32,0x50,0xd9,0x42,0xee,0x67,0x50, -0x96,0x67,0xd5,0x01,0x3d,0x82,0x16,0xf5,0x35,0x3d,0xde,0xbb, -0x47,0xa0,0x0a,0xa7,0x09,0xa8,0x60,0x28,0x81,0x2b,0xf3,0x00, -0xbd,0xd4,0x0c,0x2c,0xbe,0x12,0xaf,0x8f,0x85,0x06,0x95,0x72, -0x9f,0xbf,0x2d,0x77,0x93,0xbc,0xbc,0xbb,0x4b,0x7f,0xf7,0x27, -0xf3,0x69,0x43,0x8f,0x70,0x71,0x08,0x55,0x64,0xc1,0x6b,0x84, -0x03,0xda,0x16,0x82,0xe1,0x44,0x4d,0xaa,0x6b,0x33,0xe4,0x80, -0x1c,0x24,0xdd,0x46,0x74,0xb7,0xa0,0x96,0x77,0xea,0x7a,0x67, -0xb4,0xbb,0xab,0x59,0xe1,0x6e,0xdd,0x10,0xae,0x99,0xa4,0x6e, -0x28,0x66,0x27,0x1d,0x19,0xff,0x17,0x00,0x00,0xff,0xff,0x67, -0x2f,0x5b,0x4f,0xc4,0x23,0x06,0x00, - })) + var empty [0]byte + sx := (*reflect.StringHeader)(unsafe.Pointer(&_scripts_browser_modules_js)) + b := empty[:] + bx := (*reflect.SliceHeader)(unsafe.Pointer(&b)) + bx.Data = sx.Data + bx.Len = len(_scripts_browser_modules_js) + bx.Cap = bx.Len + + gz, err := gzip.NewReader(bytes.NewBuffer(b)) if err != nil { panic("Decompression failed: " + err.Error()) } - var b bytes.Buffer - io.Copy(&b, gz) + var buf bytes.Buffer + io.Copy(&buf, gz) gz.Close() - return b.Bytes() + return buf.Bytes() } + func init() { go_bindata["/scripts/browser-modules.js"] = scripts_browser_modules_js } diff --git a/mod/dashboard/resources/scripts-browser-scripts.js.go b/mod/dashboard/resources/scripts-browser-scripts.js.go index 661289c1a5f..92315f0e436 100644 --- a/mod/dashboard/resources/scripts-browser-scripts.js.go +++ b/mod/dashboard/resources/scripts-browser-scripts.js.go @@ -4,229 +4,36 @@ import ( "bytes" "compress/gzip" "io" + "reflect" + "unsafe" ) +var _scripts_browser_scripts_js = "\x1f\x8b\x08\x00\x00\x09\x6e\x88\x00\xff\xac\x58\x8d\x73\xdb\xb6\x15\xff\x57\x64\xd4\x71\xc9\x33\x45\xdb\xcd\xc7\x6e\x54\x39\x6f\x4d\x7c\xb7\xde\xe2\x2c\xd7\xb8\xed\x76\x3e\xdf\x0e\x12\x21\x89\x35\x05\x68\x24\x64\x59\x53\xf8\xbf\xef\xf7\xf0\x41\x91\x12\xed\xa5\xbb\x64\xab\x45\x3c\xe0\x3d\xfc\xde\x37\x80\xa3\xe9\x4a\x4e\x74\xae\x64\x20\xc2\x6d\x3e\x0d\x98\x1f\xb3\x34\xd5\x9b\xa5\x50\xd3\xc1\x58\x29\x5d\xe9\x92\x2f\xc3\xe6\x2b\x60\x72\x36\xd4\xf9\x42\x0c\x4b\x51\x70\x9d\x3f\x08\x16\x89\x70\x24\x8a\x4a\x0c\x48\x88\x1a\xff\x26\x26\x7a\x27\x42\x3c\x2e\x55\xa9\xab\x70\xa1\xb2\x55\x21\x62\x37\x4c\x45\xd0\xe2\x39\xdc\x38\x13\xd3\x5c\x8a\x93\x13\xfb\x1b\xf3\x45\x16\xda\xcf\xa0\xbd\xd7\x4a\x5a\x62\xc6\x8e\x3c\x63\x25\x2a\xa3\xcc\x11\x3e\x62\x75\x1f\x84\x61\x29\xf4\xaa\x94\x23\x1a\x2f\xf8\xbd\xf8\x30\xbb\x01\xf8\x9f\x1c\xf6\x54\xd4\x24\xad\x4f\xd2\x3a\x97\x99\x5a\x5f\xda\x9f\x58\xee\xb1\x05\x61\x32\x2b\xd4\x98\x17\x3d\x33\x75\xd0\x58\x36\xdc\xda\xed\x07\x3b\x5b\x47\x3a\x92\xe1\xd6\x8f\x07\x79\x20\x23\x07\x59\xdf\xca\x3b\xfb\x25\xcc\xd7\x03\x2f\x07\x2a\xed\xb1\x4e\x29\xfe\xbd\xca\x4b\x98\xc7\x7d\x8c\x8c\xc2\x27\x27\xca\x29\x3b\x50\x10\x7a\x74\x1e\x12\xbd\xf4\xb4\xd2\xd1\xf4\xbc\x54\xeb\x81\x14\xeb\xc1\x55\x59\xaa\x32\x60\x6f\xb9\x94\x4a\x0f\xa0\x7e\x36\xb0\x5e\x1a\x7c\xcb\x4e\xe5\x29\xfb\x96\x85\x35\x61\x58\xa5\x84\x2c\xdd\x3a\xdf\x25\xdb\xba\x1e\x11\xc2\xdb\xf3\xbb\x9d\xa6\xda\xe2\x2d\x53\x33\x73\x71\x77\xab\xef\x46\x6e\xe7\x3c\x28\x2f\xcb\x44\x87\x75\xb4\x8a\x56\x3e\x02\xc2\xda\xcd\x92\x6c\x4f\xac\xa7\x00\x64\xc5\x7c\x81\xda\x51\x95\x9e\x8f\xaa\xef\x65\x5c\x08\x39\xd3\xf3\x51\x75\x7a\x1a\xc2\x9c\xb7\xd5\x5d\xd8\x6c\x5d\x07\xdb\x8b\xe4\xb6\x41\xe9\x39\xad\x9e\x91\xc7\xb2\x65\x2b\x84\x14\xe2\x3b\x47\xe8\x8e\x1a\xdf\x64\x58\x3a\x21\xaf\x06\xc7\x14\xf1\x6a\xa5\xc1\xb8\x10\x52\x7b\xb7\xe2\xc7\x32\x25\xec\x2f\x6f\x59\x54\x4d\xd4\x52\x24\xdb\x8c\x6b\x41\xeb\x13\xf6\x67\x56\x47\x45\x2e\xef\x93\x06\x80\x59\x12\x89\x42\x90\x98\x88\x6b\x5d\x56\xd6\x70\x6e\x83\x91\x59\x10\x1f\xaf\xb9\x9e\xcc\x03\xe6\x45\xb1\xa8\x91\x40\xa4\x4f\xd8\x54\xce\x5a\x51\xb4\x5a\x12\x99\x02\x11\x21\xe7\xa4\xc7\x5a\x3c\xea\x20\xcb\xa7\x53\xbb\xdc\x70\x46\x5a\x21\x27\xc2\xba\xa5\x63\x33\xcf\xa3\x71\x13\xaf\xd7\x5c\xcf\x63\x3e\xae\x02\x1e\x4f\x0a\x85\xbc\x0b\xe3\x4a\xf3\x52\xff\x7d\x4a\xa0\x36\x2c\x8c\x89\x31\x18\x47\x34\xaa\x18\x45\x56\xf8\xfd\xc5\x25\x8f\xa7\xa5\x5a\x80\xbc\xce\xf5\x1c\xea\x7c\x5a\x4d\xa7\xf9\x63\x98\x40\x0a\x87\x97\x32\x5e\x06\xe3\xd6\xe6\x16\xf6\x7b\xfc\x57\x02\x77\x5b\x89\xc8\xd9\x23\xf5\x96\x6f\xe7\x54\x87\xad\x8e\x24\xf4\xfc\xd9\x90\x7e\xc4\x6c\x4b\x7c\x77\xc2\x98\x39\x13\x85\xe6\x69\xa3\x9d\x75\x67\xe0\xb4\xa1\x95\x61\x13\x3b\xaf\x5e\x8b\x97\x7f\x32\xeb\x2f\xe9\x73\x68\x3e\x93\x3f\x8a\x57\x8e\x88\x2f\x47\xfb\xee\x0f\xe2\xb5\x23\xbe\x01\x31\x30\x9f\xa7\x2f\xc5\xab\xf0\x05\xc6\xc9\xcb\x37\x6f\xfc\xd2\x17\x2f\xdf\x88\xd7\xb5\xd7\x09\x46\x91\x13\x51\x04\x6e\x18\x8e\x0c\x42\x80\x48\x1d\xae\x96\xaf\x29\x97\x0d\x3e\x1b\x2d\x2a\x3d\xac\x31\x8e\xc9\x04\x55\xac\x15\x2c\xd3\xf1\x42\xca\x3a\x43\x96\xcb\x81\x59\x3a\x72\x0c\xb9\x2e\xc4\xe7\xcf\x3e\x78\x88\x18\x30\x43\x64\x11\x6d\x1c\x23\x37\x17\x5c\x07\xec\x3d\xfe\xb1\x30\xf4\x41\x1c\x8f\x51\x38\x02\x76\x9c\x51\x2a\xa8\x4d\x2b\x52\xc3\xed\x53\x8a\xd6\x61\xd4\xf5\x61\x8d\xff\xd7\xd0\xd0\xb7\x87\xbd\x6e\xd1\x88\xb4\xf4\x70\xeb\xe6\x27\x4a\x22\x2a\xa1\x33\xd3\xad\x1a\xfc\x56\xc9\x69\x3e\x63\xd1\xd6\xc7\x5c\xb2\x15\x32\xd9\x16\xbc\xd2\xef\xf8\x26\x61\xb7\xff\x04\x54\x51\x22\x70\xef\xa2\xc1\xfb\x1b\xe4\x2d\x5f\x08\x3b\x73\xa3\x76\x54\x8a\x1e\x4f\x5d\x28\xd4\xca\xb5\x9b\x20\x41\xbf\x0a\x71\x9f\xb0\x0c\xff\x76\x8b\x2d\xed\x03\xbe\x06\xbb\x09\x92\x7d\x85\x16\x93\xc0\x6e\x0c\x4a\x52\xa8\xf9\xaa\xc2\x76\x2d\xf4\x96\x79\x5b\xb1\x88\x59\x47\xc2\xec\x7e\xe5\x5d\x18\x97\x2b\x19\xb4\xcc\x60\xca\xc7\xa1\xd2\xe1\x96\xcb\xd9\xaa\xe0\x25\x79\xeb\x8a\xa3\x84\x1c\xae\x69\x52\x71\xe7\x29\x34\x76\x59\xd1\x1a\x25\xa1\x1d\x55\x16\xbb\x43\x4c\x83\x80\xfe\xb4\x8c\xd9\x5a\x0c\xb7\xd1\xff\x7c\x1d\xdd\xa9\x96\x36\x5f\x91\x03\x64\xe3\x16\x7e\x4b\x3d\x42\xeb\xc3\xae\xeb\x60\x08\x14\x6f\xbb\xf9\xc9\x09\xb9\xbb\xe5\x63\x6f\x16\x57\x83\xa3\x36\x46\x26\x24\x1c\x5e\x53\x58\x5a\x2c\xc4\x8b\xa0\x02\xed\x8e\xfe\x44\x68\x49\x14\x76\xfb\x9b\x0b\x3d\xc9\xcc\xa6\xf1\x94\x4f\xb4\x2a\x37\x01\xbb\x02\xe9\x97\xef\x8c\x4b\xe6\x5a\x2f\x5b\x01\x6d\xc6\xad\xaa\x3b\x29\x04\x97\xab\xe5\x47\x54\x93\x60\x89\x3f\x56\xc7\x25\xa7\x90\xa5\x71\x5c\x2d\x8b\x1c\xc0\xcf\x58\x53\x54\xce\xd3\x34\x35\x0b\x5c\xd3\xba\x64\x2c\x09\x3c\x07\x91\xa7\x79\x41\x29\xd1\xec\xf9\xe0\xd3\x9b\xe1\x68\x92\x3e\x40\x85\xf6\xea\xdf\x54\x2e\x8d\xfc\x4e\xd5\x5b\xff\x4d\x6c\x82\x7b\xb1\xf9\x80\xd8\xb3\x98\x2a\x51\x4c\xd3\x6d\xed\x51\xd0\x30\x96\x98\x4d\xdb\x2a\x78\x8e\xc8\x4c\xcf\x84\xfe\xc8\x4b\x18\xb8\x5d\x69\x76\xfa\x35\x22\xbe\x44\x49\x07\x89\xb1\xd0\x29\x1b\x2f\xd5\x12\x45\xde\xd1\xf7\x75\x41\x44\x19\xf1\x64\xc3\xc3\xdd\x41\xc3\xaa\xd3\x3d\xe4\x1f\x4b\x1c\xe0\x1e\x4f\x1b\x58\x0d\x18\xc3\x90\xa6\xcd\x92\xb8\x5a\x8d\x2b\xdb\xf0\xce\xa3\x1d\xd5\x22\x1d\x5e\x84\x97\x0d\x2d\x21\xd6\xba\xb1\x46\x4f\xc5\x35\x01\x41\x73\x41\x83\x37\x68\xd0\x57\x6d\x16\x08\xfd\x85\x17\x2b\xd1\x65\x0d\xb6\xab\xb2\x48\x5a\xbc\x54\x69\x79\x72\x8c\x51\xc9\x17\x01\x14\x06\x4b\xe2\x79\xe1\xfb\x85\x40\x05\xcf\x12\xf6\xf1\x67\x94\x96\xb9\xe0\x99\x28\x71\x1e\x63\x48\x6c\x0d\x4f\x0d\x6f\x70\x50\x62\x09\xe3\x4b\xb8\x64\x62\x32\xf4\xec\x71\xb8\x5e\xaf\x87\x54\xbb\x87\xd8\x4b\xc8\x89\xca\x70\xca\xa5\x82\x6b\x51\xa2\x2d\x09\x2d\xe0\x85\xa7\xd4\x3b\xc4\xe8\x41\xbc\xbb\x7a\x7f\x75\x73\xf5\x75\x70\x74\xc2\xf7\x93\x46\x9f\x41\xd2\xeb\xdf\x19\xc0\x0d\x4b\x7f\xf8\xb8\x3c\xda\x8b\x1d\x62\xaa\x0e\xa2\xe7\xff\x76\xbb\x39\x31\x37\x21\x84\x48\x7d\xf8\xee\x0c\xc3\xea\x0c\xad\x60\xb7\x93\xa5\x1b\xc2\x19\xf3\x4a\x59\x57\x38\xe9\xb8\x8f\xf0\x55\x81\xb4\x70\xd6\x45\x1d\x5c\x2c\x94\xbc\x65\xff\x18\xfe\x84\x43\x2c\x75\xb0\x6c\xf8\x2b\x1a\x3a\xbb\x8b\xb6\x80\x42\x36\x4b\x9c\xed\x22\x8c\xe1\xd1\xc4\xa6\x57\x5d\xdf\xf5\xd7\xbd\x1f\xd0\xd2\x2a\x51\x52\xa5\x93\xb3\x9f\xd0\x7a\x50\x7e\x5d\x41\xec\x16\x65\x54\xc7\x5d\x15\x6e\x74\xc3\xaa\x9d\x72\x66\x05\x9a\x4b\x80\xaa\x59\x92\xa8\x8f\xa5\x7a\xc8\x33\x92\xde\xe6\xd8\x95\xd2\xce\xa2\x5d\x1e\x86\xdb\x42\xa1\xd3\x7c\x42\x29\xe6\x33\x41\x36\xfe\x51\x8b\x85\xc5\xfb\x2f\x32\x34\xaa\x4c\x97\x37\x5e\xcf\x85\xa9\x1b\x11\xdc\x63\xdb\xce\x8d\x4a\x1a\x81\x68\xb8\x4a\xcf\x45\xb9\xce\x2b\x11\x6c\x21\x6b\x09\xa5\xc4\xcf\x08\x69\xf6\x90\x8b\x75\x75\x36\xb6\x56\x88\xe7\x7a\x51\xb0\x08\x5a\xe0\x14\x53\x14\xa2\x4c\xd8\x35\xcf\xe5\x5b\x5d\x16\x0c\x41\x6a\x4d\xe0\xa6\x82\xdd\x14\x75\x09\x73\x64\x87\x9e\xc7\x04\xdd\x5c\x59\xa2\xa6\x89\xf4\x2b\x6f\x6f\x01\xcd\xfa\xc8\xae\x6e\x5b\xc1\x2e\x89\x2b\x8e\x5e\x6a\x94\x1f\xd2\xe7\x70\x0e\x8d\x59\xe4\x26\x97\xa5\x20\x15\xdc\xbc\x1b\x75\x97\x08\xc9\xc7\x85\xf8\x81\x4f\xee\xd3\xa3\x73\x4f\x5c\x97\xb9\x46\xf9\xfb\x00\xd6\xa3\x8b\x66\x25\x64\x50\x3e\xa4\x0d\x2a\x9f\xee\xc7\xdd\x2b\x89\x5f\xd8\x39\xe8\x35\xd9\xeb\x67\xa9\xac\x37\xf9\xb2\x74\x84\xbd\xad\x5a\x3d\xcb\xaf\x78\xa6\x8b\xee\x8c\xe3\xf3\xe5\xa2\xa7\x97\x9a\x31\x7a\x7d\xdd\x09\xa3\xaa\x1d\x46\x16\xfc\x3e\x96\x67\x2d\xd6\x18\xa7\xd5\x47\x70\x3c\xe9\x61\xb9\x68\x04\x61\x61\x6a\xfd\x1a\xdb\x74\x0c\xda\xb6\xd9\xdf\x9e\xce\x26\x7b\x86\x86\x80\xee\x61\x7a\xdf\x7b\x38\x0f\x1c\x9d\x9f\x9c\xec\xf6\x33\x25\x09\x17\xb5\xd5\x64\x22\xaa\x2a\x68\x5f\x19\x39\xf8\x03\xf6\x8d\x89\x14\x1b\xf1\x43\x41\xd7\x7f\x24\x2e\x05\x8c\xeb\x3c\x74\x80\x4b\x49\xea\xa5\x07\x58\xe4\x95\x4e\xcd\xd4\xfd\x43\xf5\x05\x81\x87\x06\xef\x43\x17\x20\x0b\x61\x1a\x97\x15\x60\x9a\xd9\xf3\x22\xf0\x23\x78\xc1\xa2\xae\x4a\xf6\x18\x02\xc5\x9e\x57\xaf\x07\x30\x9d\x07\xbb\x1b\x5e\x83\x13\x11\x91\xb2\x0f\x8a\xaa\x34\x35\x12\x14\x0c\x91\xc5\x48\xf2\xd8\x58\xe4\x09\xb9\xfb\xfc\x70\xe2\x20\x53\xa2\x1a\xd0\xdb\x89\x78\xc4\xae\x71\x83\xbb\x9a\xab\xb5\x2d\xae\xf6\x89\xc5\xc0\x59\x58\x4e\x73\x62\xf6\x0b\xc7\x14\x33\x87\x2e\xde\xe5\x62\xbf\x1d\xba\x89\x59\x6d\xe4\xe4\xbd\xcb\xd9\x60\x5f\xdf\xe7\x8a\x43\xa7\x0e\xd4\x7d\xe2\x3a\xd8\xba\x75\xe1\x20\x97\xdb\xca\x7f\xa2\xb2\x75\xa8\xd7\x7e\x35\x73\xee\xde\xb1\x82\xf8\x0e\xb6\xea\x61\x25\x13\x20\x8b\x7b\x82\xeb\xf3\x67\x36\x60\x3d\x51\xf1\xd5\x6a\x28\x39\x69\x67\xd7\xae\xd1\x9e\x8f\x19\x6f\x8a\xfe\x28\xf0\x12\x7b\x4f\x60\x2d\xb5\x9b\xf9\xbe\xd8\xff\xca\x5a\x1e\x2a\xd4\x51\xa6\x1d\xd4\x38\xec\xad\x8a\xcc\x84\xbf\x3b\xb8\xa0\xcd\x52\x4e\xb1\xb6\x72\x3c\xcb\xbe\x3c\x10\x7e\x57\x71\x68\xd7\x17\x06\x6a\xc0\xe2\x56\x75\x2b\x87\xf6\xb0\x10\x4f\xcd\xeb\x44\x2e\x97\xb8\x59\x63\xa4\x26\xab\xaa\xdf\x99\xe7\xf5\x13\xe9\xdb\x7a\x7d\x70\xce\x7b\xb2\x94\xda\xcd\x2c\x0e\xdc\x70\xd4\x03\x70\x4c\xec\x99\x18\xb3\xe6\x3d\x8e\x19\x99\xc9\x80\x9d\x7a\x69\xd1\x53\xd2\x60\xbb\xb7\x05\x87\xb7\x59\x47\x60\x99\xcf\xe6\x24\x8e\x60\x06\x87\x79\xf7\x3f\x31\x1b\x8b\x7f\x25\xc4\x1d\x59\x4f\xe0\x2d\xc4\xb4\x07\x2e\xea\xd9\x5f\x05\x69\xd2\x77\xbe\x0e\xec\xab\x3b\x7a\x93\x59\xd2\x62\x73\xcd\x71\x5f\x48\xa7\x4f\x36\x91\xa0\x32\x7a\xa8\x9c\x10\x22\x2b\x88\x1d\xec\x1e\x84\xc3\x57\xaf\xa9\x22\xbb\x77\x7e\x25\x4b\x51\xe5\xff\xe9\xab\x5e\xc7\x74\x9d\xd9\xd0\x63\xd5\x5d\xe7\x11\x47\xce\xae\x60\xaf\xb2\xd3\xab\xf7\x5f\xfd\x7b\x9f\x7d\xbb\x0f\x68\xc8\x1c\x00\x90\x94\x41\x88\xf8\xaa\x6a\xc9\x43\xe0\xd3\x9b\xf3\xc5\x4b\xf4\x66\xf3\x8d\x33\x6f\x3e\x99\xe3\x04\xd2\x41\xd6\xda\xdf\xd1\x05\x7a\xae\x7b\x0f\x74\x28\x49\x55\x2b\x82\xd2\x0a\xbf\xef\xec\x3d\x83\xee\x2f\xe1\xde\xeb\xd4\x1c\x06\x2a\xac\xd9\x0e\x54\x6b\xbf\x7c\xb3\x2f\x79\xe3\x66\xdf\xb0\xd3\x83\xc3\x94\x85\x36\xc7\x79\xea\xe4\xc4\x5b\x63\xb9\xeb\x72\xee\x63\x2f\xae\x7c\xcb\x66\xf4\x66\xd8\xf7\x0a\xe4\x5f\xd5\xe8\x25\x39\xc1\xd1\x14\x78\x45\xc2\xae\x1e\x97\x50\xad\x1a\xe4\x72\xf0\x02\xc6\x5d\xf2\x4a\x7b\x62\x06\xca\x80\xcf\x14\xae\x68\x09\xab\x04\xe2\x3f\xc3\x8a\x05\x2e\xb0\x83\x45\x2e\xcd\xbd\x68\x81\xd1\x8b\xcc\x0d\x31\x39\xc7\xa4\x1c\xcc\xd5\x0a\x7e\x9f\xcf\xcd\x1c\x0d\x30\x93\x11\x1b\x3d\x91\x47\x59\x66\xe8\xf6\x81\xfc\xda\x48\x43\x66\xe1\xec\x79\x7d\x6d\x85\xd1\x08\x53\x1b\x9a\xda\x08\x0e\x51\x9b\x8d\x99\xa1\x41\x45\x97\xe3\xd1\x7f\x03\x00\x00\xff\xff\xc6\x40\xd8\xa2\xa5\x1b\x00\x00" + // scripts_browser_scripts_js returns raw, uncompressed file data. func scripts_browser_scripts_js() []byte { - gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ -0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0xac,0x58, -0x0b,0x73,0xdb,0xc6,0xf1,0xff,0x2a,0xd0,0x45,0x56,0x80,0x11, -0x08,0x49,0x7f,0x3f,0xfe,0x53,0x30,0xa8,0xdb,0xd8,0x9a,0x69, -0xa6,0xb6,0xe3,0x89,0x94,0xa4,0x1d,0x8d,0xa6,0x73,0x22,0x8e, -0x24,0x22,0xf0,0x8e,0x05,0x8e,0xa2,0x58,0x1a,0xdf,0xbd,0xbf, -0xbd,0x07,0x08,0x90,0x90,0x6b,0xcf,0xd8,0x6d,0x44,0xdc,0xde, -0xee,0xde,0xbe,0x77,0xef,0x8e,0xa6,0x2b,0x39,0xd1,0x85,0x92, -0xa1,0x88,0xb6,0xc5,0x34,0x64,0x7e,0xcd,0xb2,0x4c,0x6f,0x96, -0x42,0x4d,0x83,0x3b,0xa5,0x74,0xad,0x2b,0xbe,0x8c,0xda,0xaf, -0x90,0xc9,0xd9,0x48,0x17,0x0b,0x31,0xaa,0x44,0xc9,0x75,0xf1, -0x20,0x58,0x2c,0xa2,0xb1,0x28,0x6b,0x11,0x10,0x13,0x75,0xf7, -0x87,0x98,0xe8,0x1d,0x0b,0xf1,0xb8,0x54,0x95,0xae,0xa3,0x85, -0xca,0x57,0xa5,0x48,0xdc,0x32,0x13,0x61,0x87,0xe6,0xf0,0xe0, -0x5c,0x4c,0x0b,0x29,0x4e,0x4e,0xec,0x6f,0xc2,0x17,0x79,0x64, -0x3f,0xc3,0xee,0x59,0x2b,0x69,0x81,0x39,0x3b,0xf2,0x84,0xb5, -0xa8,0x8d,0x32,0x47,0xf8,0x48,0xd4,0x7d,0x18,0x45,0x95,0xd0, -0xab,0x4a,0x8e,0x69,0xbd,0xe0,0xf7,0xe2,0xc3,0xec,0x1a,0xc2, -0xff,0xe2,0x64,0xcf,0x44,0x43,0xdc,0x86,0x38,0xad,0x0b,0x99, -0xab,0xf5,0x6b,0xfb,0x93,0xc8,0x3d,0xb2,0x30,0x4a,0x67,0xa5, -0xba,0xe3,0xe5,0xc0,0x4e,0x13,0xb6,0x96,0x8d,0xb6,0xf6,0xf8, -0x60,0x67,0xeb,0x58,0xc7,0x32,0xda,0xfa,0x75,0x50,0x84,0x32, -0x76,0x22,0xeb,0x1b,0x79,0x6b,0xbf,0x84,0xf9,0x7a,0xe0,0x55, -0xa0,0xb2,0x01,0xeb,0x54,0xe2,0xdf,0xab,0xa2,0x82,0x79,0xdc, -0xc7,0xd8,0x28,0x7c,0x72,0xa2,0x9c,0xb2,0x81,0x02,0xd3,0xa3, -0xf3,0x88,0xe0,0x95,0x87,0x55,0x0e,0xa6,0xe7,0x95,0x5a,0x07, -0x52,0xac,0x83,0xcb,0xaa,0x52,0x55,0xc8,0xde,0x70,0x29,0x95, -0x0e,0xa0,0x7e,0x1e,0x58,0x2f,0x05,0xdf,0xb3,0x53,0x79,0xca, -0xbe,0x67,0x51,0x43,0x32,0xac,0x32,0x92,0x2c,0xdb,0x3a,0xdf, -0xa5,0xdb,0xa6,0x19,0x93,0x84,0x37,0xe7,0xb7,0x3b,0x4d,0xb5, -0x95,0xb7,0xca,0xcc,0xce,0xc5,0xed,0x8d,0xbe,0x1d,0xbb,0x93, -0x8b,0xb0,0x7a,0x5d,0xa5,0x3a,0x6a,0xe2,0x55,0xbc,0xf2,0x11, -0x10,0x35,0x6e,0x97,0x78,0x7b,0x60,0x33,0x85,0x40,0x96,0xcd, -0x17,0xa8,0x1d,0xd7,0xd9,0xf9,0xb8,0xfe,0x41,0x26,0xa5,0x90, -0x33,0x3d,0x1f,0xd7,0xa7,0xa7,0x11,0xcc,0x79,0x53,0xdf,0x46, -0xed,0xd1,0x4d,0xb8,0xbd,0x48,0x6f,0x5a,0x29,0x3d,0xa5,0xd5, -0x33,0xf6,0xb2,0x6c,0xd9,0x0a,0x21,0x85,0xf8,0x2e,0x10,0xba, -0xe3,0xd6,0x37,0x39,0x50,0x27,0xe4,0xd5,0xf0,0x98,0x22,0x5e, -0xad,0x34,0x08,0x17,0x42,0x6a,0xef,0x56,0xfc,0x58,0xa2,0x94, -0xfd,0xf5,0x0d,0x8b,0xeb,0x89,0x5a,0x8a,0x74,0x9b,0x73,0x2d, -0x08,0x3f,0x65,0x7f,0x61,0x4d,0x5c,0x16,0xf2,0x3e,0x6d,0x05, -0x30,0x28,0xb1,0x28,0x05,0xb1,0x89,0xb9,0xd6,0x55,0x6d,0x0d, -0xe7,0x0e,0x18,0x1b,0x84,0xe4,0x78,0xcd,0xf5,0x64,0x1e,0x32, -0xcf,0x8a,0xc5,0x2d,0x07,0x02,0x5d,0xe1,0x50,0x39,0xeb,0x44, -0xd1,0x6a,0x49,0x60,0x0a,0x44,0x84,0x9c,0xe3,0x9e,0x68,0xf1, -0xa8,0xc3,0xbc,0x98,0x4e,0x2d,0xba,0xa1,0x8c,0xb5,0x42,0x4e, -0x44,0x4d,0x47,0xc7,0x76,0x9f,0xc7,0x77,0x6d,0xbc,0xbe,0xe7, -0x7a,0x9e,0xf0,0xbb,0x3a,0xe4,0xc9,0xa4,0x54,0xc8,0xbb,0x28, -0xa9,0x35,0xaf,0xf4,0xcf,0x53,0x12,0x6a,0xc3,0xa2,0x84,0x08, -0xc3,0xbb,0x98,0x56,0x35,0xa3,0xc8,0x8a,0x7e,0xb8,0x78,0xcd, -0x93,0x69,0xa5,0x16,0x00,0xaf,0x0b,0x3d,0x87,0x3a,0x57,0xab, -0xe9,0xb4,0x78,0x8c,0x52,0x70,0xe1,0xf0,0x52,0xce,0xab,0xf0, -0xae,0x73,0xb8,0x15,0xfb,0x1d,0xfe,0xab,0x20,0x77,0x57,0x89, -0xd8,0xd9,0x23,0xf3,0x96,0xef,0xe6,0x54,0x8f,0xac,0x89,0x25, -0xf4,0xfc,0xd5,0x80,0x7e,0xc2,0x6e,0x87,0x7d,0x7f,0xc3,0x98, -0x39,0x17,0xa5,0xe6,0x59,0xab,0x9d,0x75,0x67,0xe8,0xb4,0x21, -0xcc,0xa8,0x8d,0x9d,0x17,0x2f,0xc5,0xf3,0x3f,0x1b,0xfc,0xd7, -0xf4,0x39,0x32,0x9f,0xe9,0x9f,0xc4,0x0b,0x07,0xc4,0x97,0x83, -0xfd,0xdf,0xff,0x8b,0x97,0x0e,0xf8,0x0a,0xc0,0xd0,0x7c,0x9e, -0x3e,0x17,0x2f,0xa2,0x67,0x58,0xa7,0xcf,0x5f,0xbd,0xf2,0xa8, -0xcf,0x9e,0xbf,0x12,0x2f,0x1b,0xaf,0x13,0x8c,0x22,0x27,0xa2, -0x0c,0xdd,0x32,0x1a,0x1b,0x09,0x21,0x44,0xe6,0xe4,0xea,0xf8, -0x9a,0x72,0xd9,0xc8,0x67,0xa3,0x45,0x65,0x87,0x35,0xc6,0x11, -0x99,0xa0,0x4a,0xb4,0x82,0x65,0x7a,0x5e,0xc8,0x58,0x6f,0xc9, -0x0a,0x19,0x18,0xd4,0xb1,0x23,0x28,0x74,0x29,0x3e,0x7d,0xf2, -0xc1,0x43,0xc0,0x90,0x19,0x20,0x8b,0xe9,0xe0,0x04,0xb9,0xb9, -0xe0,0x3a,0x64,0xef,0xf0,0x8f,0x45,0x91,0x0f,0xe2,0xe4,0x0e, -0x85,0x23,0x64,0xc7,0x39,0xa5,0x82,0xda,0x74,0x22,0x35,0xda, -0x3e,0xa5,0x68,0x13,0xc5,0x7d,0x1f,0x36,0xf8,0x7f,0x03,0x0d, -0x7d,0x7b,0xd8,0xeb,0x16,0x2d,0x4b,0x0b,0x8f,0xb6,0x6e,0x7f, -0xa2,0x24,0xa2,0x12,0x3a,0x33,0xdd,0xa9,0xc1,0x6f,0x94,0x9c, -0x16,0x33,0x16,0x6f,0x7d,0xcc,0xa5,0x5b,0x21,0xd3,0x6d,0xc9, -0x6b,0xfd,0x96,0x6f,0x52,0x76,0xf3,0x4f,0x88,0x2a,0x2a,0x04, -0xee,0x6d,0x1c,0xbc,0xbb,0x46,0xde,0xf2,0x85,0xb0,0x3b,0xd7, -0x6a,0x07,0xa5,0xe8,0xf1,0xd0,0x85,0x42,0xad,0x5c,0xbb,0x0d, -0x62,0xf4,0xbb,0x10,0xf7,0x29,0xcb,0xf1,0x6f,0x87,0x6c,0x61, -0x1f,0xf0,0x15,0xec,0x36,0x88,0xf7,0x25,0x5a,0x4c,0x0a,0xbb, -0x31,0x28,0x49,0xa1,0xe6,0xab,0x0a,0xdb,0xb5,0xd0,0x1b,0xe6, -0x6d,0xc5,0x62,0x66,0x1d,0x09,0xb3,0x7b,0xcc,0xdb,0x28,0xa9, -0x56,0x32,0xec,0x98,0xc1,0x94,0x8f,0x43,0xa5,0xa3,0x2d,0x97, -0xb3,0x55,0xc9,0x2b,0xf2,0xd6,0x25,0x47,0x09,0x39,0xc4,0x69, -0x53,0x71,0xe7,0x29,0x34,0x76,0x59,0x13,0x8e,0x92,0xd0,0x8e, -0x2a,0x8b,0x3d,0x21,0xa1,0x45,0x48,0x7f,0x3a,0xc6,0xec,0x20, -0xc3,0x6d,0xf4,0x3f,0x5f,0x47,0x77,0xaa,0x65,0xed,0x57,0xec, -0x04,0xb2,0x71,0x0b,0xbf,0x65,0x5e,0x42,0xeb,0xc3,0xbe,0xeb, -0x60,0x08,0x14,0x6f,0x7b,0xf8,0xc9,0x09,0xb9,0xbb,0xe3,0x63, -0x6f,0x16,0x57,0x83,0xe3,0xae,0x8c,0x4c,0x48,0x38,0xbc,0xa1, -0xb0,0xb4,0xb2,0x10,0x2d,0x82,0x0a,0xb0,0x5b,0xfa,0x13,0xa3, -0x25,0x51,0xd8,0xed,0x1f,0x2e,0xf4,0x24,0x37,0x87,0x26,0x53, -0x3e,0xd1,0xaa,0xda,0x84,0xec,0x12,0xa0,0xdf,0x2e,0x8c,0x4b, -0xe6,0x5a,0x2f,0x3b,0x01,0x6d,0xd6,0x9d,0xaa,0x3b,0x29,0x05, -0x97,0xab,0xe5,0x47,0x54,0x93,0x70,0x89,0x3f,0x56,0xc7,0x25, -0xa7,0x90,0xa5,0x75,0x52,0x2f,0xcb,0x02,0x82,0x9f,0xb1,0xb6, -0xa8,0x9c,0x67,0x59,0x66,0x10,0x5c,0xd3,0x7a,0xcd,0x58,0x1a, -0x7a,0x0a,0x02,0x4f,0x8b,0x92,0x52,0xa2,0x3d,0xf3,0xc1,0xa7, -0x37,0xc3,0x68,0x92,0x3d,0x40,0x05,0x8b,0xf7,0x87,0x2a,0xa4, -0xe1,0xdc,0xab,0x77,0xeb,0xbf,0x8b,0x4d,0x78,0x2f,0x36,0x1f, -0x10,0x75,0x56,0x9a,0x5a,0x94,0xd3,0x6c,0xdb,0xf8,0xf3,0x69, -0x99,0x48,0xec,0x66,0x5d,0xe1,0x3d,0x45,0x6c,0xb6,0x67,0x42, -0x7f,0xe4,0x15,0x4c,0xdb,0xad,0x31,0x3b,0xcd,0x5a,0x16,0x5f, -0xa2,0x9e,0x13,0x89,0xb1,0xc8,0xa9,0x99,0x2c,0xd5,0x12,0xe5, -0xdd,0xc1,0xf7,0x75,0x41,0x2c,0x19,0xf6,0x64,0xbd,0xc3,0x0a, -0x07,0x94,0xd3,0x3d,0xb1,0x3f,0x56,0x98,0xdb,0x1e,0x4f,0x5b, -0x99,0x3c,0x03,0xe8,0x30,0x50,0x21,0x8d,0x03,0x69,0x2f,0x6c, -0x4f,0x09,0xdb,0x33,0xeb,0x2e,0x09,0x58,0xff,0xc6,0xcb,0x95, -0xe8,0x93,0x86,0xdb,0x55,0x55,0xa6,0x1d,0x5a,0xaa,0x8c,0x3c, -0x3d,0xc6,0xaa,0xe2,0x8b,0x10,0x46,0x02,0x49,0xea,0x69,0xe1, -0xab,0x85,0x40,0xc5,0xcd,0x53,0xf6,0xf1,0xe7,0x2b,0xd4,0x82, -0xb9,0xe0,0xb9,0xa8,0x30,0x40,0x31,0x64,0xa2,0x86,0x81,0x47, -0xd7,0x98,0x6c,0x58,0xca,0xf8,0x12,0x96,0x9c,0x98,0x94,0x3a, -0x7b,0x1c,0xad,0xd7,0xeb,0x11,0x15,0xdb,0x11,0x0e,0x13,0x72, -0xa2,0x72,0x8c,0xa5,0x54,0x21,0xad,0x98,0xe8,0x23,0x42,0x0b, -0x18,0xef,0x29,0xfd,0x0e,0x85,0xf4,0x52,0xbc,0xbd,0x7c,0x77, -0x79,0x7d,0xf9,0x6d,0xe4,0xe8,0x45,0xdd,0x95,0x46,0x63,0x40, -0x96,0xea,0xaf,0x8c,0xbb,0x96,0xe4,0x6b,0xbc,0x4e,0x44,0xf5, -0xb7,0xf3,0xbb,0x19,0x71,0xdb,0x48,0xca,0xd8,0xd9,0xc3,0xc5, -0x19,0x96,0xf5,0x19,0x6a,0xf7,0xee,0x24,0x0b,0x37,0x80,0x33, -0xe6,0x95,0xb2,0xae,0x70,0xdc,0x71,0x81,0xe0,0xab,0x12,0xd1, -0xec,0xac,0x8b,0xc2,0xb5,0x58,0x28,0x79,0xc3,0xfe,0x31,0xfa, -0x05,0x53,0x27,0xb5,0x9c,0x7c,0xf4,0x3b,0x3a,0x30,0xbb,0x8d, -0xb7,0x10,0x85,0x6c,0x96,0x3a,0xdb,0xc5,0x58,0xc3,0xa3,0xa9, -0xcd,0x8a,0xa6,0xb9,0x1d,0x2e,0x54,0x3f,0xa2,0x07,0xd5,0xa2, -0xa2,0xd2,0x24,0x67,0xbf,0xa0,0x57,0xa0,0x5e,0xba,0x0a,0xd6, -0xaf,0xa2,0x28,0x67,0xbb,0xb2,0xd9,0xea,0x06,0x2c,0xaf,0x1c, -0x33,0x08,0x68,0x06,0x21,0xaa,0x5c,0x45,0x9c,0x3e,0x56,0xea, -0xa1,0xc8,0x89,0x79,0x97,0x60,0x57,0xfa,0x7a,0x48,0x71,0x8b, -0x12,0x6d,0x4b,0x85,0xce,0x70,0x85,0xd2,0xc9,0x67,0x82,0x4c, -0xfc,0x93,0x16,0x0b,0x2b,0xee,0xbf,0xc8,0xce,0x2c,0x8a,0xfb, -0xb4,0xc9,0x7a,0x2e,0x4c,0xb6,0xc7,0xf0,0x8e,0x6d,0x13,0xd7, -0x2a,0x6d,0x19,0xa2,0x41,0x2a,0x3d,0x17,0xd5,0xba,0xa8,0x45, -0xb8,0x05,0xaf,0x25,0x74,0x12,0xbf,0x22,0xa2,0xd9,0x43,0x21, -0xd6,0xf5,0xd9,0x9d,0x35,0x42,0x32,0xd7,0x8b,0x92,0xc5,0xd0, -0x02,0x53,0x47,0x59,0x8a,0x2a,0x65,0xef,0x79,0x21,0xdf,0xe8, -0xaa,0x64,0x88,0x51,0x6b,0x01,0xb7,0x15,0xee,0xb6,0xa8,0xaa, -0x9b,0x11,0x1b,0x7a,0x1e,0x93,0xe8,0xe6,0x8a,0x11,0xb7,0x45, -0x7f,0x58,0x79,0x3b,0xb5,0xb7,0xf8,0xb1,0xc5,0xee,0x5a,0xc1, -0xa2,0x24,0x35,0x47,0xef,0x33,0xca,0x8f,0xe8,0x73,0x34,0x87, -0xc6,0x2c,0x76,0x9b,0xcb,0x4a,0x90,0x0a,0x6e,0xdf,0xad,0xfa, -0x28,0x42,0xf2,0xbb,0x52,0xfc,0xc8,0x27,0xf7,0xd9,0xd1,0xb9, -0x07,0xae,0xab,0x42,0x63,0x04,0xfc,0x00,0xd2,0xa3,0x8b,0x16, -0x13,0x3c,0x28,0x1d,0xb2,0x56,0x2a,0x9f,0xed,0xc7,0xfd,0x2b, -0x84,0x47,0xec,0x0d,0x66,0x6d,0xf2,0xfa,0x5d,0x2a,0xc6,0x6d, -0xba,0x2c,0x1d,0x60,0xef,0xa8,0x4e,0xa7,0xf1,0x18,0x9f,0xe9, -0x7a,0x3b,0xe3,0xf8,0x74,0xb9,0x18,0xe8,0x7d,0x66,0x8d,0xde, -0xdc,0xf4,0xc2,0xa8,0xee,0x86,0x91,0x15,0x7e,0x5f,0x96,0x61, -0x8b,0x31,0xdc,0x16,0xb3,0x3d,0x54,0xcc,0x11,0x03,0xb8,0x17, -0x2d,0x07,0x08,0x9a,0x59,0x87,0x26,0x36,0x0d,0xc3,0xae,0x51, -0xf6,0xcf,0xa5,0x21,0x62,0xcf,0xc2,0x60,0xd0,0x9f,0x7a,0xf7, -0xdd,0x86,0xc6,0x7d,0x74,0x7e,0x72,0xb2,0x3b,0xcf,0x94,0x22, -0xdc,0xa8,0x56,0x93,0x89,0xa8,0xeb,0xb0,0x7b,0xb7,0xe3,0xa0, -0x0f,0xd9,0x77,0x26,0x44,0x6c,0xa8,0x8f,0x04,0xdd,0xd3,0x91, -0xb1,0x14,0x29,0xae,0xe5,0x78,0x0b,0x7a,0xf1,0xca,0xa2,0xd6, -0x19,0x6d,0x7c,0x41,0xac,0xa1,0x13,0xfb,0x68,0x85,0x78,0xa5, -0x30,0xbd,0xca,0x10,0x27,0xa6,0x7f,0x7d,0x9e,0x05,0x7e,0x04, -0x2f,0x59,0xdc,0x57,0xc6,0xce,0x0b,0x50,0xe9,0xf3,0x8a,0xed, -0x09,0x4b,0xe3,0x5a,0xff,0xb0,0xf7,0xa0,0x42,0x00,0x64,0xec, -0x83,0xa2,0x9a,0x4c,0x6d,0x03,0xf5,0x41,0xe4,0x09,0x72,0x3a, -0x31,0x76,0x78,0x82,0xe7,0x3e,0x3d,0x5c,0x17,0xe4,0x4a,0xd4, -0x01,0x3d,0x6d,0x88,0x47,0x9c,0x98,0xb4,0x32,0xd7,0x73,0xb5, -0xb6,0xa5,0xd4,0xbe,0x80,0x18,0xd5,0x17,0x96,0xd2,0x0c,0xb4, -0x1e,0xf1,0x8e,0x22,0xe5,0xd0,0xb1,0xbb,0xd4,0x1b,0xb6,0x41, -0x3f,0x0f,0xeb,0x8d,0x9c,0xbc,0x73,0x29,0x1a,0xee,0xeb,0xfb, -0xb9,0x5a,0xd0,0x4b,0xfb,0x66,0x88,0x5d,0x4f,0xb6,0x7e,0x19, -0x38,0x48,0xdd,0xae,0xf2,0x57,0x54,0xa5,0x0e,0xf5,0xda,0x2f, -0x5e,0xce,0xd5,0x3b,0x52,0x00,0xdf,0xc2,0x56,0x03,0xa4,0x64, -0x02,0x24,0xed,0x40,0x60,0x7d,0xfa,0xc4,0x02,0x36,0x10,0x11, -0xdf,0xac,0x64,0x92,0x93,0x76,0x76,0xed,0x1b,0xed,0xf3,0x31, -0xe3,0x4d,0x31,0x1c,0x05,0x9e,0xe3,0xe0,0xbc,0xd5,0x51,0xbb, -0xdd,0x1f,0x8a,0xfb,0x6f,0xac,0xe5,0xa1,0x42,0x3d,0x65,0xba, -0x41,0x8d,0xd1,0x6e,0x55,0xe6,0x26,0xfc,0xdd,0x98,0x82,0xae, -0x4a,0x39,0xc5,0xba,0xca,0xf1,0x3c,0xff,0xf2,0x40,0xf8,0xaa, -0xc2,0xd0,0xad,0x2d,0x0c,0xd0,0x90,0x25,0x9d,0x9a,0x56,0x8d, -0xec,0x6c,0x90,0x4c,0xcd,0xe3,0x41,0x21,0x97,0xb8,0xf8,0x62, -0xa5,0x26,0xab,0x7a,0xd8,0x99,0xe7,0xcd,0x13,0xe9,0xdb,0x79, -0x1c,0x70,0xce,0x7b,0xb2,0x80,0xda,0xc3,0xac,0x1c,0xb8,0x86, -0xa8,0x07,0xc8,0x31,0xb1,0x13,0x30,0x76,0xcd,0x73,0x19,0x33, -0x3c,0xd3,0x80,0x9d,0x7a,0x6e,0xf1,0x53,0xdc,0x60,0xbb,0x37, -0x25,0x87,0xb7,0x59,0x8f,0x61,0x55,0xcc,0xe6,0xc4,0x8e,0xc4, -0x0c,0x0f,0xf3,0xee,0x7f,0xca,0x6c,0x2c,0xfe,0x8d,0x24,0xee, -0xf1,0x7a,0x42,0xde,0x52,0x4c,0x07,0xc4,0x45,0x3d,0xfb,0x9b, -0x20,0x4d,0x86,0xa6,0xe9,0xd0,0x3e,0x8a,0xa3,0x23,0x19,0x94, -0x0e,0x99,0x6b,0x89,0xfb,0x4c,0x7a,0xdd,0xb1,0x8d,0x04,0x95, -0xd3,0x3b,0xe2,0x84,0x24,0xb2,0x8c,0xd8,0xc1,0xe9,0x61,0x34, -0x7a,0xf1,0x92,0x2a,0xb2,0x7b,0x86,0x57,0xb2,0x12,0x75,0xf1, -0x9f,0xa1,0xea,0x75,0x4c,0x97,0x97,0x0d,0xbd,0x25,0xdd,0xf6, -0xde,0x58,0xe4,0xec,0x12,0xf6,0xaa,0x7a,0x1d,0x7a,0xff,0x51, -0x7e,0xf0,0x55,0xb6,0xff,0xbe,0x85,0xcc,0x81,0x00,0x92,0x32, -0x08,0x11,0x5f,0xd7,0x1d,0x7e,0x08,0x7c,0x7a,0x12,0xbe,0x78, -0x8e,0xd9,0xc3,0x7c,0x63,0xc4,0x2d,0x26,0x34,0x77,0xf4,0x24, -0xeb,0x9c,0xef,0xe0,0x02,0xfd,0xd6,0x3d,0xd7,0x39,0x29,0x49, -0x55,0xcb,0x82,0xd2,0x0a,0xbf,0x6f,0xed,0xad,0x82,0x6e,0x2b, -0xd1,0xde,0xe3,0xd1,0x1c,0x06,0x2a,0xad,0xd9,0x0e,0x54,0xeb, -0x3e,0x4c,0xb3,0x2f,0x79,0x82,0x66,0xdf,0xb1,0xd3,0xbd,0xee, -0x96,0x65,0x56,0xb4,0x39,0xa6,0xb8,0x93,0x13,0x6f,0x8d,0xe5, -0xae,0xcb,0xb9,0x8f,0xbd,0xb8,0xf2,0x2d,0x9b,0xd1,0x93,0xde, -0xd0,0x23,0x8d,0x7f,0xf4,0xa2,0x87,0xde,0x14,0x93,0x28,0xe4, -0x15,0x29,0xbb,0x7c,0x5c,0x42,0xb5,0x3a,0x28,0x64,0xf0,0x0c, -0xc6,0x5d,0xf2,0x5a,0x7b,0x60,0x0e,0x48,0xc0,0x67,0x0a,0x17, -0xb2,0x94,0xd5,0x02,0xf1,0x9f,0x03,0x63,0x81,0xeb,0x6a,0xb0, -0x28,0xa4,0xb9,0x05,0x2d,0xb0,0x7a,0x96,0xbb,0x25,0x36,0xe7, -0xd8,0x94,0xc1,0x5c,0xad,0xe0,0xf7,0xf9,0xdc,0xec,0xd1,0x02, -0x3b,0x39,0x91,0xd1,0x0b,0x76,0x9c,0xe7,0x06,0x6e,0xdf,0xaf, -0xdf,0x1b,0x6e,0xc8,0x2c,0x8c,0x9a,0xef,0xdf,0x5b,0x66,0xb4, -0xc2,0xd6,0x86,0xb6,0x36,0x82,0x83,0xd5,0x66,0x63,0x76,0x68, -0x51,0xd3,0x55,0x78,0xfc,0xdf,0x00,0x00,0x00,0xff,0xff,0x3f, -0x2a,0x63,0x77,0x44,0x1b,0x00,0x00, - })) + var empty [0]byte + sx := (*reflect.StringHeader)(unsafe.Pointer(&_scripts_browser_scripts_js)) + b := empty[:] + bx := (*reflect.SliceHeader)(unsafe.Pointer(&b)) + bx.Data = sx.Data + bx.Len = len(_scripts_browser_scripts_js) + bx.Cap = bx.Len + + gz, err := gzip.NewReader(bytes.NewBuffer(b)) if err != nil { panic("Decompression failed: " + err.Error()) } - var b bytes.Buffer - io.Copy(&b, gz) + var buf bytes.Buffer + io.Copy(&buf, gz) gz.Close() - return b.Bytes() + return buf.Bytes() } + func init() { go_bindata["/scripts/browser-scripts.js"] = scripts_browser_scripts_js } diff --git a/mod/dashboard/resources/scripts-stats-modules.js.go b/mod/dashboard/resources/scripts-stats-modules.js.go index fa4a209b006..9c61f6af195 100644 --- a/mod/dashboard/resources/scripts-stats-modules.js.go +++ b/mod/dashboard/resources/scripts-stats-modules.js.go @@ -4,13765 +4,36 @@ import ( "bytes" "compress/gzip" "io" + "reflect" + "unsafe" ) +var _scripts_stats_modules_js = "\x1f\x8b\x08\x00\x00\x09\x6e\x88\x00\xff\xdc\xbd\xeb\x76\x1b\x47\xd2\x20\xf8\x7f\x9e\x82\xa8\xf6\xc0\x55\x42\x12\x22\x64\xbb\xe7\xeb\x82\x8b\x18\x59\x92\x3f\x6b\xc7\xb7\xcf\xd2\xd7\x97\x05\x21\x1d\x90\x28\x8a\x25\x81\x00\x8c\x02\x45\xc9\x04\xfa\x59\xf6\x3d\xf6\xc7\x9e\xb3\x2f\xb4\xaf\xb0\x71\xcd\x5b\x15\x20\xc9\xee\x99\xb3\x67\xbb\x8f\x45\x54\x5e\x23\x23\x23\x23\x23\x23\x23\x22\x3b\x97\x37\x8b\x8b\x4d\xb5\x5c\xa4\xb7\xd5\x62\xb6\xbc\x35\x37\x8b\x59\x79\x59\x2d\xca\x59\x76\xa7\x59\x47\x55\xfd\x70\xbd\x9e\xbe\x9f\x57\x6f\xca\x74\x79\xfe\x3a\xbb\x7b\x3b\x5d\x1f\xcd\xcb\xc5\xab\xcd\x55\x01\xdf\x7d\xfe\x69\x36\xef\x57\x65\xf1\xfa\x3f\x6e\xca\xf5\xfb\x3e\xfe\xa6\xb2\xc3\x75\xb9\xb9\x59\x2f\x8e\x24\xbd\xaa\xff\x46\xfd\x50\xde\xa8\x33\xc8\x07\x45\x41\x6d\x2c\x96\xb3\xf2\x39\x54\xea\x76\xb9\xb5\x51\xe7\x24\x4f\xa6\xd8\x6d\x02\x25\xb0\xb9\xed\x36\x51\x88\x92\x0e\x27\x75\xbb\xe9\x09\xe4\x72\x0d\xc8\x5f\xdc\x5c\x9f\x97\xeb\x84\x33\x97\x97\x02\xa3\x36\x79\x7a\xa2\xbf\x8e\x07\x47\xd5\xe2\x08\x41\xd8\xd9\x41\x5e\xac\xcb\xe9\xa6\xfc\x69\x85\x1f\x75\xba\xe4\xbf\x3c\x54\x28\x58\x5e\x6c\x0a\x49\x7b\x34\xbd\xb8\x2a\xc7\xf2\x31\x29\xee\x76\xd1\x10\x4b\xc8\xd7\xfa\xfd\xeb\xe9\x06\xbe\x2e\x96\xeb\xf2\xe5\x7a\xb1\xdc\xdc\x5e\x55\x9b\x32\xdb\x6e\xc7\x13\x63\x11\xff\xd2\x5c\xce\xa7\xaf\xb2\x3b\xee\x65\x8c\x1f\x93\xa2\x73\xb2\xcb\x0c\xa7\x38\x10\xab\xc5\xa6\x5c\x2f\xa6\xf3\xc7\xd3\xcd\x34\x2d\xe7\xe5\xb5\x59\x4c\xaf\x4b\x33\x83\x4f\xb3\x7a\xbb\xc9\xee\xaa\xcb\x54\x80\x98\x5e\x5c\x94\xab\x8d\x2d\x98\xf1\x40\x36\x57\x15\x83\x6f\x00\x64\xa3\xcd\xfd\x8f\xf2\xbd\x4e\x5b\xf9\x6e\x35\x85\xe9\x31\xaf\xca\xcd\x37\xef\x7f\x84\xc6\x8b\xa4\xde\xac\xab\xc5\x2b\x87\x53\xea\xb2\xaa\x7f\x84\xf9\x2a\xb0\x69\x3b\x73\xe6\x02\x5b\x2e\x38\x6b\x24\x0d\x52\x5a\x4e\xb0\x56\x33\xcd\xc3\xcf\xb1\xd7\xfb\x24\x6f\xa4\x74\xbb\xde\xd7\x10\xc6\x55\xcd\xba\x5d\x6a\x6c\x5c\xcd\x20\x37\x85\xf1\x6e\xb7\x36\xa1\x8f\x38\x00\xbc\x76\x2c\xe0\xdb\x2d\x26\x01\x9d\x38\x82\x96\x69\xaa\x66\xdb\x6d\xba\x0f\x92\x02\xa0\xa4\xd9\x9a\x41\xd6\xa6\x9c\x3d\x9d\xd5\xfd\xd5\x72\x95\x42\xdb\x32\xa2\x57\x37\xd5\xac\xd7\xcb\x71\x34\xae\x5e\x66\x2c\x28\xd0\xb8\xfd\x0d\xc4\x21\xa8\xf2\x53\xfb\x9b\xe5\xff\xf6\xec\xa7\x1f\x15\xe7\x8b\xe5\x72\x95\x65\x26\x4d\x78\xba\x43\x4c\xfb\x44\x1f\x64\x64\x8c\x83\x91\xeb\xcb\x4e\xe1\xa6\x5c\xcc\x5c\x6f\x44\x23\x59\x1e\xa2\x6a\x5f\x61\xca\xe4\x1a\x99\xb1\xd4\x52\xb8\xc6\x08\xed\xa9\xcd\xa1\xf2\x8d\x04\x18\xb6\x5f\x3b\xcc\xcc\x4c\x3c\x31\x30\x12\x5b\x64\x6c\xe9\xe6\xba\x9c\x3f\x9a\xd6\x65\x4a\xc0\x4c\x0a\xae\x6a\xa7\x77\x94\xc2\x6c\xba\x96\xc7\x58\x0a\x86\x7a\x33\x9f\x17\x05\xe4\x40\x93\x61\xfe\x9e\x66\xb3\x2c\x0f\xca\xe1\xba\xd8\x35\x17\xdc\x2f\xe5\xf5\xf2\x6d\x19\x2d\xbb\x8f\x5a\x71\x95\x99\x3b\x4c\xfc\x2b\x96\x4d\xb8\x50\x27\x79\xf8\x8d\x2b\xc5\x4e\x16\x41\x87\xa0\xfa\x08\x2e\x02\xa2\x89\xc8\x02\x80\xb6\x7c\x9a\x58\x3e\xa3\x69\x84\xff\x16\xf8\x4f\xff\x62\xb9\xb8\x98\x6e\x74\xcc\xd7\xd3\x15\x95\x30\x31\x7a\x01\xaf\x98\x8e\x5c\xd6\xf6\xcc\xad\xf0\x4c\xe5\x54\xad\x68\x9f\x15\x63\xbb\x0b\xeb\x4b\x55\x82\xa3\x5e\xcd\xab\x4d\x9a\x1c\x25\x59\x36\xbc\x5c\xae\xd3\xaa\x38\x31\x73\x06\x91\x59\xfc\x70\x7e\x5a\x0d\xab\x5e\x2f\xe3\x85\x7c\x14\x92\xca\xb8\x9a\x4c\x10\x57\x1d\x5a\x43\x55\xfd\xe4\x7a\xb5\x79\x8f\x73\xf7\x13\xad\xc1\xdc\x62\x81\x32\x38\x31\x73\x48\xcc\x84\x97\xec\x98\x0d\xa5\xd2\x47\xb4\x8c\x1a\xcd\x7a\x53\x93\xe1\xfa\x8d\xe6\x7c\x5e\x4e\x17\x44\x3f\x63\x9c\xe9\x89\xe9\x9c\x64\x0a\x48\x7d\xb3\x5a\x2d\xd7\x9b\x3e\x77\xf4\x84\x27\x5b\xf8\x5f\x87\x97\x67\x9f\x37\xf0\x51\x0c\x8b\x9b\xe2\x02\x97\x47\xb6\xf3\xe8\x1b\xc1\x7c\xb8\xd9\xac\x99\xaa\xdf\x94\xef\x69\x6d\x12\xdd\xd0\x3a\x0e\x16\x29\xee\xd4\x01\xe5\x32\x85\xd3\x64\x25\x58\xfc\x38\xe9\x41\x13\xfd\x75\xb9\x9a\x4f\x2f\xca\x74\x7d\x7d\x33\xdf\x54\x8f\xa7\xf5\x95\x49\x8e\x3f\x1b\x24\x19\xf0\xbd\xef\x97\xb7\xe5\x9a\x66\x3a\x1b\x6a\x27\xd4\x26\x2c\x6c\x04\xa4\x3a\xbf\xd9\x28\x15\x34\x76\x1e\x86\x6d\xb3\x7e\x7f\x47\xf5\x92\xcd\xfa\xa6\x44\xe1\x00\xbf\x48\x5c\xb8\x9c\xce\x6b\x2f\x65\x90\x27\x38\x62\x9b\x80\x1f\x79\x0f\x7f\xf6\x12\x9b\x48\xdf\xf9\xfa\x7c\x0d\x20\xf7\x37\x65\xbd\x21\xa0\x32\x9d\x94\xd5\x74\x5d\x97\xc8\xad\x39\x39\xc7\x7f\x77\x17\xb4\xa9\xc3\xf8\x77\x52\x6a\x66\x39\x83\xc5\xe1\xae\x04\x58\x08\x64\x87\xc3\x9d\xec\x40\xd4\x88\x27\x5d\xc5\x64\x62\x65\x2c\x44\x04\x91\xb7\x2e\x06\xcc\x01\xbc\xa5\x84\x6f\x94\x82\x78\x8f\xe8\xb4\xd1\x2b\x36\xc3\x4b\x06\x89\x2d\xe1\x4d\x47\xeb\x08\x01\x77\x06\x22\xbc\x80\xbc\x61\x01\xe2\x94\xe7\x80\xdd\x34\xbb\xdb\x97\xfd\x2d\xe2\xda\xe5\x0f\x5c\x7e\x5d\x9d\xcf\x61\xde\xd2\x8b\x9b\xb5\x99\x55\xeb\xec\x6e\xb6\x3c\x82\xdf\x05\xfc\x37\x86\xef\xc9\x10\x84\xa0\x79\x89\xd9\x40\x53\x00\x0e\xfc\x70\x24\xa5\xb2\x14\x24\xba\x16\x81\xb2\x17\x20\x32\x22\x82\xcb\xc5\xa6\x36\xbf\xde\x4c\xe7\xd5\x65\x55\xae\x01\xdd\xe5\x8a\xc8\xd5\x26\x15\xf6\xd7\x76\x7b\x62\x2c\x5e\xbe\x55\x69\xcb\x66\xeb\x1a\x56\xb1\xed\x15\xd0\xad\xeb\xc2\x4a\x67\xcc\x83\x79\x3a\xa0\xc2\x5f\xa7\xf3\xa2\xd3\xb1\x8d\x00\xeb\x9a\xcf\xa5\x8c\x21\xb6\xaf\x03\x90\xb2\x45\x81\x20\xee\x88\xda\x5d\x2d\x3b\xdc\x8f\x06\x41\x11\x7d\x84\x1f\xd0\xaa\x1b\xaf\xdf\x43\x63\xc9\xb8\xd1\xd2\x00\x2e\xab\x39\x6c\x68\xe5\xac\xf8\x84\x0e\x1b\xcb\x9e\xbb\xaa\xea\x67\xd5\xf5\x6a\x2e\x6b\x66\x2f\x56\xb9\x47\x97\x6f\x14\x04\xd3\xa1\xb9\x1b\xba\x81\x7c\xa8\x46\xb6\xfb\x74\x6c\xe9\xfc\x2f\x78\x37\xa3\x89\x72\xb0\x9e\x16\x27\x16\x7d\xf1\x41\xe0\xd9\xf4\xb2\xfc\x76\x3d\x7d\x85\xad\xa7\xb3\xe5\xc5\x0d\xfe\x90\x93\x4f\x55\x6f\x0a\xc4\x06\x0a\x22\xb5\x6e\x45\xdb\x24\x33\xb5\x54\x2a\xb4\x42\x9f\x1b\x7b\x2c\x9f\xb6\x41\x42\xa1\x96\x96\x42\x4f\x78\x28\x19\xae\xf7\x21\xf6\xa1\x3b\x59\xd6\x5e\x30\xa5\x32\x24\x9c\x5a\xaa\xd3\x92\x6e\x34\xc0\x78\x66\x3f\xad\x1f\xae\x56\x28\xe7\xd1\xf8\x37\x78\xd8\xf0\xa8\x09\x79\xaf\x34\x59\x7f\xf3\xfe\xf9\xf4\x15\x0e\x2b\xc5\x52\xe3\x13\x90\x66\xa9\xc8\x94\xea\x3f\x82\x85\xcb\x8d\xf4\x97\xb7\x8b\x72\xfd\x38\x1c\xa5\xc2\x85\x55\x3d\x7c\xce\xaa\x7a\x7a\x3e\x2f\x9f\x5d\xac\xab\xd5\x46\xa6\x07\xd1\x38\x05\x7e\xdf\x64\xfe\xb8\x23\xa6\x09\x92\x6f\x62\x47\x45\x85\xe8\x74\x99\x62\xa5\x6e\x17\xff\x05\xbc\x97\x17\x38\x8f\xb3\xac\x97\xdc\x4f\x7a\xb6\x10\xad\x43\x9f\x5d\xd5\x1b\x10\xe8\x1b\xdd\xd3\xc9\xac\x58\xd7\x94\x8e\x74\xfd\xc3\xb4\x7e\x53\xce\x40\x92\x2a\x2f\x52\xdb\x98\x85\x81\x8a\x8f\x1c\x24\xf4\x3d\x1e\xf0\xd1\x05\xf6\x3c\x14\x10\xdd\x0e\x26\x03\x88\x40\xa9\xcb\xcd\xbf\xcf\x97\xe7\xd3\xf9\x93\xb7\x53\xe6\x1a\x35\x08\x9c\x97\x8a\x7c\x38\x71\xc3\xd4\x23\x6c\xcc\x50\x8a\x93\x21\x6e\x59\x9d\x82\x8a\x12\xaa\x6a\x10\x5c\x32\x12\x6b\x84\xb2\x5f\xba\xad\x27\x79\x65\xdb\x4e\x4c\xc7\x6b\xd7\x1e\x5d\xb8\xb0\x97\x03\xad\x05\xd5\xfc\x69\xbb\x98\x2f\x17\xe5\xa3\xe5\xea\xfd\x93\xb7\x38\xa9\xf5\xfa\xc2\xcc\x00\x95\xc4\x6d\x91\x27\xe0\x87\x77\x6a\x97\x2e\xae\xa6\x35\x89\x30\x50\x5c\xcf\x9c\x38\x25\x28\x06\x2f\xe7\xb3\xc7\xde\xd1\x83\x81\xc1\x72\x06\x98\x7d\x33\x07\xdb\xd7\x3a\x80\x48\x04\xa2\x2e\xe4\xbb\xcf\x9f\xb8\x88\xf8\x17\x6c\x32\x22\xf6\x70\x53\x00\xc7\x62\x36\x2f\xb5\x65\x29\x8f\x67\x75\x44\x31\x82\x84\xbb\xa9\xd4\x75\xf2\x23\x27\x8c\x31\x7f\x12\xcb\x91\x2a\x68\x63\x91\xfe\x74\x36\x63\x00\x99\xe0\xbc\x6a\x38\x41\x3b\xed\x16\x07\x02\x72\x9e\xff\x19\x9d\xbc\xe0\x80\xe8\xe7\xc2\x04\xf8\x4b\xf7\xdd\x23\x9c\x04\x5c\x12\x4f\xeb\xfa\xa6\xac\xbd\x59\x20\xf1\x40\x78\x90\x61\x1d\xc0\xb0\x6d\x66\xf8\x10\x20\x05\x5d\x16\x7e\x85\xf2\x98\xe9\x44\xb2\xe6\x62\x49\xbd\xd3\xf4\x77\xbb\x58\x33\x3e\x7c\x64\x77\xb3\xd6\x69\x63\x91\x9c\x50\x3c\x73\xd8\x57\x0c\xf2\x5a\x61\xaa\x22\x1c\x32\xf4\x32\x65\xd9\x90\x60\x8c\x17\x54\xd8\x73\xb6\x4b\x78\xdd\xa2\x18\xa7\xc3\x61\x18\x61\x3f\x7a\xb7\x01\xa9\x02\x30\x45\x3f\x47\x69\xc8\x7f\x08\x3e\xca\xb1\x45\x4c\xc8\x22\xa8\x44\x96\xbb\xe3\xb8\xed\x61\x44\x79\x28\x13\x02\xec\x38\x2b\x30\xb7\x94\xb2\x04\x10\xd7\xdf\x3d\xff\xe1\x7b\x6a\xd3\x7e\x65\x26\x42\xe9\xd5\xe6\x7a\xfe\x15\x61\xb5\xdb\xc5\x92\x20\xd7\x70\xc9\x6e\x57\xb1\x0f\x03\xbe\xe6\x46\x6d\x66\xa6\xdd\xd8\x94\x22\xa8\x8c\xc0\x56\x8b\xd5\x4d\x84\x8d\xeb\xe9\xa2\x5a\xdd\xcc\xa7\x48\x4b\x2f\xd7\x70\x06\xb8\x78\x83\x78\x40\xa2\xe0\x4d\x9b\xc6\x8f\x24\x22\xe3\x02\x29\x75\x0a\x22\xfb\x23\x2c\x09\x32\x02\xa5\x5d\xc8\x07\x96\x95\xdf\x44\x80\x7d\x60\x17\x37\xa5\xe0\x99\x7e\x2b\x90\xf4\xe1\x92\x09\x93\x2b\xd1\x5f\x38\x4c\xfa\x1d\x3e\x83\xa5\x7b\xb1\xd1\x1e\x6b\xfd\xc2\x26\xa2\x12\x79\xda\x1c\xe8\x76\x9b\xe0\x24\xc2\x9c\x4c\xfd\x64\x8b\x34\x69\xe2\xaf\x16\x2c\x3f\xc1\x5f\x6e\xb8\x0b\x81\x30\x07\x27\x5c\x22\x0a\xda\x26\x95\x19\xd7\x46\x59\xb2\xb9\x5c\x82\x3c\xaf\xf2\x95\x14\x6e\xdd\x42\x51\xb4\x45\x4d\x12\xc8\x64\xf6\x04\x30\x3a\x54\x01\x37\x4e\x18\xcd\xbd\x24\xcb\xa3\xe6\x7f\x45\xd2\x60\x24\x2c\xd7\x0f\x71\x4b\x38\xd0\x74\x5c\xd8\x6b\xd6\x16\xa6\xb3\x2f\x8d\x84\xb8\x1f\x8f\x69\x3c\xa1\x51\xd6\x85\xb6\x74\x81\xfb\x3d\xd2\x39\x6c\x21\x92\x76\x60\x47\xea\x50\x3f\x56\xa9\xc5\xd3\xe0\x44\x8e\x11\x75\xd2\x5f\xdd\xd4\x57\xbc\x0b\xeb\xd9\xf6\xba\x5c\xbf\x2a\x19\x04\x23\x73\x60\x2b\xd9\xed\x17\x3e\xfc\xd3\xe8\x76\x0b\x09\x76\xd3\xb1\x9d\xf9\x73\x37\x0a\x9a\x1f\x4b\xd6\x84\xe7\x2f\xcb\xe9\x4f\xc0\x6b\x1f\x07\xe4\x2f\x92\xc2\x07\x17\x91\x93\x14\x80\xe0\xe8\x23\x5a\x46\x94\x26\x4b\xc7\xdb\x5e\x81\xfd\xcd\x96\xeb\x9f\xd7\xcb\x15\x41\x5e\x6f\xde\xc3\x76\x45\x27\x33\xd5\xda\x20\xfb\xa4\x64\x15\xa9\xed\x69\x10\xa9\xf2\x62\x4a\x15\x45\x2b\x73\x35\x5d\x3f\xdc\xa4\x27\x78\xc4\xfe\x4f\x10\xd5\x84\xa5\xf7\x58\x55\x32\xaf\xe0\x34\x3e\xc8\xcc\x72\x5d\xbd\xb2\x75\x80\x96\x2f\xea\xfa\x67\x10\x06\xaa\x77\x20\xc4\xca\x66\x57\x1d\x1f\x0f\x33\xe9\xdf\xcf\x87\x49\xee\x49\x8f\x66\x3f\x6c\xf2\x5b\xfb\xf1\x8f\xb8\xdf\x55\xb3\x59\x29\xa7\xa9\x72\x1e\x9e\x66\xca\x39\x4b\x99\x26\x59\x00\x67\xc4\x45\xac\x1a\x91\xba\x16\xd1\x06\x78\xf8\x6a\x3e\x7d\x9f\x64\xee\xac\x8b\x13\x3a\xad\x16\x75\x8b\x3c\xca\x27\x31\x4f\xee\xba\x5a\xde\x02\x00\xa5\x3b\x2d\x60\x8a\x13\xb7\xa4\x75\x5e\xe3\x57\x04\xa9\x21\xe6\x55\xe3\x9a\x00\x01\xba\x7c\x87\xe2\x01\x5f\x78\x68\x1b\x56\x3e\xe0\xbb\x05\x2a\x35\xa4\x7f\x61\x2d\xd8\xe5\xc1\x52\x16\xa6\xf2\xe2\xea\x13\xd6\x80\x54\xb8\x7d\xc9\x2a\x5a\xa4\x39\x10\x75\xec\xa8\x8d\xfc\x2a\x5c\x13\x7d\x05\x1a\x87\x32\x0a\x9b\xc3\x9b\x10\xc4\x24\xb0\x09\x29\xb5\xdd\xa6\xcd\xaa\x45\x02\x2d\x93\x26\xa4\x99\xd7\xed\x06\x73\x96\x7d\x22\xc4\x06\xa6\xee\xa5\x2c\x84\xc7\x9c\x96\xda\x43\x24\xb1\x67\xf8\x5f\x1e\x01\x9d\x32\xea\x8b\xb0\x67\x93\x5a\x90\x9a\xa3\xea\x70\x95\xcc\xb2\x82\x7d\xf0\x70\xb9\x91\x7c\xe7\xfb\x09\x2c\x53\x85\x22\xcf\xfa\xef\x9d\x5e\x9c\x15\x0f\xe0\x36\x04\x27\xed\x19\xed\x53\x45\xb3\x1c\x4f\x72\x92\x73\x07\x59\x70\x52\x42\x98\x82\x33\xc7\xcf\xcb\xba\xda\x54\x6f\xcb\x1f\xe9\x76\x8c\x07\x4c\x4d\x99\xfa\xe6\x7c\xb3\x9e\x5e\x6c\xbc\x13\x11\xd0\xfc\x7a\x71\x73\x4d\x87\x5a\x3e\x0b\xf1\x36\x1e\x9c\x83\xca\x7a\xf4\xc3\x74\x73\xd5\xbf\x9e\xbe\x4b\x4f\x8c\x24\xc1\x61\xe8\x38\xd5\x16\xb7\xdb\x93\x2c\xeb\xa5\x9a\xf5\x00\xc1\x5d\xbd\x4b\x64\xce\x1d\x78\xd3\x1b\x3a\x10\xff\xad\x9a\x6d\xae\x7e\x5a\x7f\x57\x56\xaf\xae\x36\x9e\x8e\x1d\x38\xf6\x1a\x15\xa9\xdf\x2c\xd7\xb3\x72\xfd\xcd\xf2\x9d\x21\xb4\x78\x47\xa5\xaa\xa0\x32\x40\xc4\xa9\x57\x6c\x94\x9c\xd3\x4f\xc0\x10\x31\xfe\xc5\x26\xc9\x46\x5f\xe6\xc9\x2d\xf6\x43\x62\x02\x0a\x21\x83\xfc\x04\x11\x01\xb3\xfc\x25\x09\xf9\xc5\x83\x2c\xb9\x9e\xae\x5f\x55\x24\xab\x50\xbb\x4c\xf7\xbd\x06\x47\xa2\xcc\x1e\x7c\xb2\x2e\x16\x0f\x52\x9d\x13\x05\x2e\xf3\x21\x1e\xa5\x16\x84\xb0\xd1\xe3\x26\x9b\x5b\xc1\xb9\x02\xf5\x37\xfb\x1b\x56\xf8\x3a\x1f\x6a\x4a\xc6\x1f\xb4\xd4\x4b\x08\xcf\x89\xdf\x62\x96\xb7\x0f\xf0\x83\xb0\xb8\x12\x9d\x0f\xe0\xea\xe3\x61\x51\x22\x83\x66\x02\xb9\xec\x20\x79\x30\xf1\x12\x5d\x3d\x75\x68\x2f\x3a\x3c\xb7\xd1\x94\xf3\x66\x71\x79\x59\x4b\xa3\xb9\x97\xc0\xcd\x0b\x34\x05\x74\xfb\x8c\x7e\x09\x1b\xf2\xa6\xb4\x88\xc4\xfa\xf3\xe5\xbb\x67\xd5\x6f\x80\x0a\x58\xd4\x3c\xd6\x63\x48\x6a\xdd\xc8\x6c\x51\x18\xf8\x40\x07\x8e\xd2\xd8\xc9\x69\x01\xe0\x6e\xb7\x7c\x85\x05\x3f\x49\x0e\xc0\x11\xc0\xf9\xf0\xd1\xb3\x67\xde\xa8\x75\x02\xa0\x4e\x54\x85\x67\xc0\x63\x2a\xa2\xff\x35\xb8\xa4\x81\x55\xac\xde\xb1\xd0\x82\x65\x33\x87\xeb\x61\x03\x79\xde\x60\xa1\xcd\x7d\xc3\xfd\xa5\x9c\x57\x28\x0d\x6d\xb7\x6f\x49\xd5\xd9\xec\x17\xd3\x49\x85\xfe\xed\x7c\x39\xe5\x7e\x81\x33\xec\x5c\xd7\xbd\x8f\x61\x00\x74\x5f\x7b\x70\x6d\x9b\x78\x04\x8a\xa4\x1e\x72\x1d\x4f\xa9\xd1\xdc\x96\xec\x8e\x44\x64\x34\x5b\x5e\x58\xe5\x5d\xb0\xeb\xca\xef\xb1\x16\x9f\x28\xad\x3a\xb6\xad\xc5\x81\xfd\xdd\x4c\xe7\x71\xfb\x06\x9a\xcd\x4c\xbc\x85\xc1\xa1\xd5\xd6\xaf\x2e\xd7\x28\x76\xc9\x5f\x15\xa5\xd3\xe4\x6b\x4e\x38\xa2\x7f\x79\xec\xc5\xe7\x27\x9f\x1f\x11\x65\xd3\xaf\x2b\x42\x1b\xfe\xbc\x7f\x9a\x64\x44\x6e\x09\xfc\xf3\x1c\xb0\x97\xd8\xbd\x2d\x3f\x9f\x2f\x2f\xde\x1c\x75\xaa\x6b\x9c\xc5\x29\xa2\x2d\x13\xc5\xde\xf3\x25\xaa\x38\xfb\x3a\x70\xd5\x44\x22\xc8\x0a\xce\xf8\x64\xd2\x17\x74\xb3\x05\xc7\x76\xdb\xc8\x50\xf9\x2b\xeb\x3b\x14\x42\xb3\xb7\xeb\x0a\x55\x62\x5f\x77\xe0\x83\x54\x2f\x78\x0c\x3e\xfd\x9a\xff\x3d\x5f\xce\xde\x9f\x26\xd4\x55\xff\x62\xbe\x24\x4d\xc4\xc7\x20\x92\x3b\x07\x61\x7b\x83\xe6\x16\x19\xab\xda\x1a\x93\xa4\x68\xb6\x6d\x7a\x3b\x4f\xd8\xb8\x36\x6c\x8f\x7c\xb2\x76\x09\x31\xa1\x7e\x93\xaf\xc7\x43\xd4\xe1\x30\x1c\xe0\xd1\xb2\x07\x14\x79\x12\x46\xa0\xd5\x64\x3d\x47\xda\x02\xdf\xf9\x0d\x1c\xbe\x7e\x9e\xc2\x28\xeb\x74\x45\x62\x38\xda\x83\x18\x58\x0e\xb3\x0a\x0b\x4c\xe7\x06\xf8\xaf\x77\x4f\xe4\xae\xa2\xf5\xf2\x16\x6f\x8b\xb2\xc0\x2a\x05\x1a\xb0\x8a\xf2\xca\xbc\xc5\xdb\x34\xdb\xdc\x76\x4b\xb7\x60\x6f\xca\x0d\x33\x09\xee\x34\x1b\xa1\xae\x4b\x00\x78\x9b\xe5\x4d\xb0\x7a\xc9\x38\xe9\x35\xad\x17\xde\x8e\xaa\x1c\x04\xcc\x5e\x32\x49\xcc\xdb\x06\xd8\xbb\x6c\x48\x17\x64\x00\x74\x00\x82\x36\xd3\x29\x1a\x96\x44\x99\x07\x08\x59\x16\x51\x03\xf1\xed\xd8\x1e\xf8\xb0\x08\x81\x62\x2f\xc5\x9a\x20\x39\xd2\x98\xc1\xb4\xd2\xd1\x07\x6f\x1e\xea\x9f\xd6\xcf\xd7\xd3\x45\x8d\xab\xa6\x4e\xf1\xc8\x0d\x94\xb3\x2e\xed\x19\xc6\x22\x14\x45\x4f\x3c\x18\xc2\x1e\xb7\x2e\xeb\x1a\x92\x08\xd9\xd9\x9d\x5e\xcb\x74\xfc\x9b\xcc\xb0\x24\x70\x59\x2c\x5b\xb4\xb4\xd1\x4c\x2a\xf0\x30\x3f\x24\x76\x25\x79\xa4\x99\xd0\x8f\xba\xa5\x95\x50\xd7\xb7\xdf\x2c\x29\xa0\x22\x7b\x69\x46\xc3\xe0\x8b\x09\x6d\xda\xf6\x01\xf2\x68\xaf\x37\x19\x66\x49\x4f\xef\x54\x31\x15\x48\x7e\x94\x36\xca\xda\xb3\x28\xe9\x23\x8c\x43\xe6\x58\x4b\x4c\x8a\x96\x34\x04\x2c\xeb\xdf\x2c\xea\xab\xea\x72\x23\xc0\xe4\x9f\x56\x99\xd4\x0e\x54\x33\x30\xef\xc0\x3b\x84\xcd\x87\x66\xda\x88\x31\x17\x1d\xa0\x2b\x20\x17\x31\x12\x33\xaf\x7f\xfd\xfb\x77\xbf\xf8\x76\x72\xdc\x9e\x1d\x37\x2f\x4e\x55\x68\xe9\xba\x97\x52\xe5\xcc\x83\xbb\x63\xaf\x2a\x69\x9d\xee\x19\x86\x6f\x33\xb6\x52\xa0\x7f\x5a\x7f\x3b\x45\x4d\xcf\x7b\xd9\xc0\xa4\x82\x37\x94\xa2\x59\x36\x3d\x3c\x24\x01\x75\x2f\xdd\x7a\x8d\x6f\xb7\x75\x59\xbe\x81\x52\x5e\x4a\x73\x84\x5e\x85\xc9\x28\xae\x30\xea\xa4\x56\xe9\xd7\x52\x01\xce\x0c\xcb\x6a\x76\x74\x92\x5b\xa3\x3a\x4b\x7a\x96\x26\xda\xaa\x99\x78\x3a\x82\xcc\xce\x00\x78\x90\xd1\x7e\x77\x74\x94\x50\xb0\xd1\x6c\x2b\x06\xd2\x91\x16\x9a\x23\x5a\x2a\x89\x26\xb5\x09\x22\x2c\x04\x54\x5a\x38\x94\x00\xe1\x93\x75\x1b\x97\xc7\x95\xec\x31\x9e\xd7\xd3\x77\x4f\xf8\x62\x60\x03\x72\x7e\x09\xc2\xe8\x5a\xb6\xa4\x59\x59\xae\xc8\xba\xe0\x72\x3e\xdd\xc8\x6c\x29\x8b\xc4\x6a\xcf\xca\xcd\x06\xe0\xad\xfb\x5e\xfe\x76\x2b\x17\x1e\x50\x8f\x34\x36\xd0\x18\xfc\x37\x86\xcf\x49\x64\x80\x95\x7a\xd5\x28\x7f\xc4\x00\xe4\xd8\x2f\x59\xb8\x94\x2b\x34\xeb\xca\x28\xb3\xd0\x56\xec\x66\x86\xf9\xf6\x10\x2e\x77\x1b\x40\xd4\x32\x0a\xcc\xcd\xe4\x23\x1c\xed\x77\xa4\xf4\xff\xa5\x04\x6c\x2e\x6a\xbc\xe0\x60\x0a\x44\x95\x3c\xa7\xe8\xc5\xf5\xba\x26\xdb\x2a\xb6\x98\xda\x98\xcb\x4a\xec\x20\x29\x81\x6e\x61\x44\x08\xa9\x8b\x5a\xe5\x91\xda\x63\x89\xde\x94\xd8\xc6\xbf\xad\xca\xf9\x0c\xb3\xc2\x84\xe0\x8e\x28\xcc\xca\xe2\xe4\x1a\xc5\x63\x84\x78\x1c\x16\xe4\xcb\xa0\x49\x61\xcb\x71\x02\xeb\x16\x86\x30\xeb\x1e\xaf\x44\x1a\x19\x66\x8e\xa8\x99\xa4\x33\x18\x66\x68\x80\x93\x42\x42\xdd\xbf\xae\xae\x49\xfd\x08\xf2\x21\xf6\x8b\x9a\x64\x45\xdf\x77\xe5\x14\x64\xc3\x34\x79\xc4\xc3\x3f\x7e\x4e\x97\x90\x74\xbc\xb8\xe0\x7b\x65\x05\x5f\x11\x84\xba\x3e\xfd\xcd\x10\x76\xbb\xe1\x37\xcb\x02\x50\x9d\x2f\x7b\xc2\x65\xc7\x37\xa4\xe7\x20\x1d\xbd\xd9\x89\x39\x8f\x0e\xc8\xc7\x51\x16\xcc\x56\x38\x6e\xdc\xc4\xef\xda\x10\x5e\xd3\x01\xa8\xe3\x17\x06\x5e\x83\x53\xfb\xb6\x5c\x23\xb3\x26\xf8\x7a\xc9\x51\xd2\xf3\xcb\x4c\x80\x1f\x07\xbd\x61\x29\x01\x31\xa0\x22\x20\xea\xe0\x9b\x0a\xc2\x5a\x0c\x2a\x07\x5f\xdb\x6d\x50\x41\x4f\x31\x41\x99\x51\x1a\x7c\x76\xc2\x59\x06\x69\xbf\x81\xc3\xa0\x7c\xe6\x08\x7f\x1c\x64\x4c\x94\x0b\x86\xcb\xe7\x11\x63\x23\x75\x34\xcd\xeb\x05\xb1\xf4\x00\x6f\x16\xf1\xa6\x0a\x57\x06\x48\x60\xd7\x2b\xe3\x90\x47\x96\xa9\x81\xcc\xe0\xb3\x55\xde\xa6\x33\xdc\x64\xde\x86\xd3\x85\x66\x0b\x54\xf0\x5b\xda\x50\xc8\xce\x92\x3b\x2e\xfc\x74\x9b\x6a\x5c\xb3\x59\xe6\x7a\x1b\x0f\x26\x44\x8f\x08\x10\x71\x26\x6f\x5e\x33\x6f\x8a\xf1\x67\x28\xba\x4c\x8a\x80\x06\xf0\xe7\x84\x17\x95\x8c\xd6\x83\xb7\xd7\xab\x26\xa4\xce\x86\xf5\xc6\x86\x46\x6b\x32\xe2\xb0\x29\x38\xbe\x6e\x17\xff\x8d\xb2\xb1\xd9\xc2\xeb\x06\x8b\x10\xa9\x49\xa1\x09\x5d\x88\x68\x6e\x72\xcf\xcb\x31\x1d\xcc\xb1\xa3\x7b\x20\xcb\x4d\x07\x87\x02\xef\xf5\x8a\xda\x7e\xe0\x59\x31\xe2\xf4\x00\x52\x0a\x0b\x06\x2e\xf7\x7d\x40\x60\x59\xa0\xf4\x26\x0c\x92\x01\x6b\x95\xea\x16\x20\x57\x8c\xe2\x56\xa8\xe7\x49\xde\x48\xe9\x60\x69\xbe\xac\x26\x34\x72\x5b\xc6\x23\x8a\x15\x51\x45\x75\x7c\x6c\x4e\x94\xb4\x32\x6f\xf5\x63\x33\xd4\x88\x30\x15\xc0\x2c\x80\xb5\xb9\x5a\x2f\x6f\xeb\x64\x92\x59\x3a\xc1\x2c\x4b\x1f\x22\xc5\xa3\x51\xdf\x9e\x02\xce\xda\x8e\xd7\xdb\x5d\xbd\x81\xb3\x58\x9e\x90\x5a\x61\x5d\xae\xd7\xcb\x75\x62\xe8\x0f\x8d\x69\x04\x59\x3f\x2e\x05\xe3\x35\x5d\xed\xac\x97\xd7\x80\x1b\xc1\xde\x66\xe9\xe6\x6a\xb7\xdb\x11\x85\xeb\x67\xd8\x41\x7d\x73\x71\x01\xc2\x73\x62\xd8\x42\x50\xe0\xd9\x35\x0c\x85\xe0\x28\x3d\x9b\xae\x67\xc0\x8b\x53\xb6\x4e\xd4\x2b\x91\xf2\xf6\x88\x6d\x32\xfb\x7f\xff\xe1\xfb\xef\x36\x9b\xd5\x2f\xe5\xaf\x37\xc0\x4d\x3d\xfb\xc1\xb8\xb1\x87\x17\xa8\xa8\x3d\xd4\x94\x94\x10\x13\xbf\xe4\x87\xea\x62\xbd\xac\x97\x97\x1b\xea\xe3\xf9\xf3\x9f\x93\xec\x40\xf3\xdf\xbe\xfb\x71\x79\x6b\x8d\xf6\x50\x37\xfc\x1c\x76\x93\xe5\x0d\x0b\xd5\x24\x59\x02\xfb\xc4\x42\x9e\xd1\x62\x66\x38\xc5\x5e\xb6\x41\x0b\x71\xc3\xcf\x6f\xcb\x72\x51\xa7\xd3\x45\x75\x4d\x57\x65\xc0\x39\x96\xab\xda\x5a\x13\x93\x50\x4b\x49\x4e\x84\xc5\x4f\x56\xdb\x38\x65\xee\xc5\x72\x8e\xf2\x18\x1e\x71\xd2\x0d\x36\xc9\x64\xbf\x5c\x89\x14\x2f\xd6\xc7\x36\x0b\x65\xa9\x2c\xbe\xa0\x71\x8d\x1c\xbc\xa2\x21\x22\xd5\x92\xa2\x53\x67\x73\xbe\x70\x14\x02\xa3\x5a\xfc\x7a\x43\x7f\xa8\xe5\x58\x5f\x85\x85\x61\x45\x55\x20\x60\x04\xde\x23\x40\x39\x37\x73\x54\x2b\x2e\x57\xab\x72\x16\x43\x6b\x3b\x73\xa7\x10\xf5\xa7\x81\x09\x00\xba\x76\xd6\x7a\x8f\xe5\x1b\xce\x6f\xd3\xf9\xed\xf4\x7d\xed\xcf\x9a\x9a\x39\x57\x17\x6f\xfa\x64\x75\x04\x2c\x05\x7e\x17\x5e\x11\xe4\xdd\x0c\x84\xb3\xfe\xb4\x98\xe7\x25\x80\xe4\x50\xd0\x74\x03\x73\xf1\x49\x06\xf6\x97\xeb\x69\xb5\x00\x09\xb3\xf0\x34\xff\x16\xf8\x3e\xac\x99\x35\xd5\xee\xb9\xb4\xd9\xcd\x9a\x7e\x1c\x7b\x8d\x03\x54\x25\xb0\x3f\xdb\xda\xfd\x66\x71\xb4\xdb\x04\x44\x5e\x20\x13\x1a\x1c\x63\xf1\xbd\x38\xeb\x13\x25\x1c\xbe\x8a\x8b\x0b\xeb\x54\xaf\x6f\x80\x06\xb9\x1b\x4f\x8a\x65\x0c\x03\x91\x6f\xaa\xcb\xf7\x7f\xab\x36\x7c\x57\x6d\xc6\x1e\x4d\x70\x1d\x87\x10\xa0\xc0\xc1\xa9\xa4\x5a\xef\x25\x9b\x9b\xa7\xb6\x51\x20\x84\xe5\xfc\x6d\xd9\xd6\xea\x84\x0f\x24\x0e\xa1\x85\xad\x05\x64\x75\x5d\xc1\xce\x77\x87\x55\x72\x4b\x69\x75\xde\x30\x24\x72\x04\x98\x21\x05\xc6\x25\x40\x1c\xbf\x23\x9b\xb9\xe9\xfc\xc9\xb4\x46\xd0\x80\x41\x58\x52\xb5\x27\xc1\x9f\x6d\x2b\xb9\x4f\xd1\xe1\x39\x31\xd7\xf3\xa3\x9d\xf7\xbc\x95\x6a\x74\x4e\x73\x7b\x2e\x92\x04\xc3\x93\x91\xc3\xa1\xd6\xe3\x20\x79\xc8\x1d\x00\x6a\xb1\x21\xc3\x4c\x5d\x06\x54\x92\x11\xe8\xe6\x16\xc7\x6b\xb4\x52\x94\xde\x0f\x46\xad\xfc\x24\x2a\x53\x52\xa6\xa5\x84\x06\x89\x91\x02\x81\x7e\x67\x0c\xfb\x8e\x56\xb4\x83\xf8\xd5\x72\xb3\x7c\xa2\x00\x47\x04\x2b\x79\xa3\x3d\x84\x9b\x9f\x0c\x1b\x6b\x93\x1c\x16\x68\x79\x4a\x3a\x6c\xac\x9f\x4e\xe0\x03\x3b\x20\x05\xe1\x63\xa8\xd1\x48\x61\x10\x3a\xbd\xe2\xb8\xe1\x1c\x2e\x4d\x9e\x2e\xbb\x5d\xc6\x04\xea\x2d\x53\xfa\xa6\xc1\xe0\x2f\x11\x11\x79\x3f\x38\x34\x55\xd9\x5e\xce\xcd\x3c\xb5\x8d\x79\xee\xe1\xe4\x6e\xdd\x44\x3d\xda\x7b\x10\x6e\xd2\xda\x9f\x1f\xda\xd6\x5a\xcc\xca\xe3\x61\xe0\xb2\xc8\xba\xdd\xb6\x64\xcf\x66\xdc\x66\xdb\x26\x2f\xdf\xf5\x37\xb0\x9a\xd6\x69\xb8\x76\x91\x95\x1b\x8f\x03\x60\xc5\xdc\x81\x05\x62\xc5\x4d\x99\x47\x9d\x51\x22\x9c\xd8\x4d\x30\x0f\xaf\x50\x21\x18\x83\xab\xe9\xa8\xb8\x5f\x94\x71\x2e\xa6\xc5\xf3\x74\xb1\x44\xe3\xef\x4d\x99\xf5\x2f\xa7\xd5\x3c\xae\x81\x69\x76\x93\x8a\xf2\x38\xd5\xdb\x46\x1b\x24\x11\xd2\x80\xbb\xdd\xe3\x1b\x21\x9a\x5f\xc3\xcb\xd5\x5c\x2d\x97\x6f\x6a\x77\x5f\x8f\xb2\x36\x4f\x91\xda\xb0\x34\x3c\x8c\xa8\x5c\x26\xf5\x8b\x90\x33\xb0\x5e\x98\x0d\xd8\xa8\x19\xbd\xda\x8f\xf4\xea\x2c\x11\xa0\xbd\x0f\xb7\x42\xdf\x20\xbd\xb7\x54\x95\xbc\x13\x15\x53\xc4\xfd\x02\x9d\xf7\xa8\x14\x75\xc9\x85\x8c\x6c\xe1\x7e\xf5\x8c\x47\xe8\x5d\x28\x7c\x87\xdf\x02\x29\xe5\x75\xbb\x09\x5b\x45\x26\x30\x78\x4a\x41\x94\x21\x1c\xf4\x21\x26\x93\x02\x73\xd8\x07\xb5\x12\x62\x8f\x8b\x85\xc8\x84\x93\x72\xcb\x90\x04\x35\x21\x06\xa5\x84\x30\x53\xf6\x81\x69\xc1\xb1\x14\xf0\x6c\xc4\xf9\x4e\xce\x2e\xe3\xd4\x5b\xaf\xb4\x4a\x89\x0a\x88\xbf\xf3\xfc\x8b\x88\x24\x88\x03\x01\xfd\xd9\xd5\xf2\xd6\x6c\x96\xaf\x5e\xcd\x4b\xe6\xcf\x8c\x1d\xb4\x20\x86\x53\x3b\xd3\x2f\xf9\xf9\xf1\xcd\xa0\x77\x57\x49\x7b\x1b\x9e\x87\xd9\x00\x95\xec\xe0\xc4\x12\x25\xf0\x7b\x88\x4d\x62\x86\x6e\x95\xa1\xed\x8a\x3f\x4f\x2f\x29\x95\xa6\x4a\xee\x7d\x2f\xdf\xc1\x19\x8f\x2f\x6a\x79\x5a\x6e\x16\x54\x06\xd5\x3a\x61\x02\xec\x1a\x02\xb3\x4e\x20\xba\xf4\xf4\x69\x10\x71\x82\x2f\xdd\x85\xad\x6c\xb7\xd2\x48\x8a\x6a\xce\x30\xaf\xd7\x23\x6c\xb4\x08\x91\x7b\x92\xc3\xea\x70\xfe\x93\x61\xd2\xb7\x37\xc0\xbe\x3a\x62\xc7\x70\x22\x10\xc8\x89\x1a\xbe\x24\x30\xfa\x84\x2f\x2f\x13\x47\x6d\x72\x65\x6f\xd7\x32\x14\x22\x54\x2f\xe1\x44\x77\x39\x87\x93\xc8\x98\x6d\x63\xf4\xdb\x84\x9f\x7f\x8f\xbe\xff\x31\x31\xd0\xd6\xbc\xfa\x90\x69\x99\x9a\xeb\xb4\x15\xba\xc4\xeb\x6b\x2c\x12\xdf\x85\x73\xc3\xdf\xe0\xcd\xea\x8f\x65\x39\xab\xbf\x9f\xbe\x87\xe3\x14\x34\x25\x3d\xa2\x5e\xe1\x43\xa6\x50\x23\x86\xf7\xb7\xe5\xf2\xba\x18\xe4\x91\x89\x16\xb7\x73\x4c\x77\xb7\x49\xc6\x82\x9d\x1d\x1a\x1a\x1a\x05\x63\x2d\x12\xa6\xdc\x24\xb6\x3c\xae\xaf\xd6\xd5\xe2\xcd\xdf\xd6\xd3\x15\xc1\x5a\xb3\xfc\xd3\x32\xd7\x51\x7b\x41\x77\xa8\x0f\x88\x70\x1d\x15\x18\xc4\x05\xfe\x11\x15\x78\x30\xd9\xf9\x06\x56\x01\xc3\x6e\x61\xbc\xeb\xcb\x77\x1b\x52\x3e\x78\x96\x48\xec\x9e\xd8\xe4\x95\xb2\xfa\x0b\xfe\x83\x66\xc1\xf4\x03\xe7\x93\xd9\x04\xb7\x5f\x14\x62\x67\x36\x42\x5c\x95\x09\x1c\xf4\x81\x75\x00\x6a\x51\xef\x59\x2d\x6e\xca\xa1\xf0\x01\x16\xf9\x78\xbf\x40\xfd\x86\xc8\x72\x9a\xcb\x9f\xac\x15\x45\xe6\xd3\x66\x11\x77\xf9\x8e\xdb\x8e\x3c\x30\x82\x4c\x83\xce\xd2\x3a\x6d\x62\x31\xff\x8c\xac\xc7\xd4\x1e\x4e\x53\xfa\x62\xec\x26\x03\x45\x03\xe7\x30\xa7\x50\x73\x38\x35\x77\x93\x2b\x6c\x62\x57\x7d\xec\x2d\xcd\x48\x4a\xe0\x6d\xde\x9b\xf5\xa0\x20\xa2\x85\xd8\x46\x6b\x51\x65\xc4\x43\x1d\xd2\x3a\x72\x8e\xb6\xa3\xb6\x02\x1f\xdd\xcd\x02\x93\xd5\xbb\x68\x22\x11\xc7\xe2\x89\x01\xb3\x58\x8e\xf7\xc2\x1f\xb4\xbe\xc3\x92\x3a\x0d\x76\xee\xe9\x84\x40\x00\x7b\xa2\x1b\x1f\x23\xd4\xf6\x4f\x90\xc5\x3d\xe5\x27\x99\xd7\x6f\x11\x66\xda\xf9\x8a\x20\x65\xc9\xf6\xc8\x9b\x25\xbc\x1a\x09\xaa\x16\x04\x09\x4b\x7a\xd2\x33\xba\x5f\x53\x62\x89\x86\xe3\x5e\xb6\xf7\xdb\xb3\x53\xc2\x66\x80\x7a\x85\x2d\x4a\x02\xda\xaa\x05\xce\x21\xde\x29\x48\xcf\x62\xf6\xf8\x23\xdb\xaf\xaf\x3d\xa2\xe2\x28\xe7\x6d\x96\xb8\xa0\x80\x6f\x55\x9b\xc3\xb5\x7d\x03\xac\xc5\xb7\xef\x52\xf6\xe3\x59\x5c\xcc\x6f\x66\x25\x59\xeb\x30\x29\xdc\x5e\x55\x17\x57\x06\x1d\xb2\xea\xe2\x8e\x61\x26\xeb\x75\x52\x2f\xcb\x5c\xba\x3a\x85\xff\x81\x63\xb2\x96\x77\xc7\x41\xd3\xd4\x6a\x11\x58\x9e\x51\x17\x63\xb5\x7f\xeb\x51\x89\x49\x21\xa9\xd6\x58\x4d\x92\x49\xe9\x6f\xaf\xe6\x5c\xc3\x30\x15\x54\x03\x44\xd1\xe9\x45\xb5\x79\xcf\xf5\xfb\x6c\x49\x43\x17\x00\xdc\x4f\x64\x7d\x46\x11\x4b\xca\x36\xc7\xc2\xda\xcf\x24\xeb\xb2\xfc\x2f\xf1\x26\x37\xf2\xcd\xc2\xff\x5a\x95\xb7\xe2\x54\xc7\x7e\x24\xdc\x40\xde\x19\xec\x58\x59\x34\x9d\xbd\xff\xbe\xaa\x37\x66\xbd\x5c\x6e\xb8\x17\xd3\xb0\xf5\xd7\xfb\x58\x9b\x60\xd4\xe0\xa6\x10\x35\xa1\x35\xc0\x01\x6e\xcf\x0a\x05\xc9\xd0\x6f\xf3\x92\x1b\xd7\x74\xe9\xea\xe5\x67\x9a\xf0\x99\xb9\x98\x4f\xeb\xfa\x01\x79\xbb\xa1\x9b\x52\x18\x15\x03\xa5\x24\x4a\x12\x35\x6b\x91\x0c\xfa\x7f\xe9\x0f\x12\x4e\x64\x45\x5d\x23\x92\x06\x27\x73\x11\xe4\xae\xcd\x50\x1b\x90\x28\xc3\x45\x65\x73\x23\x9f\x52\xb9\x00\x2d\xfc\x9f\x2e\x1b\x45\x24\x9d\x0b\x6d\x96\xcf\xe8\x0a\xbb\x70\x63\xe9\x6b\x1a\x97\xb8\x9a\xd6\x3f\xdd\x2e\xfc\x7c\x4e\x11\x35\x88\x20\x1f\xbd\x72\x0a\x7f\xb8\xe4\xa7\x23\x9b\xac\x93\xc2\x6a\xf1\xba\x30\xe2\x6d\x10\x2c\x41\x3d\xe1\x2d\x78\xf1\xc5\x65\xbd\x09\xcf\x04\xdb\xab\xc5\xcd\x75\x71\x7f\xdc\x3b\x9e\x8c\xd2\x51\x7e\x36\xbb\x77\xd6\xdf\x66\x67\xb3\x1e\x7c\x8c\xcb\x27\x13\xca\x80\xcf\x6d\x76\xbf\x5f\x2f\x6f\xd6\x8a\x19\x6b\xd1\x51\xdc\x3f\x7b\xd6\xbb\xff\xca\xac\x09\xfc\xfb\x2f\xc6\x67\xf5\xd9\xcd\xb7\x4f\xbe\xfd\xf6\xec\xdd\xc3\x93\x49\x6f\x1b\x7d\x7f\x86\x45\x7f\xbd\x81\xa3\x26\xda\x8d\x40\x79\xe8\x27\xfd\x7a\x7c\x76\x7b\xf6\xb7\x49\xef\x34\x1b\xbf\x38\x9d\xdc\xdb\xfe\x29\x85\x84\xe3\xc9\xbd\x2c\xfb\xec\xbe\x59\x23\xa7\x98\x97\xcf\xa7\xaf\xa0\xf4\xd7\xe9\xd9\x6d\x2f\x3b\xab\xef\x9d\xdd\x1f\x9d\x42\xd5\xaf\xcf\xee\x9f\x0d\x4e\xb7\x54\x0e\xb6\xde\x6a\x86\x5e\x0a\x35\x81\x31\x31\xf9\xdd\xee\xac\x9e\xdc\xb3\x79\xe4\x7f\x0f\x99\x50\xef\xc5\x36\xdf\x9a\x0c\x07\x0c\x4d\x8d\x33\x1a\x00\x95\x29\xeb\x8b\xe9\x0a\x07\x75\x86\x08\x48\xce\xce\xce\xee\x9f\x5f\x2e\xd6\x9b\xc9\xf6\x66\x7c\x36\x9b\x1e\x5f\x3e\x3c\xfe\x76\x72\xf7\xe5\x2e\xb3\x35\x36\xcb\x37\x70\x7e\x2f\xee\x27\xe3\x17\x58\x7c\x7d\xb6\x98\xdc\x4b\xb6\x18\x37\x60\x4b\xb1\x02\xc8\x74\x72\x7b\xcc\xd8\xed\xed\xc5\x2e\x34\x77\x2d\xae\x10\x00\xfe\xf1\x75\x7d\x0c\x60\xcf\xa6\xf5\xd5\xc3\xf9\xea\x6a\x5a\xdc\x3f\x4e\xa9\xff\xdf\x26\x50\xb4\x32\x97\xf6\xb0\xe9\x08\x03\x4e\xfc\x70\x64\xd9\x6c\xd0\x11\x5b\x48\x82\x3f\x43\x87\x0d\x9c\x77\x3e\x57\xcf\x5c\x5d\xf2\x9e\xcb\xee\xac\x13\x32\x3a\x1f\x92\xf7\x1c\xb2\x09\xd4\x83\xc3\x66\x01\xe2\xe9\x8c\xcc\x8b\xc9\x3b\x51\xa2\x25\x69\x5b\x74\xd7\xac\x95\x89\xc1\x3c\xc3\x1b\x0e\xf6\x90\x62\xf3\x39\x63\x7d\xf3\x20\x37\x85\x8d\xc6\x70\x8e\x7f\xc0\xd8\xdb\xff\x28\xf5\x5a\xb7\xbe\x7d\x9a\x9b\x26\x8f\x7f\xfa\x41\x6e\xa4\xbf\x07\x30\xcb\x59\xe2\x46\x89\x5a\x4f\x23\x0c\xa7\xb5\x2e\x0d\x2c\x2c\x9f\xe5\xae\x3f\x86\x92\x7d\x09\x93\xe5\x82\xc0\xa7\xeb\x1b\xa0\xb5\xc5\xab\xd2\xab\x69\xbb\x89\xaa\x44\x1d\xc0\xd0\x87\x76\xa1\xaa\x48\x67\xb7\xcd\xe2\xee\x35\x79\x57\xe5\x3e\x2b\xc0\x05\xcc\xc6\x21\xcb\xb5\xa8\x5c\x0d\xae\xf0\x7c\x2f\x5b\xf0\x97\xba\xb3\xbc\x27\x1d\x15\x39\x66\x69\xf9\x40\x15\xd8\xe6\xde\x6f\x0b\xa2\x48\xcc\x0e\xcd\xc9\xd7\x38\xdd\x9a\xe1\x3c\x83\xe0\x48\x72\xda\x96\x63\xbf\x35\x32\x16\x94\x8c\xd2\x4e\x8b\x2f\x46\x63\x5c\x29\xc6\x0e\x04\xbf\x26\xb9\x63\x16\x2c\x9f\x5b\x70\x4c\x87\x80\xd9\x6e\x3b\xea\x26\x2d\x66\x05\xc0\x12\xe5\xea\x41\x3e\xad\x63\x59\x9f\x31\x3b\x4a\x6d\x86\x87\xa4\x3e\x7a\xb1\xbb\xe6\x73\x44\x48\xdf\x43\xbb\x56\x8a\x0b\x0e\x15\x2d\x78\xdf\x7c\xa7\xb6\x0e\xef\x36\xea\xe1\x86\x66\x3b\x9b\xe9\xe2\x02\x71\xc9\x5d\xa9\x17\x1d\x1c\x75\x72\x9d\xad\xc0\x91\x8c\x54\x08\x7e\x70\x10\xf4\xc0\xb4\x9d\xe8\x0c\xdb\xe1\x3a\x21\x40\x13\x02\x4f\x25\x3b\xfc\xdc\x6e\xd8\x1d\x90\xf3\x3c\xc6\xca\xf6\x17\x76\x10\xd6\xc6\xa6\xaa\x7f\x9e\x4f\xab\x85\x06\x96\x91\xf1\xd3\xb5\x33\x15\xb6\x56\x1e\x90\xda\x54\x57\xe2\x28\xc6\x54\x6e\x92\x8d\xbc\x0f\x6d\x48\xf3\x18\xd3\x28\x12\x71\xab\x26\xca\x1f\x7a\x14\x8a\x67\x24\x32\x9c\xb5\xab\xd3\x39\x39\x7e\xf3\xfe\xe9\x4c\x06\xf1\x60\xc2\xb6\xba\xdd\xae\x27\x04\xa1\x93\x21\x4d\x10\xa5\x55\x33\x38\x36\xdb\xd2\xaa\x9f\xb5\xe4\x10\x4f\x32\xc1\x28\xa7\xb3\x01\xe9\xa0\x61\xf6\x48\x04\xdb\x79\xe0\xf5\x75\xee\x2d\xa6\x29\x55\x9b\xb1\x0b\x83\x95\xd8\xf6\x86\x54\x56\x82\x9d\xc6\x34\x68\x4b\x3b\x0b\x2a\x87\xb0\x64\x79\x13\xfd\x16\xf6\x91\x37\x2a\xe6\xbd\x8e\xc8\xdd\xd2\xd4\x1f\x2d\x41\xb3\x9a\xf0\xf7\x43\x58\x14\x52\xb7\xee\x85\x28\x2c\x59\x4f\xdf\x94\xac\xd6\x0c\x2a\xe2\x1e\xa0\x09\x79\x92\x18\xbd\xa9\x30\x75\xf5\x5b\x99\x7b\xdb\x82\x8f\x62\x2e\xb4\x83\x03\x2a\xb5\xd8\x52\xcc\xc9\x76\xac\x06\xa7\xae\x76\xe8\xf7\xe9\x0a\x83\xd4\xe3\x04\x27\x52\x99\x41\x0a\x51\x69\x5f\x1a\x86\x63\xec\xc9\xa9\x26\x8e\xbd\xae\x7b\x90\x38\x21\xa2\x1d\xe3\xaf\x9d\x41\x81\x12\x76\xbb\x8b\x37\x79\x10\x20\xc4\x5e\xcd\x6e\x8a\xc6\xea\x0e\x18\x0b\xd3\x6a\xed\x87\x75\xe9\xa3\xcd\x00\xaf\x3b\x56\x28\x62\x9a\x4f\x10\x7d\xcb\xe6\xcb\xcd\xce\xe0\xdd\xb7\xeb\x1c\x07\x7d\x0e\xe0\x18\x38\xc6\xd4\xf1\x69\x82\xae\xc9\xa9\xc9\xb0\xd8\xce\x10\x6d\xb8\x56\x2e\x17\x71\x55\x2a\x60\xef\x0a\x45\x8f\x7f\xb9\x90\x2b\x19\x43\x18\xdf\x37\x6b\x16\x47\xa9\x37\x3b\xd3\xd5\x6a\xfe\x9e\x81\x01\x18\x6e\x38\x40\x05\x40\x42\x26\x50\xfb\x5a\x2a\x7f\x85\x9d\x66\x67\x40\x96\x3e\x54\xe4\x18\x6f\x3a\xcb\x5f\x5d\x89\xca\x46\x94\x2c\xbc\xb9\x34\xaf\x8b\x5e\xd5\x4b\x4f\x4e\xab\x11\x24\xc0\xf1\x77\xd8\x0e\xf4\xeb\xd3\x82\x03\x3b\x9e\xbe\x1e\x11\x2d\x8c\x5f\x4f\x26\xf9\x78\x02\x9d\x5c\x4f\x57\x4d\xd4\xef\x1b\xbc\x17\xd3\x8c\x86\xdd\x08\x01\xa4\x24\x2c\xed\x34\x63\xff\xec\x10\x41\x70\x84\xde\x8b\x68\x4b\x38\xdb\x6d\x83\xd2\x38\x38\x17\x11\x6c\x6e\x0f\x48\xa6\x5e\xae\x37\x30\x96\x3e\xfe\x35\x6c\x81\x43\x9f\xf4\x6b\x67\xc2\x33\x85\x27\xa8\xd8\x0c\x13\x5c\x24\xb9\x74\x4d\x88\x34\x3b\x18\x60\xe2\x62\xb9\x7a\xff\x94\xaf\x3a\xe8\x37\xdf\xbb\xa8\x9e\x80\xe2\x82\x88\x2d\x67\x61\x49\x83\xac\xf3\xc8\xa6\x6c\x60\x6f\xe9\x35\xcf\xd9\x33\x94\xab\x42\x6c\x0e\x92\xf3\xe5\x12\xe3\x9d\x39\x59\x86\x5b\x24\x99\x14\x8a\x89\xe1\x68\xa3\x9b\x81\x76\xf3\x20\x33\x0d\x97\x03\x2e\x6d\x95\x37\xfe\x7e\x47\x39\x19\x46\x2a\xe4\x16\x51\xe9\x26\x70\x16\x45\x85\xcc\x94\xd3\x69\xe6\x8f\x8f\x2b\x77\xf5\x48\xbe\xed\x78\x9d\xc4\x8e\xef\x82\x06\x0f\xa4\x6a\xc2\xdb\xae\xf5\x42\x90\x1b\x75\x40\xa5\x0c\x43\xae\x6b\x10\x97\x1a\x44\x54\x7d\x10\x28\x9f\xdc\xf9\x57\xef\x65\xec\x28\x3b\xd0\xc7\x9e\x9d\x7e\xf5\x1e\xc7\xe1\x4d\x52\x11\x5d\x4f\x51\x91\x2c\x1b\xf9\x65\x82\x0f\x98\x04\x9e\x46\xb4\xe3\xf5\x24\x0a\xd9\x06\xd6\x17\xd9\x08\xfe\xc1\x25\x94\xb7\x15\xf3\xc1\xb1\x85\x61\x52\xfc\xc1\x46\x61\x4e\xc8\x7a\x99\x29\x87\xa1\xcb\xf1\x4f\xbc\x9f\xf9\xf5\xb9\x98\x73\xff\x27\xd3\x61\x13\xd9\x3c\x2c\x96\x70\xb0\xb8\x84\xa5\xe0\xb1\x1b\xb2\x35\xd6\x65\xa7\xba\x0c\xab\xd3\x87\x6e\x6c\xda\xcb\xcf\x32\xc3\x08\x0f\x74\x20\x2d\x65\x25\x43\x54\x26\xba\x75\x62\x6c\xcf\x5f\x88\x31\x03\x3e\x89\x01\xff\x6d\x0a\x62\xff\xc0\x5c\x2d\xe7\xb3\x5f\x42\x8e\x8d\x49\x78\x91\x32\x9f\x8d\x7c\x8e\x8d\x15\x7a\xbd\x3c\x90\x00\x3a\x27\x4d\x86\x7f\x0b\xe5\x48\x3e\xc2\x1f\x6c\xd7\xd7\x39\x3e\x8e\x5b\xca\x5d\x1c\x38\xea\x5e\x2c\x68\x55\x1c\x23\x1f\xa5\xa6\xe9\x97\xdf\x4a\x36\x0c\x5b\x40\xc7\x04\xec\x52\x8c\x03\x9b\x5d\x9e\x9e\x00\x2d\x5a\xa5\x55\x60\x54\x60\xe5\x2c\x09\x1b\x33\xc9\x3c\x76\x05\x07\x99\x57\xaf\xd0\x7e\xd4\x79\x59\x89\xeb\x98\xe4\xa4\x09\xb5\x9a\x64\xe8\x9c\x69\x3f\xb2\x1d\x22\x5d\x97\xb5\xc3\x0f\x45\xcc\x13\xa7\x05\x2f\x5e\x6a\xc3\x87\x08\x6b\xb3\x68\x42\xff\x2a\xcd\x6f\xb7\xad\x2d\xd9\xf0\xc3\x6d\xcd\x88\xde\xae\xad\xe2\x11\xf3\x0a\x48\xe9\x76\xe1\x1f\x0e\x70\xcc\xa4\x44\xf1\x60\x61\xa4\xeb\xea\xa2\xb5\x6a\x07\xb2\xa7\x3f\xa6\x9e\xcf\x24\xb9\x3e\xe1\xdd\xe3\xb7\xc8\xe2\xb5\x7f\x04\x65\x7f\xdf\xd4\xe3\x88\xd5\x5c\x94\x99\x37\x78\x25\x7c\xb7\xc6\x96\xc5\x8a\x4e\x19\x36\x0e\xb4\x68\xbc\xe3\x61\x7b\x13\xe7\xa9\x95\xbb\x8a\x38\x38\x8f\x39\x44\xf0\x21\x2d\x72\xaf\xfb\x7d\xbc\xb6\x5b\x3f\x16\xb4\xc7\xc9\xbd\xa8\xd1\xce\x78\x0d\x2d\x24\xa1\x55\xac\xe2\xed\xa4\xdd\x6e\xc7\xd3\xec\x59\x98\x4d\xe2\x95\xc1\xab\xbc\xd6\x52\x7e\x43\x6e\x3b\x35\x09\x0c\x4c\x3f\x7e\xba\x4c\x1c\x0c\xb1\x75\xaa\xa8\x71\xdf\x94\xef\x7d\xc7\x0b\x3f\x10\x36\x24\x85\x91\x4c\x5a\x81\x85\x52\x44\x65\x5e\x3c\xc7\x08\x9d\x7b\xe3\x43\x36\x23\x3b\x8a\x79\xac\xad\x7f\x5d\xbf\xca\xee\xc8\x30\x97\x14\x93\x4f\x30\x97\x12\x41\xfc\xd0\x23\x6d\x1e\xb8\xb5\x59\xcd\x05\x06\xcb\xe3\x38\x49\x9e\x7d\x3e\x4c\x69\x9b\xa7\x50\xe6\x51\xe4\xb0\xb9\xe1\xdb\xa3\x72\xea\x35\xaa\x27\x73\xed\xb0\x40\x05\x51\x74\x6a\xdf\x6e\x95\x5b\x90\x0f\x1c\x81\x3c\x2b\xbc\x23\x33\x29\x23\x08\x02\x53\x4b\xb3\x1d\xaf\x8f\x6e\x77\x6c\xbd\x77\xb9\xf2\x68\x6c\x43\xde\x04\x9e\x9e\x9c\x8b\xe7\xee\x49\x2e\x1f\x4a\xb2\xe4\x67\x68\xe3\xf9\x91\xa3\x93\x3d\xfd\x6b\xaf\xb6\x7b\xcb\xe6\x34\xc3\xf9\x7d\x86\x51\x6a\x26\x3c\x01\x33\x2f\xf4\x4e\xa6\xb3\x82\xe1\x3b\xc3\x59\x89\xb7\x39\x2c\x61\x37\x33\xfc\x60\x0d\xc5\xa8\x91\x22\xc1\x4c\x99\x55\x70\x1c\x54\x32\x72\x6e\x0d\xb9\x2a\xf7\x8c\x45\x10\xa0\x6a\xaa\x81\x9c\xbb\x5d\x4f\xb9\xeb\x02\xa8\xba\x28\xb0\x9e\xea\xd6\x24\xff\x3d\xc9\xa2\x1c\x56\xd1\x9a\x64\xd2\xc8\x61\x95\xb0\x49\x30\x26\xc7\x08\xe9\xd4\x4a\x73\x89\x0c\x9b\x3d\x3f\x32\x38\x0d\x5a\x73\x29\x22\xe5\xe4\xe9\x82\x5a\x38\x22\x94\x69\x31\xc3\x0e\x14\x8a\xce\xbf\xc7\x34\xce\x4b\xea\xdd\xf5\x1c\x6d\xf1\x87\x9f\x42\xdb\xc8\x87\x04\xc7\x8f\x7f\xfa\xe1\x67\x32\x4c\x1f\x91\x99\x3f\x82\x6d\x93\x0c\xb4\x8d\x66\xf5\x3c\x07\xdf\xc2\x59\x4d\x38\x34\xad\x30\x8a\x62\x75\x1f\x8a\x24\xa8\xe1\xc4\xa2\x58\xf9\x90\xb9\x37\x34\x9c\x64\xd8\x68\x7f\x5a\xbf\x5f\x5c\x14\x12\xff\x96\x52\x50\xad\x09\x45\x52\x09\x2a\x67\xb9\x14\xb6\xdb\x88\x47\x0b\x89\xdd\x2e\xd6\x8a\x1c\xc0\x81\x47\x62\x6a\x6b\xac\xaa\xc0\xfe\xde\x59\x7e\xb4\xcf\x03\xe2\xda\x4e\x03\x34\xb9\x33\x18\xf4\xdc\x3f\x26\xed\x8c\x0b\x43\x18\x4f\x0b\x93\x59\x83\xfc\x9c\x78\x86\xeb\x9d\xd7\xb6\xb7\x91\x73\x5d\x2d\x01\xcd\x32\x73\x95\x67\x0e\x28\x77\x97\x71\xa9\x9d\xb1\x2a\x7b\x4f\x69\x4b\x73\xe3\x6c\xe5\x79\x07\x74\xc1\x8d\xe5\x42\xc0\x24\xd7\xf5\xb1\x4f\xbb\xf6\x5e\xc0\xbb\x08\xc8\x70\xc0\x6c\xed\x11\xea\x21\x24\xe2\x93\xef\x11\xee\x02\xb8\x05\x9f\xa1\x17\x8c\x84\xb3\x08\x13\x63\x5d\x03\xee\x23\x91\xbe\xc1\xd9\xd2\x55\xce\x3a\xd5\x7b\xc2\x41\xe4\xa1\x22\x7e\xf1\x01\x97\x03\xb7\x40\xa1\x5f\x29\x53\x02\x95\xca\x31\x49\x03\x04\x15\xf6\x78\xcc\x1a\x04\xf4\xb7\xc6\xdb\x5b\xac\x6c\x34\x68\x1c\x30\x75\x3a\x56\x39\x17\xee\xca\x8b\x6e\xfc\x91\xed\x14\xd4\x0e\xbb\x9f\xa8\x2f\xf9\x47\x82\xa6\xdb\x2c\xb6\x58\x19\xfe\xf1\x3b\xa1\x3b\xd0\x94\x07\xa0\x0d\x8d\x85\x42\x12\x92\x70\x6e\xaf\x15\x55\x10\xc1\xdf\xdc\x5a\xf2\xff\xfc\x9f\xff\xd7\xff\xfd\x7f\x24\xd9\xc8\xce\x64\x78\x97\x48\x5c\x9b\x62\x1b\x26\x49\x1e\xd5\xa5\x92\xbb\xfc\x63\x6a\x52\x5e\x2f\xf1\x69\x97\x2e\x35\x21\x05\x35\x27\xa2\x15\x74\x4d\x81\x34\x6c\xd8\x48\xd6\xd3\x9e\x49\x02\xb9\x8e\x07\xc2\x2f\x94\xa6\x08\xe7\x8e\x8e\x84\x87\x41\x46\x16\x05\x67\xc3\xc7\x28\x1a\x5b\x0f\x94\x1b\x8d\xe1\x9f\x49\x8e\x35\x9c\x4a\x84\xc7\x89\x55\xa8\x25\x56\xb2\x49\x80\xdf\x68\x71\x21\xc4\x4e\xb7\xc4\x34\xbc\x96\x7b\x00\x77\x83\x9c\xf9\xaa\x49\x49\x13\xf3\x61\xa8\x2f\xca\x1f\x92\xb2\x50\x41\x05\x69\x76\xb1\x14\xd5\x08\xb5\x53\x9e\x43\x04\xe4\xf4\xaa\x2c\xaf\x72\x32\x5d\x71\xca\x03\x22\x20\x42\x09\xfc\x03\x14\x22\x26\x02\xf6\x49\x0a\xe1\xc4\xc7\x03\xc0\x3c\xa2\xc4\x53\xf5\xa1\xbe\xcd\xd4\x25\x88\x16\x62\x5b\x3e\x2f\xf8\xcb\x81\x41\x65\x9c\xd6\x8c\xac\xc9\x9b\x6f\xa1\xc8\xa2\x38\x7d\x3d\x7c\x0d\x40\x51\x1d\x72\xdd\x97\xe6\xc6\xaf\x27\x8e\xe0\x87\x36\xcd\x3f\xa3\x0f\xdb\x6a\xa1\xf3\xbf\xf5\x84\x74\x70\x14\x15\xab\x0a\x81\xb1\x03\x79\x45\xda\x57\xc7\x96\xaa\xc5\x5b\x3f\xb6\x36\xce\x26\xc5\x5c\x0b\xe3\xad\x59\x0f\x0f\xb6\x27\x79\x5b\x74\x3a\xf0\x6f\xa8\xa2\xb1\xc1\xb9\xb5\x6d\xee\x8a\xd6\x25\x1a\xdf\xa2\x6f\x1a\x97\x01\x91\x05\xb5\xb8\x1a\x88\x90\x23\x18\x3a\xfd\x6e\xa4\x36\x8c\x00\x06\x06\xb4\x87\x8d\xfa\x80\xb6\x32\x52\xd6\x24\xcb\x10\x87\xfb\xb8\x15\x0d\x26\x64\x32\xde\x48\xa8\x7f\xc3\x4b\x4c\xc3\x6e\x42\x83\x63\x1c\x11\xb7\x20\xf6\xb9\x31\xf7\xe2\xce\xff\x15\xed\xfa\xcb\x85\x0d\x3a\x84\x45\xc3\xbc\x41\x1e\x6a\xf3\x6f\xaa\x59\x3e\x40\x8b\xa2\x77\x81\xd2\xda\x99\x45\x50\xfc\x67\x60\xe5\x5c\x86\x04\xae\x30\xe4\x40\xcb\x29\x01\xe5\xa9\xcb\x85\x0b\xe4\xa8\xa7\x02\x68\xf7\x72\x81\x72\x55\x9b\x5d\xfe\xe5\x22\x1b\xd1\xce\x55\xc4\x37\x0f\x56\x81\x67\x1e\x64\x0c\x46\xd1\x54\xdb\x5e\x2e\x64\x6c\xf6\xe8\xa1\x1a\x71\x35\x66\x49\xf7\xb6\x9b\x91\xe4\x8e\x0d\xd3\x9b\x31\x00\x67\xf0\x37\x7a\x50\x86\x4b\x7a\x21\x3a\x77\x66\x4a\xfe\x87\x31\x25\xc2\x70\x31\x0c\x00\x93\x1f\x48\xdd\xd5\x02\x43\x30\x19\x32\xb6\xfd\x77\xe0\x8a\xeb\xe9\xad\xf8\xa0\xec\x21\xcd\xf3\x9b\xf9\x9b\x82\xb7\x82\x37\xfc\xb6\x8e\x17\xea\xd6\x3f\x90\xe3\x19\x34\xbb\xb3\x5d\xa0\x03\x8a\x25\x27\xcc\xd3\xe8\x03\x04\xa6\x83\xae\x42\xf8\x24\x56\x58\x00\x95\xdd\xa5\x75\xa3\x0d\x7c\xdb\xbd\x6e\x5a\xe6\x91\x69\x0f\x75\x4d\xd3\x5b\x74\x31\xa5\x51\x20\x8d\x4e\x6f\x47\x30\xc9\x4e\x17\x5f\x8b\xbf\x1e\xd2\x04\xa9\xd4\xf3\x94\x06\xcc\x54\x12\xee\x0f\x16\x8f\x76\xbe\xb1\x28\xb7\xe5\x5b\x44\x4a\x21\xb4\x60\x06\x7a\x6a\x59\xae\x97\x0b\xb7\x94\xb0\x51\x84\x8a\xea\x70\xb8\x3b\x0f\x3a\x5e\x6c\x51\x79\x2f\xf2\x99\x45\x03\x19\x8d\xd5\x39\x02\x34\x0a\xc6\x97\xe5\xe2\x62\x66\x1b\x39\xe1\x46\x72\x45\x36\x0a\x9b\xb7\xcd\x3b\x88\x94\x0e\x23\x68\xf8\x81\x12\x3d\x6a\xff\x40\x64\xdc\x85\xa6\x1f\x7a\x83\x54\x34\x55\x37\x56\xc9\xc7\xfe\x37\xf2\xd1\xf4\x4d\x34\x1f\xb4\x3b\xd9\xab\x7b\x54\xfa\xd8\x6b\x69\x92\xed\xcd\xf9\x68\x2b\x93\x66\xc5\x16\x13\x13\x8e\x4a\xe0\x3a\xdb\x7c\xb2\x95\x49\x54\x25\xb6\x32\x21\xc5\xc5\x66\x49\xd7\x21\x78\x7a\xc4\x9f\xbc\x24\xa5\x3e\x45\xb7\xb2\xc7\x30\x67\xef\x12\x1e\xd0\x3c\x57\x5f\x68\xa1\xdb\x85\x7f\xa0\x08\x1c\x85\x96\x73\xd8\xe5\x3a\xce\xbd\x42\xd2\x28\x26\x2d\x7b\x82\x36\x54\xc5\x02\x86\xad\x0f\x88\x29\x2f\x37\x49\xc3\xff\xda\x57\x1d\x07\xed\x9a\xaf\xe0\x6c\xbd\xc7\xa0\x68\xb7\x4b\xed\xf3\x0d\x4e\x5b\xac\xd7\x95\xac\xd2\xf4\x2f\x41\x93\x6f\x58\x5b\x74\xc4\x31\x2b\x8f\xf8\xa0\x6c\x35\x00\x47\xb4\x6f\x12\x31\x1f\xfd\x52\xbe\x7a\xf2\x6e\x75\xc4\xd2\x25\xeb\xb2\x12\xdf\xb1\xde\x8b\x77\xc5\x87\x2d\x4f\xbb\x99\x8c\x99\xf3\x1d\xb9\x00\x51\x93\xb6\x83\x55\xe6\x59\xec\x14\xb1\xc6\x9a\xe6\xd2\x7f\x1b\x0e\xc3\xcc\xcb\x60\x1e\xc9\x4e\x5b\x7b\x0b\x4a\x7d\x83\xf5\x26\xa9\xb1\xe5\x49\xc6\xa8\xf5\xbd\x39\xf5\xb0\x8c\x1f\xaa\x6b\xba\x83\x6a\xce\x50\x02\xa8\xa0\xc9\xe3\x75\x79\xbd\x5c\xbf\x47\xc9\x0c\x08\x9d\xd3\xbe\xe7\x5d\x81\x3f\x9e\x92\x73\x0d\xff\x7e\x46\x36\xca\xf4\xd8\xc5\x18\xcd\xfa\x61\x18\x45\x47\xfd\x38\x61\xef\x2b\x51\x8f\x66\x42\x37\x14\x3e\x74\x93\x11\x0a\xf5\x54\xd8\x77\xf0\xe8\x93\xe3\x5f\x70\xf7\xc8\xe7\xbd\x4e\x0b\xaf\x53\xf4\x00\xf6\x3e\x8b\x93\x00\xd4\xc2\x7b\x1b\x43\x32\xc8\x37\x12\x52\xbb\x5d\xbf\xe0\xa9\xd7\xfc\xd0\xfb\xcd\x22\x39\x96\x1f\x7b\xa9\x13\xd9\xe9\x11\x42\x0d\x7c\x80\x9a\x3f\x3a\xc5\x75\xbb\x3a\x10\xf4\xc6\xfc\x89\x5f\xbf\xc9\xee\x64\x90\xb0\x88\x6d\x80\x11\x82\x66\x60\x18\x9a\x94\xb0\x36\xa2\x7f\xfb\xfa\x18\x21\x39\xc4\x70\x92\x44\x7a\xc9\x72\x6e\x68\x24\xc8\xce\xeb\x72\x7e\xd9\x97\xb8\xf5\xa9\x58\x62\x5c\x16\x77\xc0\xbb\xf2\xd0\x9f\x1b\x2b\xc8\xc5\x2d\x61\xca\x7f\x36\xa4\xe3\xc7\x33\x93\xd3\xbb\xbf\xc6\x48\xfe\xf2\x22\x4a\x59\xc9\xd6\xbf\x40\x26\x41\x00\x73\x86\xc1\xbd\x0a\x26\x2b\x7d\xf6\x6f\x16\xd5\xaf\x28\x30\x12\xd4\x57\xd3\x9a\xca\x6f\xb7\xfc\x3c\x09\x4a\xd9\xf8\x0d\xa7\xb8\x57\x78\xfa\x79\x65\xf1\xe0\xf4\x6a\xf2\x96\xa3\x80\x89\xbe\x42\x9e\x14\x25\x53\x3c\xda\x47\x02\xb9\x92\x56\xea\x93\x0c\x1b\xd8\x13\xaa\x39\x3f\xcb\x7c\x63\x20\xbc\x60\x43\x05\x6c\xcb\x4d\x3d\xcf\x5c\x84\xa8\x9b\xe8\x91\x19\xc1\x96\x0d\x42\x4b\x24\x26\x5e\x0b\xca\x58\xe5\xcd\x19\x28\x48\xd4\xc0\x8e\x6b\x59\x76\x7a\x3c\x18\x66\x04\xbf\x06\xd4\xa0\x35\x37\xd0\x81\xda\x81\x08\x11\x17\x94\x1f\x52\xf6\xf1\xb1\xbf\x74\xa2\x22\x94\x76\x7c\x9c\xed\xd4\xf8\x03\xa6\xa4\xd5\x74\xe4\x52\x1d\x44\x2c\xac\x20\x24\x11\x45\x01\x8c\x79\x27\xed\xe0\xef\xed\xb6\xe3\x21\x1b\x75\x4c\x28\x65\xec\xc1\x1b\x32\x0a\xee\x53\x88\x77\x5f\x39\xe6\x26\xb2\x7c\x9c\x15\x7b\x50\xb7\xc5\x8c\x82\x40\xd9\xa1\x4d\xfb\x9b\x96\x96\xb9\x51\xd7\x1a\x37\x8f\x01\xca\xfc\xd5\x24\x9d\x60\x1b\xad\x5d\x50\x2b\x64\xe7\x42\xd7\x98\x9e\xfd\x88\x28\xf7\x03\xcb\x1d\x3a\x5a\xe0\x3f\x14\x9f\x8d\xbe\xc6\x7e\x41\xb6\x58\x1f\xb9\x9f\x29\xad\x84\x1a\xa4\x63\xc6\x2e\xf1\x42\xd8\xab\xa9\x5b\x8e\x49\x84\xd4\xce\xfc\x41\x57\x0f\xb0\x77\x22\x65\xfa\xa9\x23\xc0\x94\x36\x2c\xe0\x68\x15\xfa\xd8\x80\xc7\xab\xda\x36\xf6\x0e\x65\xec\xec\x8b\xa6\xd8\x56\xe3\xd2\x5d\x45\x3d\x8f\xa6\x28\xca\x21\xb1\x8e\x9b\x15\xc6\xce\x1d\x8f\x13\xb9\x09\xc6\xf0\xa3\xe8\x54\x67\xe2\x0d\x11\xa5\xa2\x8b\xf2\x88\xe7\x08\xe3\x90\x4b\x85\x59\x32\x31\x58\x9b\x0e\x24\x26\x41\x4f\xe2\x8f\xaa\xfc\x9a\xa2\xab\x51\x65\x0e\xde\x00\x95\xd5\xad\xb9\xad\x01\xad\x3b\xa1\xad\x6d\x53\x16\x09\x46\xf3\xa4\xc0\xbc\x2a\x03\x4b\x20\x98\x56\x42\xdb\x94\x70\x3e\x23\xe7\xb9\x96\x7c\xeb\x34\xcf\xde\xd4\x16\xfb\xe2\x2b\x1d\xcf\xc6\xe6\xca\x8f\x7c\x20\x01\xcf\x7c\x13\x93\xe8\x81\x59\x2b\x6b\x3b\x8b\xb9\xf2\x96\x12\x43\x06\xcf\x93\xe1\x0b\x40\x94\x22\xa7\x6f\x0e\xb3\x42\x29\xb8\xdd\x39\x6b\xe7\xe0\xf4\x4c\xca\x11\xe0\x2c\xf4\x63\xa8\x03\x1b\x73\xb5\xc1\x64\x12\x7b\x86\x31\xa4\x68\xbe\xbe\xc0\x5a\xed\x66\x64\x43\x2d\xe5\x19\x95\xd8\x3e\x35\x4f\xa5\xc3\x6c\x14\xa7\xa8\x79\x9b\x8e\x5a\xad\x0e\x04\xbf\x5e\x32\x3d\xdb\xe8\xbc\xde\x6d\x0e\x53\x48\x96\x6b\xc2\x98\xb1\x81\x31\xa9\x37\x57\xc9\x84\xe0\x65\xc7\x2b\xec\x6f\x64\xeb\x59\x00\x72\xb6\x10\x5b\x8c\xc6\x0a\x1c\xaa\x29\x75\x7c\xe8\xec\x6a\x70\x06\x51\xa4\xdf\x65\xae\xda\x4e\x69\xeb\x03\x66\x03\xa3\x70\xc9\xa1\x22\x5f\xb1\x91\xcb\x8f\xdd\xce\xc5\x87\x71\x4f\x10\x4b\x66\x7f\x55\xad\x4a\x05\xbf\x8f\xf4\x65\x3e\x9a\x30\x88\x3f\xf3\xfc\x3e\x90\xb5\x21\x3e\x34\x9c\xf8\xc5\x64\x28\x0d\x3b\x2a\xe0\xfd\x18\xf6\x6f\xbf\x7c\xb7\xab\xa9\xa1\x7f\x29\x2e\x36\xaf\xd8\x8e\x3b\xaf\xc7\x83\x17\xd5\x04\xba\x54\x4e\xad\xc9\x0f\x28\x11\xd9\x75\x66\x22\xfa\x3b\x81\x9e\xf7\xaf\x3e\x5b\x28\x9e\x56\x2d\x30\xd2\xc9\x68\x67\x92\x7b\xdb\xe1\xd1\x2a\x8b\xe5\x70\x17\x8c\x75\x99\x67\xad\xc9\x87\x0f\x58\x07\xf0\x2f\x9f\xe0\x35\xc7\xb8\x22\xfa\x6b\x67\x6e\x03\x3e\x50\xdf\x60\xa8\xe7\x6a\x81\x27\x66\x75\xba\x24\x32\xa6\x87\x6b\x48\x73\x46\x9f\x8f\x78\xb7\xa1\x08\x70\xb8\x0a\xec\x37\xea\x7f\x24\x8d\xab\xec\x57\x81\x59\x5b\xb8\xa0\xbc\xca\xd8\x2e\xf4\x0f\x3e\xd7\xa8\x37\x7d\x1e\x7c\x6d\xcb\xd8\xcb\x76\x2b\xd9\x1a\x11\x5b\xd2\x45\x8f\x70\xdb\xfe\xc8\xab\x94\x37\x15\x0b\x37\xab\x19\x86\x96\x81\x1e\x0a\x8f\x74\x2f\x74\xbc\xfc\x3a\x42\x33\x2c\xaf\x68\x78\xb4\x1c\x6a\xdf\x69\xc2\xf5\x35\x85\x49\xc3\x52\xf1\x74\x30\xda\x8f\xaa\xdc\x73\x26\x16\x26\xe1\x4d\xcb\xa8\x2d\x84\x50\x0c\x62\x7e\x7c\x6c\xc7\xbc\xdd\xb6\x46\x64\x89\xab\xfc\x17\xd8\x94\xad\x0f\xf2\xe9\x20\x13\xd7\x5a\xaf\x67\xbe\x2e\x26\x31\x4e\x64\xb5\x06\x89\xb4\x14\x89\x88\xa6\x59\x22\x56\xb2\x7b\xf4\x01\xa8\x6b\x67\xe0\x61\x19\x9f\x91\xb7\xe7\x28\x43\x77\x13\x0c\xf3\x1a\x93\x73\x50\x39\x13\x5e\x1f\x85\xa7\xf1\x78\x7d\xd0\x56\x63\xad\x84\xa8\xcb\x82\x19\x71\x57\x01\x87\xa7\xe8\x30\x80\xa6\x11\xc0\xc9\x57\xb6\x89\x5b\x7e\xc3\xfa\x96\x93\x0d\xba\x8b\x4d\x0d\x3d\x75\x25\x46\xf8\xe6\x52\x4c\x4d\xf0\x58\xcf\x0f\xce\xfd\xc8\x0f\xb1\x3f\xe3\x3a\x18\x59\x0c\x24\xe7\xb7\xf1\xd3\x93\x6a\xca\x92\x40\x5e\xc2\x8f\xee\x56\x6f\xfb\xb5\xff\xdc\x6e\x42\x1a\x11\x6c\x0e\x64\xa6\x0d\xbd\x36\xf3\xd6\x7b\x6b\x2c\x39\x3a\xfa\x7a\x5e\x2d\xde\xdc\x3f\xfd\x7a\x83\x8c\xf9\xf4\xeb\xfb\xf2\x77\x7a\x74\xb5\x2e\x2f\x8b\xcf\xef\x4f\x3f\x3f\x9d\x7e\x7d\x7f\x7a\xfa\x35\x81\xcc\x07\xc8\xcf\xe9\xcd\xa3\xf3\xe5\x3b\x8c\x49\x8f\x23\x2a\xb0\xd9\x76\x5b\x82\x7b\xd0\xe9\xf4\x40\xfe\x34\xc1\xb7\x28\x4d\x07\x5a\x81\xf3\xc8\x14\xff\x9b\xcf\xf5\x48\x22\x51\xfb\x76\x43\xc6\xd4\x5e\x04\x70\x76\x42\x21\x14\xc4\x1d\x22\x78\xd7\x72\x5f\x3d\x79\xbd\x0c\x5f\xf5\xc0\xd1\x1d\x00\x93\xdf\x26\x23\x50\xa7\xf2\x7e\x8b\x44\xe1\x2f\x12\x8c\x15\x35\x58\xbd\x1b\x52\x24\x89\x1c\x55\x6b\x43\xf1\x41\xce\xfb\x5f\x25\x46\x09\xe2\x0e\x9f\x9d\xf0\x26\x27\x4f\x36\xfc\x5a\xc0\xdb\xbe\x9d\x25\x60\xd6\x53\x94\x54\xff\x86\x3e\x9e\x35\x34\x52\xe6\x5f\x14\x5c\x86\xae\xdc\x68\x38\xee\x7d\xf1\x0d\xda\x7d\xe6\x9d\xfd\x60\x53\x01\x6b\xc3\x61\x30\x2e\xff\xb3\x72\x5d\x4d\xe7\xe8\xe9\xd1\x39\x50\x11\xa9\xc2\xd5\xa3\x01\xe7\xf7\x61\xa0\xf7\xd9\x0e\x68\x1a\xbe\xea\x9c\x50\x01\xc4\x23\x52\xcd\x8f\xcb\xf5\x35\xf5\x30\xcb\x93\xfb\xf4\x72\x5b\x5c\x1c\x4b\xd1\x6c\x31\x96\xee\xbf\x38\xe9\x7f\x65\x5b\x96\x28\x13\x9c\x97\xe1\x73\x43\x64\x2d\x09\xe0\x7a\x98\xa7\x24\x43\x74\xf8\xd3\x22\xc7\x3b\x43\x98\x1f\x7e\x96\x0e\x89\xc0\xbe\x2a\x07\xbf\xed\xb3\x73\xa6\x5c\xd0\x23\x05\x38\xf0\x3d\x04\x01\xbc\xf7\x1a\x86\x2d\x05\x8d\x7b\xd4\x2f\x4f\xbe\xce\x17\xd3\xb7\xb0\x40\xe8\x0f\x4d\xdc\x9e\x36\x20\x1f\x1f\x6a\xd0\xe7\x1d\xd1\xe8\xd7\xbd\x1b\x68\x60\xd5\xfc\x00\xa9\xf3\x3c\x79\xf4\xec\xd9\xe0\xd1\xf2\x7a\x35\xdd\x04\xda\xf4\x0b\x4a\xc2\x32\x26\x78\x12\x3c\xef\x9c\x18\xff\xdd\x46\xfc\x6e\x0f\x51\x82\x36\xcb\x71\x50\x10\xb6\x63\xe6\x37\x3d\x7e\x20\x3f\xfb\x5f\xc8\x9f\x1f\x1a\x69\xbc\xf9\x81\x89\xab\xea\x5d\x39\xe7\xd7\x85\x96\x0b\xf4\x60\xe7\x15\x62\x1f\x0d\xc4\xa7\x5d\xc2\xc7\x24\xf5\x51\x34\x29\x17\x0c\x5f\x9f\x17\x94\xa5\xa9\x5a\x00\xbf\x15\x98\xa8\xc7\x36\x19\x67\x4d\x0b\x91\x12\x5d\x62\x82\x20\xbd\x6e\x82\x30\x9b\xad\xaf\xa7\x17\x00\xaf\x2c\xe8\x3d\x73\x24\xcb\x59\x06\x15\xb2\x4c\x22\xa2\x04\x6d\x25\x8c\x0b\x05\x83\x8d\xd1\x7b\x5a\x5c\xe3\x55\x4b\x0d\x6d\x8d\x2f\x62\x61\x71\xb7\xb6\x8e\x74\x05\x8d\xe3\x03\x05\x4b\xaf\x07\xfa\xfe\xab\xd6\xb4\xfd\xc8\x4d\x74\x4b\x3b\x82\x52\x61\xeb\x6d\x25\x16\x8e\xeb\xeb\x0e\xf3\xe1\xc7\x8b\x6d\xd1\x80\x7f\x52\xf3\x0e\x58\xcd\x0a\x26\x5c\xa7\x58\xca\xd0\x37\x11\x46\x61\x9b\x8c\x88\x22\xf8\x42\xc7\x22\xe6\x6e\xf6\x2d\x4a\x98\x6c\xef\xae\x05\x6d\x15\xc3\x14\x52\x15\x80\x1c\xf7\x26\x31\xfe\x29\xa4\xe5\x8d\x53\x24\x08\xde\xfb\x62\x10\x2a\xbc\x34\xc9\xec\xe5\x27\xd4\x3e\xbf\xae\x68\x59\xf0\x05\x10\xfe\xba\x04\x2c\xd5\x15\xac\x82\x93\x5d\xd6\xd8\x61\xed\x7e\x5d\x00\x34\x49\xaf\x62\x84\x0b\x10\xe3\xaa\x97\x7c\x73\x73\x0e\x64\x5c\xc3\xd9\xc2\x16\xc5\x5b\xd6\x5b\x79\x69\x45\xc6\xc4\xad\xd5\x63\x5b\x66\xa2\x0f\xa2\x92\x92\xda\xbd\x48\xf3\x56\x78\x20\xea\x3a\x5e\xad\xf1\xa5\xc3\x47\xf3\x6a\x55\xe8\x7b\x39\xf4\x3a\x51\xcb\x48\xdb\x2b\x25\x6e\xba\x60\xe3\x59\x13\xc2\xe8\x65\xa4\xb0\x3d\xd9\x81\xda\xda\x10\xa9\x27\xd6\x15\xc8\x0b\x7e\xe5\xda\x70\x50\x8f\xc7\xd5\x5b\xb3\x99\xd5\x08\xd8\x2f\x25\xe0\xaf\xd0\xb0\x1e\xf9\xc9\x90\x4b\xc0\x0f\x7e\xff\x06\x7e\x04\xaf\xda\x40\xf2\xbb\xe3\x9a\x18\x54\xee\x01\x35\x3c\xbe\x5e\xfe\x76\xbc\x2f\xef\xb6\x3c\x7f\x53\x6d\xf6\x65\x27\x06\xb7\xc5\x36\x67\x56\x7f\x0b\xe4\xad\x13\x23\x63\xe3\x2f\x0e\xda\xcc\x83\x3a\x2c\x88\x19\x5b\x2e\x16\x14\xec\x00\x29\x28\x09\xfc\x95\xa0\x2a\x27\xc3\x95\x72\xda\xe9\x39\x48\x9a\x40\x09\x43\x14\x2a\xd0\xd2\xe8\x72\x93\x1f\xff\x05\xfe\x07\xf2\x05\x23\xea\x58\xc4\x0d\x1e\x44\xb0\x4c\x6d\xc7\x59\x28\xfd\x54\x6f\x1b\x82\x9f\xca\x7b\x9b\x35\xfc\x37\x43\xb9\x6f\x46\x3f\x36\xfc\xeb\x3e\xa6\x8b\x2c\x98\xe0\xcc\x1d\x10\x8d\x36\x33\x8c\x71\x3d\xc3\xcb\xea\x78\xc0\x1f\x33\xc9\x14\x98\xcb\x17\x76\xe9\xe5\x76\x69\x2f\x78\xbd\x2b\xe8\xc3\x3d\x74\x48\xe9\x83\x46\x3a\xb7\x6b\xf9\xab\xec\x6d\x1c\xe5\xed\x27\x6a\xb6\x2e\xbc\x5e\xbb\xdd\x7d\xdd\x46\x98\xe3\xd5\xd5\x98\x59\x4b\x68\xee\x95\xb0\x06\x89\xfa\x59\x4d\x0a\xf5\x72\x15\x6f\x28\x55\x0a\xc2\xf0\x67\xb8\x2e\x98\x88\xbe\x8c\x08\xe3\xbf\xee\xa1\x25\xc8\x70\xd8\xb0\x5b\x7e\xf1\xa5\xac\x6d\xef\xdd\x34\x5b\x6a\xb6\x2c\xeb\x1f\x97\x9b\xa7\x1c\x64\x87\xa5\x86\xa7\x8b\x6f\x80\xea\x18\x7f\xa4\x33\x20\x22\xe4\xda\xcf\x97\x2b\xbd\x23\x07\x4a\x41\xd1\x06\x3a\x9f\x3d\xd3\x67\x13\xa5\xd5\x40\xb2\x28\x92\xc1\x7f\x45\x59\x2a\xdd\x53\x0f\x69\x97\x4c\x97\x32\xf4\x0d\xc4\x37\x50\x57\xcd\x41\xa8\xdc\x52\x24\x80\x0c\x64\x56\x1f\xd5\x1a\xa3\x8f\xaa\x40\xc3\xf4\xe5\x98\x15\xe1\xe4\x63\x4e\x10\xb4\xe2\x33\x57\x31\x22\x8c\x26\xa9\x28\x07\x6c\x54\xb9\x76\x52\x59\x11\xe7\x71\x10\xa3\xe4\xc4\xa7\x3d\x49\x23\x4e\x10\x53\xb9\x27\xe1\x15\x1d\xcf\xe1\x68\x2f\x62\x1c\x97\xf6\x90\xed\x41\x84\x37\x13\x62\x21\x6f\x01\xc0\xa0\x76\x6d\xaf\x15\xcb\x46\xfd\x81\x25\xa3\x78\xe8\x71\x74\x2c\xa2\x6f\x9f\xec\x95\xd6\x59\xc4\x1d\x62\x67\xf9\x20\x31\x87\x83\xf3\x15\x5f\xb4\x90\xb3\xeb\xdb\xf2\x06\x0e\xb9\xd7\xe0\x89\xf0\x0d\x5c\x0f\xff\xe5\x3c\xef\xc8\x15\xe0\xfc\x2b\x1f\xe7\xb1\xa4\x5d\x7c\xd1\xd9\xbf\xa4\xf6\xc5\x14\x4c\x69\x19\xf9\xc1\x02\x01\xe3\x94\xc6\x97\x99\x31\x7f\x37\xde\x5e\x04\xd4\x8a\x0c\xda\xd1\x2e\xbb\x1b\x67\x74\x30\x97\x83\xb3\x15\x00\xf1\x80\x3c\x65\x91\xad\xe0\xb8\x1f\x0c\xd9\x2e\x65\x63\x02\x7a\x27\xac\xa4\x60\x3d\x67\x77\x18\x44\xe8\xd9\xe4\xde\xd9\x6e\x7b\x36\xd6\xdf\x13\x0a\xfa\x73\x7d\x33\xdf\x54\x8f\xa7\xf5\x15\x94\x1c\x3f\x3c\xfe\xdf\x31\x3c\xce\x30\xba\x4c\xba\x40\xe3\x06\x74\x1e\x15\x41\x26\x4f\xb8\x40\xd2\x4b\xfd\x18\x2b\x3d\xb2\xb1\x5d\x63\x89\x6b\x10\xc5\xac\xc5\xf2\xfd\xb3\xc7\xf7\x5f\x91\x14\xbe\x58\x62\x04\xba\xfc\xae\xbc\x3e\x87\x13\x1d\x08\x64\x6c\xd9\x91\x27\x17\xf3\xba\x9a\xe5\x8f\x1f\xfc\xb7\x47\x8f\xbf\xf9\xf3\x93\xe3\x87\x4f\xfe\xfc\xf8\x78\x30\xb8\xb8\x3c\xfe\xcb\x9f\xbf\xf9\xb7\xe3\x2f\xbf\xfc\xf2\xab\xaf\xbe\xf8\xea\xcb\x13\xf8\x5f\x62\xf0\xb6\xa2\x44\xc1\x8e\x2e\x4d\xa9\xc1\xc0\x62\x2c\x7e\xb7\xd8\x0f\xf2\x65\x23\xcd\xa2\xb5\x06\x66\xc5\xaf\xe5\x4f\xf2\xb6\x54\xd3\xe9\x70\x34\x8c\x8e\x38\x7b\x61\xaf\x62\x2a\xcd\x4e\xe5\xfc\xfe\x40\xd3\x6d\xc0\x04\x2e\x40\x15\x48\x2e\x6b\x79\x30\x24\x2e\xa3\xb7\xdd\xcd\x01\x85\xfe\x07\xda\xc6\x2f\x51\x44\x3f\x2a\xb4\x33\x2f\xff\x28\x20\x86\x5c\x5b\x5f\xfe\x71\x60\xb8\x21\x34\x3c\x5c\x6d\xda\x66\x49\x03\x89\xb8\x30\xa3\x83\x4e\x23\xf2\xe8\x5f\xe2\x24\xe7\x39\x47\xbe\x75\x44\x50\x45\xe4\x93\x21\xb3\xc7\x99\xe3\x03\x1e\x1a\x6a\x14\xdd\xe1\xa2\xdb\x2d\xff\x15\x47\x5a\xaa\xf8\xaa\xa9\x84\xab\x40\x4c\xa2\x47\xec\xb1\xac\xa7\x2f\xb4\xfe\xfa\xe9\x5d\x38\x09\x9e\x09\x23\xdd\xea\x61\xf0\x3a\x79\x39\x13\x49\x54\xe2\x93\xd8\xa7\x5e\x68\x39\xa3\x12\x30\xf2\x42\x24\x94\x79\xb1\x17\x22\x9f\x2f\x1b\xb5\xb2\x35\x84\x6b\xa7\x25\xb8\x25\xbb\xb1\xe1\xf0\x6a\xf4\xdf\x22\x4b\x09\x0e\x12\x48\x95\xdd\x61\x66\x58\x7d\xcd\x31\xf8\xf4\x05\xf2\x5e\x2f\xa3\xc8\xcd\x1c\xd9\xaf\x9a\xf4\x17\xfa\xe8\xb8\x86\x76\x83\x0d\x15\x3a\x3a\xc6\xb0\x9a\x7b\x82\x3c\x7b\xef\x9e\x7f\x25\xcf\xce\x20\x28\x11\x39\xca\x23\xa5\xd6\xe7\x79\x1f\xfc\x44\x6d\x7a\xb6\xc2\x79\x11\x53\xe5\xd8\xa5\x16\x70\xca\x51\x4a\xe8\x92\xad\x11\x5d\x93\xb1\x48\x57\x0f\xe4\xdd\x69\x4d\xb6\xc4\x80\x36\x0c\x3a\x11\xda\xa5\xaa\x3f\x89\x7b\xed\x1f\xc1\x1c\x85\x03\x43\x5a\x88\x27\x8c\xcd\x4b\xc9\xd3\x2f\x4f\x3f\x1a\x38\x6b\xf0\xea\x7c\xe6\x68\x1f\x60\x4d\x43\xf3\xba\x84\x73\xc5\x71\xbd\xb9\xb8\x11\x86\x30\x08\x48\x3b\x0c\x5e\x24\x51\x0f\x4d\xde\x32\xd0\x35\xf0\x2b\xc5\x3b\x0f\x99\x07\xe9\xe2\x9c\x1f\x1f\x15\xf1\x9f\xda\x1c\x51\x04\xcb\x22\x95\xe0\x68\x18\xaf\xb8\x97\x50\xa9\x84\xe3\xa7\xb7\x04\x70\xe5\x88\x90\xe2\x04\xd9\xf9\x95\x43\x3d\x47\xe1\x12\xa8\xcb\xd1\xa1\x16\x9a\x91\x76\xd8\x2b\x2f\xa7\x4a\x6c\xe4\xc1\x29\x46\xba\x18\xbb\xe7\x9e\xcc\xac\xdc\x37\xda\xec\x6e\xa3\xef\x56\xf1\x80\x86\x76\xe0\x0a\xc9\xaf\x2e\x42\x33\x8f\x85\xec\xa2\xc4\x88\x8a\xbb\x57\x43\xba\x85\x7c\xeb\xeb\x63\x87\x03\x5a\x73\x6b\x0b\xb2\xbf\x6f\xd2\x52\x19\xf7\xbb\x1b\xa2\x36\x4d\xcd\x31\x80\x90\x2f\xe9\xc5\xcb\xb8\x4f\x0f\xba\xe3\x63\x0d\x5b\x7d\x71\xb3\x66\xeb\x6d\x0c\x16\x7d\xf9\x4e\xcd\xcf\xba\x5d\xae\xab\xef\x68\xf9\x1d\xd8\x38\xe7\xdc\x02\xda\xec\x19\xdf\x88\x9a\x00\xe7\xe6\x01\xe9\x1d\xaf\xdb\x6e\x57\x82\xaa\xb7\x84\xb0\x36\x1e\x16\x5a\xa7\x43\x3c\xbf\x0b\x7e\x98\xcc\x15\x4e\x22\x03\x8e\x97\xe1\xf2\x6c\x8b\x0e\x8c\x8b\xf0\x8e\x4d\xad\x3e\x60\xfc\x12\x5f\xb0\x6b\x5b\x71\x58\x5e\x0f\xa8\xc4\x2e\xa8\x46\x29\x59\x72\x59\xfb\xde\x13\x51\x62\xb4\xe4\x6a\x8a\x68\x58\x3c\xd8\xf7\x7a\x24\xcf\x1a\x6f\x2c\x1b\x7d\xb6\xaf\xc0\x29\x35\x5c\x15\xe7\x3c\x66\x2e\x5f\x73\xd6\x28\x20\x68\xdd\xd6\x08\xed\x39\x35\xe8\xf3\x46\xcc\xce\x5b\x19\xcd\x9e\x05\x42\xdc\xc6\x8d\x66\xd8\x42\xf5\xb6\x48\x66\x02\x22\xf4\xc9\x0e\x36\x77\xaa\x41\x2f\xbd\x45\x6b\xc1\xd5\xdf\x65\x2d\x8b\x9a\x09\xe8\x23\x78\xe4\xbe\xf6\xe6\xbe\xf7\x1b\x3e\x21\x61\xc2\x26\x2b\xb7\x4f\x5e\xbe\x1b\xd9\x5f\xf8\xe0\x46\x89\xcf\x15\x42\xfe\x64\xbb\xc5\x3f\xb9\xad\xed\xb1\x16\x0e\x26\x26\x5d\x23\xa2\x3c\xdb\x23\xbb\x94\xd8\x04\x8c\x6d\xc3\x0b\xcf\x4c\x9c\x8a\x60\x7a\x36\x74\x0b\xd2\xe7\x1b\xa4\x58\xd4\xd2\xd2\x00\x92\xe0\xce\x50\xce\x7f\x7c\x08\x57\x0e\x2e\x81\x96\xa2\x3f\x35\x6c\x6d\x08\x70\x1b\x3c\x81\x1f\xc4\xbb\xc1\x57\x86\xf8\xb2\xb6\xc5\x97\xa1\x14\x4d\x16\x5b\x0c\x54\x41\x78\x2a\xb9\xef\xf5\xc7\x71\x7c\x4c\x0d\xca\x85\x71\xe3\x89\x15\xb2\x21\x1d\xeb\x2f\x00\x91\x43\x21\xed\x59\x27\x18\x39\xc4\x2d\x13\x27\xaa\xc5\x53\x33\xac\x8e\x8f\x87\x19\x7a\x47\x35\xd8\x88\xc4\x27\x32\x0d\x86\x44\xef\xcd\x75\xbb\xe8\x8c\x4e\x4c\x86\x14\x97\x00\x79\xaf\x67\x6c\x1a\x31\x16\x35\xad\xf2\x1c\xe4\x28\x21\x95\x0b\xee\xd0\x86\x7f\x97\x89\xe8\x3c\xa3\x7e\x0c\xc6\x7c\xa0\x1f\x6b\x92\x6e\x8b\xfb\xe3\xb3\x0d\x45\x61\xc5\x90\xaa\xdc\x60\x71\xff\x6c\x8d\x5f\xa4\x3f\x27\x05\x0d\xc5\x9d\xa5\xc3\xe7\x96\x0f\xa6\x5b\xbc\x5f\x9f\xae\xcb\xe9\x16\x24\xc6\xcd\x72\xb1\x65\xc1\x0b\xce\x98\x15\x36\x5c\x5d\xbc\x71\xf5\xa6\x5b\x2c\xc8\x59\x12\x71\x82\x33\xe4\xc2\x60\xab\xf7\x7c\xdb\xe9\xcd\x66\x49\x9d\xd2\x2f\x7a\x82\x9d\xdc\xe8\x29\x22\xe3\x7a\x39\xaf\x79\x12\xb7\x7a\xc5\xb4\xe5\x60\xde\xdb\xf9\x72\xb9\xda\xd2\xd1\x16\xce\x8a\xdb\xe5\x0a\x92\xd0\xa3\x61\xb9\x98\xbf\x87\x1f\xbf\xde\xa0\x05\xe5\xb6\xbe\x58\xe2\x53\x3e\x04\xc6\x4d\x5d\x3e\xe6\xd8\xcf\xed\x90\x50\xa9\xe8\xda\xb7\x08\xed\x04\xfa\x51\xb6\x14\x7f\x4a\x27\xf4\xa8\x28\x61\x6e\xd8\xe4\xdd\x28\x4f\x7b\x31\xf6\x50\x08\x0e\x65\xcc\x16\x69\x54\x93\xa0\xaa\x71\x35\x5a\x04\x40\x2b\xf8\x3d\x6c\xf4\xf2\x09\x92\x1f\x89\xb2\xd4\x31\x9f\x38\x79\x0d\xaf\x7e\x27\xd4\x14\xd0\xfc\xe3\xa0\xfe\xb9\xd1\x8b\xb3\xc9\xf3\x0e\x18\xfc\x52\xcd\x3b\x3e\x37\xc0\x89\x0e\xdb\x6e\x1d\x15\xf9\xd2\x50\x10\x42\x8a\x65\xe5\x85\xe9\x66\xb1\xc4\xe5\xf9\x8f\xf5\xe1\x91\x76\x36\x7b\x84\x2b\x25\x8f\xcf\x01\x74\x5f\x82\x39\x65\xcd\x2e\xc9\x20\x15\x61\x80\xee\xdf\x7e\x33\xaf\xd5\x09\x35\xe0\x4e\x00\xeb\x05\x70\xf6\xa6\x77\x89\x78\x77\xd2\x9f\xf6\xd7\xbe\x83\x07\xef\xda\xa6\xed\xb5\x7d\x30\x80\xc2\x37\xf6\x15\xea\xd4\xf3\x76\xe3\x99\x90\x18\x94\x6a\xb9\x90\x61\x7c\x7f\xe8\x53\x80\xe3\x47\x32\x79\x54\x05\x57\x06\x96\x96\x1c\x7a\x9e\x3c\xf0\xaa\x76\x47\x5c\xe0\x71\x28\x26\xb6\xbd\x2f\x42\xdf\x16\x82\x11\x3a\x05\xf5\xc2\x34\x7c\x53\xd3\x73\x82\xa7\x74\x83\x69\x39\xfe\xc3\x67\x57\xf4\xaa\x26\x74\x17\x02\x2f\x3b\x3e\x67\xd0\xab\x3b\x96\xe2\xcb\x92\x58\x86\x1a\xfc\x5a\x1e\xd3\xec\x15\x2e\x6d\x18\x76\x1c\x84\x54\x81\xa2\xad\x2e\x0a\xff\x62\x7a\xc0\x0b\x8c\x78\x25\xb8\x18\x27\xff\x33\xa9\xc4\x1b\xce\xff\xff\x08\xe5\x43\x74\xa2\x2f\xd4\xee\x21\x96\x53\xa8\x86\xb4\x84\x4f\x91\xda\x0e\x82\x22\xa6\x8d\x7e\x08\x13\xa3\x98\x8a\x00\x9a\x90\x90\xf8\xbd\x90\x36\x42\x62\x7b\xdd\xbf\x4e\xe7\x9e\xe7\x8f\x4f\x01\xa6\xaa\xd1\x13\xaf\x68\x46\x6f\xd2\x8a\xd1\xd1\xa6\x41\x20\x6d\x1a\x91\x2a\xa2\x0c\x0f\xbe\x06\x65\x54\x11\x65\x38\x88\xdd\xaf\x5d\xd6\x2e\xed\x07\x21\xb3\xf9\x79\x07\xfb\xde\xa5\x6d\x0f\x17\x0b\xb9\x57\xf9\x20\x19\xcf\x4c\x19\xc3\x12\xd8\xe2\x35\xe3\xfc\x00\xe1\xb9\xd9\x71\x95\x28\x62\xc2\x30\xe3\x46\x19\xa5\x23\x76\x2e\xe8\xa8\xd3\x14\x8f\xde\x2d\x00\x82\x69\x4c\x85\x46\x89\x32\xd8\x24\x4f\xbc\x55\x94\x4c\xbc\xf2\xe4\xb2\xca\x2a\x8e\x96\xeb\x16\x58\x2d\x6e\x06\x19\x13\x1a\x40\xd8\x36\x61\x8f\x2c\x2f\x9d\x2e\x28\x79\xf9\xd2\xe6\xbf\x7c\x99\xc4\xab\x34\xfa\x2e\xc2\xcf\xed\xd6\x85\x41\xc1\x80\x23\x1f\x6c\x3e\xa3\x45\x8d\xfb\xa0\x22\xa4\x19\x43\xdd\x7b\x2c\xd6\x76\x8b\x0b\x45\xf3\x69\xad\x10\x03\xf4\xd9\xdf\x70\xee\x98\x01\xae\x7d\xe1\x05\xc1\x3b\x54\xf8\x7e\xb2\x24\x7f\x78\xc5\xdb\x75\xec\x90\x01\x6b\x58\x35\xc8\x27\x43\x1b\x0d\xce\x04\x01\x8d\x3c\xee\x8d\x11\x4c\xf8\x89\x32\xb7\x66\x02\xa5\xed\xf0\x8e\x83\xee\x04\xac\xda\x46\x0c\xb1\x75\x5a\xdc\x36\xc4\x03\xbe\x7d\xe5\x49\xec\x8a\x26\xcd\x0f\x15\x31\x3e\x56\xa0\x64\xe1\x5a\x1e\x35\x57\x27\xd1\x2f\x24\xa3\x17\xa3\xbc\xd2\x47\x7e\xcc\xf0\x1b\x4b\x17\x30\xef\x8d\x50\x24\x92\xd7\x2b\x1c\x51\x78\x4f\xeb\x65\xd2\xad\xd5\x9a\xad\xf0\x73\x9f\x66\xd4\xf6\x06\x8c\x10\x9a\xa3\x1f\x3d\x60\x80\x59\xa4\xc1\x82\x0c\x7e\x3a\x8f\xe3\x48\xe3\x93\xea\x56\xef\x12\xe6\xed\xd1\xe7\xdb\xd7\xf6\xe0\x60\x6b\x9f\xda\x53\x3d\x11\xa4\xa5\x6a\x54\x6e\xd4\xd4\xab\x13\x84\xf9\xe3\xd5\xc6\x2b\x02\x47\x29\x3b\x9c\x1f\x06\xa6\x1d\x60\xe2\xf9\x7b\x00\x3e\x74\x01\x61\x01\x7e\x15\x00\x8c\x0e\x36\xf2\xb2\x1c\x64\x88\xba\x5b\x20\x0e\x40\x46\xbf\x9b\x9c\x4a\x53\x2f\x3c\xb9\x0d\x41\x01\xf2\xb1\xa0\x5b\x23\x3c\x16\xbc\x93\x93\xf8\x72\x58\x02\x66\x86\x42\x83\xb4\xe8\x72\x75\x2c\xb9\x38\x44\xe7\x77\x41\x30\x70\xbe\xd0\x11\xa2\x8d\x2f\x0f\x18\x28\x5d\x6c\xf0\x41\x2c\x87\x1f\x11\xbd\xac\x44\x4b\xce\x85\x72\x46\x23\xf4\xba\xd3\xc0\xe6\xad\x3d\x29\x77\xb1\x56\xa2\xb8\x2c\xd5\x55\x9b\xda\xd0\x40\xc8\xec\xed\x49\x49\x7a\x94\x63\xf7\x69\x34\x67\x13\x6b\xe3\x63\x79\x50\xce\xce\xe1\x76\x7b\xc2\xaf\x59\xa9\x03\x01\x14\x18\x91\x76\x7e\x3c\x31\xd7\xd3\x77\xf4\xcd\x0f\x5d\x0d\xec\xa3\xb9\x36\xd0\x8f\xd4\x1d\x41\xc1\xdc\x16\x24\xc3\x99\x77\x8e\xbf\x71\x2d\x1b\x68\x18\x03\x71\xa4\xe2\xb7\x6d\x01\xed\x76\xab\x4e\xc1\x6e\xa3\x40\x99\xd1\x09\xd2\xb3\xb2\x14\x1f\x5f\x6b\x67\x99\xb3\x8b\x92\x34\x1e\xdd\x60\x69\x21\xa0\xa3\xed\x56\x4a\xb8\xf0\xfe\xb6\x0d\xef\x12\x8d\xe9\x36\x6d\x94\x35\x68\x74\x8d\xf6\x63\x2b\xba\x3c\xb2\x51\xb7\x84\x5d\x71\xf9\x8c\xd9\x0e\xa2\x3b\xf3\xaf\x49\x86\x8c\x59\xd6\xab\xcb\x45\x86\x9f\x8f\xe1\xd0\xe3\x79\xf7\xef\xd0\x64\x62\x1a\x4a\xfb\x30\xfc\x8d\x1f\x1f\x84\x1f\x24\xb0\x86\xe2\xcd\xe3\xa0\x0b\xd4\xef\x02\x48\xaa\xcf\x6d\x20\x09\xf1\x88\xc4\xe7\x03\x36\x94\x9d\x7e\x58\x49\x3d\x6d\x12\x5c\x71\x3c\xd0\x62\xb0\xc0\x4c\x78\xd0\x77\x57\x5e\xde\x10\x79\xdf\x01\xb9\x05\xa3\x1d\xe2\x46\xb4\x40\x76\x1f\x4a\xc4\xca\x9c\xba\x5d\xb4\x55\x58\xf0\x7e\xf0\x6f\xee\xe7\x03\xfd\x69\x8f\x00\xcc\x10\x1a\x57\x9b\x6d\xf2\xc8\xc8\x3b\x51\x37\x40\xcc\x53\x86\x8c\x4c\x87\x16\x1c\x08\xd6\x85\xab\xc0\xc0\x87\xcb\x0b\xb9\x8d\xe4\x82\xc0\xd4\xe8\x98\xde\x8c\xdc\x10\xee\x01\x88\x19\xef\xfd\x54\x8c\x26\x23\x52\x11\x9b\x95\xd3\xc1\x7f\xa4\x4a\xab\x5c\xd5\x58\x99\x8b\xe7\xe6\x06\x20\xdc\x54\x01\x88\x98\xac\x2c\x93\x36\x5e\x4b\x9d\x64\xcc\x5c\xf7\xa1\x6c\x8f\xc5\x8c\xe5\xc5\xc1\xba\xe3\x06\x8d\x63\xb6\xb6\x0a\xf1\x5c\x5d\xb2\xbc\x45\xc6\x60\xd7\xfb\xf6\x86\x5a\xe1\x7d\x6b\x9f\xde\x6d\xdb\x1e\x84\x1a\x23\x70\x8c\x6e\xc3\x99\x9d\xd2\xa6\xa2\xc7\x61\x43\x6f\x19\xf7\x68\x91\xe2\x05\x4a\x3d\x20\xe5\x58\xf9\x1d\x27\xd6\x93\xce\xe5\xdc\xda\x2e\xa4\x53\x18\x38\x2d\xdf\xed\x36\x4e\x82\x7c\x54\xb3\x57\xd0\xbe\xfc\xae\x7d\x1e\xd2\x07\xb5\x51\x54\x27\x52\xdf\x75\xbb\x9e\x52\xd0\x2f\xe8\x1b\x88\xb8\xdb\xec\x44\x9e\x8e\x3b\xe6\x28\x25\x19\xbf\x6d\x32\x56\x68\x26\x20\x5e\xe7\xcd\x14\x8f\xe6\xbd\x15\x86\x53\x42\xc3\x75\x88\xe6\x69\x8b\x40\x1c\x61\xe9\x5c\x1b\xcc\x98\xab\xc8\x3e\x4d\xee\x11\x77\x7b\x79\xa8\x17\x5c\xa6\x69\xb8\x0e\x24\xc7\x56\xed\xfa\x2a\x67\x73\x37\x60\x89\x44\x8c\xef\xa3\xad\xdf\xdf\xef\x8f\x9a\xb4\x27\x86\xf3\x22\xff\xbe\xc5\x50\x6f\xa9\xab\x47\x62\x17\x67\xee\x90\x51\xca\xf4\xc1\x80\xa6\xe7\xbc\x93\x26\xf0\x8b\x58\x6a\x62\x54\xa7\x9b\x53\xe8\xf1\x9f\xe0\x17\x7a\xa3\x2f\xd7\x20\xce\xa2\xc3\xc7\xb7\xf0\xcb\xb0\xcd\x46\x92\xfb\x0e\x54\xb0\x15\x8b\x97\x63\x02\x3f\xf9\x52\x31\x31\x17\xe5\x7c\x8e\x9e\x3a\x68\xb8\x96\xe0\xc7\x33\xfe\xe0\x1c\x35\x69\xa3\x9c\x9f\xe5\x29\x43\xb3\x5e\xde\x42\x95\x05\x00\xb0\xbc\x85\xe2\x0b\x8c\x1f\x34\xe7\x14\xf8\xc1\x29\x40\x46\x18\xac\x2e\x81\xbf\x3f\x4c\x57\x89\xa1\xb0\x41\x62\x0c\x9a\xd0\xc7\x37\xf4\x91\x18\x31\x6d\x2e\x67\x15\x19\xec\xe6\x6a\xbb\xfd\x44\x12\x92\x58\xd7\xda\xba\x71\xb8\x43\x8b\x6c\x1e\xff\x8a\x8d\xe3\x77\x72\xfb\x03\x8b\x30\xe0\xfa\x58\xc0\xe3\xfa\x2e\xce\x67\x83\x99\xff\x51\x6e\x48\x8b\x90\x55\xbe\x2c\x73\xb6\x4b\xe0\x1f\xb7\x39\xb8\xc6\x78\x66\x74\xf1\x09\x85\xee\x15\x93\xad\x74\x8c\x82\x54\x73\xbb\x20\x63\xff\x44\x29\x3e\xb1\xf2\x4c\x50\xad\xdb\x0d\x3e\x3d\x81\x9a\xec\x62\x9e\x2e\x36\x69\x58\x80\xd1\x32\x38\xc9\x72\x77\x91\xc3\x8c\x2d\x38\xa1\x80\x5c\xe8\x2e\x6c\xda\xf2\xc5\x32\x0a\x5d\xbf\x46\x27\x5e\x2c\x3c\x3a\x38\xe8\xc6\x5c\x34\x87\x2e\xa6\x63\xfa\x36\x6d\xd1\x2a\x5e\xf0\xdb\x9e\x2d\xba\x2c\x2c\xd6\x66\x93\xc5\x95\x30\xa4\x55\xd5\xa6\x02\xa3\x77\x59\xbd\x9b\x98\x6e\xb7\xc1\x44\x39\x14\x2a\xee\x66\x7f\x8c\xe3\xa3\x5f\x1b\xb6\xd2\x3e\x9d\x54\xc6\x7a\x7f\x10\xb8\xdd\x2e\xff\xed\xbf\xb5\x41\x6d\x47\x4d\xf1\xc8\x8f\x36\xb8\x87\x9f\x87\x46\x79\x9e\x42\xe7\xd0\x86\x9e\x1f\xc2\x0a\x2c\xef\x03\xc8\x08\x19\x7a\x73\xe7\x3c\xb0\xea\xb9\xeb\x4f\xda\x43\x79\xb1\xa2\xd3\x1c\xfc\xd8\xed\xcc\x61\xb8\xd3\x58\x8e\x94\x7d\xe5\x20\x3d\xb6\x0a\x6d\xcd\x69\x3b\xb8\x0b\x86\x2f\xc8\x12\x6b\x59\xe3\x5b\x39\x78\xf0\x76\x6b\x13\xbf\x98\xef\x7c\xea\xb4\x1e\xee\x3c\x6d\xf4\xae\x0f\x6a\xb3\xd8\x66\x05\x65\x94\x2b\xf9\x57\x2b\xc3\xc4\x35\xdc\xc4\xa9\x56\x89\xf5\x1e\x7d\xbe\xec\xfd\x97\x20\x97\x90\x95\x26\x15\xbd\x91\xb9\x20\xd5\x24\x7b\xbe\xb9\xcf\x8b\x25\x6c\x94\xb5\x26\x8c\x92\x84\xc3\xc5\x0a\x42\x03\x44\x67\xbf\x07\xd3\x9f\x06\xad\xbc\x10\x5c\xc7\xe5\x6c\x0b\xc1\x2b\x82\xe2\x6f\xd0\x38\x12\x58\x30\x79\xbe\x50\xdf\xa6\xaa\x1e\x3b\x70\x5d\xd1\x7b\xb8\xdf\x28\x1e\xe6\xce\x34\x16\x41\x24\x5a\xf0\x8c\x59\x4a\xc0\x67\x9c\x0e\xa3\xa6\x49\x34\x89\x15\x0f\x47\x20\xc9\x06\x24\x14\x04\x32\x19\xcb\xab\xd9\x46\x5f\xcb\x9e\x34\xc3\xfd\xb5\x9f\xfd\xa2\xf7\x89\xda\x0b\x99\xfd\x42\xae\x98\x38\x39\x38\xf7\x9d\x84\x12\xb4\x2e\x70\xe7\x20\xb1\xea\xdb\x51\x30\xd1\x48\x4a\x0e\xdd\xa2\x1d\xbb\x91\xa1\x92\x3f\xb4\x49\xea\xf5\x05\xfc\xfb\x3f\x79\xdc\x7b\x64\x8b\xfd\x07\x50\xf3\x20\x1b\x06\x1a\xd9\xc6\x41\x94\xc6\x6c\xf6\x0e\x69\xff\x08\x22\xf9\xad\x85\x21\x84\x01\xf4\x9b\xb0\x7d\x49\x8a\xc7\x06\xbe\xc9\x93\xa2\x8d\xab\x53\xc6\x07\xfb\x09\x5c\x55\xb6\xdb\x0f\x73\x82\x03\xd5\x0b\xab\xb7\x6e\xc4\x88\xe8\x7b\x9e\xea\x0e\x58\x8b\x94\x86\x5e\x29\x9c\xd9\x66\xb9\xbd\x93\xcb\xea\x37\x9e\x5f\xa7\x8a\xf3\x1e\x4a\x61\xff\x5a\xfe\x11\xa9\x3c\x25\xd1\x55\xeb\x76\x1b\x49\x61\x15\x56\x58\xb4\x4d\x8b\xf8\xd5\x87\x63\x85\xbd\x5e\x33\x8a\x04\x7e\x2c\xe9\x8c\xd4\xa8\x2b\xbe\xfe\xee\x1d\x0a\xa6\x33\x3e\x72\x9a\x44\x43\x52\xf8\xd4\x66\xe9\x2c\xb8\x05\x38\x48\x66\xf2\x6e\x4a\x93\xda\x54\x8f\x3e\x42\x37\xdf\xdc\x9d\x3c\x03\xbd\xf7\xef\x07\xa9\x75\x86\xc3\x32\x1f\x64\x59\x47\xcd\x0b\x97\x1b\x15\xbd\xd4\x6f\xbf\x5d\x43\xc9\xaa\x4e\x4f\x43\x89\x0a\x4a\x9f\xa1\xb1\x6f\x10\x46\x47\x78\x02\x45\xeb\x03\x86\x5c\x6c\x18\xf5\xa6\x7c\xcf\x8e\xd7\xf7\x5f\xc0\x4f\x74\x18\x5a\x82\x5c\xa8\x49\x50\x97\xbe\xf5\x85\x5c\xd8\xe8\x6e\xb2\x2d\x9d\x1e\xee\x8b\xb9\xd8\x0f\xcb\xf5\xea\x8a\x8b\x8a\xfb\x35\x1b\x74\xd1\xbf\xcb\x9b\xcd\xf9\xfc\x66\x4d\x9e\x48\x48\x36\xc8\x09\x28\x58\x06\x56\x18\xbf\xe8\x4f\xee\xd1\x1b\xe3\xfd\xb4\xdf\xcb\xf0\x91\x72\xeb\x99\x44\xfd\xdf\xf1\x73\x24\xe8\x99\x14\x44\xdd\x2c\xd5\x94\xb6\x36\x57\xd3\xc5\x6c\x5e\xae\xd9\xb3\xc5\x5d\x4f\xaa\x69\x21\xb5\x53\x9b\x8d\x94\xfb\xe9\xfc\xf5\xd3\x85\x21\x19\x62\x3a\xe7\xcc\xef\x28\xc3\xe5\x6b\x8b\x62\x8a\x6a\x01\xae\xcd\x72\x5d\xbd\xa2\x58\x1e\xd8\xfd\x63\xcf\x31\xc3\x99\xfa\xd9\xcb\xa4\xc7\x36\x04\xab\x34\xd7\x97\xbf\xb0\x76\x3d\x50\x0a\x05\x5f\xfe\x16\x5e\x9e\xd6\xb0\xa3\x0a\x32\xdd\x2b\xcf\xda\x01\x87\x33\x4f\xfd\x4f\x05\x90\x83\x9b\x67\x86\x5d\xe0\xf9\x06\x05\x21\x64\x34\xd7\xe8\xc2\xd1\x9e\x43\x2f\x34\xa6\x1e\x9e\x5c\x01\xee\xc7\x56\x6d\xcf\xf6\x42\x7d\x3b\x0b\x34\x3e\xb7\x31\x68\xed\xd7\xe4\xee\x84\x21\x4f\xa9\xf3\x6b\x74\xb0\x28\x60\xbd\xd3\x0d\x8e\xb7\xab\x05\x25\xd1\x6f\x10\xf5\x8d\x12\xf2\xce\x03\xad\x4f\x44\xe3\x85\x87\x33\x71\x26\x3f\x06\xc1\x74\xc5\x1e\x0a\xf5\x61\xfb\x96\x04\xd8\xc5\x86\xce\xa1\xf6\x86\x7b\xa3\x56\xa0\x21\xb5\xf3\x63\x58\x54\x72\xbc\x99\x90\x89\x02\x1b\x90\x2a\x4d\x61\xe4\x7c\x7c\xb9\xda\x91\x5b\x81\xa1\xf6\xc7\x0f\x26\x0c\x82\x84\x52\xee\xe3\x4f\x60\xb1\x68\xa5\xcf\x44\x5c\x04\xc3\x97\xc4\xb1\x5c\x54\xde\xf1\x13\x75\x85\xbd\xb7\x1f\x49\x01\x0a\xc2\xf1\x0a\xa4\x55\xec\x3b\xd7\x44\xd4\x46\x90\x4a\x68\xbb\x25\xda\xff\xc8\x2e\x2c\x59\x46\xac\x91\x55\x94\x6c\xee\x2b\xc3\xcc\xed\x1a\x22\x07\x2a\x5a\xb4\x42\xb0\xb9\x92\x3b\xbd\x5b\xe0\x53\xb1\x7b\x01\xd9\xbd\x7c\x8e\xe6\xd2\x12\xf3\xc9\xa6\x3a\xa2\xc1\xb7\xd0\x69\xd2\xfa\x7e\x41\x3e\xe4\xba\x85\x63\x71\x9d\x3b\xac\xf7\x5f\x2f\xab\x05\xe1\x79\xe7\xb3\x0c\x58\x04\xca\x16\x38\x36\x44\xcd\xe3\xcf\xdc\x82\x0b\x33\x30\x06\xab\x66\x58\x64\x3f\x22\x3b\xe7\x13\xc5\x2b\xca\xa9\x37\xab\x6e\x37\xf8\xf4\xdc\x23\x08\x3b\x1e\xff\xf1\xe8\x35\x23\x5d\x82\xde\x43\xc5\xd1\xd4\x47\xad\xa9\x6c\x78\xed\xf3\xbc\xce\x80\x0f\xea\x61\xec\x8e\x38\x25\xa5\x68\x19\x71\xe5\x2c\xb3\x24\x88\x3d\xa1\xd7\xb8\xfb\xf2\x06\x61\x91\x98\xb9\x9f\xfd\x56\x96\x15\x67\x14\xfe\x47\x96\x59\x2a\x18\x59\xbc\x4a\xe8\xde\x76\x3c\xf7\x7a\xe6\xc4\xeb\x3d\xb7\xa5\xe8\x46\xd2\x03\x2b\xa0\x6e\xde\x70\x64\x0e\x3b\x27\x64\xea\xc5\x01\x24\x9b\xc1\x8a\x5b\x76\x21\x4b\xa0\xd7\xe8\x91\x4e\x8b\x49\x0c\xf2\x5f\x7b\x3b\x0c\x6e\x4c\xb8\x10\x08\x4e\xb3\xd1\x4d\x4a\xb7\xa5\x4f\xdf\x80\xc4\x4d\x96\xb7\xa0\xc8\x82\xa8\xb1\x2d\xa1\x36\xbe\x7d\x1b\xe0\x0d\xeb\x8f\xf3\x3f\xf4\x62\xfc\x5f\xc1\x02\xd9\x05\x01\x61\xfe\x5f\xc0\x0c\x5b\x97\x39\x8f\x02\xc6\xca\x80\x76\xbb\x18\xb9\x8f\x63\xe1\xa7\x49\xfa\x62\x7b\x76\xd6\xcf\x58\xd7\x15\xb0\x17\x48\x06\xa9\xa7\x7f\x0f\xfe\x6e\x33\xf4\x80\x4b\xf1\xd7\x67\x19\x06\xda\x52\xba\x28\x5e\x17\x96\x64\x05\xb9\xaf\x11\xb9\x8e\xdb\x6a\xf6\xf8\xf5\xc4\x74\x3c\x82\xeb\x76\x15\xa5\xc0\x25\xdc\xda\xd2\xc4\xed\xd6\x8a\x22\xfe\x0a\x0b\xca\xf2\xc2\x24\xef\x84\x0e\xba\x22\x10\xe7\x74\xd9\x76\x40\xd9\x76\xeb\xb8\xaf\xf7\x40\xbe\x2b\xea\xb2\xd3\xe4\xde\x3d\xd4\xe7\x68\xca\x76\xdb\x69\x16\xf3\x19\xaa\xae\xef\xd7\x18\x96\xbb\xad\xc5\xf6\x85\x7f\x7c\x6c\x19\x13\xaf\x58\xc7\x61\xf9\xbb\x95\x3b\x65\x43\x8b\x79\x18\x73\x84\x79\x8f\xbb\x6d\xca\xe9\x7a\xb6\xbc\x5d\xb8\x46\x35\xc5\x77\x6c\xf3\x98\x76\x24\x2f\x09\xe3\x0e\xf4\xb6\xcc\x66\x2d\x3b\x69\xd4\x51\x83\xf5\x60\xe7\xd9\xd9\x97\x85\xf0\x9b\x1e\x17\xe2\x55\x1c\xac\x01\x79\xe1\xd2\xb6\xdd\xd3\x05\xd8\x64\x59\x9d\x13\xef\xdd\x61\xef\xc5\x53\xe1\x16\x19\x3d\x8c\xcd\x70\x84\xf0\xed\x75\x63\x4b\xb8\x26\x3f\x18\x2c\x32\x9c\xc7\x3e\x31\x93\x37\x39\x2a\x8d\xf7\x7c\xdf\x09\xde\xc5\x40\x82\x9b\x87\xdd\x1b\xb1\x82\xb6\xd5\xe7\x14\xa1\xe9\xb9\x27\x9c\x10\x33\xad\x98\x85\xfe\x3c\xdd\x5c\x15\x64\x8d\xe5\x1e\x27\x15\x26\xd3\x78\xde\x95\x7b\xe7\xab\x4a\x38\x78\xb1\x98\x89\xeb\x9f\x33\x3c\x26\xb4\xaf\xaa\x2d\x62\xeb\x3b\x5e\xe7\x78\x55\xce\x8f\x4c\xa1\x55\x31\xf2\x09\x2b\x67\x3a\x08\xcd\x17\x4d\xaf\xf8\x7f\x6b\x26\x75\xbc\xd3\x16\xaf\x47\x9a\xcf\x76\x39\x99\xec\x4a\x21\xdb\x19\x3a\xf7\xc9\xbc\x59\x6e\xed\x64\x64\x54\xc2\x81\xca\x98\xf2\x98\x95\x3a\x84\xfa\x49\xc4\x79\x33\x99\x93\x22\xec\x23\x67\x7b\xfb\x48\x5e\x60\x86\x19\x47\x5c\x60\x94\x61\xec\x67\x3d\x0b\xf3\x2a\xa0\x7a\x0d\xa7\xee\x52\x44\x13\xfc\x93\x71\xb3\x40\xa4\xcf\x79\xb8\x78\x73\x10\x4d\x40\xd1\x26\xd1\xc5\x85\x80\x5e\x8b\x28\x69\xf4\x47\xf9\x37\xd9\x79\x18\x5d\x7b\xf8\x2a\x9f\xe7\x77\x22\x93\x24\xef\xe3\xa7\xfe\xa7\x9c\x3d\x6c\x74\x00\x79\x0f\x96\x83\x8c\x4d\xf2\x56\xdf\x65\x23\xb9\x1f\x7d\x16\xf0\x57\x18\x70\x5f\xcb\xc0\x18\x8d\xc0\xca\xc3\x04\x3d\x43\xa9\x2c\xca\xbc\x2b\x93\x37\xa4\xbd\xb6\x30\x70\xbe\x54\x5d\x2c\x39\x88\x9a\x8b\x47\x60\x5f\x8d\xa6\x11\xf2\x76\xed\x96\x71\xd1\xb6\x07\xcb\x8e\xdb\x20\x78\x57\x8d\xc8\x8b\x1e\xe0\x17\x6b\x7d\xa7\xe0\xca\xd0\xc0\x7f\xd8\x92\x6e\x59\x04\x4b\x80\x68\xa3\x4f\x5b\x37\x16\xc7\xbf\x45\x91\x36\x95\xfa\x6e\x99\x66\x42\x7e\xae\x05\xdc\x18\xf5\x3e\xa6\x2a\x6f\x69\xbf\x94\x0e\xff\x26\x21\xe3\x38\x88\x4f\xb6\xa3\xa0\x75\xc5\xc9\x90\xe0\xb5\xcd\x90\x41\x0b\xbe\x81\xad\x04\x8d\x1e\x50\x53\xc0\x02\xb0\xc7\x67\x9b\x25\x6e\xe8\x69\x36\xcc\x1c\x73\x2a\xaa\xd3\xc1\xc8\x61\xa1\x21\xab\x04\xa2\x4a\x91\x06\x72\x20\xb2\x4f\xcb\x90\x29\x6a\xd0\xd8\x35\x3c\x89\x84\x46\x2a\xcc\xcd\x24\xba\xf7\xea\x7e\xab\x6f\xb1\xdd\xb0\xd2\x45\xb3\x0b\x66\x09\xdd\x2e\x64\x8c\xf9\xf7\xc4\xd6\xf4\xfc\xc3\x38\x56\x08\xa1\x3f\x6c\x71\x5f\xfb\xf2\xf4\x0b\x03\xbb\x5a\xd3\x5f\xb9\xb6\x4c\x59\xb0\x75\xf8\xa1\xe1\x77\xd2\x90\xd0\x15\xbb\x52\xe9\x67\x6e\x02\x71\xeb\x48\xfe\xa5\x4c\xa4\xdb\xd6\x35\xc5\x5b\x05\x21\x69\x78\x6b\x02\x6f\xae\x28\xba\xa1\x75\x48\x6e\xbf\xcb\x9b\x22\xe6\x3b\x4d\x64\xd0\xd2\x80\x1c\x46\x1b\xfc\xa0\x6b\xd1\xd0\xda\x38\x5a\x47\xd9\x9d\x6e\x27\x16\xd9\x24\xae\xa5\x7e\x12\xc7\x17\x32\x7b\x74\x29\x58\x84\x42\x76\xba\xde\x52\xef\xa1\xa9\xdd\x9e\x6a\xde\x43\x21\xcd\x0e\xf1\x01\x40\xb5\xf3\xf4\xb9\xe0\x8e\x1e\x14\x21\xd5\x4c\x24\x01\x64\x77\xbc\x4b\x04\xbd\x5d\x56\xef\x24\x97\xd4\x99\x15\x99\x47\xba\x03\x14\x1d\x48\x00\x00\xab\xac\x23\xdf\xe4\x42\xdf\xfe\x38\x10\x61\xdd\x4a\xf1\x69\x8b\x1f\xc4\xde\xd5\x41\x62\xfe\x41\x26\x1b\x14\xbe\xdb\xe9\x4b\xbd\xe3\x13\x89\x25\x29\xbc\xdf\x72\x39\x66\xfa\xd4\xad\xe5\x9d\x40\xdb\x8f\x05\x47\x8e\x30\xbd\x44\xcf\xe2\x9f\x71\xc3\xfc\xd8\xd7\x32\xfe\x87\xef\x55\xcf\x5d\x5a\x51\x36\xaa\x6d\x51\x91\x19\xe2\x4c\x82\xd3\xc2\x6f\xe9\x23\x19\x14\x02\xc0\x25\x60\xd1\x22\xfb\x93\xe1\x49\x93\xac\x79\x43\x9f\x2c\x77\x84\xb0\x79\xee\x10\x13\x75\xf5\xf4\xfa\xba\x9c\x55\x80\xab\xf6\x3e\xd3\x4e\x73\x37\xd7\xa5\xee\xa7\xed\x3f\xba\x64\x7a\x10\xee\xc7\x47\x2a\x24\x31\x99\x2f\xdc\x8f\x5d\x65\xda\x75\xf1\xfa\x2d\x4d\x5b\xa9\xa0\x79\xd8\x9a\x70\x7c\x36\xce\xd0\x93\x97\xa7\xe9\xc8\x84\xbd\x04\xa8\xe2\xe7\x95\xa1\x9f\xe8\x75\xc6\x40\xac\x40\xab\x52\x61\x8d\x69\x3b\x6f\x94\x31\xa0\x9b\xbf\x87\xc3\xd4\x7b\xe7\xd0\x12\xd9\xb2\xde\x28\x95\x39\x0e\xe8\xa7\x36\x68\xcf\x44\xab\x5b\xe7\x31\x96\xef\xaf\x02\x89\x1e\x0e\x1b\x8d\x65\x8c\xfe\x2b\xf1\x32\x0e\x35\x66\xed\xe7\x3c\x63\xf7\x52\x91\xa1\x28\xac\xba\x5f\x82\xf6\x22\xdf\x8f\x46\x68\x46\x1c\xd8\x95\x69\xa3\xac\x6d\x97\x6f\x66\x9d\x04\x3b\xb4\x3c\x5b\x24\x09\x7e\x7e\x54\x9d\x98\xa2\x2e\xf0\x53\x6d\xec\x29\xcc\xd4\xbe\x6e\xf8\x7d\x36\xc6\x80\x3c\xb6\x3b\x0c\xa0\x17\x5f\x82\x96\xf3\x7e\x35\xc1\x53\x5b\xcb\x21\x9b\xdc\xae\xa4\xcd\x31\x24\x4e\x8a\x90\x80\x82\x2c\x6f\x4d\x78\x4a\x52\xb1\x28\x42\x25\x89\x61\x6b\x78\x12\xef\x69\xbf\xc6\x9b\x26\x75\x69\x47\x5b\x7b\x2d\xc3\x81\x87\xc6\x50\x64\xa2\x71\xc8\xb3\x00\x8c\x6e\x57\xbe\x62\xe5\xdb\x50\xd3\xf5\x9c\xed\x13\x02\x17\xa6\xdd\x29\x47\x69\xc0\x92\x98\x54\xda\xb9\x78\x50\x3e\xde\xbe\x6e\x1c\xde\xf7\x35\x4a\xb0\xdb\x56\x9d\xda\x81\xb4\x0e\x41\x9b\x78\x69\xee\xb7\x82\x8f\x37\xbd\x6b\x6c\x64\x2a\x8b\xc4\x67\x9d\xcc\xdf\x10\x65\x43\x23\xbf\xf8\x8b\xe5\xea\x3d\x9f\xba\x1c\x65\x90\x06\xa8\x5a\xe0\x23\xfc\xf6\xf0\x84\xbd\x91\x75\x0e\xf9\x30\xc8\x87\x28\x02\x28\x7a\x85\x24\xa9\x7f\x53\x58\xa2\xd0\xda\xde\xd5\x9e\x3b\x41\x8a\x6f\x28\x65\xb1\x75\xa3\xbd\x12\x6c\x14\x82\x0c\x31\x80\xdc\x65\x04\xbc\x36\x4d\xb7\xc3\x35\x17\xa2\x9f\xfa\x2e\x6f\x90\x2f\x0e\xa2\xf4\x5b\xce\x86\x8d\xf3\x5f\x30\x7a\xdc\x9a\xb0\x1b\x1b\x96\x0c\x15\x60\x64\x5a\x88\xa9\xb8\x0e\x18\xdf\x98\x34\x29\x82\xaa\x9c\x36\x0c\x44\x91\xf6\x93\x57\x50\xad\x5f\xaf\x2f\xf0\xde\x34\x14\xfd\x0d\x06\xab\xf4\x2b\x05\x1e\xcb\xc1\x31\xce\x2f\xe4\x1d\x3e\x84\x5d\x5e\x97\x9b\xe9\xff\x28\xdf\x17\x9d\x4e\xf0\xad\xf3\x0b\x13\x37\xc7\x30\x3b\xe1\xa7\x70\xd2\x10\x33\xac\xa7\x60\xa3\xd4\x3a\x4f\xa6\xf3\x0d\xb4\x73\xc4\x67\x83\xfa\xe8\x62\xba\xb8\x28\xe7\xc8\x84\x8e\x2e\x36\xeb\x39\x66\x05\x5b\x33\x23\xe4\xe7\xab\x69\x8d\x21\x8c\x08\x86\xa3\x35\x54\x00\xa9\x58\x0a\xd0\xd9\x1f\x93\x79\x2c\x14\x50\xe6\xd9\x66\x7a\xbd\x3a\x7a\x0b\x67\x9d\xa3\xdb\xab\xea\xe2\x2a\x7c\xe1\x53\x68\x0e\x2f\x6b\x1d\x9d\x08\x78\x17\x57\xe8\x2a\x0a\xff\x3c\x02\x64\x60\x74\x26\xfc\x0f\x7f\x47\x4d\xe0\x70\xe3\x1d\x44\xc7\x1d\x5d\xfc\x33\x02\x09\x0e\x3b\x09\xf4\x55\xc8\x03\x4e\x52\xad\xaf\xdd\x8e\x1a\x29\xb9\x4d\x11\x68\x64\x9a\x60\x3b\xf3\x96\x83\x8e\x81\x77\x8d\x23\xfe\x03\x28\x9e\x57\x50\xf4\xef\xf2\xf7\x1f\x47\x97\xeb\xe5\xb5\x50\xce\x11\xc7\x13\xfd\xbb\xfc\xfd\xc7\x11\xec\xbd\xe5\xdf\xe9\xdf\x7f\x1c\xd5\x17\xeb\xb2\x5c\xfc\x5d\xfe\xfe\xe3\x68\xb3\x94\x5a\x9f\x84\x0a\x64\x22\x18\x74\x94\x01\x86\x23\x89\x01\x62\x35\x62\xac\x67\x87\xc5\xdf\xc6\x03\xcd\xe5\x79\x89\xc3\x16\xc4\x12\xc8\x6a\x25\xed\x30\xc7\x83\x56\x84\x43\xbf\x21\xc5\xef\x39\x3b\x23\x70\x85\xd6\x88\x9f\xcf\xe5\x08\xdf\x36\xd7\x8d\x8a\x61\x68\x74\xde\xc3\x08\xbf\xf4\x1a\x6f\xbf\xa6\x77\x6f\xbf\x2f\x2f\xa1\x2f\x8e\xfc\xcd\xc1\x59\xbd\xe4\x93\xec\xd8\x95\xe7\x16\x1a\xe5\xfd\xe4\x93\xcc\xeb\xfc\x1f\x71\xe7\xff\x68\x74\xfe\x7c\xb9\x6a\xe9\x9b\x52\x5b\xba\x8e\x4b\x7b\xa9\x27\x18\x03\x4d\x85\x2a\x6f\x25\x76\xbb\xde\x4c\x79\x42\xa0\x57\xa4\xf0\x27\x38\x64\x54\x8e\xe8\x2d\x9d\xe5\x5e\x69\x1d\x2c\x2d\x9c\xad\x44\xf6\x29\x42\x8f\x59\x7f\x69\x0d\xba\x5c\x64\x34\xc8\x1f\xe8\xcf\x2f\xf2\x2f\xf5\xe7\x83\xfc\xc4\x2d\x21\x11\x23\xf3\x3b\x7c\x76\x39\xbf\x53\xcd\x10\xc5\x75\x25\x89\x28\xbf\x6b\xe8\x84\xf7\xda\xc2\xf2\x01\x4d\x6c\x61\xbb\x5d\x67\xb0\xa3\xde\xd2\x7c\xec\x16\xd7\x77\x68\x7c\x94\xba\xdf\x20\x0a\xe3\x55\xa7\x1a\xc9\xf0\x2b\x01\xad\xbd\x4b\xfc\x4d\xff\xc9\x10\x7c\x94\xee\xad\x7b\x04\x9a\xf7\x57\x6c\x20\xc3\xa3\xb3\x1f\x78\x83\x52\xa9\x2f\x3f\x9e\x8c\x09\x6e\x97\x12\x31\x91\x49\x76\x06\xed\x62\x0e\xa1\x40\xdf\x49\x3b\x04\x08\xb6\x21\x23\xc5\x9f\xe1\x40\xdb\x7a\x5e\x02\xfe\xa0\xeb\x12\x84\x86\xf2\x66\xc1\x33\xe3\xcb\xf9\xed\x87\x73\x11\xf3\x5b\x8f\x23\xe1\xf6\xc9\xb0\xb3\x79\xb2\x5f\x15\x6d\xff\x4c\x5d\x5d\xdf\xcc\x83\xc7\x1c\xed\x75\x87\x9c\x49\x79\xff\x62\xf6\x56\x46\xe6\x04\xb1\xa8\x20\x55\x3c\x2b\x83\xaa\x7e\x26\x3d\x70\xec\x61\x1f\xb2\x1c\x43\xfb\x0c\xb9\xfd\x51\x9b\x66\x23\x65\x9f\x79\x82\x26\xdb\x63\x54\xe2\x6e\x78\x70\xab\x68\x55\x29\xed\x53\x54\x39\x9b\x59\xef\xce\xc7\x7f\x98\xdd\x26\xda\x5b\xfc\x96\xc8\x9a\x72\x21\x44\xe2\x6b\x5b\x1d\xb9\xbd\x6f\xc9\x49\xbd\xfa\x48\x24\xbb\xb6\xc0\x9d\xda\xbc\xfa\xf7\x15\x56\x95\xcf\x31\x4f\xf8\x5d\xf1\x27\xac\x8c\xf7\xfd\x26\xc5\x28\xb4\xdd\x61\xd2\x2b\xc0\xfa\xa8\xb8\x2d\xb6\x0f\x95\xce\xdd\x1b\xe4\x8c\x22\x17\x7b\x62\x7d\xc1\xd2\x4e\xb0\x42\x8e\xaa\x45\xbd\x41\x69\xc7\x9a\x14\x71\xbd\x11\x96\x87\xb3\xec\xfa\x82\xed\x85\x78\x91\x84\xb2\x35\xb6\x68\x99\x47\xa1\x45\x39\xa9\x39\xb7\x54\x60\x16\x25\x6e\xb7\x98\xea\x53\x3d\xeb\x03\x31\x15\xd8\xef\xcf\x01\x15\x30\x38\x8d\xe4\x54\x1f\xd7\x7c\xbe\x66\x23\x77\xf8\xc9\x4f\x5b\xe7\x01\x74\x3c\x7a\xcf\xca\x85\x96\x05\x71\x46\xc6\x8b\x0c\x46\x25\xb5\xc2\x61\x40\x93\xac\x36\x71\xb1\xbc\x95\x47\x78\xe3\xc3\x0a\xde\xa6\x58\x97\x82\x58\x3a\x77\x73\x10\xce\x12\xca\xf4\x9b\x25\x01\x7a\xd7\xc4\x9c\x3f\x26\xd3\xa6\x44\x8a\x0a\x1c\x50\xfd\x04\x25\xc3\x35\x16\x3f\xde\x2a\x81\x50\x82\x29\x1f\xee\x9b\x5b\x37\x01\x86\x64\xfa\x68\xfd\x8e\xe2\x84\x14\x04\xf0\x60\xda\x71\x49\x99\x48\xd5\xf2\x29\x10\x35\x87\xda\x84\x29\x6a\x1e\x16\x7b\x53\xb9\x63\x50\xe7\x89\xc2\x3f\xef\xb6\x05\x85\x03\xc6\x52\x6d\x48\xcd\x63\xf7\xf8\x83\xb8\xf7\x01\x62\x67\xfa\xb8\xf3\xc8\x31\xe0\x8e\xa4\x68\x44\xf0\x3a\x4f\xe8\x37\x70\xa5\x75\xc2\xc2\xf5\xbc\x9c\xbe\x2d\x35\x99\x76\x25\xf7\x1a\x2b\x20\x08\xcf\x13\xee\x21\xdd\x40\xcb\x86\xd9\x93\xe2\x2e\xd8\xdd\xa0\xb4\xd1\x5b\x10\xfa\x60\x6e\xd2\xd8\xce\xd4\x7d\x72\xe3\x29\x60\x45\x96\x2a\x5a\x24\x2b\xcf\xb6\x2d\x52\x15\x8a\xcc\x9c\x76\xa4\xfc\x76\x2b\x3f\xf0\x21\x75\x11\xdc\x54\xd3\x2f\x2f\x00\xd4\x29\x67\x68\x87\x9e\x06\x92\x56\x4e\x53\x7b\x48\xba\xc6\xa6\x75\x54\xeb\x73\xbe\xc6\x6b\x09\x71\x67\xc8\xda\xbf\x69\xc8\xce\xcf\x26\xc9\x53\x47\x9e\x7b\x83\x8f\x62\x29\x54\xdc\x91\x49\xda\x47\x8b\x67\xfc\x24\x1b\x3a\x8b\x84\xad\x62\x08\x4b\x2e\x41\xe2\x58\xff\x25\x37\x8f\x67\xc0\x15\x86\x6c\xd5\x04\xef\x71\x28\x95\x01\xe8\xba\x5c\x64\x59\x83\xed\x17\x07\xbd\xa4\x3c\xfe\x16\x64\xb3\x7c\xaa\x2e\x5c\xd8\x8e\x73\xa6\x41\x05\xca\x75\x1c\x3f\x1d\xd3\x4c\x12\xe0\x0a\xe5\xce\xe6\xb8\xfc\x82\x6d\xe3\xf0\xa5\x28\xc9\x7e\x79\xae\x4b\xd3\xcd\xce\xfe\x5e\xd1\x2e\x23\x88\x01\xfe\x11\xe2\x5a\xd8\x91\x67\xb8\xd1\x92\xcb\x4b\xd9\xf7\x19\xb0\x2a\x78\xb9\x5f\x8f\x8c\x71\x55\x8a\x4b\x13\x1d\x6d\xd4\x82\x08\x66\x04\xb7\x51\xeb\x98\x3f\x4c\x44\x62\xc9\xc2\x85\x2c\xa6\x1d\x5e\x9a\x94\xce\xcf\x82\x7d\x80\xd2\xb9\xd0\x7e\x4a\x77\x96\xf4\xa2\x0d\xf3\x83\x13\x65\xa3\x34\x6d\x3f\x8a\x6c\xb7\xce\xc3\xde\x26\xb6\x12\x10\x0f\x08\x37\xd5\x72\xbd\x79\xcf\xe0\xf4\x5f\xf2\xdf\x26\x21\xd9\xc7\xe4\xec\x39\x2f\x94\xbf\xb5\x1d\x84\x4e\x63\x8b\xbd\x7c\x7d\x53\x6f\xa4\xc5\x19\x6d\x08\xd1\x95\x60\x63\x7d\xee\xeb\xbd\xd9\x5c\x1b\xb5\xb4\xf6\x3a\x88\xfa\x74\x54\xa4\x9d\x79\x57\x53\x04\x23\x9f\x65\xf6\x21\x8c\x8f\x30\x74\x26\xc2\xc7\x8d\x5b\x40\x6e\xe1\x20\xc3\x78\x36\xad\x8d\x8d\xf8\x3d\xb7\x3d\xa2\x10\x90\x51\x3b\x13\xf0\x0b\xee\xc7\x5e\x27\x5a\x27\xee\x66\xda\x1e\x5b\x5c\xd2\x73\x35\xca\xf8\x08\xac\xb5\xae\xbd\x98\xb7\xb4\x0d\xa6\xc1\x5b\x0e\xec\x9a\x8c\x48\xff\xba\xc5\x93\xc2\xc5\x5c\xe9\xe3\x46\x24\x2b\xa3\xe3\x05\x64\xf2\x4f\xf4\x7e\xfa\x28\xda\x75\x0f\x6f\x81\xf6\xe0\xfb\x61\xa6\xd3\xca\x53\x04\xa7\x40\x78\x87\x57\x7d\x0b\xaf\x91\x63\xbd\x65\x36\x81\x18\xc4\xfa\x06\x7b\xf4\xe7\x93\xbf\x7f\x1e\x6f\x91\x7c\x24\x88\xc0\x94\xae\x87\x4e\xac\x7f\x48\x3c\x33\xed\xd4\x81\x02\x90\x3f\x55\x66\xcf\xe5\x3a\xd1\xc9\xb0\x55\xc8\x82\x42\x93\x16\xae\x48\x81\x4b\x05\xac\x5e\x8f\x54\x5b\xa2\x9d\x88\xad\xcd\x69\x28\x6a\x64\x48\x52\x68\xdb\xac\x60\x7b\xc7\xc7\xda\xa2\xd7\x5e\xdb\x11\xb6\xd1\x64\x7b\x7c\x7c\x5f\xae\x65\xf3\x6c\x6b\xe3\x48\x17\xb8\x97\x0b\x8a\x56\x65\x63\x6d\x92\xa2\xe0\xdb\x05\x5e\x1f\x36\x0c\xcf\x36\x6c\xc1\xdd\x88\x0e\xee\x99\xb5\xd2\x1d\xf1\x8c\x5e\x97\xb1\xfd\x58\x97\x1d\x17\x30\x7c\xe8\x9b\x6a\x72\xb3\x7c\x1a\x10\x2d\x48\x08\xa3\x98\x69\x92\x5d\x07\x02\xeb\x2f\xb6\x1d\x00\xea\x6c\xc4\x58\x47\x8b\x6f\x39\x8c\xf0\x29\x87\xa0\x99\xa2\x05\x8e\x5c\x8b\xa3\x45\x6e\x1c\x47\xce\x9a\x46\x63\x53\x04\x08\x35\xe3\xd5\x0e\x33\x0e\x0d\x38\xc3\xd7\x2c\xc8\x42\x01\xfe\x7a\xc7\xb6\xe1\x7f\x21\xcb\x55\x34\x22\xbb\x5c\xf8\x91\x6b\x75\x8c\x78\xab\x0a\x63\xc6\xe0\xeb\x34\x31\xfc\xee\x44\x83\xf4\xc3\x10\x60\x19\x3e\xe2\xa5\x64\xcd\xf5\xda\x99\xc4\x0e\xdf\xa0\x20\xff\x02\x29\x25\x4e\x08\xde\x57\xe4\x36\xd5\x1a\xc3\x31\x5a\x7a\x76\x63\x62\x82\x03\x88\x43\x87\x34\x0c\xa7\x89\x6f\x13\x7f\x80\x30\xc3\x30\xdd\xfb\xe9\x17\x0f\x9b\x30\xde\xbd\xcd\x61\x4b\xce\x82\x96\x7c\x0e\xde\x73\x4c\x98\x0d\x5b\x88\xb3\xd1\xfe\x2a\x52\x4f\x70\xaa\xbb\x9e\xd5\x88\x88\xf6\x10\x14\x15\xd0\x27\x45\x39\x39\x34\x6e\xe1\x09\x69\x31\xbc\x18\x35\xcf\x39\xbd\xa4\x9f\xf4\x5a\x8a\xe6\x2d\x47\xa2\xe6\xad\x77\xd3\xb1\x84\x67\xec\xd0\x82\xde\xb7\x18\x2f\x2f\xa3\xd5\xe8\x2d\xc4\x70\x11\xca\xf9\xcf\x12\xbe\xd8\x81\xe9\x8c\x34\xd7\x54\xc6\x6f\xad\x1c\x5a\x32\xba\x62\xb8\xa4\xaf\x09\xfa\x08\x12\xf4\x37\x33\x4b\x85\x01\x01\xe2\x01\x39\x26\x99\xc3\x84\x47\x1a\x51\x2d\xb2\x33\x37\x8b\xb6\x26\x1a\xb5\x05\x89\x52\x9d\x6a\x2a\x6d\x34\x43\xc9\x9a\x8f\x81\xa3\xb1\x4e\x10\x96\x0f\xb6\xe9\x35\x37\x68\x09\xb8\x3d\xb2\xd0\xda\x7a\xe8\xa2\x20\x5a\x37\x37\x0a\xe7\xb0\x80\xd9\xdc\x7b\x43\x6d\x4f\x54\x13\x3c\xc1\xf6\xe1\x19\x53\xcd\xb3\xad\xcb\x16\x17\x3b\xd7\xbe\x18\x31\xee\x7d\xec\x85\x84\x32\x8d\x54\xeb\xf9\xb5\xb7\x6b\xb8\x5d\x3f\x24\x0c\xc2\x0e\xea\x1c\x87\x9d\x0c\xc2\x36\xab\xc6\x7b\x95\x4c\xb3\x8e\xaa\xfa\xc7\x29\x5e\x42\xa4\x1e\x66\xd7\x0b\x4a\x62\x39\xe9\x72\x81\xe1\xe6\x76\xb6\x02\x87\xc4\x78\x84\x9b\xbb\x3e\x37\x8c\xbf\xf1\x82\x18\x8d\x5e\x14\x64\x4a\x2c\xfc\x97\xaa\x22\x77\x69\x2c\xcf\x96\x1b\xf0\xab\x57\x50\xb0\xee\x27\xe8\xe5\x47\x35\xf5\xd1\x20\x39\xe5\xf2\xe3\x7f\x54\x45\x2c\xd8\x27\xc6\xa6\x49\x50\xa8\x9d\x83\xf1\x7a\xba\x7e\x63\x83\xf6\x7a\x03\xbb\x5c\x8c\x7d\x9d\xe8\xe5\xc2\x55\xc1\x98\xe2\xeb\x4d\xaa\x5c\x16\xdf\x75\x3c\xf8\xea\xe8\xd0\xbb\x2a\xba\x5c\xd0\x2b\xbf\xee\x8a\xc8\x06\x29\xbe\xc4\x63\xdc\xfc\xfd\xdd\x4c\x1e\x86\xf4\x60\x7c\x56\xfd\xf6\xdb\xbc\x74\x74\x2a\xde\xd8\x86\x2f\x57\x90\x44\x71\x9e\x10\x16\xb2\x8d\xe1\xf9\xbd\x36\x6a\xa5\x60\x2a\x43\x71\x3c\x6b\xb3\x9c\xcf\xcc\xa2\x82\xff\xca\x5b\x31\xf8\xc1\x9f\xcf\xa4\x5d\xe4\x98\xa9\xb4\x3d\x92\xbf\xf1\x9d\xae\x24\xe7\xb0\x77\xf0\x03\x2d\x90\x95\x79\x17\x66\xe8\x07\xb4\xd1\xe2\xda\x58\xa6\x10\x17\xf2\xd7\xbb\x1b\x96\x31\x14\xf2\x97\x6c\x2d\x3b\xde\x9a\xdb\x27\x7a\x65\x36\x50\x0b\xd5\x43\xd8\x31\x78\x59\xaa\x83\xd6\xae\x5c\x4c\x41\x7e\x6d\x30\x7a\x68\x90\xfd\x33\x3a\x0a\xce\x53\x0c\x76\x86\x06\xed\xa5\xbc\xc7\x47\x08\x2d\xd6\xbf\xde\xc0\xc9\x98\x48\x8e\xfc\xd8\x2c\x10\x68\x06\x76\xcd\xd6\x8c\xe3\xc1\x84\x6a\xfc\xa5\xf0\x3a\xd1\x57\x10\x2d\x34\xee\x65\xe6\x6f\xde\x3f\x9d\xa5\xd7\x78\xeb\x4b\x47\x27\xfa\xe3\xdb\x8c\x34\xc7\x47\x25\x2a\xf4\x82\xbe\x8e\x72\x79\x71\xb0\x13\x81\x24\xed\x54\xcc\x6a\x9d\x48\xb9\x97\x29\x5a\x33\x9b\x40\x02\xee\xac\x16\x53\x89\xaf\x64\x6f\xc3\x4f\x07\xca\xa2\x75\xfc\xc0\xda\x46\x61\x51\x11\xd8\x2c\x51\x3b\x0b\xde\x26\xf2\xbc\x67\xad\x9d\x47\xef\x49\x66\x3b\x22\x52\xd6\x89\xf9\x02\x2d\x4b\xbd\x57\x5c\xbe\x79\xff\xc8\xc5\x5b\x6f\x6d\xdb\xe6\xff\x01\xf8\x6c\x1b\x38\xc9\x1e\x68\x28\xbf\x2b\x30\xbf\xd6\x53\x74\xed\x39\xbf\x79\xf5\xea\xfd\x7f\x3c\x7b\x18\xb9\x28\x13\xf5\xc0\x92\xa5\xd8\x58\x80\x62\xe1\x47\xde\xea\xd5\xf9\xf3\x57\x71\xe1\x13\xa0\x73\xcb\xa3\x47\x23\x5d\xb2\x4a\x46\x14\xe5\xd4\x2d\x94\x1f\x1b\xe1\xc9\x58\x58\x62\xfe\x51\x6c\x96\x6f\xca\x45\xf5\x9b\x8f\x74\x02\xd0\x43\x80\x17\x88\x03\x1f\xd2\xcc\x46\x08\x39\x94\x71\xc1\xa9\xcb\xfa\x62\x8a\x5e\x3e\x67\x67\x9f\x75\x61\xbb\xd5\xba\x75\x5c\x17\xc7\x9c\xd1\xc0\x93\x31\xfc\xf3\x79\xd2\x83\xdf\xbd\xe4\xf3\x09\x05\x97\x67\x88\x02\xd3\x30\x4e\x1a\x57\x93\x02\x0b\x6e\x96\x8a\x91\xd4\x66\x64\x43\x0f\x77\xeb\xba\x3a\x9f\x03\x6f\x89\xb0\xee\x48\xc2\xd7\xd5\xb4\x21\x5a\x40\x60\xa7\x20\x03\x1b\x1f\x9e\xcc\x5c\xbe\x6f\x21\x70\x98\x7e\x1c\x50\xf8\x8a\xd6\xfa\xbd\xb6\xf0\x90\xf3\x9e\xb5\x51\xb8\xec\x20\x40\x41\x4f\xd6\x6b\x8c\x79\x61\x37\x10\x40\xb6\x85\xb7\x11\x62\x94\x26\x05\x4e\xce\x6a\x02\x4c\x4d\xdb\xc1\xbb\x49\xc2\x87\x2c\x4c\xf2\xd9\x20\xc9\xda\x77\x1b\xb7\x39\x09\x16\x1f\xa1\x02\x27\x9d\x9a\x73\xd9\xe8\x6f\xd6\xc5\x79\xb7\x0b\x07\xc5\xea\xf2\x12\xad\x64\x81\xe3\xfc\xf3\xbc\x5f\x2f\x6f\xd6\x17\xe5\x53\x8e\x79\xfd\xc3\xc3\xbf\xbf\xfc\xf1\xc9\xbf\x3f\x7c\xfe\xf4\xaf\x4f\xb2\xe3\xf4\x9f\xd3\x03\xd9\x64\xdd\x0b\x4d\xe9\x82\xc4\xdf\xe2\x59\x67\x4d\x76\xd9\x18\x17\x40\x7d\xc6\x20\x91\x2b\x32\xa6\x17\xc5\xb9\xd4\x3b\x1e\xd8\x58\x8b\xa3\x41\x7e\x3c\x88\x65\x15\x8a\x3a\xf7\x73\x5d\xde\xcc\x96\xe1\x13\x67\x2d\xb1\x7a\xe8\x8a\x3c\x50\x21\x86\x2b\x47\x9f\xe0\xe3\xcb\x09\x09\xf6\x25\x0c\x93\xee\x69\xf8\x8c\xb0\x8b\x61\xf8\x86\x6e\x2b\xfe\xb5\x40\xa4\x21\x14\xf8\x14\x05\xdf\x89\x48\x42\xf6\x11\x70\xe9\x53\xf0\xd3\xb9\xc0\xe6\x49\x4b\xa1\x14\xa5\x3f\x54\xe4\xb6\xc5\x34\xa1\xe8\xe9\x2f\xd3\x5e\x13\x69\x4c\x6d\x88\x5d\x3c\x78\xb1\x19\x27\xfa\x28\xeb\x02\x64\x29\x74\xcc\x80\xa2\x1a\x9d\xdd\x76\x08\x0c\xc2\x2f\x1a\xb0\x09\xac\x30\x7e\x1d\xe4\x03\x6b\x98\xa0\x97\x2f\xe5\x80\xbc\x67\x4d\xa5\xe1\x43\x12\x33\x7e\x92\xd1\xe2\xa4\xc1\x0a\x0d\x85\xe5\xc4\x20\xb5\x9e\x28\xa6\xa3\x30\x54\x5c\x7c\xfa\xeb\xe5\xb7\xd3\xb5\x8a\x64\x20\x48\x7d\x4b\xd6\x7e\x35\x4b\xa9\x33\x66\xb2\x24\x34\x8f\x7d\xcb\x6e\x76\x24\xa5\x22\x99\x8b\x07\xc5\x3d\x8e\x4e\x72\xce\x11\xeb\xe5\x13\xd6\x37\x51\x47\xee\xc8\x29\x4c\x1c\x70\xe6\x3a\x2d\x48\xa8\xb1\xdf\x43\xaa\x32\xcc\xee\xd2\x8e\xc0\xbf\xdd\xaa\x0c\x74\xb1\xbc\xbe\x9e\x8a\xfc\x83\xa5\x32\xba\xc1\xbc\x66\xaf\x01\xed\x0b\xff\x15\x20\x78\xf3\x3d\x99\xa8\x4d\xf8\x76\xcb\xd5\x04\x0e\xf1\xa2\x23\xbc\x00\x4c\x99\xda\x8c\xe3\x6d\x81\xf1\xfa\x84\x33\xe7\x14\xa0\xaf\x83\x9e\xad\x35\xfb\x8c\xe7\xd1\x3a\xab\x72\x7b\x62\xe6\xcd\x71\xf3\x74\x22\xc8\x52\xc8\x02\x15\x71\xba\x23\x7c\xf3\xc4\xec\x19\x84\x25\xb0\x2c\xd4\xe3\x11\xee\xd8\x56\x33\x13\x9a\x61\x70\x30\x83\xb5\x06\x01\xd4\xdb\xad\x43\xfc\x58\x1c\xef\xb4\x5e\x9c\xc3\xc9\x58\xe7\x77\x8e\x94\x48\x4d\xe8\x98\x73\x3e\x6e\x80\x28\x0e\x4b\x62\x76\x0e\x0b\xff\xcd\xae\x41\x6d\x5c\x5f\x42\x2c\xd3\xc7\x88\x8f\x28\xfd\x12\xf7\x22\xb7\x97\xe6\x8e\x96\xd3\x88\x0c\x33\x4b\xaa\xfe\x9a\xb2\x5b\x36\x8f\xce\x2d\x7d\xfb\x9c\x16\x8f\x5a\xd6\xbb\x55\xa2\x24\x89\xf7\xd8\x94\x5d\x35\x52\x1a\x5f\x93\x09\x82\x56\x6b\x01\xef\x48\x37\x9b\x3d\xb2\xa4\x26\x38\xc1\x73\x96\x26\x99\x73\xe0\xa6\x7a\xdc\x83\x8d\xc6\x66\xf4\xe1\xd3\xd0\xe5\xc5\x8f\xcb\x85\x0a\x7f\x05\x96\x06\x19\xcb\x09\x10\xc8\x67\xb1\xe4\x6c\xb9\x20\x26\x5d\xe0\x8f\x5e\xcf\x9e\x80\x5d\x7b\x97\xd5\xba\xde\x44\xc6\x5f\xba\x03\xbf\xbb\x9e\x33\x46\x38\x30\x09\x59\x56\x41\xab\x93\xa1\x7a\xa3\x94\xbe\xd3\x38\x08\x01\x11\x5c\x99\xe5\xd6\x34\xbe\x66\xe3\xb1\x59\x58\xd0\x31\x0d\x1e\xd5\x07\x7c\x88\x5f\x82\x38\xb1\xa6\xa9\x85\x3d\x1e\x9d\x06\x70\x80\xeb\x9b\x45\x8d\xcc\xaa\xa7\x03\x45\x72\x3a\x08\xf6\x47\xc1\xad\x1e\x24\x2d\x20\xdb\xe7\x7e\x5c\x44\x84\xdf\x8b\x1d\x94\xb6\xed\xa0\xb8\xbe\xaa\x00\x24\xd6\x8e\x53\x09\x50\x64\x2c\x56\x5c\xb8\x3a\xd4\x1d\x0a\x90\xf4\x71\x32\xe1\x49\xa7\x97\xb7\x71\xa4\xa4\x33\xe7\x3c\x38\x2b\x16\xec\x1e\x24\x0f\xe7\x32\xfb\x46\xf4\x59\x09\x87\x33\x74\x60\xca\xfc\xe3\xee\x8a\x31\x77\xa1\x4a\x8e\xc1\xa4\xd8\x8b\x2b\x18\xb3\xed\xc6\x15\xc7\x3e\x1c\x16\xdd\x9a\x28\x19\x2f\x3f\x48\x6b\xd2\x6a\xed\xed\xf9\x9c\x60\x9f\x72\xfc\x10\xd5\xf2\x3a\x8e\xaa\xf1\x9e\xec\x98\x0e\x39\x38\xed\x9f\x66\x95\xdf\x00\xd2\xdc\x56\x38\x99\x78\xb2\xca\x12\xb6\xf4\x05\x88\x3e\x37\x0b\xb7\x03\xaf\xc4\xa2\xbe\x1d\x26\xea\x0c\xe4\xec\xff\xd4\x1a\xe2\x90\x45\x0c\xc7\x36\xa3\x3c\x87\x43\xad\x88\x9b\x01\x7c\x78\x8c\x87\xcf\xd4\xb6\x06\x1e\x38\xd0\xed\x8c\xfb\xde\x6e\xd5\x9f\xa3\x31\x36\x8c\xcb\xe0\x75\xef\x1f\x9c\xb9\x37\xa4\xff\x15\x27\x57\xce\x7f\xcf\xaf\xe3\xc9\xe3\xa5\x9d\x32\xbb\xa7\xf8\x91\x89\x98\xb5\xa1\x19\x89\xe4\xf1\x4f\xc0\x1a\xff\xb4\xa7\x0d\x7b\x76\xb1\x45\x61\xb7\x72\x1f\x76\x2d\x60\x3c\x4a\x9b\x5a\xf8\xdd\xdb\x54\xd8\xdc\x5d\x2f\xb6\x15\xfc\x68\xb6\x82\xa9\xcd\x56\x9a\xe0\x65\x87\x44\x47\x3d\xab\x34\x18\xd8\xa6\xe4\x80\x21\x38\x07\x80\x9e\x1f\xa6\x2b\x92\x87\xa0\x61\xfd\xb9\x2e\xcb\x77\x55\xbd\x81\xf3\x83\xea\xa4\x74\xe6\xb1\x52\x4d\xf2\xe0\xd6\x3e\x3e\x2b\xe7\xb6\x3a\xf5\xd5\xc4\x89\x89\x95\x4f\xa3\xb1\xa4\x4c\xf4\xbc\x8b\xcf\x23\xeb\x6c\x3c\x5d\x14\x1d\x3b\x57\x18\x9a\xa1\xc4\x29\xb7\xb7\x71\xd4\x6f\x6e\x09\x9b\x3e\x05\x78\x27\xc9\x05\x43\xd5\x86\x7f\xba\x51\x37\xdc\xf5\xc8\xe1\x11\x98\x19\xf6\x30\xb2\x75\x73\x6f\xd0\x20\xa3\xb8\x69\x1b\x8d\x27\xb9\x20\x21\xb7\xb0\x0e\x55\x7f\x83\x53\x79\x1d\x72\x87\xa7\x0b\xaf\xeb\x10\x24\xaf\x59\x92\xa1\x4a\x0c\xbe\xa0\x63\xf2\x2a\xc9\x5c\xf8\x15\xa8\x30\x20\x2c\x6c\xb0\x2a\x30\x39\x60\x23\xf2\xc8\x24\x24\xcb\xda\x73\xed\x8e\xa5\x5d\x14\xf5\xad\x80\x0f\xf0\x06\xe9\xb4\xe6\x48\x04\xb2\x0a\x40\x1f\x6d\x16\x61\x51\x8e\x5c\x28\xe1\x88\x88\x73\xb8\x5e\x5b\xa0\xf3\x40\x22\x18\x69\x10\xb4\xb4\x1d\x4c\x95\x80\x32\x74\x7d\xd0\x9d\xaf\x3f\xaf\x18\x7a\x0a\xd1\x42\xfb\x36\x47\x98\xf8\xa4\x7e\x19\x5e\xd7\xc1\x48\x02\xb9\xb0\x4a\x82\x56\x11\xc1\x90\x33\xa5\x61\xa5\xd3\xe3\x81\x9e\x92\xb0\x32\xe2\x51\xa8\x43\xbe\x19\x7f\x3b\xde\xb1\x3c\x58\x5b\xe6\xb9\x28\x74\x79\x8d\x3c\xb0\x25\xe6\x93\x47\x8f\xa6\x31\xa8\x2c\x77\x49\x3e\x5f\x19\xc5\xc8\x52\x26\xe0\x11\x17\x22\x2b\x6f\x51\xc6\x78\x2d\xfa\x87\x3c\x49\xfe\x76\xbd\xbc\x7e\x4e\xa2\x64\x43\x2e\x25\x21\x42\x75\xe9\xca\x5c\x5f\xb7\x88\xaa\xf3\x72\x8a\xe1\x72\x7f\x41\xe3\xcf\xea\x6d\xc9\xe7\xae\xb5\x7c\x8d\x45\x50\x85\x73\x09\x5f\xfa\x57\xd7\x88\x88\x6a\x63\x8b\x47\xd5\xb7\xdb\xb0\x3e\x9e\x0f\x81\xf0\xa2\x52\xa3\x41\x7e\xc2\x40\xa9\xce\x2b\x94\x73\xf7\x07\x56\x46\x89\xc4\x1b\xd9\xae\x01\x10\x5e\x1f\x71\xd3\x0f\x17\xef\x3f\xa5\xf5\x80\xca\x02\xec\x51\x31\x20\xb1\x43\x9d\xe1\x89\x75\x7c\x40\xcc\x10\xe1\x20\xc2\x03\x10\x2d\xe4\x5a\xa5\x18\xbe\x9d\x86\x42\xd4\x35\xd0\x8b\x74\x8e\xc7\xac\x00\x1a\x55\x69\x66\x8e\x85\xfb\x88\x6c\x76\x9d\x47\xc8\x68\xd9\xe6\x77\xd1\x8b\xb8\x32\xa2\x76\x52\xa8\x84\x14\x32\x37\xee\x10\xbd\xfb\xc4\x33\x45\x54\x36\x19\x06\xda\x76\xed\x88\xa5\x90\xb8\x1b\x59\x0f\xb4\x6e\x5c\x96\x6a\x60\xb4\x4d\xbb\x5b\xeb\x43\xbb\xbd\x5e\x45\x63\x7a\x0d\x9b\x7a\xeb\x30\x5e\x4b\xfb\xc3\xd7\x30\x66\x77\xfe\xb2\x60\x57\xa7\x03\x56\x3d\xb5\x0e\x85\x72\x1b\xa7\x42\x3d\x39\x9a\xea\x78\x90\x65\xad\x3a\x4b\xbb\x12\x4f\x2b\xbb\x47\x35\x56\xb1\xb4\x53\x99\xd7\x20\x4c\xc8\x28\xf6\x95\x2d\x82\x2a\x5e\x85\xe6\x99\x35\xdb\x59\x11\xd7\x67\xeb\x2e\x30\xca\x9e\xd1\xb6\xb2\x9d\x7f\xc7\xc3\xb2\x14\xad\xa3\x39\x47\x55\xac\xfd\xed\x2b\x9d\xf8\x78\x30\xfb\x05\xc4\x7c\x90\x62\xcf\xdf\x3f\x2b\x37\x9e\x44\x65\x05\x76\xcc\x92\xc3\x4f\x11\xb5\xec\x8a\xd4\x37\xab\x72\x2d\xc9\x45\x28\x64\x79\xc4\x6d\x79\x2d\x13\x88\xae\x2a\x7b\x4b\x6d\x5e\xdb\x29\xb1\x80\x90\x98\x2d\xe2\xab\xc6\x46\xad\x8a\xe4\x24\x31\x56\x82\x2e\x58\x12\x82\x72\x76\xc1\x8a\xe0\x1d\xf4\xa3\x90\x7c\x33\xbd\x78\x73\xb3\x6a\x2c\xee\x40\x70\xb3\x43\xee\x76\x65\x39\x2c\x66\xfd\xe7\x0f\xff\x3d\x45\xb1\x2d\x68\xf5\xe0\x45\x40\x66\xe4\xb4\xfb\x9f\x8b\xea\xd7\x9b\xd2\x9e\x7d\x25\x7c\x57\x00\x10\xb0\xe1\x1f\x30\x6c\xd4\x1a\x97\xce\x35\xc6\x1d\xea\x0f\x48\x8f\x64\x01\x45\x63\xa7\x08\xe8\xc2\x31\x2b\x77\xb7\xaa\x18\x77\xc7\xb8\xa2\x31\xe3\xd9\x90\x31\x94\xda\x53\x30\xdf\x7c\x20\xd3\x41\x66\xe0\x21\x80\x19\xb3\x3e\x99\xad\x5c\x22\x22\x05\x79\x3c\xdb\xb1\x91\x16\xce\x76\xd7\xbc\xf2\x1b\xb2\xee\xc8\x1f\xa2\xe0\xa8\x08\x30\xe7\x8f\xa5\xd7\x6b\x8e\x66\x47\xe4\x0b\xb5\x79\x3c\x1d\x2b\x79\xca\xfd\xa3\x4f\x40\x18\xfb\x91\x08\xe6\xa6\xe5\x28\x45\xd7\x33\x7e\xe9\x5e\x51\x19\x69\x1c\x1f\xe2\xf4\xb3\x9a\x48\xf1\x96\x8f\x20\x44\xb1\xe1\xce\x99\x8e\xb2\x03\xec\x04\xe2\xa4\xdf\xcb\xe9\x09\x5f\x5d\x90\x90\xe6\x9f\x1a\xd1\x94\x50\x9b\xa2\xcf\x34\xf8\x06\xa1\x4d\x02\x05\x0b\xda\x41\x64\xf5\x56\x95\x15\xb6\x5c\x5a\xb6\x6b\xbb\x80\x72\xd9\xc6\x9b\x26\x7b\xfa\xd0\x5c\xcb\x0b\x42\x64\xf7\x5a\x38\x0a\x70\x6b\x51\x04\xde\xd0\xe4\x3e\xc3\x78\x81\x0a\xa4\xf2\xbf\x0f\x93\xc4\xbe\xb5\xc0\xcb\x29\x73\xfc\x61\xa7\x3b\x0a\xcd\xe3\x28\x38\x10\xfa\x8c\x2b\xcb\xfd\x2f\x8f\xd1\xee\x3b\xc5\xe9\x04\xd6\xca\xd8\x9a\x9a\x48\x2d\x61\x5f\xb7\xb6\x5b\xfb\x1e\x73\x0d\x8a\xdc\xa2\xcd\x0d\xc3\xbb\x72\xff\xf8\x1e\xdc\xc8\x1d\xb0\xf5\xa8\xec\xa5\x02\xfe\xe1\xab\x05\xe4\x66\xcc\x53\x5b\xee\x6a\x49\xb5\xcb\xb3\x8b\xea\x30\x56\x27\x8b\x4c\x4d\xbe\xcf\xbc\xd3\xa9\x8a\xdc\xfe\xb0\x9a\x5a\x13\x88\xb4\xa7\x0f\xba\xdd\xe4\xe9\x63\x54\x6d\x72\xd5\xc2\x4a\xb1\x99\x98\x75\xff\xa5\x28\x1a\x76\x18\x40\x61\x91\xb5\x45\xab\xe0\x30\x50\xf9\xe7\xce\xd9\x30\x14\x8e\x63\x3f\x7d\xcc\x7d\xaa\x88\x12\xa8\xf4\x6f\x16\x72\xc7\x7c\xa9\xbf\xec\x2d\x66\x06\x05\x4d\x47\x3f\x22\x1b\x0b\xab\x4e\xb6\xf7\xa1\x3c\x72\xdd\xfb\x59\xf3\xce\xba\x64\x45\x9c\x7f\xf4\x22\xf0\x0e\xc4\x1c\x1f\x9d\xe4\x01\x06\x71\xe5\xe3\x41\xcc\x47\x1e\x3e\x15\x1c\x21\x84\x42\xdb\xd1\x60\xc5\x10\x91\xf4\xa6\x88\x06\x87\x10\xb1\x51\x94\xe3\x59\x41\x01\x7a\x3e\x0d\x41\xe1\x75\x78\x78\x20\x39\x7c\x29\x2e\xcf\x03\x2b\x96\x56\x22\x55\x0d\x5c\xd0\xee\xc2\xbb\xa4\x6b\x13\x99\x9c\xc5\xcf\xc7\x19\x5c\x90\xf8\xe1\x9b\x7b\xf0\x66\xb3\x73\xea\xf4\x55\xe5\xaf\x40\xbe\x4b\x09\xa5\x96\x80\x08\xed\xf2\xda\x63\x14\xe0\xae\xdd\x7d\x3d\x9b\x5c\xd8\xa4\xd9\xdd\x8e\x17\xa4\xa7\x5e\xc5\x99\xc1\x97\xb6\xf0\x65\x19\x43\x6f\x28\x1a\x01\x0b\x1f\xd6\x7f\x7c\x83\x58\x9a\x6e\x9a\xbc\xce\x78\xf6\x53\xc6\x8f\xa0\x81\xbb\x76\x0c\xb4\x1a\x8e\xc8\x2f\xe6\x6f\xb5\x8d\xac\xa5\x36\x3b\x06\xe3\xb6\xfe\x84\x6f\x50\x8a\x84\xb3\x2c\x92\x9a\x98\x54\xd2\x3b\x46\x2f\xdf\xc7\x08\x89\x6f\xd2\x55\xb0\x19\xa0\x0d\xd6\x61\xc4\x62\xa5\xb8\xdb\xa9\xe0\x03\x6c\x10\x95\xfc\xf0\x87\x5e\xde\x64\xa5\x79\x60\xf1\x67\xdc\xed\x4f\x94\x21\x98\x58\xb7\xe5\xf9\xee\xe3\x6a\xf4\xe5\x82\x0e\xfa\x46\x01\xc5\xe0\xeb\xaf\xbf\x18\x98\xe9\x7a\xcd\x3a\xbc\x55\x01\x3f\xfb\xf0\xd7\x20\xf5\xf0\x07\xfc\x60\xc2\xa1\x4f\xfa\x65\xe4\x08\x4a\x29\xf2\x7b\xbb\xdd\xf3\x04\xb9\xbd\x77\x42\x73\xce\x06\xa7\x97\x80\x11\xb8\x27\xf3\xf5\x82\x92\x6f\x35\x54\x7b\x83\x9d\xa1\x80\xe9\x1c\x97\x36\x19\x9f\x9d\xbd\x7b\x70\x72\x76\xb6\x39\x3b\x5b\x9f\x9d\x2d\xce\xce\x2e\x27\x89\xc1\x80\x34\xd3\x0b\x20\x83\x27\xf2\xa6\x4e\x91\xe0\xab\x28\x67\x18\x55\x16\x2a\xdc\x1e\x4f\xb6\xe3\x17\x50\xf1\xe4\xe4\x18\xfe\x9d\x02\x7b\xed\x25\xa6\x82\x3d\x7e\x83\x4f\xa1\xad\x8b\x46\x7d\xbb\xd0\x93\x5b\x7c\xa6\xea\x4f\x18\x58\x1c\xb6\x3f\xba\x46\x85\xb6\xc7\xf7\x5e\x7c\xb6\xed\xfc\x73\x32\x2a\xb2\xc4\xb8\x87\xdc\x8b\xe4\xec\x6c\x9c\xf4\x1c\xb8\xbd\xe4\x5e\x9a\xf4\x1a\xad\xf7\x92\x2c\x2e\x35\xca\x93\x9e\xed\xa1\x91\x97\x8e\x3f\x3f\x4b\x26\x59\xea\x06\xf5\x02\xff\x4e\xb2\x7b\xa3\xec\xec\xec\x8b\x2d\x74\xe2\x46\x03\xad\x6f\xe1\xff\x51\x0f\x50\x3a\x31\x2b\x32\x3b\x00\x38\xf3\x3d\x60\x51\x07\x69\xfa\xc1\xfe\xa2\x8c\x34\x03\x2c\x4f\x26\xdb\xa4\xe7\xbd\x6a\xaf\x28\xfc\xc2\xfc\x1b\xa0\x3b\xbb\x97\x6d\xfb\xf7\xa0\x36\x82\x66\xe8\xa4\x59\xf8\xc1\x82\x5f\x84\xf0\xf6\xa8\x8f\x17\xb6\x63\xed\x10\x9a\x89\x0a\x7e\x06\x13\x84\x6f\x28\xf1\xed\xfa\xa1\x36\xef\x99\xe8\x5b\x6a\xe9\xfd\xf8\xc1\xba\x69\x93\xf0\x4e\x7b\xff\x9c\xc4\x68\xc6\x26\x19\xcb\x7e\x6b\x82\x77\xc8\xf3\x88\x2e\xea\xcd\x9f\xc0\xcf\xf4\x00\x8e\x3c\xb0\xb8\x7b\xfa\x38\x0f\x0a\xff\x69\x1f\x51\x65\xe6\xd1\xf7\x0f\x9f\x3d\x0b\x4b\x63\xb0\xe5\xbd\xe5\x7f\x7c\xf8\xc3\x93\xb8\x38\xc5\x9a\x28\x88\x04\x46\x7b\x6a\x72\x26\xd2\x54\x66\xe0\xe8\x17\xb6\xd0\x56\x27\x5a\x50\xf7\x30\xe8\x33\xf6\xff\xf0\xf9\xf3\x5f\xf2\x08\x13\x8e\x86\x32\xf3\xf3\xb3\x27\xff\xf9\xf8\xa7\xb8\x84\xc5\xe7\xa3\xef\x9e\x7e\x1f\x21\x27\xa7\xe8\xb5\x5b\xba\x05\xde\x02\x5f\xdd\x6c\x17\x9b\x2b\xfc\xef\x18\x3f\xb2\xe3\xf4\xe2\xaa\x9a\xcf\xb6\xcb\xcb\x63\xda\x97\x99\xe2\xe3\xc9\x46\xbb\xef\xed\x72\x36\x03\x2a\x1c\xf7\x80\x7b\x64\xe9\xd9\xd9\xec\x5e\xb6\x68\x2c\x2b\x5a\x9c\x54\x22\xce\x80\x0a\xf8\x24\x53\x73\x1d\xd2\x02\x48\x2a\x18\x7c\xf0\x0a\xcb\x21\xe2\x1b\x23\xa9\x6d\x73\x07\x56\xf9\xeb\xf6\x15\x8c\x8e\xc7\xe6\x86\xba\x67\x34\x90\x0a\x7c\x6f\x96\x8d\x68\x10\x6d\x00\xa5\xa3\x62\xfc\x02\x46\xf1\x99\x80\xb6\xb3\xdb\x78\x71\x7f\x4c\x74\xbf\x01\xaa\x5f\x00\xb3\xbd\x37\x06\x50\xee\x1b\xb1\x5a\x2f\xee\xbf\x18\xbf\xb8\x9b\xf4\xce\xee\xce\xea\x7b\x48\x38\x98\x08\xd2\xc3\xac\x84\x22\xd6\xe4\x97\x1f\xb7\xfa\x13\x2c\x21\xe0\xc4\x80\x94\xf4\xec\x16\xfe\x05\xb2\x94\x04\x7a\xdd\x8a\xcc\xab\x0e\x3d\xb7\x25\xd1\x95\xf8\xd5\xad\xab\x72\x8a\xb7\x5c\xf7\x5f\x5c\x9d\xcd\x28\x81\x85\xb0\xe2\xfe\xe7\xdb\xb3\xb3\xfb\xaf\xcc\xda\x12\xd1\x7f\xdc\x2c\x91\x35\xdf\x3f\x2b\xa2\x81\xa4\xe3\x17\x9f\x27\xc0\xb5\xee\x65\x51\xc6\xd9\x04\x1b\x50\xb9\x0e\x6a\x9e\x01\xa0\xb3\xe9\xf1\xe5\xc3\xe3\x6f\x27\x77\x03\xf3\xe7\x5d\x58\x61\xb4\xed\x67\x50\xc3\x4a\x82\x4e\xc9\xf3\xd2\x70\x8a\x9c\x37\xae\xaa\x57\x57\x45\x72\xf2\x2e\xe9\x49\xf2\xf1\x9f\xbf\xfa\xea\x8b\x3f\xeb\x49\x06\xb3\xf1\xed\x07\xf8\x33\x92\x02\xf9\xc9\x29\x7d\x3e\x23\x1b\x6e\x8a\x39\xf6\x48\x82\xaf\xa5\x98\xd1\xa3\x06\xb2\x7c\x5f\xfe\xe9\xe9\xe0\x64\xfb\xd5\x57\x0f\xfe\xf2\x67\x33\x38\x79\xf0\x45\x17\xd3\xb6\x5f\xfd\xf9\x8b\x07\xe8\xb7\x88\xd6\x95\x9e\x48\xe8\x4b\x2f\x71\x8c\xb1\x3e\x2d\x1a\x94\x59\x6b\x90\x1a\x51\xa8\xd5\xc3\x88\xb3\xc3\x67\x21\xc1\xbd\x7f\x18\x5d\x0f\xab\x99\xfa\x78\xc2\xd6\x05\xbc\xed\x93\x22\xa0\x45\xf7\x11\x1d\xee\x76\x24\xff\x15\x72\x34\xe6\x8f\x16\x5b\xbe\x08\x6a\x12\x28\xf4\xf1\xf4\xc8\x0c\x9f\x2a\xc5\xc3\xb4\xaf\x0f\x87\xc9\xa3\xe4\xbb\xe7\x3f\x7c\x9f\x78\x8a\x24\x45\x8b\x9a\x0c\xe6\x9d\xc1\xce\x97\x3b\x15\x52\x3f\xc9\xc2\x8b\x95\x2c\xbc\x64\x51\x3c\xc2\x7f\x62\x08\x31\x2d\xf0\x12\xf0\xa0\x0b\x74\x5a\x7f\xe1\x28\x57\xde\xf9\x70\xd6\x9c\xc1\x51\xaa\x09\x58\x56\xc5\xe1\xa2\xa5\x64\x28\x22\x17\x84\x6c\xac\x9c\xa9\x08\xdb\xdf\xb0\xe1\xf8\x8f\xcb\x47\xb0\xe5\x92\xb9\x8e\x3a\x74\xe8\x10\xd1\x43\xe3\xce\x1a\x9a\xbe\xc5\x73\x48\xb9\x98\x3d\x42\x22\x4a\x29\x9a\x1b\x0b\xac\x5c\x3d\x4d\x12\x74\x56\xc7\x72\xad\xf6\xe9\xb8\xab\x8a\xe4\xb8\x73\x40\x78\x62\x57\x6b\xef\xd8\x5c\xb5\x00\x8c\xe2\xdc\x15\xc9\xd7\xcc\x4d\x4e\xbf\xbe\x2f\x3f\x92\xa1\xfa\xaf\xaa\x8c\x8c\x35\x90\x8b\x12\x98\x91\x25\xb6\xea\x39\xec\x53\xe3\xf6\x21\xed\x8e\x86\x2c\x77\x8e\x16\x6c\xfd\xe9\x40\x0d\x2d\xe6\x3f\x88\x2c\x1f\x6a\xf8\x3e\xa2\xd3\x41\xf1\xf9\x55\x35\x03\x11\xe1\xa8\xfc\x1c\xc6\x00\xc9\xa7\xcd\x3c\xcd\x49\x4c\x03\x97\x9e\xbd\xfe\xfe\xbc\x34\x29\x2d\xa6\x47\x69\x88\x8e\x0b\x0b\x3e\x14\x32\x0f\xc8\x30\xe7\x63\xda\xc9\x70\x69\x44\xa8\xd8\x8f\x05\x1a\xbe\xb5\xd3\xef\x9d\x98\x18\x1f\x53\xb6\xd7\xfd\x1c\x58\xa8\x94\x49\x70\xd4\x53\xc6\x46\x6b\x9e\x60\x84\x09\x1e\x1a\xc3\x6e\xbf\xa1\xe8\x07\xd8\xa7\xcd\xa0\x5d\x94\xc6\xca\x31\xd3\x57\x88\x58\xa4\xd5\x60\x8c\x82\xc2\xb6\xe4\x54\x7a\xd5\x91\x03\x8a\x1e\xf4\x0e\x96\xec\x9d\x68\x59\x7b\xaf\xe3\xd0\xf4\x14\xd8\xed\x86\x30\xd5\x09\x1b\x21\x8f\x12\xed\xcb\x82\xcf\x66\xea\xb2\xc2\x00\x95\x06\xe1\x07\xc4\x93\x7e\x83\x1e\x85\xe5\x57\x06\x3e\x85\xf8\xa6\x47\xf8\xb4\x6d\xf1\xf9\x9f\x18\xc5\x4c\x56\x0e\x4f\xec\xef\x2a\x0b\xc7\x25\x07\x2b\x07\xdf\x2b\x0a\xa2\x99\x25\x7f\x4a\x84\x76\xf6\xd4\x48\xf9\x3d\xdd\x6c\x97\x8d\xee\x76\xf9\x1d\x7e\x7c\xf4\x3b\xb9\xf2\x14\xef\x83\x8c\x9f\xaa\xfa\xf8\x7a\x58\x1a\xad\xfa\x4d\xdb\x04\x8c\x52\x5f\x6b\xe6\xed\x70\x56\x15\xc2\xda\x1b\x46\x46\xbb\x9f\x52\x03\x0f\xa1\x0a\x4f\xee\x9e\xf6\x39\x39\x55\x33\xbb\x23\xc2\x7e\xe6\x3b\x39\x8d\xc6\xd7\x93\x7c\x3c\x01\xc8\xbd\xab\xc9\x08\x4a\x1b\x0c\x61\xfd\x74\x56\x54\xb3\x83\x4a\xac\x61\xbb\xa1\xfc\x7e\xcc\xa1\x37\x04\xc7\x34\x80\xd6\x77\xbb\x2c\xff\xff\x14\xb2\x46\xe2\x59\x45\xaf\x71\x71\x97\xcd\x77\xcb\x1b\xdd\xb5\x3c\x6d\x4e\xe3\xec\xeb\x33\xe3\xd5\x8c\xf0\xee\xde\xd0\xfc\x9f\x3f\x03\xe4\xaa\x00\x90\x14\x5e\xd4\xc0\x0f\x8f\xa4\xb5\x14\x0f\xc6\xda\xc0\x51\xcc\x22\x12\x40\x74\x78\x76\x32\x4d\x70\xc1\x57\xec\xdd\xff\x9d\xdd\x22\x64\xb9\x79\x0e\x1f\x2c\x3d\xe4\x83\x16\x01\x3e\x3a\xe8\xaf\x06\x7d\x58\xc7\xdb\xbc\xbd\x67\x2b\x75\x6e\xd4\xa2\xe8\xc4\x0a\xa0\x1f\x6c\x9b\xfc\xde\xef\x51\xe4\x21\xf8\xf2\xec\x6e\xd5\x54\x87\xa5\xd6\x86\x41\x2c\xf0\x43\x35\x44\x0a\x84\x58\x48\xdd\x45\xf2\xac\x87\x58\x3c\x81\x17\x8d\xed\xb1\xdb\xfd\xdd\x28\xfd\x68\x7c\xd2\x80\xc9\x77\xc5\x06\x7e\x71\x60\x91\x22\xa1\xd8\xe7\xf3\x67\x81\xb3\x62\xc1\x27\x81\xe8\x64\xa1\x10\x4e\xe7\xd5\x4a\x0b\x7d\xc4\x60\xe5\x1f\xf0\x0e\xb4\x30\xe0\xb1\xd5\x57\xe3\xe2\xcc\x5b\x0d\x6f\x31\x4e\xf8\x7d\xe5\x64\x62\x7c\xf7\xc1\xc2\x3a\x65\xe3\x36\x1b\x7b\x8f\x91\x69\xe9\x27\xc9\x97\xcb\x95\x77\x0b\x55\xce\x8a\xcf\x71\xf3\xe4\x44\x4f\xf8\xa4\xad\xb4\xe1\xaa\x96\x8c\xb5\xda\xc4\x0a\x50\xdb\xad\xf3\x6e\x24\xda\x6a\x53\x32\x8e\xf2\x0b\x8e\x6b\xb5\xd5\xb7\x13\xb6\x55\x7d\x3d\x5d\x59\x2b\xcb\x2d\x08\xdc\x33\x52\x8c\x68\x0f\x5b\xf6\x10\x4f\xb2\x3d\xa0\x68\x8b\x07\x00\x71\x45\x80\x55\x7c\x14\x92\xe8\x60\xcf\x42\xb7\xca\xac\x47\x15\xa0\xe8\xfe\x7e\x8c\x54\x2f\x20\xdf\xa1\xa3\xdb\x8d\xa1\x40\x65\xec\xa4\x68\x62\xe4\x2c\x39\x4b\xb6\x9f\x7f\x7e\x60\x84\xe5\x82\x50\x75\x68\x84\x5a\xc4\x24\xb9\x62\x76\x6f\x7b\xf7\x4c\xfe\x0e\xf5\x7f\x51\x1b\xa6\x7f\x2f\x4f\x28\x1e\x97\x25\x3b\xb9\x5e\xb0\x7e\x90\x96\x04\x25\xa3\x50\xa9\x2e\x2a\x48\xeb\x83\x33\x96\xbf\xfd\xb0\x2f\xef\xb6\x3c\x7f\x53\x6d\xf6\x66\x2f\xf7\xe6\x5c\xd7\x51\x16\x52\x7f\xeb\xbc\xea\x40\x00\x27\xb0\x38\x17\x44\x50\x54\x57\x8c\x00\xe4\xb5\x1c\x94\xaf\xd9\x83\xdf\x34\xd3\xc7\x75\x07\x67\xd6\xc3\x99\xf4\x2e\x78\xeb\x14\x56\x61\x8d\xf4\xe5\x16\xb2\xa7\x29\x73\xb8\x66\x1f\xd2\x2d\x9e\x24\x43\x1e\xd0\x28\xad\xbd\x78\x35\xf4\x9a\x27\xe0\x05\x84\x12\xcd\xc9\x1c\x96\xf0\xea\x05\xe4\x2f\x3d\xa8\xab\x53\x9f\xdb\x03\xad\x0b\xe7\x14\xa3\x35\x91\x37\xf1\xd4\x19\xc1\x4d\xe3\xf3\x76\x0e\xe5\x6f\x56\xe8\xeb\x79\xee\x09\x76\xd6\xcb\x12\x7d\x2f\x6f\x56\xdb\x6d\x27\xed\xd0\x5f\xf4\xd1\x87\x1f\x9e\x1b\x46\x27\x9d\xf2\xfb\x99\x02\xeb\x28\xfc\x4c\xa1\x74\x96\x4f\xf7\xc1\xdd\xed\x0e\xfe\xdc\xdd\x9b\x4b\x95\x33\xdf\xad\x85\x46\x87\x56\x30\x6c\x7a\x71\x5e\x04\x50\xe3\x85\xce\x39\x0e\xd8\xfa\x43\x0c\x6d\x7c\x06\x77\x89\x56\x7c\x3a\x2e\xa5\x64\x80\x96\x51\xea\x5f\x04\xa2\x93\xf7\x09\xc8\x9f\x52\x12\xe0\xda\x3b\xe2\x03\xc8\x38\x80\x89\x2c\x1b\x0d\xba\x92\xbb\xdd\x4e\x83\x98\x8f\x03\x0a\x90\xe2\x25\x79\x13\xce\xea\x19\xbe\xec\xa5\x09\xf1\x95\x3a\x66\x9a\x8d\x8e\x07\xf9\xf9\xe1\x42\xe7\x19\x5a\xa2\xe6\x5f\x6a\xf7\x58\x65\xb0\x7f\x4e\x29\x3b\x9e\x33\x71\x2a\x26\x59\x68\x0a\x04\xe7\x43\xcb\x14\xe8\x27\xa0\x2d\xff\x74\x62\xce\xe1\xcf\x39\x39\x4e\x4e\x3d\x27\xe0\xa3\x06\xde\xc9\x32\x62\x4a\x74\x8a\x14\xe3\xcd\x12\x8c\xca\x1b\x20\x0c\x03\x4a\x51\x02\x99\x75\x51\x45\x84\x86\x88\x5a\xeb\x35\x3c\xa2\xc9\xd8\x0b\x1d\x91\xa7\x2d\xef\x09\x0d\xb3\xe9\xaa\x7f\xb3\x60\xe3\x02\x74\x67\xb6\xa5\xcf\x5b\x4b\x9f\xb7\x94\x1e\x92\x7d\x36\x42\x81\x7f\x81\x8c\x9d\x6b\x59\x35\x0a\xc1\xc1\x02\x86\x8a\xc1\xaa\x92\x5a\xfe\x64\xd1\xe0\xda\xd2\x61\xb4\x3b\x13\x62\x6e\x60\xc6\x80\xbc\x09\x3f\x0b\x6a\x97\x87\x53\x97\xcc\xca\x0d\xf0\x56\x5b\xa1\x2e\x82\x9b\x6f\xfb\xbe\x4a\xae\xbf\x76\x46\x14\x8f\xba\x91\xb8\x53\x04\xde\xa6\x97\xea\xa8\xa5\x42\x9a\x18\xda\x50\x26\x59\xae\xda\x10\xee\x12\xeb\x2b\x6c\xce\x6e\x58\xa1\x01\x31\x56\x67\x07\xad\xbd\x9a\xd6\xbd\x01\x4e\x28\x97\x5a\x28\x4a\x36\xda\x90\xa3\x51\xa4\xc0\x37\x49\xf1\xf9\x67\x03\x14\x04\x0c\x70\xc2\x3d\x1b\x69\x24\x3f\xea\x7e\x2e\x1c\x5f\x45\x07\xdd\x00\x38\x82\x25\xc2\xee\x6f\xfc\x2e\x35\xa3\xf8\x03\x12\xf1\x38\xdc\xd8\xdc\xb0\x91\x7e\xd7\xf8\x28\xce\xde\x2d\x91\x31\xd0\x77\xc3\x1f\x0c\x34\x44\xa3\x0d\x7a\x13\xc5\x54\xc1\x0e\xbd\x37\x10\x5a\x26\xcb\x9a\x15\xf0\xd3\x51\xd8\x9a\x7d\x3b\xea\xf4\xc4\xce\x9b\xdd\xd8\x9c\xc0\xee\xdb\x8a\x0b\x19\xb4\x47\x38\x71\xd6\x29\x1f\x17\x9d\xa6\x19\xdb\xc4\x82\x81\x73\x19\xd1\x0c\x9d\x3b\x08\xb9\x20\x87\xaa\x8f\xfd\xef\x25\x9f\x68\xd6\xe9\x50\x43\x5e\xf9\x51\x24\x0e\x93\x42\x67\x45\xa4\x12\xe3\xd0\xfa\xd9\x08\xb2\xb8\xb9\x3c\x6a\xae\xa9\x71\x1e\x35\x15\x21\x7c\x8e\x4a\xa9\x83\xf6\x53\x37\x95\xc8\x84\x1a\x9a\x75\x25\x80\x80\x7d\x09\xa0\x73\x32\xc2\x9f\x39\x34\xd8\xed\xbe\xc5\x98\xd2\x18\x59\xf2\xb2\x82\xd3\x1c\x7e\xb1\xbf\x32\x85\x39\x32\xbe\xeb\xb0\x43\xf3\x75\xfd\x0a\x23\xce\xae\x97\xb7\xe8\xe7\x76\x44\x41\x2e\xd2\xe4\xd9\x7b\x98\xa9\x77\x47\x54\xd6\x1c\xdd\x2c\xd6\xe5\xc5\xf2\x15\x5a\xb9\xcd\x8e\x90\xb0\xca\xba\xc6\xb8\xe6\x47\x49\x0f\xab\xdb\xa6\x9d\x31\xa2\x6b\xdf\xda\xf5\xd9\xc3\xf6\xcc\xf1\x29\x3a\x73\x0f\xe8\xd1\xc2\xea\x32\xda\xa6\xf7\x31\x37\x3d\xa1\x37\xb8\xa1\x5f\xbd\xf5\x3c\x3e\x21\x5b\x92\x92\xdd\x2e\x6c\xea\xf1\x00\x7d\xe2\x5e\x17\x0e\x2c\xe7\x00\x48\x8d\xd0\x7b\xf0\xb6\x53\xb6\xba\x72\x85\xf1\x59\xf8\x41\xd6\x38\xb9\x8b\x45\x92\xde\xf0\xe8\xe7\x1e\x75\x0d\x45\x20\x4f\x12\xda\x74\x6d\x60\xa5\x40\x6b\x80\xf8\x09\xa2\x1d\xf9\x61\x66\xb6\xdb\xbf\x04\x5f\x83\x41\x1c\x1b\xa9\x11\x82\x92\x1a\xc7\x35\x48\x17\xcb\x0b\x6b\xa2\x17\xa7\x13\x0a\x4a\xb5\x70\xf6\xb4\xb1\x84\x5b\xe7\x01\xec\xc7\x03\xc1\xb6\x7a\x85\x0c\x99\x47\x6a\x7d\x6b\xbf\x08\x00\xfd\xb2\x68\x04\x8a\x3a\xb2\xc3\xa6\x88\xff\x9e\xb7\x31\xe9\xb5\x88\xf8\x65\x26\x83\x6e\xe8\xba\x6c\xe8\x31\x46\xd2\x52\xb8\x1b\x36\x66\xfd\x75\x71\xe7\xc5\x32\xcb\xbf\x3a\x31\x12\x73\x83\x4e\x10\xb9\x6f\xd8\xca\x67\x91\xdc\x1a\x58\x90\xcd\x27\x3e\xbb\xa5\x06\x83\xf9\x5d\x72\x9a\xe4\x70\x78\x5d\xe7\xbe\x07\xba\x21\x1c\xd1\xfb\x3b\xc9\x51\x4b\x3e\x24\xf7\x6c\xf2\xba\x7c\x5b\x2d\x6f\x6a\x41\x5c\x50\xf7\x9f\xfb\x0a\xed\x76\xce\x1b\x31\xbf\x23\xd3\x08\xb7\x9a\xc9\x00\x2f\xf0\x1c\xb6\x9e\xca\x68\xea\x79\xd0\x32\x91\x4b\x7d\x31\x29\x24\xa4\xc4\x97\x93\xed\x96\x7f\x7d\x05\xbf\x92\x24\x3b\x5c\x3d\xf9\x67\x91\xa8\xb9\xeb\xf8\xc1\x44\xbd\x02\xb1\x41\xf4\x54\xd7\x8f\x1e\xbd\xbc\xc5\x46\xb1\xea\x60\xf2\x25\xf0\x10\x36\xd2\xf8\xe8\x91\x04\x2c\xdb\x24\x8b\xcd\x95\xeb\x7c\x60\x4d\x69\xcd\x17\xd9\xc8\x82\xb1\xdd\x06\xd1\x13\xd4\xec\x56\x07\xfe\xe5\xa4\xe8\xd9\x91\x8f\x74\xe0\x9a\xf4\x67\xa8\x3e\xc8\xf2\x07\xf7\x52\x7a\xc0\xd6\x75\x86\xed\x26\xcb\xd9\xcc\x4f\xd1\x20\x1b\x50\xdf\xb6\xf9\xdf\x26\x82\x82\x7f\x6b\xad\x90\xeb\x4f\x6b\xda\xdd\x0a\xe5\x4e\x6d\x5d\x62\x44\x11\x7b\x7d\x77\x01\x9c\xd9\xdc\x2c\x7e\x45\x61\x68\x26\x46\xfc\x00\x84\x58\x94\xc3\xac\x0c\x7d\x84\xd2\x16\x47\x78\x67\x79\xc6\xf6\x35\xc2\x1d\x23\x8f\x91\xf1\x40\xd1\xff\x25\x2a\xb5\xb9\x0f\x90\x96\xf8\xf4\xcd\x2d\x68\x32\xbd\xb9\x40\xd0\x38\xfb\x68\xcd\xa3\x70\xd8\x2e\x5f\x93\xdd\x03\xf7\x59\x62\x87\x20\xb2\xca\x31\x97\xcd\x8e\xa3\x74\xeb\x7a\xda\x66\x46\x6d\xa4\x92\xb1\xd0\xdb\xda\x51\x89\x98\x1c\xbf\x40\xdf\x4a\x7d\x1c\xee\x0e\x2d\x97\x82\x5b\x79\x52\x22\x0a\x65\xb2\x06\x58\x53\x47\x8d\x48\xd8\xe8\x3f\x6f\x2b\xd9\x72\x07\x17\x52\x44\xda\x07\xee\x58\xb4\x39\xd1\xe0\xb7\x47\x26\xf2\xc0\xc3\x90\x97\x6c\xfa\xd5\xd4\xcc\x32\x05\xad\x40\x7a\x29\xd7\x8b\xc2\x99\x9a\x8e\x6d\x09\x0e\x8e\x63\x83\x94\x50\x41\x90\xa2\xb4\x4a\xf4\xa6\x7f\xa8\x69\xcb\x92\x9e\xd7\x4e\x64\x7a\x44\x4f\xfa\xa3\xe9\xb3\xed\xd4\x53\x18\xb7\x0f\x5f\x3a\xf5\xc2\xcb\xdb\x1a\xf6\x0a\xa7\x21\x41\x7d\xf8\xde\x23\x4d\xa8\x19\x7c\xe1\x02\x03\x6c\x02\xba\x42\xf6\x4a\x2f\x28\xa9\x01\x26\x87\x23\x39\x14\x34\x4a\xde\x10\xf6\xc4\x5b\x4f\xaa\x0d\x9f\x03\xe4\xa2\x23\x54\x67\x61\x5c\x66\xe9\x22\xd7\x1f\x23\x91\xa3\x7a\x28\x25\x24\x41\x99\x91\xf4\x22\x9e\xa4\x79\xd4\xc4\xc8\x3e\x2a\x26\xd9\x2f\xc2\x6c\x4a\xed\x76\x4f\xac\x48\x64\xd7\x21\x0f\x2f\x4f\xee\xb5\x56\x68\x2d\x7c\x0a\x87\xda\xe4\xb3\x43\xe5\x79\x89\x1d\x53\xa2\x2e\x62\x07\xfa\x3f\xc3\xaa\xf8\x48\x63\x4f\x06\x8e\xbb\x46\x5b\x6f\xdb\x43\xc8\xc0\xe7\x62\xbc\x6e\x4f\x8c\xdf\x6f\x6f\x60\x7b\xee\x25\xc7\x49\x8e\xef\x20\x60\x60\x8e\x78\x27\x22\x87\x8f\xdb\xab\xe9\xc6\x86\xbd\xe2\x6d\xda\x90\x2d\x1d\x3f\xb8\x8c\x9e\xb4\x65\x41\xbb\x90\x18\x76\x78\xcc\x04\x5f\x54\xb9\x9d\xae\x67\x45\x82\x35\xa2\x02\xc7\x5f\x66\x66\x79\x49\x02\x5f\x22\xd6\x86\x38\x20\xec\xcf\x8f\xd4\x4d\x3d\xf2\x34\x61\x23\xa3\xd6\x65\xd1\x89\x83\x5b\x7e\x30\xce\x4c\x23\xc4\x0c\x49\xa3\x18\xf4\x8d\x7e\x51\xd8\x2e\x53\x6f\xa6\xeb\x0d\x9a\xa3\x17\x3c\x4e\x18\x80\x8c\x68\x94\x78\x62\x5f\xd2\x22\xcf\x30\x28\x45\x04\x16\x2d\x80\x82\x47\x7d\x90\x77\x99\x9b\x9a\xed\xd5\x8b\x0e\x80\xdc\xed\x76\xb8\x0e\x0a\xc3\xdc\x1a\x09\xb7\x0c\x95\x88\xfc\x00\xe6\x90\x7f\x5a\x79\x91\x25\x47\xfc\xc7\xc5\xa9\xe1\x86\xd8\xa2\x6a\x3f\xdf\xc4\x93\x95\xca\xd2\xf1\xe9\xbb\x33\x18\x12\x62\xd0\xa7\x0a\x5f\x70\x9b\xbf\x4f\x0a\x35\xf2\xe9\x50\x4e\xb7\x1b\xa0\x67\x67\x77\x04\x84\x99\xaa\x8e\x15\x8f\x3c\x1a\x4f\xbc\xce\x25\xc5\x9a\xd6\x4c\x94\x8a\xba\x5d\x45\x0a\x0f\xd3\x0b\xa5\xc3\x75\xfc\x60\x3a\x51\x0a\x85\xd3\x69\x84\xb7\x21\x97\x16\x0a\x08\x6b\xe7\xbc\x08\x43\xeb\xa0\x1f\x82\x86\xdb\x19\x4c\x24\xd6\xe0\xde\x12\x0f\xb8\xa1\xc2\xb6\xd6\xed\xca\x68\x9c\x7d\xe0\xd8\x66\x4e\x78\x7a\x7a\x3d\xaf\xb8\xbb\x6d\xa6\x19\x83\x81\x50\x97\x0e\xbc\x13\x8c\x72\x86\x28\x44\x37\x84\x34\xb3\xa1\x87\x82\x89\xea\x76\x7b\x3d\xac\xc7\x2d\xa9\xe7\xc0\x5d\x3c\x72\x8a\xec\x43\xbe\x24\x8e\xe4\xb1\xda\x44\xbd\x5d\xf4\xec\xa2\x78\xc7\xf7\xc7\x09\x87\x61\xac\xa2\xb6\xd8\x45\x99\xf5\x17\x6a\xa2\x2b\xf3\xd0\x38\x60\x27\x72\x3e\xe8\xa4\x7f\x1c\x21\x28\x18\x7d\x04\x8d\x77\x94\xc6\x3b\x31\x8d\x6f\xb7\x1d\x46\x1e\x74\xe5\x8d\x9b\x40\xf3\x47\x1c\x7c\x7b\x23\x76\x48\x25\x54\x66\x84\x5b\x51\x68\xc1\xc1\xda\xee\x7e\x98\x7b\x4c\x2c\x8d\xa9\x4a\x38\xdd\x76\x8b\x5f\xff\x95\x7e\x43\xda\x09\xda\xa3\x5d\x5e\xde\xa7\xef\xd3\xe2\x04\x9f\xb9\x8c\xe5\x60\x16\x43\x5d\x68\x42\xbe\x5b\x59\xbf\xc2\x00\xe5\x12\x7d\x8f\xaf\x89\xc6\xfc\x77\x22\xb1\x1d\x9c\x7f\xd1\x58\x25\x59\x1f\x45\xf1\xa9\x21\x81\x31\xb1\x66\xa2\x9c\x1d\x71\x05\xd4\x83\xf0\x2f\x67\x96\xe1\xa2\x5a\x8f\x2e\xbd\x00\x8d\xf9\xe5\xc2\x85\x71\xc2\xe4\xba\x90\x5e\xe5\x12\xcb\xc0\xfe\xae\xa5\x27\x26\x02\xb0\x7f\x35\xad\x7f\xba\x5d\xfc\x2c\xaf\x0f\xa5\x6d\xf0\x66\xa3\x4f\x89\xf4\x58\xcd\xde\xd9\x70\x80\x97\x52\xa8\x11\xdc\xd1\x8a\xdb\x12\x43\x6a\xf6\xae\x68\xc6\x30\x71\x5e\xb5\x19\xf9\x4e\x8e\xa1\x98\x1f\xe1\x91\x3e\xbd\x42\xbb\x6c\x8f\xdd\xd5\xa5\xec\x55\x27\x08\x48\x4d\xe5\x16\x78\xe4\xe5\xc9\xc3\x17\x6e\x37\xf9\xbe\x21\x6a\x44\x25\x1a\x1a\x5e\x22\xd3\xdd\xbe\x35\x29\x56\xdf\xf8\x22\x76\x5c\x6b\x8b\x6d\x90\x0d\xc3\xf8\x5b\x6e\x42\x3f\x8c\xe0\x03\xf1\xaf\x9c\xcf\xbd\x3a\x36\x53\x45\x52\xd0\xa2\x83\x3f\x86\x2b\xaa\x7c\x3f\x3e\x3f\xb6\x4c\x1c\xf2\x8a\x11\x1d\xa0\xd9\x06\x87\xc9\x0e\xed\xfd\x36\x48\x08\xe0\x08\x99\x12\x95\x50\x80\x28\xd5\x41\xa4\x1a\x3c\xd3\xb1\x16\xd8\xc8\x62\x76\xf4\x58\x7f\xfd\xc1\xb9\x38\x68\x2a\x16\xbb\xf0\x52\xa6\x53\x50\xef\x9c\xce\x78\x4f\x3f\x81\x31\x49\x6b\x1f\x69\xac\xd2\x12\xdd\x2a\x19\x1c\x3c\xa7\x20\xe9\x81\xaa\xca\x49\x99\xd4\x36\x06\x4a\x01\x30\xe6\xd3\xc5\xab\x3d\x20\x60\x96\x8b\xd3\xef\x1c\x74\xf8\x70\x82\xb9\x74\x9a\x38\xc4\x47\xa8\x75\xe0\x22\xd4\x14\xf5\x55\xe0\x3f\x7f\xe0\x98\xa8\xf4\xf6\x3d\x34\x33\x9c\x2d\x8f\x24\xac\x39\x7e\x16\xa1\xf5\x4c\xcb\x21\x08\x66\x3d\xc7\xfe\x93\x56\x3d\x74\xc2\x59\x39\xe5\x10\xc4\xde\x99\x94\x3a\xd0\x1f\x11\x88\x36\xbf\x28\x18\x2b\x27\xb2\x27\x53\x59\xc5\x3a\x66\xa1\x3c\x9e\x0d\xe1\x8c\x08\xb4\xe1\x69\x1f\xbd\xa8\xed\xec\x5a\x1d\xa8\x48\x33\x77\x87\x0c\x13\xc6\x4f\x32\xc5\x3c\x86\x5f\x6b\xa9\xaf\xd4\xef\x72\xbe\xbc\x90\x47\x3e\xa3\x04\x64\xb5\xd6\xde\x16\x7f\x77\xbb\xf8\xaf\x08\xed\x74\x72\x60\x2a\x9a\xed\xcc\x7a\xb9\x6c\x74\x14\xc6\xf8\x91\x8b\x6c\x7d\x13\xfb\x43\x65\x5b\x5f\xa2\x76\xc1\xf3\x11\xb6\x6f\xb1\x21\x77\x91\x65\x93\x68\xeb\xef\x74\x84\xe8\x49\xc9\x4a\x3f\xd1\xe4\x75\xbb\xfd\x27\x27\x4f\xcf\x49\x68\x80\xe3\xad\x18\xae\x1c\x32\x81\x55\x83\x16\x7a\x32\x05\xbd\x52\xeb\x4f\xaa\x82\xcf\x90\xb3\xfd\x4f\xdb\x5c\x58\xcd\xc8\x81\x43\x40\x33\xbe\xb3\x55\x7b\xc8\x81\x47\xad\x9a\xb6\x09\x5b\x50\xb5\x16\x52\x8b\xa6\x9d\xd1\x5f\x87\x86\xe0\xdf\xda\x47\x09\xb6\x25\x16\x15\x83\xc6\x65\xc4\xe5\xf5\x6a\xf3\x3e\x6e\xfe\xf7\x68\xd2\xf5\x35\x02\x1d\xcd\x69\xf2\xdf\x93\xed\xf6\x8b\x66\x90\xcd\x2f\x1b\xab\xa1\x19\xc8\x51\x4e\x63\xed\xc3\xee\xf8\x22\x52\x9f\x06\x20\xaa\x7b\xc3\x5e\x54\x7b\xb0\x25\x3e\x56\xfd\x96\x87\xfe\x76\x86\xa6\x6c\x5f\x45\x76\xe5\x6a\xaf\xc8\x4e\x5c\xad\x14\xf3\xfb\xa2\x81\x7b\x71\xb8\xbd\xa5\x11\x45\xe7\xc6\xc7\xd3\xde\xb5\xf2\x0c\xd4\xdd\x34\x5a\x3e\x00\x05\x74\x88\x4d\x05\xdd\x61\xf4\x49\xd2\xf5\xa4\x74\xd1\xd9\xc2\x58\xd9\xbe\x1c\x98\x2e\x16\x68\x9c\x46\x6d\x43\xf8\xb8\x16\xdd\x13\xec\x8b\x17\x1e\x2b\x21\x31\x70\x67\x46\xea\x8a\x0f\x56\x21\x19\x40\xa2\x75\x1b\x8d\x59\x21\xad\x88\x2e\x76\x80\x8d\x95\xbf\xfe\x9e\xa6\x4c\x1c\xa9\x7c\x7c\x72\xaa\x49\x23\xfd\xd1\x93\x80\xc7\x56\x0c\xde\xf1\xab\xbb\xbf\x0f\x76\xcf\xc1\x7c\x28\x72\x05\xfa\x94\x17\x0f\x32\xbf\xb8\x5e\xf9\x05\x32\x9f\xe4\x41\xf7\xcb\xd9\xec\x8f\xf6\x3e\xf8\xfd\xbd\xcf\x7f\xd7\xb4\x79\xb8\xf6\x70\xf0\x61\x74\x0f\x8f\x8f\x2b\x38\x62\x0d\x3f\x05\xc2\x57\xff\x6b\x21\xec\xf5\xaa\xaf\x45\x38\xfe\x04\x28\x77\x3b\xba\xcc\xac\x40\xea\xbd\xa3\xb7\x31\xf3\x8e\x28\x04\xcf\x97\xef\xf0\xf7\x25\x88\x1b\xf8\x17\x7d\x62\x6e\x97\xeb\x19\xfe\xae\xae\xa7\xaf\x30\x71\x97\x05\x47\x48\x10\xb3\x9b\x0f\x19\x54\x99\xeb\x80\x1f\xce\xc5\x16\x40\x68\x2e\x37\x07\x5a\x08\x9e\x21\x80\x26\xe4\x6c\x52\x58\xdb\x0c\xfe\x6c\x48\xd6\x1c\xdf\x5b\x43\xd4\x78\x41\x82\xf0\xa0\x53\xce\x83\xd8\x52\x98\x24\x51\xef\x83\x90\x10\xcd\xc0\xf7\x1d\x89\x7c\xef\x5e\x23\x81\x43\x3e\xfd\x6d\x09\x74\x93\xe9\xfb\x20\xc1\x51\x45\x3a\x92\xd3\x84\x17\xe1\x8d\x8a\xd2\x01\x91\x8b\xb8\x13\x95\x1f\xe2\x88\xe6\x50\xa0\xc8\xa3\x71\x04\x99\xc3\xb6\x11\xa5\x71\x04\xde\x4f\x0b\xee\x66\x2f\xf2\xb9\x6d\x31\x75\xd7\x5d\x71\xb1\xb9\x2a\xc2\x6d\xf2\x57\xdf\x9d\x03\x43\x9f\xd9\xa3\xba\x7b\xaa\xdf\xaf\x11\x9f\xe8\xe9\xd6\xc4\x7d\xfa\xce\xa1\x20\x30\xfb\x76\x31\xf2\xd4\x18\xfe\xb6\x8f\x71\x62\x64\x1a\x2e\x63\x9f\x1e\x6d\xbb\xf3\xf6\x33\xc7\x49\x9e\x4c\x0a\x2f\xc1\x42\x26\x69\x6c\xc3\x51\x34\x2c\x3a\x34\x7f\xd3\xb4\x6a\xd0\x2c\xf2\x01\xc5\xa0\x26\xbe\x03\xae\x89\x5e\x67\x2f\x22\x9b\xa3\x9d\xbc\x84\xc6\xee\x74\x70\xd0\xd9\x54\xf3\xe2\xfe\x7f\xe2\x1f\xf4\xf7\x66\x91\xa5\x46\xc5\x32\x3b\x7d\x4b\x02\xc6\x88\x7d\x0b\x9f\x54\x70\x4b\x26\xf8\xf7\x0d\x3d\x88\x8b\x8a\xf8\xfb\x2f\xfa\xe3\x17\xf9\x9f\xce\xc6\x67\x7d\x33\xb9\x87\xed\xf8\x4e\xf4\xc1\xf0\xf9\xe6\xcf\xcf\x36\xaf\x6e\xa6\xeb\x29\xfc\x2c\x67\x12\x74\xee\x8e\xf4\x96\xd0\x2f\x71\x0c\x3e\x45\xd6\xf8\x1b\x45\x35\xe2\x18\x00\x0c\x2e\xf1\x61\xf3\x99\xd4\xcb\xe0\x29\xbf\x48\x45\x81\x76\x1f\x28\x8e\x5e\x36\x62\x9f\x78\x36\x1b\x7b\x9f\xe4\xc2\x8a\xfc\xb6\x3e\x3f\x4f\x0c\x6b\xe3\xd9\x66\x7a\xf1\x46\x5e\x4e\x76\x9d\x09\x7d\xfa\x82\x01\x47\x53\x3a\x09\x63\xac\xc4\x0f\xe9\x63\x07\x68\xc6\x48\xef\xe5\xb9\xd0\xe5\x62\x18\x83\x36\x2b\xec\x34\xe3\xb5\xe2\x91\xa6\xf7\x4e\x20\xc7\x6e\xc1\xd1\xfa\x26\x1a\x45\x04\x36\xb6\x32\x18\x05\x74\x88\x9d\x64\x39\xfe\x83\x95\x2d\x49\x17\xfc\x54\xb0\x8d\xdc\x1c\x7c\x21\x2f\xcb\xe1\xc0\xdd\xb3\xfd\x93\x39\x08\xea\x2a\x3c\x6f\x19\x7a\x3c\x53\xe3\x7b\xd1\x57\x5d\xe8\x3b\x9b\xfc\x9a\x2f\x8d\x9a\x27\x86\xf3\x23\x2f\x4c\xea\xf4\x77\x62\x96\x67\x8d\x9b\x45\x8e\xe8\x21\x77\x67\x16\xfe\x71\xb2\xa1\x48\x89\x90\x06\xab\x67\xb1\xbc\xe5\x06\xed\x80\x3b\x83\x2c\xb3\x17\xd8\x7f\xb0\x29\x7a\x5f\xbe\xaa\xf7\x37\xd3\xe9\x78\xcf\xf4\xee\x7f\xeb\x76\x7d\x28\x60\x57\x44\xb1\x4c\xd0\x36\x34\x2c\xa9\x06\x52\x79\x80\x31\x03\xe1\x24\xb7\x64\x46\xd8\x0f\xaa\x39\xf5\x51\xee\xcf\x90\x5b\x0c\x9a\x0d\xc3\xba\x98\x2f\x6b\x00\xa4\x39\xb6\xda\x39\x2a\xd9\x08\xc4\x62\x14\x3d\xf7\xd7\xa9\x91\x45\xb0\x5a\xd6\xc5\x81\x01\xa2\x63\xc0\xde\xe5\x5c\xc7\xa3\xb7\x9d\x6f\xb7\x01\x1e\x72\x20\x2a\x21\x29\x35\x5d\x96\x95\x35\xa4\x87\x9f\xd0\x82\x39\x7a\x7d\x0e\x92\xdc\xcb\x63\x62\x53\x4a\xaf\x37\xa9\x59\x99\x46\xd8\xc6\xa0\xce\x82\x67\xb4\x76\xc6\xab\x55\x6f\x29\xc7\x16\xb4\x58\xc6\xb8\xc1\x11\x25\xc8\xde\x8c\xa6\xd2\x7c\x6f\xd2\x34\xaa\xde\xb5\xd3\x1d\x56\xb6\x4a\xf1\x3d\xcb\x7f\xc7\xd1\xa4\x0e\x1c\xd9\x47\xad\x36\x6e\xda\x5e\xf5\xff\x52\xf7\xee\xfd\x6d\x1c\x47\xa2\xe8\xff\xe7\x53\x80\xb3\x5c\x64\xc6\x18\x82\xa4\xec\x64\x93\x81\x47\x38\xb4\x1e\xb1\x76\x65\x49\x2b\xd2\x9b\xec\x05\x61\x1d\x90\x00\x49\xc8\x20\x80\x60\x40\x3d\x42\xe0\xbb\xdf\x7a\x76\x57\xf7\x0c\x28\x39\x27\xbf\x73\xee\xfd\xd9\x22\x7a\xfa\xfd\xac\xae\xaa\xae\xc7\xfc\x64\xb5\x1a\x7d\xd6\x4d\xa4\x2e\x75\x99\xe3\x57\x44\x99\x88\xd2\x7a\x4f\xaa\x3b\xc4\x1e\x83\x02\x84\x89\xf0\xfe\x2a\xa4\x8e\x76\x5b\x02\x56\xd1\x56\xb6\x1c\x50\x67\x69\x46\x8e\x7f\x51\xe7\x47\xf7\x1c\x3a\xc2\xca\x47\xe3\x86\xfb\x20\xd4\x49\xac\x50\x1a\x70\xe7\x41\x8a\xcf\x8a\xdb\x1e\x32\x8a\xdb\xd1\xaf\x13\x1e\x87\x3f\x97\x8e\xd1\xed\x0d\xe5\x6b\xd4\xb0\xf0\x4e\xe6\x46\x40\x9e\x6a\x2d\x13\x00\x4e\x0c\x66\x01\x48\xa1\xbd\x30\x03\xbc\x1b\xef\x1b\x5d\xb3\x11\x5e\xc6\x34\x4e\xb4\x23\xf9\x25\xf0\x83\x2e\x9d\x99\x2c\x0e\xa1\x39\xce\xdd\x6b\xf2\x9d\x57\x44\xdf\xb5\x13\x9d\x85\x0e\xca\x01\xa5\x3c\xc5\x9b\xd1\xc4\x70\x57\xf2\xc0\x5d\x7c\x33\x33\x84\x85\x60\x9a\x1e\xad\xbd\xe4\xcb\xca\xcb\x67\xcb\xbb\xa6\x9b\x55\xa9\x94\x05\x6e\x05\x55\xd9\xb1\x67\xa5\x33\xe3\xa9\x08\x85\x58\x19\xc1\xcc\x15\x26\xfc\x26\xe2\xe4\x4f\x73\xc2\x92\xbe\xaa\x26\xc9\xba\x65\x54\xa5\xb9\x27\xa2\xb8\x20\x85\xed\x93\x75\xb6\x65\xd4\xe6\x6b\xca\xc5\x6f\xff\xd2\x26\xec\xff\xaf\x9d\x80\x86\x96\x7f\x43\xe9\x1d\xed\xff\x43\x13\x68\x7b\x22\x79\xff\x07\x77\xe8\x1f\x5b\x8f\x58\x2c\x42\x17\x45\x66\xf0\x0b\x5b\x44\xe7\x39\x8d\x76\xe4\x66\x73\xbf\xcd\x0c\x23\x51\xc5\xe9\x1d\xb6\xfa\x55\xb5\x46\xdc\x48\x2c\xaf\x18\xee\x83\xe5\x95\xf1\x25\x63\x9c\x5e\xad\xe0\x23\xc9\xf8\x3d\x41\xaa\x08\xe5\xf3\x35\xf6\x2f\xa1\xcd\xc3\x22\x80\x38\x42\xbc\x1a\x51\x01\x92\xc5\x0b\x24\xb0\xd0\x21\x9e\x3b\xdd\x22\x0a\xef\x72\xd0\xdc\xe6\x21\xa2\x8d\xb7\xb4\x83\x8e\x4b\xc6\x70\xae\xe6\xb2\x0c\x0e\x23\xa5\x4f\xbe\xb8\x49\x42\x8b\xec\x01\x3b\xef\xe1\x98\x35\xf7\x10\x75\x27\x7c\x06\x24\xc8\x34\x17\x63\x28\x78\x97\x11\x2e\x6b\xb1\x08\xb4\xe7\xbb\x17\x53\x1d\x3c\xae\x1d\xf7\x61\x1e\x95\xb6\x74\x92\x19\x81\xf4\x05\x2f\x58\xf4\x41\xbf\x62\xf5\x86\xfa\xf5\x6b\x21\xa8\x27\x5c\x42\xec\xd1\xa9\x06\xa1\x9c\x84\x7f\x73\x83\x30\xc9\x72\x02\xdd\x99\x14\xf0\x91\x92\x5d\x95\x15\xdb\x6d\xd3\x37\x1a\xe7\x0b\xfc\x21\x9c\x82\x32\xe2\xdd\x4c\x1a\x33\xfd\x81\x7e\xa3\xe5\x8a\x26\x64\xc4\xf4\x28\x23\x6b\x99\xd1\xb1\x44\x3f\x60\x72\x30\x15\x87\xd3\xa7\x57\x64\x87\xdc\xad\x8c\xeb\x2a\xc2\x9f\xfe\x14\x21\x47\x30\x2e\x26\x47\xcb\xd2\xe8\x9e\xc7\x28\xd4\x66\xb3\x27\x37\x32\xe2\x3e\x40\xf4\x72\x21\x14\x72\xc0\xe1\x87\x15\xea\x7b\xba\xc3\x95\x72\xc1\x92\xb8\x1b\xc1\x8b\xf3\xd8\xc1\x07\x23\x80\x98\xfb\xb7\x04\xda\xd8\x68\x28\x6b\xde\x9b\x97\xf3\xe0\xd9\x80\x44\x61\x4c\xb3\x73\x91\xbc\x80\x8d\xc2\x4d\xcf\xfd\xae\x87\xa5\xef\xd9\xa7\x98\xaa\x4c\x46\x17\x17\xab\xcd\x68\xb5\x9e\x5e\xce\x26\x9b\x51\x35\x05\x50\x33\xba\x1b\x4f\x17\x9b\x8b\xf1\x74\x73\x39\x9a\x7f\x18\x55\xe4\x4f\x8b\xfe\xcc\xa6\xd5\x7a\x33\x9e\x00\x5d\x33\xab\x36\x57\xd3\xeb\xcb\x11\xbd\xc1\x60\xf0\x6e\x35\xd9\x5c\x2d\x16\xe8\x6c\x87\x5f\x09\x36\x37\xcc\x52\xc2\x87\xd4\xcd\xed\x04\x13\xe6\xa3\x0f\x9b\xc5\xdd\x1a\x2d\xb9\x2d\x57\x8b\x6b\x54\x2d\xd9\x54\x13\x1a\xed\xa6\xba\xbb\x85\x9c\x9f\x37\xeb\xe9\xed\x64\xf3\x01\xba\xb1\x48\xd0\x02\x1c\x0c\x71\xc2\x53\x5e\x1e\x0a\x34\x3a\x1f\x77\xc8\x98\x27\x5e\xbc\x1b\x34\xa9\x97\xa0\x71\xb6\xf9\xa2\xba\x99\xde\xb2\xac\x8e\x15\x4f\xfd\x9e\x0c\x69\xba\x01\xa3\xa5\xc2\xf3\xf3\xea\xf0\xf1\x50\x2c\xee\xad\xc4\xaf\xc5\x5f\xbc\x45\xd1\xc3\x5f\xce\xab\xce\x61\xbe\xfa\x74\xb3\xbe\x9d\x9d\x8d\xae\xcb\x43\xa8\x64\x8f\x0d\xce\xad\x36\x97\x8b\xd9\x66\x72\x7b\x01\xdb\xe3\x66\xb5\x99\xde\x5e\x6f\xd8\x34\x1d\x74\x94\x86\x39\xda\xc0\xf1\x1c\xdd\x66\x29\x9a\xb2\x2b\x86\x9d\x6c\xf0\xcb\xe3\xe1\x37\xd9\xf9\xe1\xe3\xc3\xeb\x69\xbe\x12\x43\x19\x58\xa5\xa4\x43\x43\xeb\x8b\xc5\x18\xc6\xf7\x3d\xfd\x92\x2d\x3b\x68\x18\xbe\x37\xed\x7f\xe9\x9f\x7f\xec\xf4\x90\xa7\xb1\x78\x81\x6f\xd8\x92\x00\x63\xaa\x2e\x57\xd3\xe5\x7a\x53\xad\x3f\xc3\xba\x61\xe3\x19\x14\xbc\x1d\xcd\xa7\xcb\xbb\x19\xbd\x6a\xbe\x5b\x11\xbf\x13\x1f\xe7\x48\xe2\x90\x38\x2a\xca\x02\xdd\x10\x53\x94\x0d\xe7\xc9\x8b\x5a\x79\x28\x81\xf3\x0a\x75\xe5\x07\xbf\x94\xc3\x4d\x09\x61\x7d\x71\xeb\x62\x03\x2b\x6e\x56\x2a\xdc\xdf\x9c\x1f\x42\xd6\xf7\xa3\x0f\xa3\xcd\xe4\xf2\x76\x94\x71\x6a\x98\xef\xa7\x51\x45\xb5\xff\xb2\x5e\xdd\x4d\x20\x7f\xf7\x1b\x1c\xf2\x25\x5a\xca\x3a\xa5\x4c\x34\xdf\xdf\x7c\xbf\x87\x0a\xfa\x83\x27\x4f\x4f\xce\x4e\xce\x07\x9b\x83\x83\x6c\x83\x11\xc3\xf3\x21\x86\x1f\x43\x8e\x7d\x58\xe6\x8f\xab\xd1\x12\x9d\x4a\xdd\xf3\xe3\x5f\x31\x38\xce\xd5\xce\x82\x33\x3c\x5e\xfe\x4e\x43\xbf\x7b\x0c\x8b\xec\x2d\x2c\x0c\x81\xc0\xbf\x06\x78\xc7\xa5\xae\xa6\x93\xd9\x18\x10\x5d\xce\xe3\xbf\x86\x39\xae\x34\xe7\x81\x0b\x84\x93\x29\x30\xcc\x69\x69\x39\x89\x7d\x3d\x73\xaa\x86\x91\x7b\x02\xbb\x89\x33\xac\x71\xe6\x39\x5d\x82\x90\xbc\x2a\x06\x8f\x5c\x1a\xaf\xb7\x64\xa1\xa0\xc9\x0a\xdb\xac\x21\xaf\xcb\x08\xc9\x74\xbc\xb8\xb4\xfb\xb2\x6d\x41\x3f\xbe\xad\x95\x5f\xaf\xa4\xbd\xd5\xe3\x86\x46\xdf\x01\xe4\x1b\xc1\xe4\x29\x08\x56\xbd\x2c\xdc\x76\xa7\x93\xd5\x14\x00\xc0\xdf\x81\x9c\x38\x42\xc9\xa5\x24\x19\xd2\x40\xff\xfa\x3d\x9b\xd3\x4a\xc4\xae\xd6\x10\x40\xd9\xe8\x6a\xf2\x7c\x35\xba\x26\xdb\x72\xcc\xf9\x3e\x35\x51\xce\xee\x5c\x96\x5f\x49\xd4\xd3\xe9\x87\xd2\x96\x8a\xcd\xc3\xf1\xc3\x37\x57\xf5\x8c\x79\xba\x29\x9b\x11\xc8\x7a\xb2\x27\xba\xb0\x25\x98\x79\x6d\x22\x50\xb7\x47\x3f\xf9\x94\xb9\x2f\x04\x56\xee\x4b\x27\xd0\x47\x30\x64\xf3\xd9\x71\x61\x7d\x55\x37\x3e\x61\x9c\xd7\x19\x81\xe1\xd3\x22\x9b\xd5\x88\x50\xaa\xd1\x25\xaa\x23\x08\x7a\xd2\x9c\x53\x8d\xef\x78\xa3\x2d\x86\x2f\x9b\x7a\xfa\x54\x5e\x70\x33\x99\xb4\x34\xad\x11\xad\x91\x02\xa5\x9b\x7f\x99\x51\x64\xec\x92\x42\x22\xb7\x8f\xd8\x34\x0a\x04\xd1\x97\x7b\xb8\xa9\x9c\x75\x76\x9a\x86\x00\x5b\xc7\xfd\x41\xdc\x05\xc7\x1d\x7e\x1e\x24\x65\x96\x0a\x24\x92\xcc\x1a\x85\x54\x56\x1c\x31\x74\xa4\x6e\x2a\xc7\x22\x67\x34\x45\x53\x94\x72\xea\xa9\x6d\xe8\xa3\x21\xe3\x7a\x98\x59\x39\x79\x58\x20\x6f\x1c\x70\xd6\x9d\xfc\x2d\x3d\x82\xc1\xce\x16\xf3\x49\xba\x77\x94\xf5\xea\xc4\x7c\xbb\x8d\x75\x85\xb6\xe0\xb4\x29\x1a\x2f\x21\x93\x11\xeb\x4f\x05\x6f\x08\xbf\xeb\x45\xc8\xb5\x91\x5a\x31\x76\xcd\x3c\x33\xa6\x51\x3c\xc0\xf0\x38\xb6\x6e\x3d\x69\x62\x2c\x3f\x85\x57\x80\x2e\x84\x78\x0d\xc2\x4d\x16\xaf\x43\xff\x2b\x17\x80\xaa\x6e\x5c\x82\xa2\xa1\x06\x65\x64\x38\x3a\x9c\x6b\x72\xb4\x05\x52\xfb\x57\x4a\x10\x54\x29\x3e\x7b\x71\x50\x31\x45\xf7\x6d\x17\x3f\x2b\xa8\x98\xcc\x01\xc5\x6c\x65\xef\xc5\x83\x26\xce\xae\x1b\x6a\xb9\x63\xf0\xbd\xdf\xbe\x09\x7d\x15\xfd\xda\x64\x14\xae\x4b\x77\xf3\xb0\x53\x11\xcf\x95\xf6\x18\x9c\xce\x78\xd2\x62\xda\x8a\xea\x4d\x10\x48\xa1\x2c\x56\xd0\x1f\x91\x0d\xfb\xcb\x74\x7d\xc3\x6c\x1a\x4b\x31\x41\xcd\x30\x41\xc8\x85\xa1\xa9\xda\xd5\x8f\xf1\xe2\xf6\x27\x44\x0f\x9c\x80\x6a\x95\xe3\x3b\x68\x48\xf6\x91\x60\x35\x65\x8f\x34\x4c\xa3\xb8\x3f\xc5\x51\x19\x03\x9b\x00\x6e\x33\x89\xca\xd4\xfd\xff\xfd\x9e\x05\x47\xdb\x71\xf8\x02\x6a\x18\x85\x51\x28\xfd\xb7\xf4\xf4\x38\xee\xa9\x59\x75\x86\x2c\x51\xc4\x8e\x9e\x50\xf3\xa3\xab\x80\x0e\xfb\x3f\xd7\x7a\x40\x52\x60\x4f\xd8\x7a\x64\x03\xf7\xf2\xd7\xc9\x64\xf9\x14\x68\x81\x48\xd6\x15\x9f\x41\xac\x93\x24\xe1\x56\xb3\x8f\xa4\x74\xcf\xbb\xf2\xdc\x41\x2c\xc7\x06\x0b\x90\x9e\xd5\xa6\x98\x10\x9b\x84\x52\x50\xfa\xcc\x82\x88\x24\x66\x4a\x51\xbe\x46\x0c\x31\x64\x2c\x80\x18\xcc\x83\xab\xad\xdd\x8e\x5f\x68\x28\x6f\x70\x65\x30\xe9\x45\x86\x05\xfe\x3c\x5b\x5c\x8c\x66\xcf\xd0\x16\x80\x69\x21\x4f\x18\xc3\x4d\xea\x4d\x05\x86\x37\xb9\x37\x01\xec\xa9\x89\x8c\xfd\x96\x99\xa4\xac\x0d\x76\xe6\x1e\x9a\x0d\x7a\x22\x8a\xaf\x27\xbe\x80\x6a\x7d\x0d\x8c\x9e\xf2\xbc\x10\x22\x54\xb9\x26\x22\x5e\x10\x2f\x60\x92\x39\xa3\xca\x9c\x5d\x2d\x9e\x1e\x45\x17\x17\xdd\xc1\x7e\xe8\x48\x54\x9e\xcc\xc7\xcf\x3e\xd0\x6e\x1e\xcb\x0a\xb9\x18\x6f\x87\xd4\xc6\x8a\x3f\xb0\x20\xae\xbf\x77\x5c\x7c\xa1\x32\x2d\x16\xc7\xf7\x83\x72\x45\x2d\x9d\xcf\x47\x74\xfd\x87\xf7\x2c\x63\x16\x04\xc0\xbf\x34\x22\x3c\x5b\x78\x73\xfc\x6f\xa0\x87\x01\xda\x01\x08\x0a\xf2\x06\xeb\xcf\x55\x88\x2c\xd5\x30\xc8\xcc\xa8\x99\x05\x3b\x88\x59\x79\xce\x30\x9b\x97\x46\xb6\x24\x38\xe0\xfc\x99\x37\x71\x49\x22\x24\xf5\x97\x6b\x6a\x73\xb3\xb1\x14\x2b\x73\xa9\xb8\xf7\x8e\x85\xd2\x4c\x5c\xb4\xdb\x96\x92\x7f\xb8\x60\x8d\x72\x87\xc2\xb5\xb8\xb0\x0a\xc1\xda\x07\xa9\x12\xe2\x80\xb2\x4f\x2e\x5d\xf2\x80\xe9\x9a\xac\xa6\x74\x3e\xcc\x70\xda\x71\x32\xd9\x51\x91\x9b\x1e\xe5\x11\x00\xf9\xb3\x7f\x0c\xa4\xd4\xfe\xa3\xc7\x49\x46\x76\xd4\x49\xbd\x47\xdf\xf0\xec\x41\xa6\xe5\x6a\x38\xc2\xe9\x17\xcf\x70\x1e\x2e\x12\xf7\x85\x8f\x69\x79\x64\xac\xc3\x6c\x99\xff\xd3\x44\x1b\x70\x91\x2f\x21\xf8\x06\xdf\xa8\xed\x53\x8f\x6c\x35\x20\x5a\xd2\x23\x55\x39\xa0\x04\xff\x1c\x5a\x46\x7b\x44\xf6\xa7\xa0\x3a\x5c\xb6\x8b\x7c\x45\xc6\x7a\x90\xd1\x04\x23\x52\x2e\xa6\xbb\xfe\x06\x94\x73\xd8\x80\x24\x10\x6f\x0b\xa5\x3e\xe2\x2b\x4d\x95\x13\xa3\xcb\xb0\xa7\x0f\x3c\x69\x84\x6e\x21\x50\x4c\x33\x29\xd5\x70\x65\xce\xc9\x45\x13\x9e\x65\xee\xe4\x97\x5e\xbe\xa4\x46\xfb\xd2\x0e\x45\x65\x40\xc6\x2a\x17\xaa\x30\x11\x35\x9e\x23\xaa\x79\x31\x42\x9d\x63\xd2\x1c\xba\x84\xa6\xdf\xc1\x85\x05\x8b\x2c\x2e\x9c\x06\x43\x56\x9a\x21\x86\x1e\x2b\xa9\x92\x42\xe7\xcd\xa8\x62\xbe\x4a\x95\x57\xf2\x8b\xe6\xdd\x95\xd2\x6e\x78\xd7\xae\x84\x43\x9d\x4f\x5f\x2d\x9e\x20\x24\x2b\x67\x07\xc7\xbc\x39\x4a\x6c\x02\xf9\xb4\x0f\xa2\xd7\xb2\xea\x00\x0f\x7c\x24\x1a\xa1\x3b\x7e\x5c\xce\x1a\x1e\xc3\x65\xf5\xa3\xd3\x4c\x6c\x26\x6a\x1e\xee\x12\x65\x4a\xd9\xd3\xfb\x10\x11\x49\x22\xee\x9e\x0c\x81\x11\x21\xcd\xc7\xd1\x3d\xdf\x29\xb4\xdc\xc9\x03\x92\x73\x6c\x10\x7a\x12\xf0\xa6\xb9\xef\x13\xe1\x81\x07\x3b\x35\xe0\x2e\xa3\x17\x81\xab\x00\x0b\xab\x2d\x16\xd3\xa8\x33\x68\x47\xe7\x5b\xa7\xeb\xe2\x0e\xae\x54\xc7\x01\xe1\xb2\x4d\xc4\x2a\xa2\x74\x4c\x3e\xb1\x42\x9c\xd6\x63\xdf\x7e\x48\x49\x58\xe3\x3d\x29\xe0\xec\x61\xfa\xc6\xa9\x90\x54\x95\x89\x4b\x6d\xec\x71\x49\x7f\xeb\x77\x3a\xef\xa3\x64\xbd\x4a\x32\xdd\x3d\xf6\x31\x45\x00\x92\xdb\x4b\x0e\x09\x52\x55\x01\xde\x79\x99\xd9\x84\xa5\x54\xe3\x1c\x3d\x09\x44\x24\xbd\x47\xbf\x29\x01\xbf\xd3\xdd\x87\xb2\xd3\x98\x18\xdc\xad\xb4\xb5\xe1\xc0\xa3\xd7\x62\x5f\xb9\x1b\x00\xbf\x27\xe9\x7e\x97\x6e\x52\x19\x8f\xa7\x65\x6e\x99\x64\xd9\x9b\xa7\x40\x05\x98\x12\x4a\x4f\xb2\x3e\xbe\x43\xbc\x5e\x9d\x78\x5a\x9c\x53\x99\x50\x2b\x34\x82\x1a\x9b\x66\x62\xdb\x48\xfa\x47\xb2\x1b\xe8\xd2\x41\x7a\x36\x08\x27\xe3\xe0\x38\x5e\x7c\x33\xd7\x3a\x98\x15\x9c\x00\x38\xfc\x3a\xb3\x88\x1d\xfa\x06\xec\x64\x6a\x1b\x28\x7c\xe5\xd9\xaf\xf2\x42\x84\x78\xa9\xc8\xc1\x23\xae\xa6\x57\xe9\x3b\x44\x57\x64\xa2\xae\x1d\xa2\x8b\x39\x62\x1c\x19\x61\xc8\x9c\xb5\x71\xb8\xb6\x6a\x75\xa9\xfc\xa8\xd1\xfb\xd1\xa7\xf4\xfe\x6e\x35\x2b\x34\x85\x8d\xad\x27\x7f\x7e\x76\x96\x10\x46\x74\x46\x9f\xb2\x14\xf9\xa8\xfa\x3c\xbf\x2c\x60\xa3\x73\x93\x18\x4a\xc8\x2c\x55\x95\x90\xf0\xac\xb2\x1f\x7d\x8f\x52\x19\x01\x09\xca\xb8\xa0\x53\xf2\x9a\x33\x61\x23\xf7\x62\x64\xbf\xc6\xb0\x9a\x11\x81\x41\x89\xb6\xe0\x78\x10\x5e\x18\x60\xa9\xe6\x81\x8c\xc4\x07\xf8\xee\x3c\x5b\x14\x09\x87\x12\xa5\x68\x31\x4a\x82\x49\x6e\x6f\x89\x22\x61\x42\x52\x63\x4f\x88\xae\x4b\x88\xbc\x4b\xf4\x7a\x45\x16\x5a\x62\xae\xda\x24\x7e\xfb\x5c\xac\xa6\xd7\xd3\xf9\x68\xf6\xc0\x0b\xa8\xbf\x6e\x2c\x0d\x51\x89\x79\x6b\x16\xda\xa3\x1e\x94\xb1\xa0\x20\xe9\x08\x94\x9c\xe8\xf6\x63\x0f\x23\x1f\x97\x1e\x69\xa9\xca\xa9\x9a\x21\xc0\xa9\x61\xf6\x8f\x63\xa4\xa9\x68\x0d\xd7\x82\x44\xca\x40\xfb\x3c\xe4\x37\x3e\xe4\x02\xc1\xcd\x15\xb8\x1e\x14\xe7\xc1\x2c\x6b\xb2\x4b\xce\x64\xc7\x4b\x65\x44\x3f\xf0\xe3\xdf\x97\x88\x08\x72\xac\x02\xc7\x40\x38\xc7\xac\xfc\xcd\xa4\x48\x3e\xcd\x61\xbb\xba\x84\xe9\xfc\xcd\xe8\xda\x83\x9e\x2f\x11\x88\x3d\xcf\xf2\xb4\xd8\xec\xef\x09\x8e\xb9\x1b\x4e\x65\x65\x19\x49\x81\x8b\xb1\x8e\xe2\x26\xdf\x27\x9d\x89\x55\x31\xe9\x24\x80\x47\xf6\xa9\x8b\x8c\x27\x52\x90\x98\xb4\x7b\x68\xd2\xd4\x30\xce\x0d\x42\xc8\xfd\x44\x9d\x77\xfc\xb4\xdc\xf5\x80\xd0\xe3\x6a\x6d\xaa\xa1\xfa\xd0\x9a\x62\x34\xa6\x39\x03\x66\x9a\x4e\x07\x19\xa2\xc4\x27\xaa\x97\x55\x23\xd5\xad\x79\xc3\x1a\xf9\x1e\xce\x4d\xc6\x00\xd3\x2c\x55\x29\xb0\x9c\xba\x9c\xd9\xb5\x2a\x2d\xbd\x6f\x89\x66\x06\x86\x3e\x23\xd1\xce\x9d\xce\x34\xb3\xf5\x42\x64\xbb\x7d\x35\xfd\xf4\x44\xa7\xf5\x45\x55\xdd\x4d\x2a\x06\x85\x51\x46\x5a\xe6\x60\x93\xa1\x98\x67\x7d\x9f\x61\xdf\x6d\x07\x4d\x98\xf4\x4e\x7d\x6f\x83\x11\xda\x0f\x97\x4f\xc6\xfb\xa5\x61\xe1\x31\xa5\xac\x4f\x16\xcb\xcf\xd4\x8d\x1d\x23\x20\x8b\x03\x51\x4e\x3a\x3c\xdc\x90\x53\xd1\xdf\x35\xf5\xf6\xaa\x37\x79\x8c\x1f\xe3\x90\x55\x12\x1c\xb7\x3d\x3e\x56\xed\xf6\x0e\x06\x4a\xd0\xaa\x9d\x41\x36\x2d\x81\xa4\x0a\x75\x02\x75\xc5\x0c\x16\x15\x82\x01\xaf\x88\xad\x37\x27\x83\x3a\x48\xf7\xc0\xf1\x7d\xce\x83\x56\x27\x9e\xeb\xdb\x65\x8e\xd6\xf4\xe9\x36\x27\xde\x72\x3e\x0b\xa4\x18\xe8\x2d\xab\xe9\x0d\xcb\x19\xba\xc4\x3e\x56\x4e\x38\xda\x8b\x01\x3b\xfe\x3e\xdd\xc8\x18\xf0\x4a\xb0\x98\x9e\xf0\xab\x21\xaa\x8d\xe9\xb3\x0e\x1c\x0c\x39\x07\x01\x5a\x43\x2d\xe4\x21\xb9\xce\xec\xb2\x82\xa1\x90\x1a\xad\x30\xc4\x36\x57\xc3\x08\xdf\xed\xb2\x84\x7f\x9b\x0d\x0e\x25\x60\xd0\xaa\x91\xcf\xc6\x77\x35\x9c\x97\x32\xa2\x93\x05\x80\xed\x24\x93\x69\x06\xf5\x69\x6c\x00\x45\x87\x8e\xe4\xee\xea\xdb\x22\xce\xb9\x01\x59\x98\x0c\xf5\x74\x84\x1b\xf5\x10\x59\xdd\xa1\xbc\x8f\x86\xf9\x7b\x2e\x75\x34\x64\x7b\x8d\x32\x40\x6f\x3f\x85\x38\x13\xff\x38\xb3\x40\x78\x7f\x34\xed\x22\xbd\x11\x4c\x94\x7b\x2e\x6b\xa8\xc0\x4f\x13\xbe\x1c\x01\x2c\x8d\xba\x41\x1b\x2d\x73\xea\xa1\x82\x64\xee\x91\x63\x89\xcd\x86\x25\x01\x4c\x2f\xfa\xee\x1d\x19\xb2\xc8\x54\x35\x65\x3b\x2a\x60\x02\xf0\x9f\x25\x13\xde\x8b\x1c\xc8\x24\x94\xb1\x52\x4c\x9c\xe6\xae\x86\xfd\xd2\x23\x69\x54\x02\xed\x5e\x2a\xc2\x0b\xb8\x23\xb7\xde\x40\x77\xd4\x38\x8a\x3c\x5a\xd2\x20\x68\xd8\xd1\xd8\x5d\xf3\x90\x41\xdf\x06\xad\x2b\x93\xa4\x17\x8e\x88\x96\x3a\x6c\x20\x48\xcf\x30\x3f\xbd\x23\xfb\xbd\xc0\x16\x5d\xbe\xbc\x96\x7c\x60\x7a\x72\x60\x00\x9c\x61\x2d\x51\x53\x0e\x1b\x71\xf6\x5f\xe5\x30\xc9\xd5\xa7\x68\x2b\x6c\xe3\xd4\xd0\x20\x55\x2e\x7a\xa4\x48\x88\x7d\x7a\xca\xc7\x40\x0a\x31\x88\xa7\xbd\x40\x59\xc5\x93\x08\x5a\x4e\xde\x73\xf0\x6b\xb3\x39\x38\xf6\x60\xc2\x4a\x2e\x1b\x18\x87\x88\xb9\x53\xbc\xf9\x5a\x04\x86\xbc\xa1\x48\x57\x6b\xe0\x81\x73\x78\xc0\xaf\xb5\xc5\xb0\x9e\x66\x46\x80\x2e\xdf\xe0\xef\x75\x4c\x90\x24\xae\xfd\x6b\x24\x09\x75\xc9\x90\x24\x4a\x1a\x35\x7a\x4b\xe1\x4b\x00\x7b\x88\x2c\x64\xe1\x8f\xc5\xf0\x1f\x79\xa6\xcb\x75\xc3\xdb\x04\x59\xf5\x9a\x8e\x09\x51\x24\xcc\x78\x3a\x47\x8b\x72\xa3\xd9\x7f\x4c\x3e\x1b\x05\x21\xd4\x4f\x13\x8b\x49\x3a\x7f\x64\x27\x6b\x0c\x95\x00\x78\x14\x2f\xcf\xd1\x16\x08\x12\x73\xb2\xc8\x3b\x72\x82\xd5\xa8\xca\xba\xee\x4a\x64\xc0\xd4\xd7\x8b\x21\xd3\xeb\x22\xf5\xbd\x77\x1b\xc9\x47\xc9\xf6\x84\x25\x46\x5f\x68\x28\xca\x66\x86\x30\xa4\x81\x95\xd3\xb1\x5a\x1e\x9a\x8e\x87\xec\x3e\x1c\xe3\xb9\x13\x8c\x9d\xe0\x44\xb4\xa6\xcc\x4e\xd7\x78\xe9\x1d\x9b\xf2\xe9\x07\x1d\x17\xae\x95\x9b\x43\x47\x98\x71\x82\xc1\x22\x68\x86\xa9\xd6\x1b\xb2\xa0\x2c\x0a\x7b\xd8\x13\xe8\x33\xcf\x51\xcb\x45\x85\x33\xda\x97\xe4\xda\xb0\x0a\x6b\xdc\x8f\x9b\xb4\xf6\xfd\x88\xb0\x08\x7c\xda\x34\x65\x4c\x4d\x8d\xac\x63\x18\xb4\x21\xd8\x05\x56\xc5\xdd\x18\xbf\x18\xab\x6e\xe9\x18\xa5\xcd\x45\x62\x8e\x45\x5e\x91\xb3\x70\x8a\x12\x57\x15\xca\xf2\x3d\x39\x3d\xcd\x57\xa3\xd9\xf2\x66\x54\x1e\xd2\xcf\x79\x3a\xf8\x25\x1b\x7e\x73\x4e\x92\x52\x0b\xb8\x0f\xa6\xeb\xcf\xe5\xa1\x04\xce\xab\x6f\x50\x9c\x8a\xb3\xa0\x08\xd4\x52\xf4\x67\x51\x32\x6b\xbd\x58\x6e\x80\x70\xba\x59\x6f\x2e\x16\xeb\xf5\xe2\x76\x33\x9b\x5c\xad\xc9\x09\xea\x78\x5a\xc1\x9d\xf8\xb9\xfa\x08\x17\x2b\x64\x9c\x23\x61\x41\x57\x42\xda\xdf\x3b\xb8\x1c\x4c\x46\xc3\xac\x4b\x42\x64\xb7\xa3\x15\x10\x5e\x90\x87\x03\x28\x33\x76\x77\x8b\x26\x8c\xd7\x65\xcd\x17\x2f\x91\x64\x90\x8c\xfe\x9d\xbb\xdf\x64\xfb\x2a\x0b\x07\x51\xd0\xc0\xf2\xd3\xc3\x25\xfa\x7b\xcb\x4f\xd9\x60\x74\xf0\xf7\x7f\x1d\x76\x5c\xd1\xd5\x64\xf6\xea\x2e\x72\xe0\x4c\x7e\x70\xb3\x32\x2a\x2f\x25\x70\x29\x64\x70\xe5\xfd\x0f\xaf\x9f\xfe\x37\xd0\xcc\xb3\xc5\xe5\xaf\x40\x05\x5f\x56\xd5\xe9\xcd\xe2\x63\x79\xaf\x53\x04\x94\xf3\x45\xb5\x98\xc1\x72\x26\xf9\x87\x69\x35\xbd\x98\xc2\xb0\x3e\x17\x09\x3b\x8c\x41\x47\x31\x54\x4f\x50\xc3\xab\x05\x4c\xc9\xec\x6c\x35\x9a\x57\xb0\xf5\x6f\xcb\x7b\x58\x5d\x58\xf7\x53\x5c\x8d\xf9\x75\x71\x94\x5f\x01\x38\xfb\xcb\x04\x27\xbd\xf8\xee\xe8\x88\xca\xf0\x8e\x2c\x07\xc9\xd9\x62\x09\xdd\x7c\x8b\x89\xf0\xfb\x03\xad\x09\x04\x5e\xc2\xaa\xa0\x6c\x56\x55\xbd\x59\xc1\x9e\xfb\x84\x88\x5f\xf2\x17\x72\xae\x02\xa9\xaf\xe1\xdf\x4f\x8b\xbf\xc3\xdf\xdb\x2a\x19\x36\xe8\x07\x42\xb9\xc8\xbe\xe4\xd7\xbf\xfd\x30\xdb\xd9\x94\xc1\x5d\x09\x17\x6f\x5e\xf1\x7e\xbc\x45\xa9\x38\x7a\x06\xea\x59\x39\x1c\xbe\x2a\xd8\x9c\x38\x01\x45\xce\x5e\xba\x8d\x2c\x50\x1e\x55\xdf\xc8\x3c\x7a\xcd\x1f\x3b\x54\x4c\x4c\x07\x00\x56\x4e\x99\x14\x06\xe2\x3b\x84\xe8\xed\xde\xb1\xf4\xc3\xe8\x63\x3b\x17\x57\xd4\xe1\xbd\x06\x39\x26\x2a\x52\x1b\x59\xd1\xd4\x08\x3e\x59\xb8\x2c\xb5\x27\x8b\xc7\xc7\x28\x60\x0f\x5b\xa6\xe1\xe9\x1c\xa3\x7f\x9c\x8e\xe5\x79\x8e\x18\xef\xb0\x6d\x9a\x9e\xf8\x83\x9c\xe8\x30\x70\x71\x7d\x3d\xb3\x4f\xe0\x6b\xb2\x36\x8e\x13\x8f\xae\x3e\x4b\xe7\xf0\xd3\xcb\x85\x63\x8e\x07\xe4\x4c\xb2\xfb\x14\xcb\xf4\x29\x5f\x31\xad\x7e\xa4\xfd\xcb\xed\x39\xc5\x38\x7e\x83\xc0\xce\xa4\x3a\x17\x12\x87\xfd\x4e\xf1\xd5\xa1\x81\x2f\x52\x55\x3f\x2e\x16\xbf\x56\xc5\xbd\x80\x9b\xe2\xbe\x66\x12\x27\x47\xed\xe9\x3b\xb4\xd8\x8b\x77\x83\xff\x50\xa9\x79\x06\x69\x42\x9d\x49\x35\xde\xbb\x69\x42\x56\x43\xd7\xfd\xe4\x38\x41\xa9\xf4\x2d\xda\x65\xc3\x63\x76\x77\x7b\x81\x96\x73\x2f\xe1\x78\xde\xce\x9f\x2c\xee\xe6\x6b\xd1\xaf\x9f\xbd\x96\x9e\xec\x05\x67\x0d\xbe\xf0\x45\xf1\x47\xf7\xb5\xf0\xd9\x16\x2b\x80\xa2\x73\x52\xb7\xfd\x38\x1d\x2f\x3e\x52\xe8\xef\xa4\xd5\x4f\xa1\xc5\xe2\x96\x0c\x66\xd3\x01\x5c\x2c\x61\xb4\xc9\xd5\x6c\x31\x5a\x27\xb1\xc0\x22\x64\x78\x8e\x09\xfd\x44\x43\x49\x91\xd0\x76\xe3\x8f\x2d\x6f\xd7\x62\xd7\xf1\xca\x61\x5e\x57\x23\x9a\x27\xc6\xa1\xbf\xad\x73\x35\xfe\x58\x8f\xa2\x4f\xaa\xd9\xcd\x2a\x5f\x8d\x37\xb8\x36\xc4\xcf\x23\xe9\x5f\x87\x5e\xdc\x4e\x66\x44\x39\xd1\x16\xe7\x3e\x95\xbe\x12\x32\x04\x6f\xf3\xcb\xb0\x07\x5a\x11\x5a\xe5\xdb\x99\x56\xc2\xf5\x3c\x5e\xac\x30\x96\x50\x7b\xaa\xd2\xf5\x01\x68\x13\xea\x94\xa9\x9b\x36\x10\x33\x18\xbd\x78\x85\x46\xbb\x6a\xf3\x9d\x8f\xca\x54\x5f\xbb\x9d\xc0\xc6\x03\x84\x57\x3f\x49\x2b\x81\xc2\xc4\xf0\x93\xc7\x4c\x99\xe0\x2c\x00\x0b\x90\xb3\xa0\x5e\x72\x27\x7a\xe2\x15\x32\x78\x37\xca\xfd\x63\xa2\x18\xe0\x64\xb5\x07\xbe\x7f\xec\x73\x2e\x62\x4b\xdc\x57\xcc\x81\x34\xe6\x71\xf6\x0d\x86\x1e\x0d\x3b\xcb\xd1\xaa\xe2\xad\x90\x36\x42\x9b\x8c\x96\xad\x4c\xe6\xb4\xbb\xc9\xc3\x08\x4b\x0f\xc8\xdb\x95\x26\xb8\x4e\xa0\x9b\xa8\x57\xee\x1d\x39\xd5\x74\xb1\xff\x6a\xa7\x93\x0f\x4c\xb0\x84\x54\xac\x53\x26\x4b\x74\x7c\x15\xef\x64\x80\x30\x2b\xe2\x52\x11\xbc\x46\x64\x79\xcf\x75\xe3\x48\x8c\x3a\x7a\x7b\xda\xf8\xbc\x81\x92\xac\xf3\x71\x42\x3a\x2d\x7e\x36\x4b\x58\x93\x9b\xc9\x6a\x8a\x98\xbc\xae\x54\x15\xae\x14\xcf\x16\xaf\x55\xa5\x6b\x65\xcf\x43\x16\x2c\x3b\x7b\x43\xb1\x4d\x50\x5e\xfb\x10\x4d\x47\xb5\xe9\x94\x51\x7d\x7a\x67\xf0\xbb\xed\x1d\xb5\xf5\x75\x47\xc5\x19\x54\xfe\xa7\x9d\x0e\x7f\xea\xfe\x09\x47\x64\xc7\x49\x40\x4f\x20\xf1\x49\x38\xd2\xa9\xc5\xc1\xdb\xe9\x95\xfc\x16\x28\xd3\xcc\xc9\x9c\xa1\x1d\x7c\xc2\x70\x70\x0f\x92\x47\x10\x4c\x45\xe4\xbe\x8e\xfb\x68\x55\xb5\x04\x71\x74\x92\x13\x70\xa7\x7e\x6c\x36\xf4\xd3\x87\xed\x7e\x5b\x9a\x53\x02\xe5\x33\xee\x29\x99\xbd\x32\xfc\x5b\xd8\xcf\xb0\xab\x2e\xb1\x00\xda\x92\x47\x2e\x17\x7a\x28\xc9\xf0\x0f\x80\xd9\x8f\x56\x32\x92\x46\x21\x12\x42\xee\x35\x8e\x9f\xb1\x1d\xbc\xe4\x97\x8f\xd9\x18\xb0\x19\xa2\xc6\x75\x58\x52\x2c\x83\x24\xd9\x6d\x7e\xc9\x38\x22\x8f\x23\x4a\x29\x23\xe0\x84\xee\x38\x7d\x02\xe4\xc7\x08\xea\x10\xb6\x8f\x26\x71\xb3\xc6\xf6\xea\x95\x6a\x07\xac\x07\x0a\xb8\x91\xc5\xaa\x1c\x2c\xed\x13\xb9\x5a\xe9\xc4\xf6\x53\x87\x6b\xc5\x2e\x41\xa4\xfc\x8e\x82\xb2\xe6\x00\x76\x50\x4f\x8f\xb6\x41\x83\xc7\x9c\xfc\x5d\x78\x91\xc3\xbd\xb9\xbe\xc9\x6f\xa7\xf3\xbf\x70\x60\xf4\x89\x03\x9a\xab\x74\xf9\x89\xd7\x1c\x20\x81\x34\x41\x92\xda\xd7\x00\xf6\x4b\xad\x90\x92\x9b\x0e\x55\x98\xd3\x0c\x3c\x19\xfe\xa1\xbc\x7e\x8d\xa9\x9d\x15\x29\x80\xa6\x03\x10\x88\xad\xd0\x49\xd2\xd7\x72\x2a\xe0\x2c\x1b\x55\xbb\x08\x73\xcc\x0c\xed\xc2\xdc\x05\x7c\x3e\x6a\xb7\x85\x28\x0a\xb5\xe4\x68\x86\x4a\xaa\xa1\x1b\xce\x96\x44\xd6\x26\x4f\xe3\x75\x2e\xc3\x6c\x51\xaa\xad\x19\x87\x18\xcc\xaa\x34\x68\xb3\x7c\x6c\xaa\xd2\x75\x21\xaa\x5b\x03\xac\x57\xb8\xf5\x8e\x89\x62\x17\x82\x5d\xd8\x30\x28\xbb\x42\xeb\x0b\x83\xfe\xd2\x1e\x64\x8e\x9f\x29\xf3\x1b\xb6\x1c\xd2\xaa\xf9\xaa\x82\xff\x91\x3e\xfa\xc7\x77\xda\x57\x6f\x24\x35\xa3\x8f\x9e\x9f\x78\x7c\xe6\x7c\x0a\x42\x60\x62\x76\x6c\x8e\x3d\x47\x81\x87\xfb\x03\x87\x23\x8b\x28\x23\xe3\x2e\x90\xce\xe8\x2d\x5f\xc3\x32\xd6\x72\x85\x2e\xc4\x2a\xcd\x87\x51\xd8\x3c\x47\x94\xb5\x39\xa5\x68\x41\xf3\x38\x4b\x82\x88\xf1\xe9\xf4\xef\x13\x35\x4b\x07\xe8\xf5\xe4\x96\x10\xec\xdc\x8d\xa2\xbb\x04\x4a\x73\x86\x95\x77\x10\x45\xb0\xe5\x9b\x1b\xe6\x88\x4c\xe0\x3a\x21\xed\xa3\xbb\xf5\x82\xf1\xf6\xf0\x6d\x7b\x90\xdc\x4c\x84\xca\xa5\x8d\x08\xd4\xad\x97\xaa\x11\x97\x5c\x8d\xd7\x5f\xf9\x00\x7d\xa1\xd8\x73\x74\xee\xfb\xfa\xfc\xd1\x5d\x5c\x5d\x01\x7e\x41\x3b\x19\xe6\xcf\xb0\x38\x78\x25\x62\x94\x2c\x91\x1c\x89\x23\x92\xba\x98\x59\x1a\x65\x66\x41\x5e\xa7\xe7\xae\xa5\x89\xd7\x2b\x26\x36\x62\xf4\x03\xd5\x19\xbe\x98\xc7\x39\x42\xae\x6a\xc3\x0d\x28\x05\x92\x3d\xe2\x13\x46\x31\xf4\xd8\x65\xf7\xbc\x73\xe8\x3f\x11\xa7\x8c\x1f\x26\x8c\x0a\xd6\x2a\xeb\x8f\xee\xe8\x8d\xe9\xc1\x8e\xc5\x78\xe2\xc5\xe2\x13\xec\x23\x40\x8c\xd1\xfe\xe1\x02\xbd\x64\x1d\x40\x94\x79\x5d\xf2\xb3\xe9\xb2\x26\x86\x6e\xd7\xdf\xe2\x88\x98\x5e\x71\xf5\x42\xa4\x05\x18\x15\xed\x06\x4d\x79\x68\x3f\x78\x3b\x90\x92\x99\x97\x39\xf5\x17\x42\xed\xa8\xf4\xeb\x87\x47\x94\x94\x3d\x38\x90\x18\x76\xe0\xd1\xef\x1e\x1d\x7f\x63\x10\x16\x66\x42\x75\xf7\x8f\xb3\x4e\x92\x14\x6e\x0b\x22\xf5\x9a\x24\x3b\x57\xd3\xac\xa3\x81\x3b\xb9\xed\x47\xfd\x58\x2b\x05\x5b\xd6\x70\x22\xae\x13\x4e\x1f\x72\x07\x53\xcd\x96\x74\x8e\x8f\x8e\xbe\x61\xa4\x3f\xc9\xb0\x3f\x62\xbe\xa7\x0c\x81\x76\xc3\xe8\x37\x9b\x2a\xf8\x4a\x92\x1e\x47\x20\x69\x5c\x1e\xe7\xdc\xe2\xe3\xf2\x18\x93\x4a\xa1\x13\xd0\x42\xa5\x7d\x66\x5c\x4d\x6f\x53\x71\x4a\xef\x7d\x2b\x62\x0f\x49\x3c\x46\x80\x69\xcc\x47\x45\x0b\xd4\x4d\xf1\x69\xc2\x55\x25\x02\x6d\x84\x32\x09\x47\xb2\xd7\x30\x14\x47\x9e\xc8\x77\xc9\x7d\xe0\xad\x21\x59\xfa\xcd\xdd\x94\x89\xcc\x0a\x4e\x46\x23\x50\x1d\x8d\xf3\x7b\xb7\x41\xd7\x46\x77\x33\x10\x8b\x53\xe4\xa1\xfe\x44\xe8\x01\xb1\xf9\x1a\x76\xf6\xad\x4f\xfd\xaa\xdd\xed\x36\x59\x0d\x46\xdd\x3b\x06\x25\x8b\x53\x1f\x38\x3e\x25\x73\x91\x07\x7c\x30\x4d\x83\xc9\xd0\x41\x9e\x6d\xfd\x15\x91\x2e\x04\xef\xa1\xd6\x73\x1a\x97\x71\x9c\x00\xf9\x35\xf1\x33\x67\xcc\xbe\x34\x00\x7e\x09\x27\xb2\x0e\xe0\x31\xf6\x41\x00\x5f\x1f\xb2\x3b\xcb\x01\xc1\x42\xd5\xc7\xb7\xaf\xab\x44\x19\x5e\x04\x1e\x5d\xd7\xd3\x8c\xdb\xa7\xbb\xce\x1d\x5c\x3f\x1b\xdb\xcc\x72\xbf\x96\x2b\x3f\x54\x63\x9d\xcf\x4b\x53\xdb\xd8\x2e\xf3\x8b\x1f\x42\x81\xcc\xcd\xf4\x3d\xba\x22\x30\x91\x0c\x87\x21\xb6\x2e\x89\xae\xfb\x89\xf9\x79\xaf\x29\x3b\xd2\x7f\xc8\xad\xc7\x43\x61\xa8\x4b\xcb\x24\xea\xca\xb6\xb0\xa4\x64\xed\xc6\xdb\xe6\x4d\x23\x21\x2e\xb8\x35\x15\x69\x87\xb2\xb7\x7b\xec\xaa\x94\x15\x0a\xb8\xf1\xc6\x43\x40\xb4\x1c\x8d\xf1\x45\x1b\x83\x7c\x8b\x14\x09\x4d\x86\x95\x4d\x5b\x12\xef\x3b\xaf\xee\xae\xe0\xa7\x69\xf7\x60\x72\x87\x93\x61\x1b\xf1\xcb\x5a\x4d\x8e\xdc\x98\x01\xcd\x39\xcb\x04\x69\x3e\x94\xb9\x5e\x57\x75\x93\x48\x54\xaa\xcf\x52\xbb\xf4\xaa\x81\x0e\x83\xb2\x42\x64\xc1\x7b\xdf\xa9\xa0\xbd\xd4\xa4\xbd\x70\x4c\xfd\xc1\x74\xe8\xba\x44\x4d\x90\x24\xbe\x84\x0e\x1e\xb9\xf0\x91\x23\xeb\xb4\xaa\xed\x36\x0f\xe8\x08\xae\x39\xab\x03\x8c\x68\xe4\xc8\x42\x29\x6b\xb7\x7d\x26\x0f\x4a\xab\x47\x47\xe5\xe1\xbf\x3e\x3a\x42\xdb\x05\x17\x2b\xa0\x48\x21\xf3\xe1\xf9\xe0\x7c\x88\x8f\x3e\x4f\xde\xbe\x7c\x0e\x5f\xab\xfe\xf9\x1c\xd3\xd9\xc0\x28\xac\x22\x72\x19\x2b\xd6\xe4\xe7\xb8\x0d\x1b\x35\xde\x90\xe9\xd2\x0d\x99\x1f\xdd\xa0\x69\x53\xd6\xec\x97\x72\x24\xf0\x4b\x85\xd8\x54\x01\xbf\x19\x6f\xf0\x15\x9c\x8c\x1a\xfc\x3a\xf9\x7c\x3d\x99\x67\x87\xd3\x86\x27\x8b\x4a\x15\x33\x1a\xf8\xe5\x92\x9b\x94\xe2\xd5\x48\x9f\xe4\xe6\x67\x07\xdc\xbc\x61\xd4\x2e\xd5\xb6\x48\xbb\x46\x5f\x6e\x49\x08\x48\x9b\x01\xa0\x20\x3a\x92\x9a\xe4\x98\xd3\x2e\x73\xdf\x09\xd0\xaa\x5d\x2c\x4d\x61\x49\xdd\x6d\x93\x7d\x44\x6c\x8d\xf9\x8c\xd8\x15\x0c\x05\x2c\x7c\x36\x2a\xbd\x17\x70\xe2\xa7\x55\x6a\xdd\xff\x23\x11\xe0\xe7\x9a\xb7\x89\x53\x42\x7c\xc5\x74\xc5\x5e\xb4\x8c\x92\x8b\x35\x14\x55\xab\x53\x04\x09\xf6\x1e\x30\xe4\x60\xca\xa1\xee\x67\x30\x73\x53\x31\x19\x22\x4e\x74\x03\x75\x5c\xf4\x0f\x9b\x46\x9e\xc8\x20\x8a\xdd\xff\x45\xef\x45\xc8\xbf\xe9\x1b\x59\x64\xe4\xcb\xd9\xf3\xab\x8b\x77\x4f\x1e\x65\x98\x09\xee\x78\xe7\xc8\xf8\xf1\x37\x36\x6e\xe5\x3c\x39\x5f\x9d\xcf\x13\xbc\x9b\x8b\xdf\x5e\x84\xe5\x44\x0d\x4e\x4a\x1b\xce\x43\xbe\x51\xbe\x46\x6b\x16\x18\x1e\x89\x32\xb0\x42\x28\x14\xcb\x02\x80\xe6\xf3\xc2\x56\xf7\x88\x9e\xd7\x3d\xa9\x2b\x33\x30\xa4\x49\xb3\xc2\xb0\x7d\xfb\x80\x48\x32\x9a\x5e\x0d\xf4\xe5\x69\x58\x4e\xe6\x97\xb0\xc8\x3f\xbf\x7d\x81\xcc\x1b\x80\xf7\xf3\x35\xb6\x02\x58\x27\xe0\x79\x0d\x69\xa2\x80\x43\xbc\x6d\xdf\xed\x88\xf1\x67\x53\x8c\x8c\xf5\xe9\x64\xbd\x46\x13\x52\xee\xca\xb3\x91\x5d\x3b\x0b\x79\xb4\xa0\x23\x34\x34\x2e\x66\xee\xbc\xf4\xf7\xb4\x7a\x33\x1b\x4d\xe7\x6c\x6a\x0d\xf2\x64\xf6\x52\x18\x59\x8a\x0a\x2d\xb7\xb9\xc3\xc0\xea\x38\x32\x90\xcc\x3b\x33\xe2\x59\x47\xde\xd9\x28\x23\x41\xbd\x37\xb8\x50\x95\xde\x17\x23\x81\x8f\x43\xbb\x5e\xb8\x3c\x9e\x34\xea\xbe\x5f\x4c\xe7\x69\xd2\xb6\xe2\xdb\x8f\x8e\xd0\x89\xaa\xb9\x06\xb1\x73\xc9\xc5\xec\x6e\xd5\x22\x0f\x0b\xfc\x17\x1a\xa5\xdf\xc5\xdd\xba\x05\xf8\xff\x18\xfd\xf1\x01\xd0\x69\x55\x97\xab\xc5\x6c\xd6\xba\x9b\x53\xe4\xe5\x6c\x7a\xf9\x6b\x6b\x7c\x31\xe3\xc0\xed\xe2\xae\x9a\x8c\x17\x1f\xe7\x1c\xba\x5b\xf2\x2f\xe2\xb7\x1c\x82\xc0\x4a\x42\x50\x2f\x05\x26\x28\x3c\xc0\xc1\xd9\x64\x04\x19\x2f\x6f\x46\xf3\xeb\x89\x98\x84\x6a\xf1\x19\x6f\xc1\x0e\xa0\x7a\xe1\x97\x3c\x28\x63\x00\xaa\x27\x77\x20\x2d\x91\x40\x02\xb0\x74\x97\x98\xbb\x6c\x27\xe1\x5d\x97\xfa\x26\x41\x16\xb4\x8c\x25\x13\x57\x7b\x0f\x3d\x62\x4d\x7f\x7d\x66\x26\x21\x07\x2d\xc4\x32\xdc\x70\xbd\x5e\x5f\x4f\x98\xdd\x19\x19\xd5\xbb\xc1\x61\xfb\xc6\xae\xe6\xaf\xe1\x1b\x4a\xbe\xbe\x5b\x47\x50\xdb\x4d\x88\x64\xca\xba\x7e\x62\x52\x2a\xb0\xd9\x48\xca\x96\x6e\x3d\xdc\xb0\x2f\x17\x97\x6f\xf0\xaa\xcd\xe5\x83\x80\x1c\x7d\xbc\x03\x7c\xe9\xd2\x9d\x49\xb4\x4f\x9a\x71\xfc\x8a\x36\x2e\xdc\x88\x7d\xb8\x1c\xc9\xa7\xc7\xe1\xbf\x74\xc9\xd6\x2f\xdc\x0f\x87\xe9\xa0\xdf\x1e\x66\xef\xca\xc1\x2f\xed\xe1\x37\x98\x81\xec\x0f\xd1\x3d\xd9\xfd\xa6\x0f\x38\x42\xeb\x7c\x3d\x44\xc1\x0b\x04\x29\x68\x8b\x67\xd5\xdf\x3f\xbc\xbe\xcd\x57\xe8\x00\x64\xf6\x06\x8d\x37\x5f\x2e\x66\x7c\x43\x8e\x2e\x60\xa5\x37\xa3\xe5\x12\xff\x1d\xa0\xe6\x05\x5e\xac\xdd\xce\x01\x5d\x88\x15\x1b\x3b\x9a\xd1\x55\xbb\xf9\x38\x1d\xa3\x61\xd9\x82\x6c\x0e\x2f\x54\x6a\x8d\xaa\xf9\xf3\xb3\xb3\xcd\x8f\xcf\x4e\x9e\x92\x08\xc7\xd2\xb7\x70\x7e\x78\x7e\x08\x31\x77\x2b\x6a\x6e\x70\xfe\x11\x6a\x1e\x76\x8a\x0c\x4d\xdd\x40\x12\xf6\xf1\xb0\xff\x2f\x05\x74\x12\x62\x8a\x14\xad\x1b\x6d\xe0\xbf\xc3\xfc\x1d\xee\x5f\x63\xe6\x10\x3f\x73\x3a\x4f\x6c\x5b\x1a\x10\xa7\x35\x72\xf5\x11\x1d\xa5\xaf\xd1\x6c\xc6\xd8\x42\xf2\xcd\x61\xd2\x65\xcd\xb1\x34\xf9\x46\x94\x13\xed\xd4\x97\xde\x0f\x0a\xd4\xe7\x5f\x6f\x82\x3c\xbb\x6c\xb0\x8c\x92\x2c\x58\x47\xaa\x03\x1d\x57\x06\xa5\x6b\x39\xb6\x76\x23\x94\x38\x1f\xfc\x72\x17\x64\x0c\x7d\x5d\x91\xd3\xba\x70\x02\x8c\x21\xb8\xd5\x8c\x8d\xf3\x54\x46\x71\xae\xc9\xae\x32\x64\x6c\xb7\x69\x32\xf5\xd5\x92\x3e\xe4\x39\x80\xd0\x0b\x77\x98\x18\x51\x33\xe6\xdc\x60\x76\xe7\xd5\x84\x5f\x74\xbd\x41\x66\xa0\x0e\x4a\x1c\x80\x7b\x83\x6b\x79\xbc\x04\xd2\x1e\x23\x19\x61\x8c\xcb\x41\x4e\x76\x2c\x03\x69\x39\x7e\x89\xd6\x25\x86\x4d\xea\x11\xd6\x9b\x19\x40\xee\x2e\x28\x1e\x67\x06\x24\x97\x8c\xb4\x94\x91\xf3\x8f\x79\xa0\x2b\x38\x06\x08\x11\x27\x6b\x2c\xd3\xa0\x09\xfc\xbe\x9a\xbc\x79\x7d\x7a\x96\x88\x7e\x9b\x97\xe8\xae\x69\xf7\xe0\x2c\x93\x62\x8f\x13\xf7\x62\xbd\x1e\x14\xd6\x65\x3d\x1f\x69\x11\x2e\xed\x31\xaa\x88\xb8\xf5\xd1\xb9\x3b\x13\x17\x4e\xfc\x55\x7a\x43\x06\x5e\xe5\x2e\xb6\x7d\x9a\xb0\x7d\x21\xa7\xc1\xea\x91\x80\x6a\x82\x12\xc4\x61\xdd\x59\x68\x8e\x1a\x0d\xe8\x99\x54\xe8\x17\x12\x94\x28\xe7\xe5\x66\xaf\xdd\x76\xbd\x7c\xff\xb7\xbf\xfe\xf8\x16\x5d\x60\xae\xef\xaa\xec\x9e\xba\x44\xb7\x8d\x7b\xa9\xd2\xca\x60\x2f\x52\xde\xae\xad\x5d\x0a\xe6\x94\x82\x4e\xcf\x72\x56\x79\x0f\x29\x71\xba\xb2\xd1\x79\x1f\xd0\xe3\x1c\x26\xd2\x1c\x83\x4f\xa4\x6b\xf2\x49\xee\xf9\x35\xd7\x1d\xc9\x04\xe9\x17\xea\x1b\x05\x74\x3c\xa1\x83\xe6\xbe\x60\xbf\x7c\x06\x84\x87\x80\x7b\x21\x9e\x4e\x21\x3e\x56\x76\x1a\xd0\xa3\x65\x9e\x00\x39\x1e\xf1\x0a\x6e\x27\xeb\x9b\xc5\x58\x5b\x19\xf0\xe7\x30\x3c\x83\x74\xc7\xb8\xe9\xe2\x5e\xed\x34\x49\x83\x99\x33\xdd\x83\xfc\x4c\xee\x8a\xba\xbd\x4d\x35\x92\x7c\xa3\xdf\xd7\xf9\xee\x9d\xc9\xbd\xf2\x7b\xd3\xed\x54\xb2\x0e\x00\x34\x2b\xcd\x63\xa1\xd5\x37\xca\xd2\xb0\x33\xa6\xe2\x88\x34\xa4\x7e\x5a\x8c\xd1\xc3\x17\x79\x83\x9f\xac\x47\xd7\xf8\x6b\xd1\xae\x82\xda\x0f\xae\x32\xa3\xfb\x36\xad\x30\x76\x56\x84\x57\x0d\xa3\xf0\x16\x04\x0e\x8e\x87\x99\xd3\x86\x43\x1b\xf3\x0b\xec\x26\x09\xb9\xc2\xa7\xe8\xcb\x39\x7b\xf4\x7c\xea\x10\x64\x4d\xb9\xc9\xc3\x4f\x07\x1f\x3f\x7e\x3c\xc0\x27\xdd\x03\xe8\x0e\xa3\x9d\xe3\x1e\xe2\x26\x2b\xa4\x42\x7f\x3e\x7b\x7e\xf0\xc7\x44\x04\x64\x51\x6a\xe6\x9b\xa4\xd0\x0b\x82\xfd\xe0\x90\x0b\x9b\xc3\x25\xa2\x83\x09\xdb\x27\xe0\x18\x3e\xda\xe8\x24\x2c\x6c\xef\x76\x96\xb7\x28\xc3\x27\x4c\x7f\x5f\x91\x8c\x9e\xc9\x80\x31\x92\x03\xcd\xc3\x89\x0c\xb1\xb1\x37\x7a\x8f\x75\x62\xe9\x43\x6e\x8e\x5a\x3a\xe4\x9a\xa8\xf4\xe1\xd6\x1d\xb7\xe7\x64\x9c\x8d\x8b\x24\x1a\xf9\xd7\x9f\x5e\x26\xd2\x77\x7b\x0a\xb9\x0d\xc0\x38\xf0\x5a\xc4\x91\x52\x27\x92\x42\x1e\x59\x4f\xe9\x3e\xc8\x69\x70\x2d\x1a\x1c\xce\x2b\x7f\x62\xab\x4e\x92\x88\xe0\xcb\xbf\x9f\xbe\x7e\x25\x89\x38\xce\x20\x0d\xda\xdf\xe6\x57\x40\xa2\xbd\xe6\x87\x62\xde\x08\xba\x48\xe4\x2b\x60\xeb\xb6\xca\xdd\x32\xb6\x47\x8f\x1c\x68\xda\x40\x5e\xfc\x4c\x22\xfa\x74\xf8\x79\x2f\x9a\xa0\x14\x6b\xc0\xfc\x33\x5f\x57\x61\x0a\x34\xe4\x14\x33\xf4\x19\xf7\xeb\x8d\xa2\x0f\x05\x20\xe1\x67\x0b\xf7\x59\xbd\x5e\x9d\x39\x5c\x22\xf5\x48\x06\xdf\xf4\x2e\xe9\xc1\x52\x1e\x19\xe1\x52\x45\x00\x26\xf4\x6d\xfd\x5e\x63\x5b\x74\x5d\x08\x00\x9d\x8f\xf0\xfc\x9d\xd2\x07\x01\x56\x5d\xdf\x4a\xdc\x53\x55\x6a\x85\x40\xc0\xa2\x1e\xeb\x9c\xf0\x6b\x7f\x51\xdf\xca\xd9\x15\xc8\x8c\x75\x95\x71\xe5\xbd\x47\xe4\x2a\x7c\x24\x1c\x6e\xf8\x2d\x1f\xe5\xf8\xce\x07\x78\xe0\x19\xfc\xa0\xad\x75\x14\xc1\x39\xe3\xa8\xd4\x26\x65\x1e\xe7\xf2\xd0\xc9\x35\xff\x23\xf7\x95\x77\x5c\x29\x3d\x47\xbe\x7c\xae\x77\xc7\x68\xfc\xf9\x94\x5a\xe4\xfe\x01\xd6\xfe\x5d\x71\xe4\x47\x4b\x2f\x99\x7a\x53\xc2\x1c\xfe\x48\xa2\xd5\x6f\x35\x39\x95\xab\xc6\x17\x40\x15\x75\xae\xa9\x7c\x74\x04\x17\xf8\xb7\x47\x47\x8f\x39\x62\xb3\xf9\xf6\x08\xdd\x86\xf1\x57\x3f\xad\xba\xd3\x2b\x85\x6c\xe8\x57\x5f\x82\x25\xf7\x0c\xb6\xc8\xdb\x60\x10\x69\xf2\x12\x40\xe1\x81\x96\x00\x54\xe1\xd6\x17\x96\x7d\x66\x81\xe5\x80\x44\xbd\x7f\x7e\xfb\x72\x58\x6a\x46\x5f\x64\x77\x23\x08\x5f\x1b\xeb\xc6\x84\xa6\x3a\xb3\xfc\x51\x30\x2c\xb7\x23\x4a\x38\x87\x66\xd1\x93\xf9\x42\x00\x4f\x92\x15\xdf\x7e\x6d\x91\xf5\xad\x1b\x6f\x61\xf2\xf1\x55\x4d\x20\x26\xad\xdc\xe4\x67\xb6\xb0\xcb\xdc\xa5\x1d\xa5\xfb\xd3\xc4\xd3\x6d\x44\xdb\xd5\x44\xf2\xf6\x35\x3d\xa2\x88\x0c\x5a\xe7\x9c\xbe\x85\x3c\xd5\x75\xdd\xf3\x91\x99\xec\x60\x1d\xc1\x84\x11\x08\xdd\x03\x2e\xb5\x3c\x42\x65\x78\x5e\x05\x89\xb1\xa8\x4b\xd7\xd4\x91\xc6\xc7\x85\x3c\x0f\x6b\x7b\x9d\x24\xf1\xbd\xed\xc3\xfe\x87\x16\x27\x63\x44\x8a\x16\xb3\x0f\x6c\xad\x4c\x2f\x5a\xf5\xb7\x32\xd0\xa3\x6a\xc6\x22\x08\x53\x61\x2a\x40\x6c\xb5\xb9\xbc\xc5\xd1\xa8\x34\x8d\x72\x18\x8e\xe7\x09\x6a\xf4\xf8\xa0\xae\x0d\x86\xcd\x41\xbd\x9a\xae\x26\xac\xc6\x02\x73\xa3\xba\xe6\xb0\x47\x9c\x5b\x07\x21\x91\xfd\x10\x03\x5c\xac\xb0\x58\x9a\xf6\xcb\xcc\x87\xe2\x1a\xda\x41\xc5\x3c\x9f\xea\x30\xb1\xfd\xaf\x1b\xe4\x30\x0b\x3b\x9b\x3e\xd0\xdb\x08\x95\xd4\xba\xa0\x8a\x83\x03\x27\x60\x8e\x8b\xea\x5e\x1d\x58\xd9\x23\x28\x4f\x58\x29\xec\x92\x6d\x8d\x72\x20\x02\x2a\x15\x00\x8e\xf4\x0a\xd3\x2d\x1e\x37\xd3\x24\xf9\x45\x6b\x37\x3d\xb1\xb4\x8f\x0a\xd6\xb9\x1e\xe3\x66\x38\x19\x80\x5e\x3b\x68\x0f\x6a\xe3\x82\x79\x15\xf1\xc8\xee\x96\x29\x20\x69\x7a\xc7\xe4\xd1\xec\x96\xce\xc9\x06\xec\xe5\xbc\x3e\x91\x3e\x9d\x7c\x7f\x07\x65\x8d\xfe\x72\x9c\xc2\xbc\x35\xf7\xa6\x15\x25\x3b\xa1\x76\x9a\xeb\x5c\x77\xba\x76\x5c\xb7\x44\x5a\xdf\x25\x9a\xe5\x89\x54\x58\xa5\x09\x32\x49\x5a\xb7\x93\xdb\xc5\xea\x73\x12\x6c\xed\xca\x6c\x7f\xb2\x32\xb4\x9a\x40\xaf\xaa\xb5\x4c\x54\x59\x8b\x61\x1b\xd0\xf7\xdb\x9c\x2f\x3e\x04\x80\xab\x93\x0b\xbc\xce\x92\xcb\x11\xb4\x82\xcc\x6d\x3e\x58\xe5\xbd\xbf\xae\xe0\x8e\xaa\x01\xef\xc2\xb2\x58\xf9\x7a\x26\xcb\xd6\xc8\xe7\x7c\x24\xe0\x76\x42\x24\xf9\x5e\xb4\x7c\x19\xfb\x17\x0a\xe2\x50\xf6\x8f\x8a\x97\xca\xcc\x61\xf6\x40\xe3\x96\xc9\x7a\x59\x14\x3f\xa0\xb2\x75\x23\x4e\xec\xa7\x6c\xf0\x68\xa8\x95\x47\xc5\xa0\xeb\x51\x89\x6d\xc0\x2e\xa7\x52\xcc\x30\xa7\xe0\x56\x2c\x8d\x84\x53\x51\x35\x3c\x72\xb8\x29\xe8\x37\x0e\x41\x9c\x48\x54\x38\xab\x66\x7d\x9a\x95\x45\x48\xe4\x8a\x04\x5f\xe7\xbb\x3c\x62\xb2\x22\x02\xba\x6c\xc7\x6c\x0d\x6b\x3e\x98\x31\x47\x71\x77\xd2\x66\x43\x6d\x86\x19\xac\x98\xaf\x92\xb7\xc8\x2c\x44\x4f\xc8\x3f\xc1\x89\x25\xc2\xc4\x23\xba\x96\xfc\xd3\x1e\x55\xdd\x5b\xc9\x48\x10\x45\x6b\xf1\x1b\xd7\x97\xbf\x1d\x89\xcb\x3b\xa4\x66\x70\x23\x61\x04\xee\xa7\xc7\xbc\x9b\xc8\x31\x0e\x24\x21\xc7\x19\x93\x7c\x1d\x03\x8c\x1e\x96\x83\x38\x06\xb5\x64\x38\x24\x3e\xf8\xf9\xd2\x18\xcd\x3e\x8e\x3e\x57\x58\xfd\xc0\xde\x22\xc3\xc8\x6c\xdf\x08\x4f\x46\xa8\x0e\x22\x17\xe1\x3d\xdb\x60\x9a\x8f\x66\x74\x6b\x56\xc1\x75\xc9\x27\xca\x55\xa5\x60\xac\xdd\x76\xc1\x2e\xd5\x9c\xba\x0a\xb2\x9c\xd0\x61\xd4\xa2\x70\x31\x6c\x98\x84\x2c\x11\xe8\xfd\x01\x54\xe2\xed\x14\x56\x9d\xfa\xec\x39\x1b\x65\xed\xa2\x01\x44\x5d\xaf\x47\xc1\x2c\xf8\x0b\x5b\x91\x04\x46\x30\x38\x7c\x35\x42\xa7\x0a\x5d\x84\xeb\x29\xa2\xeb\x30\x88\x2e\xfd\x58\x2a\x17\x2f\x7f\xc3\xa5\x47\xa6\x6c\x1e\xc4\x28\xd7\x33\x8f\xe8\xdc\x4e\x72\x78\x88\x60\x8b\x1e\xda\xf4\x96\xe8\x32\xdd\xbe\xd9\xe8\x37\xb3\x03\x7c\x3c\xc7\xe4\x8c\x38\x31\x63\xd3\x8a\xb3\xf8\x78\x40\xb0\xbf\x49\x22\x87\xd9\xec\x45\x20\x25\x25\xb4\x15\xe0\x75\x1f\x51\x5f\x9b\xf5\xd8\x87\xb9\x38\xa4\xe9\x5e\xae\x16\x40\x71\x2f\x6e\x81\x02\x06\xc0\xbf\x8c\x18\x93\x34\x03\x11\x47\x32\x0f\x0a\x95\x7b\xe9\x1e\x95\xd2\x97\xe4\xe3\x21\x1c\xfa\x68\xf0\xed\x36\xa7\x3d\xaa\xa5\x3d\x1a\x6a\xab\x83\x6f\x51\x70\x3d\xb9\x59\xaf\x97\x05\x0a\x0f\x68\x6d\xfd\x3f\x1e\x15\xdf\x7d\xf7\x2d\x0a\xe1\x87\xbc\x83\x28\x7f\xd4\xa4\x2b\x46\x1d\x1e\x93\x01\x4c\x72\x59\xa8\x1c\x06\xef\xdf\xc2\xbb\xab\x92\x7c\x32\xaf\x65\xf0\xd2\x2b\xc8\x6b\xf8\xe0\x94\xa1\x99\x99\x25\x60\x0a\x5f\xa4\x22\x01\x47\x52\x09\x70\xde\xb8\xb9\xbf\x1f\x94\x55\x84\xf1\x3d\x73\xf9\xc3\xf2\xf0\x45\x1d\xa2\x41\x47\x5e\xa8\x89\x51\x9a\x4e\xc7\x4b\x83\x34\x21\x35\xc8\x76\x73\x5b\x8f\x7f\x60\x4d\x7f\x5e\x2e\x9d\x01\x83\x0a\xbd\x6b\x2b\x2b\x7f\xcf\xb3\xf5\xc5\xc7\x17\xfb\xe4\x75\x38\x4c\x49\xdb\x22\x28\x44\x20\x4e\x66\x2f\xcc\xd6\xe1\x65\x93\xc7\x0c\x91\x85\x91\x1c\x59\x3f\x69\x03\x42\xd9\x47\x4f\x76\x3c\xbd\xa2\x14\xcb\x5f\xb4\xd5\x48\x31\x19\x1d\x73\xd3\xba\xe0\xe9\x5c\xa9\x4b\x65\x5f\x8d\x86\xfc\x39\x04\xa4\x2b\xd9\x3f\x7e\x57\x26\x1d\xff\xc4\xd2\xe9\x64\x85\xe6\xec\x7c\x55\xaf\x92\x5a\x05\xd8\xa7\x80\x88\x7c\x90\x0e\x84\x75\x61\x00\x14\x5d\x71\x69\xf2\xe2\xca\xd1\x95\x07\xa7\x53\xa8\x3b\xc9\x1f\xac\xc9\xb3\xeb\x02\xa2\xf0\xc1\x06\x5e\x01\xa0\x3b\x20\xc7\x9f\x49\x73\x69\x18\x4d\xea\x0f\x87\x5f\x4d\xfc\x32\x4c\xb7\x3d\x9c\x7e\x0f\xa4\x4c\x4a\xb6\xbb\x7d\xa9\xea\x00\xb3\x25\x79\x58\x2a\xdf\x55\xe8\x84\x58\x75\x89\x05\x78\x64\xd1\xbd\x12\xa5\x70\x7c\x81\xb6\x29\xc3\xfe\xce\x94\x0e\x3e\xfb\x20\x93\x23\x88\xee\x27\x79\x0b\x56\x54\x98\x80\x9d\xa4\xd7\xfa\x5b\x79\xd4\x3d\x22\xf1\xc8\xac\xf0\x95\x41\xd9\xa1\xf7\x75\x0b\xfb\x51\x59\x30\x3b\x7a\x3e\xcd\x5d\x16\x35\x9d\x53\x75\xd9\x18\x16\xb2\xad\x69\xef\xfa\x4f\xb6\xc4\x16\xd3\x3f\x42\xb2\x64\x25\x4f\xb7\x87\x0f\x01\x80\x90\x5b\x33\xeb\x79\xa4\x95\x62\x12\xeb\x99\x97\x49\xb0\x63\xa6\x12\xe1\x57\xef\xc6\xe2\x78\xcb\x23\x80\x4e\xa6\xae\xa7\x9e\x9d\xf3\x30\x34\xf3\xdc\xad\x1a\x34\xcb\xee\x6b\xdc\x9d\xe3\xaf\xa6\x34\x3d\x77\xdf\xd2\x6d\xb0\x18\xc8\x00\xc6\xc5\x17\xea\x08\x5f\x67\x02\x4e\x14\xca\x19\x29\x87\xca\x8a\x3d\x9a\x79\x4a\x24\x3f\x3e\xa5\xbb\x8a\x32\x7e\x0e\xac\xa4\x9f\x1e\x23\xa9\x90\x83\x18\x22\x7f\x84\x94\x64\xfe\x7d\x90\xcc\xbf\x2a\x2e\x96\x91\xfd\xb6\xd6\xa4\x47\x98\xcb\x01\x4c\x78\xb6\x65\xd3\x1b\x1a\x91\xbc\x5a\xb4\xdc\x14\xfa\x17\x32\xea\x22\x21\xd2\x6c\xa2\x2d\x64\x14\xfa\xc7\xbc\xf0\x15\x01\x45\x42\x88\xf6\x74\xc4\xbc\x7b\x39\x70\xf6\x2a\xa8\x52\xe4\xe0\x16\xbb\x9f\x28\x76\xd5\x1b\xbe\x63\x24\xc4\x19\xb6\x4f\x05\x86\xd2\x74\x4c\x75\x6e\x57\xf8\xe6\x9e\xe9\x9d\xb7\x02\xbe\xf8\x8e\x78\x74\x9f\xd2\x14\xff\xe9\xc0\xa7\x04\xec\x73\x69\xad\xd1\xff\x4a\xc4\x02\x27\xfe\xb5\x54\x61\xb0\x73\x79\x8a\x0b\x86\x6f\x8c\x7a\x30\xd6\x09\x7f\xb7\xd1\xb8\xdd\x99\x48\x9d\x41\x3f\x8f\x13\x67\xf7\xfe\xa6\xb2\x12\x2d\x1a\xbb\x77\x1c\xa1\x4d\x94\xc6\xaf\x90\xf4\x62\xa2\x37\x3d\xe6\x0c\xdb\x75\x9b\xa7\xb9\x5d\x02\x34\xa6\x62\x11\xe5\xe6\x39\x45\x88\xe4\xdf\xb0\xf1\xcb\x59\xd8\x4f\xf0\x2b\xc9\xc8\x8e\xf2\x2e\x6d\x1a\xd9\xac\xf7\x55\x60\xc5\xfe\x9d\xd9\x46\xdc\xce\xce\x67\x72\x67\x43\x85\x03\x7c\xa0\x89\xe3\xd8\xe5\x98\x27\xfc\x52\x83\xb3\xc1\x39\xf4\xe9\x26\xca\xe0\x6a\xa8\x56\x97\x8a\x7d\x70\xc4\x82\x44\x5c\x4a\xf7\x45\x00\x88\x0e\x27\x4b\xa8\x94\xa6\xdf\xd3\x8a\x00\x66\x76\x9f\x4a\x08\xf9\x88\x5c\xd0\xc3\xad\xcd\xe6\x10\x6b\x9c\x8c\x37\x0a\x34\x0f\x05\x0b\x8a\x73\x92\xaa\xed\xd7\x76\x83\x2d\xb9\x70\xb2\xb5\xcf\x5e\x8b\x0a\xac\xef\x54\x62\xfe\x52\xe6\x99\x2a\x71\x7d\xd7\x65\x48\x1f\x1d\x1d\xc1\xf9\x17\xfe\x1c\x4a\x1f\xe2\xda\x86\xe6\x7b\xcd\x86\x08\x3d\xe1\x45\xd4\x9d\x2e\xa9\xeb\x18\x0f\x2c\xf5\x10\x67\x8f\x34\x25\x44\x9c\x73\x31\x1b\x3b\x0e\x0d\x4a\x9d\xad\x10\x64\x2c\xcb\xc3\xb4\xcc\xce\xfb\x69\xbf\x6c\x6f\xf6\xb3\xcd\x79\xff\xbc\x7f\xd8\xab\x83\x10\xca\x5b\x24\x3a\x0c\x7e\x4f\x5b\x6a\x7d\x45\x24\xaa\xe8\x9e\x46\x6d\x9b\xdd\xe5\x62\x99\x66\x9e\xa5\xc7\xa6\x55\x00\x6d\x0b\xb1\x36\x4b\xd0\x0e\xb4\xa2\x21\x6e\x44\xf7\x22\xba\xf3\xb0\x63\xa7\xe8\x8d\x6c\x69\x0f\x9e\x33\x52\xe9\xde\x98\xf8\x1e\x0c\xba\xfa\x8a\x74\x3a\x01\x22\x7d\x5c\xa1\x00\xe4\xdc\xb1\xf0\x9e\xb0\xda\xdf\x64\x45\x43\x46\x35\x43\xd8\xd4\xd4\xc6\x9e\x60\xba\x3c\x91\x8a\x7e\xc3\x7e\x07\xa4\x14\xfe\x92\x3a\x7f\xe4\x94\x4f\xf0\xb7\xbd\x34\xc0\xb1\xd8\x1a\xa8\x13\xce\xf8\xe2\xab\x29\x49\x73\x06\x8d\xca\x3b\x75\x82\xbf\xe6\xe2\x92\x0e\x43\xfd\x3c\x29\x65\x0d\xc5\x4a\xed\xf8\x75\x60\xba\x64\x0d\xa2\x87\x51\x8e\xac\x1f\x45\xa4\x88\x98\x05\x31\x6e\xda\xfa\xd5\x40\x83\xc3\xd2\x84\x3d\x1d\x40\xe5\x90\x14\x48\x3a\xb6\x5b\xae\xca\x3d\x4b\x5a\x34\xd1\x2a\x3a\xfb\x9e\x50\xa1\x72\x24\xe8\x18\xd4\xc8\x38\xae\xdc\x3f\x03\x81\x7d\xfc\xba\x6a\xc4\x05\xbc\x52\x50\xbc\x17\xfc\x2e\x46\xa4\x2d\x98\xc4\x4e\xd2\xfa\x38\xaa\xd0\x11\x21\xed\x2d\x7a\x8a\xaa\x95\x87\xa9\xdf\x46\x88\x73\xc9\x57\xb8\xdd\x83\x25\xbf\x08\x0f\x6c\xf5\xc3\xbc\x29\x32\xec\x73\xd4\x98\x97\x61\xd9\xe6\x01\x37\xc9\x14\x6a\xac\xd4\x1e\x87\x2a\x4c\xa3\x55\x08\x37\x4b\x7c\xca\xa2\x6d\x10\x82\x02\x32\x73\x66\x57\xa4\x61\x92\x1c\xa1\x6c\xf6\x9f\xe9\x12\xec\x77\xf3\x95\x36\xcd\x71\x43\xa5\x76\x50\xfe\xfa\xdf\x1a\x23\x62\xaa\xc6\xc1\x50\xf3\xd3\xcd\xca\x75\x3b\x87\x8f\x53\xd6\xa7\x00\xe0\x0a\x1f\x2f\xc6\xe5\x11\xfe\xbe\x9e\xff\x4c\xa0\x97\x71\x7c\x79\xc7\x3f\x21\x7a\xff\xaf\x2c\xba\x6a\x44\x78\x18\xee\xfc\x3a\xf9\x4c\x44\x00\xfc\x22\xd1\x62\x9b\xc9\xec\x07\x72\x7c\x87\x11\x48\xef\x35\xc9\xd6\x42\xa1\xc6\x96\xfb\xb5\xed\xbc\xc7\xce\x66\x58\xee\xa3\xdd\x16\xcb\x91\x6b\x80\xc6\xa3\xd5\x18\xb6\x07\x02\x69\x8e\x94\x8a\x30\x6a\x5b\xd4\xf2\x05\xd3\xd1\x24\x04\x8c\x7d\x4a\xeb\xe6\x22\x16\xab\xaa\xdc\xdb\xb3\x85\x01\x72\x7d\x9c\xae\x6f\x9e\xac\x26\x63\xd8\xa6\x53\xa0\x41\x12\x9e\x93\x60\xb6\x6b\x6d\x39\x1b\x77\xd0\x66\x54\x63\x1e\x56\xdf\x88\xa3\x45\xa8\xd9\x5e\x80\x9b\xd5\x8d\xbf\x2f\xf4\x51\x5f\xf7\x6d\x33\xca\x25\x94\xa5\x23\xe3\xb8\x0c\xdb\x1c\xcb\xa7\xd8\xaf\x52\x26\x86\x09\xcf\xbb\x0a\x2d\x7d\xdd\x4e\xfa\x10\xd7\x5d\x2c\x27\xf3\xd4\x31\x12\x99\x71\x43\x88\x58\xee\x33\xe6\xe8\x1e\xa0\xaa\x3e\x2e\x56\xe3\xac\x78\xb0\x10\x82\x39\xc8\xc0\xb2\x29\x99\x21\x90\x7d\x24\x84\x80\xb2\x2c\x4d\x14\xfa\x6f\xf7\x1c\xf0\x76\x9b\x5a\x88\x18\xe8\xcd\xb1\x86\x71\x1e\x61\xd0\x9b\x8d\xd2\xdb\xc9\x5f\x0f\x84\x18\x9f\x8c\x0f\xc8\xa2\x34\x72\x06\x1f\x48\x2d\x93\xbf\xfe\xf4\xf2\xc7\xf5\x7a\x29\x09\xc6\x75\x05\x0d\x46\x69\x7d\xec\x50\x03\xa5\x6f\xe8\x7c\x25\x0b\x57\xab\xec\x7e\xcb\xd9\x51\x0c\x2f\x62\xa5\x90\x23\xd1\x0d\x99\x34\xf0\x12\x5e\x4d\x78\x29\x6b\x61\xd2\x9b\x74\xed\xb5\xaf\x41\x2e\x84\x7a\x8d\x36\x8d\x9c\x4c\x9f\x47\x6c\xf1\xb1\x1f\x3b\x64\xd1\x55\x93\xd5\xbc\x07\xf3\x36\x82\xb2\x34\xff\x75\xcc\xd5\xc9\x25\x2f\x96\x35\xb8\xd4\x6e\x0b\xc3\x2e\x00\x2f\x5c\x23\x40\x4a\x1d\xd8\x77\x7b\x71\x67\x78\xb5\x95\x95\x81\x14\xb3\xbb\x63\xdc\xcb\xdc\x5d\x45\x85\x9a\x27\x84\x92\x1a\xdf\xa1\xd2\x2c\xaf\x21\x49\xdc\xb8\x17\x97\x32\x32\x26\x15\x19\xdf\x2c\xe3\x1c\x9e\x43\x20\x32\x01\xbe\x2b\xf8\xe9\xf9\x01\x56\xf4\x20\xd9\x7a\xf9\x04\x05\x86\xc8\xc4\x37\xdb\xb6\x7f\xfc\xe8\xd1\xb7\x4e\x10\xc3\x0b\x27\x3c\x3a\xfa\x0e\x70\x12\x0e\x87\x5d\xeb\x03\x7a\x5f\x7c\x77\xf4\xdd\x56\xda\x44\xae\xca\xd5\xe2\xd3\x09\xa1\xfb\xcf\x3e\x21\x19\x8e\x0f\x13\xf7\x9e\x2e\x50\x59\xcf\x03\x66\xc1\x34\x64\xde\x1a\x69\x1b\x97\x5d\x26\xba\x69\xa7\xc5\xb3\x4f\xcc\x14\x82\x07\xfd\xfa\x46\xeb\x1b\xae\x8c\x23\x15\x8b\x94\x77\x45\xd9\xe9\xd0\x25\xd7\xb0\x95\x52\xbb\x89\xe0\x04\xdb\x4f\xdc\x13\x42\xb9\xf2\x85\x94\x75\x59\xb9\x21\x8d\xeb\x81\x5b\xbf\x69\x33\x3a\x13\x28\x94\xdc\xb4\xcf\x7d\x57\x1d\x61\x85\x2e\xd1\x6b\x04\x92\x3f\x6d\x2e\xdf\x0e\xea\xe8\xea\xd3\xab\xc5\x47\x7a\xda\xc7\x01\xaf\xae\x3e\xad\xbd\x22\x1b\x5b\x59\xdb\xa0\xbd\xb3\x0d\x1a\x38\x23\x91\xf9\xab\x4f\xf3\x9a\x31\xc1\x7e\x21\xf6\x04\x37\x59\x6c\x91\x90\x8d\x11\x1a\x43\x86\xab\xbb\x79\x79\x08\xc0\xea\x6e\x42\x9a\x79\x50\xe5\x68\x3e\xbd\x25\xac\xdf\xb3\xf4\xca\x81\x98\x24\x76\x51\xc3\x7c\xfd\x71\x32\x99\xd3\x0b\x34\xca\x48\x0e\x8c\xce\x23\x1c\x7a\xeb\x7f\x69\x3e\xce\xef\xe6\xd3\x35\x17\x60\xcd\x31\xb1\x28\x8b\x11\x36\xbf\x68\x34\xf2\x98\xac\x81\x2a\x91\xc6\x2b\xa9\x0a\xd4\x28\x4f\xe9\x35\x1f\x90\x9c\x0e\xa7\x6c\x36\x47\x40\xf0\x8e\x66\xc8\x9a\xbb\x1d\x7d\x7a\x01\x3d\x1c\xb1\x70\xc5\x23\xb2\xf2\x47\x15\xb3\x91\xb2\xf9\xb8\xec\xe8\xd3\x12\x75\xac\x34\x6f\x49\x35\xe3\x53\xa4\x60\x8b\xea\x4c\x6c\x72\x0a\xff\x92\x21\xae\x29\x59\xd4\x18\x21\x04\xe6\x8e\x18\xbd\x54\xee\xa5\x53\xeb\xc5\x05\xde\x6c\xa0\xdd\xcd\xe6\xb8\x37\x5e\xb4\xb8\x9f\xf4\x17\x28\xa2\xee\xef\x13\x1e\xca\x21\x47\xe5\x81\xdd\x96\xb8\x2a\xca\xd9\xc1\xe6\xb3\xde\x47\x20\xc7\x91\x4a\x87\x42\xd0\x25\x0e\xd8\x19\x3a\x14\x11\xc6\x76\x1b\x8d\xca\x53\x72\xbb\x7d\x70\x10\x4c\x4f\xb6\xe5\x02\x34\x0f\x7e\x95\xba\x3c\x26\xfa\x2b\x31\x38\x6f\xee\x7d\x8d\xbb\x91\xea\x37\x9a\x0b\x43\xe4\x03\xfe\x39\xef\x10\x58\x68\x3b\x74\x38\xe2\x89\x6e\xaa\x32\x14\x21\x76\xf1\xf9\xbd\x6c\xa8\x22\xd8\x49\x56\x4b\xa1\x41\xc0\x1f\x73\x04\xf2\xfd\x54\x84\xfe\x96\xcc\x90\x2f\xf8\x83\xfe\x1a\x15\xa2\x9e\x2a\xd5\xd2\xb4\x12\xcd\x8b\xde\x7e\xd8\x15\x1b\x67\xf6\xf6\x1d\x51\xca\x9f\xb2\xf4\xe8\x6f\xa7\x93\x61\x0e\xce\x36\xa0\x28\x7f\x1c\x44\x25\x3c\xfc\x24\x8d\x8c\x30\x0a\xa6\xbc\xba\x99\x5e\x19\x70\xb7\xf5\x5a\x2a\x7e\x0e\x1c\x67\x55\x9c\x67\x64\xf7\x12\xe8\x37\x1c\xd3\x7a\x9d\x45\x53\xae\x80\xea\xb1\xbc\x5a\x3a\x90\x25\xfd\xcd\xe9\x6f\x97\x9e\xa6\x89\xe7\x78\x0f\x84\x2a\x9a\x90\x45\x5d\x81\x82\xb3\x4c\x61\xbf\xec\xb0\x58\x45\xb3\x8a\x27\x7f\x32\xaa\x50\x56\x89\x76\x2c\x7b\x5c\x74\x46\x7d\xc5\xe3\xbe\xce\x64\x2e\x56\x20\x31\x7f\xc9\x3f\xe8\x11\xe9\x23\x59\xdf\x60\x59\xfc\x50\x5e\x8a\x23\x79\xa7\x8a\x0a\xe9\x47\x01\x2f\x04\x20\xb8\x3e\xd8\xb6\xd8\x0f\xfa\x70\xbb\xfc\xcb\x27\x9d\x0c\x0e\xc4\xfc\x24\xb6\x71\xe6\x66\x66\xc9\xba\xcc\x6e\x18\x4e\x1f\x5a\xac\x98\x39\x03\x66\x7d\x6f\xca\x8c\x7d\x11\x47\x55\x38\x8b\xef\x3e\xcb\x36\x07\xc0\x6c\x8e\xc2\x64\x75\x09\x98\xa1\x40\xd4\x51\x85\x38\xd8\x57\x76\x86\x63\x16\x55\x49\xc5\x4a\x3b\x93\xdd\xf1\x1d\xc3\x01\x67\xcc\x98\xa6\x7d\x60\x56\x62\xa8\x4d\xe7\x8d\x05\xbf\xd1\xd4\xa3\xfc\xb8\x39\x07\x1c\x40\x5b\x01\xae\x51\xaa\x2b\x73\xe0\x57\x10\x00\x08\x76\xaf\xe3\x63\xc2\xea\x00\x15\x5f\x8a\x9f\x36\x1b\xe5\xdd\x52\x75\xfd\x96\x9a\xe3\x0d\x4a\x8e\xa1\x82\x85\xa8\xdc\x42\x54\x5f\x5e\x08\x97\x45\xc4\x49\xe2\xf3\xd0\xc5\xcd\x6f\x8e\x47\x94\x9c\x47\x15\x97\xf7\xce\x61\x78\x68\x2c\x82\x80\x82\x9a\x74\xab\x20\x3d\x54\x3b\xf6\xf0\x7f\xb0\x76\x35\x02\x38\xf1\xf1\x6a\x2b\x81\x4d\x66\xc7\xf1\xb6\x54\x3f\xe5\x16\x76\x5c\x55\x3e\x27\x4a\xa7\xe4\x9c\x17\x28\x62\xb2\x87\x44\x36\xc8\x30\xa2\xcf\x3f\xc5\x51\x56\x34\xf7\x2d\x32\x18\x23\xe3\x53\x7d\x9b\x4f\xb4\x6a\x41\xaf\x76\x27\x49\xe1\xa2\x3e\xd8\x74\xc7\x68\x63\x1b\x86\xa6\xb2\x06\x2b\x84\x26\x35\xeb\xef\xba\x76\xcd\xb4\x70\x10\x76\x57\xc7\x5f\x9a\xbb\xa6\xa1\x74\x99\xb7\x76\xf3\xc8\x36\x63\x5d\xdb\x33\x80\x7c\xcd\x29\x64\x31\xeb\xbe\x69\x22\xcd\x90\xbd\x31\x55\x13\x19\xf8\x71\xfd\x52\xd7\x00\xf6\xe7\xb1\xf5\x13\x44\x31\x01\x37\x44\x24\x13\x7e\x10\xcd\x6c\x32\x74\x45\xfc\x87\xaa\x7a\x3e\x2f\x23\xcd\xdb\xde\x4e\x4d\xdc\x6a\x39\x99\xb8\xfb\xa0\xf6\xaa\x28\xb2\x45\x98\x07\xa0\x7e\xdd\x5c\x30\x26\xf4\xa9\xc5\x66\x05\x44\x56\xd5\xe5\xdb\x6e\x92\x5e\x4f\xe6\xcf\x3f\xb1\x28\x20\xfa\x5e\x6a\x6e\x3a\x54\xe5\x55\x95\xa6\x2b\x20\x56\xce\x16\x45\xd4\xed\xf5\x62\x67\xcf\xc5\x37\x32\x71\xfe\xd5\x4e\x71\x46\x87\xcb\x59\x06\xce\x8f\xd4\x4a\x31\x3b\xa0\x76\xfd\x74\x26\x88\xd7\x8b\xed\xae\x5e\x0a\x3e\x3e\x09\xf1\xa2\x1d\xb9\xf9\x7e\x40\x47\x96\x9e\x69\xfe\x0c\x3f\x45\x73\x9d\xcd\xcc\x00\x04\x85\x02\x8e\x7d\x85\x35\xed\x58\x9e\x7c\xbc\xf0\x98\x5b\x74\x1b\x62\xbf\x4a\x97\xca\x0b\x12\xe9\x87\x6d\x73\xdb\x60\xd6\x33\xb5\xa1\xf6\xe1\xb4\xba\xb1\x95\x62\x85\x70\x00\x17\xcb\x94\x6c\x4f\xa7\x34\x0e\x77\x6c\xd9\x01\x1b\x9b\xb5\xe0\xb2\x64\x82\x29\x2c\xe4\xbc\xf3\xd4\x1b\x32\x51\xb9\xd4\xcc\xdd\xea\x12\xf5\x43\x12\x14\xc6\xeb\xbb\xc9\x2e\x9b\x8b\xb2\xa5\xb6\x8c\x9d\x1d\x24\x6f\xa1\x17\xa1\x4c\x27\x79\x60\x58\xfd\x27\xe5\xbd\x86\xab\xe1\xd9\x7c\xac\x1c\x9b\xc5\x92\xa2\xfd\xf8\xe9\x6a\xf2\xa9\x6a\x5c\x16\x82\x3d\x61\x98\xf8\x18\x6a\x2a\xd5\x1a\x75\xd4\x35\xe1\x34\xb1\xfa\x2b\xf9\x4a\xd3\x19\x1f\xa4\xbc\x6c\xc3\xd7\x88\xa6\xfb\x74\x80\x2e\x2a\xce\x23\xf7\x2f\xcf\x83\xf8\xa3\xb8\xfa\x94\xe4\x83\xa1\x22\x5b\x0d\xce\xee\xc7\x13\x9e\xde\xbd\x23\xc1\xb2\x05\x74\x33\xe0\x82\xbf\x9d\xc4\x53\x9f\x09\x53\xbe\x5e\x62\x91\xbe\x72\x2b\x59\xe7\xb7\x01\xbb\xd1\x24\xc7\x95\x18\x27\x88\x78\xbb\x6d\x3e\x68\xb2\x90\x50\x93\xd9\x4e\x4d\x9a\x31\xc7\x40\x11\xea\xed\xe1\x4b\x95\x21\xc5\xcc\xcf\x3c\xdc\xf6\xce\xea\x5d\xcd\x25\x0f\xc3\x79\x1b\xc6\x38\xf2\xc6\x43\xd1\x5a\x3b\x82\xe9\x3d\xf6\xc8\xce\x8c\x3f\x37\x49\x41\x36\x9a\x2c\x67\x2c\x39\x0d\x13\xfd\x61\xd0\xbd\x91\xbb\xf9\x3f\x96\xb9\x25\x22\xe8\x72\xc2\x5d\xcb\xd1\xdf\x75\x2a\x79\x36\x9b\x3d\x2d\xe6\x58\xd5\x92\xc4\xe7\x8e\x44\xfa\xd0\x95\x2b\x1f\xa9\x1d\xf2\xcb\x66\xbf\x58\x85\x55\xdc\x2a\x0f\xec\x13\xee\xce\x8e\x95\xce\x79\x0c\x34\xbd\x66\xcf\x24\x62\x3f\xb8\x96\xc0\x9b\x69\xd8\xbc\x9b\x84\xc8\xa3\x8c\x7d\xfa\x2b\x2b\x53\x1c\xd1\x9a\x91\x73\x0e\x01\x1a\xb0\x6b\xa5\x70\x34\x0d\xb4\xe9\x03\xf4\x12\xe8\x05\x13\x94\x0a\xea\x31\x1e\x63\xcd\xf2\xdf\xb8\x3b\xc4\x5f\x7f\xe3\x8e\xf0\x36\xbe\x77\xed\x08\xbc\x08\x77\xed\x00\xbf\x55\x8f\x9a\xe9\x5d\x6a\xc3\x1d\x0a\xfb\xe5\x06\xda\x10\x69\xc6\xaa\x30\xcc\x4c\xee\xb6\xa6\x51\x7d\x5f\xcd\x00\xdf\x78\xba\xf8\x38\x2f\xf8\x06\x67\x34\x04\x2e\x70\x8c\xff\x79\xa9\xb1\x84\x95\x48\xec\x19\xbb\x1e\x90\x14\xc1\x5f\xb2\x1c\xaf\xf1\x17\x73\x6f\xe3\x9f\x6b\xda\x52\xfc\xeb\xbb\xb5\x49\xa0\xca\x38\x41\xea\xf2\x69\x52\xdd\x36\x76\x37\xc9\x7c\x87\x07\x7c\x4d\x3e\x88\xeb\x04\xa8\x0a\x73\x2c\xbe\x88\xa1\x50\x86\x1d\x0d\x5c\x09\x01\x01\x57\x13\x63\x50\x0d\x66\x0c\x18\x81\xaa\x5d\xcf\x14\x9d\xa1\x4f\x02\x11\xf6\xbb\x42\x77\xc4\x57\xf3\x76\x5b\x29\xf0\x86\xb7\x77\x2a\x03\x60\x8f\xfa\xa0\x74\x5e\x61\xbb\x54\xd8\x1a\xf8\xd7\xda\xd2\x71\x55\x71\x52\xa6\x79\xdd\xf5\x0d\x23\x71\x04\x64\xe9\x09\x84\xc5\xd5\x55\xff\xa8\xf0\x16\xe5\x65\x6c\x36\x77\xdf\x7e\x14\xf6\x03\xe1\xbb\xa1\x35\xb0\xb7\x55\x3f\x8e\x18\xd8\x12\xc3\x22\x4e\xf6\xae\xd9\xc4\xc8\x3d\x66\x17\xd8\xe9\x82\x64\xf6\x3b\x63\xd5\x2f\x89\x61\xd8\x87\xdf\x68\xae\x1b\x7f\x75\xc2\x73\xfb\x51\xd6\xed\x30\xda\x37\x67\x2e\x8e\xef\xcd\x1c\xb2\xa0\xc4\xb5\xd5\x0c\xb9\x5d\x72\x46\xea\x5e\x06\xed\x27\x76\xcb\x3d\x9a\x5a\x1c\x59\xd6\x9b\xdb\xac\x4b\xb4\x4c\x4e\x4b\x5a\x4b\xeb\xfe\x1e\x25\x85\xe1\x94\x2f\xaa\x74\xf9\x0d\x05\xdf\xbc\xc8\x0e\x1f\x79\xaa\x42\xe0\xaf\x31\x47\xf2\x29\xa6\x94\x89\x90\xf6\xc9\x50\xc4\x3e\x74\x89\xc9\x31\x52\x71\x6b\x04\xe6\xe8\xc5\x05\xa1\x17\xf1\xcd\x03\x7b\x38\xbd\xe9\xf7\x11\x64\x05\x38\x46\x31\xa5\x82\x48\xb9\x20\xf0\xa1\xd9\x45\xe1\xd5\x8a\x61\x8d\x72\xc0\xf2\xe0\x00\x40\x65\x2f\xa8\xd2\x9d\x0e\x22\x5e\x51\xac\x28\xe7\x7e\xf8\x07\xfd\x60\x64\xb7\xd6\x58\x10\x7d\x02\x41\xc7\x3d\x70\xcb\x26\x0d\x10\x8f\x8e\xb3\x78\xab\x95\x9f\x98\x33\x92\x66\xb6\x5a\xf2\xd1\x84\x26\xcc\x8e\xbf\xcd\xa3\x8c\x76\x1e\xe5\x49\x41\x51\x86\x17\x63\x14\x84\x7d\x21\x65\xd3\x70\xfe\x1b\x6a\xcf\x82\x36\x09\x41\xb5\xef\x1b\x88\x2e\xba\xca\xa4\x81\x4c\x9f\x31\xd8\x6d\x70\x1e\x1f\x26\x20\x70\x67\x8b\x8f\xc5\x1f\x8e\x8e\x00\xf8\x56\xeb\x02\x45\xd2\x1c\x97\x84\xfc\x0c\x85\xac\x01\xff\x9e\xf3\x9b\xad\x59\x0a\xb0\x1d\xef\xb0\x67\xa9\x32\x9d\xe8\x8d\x2e\xdc\x5e\x4d\x76\x41\xe4\x2a\xbe\x62\xca\x7a\x9b\xc9\x66\x08\xa8\x49\xb6\x83\xe9\x9b\x73\x6a\xfc\x80\xb2\xc6\x26\xa8\x32\x0f\xf4\x88\xbd\x69\xfd\x7c\x78\x4f\xc2\x91\x1b\x77\x07\x20\xb8\x25\xe4\x57\xb1\x31\x4d\x77\xde\x89\x45\x3a\x55\x23\x8a\xe3\x05\xf9\x0a\x45\xa9\x9d\xfc\x62\xf1\xa9\xbc\x47\x5a\x05\x19\xde\x57\xeb\x02\x66\x9a\xc6\x24\x5e\xd6\xd1\x09\xbe\xf5\x87\x18\x98\x98\x27\xb5\xaa\xc5\x65\xe6\xe8\x2c\xaa\x16\xa5\x39\x63\x49\x50\x9d\x0d\xeb\x17\x9b\xba\x40\x53\xdf\x4f\xad\xcb\xb2\xeb\xc9\xfa\x07\xf4\xc4\x01\x70\xe6\xc9\x6c\x0a\x85\xdf\xc2\xe5\xd5\xe4\xb8\x0c\xd6\x17\x7b\xbf\xbb\x10\x2a\x1b\xc1\x18\x4b\xb2\x56\x8d\x8f\x7e\xd4\xdb\x9c\x86\x0b\x25\xbb\xf0\xdb\xc1\xe7\xc0\xee\x72\x74\x3d\xf9\x6f\x9e\x32\x92\x66\x7d\x46\x5c\x24\xe5\xcb\x64\x07\xda\xdd\xee\x25\xd5\x7d\x86\xe2\x6b\x47\x19\x4f\x18\x56\x84\x01\x5f\xd3\x5f\x9b\x6b\x22\xe3\xe2\x71\x55\x18\x89\x75\x6d\x33\xac\xc8\xed\x69\xd9\x32\xf7\x6e\x25\x77\xb0\xd6\xa7\x62\xb8\x4f\x7d\xa2\x19\x6e\x1e\x1b\x44\xd5\x94\x24\xeb\x25\xf8\x58\x39\xbd\x24\xdd\x28\x67\xef\xd6\x98\x58\x75\xa6\x64\xcb\x64\x35\x99\x91\x02\x7b\xc2\xfb\x05\x10\x56\x18\x32\x32\xc1\xd9\x52\xfd\xdd\x8a\xd6\x59\xde\x53\xd9\x34\x3d\x44\x72\x4f\x4b\x49\x96\x31\xa4\x99\x58\xeb\x45\x06\x57\xad\x7b\xa4\x2f\x2d\x19\x88\xcf\x55\xcb\x41\xc6\x77\x49\xec\xe1\x12\x0d\x3b\x4e\xd4\x7c\x6f\x99\x7a\xe7\x66\x66\x44\x48\x5a\x4c\x3f\x4d\xc6\x36\xce\x03\x4d\xf5\x01\xc9\x7c\xcc\x7c\xe0\x7a\x66\xba\x30\xcc\x1e\x1f\x1c\xcb\xab\xd1\x3d\x71\xfe\x5d\x93\xf7\xdb\x5e\xad\x1f\xfd\xd4\xe6\xd0\xb1\x7b\xab\xbc\x39\x4f\x5e\x69\x72\xe1\xbe\xd3\xc9\x0c\xe2\xc9\xce\x7d\x91\x4a\x09\x63\x95\xdb\xf5\x33\xc3\xb7\x4d\x2d\x5a\xcb\x40\x3b\x8c\x76\x66\x23\xc2\x40\x40\xc7\x68\xa1\x3b\x7d\x9c\x91\x3a\x17\x9e\xfa\x65\x84\xa3\xc3\x74\xa7\x53\x2d\x44\x7a\x97\x31\x54\x0c\xdb\xf8\x03\x57\x8a\x8e\x14\xf7\x3f\x2e\x3f\x63\x63\xfb\xfa\x9c\x06\xfd\xb7\x29\xa6\x0a\x3a\x4c\x32\xc6\x2c\x4f\xd0\x48\xe1\x75\xe2\x3d\x8b\xf4\xb5\x18\x25\x98\xce\x33\x1e\x5e\xe8\x12\xe0\x16\xe2\x28\x8f\x82\x18\xd6\x9e\xf3\x91\x67\x6e\x2e\x54\x65\x61\xb0\x27\x38\x34\x75\xe8\x0d\x71\x51\x73\xfe\xc8\x99\xa7\x2a\x5b\xfd\x01\xc0\xa9\x6c\x18\xb7\x1b\x1f\x38\x9a\x7d\x39\xee\x0f\x41\xb2\x22\xb5\xbd\x91\x97\x1b\x13\x03\x3b\x4d\x6a\x31\x49\x5e\xd0\xcd\x79\xaa\xb5\x65\x10\xbe\xb3\x15\x32\x34\xcb\x1b\x8c\xcc\x66\x73\x95\x65\xc1\xe8\x69\xad\xed\xb0\x6a\x55\xb3\x29\x64\xd8\x0b\x6c\x0b\x99\xd8\xaf\x41\x0d\xb4\xd4\x5f\x51\x05\x6e\x05\x5f\x87\x40\xf0\x85\xeb\xc5\x41\xdc\xad\x83\xda\x64\xb3\x25\x62\xf2\x4f\xb8\xa7\x90\x7b\xe1\x3b\x71\x50\xeb\xd6\xae\x2a\xc8\x97\xa1\x88\x69\xe4\xb6\xbb\x5f\x69\xa3\x57\x9f\x9a\x1f\x5e\xcd\x07\x74\x2a\x6c\x36\x4f\x47\x35\xae\xaf\x2e\x2e\xa9\xaa\x2a\xec\xdf\x31\xdb\xc1\x55\x91\x05\xbd\x6b\xd8\x0b\xfc\x61\xcc\xf5\x7d\x45\xcf\x9b\x48\x7c\x77\x31\x16\x89\xb9\x39\x93\xdc\xdd\xbd\x1c\x2f\x77\xb3\xb5\xa6\x2d\x16\xcb\xd8\x12\x3b\xd1\x05\x00\x93\x0e\xff\xfb\x50\x8d\x4d\x43\xb4\x79\x7e\xa8\x59\x5d\x33\xd6\x78\xbf\xec\x3f\x52\xda\xfa\xa0\x76\x72\x43\x9c\x22\x70\x53\x11\x79\xa3\xea\x43\xd6\x3e\x76\x06\x09\x4e\x0c\xc3\x3f\x7e\x5c\x46\xcb\x5a\xdd\x5d\x53\xa5\xfd\x65\xc7\xab\xfa\x91\x4a\x05\x0e\x33\x41\xff\xa7\x7d\x2f\xd5\x94\x19\x44\x23\x25\x47\x52\x39\x66\x80\xdf\xa2\x9e\xe9\x0c\x29\x94\x2c\x68\x00\x0d\x14\xe4\x2c\x68\x0c\x08\xbe\x1f\x74\xcd\x6f\xa4\x78\x57\x8a\x57\x53\x7c\x13\x3a\xd7\x28\x74\x62\x0b\x71\x90\x12\x73\x4d\x02\xd3\x7b\x5c\xde\x19\x51\x27\x37\xe7\x49\x87\xf2\x89\x4a\x02\x5b\xa4\xc3\x07\xf9\xc5\xdd\x5a\x13\x4d\x9d\x42\x2a\x3c\x23\x1f\x1f\x18\xfb\x2a\x32\x05\xab\x71\x66\x0f\xf0\x81\xb6\xe2\x49\x97\x37\x80\x9c\x92\xdd\xef\x78\xc8\xe4\x8e\xd7\x37\x61\x9e\xa5\x1c\x83\x9d\xab\x53\x77\x5f\x61\x6e\x69\x4b\x9c\x80\xa9\x2b\xc0\xbd\xa3\xbe\x40\x15\x18\x17\x43\x3a\xa3\x28\xf1\xa5\x5d\x49\x33\x62\x3a\x0f\x7b\xa8\x17\x1b\x0d\xb4\x5b\x94\x7d\x83\xec\xdc\x72\x09\x23\xa4\x3c\xb3\xc3\xe2\x4f\x65\xe4\xa6\xb1\x9f\x2a\x29\x50\x43\xee\x89\xda\xbf\x1d\x7d\x62\x64\x12\x9d\x97\x93\xfa\x02\xec\x32\xa9\x0d\xa9\x88\x38\xca\xe4\x65\xf8\x11\xe4\x6d\x88\x0a\xfa\x97\x65\x45\xcd\xa1\x62\x3f\x06\xd9\x34\x43\xe2\x8f\x26\x78\xd2\x8d\xe6\x4f\xfd\xda\x70\x94\xdb\x04\x7d\xb1\xef\x6f\xb4\x25\x35\xc9\x9d\x00\xef\xc0\x8c\x1b\x50\x71\xf7\x7d\x81\xb3\x79\xa2\x8b\xe6\x59\x54\x5c\x1b\x8a\xbe\xe2\x6f\x77\x74\x3b\xb6\x61\xa9\x48\xa3\xd2\x84\x4d\xe4\xe0\xcb\x4a\x83\xb3\x1e\xce\x0c\xfd\x50\xb1\x46\x9f\x87\x23\x72\x5d\x2c\xaf\xf4\x99\xdd\x03\x52\x35\x69\xe1\x03\xd1\x25\xaa\xfd\xaa\x31\xb9\xdb\xe9\xfc\xd9\x6a\x85\x56\xcd\xee\x66\x9e\x83\x1f\xb1\x5f\x6e\x61\x37\x02\x44\x46\x54\x11\x6d\xe7\xb8\x73\x82\xb7\x35\x1b\xaa\x2e\x93\x41\xd2\x91\x5a\xfa\xfc\xd3\x81\xfd\x8d\x0a\xd2\x1d\x2c\xd3\x49\x86\x2d\x34\x41\x78\xbb\x44\x3c\xda\xd4\x70\x3c\x74\xb1\x27\xab\xeb\xca\x9a\x39\xa5\xb7\xac\xe9\xd5\x67\x43\x44\x5f\xbc\x77\x5d\xb4\x68\x2e\x44\xf7\xe1\x0f\x60\x01\x6c\x23\x26\xf5\xb6\x35\x0e\x5b\xe7\xf7\x83\xf3\xea\xfc\x74\x88\x36\x90\x51\x5f\x7b\x5a\xfd\xac\x93\xc2\x25\x13\x37\x49\x09\x24\x4a\x0d\x5a\x67\x81\x5a\xb1\x5d\xd7\x17\x8a\x67\x67\x7b\x32\x29\xa5\x78\x32\xd0\x51\xf8\x96\xcf\xef\xcf\xc7\x9d\xf3\xed\xe1\xb5\xb9\xc0\x50\x18\x57\x1e\x4f\x57\xd7\xc2\xfe\xef\x50\xac\xd8\xb2\x3d\xce\x0f\x8e\x1d\x38\x60\xe6\xfb\xa3\xef\xed\x0c\x09\x6c\xea\x23\x17\xa1\xb4\x09\x03\xc9\x3d\xcc\xcd\xd4\x40\xa6\xac\x0f\x7f\x9a\xa7\xa6\xff\xd0\xdc\x50\xd1\xc6\xb9\xd1\x4a\x8b\xf5\xe2\xdf\x2b\x69\x24\x13\x33\x3f\x59\xae\xf3\x22\xbf\x9d\xe4\x7c\x4e\xf6\x36\x0e\x0f\x49\x47\x09\x79\x31\xd7\x40\x4d\xad\xde\x03\xf6\xb3\xba\x3e\x4c\x3a\x1f\x26\xab\x8a\x9e\x69\xe1\x78\x75\x92\xc3\xfa\x3e\x3a\xf4\xfb\x28\x9f\x96\x8f\x7a\xd3\xef\x63\x48\xcd\xde\x97\xa3\x86\xd1\x98\xd2\xb4\x9f\xf4\xa1\x74\x1b\xcd\x32\x2c\xa1\xe6\xe9\xc1\xa3\x9d\xe6\xe6\xfd\x22\xfb\xed\x39\x05\xa0\xe3\x04\x72\x26\x1f\x5b\x64\xc2\x4c\x97\x1e\x60\x81\x3b\x47\x62\x3d\xfe\xe5\xf4\xd7\x09\x6f\x53\xf4\x47\xcb\x1c\xe2\x8b\xf7\x9b\x8d\x83\xca\x98\x26\xbc\x93\xbd\xe3\x9e\x38\xa6\xc6\xd7\x28\xdc\xbf\x32\x18\x69\xef\xb8\xa4\xc2\x46\xcc\x43\x96\x7e\xef\x28\xdc\xa7\x58\x3d\xd3\xb8\xfc\x85\xb6\x3e\x94\x3f\x59\xe3\x95\xeb\xd5\xe6\x6d\x17\x73\xe8\xe0\x98\x3c\x39\x5e\xbc\xf7\x63\x82\x5d\xfe\x0c\x2f\x69\x88\xcc\xa7\x24\x31\xba\x58\xa9\xc9\x4e\xaf\x3c\x04\x03\xc5\x66\xf1\x29\x36\x3c\x92\x99\x51\x2b\xc2\xef\xc4\xb1\x7c\xe1\xfe\x84\x78\xc0\x50\xb9\x69\xf7\x89\x50\x5e\x3f\x70\xe4\x37\xa3\xea\xf5\xc7\xb9\x7a\x54\x24\x73\x59\xed\xb6\xf6\x44\xec\x21\x88\x1d\x04\xc8\x4e\x1a\x4a\x39\x66\xe2\x87\x5d\xee\x58\x57\x46\xc1\x35\xca\xc7\x5e\x59\x4a\x28\x33\xb1\x69\x6d\x90\xae\xa2\x78\x79\x75\x6c\xe5\x51\x0f\xfe\x7e\x6f\x16\x0f\x3e\x61\x2f\x7e\x75\x27\xa3\x39\xfa\xdf\x1a\xb5\x62\xe6\x76\x0d\x2b\x52\x3c\xfa\x8f\xc9\xe7\x8a\xb7\xa5\xac\x1a\xb2\xe4\x9d\x04\xec\xd7\xb4\x8f\x65\x98\x27\x6d\x9b\xa2\xd8\x8a\xf4\x31\x6a\xdb\xe6\x94\x5a\xde\xb1\x79\x4c\xd3\x55\x19\xf5\x91\x1d\xb3\x7f\x4f\x75\x9b\xe3\xfd\xe0\x0c\xe0\x49\xa5\x59\x60\xfb\x14\xa6\x7b\xbe\x5f\xab\x09\x42\x9a\x89\x78\x5a\xd0\xea\x9e\xcf\xeb\xb7\x1e\x23\x08\x64\x9d\xcd\x67\x33\x0e\x31\xcc\xb9\x9f\x43\x27\x7e\x9e\x8e\x0d\x91\x37\x9e\x5e\x4f\xd7\x02\xda\xef\xa6\xe3\xe0\x59\xb7\x47\x70\x41\x5e\x78\x73\xca\x89\x79\xf4\xd1\x14\x15\xd9\xd1\x02\xd7\xc9\x3a\x05\x7a\xf5\xf7\xff\x56\x96\x94\x45\x79\xad\x3e\x67\x99\x9c\x24\x39\x56\xce\x4e\x20\x12\x92\x84\xf8\xd3\xd1\xde\xce\xfc\x0c\x2d\xba\x57\xab\xc5\xed\x13\x69\x24\xa5\xbc\x9d\xe3\x2c\xac\xc8\x36\x72\x94\x6c\x7d\x55\x4e\xb6\x18\xa2\xc3\x32\x66\xb7\x4d\xd6\x3f\x8e\xaa\x1b\x58\x4a\x5a\x77\xb8\xea\x6e\xe8\x62\xde\xdf\xbf\xe1\xe8\xf2\xa6\x90\x77\xe0\x20\xda\xd7\x20\xac\x9a\x71\x25\x10\xe6\xa6\x84\xa0\xcf\xa8\x2b\xab\x27\xd6\xa3\x0b\x21\x92\x00\x7f\xe0\x8c\x43\xd1\x76\xdb\x42\xb1\xa6\x15\x86\x4c\x74\x86\xc4\x23\x33\x62\x77\x66\x18\x90\x0a\xc3\xc8\xe1\xc7\xc0\x7a\xbe\x2c\xfc\x0b\x19\xf2\xe4\x5e\x70\x64\x7e\x7c\x94\xd9\x9c\xe4\x46\x5a\xd8\x2c\x91\xaf\x47\x19\x2a\x5c\x2b\xa9\x04\x0d\xda\xb5\xcd\xef\x1d\xb8\x2c\xb8\xf8\x36\xcb\x14\x69\xf5\xdb\x6f\x81\x24\xdd\xbd\x69\x91\x14\x17\xe1\xd8\xee\xbb\x76\xf6\x7d\x2a\x0d\xf1\xb9\xf3\x2b\x55\xc7\xf6\x3c\x45\x0b\x53\x61\x6f\x37\x8f\x17\x04\x65\x0d\x7e\x10\xba\x9f\xb2\x65\x9f\x7e\xa1\xe4\xde\xce\x92\x22\xb4\x16\x76\x97\x59\x8c\x14\xd7\xf0\xce\x5d\xab\x43\x2e\xca\xb0\xf1\x46\x97\x59\xb6\x94\xf8\x9b\x0c\x4b\xc5\x77\x69\x7d\xa4\xf8\xa0\x1f\x96\x19\x70\x07\x5b\x98\x34\xc4\xa2\x82\x88\x89\x14\xa3\x00\x94\x18\x8f\xd8\x51\x09\xa5\x7d\x55\x2d\xcf\x83\x9d\xee\x2a\xaa\x13\x26\xb5\x31\x88\x16\x51\x73\x07\x38\xf1\xab\x7a\x60\x10\x9e\x7b\x7f\x33\xf1\x35\xac\xf4\x09\x7f\xa9\x53\x0d\xfe\x1a\xcd\x26\x2b\x49\x30\x0f\x94\xc1\x8a\x5e\xc2\xd5\xd4\x5c\xf1\xfe\x04\xf2\x9e\xb0\x09\x20\xfa\xfe\x48\x58\xa9\x9d\x18\xd4\x99\x31\x65\xdd\xd0\x30\x61\x48\x5e\xea\xc3\x91\x11\xc6\x6f\x2a\x50\x43\x25\x88\x96\xf9\x3d\xc9\xf2\xb6\xf8\xe3\xd8\x72\x9b\x0d\x7d\xe2\xb8\x28\x80\xde\x16\x4c\x4d\xc8\x23\xdc\x8d\x59\xb1\x94\x35\x5d\xd3\x11\xc8\x6b\x80\x63\x2c\xc7\x33\x9b\x56\xec\x30\x02\x0b\xf2\x5d\xdd\x7c\x67\xd6\x0a\x21\xcc\x93\x72\x16\x76\x5d\xce\xee\xc6\x93\x0a\x60\x2c\x6c\xba\xdc\x4c\xd9\xc1\xf1\x5e\xa9\x46\x22\x7c\xa2\x2d\x19\xa7\xf1\xab\x27\x7c\xa9\x71\x89\xcc\x79\xe3\x31\x91\x34\xd5\x3d\xe3\xec\x8e\x30\x7d\xcc\x60\xd1\x00\xca\x86\xb6\xfb\x30\x05\x6f\x7b\x25\x96\x7a\xda\x3b\xdf\x13\xca\xf3\x96\x6c\xa3\x48\x6f\x0c\xef\x84\xaf\xe8\xb0\xb3\x9c\x1c\xd0\x5f\xe4\x94\x84\xbb\x11\x89\x4d\xe5\xd1\xd9\x81\x6d\xf9\x39\xad\x16\x77\xab\xcb\x49\x0e\x13\xb7\x9e\xce\x59\x30\x94\xee\x7c\x3d\x0f\x9c\x4e\x38\x3b\xef\x63\x89\x10\xab\x50\xf3\xeb\x9f\x98\x22\x4f\x2e\x97\x1f\xd1\x89\xc5\x93\xd1\xfc\x77\x6b\xaa\x7a\xaf\xf5\xd3\xe8\x57\xd8\x97\xf8\x31\x9d\x54\x2d\x38\xbc\x5c\x67\x6b\xb1\x6a\x51\x5d\xd0\xe1\x6a\x8d\xc6\x68\x2b\xd8\xa7\x64\xf9\xa1\x52\xd5\xf3\x2e\xe3\x08\x71\xb7\xb8\x71\x98\x4d\x9b\x50\xef\xc9\x34\xea\xc8\x29\x15\x6b\x8d\xe6\xe3\x96\x29\x08\xd3\x3d\x91\x0b\x08\xf6\x9b\xb4\xa8\x00\x4d\x87\x79\x6f\x0a\xc8\xba\x8a\x04\x87\x5f\x74\xce\x6b\x57\xdd\x16\x62\xed\x25\x3f\xd7\x44\xa4\x91\x01\x2d\xc5\x18\x4c\x66\x8f\x39\xe8\xf9\x31\xa9\xcd\xf8\x80\xc8\xa9\xf9\x6c\x84\x1e\x6c\xb3\x18\x5f\x96\x11\xc5\x19\x4b\xdb\x35\x8c\xc8\x7a\x16\x9f\x30\xad\xdf\x38\xbb\x5f\x3e\xb2\x94\xfd\xc3\x3f\xac\x9c\x6d\xe7\xaf\x6f\xf3\xda\x0d\x37\x18\x22\xf9\x4e\x17\x50\x53\x56\x24\x5e\x4d\x22\xbe\x38\xa1\xba\x2d\xb2\x83\x1d\xc8\xdf\x55\x2e\x48\xee\x4a\xae\xc2\xdd\xce\x12\x41\xdc\xd2\xe6\xbe\xdd\x03\xea\xe2\x0e\x95\xc9\x64\x10\xc7\x1b\x00\x50\x8b\x8f\x4f\xa8\xd0\xea\x32\x27\xfc\x0f\xfe\x20\x0a\x47\x66\xb1\xe3\xb9\x5f\x5d\x66\xf0\xaf\x99\x56\x49\xf6\xf7\x51\x5f\x05\x4d\x14\x57\x77\x17\x70\xd5\xa7\x47\xf9\x23\xea\x9f\xa2\x7a\x50\x54\x96\x46\x3b\x65\xf1\xbb\xc9\xdf\xee\x46\x33\xa0\x45\x8e\xf3\xc5\x23\x3a\x25\x0b\x22\xc5\x1f\x29\xdd\x4e\x0a\xa5\x4c\xdc\x97\x8b\x63\x96\xd5\x0d\x32\x1c\x13\x51\x8c\x4a\x97\x8b\x63\xb8\x87\xd0\x6d\x82\x2d\xed\xa9\x7e\xdc\x71\xf8\xef\x74\xb2\xce\xd7\xc7\x4e\x9c\xed\x38\x5f\x3f\x72\x1f\x8f\xc8\xda\x1e\xf4\x60\xfd\x28\x40\x77\x8e\xd9\x58\x84\x23\xfd\x8f\x33\x01\x34\xb4\xcc\xf8\xe9\x38\x65\x1c\x83\x53\xb0\x38\xf6\x2b\x8f\x9d\xf2\x5f\x7c\x56\x65\xa9\xa1\x34\x90\x9c\xee\xeb\x91\xab\x0b\xca\x7b\x4e\x12\x55\xe0\x3f\xb9\x06\xb9\x9c\x8f\x0d\x84\x83\xf2\x96\x03\x72\x1c\x7c\x3d\xb2\xec\x8b\x47\x86\x37\xc2\xd3\x52\xca\xda\x2b\x8d\x7a\x8c\x17\x40\xe2\xd6\x17\x29\x26\xa2\x96\xda\xed\x3d\xcb\x7e\x38\xe6\xe5\xe5\x29\x72\xeb\xc9\xb4\xf2\xe2\x91\x24\x46\x4d\xf1\xd6\xd8\x3b\xda\xda\xf6\x1e\x61\x7b\x7b\x9c\x61\xd7\x6e\x6b\xe8\x8c\xb4\xb1\x17\x58\x76\x0b\x7a\x58\xeb\x84\x6e\x8f\x6d\xb0\xaa\x76\x46\x20\x21\x55\x6e\xd1\xb1\xca\x0e\xd1\x22\x48\xf8\xde\x73\x25\x84\xbd\x83\x0c\x1a\xe1\x48\x7c\xcd\x44\xb8\x3e\x6c\x35\xca\xde\x6e\xe4\x17\x8d\x6e\xc2\xe3\x9c\x7e\x1e\x31\x12\x61\x7c\xeb\x61\x9a\x7a\x50\x23\x86\x26\x63\x1e\x41\x6e\x4b\x2f\x62\x16\x64\x5e\x22\x21\x57\xb1\x72\xf3\x8b\xa0\xca\xa0\x92\x20\x0b\x49\xd0\xb8\x9a\x2e\xc4\x95\xd5\x95\x13\x86\x45\x3f\xda\x9f\x43\x9e\xb2\x3a\xec\x7a\xd4\x0f\x1a\x16\x0a\xf2\x51\x56\x38\x6c\xcb\x2e\x15\x54\x88\xae\xf9\xdc\xe5\x0a\x87\x92\x0f\x46\xff\x6a\x5e\xb8\x56\x94\x2f\x5b\x27\xa6\xe2\xe6\xa1\x98\xa0\x96\xd4\x5f\x5f\x43\xd3\xbc\x69\xe7\xd0\x0f\x45\xb1\xa3\x60\xb6\x6d\x78\x87\xfe\x42\xab\x46\x71\x0a\x12\xa8\x31\x8c\xb7\xac\x0d\xe6\xee\xbe\x65\x6e\xf1\x2a\xf4\x07\xca\x3e\x53\xe9\xb3\xb7\x83\xa4\x62\x96\xde\x3e\x62\xd4\xc1\xe1\xc0\xb7\x49\x7f\x2a\x0a\x07\x09\xbc\x33\xd1\x32\xd9\xff\xcb\x8b\x57\x4f\x5f\xff\x45\x7c\x88\xb6\xdb\x4a\x2f\xa8\xd3\x74\xc9\xf5\xf4\xf5\x93\x9f\x7f\x7a\xf6\xea\x8c\x78\xd3\x04\x6a\xd4\x89\x7b\xca\x19\x4e\x9f\xbc\x7e\xf3\x2c\x21\x44\x2d\x1e\x17\xe1\xd1\x4b\xe8\xfb\xfa\xf3\x43\xc4\x2c\xe4\xea\xfb\xce\xd6\x1f\x03\xf2\x70\x96\xa4\xc6\x7e\xd2\x6a\x25\x05\xbf\x17\x79\xbe\xd8\x6a\x71\x4b\x2d\xa3\x3d\x2a\xf3\x8e\x21\xf0\x93\x62\xfb\xd4\x02\x31\x14\x38\xa2\xc0\xbf\xb6\xef\x3f\xf0\x13\xa4\xd2\x67\x70\xac\x65\x8e\x8e\xf6\x64\x72\x1c\x34\xa0\x65\x2a\x67\x68\x86\xfb\x12\x4d\x36\x27\x49\x47\xf0\x5b\x7e\x46\xdb\x4b\x93\x2b\x18\xe9\x87\xcd\x26\x39\x92\xdf\x2b\x80\x0f\x13\x09\xcf\x17\x1a\x90\xdf\x01\x92\x47\x1f\x18\xd5\x6a\x49\x15\xc7\xbd\x80\x65\xe0\x4f\x36\x9e\x53\x18\xd6\xd9\xe8\x5a\xfd\x0a\x67\xf7\x12\x28\xdf\xff\xed\x25\x50\x26\x2e\xbe\x7b\x39\x43\xc3\xa7\x6c\x3d\x45\x22\xd9\xc1\x1d\x32\x97\x9c\xd1\x94\x2d\x8e\xe7\xec\xd9\x5f\xcf\xde\xbd\x7a\xfd\xf4\x59\xf9\x2d\xbd\x26\xfe\x08\xd9\xb4\xbe\xd8\xfd\x9d\xab\x9f\xea\xe2\xea\x43\x27\xc8\x83\xa3\xa1\x63\xaf\xc3\xde\x72\x95\xf7\xfd\xa4\x69\x23\xfc\x6c\x8e\x21\x31\x91\x7e\xf8\x4b\xfa\xfd\xe0\x97\xc7\xc3\xce\xe3\xec\x30\x43\x07\x0a\xee\x5d\xe5\x97\xef\xd1\x5d\xe5\xf9\xc1\xb0\x93\x1d\x46\x6f\x3f\xb9\xf3\x72\xac\x7b\xee\xfb\xa4\xe3\x1b\x73\xa9\x5b\x33\x6e\xe9\x72\x43\x97\xec\x61\x5d\x7d\x7e\x3a\xd9\xe1\x41\xf7\xde\x0c\x7c\xbc\x2b\x93\x99\x67\x5f\x2b\x6d\x44\x40\x5b\xff\x8b\x3c\xfc\xfe\x2a\x01\xc7\x45\x7e\xe7\xd0\x66\xa4\xf2\xf0\xaa\x8e\x68\x55\x57\x84\x2d\x0b\x8a\xf5\x85\xb6\x75\xe0\xea\x2b\x85\xad\xac\x1f\x70\x80\x5d\xf5\xa5\x46\x6a\xf1\x12\x8a\xe3\x25\xd7\x3c\x64\x57\x8e\xcc\xae\x79\xe6\x13\x5e\xd6\x99\x07\x5d\x41\x82\x14\x38\x1e\x66\xfd\x2f\x55\x0a\x79\x0a\x40\xe1\x94\xf1\xde\x37\x2f\x2f\x7c\x99\xf6\x35\xc4\x74\x0a\xca\x8f\x14\x1a\x55\x0e\x1c\xc3\x1e\xe2\x87\x3e\x1e\xe1\x13\xd0\xe0\x01\xe7\x7e\xbd\x70\x13\xef\x38\xf7\x6c\x1e\xe5\xab\x78\x02\xcc\xbd\x0e\xb8\x49\x7d\xcd\xce\x39\xae\xfc\x23\x21\xe4\x91\x45\xa0\x16\xb8\xeb\xf2\x44\xb6\x9a\xb2\x1c\x09\xae\xf2\xde\x51\xd6\x31\xd9\x45\x9c\x21\x29\xcc\x8b\x9a\x66\xf7\xb9\x48\x7c\x0b\x85\x5d\xbf\xa2\xee\x0f\x5f\xaa\xf6\x83\xad\x91\xcd\xc5\xe8\x05\xc7\x1f\xce\x2d\x72\x91\x24\x06\xa5\xd7\x5a\x4e\x27\xd7\xb7\xb2\xe9\x3d\x23\xb6\xd6\x04\x36\xe0\xcf\xf3\xbf\x3e\xfa\xc3\xe1\xf5\x34\x0f\x7c\x2d\x1f\xfe\xeb\xb7\x4f\x29\xb2\x0c\x22\x1f\xfd\x40\x91\x1d\xcb\x14\x6f\xa8\x7e\x79\xb9\x7e\x46\xb1\xa7\x4b\x28\x57\xc5\x3d\x89\x4e\xa7\x6d\xe0\xbb\x23\x6a\xe0\x7f\x86\x5d\x39\xa1\xc8\x22\xec\xca\x77\x87\xd7\x39\xdc\xbf\x41\xdc\x13\xca\x98\x87\x91\xe8\x58\x3f\xea\x51\x3f\x81\x68\xa8\x30\x18\x88\x3c\xdc\xbe\x98\xc3\x29\x14\x00\x9a\x5f\x2c\x16\x6b\xb8\x10\xd1\xfd\x88\xcf\x17\x82\x5f\x05\xe9\x2c\x4d\x4e\x28\x09\xef\x01\x49\x27\xb8\x0e\x65\x54\xb2\x84\x9f\x7d\x73\xe7\xd4\x7e\x20\xa1\x21\xa9\x9e\xa3\x59\x97\xf9\x75\x01\xf9\x61\x18\xf3\xeb\x03\x0e\x7c\x3a\x70\x41\x54\x30\xd3\xaf\x61\xfe\xea\xcf\xef\x4e\xde\xbc\x79\xf7\xe4\xe5\xc9\xe9\xe9\xbb\xb7\xcf\xfe\xfc\xec\xaf\x6f\xca\xc3\xf3\x6a\x7e\x3d\x28\x00\x40\x43\x9e\xb4\x38\xaf\xbe\x21\x78\x3d\x7e\x07\x00\xbb\xd7\xcf\xfa\xe7\xd5\xa1\xe3\x16\x50\x93\xa1\x18\x53\x46\x7e\xd9\xd5\xc1\x0b\xfa\x9c\xe4\xf1\x3a\x29\x1b\xa0\xa9\x64\x2c\x3f\x7c\x7e\x31\xe6\x32\x59\xee\xdd\xd0\xe8\xd4\xc3\xfc\xe6\xc9\xf9\x39\x2c\x9b\x0e\xb6\x4b\x62\x1f\xa7\xe2\x91\xf5\x64\x86\xfe\xa3\xb4\x27\xbb\xb2\xa4\x49\x97\xe5\x63\x32\xe9\x08\x00\xd7\xaf\x2d\xd2\x91\xe6\xbf\xbe\xe0\x40\x0b\x0e\x7d\x31\x3c\x89\x51\xc9\x48\x6e\x89\xb6\x01\x52\x20\x7e\x99\x05\x49\x9f\x8d\x2a\xf2\xa5\x53\x26\x2d\x38\xeb\xd2\xac\x8b\xed\x40\x6c\xce\x0e\x88\x1a\x16\x92\x0d\x44\xb9\xcc\x19\xbb\x2a\xea\xa7\xbe\x91\x72\x12\xec\xa9\x32\x55\xaf\x46\x7c\x1f\x79\x51\x8d\xaa\x83\x87\x05\xce\x05\xa0\xc3\xd1\x1c\x8c\xd6\x80\x9f\x5d\xdc\xad\xed\x36\xc0\xb8\xec\xde\x8c\xa6\xdd\xe6\x3d\x81\x09\xe4\xf8\x1d\x0d\x93\xee\xee\x07\x65\x53\xaf\xf0\x08\xf9\x6d\x4d\xee\x48\xa5\xb5\x43\xd1\x1f\xf0\xef\x10\x28\x15\x4b\xfc\xb8\x12\x61\x3b\x95\x0a\x89\xfd\xa0\x19\xca\x50\xf2\x79\x07\x42\xe6\xb6\xe3\x74\xfe\x9e\x96\x3e\x95\x9b\x73\x3d\xba\x2e\x3d\xea\x84\x6c\x43\xd9\xf3\xfd\x44\x43\x49\xd1\x84\xfa\xf5\x62\x9e\xe2\x05\xf6\x67\x39\x86\x29\x3f\x59\x2e\x5b\x27\x33\xb2\xac\xd6\x72\xfd\x5c\x4e\xc6\x2d\xb4\xc4\x49\xe2\xb4\x2d\x99\x84\xd6\xef\xee\x8f\xb6\xbf\x4b\x72\xe8\x45\xb6\x95\x11\x96\xf2\x4b\x06\x96\x24\xec\x5e\x2a\x07\xc9\xfe\x72\xb5\xf8\x80\xfa\x9d\x7e\xed\x34\x2a\xbb\xd7\x10\xaf\x44\x9a\xec\xaf\xa0\x79\x69\x2b\xc9\x1d\x78\x1a\x66\xb5\x8a\x01\xf0\x88\x9a\x84\xce\x50\xc9\x16\xcd\x5e\xe8\x84\xe9\x02\x78\x56\x32\x27\xc0\x94\x7e\x58\xfc\x3a\xc1\xae\x61\x6b\x44\xb5\xc0\x24\x84\x4d\x27\xfb\xa8\x10\x38\x25\x3b\x1c\xfb\x5a\x12\xc3\xa2\xc6\x64\x8d\x4a\x63\x05\xda\xd7\x5c\x8a\xe5\x5a\x46\x4d\x47\xa0\x85\x6e\xc8\xd7\xdd\x67\x0a\xd0\x6c\x02\x5d\x69\x52\xb5\xb6\x8d\x69\x28\xd3\x4a\xdd\x5a\x72\x9b\x78\xe8\xa5\xf6\xee\x84\xe4\xe3\xc6\x64\x80\x01\x91\x2e\x29\xba\x45\xe8\xfb\xf4\xd9\xf3\x67\x6f\xdf\xfd\xf0\xfa\xf5\xd9\xe9\xd9\xdb\x13\x00\xbe\xbf\xd4\x23\xf7\x0e\x75\x9a\x58\x7e\xad\xdd\xde\xab\x67\x62\x21\x5f\x11\xb5\x23\x78\xd7\x37\x1b\x1b\x85\xe5\x4d\x5a\x69\xc2\xee\x98\xd7\xeb\x24\x43\x37\x72\xad\xa1\x25\x47\xd8\xc1\x0d\x47\x85\x1e\x5d\x7f\xd2\x13\xe5\xe0\x83\x89\x35\x58\xbe\x08\xd2\xe9\x8e\xa1\x9b\x4e\x22\xb7\x68\x4d\xc3\xf4\x78\xeb\x04\x6f\x3d\xd9\x34\x1f\xfd\x3a\x79\xc7\xb4\x00\x99\x58\x9d\xa0\x23\x21\x5c\x06\xef\xfa\x56\x62\x4a\x17\x02\x60\xf6\x2e\xc9\x83\xa1\x9e\xbe\x3a\xf9\x8f\x67\xef\x9e\x9c\x9c\x3e\x13\x58\xe9\x7b\x38\x03\x02\x15\x09\xd5\x85\x43\x3b\x52\x08\xf7\x5d\x6d\x24\x6b\xc5\x99\x42\x27\x4e\xdb\x88\xdd\xf2\xef\x8c\xcf\xa8\xc0\x6d\x19\xc8\x44\x8a\x94\x70\x3f\x65\xd0\xa2\x92\x91\x93\xc0\xdf\xee\xd5\x3c\xe7\x8d\x59\xfc\xfb\x7f\x62\xae\x37\x4e\x27\x94\xb7\x35\x3e\x58\xa1\xf8\xe8\x64\x55\xcb\xe0\x93\xdc\x76\xab\xe5\x71\xa7\x40\x9e\xe1\x27\x63\xf2\x12\x5d\xcf\x66\x52\x61\x8d\x24\x1d\x6f\x0a\x1e\xa2\x3c\x20\x25\x6c\x64\x1f\x15\x01\xe8\xff\xe3\x07\xb2\x92\x7d\x11\xc8\x79\xcc\xff\x3f\x90\x93\xbd\x46\x4b\x46\x40\x6b\x0b\x99\x30\x2e\xe0\x36\x67\x08\xa9\x0d\x4e\x56\xa1\xc1\xfe\x93\x15\x49\xe5\xe5\xe2\x87\x6e\x54\xc9\xf3\xce\x1e\x8a\xe6\xc5\xa0\x77\xb4\x9a\xfc\x0d\xe1\xae\x70\x80\x18\xaa\xe2\x83\xd1\xfd\xf1\x96\xb7\x11\x6c\xa8\x7e\x22\xf5\x40\x18\x7d\x9b\x4c\x57\x68\x40\xbc\xe7\x99\x4b\x0d\x5d\x78\x3e\xf7\x9d\x60\x57\x1f\x44\x7d\x9c\xcc\xe7\x8b\xb5\x3c\x39\x69\xf9\xa6\x54\xe4\x38\x33\xb9\x82\xbd\xc6\xfb\x73\x75\x8d\xbc\xbb\x01\xca\x2f\xaa\x90\x1a\x80\x17\x3f\xe4\x48\xd0\x91\x5b\x46\x6f\xb4\xad\x91\x13\x6a\xc8\x5b\xd7\xf0\x9d\x20\xd9\x72\xdd\x20\x2e\x00\xb1\x24\x1f\x69\x8c\xdb\x75\x65\x0a\xf8\x79\x23\x29\x7c\x4e\xc0\xe2\x1a\x46\xfe\x6a\x81\xaf\x3b\x96\x31\xec\x85\xd8\x3f\x31\xde\x93\x84\x9c\x63\xe4\x89\x11\x04\xab\xdd\x8a\xa3\x31\xc9\xc0\xe5\x51\x01\x7d\xce\x1b\x21\xed\x0a\x20\x03\x56\xac\xc5\x19\xb5\x15\xdf\xa9\x6b\x3a\xba\xcc\xde\x1a\xad\x6f\x72\x3c\xaa\xcf\xe7\x67\x0b\xba\x6a\x78\x57\x60\x7c\xe6\x9f\xce\xed\xe3\x0a\xc9\x50\x95\x98\x41\xc9\xed\x2e\x80\x48\x74\x2f\xf5\x42\xd8\x9f\x28\x93\x88\x26\x34\x4a\x23\xa0\x45\x22\x5b\x10\x7c\x3c\xa5\xd7\x3a\xa4\xce\x45\x16\x2b\xa7\xa7\x79\x7a\xb2\x4d\xe3\x5a\xb2\xe0\x0d\x66\x2f\xe8\x28\x6e\x86\x50\xc0\x97\x38\xbc\xb6\x0a\x7a\x59\x2e\x42\x69\x37\x74\x12\xc1\x30\xf9\xe5\x82\x2c\x2f\x8b\x84\xf4\xbd\xa1\xc6\xaa\xbb\x15\x91\xd4\xbc\x5d\xae\x46\xb8\xe6\x9f\xad\x28\xc1\x40\xbc\x3a\xa6\x2e\x5c\x4a\x2e\xc0\x83\x88\x5a\x71\xf7\x24\x2f\x5c\x29\x52\xd4\xe6\xfe\x74\x47\x45\xda\x93\x1f\x91\xcf\x4e\xe4\x58\x27\x39\x6f\xb2\x2c\x4f\xf8\x8e\x48\xf2\x58\xf4\x5a\xd0\x1c\xc3\x5b\x09\x54\x2a\xe4\x6c\xa2\x99\xef\xf9\xd5\xf4\xda\x48\xb3\x3d\xb8\x39\xb5\xb9\xcc\x55\xd0\x6e\xeb\x95\x75\x53\xcf\x9f\xb1\xe3\x6b\x4c\x96\x09\x61\xc3\xd4\x32\x2a\x49\x92\x09\x35\x7a\x4f\x5e\x22\x00\x91\x9e\x97\x23\xdc\x98\x82\x74\xad\x54\xd9\x84\x5d\x8e\xfc\x44\x1f\x0f\x88\x24\x71\x15\xff\x29\x36\x46\x7c\x11\x38\xa8\x78\xc7\x26\xc3\x74\x10\xd7\xec\x78\xdc\x0e\x83\xd3\xad\xb3\xdd\xb2\x77\x55\x1e\xba\x1c\xc3\x68\x51\x01\xc9\x5b\x40\x29\x38\x8b\xbc\xa1\x3c\x9c\xa4\x93\xf8\x61\x34\x9d\x21\xd6\xb3\xd7\xfa\xef\xc5\x5d\x6b\x02\xb8\xea\x04\x56\x6b\x5a\x55\xcb\x09\x3a\x5b\x00\xc4\x75\x22\x8b\x47\x27\x15\xdf\xe8\xe1\x98\x21\x2c\x5a\x2f\x5a\x68\x78\xb8\x35\x5d\x77\x5b\x2f\xae\x5a\xab\xc9\xf5\xb4\x82\x89\xc1\x97\xfd\x91\x16\xe1\x89\x85\x4a\x46\xeb\xd6\x67\xa8\x1f\x3d\x6c\x4d\xaf\x3e\x53\xad\x63\x32\xff\x39\x99\x5f\xa2\x10\xc0\xa8\xa2\xb8\x6a\x02\xeb\x3f\x76\x4c\xff\x2e\xc3\x71\x45\x51\xdd\xcc\x91\xbb\x95\xbb\xf9\x0f\xb3\x85\xf8\x5e\xe1\x5d\x53\xda\xf5\x09\x30\x4e\x4e\x48\xe2\xf9\x2b\xef\xdf\x99\x5a\x0b\x13\xce\xdf\xb9\xfa\x0b\x17\x72\x9b\xac\x70\xdb\x15\xbb\x57\xa8\x29\x15\x5a\xb5\x22\xec\x84\x43\xdc\x13\xcd\x40\x46\x5d\xe8\x18\xee\xcc\x2a\xe9\x68\x1a\x66\xb2\xfa\x30\xbd\x9c\xec\xcc\x29\xe9\x89\x08\x71\xec\xcc\x47\xa9\xa8\x8b\xbc\xa0\xa1\xaf\x77\x66\xd4\x0c\x10\x14\x2a\x21\xc9\xbc\x25\xe5\xa8\x98\x20\xce\x6f\x74\x64\x79\xa2\xbb\x00\x07\xc9\x16\x60\xc3\x12\x1c\xd9\x5c\xc0\xa0\x49\x61\x21\x9f\xd0\x5c\x70\x0c\x4b\x71\x89\x3a\xdf\xb5\x72\x84\xf1\x9b\x42\x2e\x27\x37\x07\x7b\xa6\xe0\x9f\xd0\x4a\xe9\x05\x2e\xb7\xf7\x56\xa2\xeb\xcf\x58\x30\x27\xaa\x77\x56\xc9\xa3\x60\x0b\x4d\x8a\x63\x28\x75\x70\x2c\x3e\xb2\xa4\x79\xe3\x39\xd3\x77\x17\xb3\x69\x75\xf3\x0c\xf0\xc9\xd5\x7c\x34\x3b\x79\xf3\x22\x15\x98\x0a\x34\x0d\x23\x99\xf2\x9d\xdf\x3b\xf2\xb8\x70\xa1\x1c\xc5\x13\x0a\xfc\x23\x28\x69\xc1\x3f\x39\x3f\x91\x16\xfc\xa3\x74\x95\xe0\x64\xca\xe4\x50\x16\x81\xc7\x3c\x43\xe2\x2f\x47\xa9\xcc\x82\x6c\xdf\xe3\xa5\x55\xe0\x1f\x79\x27\x12\x95\x89\x5c\xdf\x82\x0a\x0d\xe4\x2a\xb8\x59\x68\x20\x37\x42\x97\x56\x31\xc3\xf3\xb4\x0b\x17\xca\xf5\x15\xc9\x29\x06\x18\x25\x90\xc2\x07\x73\x15\xda\x70\xd2\x1b\xb9\x8a\x3a\x16\x1a\xc8\x9d\x88\x5b\xe1\x42\xb9\x60\x66\x85\xfc\xe6\xfb\xfb\x7c\xe5\x15\xfc\x93\x8b\x0e\x47\x21\xbf\x39\xcb\x1c\x88\x40\x4a\xee\x9e\x2e\x0a\x17\xca\xef\xd0\x93\x28\xc5\xb9\x90\xb3\x7b\x54\xa1\x3d\xa2\x3b\x94\xfc\x2b\x8e\x88\x3b\xc2\x2b\xc9\x70\xb8\xdc\x79\xc9\xd3\x03\x4f\x90\x17\x09\x75\x32\xa8\x3f\xb1\x0f\x49\xbb\xb2\xa0\x0c\x4d\x57\xa1\x0d\x1c\x83\x99\x44\xef\x73\xba\x1e\x87\x6c\x5b\xab\x00\x8a\xfa\x6a\x86\xf9\xd7\xf2\x1d\x4c\x5b\x8e\xe0\xdf\x7f\x12\x1e\xbe\xac\xeb\xce\x5e\x7a\x3f\x2a\x90\x5c\x38\x99\x5f\xde\x2c\x56\x4f\x35\x1a\xb6\xe1\xf2\x0e\x01\x13\xfc\xf5\x91\x88\x15\x02\xa6\x3f\x8a\xe3\xd1\x85\x13\x6e\xe0\x5b\x1f\x25\xfe\xf1\xf8\xc7\x44\x13\xdf\xaf\xe0\x1f\x13\x8d\xda\x74\x05\xfd\xf5\x91\xac\x9f\x5f\xf0\x8f\x8f\x9e\x5f\xff\x80\x07\x80\x7f\xe2\x68\x7c\xc2\x2a\x7c\x30\x4e\x3e\x13\xdd\xa6\x22\xfc\xb4\xd9\x9e\x20\xeb\xaf\x90\xdf\x5a\x02\xba\x8d\x2c\x4c\xb8\x96\xe1\xf5\x78\x5c\xf8\x60\x90\x5c\x2d\x0b\xfa\x1b\x96\x59\x8c\x7e\x2d\xe4\x37\x48\xf0\xf0\xd7\x7e\xd8\x2c\xcf\x71\xd6\xf9\xc7\x46\xff\x08\x2b\x5c\xf0\x8f\x8d\x7e\x71\x55\xe0\x9f\x20\x8a\xa5\x31\x0b\x17\x0a\x13\xa7\xeb\x82\x7f\x6c\xf4\xab\xc5\x1c\x67\x0e\xb1\x94\x22\xf8\xb2\x99\xde\xcc\xee\x56\x40\x53\xfc\x1d\xb3\xb8\xb0\xcd\xf0\x16\xd0\x8d\x11\x56\xcf\x01\x9b\x74\x7a\xb3\xf8\x58\xf0\x4f\x10\x4d\x3b\x44\x7e\x83\x84\x8f\x53\x38\x82\x85\x06\xea\x49\x7f\xb9\xa1\x25\xf3\x1f\xf5\x2c\x4f\xc5\x3e\x50\xf4\x6d\x33\xbe\x66\x63\x11\x85\x0b\xd9\x44\x72\xbb\xa3\x73\xe9\x3f\x6c\x16\x38\xd7\x13\xdc\x99\xf4\x6b\x13\x5e\xc2\xc5\x59\xf0\x4f\xb0\xfe\xe4\x0b\xa2\xd0\x80\x4f\x52\xc2\x59\xd1\x9e\x71\x38\xaf\x0f\x24\xd2\xc3\x59\x21\xbf\x2e\x61\x6b\x81\xc1\xfc\xfa\x44\x39\xd3\x27\xb3\xe9\xc8\x0f\xb2\x0a\x73\x91\xf7\x54\x93\x98\xd7\xc1\xcf\x3d\x20\x24\x08\x52\x4e\x49\x59\xb7\xd8\x3f\x31\x5f\x0a\x87\x72\x45\x5a\x30\x39\xc0\x5e\xf2\xfd\x0b\x40\x9f\x01\xa1\x2a\xf6\x7f\xe0\x80\x4f\x21\x4f\x96\xcf\x05\x6b\xdb\x7f\x62\xbe\x4c\x1e\x7f\x7c\xf6\x9f\xd4\x30\x96\x7c\x5f\x39\xcb\xc5\xbe\xda\x20\xf2\x69\x13\xf5\x4e\xf2\x23\x79\xec\x80\x1a\x9e\x45\x31\x3e\xaf\xe0\x55\xfb\xcf\x03\x54\x2a\xdf\x27\x0b\x57\xcb\x05\xc1\x9a\xfd\x17\xfe\x23\xca\x81\x2a\xf5\xfb\x2a\x8e\xee\xd3\x48\xb7\x71\x1f\x1d\x03\x85\x71\x3f\xc0\x3d\x86\x78\x05\x25\xc9\x87\xcf\xa1\x22\xef\xc5\xbe\x3a\x89\xb7\x69\xd7\x18\x7d\xed\x63\xe8\x19\xbe\xd8\x7f\x83\x3f\x3e\xd6\xf1\x8f\x8b\xfd\xb7\x1a\xf4\xa9\x7f\x2b\xf6\xff\xd3\x7f\x55\x80\x0a\xef\x9f\x5e\x4e\x82\x98\xa7\x00\xd9\xaf\x69\xd4\xa7\xfe\xc3\xe4\x98\x4f\xaf\xae\x70\xc2\x4e\x39\xe0\x53\x54\xed\x94\xd6\xb3\xd8\x3f\xb3\x9f\x26\x17\xbb\x73\x81\x74\x0e\xf8\x14\xbe\xaf\x8b\x7d\x96\xfc\xd1\xf8\x2d\x32\x8f\x3d\x86\xf7\xfe\x6f\xaf\xe0\x0a\x7b\x31\x76\x24\x60\xa7\xf3\xfe\x6f\x2f\xc6\x46\x20\x0d\xa8\x87\xd9\x13\x65\x95\x7a\x51\xfc\x80\x0f\xfa\xe6\xd9\x93\x17\x27\x2f\xdf\x3d\xf9\xf1\xe4\xed\x69\x8d\x15\xfa\xce\xf3\x57\x73\x61\x8b\xb2\x0e\xba\xe7\x06\xd0\x67\xdf\xb1\x43\x8d\xff\xf3\x82\x23\xb7\xfe\x19\xe9\xa7\xd7\xff\xcf\xbb\x1f\x4f\x9e\xfc\x87\xb6\x03\x84\xe3\xdf\xf7\x8f\xed\x0b\xea\x2e\x2e\x20\x11\x42\xe3\x69\xb5\xc4\x94\x33\x52\xfb\xa7\x4d\x8a\xe8\x57\x95\x33\x47\xe7\xc5\xd5\xab\xc5\x89\x93\xda\xba\x37\x6a\x6d\x58\x05\xd5\x99\x92\xd7\x79\x31\x21\x3c\x59\xe7\xa4\x3f\x81\xb2\xea\x10\x78\xc9\x8c\x1a\xf7\x52\x80\xae\x3c\x39\x91\x82\x92\x4c\xe1\xd5\x64\x4e\x7a\x01\xa5\xe9\x04\x79\xe3\x1f\xdd\xf6\x07\xd6\xda\x35\x37\x37\x2c\x28\x72\x48\x7e\x83\x08\xda\x94\x76\x28\x28\xb6\xb8\xd7\x30\x04\xb5\x71\xcb\x95\x20\x0b\xaa\x87\x8d\xaa\xd4\x37\xc5\xa0\x55\x1a\x8a\xe4\x67\x97\xcc\x8d\xa8\x3c\xf2\x63\x2a\xd9\x7c\x0b\x95\x72\x91\x8f\x35\x67\x4f\x03\x9d\x0e\x55\x19\x3d\x79\x41\xea\x40\x73\x0c\x33\x3f\x84\xbe\x3e\x8c\x88\xbf\x69\x81\x26\x80\xa9\xa1\xfc\xf2\x6a\x01\x34\x66\xe1\xc7\xbb\xe7\x82\x66\x62\xa1\x8f\x66\x6a\xcb\x54\xe7\xb6\x74\xcf\x9c\x12\x91\x66\x6a\x42\xae\x67\x0a\x3c\xf6\x35\xf5\x7c\x10\x46\x41\x13\x42\x14\x95\x58\xf8\xd0\x7a\x06\x3e\x9b\x51\x5b\x56\x77\x88\xff\xfe\xb7\x5d\xa6\xd3\x89\xa5\x65\xb3\xd5\x6c\xba\x07\x89\x41\x85\xfb\xfa\xb5\xd9\xd8\xf8\xdc\xec\x4a\x9f\x27\x28\x9a\xc7\xc6\x53\x4d\x91\xf8\xc4\x04\x8f\xc8\x12\xb6\xf2\x98\x9c\x2b\x0b\x85\xbb\x68\xe3\xd1\x58\x1b\xb2\x8a\xac\xb4\xc8\xde\x69\xf5\xed\x76\xf2\x7d\xb2\x67\x16\x48\x64\x16\x85\x43\xc4\x7b\xc6\xb3\x87\x00\x35\x06\xda\xf8\xe5\x62\x41\x6a\x19\x77\x4b\x6d\xb9\x6a\x7d\x98\x8e\x5a\x95\xbc\x9b\xd7\x95\x31\x5a\x17\x9f\xa5\xae\xbd\xd6\xe9\x64\x52\xb4\x44\x41\x1e\x6e\xba\x58\x3d\x7e\xb4\x9c\x1e\x46\x6f\x00\x49\xa0\x91\x1e\x4d\xd0\xb6\x69\x58\xf2\x0c\x3c\xfd\xb0\xd3\xe9\x32\xa4\x41\xb5\xf0\xb7\x4b\x76\x58\x7e\x3c\xfb\xe9\x65\xc9\xe2\x74\xed\x7f\x39\xfe\xc3\x51\xef\xfb\x43\x12\xad\xd3\x17\xfa\x1c\x73\x5a\x9f\xc3\xf8\x6d\x5c\x04\xcb\x73\xc7\xc9\x78\x8c\x8e\x08\xc4\x92\x09\xe6\xa1\x1d\x4a\x71\xe2\xed\x6a\x35\xba\xb6\xc7\x31\xea\x9f\x5e\xf8\xcf\x25\x5b\x8a\xa6\x8a\x25\xac\x52\x7e\xec\x2d\x85\x64\x12\x9b\x5a\x75\x13\x13\x6d\xa9\x27\x24\x73\xe8\xf6\x55\xb8\x75\x58\x22\x11\x6b\xa1\x37\xcd\xa8\xe8\xd3\xc9\x08\xee\x6f\x5f\x96\x63\x19\x94\xe3\x1b\x92\x7f\xf5\x36\x6a\x2d\x79\xf3\xf1\xa7\x9e\xe2\xe3\x75\x6f\xfa\xbd\xe6\xb0\x8a\x2f\x41\x8b\xee\xa0\x4f\x87\xb5\x4e\xbd\xbe\x72\x27\x83\x4d\x9a\x5c\xcd\xf3\xbb\xb9\xdb\x72\xaa\x1b\x20\x42\x6f\x36\xa5\x71\x73\xc3\xc5\x87\xf2\xd6\xb0\xbd\x39\xfe\x5f\x20\x22\xcd\x5a\xe3\xc5\x24\xd8\xcb\xc4\x7c\xfc\x5f\xba\xd3\xff\x97\x63\x40\xca\xeb\xf8\xe4\x03\x49\xf0\x70\x0f\x9f\xb1\xa3\xe5\xd3\x35\xfa\x96\xd6\xae\x26\x9c\x25\xc9\xc4\xa5\xe0\x83\x79\x39\x0b\xd4\xed\xdc\x0f\x5a\xc5\x54\xb2\x2e\xe4\x64\xdc\xb8\x5e\x23\x86\x82\xdf\x02\xc5\x73\x35\x57\x8e\x2b\x46\x60\x1b\x51\x7a\xf4\xcd\xf4\x7c\x1e\x4e\xa2\x2d\x85\xa6\xd4\x49\x35\x88\xab\x26\x73\xe3\xc3\xad\x17\x18\xe1\x87\x4b\xe7\x60\xca\x37\xcd\xad\xd9\xae\x5e\xcd\x33\x74\xc6\xf2\x75\xcd\x4b\x4b\x8d\xcd\x67\x85\x55\x71\xb3\x29\x24\x10\x01\xed\x6c\xb3\xda\x56\xa9\xef\x54\xe3\xd4\x43\xdc\x61\xbf\x18\x3b\x41\x0f\x40\xc4\xd8\x78\x8f\x59\x13\x38\xaf\x84\xed\x0d\x5c\xf6\x21\x82\x5b\x9b\x85\x4d\x60\x60\xbd\x4e\xf4\x94\xba\x5f\x2f\x49\xc2\x07\x7c\x09\xc8\x8b\x78\xcf\x56\xd4\x75\xab\x1d\xc4\xf2\x60\xbb\x7a\x1f\xb7\xdb\x0d\x65\xd0\xc2\xb5\xb9\xb1\xf3\xda\x51\xc9\xdc\xac\xd6\xbb\x95\x47\x13\x60\x8c\xfa\xc6\x33\xda\xb8\x5d\x23\x81\xf9\x7f\x74\x62\x37\x9b\x83\x63\x27\xf0\x19\x2b\x53\xf7\x83\x39\xd9\x6c\xd2\xb8\xcf\xbe\x55\x8f\x4f\x7f\x69\x25\xcb\xfb\x6d\x98\xc7\xe8\xc7\xab\xc8\x42\x61\xd3\xc3\xb9\x67\x75\xb9\x18\x66\xda\xdd\x16\xcd\x0c\xd9\xfc\x7f\xe8\xdc\xb3\x37\x71\x64\x72\x12\x1a\x59\xc6\xb3\x80\x15\x4a\x4c\xb9\xa7\xb9\xf0\x05\xd1\x0a\xff\x62\xf1\x29\x9a\xd9\xfe\x33\x57\xe2\xcb\x90\x66\x8e\x28\xb3\x91\x75\x0b\x54\x94\x24\xf7\xab\x61\x91\xcd\xe6\x8b\x9d\x64\xff\x05\x38\x7f\xda\x0d\xf6\x26\xe1\x27\x90\xfd\x96\x8a\x52\x91\x74\xc0\x9d\x10\xc8\x2a\x5a\x54\xa6\x59\x9b\x2a\xfe\x28\xb0\xba\x9e\x9a\x4a\x80\x08\x92\xf6\xad\xed\xca\x1f\x47\x15\x31\xb7\x5c\x1f\x15\x4e\xd7\xae\x3b\xf4\x89\xaa\xcc\x84\x3e\x42\xaf\x4e\xda\x94\x94\x26\x24\x6c\x87\x96\x23\x51\x48\x04\xa1\x9c\x97\xa1\x1b\x9c\xcf\xcf\xd7\x43\x14\xa3\x6b\x59\x27\xf2\x50\x97\x36\x4b\x05\x1e\x1f\x1c\x17\x56\xad\xc8\x82\xa4\xb0\xb7\x97\x15\x77\x1f\x85\x6e\x7c\xd8\x49\x91\xa2\x76\xb8\xeb\x99\x37\xf3\xe0\x73\x36\xc2\x62\x4d\xf6\x92\x4f\x55\xc3\x00\x73\xf8\xbe\x4d\xff\x29\x13\xe1\x84\x3d\xa1\x2e\xaa\xd5\xf5\x80\x84\x1b\x31\x4f\x16\x88\xd6\x38\x24\x66\xf7\x6c\xa0\x4f\xde\x2f\x4c\x88\xc2\x9c\x29\x2a\x5b\x5e\x4b\xd6\xf2\x9f\x31\xa2\xde\x6f\x9e\x69\x0d\x95\xe1\xf8\xf3\x03\x54\xab\x8c\xba\x18\x6c\x1c\xcd\x4a\xdd\x21\xf8\x1f\xe4\xed\x94\x41\x86\xad\x97\x5b\xdc\xbd\xa6\x51\x15\x59\xfd\xd4\x38\xf4\x11\xf9\x2b\x4e\x0a\xd9\x52\x1e\x7e\xf3\x54\xa5\x13\x69\xf6\x5a\xff\x7b\x1e\xee\xb8\x54\x51\xc9\x31\x8a\x8f\xae\xae\xbe\xca\x37\x57\x6c\xfa\x11\x43\x91\x3e\x74\x54\x0d\x21\x86\xd8\xbb\x40\x8e\x9a\x7d\x4b\xd7\xd0\x5c\xc7\x50\x8b\xee\x7a\x81\x00\x9c\xeb\x85\x15\x81\xf2\x20\x6d\x1f\x36\x8b\x08\xc3\x58\xc6\x36\xee\x0f\xfb\x15\xb7\xd9\x5c\x1b\xf1\x38\x04\xea\x37\x10\xe2\x4e\xf6\xf4\x4f\x65\xd9\xa0\x9d\x23\x56\xab\x8d\x2c\x2d\xd9\x53\x10\xf9\xa9\x0c\xad\x29\x71\xac\xb2\x0f\x70\xbd\x44\x9d\x20\x90\x71\x64\x69\x6c\xab\x98\x99\x59\x0d\xa6\x5e\xdc\xca\x52\x2c\xed\x6e\x03\x09\x1b\xd1\xc2\xc2\x4d\x46\x16\x58\xeb\x78\xd4\x85\xcf\x51\xfe\xf0\xfa\xf5\xcb\x67\x27\xaf\xde\x9d\x9c\x9d\xbd\x25\x3c\x27\x14\xac\x73\xb7\xbb\x29\xd3\x6e\x6b\xa1\x67\x2f\x9f\xa1\x8e\xdb\xa9\x6e\x13\xb7\xd3\x86\x24\x2a\xac\x05\x0c\x5f\x8c\x49\x3b\x83\xb9\xba\xee\x31\xce\x24\xb0\xc1\x64\x28\x43\x34\x59\xf0\x63\xdc\xf1\x1f\x68\x0e\x56\xf4\x2b\xac\x76\xc4\x32\x1a\xa2\xad\x68\x31\xa7\xf3\xa8\xfe\x4b\x14\xc5\xf0\x78\x52\x34\xfa\x0e\x40\xe9\x92\xd1\x35\x31\x3e\x5d\x75\x51\x7c\xbd\xbe\x4b\x24\xe1\x67\x3f\xdc\x5d\xa0\xb1\xce\xbd\x23\x57\xa1\x7a\x00\x4e\xed\x67\x29\x95\xae\x2e\x85\xc8\xf5\x46\x6a\x33\xfb\xb4\x2a\x85\xbc\x7b\x63\xfc\x44\xaa\x88\x15\x6e\xf8\xbb\x6c\x1a\x70\xef\xeb\x66\x21\xae\x19\x35\x06\xa4\x8c\x18\xb2\xfe\x40\xf4\x69\xbe\x2b\xfb\xf1\x96\x53\x08\xac\x84\x69\x75\x31\x99\xe6\x4a\xc8\x26\x5c\x6d\x51\xb6\x79\x48\x2c\x0d\xd8\x4f\x93\xcc\x22\x12\x11\xa1\xfb\x03\x55\x14\x0d\x76\x13\x82\xdd\x3f\x3e\x2e\x6f\xab\x29\x60\x0e\x8d\x33\x82\x6c\xbe\xe6\xa5\xb7\x49\x0d\xa3\x23\x19\xa3\x22\xb5\x64\x4f\x54\x77\x40\x12\xc5\xb5\x87\x89\xf5\xfa\xbd\xb3\x38\x7b\x14\xbc\x7b\x5a\x37\x44\x49\xf1\x27\xec\xc6\xdb\xb8\x72\xba\x70\x28\x8d\x76\xa6\xce\xae\x58\x71\x54\x35\x63\x9d\xcc\xa1\x64\x51\x5f\x9d\xf8\xdd\xaf\x9b\xe4\x21\x0d\xee\xc0\x3c\x56\xd6\xaf\x45\xa5\x59\x81\x71\x81\x7e\x79\xbd\x60\xe9\xcc\x93\x71\x76\x94\x7a\x93\x4e\xa0\x79\xce\xce\xaf\xd6\xf4\x16\x9a\xa8\xf8\x09\xb5\x0e\x90\xa2\xf4\x72\xce\x6c\x18\x85\x9d\xc8\xde\xb1\x53\xd6\x40\x27\x88\xc4\x37\x27\xa9\xaa\x5d\x8b\xf4\x10\x90\x9e\xe4\x0d\x1e\xce\xe2\xd3\xc9\xe5\x8c\x24\x08\x7b\xff\x63\xa7\x15\xa2\xab\x79\x3f\x95\x92\xe8\x57\x43\x82\x68\xa1\x5c\x63\x89\x9a\xf5\x86\x79\xb9\x7a\xcc\xdb\x60\xd9\xf2\xf4\xec\xed\x8b\x37\xef\x9e\xbc\xfe\x89\x00\x27\x0b\x79\x73\x3f\x4a\x2e\x27\x2a\x9c\xcf\x01\x24\xbf\xfd\xf3\xa9\xd7\x5e\x91\x5c\xa8\xcd\xc9\x28\x0d\xe7\x78\x77\xfa\xe6\xe5\x8b\xb3\xcc\x6a\xcc\x5d\x67\xf7\x28\x36\xaa\x2d\x72\x3e\x93\x01\xf6\x35\xae\xcc\xaa\x42\x0f\x16\x72\x2d\xc8\x50\xf8\xd6\x9e\x8b\x9a\x27\x50\xa0\x7e\xc0\xa5\x0e\x5c\x05\x2d\x98\x53\x80\x93\x57\xba\xd1\x1f\x1c\xe7\x56\xe4\xf6\x6a\x3e\xc0\xf4\x61\x9e\x5c\xcd\x61\xa0\x66\x16\xd9\xa2\xe8\x11\xcd\x3d\x6c\x80\xb0\x10\x65\x27\xbb\xe8\x52\x22\xbe\x40\x62\x55\x86\xb3\x05\x4a\x5b\x98\x87\x07\x61\x01\x09\xe1\x34\x96\xf7\xa3\xba\x08\x9f\xa1\xf6\x1c\x15\x6b\xa8\xad\xbe\xd3\x70\xc2\x8c\xa1\x91\x3e\x57\x69\xe6\xe8\x4e\x8d\x6a\x36\xc7\xe7\x9e\x34\x03\xd1\xb2\x77\x6c\x80\xe8\x41\x81\x48\x2f\x10\x96\x86\xfe\xc5\xa5\x06\x6f\xa5\xc2\xc7\x65\xec\xdb\x80\xbf\x4a\x0d\xbd\xf0\xaa\x1e\x24\x11\x36\x15\x2f\x5f\x5a\x28\xdf\x73\x1f\xdd\x7d\xf4\xcf\xbe\x43\xfc\x70\x79\x8d\x56\xd1\x13\x7d\x1e\x13\xf9\xc3\xdb\xbb\x6a\x2d\xd6\x84\x5b\x58\xba\x25\x32\x6f\x2d\x16\x7a\x74\x92\x7f\x6a\x9b\x4e\x4a\x33\x8d\x4f\xaa\x5b\x1a\x75\x7a\x77\x75\x35\xfd\x34\x74\xfd\x7e\x67\x54\xd5\x45\xdc\xd5\x4a\xc8\x1a\xd1\xd2\x70\x96\xef\xf7\xc9\xa1\xb1\xe6\xda\x06\xf6\x00\x69\x52\x9d\x40\xb4\x8a\x58\xfb\x4d\x62\x1b\x1a\x58\xd1\x43\x2f\x37\xe3\x34\x51\x9c\x41\xbb\x69\xd3\x0c\xdb\xea\x83\x87\x44\xd6\xee\xb1\xe8\x67\x53\xe3\xa1\x69\xbc\x2c\xb0\x7b\x41\x8d\x04\x35\x3c\xbc\x97\x9c\x2c\x60\x96\xd7\xe7\x5f\x19\x29\xfa\x26\x51\x4b\xf1\x4d\xa3\xb2\x36\xbd\x4e\xa6\x32\x91\xaf\xe8\x9d\x10\x63\x9f\xab\xcb\xb7\xd5\xd4\x3d\x14\xd7\x77\x20\x7a\x13\x37\x45\xa3\xa5\xcf\x72\x2c\x8d\xab\x57\xda\x6a\x68\x53\xf6\x6a\x31\xb1\x4f\x2e\xcc\xe0\xe4\x41\x75\x30\x2f\x22\x35\x27\x6d\x20\xb7\xd5\x19\xa5\xa8\xe6\x52\x32\x42\x32\xb8\x0d\xdb\x4b\x8f\x7c\x61\x9b\xdc\xda\x63\x8f\xf2\xb4\xa2\x8a\x13\x83\x2b\x32\xa6\x66\x44\x5e\x63\xd5\xe9\x20\x7b\x5d\x8f\x07\x19\x35\x58\xfb\x44\xeb\xa7\x39\x95\xd4\xec\x3e\x4c\x82\xfb\x51\x92\x10\xb0\x92\x38\x9a\x7d\x8e\x71\xa5\xbc\x60\xf7\xf3\x79\x19\x0a\x93\x49\x9e\x9e\xef\xb2\x97\xa2\x14\xfb\x20\xf5\xc1\x3e\x9f\x77\x9d\x2c\x73\xa6\xd9\x5c\x92\x97\xc8\x35\x0f\x12\x46\x26\xd8\x67\xb4\x72\xbc\xf8\x64\x31\x9d\x96\x26\xca\x51\x9f\x53\x11\xf3\xbf\xf7\x35\x91\x9d\x95\x40\x44\x7b\xe8\xb6\x7e\xf3\xb6\xf4\xe5\x50\x51\xbf\xa7\x79\x06\x26\xfe\x78\x38\x94\x07\x4b\x27\xd9\x6d\x52\x1f\x0d\xd5\x7c\x96\x81\xd7\x32\x7d\xfd\x48\xf4\xb4\x01\x34\xd3\x46\xd3\x15\x71\xd7\xed\x3f\x5a\xde\xde\xad\xb2\x03\x9c\x88\xbd\x17\x3c\x64\x40\x1f\xb5\xe5\x44\xeb\x65\x21\x44\xc5\xd3\x6a\xbc\x4c\xba\x62\x71\xba\xdd\x9e\xa0\x63\xb7\xcb\x5f\xdb\x6d\x64\x9e\xc8\x87\xe3\x99\xb8\x7c\xc4\x2c\x29\xdd\x27\xda\xe0\x4e\x3a\x92\xdb\x5d\xf8\xfe\xa6\xe1\x16\x57\x28\xe8\xfb\x7c\x34\x25\xe9\xf5\x45\xcb\xc0\x56\x95\x4a\x47\xb5\x93\xf1\xdd\x04\x52\x8b\xf3\x39\x69\x0b\xa9\xae\x36\x57\x0d\xb4\x83\x36\x09\x41\xf4\x96\x92\x79\x71\xf3\x3a\x7e\xc1\xa2\xbb\x0e\xcf\x20\x59\x21\xaf\x8f\x61\xc9\xed\x53\xb9\x47\x0c\x2c\x63\x2e\x18\x16\x89\x15\x16\x6c\x26\x9f\x6b\x60\xa2\x51\x73\xf5\xc5\xab\xd3\xb3\x93\x57\x67\x2f\x4e\xce\x5e\xbc\xfa\xf3\xae\x2b\xf8\x72\x3c\x41\x95\xf2\x27\xd3\x15\x3a\x80\x5a\x79\xd1\xfb\xcf\x00\x3f\x00\xbf\x2b\x70\x4a\x12\xd2\xb3\x11\xc3\x03\xad\xef\x0f\x90\xa3\xe7\xc4\x9b\x6b\x4d\x6f\x8d\x31\x0e\x2a\xa7\x3a\xa5\xb6\xdb\x79\x43\x97\x83\x0e\x37\x65\xd0\xeb\xcc\x56\xb4\xbd\xc2\xe7\xf6\xd9\xe7\x7b\xd6\xe8\x61\x29\x0a\x6b\xd8\x95\xf7\x31\xa0\x83\x64\x15\x88\x24\x5e\x85\x3f\xa0\x9a\x3d\xc4\xad\xc7\xb7\x40\x44\xc1\x15\xbb\xb4\x68\x3f\x3b\x8f\x2d\x8f\xd4\xaf\xae\xa2\xb9\x02\x2e\xd4\x9d\x2c\x83\x0c\x36\xf2\xa1\x79\x10\x4c\xd4\x9c\x56\x23\x52\xb8\x63\x3d\xa6\xeb\x5f\x01\x69\x4d\x5e\xcc\xe1\x7e\x40\x81\x36\x51\xab\x65\x73\x19\xbf\x4e\xe6\x7b\xad\x67\x9f\x96\x10\x01\x3b\x58\x66\x81\x95\x2f\x46\x55\x8b\x5b\x61\x6d\x30\x5a\x34\xe2\xf4\xe3\xb8\xf8\x88\xf3\xd0\xdb\x6d\xfe\x6d\x32\xf5\xd5\xe7\x24\x62\xbe\x17\x66\x53\x92\x65\x91\x6d\x45\xd2\x88\xa9\xc7\xe5\x81\x76\xb9\x9a\x97\x88\x9d\xd3\x04\x0c\x33\x9a\xe2\xfe\xc1\x71\x31\xf2\xe6\xa2\xb2\x7b\x14\x7c\x6e\x1d\x15\x7a\x2b\xc1\xed\xda\xa3\xa8\x63\x13\x35\x52\x28\x49\x29\x8f\xea\x29\xb4\x3e\x68\x9b\x84\x73\x7c\xbb\x3b\x07\xff\x3e\xd2\x9c\xdf\x7d\x39\x27\xff\x7e\xab\x25\x7e\xff\xf5\x25\xf8\xf7\x3b\x2d\xf9\x87\xdf\x5e\x92\x7f\x7f\xaf\x35\xfc\xdb\x3f\x5e\x03\xff\xfe\x41\x6b\xfa\xe3\xff\x7e\x4d\xfc\xfb\x6f\x5a\xe3\x9f\xfe\x79\x35\xf2\xef\x1f\xb5\xe6\xe3\xa3\x7f\x7e\xd5\xfc\xfb\xa7\x21\xba\x65\x66\x79\x5a\xd7\x44\x64\x25\xac\x0a\xc1\x85\xc7\xb7\x91\x89\x10\x00\x0c\xc5\xe6\x72\xae\x6a\x32\x66\xeb\x2f\x4f\x3c\x5e\x6e\x51\x48\xc7\x7c\x31\xe9\xde\x29\x6c\xe9\xec\x5e\x9e\xd1\xab\x3e\xfe\x1d\xe0\x1f\x7f\x2d\x16\x94\xe2\x8b\x38\x84\x9a\x6c\x56\x9a\x5a\xc3\xfe\x08\x96\xc2\xdd\x77\x5d\x96\x71\x38\xfd\x88\x34\x28\x03\xf8\x80\xfd\x2c\xb4\x98\x58\xc9\xbb\xe7\x2a\x45\x89\x26\x37\x93\x54\x98\x30\x8a\xf1\x19\xb0\x91\x2b\x08\x2d\x34\x90\x03\xcc\x29\x22\x4b\x23\x4d\x54\x5c\x93\x82\x5e\x8c\xd9\x6f\x36\x8d\x57\x23\x33\x1d\xb6\x24\xf0\x15\xdc\x27\xe8\xf5\x30\xac\xa2\x4c\xbc\xfa\x0f\x5e\x1e\x08\xfd\x03\x6c\x97\xa6\x59\x78\x46\x21\x99\x53\x3a\x0d\x87\xe2\xde\x69\x77\x85\xdc\x02\x8d\xf6\x2a\x5d\x61\xba\x62\x01\x4e\x8f\x2b\x4c\x96\x58\x55\xde\x0a\x13\xe5\x4d\xd8\x69\x6c\x85\xa9\x1a\x9d\xe5\x8e\xba\x2a\x5c\x68\xeb\x67\x41\xd1\x92\x32\x9c\xfb\xfd\x9a\xc1\x87\x08\x8b\x09\xb2\x5b\x65\xc8\x1d\x57\xda\xdd\x7c\x89\x78\xd7\xcf\xf3\x5f\xe7\x8b\x8f\x7e\xa5\x77\x62\x16\xf8\xb4\x6c\x29\x47\x5c\xb9\x98\x8e\x2a\x83\x1c\x5f\xee\x73\x90\xdd\xd8\x96\xe0\x59\xf6\x6f\x1a\x0f\x63\xf4\x26\x7b\xbc\x1d\xbf\x44\xf0\x2d\x2d\xa1\xe9\x96\x00\x5d\xd7\x46\xd4\xda\x6e\x42\x2f\x64\x55\xef\x6a\x08\xbd\x9b\xa3\x62\x96\x9d\x46\xcd\xe3\xc1\x5c\xa3\x00\xbc\x90\xbd\xe8\x45\x94\x13\x00\x9d\x78\xc6\x86\x2f\xca\xbd\xa3\x1e\x31\x48\xc7\xd3\x0a\x23\x4e\x6c\x1e\x0b\xf4\x9a\x0b\x1f\x6f\x99\xbd\x4a\x54\xf6\x20\x11\xf1\x68\xb4\xf7\xa1\x62\xe2\x6a\x21\x44\xcc\x85\x78\xde\x88\xe8\x13\xbb\x8c\x46\x2a\x3c\xc4\xa0\x9f\xa3\x70\x1e\x0f\x2b\x65\x43\xe1\xde\xca\x38\x31\xda\x6b\x33\x0d\x99\x1a\x8c\xf8\x70\x89\xcd\x26\x19\xa1\xd5\xd3\xd0\x26\x9d\x7d\x9c\x42\xd6\xad\xd4\xee\x24\xf0\xd4\xc8\xb8\xe1\x0f\xd1\x5c\xc8\xcc\x4e\x66\xb7\x08\x02\x01\x81\xd4\xd1\x20\xf8\xba\x49\x51\xdc\xab\xba\xe9\x43\x13\xb7\xe5\x2e\x1b\x4b\x98\x23\x43\x6f\x67\xea\x78\x17\xf6\xf8\xe2\xbf\xa6\x93\x8f\x68\x17\x04\x0b\x46\x53\xd0\x50\x4f\xf5\xc3\x67\x7a\xca\xe3\xaa\x76\xd4\x45\xce\x6b\x81\x7a\xc0\x4c\xed\xb6\x4c\xbf\xf7\xcd\x77\x94\x1f\x65\x45\x63\xec\x56\xbc\xb5\xb1\x19\x4b\xcd\xa2\x11\xce\xae\x42\xc3\xfe\x80\x66\xdc\x9a\x8a\x3d\xfb\xb4\xfe\xee\x13\xcf\xd9\xd6\x82\x1e\x5b\x81\x33\x90\x9f\x72\xff\xc8\xdf\x05\x85\xb6\x46\x68\x47\x14\x39\x6a\xfe\xc4\x50\x39\xc1\x69\x06\x98\x1d\xa6\x0e\xec\x5f\xdf\xad\xf1\x50\xa1\x4b\x3f\xd4\x6f\x41\x8b\x2d\x78\x1c\x91\xe6\x71\x98\x05\x71\x75\x9a\x0c\xb0\x1e\x67\x9e\x5e\x41\x03\xca\xb5\xba\x9e\xa0\x52\xe0\xc1\x41\x8e\x0e\x8e\x76\xa4\xb2\x20\x79\x43\xa2\xaa\x16\xba\x07\x62\xec\xd3\x43\xf9\x96\xe8\x59\x23\x35\xd4\x3b\x8e\xbd\x4b\x3e\xac\x52\xba\x44\x43\x5b\x97\x6f\xf8\x91\x5d\x55\x45\x50\x3c\x5d\xf4\x1e\xb2\xfb\x3d\x3f\x4f\x37\x93\xcb\x5f\x53\xff\xb8\xb2\x84\x52\xcf\xe7\x46\x4c\x91\x23\xb2\x7b\xfe\x4d\xc9\xc8\x1d\xee\x22\xae\xaa\xf4\xb5\xa6\x54\x55\xee\xdc\xb3\x6f\x03\x17\x3c\xd3\xd5\xe4\xe7\xd5\x8c\xf5\x91\xa0\x39\xb8\xaa\x55\xcf\x84\xdf\xdc\xf0\x49\x40\xd6\x18\xf2\xed\x95\x88\xed\x75\xef\x56\x33\xf4\x43\x9f\x86\x89\x26\xcd\xbd\x93\xdc\x69\xdd\x2a\xb8\x68\x46\x30\x93\xa8\xec\x5e\x43\xa9\xaf\x81\xc4\x11\x59\x3d\x61\x76\x45\x2e\x50\xf3\xd5\xe8\xa3\x8a\xf9\xba\xe3\x8d\x18\xae\x6e\x68\xb5\x2b\xe3\x40\x1c\x14\x42\xdc\x40\xe3\xe5\xd3\x4c\xb8\xa6\xf8\x98\x9c\xfc\xd3\x47\xc9\x36\x2e\x47\xba\x1e\x76\xc1\xd3\xc9\x15\x3a\xac\x27\x7b\x0b\xd4\xeb\x69\xf5\xd3\xe2\xf2\xd7\x52\x7c\x75\xed\xd8\x74\x40\xfb\x3e\xb0\x93\x90\xe9\x48\x75\xed\xef\xef\x3e\x26\xe5\xee\xa4\x5c\x0a\x4f\xe7\x97\xf5\x44\x6e\xdf\x9c\xf5\x1d\x5d\xec\x74\xb6\x5c\x0f\xe0\x9a\xd3\xab\xcf\xa8\x5b\xf7\x6a\x51\xaf\xae\xf2\x55\xa9\x32\xee\x6f\xdb\xac\x0f\x1c\xcd\xbe\xd6\x08\x70\xf4\xc1\x83\x47\xf6\xfd\xb5\xf5\x6d\x8f\x9d\x9d\xbb\x33\x90\x4b\x37\xdc\xb4\x8e\xc6\xe3\x37\x14\x55\x06\x28\x80\x7b\x6b\xf2\x4f\xfd\xa6\x16\xd8\xe7\xf6\xd8\x1e\x1f\x1d\xd9\x13\xab\x6d\x70\x5f\xa0\xb6\xfc\x6a\xce\x1d\x89\xce\x86\x87\xba\xab\xc9\x55\x7e\x01\x37\xa1\x5a\x7c\x73\x37\x0c\x8b\xa9\x60\x52\x92\xe5\xf1\x41\xec\xe9\xd1\xf0\x7d\x87\x8f\x5c\x9e\x15\x89\x5d\xa2\x4d\xc0\x75\x1b\x9d\x04\x3c\xaa\x3b\x0e\x49\x96\x43\x35\x5c\x3c\xec\x70\x89\xf1\x3c\x35\x7a\xe9\x44\x39\xb0\x7d\x85\xf0\x7a\xb8\xfa\xd2\xa1\xbe\x7c\xeb\xbb\xe7\x29\xf1\x80\x08\xa2\x24\x09\x35\x59\xa4\x9a\x05\xa7\xae\x9e\x6e\xe7\x88\x4c\xeb\xa5\x09\x4e\x5d\xb2\x2b\x1e\xcd\x5e\xa7\x76\xd6\xcc\xf4\xf4\xdd\xec\xeb\x33\x2c\xf5\x20\x58\x13\x1a\x2f\xed\xfc\xad\x57\x9a\xd0\xca\x36\x9b\x20\xaf\xb5\x89\xf9\x6f\x28\xed\xf6\xbb\x44\xb6\x5f\x1d\xde\x21\x39\xe4\x62\x51\xf1\x16\x41\x04\x2d\xe6\x62\xee\x20\x6f\xd3\x61\x92\x5e\x04\x65\xf1\x99\x3b\x9a\xf1\x76\x5b\xe4\xa2\x44\xb7\x1d\xaa\x4d\x13\xb8\x92\xd0\xa7\x33\x62\x80\x16\xc0\x67\x66\xbd\xe0\xf2\xbf\xa4\xc8\x7e\x43\x79\x9f\x1a\xd7\x50\x84\x27\x29\x8d\xea\x8f\x46\x7a\x64\x62\xdc\x8c\x44\xe7\xd6\x29\xf3\x0b\xdc\xc1\xf5\xfd\x11\x17\x24\x7a\x46\xa2\x45\xda\xb5\xf8\xba\x45\xf1\xab\x1f\x2e\xd1\x2f\xa8\x33\x53\xf5\xcf\x8b\xf3\xc3\xf3\xc3\xc1\x2f\xe7\x87\xc3\x6f\xd8\x87\x65\x92\xf8\x93\xfa\x04\x55\x73\xc8\x7a\x4e\xee\x3f\xf9\x35\xa6\x4c\xd0\x78\x12\x7e\xbf\x19\x91\x0e\x99\xe9\x63\x9a\xf1\x52\x5e\x4a\xf1\xd0\xe4\x8e\xf5\x1d\x4c\x19\x54\x77\x8e\x3e\xd8\x3e\x02\x87\xf3\x29\x9b\xbc\x27\x4d\xa4\xb9\x32\xad\xcd\x85\x27\x2d\x20\x26\x1d\x75\x8f\xf0\x98\x5a\x9f\xeb\x45\x6d\xab\xb5\x4a\x54\x2e\xb3\xd7\x12\x1b\x4e\x66\x3e\x58\xb2\xd0\x94\x8e\x9c\xec\x70\x4a\x69\x32\x20\xb3\x96\x7d\xe6\x70\xa4\x17\xd5\x44\x8b\xcc\xec\x2b\xe7\x88\xe7\x85\x4c\xf0\xdd\xcd\x27\xd5\xe5\x68\x39\x91\xaa\xc4\x17\x07\x3e\x84\x1d\xa9\x27\x80\x9e\xe9\x94\x3c\x81\x86\x12\x2d\xf5\xf4\xdd\xb5\xb2\xff\xd3\x63\x14\xa7\x35\x90\x4d\x4a\x6f\xeb\x9e\x83\xeb\x93\x59\x4a\xd5\xb4\x54\x9d\xa4\xec\x11\xe3\x23\xe9\xf8\x7d\xd2\x49\x50\x35\x01\x1f\xda\xca\xb3\x9b\xbb\xbc\x75\x74\xdc\xfa\xf7\xd1\xbc\x75\xfc\xa7\x7f\x3b\x6a\x1d\x1d\x15\xf4\x7f\xeb\xcf\x3f\x9d\x19\x6f\xa9\xce\x24\xbd\xdd\x2c\x65\xc3\x2e\x88\x5b\x4f\x3a\x12\xc1\x35\x40\xd3\xb5\xee\xa8\xde\x60\xe7\x38\xb7\xb5\x3f\xfe\xee\xe8\x4f\x7f\x00\xba\x01\x11\xd6\x8f\xa3\x15\x9c\x7c\x9e\x85\xd6\xef\xd4\x7a\xeb\xef\xe0\x46\xad\xaa\xe9\xc5\xec\x33\x6b\xf3\x4c\xd6\x68\xa0\x68\xf1\x61\xb2\xba\x42\x9a\x6e\xdc\xba\x98\x5c\x8e\xd0\x61\xf0\x74\xdd\xfa\x88\x36\x86\x16\x0b\x98\xcd\xd5\xf5\xa4\x95\x6a\xfb\xdc\x54\x27\x69\x3d\x6e\x61\x73\xad\x8b\xcf\xeb\x49\x95\xed\x21\x8b\x82\x4f\xfc\x18\x71\x29\x7b\x27\xa3\x40\x16\x4a\x16\x91\x25\x51\xbe\x69\x5f\x8c\x9d\x9a\xe2\x2e\xbc\x25\x77\x59\x2d\xf6\x6b\xa0\x88\x88\x79\x45\x18\xdc\xc0\x15\x1b\xe6\x0f\x53\x27\x5b\xee\x17\xfa\x93\x88\xd1\x40\x53\x09\x8a\xed\xb9\x2f\x3b\x42\x11\x4b\xf4\x03\x1d\x73\x61\xcf\xbc\x8b\xea\x94\xf4\x61\x3f\xdd\xd1\x73\xcd\x10\x60\xad\xae\xda\x87\x46\x43\xac\x0d\x7c\x7b\x46\x11\x7b\xc3\xcf\x88\xf4\xf2\x89\x1b\xb5\x83\xe1\x70\x8d\x3f\x72\x9d\x60\xd0\xd9\x75\x6d\xe4\x3a\x78\x62\xd0\xeb\xe9\x7b\x4d\xec\xc9\x47\x47\x48\xba\x32\x3b\x48\x49\x4b\x7c\x36\xda\x09\x08\xfa\xdc\x64\x63\xcc\x9a\x1a\xe0\x27\xbf\x17\x63\xb1\x4b\x12\xaa\x4b\x5f\xc1\xe9\x45\x0b\xe4\x6b\x7c\x65\xa4\x9f\xbd\x92\xe2\x9e\xcd\x11\xe4\xc0\x9c\xce\x26\x10\xec\x6b\xa0\x2c\x29\x93\x49\xe2\x88\xee\x3c\x2b\xc2\x98\x1c\xc8\xf6\x5f\x53\x49\xcc\xf9\x17\xd6\xc3\xc7\xe6\xda\x4e\xe6\x42\x65\x98\xd0\x65\xbc\xd0\xd0\x72\x54\x1a\xe5\xf8\x9e\x51\x46\x14\x80\x7c\xc6\x7d\x77\x91\x7b\xa5\x8b\x45\x27\x74\x1a\x6d\xc3\xdd\xa5\xcf\x93\xe5\x36\xbb\x0b\x63\xd3\x9a\x3d\x23\x45\x52\x99\x45\x74\x84\x43\x41\xb5\x94\xe6\xcc\xdd\xd9\x39\x4f\xb2\x34\x99\x4e\xc7\xe4\x29\x8c\x8b\x39\x83\x69\x23\x31\xef\xbb\x1e\xe1\x1b\x5d\x92\x4b\xbd\xac\x23\x58\x4d\xff\x3e\x41\x75\x6e\x40\x6c\xaa\x52\x74\x5a\xe0\x7e\x92\x95\xcb\xef\xa7\xe3\x42\xf2\xa3\xfd\x54\x56\xaf\x81\x1a\x96\xa3\xcb\xe9\xfa\x73\x29\xd9\xda\x6d\x09\x74\x35\x65\xb3\x61\x73\x46\xdd\x9f\x4e\xfe\xfa\xee\xbf\x4e\x5e\xfe\xfc\x2c\x9f\xad\xee\x90\x5f\x8d\xe5\xdd\xec\x33\x33\x42\x57\xd1\x32\xc2\x78\xc4\x03\x69\x7b\x58\xde\xa3\x91\x9d\x26\xa1\x36\x42\x36\x56\x77\x34\x6f\xa5\xb4\x41\x8f\x84\x80\xd6\xd9\xcf\xf2\x1e\xfe\xa2\x18\xa6\x67\x6b\xea\x66\xd4\xe2\xa1\x64\xb2\x7a\x44\x21\xc9\xb7\x42\x5d\x12\xb1\x66\x12\xce\x1a\xc0\xc6\x48\xb9\x28\xc7\xe8\xc7\x3a\x05\xed\x36\x9d\x18\xd6\x33\x74\x9b\xb7\x4b\xca\x50\x22\x39\x47\x8f\x12\x76\x50\x0f\x0c\x07\xb1\x18\xd7\xd1\x9d\x03\x70\x3d\xda\x8a\x0e\xf8\x57\x57\xaf\xb1\x78\xed\x6b\x06\x7b\x30\xdd\x9a\x69\x2a\x9d\x2d\x97\x53\xc7\x67\xcf\xa9\xcb\x39\x97\x53\xe8\x23\x72\x5b\x8b\x00\x62\xdb\x1d\x8d\x73\xe3\xa1\xb9\x3d\x38\xc8\x74\x5c\x27\xb3\x99\x75\xbb\xa3\x3b\x53\xb6\x73\xd3\x5e\x0b\xb6\xd9\x36\x17\xb5\xc4\x5a\x25\xba\x25\xe1\x38\x30\xaf\x46\xaa\xa2\x0f\xe9\x55\xb4\x39\xf1\x09\xe0\x6a\xd1\xe0\x05\xc8\x9f\x27\xaa\x30\xbf\xc7\xee\x15\xf8\x67\xab\xef\x40\x5c\x95\x31\x30\x69\x0f\x75\x17\xeb\x8d\x51\x76\x8a\xab\x3b\xfb\xe0\x8a\x8c\x02\x10\xbd\x26\xe8\x59\xbf\xc7\x52\xfe\x34\xf1\xf3\x14\xc6\xa5\xcc\x8b\xbf\x5a\x6c\xf3\xa0\xe5\x00\xce\xbb\x5a\x9a\x0f\x67\x58\xd4\xde\x7c\x8d\xb6\x49\xe2\xfb\x2f\x00\x63\xe6\x9e\xb3\xf1\x9e\xd3\x1a\x5c\x34\x89\xda\x42\xa9\x92\xe8\x0e\x0b\xcd\x78\x19\xf3\x5f\x44\xf5\x58\x7b\x3d\xb8\x49\xf4\xd1\xcd\xc5\x26\xb9\xc8\x2f\xbf\x7b\xfa\xe2\xed\xb3\x27\x67\x2f\xfe\xeb\x99\xf3\x5f\xf0\xcb\x79\xf5\x8d\x33\xf1\x73\x2e\x0e\x0c\xc6\xe8\x72\x06\x5d\x18\x9c\x57\x9d\xb4\xfb\x4d\xb6\x7f\x98\xb3\xb1\xfc\x7a\xf9\xd4\x66\x4f\xfb\xc5\x79\x91\x0e\x7e\xe9\x41\x38\xeb\xf7\xfa\xd9\x61\x3e\x42\x22\xe8\x74\x04\xb4\xde\xf4\xef\x44\x1e\xff\xe5\x06\x48\x49\x32\x09\x42\x6d\xa7\x4c\x7b\x6d\xae\xd6\xcb\xcd\xed\x68\x3a\x5b\x2f\x36\x90\xbc\xb9\x82\xf1\x66\xc5\x61\x3e\xbd\xbd\x3e\x5d\x5d\x7e\x65\x79\x2e\xb4\xc1\x9d\x5f\x4c\x6f\x47\xd7\x93\xf3\xc3\xc3\xfc\xd9\x7f\xe1\xc0\x7f\x3c\x79\xf5\xf4\xe5\xb3\xb7\xa4\x36\xe3\xc7\x9e\x2e\xe6\x83\xd1\xc1\xdf\x87\x9d\x0d\x5a\x31\x1b\xd1\x74\xc3\x60\xf5\x01\x46\x66\xa5\x34\x17\x3e\xdb\x3b\x7c\xea\x6d\x22\xb1\xb5\x15\xf9\x8c\xd7\xf7\x61\xd1\x4e\x6b\x08\xd1\xa1\xf8\x6c\x8d\x3c\xf5\x86\x89\xe3\xca\x4d\x39\x77\x5b\xe6\xc1\x1e\x68\x7c\xa9\x85\x0b\x24\xc8\x24\x24\x10\xca\xe1\xa8\x0d\x27\x2b\xc3\xda\xe1\x4d\x14\xca\xd1\x26\x35\x1b\x49\x4d\xb2\xb5\x75\x4b\x4a\x6a\x33\xc2\x6d\xd2\xba\xe8\x64\x43\xdf\x7c\xd5\xb5\x29\x10\x57\x71\xc8\x76\x0f\x6a\x2e\xf7\x63\xdb\xf8\xb5\xa5\xe9\x19\x09\x3f\x97\x98\xf5\x7d\x15\xf7\x62\x3e\xaf\x50\x29\x5e\x9f\x6b\x5b\xec\xb9\x8f\xae\x64\x6b\xb7\x7d\x14\x5e\x0b\xe8\x1a\x32\xce\x53\xd6\xaa\xa2\xac\x99\xb1\x9a\xd9\x5d\xae\xa6\x8b\x15\x62\x21\xf5\x28\xa0\x26\x4c\x46\xe3\x6a\xd6\xc4\x12\x95\x1c\x7e\x6e\x36\xe1\xe6\x54\xb9\xce\xb2\x16\xb3\xd9\xd8\x2e\xab\x32\x9e\x1d\x59\xad\x26\xa4\x93\x2f\xd7\x65\x3d\x6a\xb3\x41\x7f\xf2\x7e\xad\x99\x95\x63\xe6\xd0\xbf\x81\xc4\xdb\x84\x9e\x42\xcc\x9c\x54\xdb\x61\x16\x6d\x6d\xde\x19\x51\x9d\xba\xb4\xde\x7c\x82\x98\x33\xb6\xe2\xff\xb5\xa3\x9b\x89\xf9\x50\x7e\x2f\x7d\x00\x4c\xb9\x6d\x08\x55\x00\xc1\x6e\xb8\xc1\xaa\x9a\x29\x09\x70\x6a\x77\xd7\xc2\x79\x58\xe5\xa5\xd8\x9d\x4f\xba\xf3\x10\xd8\xfb\x0d\xfd\x79\xa8\x9a\xa0\x43\x0f\x64\x8c\x1e\x94\x2d\x34\x30\xd6\xce\x9a\x81\x43\x42\xe6\xcb\xf0\x37\x30\xf3\x85\x11\x64\x87\x8c\xdd\x53\x38\xed\xcf\x3c\x72\x64\xe1\xe9\xc5\x04\x6d\x8c\x35\x7b\xad\x30\x70\xc7\xf4\xa7\x0e\x84\xd8\x94\x5a\x64\x70\x4c\xec\xa1\x59\xc3\x71\xe6\xd9\xdb\x52\x98\xd0\xbe\x33\x5b\x17\x3d\x56\xa2\x43\x0b\x35\xba\x49\xea\xbe\xf9\xda\xd9\x02\x7c\x3e\xcf\x6f\x47\x9f\xde\xc8\x49\xce\xa7\xd7\xf3\xc5\xca\x58\xf5\x43\x82\x69\xba\xb8\xab\xe4\x86\x7f\xa2\x76\xde\x83\xea\xac\xc9\x21\xe6\xc4\x22\x93\xd7\xe6\x50\xc5\xd7\x20\x32\xf3\x6f\x6c\x61\xef\x3c\x0b\x12\x3e\x15\x96\x7e\x5b\x96\xce\xe3\xb8\x28\x96\xe9\x27\x7b\x4c\x13\x4f\x75\xe7\xa7\x9d\x43\xe4\x40\x05\x55\x0e\xa8\x92\x21\xd5\xa0\x7d\x21\x9f\xe6\xdd\x8f\xe8\x9d\x22\xf9\xbe\x5a\x8e\xe6\x8f\xbf\x3f\xa4\x9f\x24\x73\x8a\xb0\x28\x88\xb8\x65\xd2\x0d\xab\x5b\x54\x50\xf0\x25\x40\xc6\xe7\xf3\xd2\x56\xff\xd0\xec\x86\x49\xff\xc0\x5c\xd7\x6c\xa1\x8b\xbf\x08\xb4\xda\x03\x79\xe6\x50\x1a\xdf\xa5\xfc\x85\xcc\xe9\x09\xfd\x24\x5e\x0c\x7e\x1f\x41\x3a\xf6\xa2\x0c\x8b\xf6\x6b\xee\x26\x30\x99\xb5\x1f\xc3\xf5\x2a\xc2\xb1\x88\xd4\xbc\xab\xb7\x59\x66\x9e\xa6\xdc\xe5\x19\x4c\x87\xbd\xf4\x38\x5a\xca\xcd\xe6\x4f\x51\x4c\x86\x0c\x3d\xad\x76\xf2\xb7\x74\x9a\xa9\xd3\x16\x1e\x55\x2e\x6e\x59\xd4\x4f\xc8\xe8\xca\x1b\x11\x70\x05\xc9\x65\x96\xcc\x42\x34\x5d\xed\x76\xf8\x6d\x0a\x71\xcd\x79\xb4\xdc\x68\xb2\x39\x88\x90\x59\xf6\xe5\x5c\x28\xf3\x41\xfb\x6e\x1e\x74\xd1\x19\x3a\x70\xfe\x9c\x08\x57\xd0\x78\x7c\x9c\xe0\x9c\x3e\x43\xa3\xcf\xc2\x60\x13\xe2\xe4\xe1\x0b\x45\xb4\xff\x8c\xb7\x9d\x7f\xe8\xa8\x07\x8d\xd5\x67\xc0\xb5\x1a\x34\x74\x81\x12\xe3\x67\xa6\x1f\x7e\x33\x70\x69\x31\x17\xc7\x61\x3a\xe7\x14\xc1\x60\x8d\x82\xb6\x34\x3e\x2e\x4c\xf3\x39\x12\x76\x17\x3c\x5c\x6c\x12\xd1\x34\x95\xc7\x86\x8d\xa8\x3d\xa9\xed\xc3\xb0\x94\xa8\x24\xca\x17\x1a\x12\x70\x95\xcc\xb9\xa2\x19\xf5\xaa\x0a\xea\x99\x43\x3d\xb4\x95\xc3\xca\x06\xf3\x61\xee\x07\xa5\x25\x07\xd0\xea\xd0\x8e\x30\x4c\xf0\x05\xfa\xa9\x0f\xb3\x1f\x98\x7e\xea\xe7\xa1\x14\x47\x46\xf3\xc9\xc7\xd4\x09\x6b\xc6\x05\x00\x8a\x06\x00\x2d\x3c\x27\xbe\x32\xc0\x3e\xe2\x9a\xeb\xf3\x5c\x9a\xda\xfd\x2e\xaa\xe7\xdb\x6c\xf6\x6a\x2b\xdc\x6e\xdb\x7d\xd7\xf7\x35\xa5\x76\xa9\xcd\x2a\xd3\xaa\x07\xbb\xc6\x5b\x91\x0a\x76\x4e\x0c\x00\xe9\xec\xea\x96\xf2\x59\x6b\x73\xe6\x40\x67\x94\xa7\xbb\xbf\xef\x63\x48\x47\xdd\xb6\x97\x46\xb9\x73\x6d\x8e\x9e\x15\x9d\x81\x25\x32\x77\x5e\xcb\x5b\x6b\x49\xb2\xa3\xed\x8d\xb4\x61\x1a\x83\x81\x66\xc5\x57\x4e\x9a\x7b\xd7\x69\x38\x67\xb2\xcc\x0e\x6c\xf9\x0f\x73\x60\x8d\x3d\xb6\x07\x2b\xdb\xea\xe5\xb1\xe3\xe0\x7a\x54\x38\xc7\xa7\xcc\x2a\xe7\x7d\xfe\x1c\x2b\x92\x30\xbd\x1e\xf3\x03\x5c\xed\x80\xc2\x99\xa2\x62\x24\x5e\x7b\xe2\x7d\xcf\x19\x4a\xec\x12\x71\x1e\x63\x92\xbb\xb2\x27\x37\x87\xaa\xb9\xdd\xa3\xb2\x9c\xf6\x0d\x74\x2b\xfc\xa8\x22\x38\x97\xd9\xf3\x6a\xc8\x00\x71\xac\x49\x62\x5b\xbe\xb5\xb3\x05\x19\xc6\x33\xe3\xb4\xcd\x73\xdb\xbb\xc1\x2d\x71\xb1\x06\xd4\xcd\x1d\xd0\x95\xfc\x26\x07\x90\xc2\xf7\x8e\x31\x1c\x3d\x8d\x93\xd5\x2d\x9b\x9e\xdc\x33\x3d\x08\xec\xea\xed\x48\xd0\x91\x61\x4b\x45\xe3\x85\x11\xe6\xb7\xe0\xa9\x11\x1c\x14\xc1\xa6\xd5\x65\xb6\x60\x15\x23\xa2\x04\x33\xc4\xcc\xee\x92\xd2\x84\x51\x5c\x55\x8b\x6f\x36\x76\x9f\x35\xcf\x5e\xc0\xc2\x36\xf5\xf4\xa3\x9b\x8a\x26\xd9\x5e\x9a\x4d\x5b\xaa\xbe\x97\x1f\xb8\x2c\x45\xad\x91\xbc\x1c\xbb\x2b\x3a\x77\xde\x95\x03\x5c\x86\x6b\xfb\x69\xb4\x24\x4f\x88\x40\xa8\xe0\x4f\x4f\x54\x68\x1c\xc2\x73\x2f\x8a\x30\x23\x63\x9c\xdd\x6e\x3b\x17\x7c\x85\x8c\x20\xb4\x5b\xee\x1c\x28\xbf\x13\xc8\x1f\xd8\x59\xc9\xf2\xe4\x59\xf2\xe0\x10\x1c\x62\x88\xfd\x61\x8b\x35\x73\x1e\xc6\xb5\x9a\x7a\x61\xce\x79\x4e\x36\x57\x2a\x1e\x95\xf1\x10\xf9\x1e\x6e\xcb\xf7\xef\x4b\x4e\x85\xcd\x4a\xbf\x7a\xb6\xdf\xbf\x7f\xfc\xbe\xf7\x5e\x91\x40\x2c\x75\x8a\x22\x48\xe4\xfe\x72\xef\x98\xe6\xe4\xd9\x7c\x2c\x9f\xc8\x6d\xc7\x18\xa9\x6b\xf0\x7e\x98\xef\xa1\x91\x8d\xcd\x06\xff\x3e\x2e\xff\xb8\xd9\x90\x17\x49\xf6\x0c\x33\x45\x0b\x8f\xc4\x4f\x70\x1e\x28\x4d\x9f\xcb\xa6\x99\x22\x3d\x33\xf4\xaa\x89\xdc\xb5\x1f\x5e\xbc\x7a\xfa\xe2\xd5\x9f\xd9\xc7\x9e\x2f\x88\x44\x03\xd5\x6a\xfd\xd2\xb9\x54\x79\x7b\x4f\xff\x00\x13\x7b\x80\x4a\x6f\x01\x5f\xe7\x15\xb5\x6c\x72\x3b\xd1\x8d\x94\x86\xbd\xc1\x67\xb0\x7d\x92\xd9\xe8\x99\xae\x96\x65\x58\x43\x27\xa1\xdc\x09\xfa\x03\x0b\x66\x8c\xdd\x8c\x99\x39\x9b\x9b\x2e\x1d\xb1\x3b\x3c\x51\x16\xf9\x7d\xd6\x49\x26\xf3\x71\x62\x7c\xb0\x36\xe6\xfb\x03\xba\x69\x7d\x68\xc2\xe2\xfd\xa4\xdb\x78\x30\x67\x15\x3c\xd7\xa7\x4a\x63\x58\xec\x80\x4c\x4f\xe1\xba\xb5\xdb\x2c\xd9\xc2\xfe\xbf\xfa\x0d\xde\xb8\x69\x47\x05\x96\xb9\xa8\xd2\x47\xa8\x64\xea\xbd\x86\xe6\x0d\x36\x88\xe8\xc4\xce\x75\xd9\x82\x5e\xa0\xc4\x0e\x9c\x22\xcc\x6a\x0c\x9c\x1b\x8e\x68\x74\xd8\x65\x8f\xb3\x2e\xe2\xae\xe3\xc7\x27\x03\x39\x47\x0f\xa1\xd0\xc1\xa2\xd9\xf5\xca\xc8\x09\x93\x77\x00\xcb\x77\xb0\x03\x1c\x8e\xb9\xea\xd1\x7d\x98\x3c\x14\x3a\xf7\x11\x24\xeb\xcb\xee\x61\x9b\x59\xdd\x35\x0f\xb1\xd9\xce\xe5\x55\xcf\xb0\x5f\x1c\xef\x93\x87\x41\x48\x3c\xf5\xbc\xf4\x54\xf9\xb7\xc8\x1a\xf3\x03\x76\x21\xdd\x8c\x94\x8b\x79\x85\x1d\x2e\x71\x34\x54\x3d\xc1\xde\xc5\x6a\x32\xfa\x55\xd5\xfb\xa0\x8b\x68\xc6\xa4\x71\x2d\xa3\x7b\xd9\x73\x03\x82\x3a\xfe\x58\x20\x71\x25\x73\xb7\xe3\xa1\x81\x67\x2f\xaa\x84\xdd\xf1\x22\x5c\x78\x78\x26\x8f\xbf\x3c\x93\x3f\xfd\xc3\x33\x09\xcb\x94\x05\xd4\x9f\x9a\x48\xf4\xb8\x4b\xb5\x58\xad\xd3\x8b\xcf\x5a\x7b\xc0\xa2\xf4\x8a\x11\xab\xc5\xdd\xf2\xf4\x72\x24\x6c\x15\xb7\x57\x75\x9f\x7a\x0a\x8d\x50\xb6\xf1\x64\xb9\xbe\x41\xb4\xed\xca\xc3\x22\xe1\xb9\xdc\x8c\x2a\x63\x8c\xb0\x16\xe5\xf3\x67\xd9\xfd\x78\x41\x66\x05\xe8\x86\x12\xad\x20\xc1\x41\x9c\x52\x10\xac\x2b\xf4\x97\xf4\x82\x04\xcb\x41\x9d\x56\x77\xd1\xe4\xac\x7e\x2c\x2f\xec\x10\x03\x7d\xe4\xab\x17\x0d\x51\xff\xee\xfe\x18\x62\x29\x47\x37\x69\x18\x54\x2f\x66\x36\xe0\x6a\x3e\xd0\xe3\x76\x9b\x06\xde\xe9\xe4\xcd\xb9\xb0\x4e\xc9\x73\x70\xc0\xb8\xa4\x41\x7d\x38\x42\x1a\x84\x4d\x7b\x3a\xbd\x40\xed\x86\xed\xc7\x1b\xe4\xc1\x51\xa9\xc7\x47\x62\xc0\x39\x2e\xaa\x88\x8c\xa1\xe6\xac\xf1\x21\x5a\x3e\x55\xdf\x60\xbc\xe6\x2f\xe4\x3c\x18\x75\x5c\x08\x49\x6a\x58\xd1\x66\xd6\x91\x72\x1f\x18\xcd\xf1\x9c\xc5\x2a\xb6\x95\x59\xfa\x3d\xe3\xcd\xd4\x35\xb4\x23\x08\xdd\x17\xab\x37\x9c\x8b\x2f\xe2\xd9\x06\x53\xcd\x95\x25\x7a\x52\xc7\xb7\xdf\xff\xed\x09\xa3\x73\x28\x2d\xaf\x26\xd7\xdf\xf2\xed\x1b\xf0\x36\x78\xca\x2a\xf4\x05\xbb\x76\xe1\x2f\xb1\x3c\xe0\x50\x4b\x5e\x14\x1d\xa1\xb2\x4d\xd3\xbc\x44\xb3\xb0\xf6\x98\x60\xee\xf2\x81\x15\xc3\xca\xea\xf5\x90\xac\xca\xee\x57\x10\x33\x0a\xb5\xd4\x80\xd4\x3f\x76\x2a\x6e\x1d\xa2\x1e\x6c\xbe\x79\x1c\x5c\xd7\x43\x1d\xf0\x73\x27\x3d\x58\xa0\xb5\xa5\xc0\x5a\xa0\x37\x96\x88\x6f\x1a\x5c\x6e\x7f\x62\xfd\xa0\xf3\x85\x0b\x3b\x6d\x35\x45\x05\x1d\x76\x6d\x58\x8a\x4d\x59\x16\x6d\x19\xcd\x04\x2b\x74\x37\xa3\x54\x95\xb1\x3d\xc5\x5e\xf2\x0b\xa0\x15\x62\xf2\x50\x92\x8c\x05\x7b\xf2\xe9\x5a\x8a\x0d\xda\x4c\xc7\xa3\xf9\xe4\x06\x3a\x06\x4c\xee\x17\xcd\x05\x93\x56\xeb\x50\xe0\x2b\x37\xc9\x7c\xd7\x34\x10\x34\x03\x9d\xe5\xee\xe8\x60\x07\x51\x85\xc3\x14\x6d\x4c\x4a\x27\x42\xa3\x92\xf9\x1f\x4b\x5f\x2c\x30\x04\x69\x63\xf7\xf7\xed\xdb\x97\xb4\x46\x7f\x37\x9b\x5d\xf9\xf2\x5d\x09\xe2\xd4\x72\x4f\x46\xbf\xa7\x63\xda\x01\xa4\x2f\xd7\xec\x48\xd7\xf7\x9a\xe1\x71\xde\x52\x8f\x3e\x68\xe7\xdf\xed\x18\x06\xcb\x79\x0b\x20\xc7\xef\xd6\xad\x8b\x09\x03\xe8\xbd\x44\xbd\x04\x19\x72\xc6\x1a\x70\x62\x93\x40\xee\xb9\x88\x95\xa2\x75\xe5\xdd\x90\xd1\x3a\x9a\xbc\x1b\x68\x75\xe6\xc9\x89\x33\xdf\x53\x56\xde\xa4\x5f\xda\x94\x64\x97\x2c\xb2\x46\xb4\x83\x11\xc3\x00\xce\xf3\x7c\xbf\x82\xe5\xc9\x50\xd0\x71\x7c\x89\x91\x29\x00\xdb\xaf\x86\xde\xb2\x95\x7d\x56\x00\xf2\x40\x5b\xea\x07\x10\xb0\xa8\x6e\x46\xb3\xd9\xe2\xe3\x93\xc5\xf2\x73\x1a\xc2\xc6\x90\x91\x22\x0e\xa9\x53\xcf\x9a\x0e\x72\x33\x29\x0a\x83\xd7\xde\x29\x85\xea\x7a\x0b\xd5\xbd\xa8\x08\xe7\x22\xee\x53\x44\xff\xbe\x7c\xfd\xe4\xe4\x65\x20\xa8\x91\x0e\xfe\x67\xd9\x1e\x66\xe9\x79\xbf\x9f\xe1\xe7\xf9\xc7\x6f\xf0\x17\x28\x1f\x7e\x51\x09\x58\x74\x1c\xb5\xd9\xd0\xa7\xb3\xb4\xbb\xab\x49\x71\x93\x6d\x25\x4f\xa7\xf8\x1a\xb8\x90\x55\x79\xe5\x34\x77\x51\x12\x9a\xd5\xf1\xb9\x85\x3f\xa3\x9a\x2d\x37\x3f\x59\x33\x4a\x57\xfa\xd2\xf2\x6e\x64\xc7\x92\x91\x3d\xf9\x91\x52\x69\x8a\xd0\x4a\x4f\x09\x9f\x73\xb0\x80\x20\x80\x22\x6a\x68\x22\x46\xf3\x1f\x0f\x95\xc6\x97\xe1\xee\x4f\x79\x58\xe8\x98\x0c\x20\x5a\x35\x70\xb5\x0d\xd1\x1a\xce\xa4\xa3\x0d\x52\x2d\xcc\x10\x48\xfe\x67\x52\xc8\x9a\x2c\x2e\x50\xe7\x98\x91\x10\xca\xe5\x66\x42\x44\xf3\xa8\x3a\x5b\x29\x6f\x69\x21\xdc\xa0\x7d\xa9\x01\x90\x4c\xad\x03\x41\x02\x15\x28\xcd\xf2\x08\x4d\xe7\xf2\xa0\x8c\x59\x5c\xb5\x7d\xc9\x4c\xa3\xfc\x59\x6a\x2a\xcb\x2c\x22\x9e\x94\x49\x81\xba\x8e\x32\x77\x00\x75\xe2\xa2\xa2\xac\xe3\xd6\xad\xe4\xe7\xcf\x5a\x13\x7e\x3d\x4b\x97\xb7\x0b\x14\x06\xa0\xd5\x9b\x4d\x4d\x0f\xdd\x6d\x88\xb2\x36\x10\x57\x3a\xe8\x74\x1e\x83\xbf\x39\xf4\x97\x6a\x07\x10\xf8\xec\x13\xdc\xbb\x15\x7a\x7e\x14\x94\xf4\xae\x02\xe8\x07\x4b\x7d\x13\xc3\x3f\xf6\x85\x32\x3f\xe0\xa2\xe4\x40\x37\x89\x67\x77\xe7\x11\x63\xb7\xff\xac\xcb\xf1\xdb\x3a\x5f\xed\xd0\x9e\x25\xdd\x32\xca\xc8\xf5\x35\x17\x77\xc6\xf4\x7c\x4e\xa0\x43\xe3\xb6\x11\xc9\x08\x32\xb8\x6e\xf6\xbf\xd8\x61\xb6\x2b\xe1\x56\xd0\xb6\x9e\xdb\xfe\xed\xae\x28\xcb\x6c\xc6\x6d\xb0\xc7\xda\x49\xf1\xe5\xed\x53\xeb\x99\xd7\xe8\x14\x23\x1f\xc1\x2c\x44\x73\xa4\x16\x34\xb6\xd2\xae\x9a\x15\x69\xbe\x3a\xa7\xd5\xe5\x92\xec\xf8\xb0\x3b\x73\x81\x02\x2d\x6a\xbc\xe5\xe1\x10\x4a\x1a\x05\x1b\x08\xb6\x56\xb7\xf5\xd4\xa5\x17\xad\xfb\x6e\xb7\x8b\x7e\xeb\x0b\x48\x7c\x04\xbb\x0b\x3e\xd1\x87\xfd\x43\xfb\xc7\x83\xc6\xdc\xb7\xc4\xce\x65\xd1\x08\x55\xdd\x7d\x63\x65\x8c\xcb\x37\xa4\x36\x88\x3c\xa9\x72\x90\x66\x7e\x11\x9a\x19\x29\xef\x19\xc2\x14\xf2\x5e\x2a\x37\x4b\xe1\xae\x18\xba\x83\xaa\x42\xae\x49\xc3\x65\xae\xdd\xa7\xdb\x9e\x41\x5f\x9a\x04\x82\x72\x00\x97\x65\x19\x20\x49\xa6\x04\x6f\x82\x50\x64\x68\x68\x1d\x0a\x3b\x03\x7f\x06\x4d\x32\x15\x38\xbb\x29\x3b\x90\xb5\xfe\x4e\x54\xab\xde\x84\x1b\xbe\x3e\xd7\x25\x9d\xb0\x63\x01\x7e\xd8\xd0\x47\x2b\x99\xe5\x53\x4f\x2a\x51\x93\x9c\x01\xbc\xe7\x3d\xde\x9c\x6d\xd8\xd0\x27\x12\x4d\xe5\x07\x50\x43\x68\xc4\x6f\xa9\xc8\x49\x61\xf4\xc5\xe4\xc2\xb7\x90\x80\x04\xdc\x0f\x69\x40\x4e\x53\x32\xa2\xdd\x8e\x30\xb2\x30\xd9\x22\x66\x0f\xc9\x62\xe5\xa4\xd3\x0a\x37\xe9\xd9\xe8\x3a\x35\x65\x68\x67\x3f\xfc\x04\xe6\x44\x0b\xbe\xf2\x19\x0c\xe6\xc5\x92\x3e\x6a\x96\xa7\x37\x7d\x8c\x0c\x92\x83\x83\x60\x56\x7c\xc6\xff\x4f\x4d\xcb\x76\xc7\x1b\x4a\x73\xf4\x66\x73\xbf\x75\x0f\x04\x00\x62\x42\xd8\xe2\x37\x5f\x88\xc8\x7b\xe1\xa5\xbc\x11\x7a\xcc\xa2\x57\x43\xc6\xd2\xf4\x69\x4b\xd9\x57\xe5\x41\x4d\x9b\x61\x17\x90\xdb\xd1\xfb\xee\xae\xfc\x0e\xff\xfd\x62\x0d\xb5\x8c\x81\x20\x4f\x19\xe1\xd1\xfb\x91\x81\x7f\x93\x35\x33\xfc\x8a\x2f\xb7\xda\xe0\xa3\x75\x15\x31\x33\xca\x9d\x5c\x8e\xfa\x73\x7e\xc0\x2a\x91\xc3\x5d\x7b\xf3\x34\x02\x3b\x5e\xca\xd4\xe7\x42\x91\x9d\xe0\x0d\xc7\xc0\x5e\x40\x1f\x2d\x07\x21\x48\x99\xcc\xc7\x11\xf3\x30\x10\x27\x32\xbc\x25\xcb\xea\x69\xe2\x4a\xb8\x5d\xe5\x15\x1a\x6b\x9b\xe6\x71\x5d\x2c\x35\xe3\xdb\x19\x7d\x23\x84\x3b\xce\x74\x93\x45\x28\x48\x79\x28\xda\xe3\x65\x2d\xc6\x48\xa1\x1a\xe8\xab\x7d\xfb\x79\x35\xdb\x6c\x9c\xdd\xe5\xa7\x77\xcb\xd9\xf4\x12\xb1\xe3\x04\xea\x39\x94\x4b\x7f\xdc\x12\xb1\x8c\x9d\xfb\x73\xdc\xb8\xdd\x8c\xad\xae\x70\x24\xd8\xf3\x50\xb2\xc8\xce\x25\xca\x3f\x49\xcb\x4e\x0e\x6a\xe7\x49\xf2\x17\x7a\xfe\x70\x8d\x52\x93\x15\x0e\x7e\x55\x17\xee\xcd\xf7\x1a\xa7\xc8\xca\xec\x06\x17\xf5\xce\x15\x32\x17\x70\x13\x4c\x29\x9b\x22\x11\x7a\xe5\x0d\x4b\xf1\x3b\x73\xcd\xbe\x12\xdd\xd0\xcb\xc6\x7b\xd6\x48\xf4\x06\x25\x86\x3b\x57\xe8\x2b\xca\xda\x39\xde\x3d\x60\x7f\x66\x71\x75\x13\xf5\x45\x8d\xaf\x41\x41\x75\xc8\xf3\xab\x0f\x51\x4b\x57\x64\x49\xa9\x09\x9e\xec\x1a\x40\x13\x98\x32\x1d\x4e\x04\xc4\x25\xa6\x1b\x8c\xf0\xa7\x35\x00\x5e\x3f\x8b\xe6\x0c\x7f\xed\xb9\xff\x4d\x00\x37\xf2\xa5\x08\x60\x15\xbf\xc8\xb1\x4e\xb4\xe2\x05\xfa\x26\xb2\x95\x45\x8b\x44\xee\x75\x32\xcb\x7e\x2e\x6d\x47\x90\x01\x2e\x10\xf9\x2f\x40\xf2\xa5\x01\xff\x59\xfd\xac\x3a\x7b\x3f\x6e\xd4\x61\x8d\x59\x03\xa8\x76\x82\xba\xee\x0a\x0d\x80\x77\x3c\xc7\xb5\x6b\xa1\xdd\x8e\x63\xf8\x1c\xde\xef\x3a\xf1\xc5\xee\x4b\xb2\xbe\x13\x8a\xa6\x9d\x54\xbb\x23\x8b\x5a\xcc\x16\x4d\x5a\xf8\xb5\x97\x09\xb2\x5e\x29\xed\xb4\x90\x3d\xe7\x35\xf2\xab\xd3\x70\xf9\xbb\xe8\xf9\x26\x45\xdf\x06\xbf\x75\xe2\x02\x38\xe5\x20\x51\x66\x4c\xe4\xdb\xc3\x23\xc9\x49\x7d\x68\xbb\x8f\x4d\x0d\xa3\x68\xb8\x25\xc4\x1a\x63\x83\x66\x87\xef\x52\xbf\x1e\x17\x02\xdf\x60\xd7\x66\x45\x3d\x7b\xdc\xde\x78\x32\xd7\x87\x4a\x55\x0d\x8b\xef\x8f\x40\x67\xc2\xdb\x61\xa9\xe1\x1c\x66\xf4\xf1\x62\xb2\xeb\x54\x71\xf8\x15\x55\xdf\x49\xc4\x99\x6a\xb0\xb2\xc1\xd9\xd2\xea\xf0\x60\x1d\xef\xf9\x6f\x41\x4e\x36\x9b\x63\x7c\x06\x37\x5b\xc1\x89\xcc\x34\x93\xdb\xeb\xe5\x6c\x85\x02\xf9\x3a\xe2\x26\xba\x9a\x1d\x0b\xdc\x8c\xe0\x7b\xf2\x69\x74\xb9\x9e\x7d\x6e\xc1\x56\x6c\x21\x37\xd7\xb9\x89\x6b\x91\xcd\xe7\x10\xbd\x4d\xc8\x4a\xc7\x8e\xc3\x1f\xac\x95\xdd\x22\x3d\xc1\xa1\xcf\xec\xfa\x01\x5d\x8c\x80\xae\xb8\xdf\x6e\x7b\x1e\xd7\xb2\xc8\x99\x98\x36\xaf\x4b\x2b\xd9\x76\xec\x23\xf0\x92\x5c\x60\x4c\x3b\xc7\x79\x0d\xc5\x3b\xc0\xe8\x8c\x6e\xff\xa0\x17\x10\x75\x3b\x59\x5d\x4f\x6c\xa4\xb0\x8e\x6b\x7c\xe5\xb0\x60\x23\x26\xc9\x0f\x9b\xf5\x83\x1b\xed\x0a\xf2\x39\xd8\x84\x1d\x64\xff\x27\xcf\xa4\x6e\x78\x7a\xfa\xd9\xb5\xdd\x03\x69\x41\x69\xe3\xcc\x77\x38\x6d\x98\xff\x86\xd9\x9f\x86\xd0\x2c\x7a\xcf\x0c\x76\x51\x5d\xf4\x79\xc7\xdb\xe5\xff\x6d\xb8\xde\x4c\x4a\xb0\x8d\xfa\xab\xba\x8a\x99\x25\x91\x6b\x89\x0f\xc0\xb9\xfa\x84\x20\x32\xfc\xdc\x1b\x60\x23\x89\xc2\xbe\x79\xa8\x65\x3d\xe2\x5d\x4f\xe2\xc5\x4c\xb8\x02\xa6\x84\x10\xda\xf8\x28\xab\xc1\xe6\x07\xd2\xdf\x40\x78\x07\xd7\xda\xd6\xee\x76\xbe\xc7\x45\x0c\x21\x92\xee\x24\x59\xe4\x18\x9b\xfa\x09\xed\xa2\xde\x8e\x3e\xd5\xf0\xac\x06\xed\x3c\x6f\x0f\x26\x96\x13\xaf\xd3\x33\xed\x76\x2d\xaa\x6b\x65\xfa\x23\xb1\xcf\xb2\x61\xab\x08\x73\x25\xd8\xaf\xbe\xf0\xd6\xbe\xa3\xbb\x32\xa9\x81\x65\x2c\x76\xe8\xac\xdd\x3d\x24\x5b\x45\xb3\xeb\x04\x13\x81\xb8\x74\x72\x7b\xea\x7b\xb7\x2c\xcb\x58\xac\x46\xe7\x02\xe5\x44\x9d\xc4\x26\x8b\x8d\x42\xa1\x2f\x2b\xa5\x66\xca\x02\x69\x80\x1f\x95\x51\xe6\x44\x93\xb0\x46\x2b\x35\xfb\x0a\x5a\x1b\xcf\xc3\x0e\x72\x3b\x4f\xcd\x44\x58\x33\x3e\x9b\x8d\x49\x68\xa2\x78\xd1\x55\xc2\x5e\x83\xd2\xa3\x33\x5f\xe4\x6c\xa6\xb1\xe5\x00\x3f\xa3\x96\x04\x2b\xe5\xe5\xdb\xc7\xe4\xf7\x42\xe9\x17\xe1\x32\x10\x95\x5f\x98\xc5\xc0\xa7\x54\xb3\xbe\xb1\x72\xa5\xbe\xbe\xb9\x88\x2f\x68\x5b\xca\xfa\xdd\x86\x7e\xf2\x77\x5d\x59\x63\xb4\x1f\xb8\xba\x64\x8e\x34\x04\x30\xa9\x84\x5f\x7e\xe7\xcc\x21\x99\x62\xe0\x57\x62\x1c\x15\x41\x51\xfa\xe5\x9e\x22\xc7\xd6\x2a\x2c\x4b\x2d\x90\x59\x87\x64\x1f\x88\x31\x08\x79\x89\x03\x9c\xcd\xd5\x25\x99\x4e\xd0\x67\xea\x4e\x99\x26\xd5\xfa\xf3\x6c\x82\x26\x54\x21\xa1\x9f\xf4\x92\x02\x69\x8b\x8e\xe6\xcc\x72\x6a\xb6\x9a\xac\xbd\x85\x8d\x1c\x00\x80\xf4\x9c\xf3\x20\x57\x56\xfb\x03\x09\xcd\xfd\x61\xa7\x9f\xdc\x4c\xda\xac\x69\x24\xa2\x94\xd0\xe2\x40\xb2\x0f\xcb\xd4\x7e\xf5\xed\x07\x12\x41\x05\xfa\x46\xe5\x62\x85\x1b\x09\xb6\xe0\xb5\x94\xd6\x68\xbd\x8c\x93\xf2\xc6\x58\x34\xe4\x94\xb8\x4a\xf6\xb9\x02\x3f\x6d\x9b\x0d\xce\x40\x83\xfb\x81\xcd\x86\xfa\x66\x8c\x7d\xc8\xe2\x71\x54\x30\x43\xdb\xac\xa6\x0f\xd5\x7c\x43\x47\x77\x30\x5f\x2f\xc1\x13\xfe\x57\x5f\xbe\xbb\x04\x87\x88\x4f\x76\x05\x70\x5a\x7b\xf0\xca\xeb\x4d\x04\xf1\x4f\x8c\x50\x01\xde\x39\xec\x9d\x66\x30\xcc\x2f\x26\xb0\xda\x6e\x00\x4f\x1e\xa0\x42\x91\x0d\x48\xc6\x6d\x1b\xd0\x96\x4a\xdd\x5f\xe4\x63\x38\xc8\x1f\x26\xe3\xd3\x20\x9f\xb1\x3c\x53\xab\x24\xbf\x37\x08\x19\xab\x27\x98\x47\x19\xfa\x16\x74\x89\xc5\xea\x3d\xb6\x85\x06\x15\xcd\xd5\x5c\xaf\x3a\x40\xf5\xfa\x0f\xa7\xa7\x0d\x8b\x95\x15\x0f\x97\xe9\x39\xab\x15\x4d\x94\x23\xa9\xd7\x12\xa4\x46\x5d\x59\x0c\x9c\xad\x80\x0c\x98\x8c\xdf\x4e\xaa\xc5\xdd\xea\x92\x5a\xb5\x3d\xcc\xef\xc9\xfa\x45\x11\x6a\xe4\x6e\xb3\x6e\x75\x77\x79\x39\x81\xc3\xe5\x15\x92\x98\xb6\xd1\xa7\xb0\x10\x95\x09\xf0\x66\x4f\x3e\xf5\xc8\xee\x1c\x95\x6b\x24\xd4\xb4\xce\x86\x25\x0a\x48\xb5\x87\xe9\x31\xad\xe5\xff\xf7\x84\x58\xc3\x2c\x10\xee\x60\x17\xac\x57\x9b\x6e\x43\x65\x05\x5c\x9b\xe0\xd4\xef\x24\xdc\xf2\xaf\xa5\xbd\xea\xeb\xfc\x00\x5d\x25\x42\x8c\xb5\x12\x4c\x3c\xd9\x25\xd9\x09\x0b\x1a\x78\x23\xba\xd0\xf4\x68\x63\xe0\x80\x7a\xdc\x69\x82\x03\x59\xbe\x03\x56\x95\xbf\x4d\x3a\x73\x07\xa2\x1e\x4e\x6c\x03\xa4\xf9\x6d\xc0\xd5\x2b\x8b\x37\xaa\x0c\x92\x50\xf3\x34\xbb\x27\x01\x5c\xbb\x4f\xf1\xdd\xc1\x94\x00\xfc\x2a\x06\xa5\x78\xc7\xee\x82\xce\xbb\xf5\xbc\xf1\x99\xd7\xbc\x1f\xd6\xe9\x94\x9e\x03\xec\xde\x51\x33\x61\x26\x84\x8e\xfb\x44\x05\xd5\xe1\x72\xbf\x54\x6d\xed\x7a\x46\x8c\x79\xeb\x47\xd4\x90\xc3\x3c\x3e\x37\x17\x6f\xba\x51\x7a\xcd\x1d\x80\x53\xbf\x73\x23\xe2\x73\xb3\xd6\xb6\x8b\xa9\x17\x1c\xbc\xa8\xeb\xca\x31\x6d\x6e\xd9\x77\x35\xdb\xb9\x55\xd3\x9d\xd7\xea\x43\xb2\x7a\x7e\x7e\xb2\xad\xbf\x7e\xf9\x3e\x13\xb3\xe7\x46\xa4\xb0\x5a\x2e\xe6\x15\x82\x06\xa8\xe6\x66\x32\x42\x4f\xa7\x58\xc3\xd5\xf4\xda\x79\xba\xa8\xc3\xbe\xc5\x68\x1c\xb8\x18\xc3\x88\x96\x42\x2b\xf1\x78\xc1\x95\xa0\x49\x64\x42\xf3\xb4\x45\xa6\x62\xea\x63\xa1\x2d\xbe\x63\x18\xf7\x6e\x18\xfd\xd4\x2f\x39\xa1\xdf\xa2\xd1\x1e\xc5\xce\xdd\x04\x9b\x48\x53\x79\x2d\xcd\x34\x96\x15\xff\xe0\x6a\x3c\x34\x02\x23\x1c\xee\x55\x19\xd2\x51\x7e\xa1\xd6\x79\xae\xae\xca\x0b\x47\xea\x1c\x8c\x5c\x50\xaf\xfd\x23\x7a\x21\xb9\xba\xea\xe3\x9f\x62\x44\xf7\x03\x6e\x5e\x0a\xf4\xf9\xfb\x7b\xf9\x3a\x38\x2e\x8e\x21\x0b\x91\x45\x07\x17\xfc\x6b\xc8\xd5\x1a\x1b\xea\xe3\xcd\x68\xed\x00\x53\x13\x03\xca\x49\x38\xc3\x95\x5c\xcb\xb7\xe3\x86\xbc\xfd\x7f\xb9\x7b\xd7\x85\x36\x92\xab\x51\xf4\xf7\xde\xfb\x25\x44\x87\xc8\xdd\x56\x21\x10\x9e\x64\x27\xad\x69\x6b\x33\x18\xc7\x4e\x8c\xe1\x03\x26\xc9\x04\x34\x8e\x40\x0d\xc8\x16\x12\x23\x09\x6c\x82\xf4\x2e\xe7\xc7\x79\x92\xfd\x64\x67\xdd\xaa\x6a\x55\x77\x0b\xb0\x27\x67\x7f\xe7\x3b\x93\x18\x75\xdd\x56\xad\xba\xad\x5a\x55\xb5\x2e\x37\xc3\xd9\xa0\x4f\xaa\x0f\xbb\xf8\x79\x3d\xcc\x95\x56\x47\xed\x18\xe6\x88\xc1\xdd\xaf\x0b\x08\x7d\x42\x65\x07\xdc\x41\xef\x37\x17\x35\x92\x06\x7a\x81\x2e\x53\x8a\x35\x15\x0d\xe1\x50\x90\x90\xd7\xb7\x13\x4a\x58\x42\x9d\xd0\x9f\xa2\x65\xe3\xd9\x5c\x25\x01\x08\x54\x32\x10\x08\xc4\x4c\xe4\x14\x32\xc8\xa3\x9e\xfe\xe4\x5c\x78\x6f\x47\x30\xdd\x30\x45\xa3\x46\x05\x95\x05\x9a\xad\x4a\x82\x8d\x55\x2c\xac\x99\x0e\xd2\xc5\x46\x99\x4a\x91\x30\xb3\x52\x35\x12\x1d\x91\x40\x67\xdb\x66\xe2\xca\x03\xdc\x0a\x6f\x9e\xd5\x50\x5c\x2d\x09\xbd\x83\x3a\xe0\xa1\xc0\x5d\x00\xb7\x24\xa4\x39\x92\x8d\xc3\xe9\x18\x59\x29\xcd\x45\x70\x8e\xf1\xcc\xa9\xec\x7e\x5e\x67\xc7\xe9\x1e\xb1\x62\x99\x08\xaa\x45\x5f\x70\xb5\xbe\x61\x75\xbb\x72\xb6\xf9\x3c\x7a\xbb\xfb\x27\x38\xb5\x16\x74\x46\xf1\x81\x11\x0e\x53\xcb\xca\x8c\xd0\xe8\x4e\x65\x62\xd2\x21\x26\xfa\x60\xe7\x70\xef\xc7\x83\xed\x9d\x0f\x3f\x1e\xbc\x4b\xd9\x76\x63\x30\xa5\xbe\x56\x09\xcf\xc9\xf0\x3e\x30\xbb\xec\x39\x99\xae\x91\xc3\x51\xc4\xc5\xc7\xab\xea\x9a\xcf\xdd\x23\xba\xd9\x88\x0e\x77\xde\xed\x6c\x1f\x51\x44\xd8\xfe\x25\x0b\x73\x9a\x0f\x09\x0a\x2c\x4c\x91\xd6\x45\x0a\x3e\xbb\x04\xae\xd5\x42\x7f\xe6\x15\x94\x58\xcc\x73\x66\x7d\x44\xe7\xfd\x66\x4d\xc8\x9b\x90\x79\xbd\xf4\xb8\xda\xf6\xf2\xa5\xb0\x86\xe6\xf9\x1f\x59\x0d\x5a\x8c\x48\x3c\x91\x7b\xe9\x5e\xd6\x8a\x55\x11\x28\x96\x50\x88\xca\xee\x17\xd4\x7d\x4b\x6d\xcf\x89\x72\x2c\xdd\x81\x55\xf7\x11\x34\x64\x7c\xc5\xbe\xf4\x49\xba\xd1\x8e\x03\x1a\x26\x25\x42\xf5\xe6\x68\xf7\x5d\xed\xd5\xde\x2e\x7b\x9c\xf7\xdd\x35\xad\xc1\xea\xaa\xa1\xf7\xa0\x21\x99\x5f\x6e\xd6\x6a\xfb\xc3\x1c\xf5\xf4\xd0\x06\x33\x76\x32\x2c\xae\x1a\x62\x49\xa0\x62\x38\x6d\x5d\xa2\x53\x43\x88\x3d\x03\x6a\xfc\x89\xac\x0d\xc1\x2e\x5c\x43\xc7\xf3\x23\x8a\x4a\x6c\x5c\x33\x4a\xda\xff\xfd\x81\xb9\x83\x48\x88\xf1\xb6\x95\x0d\xb3\x64\x95\x71\xab\x4d\x81\x78\xa9\xb2\x59\x90\x64\xf7\xd7\x58\x4b\x58\x53\x3e\xb4\x82\x54\x8c\x83\x33\x7e\x92\x90\x83\x0b\x00\x81\xd7\xbd\xa5\xb1\xa9\xd7\x8b\x31\x62\x54\x4c\x64\xb5\x45\x0c\x3d\x79\x1a\xc1\x61\x58\xd3\x3c\x70\xe9\x8c\x8e\x49\x92\xc0\xa9\xf0\xd2\xc3\x91\x4c\x35\x38\x02\x1c\x90\x25\x50\x7c\x09\x7a\xef\x88\x31\x69\x33\x9c\xa3\xaf\x21\xc9\x6f\xb3\x65\xc5\x72\xfc\x6e\x8e\x5f\xec\xc5\xa3\x98\x2e\x2c\xb2\xc8\xf5\x66\x55\x30\x85\xdc\x63\xed\x38\x77\x35\x9a\x89\x32\x03\xa3\xe3\x4b\xb7\xae\x85\x72\x78\x18\xa8\xac\x4b\xfc\x08\xb9\x6c\x8d\x46\x37\x93\x96\x3b\xb1\xb8\x8f\xd9\xc0\x7c\xdc\xcc\x3e\x36\x54\xbb\xd6\x5a\xa8\x12\x5f\x85\x82\x55\x88\x87\x22\x80\x07\x86\x36\x3b\x41\x1d\x1f\xbb\x59\x18\xde\xec\xa6\x62\x01\xb5\x90\xaf\x5d\x01\x7f\x2d\x0b\xb0\x60\xc9\xe4\x05\xf7\x58\xbd\x2e\x1b\x9a\x8c\x32\xf1\x65\xb1\x34\xa7\x72\xf8\xf8\xcd\xf2\x7c\xd2\xbb\x08\xfd\x7b\xb0\x74\x85\xb5\xe5\xfe\x5a\xd2\x63\x38\x75\xca\x27\xba\x3d\xca\x47\x7d\xae\xa1\x12\xb2\x9d\x40\xc7\xcc\xfa\x37\xf3\x2f\xd7\xbd\x51\x7f\xdc\xad\x1c\x86\x62\x26\xd7\xf7\x9f\xb2\x96\xf9\xf4\x69\xd9\x2c\x6a\x7f\xfa\xf4\xf2\x53\xfb\x93\xb5\x3e\x60\x6f\x79\x4b\x73\xf2\x53\xb7\x2d\x07\x10\xcb\x11\x59\x03\xc2\x68\xb3\xba\xa2\x45\x36\x9b\x35\x4d\x5b\x01\x71\x51\x31\xf3\xed\xd4\x29\xad\x26\xc1\x37\x6b\x91\x75\x5a\x7f\x4b\x90\x15\x9d\x91\x09\xa9\x67\x4b\x77\xab\x85\x06\x89\x01\x3c\xcc\x42\xc4\x1f\xc5\x9f\x16\x6d\x0f\x4d\xb9\x9a\xbc\x5f\x75\x37\x4d\x5e\x80\xc0\x6d\xed\x66\xd5\x1a\x9e\x4a\x95\xb1\x1d\x08\x02\xab\x92\xdc\xdb\x2f\x34\x9e\xc5\x5f\x82\xfe\xcb\x8d\x7a\xdd\x9a\x9e\xf3\xa6\xab\x42\x5c\x8d\x83\xb3\x80\xa3\x18\x4f\xd5\x5f\x57\x91\x02\xf2\x50\x5d\xd3\xbc\xca\x92\xb5\xf9\x8c\x9a\x7d\xd8\x45\x4e\xcf\x47\xa9\x7d\x92\x7f\xdd\x57\x64\xac\x3d\x1f\xa1\x2b\xe6\xd9\xa4\x05\xbb\xf8\x64\x53\x29\x30\x92\x92\x34\x65\x9c\xb6\x32\xcc\x20\x0e\x28\xd6\x4f\xa6\x8d\xf5\x44\x52\x36\x31\x65\x33\x48\x41\x7f\x5d\xf9\x24\x75\xce\xc9\xc9\x32\x8e\xc0\xd1\x86\x71\xee\x6d\x0e\x4a\xcb\x24\x07\xbe\x0e\x7d\x84\x22\x1f\xa5\xc8\xa6\x23\x2f\x4c\xdc\x38\xb3\xe4\xde\x04\x5a\x91\xe0\x09\x6b\x30\xba\xc9\x6b\x54\x6f\x9b\x71\x67\xa6\x83\x72\xb9\x47\x16\x4e\x41\x3b\x05\xfa\x3d\x41\x34\xe9\xac\x9e\x61\x14\xb1\x31\xbc\x9b\x09\x91\xe9\xb0\xdf\xe5\xea\x9f\x4b\x27\x94\x99\x33\x04\x43\x55\xe8\x5b\x01\x25\x3b\x93\xf5\xff\x18\x89\xf5\xcb\xe6\xaa\x9f\x4f\x85\x82\x3c\x8e\x52\x5c\x17\xa4\x8b\x34\x51\x66\xb7\x3c\x2b\x4c\x06\x63\x79\x3f\x73\xca\xc6\x24\xfe\x92\xdf\x65\x15\xa6\x25\xc2\x36\xe1\xbe\x45\xde\x93\x7d\x21\xe0\x05\x0a\xed\x86\xf5\x75\xad\xac\xa4\xbb\x29\x95\xf9\x42\xdc\x1c\xfd\xa2\x61\x33\x75\xfc\xea\xe5\x64\x9b\x90\x3a\x5d\xaf\xac\x90\xc5\x95\x05\x0e\x63\x59\x69\x6d\xc6\x04\x51\x23\xab\x25\x89\xeb\x04\xc5\x09\x87\x8d\x01\x46\x26\xda\xd2\xac\xb2\x37\xe3\x81\xf3\x41\x4e\x12\x61\x3a\x1f\x21\x68\xba\x40\xe7\x84\xa6\x5b\x48\xb8\x56\x8f\x03\x7a\x29\xc2\x2b\xf7\xe1\xad\xf5\xf6\xc1\x0e\xa5\xc8\xde\x45\x90\x13\x4f\x27\xaa\xea\x7a\x7d\x25\x48\x16\xcd\xbd\xe5\xb6\x4a\x71\x0a\x7a\xd4\x96\x14\x7f\xc0\xb2\x68\x92\xc8\x50\xab\x61\xcb\xa2\x9b\x11\x9e\x15\xd3\xa8\x11\x40\x83\xae\x75\x24\x05\xda\xb1\xd2\xc2\x66\xdf\x0c\x87\x59\x66\x97\x4e\xd9\x29\x4b\x61\x16\x89\xa9\x75\x5c\x43\x8e\x26\xa5\x55\x2b\xcc\xa9\x00\x72\xf7\xb1\x93\x39\xcd\xdd\x4b\x21\x17\xd1\x0e\xd8\x4b\x77\x8f\xaa\x38\x4c\x9a\x55\x81\x4b\x31\x76\x63\x28\x23\xf4\xa8\x51\x5c\x20\xb4\x02\x2c\x24\xb6\xe7\x5a\x0b\x96\x7d\xe0\x15\x8f\x2a\xd3\x8a\xb3\xca\xb4\x78\x58\x31\x43\xe7\x8c\xaa\x88\x76\x81\xcb\xa6\xa1\x02\x26\xdb\xdb\x85\x0a\x7c\x36\xa1\x7f\xb3\x6a\x17\x91\x4a\x53\xce\x97\x11\x46\x7d\x3e\x47\x77\xd4\x24\x2d\x2a\x6f\x8e\xe8\x24\x8d\x7d\x2f\xd1\xe9\xee\xf0\xee\xea\x74\x3c\x0c\x0e\x1b\x4d\x95\x00\xfc\x04\xf0\x10\x55\x99\x5c\x34\xbd\xd5\x95\x9e\x81\xb2\xe8\xfe\x3e\x22\x7b\xff\x16\x14\xcc\xe9\xc5\x22\x42\x3f\x1d\x52\xb0\x33\x80\x72\x33\x3c\x3a\x7a\xf3\x79\x56\xa6\xd0\xaa\xb1\xb9\xb7\x1c\x67\x97\xe8\xe4\xfe\xe4\x7e\xfd\xc2\x28\xc0\x89\x4f\x5c\x2c\x20\xc9\x55\x00\x83\x5b\x30\x9d\x94\xad\xff\xcc\x56\x8c\x8e\xb7\xd6\xfe\xd1\x5d\x77\xc6\xf4\xf9\x84\xa8\x4d\xc3\x97\x79\x8c\xd0\xd9\xf3\x19\x04\x86\x64\x61\x68\xa4\xcd\x26\xed\x1f\xec\xbc\x7e\xfb\x77\x39\x8b\xa2\xd8\x5d\x12\x58\x9b\xb7\x77\x86\x05\x77\xb5\xca\xd4\x04\x9a\x99\xdf\x7e\x7f\x74\x40\x2a\xbd\x68\x48\xfd\xe4\xb0\x91\xc4\xb0\x01\xf7\xa6\x68\x32\xfe\xe4\x33\xda\x7f\xb7\x86\xd4\xad\xe9\xe5\x82\xef\xab\x33\xef\x56\xdb\x9a\x57\x5d\x66\x29\xfd\x4c\x2b\xf3\x7b\x53\x8d\x74\x51\x22\x8f\xad\x0a\x39\x3e\x61\xa6\x2a\x46\xce\x86\xaa\xc6\x07\xac\x1a\x5b\x0f\x36\x55\x16\x86\xc5\x03\x5a\xc9\xf0\x47\xee\x94\x56\xab\xdd\x98\x8b\x29\x33\xe5\x48\x9c\x67\xd5\xf9\x80\x95\xd3\x9d\x1d\x06\x0f\x08\x89\x23\x8b\x73\xf8\x38\xa1\xa9\xae\xe7\x13\x0d\xd2\x29\x46\x2b\xd8\x4e\xb9\xda\x78\x20\x4a\xa4\xbf\x24\x95\xa3\xc7\xa4\xa3\x3b\x50\x25\x74\xd1\xf3\xf8\x0c\xed\xa2\x68\x4d\xb8\xa0\x71\x2b\x28\x7b\x20\xb9\xac\xb7\x9e\x42\xba\x71\x06\x75\x5f\x07\xdd\x57\xc8\xe6\x5d\x2b\x07\xb6\xdd\xbd\xeb\xf8\x72\xcf\xab\xe6\xd3\x8d\xd5\x0a\xc7\xc3\xc2\x1e\xd3\xbc\x81\x7d\x10\x79\xf5\xf1\x79\x2d\x68\x40\xc9\x1f\x8d\x9a\x73\x78\x1d\xc3\x6a\xa6\xdb\xe4\x61\xbd\x06\xb5\x8e\x27\x33\xb5\x22\xe4\xbd\xb5\x37\x65\xd5\xe4\x95\xda\xfb\x71\x8d\xe1\xd6\xb8\x56\xeb\xf0\xba\x5f\xbb\x1d\xf4\x6a\xff\xe4\xaa\xff\xd9\x8c\x74\x83\xe7\x73\x35\xd4\x34\xf3\x55\x53\xda\xa1\xe2\xa1\x4f\xe9\x66\x05\x1f\xf2\x6e\xda\x2d\x02\x2f\x12\xf6\x58\xf9\x98\xe7\x26\x37\xa9\x0b\x93\x3d\x70\x02\xe8\xdc\x0d\x87\x9e\x2a\x76\x0a\xbb\xd7\xb2\xba\xc8\x37\x94\x5f\x60\x10\x2c\xaf\x29\xed\x25\x57\xdc\xfd\x92\x9f\x27\xe7\xe4\x37\x09\xda\x47\xaa\xc8\x6f\xf8\xf9\x28\x96\x67\x24\x5e\x85\xc2\x39\x9a\x81\xa1\x3c\x7d\xe5\x6f\x44\xf2\x75\x62\xe7\x87\x80\x23\xac\xb7\xbb\x93\x51\x94\x68\x77\xb4\x23\x14\x0a\xc0\x37\x3f\xe5\xb1\x2e\x85\x2c\x50\x85\x72\x60\x4d\xa2\x01\x94\xc9\x99\x86\x1b\x24\x6c\x04\x23\x2b\xa5\xb1\xe4\x30\x31\x50\x31\xe3\x47\x7b\x60\x47\x7d\x37\xb2\xc8\xd4\x48\xce\x27\x55\xb1\x19\xb9\xea\x4d\xa4\x51\x89\x24\xf9\x0e\x91\xa6\xfc\x89\x97\x61\xd0\x23\x12\x00\x3a\x9a\x39\x6a\x6a\x33\x74\xe4\xc3\x9b\xf8\x2c\x59\xc8\x0e\xb6\x18\x0f\x0b\x5d\x4c\x78\xc0\x95\xe3\x91\x10\x6d\xee\xf8\x6c\xc7\xbe\xd7\x08\x2a\xb0\x1b\x64\x48\xd3\xe7\x50\xb7\x68\x24\x28\x83\x9e\x3a\xd0\x62\x4c\x8c\xcf\x07\xee\xbd\xf0\x7c\x34\x55\x86\xf0\x5f\x7b\x4e\x0b\x1a\x04\x7f\x82\xcc\x49\xea\x86\xfb\x5c\xbb\xa0\x45\x66\x8a\xbc\xe6\x00\x33\x12\xe4\x17\x67\x51\x6a\x93\x04\x7a\x2d\x42\x2a\xe8\x0b\xe7\xc6\xd7\xcd\xc1\x97\xd9\xe6\x06\x9c\xa5\x5f\x6c\x6c\xbc\xe4\x08\xb5\x44\xde\xcc\x66\xd7\x85\x8d\xf5\xcf\x87\x7b\xef\x3f\x1c\x1e\x6d\x1d\x1c\x89\x99\x8f\x93\xe3\xf9\xc9\xfd\xf1\xcf\x27\xf7\xdd\x64\xdd\x50\xea\xce\xfb\x57\xd9\xfa\xf1\xc9\xe2\xa4\xdb\x65\x83\x1f\xfb\x07\x7b\x47\x68\x61\x0d\xd2\x78\x37\xc7\xa2\xc9\x49\xf7\x64\xf1\xcc\x74\x4e\x46\xeb\x66\x7b\xef\xfd\x11\xde\x3e\x1f\xfd\xb4\xbf\xf3\x61\x6b\x7f\xff\xdd\xdb\xed\x2d\xca\x8e\xf0\xb2\x7b\x52\x80\x86\x55\xb4\x86\x22\x26\x30\x87\x71\x81\x0d\x58\x78\x71\xfd\xe3\x74\x3c\x6a\xa3\x14\x19\x1c\xee\xb3\x9b\xd9\xf9\xda\x1f\x22\x74\x31\x44\x5a\xf8\xc2\xfd\xba\xd0\xbd\x1b\x94\x03\x79\xc8\x4d\x8f\xbd\x16\x3b\xf6\x9a\x1f\x16\xd9\xe3\x28\x16\x05\x22\xb1\xaf\xf1\x8f\xe2\x4b\x0a\x8d\x42\xde\xc4\xf8\xee\xe1\x6b\x73\x29\x6f\xfb\x25\x88\x64\xa0\xe7\x93\xf1\xd5\x9f\xa7\x16\x81\x84\x87\x6f\xd1\x35\x0a\x57\x72\x72\xa7\x51\x55\x78\x5a\x5d\x40\x80\xb7\x02\x93\x89\x6c\x8b\x25\x9d\xd9\x98\x41\x26\x69\x1f\x40\xd9\x55\x82\xee\x42\xae\xc6\xa3\xf4\x7e\xeb\x0c\x49\x5f\xb9\x23\x4d\x0d\xaf\xbe\xd7\xa1\x81\x03\xf8\x7e\xbe\xfe\x1c\xfa\x12\xc5\x31\xd2\x07\x07\xc8\xa0\x87\xb0\x47\x72\xe0\xd6\xfe\x70\x9e\x85\xf9\x32\x9d\x9c\xb3\xc3\x46\x3a\xf2\x46\x7f\x3f\x3c\x78\xbd\x76\xb4\xf7\x97\x9d\xf7\x11\xa5\xf1\x02\x95\xb4\x35\x95\xba\xe0\x0b\x7a\x6c\xd3\x78\xc2\x1e\x36\xd0\xdb\x27\x7b\xa7\xf0\x29\x74\x4d\x63\x9f\xf0\xdf\x2e\x2d\x51\x91\x83\xbc\xc0\x04\x9b\x02\x0a\x90\xfd\xd0\x3b\xfb\x44\xe6\x35\xa3\xd5\x53\x76\xf7\x47\x3e\x22\x02\xa7\x4a\x05\x2f\x11\xbf\xb0\x2b\x0a\xcb\xc0\xf9\x5d\x45\xc1\x33\x16\x9a\x09\x60\x05\xbe\x1e\x7e\x31\x0e\x8a\xba\xab\x22\x28\x64\xd4\x88\xdc\x48\xb2\x54\x42\x99\x22\xd9\xc9\xef\xc4\x19\x78\x65\x63\x48\x89\x03\x3a\x59\x07\xa2\x33\x69\x48\xcf\x6c\x22\xbd\x8b\xba\xac\xcd\x50\x22\xa2\x59\xaa\x31\xf1\x2e\xe0\x3c\x55\x72\xa5\x85\x3c\x75\x30\x22\x5d\xfd\x05\x06\x82\xa6\x36\x06\x93\x82\xc4\xaf\xa5\xd5\x67\xc5\x46\xa2\xe9\xc6\xca\x8d\x95\x03\x42\x48\xda\x85\x2d\xd4\x0f\x04\x47\xbc\x1e\xc9\x9a\x81\x1d\xd4\x13\x68\x9b\x86\x4b\x37\x00\x97\xd9\x14\x38\xb8\xe1\x8e\xb0\x92\x05\xc9\x76\x0b\x39\xe6\xdf\x6e\x98\x6a\x2f\xf6\x0b\x99\xf0\x4d\x86\x64\x1d\x72\x35\xef\x8d\xdb\x81\x5e\x05\xd1\x30\xe4\x2a\xe4\x8a\x4c\x33\x4b\xfa\xdc\xc8\xb8\x9c\xda\xe7\xa0\x0c\x97\xed\xaf\xb6\x02\xe0\x33\xf9\xc8\x26\x93\x11\x15\xa3\x76\x46\x81\x75\x45\x76\xce\x92\x2e\x1c\x76\xd5\x88\xf8\x12\x61\xbc\xc7\x2a\xb1\x16\x54\x24\xfc\x16\xd6\x20\xd1\x27\xba\x25\x0d\x3a\x83\x9c\x02\x7a\x80\xf7\xd5\x7d\xa3\x58\x9d\xa0\x34\x4b\xc7\x05\x1d\x87\x00\x15\x26\xe4\x83\xdc\x96\x0d\x32\x26\x68\x8e\xab\xb2\x36\x7f\xc5\xba\xa4\x19\x6d\x5f\xc1\x71\x80\x4f\x37\x53\xdd\x19\xa6\x2c\x9c\xd3\x41\x5b\x74\x21\x47\x5c\xe8\xe9\x60\x53\xe3\x8d\xc2\x0d\x7a\x31\xc5\x94\xf7\xbf\xaa\xbc\x9c\xb4\xb0\xdb\x46\x16\xac\xb8\x90\xba\xb4\xfd\x89\x16\x82\x26\x4c\x2c\xcc\xab\xac\x40\x1d\x78\x8a\x64\x37\x68\x86\xb0\x62\xea\xd0\x85\x0a\x92\x7d\x96\x8d\xb8\x99\x0c\xdf\x4e\x0f\xa1\x3b\xf6\xc8\xe4\x41\xac\xe4\xa5\x3a\x96\x60\x5a\xc7\xd5\x71\x72\x2c\xc9\xe1\x96\x32\x9f\xbb\xe6\x86\x09\x5d\xc5\x40\xba\x2a\xdd\x2a\x9f\x6a\x68\x7e\x5c\x0a\xd0\xf4\x50\x3a\x18\xe2\xea\x93\xee\xa3\x64\x10\x32\x2d\x2a\x4c\xc4\xcb\xf6\x4f\xd8\x5d\x6d\xa6\xc9\xbf\x20\xbd\xcd\x42\xea\x2b\x19\x35\xdf\x57\xe0\x9c\x2b\x28\x30\x55\xae\xc8\xaf\xf7\xb7\xa9\xc0\x25\xfe\x4a\xb0\x44\x1a\xf9\xaa\xda\xd2\xc5\x48\xa4\x4c\xd7\xf0\x7c\x1a\xe9\x05\x21\x25\xc9\x34\x6a\x15\x65\x5b\x84\xde\x3e\xa5\x76\x34\x9a\xb5\x3d\xc9\xe9\xa4\x88\x27\x63\x62\x67\x7c\x2e\xd7\xd5\xe5\x7c\x4b\x20\x64\x4b\x8b\x98\x29\x4c\x59\xe8\x10\x35\x6b\x5f\x69\x16\xba\x39\xbb\xcc\xc5\xf5\x83\x5e\x10\xe5\xd5\x93\x00\xe9\xbc\x04\x56\x29\x3b\x0e\x46\xd8\xdb\x8f\x41\x5b\xff\xe3\xab\xc1\x34\xcf\x60\x3b\xfb\x8c\x50\x65\xcc\x89\xf8\x78\xcb\x85\xe4\xfc\xab\xaf\x19\x0e\x25\x01\xe8\x63\x93\x7b\x1d\x6a\xca\x6a\x9b\xcf\x2b\x22\x77\xf0\x24\x9a\xa0\x4e\x16\xe0\xe7\x24\x7f\x2b\x32\x9a\xa5\x85\x4d\x21\x3b\xb7\xb9\x58\x1b\xc7\x86\xd5\x79\xa1\xaa\x42\x36\xb3\xbc\x2c\x70\x06\x5c\x38\x90\x8f\xc5\x91\x40\xc1\x5b\x4a\xb1\x42\xab\xcc\x17\x14\xa3\xdb\xb6\xab\xe5\x57\x46\x91\x00\xb8\x22\xee\x85\xcb\x66\x12\xf9\xfd\x2c\x38\x58\x15\xf2\x10\xa0\x92\x0c\x68\x82\x57\xe2\x4b\xf8\x20\xe6\x64\x96\xf1\x45\x74\x10\x66\xd8\x0b\xfb\xc1\x57\x07\x8f\xa3\x41\x4a\x17\xff\x2f\xe3\xa2\x74\x68\xe8\x8d\xff\xf0\x72\x3c\x99\xb1\xe1\xd2\x69\x4c\x8f\x91\x34\x6d\xdd\xdd\x86\x29\x1c\xb6\x89\x0d\x95\x6b\x4c\x97\x04\x84\xda\x09\xca\x5a\xdd\x0c\x62\x57\x83\x1d\x84\xac\x75\xdc\xf3\x0e\x90\xd2\xad\x12\x94\x4b\xe1\x1f\x8a\xa9\x04\xda\x3d\x25\xcc\x50\x5e\x85\xa8\xe3\xaf\xc1\x90\xba\xee\x57\xa0\x49\x00\x52\x3a\xc1\x15\x10\x5e\x42\x73\x14\xbf\xe1\x99\xd8\x3e\xca\x4d\x17\xc6\xcd\x52\x27\x3e\x97\x26\xac\x8f\x02\xc4\xaf\x74\xba\xef\xb0\x8b\x59\x38\x93\x51\x8b\x8e\x8b\x70\xaa\xee\x3a\x04\x68\x37\x61\x97\xd3\x56\xf2\x01\x37\x57\x5c\x6e\xf4\x40\xb7\xcf\x93\xc3\xcb\x40\x0b\xe0\x10\x44\xf0\xa0\xb2\x7a\x7d\xd9\x43\x8a\xa1\xe3\xf8\x72\x2c\xd1\xe2\x46\x4f\x01\x9f\xdc\x73\xd8\x6b\xc0\x4a\xfa\x06\xd2\xa9\x52\x1f\x90\xd3\x78\x20\xf9\x4d\x81\x9e\xaa\x08\xa4\x04\x49\xcc\xa7\x9a\x42\x6d\x69\x58\x69\xfa\xe0\xe6\x9a\xf2\xcf\x22\x68\x0b\xf6\xdb\x7e\x6e\x9d\xd3\x5b\xc7\xc1\x7d\x71\x82\xb9\x77\xce\x27\xbd\xe6\x75\xae\xfd\xd7\xbb\xc5\xd8\x46\x95\xcd\x0c\xb2\xd7\xeb\x95\xf9\x9c\x62\x79\xff\x8b\x69\x25\xde\x8d\x31\xbb\x01\x46\xed\xa5\x6b\x63\x5b\x8a\x5b\x0e\x7d\xc7\x6e\x85\x67\xae\x17\x24\x02\xa7\x6d\x76\x7a\x33\x18\xf6\x51\xe1\xc9\xb3\x54\x96\x7d\xb8\x26\x7f\x94\x24\x1e\x58\x8d\x8f\x95\x06\x27\x76\x2f\x20\x54\xc5\xae\x30\xc5\x08\x18\x37\xa9\x85\x90\x57\xdc\x14\x85\x71\x33\x51\xc9\xf2\x8e\x1a\xfd\x69\xe7\x88\xed\xfa\x79\x4e\x11\x39\x00\xcc\xe2\xaf\x07\x75\x41\x7a\x1a\x70\xa1\xd4\x5f\x98\x84\xb5\x75\xc2\xb0\xe5\x8b\x49\xcd\x2b\xe1\xee\x4d\xac\x67\x78\xea\x67\xf1\xe4\x8c\x3a\x64\xb8\x4c\x8c\xf7\x68\xe9\xf3\x24\x74\xa7\xef\xc3\xd4\x33\x89\x76\xe5\xec\xa3\x9f\xd2\x61\xbe\x48\xdb\x5a\x44\x56\x95\x75\x0a\x2b\xc9\x27\xe1\x3b\x8b\x0a\x6d\x40\x08\xcd\x05\xab\x28\x74\x2c\x90\x2e\x2d\x6f\x36\x37\x36\xcc\xfd\xc2\xaa\x25\x05\xf4\x45\x06\xbd\x9a\xb1\xf4\xc8\xc9\x84\x96\x2b\x8e\x90\xd5\xc7\x59\xe8\x08\x22\xd2\x3e\x45\x15\x1d\x23\x3b\xb8\xca\xc7\x37\x33\x53\xcd\xeb\xd9\x68\xbb\x9c\x49\x15\xad\xbc\x9d\xb9\x99\x4e\x88\xf3\xdc\xa6\x57\x17\xf9\x96\x26\x40\x6a\x5b\xc4\xcb\x67\x15\xfe\x7f\x0f\x49\xb0\x37\xe6\x32\x95\x4a\xb8\x7c\x07\x20\xe6\xb4\x6f\xd9\x2a\xaa\xe3\x09\x89\x66\xf3\xe0\xf1\x83\xfa\x7c\x6e\x8d\x59\xd3\x4f\xd7\x6b\x37\x31\x48\x5f\x03\xde\x46\xc8\xf4\xbd\x25\x1b\xd8\x99\xdc\xf3\xdd\xb2\x15\x54\xbb\x20\xf3\x11\xb9\x57\x99\x0c\xfe\xe3\x26\x9f\xb0\x16\x6d\x23\xca\xa2\x46\x21\xfe\x36\x61\xe9\x4f\xec\xff\x46\xbc\xd6\xca\xf0\x90\xe5\xdf\x0a\x3a\x51\xd2\x81\x3f\x69\x54\x8f\x92\x06\x43\x67\xc1\x1a\x08\xdb\xab\x09\xb7\x46\xb2\x82\x7b\x70\x76\xa6\x9a\x98\x2a\xde\xb6\xc2\xa5\x72\xd5\xfd\x5d\x25\x0b\xac\xbc\x56\x97\x01\x7b\x3e\xd7\x5e\xdf\x56\x94\xec\x84\xaa\xfa\x15\x39\xd2\x92\x67\xe6\x8a\x4c\x5a\x19\xfb\xa1\x3b\xc5\x87\x5a\x61\xc4\x9f\xa9\xbd\x7d\x43\x18\xc0\xd6\xfe\x1f\xc0\xbe\x5d\xd9\x7d\x76\x77\x21\x7f\xcd\xb0\xe2\x2d\x4e\x69\x05\xc3\xe9\xee\x24\x2c\xda\xb1\x3d\xe2\xb8\x3c\x89\xbf\x44\x24\x1e\xff\xc9\x60\xd4\xc5\x9f\x00\x42\x11\x65\x53\xb9\x05\xe1\x9d\x6e\x05\x9b\x1a\x41\xef\x44\x26\xe2\x13\x28\x7c\xe0\xde\x0d\x3f\x78\xc9\x8d\xd3\xf2\x01\xf6\x31\xc2\x0b\x6f\xc8\x0a\x04\xce\x69\x0a\xbb\x17\x04\xfb\xc1\xf1\xc1\x73\xe2\x1b\x4f\xde\x96\xbd\x24\xaa\x5b\x62\xfb\x80\xa9\x1d\x07\xfb\xdb\x60\x77\x01\x2c\xef\xd1\x2e\x93\x17\x96\xa0\x06\xa8\x3a\x7d\xa1\xbf\xbf\x39\x70\x57\xc8\xcc\x01\x58\x38\xcd\xde\xe8\xe2\x66\xd8\x9b\x90\x43\xd7\x53\x28\x37\xf5\x90\x71\x57\xb0\xd9\xac\x65\x08\x16\x16\x3d\x1b\x0f\xdd\xa3\x07\x50\x03\x16\xc1\xe8\x16\x59\xf1\x47\x51\x79\x45\x98\xf8\x9a\x27\xbd\xcf\xf6\x95\xd7\x59\xfd\xd8\x97\xfa\x14\x2f\x4c\x23\x86\xac\x14\x71\xd7\xe3\x51\x6e\x15\x27\x27\x83\xeb\x59\xa6\x80\x88\x40\xb2\x48\x0d\xc7\x11\xe7\x80\x21\xc4\x42\xe2\xed\x22\x53\x0f\xb6\xba\xe8\xe9\xb8\x7f\x67\x85\x48\x49\xb8\x97\x0b\x73\xd9\x7a\x9d\xf8\xf1\xc4\x3d\xc4\x72\x62\x93\x84\x68\x23\x7a\x30\xf9\xd8\xbb\xed\x49\x7d\x46\x52\xa7\x93\x33\x24\xa7\x06\x45\xdf\x3a\x12\x37\x1e\x01\x8a\xfd\x3b\x64\x32\x73\x38\xbb\x8e\x2e\x72\x8d\xd0\x3a\xaa\x05\xe6\xfd\x39\x8a\xce\xe0\xac\x5d\xe7\xc7\x22\x29\x4b\x25\x0f\xb1\x64\xc2\x08\x59\x07\x1e\xc9\x22\x75\xe0\x11\x42\xe6\x42\x7c\xa4\x54\x79\x4d\xa9\xcd\x5a\xa0\x59\xb7\x59\x0a\x2c\x8a\xaf\xa9\x6a\xc3\x26\xbb\x38\x76\x38\xdd\xdb\xa6\xdd\xa7\x8b\x1b\x74\xb0\x33\xab\x37\x09\xce\x2e\x8b\x39\x76\x5c\x3e\xca\xce\xe3\xc0\xbf\xa2\xfe\x77\x9f\xc0\xca\x7e\xb9\x9c\xd4\xeb\xf0\xa7\xd9\x3b\x45\x4f\x47\x05\x83\x0f\xd8\x6f\x16\x98\xc3\xed\x91\x93\x14\x6d\xf5\x32\xf1\xb2\xe2\x4c\x9c\xcf\x95\x88\x22\x32\x7a\x6e\x4d\xb4\x05\xf7\xb7\xb0\xa3\x07\x73\x1c\x56\xd7\xe8\x2c\x1f\xc6\x2e\x3d\xf1\x6d\xc9\x00\x73\xd2\x2d\x15\xa4\xb2\xe8\x7c\x40\xea\x49\x16\x6a\xc7\x62\xd9\x01\x9e\x2b\xfd\x6e\xe3\x3b\x7b\x24\x91\xfc\xad\xcd\xcd\x17\x24\x03\x06\x01\xc8\xe2\x92\x6d\x63\x1f\x39\x36\x7a\xc2\xb0\xba\x6a\xfb\x6b\xef\x66\x86\x22\x18\x8a\xa8\xc6\xa3\xf1\xf8\x9a\xf7\x78\x06\x47\x07\x00\x5f\x72\x30\x3a\x2b\x17\x22\x6d\x83\x98\x98\x09\x9c\xf8\x70\xea\xb3\x05\x20\x04\xf1\x42\x7b\xf5\x45\xa1\xdc\xf4\x8a\xc4\x96\xb4\xe0\x6d\x3f\x8b\x3e\x44\x0d\x37\x7c\xf8\xdc\x70\x83\xbb\x54\xa3\x81\x7e\x0d\x65\x6b\x7c\xf1\xfb\xa4\xed\x72\x1c\xfb\xb2\xea\x3c\xcf\x0f\xba\x95\x99\x9a\xee\x39\x97\xe5\xf6\xfc\xf8\x68\x72\xe3\xa9\x1e\x3d\xdf\x6e\x6f\xbd\x7b\xf7\xc3\xd6\xf6\x5f\x80\x02\x96\xe8\x68\x33\x6a\x78\xf0\x4a\x0a\xe3\xa1\xfa\x3b\x4b\x27\x2c\xb2\xdb\xcb\xcb\x25\xe9\x23\x33\x7d\x3e\x5f\xdb\x74\x6a\x09\x95\x70\x16\x4a\x30\x9a\x66\x64\xfe\xb9\x06\xb4\xba\x8d\xeb\x0a\x4e\xe9\xc1\x3a\x47\xf9\xa7\x47\xee\x84\x89\xe9\xf5\x07\xa0\x5b\x31\x47\x8b\xd0\xa6\xb9\x5d\xdc\xcc\xd0\x2b\xd1\xe8\x05\x2d\xe6\x47\x88\x22\x4c\xbc\xef\x32\x5c\x35\x9a\x00\x06\x0c\x93\x7d\xa9\xc2\x3c\xe8\x3f\x70\x38\x3c\x08\x53\x62\x98\x29\x8f\x75\x18\xa1\xca\x2b\x87\xeb\xf2\xd4\xaa\xa3\x23\xd2\x20\x15\x75\x5b\x0b\x68\x00\xab\x52\xa4\x7d\xc0\xa8\x63\xa9\xe2\xf5\x34\xf6\xab\xae\x47\xb2\xe9\xa8\x2c\x3c\xcf\x70\x7f\xc2\xfe\x8a\xa4\x97\x65\x4f\x12\x94\xd1\x17\x42\xf3\x72\x23\xa1\x2b\x53\x4b\x76\xb2\x80\x2a\xc5\x21\xa9\xb5\x84\x3a\x61\x33\x6d\x12\xaa\xd7\xe5\x83\x8e\xa4\x61\xa8\x00\x40\x6b\x8c\xad\x2a\xbd\xce\x82\xb8\x88\x96\x88\x45\xf9\x55\x25\xfb\x8a\xb2\xab\xfc\xa8\xae\x33\x15\x95\xd6\xb4\x46\x42\x27\xd6\x39\x79\xee\x21\x80\x24\x55\xf1\x22\x2d\xe9\xeb\x79\x18\xa2\xcf\xa7\xe1\xb9\xd8\x82\xec\x25\x5d\x9e\x21\xd7\x56\x14\x7d\xc6\xb8\xe9\x59\xae\x99\x38\xca\x6b\x4a\x39\x0d\xe6\xd3\x8f\xf6\x25\x75\x69\x34\x84\xf2\xa6\x77\x9b\x7b\xef\x22\x66\x16\xe8\x26\x7a\x85\x10\x6a\xf7\x5b\xe2\xd7\x01\x65\xfe\x38\x1f\xa1\x00\x25\x9f\x2d\xb2\x0d\x63\x4f\xb1\x46\xf4\x8b\xc8\xb8\xb8\x68\xd8\x5d\xf6\xa6\x81\xb6\x26\xba\x63\x65\x5b\x93\x78\x46\x13\x45\x1b\x02\xd4\x4e\xf0\x6a\x2a\xf6\xf5\x31\x1c\x7b\x58\x54\xfd\x2f\x87\x1a\xb1\x3f\x16\x5b\xb4\xc2\xfc\xae\x7f\x8d\x87\xd8\x50\x40\xde\xb1\xf7\xc7\xa4\xc3\x87\x91\x95\xcc\x67\x23\x0d\x36\x7b\xd2\x25\xa0\x2c\x9c\xc6\x07\x26\xec\x00\x9f\x37\x3c\x16\x9f\x8f\xac\x13\x10\xe8\x9f\xac\x50\xf4\x21\x3c\x5c\xdf\xa2\x78\xce\xf9\x08\x55\xd0\x32\xdf\xc5\x36\xb1\xe1\x1a\xf5\x6e\x59\xef\x6e\x24\xa9\x6d\x10\x77\xee\xa3\x8d\x21\xa1\x52\xac\xc5\xfa\xc3\x44\x13\xe6\x32\x92\x5c\x54\x12\xe6\xf3\x58\x81\x42\x51\x24\xab\x4f\x96\x14\x66\x8f\xad\x53\x86\xb7\x65\x95\x7a\xd5\x3c\xf4\x8a\xbd\x3c\x1b\xd0\xd5\x0d\x32\x93\x35\x72\x5e\xa8\x34\xb2\x01\x4f\x52\x6c\x3e\x19\x1d\x92\x71\xb9\x9a\xd4\x71\xd3\x1b\xd6\x76\xa6\x67\xbd\x6b\xd4\x97\xb6\xfa\xbd\x53\x5d\x10\xf5\x79\x67\x97\x3d\x92\x4b\x45\xb3\x0a\xe8\xe9\xb1\x66\x35\xaa\x6b\x5e\xb6\x74\x5a\xc3\x43\x65\xad\x57\x93\x46\xf0\xba\x45\x3d\x6b\xeb\x6d\xac\x59\xab\x1d\xe6\x79\x0d\x0f\x64\xe9\xfa\x3a\x9c\x67\xa6\xf6\x9c\xf3\x71\xda\x1c\x4f\x2e\xd6\x01\x8b\xf5\xd1\x45\x93\xd7\x27\xad\x1f\xe1\xe4\x57\xca\x2b\x6d\x3e\x2f\x8e\x5a\x27\x66\x0c\xad\x7e\x9e\x2c\x1d\x98\x4c\xa1\xcd\x27\x5c\x96\xa4\x28\x21\x4b\x13\x3b\xd9\xda\xfe\x2b\xaa\x9e\x46\xb6\x28\xb0\x43\x2c\xdf\x4b\x83\xc7\x23\x7a\x3c\xe8\x26\x2c\x5f\xc9\x11\x0e\x3c\xcd\xe5\x2c\x1c\xcc\x4e\xc1\x80\x55\x1c\x26\x53\x91\x24\xa5\x4c\xd4\x71\x7b\x5c\x13\xcb\xa9\x64\x04\x7f\x3e\x27\xb8\x4a\x19\x85\xc2\x51\x84\xa6\xde\x70\x26\x7a\x21\xe4\x6b\xf1\x29\x48\x78\xf0\xd5\x13\x41\x23\x93\xd6\xd8\x49\xa8\x44\x8b\x31\x5e\x0d\x80\xba\x8e\xef\x8d\x22\xa7\x3b\x32\x11\x55\x75\xe0\x3b\x60\x5e\x65\x55\x53\x8f\xa2\x26\x2c\xc5\xfc\x6c\xa6\x8d\x00\xc8\x84\x23\x53\x50\xd4\x46\xc8\xe7\x19\xc3\x24\x69\x97\xf5\x52\xb8\x1e\xdc\x9b\x65\xf5\x52\x39\xf8\x66\xe2\x48\x7f\x51\x3f\xc5\xda\x2d\x28\xec\x5c\xbc\x9e\xb5\xe6\x85\x25\xa0\x85\x25\xef\x35\x31\xec\x33\xa6\x7d\x43\x5a\xa2\x0c\x12\x1c\x48\x9d\x54\xa6\xdf\xca\xaa\xf5\x43\x2a\x4a\xa9\x0d\x4b\x97\x09\x6e\x28\x68\x5a\xdf\xa2\xcd\xd0\xea\xeb\x89\x40\x52\xcd\x5f\x50\xfc\xa2\x37\x35\x25\x83\x66\xef\x26\x7e\x51\x9b\xd9\x40\xaa\x00\x5a\x8b\x0c\x68\xef\xce\x10\xf3\x6e\xf8\x26\x6a\x0b\xdf\x7e\x84\x2f\xc8\x67\x16\x9d\xcc\x5e\x3b\xa8\x38\x73\x36\xcc\x7b\x93\x52\x8e\x20\xf6\xeb\x5e\x3a\x08\x8f\x4c\xdd\xd2\x63\x18\x1f\x07\xe0\x27\x85\x45\x6a\x05\x74\x60\xc1\x4e\x3f\x0d\xae\x09\x55\x95\x5d\x37\x00\x45\x13\x7c\xb0\xbd\xf4\x99\x96\xdf\x6a\x47\xfe\x61\x44\x34\x88\x00\x77\xe0\xce\x54\x6b\xb5\xbe\x91\x43\x7c\x34\x9e\x0d\xce\xef\x62\x87\x18\x90\x0d\x6e\x04\x6a\xbe\xba\xd8\x97\x19\xc5\xa1\x30\x69\xe1\xc9\xcb\x97\x4c\xc2\xde\x8c\xab\xd0\x71\xe7\x05\x1b\x35\x3d\xae\xca\x86\xfe\xb2\x6c\xef\x54\xbf\xeb\xf1\xb0\x8b\xc9\x84\xe5\x70\xdc\x00\xb9\x5b\x7a\x67\xea\xe3\x96\x5c\x45\xb9\xdb\x16\x1b\x27\xc7\x6a\x3f\xf7\xed\xcb\x43\xe1\x9d\x1c\xb6\xb7\x8a\x0a\x51\xb6\xcb\x41\xef\xc4\x8f\x60\x67\x6f\x20\x23\xae\x33\xef\x47\xff\xce\x3e\xc4\x23\x55\xba\xd2\x5a\xb8\x3e\x0a\x56\xe9\x3b\x54\x9d\xc8\x2b\xd7\x68\x69\xd9\xdf\x0f\xfa\x69\x94\x8f\xd6\x6e\xa6\x91\x79\xff\xe3\xee\x0f\x3b\x07\x1f\x5e\xef\x1d\xec\x6e\x1d\x1d\xa6\xf7\xaf\x76\xb6\xdf\xee\x6e\xbd\xfb\x70\xb8\xb3\x9f\x46\xcd\xc8\xfc\xe9\x60\xef\xc7\x7d\x0e\x99\xc8\xec\x6f\x1d\x1d\xed\x1c\xbc\x3f\x4c\x8f\xef\xaf\x06\x23\x68\x56\xda\x32\xf0\xf1\x7a\xd2\x3b\x83\xd5\x70\xd5\xfb\x42\x5f\x2f\xf0\x96\x67\x7f\x92\xa7\x51\x84\x5f\x87\x37\xe7\xf8\x35\xca\x2f\x28\x6e\x8d\x3e\x25\xf2\xe2\x10\xf5\xc8\x5f\x98\xa1\x7c\xe0\xfb\x78\x11\xf0\xa6\x03\xbc\xe9\x00\xff\xef\xff\xbb\x0a\x74\x8c\xd1\x16\x78\x52\x01\xbd\x6b\xb6\x7f\x3c\x38\xd8\x79\xbf\xfd\xd3\x87\xc3\x9f\x76\xd1\x9e\xe8\xc2\xbc\xda\x3a\xda\x39\x7a\xbb\xbb\xe3\xfb\x60\x77\xef\xfd\xd1\x9b\x34\xfa\x73\x6f\x74\xd3\x9b\xdc\x99\xd7\xf9\xe9\x84\x3e\x76\x7b\x93\xb3\x4b\xb3\x75\x3d\x19\x0c\xe1\xfb\xce\xfc\xf9\x66\x94\xc3\x9f\xe1\x9d\xd9\xba\xb9\x80\xfd\xd3\x1c\xc2\xfe\x91\xa3\xe3\x21\xb3\x77\x36\x1b\xe3\xef\xfb\xf1\x2d\x47\xbc\xca\xcf\xe8\x23\xb2\x7a\x17\x06\x26\xc7\xe1\x9b\xbd\x83\x23\x5f\x1b\xd6\x84\x95\x60\x15\xb6\x02\x84\x8f\xe0\x11\x36\x42\x45\x88\x08\x2c\x80\xf3\x6a\xeb\xa7\x34\x3a\x84\x9d\x18\xca\xec\x8e\xe9\xe7\x08\x4e\x5b\xf8\xfb\xb7\xbc\x3f\xe2\xaf\xa3\xcb\x9b\x09\x7d\xbc\x9e\x0c\xf0\xe7\x10\xce\xad\x13\xf8\x28\x63\x64\xc1\x21\x2c\x04\x84\x40\xb0\x38\x96\xc4\x62\x41\x89\xad\xdd\xfd\x5d\x98\x0f\xd1\xd6\x2e\x04\xf7\x77\xa3\xae\xb9\xca\xfb\x83\x9b\xab\x34\xda\xdd\xdd\xad\xf5\x4d\xed\xae\x76\x99\x5e\x5d\xa5\xd3\x69\xad\x07\x39\xa6\x78\x79\x0e\x3c\xc2\xee\x7a\x7f\xfd\x8e\x93\x30\xfe\x1c\x88\xde\x2b\xdc\xa5\xa3\x1d\xf8\xcf\xd4\x76\x6d\xe1\xc8\x0c\xc7\xa3\x0b\x4e\x52\x91\x5c\x87\x8b\x96\x58\x02\x2e\x91\x04\xdf\x66\x3c\x1a\xa0\xf4\xb7\xc2\x83\x72\xfa\x58\x88\x5a\x98\xeb\xe1\xcd\xa4\x37\xdc\xee\x29\x6d\xff\xd1\xcd\x95\x23\x14\x2d\xd4\x58\xbe\xb9\xea\x44\xe3\x11\x6a\x13\x8c\x81\x62\x4f\xe0\x30\xaa\x0e\xb4\xfc\x50\xb6\xdf\x9b\x5d\x02\x7f\x03\xcc\xb5\x3f\x6f\xe5\x64\x08\x02\x59\x86\xd9\xa5\xed\xbc\x75\x54\xf4\xcb\x6c\x92\x53\xdc\x5f\x5b\x6b\x27\x36\x12\xf9\x22\xf7\xfa\x76\xc8\x91\xb1\x4a\x74\xcf\xa7\x0e\x0a\xb3\x4c\x00\xba\xa0\x35\xb4\x75\x0a\xd4\xfd\x86\xed\xa2\xf6\xfc\xb7\xbb\x42\xdf\x3b\xfd\xe8\x8c\x63\x4d\x73\x7c\xed\xd4\x4a\xd6\xaa\x04\x2b\x69\x49\x11\x94\xd5\xb0\x57\xa0\xae\xa0\xbb\xea\x34\x61\xce\x4b\x74\x29\xed\x73\x61\x30\xb0\xdd\x2d\xf0\x60\x5c\xd0\x3a\x4e\xac\xe0\x41\x14\x1c\x54\x5e\xed\xbc\xde\xfa\xf1\xdd\xd1\x87\x7d\x98\xa1\x87\xc7\xe5\xea\x44\xaf\xa6\xd8\xf0\xeb\x6b\x6c\xf3\x24\x47\xb6\xfc\x76\x49\x9b\x27\xb0\x5d\x7f\x01\x8e\x00\x3a\x6e\x25\xcb\x54\x5e\x6f\x68\xb8\x6d\x33\x91\xb1\x7f\x97\x01\x8b\x34\x54\x38\x51\x46\xc2\x55\x07\x06\x39\x0a\x0d\x86\x29\x01\xdb\x1a\x0d\xf2\xc1\x5b\xb4\x8c\x08\xf3\x0b\xdb\xef\xea\x83\x2a\x32\xf1\x10\xdb\xc4\xdc\xa4\xe7\xea\x10\xeb\x14\x12\xfc\x49\xb0\x95\xa4\x61\x5a\x52\xe8\xeb\x69\x8e\x04\x8d\xc7\xe4\x2f\xf9\x1d\x89\x9e\xb2\x42\x66\x93\x93\x8a\x05\xe0\x90\x53\x89\x2b\x97\xc1\xd4\x62\x09\xac\x9a\x5a\xb0\x92\x95\x13\x02\xeb\xd7\x15\xbd\x82\x7d\x5b\x8e\x56\x53\xfb\x34\xbf\x18\x8c\xe8\x0d\x2e\xa6\x4f\xf3\xf9\x72\x3c\xf4\x5b\xfb\x46\x96\x51\x84\xbb\x3f\xa0\x4c\x49\x87\x23\x45\x39\x8d\xe2\xec\x71\xb8\x64\x81\x0c\xbb\xf2\xfa\x0d\x34\x8c\x6e\xf1\x45\xe3\x15\x0f\xd6\xc1\x9b\xf7\x6f\x22\xbb\x12\xf1\x39\x9c\xa2\x3b\x22\xda\xa5\xf4\xe3\xe8\x60\x5e\x00\x4d\x9a\x37\x04\xda\x4b\x11\xf8\x22\x61\xed\x4d\x74\x67\xfa\xd6\xd6\x09\x6b\xbc\xd1\x0a\x24\xc5\x50\x96\xf4\x07\xbc\x20\xaf\x86\x87\x53\x62\xc3\x04\x78\xaf\x47\x85\x30\x02\xdd\xd4\xaa\xd2\xef\xa4\xfb\xdf\xcc\xae\x86\xbf\x23\xe2\x71\x7d\x8d\x75\x98\x53\xf8\xb3\x4f\x53\xd4\xd9\x94\x41\x2b\xb2\xbf\x43\x83\x54\x3e\x2d\xf3\x9f\xce\xf2\x87\x40\x78\x3f\xc6\xb6\x67\xbe\x17\x24\x3e\x69\x97\x09\x96\xd4\x49\x97\x6c\x72\xb3\xc6\x57\x32\x81\x18\x9e\x25\x60\xb3\x4b\x5c\x98\x6a\x6e\x04\x35\x62\x8b\xe9\x3e\x64\xc5\x3d\x98\x4b\x11\x67\xa7\xcc\xce\x39\x77\xa8\xbc\x9e\x5d\x5e\x4f\xce\xbf\x44\xe6\x99\xf5\xc1\x0a\x40\x6a\x11\x99\x66\xab\x01\xa7\x36\xc5\xfb\x0a\x84\x22\xa4\x04\x92\xe0\x98\xd9\x7c\x46\x37\xe1\x41\xed\xb6\x75\x4c\x95\xa4\xe2\x62\xc3\xd0\x54\x72\xac\x42\x19\xed\x15\x12\x81\x97\xd2\xe3\x29\x3e\x2c\x16\x62\xb2\x82\xa3\x5e\x59\xdd\xb3\xb1\x5b\xda\x92\xdf\xae\x6d\x5a\xcd\x76\xec\xe0\xbb\x03\xf3\xb8\x51\xda\x6e\x54\x86\x24\x75\xf6\x58\x56\xf1\xbd\x46\xed\x77\x0a\xdb\xa4\x11\x73\x0d\x28\x06\xd2\xe0\x4f\xb2\x04\x8f\x20\x2c\xce\xb0\xa3\xe0\x20\x05\x7d\xd3\xf0\xa0\xbd\x87\x7d\xd7\xca\x49\x4e\xd6\x31\x2a\x46\xbc\x47\x9d\x49\x96\x2c\xb9\x5f\x65\x2d\xc6\x9c\x50\x31\x13\x68\x0e\x24\x2b\xfa\xf2\x22\xf6\xc5\x33\x01\x58\x51\xde\x4f\x67\xc3\x89\x05\x28\x61\x7b\x62\x55\x12\xd7\x9a\x14\x99\xcf\xe5\x23\x95\xec\x0d\x5f\x77\x92\x2e\xc7\x5a\xcd\xdf\x87\x6a\xe5\xf2\x69\xb8\xcc\x90\x5e\x35\x00\x87\x30\xaf\x25\x77\x15\xeb\x1d\xc6\xea\xb4\x37\xba\xd0\xcb\x0f\xc7\xcf\x2e\xf9\xff\x03\x0b\x19\x5f\x3e\xc6\x37\x33\x2c\xf0\xc0\x28\xce\xe7\x0f\x76\x94\x11\x28\xd8\x1e\xda\xae\x7f\x03\x7b\x69\x08\x59\x6d\xa5\x0a\x94\x6f\xac\x09\xb3\x3b\x83\x2a\x44\xee\x3a\x61\x22\x2e\x90\x80\xb4\x84\xd5\x2f\xa7\x30\x97\xd3\x27\x50\x18\xc4\xa9\x8a\xc2\xa8\x81\x09\xc8\x4b\x58\x79\xd8\xeb\xff\x7f\x24\x23\x8d\xd8\x43\xed\xf8\x4e\x51\x74\x05\x01\x3c\x42\x51\xdc\x7d\x9a\xdd\x7a\xed\x74\xce\xb2\x70\x3b\xa6\x1d\xfe\xd1\x05\xf4\xb6\xbc\x75\xea\x75\x54\xdc\x3a\x2b\xd6\x9f\x18\x06\x20\xcb\x37\xde\x30\xc0\xd3\xb6\xd2\x27\xd2\x4e\x7b\x8c\x90\x72\x95\x2d\x7d\x1a\x5d\xb2\xf4\xa5\xa1\xba\xbf\x9a\x20\x3d\x95\x22\xd9\xa5\x22\xd2\xd4\xd7\x62\xc7\xa3\xca\x98\x82\x35\x55\x83\xf6\x96\x6c\xbe\xee\x52\x50\x87\x21\x40\xdc\x3f\xe0\x1b\xe5\xc1\xcb\xb0\xc3\x37\x40\x2d\xac\xca\x29\x9d\xb0\xce\x34\x0e\xc3\x99\x87\x2d\x05\x4a\xcb\x90\x57\xe7\xa2\x70\xa3\x23\x32\x25\xfa\x69\xd4\x77\x6c\x16\x45\x86\x26\xce\x2e\x1a\x4f\x5f\x69\xf1\x68\xab\x74\x75\xe7\xc5\xb3\xcd\xe1\x6f\xef\x27\x25\xa1\xa3\xc8\x5d\xc6\x71\xf2\xa2\xe9\xe3\x65\xd9\xf8\x0a\xbd\x74\x0f\x19\xf3\x2c\xc1\xa6\x68\x80\xec\x0a\x5c\x91\xa9\x7f\x06\x6b\x23\x0b\x0f\xa5\xc1\x5d\xb2\x92\x7b\x9b\x8e\xc8\x6a\x9b\x55\x8c\x16\x69\xad\x25\x37\xcc\x4e\x0c\x4e\x4a\x05\x86\xdb\xd5\xad\x33\x59\x1a\xb7\xbd\xbc\x4d\x32\x04\xf1\x98\x24\x7d\xc5\x76\xa7\xdc\x51\x02\xbc\x5e\xff\x0c\x78\xef\x38\x72\x84\x9b\xb3\x8b\xfe\x40\x64\x5c\x7c\x93\x29\x12\x8c\xa7\x40\x62\xcb\x5a\xce\x3f\x95\xad\x6e\x97\x8c\x31\x2b\x5e\x1a\x4d\x90\xb9\xa7\xf7\xa6\x8d\x89\xf1\x46\x74\x80\x2f\xff\xb8\xf4\x42\xd1\x98\xb6\xeb\xc4\x8e\x5d\x89\x99\x3a\x04\xf8\x72\x40\x49\x2d\x3c\x60\xc2\x91\x95\xd4\x58\x64\xb6\x9b\x9a\x30\x10\x28\xe5\xd9\x29\xf2\xfc\xe9\x72\x9a\xc6\xec\x0a\x57\xed\x48\x86\xaa\x39\xac\xa9\x82\xb8\x25\xbe\xeb\x48\x9c\x44\xe7\x77\x04\x13\x37\x14\x45\x34\x55\x6f\x0b\xf7\x10\xeb\x18\x21\x76\x1a\x8d\x24\x98\x02\x4d\x98\x2f\x11\x59\xfd\x55\x13\x88\x4c\x0c\xb3\x58\x37\x7d\x36\xcf\x66\x93\x21\x99\xea\x93\xf0\x55\x3e\xeb\x51\x78\x73\x25\xe3\x98\xcf\x97\x83\x33\x75\xc9\x93\x0f\xaf\xac\x03\x15\x4e\x9f\x01\xb5\xce\x67\x49\x3b\xea\xe1\xad\x82\x97\x5f\x82\x8c\xd6\x80\x37\xab\x14\xa3\x8a\x00\x47\x02\x59\x0c\x2c\xb5\x6e\x74\xe7\xf3\x15\x4c\xcb\xe0\x9f\x33\x56\x9e\xa0\xc7\x09\x5e\x6f\xbc\x0d\x9c\x4e\x69\xfa\x50\x1e\x34\x24\xc8\x36\xef\x50\x50\x04\xbb\x62\x96\x8f\x68\xf6\x54\xf4\x91\x94\x4c\xda\xf2\x81\xad\x05\x20\x6c\x85\x91\xf1\x8f\x12\xf4\x0b\xeb\xe1\xb8\x0e\xa1\x95\x8e\x62\xad\xfb\x13\x8a\x80\x25\x8f\xe7\x78\x4e\xbc\xe6\x38\xc9\x11\x87\xa8\xac\x14\x66\x32\xfa\xd7\x28\x0d\xa9\x2e\x10\xea\x07\xc9\x9b\x81\x09\xa5\x53\x8f\xa3\xf3\xf3\xb5\xdf\xff\xe1\xbb\xcd\x8d\x3f\xac\x85\xb5\x47\x5d\x7c\x42\x47\x69\xe0\x8a\x75\xba\x92\xf9\x79\xe2\x25\xf2\x08\xaf\x8a\x45\x8d\xd2\x37\x2e\xcf\x18\x51\x13\xba\xe1\x6f\x0c\xf3\xcf\x44\x40\x50\xf4\xad\xa2\x32\x9d\xfc\x34\x12\x83\xef\x7a\xe4\xee\x1b\x59\xb8\x32\xc8\xc4\x4a\x19\xbb\x51\xb0\xfa\x08\x8f\xb4\x25\x31\xbc\xd5\xb6\x1f\xb5\x65\x87\x53\x8c\xa9\x59\x56\x06\xd3\x2e\x8f\x9c\xb4\xd1\x3c\x40\x5f\x17\x8f\x6b\x7c\xf5\x07\x17\x24\xdb\x09\xc3\xc6\x06\x43\x09\xc0\x36\x0b\xf5\x65\x1b\xee\x89\x54\x15\x61\x6b\xd6\x4b\x30\x0f\xc5\x0a\xc5\xe8\xa7\x38\xc0\x2f\xac\x0d\x8a\x73\x6f\xd1\xba\xde\x7a\x5d\xc0\x55\xf4\x04\x9c\xd9\x83\xbc\x8d\xc6\xe3\x66\x02\xbf\x6a\x0e\x2c\xdf\x64\x4a\x73\xa0\x53\x1e\x15\xc9\x9a\xc6\x8f\xcd\xf1\xb0\x6b\x1e\x1c\x46\x54\x04\x31\x15\x7d\x47\x02\x41\xba\x2f\xf4\xd2\x2b\x3c\x5b\x5d\x14\xf8\x9b\x7e\x7e\x7a\x73\x81\xac\xf0\x34\x1f\x9e\xd3\xb9\xa2\x2d\xe6\x78\x20\x7e\x67\xd4\x3b\x1d\xe6\x7d\xa5\xf0\x3a\xec\x5d\x54\xf0\x1e\x14\xdd\x89\x19\x16\x06\x84\xef\xa0\x88\x02\xcf\x51\x36\x86\x67\xad\x82\x39\x34\x51\x7b\xbb\xc7\xfa\xcd\xa8\x1d\xea\x2a\x84\x6f\x67\x89\x6c\x7c\x5e\xa3\x0c\x68\xe7\x7d\x72\x81\xaf\xf5\x67\x9f\x3a\xf0\x95\x61\xe8\x0a\xb8\x84\xde\x45\x8e\xc2\x4d\x40\xe2\x5d\xba\xbb\x79\x53\x79\x92\x0e\x0b\xcd\xa0\x97\x71\x15\xdd\x40\x43\x5d\x0d\x57\x32\x2d\x7c\xb1\xfb\xb1\x83\x77\x5c\x7b\x56\x5d\xd0\x66\x6a\x44\x29\xc7\x90\xb5\xaf\x04\x4f\x15\x5a\x1e\x7a\x04\x07\xf6\x1c\x06\x26\x9e\x91\xd4\xb5\x18\x8d\xc0\x48\xff\xa2\xce\x61\x52\x68\x1d\x8e\x2f\xc8\xc9\x11\xc5\x1c\x63\x19\xd8\xb4\x24\xd8\x84\xc4\xf9\x1c\xe5\x82\x9d\x31\x4d\xcc\xcd\x47\x9a\x4e\x61\xa5\x02\x1a\x55\x4a\x3e\x15\xea\xb8\x34\x06\x98\x5d\x24\xb4\x0a\xc3\x83\xe4\x45\xd5\x13\x0b\x32\xd8\x4e\x60\xb1\x53\x0d\xa6\x85\x91\x9b\x68\x05\x04\x8d\xf4\x51\x04\x8b\xb8\x60\x74\x27\x8a\x52\x4a\x5e\x88\x44\x3b\x66\x4b\x55\x07\x45\x68\xe4\x0d\x59\xb4\xf3\x71\x10\x8d\x11\x10\xff\xb9\x37\x19\x05\xf1\x18\x01\xf1\x24\x61\x1f\x24\x50\x0c\x2a\x1d\xe0\xfc\x4c\x0b\xfd\x72\xee\x7a\x97\xf2\x52\x1e\x77\xe1\x5c\x0b\x84\x00\x20\xa5\x5e\x3f\xb7\xed\xc6\x15\x14\xd8\x92\x8b\x43\x7b\x72\xf9\x68\x7a\x33\xc9\x0f\x7b\xe7\xf9\x2e\xbd\x4d\x92\xf1\x62\x7a\x93\xc1\xe7\x38\x2f\xdd\xc4\x5e\x31\x94\x15\x3f\xeb\x18\xc3\x5e\x67\x10\x85\x71\x77\x19\xd3\xfc\xec\x7c\xd8\x8f\xcc\xb3\x83\x9c\x2d\x2d\xe3\xdd\x45\x50\xbc\x76\x3e\xc8\x87\xf4\xb8\xbe\xc5\x9b\x77\x20\xb9\x35\x98\x2a\xdf\x0e\x2b\x35\x8f\x07\x89\xf0\x3c\x2b\x22\x67\x3b\x02\x11\xaa\x6a\x9b\xa8\xbe\x8d\x4f\x3f\x56\x35\x0b\xa2\x81\xa8\x9f\x7e\x6c\x6a\x33\x93\x59\x06\x31\xcb\x1b\x37\x82\x73\x88\x6e\xdb\x6b\x2f\xc7\xf2\x0d\x0d\x8a\x4a\x0d\xd2\x68\x59\x45\x1b\x0e\x59\x1a\xca\xa1\xde\x30\x9f\x48\x82\x12\x0b\x59\x8a\xb7\x93\xcd\xd1\xb8\xa3\xb7\x8c\xbf\x51\xc2\xbf\x1b\x7b\xfc\xeb\x98\xdc\xf9\x1c\x43\x16\x71\x20\xd2\xfd\x64\x29\x9e\xfd\xf1\x55\x01\x49\xf4\x01\x82\x90\xa6\xff\x2e\x1c\x65\xca\x00\x2e\xfa\x01\x85\xee\x03\x70\x9e\xe0\xfd\x13\xec\x3f\xb3\xbf\x8a\x26\x25\x15\x36\x63\x12\x0e\x9b\x26\xf7\xf2\x91\xc9\x2f\x52\x41\xef\x56\x20\xbf\xb3\x36\xfa\x83\xd7\xde\x26\xbd\xf6\x6e\xb4\xf3\xc0\xe7\xc2\xcb\x16\x5b\x6a\x47\xe3\x8a\x95\xab\xd1\x66\xb7\x46\x2c\x04\x17\x66\x89\xec\xfd\x02\xda\x22\x04\xb4\xc9\x64\x62\x5b\x45\xa2\x70\xa7\xca\x82\xce\x3a\x25\x57\xa6\xe2\x13\x8c\xd5\x11\x18\x16\x49\x6e\x69\x61\xf3\x66\xf4\x79\xd2\xbb\x16\x15\x5f\x14\x4d\x17\x09\x95\xbf\x01\x41\xc3\x6b\x4e\x8b\x16\x9c\xcf\x57\x6f\x81\xfc\x61\xcf\xc2\xc9\xa5\x2c\x74\x53\x6d\x1c\x03\x67\xed\xbd\x17\x7a\xb9\x45\xa1\xea\x45\xb2\xc0\xb1\x20\xec\x28\x2e\x0b\x94\x61\x6d\x2c\x5a\x76\x1e\x73\xfb\x31\xec\x2c\x75\x7c\x75\x8f\xfa\xbe\x71\x03\x6f\x3f\xd4\xe6\x38\xbd\x46\x70\x7c\x81\xf4\x9a\x0c\x54\x6f\xa0\x1e\x43\x0b\xff\x6c\xe2\x9f\x17\xf8\xe7\xbb\xf2\x9c\x71\x02\x77\x15\x28\x11\x14\x87\xc7\xb2\x2c\xad\xc7\xb3\x6c\x3e\x9e\xe5\xc5\xe3\x59\xbe\x53\x7d\x52\x39\xfe\x7e\xe3\x16\x0f\x79\xca\x1e\xb0\x15\x9c\xc3\xb9\x8f\xa6\xda\x39\xad\x5e\x17\xd3\xae\x65\x4f\xbd\x1b\x49\x87\xd3\x52\x02\xd6\x56\x0e\xb7\xb3\x2c\x13\x30\x28\x6c\xca\xf0\x42\x81\x53\x8a\x4b\x63\x9b\x28\xbf\x38\x8c\x1b\x5d\x8b\x02\x0a\x2e\xd3\x87\xcc\xe9\x47\xe7\xae\xab\x33\x76\xb6\x61\x38\xc6\xe8\x29\xea\xd0\x30\x5f\x33\xa7\x13\x53\xc0\x95\x26\x2d\x4d\xa1\x7a\x9d\x15\xc0\xb3\x22\xde\x85\x77\xaa\x72\x5b\x5b\xff\xd5\xda\xba\xf9\xed\x6d\xdd\xfc\xaf\xd6\xd6\x17\xdf\xde\xd6\x17\xff\xd5\xda\xfa\xdd\xb7\xb7\xf5\xbb\xff\x52\x6d\x95\x00\x5a\x24\x7e\xf4\x43\x1d\x38\x2a\xc8\xe5\x7f\x3e\x99\xfc\xd5\xb4\xe7\x57\x2f\xe8\x5f\xbd\x4a\x9e\x3e\xf5\x2a\x67\xde\x93\x46\x31\x70\xad\xc8\xbb\x3f\xb1\x89\xb2\x45\xba\x3d\x13\x0f\x15\x36\x07\xd9\xb8\x28\x0e\x26\x3d\x6e\xda\x4b\xba\x20\xe7\x31\x26\x75\xd9\x71\xd6\x88\xe6\xd8\x5f\xf2\xbb\x69\x89\x8d\xb4\x09\xef\xac\x3e\x0f\x07\x9d\x04\x21\xf0\xdf\xb2\x6d\x03\xbb\x92\xc0\xe9\xf1\xf7\x2f\xc3\x22\x9d\x22\x1b\x63\x93\x8f\x65\xd7\xa4\xef\x96\xfa\xde\x54\xdf\x2f\xd4\xf7\x77\xdd\x12\xa7\xf3\xd0\x6c\xbf\x45\x13\xe5\xc0\x04\xf7\xc7\xf8\x99\x2d\x45\x03\x9d\xa5\x99\x6f\x0d\x15\x11\x0a\xf1\x90\x5f\x45\x0e\xd8\x99\x32\xe0\xd3\x26\x35\xa5\x38\xec\xab\x97\x03\x77\x5a\x40\x82\xd0\x76\x0a\xb0\xf8\x66\x9d\x45\xf8\x35\x34\x38\x5c\xb5\xeb\xf6\xc9\x28\x72\x26\x62\x2d\x14\x7f\x69\x81\x27\x03\xb1\xdc\xb1\x8c\xf3\xf2\x8c\x17\x42\x6f\x64\x11\x8c\xe5\xb4\x06\x8b\x99\xd6\x79\x6d\x3e\xaf\x71\xc8\x21\x9f\xd4\xec\x73\x35\x54\x3e\xcc\xf0\xef\x34\x8b\x1a\xac\x04\xd6\x89\xa6\x51\xfa\x2c\x8e\x3f\xd5\xeb\x9f\x8a\x93\x30\x7a\xd6\x80\xfa\x1a\xcf\xa2\x24\xe9\x7c\x4a\xa7\xc9\xb3\xa4\xf1\xec\xd8\x45\x76\x01\xce\xb3\x46\xbc\x84\xfb\x7b\x36\x38\xaf\x01\x5a\xf5\x7a\x6d\xca\xa6\x89\x6a\xf7\x27\x40\x85\x3f\x23\x50\x69\x80\xf2\x12\x12\xad\x5f\x98\x67\x27\x27\x11\xd6\x10\x25\x00\xb8\x86\xc5\x57\x62\xa2\xde\x78\x9e\x9b\x26\x02\x80\xd0\xaf\x5d\x23\x85\xad\xa9\x36\x72\x64\x81\x58\x43\x91\x6b\xa6\xd2\xed\x05\x02\x5d\x40\xa6\x69\x86\x2f\xaa\xb7\x14\x78\x46\x2f\xfd\xae\x1f\x5d\x2f\xb1\x08\x1a\xde\xc5\xe6\xfd\xd7\xf2\x10\x9c\x39\x53\xbd\x11\x3a\xa2\xfc\x84\x56\x3c\x3e\x47\x54\x34\x69\x87\x59\x9d\xc2\x50\x85\x2e\xcd\x19\x3d\x67\x6a\x05\xaf\x70\x01\xd8\x13\x40\x00\x30\xc8\x63\xc2\x9d\xce\x5d\x3b\x45\xe1\xe0\x45\x42\xe4\x60\x6b\xac\x20\x1f\x19\xaa\x8e\x9c\x8f\xd4\x1d\xeb\x3e\x1e\xb0\x8b\x8e\x4e\xc8\xfa\x0e\x9c\x08\xf9\xf8\xbd\x27\x27\xda\x7b\x58\x92\xe9\x4a\xcb\x84\x23\x8e\x31\x43\xba\xa8\x55\xd8\x41\xec\xc6\x82\xaf\x66\xc3\xdc\xcb\x34\x67\x8b\x9a\xde\x9d\x38\xa8\xbb\x08\x65\x65\xe5\x56\x69\xd6\x3e\x94\x55\xee\x73\xcb\x18\x7e\x2b\x26\x15\x90\x96\xe3\x52\xce\x5c\xb8\x5f\x3e\x1f\x0c\x67\xa5\x57\xec\x82\x77\x08\xce\xa3\x5e\xac\xb5\xbb\x88\xb0\x3e\x18\x21\xff\x64\x0b\x81\xc2\xa4\x51\x57\xe3\x76\x43\x7a\x0c\xdf\x7a\x7d\x25\x84\x51\xb9\x6d\x59\x70\x25\x86\x8c\xe7\x9e\x24\xe3\x53\x1b\x61\xdf\xc4\xbb\xce\x38\x3a\xe6\xca\xbb\x35\xa9\xb3\x76\x3e\x86\x85\x8d\xcb\x1e\x2f\x9e\xfc\x15\x4e\xed\x9f\x91\xa5\x1d\x8d\xe8\x9f\xcd\xda\xd6\xcd\x6c\x7c\xd5\x9b\x0d\xce\x6a\x3c\xd6\xa4\x11\x8a\xaa\x84\x32\xe6\x0f\x5d\x04\xe5\x10\x44\xdd\x50\xf4\xe5\x9a\x2c\xd4\x13\x2f\x76\x86\x97\x71\xf7\xf7\x41\xed\xe9\xe7\x01\xbe\x1c\x89\xba\x22\xe5\xc3\x97\x5a\xab\xc8\x98\x5a\x73\x6c\xc5\x3e\xc1\x9c\x81\x61\xb6\x63\x88\xe1\x0d\x7c\x98\x7f\xc9\xd9\x42\xc2\x3b\xfc\x0a\x67\x98\x78\xb4\xe0\x74\x5a\x9e\x93\x98\x0a\x18\x37\x13\x82\xec\x4e\x59\xac\x80\x38\x8b\x6e\x4f\x9a\x4e\x3d\xd9\xac\xb4\x80\x2f\x2e\x93\x0b\x48\xb2\xf6\xee\x08\xc5\xac\x08\xc9\xfa\xd8\x50\x9d\x42\x3d\xe0\x34\x4e\x53\x4b\xbe\xbe\x5c\x5b\x5b\xdb\x36\x0a\xaf\xf7\x17\xa1\x23\x96\xff\x78\x92\xb2\x60\x59\x1b\xbe\x52\x9e\xa3\x98\xcd\x2d\x8c\x5f\xac\xdd\x30\xaf\x52\x2b\x16\x1a\xc2\x97\x35\xff\xf4\xe6\xd2\x17\x15\x60\x75\x0b\x1c\xe8\x51\xfe\x65\x76\x34\x38\xfb\x64\xca\x58\x78\xc3\x9b\xd6\xa2\x99\x18\x53\xb9\x2d\x5b\x0c\x58\x14\x33\xef\x4c\x26\x94\x77\x92\xf7\xa6\xe3\x91\xb2\x68\x25\x36\xac\x28\x56\x79\xc2\x1e\x3a\xe5\xb1\xa9\x7d\x27\x13\x35\x46\xab\xc3\x78\x66\x9f\x46\xd1\x3e\x04\x99\x9d\xb2\x06\xe3\x5c\xc1\xce\x71\x37\xf5\x2a\x72\x8e\x55\x91\x64\x53\xbc\xae\x63\x0b\x1d\xce\x8e\x8a\x41\x81\x14\x7b\x93\x57\x3e\x3d\x71\x93\xa7\x6c\x37\xbe\x74\x7e\xc1\xe7\x51\x49\xd5\x2e\x0c\xd7\xd6\x04\x3c\x19\x72\x0c\x35\x11\x25\x7b\xb0\x86\x7d\x77\x3d\x50\x51\xc1\x6a\xa7\xeb\x4d\x64\x08\x36\xb2\x2c\x3b\xb3\x8f\xb9\x4b\xab\x34\x45\x45\xd0\x85\xeb\xf2\xa2\x7c\x24\xb7\xc3\xeb\x25\xb2\x9d\x31\xf5\x8e\xe5\x06\xea\xde\x5a\x15\x0d\xcf\x9c\xe8\xee\x9f\x0b\x85\x16\x6d\x80\xf1\xe7\xe8\xb6\x85\xe9\x79\x56\xaa\x34\xc3\xf1\x40\x10\xde\xea\x4b\xd3\xda\x07\xb0\xd3\x56\x3f\x36\xdb\x2b\x6a\x67\xc6\x44\xd4\xcc\x8b\xa5\xbd\xc2\xb9\x4d\xf1\x59\xd0\x7b\x28\x55\xce\xe3\x6f\xe3\xc9\x38\xa4\xfd\x6e\xa9\xef\x4d\xf2\x48\xb0\x10\x4b\xca\x69\x69\x1c\x2b\x46\x40\x0f\x19\x0c\x3d\x2b\xb4\xa6\x7a\x6e\x5e\x4c\x48\x48\xef\xc9\x1d\xf7\x7f\xa4\x57\x54\x93\x3d\x8e\xd4\x76\x99\x42\xe9\x3d\x76\x59\x5a\xa2\x55\xf8\x1c\x48\xbf\xb6\x14\x13\x30\x31\x56\x03\xd3\xd1\xad\x70\xda\x08\xf3\xbe\xa5\x33\x25\xf6\x06\xed\x08\x70\x11\xd7\x9d\xb1\x72\x44\xe1\x88\x5f\xc7\x7e\xa5\x05\xca\x95\x08\x24\xe5\xad\xd1\xc1\xa3\x61\xc9\x93\x12\x1d\x84\xb8\xff\x8e\xd6\x6b\x18\x37\x21\x6b\x59\x69\xa0\x1f\xc1\x4d\xfa\x20\xe9\xc8\x47\x1a\x92\xc9\xc4\x4f\x88\xaf\xc2\xcc\x23\xb6\xaf\x3a\x37\xab\x98\x4c\x0a\x3f\x51\xa1\xd6\xe8\x05\x43\xd3\xd1\xa1\x72\x17\x3a\x98\x0a\xd7\x4a\xd4\xdc\xae\xce\xb3\xb4\x63\x67\x2b\xbd\x69\x1f\x17\x06\xbb\xd0\xc1\x55\xcd\xea\x26\xa9\x5a\x9a\x5f\x5f\x3e\x91\x8d\xc3\xd1\x3c\x13\x51\x03\xa2\xf2\x9c\x0d\x64\x6d\x95\xee\xba\xda\x5f\x23\xa0\x54\x10\xba\xab\x2c\xed\xf6\xb5\xab\xde\x27\x67\x39\x96\x29\xa9\xcc\x8e\x7e\xd5\x0a\x68\x7b\xbb\x8f\x94\xa7\x53\x98\x51\x3c\x7b\xd3\x70\x66\x70\x64\xb1\x6d\x0e\x83\x4b\x1a\x91\x70\xef\x36\x83\xe9\x41\x80\x86\xc5\x7c\xef\x66\x76\x7d\x33\x23\x53\x6c\x6d\x9c\x31\x85\x78\xd7\x42\x67\x96\xd8\x4f\x8b\x60\xe6\x52\x2b\x74\xdb\x73\xe4\xdc\x16\x8e\x95\xd4\x50\xeb\xf5\x8a\x35\xcc\x49\x7c\xfc\xee\x54\x44\xc6\xe5\xf3\x69\xb9\xab\x55\x2b\x35\x9b\x4c\x86\xf5\x2b\x91\xc4\x14\x42\x14\x35\x0b\x1f\x82\x56\x9a\x1d\x4b\x4e\x62\x95\x9d\xbf\xb2\xb1\x1c\x9d\x42\x01\x85\x11\x6a\xe2\x22\x87\xf2\xa0\xe1\xa6\xa0\x2f\xfd\x62\x49\x3a\xf4\xbd\x8c\x42\x3f\x34\x15\xab\xb6\x94\xc2\xac\x3c\x0b\xf9\x42\xe4\x42\x0a\x93\xd1\x6d\x90\x65\xba\x29\x82\x27\x0f\x6f\x1d\x5f\x8b\xe0\x7f\x16\x31\xae\x6a\x38\xda\xe9\x29\x8c\x99\x79\xc2\xd6\x28\x06\x9f\xbf\x61\x7f\x24\xd5\xb0\x6f\xda\x16\x4b\x2b\xb8\xa2\x91\xc6\x75\xc1\x93\x37\xc5\x22\x42\x4f\xed\xfe\x6f\x45\xe7\x69\x5b\x61\x11\xab\x6f\xda\x02\x9f\xb8\x03\x56\x2f\xa2\x73\xeb\xdc\xbb\x92\x82\xe0\x04\x80\x33\x05\xfe\xe0\x9d\x43\x61\x3a\x17\x4b\x7f\xc3\x6e\x58\x79\xee\x78\xb8\xd6\x10\xaa\x5b\x26\x1a\x90\xef\x66\x06\x15\xf2\x1c\x15\x78\x68\x7e\xa2\x82\x74\x48\x17\x32\x23\xcd\xfe\x0a\x2c\xb3\xcd\x3f\xb4\xc0\x52\xfc\x63\xa0\xf1\x29\xfc\xd3\x4a\x26\x07\xf6\x94\x5c\xb8\x1f\x3c\x3a\x7a\x97\xb5\x36\x94\x00\x2b\x0b\xe8\x64\xce\x77\xac\x3f\xc8\x8b\x96\x11\xcb\xeb\x1e\xcd\x96\x5a\xce\x73\x82\x68\x8e\x1d\x8f\xb1\x16\xd9\x9e\xe1\xf3\x01\x87\xc5\x55\x76\xf3\x9c\x51\x3d\xa7\x27\xe2\x6f\x0f\xbc\x3b\xe3\xb2\x1d\x3d\x31\xb0\x27\xa5\x14\x2f\x42\xcd\x71\x17\x16\x83\x7e\x86\xd3\xf2\xc7\x41\x3f\xf6\xaa\x8a\x28\xa6\x2c\xba\x6e\x2c\x8c\x6f\x15\xdf\x48\xf4\x58\x39\x4e\xc7\xa2\x87\x83\xd3\x21\x1e\xdb\x6c\xe1\x49\x7e\x5b\x88\x3a\x43\x0b\xba\x68\x02\x32\x88\x38\xea\x0d\x86\x6c\xed\x95\xb4\x88\x22\xfc\x1b\x75\x25\x0b\xf6\x3b\x7b\x41\x97\x22\x7d\xe8\xf5\xc9\xf8\x0e\xce\x97\x2b\x2d\xa7\x16\x87\x97\x1d\xff\x71\x93\xa3\x75\xf8\xae\x43\x7e\x3c\x9d\xbd\xa2\x41\x2a\xa6\x78\xcf\xe8\xf7\x4e\x39\x6e\x30\x25\x4b\x50\x3f\x0c\x88\x27\xc5\x94\x82\xda\xfd\x3e\xf6\x45\x4c\x3d\x52\x12\x67\xe7\x8e\xb2\x02\x5e\x85\x39\x84\xe2\x91\x38\xa7\x61\xe8\xee\x37\x16\xb5\xde\x90\x4c\x74\xe2\x85\x9e\x9d\xe9\x51\x85\x70\x78\xd2\x2e\xc7\x65\xf4\x57\x49\xe7\xa0\xad\x1d\xc6\x2b\xbc\x00\x92\xec\xa1\x79\x08\xf1\x35\x7e\x34\x66\xef\xcc\xec\x52\xdb\x0a\x5c\x7a\x8b\x7f\x6e\x23\xd5\xde\x9c\xcf\x47\x9c\x3d\xb8\x68\x47\x45\x82\xbf\xe1\x3c\xf8\x6b\x6f\x08\x08\x2c\xb8\x72\x32\x4f\x41\x46\x9f\xef\x95\x88\x61\x2a\x77\x5b\xa3\xfc\xb3\xdf\xd6\xa5\xcf\x19\x09\xb2\xae\x6c\x68\x42\x78\x27\x09\x94\xde\x89\x29\x96\x2e\x0f\x19\x0c\x85\xd5\xdc\xe0\x4f\x1b\xad\xa7\x43\x69\x82\xb8\x4c\xc5\xe9\x51\x3d\x6b\x92\x34\x26\xc4\xf4\x2d\xc8\xc2\x50\x94\x6a\x29\xcd\x4f\x8f\xa4\x6a\x4a\xb8\xae\x12\x8e\x74\x53\xfc\x4c\x67\x0c\xa7\xa5\x44\xaa\x55\xe7\xf2\xb9\xb5\x67\x23\xf4\xe2\xb3\x71\x7e\xa9\x05\x31\x7e\xad\xb9\x7e\x58\xb2\x4c\x31\xa7\x29\xae\xdb\x4e\x5c\xcc\x52\x55\xbd\x29\xad\x6e\xfa\x72\x9a\xc4\x0f\x90\x01\xd5\x23\x0b\xc3\x2d\xf5\xf3\x85\x82\xf8\xd4\x69\x7b\xca\xb0\x23\xee\x9d\x5f\x6e\x7a\xc3\xc1\xcc\xda\x4c\xa3\xf7\x4d\xea\x31\xa4\xac\x7a\xda\x3b\x00\x11\x7d\x45\x28\x0a\x3e\xe9\xdd\x65\x53\x59\x36\xb6\x67\x8d\x7c\x64\xf7\xe7\xa3\xd4\x55\x86\xe6\x28\x52\x3d\xe9\x11\x3e\x7a\x4e\x47\x43\xa2\xa9\x83\x9d\xff\x92\xae\xac\x84\x88\x2d\x44\x49\xda\x31\x17\x16\xa6\x18\x75\xe6\x20\x49\x95\x7b\x64\x6d\x1e\x96\x27\x37\x91\x0d\xc3\x06\x24\xe8\x35\xf5\xe3\x1b\x4c\x3c\x44\x69\x3c\xec\xe3\x0f\xfb\x3f\xbf\xb7\x80\xf9\xd9\x0d\x38\x10\x14\x6c\x96\x5b\x7e\x6b\x53\xb1\x66\x31\xaf\xd7\xa1\x2d\x2c\x17\xdc\x43\xdd\x30\xd2\x26\x21\x37\x82\xbd\x21\xe0\xe6\x2b\x7d\x7a\xfd\xbe\x38\x19\x81\x66\xa5\xde\xaa\x9c\x3c\x10\x07\xec\x50\x89\xbe\xed\x18\xb8\x57\xc1\x1a\x45\x03\x33\x52\x3d\x66\x40\xdf\x05\x8a\xf3\x6f\x61\x21\x68\x43\xd3\x0f\xd5\x23\x13\x6e\x1b\xdd\xbf\x53\x7e\x3f\xf5\x50\x4a\xd6\x0d\x9a\xb2\x4a\x5b\x28\x40\x33\x03\x6a\xe1\x26\x02\x55\x81\x72\xf6\xe1\x33\x1f\x9e\xb3\x18\x2b\x24\x8a\xe9\xd3\x4f\xf9\x1d\xce\x0c\xe7\xae\xc4\x16\x4b\x12\x8a\xa5\xdb\xec\x77\x83\x4f\xb9\x4a\xb8\xe7\x6e\x43\x5f\x29\x19\x19\x46\x83\xde\xa5\x7c\x28\x23\x2a\x49\x61\x02\x76\xb4\x88\x4e\xd8\x0c\xd6\x48\xe7\x86\x68\xad\xbc\xca\x67\x50\x7d\xde\x47\xe3\x79\x0e\xbb\xcc\x56\x6a\xad\x38\x3a\x38\x50\xb5\xcb\x85\xcf\x2c\x05\x18\xa6\x58\x8d\x47\xc0\x15\x4b\x9c\x00\x31\xca\x47\xb8\x68\xb9\x71\xb4\x00\x8e\x07\x5d\xae\xcb\x86\x1e\xaa\x0d\x8d\x47\xa9\xac\x62\xbb\xbb\xa2\xbb\xb8\xb3\xab\xfa\x8b\x53\x48\x70\xd8\xa1\xfc\x70\x1f\x6d\x50\x3b\x60\x1c\x71\x3f\x77\xa3\xe4\x7a\xae\xe2\xba\x1e\xea\x75\xc5\x75\x5f\x55\x64\xed\xb8\xa6\xe1\x0b\x82\xee\x09\x0c\x3f\xd8\x17\xf4\xe4\x10\x64\x87\x9d\xcc\x35\xea\xe1\x9c\xe5\x06\xb3\x90\xbb\x2d\xfd\xd2\x8f\xe2\x7d\x31\xab\xee\x0e\x5b\x45\xf2\x50\x1b\xeb\xf5\x95\x47\x7a\xcb\xd5\xbb\xb6\x66\x2d\x07\x06\xc8\xc3\xc2\x25\xe3\xdc\x6a\xa4\x2d\x44\x3d\xc6\x36\xae\xdc\xba\x50\xf3\xcd\x26\x2c\x96\x2e\xf2\x2d\x4b\x4b\x2c\x41\x70\xcb\xd3\xf5\x2a\x29\x73\xb1\x2e\xb4\x8b\x72\x99\x9c\xa2\x57\x01\x15\x98\x6a\x8e\x5c\x58\x86\x8c\x24\xc2\x83\xe5\x8c\x7c\xec\x83\xf3\xb5\xad\x2f\xa3\x44\x47\xb0\x92\x4e\x95\xc8\x1d\x37\x0c\xdf\x0a\xf9\x70\x53\x54\xcc\xa1\xec\xfc\x1c\x42\x9b\xa1\x71\xfb\xa5\x50\x88\xfe\x00\x0d\x0b\x20\x3f\x60\xb5\xea\x50\x8e\xe2\x6d\xff\x0b\xfe\xec\x4e\x2f\x0c\xf1\x61\x47\xbd\xe9\x27\xf3\x20\x8f\xf6\x34\xde\xcc\xcc\xe0\xf0\x05\xe7\x28\xc3\x8a\xbb\xdc\xa5\x84\xd2\xbb\x31\x3d\x48\x29\xbe\x3d\x92\x26\xc1\xe6\xd9\x1f\xd3\x1b\x08\xe1\x4a\x8a\x7b\x8c\x69\xc6\x50\xda\x1e\x0d\xe7\x63\x13\x2f\x08\x1c\xe6\x99\xca\x60\x65\xea\x5d\x62\x73\xea\x5f\x60\x63\x1f\xeb\x5f\xec\xd5\x0d\x41\xd9\xfe\x2e\x6c\xd0\x80\x1c\x2c\x35\xcf\xe1\x31\x6e\x6a\x9b\x4b\x10\x77\xa1\xab\x36\xce\x22\x3a\x94\x65\xc2\x18\x53\xaa\xcb\x73\xcc\x89\x5d\xee\x20\xf4\x8a\x45\xab\x82\x42\xe4\x23\x49\xaa\x22\x53\x3b\x31\x8e\xae\xa4\xe1\x27\xae\x52\x46\xaa\x99\xff\xd2\xc9\x91\xab\x11\xc3\x0e\x34\x0f\x92\x34\x1a\xdd\x90\xb1\x47\xc7\x53\xc8\xed\x63\x29\x1e\xb3\xe3\xa5\xe4\xfb\xde\x7b\xe7\x00\x81\x43\x04\x88\xdc\xfd\xf3\xc0\x6c\x18\x5f\x7f\xe6\xea\x26\x8f\x6c\x72\xed\xcd\x08\x70\xd2\xf9\x48\xe1\x93\x65\x99\x66\xd3\xe4\xb2\x93\x66\xac\x6d\xa5\xf9\xdd\x4b\x98\x3d\x64\x86\x0d\xe7\x67\xf6\xdd\x1a\x04\xdd\xe4\x39\xe6\xd8\x2e\x30\x1b\xc5\x28\x62\x33\x78\x36\x67\x8a\xa1\x13\x04\xe1\xf8\xd4\x89\x80\x67\xac\x45\x0d\x1f\xd5\x1c\x91\x42\x90\x0f\x2b\x2b\xcc\xa9\x8b\x15\xa0\x8d\x2c\x6a\xcb\x46\x82\x50\xac\xe7\x32\x6a\x71\x03\x92\x98\x9c\xa8\x24\xea\xb7\x12\xe6\xfc\xa8\xc2\x20\x93\xe4\xb1\x59\x87\xdc\x29\xbd\xfc\xab\x09\xe7\x98\xf4\xf9\x5c\x22\x61\x62\xf0\x1a\xa9\xd7\x7d\x36\xc5\xfc\x27\x34\x37\xdb\xe5\xdc\x25\xd8\xba\x50\x3b\xb1\x0b\xd0\xa5\xf3\xa1\x67\xc1\xe2\x89\x36\x75\x44\x16\xc9\x01\x53\x9a\x1f\x00\x14\x06\x6c\x6d\x4d\xce\xdc\xfa\x24\x5c\xba\xc4\x21\xfd\x44\x58\xfe\x72\x00\x77\x9a\xd6\x35\x67\xfa\x17\xad\xa4\x93\x13\xbe\x66\x6d\x0b\xfd\xe5\x00\x5e\x2b\x27\x4c\x24\x61\xe9\xd4\xce\xd1\x7e\xba\x95\xe1\xc1\xfe\xae\xfd\x4e\x95\x4d\x6b\x64\x63\x9b\x08\x11\x0f\x89\x1d\x0c\xe8\x77\x6e\x03\xa1\x3c\x9f\x97\x48\x0b\x73\x41\x1a\xf9\x76\x81\xc6\x05\x34\xa8\x98\x26\xe4\x27\x7e\x6c\x78\x91\xa0\xf3\xad\x89\xa6\xe8\xc1\x1d\xc6\x0a\x51\x4f\xec\xd5\xc2\x2d\x8b\x93\x20\xf2\x97\x3f\x1c\x10\x5b\x3b\x5a\x8b\x5b\x0a\x79\xdb\x6d\xea\xae\x86\x3c\x2c\x14\x66\x56\x26\x95\xc6\x15\x49\xe5\x7b\xa5\xa4\x00\x81\xce\x8b\xd5\x10\x28\xa9\x7c\x0d\xe5\xef\xb5\x7c\x1c\x94\x2d\x47\x56\x5f\x68\x05\xb8\x94\xe3\x3c\x24\x15\x59\x7d\xc0\xae\xc0\x29\xbc\x5e\xfb\xb7\x5c\xa7\xe1\xb8\xe3\x46\x94\x6a\x89\xb0\x49\x51\x3e\xd3\xdf\xfc\x4c\x12\x3e\x9f\x49\x06\x29\x4d\x82\x44\x21\x88\xaa\xcb\xa6\xd0\x98\xc1\x6a\x69\xaa\x2b\x47\x47\xfc\x7c\xb1\xc4\x2a\x40\xb9\x64\xbd\x5e\x69\x26\x21\x29\x5f\xfe\x31\xd1\xbb\xa7\x3d\x98\xee\x1a\x8c\xdf\x78\x53\xfc\x5c\x60\xa3\x14\x1b\xe1\x9b\x85\x8e\xa4\xab\xb9\x0c\xeb\x14\x03\x8b\x92\x9a\x71\xb1\x2f\xfc\xf3\x41\x2d\x60\x38\x28\xb3\x5b\x0b\xc4\x11\x50\x81\x47\x16\xab\x3c\x23\xdf\x07\x54\x01\xeb\xa8\xec\x05\x07\x4b\xee\x1e\xab\x9e\x44\xe8\xd9\x69\x55\x9f\x66\xd9\x0e\xad\x3b\xce\xd2\xa1\x14\xa2\xfa\xef\xdc\x0d\x54\xf1\xa6\x94\xbd\x40\x6b\xd5\x63\x9f\xd9\x1b\x6f\x2c\x64\xcf\x0a\x40\x71\xeb\x0c\xa3\x64\x9b\xb2\x98\xe8\x83\x7a\x98\xf1\xd8\x5a\x0e\x08\xa3\x7d\x23\xba\x7e\xca\x5f\x0d\x66\x61\x53\xab\x5a\x68\x06\x96\x69\xcd\xaf\xae\x67\xc4\x55\xab\x8b\xa3\xe9\x6c\x8c\xe2\xb6\xd7\xbd\x0b\xe7\xf0\x85\x2c\x4d\x64\x84\x16\xfb\xb2\xe6\xdd\x8d\x06\x84\x75\x54\x8a\xa5\x34\xb1\x2d\x01\xdc\x58\x98\xd0\x6a\x8b\xce\xcd\x36\x65\x8a\x46\x2e\xa8\x50\x31\x92\x2c\xa4\xdb\x5e\xd8\x42\x8b\x02\xec\x4c\x22\x3e\xa6\x0c\x5d\xaf\x0f\x6f\x5a\x8e\xf7\x2d\x8c\x8b\xbd\x4a\x29\x0c\xdf\x7c\x4e\x5d\x63\xc4\x3e\x10\xef\xc3\xd4\x5e\x2e\x41\x92\x43\xc2\x8a\x16\x86\x35\x60\x48\xe5\x50\x3f\x28\x0e\x1e\x1e\xd2\x71\x62\x97\x62\x45\x9d\x9f\xae\x29\x75\xd3\x1e\x5b\x3a\x74\x0c\x2c\x60\x62\x9d\x75\x42\xeb\xcd\x00\x4e\x90\x8e\x47\x46\x55\x83\x70\x50\x12\x27\xb1\x8e\xdb\xdb\xd4\x37\xb4\xc0\x91\xf0\x8d\x55\x5b\xe7\x5e\x18\xbf\x11\x56\xcd\xbd\x61\x79\xda\xe9\x43\x4b\x78\x04\xa1\x03\x94\xfd\x7e\x70\xda\x49\x9e\xff\xbc\x89\xa4\x99\xb3\xaa\x79\xe2\x4e\x81\x6e\xae\x79\x26\xb0\x34\xdb\xf0\x60\xeb\x67\xd4\xf0\xb1\xc9\x34\x2c\xce\xa3\xe1\xbf\x6b\x0a\x0d\x9f\x34\x7b\xfe\x8b\x70\xcc\xc1\x34\x65\x3f\x7e\x7e\x27\xf1\x6f\x2a\x65\xfb\x42\x81\xe5\x0c\x74\x52\x94\xbf\x1e\x4f\x0e\xf2\x0b\xd8\xc4\x62\xcf\x3e\x4c\xbd\xea\x4b\x7c\xbc\x16\x27\x27\xc7\x27\xdd\xfb\x45\xa3\xf3\xbc\xb9\x7a\xf2\xf3\xdc\xa4\xbf\xf9\x7e\xe5\xe4\xa4\x9b\xac\x5f\x98\xe8\xe4\x64\xb5\x15\x25\x4a\x55\xe6\xcb\xc6\x1f\x38\x1e\x3e\xb4\x05\xf7\x5e\xff\xe3\xcd\x74\xb6\xcb\xec\x37\x1b\xdb\xc6\x6d\x8a\xee\xae\xf3\xe1\xb9\xb3\x0e\x0e\x91\x4e\x6a\x87\x82\x7c\x99\x2a\x87\x2b\x9b\x85\x0a\x4a\xc0\x9b\x7e\x7e\xfe\xfc\x79\x94\xbc\x5c\x73\x6e\x9d\xa6\x67\xfe\xb4\xf0\xf9\xac\x37\x41\x37\xb8\x6f\x87\xc3\xfc\xa2\x37\xac\x4d\xd1\x97\xdb\xe8\x2c\xaf\x41\x21\xd2\xe0\xa1\x0a\x6c\xa5\xe8\x58\x89\x22\xc4\xf6\x82\xad\xb7\x1d\xe2\x96\x15\xfb\xd0\xe6\xf3\xfe\x13\x4f\x4e\x9e\xc3\xff\xa1\xe2\xe6\xf3\x28\x8c\x86\xb8\xe3\x9f\xd3\xf5\x66\xa7\xde\xee\x42\x1a\xde\x25\xd5\x00\xcc\x0e\x80\x89\x7e\x8e\x1a\x02\xaa\x11\xad\x46\x74\x98\x43\xd1\x24\x4a\x74\x7d\xe0\x04\x0f\xaa\xca\x89\xa5\x1c\x2a\xde\xae\xe8\x0f\xc9\x16\x99\x67\xbb\xf6\x48\x74\xd5\xbb\xab\x8d\x47\xc3\x3b\x60\x79\x6a\x3c\x28\xc6\x76\xcb\x75\x0f\x2f\xb0\xe0\x4c\x35\x9e\x48\x5d\x35\x7e\x0d\x99\x3e\x5b\x36\xc6\x53\x8b\xa8\xc8\x7a\x73\x62\xde\xdf\xf5\x77\xfb\xed\xb2\xbd\x45\x5b\xa4\x5e\xb7\x92\xdd\x36\xca\xf3\x11\x6e\xf2\x14\x41\x32\xe7\x51\x3d\xd3\x90\xb9\x2c\xe6\x57\xd7\x81\x87\x67\xf9\xab\x1c\x67\xc6\xac\xe4\x7f\xe4\x70\x7b\xe7\xc3\xf6\xde\xfb\xa3\x9d\xbf\x1f\x1d\x66\x3a\xd0\x16\x81\x25\x36\x49\x34\x19\xfe\xed\x12\x0e\x90\x48\x67\xb2\x63\xee\xbe\xae\x51\xa9\x3f\xc0\xc0\x7f\xe2\xd4\x2e\x9f\xb3\x2a\x8b\x3e\x5d\xf0\xa0\xb2\x78\x61\x08\x44\x5a\xcb\x54\xe5\x95\x67\xf2\x4a\x14\xbf\x09\x0b\x5f\xfc\x71\x2c\x5c\xde\x82\xc0\x04\xe9\x14\x69\xcf\xd0\x5a\x86\xc2\xcb\x46\x40\x36\xef\xbf\xd9\xb8\x2c\x81\x50\x26\x54\x8e\xc6\xc0\x64\x02\x18\xe7\x6c\xc1\xb6\xa6\x48\x77\xd0\x42\xec\xdb\xe9\x21\xec\x5b\x7b\xf4\xc2\xe5\xbd\x37\x24\xe9\xca\x8a\x5d\x56\xf9\x97\xfc\x4c\xf9\x75\x40\x23\x85\x6a\x05\xb0\xfc\xa0\xb4\x71\x8b\x4d\xb9\xfc\x70\xb7\x3f\x86\x1d\xe7\x4e\x59\xdf\x37\x23\x53\xe9\xa3\x02\xad\x86\xab\x8b\x24\x23\xc6\x60\xd0\x2e\x2a\x6e\x2a\xb8\x93\x8e\xb2\xaa\xb1\xb4\xfb\xe9\xc8\x6f\xa5\xae\x03\xaa\xf2\xa3\xb0\xb7\x6f\x1e\xac\x24\x5b\xd1\x46\xfb\x74\x92\xf7\x3e\x21\xc1\x91\xb8\xa4\xb2\x66\x37\x7e\x4f\xac\xd9\xe5\x5f\x5a\x73\x4b\x6a\xb6\x53\x8d\xe3\xb5\xc6\xf2\x08\x6f\x67\xf2\x37\xe3\x21\x2c\x4e\x74\x0b\x4a\x16\x41\xc5\xa0\xac\x8b\xf4\x53\x57\xdc\xc1\xfd\x55\xe4\xe6\xc4\x7e\x32\x69\x62\x1d\xa9\xa4\x0a\x2d\x48\x5d\xd2\x0b\x78\x61\x6d\x30\xe1\x7d\x55\x4a\x58\x00\x69\x30\x61\x63\xaa\x92\xad\xff\xb9\xaa\xaa\x96\xa1\x85\x46\xad\x2b\x61\x3d\xa0\xbd\xf9\x00\x30\x35\xad\x34\x5c\xdf\xbd\xd4\xe6\xad\x29\xe9\x8f\x99\xb0\xeb\xac\xf1\x34\x6b\x59\xea\xf4\x8e\x50\x2a\xbc\xec\x90\xa1\xb5\x0e\xa7\xb1\x05\xb5\x94\x44\x8c\x91\xab\x52\xa5\xab\xf6\x66\xf1\xf2\x07\x8b\x7d\x0b\xb6\x19\x38\x9f\xe4\xfd\xda\x6c\xcc\x28\xd5\x7a\xd6\xed\x21\x0a\xa7\xdc\x92\x59\x71\xc9\xdf\xb4\x2e\x17\x69\x93\x6e\xd7\x08\x5d\xb9\xb9\x2b\x37\x03\x11\x11\x4b\x03\x3a\x7e\x3e\xd7\x21\x6d\x73\x00\x9d\x3d\x14\x32\x27\x15\x13\xa4\xad\x9e\xdf\x9d\xb7\xc0\xa0\x50\x45\x8b\x31\xdb\xb2\xe6\x8e\xc6\xa3\x35\xd9\x76\x5d\xcb\x7b\xdc\xe6\xd1\x4c\x5c\x3f\x62\x62\x4f\x36\xe7\x34\xe8\x06\x6e\xba\x92\x9c\xfd\xac\xc7\x2e\x5c\x14\x7e\xf4\x65\x82\xc2\xd2\xbd\x3b\xcd\x65\xe6\x28\xf1\x69\x1f\xa9\x4d\x05\x6a\x58\xbc\x2a\xd1\xd2\x6d\x47\x67\x5f\x32\x1b\x53\x9d\x27\x30\x4a\xe0\x7c\x39\xe2\xf8\x85\xd8\xf8\xf1\xd3\xf1\xf3\xb9\x0e\x55\x8c\x5f\x00\xa4\xa2\x45\xed\x5f\x3d\xbd\x55\xe1\x7a\x7d\x59\x67\xe9\x25\x50\x81\xc5\x92\x8e\x42\xf0\x44\x60\xb2\x80\xf3\x68\x1e\xec\x1c\xee\xfd\x78\x00\x31\x3f\x1e\xbc\xa3\x9e\x79\x70\xe3\x09\xba\xa0\xb2\x0f\x2a\xa6\xe8\x68\x9a\xc3\xa1\x63\x08\xb3\xf4\x87\xe1\xf8\xec\x13\xb4\x06\x5d\xec\xe3\xc4\xb3\x64\xbd\x76\x3e\x19\x5f\x91\x8d\xff\xd1\x78\x66\x09\x76\xed\xf4\x8e\xe0\x58\x6e\xaa\x76\x4d\x38\x00\x43\x0d\xa8\x3a\x6e\x5a\x35\x5c\xed\x77\x8b\x25\xad\x7d\x73\xb4\xfb\xce\xa2\x8d\x06\xa3\x0f\x7b\x68\xee\xf6\x5f\xc4\xdd\xa9\x96\x55\xb4\xe2\x66\x34\xed\x9d\xab\x95\x46\xd6\xe2\xc6\xb5\x1b\x38\x10\xf6\x46\x35\x4e\xd5\xab\x8c\xc2\x96\xbc\x44\xfc\xd8\x1b\xd4\xa8\xa9\xee\xbf\xbf\xba\xb6\xe3\x64\x70\xfa\xc5\xd1\xea\x54\xea\x45\x03\xc6\x71\x88\x88\xcf\x8a\x2f\x7d\x3a\x2b\x4b\xa4\x54\xaf\xce\xac\x62\x1b\x4d\x0c\x4f\x6b\xe5\x79\x50\xe6\x79\x69\x14\xba\x55\xe5\xab\x6b\xb2\x50\x43\x20\xdb\x87\x87\xbf\x1a\x06\x4c\xa4\x5f\x0d\xe3\xcf\xbf\x1e\x0d\xbd\x06\x2b\x81\x2d\xc3\x3d\x31\xf7\xb2\xdb\xa6\xf2\x6b\x3c\xe5\x4b\xfd\xa7\x11\xaa\x9c\xca\x6f\xa8\x9e\x0c\x27\x96\x82\xbc\x73\x2e\x86\x65\x81\x89\x23\x4e\x20\x2f\x1a\x9a\x7d\x94\x99\x77\x10\x56\x44\xa2\x59\x22\x96\xb9\x02\xd7\x6c\xba\x5a\xf3\x15\xae\xc0\x3d\xaf\xae\xf2\x11\xe5\x92\x2a\x80\x72\x4e\x07\xb9\xd5\x8b\xe0\xcc\x6c\xda\xdd\x86\xd0\xde\xb8\x4e\xa1\x77\x43\x15\x5e\x09\x8c\xeb\xfd\xe1\xa5\x4e\xab\xda\x83\x69\x23\xfd\x84\xd6\x32\x1e\x72\xe0\x3c\xce\xa7\x44\xde\xa6\x37\xd7\xe8\xc3\xad\x46\x16\x2a\x47\xf9\x0c\xed\x33\x0e\xc7\x93\x7c\x52\xbb\x85\x23\x0e\x8e\xc8\xf7\xb5\x3f\xe2\xaa\x66\xa8\x35\x74\x14\x00\x44\xef\xa7\xf1\x4d\xed\x0c\xd6\x3e\xba\x3b\xc1\x3e\x44\xf2\xd8\xeb\xf7\xad\xd5\x4a\xac\xaf\xf6\xfd\x0a\xa0\x8a\x74\x8f\x48\xcd\x4b\xe2\x05\x30\x6d\x7c\x8d\xd6\x02\xee\x80\xd6\xd6\x70\xed\xb9\x6e\xf9\x1a\xaf\xcf\xa8\x20\x0d\xb8\x4c\x90\xe0\xb0\x89\x59\x40\xd5\x52\x36\x48\xcf\xe8\xf1\x5d\x4f\xd1\xa4\x8d\x1e\x93\x07\xd3\x92\x99\x62\x65\xff\x4f\x26\x05\x66\x94\x19\x9c\xe9\x71\xb5\x91\x26\x74\xd0\x1c\xe6\x51\xf3\x5c\xb9\x68\x0e\xf3\x48\xa4\x9d\x86\xf3\x79\xac\xab\x2c\x40\xf7\x8c\x3f\xf2\x0e\x15\x8a\x53\x8b\xa0\x22\x58\x38\xa3\x19\x0a\x98\x52\x2c\xbb\x9c\x99\x16\x80\xf0\x63\x90\x37\xb7\x90\xe9\x87\xb5\xd0\xa0\x41\x73\x48\xaf\xc7\x64\xb2\x89\xc2\x56\xe6\xb2\xc3\x61\x65\x52\x08\x2d\xda\x16\x1e\xec\x8a\x9e\xac\xb1\x72\x2e\x17\x64\x4f\xe4\xb2\x8f\xdd\x0d\x29\xbc\x15\x01\x29\x74\x8b\xd1\xbd\x25\xdf\x45\xe5\x79\x3d\xfc\x4a\x69\x6d\x74\x73\x25\x52\x4e\xfe\xce\x1b\xad\xfc\x28\x37\x00\x94\x82\x13\xe6\xf8\x0c\xbe\x86\xdb\xf4\x4a\x46\x38\x7d\xe8\x4d\x3f\x44\x0d\xca\x9f\x74\xb3\xc2\xf3\x9a\xee\x39\x55\x0f\x3f\xa5\x99\x02\x38\x68\xcb\x07\xa1\xcb\x55\x10\xc3\x81\x56\x9d\xe8\xe1\x8a\xe6\x53\x11\x30\x01\x5d\x82\x67\x08\xd5\x1e\x8e\x4a\x20\x17\x34\x7b\x42\xba\xcc\x66\x4d\x96\x19\x8f\xf0\xce\xa5\x3d\xf1\xf4\xd4\xd2\x3a\x96\xb6\x69\xa2\x9e\x9f\x43\xec\x44\x3c\x39\xd1\xe9\x9a\xef\xe5\x0f\x99\x26\xa1\xac\x58\x6f\xd4\x9f\x8c\x07\x7d\xf2\x3a\x19\xaf\x4b\xa8\x16\x9f\xf4\x1b\xc9\x3a\xdf\x59\xf8\x41\xb3\xd5\x34\x47\xbd\xdb\x01\xac\xb3\xf1\x04\xad\xbf\x26\x4d\x60\x53\x26\x5b\x17\x24\x2f\x84\x17\xf7\xc9\x71\x0b\xb6\xaa\xd3\xf1\x97\x3c\xcf\xd6\x7f\xc0\x9f\xf5\x41\x73\x86\x4f\x94\x8f\x03\x30\xb6\x01\x01\xf9\x26\x53\xdb\xdc\x18\xba\x2f\xcd\xd6\x7f\x8e\x77\xc7\xff\x9a\x7f\xce\x4f\x3f\x0d\x66\xf3\xbd\xf9\xd5\x34\x89\x3b\xd9\xf1\xd6\xda\x3f\xba\xc9\x3a\x54\xdd\xbf\x3b\x9c\xdd\x0d\xf3\xcc\x51\x3e\x8c\xaa\xd7\x83\x60\x73\x8a\x59\x60\x9a\xf7\x46\xd3\x01\x09\x6f\xe0\xcb\x1e\x30\x43\x4c\xeb\x30\x84\x3b\x85\x03\xe6\xb5\xe5\xd1\x5c\x2c\x52\x6c\x9f\x64\x2f\x2f\x32\x85\xa4\xdc\xf8\x40\x5e\x14\xec\x56\x23\xc1\xf7\x46\x68\x35\x20\x88\xd5\x01\xef\x05\xb1\x85\x27\xf0\x1f\xaf\xe1\x2c\x41\x73\x2f\x69\x54\x65\x6b\x25\x72\x07\xa2\x13\x81\xe8\x05\xf0\xa3\xbf\x51\x6f\xed\x5d\xf7\xce\x80\x7a\x45\x1a\xff\x7a\x3d\xe2\xae\xc4\x17\x6a\xdd\x1f\x2b\x38\xd9\x6d\x38\x28\x32\x9f\x6b\xe0\x8d\xe8\xa8\x3a\x5b\x12\x74\x28\x80\x73\xc1\x07\xa1\x6d\x55\xe6\x4a\xcc\x8a\x4c\x50\x3c\x7c\x3b\x34\xeb\x75\x5f\x07\x3e\x3f\xab\x06\xb8\xfb\xff\x8a\x91\x6f\x72\x93\x3d\xe2\x01\xae\x4f\x28\xe9\x90\x4c\xac\xce\xef\xbd\x78\x91\x49\x57\xe2\x15\x3b\xd5\x25\x6a\x3e\x2f\xc6\xd0\xa5\xf3\xe1\x0c\xb6\xab\xf9\xfc\xbb\x97\xae\x61\xb4\x6e\xd8\xb1\x18\x4b\x84\xa6\xd1\x78\xe4\x43\xd8\x21\x02\x08\xf8\xae\x15\x87\x9d\x66\x5a\xe6\xf3\xca\xe8\x97\xff\x93\xc0\xee\x20\x05\x50\x72\x0b\xce\xf7\x0b\xaa\x26\xdd\xcc\xe0\xe8\x4b\x51\xf5\xfa\x1f\xe1\x0c\x8c\xac\x15\xb7\x8d\x57\x83\xf6\xfe\xa4\x49\x89\xbc\x0b\x8a\x0a\x43\x7f\x70\xbb\x33\xbc\xf2\xab\xef\x0c\x26\xe7\x2c\x17\xbf\x2e\x71\x04\xc9\x11\x1a\x17\x2b\x15\xcf\xa0\xa9\x51\x83\xbe\x71\x7d\x31\x98\x85\x7e\xb4\x0a\xf3\x2f\x0c\x9a\xec\x72\xd5\xd0\xc1\x13\xe6\x36\x9f\x5e\x3b\x4b\xe2\x81\x45\x41\xd9\xd6\xdb\x1c\x2d\x7b\xe9\x69\x97\x06\xe4\x52\x4d\xa3\x54\x7d\xab\x39\x92\xfa\xcf\x90\xc3\x46\xd7\xca\xe3\x9b\xd9\x93\xcc\x00\x29\x3f\x4f\xbf\x7c\x85\x59\x20\xe7\xe6\xe9\x97\x2a\x13\x41\xfe\x72\x8e\x31\x41\x95\x2d\xf2\x31\x6f\xb4\x4f\x7e\x1a\x2a\xc9\xf1\xb6\xef\xac\xb8\x64\xab\xbf\x34\xad\x52\xb1\xb5\x29\x5a\xb4\x08\xf3\x8d\xfe\xfe\x5d\x65\xd9\x72\xf9\x1f\x7c\xbc\x2d\x19\x46\x39\x1f\xc5\x5a\x6e\xb1\x68\xe3\x06\xd6\xcb\x83\xe2\x33\x22\x29\x6e\x8b\x69\x1f\xf3\x0e\xa5\xee\xe2\x89\x5e\xfa\x2b\xca\x66\xe5\x5e\x9c\x1e\x7b\xc8\xd5\x8e\xfb\x5d\x4e\x75\x7c\x96\x32\xdf\xe2\xb7\xdf\x55\x47\x2b\xc7\xc2\x46\x37\x20\x0f\x35\xba\xca\x6b\xff\x93\x7a\xcb\x84\x23\x28\x18\xc7\x15\x59\x13\x76\xdc\x2f\x61\xbd\x50\xc2\x07\x0c\xb9\x90\x47\x2b\x06\x37\xde\x2b\x1f\x52\x20\x20\x74\x10\x83\xa6\xc2\x80\x20\xbf\xc7\xf3\xd1\x34\x9f\x6d\xcd\x80\x3e\x9f\xde\xcc\x72\x71\xf8\x64\xe8\x45\xc5\xd8\xf2\x3a\x37\xa7\x3c\x19\xc4\x3d\xfe\xa4\x15\x30\xcc\xb5\x78\xdb\x2e\x26\xda\xf8\xce\x92\x78\xff\xcc\x9d\xae\xae\x9b\x28\x42\x27\x90\x06\x7d\x80\x97\x6a\x81\x38\x23\xee\x1e\x4b\xf8\x92\x3f\xc8\xca\x58\x0f\xff\xe7\x93\x8e\xaf\x00\x36\x8d\x52\x05\xe8\xa3\xb2\x22\x4e\x41\xf8\x4d\x80\x21\xc9\x9a\x54\x60\x49\xe2\x27\x48\x8a\x4b\x7d\x01\x71\xc6\x3a\xdd\x2e\x25\x4a\xbc\xf5\xeb\xbd\x24\x59\x79\x25\x5d\x92\x23\x45\xf7\xd8\x4b\xd2\x16\xc1\x0c\x0b\x5e\xe6\xf0\x12\x1c\x18\xd1\x1f\xbd\xbf\x64\x3a\x9d\xb9\xcd\x5e\xa5\x77\x6e\xb4\x07\x73\x17\x9d\xfa\xef\xc2\x39\xce\x0e\x36\x3a\xd0\xa0\xda\x7e\x54\xfe\xd9\xdd\xe9\x8e\x1c\xc1\xeb\x1c\x18\xa1\xf6\x0e\x76\x47\x51\xb9\x75\xd0\xd1\x01\x15\x17\xd9\x43\x8f\x2a\xf4\x9a\xec\xe6\xb9\x42\xe8\x97\x1e\xbf\xd4\x46\x30\xc9\x2f\x50\x8e\x65\x22\x1e\x4e\xd8\xbe\x9b\x5c\x00\x5b\x7d\x33\x3c\xc9\x88\x0e\x30\xc1\xd3\x84\xc9\x9e\xf8\xc4\x41\x8a\x15\x94\x64\x7b\x7d\x64\x29\x4d\xca\xb0\x0a\x91\xab\x8e\xec\xba\x52\x0a\xbe\xa3\x4b\x9e\x85\x93\x36\x65\x3c\x9b\xe7\xd6\xde\x1c\x40\x6f\x4c\x6f\xce\x71\xff\xb5\x38\xf2\x85\x03\xc5\x65\x11\x37\x34\xb2\xcf\xe0\x5c\x89\xab\x6d\x99\x0a\x7d\x59\x35\xbe\xec\xe4\x92\x8f\xaa\x16\xb1\xe0\x8a\x54\x61\x85\x2e\x64\x8c\x6b\x5c\xc4\xc2\x35\x67\x77\x91\xb1\x5f\x8c\x60\xa2\xf2\xf4\xe9\x5e\x0b\xff\x96\xd3\xac\x75\x4c\xfe\x2d\xa7\x7f\x9c\x02\x3f\x64\xf0\x6f\x39\x6d\x38\xb8\x02\xe6\x75\x1c\x19\xf9\xa8\xc8\x61\x0f\x6e\x90\xc7\x7e\x96\x73\x89\x52\x86\xe1\xdf\x72\xfa\x78\x82\x97\x99\xd0\x44\xf9\x28\xe7\xb8\xc1\xd3\x09\xd7\xe3\x3e\x25\x97\x9f\xa3\xba\x89\x71\xb9\xfb\x59\xfb\xd2\x4b\xe8\x1a\x54\x7e\xc5\x7b\xd1\x31\xcb\xfa\xac\x58\x73\x7e\x94\xd1\xbd\x45\x50\x48\xdc\x81\xe4\xfd\x01\x1a\xbe\x24\xf9\x10\xb1\x69\xe9\x23\x81\xa2\xe4\x15\x46\x51\xec\x89\xee\x63\xb6\xd1\xfe\xf8\xbd\xca\x2e\x8f\xd3\x1f\xf9\x61\x7a\xc5\xa7\x1c\x7f\xec\x5a\xb1\x04\xc7\x17\xcb\xc7\xc6\xc2\xc8\x2b\x9e\x46\xbe\x60\x49\x92\xce\x69\x6c\x5e\xf2\x74\x3c\x1e\xe6\xbd\x11\x59\xd8\x6c\x65\x59\x50\xc8\x7d\x67\x81\xd6\x29\x5e\xfb\xf9\x7b\x58\xbe\xbb\x6b\x8a\x0a\x8e\x4b\x5f\xc8\x61\xd0\x5a\xaa\x7c\x12\x30\x0c\x64\x71\x14\x35\x28\x16\xce\x9c\xef\x70\xca\xf0\x99\xd3\x60\x3c\x2a\x9f\x85\xd1\x4e\x7e\x8a\x8a\xbc\x5c\x6b\x2d\x16\xca\x67\x72\x45\x55\x55\xfa\xc6\x98\x02\xbb\xc1\x0a\xbd\x98\xe2\x6b\x86\x23\xfd\xb6\x7f\x19\x9e\x03\xe3\xcf\xbc\x49\xc1\x74\x29\x62\x78\x1f\xf6\x2c\x85\x64\x86\xa7\x81\x55\x53\x67\xb7\xd8\xf6\x8d\xc7\x93\x47\x87\xc5\x94\xa2\x34\xac\x83\x1b\x12\x64\x78\x08\x92\x1d\x02\x3b\xcf\x20\xe5\x2f\xa2\x0f\x09\xc8\x62\x7f\xac\xa2\x85\x52\x8e\xf6\x4d\xaf\xd7\x7d\xa3\x8f\x39\xb1\xcb\x10\x13\x37\xd7\xdc\xec\x23\x64\x68\x21\x44\xa9\xd6\xa3\x1d\x7c\x4f\x4e\x93\xc4\x7e\x1f\x4f\x64\x05\x76\x50\x84\xd8\x76\x10\x43\x13\xa7\xc0\xbc\x2d\xca\x46\x62\x9d\x1b\xa9\xa7\x76\xb8\x2f\x93\xdd\xaf\xaa\xd0\x22\xec\x6e\xe5\x4a\x08\xbb\x49\xd5\x04\x5d\x95\x65\x10\xdd\x59\x09\xf5\x56\x7c\x16\xd6\x00\x75\xae\x05\x30\x73\x5b\xad\x69\x96\xdd\xaf\xbe\xa8\x93\x8e\xb9\x75\xd7\x89\x02\x91\xac\x4a\x93\x81\xad\x18\xd8\xd8\xb0\xe2\xc8\xbd\x1a\xf8\xf7\xfb\x7f\x27\x32\x6c\xdf\x5a\x70\x22\xc3\xf9\xcb\x30\x6b\x2b\xaa\xe2\x49\x4d\xb1\x36\xd5\x97\x52\xa0\x60\xcb\x96\x26\xd1\xc2\x8e\x00\x53\x6c\xe0\x8e\x8e\xbb\x86\xe9\x23\x2b\xd4\x2b\xd2\xe8\x6d\x80\x66\x94\x06\xc4\xb1\x5d\xa4\xb9\x4e\xb3\xcf\xc2\x63\x64\xe4\x1e\xd4\xee\x03\x92\xa6\x58\xb7\x70\x47\x65\x5f\x8d\x43\x6b\xa3\x84\x1e\x24\xa6\x99\x44\x36\xdf\xff\xb8\xfb\xc3\xce\xc1\x87\xd7\x7b\x07\xbb\x5b\x47\x87\x25\xe7\x70\xbd\x2b\xb4\x79\xea\xf6\xe8\xc3\xbb\xab\xd3\xf1\xb0\xd2\xa1\x75\x21\x0b\x6a\x55\x07\x31\x99\xd4\xdc\xdc\xfe\xf1\xe0\x60\xe7\xfd\xf6\x4f\x1f\x0e\x7f\xda\x05\xbe\x86\x62\xdf\xd3\xb4\xb7\xb5\xd9\x9c\xfb\x5b\x47\x47\x3b\x07\xef\x0f\xd1\x32\xa8\x8d\xfb\xd3\xc1\xde\x8f\xfb\x1f\x0e\x77\xf6\x5d\xcc\xab\x9d\xed\xb7\xbb\x5b\xef\x28\x6e\x53\x8b\xc5\xde\x6c\x6c\x6c\x7d\xb7\x7e\x51\x44\x5e\x75\x94\xde\xb8\x7f\x55\x37\x31\x20\x73\x3e\xe9\x51\xf8\x70\xf0\x2f\x3f\x25\x83\x16\xda\x8c\xc5\x16\x6e\x3c\xb5\x85\x41\x15\xaa\x29\x55\xb5\x88\x04\xa9\xb9\x98\x8c\x6f\xae\x0f\xf3\x6b\x38\x9e\x9e\x0d\xae\x7a\x43\xfc\x0c\x51\x25\x66\x16\xb5\x47\xb9\x64\x32\x9f\xa3\x95\x0d\x54\xfe\xcc\x6d\x94\x10\xbb\x88\x2d\xe1\x43\x6e\x7c\x2e\x1a\xdc\xe6\xd9\xc6\x4b\xce\xd1\xe6\x9f\x6c\x17\xdd\x4e\xf4\x4e\xa7\xb6\x20\xdb\x6a\xb8\xb9\x82\x73\x42\xc6\x51\x0d\x38\x20\x31\xbe\x79\xff\x08\x37\x4f\x08\x03\x8f\x3f\x43\xfe\x83\xae\xd9\xbe\x5c\x8f\x47\x78\x91\xcd\x17\x67\x6b\x2d\x54\x45\x27\x00\x5e\xee\x18\xdf\xdc\x69\xa4\xf8\xf2\x58\x92\x29\x80\x12\xd4\x27\xfd\x93\x66\xb7\x91\xe4\xf1\x5a\x27\xe1\x1b\xf9\xa4\x7d\xc5\x1a\xbb\xd1\x5a\x24\xf2\x87\xc7\x9b\x5d\x14\x1e\xc1\xaf\x17\xdd\x97\xba\x47\x1a\xad\x8e\x60\x1c\x6d\x44\x69\x1c\x20\xcb\x09\x21\x9e\x1b\x24\x47\xa1\xa2\x12\x0d\xed\xe5\x06\x3a\x19\xc1\x96\xc3\x4e\x5f\xaf\xb7\xa4\xc3\x60\x89\x14\x01\x9f\x92\xab\x80\xd7\x83\x2f\xe8\x8b\x54\x8f\x50\xe2\x1d\x48\xd8\xf8\x77\xf9\x28\x8b\xa5\xdd\xec\xe5\x43\xcd\x13\x7c\x48\x40\x61\x9c\xc4\xed\x62\x6a\xa9\x06\x90\x11\x0b\x15\xe6\xe1\xbb\x82\x03\x1f\x7f\xf4\xbe\xc4\x32\x8d\x30\xf2\x35\xe4\x34\x0a\x81\xc4\xb8\xc4\xde\x17\x4c\x14\x41\x88\xeb\xf1\x67\x06\x04\x1f\x71\x6b\x23\x9c\x6d\xc1\x44\x99\xa0\x5d\x77\x99\x2a\xcf\x21\x77\xb2\x0e\x7f\xda\xba\x9d\xc4\x5a\xc9\x5c\xaa\x68\xa8\xf9\x7c\x89\x1e\x46\x6d\x6e\x58\x49\x6d\x57\xd2\x45\x72\x67\x08\x6a\x53\x54\x8e\xa0\x35\x91\x59\xec\x87\x17\x88\x99\x09\x23\x29\x0e\x27\x20\xd5\x60\x3d\xef\x65\x5c\xb4\x41\x7f\x13\xd4\x25\xcd\x74\xfa\x1a\x27\x07\x66\x39\x20\x8f\x48\x6a\xc6\xf0\xb9\x36\x48\x7e\xcb\x15\x65\x19\x4c\x8c\x0d\xb4\xa8\x51\x98\x0a\x8d\xcc\x2e\xd9\xc4\x84\xf1\x5c\x95\x70\x3b\x83\x64\xc1\x82\xa2\x00\x15\x78\x16\x8d\x06\xd7\x16\x20\x06\xd5\x0e\xff\x5d\xf5\x52\xf3\x5c\x3f\x4b\x0d\xdf\xeb\x51\x6e\xbb\x15\xd0\xc0\x35\xd4\xd6\x69\xb0\x06\x37\x90\x77\x0b\xe3\x0a\x98\x78\x5a\xd5\x70\xf5\xb8\xa7\x9d\x70\x6d\x2c\xfc\x6d\xc2\x4c\x36\x6c\x4f\x9f\x3a\x76\x38\x47\xf9\xc5\xfe\x24\x4f\x6d\x10\xfa\x0c\x82\x89\x51\x85\x34\x02\x41\x42\x35\xb4\xc3\x9b\x73\x0d\x0d\x82\xb6\xd0\xc7\x31\xac\x9f\x48\xab\x62\x5c\xf7\xfa\x9e\x36\x9b\xfe\xe0\x62\x30\x9b\x1a\x60\xef\xae\x44\xa7\x2e\xbf\x00\x22\x48\xbd\x4a\xe4\x94\x8c\x9b\x40\xd4\x1a\x1d\x2e\xb3\x35\xf8\x93\xd0\x17\x2f\x05\x5c\x41\xb6\xd3\x19\x56\x3b\xa1\xd4\x0d\x4e\x76\x4f\xa8\x03\x82\x04\x29\x58\x40\x7a\xcf\x97\x5d\xe3\xb2\x09\xea\x41\x5c\x60\x41\x65\x6d\x1e\xfa\x41\x0c\xee\xd0\xbd\xc5\x14\x97\xc7\xf8\xfc\x7c\x9a\xcf\x04\x6d\xeb\x0a\x93\xe2\x32\xfe\x99\xcf\x37\xfc\x8d\x41\xdf\x99\x24\x63\x36\x07\xc3\xc7\xf8\xb2\x0c\x38\xa2\xa2\x92\x33\x35\x1a\x73\xe1\x97\x1b\xf3\x39\xe5\x7c\xb9\xc6\x11\x89\xb5\xae\xd0\x10\xf0\x6c\x85\x5d\x4c\x22\xac\xb5\x36\x33\x89\x77\x56\x18\x5a\x9b\x38\x02\xb6\xa7\x99\xfb\x23\xcc\x09\xe5\x45\xd8\x3c\xa0\x9b\x41\x0b\x2f\xc7\x93\xd9\x6b\x98\x01\xe5\x13\x36\xe6\xb6\x9b\xf1\xc3\x2d\x22\x4b\x59\xee\x24\x1f\x3b\x98\x9d\xe8\xf0\xcd\xde\xc1\x11\xe7\x4b\xf9\x05\x3f\x60\x0f\xa6\xc7\x50\xb2\x7b\x4c\x70\xbb\x0a\x51\xbc\xfa\xfd\x07\xec\x28\x82\xa9\xef\xd3\x7f\xa1\x25\xc9\xb5\xd6\xf3\xbe\x95\xeb\x80\x8c\x18\xb7\x47\x3d\x12\x53\x3f\xf4\xf3\x3e\x96\xcd\x30\xfe\x65\xb6\xd1\x89\xd0\x81\x73\xe4\xef\xde\x6c\x86\x46\xe6\x3b\x0d\x89\xf2\x31\x15\x80\xfc\xe7\xc3\xf1\x18\x0e\x23\xd1\x59\x3e\x18\x46\xdd\x18\xa3\xd7\x7f\xbf\x91\x00\x9b\xd5\x08\x4b\xd0\x7e\x8f\xc9\xbf\xe5\x64\xa5\x9f\x72\x75\x7d\xa5\x90\xf7\xdd\x68\xad\xdc\x0b\xfe\x6f\xc6\x37\x93\x69\x9c\x7c\xdf\xda\xec\x58\xb6\x67\x6b\x77\x7f\x17\x39\xa3\x34\x8c\x68\x75\xc3\x71\x2c\xb2\x6f\x2e\x11\x2f\x7f\xf8\x72\xf2\x68\xfc\x0a\x32\xc6\x7c\x9e\x52\x6c\x43\xdb\x3d\x3e\x73\x92\xf0\x0f\x07\x1f\xde\x1e\xee\xfd\xe1\xf7\x1b\xad\x0f\x87\x47\x07\xf6\x5d\x0e\x00\x90\xc8\x39\x41\x82\x46\xce\xfe\x85\x28\xc3\x56\x32\xfb\xd7\xee\x60\x04\xbf\x34\xa7\xd8\xe8\x0c\xf3\x14\x7f\xe8\x76\xa8\x75\x30\x1e\x3f\x1e\x6d\xbf\xbe\x19\x0e\x7f\x82\xd3\x49\x6a\xe3\x6c\x04\x5d\xef\x3f\x50\x90\x7a\xc6\x95\xa2\x10\xb3\x32\xc7\x7f\x44\xdb\x45\x82\x07\x0a\x21\xd8\x58\xd6\x47\x3a\x6e\x6d\x74\x13\xc1\xae\x2a\xb5\xd5\x85\x85\xef\x91\x66\xc3\x62\x34\x46\x3e\x37\x4a\x23\xf8\xd0\x66\x37\x59\x6b\xa9\xf0\x8b\xae\x6c\xf7\x97\xaa\x82\xef\x60\xab\xdd\x48\xd6\x18\x2d\x73\xa5\x52\x7e\x67\x53\x00\x23\x33\x55\x09\xbf\xa7\x04\x73\x35\xd5\x2c\x41\x2b\x7f\xf1\x9c\x2e\x7e\x5f\x0f\xc7\xbd\x59\x1c\x6d\x34\xa3\x86\xe4\xff\x9f\x94\x3f\x49\xf4\xd3\x4f\xa9\x11\x97\xe6\xca\x4c\x01\x26\xb7\xd1\x6e\x11\x3c\xce\x74\xd9\x12\x8c\x32\x8a\x45\x9c\xf4\xef\xbf\x5b\x24\x6b\x1d\xf8\x38\xe9\xbb\xdf\xb8\x93\x1e\xb9\xaf\xb4\x53\xf5\x79\xd2\x24\x16\x33\xe9\xe0\xff\xe2\x7f\xcc\xe3\xe3\xc6\x5a\x37\xe1\x64\x9b\x0d\x93\x56\xd7\x4b\x07\x07\xb5\x26\xac\x0d\x49\x91\x36\x99\x15\x18\x63\x9c\xab\x9c\x51\x4e\x52\xc0\xd1\x5c\xc1\x79\xf1\xe6\xea\x15\x8b\xff\x71\x9a\x3d\xb0\xbc\xda\x3a\xda\x39\x7a\xbb\xbb\x63\x8f\x2c\xc7\x9c\x0e\x3d\xc7\x1f\xc6\xbf\xd2\x23\x4d\x41\x7b\x31\x38\xc3\xe5\x9c\x03\x3d\xf2\xf6\xfd\x9f\x58\xf2\x84\xd2\x3b\x38\x5a\xf4\x95\x96\x96\x15\x45\xc3\x3c\x99\x0a\x41\x08\x00\xba\x25\x23\xb9\xe0\x5c\xa1\x82\x8a\x08\x08\x4b\x41\xc8\xb5\x13\x5e\x95\xd8\x08\xdb\x80\x0f\x87\xfb\xef\xde\x1e\xb1\x64\x88\x74\x18\xf7\x14\x3a\xf7\xc3\x2e\x12\x35\x5b\x0a\x48\x1f\xb6\x2c\x1b\x93\xc9\x66\x3e\xbe\x46\x5b\x31\x71\x69\xcf\x77\xf9\x50\xbe\x3c\x29\x39\x3a\x21\x90\xa5\x0b\xd0\x51\x80\x9f\xd0\x6e\x1a\xb7\x46\x76\x3e\xea\x9c\xcb\xe8\x2e\x1b\x11\x6b\x75\xdf\xeb\x7d\xfe\xfc\x6c\xfe\x6c\x95\x54\x3c\xb5\x7a\xe7\xb3\x67\x18\xf3\x8c\xdc\x92\x21\x70\xb5\x3b\xf8\xeb\xed\x8a\x7b\x61\xbe\x16\xf2\xb7\x94\x6c\x53\x85\x63\xc9\x56\xba\x87\x13\xdc\x83\x57\x80\x22\xe9\x06\xbe\x2d\x0f\xaf\x94\x29\x81\x5e\xac\xed\x6c\xe2\x18\x3b\xb0\x14\x6a\x93\x92\x31\x94\xa5\x25\xcf\x50\xcc\x92\x02\x94\xda\xa1\xbf\xb8\x63\x51\x62\x73\x4a\xca\xc3\x1b\x82\x40\xaa\x23\x29\xca\x70\x8c\xd8\x82\x49\xed\x11\xd5\x8c\xcc\xf8\x86\x54\xff\xc8\xd6\x13\xc1\xd4\x39\x3b\x16\x29\x1f\x95\x52\xd4\xf7\x6b\x3a\x0e\x8d\x1b\x51\xc6\x20\x36\x61\x6c\x60\x97\x14\x45\x2d\x41\x0e\x42\x3a\x5b\x83\x11\x1c\x05\x91\x89\xd3\xdc\xc2\x07\x6b\xe6\x30\x31\x19\x15\xb0\x9d\x8b\xf1\x1b\x3d\xd0\xc1\x03\x82\x08\xf8\x2e\x7b\x09\x98\x02\xcf\xb1\x6f\x6f\x92\x0c\xaa\xa5\xa3\x47\x28\x04\xa0\xf6\x46\xbc\x72\xed\xc9\x95\x6b\xcb\x8c\x37\xfd\x7d\x3e\xdf\x7f\x06\x40\xf4\x99\x42\x34\x93\xae\xae\xb3\x20\x0b\xa0\x2e\x80\x70\xb4\xf1\x6c\x81\x79\x12\x75\xc1\x6b\x29\xf0\xc6\x42\xbd\xac\x11\x6e\xdb\x1e\x17\xcc\x68\xfa\xf9\xf4\xcc\xfa\x67\x71\x93\xf7\x07\xbe\x26\x8d\x55\xa2\x77\x27\xdd\x33\xa7\xca\x9d\xde\xd5\x75\x7c\x6a\x7a\xc9\x82\x6e\xef\x17\x85\x26\xe7\xf1\x6d\xcb\xdc\x6e\x8a\x04\x47\xcb\xd9\xd9\x6a\x99\xd9\xa6\x0b\x6c\xba\x9d\xa5\x95\x65\xb3\xcd\x8e\xbe\x7b\x6f\x21\xb7\xd9\xca\x6e\x5b\x85\x8b\xfe\xdb\xcd\xec\x76\x33\x8c\x83\xc8\x16\x32\xac\x9b\x9d\x8d\xf4\x76\xf3\xe5\x6d\xab\xb3\xd6\x4a\x5b\x49\x3a\xdb\x7c\x39\xe3\xef\xc5\x63\xef\x33\x98\x1e\x74\x74\x98\x1c\x24\x39\xbf\x4d\x61\x81\x4e\x10\x4c\x8f\x83\x60\x37\x9c\x2e\xc0\x80\x5c\x87\xc5\xb5\x05\x72\x0b\x51\xfc\x48\xd9\x81\x40\xd9\x3f\xe4\x7b\x5d\x86\xf9\xdc\x0a\xfa\x7a\x6d\x63\x59\xed\x1e\x08\x74\x63\x0c\x0c\x69\xe6\x8b\xf9\x4b\x7c\xd8\xda\xd6\xaa\x53\x68\x5f\xf1\x35\x2f\xcb\x66\x5c\x9c\x4a\xe5\xf3\x0f\x62\xd1\x4a\x98\x53\x17\xd1\x62\x8f\x54\x62\xca\x73\x72\xe9\x2c\xc3\xc9\x84\x8f\x9c\x3d\x02\x16\x9f\x26\x24\xe8\xe2\xa6\xe7\xc2\xdb\xe6\xa4\xc1\xda\x1e\x5f\xb3\xd1\x41\x5a\x60\xc1\x85\x30\x2e\x2c\x97\x47\x34\xaa\xe9\x52\x58\x11\x04\x9f\x8e\xe3\x13\x57\xaf\x1d\xfe\x0c\x17\x7d\x70\xe1\x79\xf1\x6a\x30\xc9\x49\x84\x0b\x8d\x69\xf1\x97\xba\xce\x75\x76\xd8\x7c\x22\x5b\x92\xe3\x40\x76\x3f\x1c\x8c\x3e\xa5\x2e\x0c\x3b\x92\xfb\x46\x81\x23\x92\xe9\xcf\xca\x51\x30\x9e\x5b\xdb\x91\xb1\xaf\xf0\x1e\xb8\xc7\x0c\x4f\x49\xa8\x0c\x30\x41\x2b\x8a\x13\xeb\x6d\xdf\xc0\xf1\x1a\x95\xd9\xfd\x79\x68\x7c\x71\x31\xcc\xff\x8a\x2a\x89\xdb\xd3\x29\x1c\xcc\xe9\xd3\x90\x8e\x22\xc9\x93\xed\xa0\x67\x8f\xbf\xe4\x24\xa2\x55\x8c\xcb\xca\x51\x1d\x98\x41\x8d\xe9\xa8\xf7\x29\xff\x40\x27\xb6\x72\x0e\x03\x39\x48\xa4\x43\x50\x82\x46\xa1\xd1\xdb\xed\x61\x0f\xea\xb7\x08\x74\xde\xbe\xff\xeb\xd6\xbb\xb7\xaf\x3e\x6c\xbf\xdb\x3a\x3c\x4c\xd5\x77\xd2\xa8\xc0\xad\x09\x47\xaf\x02\x00\x5d\x3c\x00\x56\x09\x60\x61\xef\xa8\xd9\xee\x09\x9b\xad\xc0\x2e\xcc\x2c\x96\x1c\x15\x27\x28\x13\x6f\x3b\x35\xc2\x12\x11\xac\x2e\xe4\x72\xa8\xc3\x67\x93\xa1\x11\x0d\xcf\x6d\xbc\x7a\x87\x9d\x8c\x5c\xa3\x4c\x89\xc7\x6c\xae\x52\x80\xcc\x68\x33\x94\xa9\xec\xa6\x90\x84\x87\xd8\x8c\x86\x47\xec\xf2\xc9\xf7\x05\x02\x36\x9c\xc7\xd9\x86\xe4\xe0\xf5\x64\x30\x9d\x0d\xd8\x03\x02\xc7\x50\xcd\x3e\x28\xa8\x60\x09\xdf\xa4\xe6\x2a\xf6\x16\xd7\x4f\xec\x5a\xe2\x86\xc2\x75\xe3\xfe\xc1\xdb\xc3\xa3\xb7\xef\x77\xa4\xcb\x4c\x61\x9a\x00\xb7\x23\x35\x78\x50\xfe\x95\x55\xda\x96\xdc\xb3\xa5\xf6\xf7\xe3\xd9\x9b\x50\x0f\x51\x32\x70\x9b\x8d\xc8\x7a\x26\xae\x4f\x78\xc1\x5a\x28\x26\xc8\x2d\xd7\x5a\xc7\x41\x64\x37\xb3\x99\x17\x82\x96\xcc\xa9\xa5\x98\x15\x60\x56\x81\xcc\x2c\x50\x74\x4f\x47\x52\x68\x15\xb9\x8c\x65\x69\x79\x94\x3d\x71\xff\x85\x0c\x81\xfa\x99\x76\x34\xfe\x94\x8f\x88\x29\x00\xec\xe0\xd8\x49\x5d\x09\x04\x3d\x2e\x64\x01\x46\xd2\xb8\xc6\x84\xd6\xa7\x6d\xf7\x98\x62\x63\x15\xb8\x40\xaa\x20\x80\x6b\x57\xb6\xeb\x02\x9c\xf1\x84\x65\xc6\xb8\x1f\x17\x4a\x74\x59\xc6\x96\x4a\x25\x94\x11\xfa\x5e\xe3\xc3\x4d\x74\xc3\xf4\x4b\x60\x54\x2d\xd6\xab\x60\x6d\x2d\x58\x14\x28\x18\x5d\x49\x76\x92\xc7\x66\x71\x62\x96\xe0\x4a\x3e\x17\x8a\x73\xb4\xd4\xfd\xc1\x22\x78\x64\x10\x68\x61\xd8\x67\x02\xec\x89\xa0\x01\xcb\xf0\xa7\x4e\xe0\xe7\x9f\xd1\xd9\xf0\x06\x76\xaf\x42\x37\x09\xbb\xc1\xf6\x7c\x96\x35\xe6\x17\xeb\x17\x42\x57\xda\x68\x94\x5a\xd8\xfa\x15\x2d\x64\x1a\x92\xb4\x7f\xf1\xb6\xe4\xdc\x50\x06\xa3\xd0\x2a\x8e\xc2\xc6\x42\x4d\xbc\x57\x44\x91\xb4\x65\xa7\x0a\xaa\x5e\xa0\x26\x9e\xcc\xbc\x7a\x7b\x70\xf4\x93\xa5\x31\x01\x89\xdb\x28\x91\xb8\x56\xb1\x6d\x54\x75\xac\x57\xc1\xbe\xcd\xfc\x08\x3e\xba\xda\xe5\x34\xef\x29\x24\x97\xd6\xbe\x5b\x98\x0f\xd0\x19\x85\x5d\x4c\x2f\xe5\x7e\x07\x86\x23\xe8\x5b\xa4\x80\xa4\x6f\xc9\x06\xcc\xf4\x6e\x6d\xce\x70\x53\xf1\x1e\x89\x9d\xbf\x14\x6d\xbf\xab\xd2\x19\xa6\xdb\xbb\xd0\xc4\x5f\xd2\xf6\x3c\x3f\x82\x85\x8d\xe5\x68\x32\xb8\x02\x4e\xe2\x28\x72\x17\xc3\x19\xde\xeb\x3a\x53\x27\x58\x31\x4c\x83\x81\xd8\x87\x5e\x71\x57\xc5\x53\x2d\x8d\xac\x6a\xe6\x02\x38\xed\x6c\x19\xfb\xb6\x9e\x90\xa7\x01\xe7\x2e\xd9\x8a\xfe\x5b\x39\xff\x24\xb1\xa8\x02\x1c\x89\xf3\x26\xf4\xfc\x53\x9d\x08\xee\xb2\x64\xf3\xbb\x8a\xb6\x4b\x06\xa4\x30\xfc\xf5\x80\x74\xb7\xb3\x92\x9d\x58\x81\x60\x36\xd5\x07\x0c\x5e\x5b\xa3\xf3\x29\xbf\xeb\x8f\x3f\x8f\x94\xec\x9e\xe8\x29\x88\x6c\x08\xeb\x28\x34\xe1\x6b\x1b\x35\x3a\xff\x88\x47\x14\x08\xcc\xe7\xf0\xe7\x65\xeb\x77\xf5\x7a\xeb\x8f\x2f\x5d\x38\x7b\xf1\x3f\xeb\xf5\xef\x36\x5e\x72\x8e\xa0\x1d\x64\x3b\x52\x57\x2c\x5a\x16\xbe\x23\x4c\x45\x07\x5e\xf7\x20\x11\x47\x50\x97\xa4\xc8\xda\x19\xf6\x62\x50\x45\xb2\xe0\x21\x82\x75\xd4\x0f\x3b\x4e\x4f\x15\xce\x33\x98\xee\xa0\x95\xbd\xb8\x30\x0b\x92\x4e\x14\xa5\xc5\x38\xab\xd1\x47\x6f\x37\x7f\x65\x52\x03\x6c\x33\x5f\x1a\x49\x74\x26\xb3\x6e\x5f\x1e\xd1\x85\x22\xe5\xda\x1b\x17\xda\x9f\x2a\x68\x3d\x86\xc8\x70\x1a\x7a\x6b\xba\x20\x83\xc5\x78\x9f\xc6\x71\x9d\xd8\xcf\x3e\x4b\xf4\x22\xa9\x3a\xc2\xeb\x19\x81\x9b\x3e\x9c\x0f\xb6\x18\xef\xd4\x9f\x26\xad\xa4\xc1\x12\xe1\x3b\x34\xff\x40\x4b\x4f\xe3\x3f\x9f\xac\xc7\xcd\xe7\x09\xfc\x3d\xbe\x18\x5c\x75\x9f\x27\xab\xeb\xea\x1e\x8d\x5b\xae\xac\x5b\xd9\x6b\x60\xe3\x6e\x80\x4d\xb1\xdb\x1e\x70\x9c\x47\x1d\x66\xc1\x5a\xfd\xbd\x24\x7d\x14\x82\x1a\x9d\xbd\xd3\x8f\x99\x36\x06\x2e\xd1\x74\xcd\xb0\xe2\xf3\xcc\xe7\x2a\x40\xfd\x2c\xea\xc7\xd6\xf1\x94\x1b\xc8\x28\x81\xc0\x98\x07\x24\x32\x11\x34\x92\x8c\xc6\xa3\x95\x04\xd4\x02\x3e\xcd\x6b\x3d\x6b\x6e\xeb\xf4\x66\x56\xfb\xdc\x9b\xa2\x75\x93\x66\x4d\x54\x70\xd2\xda\xfd\xe6\x22\x72\xa2\x15\xbe\x52\x33\x9d\xf5\xc8\xe2\xf1\x51\xef\xc2\x1e\x5f\xfc\x15\x75\xb1\x33\xb0\x80\x55\x91\xe4\x01\xe6\x0b\xc8\x99\x33\xa9\x55\xec\x24\x4b\xdf\xd8\x75\xf7\xb2\x4c\x89\x91\x69\xbb\x3b\x18\xc9\x65\x13\x3f\x77\xd8\x20\xdd\xc1\x95\xf2\x18\x48\x67\xbb\xf7\x8f\x0d\xeb\x4a\xd5\x04\xaf\xd7\xf9\x2a\x53\x9e\x24\x5d\x65\x95\x93\xdc\xa5\x16\xa7\x6f\xe5\x54\xd7\xb9\xdd\xa2\x58\xb4\x2b\xfa\xa2\xdc\x84\x64\x49\xd7\x56\xe4\x24\xe3\x4b\xd2\x2b\xbd\x2f\x41\xcf\xd9\x60\xd0\x73\x2e\x0f\x2c\x8c\x2f\xff\xb6\x9e\x7b\xe9\x2a\xab\xee\x39\x9b\xfa\xb4\x9e\x53\xb9\x1f\xe9\xb9\x52\x13\x96\xf6\x5c\x39\x67\x49\x9c\xea\x9b\x98\x83\x01\x8b\xc2\x7e\x4b\xd9\xaa\x75\x51\x45\x50\xd8\x3e\x6d\xd5\x20\x38\xdb\x8e\x18\x37\x9f\xcb\x1b\xc8\xc1\xce\x9f\x76\xfe\xbe\xff\x28\xcd\xb6\x42\xe0\xd8\xc7\x64\xfc\x86\xf2\x76\x70\x6b\x4e\x09\xa0\x98\xcd\x57\x6f\x59\xc2\xb0\x54\x8e\x9a\x03\x17\x52\xf6\x65\x03\xb2\x84\xf6\x56\xb5\x12\x37\xc2\x28\x6a\x50\x60\x21\x74\x02\x56\x82\xa3\x0f\x99\xc2\xd0\x25\x22\x61\xf8\xc6\x89\x0d\x25\x5f\x72\x5f\x2c\x59\xd7\x4f\x5d\xff\x4f\x59\xf9\x4f\x59\xf3\x15\xab\x1d\xe6\xb3\x5b\xe5\xe5\xf6\x43\x22\x2e\xef\x5f\xb3\xb0\x71\x45\x2f\x5b\xcb\x4f\x5d\xc5\x4f\x59\xbf\x4f\x59\xb9\xaa\xfd\xbf\x7a\x32\xcd\xe7\xee\xb9\xef\x89\x6b\xe3\x01\x76\x66\xd9\x9c\x0f\x95\x91\xbe\x85\xac\x7c\x2b\x4d\x21\xfe\x10\x2a\x7d\x94\xc9\xa9\xee\x9b\x1f\x0f\xde\x3d\x95\x7c\x90\xb9\xa7\x47\xfa\x87\xf3\x04\x9d\xf3\xdf\xed\x44\x28\x8e\xa1\x46\xba\x92\x34\x06\x19\x94\x65\xda\xab\xde\xe0\x3f\xa1\x8f\xa9\xda\x6f\xec\xe5\x9d\xdd\xad\xb7\x4f\xee\x67\xaa\xe8\xd1\x9e\xb6\xb9\x0a\x6c\x75\x75\x57\x87\xb8\x57\x76\x76\x21\x8b\x7a\xfa\xea\xf5\x07\xe3\x47\x7b\x0d\xf6\x45\x25\xb6\xc9\xbc\x07\xca\x0e\xf9\xd3\x13\xc6\xc1\x02\x82\xc8\xc8\x28\xf7\x88\xc1\xb1\x6c\x38\x38\xfb\x14\x99\xf2\xe1\x09\x8d\xfb\x90\x9c\x39\xda\xee\xf9\x8a\x33\x32\xe1\xe1\x0e\xca\xb6\xdd\xe5\x33\x9a\x92\x70\x77\x25\xda\xe5\xba\x79\xc3\xcc\xb2\xc2\x21\x6d\xc1\x3b\xd4\xea\xf8\x14\xba\xf3\x36\x8f\x23\xca\x16\x05\xb5\xa9\x0e\x25\x60\xa7\xe3\x2f\x4f\xe8\x53\xb1\xd4\x95\xff\xd5\xe3\x86\x97\x0b\x12\x61\xce\x7b\x70\x7c\x0f\xd2\x5e\xbb\x98\xb6\x7b\xc8\x72\x00\x12\xb2\x9b\x60\xa1\xe1\xfc\x72\x79\x3c\x24\xcc\xa4\xe0\xd2\xcd\xe0\x83\x43\xf4\x15\xc3\x51\xee\xd1\x87\x87\xa5\x62\x04\x4a\x5d\x1f\x2c\xb8\x07\x3d\xa3\xaf\x90\xb5\xc6\x3c\x28\xf7\xc4\x7d\x45\xc6\x5d\x95\x7f\x12\x2f\xa7\x4b\x77\x5c\xd9\xd4\x77\xaf\xde\x38\xce\xf0\xaa\xcc\xbf\x79\xb1\xa4\x5f\x3e\x0c\x35\x19\xe9\x29\x03\x4e\x88\x74\xaf\xc6\x32\x7a\xa6\x64\x82\xe9\xde\x3e\x66\xa5\xf4\x96\x45\x0f\x61\xde\xbe\x50\x69\xae\xa9\x37\x2b\x01\x4c\x6e\x70\xc4\xe7\x19\x3b\x26\x4a\xee\x63\x8b\x0b\x74\xc3\xca\xc6\x7c\x2e\xc3\x4e\x52\xee\xbf\xdd\x84\x48\x87\xab\x73\xc1\x56\xaf\xaf\x88\xd6\x5a\xe0\xff\x11\x32\xe8\x2b\x43\x89\x34\xee\xb6\x50\x6a\x4c\x24\x3b\xdb\xa0\x92\x48\xfd\x22\xef\x41\x50\xd7\x11\x96\xbc\x10\x75\xda\xf9\x10\xc7\x77\x44\x81\x5c\x65\x0d\x0c\x53\xf7\xab\xe1\xb8\x84\x27\x00\x59\x96\xe5\xde\xfb\x75\x54\xe5\xec\x6e\x21\x11\x22\x86\x5b\x8b\x92\xc0\x39\xa4\xcf\xcf\xb2\x0c\x0c\x5b\x99\xc6\x76\xf7\xa4\x92\x51\x49\xfe\x18\xf4\x22\x5f\xaf\x4b\x19\x9e\x9f\x9f\x68\xb1\x49\x8c\xab\x72\x51\x40\x46\xf9\xb0\xf3\x26\xd3\xda\x32\xde\xec\x5c\x0e\xbb\x87\x6d\xf8\x9b\xf2\x94\xa1\x9d\xab\x40\x11\x09\x76\x40\x36\x2a\xa6\x5a\xe0\x49\xcd\xd5\x40\xb6\xb9\xdd\x94\x47\xfd\x0a\x7a\x58\x0a\xf0\x89\x78\x1e\x86\x4a\xc0\x10\x81\x53\x88\xbf\x84\x77\x1e\xf7\xb3\x56\x9d\x63\xda\x10\x40\xad\x3c\x9b\xa3\x8e\xe2\x12\x98\xc1\x4f\xe5\x8e\x9b\x00\xcb\x50\x4b\xf5\x54\x5b\x96\x89\x6e\xa5\x59\x6d\xd2\xa9\xe8\x7a\x4a\x65\x45\x3d\x8b\xf2\x07\x12\xdf\x11\x79\xcf\x40\x5c\x23\x15\xe1\x40\xaf\x8a\xfb\x0d\xf0\x94\xbd\x21\x07\xef\xaa\x37\x82\x05\xfb\xae\x02\xcb\x0a\x80\x00\xcb\xcb\x7e\x91\x59\xa6\xf5\x0b\x75\x53\x7f\xe9\x8a\x70\x81\x26\x1a\x04\xdd\xbe\xec\x4d\xf0\x32\x37\x7e\xb1\x39\x3f\xbb\x24\x31\x08\x0c\xb2\xc4\xc4\x02\x10\xb1\x38\xfc\x58\xd1\xb2\x47\x71\xe8\xad\xfd\xeb\x6b\x70\x58\x7b\xf1\xa2\x5e\x8d\x44\x3b\x1a\xe0\x54\x8b\xde\x46\x61\xc7\x93\xfb\x38\xdb\x39\x85\xce\x52\x83\x51\x68\x02\x33\x90\x68\xe4\xc2\x7c\xfc\xe5\xdd\x60\x06\x3f\xff\x71\x93\x4f\xee\x8c\x28\xf7\xee\x8e\xfb\x37\xc3\xdc\x8c\x00\x07\xb4\x34\xf6\xc1\x90\xc4\x18\xac\x73\x16\x1d\x33\xb8\x78\x31\x84\xbf\xc6\x59\x78\x66\x22\x51\x61\xfb\xd9\xd0\x7d\x99\xf6\x65\x3f\xba\x40\xc7\xc7\x5c\x59\x26\xa6\x05\xac\x55\x40\x13\x06\xd1\x01\x5e\x10\x91\xdd\x2f\x80\x16\xdf\x0c\xfa\xd9\x71\xb4\x11\x19\xf9\xd7\x6d\x63\x73\xc4\xa0\x18\x7e\x2e\xb1\x26\xe6\x6c\x80\x55\xdb\x0f\x63\x55\x2e\x32\x40\x84\xab\xcf\x83\x84\x86\xa0\x60\xce\xc9\x7a\xf3\x79\xbb\x36\xb9\x4d\xbf\x05\x7a\x62\xd0\x8d\x72\x53\xf4\xfc\xe9\xd5\x4f\x84\x7d\x54\x9c\x58\x43\x1d\x5c\x55\x98\x32\x94\x39\xa3\x7a\x18\xf2\x75\x96\xec\xf2\x6e\x4e\x0a\x6b\xcf\xb7\x65\xf4\xec\x23\xf2\x93\x8b\xf4\xab\x8a\x2a\x73\x1f\xd3\xe7\xeb\xa1\xa4\x25\xc4\x88\x8d\x11\x86\x8c\xea\xa1\x6e\xfa\x64\x7f\x7c\x89\x3d\xe9\x11\xb5\x77\xbc\xce\x44\x23\x87\xdd\x2b\x96\x2d\xd9\x91\x88\xd4\x33\x5e\x8e\xf9\x23\xe2\xf6\x9e\xad\x7a\xa0\xbd\x49\x58\x1e\xa5\xa4\x8e\xd7\x27\x28\xa5\xa1\x40\x7f\xa3\x58\x61\x92\x16\x63\x54\x1f\x2d\xc1\xba\x84\xad\x8b\x50\x68\x7b\x78\x34\xbe\x87\xef\xb7\xfe\xb2\xf3\x61\x7b\xeb\x70\x47\x0e\x61\x99\xa3\x57\x62\x9f\x33\x03\x26\x68\x38\x4c\xa3\x56\x73\xb3\xb9\xb1\x36\x39\x6b\xbe\x40\x43\xc4\x1f\xc7\x93\xb4\x85\x17\x4b\xf0\xbb\x69\xfa\xe3\x59\x8a\x32\x04\x52\x5b\x04\x67\xc4\xf1\xd9\x60\x7c\x33\x5d\x9b\x91\xe2\x74\xb4\x80\xe5\xbd\x8d\x5e\x05\xb3\x3f\xff\x07\x2e\xf3\xe6\x19\x05\xee\x31\x9e\x4c\x23\x4a\x74\xfe\xe5\xba\x07\x4b\x0c\x19\xb9\xb5\xa8\x11\x5b\xc1\xe3\xc4\x6a\x47\xc4\x09\x14\x78\x0b\x5b\x15\x32\x45\xf4\x1a\x65\x9f\x99\xd0\x29\x37\xaf\x4e\x67\xf7\xaa\x98\xa5\x34\xec\x6c\xed\x1c\xdd\x9d\x29\x61\x94\xa0\x48\x2c\x39\xf0\x34\x59\x1e\x80\x8a\xf2\xb3\x19\x34\x4c\x9e\xc9\xd0\x98\x97\xcd\xb1\x30\xbc\x1f\x3e\x86\x73\x45\xae\x27\xa0\x5d\x51\xea\x6b\x31\xef\xe7\x4b\x31\x3f\xdc\xdf\xd9\x7e\xbb\xf5\xee\xc3\xf6\x9b\xad\x83\x43\x37\x4d\xe2\xe3\x93\xf4\x64\xed\xe4\x43\xb7\x11\x37\x13\x14\x78\xde\xdd\xfb\xc7\x87\x37\x5b\xdb\x7f\x71\x39\x7e\xbe\x1a\xff\x2b\xb6\x46\x09\x99\xbc\x17\xa8\x2f\x47\x02\x05\xe6\xe1\xdf\x77\x7e\x01\x64\x3a\x78\x47\x01\x30\xd5\x7b\xfd\xbb\xd0\x47\x9d\x7f\xce\x9e\x0c\x2e\x2e\x48\xe8\x99\xbc\x53\xe2\x39\x0d\x7f\xe9\xbd\x9c\x2c\x67\xb1\x3e\xf7\x84\xdd\x25\x44\x28\x64\x87\xc2\x35\x78\xd1\xab\x3a\xbe\x87\x66\xf6\x60\xb2\x75\xa6\x3c\xd7\xd0\x70\x98\x40\x4e\x52\xf6\xea\x86\xe7\xbc\x57\x7b\xbb\xdb\x6c\x53\xfe\xdd\xb8\xd7\xcf\xfb\x91\xb1\x99\xa4\x15\xd6\x40\x0d\xe5\x46\xdb\xdf\x3e\x07\x74\xa6\xdd\x8a\x8a\xfe\x84\xf9\x20\x55\x66\x60\x49\x36\xcc\x0f\x1f\x89\xc4\x59\xe1\x89\x08\xe8\x06\xf1\x80\xc7\x72\x25\x2c\xac\xab\x01\xde\xb5\x11\x75\x61\xd5\xe5\xbf\xa4\x4a\x00\x9c\xb8\x3d\x27\xd5\x0d\x21\x14\xcf\xe6\x31\xa0\x7a\xd8\xb1\x5c\x37\x49\x75\x1c\x35\x5c\x04\xa1\x25\x7d\x21\xbe\x9f\x60\xcd\x23\x1e\x29\xed\xbe\x28\xd1\x98\xe2\xc6\x8c\x06\x90\xd8\x57\x14\x05\xe9\x6b\x61\x7e\xd8\xdb\x7b\xb7\xb3\xf5\xfe\xc3\xd6\xd1\xd1\x01\x1a\xd6\xb1\xed\x8b\xae\x6e\x86\xb3\x01\x8c\x87\x1c\xe9\xf2\xbe\x91\x13\xad\xe9\x0f\xa6\x64\xbe\xd6\xe0\xd8\xec\x8d\x86\x77\x86\x8d\xf8\x43\x0c\x90\xce\x51\x24\x8a\xa4\xb0\xeb\x26\x25\xc1\x7e\x5d\xdd\xb1\xdf\x16\x39\xb5\x9b\xc9\x05\x3a\x91\x40\x9b\x75\xe7\xdd\xce\xee\xce\xfb\xa3\xc3\x00\x3b\x96\x9a\x67\xd4\xa0\x56\xe2\xe2\xf1\xb2\xac\x07\x38\x99\xd3\x9b\xd9\x0c\xc2\x78\x54\x36\xb8\x7e\x06\xc3\xe9\x93\x90\xb2\x35\x1d\xfb\x5d\xc1\x22\xb6\xb2\xb1\x48\x9c\x90\xc7\x7d\xbf\x37\xeb\xa5\x3c\xab\x80\x0c\xf6\xcc\x60\x74\x99\x4f\x20\xd0\x7f\xe5\x13\xde\xea\x38\xf6\xdb\xb7\x7c\xa7\xa8\x28\xe2\x48\x02\x9a\xa2\x46\xab\x7c\x89\x93\x14\x1c\xe6\x13\xa9\xc4\x4b\x75\x1a\x6b\x1b\xe8\x5b\x2b\x71\x76\x89\x1c\x4d\x44\x3b\x64\x65\x0a\xc5\x16\x89\x42\x0a\xe7\x2d\x96\x51\xea\x02\x15\xb8\xe9\x84\x21\x68\xbe\x91\xa0\x39\x83\xa8\x4a\x88\xf6\xa5\x5f\x1c\xf0\x65\xde\xac\x2d\x41\x24\xd5\x16\xb9\xaa\x93\x17\x1e\xbb\x3a\xed\xc2\xfc\xc3\xcb\x4c\x8c\xb2\x41\xa4\xdb\xe9\xad\xa7\x37\x36\x27\x5a\x15\x2b\xa7\x42\xfb\xbc\x24\xe5\xa3\xde\xcd\x6c\x1c\x25\x74\xa1\x89\xd1\xf3\xb9\xe3\x0f\x54\xa1\xb0\x4e\x0b\xa2\xe3\xce\xa0\xc8\xeb\x10\x88\x45\x45\x69\x99\xec\x74\xf0\x7c\xb8\x53\x82\x73\x58\xbf\xd2\x64\x31\xf4\x5b\xf5\xd2\xa2\xec\x5d\xab\x59\x52\xe8\xc3\x90\x4f\xb1\x3e\xee\x62\x7d\x01\x4a\xc3\x3a\xc5\x7d\x1e\x01\xf5\xdf\xce\xf2\x2b\xae\x72\x3e\x47\x5e\x15\xb5\xc6\xf3\xb3\xc1\xf9\x20\xef\x77\xc2\x3a\x53\x7f\x12\x97\x97\xaa\xa0\x1a\xdc\x04\x5c\xa7\x68\x9b\x77\xec\xf7\x33\x00\x85\xcf\x79\x85\xc2\x2d\xb3\x6c\x06\x16\x8b\xb2\x78\x1d\x5d\x52\x14\x5a\xbf\xbc\x7a\xce\xd0\xb6\x25\x6d\xc6\x0b\x95\x91\x47\x05\x3a\x30\xab\x4a\x65\x30\x9b\xde\x6d\x09\x3b\xfa\x80\x90\x9a\x1c\x10\x5a\xa0\x87\xcd\xf1\xf5\xc3\xc3\xef\x78\xee\x10\xff\x62\x87\x52\xac\xb9\x45\x3b\xc9\x1b\x8e\x4d\xe5\xb4\x05\xd1\x46\xbd\xb5\xb9\x3d\x1a\x19\x38\x48\x73\x15\xab\x29\x87\x65\x50\x4e\x37\x7b\xbf\xf7\x6a\xe7\xc3\xd1\x4f\xfb\xf0\x67\xe7\xef\x47\x1f\xf6\x0f\xf6\xf6\x77\x0e\x8e\x7e\x3a\xd6\xec\x2c\x5e\x0b\x2b\x07\x68\xfe\x66\x5d\x90\xb5\xc0\x2c\x0f\x7c\x6c\x23\xba\x69\x14\xf9\xe1\x75\xb1\x61\x73\x88\x53\x58\x82\x86\xda\x9c\xe5\x18\x11\x2f\x43\xb8\xd5\xcd\x60\xe7\x00\x2e\xec\x88\xdc\xf3\x2c\xcb\xf6\x02\xb2\x61\x9b\xe8\xc6\x13\x4e\x2c\x0f\x80\x7b\x08\x04\xb6\x45\x58\x92\xc8\x48\x3f\x37\x57\xfb\xb7\xa8\x79\x28\xc1\x45\x4c\xe4\xa1\x3c\xfe\x9e\x1c\x96\xfb\x92\xad\xfc\x1c\xc2\x6e\xb5\x7d\x84\x24\xc7\x1d\xa3\x1c\xc5\xf7\x94\xce\xee\xe2\x76\xba\x4e\x21\x5c\xc1\xcd\xb8\x0b\x73\xda\x49\x15\x63\xc3\x11\xc9\x3d\xff\x36\x2d\x2f\x80\x17\xa2\x08\x8a\xf9\x1d\x49\x24\xec\xe6\x73\x09\xb1\xcd\x2f\x56\x2b\x97\xcc\x22\x7e\x41\xcf\xf7\x1c\xb5\x28\x9c\x94\x98\x24\x06\x21\x4b\x26\xd1\x97\xc0\x57\xf6\x53\x01\x38\x8d\x3c\x9e\x04\xb5\x01\x09\x43\x1e\x31\xd1\x50\xe3\xd4\x6f\x1a\x2e\xaa\x3d\xf8\xde\x07\xb4\x46\x89\x6c\xfe\x79\x6f\x38\x3e\x8b\x7d\x16\x52\x2a\x29\xd5\x27\x4d\x58\x28\x0b\x85\x23\xd9\x47\x8b\x0c\xb5\xac\x65\xa5\x76\x76\xd1\x42\xaf\xa6\x9b\xd6\xd3\x1a\xda\x6b\x82\xd6\xc6\x9b\x59\x76\x3e\x72\x5a\x74\xe7\xa3\x95\x2c\x0b\xf7\xda\x20\xd2\xf3\x09\x1d\x84\x98\x12\x44\xed\x5c\x28\xb4\xb9\x88\x79\x12\x56\x5b\x63\x8d\x1a\xc5\x6c\x5a\x9b\x5d\xe7\xa3\xcc\x42\x47\x4e\x22\x01\x8e\x9e\x79\x55\xf2\xc2\xda\x12\xea\x89\x20\xc4\x66\x16\xc5\xaa\x5c\x68\x83\x11\xe3\xd8\x22\x95\xd3\x0a\x83\x54\x67\xeb\x89\x27\x00\xb4\x13\x56\x8d\xf9\xf8\xd1\xbe\x66\x39\xac\x3b\xce\xa0\x8a\x42\xd0\xa0\x16\x98\x0a\x92\x91\xa8\x8f\x2f\x3f\x7a\xd3\x50\x6e\x6d\x67\x16\x9d\x8f\x8c\x34\xf7\x74\x5b\xcd\x3b\xbe\xd9\x68\xb8\x12\xa9\xfb\x5a\x04\xae\x18\x96\x77\x55\xa1\x5f\xa4\x0a\xdd\x5a\xcd\x59\xf2\x6e\xa6\xd8\xc8\x03\x17\x01\xac\x2c\xce\xb6\x54\x4f\x3d\xa3\x3c\x38\xd7\xc6\xa3\xd7\xa5\x83\xa4\xb9\x19\x89\xa7\x0f\x37\xc6\x76\x17\xd1\x29\x22\xd0\xa7\x0f\x83\x78\xda\x04\x5c\xd1\x9f\x08\x47\xff\x06\x37\x8e\xb2\x03\x11\x74\xed\xf1\x4f\x7b\xff\xfc\x4f\x74\xa8\xf9\x4f\xf6\xf4\x0b\x28\xff\x13\xcd\x86\xc0\x84\xc6\xe7\xaa\x48\x1e\xa5\x31\x6d\x2a\x13\x67\x87\xaf\x15\x0e\xa1\x64\xee\x59\x51\xce\x12\xa1\x55\x70\x34\x99\xfc\x60\x5e\xce\x62\xcd\x76\x4f\xe7\xf3\x27\x00\x36\x82\xc3\xfd\xc2\x56\xf1\x70\x31\xa9\xc3\xa2\x23\x56\xc3\x11\x86\xb3\xe9\x2c\x59\x19\x70\xe2\xc7\x93\xee\xe0\xe4\xe4\x51\xd3\x27\x0f\x72\xbf\x75\xef\x3a\xe4\xf5\x68\xca\xa2\xc3\x53\xf6\xc5\x45\x12\x98\x36\x85\xe9\xfd\xd5\xf8\x66\x9a\xe7\xe8\xad\x45\xac\x14\xa2\x8e\x3a\xc6\x0d\xf3\xde\x6d\x2e\x71\xce\xb9\x1d\x1c\x7a\x00\x64\x68\x24\xde\x46\x2b\x3b\xec\x12\x4f\xfa\x76\xaf\x24\x72\x7f\xcc\x36\xc5\x0b\x2a\xa0\xec\xf3\x74\xfc\x79\x94\xfd\x11\x16\x7e\xcf\x6d\xfa\x9d\x9e\xbb\x25\xb1\x62\x9c\x78\x02\xbb\xce\x4e\xeb\xf5\x53\x51\x95\x42\xda\xe8\x74\xed\xa0\x34\x24\xcf\xe7\x2b\xf1\x0a\xfd\xa2\x05\x2a\xf8\x70\x00\x31\x85\x2a\x72\x08\x77\xc2\x60\x0c\xb9\x93\xb4\xb7\x0c\xef\x7a\xbd\xf5\xfb\xfa\xd2\x54\x2a\x9c\xe8\xab\x17\x6a\x1d\x7a\x54\x60\x47\xc7\xa7\x59\x80\x35\x92\xba\x53\x6c\x70\xd9\x20\xe1\xa2\xad\x87\xcc\xde\xcc\x52\xdc\x55\xef\x3a\xbb\xf7\xa3\x93\xf2\x48\x8d\x51\x08\xdb\x8f\xa3\x8d\xbd\x85\x21\x5d\xb4\x83\xd5\x6b\x81\x30\xe8\x4a\x69\x73\xed\x9b\x7b\x92\x0f\xd1\xb6\x8e\x48\x9f\x4b\xe8\x88\x32\xb4\xe3\x15\x09\xa3\xa4\x34\x7d\x68\x1f\xce\xae\x4f\xc5\x4d\xb7\x64\x41\x5d\x50\x9e\xee\x5c\x25\x3b\xe1\x5b\x08\x33\x5d\xbe\xe3\x0b\xc9\x0e\x97\x4c\x4c\xb1\x7b\x2a\xa7\xfa\xc2\xc6\xca\x8b\xf4\x08\xab\x41\x73\x38\x42\xe8\xf6\xce\xcf\x8d\xdc\x22\xff\x6d\x30\xbb\x2c\xb3\x00\x92\x88\xa3\x25\x9b\x24\x3d\xa4\xf1\x18\x16\x74\xf6\x68\x48\x83\xbd\xdb\xb2\x4c\x6e\xc9\xe2\xdd\xa6\x5c\x18\x69\xc8\x7e\x0c\x46\x54\x11\x55\xd2\x61\xa8\x4d\x74\x2e\x37\x99\xfd\x90\x9f\x23\xdd\xc0\x74\x43\xc9\x4d\xed\x11\x3b\x95\xbc\x02\x74\x1b\x79\x06\xce\xeb\x50\xa0\x42\xc4\xce\x61\x17\x10\x57\x01\x45\x2a\x0e\xf4\xb4\xc8\x25\xf9\x01\x6e\xce\xf3\x25\xa6\x0c\x03\x55\x13\x8a\x0c\x7c\xbd\x6e\xa1\x8a\x88\x8d\x64\xc6\x47\x58\x49\xe0\x4b\x08\x1c\xbe\xc7\xef\xbd\x7d\xfd\xf8\xc2\x01\x27\xdd\xeb\xeb\x7c\xd4\xaf\x38\xec\x50\x8f\x56\x0c\xc0\x28\xec\x79\x82\x97\xdc\xc7\x55\xa8\x03\x1d\xa9\x8a\xd6\xe2\x3c\x54\x3b\xf7\x3b\x43\xc2\x5e\xbe\x86\xf1\x78\x00\x29\xb6\x81\x5b\x06\xeb\x27\x9a\x4b\x3b\x1f\x4c\xa6\x33\x02\xdf\x7e\x72\x53\x3c\xab\xa8\x26\x10\xa5\xf1\x64\xa0\x87\x58\x83\x7e\x14\xcb\x08\x62\x2c\x4f\x7a\xfb\x95\xc9\xfd\xa0\x4b\x41\x83\x6d\x62\x4f\x7c\xc9\x5a\xe0\x4f\x32\x03\x5e\x9a\x9c\x16\x8c\x9f\xa0\x36\x26\xe8\x4b\x37\x49\xe4\xda\xa8\x62\x5e\x54\xae\xb8\xaf\xc5\x8c\x9e\xad\x0b\x35\xf6\xce\x91\x8e\x96\xc7\x2e\xff\xbc\xa3\xd7\x4a\x30\x50\x0f\x50\x86\xaa\x71\xf3\xa0\x8a\x24\xe0\x2b\x16\x7f\x61\x69\xdb\x57\x7a\x21\x71\x5b\x12\x34\xea\x6d\x3e\x60\xff\x38\x95\x55\xfe\x38\xb5\xd4\x64\xcb\x81\xe1\xf2\xec\xb3\xdf\x98\x40\xc0\xcd\x47\xa3\xf5\x50\x1b\xc8\x56\xc2\x43\x43\x09\x1c\xf0\x32\x3e\x77\x27\x44\xb7\x8c\x62\x52\x2e\xbf\x90\xee\x5e\x42\xc5\x96\x8f\x85\xb7\x3b\x47\xd3\xa0\x85\x6c\x82\xf4\xb8\x63\x3d\x04\x34\xa9\x6a\x91\x88\x5e\x45\x2d\xea\xfe\x06\x73\xc8\x60\xda\x81\x29\xbe\xd4\x95\x72\xb8\xb3\x62\x3e\xf4\x0a\xe0\x6a\x68\xdb\x58\x39\xbe\x2c\x5e\x01\x8e\x2b\x48\x29\xae\x1c\x7e\xed\x84\x0b\x5d\x05\x05\x5c\x95\x57\x70\x20\x1c\x54\x91\x9e\x92\x10\x95\xba\x62\x12\xec\xa6\x3f\xdc\x1d\xf5\x2e\xf0\xd4\x1f\xab\xae\x3e\x1b\x8e\x47\xb9\xbd\x1a\xc6\x6f\xfb\xc2\x21\x8c\x6a\xb9\x2a\xda\x7c\x11\x8c\x71\x4c\x7b\x81\x2d\x8d\x9f\xc2\xaa\x93\xbf\xac\x63\x07\x4c\xb6\x7a\x84\x96\xb9\x2f\xdc\x03\x3c\x87\x94\x1d\xdf\x5f\x4f\x38\x4d\xac\xfd\xe2\x75\xa2\x99\xce\xc6\xd7\x78\xff\xd3\xbb\x20\x05\x55\x8a\x5c\x74\xdd\xc2\xb4\x58\x99\xf0\xc1\x69\xd4\x64\x61\xbe\xa0\x55\x4d\x31\x5a\xe4\x1b\xc6\xd4\xf4\x57\x1c\xc2\xf1\xcf\x0b\x6f\xce\x85\xaf\xa8\xaa\x8f\x7d\xa5\xa3\x6a\x7c\x1b\x9c\x38\xf5\x71\x90\xc1\x1a\x2b\xf3\xa7\xbc\x3e\x4a\x21\xa1\xeb\x72\xad\x91\x78\xb2\x41\x9b\xab\x18\xe3\x5b\x0e\x59\x6b\x4a\x85\x48\x61\x01\x79\x90\x37\xc5\x5e\x68\x9e\xc2\xe4\x2c\xbd\xf8\x35\xc7\xa3\x72\xce\x9b\xd1\x92\xbc\xe7\xe7\xc0\x3a\x00\x71\xb9\xdc\xed\x5d\xeb\x57\xc3\xeb\x1b\xb5\x54\xf1\x22\x40\xee\x71\xa8\x01\xe8\xf8\xe3\x2f\xf9\x1d\xc6\xbb\x27\x21\xbc\x30\x0b\x4a\x04\x4e\xc2\xc3\x22\xe5\x8d\xe8\x53\x7e\xa7\x5f\xf3\xa8\x08\x6a\x6b\x06\xc5\x6c\x27\x89\x86\xbf\xcd\xd4\x35\x72\x7b\x43\x73\xf7\xf5\xfb\x0f\x5b\x07\x7f\x3a\xcc\xd6\x7f\xb6\xc0\x4f\xa6\xcf\x8f\x7f\x3e\x89\xbb\xcf\x4f\x62\xf8\x8c\xe1\x3b\xe9\x3e\x4f\x4e\x92\xf5\x2b\xc3\x99\xd9\xb8\x56\xb6\x6e\xd6\x25\x22\x23\xf9\x88\xf8\x03\x5a\xcf\x3d\x6c\x74\x92\x93\x16\x0b\x47\xa0\x61\xb0\xfd\x0f\xdb\x7b\xbb\xfc\xdc\xb5\x1e\xc7\x27\xeb\x28\x4e\xb2\x9a\xcc\xf1\xeb\xf9\xf1\xc9\xf4\xe4\xb0\xfb\xbc\x73\xf2\xfc\x64\x1d\x5f\x76\xaf\x8c\x7b\xb4\x29\xbc\xdf\xaa\xc7\x1c\xb3\xca\xee\xa0\x8a\x4f\xbc\x36\x1a\x73\xb0\xc7\x30\xe7\x7e\x95\x5c\xa2\xf2\x77\xe0\xf9\xd4\x7a\xef\x10\xb7\xec\x96\xee\xe0\x41\xda\x84\x9e\x2f\xfc\x46\x19\xb8\xf5\xb0\x4a\xb2\xe4\xb9\x22\x5a\xf3\x3e\xd7\xda\xf2\xec\x53\xaf\x47\xcd\x68\x25\x0b\x7d\xad\x58\x5f\xa7\x41\x3b\x50\xcf\x70\x76\x06\x28\x38\x35\x43\x74\x41\x4a\x42\x7b\x35\x8b\x58\xcd\x6a\x0d\xd6\x3e\xc3\xc1\xa1\xf6\xac\xf9\xac\x76\x31\x9e\xd5\x9e\xdd\x6f\x2c\x9e\x35\x23\x5e\xfb\xed\x62\x63\x68\xed\x7b\x63\xfe\x5d\x54\xce\x35\x25\x8f\x20\xe4\x3f\xc4\x7a\x02\x09\xfd\x7b\x88\xfe\xb0\xee\x39\x89\x72\xd2\xae\x7c\xfa\x2b\xd1\x62\xde\xcd\x98\xa7\x31\x7d\xa0\xde\x72\xe8\xc6\x30\xb1\x77\xf4\x55\xaf\xd3\xcf\x31\xfd\xb5\xc6\x46\x5b\x5d\xe3\xb7\xce\x2c\x64\x9d\xc8\x51\xa1\x03\x22\xc5\xc5\x33\x8d\x2d\xc2\x95\xbe\xf7\xbb\x54\x56\x59\x40\x6d\x63\x6c\xe8\xa4\x5d\x38\x72\x54\xb3\x48\x54\xb6\xcc\x26\x15\xeb\xc4\x63\x06\x36\xbb\x5e\xb7\x3d\x16\x63\xd0\x6c\x90\xac\x84\xe1\x93\x74\xa9\xd7\xb8\xa3\xc2\xd7\xa0\xf8\x41\x40\x95\x4c\x6a\x55\xef\x8b\xd7\xe1\x99\xba\xe1\x29\xe7\x52\xdc\x5c\x09\x26\xcd\x47\xda\x5d\x19\xa0\x0b\x7b\xd7\x3e\x2e\x2a\xe9\xb8\xcf\x34\x10\xbf\x2d\x24\x7a\x31\xd8\x94\x4d\x7d\x57\xf7\x7f\x81\xf7\xb6\x3c\x5b\x19\xb5\x47\x7a\x5d\xb3\xa4\xff\x1f\x6e\x9f\x62\x42\xbf\xba\x89\xe2\x8d\x97\x79\x8d\x45\x77\xd1\x35\xab\x78\x79\x34\x18\x96\x68\xa6\x44\x47\x49\x7b\x75\x9b\x3f\x2d\xcd\xf4\xf2\x79\x9e\x76\x32\xb7\xb3\x7f\xb0\xf3\xfa\xed\xdf\xbd\xf0\x4d\xfc\x85\x04\x74\x3e\x74\xe7\x28\x45\x20\xdf\xc9\xfa\xc0\xfc\xfd\xcd\x81\x15\x3b\xfa\xfb\xee\xbb\x37\xb3\xd9\xf5\x01\xfb\x74\x9a\xcf\xb5\x55\x83\xc9\x9d\x13\xad\x87\x53\x0a\xbb\x0c\xfc\xbb\xdc\xdb\x47\xbb\xd3\x2f\x57\xc3\x4d\x2a\x7f\x74\xb4\xdf\xfc\x7d\x73\x23\x72\xfe\xe9\x5a\xc9\xfd\xe2\xab\x4a\xbf\xd0\xa5\x37\xbf\xb2\xb4\x2f\xf9\x02\x4b\x06\xfa\xe2\xab\xe8\xea\xf9\x87\xde\xd9\x27\x38\x43\xb2\xca\xf8\x97\x4b\x74\x35\x78\x44\x0e\xa5\x59\x85\xa9\x7c\xd1\x1c\x76\x0a\xfa\x7c\xc6\xfd\x0f\xd6\xe1\xf5\x78\x58\xb1\xbf\xa9\x24\xd8\xe3\xf6\xb7\x8e\xde\x7c\xd8\xdd\x3a\xda\x7e\x83\x43\x00\x9b\x74\xe7\x37\xb0\x4b\xc7\x27\x1d\xf8\xc6\xaf\xa4\x13\xff\x06\x75\xf7\xd1\x90\xa8\x79\xb5\xf3\x7a\xeb\xc7\x77\x47\x1f\xf6\xf7\x0e\x50\xec\x84\x1c\x53\xff\x61\xc3\xe0\xef\x34\xfd\xee\xbb\x17\xe6\x1c\x22\x36\x5b\x0b\x36\x3f\x89\x03\x53\xac\xdc\xc6\xc3\x4c\x79\x27\x9f\xc8\x05\x9d\xf6\x46\x17\x6f\x47\x6f\x66\x57\xc3\xdf\x39\x2f\x5c\xc4\x10\x15\x33\x2d\x49\x2d\x17\xbc\x5f\x5d\xc5\x87\xb1\xdf\xa1\xd3\xc8\xd5\x55\x39\xbe\x63\xa0\x77\x3a\x85\x9c\xa9\x45\x44\xcc\x04\x47\xab\xab\x9c\x10\x91\xe3\x39\xbf\xa2\x21\x60\x2f\xd2\x4a\xaf\x69\xe8\x04\x2f\x51\xdc\x16\x6c\x97\xe8\x09\xcf\x3b\x08\xf0\x9d\xcb\xe2\xae\x58\xc0\x39\xca\x13\x23\x07\xf5\x3a\x15\x45\xf7\x20\x71\x3f\x47\x91\xc4\x1f\x0f\xde\xe2\x22\x22\x93\xfe\xde\x24\x2e\x9c\x37\xad\x39\x84\xf9\xdc\xc5\x4a\x69\xf1\x40\x62\x2d\xe4\x92\x11\x7e\xde\x7d\x91\x93\x53\xa6\x70\x81\x72\xd8\xd6\x50\xfa\xc2\xfb\xc8\x2b\xf7\x88\x4d\xc2\x17\x07\x74\x7e\x57\xce\x81\xd1\x90\x4a\x7e\xe5\x2a\xca\x43\x74\x44\xb6\x1b\x2e\x0b\xa9\x87\x3e\x4f\x0f\x75\xc5\x5d\x77\x93\xdb\x14\x59\x4b\xe4\x72\x0e\x23\x94\x7b\x39\x82\x85\xbe\xe4\xf0\x03\x9d\x32\xb3\xef\x3d\x2f\x58\x4e\x61\x43\xaf\x2b\xac\x04\x75\x2f\xae\x70\x8a\x5e\xf0\xd9\x13\x4e\x6d\x23\x0d\x07\x90\x01\x90\x6b\x96\x5a\x2b\xa5\x01\xb7\x02\xea\x94\x82\x5c\x97\xca\xc8\xfa\xbb\xc0\x2a\xb3\x5a\x94\xe4\x71\x06\x91\x56\xdc\xcb\xa1\x2f\x4d\x3c\x5b\xb8\x3e\xe2\x68\x30\x9d\x60\x23\x2f\x68\xb1\xe7\x35\xba\x32\x73\x8e\xfb\xd1\x3b\x3c\xbd\x26\xb9\x62\xbf\x91\x01\x4f\xfe\x59\x43\xe3\xc3\xb5\xab\x1b\xc8\x4e\x56\x24\x58\xfc\x1f\x5f\x9b\x7a\xe4\xb4\x08\xe5\xca\x23\xc5\xcd\x11\xd2\xfc\xb3\x08\x9d\xca\xf8\x3e\x53\x07\x21\xe6\x66\xe8\x82\x41\x12\x3b\xea\x18\xe0\x20\x1e\xf3\x4f\x37\xad\x8c\x55\xa5\x17\x61\x6f\xe3\xbe\x31\x26\x4b\x96\x3c\x19\xc9\x05\xe2\xb2\xa9\x82\x89\x91\xf1\xee\xdc\xed\xa2\x2e\x4b\x7b\x0b\x74\xc9\x80\x12\x35\xfc\x9e\x28\x9e\xc7\xc8\xe5\xe5\xdf\x7a\x93\x11\x4a\xd8\xb3\xde\x57\x91\x4e\x51\x64\xe4\x9c\x87\x4a\xe6\x6d\x2b\x04\x8c\xc2\x0c\x5b\x47\x7b\x07\x40\x04\x23\xec\xa0\xa8\x02\x09\xbe\x7c\x41\xe7\xe3\x79\x45\x32\x7a\x1f\x8b\x48\x7d\xac\x2a\x11\x48\xa8\x17\x87\xa1\xf3\x3f\x9a\x51\x57\xb3\xdc\xb9\x8a\x37\xda\x40\x64\x2f\xeb\x05\x6e\xe4\xcd\x69\x76\x1a\x46\xf8\x57\xce\x5e\xd2\xf1\x81\xd3\xa4\xd3\x6b\x9c\xa6\xbd\x34\x88\x3a\xf5\xa2\x4a\x0b\xb4\x48\xf8\xef\x40\x20\x0e\x30\xe8\xa5\x1b\xc9\x5a\x5c\xa8\x74\x03\x36\xb0\xe8\xf9\x53\x6a\x0b\x40\x3f\x0f\x6b\x02\x18\xeb\x5f\x0d\x63\xbd\x04\xe3\xb7\x5f\x0d\xe3\xb7\x25\x18\x3f\x7f\x35\x8c\x9f\x4b\x30\xb2\x48\xe6\x41\x96\x65\x5f\x0d\x0e\x5f\x16\x8b\x00\x57\xbe\x01\xce\x4a\x05\x9c\x6f\x42\xa7\x02\x9b\x6f\x40\xa6\x04\xe5\xfb\xaf\x06\xf2\x7d\x09\xc6\xcb\xaf\x86\xf1\xb2\x8c\xc7\xd7\xb7\xe6\xfb\x72\x6b\x5e\x7e\x3d\x94\x97\x65\x28\xf5\xfa\x57\x43\xa9\xd7\x4b\x50\x80\x73\xf8\x5a\x28\xf3\x79\x19\x97\xaf\x47\xa5\x8c\xc9\x13\x60\x84\x85\xc2\x6c\x41\x12\x4d\xbd\x65\x00\x1d\x35\x0e\xcb\x2c\x16\x66\xe7\x70\x7b\x6b\x7f\x27\xbb\x1f\xa5\xd1\x09\xda\x3c\x83\x9f\x73\x60\xac\xe0\x07\xb8\xf5\x59\x1a\xfd\xb7\xc8\xdc\xa6\xd1\xff\x88\xd0\x82\x7a\x0a\xff\xcc\xb3\xe8\x59\x0a\xff\x16\xe6\x1d\xf0\x81\xda\xad\x23\x8b\x95\xc9\x89\x5a\x42\x99\xfc\x2e\xda\x94\x5b\x33\xb5\x67\x10\x0d\x1b\x0a\x89\x33\x53\x22\x1c\xfe\xbf\x78\xf4\xd9\xdb\x21\xc1\x22\x4b\xfe\xf8\x87\xd9\x40\x7e\x66\xe1\x7d\x10\xd8\x29\xbf\xc1\x73\x0c\x1c\x08\x67\xdb\x97\x30\xdf\x22\x0e\xcf\xd0\xfe\xa1\xb5\xbb\x4a\x57\xba\x14\x43\xae\x4d\x31\xd6\xc3\xfc\xde\xd5\x66\x2f\x79\xd9\xae\x8e\xd4\xe3\x53\x85\x87\xf3\x25\x85\x41\x1d\x4c\xe3\xe8\x04\x2d\xcd\x27\x72\x55\xd6\xeb\x5b\x6d\x77\x86\xe1\xc5\x4d\x25\xbf\x98\x00\xb1\xc9\xf3\xb9\x83\xd3\x8c\x2c\x43\x1c\xe6\xba\xce\xf3\x4f\x71\xa2\x6a\x90\xc4\x12\xe8\xb7\xc8\x5c\x38\xc8\x3e\x3f\xc7\x0b\xc6\x9f\x7b\x50\xd5\xbd\xc1\xba\xa2\x7b\xa4\xc7\xd8\x2b\xc7\x1b\xe4\x17\x03\x7b\x29\x53\x5d\x78\xac\xbe\xfd\x9d\x54\x62\xb3\x36\xa9\x43\xd7\xf0\x25\x95\xc3\xd4\x55\xce\xdf\x18\xb4\xa7\x84\x62\x1c\x01\x9b\xb0\x38\xee\x36\x4d\x3b\xed\xd8\x5e\x93\x0a\xe8\x6d\x9a\x5f\xe0\x53\xdf\xd1\x2c\xe3\x2a\x8d\xa2\x21\x4c\x7d\x3b\xd2\x63\xa3\x3a\x0d\xda\x75\x1c\xe9\x1e\x5d\x74\x53\xc3\x4e\x00\x74\x86\x7a\x1d\xa1\x34\x6f\x46\xd3\xcb\xc1\xb9\xef\x2f\xa3\x8a\xb9\x4c\x9c\x25\x51\xd3\xb0\xd1\x70\xfc\xb1\x14\xf8\xdb\xe5\x60\x96\x4f\xaf\x51\x1f\xce\xf5\xfd\x7d\x50\x00\xdf\xd6\x07\xa3\x1b\xf6\xc4\x7e\x76\xb9\x99\x49\xbe\x86\x1a\x5f\x73\x76\xf9\x22\x83\x34\x15\xb7\x99\x98\xf3\x51\xe6\xb8\xb5\x63\x29\xd5\x85\xd8\x4d\x15\x0d\x85\x30\xea\x45\x10\xf5\x02\x26\xff\xe8\x45\x27\x7e\x6a\x07\x43\x09\x80\x91\x42\x99\x45\xd0\xda\xec\x45\x02\x91\x9b\x5f\x03\x68\x93\x01\x6d\x16\x00\x6d\x22\xa0\xa7\xc3\xb1\x23\x4e\xb0\x8a\xe3\x7e\x6c\xd2\x60\xdc\x1b\x6b\x6a\x98\xb9\x3a\x2b\x2e\x48\x47\x16\xb2\x02\x1d\x47\x3f\x8e\x72\x6b\xbb\x0e\xef\x3c\x6b\xb8\xaa\x7b\x10\x9c\xd4\x22\xa3\xeb\xf7\x60\x5a\xc9\x42\x93\x18\x41\x2a\x38\x07\x70\x5b\x16\xc0\x9c\xa6\x4a\x0e\xa0\x37\x71\x5a\xc2\xe4\x8c\x8f\x62\xbc\x0f\x5b\x99\x27\x0b\x03\xcd\xf9\xca\x62\x8c\x0a\xbe\xc0\xc2\x1c\x51\xfa\x46\x22\x04\x79\x73\x95\x0d\xe6\xf3\x96\x16\x45\x94\xb6\x40\x52\x89\xde\x75\x1e\x20\x71\x58\x80\xbd\xe9\x5b\x7a\x94\x56\x29\x34\x9f\x5d\xbe\x44\x2f\x5a\x40\x4e\xfe\x08\x9b\x3e\x74\x8e\xd1\x4b\xa2\xaa\x48\x54\x43\xba\x73\x76\x09\xdb\xf2\xc9\xc4\x7d\xfe\x37\x1f\x39\x72\x9f\xff\xc3\x7d\xfd\xef\xff\x8b\x3f\x11\x3c\x11\xb4\xe5\xc8\xf4\x10\x99\x7f\x11\x32\xf3\x39\xc5\x6c\x61\xcc\x3f\x24\x26\xfa\xe0\x80\xae\x7a\x98\x3b\x5f\xae\xf7\xc4\xa5\x6e\x25\xce\x6b\xae\x50\xc3\x7e\x85\xb4\x9a\x86\xd3\xcf\x36\x75\xad\x8a\x41\xb6\x75\x68\xf2\x11\x8a\x87\x8c\xfa\x19\xfc\xb3\x00\xc8\x08\x00\x0b\xf8\x0d\xd1\x7f\xa2\x3f\x50\x50\x99\xa4\x13\x4d\x6b\x51\x83\xbe\x1b\x68\x7b\x5d\x8d\x50\x54\x3b\x96\xb0\x72\x56\xcc\x4a\xf6\xb6\x36\xd4\x4c\x4b\xa1\xc7\x1b\x10\x68\xcb\xf9\x5d\x1d\x1a\xe3\x68\x88\x3b\x31\xde\xd3\xd1\x96\x5c\x63\xe4\xc9\xb6\x63\x6f\x86\x28\xdd\x5c\x8d\xee\x5b\x8b\xd0\x3d\x6e\xed\xf8\x7e\x73\xd1\x6d\x46\x6c\xac\xd9\x30\xe2\xc6\x21\x42\x17\xcc\x76\x9f\x0a\xf4\x13\x64\x3f\x16\x77\x86\x51\xc4\xdd\x92\xa9\x8e\x78\x78\x73\x66\x02\xaa\xf4\x64\x1e\xda\xa2\x49\x7f\x06\xb6\xa1\x25\xa3\x95\x88\x6b\x4d\x48\xf5\xf6\x67\x71\x51\xd9\x85\xce\x84\x99\x80\xe4\x04\xc4\x51\x1c\x35\x59\x62\x2e\x51\x82\xa6\x36\x3d\x9d\x1b\xea\xad\xd7\xb9\x48\x71\xb3\x17\x40\x30\x53\xb1\x36\x71\x6d\x29\xad\x92\x90\xdd\x83\xcb\xb8\xe3\x75\x4d\x75\x75\xf3\xf9\x83\xd5\xc1\x84\xce\xf1\x71\xee\xe1\xea\xf8\xb2\xa5\x44\x50\xdf\xb2\xc5\x68\x9c\x19\x74\xe1\x07\x54\x78\xa1\x77\xbe\x85\x0c\x74\xeb\xb9\xf8\x57\x5d\x46\xfd\x79\xbe\x12\xe1\x97\x9c\x44\xf1\x49\xa5\xb4\xf2\x76\x02\xf3\x2c\x16\x32\xd1\x0a\xf4\xc0\xcf\x33\xa4\x95\xaf\xc6\x33\x22\x95\x6f\x89\xb6\x5f\xe5\xb3\xcb\x31\xe9\x10\x19\xbe\x69\x9b\x02\x7a\x24\x6d\x49\x57\x33\x5f\x3b\x25\x51\xea\xe6\x11\x3e\x91\x26\xa0\x9e\x81\xcc\x8e\x69\xde\x4f\xcd\xc9\x76\x62\xf3\xa3\x89\x07\x46\x3f\xd3\xe0\x08\x4d\x1c\xf7\x90\x27\x41\x87\x41\x9c\x9b\xe4\x5d\x5d\x83\x75\x3b\x5c\xe4\x12\xb6\xb7\xaa\x25\xae\x0c\x34\x24\x66\xc4\x92\x7b\xdf\x89\x19\xa1\x63\x1f\x56\x05\x83\x35\xa6\x55\x2d\x41\x36\xcc\xb3\x61\x82\x5c\x7a\xd3\xce\x5c\x65\xe2\x53\x5e\x4d\x6a\xc5\x64\x61\x37\xf1\x7c\x74\xf9\x61\xa6\x39\x36\x3f\x2b\x4f\x29\x42\x80\x4c\xfd\x3a\x16\x09\xaf\x96\xb5\xc3\x02\xca\x02\xfc\x28\xf1\xb1\x01\xd7\x45\x29\x5d\xa3\x38\xde\x62\x9a\xa7\x1e\xec\x44\x3b\xe3\x9f\xd7\x23\x86\x6a\xf4\xf1\x48\x11\xc9\xb6\xab\x0c\x42\x40\x9d\xe3\xaa\x83\x9c\x9b\xf2\xe2\x9f\x3b\x3c\x4f\xa2\xff\x85\xc1\xc5\xa8\x70\x04\x0c\x15\xcc\xa6\xaa\x20\xe3\xc3\x82\x25\x3c\xf7\x09\x15\xfb\x6d\x0f\x74\xb8\xb0\x4a\x2b\x75\xc6\xe6\xe7\xfd\xd8\xe3\x31\x60\xc9\x72\xb6\xeb\x8e\x7a\x1f\x26\xb4\xac\xe6\x96\xe5\xd1\x96\x17\x69\xb4\xb8\x90\x5a\xa7\xae\xac\xac\xf6\xa2\x46\xf7\x2f\x37\x63\xab\xb8\xf7\xc0\xd2\x85\x35\x62\xa9\x02\xef\x91\xb8\xd4\x27\xbd\xcf\x62\xc4\x84\x80\x98\x7c\x7a\xd6\xbb\x46\x63\x63\x4f\xdb\x86\x1e\x5a\xf7\x38\xdb\x1c\x78\x5a\xaf\x0c\x5c\x9c\xb9\xdb\xa5\x44\xbe\x00\xed\x32\x2d\x6c\xe2\x9a\x0f\xd5\xab\xfd\x77\x49\x1b\x8a\x58\xf3\xd4\xc7\x27\xfd\xde\xda\x79\xf7\xfe\xbb\xc5\xfa\xc0\x92\x94\x2a\x1a\x7d\x33\x1a\xe0\xfb\x4c\x8d\xd1\xa8\x1d\x9f\x9c\xdc\x44\x8d\x4b\x64\x22\xba\x51\xc8\x3a\x7f\x67\xa6\x82\x75\x95\xb1\x1a\x9a\x2b\x6f\x81\x82\x41\x59\xd3\xfa\xbd\xa8\x62\x8a\x22\xda\x75\xc6\xb7\x0b\x70\x02\xe9\xb6\x2d\x14\x88\xee\xc0\x3f\x38\x1a\x2c\x5c\x07\x3b\xfd\xcd\xe8\xe4\x44\xfa\xc2\xa6\x6d\xb8\x1d\x0d\xba\x38\xe3\x91\x49\xca\xec\x6c\xe3\x29\x9b\x89\x1b\x01\x69\x92\x18\x18\x7a\x78\x77\x11\x23\x44\x30\x59\x59\x4f\xd1\x35\x04\x1a\x10\xec\x6e\x15\x07\x0c\x58\x6a\x57\x83\x11\x0a\xdb\xd7\x08\x71\xd9\x4e\x12\xb9\xbe\xdf\xe7\x0d\xc7\xd5\x49\xcc\x97\x59\x65\x8f\xf4\x26\xbc\x50\xa1\xb4\x8c\x73\xf0\x9b\x00\x67\xcb\x6c\xf6\xea\x6b\x17\xae\xa2\xf9\x8f\x9d\x83\xbd\x0a\x53\x32\x1b\x0b\x23\x19\x96\x5c\xcc\x70\x2a\x53\x84\xf0\x6e\x86\x3a\xad\xfa\x82\x86\xc8\x22\xfe\x09\x2e\x60\x7c\x2b\xf0\x2f\xdf\xef\x10\x10\x4b\x39\x98\x7a\xe1\x23\x91\xe4\x1d\x5f\x0c\x80\xb2\xed\x1d\x30\x18\x5b\xfb\x76\x6f\x38\xe4\x0c\xe7\x83\x7c\xd8\xdf\x3a\x3b\x03\x2e\x94\x23\xf8\x8d\x48\xed\x74\xdc\x35\x30\x59\x07\xa3\xc0\x6b\x40\x71\xa4\x06\xfc\x0a\xcd\x4b\x03\x71\x8a\xcc\x3d\x9f\x3b\xb1\x51\x3c\x8f\x36\xd0\x28\x60\xdb\x0b\x98\x61\x36\x3e\x30\x5d\x4f\x06\x57\xbd\xc9\x5d\x2c\x67\xcc\x29\x1a\xb2\xa0\xe7\x39\xe7\xa1\xb7\x86\x0e\xdd\xca\xd7\x27\xc2\x84\x15\x30\xe9\x8d\x60\x75\xba\xa3\x29\x15\x08\xae\xb2\x8e\xd1\x2d\x29\x9b\x9c\x18\xc2\x36\x38\xe9\x0d\xb3\x95\x95\x20\x2c\xa9\x34\x90\x3d\x74\xaf\xbe\x12\x46\x88\xb4\x9b\x11\xc4\x8b\xe6\x30\x24\xba\x6d\x79\x56\xc6\x25\x86\x1d\x3f\x49\x24\xad\xd4\xbd\xf6\x56\x04\xab\xb8\xb9\xca\x21\x6b\x54\xb8\xee\xb1\x50\x8e\x8b\x50\xc8\xc7\xcc\xab\xfc\x6c\x88\xce\xb7\x08\x8b\xea\x82\xf7\xc5\x82\x3c\xdc\xb1\xf6\x17\xe1\xef\xac\xa4\x54\xd2\x76\x45\x64\x7f\xb5\xad\xae\x20\x8e\x38\x07\x7a\xb6\xf9\xea\x80\x13\x19\xd9\xee\xfc\xb6\x0f\x93\x56\xf2\xa9\x6e\xde\xb0\xb0\xfd\xc0\x6c\x24\x4e\xd1\x11\x2f\x19\x58\xb1\xe2\xcb\xac\x3d\xa2\x25\x13\xf6\x2e\xda\x12\x41\x06\x11\xd9\x3e\xe2\xae\x30\x13\x2d\xae\x4e\x1c\xf6\xbb\x5a\x0a\x36\xc5\x42\x4e\xec\x07\xbb\xe2\x4c\x01\x66\x08\xc9\x26\xdb\x72\x15\x5d\x4a\x2b\xc8\x02\x4e\x92\x94\x99\xd0\xa7\xc2\x50\xcb\x52\xc1\x28\xef\x44\xbb\xfb\x7b\x87\x87\x6f\x7f\x78\xb7\x13\xb9\x65\x22\xd9\xab\x4f\xce\x57\xd3\x0b\x19\x87\xfb\xaa\x93\xeb\xf4\x6e\x34\x43\x5b\xcf\xd1\x21\x7d\xd8\xa3\x2b\xb9\xbf\x61\x79\x40\xf4\x43\xe0\x8f\xb1\xe8\x87\xc0\xbe\x46\x07\x67\xd9\x17\x8b\xae\x97\x2a\x84\xec\xc7\xb0\x99\xe2\xe9\xd6\xdf\x6a\x1a\x87\x4a\xb8\x21\x7b\x12\x58\xdc\xbb\x7d\xde\x44\xee\x6a\x08\x2f\xbd\xf2\xd0\xed\x63\x56\x45\x25\x92\xaa\xd6\xde\xe4\x63\xd8\x4d\xf4\x1d\x16\xf0\x8c\xd8\x1c\xdf\x14\x3a\xb6\x47\x9a\xc5\x2c\x5f\x54\x01\x2d\x29\x5e\x1d\xe5\x2d\x93\x6f\x9a\xfc\x85\xc9\xbf\xf3\xd7\xdf\x01\x42\x2f\x37\x92\xe2\x6a\x73\xd0\xcc\x4c\xdd\xfe\x12\x05\x41\xcd\x9a\x16\x2c\x36\xfc\xdd\x94\xdf\x17\xf2\xfb\xdd\x7c\xbe\x92\xb7\xd0\x22\xe7\x26\xfe\x79\x81\x7f\xbe\x73\xbb\x3b\xc2\x59\xf8\x57\x64\x6e\xe9\x12\x44\x0b\xf8\xd0\x61\x5d\x67\x68\x6b\xa0\x72\xef\xc8\xeb\x78\x45\x2f\xea\xa7\xec\x0c\x96\x16\xa8\x7e\x71\x17\xc3\x94\x44\xb7\x64\x42\x08\x35\xba\x56\xd8\x90\x17\x3c\x84\x2b\x48\x10\xd4\xe7\x89\xbf\xa9\xe5\x4e\xec\xf5\x38\x6a\xe4\x2d\xe4\xce\x8c\xba\x8d\x48\xf0\x6d\x1d\x16\xcc\xeb\x91\xb6\xce\x64\x26\x83\x8b\x4b\xa5\xa9\xf5\x84\xf3\xc4\x79\xf8\x5e\xc4\x00\xe0\x44\x61\x69\x5b\x4a\x31\x8e\xd4\xe1\x51\x1b\xfd\x66\x14\xaa\x1e\xe6\xe7\xb0\x36\x06\xfd\xfe\x30\xff\x06\x1c\xb0\x74\x90\xd0\x61\x50\x41\x1c\x23\x52\x3c\xf8\x38\x34\x11\x86\xc3\x12\x4d\xe8\x23\x00\x15\x51\x6e\xc6\xe9\xa0\xb2\x15\xff\x8e\x5e\x24\x40\xe5\xae\x2c\xe0\x58\x46\xc9\x33\x10\x55\xb7\x18\x3e\x15\x5f\xaf\x48\x77\xb5\x6a\x89\xe2\xc4\x76\x13\x25\x5a\xa0\x4d\x24\xf8\xd7\x86\x7f\xf4\x9e\xe1\xa1\xc8\x79\xae\xb4\x9f\x27\x66\x25\xd8\x9f\xda\x91\x13\x10\xc3\x37\x1e\x05\x40\x2e\x8f\x7d\x0c\xf9\x9c\xaf\xec\xa8\x2a\xad\x8a\x12\x24\xef\x27\xd7\x25\xa9\xea\x8e\x07\x78\x96\x90\x90\xd3\x9e\x70\x31\x85\x37\xd1\x40\x81\x1f\x75\x71\x5c\x0b\xab\xfa\x96\x5f\x06\x71\x80\xdc\xde\x2c\xc4\x14\x18\x09\xbe\x16\x59\x89\xcb\x4c\x46\x34\x87\xbe\x49\xd4\x34\x6e\x7b\x10\x76\x7e\xf1\xb4\x72\x4c\x88\xea\x6f\x5c\xc6\x16\xb3\xa5\x48\x85\x4c\x0d\xbe\x04\xe9\x43\x40\xec\x69\x2e\xfa\x10\xbc\x98\xbe\x1e\xf1\xe4\x58\x8a\x71\x8a\x18\x8b\xc3\x74\x38\x13\x42\x86\xac\xf2\xe5\x98\xbd\x4b\xdf\x7b\x47\xac\x17\x30\xed\x28\xd2\x3b\x62\xc5\xea\x42\x4f\xac\x17\x53\xeb\x84\x15\xd3\xd0\xb7\x71\x30\x2a\x4e\x97\xd5\xea\xf4\x50\x2a\x66\x4e\x16\x25\x0f\xef\x6a\x75\x31\xa2\x8b\x85\x54\xe9\x27\xae\x1e\x27\xec\x4c\xb5\x09\x2e\x91\xa8\xf2\x87\x0d\xf4\x28\xe7\x43\x45\x5e\x98\x56\xa7\x29\xce\x0b\xa1\x7d\x8e\xbf\xaf\xea\xe1\x2c\x42\x81\x53\x5a\xec\x0c\xbe\x8a\xda\x5f\x5d\x0f\x07\x39\xb0\xfc\xae\x7e\xf2\x99\xb4\xe4\x1a\x7f\x43\xf3\x1c\x78\x95\x5f\x3b\x83\xb3\x02\x6e\x4f\x28\x2e\x47\x20\xe8\xc0\xe0\xb6\x28\x42\xbe\x80\xb1\x97\x4f\x64\x9b\xdb\x15\x44\x58\xf0\x95\x0c\x42\x70\x75\x66\x47\x79\x13\x22\x66\x6e\x2f\x28\x76\x9e\x6c\x05\xc5\xde\x73\x07\xbb\x07\xfb\xaf\x43\xfd\xc7\x20\x8a\x6d\x58\x36\xa3\x3b\x3a\xdf\xeb\x70\x43\x0a\x41\x58\xab\x82\x7a\x34\x1c\x2b\x95\xba\x1e\x14\xd3\x44\xb6\x9d\x0e\xf3\xa7\x10\x0f\xc9\xbc\xf5\xfe\xd5\x23\xc4\xe3\x1b\xa9\x87\x86\x8f\x93\xde\x87\x8b\xe3\x50\x22\x6b\x68\xff\x1c\xfd\x35\x3c\xd4\xff\xf5\x7a\x44\xde\xa2\xbf\x01\x99\xc4\x70\x77\xd9\x7a\x1e\xc3\x87\x0c\x12\x40\x32\xb9\x2d\x7c\x60\x45\x65\x11\x49\x57\xb1\xe0\x18\x8b\x7d\x7d\x05\x8a\xbe\xd5\x0e\x41\x5f\xf1\x63\x28\xf6\xfa\xa8\x1e\x8b\x5a\x27\x0f\x20\xf8\x7d\x84\x32\x57\x24\x33\x45\x22\x4f\x5f\x81\x9c\xee\x02\x87\x9e\xad\xf4\x29\xb3\x4d\x8c\x40\xa1\x1b\x6e\x46\xb3\x02\xbf\x46\x44\xce\x98\xdb\xc9\xd3\x70\x2a\x82\x74\x74\x9b\xb1\x0b\x93\x9f\x82\xe3\x8d\x10\xcd\x0a\xd4\x9e\x47\x28\xe3\x88\x32\x8a\x4f\x46\x4f\xa0\x15\xb0\xba\xa9\x22\x44\x54\x30\x38\xfe\xf8\x2e\x49\x8a\x97\x38\x55\x23\xbb\xe6\x88\x8b\xc3\x48\xdd\xac\x2d\x41\xac\x12\xd2\x8a\x83\x24\x9c\x7b\xbc\xa4\x70\x88\x15\x32\x08\xee\x58\xed\x9b\xc7\xe7\x75\xa7\x66\xed\x9e\xaf\x28\x73\xc8\x33\xf0\xe9\xb4\xf8\x26\x41\x19\x97\xbe\x49\x2c\xe3\x7d\xd5\x66\x80\x8a\xcc\xe7\x55\xef\x12\xf3\xb9\xdc\xcf\x54\x6f\x1d\x15\xaf\x15\x94\x8f\x39\xc3\xc2\xae\x24\xaf\x16\x95\x00\xb9\x05\x4f\x7a\xc5\x30\xea\x7a\x23\xe8\xc3\x72\x07\xd2\x1e\xfb\x7a\x54\xc1\x06\xea\x59\xe4\x6e\xbb\xf0\xd2\xfc\xc1\x13\x02\x31\xbd\xe6\xda\x8c\x33\xa8\xae\x20\xc7\x9c\x49\x65\x41\xb4\xad\x67\xdc\x89\x6f\x33\x60\xec\x6f\x26\xf9\x61\xef\x3c\x17\x11\xfc\x31\x6a\x00\xab\xd6\xc2\x4e\x55\xaf\xdf\xc2\x6e\x96\x8f\x48\x25\x50\xb5\x1c\x66\x14\x9a\x74\xd8\x67\xe9\xef\x29\x5e\x5c\x65\xb7\x26\x5a\x5d\xbd\x8d\x06\xc0\x18\xcf\xe7\xf1\x75\x13\x02\x4a\x8c\xee\x9a\xe0\x04\x7e\x43\x92\x7b\xce\x84\x66\x2e\xd1\x60\x66\x76\x8b\x41\xf8\x48\xb4\x5c\xf5\x03\x0f\x50\x41\x47\xa0\x7e\x68\x55\x9b\xcd\x14\x5a\x58\xd1\xd8\x62\x8f\xe9\x86\xdb\x7e\xc2\xa2\xa4\xca\x6b\x0d\xb1\x25\x5e\x0b\x1c\xef\xc9\x82\x03\xb2\xdc\x5f\xb1\x54\xbe\x68\x60\x3a\xae\x19\x9f\x21\x92\xc8\x5e\xd8\xba\xbb\x1a\x59\x41\x49\x7f\x5c\x7b\x98\xff\x6c\x7f\xbe\x1c\x0c\xf3\xf0\xee\xd2\xa0\xf4\x5b\xf9\x82\xb4\x30\xeb\x4a\xbc\x6f\xc8\xa2\x85\x4c\x78\xd7\x5d\xf4\x05\xad\xe9\x04\xa1\x10\x42\xca\xa1\xaf\xe2\xdb\x75\x79\x46\xf8\x7c\xb4\x3f\x9b\xa0\x8e\x7b\x40\x06\xec\xf5\x23\x9b\x0c\x6d\x97\x06\x91\x4a\x85\x23\x47\x6b\x22\xa3\x04\x3e\x07\x74\xd4\xb7\xbd\x63\xe4\x63\x41\x4a\x29\x84\x17\x5e\x32\xd1\x6f\x4b\x7e\x37\xe5\xf7\x85\xfc\x7e\xe7\x0d\xcd\x95\xb0\xb8\x0d\x30\x00\xa6\xa9\x78\xf9\x5c\xdc\x39\x44\xcf\xf0\x35\x49\x90\x1a\x98\x47\xdb\xfe\xb6\x97\x66\x4a\xf7\xa1\x99\x12\x82\xa8\x20\x26\x1e\x7c\x60\x8c\xe7\xf5\xc8\xf9\x37\x7a\x3d\x72\x77\x03\xb0\x54\x03\x04\x5a\xc9\x62\xd9\x54\xfb\x36\x22\xe5\x27\x18\x74\x0a\x79\x26\xa0\x99\xa2\x90\x0c\x66\x0b\x64\x2a\x60\x3d\x5d\x7a\xd4\xa3\xdc\x40\x21\xe4\x62\x3c\x65\xb7\xf7\x7c\x21\xa2\x1a\xe5\x89\x74\x71\x20\x3e\x89\x5e\xd2\xb2\x71\x58\x3c\x3a\x0e\x55\xe7\x68\xa4\x44\xbc\xff\xf2\x19\x0b\x3d\xbd\xbb\xdb\xcb\xb0\xf7\xd2\x48\xbf\xfe\x94\x87\xd2\x21\x28\xaf\x8f\x10\x4e\x9d\xd5\x03\xb1\xbf\x90\x14\x5e\x66\x7e\xed\x88\x2e\x9e\x3a\xa2\xdc\xa7\xa8\xee\xc3\x43\xea\x91\x2d\x5e\xb9\xd8\x94\xcc\x65\xc1\x0b\x17\x2e\x7f\x6c\xe3\x9a\x44\x68\x5d\xe8\x56\x14\xc6\xd4\x9d\x9c\xdd\xbd\xa8\xdc\x93\xc6\x9d\x9f\x44\x2d\x63\xe2\xd4\x93\x56\xa7\x67\x25\x8d\x26\x88\x82\x86\x1f\x6e\xef\x7c\xd8\xde\x7b\x8f\x36\x59\x0f\xb3\x7b\xb4\xc1\x99\x46\xa8\x27\x19\x99\xed\xc3\xc3\x34\x3a\x43\xd7\x3e\x3f\x1e\x40\x24\xb9\xff\x3b\xd8\x39\xdc\xfb\xf1\x00\x8a\x50\x14\x0c\xdb\xf8\x66\x72\x96\xa3\x96\xa4\xf9\x33\xe4\xfe\x38\x8d\x16\xa8\x2d\x89\x5c\x1d\xcc\x03\x52\xb1\x77\x16\xf4\xc4\x1a\x20\xcf\xf2\x38\xea\x41\xe5\x63\x38\x16\x0f\x46\x50\x3c\x83\x42\x07\x00\x6d\x78\x6b\x8d\xcf\x37\xad\xd2\x57\xf3\x72\x92\x9f\xa3\xdf\xa1\xf6\xea\x6b\xba\xa6\x79\x50\x91\xd8\xb0\xc1\xea\xb3\x3b\xce\xab\xf3\x50\xa7\x62\x16\x96\x74\x5a\x9e\x81\xba\xf0\xd5\xce\xf6\xdb\xdd\xad\x77\x1f\x0e\x77\xf6\x33\x94\xb6\x78\xb5\x75\xb4\xf3\xe1\xf5\xde\xc1\xee\x16\xf6\xd3\x1d\xfc\x97\xa2\x27\x65\xab\xdb\xf8\xfa\x66\x38\xfc\x29\xef\x4d\x22\xf3\x5d\x62\x96\xa6\x6d\xa2\x22\xf5\x06\x64\x58\x92\xde\x4a\xcc\x2e\xfc\x47\xa9\x87\xb3\x89\xcd\xb0\x0b\x44\xfc\x32\xa2\xb4\xea\x24\x02\x2a\x89\x85\x94\x4d\x02\x5a\x95\xd2\xc2\x94\x7e\x3f\x48\x42\x57\x15\x50\x06\xe2\x2b\xa2\x21\xfb\x9b\x37\x41\xfc\x1b\x18\xfe\x29\xe5\xaf\x8c\x87\x02\x97\x97\x95\x05\xcc\x5a\x0b\x0a\x55\xa6\xb5\x38\xed\xea\x2a\xc4\x79\x30\x42\xeb\xda\x54\xd7\x92\x14\xa8\x0d\x98\x7a\x9d\x74\x98\xa3\x41\x26\x2e\xb4\x24\x85\x0a\x4d\x0b\x00\x87\x43\xe0\xf1\x24\xc3\x8b\xc4\xec\xc0\x7f\xc5\x7e\x7f\xd5\xbb\x8b\x28\xa5\x2a\x81\xbd\x64\xa5\xbd\xab\xeb\x2b\x8e\x36\xff\x48\x51\x9d\xfe\x1f\xe3\x91\xd4\xb2\x08\xe6\x93\xb5\x7b\x12\xc7\x9d\xf4\xf8\xe7\xbb\xdd\xfe\x9b\xcb\xab\x69\xef\x1f\x3b\xcf\xba\x8d\x64\x0e\x71\xcf\x28\xfe\x59\x77\xfe\xec\x59\xf2\xfc\x19\x45\xed\x34\xe6\x77\x8d\xf9\x6e\x63\xde\x6f\xcc\xdf\x34\xe6\x97\x8d\xf9\x55\x63\x3e\x6d\xcc\x7b\xf3\x7f\x24\x09\xaa\x67\xaf\x1b\x71\x20\x8c\xb6\x52\xde\x93\x25\x95\xb5\xce\x49\xbf\xb1\xba\xde\x46\x8c\x1f\x99\xfd\x4e\x18\x94\xf3\x31\x37\x88\x07\x48\x1b\x9f\x78\x8f\x48\x85\x2c\x37\xde\x33\xd2\x78\x02\xcb\xf4\x87\x8a\xa5\xc8\xca\x94\x5c\x15\x12\x9b\xad\xd1\xd9\xe5\x78\xe2\x1c\xe4\x39\x58\xca\xdb\xdd\x4e\x64\xc4\xa6\x40\xd9\xb8\x02\x7b\xba\x2b\x59\xb7\x27\x3f\x65\x48\x3d\xe6\x73\xe7\xb4\x52\x3e\xd1\x77\x61\x1c\x61\x5a\x84\x9e\x76\x4c\x68\xf9\x2e\x2e\xd0\xac\xed\xf1\x15\xd3\xac\xb7\x3b\xb5\xf3\xc1\x17\xbc\x68\x2a\x5e\x00\x3a\x23\x0b\xce\x3e\x74\x85\x43\x45\xb1\x49\xa9\x8d\xc6\x0b\x12\xc0\xf0\xb1\x25\xa6\xd0\xd2\x53\xcc\xee\xc5\x12\x33\xba\x70\x06\xd3\xb7\x86\x83\x9e\xf7\x25\x38\xd5\xbe\x1e\xb4\x5d\x7b\xa5\x41\x3d\x19\x5f\x93\xd0\x17\xd6\x47\x46\x1e\xd8\x4a\xaa\xd8\xbb\x86\x1d\xdf\xe6\xb0\x36\x7f\x27\x57\xbd\xe1\xe0\x5f\x79\x3f\xeb\xdb\x6a\xde\xdb\xb8\x98\xfd\xda\x38\x50\xed\xe5\x98\x1d\x7b\x40\xdd\xb2\xd4\xcc\x3d\x1c\xca\x81\xfc\xcf\xee\xd2\xd6\xc6\x46\x79\x68\xfd\x7d\x75\x65\x3f\xcb\x90\x57\x38\xae\xf3\x75\x96\x9c\x57\xf8\x91\xb7\xe8\x1b\x91\xe9\x10\x2f\x6e\xda\xbe\xf0\x71\x34\x9d\x9c\xc1\xe4\x80\xbf\x50\x02\x3e\x68\x9c\x14\x50\xdf\x9b\xff\x19\x9d\xf6\xc7\x3f\x3e\xc1\xf7\x63\xe8\xa8\xcf\x43\x2e\xf5\x0c\xfd\xd8\x15\x13\xf6\x10\xe7\x30\xbc\xa4\x9c\xf1\x39\x98\x30\x3e\x0b\x75\xbc\x0d\x75\xad\x4b\x3c\xe6\xf3\x50\x7c\xe2\x35\x54\xbc\x3d\x83\x4d\xfe\x9e\xbc\x2e\xb2\x81\x6d\x56\x5e\xb5\xee\x1c\x83\x38\xe5\x0b\xd7\xc7\x40\x17\xe9\xe0\xfe\x64\x30\x9d\xc1\x79\x99\x4d\x91\xb4\xa9\x0a\x67\xb9\x5b\x93\x1a\x41\x19\x06\x70\x15\x51\x44\x23\xcd\xe2\x25\x84\xc9\x0f\xba\x3e\x71\xdd\xff\x9a\x0d\x15\xf9\x8e\x1f\x4c\xdf\x5f\x20\x6c\x3b\x00\x8f\x58\x2d\x2a\x01\xcc\xee\x47\xec\x4e\x0a\x22\xd1\xd0\x82\x50\x33\x0b\xb6\x13\xed\x6c\x6d\x47\x42\xdc\x9c\xc3\x92\xb0\x31\xcb\xd7\x06\x9a\x85\x2b\x4e\x00\xac\x68\x47\x4d\x02\x05\x97\xef\xce\x69\x88\x7b\x54\xc4\xca\x29\x69\x8a\x63\x0d\xd5\x66\x25\xa2\x55\x41\x9e\x3a\xd5\x34\x2b\xb5\xc6\x75\x11\xc9\xbf\x8a\xab\xd9\x45\xbb\xc2\x16\xae\xc2\x16\x0f\xa4\xd1\xf4\xe6\xf4\x6a\x00\x5d\x5b\x8d\x53\xa2\x5b\x47\x04\x76\xb5\x8f\x3d\x3a\xbe\x0b\xbc\x4f\x3a\xcb\x35\x41\x87\x55\xb8\xae\xfa\xda\xea\x17\x62\x06\x87\x7d\x2c\xb0\x7d\x23\x37\xb7\x35\x6a\x9c\x04\x07\x27\xdf\xf9\x31\x4f\x01\x60\x0b\x70\xbd\x67\xc5\xfd\x68\x44\xd3\xa1\x4d\x89\xf5\xfa\x54\xdf\x3a\x50\x9c\x1a\x46\x8e\x48\x4c\x58\x7f\xbd\xfe\xa4\xbe\x09\x0b\x35\xc3\xe5\x17\xab\xb9\x62\x96\xa3\xe2\xaf\xb8\x04\x13\x39\x3b\x29\x14\xf5\xaa\x17\x72\xa0\x2c\xef\xfa\xe5\xb1\xe8\x2e\x4c\xb8\x5e\xaa\x96\x63\x8c\xfb\xdf\xeb\xc7\xb3\x21\xd7\xe5\x7d\x9e\xa3\xf5\x9a\xf3\xd9\xf5\x1c\x6d\xd1\xd0\x9f\x69\x92\xa2\x95\xb8\xf8\xe4\x73\x23\xbd\xdf\x30\xad\xc5\xc9\xe7\xe7\xff\x2b\xe9\xa0\x65\xb9\x24\x4e\x8f\x37\xd6\xfe\x08\xec\x16\x04\xd7\xe7\x27\xe8\x1a\xec\xf3\x6f\x56\xd2\x66\xa7\x91\xd5\x7f\xfb\xbf\x56\x4e\xd6\x4e\xd6\xbb\x6c\xf1\x46\xbb\xfb\x86\x2a\xd0\x39\x58\x6f\xed\x5f\x50\xb8\xf9\xe1\xb7\x8d\xb5\x6e\xe3\x7f\xa9\x18\x08\x9e\x34\x25\xdc\xbd\xdf\x34\xbf\x5f\xac\x3a\xb6\xcc\x41\x40\x03\x77\x27\x6b\xf3\x13\xaa\x1b\xb8\x39\xf8\x03\x11\x4d\xf8\x0b\x6c\x06\x9b\xbb\x1b\x58\x6f\xd2\xff\x0f\x79\xef\xb6\xdd\xc6\x91\x24\x8a\xbe\xef\xaf\x20\x31\x6a\xb8\x0a\x48\x80\x00\x28\xc9\x36\xa0\x22\x46\x96\x65\xb7\xf6\x96\x65\x6d\x49\xee\xb1\x9b\x84\xb9\x71\x23\x09\x13\x04\x68\x5c\x28\x80\x24\xce\xea\x5e\x6b\x56\x7f\xc1\xf9\x93\xf6\x3c\xf5\x3f\x48\xbf\x74\xe2\x96\xd7\xaa\x02\xa9\xbe\xac\x39\x67\x9d\xe9\xb1\x88\xca\x4b\x64\x64\x64\x64\x64\x64\x66\x64\x44\x62\xed\x29\x4d\x84\x69\xd9\xce\xc8\x1b\x0f\x9b\x8c\x4e\x6b\xdc\x80\xf5\x8a\xa2\x80\x37\xfd\x08\xeb\x8a\x22\x80\x37\xfd\x38\xe0\x4a\x47\xb1\x6e\xa6\xc2\x59\xab\xb3\xd1\x60\x30\x64\x67\x93\x12\x3c\x4a\xbc\x51\xd2\xd4\xe1\xdf\x20\xe6\x86\xfc\x73\xc3\xa8\xdb\x71\x03\xf9\x29\x5e\x8b\x48\x0e\x73\x30\x76\x57\x94\x4a\xae\x09\xd4\x9e\x11\xf7\xf8\x79\x41\x47\xd1\x6a\x16\xda\x93\xd3\xef\x60\x93\x39\xbe\x3b\x12\xb2\x44\xdd\xee\xd3\x5c\x89\x0c\x45\x9d\xc8\x5a\x34\x11\xc9\x7b\x77\xe7\xf6\xd6\x14\xe0\xc3\x8f\x4f\x08\x28\xff\x3f\x80\xd9\x3b\xea\x0f\x4f\x5f\xbe\xf8\xfa\xf8\xd9\xcb\xa7\x6f\xdf\x52\x28\x40\x32\x7d\x2a\xa8\x17\xaf\xc2\x8c\xd1\x44\xb2\x5e\xbf\x79\xf1\xf6\xdd\x8b\x57\xcf\x9d\xbc\x4b\x59\xe4\x60\xf7\xf9\xe2\xcd\xbb\x9f\x9c\x9c\x01\xae\x86\x05\xf5\x8a\x7b\x6f\x17\x0b\x24\x30\xaf\x6e\x40\xde\xe1\xaa\x3f\xa4\xd3\x6b\xf1\x69\xea\xad\x81\xce\xba\xc8\xda\xb8\x33\x08\xdc\xd9\x54\x7d\x45\xb5\x95\xae\x2a\x8e\x52\xdc\x28\x76\xe4\x76\x97\xd6\xee\x67\xf3\x39\x2c\x9e\xf4\x53\x5d\x71\xe4\x79\x28\x41\x57\xc4\xff\x8b\x5d\x4c\x86\x69\x49\x3a\xa9\x8d\x4f\xf3\xe6\x93\xee\xf9\xf0\x58\x87\x18\x0b\x4a\xd0\x4d\x1c\x7a\x3f\x7b\xe0\xbb\xb5\x63\xf7\x66\x1a\x83\xb6\x47\xf6\xa6\xf3\x3b\x2e\x67\x20\x57\x35\xa1\x81\x0d\x00\xb7\xba\x07\x2c\x13\x00\x1b\xb1\xdb\x48\xe6\x09\x3f\x46\xaa\xbe\xea\xbe\x12\xab\xf3\x0b\x1c\xb6\x9c\x3c\x09\x3b\x8e\x27\x74\x62\x9d\x62\x02\x99\xdb\x34\x84\xfd\xec\xac\x3b\x39\x1d\xea\xe5\xc9\xc9\xd4\x7c\xa3\xfd\xdc\x54\x1f\x10\xbf\x60\x2c\x28\xa9\x8d\x38\xdb\x5c\xe1\x41\x9b\x4f\x81\xc5\x1e\x98\x45\x8a\x35\x39\xe6\x35\xd8\xbf\x26\x8c\x62\x24\x05\x38\x1d\x65\x34\xfd\x78\x0b\x05\x6c\x59\xb1\x82\xa0\x28\x0d\xb6\x40\xec\x7b\x39\xd3\x93\x98\x1c\x9c\x4d\xb8\x06\x7b\xab\xd4\xf6\xa4\x6c\x79\x4a\x66\x84\x93\x0a\x17\x40\x4f\x74\xd5\x1d\x1d\x48\x01\xad\x52\x0b\xca\xc3\x48\x69\xf3\xd3\x77\xdd\xd3\x48\xf3\x87\xbe\x40\xd0\x11\xe9\x49\x60\x09\x15\xb6\xc7\x99\x4f\x47\xcd\x41\xe7\x59\x1c\xc7\x0c\x83\xf9\x48\xe0\x2a\xf9\xd2\x1e\x6a\xf9\xc4\xb2\xe5\x2b\x0c\xc9\x03\xeb\xcd\xdc\x0d\x31\x57\xa0\xa1\xb6\xb3\x19\xb7\x85\xee\x52\xaa\x64\xa0\x9e\x4d\x97\x93\x45\x52\x03\xa6\x47\x8e\x13\x1b\x26\xfe\x0d\xbb\x41\x03\xdc\xb0\xb1\x2f\x5a\x62\x15\xcc\x53\x5c\x37\x19\x86\xa3\x79\x7b\x64\x08\x67\x9d\x4c\x0b\xd0\xb5\xa8\xd5\xc3\x74\x91\x0e\xf4\x7d\x57\x8a\xa1\xbc\x95\x79\x14\xe5\x57\x28\x16\xdd\xde\x55\x2a\x5e\x67\x6f\x6f\xa3\x5c\xa4\x73\x99\x99\xef\x8e\xfd\x5a\xf5\x38\x2c\x56\xf3\xc1\xd4\xbd\x76\xcb\xe5\x58\xe5\xa3\x6c\x7a\x18\x52\x74\x8b\xe4\x73\xf4\x36\x8f\xde\x5b\xc8\x4c\x08\xc6\xc6\x7b\xaa\xb3\xfb\x49\x3d\xc7\x08\x27\xba\x2b\x09\x32\x45\xa4\xb3\xb0\xc4\xb9\x0c\xe3\xb4\xfc\x07\x23\xd3\xc2\x69\x12\xca\x3c\xbe\xb0\xf4\xd1\xd0\x8f\x54\x34\x9a\xb5\x14\x9a\xf5\x6c\x34\x03\x84\x2c\xa6\x2e\xfe\x21\x69\x69\xcb\x18\xb9\x31\x66\x5c\xf9\xea\x3b\xc6\x36\x2e\xa7\xf5\x79\x40\x4a\x50\xeb\xc9\x6c\xfa\xe0\xc8\x70\xee\xab\x95\x6f\x7a\x09\x95\xcd\xb3\xdf\x7e\x86\xe8\xb6\xb8\x8c\xf5\x66\x83\xdc\x57\xea\xaf\xc8\xb8\x88\xc4\x29\x17\x2c\xcb\x90\xb8\xc1\x87\x7c\xfc\xde\x1d\xf7\x22\x74\x0b\xfa\xc0\x3b\x1c\x09\xee\x81\xb8\xbb\x56\x4c\x0b\xc2\xf4\xc2\xae\x4f\x3b\x83\x74\xc7\xed\xf6\x56\x16\x23\x2e\x68\x13\xd4\x68\xb0\x4a\xec\xa7\xbe\x14\xc1\xdb\x93\x10\x26\x43\x6c\x41\x85\x4a\xa5\x25\x1e\xc0\x6d\xcd\x43\x48\xef\xc8\x48\xb4\xb8\xaa\xe1\x2b\x67\x18\x82\x1c\x19\x85\xbe\x6c\x6c\x50\xbc\x93\x49\x57\x0c\xea\x98\xf4\xd4\xd9\x40\xa4\x36\xd3\x5a\xa7\x3c\x34\xcb\x91\x2a\xfc\xdc\xa6\x5d\x5b\xc7\xdd\x9d\xa7\x74\xae\xfb\xa9\x9e\x72\x69\x4f\x24\xa0\x0d\x23\x25\xe2\xb6\xf3\x44\x6f\x21\x39\xa5\xde\xf1\x85\x7e\xeb\xc4\xec\xd6\xec\xe9\x49\x64\xe0\xd8\xb3\xcb\xdc\x4d\xdf\x49\xce\x76\xcf\xc2\xa0\x2d\x1a\x10\x89\xd9\x32\xf3\x1c\x56\x14\xee\xbf\x47\xdf\xce\x64\x79\x89\x61\x63\xb1\x14\x76\x1d\x42\x9b\x91\x2c\xe3\x5c\x85\xb0\x8b\x4d\xe4\xdc\xfb\x8c\xe1\x9d\xfb\x82\x0b\x17\x47\xe1\x79\x39\x2f\xd3\xed\xe0\x95\xa8\x4c\x15\x94\xcb\xd3\x59\x9e\x62\xe0\xd5\x2a\x16\xb9\xc7\xa2\x4c\x98\x00\x94\x9c\xea\xca\xfb\x82\xae\x52\xc0\x43\x05\x13\x8d\xf2\x8e\x92\x35\xb9\xfc\x84\xe9\x1e\x4e\x3f\xa6\xa8\x41\x37\x96\x99\x20\xe2\xce\xf8\xe6\x71\x0a\xf8\xe7\x83\x4e\x33\x9e\xb4\x90\xe2\xe1\x74\x93\x7d\x3d\x70\x0d\x8e\xe9\xbd\x46\xe5\x53\x99\xc7\xfa\x4d\xc5\x7d\x7b\xb5\x14\x1f\xed\xed\xb1\xdf\x54\x61\x10\x6c\x19\x5d\x7e\xa2\x3b\x49\x1c\x21\x2a\x5c\x2c\xa2\xcb\xdd\x37\xc3\x53\xd0\x1f\xad\xb7\x54\x73\xd0\x82\x75\x40\x73\x53\x05\x36\x62\x70\x46\xd5\xf4\x4c\x1c\x75\x3a\xfa\x9e\xc9\x92\x10\xb3\x00\xc3\x89\x26\x64\xb2\xe9\x14\x86\x44\xbd\x49\x92\x98\x66\x06\xc7\x74\x4c\x65\x11\x67\x08\x53\x8c\x61\x66\xa3\x0b\x1d\xc1\x00\x18\x1f\x33\x36\x7a\xb8\xf5\x68\x52\x41\xde\x7f\xa9\x6c\x46\xc8\x55\x63\xd9\xa1\xb4\x30\x26\xdf\xa4\xdb\xa0\xdb\x8e\x21\x92\x06\xbc\x5d\x2f\xde\x15\x85\x57\xbf\x9d\x24\x99\x4f\x8c\xf1\xec\xfb\x57\x6f\xdf\x3d\x7d\xf5\xee\x18\xb6\x4b\x3f\x3c\x77\x4e\x65\xd0\x01\xe6\x2d\x79\xba\xbc\x3d\x1a\x94\xe3\x07\x7b\xc0\x43\x44\xab\x6d\x4c\xb4\xfd\xba\x60\x71\x39\x56\xf0\xdf\x53\xf7\x22\x28\xb3\x7d\xd8\xce\xcf\x17\x91\x14\xad\x4a\xbb\x71\x3b\x47\x40\xb8\x77\x06\x05\xea\x60\x41\x65\xc8\x29\x61\x8d\x4d\x9e\x9c\xc9\xb8\xa8\xd0\xb5\xb6\x5c\x52\xe8\x06\xaf\xfc\xd9\xf6\xd5\x68\xe2\xc8\xc0\xc9\xa9\xf9\x1d\x5a\x16\xfa\x77\x01\xa9\x4d\x01\x1e\x23\x60\x2c\x0a\xd8\x20\x15\xe2\xea\x80\xf7\x1f\x3a\x41\x09\x8e\xd8\x58\xac\xb2\xb0\xc7\x9c\x14\xf2\xba\x11\x3c\x33\x91\x27\x25\x4e\x08\x8d\x42\xa1\x69\xfa\x12\x4b\x4f\xde\x0d\x2f\x2e\xd1\xf1\xb4\x77\x44\xe4\x3a\xa4\x76\x8e\x25\x9c\xe4\xfb\x5d\x0f\x71\x40\x23\x53\xe9\x9b\x49\xe2\xc2\xf0\x82\x37\xcb\xd5\x87\xd3\x39\x8d\x59\x6c\xc3\x94\xde\x93\x76\x5e\x93\x29\x29\xeb\x43\x2f\xdc\x83\x86\x34\xf6\x1d\x21\x18\x7a\xb6\xf6\x88\x85\x66\x1c\xd9\x87\x38\x43\x73\x42\x73\x1f\x62\x99\x53\x9c\xd3\xe1\x82\xdf\xf7\x13\x83\x9a\x19\xc8\xd2\x66\xc0\x75\x69\xef\x1b\x57\x17\x53\xf1\xcc\x17\x6f\xfe\x7e\xfe\xc2\x1e\x39\x46\x7c\x03\x7d\xbc\x90\x2a\xe2\xb1\xa1\x8f\xa5\xbb\x62\x69\x4c\xf0\x62\x99\xc8\x80\xc1\x80\xde\xcd\x96\xa0\x78\x10\x20\xbf\x23\xdc\x13\x4d\x61\x42\xdd\x92\xb7\xef\x7d\x46\x05\x5e\x80\xa5\xd8\xf7\x83\x41\x7e\x49\xc8\x2c\x28\x5b\x16\x6f\x1e\xf2\x0b\x63\x2e\x99\x22\x60\xe9\x69\xf7\x3c\x7b\x7a\xdf\xdc\x71\xff\xed\x48\x0e\x81\x53\xb0\x67\xf5\xb1\xca\xda\x56\xe1\x18\xf5\xa9\x64\xcc\xb7\xcc\x56\xab\x75\x78\x2c\x2d\x8c\x39\x46\x3f\x5b\x21\x69\xbd\xb8\xf0\xef\x05\xa1\xe1\xfc\xd2\x67\xd0\xf4\x29\x6f\x78\xb8\x6b\x85\xfb\x70\x3f\xeb\xbe\x4b\x57\xa8\xf6\xe7\x97\xa0\x97\x6d\xb8\x21\xba\xce\xc9\xbe\x09\xe7\xab\xf7\x9d\x41\x6f\xcc\x3f\x28\x7c\x24\x86\xc6\xe4\x5f\xcb\xcb\x1d\x13\x50\x72\x47\x07\x9c\xe4\x1f\x48\x9c\x1d\x1b\x79\x72\xc7\x86\xa6\x44\xa3\x2e\x02\x01\x7f\x01\x00\xfc\x4b\xc7\x53\x3b\x7c\xf4\x0d\x1b\xa3\xfe\x72\xbe\xd3\x1b\x2f\xd1\xe9\xd8\xe5\x7a\xa7\x0f\x10\x2f\xbb\xc0\x74\x85\xcc\xb0\xa6\x13\x73\x75\x6c\xef\x8b\xf1\xd4\x2d\xf7\xf6\x78\x22\x37\xc7\x41\xaf\x0f\xbd\xea\x1d\x6b\x5d\xe1\xc6\xa3\xb9\xbf\x14\x60\xd3\x54\x77\x0a\x06\x0d\x58\x69\x38\x75\xcc\x41\xb8\x3b\x29\x4b\x07\x99\xb1\x6c\x8a\xea\xee\x48\x74\xe3\xb0\x9b\xc5\x92\x7c\x63\xb8\xa1\x85\x01\xe3\x4a\xc8\xdd\xf1\xe9\x8b\x13\x8f\x93\x74\x20\x1e\xa7\x6b\x92\x64\x38\x69\x31\xeb\x4e\xe6\xfd\xf1\x72\x00\xea\xa6\x39\xe2\x36\xec\xf5\x18\x74\x07\xf1\x26\x42\x46\x74\xf6\x52\xe1\xe9\x1d\xe6\x25\xca\x02\x4e\x13\xd2\x1c\x98\xeb\x83\x71\x4b\x4b\x8a\x50\xa8\x2f\x64\xe9\xe3\x2d\x96\x0d\x76\xe7\x7a\xdd\x79\x71\x92\x5a\x0e\x5c\x00\xb0\xe7\xd5\x1d\xae\x12\x4b\x46\x6e\xae\x44\x9e\x94\x2f\x27\x60\xb5\xd3\x2e\xf9\xd6\xd2\x1f\x55\xbd\x5b\x8c\xdc\x22\x6e\xc5\xc5\xf4\xab\xe9\x14\x5a\xd2\xd8\x78\xd5\x13\xdd\x07\x50\xb7\xf1\xd5\xb6\x21\x90\x53\xc6\x09\xe3\x38\xe6\xb8\x27\x2e\x8a\x94\x66\xc2\x2e\x49\xfc\x18\x49\x1c\x06\x77\xa9\x26\x25\xde\x68\x0b\x03\x94\x01\x2f\x26\xd4\xa4\xc7\x26\x78\xd9\x87\x4b\xe2\x42\xd6\x58\xb2\x82\xa4\xcb\x0f\x32\x33\x7a\xdb\x47\x89\x85\xdf\x3a\x48\x89\x72\x58\x4b\x16\x54\x3b\xb6\x08\x4d\xf9\xb0\x94\x07\xc9\xc4\x40\x31\x3d\x21\x73\xcb\xac\x9b\xab\x67\x4f\x1d\x66\x7c\x18\x30\x63\x26\xeb\xde\x8f\x29\xe7\xe6\x36\x7f\x3e\xeb\xc3\xb6\x27\xd1\xfc\xc4\xd4\x91\xbd\x0f\xe4\xa9\xe9\x04\xc4\xfd\xc0\x14\xe1\x4f\x8a\x02\xd1\x5d\xc2\x6a\x4e\x1d\x32\xb9\x98\x34\xa7\xa4\x1c\x9b\xfa\x07\x5e\x84\x42\x36\xbd\x5c\xf0\xc0\xcb\x87\x65\x7d\xdc\xca\xd3\x99\xea\x70\x86\xb1\xe4\x91\xab\x96\x97\x2f\x41\x40\x0a\x92\x12\xff\xdf\x55\xfd\x5d\x80\xc8\x7a\xce\xa7\xc7\xbb\x4e\x3a\xfb\xeb\x08\x5a\xcf\x98\x37\x5e\x7e\x58\x9e\x81\x6c\x7c\x85\x83\x34\x09\x12\x8b\x4f\xe7\x6f\xac\xe1\x6b\xc4\x14\x77\x0d\xc2\x66\x7d\x79\x2f\x75\x36\x9a\xf3\x11\xc6\x8b\x41\x52\x2e\x7b\x24\x68\x41\xa9\x36\x33\x17\xaa\x27\x58\x49\xdd\xf4\x91\xb9\x9a\x3e\xaf\x6d\xe2\xea\x7c\xc9\x6e\x40\x4c\x0b\xc0\x51\x97\xd3\xc9\x7c\x68\x3c\x4b\x98\x56\xc8\xbb\xa9\x6d\x45\x6c\x93\x86\xef\x99\x36\xee\x7c\x6d\xb9\xbc\x13\xe9\x22\xe9\xf9\x9a\x37\x4e\x29\xba\x6b\x08\x01\x29\x79\x3a\xfb\x89\xac\x9a\x99\x5e\xa4\x24\x80\xcf\x3a\x38\x18\x96\xd3\xcc\x6c\x0b\x8a\x55\x75\xb0\xdd\x28\x8e\x3d\x4e\x89\xd5\xae\x13\x61\xc0\x65\x72\x3c\x13\x70\xbf\x8b\xc5\x5d\x6f\x73\x17\x94\xf5\x26\x7d\xd0\x7d\xa8\x72\x81\x4b\x3c\xec\x2f\x5c\x22\xbd\x84\xc9\x35\x1c\x14\x62\x6f\xd3\x68\xa6\x20\x1f\x6c\x55\xe9\x7a\x21\x0a\x4e\xf2\xf3\x46\xb4\x58\xdc\x32\x22\x1b\xdb\x50\x16\x36\x12\x0f\x08\x11\x8a\x9b\x5b\xc1\x58\xe9\x3a\x5a\xdc\x57\x19\xdd\x66\x95\xe4\x0a\xad\x79\xfa\xa8\x6f\x5e\xe5\xa6\xd8\x6e\x4c\xa1\xe1\xf8\x04\xd5\x7e\xbc\xe2\xcb\x69\xde\x15\x9c\xae\xf6\x48\xd5\x5f\x83\x12\x46\xaa\x93\xb7\x2c\x88\x81\xab\xca\xdd\x5f\x72\x01\xe5\xef\x33\x71\x02\x7d\xf5\xe6\xe9\xb3\xe7\xc9\xde\xcd\x66\x4f\xc7\x24\x75\x85\xfa\xd3\xbb\xcf\xb3\x8c\x1f\xe1\xde\x70\x66\xa4\x6b\x1f\x47\x54\xbd\x3f\x1b\x4e\x4c\x12\xeb\x01\xef\xe9\xfd\x59\xde\xee\x14\x73\x63\x8c\xfa\x0d\x9a\xbe\x08\x71\xfa\x7d\x7b\x5b\x23\x60\xf3\xc4\x25\xaf\x80\xa7\xb0\xa3\x9c\x0d\x5f\xf8\x0e\xe7\x66\xc3\xf7\xa3\x6f\xd7\x17\xbd\xe9\xd8\xdb\x18\x57\x9d\x0c\xe0\xf3\xe1\x64\x90\x55\xc8\x24\x47\x31\x79\x27\x1e\x4e\x92\xbd\x9f\xb1\x85\x08\x2d\xb2\xe7\x71\x3b\xaa\xe2\xd9\x8e\x51\xc9\x69\xc1\xb2\xcb\x98\xb9\xdb\x25\x02\x91\x29\xa4\xd8\xa6\x12\x2c\x3e\xa7\xf1\xb3\x40\x8c\x53\x0f\x02\x9b\x0d\x53\x40\xc7\x64\x8e\x0a\x58\x8c\x0c\x7b\x6d\x12\x21\xc5\x2f\x77\xe3\x4e\x92\x43\xdc\x43\x0f\x1e\x99\x32\x9a\xcb\x1b\x6a\x3a\xb3\x03\x14\x33\xd3\xa1\x2d\x3f\xa0\xf3\x8f\x1a\x4c\x69\x83\x11\x31\x95\x3b\x06\x65\xc3\x1f\xe4\x92\x98\x47\xb5\x6c\xc8\x1c\x3b\xd3\x7b\xcb\x9d\x0e\xad\x50\xdf\x80\x84\x91\x17\x9e\xc2\x07\x06\xb6\x7d\x04\x33\x9a\xbf\xea\xbe\xd2\xe7\x80\x85\x42\x93\x7f\xa2\x53\x62\xea\xcb\xed\xad\x9c\xe0\xc8\xbc\xa8\x5e\xd2\xac\x7a\xd6\x95\x53\x89\x0a\x23\x18\xc7\x2e\x5b\x1d\x52\x56\x27\x98\x02\xb0\x69\x76\xde\x11\xe2\x62\xf3\x07\x7c\x0c\xe9\x9d\x74\x48\x22\xc9\x1e\x14\x3d\x6f\x86\x97\xc3\xae\x6f\x39\x24\xfb\x9a\xcc\x7d\x00\xe5\x29\xbb\x1d\x70\x4f\x33\xbe\x82\x0e\x9c\xeb\x68\xc0\x51\x0f\xbf\x68\xe5\xa4\x5f\xcc\xec\xf4\x52\x25\x49\x38\x05\x48\x4e\x31\xc1\x85\x4e\x12\x52\x36\x28\xcd\x5b\x14\xe9\x42\x12\x64\xea\x9e\xcf\x93\x43\xf9\xd5\x69\x0d\xa6\x4e\x80\xe5\xac\xf8\xc8\x6a\x57\xaf\x71\xec\x89\x55\xc3\x08\x02\xcc\xf3\xd3\x26\xf9\xdc\x0d\x71\x6e\xf9\x48\x6b\x18\x6c\xb2\xf8\xea\xdb\xe3\x37\xcf\xbf\xfb\xfe\x0f\xcf\xbf\x4e\x0a\x0f\x1e\xd8\xaf\x82\xa1\x77\xf0\x46\x48\x27\x1b\xdf\x62\x77\xec\xaf\x70\xfb\xee\x4a\xe6\xed\xda\x2b\x4a\x4d\x7b\x08\x70\xaf\xed\x14\x34\xdf\x3f\xff\x6a\x0d\xcc\xa6\xec\x4f\x79\x63\x21\x09\x2f\x06\xc4\x8a\xf6\x93\xce\xbb\xdd\x84\xef\x7b\xbf\xc0\xe7\xf8\x6c\xae\x66\xf0\x1f\x71\x2c\xf9\x0f\x1e\x9d\x8c\x00\x0c\x4c\x5e\xe7\xcb\x4e\x5c\x6d\x0f\x23\x24\x51\x7c\x31\xe1\x4c\x6c\xf1\x1d\x4a\x06\x7d\x20\xfa\x8e\xe6\xe5\xd1\x04\xfe\x89\xaa\xa5\x76\x4c\x46\x7e\xf3\x32\xa1\x00\x7f\x7b\x6b\xca\xc0\x42\x25\xb4\x2a\x8c\x29\xce\xd5\x37\x93\x97\xf4\xee\x2b\xb9\x79\x30\x1a\x34\x25\xe2\x2e\xb9\xd8\xdd\x25\xd8\x62\x35\xe3\x8f\x55\x54\x18\x01\x0e\x26\xb6\xeb\x70\xe0\xba\x61\x1b\xb1\xbd\x25\x7a\x34\xfb\xec\x18\xf8\xe1\xe2\x18\x93\x8e\xfb\x78\x62\x43\xb4\x3e\x3e\x64\x92\xee\xf4\xd6\x3b\xc7\xa3\xc1\x71\xe7\x33\x72\x9d\xe2\x06\x7f\xb5\xe0\xe8\x76\x18\xc8\x96\xe8\x4b\x15\xa4\x5f\xa2\x63\xd0\x39\x03\x22\x69\x0f\xdd\xb4\x76\x14\x0e\x98\x3e\x64\xb0\xe9\x71\x30\x86\x76\x2f\x60\x9e\x00\x4a\xb0\x7f\xcd\x33\xde\x60\xc1\xfa\xe0\x92\xf1\xd0\xcb\xa4\x87\x76\x3e\x9d\x0f\x83\xa1\x97\x47\xcf\x5e\x19\x34\x14\x41\x47\x98\x12\xc5\x20\xe8\x82\x66\x55\xb7\x06\xfa\x73\x89\x42\xe6\xcb\xe8\x89\xe9\x83\x0e\xad\x2c\x67\xbd\x01\x9f\x7a\x35\xdd\x7e\xc3\x6a\xc0\x2c\x08\x23\x62\x78\x2f\x6a\x37\xa3\xc3\xa3\x07\x47\xef\xf1\xe5\xd0\x51\x64\x7e\x03\x9f\x29\x8a\xb9\xac\xbf\xe3\x18\xb9\x6e\x3b\x5f\x8d\x06\xcc\x59\xc2\x3a\x9f\x21\xef\xe4\xb0\xd1\x67\x3b\xf3\xb3\xe9\x72\x3c\x20\x1f\x3a\xc0\x77\x86\xa6\x18\x77\xee\xb3\xe8\x18\xf0\x3d\x56\x3b\xc7\xd4\xc5\xe3\xf8\x33\x87\x47\x55\x8a\xe1\xa0\x3f\x28\x5c\xbd\xa1\x49\x6c\x20\x43\xc3\x7c\xde\xf0\x1a\x36\x6c\x69\x1f\xe5\x24\xf5\xbf\xeb\x5e\x92\x49\x94\xbb\x6e\x3e\x33\x68\x47\x38\xff\xed\x8e\xc7\xa4\xeb\x9b\x03\x1c\x72\xbe\xd0\xa2\x90\xf9\x1c\xfe\x17\x07\xf4\x25\x27\x3a\xe7\x1c\x96\x41\xcd\xf0\xd9\x5f\xdf\xe0\x5b\x79\x0d\x1c\x86\x7a\xae\x48\x68\x9b\x55\xe2\xdd\x94\x83\xb2\x92\xe1\xf9\x68\xba\x9c\xd3\x72\xa4\xe5\x1f\xda\x08\x60\xf3\x4e\x87\xec\xf7\xf7\x33\x0a\x43\x4d\xef\xed\xe5\x6a\xef\xe5\xe8\x7c\xe8\x76\x26\xf6\xdb\x4e\xec\xa7\x8b\x61\xe2\x4f\xbd\xdb\xdb\x90\x85\xd9\x3f\xe8\xbd\x6a\xb0\x80\x0d\x9a\x95\xd0\x41\x40\x28\x64\x1e\x07\x3f\xfb\x33\x74\x20\x8e\x1c\x5f\x2c\x16\x1e\x14\x76\x71\xf2\xda\xa0\x8f\xb0\x09\xf2\x60\xf3\x0a\x89\xa5\x5b\x41\xc6\x7c\x3a\xc3\x0d\x0d\x36\xec\x0c\x5c\x12\x94\x92\x31\xe6\x3f\x89\x4f\x5b\xc9\xcc\xa9\x22\x51\x94\x5a\xe2\x2f\x8e\x1d\x64\x8b\x2b\x63\xf4\x29\x87\x2f\xa3\x6d\x4d\xdc\xcc\x79\x60\xda\xec\x14\xd7\x4f\x3c\xa4\x44\x09\xa6\xee\x54\xe0\x10\xeb\x86\xc8\x89\x33\x14\x29\xf9\x88\xbe\xb8\x86\xa8\x8b\x2c\x7e\xef\x93\xd4\x72\x67\xe1\xff\x68\xc9\xff\x7f\x60\xbe\xc2\x1e\xd5\x9d\x34\xe1\x48\x98\x6a\x71\x4c\xac\x9b\xb8\x85\x0f\x4d\x6e\x47\x49\x2c\xc8\xbc\x6c\x97\x8f\x9d\x74\x56\x62\x02\xae\x16\x3a\x70\x9e\xf1\x5f\xed\x42\xd8\x82\x25\x8b\x34\xad\xbb\xfb\x80\xed\x94\x17\x5d\x90\xfe\x14\x8b\x81\x12\x27\xc1\x02\x0c\xba\x9c\x3d\xd2\xd8\xf2\x1d\xa6\x2f\x32\x07\xcb\x4b\x7c\x89\x5a\xf8\x7a\xc9\x1e\x7d\x86\x73\xe4\xf5\x2e\x3a\xf1\xc6\x77\x84\xe8\x13\x60\x48\xae\xc6\xba\x63\xdc\xc3\x0c\xaa\x3b\x3f\xcc\x87\x3b\x9f\xe9\x81\x70\xe5\xe2\xce\x62\xba\x33\x87\x15\x7d\x74\xb2\x46\x37\xe3\xb0\x73\x47\xa1\x3f\xaf\xee\xbc\x11\x58\xe4\x98\x54\xed\x98\xa6\x30\x5b\xcc\x60\x1d\xe9\x6a\x49\xd2\xca\xa6\xee\x0d\xa8\x1a\xa6\xd0\x26\x77\x80\x76\xeb\x1b\x67\xf6\xba\x74\x89\xb7\xf1\x0d\xcf\xe0\x28\x83\x67\x88\x99\x43\x01\x98\xe4\x68\xeb\x2a\x38\xbe\x0b\xeb\xc5\x61\xf4\x6b\x2b\x52\x43\xd5\xd3\x6c\x3b\x0e\xad\xfe\xdb\xc1\x3b\xa5\x58\xc9\xf8\x07\xc7\x8b\x31\x89\x2d\x1d\x2e\x6d\x9b\x34\xc8\x96\x02\x37\xff\x12\x31\xc0\x14\xcd\x1c\xd2\xcc\x69\x54\xc1\xb8\xc2\x91\xb7\xbe\xe4\x14\x33\x5b\x09\x15\xee\x78\x6e\x9c\xb3\x7f\x87\x56\x66\x6d\x4c\x5c\xf0\xb0\xdf\xd9\x31\x19\xfa\x87\xbb\xdd\x11\x07\x2f\x3a\x0b\xad\x76\xf8\x97\x3b\x30\x71\x2b\xb5\x49\xd3\xc5\xe8\x40\x8e\x99\x82\x02\xd9\xe7\xb1\x0e\x1d\x1f\xca\x76\xc8\x45\x30\x8e\xfd\xe5\xd6\xdb\x47\xb1\x2b\xfd\xbc\xbb\x8e\x96\xcd\xc8\xd3\x21\x43\xc5\xd4\xa9\x91\xa5\x96\xba\x97\x32\xae\xc2\xe9\xa6\x53\x08\xe0\x04\xbd\x1b\xa7\xf3\x70\x6a\x71\x32\x64\x3b\x2b\x5d\xa5\xee\x15\x13\x3f\x80\xbb\x51\x0a\x2e\x06\x75\xf2\xc1\xf9\x48\x4d\x07\x83\xa0\x1a\x5e\xd9\x71\x93\xbf\x6b\x24\x49\x2d\xc5\x2e\xb7\xb7\xbc\xbf\xdb\x7a\x1b\x84\x7f\x0e\xe9\x5f\x99\x43\xe5\x72\x27\xf4\x18\x61\x5e\x5f\x93\x73\x66\x2d\x72\x9b\x18\x7e\xc9\xc8\xb9\x32\x5d\xae\x66\xde\x24\xdd\x77\xfc\xb9\xb4\xc3\xd7\x89\x83\x78\xc8\x85\x6e\xcd\x62\xd1\xfd\xd2\x1c\xd4\xce\x4a\x6c\x72\x87\x6b\x32\x7f\x75\x72\x92\xa6\x03\x4c\x44\x5f\x16\x07\xab\xcf\x26\xde\x78\x4a\xae\x5b\xd6\x9e\xdf\xbe\x3d\x9b\xbe\xff\xc4\x1b\xd4\x4f\x7c\x70\x5d\xe5\x46\x52\xd7\x96\x1a\xec\x61\x78\x85\xd8\x2e\x38\x06\x08\x85\x66\x41\xdb\x8b\x14\x3a\xe6\x70\x00\xef\xbb\xcf\xd0\x9b\x87\x31\xcb\xf8\xfd\x28\xb8\xe2\xfb\x57\xf4\x03\x1b\xf9\x94\x7e\x18\xcc\x9b\x5e\x97\xb6\xf5\xe3\xed\x62\x9d\x77\xa2\xfd\xe9\x64\x47\x58\xde\xa1\x1a\xa5\xcc\xd5\x14\x98\x96\x7e\xc5\x37\xe6\x27\xd9\x44\xf2\xcf\xdd\x24\x71\x92\xf5\xba\x69\x92\x3c\x0a\xa8\x39\xa6\xd9\x93\xba\xfe\x7c\x1e\x51\x92\x22\x9b\x19\x65\x80\xda\xe3\x6a\x2c\x62\x92\x61\x37\x8b\xbe\xfb\x51\x51\x7a\x4b\x41\xd7\x3f\xf5\x3a\xdf\x3f\x66\x77\x0c\x49\x19\x9c\xf7\x6c\xda\x3e\x7d\x0b\x5f\x45\xe0\x51\x31\x9e\x78\xc3\x18\xdc\x69\x79\xaa\x41\x3f\x73\xde\x4d\xd3\xc5\xea\x70\x4c\xc7\x2b\xef\xcc\x01\x18\x39\xed\xa3\x34\xbd\xe4\x28\x1a\x20\x7c\xb6\xa4\x2f\x60\x19\x96\xdc\xbf\x82\x3a\xa6\x6b\x90\x54\xa1\x3d\x91\x37\xb4\xa6\x7e\x8a\x0f\xb5\x0b\xa2\x11\xe8\x20\xa3\x51\xe2\xc3\x31\xee\x87\x46\x07\x23\xf2\x41\xe4\x67\x1f\x8e\x3a\xee\x75\x69\xa0\x4b\x85\x9d\x80\xd2\x71\x2b\x4c\xc4\xb7\x65\x29\xdc\x55\x94\x41\x94\x24\x4d\x40\xa6\xff\x61\x61\xb7\x50\xe6\xe3\xe1\xdb\xdb\xfc\x42\xed\x02\x05\x43\x4d\x59\x59\xf2\x2d\x98\x55\xf4\xb2\xc6\xc3\xf5\xde\x14\x64\xfa\x83\x98\x71\x1d\x1a\x50\x94\x76\x94\x5e\x5a\xac\xd2\x50\xab\x8e\xd1\x83\x57\xd8\x59\xe9\xa0\x5b\xcf\xdd\x5b\x72\xbe\x45\x4c\x32\x80\x09\x1f\xa6\x88\xcf\xc8\xb8\x80\x52\x2b\x9d\xcd\x53\x0c\xdf\xda\x4d\xf0\x37\x99\xd6\xd8\xd3\x75\xa6\x3e\x5e\xb4\xe4\xdd\xaf\x6d\x3d\xe7\xfd\xa2\x56\xb3\x93\xf1\x67\x77\x36\x6e\x39\xef\x9d\x6f\xb5\xa1\xd9\xfe\x82\xc0\x72\x90\xbe\x2f\xb4\x3d\xe8\x24\x77\x17\xb9\xbd\x45\x4f\x7f\x77\x16\x13\xa7\x63\x4e\xe7\xed\x4f\x8d\x5b\x53\xfe\x1a\xcf\x2c\x22\xd9\xd8\x59\xc0\xff\xa7\xc8\x09\x73\x2d\x09\xbe\x53\x84\x82\xb4\xbf\x8b\x2a\x96\xaf\x73\x2f\x90\x5d\xc9\xed\x3c\x47\xb6\xc0\xdd\x95\xc1\x5c\x06\xb8\xbd\xc6\xa3\x71\x37\x21\x7c\x56\x6a\x91\xc0\xb7\xa5\xd3\xd9\x25\x08\x11\x68\xe2\x05\xb4\x7a\xda\x1d\xef\x2c\x41\xc7\x9f\x9e\xec\xb8\xe5\xac\x35\x1e\xee\x73\x31\xaa\x89\xb6\xc6\xd8\xdd\x79\x35\x95\x37\x9c\x4e\xa1\xc5\x59\x77\xb1\x23\x63\x37\x87\x9d\xbe\x6b\x55\xb1\x73\x32\x5d\x4e\x06\xee\x1b\x55\x8c\x24\xb2\xf5\x55\xaa\xed\x4b\xe2\xfc\x4e\xad\x5b\x99\x37\x24\x73\xcf\xd7\x47\xdf\xf1\x86\xe2\x48\xa9\x50\x03\x7f\xe0\x59\xcb\xa2\x2b\x24\xfb\x7c\x94\x7d\x21\x71\x41\x7e\x83\x33\xef\xcf\x46\x97\xfe\xa5\x5c\x60\x64\xe5\x38\x45\x71\x33\xb2\x9f\xf1\xdf\xfb\xaa\x88\x3d\x16\xe1\x3d\xe1\x1e\xe8\x54\x1a\x72\x21\x49\xec\x9b\x7d\xbe\x1e\x92\x1c\xf4\x28\x47\x59\xf8\x78\x12\x9d\x7b\xda\xa3\x54\x76\x4a\xe8\x63\x07\x0c\xbe\x88\x9c\xba\x4a\xfc\x5a\xb2\xac\xfc\x9e\x3d\xc0\xa6\xae\xc6\x24\xbd\x10\xdb\x32\x19\x2f\xa7\x9c\x3e\x6e\xf4\x12\xe2\x11\xd0\x31\x42\x4b\x19\x6e\x1a\xc3\x32\xb1\xe0\x94\x4b\xbc\xef\x5f\xbf\x7b\xf1\xfd\xab\xb7\xbe\x0f\x09\xbc\x66\x8a\xda\xcd\xa3\x79\xb9\x3b\xd7\xd7\x4e\x71\x9b\x53\x4e\x67\xd3\xe5\xa5\xbd\x77\x2a\x41\x06\xfc\x80\xa5\x14\x3f\xcd\x95\x01\xff\x5b\x22\xff\x62\x47\x91\xbd\x2e\xd0\xe9\xfe\x25\x82\x93\x7a\x44\x9e\x2a\x9c\xeb\x2e\x6e\x35\xb8\xed\x22\x7c\xf0\xb1\x07\x6c\xc8\xbe\x33\xcf\xe0\x6e\xbc\x57\xa5\xe2\xce\x67\x9b\xcf\x87\xc3\x02\x13\x11\x08\x66\x5e\x79\x75\x54\xae\x34\x31\xde\x10\xc4\xf3\x41\x86\x44\x91\xc9\x24\xc6\x29\xda\x6b\x11\x0f\xa9\x5a\x4e\xce\x27\xd3\xf7\x13\xf9\x42\xef\x8c\xec\xda\x58\xfc\x02\xeb\xf3\xf8\x53\xdb\x23\xaf\x7f\xb8\x98\x9f\x90\xf5\x7b\x0f\x85\x01\x5f\x22\xce\xed\x13\x75\xcc\x1d\x4d\x46\x8e\xd9\x9b\x03\xea\x58\x59\x44\x8e\x7d\x4c\x8e\xe3\x1b\xaf\xcd\xec\x4a\x49\x6e\xfd\xc4\x87\xb6\x61\x4c\x60\x4b\x23\xd9\xa9\x67\x29\x39\xe7\xc9\x7c\xda\xf1\x59\x81\xc9\xc1\x36\x08\x85\xcf\x62\x87\x3c\x62\x12\x80\x8f\x7d\x1d\x2c\xdd\x77\x9b\xe6\x41\xa7\x11\x3d\xa8\xf5\xc9\xdb\x59\x0f\x4f\xa3\xd5\x14\x8b\x7e\x3a\x6f\xc0\x28\x94\x0f\x75\x84\xbf\x73\xfa\x42\xa2\x16\x0f\x2a\x2f\x9d\x54\x68\x5e\xce\xae\x53\xa8\xdf\x85\xb7\xb8\x9e\xc7\x07\xa7\x3f\xb8\x58\x99\x37\x5c\x1a\xa7\x54\x09\x0f\x35\x66\x3d\xe9\x17\xb4\x90\x14\xda\x3b\x85\xb2\x73\x3b\x18\x97\x0b\x3b\xed\x42\xcb\xef\x39\x92\xca\xd6\x71\xc4\xf7\x25\x1e\x3a\x83\xfc\xf6\x8a\x3b\xf9\x61\xc5\x80\xd0\xe8\xe1\xab\xa0\x35\x51\x7a\xf7\x20\xdd\x30\x84\xcb\x7b\x0f\x69\xe9\x17\x9e\x06\xeb\x1b\x4e\x7d\x98\x96\xfb\x62\x35\x97\x5e\x24\x1e\xe2\xfb\xec\xe3\xe4\xcd\xad\xb1\x03\x79\x0b\x4b\x2e\x46\x41\xa2\x92\xdc\x2f\x63\x60\x68\x87\x57\x72\xe8\x51\xec\x8d\x37\xec\xe2\x2f\x22\xb4\xb9\x31\xcc\x90\xcd\x23\x2d\x0b\xcf\xe5\x38\xf3\x9c\xb0\x1d\x7d\x1a\x7f\xfb\x98\xf3\x3c\x31\xc0\x14\xbb\xa3\xb0\x8f\x11\x87\xf8\x6e\x2f\x7f\x40\xe3\x66\xf6\x1b\x47\xaf\x62\x3b\xdd\x22\x68\x09\x4d\xa7\x5f\x99\x9c\x6f\x5f\x89\x06\x28\xe3\x98\xf3\x5e\xce\x1f\xf1\x9c\x97\x02\x9f\x48\x1e\x6f\x14\xdc\x85\x25\x4a\x77\x23\xe6\xe7\x06\x86\x43\xbe\x13\x87\x88\x99\x3c\x62\x1f\xa4\xe2\xb1\x1b\x82\x6d\xf5\xb7\xf0\x05\x5e\x92\xcf\xd1\x26\x76\x07\xe4\xe5\x19\x4c\x85\xd4\xeb\xd9\x96\xbf\x91\xd5\x88\xc1\x58\x0c\x22\x91\xa6\xee\x73\x11\x4e\x89\x6f\xf8\x6f\x55\x8f\xa3\x13\xd8\x9e\x9a\x24\x5b\xe2\xa9\x91\x0c\xfc\x82\x74\x93\x6b\x36\x46\xc1\x3c\xe6\x91\xee\x92\x0a\x91\xbc\xbd\x35\x79\x09\x3e\x6a\x49\xf5\x22\xf5\xda\x3e\xe4\xd0\x4f\x1c\x6e\xd7\x6f\xf6\x3f\x87\x42\xc5\xa2\xe3\x62\xdb\xa3\x8c\x79\xe0\xea\xf1\x09\x95\x0e\x38\x43\xb4\xbb\x2d\x8c\x61\x8a\x6a\x4a\x50\x4f\xb8\xb9\x6f\x51\xf9\x22\x47\x89\xce\xb7\xfc\x56\xc3\x15\x79\xbe\x38\x7d\x4d\x9c\x6d\x3e\xa5\xc5\xe0\x9b\xac\x0c\x48\x97\x7b\xe9\xde\x58\x73\x12\xc7\x62\xe7\xab\x01\xdd\x79\xba\xd3\xd5\x98\x6a\xf1\x38\xee\xf6\x40\xf3\x70\x70\x99\x27\x37\x85\x42\xf3\xb0\xb3\x51\x01\xc2\x73\xd0\x4f\x41\xb7\x72\xdc\x36\x84\x7e\x1c\xf8\x76\x6a\xce\x2b\xe1\xfc\x9b\x89\x79\x1d\x78\x48\xf6\x19\x73\xbc\xdd\x40\x48\x6d\xbc\x88\x1f\x0e\xf0\x5e\x8b\x57\x81\x79\xcc\x6f\x41\xe7\xe2\xeb\x9a\xec\x51\xf5\x09\xca\x10\x9d\x78\xa3\x45\x83\xf6\x51\x4a\xe1\xd9\x50\x9b\xfc\x66\x52\x2c\xea\x17\xcc\x16\x8b\x98\x56\x0c\x53\xd7\x9d\x76\x87\x9d\xd8\xc4\xa8\x26\x08\x2f\xe4\xc6\xde\x7e\x3c\xb1\x80\xf4\x41\x9a\xcd\x2c\x97\xe3\xb1\x63\x2c\xc4\x6f\xac\x6c\x85\x43\x5b\xb2\xe3\xe2\xcf\x3b\x0a\xc6\x38\x88\xb4\x91\x5d\x99\x23\x3e\xef\xe4\x75\xc3\xe9\xab\x7b\x1d\x29\x37\x8e\xc9\x79\xca\x8a\x21\xe7\x12\x92\xd9\x43\x06\xc5\xa4\x9e\x3b\xd7\x8d\x85\x07\xb8\x88\x78\xe6\x16\x31\x2a\xd7\xa3\x09\xac\x66\x63\x63\x58\xc5\x84\x40\x33\x24\xb4\x09\x4b\x51\x88\xc7\x96\xaf\x2a\x03\xa6\x4a\x88\x5b\xbf\x5a\x87\x84\xa1\x47\x28\x91\x53\x38\x71\x59\xf4\x30\x80\x82\x0f\xfa\xef\x5f\x18\x0f\x2b\x43\xde\x76\x25\x82\x49\x8d\x63\x65\x78\xce\xc8\x58\x77\x5c\x65\xa5\x91\xa1\x6d\x67\x0e\x71\x53\x6f\xfe\xfc\xb0\x0f\xbb\xce\x5b\x68\x63\xce\x20\x00\x5c\xe7\x20\x78\x95\x77\xb3\x69\x99\x8f\x6c\xce\x33\xdc\x96\xf8\x38\x98\x6a\x31\x86\x41\xcd\x42\xcf\xe7\x34\x07\xa4\xf6\x5e\x15\x56\x68\xb9\x5c\xe9\xfc\xbe\xbd\xd5\x1f\x1b\x92\x29\xc9\x60\x34\x87\x2d\x74\x6a\x64\x59\xe2\x24\xfc\x6f\xf0\x20\x3c\x25\x8c\x74\xd4\x71\xb1\x4a\xc8\xa5\xb1\x16\x2c\x0e\xef\x36\xc5\x62\x0b\x41\x0a\x60\x8d\x60\xd3\x60\xca\xc6\x40\x7a\x90\x81\x89\xec\x16\xc9\xf8\xf0\xb2\x24\x69\x7b\x6c\x05\xc2\xd0\xb8\xd1\x40\x0c\x81\x61\xb9\x31\xf8\x61\x5a\xda\x75\x28\xb4\x89\x9b\x1e\xbd\xb6\x43\x6b\x67\x82\xab\x61\xa8\x18\x2b\xe1\x93\x9a\xbb\x02\x24\x29\xae\x16\x09\xe6\x94\x39\xb0\x95\x5b\xf6\x27\x0a\x05\xa4\x44\x38\x3b\x43\x80\x87\xb6\x8a\x37\x87\xb6\x4f\x38\x6f\x79\xe1\x33\x16\x46\xec\x49\x62\xe1\xb5\x23\x7f\xf5\x4b\xf4\x3d\x53\x13\x6a\x53\x55\xfd\x4a\xbe\x4a\x87\x50\x51\x2c\x2e\xb3\x89\x4a\x85\x70\x46\x0b\x9f\x35\x5d\x66\xa2\x94\x4d\xb8\xac\x26\x87\x7e\xc3\x59\xe8\xb2\xd1\xb5\x5f\x2f\x54\x6e\xe4\x84\xcc\x07\x56\x35\x67\x7a\xcd\xb0\x7e\x92\x6a\xc6\xa3\x6e\x40\x8d\xa0\x32\xde\x1a\x07\x2d\x51\xef\x76\xc3\x21\x03\xf5\x3d\x13\x21\x9f\x78\x59\xa0\x42\x48\x71\xec\x2a\x10\x74\x98\xa0\x02\x43\x18\x8f\xd8\x5b\xac\x60\xb8\x9c\x5b\x5c\x2f\x38\x01\x91\x52\xdd\x96\xd8\xd0\x1d\x0c\x03\xe9\x20\xe3\x17\xd3\x7d\x0c\x46\x5a\x53\x48\xda\xe5\xcf\x62\xd1\x81\xc3\x8f\x0f\xb2\x6a\x79\x75\xe2\x10\xf0\x68\x60\xa1\xa2\x87\x3b\x17\x24\x6e\x2e\x52\xc5\x6d\x61\x8f\xa8\x78\x22\xa9\xe7\xba\x85\x68\x95\x57\x17\x6e\xb8\x85\x0b\xda\x30\x12\x3d\x00\x82\x8c\x48\x3b\x43\x07\x5d\x2b\xf4\xda\x43\x67\x74\x39\xa9\x69\x5e\x29\x70\x8d\x70\x16\xc6\xd4\x43\xdb\x1f\x61\x2c\x8b\x58\x88\x00\x13\x79\x1b\x3d\xe7\x59\xf3\xcd\xca\x03\x4f\x75\x6d\xba\x80\x14\x08\x4e\x83\x89\x95\x9a\x01\x06\x1b\x8f\xe6\x6d\x97\xa8\xdd\x13\xbc\x40\xd3\x73\xb6\x99\x37\x75\x64\xa6\xeb\xfb\x37\x8f\x13\x25\xd1\x2a\x66\xe5\x72\x2b\xec\x9e\x37\x29\xe2\x54\xe7\x61\x60\x63\xd3\x98\xde\xcd\xd2\x4a\xd5\xca\x93\xa4\xae\x58\x8f\x33\xe4\x17\x82\x44\xee\x4a\x41\x35\x5e\x95\xe8\x65\x01\x7b\x48\x92\x91\xc6\x57\x3c\x62\x4b\x9e\x3a\x66\x8e\x63\x63\x27\xee\x9d\x88\xdf\xff\x01\x02\x8f\xc6\xf1\x4e\xd4\x9d\xef\x1c\xd3\xe8\x1d\xc7\x6d\xcc\xdf\x11\x03\xf1\xb8\x2d\x16\xe2\x69\xf3\x72\xdf\x48\x3c\x70\xfb\x69\x91\xf7\x6e\x57\x3c\x61\x2d\x61\xb7\x8c\x8a\xa2\x5f\x1f\x68\xbb\x71\x6b\x58\x2e\x1e\xb6\x68\x35\xd4\x0f\x19\x74\xee\xe3\x8e\x56\x9e\x25\xeb\x51\x47\x19\x7d\xd6\x07\x89\xb6\xea\x78\x9f\x22\x2a\x55\xd8\x5e\x5b\x37\xd7\x34\xad\x49\xd9\x79\x58\xf8\xf3\x8e\x3c\x8c\x90\x36\xbf\x10\x2b\x60\x6d\x82\xdd\xf6\x4a\x7f\xd1\x89\x9b\x24\xa9\x53\x3c\x91\x1c\x1e\x9a\x19\xe5\x6f\x62\xb5\xda\xb1\xe9\x74\x5a\x56\x0e\xe0\x89\xac\x7e\x6c\x6b\x53\xc5\x17\x72\xec\x1c\x33\xa7\x7c\x8a\xf0\xa1\x7b\x46\x11\xf4\xc6\xe8\xaf\xa1\xe6\xde\xe9\x1f\x3e\x37\x70\x77\xd7\xd6\x1e\x9a\x53\x75\x47\x59\x43\x74\xf6\xcb\xee\x0e\x5a\x36\xd5\x76\x57\xe6\x98\xb0\x67\xee\x70\xcd\xd6\xd5\xdb\xab\x8a\xad\x06\x2a\xd4\x87\x9d\x7b\x69\x6e\xde\xbc\xbe\x97\xea\x16\x68\x6e\x77\x28\x16\xbc\x66\xd7\x3f\x75\xcd\x86\x4e\x45\x1e\xf9\x32\x96\x6f\xa3\xed\xb8\x0b\x99\xd9\x58\x7a\xb5\xf9\xd0\x4d\xcf\x1f\xb4\xae\x4e\xef\x26\x63\xcd\xd9\xd4\xc3\xbc\x0d\xbb\xf3\x62\x80\x11\xb7\xc0\x9c\xbd\x92\x63\x32\xeb\xee\xd2\x33\xb7\x13\xf4\xc0\x20\xf6\xf7\xfd\xb4\x47\xdb\xd9\x0a\x17\xf7\xb7\xfc\x54\xc5\x38\xdf\xcb\xd8\xf8\x6d\x78\xc7\x25\x34\xc9\x38\x88\x54\xa0\xfa\xd3\x9e\x5b\x3c\x61\xfa\xdb\x44\xac\x58\xf0\xf2\x71\x5a\x99\x2c\xb3\x83\xfc\x24\x8a\xf9\x5d\xcd\xdc\xc3\x7f\x0a\xfd\x18\x3b\xf1\xa2\x9a\xbd\x89\xa4\x47\x8d\x42\x8a\x3b\x89\x7a\x17\x97\x6c\x6b\x28\xe3\x50\xef\xca\xba\x1d\xf3\x4e\x6d\xf9\x8f\x7f\x3c\xca\x69\xf1\x46\xbc\x42\xa2\x43\x4e\x6b\x7e\xe5\x1c\x8c\x3b\x57\x05\xd6\x97\xa7\x7b\x27\xa7\x6b\x9b\x13\x05\xbe\x92\xd6\x5f\xce\xaa\xa5\xad\xc5\xf4\xa9\x9f\x73\x81\xb7\x5b\x57\xbe\x12\x96\x88\x29\x6b\x5e\x58\x35\x7d\x30\x4a\x57\x70\xde\x26\xea\x1e\x35\x49\x68\x60\x5d\xff\xae\x30\x5b\x0d\xc4\x20\x78\x6c\x23\x0c\x24\x33\x0f\x5a\x75\x02\xe6\x8f\x12\xfd\x95\xb2\x50\x13\x9e\xd6\xf9\x68\xa2\x26\xb7\x48\x0e\x89\x1d\x9d\xd4\x42\x1a\xfe\x1a\x8d\x34\x37\x01\x14\xe7\x22\x02\xef\x51\xdd\xeb\x53\x95\x77\x8d\x6b\x0f\x79\x74\xcc\x1b\xed\xfa\xd2\x78\x89\x7c\x23\x09\xe2\xf0\x51\xe7\xff\xe1\x2e\x67\xa0\xa9\x0b\x88\x2c\x27\x9e\x41\x93\x72\x8d\x78\xe5\x9c\x40\x0a\x87\x6f\x5a\x1e\x3c\xcf\xf9\x63\x0a\xa5\xe0\xf6\xc3\xf1\x07\xa9\x4f\x19\x32\xaa\xdc\xc7\x05\x68\xaa\x5a\x94\x7d\xd9\x85\x13\xcc\x32\x75\xdb\x3f\x35\x1f\xa6\x2f\xdb\xe2\xa6\x1e\x86\x76\x70\xf7\x92\x59\xd8\xbb\xc1\xcb\x28\xe1\xde\xdd\xb1\xed\x06\x23\xf3\xc9\xae\x05\xb5\x05\xc0\x5b\x3b\xc1\x6f\xcc\xcd\xb8\x0e\x71\x61\xef\x97\xb7\xd8\x2c\x6c\x77\x5a\x99\xb2\x70\x71\xaf\xe3\x68\x60\xe4\x0a\xe7\xbe\x8e\x0c\x69\x9b\x15\x53\xac\x44\xaf\xb4\x17\xf4\x8c\xaf\xe8\x95\x5f\xc7\x84\x9a\xbc\xd3\x89\x85\x25\x32\x69\xc1\x85\x07\x92\x60\x9d\xd6\x8b\x1b\x6e\x23\x11\x8c\x25\xa2\x23\x30\x39\x8d\xbd\x02\xfa\x10\x41\xb3\x92\x4c\x13\x45\x27\xab\x94\x73\xb1\x8a\x41\x6a\x8c\x0c\x30\x96\x16\xed\x61\xce\xae\x79\xb7\xee\xde\x5e\x26\xfe\x40\xfb\xbe\x1b\xdb\xde\xc2\xe0\x65\x85\xde\x0c\xd0\xea\x9a\x9c\x1a\xa4\x29\x2d\x9e\x0d\xe4\x2f\x9b\x5f\xc3\x0f\x0f\x6d\x97\x9f\x22\x01\xe5\xd0\xcb\x1a\x66\x58\x3f\x09\xcd\xcc\x6c\x87\x6d\xee\x76\x0c\x9d\x87\x80\x03\x44\xbf\x25\x98\xfb\x96\xeb\xc6\xbc\x29\xcf\x9e\x6b\x03\x22\x1a\x34\xc4\xff\xf9\xbf\x97\x43\x0a\xe0\x73\xb9\xec\x8d\x47\xf3\xb3\xe7\x2b\x28\x03\x25\x9e\xbe\x7e\x11\x81\x72\xbf\x1c\x77\x41\x06\x05\xeb\x12\x3a\xed\xe8\x0e\x3c\xf5\x5e\x8a\xa2\x8f\x18\x53\x4c\xf5\xa6\xd3\x05\x34\xde\x25\x3f\x3a\x12\x35\x54\x19\x20\x4a\xea\x68\x05\xd5\x01\xcf\x37\x88\x67\xd0\x48\x21\x36\x66\x12\x9f\x3d\xa1\x2e\xee\xa0\x31\x59\xc2\xa6\x66\x18\xfb\xf4\xe0\xdf\xf1\x1a\x04\x86\x73\xa7\xf0\xc3\xbb\x6f\x2a\x5f\x14\x5a\x87\x93\xd3\xa3\xa3\x26\x79\x4d\xec\xa8\x43\xed\x40\x11\x7e\x22\xeb\x55\x9c\xef\x95\xf3\x51\xd5\x3f\x55\xd5\x26\x53\x2a\x3e\x10\xb8\x91\x5d\x29\x48\x92\xc9\x70\x67\x77\x74\x71\x39\x9d\x61\x3c\xd7\xd6\x86\xda\x42\x49\x6e\x8a\xf0\x6b\xc8\xcd\x93\x3d\x42\xf7\xe0\x33\xe7\x06\x54\x48\x20\x1d\x77\x7c\x3f\xde\x14\xd0\xd2\x91\x07\xaa\x40\xdb\xe0\x07\x3a\x74\xab\x58\xb5\x69\x62\x3d\x78\x60\xc2\xc3\xea\x12\x85\xb8\xa5\x73\x2f\xa6\x83\xe5\x78\xc8\x6e\x21\x24\x53\xa1\xbb\xf4\x42\x07\x88\x2a\x41\x98\x9c\x8a\xca\xf1\xc5\x66\x7c\x77\xfc\x9a\xf6\xad\x26\xbe\x3b\x7e\x75\xae\x51\x87\x93\x3e\x88\xf5\x1f\x66\xa3\xb7\xc3\x53\x1a\x3d\xb2\x8f\x31\xf1\xa8\x25\x93\x99\x0b\xdf\x29\x80\xc8\x33\x8e\x55\xf6\x7e\xd7\x78\xbc\x77\x3a\x52\x85\x62\xc1\x4d\xdc\xff\x9a\x12\x13\x2f\xb1\xf1\x15\x25\x96\x0b\xce\x6d\x6f\x06\xf8\x4b\x50\xd8\x29\xf5\xed\x25\xd4\x9b\x87\x98\xbc\x79\xf1\x0c\x64\x3b\x8c\x9d\x20\xea\x34\xf0\xb0\x46\x0d\xfc\xbb\x8f\xca\x53\x4a\x6c\xfa\xa8\x3c\xdc\x3b\xc5\xdb\x37\x2f\xed\x19\x15\x54\x7e\x22\x80\x0c\x31\x6a\x17\x20\x19\x00\x7a\x1d\x79\x33\x5d\xc2\xb4\xd2\x76\x8d\x6a\xc0\x16\xca\x73\x31\x80\xd2\xe9\x89\x29\x40\xc9\xba\x54\xa2\x7f\x90\xe7\x20\xca\xe2\xc0\xbe\xdd\x0b\x7a\x4a\x61\x5b\x91\xe1\xd6\x51\xb8\xa0\x14\xae\x00\xdd\x0b\xb1\x88\x9e\x2b\x8e\x36\xe7\x1a\xe3\xc0\xec\x82\x5d\xc4\x82\xa1\x45\x38\x71\xa4\x10\xa7\xc8\x82\x37\xa0\xad\xb5\xf6\xb2\xee\xe4\x27\x12\x71\x0c\x10\xcb\x6a\x49\x80\x98\xa7\x02\x6e\x6a\xf0\xb0\x82\x7d\xf7\x8c\xe6\xdf\xf8\xda\x1c\xa8\x85\xce\x0e\x03\x0f\x2d\x00\x19\x27\x2e\xbe\x56\xd6\xf8\x8a\xb4\x58\x84\xf1\x95\xbb\x33\x7b\x6b\xda\xe6\x20\xcc\xdc\x3b\xce\x9b\x2f\x7b\x30\x09\x23\x0c\x4c\xc2\xea\x1d\xc1\xb5\xa4\x14\x92\xbf\x11\xc7\x68\x2f\x70\xc1\xc1\x58\x0f\xa0\x71\x59\x97\x6f\x42\x0f\x9d\x50\xd5\xd3\x2d\x3d\x22\xc6\x85\x24\x5a\x6e\xd0\x6f\x33\x96\x7c\x0a\x38\x43\x06\xa1\x7b\x66\x66\x15\x18\xba\xd8\xa7\xb7\x4b\xea\xaf\x9f\x7f\xf3\xf4\x87\x97\xef\x8e\x9f\x3e\xa3\x63\x41\x33\xac\x01\x9d\x1d\x12\x73\x82\xb2\xae\x4e\xcf\xba\xf3\xaf\xa6\x83\x35\x3a\x01\x7f\xfd\xfd\xdb\x77\xb7\xaf\x7f\x80\xff\x9e\xbe\x7b\xf6\xfb\xf8\xc1\xde\x48\x5c\x3e\xf1\x2e\xf5\x62\xb8\x38\x9b\x0e\xe2\x96\xee\xcb\xe1\x84\x36\x7f\x16\x74\x5d\x75\x1b\xaa\xbb\xaf\xba\x0f\xc5\x8b\x2a\x92\x59\x3c\xe5\x29\x72\xa9\xc6\xbc\x41\x3c\x34\x27\x83\x7a\x0c\x7c\xbf\xe4\x17\x15\xa2\x65\xdf\xa0\x91\xfb\xce\xc3\x26\xc7\xce\xe9\x3e\xd4\x00\x92\xee\x7e\x8b\xb2\xf6\x9b\xf4\xa7\xd1\x64\x8f\xf0\x86\x4d\xba\x8d\x98\x82\xe9\x21\x7c\x40\x05\x1b\x4f\x00\x1f\xa7\xba\xd9\xa0\xb8\xb5\xea\x68\xa9\xa0\xcb\xd4\x95\x34\xdb\x90\xc2\x26\xa7\xa1\x73\x04\x8b\x7a\xd3\x07\xd2\xb6\x30\x9a\x42\xd2\x36\xa3\x50\x6f\x1a\xa4\x18\x28\x03\xa8\x35\xf9\x43\x18\xac\xc9\x07\xb9\xc1\x22\x10\x15\x7a\xdd\x01\xc6\xfe\x77\xcf\x72\x97\x97\xf8\x10\xfd\xe1\x8e\x21\xdd\xce\x21\xb7\xa0\x98\xe2\x3b\x9a\xe4\x3b\x84\x72\x47\xd1\x29\xed\x4d\x6d\x63\x6b\x14\x54\x8a\xee\xc4\x7d\xa3\xf9\x0b\x0a\x5b\xde\x1f\x3e\xeb\x8e\xc7\x09\x82\x03\x25\x97\x53\xa6\x27\x86\x8b\x65\xaf\xef\x97\xa6\xee\x36\x85\x57\xc4\x20\xa4\x7d\xd8\x69\xb2\xe7\x7f\x61\x7f\x2c\x13\x2b\x5c\x63\x40\x3b\x3d\x19\x9d\xa2\x8d\xc9\x2c\x3d\xb7\x12\x0d\xc6\x26\x15\x8b\xe9\xb4\xaa\xae\x7a\x7b\x9b\x3f\x4f\x0d\x7c\x8a\xe1\xf3\x0f\x34\x42\xf5\x6f\x6f\xed\x49\x90\x3f\xc9\x32\xc5\x58\x81\x47\x86\xbd\x65\x80\x50\x12\xba\xd8\x6f\xdb\x8e\x4e\x8b\x2c\x75\x58\xbc\x59\x59\x41\xc6\x5c\xc2\x5d\x5e\x41\xd2\xb1\x13\x26\x2e\x49\x90\x2a\xa8\x47\x3f\xe8\xb5\xc1\x29\xa8\xac\xec\xc8\x15\xf8\x55\xc6\x19\x05\x53\x2c\x13\x36\xd6\x59\x24\x8e\x38\x44\xe9\xf4\x62\x34\x1f\x26\xa4\x2f\x38\x0d\xc4\xd5\x05\xfa\x9b\xcb\xf0\x8b\xa9\x25\x42\x62\xc4\x24\xb5\xa9\x01\xb1\x38\x7e\x20\x9f\x78\x7a\x4b\xdd\xc1\x3d\x98\xd6\x75\xb4\x95\x11\x65\xec\x26\xbb\xbb\x3e\xb3\xc5\xf9\x53\xa8\x7f\x72\x8a\x33\x08\x07\x10\xef\x31\x74\x89\x9d\x3e\xe1\xbc\x9c\x71\x54\xfb\x1d\x33\xc5\x34\x8a\x38\xd1\xd0\xd2\xa6\x8b\x2e\x23\x26\x34\x87\xf4\xad\x07\x7e\xe2\x45\x47\xc0\xf0\x05\x32\x56\x83\x65\x7f\xda\xfb\x85\x2c\xe2\x33\x91\x4f\x15\x03\x85\xce\x87\x13\xb9\x47\x0f\x49\xcd\x08\x74\xa2\x99\xa1\x2e\x1a\x18\xca\x11\x1f\x9f\x40\x78\x93\x8d\x72\xf1\x8d\x59\x33\x22\x1e\xb2\xcb\x9e\x9c\x65\x18\x72\x27\xf2\xd7\xec\x39\x25\x7b\xc6\xf1\xff\x29\x24\x57\x6a\x75\x93\x67\xf5\x3a\xdd\xf1\x26\x97\x5a\x19\x33\xc0\x45\x43\x9e\x4e\xb8\x53\x8f\x5d\xc7\xa3\xbf\x42\x03\x48\x13\x9b\xb6\x31\x2b\x60\x80\xee\x56\x5e\x63\x19\x95\x21\x5c\x6c\x39\x01\x1b\x89\xc0\xd4\xc8\xf8\xfd\xaa\xe2\xae\x64\x38\x9b\xeb\xf3\x9f\x5c\x79\x82\x3e\x17\x3d\x81\x28\x58\x8a\x17\xbf\x14\xb1\x55\x8a\x2a\x75\x1d\x58\x42\xe9\x31\xc4\xad\xf2\x62\x8a\x1b\x20\x50\xdc\xd9\xe1\xb8\xb3\xe2\x8a\xd8\xf7\x96\x58\x4f\x75\x92\xd9\x0b\xb2\x82\x57\x2f\x5d\x52\x2f\x56\x02\xc0\xac\xca\x3c\xb7\x01\x21\x10\xa5\x89\xbf\xce\xeb\xc6\xe8\xa5\x85\xdf\x62\xcb\xea\x3f\x50\xcf\xf4\xf3\xf6\x96\x13\xf0\x95\x92\xe9\x0f\xee\x40\x1d\x95\x61\x30\x18\xe1\x8f\x2e\x4b\xaa\xaf\x8d\x3a\x2c\x10\xb3\x14\xd9\x5c\x45\x33\x07\x96\x51\x8a\x2c\x55\x69\xad\x0b\x74\xa7\xe4\x06\x94\xc3\xe6\x0d\x2b\x39\xcd\xc2\xb7\xcf\xdf\x15\x36\x6a\xde\xbd\x1a\xda\x34\xd4\x8f\x20\xf1\x57\xdc\x83\xf8\x25\x95\x4c\x5b\xdc\x65\xcb\x41\x94\x2d\xf0\xf5\xf3\x97\xcf\xdf\x3d\x87\x8a\x05\x7e\xcd\x50\x48\x67\x6d\x14\x72\x9e\xd9\xf2\xd1\x79\x96\xcc\x78\x93\x28\xdf\xd2\x7f\x93\xcc\x9f\x0a\x27\xb8\x49\xc3\x0f\x65\xd9\x20\xb1\x32\x48\x27\x29\xd6\x84\xed\x48\x80\x1c\x02\x7a\xa2\xfe\x25\xa4\x97\x0b\x4c\x4a\xc3\xdc\xd8\xa8\xfd\xa4\x98\x5a\xc6\x4c\x6e\xdc\xb5\xa6\xe9\xf8\xee\xa2\x05\x47\xd8\x86\x07\xe1\x07\x6d\xb1\x8d\x3b\x38\xde\xa9\xe1\xb9\x8a\xf3\x86\x67\x89\xcf\xc4\x74\x59\xb2\x66\x3b\x31\xdb\x22\x65\x77\x3b\x94\xee\x6e\x7e\xcc\x62\x0c\x89\xe2\xdc\x7f\xef\xe8\x3f\xf6\x9c\xad\x38\x21\xc2\xaf\xd6\xfc\x37\x10\x68\x87\xc1\x99\xf9\x0b\x08\x4e\x01\x0c\x38\xef\x55\xe4\xe0\x97\xb0\x14\x70\x40\xaa\x1d\x02\x32\x44\x9f\x42\x58\xbc\x0a\x42\x7d\xd7\x09\x7b\x54\xf8\xf9\xe8\x68\x50\xc6\xcd\x24\x29\xd5\xdc\x62\xb1\x48\x7f\xbd\xf8\x48\x85\xe8\xe7\xdb\x43\x28\x7c\x74\xd4\x89\x61\xfb\x48\x05\xca\x85\xe8\xe8\xe8\x3f\x6e\x1f\xc4\xba\x3a\x2e\xc4\x30\xad\x0d\x09\x58\x05\xc4\x27\x3d\xa8\x23\x20\x15\x97\x74\x84\x24\x3b\xd4\xa3\xa3\x26\x6e\x65\x61\x7b\xab\x27\xbb\x5d\xe4\xdd\xf7\xdb\x0e\x51\x2d\xe5\x8e\x0d\xe1\x69\xad\x91\xaa\xe1\x4b\x12\x53\xa6\xcd\xf9\x87\x3a\xa1\xd3\xe4\x07\x57\x32\x21\x6d\xba\xb3\x36\x7e\x6d\xa3\x78\xc6\x6c\x0a\xc3\x41\xe5\xda\x91\xe5\x92\x24\xf3\x04\x22\xd5\x57\x97\x92\x40\x3e\xdd\x9a\xa5\xa0\x2a\x9c\x16\x62\x87\xfb\xca\x85\x07\x75\x74\xbf\xbc\x0d\x50\xb4\xd7\x8e\xb7\x00\x33\xa4\xa2\xfb\x7e\x35\x86\x15\x63\x34\x39\x7d\x3b\xee\xce\xcf\x86\x20\x32\xbb\x23\x73\x52\x52\xd8\x03\x76\xc3\x04\x67\x57\x8a\x9f\x4d\xbf\x4e\x19\xd3\x36\xd9\x43\xb9\x57\x7e\xb0\x87\xce\x1e\xb2\xb2\x8e\xaa\x51\x3b\x39\x7a\x5f\x8e\x1e\xdc\x1e\xb5\xe3\x18\xca\x01\x1b\x2a\x9e\x8a\xd5\xac\xf2\x47\x47\x55\x28\xb4\x87\xa5\x34\x1b\x5c\xe6\xef\xca\x7d\x0e\x21\xf5\xf4\xf6\x56\x66\xba\xa8\x8d\x89\xf7\x45\xfc\xe5\xa5\x38\x5b\x76\x3a\xc9\x54\x81\x8c\xdf\x74\xec\xc9\x21\xff\xa9\x9a\x53\xc9\x9c\x63\xb5\xe0\x30\xcd\x6c\xba\x1f\x90\xa4\x02\x06\xbd\x1a\xf1\x03\x00\x93\x23\x31\x5e\xa3\x4b\x6d\xe2\x0f\xaa\xb0\x0d\x1e\xe7\x49\x56\x64\x83\x28\x48\x72\x8e\x41\x37\xea\xc6\x48\xc2\x26\x83\x43\xe3\x50\x86\x68\xf7\xff\x28\x46\x85\x97\xf0\x27\x5e\x83\xe8\x23\x15\xd0\x24\x26\xf3\x11\x1d\xe1\x62\x2a\x3d\xf2\x7e\x61\x13\xbf\x23\x8e\x02\xd4\x92\x9b\xe9\x6c\x74\x3a\xa2\xf5\x6d\x71\xd6\x24\x30\xb3\xe1\xe9\x70\x75\x49\xbf\x37\x6c\xdc\x0f\x05\xab\xe7\xe2\xdb\x4f\x2b\x4d\x90\x9b\xe0\x3f\x76\xe0\xa3\xc3\x28\xae\x76\x62\x14\x09\x47\x47\xc8\xfd\x4e\xd6\xd1\x5e\xdc\x6e\x62\x44\xf6\x38\x3a\x3c\x6a\xdf\x1e\x95\x3a\x71\x1b\x0a\x3a\xd2\x60\x8e\x3c\x4a\xf6\x1c\xfa\x81\x87\xb5\xf5\xa0\x37\x72\xd6\xb2\x4d\x8c\x75\xd9\xfa\x05\x0d\x81\x92\x42\x29\x3b\xb7\x65\xdd\x68\xca\x6d\xda\x0d\x4a\xd1\xa6\x6d\x06\x0f\xb1\x77\xf5\x4f\x7c\xd5\x82\x68\x24\xf4\x2f\x19\xaa\x17\x0a\xe5\x48\x67\xa3\x1d\x33\xe5\xc4\x30\x55\xdb\x4d\x37\x87\x92\x9b\x30\x7f\xca\x11\x22\x04\x9b\xb0\xa8\x5a\x6e\xc7\x05\x80\x11\x1d\xfe\xbc\xd7\x29\xc7\x98\xa5\x8b\x93\xc1\x50\xb9\x10\x17\x82\xa4\x8d\x4b\xb2\xc3\xa3\xbd\x07\x48\x27\x4b\x4f\x1c\xb0\x2a\x8f\x4e\xe2\xad\x01\x28\xcf\x17\x67\x20\x72\x30\x26\x95\x19\xe4\x76\x61\x54\x40\x94\xb0\x9a\x3d\x06\xa2\x75\x8d\x8e\xf9\xd0\x9d\xb3\xab\xf6\xe1\xd0\x63\x09\x5a\xce\xb8\xec\x21\xa6\x76\x02\xb5\x00\x2f\x03\xd0\xb3\xfd\xf7\x93\xb7\xc3\xee\xac\x7f\xc6\xfa\x09\x96\xa7\xc5\x1e\x57\x1f\x9f\x2f\x19\x6a\x2c\xaa\x00\x2b\x83\xfc\xb2\x1e\x99\x2e\x21\xe9\x85\x79\xd4\x9a\xf5\x8b\xd4\xa6\x4f\x39\x4b\xab\x29\x2f\x33\x6e\x72\x8f\xf7\x0a\xad\xff\x21\xa8\xba\x30\xb3\x50\xe6\xdc\x77\x53\x61\x6c\x07\x49\xb7\xa6\x46\x56\xef\x59\x90\x52\x72\x2c\x3a\x85\x8d\xc1\xec\xfd\xc8\x0d\x25\x28\xaa\xf0\x8d\x53\x98\xc8\x4a\x66\x54\x66\x9b\xeb\xc0\x78\x00\x2a\x12\x5e\x4f\xce\xa6\x53\x0e\x28\x80\x47\xe5\x68\xd0\x40\x97\xf9\x0a\x33\x50\xba\xf0\xfe\x9e\x4f\xd1\xf1\x2e\x13\xb7\x2e\x53\x8a\xeb\x9e\x1b\xf3\x24\x88\x66\x62\x1a\x50\x06\xbc\x72\x81\xc3\x96\x48\x19\xc0\x2a\x3b\xf8\x09\x05\x37\x31\xd2\x86\xcf\xd6\x48\xfa\x91\xf8\x00\xe1\x07\x30\x85\x63\x70\x50\x59\x54\x90\x1e\x87\x3f\x9d\x63\x39\x3c\x54\xe3\x0c\xe6\x5e\xed\x71\x9b\xa6\x28\x99\x29\x26\x6e\x36\x87\x7e\x14\x3f\xc4\xbb\x17\x5e\x69\xeb\x1f\x87\xcc\x9c\x92\x0b\xc7\xb2\xe9\x60\xd4\x2a\x97\x47\x06\x19\x79\xee\x82\x1e\xb6\x50\xc1\x28\xcc\x29\x8c\x18\xae\x94\x20\x56\xa7\x27\x3b\x17\x87\xa3\x4e\x7b\x30\x4c\x9d\xec\x63\x7a\xdc\xc4\x7f\x5b\x74\x76\x72\x85\x77\x78\x91\x5d\x68\xaa\x13\xfd\xf4\xc5\x70\x09\x67\x5a\xc9\xbc\xbc\x84\x6d\xf1\x90\x0f\x5b\x75\x6c\x8e\xd5\x82\x7d\xb6\x4b\x2a\x99\x96\x26\x3c\x24\x55\x09\x31\x41\xae\x1b\xd9\x16\x98\xdd\xe4\x55\x1f\x70\x01\x10\x6e\x98\xa8\x3f\x8b\x45\xc3\xde\xfc\xb0\x8f\xca\x22\x47\xcb\xe0\x52\x61\xfb\x0d\xda\xcf\x2e\x15\xf1\x27\x2e\xa4\x02\x39\xfb\xc3\x37\x94\xca\x66\xd7\x7a\xc1\x15\x88\xac\x66\x3b\x3b\x00\xb7\x90\xc7\x50\xb1\xb2\x3c\x57\x7d\xd0\x9b\x01\x44\x58\x74\x30\x38\x05\x15\xfa\x81\x28\x52\x20\xcc\xf0\xec\x00\xe1\xdf\xde\xd2\x17\x10\xd7\x41\x83\x42\x43\x6f\x83\xc4\x11\x33\xde\xa2\xfd\x67\x81\x5c\xa7\x31\x4c\xe5\x93\x92\x3a\xa0\x98\x9e\xd2\x75\x2d\x00\xd0\x16\xc4\xe8\x88\x12\xd7\x2e\x28\x12\xb7\xcd\xac\x21\x32\xba\x77\xc2\x61\x59\xe5\x90\x2a\x8e\xab\x73\x22\x6d\xe4\x26\x1a\xa1\x1e\x37\x2d\x58\x50\x78\x42\x48\xa9\x61\x0c\x90\x88\x9d\x04\x69\x26\x8e\x1d\xe8\x74\xce\xc1\xe2\x07\xfd\x5a\x04\x07\x19\xb1\xf6\xaa\xea\xbb\xd0\x50\xae\x3f\x0c\x31\x74\x0c\x85\x27\xbb\x0b\xad\xca\x99\x82\xbd\x3a\xf0\x37\x91\x62\xc1\x95\xa9\xe3\x59\xe3\xae\x4e\x92\x22\x3e\xeb\x6d\x6d\x23\x8e\xe8\x71\x2b\x27\x36\x6d\xe2\x68\x72\x35\x3d\xb7\x26\x5e\x19\x7a\xbe\xb9\xe9\x22\x6c\xf5\x57\xdc\x4e\xef\x54\x4d\x51\x64\xbe\xd4\x05\x99\x37\x78\x71\x33\xbf\x21\x74\x3f\xe2\xb5\xf5\x03\x6f\xa0\xb6\x34\x28\x25\x5c\x08\xce\x6f\xbf\x65\x77\x68\x92\x20\xc0\xa1\x1b\x99\xc8\x85\xbd\x85\x2c\xd2\x34\x35\x31\xa6\x80\x35\xef\xb2\xb1\x30\xed\x26\x36\x6e\x91\x9b\x9d\x17\xbf\x28\xef\xe0\x2c\xbc\xc9\xc2\x23\x43\xd0\x67\xb7\xe1\x1a\x1b\x83\xc0\xea\x83\xd4\xf8\x10\x9f\x77\xc7\xe3\x48\x3f\x74\x4b\xb5\x2d\x19\x37\x24\x75\x03\x21\x2b\x24\xb0\x94\x20\x96\x17\xde\xf5\x44\x9d\x2b\x03\x3d\x51\x77\x87\xac\x13\x09\xc5\xc7\x59\xae\x8c\x72\x83\x63\xc8\xb9\x5a\x36\x86\xf7\x00\x4f\x07\x85\x0e\x70\x39\x36\x43\xe7\x5f\xce\x06\xc1\xae\x37\x37\x12\x78\x13\x3b\xc3\xaf\x02\x72\x66\x31\xab\x53\xce\xb9\x2b\xaf\xf0\xbb\x12\x73\x59\x16\xc2\x24\x43\x1d\x48\x89\x2b\xd1\xa5\xa0\x12\x3f\x3e\xd0\x3b\x24\x56\x16\xe5\x5e\xad\x99\x96\x36\x69\x31\x67\x74\x29\x2b\x1d\xe5\x06\x4c\xbb\xc7\x37\x4b\x25\x37\xaa\x93\x6f\x6f\x45\x3d\x44\xed\xa1\x53\x2c\x7a\x28\x48\xaa\xc1\xe4\x66\xe3\xc2\xbf\xd9\xb8\x6f\xc3\xdd\x15\x80\x35\x09\x8d\xd3\x8d\x73\xbc\x69\x37\x48\x21\x59\xa5\x12\xc7\x53\x95\x40\x91\x4d\x77\x97\x3f\xe7\xf3\x07\x35\x22\x41\x4d\xee\x5f\x63\x39\xfc\x14\x27\x75\xa7\xfc\xfc\x84\x9e\x94\xb2\x89\x16\xa5\x10\xf9\x13\xf9\xd0\x11\x02\x78\xaf\x55\x2d\xc5\x7b\x64\x89\x9d\xb8\x65\x0f\xeb\x88\xa5\x85\xec\xec\xa1\x71\xb3\x90\x6a\xf2\x3b\xf3\x74\x02\x77\x13\xe2\x3d\xc5\xa9\xc4\x51\xdb\xa9\x1a\x07\x98\x2e\xf0\xc5\x5f\x7a\x55\xc7\xf1\xb9\x61\xd2\x37\x85\xcf\x58\x25\x71\xc3\x4b\x79\xd5\x6a\xde\x54\x43\x37\x81\x4f\xe7\xeb\x49\x3f\x72\x34\x2c\x98\xff\x9a\xe8\x6e\x59\xb2\x86\xd2\x8c\x14\x4c\x49\xb7\xb6\xe0\xb5\xe9\x6c\xc2\x3d\x3e\xf5\xd0\xd9\xe9\x5b\x0d\xde\x33\x9a\x24\x73\xc0\x8d\x63\x29\x31\x39\x45\x8b\x64\x7d\xbc\xcc\xe0\x73\x43\x09\x5a\x87\x46\x6a\x9b\x57\x4c\x8c\x28\x98\x19\x0b\x8b\xa2\x0b\x7e\x7a\x44\xc1\xbc\x98\x2c\x81\xc3\xb1\x20\x64\xb0\x13\x4c\x0c\xbb\xf8\xdf\x1f\xba\x2f\xd0\xb8\x45\xcc\x89\x4c\xcf\x90\xaa\xce\xb7\x48\x7e\xbb\xd8\xf1\x37\xa8\xdf\xc1\xba\x83\xbb\x10\xb3\xf4\x6c\x09\xb4\x27\x6e\x90\xef\x13\x62\x8f\x89\xa7\xd8\x11\x30\xbd\x8c\x71\xd6\xb6\x5c\xe7\xc6\xc6\x8b\x1c\xc7\xbe\x80\xe6\x12\x1b\x24\x8f\x40\x39\xb1\xf1\x34\xbd\xc2\x2d\x06\x9a\xbb\xbb\x83\xa1\xa9\x31\xbf\x7f\x74\xbf\xaa\xeb\x7f\x4e\x53\x6b\xee\x42\x23\x18\x84\xa5\x2d\x9a\x38\xac\x1e\xa5\x21\x59\xb3\xfe\x54\xd6\xd3\x79\xc0\x59\x87\x59\x65\xf0\x4e\xdb\xa0\x25\xa4\x95\x68\xd4\x6e\xbc\x63\xd7\x80\x35\x5d\xc1\x5a\xb5\x7f\x42\xdd\x0d\x0e\x45\x10\x8b\x30\x33\x68\x20\x5a\x50\x87\x11\x03\x83\x82\x57\x5e\xe0\x40\x15\xc4\x22\xdc\xc8\xa3\xfe\x14\x2b\x6d\xee\x8a\xcb\x99\x1b\x0f\xb4\xe5\xfa\x68\xca\x52\x5f\x78\x66\xc5\x4a\xcf\x30\xb4\x13\xe5\xa8\xbd\xbc\xf8\x93\xad\x60\x92\x61\x3a\x88\xb9\xd6\x6e\xf0\x52\xcb\x51\x69\xa5\xa0\xfc\x93\x54\x0c\x79\xec\x00\x4c\x95\xd7\x67\x22\x59\xb2\x99\x02\x34\x38\x75\x07\x36\xe4\x36\xcb\x62\x0c\x7c\xe5\xc8\xe4\xd8\xff\xac\xca\x26\x83\x0f\x66\x96\x8b\xbb\x62\xc8\xf6\x1d\x36\xb0\xde\x8d\x3b\x9f\x7a\xc2\x9c\x6b\xb8\x99\xa6\xe5\xb3\xe9\xf4\x7c\x84\x11\x27\xd2\x56\x98\x7d\x9b\xe5\x9f\x2a\x81\xde\xf8\x7e\xee\x87\xe3\x76\x0e\x85\x24\xd7\x91\xed\xb4\xe2\x8b\x7c\x43\x27\x36\xbc\x7b\x93\x72\x82\x80\x30\x01\x59\xa9\x50\x28\x68\x1d\x24\x42\xf2\x63\xd7\xb2\x5d\x30\xe3\x9b\x5e\x50\x02\x75\x9b\x55\xc9\x20\x00\x0e\x11\x3c\xa0\x52\x46\x1e\x42\x79\xa0\x54\xde\x06\x92\xfe\xec\xf2\x51\xc9\x33\xb7\x06\x2e\x2e\x99\x8d\x8b\xdd\x82\xf4\x0a\xef\xb9\x32\x76\x7b\x29\x70\x71\xdb\xc3\x27\xdd\x5e\x53\x74\x24\xaf\x18\x4a\x5e\x69\x28\xbe\x31\x2d\xd6\x03\x0a\x27\x29\x44\xb3\xe9\xe2\x81\xc6\xe8\x70\x1e\x14\xd3\x6d\x77\x3c\xb2\x8a\xc4\xed\x2c\x54\x9b\x7e\xff\xb2\x2a\xba\x34\x63\x79\x40\x6e\xea\x7d\x6e\x11\x30\x68\x8f\xe5\xd0\x88\xcc\xb3\x96\x93\xe7\x78\x28\x07\xfd\xcf\xba\x64\x36\x48\x3b\x87\x05\x26\xcd\x28\x7b\x9a\x52\xdd\xc1\xe0\x35\x4c\xc7\x6f\x26\xe1\x4b\x51\x91\x81\x5b\x48\x9b\x46\x7a\x37\xf1\x6b\xe9\xe8\x30\xfe\x28\xf9\x65\x14\xfb\xcc\xf2\xd3\xdc\x69\x91\x59\x40\x0f\xa5\x26\x86\xbf\xed\xe3\xa7\xaf\x68\x69\x13\xc5\x96\x5c\xbe\x3a\xcc\xef\x15\x70\xe2\xc6\x1a\xda\x26\x43\x38\xbc\x85\x2f\xb6\xd2\x36\xc2\xc2\x75\x3d\xca\x58\x68\x9d\x13\x0d\x16\x4c\xae\x3c\xf6\xd3\x76\x2f\xba\x30\xbf\x87\x74\xcd\x70\xcc\x19\xcb\xc9\x6c\x7a\xf1\x3f\xe7\xc6\xc0\x56\xec\x5f\xd5\xe5\xd2\x87\xaa\x43\xa3\x79\x10\xcd\x60\x2f\xa6\x0e\x08\x63\xfb\xe0\x63\x25\x7c\xeb\x01\xc0\x17\x0c\xff\xb4\x8b\xbe\x8b\xee\xf9\x10\x5d\x32\xcd\xf5\x8b\x98\x91\x9a\xf6\x7e\x41\xe6\x65\xa7\x6f\x73\x8c\xb4\xcd\xfb\x38\x55\x10\xa7\x00\x49\xad\x35\x7a\xc2\x0e\xda\xf4\x43\x38\xf4\x89\xd1\xfb\xe5\x90\x12\x0f\x47\x1d\xbc\x4c\xd7\x94\x83\x74\xab\xc0\xa2\x16\xf8\x9a\xde\x7d\x45\xf8\x53\xc1\x22\x3c\x18\xfb\x02\x1a\x73\xe9\xdc\x13\x1f\xbd\x2f\xba\xa7\x0a\xfe\x23\x7f\x63\xb8\x4f\x00\x41\xda\x9d\xad\x39\x52\x34\x27\x1b\x72\xda\x30\xaa\x92\x23\x31\x3c\x9e\x6b\x4f\xf0\x92\xdc\xa1\xc7\xb8\xad\xf9\xa2\xdb\x3f\xaf\x22\xfb\xa2\x03\xbe\xd1\x04\xb4\x9b\xa1\x29\xea\x66\x76\x5a\x31\xe1\xf4\x7c\x32\x40\x8c\x0a\xe4\xf3\xd8\xe4\xc6\x2d\x7d\xb7\xc5\xf9\xa9\xd6\x8a\x45\xb7\x38\x5e\x60\x9f\xf2\x53\xd0\x00\xa8\x41\x9a\x7a\x98\x5c\x4d\x47\x83\x14\xac\xdb\xdb\xdd\x5d\xca\xe6\x42\xb7\xb7\x0c\x9a\xd6\x34\x5d\x9f\xf4\x51\x72\xcb\xca\xe6\xe6\xf3\x85\x39\x3a\x7d\xfa\xee\xdd\x1b\x71\x7f\x10\x5e\xbb\xd3\x3a\x31\x98\x2e\x7b\xe3\xe1\xff\x5e\x4e\x17\x74\xb9\x8f\x5e\x99\xe8\x91\x9a\x9b\xb2\x9c\xfc\x6a\xbf\xdc\x59\x93\xaa\x0c\xd8\x85\xb5\xd1\x02\xf4\x57\xf7\x13\x14\x33\x42\x55\x44\x30\x5f\x53\x3c\x9f\x2c\x46\x0b\x14\x5c\xe6\x10\x54\xb8\x84\x23\x96\x14\x8b\xde\xa7\xee\xb8\x78\x11\x67\xfe\x08\x0e\x87\x84\xca\x0e\x33\x69\x5e\xbf\x9c\xce\x91\x9b\xef\xc3\x4d\xc4\x36\x58\x5e\xc6\x53\xee\xe8\x5a\x90\x74\x90\xd4\x64\x98\x0f\xe1\x0b\x16\x2a\xa9\x83\x79\x95\x0a\x5d\xf0\x70\x29\x7e\x7a\x3b\x0a\x41\x8c\x0e\x12\xc8\x6f\x8d\xa0\xac\xee\xdb\x70\x32\xb0\x1d\xc5\x03\x23\x06\x4f\xd1\x0c\x9c\xca\x58\x8f\x17\x26\x1d\x45\x07\xfa\xcb\x67\x42\xcc\xa6\xf8\x68\x9f\xee\x60\x70\xba\xd1\xf4\xb5\xec\x98\xde\xe0\xef\x70\x2b\x3e\x7a\x9d\x4d\x8b\x2a\xd3\xac\xa3\x06\x50\x42\xfb\xf3\x87\x62\x7b\xc1\x1c\xc8\x9c\x40\x31\x56\x27\x04\xb2\x0d\x44\xaa\xa5\xf8\xc9\x11\x3a\x5e\x46\xa3\x8a\x79\xa9\x50\x76\x6b\x97\x0b\x87\x3f\x1f\x74\x4a\x07\x20\x7e\x46\xee\x41\x52\x77\x2c\x8e\xad\xcd\xb5\x24\x9e\x31\x2e\xf8\x04\x9f\x1b\x79\xf6\xfd\x77\xdf\x3d\x7f\xf5\x4e\xb3\x7c\xc1\xbb\xad\x7f\xf6\xf5\xd3\x77\x4f\xbd\x2c\xc3\x66\xd4\x47\x4b\x7d\xfa\x8c\x02\xde\xa4\xa6\xd1\x47\x29\x1b\xfa\xe6\x8b\x07\xfd\xb0\x1d\x8f\xba\x88\x04\x34\x50\xdf\x9f\x44\x85\x27\xbb\x95\x4a\x21\x6e\x8b\xf7\x3b\x3f\x0f\x72\xd4\xc3\x98\x5d\x1d\x10\x77\x51\xf6\x98\xc2\x9b\x9b\x22\x07\x05\x89\x8f\xa7\x83\x28\x51\xc8\x50\x41\x9a\xc3\x0c\x39\xbd\x90\xb8\x43\x04\x88\xaf\x9b\x51\xa9\x7c\x28\x20\xd0\xb2\x5c\x0f\x92\xcd\x65\xdf\x09\xfb\xb1\x92\x61\xc7\xb7\x1e\x5f\x7f\xff\xec\xdd\x4f\xaf\x9f\x0b\xe9\xd8\xa4\x0a\xab\x41\x4f\xf8\xfc\x93\x60\xf0\x01\x9d\x5f\x56\x0e\x2b\xc9\xfc\x3a\x64\x08\xf6\xf6\x51\xeb\x90\x75\x8e\xd3\xda\x57\xcf\xbf\x7d\xf1\xea\xdb\xe3\xe7\xaf\xbe\x3e\x7e\xf7\xf4\xdb\xfb\xb5\x2a\x85\xb7\xb4\x6a\x7b\xa8\xdb\x35\x2f\x95\x4d\x82\xc6\xcc\x87\xe6\x0e\x76\x1a\x51\xa7\xa0\x83\xa2\x39\x19\x76\x70\x7c\xfb\xee\xe9\x9b\x77\xff\x44\x2c\x43\x78\xca\x5b\x47\x5d\x4c\xf9\x27\x4c\xe6\x0c\xb6\x7b\x52\xe0\xf5\x83\xe6\x52\x8d\x1d\x68\xb4\xb1\x48\x33\x40\xa8\x26\x6c\x43\x92\xc1\x94\x2c\x14\xc2\x72\x12\xbf\xf1\xd3\x67\x16\xbe\x35\x21\xd8\xb4\xef\xd0\xe6\x85\x73\xd8\x86\x2e\x46\xd7\xae\x79\xa1\x7e\x8a\xf7\xee\x0c\xb4\x1b\xc9\x9e\xed\xbc\xef\xce\x77\x60\x29\xe8\x8d\xc9\x0e\x9d\xca\x50\xf8\x83\x93\x29\x86\x2a\x04\xcc\x76\x48\x37\x40\x4f\x3e\xd4\x3d\x0e\x66\x40\xa3\xd5\x32\xe2\x72\xe3\x4e\xec\xd8\x7d\xd8\x94\xb1\x4c\x99\xf8\xb7\xa3\xc1\x60\x38\x79\x3d\x1b\x02\x55\x27\xc3\xd9\xef\xdf\x7d\xf7\x52\x0c\xf4\x8d\xd1\xcb\x13\x34\x76\x29\x8e\x17\x2d\x94\x39\x7e\xf1\x77\x74\x61\x6c\x13\x91\x18\x72\x96\x82\xab\x65\xf8\xac\x2f\x07\x87\xa0\xbd\x22\xb5\xd7\xbd\xb8\x6c\x39\xf2\xef\xd5\xf7\xaf\x8e\x9f\xbe\x7c\xfd\xfb\xa7\xaf\x7e\xf8\xee\xf9\x9b\x17\xcf\x52\x8a\x81\x07\xb2\x50\xfc\x37\x09\x77\x43\x83\xf6\x0c\xda\x26\x03\xbc\x72\xa1\x55\x70\xed\x79\x9c\xae\xd9\xc4\x03\x4a\x3c\xc5\x44\x5f\x13\x7c\x2b\xe3\xf5\x1f\xb3\x11\x1e\x02\xf6\x96\x27\xb2\x2e\x9f\x4e\x40\x97\x27\x6f\x11\xcb\x85\x59\x92\xd1\xfe\x19\xcb\x28\xf8\x8f\x74\x1e\x13\xc6\x9b\x14\x01\xab\x39\xe3\x3a\xef\x6a\x03\x37\x90\x90\xbd\xb0\xc7\x6a\x97\xdb\x4a\xaf\x60\x90\x4b\x5b\x4b\x46\x85\xca\xf2\xef\xdb\xdb\xfa\x6e\x42\x56\xab\x5e\x59\x74\xe0\xb9\x5c\xd0\x14\x42\xac\x19\x7a\x78\x3d\xc2\x58\x65\x5d\x1d\xd3\xde\x12\xaf\x30\xd2\x78\x92\x2f\x17\xdd\xe6\x53\x52\x97\xc6\x6c\x3e\xb8\x9c\x8d\x9c\x6f\x58\x05\x76\x61\xa5\xd8\xe5\x61\x62\x31\xf3\xc3\x9b\x17\x5a\xc0\x68\x04\x77\x04\x41\xf2\x3c\x82\x3f\x3e\x4b\xc8\xc5\x3d\xfc\xca\x64\x2a\x29\x04\x65\x50\x15\xc3\xdf\x44\xd5\x76\x61\xef\xa0\xd0\x2c\x1c\x14\xf0\xbe\x0f\x14\x14\x8f\xfc\x5b\x49\x7e\x3f\x32\xee\xb9\x74\xa4\x24\x6c\x0a\xf5\x37\x58\xec\x64\xcc\x22\xc3\x28\x80\x03\x09\x13\xc7\x86\x5a\xa4\x9c\x34\x96\xd1\x3d\x2e\x11\xcb\x41\x5f\x20\x5b\xb2\xde\x10\xeb\x12\x18\x56\x44\xff\xb6\x3a\x14\x09\x0f\x1a\x47\xe0\x4f\xe7\x8a\x2c\xb5\xe5\xc9\x66\xfc\x98\xd8\xda\xdc\x2c\xa9\x50\xa0\x27\x7b\x3f\x3f\xe1\x70\x15\xef\x9b\x95\x4e\x39\x8e\x38\x2a\x85\x7c\xd2\x47\x29\xc1\x02\xed\x26\x5a\xed\x1d\xfe\x5c\xe8\x94\x0a\x14\xfc\xe2\xb3\xc3\x9f\x3f\xeb\x94\x3e\x8b\x6f\x41\x91\x3a\x9a\x43\xd5\xb8\x1d\x97\x38\x86\xfb\x1e\xc5\x72\x3f\xd8\x53\xfe\x22\x27\x6d\x81\x3a\xe6\x34\xc8\x6a\xd8\x9e\x72\x76\x11\xc9\x9e\x45\x27\x44\x00\xf1\x43\x24\xa0\x29\x41\x83\x53\x3e\xc3\x14\x40\x26\x32\xd8\x90\x8d\x64\xb8\x7c\x22\x0a\x7b\x2a\x7b\xf5\x37\xe8\xed\x29\x5f\xc5\x4b\xf6\x50\xa7\xa2\xd8\x1c\xa0\x1d\x01\x54\x5f\xff\xc0\x6c\x49\x89\xa8\x37\xed\xf8\x60\x6f\xa4\x5c\x4d\x10\x8b\x1c\x1d\x52\xca\xd1\x21\x01\xea\x74\x10\x90\x9d\x47\xf8\xda\x33\x3a\x59\x5c\xde\xa2\xd1\xc0\xbc\x1d\x37\x81\x4a\x7b\xb7\x17\xdd\xd1\x78\x31\x6d\xde\x2e\x86\xe3\xe6\xed\xbf\xc5\x00\x35\x47\xbe\x22\xc9\x7e\x3e\xfa\xb7\xca\xff\x75\xbb\x73\xbb\x4b\x66\x8f\xee\x4e\x2f\xd1\xdb\xf1\x42\x77\x36\xec\xaa\xde\x0c\x7d\x76\xa9\xb3\x99\x1a\x5d\x9c\xaa\xf7\xbd\x59\x21\x56\xfe\x76\xd3\x8b\x11\x6a\x6b\x43\x2d\x72\x7e\xa3\x06\x03\x35\x58\xa8\x31\x6c\x74\xd4\xa2\x37\x1d\xac\xd5\x62\xa0\x16\x27\xd3\xe9\x42\xa1\x6b\x30\x7c\x9e\xa3\x16\x69\xa8\x2f\xbc\xdd\xb0\x05\x3b\xbb\x54\xb3\x45\xaa\xb4\x29\x97\xbe\xe2\xde\x06\x76\x5b\x4f\x82\xad\xfb\x9d\x90\xbd\xca\xca\x12\x71\x30\x40\xd7\x7d\x0a\x9d\xe8\xf5\xc7\xb0\x45\x9c\x63\x50\x44\x02\x4d\xdb\x50\xd5\xef\xb2\x3f\x9d\x3e\xdd\x42\xe1\xa5\xaf\x1a\x8c\xe0\xef\xe8\x4a\x0d\xc6\x8a\xde\xb7\x0d\xf1\x8f\x2e\x87\x94\x83\x72\x67\x75\x75\xd6\x50\x67\xfb\xea\xec\xa1\x3a\x7b\xa4\xce\x1e\x2b\x7e\xe8\xa4\xce\x98\xea\x38\x60\x13\xc4\xe3\x52\x01\x46\x4b\xd8\x5d\x5f\x29\x18\xc7\x4b\x00\xc6\x31\x87\xd4\x9c\xbd\x43\x81\x38\x03\x15\x45\x2d\xc7\x28\xda\xfc\x33\x88\x4f\xa5\xa6\xed\xb4\xea\xf6\x80\x71\xba\xfd\xd9\x74\xb2\xbe\x50\x3d\xd5\x1b\x8c\xe0\xbf\xa9\xea\x8d\x4e\x89\xa3\x46\xd8\x71\x0c\xfe\x4e\xdd\x3d\x99\xa8\xe1\x05\x74\x0c\xef\xe4\x89\xcd\x10\xf3\xf3\xde\x40\x1c\xab\x53\x1f\xba\xb3\x73\xf5\xab\x9a\x2d\x7b\x6b\x45\x2c\xa0\xe6\x6a\x0e\xea\x84\x9a\x5f\xe0\x2e\x6c\x7e\xd9\x9d\x28\xd4\xf7\xce\x87\xf8\x67\x3a\x39\x55\xa0\x00\xc2\x7f\xc0\x73\x23\xd8\x98\x2f\x16\x6a\xa9\x40\x2e\x62\x1f\x83\x55\xd6\xb2\x96\xa6\x0b\xba\x81\x60\x37\x83\xce\xac\x48\x93\xc2\x9d\x34\x3e\xaf\x04\x64\xcc\xe1\x55\x7c\x01\xc0\x6b\xa7\x45\xa1\x07\xdb\x36\x1c\x3f\x7c\x12\x84\x34\x3a\x9b\x0d\x4f\xd4\x18\xba\x33\x18\xce\xfb\x6a\x3e\xeb\xab\xe5\x7c\x08\xf4\xd0\xe8\x71\xf5\x34\x6e\x1a\xb2\x33\x24\x34\x20\x63\x58\x8b\xe0\xdf\x85\xea\xae\x46\x80\xf4\x29\x4c\xd2\xe9\x4c\xf5\xa6\x14\xc4\xbb\x3f\x1c\x8f\x2f\xf1\x1d\x16\x50\x0f\x7f\x03\x51\xfb\xf4\x1b\xbd\x10\x2a\xbc\x09\x23\x59\x30\xa5\x7f\xe7\xf4\x0f\x92\x1d\xef\x6b\xba\xfd\x05\x9e\x8b\xce\x06\x73\x62\xde\x13\xd0\xb9\x84\x1f\xe7\xf0\x77\x74\x7a\xb6\xa0\xae\x8c\x01\x53\x75\x86\x70\x87\x6a\x34\xc7\x71\xa5\x14\xfc\x67\xd9\x3d\x1d\xaa\xc9\x94\x3a\x3c\x99\xbe\x9f\x75\xd1\x25\xd0\x18\xfe\xbb\x52\x78\x20\x4c\xff\x60\x6b\xb3\x25\x86\xd4\x94\xb8\x01\x74\x67\x84\x28\xce\xcf\xba\xf8\xcd\x5c\xd0\x45\xee\x58\x5e\x5c\xe0\x21\x15\x7c\x9c\x0e\x41\xd2\x8c\x16\xc0\xe3\x68\x8e\x4a\x74\x03\x22\xb0\xde\x73\xc5\xb8\xbc\x1f\x0d\x16\x67\xc8\x1b\x46\xeb\x0d\x43\xe6\x1a\x9f\x5e\x30\x81\x32\xfd\x77\xe8\xe5\xb4\xa0\x0e\x3b\x31\xbb\x7a\x71\x97\x6b\xbb\x5a\x5b\x8d\x2c\xa3\x32\xba\x52\x19\xe3\x72\x5c\xc0\x9b\x4e\xdf\xb7\x0c\x2e\xeb\x2f\x5f\xbc\xfa\x5f\x3f\x1d\xff\xf0\xe6\xa5\x15\xe5\xe9\x85\x20\x92\x95\x20\x6e\x1f\x3e\xad\xfc\xb1\x5b\xb9\xae\x55\xbe\xac\x1e\xff\xae\x0c\x6b\xe4\xbf\xc7\x47\x6f\x4b\x20\xfc\xe7\x47\xd5\xa3\xd6\x91\x3a\x8a\x8e\xe2\xa3\x9b\xa3\xcd\xd1\x93\xa3\x83\xce\x9e\xfa\xee\xe9\x8b\x97\xef\xbe\xb7\x8b\x8c\xc0\xd9\x6b\x85\xb6\x0b\xb8\x25\x10\xd2\x72\xc8\x38\xda\x30\x39\x87\x20\x2d\xe3\x12\x15\x9f\xb9\xc0\xac\x9e\x75\xdf\xd3\xc1\x08\x6f\xf1\x0f\x3b\xea\x3d\xa9\x1d\x49\x86\x26\x42\x4a\x0c\x3e\x2c\xc6\x87\x49\x7c\x8b\x41\x87\x46\x19\x36\x6c\x8c\x01\x85\xe1\xd6\xa5\xab\x9c\x98\x48\x5e\x8b\x77\xbd\xd0\xbc\x68\xa3\x21\x05\xe3\x56\x8c\x0f\x6b\xcc\xde\x5f\x7b\x13\x4d\x12\xed\x74\x94\xdf\x69\x25\x05\xa1\x06\x3d\x23\x42\x57\x6d\x6c\x0c\xc5\x67\x5e\xdc\x1b\xd9\x4c\x62\x63\xc6\x98\x7e\x14\xbb\x7d\xa9\x22\x7f\xe3\x2b\x1e\x5d\x83\xcf\x0f\x0b\xdd\x82\x53\x28\xf6\xc1\xa5\xf6\xd9\xde\x30\xe1\x61\x85\xa9\x80\xb3\x1f\x60\xc5\x44\x6e\x8b\x06\xed\x82\xcb\xe1\x06\x5e\x8f\x6a\x88\x3b\x1f\xc3\x58\x35\x70\x93\x7f\xd0\x3f\xd8\x77\xcf\xef\xec\xe6\x74\xff\x78\xb2\xbc\xe8\xc1\x58\xae\xac\x1b\x37\x7a\x14\xb6\x82\xbd\xc1\x68\xfe\xaa\xfb\x0a\x72\x36\x6e\xf9\xeb\x91\x38\xe8\x8c\x06\xa6\xca\x40\x30\xf5\x0a\xce\xf0\xca\xfc\x18\x6d\xc4\x4e\x87\xb3\xb7\xfd\x2e\x4c\xa0\x95\xf5\x26\x78\x9e\xd4\x5b\xab\xd2\xf9\xef\xea\xad\xf8\xbc\x94\xd4\xcd\x05\xc0\xb9\x07\x83\x04\x5a\x44\xf6\xf8\x0e\xd5\x6f\x16\xb3\xb5\x05\xc4\x61\xf9\x9d\xec\xef\xe9\xa9\x77\x95\xaf\xc6\xcc\xb3\x39\x32\x98\x35\x4f\x88\xe8\x55\x0d\x3f\xe8\x6e\xda\xaa\x12\x99\x1f\x48\x32\x9c\xe1\x5a\xdb\xdc\xad\x03\x4d\xfb\xc4\x8e\x68\xad\xe2\x81\x48\x6c\xbd\x8d\x87\x34\x4a\xf1\xf8\xc6\x4b\x7a\x3b\x5c\x84\x49\xb3\x21\x6d\x51\x45\xec\x89\xbf\x07\x71\xfd\x91\x32\x41\x72\xcf\xce\xb9\x4c\x95\x2f\xc3\xa4\x9e\xf1\x32\x11\x7b\x57\x50\xb0\xc9\xe3\x02\x6d\x6e\x46\xae\x9e\x3c\x3c\xae\x80\x15\xa7\xb3\xb7\xeb\x8b\xde\x74\x1c\xf1\x2b\x79\xf1\x5a\x82\xb6\xd1\x72\xc5\xca\xe9\xe6\xd5\x01\x9e\x56\xe3\x3f\x09\xbd\xbf\x34\x0f\xec\xaa\x8b\xe9\x0f\x97\x40\x92\x67\xb8\x7b\x8b\xe9\x11\xb5\xc3\xd7\xf5\xb8\xe5\x46\xf2\x9d\x24\xa6\x71\x90\xe2\x27\xa3\x95\x75\xe0\xef\xbd\x5a\xb8\xa4\x4c\x3a\x6e\x4f\x55\x38\x1c\x75\xa8\x15\x3a\x33\x37\xe5\xd2\x08\xdb\x3c\xbf\xef\xf8\xfe\x37\x1c\x17\x74\x36\x45\x03\x9e\x93\x7e\x8c\xc1\xdf\x17\x91\xfe\x74\x3d\xfb\x50\x86\x65\xf1\xb1\xba\x4e\xc6\xa3\x39\xac\xf4\xb8\xb8\x8e\x92\x4a\x1d\x3a\x7d\xad\x3b\x09\x1d\x7c\x32\x69\xc5\xd1\x38\xb9\x46\x0f\x90\xd3\x09\x48\xc7\xb1\x8c\x2a\xbd\xd0\x4d\x8f\xa9\x6e\x93\xef\x96\x35\x64\x3a\xb4\x97\x8f\xaf\xd8\x3f\x33\x1e\x98\x33\x23\xea\xaa\x84\x5b\xd5\x0d\x5c\x46\x97\x38\xba\x9e\xbe\xe1\x18\x27\x3e\x24\xb2\x73\x26\x6e\x30\x36\x9c\x81\x3f\x93\x27\x8d\x36\x7a\x95\xaf\xa2\xeb\x74\x7c\x13\x82\xbf\xd8\x5b\xbe\x45\xd0\xfc\xaa\xce\xc7\x23\x10\x8b\x20\x6b\x9d\x34\x7d\x9e\x38\x8e\x63\x34\x5e\x82\xd9\x16\x85\x15\x46\x65\x3c\x8a\x0c\x50\x13\xaf\xca\x13\x71\xd6\x2f\x99\x80\x8b\xa9\x4c\x57\x5d\x41\xad\xb9\x74\x48\xdd\x00\xc6\x3a\x0f\x5f\x12\x9a\x01\xdd\x28\x22\x96\x37\xf8\x94\x02\x5c\x87\x7f\xe4\xc1\x3b\x0c\xf4\x60\xbf\xca\x74\xbd\x0c\x72\xd2\x75\xdf\xb2\xa7\x08\xcb\x1c\x73\x85\xfc\xcc\xf5\x5b\xf3\x04\x10\xa3\x12\xcf\xe9\x3b\x1e\x26\x73\x33\x72\x69\x60\x5f\x6b\x16\xd5\x0b\xbb\x06\xaa\xfb\xa0\x19\x40\x7f\x2b\x9e\x70\xe1\xd8\x69\x1e\xd4\xc5\x0e\x4d\x01\xba\x95\xdd\xc2\xf4\x21\x47\x56\xa7\x27\x96\xb3\x80\x7b\xaf\x0d\xf7\x5e\xa7\x85\xd9\xb0\xce\x12\x9c\x9c\xca\xd6\x92\x61\xdd\xed\xbb\x25\x0a\xa4\x7b\x1a\x82\xd2\x39\x50\xc3\x8c\xd6\x21\x96\x02\x61\xdc\xb1\x33\xc7\x6d\x7b\x73\x82\xc6\xa1\xe3\xf5\x8d\xad\x5b\xdb\x6c\x36\xa6\xba\x47\x5b\x76\x09\x88\x18\xd2\xc6\xcc\xbe\x61\xc3\xbc\x65\x8f\xd7\x22\xce\x52\x6e\xf1\xd7\x7a\x45\x90\x58\x25\xf3\x6c\xa8\xf2\x0b\x2f\xd5\xe4\x87\x39\x2f\xd5\xe5\xed\x43\x2c\x5d\xa4\xad\x7f\x34\xd3\xd7\x6f\x06\xb8\x01\x28\xfe\xad\xee\x68\xff\xe9\x78\xfc\x4f\x47\xc1\x85\xb9\x1d\x0b\x0a\x98\xe0\x58\x1a\x59\xf9\x89\x39\xaf\x96\x68\xd3\x77\x23\x61\x25\x71\x82\xe3\xee\x68\xd4\x43\x7b\x7d\x9a\xe9\x9b\x54\xf1\x57\x6f\x73\x2a\x40\x06\x2f\x41\xb4\x73\xa0\x9f\x64\x50\x19\xc0\x78\x36\x25\x5f\x22\x0b\x0d\x05\x24\x84\xdf\xa6\x76\x16\x92\x59\xcb\xb6\xee\xd6\xcb\x6b\x3a\x13\xd4\x37\xfe\x2a\xbf\x92\x1b\x80\xec\xa5\x80\xc3\xf4\xac\xda\xf9\x14\x6a\xe6\xf4\x62\x95\xd3\x2c\xf5\xe0\x1f\x6b\x38\x97\xd2\x69\x5c\x72\x29\xb3\x32\x2f\x07\x27\xbc\xd6\x57\x27\xf3\x2a\x3e\xe5\x1b\x9d\xac\xb9\x63\x8a\x51\x60\xe3\x1a\x5b\xb3\x6d\x19\xa1\xa9\x7f\x36\x33\xf8\x39\xb3\x9e\x25\x42\xd3\xfd\x6c\x06\xc5\xec\x68\x37\xdd\x4f\x5f\x57\x85\x1d\x6e\xf7\x12\x34\x1f\x2b\x38\x60\x8f\x33\x1b\x5d\x44\xce\xed\xc6\xd1\xbc\x8c\xf7\x1b\x3b\x85\x38\x67\x7a\x90\x94\xc1\xb7\x4c\xdc\x67\xa3\x94\xbb\x57\xe0\xed\xe6\xcf\xb7\x47\x00\x29\x2e\x94\x81\x4c\xe8\xa2\x78\xaa\x07\x9f\x5e\x61\x63\x9e\xf1\x99\xb0\xbd\x9d\xec\x99\x28\x99\xce\xcc\xb0\x9a\x5b\xa5\xae\x97\x0d\xac\x0a\x0b\x05\x73\x4a\xc8\xd8\x02\xe2\x1b\xef\x0d\x83\x01\xa1\xb6\xf3\x5a\x56\x03\xc0\x1f\x56\xe5\x14\xb2\x6a\xff\x23\x40\xd4\x18\x76\x8d\x97\x51\x56\x17\xad\xa1\xcb\x84\x2b\xcb\xe2\x97\x2b\xf8\x98\x51\x82\x0e\x34\x03\x9a\x6c\x27\x2b\xb6\x19\x59\xdf\x7f\x33\xd2\x5d\x73\x07\x39\xb5\x6b\x9f\xe0\x81\x9b\x0c\x0a\x9a\x51\x24\x98\x50\xa5\x5a\x2f\x41\x65\x89\x3d\x33\xb3\x3e\x1a\xfb\xc9\xd4\xef\x7b\x5a\x11\x5b\x9b\x73\xe5\x53\x57\x1e\x14\x08\x54\x81\xe2\xce\xb1\x0f\xff\x76\x34\x1b\x5a\x8b\x81\x04\xbd\x50\xf1\x9d\x74\x3f\x46\x6f\x4d\x83\xa1\x2f\x50\x04\x80\x72\xd9\xbe\x5f\x06\xb6\x2e\xb3\x3a\x16\x37\xef\x59\xc7\xcc\x8c\xd9\x90\x66\x45\x9c\xbf\x7a\xe1\x61\x9f\xc3\xae\xe6\xb9\x87\xfb\x1e\x1b\xcb\xb8\x4b\x08\x25\x08\x4d\xcc\x4e\x30\x58\x47\xa8\x4c\x6a\x11\xa0\x8a\x80\xbf\x57\x2b\x6c\x39\x00\xf2\xf7\x0b\xf2\x7c\x3c\x9b\xdb\xd0\x59\x39\xa8\x38\xbb\x77\x2d\x23\x0d\x39\xf2\xc5\xa1\x87\x77\xd3\x23\x45\xce\x38\x5c\xa6\x09\xe2\x1a\xe4\x49\xae\x0c\x82\x58\x24\x62\x1f\xd8\x6a\x6b\x93\x2a\x19\x50\xfe\xd0\xbc\x24\x5f\x0e\xd3\x85\xff\x2e\x0a\xa7\x90\x68\x3a\x0d\xad\xb2\x08\xe7\xf6\x22\x9f\x76\x21\x52\xcd\xb0\x4b\x79\x32\x02\xcf\x29\xc5\x7e\x79\x8b\xfe\x85\xd9\xb4\x5c\x35\xa3\xbc\x15\x31\x96\x25\x2a\xad\x98\xb1\x77\x86\xf7\xb0\xb7\xf9\x3a\xf3\x78\x34\x5f\x37\xca\xd0\xf2\xee\x01\x8c\x8a\x20\x18\x02\xf8\xc3\x9b\x17\x7c\x86\x90\x37\x99\xf1\x21\xc9\x2b\x90\x10\xe2\x7b\x50\x6e\xe6\x8f\x29\xfd\xf8\xb8\x49\xaf\x61\x73\xaa\xca\xb1\x6b\xa8\xc0\xee\x6c\x51\x4e\xf9\x31\xe6\x9c\x19\xc4\x54\xcc\x15\x34\xd3\xd9\x02\x3d\x1f\x74\x67\x34\x4a\x7d\xf3\xd3\x3a\xae\x09\xf6\x51\xe4\x9a\x47\x97\xc2\x71\xea\xce\xfb\xc3\x09\x9e\xce\xbb\x26\x68\xaa\x67\x01\x14\x8b\xbd\xb6\xad\x12\x75\xab\xba\xeb\xaa\x67\x7e\xc6\xcd\xdd\x6e\x65\xb7\x97\x87\xe6\x10\x4d\x04\x64\x37\x02\x03\x37\xc6\x0b\x09\xbb\xc8\xfe\x02\x22\xfc\x82\xe3\x0d\x9a\xdd\xde\xc5\xc1\x2f\xad\x5f\x24\x80\x0e\x2d\x87\xb8\xca\x50\x11\x29\x78\xf8\x4b\x47\x76\x8a\xa7\x6e\x80\x9c\x89\xc4\xad\xa0\x65\x89\xb3\xd0\xb4\xb0\x58\xd4\xcd\xf2\x7a\x35\x52\xbf\x98\x95\x6c\xeb\x3e\x88\x1f\x71\x99\xef\xcc\x6d\x96\xc9\x55\xe9\xaa\xce\x7e\xcb\xe4\x6c\x6b\x0a\x9d\xfc\xcc\x16\x5f\x0d\xa1\xdf\xc3\x88\x52\xe4\xb8\xa3\xa6\x7e\xa9\xa5\xd6\xde\x01\x75\xe4\x26\x20\x10\x55\x03\xfa\x54\xd9\xc6\x3f\x24\x11\x92\xf4\x97\xdd\xe4\x97\x5a\xb1\x18\xfd\x52\x4b\x7e\x51\xa3\x5a\x52\x8b\xd5\xe8\x20\x19\x61\x12\x7c\xe1\x11\x46\x6b\xd7\x23\x61\x0d\x69\x08\xaa\x4e\x0d\x75\x1d\x8d\x07\x16\xc8\x1b\x72\x7a\x6f\x3b\x5b\x5e\x2e\xec\x9b\xbe\x73\xf2\xe6\x06\x3c\x43\xef\x1d\xc9\x2f\xdf\xf1\x31\xc6\xaa\x3d\x47\xd0\xf8\xb7\x8a\x5e\xde\xae\x86\x39\x20\x71\xbb\x8e\xc7\xa1\xfa\x24\x84\x2e\x39\x97\x33\x57\xa4\x4f\x27\x6f\x24\x7e\x15\x37\x9a\x58\xe9\xd9\xc2\x13\x1f\x67\x43\x40\x3b\xf8\x97\x02\x4a\x00\xab\x71\xf5\x81\x79\x21\x6b\xab\x3a\x6b\xe7\x74\xf2\x14\x14\x18\x0d\x1d\x2f\x94\xcc\xa1\x0d\x8e\x3e\x79\xf0\xb4\xfe\x8b\xd1\x88\x5a\x30\xaa\x22\x03\x52\xfb\xec\x50\x06\x15\xa1\x0c\x14\x1c\x69\x4f\xd8\x24\xba\x8f\xf0\x71\x6c\x0e\xa2\x2c\x3e\xb2\xb9\xa1\x05\xcc\xde\x8e\xcc\x86\xbe\x83\xa1\x63\xa4\xdd\xe1\xcf\xd5\x4e\xa0\x86\x13\x63\xa2\xdf\x21\x7b\xdc\xa9\x4f\x51\x09\x53\x0c\x97\xc5\x67\x34\x28\x74\x2f\x24\xee\x4e\x9c\x41\xdc\x5c\xc2\xea\xf0\x69\xb9\xc4\x95\xd7\x68\x14\xa4\x02\xf1\x2c\x94\x89\x10\x23\x5a\x5a\xac\x01\x1f\x3a\x23\x43\x72\x27\x01\x4b\xe8\x86\x5a\xa3\x03\xe4\x5e\x3a\xec\xa6\x9a\x9e\x49\x9f\x44\x95\x63\x79\x1c\x82\xf8\x86\x52\x39\x26\x3b\x51\xc4\xcc\x33\xca\xd0\x50\xf9\x2b\x13\x09\x86\x00\x73\xa8\xad\x47\xa8\x4d\x8c\xd2\xd4\xa3\x6f\x4e\xef\xda\x72\x92\xdb\x34\x03\x97\xcb\xec\x86\x84\x86\xc1\xb6\x9d\x53\x89\xbf\x2d\x7b\x24\x65\x4f\x91\x6c\xbd\xc3\x5a\x47\xcf\x41\x96\xdb\x2d\x3c\xdb\xd2\xf0\xb3\x54\x93\xac\x63\xa9\xe9\x26\x6f\xd2\x6b\x4a\x64\xa2\xcc\x3c\xf3\x09\x9d\x4c\x89\x3b\xe9\xa4\x3e\x69\x43\x3c\x67\xc3\x31\x3d\x63\x42\x85\x94\x7e\xbd\xa3\xcc\x96\x7c\xc1\xd8\xe9\x12\x89\x1c\xcd\xdd\xde\x7e\x51\x94\xb4\x2a\x2f\x6b\x43\xad\x29\xbc\x9e\xb2\x54\xd2\x67\x95\xb0\x9d\x18\xcb\xbc\xe5\x53\xbd\x50\x49\x20\x7a\x1c\x0f\x66\xdd\xd3\xb7\xcb\x4b\x0a\xa6\xe8\xbc\xc0\x4b\x0a\x55\xcc\x99\x4b\x4e\xa5\x50\xde\x29\x97\xbd\x4a\x2f\x06\xaa\x3f\x1e\x81\x5c\x2c\xd0\x1f\xde\x90\xa8\xf7\x38\x86\x72\x58\x06\xc5\xf9\x76\x2c\xa6\xe8\x20\x8b\xe9\xb2\x7f\x86\x7c\x23\x45\xb3\x4f\x7b\xb9\x2c\xb5\x4d\x5e\x76\xee\x2e\xcb\xad\xdd\xa3\x34\xb9\x60\xf6\xfa\x4d\x35\x25\xe4\x0c\x6a\xe4\x74\x12\x2b\xf4\xd4\xc1\xd3\x28\x43\xd6\xbf\x84\x3e\x0e\x33\x80\x74\x5a\xf9\x59\x49\x01\x43\x6e\x14\xd2\xd1\x6f\x84\xb8\xcf\x90\x7e\xee\x3a\x70\x72\x02\x23\xf1\xbe\x4a\xaf\xb7\x21\x4b\xf1\x73\x73\x4a\xa0\xfe\xf1\x1b\xf6\x8c\xa6\x80\x63\xb6\xa0\xc1\x7d\x88\x95\xd7\x2e\x54\x71\x5a\x72\x74\xbb\xbc\xc3\x78\x45\xe8\x6d\x30\x1a\x06\x50\x65\xf1\x6e\x74\x31\x44\x2b\x3e\x48\x55\xb5\x70\x53\x79\x31\x5d\xce\x87\xaf\xa7\xb0\x9c\x46\xe2\x74\x1a\xe6\x08\x86\xdf\xaa\x72\xd8\xc5\xc1\x3b\xe4\x09\x7a\xe7\x36\x4c\xc2\x44\x98\xef\x2c\xf8\xe6\x57\xa7\x89\xa9\xce\xea\xf1\xdb\x3f\x7c\x2b\xe3\x73\x7b\x6b\xb2\x70\x7c\xa1\xac\xa8\xcc\x50\x84\x5a\x96\xcb\x2d\xfc\x99\xa4\x73\x23\xe2\x8a\xda\x81\x46\xf6\xb8\xb7\x3c\x7d\xf8\xb0\xf6\xc5\x3e\xa0\x64\xf8\xb7\xca\x56\x15\x3f\xde\xde\x86\x49\x3f\xa1\xaf\x7d\x40\xcf\x32\x7d\x01\xed\xb9\x0a\xb1\x0e\xe6\x5a\x80\x5c\xf4\xb9\x41\x7b\xea\x9b\x4b\x99\xa3\xcd\x42\xb7\x37\x9f\x8e\xe9\xb5\xef\x02\xe4\x29\x46\x3a\x3e\x59\xc0\x9f\x0b\x98\xaa\xa3\x49\x13\x1d\xce\x91\xd1\x09\xfc\x62\x4b\x94\xa6\xf0\x90\x2a\x98\xc0\x2d\x62\x14\xde\x5f\x5c\x60\xbf\x80\x5a\x78\x91\x0d\x13\xfd\x2d\xf4\x70\x38\x79\xf6\xee\x3b\xe8\x5b\xaa\x5b\xc9\x6e\x04\x15\xaa\x27\x40\x36\xf8\x83\x8a\x1d\x90\xc4\x44\x6d\xb5\xfa\xa1\x5f\xab\x1d\x11\xfd\xaa\x2b\x18\xa4\xcb\xee\xe9\xf0\x47\xc5\xdf\x6b\xf9\xfe\x29\x6e\x3a\x25\x80\x95\x60\x60\xdc\x32\x9c\xf2\x53\xcc\x49\x09\x67\xc0\xe2\x3a\x1b\xad\xde\xa1\x3a\x85\xc1\x67\x2c\x87\x04\x9d\x40\xe7\x4b\x43\xf4\xdc\x1b\xc7\xea\x50\x5a\xd1\xb0\x3b\xec\x5b\x10\xa7\xa6\x57\xfd\x2b\xb4\x17\x02\xfa\x3d\xa3\x86\xdf\xe0\xc0\x68\x89\x7c\x68\x10\xac\x60\xbd\x2a\x52\xbe\x62\x2b\x73\xde\x4b\x48\x54\x06\x6f\x2e\x08\x03\x95\x2a\xf7\x6e\x7a\xd9\xf1\x97\x93\xd3\x89\x73\xc3\xbf\x3a\xa8\xb5\xeb\xcd\xda\xc1\xaa\x5d\x81\x3f\x5e\xc1\x6e\x7f\x3a\xf7\x4a\xd6\xdb\xb5\x66\xa5\x0e\x45\x3f\xfe\xa9\xf9\x1d\xba\x1b\x94\x12\x7e\xad\xf9\x68\x12\xd4\x3a\xeb\x8e\x4f\xa0\x0a\x55\xad\xc8\x07\xd7\xe7\xb2\x3e\x7a\xa3\xc9\x99\xad\x1f\x45\xab\x84\x8a\x0e\x41\xd1\x5a\xc5\x71\xa5\xbe\xb7\x8a\xf7\x1a\xc1\x61\xeb\x7c\x4b\x8d\x72\x46\x0d\xe0\xcd\xbc\x1a\x8d\x12\xb5\x12\xef\x45\x2b\xd0\xd2\xbd\x5a\x67\x5d\x1c\x64\xb7\x77\xba\x22\x25\xee\x35\xe2\xb8\xb4\xf2\x6a\x3c\x43\xbb\xab\xf0\xf2\xf8\x6c\x3e\x8e\xce\xd4\x5c\x8d\xbd\x03\x5d\xc8\xf8\xbd\xc9\xf0\xca\xdb\x64\x3e\x8e\x3a\x4b\xce\x58\xc3\x9d\x27\xec\x19\xbc\x3a\x4e\xc6\x61\x0b\xc7\xb3\xd3\x9e\xae\x65\xb2\xae\x22\xeb\x6a\xfa\xec\x60\xff\x71\xad\x7d\x56\x49\xe0\x0f\x0c\x3f\xbd\x1f\x29\xe3\x47\xac\x1e\xc3\x67\xfb\xa2\x5e\x8e\x2e\x1a\x95\x8b\x7a\x5c\x3a\xdb\x83\x22\xf5\x2f\x28\xb5\xd1\x6c\x3c\x0c\xb2\x23\x48\xa9\x9c\xc5\x58\xe8\xa2\x6e\xf1\xb8\x72\x5b\x23\x2a\x91\x85\x5c\xd4\x78\xf4\xa8\x84\x59\x12\x80\xb9\xae\x2e\x1a\xc6\xe0\x39\x61\x2b\x93\xb3\x18\x18\x2d\x3a\xfb\x1d\xa1\xf3\x04\xb0\x2c\x23\x92\xd0\x1b\xc9\x9f\x63\x7e\xed\x60\x0e\xff\xce\x81\xbd\xea\x4d\xe8\x66\x52\x3b\x18\xc3\xf7\x98\xbe\xc7\x00\x34\xa9\x3e\x3a\x48\xc6\xed\x71\x29\xaa\x97\xe7\x71\x73\x5c\x9e\x57\xc6\xa5\xb9\xba\xa8\x27\x8d\xd2\xb8\x72\xd1\xc0\xc5\x09\x89\x84\x78\x96\xeb\x0d\xe8\x37\x61\x4c\xff\x56\xf0\x3b\x18\xfc\x3e\x8e\x42\x3f\x63\xd4\x4c\x86\x3f\x6a\xb6\xbc\x3f\x6a\xfd\xa4\x9f\x33\x6a\xfd\x31\x46\x83\x0e\x5a\xd1\x14\xc1\xf1\xa1\x6d\x24\x7d\xf7\xf1\xbb\x8f\xdf\x50\x11\x2b\x8d\x15\x91\x18\xe7\xe3\x59\x29\x21\x8b\x9b\xc1\x08\xc4\x56\x5c\xea\x2b\xc3\xa2\x67\xf0\xe5\x63\xc9\x55\xdd\xe3\x08\xe9\xd5\x4b\x93\xe1\x95\xb7\xc9\x37\xba\x0f\xb2\xdb\x4a\xba\xfc\xa3\x97\xf4\xc2\x16\x88\xcc\x52\x0b\xc7\x7c\x9d\x44\xe3\x72\xfd\x71\xbc\x57\xaf\x3f\x56\xab\x64\x5d\xee\xee\x3d\xaa\xd5\xd4\x75\xb2\xae\xf4\xf6\x1a\x35\xb3\x0d\x5f\x25\x52\x7f\xb5\xbe\x86\x39\x57\x92\xaf\x1f\xd5\xda\xcd\x58\x9b\x8c\x9f\x00\x84\x93\x71\x6d\x32\xfe\xa8\xc7\x1a\xfe\x40\x0e\xfd\xdc\xaf\x02\xdb\x3e\x7c\xf4\x10\xe6\x7b\xa5\x5e\x7d\xb4\xff\x79\x7d\xff\x8b\x47\xa5\x75\xa5\xfa\xf0\xcb\x2f\x1e\xed\xd7\x1f\x96\xae\x89\xb6\xba\x78\xa5\xfa\xe5\xe3\x2f\x1b\x8f\x1f\x97\x40\x2a\x54\xbf\xf8\xfc\x71\xad\x5e\xfb\xa2\xb4\x2e\x57\x6b\x0f\xeb\x8f\x1e\x3d\x0e\x0a\x57\x6b\x90\xf6\x70\xff\x21\x80\xae\x36\x6a\x0f\x6b\x8d\x47\x5f\x42\xd9\x3a\x24\x7f\xde\x68\x3c\x6a\x40\xe9\xd4\x28\x10\x7f\xf9\x23\x31\x06\xc9\x2c\x8c\xc7\xc2\x12\xa4\x56\x23\xea\xa9\x2e\xf5\x6b\x30\x3c\x85\xe5\x67\x2e\x83\xfb\xeb\x6c\x11\x75\x4b\xdd\x72\xaf\xd4\x83\x9d\x6c\xdc\x94\x7a\xb5\xbd\x9a\xc2\xff\xc6\xe9\xf6\x98\xa6\x56\x4a\x03\xa2\x8f\xbf\xf8\x72\xbf\xb6\xff\xb0\xbd\x2a\xc1\xff\x9a\xd1\xaa\xf2\x70\xaf\xf1\x65\xbc\xf7\x79\xf5\xf3\x2f\x3e\xaf\xed\x7f\xee\x81\xc0\xae\x22\xf3\x78\x20\x6a\xb5\x2f\xbe\x78\xf4\xb8\x4d\x28\x5d\x4e\xdf\x47\x2b\x55\xdf\xdb\x8f\x9b\x1a\x00\xd0\x0e\x21\xa6\xe0\x20\xc9\x66\x79\xa2\x02\x88\x59\xdb\xaf\x3d\x3c\x48\x66\xed\x7a\xa3\xfa\x65\xa3\x34\x6b\x22\x21\x1f\x95\x4c\x2b\x33\x68\xa5\x51\x7d\x18\x57\x30\x39\xa0\x2c\x80\x7e\xc5\x36\x6d\xfe\x3b\x27\x3d\xf7\x31\xf1\xe0\x00\xd8\x50\x7e\x7d\x51\x84\x36\x15\xfc\x57\x0c\xef\xb0\xb8\x86\xe7\x50\xc3\x07\xe6\xb5\x53\x76\x5f\x5c\x49\x63\x33\x75\x9a\x9e\x67\x6f\x4c\x86\x57\xde\x26\xcb\x6d\x76\xc2\x37\xe9\xd5\xd3\xe4\x34\x7b\x9e\x41\x0b\xc7\x67\xc3\x55\x74\x65\x5a\xa8\x3f\x3e\xb8\x6a\x17\x6a\x85\x32\x51\xea\xa2\xbb\x82\x8d\xfa\x15\xda\x65\x49\x27\x60\x06\xf2\x2a\x7c\x01\xa2\x01\xbb\x1d\x64\xa6\xc0\x73\x70\x02\x8e\xa7\xaa\x20\x41\xc1\x3a\x23\xa7\x22\x28\xc5\xf9\x29\xf8\x0c\xc3\x7b\xc3\x7f\x3d\x7e\x1d\x0d\xe2\x76\x2f\x3a\xec\x56\xae\x3b\xe5\xf8\x28\xc2\x07\xbb\x47\xf1\xde\x88\x3d\xd8\x32\xa4\x38\x96\xa8\x54\x20\xb2\x2f\xea\x87\x8d\x8e\xe3\x35\x00\xa4\x35\x85\x1d\xc6\x27\x40\x05\x68\xad\xd0\xd4\x2b\x05\xac\x89\x84\xce\x37\xe3\x69\x77\x01\x55\x51\x13\x57\x7e\x0a\x54\xdc\xc3\x78\xa3\x7e\x6a\x83\x53\x63\x0a\x08\x55\x80\x5e\x18\x98\x22\x20\x4c\x4f\x65\x40\x05\x76\x76\x4e\x3d\x37\x07\xda\xd1\xca\xaa\x3e\xb2\xa7\x62\x74\x2e\x4e\x47\x21\xba\xfb\x6d\x6c\x98\x0e\x80\x66\x7c\xf8\x7e\xca\x7f\x7a\xe8\x06\x96\xec\x2f\xb9\x64\xb1\x58\xf8\x37\x74\xe3\xcd\x5f\xd6\xd2\x0e\x56\x82\x87\x36\x99\x0f\x21\xdb\xd1\x2c\x28\x57\x8f\xd5\xac\x0c\x5c\x74\x1a\xa4\x37\x62\x75\x5a\x06\xa6\xea\x05\xe9\xfb\xb1\xea\x95\x13\xc0\xe1\xf3\x10\x36\xee\xf9\x75\x8a\x63\xd1\xa7\xa0\xc6\x69\x3a\x7d\x5f\x3d\x8a\x2d\x70\x9b\xfe\x48\x7d\x0e\xca\xf2\x8c\x7d\xfe\xbe\x80\x5d\x0e\x4c\xe3\xc7\x08\xc1\x24\x9c\x52\x42\xcf\x26\xf4\x30\x21\x56\x76\x2e\xa5\x59\x14\xf9\x42\xe6\x0d\xc5\x59\x03\x05\x08\xb8\x3b\x31\x6c\x3e\xdb\x4b\x90\xd3\x4f\xf9\x4f\x8f\xfe\xe0\x9b\x58\xd1\xe3\x70\x92\x70\x75\x35\x48\xe0\xa3\x02\x75\x40\xb5\x88\xe0\x67\x19\x7e\x82\x1a\x69\x8c\x9d\xda\xd1\x1c\x35\x8c\x71\x7b\xb0\x67\xb2\x9b\xf0\x1b\x74\x22\xae\x17\xab\xb3\x64\x86\xf6\xc8\xab\x76\x74\x5a\xe9\xc5\x7b\x83\x72\xd4\x3b\x38\x6d\x3f\x6e\xd6\xe2\xe6\xa9\x64\xf4\x2a\x33\xcc\x68\x34\xa3\x59\xe5\x14\x7f\x3d\x54\xb0\x7c\x83\xd6\xd3\xc4\xd5\x1e\x24\xf7\x3c\x19\xe3\xb9\x5b\x9d\x94\x9b\x33\x62\xb6\xb3\x6c\x35\x11\xbb\x8f\x12\x59\x8b\x19\xcd\x70\x28\xe8\x67\x48\x58\xe7\xfb\x14\xe9\xea\x7c\xf7\x78\x9f\x46\x0b\xae\x96\xec\x51\xf5\x61\xbd\xf1\x10\xd6\xb1\xd2\xac\x5c\xdd\x7f\xf4\xf9\xa3\xcf\x1f\xd7\x4b\xa7\xe5\x2a\x28\x82\x0f\xf7\x3f\x7f\x54\xc2\x1e\xc9\x82\x1c\xf3\x8a\x6c\x6b\x36\xea\x8d\xc7\x9f\x37\xbe\xc4\x9a\x9f\xd7\x1f\xd5\x1f\x35\x1a\x58\xb3\xf6\x79\xa3\xee\x55\x84\xdd\xd6\xb5\x5f\xb1\x56\xff\x72\x7f\x7f\x9f\x2a\xd6\xeb\x5f\xd6\xbf\xa4\x7a\x5f\x3e\xc2\x15\xa0\xee\x54\xfc\xa3\x35\x3a\x63\x1d\x06\x34\x09\x58\xbb\x41\x8e\x83\x22\x51\x82\x95\x6b\x1d\xab\x06\xfe\x5a\x57\xc2\xb5\xd6\x92\x44\xeb\xf0\xcc\x13\xf1\x93\x04\x16\x73\xd0\x08\xda\xb3\x3d\x5a\x68\x9a\x66\x81\x89\x00\x1b\x5c\x5a\xf6\x68\xe1\x51\xb8\xd6\x64\xcb\x46\x99\xfd\x7d\x66\xbe\x93\xc4\x91\x3c\x7d\x8d\x71\xe1\x77\x38\x87\xfb\x7a\x9a\xf5\xad\xcb\xf6\xb6\xbb\xf2\x55\x61\x85\x3b\x89\x9b\x27\x5e\x43\xf4\x1b\xcf\x91\xb7\xdc\x14\x5e\xb5\xaf\x32\x6e\xed\xae\xfc\x43\x8f\xd1\x60\x88\x6f\x19\xd7\xae\xd9\xb6\xbf\x2c\x9f\xcd\xde\x01\xbc\xb4\x93\x5a\x03\x1a\xad\xe1\x2f\x46\x17\x43\x2c\xe6\xdc\x39\x49\xb1\x06\xf4\x31\xbc\x19\x03\xf9\x95\x46\x57\x83\x40\x75\x56\x80\x24\x06\xac\xfe\x91\x98\xd3\x24\xc0\xcb\x6f\x59\x23\x68\x51\xd8\x84\x1d\xb9\xab\xc2\x8d\x73\xd8\x8f\x79\xfa\xfa\x81\x5d\x76\xe2\xcb\x94\xc5\x12\x63\x3c\xfc\xba\x44\x97\x28\xfc\x49\x9e\xd7\xf9\x67\xb1\xa8\xb3\x34\x68\x7e\x8a\xcd\xb9\x07\x09\xf0\x61\xb1\xb8\x5f\x83\xed\x0a\x25\xdc\xde\x02\x2f\xa3\x2d\x36\x7d\xb1\xf1\xa3\x38\x47\x35\x0e\x80\xe9\x48\x14\x30\x57\x02\xd9\x31\x3b\x0f\x6d\x2d\xc9\xab\xad\xad\x80\x41\xc6\xa6\xa3\xc1\x4e\x6d\x63\x4a\xa0\x23\x3b\x17\x22\x36\x25\x6e\xf2\xf2\xa0\x98\x66\x49\x28\x9c\xcd\xd0\xa9\x8f\x31\x27\x1d\xec\x57\x8d\x59\x74\xa1\x47\x17\x69\xf3\xe1\x64\x00\xcb\xf5\xe5\x6c\x7a\x3a\x23\x9f\x78\x05\x6c\x14\xfe\x10\x5c\x7c\xab\xce\x8f\x8a\x38\xf0\x20\xc1\xa6\xfb\x93\x1f\xbf\x7b\xf9\xfb\xc5\xe2\xf2\x0d\x27\x99\xa1\x31\x43\x2e\x53\x66\xd7\x1e\x69\xfd\xf8\xf5\xf4\xa2\x3b\x9a\x48\x8d\xdb\xdb\x02\xa8\x0e\x67\xcf\x66\x43\xe2\xe7\xee\x78\x5e\xa0\x40\x73\x92\xb9\xbb\xf7\x33\x05\xc9\xc3\xfd\x62\x33\x6e\xe3\x2b\x9b\x3d\x1b\xc8\xe7\xf6\x36\xf2\x70\xf1\x20\xc7\xaa\xc0\x2e\x00\x1d\x80\x6d\x3d\xce\x9c\x93\xd8\x4f\x8e\xf3\x25\xdc\xd3\xb4\xe9\x14\xbf\x19\x07\x7a\xc8\xc7\x86\xbe\xdb\x14\xcd\x34\x50\xe6\x2d\x96\x39\xd8\x47\x56\x12\x0e\x34\x84\x02\x30\x9a\xae\xb6\x3a\x1d\x77\x6e\xb9\x91\xa0\x4f\xe4\x2c\x33\xc2\x1a\x46\xc6\x20\x67\xdd\x3f\x28\x28\x21\x31\xd8\x02\x2b\x71\x5f\xf5\x25\xcd\x86\x32\xca\xe8\xb7\x77\x31\x7d\x89\x2f\xb6\xd9\xe6\x3f\x15\xee\xf2\x49\xa3\x2d\x8c\x20\xa6\x17\x91\x6b\x67\x21\x17\x57\x7e\x09\xef\x8b\x0b\x42\x43\x88\x5d\xcc\x38\x1a\x11\x61\xb0\xf4\x11\x0c\x71\x68\x47\x9e\x50\x31\x16\x84\x68\xdc\xe1\x81\x6f\xea\x72\xdc\x8e\xc7\x9b\xf7\x6e\xcb\xab\xc5\xf6\x42\x04\xdb\x4d\xf7\xe1\xe7\xc1\x36\xf9\x06\xca\x46\x1d\x16\x40\x8d\xc4\x79\x37\x9d\x2f\x0a\x1d\xe3\x2a\xc0\x40\xd0\x0f\x48\xa0\xf0\x21\xff\xee\x64\x38\xee\xc1\xd6\x71\x02\xcb\xed\x14\xb2\x86\x2e\xad\x4d\xf0\xb3\x2e\x5e\xf1\x85\x91\xd2\x75\x93\xa0\x4d\x0a\x04\xeb\x48\x58\x90\x96\xf7\x5e\x11\xb0\xaa\xbb\x1a\x10\x28\x8a\x61\xc9\xab\x80\x99\x17\x97\x43\xd3\x1c\x4a\x78\x3c\xec\xe7\x21\xd5\x23\x07\xe2\xa1\xdb\xc7\x30\x81\x38\x89\x85\x93\x60\xd6\xcb\xaf\x2a\xe7\x99\xd2\xe5\x82\x2a\xed\x95\x0a\xb6\x81\xf9\x70\x21\xd2\xe0\xf7\x54\x23\x0e\xaf\x70\x75\x8c\xc2\xbc\x0a\x3c\x61\x3c\x16\xb6\x57\xfb\xa4\xd8\xdb\xd5\xcf\xf4\xea\x0a\x84\x09\x2c\xcf\xdf\xdd\x99\x63\x18\x99\xbb\xbd\x9b\xb8\x4c\x45\xf7\x72\xc1\xaa\x84\x3c\xe8\x7e\xe8\x7a\x9a\xd2\xc5\x22\x0e\x27\x5e\x57\xb1\xd8\xb6\xe3\x47\x89\x2c\xd2\xcd\x40\x6b\xf1\x71\x63\x0d\x3f\xf0\xa1\x87\x91\x2a\xf6\x11\x45\xd5\x2e\x12\x69\xe9\xe3\x50\x1b\xc3\x3e\xd1\xf8\xe1\x60\xf3\x8c\xe4\xe0\xaa\xc4\xec\x88\x5b\xb7\x37\x9d\x65\xf9\x9e\xd2\x30\x28\x3f\x92\x0a\x74\x0d\x4f\xcf\xaa\xb0\x3d\xf3\xfa\x01\xe4\x7a\x41\x33\x8a\x46\xbd\x0d\x25\x9a\xd8\x00\x6e\xcb\x58\x69\x38\x3e\x19\xad\x9e\xe9\x9e\x19\x42\xc4\xa1\x6e\x91\x5d\xcc\xec\xc0\x13\xdb\x86\x96\x09\x56\x86\x53\x6c\xe6\x6c\x32\x26\x09\xe5\xea\x15\xa7\xc9\xb7\x6b\x4d\x5d\xc6\x3f\xcb\x06\x36\x98\x1d\xcf\x17\xc3\x4b\x7d\x37\x3a\xc5\x1b\x4e\x49\xc7\x17\xd8\x11\x59\x05\x74\xd7\x36\x75\xfe\x7e\x88\xc5\x2b\x50\xb4\x45\x59\x07\x8d\x87\x6d\x8c\xd0\x3c\x9a\x8c\x16\xc3\x88\x92\xe8\x84\x11\x5f\x0f\xeb\xeb\x33\x53\x7b\xc1\x09\xa4\x90\x79\x29\x89\x73\xd9\xe6\x61\xc6\xed\xc7\x4e\x0d\x32\x5f\x41\x97\x8d\xb8\xd5\x49\xa7\xd6\x6d\xc9\x13\x8c\xef\xe7\x83\x0b\xc6\xc1\xed\xa9\xa1\xc0\xd7\xe8\x22\x1a\x7e\x88\x7f\x50\x53\x8c\x2d\x60\x2c\x29\x80\xc0\xcb\x21\xce\xd7\x56\x50\xa4\x15\x43\xed\x83\x24\x48\xad\x2e\xf8\xd6\xcd\x4b\x3b\x49\x95\xea\x47\x50\xb9\x92\xaa\xeb\x52\x20\x44\x44\xca\x4c\xac\xf1\xcf\xfb\xac\x71\xe6\x91\x33\xb6\x5d\x8b\x9a\x5a\xd4\x33\x7a\x43\x0f\xe8\x93\xfa\x5e\xad\xb5\xa8\xb7\xe2\x45\xbd\x7a\xd2\x86\x72\x8b\x5a\x7b\x51\xab\x4e\x12\xf8\x9e\x34\xd3\x95\x38\x3d\x82\x7f\x17\x4f\x30\x0b\x2d\x33\x10\x0c\x26\xc4\xd8\x4e\xb4\xa8\xc1\x47\x5c\x9d\xb8\xfb\x2f\x07\xc8\xbb\xee\x68\x9c\x20\x4a\x90\xe4\xef\x5c\x68\xfb\xcf\x4f\xf5\xd0\xaa\x4a\x82\x10\x25\x66\x8f\x55\xaf\xa9\xfd\x52\xb7\x37\x8f\xbe\xa8\xa0\x31\x89\x76\xcf\x83\x2f\x45\x9b\xa3\x83\x2f\xec\xb4\x71\x76\x2c\x7b\xe7\x8e\x81\xa2\x93\x5e\x3a\xdf\xa8\x39\xbd\x61\x6c\x0e\x36\x19\x68\x1c\x5f\xce\x86\xfd\xd1\x1c\x6b\xad\xd4\xa5\xa9\x77\x59\x89\x56\xbc\x07\xeb\x0f\x47\x72\xe6\x3a\x9e\x9e\x46\xab\x78\x8f\x7e\xbf\x7c\x55\x07\x66\x0d\xee\x84\x04\x22\xae\x5d\xfa\xfa\xd9\x39\x12\x0d\x8f\x01\xbb\x03\xf6\x94\x9f\x4e\x7c\xbb\xbc\x40\xab\x40\x35\xd5\xc6\xac\xd3\xea\x3c\xe9\x96\x7b\xaa\x77\x95\xac\x2a\x5d\xd5\xc5\x3f\xbd\xab\xd6\xb4\xba\x48\xba\x95\xee\x55\x39\xea\xc1\xa7\x8f\xcb\xe9\x70\x0a\xd3\x03\x14\xca\x8b\x6f\x87\x53\x58\x19\x67\xeb\xe8\x54\x7e\x38\x8f\xfb\x74\x52\xb1\x98\x59\x05\x57\x86\x30\xca\xa4\xae\x42\xaf\xab\xe2\x2d\x15\x0f\xbd\x92\x9d\x8c\xe6\x73\x10\x7e\x39\x9a\x0c\x23\x72\x45\x00\xca\x28\x86\x3d\xb0\x36\x68\xe3\xe9\x1c\x3d\x02\xb3\xb3\x74\x5d\x40\xbf\xa0\x74\xaa\xe8\x8d\x34\x57\xa0\x49\x6f\x0c\x6f\xd0\xaf\x03\x79\x34\x8b\xcc\x7b\x02\x5b\xd3\x05\x72\x38\xb2\xef\x27\xab\x97\x62\x15\xa0\x73\xf1\xa1\xb9\xf3\x55\xf7\xbe\x1a\x1d\xf4\x70\xef\xb4\xf7\x9c\xd4\xf8\xec\xee\xbe\x9e\x8e\xd7\xa7\x14\xae\x35\xdd\x63\xb1\x47\xcc\xeb\x9f\xdf\xb1\x4b\x06\x14\xf4\x6d\x2b\x69\xdd\x2e\xaa\xba\x83\xb4\xc0\xca\xc6\x1b\xbd\xca\xbc\xc1\x50\x99\xdc\x92\xdd\x33\x63\x80\x15\x36\x53\xf8\xf0\x37\xf5\xf1\x2f\xf1\xcd\x87\xbf\xb9\xf7\x4f\x90\x94\x7c\xfc\x4b\xc9\x26\xec\x35\xca\x1f\xff\xb4\xf7\x90\xce\x9b\x06\x1f\xfe\x96\x7c\xf8\x5b\xe5\xc3\xdf\x6a\x40\xc9\x39\x94\x34\x77\x58\x00\x08\x5d\x88\xea\x24\xbc\xbc\xc2\xa4\xf3\x84\x12\x6b\x25\xfa\xa3\x96\x09\x55\xab\x95\xe8\x4f\xf9\xbc\x64\xea\x03\x64\xd8\x09\x27\x92\x82\xd5\x31\xa5\x15\x76\x66\x79\x41\xef\x25\x9c\xeb\x95\x2b\xb5\x04\x21\x0d\x18\x01\x62\x8c\x54\x8d\x1b\x61\x74\x6a\x8c\x00\x6d\x8c\xa1\x50\x0d\xba\x07\xff\x98\x0e\xd4\xa4\x94\xdb\x10\xb3\x91\xd5\x5a\x84\x4c\xe9\x12\x86\x92\xd4\x7c\x84\xe0\x01\x89\xd8\xa1\x9d\x46\xc8\x21\x53\x12\x21\x06\x40\xe2\x38\x4d\xe3\x58\xe3\xec\xd2\x10\x15\x23\xdb\xb2\x30\xaa\xab\x53\xb9\x03\x2a\x1d\x0c\xce\x54\xb0\x76\x1f\xd8\x60\x38\x87\xc6\xa2\xf9\x25\x46\x8c\x41\x73\xf1\x1b\xa6\x4a\x62\x52\x70\xc2\x00\x8a\xf6\x9b\xa6\x4c\x38\xd0\xda\xe8\x81\x72\xec\x10\xe2\x08\x3a\x49\x88\x3e\x26\xb9\x7d\xe9\xe4\xa3\xf5\xf5\x74\xe1\x5b\x59\x03\x2e\xa5\x1e\xfc\x53\xee\x02\x16\xf0\xab\x8e\xbf\x1a\xf8\xab\xb1\x05\xcc\xb3\xd9\x74\x3e\x77\x01\x1d\x4a\xf5\x46\xa7\x22\xd5\xa1\x4f\xf2\xab\x86\x69\x35\xce\x55\xf2\xab\x8e\x69\x75\xce\xdd\xd2\x0e\x1a\xbd\x52\x2b\x58\xad\x9c\x60\x69\x85\xf5\xf0\xa7\x34\x80\x3f\xb7\xa1\xca\x5e\x15\xae\xf8\xd5\xa5\xd1\x41\x0f\x39\x01\x91\x39\x57\xf2\xbb\xee\xfc\x06\xc4\xcf\xb7\x00\x7d\x85\xcb\xdb\x78\x74\x3d\x8c\x06\xda\xc0\xd0\x5e\x38\x0e\x10\x2a\xfe\x53\x1e\x20\x4c\xfc\x07\x7e\x35\xf0\x17\x4a\x43\xcc\xd9\x4b\xc6\x0a\xd3\xf9\x6f\x03\xff\xa6\x45\xa2\x66\x8f\xc8\x34\x6b\xb0\x77\x66\xa6\xc9\x24\x2e\x32\x1f\x72\x37\x43\x96\x24\x36\x15\xda\x4f\xf7\xca\x34\xf4\x1c\x1f\x4e\xf8\xec\x01\x9a\x07\x92\xbe\x82\x94\x8f\x9f\x7c\xf8\xaf\x62\x91\x93\xea\x98\x54\xa7\xa4\x34\x95\x86\x18\xe7\x60\x34\xd8\x22\xfe\x24\xc6\x47\xc0\xf2\xde\x05\x7d\x2b\x0b\xda\x8f\x3f\xfd\x31\xfa\xf4\xf9\x90\x16\xdb\x29\xa0\x2b\xb5\x56\xd7\xf1\x0d\x19\x4a\xba\x05\xfe\xa3\xa6\x82\x94\x1f\x6b\xe5\x24\x5a\x55\x52\xa9\x74\x08\xbf\xbd\xea\x4f\x58\x75\x1d\x56\xfd\xe9\x3e\x55\xff\x88\x55\xaf\xc3\xaa\x7f\xcc\xaa\x9a\xdb\xd9\x97\x76\xb5\xbf\xf7\x3a\x75\x8f\x81\x52\xab\x24\x63\x48\xd6\x49\xc6\xa0\x5c\xfb\xcb\xd6\xfb\xc4\xe1\x63\x3b\x7f\xa2\xf7\xc9\xba\x56\xba\xae\x5c\xd7\x4a\xeb\xb8\xf4\xbe\x0c\xdf\xf0\x73\x55\x59\x41\xa2\x7c\xc3\xcf\x75\x05\x0a\xad\xe0\x1b\x10\x80\x1f\x65\xf8\x5a\x97\xaf\xb1\x48\xc8\x39\xff\x51\x2f\x27\xef\x53\xc3\x88\x89\xa5\x68\x55\x2b\xc3\x7f\xc9\x8a\x77\x20\xde\xb0\x70\x81\x35\x14\x58\xd7\x92\x75\xba\xc0\x1f\xb9\xc0\x35\x14\xb8\xae\x25\xd7\xe9\x02\x96\xb9\x6a\x6a\x5d\x53\xd7\x35\x39\x38\x96\xaf\x10\xcd\x9c\x05\xf1\xd3\x27\xce\xaa\x96\x39\x20\xb5\xcc\x11\xf1\x57\xc1\xb0\x0b\xa9\x25\xf8\xee\x2e\x6e\x65\x3f\x52\xa2\x6e\xb2\x1b\xc9\x02\x9d\x0b\xec\xef\xd1\xb9\xfe\xa5\xbc\xdc\x5f\x59\xa6\x55\xfd\xb5\xe5\x58\xd5\xbf\xb6\xec\xaa\x2e\x9c\x75\xa2\xbf\x2a\xf5\x57\xe5\xfe\xba\xd4\x5f\x97\xfb\xd7\xa5\xfe\x75\x0c\x9a\x9b\xcf\xcb\xa0\xab\x5d\x14\x8b\x38\xed\xc9\xa4\x70\x19\xef\x5d\xf8\xb3\xe6\x02\xf4\xb2\x90\x91\x7e\x6c\x94\x93\x2b\x00\x9d\xe2\x68\x4e\x5f\xa7\x18\x99\xd3\xaf\xc3\xf4\xff\x97\xcd\x1b\xab\x57\x7e\xf2\x0c\x22\x8d\x51\x89\x46\x78\x27\x8f\xff\xb7\xce\xb8\x8c\xce\xa7\x10\xbe\xa7\x76\x9a\x5b\x2f\x7b\x62\xe6\xd0\x25\x73\x5a\xfa\x27\x30\xb3\xe5\xd0\x1c\x3a\xee\x66\xac\x3f\xe3\xd1\xa5\xde\xd9\x49\x68\xc0\xb9\x92\x27\x10\xf8\x10\xe1\x92\x26\xf2\x8b\x09\xf9\x92\x74\xe2\x25\x06\x5b\xbf\xf9\xb2\xc7\xd1\x96\x3a\x54\x09\x1d\xc2\xa2\xd5\xb8\x00\x4c\x1f\xf3\xeb\xc0\x87\xe4\xd2\x2d\x8a\x26\x26\xd4\xa1\xb9\x59\x7e\x92\xd4\xe4\x4d\xd1\x44\x5d\xd6\x74\x3e\xea\x98\x97\x75\xf3\x35\xe9\xc8\xe3\x83\x0c\x55\xe9\xb2\x06\x25\x01\x44\xe6\x86\xda\x71\xe4\x64\xbc\x35\x05\x7b\xe8\xc8\x69\x75\xd4\x89\xa9\xe5\x1a\xaa\x55\xfa\x18\x29\xbd\x73\xd6\xf7\x98\x14\x94\x42\xfb\xb0\x09\xa8\xfc\x02\x69\x28\x0e\x31\x11\x47\x1d\x22\x92\x4e\xa2\xf1\x3e\xa0\x77\xcf\x8a\x54\xa1\xab\x76\x61\x3b\xdc\xad\x4e\x13\x74\x41\xc9\x7e\xc3\xc8\x6b\x50\x37\xa6\x81\xe0\x8f\x5e\xac\xee\x87\x4e\x3d\x40\xa7\x7e\x5f\x74\xea\x06\x1d\xe8\xc1\xdd\xe8\xfc\x0f\xbc\x88\xa1\x04\x7c\xac\x2a\x8f\x4f\x87\x76\x52\xd8\x66\x80\xff\xcf\x9f\x8d\x66\x7d\xf4\x02\x17\x09\xc8\x3b\xcb\x61\x7a\x6c\x10\x10\x27\x74\x37\xae\xf3\x2e\x9c\x30\xb0\x72\x04\xfc\x3d\xa1\x94\xd0\x8b\x17\xa6\xa1\x87\xab\x61\xc2\xb5\x76\xe9\x8f\x61\x21\xe2\x96\x39\x5b\xce\xb3\x4b\xc8\x44\x5a\x06\x96\x69\xb5\x6c\xbb\x3a\x78\x1f\xbb\x8d\x1c\xcd\xdf\xca\x9c\xd9\xad\x99\xf8\x78\x57\x2d\x7c\x5e\xa7\x83\x0a\x99\x38\x7e\x93\x98\x2f\xdf\x67\xda\x73\x42\x8b\x1b\x35\x05\xae\x55\x26\x97\x0f\xa6\x37\x36\x34\x60\xf5\xca\x14\x9f\xc2\xef\xdd\x9a\x09\xfd\xc7\x4e\x1a\x0c\x42\xb1\xef\xe5\x8c\x9b\x0a\x89\x92\x9a\x2c\xf6\x05\xc2\xdc\x4c\x17\xfc\xa2\x19\xc3\x61\x21\x9c\x68\xab\xba\xe9\x95\x41\x62\x02\xbf\xeb\x56\xaa\xb4\x52\x34\xa0\xeb\x7f\x1f\xd1\x9b\x80\x0c\x97\xd5\x6b\x67\x66\x7b\x88\x73\xdc\x91\x5a\xab\x52\xf9\x44\xdc\x37\xf7\xc0\xfd\x12\x7e\x57\xb6\x21\x7f\xb9\x09\x53\xa6\x2a\x35\x84\x0e\x47\x98\x9f\x9b\xf7\x67\x18\x09\x72\xd7\x8c\x61\xe6\x81\x5d\x86\x5e\x97\x33\x33\xe8\x4a\x94\x3d\xe3\x25\xf4\x3b\x35\x3b\x26\xaa\x47\x6f\xa8\xbb\x9c\x8f\x3c\x2c\x27\x74\xdd\xea\x24\xe9\x49\xea\xa8\xa3\x7a\xd5\xcb\xa4\x0b\xc5\x7a\x2d\x37\xa3\x26\x19\x5b\x51\xf2\x65\x07\x4d\x1c\x99\x45\xd3\xc5\x19\x3e\x65\xc2\x09\x26\x56\xa4\x2b\x1e\x18\x36\x20\xbd\x96\x51\xe2\xaf\x69\xc2\xc5\xe9\x43\x66\x27\x7f\x5c\xa1\x37\x7d\x7e\xd2\xcf\x6f\x1b\x2f\xe9\x22\x36\x13\x27\x46\xe0\x0f\xa3\xf9\x08\x9d\x13\x63\x0a\xae\xb9\xde\x5a\x67\xe4\x44\xda\xa4\x68\x36\x5d\xf8\xab\xa1\x69\xe2\xd2\x55\x76\xed\x5b\x29\xae\x20\xe9\x2d\xb7\x6d\x48\xe3\xee\xc9\x11\x94\x61\x41\xeb\x12\xae\xea\x02\xdd\xf8\x4d\xd1\x99\xe9\x1d\xcd\x21\xcb\x08\x0c\xd3\x92\xe5\x74\x03\xcf\x91\x21\x37\x2c\xb6\xed\x0c\x21\xe2\x10\x1c\xa7\x94\x5f\x95\x37\x12\x61\x45\x5b\x29\x38\xad\xa5\x5c\xdc\x33\x68\xec\xd1\xd6\x92\x57\x8a\x43\x4a\x91\xf7\x6a\x59\x1d\xc2\xa2\x2f\x7d\xe2\x6c\xeb\xd8\xcc\xd9\x99\x78\x55\x9d\xbe\x28\xcc\x00\x05\xc6\xaf\xc5\x7d\xb2\xa8\x62\x1a\xbf\x0d\x53\xfa\x27\xda\xd5\xa6\x80\x52\xbd\x96\x13\xd9\x99\xbc\x1b\x4f\x12\xaf\x20\x25\x49\xcb\x6f\x45\x67\x4a\x7a\xcb\x93\x13\xc8\xe3\x3f\x90\xc9\x75\x74\xb6\x89\x45\x76\x12\x31\xb9\xd0\x11\xa4\x92\xe3\x70\x26\xde\x8c\xfc\x37\x50\x6f\x68\x71\x9e\xd0\xbc\xaf\x17\xa9\xb5\xf8\x46\xf0\xf1\xa0\xe2\x6c\x37\xa4\x56\x69\xad\x8c\xae\x30\xee\xbc\xa6\xc8\x96\xaf\xa1\x2e\x65\x16\x87\x3b\xd5\xa9\xc9\x41\xbd\x58\x6c\x30\xde\xc5\xa2\x47\x05\xd3\x51\x9b\x82\x84\xd0\x46\x20\x5e\xce\xfc\x6c\x74\x82\x41\x8a\x62\x35\xcf\xaf\x2e\xde\x39\x1c\xf9\x20\x59\xec\x33\xb6\x1e\x4b\x64\x44\xa3\x2b\x0b\xc9\x89\xce\xc4\xe2\x89\x16\x21\x86\x46\x18\x34\x1d\x99\x76\xf0\x4c\xcb\x11\xe1\x62\x7e\x82\xb7\x88\x8c\x7c\xa1\xdb\x1a\xf3\x81\x0c\x45\xda\x34\xf3\x5d\xd3\x4e\x23\xca\x6f\x9a\x5f\x4a\x48\xd6\x94\xbf\xca\xbd\x62\x71\x2d\x2b\xc3\x39\xf9\x86\x9d\x6e\xa3\xce\xa3\x61\x25\x66\x8e\xd8\x0c\xdc\x97\xc8\x24\x30\xb4\x43\x55\x5f\x9a\x71\x1d\x78\x06\xb7\x3b\x1b\x65\x6f\x68\xb6\x21\x12\x22\x61\xbb\xe6\x21\xa1\xfb\x67\x90\x18\xec\x57\x2f\x86\xb3\xd3\xa1\xd9\x6a\xc8\xe3\x4d\x5f\xb1\xd3\xfb\x24\x6a\xea\xc5\x44\xef\x77\xc2\x51\xd1\xb8\xc6\xad\xb9\x3f\xcb\xda\xdb\x36\x4b\x2e\xab\x4c\x05\xe1\x3b\x37\x4d\xcd\xa0\x14\x3a\x1e\xcd\x92\x45\xae\xde\xc1\x93\x18\xff\x71\x54\x0d\x95\x9e\x36\x71\x6a\x30\x78\x36\x19\xa2\xe9\x61\xa3\x25\x51\xd1\xb6\x69\xe8\x0e\x4e\xce\x58\x66\xea\x98\x7f\x17\x86\xd9\x08\x6e\x36\x8a\x85\x5d\xe2\x90\xf4\x2b\x4a\x31\x1e\x01\x7c\x09\x6b\xa7\x1a\x57\x34\xb2\x04\xd3\xb3\x35\x10\x7f\x36\xdb\x7d\xa8\x76\x78\xe7\x09\xbc\x83\x7a\x26\x8c\x10\xa7\x1b\x09\xdd\xcd\xeb\xe2\xdc\x46\x43\xb9\xb1\x73\xd5\x23\x2f\x14\xd2\xbe\x5d\x41\x5e\x1c\x76\x68\x9a\xe0\x0c\x0f\x4f\x47\x78\xbd\x76\x57\xc2\x8d\x99\xeb\x03\x76\x1a\x21\x34\x73\x1b\x90\xa8\x2c\x48\x49\x6a\xcb\x4a\x58\xc6\x8e\xc5\x14\x8d\x13\x17\x43\xc3\x4d\x74\xc2\x9d\xc6\x52\xd3\x01\x55\x10\x0f\xeb\x40\xd0\x72\x92\x91\xb0\x39\x1a\x29\xce\x0f\xf7\x0a\x23\x8a\x40\xcb\xac\xae\x70\x3d\x78\x52\x6b\xd3\xc5\x05\xbf\xb4\xad\x7c\xf8\xaf\xa6\xfc\xc2\xd4\xb8\x12\x45\xbd\xa4\x67\x4a\xf6\x32\x4b\xf6\xfc\xb5\x3e\x7b\xce\x6b\x6d\x93\x27\x3a\x3f\x04\x1a\xce\x46\x78\x56\x64\x95\x2f\x74\x6b\x34\x1e\x0f\xc7\x46\x05\x53\xba\x0c\xdf\x2d\x25\x87\xe6\x94\x48\x67\xc4\xaa\x62\xce\x97\x6c\x1a\xad\x73\xa0\x77\x3f\xc5\xd0\x90\xa0\x54\xa3\x45\x30\xae\xc7\xb5\xac\x5b\x5d\x34\x37\x0b\xce\x26\x68\xfb\xc5\x4b\x7a\x96\x6b\x69\x5a\xdb\xa5\x00\xaa\xe4\x17\x24\xa5\x1d\xd5\xe0\x22\xf6\x76\xa9\xb5\x44\x74\x15\xba\xad\xe5\x24\x56\x36\xcd\x57\xbd\x23\x17\xb1\x19\xf7\xb0\xb5\x38\xe3\x32\x17\x12\x7f\x49\xea\xb8\xcf\xfd\x05\x36\xa9\x17\xe8\x3f\x08\x9f\x78\x8a\xb2\x86\xcd\xfd\xc2\x3a\x45\x8e\x76\xeb\x35\x17\x1c\xdb\xa6\xef\xd7\x9d\xdb\xf7\xee\x84\x8c\x78\x78\xe8\xf0\xe2\x0b\x6f\xcb\x0f\x3e\xfe\x29\xb8\x70\x77\x0e\x88\xb6\x5c\xa1\x07\x97\xee\x6a\xdb\x1d\x7d\x1c\x3b\x83\x5a\x4e\x5c\x3c\xda\x90\x5f\xc6\x42\xb0\xd5\x6c\x7f\xfc\xcf\x66\xe5\xe3\x7f\xc6\x4d\xf8\xf4\x90\xfd\x19\xb0\x3f\x48\x9c\x2f\xfb\xc1\xc3\xda\x9d\xf5\x93\xec\x4b\xdd\xd4\x45\x36\x8f\x9a\x3d\x10\xf1\x33\x62\x7b\xfc\x9c\xbe\x1c\x85\x56\x78\xb1\x1c\x39\xbb\xb1\x9c\x76\x7d\xf6\x57\x54\x35\x1f\xb2\x0b\x90\x9b\xf8\xf8\x17\xec\x53\xe4\x51\x41\xa8\x54\xa9\x37\xeb\x64\x07\x40\xb7\xa1\x6e\x55\xba\x90\x8d\xf4\x5c\x3c\x20\x18\xb7\xb7\x66\x6e\x26\x09\xa5\x00\xc3\xc1\xbf\xc0\x54\xb7\xb7\xf8\x17\x44\x00\x1a\x01\xca\x44\xf3\x47\x47\x37\x59\x6f\x56\xea\x14\x72\x6f\x17\xbd\x89\xf5\x80\x2b\xce\x5b\x62\x3d\xe1\x1a\x4c\xf8\xa6\x14\x32\x81\x2e\xf9\x88\x55\xe4\x17\x48\x9f\x03\xcb\x0b\xb7\xb7\xde\x67\xb1\x48\xfe\x2f\x02\xce\x8b\x7f\xae\x17\x05\xbb\x4c\x19\xf9\xd4\x41\xd8\xd7\x22\xd9\x9c\x08\x95\x60\x9a\xbe\xf8\xd2\x0a\xd1\xa3\x17\x57\x92\x70\xc7\xba\x93\xb1\x7c\xf3\x6e\xa4\x9e\xb1\x00\xd5\x01\x7a\x5d\x4e\x77\xe1\x0b\xa8\x53\x3f\xa8\xa1\x17\x85\x0a\xcc\x31\x9c\x87\x38\xe7\x20\x11\x27\x63\xdc\x92\x09\x08\x79\x78\xed\xdc\x8e\x52\x3b\x56\x46\x16\x05\x46\x19\xe1\xee\x35\x00\x98\xf6\xb1\xc0\x7f\xe3\xd0\x8a\x69\x2e\xb5\xb6\xab\x10\x6e\x6f\xd2\x00\xea\x01\x00\x8d\x8e\xa4\x73\xf7\x6b\x71\x13\xdb\xda\x4d\x12\xac\x51\x2c\x12\x9f\x7c\xfc\x13\xb2\x16\xf5\xb1\x56\xc1\x6c\xbe\x62\xa7\x4f\x2c\x57\x2b\x7d\xf8\xaf\x58\x69\x22\x60\x45\x53\xa0\x4e\x05\xea\x54\x00\x7b\x9d\x33\xbc\xe6\x18\x44\x93\x47\x69\xaa\xff\x0b\x29\xa1\x7b\x9c\x26\x09\x4e\x00\x2a\x94\x10\x0a\xc4\x53\x58\xcf\xaa\x1c\x79\xcc\xc4\x78\x60\x79\x61\xc8\x0d\x37\x93\xf1\xa8\xab\x51\xa1\x9c\x1c\x05\xe1\x7e\xd4\xb9\x31\xb7\x3e\x62\xc2\x54\xc7\x79\x0b\xa5\x8e\x91\x4f\x9d\x0b\x9d\x5a\x05\x47\xa5\xe5\xd8\x4b\xd8\x72\xf1\x01\xb0\xa9\x59\x01\xa2\xc8\x5b\xe7\x4a\x6c\xc8\x50\x77\x17\x1f\xd8\xfa\xb9\x97\x45\xf5\xb8\xe2\x54\xa9\xeb\x2a\xc1\xda\xe8\x55\x81\xcf\xbd\xc8\x5d\x59\xea\x25\x07\x9f\xb8\xe9\x4c\x8e\xfb\x51\x47\xd4\x6e\x8c\xcf\xae\x16\x53\x8c\x87\x24\x3e\x07\x7d\x99\xc1\x8b\x20\xdb\x68\x63\xd9\x18\x56\x55\x53\xb6\xc4\x93\x2f\x64\x07\x9c\xe4\xb8\xe0\x06\xc9\xb5\xac\xc4\xec\xa2\x90\x9a\xe6\x32\x48\xac\x64\x82\xcd\x4a\xad\xe4\x94\xae\x64\x82\x16\x8c\x4d\x08\x62\x1c\x6e\xec\x2d\x5a\xc8\x2c\xa6\x68\x22\x03\x23\x2e\xf2\x2c\x91\x8c\x27\x94\x21\x42\xad\xe5\x91\x65\xbe\xd7\x48\xb5\x30\xbf\x37\x49\xa8\x24\x1f\x1d\x07\x39\xd4\x22\x8c\x56\xa6\xb2\x8a\xa3\x8c\x67\xb5\xe3\x61\x84\xf7\x97\xf8\xd6\xce\xfa\x26\x31\xa7\x84\x7c\x48\xe6\x78\x1e\x90\x4b\xcd\x92\xab\x2a\x1d\xf4\x67\xae\x33\xeb\xf0\x48\xc2\x1e\x11\xc2\x1c\xaa\xa9\x2b\xfc\xff\x1a\xcf\xda\xad\x8b\x08\x94\x4a\xae\x6a\x78\xbc\x9a\xbf\x72\xf8\x47\x90\x0d\x2e\x50\x4f\x64\x0b\xa3\xae\x12\xbf\x2b\xaa\x9f\x50\x68\xb5\x37\xd4\xe3\xf6\x55\xbb\xd6\xc4\x80\x6d\x92\xdb\xbc\x6a\xcb\x57\x39\xaa\x1d\x7c\xf8\x9b\x0c\x56\x4c\x79\xf4\x5c\x7f\x97\xbb\x01\x92\x57\x90\xbb\x72\x4f\x4c\x5d\xa1\x78\x05\x12\xfe\x0a\x0b\x32\x62\x89\xd1\x36\x44\x8f\x12\x4c\x63\x95\x77\xa5\xe8\x94\x6a\xc4\xe4\xb6\x2a\xaf\x54\x5d\x97\x8a\x75\x73\x75\xb2\x92\xfb\xf0\x5f\xd2\x08\x19\xca\xc1\x97\xa5\x86\x29\x65\x4b\xa0\xae\xc9\x48\xc7\x22\xb7\xd5\x55\x3b\xfb\xa4\x20\xbb\x4b\x82\x46\x7a\xb6\x70\x71\xd3\x18\x79\x05\xd0\xde\xa6\xb6\x10\xe6\x1e\x50\x32\x4f\x24\x5c\xed\xa9\x61\xa6\xe8\x64\xba\xf8\xfd\xf0\x62\xc4\x27\x10\xc5\xa2\x1e\x47\x87\x1b\x7e\xbe\x12\xf7\x7e\xad\x7e\xb1\x5f\xbb\xbd\xdd\x8d\x16\x79\x5d\xc4\xab\x47\x7c\x20\xa9\x09\xe5\xf2\x54\x36\xc9\xc2\x99\x29\xe7\xb9\x0b\x7d\x98\x9b\xca\xaf\x4b\x7e\xbd\x93\xd7\xd1\x66\xa8\xf8\xdc\xa3\xd2\xbd\x16\xf0\x10\x39\xd8\x63\xef\x5e\x81\x36\x2c\x24\xbb\x9b\x5f\xeb\xe8\xb8\x30\x63\xf4\x02\x86\xf3\x46\xaa\xae\x70\x36\x81\x7c\x48\xfa\x99\x4a\x00\xce\xa5\x8c\xfe\x08\x08\x3e\x5e\xca\x53\x04\x28\xfd\x16\xa7\x6c\xb1\x08\xec\xfd\xe4\x49\xdd\xd5\x09\xec\x18\xe3\x03\x83\xc5\x7b\x79\x62\x70\xd9\x4d\x6d\x50\xf0\x6a\xf9\xb2\x97\x4e\xee\xc5\x6a\x02\x32\xa7\xae\x6a\xb8\x0b\x9f\x34\x72\x76\x36\x97\x5d\xa8\x0d\x45\x1b\x19\x25\xd0\x1e\x77\xd2\x80\x3c\x04\x05\xf9\x13\xe2\xf4\xc1\x10\x50\xbb\x18\x4d\xba\x68\x97\x02\xd5\x2a\x98\x57\xc2\x7f\x48\x18\x39\xd9\x72\x59\xbb\x0b\xe8\x17\x8b\x5d\x3e\x98\xac\x27\xfd\x59\x09\xab\xed\x39\x05\x55\xbf\x91\x54\x30\xbd\x1e\xa4\x4f\xea\xab\x5c\xcc\x27\x75\xc2\xec\x69\x2a\x9b\x8d\x6a\x21\xbb\x0f\x13\xf6\xab\xbc\xec\x06\x34\x9a\xde\xc4\xa1\x4d\xef\x53\xf5\x15\xef\xda\x96\x09\xb5\x4f\xce\x27\xd3\x94\x79\xaa\x96\xb1\x5a\x2e\x33\xf3\x96\x98\x07\x72\xe4\x7d\xe9\x7d\x65\xb9\x2c\xa5\xf6\xad\x5c\xff\x29\xfa\x2c\x23\xa2\x81\x64\x47\x51\xc9\x53\xdd\xb1\xc2\x82\x44\xf5\x6b\x4e\x0f\x96\x2a\xaa\xbc\xaf\x2c\xe2\xbd\xe5\x32\x76\x36\xf9\x5e\x57\x7e\x85\x36\x2c\x00\x6b\xb4\xfb\x6b\xac\x70\x54\x74\x1c\x82\x5f\xa9\xbb\xd7\xa4\xc1\x76\xf9\x78\xa4\xce\xf6\xcc\xa8\xd0\xe1\x09\x14\xea\x9d\x64\xd6\x8c\x1b\xc5\x03\xda\x24\x44\xd7\x09\x2a\xa6\x5a\x6f\xc6\x2a\xd7\x4c\xb7\x0f\xbf\xd1\x29\x05\xed\x8c\xf8\xa8\x80\xf7\x4a\xbf\x99\xfd\x91\x72\xce\x9c\x20\x9b\x36\x8e\x98\x2d\xcb\x19\x24\x15\x8b\xd0\x34\xec\x79\xb9\x21\xd4\x74\x45\x31\x27\x4c\xae\x63\x03\xa1\x4d\xc5\xdb\xa2\x3a\x1e\xd4\x7e\xfe\x15\xb0\x7c\x42\xda\xcf\xaf\xa8\xfa\xc8\xc6\x05\x0b\x34\x51\xb7\x84\x7f\x6a\x4f\x12\x2c\x54\x2c\x52\x51\x04\xda\xc4\xc6\x0f\x3e\xfe\xe9\x67\xd4\x52\x31\xb7\x46\xb9\xa0\x20\x25\xa4\x9a\xd2\xc0\xfc\x5a\xdf\x36\x12\x65\x19\x09\xfb\xc6\x2a\x3d\x1a\x75\x1c\x8e\xc3\x5f\x55\x7a\x3c\xea\x71\xc7\x9d\xfe\xce\xfa\xcf\x07\x5d\x9e\x86\xc0\xaa\x51\xf3\xe3\x9f\x2a\xfc\x8b\xe2\xbb\x26\x7a\xf3\x5b\x99\xa1\x96\x80\x49\xb7\x09\x74\xec\x6f\x07\xe8\xac\x97\x3f\x81\x9b\x20\xf7\xe3\x5f\x24\xf7\x21\xd0\xc2\xc9\xfd\x22\x26\x40\x74\xbd\xd3\x9f\x59\x3d\x5e\x34\x31\x77\x41\x81\x49\x7c\x50\x53\xde\xf2\x45\x43\xdc\x9f\xa1\x92\xe9\x9e\x89\x1b\x92\x91\x56\xe7\x6a\xed\x82\xfb\xe3\x92\x6b\xac\x16\x50\x0f\xaf\xa7\xaf\xb6\xdd\x4c\xbb\x64\x39\x04\x2d\x9a\x81\x76\x9a\x87\xa8\x0d\xf3\x07\xfc\x4a\x6b\x9a\x17\xc7\x46\x25\x64\x5b\xb6\x55\x5d\xad\xeb\xe9\xeb\x6d\x14\xea\x32\x04\xaa\x4b\xa7\xc9\x55\x10\xc8\xfc\xa3\xa7\xba\x2b\x3c\xbe\x55\xdd\x35\xfc\x59\xab\xde\x0a\x8f\x68\x55\x6f\x0d\x7f\xd6\x0a\x16\x00\x7a\xe6\x57\x57\x03\x48\x5f\x55\xba\x2b\x35\x80\x9c\x75\xa5\xbb\xa6\x4b\xcb\x64\x55\xa3\xb4\x15\x2e\xea\x40\xcd\x98\x6e\x27\x67\x7b\xc9\x60\xa5\x6a\x07\x83\x15\x7d\x2e\x6a\x07\x33\x6d\xf2\xb2\xa8\xd3\x58\x01\xc8\x99\x36\xca\x80\x49\xbf\x82\x9a\x54\xf1\x00\x56\x38\x29\x09\xbf\xc9\xed\x73\x0d\x4b\x72\x5b\x75\xdb\x16\x82\xcc\x6a\x2b\x17\x42\xaa\xad\x5c\xac\xa8\xad\x35\xf4\x6b\x0d\x7d\x4d\xf5\x6b\x8d\x6d\xad\xef\xd9\xaf\xf5\xbd\xfa\xb5\xae\xdb\xb6\xbc\x7e\x39\x6d\xdd\xdd\xaf\xf5\x5d\xfd\xd2\x61\x14\x6a\xe4\x4e\x9b\xb9\x20\xb9\x59\x35\xbb\xab\xf2\xa2\x56\x02\x22\xae\x9b\xdd\x35\xfd\x5c\x6f\x62\x55\x87\x06\x75\xb1\x9e\x2e\x56\xb7\xc5\xea\x5c\x0c\xf3\x37\xf4\x7f\x59\xbb\xa0\xe7\x18\x60\x78\xe1\x33\xa7\x23\x1e\x66\x78\xf1\x72\x69\x77\xbc\xde\xa3\x88\x4b\x94\x7b\x2b\x16\x7b\xa6\xc4\x41\x0d\xf6\x15\xfb\x4d\x9b\x5f\x4f\xe5\x37\x9a\x75\xc9\xaf\x77\x2a\xeb\x74\xfd\x7a\xb3\xd6\x74\xbf\xf7\x9b\x0d\x57\x64\x91\x0d\x9b\xf7\x42\x43\xd2\x5e\x93\xcd\x4a\x84\x53\x05\xef\x30\x52\x75\x74\xbe\xf6\x00\xd5\xef\x26\xd2\xc3\xae\x82\x25\xbc\xdf\xd3\x9f\x3d\x65\x4f\x33\xfa\x5d\xd8\x17\xf4\x7b\xed\x7e\xb7\xd2\xef\x35\x6b\xf4\x6a\xbb\x6d\xde\xee\x34\xf9\x19\x77\xdb\xbc\x0f\x69\x36\x74\x82\xbc\x0e\x69\x9a\xb7\x3f\x29\x07\xca\x19\xc6\x2c\x23\xba\xa5\x34\x97\x28\xd6\x5c\xe8\xfd\x24\xe3\x96\x42\xad\x13\xa4\xa1\x72\xad\x2b\x75\x85\x1e\x5e\x17\xc0\xae\xc7\xbb\xb6\xb8\xd2\x15\xbb\xc9\x15\xda\x20\x60\xa4\x87\x72\xf9\x97\xb8\x07\x9f\xbf\xf0\xab\xa2\x27\xc9\x9a\xba\x77\xb0\x2e\x16\x47\x73\x74\x24\x4b\xaa\xe1\x65\x8c\x3c\x59\x2e\xbf\x9f\x34\x7b\x5c\x2a\xc8\x7e\x02\xd9\x95\xca\xfb\x09\x59\x2a\x49\x4f\xf1\xa4\xef\xbd\x13\x72\xc1\xa9\xd0\x37\x77\x53\x86\x3e\x78\xb4\xa3\xe9\x1a\x57\xa2\xbe\x93\x6e\xe8\x1d\x07\xaa\xeb\x3d\x4f\x66\xba\x68\x68\x55\x67\xaf\x73\xf6\x7c\x06\x36\x31\x86\x03\x08\x82\x65\xf3\x18\x30\x8f\xa0\x86\xe4\xba\xa0\x71\xf3\xe3\x33\x94\xb4\x0e\x24\xf8\xd9\x61\xdb\xf8\x66\x30\x0d\x0f\x25\x90\x7d\xba\xb7\xb7\xfb\xf8\xa7\xbd\xaa\x35\x61\xc2\x75\x0f\xea\xed\x75\xbd\x09\xd3\xa0\xc5\x36\x68\x80\x53\xd4\x2d\x1b\x40\xe5\x87\xf1\xef\x1e\x22\x3a\xdd\xfa\xa7\x1c\x74\x78\x06\x4f\x20\x11\x1c\x57\x8c\xb0\x22\x14\x8b\xab\xfa\x01\x86\xee\x5d\x1f\x80\x1c\x85\x3f\xf0\xb5\x0e\x2a\x73\xad\x14\x9c\x94\x91\x14\x26\xde\x69\xd2\x84\xc9\xaf\xdd\x2b\x41\xdc\x85\x06\x36\x34\x78\x41\xab\x4e\x46\xb3\x39\x9a\x6a\xaa\xab\x63\x3c\xfc\x58\x1d\x27\xeb\x63\x3a\xe9\xcc\xb0\x7c\xd2\x17\xed\x22\x04\x5f\x33\x3a\xc7\xc7\x6a\x7d\x7c\x0c\xfb\xf9\xe3\x63\xd8\xf1\xc0\x7f\x3d\xef\x1a\xb9\xca\x57\xb0\x51\x68\x9d\x12\x94\xd2\xe6\x40\xb1\x36\xa8\x75\x6c\x28\x10\xe8\xb6\xe7\xbc\x0e\x32\x48\x42\xc7\x7f\x5c\x25\x25\x79\xbf\x7b\xfa\xa3\x32\x1e\xe8\x32\x73\xf1\x79\xc1\xfa\x1f\x03\xb1\x96\x10\x0c\x57\x49\x6a\x38\x29\x22\xae\x1e\x12\xc7\x20\x0c\xf2\xf4\x68\xc4\x40\xd2\x64\x85\x44\x4d\xd6\x48\x55\xd8\xa9\xca\x28\x81\x7c\xc9\x33\xa8\xc8\x60\x12\x7b\x64\x78\x85\x03\x13\x9a\x2f\x11\x36\x64\x7b\xca\x6f\x09\x6f\xba\x4d\x58\xd3\x56\x30\x9a\xcd\xf5\xf1\x46\xf5\xe8\x0b\x3f\x36\x9b\x96\x3d\x67\x8b\xdb\xd1\xd5\x31\x4c\xe4\xfb\x20\x31\xae\xe2\xd2\x80\xff\xae\xe3\x8c\xcc\x1e\x65\x82\x4e\x05\xbc\xe3\xec\xe1\xed\x7e\x9b\x4f\x3c\x76\x41\xb7\xff\x94\x6e\xdb\x6a\x9b\x15\xd3\x91\xc8\x98\x5c\x6d\xb1\x6f\x12\x26\x46\x6a\x2b\xa4\x00\xfe\x62\xa2\x33\x34\xd3\x8e\x0d\x4b\xa2\x7c\x06\xde\x6a\xdb\xa1\xc9\x97\x6c\x55\x52\xff\xc9\x36\x51\x1a\xd1\xc4\x47\x34\xcf\xd0\x49\xa8\x56\xcb\xb3\x6c\x32\xe0\x0c\x29\x3e\xd9\x5a\xc9\x5f\x6c\x0f\xb1\xef\x78\x32\xc3\xc9\x89\x98\xc1\x85\x66\x46\xb2\x4d\x48\x02\xd3\xa5\x56\xc4\xd5\x6e\x6f\xa5\x40\xec\xb2\x48\x60\xe0\x33\xfa\x17\xd8\x22\x49\xb3\xe6\x88\xea\x1f\x7f\x70\x92\x6d\x3d\x94\x61\xdf\x64\x4c\x1f\x61\x66\x6e\x37\x0c\x82\xe6\xa7\x73\xd1\xdd\x3c\xed\x0c\x53\xbd\x55\x2a\xe9\xf2\xe4\xe9\x45\x2b\x5c\xde\x56\xb4\xba\x69\xdd\x53\xec\xf9\x40\xb4\x9b\x5f\x91\x40\x91\x04\xfb\x2c\xc8\x87\xaa\x2b\x30\x70\x58\x00\xbb\x26\xc1\x36\x13\x2b\x01\x96\xd1\x81\xc9\xa8\x1f\x5d\xce\xa6\x68\x33\xfe\x74\xa1\xaf\x7d\x70\x0b\x86\xc7\x05\x1f\xff\xb4\xb7\x0f\x5a\x96\xb6\x88\xe1\x62\x50\xfd\xbb\xe5\x82\xc2\x77\xd9\x8a\xea\x32\xb9\x88\xf8\xb4\xc1\xaa\x9a\xb0\xca\xc8\xa5\xbb\xe3\xce\xed\x38\xdf\x85\x18\x41\x48\x8e\xf1\xa5\x35\xb4\x5c\xff\x82\x91\x38\xc6\xf7\xd6\xfc\x1d\x37\x0f\xa1\xc4\xde\xc7\x3f\x95\x24\x53\x7e\x76\x60\x52\x65\xf7\x8d\xce\x33\x9f\xce\x86\x5d\x83\x9c\x1d\x27\xd0\x2c\xdf\x77\x67\x03\xf7\xbc\xe0\xe3\x9f\x9d\x43\xa4\x67\x95\x46\x69\x52\xf2\xde\x1c\xef\xe9\x4b\x8e\xc3\x8f\x7f\x76\x6f\xe8\x4a\xc9\x24\x56\x1f\xff\x5c\xab\xe8\x64\xb9\x59\xe1\x78\x08\x99\x16\x31\x93\x24\xe2\xf4\xb2\x77\x1d\x18\xef\x35\xd4\xb3\xa4\x5e\x16\x53\x94\x08\x50\xa8\xf0\x6f\x6a\xc0\x45\xcf\x62\xa3\xbb\x92\xcd\x2c\xdc\xb1\xda\xf1\x3a\x41\x0c\xd7\xad\xf4\x73\xf3\x95\xa2\x7c\x00\x68\x9e\x97\x43\xef\xa3\x55\x69\x55\xa6\x8c\x12\x67\x03\x35\x26\xf1\x1e\xa2\x14\x43\xcf\x94\x34\x9a\xb6\x9c\x02\xc0\x4f\xef\x76\xa1\x41\xb8\x05\x55\x64\x82\x97\xf1\xd9\x25\xbd\xb4\x5c\x2b\x7c\x90\x88\xaf\xee\xd6\xf2\xce\x16\x05\xba\xbc\x16\x6c\x05\xb5\xc3\xa7\x82\x59\x2d\xa4\x1e\x06\x02\xc4\x44\xda\x80\x46\xb0\x1d\x15\x56\xd9\xfa\x34\x4f\x30\xca\x78\x23\x8b\xf5\x29\xf0\xc5\x3c\xbb\xc3\x9c\xf7\x63\xed\x60\xc5\x6e\x96\xc2\xf4\x64\x05\x33\xfa\x20\x9d\x51\xcf\x2c\x5e\xc7\xe2\xa9\xe4\x9f\x6a\xb8\xef\x49\x17\xff\x09\x5f\x71\xaa\x75\x1a\xfa\x4f\x99\xd0\x7f\xaa\x27\xeb\x0c\x13\x39\xcc\x17\xbd\x32\x7c\xbf\x40\x9d\x15\x53\x74\xd2\xc0\x0a\xdf\x15\xd4\x4a\x15\xe0\x7f\x6b\x3e\xe8\xe7\xfb\xca\xac\xd7\x08\xcc\x34\xdb\x20\x80\xd0\x26\x7f\x2f\xe1\x03\x83\xac\xa7\x0d\x69\x38\x2f\x2d\x9c\x4c\x35\x3c\x05\x3a\x55\x4a\x2f\x44\x01\xe0\x3f\x16\xf8\x55\xab\xd3\xbf\x24\x45\x2b\xb9\xa7\x7d\x58\x45\x5f\xd1\x6c\x6d\x09\xea\x01\x37\x9a\xa7\x9d\x38\x9c\x97\xdd\x71\x2a\xb6\xb9\xb7\xee\x22\x40\x02\xf3\x68\xe9\x55\x9e\x86\x92\x5d\x29\x63\x24\xe4\x6a\x97\x8f\x17\x9b\x19\x92\xff\x3e\xf4\x39\xd6\x63\xbc\x11\x3f\xae\x2e\x2a\xa0\x73\x0b\xe1\xf5\x33\x24\xeb\xc4\x56\x3f\x80\xa0\x5d\x51\xa1\x60\x56\x23\x4b\x6b\x2e\xb7\xb1\xab\xbb\x34\x94\xcf\xdf\xc1\xdd\xba\x38\x24\xbe\xa8\xa9\x42\x99\xd3\xca\x85\xae\xfd\xe9\xa4\xee\xd4\x76\xea\xaa\xbe\x83\x05\x61\x35\xb9\x77\x59\x5b\xf4\xba\x90\x89\xd6\x33\xcf\x1d\x87\x2b\x59\x5c\x97\x16\xe9\xd7\xe2\x90\xb8\x56\x29\x87\x18\x7f\x4c\xbf\xef\x75\x1b\x79\xb9\xdd\xc7\x84\x59\x64\x06\xab\x04\x85\x36\x9e\xd7\xe2\xcb\x78\xf3\x9a\x9e\xfc\xa7\xac\x4a\x83\x55\x79\xb0\x2e\x0d\xd6\xe9\xb7\xed\xf5\x72\x72\x4d\xcf\xcf\x31\x78\x4d\xd6\xdb\xf3\x6b\x7a\x7b\xbe\xce\xc8\xc5\x87\xe7\xd7\x2a\x9f\x32\x7a\xf5\x70\xdc\x34\xb4\x32\x4a\xdf\xb5\x74\x3c\xbb\xc3\x77\xc2\xd6\x96\xef\x24\xae\xe7\x41\x21\xa3\xb9\xbc\x46\xb6\x02\xbe\xcf\xf2\xfb\xdf\x3b\x6a\xd7\xce\x32\x1f\x02\x6f\x58\xe0\x59\x2e\x0f\x34\xe8\x2c\xb7\x07\xfb\xa5\x4f\x61\x88\x1c\x7d\xe2\x5f\xc9\x18\x81\xc2\x11\x6f\xb2\x0a\xff\x23\x5a\xc7\xb3\x29\xec\x7f\x56\x1c\xe4\x0d\xfe\x66\x2f\xce\x92\x59\xc5\x58\x63\xef\xa6\xb2\xb5\x97\xb4\xee\xac\x4f\xde\x73\x1c\x39\xae\x40\x7d\xfe\xcf\x3b\x56\xeb\x2c\x90\x9f\xb8\x50\x6b\x10\xd8\x7b\x01\xf1\x0f\xaf\xd1\x1a\x26\x79\xcb\x7b\x0d\xf4\x89\x9c\x65\x5a\x2e\xe6\x60\x39\xfe\xff\xe1\x1a\x2c\x9d\x3f\x0e\x97\x5a\x79\x23\x72\xd7\x02\x09\xa5\xbb\x17\x97\x68\x82\xc4\x1b\x41\xdf\xbd\x3c\x67\x71\x55\x73\x3a\x7e\xd1\x5d\x7d\x3d\xbc\x84\xcd\x9e\x2e\xf0\x66\xd8\x5f\xce\xe6\xa3\xab\x61\x53\xa7\xbc\x9a\x4e\x86\xb1\xae\xb7\x49\x41\xc4\xec\x00\xaa\xc6\x67\xa1\x83\xd7\xf1\x24\xe1\x42\xca\x9f\xbb\xab\x44\x90\x4d\xb3\xa7\xb7\x67\x4e\x37\x6c\x30\x35\xad\x67\x4d\xab\x7b\xc2\xcf\x3c\x60\x5e\xb1\x55\xb6\x6e\x30\x75\xd6\xec\x8c\x7e\xe6\x03\x5a\xee\xb6\xb3\x9f\x4d\xbf\x07\x30\x0f\x82\x60\xc7\xae\x11\xd5\x6f\x62\xa5\xd9\x97\x32\xf5\xe8\xe8\x8c\x5e\x4c\xd4\x54\x8f\xac\xfd\x00\x3f\xbc\x03\x43\xf9\x49\xf7\x34\x62\xfc\xde\xad\x25\x7d\x7a\xc0\x8d\x7f\xd1\x61\x19\xfe\x6d\x74\x94\x1e\x6b\xc1\x3b\x24\x06\xc2\xcf\x9c\xdc\x41\xff\x2f\xc3\xbe\x07\xc7\xd3\xee\xe3\x5c\xf7\xdc\x29\x00\x83\xc5\x98\x8c\x26\x23\x78\xa3\xe8\x83\xf4\x88\xe9\x52\xd7\x73\x3c\x13\xcb\x06\x52\xa4\x39\xd0\xa2\x96\x10\xbd\x6a\x09\xd1\x0c\x48\x91\x3f\xa0\x59\x4f\x85\xef\x33\x0a\xbc\x66\x91\x53\x18\x68\x0f\x1b\xc3\x96\xd2\xf4\x4e\x75\x54\xcb\x0b\x4b\xc5\xd0\x09\x0f\x83\x76\xa1\x62\xdb\x8c\x85\xc1\x40\xc3\xfd\xe7\x1d\xa6\x1a\xce\xf0\xce\x15\x3d\xfc\x83\x57\xa6\x77\x48\x48\xef\x05\x63\x06\x1c\xf3\xcb\xee\x02\x74\xa9\xf4\xcc\xdf\x32\x16\x78\xb2\x4c\x06\xe6\xdd\xba\xea\xa1\xe9\x94\x1a\xb8\x43\x60\x34\xac\xba\x56\xb1\xea\xa8\x63\x0d\x1a\x89\xa3\x5a\x91\x15\x52\xe3\xe0\x61\xe9\xc3\x6f\x8d\x62\x91\xea\x57\x2a\xfa\xa2\xaf\x5b\x2b\x23\x68\xe0\xa6\x32\xc2\x07\x76\x2a\x43\x23\xae\xcb\x29\x89\x85\x56\xee\x97\xfa\x68\xdd\xda\x48\x6c\xb8\xc9\xfe\x5e\x72\x81\x06\xf6\x0d\xb2\x32\x21\x4b\x13\xb4\x9d\xa2\x97\x16\xd0\x8f\x32\x1a\xe9\xec\x49\xf4\x17\x1b\x72\xcd\x13\x0d\x0d\x04\x09\x5c\xde\x90\xa9\xdf\xe0\xa9\x3f\x80\x84\x55\x83\x3b\xd5\x48\xd6\x0d\xea\xd5\x75\x82\xfd\x81\x64\xec\xdc\xba\xd1\x8a\x06\xd7\xa5\xc1\xf5\x1e\xf4\x0d\x7a\x76\x7b\x8b\xed\xb3\x4a\xd9\xe0\x8e\x37\x62\xc8\xab\xc0\x46\xf8\xa0\xba\x8f\x97\x90\xf3\xef\x46\x93\xaf\x47\x73\x40\xa4\x3f\x3c\xe8\xd6\x4a\xdd\x7a\xb9\x57\x2b\xf5\xea\xe5\x7e\xad\xd4\xaf\xc7\xe4\xe8\xfd\xce\x11\x69\x00\x8e\xd8\x65\xd5\x85\xce\x63\xd8\xa3\x0b\xd5\xf7\x07\x25\x90\x43\x58\xc1\xf2\x88\x86\x6c\xc1\xe0\xed\xee\x9d\x03\x2d\xcf\xbb\xa1\x9f\x09\x68\x12\x7e\x57\xac\x49\xd0\x7e\xcd\xb5\xda\x31\x13\x36\xa9\x3f\x0e\x99\xb0\x6a\xfc\x36\xdf\xeb\xe8\xd4\xac\xab\x49\x84\x28\x1c\x97\x8e\xe9\x7e\xbb\xfe\xd8\xf4\x4b\xe2\x9f\x11\xc7\x40\x11\x0a\xba\x11\xf2\xbb\xa3\x42\xbe\x66\x06\x78\x4b\xbd\xb3\xeb\xbb\x6c\xb3\x79\xf6\x87\x5a\x40\xe6\x19\xb5\xe6\xa4\xc3\x95\x1b\xc9\x6f\xed\x7c\xe0\x6a\x9b\x0a\xcf\x9d\xbd\xc0\xdb\x23\xe8\x37\x4c\xc3\x28\xd4\x33\x32\x34\x62\xa3\x15\x18\xa0\x1c\x51\x94\xb5\xbd\x1c\xd5\x26\x5b\x95\x20\x8e\xd1\x48\x79\xc2\x2f\xfd\x1a\x5a\x98\x8c\x33\x22\x79\x80\xbb\x4d\x0b\x34\x4f\xdf\x33\x2c\x66\x53\xab\xdf\xa7\x49\xd2\x4f\x10\x96\x19\x76\x35\x96\xea\x96\x11\xf2\xc6\x45\x5f\x0d\xa4\x0d\x77\xa5\xcc\x26\xce\xf0\xb8\xbc\xed\x6a\xc1\xd1\xb2\x1c\x3c\xdc\x81\x10\x0f\x1f\x92\xfd\x86\x1d\x7d\x50\x1a\x39\x6d\xb5\x2e\x85\x39\xad\xde\xf1\x26\xe1\xa1\x29\x79\x5e\xfe\xf0\xdb\x4a\x7d\xf8\x6d\x5d\x31\x05\x5d\x7f\xb1\x72\xb1\x72\x67\xd3\xfa\x06\xc6\xa0\x50\x01\xb0\xf1\xde\xb9\x8a\x5c\xd0\x90\x20\x16\xcf\xc5\xe2\xa1\x8b\xac\x9e\x22\x2e\xb2\x7a\xa6\x78\x4b\x13\x3e\xfa\xbd\xf1\x5a\x4e\x82\x7b\xa9\x99\x97\x4a\x5f\xf4\xd2\xe2\x37\xd0\x5f\x3e\xfc\xf6\xf1\x2f\xf0\xcf\x87\xbf\x02\x16\x32\xa8\x7c\xa9\x38\xc4\xbb\xb3\xb4\x6e\x48\xbd\x85\x8e\x24\xab\x0a\x17\x61\x37\xbb\xd0\xa5\x64\x5d\x96\x14\x72\xb6\x0b\xbd\xef\x8e\x47\x03\x1c\x03\xcf\xba\xc5\xa6\xde\x78\xbb\x0a\x0a\x92\x4e\x6c\x48\x25\xf0\x16\x5e\xa6\x26\xc7\x40\xb1\xa3\xce\x9b\x36\xfe\x14\xd7\x18\xca\xeb\xbf\x92\xf1\x53\x7a\xba\x4a\xe6\x27\x09\xac\x40\x81\x3f\x5c\xf9\xac\xb1\x12\xb6\x40\x07\xdd\xf5\x47\xb0\xde\x24\x0f\xbf\x80\x55\x28\x69\x3c\xc2\x85\x88\x2e\xce\xe0\x1f\x32\xe5\xad\x11\x91\xf9\xeb\xaf\xf0\x07\x45\x3a\x5e\x41\xe7\x3c\xcd\x52\x18\xdb\x46\x17\xd0\x51\xcb\x14\x97\xc2\x67\xe0\x74\x6d\x6a\x0d\x21\xdc\xa8\x51\xce\xd4\xd0\x72\xcd\xf4\x6d\xba\x5c\x5c\x2e\x17\x77\x12\x5d\xaf\x8a\xe9\xdb\x3e\x2d\x6a\x05\xfd\x28\xa0\xbc\xd9\x03\x0a\xf6\xba\x41\x72\x98\xe2\xb5\x51\x33\xdb\x4c\x07\x5d\xdb\xbf\x7b\xad\x73\x9a\x86\x6c\xf5\x74\xdc\x8e\x6c\xfd\x63\x95\x43\x58\x0a\x11\x1b\x3c\x93\x72\xea\x95\x8f\x63\x4f\x20\xb8\xac\xca\x0e\x37\xa8\x5c\x0a\x6b\x19\x86\x7b\xa1\xed\x94\x3f\xb6\xe3\x7c\xdc\x4e\x5b\x2e\x1e\xcb\x83\x8d\x63\x7e\xb0\xa1\x8e\xe5\x21\xc8\xb1\x3c\x04\x69\xba\xcc\x91\x46\x95\xa1\x78\xb8\x52\x60\x89\xfb\xa1\x79\x0e\xc4\x50\x22\x5a\xe2\xe6\xb9\x07\x86\x56\x44\x32\x4d\xbe\x17\xa8\x15\x80\x22\x85\x11\xff\xd6\x3b\x16\x2a\x99\xe2\xf8\xc4\x64\x91\x73\x2f\xb0\x30\xad\x10\xec\xef\xf6\x1f\xbb\xaa\x14\x4e\x3a\x6c\x25\x4c\xb6\x8d\x7e\xf8\x9b\x2b\x5b\xd9\x4b\xbf\x91\xaa\x2e\x32\x22\x33\xef\x87\xcc\x6f\x39\xe8\xd0\xbc\xcf\x42\x88\x24\xc1\xb1\x76\x8f\xd1\x68\x1f\xc3\x3e\x39\x28\xd3\xac\xb9\x68\xff\xe6\x23\x8e\x80\xfd\xef\x0f\x7f\xf5\xbb\x62\x43\x01\xd9\x4e\x85\x73\x15\xc3\xd7\x89\x76\x59\x88\x55\xee\x52\x9d\x98\x45\x58\xe2\x65\xa1\xde\x64\x23\x9d\xb9\xa2\x59\xdf\x3c\x4b\x8a\x31\x63\xe0\xbf\xba\x43\xdb\x54\x0b\x2d\x65\xfe\xfe\x93\x1d\x5f\xab\x77\xa9\xbe\x76\xe7\xf7\x26\xad\x7e\x0c\x7f\x5d\x92\x79\x62\x77\x72\x4a\xde\xee\xbc\xa7\x92\x72\x60\x92\xaa\xa4\x27\xe1\x1b\xb3\xac\x06\xb5\x0e\x3e\xfe\xa9\x8d\x2f\x37\xd0\x9d\xc3\x9f\xf0\x6d\xc1\x87\xbf\x95\xe1\x23\x07\x5c\xde\xea\x7c\x63\x96\x5e\x84\x00\xc9\xb7\xb7\x98\xde\x0e\x96\xfa\x00\xca\x87\xbf\x11\x1c\x73\x3e\xae\xd3\x3f\xfe\xe5\xc3\x5f\x23\x0b\xdc\x48\xc6\xb0\x5e\x33\xdd\x52\x0e\x88\x66\x0e\x39\x52\x1d\x14\x73\x81\x37\x41\x4b\xa9\xa7\x03\xfe\x3b\x55\xa0\x59\x42\x14\xb9\x93\xa0\xb9\x14\x35\x2d\xd1\xe6\x45\x12\x93\xed\x58\x99\xed\x98\xa4\x6b\xfe\xce\xad\x55\x61\x72\xcf\xf2\x7a\x9f\x43\x3d\xe7\xc0\x34\x60\xa3\x4c\x0f\xc5\xe8\xcb\xda\x7b\xb9\xcb\x3e\x26\xd6\xee\xe3\x75\x9d\x78\x1d\x86\x13\xb9\xc6\x0c\x6c\xba\xbc\xa2\xf7\xe3\xf0\x2b\xc3\x16\x64\xcd\xa5\x3e\xfc\xb5\x72\xce\xa5\x3e\xfc\x55\xad\x74\xcd\xca\xb5\xae\x69\x03\x11\x9c\xeb\x0a\xe5\xb5\xae\x20\x76\x2f\x52\xc9\xc1\xf8\x37\x89\x75\x62\x93\x09\xe7\xdf\xc4\x6b\x0b\x49\x47\xa7\x34\x2a\xa7\x02\xd2\x2d\x0d\xc9\x79\xe3\x93\xf5\x9a\xf8\x5f\x42\x45\xa0\x8f\xe9\xee\x36\x2a\x96\xaf\xd3\x54\x2c\x9f\xe7\x52\x11\x48\x6c\x06\x07\x6d\x6c\x72\xf9\x29\xf7\x5d\x90\x91\xec\xd2\xf9\xac\x07\x49\x33\xdb\x29\x49\x4b\xed\xbf\xef\xb2\x40\xc7\xd0\x64\xce\xab\x77\xd3\x6a\x4b\xa2\x78\x43\xf5\x36\x01\xf1\x1f\x32\x91\x26\x15\xf5\x67\x8a\xfc\x09\x40\x03\x39\xd9\x8b\x29\x3e\xa6\x76\xde\x4c\x2c\xa6\x07\x58\xa5\x89\xff\x1c\x40\x2e\x28\xb0\xf8\xb3\xec\xe0\xf0\xf1\x3f\xf1\x1d\x2d\xb5\x29\xb7\xe7\x6e\x1e\xb6\x25\x0f\x9b\xaa\x8f\x4a\x14\x59\xcd\x77\xc5\xab\x16\x84\x75\xcb\x6b\x15\x9a\x6a\x42\xd3\x8b\xd6\xa2\x92\x50\x9d\x6c\x1f\x85\xa9\x87\x69\x87\xd0\x87\xca\x7c\x66\x8d\xc1\x16\xb1\xfd\x46\xba\x2f\xe2\x4e\x1c\x38\x8c\xcd\x19\x63\x43\x13\xd9\x78\xf2\xb1\x60\xea\x5c\x9f\x73\x5b\xf4\x70\x23\x81\xd2\x61\x01\xc7\x81\x0f\x6f\xf4\xba\xa4\xfc\x0e\xc4\x0b\x3d\xbf\x46\x68\x69\xff\x5a\x14\xf0\xe5\x49\xad\x5d\xa1\x52\x4d\xfa\x37\x2e\x37\x18\xff\xd7\x2f\x2a\x1f\xfe\x2b\xfe\x5d\x64\x3e\xd3\x2b\xea\xe9\x0c\xd8\xb6\xbf\x1c\x0f\x7f\x8c\xd6\x68\x86\xaa\x06\x72\xdb\x8c\x81\xf9\xaa\x33\x0c\x10\xcb\x19\x00\x09\xf3\xb4\x3f\xb0\x75\x3d\xcd\x8a\x36\xc6\xd8\xba\x7a\xd1\xbd\xb4\xbb\x36\xb3\x67\x63\x95\x32\x83\x82\x06\x8d\x9f\xf0\xa8\x70\x85\x6f\xca\x74\xcc\x31\x83\x06\x65\x10\x1a\x2b\x83\xc6\x2a\x03\x0d\x67\x87\xe8\xa3\xb1\xda\x8a\xc6\x7c\xba\x9c\xf5\x87\x6e\xc8\xb6\x2a\x27\xf9\xee\xd6\x41\xab\xc2\x78\x8c\x4e\x29\x4e\x4a\xab\x1d\xce\x94\xf7\x5e\x39\xd1\x6c\x5f\x3b\x7e\x44\xd6\x35\x98\xea\x6b\xc7\xee\x10\x13\xfa\x6b\xc7\x39\x09\x1a\x45\xcf\xd7\x8e\xeb\x13\x4c\x38\x47\xc7\xf7\xeb\x9a\x65\xdd\x15\x54\x3b\x5f\x3b\x89\x58\x92\x12\x57\x75\x48\xac\x3b\x25\xb1\xfa\xda\x49\xa4\x92\x75\x0c\x30\x2f\xbc\x42\xb2\xce\xb1\x2f\xd8\x3f\x3e\xeb\x82\xe0\xe6\xb6\xf1\xed\x54\x19\x5b\xc1\xfa\x6e\x16\x9d\x9e\xe3\x7e\x12\xb6\xdc\x7b\xd6\xe1\x56\xec\x3f\x98\xb4\x71\xf3\x64\x96\x7c\x65\x3b\xb6\x28\x25\x83\xb8\x74\xae\x9e\xda\xa4\x41\x65\x81\x29\xab\xe4\x69\x09\xba\x5c\xfe\x0a\xfe\x05\x42\xe2\xd7\x9a\xbe\x80\x69\xaf\xe1\x6b\x4e\x5f\x40\xa5\x2c\x29\xaf\x56\xb1\xab\x8d\x3b\x59\xd7\xca\xf6\x12\xed\x28\xd7\xa5\x75\x1c\x7b\x9a\x7a\xfa\x6d\xfb\xe1\xaa\xe6\x9d\x91\xd6\xbc\xf2\x9a\x21\x9d\x3e\x63\x70\x6a\x3e\x61\x1e\xb8\xa4\x48\xf1\x0c\x6f\x37\xee\x30\xe3\x71\x56\xcc\x94\x4f\x35\x3f\x9e\x46\xc6\x82\xba\xa0\x0b\x86\xc8\x8f\xb2\x10\xd3\x23\x62\x5a\xd8\xff\x6f\xd8\x28\x39\xb2\xb0\xe5\x21\xf6\x76\x09\x72\x3c\x3b\x6a\xcc\x22\x8c\xb6\xb0\x00\x2a\x2e\xca\x3a\x5d\x7c\xb5\x89\x85\x2c\x2f\xd9\x25\x69\x0f\xca\xe9\xa0\x60\x5c\xaa\xec\xba\xd0\x31\xa5\xd4\x5d\x1e\xbc\xf4\x15\x99\x94\x90\x3c\xbf\x07\xf9\x41\x2a\x12\xde\xbf\xe9\x9d\x48\xda\x14\x38\x83\xb4\xb5\xc0\x77\x5d\x46\x4b\x66\xd8\x36\x41\x76\x86\xd5\x48\x56\xfd\xec\x4a\xfa\xda\x3f\x1d\x11\xef\x7a\x74\xb1\x5c\x9c\xc1\xc2\x46\x87\x09\x8a\xd7\x03\xcb\x45\x36\xdf\xd7\xbb\xdc\x51\x37\xee\xf1\xfc\xde\x9d\x27\x04\x32\xa2\xd2\x3c\x36\x26\x4a\xda\x79\x29\x23\xd6\xc6\x79\xc9\x0f\x8b\xa6\x77\xe7\x1a\x85\x6d\x46\xd0\x49\x7a\x52\xaa\x7e\x42\xbd\x89\x3e\xfe\x99\xd8\xa5\x6f\x07\xa7\x4f\x08\xf7\x2d\xbe\xfd\x38\xcb\x78\x1a\xb9\xab\xaf\x3e\xfe\x19\x91\xed\x4b\x90\x2a\x92\x73\x1f\xff\x5c\x2c\x92\x92\xd8\xdf\x03\xe0\xa0\xcd\x19\x1c\xb3\x8d\xd5\x9f\x4d\x27\x14\x62\x73\x7c\x6f\x63\x75\x7d\xab\x07\x49\xda\x19\x1a\xde\xed\xd5\x9a\xdf\xb0\x98\xc4\x68\xa3\x0b\x22\xb3\x89\x62\xea\x59\xae\x4f\x4a\x48\xd1\x6f\x3c\xb3\x75\x4a\x33\x0a\x7c\xda\x6b\xa3\x43\xd9\xd0\x9f\x10\x3a\xc4\x42\xa7\x8c\xe5\x8f\x7f\xd9\xc3\x1b\xa6\x09\xf9\xf5\x42\x87\x7b\xf5\xb6\x67\xfd\xce\x77\x51\x18\x70\x94\x9b\xd8\xf3\xfd\x65\xed\x99\xec\x05\x25\xec\x2d\xd8\x29\x96\xfa\x46\x4f\x7b\xaf\x77\x68\x4d\xef\x18\xc3\x4f\x40\xf1\xbc\x9f\x3d\xfc\x37\x95\x35\x8c\x1b\xb2\xfd\xfc\x14\x88\xa1\x9d\x6e\x69\xf6\x70\x6d\xdf\xb7\x5b\xcd\xeb\xd5\x0d\x09\x60\x50\xfb\x06\x46\x5d\xd5\xf7\x26\xb1\x1e\x1b\x6b\x33\x6f\x76\xcd\x17\xc3\x59\x1f\xef\x3b\x72\x5f\x2f\x8c\x58\xc4\x2f\xee\xcd\x12\xdf\x56\xec\xb6\x2e\x3d\xd4\xdf\x7e\xd2\x50\xe7\x8f\x9f\x8c\x5b\x25\x18\x37\xfc\x53\xa1\xaa\xdf\xca\x48\xed\x4d\xca\x28\x2d\x9d\x57\xd6\x13\x7e\x1f\x9d\x79\xf2\xd2\xbc\xef\xc8\x7d\x7b\xd7\x3b\x86\x6f\x2b\xf7\x19\x55\x67\x44\x52\x23\xa0\x87\x26\x7d\x1a\x64\xb8\x33\x83\xe9\x33\x42\xf1\x69\x40\xaf\x33\xae\xd1\xf4\x33\xae\xa7\x4b\xd8\x6b\x65\x9c\xbd\x9b\x92\x8a\x4f\x71\x2f\xf8\x34\x57\xd9\xc3\xd8\x0b\x7b\x30\xeb\x5e\x0e\x5c\x38\x47\xd4\x9a\xfa\x17\xe1\x51\x30\xb7\x7f\xc5\x12\x58\xce\xf8\x2e\x9c\x03\x3e\x5d\xf1\x0a\x3d\xb4\xb6\x35\x9e\x6d\x17\x36\xbd\xe8\x8a\x9b\x17\xcd\xab\x8d\xba\xc8\x3a\x23\xd6\x6d\x98\xac\x7f\x46\x3b\x39\xc7\xef\xdc\x92\xcd\xcc\x6a\x0a\x9f\x6b\x62\x33\x64\x44\xae\x9b\xd2\xd7\x09\x3a\x54\xf3\xc7\x3f\x95\x78\x55\x42\x71\x67\x30\x8f\xe2\x96\x83\xd0\xe1\x21\xdd\xed\x9d\x93\x8b\xa6\xca\x79\x47\xd1\x77\x99\xbf\xcb\xe7\x1d\xdc\xd1\xd1\x03\x51\xdd\x08\x3a\x1a\xe3\x3b\x2d\xd3\x5f\xbf\x2f\xdc\xc7\xec\xab\x68\xd4\x84\x87\xdf\x05\x2c\xe9\x28\xb9\xb2\x9c\x7a\x4b\xa5\x37\x43\x90\x5f\xa3\x7a\xf9\x2b\x98\xa4\xf5\xca\x57\xf4\xce\x28\x54\xb9\x88\x93\xff\x22\x2b\x98\x2c\xd9\x69\x7e\xbe\x60\xd7\xc8\x3f\xba\x9b\x15\xf4\x06\x90\x5d\xec\x27\xaf\x58\x3d\xa3\xd8\x19\x74\xfa\xd9\xb3\xff\x88\x46\x75\x35\x6a\xa8\xd1\xbe\xd2\xae\xbb\xc4\xf2\x62\xa0\x86\xea\x44\x53\x6c\x91\x5c\x1d\x8e\x30\x48\x68\x42\xfe\x80\x7b\x09\x79\x59\xa6\xd4\x46\x07\x96\x72\x4a\x1d\x38\xa9\xfb\x1d\x35\xe4\xd4\x13\x4e\x8d\x4e\x2a\x3d\x7c\x99\x5f\xe9\xc6\x15\xd8\x03\xe0\xef\x21\xfc\x3e\x48\x99\xc0\x63\x07\x24\xea\x08\xbe\x29\x8c\x2e\x95\xeb\x81\xda\x7d\xe5\x7f\x69\x5e\xf3\x3f\x71\x5e\xf6\x97\xd8\x5f\x04\x95\xd9\x02\x5b\xfb\xab\xc2\x8e\x1a\x4f\x0e\xb8\xb1\x22\x5b\xc1\x7d\xf6\x2b\xb4\x6a\xd4\x93\x01\x7b\x9f\x50\xab\x87\xfb\xe4\x63\xa8\xb2\xda\x57\xb8\xd7\xc2\x4e\xad\xf7\xd9\xdb\xd0\x9a\xca\xa1\x17\x0a\xd8\xca\x50\x39\xfc\xbd\xaf\x96\xdd\x24\x82\x7a\x25\xda\x64\xed\xc7\x95\x35\xfe\xc6\x5d\xd5\x3e\x0a\x6d\xfc\x82\x16\x2a\x58\x02\x20\x18\xbe\x59\xd5\xcb\xcb\x2e\xe6\x40\x3b\xf8\x0b\xf2\x32\x87\x99\x7a\xf2\x8c\x42\x51\xbb\xc1\x97\xf5\x51\x85\x1b\x8f\x99\xc6\xcc\x4d\xc8\x08\x6a\x5d\xd7\x6e\xd0\x77\x6d\x98\xd2\xdb\x5b\x1b\x9e\x34\x8d\xc2\xd5\x74\x36\x9d\x4c\x47\x5f\x0d\x31\x56\x98\x56\x78\x4d\xf2\x9b\xe1\xe0\xab\x71\xb7\x7f\xfe\x0a\x9d\x03\xe1\x15\x46\x2c\x91\x5f\x06\xa7\x43\x8e\xf2\x32\x1f\x2d\xe4\x17\x1f\xb9\x64\x06\x7d\x31\xf0\x9e\xcd\x86\x80\x2e\x37\x86\x35\xc5\x71\x3a\x7e\x27\x59\x18\xbd\x9e\x4e\xc7\xec\xf5\xfc\xf6\xd6\xc6\xc4\xf2\xcb\x98\x47\x2a\xf8\xc1\xf8\xe0\x3f\x8a\xbe\x73\x31\xf9\x7a\xb8\x80\x6c\xc6\x84\x4a\xa6\xfa\xce\x25\xe4\xce\x93\x8b\xa8\xac\xf6\x81\xfa\xb3\x21\x9a\x6d\x6f\x2b\xc4\x1d\x21\xe7\x02\x99\x85\x3c\x42\x73\x91\x5c\xd4\xdf\x50\x6b\x1e\xea\xb4\x16\x32\xfd\x99\x0c\xfc\x81\xe1\x1b\xe9\x47\x75\x85\x41\x1b\xf9\x67\x7f\xad\x50\x3f\x18\xae\x12\xf3\x8a\x1e\x4f\x3e\xaf\x46\xd3\xe5\x5c\x6a\xbf\x56\xb8\x49\x92\x8f\x57\x0a\xf4\x28\x58\x0d\x86\x5d\xf6\xcc\x40\xa9\x9d\xd6\x9d\xf4\x34\x47\x84\xe3\xa7\xb3\x7e\xa2\x5b\x68\xe1\x97\xa0\xc7\xc1\x73\x57\x15\x27\xa9\xba\xb2\x51\x75\xd7\x5e\x46\x7f\x8d\x39\xad\xd8\xa0\x4a\x99\xaf\x3d\xe4\xaa\xcb\x09\xfb\xc3\xc7\xbc\x14\x8d\x5d\x14\xb9\x80\x8f\xd9\xa7\x42\x12\xd6\xa0\x12\xa6\xb3\x33\x04\x89\xd4\x6b\xcd\xd2\x1d\x9d\xe5\x75\x74\x96\xd1\x51\x1a\x02\xca\xf0\x47\x40\xbc\x64\xdc\xd5\x43\x2e\x60\xd1\xf9\x24\x10\xfa\xd1\xd6\x53\xe3\xb0\x1c\x7e\xa9\x09\xfc\x33\x4f\x3c\x40\x4e\xb0\x79\x2c\x92\xd4\x5b\x54\xe8\x00\x3f\xd0\x07\x0d\x02\x20\x1c\xdc\x5a\x87\x98\xdc\x61\xea\xa7\xd2\x2b\x68\x21\xe9\xe3\xf4\x76\xb8\x78\x0e\x02\x07\x6d\xac\x88\x1e\x28\x7d\xa8\x36\xcd\x76\xea\x23\xff\x62\xbe\x8e\x5b\x69\xc8\x18\x33\x28\x95\x48\xa8\x62\x7b\x06\x6a\x28\x84\x58\x5e\x61\xe3\x51\x56\x7b\x64\x56\x22\x8d\x86\x48\x3f\x5d\x84\x3c\xb2\xb5\x04\x91\x3a\x77\xca\x3f\x1d\x0c\x5c\xa1\xe9\xce\x2c\x42\x47\x0d\x56\x63\xf8\x6f\x06\x13\x1e\x4b\xc0\x74\xe7\xa3\xf8\xd9\x48\x12\xd6\x6a\x32\x1d\xa4\xba\xa7\xe5\xd7\x71\x0b\x73\x29\x10\x1e\x00\x0a\x4b\xa1\xa3\x9e\xaf\xd0\x63\x3b\x1f\x76\x61\x51\x0b\x3e\xae\xa0\x93\xaf\x31\xfa\xf9\xa5\x16\xf0\x9f\xea\xcb\x96\x0e\x1c\x07\x38\x25\x3a\x38\xb4\x15\x60\xa3\xd3\xb3\x2d\x10\xd5\x2e\x56\x43\x88\x20\x93\x01\x74\x85\x9c\x9b\x13\x23\x23\xf0\xd7\xc2\xd4\xf0\x1b\xf6\x82\xff\x0f\x75\xef\xba\xdd\xc6\x91\x24\x08\xff\xfe\xf6\x29\x2c\xcc\x98\x53\xc5\x4a\x80\x55\x00\x2f\x22\xa0\x22\x8e\xdb\x6e\xaf\xb5\x2b\x5f\x56\x74\xcf\x74\x9b\x42\xf3\x14\x80\x02\x59\xc4\x55\x28\x50\x04\x44\xa2\xcf\xec\x39\xfb\x4a\xf3\x02\xdf\x03\x4c\xbf\xd2\x17\x97\xbc\xd5\x0d\x84\x64\xbb\xf7\x9b\x6e\x8b\xa8\xcc\x8c\x8c\x8c\x8c\x8c\x8c\x8c\xbc\x45\x02\x60\x36\xdd\xa4\x36\x7e\x70\xdb\x14\xad\x63\x3a\xe4\x78\x9e\xdc\xd0\x13\xc0\x5b\xf7\x71\x92\x4b\x33\x95\x78\x4b\x93\x3b\x18\x75\xbe\xd2\x6f\x03\x54\x8f\x63\xc6\x1b\x62\x81\xb9\xc9\x2c\xc5\x23\x6d\xd4\x56\x8c\x8d\x15\xcf\xd3\x13\xb5\x39\x72\x89\x88\x08\x89\x4e\x37\xb3\x79\xbf\x43\xe3\x48\x79\xae\xa6\x4b\x0b\x6c\xe5\xb0\x25\x29\x63\xa2\x04\xab\x03\x0e\x70\x67\xf8\xb4\x6e\x21\x73\x96\x96\xf8\x39\x9d\x41\x3e\x45\x45\xfe\x19\x6d\xce\xd8\x14\x7e\x32\x5d\x12\xe7\xf3\xdc\xdd\x4f\x2d\x4e\x2e\xd1\xdc\x30\x65\x45\xeb\x90\xa2\xd8\x43\x1f\x7f\x92\x97\x3e\xee\x95\xe8\x11\xaf\xbf\x91\x3d\x12\xdd\xc8\x2d\x29\xbf\x51\x28\x83\x75\xb8\xbc\xd4\xa0\x83\x8d\x0c\x11\x2c\x6e\x04\x38\xfd\xf5\xe1\x60\x53\xef\x6f\x0e\x07\xd0\x4f\x6e\xfb\x21\x84\xfb\x6b\x0f\xc2\xfd\x8d\xb8\x1d\x84\xd6\x13\xca\xd6\x20\xef\x40\xc4\x6d\x1f\x73\xdd\x0e\xdc\xa3\xa1\x17\xe1\xa8\x8f\xa8\x6e\x07\x75\xc8\x71\xdb\xa7\xc8\xcd\x36\xcf\x96\x72\xbd\x7b\x49\x3a\xf0\xd2\xd2\xb7\x62\xcf\x26\xa1\x3c\x45\xc5\xb9\x8f\x98\xa5\xa6\xd0\xdf\x5c\xe9\x56\x6a\xdd\x9c\xe2\x8b\x50\xcf\x6a\x2d\x25\x57\xfb\xa1\xf9\x22\x3d\x30\x8c\xe6\x03\xad\x80\x31\xa0\x9a\x5a\x2c\xfa\x9b\x66\x48\x31\x75\x8d\x81\x1d\x8f\x42\x82\x12\x6c\xca\xde\xd1\x16\x13\xa2\xfd\x89\x80\x26\x46\xfa\xeb\xc1\x91\xdf\x49\x33\x42\xc7\x39\xec\xa2\x27\x99\xa2\x27\x58\xf6\xa4\xac\xec\x89\x55\xf8\x44\x17\x7e\x3b\x21\x68\xb0\x56\xf0\xaf\x88\x30\x7b\x70\x84\x84\x42\xe1\x94\x07\xe6\x21\xb7\x13\xfe\x34\xab\x53\x9b\x66\xd7\xa9\xf7\x3d\xb3\x60\xd4\x3f\xec\xd7\x9b\x87\x98\x70\xe8\xdc\x4e\x0e\x21\x87\x03\x61\x2e\xb4\x4e\xf4\x78\x14\x38\x6a\x7a\xcc\x99\x05\x05\x5c\x98\x56\x61\x26\x8a\x04\xd9\xa5\x1f\x8f\x28\x2a\x71\xd2\x5f\x39\xa6\x94\x34\xd6\x52\x71\xf5\x07\xf2\x86\x59\xad\x6c\x73\xed\xce\x03\xac\xc6\xd5\x29\x34\xbc\xbe\xea\x47\x1c\x07\x45\xae\x81\xbb\xdc\x22\xed\xe0\xa8\x64\x8e\xac\xa4\x3c\x9e\x4c\xe4\xb8\x6e\x66\x54\x24\x4f\x7a\xaa\x95\x66\xde\x46\xcc\xe7\xc7\xd9\x23\x22\x49\x9d\x98\xd6\x42\x8c\x7d\xc0\x37\x31\x70\xb2\xdb\x12\x03\x80\x10\xc9\x77\xd1\x64\x84\x1d\x4a\xdc\xca\x8f\x54\xcc\xbe\xd3\x9f\xfc\x4c\x6e\x3c\x1b\xe2\x75\x35\x46\x26\x8f\x54\xc2\xac\x5a\x86\xf9\x48\xe5\xc6\x4e\xc7\xd9\xb3\x9d\x8e\xf7\xd8\x90\x9e\x42\x7f\xc6\x48\x41\x3f\x21\x01\xe8\x67\x93\x30\xaa\x5e\x27\x03\x64\xa0\x53\xaf\x28\x9a\x71\x1d\x1c\xe0\x5f\xbc\xd9\x41\x1e\x1f\x5d\xf2\x6d\xa8\xeb\x40\xf0\xcc\x29\xab\x3a\xa1\x4e\x57\xae\x0e\x75\xf5\xd1\x49\xa2\x86\xbb\xd0\xd1\x1d\x17\xea\x6e\xb2\x5d\xe9\x84\x5e\x23\xa6\xdb\x50\xeb\x56\x08\x1f\x38\x9b\xe2\x8f\x8d\x7c\x7d\xd8\x64\x01\x9b\x57\x65\xfa\xd2\x14\xd1\x6b\xa4\xf2\x5e\xd6\xba\xc9\xaf\x0a\x23\x0e\xf5\xb9\x11\xb4\x13\xb1\x6e\xd5\xd7\x4d\x74\x66\xcb\xd7\x7d\x36\xad\xfa\x86\x82\x78\x60\xc3\x54\x25\x5d\x4c\x92\x41\xec\x98\x96\xf4\x45\xc9\xcc\x58\xa5\xe6\x8d\x12\x69\x20\xcc\x97\xc3\x78\x49\xe9\xc4\x39\x92\x36\x6c\x76\x45\x86\x7c\x78\x85\x96\x3c\xf0\xd1\x0e\x9c\x29\xfa\xe8\x4b\x14\xd3\x9b\xca\xd1\xe7\xa6\xd9\xde\x04\xdb\xb6\x22\x56\x3e\xc6\x42\x6b\x23\x32\x13\x25\x35\x65\x52\x77\xdd\x44\x2f\x87\x1b\x9d\x09\x4b\x92\x99\x30\xbf\xaf\x4a\x0a\xac\x92\x02\x5d\x92\x6f\x4a\x92\xe4\x11\xa5\xb9\x92\x7c\x5d\x12\xd2\x8b\x99\xf8\x28\xbc\xa9\x27\x85\x5d\xe1\x79\xa6\x79\xaa\x0d\x6f\x05\xf2\x23\x32\x2c\xe3\x68\xb4\xdf\xa0\xbd\xaf\x7a\xc4\xbf\xd5\xdd\x93\x87\x9a\xbd\x97\x56\xd6\xbc\x9a\xb2\xe1\x1f\x7c\xc3\x29\xbf\xce\xb2\xd9\xbd\xc6\x92\x19\xe1\xf0\xd9\xa8\xc7\xec\x90\x22\xb2\x7a\x10\x13\x0e\x0e\xd8\x14\x2d\x33\x69\x06\x97\xd9\x31\x2e\x6b\xb1\x10\x06\x8c\x7a\x11\xb2\xad\x22\x57\x74\xd6\xe1\x40\xda\x41\x60\xef\x0c\xa4\x1d\x64\xac\xa3\x7a\xdf\x36\x90\xc0\x32\xb1\x2d\x9f\x7e\xc6\xf2\xe9\x4b\xcb\x27\x22\xcb\x27\x22\xcb\x87\x1d\x98\x0f\x2f\x42\x30\xfd\x95\x0f\xf3\xdb\x28\x04\x90\x68\x0d\x86\xcc\x61\x54\x30\x86\xd6\x21\x19\x41\x11\x22\x20\x23\x48\x6c\x42\x44\x29\x2d\xa0\x08\x63\x06\x78\x1d\x03\x49\xb1\xbd\xab\xe4\xda\xf1\x99\x05\x29\x06\xea\xc8\x99\x7c\xc4\x7c\x96\x08\xb9\x09\x89\x17\x2a\x66\x1d\x82\xed\xb6\x56\x21\x60\xd4\xc6\x2b\xdb\x65\x55\xeb\x02\x90\x8e\xef\x80\xa9\x95\x36\xfe\xd1\x2b\x0f\xfd\x18\xbe\xe4\x55\x87\xb2\xc9\x1e\xd3\x96\x99\xec\xa9\x82\x5f\xd1\x84\x67\xf3\xf4\xa4\x29\x09\x79\x0e\xb4\x01\xbd\x2b\x49\x7d\xc5\x31\xec\xb8\x99\xa7\x4f\x6f\xdc\x47\x55\x28\x4d\xcc\x0a\x53\xa8\x37\xfa\x01\x71\x3d\xdf\xb2\x32\x94\xcc\xb8\x2a\x48\x96\x13\x15\xce\x2b\xf9\xe1\x0a\x0e\x3e\x3d\xe5\x75\xdc\xb7\xe8\x1c\xf1\x6b\x9b\x47\x6e\x89\xcb\xe3\x72\xd3\x5e\xf7\x17\xc9\x63\xc3\xee\x8e\x5a\xca\x51\x5c\xfb\x69\x9f\x92\x61\x0e\x9a\xb7\x40\x55\xa5\xe4\xa2\xa1\xaa\xcd\x0e\x71\xc3\xc5\x9a\x0a\xb0\x8c\x16\x51\x30\x96\x8c\x90\xb2\xfb\x6f\x3b\x6c\x87\x64\x41\x1a\xb0\x60\x3a\xc0\x78\x40\x63\x68\x2e\x03\xdb\x09\xd6\x55\x1c\xcb\x61\x64\xc6\x60\xc8\x58\x07\x19\xd3\xc1\xb6\x13\x5c\x91\x84\xb1\x35\x42\x77\x12\xb4\x02\x62\x8e\x43\x3f\xc5\xce\x8b\x3c\xc5\xf3\xd9\x0c\x6c\x2b\x1a\xbb\x62\x89\x0b\xfa\xe2\x0b\xba\x3e\x13\xbb\x3c\x74\xc6\xe8\xdd\xb3\x1e\xa3\x1b\x4f\xb3\xca\x86\x91\x1b\x8a\xa4\x05\x36\x1c\x55\xd1\xa7\x36\x3a\xcc\xa6\x3e\x13\x67\x86\x57\x81\xcf\x0e\x57\xb2\xcd\x26\x02\xc7\x60\xc5\x3d\xda\x45\xeb\x13\xf9\x8d\x3e\xed\x97\xf5\xd5\x8b\x14\x3e\xd9\x8d\xa3\xa9\x18\xf5\xc5\x87\x88\x41\xa2\x5f\x6d\x67\xb1\xb6\x26\x64\x13\xa9\x9b\x29\xb0\x14\x13\x33\x15\x9d\x98\xa9\xe8\x52\xab\x59\xb1\xd4\x3a\x56\x8c\x40\x3d\x4e\xd6\xde\x92\xbc\xd1\x8c\x40\x33\x4e\x36\xde\x12\x9d\xcf\x90\xa1\x8c\xda\x60\xc2\x8e\xcd\x61\xb8\x1d\xad\x9f\x9e\x46\xe8\x39\x58\x7b\x38\xc7\x21\x60\x7d\xb1\x64\xcd\xf0\x21\x92\x3f\x0d\xf4\x27\xac\x60\x78\x37\xef\x03\x79\x30\x1f\xad\x79\x60\xee\x7c\xe8\x9b\x60\xb0\xd5\xaa\xc2\xc2\xf0\x0a\x86\xf3\x2a\x04\x41\x16\x81\xbf\xdd\x6a\xef\xea\xa3\x29\x56\xa7\x8e\xd5\x01\xea\xeb\x40\x3b\x30\x3d\x1c\x6d\xea\xa3\xe9\x21\x40\xd7\x83\x8b\xd1\x14\x2a\x31\xbd\x08\xdc\x4f\xa4\xdd\xd9\xf8\xf5\x11\x4c\x97\xa1\x19\xed\x3a\xe0\xde\x90\x89\xfe\x94\xba\xe4\x73\x6a\x84\xd9\x72\x74\xd5\x96\x9b\x0b\xd9\x14\x06\xb5\xdd\x18\x36\x6e\xb2\x80\xa0\xca\x6b\xdf\x1b\xf5\x15\x66\x32\xae\x30\x32\xc0\xc8\x12\x42\xd7\xaf\xd6\x65\x84\x66\xf3\x29\x64\x99\x12\xd4\x71\x22\x16\xed\xf0\x43\x24\xb8\x1f\x84\x1f\xfa\xe2\x45\xf5\xfc\xc7\x5a\x1b\x90\x16\x04\x99\x39\x13\x96\x59\x36\x8a\x96\x2c\xaa\x1c\x88\xd8\x0e\xea\xef\xb3\xd5\x94\x47\x0e\x5d\x4f\x7c\x90\x3b\x85\xb4\xe4\x50\x32\x88\x17\x08\xea\x94\x4f\x13\xd9\x1e\x27\xed\x8c\xa8\x5c\xc0\xad\x5c\xae\x96\x2d\xa1\x14\x57\x4f\x22\xc8\xd2\x7f\x36\x0b\x03\x73\x46\x20\xbd\x30\x50\xd0\x54\x89\xfb\x77\xd2\xe3\x49\x10\x13\xb5\x6b\x5a\x90\x27\xc6\xad\xc0\xbb\xfc\x1c\xbc\x16\xc5\x80\x17\xa3\x9e\xdb\x10\x34\xf3\x91\xc9\x67\xb4\x52\xe6\x60\x40\x99\xf4\x3d\xd7\x70\x3b\x49\xdc\xd9\x84\xbc\x0c\xf5\xc8\x85\x3e\x3d\x71\x91\x5d\xd6\xc7\xa1\x34\x89\xbb\x8a\x10\x02\x6e\x2b\x02\x39\xe4\x64\x82\x42\xe6\xe4\x32\x58\x93\x4b\xbb\xfa\xd9\x29\x4a\xb1\x4d\x79\x38\xd2\x63\x8d\x19\x98\xcc\x1e\x2e\xe5\x31\x13\x0c\xab\xcb\xf1\x19\x77\xae\x80\x75\xda\x41\x99\xe5\x7a\x40\x91\x46\xbb\x1c\x6b\xdc\xb6\xae\xfb\x24\x9f\xf5\x03\x0c\xc7\x75\xd4\x30\x02\xf5\x61\xfd\x03\x0e\xc4\xb6\xff\x0c\x4c\xc2\x58\x48\xef\x63\x3a\xfa\xe6\xde\xb1\x09\xca\x86\xcf\xcf\xcb\x18\x27\x57\x44\xf2\xf5\x6e\x8d\x90\xb1\x95\x68\xfb\xe0\x91\xcc\xce\x3f\xb1\xf5\xf9\xb5\xb4\x59\xa5\x2d\x2a\x6d\x5a\xb9\x93\xb0\x1f\x66\xbe\x4a\x4d\x0f\x1b\xac\x80\x2e\xc1\x85\xd0\x35\x09\xde\x9b\x78\x2f\x91\xe3\x73\xd7\x78\x02\x67\xd1\xf8\x53\x87\x3f\xbb\xfc\x03\xc5\x87\xe1\xc2\x04\xde\xb7\xe5\xe7\x5b\xf8\x44\x9c\x50\xc9\xf7\xe2\x3d\xd0\xcc\xf1\x02\x30\x40\xc4\x02\xd3\x1b\x6f\xf0\x17\xdf\xbe\x6b\xbc\x45\x00\x17\xe0\xde\x84\x05\xef\xbe\x15\x54\xd3\x02\xda\x6e\xb2\xdf\xfc\xf6\x64\x03\x28\x72\xa3\xf1\x86\xc8\x7e\xa3\xc8\x7e\xbb\x07\xd9\x64\x6c\xcb\x76\x44\xab\xb5\x23\xb7\x9e\x32\xfb\x50\x6a\xcf\x00\x1f\x19\xaa\x42\x48\x8b\x6f\xa9\xe8\xf7\xe7\xea\x86\x03\xdd\x0d\x90\x5b\xe8\x7a\x39\x0e\xba\xc8\x9c\xcf\xdd\xdb\x99\xff\x95\xfa\x2d\x2d\x0f\xb8\x3c\x31\xa4\x09\x59\x99\xc6\x41\xff\x99\x65\x4a\x96\xb4\xdb\x57\xcb\x65\xb4\x61\x4a\x94\x63\xce\x8a\x6d\x9b\x32\x65\x68\xf7\x86\x8a\xf9\xc6\x73\xd9\x3a\xd6\x8c\x30\x6f\xf5\x5b\xf3\x1a\xe2\x07\x34\xd6\x0b\x86\x7c\x7a\xe2\x45\xcf\x57\x6a\xe6\xa8\x22\xf0\xe9\x13\x19\x75\x70\xc0\xab\xa0\xaf\xf4\xce\xb7\xeb\x70\xcc\x8b\x30\x5c\xfb\x2a\x07\x04\xc0\x44\xd2\x2e\x74\x73\x03\x51\xca\xe3\x50\x59\x2d\xcc\x02\x6a\x61\xb1\x3f\xb3\x6d\x8a\xc6\xb6\x5c\x21\xdf\xc8\x2f\xf4\x32\x66\xda\x97\x9b\x4f\x4f\x59\xe5\x74\x8a\x1f\xab\xde\x71\x06\xc3\xcc\xf6\xdd\x2d\x0a\x51\x49\x15\xf4\x34\x8b\x84\xac\xd0\x42\x66\x05\x97\xd2\x79\x89\x79\x98\x44\x37\x4b\xf4\xa9\x47\x8b\xa1\xed\xd2\xb5\x54\x1a\x95\xf3\x49\x54\xd2\xb6\xc2\x68\x51\x42\x54\x21\x24\xa5\x72\x5b\x2e\xb4\x38\x67\x92\x34\x56\x76\x2d\xab\x77\xe4\x16\xcf\x36\xf5\x08\x85\x05\x87\x04\x50\xfb\x95\x08\x7e\x5e\x26\x34\x10\x91\x1f\xf2\xec\x63\x31\x38\x5a\x0c\x40\x98\x0e\x1d\x89\xcd\xad\x53\x5c\x9f\xe2\x06\x32\xae\x88\xf9\xfd\x7d\x34\x44\x95\xf4\x35\x3a\xc0\xcf\x9e\xc5\x2b\x23\x24\x0b\x9e\x39\x93\xd7\xd8\x54\x83\xd3\x20\xa3\xdd\xdd\x4c\xe2\x68\xd4\x7e\xe1\x93\x66\x4d\xdb\x57\xf2\xea\x1a\x2d\x4d\x8a\x35\xff\x6c\xda\xec\x36\xbf\x12\x23\x3e\xd5\xb1\x72\x46\x84\x43\xfa\x54\x62\xff\x4b\x24\xac\x23\xa7\x10\x2f\xb7\xa8\xd6\x61\xe3\x04\x4f\xac\x79\xeb\x26\x5e\x59\xa2\xd0\x26\xf0\xd0\x71\xd3\xe0\x36\x99\x0c\x41\x1f\xb3\xb6\x24\xe2\x3a\x2a\x8e\xde\xa3\xab\xa2\xc1\x02\x92\x45\xa6\x6b\xc0\x6d\x30\xd2\x5b\x77\xcf\xe6\x86\x09\x2c\xe4\x63\x82\x33\xb9\x9b\xfb\xe4\x6e\x52\xd9\xe9\x86\x70\x58\xb5\xb9\x6a\xed\x93\xbb\xd5\x63\x9a\x25\xb3\xb2\x8c\xb7\x6e\x04\xbd\xbd\xe9\xb3\xe8\x46\x74\xef\x0d\x43\xae\xe8\xab\xef\xbe\xbc\x10\xb8\x0c\x23\x98\x73\x47\x37\xf0\x73\x23\xa2\x3e\xfc\xf4\x45\x7f\x19\xf6\x1b\xcb\x7a\xb4\x14\xfd\x1b\xf8\xba\xa9\x47\x37\x30\xc8\xc0\x57\xbf\x1e\xf5\x0b\x77\xe4\xb4\x9b\x91\xda\x3f\xd5\xd0\xc9\x30\x60\xbf\xbe\x8d\xd7\x7c\xbc\x74\x89\xae\xc4\x9d\x68\xe9\xf5\x97\x87\x2b\xd7\xad\x4a\xbf\xf1\xfa\x37\xbb\xd2\xfb\x5e\xbf\x8f\xe9\x95\x95\xfd\xb1\x7f\x27\x1f\x00\x95\xc7\x79\x45\x12\x3e\x6e\xc5\x00\xfe\xd0\xb0\x36\xfe\x22\x99\x7d\x11\xb9\xf4\xd3\xef\x26\x57\xe3\x5e\x98\x45\xe0\x44\x10\x27\xfa\xf0\xc7\x6d\x0f\x30\x19\xc3\x26\x6b\x9f\xb3\x46\xf8\x66\x2c\xa6\x12\x60\x19\x2b\x74\x8e\xc4\x25\x40\x2c\x0a\x2f\x3f\xa9\x37\x25\x2a\x6b\xf0\xc3\xfd\xb4\x9f\x57\x36\xf5\x30\x0a\xbd\x48\x14\x59\xfd\x05\x7a\x7f\x5b\x55\xe2\xba\x5c\xe1\x31\x21\xc3\x8d\xa9\x48\xc4\x9d\x98\x89\xb9\x48\xf1\x99\x87\x34\xc0\x3f\x38\xaa\xbf\x87\x3f\x54\xc9\xc8\x0b\x6b\x35\xd1\xa7\xbf\x59\x5c\xd7\x33\x22\xac\x31\x89\xd2\xd5\xeb\xd9\x30\x5e\x43\x5e\x7c\xab\x6b\x1a\x96\xc3\xc5\xeb\x78\x80\xd2\x85\xef\x78\x4d\x1b\x09\xe6\x80\x71\x54\x1e\x21\x6c\xa4\xf7\xfd\x94\x89\x4b\x89\x0e\x09\xe1\xa2\xed\x44\x20\x8f\x49\x5b\xef\x7a\xad\xdb\xf8\xee\xf6\x16\x3a\xbf\x9c\xbf\xd1\x86\x48\xea\x57\x94\xac\x29\xa4\x1a\xa5\xfe\xab\xbe\x44\x54\x51\xbe\x8b\x8b\x78\xf8\x20\xd9\x7b\xfd\xfe\xc9\xce\x4a\x45\xf8\x3e\xb3\x7a\xa2\x0c\x54\xd7\x3c\x7c\x8f\x2b\x7d\xf3\xc6\x3a\x0c\x91\x50\xd2\x67\xf3\x06\xa5\xf1\xd9\xf2\xf4\x0a\x82\x5e\xd0\xa3\x4d\x3f\x0a\xd4\xf1\x25\x67\xc8\x21\xf4\x82\x1d\x44\xe2\xc3\x6d\x77\x21\x00\x76\x66\xf2\x2d\xb3\xf7\x57\x77\xbd\x06\x2e\x29\xd2\x42\x46\x21\xb7\xa7\x10\x67\xd1\x34\x2b\xd0\x84\xd6\x2b\xca\x59\xba\xe8\xa3\x87\x28\x73\x25\xf5\xaa\x8b\xf1\x82\x1d\xf4\xbe\xb7\xd7\x21\xc5\xac\x5e\x17\x10\xcb\xcb\x31\xc8\xa7\x52\x91\x07\xcb\x3a\x8d\xbf\x9d\xcc\xa3\x95\x43\x6c\x14\x56\xc4\x1c\x8d\xae\x0e\x5b\xc9\xc0\x79\x17\x78\xce\xd6\x8e\xc8\xd6\xa4\x6b\xa8\x6e\xef\x57\x03\xa2\x4e\x75\x4b\x7c\x01\x4f\xbb\x78\x51\x98\xfd\x5e\x97\xda\xd8\xef\x41\x7b\x95\xf4\xc1\x39\x7c\x7b\xb5\xda\xd6\x2d\x79\xb1\xb8\x6f\x5d\xf5\x94\x5a\xc1\x7e\xe0\x2e\xbd\x92\xd2\xe3\x4a\x42\x2d\x15\x91\x6a\x2f\xf6\x95\xfd\x5b\x3e\x3e\x23\x57\xbc\x47\x20\xc6\xa0\xd5\x4d\xfa\x7c\xa9\x97\xa5\xeb\xf5\xe4\x22\xf4\x0f\x0e\x5e\x38\xa3\x02\x0c\x14\x4f\x88\xdc\x8e\xd1\x64\x55\x25\xb2\x95\xaf\x15\x4a\x22\xd6\xf4\xbe\x11\xfe\x99\x45\x30\x68\xc8\x1e\x3b\xc3\x51\x42\x7d\xcb\x9b\x45\xf8\xa2\x97\x06\x50\xa9\xdc\xa6\xc4\x12\x5f\xf2\x64\xcd\x7d\xb4\xa0\x96\x13\x54\xcb\xc0\x29\x25\x06\x91\x84\x1f\x40\x64\x88\xc9\x32\xbe\x6f\xc7\x63\x8e\x2a\xf5\x9c\x29\x95\xa0\xd7\xc8\x8a\x4a\x25\x1d\x47\x69\x7c\x3d\x98\x44\xd3\x85\x33\x2a\xba\x2e\x31\xf2\xe0\x5f\x84\xab\xae\xdf\x5e\x5d\x84\x41\x37\x68\x8f\x20\xa5\x04\xcf\x32\xa6\x5b\x1d\xbb\x31\x05\xf5\x91\x13\xd4\x2b\xf2\x8f\x26\x38\xd2\xed\xca\x8f\x86\x52\xe3\xe4\x62\xd5\x1d\x39\x4d\x18\x35\xdb\x4d\x40\xd7\xac\x37\x8b\x03\x28\x61\x44\x43\xc3\x2a\x7b\x75\xb8\x2a\xa9\xfd\x7d\x3f\x19\x64\x81\x2a\xc1\x5e\xcf\x7e\xbc\x5f\x21\x2c\xa8\x1b\x64\x89\x3a\xab\x42\x07\xe2\x90\x39\x2a\x22\x20\x0b\x64\xd5\x0c\x01\x97\x58\xb5\xc2\x15\x50\xa8\x9a\xe0\x58\xd6\x60\xd5\x6a\xb7\x0e\x9d\x55\x7d\xd5\x74\xbd\x55\xab\xde\x38\x3b\x71\x8b\xc5\xe2\x6d\x04\x27\xde\xc5\x50\x73\x75\x50\xc4\x65\x3c\xe0\x5b\xc6\x86\xfd\xe6\xda\xf2\xa1\xbc\x62\x59\xcc\x13\xaf\x17\x65\x25\x34\x45\xe0\x23\xc5\xf9\x6d\x18\x66\x10\x6f\xd6\x15\x8a\xa2\x7d\x53\x68\x71\x68\xa2\x92\x82\x70\x60\x03\xf6\x47\x62\x21\x2d\xe3\x4e\x85\x77\xaa\x57\x4d\x5c\x8d\x08\x1b\xc7\x27\xae\x28\x78\xae\x4a\xc3\xc5\xd1\xdf\xff\x8f\x75\x25\x3f\x38\x8a\xdc\xb6\x13\x85\x60\x8a\x42\xd2\xb1\x5b\xa6\xe3\x02\x2f\x3a\xb4\xaa\x56\x87\xba\xad\xac\x0b\x47\x50\xcf\xd4\x3d\xfc\xfb\xff\x39\x5a\x94\x71\xb5\x0f\x33\x74\x27\x35\x5e\xe8\xc0\x78\x4a\xc3\xa0\x71\xe6\x07\x27\x2f\x4b\x4b\x43\xb1\x72\x9c\xd4\x0b\xdc\x43\x44\x5c\x86\x12\xec\xc1\x41\x86\x81\x47\x4d\x10\x0a\x90\x94\xb3\xc6\xc9\x69\xf3\x04\xe5\xb2\xdd\xcc\xc5\x01\x95\x50\xec\x09\xc5\xe2\x4d\x72\xf8\x69\x37\x65\x38\x0b\xd5\x6c\x34\x2d\xb0\xf3\x16\x00\x66\x52\x4f\x33\xc9\x2f\x8f\x01\xa0\x4a\x63\x7e\x37\x98\x58\xe6\xf7\x2d\x86\xa4\xf9\x8d\xdf\xca\xfc\xbe\x05\xdd\x79\x2b\xa2\x01\xfc\x0c\x44\x34\x41\x4d\x2a\xfa\xb7\xa0\x44\x6f\xeb\x11\x68\xcb\x01\x7c\x0d\xea\xd1\x40\xf4\x27\xa8\x58\xeb\x91\x76\xfe\x97\xa4\x3f\x44\x3f\x38\xfd\x01\xae\x3b\x00\x98\x8f\x38\x38\x2e\x1a\xb8\x5d\xc8\xd5\x86\x5f\x21\xa1\x6e\xdd\x2e\xfc\x41\xa0\x5b\x05\x74\x8b\x40\xb7\x6d\xf8\x6d\xf7\x6f\x2f\x82\x97\x7e\xb7\x7f\x5b\x0f\x5b\xa7\x7e\xbb\x0e\x81\x8b\xfe\x2d\xe2\xbd\xf5\x30\xa6\xb4\xa9\xa0\xb2\x50\x8f\xeb\x49\x04\xd3\x8a\x5b\xaf\x7f\x0b\x5d\x38\x1a\x78\xfd\x01\xfe\x4e\xbc\xfe\xe4\x90\xc7\xc6\x4a\xee\xa4\x19\xee\xa4\x16\x77\xd2\x02\x77\x52\xf8\x49\x4b\xb8\x93\xc2\x57\x5a\x8f\xd2\x4a\xee\xa4\xc4\x9d\x14\x2b\x9e\xaa\x8a\xa7\x58\xf1\xb4\x1d\xa5\xbf\x37\x77\xd2\xc9\x35\x4d\xba\x14\x77\x52\xaf\x9f\xee\xc9\x9d\x37\x91\x3d\x75\x23\x1e\x4b\xee\xe0\xb7\xe2\x0e\xf3\x23\xc2\xe1\x37\xd2\x53\x37\xc5\x09\xd1\x8f\xe0\x2b\xaa\x47\xd1\x3e\x53\x37\x2c\x1f\x50\x33\xc1\x4c\x20\x20\xf6\xfa\x11\xfe\xca\xc9\xd7\x2e\x82\xdf\xf2\x34\x2d\x37\x77\x29\xe3\x8c\x3d\xab\xc3\x39\x4c\xae\x93\x1b\xbf\xbd\xd2\x8f\xf6\xd2\x0f\xaf\xa6\x50\xbf\x69\xa3\xdf\x13\xcb\x00\x03\x03\x08\x0c\x7b\x62\x4c\xe6\xa4\xce\x60\x7c\xdd\x2c\xd1\x4d\xc9\xc7\x4c\xe2\x37\xf3\x15\x44\x43\x7e\xf4\x55\x52\x91\xcd\x8e\xfd\x7a\x3e\xed\xe3\xc9\x80\x65\x20\x20\x5b\x7d\xfc\x11\x1f\xcc\xf5\x3b\x4b\x1f\xfd\x82\x2e\x03\x7c\x3c\x18\xfe\xe2\xb7\x4f\x0b\x06\x0e\xa7\x84\x75\x84\x47\x67\xa1\xf8\x35\x5e\xf3\xcf\x47\xfc\x91\x87\xa4\xa4\xa7\x41\x67\xbc\xce\xec\x4d\xf8\xe4\x2d\x11\x71\x64\x36\x16\xea\x54\x88\xa0\x02\x33\x9e\x44\x18\x99\xb9\x79\x0b\x5c\x89\x81\x2b\xa3\x9e\xdc\x12\xa1\x2b\xbf\x57\xe3\x35\xd4\x56\x45\x8d\xe3\x87\x70\xbc\xb1\x4b\x1d\x7f\x84\x64\x1b\x6b\xdb\x2f\x6f\x0b\xe4\x9e\xdd\xb6\x78\x43\xef\x10\x6f\xe8\x79\x78\x41\xef\xb0\x9f\xbf\xe2\x59\xc2\xdc\x48\xdd\xb2\xcd\xf8\x81\xc9\x95\x00\x13\x2c\x25\xa2\x63\xe0\x29\x16\x73\x14\x8e\xe9\x45\x67\xf8\x85\xa6\x2b\x2f\x45\x35\x16\xae\xb2\x8d\x33\x44\x7a\xe1\x98\xe8\x24\x14\x1c\xc0\x7b\xa4\x55\x52\xfc\xb3\x16\x3e\x6d\xe8\xce\xcc\x3c\x59\x7c\x85\x1d\xd0\x08\x28\x74\xc9\x3f\x64\x63\xfa\xd0\xc8\x51\xf8\x95\x75\x39\x7b\xd5\x0f\xff\x60\x05\x97\x98\x2a\x5d\xa9\x2e\x31\x49\x7e\x3f\x60\x3c\xb6\x90\x78\xc0\x58\xfa\x1a\x53\x1c\x5d\xfc\x1e\x53\x24\x7e\xea\x2b\xb1\x58\xbb\x17\xe1\x8a\xaf\x49\xae\xb0\x7a\x14\x82\x89\x90\x23\xe7\xba\x35\x73\x85\xb9\xc6\x47\xcf\xf1\xf5\x33\xfe\x70\x6b\xf6\x7c\x3b\x80\x89\x76\xe9\xc4\x8c\x4a\x11\x54\x88\xbb\x15\x00\xd9\xda\x01\x19\x10\x24\x3d\xa3\xe1\xb6\x15\x61\x44\x51\x09\x41\xde\xaa\xef\x21\x15\x6d\x95\x06\x04\x2d\xa3\x17\xe1\xb2\xdf\x75\x96\x51\x7d\xd9\x27\x9d\xbb\xec\x93\x86\x6d\x2f\xfb\xf5\x65\x84\x31\xd8\xd1\x22\xa9\x75\xed\xf5\x02\xfa\x92\xcb\xe0\x5e\x8d\x99\xea\x58\x95\xad\x37\xab\x08\x5f\x46\xd0\x10\x44\xf2\xb2\xaf\x97\x09\x0a\x98\xbc\x25\x93\x0b\x0d\xf5\x22\x7c\xe8\x77\xab\xcb\xc6\xa6\xfb\xf3\x7e\x45\x3f\x44\xd0\xda\x50\x74\xfb\xa1\xa4\x64\x89\xc7\x7b\x90\x05\x8f\xb9\xc1\xc7\xcc\xd7\x31\x37\xf8\x98\x1b\x7c\x16\x16\x72\xd3\x3d\xf6\x67\x9a\x7d\x56\x3f\xae\x22\x8d\x4a\x13\x63\xd3\xf0\xb3\xea\x6a\x10\x2d\x62\xac\x9b\xde\x09\x34\x99\x81\x24\xd1\x2d\xa9\x1f\x53\xe8\x8d\x65\xfd\xcc\x22\x8c\xc8\x4f\xd7\xb0\x1b\xce\x61\x8a\x5b\x0f\x70\xc2\xf6\x6a\xd6\xf9\xac\xd9\xf3\xfd\xec\x99\xb5\xb6\xb0\x5f\x77\x70\xb5\xcd\xed\x06\x47\x0e\x8c\x96\x6e\xdb\x17\x45\x77\x5f\xce\x1a\x52\x0e\xfb\x3b\x70\x7f\x4d\x13\xc5\xcf\x40\xfd\x85\x7e\x1e\xdb\x37\x4f\x61\x07\x42\x96\x98\x9f\xba\x4d\xa2\xcd\xfc\x7e\x75\xdd\x87\x81\x74\xc2\xef\x1b\x4d\x92\xd9\xd8\x70\x2c\x55\x2f\x5e\x8c\xa5\xeb\x31\x3c\x26\xcd\x61\x76\x32\x26\x26\x03\xf2\x28\x97\x41\xf4\x06\x8c\xec\x15\xe8\xd3\xe9\x7c\xf6\x15\xd8\xd9\x29\xfa\x1d\xd0\x67\xeb\xa5\xeb\x72\x50\x88\x14\xd5\xeb\xd0\xcf\x8b\x30\x04\x4c\xd0\x28\x54\x20\x9f\x0b\x57\x7b\xa5\x04\x2e\x5b\x1e\x13\xcc\x6d\xd3\x71\x28\x13\xe5\x3a\x05\xa0\x57\x88\x64\x82\x5c\xae\x19\x0b\x9f\xcb\x46\xf2\xf1\xe4\x3a\xe3\xd6\x3e\xb0\x09\xb8\x9a\x33\xaa\x16\xa9\xb5\xdf\xca\x8e\xf0\x64\x3c\xbd\xd8\xcc\x9b\xc3\xb4\x97\x20\xd1\xb3\x47\x43\x19\x70\x35\xb4\x5c\x87\x44\x54\x7c\x54\x55\xd5\x54\x6e\x2f\xf3\xde\x71\x96\xc4\xd2\xcc\x3a\xb2\x9a\xf4\xb2\xb6\x20\xb1\x82\x69\x74\x14\x86\xa1\x3a\xa0\xf7\x45\xa4\xb9\x1a\xe1\xe6\x4d\x5a\x68\x55\xc3\x04\x34\x23\x9f\x83\x81\x51\x8c\xf0\x84\x8c\x4d\x2e\xb7\x51\xb6\xb0\x6f\x47\xa5\xb7\x50\xcf\x21\xc5\x93\x5b\x72\xce\x15\x32\x10\x08\x84\x49\xa6\x94\xfd\xb0\xea\x7e\xac\x73\x97\x71\x08\x2a\x3c\x88\xbf\x59\x46\x37\x7c\x3b\x61\xe8\x3e\x0e\x1b\xa3\x64\x1d\x0f\x9f\xc2\xe6\x4e\x78\xbc\x0d\x61\xa0\x0f\xc2\xfa\x59\x25\xf8\xf7\xf3\xfb\x34\x9e\x7f\x00\x3d\x61\xa3\x3f\x16\x20\x82\x60\x79\xe2\x4d\xd5\xc6\x02\x6c\xc9\xdc\x5e\x5a\x11\xc3\xfd\x2a\x5b\xe2\x49\x25\xfc\x57\x83\xc1\xfd\xf4\x9e\x46\x48\x5c\x94\x81\xa9\xc2\xe2\x36\x12\x83\x5b\xec\xaf\xd2\xb3\xc2\x00\x97\xdf\x07\x9b\x90\x16\x53\x10\xa8\xc1\xc9\x10\xfb\x82\x82\xb8\x61\xe7\x2a\x79\x18\xf0\xbe\x5d\x48\x29\xf4\x29\x78\xcb\xcc\x5c\x2c\xb1\xd4\x2a\x5f\x27\xa5\xd3\xab\x2c\xfe\x03\xde\x10\xae\x20\x71\x90\xa3\x4f\x58\xe4\x78\xe1\x40\x7e\x89\xc1\xda\x04\x0e\xe1\x03\xcf\xc8\x67\x63\xf8\x01\x7f\xca\x2d\xdd\x5d\xea\x9a\x3c\x3d\x59\xf1\x8d\xb5\xf4\x18\x07\x76\xc4\x70\x3e\x75\xdc\x7a\xe3\x44\x58\xc9\x9b\x62\x32\xcf\x94\xc6\x21\x51\x7a\x28\x29\xbd\xb2\xf2\xd0\xee\x43\xaf\x93\x21\xdd\x24\x7f\xcd\xbc\x1d\x53\x25\xc6\x87\x36\x29\x54\x8b\x4c\xd4\x66\xcb\x58\xd6\xe1\x60\x7d\x64\x21\x94\x7c\xc1\x43\xf0\x76\x74\x99\x14\xdc\x26\xf1\x32\x5a\x0e\x6e\x37\x6f\xd9\x9b\xf7\x9c\x76\xb5\x84\x8e\xb6\xa6\x6b\xca\xe1\x77\x1e\x44\xd4\xf0\xb0\x07\x8c\xf9\x6a\xbf\x10\x3e\x3f\x44\x93\xfb\x18\x06\x59\x86\x65\x41\x08\x65\x46\x19\x87\x03\x83\xad\x14\x34\xbe\x37\x98\x20\xa1\x76\x92\xfc\xb5\x2c\x2f\xb3\xcf\xac\x88\xd8\x99\xf3\x5f\x91\xbc\x4c\x36\x22\x78\x67\x9e\x4b\x3c\xd1\x92\xdd\x98\xa7\x4c\xf5\x68\x8f\xcc\x54\x27\x52\xc4\xa9\xcd\x51\x7e\x6e\x9f\xbb\x47\xc6\xd9\x27\xeb\x73\x6d\x02\x48\x2d\xaf\xea\xf6\xf4\x74\x05\x86\x48\x26\x03\x25\xe9\x3d\x74\x1e\x7f\xe5\xc1\x22\xc1\xa3\x6f\x9b\x40\xb6\x5b\x34\x99\xca\x68\x05\xbd\x36\x18\xef\xda\xe2\xb7\xe1\x76\xed\xed\xdb\x70\xb8\x2e\x3b\xc4\x13\x42\xf8\x9c\x65\x63\x43\x6f\x97\xc1\x6f\xb8\x23\x07\x1e\x80\xf8\x26\x1e\x45\xf7\x13\xc8\x0a\xd3\xc6\x8c\x04\x92\x2b\x55\x8c\x55\x8b\xfa\xd5\x78\x46\xa3\x34\x5e\xfd\x12\x2f\xe7\x12\x8b\xd2\x50\x77\x38\x43\x9e\x86\x18\x89\xfb\x2c\x52\x2b\x01\x69\x57\x3d\xdc\x58\x7a\x35\xed\xb8\x1b\xff\xea\xae\x17\x6a\x77\x62\x1b\xbf\xb2\x94\xef\xa3\x35\xed\xff\x39\x11\xee\x56\x98\x42\x70\x33\x38\x10\x77\xa0\x23\x3f\x84\x94\x24\xcf\xa4\xcf\x38\xa4\x6c\x10\xb5\x39\xe3\x8c\x25\x54\x42\x27\xfd\x2f\x3e\x80\x1a\xbc\x0b\x13\xc8\x3c\xd6\xc3\xd3\x5d\x25\x11\x6f\xe3\xe1\xfd\x20\xbe\xbc\x9f\x66\x1a\x65\x49\xb1\x4e\x0e\x16\xa0\x84\x5f\xcd\x35\x44\xb2\x10\x06\xcd\xc2\x2b\x78\x53\xd2\xa2\x0d\x63\x37\x1e\x68\xf9\x43\x32\xbb\x04\x58\x3c\xb1\x43\xcd\x23\xa8\x37\xa4\xd9\x75\x9e\x42\x96\x6f\x71\x68\x92\x19\x78\xe9\x3b\x4e\x26\x8e\x76\x21\xc5\x38\x54\x23\xb3\xa3\xbe\x37\x3f\x34\xbd\xfc\x12\xf7\x33\x98\x67\xd6\xfd\x4e\x6c\xf6\x7e\xe8\x51\x02\xce\x35\xa6\x21\x03\x91\xc3\x1f\xf4\xe5\x36\x62\x09\xc0\xdb\x3b\x1d\x77\x74\xb5\xee\x85\xd3\xc3\xb5\xd7\x2f\xdf\xa8\xca\x17\xfc\x96\x24\x33\x5b\xf5\x2b\xec\xe1\x60\x51\xcb\x58\x7c\x33\x01\x6d\x6d\x19\x2c\xe5\x2a\x1e\xbf\xb8\xc4\x7b\x9b\xec\xa4\x3d\xb3\x04\x22\x8d\x3c\xb0\x76\xe4\x57\x37\x68\x97\x9a\x1e\x88\x84\x0e\x56\x9a\xc3\x89\xd9\xd3\x2b\x2a\xa4\x77\x9f\x64\xf8\xe0\x40\x7d\xa9\x15\x7c\xeb\xf0\x4a\x9b\xb2\x12\xf2\xc6\xea\x76\x19\x47\xc3\xaa\xb2\xf9\x78\xa4\x29\x7c\x26\xf6\x2d\x1e\x26\x96\x39\x0a\x5c\x43\xc2\xac\x1e\x7c\x02\x11\x97\x31\x2a\x5e\x3e\xe9\x83\xaf\x17\x00\xc7\x76\xb2\x02\x0f\xf4\xed\xa2\xc3\x58\x38\x98\x20\x66\x59\x3b\x86\x0b\x60\x0c\x61\x29\x21\xba\x1a\x60\xeb\x28\x7a\xf8\xf4\x28\xbe\xa0\x46\x74\x72\x99\x6e\xe5\x41\xcc\x3c\x8b\xa7\xf3\x34\xb7\x4a\xc6\x47\xba\x76\xc9\x44\x21\x4f\xd9\xa9\x32\x2b\xcf\x37\x71\xbc\x88\x0b\xc5\xf0\xe3\x81\x7d\xfe\xad\xca\x4a\xc7\x88\xbe\x1a\xc1\x34\x55\xb6\x42\x34\x99\xe0\xfe\x8c\xe5\x5b\xf2\x03\x9d\x34\xa2\x54\xe5\x5a\xe8\x32\xe9\x83\x65\x70\xf3\xdb\xb6\x95\x42\x4e\x16\x03\x1f\xc7\xcb\x34\x1f\x45\xdb\x2d\xc4\x84\x71\xf6\x4c\x66\x37\x87\x8b\x40\x3a\xaa\x6a\xe5\x55\xd9\x5a\xd5\x2c\xfa\xde\xb3\x45\x85\xdc\x28\x65\xa7\x8d\x4c\x03\x39\x58\x42\x3b\xfc\x16\xf5\x0c\x7d\x94\x71\x06\x6a\x95\xe3\x85\xdc\x8e\x2f\xa9\x23\xf7\x22\x46\x84\x97\xc5\x27\xc9\xd4\x0b\xa9\x1c\x19\x37\x9d\x0f\x55\x04\xfd\xf5\x38\x77\x83\x03\x0e\x93\xa2\x22\x39\x54\x59\xb3\xef\xf1\xfa\x9e\x9a\x81\xf2\xb1\x3c\x42\xe3\x3e\xaa\xc8\x50\x7d\x48\xba\xcc\xc9\x64\x0a\xf3\xa3\x61\x5c\x7b\xca\x79\xe4\xf0\x41\x3c\x5a\x57\xa9\xeb\xcc\x1c\x76\x3b\x3a\xc2\x90\xc9\x23\x02\xb3\x0b\xbf\xeb\x99\x38\x59\x45\xae\x2f\xcf\xcb\x33\x3c\xa1\x28\xc3\x92\xaa\x8a\xea\x79\xf3\x87\x64\xca\xf5\x54\x94\xe8\xee\x03\x29\x52\x83\x69\x22\x95\x66\xb7\xd6\x03\xba\x45\xb0\xb6\xfa\x28\x2b\x7c\x81\xa3\x77\xde\x44\x95\xa6\x69\xbd\x5f\x6d\xa2\x62\xbe\xd7\x7c\x67\x54\x2f\x41\xe3\xde\x20\x25\x5c\x93\x3b\x2c\x3b\x10\xf6\x45\x5f\x06\x51\xcc\xc3\x48\x07\x29\x75\x20\x06\x76\x6a\xbf\x92\x52\x5e\x6a\xe1\x1d\xa7\x5d\xe8\xab\x49\x96\x1e\x0d\x53\x43\xf6\x70\x1d\x4a\x6d\x86\x4f\xa0\xca\x93\xae\x62\x48\xe7\x0d\xbd\x7e\x63\x29\x15\x6b\xe3\xfc\xfc\xfc\x70\xb8\x84\xff\x2e\xac\xd7\x51\x2b\x29\xe5\x4e\xac\x57\x73\x14\x54\x9f\x36\x94\x38\x72\xfd\x7d\x32\x33\x27\x4f\xf8\xee\x6f\x9d\x7e\x96\x02\xd3\x5c\xf8\x1b\xc9\xc7\x40\xd0\x02\x60\x08\x4f\x43\x44\x6b\x57\x6c\x8a\x38\x36\x0a\xc7\x86\x70\x6c\x8a\x38\x36\x9e\x86\x00\x1c\xe8\xac\x89\x27\x13\x59\xad\xe0\x92\x92\xb4\x27\xe1\xf2\xbc\x2b\xcf\xcf\xd9\xff\x25\x1e\xd7\x1b\x83\xaa\xa4\xba\x04\x47\x3e\x93\x8c\x3e\x54\x98\x34\xfe\x90\x51\xa8\x81\x19\x1f\xa8\xa9\x3f\xe2\xc1\xee\x2c\xd3\x70\xba\xe3\x8a\x48\xce\xee\x71\x37\xa4\x01\x19\xe9\xd8\x27\x18\xff\xbe\x60\xe6\x45\x30\x00\x5e\x04\xb8\xaf\x29\x01\xc1\x40\x86\x06\xc4\x93\xa0\xf0\x6b\xe0\xfa\x08\xd7\x64\xad\xae\x56\x0c\x9a\x74\x3f\xc0\x2a\xf2\xa7\x49\xc4\x02\x25\x06\xae\xcc\xc7\x1e\x6d\xca\xe4\x1c\x52\x22\x5b\xcc\x06\xe5\x80\x03\x90\x2c\xd1\xcf\x74\x06\xd0\xb0\x2d\x32\xdb\x13\xcf\x23\x3f\x05\xe5\x24\xe8\x75\x0d\xe9\x12\x0e\x25\x95\x8f\x41\x06\x22\x6d\x86\x01\xad\x9f\xdd\x85\x76\xdf\xe9\xdc\xbd\x00\xd3\xae\x73\x17\xde\xd9\xc5\xa5\x01\x94\xc3\x0e\xb3\xca\xa5\xff\x0e\x2a\xe3\x3e\x72\x01\x81\x71\xda\x15\x84\x21\xc5\x11\xd3\xc6\xa1\x5d\xdb\xce\x18\x0a\xba\xb3\x22\x0e\x0e\x5e\x54\xa2\x1f\x03\xfa\xce\x38\x1c\x5b\xe0\x50\x01\xa0\xa9\x43\xe8\xbb\x4e\xda\xbc\x48\x83\xa7\x27\xa8\x59\x98\x36\x0f\x0e\xa0\xed\x5e\x41\x3b\x77\xab\xba\x7b\x78\x47\x4e\xa9\x4b\xd2\xc2\x31\x72\x3b\xa9\xd7\xdd\x76\xa1\xb6\xa6\xd9\xfa\xd0\x56\xaa\x75\xf9\x59\xe9\xc1\x3a\x74\x50\x68\x3d\xea\x47\x47\x4d\x5c\xab\x72\x50\x64\xbd\x8d\x8a\x58\xc2\x00\x68\x9f\x87\xc3\xb6\xb3\x96\x9e\x06\xd0\x5b\x43\x5c\x25\x82\x1e\x87\x7e\x03\xd4\x63\x36\xd8\xcd\x06\x4b\x88\x5e\x5a\xae\x06\x00\xf8\x10\xfe\x79\x00\x0c\xbf\x1b\xd7\xe5\xbb\x34\x4b\x7c\x0c\x65\x57\x97\xf8\xd3\x8c\x56\xcb\x4b\x97\xd4\x55\x97\xb1\x6f\x5a\x59\x3a\xd1\x0a\x93\xb4\x56\x19\x86\xa6\x18\x89\x68\x18\x4f\xe2\x55\xfc\x45\x0e\x9f\x28\x46\x23\xda\x2a\x8c\x66\x8b\x92\x4f\xcd\x8b\x0d\x6e\x79\xee\xb6\xce\x58\xc1\x85\xb4\x84\xc5\xdf\x3c\x7e\xa2\xd7\x08\x15\xc5\x6e\xf6\x1a\xcb\x43\x5c\xec\x52\x9a\x4a\x59\x3d\x68\x9d\x89\x82\x61\xa7\xec\xb5\x2a\x02\x6d\x0b\x8e\xe9\xac\xaa\x94\xad\x2d\x78\xec\xe8\xd3\x28\x01\x6d\x2d\x4a\x87\x11\x72\x5a\xd7\xc7\xd5\xc9\xf5\xd3\xd3\x70\x23\x15\xe8\x10\x4f\x3c\xc8\x5c\x83\xcc\x5b\xdb\xc3\xe8\x30\x1c\x46\x62\xd8\x87\x9f\x7e\x87\x27\xa0\x8d\x13\x0f\x70\xd4\x87\x91\x7b\xe4\x34\x0f\x87\x20\xcd\x1b\x6b\x73\xda\xda\x8b\x81\xc4\xe8\x10\x40\x3d\x80\xa9\x63\x96\x10\x3e\x0e\x29\x2b\xa4\xb8\x2a\x7b\x07\xe4\x10\xa8\x5e\x7b\x54\x2e\x16\x8b\x22\x0c\x31\x1b\x8c\xd9\xd4\xf1\x91\x6c\xe9\x1b\x5a\x02\x0e\xfb\x0a\xa2\x8c\x31\x83\xc9\x7d\x0a\x1d\xff\x2f\x9a\x8f\xd6\x99\x2c\x39\x77\xd5\x86\x66\xf9\xc2\x13\x8b\x04\x60\x2f\x65\xbc\xc4\xff\xe7\x22\x7e\xdd\xce\x72\xd1\xc2\x6c\x55\x89\x2c\x7e\xe8\x77\x54\xc2\x7a\x2b\x7c\xf7\x28\x27\x1e\x3b\xca\xfc\x8d\xa6\xc3\xe5\x78\xad\x49\xb2\xdb\xae\xea\x9c\x32\xc3\x6f\x3b\x37\xae\x40\x9f\x99\x32\x57\x93\x84\xc6\xe5\x34\x5a\xfc\x14\x0d\x7f\x80\x59\x89\x24\x4a\xae\x20\xae\xdb\xb2\xdf\x6e\xda\xb2\xb7\x0e\x65\xd4\x10\xbb\x85\xfc\xdc\x94\xaa\x33\x83\x57\xce\x87\xa2\xe1\x50\x4f\xe9\xd6\xd2\x37\x8a\x27\x63\xf1\x72\xc9\x46\x7a\x50\xd1\x71\xe8\x5d\x5b\x02\x0e\xd7\x75\x15\x1b\xf4\xea\x56\xa6\xa1\xcc\x05\x82\x6e\xb2\xe9\xcf\xa6\x3e\x8b\xeb\x83\x95\x07\xdc\x03\x5d\x34\x5c\xc3\x60\x00\x78\x7d\x57\x40\xe4\x06\x22\x41\x1b\x0d\x37\x18\xb9\xc1\x48\xe5\xe3\x17\xab\xca\xb5\xcc\x57\x90\xf6\x90\xa5\x17\xf5\xe1\x7c\x1a\x25\x33\xf5\x4c\x19\xee\x44\x72\x0c\x12\x0f\x46\xfa\x42\x05\xf9\xa7\xe0\x75\xfa\x0b\x84\xb9\xa0\x8c\x5d\xde\xdd\xa4\x5c\xbd\xf6\x15\xfe\xb0\xaf\xac\x5e\xb1\x70\x5e\x67\xa2\x4f\x73\xca\x91\xfc\xea\xd3\x62\x16\xd3\xd6\x2d\xc4\x38\x34\xea\xda\xe4\x5b\x20\x4e\x6e\x51\x8d\x92\xae\xfb\x09\xbe\xd7\x12\x2d\x65\x45\x05\xaf\xa8\x65\x36\x9e\xed\xd7\x80\x98\x11\xf7\x61\x06\xc0\x31\x2c\x91\x5f\xec\xc7\x24\xc9\x3c\x28\x27\x17\xe4\xd4\x6a\xdc\xae\xa7\xb9\x12\xe7\x1e\x42\xf9\xb7\xaf\x88\xde\x19\x5a\x11\x92\x56\xfc\x56\x73\x02\x91\xe0\x05\x94\x24\x08\x73\x0d\x81\x57\x1c\x25\x51\x09\x7b\x13\x51\x21\xd3\x46\x6b\xff\x62\x1d\x90\xd2\x0f\x13\x1f\x11\xa1\xf7\x78\x00\x44\x3f\xae\x21\xbe\xe8\xe5\x87\xe8\x2a\x1d\x63\x5c\x61\x90\x85\x58\x7e\x63\x34\x99\xc3\x68\x86\x2f\x58\x19\xc4\x9c\x42\x6b\x9d\xf4\x60\x15\x25\x54\x54\xe6\x72\x15\x2f\x1c\x7a\x0d\x4e\x37\x34\x04\xba\x8f\x84\xb7\x5d\xb5\x9b\x2f\x4b\x3d\xa2\x8c\xf4\x00\xdd\x56\x60\x79\x95\x19\x98\x18\x1b\x7e\xdb\xce\xd0\xf1\x5a\x3e\xb9\x59\x42\x26\x1e\x83\xfe\x2c\x29\xb9\xea\x81\x14\xc0\x1f\x5c\x21\x1f\x9b\x69\x4f\xa6\x89\x18\x97\xd2\x75\x75\xb6\x99\x25\x2b\xc7\xbd\x57\x5a\xb4\xb0\x7d\xe8\x5b\xb5\x70\x4a\x06\xa5\xdb\x50\x07\xdf\x5d\xc6\x14\x32\xbe\x42\xaa\x4b\xcb\xfe\xe1\xb8\xe3\xde\xf3\x7e\x79\xa9\x08\xdf\x91\xb3\x65\xf9\xdd\xc3\x8b\x3c\x0c\x5c\x14\x65\x82\x94\x9f\x3d\xb7\x54\x9a\x79\x1f\x02\xc6\xd4\x3e\x59\xd1\x8a\x79\x6b\x11\x80\xcd\x52\x0f\xf4\x99\x52\x40\xe0\xdc\xe3\x9f\x0a\x99\x2f\xe3\xbd\xcd\x79\xba\x42\x60\xbf\xfe\x19\xcb\xf7\x26\xd8\x7d\x19\xe5\xde\x8b\xfb\x17\xcd\x6e\x49\xa3\xb7\x0b\xca\x22\xdb\xf2\x21\x95\xdf\x2d\x3d\xaf\xd2\x2e\x3f\x21\xa3\xaa\xce\xaf\x7b\x87\x9f\x26\x5c\x10\xb0\x32\x31\xb4\xcc\x9a\xcd\x94\x3d\x56\x24\x5f\x3d\x31\xfc\x65\x26\x99\x7e\xc2\xc4\x40\x04\x4d\x38\x38\xc4\x65\x75\x32\x2a\x38\xff\x9a\x8c\xd9\x3d\x25\x60\x08\x6f\xb9\xa8\x86\x14\xd8\x92\x3e\x59\x7c\xfe\x59\xc2\xf2\x73\x80\xcc\x25\x57\xe8\x96\x04\xc5\xce\x4a\x44\x58\x1a\x7d\x3f\xc4\x0c\xba\xb6\x71\x51\x54\x06\x15\x9d\xb3\x2d\xc3\x67\x0f\x20\x6b\xb7\x91\xe9\x32\xc5\x93\xba\xba\xe6\x24\x12\xfb\xd1\xc7\xa0\x19\xfa\x28\x4a\xa1\xb2\x25\x6d\x2f\x84\x76\x86\x0c\x5a\xfb\x55\x3b\x89\x7c\x95\x0c\xc6\xa9\x41\x9b\x79\x2a\xda\xee\x7c\x3f\x23\x9c\x12\xd0\xa9\x6b\x67\xff\x16\xcf\xa2\x5a\xe2\x30\x15\xf4\xee\xd6\x6a\x17\x2a\xce\xa3\xf1\xa9\x1c\x0a\x2d\xea\xe3\x3d\x88\xfa\xc1\x1a\x0b\xa7\x96\xb0\xe8\x46\x98\x2f\x36\x61\xf1\x0a\xd9\x27\xea\x95\xad\x85\xb8\x4a\x37\xc9\x63\x05\x7c\xaa\x94\xa3\x4a\x8e\x14\xd8\xc9\xa2\x46\xe5\xd5\xe4\x2f\x49\x0f\x04\xac\xe2\xf1\xa8\x01\x12\x50\xab\x2e\x36\xcb\x80\x42\xfd\x6c\x5b\xa1\x38\xe6\x96\x34\x0b\x9b\x5d\x1a\x21\x98\x97\xa5\x36\x53\x65\x86\x47\xbe\xe1\x37\x85\xb1\x6a\x1a\x06\x3e\xaf\x0a\xb1\x77\xb2\xb0\xdc\xae\x14\xe9\x22\x9a\x19\x07\x66\x75\xed\xda\x4c\xd0\x9b\xb8\xfa\x86\x4c\x20\x8f\xee\xf1\xb8\xaf\x37\x50\x31\xf7\xd1\x54\xef\x9c\x06\x78\xf5\x34\x5e\x2e\xc3\xe9\x11\xa6\xd0\x38\xaf\x96\x43\x83\x93\x8b\x10\x92\xba\x18\x77\x08\xc4\xb5\x1b\xad\x4c\xcc\x49\x1b\x2f\xad\x60\x04\x50\xcf\x51\x4d\xd7\x38\xad\x0b\x8d\x15\xa1\xe3\x2c\x6b\xc2\xf8\xb0\x0b\x2d\x3a\x75\xa4\x05\xe9\xc9\x07\x73\x55\x8e\x66\x2f\xb4\x82\x3b\xb9\x9d\x96\xb5\x35\xeb\x26\xf9\x38\xd3\xb0\x25\x9e\x6f\xa5\xdd\x4d\x5a\xd1\x35\xf9\xce\x00\xa9\xd2\xe7\x4b\x30\x9e\x26\x1a\x9c\xdd\xe1\x9f\x2e\xff\x40\x7f\x58\xd0\xca\x04\x60\xe2\x98\xeb\x65\x6c\xe6\xda\xd6\x4b\x45\xe2\x46\xdc\xe2\x9a\xad\xde\x6b\xce\xa5\x3c\x3d\xd5\x1a\x74\x37\xde\x26\x88\x6b\xf0\x93\x7a\x44\xd9\xb9\xe3\xae\xed\x8a\xbb\x9e\x39\xc3\xea\xb6\x6b\xa2\x98\xd5\x64\x62\xcb\x06\x3a\x80\x57\x1b\xed\xe8\x81\x26\x03\x6d\x3d\x28\x3a\xeb\x65\xc2\xca\x10\x46\x56\xbd\x86\x1f\x54\x63\xce\xd7\x62\xb5\x59\xc4\xb2\x22\xd2\x11\xcf\xf3\xb4\xeb\x53\xed\x90\x17\xaf\xce\x97\x16\x71\x99\xdc\xcc\x92\x51\x32\x88\x60\x46\xc5\x97\x18\xfa\xa9\xb3\xa8\x57\x62\xd7\x2b\x38\x1a\x58\xcd\x6d\xd4\x73\x8d\x3a\x0a\x2f\x57\xb8\xae\xf7\x85\xe7\xd4\xe2\xda\x8b\x30\x44\x3a\xdc\xf6\xa2\xde\x3c\x74\x6a\x5f\xd6\x42\x19\x51\xc6\x02\x60\x97\xd4\x93\xfd\x28\x85\xb9\xf5\x3c\x4d\x56\xc9\x07\x65\xe7\x58\xb6\x1e\x42\x9a\x43\xc4\x0a\xae\xab\x99\x0e\x53\x9b\xae\xdf\x5e\xbb\xed\xba\x8e\x5a\x5f\xf8\x10\x55\x5f\xdb\x4f\x23\x62\x29\x16\x21\xa8\x72\xcc\x10\x9b\xc5\x8a\x69\x44\x94\x46\xaa\x63\x00\x67\xb5\x79\x25\x47\x1b\x26\xd8\xdd\xee\x34\xa9\xec\xb2\x1f\x24\x27\x24\x10\x65\xfe\x0c\xfb\x4a\x55\x22\x5c\x43\x4b\x5d\xc0\x84\x44\x62\x65\x1c\x4e\xa9\xfd\xc5\xa7\xaf\x80\x62\x57\x1a\x8f\x79\x33\x0c\x6b\x6d\x8a\xbf\xde\x51\x3c\x41\x7a\xd7\xb9\x52\xa5\x2d\x5e\x28\x05\xa1\xcb\x8d\x01\xb9\x9e\x04\x71\xc3\xb0\x6c\xa0\xd3\xb8\x44\xa6\xd5\xda\xb6\x68\xe1\xb0\xf9\x43\x7c\x13\x61\xaa\xee\x23\x59\xc2\xa8\x00\x35\x73\xa5\xe9\xec\x90\x30\x43\x73\x28\x3b\xba\xdc\x82\x92\x2e\xea\x76\x8e\x79\x9c\x05\xe6\x85\xf7\xc6\x95\xa7\xf8\x60\x46\x41\x98\x34\x5a\xfa\x03\x05\xe6\xde\x45\xc7\x00\x66\x08\x22\x75\x82\x37\xed\x43\xe4\xd4\x97\x41\xb7\x49\x2c\xc3\xe5\xdb\x24\xfd\x36\x99\x25\x60\x9e\xde\xd5\x13\x97\xce\x34\x2b\x46\x48\xa7\x58\x77\x72\x6b\xc0\x9c\x18\x47\x77\x03\xe3\xce\x18\xe2\x88\x32\x9e\xf1\xa1\xe0\x25\xee\xe1\xd8\xed\x14\x22\xdd\xad\xf6\x69\x50\x4c\x43\xdc\xaf\xee\x3a\x16\xfa\x19\x4c\xf5\xc6\x17\x7e\x67\x5c\xaf\x97\x17\xa0\x36\x2d\x28\xf1\x2a\xe9\xbd\xba\x27\x02\xe5\xee\x11\x67\x91\x4b\xe3\x0c\x82\x73\xcf\x8b\x0f\x9d\x3b\x40\xc8\x31\x12\x28\x95\x1e\x12\xee\x74\xf7\xa2\xf8\x5d\xd6\xea\x4c\x0f\x70\xe8\x33\x27\x2f\xb7\x6e\xc1\x64\x9c\xdf\x30\x86\x4e\xf1\x8a\xb0\x1c\xdd\xc2\x22\x70\xbb\xa6\xca\x03\x2d\x88\x3a\x6f\x3e\xfa\x82\x81\xf1\xe5\x7d\x95\x2b\x3b\x56\x4a\x9f\x4f\xb1\x5e\x38\x00\x8d\xdb\x08\xc4\xec\xc8\x12\x3b\x98\xdf\x2b\xcf\x60\xa3\x50\xcb\xbb\x13\x87\x41\x5c\x0f\x9a\xe6\xc4\x99\xdb\x86\xb8\xba\x15\x49\x92\x55\x9c\xb3\x5b\x07\xeb\x8e\xb0\x79\x46\x24\x68\x43\xd7\x8b\x5d\xf7\x55\x38\x96\x15\x84\x88\x36\x5e\x59\xdc\xd7\xca\xd6\xba\x9c\x40\xf1\x70\x7a\x99\x4a\xdf\xe6\x2d\x97\x32\x9b\xba\x74\x71\x46\x29\x48\x30\xa1\x16\xf3\x19\x1e\x05\x2d\x8c\x12\x15\xba\x18\xf2\x2e\x48\x9f\xd2\x80\x0a\x81\xd0\x46\xfb\x13\x60\x56\x28\x51\x9d\x3c\xf4\x0b\xc9\xc1\x91\x06\xd8\x3d\x47\xce\x28\xf4\xfe\x6f\xa3\xd1\xf7\x56\xe0\x58\xb3\x2a\x0d\xfe\x5f\x6b\x02\x68\x33\x29\x3f\x81\xc9\x4e\x86\x34\x2a\xd5\x40\xfb\x8e\x92\x3b\x84\x20\x5c\x3f\x2f\x06\xd5\x03\x5c\xb6\x15\x54\x86\x7d\x7b\x91\x65\x07\xc8\x5e\x94\x97\xf6\x5f\xd1\x81\xa8\x92\x45\x37\x0f\x6b\xcb\x03\xc7\xba\x6b\x6c\x9d\xfa\x5a\xc4\xf2\x3a\x2d\x19\x4a\x05\xd7\x0f\x8c\x98\xdf\xa2\x9c\x64\xe6\xd7\xcb\x1d\x9d\x92\x6d\x46\xc7\xa1\xab\x03\x8d\x9b\x18\x3b\x06\x18\xf8\x3c\x4f\x46\xd7\xa9\x84\xa0\x01\x3a\x93\x21\x52\x84\x90\xb5\xe7\x21\x05\x3a\x92\x5b\x0f\xdc\x2f\xed\x45\x3d\x6b\x77\x01\xe7\x59\xa9\xa3\x76\x21\xac\x15\x67\x73\xe6\xda\x5e\x1b\xcc\x9d\x3d\x4f\xac\x05\x6a\xc0\xe0\xd1\x1c\x31\xd9\xb2\xf2\x20\x8a\xe4\x12\x02\xfd\xfd\x43\x34\x1b\x76\x4a\xa4\x36\x23\x86\x3b\x87\x1c\x02\xef\xc8\x5c\xb8\x92\x4c\x3e\xa2\xa4\xb7\xc3\xef\xa3\x85\xbe\x9c\xb4\x4e\x84\xdc\x48\x5e\xe7\x76\x90\x99\x4f\xb7\x51\xea\xac\x13\xf2\xc5\x02\xfc\xb4\x78\x97\x64\x99\x97\xb8\x59\x25\x76\xa5\x18\xde\x93\xb3\x4b\x96\x24\x19\x1b\xb9\xbf\x66\x2d\x4e\xb1\x08\xec\x50\xc6\x17\x3e\xc2\x28\x29\x97\x44\xa2\xb6\xce\xb9\x55\xfd\xa5\xb8\x64\xf7\x13\xdf\x99\xb3\x36\x51\xf5\x0e\x5c\xb9\xfb\x0e\x4e\x0d\xe5\xb2\x04\x6f\x66\xad\xf5\x3e\xd6\x9a\x7c\xcf\xe1\xa2\x83\x83\xe1\x3a\xdf\xab\x3b\x32\xd3\x9d\x40\xe4\xb6\x56\x5c\xb5\x91\xa7\xd9\xc6\xd5\x63\x31\xcb\x00\x83\x71\xe0\x28\xa9\x99\x2f\xdc\xa3\x66\xdb\x92\x21\x89\xe5\xa8\xc9\x42\xb9\x93\x39\x5c\xe9\x32\x16\x65\x78\x83\xb9\xcb\x58\x23\xe6\xf7\xab\x78\xf9\xd3\x9e\x7c\x2a\xb8\x3a\x79\xd5\x82\x74\x1b\x45\xa8\x39\x40\xcb\x03\xbc\xd7\x40\xac\x7c\x25\x39\xcb\x4c\x96\x29\x75\x8b\xd9\x75\x19\x57\xca\xf4\x2c\xa7\x65\x21\x5e\xf3\x30\x43\x7e\x19\xd7\x2d\xb6\xda\xb0\x8a\xb3\x5c\xe4\xc1\x01\x2b\x88\xdc\xce\x09\xb1\x9c\xb2\x3a\x81\x2a\xd3\x2d\xb4\x00\xb1\xf6\xd9\x06\xa0\xf5\xbc\xff\x7a\xad\x60\x4d\x3b\x3e\xa3\x41\x68\xed\x6d\xbe\x0c\x4d\xce\x7a\x79\x46\xd7\x5e\x99\x2b\x69\x3f\xcb\x97\x05\x61\x3c\x42\x3f\x95\xfb\xb6\xa0\x95\x39\xdf\x98\xc5\xd6\x34\xed\xb4\x57\x9f\x2a\x19\xa1\x75\x5a\x06\x38\xff\x9e\x7a\x61\x40\x97\x53\x42\xa5\x4c\xf9\x42\xf8\x7e\xa6\x40\xf9\xb8\x9a\xb5\x20\xd5\x5c\xb3\x64\x58\x7e\x7f\x1f\xcd\x56\xc9\x24\xce\xe4\xaf\xdc\x3d\x1b\x83\x12\x7a\x1f\xda\x03\x2a\x8d\x3b\x78\x2b\x21\xbd\x9d\x4f\x86\x29\x5f\x27\x19\xbf\x7a\xdf\x71\x4d\xe4\xd5\x18\x66\x68\x38\xa5\xc9\x17\x36\x3e\x7a\x9f\x1d\x64\xaa\xd7\x4b\xd8\x63\xcc\x3a\xf4\xd6\x6e\x97\x1f\x69\xe4\x41\xe0\xca\x6c\x28\x9a\x02\xc5\xda\xed\xd1\x40\x6c\xa2\x9e\x1d\x7d\x9f\xdf\x8a\x1a\x25\x13\x3c\xd7\x5f\x9c\x1f\xbd\x60\xe2\x86\xee\xd6\x55\x3e\xa0\x1b\x51\x3a\x88\x67\x52\x67\xfc\x03\xb6\xad\x14\x67\xd3\x12\x51\x31\x4c\x30\x9b\x48\x38\xd3\xc8\x4b\xa5\xd9\xb9\xdb\xc8\x26\x26\xeb\xe0\xc7\x11\xa4\x08\xff\x62\xd3\xbd\xf2\x8f\x7c\x01\xff\x7a\xed\xab\xcd\x85\xdf\xb5\x1a\x78\x83\xd7\x54\xcc\xb9\x87\xcd\x2b\x93\xa6\xaa\x60\x43\x6b\xd8\xfc\x39\x91\xde\xbe\x52\x5f\x2e\xb6\xbb\x37\x67\x38\xcf\x47\x90\x2a\x1f\x1d\xc3\xe6\x25\xbd\xdc\x00\x2d\xf3\x16\x90\xd8\xbb\x0f\xe3\xf5\xa1\xb3\xc6\x17\xcf\xdc\xcc\x83\xfe\xa6\xdf\x28\x57\x2b\xeb\x4c\xbf\x39\xa2\x37\xe2\x7d\x3c\x18\x62\x47\x83\xe1\xc6\x1d\x81\x7a\xdb\x5a\x24\x9f\x2f\xb7\x6b\x1f\x3a\x8b\x7c\x33\x06\x3e\xd6\x86\xcb\xb6\x14\x5d\x11\x33\x7a\xbf\x83\x48\x7e\x86\x90\x6d\x42\x14\x33\x90\xb0\xf6\xe6\x68\xbc\xf6\x80\xb4\xab\x8d\xd8\x78\x01\x04\x3e\x51\x32\x72\xad\xbc\x5b\x30\xb4\x68\x56\x29\xc1\xbc\x68\xe0\x8b\x2a\xdd\xbc\xfa\x51\xe7\x19\xdc\x5e\x9b\x15\xd4\x76\x67\xd3\xed\x5a\x1e\x95\xb0\xd7\xe5\x93\xf4\x5c\x1b\xed\x42\xc4\xa0\xd7\xa5\x66\xf3\x67\xb4\x8f\xec\xaf\xd4\xdd\xd5\xb9\x90\xcd\xfe\x9d\xb6\x82\xcf\xdb\xfc\x51\x04\x9d\x21\x91\xa7\x70\x9c\xc2\x12\x8e\x4e\xd1\x8d\xe2\xad\x15\xbf\x55\x9a\x5a\x76\x51\x61\xa1\x13\x24\x7f\x75\xf8\x13\xa4\x3e\xb3\xae\xa2\x10\xb8\x1a\xb5\x69\x2b\x8d\xfc\xb3\xd7\x54\x32\x18\x7e\xd3\x65\x15\x8d\xf9\xb9\x16\xcb\x37\x80\xc9\x9a\x6d\xad\x0f\x37\xd7\xd1\x72\xf0\x7a\x36\x8b\x97\x6f\xa3\x61\x72\x9f\x66\x6e\xe5\x26\x26\xbe\x2c\xdb\x8f\x68\x35\x96\x64\x9b\x9b\xf8\xb2\x6c\x97\x68\x1c\x7e\x85\xce\xdf\x33\xb9\x52\x1d\x5d\x96\xe9\x8f\xb3\x61\x31\x4b\x2c\x23\x0b\x19\x90\x95\xce\x62\x39\xc7\x5b\xf9\x10\x6d\x6f\x34\x61\x8a\xbc\x63\xad\xb5\x44\x7c\x33\xa5\x63\x87\x8f\xf2\x4b\x79\x5d\xfa\xbe\x66\x1f\x24\xb0\x10\x3a\xec\x57\xc5\x15\xd0\x07\x71\x37\x0d\x37\x63\xe5\x34\x7e\x28\x14\x92\x50\xb9\x83\x4f\xf9\x04\x19\xcd\xed\xad\xeb\xe0\xf8\x3a\x17\x6e\x9d\x22\x19\x78\x0a\xce\xc5\x17\xba\xac\x88\x8d\xab\x8f\x90\xc7\x23\x20\x7b\xd8\x80\xa6\x9d\xd0\x93\x92\x62\xc8\x57\xc2\x13\xc0\xeb\x76\x6d\x3f\x32\x57\xde\x68\x6d\xc3\x41\xba\xf0\x46\x9b\x5c\x54\xcf\x6d\x67\xfc\xcb\xe0\x5e\xb9\x62\x82\xa1\x59\x9b\x78\x39\x58\x03\xaa\xf9\xa5\x7c\x68\xaa\xb0\xda\xa0\x65\xdf\xf7\x7c\x94\x76\x28\x3d\xb7\x13\xb6\x3f\x8b\x4d\x36\xe2\x2f\x42\xd6\x92\xdd\xcf\xdd\x67\xce\x70\x84\x56\xbb\xbe\xe1\x55\x62\x2b\xf5\x7f\xc6\x1b\xfb\x90\x66\x63\x1c\x6f\x54\xc3\x84\x8d\x33\x7b\x67\xa8\xb1\xde\x4f\xfd\xae\x43\xde\xe2\x72\xdb\xeb\x2d\x7d\x34\x36\xfb\x65\xdc\xe8\x8c\x1b\x99\x51\xd5\x6a\xbf\xf1\x43\x02\x2b\x24\x32\x2c\x51\x95\x1e\x21\xda\x85\x2e\xc7\x23\xb3\x7d\x11\xaa\xed\x8b\xeb\xae\x8d\xf4\xba\xed\x54\x30\xfd\x75\xc6\x79\x32\x2e\xea\x5d\xbb\x4f\x4f\x85\x56\x71\x89\xf7\x4c\x7b\xb6\x70\x59\x05\xd5\x2c\x7b\x91\xaf\x80\x15\x37\x64\x98\x51\x95\xf6\xf9\x37\x6a\x43\x80\x7a\xe7\x63\x56\x7c\x59\x26\xdf\xe4\x8f\x01\x64\xb2\xd2\x8b\x1d\xc3\x3c\x82\xca\x12\xbc\xda\x2f\xb5\x52\x6c\x74\x34\x47\x61\x31\xf7\x44\xd0\x9b\x7a\xa6\x33\x89\x85\x0c\xa3\xbd\xb7\x00\xe3\x34\xbc\x5a\xe0\x27\x3a\x25\x5b\xc0\xcc\xbf\xa7\x54\xc0\x82\x96\x69\x49\x2b\x7d\x57\x13\x0e\x02\x79\x8e\xce\x9c\xf4\x5c\x9c\x7a\x1e\x35\x45\xed\x5f\x39\xa3\xb9\x98\x8d\x97\xd6\xb2\xb9\x17\xd2\x8b\x38\xc4\xe9\x83\x14\x55\x95\xf8\x03\xbd\xc5\xf9\x7b\x55\xe5\x5f\xb1\x2a\x76\x25\xc0\x3c\xd1\x14\x6a\xed\xb3\x17\xa1\x7c\x77\xfb\xf7\x64\x79\x96\xd9\x45\x4e\x3f\x4f\xe7\xd7\x11\xcf\xf9\x7f\x5c\xc4\x6a\x08\xd1\x23\xc8\x63\x99\xae\x7d\x75\xdc\xad\x94\x3d\xa9\xc1\xd1\x73\xa3\x05\xf3\x5d\xbc\x9c\xe2\xb6\xb1\xf2\x12\x46\x5b\xa9\x81\xc8\x60\x45\x6f\x9b\x25\x54\xfd\x8c\xc6\x14\x40\xe5\x29\xdb\x5d\x97\x4c\x8f\x79\xa6\x36\xad\xe7\x6b\xe3\x97\xd6\xc6\xb1\x87\x38\x0d\xa9\x86\xaa\xdd\xb5\xb9\x92\xf0\x59\x16\x34\x7b\x3d\x30\xa2\x66\x03\x30\xb3\x24\xe9\x57\x9a\x9f\x3d\x77\xcf\xba\xff\xce\xb5\x96\xd8\x7f\x75\x53\xe5\xf0\xad\x64\x66\xda\xb8\x50\x01\x45\x6c\xf0\xf4\x94\xa1\xfe\x45\x98\x83\x00\x65\xb2\x33\xdd\x6b\xba\xcf\x69\x4d\x5a\xdf\x44\x57\x4f\xe1\x6e\x54\xdc\x3b\x6b\xd0\xcd\x7c\xbb\xcb\x86\xba\xa1\xc4\xca\xd7\x79\xc8\x15\x67\xb8\xf2\xc5\x22\x09\x03\xe5\x30\x8b\x56\x60\x81\xa6\xb0\xf6\xbf\x6a\x1e\x29\x4d\x74\xf3\x8e\x7e\x64\x8f\x5a\xa0\xbc\x05\x45\x06\x32\x32\xd0\x91\xa4\x5d\xf9\x03\x0a\x31\xa5\x63\x20\xc1\xf3\x84\x39\x42\x2f\x02\xf7\x71\x65\x28\x09\x2c\x22\x17\x09\xe6\xf1\x3c\x21\xe9\xf8\x1a\x8b\x44\x02\x3c\x22\x43\x11\x81\xa5\x7b\x44\x83\x26\x0b\x68\x5d\x59\x10\x48\xe6\x4a\xa7\x67\x28\xec\x18\x6d\xd7\xec\x24\xaf\x72\xc4\x75\xa8\x74\x3c\x89\x91\x21\xca\xa2\x37\xe9\x29\xea\x2e\x3f\xb1\xe4\xad\xe4\xb3\x3c\xa2\x6f\x17\xd0\xc9\x31\xde\x03\x1e\xe7\xf9\x4e\x71\x86\xed\x13\x8d\x7b\x42\xc8\x2d\xbd\xba\xb3\x1b\x56\xf5\x06\x3d\x0a\x2c\x7c\xdd\x62\x68\x8a\x47\x21\xfa\x92\x97\x40\x30\x66\x2e\x02\x5b\xbc\x9a\x56\x6b\xa3\x5f\xa4\xdc\xf0\xa1\x47\x07\x90\x8a\x80\xb2\x36\xad\x3c\xc8\x58\xc5\x7e\xb6\xc7\xa3\x43\x67\xd1\xa4\xcb\x5e\xd4\xde\x82\xc3\x01\x85\x81\xa7\xd6\xe9\x3e\x99\xaf\xb4\xaa\x7f\x88\xd2\x24\xd5\xa3\x1b\x9d\x06\xca\xa8\x98\xfd\xfa\x5c\x49\x75\x50\xa0\x4d\xe5\xd7\x50\xa9\x44\x3e\x4f\x0c\x1f\x28\xc8\xeb\x90\x56\x9a\xe8\x3f\xc7\x00\x07\x34\x08\x42\xfa\x26\xbc\xda\xd0\xb3\x7a\x1b\xec\x7a\xa8\x3d\xe5\xa0\x0a\xf0\x38\xa4\x42\x34\xd8\x5c\xb6\x12\xfb\x66\xbe\x3a\x76\xf2\x75\xfb\x43\xfc\x31\x89\x97\x2d\x28\xcf\xa5\x6c\x7b\xc3\x6f\x5c\x96\xff\x92\x11\x82\x2e\x14\x52\x7b\xa1\x93\x24\x43\x3b\xca\xfb\x9a\x7d\x68\xe0\x74\x47\x3e\xac\x41\x15\x87\xe0\xc6\xa4\x6c\x74\x0a\xde\x04\x2b\x27\x81\x14\x0c\x20\x41\x4a\x72\xf3\x25\xf6\xe1\x68\x99\x11\xc0\x87\x45\xb2\x8f\xe5\x45\xf8\x2d\x2b\xa1\xa4\xc9\x8f\x9f\x6f\x72\x2d\xfe\x89\x6c\x12\x3d\x0b\xcd\x89\x00\xb4\xb1\xcf\x4d\xe9\xb3\xf5\xd3\x2a\xb2\x2b\xc7\x23\x8b\x31\xcc\x7e\x5d\xcb\x4f\x68\x69\xea\xeb\x9f\xd0\xd2\xae\xa8\xd7\x13\xdd\x01\xff\x01\xed\xf9\xdf\xf6\x36\xed\x08\x43\x6e\xf6\xa0\xf9\x4f\x18\x93\x72\xe6\x4f\xc3\x99\x77\x4c\x4d\xc0\x2d\xc0\x0d\x70\x9c\xa9\xdf\x97\xb3\x7d\x9b\x20\xbc\xfa\x3d\x3b\x9a\x66\xff\xb4\x8c\xbc\xdf\xb3\x43\x3d\xcf\x7f\xf2\xd2\x5a\x18\x01\xe8\x94\x69\x98\xb3\x7e\xe9\xd2\xbf\x3e\xd9\xb8\x10\x2b\xd2\x7b\x4a\x09\x2a\xf5\xa7\xc3\x01\xdd\xef\x55\x3a\x05\xd3\xeb\xa0\xdb\x86\x1b\x2b\x0a\xd4\xf9\xc6\x37\x5e\x9a\x48\xdd\xd8\x43\x42\x98\x1c\xcd\x68\x52\x13\x4a\xca\x0e\x79\x02\x67\x86\xa1\x43\x67\xed\x7b\xab\x43\xbc\x9a\x89\x63\x9f\x05\x17\xe4\xe0\x36\x04\xb7\xb1\x1f\xaf\x2e\x1d\x25\x4a\xd9\x44\x4d\xfc\x8c\xaf\x7a\xf8\x6a\xe2\x57\x13\xbf\x5a\xf8\xd5\xea\x55\xcb\xbc\xdd\x68\x6b\xf4\x9c\x68\xe9\xbb\xaf\xf7\x15\xaf\x40\x7c\x92\x34\x06\x50\xce\x27\x80\x37\x3f\x0d\x7b\xf3\xd3\xb0\xb7\x3e\x0d\x7b\x0b\xb0\x57\xcc\x62\x27\xf3\x05\x48\x30\x0c\xa3\x81\x39\x09\x1f\x18\x53\xe1\x08\x43\xda\x90\x28\x45\xc1\x67\x86\xbf\x49\x46\xa3\x78\x19\xa3\xff\xa5\xd2\x09\xf1\x5d\xbe\x43\x80\x12\xba\xea\xe5\xac\xec\xc0\x32\x83\x86\xf4\x7e\x5b\x58\x41\x2b\x89\xfc\x5d\xc7\x9d\xe2\x53\x51\xce\xd0\x73\x86\x65\x90\xca\x5a\x92\x9d\xc2\xa3\x4b\x05\x47\x4d\x25\xc5\x94\x79\x28\xa6\xa5\xd5\xfa\x7e\x3e\x9b\xaf\xe6\xb3\x38\x67\xe8\x99\x5a\x0d\xc9\x2f\x50\x9a\xb1\xf2\xa6\xe1\x3e\x8c\x61\xa5\x5c\xe0\x88\xaa\x53\x59\x55\xcc\xa0\x63\xd5\x45\xe0\x75\x89\xa1\xfb\xea\x3f\xff\xa3\x4b\x55\x99\x52\x74\xe8\xe3\x03\x42\x18\x71\x34\x14\x7d\x19\x09\x9f\x69\x18\x1d\xe2\xf3\x1e\x40\xf3\xc5\x39\x2e\xbe\x86\xad\xc3\xe1\x91\x71\xaf\x01\x74\x11\x96\xf4\x30\x12\x12\x53\x8a\x3e\xce\xe5\x19\x6b\xa5\x66\x80\xc0\x34\x54\x04\xe9\x5d\xce\x3b\x01\x19\x5c\xd2\x53\x99\x24\xda\x0b\x4d\xea\x9c\x04\xd2\x74\x7a\xe8\x04\x1e\x16\x73\x38\xa5\xf7\xc3\xf2\x96\x6b\xfa\xf4\xe4\x13\x19\x87\xf8\xb5\xa7\xa1\xaa\x9a\xaa\x62\x0d\xee\xb7\x9f\x12\x57\xc9\x46\x79\x07\xc1\x5d\x09\x3d\x6f\xb7\x46\x68\x0c\x8b\x25\x48\x51\xd9\x18\xad\x2d\x0e\x8c\xb4\xd4\xfa\x92\xbf\xc9\x65\x94\xfc\x34\x8b\x31\xb8\x3f\x42\x5e\x5d\x85\x02\x0a\x97\x87\xfa\xdd\xac\x48\xae\x5d\xa0\xaa\x5f\x9a\xf7\xa2\xa2\x9c\x09\x59\xb2\x13\x12\x47\xe5\x1b\x1b\x94\xf2\x5b\x6d\x6c\x04\x66\x05\x44\xbc\xb1\x41\xdf\xca\x93\x3a\x85\x1c\xbe\x75\x45\xdd\xe4\xad\xfd\x52\xb3\x77\x46\xd0\xcd\x4d\x71\x77\xc4\x37\x9f\x41\xe5\x4e\x89\x9f\xd9\x2a\xc1\x57\x85\x36\x99\xa8\x0d\x46\xad\x83\x10\x00\xc3\x70\x1d\x74\x8b\x7b\x63\x6b\xf2\x14\xa0\x51\x04\x88\x22\x08\x37\x08\xbf\x29\x83\xdf\x64\xe0\x37\xc1\xbe\xbb\x31\x9a\x21\xdc\x8b\xd6\xa1\x07\xd4\xe7\xb7\x64\x36\x10\xbb\xc9\xc7\xea\x05\xad\xc0\xec\xe5\x04\x25\x9b\x39\xf9\x38\xe8\xbb\x6d\x55\x68\xe5\x7e\x4e\x8e\xcb\x39\x41\xf3\x7f\xe5\xe6\x8e\x9f\xdf\xdd\x41\x37\x13\xb9\xfd\x1e\xf4\x6c\xb1\x09\x7e\xd7\x5d\x9f\xa2\xa4\xda\x20\xe2\x4d\x88\x53\xb0\xe2\xd6\x10\xf6\x9d\xbd\xb7\x86\xd0\x8b\x46\x78\x2d\x30\x8f\xdb\x5e\x07\x5b\xc1\xb9\xfd\xbd\xb3\xeb\xbc\xbe\xca\x1b\xec\x99\xb7\xa4\xdc\x7d\xf7\xa5\x7c\x10\x72\x9d\x7b\xa3\x73\xef\x49\xf5\xc6\x50\xbd\x51\x54\x6f\xf6\xa4\xba\xac\xdc\xcf\xdc\x14\x63\x2c\x7a\x53\x8c\x50\x25\xff\x45\x36\xc5\x76\x8b\xa6\xd2\x9e\x78\xb0\xdb\x96\x57\x1b\x64\x40\xb3\xcb\x2e\x28\xef\x36\x8a\x31\x33\x23\xc9\xed\xb2\x11\x4f\x3e\x73\x97\x8d\x31\xea\x5d\x36\x0c\x16\x06\xa0\xc1\xed\x7c\x39\x2c\xd9\xf3\x5f\x96\x6f\xf7\xe3\x3b\xf2\xf3\x59\x34\xf9\xc9\x8c\x17\x43\xe3\x71\x9a\x1e\x89\xd8\x14\x27\x17\x2a\x17\x8f\xd6\x56\x5e\x7b\xe4\xcb\x5f\x73\x90\x2e\x74\x42\x03\x23\xcf\xbb\x93\x9e\xd4\xf5\x76\x61\xd8\x1e\xf2\x90\x3d\x2c\x1b\xae\xa5\x4e\xb8\xca\x0e\xd6\x99\x31\xba\xb7\x2d\x50\x9c\x6e\xa6\xfd\xf9\xe4\x12\x8f\x2f\x69\xca\x4e\x8f\x2b\xc0\x7e\x06\x31\xd3\x60\x35\x7e\xf3\xb2\xb8\x8d\xc8\xb0\xfc\x7e\xbf\x93\x02\x66\x79\x79\xdb\xf2\xc2\x86\xb1\x47\x7f\xff\x77\xa5\xcb\x6b\xdf\xfb\xa2\xe6\x2d\xbd\xda\x57\xf4\x97\xbf\xbf\xf0\xbf\x08\x44\xf0\x05\xa6\xd4\xab\x93\x96\x85\x9d\x4c\x7a\x8d\x2a\x21\xce\xde\x2c\xe7\xf7\x8b\x54\x24\xc3\xcc\x86\xe8\x7d\x7f\x30\x89\xd2\x54\xa5\x66\xb2\x40\x93\xad\xe6\x74\x2b\x58\x70\x72\x23\x19\x86\xc9\x50\x06\x2a\xca\xb9\x5e\x3d\xc4\xb1\x2e\x6d\x16\x4d\xa5\x77\x76\x41\xf1\xf2\x61\xdd\x61\xa8\x11\xda\x13\xe0\x78\xc2\x4d\x7e\x1d\xa3\x33\x43\x89\xa2\xa4\x6b\x13\x42\x33\xd8\x93\xc7\x2f\xba\x98\xce\xfe\x05\x6d\x6a\xae\xaf\x92\x61\xaf\x41\x65\xd3\x3d\x0b\x22\x88\x49\x24\x2c\x3c\x08\xb3\x9f\x64\xce\x8c\x46\xc0\xf5\x35\xe1\x03\xf3\xb3\xcd\x60\xa1\x95\xc5\x7a\x0b\xd1\x76\x9e\xf8\x6c\xa9\x9c\x39\xff\x14\x82\xcd\xb9\x78\xbd\x72\xcc\xa4\x9e\xfd\x37\xf4\xc9\x59\x29\x3e\x1d\x66\xf5\x11\x7e\x18\x0f\xc0\xbf\x9e\xcf\xe8\x00\x59\xbf\xe4\x81\x41\x42\x8a\x6f\xcb\x28\x06\x41\xd3\x25\xb3\xdb\x78\x99\x48\x0f\x02\x93\xf9\x60\x1c\x96\x10\xff\xf4\xe4\x94\xc4\x86\x8f\xd1\x00\xef\x08\xb6\x7d\x31\x98\xdf\xcf\x56\x6d\x7f\x0b\x36\xa2\x06\x08\x11\x1b\x56\x1b\xd7\x65\x5e\x98\x78\x2e\x6a\x95\x4c\x51\x53\x52\xe1\x0d\x0c\x74\x4a\x72\x86\x8f\xc4\xb1\xb6\xb9\x56\x23\x10\xb4\x8d\x7f\x04\x3e\x8c\xda\x56\x18\x30\x80\x3e\x25\xa3\x8d\x8e\xa2\x90\x18\xde\xb3\x1b\x7b\x13\x2d\x23\xb6\xc2\xf3\xb0\x9c\x06\xd1\x8e\x7e\xf1\x11\xab\x75\x9c\x18\xb2\x2f\x40\x31\xdb\x76\x37\xbd\xba\xa3\xe3\xd5\x01\x14\x44\xc2\x9c\xb8\x48\x86\x5d\x3c\x5e\xef\xb8\x6d\xc7\x8a\xc6\x2e\x62\x6a\xd7\x80\x21\x61\xb6\x3a\x38\xc8\xc7\xf0\x49\x29\x4b\xfc\xc8\x2e\xb4\xa0\x58\x7a\x94\x67\x4f\x4d\x27\x8e\x42\xd2\xbd\x81\x14\xcd\xbc\x1c\x23\x22\x17\x0a\xc4\xfc\x60\xe0\x92\x11\xaa\x64\xaf\xa4\xe2\xe6\xa0\x32\xc6\x37\x06\x74\x7d\x57\x55\xfb\xe9\x29\x20\x97\x7f\x68\xd4\xb5\x31\x41\xc0\xe0\xef\x53\xb3\xb8\x82\xcf\x59\x5a\xe2\x6c\xe7\xa4\xc5\x6e\x8b\x2b\x2f\x42\xe0\x8b\xab\x2f\x6d\xd1\x2b\x48\x6a\x4e\xb1\x0a\x65\xa6\x23\xd5\x5c\x22\x0e\xb1\x8d\x9d\x15\x5e\xab\x56\x35\xd1\x2f\x58\xf8\x1d\x57\xc6\x5d\xd5\xeb\xb3\x9e\x55\x77\x73\x85\x9c\x1e\xa8\x76\xf6\x68\x07\x7c\x6e\x2b\xd7\x0a\x4c\x9e\xab\x0e\x92\x5a\x12\x81\xf1\xca\xad\x44\xdd\xc8\x53\x57\xfa\x37\x55\x92\xdc\xce\xf8\x3b\xcd\x74\x23\x60\x20\x8f\x70\x59\x75\x83\xb5\x0d\x6d\xda\xb4\x84\xdb\xb1\x59\x21\xcf\xa4\x28\xc6\x51\x2b\x92\x1d\x8e\x1f\xd7\xcc\x7c\x21\xd9\x85\x0b\xd3\x99\xd6\x5e\x85\x84\xd2\xe3\x2e\xc6\x8d\x70\xc1\x71\xdd\x8c\xfc\xd7\x29\x0e\x04\x5d\x49\x09\xdf\xdc\x53\x22\xa0\x64\xa2\x38\xa6\x46\xeb\x24\xfd\xb3\xa3\x55\xbb\x58\xe3\x34\x76\xa2\x86\xf6\xd5\x6a\x29\x9f\x4c\xc4\xd3\x91\x35\x51\x3c\xdd\x9f\x79\x50\x71\x8d\xf7\x9f\x6b\xc2\x77\x6b\xdb\xe2\xca\x00\x96\xf4\x17\xab\xa4\xcd\xe7\x97\x84\x83\xe9\x86\xca\x2a\x94\x04\xed\x15\x5f\xdf\xaf\x06\x4a\x0b\x5f\xd3\x2d\xc0\x6f\xe8\x15\xf7\x9c\x55\x76\x11\x74\x31\xbe\xf1\xa7\x9f\xbf\x2e\x37\x62\xcc\x1d\x97\xc2\x5a\x20\x72\x93\x3d\x5a\x41\xd7\xc5\x7e\x14\x4d\xe8\xd6\x8d\x90\xde\xe6\xad\x2b\x22\x74\xbb\x66\xa8\x3d\xdd\x0d\xfd\x90\xc0\x39\x4a\x0c\x83\x70\x4e\x26\x91\x33\xf4\x45\x60\xd6\x9b\x83\x3a\xa6\x5f\xe0\x9f\xfa\xd0\xef\x0e\xfd\xf6\x30\x30\x14\x90\x63\x03\xc6\xa9\xbb\x2c\xba\x29\x42\xdb\x99\xb1\x4b\x25\x3d\x24\xe7\x5b\x88\x24\x70\x5d\x81\x47\x52\xd0\xa1\x95\xc6\xa3\x8a\x46\x3b\x78\x5c\x82\xcb\xc6\xe2\x31\xc1\xe3\x3c\x0e\xbe\x0d\xba\x02\x09\x0b\xc4\x70\x65\x8d\x27\x44\xe4\xca\x77\x49\xf4\xe8\xee\x0b\x7a\xcc\x5d\x5d\x04\xb4\x2b\xd0\x59\x05\x17\x34\xd2\xb8\x33\xf9\x7c\x27\x0a\xe8\x97\xc3\xd5\xd3\x13\xc1\xcb\x87\xea\x54\xe3\x51\x17\x70\xf9\x6a\x13\x81\x22\xb3\xb4\x6b\x05\x83\x6c\xef\xbc\x56\x3f\x4b\x73\x95\x21\xf9\x31\x15\x5a\x2d\x37\x8f\x92\x09\xa1\x25\x61\xb4\xd9\x0b\xbf\x36\x93\x28\x5e\x62\x86\x4f\x90\xbe\x95\xbc\x97\xe8\x40\x50\x22\xdc\x8e\x70\x3f\x7d\x62\x90\x22\x91\xdb\x2d\xb5\x1b\x5f\x9b\xe0\x36\x14\x1c\x43\x02\x14\xd2\x5f\x19\x83\x8c\x25\xee\xca\x30\xb7\xa2\x94\x23\x95\xcb\xf8\x37\xd4\x84\x72\x0a\x7e\xe5\x25\x98\x6a\x4c\xc9\xae\x4d\x3e\x13\x83\x94\x63\x88\x09\x29\xcd\x4a\x49\x2e\x41\x11\x71\xa5\x40\xe4\x5d\x81\x60\x24\xc1\xa5\x50\x9c\xc6\x70\x56\x25\x30\x8d\xab\x56\xdd\x0f\x29\xff\x34\x5e\xdd\xce\x87\xc5\x99\x8b\x92\xf1\x5f\xd5\x9a\x84\x84\x4b\xa0\x06\x1d\xbb\x8d\xeb\x8a\xe6\x2c\x92\x29\x3d\x42\xac\xe2\xe9\x82\x1d\x5f\x6a\x08\xe5\x2b\x82\x63\xf5\xbb\x79\x0b\x31\x02\x99\x5d\xe2\xad\x45\xb5\x62\x77\x17\xfa\x6a\x85\xac\x75\x86\x4e\x81\x24\x32\x7a\xcf\xed\x6b\x18\xb2\xbe\x5a\x39\x89\x4b\x5e\xb9\x30\x1f\x77\x04\x0d\x04\xd3\x09\x8e\xc7\xe5\x6b\x74\x89\x42\x2f\xec\x39\x8b\x30\x4b\xe2\x4f\xd1\x30\xbd\x1a\x64\x71\x03\x5e\x7c\x96\x0a\x9f\x51\x75\xca\x93\x5c\xe1\x8c\x72\x88\x00\x8b\xcc\x30\x62\xfe\xb3\xcd\xbc\xe8\xd6\xe8\x92\xfa\xa0\x5b\xfb\x02\xa6\xd9\x7e\xad\x4d\x97\xfd\x2d\x8a\x07\xe8\xc2\x05\x17\xd8\xf5\x05\x9f\x7d\xaa\x23\x81\xf4\x16\x26\xef\x4b\x6a\x70\x69\x9f\x28\xc1\x60\xf7\x5a\x8b\x68\x69\xbb\xe5\x61\x14\x6a\x96\xfb\xb8\x69\x07\xe7\xbe\x2f\xa6\x60\x54\x0f\xdb\x81\xf8\x0e\x7e\xbf\x87\x7f\x97\xf0\xef\x0d\xfc\xfb\x85\x17\xe7\xa0\x69\x54\xbd\x09\x9d\x03\x06\xa6\x2c\x54\xd2\x04\x43\x22\xf9\x9b\x79\x11\x4a\x1a\xb3\xf7\xd7\xe9\xe5\xca\xda\xa2\x86\x6e\xa7\xd0\x77\x69\xe3\xbb\x10\xfe\x7d\x19\x34\xbd\xa0\x79\x08\x26\x22\x1f\x2c\x21\xe9\xff\x25\xe4\x46\x1b\x36\x7e\x39\x38\x90\x12\x07\xb6\x9b\x25\xb1\x42\x69\x6e\xee\xd0\xbf\x74\xad\xb4\xb6\xfc\xd6\x93\xd9\x3b\x2a\xb3\x3b\x24\x76\xc6\xab\x6f\x01\xf7\x5f\xc8\xcf\x5f\x63\x23\xf0\x5d\xb6\x3b\xb7\x5d\x7b\x50\x74\xd5\xfe\x8d\xbe\x9e\x9e\x6a\x7f\xa2\x0f\xb7\xeb\x54\x64\x94\x03\x4d\x49\x8a\xc4\x01\x39\x1b\x0f\xde\xa9\xfb\xe5\x99\x77\x06\x35\xfc\xb7\x3a\x63\xba\x89\x57\xdf\x44\x1b\xc7\xf5\x4e\x20\xa5\x8d\x20\x98\xfa\xa7\x7c\x2a\xe6\xc3\xfb\x82\x65\x45\x0c\x1b\x53\xf8\x37\x34\x04\x7c\x37\xbf\x5f\xa6\xc8\x53\xcf\xba\x97\x06\xec\x3b\x0a\x7c\x74\xc4\xdb\xf8\xde\x83\xc0\x97\x10\x80\xef\x4b\xf8\xf7\xc6\x15\x8a\x71\x88\xe1\x9a\x8a\xd9\xca\xeb\x21\x8d\xd5\xfc\x92\x7b\x64\x89\x81\x2e\xdb\x5c\xc1\x16\x35\x80\x94\x0e\x7a\x94\x3b\x27\x20\x77\xd9\x8e\x8f\xaf\x4a\xe0\xa1\xda\x9c\xf0\x8a\x69\x56\x80\xe8\xed\x02\x32\xe5\xef\x2e\xc2\xa9\x14\x27\xde\x7b\x1f\x94\xeb\x06\xcf\x73\x05\x69\x8e\x01\x9f\x4c\x2c\x74\x65\x82\x58\x64\x05\x7a\x99\x5e\xad\xa4\x4f\xb4\x9c\x9a\xe8\x96\x65\x6f\xaf\x7a\xe2\xc5\x02\xa6\xa9\x77\x21\xdb\x11\xa6\x92\xee\x2b\x5f\x53\xc9\x9e\x91\x90\x54\xdd\x2d\x2c\x4a\xef\x00\x91\x01\x95\x2c\xbe\xab\xd2\xaa\x6f\x63\x9a\xc4\x9b\xad\x30\x54\xe3\x6f\xe3\x9b\x3f\xae\x17\x4e\xed\xaf\x4e\xb7\x5d\xf3\x28\x9d\x2e\x26\x91\x97\xc9\xf7\xf7\x73\xe8\x07\x52\x59\x3c\xd5\xc8\x96\x14\xb5\xa4\x56\x62\xe7\x71\x11\x6f\xe6\xf3\xf1\xfd\x42\x15\xa3\x1a\x0b\xf0\x59\x7e\x26\xd4\xb6\x0a\x97\x95\xdd\xd8\x02\x48\xbd\xd8\x80\x3b\x5b\x20\x4a\x6f\xe6\x0f\xf1\xf2\xeb\x88\x2e\x65\x27\x5a\xd7\x01\x60\x15\x0d\xe8\x40\x9d\x97\x6b\x46\xc9\x64\x22\x1e\x92\x21\x28\x11\xf6\x37\x9e\xdc\xcc\x42\xff\x82\x97\x5e\x6a\x75\x50\xab\x35\x35\x76\x38\x98\xd6\xad\x53\x52\x9b\x67\x9a\x1e\xa4\x32\x75\x39\x79\x52\xea\x16\x72\x78\x0e\xa1\xbf\x90\x4b\x98\x58\xc9\xaf\xf0\x45\x42\x8e\xae\xcb\x83\xa8\x81\x64\x21\xd2\xe3\x7a\x8c\xac\x2d\xb5\x68\x45\x07\xf8\xb7\x38\x1e\x0f\xa3\xcd\x57\xfd\xfe\x32\xfe\x90\x91\x8f\x84\xde\x4f\x21\x48\x0d\xf0\x16\x64\x3f\x4a\x57\xf4\xa6\x22\x0c\x7e\xac\xd3\xcb\xa0\xe2\x75\x3c\x50\xc3\x9e\x19\x19\x2c\x47\x1d\x33\xa9\xae\x1e\x8a\xd9\xb9\x6d\x69\x75\x19\xaf\xe8\x66\x9b\x06\xda\xc6\x9b\x99\x87\x21\xdd\x76\x3d\xd8\x5d\xb3\x1d\x75\x7a\xae\x36\x9f\x5d\x8f\xdf\xb6\x06\xf2\x6d\xf2\x8a\x7a\xb0\xc1\xbe\xb3\x2a\x1a\xa4\xa2\x36\xb4\x7b\x5e\x52\x23\x22\xf3\x93\xa8\x65\x52\x2f\xc1\xfe\xac\x66\xfc\x6f\x40\x70\x91\xd8\x3f\x7d\x36\xb1\xdf\xcf\xff\xe1\xc4\xfe\xdb\x27\x12\x0b\x24\xae\x6e\x77\xf7\xd0\xa9\x01\xd9\x49\x6c\x16\x6e\x6f\x8a\xa7\x65\x08\x7e\x23\x29\xa7\xda\xed\xac\xd7\xf3\x35\xfa\x15\x75\xf9\x8d\x6a\xf1\x06\xad\x94\x18\x8d\x9f\x7c\x55\xcc\x9a\x7d\xde\xec\xc8\xd9\xe7\x8d\x81\x36\x68\x1c\xd7\x20\xd8\x59\xe0\x37\x6a\x49\xe1\xb3\x0a\x5c\x7f\x7a\x81\x3f\x43\xf8\xf3\x0b\xfc\xf3\xa7\x14\x68\x2c\xc9\xdf\x53\xf3\x1d\x17\x25\x64\xf3\x89\xfd\xf3\xf7\x27\xb2\x59\x46\xa4\xca\x1f\xaf\x17\xd1\x6c\x48\x44\x10\xc1\xfb\x0b\xed\x2f\xf3\x59\x45\x63\x1e\xfd\xf5\xca\xab\xf7\xde\x0d\x1f\x8f\xb7\xff\x7c\xd4\x58\xe1\x13\xa3\xd2\x80\x29\xa5\xef\xc4\x55\xd3\x8f\x5f\x42\x4f\xe1\x82\xd8\xd2\xa2\x2d\x7a\xad\xfd\x35\xcf\x19\x5e\x9c\xbe\xec\xe2\x14\xaf\xdd\x8c\x5b\x55\x42\x41\xda\xe2\x1f\x31\x22\x96\xb0\x7c\x1a\x22\x5b\xc1\xa6\xdc\x97\xbf\xdf\xfc\xae\x03\x4b\x39\x91\xc3\x4f\x14\x5e\xa0\xf1\xc7\xd1\xef\x2f\xc1\xad\x22\xa5\x77\x9f\x48\x29\xce\x1d\x9b\xc7\xff\x70\x86\x7e\xf7\xa9\xa3\x75\x32\xbb\x87\x0e\xf3\x0f\xa7\xf3\xfb\x4f\xa4\xf3\x32\x1e\x80\xe9\xf3\x8f\xa7\xf3\xf2\x93\xf9\x39\x99\xa0\xdb\x92\xdf\x9f\xd8\x12\x19\x7d\xf3\x89\xc4\x7e\x35\xfd\x69\x9a\x27\x32\x4b\x47\x04\x10\x96\xcd\x51\x4a\x48\xd8\x74\x73\x76\x48\x27\xb3\xcf\x0d\x13\xc8\xa0\xcd\x04\xc2\x84\xb7\x74\x04\xfd\x48\xaa\x9d\x0b\xff\x18\x0e\xb1\x2c\x1c\xba\x31\x9a\x4f\x7d\xc0\x00\xfc\x31\x0d\x3f\x5e\xf8\x3c\x41\xf5\x6a\xe2\xe3\x6d\xf8\xb7\xbf\x39\x78\xaa\xf7\xa3\x7b\x74\xea\x43\xfa\x34\xe4\xd0\x97\xa7\xfa\xb1\xfd\x8f\xa9\x57\x9c\x05\x7f\xbc\x15\x35\xbf\x26\x9a\x6e\x59\xda\x54\xa6\x55\x99\x15\xc9\x2a\x5e\x46\x93\x9f\xe2\xe5\x80\x5c\x4d\x97\x37\xf0\x82\x93\x77\xb6\xb0\x81\xd9\x73\x9e\x93\x69\xd4\xd2\x36\xe5\x6a\xfc\x09\x77\x20\x9e\x59\x28\xfe\x84\x95\xec\x92\x55\x6c\xde\x26\x82\x88\xaa\xfd\x08\xbd\xe4\x18\x56\x2d\x60\xef\xb7\x94\xba\x8b\x4c\xb3\x61\xc5\xb9\x55\x26\xbd\x0d\x06\xe9\x07\x07\xbc\x0e\x57\x41\x66\x61\x61\x8e\xd1\xa9\x60\xe5\x5a\x1c\xc7\xbf\x4e\xe7\x3f\x90\x6f\xf3\xec\x6e\x1a\x95\xb8\x9a\xbf\xbe\xfc\x51\x99\x8e\x45\x04\xec\xf6\x48\xba\x53\xe6\x3d\x81\x54\xbb\x9d\x31\x7b\xd2\xe5\xde\x94\xd7\xb9\x6d\xf9\xef\x79\x4f\x81\xdd\x9a\xf3\x01\x0e\xb9\xae\x53\xf5\xe6\xc7\x2a\x5a\x42\x07\x0b\xe9\x55\x0f\x3e\x34\x91\xd8\xcf\x47\xd9\x44\xe2\xcd\xfd\x54\x66\xd0\xac\x4d\xc2\xb0\x08\xa4\xce\xab\x5d\xc9\xfa\x34\x36\x58\xb9\x61\xd1\x79\x0e\x3f\x62\xc1\xd4\x64\x5d\xbf\xda\x7e\xb1\x5b\xc1\x49\xeb\x34\x3e\xdd\xba\xb2\x46\x57\xcd\x5e\xaf\x9d\x74\x25\xf2\x2b\xa6\xe8\xa8\x48\xc6\x55\x52\x0f\x7a\xaf\xca\xe2\x7b\x47\x9c\xa9\x0b\x10\xed\xa4\xd7\xbe\xca\x00\xd9\x7a\xfb\x39\xaa\x2d\x92\xf6\x7d\x5f\x20\x53\x16\x4d\x83\x7e\x07\xdf\xd4\x6b\xed\xf6\x38\x1b\xef\xca\xa5\xcb\x1c\x09\x15\x6e\xa0\xd5\x96\x98\x48\xc7\x89\xfd\xdc\x17\x06\x17\xf1\x30\x23\xed\xca\x41\x1f\x46\x1d\x1c\xbc\x50\x59\x95\x9b\x5f\x7b\x52\x65\x6a\x4e\xbb\xc1\xa0\xdb\xb8\x00\xf5\xdc\x26\xf5\x69\xae\x7a\xc6\xc3\xa2\x7a\x1c\xa6\xca\xcb\x3f\x4b\x44\xc8\x43\x8d\x22\xa0\x5b\xec\x18\x81\xef\xb6\x6b\x3c\xac\x9a\xe3\x66\x0a\xfe\xe0\xa0\x98\x41\xa5\x99\xc5\x55\x4a\x3e\x38\xd0\x1c\x0a\x39\x86\x7c\x7f\x42\x55\x54\x10\x6f\x9b\x94\x7b\xd0\xb6\x5f\x0a\xc2\x1c\x17\x41\xe1\x01\x3e\xb3\x91\xd7\xb1\x19\xae\x6b\xa6\x36\x21\x10\xcc\xed\xb8\x4a\x35\xe6\x58\x2c\xb7\xed\x6d\x5d\x98\x7f\xb8\xef\xd9\x82\xcc\x41\x81\xea\x72\x54\x53\x66\x0a\xda\xb6\x35\xeb\xdc\x0a\xff\xe7\x39\x19\xab\x7e\xcb\x21\x2b\x0a\xbf\x51\x6b\x97\xc0\x6b\x8a\xdb\x39\x21\x3e\x38\xb8\x7a\xa4\x0f\x5d\xa9\x2d\x11\xdd\xfb\x1c\xa9\xc8\xf5\x0f\xa3\x91\x4b\x38\xab\x15\x37\xf6\x94\xe0\x02\x51\x75\x83\x36\xf1\x6b\x97\x57\x78\xb3\x7b\xcd\x83\xd7\xb3\x1e\xe4\x0a\x03\x92\x72\x7c\x9e\x1b\x97\x3e\xdd\xef\x79\xae\x46\xab\xcc\x06\x8b\x8c\xaa\xc8\xf0\xad\xfd\x44\x43\x5a\xbe\x23\x6f\xdf\x99\x53\x2b\x36\xfa\xc2\xdc\x48\x45\x81\xde\xef\xbc\x18\x01\x1b\x39\x4f\xc7\x35\x29\xf5\x7a\xa2\x5b\x71\x04\xad\xc0\x10\xd9\xdd\xf7\xbb\x14\x1f\xfa\x89\xdf\xdf\xc7\xa9\xa1\xff\x7f\x5c\xfe\xf8\x83\x34\x3a\x64\x52\x63\x19\xa7\x8b\xf9\x2c\x8d\x7f\x86\x66\xcb\x56\xea\x76\x35\x9d\x18\x0c\x99\xe7\x9d\x86\xf3\x01\xe9\xf3\xc6\x60\x19\x43\xd1\x3c\xc4\x64\xfd\x2d\x37\xf8\x9c\x13\x9d\xf5\xb4\x73\xf4\xe7\xc3\x8d\x74\xb3\x2b\x73\xd3\x79\xd1\xf5\xea\x3e\x9a\x7c\xbb\x8c\xf8\x8a\x48\x39\x71\xa4\x68\x5b\xe1\x23\x9e\x67\xc7\xf3\x94\xb5\x56\xa3\xd5\x38\xaf\x6d\x3b\xd8\x24\x8d\xd9\xfc\xe1\xe9\xc9\x51\x9f\x45\x91\xf1\x54\xe3\x6d\x79\x67\x19\x88\x8a\x70\x6b\xe7\x12\x1d\xd9\x84\x57\x3d\xf6\x68\x23\x54\xb4\x41\x00\xa3\x6b\xc6\xa7\x9e\xc9\xc5\x67\xf4\x28\x9d\x84\x4c\x55\x32\x54\x1f\x76\xe4\x1f\x27\xf1\x54\xea\x09\xcd\x8c\x5c\x1a\x82\x3f\x80\x68\x02\xf9\xfc\xd3\x91\x76\x24\x15\xe9\x14\x91\xf1\x3b\xc9\xc8\xe3\x14\x9d\x26\x34\xf0\xfc\x1e\x9e\x01\xdf\x0e\xa2\xd5\xe0\x16\x5d\xf6\x57\x55\xc7\x48\x20\x86\xd5\x4e\x2b\x83\x9a\x5d\xaf\xc4\xed\x24\xf5\x7a\xc7\xa5\x78\xbc\x36\x88\xc0\x57\x46\xf8\x28\x7e\xbb\x55\xd6\x6e\x56\x2a\x24\x8d\x4e\x6d\x98\x7c\xa8\xb9\x8d\x74\xb5\x99\xd0\xde\xf4\x4f\xcb\x39\x4c\x21\x56\x1b\xa7\x36\x5f\x44\x83\x64\xb5\xa9\xe1\x16\x79\xcd\x55\x44\xa3\x43\x67\x79\x18\xa1\x75\x1d\x33\x92\xeb\x85\x3a\xf7\x1d\x6a\x1e\x35\x14\x13\x74\x9a\xb0\x32\x40\x41\x5f\xad\xc0\x8e\xed\xdf\xb3\xfe\x2f\x20\x6a\xd8\x10\x55\x39\x7f\xb8\x7c\x3e\xef\x0f\x97\xa4\x60\xb0\x7a\xa5\x64\x7e\x7d\x79\x79\x89\x89\xdf\xc4\x83\x49\xc4\x27\x22\xb3\x24\x73\x56\x8b\x33\x61\x11\x9f\xcd\xb8\xce\xb3\x24\x99\xd6\xb6\x4e\x79\x3f\x56\xd4\xd1\xba\xf3\x65\xa0\x3d\x6c\x10\xb1\x47\xdd\xad\xb9\x0f\xb4\xa6\x9c\x62\x3d\x53\xe2\x0f\x97\x56\x99\x85\x6c\xba\xe8\x1d\x2c\x28\xab\xa0\x58\x2c\x93\xf9\x12\x3d\x6e\x3e\x96\x31\xb5\xaa\x9a\x26\xdb\x76\x6b\x7b\x69\x36\x45\xd8\xb7\xda\xfb\x17\x11\x2e\x0a\x44\x17\xfd\x2e\xfe\x0d\xfb\x5d\xbf\xed\x1f\xf9\x48\x6f\x03\x7a\xe1\xce\xbc\x98\x09\xf2\x22\x0a\xfc\x1b\x46\x56\xde\xa9\x6d\x38\x53\xb7\x12\x23\xee\x04\x78\xff\x58\xee\xb0\x53\xbc\x76\xbe\x33\x72\x82\x30\x0c\x0b\xcf\x43\xb0\x65\x44\xfb\xef\x60\xd8\x3a\xf2\xb8\x53\x14\xaa\x3e\x0c\xe6\x2e\xd0\x1d\xc1\x80\x12\x85\x7c\x54\xb6\x63\x72\xe0\xc1\x44\x82\xef\x67\xe1\xf1\xc4\x7f\x14\xf6\xf9\x61\xa7\xaa\x12\x46\xcc\x62\xa6\x5e\x65\xe7\x83\xdf\x7b\x95\xb8\x33\xbf\xa6\x40\x31\x93\xd9\x16\xad\xff\xff\xcb\x36\x68\xe8\xff\xcb\x6c\xb3\x28\xc8\xb0\x2d\xce\xb9\xf8\x2d\x70\x6e\xf0\x9b\xf0\x6e\x50\xca\xbd\xc1\xbe\xfc\x73\x4c\xab\x8b\xfe\xc5\x80\xce\xd4\xf5\xdd\xdd\xcc\x1c\xfc\x9e\xec\xac\x22\x48\x5e\xec\x8a\xc4\xa0\x47\xec\x4d\xef\xa7\x95\xbc\x4d\xe9\x88\x93\xcd\x56\xf6\xee\x51\xc5\x5b\x9b\x46\x9e\xb3\x46\xa1\xa7\x99\x04\x16\x4f\xea\x61\xa5\xcc\xa9\xe0\x3c\x6c\x65\x7d\x74\x5e\xb5\x12\xc0\x3d\x2a\x8e\xaa\x35\x51\x8e\xf0\x69\xe8\x9b\x13\x9a\xfb\x54\x00\xf4\xb2\x3c\xf8\x9c\xe9\x58\xce\xd4\x83\xb6\xab\x4f\xdd\x23\xcf\xbb\x2b\xa9\x8b\x9d\x6d\x57\xf3\x64\xd1\xa8\xe3\x53\xdd\xa9\xba\x34\x21\xac\x07\x07\x4c\x1d\x69\x20\x48\x85\x9c\xcd\x7d\x17\xca\x08\xe3\xd2\xf0\x70\xe1\x05\xe2\xd6\x7e\xfb\xe2\x3b\x57\x7c\x08\x3d\x86\xbb\x22\xff\xe9\x71\xf8\x5d\x5d\x9f\x24\x8a\xbb\x1f\xbc\xf8\xd0\x51\xe9\xbd\xfa\x07\xb7\xfd\x41\x72\x77\x98\x94\xf2\xb7\x62\xa1\x04\x5d\x6c\x32\x90\xe4\x3c\x2d\x3b\xb9\xae\xb0\xe3\xe4\x2b\x04\x9a\x4b\x32\x55\xad\xb5\xd8\xaf\x2c\x70\x42\xc9\x8b\x04\x8d\x13\xd7\xe6\x12\xaf\xa8\xcd\x97\x86\x52\x4d\xe3\xe3\x24\x1e\xad\xda\xd6\x00\xb7\x86\xc1\x5b\xdc\x26\xac\x05\x4a\xdf\x22\x99\xcc\x4b\x9f\x29\x39\x86\xa4\xdb\x24\x54\x77\xf1\xdd\xce\x6d\x72\x31\x99\x77\xb8\x1d\xa6\xc9\x30\x9c\xcc\x3d\x88\xba\xb8\x08\x3a\xaa\xd5\x45\x74\x05\x09\x3d\x01\x7f\xdc\x57\xeb\x2e\x20\x86\x2f\x2f\x68\x03\x1a\xf8\xd8\xea\xfb\x50\x5b\xb1\x4c\x6e\x6e\xff\xe1\x64\xae\x5f\x95\x11\xda\x65\xea\xda\x8a\x5a\x8b\xcc\xed\x56\xcd\x45\x34\xc7\x2d\xee\x97\x2e\x30\xc2\xf4\x45\x83\xbc\x81\xa6\x08\xad\xcc\x0d\x6c\x1b\xd3\x7e\x06\xd7\x5b\xe4\x46\x06\x92\xf8\x43\xca\xea\xf6\x7e\x34\xb2\xfb\x03\xc9\x88\x99\x21\xac\x44\x02\x1d\x3d\x33\x0e\x4c\x41\xbd\xc8\x67\x54\x22\x30\x71\xa7\x8e\x7b\x38\xad\xd7\x9f\x7c\xb1\x92\xbd\x7a\xda\x13\xea\x43\xf7\x73\xdd\x59\xc3\x55\x76\xfa\x80\x44\x80\x85\x36\xcd\xd8\xad\xdc\x31\xc8\xd1\x7c\x9c\x71\x32\x23\xa3\xb4\x87\x31\xce\x98\x96\x4c\x5a\x54\x12\x16\x29\xcb\xe6\xbc\x10\xa1\x67\x30\x0a\x88\xa9\x88\x92\x65\x5a\xc9\x88\x85\x2f\xcf\xa7\xda\xbc\xa8\x93\xef\x19\x46\x4f\x5e\x5f\x11\x83\xa1\xc5\xbf\x98\x81\x91\x37\x73\xf9\xcc\x2a\xa5\x22\x39\x57\xda\x6b\x0d\xe7\x04\x3d\x67\x91\x84\x50\x44\xcf\xc7\x64\x61\xcf\x63\xf1\x12\xa3\x33\x2b\x6a\x58\x79\x62\xf4\xca\x76\xd7\x48\xae\x77\xd8\xae\x34\xc2\x8e\x46\x35\xe0\x7c\xc3\xd9\x04\x7c\xda\xb4\x4e\x5d\xe5\x81\x4b\xa1\x99\x81\x62\x07\x44\x48\x06\xc2\x22\xe5\x06\x7c\x86\xe0\x77\xa8\x9e\x21\xed\xea\xae\x67\xe8\x82\x80\x35\x47\xc4\x9c\x54\x1b\xba\xca\xb4\x98\xdb\xdb\x25\xd3\x08\x66\x04\xf6\x5a\x33\x56\xd9\x3c\x55\x2d\x93\x29\xf7\x38\xde\xd8\x2e\xf3\x23\xa9\xae\x29\xda\xaa\x39\x84\xf1\xd0\x2e\xa6\x63\x12\x1f\x9e\x87\x2f\x3d\x1c\x60\x2c\x21\x64\xed\x5c\x82\x52\x26\x94\x23\x95\x63\x03\xa1\x85\xf0\x15\x24\x19\x8f\x17\x9c\xc8\x16\xd6\x0c\x48\x2f\x45\xaf\x52\xca\xf1\xcb\x54\x2e\xe0\x11\x12\xda\xfa\x72\x65\x5b\x95\xb7\xd5\x05\x4a\x68\x39\xb4\x2c\x6f\xf2\x1d\xc8\x3e\xb3\x2b\x08\x60\xc8\xdd\x50\x49\x71\x6a\xdb\x20\xf6\x45\x8b\x3b\x4f\x02\xe0\xa1\x5d\xeb\x5d\x1f\x46\x62\x89\x01\x0c\xb4\xf5\xfa\xec\x02\x72\xba\xac\x58\xf5\xb8\x84\xfe\xcb\xf2\x9a\xa3\x5e\x9f\x12\x28\xa3\xb9\xaa\xd7\xef\x7a\x5a\x65\x98\x75\x49\x4c\x63\xc5\x18\xf5\xd3\x50\xbd\x19\xdd\x51\xaf\xf0\xd9\xdb\x6d\xfc\x54\xdf\x7c\x21\xdf\xeb\x83\x2e\x52\xaa\xda\xe9\xf1\xaa\xa0\xa8\xdb\x9b\x94\x36\x5f\x84\x0a\x13\x3e\x84\x85\x8f\xc4\x67\x5e\xb7\xa2\xdc\x61\x18\x1c\xf9\xf8\x84\xd1\xfc\x81\xd6\x01\xff\x88\xeb\x14\x4e\x2d\x99\x8d\xc8\x03\x14\xaf\x7a\xd5\x78\x6d\x49\xbe\x28\x8e\x37\x58\xc6\xa8\x75\xf9\x36\x0f\x2e\xa2\xde\xc4\xcb\x4b\x5a\xb2\xc4\x1d\x5f\xa2\xd9\xd5\xf6\x1f\x95\x76\x18\x8e\xa9\x42\xfc\x1b\xd3\xaf\x7f\x41\x90\x64\x12\x39\x77\xa1\xf5\x86\x19\xde\x41\xb9\x40\xf0\x8e\xcb\x4b\x6a\x24\x37\x77\x47\x63\xdb\x8a\x2a\x66\x79\x55\x9e\xc5\x5e\xbf\x93\xd3\x2b\x4b\x01\xcd\xfb\xe8\xfd\x40\x0e\x7b\x99\x13\xe0\x48\x3c\xa7\x82\x18\x43\x51\xb3\x01\x2e\x5a\x73\x9a\xcb\x09\x3b\x2f\x0d\xab\x53\xe2\x26\x6a\x6b\xd1\x6f\xf5\x10\x49\x82\x0d\xcf\x51\xd9\x7e\x88\x87\xc9\x51\xd7\xb1\x1b\x01\xa6\x43\x3c\xde\x46\x69\xdb\x2e\xdd\x5e\xd3\x83\x1c\xd7\x8b\x65\x3c\x4a\xd6\x9e\x2c\x0a\x17\x0a\xb6\xe2\x26\x5e\x95\xe7\xc1\xe4\xab\x42\x46\x7c\xf7\x24\x97\x43\x64\xde\x7d\xaf\xca\xc7\x17\xa6\xf1\x4d\x9a\xe9\xfc\x43\x5c\x5e\x24\x7c\x87\x85\x9c\xc2\x22\xf7\xe0\x40\x5e\xf1\xa5\x42\x98\x1c\x54\x77\xed\x9c\x27\x0b\xa5\x33\x2d\x92\x4a\x9b\xc7\x7d\xcc\xaa\xd0\xad\x2b\x58\x7b\xb2\x9e\xcb\xa3\x35\x7a\xf3\x39\xc4\x8a\x27\xb6\x32\xd5\xb7\xc1\x95\x12\x95\xba\x2d\x5f\x8a\xa5\x3e\xf7\x2d\x66\x97\x4e\x65\xbe\x63\xb9\x5a\x97\x4a\x54\x6d\xdb\xf4\xcd\xc9\x21\x16\x89\xe3\x8b\x7d\x77\xc3\x77\xc3\x30\xdb\x3e\x98\x70\x70\x30\xb2\x16\x9e\x30\x8b\x39\x28\x81\xbe\xb0\x54\x4b\xb9\x5b\xb3\x2c\x6d\x30\x84\xb5\x77\x6b\xdf\x27\x07\x80\x59\xb4\xd9\x82\xb2\x64\xa0\x9e\x9c\xc5\x69\x66\x6b\x45\xaf\xeb\xe3\xfc\x01\xfe\xe1\x22\xb1\x1c\x09\x86\xf1\x02\xa7\xf0\x78\x4d\x15\xbf\x2e\x42\x6a\xf4\xec\x5d\xad\xe5\x7c\x32\xb9\x5f\x74\xf9\x47\xde\x4c\x87\x12\x18\x83\xdb\xc6\xf9\xc4\xbf\x52\xb3\x75\xad\x09\x86\x89\xc5\x4b\xc5\x10\x6d\x8f\x77\x94\x20\x3b\x2f\xf6\x19\x98\xc0\xc8\x96\x2f\x5b\x79\x40\xc6\x11\x5d\x57\x44\xa3\xe7\xf5\x24\xf0\x1f\x36\xe8\x69\xc7\xd2\x44\x72\xf8\x92\x73\xaf\xd0\x82\xa2\x63\x40\xaa\x64\x44\x26\x55\x96\x99\x88\xba\x6e\xd7\x96\x48\xa9\x6c\xda\x36\x8e\xd4\xc2\x21\xae\x18\x22\xa3\x77\x90\xb1\x5d\x85\x59\x86\xc9\xe3\x15\xd6\x30\xb4\xa5\x93\x71\x30\x72\xf7\x51\x6a\xc9\x0c\x7e\xbb\xad\x24\x6f\xb8\xb1\x40\x5c\xdb\xaa\x8c\xc7\xed\xbe\xc8\xaf\x54\x7c\x2f\xac\xc4\xac\xfb\x1f\x57\x57\xf5\x2a\x2e\xc1\x95\xed\x65\x76\x89\x64\x9f\x41\x5c\x7b\xc9\x11\xc0\xf1\xb0\x4e\x56\x02\x8c\x8e\x28\x23\xd8\x82\xf2\xd7\xb4\xaf\xc5\xd2\xf2\xbe\x0d\xff\x70\x74\x79\x64\x31\x29\xe9\xda\x69\x5b\x51\xc7\xa0\xba\x86\x50\x47\x59\x9c\x2d\xad\xa5\x2b\xb6\x12\xce\x89\xc8\xd1\x52\xbf\xc1\x5a\x90\xa5\x99\xb6\xa3\x8c\x8c\x0b\xee\x1c\x82\x7a\xde\x23\x2b\x5e\xab\x86\xb6\xc2\x42\x90\xec\xd0\xca\x3d\x51\xb6\x88\x2e\xbd\xd8\x57\xd1\xab\x01\xe5\x2e\x98\x97\x72\xae\x92\xb5\x0b\x1d\x79\xf1\x0b\x9b\x47\x21\x30\x38\xb0\x4f\x95\x4c\x30\x8d\xce\x1f\xba\x04\x28\xc1\x75\x45\x8c\x41\xb0\x1c\xe2\xad\xff\x2c\xaf\x48\x9d\x99\xc5\x93\x5e\x48\x50\x79\x44\xff\x9a\x33\xbe\x8b\xa8\x24\x44\x21\x37\xb3\xb9\x64\x5d\xe2\x0b\x95\x62\x41\xf3\x6a\x5c\x9c\x5f\xe9\x94\xa7\x8c\x20\x5e\x6a\x8e\xcb\x78\xd5\x21\xd3\x11\x13\xb3\x2f\x7c\x64\xcc\x57\x9c\xc5\xa1\xfd\x04\x59\x1b\xd1\x70\xe8\x68\xd5\xa1\xd7\xd0\xe2\x55\xd6\xf2\x00\xcc\x39\xcb\x23\x6b\x0f\x64\x07\x74\x4a\x33\xd6\x07\x14\x51\x95\xaf\xc4\x8e\xf8\xc0\x7d\xfb\x85\x2f\x2a\x8c\x89\x2c\x06\x76\xd4\x52\x82\x42\x64\xa8\xc8\x1a\x15\x5c\xc6\xaf\xb4\x00\xf6\x19\xfd\x77\x0d\xc2\x19\xfa\x5c\xe9\x6a\xe6\xd3\x06\x62\xce\x64\x0f\xc5\x38\x00\xd3\xca\x49\x7c\x1b\x7d\x48\xe6\x4b\xec\xc3\x74\x67\x12\xcf\x0a\x18\xf9\xe1\x93\x58\xd0\xab\xef\x97\x03\x6b\x23\x9f\x8f\x1c\x48\xaf\xfe\xf9\x59\x86\x1a\x94\x38\xaf\x3c\x66\x66\x4d\x97\x61\xee\x4f\x93\x04\x3e\x96\x90\x29\x41\xfe\xc8\x3c\xb6\xa7\x54\x04\xd2\x8b\x47\x1f\xe2\xd9\x70\xbe\xfc\x89\x2a\x8b\x0d\x50\x7b\x88\xfb\xe3\x64\x55\x13\xb5\x69\x8a\x7f\xe6\x1f\xe1\xef\xf7\xf4\x77\x0e\xff\x7e\xac\xf5\xd0\x48\x18\x26\xe9\x02\xb7\x5c\x33\x86\x82\x72\xa9\xa9\xd2\xd4\x81\x4a\x19\x36\xa3\x73\x79\x1d\x15\xdc\x55\xa1\x7a\x2a\xe5\x9a\x5c\xdc\x38\x2a\x68\x4e\xe0\xc8\x08\xde\x49\x97\x01\xb3\x51\xda\xb0\xbd\xce\xd1\xce\x29\xee\x45\xc7\xb3\x58\xee\x15\x27\x74\x56\x46\x3f\x4d\x58\x6b\xd4\x5c\xda\xe7\x0b\x6b\x35\xba\x85\x0e\xd3\x4e\x98\xeb\x51\x0c\x6f\x26\x9a\xf3\xaa\x78\x4e\x85\xf6\x6c\x73\x09\x3e\x5d\xb0\x27\x67\x63\x15\xcb\xb3\xaf\x9a\x5d\xea\x16\x08\xd3\x6b\xc8\xdd\x48\xb7\x5d\x8c\x33\xc4\x22\x31\xcd\xd2\x4d\x14\xf4\x1d\x4e\xa7\x83\x34\x2c\x3d\xa4\x0b\x68\xb4\xac\x53\x7f\x02\x6d\xf2\xe3\xc3\x4c\xef\xa2\x13\x7d\x07\x07\x25\x65\x22\x36\xd7\xee\x8a\x5b\x5e\x9a\x40\xfe\xd3\x41\x24\x61\x2e\x05\x5b\x73\x6a\xa3\x86\x21\x71\x31\x89\x06\x74\x80\x43\xc2\xc1\xaf\xa8\xbd\x7b\xf7\xcf\x07\x35\x57\x4b\x9f\x49\x0a\x8f\xae\xde\xbd\x7b\xf7\xd7\x77\xff\xfc\xee\xf0\x9d\xf7\xae\xfb\xee\xe9\xdd\xd5\xbb\xde\x3b\xe7\x9d\xfb\xae\xf1\xee\xf1\xdd\xb6\x77\x74\x43\xfb\xba\xd2\xc1\x1b\x74\xb1\xc6\x35\xef\xef\x5e\x5f\x77\x73\xf3\x4c\xa1\x5b\x5e\x1b\x47\x1a\x36\xd4\x69\xdb\xf6\x8e\x6c\x7a\xe1\x4e\xb2\x0a\xd9\x68\x92\xa5\x51\xa4\x12\x7b\x06\xa9\x89\xe3\x6d\x68\x3a\xc8\x62\x31\x48\x18\x1f\x81\xb3\xc6\xfb\xfb\x78\xb9\xb9\x24\x10\x28\x2e\x75\xad\x2c\x5f\x41\x63\xee\x93\x0b\xe0\xb2\x19\xbf\x47\xd1\x67\x8f\x49\xb9\x03\x1f\x57\xba\xb3\x5f\x92\x13\xbf\x92\x33\x21\xd0\xdd\x29\x7b\xaa\xd0\xd7\xdc\x5e\x1e\xb5\x35\xf2\xce\x44\xf6\x35\x32\x9b\x00\x69\xe9\x03\xc4\xb6\x53\xe2\xf2\xee\x32\xf9\xf8\x71\x12\xa3\xef\x86\xdd\x4c\x62\x38\x8a\xba\xf2\x7b\x4f\x4f\xec\x3f\xe2\x79\x36\x71\xbe\xc6\xfd\x2c\x01\x76\x5d\xa2\x91\x66\x61\x2a\xf2\x2b\x0d\x65\x86\x5c\xfd\x5d\x36\x02\xa4\xd7\xa5\xf2\xb3\x64\x3a\xf9\xed\x7c\x6e\xd4\xaa\x8e\xfd\xc9\x3a\xd6\x61\x50\x19\xe5\x84\x43\x5e\x29\x7c\xa3\xc0\x17\x45\x14\x9b\x20\xf7\x7d\xf2\x28\x28\xe0\x83\x5c\x7d\x71\x88\x3e\x55\x1a\x8d\xa7\x2a\x5b\x68\x97\x72\x9d\x6a\xb1\x53\x58\xf5\x14\xeb\x8e\x57\x80\x49\x61\x68\x0d\x7d\x87\x8b\xbb\x8f\x1a\x31\x0f\xbd\x2a\x88\xae\x7b\x75\xa1\x78\x48\x0c\x64\x09\x4f\x16\x85\xec\xf4\x90\x50\x5d\xdd\xf5\x5c\x2b\x29\xbb\xde\x3c\x63\x07\x8a\xb9\x01\x81\x5c\x98\x71\x0a\x1a\x4a\x5d\xc7\x14\x21\x4b\x27\x00\x55\x83\xdd\xbe\x0f\x15\x9f\x0e\x0e\x6a\x2a\x01\x3d\x6b\x70\x94\xc2\xa5\xf3\x64\xdd\x99\xb9\x30\x4f\xcd\x14\x4d\xea\x71\x5b\x22\x03\x9a\xc4\x4c\xa7\x2c\x69\xd6\xac\xe4\x96\xb7\xec\xa7\xb6\x25\x29\x83\x4f\x68\x4e\x05\x91\x69\x24\xb1\x7f\x83\x30\xdf\x4a\x05\x42\x1f\x3c\xdb\xaf\x71\xca\xa5\x87\xdc\x51\x96\x39\xd4\xb4\xb9\xac\x3a\xdc\x2c\x65\x1b\x26\x7c\x4c\x3f\xdc\xb4\x6b\xb7\xab\xd5\xa2\x7d\x74\xf4\xf0\xf0\xd0\x78\x68\x35\xe6\xcb\x9b\xa3\xa6\xef\xfb\x47\x90\x56\x13\x6b\x3c\x9e\x58\x06\x12\x9c\x9f\x9f\x1f\x51\x2a\x00\x4d\x92\xd9\xb8\x1a\x08\x53\x01\xa8\x1c\xcf\x9f\xbf\x7f\x83\x60\x2f\x8f\xc8\x77\x05\x1e\x52\x22\xd0\x59\x5a\x49\x17\xa5\x1e\xd5\xb6\xb4\x00\x03\xa3\x1b\x1b\x9f\x6d\xab\x5e\xe2\xfd\x7d\x34\x49\x46\x9b\x76\xe6\xf8\x92\x32\x60\xf0\xdb\x18\x30\x6d\x30\x60\xe4\x2a\x10\x26\xe8\x2b\x0c\x6c\xc8\x58\x29\x39\x8b\x85\xad\x9e\x5c\x02\xdd\xc8\x11\x16\x25\x79\x2b\x82\xf1\xb9\xdd\x47\xaa\xa9\x4d\xf3\x15\x27\xf5\xf8\x88\x56\x1b\x11\x6f\xf9\x6f\x55\xe7\x40\xbf\x76\xe5\x27\xd0\xca\x96\xe7\x9b\x14\x5d\x63\x42\xcd\xe0\x62\x18\x43\xf2\x4a\x92\x8f\x5f\xe6\x98\x29\xd5\x93\x58\xdd\x90\x6c\x65\x6e\x12\x03\x1a\x44\x6d\x97\xc4\xf4\x26\x73\xf4\xcc\x61\xd6\xd0\xa9\x33\x03\xe9\xb6\x0b\xa0\x8c\x4d\x3a\x99\x97\x33\x0e\x8a\x22\x5a\xc8\x1d\x6c\xa6\x07\x93\x3b\x3f\x9e\x41\x21\x98\x9c\x2b\xb9\x59\x0b\xcc\x9e\x18\x55\xa0\xb0\x18\x56\xa9\x7f\xc8\x7e\xb2\xdd\x5c\xff\xe6\x5c\x06\xed\xe1\x18\x49\x82\x6c\x53\xc7\x05\xae\x4d\x92\x95\x73\xf4\xd7\xa7\x77\xa9\x77\x74\xe3\xba\xf9\x13\x2c\x3c\xa7\x24\x4e\x12\x85\x6f\xf4\x59\x53\x7d\x2c\x65\xe4\xbc\x90\x33\xb6\xf9\x6c\x15\x25\xb3\x94\x4a\xa1\xe5\x37\x79\x33\x43\xfa\xc8\xd1\x7c\x0f\x8b\x2d\x53\x23\xec\x35\xd7\xc6\x9a\x61\xa4\xe4\x8f\xf4\x90\x83\xd8\xf9\x72\xb5\x64\xab\x2a\x49\x36\xcd\x0b\x7f\xff\x56\x96\x98\x7f\x65\x43\x2b\x2c\xfb\xb4\x35\x9d\x64\x7c\xe6\xc0\x23\xdf\xd0\x2b\x4c\xca\x80\x2f\xad\x8b\x59\xa6\xe9\x5f\x64\x9b\xbe\x79\x31\x03\x6d\xc2\x7c\xae\xd5\xe4\x73\x4f\x12\xed\x73\xac\x20\xc2\x34\x34\xf3\x42\x85\x7a\xaa\x56\x19\xa6\xe0\xdc\x07\xe8\x31\xa3\x81\x3c\x22\x0b\x8d\xfb\xf5\x7c\xba\x80\xa6\x1d\xd2\x51\x59\x27\x23\x88\x38\x4e\x23\x88\x52\x56\xb4\x34\xc4\xbd\xb3\xa3\xca\x03\xe2\x9f\x61\x39\x53\x5b\xc6\xbd\x4a\xce\x2f\x76\x9c\x36\xfd\xf4\x6e\x66\x93\xc7\x55\xbb\xc2\x78\xb5\x0d\xbc\x87\xec\x29\x7a\x7e\xa5\xf0\x69\x34\xfb\x48\x1f\x1e\xcd\x0f\x2b\x96\x90\x0a\xd7\xba\x4c\x81\xcf\x7b\xc8\x96\x4b\x46\xd2\x55\x70\xa9\x2f\xd2\x4e\xc1\xa7\x34\x4f\x90\x3f\x74\x6b\xb5\xf6\x87\x6d\x5b\x86\xf2\x78\x0b\xb9\x40\x36\xda\xbb\xd2\xe5\x26\x50\xdb\x6a\x1a\x3b\xbd\x8a\x35\x68\x63\xfc\xdf\x65\x0d\x3d\xa3\xff\xdd\xcf\xdf\xbf\xf9\x04\xc6\x98\x3c\x65\x6c\x31\xa9\x65\x4c\xd1\xa9\x95\x43\xff\x62\x11\xcf\x72\xc3\x92\x99\xf6\xf2\x80\x6d\xab\x41\xbc\x28\x30\x5f\xca\x71\x9b\x4a\xe2\xc4\x32\x57\xd0\x98\xca\xf8\xbf\xc6\x5b\x0f\x3c\x8e\x97\x32\x07\x17\xf9\x2b\x08\x84\x01\x27\x73\xd1\x91\xba\x40\x9f\x9e\xa2\xde\x9b\x4e\x06\xaf\x98\x8a\x49\x5c\xfb\x54\x86\x69\x91\xef\x60\x57\xd7\x46\x96\x57\x9e\xc8\xb3\xe9\x1d\x15\xe6\x45\xe0\x32\xcf\x7d\x5a\x20\x73\x72\xc7\xc6\x3b\xdb\x02\xd6\x5c\x8f\x3f\xf1\xc5\x6f\xfc\x95\x78\xb9\x25\x68\x6d\xaa\x9a\x04\x9c\x24\xe4\xfa\x89\xa0\xed\x65\xbd\x8f\x44\x0b\x9f\x3c\x5f\xa2\xbf\xdf\xd0\x6b\x43\x64\x15\x0b\x3d\xdd\xc0\xc8\xdc\x9c\x06\xa6\x44\x1a\x5e\x45\xcd\xfc\x50\x3f\x58\x35\x13\x53\x57\xdc\x2f\xf0\x7a\x14\x5d\x92\xc9\x9c\x76\xc2\x1d\xdf\x78\x59\x16\xbf\x4e\x56\xf9\xe8\x19\x2d\xdc\x11\xd5\x99\x1d\x4c\x24\x8c\xf6\xca\x78\x5b\xd1\xf2\x8a\x87\xb5\x2e\x4f\x51\x99\xf5\x19\x20\xfd\xee\x16\x98\x32\xf6\x0e\xa5\x99\x6f\xe9\xf9\x5a\x7e\xe2\xe5\xe6\x28\x40\xbb\x5e\x6f\x00\xba\x5d\x5d\x15\x3a\xbb\x05\x1f\xed\x1c\x78\x66\xc3\x91\xe6\x6a\x86\x3c\xbd\xf3\xcf\xc8\xb2\xa4\x4e\xcb\x48\xd5\x8d\xa1\x1b\xcc\xb4\x0f\x1f\xd3\x15\x5c\x9d\x1c\x15\xf6\xd6\x2c\xba\xb8\xb1\x9a\x4c\x11\x2e\x8a\x53\x7b\x92\x92\x76\x96\xcf\xd9\xfa\x3f\x3d\x39\xa6\x91\xe9\x1d\x38\xbb\xcf\x62\x4e\xfd\x16\x01\x21\x73\x73\xad\x56\x64\x0f\x81\xe5\x79\xce\x7d\xa1\x8a\x53\x78\xb8\xbc\xba\x8d\x52\x39\x09\xcf\xb4\x94\x9e\x9d\x9b\xb3\xee\x06\x9d\xcf\xf8\xb2\x22\x51\xe4\x35\xc6\x76\x9d\x72\xae\x89\x12\x06\xbb\xed\x4f\x62\x15\x1f\x9e\x9f\xca\x0d\xb1\x7d\xb2\xda\xd4\xc9\xec\x6a\x3f\xad\xb4\xf2\x5b\x83\xb4\xc1\xf4\x86\x16\xd9\x56\xef\xb5\x97\x1b\x2c\x08\x3b\x5a\x17\x60\x47\xe6\xd7\x2a\x18\x25\x4b\xbd\xc1\xae\x14\x08\xc7\x5b\xf8\x59\x51\x48\x70\x85\x9f\x6f\x29\x5a\x4b\x3f\x72\x25\xc6\x5e\xb9\xc1\xd9\x4a\xf9\x1d\x8a\x0f\x5a\x55\x48\xad\x93\x59\x7c\xf3\x7b\xae\x3e\x83\x5c\xb9\x98\x43\x28\x54\x9b\x9a\xc5\xaf\xcc\x11\xc5\xad\x3c\x48\x13\xe7\xd6\xa0\x28\xca\xc1\x75\x40\xc9\xef\xcc\x7a\x0d\xc6\x63\x3d\x0b\xb1\x58\xa1\x2a\xa3\x26\x73\xfc\xdf\xa8\x79\xae\x50\x22\x67\x09\x96\xf2\x10\xf9\x46\xb1\x75\x5d\xc9\xd5\x80\x2a\x94\xc6\xa5\x37\x37\x1e\xd7\xb6\x38\x0a\x52\xfc\x56\x35\x31\x55\xaf\x04\x08\xa2\x15\xcc\x8e\x41\xce\xbe\xf5\xb1\x8f\x35\xa8\xed\x70\xb3\x98\x29\x89\x6f\x57\xa5\x57\x8e\xb1\x7c\x07\xc0\xda\x16\xa7\x70\x6e\x2d\xb2\x6a\x45\xd2\x34\x9e\x9e\x1d\x72\x7e\x90\x27\x89\x38\x23\x27\xf2\xc2\x81\x2c\xc3\x5e\xa5\xf4\x73\x8b\x94\xd3\x8b\xbb\x0e\x3a\xb5\xfd\x7d\x16\x9d\xfd\xfc\x12\x27\xea\x13\x74\xc6\x5b\xe8\x16\x4c\x6a\xf5\xfa\x25\x80\xe4\x16\x87\x51\x23\x7e\xe6\xe2\x30\x9d\x59\x28\xdb\x54\x7d\x7e\x15\xd7\x6c\x02\x58\x4b\xb9\x76\x15\x51\x99\xe0\x8c\x4c\xd5\xad\x53\xaf\x27\x74\xa2\xb6\xa8\x09\x10\xee\xe0\x00\xff\xbe\x08\x59\x1b\xe0\xf7\x65\xd2\x9f\xc0\xdc\x94\x13\xec\x7e\x96\xb5\x46\x89\x47\x78\xeb\x9a\x8b\xcb\xac\xe2\xf2\xe1\x84\x8a\xc5\x8a\xb9\x6d\x5d\x0f\x60\x52\x8f\x57\x5e\x71\x55\xdc\x7c\xe7\x2c\x67\xc8\xf1\xb5\x4e\xab\x98\xed\x3c\xcf\x41\xc9\x2e\x3e\xcc\x63\x95\x9b\x39\x8a\x40\x2d\xe3\x54\x36\x1c\x5a\xc3\x16\xf1\x20\x2c\xfd\x68\x30\x2e\xdd\x27\xe2\xe7\xa0\x88\x4c\x33\x97\xd0\x6f\x3e\xa9\xac\x3b\x17\xcc\xab\x0d\x66\xcc\x55\x46\x07\x1f\x9d\xba\x49\xcd\xca\xbc\xc5\x22\x49\xa4\x2e\x9a\x19\x89\xe0\xf4\x3a\xb4\x64\x68\xca\x33\x93\x4a\x0e\x4c\x17\xab\x12\x77\x0b\x2f\xac\x49\x60\x55\x56\x92\xbf\x52\xa1\xaf\x52\x0a\x55\x5b\x17\x95\xdd\xda\x2c\x54\x6a\xf1\xff\x6f\xb8\x6b\x8e\xc2\xa9\xe6\x6e\xf0\xbd\xb5\x9c\x78\x55\xca\x69\xf2\x31\x43\x2d\x2f\xa0\xf9\x9d\x5d\x93\x23\xcf\x9b\x6d\xa1\x3f\x14\xb7\x07\x79\xe0\xfc\x29\xbb\x1b\x68\xed\x13\x56\x0d\xb5\xa2\x08\xd5\xc8\xdc\x20\xaf\x2a\x41\xec\x48\x53\x73\xf0\x5d\x13\xf4\x9d\xf9\x59\x06\x76\xc8\xc7\xce\xdc\x33\x79\x3a\xb5\x42\x42\x76\xe6\x25\xb9\xaf\xee\x12\x3b\xf3\x52\x83\x56\xb7\xf5\xee\xbc\x95\x9b\xb4\x4a\x44\x0b\x1b\xb5\xf7\x8b\x9d\x43\xaa\xd8\xb1\xff\x2a\xb3\x16\x86\x38\xb6\x2f\xc4\xa7\x8e\x94\xf9\x88\xdf\x60\x53\x56\x52\x48\xef\x44\x97\xee\xcf\x16\x2c\xe7\x1d\x7b\xb5\x55\xdb\xb1\xbf\x7a\x37\x36\xdf\x8a\xfb\x2c\xed\x94\x5e\x00\x29\x5b\xcf\x21\xdc\xaf\xed\x21\x91\x16\x39\xdc\x9d\xcb\x4a\x79\x17\x03\xb2\xf0\xca\xa5\x55\xf3\xdc\x5c\x89\xde\xcc\xee\x20\x27\xf2\x1d\x5b\x95\xe3\x35\xbf\x25\xf7\x7a\xf8\xf4\xe4\x79\xd9\x94\x61\x56\x12\xad\x52\xca\x30\x3c\x3d\x3d\xa2\xf7\x9a\xb6\x72\x9c\x82\x6e\xab\xf0\x59\x3b\x80\xc5\xdf\xeb\xc1\x7d\x3f\x19\xbc\x9e\xfd\x78\xbf\x92\x8f\xdb\xf9\xe6\x41\xbb\xe6\x89\xbf\xdd\x25\xe9\x3b\x44\xb9\xf3\xeb\xf7\xad\x87\x3b\xde\x14\xb4\xde\xfa\x13\xbb\x0d\x3b\xeb\x35\x4a\x4d\x2e\x3e\x48\x59\xbd\x80\x08\x82\xb1\xbc\x5f\x94\xcd\x13\x2a\xd6\xdb\x75\x16\x77\x6b\x34\x7e\x98\x31\x1b\x78\x00\x92\xdc\x29\x6e\x1f\xe0\x54\x5e\xa3\x74\x94\x1e\x55\x07\x7f\x78\xef\x52\x1f\xba\x2c\x68\x88\x92\x33\x42\xd9\x29\x1c\xe5\xe8\xd9\xd4\x65\x4e\x36\xcc\x68\x62\x6b\x51\xa8\xcd\x8f\x52\x4a\xd3\xae\x7d\xb4\x87\x73\x17\xc9\x35\x16\xcb\xe7\xd3\x5b\x18\x88\xf1\xf4\x8e\x39\xa2\x53\x72\x38\xca\xad\x38\xa6\x53\x79\xb8\x50\x0c\xa2\x05\x50\x19\x7f\xd6\x16\x1b\x1f\x43\xe3\x2d\x36\x85\x30\x7c\x11\xb0\xf0\x4b\xc4\x74\xb6\x0f\xe1\x2a\x84\x87\x0c\x40\x82\xa4\x93\x88\x57\x32\xd0\x33\x27\x0a\x77\x6c\xb1\x39\x33\x7b\x4b\xf7\x0a\xe6\x91\x30\xc7\xf3\xe8\x94\x20\x74\xa0\x59\xe3\xba\x23\xd1\x01\x55\xcf\x6c\x1a\x55\x32\xa6\xa4\x19\x00\xf8\x5b\x9a\x70\xa5\x21\x9f\x3e\x77\x1e\xa7\xf3\xfb\x34\x26\xa5\xda\xae\xd1\xf7\xfc\x43\xbc\xac\x09\xfa\x9c\xc4\xd1\x87\x58\x45\xdf\xaf\x6a\xdb\x6c\x2b\x19\x6c\x25\x37\x01\xdc\xc7\x1a\x56\x69\x2c\x5f\x6a\x51\xcd\x4d\xda\xa1\x0c\x83\x5a\xaa\x73\xcd\x75\x1b\x3a\x0f\x79\x3d\x5c\x46\x37\x7c\x62\x2c\xac\xa1\xcb\x29\xfc\xa2\x5b\x73\xb5\x2c\xe6\x2e\x0e\x54\xed\xe7\x8f\xe1\xf1\xb6\xad\xa8\x41\x9d\x96\x8c\xb7\x46\x83\x87\x29\xed\xf5\x10\xec\x4d\xe4\x0f\xd6\xdb\x9e\x33\xd1\xbe\xb8\x75\x1c\x1e\xcf\x32\x23\xcc\x4f\xf8\xae\xb9\x49\xd7\xd8\x2e\xe9\x8c\xb0\x63\xb5\x04\x81\x5f\xf7\xef\x6f\x8e\x8f\xfd\x97\xad\xf0\xe8\xdf\xe2\xfe\xff\x4c\x56\xd2\xc1\xb8\xd9\x76\x9d\x45\x1f\x92\x1b\x9a\x75\x21\x95\x5f\xdd\x60\x07\x41\x0f\x35\x44\xd6\x6a\x7e\x9f\x3d\x29\x68\x0a\x96\x49\xbb\x47\x55\x95\x3f\x4f\xa5\x42\xec\x2a\x34\x5d\xad\x4e\x54\x4a\xd6\xb9\x26\xc5\xca\xad\x09\xe4\x40\x58\xc5\x0f\x06\xd4\xd7\xa8\x31\xb5\x91\x0c\xa1\xe8\x64\x94\x40\xc7\xa3\x64\x2b\x82\xdf\xac\xdf\xba\xed\x2b\xf6\xda\xa1\x4e\x7e\x37\xb0\x71\x4a\x2f\x4f\x61\x82\xda\x28\x83\x14\x96\x58\x60\xe4\x8c\xb2\x48\x69\xc6\xb0\x4b\xc9\x54\x20\x3f\xa9\xca\xe9\x26\xc2\xf6\x84\x8a\x91\xc9\xd0\x3e\x8e\x28\x5f\x01\x1d\xdc\xe2\x45\xcc\xe1\xcf\xcc\x15\xf4\xef\x65\x88\xcf\xe5\x5f\xcc\xe5\x18\xc6\x6a\x34\xfb\xa2\xb2\xe2\xb8\x4c\x74\x95\xc3\x08\x5d\xc5\x85\x06\x5e\xd8\x0c\xc3\xd7\x51\xb7\x78\x64\x73\x9b\xe1\x00\xbf\xc1\x09\x83\xac\x2a\x54\x60\x97\x12\xd0\x1d\xca\x9e\xce\x36\x37\xcf\x00\x68\xa8\xb6\x98\xc2\x3c\xc1\x54\xe3\xd7\xf8\x82\xd5\x3a\x5c\xa0\xf7\xf7\xf9\x32\xb9\x49\x66\xd7\xe4\x63\x70\x03\x51\x81\x89\x0a\x60\x7a\x05\x74\x00\x6b\x9e\xc2\xe1\xfa\x69\xb8\x11\x32\x25\x5c\x30\x9e\x6b\xe7\x11\x75\x55\xbb\xc6\x6c\x5f\xb7\x11\xa3\xc7\x8f\xec\x21\xc2\x4d\x1b\xf1\xa9\x88\x00\x4a\x58\xb7\x87\x6b\x28\xa7\x3d\xdc\xd8\x4f\x6b\x42\x95\x88\xe4\x07\x6c\x4e\xa4\xdf\xab\x35\x6a\x1e\x22\x95\x87\x12\x20\x1a\x60\x72\xb1\x02\x3f\xdf\x42\x4f\x43\xab\x51\x12\x8a\xfa\x48\xb6\xaa\xf4\x61\x1b\x86\x14\xfc\x99\x1d\xd4\x96\xd0\x0d\x88\xf1\x9d\x4f\x64\x97\x7c\xe4\x50\xe6\x24\xfb\x52\x6d\xd2\x51\x54\x66\x0d\x9b\x10\x31\xf2\xc6\x7c\xa4\xae\x13\x58\xdb\x89\x56\xb9\x61\x8e\x2a\xd5\x0c\x21\xca\x23\xd5\x43\x6e\x2c\xcb\xf8\x2e\x73\x94\x09\xac\xd7\x3c\x19\x6d\xf8\x5f\xd9\xaa\xcc\x06\x98\xda\x3f\x64\xc7\x67\xd6\x44\x6e\x91\xc1\x24\x2e\x45\x0e\x53\x93\x64\x58\x1c\x66\x55\xf8\xfd\x62\xb1\x8c\xd3\xd4\x71\x3b\x4c\x54\xd7\x51\x2f\x45\x52\x50\xad\x2f\x15\xb8\x22\xa1\xae\xf8\xb7\xb1\xb6\x05\x50\xc6\x6d\x2c\x09\xec\xb9\x6d\x95\xc3\x17\x00\x52\x6c\x3f\x1e\x3c\xa0\x05\x79\xb5\x9d\x0e\xdd\x2b\x52\xbf\x91\xd7\x1b\x48\x3c\x84\x14\x53\x2b\x97\xd0\x12\xe0\x4a\xde\xf2\x79\x7d\xad\x60\x42\xd3\x11\xf1\xa8\xde\x7c\xbe\x10\x6a\x24\x11\xac\x96\x90\x7f\x35\xf9\x7d\xbf\xa8\xb9\x96\xf2\xb1\x32\x4b\xe5\x23\x32\x4a\x44\xb0\xe6\x92\x18\x58\x79\x20\x29\xfa\x14\x27\x64\x6f\x48\xb2\x74\x4f\xdf\xe5\xec\x57\xc2\xae\xa9\xd9\x80\x71\x14\xb4\xae\xd7\x30\x1b\x88\x33\x02\x87\x73\x39\x90\xfd\xfd\xdf\x79\x27\xf7\xa7\xd7\xe2\xef\xff\x27\x6c\x1e\xfe\xfd\xdf\xc5\x6d\x34\x19\x41\xf4\xdf\xff\xfd\xa8\x29\xfe\xf3\x3f\xc2\x20\xae\x9f\xc2\x6f\x33\xfc\xcf\xff\x38\xfc\xcf\xff\xc0\x01\x71\x89\x0f\xfb\xcf\x52\x84\x08\x5e\xfa\x64\x76\xc6\x37\xcb\x18\xc6\x21\x08\x1e\x01\x86\xbf\xff\x6f\x46\x7a\xf9\xbf\xde\xfe\xdc\x84\x50\x33\xc4\xbf\xc7\xe1\x31\x8e\x79\x64\xa9\x2f\xe6\xe8\xe8\xfb\x97\xf9\xdc\x5a\xce\x5f\xf8\x62\x11\x58\xca\xcc\x02\x74\x94\xd3\xea\x70\x75\x78\x49\xaf\xbd\xca\x65\xf7\xc1\x3c\xbd\x5d\xfa\xd8\xe6\xf8\xe5\x2c\x7d\x57\xdc\x87\x0f\xfe\x91\x03\x85\x1e\x0e\x03\xf7\xd0\x61\x88\x43\x9c\x85\x44\xb3\x5b\x88\x3f\x4c\x3d\x00\xab\xa3\xed\x92\xcc\x28\x8b\x62\xfa\xd5\xfd\xda\xf7\xee\x0f\x41\x4b\xdd\x6f\xe8\x63\x23\x1e\xfc\x43\x46\x70\xa4\x8a\x50\x08\x94\x73\x67\xca\xb4\x52\x99\x56\x32\x13\x55\x3f\x5e\x2f\x08\x1c\x60\xc9\x75\xfa\xda\x0f\x17\x3e\xca\x3a\x40\xe2\x17\x28\xc5\x07\xfa\x68\x42\xd4\x3a\x08\x17\x01\x27\xd2\x17\x26\xd2\x47\x13\x55\x67\x08\xe9\x75\x40\x80\x7a\x1a\x00\xea\x80\x41\x0c\x9b\xa0\x99\xa1\x60\x6f\xb8\xc1\x52\x87\x01\x33\x3d\x7d\x8f\x02\xdb\x74\x45\xdf\x0f\x9d\x87\xe0\xf0\x21\xa8\x03\x41\x0f\xbe\x07\x2d\x70\x08\xf1\x47\x4e\x13\x42\x87\x92\x41\xa2\x1f\x64\xa0\xea\x36\x54\xa0\xa1\x96\x72\xc3\x7f\x32\xbf\x71\x4c\x29\x7d\xff\xb0\xef\x7b\x81\x5b\xef\x03\xdf\x97\x41\x29\x48\x70\xd8\x0f\x08\x04\x9f\x6e\x5c\x86\xcb\xa0\xbe\xf4\xc5\x65\x08\x2d\xf8\xf4\xa4\xe1\x1f\x82\xa3\x07\x68\x87\xa3\xbf\xff\x6f\x7d\x04\xd7\x34\xbe\x7e\xe3\x1a\x44\xb1\x75\x78\x29\xac\xa4\xac\x4d\xf1\x31\x23\x4e\x96\x24\x61\x82\x73\xe3\x3e\xde\xb0\x0a\x94\xfd\xdb\xf4\x74\xa9\x04\xa1\x89\x15\xae\x6b\xcc\xf2\x6f\xb7\x71\x3c\x01\xb5\x88\xdf\xd2\xf0\x78\xc0\x28\x09\xae\x55\x80\x95\x04\x6a\x11\x46\x1a\xb2\x4c\x86\xfd\xc9\x60\x92\x0c\xc6\x32\xbb\x0a\xca\xcc\x46\x53\x58\x4a\x03\xd2\xcc\xc0\x88\x47\x66\xb3\x96\xc3\x95\x43\xa3\xf6\x87\x24\x7e\x68\xac\xdd\x23\xfa\x1d\x0b\x87\xc6\x6d\x0a\x6c\x54\xa4\x65\x4d\x90\x01\xe6\x4c\x34\x8e\x09\xa0\x38\x64\x28\x8f\x31\x89\x09\x20\xc8\x44\x6d\xac\xfc\xe4\x71\xf7\xe7\x39\x5e\x59\x62\x10\x79\xf2\x23\x5a\x3b\x96\x2b\x66\x14\x5c\x7d\x22\xc4\x8e\x07\x31\xc6\xb3\x42\xc6\x92\x52\x6f\x6d\x03\xca\x85\x00\xb2\x26\xa1\xa2\x50\x30\x39\x1e\xdb\x26\x48\x27\xc7\x6c\x3c\x36\x4d\x90\x4c\x83\x08\x38\x4d\x3e\x3e\xdc\xc7\x75\x70\x70\xb0\x0e\xb4\x9b\x73\x5f\xfa\x4f\xce\x59\xb8\x5a\x77\x3a\xeb\x1c\x07\xb7\x0c\x09\x19\xd9\xdd\xba\xeb\x8a\x0d\xa0\xdc\x68\x94\x9b\x0a\x94\xfa\x2e\xb0\xb3\xc9\xf1\x5f\xa2\xdc\x18\x94\xdb\x8c\x30\xca\xd6\x76\x48\x1d\xbb\x8f\x7c\x5b\x4f\x8e\x6b\x3a\x3d\xf3\x02\x39\xc6\x9a\x0c\xba\xf2\xa2\x90\xb5\xc6\x3e\x9d\xdb\x52\x3a\x34\xbf\xdb\x57\xb2\xb5\x55\x0b\xe7\x91\x93\x1d\x56\x45\x90\x34\x94\x2c\x43\x53\x77\x9d\x32\xfb\x53\x99\x22\x81\xb0\xdb\x5b\x8d\x9e\xd2\x32\x70\xa1\xf9\x85\x5d\xaf\xeb\x6a\xb3\x50\xf5\x34\x6d\xcd\xe0\x13\xc5\xec\xb0\x3f\xd7\xf9\xda\xda\x6a\x94\xc3\xf2\xaf\x30\x18\x73\x7c\xb9\x66\x4b\xd1\x36\x11\x7f\x9d\x25\xb8\x9f\xc5\x66\xb4\xcc\x07\x5b\xf3\x40\xcd\xa4\xa1\x36\x09\xb5\xae\xc8\xb3\x78\x3f\x2b\xae\x7c\x25\x4d\xae\xb4\x5a\xbc\xc8\x88\xed\x75\x7e\x7e\xa5\x52\xed\x37\xea\x63\xa2\x2c\x96\x53\x12\x33\x5d\xd5\xd3\xa5\xdc\xe3\x11\x24\xbb\x7e\xa8\x84\x97\x81\x8a\x4b\x12\xf8\x78\xba\x35\x8b\xc2\x85\x09\xc5\x82\xd4\x27\x2f\x73\x2a\x70\x65\xc3\xf5\x0c\xa3\xb0\x4b\xea\xd9\xb1\xa5\xed\x4c\x1d\xd4\xa3\xb9\x3c\x41\x0c\x2b\x66\x8c\x72\x2b\x4d\x46\xe6\xef\xb0\x5b\x74\x48\x08\x74\xf5\x64\x13\x44\x2f\x34\xdb\xbc\x31\x2c\x13\xe8\x62\xdb\x2c\x19\x2b\xe7\x8f\x8a\x29\xd6\x95\xd6\x13\xdf\xbf\x00\x98\x3a\x25\xd1\x9b\xf4\x72\x06\xb8\xd2\xd3\x5a\x4b\x48\x80\x18\x7b\x0a\xda\xeb\x28\x15\xdf\x94\x83\x80\x2b\xf2\x5a\x5a\x2f\x83\xfc\xb4\x64\xdb\x3a\x1e\x45\xf7\x13\x7a\x77\x27\xdb\x7f\x35\x05\x21\xd0\xa3\xdf\xe1\xcd\x92\x7c\x11\x94\x90\xf7\x5e\x07\x68\xaa\xc8\x63\xc0\x7b\x7b\x66\xfa\x9e\xe6\xa4\x09\xbb\x3e\xf2\x6d\xd3\x67\xbb\x2d\x28\x20\xcb\x9f\xdc\x04\xcd\x4a\x31\x09\x44\xb5\xf4\xa9\x37\x91\x33\x64\xaa\x46\x84\xac\xd4\x4a\x78\x01\x62\x11\x68\x32\x93\x1e\x26\xd8\x3c\x0d\x6c\xa6\x52\xbb\x4c\x7c\xb7\xbf\x8c\xa3\x71\x87\x5d\xd1\x4d\xfc\x70\x12\xe0\xe2\xe1\x44\x72\x40\xd5\x06\x4c\x30\xf3\x49\x56\x60\x9d\x0c\x45\xf7\x50\x47\x7b\x59\x88\x80\x20\x02\x1b\x82\xf5\x7e\x10\x6a\x16\x1d\x1c\x58\xf6\xa0\x82\x3a\xd2\xc9\x2e\x52\x05\x26\x05\x16\xe4\x51\xa1\x2e\x18\xfc\x0e\xa1\xf5\xa8\x08\x08\xf7\x90\xe8\x2b\xa8\x08\x02\x4d\x34\xd0\x84\x80\x26\x1a\x48\x1b\x09\x44\xc2\x21\xf7\xe2\x8c\x34\xe0\xcc\x2a\x23\x55\xd8\x30\xcf\xab\x7f\x34\xf5\xb5\xd2\xcc\x09\xfe\x6f\xd2\x43\xa5\xe3\x85\x67\x3b\xaa\xd9\xf1\xab\xd0\x3a\x6a\x73\x9c\x1c\x7c\x9a\x4e\xbc\x7d\xd0\xd6\x1e\x29\x71\x3d\x3a\xa9\x29\x9f\x1c\x9e\x56\xbb\x4d\xd3\x2a\x73\x31\x33\xac\x55\x0d\x58\xdc\x1c\xe2\xd3\xc6\x2d\x53\x33\xf2\xd2\xa0\xfb\x9d\xaf\xd7\x2f\xf6\x5a\x43\x13\xba\xea\xa1\x99\xf1\x92\x49\x6c\x2d\x6d\x28\x5e\x84\x7a\x22\x9c\x87\xa8\x1e\x1f\x0d\x6b\xcd\xf8\xa8\x79\x2b\x07\x48\x7b\xbf\x42\x76\xf9\x2c\xbb\xb3\xad\x62\xf9\xa9\xdb\x77\x1d\x64\x9f\x11\x54\x8f\x2d\xbb\xc7\x52\x7b\x9e\xa1\xfc\x71\xe5\x5b\x2b\x77\x72\xc7\x64\xc1\x77\xd1\x96\xdd\xc1\x84\xde\x25\x9a\xe2\x4a\xbf\x93\x4b\x74\xdb\x15\x1b\x67\x66\x63\xb5\x94\x40\x57\xe4\x10\x85\x29\x3f\xc3\x86\x65\x58\x93\xae\x3c\x14\xf5\xfc\xa2\x5c\x8a\x13\xbf\x7a\x54\xe9\x98\x75\xe8\x01\xed\x67\x3c\x3d\x19\xc3\x06\x09\xec\x68\x55\xe2\x3f\x3d\x39\x26\x60\xc6\x77\xca\xed\xba\x5a\x33\x91\x2a\x5c\xc0\x38\xda\x14\x0d\xdf\x6f\x1e\xe6\xe7\x7a\xdf\xc4\x93\x55\xe4\xb8\x6e\xf9\x10\x88\xc8\x4c\xcc\x0e\xbd\x95\x33\x46\x71\x21\xdb\x10\x47\x67\x65\xcc\x32\xaf\x9e\x10\xee\xd7\xca\xf4\x76\xbe\xcd\x04\xdb\xee\x5b\xb8\x62\x6c\xa6\xdf\xb2\x0e\x47\x14\xf1\xe6\x87\x66\xa7\xa4\x3d\xcb\x38\xa3\x7b\x74\x7a\x9b\x8c\xc8\x79\x13\x25\xd2\x5b\x3f\x63\xb7\x1e\xb4\x2d\xf7\xcc\x63\x97\xae\xbb\x16\x2c\x85\x2c\x67\xaa\x4c\x68\xcd\x14\xc1\x0d\x9c\x17\x2e\xa1\x87\x0e\xb1\xf6\xc5\x3a\x10\x1b\x1f\x26\x65\x34\x77\x09\x1f\xd7\x6d\x08\xc0\xbf\x71\x3b\xd8\x0a\x3a\x93\x72\x75\x7e\xea\x83\x44\xf9\x72\x24\xfa\xa3\x7e\x2d\x28\xd3\xc6\xaf\xd9\x8d\xe4\xc6\x5a\xe2\xb3\xb6\x17\xac\x59\x3e\xab\x2b\xfd\x69\x27\xdd\x2f\x42\xb5\xde\x27\xa3\xad\x35\x3f\x7b\x37\x82\x13\x2b\x96\x24\x31\x51\x58\x33\x3d\x21\xa7\x6e\x7a\xae\xa5\x0d\x62\x8c\x90\xde\x44\x74\x27\xa3\x95\x8c\xb2\x5b\x25\xcf\x49\x10\xb2\x2f\x20\xe3\xba\x53\x7e\xd0\xa1\x6b\xe9\x49\x14\x31\xeb\xf8\x04\x4c\x7e\xf9\x66\x95\x5d\x3d\xbb\x7c\x6c\x1a\xda\xb5\xb9\xbe\x46\x47\x41\xab\xeb\xeb\xa7\xa7\x6c\x53\x95\xa9\x3d\x28\xe2\x21\x8e\x67\x0e\xd5\xbc\x5d\xc4\x8a\x56\xd2\x3a\xc4\x46\x96\xb6\x20\x7d\x82\x8d\x38\x58\x83\x09\x08\xb6\xc8\x60\x13\x0e\x37\xf0\x4b\x8f\xd4\xe5\xd6\x16\x9d\x2b\x67\x90\x9f\xed\x0b\x67\x90\x9f\xad\x83\xbd\xa9\xd6\x4d\x84\xce\x11\xa8\x2c\x81\x95\x27\x50\x99\x02\x93\x0b\x3e\x8d\xcf\x20\x7b\x86\x42\xef\x1c\x86\x09\x7c\xa2\x83\xd2\xf5\xd1\xe4\xaa\xd9\xeb\x64\x39\x14\x2a\x81\x86\x42\x69\x61\x66\x0c\xdc\x82\xb2\x68\x45\x66\x0c\x5c\x1b\x6f\xf3\xaa\x66\xab\xda\x41\x8d\x92\x19\x7e\x15\xbb\x1b\xe8\xfb\x92\x32\xcb\xda\xe2\xd9\xae\x4b\xa7\x29\x49\x20\x75\xff\x35\x42\x79\xbd\x63\x6d\x5a\xd5\xd2\xbb\xe6\x9d\x21\x0f\x57\xf7\xa1\x76\x72\xb1\x44\x98\xe5\x0c\xc4\xee\x16\x96\x2a\xb8\x50\x82\xf9\xb4\x02\x25\x9e\x0d\x7f\x6c\xa0\x48\xef\xba\x58\x9c\x22\xc3\x94\xf2\xc7\xdc\xdb\x0d\xbb\xca\xb2\x33\xf0\x46\xce\x75\xb7\x4a\xf3\xb4\xaf\x98\x05\xc4\x80\x9e\x2c\xde\x42\x20\x69\x60\x9d\xb8\x5f\xf1\x12\xf6\xfa\xe0\xa0\x0c\x37\xa7\xaa\xaa\x65\xce\x63\xee\xac\x13\x42\x56\xa0\xc4\x34\x89\xd0\x7a\x17\xe4\xe3\x0e\x6c\xeb\x20\xfc\x08\x6a\x3c\xfc\xa8\x1e\x0f\x2b\x51\xe3\x8c\x7b\x2d\xbf\x1a\x9b\xfd\x30\x6f\x10\xf3\x66\x1f\xcc\x9b\xc0\xde\x0b\x21\xfd\x5b\xd8\x0b\x29\xb5\x09\x44\x55\x5b\xe2\xa6\x54\x70\xc4\xcf\xb3\x15\x97\x8d\xf1\x80\x03\x0d\x67\xf9\xa3\x0d\x85\x55\x66\x2a\xa5\x78\xd0\xa9\xae\x07\xe3\x21\x02\xfc\xe5\xd0\xc9\x46\x7c\x8f\xe7\x95\x82\xa6\xdf\x0e\xa0\x53\xd6\xb8\x28\xb7\x0d\xa5\x9a\x91\xf4\x73\x8b\x36\xbb\xe3\x84\x86\xa0\xb6\x72\x9b\x4b\x15\xf4\x39\xd5\x58\x45\xc9\x64\x4b\x3e\xd2\xbe\xa7\xa3\x05\xc9\x3a\x9e\x5c\x0e\xd0\xa5\x60\x8d\x2c\xe8\xaf\xe7\x93\xb9\x75\x38\xd7\xbc\xbc\x5a\x71\x0a\x6c\x79\xd3\x77\x5c\xaf\x56\xa3\x96\xbd\x4d\xad\xd3\x54\xb7\x22\x15\x7a\xc9\xfb\x8b\xb2\x27\x33\xba\xb7\x39\x37\xca\xdf\xa5\x13\xec\xb6\x80\xc6\xb9\x6d\xdc\x0a\x98\x37\xc3\xbf\x89\x8b\xa7\x50\xa0\x1c\x7e\xdf\xd7\xa9\xd5\xbc\x5b\x21\x63\x00\x52\x70\x06\x02\xc2\x8c\x90\xe8\xa5\xc2\x9b\x18\x79\x82\x58\xdb\xaf\x11\x96\x62\x9d\x3e\x96\xf7\x15\xa9\xde\x9d\x1c\x74\xa3\x4f\xaf\x07\x64\x7c\x8e\x1a\xff\x91\xa1\xb6\xda\x1a\x67\x05\x8f\xdb\x5d\x90\x7b\x57\xd2\xc6\xca\xff\x56\x5e\x95\xe5\x9f\xc9\xd1\x98\x8f\xde\x65\xca\x1b\x46\xcb\xf1\x6f\x5c\xda\xf8\x90\xcb\x2b\x29\x0d\x38\x58\xee\x25\x0a\xa0\x90\xbd\xa5\x74\xf3\xe9\xb9\xdb\x41\xa6\xa5\x07\x9f\xdc\xd2\x03\x6e\xe9\x81\x6a\xe9\x01\xb7\x74\x1e\xee\x4d\xd4\x47\xb8\x49\xd4\x97\xb0\x13\x80\x8b\xe0\x5f\x9f\x1a\x5c\xc5\x3b\xb7\xa1\x94\x08\x88\xe1\x10\x49\xe6\xad\xeb\x36\x96\x00\x7d\x43\x39\xdc\x5c\x6e\xcc\x89\xe2\x32\xc8\x8a\xcb\x20\x27\x2e\x83\x9d\xe2\x32\xd8\x53\x5c\x64\x79\x36\x4b\x07\x66\xbb\x26\xf0\x7d\xc9\x5f\x4f\x56\xeb\x7f\x1e\x16\xbc\x2e\x50\x2b\x4b\x0f\x06\x99\x72\xab\xc5\x66\x47\xa9\xd1\xda\x51\x65\xd6\x3f\xa3\xcc\x6a\xe1\x19\x4c\xa8\x15\xb2\x65\x4a\xe1\x61\x7d\x41\x22\x04\x30\xd6\x63\x7d\xc2\x76\x47\x5b\x2a\x42\x93\x4a\xd1\x70\x26\xd0\xb0\x13\x68\xd8\x09\x36\x6c\x1e\xce\x88\x9a\x05\x0b\x02\xdb\xb8\xd5\x8a\x85\x84\x66\xa2\x84\x66\x42\x42\x33\x01\xa1\x99\x18\x31\x73\xbc\x89\xf0\x22\xe1\xf5\x8d\xa0\x10\xc7\xc2\xe0\xa5\x90\xdf\x7f\x0e\x1b\xe7\x27\xfe\xf1\x99\x0a\xff\x25\x0c\xd4\xe7\x2f\x61\xd0\xf0\x5f\xbe\x7c\xd9\x92\x11\x19\x01\x83\x8a\xec\x10\x30\x1b\xfa\x39\x01\x43\x3a\x3f\x47\xa4\x18\x2e\xe2\x9f\x3e\x37\x76\xa6\xdc\x9d\x02\x66\x4a\xfd\x14\x91\x7a\xb6\xcc\x4a\x01\x43\xb4\x5a\x3b\x4d\x4a\xf0\x64\xb3\x2e\xc5\xcd\x73\xa2\xb5\xcc\x89\xcc\xdb\x1b\x12\x2d\x2c\x64\x09\xa2\xb0\x04\x51\x58\x4a\x51\xc8\x8c\x43\x4b\x39\x0e\x09\xa3\x33\x15\x90\xf3\xb7\xbf\x2d\xc5\xdf\xfe\x76\x03\xff\x2c\x91\x81\x84\x4c\xd3\x43\x41\x3b\x9a\xde\x86\x2e\x6f\xfa\x3d\x46\x05\x2a\x7a\xc9\xb3\x42\x60\x05\x7f\x00\x47\xf8\x03\xdf\x8e\x6b\xe9\xdb\x40\xcb\xa7\xa7\x9b\xa7\xa7\x7e\xd7\x59\x1e\x1c\x24\x17\x78\x2b\x72\x19\x26\xae\xb8\xc1\x10\xfc\x71\x6e\x30\xd4\xc7\x10\x3e\x0f\xd6\xc7\x90\xac\xad\x96\xba\xe6\xc9\x09\x54\xd9\x59\xc2\xf0\xe6\x8a\x7c\xec\x4d\x69\x6c\x1f\x63\x5d\xcd\xb8\x04\x4f\xb6\xb3\x3c\x64\x18\xf0\xab\xc7\x46\x6e\x15\x47\x0e\x88\x4b\x57\x98\xc0\x8d\x1d\xe8\xbb\x25\xa5\x67\x4c\x1b\x5b\x1a\xa5\x19\xe2\x48\x06\x4b\xf6\xda\x52\x9d\xc1\x53\x6d\x4f\xd5\xfe\xa9\xe6\x29\x7c\xf1\x5a\xe2\x73\x0b\x51\x37\xc5\xa8\xac\x80\x5d\x93\x1f\x37\x7d\x20\x39\xc2\x07\x5c\xfb\xf8\x04\x40\x70\x72\x76\xde\x6c\x81\x0e\xc2\x37\xb2\xde\xdf\x83\x19\x99\xac\x20\xf6\xf4\xf8\xf8\xb8\x75\x76\x22\xa2\xf7\xf7\x51\xfb\xf4\xe4\xa4\xc5\x9f\xd3\x08\xc8\x8c\xdb\x2f\x5b\x2f\x5f\x9e\x9c\x1e\x8b\xe8\xe3\xfd\x92\x51\x1c\x07\x00\xdc\x8f\x93\x1b\xcc\x1b\x04\xe7\xcd\x53\x5f\xf4\x93\xf4\x3d\x96\x70\x7a\x76\xe6\x37\x8f\x8f\x45\x7f\x12\x0d\xc6\x60\xfc\xc3\xef\x6c\x70\x1b\x0f\xa3\xc9\x74\x3e\x1b\x52\x7a\xd3\x3f\x86\xec\x48\x0f\x36\x3f\x7e\x80\xf9\x3a\x89\x57\xed\x73\xff\xe4\xa4\xe9\x37\x45\x7f\x39\x7f\x98\xb5\x03\xff\x65\xf3\xb8\xd9\x02\x54\xf7\xcb\xc9\xe6\x61\x3e\x87\xdc\xc7\x27\xe7\xa7\xcd\x56\x20\x06\x11\xd8\xb1\x84\xe2\xb4\x79\x7a\x7a\xd2\x7c\x29\x68\x5a\xbd\x8c\xc1\xa0\x25\x82\x5b\x27\x4d\x88\x9a\x0f\x68\x11\xa2\x1d\xb4\xce\x5e\x9e\x1f\x9f\xf9\x62\x30\x5f\x46\x13\x24\xe2\xf8\xb8\x79\xd6\xc4\xe0\x6c\x34\x99\x3f\xc4\x4b\xc6\x75\x72\x1e\x9c\xbf\x0c\x28\x3a\x4d\x26\x63\xa2\xf6\x04\xb0\x89\xc1\x32\x99\xa6\x73\xa0\x09\xf2\xb5\x50\xad\x0e\x36\xd1\x4c\xb2\x0a\xa5\x92\xb9\xdb\x3a\xa7\x00\xa5\xb5\x4e\xce\x9a\x2d\x0a\xde\xcc\x27\xc3\x78\xb6\x44\xf2\x9b\xfe\x79\xf3\x5c\x42\xdd\x2c\xa3\x4d\x3b\x80\xff\x9d\xfb\xc1\x99\x8c\x89\x63\xbc\x69\x72\x0a\xf8\x65\x38\x07\x31\xbe\x8d\xc6\x09\xa0\x39\x6e\xb5\x9a\x27\x8c\x66\x8a\x87\xe7\x56\x51\xfb\x3c\xf0\xcf\x4f\x8f\xb9\xc4\xf9\x24\xf9\x10\x33\xb6\x93\x93\xf3\xb3\xf3\x73\x06\x9d\xf3\x1b\xd9\x58\xfb\x33\xe0\xb3\x8c\xc3\x43\x71\xc0\x6b\xff\xd8\xf7\x83\x26\xc5\x2d\xe3\x21\xa1\x83\x91\x8c\xc2\x29\xb5\x1d\xb4\x7c\xcb\x7f\x79\x1c\x70\xbe\x34\x8e\xb8\x00\x10\x86\x73\xe0\x1a\x47\x22\xb3\x89\x15\xc7\x67\xad\xe3\xd6\xf1\x99\x89\xa5\xda\x22\xe7\x8e\xcf\x4f\xec\xd8\x38\x1b\x0b\xbd\xe0\xfd\xfd\x3c\x81\x46\x3c\x69\x9e\x1f\x73\x9c\x12\x8e\xd3\xf3\xf3\x13\xe4\x5d\x1c\x2f\x16\xe8\xeb\x10\xea\x11\x9c\x9e\x63\x21\x10\x93\x8e\x37\x5c\xf0\x79\x70\x02\x63\x6e\x32\xa5\x02\x4f\xcf\x41\x86\x4e\x4f\x38\x1c\x5b\xe1\xf9\xf0\x46\xb6\x79\xd3\xf7\x5b\x50\x03\x31\x4a\x60\x52\xba\x4c\x40\x66\x03\x64\x50\x70\x7c\x2a\x40\x32\x40\x5a\x54\x1f\x01\x49\x38\x07\xa6\xe1\x35\xa7\x74\x25\x9b\xaa\x79\xda\x7a\x79\xdc\x14\x23\x5c\x6e\x4c\x22\xa2\x28\x38\x07\x91\xb8\x41\x6f\x6f\xfd\xf9\x72\x8e\x02\x03\xb2\x06\xfd\xe3\xe6\x76\x9e\xae\x14\xae\x56\x70\x0a\xa0\x02\x25\x03\x33\x41\x00\x30\x5b\x72\x72\xdc\x6a\x9e\x07\x18\x85\x95\x80\x12\x02\x6c\x0a\x2e\xb3\xd5\x3c\x3b\x7d\xc9\xdf\x9b\x78\x02\xb2\x0b\xf4\x1e\xfb\x2d\xe8\x39\x82\xaa\xa8\xa0\x6f\xe7\xb3\x78\x33\x8c\x1f\x64\x87\x05\x0a\x6e\xe7\x2b\xc5\xb7\xd6\xcb\xb3\x63\x1f\x9f\x1c\x4b\xa2\x19\xb6\x76\xd0\x3a\x3e\x79\x79\xd2\x3c\xa6\xa8\x9b\x39\x71\xb1\xd5\x02\x88\x0f\xf3\xe5\x86\xea\x0e\x04\xc2\x44\x9e\xc5\xef\xe4\xec\x25\x90\xec\x8b\x49\x84\x17\x03\xe2\x25\xc4\x04\xad\x26\x4a\x86\x8a\x01\xce\xa6\xb7\x94\xaf\xd5\x02\x76\x4f\xa2\x87\x19\x53\xff\x12\x64\xf9\xfc\xec\x54\x4c\x62\x90\x28\x90\xbc\xd1\x08\x05\x0b\x79\x0b\x3a\x46\x4c\x70\x7c\xe3\xae\x04\x7d\x09\x44\xfc\x98\xa3\x64\xaf\x3d\x39\x3b\x05\xb2\x4e\x65\x1c\x76\xb2\x00\x98\x0b\x12\x7e\xce\x51\x9a\x81\x8a\x31\xa0\xd7\x5e\x36\x91\x2c\x4a\xa5\xfe\x06\x9d\xb9\xd9\x82\x8e\x29\xa3\x58\x82\xcf\x5f\x42\xa7\xd3\x51\x79\x28\xc5\xb4\x93\x97\xc7\xa7\x92\x46\xd5\x23\x20\x12\x9a\xa3\x29\x23\x55\x97\x68\x06\xc7\xcd\x97\xe7\xb2\x58\x25\x98\x10\xe1\xb7\x8e\x65\x29\xa6\x4b\x9c\xbd\x6c\x81\xe6\x6d\x65\xa2\xe3\x7c\xf4\x0a\x26\xfb\x92\x2d\x40\x04\x74\x2d\x8e\xd7\xd5\x84\xe6\x09\x5e\x62\xe4\x14\x75\x58\xf3\xa5\x4f\x9f\x52\x5e\x40\x94\xb0\x29\xf1\xf1\xf7\x19\xb1\xe4\xe4\x14\x14\xa1\x52\x1b\x5a\x64\x41\xd9\xcf\xa1\x4a\xa8\x3b\x4f\xfd\x97\x02\x5f\x6f\xbc\x9f\x5a\xa3\x00\x08\xcd\x59\xab\xd9\x94\x09\xb2\xeb\x9c\xc8\xa0\xd2\x22\xcd\x66\x80\x92\x2d\x63\x17\xf7\xcb\xc5\x24\x86\x8e\x0b\x3a\x1a\xc6\x1c\x8e\xd4\x5c\x6a\x9d\x9f\xbd\x04\x59\x50\xd1\x5a\x75\xbc\xf4\x5f\x9e\x9d\x01\xf7\x64\xfc\x02\x87\x4a\xce\x71\x7a\x1c\x80\x44\x70\xbc\x51\x14\xc7\x20\x9b\x2d\x5f\xc1\xb3\xb2\x60\x99\x06\x43\x3c\x38\x6b\xe1\xbb\x84\x33\x23\x58\xc0\x00\xe8\x5a\x10\x39\x5b\xa1\xab\xab\x29\x8e\x60\xcd\xe0\xe5\x09\x20\x48\xd2\xd5\x66\x39\x4f\xd5\x20\x86\x59\xe7\x83\x41\x94\x26\x33\x19\xd3\x3c\x17\xb3\xe8\x43\x74\x37\xd7\x3a\xe1\xf4\xe5\x29\xc8\x2d\x44\x82\xd0\xc0\x20\x04\x02\x88\x4e\xc4\x21\x05\x54\xf1\xc9\x09\x46\x80\x26\xa6\x3e\xd9\x02\xa9\xa7\xd0\x70\x19\xf5\xdb\x67\xfe\xf1\xcb\x33\x50\x66\x46\x25\x83\x6a\x83\x0e\xcf\x61\x22\x1f\x74\xc2\x79\x0b\x06\x52\xc5\xdb\xe3\x16\x74\x00\x68\xfa\x45\x34\x89\x2d\x55\x71\x72\x7a\x72\x06\x55\xe5\x68\x62\x13\xa8\xd3\x26\x74\x27\x8e\x32\x7c\x02\xd9\x69\x9e\x43\x5b\x50\xb4\xc5\xa6\xe3\xd6\x4b\x50\x35\x2d\x88\x5e\x44\x9b\x08\x6a\xb6\xe0\x8e\xeb\x9f\x9d\x89\x05\xae\x71\x2f\xee\x47\x23\xaa\x2b\xfc\xbf\x85\xaf\x10\xde\xa3\xbe\x38\x7d\x09\x6a\x5f\xa8\xbe\x71\x1a\xf8\x20\x43\x8b\xc9\xfd\x14\xc7\xe8\xe6\xf1\x69\x0b\x32\xcf\x1f\x86\x52\xc9\x42\xd9\x30\x46\x40\x4f\x94\x22\x81\x52\x76\x06\x3d\x59\x56\x14\x84\x06\xc8\x05\xd6\x6f\xe4\xc8\xdf\x84\xd1\xf3\x04\x06\x95\xe5\x7c\x13\xb1\xe4\x43\x8f\x3a\xc5\x01\x21\x8d\x86\xc3\x49\xcc\x60\xd0\x8e\x20\xf7\x67\x42\xf7\x46\x50\x73\xd0\x71\x21\x3c\x1b\x2a\x4c\xa7\x7e\x0b\x72\x1e\x0b\x23\x76\xfe\x09\x44\x9d\x61\x44\x7a\x0b\x1d\x88\x2a\x0b\xf4\xbe\x14\x69\x12\xcf\x66\xd0\x23\x00\xe0\xf4\x0c\x04\x13\x2c\x80\x0f\xa8\xdc\x40\xc9\x37\x51\x3f\x64\x7a\x32\xd8\x20\x46\x64\x4f\xcf\xcf\x7c\xff\x54\xc6\x70\xb7\x6e\x41\xeb\x41\xe3\x59\x3d\x5a\xc5\xcc\x64\x97\x3d\x39\x87\x56\xcb\x88\xf7\xc9\xb1\x7f\x26\x4c\x67\x3f\x3e\x05\x73\xe1\x25\x1e\x18\x98\xa1\x46\x82\x6e\x81\x81\x18\x34\x21\x54\xe9\xfc\x94\x0c\xcc\x15\x30\x13\xb4\x0d\x48\x13\x18\x29\xab\xf9\x34\x5a\xcd\x49\xbf\x9f\xc1\xe8\x2d\xac\x3e\xd2\x3c\x01\x11\x3f\x15\x72\x28\x05\xa1\x81\x41\xf7\xe5\xa9\x78\xb8\x8d\xa3\x15\xd9\x70\x2d\xac\x91\x19\xea\xce\x60\x10\xe1\x60\x3a\x9d\x8f\x95\x99\x07\xa2\x6e\xe9\x9c\x53\xdc\xed\xe3\xb0\x12\x3c\x68\x7b\xff\xec\x98\x2f\xb5\x69\x3b\x74\xe7\x83\x55\x19\x40\xf5\xd0\x19\x47\xfe\xc0\xef\xe0\x2a\xa7\x8d\xe4\xdb\x9c\x50\xb0\x5b\x0d\xf9\x89\xb1\xeb\x5b\x8a\x81\x1f\x7a\x83\x08\x3e\xf9\xe4\xc3\x6a\x43\x99\x86\xe9\x07\xeb\xf5\x97\x18\x74\x63\x42\x7b\x9d\xa0\x23\xf9\x11\x1a\xb3\x19\x9c\x7e\x70\xc0\xd8\x04\xb9\x7b\x10\xc6\x17\x45\xe9\x33\x7b\x2a\x39\x44\x58\xf8\xc7\x87\x93\xc8\x08\x67\x72\x90\x2a\x42\xa6\xca\x31\x18\xd5\x64\x0b\x20\x1a\x98\x75\x9f\xcd\x00\x82\x8d\xd3\x05\x5e\xd8\x93\x8f\x27\x38\x98\xf7\xda\xed\xaa\xe8\x36\x4e\x2f\x86\x6f\x15\x10\xba\x01\x07\x88\xad\x90\xc5\x20\xb8\x9b\x39\xb9\xcb\x70\xf8\xa6\x41\x9c\xae\xcc\x8c\x26\xfd\xd0\xe0\xf9\xad\x4c\xd1\xe5\xfe\x8c\x5e\x52\xac\xe3\x86\x99\xe2\x46\xc5\x2b\x51\x9f\x86\x59\x8c\x5c\xeb\x08\x19\x08\x0d\x48\xf3\x5b\x49\xb6\x79\xf8\xe6\x81\x0f\xff\x52\x2a\xbb\x20\x6b\xdc\xcd\x13\xab\x5d\xdd\x3c\x0e\xf6\xa1\x8a\x4e\x2e\x0d\x9a\xf8\x5b\x4a\xe2\x5b\x82\x94\xd4\xfd\x97\xda\xbf\x78\xf8\xa5\x5f\x82\x38\x7a\x57\x3b\xba\x11\xff\x52\xab\xfd\x8b\xeb\x41\x62\x1b\x13\x69\xdb\x5c\xe5\xa6\xd9\xfa\xdb\xf8\xe6\x8f\xeb\x85\xf3\x2f\x57\x90\x5d\xd3\xe0\xd5\xde\xcd\x7a\x78\x09\x52\x45\xf0\x43\x66\x2a\x94\x7b\xc4\x2c\xcf\x1d\xeb\x7e\x2e\xb3\x85\xf6\x30\xe7\x05\x38\x60\x4e\x2a\x41\x34\xc7\x41\x18\x13\x3a\xaf\x35\x57\x27\xa1\xe6\x1c\x59\x97\xeb\x00\x11\x51\xfd\xad\xca\x50\x1b\xd6\x44\x4d\x42\x3e\xd6\x3c\xcd\xdf\x8c\x67\x81\x44\x33\xee\x7f\x5c\xfe\xf8\x43\x83\xef\xfe\x6a\x27\xd4\x5e\xad\xfd\xc5\xf0\xaa\xe6\x25\x5e\xad\x57\xdb\xca\xe6\x00\xac\x90\xb0\xad\xb9\x9d\x79\x38\xea\xe6\xe9\x53\x82\xe2\x44\xd4\xba\x38\xd9\x6f\x47\xe4\x24\xc6\xae\x5b\x91\x0f\xd6\x49\xd7\x71\x2c\xdf\x89\x7d\x7d\x11\xfe\xa0\xea\xfa\xc7\x1f\xbf\xc5\xf3\x99\xf1\x7c\xa2\x62\xe0\x33\x7c\x11\x88\x3f\xfe\xf8\x86\x5f\xab\x0c\x5f\xd3\x35\xe6\x63\x3c\xc1\x89\x8d\x6d\xb5\xc4\x9d\x6b\x3f\xbe\x7b\x87\x6e\x58\x5e\xfd\x40\x0e\x97\x4b\xc1\x13\x97\x8a\x6f\x1d\xbf\x28\x49\xc3\x43\x19\x7c\xe6\xd0\xf3\x92\xed\xeb\x30\xf1\x9a\x54\xfe\xa0\x14\x54\xb5\x6c\xd0\x82\x72\x06\x5d\x87\x88\xf6\x45\xe0\x97\x15\xeb\x35\xdd\x83\x03\xcf\x7b\xed\xb6\x29\x1d\x9f\x66\x67\x78\x57\x10\xac\xf1\x84\x7e\xe7\x05\xc0\x59\x23\xcf\x35\x96\xe7\x7f\x61\x47\xdf\x9d\x1f\x2e\x5e\xcb\x97\x9d\x8b\x54\xbd\xf6\x3c\xdc\x2e\xe7\xb7\xda\xa9\x18\x97\x0b\xe9\xa8\xc3\xa4\x4c\xab\xbb\x8b\xd4\xd7\xe8\x9a\x09\x28\x15\x9e\xa7\x1e\xe9\x84\x8c\x03\xe0\x57\xa6\x5f\xb8\x78\xad\x35\x99\xdd\x6b\xbf\xef\xf9\x6a\x88\xd7\xf5\xb1\xf6\x63\x90\x4f\x54\x5e\xcb\xf1\xf5\x66\x20\x06\x5b\x1a\x4f\xca\x81\x24\xe0\xcf\x12\xc5\xe8\xaa\x27\x7e\x60\xe2\xa4\xdf\xcc\xd7\x74\x14\xd1\xef\x38\xab\x50\x0a\x92\x0b\x44\xa1\xf0\x18\x09\x88\xd0\xb5\xcd\x8a\xa2\xdf\x1c\x1c\xac\x54\x7a\xc4\xae\x15\x56\x78\x9a\x4d\xe6\xed\x38\x2f\x46\x4f\x4f\xf8\x5e\xbb\x13\x89\x19\x30\x0e\x1f\x68\xc7\x82\x19\x34\xd2\xa4\x63\x1c\x8b\x38\x2b\xa6\xd0\xee\x16\x29\x49\x13\xf9\xc2\x6b\x24\x29\xfb\xc4\xc3\x68\x3c\xeb\xe9\x5a\x3d\x5f\xeb\xc5\x94\x73\x91\x58\x8d\x92\x78\x32\xbc\xcc\x3c\xf2\x25\x28\xce\x7e\xa1\x8a\x48\x2a\x8c\xc5\xa4\x5d\x55\xa5\x29\x0f\x1e\xa8\xc4\x58\x85\x94\x7c\x38\x52\xc0\x7d\x7a\x62\xac\x5c\x33\x0d\x80\xef\x83\x31\x00\x0e\xd3\x57\x12\xe6\x39\x3d\xdd\x43\xbf\xe8\x83\x68\x45\xf5\xc8\x6a\x1d\x5b\xe5\xdb\xd8\x8c\xcb\x37\x2c\x4c\x43\x58\x6a\x1e\x72\x72\xf9\x3d\x77\x5b\x1c\x19\x5c\xa5\x9d\xde\xcd\x6a\xae\xdd\x14\x59\x75\xc3\xcd\x61\xb3\xcd\xd4\xe5\x2d\x1e\x78\xcc\x21\xa1\x15\xe7\x01\x58\x19\x6c\x6c\xa0\xf2\x13\x35\x14\xb8\x23\x88\xa4\xbb\xf0\x8d\x95\x95\xfa\xff\xa8\xd4\x55\xd4\xaf\xa7\x31\xb9\xef\x8a\x87\x75\x7e\x0f\xac\xa6\x2f\xeb\xe3\xb1\xab\xe5\x35\x8c\x93\xf7\xf1\x77\x71\x44\x8e\x8c\xac\xa8\x9f\xa3\x64\x62\xa2\xe8\xc4\x0d\x20\x30\x31\x2b\x3e\x13\x68\x22\x22\xa8\x1b\x5f\xdd\xe0\xf0\x68\x29\x7d\xff\xf2\x29\xce\xb2\xe7\x76\x38\x05\xc7\x07\x1a\xad\xbf\x9a\x25\x53\x3a\xe7\xf6\x2d\x66\xad\xb9\x3d\x10\x87\xa2\xe7\x2e\xeb\x3c\xa2\x8a\x14\xc1\x99\x4b\x4f\x45\x50\xc9\x45\x77\x5f\xec\x6e\x05\x0c\xda\x78\x56\xe9\x7c\x82\x5d\x4b\x10\x20\x3e\x5d\xdf\xa2\x20\xe6\xb0\xce\xe6\x33\xeb\xe8\xa4\x33\xa6\x78\x12\x2d\x15\xfa\x38\x68\xeb\xe2\x56\x6d\x3a\xd2\x36\x6a\xc3\xc8\x30\xa3\x9b\x2a\xdb\x4e\x91\xbb\xdd\x62\x54\x63\x16\x52\x4c\xbb\xd8\x3c\x9c\x50\xd2\x48\xf9\x04\xd5\x54\xa0\x32\xf2\x8d\x15\x66\x4e\x8c\xe6\x53\xdd\x22\x0e\xde\x5e\xb2\xda\xd3\x64\xe2\x07\x97\xf9\x35\x72\x8c\x68\x8c\x70\x4d\xc6\x5e\x73\xd6\xa0\x30\xc3\x1f\x3b\x16\xf6\xf4\x21\x8e\x17\x8e\x59\x51\x66\xc1\xbf\x1e\xc6\x03\x68\xff\x09\xb9\x22\x08\x6b\x0d\x7a\xaa\x55\x26\xad\x6e\xe7\xf7\x38\xeb\x4a\x2f\x63\xe5\xf8\x0e\xfb\x80\x01\x20\x37\x26\xb8\xe6\x7d\xd5\x12\xad\x9e\x95\x30\xb8\x5f\x2e\xe3\xd9\x60\xc3\x42\x17\xd6\xfe\xd9\xca\x65\x3d\xf2\xb6\xc1\x43\x79\xf0\x2f\x82\x7f\x23\xf8\xb7\x80\x7f\x33\xf8\xf7\xff\xfe\x07\x5e\xfc\x85\x7f\xf0\xdf\x18\x1f\x7e\x83\x7f\xff\x1d\xfe\xfd\x0c\xff\x7e\x82\x7f\x7f\x84\x7f\xbf\xc0\xbf\xbf\xd4\x7a\x0d\x7e\x21\x32\x83\x9b\x1e\x90\xb5\x23\xf2\xbe\x9b\x17\x4b\xa8\x74\x0a\x61\xf5\x70\x89\x9f\xf1\x2e\x0a\x22\xe8\x5f\xa8\x2f\xfa\x3d\x0c\xc1\xd0\x32\xd9\xf4\x83\x03\xb8\x25\x35\xbf\x9f\xa9\xc7\x14\x0c\x03\x34\x68\xa1\x44\x7a\x2e\x3b\xf0\xac\xc3\x9e\x41\x5c\x0f\x9a\x9e\x39\x5f\x4a\x6a\x55\x1d\x2f\x0d\x7c\x84\xd7\xbb\x70\xf5\xe6\xb1\xb5\xcd\x72\x2c\x5a\x87\x16\x22\xa0\x3a\x4c\xba\x60\x7a\xb4\xd1\x2e\x3c\x6a\xe1\xe6\x4b\x91\xef\x57\x2f\xbd\xe4\xa8\xc5\x1e\x23\x88\x78\xeb\x0a\xb4\xfd\xc4\x17\x1f\x4f\xe5\xea\xad\x0f\x9d\x99\xd9\x8b\x81\x19\x3f\xd6\xe3\x68\xd6\xb6\x41\x58\x2a\xf3\x63\x60\xba\x80\x8a\xe3\x59\x75\xf5\xec\x36\x3e\xc9\x67\xf8\xb4\x44\x97\xa5\xf1\xc0\x02\x83\x11\x0e\x66\x41\x04\x77\x15\x80\x2e\xaa\x7d\x51\x13\xd1\x24\xb9\x99\xc9\xb8\x26\xc6\x5d\xd4\x60\x1e\xaf\xa3\x5a\x18\x05\x31\x2c\x6c\x1c\x77\xcc\x71\x1f\x2d\x6c\x27\x3d\xf1\x90\x0c\x57\xb7\xa1\xc7\xe1\xd3\x9e\x18\xcc\xa7\xd3\x48\x26\x9f\xf5\x4c\x2b\xc9\xa8\x97\x3d\x7e\x56\x8f\x43\xe7\xf2\x00\x2c\x74\xcf\xf4\x7e\x44\x8f\x1b\xd7\x84\x7c\x12\x1d\xac\xd1\x4e\xfa\x90\xe0\xf1\x53\x5b\x4a\x0c\x42\x4f\x7f\x66\x1f\x6d\x14\x0e\x91\x08\xc4\xfa\x35\x30\xb6\xf0\xfb\xe0\xa0\x16\xe2\x37\xd5\x1a\x8d\x2d\xae\x04\xfd\x01\x28\xc9\x0d\x80\x61\xf2\x01\x80\xaa\x55\x0f\x6d\x51\xe0\x28\x90\x82\x63\x57\x3d\x01\xf8\x38\x88\xd2\x18\xfa\x57\x9b\x6b\x0d\xf6\x1d\x55\xae\x76\x53\xeb\xb0\x51\x4b\xe9\x5f\xd6\xda\xb2\x96\xbe\xaf\xeb\xf9\x65\x4d\xc2\x8e\x32\xb0\x8b\x9d\xb0\xcb\x0c\x6c\x1f\xca\xc5\xdf\xb9\xfc\x5d\xcb\xdf\x3f\xd7\xda\xb5\x7f\xc2\xea\x72\xf3\xe1\x6b\x4e\x52\x69\xf8\xec\xe8\xa7\xb1\x9a\xbf\xc1\xbd\x9c\xaf\x01\x1a\x07\x03\xca\x35\x90\xb9\x87\xb5\xb6\x6e\x01\xdf\x6a\x3f\xdf\x2e\x3a\x55\x64\xd6\x03\x4d\xda\xd6\x2e\xb4\xab\x8a\x04\x5a\xfe\xb9\x8c\x96\x2a\xcd\xe6\xc2\x28\x2a\xbd\x25\x3d\x3d\xe9\xd2\xf1\xd4\xbc\xe4\x2c\x3f\xfe\xf1\x22\xb4\x85\xa2\xa6\xdc\x4d\x75\x0d\xbd\xba\x83\x07\x56\xf7\x0e\x6c\xb5\xd1\x76\x6a\xb1\xcc\x07\xc2\x32\x92\x9f\x6e\x46\xc8\xac\xcd\x7a\x83\xc5\xcf\x28\x1f\xf9\x50\xa4\xa5\xe4\x21\x98\x92\xff\x72\xc2\x87\x97\x01\xec\x24\x79\x6d\x80\x06\x8d\x8f\x2c\x38\x1f\x59\x48\x29\x50\x38\x9a\x6b\x9e\x16\x91\xed\x72\x70\x40\x51\x5f\x06\xd2\xdc\xad\xd5\x08\xd7\x2c\xbe\x89\xd0\x72\x09\xa5\xa6\x7d\x7a\xc2\xb9\x86\x54\xba\xfe\x45\x70\xc4\x6f\x3f\x48\x3d\xcb\x56\x94\xa8\xd5\x6b\x78\x44\xf2\x66\x86\x13\x18\xff\x82\x1a\x55\x5e\x77\x63\x3d\x9f\xd3\xfb\x05\xe5\xdb\x61\x74\xf2\xd1\x71\x3e\xa8\xca\x14\x2b\xf1\x55\x49\xd4\xbc\x7c\xc7\x4e\x8e\x00\x04\x2d\x11\x20\x67\xca\x90\xe3\x58\xc2\x0e\xa1\x27\x51\xba\x7a\x6d\x3f\xe5\x29\x7d\xf4\xf9\x17\x09\xbf\xd1\xdd\xce\xbf\xe0\x4a\x4f\x5f\x45\x23\x3c\x09\x80\x40\x20\x8b\xe5\x83\xb4\x97\xcf\x48\x73\xcd\x17\x76\xab\x64\x5c\x02\xda\x63\xb5\x7a\xdc\x81\x69\x65\x1b\x4c\x0a\xbb\xb4\xc8\x3c\xf9\x56\x83\x0c\x11\x3d\x2a\xe0\xb0\x00\x74\xfd\xb6\x6a\x3d\x75\x5d\x4c\x2c\x60\xb2\x80\x86\x00\xa9\x9d\x0b\xb9\xb8\x65\xdc\x80\xcb\x92\x58\x29\x49\x6c\x81\x34\xb8\x91\x6e\xb7\x0d\x62\xa1\x0e\xe5\x3f\x53\x05\x59\x94\xa7\xaa\x22\x14\x31\x9e\xac\x09\x2f\x42\x86\x9c\xce\x35\x10\x4e\xed\x95\x56\xa9\x5d\x9d\x81\x00\x3d\x89\xb0\x0d\xa3\x8a\x06\x51\x85\x64\x41\xdb\xb5\xbf\x16\x40\x32\x2d\x28\xef\x60\x5e\x84\x81\xeb\x95\x97\x62\x81\x4b\xde\x69\x66\x6a\xfe\x72\x59\x8a\x04\xb9\x52\xea\xb1\x80\x6e\xf3\xf6\x1b\xbe\x59\xea\x74\xdb\xce\xd5\x5f\x1f\xd1\x0f\xe6\xd5\xab\x8b\xf0\xaf\x3d\x7c\x82\xe0\xca\x7b\x57\xff\x82\xa2\xfe\xf9\x9f\xf0\xc7\x87\x7f\xef\x86\x1e\xfc\x15\xf8\xd5\xa8\x77\x39\x74\x15\xd5\x3f\x7e\x09\x00\x47\x89\xc8\x69\x05\x7d\xd4\xa0\xdf\x2e\xf1\x92\xb2\xd6\x47\x1e\x9c\x26\x8c\xfd\x83\x32\x18\x4e\x6f\x8c\x96\xf3\xe9\xd7\x72\x05\x81\x0c\x85\xf9\x33\x08\x5f\x02\xcc\xfa\x19\x98\xe0\x14\x80\xfe\xfc\x3c\x10\x04\xfe\xb4\x58\xa8\xf1\x63\x2b\x6e\xac\x2c\x62\x91\xc9\xf4\x93\xb6\xda\x16\x00\x18\x57\x03\xfe\x71\xbd\x98\xcf\x70\x2d\x3a\x9a\x10\xe8\xa8\x1a\xf4\x5b\xb0\xb9\x86\x04\xb4\x2c\x05\x72\xd6\xc6\x8a\x5c\x97\x5a\x90\x08\xec\xba\x1a\x55\x85\x92\x2f\xcd\x79\xe8\x04\x1e\x9a\x97\x27\x2e\xe1\x90\x4f\x44\x67\xfa\x53\x68\xad\xab\x93\xf3\x97\x82\x75\x2f\x2f\x8a\x14\xe2\xdf\x70\xa7\x2e\x26\xa8\xc9\x5d\xbe\xa4\xfc\x20\x61\x96\xe5\xa4\xce\xe4\x75\x9c\x15\xbb\x9f\xf5\xc5\x4d\x09\xee\x2b\xbf\xd7\x49\x2e\xfc\x83\x83\x9b\x0b\xbf\xe3\xae\xac\xc7\xb7\x6d\x8d\x58\x0f\x6f\x44\xe2\xdd\x80\x72\x28\xc5\x71\x17\xe2\x2a\x9a\xfb\x65\x55\x9d\xcc\xf3\xdf\x0d\xbc\x3c\x87\x4b\xdb\x6a\xd5\x61\xc7\xe4\x08\xb8\x0b\x2d\x79\x13\xcf\xf9\xe5\xed\x6b\xe8\xbf\x19\x77\xc8\x8f\x69\x1b\x4c\x2e\xf8\x97\x79\x15\x7d\x43\x53\x36\x82\xbd\xbc\x9f\x3a\x1b\x3e\x61\xb4\xd2\x08\x7e\x8e\xa7\x0b\x57\xd8\x20\x76\x8a\x3a\xbd\xcb\x3f\xea\xd1\x1e\xf6\xce\xbf\xf2\xc2\x0c\xe8\x8a\x9d\xf2\xa7\xb9\x58\xba\xb2\x11\xaf\x0a\x0f\x29\xa5\x7c\x90\x6c\x15\xfa\xf2\xb5\xf4\x1f\x47\xed\x8a\xb3\xe2\xa9\xd1\x4a\x1a\xb1\x5a\xb4\xa2\x88\x0e\x33\x06\xd7\x98\xe3\xc8\x72\x23\x23\x9f\x3c\x36\x0f\x61\x73\x04\x39\x3f\x04\xf8\x6b\x86\xff\x91\x22\x7f\xa6\xd3\x5a\xd9\x47\x1f\xe5\xbb\xca\x64\xb9\x74\x2b\xf2\x5c\x59\x40\xbd\x42\x89\xed\x4c\xae\xff\x1e\xcf\xff\x3f\xea\xde\x76\xb9\x8d\x23\x59\x14\xfc\x7f\x9f\x42\xc4\xae\x19\xdd\x44\x01\xec\x06\x45\x4a\x02\xd4\x44\xd8\xb2\xbd\xf6\xbd\x92\xed\x95\x3c\x1e\xc9\x18\x0c\x03\x04\x9a\x62\x8b\xf8\x1a\x34\x28\x76\x4b\x44\xc4\xdc\xb8\x37\xce\x13\xec\xab\xf8\xd7\xfc\x3b\x0f\x60\xbf\xd2\xe6\x47\x7d\xf6\x07\x08\xca\x3a\xe7\xec\x86\x2d\xa2\x3b\x2b\x2b\x2b\x2b\x2b\x2b\x2b\xab\xba\x2a\x6b\x16\xaf\x57\x79\x09\x4d\xd7\xaf\xaa\x84\xe8\xe3\xb7\xf1\x08\x43\x49\x5a\x97\xbd\x33\xc0\xaa\x59\x75\x39\x12\x0f\x85\x43\x00\xab\x44\x21\x89\x3e\x5b\x4c\xe5\x71\xd2\xd2\x55\xd1\x86\xba\x5e\xea\xe3\x3c\x69\x24\x2b\xad\xde\x55\x5c\x5a\xf5\x5e\xbc\xf4\x7c\x1b\x73\x78\x1c\xbf\x8a\x41\x52\xf2\x72\x36\x96\xc8\xab\xe5\x65\x6c\x0b\xa4\xc4\xb1\x7a\x6a\xa7\x84\x89\x96\x99\xdc\x9b\x2d\x59\x18\xa0\xaa\x36\x86\x79\x0e\x0c\x90\xa3\x35\xd4\x4e\x13\xe3\x70\x37\xf2\x66\x6c\x0c\x7c\xc6\x4f\xa1\x7e\xea\x60\x94\xd8\x17\xe0\xd1\x26\x77\x95\xa6\x4f\xc0\x9b\x72\xaa\x8a\x96\x82\xb5\x40\x05\xd9\x4a\xae\x2d\x04\x0a\x6f\x70\x3f\x8e\x9f\x27\xf3\x98\x87\xb4\x2d\x1c\x3b\xcd\x81\x39\xbc\x2d\xa2\x12\x81\x12\xc4\x4e\xb4\x3f\x97\x34\xca\x3c\xd6\x08\x86\xf8\xfb\x69\x31\xcd\xdf\x6e\x55\x7c\x87\x9e\x44\xdf\x56\x6d\xd3\xfa\x77\x51\xfe\x0f\xa9\xb1\xe2\xb0\xa6\xd2\xc0\x9d\xea\x46\xf7\xea\xf6\xb2\x77\x26\x86\x45\x03\xd1\x21\xa9\x35\x64\xa7\xce\x6f\xd0\x5d\x06\x37\xca\x96\x8f\x00\xbd\x68\xc9\xed\xfd\xb3\x48\x13\x71\x60\xc8\x8a\xd0\x3d\xb1\x06\x00\x55\x11\x0b\xcb\x17\x6e\x16\xc7\xe0\x4a\x98\x8d\xf2\x12\xd4\x15\x29\xdb\xa3\x8c\x9d\x0e\xa3\x6d\xc1\x02\x69\x5d\x91\xd4\x9a\xd1\xc3\x83\x3f\xfe\xb9\xe1\x20\xaa\x74\x5f\x30\x06\x21\x9c\x52\x67\x18\xad\x5c\xc8\x37\xf3\x89\x7e\x5f\x72\x13\x32\x52\x35\x75\xc9\x5c\x5b\x1e\x53\xb7\xf9\x6a\xeb\x02\xa2\x62\x06\x04\x6e\x14\x79\x2c\xb1\x70\x54\x97\xaa\x45\x87\xeb\x0b\xe5\xf4\x8a\x15\x0b\x4e\xf1\xb1\x8f\xf5\x6b\xe2\x53\x17\xff\xdc\xcd\x45\x5b\x56\xd5\x81\xe9\x88\xb5\x58\x79\xd3\xfa\xe7\xe8\xb0\xa6\x95\x01\xe1\xd8\xa2\xfd\xfe\x2f\xf1\xc7\xbf\x81\x3e\xe0\xce\x27\xf9\x65\x89\x9e\xa3\xc1\xef\xff\x0a\x22\x48\xfd\xfd\x5f\x21\xfc\x0c\x7d\x40\x0b\x4e\xff\xf8\x37\x98\xf0\xc1\x43\x04\x79\x00\x00\xef\x21\x01\x42\x04\x58\x61\xdb\x80\xc1\x9f\x6c\xf2\x1c\xea\x46\xf2\x3b\xc6\x13\xb7\x69\x32\x9a\x7b\x50\x06\xca\x49\xc6\x52\x04\x4c\xeb\x8d\xaf\x7b\x5a\x06\xea\x32\x0e\xf0\xe6\xa6\x25\x0a\xcf\x56\x8b\x34\xa5\xa0\x89\xbe\x88\xff\x71\x8d\xee\x1d\xf8\xfa\xd1\x80\xd1\xd1\x3a\xb7\xe4\x23\x98\x6c\xf8\x3f\x99\x5f\xa8\xeb\xf9\xab\x49\x19\x22\x82\x33\xfa\xbd\x22\xe2\x0f\x04\x4f\x3e\xc4\x9e\xa1\xe6\x57\x50\x26\xc5\x4e\xc6\x30\xf5\xb0\xf0\xb8\xbb\xfc\xfe\x2f\x10\x69\xeb\xf7\x7f\x9d\x09\xf0\xf2\x7e\xff\xd7\x69\xd0\x0f\xbb\xd0\xf9\x7f\xff\x57\x12\x19\x5c\x3c\x0a\x3d\xd1\x61\x25\x0f\x52\xda\x36\x3e\x03\x92\x28\x9e\x68\x74\x9e\x7a\x90\xd5\x3f\x0d\x1f\x07\x28\x28\x3b\xf1\xef\x20\xf6\xe4\x34\x3d\x80\x02\xf6\xf7\xf1\xe7\x14\xde\x7d\x16\xe4\x1f\xff\xe6\x94\x11\xda\x65\xf4\x20\xd1\x69\xd2\xc4\xd7\xe1\x8b\x90\x37\xfc\xd3\x3c\x3a\x09\xfc\x2f\xe0\x4f\x0b\xc3\x5e\xde\xa7\xd4\xd6\x96\x62\x51\xb3\x12\xad\x5a\xaa\xd8\x3b\x35\xae\x67\x97\xdf\x87\x82\xa1\xc8\x3e\x68\xef\x34\x06\x5e\x02\x81\xe2\xb1\xdf\x42\x5c\x86\x63\x7d\xf6\xbb\x2a\x81\xe0\x55\x68\x01\xa1\xc1\xeb\x29\x3c\x04\x7d\x84\x00\x79\x93\x24\xa8\x86\xa1\x21\x89\xc8\x00\x39\xfb\xbc\x1c\xb0\x28\xec\xce\x4a\x91\x94\x00\xeb\x0c\x92\xdd\x3e\x47\xa6\x02\xfa\x38\xf5\x7a\x69\x12\x74\x5f\x74\x51\xc1\x7e\x78\xb2\xdb\xe3\xb5\x3d\x58\x2e\xbf\x84\xf8\x12\x0a\x9b\x04\x87\xfe\x58\x16\x83\x77\xa0\x2b\xe2\xf4\x72\xab\xbf\x1a\x15\xef\xc1\x23\x59\x3b\x5b\x63\xfb\xf0\xd0\xf4\x58\xf5\x41\x97\xba\x2d\xd4\xaa\x2e\xbc\x73\x6d\x21\x1b\x56\x0e\x88\xc2\xef\x1f\xff\xd6\x2b\x19\x3a\x59\xa2\x28\x1a\x1a\x97\x39\x25\x8e\x4a\x8b\xea\x15\x90\xa5\x40\xac\x3a\x9d\x9d\x11\x03\xe5\x71\x81\x50\x85\xac\x0f\xd4\xcd\x3f\xfd\xfd\x37\xd9\x60\x2d\x6a\x5d\xa8\x21\xcc\x69\xeb\x85\x5b\x12\xa5\xdb\xf8\x6a\xc9\x01\x9e\x5b\x98\xd9\x7f\x1a\x80\x72\x87\xd8\xf5\x50\x1f\x4d\x36\xbe\x7e\x2a\x7e\x49\xc6\xdb\xb3\x4f\xf0\x8d\x30\x7e\xd6\xb9\x13\x52\xfb\x26\x81\x49\xe0\x9c\x70\xbd\x8c\xf9\x31\x9f\xd9\x25\xab\x4f\x23\xc5\x67\xdf\x80\xb2\xfd\xfd\xcc\x24\x74\xb3\xa7\x2a\xe9\xf6\x56\x01\x9f\x66\xb4\x37\x0b\x2b\x00\x7d\x15\x2b\x01\xbf\x68\xd0\xce\x84\x96\x23\xd4\x5a\xb0\xbc\xb8\xf0\x94\x7f\x58\xd3\xa2\x8f\x3c\xbe\xb3\xaa\x99\xd1\x5d\x3f\xe9\xf1\x5d\xfe\xd6\x8e\xef\xb6\xe2\xea\xd6\x94\xea\x6c\x86\x53\xad\x1e\x56\x0a\x0e\xa9\x52\x13\x24\xa3\xe4\x10\xd9\xaa\x67\x8a\xc4\x89\x50\xb5\x0f\x50\x91\x83\x15\xa6\xdc\xa5\x8a\x5c\x99\xda\xba\x5c\xa9\x3a\x07\xa7\x65\xaf\xa2\x5f\xd0\x3b\x6c\x00\x78\x45\x0b\xf9\x24\xf0\xa9\x53\x01\x16\x68\x68\x9f\x41\xdd\xd6\xef\xbf\x9d\x32\x50\x5a\xd6\xd6\x93\x60\x8b\xb2\x82\x2f\x51\xfc\x78\x20\x27\x9c\xfe\x47\xa4\x88\xe5\xb6\x88\x07\xa4\x15\x1e\x2a\x5a\xb4\xf3\xc7\x75\x27\xd5\x6c\x9b\x2a\xc7\x43\xe1\x3c\x92\x9e\x87\xb9\xe5\x62\xae\xbd\x11\xeb\x8a\x35\xa9\xe3\xe6\x3a\x17\x3c\x38\x17\x8a\x91\xcc\x8e\xc3\x3b\x8c\x03\x18\x01\x6d\x30\x1a\xea\xc0\x66\xe7\x2a\x39\xc1\x6f\x8a\x46\xfd\xb1\x63\x88\x91\x7f\x7b\xeb\x02\x43\x04\xf6\x3d\xee\x8c\xd8\x83\x04\xc2\x94\x69\x26\xc0\x08\x01\x20\x37\xfc\x8d\x28\x55\x70\xea\xb9\x4e\xad\x46\x0f\x10\x1d\x77\x13\x75\x99\x4f\xb9\x3d\x29\x3a\x37\x55\x82\x46\x11\xe7\xe2\x3c\x4e\xd7\x30\x13\x38\xc4\x4d\x52\x12\x55\xdf\xc8\x87\x51\xdc\x46\x12\x3a\x98\x63\x3d\xa3\xa4\x07\x34\x04\x57\x56\x26\x40\x65\xd1\x30\x45\x8a\x8f\x10\xeb\x81\x45\x9f\x22\x6d\x5a\xb1\x87\x22\x26\x34\xf0\x04\xc4\x15\x39\x79\xc4\xeb\xc6\x36\x08\x29\x8b\x8f\xc3\x57\x01\x47\x51\x84\xda\x71\x7b\xcb\x8f\xd0\xdc\xfd\xc1\x20\x00\x46\x03\x8c\x00\xa1\x9f\x86\xdd\xc1\x40\x1a\x02\x80\x4a\x53\x30\x1c\x6e\x36\xe4\x61\x93\x3e\x60\x44\x8e\xd5\x22\x99\x94\xe7\x25\xca\xd9\x92\x08\x7f\x0d\xa2\x22\x24\x2c\x42\x5e\x97\x70\xde\x94\x20\xbf\x96\x20\xaf\x4b\x74\xde\x94\x20\xbf\x96\xcb\xea\x94\x72\x95\x20\xbf\x76\xee\x98\x47\x29\x44\xb9\x5f\xb9\x5c\x86\xc8\xcb\xa5\x88\x0f\xe5\x72\xc4\x2c\xca\x0e\xb2\x66\x7e\x90\x37\x3f\x1c\x7c\x50\xdd\xf4\xf7\xdf\x3a\xa7\xd8\xb5\xcb\x84\xc3\x0a\xc2\x61\x05\x61\x30\xdc\xbf\x9d\x96\xe4\x5e\x49\x32\xa8\x20\x19\x54\x90\x04\xab\xe0\x30\x2b\x88\x4b\xbf\xaf\xb5\xa6\x3b\xa0\x55\xf2\xd1\x7a\x34\xef\x78\xb9\xc8\x7c\xcb\x41\xa4\x65\xd5\x34\x99\x7b\x1f\x0e\x4d\x34\xc7\x99\x6f\xa3\x0c\x9d\x89\xa8\x51\x9f\xa2\xcc\xff\x1a\x16\x21\xaf\x4b\x38\x6f\x4a\x90\x5f\x4b\x90\xd7\x25\x3a\x6f\x4a\x90\x5f\xcb\x65\x75\x4a\xb9\x4a\x90\x5f\x4b\x10\x3d\x47\x36\x53\x5b\x39\x15\xb6\xb1\x7e\x2a\x8c\x9c\x85\xe4\xe7\xa5\x81\xb4\x02\x61\xdb\xb8\x5a\x40\x2f\x4f\x4d\x55\xca\x9d\x93\xe4\x9d\x29\x69\x9e\xcd\x1a\xe6\x78\x9a\x2c\xbf\xb4\xe7\x41\x16\x9c\xb6\x5f\xad\x78\x7f\x4f\x15\x2e\x92\xab\x4b\xfb\xde\x04\xcd\x12\x83\xd6\x1f\xff\x14\x2d\x0c\xfd\x3f\xf4\x6d\x7c\x0e\x4f\xf4\xe2\xcb\xd7\x51\x18\x3f\x51\x13\x6c\x93\x50\xbc\xe5\x31\x0b\x30\x60\x1b\x86\x6d\x0b\x05\xdb\x01\x81\xc8\xc2\xca\xf1\x91\xe1\xd6\xca\xd1\xf5\x7a\x79\x6d\x96\x67\x38\x19\xf7\x37\xd0\x43\xfb\x3d\xcc\x3b\x27\x78\xb1\x93\x24\x18\x51\xb5\x65\x26\xe1\x22\x05\xf2\x7d\x23\x62\x2a\xac\xbb\x5b\x24\x25\x20\x18\x95\xea\xec\x65\x41\x44\x51\x90\x28\x74\x95\x7a\x86\x71\x25\x0b\x23\x8a\x8b\x44\x70\xf5\x8c\x83\x62\x1d\xe7\x8a\x71\x0e\xf8\x69\x4a\xf0\x61\xd0\x20\x81\xc1\x90\x41\x22\xc3\x01\x43\x5f\x35\xaa\xd1\xda\x5c\x17\x6f\xc0\x97\x61\xe8\x90\x77\xb8\xf6\xea\xa9\x16\x59\xcc\x93\xf1\x37\x30\x8f\x9f\x7e\xe9\x2c\x7d\x15\x17\xbd\x08\xcf\xb3\x5f\x74\x26\xdc\xbd\xba\x1a\xdd\x44\x95\x89\xca\xb6\x8f\xa6\xe7\x78\xf1\x54\x65\xe4\xa0\x32\x1f\x1e\x50\x5c\xac\x31\x1c\x2a\x30\x2d\xf0\x86\x6d\x0e\x4b\xe5\x0d\x5a\xed\x13\x71\xf4\xb8\xfd\x88\x2f\x1f\x1e\x4d\xa7\x30\x05\xf2\x06\x9d\x27\xed\x63\xf1\xf0\xb8\x7d\x0c\x60\xde\x00\x11\x06\x8f\x02\xde\xbc\x65\x4a\xff\x4b\x3a\xaa\x5c\xdb\xd1\xa9\xf6\x0a\xa6\x54\x4b\x67\x89\x1b\xdb\xcd\x01\x84\x43\xe7\x5e\x23\x1e\xfb\xe9\x90\xf6\xc3\xc7\x3c\x31\xca\x44\xee\xb3\x09\xc2\x7d\xd7\xd3\x51\x7a\x35\x2a\x26\x80\x67\x75\x39\xba\x19\x25\x49\x31\x61\xa3\x83\x5a\x3a\x34\x85\x45\x46\x58\x39\x15\x52\xe4\x54\x1a\xe7\x5e\x84\x1f\xdd\x25\x69\x3c\xb1\xe8\x88\xba\x23\x8e\x1f\x93\x48\x2d\x49\x1f\x1f\x8b\x13\x00\xc9\x72\x77\xa0\xf9\xc8\xa5\x79\x24\xc2\x27\xed\x27\x2e\xcd\xc7\x22\x7c\x3c\x94\x75\x7e\xc5\x1a\x2f\x27\x38\xd6\xb7\xe5\xdc\x67\x58\x34\x80\x67\x4b\xdd\x75\xe3\xc9\xc0\xf9\xf6\xcd\x58\x85\xc6\xbc\x8a\xa4\x84\xda\x2a\x9a\xdb\x5a\x43\x74\x7c\x3a\x80\x66\x91\xe7\xb6\x7a\x6b\x4d\x61\x8c\xaf\xa0\xf5\x3d\xb7\xf9\x5b\x6b\x0a\x68\x7c\xd5\x53\x61\xfd\x4f\xa3\x76\xd8\xd9\xdf\x6f\x77\x8e\x1e\x9e\xe6\x30\x15\x3c\x8d\x5a\xed\x87\x9d\xe3\xfd\xfd\x56\xbb\x13\x3e\x3c\xcd\xfa\xdc\x18\x5d\x42\x3c\x39\x29\x60\x02\x0a\x62\x86\xe1\x31\x60\xb2\x88\xbb\x92\x43\x5f\xd6\xcf\xa9\xd6\x46\x98\xea\x17\x3f\x43\xf2\xbb\x0c\x2a\xc8\x34\xa4\x70\xb5\x14\xd8\xdf\x92\x88\x52\x4d\x24\x0e\xbf\x14\x51\x98\x25\x89\xc2\x2f\x05\x14\x29\x88\xca\xf6\x73\xb8\x90\x4b\x13\xa4\xed\x76\xc9\x36\xdc\x2e\xce\x82\x6f\x44\x79\x19\xdc\xa5\xad\x3e\xbf\xb9\x94\x35\xd4\xa1\x6b\x3e\xd5\x19\x77\xa0\x96\xb0\xb5\x20\xe2\xd2\xb6\x13\x1c\xf2\xf6\x12\x8a\xf6\x26\xb6\xd2\x97\x0b\x26\x45\xea\x0c\x2e\xd1\x26\xf0\xa6\xd6\x11\x29\xca\xdc\x9e\x93\x17\xe5\xee\xa4\x15\x64\xbf\xcb\x5c\xbe\xb2\xac\x8a\xda\x38\x29\x55\xe5\x70\x9d\x36\xb6\x76\x9b\xfd\x80\x3b\x8d\xcc\x4a\xc5\xcd\xb6\x91\x33\xc5\x84\x0b\x93\x4a\x5c\xc0\x93\xa5\xfa\xdd\x32\x1d\xb7\xcf\xed\x1e\x60\xd2\x35\x3d\x86\x1b\x7e\x6f\x1f\x1d\x1f\x18\x6e\x2c\x1c\x55\x92\xb1\x4f\x15\x16\xcb\x37\x8c\x4a\xcb\xb6\xa9\xca\xea\x30\x9a\x5c\x78\x7b\x45\x2e\xd5\xe1\x9d\x8a\x22\x7a\xd5\x06\x34\x8b\x64\x84\x4e\xf6\x66\x7a\x2e\x01\xde\x7f\x5f\xa6\x76\xe6\x5b\x9e\x20\xb8\x27\x19\xd8\xc8\xe3\x63\x0c\x65\x0a\x26\xf0\xe8\xf1\x01\x46\x55\xcd\x9a\x0a\xd6\x94\x30\x1c\xdb\xd9\xd6\x58\x23\x85\xcf\x76\xc1\x1e\x13\x95\xed\x32\xe5\x61\x01\x47\xc1\x23\x49\x2c\x08\x81\x58\x05\x07\x1d\x28\xad\xf9\xfb\x6f\x88\x13\x76\xe8\x11\xd9\x20\xa3\x7c\x70\xd5\xe2\x04\x30\xd6\xf4\xbc\x9d\x19\x6b\x50\x56\x56\xd2\x65\xa6\x13\xa8\x9a\x61\x49\x15\xcc\x50\x99\x8a\x99\x93\x13\x8b\x1b\x18\x1b\xee\xc9\x8d\xd6\x84\x92\xe6\x4a\x2f\xc9\x9e\x0c\x2e\x61\xca\xf8\xa5\xfb\x65\x52\x81\xe4\x67\x5e\x1b\xfc\x15\x7d\x30\x33\x93\x42\x03\x7b\x53\x01\x33\xd3\x40\x0b\x2f\x2c\x16\xa3\xc6\xfe\xcf\xf6\xf1\xb2\xc0\x7f\x14\x14\x4b\x2c\xcf\xa9\x54\x8a\x9e\x9d\xfd\xb7\x3b\xa6\x67\x77\x93\x2a\x7e\x87\xd4\x70\xe7\x5b\x64\x91\x35\xf3\x2d\xa0\xb2\x2e\x87\x1d\x6b\xc7\x8a\x11\xaa\x25\x42\x37\xa1\x62\xda\x7b\xb7\x44\x0b\x40\x0b\xcf\x29\xfa\x99\x9e\x80\x97\x0a\x7f\xb6\x7d\xd6\x6d\xa3\xd4\xce\xbc\x8b\x48\x3b\xcc\xbe\xed\x2c\xd5\x0d\xf3\xec\x9e\xb3\x70\x87\xa2\x6e\xb8\xea\x6a\xde\x9b\x09\x53\xf5\xba\xac\x05\xf5\x29\x88\xc3\x7c\xc7\xc6\xc4\xea\xaf\xd8\x90\x50\xdc\xd5\xa0\x76\xa7\x79\x0d\x85\x66\x2e\x8e\xa6\x2a\xbe\x1c\x4d\x92\xeb\x14\x77\x61\xcf\xf1\x28\xa0\xed\x8e\x71\x92\xd7\xb4\x5e\xd4\x55\x8e\x6e\x20\x71\x98\xbd\x8e\xc6\x97\xf1\x2b\x39\xd1\xb5\x5e\x78\xb2\x0b\x93\x23\x0b\x16\x2d\x57\x0b\x64\x8a\xdf\x3c\xa7\x60\xdf\xaf\x5e\x2e\xb4\xb2\x63\x69\x0e\xaf\xab\x38\xa5\x5b\x02\x9c\x88\x02\x14\x6b\x5f\xcd\x99\xad\xb2\x69\x0e\x87\x82\xe2\xe9\x17\x73\x02\x79\x14\x4d\xe1\x30\xe7\x96\x64\x73\x21\x2c\xa1\x44\x0f\xdb\xc7\x7a\xa6\x48\x2b\x77\x3b\xec\x33\xb1\x6c\x40\xdd\x1a\xa9\x2b\xa7\x42\x3e\xdf\xaf\xb0\x26\x1b\xaa\xda\x96\x35\xe5\xc2\xb4\xff\xff\x77\xcb\xc6\xb5\x22\x51\xbd\xc5\x88\xc5\x10\xeb\x0f\x4a\x3c\x1c\xde\xb9\xd2\xf8\xa6\x02\x67\x58\x5c\x28\xfc\x35\x2c\xd3\x0e\x4b\xf9\x2a\xd6\x47\xcb\x38\x65\xda\x41\x99\x76\x50\xca\x57\xb1\x62\x5b\xc6\x19\x76\xf5\xda\xb2\xd4\x90\xe2\x5e\x98\x7a\x1d\x55\x83\x7b\x54\x31\xb8\x47\x2d\xaf\xc2\x5d\xa8\xc0\x94\x1f\xc3\xee\xd7\xa4\x9c\x17\x1a\x74\x30\xd8\xd1\x29\x01\x1f\x6a\x47\xb7\x64\xa8\x04\x61\x2c\xc0\x6e\x8e\xbe\xc3\x6d\xe4\x59\xd9\xcf\xfc\xfe\x99\xac\x1a\x9d\x48\x52\x45\xfe\xe4\xd4\xef\x8c\xb6\xef\xaa\xdd\xeb\x42\x5a\x2a\xbf\x6b\x08\xa9\x3e\xcc\x96\x67\xc7\xa0\xef\xb6\x99\x92\x51\xe7\x15\x10\x19\x93\x5b\xce\xb4\x14\xae\x2f\x2e\xe2\x55\xb7\x00\x7d\xc6\xf8\x38\x33\x31\x83\xc5\xde\x3d\x07\x0b\xeb\xd9\x37\xb5\x93\xf8\x4a\xe6\x96\xe9\xdc\x4d\xe8\x56\x86\x8a\x71\xec\xac\x7f\xd6\xf5\xb6\x8c\x5f\x50\x21\xfc\x87\x65\xfb\x5d\x2b\xa5\xa4\x02\x5e\x71\x41\x11\x78\x57\x0d\x41\xa1\x6e\xcc\x9a\x23\x79\xfc\xb8\xaf\xde\xd4\x60\x16\xaf\x2f\x17\x13\x7d\x5e\xbf\xb4\xa0\x6d\x2f\xd9\x98\xec\x56\x23\x68\xa0\x5e\x6e\x51\x9f\x43\xe9\xaa\x7d\x45\x5f\xa3\x0d\xae\x86\x91\x04\xc2\xa3\x3e\x00\xa0\x92\x37\xc6\x79\xd4\x30\x7b\x6f\x7f\xd5\x22\x0e\xef\x8d\xdf\x65\x49\xc6\xc6\xbc\x63\x91\xc5\x46\xbd\x6b\xc1\xa4\x88\x7b\xc7\x02\x88\xcb\xef\x2e\xcb\x18\x15\x39\xe4\x82\x84\x6a\x5a\xab\x4f\xab\xde\x61\xdc\x84\x12\xce\x8b\xeb\xf5\x48\x06\x82\x72\x51\x65\x82\x5e\x8b\x8f\xff\x71\x9d\xac\x20\x69\x34\x7f\x7b\x3d\x1d\xad\xea\x17\xe3\x1d\x7d\x24\x48\x21\x6b\x61\x49\xbe\x90\xaa\x56\x4a\xab\x53\x55\x05\x68\x09\xd7\xb1\x7a\xbc\xa8\x6b\xb9\x93\xa0\x2f\x37\xa3\xd5\xc4\x5d\x69\x55\x2e\x95\xb5\xc3\x58\xae\x06\xbb\x8b\xaa\xf6\x5e\x46\x77\x51\xd5\x4a\xf1\x45\x21\x53\x64\x7d\xe8\x2c\xe4\xaa\x49\x32\x21\x4d\x90\x09\x55\x69\x55\x3b\x59\x29\x20\x8d\xdb\xf4\x6c\x9e\x64\x82\xc3\x8e\x04\xaa\x6b\x09\x3b\x7d\x89\xd4\xb1\x91\xba\x30\x8e\x49\xc9\x6c\x5f\x93\xae\x95\x54\xc5\x52\xef\x7f\xa6\xc0\x14\xf7\xda\x38\xa8\xc1\xe8\xa5\x14\xda\x6e\x0a\x34\x4e\x56\x63\x7b\x75\xcc\x52\x1c\x4e\x92\x9f\xfe\xe4\x7d\x24\x15\x66\x5b\xde\x15\xef\xde\x11\xef\x4e\x2c\xe4\xb5\xe5\xa2\xa6\x75\x5b\x4c\xbc\x20\x3d\x05\x75\xdb\x36\x50\x4b\xec\x82\x42\x1e\x9b\x68\x34\xf6\xb5\xe2\x34\x3b\xe0\x2d\x1e\xa2\xd2\x3e\xd2\xd9\x40\xda\xae\x92\x29\xbd\x47\xb8\x2f\xb2\x62\x6b\x64\x85\x36\xc0\x73\x6c\xf2\x02\x5d\x39\xbd\x6f\xd8\xcd\xd2\x1d\x20\xe9\x21\x5f\x5c\x4f\xbb\x55\xec\x7b\xad\xd5\x85\xf4\xfc\xa9\xcf\xac\x96\x9e\xe8\xcf\x82\x24\xf2\x4f\xbb\x20\x9e\xf3\x9a\x2b\xe2\x25\x2d\x62\x62\x37\x52\x16\xa7\xda\x95\x27\x22\xd6\xa7\x64\x8f\xf7\x12\x45\x4d\xde\xce\xa0\x9a\x45\xd7\xc5\xd5\x72\xc9\x12\x65\xd1\x1c\xdd\x73\x95\x78\x27\xae\xa8\x04\x9b\x21\x3b\x00\xc2\x99\x5f\xc9\x95\xc6\x70\x65\xe5\x3d\xb1\xbe\x49\xaa\x0b\x12\x0d\xab\xb4\x25\x91\xc3\x43\xfd\xfe\xff\xfc\xfe\xaf\x88\xb6\x49\xb5\x46\x7c\xad\xa7\xbd\xf1\x3b\xa0\xed\x47\x2e\x2c\xa4\x0d\x56\x36\x2c\x4d\xe6\x44\x86\x77\x81\x24\x73\x0f\xdf\xd0\x3e\xa4\x06\x0c\x2f\x12\x0c\x08\x48\x58\x23\xc3\x0b\xe1\x6a\x20\xa2\x12\x90\x30\x43\x1b\x33\x94\x98\xa1\x8d\x69\x62\x85\x59\x7b\x55\xcc\x8e\x14\x0f\x4f\x0f\x61\x9e\x03\xc9\xa7\x7f\xb0\x6e\x2a\x60\x70\xc0\x85\xb4\x98\xab\x03\x89\x29\x59\x07\x4c\xc5\xaf\xc4\x6b\xca\x5c\x05\x3c\x2d\xeb\xb7\x78\x1d\xfc\xf8\xba\xc6\x1a\xe9\x54\x3d\xd0\xca\x6e\x58\x38\x49\xe4\x76\x47\xf4\x3d\x52\xcf\x8e\x03\x28\x21\xd6\x97\x6d\xbe\x04\xdc\x5c\x89\x07\xd3\xb3\xaf\x5f\xfb\x07\x5f\xbf\x16\x30\xe9\x80\x27\x3a\x89\xfc\xda\x57\xd1\xa5\x2a\x72\xc0\x64\xed\xeb\x37\x90\xe3\x8d\x78\x03\x39\xde\x70\x8e\x37\xfe\xb6\x2c\x19\xcc\xef\xb0\x0b\x65\xb8\x9d\x02\x9e\xda\x17\xc9\x14\x3f\xbe\x56\x75\xd4\xf3\xd4\xcb\xbe\x00\x46\x4e\x7f\xff\x4d\x5d\x6a\xbe\x95\x76\x0e\xb4\x73\xa0\x9d\xe3\x3e\x0d\x78\x2a\xd1\xce\x1d\xda\xf9\x17\xc0\xb2\xa1\x0d\x66\x90\xac\x17\xf0\x95\x05\x28\x82\xd7\x74\x47\x5f\x1e\x60\xe5\xde\x04\x02\x0c\xa5\x78\x2d\xde\xe0\x6d\xbe\x18\xbb\x3b\x8f\xa0\x0e\x5f\xbf\x8e\x9e\x04\x50\xf3\x08\x86\x67\xcb\xae\x75\xcc\x82\x8a\x6e\x3e\x72\x07\xab\xf6\x18\xc8\x86\x71\x03\x73\x55\x0c\xc5\xb2\xd9\xeb\x5a\xdc\x1e\x23\x31\x02\xa0\x29\x78\x71\xbd\xc6\x32\xca\x45\x6f\xb5\xe7\xaf\x41\x1f\xb4\xb0\xdf\x78\x6f\x42\xbf\x9d\x62\xfc\x7b\x0f\xfa\x13\xa4\x85\xbe\x75\x50\x57\x27\x00\x5e\x50\x95\xe0\xc3\xc8\x60\x71\x14\xef\x7e\x1f\x99\xc9\x34\x1b\xbd\x5b\xac\xe4\x67\x88\x33\x1f\x0f\x61\x5b\xaf\x5d\x0b\xcf\x4a\x70\xe4\x60\x11\xd8\xcd\x02\xbf\xb6\xb6\xca\xbc\xb6\xb6\xc7\xbc\xb1\xb6\xcd\xbc\x31\x5b\x65\x40\x61\x4e\x5f\xe3\xde\xba\xb3\x08\x54\x07\x32\xa3\x0e\x85\x30\x7f\x85\x0c\xa7\x6f\x38\x01\xf4\x08\x72\xa3\x42\x51\x82\x61\xce\x7c\xc3\xb3\x23\x90\x0c\x06\xaf\x31\xc3\x50\x0c\x5e\x63\x16\x9c\xf6\x57\x56\x74\xb7\xfa\xd8\x5b\x7f\x9c\xed\x3e\x56\x7d\xac\xad\x3f\xd0\x0d\x4e\x33\x66\x1b\x2f\xad\x0c\x22\xec\x19\xc4\x76\x1e\x9c\xe6\x9c\x80\xf7\x58\x06\x11\x76\x93\x1d\xeb\x53\xdc\x19\x64\x65\xc1\x48\x5b\x9f\xa2\x13\xaf\x20\x9f\xd6\x08\xf9\x52\xd4\x07\x02\x97\xb5\xe1\xd5\xae\x45\x7a\xd0\xd3\xf9\x63\x22\x74\x76\xbe\xef\x4f\x93\x82\x6a\x81\xcd\xfc\xfa\x4d\xb9\x71\x76\x27\x3f\x51\xdf\x2a\x27\x79\x05\x79\xb0\x34\x93\xdc\x21\x7f\x4f\x5f\xc2\x76\x0a\x84\xde\xfa\xa9\xc9\xbd\xf6\xf8\x3e\x52\xdc\x3d\x9e\x97\x52\xdf\x78\x7c\x67\xa9\x69\x48\xf1\xba\x82\xc4\x1b\x32\x94\x48\xe2\x4d\x05\x89\xd7\x64\x52\x2d\x12\x56\xfd\x2c\x87\xa4\xba\xbb\x0f\x06\x74\x44\xa9\xf5\x24\xe0\x2f\x8d\xf8\xf2\x24\x90\xdf\x16\xed\xfe\xae\x10\x1f\x53\x22\x23\x3e\xa6\x4c\x43\x7b\xbc\x8d\x47\xeb\x2f\x57\xe3\x9a\xe1\x96\x13\x8b\x46\xb2\xce\xec\x0e\xd2\xc5\xf5\x6a\x1c\x9f\xdd\xde\xf2\x43\xf5\x44\x40\x5e\x56\x0d\x58\xfc\x50\x8d\x25\xdd\x67\x49\x51\xe5\x11\xfc\x8e\x32\xe5\x27\x75\xf3\x35\xee\x91\xa4\x27\x33\xd6\x30\xef\x15\xde\x5b\x71\x47\x9b\xc2\xf0\x3e\x23\xfb\xd4\xbd\x24\x07\x92\xe7\x9d\x94\x53\xe2\xaa\x8a\x9e\x55\xaf\x91\xe1\xb4\xa6\x7b\xa6\x4b\xf0\xbb\x8c\x6d\x95\x29\xc5\xb2\x53\x99\x12\x57\x0b\x79\xc7\x32\x19\xdb\x2a\xb3\xa2\x1f\x6e\xb3\x5b\x9c\xab\x1b\x18\x0a\x5a\x2b\x6d\x77\xdf\xac\xb9\xd9\x0d\x5e\x5c\x68\xb1\x67\x7e\x8c\x58\x98\x49\x4a\xa0\xeb\x78\x33\xad\x02\xa6\x04\xba\x93\x75\xcd\x9a\x8c\x25\x74\xc7\x6a\x3b\x63\xdd\x7d\xf6\x58\x05\x30\x2a\x66\x73\xbe\xf4\x6b\xa8\x8b\x56\xbf\x7f\xba\xfc\x35\xde\xe4\xaa\xfc\x6c\xfb\x49\x9f\x92\x47\x1f\x92\xd9\xf5\xfa\x72\x34\x35\x9b\x52\x8b\x29\xb6\x0f\x97\xfe\xfe\x2f\x72\xfa\xb5\xa8\xcc\x0c\xa3\x73\xe8\xd1\xc4\x40\x61\x60\x48\x1b\x95\xf1\x8f\xff\xa9\x33\x74\x38\xda\x20\x6d\x99\xff\xe3\x7f\xe2\x17\x75\xbf\xa7\x97\x79\xcb\xcc\xdc\x63\x51\xae\x9c\xbb\xb0\x2e\x57\x46\x70\x0e\x88\x5b\xa9\x09\x5b\x93\xf5\x8e\xb2\xe0\x48\xc9\x5c\x2f\x9c\x8d\x59\x69\x7a\x3d\x60\x7f\x7f\x7c\xa8\xe7\x70\x63\xbe\xfc\x49\xc7\xa2\xaf\x94\x80\x66\xe2\x13\x65\xa0\xf2\x6f\x91\x82\x42\x71\xf7\x27\x3f\x5b\xcc\x2f\xf8\x48\xf3\x3d\xf6\x27\xeb\x4c\x55\xfb\x93\x75\x62\x69\x23\xf4\xdd\x95\xac\xdc\x0a\x5d\x53\xb7\x62\xb2\xdd\xbe\x6f\xe7\x8b\x19\x26\xdf\x4f\xbf\xc3\x43\x03\xdc\x08\x3d\xb5\x36\x2d\xa6\xc9\xde\xa3\x95\x54\x9e\x02\xf7\x0a\xac\xfa\xe6\x2c\x5e\x8d\x71\xc5\xba\xb4\xac\xc9\x2b\x5f\x54\xc8\x20\x13\xaa\x47\x01\x5b\x3c\x75\x8c\x33\x9a\xfb\xb5\xa0\x6e\x17\x7f\xfc\x73\x68\x76\x9f\x2b\x82\xf5\xbc\x2a\x8c\x9f\x4a\x3c\xab\x94\x02\xcf\x0a\x6c\x4b\x7a\xb1\x5a\x5f\x2e\xc0\xef\x59\x5e\x6e\x95\xb6\x91\xb1\x12\x2c\x74\x0d\x23\x58\x87\xca\x3d\x84\x6b\xe7\x2b\x30\x6b\x27\xb9\xc1\x78\xc0\x02\xef\xc0\x71\xa5\x7e\xb8\x56\x6f\xbb\xd1\xc3\x26\x02\xb0\x65\xf2\xdc\xa2\xef\x51\x4d\x27\x63\xa1\x9e\x4e\x9a\xf3\xc9\x89\xa6\xb1\x2f\x76\xd1\x2b\x6b\x15\x09\xfd\xb4\x64\x7e\xe9\x65\xf2\x7a\x3e\x34\x71\x39\x7f\xd7\x27\x33\xae\xcd\x5a\xee\xe3\xa7\x6e\x48\x46\x5c\xdf\x52\xbc\x72\xd9\x9f\xa2\x82\x65\x2a\x85\x7a\x97\x11\xe4\xb8\x3d\xe3\x70\x5a\xf4\xd8\xbe\xc4\xaf\xb1\xba\x78\x94\x00\x4c\xe9\x53\xcb\x63\x46\x04\x6f\x02\x75\xa7\x6d\x9b\xf8\xa0\xaf\xf7\x90\x5b\x36\x07\x43\x52\x1f\x95\x57\x4c\x44\x22\xde\xc9\xe3\x34\x59\x47\xe4\x1d\x71\x2d\xde\x8b\x91\x48\x97\xe2\x22\xb3\xae\x24\x41\x11\x5e\xe4\x36\x20\xf7\xc5\x3c\xb2\x8a\x10\x4b\xf8\x8d\xe6\xad\x90\x1d\x01\x3a\xa8\x0a\xa6\x6c\x7c\x85\x0f\x97\x11\x85\x53\x00\x92\x91\xac\xf2\xec\x8c\xd0\x5e\xef\xef\xe7\x45\xd8\x1b\x5f\xb1\x47\xf4\x39\xca\x3e\x7e\xbf\xc4\x83\x9a\x3a\x69\x60\x0e\xa4\x2a\x18\x2f\xa7\x0f\x9a\x17\x59\x1b\xe3\x62\xb3\x53\x3c\x21\x22\x78\x7c\x33\xf1\x45\xf3\x22\xb7\x93\x00\x34\xe4\x4f\xa3\x49\x14\x6a\x72\x5a\xb2\x90\x09\x4f\x61\xeb\xd7\xcb\x21\x05\xe5\x75\x93\xa3\xc8\x4d\xdf\xdf\xb7\xd3\x03\x37\x7b\x40\xe7\x56\x2f\xa3\x44\x95\x1a\xe8\x52\x93\xbd\x28\xba\x84\x44\x98\xbd\xbb\x05\xb4\x5c\xfa\x38\xef\x77\x4b\x68\xb9\x25\x48\xf9\xb3\x30\x3e\xd2\x7a\x72\xd7\x3e\x05\x88\x2b\x07\x18\x59\x63\x12\x67\xdd\x64\xe3\x33\x27\x32\x0f\x9d\x0d\x76\xd7\x99\x95\x07\xcd\x2b\xd3\xad\x73\xfe\xdd\xf8\x62\xc4\xc7\xad\xf1\xe3\x11\xc3\xc4\x7b\x0b\x42\xf4\xc5\x35\xb4\x18\x8a\x16\x75\x43\xd6\x13\x35\xf3\x9d\x42\x4c\x14\xe2\x28\xb2\x40\x44\x8d\x10\xed\xba\xbe\xaf\xaa\x6b\xee\x22\x54\x48\xab\x63\x10\xde\x55\x52\x70\x11\xaa\xe4\x7d\x90\x85\xcd\x3c\x3c\xc8\xc3\xd3\x28\xeb\x1c\x64\x9d\x66\xde\x39\xc8\x3b\xf2\x9c\x89\xa9\x44\xd4\x0a\x7b\xea\xde\x87\x8d\x4c\xbb\x36\x69\x9b\x51\xb1\x8a\x20\x9e\x04\x84\xf6\x8e\xae\x7b\xa0\xbe\xc2\x8d\x76\xe9\xd3\x99\xe4\x77\xa0\x1d\x1d\x92\xda\x3b\x5f\xe6\x7c\x27\xc9\x9d\xb6\x42\x3a\x05\xa6\xb3\x14\xd2\x81\x40\xb3\xc9\x2d\x9b\x2e\x23\xc6\x93\x67\xc1\xa9\x29\xde\x11\x51\xdc\xda\x5d\xcc\xf9\x34\x90\x57\x86\xf4\xf6\x54\xa7\x44\xab\xf2\xec\xd9\x5f\xb9\xb8\x41\xba\x6c\x75\x64\xcf\xc6\xe7\x50\xe9\x9b\xa6\xa0\xbb\xa8\xdf\xf3\x5b\xad\x74\xd9\x53\xb8\xcd\xe6\x30\x2a\xe0\xca\x93\x4b\xd3\x1c\xbb\x33\xf7\x08\xa4\xd9\x4b\x4e\xa1\xee\xad\x56\xe2\x63\x1a\xd7\x90\x7a\x31\x93\x4a\x86\x43\xdf\x9c\xa7\x9a\xe6\x1b\xfb\xc4\xaf\x32\x2c\x7a\xf5\x44\x59\x95\x5e\xdd\x5c\x90\xac\xa6\x66\xba\xeb\xe1\x7b\x3b\xdb\x71\x1d\x0f\xe6\xad\x88\xef\x77\xb3\x0d\x3d\xb4\xf3\xdd\x32\xe6\x3a\x63\xce\x19\x7d\x63\xe7\xe5\xac\x67\xfb\xd7\x60\x1c\xdb\xae\xcf\xc7\xd3\x51\x9a\xda\xe9\xc2\x54\x9a\x88\xe8\x3b\x5a\x9d\x0f\xbe\xf6\x04\xaf\x8c\x1a\x15\x18\xb1\xb6\x7a\x40\x33\xd5\xe5\x2a\xec\x55\xb4\xae\x1d\x51\x41\xb8\xf8\x6a\x63\x1e\x2d\xf8\xa6\xde\xc1\x1c\x15\x88\x32\x06\x2a\x22\x17\x9e\xbc\x97\xa9\x09\xa7\x35\xf9\x33\x96\xfe\xca\x75\x70\x6e\x8e\x10\xb4\x8f\x0f\x10\x65\x53\x5b\xef\x8a\x2d\x8c\x57\x32\xa4\x94\x18\x57\x71\x96\x45\x78\xfc\x3a\xb0\x39\x24\xd5\x2c\xb6\xe1\xed\xad\x77\x85\x21\x05\xbc\x13\xbe\x60\x77\x44\x27\xda\xc0\x9e\x56\xd6\x63\x1c\x69\xce\x29\x82\xc8\x01\x45\x10\xc8\x9a\x11\x45\x31\x68\x52\x28\x81\x83\xb1\xc8\x09\x10\x22\x20\x44\x40\x4f\x39\xcb\x78\x0c\xe0\xe0\x6a\xb8\xad\x9e\x78\xc8\xd4\x2c\x57\x5c\xcb\x95\x01\x1d\x67\x74\xbe\xbc\x5e\xc3\x58\x3f\x13\xe7\x50\xef\x89\x18\x4f\x17\x69\x3c\x89\x0a\xe4\x9e\x11\x54\xe7\xae\x90\x4f\xab\x3a\x07\x47\xe0\x1c\x59\x32\x93\x62\x20\x06\xa8\xf0\x48\x52\x95\x9f\x22\x30\xea\x34\xbf\xcb\x05\x85\xc0\x11\x17\x65\x19\x78\x33\x7e\x50\x85\x33\xd7\x3e\x6a\xcd\x3b\xb4\xb7\x60\xc6\x9e\xce\x7a\xfe\x44\xa2\xbf\x1b\x16\xe5\xf3\xfd\x3c\x85\xc9\xaa\x37\xa1\x9b\xd8\xfb\x5e\x75\xea\x98\x52\x6f\x6f\x15\x43\x6c\x70\x8a\xa8\x6b\xf4\xcf\xc6\x6b\x0f\xa5\x87\xf8\xa6\x02\x8c\xef\xab\x78\x9a\xd5\xe4\xf7\xf7\xef\x4b\x1e\x3c\xfa\x1e\x57\xb8\x90\x57\xbe\xa0\xce\x80\xc8\xcf\xd5\xce\x15\x09\x76\x3b\xf7\xfb\xc5\x6a\x31\x5f\x24\xdf\x4c\xde\x5a\xc6\x41\x02\x9f\xc5\xd3\x69\x09\xf8\x15\x5e\x90\x58\xc6\xfd\x36\x59\xa5\xeb\x67\xf4\x91\xba\x44\x86\xa0\xd5\x84\x7e\x5a\x2c\xa6\x1c\xb3\xa4\x22\x8b\x4c\xec\x55\xb0\x65\x6c\x0e\x2e\xec\x61\xac\x92\x2a\xd3\x12\x43\xad\x04\xce\x16\xa9\x7a\xac\xa6\x31\xd5\x34\xf9\x4e\x42\x23\x9d\xac\xa3\xa1\xa8\xa4\x56\xab\xe7\xc7\x0e\xca\x40\xa7\x0d\x89\x8e\xc0\x3f\xed\xf3\xfd\x7d\xfa\x1d\xe1\x01\x5c\x45\x2c\x5d\x92\x16\xeb\x0c\xc2\x7c\xc7\xb6\x90\xd0\x9f\x2a\xd4\x4e\x65\xf8\x71\x35\xc1\x1b\x1f\x8a\xec\x6d\x8a\xa2\x52\xf8\x4e\x30\xdd\xe2\xee\x35\x3b\x20\x2d\xf1\x3a\xc5\xcb\xab\x68\x83\x5a\xb2\x8e\xfb\x06\x3e\xea\x9a\xe7\xf3\x8d\x88\xdd\x2d\x6d\x3b\x93\x39\xb7\xc8\x8c\x36\x25\x9e\x5f\xc6\x93\xaf\xf0\xb2\xec\x9f\x57\xb1\xc3\x77\x32\x4f\x63\x9b\x71\x8e\x4f\x3e\xc7\x1b\xb3\x38\x9a\x1d\xb4\xf3\x7c\x8d\x9f\x1e\xe6\x93\xe5\x48\x00\xda\x34\xa6\x38\x6c\x88\x47\x1e\x21\xe2\xb6\x7f\x8a\x4c\xc6\xf6\x0f\xfc\xd2\xfe\x41\xc8\x5f\x8c\x0c\xc3\x4f\x80\x48\xb4\x55\x0a\xbd\xc9\x97\x97\xac\x45\x1c\x73\x5e\x82\x7a\xfc\xfb\x1c\x8c\xb7\x05\x7e\xae\xc0\x94\x9d\x63\x69\xc9\x0c\x04\xe9\x31\xd7\x8c\xcd\xc9\x24\x9c\xb3\xbe\xa4\x5e\x23\x1c\xea\x51\x7c\xd7\xf9\x99\x2f\x64\xc5\x78\xdf\x90\x55\x2f\xc9\xee\x4f\x91\xcd\x84\xb0\x8b\x04\x97\x45\x65\xe6\x7c\x44\x83\xe9\x3a\xd8\x7c\x9d\x23\xd6\x9a\x10\x99\x14\x56\x42\x97\xf9\x97\x48\xb6\x00\xbd\x3d\xc3\x78\x07\x5c\x05\xab\x9e\xfb\xfb\xfc\xdb\x7e\xd6\xf3\x65\x43\xc9\x5c\xed\xbf\xa8\x92\xa2\x48\xa6\xb4\x9f\xf7\x3d\x6a\x45\x0d\x78\xc9\xad\xba\xbf\x4f\x3f\xed\x67\x7d\x4f\x91\x8b\x24\x04\x03\x1a\x28\x6c\x8b\x05\x09\x82\xda\xf2\x7b\xa4\x4a\x7d\x89\x97\x28\x55\x8b\x98\x36\xa2\xc5\xcf\xe3\x0b\x96\xb3\x64\x4f\xdd\x34\x20\xeb\x6a\xcb\xb2\xfd\x17\x5f\x68\x7e\x8a\x7c\x6c\x2d\xe4\x25\xde\xc7\xcb\xa5\x28\x4e\xf1\xde\x0a\xb7\xf2\xcf\x3f\x7b\xe5\x9f\xdf\x55\x79\x8b\xaf\xff\xc0\xda\x1b\x11\x9b\xca\x17\x48\xf7\x58\x23\x89\x32\x86\xdd\x9e\x2d\xde\x5b\xdb\x80\xd9\x06\xb0\x02\x43\x95\xf8\x41\xa9\xf4\x4f\xaa\x7b\xa8\x94\x9f\x64\x5f\x6e\xff\xe0\xab\xce\x62\xf5\x1f\x1a\xff\xd2\xe4\x7c\x9a\xcc\xdf\x8a\x39\x1e\xc5\x59\xc5\x13\xdd\x09\x48\xcf\xc5\x14\x18\xe6\xe7\xe7\x62\x85\x22\x92\x9d\x81\x62\x65\xe1\x5e\x77\x44\xe8\x53\x4a\x7f\x6b\xff\x25\x14\xbf\x8b\xe8\x5d\x7a\x96\x05\xf5\x55\x03\x41\x5b\x21\x69\xf3\x8e\xe4\xbb\x4a\x77\xf9\x4d\x75\x56\xe4\x15\x29\xed\xef\x73\xc9\x1e\x30\x4e\xd0\xf6\x33\xc1\x3f\xcc\xa6\x7c\x7b\x4e\x5c\x52\x0e\xe8\xba\x94\x1b\xff\xec\x45\x91\xcc\xae\xea\x8c\xc8\x7f\xe1\x3c\x7f\x51\x12\xc0\x1f\x4e\x79\x29\x0c\x6f\x68\x2e\x18\xc8\x34\x58\x38\x92\x3a\x5a\x1a\xa6\xe1\x2a\x0f\x97\x6c\xe8\xa1\xe2\x13\xe7\x92\x57\x95\xc2\x8d\xa5\x1a\x51\x51\xf1\xc5\x1e\x20\x6b\xeb\xbe\xbf\xcf\xd9\xd4\x21\x63\x65\x8a\x42\xf1\x1e\xbc\xf8\x07\x41\x6f\xb2\x50\xa8\x6a\x70\x3a\x93\x17\x53\x1a\xb0\xaa\x10\x51\x95\xaa\xa0\xed\x85\x90\x80\xf6\x33\x9c\x44\xab\x67\x2c\xc1\x28\xff\xce\xfa\xae\xea\x50\x2c\x44\x43\xb0\x8b\xea\xc7\xf6\x33\xf0\x32\xe5\xcb\x4b\x03\x7f\x09\xd5\xfd\x58\x0d\x87\x69\x86\x95\x9b\x82\xbd\x18\x8e\x77\xb7\x49\x32\xd3\x36\x3e\x9f\x29\xd8\x33\x23\x07\x8b\x11\x2c\xfa\x9e\x32\xa1\xc6\xe0\x16\xe2\x2b\x89\x36\x3a\xf2\x69\x81\x8d\xe7\x9f\xa1\x4d\x2a\x2c\x5d\xb1\x94\x4f\x6f\x94\x22\xbe\x57\x90\xcc\xee\x8d\x62\xc4\x54\xd7\x26\xcf\x77\x6b\x93\xe7\x3b\xb4\x49\x85\x4c\xca\x8d\xe2\xf0\x4e\xc9\x6e\xf7\x56\xa3\xfb\xe6\xe6\x32\x99\xc6\xde\x9e\xec\x9f\x3d\xbb\x33\x23\x2b\xfe\x66\x63\x16\x31\x24\x3b\xc6\x6b\xe7\xa5\x1f\x6b\xb5\x5a\x62\xc8\x05\x6b\xb5\x12\x04\x73\xa1\x34\x32\x97\xe7\x58\x2b\xcc\x3e\x6e\x7c\x32\x07\xe0\x4d\xe0\x24\x17\xc6\x4b\xa4\x16\x4c\x07\x52\x72\x61\x26\x0a\x41\x41\x82\x1e\xfa\xba\x29\xb3\x65\x07\x53\x6a\x8f\x71\xba\x54\xbe\x6e\x14\xc1\x78\x03\xb0\x9a\x90\xa4\x11\x42\xe4\x3c\x04\x69\xf1\x3b\x3e\xa9\x0f\xda\x91\xaa\x29\xcc\x75\xa3\xd8\xf2\xfe\xfb\xfc\xe2\x6c\x40\x94\xbe\x71\x1a\xe1\xbe\x2c\x3a\x91\xa2\x56\x05\xd2\x76\x26\xd2\x76\x3e\xdc\xe0\xc5\x50\xeb\xb8\x9d\x9d\x46\x59\x80\x8a\x8a\xcf\x4f\x23\xdc\x31\x46\xcf\xf9\x69\x94\x2b\x78\xfe\x34\xca\xc3\x3e\xef\xfe\x0a\xf5\xee\x2f\xfe\xa5\xdd\x60\xb4\x2b\x6c\xd8\x85\x69\x99\x5e\xfe\xe1\x03\xcc\xbc\x8c\xbe\xf1\x55\x25\x52\xb3\xa3\xd5\x92\x98\x5e\x9f\xc2\x86\x73\xea\x31\x31\xd7\x24\x7f\xcc\xec\xdb\xf3\x2e\x32\x4a\x3b\xfc\xfd\x37\xff\x00\x03\x16\x38\x69\xb9\x93\x96\x74\x13\xdc\x52\xb9\xd3\x82\x1f\x7e\xc1\xc8\xf0\xab\x45\x6e\x47\x07\x2b\xce\x35\x75\x8a\x13\xa1\x29\xed\x2b\x65\x90\x6a\xdb\xf5\x24\x00\x37\x8f\x5e\x59\x9b\x47\xdd\x4a\xd7\x6b\x92\xcf\x0a\x51\xda\x08\x8b\x50\x9d\x9f\xe7\x5c\x72\xba\xb9\xda\x14\x76\xa2\xda\xa8\x1f\x79\x47\x49\x97\x5a\x85\xb3\xb5\x93\xa1\xdc\x3d\x62\x41\x57\x00\xa5\x4d\xa8\x8a\xfd\xf5\x2a\xa1\xc5\xe7\x52\x15\xf8\x30\x99\x4a\x1c\xec\xd2\x3b\xee\xec\x12\x7a\x9e\x1e\x88\x34\x28\xf6\x85\x62\x5f\xd9\x65\xb6\x8c\x8b\x3e\x62\xe6\x74\x19\x11\x87\xfc\x3e\x98\xb5\x42\x39\x6d\x4f\x01\x14\xd2\xe4\x95\xe6\xad\xf0\xbc\xea\x22\x40\xad\x17\xc5\x01\xa4\x03\x4b\x51\x1a\x9a\xec\xef\xee\xc8\x2c\x92\xa7\x69\xd0\x4e\xf6\xf7\xe1\x37\xc4\xdf\x02\xab\x3f\x4b\xe1\x51\x68\x2b\xaa\x21\xd6\x39\xf4\x9f\x42\xd7\xd3\x82\x95\xdf\xa8\xd4\x37\x29\x5e\xd5\x0e\xda\xfa\x11\x08\xe3\x06\x39\x5f\xe8\x1c\xa6\xe9\x76\x5c\x93\x2e\x7c\xb8\xc3\xcd\xa0\x92\x02\xad\x52\x2b\x6a\x3b\x2e\x54\x17\xbe\xfa\x39\xd4\x72\x43\xad\x2a\x62\xdf\x5d\x21\xf1\x24\x3e\x1f\x32\x3d\x2b\xba\xd9\xa6\x67\x76\xcf\x4c\x91\x56\xbe\xa8\xbe\x2f\xf3\x76\x31\x83\x6b\xf8\x4c\x93\x0f\xbb\xed\x85\x2b\x57\xcc\x3b\xdb\xdf\x97\x21\xf2\xce\x86\x9f\xca\xc9\xfe\xbe\x33\x0a\x69\xc6\xfc\xca\x55\x3b\x43\x27\xc2\xfd\x94\xf1\x89\xc0\x3f\xb8\x9b\x12\x1e\xf1\x69\xd8\x53\xe3\x2d\x5e\x0c\x7c\x3d\x1f\xe5\x55\x1f\x88\x9d\x0d\x87\x7a\x68\xf6\x7c\x63\x0c\x0c\xb2\x19\xc1\xff\x71\x3d\x9a\xac\x57\x71\x5c\x1c\xc2\xed\xcf\xc5\xd6\x70\xae\xd0\xa5\x39\xd1\x70\x5e\xf3\xf1\xe6\x62\x42\xfb\xf6\x9d\xcc\xf8\x05\x2a\x49\x7f\x18\xfd\xe0\x65\xfe\xfe\xbe\x7c\xcc\x7d\xfc\x32\x35\x87\x86\x18\x5d\xc8\x7b\x0a\xb2\x68\x0e\x43\xdd\x3c\x87\x1f\xba\xa7\x8b\xef\x95\x9c\x67\x88\x88\xe7\x07\xe6\x59\x2b\xf3\x9b\xf4\x94\xb7\x72\xff\x69\x3b\x08\x7d\x2e\xf7\x19\xb8\x2c\x93\xca\xc2\xe9\xbb\x32\x93\xe7\x08\x3c\x72\xa0\xeb\x41\x51\x58\x8e\x7a\xe7\xe5\x11\x97\x1a\xe7\x10\x73\xe4\xc9\x21\x2a\xee\x2a\x95\x1d\x60\x2c\x02\xf2\xb6\x71\x6c\xd2\xe3\xab\xf4\x8d\xef\x22\x50\x10\x6c\x3d\x26\x7b\x0d\x59\xd4\x3e\x3e\xf0\xb2\xb0\x99\x01\x77\x69\x4e\x6f\x79\xd8\x44\x5e\x79\xde\x0b\x4e\x43\x9a\x89\xf3\xc5\x7a\xbd\x98\x45\xe0\x29\xa4\xb9\x48\x22\x8f\xdf\x9f\x3e\x0d\xfd\x26\xa1\xf5\xb8\x3d\xd0\xe1\x9c\x83\x64\xd0\xe7\x43\xe7\x05\x1c\x61\xf3\xa2\x7b\x81\x52\x85\x1f\xf0\xca\x3b\x5f\x16\xd4\x07\xaf\x2c\xcd\xba\x59\xc7\x14\xd7\x07\xa7\x2c\xcd\xbb\x79\x87\x0a\xad\xd7\x13\x1a\xef\x01\x9c\x8a\x3c\x15\x89\x98\x43\xe2\x19\xa4\x9e\x41\x32\xfc\xc2\xbf\x3b\xf7\x2a\x18\xa5\xc9\x42\x1f\x72\x47\x44\xfe\x2c\xa2\x22\xe0\x0d\x8b\x81\xb7\x8e\xbe\xd2\x3f\xa3\xd7\xb3\xa8\xe5\x65\x84\x76\xc6\x71\x07\x32\xda\xd7\x90\xd3\x5f\x77\xfb\x03\x45\xd4\x5e\xfb\xc5\xcf\xfa\x66\xfb\xc1\x04\x5c\x32\xa0\x85\x01\x7f\x81\x1a\xbc\xf9\x00\xca\x9f\xe6\x04\xca\x09\x94\x23\x28\x3b\x85\xb2\x11\xab\x63\xb0\x4e\x73\x02\xe5\x1d\x89\x95\xc9\x71\x84\xd2\x73\xfd\x92\xfb\xce\x66\x09\xcd\x04\x87\xa8\x3c\x8b\x9a\xe8\x69\xd9\xfb\x21\x72\x84\xb1\x87\xd5\x03\xae\x4e\x33\xc5\x5e\x86\x47\x0b\x00\x90\x2b\xe6\x72\x00\x64\x67\x16\x6b\x84\x71\x66\x31\x46\x18\x92\x15\x4a\x94\xcf\x00\xe7\xf6\x03\x95\xef\x9c\xb5\xb0\xe9\x26\x39\xca\xb6\x05\x74\x7b\x93\xec\x74\x92\xf7\x29\x7f\x78\xd6\x9c\xa0\x7e\x60\xe3\xc0\x63\x4e\x06\x71\xb5\x58\xac\x6b\xf4\x0a\xdb\x14\x93\xf1\xae\x7e\xcb\x99\xf1\xe5\x2a\x33\xa5\x81\x32\x51\xa5\x29\x96\xb7\x50\x75\x2d\xea\x0f\xde\x4f\x88\x84\xde\x27\x30\x6c\x1b\x52\x17\x2a\x9e\x8f\x29\xf9\x17\xc4\xf0\x2e\x08\xbd\x82\x0a\x7f\x2e\xa3\xf1\x0c\xf4\x8c\xbf\xa3\xcb\xef\x60\x0e\x53\xb2\x05\x32\xfa\xd6\x95\xd3\xdf\x22\x31\xfc\xfc\xcd\x16\x81\xb4\x4c\xb9\x58\xaa\xc2\xfa\x63\x03\x68\x64\xce\x1b\x65\xa2\x09\x9b\x2a\x44\xd9\xf0\xe5\x4d\xa8\x92\x62\xd7\x2f\xe3\x1c\x14\x7e\x5d\xba\xa7\xde\xef\x7b\x59\xa9\x09\x9e\x11\xae\x4d\xcb\x4d\x79\x23\x8e\x60\x58\x64\x8a\xa4\x20\xdc\xd5\xb8\xdf\xc1\x5f\xbc\xf0\x5e\x8f\x18\xd2\xc7\x06\x27\x5b\x81\xee\xf3\xfd\x5d\xe5\x21\xef\x46\x13\xb8\xc7\x77\x78\x43\x20\xb7\x08\xdc\xe3\x64\x94\xa7\x5c\x18\xa8\x17\xd6\x0e\x2a\xdb\xa1\xb6\xe8\x7a\x7c\xb8\x27\xb0\x62\xf9\x06\x6a\x5f\x8a\x39\xf4\xd3\xb1\x62\xfc\x1a\x66\x94\x1e\xb1\x0b\x31\x30\x33\x35\x34\x56\x78\x5e\x47\xb3\xba\xb3\x53\xb3\x9d\x51\x80\x04\x92\x31\x8b\xab\x2d\x2c\x65\x9d\x16\x32\xd5\x81\x9e\x6c\xb1\xc3\x2e\x84\xb5\x25\xfe\xe5\xdb\x73\xba\x68\xd3\x81\x14\x70\xf8\x42\xc1\x02\xda\x8f\x6a\xb7\xba\x8d\xf9\xc3\xf5\xec\x9c\x3f\xaa\x94\x80\x05\x4c\x3e\x24\x52\xc0\x64\xa0\x72\xb6\xac\x84\xb3\x39\x13\x3e\x1c\xb4\x9a\xc3\xbe\xd7\xef\xfe\x6d\xd2\xfc\x5b\xbb\xff\xb7\xc9\xc1\x2d\xfd\x34\x7d\x80\x0d\xe2\x6f\x86\x94\x4e\xd7\xc7\x1e\xbe\xed\xb9\x25\x16\xca\x72\xf9\x59\xac\x60\xd8\xa8\x38\xc4\xab\x0e\x39\x9c\xab\x7e\xd8\xe0\x9b\x3c\xf1\xb6\x5d\x5a\x7c\x5e\xbd\x3d\x3f\x9b\x8f\x66\x30\x79\xb8\x1c\xa5\x1e\x7e\xa8\x3e\xfc\xbb\xf7\x7f\xdc\x02\xf8\x6f\xde\xed\x65\x3a\xfd\x9b\xe7\x1f\xb6\x61\x52\xb6\x86\xb4\x7e\x49\xd0\xdd\xaa\xfa\x77\xcf\xd1\x7d\xa0\x73\x27\x50\x32\x60\x3c\x5b\x4c\x17\xab\x8a\xcc\x0d\x3e\x31\xc0\xbc\xd0\x92\x4c\x3b\x49\x79\x69\xa6\x54\x18\x81\xbb\x55\x4d\xd8\xad\x6a\x2d\x9f\x64\xb0\x19\x16\x5a\x8d\xa8\x44\x15\x94\x55\x9b\xc5\xa3\x34\x3e\x9b\xf0\xd5\xdc\xd5\xbb\x31\xd5\xf6\x74\xfa\x2e\x89\xe8\xfa\x32\x5f\x3c\x75\x30\x5a\x75\x0b\x54\x68\x45\x43\x03\xf1\x85\x94\xb9\xe2\xcb\xa8\xc2\xc1\xe4\x8d\x18\x5f\x9f\x27\xe3\x2d\x58\x94\xbe\xc1\x23\xc8\x5b\x90\x52\x3c\x29\x1f\x67\xcb\x2d\x28\x90\xaa\x0e\x88\x6f\x2b\x8d\x10\x80\x16\x5e\xbf\x0d\x7c\xe9\xdc\xfc\x2e\xce\x47\xe3\x2b\x0d\xc4\x17\xba\x32\x64\xbc\x8d\x24\x23\xc8\x8b\x73\x09\x32\xc3\xe5\x41\x25\xcd\x46\x32\x6f\x38\x41\x88\x16\xd7\x6b\x5d\x82\x3c\x7a\x2a\x00\xa9\x05\xf0\x86\x95\x40\x37\x56\x89\x06\x40\x5b\x48\xc1\x1e\x70\x0b\x0c\x48\x5c\xaf\x40\x14\x30\xf1\x3a\x5f\xec\x7d\xd4\xbc\xe6\x33\x12\x74\x13\xee\x56\x49\x84\xcf\xbc\xcd\x0c\xaf\x21\x6f\x35\x30\x8c\x33\x6e\x2c\xeb\x53\x82\x7b\xf7\x78\x17\x61\x62\x56\x99\x8e\x57\x8c\x77\xb1\xaa\x3a\x2e\x4e\x24\xd9\xe1\x9b\xe3\xf9\xda\x78\x47\x9f\x66\x91\x2d\x2f\x42\x9b\x31\x9a\x16\x95\x6e\xb5\xe9\x68\xb6\xf4\x66\x9e\x3a\xc9\x45\x43\x38\x6e\x50\x46\x95\x7f\x85\x7b\x0c\x78\xc7\xaa\xb6\xe5\x22\xa4\xcb\x8c\x0b\xbd\xe6\xbb\xf1\xb4\xd0\x67\x00\x52\xc4\x49\x4b\x38\x69\x11\xe7\xf9\xa8\x68\xab\x01\x52\xb4\xe7\x74\xb7\x50\xc1\x56\x20\x0c\xf1\x2a\xc2\x19\xb1\x1c\xb9\x59\xc8\x1c\x4f\x16\x63\xaa\x4b\x7b\x8c\x47\xaf\xe2\x6f\xa6\x31\xbe\xfd\xf0\x0a\x77\x5d\xcf\xd3\xb6\xba\x86\xfe\xfd\x5b\xd1\x78\xdb\x50\x9e\x8e\xb7\x9d\xb8\x76\xf0\x15\xe0\x6d\x3b\x8d\xd7\x5f\xae\xe1\xed\xfc\x7a\x1d\x7b\x0d\x9d\xb7\x21\x24\x4a\x8f\xcd\xef\x5b\x43\xb6\x7d\x0e\x4d\xf2\xcb\x68\x8a\xa7\x3c\xd2\xc5\x34\x99\x50\x4c\x5b\xb5\xbf\x46\x06\x58\x32\xc1\x95\xd6\xfd\x35\x74\x04\x20\x96\x75\x6d\xf8\xf7\xea\x74\xcc\xc6\x57\xec\x91\x25\xaa\x88\x9d\xa4\x2f\xeb\x2e\x1b\xb2\x86\x09\x05\xdb\x68\xf2\x6d\xc4\x0a\xd0\x6c\xf8\x32\x64\x88\x4c\xe1\x37\x00\xa7\x57\xf1\xcd\x6b\x05\xc5\x17\x84\x51\x08\x57\x05\xc3\x17\x00\x36\xf4\x7a\x43\x89\xed\xe8\xe3\xa8\x1b\x8a\xf3\x6e\x20\xc6\xf0\x6f\x02\xcf\x31\xfc\x5e\x74\x83\x4d\x61\xb4\xfb\x59\xb7\x86\xab\x0b\x1a\x8e\xfa\x30\x1d\xe5\xd0\xcf\xe5\x36\x76\x7e\x69\x9f\x83\xae\x4c\xab\x8e\x44\x6a\x08\x2d\xb2\x9a\xf5\x43\x8c\xae\x45\x73\x2e\xb9\x1f\x8d\x92\x0b\x37\x79\x12\x8e\xde\x5b\xc5\x25\x9d\x71\x49\x3f\x61\x38\x4b\xca\x03\x9e\xb6\xef\xdb\x11\x16\xd3\x8d\xcd\xd8\xf8\x72\xb1\x9a\x54\x1e\x46\x5d\xc5\x8c\x22\x63\xd1\x5c\xc1\x1c\x35\x0b\x68\xf3\x3c\x18\x0a\xba\x40\x3b\xc5\x3a\xd2\xd3\xab\xeb\x19\x31\x4b\x2f\xdf\xd3\xf6\x5f\x1d\x22\x61\xee\xeb\x0c\x9c\x22\x37\xbe\x52\xc9\x26\x17\x3d\x5d\xd1\xee\x69\xda\xe5\x66\xb6\xd2\x65\xb4\x33\x58\x6e\x7d\x03\x60\xd6\x8c\x58\x05\x71\x33\xf8\xbb\x61\x4f\x73\x20\x67\x62\x85\xf2\x94\x80\x34\x3b\xbe\xb8\x6a\x46\x19\x6f\x41\x5e\xac\xd6\xff\x17\x95\xbe\xbf\x6f\xe5\xa8\xdf\x14\x6e\x32\x78\xba\xd8\xc1\x68\x68\xa4\x30\x38\xa7\xb5\x4c\xc4\x7b\xa5\xc4\x44\x7b\xd9\x2c\xf2\xa5\xc5\x63\xfa\x4e\x30\xb9\xa3\x58\x4d\xce\x33\xb5\x87\x92\xcd\x0b\x95\x0c\x65\x5f\x45\xde\x1f\xff\xbb\xb5\x84\x29\x13\xf4\xb2\x83\x39\x46\xe7\xcf\x2a\xe5\x1a\x44\x99\x23\x58\xbe\x71\x2f\x89\x0c\xab\x34\x8b\x7f\x17\x39\xec\x0f\x26\x24\x76\xf1\x5e\xb5\x02\xbe\x4f\x00\x30\x42\x7a\x23\xf0\xb1\x9b\xd1\xfb\x83\xab\x9e\xd6\x12\x40\x68\xc2\x60\xd4\x04\x1c\xdc\x0f\x85\xbb\xf0\x27\x09\xb6\x91\x7c\x7e\x27\xe8\x23\xcf\x97\xb4\x6f\x7f\x14\xe0\x0e\x2d\xf9\x1c\xf2\x4d\xe4\xdd\xf7\x9b\x8d\x26\xe5\xd0\x30\xf9\x32\x2b\x5f\x26\xb3\x79\x59\x2b\x0b\xa0\xfa\x1b\xdc\x75\x2a\xe5\xb1\xe1\x95\x02\x23\x0a\x7c\x7f\x17\x25\x05\x29\xc8\x63\xc1\xa6\x0e\x5c\x85\x77\xea\x7b\x83\x95\xf2\x8e\x52\x92\x61\x4f\x9e\x83\x6d\x53\xe1\xfa\xb4\x32\x5f\x7e\xbf\xbf\xcf\xea\x2e\x77\x33\x5a\x88\x4f\x6d\xbc\xbe\xfa\xc8\xc1\x40\xf5\x71\x43\x9e\x36\xee\xaa\x54\xe7\x64\xae\x3a\x17\xec\x6f\x50\x4b\x9e\x51\x31\xfb\xfb\xab\x98\xb4\xc9\x0d\xec\x4a\x90\x8f\x92\x93\xed\xda\xc6\x74\x3c\x6f\xd4\xb6\x99\x6d\x8e\xda\x4e\xb5\x0e\x3b\xc2\x3b\x77\x31\xce\x8b\x18\xbe\xfc\x7a\xc5\xc5\xca\xce\x2e\xd5\x56\xcc\x85\xe9\x51\x6e\xa7\x11\x86\x0b\xc1\x46\xea\x23\xc6\x1c\xa4\x46\x8c\x02\x7d\x56\x13\x53\xe4\x88\xb4\x5b\xbc\xa5\x79\x24\xfb\x4f\x84\x0b\xb0\xfc\xa8\x57\xb4\xd8\x24\x49\x7b\x44\x7e\x09\x81\xfc\x2e\xe3\x6d\xf8\xb5\xad\xd8\xd8\xa9\x40\x85\x9c\x6d\x21\xaf\x94\x53\xd2\x37\x32\xd9\xad\x08\x0b\x7f\x5b\x29\x06\xcd\x2e\x48\x4b\x7c\xf7\xb2\x4c\x16\x5d\x5c\xb1\x1c\x8d\x62\x17\xc5\xcd\xb9\x7b\x39\x12\x5f\x15\x62\xb4\xda\x2e\x89\x91\x54\x31\xe3\x42\x11\x26\x96\x1c\x25\xdc\xde\x9a\x61\x4d\x52\x55\x39\x8b\x42\xd0\x39\x39\xc1\xc9\xe9\x54\xcd\x1e\x4e\xc1\x9a\x8c\xab\x43\x29\xad\xe2\xe5\xf5\x14\xfc\x78\xde\xf6\x55\x1c\xfd\x71\x76\x85\x0b\xc0\xb8\x1a\x46\x9e\x1d\x02\x78\x99\x7c\x8f\x77\x53\x49\xeb\x9c\x45\x94\x32\xce\x5a\xb4\x09\x01\x63\x73\x48\x48\xce\x10\xf0\xb3\xe7\x51\x68\x5d\x70\x36\xc9\x0e\x26\x59\x73\x92\x1f\x4c\x78\x51\x78\x7d\x19\xaf\x47\xa7\x1e\xad\x64\xf8\x07\x93\xb9\xba\x01\x86\xa9\x5c\x62\xe7\x05\x28\xfc\xdf\xb3\xb7\x23\x2d\xb3\x56\x34\xc1\x3d\xf9\xfc\x96\xc3\x5b\x0e\x6f\x7b\xc1\xc6\x61\x76\x7f\x3f\x49\xbf\x4d\xe6\x09\x38\x6a\x40\xd9\x21\x4d\xe9\xcf\x6c\xfa\x5b\x08\x6f\xa4\x07\xba\x67\x71\x65\xc5\x93\x9a\xac\x46\x6f\x71\x37\x1d\xae\x7d\x02\x65\xf2\x36\x60\x9e\x04\xfe\x35\x08\x04\x88\x98\xf7\x5c\x50\x93\x50\xa0\xeb\x19\x7a\xb6\x24\x45\xc8\x2e\x46\xd3\xe5\xe5\x48\xa8\x58\x14\x29\xdd\x28\x85\xaa\x87\xe6\x06\x8b\x4b\x39\x27\x9a\x1d\x22\x85\x34\x01\x1b\x5c\x28\x18\xba\x1b\x34\xf4\x34\x44\x03\xa6\x99\x57\xf0\x03\x63\x4f\x03\xf7\xb0\x7c\x88\xa3\x41\x28\xc2\xa1\xb8\x58\x25\xc4\x6a\xd4\x7e\x82\x07\xff\xaf\xbe\x56\x51\x31\x8c\xa7\x46\xe4\x9f\x5b\x69\x84\xf8\x4a\xb2\x51\x85\xa8\xd2\x24\x87\x51\xeb\x28\xc0\xdd\x87\xef\xd1\x85\x6d\x87\x82\x5a\x36\x6a\x3f\x26\x51\x92\x3f\xc5\x5f\xee\xcd\x97\x6d\x96\x05\xf2\x6c\xeb\x28\xb4\xa0\x47\xd2\x38\x00\x6e\x9f\xe0\xb7\xa4\xe0\x58\xed\x45\x63\x29\x42\xf5\x3c\x19\x8b\x04\x6b\xca\xb2\xeb\xd2\xdf\x28\x00\x97\x63\x2f\x20\xdf\xfa\x1f\xe0\x1e\x2e\x44\x2a\xd6\x62\x2a\xae\xe8\x9b\xc6\x9c\x74\x57\x7f\xb8\x9e\xb9\x7e\xac\x5a\xb5\x9f\xc9\x55\xfb\x45\xa4\x5c\x56\x91\x46\x8b\xb6\x1a\xe6\xe0\x51\x8e\x87\x59\x04\x2d\xdc\xc2\x4d\x20\x39\x3c\xe5\xf0\x94\x0b\x6f\xaa\x2e\x04\xc4\x93\x86\xd3\x88\x6b\xa2\x5b\x13\x88\x1d\x78\x00\x36\x1d\x62\xea\xfb\x2d\xdd\xec\xe8\x1e\x1f\x4e\x45\x76\x10\x4d\x45\x8e\x7f\xb0\x00\x20\xe8\x5d\x45\x69\xfb\x26\xc6\xcf\x38\x87\x30\x53\xe5\xa7\xa6\x02\x81\x0f\x89\xc5\x47\xd8\x05\x80\x9b\x26\x67\x08\x5b\x57\x3e\xee\x4e\x69\x22\x9c\xfa\x1a\x00\x99\x1d\xd9\x4a\x3e\x5d\x7e\x88\x6a\x32\x08\x86\x87\x78\x41\x23\x3d\x87\xf8\x4c\x8e\xfe\x95\xef\xdb\x4b\xe8\x8b\x48\x7d\x69\x12\x0b\x2c\x06\x3c\x1a\xf8\xf5\xa1\xd4\x05\x16\xe3\xe5\xf0\x9a\xc3\x2b\x96\xc5\x3a\x41\xd9\x0b\xaa\xf3\xe5\x18\x26\x9f\xd7\x34\xaf\xfa\x47\x54\xfc\xd6\x49\x06\x09\x0f\xb2\x50\x77\x90\xaa\xef\x3b\xde\xa2\x67\xd8\xc0\xb0\x66\x59\x3c\xb9\xbd\xfd\x07\x7f\x3a\xf0\x94\x59\x5b\xf8\xea\x80\xa7\xc9\xe7\x70\x4f\xf9\xfa\x1e\xb0\x0f\xcd\xb9\xcc\xb0\x02\xf8\x90\xfb\x5d\x84\xb5\x22\x0f\xa1\x2d\xfa\x1b\x61\x1d\xfd\x03\xd5\x85\x10\x95\xd3\x73\x4a\xc7\x7c\xb9\x95\xee\xf7\x58\x49\x51\xad\x95\x96\x72\xb7\xb4\xd4\x14\x37\x84\xb0\xf6\x73\xe7\xd8\xcd\x57\x20\xd4\x8c\x33\xfa\x5d\x7a\x55\x64\x0a\x9b\x62\xb6\x91\x61\x54\x4d\x86\x5e\x15\x19\x77\x21\x7b\xeb\x80\x88\x98\x9a\x08\xbe\xd9\xac\x7c\x5d\x8a\xba\x73\x17\x47\x3a\x47\x45\xf4\x99\xac\x9f\x75\x9b\x0e\xc3\x0a\x5b\x95\x69\xa2\xfc\x94\x58\xb0\xb8\xd2\xc6\x6c\x67\xae\x74\x8e\x9d\xb8\x52\xd8\x8a\x2b\x6d\x75\x77\x2a\x4f\x63\x1b\xa2\x0a\xa4\x08\x4a\x43\xbb\x13\x39\x89\x7b\x27\xe3\x72\x34\x93\x25\x28\xfb\xbd\x53\x11\x0a\xd9\x10\x93\x10\x45\x8d\x87\x80\x9d\x68\x31\xaa\xa1\x44\xef\x8a\x0e\x9b\xf6\x9d\xe8\x64\x48\x83\xf0\xfb\x9c\x2b\x3b\x0d\xa0\xbe\x41\x17\x7e\xc1\xdc\x71\xdf\xe4\x6d\x7c\xb2\x73\xca\xb1\xd3\x1e\x44\x32\x5a\x70\x6d\xd3\x45\x9a\x9e\x19\xa5\x7c\x5f\x71\xc7\x9d\x58\x75\x1a\xba\x28\xa4\xca\xcb\x5a\x2e\xc0\x28\xd1\x94\x1a\x28\xcd\x31\x36\x12\x0d\xca\xbc\xef\x62\x8e\xb6\xfb\x7c\xb1\x92\xcb\x2c\xfa\xd5\xda\x7c\x39\xf7\xe9\x10\xb2\x3a\x2f\xac\x51\x60\xc2\xab\xd6\x2d\xde\xd1\x80\xc5\xe9\xe4\xdf\xa8\x51\xeb\xdd\xb0\x67\xf0\xd5\xe8\xc5\xcb\xb1\x43\x9e\xf7\xa9\x71\xcc\x17\x36\xa2\x9c\x2f\xb9\x88\x9c\x1b\x63\x68\xca\x05\x21\x70\x82\x47\xf3\x09\xad\xcf\x21\xc3\x2a\x7b\x32\x54\x3b\xc4\x4c\xb2\x59\x26\xa2\xad\x5f\xd6\x96\x13\x0b\x09\x8f\x7e\x6b\x21\x0d\x7d\x35\xde\x67\x4e\x68\x52\x3c\xd9\xb0\x98\x79\xfe\x01\x99\x1b\x5a\x64\x36\xac\xc3\x50\xbf\x75\x78\x17\x37\x6a\xa4\x13\x97\x6a\x9c\x2b\x05\x01\x70\xc6\x16\x3e\x38\x0e\x3e\x84\x1c\x66\x01\x6f\x8b\x8b\xd0\xe0\xcf\x57\x56\x50\x64\x29\x35\xd0\x3a\xf5\x28\x89\xab\xd7\xa1\x5f\x95\x8b\x1b\x80\x72\xc9\xd9\xbd\xca\xc5\xaf\x90\xab\xd9\xd4\x0d\xca\xac\x11\x44\xb6\x1c\x43\x4a\x55\xb3\x86\x3f\x16\x3f\x0e\x6b\x54\x4a\x16\x69\x3d\x6d\x64\x0d\x71\x03\x6a\xae\x30\x72\xc6\xc8\x2d\x8c\xbc\x21\x2e\x2d\x8c\xa5\x24\x22\xc7\x49\x03\x97\x59\xe5\xf8\x88\x2e\x81\xf6\x72\xd0\x17\xac\xb0\x4b\xb6\xd5\xf6\x8b\x92\xb6\x5d\xa4\xa8\x69\xa3\x5a\x01\x25\x74\x63\x24\x85\x8d\x18\x95\x54\x6c\x22\xc8\x9f\x76\xd2\xb6\xf0\xa7\x4c\x7c\x89\x3f\xdb\xc3\x63\xfe\x14\xea\x7d\xf8\x73\xa8\xd8\x44\x8c\x4b\x55\xc7\x9d\xe5\x70\xd9\xcd\x2e\x33\x11\x57\xfc\x6c\xf1\x63\x34\xa2\x66\xe3\x8a\x95\x9b\x1f\x5d\xd7\x5d\x4f\x63\x9c\x69\x4d\xd5\xba\xb2\x31\xe3\x5e\x3b\xf4\x8d\xe5\x5c\x2c\xef\xc0\x0e\x34\x32\x4e\x93\xaa\x3e\x37\x02\xf8\xf6\xd6\xa3\x54\xb0\xd9\xe7\xf1\x25\x0c\x41\x8b\x15\xa1\x7b\xbe\x8c\xa3\xed\x59\x5f\x7e\x00\x06\x6a\x8c\xc9\x64\xb8\x79\x92\xdc\x10\x05\x2f\xf5\x6b\x95\x6e\xd0\x35\xa6\x9c\xee\x99\x14\x98\x85\x6c\x21\x03\xa9\xd0\x61\xaa\x86\xbc\x24\x25\x12\xb3\xc5\x75\x1a\x03\xc1\x55\x0d\x91\x17\x2a\xdd\xb7\xd0\xaf\xeb\x38\x7f\x21\x93\x7d\x6e\x68\xe4\xcf\x97\x47\x77\xfc\x2e\xbe\xd1\x0a\xc1\x2a\xc6\x05\x4f\x1e\xdc\x78\x46\x29\x1a\xa0\x4e\x66\x1b\x64\xdd\xb4\x30\xea\x04\xc5\x12\xed\xc9\x60\x14\xf6\xcc\xfa\xc3\x65\x12\xaf\x46\xab\xf1\x65\x5e\xb3\x06\x31\xbe\x5e\xc9\x35\x08\x31\x89\x97\x60\xa3\xd9\xfb\xe7\xb0\x68\xb8\xaf\x2e\x8d\xe8\x07\x48\x73\x65\x34\x41\x61\x32\xf9\xea\xd8\x51\x9b\x5e\x23\x8b\x12\x0f\x5f\x7c\x04\x95\xe9\xe1\x71\x09\xa6\xa9\x77\xd7\x98\xaf\x1c\xb8\xf5\x8d\xe6\x3d\x63\x3e\x39\xa5\xca\x76\x47\xe4\xf7\xf4\x01\x80\x4a\x69\xea\xc9\xc5\x24\x1a\x63\x27\x51\x55\xe2\x12\x68\x38\x94\x75\xc2\x85\x00\x73\x00\x4e\xbc\x6f\x46\x13\x5e\x93\xec\xe1\xaa\xd1\xfe\xfe\x98\x57\x40\xf1\x8f\xcf\x0b\xc6\xea\x64\x07\xbd\x44\xef\xe5\xce\xc5\x49\x3c\x8d\xd7\xf1\x03\x87\xbf\x2a\xfc\x26\xfd\x6c\x91\xda\xed\x6d\xe0\xdb\x6b\x2a\xf6\xd2\x2c\xe5\xb5\xda\xc5\x6a\x11\x12\x87\x5b\x38\xc7\x16\x52\xef\x46\xc2\xd8\x6a\x4a\xc6\x46\xc4\x24\xe0\x92\xfc\x40\x1e\xaa\x58\x95\x97\xa4\x27\xed\x92\xaa\xe0\xfb\xfb\xd5\xab\x52\x8e\xe2\xbd\xa9\xaa\xa6\x80\x4b\x37\xc4\x9f\x5c\xab\x50\x14\x54\x83\x4e\x44\xa0\x1a\x92\xe7\x5d\xbc\x2c\xcf\x77\x2b\xc8\xbe\xa0\x89\xbd\x02\xb8\xd0\xd2\xaa\x40\x78\xe6\x34\x5c\x15\xc6\x2f\xa4\x1a\xea\x3c\xbb\x82\x92\x8e\xec\xbe\x60\x09\xf3\x33\x9d\x95\x57\x27\x37\x06\x60\xb4\x7b\xc7\xe9\x84\xc4\x76\x68\x2a\xa8\x4d\x97\xeb\xb4\x13\x51\x46\x75\x28\x12\xc8\x26\x27\xd5\xc2\xbe\xdd\x64\x61\x02\x6e\x2a\xa5\xa1\x6d\x7d\x81\xcf\x7b\xef\x4c\x6e\x7b\x31\x14\xba\xdf\x3a\x71\xa7\x63\x55\xae\x3a\xa9\x52\x26\x28\xce\xf0\x36\xbd\xd7\x56\x87\x76\x12\xd3\x5a\x67\x64\xac\x10\x46\x5e\xe7\xb7\x0c\x83\xa3\xf3\x63\x1e\x01\x74\x7b\x3f\x61\x25\xa4\x00\x21\xb4\x76\x41\xeb\x27\x59\x64\x14\xb8\x3f\xc9\x0e\xcd\x5b\x57\x47\x6d\xd1\xfc\x8f\x23\xbb\x03\x41\x2d\xa2\x31\xe3\x1e\x70\x95\xf0\xeb\xd3\xc4\x5e\xbc\x44\x76\x3d\xb3\xa6\x5b\xd3\xcb\x27\xf7\xef\xe5\xc6\x36\x92\xf7\x3e\x1b\xe1\x3e\x50\x2e\xce\x62\xd1\x7c\x0e\x0e\x9b\x13\xc3\x96\x6e\x2d\xfe\x1c\x69\xfa\xa6\xa5\xc0\x4e\x20\x31\x73\x7c\xc8\x6a\x48\xba\x0a\x3e\x10\xca\xf1\x57\xcb\x5b\xa6\xce\x88\xe0\xdb\x7d\xda\x0c\x59\x15\xc3\x98\x67\x2f\xaa\x9a\xaf\xd8\x92\xd3\xfb\xaf\x9e\xe8\xac\x6a\x05\xa5\xc2\x12\xbc\xe4\x81\x5a\xa3\x5a\x9d\xc5\xd1\xee\xa4\x7a\xa1\x1f\xe0\xd6\xf9\x22\x52\x04\xde\xad\x5a\x7b\x36\xcc\xb6\xb0\x90\x9b\x84\x4b\xe1\xc6\x9a\x55\x37\x4f\x9a\x8f\x9e\x7d\xf3\x58\x73\x05\x8c\x41\xa0\xcf\xc2\x55\xe4\xd4\x57\xd3\xbe\x7a\xa8\x21\xa5\x92\xfd\xd6\x08\x43\x07\xb6\xc1\xef\x64\x6b\x92\xca\xd0\x6a\xe6\x8b\xbf\x7d\x82\xb1\x27\xb7\xa8\xd0\x90\x9b\x98\xaf\xeb\x64\x2d\x23\xcb\x10\x43\xc5\xd1\x8a\x7f\xc5\x86\xb8\xaf\xe5\x94\xc0\x90\xf4\xd1\x1e\x5d\x70\xe6\xda\x92\x4f\xc9\x70\xd3\xad\xc4\xe4\x83\x55\xf6\x91\xa3\x77\xf8\x75\x9c\x5c\x2d\x68\xca\xd4\xb9\xc3\xa6\xf2\x9b\xbc\xec\x02\x93\x1e\xa2\xa3\xa7\xf1\x11\xc9\x74\x15\x4d\x36\x06\x93\x48\x33\xe2\x7c\xc4\xb6\xbe\x61\x43\xdf\xc7\xcf\x19\xe8\x8e\x8e\x59\xe1\xd9\xba\xca\xfd\x9b\x85\x11\xcd\x95\x82\x45\x33\x32\xdf\xb7\xa3\x3f\xfe\xb7\xee\x09\x49\xfc\x29\xd6\x1f\xb2\x69\xbb\x8f\x24\xee\x3d\xc8\x11\x01\x1e\xde\x28\xbf\x61\x73\x37\x2a\x06\x5f\xd1\xd2\x10\xa6\xa8\xeb\xba\x13\x3d\x8d\x2d\xa9\xa9\x77\xa2\x55\xf4\xaf\x5d\x11\x47\x1f\x37\x96\xfb\xcc\xe1\x1b\xab\x7a\x35\xa5\x90\x46\xb1\xb6\xcb\x6d\x02\xf1\x2a\xb9\xab\x77\x4b\xad\xe5\xee\x4d\x64\x54\x07\x97\x61\x23\x99\x48\x21\xbf\x99\x6d\xb9\x09\xef\x0d\xe1\x41\x66\xd3\x7c\x4f\xa7\x1b\x8a\x90\x21\xed\x08\x59\xe0\x51\xc2\x34\xa2\x1f\x1b\x43\x9e\x6e\xe2\x0a\xf5\x54\x65\x8e\xda\xcb\x78\x35\xc3\x1d\x64\x0c\x91\xd9\x35\xbf\x16\x82\x24\x20\x11\x7a\xbc\x40\xf4\x4e\x2c\xc4\xe2\xe2\x22\x8d\x01\x97\x7f\x6b\xcb\x14\x73\x55\x7b\xbd\x82\xc4\xef\x18\x50\xd1\xfa\x48\x64\xaf\xb9\xe1\x3b\xed\x5c\x32\x34\x75\x1e\xdc\x97\xb2\x90\x85\xa2\x9d\x10\x3a\x3c\x23\x47\x3a\xf4\x85\x1d\xf6\xb2\x9a\x0a\xef\x6e\x59\x34\x75\xec\xc2\x56\x28\x73\x0b\x1d\xcd\x50\x52\xeb\x59\x27\x7f\x37\xb6\xb5\xb7\x37\x88\xa2\x6c\xac\x1e\x4e\x65\xd1\xd9\xce\xaf\xe5\xd6\x49\x66\xb7\x84\x42\xd0\x5f\xe3\xd5\x02\xf7\x98\x96\x52\xaf\xe5\x29\x07\x1b\x28\xcf\x26\xd8\x20\x1d\x03\x90\x23\x22\x4a\xfe\x76\x37\x16\xf8\xd9\x82\xb2\x4a\x7b\x91\x6e\xf8\xb5\xcd\xd5\xda\x89\x10\xa3\xd6\x2c\x8b\x57\x09\x26\xa5\xed\xa3\x19\x6f\x1f\xad\x15\x9b\x64\x8b\xa8\x6b\xae\x58\x94\xbb\xb1\xc5\xb8\x3b\xf2\xc5\x2a\x55\xcb\x98\x69\x2c\xc5\x16\x41\x14\x5f\xd6\xde\x94\x0f\x5b\x97\xd2\x3f\xa8\xfc\xd9\x7f\x53\x79\xf3\xdd\xf2\xe6\x26\x6f\xae\xc5\x71\xbd\xde\x2d\x33\x22\xea\xec\xf0\x22\x09\x14\x4d\xa7\xd5\x44\xf6\x8e\x68\x0c\xf9\xc6\x1b\x9e\x2b\x0e\x58\xa3\x3d\x70\xcf\x8d\x81\x5b\x6a\x2c\x66\x81\xf8\x8b\x51\xf6\x3d\x5b\x86\x14\xb7\x14\xd6\xa1\xbd\x8c\x27\xd7\xe3\xf8\xd5\xf5\xac\xe4\x7b\xcc\xfd\x2d\x1b\x9b\xa0\xe4\xc1\x68\xd8\xc2\x9f\x73\x3c\xd6\x8f\x8b\x61\x81\x3a\x08\x18\xe0\x2b\xad\xf5\x31\x20\x35\x11\x3a\xcd\xf2\xdc\xbb\x28\x51\xfb\xe2\x18\xeb\x14\x32\xf5\x3d\xf8\xd3\x8c\x90\x63\x34\x1d\x48\x86\xd7\x41\xde\xe1\x01\x1f\xc6\x33\xc9\x92\xba\xc6\x50\x3d\x54\xc1\xcd\x05\x46\xf2\xfe\x23\xa4\x87\xe7\xb5\x18\x5e\x94\xb1\x39\xe9\x5a\x76\xbe\x0c\xad\x8d\x68\xc8\x9d\xda\x8d\xca\x2e\x27\x3b\x15\x6f\x7b\xaf\x50\x7c\xdd\xdc\x69\x32\xbd\x5c\x5c\xc7\xeb\x75\x89\x11\x63\xfc\x0b\xcd\xcd\xc7\xef\xb4\x41\xe7\x96\x1d\x0c\x49\x0f\xe8\x06\x9f\xaa\x0f\x2b\x2a\xfa\xf1\x42\x0a\x1f\xa3\xbb\x82\x4d\x96\xee\x17\xdb\xdf\xde\xe2\x14\x68\xc0\xc4\x08\x29\x2d\x58\x65\xe4\xf7\x13\x7f\x53\xa4\x98\x07\xf8\x0d\x07\x51\x5b\xb2\x25\xfc\xc3\x8e\x92\x7d\x1e\x6c\xc4\x4d\xf2\xf6\xed\xb4\xb6\x56\x30\x3c\x84\x22\xed\x88\xf4\x08\x27\xaa\x60\x93\x83\x42\x35\x33\x55\x4d\xa8\xb0\xde\x50\x66\x2a\x07\xc5\x07\xc3\x08\xc6\xa6\x80\x16\xb3\xc2\x72\x4d\xd3\xd0\xd2\xb3\x34\x2c\xd6\x55\xa3\xd1\xfd\xde\x30\x05\x96\xc1\x7d\xe1\xb7\x45\x87\xaa\xcc\xd9\x47\x44\xc5\x9d\xb4\xe9\x51\xe4\x39\x44\x5a\xfa\xad\xc5\x67\xaf\x0e\xbd\x0e\x4c\x52\xfd\x5e\x72\x7a\x05\x59\xaf\xfc\xf4\xa8\x29\xb3\x5c\x39\x59\xae\xac\x2c\x93\xac\x97\x76\x40\x9d\x8f\x0e\x1c\xda\x1b\x16\xf0\xa2\x15\xa5\x61\x3f\xed\x1c\xa6\x21\x50\xee\x42\x03\x06\xa7\x0b\xfc\x2e\x11\x44\x75\x8d\xd2\x02\xa1\xd8\x0d\x11\x67\xcb\x91\x1d\xa4\xef\x9e\xea\x75\x15\x85\x87\xf3\x3f\xa5\x56\x30\xeb\x5e\x38\xcb\xfa\x88\xe5\xa0\x1c\x46\x8b\xf2\xf2\x7d\x09\x2b\xba\xaa\xd1\x42\xa7\xba\x1f\x60\xd4\xa8\x1e\x6e\x70\x3c\xe1\x23\x1f\x7a\x5a\x9c\xae\x31\x56\xfc\xac\xd2\x45\xd5\xa9\xd2\x4d\x35\xcb\xac\x30\x9f\x15\x99\x80\xbf\xd4\xef\x8a\x53\x52\x7a\x5f\x09\x8e\x67\x4a\x46\x24\xa2\xbf\x2b\x6b\xbe\xcf\x79\xd0\xc5\x5c\x5f\xae\xe2\xf4\x72\x81\x2b\xc5\x40\x70\xee\x60\x51\x36\x0b\x57\xee\x5d\x97\x0e\xb0\x6e\x30\x43\x42\x05\x39\x0d\xa1\xd5\x2e\x56\xf1\x3f\xae\xe3\xf9\x38\xef\x87\x5d\x68\x41\x5a\xcb\x98\xf5\x7c\x28\x04\x0b\xa2\x29\x18\xda\xe6\x64\x8e\x2b\x3c\x86\xc4\x20\x69\x82\x96\x7a\x08\x77\xc1\xe0\xec\x21\x30\xe7\x65\x94\xd9\x69\xe0\x17\xf7\xa7\x64\xd6\xec\x2d\x3b\xe5\x4a\x83\x22\xed\xef\x67\x4f\xe5\x0b\x06\x37\xf7\x34\x07\xf8\xb1\x23\xa1\x88\xa5\xa6\x1c\x90\x6b\x28\x66\x14\x9e\x95\x4a\x6b\x46\x57\xf4\x60\x22\x38\xdb\x0b\x2e\x48\x86\x1c\x45\x5d\x59\x0c\x40\xc4\x0d\xa0\x66\x84\x2c\x7b\x67\x89\x52\xb6\xeb\x4b\x12\x2f\x4b\xbd\x2a\xfd\xab\x64\xfe\x0a\x8a\x79\x1b\xa7\x66\x1d\x53\x26\xdd\x7b\x96\xb8\xa2\x45\x42\x99\x5b\x3a\x7f\xab\x8d\x01\xf1\x68\xb3\x1b\x41\x53\x23\xeb\xbc\xba\x45\x9c\xd3\x6d\xe2\xa4\xaa\x3b\xd1\x96\xd2\x28\x7d\xd5\xcd\xcc\x0a\x02\x91\xb7\xcf\x59\x55\xc8\xed\x5b\xdc\x9d\xc4\x98\x22\xf3\x37\xdd\x5a\x5e\xb9\x3c\x9b\x57\xd3\x94\x3b\x6e\x3a\xd1\x2d\xbf\xe7\x88\x58\x27\x58\xc4\xed\xf5\x26\x37\x30\x85\xd5\xef\x39\x04\x05\xf5\x77\x05\xba\xc0\x40\x7d\x7f\x1d\x4d\xaf\x78\x71\x75\x09\x7e\x40\xb2\xb8\x4e\x5f\xc9\xf0\x5b\xdb\x96\x1d\xe5\xe9\x15\x02\x9e\x21\xe9\xbb\x17\x21\xad\xcf\x39\xaa\x24\x5a\x6f\xe7\x95\x57\x41\xcc\x10\xc0\x2c\x94\xc2\x30\x49\x5f\x85\xe9\x7a\x09\x9d\xee\x2c\x62\x8e\xdd\x1c\xd0\x47\x4d\xad\x98\xb0\x53\x98\x6f\x08\x8e\x96\x4b\x70\x01\x69\x69\xb6\x8c\xa7\xd1\x7c\x17\xce\x25\xf5\x8c\x7a\x60\xdd\x5f\x5d\x26\x17\x6b\x5e\xaa\x25\x57\x78\x96\x4c\x38\x74\x05\xc6\x94\x30\x7c\xb3\xa0\xf0\x14\xd7\x34\x99\x35\xc7\x25\x90\xdf\x2b\xb4\x40\xdf\x53\x8b\x88\x94\x1e\x15\x92\x5d\x7a\x29\x06\x03\x1e\x99\xa5\xf2\x62\x6b\xca\x26\x6b\xcf\x16\x93\xc8\x21\xdb\x52\xec\xfa\x5d\xb7\x38\x05\xe7\x0f\x5c\x05\x7a\xb8\x89\xf2\xf3\x31\x67\x1d\x40\x00\xcb\xb9\x98\x4f\x8c\x56\x66\x32\xfa\x65\x66\x29\x9b\x22\x9b\xf5\xb6\x7f\x07\xd8\x69\x4d\x5f\xaf\xe7\x67\x4d\xbb\x08\x90\x93\xd2\x31\x8b\x25\x67\x0d\xdf\xbe\xbb\xd6\x68\x53\x55\x05\x8d\x3a\xe1\x16\xa3\x52\x47\x53\x3d\x23\x45\x45\x12\xef\x93\xa5\xfc\x18\xb9\x50\x0f\x65\xf9\x42\xe2\x8c\xd9\x95\x51\xe8\xec\x4e\x93\x02\x05\xa0\x62\x6a\x22\x52\x20\x05\xe4\x6c\x08\xd0\x04\xba\x0e\x0e\x40\x16\x16\xa4\x87\xe5\xba\xaa\xce\x71\xf3\x00\xee\x13\x9b\x6e\x22\x45\xf0\x03\x30\xa6\xcd\xf6\xf7\xe1\xa9\xe7\x23\x9b\x55\x58\x0b\xa4\xb0\x28\x52\x90\xe4\x17\x4b\x4a\x94\x9c\xe8\x1e\x4b\xb2\x20\x19\x59\x9c\x2b\x0d\x03\x45\x36\x55\x96\x9a\x0d\x72\xb0\x55\x0f\x32\x09\x62\x8f\x68\xd0\xbe\x33\xb7\xf8\x17\xb4\x59\xdc\x01\x7d\x29\x0b\xa7\xcc\x1c\x26\x59\x5b\x06\xc3\x0f\x7e\x7f\xc0\xb9\x1d\xb5\x5f\xba\x50\x8f\x08\x86\x39\x5d\x41\xcc\x88\x59\x6c\x1d\xc4\x5a\xb8\x8d\x81\x58\x76\x8b\xf5\x48\xa8\x7b\x75\x02\xc3\xaf\xb0\x1a\x1d\x3d\x8f\xd1\x24\xa2\x0a\xdb\x34\x80\x2f\x14\x0b\xc9\x37\x59\x96\xa8\xe9\xa6\x21\x62\xb3\x22\xb1\xa5\x70\x78\x44\x62\x4b\x20\x36\x13\x4e\x0b\xe9\x63\xa1\x0a\xba\xb9\xfb\x23\x11\x7d\x1f\x8c\xd4\x87\x9f\x82\x79\xa5\xd8\x22\x5f\x62\xb0\x5c\xfe\x9c\x68\x0e\x35\x57\x8e\x5a\xa6\x0f\x47\x1f\x15\x0b\x5d\x85\x0a\x5a\x01\xb3\x0d\x60\x1e\x4f\x71\x5e\xe2\x40\x0e\x0f\xd4\x56\xf0\xcb\xde\x41\xb7\x68\x85\xab\x2d\x1b\x23\x37\xc3\x6e\x80\xcb\xf6\x66\xd0\xa1\xcf\xa0\xc2\x32\x19\xc4\x73\x8b\xe2\x93\xb8\xd6\x1e\xc5\x42\xe1\x76\x0b\xa3\x49\x8c\x12\x92\x01\x51\x4a\xcd\x33\x5b\xa4\x6b\x15\xa1\x68\xd7\x7c\xa4\x24\x9c\x71\x12\xc7\xc5\x3e\x57\x9f\xef\x6b\x40\x8e\x31\x57\x16\x50\x40\x5d\xdc\x54\x6f\x7a\x13\x85\xd8\xe5\x28\xbf\x87\x1d\x0c\x2d\xc9\x11\x71\x33\xbb\xc7\x71\xa8\x5c\xc4\x44\x9c\x3c\x8c\xb0\x7c\xfe\x12\x7b\x7b\x1b\xf6\x4a\xce\x56\x55\x73\x63\xd3\xbd\x4a\x3e\x58\x1f\x7b\xac\x98\xc8\xd9\x81\xde\x3d\x58\xfe\xd6\x2b\x3f\x2b\x0a\x7b\x6f\x06\x15\xb1\xa9\x8c\xaf\x9c\x45\xf2\xc3\x31\x9d\x07\xf4\xb2\x10\x7f\x0f\x6a\xc9\x1f\xe6\xe1\x96\x12\xee\xf1\xed\x92\x97\xa2\x28\x08\x79\x5b\xee\xf0\xc0\x67\x61\xe4\x58\x6a\x31\x95\x60\x9f\x25\x51\x72\x8a\xf6\xb4\x60\x39\x08\x8a\x21\xb3\xdb\x67\x17\x83\x9f\x09\x0e\x6f\x62\x40\x1b\x51\x11\x58\xe5\xae\xcd\xf1\xc4\x15\x87\x48\x91\x5f\x58\x7d\x49\x58\xb7\x2d\xc5\x4d\xe1\x6f\xad\xdc\xbb\x54\xb6\xfb\x17\xb2\x57\x5b\x08\x82\x29\x54\xcb\xb6\x0f\xba\x98\xa7\xee\x5b\x6e\xdd\x67\x9f\x25\x7d\xf5\xd9\xed\x7b\xb8\x6b\xea\xaa\x76\xbf\x8a\x95\x14\x16\xde\x72\x7a\x9d\xf6\xf5\x99\x94\x6e\xc5\x2a\xb4\xc4\xe1\x9f\x8a\x88\x15\x9c\xb0\xd1\x31\xa1\xb2\x88\x7e\xf2\x28\x10\x3b\x5a\x59\x3a\x3c\xb5\x8a\x9a\x2b\x4f\xee\x8c\xf2\x9d\xf5\xbf\xaa\xbc\xd6\x67\x34\x90\x8c\xb4\x9b\xf8\x69\x88\x0f\x30\xca\x6f\xa6\x2b\x75\xda\xf6\xc0\x93\xb5\x08\xbb\x7a\x47\x42\xe7\x80\xf8\x5c\x1d\xde\x08\xfd\x08\x3e\xdb\x61\x67\xd7\xc1\x81\xd9\x6e\x46\x93\xd5\x27\xb3\x7b\xaf\x92\x5a\x54\xd2\xa6\x64\xd2\x96\x74\xdd\x83\x8a\x8c\x40\x39\x6f\xc0\x16\x5e\xc2\x3f\x5d\x6b\x79\x10\x6f\x4b\xbd\x2d\x6b\xc2\xb9\xc4\x4e\x46\xa5\x50\x33\xda\xc4\xa6\x4f\xc7\x56\xee\x99\x18\x5f\xdd\x23\x6a\x12\x61\x9e\x89\x25\x7d\x10\xe0\xde\x4b\x14\x98\xc5\xdd\x68\x48\x5c\x19\x80\xe9\xf6\xb6\xf2\x96\xe3\xb3\x6e\x53\x15\x23\x15\x9a\x0b\x2a\x1d\xb1\xdd\x7a\xc3\xb8\x44\xd6\xa4\xf4\x79\xda\x6d\xdb\x3b\xc6\x57\x35\xd6\x60\x3c\xbd\xc6\x6b\x19\x2b\x0d\x82\x4c\x33\x36\x41\xf9\x10\x18\xa1\x4d\xdc\xcb\x40\x64\x51\x70\x1f\x7f\x68\xeb\xd6\x28\x33\x1f\x2a\x4c\x86\xfa\x6a\xcb\x94\x29\x49\x56\xe1\xb5\x9e\xf2\xf8\x6a\x08\x2c\xe1\xbc\x31\x38\xea\xce\x8d\x2c\xb2\x6b\xdc\x87\xa9\x55\xdd\x14\x10\xd3\xfd\x6e\xa0\x88\x07\x4e\x82\x74\x28\x2b\x7d\x25\x59\x38\xf9\xac\xec\x74\x15\xbd\x22\x89\xc1\x3e\x32\xa3\x7c\x1e\x4f\xe6\x73\x38\x2e\xda\xd9\x60\xb3\x5c\x72\x34\xd8\x5c\xab\xe3\xba\xbe\x72\x35\xfe\x84\xeb\xe2\x85\x2d\x49\xb4\xcf\x90\x43\x7e\xeb\x86\xbe\x21\xff\xff\x09\xbf\x45\x36\xdc\x9f\x73\x5d\x24\x11\xd7\x7b\xd1\x94\xff\xb4\x03\xa3\xc9\x57\xf9\x30\xaa\x98\xcf\xe3\xc6\x94\x8b\xda\xc9\x93\x91\xd9\x6a\xcc\x17\x36\xc8\x6c\xb4\xac\xde\xc6\x42\x91\x71\xf4\x62\xdf\x95\x59\xa1\xe0\xa5\x32\xbc\xd2\x4d\x2d\xdb\x17\x2c\x89\xbe\xe5\x0d\xef\xae\xf3\x4a\xab\x73\x52\x53\x0e\xbc\xe0\xf4\xaa\x1f\x74\xaf\xe4\xfe\x6f\xbe\x23\x8f\xcf\xa8\xe0\x23\xee\x16\x3e\x8d\xf0\x09\x90\xe8\x1e\x3b\xc3\xdb\x3f\xae\x47\xab\xe4\x22\xbf\xdb\xda\x39\x0b\x40\xc5\xd5\x1f\x93\x51\xa4\xe3\xc5\x0a\x8c\xb2\x58\x61\xa0\x3d\x18\x13\xe4\xc6\xf4\xd5\xe2\x06\x3f\x25\xac\x40\x4a\xc9\x1c\xc7\x0d\x4d\x42\xdd\xd3\x76\x0e\x93\x23\x0c\x7f\x2a\xae\xa3\x06\xc1\x30\x1a\x1c\x06\xb6\xea\x23\xa9\xf6\x24\xeb\x36\x26\x65\x68\xde\x65\xe4\x96\x93\x16\xee\x9b\x99\x85\x46\x54\x64\xd8\x3b\x48\xc0\x0a\x31\x40\x48\x04\x79\x93\x25\xb5\x96\xe6\x53\x26\x66\x07\x12\xc9\xda\x96\x4a\x95\x6a\xcb\x6b\x05\xbd\x79\xa4\xf3\x28\xa9\x9c\x06\x3d\x1f\x51\xe8\xbb\x04\x37\x9e\xc6\xa1\xdb\xeb\x0c\x85\x66\x64\x5a\x4e\x34\x54\xab\x34\xf6\xb8\x3e\x78\x11\x04\x8a\x35\xba\x59\xd0\x65\x30\x8b\x1b\x71\xed\xfb\x4f\x23\x14\x59\xdf\xf0\xda\x5e\x2e\x96\x4a\x90\x84\x8f\x17\xa4\xc8\x02\x5a\x11\x71\x82\x08\x5c\x88\xde\x41\x4a\xd4\xa8\x9a\x62\x2f\xf4\x41\xfa\xb5\xf2\x21\x6e\xe5\x35\x7a\xa6\xea\xba\xe1\xfc\x9e\x49\xdf\xdf\xf7\x2a\x0b\x08\x6a\x88\xf8\x7a\xc7\xb0\xde\x15\xa8\x84\x60\xaf\x0b\x62\x84\xb9\xab\xcf\xaa\xae\x45\x35\xad\xd7\x4f\xd6\xe0\x3f\xa3\x23\x05\x15\xe0\xd6\x28\xaa\x48\x8d\x4e\xb0\x11\x63\xc0\x87\xa2\x74\x25\x58\x77\x14\xc9\x87\x94\x79\x49\x2f\x6b\x9a\xa0\x57\x6e\x02\x29\x6e\xbb\x09\x6c\x15\xd4\x86\x6c\x25\x52\x4d\x49\xac\x78\x93\xc1\x0a\x54\x88\xfa\xb3\x34\x6d\xa6\x50\x7d\xc6\x7d\x85\x40\xba\xa8\x16\xad\x14\xd4\x0a\xf3\x9c\xae\xe4\x43\xb4\x02\x4e\x4f\x57\xbc\xdd\x80\x88\xae\xcc\x57\xbd\xf4\x20\x4a\xc5\xf5\x41\x74\x2d\xe4\x94\x0e\xdd\xfd\xeb\x03\x44\x3b\xa0\xe1\xe9\x30\x15\xe9\x21\x41\x92\x39\x43\xc0\x9d\x02\x76\x36\xe5\xcd\xf0\x96\x82\x5e\x80\x95\x97\xfa\xb1\x28\x33\x0e\xee\x23\x89\x16\x43\x21\xd0\x43\x2e\xde\x47\xd7\x7d\xbe\x43\x42\xd5\xff\xf0\x1a\x1c\x30\xd4\xbf\xeb\x28\x92\x0d\xc2\x82\xf2\x88\xf8\xed\xed\xfb\x53\xd5\xa1\xe8\xac\x87\xb6\x3e\xfa\x04\x3f\x0b\x05\xa3\x0f\x44\x7c\x66\x3f\x87\xbf\x93\x3c\x7a\x8f\xbb\xda\x17\xf8\x29\xd6\xed\xa2\x59\x53\x96\xd3\xca\xc4\x7b\xc9\xcd\x82\x79\x79\x0f\xbc\xf8\xbd\x45\xfb\x03\x7e\xf3\xc4\xac\xcd\xa8\x94\x85\x6b\xd2\x04\xf2\x92\x95\x56\xf4\x9e\xbe\x52\x54\x72\x9d\xd9\x5c\x67\x77\x72\x9d\x01\xd9\x9c\xb8\xce\x0b\x5c\xe7\x4d\x55\x5c\xbe\x85\xeb\x90\xaa\x2e\xb9\xb6\xb3\xc8\x6a\x68\xae\x33\xe4\x7a\xe3\x7c\xa0\xa3\x4d\x4c\xf6\x2a\x02\xeb\x73\x9c\xde\xde\xda\x47\x62\x0a\xcb\xa5\x6a\xaa\xcf\xf3\xfb\x40\xe8\x19\x3e\x3d\x4c\x74\x28\x09\xf9\xae\xc3\x49\x08\x45\x7d\x7f\xbf\x74\xae\x43\xad\x63\x92\xdd\x18\xe0\x8b\xca\x9e\x1d\x48\x32\xec\x3e\x4a\xa3\xe1\x29\x5a\x7d\xd5\x05\xbb\xda\x1c\x2a\x5a\x98\x90\xd3\xfd\xcc\x8c\x1a\x95\x4f\xee\xc8\x94\xbb\x66\xb7\x82\x84\x1f\x99\xfb\x50\x6c\x87\x52\xcd\xf3\x28\xd8\x10\xbc\x14\xbc\x50\x90\xf1\x4f\xa3\xc9\x0f\x98\xc8\x1c\x61\x93\x51\xf8\x4e\x33\x8a\x09\xea\x7e\xf4\xfd\x2e\x6c\x9a\x98\x20\xc7\xa6\x37\x4b\x4a\xf7\x3f\x5b\x20\x33\xfa\xd6\x6a\x17\xd2\xa9\x0a\x16\x65\xad\x2f\x4d\xbe\x2d\x4f\xf2\x96\x91\xba\xcf\x9d\x69\xf0\xf7\x08\xe3\x46\x98\x63\x64\xe4\xb9\x2e\xfb\x75\x72\x60\x9a\xdd\xaa\x64\x9e\xab\x95\xbe\x9b\x2f\xfb\x83\xa5\xa0\xff\x86\xdd\xa5\x6f\x9f\x0e\x99\x3c\x5b\x50\xfc\xdc\xb5\x1b\xc2\xa8\x9e\x76\xe6\x63\x6c\xa0\xbd\x52\x70\x6d\xbd\x22\x41\x72\xe1\x90\x94\x50\xb8\x59\xa9\x98\x28\x8f\x5c\x87\xce\xf2\x6b\x6b\xe8\xac\x9c\x79\x74\xe7\xa6\xda\x01\xe0\xf7\x2d\xe9\x76\x4d\x4d\x09\x01\x37\x5b\x0e\x32\x41\xff\x91\x62\xa9\xda\xd1\xe2\x81\x7a\x51\x8d\x68\x5a\x93\xd5\x73\xb7\x6d\x0f\x84\x9a\xf5\x8d\x2e\x77\xe5\x16\x0f\xad\x29\x04\xdd\x93\x3b\x3f\x4c\x21\x52\x7b\x77\xdc\x4b\x4f\xb8\x99\x30\x7d\x8f\x2e\xc3\xd4\xca\x48\xe9\x56\x05\x48\xff\x77\xdc\xb7\x81\xa8\x96\x62\x13\xc0\x90\xa2\xae\xb5\x13\x25\xc2\xcc\x9a\x8d\x86\xa1\x85\xa0\xbb\x56\x6b\x51\xf5\x0b\x73\x1c\x8e\x48\x10\xc1\xe4\x18\x86\xd7\xa9\x99\x32\xff\xfb\x6f\xe2\x8f\xff\x25\xed\x6b\x29\x9e\xbb\x52\xad\xce\x29\x7e\x30\xfe\xe3\x7f\x45\xe0\x5b\x86\xf4\xfc\xef\xbf\xa1\xbf\x67\xcd\x94\x38\xd4\x02\x98\xf4\xde\x64\xf1\x20\x8b\x3a\x07\x4e\x1c\x04\x18\x83\xf3\x0a\xd8\x4a\x45\x20\xea\xc9\xfb\xc2\x56\xb7\xb7\xab\x53\x73\x3d\xee\xbf\xff\xd6\xfc\xe3\x7f\x1d\x64\x07\xc6\xde\xb4\x24\x91\xe9\xe2\xad\xb7\xf2\x0f\x57\xe0\xd5\x08\x78\xfe\xa1\x50\x2b\xe6\x47\xd6\x59\xd7\xbe\xcd\x95\x97\x07\x75\x26\x47\xd6\x31\x9d\x5e\x31\xb0\x98\x6e\x14\x2a\x2e\xce\x96\x9e\xe2\x1b\x8b\x4c\x56\x37\xc9\xfc\x3b\xb0\x34\xa6\xc8\x59\x39\x38\x99\xf5\x19\x5b\x5e\xd2\x8f\xdb\xd9\xde\x35\x9b\x7e\xda\x8c\x1c\x59\x68\x8f\xe8\x70\x26\x6f\x46\xa3\x61\x06\x0f\x59\xc8\x9d\xc4\xf4\x7e\x36\x07\x37\xd6\x04\x5f\xbd\x98\x2e\x16\x2b\x27\xaa\xf2\x38\x4e\xa6\x36\x80\xce\x68\x50\xd6\x36\xc7\xb3\xae\x8e\x82\xcd\xc4\x19\xc3\x1b\x04\x38\x64\xf0\x5f\x37\x5a\x2b\xce\x2d\x0a\x0c\x71\x9e\x6f\x51\xae\xeb\x57\xc9\xdb\x79\x72\x91\x8c\xa1\xff\x47\x1f\xd3\x6e\x28\xde\xc2\xbf\x25\xfc\x5b\x51\x60\xd8\xd0\x66\x66\x51\x79\x40\x5e\x53\x85\xd6\x2d\x30\x0e\x53\x1e\x90\x14\xf8\xc0\xcc\x1f\xe8\x61\x00\x33\x11\x81\xe3\x5b\x30\x2c\x71\xb5\x78\xcb\x2c\x61\xdb\x5f\xd0\x93\xd7\x68\x07\x71\x83\x56\xb0\x35\xce\x0f\x20\xcd\x1f\xe2\xb7\xd0\x39\xdf\xc7\x4a\x9a\xe5\x6e\xd9\xa2\x96\x42\xc9\x7a\x2d\x30\xcd\x2c\xe3\x3a\x34\x22\x42\x78\x56\x6d\x97\x30\xeb\xd8\x52\x5b\x48\x2e\xd7\x56\x84\xdc\x04\xbe\xd1\x06\xea\x01\x95\x84\x4c\x39\x20\x26\xd0\xd5\xc5\x1c\x2f\x21\x6a\x1f\xdb\x99\xf9\x46\xff\xe9\x36\x46\x24\x8a\x07\x73\xfc\x8f\xeb\x6e\x83\x76\x75\x35\xc4\xa8\x3b\x18\x0c\xf1\x3c\x8c\xa1\x35\x06\x65\x78\xbb\x58\xe5\x61\xb0\x8d\x1d\x45\xce\x57\x3b\xac\x8f\xce\x4c\xc6\x2a\x72\x9d\x4f\x25\xd7\xa9\x21\x77\xfe\xc9\xf4\xce\xab\x09\x8e\x3f\x99\xe0\xd8\xe8\xa8\x25\xbd\x41\x27\x38\xe9\x74\x4e\xa0\xa1\x4f\x1e\x3d\x7c\x08\x2f\xa2\xf3\xa4\xf3\xf0\xf8\xf1\x63\x11\x3e\x0c\x8e\x1e\x3e\xea\x3c\x16\x4f\x1e\x75\xe0\xfd\x58\x3c\x09\x9f\x3c\x0a\x8f\x40\xdd\x1f\x3e\x09\x1e\x1d\x1d\x05\xe2\xf1\xd1\xf1\xf1\xa3\x10\x00\x9d\xa3\x93\x27\xe1\xe3\x13\x11\x1e\x1f\x9f\x84\x8f\x8e\x87\x6a\xe3\xff\xea\xed\x39\x47\x8c\x26\xad\xb7\x84\x6c\x4a\x0d\x1f\x1e\x3f\x7c\xf8\xd0\x2a\x1e\x1e\x8e\x9f\x9c\x3c\xea\x18\x3e\x82\x00\x68\x3f\xb1\x18\x42\x94\xe0\xd1\xc3\x8e\xe6\x2c\xec\x3c\x39\x3e\x7e\x7c\x12\x1a\x16\x3b\x00\x0e\x91\xac\xe2\x35\x3c\xe9\x1c\x3d\x0c\x9f\x3c\x34\x4c\x1f\x05\x4f\x3a\x8f\x83\x47\x16\xf7\xc0\xc7\xe3\xe3\xc7\x4f\x54\x35\xa0\xe4\x87\x61\x00\x45\xdc\x59\x9f\xf3\x68\x70\x84\x2c\x3d\x7a\x24\x8e\x8f\x9e\x40\xe6\x27\xe2\x51\xf0\x30\x78\x04\xbf\x61\xd0\x39\x79\xd8\x01\xea\x27\xc7\xe1\x93\xe0\xc9\x23\xf1\xa4\x13\x9e\x1c\x03\x1f\x61\xf8\x24\x3c\x0e\x43\x28\x05\x18\x3a\x39\x39\x3a\x81\x94\xa0\xf3\xe4\xc9\x11\x30\xf4\xb0\x73\xf2\x38\x40\x3e\x80\x2d\xa8\x06\x3e\x40\xde\xa3\x8e\x78\x7c\x72\x72\x12\x9e\x40\x0a\x64\x3a\x81\xaa\xa1\x4c\x1e\x3e\x39\x79\x78\x84\x28\x8f\x1e\x1d\x81\xd8\xa0\x4a\x8f\x4e\x1e\x03\x20\x78\x7c\xf4\xf0\xa8\xf3\x10\xe9\x77\x1e\x1f\x23\x39\x10\xe7\x93\x93\xe3\x87\x77\x56\x67\x0c\xd5\xe9\x3c\x7c\x08\x72\x83\x6a\xa0\xb0\x02\x14\x45\x70\x72\xd2\x41\x6e\x03\x48\x43\x11\x03\xc9\xe3\xe0\x18\x0a\x42\x96\x1e\x9d\x3c\xc4\x87\xce\x71\xe7\xf8\x09\x3e\x1c\x3d\x84\x46\x13\x47\x9d\xe3\xa3\xe0\xd1\x89\x78\x74\x72\xdc\x79\xf8\x08\xa9\x9c\x04\x8f\x82\x00\xf2\x1c\x85\x01\x10\x78\x08\x29\x4f\x80\x3b\x34\xa1\x47\x4f\x80\x6a\x87\x5b\x03\xeb\x11\x3e\x3c\x02\xb9\x3d\x09\x51\x70\x47\x20\x5b\xf1\xe4\xf1\xc9\x63\x28\x11\xc5\x73\xf4\xf0\x31\x88\x3a\x7c\xd8\x79\xf4\x28\x78\x1c\x56\xd4\xc7\x58\x3e\x98\x3d\xc0\x38\x54\x19\xc0\x5b\x5b\x1d\x85\x83\x66\x67\xe0\xd8\x3b\x4e\xf9\xb0\x43\xee\x0f\xb1\x17\x54\x59\x4c\xbd\xdf\x78\x1b\x09\x8d\xe4\x0d\xda\xc7\xc3\x32\x11\x35\x98\x6e\xa3\xa1\x70\x3c\x3b\xf7\xfb\xb7\x32\x98\x39\x3c\xc1\xa4\x78\x5c\xb9\xb2\x0c\x70\xe5\xae\x04\x11\x6d\x92\x7d\x49\xdf\xf3\xaa\x8f\x13\x8b\x55\x18\x81\xd7\x77\x07\xd2\x28\x88\xee\x3a\xe2\xdc\x44\xd6\xdf\xbf\x3d\x83\xe2\x79\xd3\x3c\x46\xa0\xde\x7e\x94\xb9\x9c\x65\x32\x8a\xbc\x51\x70\x3a\x0a\x71\xdf\xd8\x28\x1a\x05\x58\xf2\x28\x44\x52\x93\x11\xb0\x11\xb6\x46\xe0\x25\x4e\x2e\xa2\x3f\xfe\x79\x3a\x19\xf5\x1b\x41\xa3\xdb\x08\x1b\x62\x1c\xb0\xf7\x33\x5e\xa4\x1e\x22\xa4\xf2\x3d\x85\xd1\x1d\xdf\xc7\xa1\x95\x0e\x2e\x67\x1a\x5a\xe9\xa1\x75\x7a\xf0\x34\x32\x2c\xbd\x18\x65\xfd\x55\xd0\x6f\xbc\x08\x44\xa3\xb9\x0a\x9b\x8d\x2f\xf9\x47\xbe\x3d\x08\x1e\x80\x9e\x3f\xc0\xc4\xd6\xd6\x54\x7c\x65\x1a\x01\x63\x05\x8c\x15\x48\xac\x40\xd2\xd8\x96\x8a\xaf\xbf\x42\x5d\x3f\x07\x2f\x40\x87\xaa\x85\x6f\x07\x63\x59\x5c\x78\x90\x06\x15\xf9\x1a\xcd\xc9\x05\xbc\x87\x0f\x18\x39\xd4\xc8\xf0\xf4\x1c\xd9\xd2\xb0\x80\x60\xa5\x0a\x48\x02\xf8\x80\xc8\x81\x46\x56\x15\xfa\xd3\x5c\x04\x22\xf8\xb5\xc1\x11\xd1\x8c\xae\x5b\xcd\xf8\xbd\x81\x0a\x4b\xd1\x2d\x8c\x1f\x0d\xd4\x3e\xe7\x6d\x10\x5e\x69\xa0\x39\xfa\x6d\x52\xbf\x91\xa0\x9e\x9e\x74\x8d\xdb\x36\x2b\xe6\xc4\xf0\x96\x79\x59\x81\x77\xb5\xe5\xfd\x3d\x9d\x56\xf7\xbb\x56\xf2\x06\x21\x6d\xbb\x26\x3b\x15\x50\xa8\x7a\xb1\x00\x2b\x99\x0b\xa8\x3a\x49\xbe\x8d\xbe\x2b\xb8\x22\x79\xfb\x54\x39\x52\x2f\x9f\x2a\xdf\x46\xdb\x16\x7a\x91\xb2\x39\x61\x8e\x74\xc7\x90\x6f\xb5\x48\x1c\xeb\x4c\x96\x30\xf2\xee\xb4\x84\xcd\x3b\xcd\x20\x7e\xba\x06\x03\x75\xa7\x25\xdc\x6e\xf5\x80\x4a\xc9\xf0\x49\xdd\x19\x18\x2b\xe5\x1f\xac\x84\xb1\x51\xf0\x36\xa4\x1a\x9a\x79\x89\x9d\x3d\x6a\x5d\x8e\xa6\x17\x7f\xfc\x53\x38\xd6\x2b\xfa\xe3\x7f\xb7\x7e\xff\xad\x27\x87\x0b\x9c\x07\xd4\x0c\x38\x90\x03\x53\x9d\x48\x61\x4e\x41\x98\xfa\xbd\x73\x99\x54\xf9\x52\x23\x85\xf7\x9c\x20\xa2\xc1\x29\xad\xf1\x74\x91\xc6\x93\x46\x19\xe3\x19\x25\x40\x87\x8b\x97\x76\xe2\x2b\x78\x17\x0d\x84\xb6\xce\x63\x98\x69\xc5\x8d\x62\xea\x57\x04\x96\x38\x23\xdc\x0b\x50\x42\xa1\x1d\x02\xe2\x7c\x94\x26\xa9\x9d\xf4\x15\x02\x44\x83\xe0\xad\xc5\x32\x9e\x37\x4a\xa9\x3f\x02\x54\x61\x54\xb0\x4e\x38\x92\x73\xbe\xdf\xc3\x49\x25\x88\x18\x8f\xd8\x6f\xb7\x93\x9e\x49\x98\x68\xa8\xd4\x32\x03\x0a\x87\x79\xd0\x78\x15\x6c\x28\x4c\xc9\xc9\x6c\x31\x5f\xac\x61\x8a\x66\xa3\xbc\x90\x30\x3a\xb6\x56\xdd\x88\xe5\x50\x1b\x57\x71\xce\x67\xa0\xb0\xd7\x60\x24\x14\x00\x44\x1a\xd8\x66\x4e\xa2\x43\xc9\xd2\xff\x29\x6f\xfe\x02\x04\xb5\x7f\xa4\x28\xaa\xaf\xe2\x0f\x49\x0c\x1e\x06\x38\x31\x9d\x43\xf0\x13\xe1\x5f\x30\x14\xd5\x58\x1d\xc4\x42\x8c\xce\x36\xac\x23\xc6\x3a\x91\xf4\x4e\x86\xb6\x7e\xd3\xae\x28\x77\x4a\x4a\xbb\x5a\x50\xf5\x0b\x8a\xae\x9e\x5f\x52\x0e\xed\x02\x68\x2a\x60\x26\xe9\x39\x53\x1b\x4f\xe5\x1b\xfd\xd0\xc5\xb5\x9c\x9e\x3b\xe9\x39\xa5\x6f\x44\xa5\xca\xaa\x43\xc0\x51\xa5\xb6\x56\x01\xeb\x72\xc8\x2e\xa0\x1d\xc1\x78\x54\xdf\xb3\x31\xd5\xed\xd9\x76\xbe\x1a\x89\x11\xc9\x02\x81\x1a\x89\x69\x2a\x20\x31\x7a\xd6\x12\x93\x6f\xf4\x63\x8f\x6f\x0c\x0f\x5c\xb4\x80\xf1\xec\x61\x8a\x13\x42\x17\x2f\x64\x3c\x6e\x01\x7a\xcc\x1d\x84\x9c\xd3\xad\xe1\x88\xc1\x6e\x71\xb9\x2c\x4e\x8f\x2c\x0c\x75\xcb\xca\xb9\x2c\x2d\xaf\xfa\x1b\x7a\x28\xc5\xec\x44\x4b\xf5\xa5\x24\x6c\xfb\x65\xf8\x74\xda\x79\xb6\x2e\xa4\xc9\x78\xea\x56\x18\x27\xf4\x89\xd2\xf6\x32\x68\xa2\x23\x9f\xb6\x57\x02\x5e\x42\xf8\x03\xce\x2f\xfc\x09\xfc\xa6\x17\xc3\x1c\x65\x9a\x7a\x90\xd9\xef\x8f\xaf\x57\xef\x63\x1b\x8d\x1e\x02\xbf\x5b\x48\x58\xc3\xc3\x1a\x13\x88\xac\x7c\x43\x30\x90\x5d\x13\x59\xce\xa0\x53\x34\x25\x1f\x3d\x35\xeb\x6b\xbb\x62\x3f\x8d\xa7\x17\xe2\x42\x98\x5a\xcb\x84\xe8\x42\xc6\xf2\xc0\x74\xde\x6d\x27\xb7\xd9\x5a\x70\x85\x8c\xc9\xee\xe4\xa2\x12\x67\xfb\xe4\x62\xb7\x2c\x52\xbc\x1f\x57\xdd\x15\x14\x89\xf7\xdb\x8c\x42\xbc\xd9\x66\x19\x74\x07\xab\x03\x67\xfe\x20\x5f\xe5\xf4\x61\x28\x96\xa1\x8b\x12\xba\x28\xa1\x3f\xb4\x3e\x32\xca\xd6\xb1\xc3\x1d\x80\xbe\x06\x51\x74\x0e\x7f\xf7\xf7\xe1\x39\xa4\xe7\x70\xe3\x4c\xd9\x56\x62\x29\x74\x14\x01\xf2\x81\xd9\xcd\x95\x1e\xf0\x83\xa6\x37\x3a\xfd\xe3\x9f\xbe\xf4\x83\x97\x26\x33\xb7\xdb\x2a\x80\x8a\xc0\x64\x0e\x78\xd5\x44\xfe\x6f\xf0\xf9\x31\xef\x32\xdc\x58\x57\xe9\xa0\x5c\xec\x6b\x6b\x10\x20\xb5\x70\xe5\xb8\xc6\xa4\xd5\x9f\xcf\x2f\xe6\xcb\x3d\x8a\xfb\x56\xb7\xf9\x7d\xab\x2a\x77\x55\x5e\x38\xa2\xf6\xa8\xaa\x3b\x4d\xae\x9d\xeb\x3e\xb6\x7a\xaa\x5a\x0c\x65\xaa\xf2\x7e\x1f\x49\x55\xca\x67\x27\xaa\x46\x96\x65\xaa\xf2\x5e\x20\xc5\xeb\xe7\xf2\xac\x15\xcf\x96\x6f\xcd\x25\x7c\x16\xef\x5a\x52\x37\xfe\xb5\xb9\x62\x05\x8d\x21\x8c\x01\x6f\x17\x85\xe5\x5f\xad\x92\x2a\xd1\xda\x9c\x0b\x7d\x9c\xa3\x36\x17\x76\xe2\x2e\x8f\x22\x19\xbc\xb9\x90\x30\x8b\xbc\x65\xd0\xce\x9b\xcb\xa3\x76\x8e\x8e\xf7\x32\x1a\x80\x82\x7f\xcc\xba\x00\xcd\x44\xde\x9d\x6d\xe8\xe5\x48\xbd\x2c\x8f\xcc\x16\xeb\x68\x49\xde\xe9\x72\xb5\xc0\x1b\x40\x81\x23\x5f\xa0\x55\x5d\x0e\x82\x61\xb3\xf1\x0c\x1f\x42\x78\xc0\x9e\x31\xe8\xa8\x87\xa3\xe1\x8e\x9d\xc4\x50\x55\xca\xae\xaa\xfb\x93\x4e\xd1\x2b\x0a\x32\xa5\xa4\xa0\xdb\x43\x7b\x15\x15\x14\x6f\xe2\x96\x94\x8c\x8e\x6a\xda\x45\x35\xdd\x1a\x6b\xbe\xa4\xa6\x0e\x6d\xa5\xa9\x9a\xb6\x55\xd9\x9d\xe8\x5b\xf8\x99\x45\xd7\x80\x0d\xed\x92\x2a\xd5\xf8\x22\x5a\xd3\x0a\xe8\x9e\xbf\x53\x53\x58\x48\x67\x51\x45\xbd\x4a\x4d\x75\xcf\x2a\x5b\xe4\xbd\x02\x13\xec\x29\x19\x56\x1c\xe9\xf8\x77\x48\x25\xcd\x67\xe7\x8b\xea\xee\xc5\x49\x76\x68\x33\x55\x32\xa7\x70\x74\x24\xba\xe3\xd1\xed\x54\x1c\x2f\xc9\x20\x3e\xa3\x6b\x5b\x7d\xda\x37\x51\x44\xdd\xa8\xed\x00\x91\x93\xe5\x67\x80\xf0\x4e\x10\x07\x8c\x5b\x66\xf5\x37\x47\x82\xb4\x29\xef\x6e\x4a\x29\x4b\xb1\x54\x92\x69\x80\x42\x42\xd2\x46\xbe\xdd\x77\x23\x48\x35\x45\xde\x13\xc2\x6f\xce\x24\x57\xca\x4e\x4f\x6d\xe5\xa5\xb7\x15\x02\x13\xe3\xd5\x22\xb5\x4e\x02\xf1\xc5\x06\xbc\xce\x60\xbe\x2c\x23\xf4\xf0\xd8\x6c\x65\x47\x0b\xd4\x3a\x3a\xe0\x61\xbd\x05\x3f\xdf\xf1\xcf\x2f\x0a\xfc\x1d\x8d\xf6\xbf\xe8\x44\x02\xfe\x42\x40\x9d\xa4\x10\x5b\x76\x12\x67\x07\x27\x0d\xb5\x08\x26\x83\x93\x4a\xd6\xf2\x22\x6f\x18\x72\xc6\x69\xda\xd1\xfc\x08\xdc\x3d\xb1\xca\xa2\x55\x5e\x91\xa4\xeb\xc1\xeb\x8c\x39\x2f\x01\x66\xb4\xbc\x47\x8b\x8b\x39\x99\xd2\x16\x83\x90\x1d\xda\xec\x13\xef\x22\xa8\x82\x98\x8c\x90\x9e\x2b\x47\x08\x5f\x1e\xd8\x5e\x91\x46\x93\x75\x6f\xac\x57\x09\x4d\x0b\x5a\x93\xc5\x8d\x7d\xb5\xaf\x55\x6a\x56\x94\x81\xab\xc4\x00\x3e\x82\xfa\xe7\xd1\x2a\x3b\x28\xa7\x58\x2c\xaa\xea\x2a\x01\x48\x79\xe8\xda\xcb\x77\x97\xad\xeb\xe5\x7f\x3c\x53\x2d\x97\xab\x02\x53\x92\x27\x0e\x65\x63\x8c\x0c\xf6\xe9\x34\x2a\x98\x10\x98\xd5\xa7\x9e\xef\xdc\xd4\x4a\x1b\x23\xbf\x9f\x5f\xc6\xab\x64\xfd\xfd\x44\x54\x81\x45\x99\x43\xab\x76\x47\xbe\x28\xab\x15\xa7\xaf\x47\x73\xef\x28\xc0\x0a\xd2\x28\x30\xe7\xb3\x61\x86\xfe\x4f\xea\xda\x5a\xdc\x30\xee\x96\x3c\xe1\xe3\x3b\x15\xb8\x64\xd3\xa8\x62\xf1\x94\x0d\xad\x9b\x54\x53\x44\x3b\x9e\x2d\xd7\x79\x4d\x3e\x4a\xab\xcb\x88\xdb\xad\x6a\xf2\x71\x9c\xee\xea\x6c\xda\x9e\x96\xb3\x61\x92\xbe\xdb\xb8\x10\xc3\x59\xa3\xd7\x5b\xc2\xea\xa6\xeb\xeb\x9c\x16\x5d\x0f\x8f\x71\x48\x70\xd7\x66\xe6\x25\xee\x41\xb4\xf1\x36\x2e\x43\xe6\x4a\xe1\xa8\xba\x7e\xb5\xd5\xa6\x12\x8a\xf5\xc1\x08\x0e\xf6\x7c\x12\x27\x75\x7a\xdb\x9d\x48\x26\x11\x5d\x1e\x90\x4c\xac\xdb\x6f\x07\xc3\x9e\xca\xeb\x88\xf1\x4c\x41\x0d\xe9\x9e\xda\x41\x23\x2f\x6b\x21\x62\xee\x35\x2d\x1f\x35\x61\x79\x49\xa8\x9a\xd8\x0e\x86\x26\x3b\x43\x30\x37\x86\x92\x93\x9b\x82\x09\x58\xdc\xcf\x4c\x3a\x41\x29\x78\x38\x03\xf7\x66\x72\x7d\x22\xc5\x13\x8f\xbc\x66\x67\xe1\xd9\x19\x86\x2b\x3a\x3b\xc3\x2b\x7e\x7d\xbf\xef\x35\x14\xa0\x91\x70\xbc\x7a\x43\x43\xe3\x46\xce\x5b\xa1\xef\xe0\x31\x2f\x95\x03\x88\x82\xe8\x24\xb6\x41\x39\x3b\x1b\x24\x93\xa1\xb9\xb0\x57\xd7\x9c\xb6\x18\x82\x66\x38\x70\x3a\x84\x64\x9d\xc3\xb4\x94\x43\xcb\x0e\x8a\xb1\xae\x9e\xae\x6c\xf9\x2f\xa7\xd3\x5d\x1b\x3f\x95\x91\x20\x64\x25\x0c\xcd\x4f\x54\x88\x2f\x79\xc5\x60\x67\x9d\xb8\x77\xb3\xcb\x10\xe9\xa6\xe1\x3f\x5a\x3d\xb8\x46\xfc\xba\xb2\xbb\xa9\x86\xd8\x45\x4f\xaf\x90\xc5\x45\xa4\x28\x1b\x2e\xaf\x9e\x2e\x40\x39\xb5\x2e\xca\xf4\xc1\x15\xaa\x68\xbd\xf6\x5c\xa1\xf6\x98\xb4\x3a\x85\xf9\x33\xba\x71\x91\x4c\x9d\x33\x9f\xfc\x5e\x50\x0b\xfe\xab\x74\xc2\xb4\xba\xd9\x84\xba\xa7\xb6\x9f\x72\x7e\xe8\x33\x92\xb0\xa3\x0f\x0c\x53\x65\xd8\x9a\x10\x14\x14\x41\xed\xb4\xfb\x24\xeb\x10\x14\xb5\x44\x85\xa9\x2b\x59\x07\xe6\xa4\xbe\xdd\x7d\x73\x87\xb5\x75\xc5\xc7\x5d\xf2\x96\x3d\xa4\x5e\xe8\xeb\x9b\xd8\xbe\x0e\x61\x3e\x9a\x41\x27\x43\x98\x8c\xf7\xa7\x3b\x59\xaf\x66\xa4\x79\xda\xe9\x13\x06\xb2\xe3\xf9\x15\xca\xcd\x45\xd0\xcc\x04\xa9\xfb\xce\x10\x73\x16\xe3\x77\x08\xbe\x98\x87\x76\x1c\x13\x72\xe5\xa9\xce\x5a\xca\xab\x98\x2e\x22\x25\xe2\xd5\x47\x38\x6b\xb3\xa6\x92\x29\x59\xe5\x4d\xbd\x9c\x46\xeb\xf5\xca\x15\xd3\x0f\xaf\xd4\x27\x13\xb3\x9c\x07\x48\xb4\xe9\x1b\x3a\x3d\x5d\x91\x4f\x9c\x7d\x09\xd0\xe4\x1c\x83\x35\x33\x8b\x25\xf4\x1f\x5e\xd5\x64\x80\x04\xcc\xd2\x4e\x97\xa3\x31\x28\x04\x3e\x4e\x17\xa0\x21\x05\x1a\x3f\x23\xf3\x9e\x59\x75\x64\x49\x9e\xf7\x15\xfd\xae\x77\xde\x8c\x1a\x8d\xe2\x26\xdb\x44\x8c\xb8\x71\xa1\x69\x0a\x3c\xea\xc6\xde\x03\x3a\xd0\x83\x92\xc8\xda\xbc\x49\x2b\x9c\x86\xd8\x5a\xf2\x9e\x16\xa9\x88\x04\xd2\xe8\x42\xf4\x0a\x76\xa1\x6a\xf5\x0c\xff\xf0\xea\x3e\x2c\xd7\x4a\xe9\xcf\xd6\xa2\x8e\xb0\x55\x31\xb0\xf6\xe5\x0e\xa1\x76\xeb\x82\x72\x3c\xc0\x91\x9b\x94\xc5\x27\xf2\x58\x43\x4e\x11\x0c\x1e\xd0\x8b\x09\x64\x8d\x58\x72\x53\x83\x66\x35\x6a\xac\x55\x90\x83\x46\x14\x71\x3e\x74\xf0\x2c\x14\x1d\x05\xa1\xeb\xc2\xa9\x14\x9c\xd9\xce\x53\xdc\x04\x35\xa5\xd3\x72\xcc\x4f\xaf\xca\x74\x9c\x51\x47\xe0\x0e\xd9\x40\x66\xdb\x8d\xa6\xad\xec\x96\x14\xfa\x56\x5b\x76\xf5\xf3\xf6\x1e\xf4\x73\xd9\xda\x00\x65\x69\x6f\x2a\x74\x5a\x2f\x23\x5e\xb0\x61\x70\xd7\x2a\x44\x8d\xf6\x9a\x3d\xd8\x60\x56\xef\x56\xd3\x0b\x6a\xcd\x6a\x1d\xbd\x2f\x03\xb5\xba\xb8\x33\x4f\xb5\x4a\x27\xd9\xa4\xfd\xf5\x77\x36\x2a\x91\xe5\xb6\x2c\x34\xe3\xa7\x37\x60\xba\xce\xed\xa5\x65\x92\x1d\x2b\xc5\x72\x95\x2c\x56\xf8\x89\xd1\x8e\xd6\x0f\xc8\xb6\x2d\x24\x80\x34\x70\x40\x73\x19\xaf\xd6\x79\xd1\x20\x12\x0e\x6f\xfd\xab\xb0\x0e\x9a\xe4\x16\xe3\x00\xac\xdf\x24\xf3\xc9\xe2\x06\x5b\xe5\xd9\x62\xb6\x04\x89\x4e\x5e\x61\x46\x33\xc8\xf8\x98\xa6\x58\xf8\x85\x0f\xd7\x57\x1b\x3d\xb7\x2f\xd5\x59\x0c\xaa\x58\x6a\x48\x1a\xe3\x67\x04\xb3\x51\x77\xf0\x57\x1c\x8d\x00\x1b\x72\x74\x3a\xa7\xe8\x75\x8d\x94\x2a\x6f\x7c\x18\x62\xec\x23\x9f\x9a\xe0\x00\xa2\x8d\x06\x7b\x1b\x8a\xb4\xb2\x30\xbe\x61\x46\xa7\x51\x63\x0f\xd4\x9b\x8c\x3b\xeb\x9a\x1a\x95\x08\x74\x2b\x7d\x09\xc7\x20\x70\x55\x59\x95\x64\xd3\x5b\x4d\x76\x87\xe6\x54\xf4\x7d\xee\xf9\xb5\xea\xb3\x93\x0d\xf8\x33\x2d\xbe\xbd\x53\x56\xb7\xeb\x85\xdb\xae\x9b\x3a\xb7\xe8\x08\x8f\xe5\x1a\xe1\xfa\xc2\xee\x92\x8e\x20\x4d\x5d\xb7\xb8\x69\x71\x66\x7f\x8f\x62\xbf\xe3\xce\xf6\xc2\x5c\x0d\xd9\x50\x05\x2c\x48\xa9\x2f\x8d\x7b\x69\xc5\xd6\x02\xaa\x03\xb9\x6c\x78\x0b\xb9\xb5\x14\x50\xea\x8c\xcb\x1e\xe1\xba\x9e\x57\x7b\xbc\xb8\x9e\xaf\x9f\x76\x50\x34\x3c\x8e\x73\x80\x45\x8a\x53\xb2\xbf\xbf\x94\x05\x53\x44\x4f\xaa\xc2\x36\x8f\x2c\x1e\xa5\x71\x49\x24\x3b\xfa\xac\xe1\x5d\x3e\x2b\x12\xef\x7a\x15\x93\x0a\x7d\x2b\x9b\xbc\xc5\x8c\x30\xab\xce\x01\xf1\x8a\x57\x85\x9f\xbb\xa3\x83\x4a\xd5\xe3\x1b\x62\xfc\x7a\x21\x4c\xe2\xe9\x28\xdf\x51\x0a\x35\xfc\x54\x04\x05\x7a\xef\x5e\x00\x24\x97\x0f\xde\xd5\x33\xcb\x6c\xd8\xb7\x28\xd5\x4c\x5d\x37\x5d\xcf\xbe\xab\x6f\x57\x61\x30\xfd\xbb\xa5\x71\x5d\x8c\x68\xf2\x5f\x26\x10\xc5\x89\x3e\x8d\x1e\x8a\xbb\xa5\x63\xc4\x53\xcc\xe6\xef\x2c\x29\x55\xf0\x9d\xc2\xc2\x0a\x1b\x41\xd1\x72\xdd\x34\x49\xd7\xf1\x5c\xcd\xb9\x2d\x81\xd5\xb8\xb9\xec\xa7\xd2\xda\x62\x54\xb9\x28\x9c\xa8\x85\xc7\xca\xe4\xef\x27\xbd\x6a\x70\x94\x4c\x76\xea\x3d\xdc\x06\x55\x34\x6a\x57\x5a\xcc\xc7\xb1\x3a\x0d\xf5\x2b\xd7\xb7\xa3\xc4\xac\x73\x57\x72\xac\x9e\xe4\x95\x95\x3b\xf5\x7d\xfa\xd2\x76\xe7\xea\x50\xcf\xb3\x96\x5c\xe9\xda\xd2\xdb\xdb\x12\x08\x4d\xd1\x24\x01\xa7\x71\x8d\xc6\x59\xde\xef\x8d\x46\xba\xe1\xfb\x74\x85\xaa\xdb\xc0\x1b\xd7\x0b\xa8\x1d\x76\x2a\x16\x9f\xad\xb3\x88\x15\x8b\x32\xce\x02\x5d\xa0\x57\xe8\x92\x49\x18\x35\x9b\x85\xe5\x7b\x67\x05\x47\xfc\x97\xad\xbc\x58\xb2\x8c\x7e\x3c\xc7\x0f\xb3\xed\xf1\x2a\x46\x87\xaf\xba\x41\x82\xa1\x6f\x55\x93\x6d\x53\x33\xb2\x21\xb2\x0f\x56\x2c\xc6\xea\x95\xd8\xd0\x5e\x4c\x2b\xae\xa6\xed\xb8\x94\x16\x5a\x5b\xfa\xb2\x24\xad\x3e\x14\x02\x09\xde\x5b\xff\xe3\x5b\x1e\xb6\x0b\xa3\x34\x9e\x4c\xd6\x53\x47\x41\x97\x09\xb3\xda\xf2\xe8\xcb\x21\x0a\x64\x2b\x9e\x9d\xe1\x75\xc8\xeb\xb3\xb3\xdb\x5b\x02\x73\x62\x58\x48\x8c\xe4\x41\xb7\xc5\x12\x43\x09\x20\x7d\x15\x7f\x0e\x9f\xc9\xf7\x4a\xfb\x9c\x93\xee\x97\x77\x5f\xe4\x40\xca\x20\xca\xfd\xa5\x32\x3b\x67\x7e\x57\x62\xca\xd3\x9b\x7e\xd7\x50\x24\x54\x79\x50\xd3\x94\xc6\x80\x33\xbb\x04\x06\x95\x21\xdb\x0b\xb6\x76\x51\x76\x2d\xc2\x84\x17\xbd\x35\x8b\xda\x5e\x83\x48\x36\xfc\x36\x1a\x14\x8f\xaa\x24\xc5\xc4\xb2\xf8\x06\x67\x13\xc4\x5b\x3b\xc6\x47\x0f\x6f\x5c\x4f\xc1\xb9\xf4\x1a\x6f\xa1\xbb\xca\xaa\x41\x7e\x5a\x27\x68\x8c\xa7\xa3\x34\x05\xb8\x24\xca\xce\x7d\x63\x01\x33\x43\xe0\xa4\x21\x7e\xff\x4d\x52\xcd\xc8\xfe\xda\x9f\x6a\xb8\x08\x80\x7b\x7e\x65\x3e\xb5\x68\xc6\x04\xfe\xb2\xc4\x0b\xe9\x2b\x48\x94\xf3\x86\xea\x82\x8a\x89\x3c\xb0\x44\x57\x22\x78\xaa\x92\x60\x7f\x2e\x0b\x12\xd1\x75\x22\x99\xe0\xdd\x93\x84\x25\xcb\xf4\x96\x74\x2a\x5a\x0a\x03\x5a\x01\x4c\x96\xd7\x40\x60\x49\x0a\x8a\x90\xfb\x11\x8a\x08\x80\x27\xaf\xc5\xab\x89\xe0\x9e\xd5\x86\x25\x77\x9d\x40\x6e\xb1\x8c\x7b\x07\x38\xa6\x51\x18\x4d\xf6\x00\x95\x1f\x7f\x24\xb3\x46\x56\x45\x24\xf2\xcf\x49\xe8\x2a\x81\x0b\x21\xc7\xdd\x33\x3a\xc3\x98\x75\x25\x72\x1e\x42\xa9\x2f\x51\x72\x9f\xde\x24\x68\xea\x61\x7e\x01\xc2\xf3\x3f\x8e\xc1\x5d\x6c\xf0\x05\x46\x8d\xae\xd3\xad\xf5\xce\x3c\xb0\x04\xaf\x85\xae\xb0\x14\x6e\xde\x69\x08\xda\xa3\xfb\x73\x82\xe1\x39\x3f\xc4\x16\x87\x0a\xa5\x21\xb4\x3b\xe2\xa0\x8a\xc0\x6f\x62\x51\x3f\xc9\xd0\xa3\x96\xa8\x64\xd6\x0c\xa8\x07\xfe\xf6\xa2\xdc\x0c\x36\xfe\xce\xe5\x22\x19\x99\x69\x92\x63\x3f\x7a\x14\xc6\x33\xd3\x61\x30\xb9\x35\x9a\xe3\x3e\x36\x48\x9c\x25\x93\xc9\x14\xdb\xcc\x68\xa1\xca\xdb\xa0\xdd\x62\xea\x16\x12\xfa\x72\x4e\xfb\x92\x55\xc1\xcd\xc6\x2f\xc1\x77\x0a\x01\x77\x92\xfd\x52\x40\xf0\x7b\xe7\x30\x72\x5c\xf5\xa8\x35\xd6\x8b\xe5\xbd\x9b\xa2\x75\x77\x5b\xb4\xbc\xdd\xa4\xb2\x4b\x73\x54\x15\xb7\xad\x3d\x76\x2f\xbb\xd8\x24\xc1\xe7\x6a\x90\xd6\x9d\x2d\xd2\xaa\x6f\x12\x8c\x31\xb9\xa5\x4d\xde\x94\xda\x24\xdb\xa1\x4d\xb2\x3f\xdd\x26\x85\x12\xac\x06\x0a\xaa\xdb\x64\xe7\xf2\x4c\xd3\x05\x15\xdd\xe4\xa8\x53\xdf\x2a\xe4\x3a\xd6\x36\x49\xb1\x15\x84\xdd\x4a\xdf\x05\xbf\xd8\x4d\xf2\xdd\xd6\x26\xa1\xe0\x9f\xf7\x6e\x93\xbb\x9b\xe4\xcf\x98\xac\xfb\x36\xc7\x6e\x65\x7d\x7a\x5b\xb0\x3b\x5f\xdf\x1a\xf7\x6b\x0c\xb7\x2d\xf0\x13\x86\x74\x87\x08\xeb\xab\xd1\x5c\x46\xc9\xc2\xe0\x56\x85\x04\x8f\x82\xb8\x1a\x47\xd3\x5c\x92\xca\x98\x00\x68\x4e\xb2\x8d\x35\x18\xf3\x62\x9d\xe3\x66\x66\xb6\xd3\x51\x8d\xc0\x93\xa8\xad\x54\xd8\x35\xbd\x8b\x94\xe3\x7c\x81\x3b\xb4\x05\x47\x2e\xd0\xda\xfe\x1d\x87\x4e\x29\x47\xd4\xe0\x41\xd7\xd6\x52\x79\x7b\xdf\x8f\x94\xe0\xea\x4f\x74\x22\x1c\x89\xc3\xbb\xa5\x15\xd1\x51\xc1\xdb\x8c\x06\x61\x30\x14\xc6\xb5\x25\x7f\x56\x2f\x64\x65\xb8\x30\x49\x5c\xed\xb6\x1d\x92\x50\x33\x81\xf9\xa4\xff\xbc\xa1\x97\xb6\xac\xc2\x4e\x54\x24\x6e\x86\x4b\xcd\x56\x9d\xb9\xb2\x69\x1f\xa3\x0a\x75\x6b\x65\xc1\x45\x33\x09\x59\x36\xcf\x0a\xca\xab\x8b\x15\x1b\x44\x5d\xd1\x68\x04\x49\xd5\x4d\xb6\xa8\xff\x72\x8f\x5b\x56\x2d\xfc\xcc\xa2\xfb\x8b\xbc\x6c\x55\xd3\x94\xd3\x8b\x9d\x69\x4a\x25\x72\x88\x4a\x98\x45\xb5\x14\xc1\x76\x7b\xfc\xa4\x79\xdf\xb5\x31\x51\x33\x2b\x68\x57\x53\x67\xa5\xa0\xa2\xb2\x70\x27\x93\x2c\xde\x25\xb4\x53\xbd\x4a\x65\xd7\x53\x77\xb9\xda\x4d\xcf\xdc\x8a\x28\xea\x0e\xd8\x12\xdd\x4f\x15\x61\xdd\xb6\xb6\x88\xca\xd0\xb4\x9b\xe4\x27\x15\xa8\xdc\x34\xc9\xf5\xf9\x24\x79\x9f\x4c\xaa\x16\xc0\x8b\x84\xf7\xf7\x31\x1b\x67\x76\x8f\xe8\x16\x3b\x42\xa4\xbc\x73\x51\xee\x42\xd1\x47\xf0\x15\x31\xbc\x11\x8e\x85\xf0\xcb\x98\xf0\x80\xfe\x8a\x0a\x77\x04\xb3\xfc\xf3\xd5\x75\x7a\x59\x39\xcd\xa7\x94\xfa\x79\x7e\x69\x66\xaf\xc6\x19\xba\x84\x2a\x5e\xb5\x68\x0d\x09\x27\x59\x60\x1e\xcd\x28\xd4\xba\x89\xcf\xaf\x92\x75\x6b\x3d\x5a\xb6\x2e\x81\xb7\x29\xf2\xd7\x1a\x2f\xa6\x34\x2a\xad\xde\x9e\x8f\xbc\x40\xd0\x7f\x7e\x83\x16\x99\x1a\xb3\xc5\x75\x1a\xe3\x36\x5c\xe6\xb5\x21\xe8\x87\xc6\x2f\x46\x58\x2f\xae\xc7\xfc\x5e\x81\x21\xce\x47\xe3\xab\xb7\x1c\xb5\xcd\x9d\x44\x9a\x04\x7b\x22\xd9\x33\xe0\xd2\x14\x12\xe3\x40\x96\xa6\x90\x0e\x19\x59\xc9\xf7\x49\x9a\x9c\x27\x53\x9a\xdd\x36\x2e\xc1\x17\x8d\xe7\x26\x71\x7c\xbd\x4a\xa9\xb2\xb4\xf3\xfb\x72\x94\xac\x60\x18\x76\x59\x83\x91\x1a\xca\xb6\xd9\xda\x8d\x17\x9d\xaf\x58\x14\x4e\xcc\xe5\xf4\x74\x15\xd3\x0e\x51\xb7\x40\x06\xaa\x02\xf9\x2d\x15\xf6\x49\xcf\x1e\x03\xe5\x02\x80\x99\xeb\x2b\x70\x81\xbd\xb7\x45\xde\xca\xe3\x7b\x83\xb3\x62\x44\x86\x4d\x89\xe3\x0a\x77\x40\xea\x38\xb5\xed\x33\x42\x1b\x4c\x86\x9b\x1a\x89\x64\x55\x24\x0e\x07\xf1\xcd\x50\x9d\x34\x9e\xf8\xfd\xd6\x11\x07\xe5\xb6\xdc\xa8\x8a\x4c\x7f\x1f\xcc\xd3\x61\x7d\xa6\x9b\x64\xb2\x06\x95\x3b\x51\xef\x97\x31\x79\x9f\x08\xd8\xa6\x0d\x4a\x70\x12\x07\xd7\x59\xa7\xa3\x5c\xaa\x2e\x6f\x0c\xf6\xfc\x7e\x63\xbe\x98\xc7\x0d\x2a\x50\xb6\x1e\x5d\x66\xf8\xb6\x72\x75\xe5\xad\xad\xec\x95\x18\x26\xd9\xef\x51\xcc\xdb\x8a\x65\x97\xac\x4c\xc5\xc8\x54\xf9\x80\x85\xca\x2b\x5f\xb0\xa5\xd3\xa1\x7a\x93\xd5\xe8\xe6\xb5\x27\x59\x85\x69\x4a\x5e\x53\x60\x5e\x5b\x60\x5e\x2e\x50\x49\xb7\xb6\xc4\x37\x56\x89\x0c\xd1\x00\x7b\xd3\x90\x4a\x42\xeb\x56\xd9\x11\xb8\x38\xb3\x49\xa6\x4a\x81\x29\x95\x3e\xe3\x37\x9a\xd9\x37\xd4\xf7\x06\xcd\xc3\xd8\x68\x18\x4f\x2f\x73\x9d\xf4\xf7\xd4\x4e\xf2\x1b\x65\x8e\x5e\xdb\x2c\xd9\x96\x40\x37\x81\x2a\x08\xab\x5c\x65\x35\x44\x7b\x4e\x81\x74\x45\x3b\x3d\x75\xba\x84\x6c\x2b\x95\x1f\x64\x67\x91\x2a\xf2\xf1\x66\x3b\x1f\xd0\x32\xf9\x9d\x7c\xc4\x92\x8f\x1b\x97\x0f\xd5\x84\xb9\x61\xc4\xa2\xf5\xdf\x36\xee\x00\x44\x56\xdc\x1e\x96\xae\xe2\x1c\x07\x03\x00\x1d\x75\x22\xfa\x5a\x8a\xe3\x0c\x1e\x31\x78\xc6\xdb\x99\x81\xfd\xb7\x6f\x61\xf8\xbd\xbd\xf5\xc6\x64\x96\x38\x76\x26\xb8\x8b\x00\xc5\xcb\x95\x23\x23\x02\x05\x05\x1e\x22\xc3\x8f\x50\x24\xa2\x0e\x7d\xb9\xa1\x12\x7e\x5a\xd1\x8f\x1c\x80\x3d\xfb\x0e\x44\x28\xfb\x7a\x59\xcb\x10\x02\x25\x3d\xbc\x32\x59\xb1\xd1\xac\x64\xa3\x59\xc9\x46\xb0\x0b\x17\x24\x2e\xb6\xca\xfc\xb9\x9c\xee\xb4\xc4\x13\x48\x38\x80\xca\xd3\x7a\xbe\x40\x8c\x09\x5e\xde\xc0\x97\xd9\x53\xa4\xef\x84\x64\xdf\x8c\x18\x84\xa1\x87\x19\x16\x1a\x18\x46\x35\xb4\x24\xab\xab\x39\x9a\xae\xff\x47\x9c\xf7\xa5\xa8\x8d\xcc\x07\x9e\xd1\xae\xa6\xa9\x29\x4e\xf6\x3c\xd3\xdc\xcd\xdc\x4e\x19\xfa\xa6\x95\xb4\x74\x9a\x9a\xbf\xa7\x4c\x1a\xb5\xc4\x92\x98\x16\x98\x42\x0c\x35\x22\x10\x1d\xfa\x5d\x4b\x09\xa4\xd5\x85\x2a\xbc\xde\xdf\x47\x41\x84\x9c\x47\x64\xe0\x6e\xa0\x79\xd2\x7d\x50\x89\x29\x30\x59\xde\xb8\x59\x72\x11\x9a\x2c\x6f\xdc\x2c\xf4\xa4\x13\x31\x8d\x84\x75\xe6\x7d\xc4\x4f\x60\xdd\x86\x74\x53\x30\x5a\x6a\x57\x09\xb5\xcf\x83\x74\x57\x0e\x8b\x8d\x8d\xdd\xb6\x76\xc1\xfc\xa5\x44\x6e\x60\x99\x25\x73\xbc\x72\xbd\x7e\x01\x1d\xf8\x0f\xf4\x25\xc8\x18\xa0\x4b\x59\x4e\x1d\x9f\x3f\x92\xa2\x4c\x86\x82\x7b\x6e\x94\xf4\xa5\x4c\xbb\xb2\x11\xf8\x90\x5e\xac\xfb\x6b\xac\xda\x4f\x7f\xf6\x36\x0a\xbe\x0a\x5a\x91\x22\x0d\xe5\xb5\x28\x7a\x75\x53\x41\x40\x36\xc9\x3c\xf2\xa0\x84\x67\xd3\xd1\x6c\xd9\xcd\xe8\xc7\x37\x11\xd6\x57\x81\x30\x51\xbc\x43\xa9\x87\x78\xdb\x72\x57\x3d\x6a\x35\xec\x63\xc4\x76\x0f\x10\x9b\xba\x40\xbf\x49\x57\x6d\x48\x25\xb4\xa2\xd8\x47\x75\x05\x74\x0e\xa4\xb2\x24\xc3\x16\x80\x7c\x9f\x38\x3c\x55\xf9\xfa\x74\x0d\x3d\x49\x19\xf8\xd6\xc5\x74\x11\x6a\xea\xa4\xe5\xb1\x87\xa8\xb7\xb7\x5a\x52\xf0\x3e\xca\xfa\x9e\x16\xe8\xd7\xf4\xbd\x81\x74\x51\xc9\xd6\x02\x19\x3a\x54\xa2\xa6\x82\x44\xf0\x66\x83\xee\xfb\x45\x32\x79\x10\x14\xba\x3c\x3a\x3f\xfe\x47\xab\xf7\xa3\x39\xde\xee\x8a\x57\x0d\x78\xbb\x3b\x22\xc2\x38\xfe\x30\x01\x99\xe4\x65\x7f\x93\xd1\x6e\x8c\xfb\x8e\x8c\x29\xe7\x9c\x77\x4f\xe9\xa4\xeb\x65\x39\x81\x3c\xfa\xea\x3c\x94\x84\x1b\x85\x4a\x29\x72\x60\xa8\x4c\x28\x14\x42\x2a\xf4\x92\xae\x9a\x44\x79\x55\x74\x4e\x5c\xaa\x94\xab\x37\xac\x1f\x82\xed\xa0\x3a\xc2\x4d\x9f\xde\x29\xdf\xcf\xea\x14\xb4\x92\x89\x36\x8d\xca\xe2\x32\xf9\x88\x81\x8b\x0b\x69\x89\xed\x90\x78\xce\x5c\x4a\xe6\x52\x66\x27\xb2\x4a\x41\xf7\xfc\x7a\xe6\x59\x66\x2c\xda\x3b\xfc\xbb\x37\xbf\x4d\x7d\xe5\x74\xa8\x14\xb0\x4d\x99\x31\x5d\x84\x16\xdf\xde\x54\xa1\xe5\x66\x9c\xb1\x8b\x22\xbf\x3d\x06\xbf\x5a\x0d\xfd\xb6\xd0\x22\x35\x1e\x9d\x21\xf0\xd5\xf5\x72\x09\x04\x53\x4f\xd9\xef\xf2\xa8\x73\xe3\x08\x48\x6e\xaf\xab\x6c\x37\xf9\x5a\xd1\x74\xf4\xe2\xe3\xc6\x11\x2d\x62\xbe\x13\x74\xf2\x33\x29\x45\xda\xbf\xa9\xd6\x1e\xdd\x37\xaa\x55\x48\x35\xb8\xdf\xad\xd1\xd8\x42\xfe\x82\xda\xea\xec\xd0\xef\xa8\xbb\xad\xae\x97\x38\x45\xb2\x7a\xd9\x41\xc3\xb7\x93\xb4\xc0\xfd\xf2\x70\x07\xfe\x89\x06\x56\x0c\x72\x26\x39\x1c\xf6\x68\x45\x13\x6f\x0a\x54\x8d\x49\x43\x42\x9c\x45\xcd\xc3\x9b\x52\x43\x8b\x38\x07\xf8\xdf\xe7\x45\xb8\xf4\x03\xa2\x81\x1e\xa4\xc1\xbc\xdb\x4c\xe4\x06\x9e\x5b\xa5\x0f\x2b\xc6\x6a\xc8\x58\xc1\x33\x64\x53\x3b\x58\x1c\xa7\x01\xcc\xb3\x1c\x9b\x39\x8f\xba\x08\xc5\xef\xd5\x1b\x30\x32\x43\x3b\x5a\xb0\x1d\x8d\x95\xad\xf4\x6e\x92\x5f\x69\x1a\x78\xe5\x7c\xe3\x0b\xcb\xe4\x92\xa1\x70\x0c\xba\x70\x2c\xbe\x50\x1b\x6a\xb8\xd7\x7c\xad\x76\xd5\x10\x09\x61\xd3\x15\xca\x39\x30\x76\xc8\x17\x19\x0f\x0f\x39\xff\xc8\x72\x30\x78\x95\x69\x1e\xf9\xc6\xe3\x29\xde\xc5\x91\xeb\x27\x39\x9d\x8f\xec\x09\xf4\x4b\x86\x59\x43\xb8\xb4\xf7\xef\x9d\x25\xdc\xda\xd5\x9f\x92\x4d\x2b\x04\xf9\x64\xa3\x11\x46\x1f\x33\xed\x43\xe4\x5d\xc9\xaa\x48\xdc\xb1\x4f\xbc\xeb\x3a\xc2\xda\xc8\xdc\xe5\x4d\x22\x92\x6a\xaf\xb0\x3f\x44\x82\x6b\xf6\x51\xf5\x8b\xeb\x54\xf6\xf1\x53\xb9\xf3\xd4\x59\x3d\xb2\xaa\xea\x8e\xd1\x92\xad\x76\xe2\x36\xae\x86\xbf\xd3\x4d\xa3\x20\x99\x6e\x1e\x05\xc9\xb7\xa9\xd4\xc6\x57\xbb\x79\x09\xdc\x2d\x33\x44\x7a\x56\xdc\x45\xfe\xe5\x6a\x35\xca\x95\xbf\xad\x44\xdf\x86\x89\x7c\x5e\x8d\x99\x17\x30\x73\xbd\x73\xcb\xad\x6f\xd9\x69\x71\xb6\xaa\xbb\x95\x0d\xf1\x5a\x16\xda\xa1\x9e\x17\xe0\x79\x94\x13\xbc\xde\x07\x36\x3e\xef\x66\xe3\x5b\x9b\x81\x77\x6b\x90\xb0\xa6\x41\x42\x68\x90\x5d\xca\xdc\x32\xfe\xfb\x5d\x6f\x9b\x05\xf8\x73\xd4\x69\x27\x32\x77\x3b\xeb\xfb\xd7\x87\x2d\x2b\xce\x59\xf4\x61\x6b\x6f\xde\xcb\x9e\x3e\x0d\x6f\xf7\xf2\x21\x93\xf5\xbb\x99\x2a\x20\xdf\xad\x80\xfc\xbe\x05\xe4\xaa\x80\x31\x59\x9b\xdd\x6a\x01\x3e\x07\xfc\x95\xf6\x69\xef\x03\x0d\x32\xe6\x0d\x66\x6d\xdd\xac\x6f\x92\xbb\x38\x50\x98\x74\x5f\xd7\x0d\xc9\x0c\x18\x4f\xe6\x1f\xea\x8c\xdd\x5c\xcd\x30\xf8\xda\x3d\x69\xdd\x58\x31\x6d\x2e\xa9\x43\x05\x22\x0b\x45\x0e\x36\x33\x14\xeb\xba\x0d\xde\xc4\xb7\x97\x05\x11\xf1\x9b\x85\x11\x72\x4a\x8b\x5a\x00\xcb\x02\x09\xcc\x42\x5a\x10\x71\xd5\x74\x40\x05\x40\x3a\x38\x01\xef\xe3\xd5\x5a\xe6\x81\x3f\x3e\xe5\xf1\xb2\xd0\xc7\xbb\x2e\x4f\x33\x8c\x55\xbc\x8e\x10\x1d\xd2\x43\x4c\x84\x7e\x03\x78\x7b\x96\x7f\x70\x7b\x9b\x85\x7b\xd6\xe2\x01\xce\x42\xf5\x90\xc0\x25\xc9\xd5\xb6\x5c\x32\x9b\x4b\x66\x33\x86\xe5\x01\x71\x1e\x46\x79\x48\xd3\x7a\xb7\xef\x0c\x48\x0e\x90\x6e\x98\xc5\x3c\xf0\xc7\xa7\x3c\x5e\x8e\xcc\xe6\xc1\x69\xce\xcc\x22\x3a\xa4\x87\x98\x88\xcc\xe6\xc0\x6c\x6e\x31\x9b\x87\x7b\xd6\x9a\x06\x32\xab\x47\x2c\x2e\xc9\xd7\x2d\xca\x12\xb6\xb9\xe9\x93\xa0\x6c\x88\x92\xb3\x03\x43\x95\xb1\x30\x5d\x9c\x41\x59\xf2\xf2\x4d\x37\x80\x7e\xdf\xd6\x0e\x4a\xa8\x2e\x7f\x58\xf7\x22\x7f\x28\xa6\x32\x7f\x06\xd3\xc5\x19\x94\x85\xad\xf8\x51\x32\xd7\xef\xdb\x44\x8f\xac\x53\x97\x40\x25\xc8\xa1\x88\x01\x2a\x35\x38\x6a\xd0\x29\xa4\x5e\x60\x57\x52\x52\x37\x1d\xb7\xfa\xe6\x0d\x67\xfe\x77\x7b\xeb\x6d\xf5\x3a\xee\x18\x36\x64\x0b\x6f\xec\x49\x65\xb9\xc8\xbd\x3d\xe0\xdf\x34\x61\x64\x35\xe0\xed\xed\xde\x1e\xf0\x9e\x5b\x89\x46\x7a\x7c\x85\x0c\x5f\x30\xc3\x0e\x15\x99\x5c\xd1\x58\xcc\x1b\x6e\xec\x59\xeb\xdb\x41\xf4\x11\x6f\x0f\x4a\x5b\xd2\x4e\x0b\x30\xcd\xf1\x8d\x7e\x4b\x9d\xb4\x1b\x27\x6d\x0e\xaf\xf3\x9b\x34\x36\x00\xc8\x3b\x8f\x53\x2b\x77\x5c\xc0\x48\x6f\x5c\x0c\x1d\xd4\xd3\x36\xae\xd1\x60\xd0\x98\xe3\x47\x1c\xdc\x9d\x01\xff\x6e\xd0\xdd\xa5\x3f\x04\xa1\x3f\x37\x0d\x68\x56\xc2\xe0\xa7\x39\xe1\x62\x68\x7a\x0e\x64\x92\xf0\x21\x3e\xfa\xa5\xe8\xee\x67\xf4\x15\xe0\x6b\x3c\xb6\x29\xd3\x01\x92\xbf\x92\xe1\x89\x06\x8d\x57\xd7\xf3\x09\xfa\xca\x8d\x17\x0b\xf9\xf0\xf3\x75\x9c\xf2\xd3\x5f\xe3\xc9\x5c\x3d\xff\x7c\x09\x52\xe3\xc7\x6f\x57\x09\x3f\xbc\x82\xc9\xe8\x0a\x1f\x31\x6e\x2a\x15\x74\x76\xbd\x1e\x5b\x11\x3c\x3e\x82\x4b\x8d\x45\x57\xdc\x73\xce\x1e\x1c\x9e\xa1\xfa\xcb\xcf\xcf\x10\x07\x23\x81\x10\x7a\x7e\x37\x76\x2e\x91\xbf\x05\xd5\x7a\x83\x21\x83\xef\xc8\xa1\xf0\x64\xb6\xef\x16\x50\x97\xbb\xf2\x10\x92\xcc\xf0\x22\x99\x4e\x93\x34\x1e\x83\x88\xee\xcc\x67\xe3\xea\xec\xf3\x6b\x98\x73\xdd\x9d\x93\xd0\x54\xa6\xc5\x7c\x7d\x79\x67\x16\x44\x92\x19\x5e\xed\xc6\xdf\x2b\x87\xb5\x9f\x41\x21\xb6\xe6\x40\x04\x0b\xf7\x03\x4c\xbf\x38\x02\x66\x45\xae\x60\xc3\x07\x4b\x7e\xbc\xa8\x27\x29\x11\x90\x64\x5a\x56\x0e\xa5\xa2\x4b\x2b\xb8\x87\x52\x10\x2b\x00\x76\xfb\xcc\x9a\x6a\x48\x42\xf9\x6e\x74\xf2\xad\x64\xaa\x14\xaa\x8e\x96\xc2\xdd\x4a\xb0\xa4\x6a\x75\xd4\x08\x71\x2b\xa9\x3a\x25\xac\xa3\x68\xe3\xdf\x41\xb8\xa4\x9e\xf5\x34\x09\x75\x3b\xb9\xa2\xe2\xd6\x12\x43\xc4\xad\xa4\x2a\x54\xba\x8e\xd8\xab\x1d\x2a\x5a\x54\xf6\x4a\x5a\x88\x54\x4b\x45\x0f\x26\x6e\x3e\xb2\xaf\xc6\xee\x69\x4b\xcb\x97\x3b\x61\x22\x52\x8d\x1a\x5f\x8c\x1e\x7c\x71\xfe\xe0\x8b\xf8\xc1\x17\xaf\x1f\x7c\xf1\xa6\x51\x81\x07\x38\xb3\xc3\x2f\x26\x87\xe5\x54\x49\xe1\xbb\xee\x17\x2f\xba\x5f\xbc\x6a\xd8\xd6\xfc\xf3\xd9\x71\x9b\xea\x97\xe7\xe7\x78\xb3\x3b\x9d\x3a\x91\x25\x30\x79\xa6\xcd\x74\x99\x22\x13\x63\x4a\x16\x91\x19\x36\x30\x66\xfd\xef\xa3\xf9\xf5\x68\x45\x65\xc6\x30\xe2\xf1\xe3\x0b\xbc\xff\x0d\x7e\xbf\x5c\xae\x92\x29\xbd\x23\xf4\xbf\x5f\xd3\x68\xf7\xdf\xaf\xa7\xf8\xf6\xe5\xf5\xdb\xeb\x14\xd7\x44\x5e\xc5\xcb\x75\x4c\x37\xfd\x89\xc6\x8f\xe3\xf5\x82\x9f\x7e\x58\xbc\x57\xc0\xaf\xe3\x31\x3f\x16\x8a\x2f\xd6\x02\x58\x61\x36\x98\x03\x2e\xdf\x2e\x9d\x0b\xe7\xb2\xb9\x60\x2e\x92\x8b\xe3\x92\x78\xc0\xc3\x42\xda\x39\x3a\x4f\xaa\x44\x9a\x68\x83\x85\x33\x8b\x25\xf4\xdd\x59\x6f\x60\xe0\x0f\xf2\x9c\x11\x24\xcc\xa9\x42\xee\xc4\xe7\x8e\x83\x37\xc5\x30\x68\x23\x1c\x2a\x72\xfd\x19\xb4\x56\xa2\xeb\x41\x8d\x00\xd6\x68\xe8\xf9\x4d\x89\x5b\x20\xe1\x30\xe2\xe6\x90\x27\xd4\x54\x8d\xd2\xc8\x7e\xe3\xbd\x9a\x2e\x42\x1b\x46\x7a\x17\x09\x87\x7e\x17\x11\xaa\x78\xa7\x68\xa8\x3f\x01\xde\x3c\xbe\x79\x20\x9d\x08\xaf\x13\x1f\x89\xc0\xba\x48\x25\xbf\xa3\xba\x42\x81\xe4\x70\xa8\xdf\xd9\xaf\xc0\xf7\xfc\x2e\x69\x12\xaa\x9b\x6f\x57\x29\x32\x76\x2b\xb4\x64\x48\x9d\xd2\x7a\x29\x0b\xc6\x15\x20\xa2\x54\xca\xef\xc7\x8b\x37\x8e\x77\x6e\x84\x66\x2b\x1e\x35\x00\xa7\xf5\xec\x8b\x01\xf9\xbe\x37\x4a\x68\x21\x4a\xeb\x24\x7e\x78\xa0\x6b\xe9\x8e\xe7\x50\x03\x6a\xc6\x8a\x04\xdf\x3f\x7c\x7c\xf2\x30\x3e\x96\xe7\x3e\x5d\x1f\xb2\x1c\xf8\x1f\xd2\xf0\xeb\x24\xb5\x3e\xd4\x6b\xbd\x78\xbe\xb8\x89\x57\xcf\x46\x29\x7e\x6a\x49\xa2\x47\xad\xa4\xa7\x83\x91\xbc\xa7\x50\xac\x44\x73\x00\xc8\xc3\x1d\xbb\x92\x57\xdd\x95\xfc\x9a\x96\x6c\x59\xef\xe0\x3b\x36\x13\xff\x8b\x47\xbb\x75\xb2\x2a\xb5\x78\x74\x60\x09\x57\x66\x28\x2b\x89\x52\xec\x72\x13\x69\x3e\x2a\xdb\xaa\xd8\xf6\x9c\xa7\x89\x52\x65\xbe\x0f\x1f\x51\x7d\xf2\xbd\x28\x4a\x7c\xbe\x91\x41\x89\xaf\x89\x73\x81\x48\x89\xce\xd5\x26\x9d\x4e\x8a\xa7\x71\x4a\x5a\xc7\x78\x5c\x36\x10\xfb\x4f\xac\xd5\xc6\xea\x40\x37\x71\x7c\xa5\x0b\x49\x69\x6c\x73\xd2\xd2\x42\x62\xa1\xe7\x10\x8a\xd3\xc3\x24\x5a\xb9\x93\x21\xaa\xec\x65\x2e\x2e\x03\x35\xde\x85\xba\xb2\xd1\x1e\x94\x1d\x77\x00\xf2\xbc\x8c\x0b\x08\x2f\x63\xcf\x1e\xa8\x7d\xbb\x03\x3d\x5f\x2c\xae\xae\x97\x85\x0c\x0c\xac\xcf\xc4\xe3\xd9\x1d\xe5\x38\x83\x5e\x45\xf6\x1d\x4b\xae\x21\x43\xe3\xea\x36\x0e\x78\xdc\x2f\x64\xd8\xa5\xcc\xca\x8c\x77\xd7\xb8\x3c\xd0\x57\x92\xd8\x99\x83\x1a\x52\xcb\x78\x85\x9f\xab\x80\x93\xc3\xbf\x7f\x71\x58\x70\xcf\x7e\x1a\x4d\xd2\xe8\x63\xa3\xd5\xe8\x36\x1a\xe2\xac\xdb\x78\xd0\x10\x41\xb7\x11\x34\x36\x05\x3c\x40\x1a\x75\xab\x37\x37\x56\x8a\x7d\x30\xd1\x1d\x6b\xb8\x11\x5f\xde\x91\xb5\x80\x7e\xbe\x0d\xbd\x5c\x55\xce\x2c\x07\x51\xc8\xfe\xd5\x9d\xd9\x4b\x59\xc6\x5d\xb7\xba\x5e\xb5\x2b\x0c\x42\xb5\x68\x8b\x65\x89\xba\x16\xaa\x37\x31\x96\x57\x2c\x45\x07\x0c\x6d\xfc\xe9\x59\xbf\xbb\x57\x56\xb9\x04\x20\xf3\x7e\xff\x29\x79\xbf\x08\x3b\xb7\xb7\x61\x47\x92\x78\xb7\x2b\x89\xb0\x59\x61\x31\x91\x91\x23\xa0\xf2\xfc\x5e\x8c\xb8\xeb\x11\x92\xc4\xec\x7e\x24\xb8\x79\x9b\xa1\xac\xc6\x8b\x7b\x32\x20\x97\x35\x64\xee\x65\xa5\x56\x59\x32\x3b\x8d\xc2\x4e\xbf\xf1\xd3\x0b\xe8\x4a\x5f\xbe\x80\x0e\xf4\xea\x5e\xc5\xbd\xb2\xaa\x8a\xc5\xfd\x65\xe7\xdc\x15\xd6\x9f\x85\x8e\x64\x6e\x6a\xb9\x96\xcb\x51\x7f\xbd\x6f\x39\xb3\x45\x55\x39\xd9\xdd\x1d\xc8\x17\xaf\xb7\x23\x71\x0f\xcb\xef\x25\x36\xe3\x4d\x7f\x11\x06\x81\x64\xe6\xcd\x27\x93\x88\x1f\x02\x89\x87\x40\xe2\x57\xcd\x2a\x7a\x94\xa2\xf1\x45\xa3\xbc\x36\x04\xc0\x8d\xb1\x93\x4b\x98\x5d\xc4\x2b\xb2\x93\x0e\xe8\xaf\x30\x50\x6b\xeb\x08\x96\xb0\x2a\x11\x2c\x9e\x03\x7e\x61\x6c\x1c\x58\xb3\x72\x92\x65\xaf\x08\xf8\x1c\x63\x98\xc5\x58\x11\x30\x50\x4e\x0a\xb4\xb2\x88\xcb\xa0\xef\x5c\x10\x6a\x70\xe7\x21\x98\x8a\x2a\xe8\xbb\x52\x76\xe9\x62\x3c\x2f\x70\x66\xf5\x58\x31\xab\xe0\x5a\xde\x37\xff\xa2\x98\x8d\xfa\x99\x58\xba\xe0\x2f\x67\x3f\xcd\xa0\x0b\x39\x30\xd9\x47\xa0\x6f\x94\x84\xc8\xb4\x79\x35\x01\x94\xbe\x4a\xc8\xb2\xf8\xbf\xd6\x65\xe6\x15\x08\x4b\x93\x2d\xd1\xd2\x42\xf4\xeb\xaa\x14\x54\x5a\xd0\x59\x27\x85\xa4\xf3\xc6\x85\x29\x2d\xb3\x34\x8b\xe0\xbf\x2a\xf5\x72\x69\x27\xe0\xe7\x8e\xa6\x3f\xf1\xd0\x6d\x94\x6c\x4e\x9c\xd2\x50\xfe\xb7\xf4\xe0\x6f\x93\xa6\x19\xcf\x47\x20\x30\xed\x2a\x70\xf4\xfe\xd1\xac\x0b\x5d\x62\xd6\x0d\x8d\xbb\x2d\x95\xdf\xf6\x31\x25\xe8\x2f\xeb\xb1\xe3\x16\x32\xf4\xfb\x74\x51\xc6\xf3\x1a\x5f\xbc\x69\x7d\x31\x6b\x7d\x31\xf9\x59\x2d\xec\xb4\xbf\x78\xfe\x6b\xa3\x54\x4a\x02\xb9\xdd\x35\x26\x98\x56\x7d\xff\xea\x47\x0e\x92\xb6\xbf\xdf\xc4\xe9\x33\x8d\x74\x8d\x4e\x10\x04\xad\x20\x84\xff\x7f\x0e\x82\x2e\xfd\xdf\x06\x10\x10\xed\x97\x38\xfa\x81\x2e\x17\xef\x96\xe0\xa2\x06\xb3\x4d\x32\xb5\xc2\x2c\x73\x8c\x36\x39\x2b\x80\x19\x99\x66\x43\xa6\xa8\xb9\x40\x92\xfe\x30\xfa\x81\x7d\xfe\x3e\xed\x7d\xe4\xb9\x57\x5d\x31\xeb\x05\x57\x2c\x2a\x21\xe8\x24\xed\x97\x73\x47\xd9\x75\x09\xc6\x5e\x66\x08\xe3\x23\x7b\x26\x87\xb0\x43\x80\x95\xe6\x72\xa5\x49\x21\xaa\xaa\x33\x8b\xc6\xb1\xd1\x25\x56\x3b\x2d\x2c\xac\x1d\x98\xd5\x78\xbf\x50\x23\x6b\x86\x43\xef\x85\x69\x8b\x5a\xd8\x75\xe6\x38\x8c\x58\x9e\xe3\xcc\xc8\x34\x7c\x8a\x8c\x70\xc1\xa0\x28\x23\x80\x7d\x9a\x8c\x0a\xc4\x76\x95\x91\xf9\x2e\xe2\x17\x6a\x64\x64\xc4\xef\x95\xd5\x76\x65\x24\x11\xcb\x32\xba\x04\x23\xbd\xd3\x6a\x15\x22\xe0\x58\x16\xd5\xad\xa3\x1c\x9e\x04\xbd\x0a\x51\x1e\x9d\xc4\xc7\x07\x5e\x51\x98\x08\x6d\x29\x92\x78\xb6\x5b\x3e\x7d\x92\x80\xa9\x88\x4f\x90\xb0\xfa\xd6\xe5\x3b\xd2\x30\xd2\xc5\xb7\x0a\x71\xb9\x92\x25\xa4\x0a\xdd\xc3\x11\xeb\x33\xad\x90\x92\x69\xd9\x71\x79\x94\x9d\xd6\xc2\xca\xe0\xae\x4b\x7a\xea\xbb\x9a\xef\x56\xc3\xd2\x37\xfa\x76\x51\x51\xd3\x82\xb6\x11\x9a\x16\x8a\x33\x2e\xd0\xee\x7c\xbc\x80\x31\x8d\x06\x60\x3d\xc4\x71\x8c\x37\xb5\xc3\x9f\x23\x70\x9e\xa0\xab\xc0\xef\xb1\x78\x02\xff\xc2\xc7\xf0\x07\x9b\x16\xef\x6d\x87\xbf\x9d\x10\x9f\x1f\x1e\x75\xe0\x2f\xad\xcb\x89\xf0\x51\x07\x13\x4e\x82\x87\x94\x7e\xfc\xa4\x13\x9f\x88\x47\x8f\x1e\x9d\xc0\xcf\x51\x78\x0c\x79\x4f\xcc\xb2\x38\x95\x4b\x63\xee\x8b\x78\x7d\xb9\x98\xe0\x77\x67\xd7\x82\x88\x70\x28\x8a\xa0\xe3\x32\x28\xac\x80\x1d\x05\x16\x8c\xfb\x9a\x43\x4d\x82\x8e\x2b\xb0\x2a\x60\x0e\x35\xd4\x2f\x87\x16\x01\x8e\x8a\x80\x93\x52\x96\x8e\x05\x41\xc7\x24\x2c\xbc\xdb\xe9\xb8\x1a\xe4\x32\x4c\x6e\x62\x19\x62\x97\x8b\x8b\x60\x22\x1c\x56\x89\xf8\x5b\x39\xe9\xd7\x22\x56\x1e\x3b\x8c\xfc\x0d\x19\x15\xee\x3a\x36\xb4\x4c\xf2\x57\x0d\xbf\xf2\x10\xa4\xad\x9e\x55\xf9\xce\x1f\x7c\x31\xa9\xce\x1a\xee\x45\xd6\x8c\xb8\x32\xf3\xa8\x36\xb3\x99\xea\xec\xef\xef\x40\xe8\xfb\x07\x5f\x2c\xb7\x10\x52\x06\xa7\x32\x2b\x38\x42\xdb\xea\xae\x47\x83\x8a\xcc\xf8\x65\xac\x3e\xab\x19\x6c\x2b\xb2\x82\xe3\xb5\xb5\x54\xe7\xab\xfe\x96\xb6\x8e\x9c\x84\x6f\xdd\xf9\x59\x49\x31\x8c\xa7\x57\xea\x96\xed\xd2\x0a\xbf\x19\xfc\xdd\x68\x0d\xa5\x89\x19\xa5\x7b\xe5\x18\x13\xb5\x65\xd5\xd6\xc6\xdf\x94\x6d\x96\x2d\x8b\xe8\x23\x19\x36\xeb\x2a\x22\xdc\x1a\x29\xd2\xf5\x62\x49\xb7\x31\x5b\xac\xb1\x09\xf4\x9a\x8c\xd1\x34\x28\xe4\x63\x3b\x25\xd0\x2c\xd7\x9a\x12\x12\x10\xbf\x18\x4b\x83\x55\x2f\x33\x24\xa5\x79\x99\xa9\xd2\x07\xb3\x41\x40\xeb\xde\x62\x86\x1b\xd7\x37\x7e\x89\x72\x5d\x3f\x95\x4e\xfa\x96\xae\xca\x18\xdb\x7a\xab\xb5\x4f\xa3\x26\xf7\xdd\x7d\xd6\x6c\x92\xa9\x21\x71\x47\xcf\x55\xdb\x66\x4c\xe7\xbd\x93\xe2\x1d\x5d\xd8\xda\x22\x53\x47\x60\x7b\x47\x76\x76\xbc\x54\x93\xd8\xde\x9d\x9d\xcd\x2c\xd5\x04\xb6\x77\xea\xf2\x6e\x9d\x62\xbf\xd6\x9a\x71\x77\xaf\x36\x4a\xe4\x44\xc1\xaf\x50\xdf\x72\xc7\x26\xd5\x74\x3a\x37\x79\x11\x7f\xb2\x83\x9b\x12\x6b\x2a\x25\xef\x5e\xc2\x30\x73\x80\x90\x5d\xae\xf0\xee\x2e\xd3\x7d\x56\xf1\x3f\xae\xe3\x74\xad\x0b\x97\xef\x78\xde\x62\xb9\x98\xa7\xf1\xcf\x90\x91\xba\x52\xfb\x5d\x6a\x07\xf3\xbc\x5e\x4d\x05\x86\xc4\xc1\x43\xdd\x36\xe7\x50\x00\xa5\x35\x70\x77\x45\x32\xa6\x35\xe7\x43\xcc\x4a\x9b\x19\xf0\xc1\x64\x23\xce\x2e\xd7\xb3\xe9\x3d\xc8\x62\x45\x0e\x31\x0f\xd1\xc3\x87\x02\xbd\x6c\x36\xfd\xa4\x8a\xbe\x7e\xf1\x9c\xea\xb9\xf1\xfc\xe2\x56\xfc\xd5\x62\x21\xcf\x82\x2d\x71\x1d\x7d\x71\x9d\xfe\x65\x3e\x89\x57\xe9\x18\xcf\x48\x61\x62\xfb\x4c\x50\x60\xaa\x78\x85\xfb\xf1\x68\xfb\x3d\x85\x42\x8d\xe8\xd1\xda\x33\xf2\xe3\xf9\x3b\x4e\x90\xe1\x42\x4d\xca\xb7\x50\x24\x27\x7d\x2b\x0b\xb7\x12\x31\xb6\x67\x64\xc8\x52\xac\x4f\x41\xe7\x68\x6c\x28\x01\x04\x68\x39\x48\xdd\x86\x33\x44\xe8\xf9\xb6\x62\xc2\x4c\xb3\x2f\x47\xe9\x8f\x37\x73\x15\xae\xdc\x20\xb8\x70\x31\xa7\x89\xfb\xb7\xfc\x29\xd8\x2e\x41\x7e\x1d\x96\x08\x2f\x46\x4b\x3b\x11\xac\xb5\x4c\x78\x19\x4f\xae\x5d\x96\x57\x04\x71\x92\x5f\xe2\xb9\xf1\x32\x0e\x81\x15\x07\x7c\x2d\x8f\xcd\x00\x41\x64\xf2\x37\xef\xe3\x55\x6e\xa7\xe2\x95\xe8\x8a\xfb\x57\x8b\x99\x2b\x34\x78\x97\x49\xdf\x43\xa3\x66\x3f\x5e\xd8\xa9\x09\x83\x24\xc2\xf3\x51\xba\xae\x40\x9a\x1a\xb0\xa2\x94\x52\xb2\x6c\xfe\x84\xdf\x64\xda\xff\x88\xf3\x54\x35\x3f\xde\x98\x27\xc1\x5f\x41\x51\x91\x56\x82\x36\x6e\x73\x15\x67\xa6\x5f\x2c\xce\xdf\x69\xcd\x85\xe7\x07\xc9\x1c\x46\xd8\xf9\x18\x43\x4c\x9f\xf5\x01\xd0\x45\xf5\x74\xa1\x72\x67\xd3\xcd\x0a\x43\x67\x4c\x22\x40\x12\x7c\xb2\xd4\xef\xe2\x8c\xf5\x8c\x68\x6e\x7a\x8d\x6b\x60\xfd\x02\x6c\xcb\xc4\xc4\x2a\x8f\xb3\xe5\x62\xb5\x4e\xfb\x5e\x55\xe2\x6c\x31\xb9\x9e\xc6\x78\x4c\x1a\x7f\xdb\x12\x77\x7f\xdf\x93\x4f\x91\x9b\x10\x9d\xe1\x39\x21\x7a\x6c\x9f\xc1\x4b\x97\x3b\x4c\x74\x26\xce\xda\xbf\x7c\xf3\xf2\xd5\xf7\x3f\xfe\x10\x35\xc2\xf6\x71\xbb\xd3\x20\x2f\x84\xa2\x4b\x9f\xb5\xe5\x8f\xd4\x2c\x47\x10\x82\x96\xfb\xd6\x8b\x15\xea\x3a\xc5\xa3\xc7\x1b\x10\x70\xc5\x69\x0f\x6b\x49\x17\x69\xd9\x9a\xba\xbf\x0f\x50\x4d\x09\xef\x41\xb1\x12\x7d\x2b\xcd\x2b\x11\xd6\x87\xef\x10\x8b\x4f\x08\x00\x81\xa6\x79\x33\x31\x8d\x31\x70\xb0\xc4\x30\xc9\x3d\xfe\x91\x01\x84\x81\x8e\x2a\x81\xc3\x89\xc9\x62\x04\x64\xc0\x23\xcf\x09\x3e\xf8\x50\x80\x34\x26\x3e\x37\x38\x1f\xab\xd3\xd7\x73\xa1\xfe\x9c\xf1\x65\x8b\x88\x2e\xac\x82\x11\x78\xff\x92\x31\x17\x14\x3f\x14\xf2\xa1\x86\x8b\xde\x19\x76\xe5\x08\x63\xf1\x4f\xdd\x5b\xf8\xaa\xdb\x44\x86\xa1\xb9\x9e\xae\xe9\x9e\x2d\xe7\x4e\x0e\xc8\xd1\x97\x69\x5d\x6d\x34\xb8\x9d\xb0\x0c\xd5\x46\x00\xec\x4b\x58\xb9\x6d\xba\x1e\x1d\x21\xc2\xc2\x35\x27\x1c\x1b\x9e\x7a\x2d\x85\xac\xc6\x3e\x43\xa5\x70\x78\xe4\x6a\x19\x94\x32\xe1\x92\x80\xcc\x27\x5d\x63\xb6\x41\xdf\xac\x56\x8b\x55\xd4\x60\x3b\xf5\x00\xfb\x09\x6e\x71\x7f\x30\xc2\xde\xfd\xe0\x26\x59\x5f\x3e\x98\x2f\xa0\x0f\x26\xeb\x64\x34\xe5\xc8\xef\x0d\x10\x1a\xe7\x25\x5d\x9e\x4e\xa6\xf0\x9b\xcc\xdf\xd5\x8b\x6f\x16\xcf\x16\xae\x0c\x25\xc1\x52\x64\xaf\xd3\x4e\x4f\xaa\x3d\xc9\x13\xa3\x4e\x9c\xbf\xc3\xc8\xd6\x8e\x39\x65\xa1\x4a\x1e\x94\x5c\x39\x49\x36\xec\x03\x59\x1a\x5e\x5c\x22\xb9\x00\x26\x69\x9f\x7d\x49\xe6\xa0\x6d\xcc\x4d\xdf\x50\xf5\x1c\xde\xfd\x6e\x45\x0a\x1d\xe7\xdd\xa5\xb5\x14\x75\xa4\x14\x55\xb7\x16\x09\xa8\x94\xb3\xeb\x51\x16\x05\x67\x89\xed\x05\xd8\x92\x7b\xf2\xd5\x5f\x5f\xae\x16\x37\xb4\x5a\x87\x9e\x01\x35\xa6\x67\x35\xac\xf6\xe8\x90\xd2\x46\x9c\xd9\x03\x8f\x6c\xbd\xd5\x7f\x52\xa3\x51\x99\x76\xcb\x31\x13\x6e\xf3\x11\xec\x33\xb5\x21\xd1\xaa\x6f\x48\x37\x59\x06\x40\x2e\xd9\x3a\xa8\x18\x3f\xed\x81\x89\x54\xe6\xb1\xca\x64\xf5\xca\xe6\x6a\xb3\xb3\x82\xc0\x33\x65\xec\x93\xb5\x6a\xb5\x38\xff\xb0\xab\x9e\xc4\xae\x3a\x44\x46\x17\xc9\x0d\x2b\x14\xc9\x49\xfc\x2c\xda\x74\x81\x03\xfc\x59\x7b\x12\xaf\xef\x65\x3b\xf5\x41\xb5\x79\x7e\xb7\xa9\xe3\xcf\x30\xbb\x19\xb9\xbe\xc7\x05\xc8\x3e\x63\x05\x9e\x50\xc6\x8f\xb9\x26\x57\xeb\xac\x74\xf3\xea\xe7\xb0\xf9\xdf\xca\x7b\x16\x69\x08\xe6\x82\xf4\xe8\x4c\xaf\x7d\x93\xf2\xa9\xf6\x7f\x47\x69\xec\xef\x3b\x03\x05\x5f\x9d\x61\x0f\x03\x64\x0f\xb6\xd8\x6d\x2d\x02\x59\x67\x25\xba\xdd\x1a\x6d\x6f\x47\x36\x37\xba\x1c\x64\x88\xfc\x5a\x68\x9b\x91\x7d\x2b\x6a\x8d\x83\x24\x21\xb7\xb7\xb6\x81\x30\x71\xdf\x4c\xc3\x81\x9e\xd7\xb5\x5b\xd7\xf2\xaf\xb9\xd5\x98\x03\xd5\x68\x94\xd0\xd7\xf0\x4f\x1f\xb2\x69\xf7\xa7\x64\x87\x7f\xf6\xf7\x77\x1d\xbb\xfb\xac\xc5\x5d\xe9\xbd\x60\x97\xdd\x63\x1a\x72\x2c\x87\x9e\x84\xfa\x8c\xf3\x00\x10\xdd\x3c\xff\x9c\xa2\x0b\xef\x10\x1d\xce\x3e\x58\x72\x54\xbe\x12\x1c\x82\xfb\x0a\xfa\xe7\xc4\x26\x59\x01\x66\x25\x4f\x3b\xcb\x4d\x0a\xcc\x58\x01\x4b\x6e\xe8\xf4\xcd\xd7\x23\x98\x5f\x90\xff\x32\x9e\x5e\xdb\x41\x2e\x91\x31\x19\x02\xe4\x63\xa9\xfe\x7b\x61\xd7\x99\x5e\x71\xf5\xe5\xc4\x4a\x4b\x40\x26\xf6\x5b\x21\xf9\xf0\x2a\x5d\x45\x16\xe9\x56\x9b\x3f\x5d\x1c\xdf\x7d\x13\x45\x8c\xbe\xa1\xde\x91\xcc\xdf\x2f\xae\x0a\x5c\xce\x68\xa9\x84\xed\x14\xa0\xa6\x11\xcd\x99\x59\x34\x26\x46\x6c\x07\x86\xc8\x14\xe7\x62\x58\xd9\xf4\x5b\xbd\xd2\xc8\x99\x7b\xba\x87\xa3\x5b\x5a\xcf\x94\xc7\x59\xfb\x9c\xad\x4b\x69\x03\x7e\x19\xfa\xf2\xf0\x09\x37\x03\x72\xe2\x33\xd3\xcb\xe9\xf5\xf8\xca\xe5\x19\xc6\x39\xcb\xaa\x6c\x2f\x93\x05\x81\x1e\xfd\x90\xe9\xdd\x5c\xc6\xab\x82\x0c\x30\x3e\x5b\x2a\x2e\x92\x55\x6a\x5b\xab\x24\xfd\x86\x0e\x81\x52\xaa\xdf\xa7\x64\xd5\x95\x06\xc3\xee\xd9\x80\x21\x0d\x1c\xc6\x1a\xdd\x06\x1b\xb7\xc6\xb0\x92\x17\x6b\xaa\x82\x91\x7f\x99\x24\xde\x04\x84\x0f\xc4\x1c\x78\x08\x86\x53\xe9\xc3\xe8\xce\xb3\x17\x30\xef\x58\xd4\x5f\x6b\xf8\xb7\x38\xa7\x2a\x5a\x35\xc3\x71\x5a\xa0\xa0\xb2\x1d\x66\x8d\xda\xf0\xee\xef\x9f\xa9\x89\x3a\xf9\x28\xa4\xa6\x74\x04\x95\xa6\x7b\xf0\xc4\x8a\x2b\xef\x30\x3a\x39\x3e\x3e\x3a\xf6\xed\x4d\xd0\x50\x9e\x6c\x54\x7c\xa5\x39\x54\xaf\x5c\x00\x0b\x19\x13\x65\xe6\x56\x78\x18\xd8\x26\xe4\xe3\x58\xde\x44\xd7\xc5\x14\xee\xa7\xf4\xb8\x51\x7a\xb7\x8b\x2d\xa0\x88\x46\x92\x90\xb6\x00\xfd\x1d\x4d\x01\xab\x6a\x4f\xe5\x3f\x65\xce\xda\xea\x9d\x22\x9e\x31\xaf\xcc\x1d\x13\xd0\x7c\xab\x87\x8d\x19\x3f\xf9\x40\x20\xb5\x4a\x32\xff\xcf\x6d\x95\x64\xfe\x29\xad\xf2\xa0\xb6\x55\x4c\xa3\xfc\x57\xb6\xc9\xd3\xcf\xd9\x26\xe9\xe5\xf5\xc5\xc5\x34\x2e\x2c\x36\xc9\x38\xa4\x13\xe6\x22\x0a\x84\x44\x9b\x58\x8e\x5d\x4d\xb5\x7d\xfc\x5e\x84\xce\x2e\xfe\x2c\x66\x1e\x51\x68\x36\x7d\x4d\x82\x5d\xeb\x56\x38\x8c\x34\x04\x51\x87\xc2\x7d\x55\x57\x91\x69\x30\x71\x3b\x9a\x2d\x0b\xcc\x8a\xb9\x78\x7b\x3d\x5a\x4d\x6a\xa3\x51\x3c\xed\xdc\xde\x12\x06\x0e\xb2\x03\xcd\x97\xd1\x9c\x56\xe8\x83\x8d\x53\x92\x20\x01\xc8\x38\x45\x81\x89\xd1\x1f\x88\xb9\x2f\x7d\x88\x29\x6d\x1d\xfb\x5e\xb9\x03\x35\xc6\xd8\x19\x3d\x38\xa9\xcf\xcd\x53\xb7\xae\x27\x2f\xf5\xdd\xe0\x88\x9b\x2e\x56\xeb\xaf\x0a\xde\x89\x6a\x56\x7b\xa6\xa9\x98\x70\x79\xf2\xdc\x2b\x3b\xe5\xc8\xe2\xd5\x0d\x20\x15\x9e\x84\xa3\x49\x94\xdc\x65\xa4\xf1\x0a\xcb\x4c\x46\xdd\x5d\x9d\x56\x0c\xfc\x0b\x95\x31\x6b\xf1\x18\x9d\x9a\x63\x56\xcb\xa1\x38\x42\x48\x5b\x11\x16\xe7\x11\x25\x6a\x00\x5d\x23\x87\x97\xab\x92\x75\x18\x9d\x9e\xdf\xde\xf2\xd0\x04\x76\x60\xa4\xbb\x2c\x62\x9d\x9f\x8e\xac\xb4\x73\x6d\x64\xd5\x9d\x58\x54\x0e\x6b\x1f\x17\x41\xcf\xa0\x62\x0d\x5e\xaf\x91\xed\xcb\x77\x82\x69\x7e\xcf\xe3\xcb\xd1\xfb\x04\x26\x75\xaa\xa9\xee\x6a\x14\xd5\x19\x37\xda\xbe\xc9\xdb\xa5\xf8\x3e\x40\xe3\x3e\x76\xb7\xb6\xda\x56\xb3\xa2\xe7\xd6\x3b\xb8\x78\x64\xf7\x54\x2d\xa4\xa9\x40\xc7\x42\x14\x26\x3a\xa4\x78\x54\x79\xd0\x3c\xfa\xb5\x3f\xa0\x14\x72\x7d\x04\x6d\xba\x1c\xa5\x56\x82\x2f\xbd\x5d\x1a\xd6\xbb\xd6\x33\xae\x70\x14\x66\x56\x67\x2c\xfa\x5d\xca\xb1\x09\x29\x93\x20\xaf\x04\xdd\x96\xdd\xff\xb8\x95\xbf\x66\xd3\xe1\x30\x24\xa2\xa8\xa7\xf1\x84\xbc\x51\xd3\xfc\xb4\xc0\x27\xb6\xcf\x0e\x64\xfb\x6a\xbb\x5e\xdf\xc4\x66\x19\x45\xf9\x49\xec\xbd\xd6\x2e\xd0\xfa\x62\xba\xb8\x01\x03\x8c\x81\xd4\x23\xe2\x45\xad\xb8\x20\xe4\x14\x12\x7b\x2a\x4c\xe8\x04\xac\xc0\x4d\x93\xc0\xa7\xa7\xd0\x21\x2a\x49\x12\x89\x01\x20\x0f\xfd\xa7\xac\x8f\x48\x1f\xde\x9b\x61\x97\xca\x80\x47\xdd\x5f\x16\x37\x68\x74\xd7\x0b\xfe\xa0\x51\x67\xb7\xfa\xee\x30\xdd\xb7\xdc\x6a\x7c\xef\xd6\x2d\x9f\xf7\x8d\x3d\xb2\xa6\x54\x5d\x19\x41\x80\x57\x8b\xc0\xfd\x24\xbb\xef\xdc\x06\x60\x97\x6f\xe6\x1a\x41\x7d\x41\xe6\xb1\x6b\xad\x44\xa9\xd5\x27\xf2\x36\xc1\xbf\x85\x31\xeb\x32\x1e\xe1\xd0\xb5\x1e\xd9\x73\x07\xd6\x80\xe2\x30\xc3\x05\x53\x9a\x72\x95\x19\x34\x57\xc3\x0d\x8b\x3a\x18\x76\x9d\x79\x06\xd2\xc2\xc1\x84\x67\x29\xbc\xba\x74\x47\x51\x15\xf9\x6d\x4d\x68\x79\x85\x82\x61\xd2\xe5\x13\x7d\xfc\x38\xf5\xb9\xea\xe1\x94\x18\x56\x54\x4a\x0f\x95\x0e\xe6\x5c\x04\xbe\x5c\x40\x21\x09\xc3\x3c\x12\x97\xc1\x27\xab\xc5\xf2\xde\xd5\xae\xa8\xe6\x86\x6c\xc1\x6c\x39\x1a\x17\x2b\x5a\x5e\x8c\x61\x22\x96\xaa\xb1\xb9\xbf\x98\xc2\xd4\xc1\xbe\x63\x3a\x99\x83\xbb\x04\x6e\x07\x14\xbc\xb8\xc1\x7b\x32\xe0\xd5\x30\xc5\x60\x74\x1d\xe5\x3a\x07\x61\xeb\x2e\xe0\xf7\xf9\x8b\xab\x74\x3c\x39\xb7\x20\x24\x35\x95\xe7\x1c\x45\xb7\xc1\x74\x22\x06\xdc\xde\x32\x44\x3a\x33\xca\x85\x90\xe5\xf7\xd1\x9e\xba\x65\x30\x42\x57\x56\x47\x0e\x16\x85\x5a\x74\xf9\xb7\x60\x8d\x65\x22\x9a\xff\x92\x30\x58\x68\x92\x8c\x19\x02\x65\x21\x4e\xaa\x18\x0c\x79\xda\x99\xc0\x34\xf7\x7a\x4b\x7b\x4c\x92\x8b\x0b\x98\xb6\xcd\xc1\xc3\x92\x04\xaa\x26\xe2\x21\xeb\xcd\xf5\x3c\xf9\x47\xc4\x3f\xd7\xa5\x4e\x99\xa4\xaf\xc8\x66\x57\xd8\x66\xc7\xff\x52\x78\x18\x9b\x4a\x62\x68\x93\x6b\x46\x69\x43\x4d\x3e\x44\x7b\xa1\xdf\xb3\x57\xf4\x2d\x0b\x8f\xa6\x4b\x32\x51\x5a\xb2\x61\xb8\x59\x0b\x15\x29\x5e\x62\x5e\x70\x9d\x25\xcd\x9a\xe1\x5d\xb3\xdc\x27\xc0\xfe\x3e\x92\x18\xe0\x1f\xd3\x07\x23\xe9\x52\x74\xcd\x0a\x8d\x87\x18\x52\x15\x30\xa4\x13\x65\xb0\x5a\x5b\x38\x2b\x9d\xdc\xb3\x79\xa9\xdb\xfe\xf0\x25\xe5\xee\xde\xdf\xaa\x9b\x0f\x9b\xc2\xd3\xaa\x62\xb5\xd8\x9e\xec\xea\xb4\x05\x38\xe5\x6b\x6c\x4b\x5a\x20\x3d\xa4\x75\xf5\xea\x4b\x68\x39\xad\xb2\xdb\xca\x02\x39\xbb\x91\x16\x48\x7d\x66\x31\xc5\x9d\x11\x09\x1b\x94\xc5\xfa\x32\x5e\xd9\x0e\xb9\x5c\x57\x22\xb8\x20\x02\xa7\x51\xb0\x91\xab\x2f\x46\x2b\xeb\x79\x76\xfa\xb6\xd9\x05\x50\xab\xc0\xa5\xca\xb0\x66\x54\x4f\x19\xf6\xac\x66\xa4\x8a\xa8\x0e\x8a\xdc\x7d\x48\x96\x55\x97\xe9\xca\xe1\x8e\x56\x3c\x3c\xe5\xe7\x1b\x16\x1a\x9c\xde\xf0\xe5\x3e\x10\x2f\xf0\x75\x23\xd3\x51\x0e\xb6\x38\xf2\x7b\x0d\x7e\x4d\x76\x3f\x1d\x4b\xd4\x41\x32\x8c\x2a\x88\x37\x9a\x89\xae\xa1\xa5\x39\x8b\x73\x77\xa9\x1c\xe7\x01\x5c\x97\x54\x7f\xac\x8f\x22\x9a\x1d\xc8\xc9\xc6\x46\x7b\x43\x96\xef\x6c\xbe\x6c\x23\x6a\xe5\x97\x6d\x26\xaa\xfc\x3a\x44\xc3\x4f\xd9\xdc\x29\x90\xe9\xae\x9b\x02\x63\xf1\x30\x52\xcf\xe1\xd0\x65\x7d\x23\xb4\x82\x94\xac\x0c\x68\x8a\xb6\x09\x56\x15\x58\x3b\xd4\x2c\xa3\x57\xd8\x08\xe0\x38\x6b\xf8\x0d\xde\x26\xd0\xe0\xe3\x49\x66\x63\x85\x4e\x55\x1f\x72\x22\xc7\x1f\xb5\x38\xf1\xa5\x0f\x97\x60\xff\x47\x40\x3f\xe9\x42\xe9\x50\xf2\xa9\xb6\x19\xd6\xb4\x95\x19\x68\x6a\xfa\x5d\xf5\xb4\xd1\x5b\x25\xf4\xe2\x2c\xb3\x5c\xb3\x3c\xeb\xeb\xf9\xb5\x85\xe4\xb9\xb2\xa1\x76\x2c\xed\x3e\x28\xf0\xab\xab\xd8\xd3\x13\x34\xe9\xab\x7c\xbf\x45\xfe\x17\xab\xc5\x6c\xbb\xec\xc1\xdd\x67\xdf\x9d\x77\x83\x60\x86\x9e\xae\xa3\xb5\x7d\x47\xd5\xd3\x2e\x51\xd5\xd5\x42\x53\x6c\x2a\xb2\xfd\x52\x2e\xcf\x30\xd6\xad\x4e\x34\x6e\x7e\x12\x69\x3a\x98\xa1\xeb\x6a\x47\xab\xd5\xdb\x59\x50\x7c\x0f\x40\xfd\x76\xdc\xd2\xfa\x47\x84\xa1\x04\x11\x21\x22\xdc\xdb\xdb\x40\xd0\x43\x14\xf8\x94\xc5\x7c\xa7\x1e\x74\x86\xb7\xb7\x61\xaf\x60\x5c\xb4\x32\xd1\xc3\x38\x4e\xa6\x1e\x91\x6b\xf1\xf5\x4b\x87\x54\x2c\xde\x4e\x98\x4c\x70\xa9\x88\xf9\x2b\x19\x17\xad\x15\x93\xac\xe7\xf3\x0d\x05\xf0\xd8\x6c\x0e\x23\x55\x03\xf8\xdb\x8c\x90\x98\xee\x98\x88\xc5\x4e\xda\xd8\x59\x69\xf1\x3f\xa2\xaf\x82\x5f\xb8\x0d\x0c\x1f\xdc\x79\x38\xae\xac\x8b\x73\xbc\x72\xc6\xa8\x01\xee\xb4\xda\xdf\x47\x5c\xce\xae\xda\x1d\xe1\x4a\xdc\x06\x22\x0d\x3d\x91\xae\x35\xf1\xb8\xa2\xe8\x38\x1b\x88\xee\x57\x7d\x22\xee\x99\x15\xaa\xfa\x8f\x0f\xc4\xb0\x5d\x55\xa4\xef\x15\x77\x78\x11\x96\x5e\xb0\xa4\xfa\x30\xaf\x66\x9e\xf7\x36\x55\x46\xbf\xaa\x28\x1f\x38\x47\xa1\x5a\xb1\xfc\x88\x8a\x7e\x25\xb1\xa7\xf1\xf4\x82\x9a\x12\x51\x8b\xf8\x74\xc5\xa0\x65\xb4\x2d\x2e\x30\xdf\x4e\x2c\xc8\x0a\xf0\x96\x38\x39\x59\xc7\xed\x47\xfc\xa4\xbe\x9f\x21\xb9\x0d\x7d\x21\x01\x1d\x71\x26\x4c\x24\xeb\x3b\xbe\xe8\x18\x9f\xa2\xec\xcd\x58\x3c\xab\xf8\xd6\x77\xf1\x4c\x8c\xa0\xf2\x7c\x59\xf8\xf2\xca\x7c\x20\xa4\x9e\x11\x68\x4c\x5c\x9b\x22\x24\xb5\x8b\xcc\x68\x0a\x6f\x24\x68\x48\xe2\x0f\x66\xd7\xe9\xfa\xc1\x79\xfc\x60\x49\x61\xfa\x35\xf7\xa0\xa0\xb3\x38\x6d\xb8\x0b\x45\x44\xd1\x38\x16\x17\xfe\x47\x5c\x51\xbc\x18\xaa\x9d\x20\xfc\x46\x2b\x0a\xe8\xf1\x9f\xbf\xa3\x05\xf9\x78\xb6\x70\x66\xd6\xa4\xe9\x60\xa8\xc8\x6f\xa2\x55\x05\xdc\x0f\xf1\x51\x2f\x75\x73\x12\xf8\x95\xfc\x60\x7f\x0a\x2d\xee\x9b\xc5\xb5\x29\xc6\x2a\xc8\x57\x0a\xd2\xb8\x46\xb8\x4c\x43\x9b\x32\x68\x91\x06\x9f\x78\x09\x49\x3f\x45\x15\xad\xa4\xe3\xb5\xa1\xe7\x86\x77\xe9\x17\x2a\x71\x33\x4a\xd6\x77\x7f\xea\x53\x4c\xc8\xd3\x76\x30\x6b\xf1\xb6\xea\x08\x05\xc8\xe6\x2f\x76\x5c\x02\x17\x0f\x7e\x63\x51\x25\xcd\x64\x07\x99\x93\xd9\xcf\xc4\x80\x98\x0b\x87\xdb\x74\x0c\xad\x0a\x11\x46\xc5\x58\xaf\xa7\xc5\xf6\xc1\x82\xc5\x62\x89\x80\x54\x7d\x59\x30\x9d\x5e\xba\x77\x62\xcd\xf5\xe1\x98\xde\x6a\xeb\x32\xf8\x77\x32\x23\xb4\xa1\x7c\x82\xe6\x95\x9b\x7a\x46\xeb\xd8\x31\xb2\x3a\x97\xc4\x04\x85\x1d\xd1\x55\x88\x11\x4c\x8c\xfa\x41\x57\x9d\x05\x76\xcb\x2a\x5b\x03\x9b\x3d\x7f\x53\xd1\x19\xe9\xf6\xca\xc5\x8d\x3e\x5c\xdc\x53\x25\xdf\xde\x16\x8a\xde\xc3\xa2\x81\x77\xcd\x1a\x64\x53\x5f\xe1\x31\x16\x2d\x72\x87\xf2\x69\x79\x90\xd0\x52\x58\xba\x9d\xd5\xdc\x4f\xf7\x75\xeb\x72\xd0\xe0\x34\xd2\x34\xfa\x1e\xc5\xcc\x55\x3a\x21\xeb\xe7\xd7\x08\x15\x8a\xba\xab\xd6\x78\x6b\x24\x65\x35\x35\x5a\xaf\x46\xc9\x54\x49\x13\xaa\xa4\x68\x5b\xba\x48\x2d\x22\x34\x57\xbe\x59\xa9\x25\xb5\xc3\x41\x60\x5c\xa9\x1d\xc9\x6c\x16\x4f\x12\xe7\x40\x2b\xd0\x63\xfd\x50\x7c\x21\x10\xc6\x93\xd9\x52\xb8\x9b\x8d\xac\x76\xd9\x26\x2e\x9d\xdd\xb4\x5a\xa5\x12\x31\x10\xe6\x4e\x0a\xad\xa5\x73\xf6\x90\xd7\xff\x97\xb6\x63\x59\x72\x1b\x37\x9e\x93\xaf\x58\x21\x59\x89\x0c\xe9\x91\xc6\x97\xc4\xa2\x69\x55\x65\x2a\xa9\xda\x43\x7c\x88\x8f\x92\xa2\x22\x67\xe8\x59\x6e\x34\x92\x46\x8f\xc4\xae\xa1\xfe\x3d\xfd\xc4\x83\x0f\x67\x6a\x9d\x5c\x24\x10\xcf\x46\xa3\xd1\xe8\x6e\x34\x80\x0f\x98\xb8\x18\xec\x3c\x0d\x26\x66\x89\xe7\x51\x80\x7d\xdb\x45\xe7\xed\x30\x88\x7c\x74\x9d\x81\x29\xf6\x11\x48\xcc\x16\x1b\x8f\x47\x52\x9f\x76\xdc\x8e\xd0\xf0\x50\xd0\x9c\x8f\xb5\x32\xb7\xf1\x36\xd8\xb2\x3f\x60\xfb\xce\x60\x39\x1e\x8b\x52\x93\xe7\x41\xd2\x65\x42\x90\x4e\xac\x71\x1e\x51\xce\x19\xb9\xb1\x7d\x83\x31\x12\x3b\x66\x4c\x91\x43\x1f\x41\x80\xee\xe0\x6d\x72\x21\x0f\xf1\x9e\xcd\x0d\x8f\x79\x12\xd3\xb2\xea\x92\x67\x79\x22\x7a\xf2\xda\x94\xca\xda\x0b\x2a\x37\x8e\x66\xba\xfd\xa9\x6a\xd3\x06\xaf\x97\xb6\x92\x1e\x0c\xa8\x38\xda\x22\xc0\x3a\x58\x45\x51\x03\x02\x65\x1e\x45\xe9\xd8\x01\x8d\x4a\x60\x07\x9a\xb5\x2f\x87\x81\x4e\x48\xd0\x15\x9f\x03\xba\x25\x1a\x4d\x03\x56\xde\x19\x93\x37\x4c\xc9\xef\x6f\x17\xc3\xc3\xa0\x0e\x35\xd8\x04\x79\x42\xba\x73\x00\x4d\x13\x0a\x0f\xec\x68\x0e\x0c\x4e\x84\x21\xda\x91\xee\x73\x34\x34\x3f\xed\x40\xc3\x85\x6a\x59\xd1\x96\x47\x3f\xa9\xf6\xe5\x3a\x6b\xf9\x5f\x60\x2d\xbc\xc4\xaa\x3b\xc9\x78\x4c\xae\x97\x64\x00\xc2\x6f\xc5\x06\xc6\x22\x98\xff\x6a\xbd\x82\x4c\xc0\x0d\x3a\xa0\x77\xbd\x39\x45\xd5\x7f\x8d\x79\xc1\x2a\xea\xb9\xef\x8a\x9e\xf9\xee\x2c\x27\x16\xfb\xea\xe3\xf7\x40\xc4\xe5\x5f\x01\x10\x65\x44\x78\x96\x9e\x3b\xbc\x05\xcc\x4d\x01\xcc\x26\x6e\x46\xd5\xf1\x3c\x00\x99\xb3\x65\xfc\x2a\xb7\x7d\xb1\x5d\x04\x00\xe4\x12\xea\x9a\x2e\x14\x04\x6c\xe7\x49\x0e\x51\x76\x85\x53\x92\x1c\x07\xa9\xc4\xd3\x61\xa4\x55\xbc\xcc\x9f\xca\x74\xc9\x85\xa3\x69\x6b\x97\xfd\x11\xf1\xae\xf6\x87\xfe\x4d\x1a\x92\x50\x07\x44\x1d\x27\x37\x9e\xf6\x97\xe3\x7d\x45\x13\x41\x82\x0a\x25\xe8\x1c\x07\x04\x53\xa2\x11\x38\x8c\x02\xd5\x91\x22\xf8\xc3\x89\xb5\x87\xba\xe5\x46\xa5\x52\xd2\xe1\xab\x1d\x8d\x5f\x6b\xc9\xa3\xae\xd0\xd1\x1d\xdb\x00\xed\x36\x3a\x44\x92\x89\xf9\xf0\x95\x25\x57\x8b\x47\x80\x0e\x63\x69\x1d\x78\xaa\xdb\x24\xf3\xfd\xe0\xf5\x0e\xa8\x35\x27\x12\xc0\x08\x27\xbe\xde\xd8\x05\xce\x89\x47\x0c\xe1\x03\xbf\x3d\xd9\xa6\xa0\xff\xcf\x68\xda\x0d\x7b\x3b\xac\xec\x18\xdf\x33\xc4\x9e\xea\x72\xbf\xc5\x0b\x3a\x7a\xe1\x43\x42\xf6\x78\x68\x7b\x7b\x92\x5c\x2d\xf9\x19\xaa\xb9\x92\x6d\x04\x78\xd7\xbd\x4a\x92\xbc\xfd\x75\x92\xf6\x7e\xd1\xa4\x7d\x5f\x1d\xce\x9e\x1f\x80\x17\xc7\xb3\x1a\x0b\xf3\x31\xa6\x67\xcf\x84\x95\x96\x69\xf1\xe9\x5c\xdc\xff\x33\x2d\xe9\x8f\xdd\x18\x3c\x07\x85\x1f\x66\xc0\xf4\x8b\xa6\xb9\x9d\x42\xec\xed\xb4\x74\x87\x04\x20\x92\x03\x36\x2b\x95\xcb\x8a\xe0\xb0\x17\xa0\xab\xc8\x0b\x7b\xd8\x2b\x4e\xcb\x76\x72\x99\x97\x2e\x99\xed\x29\xf8\xf4\xdb\x47\x98\xc2\xb9\x9e\x03\x94\xb1\x24\x15\xd5\xa6\x8e\x5a\xc9\x65\xec\xfc\xf5\x4e\xff\xae\xf1\x7d\x2b\x9b\x17\xc4\xc5\xe2\x54\x99\x25\xaf\x4a\x3f\x70\xb1\xb5\x99\x3b\xc8\x39\x0a\x2a\xc9\x82\x9c\x7c\xa7\x94\x97\x73\x94\x27\xc5\xa2\x84\xdf\x72\x0e\x74\x51\x2c\x14\x31\x73\xf8\x4b\xca\xb0\x30\x4a\x94\x50\x34\x88\xfb\xf3\x7e\x0f\xd2\xfb\xce\xd6\x98\xf4\x94\xfb\x7b\xf5\xf8\x97\x2f\x07\xaf\xd1\x1b\x26\x34\x40\xaf\x84\xc6\xe3\xe2\xe6\x71\xbb\x2f\x0b\xc4\xbf\x84\x30\xee\x09\x66\x06\x0a\xee\x94\xd5\x7e\x60\x4a\xfd\xb8\xdb\x1f\x2b\xbc\x06\x17\x93\xdc\x17\x5a\x5a\x8d\x2c\xd6\xd6\xe0\x0b\x63\xdb\x89\x2b\x1d\x76\x5b\x36\x38\xa6\xa0\x70\x9d\x50\xa3\x21\x25\x2d\xe5\x28\x84\xe7\x13\x53\xb6\x12\x4a\xf6\x80\xbe\x43\x53\x5a\x81\xbc\xe1\x74\x3e\x5e\xd0\xa6\x93\x96\x14\x57\xfa\x71\xe4\x75\x83\xd1\xe8\x79\x73\x47\xce\x72\xa3\x28\x58\x22\x28\x15\xd6\x07\xfa\xf7\x29\xae\xb8\xb3\xbe\x75\x36\x73\x29\x99\xcb\x76\x66\x4e\x70\xfd\x96\x8e\xf2\xf6\x54\x2c\x53\x86\x3f\x4b\x26\x5d\xf2\x05\x98\xa5\xce\x61\x1d\xb1\xab\xc3\x4a\xf3\x7c\x6d\xf2\xdc\x23\x4b\xe4\x40\x58\xa6\x50\x71\x40\x8a\x52\x24\xf4\x5a\xa4\x01\x32\x5d\x63\xdc\x9b\x37\x4e\xb0\xaf\x9e\xa3\x62\x89\x91\xeb\xb4\x94\xff\x70\x32\xc7\x59\x9c\xd1\x19\xbd\x8e\xe3\x2b\x8e\x0d\x4b\x5e\x85\xc8\x5d\x04\x46\x92\xa4\x23\xad\x9d\x93\x4b\x49\xa6\xb6\x90\x1f\x43\x53\xf4\xd7\x6a\x29\x8e\xc9\x4b\x3b\xe3\xe7\x00\xd1\x50\x46\x6d\x4a\x7b\xa5\x6b\x4f\x2b\x1c\x71\x6f\xa4\x98\x22\x0c\x23\xaf\xea\xe2\xa1\xe8\xde\x1f\x22\x87\x6c\xfa\x10\xd1\x22\x23\x17\xc9\xe7\x4b\xe0\xa5\x90\x96\x6e\x31\x78\x26\x0e\xb7\x5c\xeb\xee\xaf\x38\x54\xe6\x1d\xe1\xd6\x7a\x6a\xe8\x60\xd3\xc8\x85\xcc\x99\x37\xbd\x85\x4f\xf8\x2e\x99\xb2\x3e\xe8\x04\xe8\x59\xed\x6c\xef\x55\xce\xed\xf1\x2c\x66\xf0\xb6\xd5\x53\xf0\x44\x94\xb7\x76\x8c\xa2\x11\x7c\x01\x2b\x1e\x71\x73\xbb\xfd\x43\x85\x82\xb7\xf4\x8c\xfd\x60\x82\x63\xbe\x6d\x29\x9e\xeb\xe9\xd2\x63\xc8\x47\xc9\x0c\x97\xba\x85\x6a\xd0\xb7\x26\x0f\xd4\x81\x6b\x4a\x6b\xef\xd2\x58\xb7\x00\xbc\xe9\x5e\x8a\xe2\x55\xf6\xd4\x04\xde\x63\x7f\xd1\x4b\xf3\x81\x3d\xc2\x1f\x73\x3b\xb3\x76\xcb\xf3\x8e\xa6\xc6\x66\x69\xea\x93\x49\xf0\x63\xdd\x0f\x43\x17\xee\x3c\xb7\xbd\xe3\x92\x89\x59\x9b\x2b\x3b\x75\x79\xfe\x0a\x4e\x07\x02\x99\x23\x48\xfa\x26\xe6\x47\x6e\x10\x0d\x36\x59\x55\x86\x76\xa6\x8d\x16\xb2\xdc\x72\x7a\x33\x85\x29\xe5\x73\x99\xde\x8a\x5d\xc1\x5c\xd9\x2c\xae\xd3\x0c\xee\x5f\x71\x13\x7e\x40\x8c\xd0\x54\xf1\x3d\x1e\xf1\x0d\x77\x7c\x43\xe1\x76\x5f\x88\x82\xc6\xa3\x08\x29\xc3\xb2\x08\x0f\x86\x73\x61\x1e\x91\x6f\xd2\x6f\xb9\xa4\xac\x55\xdf\x22\x80\xd1\xac\x69\x24\x74\x0b\x8b\x46\x67\x95\x1b\x26\xae\x8f\x97\x8e\x89\x3a\xf0\xfa\xc9\x45\xdc\xa9\xe9\x4e\x03\x3a\xea\xdd\x9f\xdd\x97\xd1\xb0\x00\x0c\xd1\xf0\x39\x85\xf0\xe2\x00\x0b\x13\xe5\xc1\xc2\xbb\xfd\xdd\x7e\xf7\x19\x24\xb0\x73\x8f\x7b\x81\x1c\x10\xef\xde\xb5\x90\xa2\x66\x7d\xf5\xdc\x77\x86\x1c\x6f\xad\x6b\x33\x69\xe7\x2e\xd7\xae\xc7\x49\xc4\x9a\x16\xee\xef\x2f\x4f\x7c\x62\x3f\x0a\x5d\x7e\x49\x47\xdc\x89\x4a\x46\xd9\x50\x3d\xec\xf7\xad\x73\xbb\xe1\x94\x51\x36\xeb\x1e\xf6\x4f\x0e\x08\x79\xcc\xbb\x35\x0e\x10\x03\xa4\xec\x3f\x43\x3d\xa3\x37\xaa\x13\xef\xa6\x37\x0a\x8a\xef\x72\xfc\x07\xcc\x8c\x8f\x59\x27\xb7\xea\x95\xcc\x48\xc1\xeb\x17\x5e\xaa\xd3\x7d\x71\xa8\xe6\x2f\x66\x6c\xe6\x66\x8c\x86\x36\x60\x01\xef\x31\xbc\x3d\x63\xf0\x03\x06\x1f\x31\x38\x31\x13\x08\x3e\x5f\xf6\x14\x3f\xc1\xf8\xdf\x7d\x79\xfb\xc7\x0c\xe6\x73\x66\x2b\xbc\xb9\xec\xb8\xc6\x5c\x15\xde\xc8\xa5\x71\x4a\xec\x81\x00\xbc\xa6\xfa\x02\x88\x57\x30\x50\xed\x66\xfe\x03\xcb\xb4\x49\x44\x0d\xee\xd4\x70\xf3\xcb\xbe\xde\x45\xc6\xc4\xc8\x4e\x52\xf3\x68\xe2\x54\xdb\x0d\xaa\x88\x7a\xaa\xd0\x8c\x5a\x49\x83\xb5\xc4\x5c\x0b\xae\x60\xc2\x35\x39\x13\x44\x6b\x7e\x9f\x21\xea\xe9\xa0\x8d\x1e\xd3\xe9\x5e\x82\x19\x8c\x19\x47\x2e\x8c\x99\x03\xd0\x89\x64\xb9\x39\x56\x87\x6d\x01\xaa\x45\x80\x0a\xad\xd1\x6b\x0c\x9f\x86\x75\x6b\xa8\x76\x44\x33\x2e\x29\x7d\x4d\x6f\x65\xb2\xeb\x9c\x1a\x19\x75\xc2\x01\x03\x48\x0f\x32\xc3\xc2\xd5\x15\x52\xe2\x60\xd6\x66\xce\xcb\x94\x93\x97\x5a\x70\x9d\xfd\x37\x5f\x75\x3b\x79\xb1\xd6\xb9\x77\x90\xe3\x4b\xdd\x66\x59\x84\x63\xcf\x0e\xc1\x2a\x51\x6b\xbd\x51\x83\x5f\xbe\x59\xf2\x9a\x83\x7a\x1e\x85\x60\x94\xec\xce\xe2\xb2\xb5\x1e\xf9\xa6\xc8\xe0\x5e\x8b\xd7\xd8\x24\xf5\xb0\x04\x76\x84\x8c\x73\x9e\xb1\x6e\x23\x56\x5a\x40\x33\x4f\xa2\xfa\xe1\x0e\x1d\x8b\xab\x23\x4c\x7a\xf5\x39\xfb\xc9\xe3\x89\xc0\x96\x3e\xd7\xe2\x7c\x5d\x3f\xe4\x49\x62\x0b\x24\xc6\x58\x60\x28\xd3\x82\xff\x20\xc7\xbc\xa6\xb3\x0b\xe7\xea\xe9\x80\x56\xe4\x4f\xd5\xf9\x0c\xb4\x82\x13\x04\x11\x8a\xef\x63\x4d\xdf\xff\x18\x2d\x57\xa7\xd5\xa7\x75\xb2\x88\x7f\xfc\x30\x7d\x4c\xbd\x07\x5d\x31\x35\x6f\x25\xcb\xbc\x80\x94\x37\x61\x0a\x77\x63\xb7\xff\x1b\x12\x50\x3e\x8d\x6e\xe2\x7f\x4c\x25\x37\x5e\x5e\x8f\x33\x7c\x02\x13\x60\xb5\x82\x00\xfc\x40\xe8\x08\x21\x94\x14\x56\x3b\x08\xa0\x10\xf1\x1b\xf8\xc7\xb7\x70\x56\x97\xb7\xb3\xb7\x7f\x82\x0f\xfe\x97\x88\x77\x12\xf1\xce\x5c\xa5\xde\x63\x3e\x5d\xad\x9a\x49\xb3\x3a\x36\xab\x5d\xb3\x3a\x37\x5c\x90\xff\xde\x4d\x1f\x33\xd7\x75\xcf\xbe\x8a\xfc\xf2\xa1\x38\x17\xf8\x6a\x13\xa1\x43\xbd\xac\x90\xe1\x67\x1a\x99\x3b\x5b\x04\xaa\xe8\x1a\xdd\x83\x4d\xe6\x40\x34\x6f\x00\x22\x8f\x61\x2c\x23\x2d\x25\x9c\x06\x74\x6a\x46\x4f\x2c\x8a\x5e\xea\x72\x78\x68\xff\x56\x36\x1d\xb8\x4e\x9e\xb5\xcf\x80\x9a\xdf\x0b\x1f\xb3\x87\x68\x58\xc3\x34\x9b\xcd\x21\xc9\x27\x26\x43\x24\x58\x8e\x21\xa0\xb7\x78\x84\xe0\xd8\xa7\x87\x54\x5b\xb7\x17\x62\xea\x26\x26\x55\x9f\xe4\x54\x2d\x5b\x38\xe4\x00\x00\xe7\x73\xcc\x89\xc7\x6d\x80\x1d\x21\x5d\x24\x42\x32\x96\x11\x09\x1c\xa8\xc3\x48\x2b\x66\x92\xac\x76\x51\xb4\xd9\x9c\xf3\x48\xf3\x03\xd3\x05\xf1\x10\x59\xd1\x62\x32\x41\xe3\x0a\xc5\x62\x9e\x38\x86\xdc\x13\xc2\x85\xed\xc8\x60\x65\x5e\x9e\x56\x8d\x50\x91\xd4\xa3\x48\x08\x2a\xc9\x80\x88\x13\x4d\x49\x80\xa4\x05\xd3\x3a\x04\x8c\x88\x84\x3a\xa5\xca\x20\x7d\xe0\x31\xa3\xa0\x16\x4b\x6a\x37\x40\x56\x75\x51\x6e\x71\x9b\x49\xc7\x0f\x1d\x5c\x23\x92\x56\x5f\xae\xf1\x0b\xb6\x29\x85\xcd\x15\x3e\x62\x3b\xce\x48\x91\x00\x72\x0a\x50\xe4\x93\x09\xfc\xfd\xd2\xbe\x13\x8a\xc8\x05\x18\x79\xbd\x0b\x64\x21\x02\x1b\xb2\xb7\xed\x6b\x93\x09\x28\x9b\x99\xdf\xa0\x32\xf0\xcd\x01\xa3\xb3\xf3\xf1\xeb\x0b\x4f\x21\x9a\x02\x96\xa7\xf7\xf4\x06\x8d\x0f\x40\x9f\x1b\x23\xe0\xc6\xd7\x7b\x7a\xaf\x1c\xb8\x34\x6f\x3a\x54\x6a\x11\x11\xf2\x27\x33\x06\x4e\xd9\xd8\x1a\x9f\xb1\x21\x8a\x4a\x37\x3c\xff\xba\x13\x9d\x0a\xbc\x04\x25\x3c\x5e\x2c\x65\xed\x9e\xb0\x96\xd7\xa6\xad\xf0\x0e\x54\x31\xd8\x09\x5c\xeb\xc3\x51\x30\xa9\x56\x44\xa6\xc2\x9f\x8b\xce\x6a\xa5\x78\xe3\x83\x00\x94\x23\xb2\xd7\xbd\x74\x56\xdb\xd6\x13\x87\x94\x7d\x11\x96\x25\xcb\x61\x26\x4b\x63\x84\xd7\x1e\xb1\xcc\x01\xca\x34\x60\xf9\xc0\xef\xbd\x1f\xd1\x4b\x95\x9f\xf9\xfc\xb9\xfe\x8c\x4c\x16\xed\xe7\xf8\x87\xd7\xa5\xb1\x64\xc2\x29\x1d\x4d\x8d\x37\x0a\x51\x3a\xf0\x6e\x9f\x7a\xed\xd2\x89\xda\x43\xb0\x70\xba\xfb\x29\xb0\x46\xf5\x23\xc7\x53\xb4\x6e\xe1\x14\x18\x47\x39\x56\x39\x1e\x2b\x8c\xfc\xdd\x34\xb3\x91\xaf\x91\x37\xcd\x43\xb5\xad\xce\x15\x9d\x63\x9b\xad\xbb\xab\x2e\x29\x25\x57\x8b\x16\xb6\x6f\x43\x87\x71\x02\x20\x02\xa8\xee\xff\x5d\xaf\x83\xfd\x11\x0f\x8c\xa0\xc3\x01\x4a\xbc\xae\x8b\xd8\x25\xa6\x61\xaf\x9d\xf4\x85\x46\x7b\xf0\xed\x4b\xa6\xb4\xd1\x4c\x74\x95\xd6\x71\xbf\x56\x66\x69\x17\x65\x0f\x07\x60\x9c\xfd\x27\x00\x00\xff\xff\xf9\xf2\x2c\xdf\xd7\xe3\x08\x00" + // scripts_stats_modules_js returns raw, uncompressed file data. func scripts_stats_modules_js() []byte { - gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ -0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0xdc,0xbd, -0xeb,0x76,0x1b,0x47,0xd2,0x20,0xf8,0x7f,0x9e,0x82,0xa8,0xf6, -0xc0,0x55,0x42,0x12,0x22,0x64,0xbb,0xe7,0xeb,0x82,0x8b,0x18, -0x59,0x92,0x3f,0x6b,0xc7,0xb7,0xcf,0xd2,0xd7,0x97,0x05,0x21, -0x1d,0x90,0x28,0x8a,0x25,0x81,0x00,0x8c,0x02,0x45,0xc9,0x04, -0xfa,0x59,0xf6,0x3d,0xf6,0xc7,0x9e,0xb3,0x2f,0xb4,0xaf,0xb0, -0x71,0xcd,0x5b,0x15,0x20,0xc9,0xee,0x99,0xb3,0x67,0xbb,0x8f, -0x45,0x54,0x5e,0x23,0x23,0x23,0x23,0x23,0x23,0x23,0x22,0x3b, -0x97,0x37,0x8b,0x8b,0x4d,0xb5,0x5c,0xa4,0xb7,0xd5,0x62,0xb6, -0xbc,0x35,0x37,0x8b,0x59,0x79,0x59,0x2d,0xca,0x59,0x76,0xa7, -0x59,0x47,0x55,0xfd,0x70,0xbd,0x9e,0xbe,0x9f,0x57,0x6f,0xca, -0x74,0x79,0xfe,0x3a,0xbb,0x7b,0x3b,0x5d,0x1f,0xcd,0xcb,0xc5, -0xab,0xcd,0x55,0x01,0xdf,0x7d,0xfe,0x69,0x36,0xef,0x57,0x65, -0xf1,0xfa,0x3f,0x6e,0xca,0xf5,0xfb,0x3e,0xfe,0xa6,0xb2,0xc3, -0x75,0xb9,0xb9,0x59,0x2f,0x8e,0x24,0xbd,0xaa,0xff,0x46,0xfd, -0x50,0xde,0xa8,0x33,0xc8,0x07,0x45,0x41,0x6d,0x2c,0x96,0xb3, -0xf2,0x39,0x54,0xea,0x76,0xb9,0xb5,0x51,0xe7,0x24,0x4f,0xa6, -0xd8,0x6d,0x02,0x25,0xb0,0xb9,0xed,0x36,0x51,0x88,0x92,0x0e, -0x27,0x75,0xbb,0xe9,0x09,0xe4,0x72,0x0d,0xc8,0x5f,0xdc,0x5c, -0x9f,0x97,0xeb,0x84,0x33,0x97,0x97,0x02,0xa3,0x36,0x79,0x7a, -0xa2,0xbf,0x8e,0x07,0x47,0xd5,0xe2,0x08,0x41,0xd8,0xd9,0x41, -0x5e,0xac,0xcb,0xe9,0xa6,0xfc,0x69,0x85,0x1f,0x75,0xba,0xe4, -0xbf,0x3c,0x54,0x28,0x58,0x5e,0x6c,0x0a,0x49,0x7b,0x34,0xbd, -0xb8,0x2a,0xc7,0xf2,0x31,0x29,0xee,0x76,0xd1,0x10,0x4b,0xc8, -0xd7,0xfa,0xfd,0xeb,0xe9,0x06,0xbe,0x2e,0x96,0xeb,0xf2,0xe5, -0x7a,0xb1,0xdc,0xdc,0x5e,0x55,0x9b,0x32,0xdb,0x6e,0xc7,0x13, -0x63,0x11,0xff,0xd2,0x5c,0xce,0xa7,0xaf,0xb2,0x3b,0xee,0x65, -0x8c,0x1f,0x93,0xa2,0x73,0xb2,0xcb,0x0c,0xa7,0x38,0x10,0xab, -0xc5,0xa6,0x5c,0x2f,0xa6,0xf3,0xc7,0xd3,0xcd,0x34,0x2d,0xe7, -0xe5,0xb5,0x59,0x4c,0xaf,0x4b,0x33,0x83,0x4f,0xb3,0x7a,0xbb, -0xc9,0xee,0xaa,0xcb,0x54,0x80,0x98,0x5e,0x5c,0x94,0xab,0x8d, -0x2d,0x98,0xf1,0x40,0x36,0x57,0x15,0x83,0x6f,0x00,0x64,0xa3, -0xcd,0xfd,0x8f,0xf2,0xbd,0x4e,0x5b,0xf9,0x6e,0x35,0x85,0xe9, -0x31,0xaf,0xca,0xcd,0x37,0xef,0x7f,0x84,0xc6,0x8b,0xa4,0xde, -0xac,0xab,0xc5,0x2b,0x87,0x53,0xea,0xb2,0xaa,0x7f,0x84,0xf9, -0x2a,0xb0,0x69,0x3b,0x73,0xe6,0x02,0x5b,0x2e,0x38,0x6b,0x24, -0x0d,0x52,0x5a,0x4e,0xb0,0x56,0x33,0xcd,0xc3,0xcf,0xb1,0xd7, -0xfb,0x24,0x6f,0xa4,0x74,0xbb,0xde,0xd7,0x10,0xc6,0x55,0xcd, -0xba,0x5d,0x6a,0x6c,0x5c,0xcd,0x20,0x37,0x85,0xf1,0x6e,0xb7, -0x36,0xa1,0x8f,0x38,0x00,0xbc,0x76,0x2c,0xe0,0xdb,0x2d,0x26, -0x01,0x9d,0x38,0x82,0x96,0x69,0xaa,0x66,0xdb,0x6d,0xba,0x0f, -0x92,0x02,0xa0,0xa4,0xd9,0x9a,0x41,0xd6,0xa6,0x9c,0x3d,0x9d, -0xd5,0xfd,0xd5,0x72,0x95,0x42,0xdb,0x32,0xa2,0x57,0x37,0xd5, -0xac,0xd7,0xcb,0x71,0x34,0xae,0x5e,0x66,0x2c,0x28,0xd0,0xb8, -0xfd,0x0d,0xc4,0x21,0xa8,0xf2,0x53,0xfb,0x9b,0xe5,0xff,0xf6, -0xec,0xa7,0x1f,0x15,0xe7,0x8b,0xe5,0x72,0x95,0x65,0x26,0x4d, -0x78,0xba,0x43,0x4c,0xfb,0x44,0x1f,0x64,0x64,0x8c,0x83,0x91, -0xeb,0xcb,0x4e,0xe1,0xa6,0x5c,0xcc,0x5c,0x6f,0x44,0x23,0x59, -0x1e,0xa2,0x6a,0x5f,0x61,0xca,0xe4,0x1a,0x99,0xb1,0xd4,0x52, -0xb8,0xc6,0x08,0xed,0xa9,0xcd,0xa1,0xf2,0x8d,0x04,0x18,0xb6, -0x5f,0x3b,0xcc,0xcc,0x4c,0x3c,0x31,0x30,0x12,0x5b,0x64,0x6c, -0xe9,0xe6,0xba,0x9c,0x3f,0x9a,0xd6,0x65,0x4a,0xc0,0x4c,0x0a, -0xae,0x6a,0xa7,0x77,0x94,0xc2,0x6c,0xba,0x96,0xc7,0x58,0x0a, -0x86,0x7a,0x33,0x9f,0x17,0x05,0xe4,0x40,0x93,0x61,0xfe,0x9e, -0x66,0xb3,0x2c,0x0f,0xca,0xe1,0xba,0xd8,0x35,0x17,0xdc,0x2f, -0xe5,0xf5,0xf2,0x6d,0x19,0x2d,0xbb,0x8f,0x5a,0x71,0x95,0x99, -0x3b,0x4c,0xfc,0x2b,0x96,0x4d,0xb8,0x50,0x27,0x79,0xf8,0x8d, -0x2b,0xc5,0x4e,0x16,0x41,0x87,0xa0,0xfa,0x08,0x2e,0x02,0xa2, -0x89,0xc8,0x02,0x80,0xb6,0x7c,0x9a,0x58,0x3e,0xa3,0x69,0x84, -0xff,0x16,0xf8,0x4f,0xff,0x62,0xb9,0xb8,0x98,0x6e,0x74,0xcc, -0xd7,0xd3,0x15,0x95,0x30,0x31,0x7a,0x01,0xaf,0x98,0x8e,0x5c, -0xd6,0xf6,0xcc,0xad,0xf0,0x4c,0xe5,0x54,0xad,0x68,0x9f,0x15, -0x63,0xbb,0x0b,0xeb,0x4b,0x55,0x82,0xa3,0x5e,0xcd,0xab,0x4d, -0x9a,0x1c,0x25,0x59,0x36,0xbc,0x5c,0xae,0xd3,0xaa,0x38,0x31, -0x73,0x06,0x91,0x59,0xfc,0x70,0x7e,0x5a,0x0d,0xab,0x5e,0x2f, -0xe3,0x85,0x7c,0x14,0x92,0xca,0xb8,0x9a,0x4c,0x10,0x57,0x1d, -0x5a,0x43,0x55,0xfd,0xe4,0x7a,0xb5,0x79,0x8f,0x73,0xf7,0x13, -0xad,0xc1,0xdc,0x62,0x81,0x32,0x38,0x31,0x73,0x48,0xcc,0x84, -0x97,0xec,0x98,0x0d,0xa5,0xd2,0x47,0xb4,0x8c,0x1a,0xcd,0x7a, -0x53,0x93,0xe1,0xfa,0x8d,0xe6,0x7c,0x5e,0x4e,0x17,0x44,0x3f, -0x63,0x9c,0xe9,0x89,0xe9,0x9c,0x64,0x0a,0x48,0x7d,0xb3,0x5a, -0x2d,0xd7,0x9b,0x3e,0x77,0xf4,0x84,0x27,0x5b,0xf8,0x5f,0x87, -0x97,0x67,0x9f,0x37,0xf0,0x51,0x0c,0x8b,0x9b,0xe2,0x02,0x97, -0x47,0xb6,0xf3,0xe8,0x1b,0xc1,0x7c,0xb8,0xd9,0xac,0x99,0xaa, -0xdf,0x94,0xef,0x69,0x6d,0x12,0xdd,0xd0,0x3a,0x0e,0x16,0x29, -0xee,0xd4,0x01,0xe5,0x32,0x85,0xd3,0x64,0x25,0x58,0xfc,0x38, -0xe9,0x41,0x13,0xfd,0x75,0xb9,0x9a,0x4f,0x2f,0xca,0x74,0x7d, -0x7d,0x33,0xdf,0x54,0x8f,0xa7,0xf5,0x95,0x49,0x8e,0x3f,0x1b, -0x24,0x19,0xf0,0xbd,0xef,0x97,0xb7,0xe5,0x9a,0x66,0x3a,0x1b, -0x6a,0x27,0xd4,0x26,0x2c,0x6c,0x04,0xa4,0x3a,0xbf,0xd9,0x28, -0x15,0x34,0x76,0x1e,0x86,0x6d,0xb3,0x7e,0x7f,0x47,0xf5,0x92, -0xcd,0xfa,0xa6,0x44,0xe1,0x00,0xbf,0x48,0x5c,0xb8,0x9c,0xce, -0x6b,0x2f,0x65,0x90,0x27,0x38,0x62,0x9b,0x80,0x1f,0x79,0x0f, -0x7f,0xf6,0x12,0x9b,0x48,0xdf,0xf9,0xfa,0x7c,0x0d,0x20,0xf7, -0x37,0x65,0xbd,0x21,0xa0,0x32,0x9d,0x94,0xd5,0x74,0x5d,0x97, -0xc8,0xad,0x39,0x39,0xc7,0x7f,0x77,0x17,0xb4,0xa9,0xc3,0xf8, -0x77,0x52,0x6a,0x66,0x39,0x83,0xc5,0xe1,0xae,0x04,0x58,0x08, -0x64,0x87,0xc3,0x9d,0xec,0x40,0xd4,0x88,0x27,0x5d,0xc5,0x64, -0x62,0x65,0x2c,0x44,0x04,0x91,0xb7,0x2e,0x06,0xcc,0x01,0xbc, -0xa5,0x84,0x6f,0x94,0x82,0x78,0x8f,0xe8,0xb4,0xd1,0x2b,0x36, -0xc3,0x4b,0x06,0x89,0x2d,0xe1,0x4d,0x47,0xeb,0x08,0x01,0x77, -0x06,0x22,0xbc,0x80,0xbc,0x61,0x01,0xe2,0x94,0xe7,0x80,0xdd, -0x34,0xbb,0xdb,0x97,0xfd,0x2d,0xe2,0xda,0xe5,0x0f,0x5c,0x7e, -0x5d,0x9d,0xcf,0x61,0xde,0xd2,0x8b,0x9b,0xb5,0x99,0x55,0xeb, -0xec,0x6e,0xb6,0x3c,0x82,0xdf,0x05,0xfc,0x37,0x86,0xef,0xc9, -0x10,0x84,0xa0,0x79,0x89,0xd9,0x40,0x53,0x00,0x0e,0xfc,0x70, -0x24,0xa5,0xb2,0x14,0x24,0xba,0x16,0x81,0xb2,0x17,0x20,0x32, -0x22,0x82,0xcb,0xc5,0xa6,0x36,0xbf,0xde,0x4c,0xe7,0xd5,0x65, -0x55,0xae,0x01,0xdd,0xe5,0x8a,0xc8,0xd5,0x26,0x15,0xf6,0xd7, -0x76,0x7b,0x62,0x2c,0x5e,0xbe,0x55,0x69,0xcb,0x66,0xeb,0x1a, -0x56,0xb1,0xed,0x15,0xd0,0xad,0xeb,0xc2,0x4a,0x67,0xcc,0x83, -0x79,0x3a,0xa0,0xc2,0x5f,0xa7,0xf3,0xa2,0xd3,0xb1,0x8d,0x00, -0xeb,0x9a,0xcf,0xa5,0x8c,0x21,0xb6,0xaf,0x03,0x90,0xb2,0x45, -0x81,0x20,0xee,0x88,0xda,0x5d,0x2d,0x3b,0xdc,0x8f,0x06,0x41, -0x11,0x7d,0x84,0x1f,0xd0,0xaa,0x1b,0xaf,0xdf,0x43,0x63,0xc9, -0xb8,0xd1,0xd2,0x00,0x2e,0xab,0x39,0x6c,0x68,0xe5,0xac,0xf8, -0x84,0x0e,0x1b,0xcb,0x9e,0xbb,0xaa,0xea,0x67,0xd5,0xf5,0x6a, -0x2e,0x6b,0x66,0x2f,0x56,0xb9,0x47,0x97,0x6f,0x14,0x04,0xd3, -0xa1,0xb9,0x1b,0xba,0x81,0x7c,0xa8,0x46,0xb6,0xfb,0x74,0x6c, -0xe9,0xfc,0x2f,0x78,0x37,0xa3,0x89,0x72,0xb0,0x9e,0x16,0x27, -0x16,0x7d,0xf1,0x41,0xe0,0xd9,0xf4,0xb2,0xfc,0x76,0x3d,0x7d, -0x85,0xad,0xa7,0xb3,0xe5,0xc5,0x0d,0xfe,0x90,0x93,0x4f,0x55, -0x6f,0x0a,0xc4,0x06,0x0a,0x22,0xb5,0x6e,0x45,0xdb,0x24,0x33, -0xb5,0x54,0x2a,0xb4,0x42,0x9f,0x1b,0x7b,0x2c,0x9f,0xb6,0x41, -0x42,0xa1,0x96,0x96,0x42,0x4f,0x78,0x28,0x19,0xae,0xf7,0x21, -0xf6,0xa1,0x3b,0x59,0xd6,0x5e,0x30,0xa5,0x32,0x24,0x9c,0x5a, -0xaa,0xd3,0x92,0x6e,0x34,0xc0,0x78,0x66,0x3f,0xad,0x1f,0xae, -0x56,0x28,0xe7,0xd1,0xf8,0x37,0x78,0xd8,0xf0,0xa8,0x09,0x79, -0xaf,0x34,0x59,0x7f,0xf3,0xfe,0xf9,0xf4,0x15,0x0e,0x2b,0xc5, -0x52,0xe3,0x13,0x90,0x66,0xa9,0xc8,0x94,0xea,0x3f,0x82,0x85, -0xcb,0x8d,0xf4,0x97,0xb7,0x8b,0x72,0xfd,0x38,0x1c,0xa5,0xc2, -0x85,0x55,0x3d,0x7c,0xce,0xaa,0x7a,0x7a,0x3e,0x2f,0x9f,0x5d, -0xac,0xab,0xd5,0x46,0xa6,0x07,0xd1,0x38,0x05,0x7e,0xdf,0x64, -0xfe,0xb8,0x23,0xa6,0x09,0x92,0x6f,0x62,0x47,0x45,0x85,0xe8, -0x74,0x99,0x62,0xa5,0x6e,0x17,0xff,0x05,0xbc,0x97,0x17,0x38, -0x8f,0xb3,0xac,0x97,0xdc,0x4f,0x7a,0xb6,0x10,0xad,0x43,0x9f, -0x5d,0xd5,0x1b,0x10,0xe8,0x1b,0xdd,0xd3,0xc9,0xac,0x58,0xd7, -0x94,0x8e,0x74,0xfd,0xc3,0xb4,0x7e,0x53,0xce,0x40,0x92,0x2a, -0x2f,0x52,0xdb,0x98,0x85,0x81,0x8a,0x8f,0x1c,0x24,0xf4,0x3d, -0x1e,0xf0,0xd1,0x05,0xf6,0x3c,0x14,0x10,0xdd,0x0e,0x26,0x03, -0x88,0x40,0xa9,0xcb,0xcd,0xbf,0xcf,0x97,0xe7,0xd3,0xf9,0x93, -0xb7,0x53,0xe6,0x1a,0x35,0x08,0x9c,0x97,0x8a,0x7c,0x38,0x71, -0xc3,0xd4,0x23,0x6c,0xcc,0x50,0x8a,0x93,0x21,0x6e,0x59,0x9d, -0x82,0x8a,0x12,0xaa,0x6a,0x10,0x5c,0x32,0x12,0x6b,0x84,0xb2, -0x5f,0xba,0xad,0x27,0x79,0x65,0xdb,0x4e,0x4c,0xc7,0x6b,0xd7, -0x1e,0x5d,0xb8,0xb0,0x97,0x03,0xad,0x05,0xd5,0xfc,0x69,0xbb, -0x98,0x2f,0x17,0xe5,0xa3,0xe5,0xea,0xfd,0x93,0xb7,0x38,0xa9, -0xf5,0xfa,0xc2,0xcc,0x00,0x95,0xc4,0x6d,0x91,0x27,0xe0,0x87, -0x77,0x6a,0x97,0x2e,0xae,0xa6,0x35,0x89,0x30,0x50,0x5c,0xcf, -0x9c,0x38,0x25,0x28,0x06,0x2f,0xe7,0xb3,0xc7,0xde,0xd1,0x83, -0x81,0xc1,0x72,0x06,0x98,0x7d,0x33,0x07,0xdb,0xd7,0x3a,0x80, -0x48,0x04,0xa2,0x2e,0xe4,0xbb,0xcf,0x9f,0xb8,0x88,0xf8,0x17, -0x6c,0x32,0x22,0xf6,0x70,0x53,0x00,0xc7,0x62,0x36,0x2f,0xb5, -0x65,0x29,0x8f,0x67,0x75,0x44,0x31,0x82,0x84,0xbb,0xa9,0xd4, -0x75,0xf2,0x23,0x27,0x8c,0x31,0x7f,0x12,0xcb,0x91,0x2a,0x68, -0x63,0x91,0xfe,0x74,0x36,0x63,0x00,0x99,0xe0,0xbc,0x6a,0x38, -0x41,0x3b,0xed,0x16,0x07,0x02,0x72,0x9e,0xff,0x19,0x9d,0xbc, -0xe0,0x80,0xe8,0xe7,0xc2,0x04,0xf8,0x4b,0xf7,0xdd,0x23,0x9c, -0x04,0x5c,0x12,0x4f,0xeb,0xfa,0xa6,0xac,0xbd,0x59,0x20,0xf1, -0x40,0x78,0x90,0x61,0x1d,0xc0,0xb0,0x6d,0x66,0xf8,0x10,0x20, -0x05,0x5d,0x16,0x7e,0x85,0xf2,0x98,0xe9,0x44,0xb2,0xe6,0x62, -0x49,0xbd,0xd3,0xf4,0x77,0xbb,0x58,0x33,0x3e,0x7c,0x64,0x77, -0xb3,0xd6,0x69,0x63,0x91,0x9c,0x50,0x3c,0x73,0xd8,0x57,0x0c, -0xf2,0x5a,0x61,0xaa,0x22,0x1c,0x32,0xf4,0x32,0x65,0xd9,0x90, -0x60,0x8c,0x17,0x54,0xd8,0x73,0xb6,0x4b,0x78,0xdd,0xa2,0x18, -0xa7,0xc3,0x61,0x18,0x61,0x3f,0x7a,0xb7,0x01,0xa9,0x02,0x30, -0x45,0x3f,0x47,0x69,0xc8,0x7f,0x08,0x3e,0xca,0xb1,0x45,0x4c, -0xc8,0x22,0xa8,0x44,0x96,0xbb,0xe3,0xb8,0xed,0x61,0x44,0x79, -0x28,0x13,0x02,0xec,0x38,0x2b,0x30,0xb7,0x94,0xb2,0x04,0x10, -0xd7,0xdf,0x3d,0xff,0xe1,0x7b,0x6a,0xd3,0x7e,0x65,0x26,0x42, -0xe9,0xd5,0xe6,0x7a,0xfe,0x15,0x61,0xb5,0xdb,0xc5,0x92,0x20, -0xd7,0x70,0xc9,0x6e,0x57,0xb1,0x0f,0x03,0xbe,0xe6,0x46,0x6d, -0x66,0xa6,0xdd,0xd8,0x94,0x22,0xa8,0x8c,0xc0,0x56,0x8b,0xd5, -0x4d,0x84,0x8d,0xeb,0xe9,0xa2,0x5a,0xdd,0xcc,0xa7,0x48,0x4b, -0x2f,0xd7,0x70,0x06,0xb8,0x78,0x83,0x78,0x40,0xa2,0xe0,0x4d, -0x9b,0xc6,0x8f,0x24,0x22,0xe3,0x02,0x29,0x75,0x0a,0x22,0xfb, -0x23,0x2c,0x09,0x32,0x02,0xa5,0x5d,0xc8,0x07,0x96,0x95,0xdf, -0x44,0x80,0x7d,0x60,0x17,0x37,0xa5,0xe0,0x99,0x7e,0x2b,0x90, -0xf4,0xe1,0x92,0x09,0x93,0x2b,0xd1,0x5f,0x38,0x4c,0xfa,0x1d, -0x3e,0x83,0xa5,0x7b,0xb1,0xd1,0x1e,0x6b,0xfd,0xc2,0x26,0xa2, -0x12,0x79,0xda,0x1c,0xe8,0x76,0x9b,0xe0,0x24,0xc2,0x9c,0x4c, -0xfd,0x64,0x8b,0x34,0x69,0xe2,0xaf,0x16,0x2c,0x3f,0xc1,0x5f, -0x6e,0xb8,0x0b,0x81,0x30,0x07,0x27,0x5c,0x22,0x0a,0xda,0x26, -0x95,0x19,0xd7,0x46,0x59,0xb2,0xb9,0x5c,0x82,0x3c,0xaf,0xf2, -0x95,0x14,0x6e,0xdd,0x42,0x51,0xb4,0x45,0x4d,0x12,0xc8,0x64, -0xf6,0x04,0x30,0x3a,0x54,0x01,0x37,0x4e,0x18,0xcd,0xbd,0x24, -0xcb,0xa3,0xe6,0x7f,0x45,0xd2,0x60,0x24,0x2c,0xd7,0x0f,0x71, -0x4b,0x38,0xd0,0x74,0x5c,0xd8,0x6b,0xd6,0x16,0xa6,0xb3,0x2f, -0x8d,0x84,0xb8,0x1f,0x8f,0x69,0x3c,0xa1,0x51,0xd6,0x85,0xb6, -0x74,0x81,0xfb,0x3d,0xd2,0x39,0x6c,0x21,0x92,0x76,0x60,0x47, -0xea,0x50,0x3f,0x56,0xa9,0xc5,0xd3,0xe0,0x44,0x8e,0x11,0x75, -0xd2,0x5f,0xdd,0xd4,0x57,0xbc,0x0b,0xeb,0xd9,0xf6,0xba,0x5c, -0xbf,0x2a,0x19,0x04,0x23,0x73,0x60,0x2b,0xd9,0xed,0x17,0x3e, -0xfc,0xd3,0xe8,0x76,0x0b,0x09,0x76,0xd3,0xb1,0x9d,0xf9,0x73, -0x37,0x0a,0x9a,0x1f,0x4b,0xd6,0x84,0xe7,0x2f,0xcb,0xe9,0x4f, -0xc0,0x6b,0x1f,0x07,0xe4,0x2f,0x92,0xc2,0x07,0x17,0x91,0x93, -0x14,0x80,0xe0,0xe8,0x23,0x5a,0x46,0x94,0x26,0x4b,0xc7,0xdb, -0x5e,0x81,0xfd,0xcd,0x96,0xeb,0x9f,0xd7,0xcb,0x15,0x41,0x5e, -0x6f,0xde,0xc3,0x76,0x45,0x27,0x33,0xd5,0xda,0x20,0xfb,0xa4, -0x64,0x15,0xa9,0xed,0x69,0x10,0xa9,0xf2,0x62,0x4a,0x15,0x45, -0x2b,0x73,0x35,0x5d,0x3f,0xdc,0xa4,0x27,0x78,0xc4,0xfe,0x4f, -0x10,0xd5,0x84,0xa5,0xf7,0x58,0x55,0x32,0xaf,0xe0,0x34,0x3e, -0xc8,0xcc,0x72,0x5d,0xbd,0xb2,0x75,0x80,0x96,0x2f,0xea,0xfa, -0x67,0x10,0x06,0xaa,0x77,0x20,0xc4,0xca,0x66,0x57,0x1d,0x1f, -0x0f,0x33,0xe9,0xdf,0xcf,0x87,0x49,0xee,0x49,0x8f,0x66,0x3f, -0x6c,0xf2,0x5b,0xfb,0xf1,0x8f,0xb8,0xdf,0x55,0xb3,0x59,0x29, -0xa7,0xa9,0x72,0x1e,0x9e,0x66,0xca,0x39,0x4b,0x99,0x26,0x59, -0x00,0x67,0xc4,0x45,0xac,0x1a,0x91,0xba,0x16,0xd1,0x06,0x78, -0xf8,0x6a,0x3e,0x7d,0x9f,0x64,0xee,0xac,0x8b,0x13,0x3a,0xad, -0x16,0x75,0x8b,0x3c,0xca,0x27,0x31,0x4f,0xee,0xba,0x5a,0xde, -0x02,0x00,0xa5,0x3b,0x2d,0x60,0x8a,0x13,0xb7,0xa4,0x75,0x5e, -0xe3,0x57,0x04,0xa9,0x21,0xe6,0x55,0xe3,0x9a,0x00,0x01,0xba, -0x7c,0x87,0xe2,0x01,0x5f,0x78,0x68,0x1b,0x56,0x3e,0xe0,0xbb, -0x05,0x2a,0x35,0xa4,0x7f,0x61,0x2d,0xd8,0xe5,0xc1,0x52,0x16, -0xa6,0xf2,0xe2,0xea,0x13,0xd6,0x80,0x54,0xb8,0x7d,0xc9,0x2a, -0x5a,0xa4,0x39,0x10,0x75,0xec,0xa8,0x8d,0xfc,0x2a,0x5c,0x13, -0x7d,0x05,0x1a,0x87,0x32,0x0a,0x9b,0xc3,0x9b,0x10,0xc4,0x24, -0xb0,0x09,0x29,0xb5,0xdd,0xa6,0xcd,0xaa,0x45,0x02,0x2d,0x93, -0x26,0xa4,0x99,0xd7,0xed,0x06,0x73,0x96,0x7d,0x22,0xc4,0x06, -0xa6,0xee,0xa5,0x2c,0x84,0xc7,0x9c,0x96,0xda,0x43,0x24,0xb1, -0x67,0xf8,0x5f,0x1e,0x01,0x9d,0x32,0xea,0x8b,0xb0,0x67,0x93, -0x5a,0x90,0x9a,0xa3,0xea,0x70,0x95,0xcc,0xb2,0x82,0x7d,0xf0, -0x70,0xb9,0x91,0x7c,0xe7,0xfb,0x09,0x2c,0x53,0x85,0x22,0xcf, -0xfa,0xef,0x9d,0x5e,0x9c,0x15,0x0f,0xe0,0x36,0x04,0x27,0xed, -0x19,0xed,0x53,0x45,0xb3,0x1c,0x4f,0x72,0x92,0x73,0x07,0x59, -0x70,0x52,0x42,0x98,0x82,0x33,0xc7,0xcf,0xcb,0xba,0xda,0x54, -0x6f,0xcb,0x1f,0xe9,0x76,0x8c,0x07,0x4c,0x4d,0x99,0xfa,0xe6, -0x7c,0xb3,0x9e,0x5e,0x6c,0xbc,0x13,0x11,0xd0,0xfc,0x7a,0x71, -0x73,0x4d,0x87,0x5a,0x3e,0x0b,0xf1,0x36,0x1e,0x9c,0x83,0xca, -0x7a,0xf4,0xc3,0x74,0x73,0xd5,0xbf,0x9e,0xbe,0x4b,0x4f,0x8c, -0x24,0xc1,0x61,0xe8,0x38,0xd5,0x16,0xb7,0xdb,0x93,0x2c,0xeb, -0xa5,0x9a,0xf5,0x00,0xc1,0x5d,0xbd,0x4b,0x64,0xce,0x1d,0x78, -0xd3,0x1b,0x3a,0x10,0xff,0xad,0x9a,0x6d,0xae,0x7e,0x5a,0x7f, -0x57,0x56,0xaf,0xae,0x36,0x9e,0x8e,0x1d,0x38,0xf6,0x1a,0x15, -0xa9,0xdf,0x2c,0xd7,0xb3,0x72,0xfd,0xcd,0xf2,0x9d,0x21,0xb4, -0x78,0x47,0xa5,0xaa,0xa0,0x32,0x40,0xc4,0xa9,0x57,0x6c,0x94, -0x9c,0xd3,0x4f,0xc0,0x10,0x31,0xfe,0xc5,0x26,0xc9,0x46,0x5f, -0xe6,0xc9,0x2d,0xf6,0x43,0x62,0x02,0x0a,0x21,0x83,0xfc,0x04, -0x11,0x01,0xb3,0xfc,0x25,0x09,0xf9,0xc5,0x83,0x2c,0xb9,0x9e, -0xae,0x5f,0x55,0x24,0xab,0x50,0xbb,0x4c,0xf7,0xbd,0x06,0x47, -0xa2,0xcc,0x1e,0x7c,0xb2,0x2e,0x16,0x0f,0x52,0x9d,0x13,0x05, -0x2e,0xf3,0x21,0x1e,0xa5,0x16,0x84,0xb0,0xd1,0xe3,0x26,0x9b, -0x5b,0xc1,0xb9,0x02,0xf5,0x37,0xfb,0x1b,0x56,0xf8,0x3a,0x1f, -0x6a,0x4a,0xc6,0x1f,0xb4,0xd4,0x4b,0x08,0xcf,0x89,0xdf,0x62, -0x96,0xb7,0x0f,0xf0,0x83,0xb0,0xb8,0x12,0x9d,0x0f,0xe0,0xea, -0xe3,0x61,0x51,0x22,0x83,0x66,0x02,0xb9,0xec,0x20,0x79,0x30, -0xf1,0x12,0x5d,0x3d,0x75,0x68,0x2f,0x3a,0x3c,0xb7,0xd1,0x94, -0xf3,0x66,0x71,0x79,0x59,0x4b,0xa3,0xb9,0x97,0xc0,0xcd,0x0b, -0x34,0x05,0x74,0xfb,0x8c,0x7e,0x09,0x1b,0xf2,0xa6,0xb4,0x88, -0xc4,0xfa,0xf3,0xe5,0xbb,0x67,0xd5,0x6f,0x80,0x0a,0x58,0xd4, -0x3c,0xd6,0x63,0x48,0x6a,0xdd,0xc8,0x6c,0x51,0x18,0xf8,0x40, -0x07,0x8e,0xd2,0xd8,0xc9,0x69,0x01,0xe0,0x6e,0xb7,0x7c,0x85, -0x05,0x3f,0x49,0x0e,0xc0,0x11,0xc0,0xf9,0xf0,0xd1,0xb3,0x67, -0xde,0xa8,0x75,0x02,0xa0,0x4e,0x54,0x85,0x67,0xc0,0x63,0x2a, -0xa2,0xff,0x35,0xb8,0xa4,0x81,0x55,0xac,0xde,0xb1,0xd0,0x82, -0x65,0x33,0x87,0xeb,0x61,0x03,0x79,0xde,0x60,0xa1,0xcd,0x7d, -0xc3,0xfd,0xa5,0x9c,0x57,0x28,0x0d,0x6d,0xb7,0x6f,0x49,0xd5, -0xd9,0xec,0x17,0xd3,0x49,0x85,0xfe,0xed,0x7c,0x39,0xe5,0x7e, -0x81,0x33,0xec,0x5c,0xd7,0xbd,0x8f,0x61,0x00,0x74,0x5f,0x7b, -0x70,0x6d,0x9b,0x78,0x04,0x8a,0xa4,0x1e,0x72,0x1d,0x4f,0xa9, -0xd1,0xdc,0x96,0xec,0x8e,0x44,0x64,0x34,0x5b,0x5e,0x58,0xe5, -0x5d,0xb0,0xeb,0xca,0xef,0xb1,0x16,0x9f,0x28,0xad,0x3a,0xb6, -0xad,0xc5,0x81,0xfd,0xdd,0x4c,0xe7,0x71,0xfb,0x06,0x9a,0xcd, -0x4c,0xbc,0x85,0xc1,0xa1,0xd5,0xd6,0xaf,0x2e,0xd7,0x28,0x76, -0xc9,0x5f,0x15,0xa5,0xd3,0xe4,0x6b,0x4e,0x38,0xa2,0x7f,0x79, -0xec,0xc5,0xe7,0x27,0x9f,0x1f,0x11,0x65,0xd3,0xaf,0x2b,0x42, -0x1b,0xfe,0xbc,0x7f,0x9a,0x64,0x44,0x6e,0x09,0xfc,0xf3,0x1c, -0xb0,0x97,0xd8,0xbd,0x2d,0x3f,0x9f,0x2f,0x2f,0xde,0x1c,0x75, -0xaa,0x6b,0x9c,0xc5,0x29,0xa2,0x2d,0x13,0xc5,0xde,0xf3,0x25, -0xaa,0x38,0xfb,0x3a,0x70,0xd5,0x44,0x22,0xc8,0x0a,0xce,0xf8, -0x64,0xd2,0x17,0x74,0xb3,0x05,0xc7,0x76,0xdb,0xc8,0x50,0xf9, -0x2b,0xeb,0x3b,0x14,0x42,0xb3,0xb7,0xeb,0x0a,0x55,0x62,0x5f, -0x77,0xe0,0x83,0x54,0x2f,0x78,0x0c,0x3e,0xfd,0x9a,0xff,0x3d, -0x5f,0xce,0xde,0x9f,0x26,0xd4,0x55,0xff,0x62,0xbe,0x24,0x4d, -0xc4,0xc7,0x20,0x92,0x3b,0x07,0x61,0x7b,0x83,0xe6,0x16,0x19, -0xab,0xda,0x1a,0x93,0xa4,0x68,0xb6,0x6d,0x7a,0x3b,0x4f,0xd8, -0xb8,0x36,0x6c,0x8f,0x7c,0xb2,0x76,0x09,0x31,0xa1,0x7e,0x93, -0xaf,0xc7,0x43,0xd4,0xe1,0x30,0x1c,0xe0,0xd1,0xb2,0x07,0x14, -0x79,0x12,0x46,0xa0,0xd5,0x64,0x3d,0x47,0xda,0x02,0xdf,0xf9, -0x0d,0x1c,0xbe,0x7e,0x9e,0xc2,0x28,0xeb,0x74,0x45,0x62,0x38, -0xda,0x83,0x18,0x58,0x0e,0xb3,0x0a,0x0b,0x4c,0xe7,0x06,0xf8, -0xaf,0x77,0x4f,0xe4,0xae,0xa2,0xf5,0xf2,0x16,0x6f,0x8b,0xb2, -0xc0,0x2a,0x05,0x1a,0xb0,0x8a,0xf2,0xca,0xbc,0xc5,0xdb,0x34, -0xdb,0xdc,0x76,0x4b,0xb7,0x60,0x6f,0xca,0x0d,0x33,0x09,0xee, -0x34,0x1b,0xa1,0xae,0x4b,0x00,0x78,0x9b,0xe5,0x4d,0xb0,0x7a, -0xc9,0x38,0xe9,0x35,0xad,0x17,0xde,0x8e,0xaa,0x1c,0x04,0xcc, -0x5e,0x32,0x49,0xcc,0xdb,0x06,0xd8,0xbb,0x6c,0x48,0x17,0x64, -0x00,0x74,0x00,0x82,0x36,0xd3,0x29,0x1a,0x96,0x44,0x99,0x07, -0x08,0x59,0x16,0x51,0x03,0xf1,0xed,0xd8,0x1e,0xf8,0xb0,0x08, -0x81,0x62,0x2f,0xc5,0x9a,0x20,0x39,0xd2,0x98,0xc1,0xb4,0xd2, -0xd1,0x07,0x6f,0x1e,0xea,0x9f,0xd6,0xcf,0xd7,0xd3,0x45,0x8d, -0xab,0xa6,0x4e,0xf1,0xc8,0x0d,0x94,0xb3,0x2e,0xed,0x19,0xc6, -0x22,0x14,0x45,0x4f,0x3c,0x18,0xc2,0x1e,0xb7,0x2e,0xeb,0x1a, -0x92,0x08,0xd9,0xd9,0x9d,0x5e,0xcb,0x74,0xfc,0x9b,0xcc,0xb0, -0x24,0x70,0x59,0x2c,0x5b,0xb4,0xb4,0xd1,0x4c,0x2a,0xf0,0x30, -0x3f,0x24,0x76,0x25,0x79,0xa4,0x99,0xd0,0x8f,0xba,0xa5,0x95, -0x50,0xd7,0xb7,0xdf,0x2c,0x29,0xa0,0x22,0x7b,0x69,0x46,0xc3, -0xe0,0x8b,0x09,0x6d,0xda,0xf6,0x01,0xf2,0x68,0xaf,0x37,0x19, -0x66,0x49,0x4f,0xef,0x54,0x31,0x15,0x48,0x7e,0x94,0x36,0xca, -0xda,0xb3,0x28,0xe9,0x23,0x8c,0x43,0xe6,0x58,0x4b,0x4c,0x8a, -0x96,0x34,0x04,0x2c,0xeb,0xdf,0x2c,0xea,0xab,0xea,0x72,0x23, -0xc0,0xe4,0x9f,0x56,0x99,0xd4,0x0e,0x54,0x33,0x30,0xef,0xc0, -0x3b,0x84,0xcd,0x87,0x66,0xda,0x88,0x31,0x17,0x1d,0xa0,0x2b, -0x20,0x17,0x31,0x12,0x33,0xaf,0x7f,0xfd,0xfb,0x77,0xbf,0xf8, -0x76,0x72,0xdc,0x9e,0x1d,0x37,0x2f,0x4e,0x55,0x68,0xe9,0xba, -0x97,0x52,0xe5,0xcc,0x83,0xbb,0x63,0xaf,0x2a,0x69,0x9d,0xee, -0x19,0x86,0x6f,0x33,0xb6,0x52,0xa0,0x7f,0x5a,0x7f,0x3b,0x45, -0x4d,0xcf,0x7b,0xd9,0xc0,0xa4,0x82,0x37,0x94,0xa2,0x59,0x36, -0x3d,0x3c,0x24,0x01,0x75,0x2f,0xdd,0x7a,0x8d,0x6f,0xb7,0x75, -0x59,0xbe,0x81,0x52,0x5e,0x4a,0x73,0x84,0x5e,0x85,0xc9,0x28, -0xae,0x30,0xea,0xa4,0x56,0xe9,0xd7,0x52,0x01,0xce,0x0c,0xcb, -0x6a,0x76,0x74,0x92,0x5b,0xa3,0x3a,0x4b,0x7a,0x96,0x26,0xda, -0xaa,0x99,0x78,0x3a,0x82,0xcc,0xce,0x00,0x78,0x90,0xd1,0x7e, -0x77,0x74,0x94,0x50,0xb0,0xd1,0x6c,0x2b,0x06,0xd2,0x91,0x16, -0x9a,0x23,0x5a,0x2a,0x89,0x26,0xb5,0x09,0x22,0x2c,0x04,0x54, -0x5a,0x38,0x94,0x00,0xe1,0x93,0x75,0x1b,0x97,0xc7,0x95,0xec, -0x31,0x9e,0xd7,0xd3,0x77,0x4f,0xf8,0x62,0x60,0x03,0x72,0x7e, -0x09,0xc2,0xe8,0x5a,0xb6,0xa4,0x59,0x59,0xae,0xc8,0xba,0xe0, -0x72,0x3e,0xdd,0xc8,0x6c,0x29,0x8b,0xc4,0x6a,0xcf,0xca,0xcd, -0x06,0xe0,0xad,0xfb,0x5e,0xfe,0x76,0x2b,0x17,0x1e,0x50,0x8f, -0x34,0x36,0xd0,0x18,0xfc,0x37,0x86,0xcf,0x49,0x64,0x80,0x95, -0x7a,0xd5,0x28,0x7f,0xc4,0x00,0xe4,0xd8,0x2f,0x59,0xb8,0x94, -0x2b,0x34,0xeb,0xca,0x28,0xb3,0xd0,0x56,0xec,0x66,0x86,0xf9, -0xf6,0x10,0x2e,0x77,0x1b,0x40,0xd4,0x32,0x0a,0xcc,0xcd,0xe4, -0x23,0x1c,0xed,0x77,0xa4,0xf4,0xff,0xa5,0x04,0x6c,0x2e,0x6a, -0xbc,0xe0,0x60,0x0a,0x44,0x95,0x3c,0xa7,0xe8,0xc5,0xf5,0xba, -0x26,0xdb,0x2a,0xb6,0x98,0xda,0x98,0xcb,0x4a,0xec,0x20,0x29, -0x81,0x6e,0x61,0x44,0x08,0xa9,0x8b,0x5a,0xe5,0x91,0xda,0x63, -0x89,0xde,0x94,0xd8,0xc6,0xbf,0xad,0xca,0xf9,0x0c,0xb3,0xc2, -0x84,0xe0,0x8e,0x28,0xcc,0xca,0xe2,0xe4,0x1a,0xc5,0x63,0x84, -0x78,0x1c,0x16,0xe4,0xcb,0xa0,0x49,0x61,0xcb,0x71,0x02,0xeb, -0x16,0x86,0x30,0xeb,0x1e,0xaf,0x44,0x1a,0x19,0x66,0x8e,0xa8, -0x99,0xa4,0x33,0x18,0x66,0x68,0x80,0x93,0x42,0x42,0xdd,0xbf, -0xae,0xae,0x49,0xfd,0x08,0xf2,0x21,0xf6,0x8b,0x9a,0x64,0x45, -0xdf,0x77,0xe5,0x14,0x64,0xc3,0x34,0x79,0xc4,0xc3,0x3f,0x7e, -0x4e,0x97,0x90,0x74,0xbc,0xb8,0xe0,0x7b,0x65,0x05,0x5f,0x11, -0x84,0xba,0x3e,0xfd,0xcd,0x10,0x76,0xbb,0xe1,0x37,0xcb,0x02, -0x50,0x9d,0x2f,0x7b,0xc2,0x65,0xc7,0x37,0xa4,0xe7,0x20,0x1d, -0xbd,0xd9,0x89,0x39,0x8f,0x0e,0xc8,0xc7,0x51,0x16,0xcc,0x56, -0x38,0x6e,0xdc,0xc4,0xef,0xda,0x10,0x5e,0xd3,0x01,0xa8,0xe3, -0x17,0x06,0x5e,0x83,0x53,0xfb,0xb6,0x5c,0x23,0xb3,0x26,0xf8, -0x7a,0xc9,0x51,0xd2,0xf3,0xcb,0x4c,0x80,0x1f,0x07,0xbd,0x61, -0x29,0x01,0x31,0xa0,0x22,0x20,0xea,0xe0,0x9b,0x0a,0xc2,0x5a, -0x0c,0x2a,0x07,0x5f,0xdb,0x6d,0x50,0x41,0x4f,0x31,0x41,0x99, -0x51,0x1a,0x7c,0x76,0xc2,0x59,0x06,0x69,0xbf,0x81,0xc3,0xa0, -0x7c,0xe6,0x08,0x7f,0x1c,0x64,0x4c,0x94,0x0b,0x86,0xcb,0xe7, -0x11,0x63,0x23,0x75,0x34,0xcd,0xeb,0x05,0xb1,0xf4,0x00,0x6f, -0x16,0xf1,0xa6,0x0a,0x57,0x06,0x48,0x60,0xd7,0x2b,0xe3,0x90, -0x47,0x96,0xa9,0x81,0xcc,0xe0,0xb3,0x55,0xde,0xa6,0x33,0xdc, -0x64,0xde,0x86,0xd3,0x85,0x66,0x0b,0x54,0xf0,0x5b,0xda,0x50, -0xc8,0xce,0x92,0x3b,0x2e,0xfc,0x74,0x9b,0x6a,0x5c,0xb3,0x59, -0xe6,0x7a,0x1b,0x0f,0x26,0x44,0x8f,0x08,0x10,0x71,0x26,0x6f, -0x5e,0x33,0x6f,0x8a,0xf1,0x67,0x28,0xba,0x4c,0x8a,0x80,0x06, -0xf0,0xe7,0x84,0x17,0x95,0x8c,0xd6,0x83,0xb7,0xd7,0xab,0x26, -0xa4,0xce,0x86,0xf5,0xc6,0x86,0x46,0x6b,0x32,0xe2,0xb0,0x29, -0x38,0xbe,0x6e,0x17,0xff,0x8d,0xb2,0xb1,0xd9,0xc2,0xeb,0x06, -0x8b,0x10,0xa9,0x49,0xa1,0x09,0x5d,0x88,0x68,0x6e,0x72,0xcf, -0xcb,0x31,0x1d,0xcc,0xb1,0xa3,0x7b,0x20,0xcb,0x4d,0x07,0x87, -0x02,0xef,0xf5,0x8a,0xda,0x7e,0xe0,0x59,0x31,0xe2,0xf4,0x00, -0x52,0x0a,0x0b,0x06,0x2e,0xf7,0x7d,0x40,0x60,0x59,0xa0,0xf4, -0x26,0x0c,0x92,0x01,0x6b,0x95,0xea,0x16,0x20,0x57,0x8c,0xe2, -0x56,0xa8,0xe7,0x49,0xde,0x48,0xe9,0x60,0x69,0xbe,0xac,0x26, -0x34,0x72,0x5b,0xc6,0x23,0x8a,0x15,0x51,0x45,0x75,0x7c,0x6c, -0x4e,0x94,0xb4,0x32,0x6f,0xf5,0x63,0x33,0xd4,0x88,0x30,0x15, -0xc0,0x2c,0x80,0xb5,0xb9,0x5a,0x2f,0x6f,0xeb,0x64,0x92,0x59, -0x3a,0xc1,0x2c,0x4b,0x1f,0x22,0xc5,0xa3,0x51,0xdf,0x9e,0x02, -0xce,0xda,0x8e,0xd7,0xdb,0x5d,0xbd,0x81,0xb3,0x58,0x9e,0x90, -0x5a,0x61,0x5d,0xae,0xd7,0xcb,0x75,0x62,0xe8,0x0f,0x8d,0x69, -0x04,0x59,0x3f,0x2e,0x05,0xe3,0x35,0x5d,0xed,0xac,0x97,0xd7, -0x80,0x1b,0xc1,0xde,0x66,0xe9,0xe6,0x6a,0xb7,0xdb,0x11,0x85, -0xeb,0x67,0xd8,0x41,0x7d,0x73,0x71,0x01,0xc2,0x73,0x62,0xd8, -0x42,0x50,0xe0,0xd9,0x35,0x0c,0x85,0xe0,0x28,0x3d,0x9b,0xae, -0x67,0xc0,0x8b,0x53,0xb6,0x4e,0xd4,0x2b,0x91,0xf2,0xf6,0x88, -0x6d,0x32,0xfb,0x7f,0xff,0xe1,0xfb,0xef,0x36,0x9b,0xd5,0x2f, -0xe5,0xaf,0x37,0xc0,0x4d,0x3d,0xfb,0xc1,0xb8,0xb1,0x87,0x17, -0xa8,0xa8,0x3d,0xd4,0x94,0x94,0x10,0x13,0xbf,0xe4,0x87,0xea, -0x62,0xbd,0xac,0x97,0x97,0x1b,0xea,0xe3,0xf9,0xf3,0x9f,0x93, -0xec,0x40,0xf3,0xdf,0xbe,0xfb,0x71,0x79,0x6b,0x8d,0xf6,0x50, -0x37,0xfc,0x1c,0x76,0x93,0xe5,0x0d,0x0b,0xd5,0x24,0x59,0x02, -0xfb,0xc4,0x42,0x9e,0xd1,0x62,0x66,0x38,0xc5,0x5e,0xb6,0x41, -0x0b,0x71,0xc3,0xcf,0x6f,0xcb,0x72,0x51,0xa7,0xd3,0x45,0x75, -0x4d,0x57,0x65,0xc0,0x39,0x96,0xab,0xda,0x5a,0x13,0x93,0x50, -0x4b,0x49,0x4e,0x84,0xc5,0x4f,0x56,0xdb,0x38,0x65,0xee,0xc5, -0x72,0x8e,0xf2,0x18,0x1e,0x71,0xd2,0x0d,0x36,0xc9,0x64,0xbf, -0x5c,0x89,0x14,0x2f,0xd6,0xc7,0x36,0x0b,0x65,0xa9,0x2c,0xbe, -0xa0,0x71,0x8d,0x1c,0xbc,0xa2,0x21,0x22,0xd5,0x92,0xa2,0x53, -0x67,0x73,0xbe,0x70,0x14,0x02,0xa3,0x5a,0xfc,0x7a,0x43,0x7f, -0xa8,0xe5,0x58,0x5f,0x85,0x85,0x61,0x45,0x55,0x20,0x60,0x04, -0xde,0x23,0x40,0x39,0x37,0x73,0x54,0x2b,0x2e,0x57,0xab,0x72, -0x16,0x43,0x6b,0x3b,0x73,0xa7,0x10,0xf5,0xa7,0x81,0x09,0x00, -0xba,0x76,0xd6,0x7a,0x8f,0xe5,0x1b,0xce,0x6f,0xd3,0xf9,0xed, -0xf4,0x7d,0xed,0xcf,0x9a,0x9a,0x39,0x57,0x17,0x6f,0xfa,0x64, -0x75,0x04,0x2c,0x05,0x7e,0x17,0x5e,0x11,0xe4,0xdd,0x0c,0x84, -0xb3,0xfe,0xb4,0x98,0xe7,0x25,0x80,0xe4,0x50,0xd0,0x74,0x03, -0x73,0xf1,0x49,0x06,0xf6,0x97,0xeb,0x69,0xb5,0x00,0x09,0xb3, -0xf0,0x34,0xff,0x16,0xf8,0x3e,0xac,0x99,0x35,0xd5,0xee,0xb9, -0xb4,0xd9,0xcd,0x9a,0x7e,0x1c,0x7b,0x8d,0x03,0x54,0x25,0xb0, -0x3f,0xdb,0xda,0xfd,0x66,0x71,0xb4,0xdb,0x04,0x44,0x5e,0x20, -0x13,0x1a,0x1c,0x63,0xf1,0xbd,0x38,0xeb,0x13,0x25,0x1c,0xbe, -0x8a,0x8b,0x0b,0xeb,0x54,0xaf,0x6f,0x80,0x06,0xb9,0x1b,0x4f, -0x8a,0x65,0x0c,0x03,0x91,0x6f,0xaa,0xcb,0xf7,0x7f,0xab,0x36, -0x7c,0x57,0x6d,0xc6,0x1e,0x4d,0x70,0x1d,0x87,0x10,0xa0,0xc0, -0xc1,0xa9,0xa4,0x5a,0xef,0x25,0x9b,0x9b,0xa7,0xb6,0x51,0x20, -0x84,0xe5,0xfc,0x6d,0xd9,0xd6,0xea,0x84,0x0f,0x24,0x0e,0xa1, -0x85,0xad,0x05,0x64,0x75,0x5d,0xc1,0xce,0x77,0x87,0x55,0x72, -0x4b,0x69,0x75,0xde,0x30,0x24,0x72,0x04,0x98,0x21,0x05,0xc6, -0x25,0x40,0x1c,0xbf,0x23,0x9b,0xb9,0xe9,0xfc,0xc9,0xb4,0x46, -0xd0,0x80,0x41,0x58,0x52,0xb5,0x27,0xc1,0x9f,0x6d,0x2b,0xb9, -0x4f,0xd1,0xe1,0x39,0x31,0xd7,0xf3,0xa3,0x9d,0xf7,0xbc,0x95, -0x6a,0x74,0x4e,0x73,0x7b,0x2e,0x92,0x04,0xc3,0x93,0x91,0xc3, -0xa1,0xd6,0xe3,0x20,0x79,0xc8,0x1d,0x00,0x6a,0xb1,0x21,0xc3, -0x4c,0x5d,0x06,0x54,0x92,0x11,0xe8,0xe6,0x16,0xc7,0x6b,0xb4, -0x52,0x94,0xde,0x0f,0x46,0xad,0xfc,0x24,0x2a,0x53,0x52,0xa6, -0xa5,0x84,0x06,0x89,0x91,0x02,0x81,0x7e,0x67,0x0c,0xfb,0x8e, -0x56,0xb4,0x83,0xf8,0xd5,0x72,0xb3,0x7c,0xa2,0x00,0x47,0x04, -0x2b,0x79,0xa3,0x3d,0x84,0x9b,0x9f,0x0c,0x1b,0x6b,0x93,0x1c, -0x16,0x68,0x79,0x4a,0x3a,0x6c,0xac,0x9f,0x4e,0xe0,0x03,0x3b, -0x20,0x05,0xe1,0x63,0xa8,0xd1,0x48,0x61,0x10,0x3a,0xbd,0xe2, -0xb8,0xe1,0x1c,0x2e,0x4d,0x9e,0x2e,0xbb,0x5d,0xc6,0x04,0xea, -0x2d,0x53,0xfa,0xa6,0xc1,0xe0,0x2f,0x11,0x11,0x79,0x3f,0x38, -0x34,0x55,0xd9,0x5e,0xce,0xcd,0x3c,0xb5,0x8d,0x79,0xee,0xe1, -0xe4,0x6e,0xdd,0x44,0x3d,0xda,0x7b,0x10,0x6e,0xd2,0xda,0x9f, -0x1f,0xda,0xd6,0x5a,0xcc,0xca,0xe3,0x61,0xe0,0xb2,0xc8,0xba, -0xdd,0xb6,0x64,0xcf,0x66,0xdc,0x66,0xdb,0x26,0x2f,0xdf,0xf5, -0x37,0xb0,0x9a,0xd6,0x69,0xb8,0x76,0x91,0x95,0x1b,0x8f,0x03, -0x60,0xc5,0xdc,0x81,0x05,0x62,0xc5,0x4d,0x99,0x47,0x9d,0x51, -0x22,0x9c,0xd8,0x4d,0x30,0x0f,0xaf,0x50,0x21,0x18,0x83,0xab, -0xe9,0xa8,0xb8,0x5f,0x94,0x71,0x2e,0xa6,0xc5,0xf3,0x74,0xb1, -0x44,0xe3,0xef,0x4d,0x99,0xf5,0x2f,0xa7,0xd5,0x3c,0xae,0x81, -0x69,0x76,0x93,0x8a,0xf2,0x38,0xd5,0xdb,0x46,0x1b,0x24,0x11, -0xd2,0x80,0xbb,0xdd,0xe3,0x1b,0x21,0x9a,0x5f,0xc3,0xcb,0xd5, -0x5c,0x2d,0x97,0x6f,0x6a,0x77,0x5f,0x8f,0xb2,0x36,0x4f,0x91, -0xda,0xb0,0x34,0x3c,0x8c,0xa8,0x5c,0x26,0xf5,0x8b,0x90,0x33, -0xb0,0x5e,0x98,0x0d,0xd8,0xa8,0x19,0xbd,0xda,0x8f,0xf4,0xea, -0x2c,0x11,0xa0,0xbd,0x0f,0xb7,0x42,0xdf,0x20,0xbd,0xb7,0x54, -0x95,0xbc,0x13,0x15,0x53,0xc4,0xfd,0x02,0x9d,0xf7,0xa8,0x14, -0x75,0xc9,0x85,0x8c,0x6c,0xe1,0x7e,0xf5,0x8c,0x47,0xe8,0x5d, -0x28,0x7c,0x87,0xdf,0x02,0x29,0xe5,0x75,0xbb,0x09,0x5b,0x45, -0x26,0x30,0x78,0x4a,0x41,0x94,0x21,0x1c,0xf4,0x21,0x26,0x93, -0x02,0x73,0xd8,0x07,0xb5,0x12,0x62,0x8f,0x8b,0x85,0xc8,0x84, -0x93,0x72,0xcb,0x90,0x04,0x35,0x21,0x06,0xa5,0x84,0x30,0x53, -0xf6,0x81,0x69,0xc1,0xb1,0x14,0xf0,0x6c,0xc4,0xf9,0x4e,0xce, -0x2e,0xe3,0xd4,0x5b,0xaf,0xb4,0x4a,0x89,0x0a,0x88,0xbf,0xf3, -0xfc,0x8b,0x88,0x24,0x88,0x03,0x01,0xfd,0xd9,0xd5,0xf2,0xd6, -0x6c,0x96,0xaf,0x5e,0xcd,0x4b,0xe6,0xcf,0x8c,0x1d,0xb4,0x20, -0x86,0x53,0x3b,0xd3,0x2f,0xf9,0xf9,0xf1,0xcd,0xa0,0x77,0x57, -0x49,0x7b,0x1b,0x9e,0x87,0xd9,0x00,0x95,0xec,0xe0,0xc4,0x12, -0x25,0xf0,0x7b,0x88,0x4d,0x62,0x86,0x6e,0x95,0xa1,0xed,0x8a, -0x3f,0x4f,0x2f,0x29,0x95,0xa6,0x4a,0xee,0x7d,0x2f,0xdf,0xc1, -0x19,0x8f,0x2f,0x6a,0x79,0x5a,0x6e,0x16,0x54,0x06,0xd5,0x3a, -0x61,0x02,0xec,0x1a,0x02,0xb3,0x4e,0x20,0xba,0xf4,0xf4,0x69, -0x10,0x71,0x82,0x2f,0xdd,0x85,0xad,0x6c,0xb7,0xd2,0x48,0x8a, -0x6a,0xce,0x30,0xaf,0xd7,0x23,0x6c,0xb4,0x08,0x91,0x7b,0x92, -0xc3,0xea,0x70,0xfe,0x93,0x61,0xd2,0xb7,0x37,0xc0,0xbe,0x3a, -0x62,0xc7,0x70,0x22,0x10,0xc8,0x89,0x1a,0xbe,0x24,0x30,0xfa, -0x84,0x2f,0x2f,0x13,0x47,0x6d,0x72,0x65,0x6f,0xd7,0x32,0x14, -0x22,0x54,0x2f,0xe1,0x44,0x77,0x39,0x87,0x93,0xc8,0x98,0x6d, -0x63,0xf4,0xdb,0x84,0x9f,0x7f,0x8f,0xbe,0xff,0x31,0x31,0xd0, -0xd6,0xbc,0xfa,0x90,0x69,0x99,0x9a,0xeb,0xb4,0x15,0xba,0xc4, -0xeb,0x6b,0x2c,0x12,0xdf,0x85,0x73,0xc3,0xdf,0xe0,0xcd,0xea, -0x8f,0x65,0x39,0xab,0xbf,0x9f,0xbe,0x87,0xe3,0x14,0x34,0x25, -0x3d,0xa2,0x5e,0xe1,0x43,0xa6,0x50,0x23,0x86,0xf7,0xb7,0xe5, -0xf2,0xba,0x18,0xe4,0x91,0x89,0x16,0xb7,0x73,0x4c,0x77,0xb7, -0x49,0xc6,0x82,0x9d,0x1d,0x1a,0x1a,0x1a,0x05,0x63,0x2d,0x12, -0xa6,0xdc,0x24,0xb6,0x3c,0xae,0xaf,0xd6,0xd5,0xe2,0xcd,0xdf, -0xd6,0xd3,0x15,0xc1,0x5a,0xb3,0xfc,0xd3,0x32,0xd7,0x51,0x7b, -0x41,0x77,0xa8,0x0f,0x88,0x70,0x1d,0x15,0x18,0xc4,0x05,0xfe, -0x11,0x15,0x78,0x30,0xd9,0xf9,0x06,0x56,0x01,0xc3,0x6e,0x61, -0xbc,0xeb,0xcb,0x77,0x1b,0x52,0x3e,0x78,0x96,0x48,0xec,0x9e, -0xd8,0xe4,0x95,0xb2,0xfa,0x0b,0xfe,0x83,0x66,0xc1,0xf4,0x03, -0xe7,0x93,0xd9,0x04,0xb7,0x5f,0x14,0x62,0x67,0x36,0x42,0x5c, -0x95,0x09,0x1c,0xf4,0x81,0x75,0x00,0x6a,0x51,0xef,0x59,0x2d, -0x6e,0xca,0xa1,0xf0,0x01,0x16,0xf9,0x78,0xbf,0x40,0xfd,0x86, -0xc8,0x72,0x9a,0xcb,0x9f,0xac,0x15,0x45,0xe6,0xd3,0x66,0x11, -0x77,0xf9,0x8e,0xdb,0x8e,0x3c,0x30,0x82,0x4c,0x83,0xce,0xd2, -0x3a,0x6d,0x62,0x31,0xff,0x8c,0xac,0xc7,0xd4,0x1e,0x4e,0x53, -0xfa,0x62,0xec,0x26,0x03,0x45,0x03,0xe7,0x30,0xa7,0x50,0x73, -0x38,0x35,0x77,0x93,0x2b,0x6c,0x62,0x57,0x7d,0xec,0x2d,0xcd, -0x48,0x4a,0xe0,0x6d,0xde,0x9b,0xf5,0xa0,0x20,0xa2,0x85,0xd8, -0x46,0x6b,0x51,0x65,0xc4,0x43,0x1d,0xd2,0x3a,0x72,0x8e,0xb6, -0xa3,0xb6,0x02,0x1f,0xdd,0xcd,0x02,0x93,0xd5,0xbb,0x68,0x22, -0x11,0xc7,0xe2,0x89,0x01,0xb3,0x58,0x8e,0xf7,0xc2,0x1f,0xb4, -0xbe,0xc3,0x92,0x3a,0x0d,0x76,0xee,0xe9,0x84,0x40,0x00,0x7b, -0xa2,0x1b,0x1f,0x23,0xd4,0xf6,0x4f,0x90,0xc5,0x3d,0xe5,0x27, -0x99,0xd7,0x6f,0x11,0x66,0xda,0xf9,0x8a,0x20,0x65,0xc9,0xf6, -0xc8,0x9b,0x25,0xbc,0x1a,0x09,0xaa,0x16,0x04,0x09,0x4b,0x7a, -0xd2,0x33,0xba,0x5f,0x53,0x62,0x89,0x86,0xe3,0x5e,0xb6,0xf7, -0xdb,0xb3,0x53,0xc2,0x66,0x80,0x7a,0x85,0x2d,0x4a,0x02,0xda, -0xaa,0x05,0xce,0x21,0xde,0x29,0x48,0xcf,0x62,0xf6,0xf8,0x23, -0xdb,0xaf,0xaf,0x3d,0xa2,0xe2,0x28,0xe7,0x6d,0x96,0xb8,0xa0, -0x80,0x6f,0x55,0x9b,0xc3,0xb5,0x7d,0x03,0xac,0xc5,0xb7,0xef, -0x52,0xf6,0xe3,0x59,0x5c,0xcc,0x6f,0x66,0x25,0x59,0xeb,0x30, -0x29,0xdc,0x5e,0x55,0x17,0x57,0x06,0x1d,0xb2,0xea,0xe2,0x8e, -0x61,0x26,0xeb,0x75,0x52,0x2f,0xcb,0x5c,0xba,0x3a,0x85,0xff, -0x81,0x63,0xb2,0x96,0x77,0xc7,0x41,0xd3,0xd4,0x6a,0x11,0x58, -0x9e,0x51,0x17,0x63,0xb5,0x7f,0xeb,0x51,0x89,0x49,0x21,0xa9, -0xd6,0x58,0x4d,0x92,0x49,0xe9,0x6f,0xaf,0xe6,0x5c,0xc3,0x30, -0x15,0x54,0x03,0x44,0xd1,0xe9,0x45,0xb5,0x79,0xcf,0xf5,0xfb, -0x6c,0x49,0x43,0x17,0x00,0xdc,0x4f,0x64,0x7d,0x46,0x11,0x4b, -0xca,0x36,0xc7,0xc2,0xda,0xcf,0x24,0xeb,0xb2,0xfc,0x2f,0xf1, -0x26,0x37,0xf2,0xcd,0xc2,0xff,0x5a,0x95,0xb7,0xe2,0x54,0xc7, -0x7e,0x24,0xdc,0x40,0xde,0x19,0xec,0x58,0x59,0x34,0x9d,0xbd, -0xff,0xbe,0xaa,0x37,0x66,0xbd,0x5c,0x6e,0xb8,0x17,0xd3,0xb0, -0xf5,0xd7,0xfb,0x58,0x9b,0x60,0xd4,0xe0,0xa6,0x10,0x35,0xa1, -0x35,0xc0,0x01,0x6e,0xcf,0x0a,0x05,0xc9,0xd0,0x6f,0xf3,0x92, -0x1b,0xd7,0x74,0xe9,0xea,0xe5,0x67,0x9a,0xf0,0x99,0xb9,0x98, -0x4f,0xeb,0xfa,0x01,0x79,0xbb,0xa1,0x9b,0x52,0x18,0x15,0x03, -0xa5,0x24,0x4a,0x12,0x35,0x6b,0x91,0x0c,0xfa,0x7f,0xe9,0x0f, -0x12,0x4e,0x64,0x45,0x5d,0x23,0x92,0x06,0x27,0x73,0x11,0xe4, -0xae,0xcd,0x50,0x1b,0x90,0x28,0xc3,0x45,0x65,0x73,0x23,0x9f, -0x52,0xb9,0x00,0x2d,0xfc,0x9f,0x2e,0x1b,0x45,0x24,0x9d,0x0b, -0x6d,0x96,0xcf,0xe8,0x0a,0xbb,0x70,0x63,0xe9,0x6b,0x1a,0x97, -0xb8,0x9a,0xd6,0x3f,0xdd,0x2e,0xfc,0x7c,0x4e,0x11,0x35,0x88, -0x20,0x1f,0xbd,0x72,0x0a,0x7f,0xb8,0xe4,0xa7,0x23,0x9b,0xac, -0x93,0xc2,0x6a,0xf1,0xba,0x30,0xe2,0x6d,0x10,0x2c,0x41,0x3d, -0xe1,0x2d,0x78,0xf1,0xc5,0x65,0xbd,0x09,0xcf,0x04,0xdb,0xab, -0xc5,0xcd,0x75,0x71,0x7f,0xdc,0x3b,0x9e,0x8c,0xd2,0x51,0x7e, -0x36,0xbb,0x77,0xd6,0xdf,0x66,0x67,0xb3,0x1e,0x7c,0x8c,0xcb, -0x27,0x13,0xca,0x80,0xcf,0x6d,0x76,0xbf,0x5f,0x2f,0x6f,0xd6, -0x8a,0x19,0x6b,0xd1,0x51,0xdc,0x3f,0x7b,0xd6,0xbb,0xff,0xca, -0xac,0x09,0xfc,0xfb,0x2f,0xc6,0x67,0xf5,0xd9,0xcd,0xb7,0x4f, -0xbe,0xfd,0xf6,0xec,0xdd,0xc3,0x93,0x49,0x6f,0x1b,0x7d,0x7f, -0x86,0x45,0x7f,0xbd,0x81,0xa3,0x26,0xda,0x8d,0x40,0x79,0xe8, -0x27,0xfd,0x7a,0x7c,0x76,0x7b,0xf6,0xb7,0x49,0xef,0x34,0x1b, -0xbf,0x38,0x9d,0xdc,0xdb,0xfe,0x29,0x85,0x84,0xe3,0xc9,0xbd, -0x2c,0xfb,0xec,0xbe,0x59,0x23,0xa7,0x98,0x97,0xcf,0xa7,0xaf, -0xa0,0xf4,0xd7,0xe9,0xd9,0x6d,0x2f,0x3b,0xab,0xef,0x9d,0xdd, -0x1f,0x9d,0x42,0xd5,0xaf,0xcf,0xee,0x9f,0x0d,0x4e,0xb7,0x54, -0x0e,0xb6,0xde,0x6a,0x86,0x5e,0x0a,0x35,0x81,0x31,0x31,0xf9, -0xdd,0xee,0xac,0x9e,0xdc,0xb3,0x79,0xe4,0x7f,0x0f,0x99,0x50, -0xef,0xc5,0x36,0xdf,0x9a,0x0c,0x07,0x0c,0x4d,0x8d,0x33,0x1a, -0x00,0x95,0x29,0xeb,0x8b,0xe9,0x0a,0x07,0x75,0x86,0x08,0x48, -0xce,0xce,0xce,0xee,0x9f,0x5f,0x2e,0xd6,0x9b,0xc9,0xf6,0x66, -0x7c,0x36,0x9b,0x1e,0x5f,0x3e,0x3c,0xfe,0x76,0x72,0xf7,0xe5, -0x2e,0xb3,0x35,0x36,0xcb,0x37,0x70,0x7e,0x2f,0xee,0x27,0xe3, -0x17,0x58,0x7c,0x7d,0xb6,0x98,0xdc,0x4b,0xb6,0x18,0x37,0x60, -0x4b,0xb1,0x02,0xc8,0x74,0x72,0x7b,0xcc,0xd8,0xed,0xed,0xc5, -0x2e,0x34,0x77,0x2d,0xae,0x10,0x00,0xfe,0xf1,0x75,0x7d,0x0c, -0x60,0xcf,0xa6,0xf5,0xd5,0xc3,0xf9,0xea,0x6a,0x5a,0xdc,0x3f, -0x4e,0xa9,0xff,0xdf,0x26,0x50,0xb4,0x32,0x97,0xf6,0xb0,0xe9, -0x08,0x03,0x4e,0xfc,0x70,0x64,0xd9,0x6c,0xd0,0x11,0x5b,0x48, -0x82,0x3f,0x43,0x87,0x0d,0x9c,0x77,0x3e,0x57,0xcf,0x5c,0x5d, -0xf2,0x9e,0xcb,0xee,0xac,0x13,0x32,0x3a,0x1f,0x92,0xf7,0x1c, -0xb2,0x09,0xd4,0x83,0xc3,0x66,0x01,0xe2,0xe9,0x8c,0xcc,0x8b, -0xc9,0x3b,0x51,0xa2,0x25,0x69,0x5b,0x74,0xd7,0xac,0x95,0x89, -0xc1,0x3c,0xc3,0x1b,0x0e,0xf6,0x90,0x62,0xf3,0x39,0x63,0x7d, -0xf3,0x20,0x37,0x85,0x8d,0xc6,0x70,0x8e,0x7f,0xc0,0xd8,0xdb, -0xff,0x28,0xf5,0x5a,0xb7,0xbe,0x7d,0x9a,0x9b,0x26,0x8f,0x7f, -0xfa,0x41,0x6e,0xa4,0xbf,0x07,0x30,0xcb,0x59,0xe2,0x46,0x89, -0x5a,0x4f,0x23,0x0c,0xa7,0xb5,0x2e,0x0d,0x2c,0x2c,0x9f,0xe5, -0xae,0x3f,0x86,0x92,0x7d,0x09,0x93,0xe5,0x82,0xc0,0xa7,0xeb, -0x1b,0xa0,0xb5,0xc5,0xab,0xd2,0xab,0x69,0xbb,0x89,0xaa,0x44, -0x1d,0xc0,0xd0,0x87,0x76,0xa1,0xaa,0x48,0x67,0xb7,0xcd,0xe2, -0xee,0x35,0x79,0x57,0xe5,0x3e,0x2b,0xc0,0x05,0xcc,0xc6,0x21, -0xcb,0xb5,0xa8,0x5c,0x0d,0xae,0xf0,0x7c,0x2f,0x5b,0xf0,0x97, -0xba,0xb3,0xbc,0x27,0x1d,0x15,0x39,0x66,0x69,0xf9,0x40,0x15, -0xd8,0xe6,0xde,0x6f,0x0b,0xa2,0x48,0xcc,0x0e,0xcd,0xc9,0xd7, -0x38,0xdd,0x9a,0xe1,0x3c,0x83,0xe0,0x48,0x72,0xda,0x96,0x63, -0xbf,0x35,0x32,0x16,0x94,0x8c,0xd2,0x4e,0x8b,0x2f,0x46,0x63, -0x5c,0x29,0xc6,0x0e,0x04,0xbf,0x26,0xb9,0x63,0x16,0x2c,0x9f, -0x5b,0x70,0x4c,0x87,0x80,0xd9,0x6e,0x3b,0xea,0x26,0x2d,0x66, -0x05,0xc0,0x12,0xe5,0xea,0x41,0x3e,0xad,0x63,0x59,0x9f,0x31, -0x3b,0x4a,0x6d,0x86,0x87,0xa4,0x3e,0x7a,0xb1,0xbb,0xe6,0x73, -0x44,0x48,0xdf,0x43,0xbb,0x56,0x8a,0x0b,0x0e,0x15,0x2d,0x78, -0xdf,0x7c,0xa7,0xb6,0x0e,0xef,0x36,0xea,0xe1,0x86,0x66,0x3b, -0x9b,0xe9,0xe2,0x02,0x71,0xc9,0x5d,0xa9,0x17,0x1d,0x1c,0x75, -0x72,0x9d,0xad,0xc0,0x91,0x8c,0x54,0x08,0x7e,0x70,0x10,0xf4, -0xc0,0xb4,0x9d,0xe8,0x0c,0xdb,0xe1,0x3a,0x21,0x40,0x13,0x02, -0x4f,0x25,0x3b,0xfc,0xdc,0x6e,0xd8,0x1d,0x90,0xf3,0x3c,0xc6, -0xca,0xf6,0x17,0x76,0x10,0xd6,0xc6,0xa6,0xaa,0x7f,0x9e,0x4f, -0xab,0x85,0x06,0x96,0x91,0xf1,0xd3,0xb5,0x33,0x15,0xb6,0x56, -0x1e,0x90,0xda,0x54,0x57,0xe2,0x28,0xc6,0x54,0x6e,0x92,0x8d, -0xbc,0x0f,0x6d,0x48,0xf3,0x18,0xd3,0x28,0x12,0x71,0xab,0x26, -0xca,0x1f,0x7a,0x14,0x8a,0x67,0x24,0x32,0x9c,0xb5,0xab,0xd3, -0x39,0x39,0x7e,0xf3,0xfe,0xe9,0x4c,0x06,0xf1,0x60,0xc2,0xb6, -0xba,0xdd,0xae,0x27,0x04,0xa1,0x93,0x21,0x4d,0x10,0xa5,0x55, -0x33,0x38,0x36,0xdb,0xd2,0xaa,0x9f,0xb5,0xe4,0x10,0x4f,0x32, -0xc1,0x28,0xa7,0xb3,0x01,0xe9,0xa0,0x61,0xf6,0x48,0x04,0xdb, -0x79,0xe0,0xf5,0x75,0xee,0x2d,0xa6,0x29,0x55,0x9b,0xb1,0x0b, -0x83,0x95,0xd8,0xf6,0x86,0x54,0x56,0x82,0x9d,0xc6,0x34,0x68, -0x4b,0x3b,0x0b,0x2a,0x87,0xb0,0x64,0x79,0x13,0xfd,0x16,0xf6, -0x91,0x37,0x2a,0xe6,0xbd,0x8e,0xc8,0xdd,0xd2,0xd4,0x1f,0x2d, -0x41,0xb3,0x9a,0xf0,0xf7,0x43,0x58,0x14,0x52,0xb7,0xee,0x85, -0x28,0x2c,0x59,0x4f,0xdf,0x94,0xac,0xd6,0x0c,0x2a,0xe2,0x1e, -0xa0,0x09,0x79,0x92,0x18,0xbd,0xa9,0x30,0x75,0xf5,0x5b,0x99, -0x7b,0xdb,0x82,0x8f,0x62,0x2e,0xb4,0x83,0x03,0x2a,0xb5,0xd8, -0x52,0xcc,0xc9,0x76,0xac,0x06,0xa7,0xae,0x76,0xe8,0xf7,0xe9, -0x0a,0x83,0xd4,0xe3,0x04,0x27,0x52,0x99,0x41,0x0a,0x51,0x69, -0x5f,0x1a,0x86,0x63,0xec,0xc9,0xa9,0x26,0x8e,0xbd,0xae,0x7b, -0x90,0x38,0x21,0xa2,0x1d,0xe3,0xaf,0x9d,0x41,0x81,0x12,0x76, -0xbb,0x8b,0x37,0x79,0x10,0x20,0xc4,0x5e,0xcd,0x6e,0x8a,0xc6, -0xea,0x0e,0x18,0x0b,0xd3,0x6a,0xed,0x87,0x75,0xe9,0xa3,0xcd, -0x00,0xaf,0x3b,0x56,0x28,0x62,0x9a,0x4f,0x10,0x7d,0xcb,0xe6, -0xcb,0xcd,0xce,0xe0,0xdd,0xb7,0xeb,0x1c,0x07,0x7d,0x0e,0xe0, -0x18,0x38,0xc6,0xd4,0xf1,0x69,0x82,0xae,0xc9,0xa9,0xc9,0xb0, -0xd8,0xce,0x10,0x6d,0xb8,0x56,0x2e,0x17,0x71,0x55,0x2a,0x60, -0xef,0x0a,0x45,0x8f,0x7f,0xb9,0x90,0x2b,0x19,0x43,0x18,0xdf, -0x37,0x6b,0x16,0x47,0xa9,0x37,0x3b,0xd3,0xd5,0x6a,0xfe,0x9e, -0x81,0x01,0x18,0x6e,0x38,0x40,0x05,0x40,0x42,0x26,0x50,0xfb, -0x5a,0x2a,0x7f,0x85,0x9d,0x66,0x67,0x40,0x96,0x3e,0x54,0xe4, -0x18,0x6f,0x3a,0xcb,0x5f,0x5d,0x89,0xca,0x46,0x94,0x2c,0xbc, -0xb9,0x34,0xaf,0x8b,0x5e,0xd5,0x4b,0x4f,0x4e,0xab,0x11,0x24, -0xc0,0xf1,0x77,0xd8,0x0e,0xf4,0xeb,0xd3,0x82,0x03,0x3b,0x9e, -0xbe,0x1e,0x11,0x2d,0x8c,0x5f,0x4f,0x26,0xf9,0x78,0x02,0x9d, -0x5c,0x4f,0x57,0x4d,0xd4,0xef,0x1b,0xbc,0x17,0xd3,0x8c,0x86, -0xdd,0x08,0x01,0xa4,0x24,0x2c,0xed,0x34,0x63,0xff,0xec,0x10, -0x41,0x70,0x84,0xde,0x8b,0x68,0x4b,0x38,0xdb,0x6d,0x83,0xd2, -0x38,0x38,0x17,0x11,0x6c,0x6e,0x0f,0x48,0xa6,0x5e,0xae,0x37, -0x30,0x96,0x3e,0xfe,0x35,0x6c,0x81,0x43,0x9f,0xf4,0x6b,0x67, -0xc2,0x33,0x85,0x27,0xa8,0xd8,0x0c,0x13,0x5c,0x24,0xb9,0x74, -0x4d,0x88,0x34,0x3b,0x18,0x60,0xe2,0x62,0xb9,0x7a,0xff,0x94, -0xaf,0x3a,0xe8,0x37,0xdf,0xbb,0xa8,0x9e,0x80,0xe2,0x82,0x88, -0x2d,0x67,0x61,0x49,0x83,0xac,0xf3,0xc8,0xa6,0x6c,0x60,0x6f, -0xe9,0x35,0xcf,0xd9,0x33,0x94,0xab,0x42,0x6c,0x0e,0x92,0xf3, -0xe5,0x12,0xe3,0x9d,0x39,0x59,0x86,0x5b,0x24,0x99,0x14,0x8a, -0x89,0xe1,0x68,0xa3,0x9b,0x81,0x76,0xf3,0x20,0x33,0x0d,0x97, -0x03,0x2e,0x6d,0x95,0x37,0xfe,0x7e,0x47,0x39,0x19,0x46,0x2a, -0xe4,0x16,0x51,0xe9,0x26,0x70,0x16,0x45,0x85,0xcc,0x94,0xd3, -0x69,0xe6,0x8f,0x8f,0x2b,0x77,0xf5,0x48,0xbe,0xed,0x78,0x9d, -0xc4,0x8e,0xef,0x82,0x06,0x0f,0xa4,0x6a,0xc2,0xdb,0xae,0xf5, -0x42,0x90,0x1b,0x75,0x40,0xa5,0x0c,0x43,0xae,0x6b,0x10,0x97, -0x1a,0x44,0x54,0x7d,0x10,0x28,0x9f,0xdc,0xf9,0x57,0xef,0x65, -0xec,0x28,0x3b,0xd0,0xc7,0x9e,0x9d,0x7e,0xf5,0x1e,0xc7,0xe1, -0x4d,0x52,0x11,0x5d,0x4f,0x51,0x91,0x2c,0x1b,0xf9,0x65,0x82, -0x0f,0x98,0x04,0x9e,0x46,0xb4,0xe3,0xf5,0x24,0x0a,0xd9,0x06, -0xd6,0x17,0xd9,0x08,0xfe,0xc1,0x25,0x94,0xb7,0x15,0xf3,0xc1, -0xb1,0x85,0x61,0x52,0xfc,0xc1,0x46,0x61,0x4e,0xc8,0x7a,0x99, -0x29,0x87,0xa1,0xcb,0xf1,0x4f,0xbc,0x9f,0xf9,0xf5,0xb9,0x98, -0x73,0xff,0x27,0xd3,0x61,0x13,0xd9,0x3c,0x2c,0x96,0x70,0xb0, -0xb8,0x84,0xa5,0xe0,0xb1,0x1b,0xb2,0x35,0xd6,0x65,0xa7,0xba, -0x0c,0xab,0xd3,0x87,0x6e,0x6c,0xda,0xcb,0xcf,0x32,0xc3,0x08, -0x0f,0x74,0x20,0x2d,0x65,0x25,0x43,0x54,0x26,0xba,0x75,0x62, -0x6c,0xcf,0x5f,0x88,0x31,0x03,0x3e,0x89,0x01,0xff,0x6d,0x0a, -0x62,0xff,0xc0,0x5c,0x2d,0xe7,0xb3,0x5f,0x42,0x8e,0x8d,0x49, -0x78,0x91,0x32,0x9f,0x8d,0x7c,0x8e,0x8d,0x15,0x7a,0xbd,0x3c, -0x90,0x00,0x3a,0x27,0x4d,0x86,0x7f,0x0b,0xe5,0x48,0x3e,0xc2, -0x1f,0x6c,0xd7,0xd7,0x39,0x3e,0x8e,0x5b,0xca,0x5d,0x1c,0x38, -0xea,0x5e,0x2c,0x68,0x55,0x1c,0x23,0x1f,0xa5,0xa6,0xe9,0x97, -0xdf,0x4a,0x36,0x0c,0x5b,0x40,0xc7,0x04,0xec,0x52,0x8c,0x03, -0x9b,0x5d,0x9e,0x9e,0x00,0x2d,0x5a,0xa5,0x55,0x60,0x54,0x60, -0xe5,0x2c,0x09,0x1b,0x33,0xc9,0x3c,0x76,0x05,0x07,0x99,0x57, -0xaf,0xd0,0x7e,0xd4,0x79,0x59,0x89,0xeb,0x98,0xe4,0xa4,0x09, -0xb5,0x9a,0x64,0xe8,0x9c,0x69,0x3f,0xb2,0x1d,0x22,0x5d,0x97, -0xb5,0xc3,0x0f,0x45,0xcc,0x13,0xa7,0x05,0x2f,0x5e,0x6a,0xc3, -0x87,0x08,0x6b,0xb3,0x68,0x42,0xff,0x2a,0xcd,0x6f,0xb7,0xad, -0x2d,0xd9,0xf0,0xc3,0x6d,0xcd,0x88,0xde,0xae,0xad,0xe2,0x11, -0xf3,0x0a,0x48,0xe9,0x76,0xe1,0x1f,0x0e,0x70,0xcc,0xa4,0x44, -0xf1,0x60,0x61,0xa4,0xeb,0xea,0xa2,0xb5,0x6a,0x07,0xb2,0xa7, -0x3f,0xa6,0x9e,0xcf,0x24,0xb9,0x3e,0xe1,0xdd,0xe3,0xb7,0xc8, -0xe2,0xb5,0x7f,0x04,0x65,0x7f,0xdf,0xd4,0xe3,0x88,0xd5,0x5c, -0x94,0x99,0x37,0x78,0x25,0x7c,0xb7,0xc6,0x96,0xc5,0x8a,0x4e, -0x19,0x36,0x0e,0xb4,0x68,0xbc,0xe3,0x61,0x7b,0x13,0xe7,0xa9, -0x95,0xbb,0x8a,0x38,0x38,0x8f,0x39,0x44,0xf0,0x21,0x2d,0x72, -0xaf,0xfb,0x7d,0xbc,0xb6,0x5b,0x3f,0x16,0xb4,0xc7,0xc9,0xbd, -0xa8,0xd1,0xce,0x78,0x0d,0x2d,0x24,0xa1,0x55,0xac,0xe2,0xed, -0xa4,0xdd,0x6e,0xc7,0xd3,0xec,0x59,0x98,0x4d,0xe2,0x95,0xc1, -0xab,0xbc,0xd6,0x52,0x7e,0x43,0x6e,0x3b,0x35,0x09,0x0c,0x4c, -0x3f,0x7e,0xba,0x4c,0x1c,0x0c,0xb1,0x75,0xaa,0xa8,0x71,0xdf, -0x94,0xef,0x7d,0xc7,0x0b,0x3f,0x10,0x36,0x24,0x85,0x91,0x4c, -0x5a,0x81,0x85,0x52,0x44,0x65,0x5e,0x3c,0xc7,0x08,0x9d,0x7b, -0xe3,0x43,0x36,0x23,0x3b,0x8a,0x79,0xac,0xad,0x7f,0x5d,0xbf, -0xca,0xee,0xc8,0x30,0x97,0x14,0x93,0x4f,0x30,0x97,0x12,0x41, -0xfc,0xd0,0x23,0x6d,0x1e,0xb8,0xb5,0x59,0xcd,0x05,0x06,0xcb, -0xe3,0x38,0x49,0x9e,0x7d,0x3e,0x4c,0x69,0x9b,0xa7,0x50,0xe6, -0x51,0xe4,0xb0,0xb9,0xe1,0xdb,0xa3,0x72,0xea,0x35,0xaa,0x27, -0x73,0xed,0xb0,0x40,0x05,0x51,0x74,0x6a,0xdf,0x6e,0x95,0x5b, -0x90,0x0f,0x1c,0x81,0x3c,0x2b,0xbc,0x23,0x33,0x29,0x23,0x08, -0x02,0x53,0x4b,0xb3,0x1d,0xaf,0x8f,0x6e,0x77,0x6c,0xbd,0x77, -0xb9,0xf2,0x68,0x6c,0x43,0xde,0x04,0x9e,0x9e,0x9c,0x8b,0xe7, -0xee,0x49,0x2e,0x1f,0x4a,0xb2,0xe4,0x67,0x68,0xe3,0xf9,0x91, -0xa3,0x93,0x3d,0xfd,0x6b,0xaf,0xb6,0x7b,0xcb,0xe6,0x34,0xc3, -0xf9,0x7d,0x86,0x51,0x6a,0x26,0x3c,0x01,0x33,0x2f,0xf4,0x4e, -0xa6,0xb3,0x82,0xe1,0x3b,0xc3,0x59,0x89,0xb7,0x39,0x2c,0x61, -0x37,0x33,0xfc,0x60,0x0d,0xc5,0xa8,0x91,0x22,0xc1,0x4c,0x99, -0x55,0x70,0x1c,0x54,0x32,0x72,0x6e,0x0d,0xb9,0x2a,0xf7,0x8c, -0x45,0x10,0xa0,0x6a,0xaa,0x81,0x9c,0xbb,0x5d,0x4f,0xb9,0xeb, -0x02,0xa8,0xba,0x28,0xb0,0x9e,0xea,0xd6,0x24,0xff,0x3d,0xc9, -0xa2,0x1c,0x56,0xd1,0x9a,0x64,0xd2,0xc8,0x61,0x95,0xb0,0x49, -0x30,0x26,0xc7,0x08,0xe9,0xd4,0x4a,0x73,0x89,0x0c,0x9b,0x3d, -0x3f,0x32,0x38,0x0d,0x5a,0x73,0x29,0x22,0xe5,0xe4,0xe9,0x82, -0x5a,0x38,0x22,0x94,0x69,0x31,0xc3,0x0e,0x14,0x8a,0xce,0xbf, -0xc7,0x34,0xce,0x4b,0xea,0xdd,0xf5,0x1c,0x6d,0xf1,0x87,0x9f, -0x42,0xdb,0xc8,0x87,0x04,0xc7,0x8f,0x7f,0xfa,0xe1,0x67,0x32, -0x4c,0x1f,0x91,0x99,0x3f,0x82,0x6d,0x93,0x0c,0xb4,0x8d,0x66, -0xf5,0x3c,0x07,0xdf,0xc2,0x59,0x4d,0x38,0x34,0xad,0x30,0x8a, -0x62,0x75,0x1f,0x8a,0x24,0xa8,0xe1,0xc4,0xa2,0x58,0xf9,0x90, -0xb9,0x37,0x34,0x9c,0x64,0xd8,0x68,0x7f,0x5a,0xbf,0x5f,0x5c, -0x14,0x12,0xff,0x96,0x52,0x50,0xad,0x09,0x45,0x52,0x09,0x2a, -0x67,0xb9,0x14,0xb6,0xdb,0x88,0x47,0x0b,0x89,0xdd,0x2e,0xd6, -0x8a,0x1c,0xc0,0x81,0x47,0x62,0x6a,0x6b,0xac,0xaa,0xc0,0xfe, -0xde,0x59,0x7e,0xb4,0xcf,0x03,0xe2,0xda,0x4e,0x03,0x34,0xb9, -0x33,0x18,0xf4,0xdc,0x3f,0x26,0xed,0x8c,0x0b,0x43,0x18,0x4f, -0x0b,0x93,0x59,0x83,0xfc,0x9c,0x78,0x86,0xeb,0x9d,0xd7,0xb6, -0xb7,0x91,0x73,0x5d,0x2d,0x01,0xcd,0x32,0x73,0x95,0x67,0x0e, -0x28,0x77,0x97,0x71,0xa9,0x9d,0xb1,0x2a,0x7b,0x4f,0x69,0x4b, -0x73,0xe3,0x6c,0xe5,0x79,0x07,0x74,0xc1,0x8d,0xe5,0x42,0xc0, -0x24,0xd7,0xf5,0xb1,0x4f,0xbb,0xf6,0x5e,0xc0,0xbb,0x08,0xc8, -0x70,0xc0,0x6c,0xed,0x11,0xea,0x21,0x24,0xe2,0x93,0xef,0x11, -0xee,0x02,0xb8,0x05,0x9f,0xa1,0x17,0x8c,0x84,0xb3,0x08,0x13, -0x63,0x5d,0x03,0xee,0x23,0x91,0xbe,0xc1,0xd9,0xd2,0x55,0xce, -0x3a,0xd5,0x7b,0xc2,0x41,0xe4,0xa1,0x22,0x7e,0xf1,0x01,0x97, -0x03,0xb7,0x40,0xa1,0x5f,0x29,0x53,0x02,0x95,0xca,0x31,0x49, -0x03,0x04,0x15,0xf6,0x78,0xcc,0x1a,0x04,0xf4,0xb7,0xc6,0xdb, -0x5b,0xac,0x6c,0x34,0x68,0x1c,0x30,0x75,0x3a,0x56,0x39,0x17, -0xee,0xca,0x8b,0x6e,0xfc,0x91,0xed,0x14,0xd4,0x0e,0xbb,0x9f, -0xa8,0x2f,0xf9,0x47,0x82,0xa6,0xdb,0x2c,0xb6,0x58,0x19,0xfe, -0xf1,0x3b,0xa1,0x3b,0xd0,0x94,0x07,0xa0,0x0d,0x8d,0x85,0x42, -0x12,0x92,0x70,0x6e,0xaf,0x15,0x55,0x10,0xc1,0xdf,0xdc,0x5a, -0xf2,0xff,0xfc,0x9f,0xff,0xd7,0xff,0xfd,0x7f,0x24,0xd9,0xc8, -0xce,0x64,0x78,0x97,0x48,0x5c,0x9b,0x62,0x1b,0x26,0x49,0x1e, -0xd5,0xa5,0x92,0xbb,0xfc,0x63,0x6a,0x52,0x5e,0x2f,0xf1,0x69, -0x97,0x2e,0x35,0x21,0x05,0x35,0x27,0xa2,0x15,0x74,0x4d,0x81, -0x34,0x6c,0xd8,0x48,0xd6,0xd3,0x9e,0x49,0x02,0xb9,0x8e,0x07, -0xc2,0x2f,0x94,0xa6,0x08,0xe7,0x8e,0x8e,0x84,0x87,0x41,0x46, -0x16,0x05,0x67,0xc3,0xc7,0x28,0x1a,0x5b,0x0f,0x94,0x1b,0x8d, -0xe1,0x9f,0x49,0x8e,0x35,0x9c,0x4a,0x84,0xc7,0x89,0x55,0xa8, -0x25,0x56,0xb2,0x49,0x80,0xdf,0x68,0x71,0x21,0xc4,0x4e,0xb7, -0xc4,0x34,0xbc,0x96,0x7b,0x00,0x77,0x83,0x9c,0xf9,0xaa,0x49, -0x49,0x13,0xf3,0x61,0xa8,0x2f,0xca,0x1f,0x92,0xb2,0x50,0x41, -0x05,0x69,0x76,0xb1,0x14,0xd5,0x08,0xb5,0x53,0x9e,0x43,0x04, -0xe4,0xf4,0xaa,0x2c,0xaf,0x72,0x32,0x5d,0x71,0xca,0x03,0x22, -0x20,0x42,0x09,0xfc,0x03,0x14,0x22,0x26,0x02,0xf6,0x49,0x0a, -0xe1,0xc4,0xc7,0x03,0xc0,0x3c,0xa2,0xc4,0x53,0xf5,0xa1,0xbe, -0xcd,0xd4,0x25,0x88,0x16,0x62,0x5b,0x3e,0x2f,0xf8,0xcb,0x81, -0x41,0x65,0x9c,0xd6,0x8c,0xac,0xc9,0x9b,0x6f,0xa1,0xc8,0xa2, -0x38,0x7d,0x3d,0x7c,0x0d,0x40,0x51,0x1d,0x72,0xdd,0x97,0xe6, -0xc6,0xaf,0x27,0x8e,0xe0,0x87,0x36,0xcd,0x3f,0xa3,0x0f,0xdb, -0x6a,0xa1,0xf3,0xbf,0xf5,0x84,0x74,0x70,0x14,0x15,0xab,0x0a, -0x81,0xb1,0x03,0x79,0x45,0xda,0x57,0xc7,0x96,0xaa,0xc5,0x5b, -0x3f,0xb6,0x36,0xce,0x26,0xc5,0x5c,0x0b,0xe3,0xad,0x59,0x0f, -0x0f,0xb6,0x27,0x79,0x5b,0x74,0x3a,0xf0,0x6f,0xa8,0xa2,0xb1, -0xc1,0xb9,0xb5,0x6d,0xee,0x8a,0xd6,0x25,0x1a,0xdf,0xa2,0x6f, -0x1a,0x97,0x01,0x91,0x05,0xb5,0xb8,0x1a,0x88,0x90,0x23,0x18, -0x3a,0xfd,0x6e,0xa4,0x36,0x8c,0x00,0x06,0x06,0xb4,0x87,0x8d, -0xfa,0x80,0xb6,0x32,0x52,0xd6,0x24,0xcb,0x10,0x87,0xfb,0xb8, -0x15,0x0d,0x26,0x64,0x32,0xde,0x48,0xa8,0x7f,0xc3,0x4b,0x4c, -0xc3,0x6e,0x42,0x83,0x63,0x1c,0x11,0xb7,0x20,0xf6,0xb9,0x31, -0xf7,0xe2,0xce,0xff,0x15,0xed,0xfa,0xcb,0x85,0x0d,0x3a,0x84, -0x45,0xc3,0xbc,0x41,0x1e,0x6a,0xf3,0x6f,0xaa,0x59,0x3e,0x40, -0x8b,0xa2,0x77,0x81,0xd2,0xda,0x99,0x45,0x50,0xfc,0x67,0x60, -0xe5,0x5c,0x86,0x04,0xae,0x30,0xe4,0x40,0xcb,0x29,0x01,0xe5, -0xa9,0xcb,0x85,0x0b,0xe4,0xa8,0xa7,0x02,0x68,0xf7,0x72,0x81, -0x72,0x55,0x9b,0x5d,0xfe,0xe5,0x22,0x1b,0xd1,0xce,0x55,0xc4, -0x37,0x0f,0x56,0x81,0x67,0x1e,0x64,0x0c,0x46,0xd1,0x54,0xdb, -0x5e,0x2e,0x64,0x6c,0xf6,0xe8,0xa1,0x1a,0x71,0x35,0x66,0x49, -0xf7,0xb6,0x9b,0x91,0xe4,0x8e,0x0d,0xd3,0x9b,0x31,0x00,0x67, -0xf0,0x37,0x7a,0x50,0x86,0x4b,0x7a,0x21,0x3a,0x77,0x66,0x4a, -0xfe,0x87,0x31,0x25,0xc2,0x70,0x31,0x0c,0x00,0x93,0x1f,0x48, -0xdd,0xd5,0x02,0x43,0x30,0x19,0x32,0xb6,0xfd,0x77,0xe0,0x8a, -0xeb,0xe9,0xad,0xf8,0xa0,0xec,0x21,0xcd,0xf3,0x9b,0xf9,0x9b, -0x82,0xb7,0x82,0x37,0xfc,0xb6,0x8e,0x17,0xea,0xd6,0x3f,0x90, -0xe3,0x19,0x34,0xbb,0xb3,0x5d,0xa0,0x03,0x8a,0x25,0x27,0xcc, -0xd3,0xe8,0x03,0x04,0xa6,0x83,0xae,0x42,0xf8,0x24,0x56,0x58, -0x00,0x95,0xdd,0xa5,0x75,0xa3,0x0d,0x7c,0xdb,0xbd,0x6e,0x5a, -0xe6,0x91,0x69,0x0f,0x75,0x4d,0xd3,0x5b,0x74,0x31,0xa5,0x51, -0x20,0x8d,0x4e,0x6f,0x47,0x30,0xc9,0x4e,0x17,0x5f,0x8b,0xbf, -0x1e,0xd2,0x04,0xa9,0xd4,0xf3,0x94,0x06,0xcc,0x54,0x12,0xee, -0x0f,0x16,0x8f,0x76,0xbe,0xb1,0x28,0xb7,0xe5,0x5b,0x44,0x4a, -0x21,0xb4,0x60,0x06,0x7a,0x6a,0x59,0xae,0x97,0x0b,0xb7,0x94, -0xb0,0x51,0x84,0x8a,0xea,0x70,0xb8,0x3b,0x0f,0x3a,0x5e,0x6c, -0x51,0x79,0x2f,0xf2,0x99,0x45,0x03,0x19,0x8d,0xd5,0x39,0x02, -0x34,0x0a,0xc6,0x97,0xe5,0xe2,0x62,0x66,0x1b,0x39,0xe1,0x46, -0x72,0x45,0x36,0x0a,0x9b,0xb7,0xcd,0x3b,0x88,0x94,0x0e,0x23, -0x68,0xf8,0x81,0x12,0x3d,0x6a,0xff,0x40,0x64,0xdc,0x85,0xa6, -0x1f,0x7a,0x83,0x54,0x34,0x55,0x37,0x56,0xc9,0xc7,0xfe,0x37, -0xf2,0xd1,0xf4,0x4d,0x34,0x1f,0xb4,0x3b,0xd9,0xab,0x7b,0x54, -0xfa,0xd8,0x6b,0x69,0x92,0xed,0xcd,0xf9,0x68,0x2b,0x93,0x66, -0xc5,0x16,0x13,0x13,0x8e,0x4a,0xe0,0x3a,0xdb,0x7c,0xb2,0x95, -0x49,0x54,0x25,0xb6,0x32,0x21,0xc5,0xc5,0x66,0x49,0xd7,0x21, -0x78,0x7a,0xc4,0x9f,0xbc,0x24,0xa5,0x3e,0x45,0xb7,0xb2,0xc7, -0x30,0x67,0xef,0x12,0x1e,0xd0,0x3c,0x57,0x5f,0x68,0xa1,0xdb, -0x85,0x7f,0xa0,0x08,0x1c,0x85,0x96,0x73,0xd8,0xe5,0x3a,0xce, -0xbd,0x42,0xd2,0x28,0x26,0x2d,0x7b,0x82,0x36,0x54,0xc5,0x02, -0x86,0xad,0x0f,0x88,0x29,0x2f,0x37,0x49,0xc3,0xff,0xda,0x57, -0x1d,0x07,0xed,0x9a,0xaf,0xe0,0x6c,0xbd,0xc7,0xa0,0x68,0xb7, -0x4b,0xed,0xf3,0x0d,0x4e,0x5b,0xac,0xd7,0x95,0xac,0xd2,0xf4, -0x2f,0x41,0x93,0x6f,0x58,0x5b,0x74,0xc4,0x31,0x2b,0x8f,0xf8, -0xa0,0x6c,0x35,0x00,0x47,0xb4,0x6f,0x12,0x31,0x1f,0xfd,0x52, -0xbe,0x7a,0xf2,0x6e,0x75,0xc4,0xd2,0x25,0xeb,0xb2,0x12,0xdf, -0xb1,0xde,0x8b,0x77,0xc5,0x87,0x2d,0x4f,0xbb,0x99,0x8c,0x99, -0xf3,0x1d,0xb9,0x00,0x51,0x93,0xb6,0x83,0x55,0xe6,0x59,0xec, -0x14,0xb1,0xc6,0x9a,0xe6,0xd2,0x7f,0x1b,0x0e,0xc3,0xcc,0xcb, -0x60,0x1e,0xc9,0x4e,0x5b,0x7b,0x0b,0x4a,0x7d,0x83,0xf5,0x26, -0xa9,0xb1,0xe5,0x49,0xc6,0xa8,0xf5,0xbd,0x39,0xf5,0xb0,0x8c, -0x1f,0xaa,0x6b,0xba,0x83,0x6a,0xce,0x50,0x02,0xa8,0xa0,0xc9, -0xe3,0x75,0x79,0xbd,0x5c,0xbf,0x47,0xc9,0x0c,0x08,0x9d,0xd3, -0xbe,0xe7,0x5d,0x81,0x3f,0x9e,0x92,0x73,0x0d,0xff,0x7e,0x46, -0x36,0xca,0xf4,0xd8,0xc5,0x18,0xcd,0xfa,0x61,0x18,0x45,0x47, -0xfd,0x38,0x61,0xef,0x2b,0x51,0x8f,0x66,0x42,0x37,0x14,0x3e, -0x74,0x93,0x11,0x0a,0xf5,0x54,0xd8,0x77,0xf0,0xe8,0x93,0xe3, -0x5f,0x70,0xf7,0xc8,0xe7,0xbd,0x4e,0x0b,0xaf,0x53,0xf4,0x00, -0xf6,0x3e,0x8b,0x93,0x00,0xd4,0xc2,0x7b,0x1b,0x43,0x32,0xc8, -0x37,0x12,0x52,0xbb,0x5d,0xbf,0xe0,0xa9,0xd7,0xfc,0xd0,0xfb, -0xcd,0x22,0x39,0x96,0x1f,0x7b,0xa9,0x13,0xd9,0xe9,0x11,0x42, -0x0d,0x7c,0x80,0x9a,0x3f,0x3a,0xc5,0x75,0xbb,0x3a,0x10,0xf4, -0xc6,0xfc,0x89,0x5f,0xbf,0xc9,0xee,0x64,0x90,0xb0,0x88,0x6d, -0x80,0x11,0x82,0x66,0x60,0x18,0x9a,0x94,0xb0,0x36,0xa2,0x7f, -0xfb,0xfa,0x18,0x21,0x39,0xc4,0x70,0x92,0x44,0x7a,0xc9,0x72, -0x6e,0x68,0x24,0xc8,0xce,0xeb,0x72,0x7e,0xd9,0x97,0xb8,0xf5, -0xa9,0x58,0x62,0x5c,0x16,0x77,0xc0,0xbb,0xf2,0xd0,0x9f,0x1b, -0x2b,0xc8,0xc5,0x2d,0x61,0xca,0x7f,0x36,0xa4,0xe3,0xc7,0x33, -0x93,0xd3,0xbb,0xbf,0xc6,0x48,0xfe,0xf2,0x22,0x4a,0x59,0xc9, -0xd6,0xbf,0x40,0x26,0x41,0x00,0x73,0x86,0xc1,0xbd,0x0a,0x26, -0x2b,0x7d,0xf6,0x6f,0x16,0xd5,0xaf,0x28,0x30,0x12,0xd4,0x57, -0xd3,0x9a,0xca,0x6f,0xb7,0xfc,0x3c,0x09,0x4a,0xd9,0xf8,0x0d, -0xa7,0xb8,0x57,0x78,0xfa,0x79,0x65,0xf1,0xe0,0xf4,0x6a,0xf2, -0x96,0xa3,0x80,0x89,0xbe,0x42,0x9e,0x14,0x25,0x53,0x3c,0xda, -0x47,0x02,0xb9,0x92,0x56,0xea,0x93,0x0c,0x1b,0xd8,0x13,0xaa, -0x39,0x3f,0xcb,0x7c,0x63,0x20,0xbc,0x60,0x43,0x05,0x6c,0xcb, -0x4d,0x3d,0xcf,0x5c,0x84,0xa8,0x9b,0xe8,0x91,0x19,0xc1,0x96, -0x0d,0x42,0x4b,0x24,0x26,0x5e,0x0b,0xca,0x58,0xe5,0xcd,0x19, -0x28,0x48,0xd4,0xc0,0x8e,0x6b,0x59,0x76,0x7a,0x3c,0x18,0x66, -0x04,0xbf,0x06,0xd4,0xa0,0x35,0x37,0xd0,0x81,0xda,0x81,0x08, -0x11,0x17,0x94,0x1f,0x52,0xf6,0xf1,0xb1,0xbf,0x74,0xa2,0x22, -0x94,0x76,0x7c,0x9c,0xed,0xd4,0xf8,0x03,0xa6,0xa4,0xd5,0x74, -0xe4,0x52,0x1d,0x44,0x2c,0xac,0x20,0x24,0x11,0x45,0x01,0x8c, -0x79,0x27,0xed,0xe0,0xef,0xed,0xb6,0xe3,0x21,0x1b,0x75,0x4c, -0x28,0x65,0xec,0xc1,0x1b,0x32,0x0a,0xee,0x53,0x88,0x77,0x5f, -0x39,0xe6,0x26,0xb2,0x7c,0x9c,0x15,0x7b,0x50,0xb7,0xc5,0x8c, -0x82,0x40,0xd9,0xa1,0x4d,0xfb,0x9b,0x96,0x96,0xb9,0x51,0xd7, -0x1a,0x37,0x8f,0x01,0xca,0xfc,0xd5,0x24,0x9d,0x60,0x1b,0xad, -0x5d,0x50,0x2b,0x64,0xe7,0x42,0xd7,0x98,0x9e,0xfd,0x88,0x28, -0xf7,0x03,0xcb,0x1d,0x3a,0x5a,0xe0,0x3f,0x14,0x9f,0x8d,0xbe, -0xc6,0x7e,0x41,0xb6,0x58,0x1f,0xb9,0x9f,0x29,0xad,0x84,0x1a, -0xa4,0x63,0xc6,0x2e,0xf1,0x42,0xd8,0xab,0xa9,0x5b,0x8e,0x49, -0x84,0xd4,0xce,0xfc,0x41,0x57,0x0f,0xb0,0x77,0x22,0x65,0xfa, -0xa9,0x23,0xc0,0x94,0x36,0x2c,0xe0,0x68,0x15,0xfa,0xd8,0x80, -0xc7,0xab,0xda,0x36,0xf6,0x0e,0x65,0xec,0xec,0x8b,0xa6,0xd8, -0x56,0xe3,0xd2,0x5d,0x45,0x3d,0x8f,0xa6,0x28,0xca,0x21,0xb1, -0x8e,0x9b,0x15,0xc6,0xce,0x1d,0x8f,0x13,0xb9,0x09,0xc6,0xf0, -0xa3,0xe8,0x54,0x67,0xe2,0x0d,0x11,0xa5,0xa2,0x8b,0xf2,0x88, -0xe7,0x08,0xe3,0x90,0x4b,0x85,0x59,0x32,0x31,0x58,0x9b,0x0e, -0x24,0x26,0x41,0x4f,0xe2,0x8f,0xaa,0xfc,0x9a,0xa2,0xab,0x51, -0x65,0x0e,0xde,0x00,0x95,0xd5,0xad,0xb9,0xad,0x01,0xad,0x3b, -0xa1,0xad,0x6d,0x53,0x16,0x09,0x46,0xf3,0xa4,0xc0,0xbc,0x2a, -0x03,0x4b,0x20,0x98,0x56,0x42,0xdb,0x94,0x70,0x3e,0x23,0xe7, -0xb9,0x96,0x7c,0xeb,0x34,0xcf,0xde,0xd4,0x16,0xfb,0xe2,0x2b, -0x1d,0xcf,0xc6,0xe6,0xca,0x8f,0x7c,0x20,0x01,0xcf,0x7c,0x13, -0x93,0xe8,0x81,0x59,0x2b,0x6b,0x3b,0x8b,0xb9,0xf2,0x96,0x12, -0x43,0x06,0xcf,0x93,0xe1,0x0b,0x40,0x94,0x22,0xa7,0x6f,0x0e, -0xb3,0x42,0x29,0xb8,0xdd,0x39,0x6b,0xe7,0xe0,0xf4,0x4c,0xca, -0x11,0xe0,0x2c,0xf4,0x63,0xa8,0x03,0x1b,0x73,0xb5,0xc1,0x64, -0x12,0x7b,0x86,0x31,0xa4,0x68,0xbe,0xbe,0xc0,0x5a,0xed,0x66, -0x64,0x43,0x2d,0xe5,0x19,0x95,0xd8,0x3e,0x35,0x4f,0xa5,0xc3, -0x6c,0x14,0xa7,0xa8,0x79,0x9b,0x8e,0x5a,0xad,0x0e,0x04,0xbf, -0x5e,0x32,0x3d,0xdb,0xe8,0xbc,0xde,0x6d,0x0e,0x53,0x48,0x96, -0x6b,0xc2,0x98,0xb1,0x81,0x31,0xa9,0x37,0x57,0xc9,0x84,0xe0, -0x65,0xc7,0x2b,0xec,0x6f,0x64,0xeb,0x59,0x00,0x72,0xb6,0x10, -0x5b,0x8c,0xc6,0x0a,0x1c,0xaa,0x29,0x75,0x7c,0xe8,0xec,0x6a, -0x70,0x06,0x51,0xa4,0xdf,0x65,0xae,0xda,0x4e,0x69,0xeb,0x03, -0x66,0x03,0xa3,0x70,0xc9,0xa1,0x22,0x5f,0xb1,0x91,0xcb,0x8f, -0xdd,0xce,0xc5,0x87,0x71,0x4f,0x10,0x4b,0x66,0x7f,0x55,0xad, -0x4a,0x05,0xbf,0x8f,0xf4,0x65,0x3e,0x9a,0x30,0x88,0x3f,0xf3, -0xfc,0x3e,0x90,0xb5,0x21,0x3e,0x34,0x9c,0xf8,0xc5,0x64,0x28, -0x0d,0x3b,0x2a,0xe0,0xfd,0x18,0xf6,0x6f,0xbf,0x7c,0xb7,0xab, -0xa9,0xa1,0x7f,0x29,0x2e,0x36,0xaf,0xd8,0x8e,0x3b,0xaf,0xc7, -0x83,0x17,0xd5,0x04,0xba,0x54,0x4e,0xad,0xc9,0x0f,0x28,0x11, -0xd9,0x75,0x66,0x22,0xfa,0x3b,0x81,0x9e,0xf7,0xaf,0x3e,0x5b, -0x28,0x9e,0x56,0x2d,0x30,0xd2,0xc9,0x68,0x67,0x92,0x7b,0xdb, -0xe1,0xd1,0x2a,0x8b,0xe5,0x70,0x17,0x8c,0x75,0x99,0x67,0xad, -0xc9,0x87,0x0f,0x58,0x07,0xf0,0x2f,0x9f,0xe0,0x35,0xc7,0xb8, -0x22,0xfa,0x6b,0x67,0x6e,0x03,0x3e,0x50,0xdf,0x60,0xa8,0xe7, -0x6a,0x81,0x27,0x66,0x75,0xba,0x24,0x32,0xa6,0x87,0x6b,0x48, -0x73,0x46,0x9f,0x8f,0x78,0xb7,0xa1,0x08,0x70,0xb8,0x0a,0xec, -0x37,0xea,0x7f,0x24,0x8d,0xab,0xec,0x57,0x81,0x59,0x5b,0xb8, -0xa0,0xbc,0xca,0xd8,0x2e,0xf4,0x0f,0x3e,0xd7,0xa8,0x37,0x7d, -0x1e,0x7c,0x6d,0xcb,0xd8,0xcb,0x76,0x2b,0xd9,0x1a,0x11,0x5b, -0xd2,0x45,0x8f,0x70,0xdb,0xfe,0xc8,0xab,0x94,0x37,0x15,0x0b, -0x37,0xab,0x19,0x86,0x96,0x81,0x1e,0x0a,0x8f,0x74,0x2f,0x74, -0xbc,0xfc,0x3a,0x42,0x33,0x2c,0xaf,0x68,0x78,0xb4,0x1c,0x6a, -0xdf,0x69,0xc2,0xf5,0x35,0x85,0x49,0xc3,0x52,0xf1,0x74,0x30, -0xda,0x8f,0xaa,0xdc,0x73,0x26,0x16,0x26,0xe1,0x4d,0xcb,0xa8, -0x2d,0x84,0x50,0x0c,0x62,0x7e,0x7c,0x6c,0xc7,0xbc,0xdd,0xb6, -0x46,0x64,0x89,0xab,0xfc,0x17,0xd8,0x94,0xad,0x0f,0xf2,0xe9, -0x20,0x13,0xd7,0x5a,0xaf,0x67,0xbe,0x2e,0x26,0x31,0x4e,0x64, -0xb5,0x06,0x89,0xb4,0x14,0x89,0x88,0xa6,0x59,0x22,0x56,0xb2, -0x7b,0xf4,0x01,0xa8,0x6b,0x67,0xe0,0x61,0x19,0x9f,0x91,0xb7, -0xe7,0x28,0x43,0x77,0x13,0x0c,0xf3,0x1a,0x93,0x73,0x50,0x39, -0x13,0x5e,0x1f,0x85,0xa7,0xf1,0x78,0x7d,0xd0,0x56,0x63,0xad, -0x84,0xa8,0xcb,0x82,0x19,0x71,0x57,0x01,0x87,0xa7,0xe8,0x30, -0x80,0xa6,0x11,0xc0,0xc9,0x57,0xb6,0x89,0x5b,0x7e,0xc3,0xfa, -0x96,0x93,0x0d,0xba,0x8b,0x4d,0x0d,0x3d,0x75,0x25,0x46,0xf8, -0xe6,0x52,0x4c,0x4d,0xf0,0x58,0xcf,0x0f,0xce,0xfd,0xc8,0x0f, -0xb1,0x3f,0xe3,0x3a,0x18,0x59,0x0c,0x24,0xe7,0xb7,0xf1,0xd3, -0x93,0x6a,0xca,0x92,0x40,0x5e,0xc2,0x8f,0xee,0x56,0x6f,0xfb, -0xb5,0xff,0xdc,0x6e,0x42,0x1a,0x11,0x6c,0x0e,0x64,0xa6,0x0d, -0xbd,0x36,0xf3,0xd6,0x7b,0x6b,0x2c,0x39,0x3a,0xfa,0x7a,0x5e, -0x2d,0xde,0xdc,0x3f,0xfd,0x7a,0x83,0x8c,0xf9,0xf4,0xeb,0xfb, -0xf2,0x77,0x7a,0x74,0xb5,0x2e,0x2f,0x8b,0xcf,0xef,0x4f,0x3f, -0x3f,0x9d,0x7e,0x7d,0x7f,0x7a,0xfa,0x35,0x81,0xcc,0x07,0xc8, -0xcf,0xe9,0xcd,0xa3,0xf3,0xe5,0x3b,0x8c,0x49,0x8f,0x23,0x2a, -0xb0,0xd9,0x76,0x5b,0x82,0x7b,0xd0,0xe9,0xf4,0x40,0xfe,0x34, -0xc1,0xb7,0x28,0x4d,0x07,0x5a,0x81,0xf3,0xc8,0x14,0xff,0x9b, -0xcf,0xf5,0x48,0x22,0x51,0xfb,0x76,0x43,0xc6,0xd4,0x5e,0x04, -0x70,0x76,0x42,0x21,0x14,0xc4,0x1d,0x22,0x78,0xd7,0x72,0x5f, -0x3d,0x79,0xbd,0x0c,0x5f,0xf5,0xc0,0xd1,0x1d,0x00,0x93,0xdf, -0x26,0x23,0x50,0xa7,0xf2,0x7e,0x8b,0x44,0xe1,0x2f,0x12,0x8c, -0x15,0x35,0x58,0xbd,0x1b,0x52,0x24,0x89,0x1c,0x55,0x6b,0x43, -0xf1,0x41,0xce,0xfb,0x5f,0x25,0x46,0x09,0xe2,0x0e,0x9f,0x9d, -0xf0,0x26,0x27,0x4f,0x36,0xfc,0x5a,0xc0,0xdb,0xbe,0x9d,0x25, -0x60,0xd6,0x53,0x94,0x54,0xff,0x86,0x3e,0x9e,0x35,0x34,0x52, -0xe6,0x5f,0x14,0x5c,0x86,0xae,0xdc,0x68,0x38,0xee,0x7d,0xf1, -0x0d,0xda,0x7d,0xe6,0x9d,0xfd,0x60,0x53,0x01,0x6b,0xc3,0x61, -0x30,0x2e,0xff,0xb3,0x72,0x5d,0x4d,0xe7,0xe8,0xe9,0xd1,0x39, -0x50,0x11,0xa9,0xc2,0xd5,0xa3,0x01,0xe7,0xf7,0x61,0xa0,0xf7, -0xd9,0x0e,0x68,0x1a,0xbe,0xea,0x9c,0x50,0x01,0xc4,0x23,0x52, -0xcd,0x8f,0xcb,0xf5,0x35,0xf5,0x30,0xcb,0x93,0xfb,0xf4,0x72, -0x5b,0x5c,0x1c,0x4b,0xd1,0x6c,0x31,0x96,0xee,0xbf,0x38,0xe9, -0x7f,0x65,0x5b,0x96,0x28,0x13,0x9c,0x97,0xe1,0x73,0x43,0x64, -0x2d,0x09,0xe0,0x7a,0x98,0xa7,0x24,0x43,0x74,0xf8,0xd3,0x22, -0xc7,0x3b,0x43,0x98,0x1f,0x7e,0x96,0x0e,0x89,0xc0,0xbe,0x2a, -0x07,0xbf,0xed,0xb3,0x73,0xa6,0x5c,0xd0,0x23,0x05,0x38,0xf0, -0x3d,0x04,0x01,0xbc,0xf7,0x1a,0x86,0x2d,0x05,0x8d,0x7b,0xd4, -0x2f,0x4f,0xbe,0xce,0x17,0xd3,0xb7,0xb0,0x40,0xe8,0x0f,0x4d, -0xdc,0x9e,0x36,0x20,0x1f,0x1f,0x6a,0xd0,0xe7,0x1d,0xd1,0xe8, -0xd7,0xbd,0x1b,0x68,0x60,0xd5,0xfc,0x00,0xa9,0xf3,0x3c,0x79, -0xf4,0xec,0xd9,0xe0,0xd1,0xf2,0x7a,0x35,0xdd,0x04,0xda,0xf4, -0x0b,0x4a,0xc2,0x32,0x26,0x78,0x12,0x3c,0xef,0x9c,0x18,0xff, -0xdd,0x46,0xfc,0x6e,0x0f,0x51,0x82,0x36,0xcb,0x71,0x50,0x10, -0xb6,0x63,0xe6,0x37,0x3d,0x7e,0x20,0x3f,0xfb,0x5f,0xc8,0x9f, -0x1f,0x1a,0x69,0xbc,0xf9,0x81,0x89,0xab,0xea,0x5d,0x39,0xe7, -0xd7,0x85,0x96,0x0b,0xf4,0x60,0xe7,0x15,0x62,0x1f,0x0d,0xc4, -0xa7,0x5d,0xc2,0xc7,0x24,0xf5,0x51,0x34,0x29,0x17,0x0c,0x5f, -0x9f,0x17,0x94,0xa5,0xa9,0x5a,0x00,0xbf,0x15,0x98,0xa8,0xc7, -0x36,0x19,0x67,0x4d,0x0b,0x91,0x12,0x5d,0x62,0x82,0x20,0xbd, -0x6e,0x82,0x30,0x9b,0xad,0xaf,0xa7,0x17,0x00,0xaf,0x2c,0xe8, -0x3d,0x73,0x24,0xcb,0x59,0x06,0x15,0xb2,0x4c,0x22,0xa2,0x04, -0x6d,0x25,0x8c,0x0b,0x05,0x83,0x8d,0xd1,0x7b,0x5a,0x5c,0xe3, -0x55,0x4b,0x0d,0x6d,0x8d,0x2f,0x62,0x61,0x71,0xb7,0xb6,0x8e, -0x74,0x05,0x8d,0xe3,0x03,0x05,0x4b,0xaf,0x07,0xfa,0xfe,0xab, -0xd6,0xb4,0xfd,0xc8,0x4d,0x74,0x4b,0x3b,0x82,0x52,0x61,0xeb, -0x6d,0x25,0x16,0x8e,0xeb,0xeb,0x0e,0xf3,0xe1,0xc7,0x8b,0x6d, -0xd1,0x80,0x7f,0x52,0xf3,0x0e,0x58,0xcd,0x0a,0x26,0x5c,0xa7, -0x58,0xca,0xd0,0x37,0x11,0x46,0x61,0x9b,0x8c,0x88,0x22,0xf8, -0x42,0xc7,0x22,0xe6,0x6e,0xf6,0x2d,0x4a,0x98,0x6c,0xef,0xae, -0x05,0x6d,0x15,0xc3,0x14,0x52,0x15,0x80,0x1c,0xf7,0x26,0x31, -0xfe,0x29,0xa4,0xe5,0x8d,0x53,0x24,0x08,0xde,0xfb,0x62,0x10, -0x2a,0xbc,0x34,0xc9,0xec,0xe5,0x27,0xd4,0x3e,0xbf,0xae,0x68, -0x59,0xf0,0x05,0x10,0xfe,0xba,0x04,0x2c,0xd5,0x15,0xac,0x82, -0x93,0x5d,0xd6,0xd8,0x61,0xed,0x7e,0x5d,0x00,0x34,0x49,0xaf, -0x62,0x84,0x0b,0x10,0xe3,0xaa,0x97,0x7c,0x73,0x73,0x0e,0x64, -0x5c,0xc3,0xd9,0xc2,0x16,0xc5,0x5b,0xd6,0x5b,0x79,0x69,0x45, -0xc6,0xc4,0xad,0xd5,0x63,0x5b,0x66,0xa2,0x0f,0xa2,0x92,0x92, -0xda,0xbd,0x48,0xf3,0x56,0x78,0x20,0xea,0x3a,0x5e,0xad,0xf1, -0xa5,0xc3,0x47,0xf3,0x6a,0x55,0xe8,0x7b,0x39,0xf4,0x3a,0x51, -0xcb,0x48,0xdb,0x2b,0x25,0x6e,0xba,0x60,0xe3,0x59,0x13,0xc2, -0xe8,0x65,0xa4,0xb0,0x3d,0xd9,0x81,0xda,0xda,0x10,0xa9,0x27, -0xd6,0x15,0xc8,0x0b,0x7e,0xe5,0xda,0x70,0x50,0x8f,0xc7,0xd5, -0x5b,0xb3,0x99,0xd5,0x08,0xd8,0x2f,0x25,0xe0,0xaf,0xd0,0xb0, -0x1e,0xf9,0xc9,0x90,0x4b,0xc0,0x0f,0x7e,0xff,0x06,0x7e,0x04, -0xaf,0xda,0x40,0xf2,0xbb,0xe3,0x9a,0x18,0x54,0xee,0x01,0x35, -0x3c,0xbe,0x5e,0xfe,0x76,0xbc,0x2f,0xef,0xb6,0x3c,0x7f,0x53, -0x6d,0xf6,0x65,0x27,0x06,0xb7,0xc5,0x36,0x67,0x56,0x7f,0x0b, -0xe4,0xad,0x13,0x23,0x63,0xe3,0x2f,0x0e,0xda,0xcc,0x83,0x3a, -0x2c,0x88,0x19,0x5b,0x2e,0x16,0x14,0xec,0x00,0x29,0x28,0x09, -0xfc,0x95,0xa0,0x2a,0x27,0xc3,0x95,0x72,0xda,0xe9,0x39,0x48, -0x9a,0x40,0x09,0x43,0x14,0x2a,0xd0,0xd2,0xe8,0x72,0x93,0x1f, -0xff,0x05,0xfe,0x07,0xf2,0x05,0x23,0xea,0x58,0xc4,0x0d,0x1e, -0x44,0xb0,0x4c,0x6d,0xc7,0x59,0x28,0xfd,0x54,0x6f,0x1b,0x82, -0x9f,0xca,0x7b,0x9b,0x35,0xfc,0x37,0x43,0xb9,0x6f,0x46,0x3f, -0x36,0xfc,0xeb,0x3e,0xa6,0x8b,0x2c,0x98,0xe0,0xcc,0x1d,0x10, -0x8d,0x36,0x33,0x8c,0x71,0x3d,0xc3,0xcb,0xea,0x78,0xc0,0x1f, -0x33,0xc9,0x14,0x98,0xcb,0x17,0x76,0xe9,0xe5,0x76,0x69,0x2f, -0x78,0xbd,0x2b,0xe8,0xc3,0x3d,0x74,0x48,0xe9,0x83,0x46,0x3a, -0xb7,0x6b,0xf9,0xab,0xec,0x6d,0x1c,0xe5,0xed,0x27,0x6a,0xb6, -0x2e,0xbc,0x5e,0xbb,0xdd,0x7d,0xdd,0x46,0x98,0xe3,0xd5,0xd5, -0x98,0x59,0x4b,0x68,0xee,0x95,0xb0,0x06,0x89,0xfa,0x59,0x4d, -0x0a,0xf5,0x72,0x15,0x6f,0x28,0x55,0x0a,0xc2,0xf0,0x67,0xb8, -0x2e,0x98,0x88,0xbe,0x8c,0x08,0xe3,0xbf,0xee,0xa1,0x25,0xc8, -0x70,0xd8,0xb0,0x5b,0x7e,0xf1,0xa5,0xac,0x6d,0xef,0xdd,0x34, -0x5b,0x6a,0xb6,0x2c,0xeb,0x1f,0x97,0x9b,0xa7,0x1c,0x64,0x87, -0xa5,0x86,0xa7,0x8b,0x6f,0x80,0xea,0x18,0x7f,0xa4,0x33,0x20, -0x22,0xe4,0xda,0xcf,0x97,0x2b,0xbd,0x23,0x07,0x4a,0x41,0xd1, -0x06,0x3a,0x9f,0x3d,0xd3,0x67,0x13,0xa5,0xd5,0x40,0xb2,0x28, -0x92,0xc1,0x7f,0x45,0x59,0x2a,0xdd,0x53,0x0f,0x69,0x97,0x4c, -0x97,0x32,0xf4,0x0d,0xc4,0x37,0x50,0x57,0xcd,0x41,0xa8,0xdc, -0x52,0x24,0x80,0x0c,0x64,0x56,0x1f,0xd5,0x1a,0xa3,0x8f,0xaa, -0x40,0xc3,0xf4,0xe5,0x98,0x15,0xe1,0xe4,0x63,0x4e,0x10,0xb4, -0xe2,0x33,0x57,0x31,0x22,0x8c,0x26,0xa9,0x28,0x07,0x6c,0x54, -0xb9,0x76,0x52,0x59,0x11,0xe7,0x71,0x10,0xa3,0xe4,0xc4,0xa7, -0x3d,0x49,0x23,0x4e,0x10,0x53,0xb9,0x27,0xe1,0x15,0x1d,0xcf, -0xe1,0x68,0x2f,0x62,0x1c,0x97,0xf6,0x90,0xed,0x41,0x84,0x37, -0x13,0x62,0x21,0x6f,0x01,0xc0,0xa0,0x76,0x6d,0xaf,0x15,0xcb, -0x46,0xfd,0x81,0x25,0xa3,0x78,0xe8,0x71,0x74,0x2c,0xa2,0x6f, -0x9f,0xec,0x95,0xd6,0x59,0xc4,0x1d,0x62,0x67,0xf9,0x20,0x31, -0x87,0x83,0xf3,0x15,0x5f,0xb4,0x90,0xb3,0xeb,0xdb,0xf2,0x06, -0x0e,0xb9,0xd7,0xe0,0x89,0xf0,0x0d,0x5c,0x0f,0xff,0xe5,0x3c, -0xef,0xc8,0x15,0xe0,0xfc,0x2b,0x1f,0xe7,0xb1,0xa4,0x5d,0x7c, -0xd1,0xd9,0xbf,0xa4,0xf6,0xc5,0x14,0x4c,0x69,0x19,0xf9,0xc1, -0x02,0x01,0xe3,0x94,0xc6,0x97,0x99,0x31,0x7f,0x37,0xde,0x5e, -0x04,0xd4,0x8a,0x0c,0xda,0xd1,0x2e,0xbb,0x1b,0x67,0x74,0x30, -0x97,0x83,0xb3,0x15,0x00,0xf1,0x80,0x3c,0x65,0x91,0xad,0xe0, -0xb8,0x1f,0x0c,0xd9,0x2e,0x65,0x63,0x02,0x7a,0x27,0xac,0xa4, -0x60,0x3d,0x67,0x77,0x18,0x44,0xe8,0xd9,0xe4,0xde,0xd9,0x6e, -0x7b,0x36,0xd6,0xdf,0x13,0x0a,0xfa,0x73,0x7d,0x33,0xdf,0x54, -0x8f,0xa7,0xf5,0x15,0x94,0x1c,0x3f,0x3c,0xfe,0xdf,0x31,0x3c, -0xce,0x30,0xba,0x4c,0xba,0x40,0xe3,0x06,0x74,0x1e,0x15,0x41, -0x26,0x4f,0xb8,0x40,0xd2,0x4b,0xfd,0x18,0x2b,0x3d,0xb2,0xb1, -0x5d,0x63,0x89,0x6b,0x10,0xc5,0xac,0xc5,0xf2,0xfd,0xb3,0xc7, -0xf7,0x5f,0x91,0x14,0xbe,0x58,0x62,0x04,0xba,0xfc,0xae,0xbc, -0x3e,0x87,0x13,0x1d,0x08,0x64,0x6c,0xd9,0x91,0x27,0x17,0xf3, -0xba,0x9a,0xe5,0x8f,0x1f,0xfc,0xb7,0x47,0x8f,0xbf,0xf9,0xf3, -0x93,0xe3,0x87,0x4f,0xfe,0xfc,0xf8,0x78,0x30,0xb8,0xb8,0x3c, -0xfe,0xcb,0x9f,0xbf,0xf9,0xb7,0xe3,0x2f,0xbf,0xfc,0xf2,0xab, -0xaf,0xbe,0xf8,0xea,0xcb,0x13,0xf8,0x5f,0x62,0xf0,0xb6,0xa2, -0x44,0xc1,0x8e,0x2e,0x4d,0xa9,0xc1,0xc0,0x62,0x2c,0x7e,0xb7, -0xd8,0x0f,0xf2,0x65,0x23,0xcd,0xa2,0xb5,0x06,0x66,0xc5,0xaf, -0xe5,0x4f,0xf2,0xb6,0x54,0xd3,0xe9,0x70,0x34,0x8c,0x8e,0x38, -0x7b,0x61,0xaf,0x62,0x2a,0xcd,0x4e,0xe5,0xfc,0xfe,0x40,0xd3, -0x6d,0xc0,0x04,0x2e,0x40,0x15,0x48,0x2e,0x6b,0x79,0x30,0x24, -0x2e,0xa3,0xb7,0xdd,0xcd,0x01,0x85,0xfe,0x07,0xda,0xc6,0x2f, -0x51,0x44,0x3f,0x2a,0xb4,0x33,0x2f,0xff,0x28,0x20,0x86,0x5c, -0x5b,0x5f,0xfe,0x71,0x60,0xb8,0x21,0x34,0x3c,0x5c,0x6d,0xda, -0x66,0x49,0x03,0x89,0xb8,0x30,0xa3,0x83,0x4e,0x23,0xf2,0xe8, -0x5f,0xe2,0x24,0xe7,0x39,0x47,0xbe,0x75,0x44,0x50,0x45,0xe4, -0x93,0x21,0xb3,0xc7,0x99,0xe3,0x03,0x1e,0x1a,0x6a,0x14,0xdd, -0xe1,0xa2,0xdb,0x2d,0xff,0x15,0x47,0x5a,0xaa,0xf8,0xaa,0xa9, -0x84,0xab,0x40,0x4c,0xa2,0x47,0xec,0xb1,0xac,0xa7,0x2f,0xb4, -0xfe,0xfa,0xe9,0x5d,0x38,0x09,0x9e,0x09,0x23,0xdd,0xea,0x61, -0xf0,0x3a,0x79,0x39,0x13,0x49,0x54,0xe2,0x93,0xd8,0xa7,0x5e, -0x68,0x39,0xa3,0x12,0x30,0xf2,0x42,0x24,0x94,0x79,0xb1,0x17, -0x22,0x9f,0x2f,0x1b,0xb5,0xb2,0x35,0x84,0x6b,0xa7,0x25,0xb8, -0x25,0xbb,0xb1,0xe1,0xf0,0x6a,0xf4,0xdf,0x22,0x4b,0x09,0x0e, -0x12,0x48,0x95,0xdd,0x61,0x66,0x58,0x7d,0xcd,0x31,0xf8,0xf4, -0x05,0xf2,0x5e,0x2f,0xa3,0xc8,0xcd,0x1c,0xd9,0xaf,0x9a,0xf4, -0x17,0xfa,0xe8,0xb8,0x86,0x76,0x83,0x0d,0x15,0x3a,0x3a,0xc6, -0xb0,0x9a,0x7b,0x82,0x3c,0x7b,0xef,0x9e,0x7f,0x25,0xcf,0xce, -0x20,0x28,0x11,0x39,0xca,0x23,0xa5,0xd6,0xe7,0x79,0x1f,0xfc, -0x44,0x6d,0x7a,0xb6,0xc2,0x79,0x11,0x53,0xe5,0xd8,0xa5,0x16, -0x70,0xca,0x51,0x4a,0xe8,0x92,0xad,0x11,0x5d,0x93,0xb1,0x48, -0x57,0x0f,0xe4,0xdd,0x69,0x4d,0xb6,0xc4,0x80,0x36,0x0c,0x3a, -0x11,0xda,0xa5,0xaa,0x3f,0x89,0x7b,0xed,0x1f,0xc1,0x1c,0x85, -0x03,0x43,0x5a,0x88,0x27,0x8c,0xcd,0x4b,0xc9,0xd3,0x2f,0x4f, -0x3f,0x1a,0x38,0x6b,0xf0,0xea,0x7c,0xe6,0x68,0x1f,0x60,0x4d, -0x43,0xf3,0xba,0x84,0x73,0xc5,0x71,0xbd,0xb9,0xb8,0x11,0x86, -0x30,0x08,0x48,0x3b,0x0c,0x5e,0x24,0x51,0x0f,0x4d,0xde,0x32, -0xd0,0x35,0xf0,0x2b,0xc5,0x3b,0x0f,0x99,0x07,0xe9,0xe2,0x9c, -0x1f,0x1f,0x15,0xf1,0x9f,0xda,0x1c,0x51,0x04,0xcb,0x22,0x95, -0xe0,0x68,0x18,0xaf,0xb8,0x97,0x50,0xa9,0x84,0xe3,0xa7,0xb7, -0x04,0x70,0xe5,0x88,0x90,0xe2,0x04,0xd9,0xf9,0x95,0x43,0x3d, -0x47,0xe1,0x12,0xa8,0xcb,0xd1,0xa1,0x16,0x9a,0x91,0x76,0xd8, -0x2b,0x2f,0xa7,0x4a,0x6c,0xe4,0xc1,0x29,0x46,0xba,0x18,0xbb, -0xe7,0x9e,0xcc,0xac,0xdc,0x37,0xda,0xec,0x6e,0xa3,0xef,0x56, -0xf1,0x80,0x86,0x76,0xe0,0x0a,0xc9,0xaf,0x2e,0x42,0x33,0x8f, -0x85,0xec,0xa2,0xc4,0x88,0x8a,0xbb,0x57,0x43,0xba,0x85,0x7c, -0xeb,0xeb,0x63,0x87,0x03,0x5a,0x73,0x6b,0x0b,0xb2,0xbf,0x6f, -0xd2,0x52,0x19,0xf7,0xbb,0x1b,0xa2,0x36,0x4d,0xcd,0x31,0x80, -0x90,0x2f,0xe9,0xc5,0xcb,0xb8,0x4f,0x0f,0xba,0xe3,0x63,0x0d, -0x5b,0x7d,0x71,0xb3,0x66,0xeb,0x6d,0x0c,0x16,0x7d,0xf9,0x4e, -0xcd,0xcf,0xba,0x5d,0xae,0xab,0xef,0x68,0xf9,0x1d,0xd8,0x38, -0xe7,0xdc,0x02,0xda,0xec,0x19,0xdf,0x88,0x9a,0x00,0xe7,0xe6, -0x01,0xe9,0x1d,0xaf,0xdb,0x6e,0x57,0x82,0xaa,0xb7,0x84,0xb0, -0x36,0x1e,0x16,0x5a,0xa7,0x43,0x3c,0xbf,0x0b,0x7e,0x98,0xcc, -0x15,0x4e,0x22,0x03,0x8e,0x97,0xe1,0xf2,0x6c,0x8b,0x0e,0x8c, -0x8b,0xf0,0x8e,0x4d,0xad,0x3e,0x60,0xfc,0x12,0x5f,0xb0,0x6b, -0x5b,0x71,0x58,0x5e,0x0f,0xa8,0xc4,0x2e,0xa8,0x46,0x29,0x59, -0x72,0x59,0xfb,0xde,0x13,0x51,0x62,0xb4,0xe4,0x6a,0x8a,0x68, -0x58,0x3c,0xd8,0xf7,0x7a,0x24,0xcf,0x1a,0x6f,0x2c,0x1b,0x7d, -0xb6,0xaf,0xc0,0x29,0x35,0x5c,0x15,0xe7,0x3c,0x66,0x2e,0x5f, -0x73,0xd6,0x28,0x20,0x68,0xdd,0xd6,0x08,0xed,0x39,0x35,0xe8, -0xf3,0x46,0xcc,0xce,0x5b,0x19,0xcd,0x9e,0x05,0x42,0xdc,0xc6, -0x8d,0x66,0xd8,0x42,0xf5,0xb6,0x48,0x66,0x02,0x22,0xf4,0xc9, -0x0e,0x36,0x77,0xaa,0x41,0x2f,0xbd,0x45,0x6b,0xc1,0xd5,0xdf, -0x65,0x2d,0x8b,0x9a,0x09,0xe8,0x23,0x78,0xe4,0xbe,0xf6,0xe6, -0xbe,0xf7,0x1b,0x3e,0x21,0x61,0xc2,0x26,0x2b,0xb7,0x4f,0x5e, -0xbe,0x1b,0xd9,0x5f,0xf8,0xe0,0x46,0x89,0xcf,0x15,0x42,0xfe, -0x64,0xbb,0xc5,0x3f,0xb9,0xad,0xed,0xb1,0x16,0x0e,0x26,0x26, -0x5d,0x23,0xa2,0x3c,0xdb,0x23,0xbb,0x94,0xd8,0x04,0x8c,0x6d, -0xc3,0x0b,0xcf,0x4c,0x9c,0x8a,0x60,0x7a,0x36,0x74,0x0b,0xd2, -0xe7,0x1b,0xa4,0x58,0xd4,0xd2,0xd2,0x00,0x92,0xe0,0xce,0x50, -0xce,0x7f,0x7c,0x08,0x57,0x0e,0x2e,0x81,0x96,0xa2,0x3f,0x35, -0x6c,0x6d,0x08,0x70,0x1b,0x3c,0x81,0x1f,0xc4,0xbb,0xc1,0x57, -0x86,0xf8,0xb2,0xb6,0xc5,0x97,0xa1,0x14,0x4d,0x16,0x5b,0x0c, -0x54,0x41,0x78,0x2a,0xb9,0xef,0xf5,0xc7,0x71,0x7c,0x4c,0x0d, -0xca,0x85,0x71,0xe3,0x89,0x15,0xb2,0x21,0x1d,0xeb,0x2f,0x00, -0x91,0x43,0x21,0xed,0x59,0x27,0x18,0x39,0xc4,0x2d,0x13,0x27, -0xaa,0xc5,0x53,0x33,0xac,0x8e,0x8f,0x87,0x19,0x7a,0x47,0x35, -0xd8,0x88,0xc4,0x27,0x32,0x0d,0x86,0x44,0xef,0xcd,0x75,0xbb, -0xe8,0x8c,0x4e,0x4c,0x86,0x14,0x97,0x00,0x79,0xaf,0x67,0x6c, -0x1a,0x31,0x16,0x35,0xad,0xf2,0x1c,0xe4,0x28,0x21,0x95,0x0b, -0xee,0xd0,0x86,0x7f,0x97,0x89,0xe8,0x3c,0xa3,0x7e,0x0c,0xc6, -0x7c,0xa0,0x1f,0x6b,0x92,0x6e,0x8b,0xfb,0xe3,0xb3,0x0d,0x45, -0x61,0xc5,0x90,0xaa,0xdc,0x60,0x71,0xff,0x6c,0x8d,0x5f,0xa4, -0x3f,0x27,0x05,0x0d,0xc5,0x9d,0xa5,0xc3,0xe7,0x96,0x0f,0xa6, -0x5b,0xbc,0x5f,0x9f,0xae,0xcb,0xe9,0x16,0x24,0xc6,0xcd,0x72, -0xb1,0x65,0xc1,0x0b,0xce,0x98,0x15,0x36,0x5c,0x5d,0xbc,0x71, -0xf5,0xa6,0x5b,0x2c,0xc8,0x59,0x12,0x71,0x82,0x33,0xe4,0xc2, -0x60,0xab,0xf7,0x7c,0xdb,0xe9,0xcd,0x66,0x49,0x9d,0xd2,0x2f, -0x7a,0x82,0x9d,0xdc,0xe8,0x29,0x22,0xe3,0x7a,0x39,0xaf,0x79, -0x12,0xb7,0x7a,0xc5,0xb4,0xe5,0x60,0xde,0xdb,0xf9,0x72,0xb9, -0xda,0xd2,0xd1,0x16,0xce,0x8a,0xdb,0xe5,0x0a,0x92,0xd0,0xa3, -0x61,0xb9,0x98,0xbf,0x87,0x1f,0xbf,0xde,0xa0,0x05,0xe5,0xb6, -0xbe,0x58,0xe2,0x53,0x3e,0x04,0xc6,0x4d,0x5d,0x3e,0xe6,0xd8, -0xcf,0xed,0x90,0x50,0xa9,0xe8,0xda,0xb7,0x08,0xed,0x04,0xfa, -0x51,0xb6,0x14,0x7f,0x4a,0x27,0xf4,0xa8,0x28,0x61,0x6e,0xd8, -0xe4,0xdd,0x28,0x4f,0x7b,0x31,0xf6,0x50,0x08,0x0e,0x65,0xcc, -0x16,0x69,0x54,0x93,0xa0,0xaa,0x71,0x35,0x5a,0x04,0x40,0x2b, -0xf8,0x3d,0x6c,0xf4,0xf2,0x09,0x92,0x1f,0x89,0xb2,0xd4,0x31, -0x9f,0x38,0x79,0x0d,0xaf,0x7e,0x27,0xd4,0x14,0xd0,0xfc,0xe3, -0xa0,0xfe,0xb9,0xd1,0x8b,0xb3,0xc9,0xf3,0x0e,0x18,0xfc,0x52, -0xcd,0x3b,0x3e,0x37,0xc0,0x89,0x0e,0xdb,0x6e,0x1d,0x15,0xf9, -0xd2,0x50,0x10,0x42,0x8a,0x65,0xe5,0x85,0xe9,0x66,0xb1,0xc4, -0xe5,0xf9,0x8f,0xf5,0xe1,0x91,0x76,0x36,0x7b,0x84,0x2b,0x25, -0x8f,0xcf,0x01,0x74,0x5f,0x82,0x39,0x65,0xcd,0x2e,0xc9,0x20, -0x15,0x61,0x80,0xee,0xdf,0x7e,0x33,0xaf,0xd5,0x09,0x35,0xe0, -0x4e,0x00,0xeb,0x05,0x70,0xf6,0xa6,0x77,0x89,0x78,0x77,0xd2, -0x9f,0xf6,0xd7,0xbe,0x83,0x07,0xef,0xda,0xa6,0xed,0xb5,0x7d, -0x30,0x80,0xc2,0x37,0xf6,0x15,0xea,0xd4,0xf3,0x76,0xe3,0x99, -0x90,0x18,0x94,0x6a,0xb9,0x90,0x61,0x7c,0x7f,0xe8,0x53,0x80, -0xe3,0x47,0x32,0x79,0x54,0x05,0x57,0x06,0x96,0x96,0x1c,0x7a, -0x9e,0x3c,0xf0,0xaa,0x76,0x47,0x5c,0xe0,0x71,0x28,0x26,0xb6, -0xbd,0x2f,0x42,0xdf,0x16,0x82,0x11,0x3a,0x05,0xf5,0xc2,0x34, -0x7c,0x53,0xd3,0x73,0x82,0xa7,0x74,0x83,0x69,0x39,0xfe,0xc3, -0x67,0x57,0xf4,0xaa,0x26,0x74,0x17,0x02,0x2f,0x3b,0x3e,0x67, -0xd0,0xab,0x3b,0x96,0xe2,0xcb,0x92,0x58,0x86,0x1a,0xfc,0x5a, -0x1e,0xd3,0xec,0x15,0x2e,0x6d,0x18,0x76,0x1c,0x84,0x54,0x81, -0xa2,0xad,0x2e,0x0a,0xff,0x62,0x7a,0xc0,0x0b,0x8c,0x78,0x25, -0xb8,0x18,0x27,0xff,0x33,0xa9,0xc4,0x1b,0xce,0xff,0xff,0x08, -0xe5,0x43,0x74,0xa2,0x2f,0xd4,0xee,0x21,0x96,0x53,0xa8,0x86, -0xb4,0x84,0x4f,0x91,0xda,0x0e,0x82,0x22,0xa6,0x8d,0x7e,0x08, -0x13,0xa3,0x98,0x8a,0x00,0x9a,0x90,0x90,0xf8,0xbd,0x90,0x36, -0x42,0x62,0x7b,0xdd,0xbf,0x4e,0xe7,0x9e,0xe7,0x8f,0x4f,0x01, -0xa6,0xaa,0xd1,0x13,0xaf,0x68,0x46,0x6f,0xd2,0x8a,0xd1,0xd1, -0xa6,0x41,0x20,0x6d,0x1a,0x91,0x2a,0xa2,0x0c,0x0f,0xbe,0x06, -0x65,0x54,0x11,0x65,0x38,0x88,0xdd,0xaf,0x5d,0xd6,0x2e,0xed, -0x07,0x21,0xb3,0xf9,0x79,0x07,0xfb,0xde,0xa5,0x6d,0x0f,0x17, -0x0b,0xb9,0x57,0xf9,0x20,0x19,0xcf,0x4c,0x19,0xc3,0x12,0xd8, -0xe2,0x35,0xe3,0xfc,0x00,0xe1,0xb9,0xd9,0x71,0x95,0x28,0x62, -0xc2,0x30,0xe3,0x46,0x19,0xa5,0x23,0x76,0x2e,0xe8,0xa8,0xd3, -0x14,0x8f,0xde,0x2d,0x00,0x82,0x69,0x4c,0x85,0x46,0x89,0x32, -0xd8,0x24,0x4f,0xbc,0x55,0x94,0x4c,0xbc,0xf2,0xe4,0xb2,0xca, -0x2a,0x8e,0x96,0xeb,0x16,0x58,0x2d,0x6e,0x06,0x19,0x13,0x1a, -0x40,0xd8,0x36,0x61,0x8f,0x2c,0x2f,0x9d,0x2e,0x28,0x79,0xf9, -0xd2,0xe6,0xbf,0x7c,0x99,0xc4,0xab,0x34,0xfa,0x2e,0xc2,0xcf, -0xed,0xd6,0x85,0x41,0xc1,0x80,0x23,0x1f,0x6c,0x3e,0xa3,0x45, -0x8d,0xfb,0xa0,0x22,0xa4,0x19,0x43,0xdd,0x7b,0x2c,0xd6,0x76, -0x8b,0x0b,0x45,0xf3,0x69,0xad,0x10,0x03,0xf4,0xd9,0xdf,0x70, -0xee,0x98,0x01,0xae,0x7d,0xe1,0x05,0xc1,0x3b,0x54,0xf8,0x7e, -0xb2,0x24,0x7f,0x78,0xc5,0xdb,0x75,0xec,0x90,0x01,0x6b,0x58, -0x35,0xc8,0x27,0x43,0x1b,0x0d,0xce,0x04,0x01,0x8d,0x3c,0xee, -0x8d,0x11,0x4c,0xf8,0x89,0x32,0xb7,0x66,0x02,0xa5,0xed,0xf0, -0x8e,0x83,0xee,0x04,0xac,0xda,0x46,0x0c,0xb1,0x75,0x5a,0xdc, -0x36,0xc4,0x03,0xbe,0x7d,0xe5,0x49,0xec,0x8a,0x26,0xcd,0x0f, -0x15,0x31,0x3e,0x56,0xa0,0x64,0xe1,0x5a,0x1e,0x35,0x57,0x27, -0xd1,0x2f,0x24,0xa3,0x17,0xa3,0xbc,0xd2,0x47,0x7e,0xcc,0xf0, -0x1b,0x4b,0x17,0x30,0xef,0x8d,0x50,0x24,0x92,0xd7,0x2b,0x1c, -0x51,0x78,0x4f,0xeb,0x65,0xd2,0xad,0xd5,0x9a,0xad,0xf0,0x73, -0x9f,0x66,0xd4,0xf6,0x06,0x8c,0x10,0x9a,0xa3,0x1f,0x3d,0x60, -0x80,0x59,0xa4,0xc1,0x82,0x0c,0x7e,0x3a,0x8f,0xe3,0x48,0xe3, -0x93,0xea,0x56,0xef,0x12,0xe6,0xed,0xd1,0xe7,0xdb,0xd7,0xf6, -0xe0,0x60,0x6b,0x9f,0xda,0x53,0x3d,0x11,0xa4,0xa5,0x6a,0x54, -0x6e,0xd4,0xd4,0xab,0x13,0x84,0xf9,0xe3,0xd5,0xc6,0x2b,0x02, -0x47,0x29,0x3b,0x9c,0x1f,0x06,0xa6,0x1d,0x60,0xe2,0xf9,0x7b, -0x00,0x3e,0x74,0x01,0x61,0x01,0x7e,0x15,0x00,0x8c,0x0e,0x36, -0xf2,0xb2,0x1c,0x64,0x88,0xba,0x5b,0x20,0x0e,0x40,0x46,0xbf, -0x9b,0x9c,0x4a,0x53,0x2f,0x3c,0xb9,0x0d,0x41,0x01,0xf2,0xb1, -0xa0,0x5b,0x23,0x3c,0x16,0xbc,0x93,0x93,0xf8,0x72,0x58,0x02, -0x66,0x86,0x42,0x83,0xb4,0xe8,0x72,0x75,0x2c,0xb9,0x38,0x44, -0xe7,0x77,0x41,0x30,0x70,0xbe,0xd0,0x11,0xa2,0x8d,0x2f,0x0f, -0x18,0x28,0x5d,0x6c,0xf0,0x41,0x2c,0x87,0x1f,0x11,0xbd,0xac, -0x44,0x4b,0xce,0x85,0x72,0x46,0x23,0xf4,0xba,0xd3,0xc0,0xe6, -0xad,0x3d,0x29,0x77,0xb1,0x56,0xa2,0xb8,0x2c,0xd5,0x55,0x9b, -0xda,0xd0,0x40,0xc8,0xec,0xed,0x49,0x49,0x7a,0x94,0x63,0xf7, -0x69,0x34,0x67,0x13,0x6b,0xe3,0x63,0x79,0x50,0xce,0xce,0xe1, -0x76,0x7b,0xc2,0xaf,0x59,0xa9,0x03,0x01,0x14,0x18,0x91,0x76, -0x7e,0x3c,0x31,0xd7,0xd3,0x77,0xf4,0xcd,0x0f,0x5d,0x0d,0xec, -0xa3,0xb9,0x36,0xd0,0x8f,0xd4,0x1d,0x41,0xc1,0xdc,0x16,0x24, -0xc3,0x99,0x77,0x8e,0xbf,0x71,0x2d,0x1b,0x68,0x18,0x03,0x71, -0xa4,0xe2,0xb7,0x6d,0x01,0xed,0x76,0xab,0x4e,0xc1,0x6e,0xa3, -0x40,0x99,0xd1,0x09,0xd2,0xb3,0xb2,0x14,0x1f,0x5f,0x6b,0x67, -0x99,0xb3,0x8b,0x92,0x34,0x1e,0xdd,0x60,0x69,0x21,0xa0,0xa3, -0xed,0x56,0x4a,0xb8,0xf0,0xfe,0xb6,0x0d,0xef,0x12,0x8d,0xe9, -0x36,0x6d,0x94,0x35,0x68,0x74,0x8d,0xf6,0x63,0x2b,0xba,0x3c, -0xb2,0x51,0xb7,0x84,0x5d,0x71,0xf9,0x8c,0xd9,0x0e,0xa2,0x3b, -0xf3,0xaf,0x49,0x86,0x8c,0x59,0xd6,0xab,0xcb,0x45,0x86,0x9f, -0x8f,0xe1,0xd0,0xe3,0x79,0xf7,0xef,0xd0,0x64,0x62,0x1a,0x4a, -0xfb,0x30,0xfc,0x8d,0x1f,0x1f,0x84,0x1f,0x24,0xb0,0x86,0xe2, -0xcd,0xe3,0xa0,0x0b,0xd4,0xef,0x02,0x48,0xaa,0xcf,0x6d,0x20, -0x09,0xf1,0x88,0xc4,0xe7,0x03,0x36,0x94,0x9d,0x7e,0x58,0x49, -0x3d,0x6d,0x12,0x5c,0x71,0x3c,0xd0,0x62,0xb0,0xc0,0x4c,0x78, -0xd0,0x77,0x57,0x5e,0xde,0x10,0x79,0xdf,0x01,0xb9,0x05,0xa3, -0x1d,0xe2,0x46,0xb4,0x40,0x76,0x1f,0x4a,0xc4,0xca,0x9c,0xba, -0x5d,0xb4,0x55,0x58,0xf0,0x7e,0xf0,0x6f,0xee,0xe7,0x03,0xfd, -0x69,0x8f,0x00,0xcc,0x10,0x1a,0x57,0x9b,0x6d,0xf2,0xc8,0xc8, -0x3b,0x51,0x37,0x40,0xcc,0x53,0x86,0x8c,0x4c,0x87,0x16,0x1c, -0x08,0xd6,0x85,0xab,0xc0,0xc0,0x87,0xcb,0x0b,0xb9,0x8d,0xe4, -0x82,0xc0,0xd4,0xe8,0x98,0xde,0x8c,0xdc,0x10,0xee,0x01,0x88, -0x19,0xef,0xfd,0x54,0x8c,0x26,0x23,0x52,0x11,0x9b,0x95,0xd3, -0xc1,0x7f,0xa4,0x4a,0xab,0x5c,0xd5,0x58,0x99,0x8b,0xe7,0xe6, -0x06,0x20,0xdc,0x54,0x01,0x88,0x98,0xac,0x2c,0x93,0x36,0x5e, -0x4b,0x9d,0x64,0xcc,0x5c,0xf7,0xa1,0x6c,0x8f,0xc5,0x8c,0xe5, -0xc5,0xc1,0xba,0xe3,0x06,0x8d,0x63,0xb6,0xb6,0x0a,0xf1,0x5c, -0x5d,0xb2,0xbc,0x45,0xc6,0x60,0xd7,0xfb,0xf6,0x86,0x5a,0xe1, -0x7d,0x6b,0x9f,0xde,0x6d,0xdb,0x1e,0x84,0x1a,0x23,0x70,0x8c, -0x6e,0xc3,0x99,0x9d,0xd2,0xa6,0xa2,0xc7,0x61,0x43,0x6f,0x19, -0xf7,0x68,0x91,0xe2,0x05,0x4a,0x3d,0x20,0xe5,0x58,0xf9,0x1d, -0x27,0xd6,0x93,0xce,0xe5,0xdc,0xda,0x2e,0xa4,0x53,0x18,0x38, -0x2d,0xdf,0xed,0x36,0x4e,0x82,0x7c,0x54,0xb3,0x57,0xd0,0xbe, -0xfc,0xae,0x7d,0x1e,0xd2,0x07,0xb5,0x51,0x54,0x27,0x52,0xdf, -0x75,0xbb,0x9e,0x52,0xd0,0x2f,0xe8,0x1b,0x88,0xb8,0xdb,0xec, -0x44,0x9e,0x8e,0x3b,0xe6,0x28,0x25,0x19,0xbf,0x6d,0x32,0x56, -0x68,0x26,0x20,0x5e,0xe7,0xcd,0x14,0x8f,0xe6,0xbd,0x15,0x86, -0x53,0x42,0xc3,0x75,0x88,0xe6,0x69,0x8b,0x40,0x1c,0x61,0xe9, -0x5c,0x1b,0xcc,0x98,0xab,0xc8,0x3e,0x4d,0xee,0x11,0x77,0x7b, -0x79,0xa8,0x17,0x5c,0xa6,0x69,0xb8,0x0e,0x24,0xc7,0x56,0xed, -0xfa,0x2a,0x67,0x73,0x37,0x60,0x89,0x44,0x8c,0xef,0xa3,0xad, -0xdf,0xdf,0xef,0x8f,0x9a,0xb4,0x27,0x86,0xf3,0x22,0xff,0xbe, -0xc5,0x50,0x6f,0xa9,0xab,0x47,0x62,0x17,0x67,0xee,0x90,0x51, -0xca,0xf4,0xc1,0x80,0xa6,0xe7,0xbc,0x93,0x26,0xf0,0x8b,0x58, -0x6a,0x62,0x54,0xa7,0x9b,0x53,0xe8,0xf1,0x9f,0xe0,0x17,0x7a, -0xa3,0x2f,0xd7,0x20,0xce,0xa2,0xc3,0xc7,0xb7,0xf0,0xcb,0xb0, -0xcd,0x46,0x92,0xfb,0x0e,0x54,0xb0,0x15,0x8b,0x97,0x63,0x02, -0x3f,0xf9,0x52,0x31,0x31,0x17,0xe5,0x7c,0x8e,0x9e,0x3a,0x68, -0xb8,0x96,0xe0,0xc7,0x33,0xfe,0xe0,0x1c,0x35,0x69,0xa3,0x9c, -0x9f,0xe5,0x29,0x43,0xb3,0x5e,0xde,0x42,0x95,0x05,0x00,0xb0, -0xbc,0x85,0xe2,0x0b,0x8c,0x1f,0x34,0xe7,0x14,0xf8,0xc1,0x29, -0x40,0x46,0x18,0xac,0x2e,0x81,0xbf,0x3f,0x4c,0x57,0x89,0xa1, -0xb0,0x41,0x62,0x0c,0x9a,0xd0,0xc7,0x37,0xf4,0x91,0x18,0x31, -0x6d,0x2e,0x67,0x15,0x19,0xec,0xe6,0x6a,0xbb,0xfd,0x44,0x12, -0x92,0x58,0xd7,0xda,0xba,0x71,0xb8,0x43,0x8b,0x6c,0x1e,0xff, -0x8a,0x8d,0xe3,0x77,0x72,0xfb,0x03,0x8b,0x30,0xe0,0xfa,0x58, -0xc0,0xe3,0xfa,0x2e,0xce,0x67,0x83,0x99,0xff,0x51,0x6e,0x48, -0x8b,0x90,0x55,0xbe,0x2c,0x73,0xb6,0x4b,0xe0,0x1f,0xb7,0x39, -0xb8,0xc6,0x78,0x66,0x74,0xf1,0x09,0x85,0xee,0x15,0x93,0xad, -0x74,0x8c,0x82,0x54,0x73,0xbb,0x20,0x63,0xff,0x44,0x29,0x3e, -0xb1,0xf2,0x4c,0x50,0xad,0xdb,0x0d,0x3e,0x3d,0x81,0x9a,0xec, -0x62,0x9e,0x2e,0x36,0x69,0x58,0x80,0xd1,0x32,0x38,0xc9,0x72, -0x77,0x91,0xc3,0x8c,0x2d,0x38,0xa1,0x80,0x5c,0xe8,0x2e,0x6c, -0xda,0xf2,0xc5,0x32,0x0a,0x5d,0xbf,0x46,0x27,0x5e,0x2c,0x3c, -0x3a,0x38,0xe8,0xc6,0x5c,0x34,0x87,0x2e,0xa6,0x63,0xfa,0x36, -0x6d,0xd1,0x2a,0x5e,0xf0,0xdb,0x9e,0x2d,0xba,0x2c,0x2c,0xd6, -0x66,0x93,0xc5,0x95,0x30,0xa4,0x55,0xd5,0xa6,0x02,0xa3,0x77, -0x59,0xbd,0x9b,0x98,0x6e,0xb7,0xc1,0x44,0x39,0x14,0x2a,0xee, -0x66,0x7f,0x8c,0xe3,0xa3,0x5f,0x1b,0xb6,0xd2,0x3e,0x9d,0x54, -0xc6,0x7a,0x7f,0x10,0xb8,0xdd,0x2e,0xff,0xed,0xbf,0xb5,0x41, -0x6d,0x47,0x4d,0xf1,0xc8,0x8f,0x36,0xb8,0x87,0x9f,0x87,0x46, -0x79,0x9e,0x42,0xe7,0xd0,0x86,0x9e,0x1f,0xc2,0x0a,0x2c,0xef, -0x03,0xc8,0x08,0x19,0x7a,0x73,0xe7,0x3c,0xb0,0xea,0xb9,0xeb, -0x4f,0xda,0x43,0x79,0xb1,0xa2,0xd3,0x1c,0xfc,0xd8,0xed,0xcc, -0x61,0xb8,0xd3,0x58,0x8e,0x94,0x7d,0xe5,0x20,0x3d,0xb6,0x0a, -0x6d,0xcd,0x69,0x3b,0xb8,0x0b,0x86,0x2f,0xc8,0x12,0x6b,0x59, -0xe3,0x5b,0x39,0x78,0xf0,0x76,0x6b,0x13,0xbf,0x98,0xef,0x7c, -0xea,0xb4,0x1e,0xee,0x3c,0x6d,0xf4,0xae,0x0f,0x6a,0xb3,0xd8, -0x66,0x05,0x65,0x94,0x2b,0xf9,0x57,0x2b,0xc3,0xc4,0x35,0xdc, -0xc4,0xa9,0x56,0x89,0xf5,0x1e,0x7d,0xbe,0xec,0xfd,0x97,0x20, -0x97,0x90,0x95,0x26,0x15,0xbd,0x91,0xb9,0x20,0xd5,0x24,0x7b, -0xbe,0xb9,0xcf,0x8b,0x25,0x6c,0x94,0xb5,0x26,0x8c,0x92,0x84, -0xc3,0xc5,0x0a,0x42,0x03,0x44,0x67,0xbf,0x07,0xd3,0x9f,0x06, -0xad,0xbc,0x10,0x5c,0xc7,0xe5,0x6c,0x0b,0xc1,0x2b,0x82,0xe2, -0x6f,0xd0,0x38,0x12,0x58,0x30,0x79,0xbe,0x50,0xdf,0xa6,0xaa, -0x1e,0x3b,0x70,0x5d,0xd1,0x7b,0xb8,0xdf,0x28,0x1e,0xe6,0xce, -0x34,0x16,0x41,0x24,0x5a,0xf0,0x8c,0x59,0x4a,0xc0,0x67,0x9c, -0x0e,0xa3,0xa6,0x49,0x34,0x89,0x15,0x0f,0x47,0x20,0xc9,0x06, -0x24,0x14,0x04,0x32,0x19,0xcb,0xab,0xd9,0x46,0x5f,0xcb,0x9e, -0x34,0xc3,0xfd,0xb5,0x9f,0xfd,0xa2,0xf7,0x89,0xda,0x0b,0x99, -0xfd,0x42,0xae,0x98,0x38,0x39,0x38,0xf7,0x9d,0x84,0x12,0xb4, -0x2e,0x70,0xe7,0x20,0xb1,0xea,0xdb,0x51,0x30,0xd1,0x48,0x4a, -0x0e,0xdd,0xa2,0x1d,0xbb,0x91,0xa1,0x92,0x3f,0xb4,0x49,0xea, -0xf5,0x05,0xfc,0xfb,0x3f,0x79,0xdc,0x7b,0x64,0x8b,0xfd,0x07, -0x50,0xf3,0x20,0x1b,0x06,0x1a,0xd9,0xc6,0x41,0x94,0xc6,0x6c, -0xf6,0x0e,0x69,0xff,0x08,0x22,0xf9,0xad,0x85,0x21,0x84,0x01, -0xf4,0x9b,0xb0,0x7d,0x49,0x8a,0xc7,0x06,0xbe,0xc9,0x93,0xa2, -0x8d,0xab,0x53,0xc6,0x07,0xfb,0x09,0x5c,0x55,0xb6,0xdb,0x0f, -0x73,0x82,0x03,0xd5,0x0b,0xab,0xb7,0x6e,0xc4,0x88,0xe8,0x7b, -0x9e,0xea,0x0e,0x58,0x8b,0x94,0x86,0x5e,0x29,0x9c,0xd9,0x66, -0xb9,0xbd,0x93,0xcb,0xea,0x37,0x9e,0x5f,0xa7,0x8a,0xf3,0x1e, -0x4a,0x61,0xff,0x5a,0xfe,0x11,0xa9,0x3c,0x25,0xd1,0x55,0xeb, -0x76,0x1b,0x49,0x61,0x15,0x56,0x58,0xb4,0x4d,0x8b,0xf8,0xd5, -0x87,0x63,0x85,0xbd,0x5e,0x33,0x8a,0x04,0x7e,0x2c,0xe9,0x8c, -0xd4,0xa8,0x2b,0xbe,0xfe,0xee,0x1d,0x0a,0xa6,0x33,0x3e,0x72, -0x9a,0x44,0x43,0x52,0xf8,0xd4,0x66,0xe9,0x2c,0xb8,0x05,0x38, -0x48,0x66,0xf2,0x6e,0x4a,0x93,0xda,0x54,0x8f,0x3e,0x42,0x37, -0xdf,0xdc,0x9d,0x3c,0x03,0xbd,0xf7,0xef,0x07,0xa9,0x75,0x86, -0xc3,0x32,0x1f,0x64,0x59,0x47,0xcd,0x0b,0x97,0x1b,0x15,0xbd, -0xd4,0x6f,0xbf,0x5d,0x43,0xc9,0xaa,0x4e,0x4f,0x43,0x89,0x0a, -0x4a,0x9f,0xa1,0xb1,0x6f,0x10,0x46,0x47,0x78,0x02,0x45,0xeb, -0x03,0x86,0x5c,0x6c,0x18,0xf5,0xa6,0x7c,0xcf,0x8e,0xd7,0xf7, -0x5f,0xc0,0x4f,0x74,0x18,0x5a,0x82,0x5c,0xa8,0x49,0x50,0x97, -0xbe,0xf5,0x85,0x5c,0xd8,0xe8,0x6e,0xb2,0x2d,0x9d,0x1e,0xee, -0x8b,0xb9,0xd8,0x0f,0xcb,0xf5,0xea,0x8a,0x8b,0x8a,0xfb,0x35, -0x1b,0x74,0xd1,0xbf,0xcb,0x9b,0xcd,0xf9,0xfc,0x66,0x4d,0x9e, -0x48,0x48,0x36,0xc8,0x09,0x28,0x58,0x06,0x56,0x18,0xbf,0xe8, -0x4f,0xee,0xd1,0x1b,0xe3,0xfd,0xb4,0xdf,0xcb,0xf0,0x91,0x72, -0xeb,0x99,0x44,0xfd,0xdf,0xf1,0x73,0x24,0xe8,0x99,0x14,0x44, -0xdd,0x2c,0xd5,0x94,0xb6,0x36,0x57,0xd3,0xc5,0x6c,0x5e,0xae, -0xd9,0xb3,0xc5,0x5d,0x4f,0xaa,0x69,0x21,0xb5,0x53,0x9b,0x8d, -0x94,0xfb,0xe9,0xfc,0xf5,0xd3,0x85,0x21,0x19,0x62,0x3a,0xe7, -0xcc,0xef,0x28,0xc3,0xe5,0x6b,0x8b,0x62,0x8a,0x6a,0x01,0xae, -0xcd,0x72,0x5d,0xbd,0xa2,0x58,0x1e,0xd8,0xfd,0x63,0xcf,0x31, -0xc3,0x99,0xfa,0xd9,0xcb,0xa4,0xc7,0x36,0x04,0xab,0x34,0xd7, -0x97,0xbf,0xb0,0x76,0x3d,0x50,0x0a,0x05,0x5f,0xfe,0x16,0x5e, -0x9e,0xd6,0xb0,0xa3,0x0a,0x32,0xdd,0x2b,0xcf,0xda,0x01,0x87, -0x33,0x4f,0xfd,0x4f,0x05,0x90,0x83,0x9b,0x67,0x86,0x5d,0xe0, -0xf9,0x06,0x05,0x21,0x64,0x34,0xd7,0xe8,0xc2,0xd1,0x9e,0x43, -0x2f,0x34,0xa6,0x1e,0x9e,0x5c,0x01,0xee,0xc7,0x56,0x6d,0xcf, -0xf6,0x42,0x7d,0x3b,0x0b,0x34,0x3e,0xb7,0x31,0x68,0xed,0xd7, -0xe4,0xee,0x84,0x21,0x4f,0xa9,0xf3,0x6b,0x74,0xb0,0x28,0x60, -0xbd,0xd3,0x0d,0x8e,0xb7,0xab,0x05,0x25,0xd1,0x6f,0x10,0xf5, -0x8d,0x12,0xf2,0xce,0x03,0xad,0x4f,0x44,0xe3,0x85,0x87,0x33, -0x71,0x26,0x3f,0x06,0xc1,0x74,0xc5,0x1e,0x0a,0xf5,0x61,0xfb, -0x96,0x04,0xd8,0xc5,0x86,0xce,0xa1,0xf6,0x86,0x7b,0xa3,0x56, -0xa0,0x21,0xb5,0xf3,0x63,0x58,0x54,0x72,0xbc,0x99,0x90,0x89, -0x02,0x1b,0x90,0x2a,0x4d,0x61,0xe4,0x7c,0x7c,0xb9,0xda,0x91, -0x5b,0x81,0xa1,0xf6,0xc7,0x0f,0x26,0x0c,0x82,0x84,0x52,0xee, -0xe3,0x4f,0x60,0xb1,0x68,0xa5,0xcf,0x44,0x5c,0x04,0xc3,0x97, -0xc4,0xb1,0x5c,0x54,0xde,0xf1,0x13,0x75,0x85,0xbd,0xb7,0x1f, -0x49,0x01,0x0a,0xc2,0xf1,0x0a,0xa4,0x55,0xec,0x3b,0xd7,0x44, -0xd4,0x46,0x90,0x4a,0x68,0xbb,0x25,0xda,0xff,0xc8,0x2e,0x2c, -0x59,0x46,0xac,0x91,0x55,0x94,0x6c,0xee,0x2b,0xc3,0xcc,0xed, -0x1a,0x22,0x07,0x2a,0x5a,0xb4,0x42,0xb0,0xb9,0x92,0x3b,0xbd, -0x5b,0xe0,0x53,0xb1,0x7b,0x01,0xd9,0xbd,0x7c,0x8e,0xe6,0xd2, -0x12,0xf3,0xc9,0xa6,0x3a,0xa2,0xc1,0xb7,0xd0,0x69,0xd2,0xfa, -0x7e,0x41,0x3e,0xe4,0xba,0x85,0x63,0x71,0x9d,0x3b,0xac,0xf7, -0x5f,0x2f,0xab,0x05,0xe1,0x79,0xe7,0xb3,0x0c,0x58,0x04,0xca, -0x16,0x38,0x36,0x44,0xcd,0xe3,0xcf,0xdc,0x82,0x0b,0x33,0x30, -0x06,0xab,0x66,0x58,0x64,0x3f,0x22,0x3b,0xe7,0x13,0xc5,0x2b, -0xca,0xa9,0x37,0xab,0x6e,0x37,0xf8,0xf4,0xdc,0x23,0x08,0x3b, -0x1e,0xff,0xf1,0xe8,0x35,0x23,0x5d,0x82,0xde,0x43,0xc5,0xd1, -0xd4,0x47,0xad,0xa9,0x6c,0x78,0xed,0xf3,0xbc,0xce,0x80,0x0f, -0xea,0x61,0xec,0x8e,0x38,0x25,0xa5,0x68,0x19,0x71,0xe5,0x2c, -0xb3,0x24,0x88,0x3d,0xa1,0xd7,0xb8,0xfb,0xf2,0x06,0x61,0x91, -0x98,0xb9,0x9f,0xfd,0x56,0x96,0x15,0x67,0x14,0xfe,0x47,0x96, -0x59,0x2a,0x18,0x59,0xbc,0x4a,0xe8,0xde,0x76,0x3c,0xf7,0x7a, -0xe6,0xc4,0xeb,0x3d,0xb7,0xa5,0xe8,0x46,0xd2,0x03,0x2b,0xa0, -0x6e,0xde,0x70,0x64,0x0e,0x3b,0x27,0x64,0xea,0xc5,0x01,0x24, -0x9b,0xc1,0x8a,0x5b,0x76,0x21,0x4b,0xa0,0xd7,0xe8,0x91,0x4e, -0x8b,0x49,0x0c,0xf2,0x5f,0x7b,0x3b,0x0c,0x6e,0x4c,0xb8,0x10, -0x08,0x4e,0xb3,0xd1,0x4d,0x4a,0xb7,0xa5,0x4f,0xdf,0x80,0xc4, -0x4d,0x96,0xb7,0xa0,0xc8,0x82,0xa8,0xb1,0x2d,0xa1,0x36,0xbe, -0x7d,0x1b,0xe0,0x0d,0xeb,0x8f,0xf3,0x3f,0xf4,0x62,0xfc,0x5f, -0xc1,0x02,0xd9,0x05,0x01,0x61,0xfe,0x5f,0xc0,0x0c,0x5b,0x97, -0x39,0x8f,0x02,0xc6,0xca,0x80,0x76,0xbb,0x18,0xb9,0x8f,0x63, -0xe1,0xa7,0x49,0xfa,0x62,0x7b,0x76,0xd6,0xcf,0x58,0xd7,0x15, -0xb0,0x17,0x48,0x06,0xa9,0xa7,0x7f,0x0f,0xfe,0x6e,0x33,0xf4, -0x80,0x4b,0xf1,0xd7,0x67,0x19,0x06,0xda,0x52,0xba,0x28,0x5e, -0x17,0x96,0x64,0x05,0xb9,0xaf,0x11,0xb9,0x8e,0xdb,0x6a,0xf6, -0xf8,0xf5,0xc4,0x74,0x3c,0x82,0xeb,0x76,0x15,0xa5,0xc0,0x25, -0xdc,0xda,0xd2,0xc4,0xed,0xd6,0x8a,0x22,0xfe,0x0a,0x0b,0xca, -0xf2,0xc2,0x24,0xef,0x84,0x0e,0xba,0x22,0x10,0xe7,0x74,0xd9, -0x76,0x40,0xd9,0x76,0xeb,0xb8,0xaf,0xf7,0x40,0xbe,0x2b,0xea, -0xb2,0xd3,0xe4,0xde,0x3d,0xd4,0xe7,0x68,0xca,0x76,0xdb,0x69, -0x16,0xf3,0x19,0xaa,0xae,0xef,0xd7,0x18,0x96,0xbb,0xad,0xc5, -0xf6,0x85,0x7f,0x7c,0x6c,0x19,0x13,0xaf,0x58,0xc7,0x61,0xf9, -0xbb,0x95,0x3b,0x65,0x43,0x8b,0x79,0x18,0x73,0x84,0x79,0x8f, -0xbb,0x6d,0xca,0xe9,0x7a,0xb6,0xbc,0x5d,0xb8,0x46,0x35,0xc5, -0x77,0x6c,0xf3,0x98,0x76,0x24,0x2f,0x09,0xe3,0x0e,0xf4,0xb6, -0xcc,0x66,0x2d,0x3b,0x69,0xd4,0x51,0x83,0xf5,0x60,0xe7,0xd9, -0xd9,0x97,0x85,0xf0,0x9b,0x1e,0x17,0xe2,0x55,0x1c,0xac,0x01, -0x79,0xe1,0xd2,0xb6,0xdd,0xd3,0x05,0xd8,0x64,0x59,0x9d,0x13, -0xef,0xdd,0x61,0xef,0xc5,0x53,0xe1,0x16,0x19,0x3d,0x8c,0xcd, -0x70,0x84,0xf0,0xed,0x75,0x63,0x4b,0xb8,0x26,0x3f,0x18,0x2c, -0x32,0x9c,0xc7,0x3e,0x31,0x93,0x37,0x39,0x2a,0x8d,0xf7,0x7c, -0xdf,0x09,0xde,0xc5,0x40,0x82,0x9b,0x87,0xdd,0x1b,0xb1,0x82, -0xb6,0xd5,0xe7,0x14,0xa1,0xe9,0xb9,0x27,0x9c,0x10,0x33,0xad, -0x98,0x85,0xfe,0x3c,0xdd,0x5c,0x15,0x64,0x8d,0xe5,0x1e,0x27, -0x15,0x26,0xd3,0x78,0xde,0x95,0x7b,0xe7,0xab,0x4a,0x38,0x78, -0xb1,0x98,0x89,0xeb,0x9f,0x33,0x3c,0x26,0xb4,0xaf,0xaa,0x2d, -0x62,0xeb,0x3b,0x5e,0xe7,0x78,0x55,0xce,0x8f,0x4c,0xa1,0x55, -0x31,0xf2,0x09,0x2b,0x67,0x3a,0x08,0xcd,0x17,0x4d,0xaf,0xf8, -0x7f,0x6b,0x26,0x75,0xbc,0xd3,0x16,0xaf,0x47,0x9a,0xcf,0x76, -0x39,0x99,0xec,0x4a,0x21,0xdb,0x19,0x3a,0xf7,0xc9,0xbc,0x59, -0x6e,0xed,0x64,0x64,0x54,0xc2,0x81,0xca,0x98,0xf2,0x98,0x95, -0x3a,0x84,0xfa,0x49,0xc4,0x79,0x33,0x99,0x93,0x22,0xec,0x23, -0x67,0x7b,0xfb,0x48,0x5e,0x60,0x86,0x19,0x47,0x5c,0x60,0x94, -0x61,0xec,0x67,0x3d,0x0b,0xf3,0x2a,0xa0,0x7a,0x0d,0xa7,0xee, -0x52,0x44,0x13,0xfc,0x93,0x71,0xb3,0x40,0xa4,0xcf,0x79,0xb8, -0x78,0x73,0x10,0x4d,0x40,0xd1,0x26,0xd1,0xc5,0x85,0x80,0x5e, -0x8b,0x28,0x69,0xf4,0x47,0xf9,0x37,0xd9,0x79,0x18,0x5d,0x7b, -0xf8,0x2a,0x9f,0xe7,0x77,0x22,0x93,0x24,0xef,0xe3,0xa7,0xfe, -0xa7,0x9c,0x3d,0x6c,0x74,0x00,0x79,0x0f,0x96,0x83,0x8c,0x4d, -0xf2,0x56,0xdf,0x65,0x23,0xb9,0x1f,0x7d,0x16,0xf0,0x57,0x18, -0x70,0x5f,0xcb,0xc0,0x18,0x8d,0xc0,0xca,0xc3,0x04,0x3d,0x43, -0xa9,0x2c,0xca,0xbc,0x2b,0x93,0x37,0xa4,0xbd,0xb6,0x30,0x70, -0xbe,0x54,0x5d,0x2c,0x39,0x88,0x9a,0x8b,0x47,0x60,0x5f,0x8d, -0xa6,0x11,0xf2,0x76,0xed,0x96,0x71,0xd1,0xb6,0x07,0xcb,0x8e, -0xdb,0x20,0x78,0x57,0x8d,0xc8,0x8b,0x1e,0xe0,0x17,0x6b,0x7d, -0xa7,0xe0,0xca,0xd0,0xc0,0x7f,0xd8,0x92,0x6e,0x59,0x04,0x4b, -0x80,0x68,0xa3,0x4f,0x5b,0x37,0x16,0xc7,0xbf,0x45,0x91,0x36, -0x95,0xfa,0x6e,0x99,0x66,0x42,0x7e,0xae,0x05,0xdc,0x18,0xf5, -0x3e,0xa6,0x2a,0x6f,0x69,0xbf,0x94,0x0e,0xff,0x26,0x21,0xe3, -0x38,0x88,0x4f,0xb6,0xa3,0xa0,0x75,0xc5,0xc9,0x90,0xe0,0xb5, -0xcd,0x90,0x41,0x0b,0xbe,0x81,0xad,0x04,0x8d,0x1e,0x50,0x53, -0xc0,0x02,0xb0,0xc7,0x67,0x9b,0x25,0x6e,0xe8,0x69,0x36,0xcc, -0x1c,0x73,0x2a,0xaa,0xd3,0xc1,0xc8,0x61,0xa1,0x21,0xab,0x04, -0xa2,0x4a,0x91,0x06,0x72,0x20,0xb2,0x4f,0xcb,0x90,0x29,0x6a, -0xd0,0xd8,0x35,0x3c,0x89,0x84,0x46,0x2a,0xcc,0xcd,0x24,0xba, -0xf7,0xea,0x7e,0xab,0x6f,0xb1,0xdd,0xb0,0xd2,0x45,0xb3,0x0b, -0x66,0x09,0xdd,0x2e,0x64,0x8c,0xf9,0xf7,0xc4,0xd6,0xf4,0xfc, -0xc3,0x38,0x56,0x08,0xa1,0x3f,0x6c,0x71,0x5f,0xfb,0xf2,0xf4, -0x0b,0x03,0xbb,0x5a,0xd3,0x5f,0xb9,0xb6,0x4c,0x59,0xb0,0x75, -0xf8,0xa1,0xe1,0x77,0xd2,0x90,0xd0,0x15,0xbb,0x52,0xe9,0x67, -0x6e,0x02,0x71,0xeb,0x48,0xfe,0xa5,0x4c,0xa4,0xdb,0xd6,0x35, -0xc5,0x5b,0x05,0x21,0x69,0x78,0x6b,0x02,0x6f,0xae,0x28,0xba, -0xa1,0x75,0x48,0x6e,0xbf,0xcb,0x9b,0x22,0xe6,0x3b,0x4d,0x64, -0xd0,0xd2,0x80,0x1c,0x46,0x1b,0xfc,0xa0,0x6b,0xd1,0xd0,0xda, -0x38,0x5a,0x47,0xd9,0x9d,0x6e,0x27,0x16,0xd9,0x24,0xae,0xa5, -0x7e,0x12,0xc7,0x17,0x32,0x7b,0x74,0x29,0x58,0x84,0x42,0x76, -0xba,0xde,0x52,0xef,0xa1,0xa9,0xdd,0x9e,0x6a,0xde,0x43,0x21, -0xcd,0x0e,0xf1,0x01,0x40,0xb5,0xf3,0xf4,0xb9,0xe0,0x8e,0x1e, -0x14,0x21,0xd5,0x4c,0x24,0x01,0x64,0x77,0xbc,0x4b,0x04,0xbd, -0x5d,0x56,0xef,0x24,0x97,0xd4,0x99,0x15,0x99,0x47,0xba,0x03, -0x14,0x1d,0x48,0x00,0x00,0xab,0xac,0x23,0xdf,0xe4,0x42,0xdf, -0xfe,0x38,0x10,0x61,0xdd,0x4a,0xf1,0x69,0x8b,0x1f,0xc4,0xde, -0xd5,0x41,0x62,0xfe,0x41,0x26,0x1b,0x14,0xbe,0xdb,0xe9,0x4b, -0xbd,0xe3,0x13,0x89,0x25,0x29,0xbc,0xdf,0x72,0x39,0x66,0xfa, -0xd4,0xad,0xe5,0x9d,0x40,0xdb,0x8f,0x05,0x47,0x8e,0x30,0xbd, -0x44,0xcf,0xe2,0x9f,0x71,0xc3,0xfc,0xd8,0xd7,0x32,0xfe,0x87, -0xef,0x55,0xcf,0x5d,0x5a,0x51,0x36,0xaa,0x6d,0x51,0x91,0x19, -0xe2,0x4c,0x82,0xd3,0xc2,0x6f,0xe9,0x23,0x19,0x14,0x02,0xc0, -0x25,0x60,0xd1,0x22,0xfb,0x93,0xe1,0x49,0x93,0xac,0x79,0x43, -0x9f,0x2c,0x77,0x84,0xb0,0x79,0xee,0x10,0x13,0x75,0xf5,0xf4, -0xfa,0xba,0x9c,0x55,0x80,0xab,0xf6,0x3e,0xd3,0x4e,0x73,0x37, -0xd7,0xa5,0xee,0xa7,0xed,0x3f,0xba,0x64,0x7a,0x10,0xee,0xc7, -0x47,0x2a,0x24,0x31,0x99,0x2f,0xdc,0x8f,0x5d,0x65,0xda,0x75, -0xf1,0xfa,0x2d,0x4d,0x5b,0xa9,0xa0,0x79,0xd8,0x9a,0x70,0x7c, -0x36,0xce,0xd0,0x93,0x97,0xa7,0xe9,0xc8,0x84,0xbd,0x04,0xa8, -0xe2,0xe7,0x95,0xa1,0x9f,0xe8,0x75,0xc6,0x40,0xac,0x40,0xab, -0x52,0x61,0x8d,0x69,0x3b,0x6f,0x94,0x31,0xa0,0x9b,0xbf,0x87, -0xc3,0xd4,0x7b,0xe7,0xd0,0x12,0xd9,0xb2,0xde,0x28,0x95,0x39, -0x0e,0xe8,0xa7,0x36,0x68,0xcf,0x44,0xab,0x5b,0xe7,0x31,0x96, -0xef,0xaf,0x02,0x89,0x1e,0x0e,0x1b,0x8d,0x65,0x8c,0xfe,0x2b, -0xf1,0x32,0x0e,0x35,0x66,0xed,0xe7,0x3c,0x63,0xf7,0x52,0x91, -0xa1,0x28,0xac,0xba,0x5f,0x82,0xf6,0x22,0xdf,0x8f,0x46,0x68, -0x46,0x1c,0xd8,0x95,0x69,0xa3,0xac,0x6d,0x97,0x6f,0x66,0x9d, -0x04,0x3b,0xb4,0x3c,0x5b,0x24,0x09,0x7e,0x7e,0x54,0x9d,0x98, -0xa2,0x2e,0xf0,0x53,0x6d,0xec,0x29,0xcc,0xd4,0xbe,0x6e,0xf8, -0x7d,0x36,0xc6,0x80,0x3c,0xb6,0x3b,0x0c,0xa0,0x17,0x5f,0x82, -0x96,0xf3,0x7e,0x35,0xc1,0x53,0x5b,0xcb,0x21,0x9b,0xdc,0xae, -0xa4,0xcd,0x31,0x24,0x4e,0x8a,0x90,0x80,0x82,0x2c,0x6f,0x4d, -0x78,0x4a,0x52,0xb1,0x28,0x42,0x25,0x89,0x61,0x6b,0x78,0x12, -0xef,0x69,0xbf,0xc6,0x9b,0x26,0x75,0x69,0x47,0x5b,0x7b,0x2d, -0xc3,0x81,0x87,0xc6,0x50,0x64,0xa2,0x71,0xc8,0xb3,0x00,0x8c, -0x6e,0x57,0xbe,0x62,0xe5,0xdb,0x50,0xd3,0xf5,0x9c,0xed,0x13, -0x02,0x17,0xa6,0xdd,0x29,0x47,0x69,0xc0,0x92,0x98,0x54,0xda, -0xb9,0x78,0x50,0x3e,0xde,0xbe,0x6e,0x1c,0xde,0xf7,0x35,0x4a, -0xb0,0xdb,0x56,0x9d,0xda,0x81,0xb4,0x0e,0x41,0x9b,0x78,0x69, -0xee,0xb7,0x82,0x8f,0x37,0xbd,0x6b,0x6c,0x64,0x2a,0x8b,0xc4, -0x67,0x9d,0xcc,0xdf,0x10,0x65,0x43,0x23,0xbf,0xf8,0x8b,0xe5, -0xea,0x3d,0x9f,0xba,0x1c,0x65,0x90,0x06,0xa8,0x5a,0xe0,0x23, -0xfc,0xf6,0xf0,0x84,0xbd,0x91,0x75,0x0e,0xf9,0x30,0xc8,0x87, -0x28,0x02,0x28,0x7a,0x85,0x24,0xa9,0x7f,0x53,0x58,0xa2,0xd0, -0xda,0xde,0xd5,0x9e,0x3b,0x41,0x8a,0x6f,0x28,0x65,0xb1,0x75, -0xa3,0xbd,0x12,0x6c,0x14,0x82,0x0c,0x31,0x80,0xdc,0x65,0x04, -0xbc,0x36,0x4d,0xb7,0xc3,0x35,0x17,0xa2,0x9f,0xfa,0x2e,0x6f, -0x90,0x2f,0x0e,0xa2,0xf4,0x5b,0xce,0x86,0x8d,0xf3,0x5f,0x30, -0x7a,0xdc,0x9a,0xb0,0x1b,0x1b,0x96,0x0c,0x15,0x60,0x64,0x5a, -0x88,0xa9,0xb8,0x0e,0x18,0xdf,0x98,0x34,0x29,0x82,0xaa,0x9c, -0x36,0x0c,0x44,0x91,0xf6,0x93,0x57,0x50,0xad,0x5f,0xaf,0x2f, -0xf0,0xde,0x34,0x14,0xfd,0x0d,0x06,0xab,0xf4,0x2b,0x05,0x1e, -0xcb,0xc1,0x31,0xce,0x2f,0xe4,0x1d,0x3e,0x84,0x5d,0x5e,0x97, -0x9b,0xe9,0xff,0x28,0xdf,0x17,0x9d,0x4e,0xf0,0xad,0xf3,0x0b, -0x13,0x37,0xc7,0x30,0x3b,0xe1,0xa7,0x70,0xd2,0x10,0x33,0xac, -0xa7,0x60,0xa3,0xd4,0x3a,0x4f,0xa6,0xf3,0x0d,0xb4,0x73,0xc4, -0x67,0x83,0xfa,0xe8,0x62,0xba,0xb8,0x28,0xe7,0xc8,0x84,0x8e, -0x2e,0x36,0xeb,0x39,0x66,0x05,0x5b,0x33,0x23,0xe4,0xe7,0xab, -0x69,0x8d,0x21,0x8c,0x08,0x86,0xa3,0x35,0x54,0x00,0xa9,0x58, -0x0a,0xd0,0xd9,0x1f,0x93,0x79,0x2c,0x14,0x50,0xe6,0xd9,0x66, -0x7a,0xbd,0x3a,0x7a,0x0b,0x67,0x9d,0xa3,0xdb,0xab,0xea,0xe2, -0x2a,0x7c,0xe1,0x53,0x68,0x0e,0x2f,0x6b,0x1d,0x9d,0x08,0x78, -0x17,0x57,0xe8,0x2a,0x0a,0xff,0x3c,0x02,0x64,0x60,0x74,0x26, -0xfc,0x0f,0x7f,0x47,0x4d,0xe0,0x70,0xe3,0x1d,0x44,0xc7,0x1d, -0x5d,0xfc,0x33,0x02,0x09,0x0e,0x3b,0x09,0xf4,0x55,0xc8,0x03, -0x4e,0x52,0xad,0xaf,0xdd,0x8e,0x1a,0x29,0xb9,0x4d,0x11,0x68, -0x64,0x9a,0x60,0x3b,0xf3,0x96,0x83,0x8e,0x81,0x77,0x8d,0x23, -0xfe,0x03,0x28,0x9e,0x57,0x50,0xf4,0xef,0xf2,0xf7,0x1f,0x47, -0x97,0xeb,0xe5,0xb5,0x50,0xce,0x11,0xc7,0x13,0xfd,0xbb,0xfc, -0xfd,0xc7,0x11,0xec,0xbd,0xe5,0xdf,0xe9,0xdf,0x7f,0x1c,0xd5, -0x17,0xeb,0xb2,0x5c,0xfc,0x5d,0xfe,0xfe,0xe3,0x68,0xb3,0x94, -0x5a,0x9f,0x84,0x0a,0x64,0x22,0x18,0x74,0x94,0x01,0x86,0x23, -0x89,0x01,0x62,0x35,0x62,0xac,0x67,0x87,0xc5,0xdf,0xc6,0x03, -0xcd,0xe5,0x79,0x89,0xc3,0x16,0xc4,0x12,0xc8,0x6a,0x25,0xed, -0x30,0xc7,0x83,0x56,0x84,0x43,0xbf,0x21,0xc5,0xef,0x39,0x3b, -0x23,0x70,0x85,0xd6,0x88,0x9f,0xcf,0xe5,0x08,0xdf,0x36,0xd7, -0x8d,0x8a,0x61,0x68,0x74,0xde,0xc3,0x08,0xbf,0xf4,0x1a,0x6f, -0xbf,0xa6,0x77,0x6f,0xbf,0x2f,0x2f,0xa1,0x2f,0x8e,0xfc,0xcd, -0xc1,0x59,0xbd,0xe4,0x93,0xec,0xd8,0x95,0xe7,0x16,0x1a,0xe5, -0xfd,0xe4,0x93,0xcc,0xeb,0xfc,0x1f,0x71,0xe7,0xff,0x68,0x74, -0xfe,0x7c,0xb9,0x6a,0xe9,0x9b,0x52,0x5b,0xba,0x8e,0x4b,0x7b, -0xa9,0x27,0x18,0x03,0x4d,0x85,0x2a,0x6f,0x25,0x76,0xbb,0xde, -0x4c,0x79,0x42,0xa0,0x57,0xa4,0xf0,0x27,0x38,0x64,0x54,0x8e, -0xe8,0x2d,0x9d,0xe5,0x5e,0x69,0x1d,0x2c,0x2d,0x9c,0xad,0x44, -0xf6,0x29,0x42,0x8f,0x59,0x7f,0x69,0x0d,0xba,0x5c,0x64,0x34, -0xc8,0x1f,0xe8,0xcf,0x2f,0xf2,0x2f,0xf5,0xe7,0x83,0xfc,0xc4, -0x2d,0x21,0x11,0x23,0xf3,0x3b,0x7c,0x76,0x39,0xbf,0x53,0xcd, -0x10,0xc5,0x75,0x25,0x89,0x28,0xbf,0x6b,0xe8,0x84,0xf7,0xda, -0xc2,0xf2,0x01,0x4d,0x6c,0x61,0xbb,0x5d,0x67,0xb0,0xa3,0xde, -0xd2,0x7c,0xec,0x16,0xd7,0x77,0x68,0x7c,0x94,0xba,0xdf,0x20, -0x0a,0xe3,0x55,0xa7,0x1a,0xc9,0xf0,0x2b,0x01,0xad,0xbd,0x4b, -0xfc,0x4d,0xff,0xc9,0x10,0x7c,0x94,0xee,0xad,0x7b,0x04,0x9a, -0xf7,0x57,0x6c,0x20,0xc3,0xa3,0xb3,0x1f,0x78,0x83,0x52,0xa9, -0x2f,0x3f,0x9e,0x8c,0x09,0x6e,0x97,0x12,0x31,0x91,0x49,0x76, -0x06,0xed,0x62,0x0e,0xa1,0x40,0xdf,0x49,0x3b,0x04,0x08,0xb6, -0x21,0x23,0xc5,0x9f,0xe1,0x40,0xdb,0x7a,0x5e,0x02,0xfe,0xa0, -0xeb,0x12,0x84,0x86,0xf2,0x66,0xc1,0x33,0xe3,0xcb,0xf9,0xed, -0x87,0x73,0x11,0xf3,0x5b,0x8f,0x23,0xe1,0xf6,0xc9,0xb0,0xb3, -0x79,0xb2,0x5f,0x15,0x6d,0xff,0x4c,0x5d,0x5d,0xdf,0xcc,0x83, -0xc7,0x1c,0xed,0x75,0x87,0x9c,0x49,0x79,0xff,0x62,0xf6,0x56, -0x46,0xe6,0x04,0xb1,0xa8,0x20,0x55,0x3c,0x2b,0x83,0xaa,0x7e, -0x26,0x3d,0x70,0xec,0x61,0x1f,0xb2,0x1c,0x43,0xfb,0x0c,0xb9, -0xfd,0x51,0x9b,0x66,0x23,0x65,0x9f,0x79,0x82,0x26,0xdb,0x63, -0x54,0xe2,0x6e,0x78,0x70,0xab,0x68,0x55,0x29,0xed,0x53,0x54, -0x39,0x9b,0x59,0xef,0xce,0xc7,0x7f,0x98,0xdd,0x26,0xda,0x5b, -0xfc,0x96,0xc8,0x9a,0x72,0x21,0x44,0xe2,0x6b,0x5b,0x1d,0xb9, -0xbd,0x6f,0xc9,0x49,0xbd,0xfa,0x48,0x24,0xbb,0xb6,0xc0,0x9d, -0xda,0xbc,0xfa,0xf7,0x15,0x56,0x95,0xcf,0x31,0x4f,0xf8,0x5d, -0xf1,0x27,0xac,0x8c,0xf7,0xfd,0x26,0xc5,0x28,0xb4,0xdd,0x61, -0xd2,0x2b,0xc0,0xfa,0xa8,0xb8,0x2d,0xb6,0x0f,0x95,0xce,0xdd, -0x1b,0xe4,0x8c,0x22,0x17,0x7b,0x62,0x7d,0xc1,0xd2,0x4e,0xb0, -0x42,0x8e,0xaa,0x45,0xbd,0x41,0x69,0xc7,0x9a,0x14,0x71,0xbd, -0x11,0x96,0x87,0xb3,0xec,0xfa,0x82,0xed,0x85,0x78,0x91,0x84, -0xb2,0x35,0xb6,0x68,0x99,0x47,0xa1,0x45,0x39,0xa9,0x39,0xb7, -0x54,0x60,0x16,0x25,0x6e,0xb7,0x98,0xea,0x53,0x3d,0xeb,0x03, -0x31,0x15,0xd8,0xef,0xcf,0x01,0x15,0x30,0x38,0x8d,0xe4,0x54, -0x1f,0xd7,0x7c,0xbe,0x66,0x23,0x77,0xf8,0xc9,0x4f,0x5b,0xe7, -0x01,0x74,0x3c,0x7a,0xcf,0xca,0x85,0x96,0x05,0x71,0x46,0xc6, -0x8b,0x0c,0x46,0x25,0xb5,0xc2,0x61,0x40,0x93,0xac,0x36,0x71, -0xb1,0xbc,0x95,0x47,0x78,0xe3,0xc3,0x0a,0xde,0xa6,0x58,0x97, -0x82,0x58,0x3a,0x77,0x73,0x10,0xce,0x12,0xca,0xf4,0x9b,0x25, -0x01,0x7a,0xd7,0xc4,0x9c,0x3f,0x26,0xd3,0xa6,0x44,0x8a,0x0a, -0x1c,0x50,0xfd,0x04,0x25,0xc3,0x35,0x16,0x3f,0xde,0x2a,0x81, -0x50,0x82,0x29,0x1f,0xee,0x9b,0x5b,0x37,0x01,0x86,0x64,0xfa, -0x68,0xfd,0x8e,0xe2,0x84,0x14,0x04,0xf0,0x60,0xda,0x71,0x49, -0x99,0x48,0xd5,0xf2,0x29,0x10,0x35,0x87,0xda,0x84,0x29,0x6a, -0x1e,0x16,0x7b,0x53,0xb9,0x63,0x50,0xe7,0x89,0xc2,0x3f,0xef, -0xb6,0x05,0x85,0x03,0xc6,0x52,0x6d,0x48,0xcd,0x63,0xf7,0xf8, -0x83,0xb8,0xf7,0x01,0x62,0x67,0xfa,0xb8,0xf3,0xc8,0x31,0xe0, -0x8e,0xa4,0x68,0x44,0xf0,0x3a,0x4f,0xe8,0x37,0x70,0xa5,0x75, -0xc2,0xc2,0xf5,0xbc,0x9c,0xbe,0x2d,0x35,0x99,0x76,0x25,0xf7, -0x1a,0x2b,0x20,0x08,0xcf,0x13,0xee,0x21,0xdd,0x40,0xcb,0x86, -0xd9,0x93,0xe2,0x2e,0xd8,0xdd,0xa0,0xb4,0xd1,0x5b,0x10,0xfa, -0x60,0x6e,0xd2,0xd8,0xce,0xd4,0x7d,0x72,0xe3,0x29,0x60,0x45, -0x96,0x2a,0x5a,0x24,0x2b,0xcf,0xb6,0x2d,0x52,0x15,0x8a,0xcc, -0x9c,0x76,0xa4,0xfc,0x76,0x2b,0x3f,0xf0,0x21,0x75,0x11,0xdc, -0x54,0xd3,0x2f,0x2f,0x00,0xd4,0x29,0x67,0x68,0x87,0x9e,0x06, -0x92,0x56,0x4e,0x53,0x7b,0x48,0xba,0xc6,0xa6,0x75,0x54,0xeb, -0x73,0xbe,0xc6,0x6b,0x09,0x71,0x67,0xc8,0xda,0xbf,0x69,0xc8, -0xce,0xcf,0x26,0xc9,0x53,0x47,0x9e,0x7b,0x83,0x8f,0x62,0x29, -0x54,0xdc,0x91,0x49,0xda,0x47,0x8b,0x67,0xfc,0x24,0x1b,0x3a, -0x8b,0x84,0xad,0x62,0x08,0x4b,0x2e,0x41,0xe2,0x58,0xff,0x25, -0x37,0x8f,0x67,0xc0,0x15,0x86,0x6c,0xd5,0x04,0xef,0x71,0x28, -0x95,0x01,0xe8,0xba,0x5c,0x64,0x59,0x83,0xed,0x17,0x07,0xbd, -0xa4,0x3c,0xfe,0x16,0x64,0xb3,0x7c,0xaa,0x2e,0x5c,0xd8,0x8e, -0x73,0xa6,0x41,0x05,0xca,0x75,0x1c,0x3f,0x1d,0xd3,0x4c,0x12, -0xe0,0x0a,0xe5,0xce,0xe6,0xb8,0xfc,0x82,0x6d,0xe3,0xf0,0xa5, -0x28,0xc9,0x7e,0x79,0xae,0x4b,0xd3,0xcd,0xce,0xfe,0x5e,0xd1, -0x2e,0x23,0x88,0x01,0xfe,0x11,0xe2,0x5a,0xd8,0x91,0x67,0xb8, -0xd1,0x92,0xcb,0x4b,0xd9,0xf7,0x19,0xb0,0x2a,0x78,0xb9,0x5f, -0x8f,0x8c,0x71,0x55,0x8a,0x4b,0x13,0x1d,0x6d,0xd4,0x82,0x08, -0x66,0x04,0xb7,0x51,0xeb,0x98,0x3f,0x4c,0x44,0x62,0xc9,0xc2, -0x85,0x2c,0xa6,0x1d,0x5e,0x9a,0x94,0xce,0xcf,0x82,0x7d,0x80, -0xd2,0xb9,0xd0,0x7e,0x4a,0x77,0x96,0xf4,0xa2,0x0d,0xf3,0x83, -0x13,0x65,0xa3,0x34,0x6d,0x3f,0x8a,0x6c,0xb7,0xce,0xc3,0xde, -0x26,0xb6,0x12,0x10,0x0f,0x08,0x37,0xd5,0x72,0xbd,0x79,0xcf, -0xe0,0xf4,0x5f,0xf2,0xdf,0x26,0x21,0xd9,0xc7,0xe4,0xec,0x39, -0x2f,0x94,0xbf,0xb5,0x1d,0x84,0x4e,0x63,0x8b,0xbd,0x7c,0x7d, -0x53,0x6f,0xa4,0xc5,0x19,0x6d,0x08,0xd1,0x95,0x60,0x63,0x7d, -0xee,0xeb,0xbd,0xd9,0x5c,0x1b,0xb5,0xb4,0xf6,0x3a,0x88,0xfa, -0x74,0x54,0xa4,0x9d,0x79,0x57,0x53,0x04,0x23,0x9f,0x65,0xf6, -0x21,0x8c,0x8f,0x30,0x74,0x26,0xc2,0xc7,0x8d,0x5b,0x40,0x6e, -0xe1,0x20,0xc3,0x78,0x36,0xad,0x8d,0x8d,0xf8,0x3d,0xb7,0x3d, -0xa2,0x10,0x90,0x51,0x3b,0x13,0xf0,0x0b,0xee,0xc7,0x5e,0x27, -0x5a,0x27,0xee,0x66,0xda,0x1e,0x5b,0x5c,0xd2,0x73,0x35,0xca, -0xf8,0x08,0xac,0xb5,0xae,0xbd,0x98,0xb7,0xb4,0x0d,0xa6,0xc1, -0x5b,0x0e,0xec,0x9a,0x8c,0x48,0xff,0xba,0xc5,0x93,0xc2,0xc5, -0x5c,0xe9,0xe3,0x46,0x24,0x2b,0xa3,0xe3,0x05,0x64,0xf2,0x4f, -0xf4,0x7e,0xfa,0x28,0xda,0x75,0x0f,0x6f,0x81,0xf6,0xe0,0xfb, -0x61,0xa6,0xd3,0xca,0x53,0x04,0xa7,0x40,0x78,0x87,0x57,0x7d, -0x0b,0xaf,0x91,0x63,0xbd,0x65,0x36,0x81,0x18,0xc4,0xfa,0x06, -0x7b,0xf4,0xe7,0x93,0xbf,0x7f,0x1e,0x6f,0x91,0x7c,0x24,0x88, -0xc0,0x94,0xae,0x87,0x4e,0xac,0x7f,0x48,0x3c,0x33,0xed,0xd4, -0x81,0x02,0x90,0x3f,0x55,0x66,0xcf,0xe5,0x3a,0xd1,0xc9,0xb0, -0x55,0xc8,0x82,0x42,0x93,0x16,0xae,0x48,0x81,0x4b,0x05,0xac, -0x5e,0x8f,0x54,0x5b,0xa2,0x9d,0x88,0xad,0xcd,0x69,0x28,0x6a, -0x64,0x48,0x52,0x68,0xdb,0xac,0x60,0x7b,0xc7,0xc7,0xda,0xa2, -0xd7,0x5e,0xdb,0x11,0xb6,0xd1,0x64,0x7b,0x7c,0x7c,0x5f,0xae, -0x65,0xf3,0x6c,0x6b,0xe3,0x48,0x17,0xb8,0x97,0x0b,0x8a,0x56, -0x65,0x63,0x6d,0x92,0xa2,0xe0,0xdb,0x05,0x5e,0x1f,0x36,0x0c, -0xcf,0x36,0x6c,0xc1,0xdd,0x88,0x0e,0xee,0x99,0xb5,0xd2,0x1d, -0xf1,0x8c,0x5e,0x97,0xb1,0xfd,0x58,0x97,0x1d,0x17,0x30,0x7c, -0xe8,0x9b,0x6a,0x72,0xb3,0x7c,0x1a,0x10,0x2d,0x48,0x08,0xa3, -0x98,0x69,0x92,0x5d,0x07,0x02,0xeb,0x2f,0xb6,0x1d,0x00,0xea, -0x6c,0xc4,0x58,0x47,0x8b,0x6f,0x39,0x8c,0xf0,0x29,0x87,0xa0, -0x99,0xa2,0x05,0x8e,0x5c,0x8b,0xa3,0x45,0x6e,0x1c,0x47,0xce, -0x9a,0x46,0x63,0x53,0x04,0x08,0x35,0xe3,0xd5,0x0e,0x33,0x0e, -0x0d,0x38,0xc3,0xd7,0x2c,0xc8,0x42,0x01,0xfe,0x7a,0xc7,0xb6, -0xe1,0x7f,0x21,0xcb,0x55,0x34,0x22,0xbb,0x5c,0xf8,0x91,0x6b, -0x75,0x8c,0x78,0xab,0x0a,0x63,0xc6,0xe0,0xeb,0x34,0x31,0xfc, -0xee,0x44,0x83,0xf4,0xc3,0x10,0x60,0x19,0x3e,0xe2,0xa5,0x64, -0xcd,0xf5,0xda,0x99,0xc4,0x0e,0xdf,0xa0,0x20,0xff,0x02,0x29, -0x25,0x4e,0x08,0xde,0x57,0xe4,0x36,0xd5,0x1a,0xc3,0x31,0x5a, -0x7a,0x76,0x63,0x62,0x82,0x03,0x88,0x43,0x87,0x34,0x0c,0xa7, -0x89,0x6f,0x13,0x7f,0x80,0x30,0xc3,0x30,0xdd,0xfb,0xe9,0x17, -0x0f,0x9b,0x30,0xde,0xbd,0xcd,0x61,0x4b,0xce,0x82,0x96,0x7c, -0x0e,0xde,0x73,0x4c,0x98,0x0d,0x5b,0x88,0xb3,0xd1,0xfe,0x2a, -0x52,0x4f,0x70,0xaa,0xbb,0x9e,0xd5,0x88,0x88,0xf6,0x10,0x14, -0x15,0xd0,0x27,0x45,0x39,0x39,0x34,0x6e,0xe1,0x09,0x69,0x31, -0xbc,0x18,0x35,0xcf,0x39,0xbd,0xa4,0x9f,0xf4,0x5a,0x8a,0xe6, -0x2d,0x47,0xa2,0xe6,0xad,0x77,0xd3,0xb1,0x84,0x67,0xec,0xd0, -0x82,0xde,0xb7,0x18,0x2f,0x2f,0xa3,0xd5,0xe8,0x2d,0xc4,0x70, -0x11,0xca,0xf9,0xcf,0x12,0xbe,0xd8,0x81,0xe9,0x8c,0x34,0xd7, -0x54,0xc6,0x6f,0xad,0x1c,0x5a,0x32,0xba,0x62,0xb8,0xa4,0xaf, -0x09,0xfa,0x08,0x12,0xf4,0x37,0x33,0x4b,0x85,0x01,0x01,0xe2, -0x01,0x39,0x26,0x99,0xc3,0x84,0x47,0x1a,0x51,0x2d,0xb2,0x33, -0x37,0x8b,0xb6,0x26,0x1a,0xb5,0x05,0x89,0x52,0x9d,0x6a,0x2a, -0x6d,0x34,0x43,0xc9,0x9a,0x8f,0x81,0xa3,0xb1,0x4e,0x10,0x96, -0x0f,0xb6,0xe9,0x35,0x37,0x68,0x09,0xb8,0x3d,0xb2,0xd0,0xda, -0x7a,0xe8,0xa2,0x20,0x5a,0x37,0x37,0x0a,0xe7,0xb0,0x80,0xd9, -0xdc,0x7b,0x43,0x6d,0x4f,0x54,0x13,0x3c,0xc1,0xf6,0xe1,0x19, -0x53,0xcd,0xb3,0xad,0xcb,0x16,0x17,0x3b,0xd7,0xbe,0x18,0x31, -0xee,0x7d,0xec,0x85,0x84,0x32,0x8d,0x54,0xeb,0xf9,0xb5,0xb7, -0x6b,0xb8,0x5d,0x3f,0x24,0x0c,0xc2,0x0e,0xea,0x1c,0x87,0x9d, -0x0c,0xc2,0x36,0xab,0xc6,0x7b,0x95,0x4c,0xb3,0x8e,0xaa,0xfa, -0xc7,0x29,0x5e,0x42,0xa4,0x1e,0x66,0xd7,0x0b,0x4a,0x62,0x39, -0xe9,0x72,0x81,0xe1,0xe6,0x76,0xb6,0x02,0x87,0xc4,0x78,0x84, -0x9b,0xbb,0x3e,0x37,0x8c,0xbf,0xf1,0x82,0x18,0x8d,0x5e,0x14, -0x64,0x4a,0x2c,0xfc,0x97,0xaa,0x22,0x77,0x69,0x2c,0xcf,0x96, -0x1b,0xf0,0xab,0x57,0x50,0xb0,0xee,0x27,0xe8,0xe5,0x47,0x35, -0xf5,0xd1,0x20,0x39,0xe5,0xf2,0xe3,0x7f,0x54,0x45,0x2c,0xd8, -0x27,0xc6,0xa6,0x49,0x50,0xa8,0x9d,0x83,0xf1,0x7a,0xba,0x7e, -0x63,0x83,0xf6,0x7a,0x03,0xbb,0x5c,0x8c,0x7d,0x9d,0xe8,0xe5, -0xc2,0x55,0xc1,0x98,0xe2,0xeb,0x4d,0xaa,0x5c,0x16,0xdf,0x75, -0x3c,0xf8,0xea,0xe8,0xd0,0xbb,0x2a,0xba,0x5c,0xd0,0x2b,0xbf, -0xee,0x8a,0xc8,0x06,0x29,0xbe,0xc4,0x63,0xdc,0xfc,0xfd,0xdd, -0x4c,0x1e,0x86,0xf4,0x60,0x7c,0x56,0xfd,0xf6,0xdb,0xbc,0x74, -0x74,0x2a,0xde,0xd8,0x86,0x2f,0x57,0x90,0x44,0x71,0x9e,0x10, -0x16,0xb2,0x8d,0xe1,0xf9,0xbd,0x36,0x6a,0xa5,0x60,0x2a,0x43, -0x71,0x3c,0x6b,0xb3,0x9c,0xcf,0xcc,0xa2,0x82,0xff,0xca,0x5b, -0x31,0xf8,0xc1,0x9f,0xcf,0xa4,0x5d,0xe4,0x98,0xa9,0xb4,0x3d, -0x92,0xbf,0xf1,0x9d,0xae,0x24,0xe7,0xb0,0x77,0xf0,0x03,0x2d, -0x90,0x95,0x79,0x17,0x66,0xe8,0x07,0xb4,0xd1,0xe2,0xda,0x58, -0xa6,0x10,0x17,0xf2,0xd7,0xbb,0x1b,0x96,0x31,0x14,0xf2,0x97, -0x6c,0x2d,0x3b,0xde,0x9a,0xdb,0x27,0x7a,0x65,0x36,0x50,0x0b, -0xd5,0x43,0xd8,0x31,0x78,0x59,0xaa,0x83,0xd6,0xae,0x5c,0x4c, -0x41,0x7e,0x6d,0x30,0x7a,0x68,0x90,0xfd,0x33,0x3a,0x0a,0xce, -0x53,0x0c,0x76,0x86,0x06,0xed,0xa5,0xbc,0xc7,0x47,0x08,0x2d, -0xd6,0xbf,0xde,0xc0,0xc9,0x98,0x48,0x8e,0xfc,0xd8,0x2c,0x10, -0x68,0x06,0x76,0xcd,0xd6,0x8c,0xe3,0xc1,0x84,0x6a,0xfc,0xa5, -0xf0,0x3a,0xd1,0x57,0x10,0x2d,0x34,0xee,0x65,0xe6,0x6f,0xde, -0x3f,0x9d,0xa5,0xd7,0x78,0xeb,0x4b,0x47,0x27,0xfa,0xe3,0xdb, -0x8c,0x34,0xc7,0x47,0x25,0x2a,0xf4,0x82,0xbe,0x8e,0x72,0x79, -0x71,0xb0,0x13,0x81,0x24,0xed,0x54,0xcc,0x6a,0x9d,0x48,0xb9, -0x97,0x29,0x5a,0x33,0x9b,0x40,0x02,0xee,0xac,0x16,0x53,0x89, -0xaf,0x64,0x6f,0xc3,0x4f,0x07,0xca,0xa2,0x75,0xfc,0xc0,0xda, -0x46,0x61,0x51,0x11,0xd8,0x2c,0x51,0x3b,0x0b,0xde,0x26,0xf2, -0xbc,0x67,0xad,0x9d,0x47,0xef,0x49,0x66,0x3b,0x22,0x52,0xd6, -0x89,0xf9,0x02,0x2d,0x4b,0xbd,0x57,0x5c,0xbe,0x79,0xff,0xc8, -0xc5,0x5b,0x6f,0x6d,0xdb,0xe6,0xff,0x01,0xf8,0x6c,0x1b,0x38, -0xc9,0x1e,0x68,0x28,0xbf,0x2b,0x30,0xbf,0xd6,0x53,0x74,0xed, -0x39,0xbf,0x79,0xf5,0xea,0xfd,0x7f,0x3c,0x7b,0x18,0xb9,0x28, -0x13,0xf5,0xc0,0x92,0xa5,0xd8,0x58,0x80,0x62,0xe1,0x47,0xde, -0xea,0xd5,0xf9,0xf3,0x57,0x71,0xe1,0x13,0xa0,0x73,0xcb,0xa3, -0x47,0x23,0x5d,0xb2,0x4a,0x46,0x14,0xe5,0xd4,0x2d,0x94,0x1f, -0x1b,0xe1,0xc9,0x58,0x58,0x62,0xfe,0x51,0x6c,0x96,0x6f,0xca, -0x45,0xf5,0x9b,0x8f,0x74,0x02,0xd0,0x43,0x80,0x17,0x88,0x03, -0x1f,0xd2,0xcc,0x46,0x08,0x39,0x94,0x71,0xc1,0xa9,0xcb,0xfa, -0x62,0x8a,0x5e,0x3e,0x67,0x67,0x9f,0x75,0x61,0xbb,0xd5,0xba, -0x75,0x5c,0x17,0xc7,0x9c,0xd1,0xc0,0x93,0x31,0xfc,0xf3,0x79, -0xd2,0x83,0xdf,0xbd,0xe4,0xf3,0x09,0x05,0x97,0x67,0x88,0x02, -0xd3,0x30,0x4e,0x1a,0x57,0x93,0x02,0x0b,0x6e,0x96,0x8a,0x91, -0xd4,0x66,0x64,0x43,0x0f,0x77,0xeb,0xba,0x3a,0x9f,0x03,0x6f, -0x89,0xb0,0xee,0x48,0xc2,0xd7,0xd5,0xb4,0x21,0x5a,0x40,0x60, -0xa7,0x20,0x03,0x1b,0x1f,0x9e,0xcc,0x5c,0xbe,0x6f,0x21,0x70, -0x98,0x7e,0x1c,0x50,0xf8,0x8a,0xd6,0xfa,0xbd,0xb6,0xf0,0x90, -0xf3,0x9e,0xb5,0x51,0xb8,0xec,0x20,0x40,0x41,0x4f,0xd6,0x6b, -0x8c,0x79,0x61,0x37,0x10,0x40,0xb6,0x85,0xb7,0x11,0x62,0x94, -0x26,0x05,0x4e,0xce,0x6a,0x02,0x4c,0x4d,0xdb,0xc1,0xbb,0x49, -0xc2,0x87,0x2c,0x4c,0xf2,0xd9,0x20,0xc9,0xda,0x77,0x1b,0xb7, -0x39,0x09,0x16,0x1f,0xa1,0x02,0x27,0x9d,0x9a,0x73,0xd9,0xe8, -0x6f,0xd6,0xc5,0x79,0xb7,0x0b,0x07,0xc5,0xea,0xf2,0x12,0xad, -0x64,0x81,0xe3,0xfc,0xf3,0xbc,0x5f,0x2f,0x6f,0xd6,0x17,0xe5, -0x53,0x8e,0x79,0xfd,0xc3,0xc3,0xbf,0xbf,0xfc,0xf1,0xc9,0xbf, -0x3f,0x7c,0xfe,0xf4,0xaf,0x4f,0xb2,0xe3,0xf4,0x9f,0xd3,0x03, -0xd9,0x64,0xdd,0x0b,0x4d,0xe9,0x82,0xc4,0xdf,0xe2,0x59,0x67, -0x4d,0x76,0xd9,0x18,0x17,0x40,0x7d,0xc6,0x20,0x91,0x2b,0x32, -0xa6,0x17,0xc5,0xb9,0xd4,0x3b,0x1e,0xd8,0x58,0x8b,0xa3,0x41, -0x7e,0x3c,0x88,0x65,0x15,0x8a,0x3a,0xf7,0x73,0x5d,0xde,0xcc, -0x96,0xe1,0x13,0x67,0x2d,0xb1,0x7a,0xe8,0x8a,0x3c,0x50,0x21, -0x86,0x2b,0x47,0x9f,0xe0,0xe3,0xcb,0x09,0x09,0xf6,0x25,0x0c, -0x93,0xee,0x69,0xf8,0x8c,0xb0,0x8b,0x61,0xf8,0x86,0x6e,0x2b, -0xfe,0xb5,0x40,0xa4,0x21,0x14,0xf8,0x14,0x05,0xdf,0x89,0x48, -0x42,0xf6,0x11,0x70,0xe9,0x53,0xf0,0xd3,0xb9,0xc0,0xe6,0x49, -0x4b,0xa1,0x14,0xa5,0x3f,0x54,0xe4,0xb6,0xc5,0x34,0xa1,0xe8, -0xe9,0x2f,0xd3,0x5e,0x13,0x69,0x4c,0x6d,0x88,0x5d,0x3c,0x78, -0xb1,0x19,0x27,0xfa,0x28,0xeb,0x02,0x64,0x29,0x74,0xcc,0x80, -0xa2,0x1a,0x9d,0xdd,0x76,0x08,0x0c,0xc2,0x2f,0x1a,0xb0,0x09, -0xac,0x30,0x7e,0x1d,0xe4,0x03,0x6b,0x98,0xa0,0x97,0x2f,0xe5, -0x80,0xbc,0x67,0x4d,0xa5,0xe1,0x43,0x12,0x33,0x7e,0x92,0xd1, -0xe2,0xa4,0xc1,0x0a,0x0d,0x85,0xe5,0xc4,0x20,0xb5,0x9e,0x28, -0xa6,0xa3,0x30,0x54,0x5c,0x7c,0xfa,0xeb,0xe5,0xb7,0xd3,0xb5, -0x8a,0x64,0x20,0x48,0x7d,0x4b,0xd6,0x7e,0x35,0x4b,0xa9,0x33, -0x66,0xb2,0x24,0x34,0x8f,0x7d,0xcb,0x6e,0x76,0x24,0xa5,0x22, -0x99,0x8b,0x07,0xc5,0x3d,0x8e,0x4e,0x72,0xce,0x11,0xeb,0xe5, -0x13,0xd6,0x37,0x51,0x47,0xee,0xc8,0x29,0x4c,0x1c,0x70,0xe6, -0x3a,0x2d,0x48,0xa8,0xb1,0xdf,0x43,0xaa,0x32,0xcc,0xee,0xd2, -0x8e,0xc0,0xbf,0xdd,0xaa,0x0c,0x74,0xb1,0xbc,0xbe,0x9e,0x8a, -0xfc,0x83,0xa5,0x32,0xba,0xc1,0xbc,0x66,0xaf,0x01,0xed,0x0b, -0xff,0x15,0x20,0x78,0xf3,0x3d,0x99,0xa8,0x4d,0xf8,0x76,0xcb, -0xd5,0x04,0x0e,0xf1,0xa2,0x23,0xbc,0x00,0x4c,0x99,0xda,0x8c, -0xe3,0x6d,0x81,0xf1,0xfa,0x84,0x33,0xe7,0x14,0xa0,0xaf,0x83, -0x9e,0xad,0x35,0xfb,0x8c,0xe7,0xd1,0x3a,0xab,0x72,0x7b,0x62, -0xe6,0xcd,0x71,0xf3,0x74,0x22,0xc8,0x52,0xc8,0x02,0x15,0x71, -0xba,0x23,0x7c,0xf3,0xc4,0xec,0x19,0x84,0x25,0xb0,0x2c,0xd4, -0xe3,0x11,0xee,0xd8,0x56,0x33,0x13,0x9a,0x61,0x70,0x30,0x83, -0xb5,0x06,0x01,0xd4,0xdb,0xad,0x43,0xfc,0x58,0x1c,0xef,0xb4, -0x5e,0x9c,0xc3,0xc9,0x58,0xe7,0x77,0x8e,0x94,0x48,0x4d,0xe8, -0x98,0x73,0x3e,0x6e,0x80,0x28,0x0e,0x4b,0x62,0x76,0x0e,0x0b, -0xff,0xcd,0xae,0x41,0x6d,0x5c,0x5f,0x42,0x2c,0xd3,0xc7,0x88, -0x8f,0x28,0xfd,0x12,0xf7,0x22,0xb7,0x97,0xe6,0x8e,0x96,0xd3, -0x88,0x0c,0x33,0x4b,0xaa,0xfe,0x9a,0xb2,0x5b,0x36,0x8f,0xce, -0x2d,0x7d,0xfb,0x9c,0x16,0x8f,0x5a,0xd6,0xbb,0x55,0xa2,0x24, -0x89,0xf7,0xd8,0x94,0x5d,0x35,0x52,0x1a,0x5f,0x93,0x09,0x82, -0x56,0x6b,0x01,0xef,0x48,0x37,0x9b,0x3d,0xb2,0xa4,0x26,0x38, -0xc1,0x73,0x96,0x26,0x99,0x73,0xe0,0xa6,0x7a,0xdc,0x83,0x8d, -0xc6,0x66,0xf4,0xe1,0xd3,0xd0,0xe5,0xc5,0x8f,0xcb,0x85,0x0a, -0x7f,0x05,0x96,0x06,0x19,0xcb,0x09,0x10,0xc8,0x67,0xb1,0xe4, -0x6c,0xb9,0x20,0x26,0x5d,0xe0,0x8f,0x5e,0xcf,0x9e,0x80,0x5d, -0x7b,0x97,0xd5,0xba,0xde,0x44,0xc6,0x5f,0xba,0x03,0xbf,0xbb, -0x9e,0x33,0x46,0x38,0x30,0x09,0x59,0x56,0x41,0xab,0x93,0xa1, -0x7a,0xa3,0x94,0xbe,0xd3,0x38,0x08,0x01,0x11,0x5c,0x99,0xe5, -0xd6,0x34,0xbe,0x66,0xe3,0xb1,0x59,0x58,0xd0,0x31,0x0d,0x1e, -0xd5,0x07,0x7c,0x88,0x5f,0x82,0x38,0xb1,0xa6,0xa9,0x85,0x3d, -0x1e,0x9d,0x06,0x70,0x80,0xeb,0x9b,0x45,0x8d,0xcc,0xaa,0xa7, -0x03,0x45,0x72,0x3a,0x08,0xf6,0x47,0xc1,0xad,0x1e,0x24,0x2d, -0x20,0xdb,0xe7,0x7e,0x5c,0x44,0x84,0xdf,0x8b,0x1d,0x94,0xb6, -0xed,0xa0,0xb8,0xbe,0xaa,0x00,0x24,0xd6,0x8e,0x53,0x09,0x50, -0x64,0x2c,0x56,0x5c,0xb8,0x3a,0xd4,0x1d,0x0a,0x90,0xf4,0x71, -0x32,0xe1,0x49,0xa7,0x97,0xb7,0x71,0xa4,0xa4,0x33,0xe7,0x3c, -0x38,0x2b,0x16,0xec,0x1e,0x24,0x0f,0xe7,0x32,0xfb,0x46,0xf4, -0x59,0x09,0x87,0x33,0x74,0x60,0xca,0xfc,0xe3,0xee,0x8a,0x31, -0x77,0xa1,0x4a,0x8e,0xc1,0xa4,0xd8,0x8b,0x2b,0x18,0xb3,0xed, -0xc6,0x15,0xc7,0x3e,0x1c,0x16,0xdd,0x9a,0x28,0x19,0x2f,0x3f, -0x48,0x6b,0xd2,0x6a,0xed,0xed,0xf9,0x9c,0x60,0x9f,0x72,0xfc, -0x10,0xd5,0xf2,0x3a,0x8e,0xaa,0xf1,0x9e,0xec,0x98,0x0e,0x39, -0x38,0xed,0x9f,0x66,0x95,0xdf,0x00,0xd2,0xdc,0x56,0x38,0x99, -0x78,0xb2,0xca,0x12,0xb6,0xf4,0x05,0x88,0x3e,0x37,0x0b,0xb7, -0x03,0xaf,0xc4,0xa2,0xbe,0x1d,0x26,0xea,0x0c,0xe4,0xec,0xff, -0xd4,0x1a,0xe2,0x90,0x45,0x0c,0xc7,0x36,0xa3,0x3c,0x87,0x43, -0xad,0x88,0x9b,0x01,0x7c,0x78,0x8c,0x87,0xcf,0xd4,0xb6,0x06, -0x1e,0x38,0xd0,0xed,0x8c,0xfb,0xde,0x6e,0xd5,0x9f,0xa3,0x31, -0x36,0x8c,0xcb,0xe0,0x75,0xef,0x1f,0x9c,0xb9,0x37,0xa4,0xff, -0x15,0x27,0x57,0xce,0x7f,0xcf,0xaf,0xe3,0xc9,0xe3,0xa5,0x9d, -0x32,0xbb,0xa7,0xf8,0x91,0x89,0x98,0xb5,0xa1,0x19,0x89,0xe4, -0xf1,0x4f,0xc0,0x1a,0xff,0xb4,0xa7,0x0d,0x7b,0x76,0xb1,0x45, -0x61,0xb7,0x72,0x1f,0x76,0x2d,0x60,0x3c,0x4a,0x9b,0x5a,0xf8, -0xdd,0xdb,0x54,0xd8,0xdc,0x5d,0x2f,0xb6,0x15,0xfc,0x68,0xb6, -0x82,0xa9,0xcd,0x56,0x9a,0xe0,0x65,0x87,0x44,0x47,0x3d,0xab, -0x34,0x18,0xd8,0xa6,0xe4,0x80,0x21,0x38,0x07,0x80,0x9e,0x1f, -0xa6,0x2b,0x92,0x87,0xa0,0x61,0xfd,0xb9,0x2e,0xcb,0x77,0x55, -0xbd,0x81,0xf3,0x83,0xea,0xa4,0x74,0xe6,0xb1,0x52,0x4d,0xf2, -0xe0,0xd6,0x3e,0x3e,0x2b,0xe7,0xb6,0x3a,0xf5,0xd5,0xc4,0x89, -0x89,0x95,0x4f,0xa3,0xb1,0xa4,0x4c,0xf4,0xbc,0x8b,0xcf,0x23, -0xeb,0x6c,0x3c,0x5d,0x14,0x1d,0x3b,0x57,0x18,0x9a,0xa1,0xc4, -0x29,0xb7,0xb7,0x71,0xd4,0x6f,0x6e,0x09,0x9b,0x3e,0x05,0x78, -0x27,0xc9,0x05,0x43,0xd5,0x86,0x7f,0xba,0x51,0x37,0xdc,0xf5, -0xc8,0xe1,0x11,0x98,0x19,0xf6,0x30,0xb2,0x75,0x73,0x6f,0xd0, -0x20,0xa3,0xb8,0x69,0x1b,0x8d,0x27,0xb9,0x20,0x21,0xb7,0xb0, -0x0e,0x55,0x7f,0x83,0x53,0x79,0x1d,0x72,0x87,0xa7,0x0b,0xaf, -0xeb,0x10,0x24,0xaf,0x59,0x92,0xa1,0x4a,0x0c,0xbe,0xa0,0x63, -0xf2,0x2a,0xc9,0x5c,0xf8,0x15,0xa8,0x30,0x20,0x2c,0x6c,0xb0, -0x2a,0x30,0x39,0x60,0x23,0xf2,0xc8,0x24,0x24,0xcb,0xda,0x73, -0xed,0x8e,0xa5,0x5d,0x14,0xf5,0xad,0x80,0x0f,0xf0,0x06,0xe9, -0xb4,0xe6,0x48,0x04,0xb2,0x0a,0x40,0x1f,0x6d,0x16,0x61,0x51, -0x8e,0x5c,0x28,0xe1,0x88,0x88,0x73,0xb8,0x5e,0x5b,0xa0,0xf3, -0x40,0x22,0x18,0x69,0x10,0xb4,0xb4,0x1d,0x4c,0x95,0x80,0x32, -0x74,0x7d,0xd0,0x9d,0xaf,0x3f,0xaf,0x18,0x7a,0x0a,0xd1,0x42, -0xfb,0x36,0x47,0x98,0xf8,0xa4,0x7e,0x19,0x5e,0xd7,0xc1,0x48, -0x02,0xb9,0xb0,0x4a,0x82,0x56,0x11,0xc1,0x90,0x33,0xa5,0x61, -0xa5,0xd3,0xe3,0x81,0x9e,0x92,0xb0,0x32,0xe2,0x51,0xa8,0x43, -0xbe,0x19,0x7f,0x3b,0xde,0xb1,0x3c,0x58,0x5b,0xe6,0xb9,0x28, -0x74,0x79,0x8d,0x3c,0xb0,0x25,0xe6,0x93,0x47,0x8f,0xa6,0x31, -0xa8,0x2c,0x77,0x49,0x3e,0x5f,0x19,0xc5,0xc8,0x52,0x26,0xe0, -0x11,0x17,0x22,0x2b,0x6f,0x51,0xc6,0x78,0x2d,0xfa,0x87,0x3c, -0x49,0xfe,0x76,0xbd,0xbc,0x7e,0x4e,0xa2,0x64,0x43,0x2e,0x25, -0x21,0x42,0x75,0xe9,0xca,0x5c,0x5f,0xb7,0x88,0xaa,0xf3,0x72, -0x8a,0xe1,0x72,0x7f,0x41,0xe3,0xcf,0xea,0x6d,0xc9,0xe7,0xae, -0xb5,0x7c,0x8d,0x45,0x50,0x85,0x73,0x09,0x5f,0xfa,0x57,0xd7, -0x88,0x88,0x6a,0x63,0x8b,0x47,0xd5,0xb7,0xdb,0xb0,0x3e,0x9e, -0x0f,0x81,0xf0,0xa2,0x52,0xa3,0x41,0x7e,0xc2,0x40,0xa9,0xce, -0x2b,0x94,0x73,0xf7,0x07,0x56,0x46,0x89,0xc4,0x1b,0xd9,0xae, -0x01,0x10,0x5e,0x1f,0x71,0xd3,0x0f,0x17,0xef,0x3f,0xa5,0xf5, -0x80,0xca,0x02,0xec,0x51,0x31,0x20,0xb1,0x43,0x9d,0xe1,0x89, -0x75,0x7c,0x40,0xcc,0x10,0xe1,0x20,0xc2,0x03,0x10,0x2d,0xe4, -0x5a,0xa5,0x18,0xbe,0x9d,0x86,0x42,0xd4,0x35,0xd0,0x8b,0x74, -0x8e,0xc7,0xac,0x00,0x1a,0x55,0x69,0x66,0x8e,0x85,0xfb,0x88, -0x6c,0x76,0x9d,0x47,0xc8,0x68,0xd9,0xe6,0x77,0xd1,0x8b,0xb8, -0x32,0xa2,0x76,0x52,0xa8,0x84,0x14,0x32,0x37,0xee,0x10,0xbd, -0xfb,0xc4,0x33,0x45,0x54,0x36,0x19,0x06,0xda,0x76,0xed,0x88, -0xa5,0x90,0xb8,0x1b,0x59,0x0f,0xb4,0x6e,0x5c,0x96,0x6a,0x60, -0xb4,0x4d,0xbb,0x5b,0xeb,0x43,0xbb,0xbd,0x5e,0x45,0x63,0x7a, -0x0d,0x9b,0x7a,0xeb,0x30,0x5e,0x4b,0xfb,0xc3,0xd7,0x30,0x66, -0x77,0xfe,0xb2,0x60,0x57,0xa7,0x03,0x56,0x3d,0xb5,0x0e,0x85, -0x72,0x1b,0xa7,0x42,0x3d,0x39,0x9a,0xea,0x78,0x90,0x65,0xad, -0x3a,0x4b,0xbb,0x12,0x4f,0x2b,0xbb,0x47,0x35,0x56,0xb1,0xb4, -0x53,0x99,0xd7,0x20,0x4c,0xc8,0x28,0xf6,0x95,0x2d,0x82,0x2a, -0x5e,0x85,0xe6,0x99,0x35,0xdb,0x59,0x11,0xd7,0x67,0xeb,0x2e, -0x30,0xca,0x9e,0xd1,0xb6,0xb2,0x9d,0x7f,0xc7,0xc3,0xb2,0x14, -0xad,0xa3,0x39,0x47,0x55,0xac,0xfd,0xed,0x2b,0x9d,0xf8,0x78, -0x30,0xfb,0x05,0xc4,0x7c,0x90,0x62,0xcf,0xdf,0x3f,0x2b,0x37, -0x9e,0x44,0x65,0x05,0x76,0xcc,0x92,0xc3,0x4f,0x11,0xb5,0xec, -0x8a,0xd4,0x37,0xab,0x72,0x2d,0xc9,0x45,0x28,0x64,0x79,0xc4, -0x6d,0x79,0x2d,0x13,0x88,0xae,0x2a,0x7b,0x4b,0x6d,0x5e,0xdb, -0x29,0xb1,0x80,0x90,0x98,0x2d,0xe2,0xab,0xc6,0x46,0xad,0x8a, -0xe4,0x24,0x31,0x56,0x82,0x2e,0x58,0x12,0x82,0x72,0x76,0xc1, -0x8a,0xe0,0x1d,0xf4,0xa3,0x90,0x7c,0x33,0xbd,0x78,0x73,0xb3, -0x6a,0x2c,0xee,0x40,0x70,0xb3,0x43,0xee,0x76,0x65,0x39,0x2c, -0x66,0xfd,0xe7,0x0f,0xff,0x3d,0x45,0xb1,0x2d,0x68,0xf5,0xe0, -0x45,0x40,0x66,0xe4,0xb4,0xfb,0x9f,0x8b,0xea,0xd7,0x9b,0xd2, -0x9e,0x7d,0x25,0x7c,0x57,0x00,0x10,0xb0,0xe1,0x1f,0x30,0x6c, -0xd4,0x1a,0x97,0xce,0x35,0xc6,0x1d,0xea,0x0f,0x48,0x8f,0x64, -0x01,0x45,0x63,0xa7,0x08,0xe8,0xc2,0x31,0x2b,0x77,0xb7,0xaa, -0x18,0x77,0xc7,0xb8,0xa2,0x31,0xe3,0xd9,0x90,0x31,0x94,0xda, -0x53,0x30,0xdf,0x7c,0x20,0xd3,0x41,0x66,0xe0,0x21,0x80,0x19, -0xb3,0x3e,0x99,0xad,0x5c,0x22,0x22,0x05,0x79,0x3c,0xdb,0xb1, -0x91,0x16,0xce,0x76,0xd7,0xbc,0xf2,0x1b,0xb2,0xee,0xc8,0x1f, -0xa2,0xe0,0xa8,0x08,0x30,0xe7,0x8f,0xa5,0xd7,0x6b,0x8e,0x66, -0x47,0xe4,0x0b,0xb5,0x79,0x3c,0x1d,0x2b,0x79,0xca,0xfd,0xa3, -0x4f,0x40,0x18,0xfb,0x91,0x08,0xe6,0xa6,0xe5,0x28,0x45,0xd7, -0x33,0x7e,0xe9,0x5e,0x51,0x19,0x69,0x1c,0x1f,0xe2,0xf4,0xb3, -0x9a,0x48,0xf1,0x96,0x8f,0x20,0x44,0xb1,0xe1,0xce,0x99,0x8e, -0xb2,0x03,0xec,0x04,0xe2,0xa4,0xdf,0xcb,0xe9,0x09,0x5f,0x5d, -0x90,0x90,0xe6,0x9f,0x1a,0xd1,0x94,0x50,0x9b,0xa2,0xcf,0x34, -0xf8,0x06,0xa1,0x4d,0x02,0x05,0x0b,0xda,0x41,0x64,0xf5,0x56, -0x95,0x15,0xb6,0x5c,0x5a,0xb6,0x6b,0xbb,0x80,0x72,0xd9,0xc6, -0x9b,0x26,0x7b,0xfa,0xd0,0x5c,0xcb,0x0b,0x42,0x64,0xf7,0x5a, -0x38,0x0a,0x70,0x6b,0x51,0x04,0xde,0xd0,0xe4,0x3e,0xc3,0x78, -0x81,0x0a,0xa4,0xf2,0xbf,0x0f,0x93,0xc4,0xbe,0xb5,0xc0,0xcb, -0x29,0x73,0xfc,0x61,0xa7,0x3b,0x0a,0xcd,0xe3,0x28,0x38,0x10, -0xfa,0x8c,0x2b,0xcb,0xfd,0x2f,0x8f,0xd1,0xee,0x3b,0xc5,0xe9, -0x04,0xd6,0xca,0xd8,0x9a,0x9a,0x48,0x2d,0x61,0x5f,0xb7,0xb6, -0x5b,0xfb,0x1e,0x73,0x0d,0x8a,0xdc,0xa2,0xcd,0x0d,0xc3,0xbb, -0x72,0xff,0xf8,0x1e,0xdc,0xc8,0x1d,0xb0,0xf5,0xa8,0xec,0xa5, -0x02,0xfe,0xe1,0xab,0x05,0xe4,0x66,0xcc,0x53,0x5b,0xee,0x6a, -0x49,0xb5,0xcb,0xb3,0x8b,0xea,0x30,0x56,0x27,0x8b,0x4c,0x4d, -0xbe,0xcf,0xbc,0xd3,0xa9,0x8a,0xdc,0xfe,0xb0,0x9a,0x5a,0x13, -0x88,0xb4,0xa7,0x0f,0xba,0xdd,0xe4,0xe9,0x63,0x54,0x6d,0x72, -0xd5,0xc2,0x4a,0xb1,0x99,0x98,0x75,0xff,0xa5,0x28,0x1a,0x76, -0x18,0x40,0x61,0x91,0xb5,0x45,0xab,0xe0,0x30,0x50,0xf9,0xe7, -0xce,0xd9,0x30,0x14,0x8e,0x63,0x3f,0x7d,0xcc,0x7d,0xaa,0x88, -0x12,0xa8,0xf4,0x6f,0x16,0x72,0xc7,0x7c,0xa9,0xbf,0xec,0x2d, -0x66,0x06,0x05,0x4d,0x47,0x3f,0x22,0x1b,0x0b,0xab,0x4e,0xb6, -0xf7,0xa1,0x3c,0x72,0xdd,0xfb,0x59,0xf3,0xce,0xba,0x64,0x45, -0x9c,0x7f,0xf4,0x22,0xf0,0x0e,0xc4,0x1c,0x1f,0x9d,0xe4,0x01, -0x06,0x71,0xe5,0xe3,0x41,0xcc,0x47,0x1e,0x3e,0x15,0x1c,0x21, -0x84,0x42,0xdb,0xd1,0x60,0xc5,0x10,0x91,0xf4,0xa6,0x88,0x06, -0x87,0x10,0xb1,0x51,0x94,0xe3,0x59,0x41,0x01,0x7a,0x3e,0x0d, -0x41,0xe1,0x75,0x78,0x78,0x20,0x39,0x7c,0x29,0x2e,0xcf,0x03, -0x2b,0x96,0x56,0x22,0x55,0x0d,0x5c,0xd0,0xee,0xc2,0xbb,0xa4, -0x6b,0x13,0x99,0x9c,0xc5,0xcf,0xc7,0x19,0x5c,0x90,0xf8,0xe1, -0x9b,0x7b,0xf0,0x66,0xb3,0x73,0xea,0xf4,0x55,0xe5,0xaf,0x40, -0xbe,0x4b,0x09,0xa5,0x96,0x80,0x08,0xed,0xf2,0xda,0x63,0x14, -0xe0,0xae,0xdd,0x7d,0x3d,0x9b,0x5c,0xd8,0xa4,0xd9,0xdd,0x8e, -0x17,0xa4,0xa7,0x5e,0xc5,0x99,0xc1,0x97,0xb6,0xf0,0x65,0x19, -0x43,0x6f,0x28,0x1a,0x01,0x0b,0x1f,0xd6,0x7f,0x7c,0x83,0x58, -0x9a,0x6e,0x9a,0xbc,0xce,0x78,0xf6,0x53,0xc6,0x8f,0xa0,0x81, -0xbb,0x76,0x0c,0xb4,0x1a,0x8e,0xc8,0x2f,0xe6,0x6f,0xb5,0x8d, -0xac,0xa5,0x36,0x3b,0x06,0xe3,0xb6,0xfe,0x84,0x6f,0x50,0x8a, -0x84,0xb3,0x2c,0x92,0x9a,0x98,0x54,0xd2,0x3b,0x46,0x2f,0xdf, -0xc7,0x08,0x89,0x6f,0xd2,0x55,0xb0,0x19,0xa0,0x0d,0xd6,0x61, -0xc4,0x62,0xa5,0xb8,0xdb,0xa9,0xe0,0x03,0x6c,0x10,0x95,0xfc, -0xf0,0x87,0x5e,0xde,0x64,0xa5,0x79,0x60,0xf1,0x67,0xdc,0xed, -0x4f,0x94,0x21,0x98,0x58,0xb7,0xe5,0xf9,0xee,0xe3,0x6a,0xf4, -0xe5,0x82,0x0e,0xfa,0x46,0x01,0xc5,0xe0,0xeb,0xaf,0xbf,0x18, -0x98,0xe9,0x7a,0xcd,0x3a,0xbc,0x55,0x01,0x3f,0xfb,0xf0,0xd7, -0x20,0xf5,0xf0,0x07,0xfc,0x60,0xc2,0xa1,0x4f,0xfa,0x65,0xe4, -0x08,0x4a,0x29,0xf2,0x7b,0xbb,0xdd,0xf3,0x04,0xb9,0xbd,0x77, -0x42,0x73,0xce,0x06,0xa7,0x97,0x80,0x11,0xb8,0x27,0xf3,0xf5, -0x82,0x92,0x6f,0x35,0x54,0x7b,0x83,0x9d,0xa1,0x80,0xe9,0x1c, -0x97,0x36,0x19,0x9f,0x9d,0xbd,0x7b,0x70,0x72,0x76,0xb6,0x39, -0x3b,0x5b,0x9f,0x9d,0x2d,0xce,0xce,0x2e,0x27,0x89,0xc1,0x80, -0x34,0xd3,0x0b,0x20,0x83,0x27,0xf2,0xa6,0x4e,0x91,0xe0,0xab, -0x28,0x67,0x18,0x55,0x16,0x2a,0xdc,0x1e,0x4f,0xb6,0xe3,0x17, -0x50,0xf1,0xe4,0xe4,0x18,0xfe,0x9d,0x02,0x7b,0xed,0x25,0xa6, -0x82,0x3d,0x7e,0x83,0x4f,0xa1,0xad,0x8b,0x46,0x7d,0xbb,0xd0, -0x93,0x5b,0x7c,0xa6,0xea,0x4f,0x18,0x58,0x1c,0xb6,0x3f,0xba, -0x46,0x85,0xb6,0xc7,0xf7,0x5e,0x7c,0xb6,0xed,0xfc,0x73,0x32, -0x2a,0xb2,0xc4,0xb8,0x87,0xdc,0x8b,0xe4,0xec,0x6c,0x9c,0xf4, -0x1c,0xb8,0xbd,0xe4,0x5e,0x9a,0xf4,0x1a,0xad,0xf7,0x92,0x2c, -0x2e,0x35,0xca,0x93,0x9e,0xed,0xa1,0x91,0x97,0x8e,0x3f,0x3f, -0x4b,0x26,0x59,0xea,0x06,0xf5,0x02,0xff,0x4e,0xb2,0x7b,0xa3, -0xec,0xec,0xec,0x8b,0x2d,0x74,0xe2,0x46,0x03,0xad,0x6f,0xe1, -0xff,0x51,0x0f,0x50,0x3a,0x31,0x2b,0x32,0x3b,0x00,0x38,0xf3, -0x3d,0x60,0x51,0x07,0x69,0xfa,0xc1,0xfe,0xa2,0x8c,0x34,0x03, -0x2c,0x4f,0x26,0xdb,0xa4,0xe7,0xbd,0x6a,0xaf,0x28,0xfc,0xc2, -0xfc,0x1b,0xa0,0x3b,0xbb,0x97,0x6d,0xfb,0xf7,0xa0,0x36,0x82, -0x66,0xe8,0xa4,0x59,0xf8,0xc1,0x82,0x5f,0x84,0xf0,0xf6,0xa8, -0x8f,0x17,0xb6,0x63,0xed,0x10,0x9a,0x89,0x0a,0x7e,0x06,0x13, -0x84,0x6f,0x28,0xf1,0xed,0xfa,0xa1,0x36,0xef,0x99,0xe8,0x5b, -0x6a,0xe9,0xfd,0xf8,0xc1,0xba,0x69,0x93,0xf0,0x4e,0x7b,0xff, -0x9c,0xc4,0x68,0xc6,0x26,0x19,0xcb,0x7e,0x6b,0x82,0x77,0xc8, -0xf3,0x88,0x2e,0xea,0xcd,0x9f,0xc0,0xcf,0xf4,0x00,0x8e,0x3c, -0xb0,0xb8,0x7b,0xfa,0x38,0x0f,0x0a,0xff,0x69,0x1f,0x51,0x65, -0xe6,0xd1,0xf7,0x0f,0x9f,0x3d,0x0b,0x4b,0x63,0xb0,0xe5,0xbd, -0xe5,0x7f,0x7c,0xf8,0xc3,0x93,0xb8,0x38,0xc5,0x9a,0x28,0x88, -0x04,0x46,0x7b,0x6a,0x72,0x26,0xd2,0x54,0x66,0xe0,0xe8,0x17, -0xb6,0xd0,0x56,0x27,0x5a,0x50,0xf7,0x30,0xe8,0x33,0xf6,0xff, -0xf0,0xf9,0xf3,0x5f,0xf2,0x08,0x13,0x8e,0x86,0x32,0xf3,0xf3, -0xb3,0x27,0xff,0xf9,0xf8,0xa7,0xb8,0x84,0xc5,0xe7,0xa3,0xef, -0x9e,0x7e,0x1f,0x21,0x27,0xa7,0xe8,0xb5,0x5b,0xba,0x05,0xde, -0x02,0x5f,0xdd,0x6c,0x17,0x9b,0x2b,0xfc,0xef,0x18,0x3f,0xb2, -0xe3,0xf4,0xe2,0xaa,0x9a,0xcf,0xb6,0xcb,0xcb,0x63,0xda,0x97, -0x99,0xe2,0xe3,0xc9,0x46,0xbb,0xef,0xed,0x72,0x36,0x03,0x2a, -0x1c,0xf7,0x80,0x7b,0x64,0xe9,0xd9,0xd9,0xec,0x5e,0xb6,0x68, -0x2c,0x2b,0x5a,0x9c,0x54,0x22,0xce,0x80,0x0a,0xf8,0x24,0x53, -0x73,0x1d,0xd2,0x02,0x48,0x2a,0x18,0x7c,0xf0,0x0a,0xcb,0x21, -0xe2,0x1b,0x23,0xa9,0x6d,0x73,0x07,0x56,0xf9,0xeb,0xf6,0x15, -0x8c,0x8e,0xc7,0xe6,0x86,0xba,0x67,0x34,0x90,0x0a,0x7c,0x6f, -0x96,0x8d,0x68,0x10,0x6d,0x00,0xa5,0xa3,0x62,0xfc,0x02,0x46, -0xf1,0x99,0x80,0xb6,0xb3,0xdb,0x78,0x71,0x7f,0x4c,0x74,0xbf, -0x01,0xaa,0x5f,0x00,0xb3,0xbd,0x37,0x06,0x50,0xee,0x1b,0xb1, -0x5a,0x2f,0xee,0xbf,0x18,0xbf,0xb8,0x9b,0xf4,0xce,0xee,0xce, -0xea,0x7b,0x48,0x38,0x98,0x08,0xd2,0xc3,0xac,0x84,0x22,0xd6, -0xe4,0x97,0x1f,0xb7,0xfa,0x13,0x2c,0x21,0xe0,0xc4,0x80,0x94, -0xf4,0xec,0x16,0xfe,0x05,0xb2,0x94,0x04,0x7a,0xdd,0x8a,0xcc, -0xab,0x0e,0x3d,0xb7,0x25,0xd1,0x95,0xf8,0xd5,0xad,0xab,0x72, -0x8a,0xb7,0x5c,0xf7,0x5f,0x5c,0x9d,0xcd,0x28,0x81,0x85,0xb0, -0xe2,0xfe,0xe7,0xdb,0xb3,0xb3,0xfb,0xaf,0xcc,0xda,0x12,0xd1, -0x7f,0xdc,0x2c,0x91,0x35,0xdf,0x3f,0x2b,0xa2,0x81,0xa4,0xe3, -0x17,0x9f,0x27,0xc0,0xb5,0xee,0x65,0x51,0xc6,0xd9,0x04,0x1b, -0x50,0xb9,0x0e,0x6a,0x9e,0x01,0xa0,0xb3,0xe9,0xf1,0xe5,0xc3, -0xe3,0x6f,0x27,0x77,0x03,0xf3,0xe7,0x5d,0x58,0x61,0xb4,0xed, -0x67,0x50,0xc3,0x4a,0x82,0x4e,0xc9,0xf3,0xd2,0x70,0x8a,0x9c, -0x37,0xae,0xaa,0x57,0x57,0x45,0x72,0xf2,0x2e,0xe9,0x49,0xf2, -0xf1,0x9f,0xbf,0xfa,0xea,0x8b,0x3f,0xeb,0x49,0x06,0xb3,0xf1, -0xed,0x07,0xf8,0x33,0x92,0x02,0xf9,0xc9,0x29,0x7d,0x3e,0x23, -0x1b,0x6e,0x8a,0x39,0xf6,0x48,0x82,0xaf,0xa5,0x98,0xd1,0xa3, -0x06,0xb2,0xbc,0x2d,0xff,0xab,0xaf,0x1e,0xfc,0xe5,0xcf,0x5b, -0x2c,0x75,0x7a,0x3a,0x38,0x31,0x5f,0xfd,0xf9,0x8b,0x07,0x27, -0xdb,0xc1,0xc9,0x83,0x2f,0xba,0x98,0x96,0xed,0xc8,0xa8,0xde, -0x13,0x09,0x7d,0xe9,0x25,0x8e,0x31,0xd6,0xa7,0x45,0x83,0x32, -0x6b,0x0d,0x52,0x23,0x0a,0xb5,0x7a,0x18,0x71,0x76,0xf8,0x2c, -0x24,0xb8,0xf7,0x0f,0xa3,0xeb,0x61,0x35,0x53,0x1f,0x4f,0xd8, -0xba,0x80,0xb7,0x7d,0x52,0x04,0xb4,0xe8,0x3e,0xa2,0xc3,0xdd, -0x8e,0xe4,0xbf,0x42,0x8e,0xc6,0xfc,0xd1,0x62,0xcb,0x17,0x41, -0x4d,0x02,0x85,0x3e,0x9e,0x1e,0x99,0xe1,0x53,0xa5,0x78,0x98, -0xf6,0xf5,0xe1,0x30,0x79,0x94,0x7c,0xf7,0xfc,0x87,0xef,0x13, -0x4f,0x91,0xa4,0x68,0x51,0x93,0xc1,0xbc,0x33,0xd8,0xf9,0x72, -0xa7,0x42,0xea,0x27,0x59,0x78,0xb1,0x92,0x85,0x97,0x2c,0x8a, -0x47,0xf8,0x4f,0x0c,0x21,0xa6,0x05,0x5e,0x02,0x1e,0x74,0x81, -0x4e,0xeb,0x2f,0x1c,0xe5,0xca,0x3b,0x1f,0xce,0x9a,0x33,0x38, -0x4a,0x35,0x01,0xcb,0xaa,0x38,0x5c,0xb4,0x94,0x0c,0x45,0xe4, -0x82,0x90,0x8d,0x95,0x33,0x15,0x61,0xfb,0x1b,0x36,0x1c,0xff, -0x71,0xf9,0x08,0xb6,0x5c,0x32,0xd7,0x51,0x87,0x0e,0x1d,0x22, -0x7a,0x68,0xdc,0x59,0x43,0xd3,0xb7,0x78,0x0e,0x29,0x17,0xb3, -0x47,0x48,0x44,0x29,0x45,0x73,0x63,0x81,0x95,0xab,0xa7,0x49, -0x82,0xce,0xea,0x58,0xae,0xd5,0x3e,0x1d,0x77,0x55,0x91,0x1c, -0x77,0x0e,0x08,0x4f,0xec,0x6a,0xed,0x1d,0x9b,0xab,0x16,0x80, -0x51,0x9c,0xbb,0x22,0xf9,0x9a,0xb9,0xc9,0xe9,0xd7,0xf7,0xe5, -0x47,0x32,0x54,0xff,0x55,0x95,0x91,0xb1,0x06,0x72,0x51,0x02, -0x33,0xb2,0xc4,0x56,0x3d,0x87,0x7d,0x6a,0xdc,0x3e,0xa4,0xdd, -0xd1,0x90,0xe5,0xce,0xd1,0x82,0xad,0x3f,0x1d,0xa8,0xa1,0xc5, -0xfc,0x07,0x91,0xe5,0x43,0x0d,0xdf,0x47,0x74,0x3a,0x28,0x3e, -0xbf,0xaa,0x66,0x20,0x22,0x1c,0x95,0x9f,0xc3,0x18,0x20,0xf9, -0xb4,0x99,0xa7,0x39,0x89,0x69,0xe0,0xd2,0xb3,0xd7,0xdf,0x9f, -0x97,0x26,0xa5,0xc5,0xf4,0x28,0x0d,0xd1,0x71,0x61,0xc1,0x87, -0x42,0xe6,0x01,0x19,0xe6,0x7c,0x4c,0x3b,0x19,0x2e,0x8d,0x08, -0x15,0xfb,0xb1,0x40,0xc3,0xb7,0x76,0xfa,0xbd,0x13,0x13,0xe3, -0x63,0xca,0xf6,0xba,0x9f,0x03,0x0b,0x95,0x32,0x09,0x8e,0x7a, -0xca,0xd8,0x68,0xcd,0x13,0x8c,0x30,0xc1,0x43,0x63,0xd8,0xed, -0x37,0x14,0xfd,0x00,0xfb,0xb4,0x19,0xb4,0x8b,0xd2,0x58,0x39, -0x66,0xfa,0x0a,0x11,0x8b,0xb4,0x1a,0x8c,0x51,0x50,0xd8,0x96, -0x9c,0x4a,0xaf,0x3a,0x72,0x40,0xd1,0x83,0xde,0xc1,0x92,0xbd, -0x13,0x2d,0x6b,0xef,0x75,0x1c,0x9a,0x9e,0x02,0xbb,0xdd,0x10, -0xa6,0x3a,0x61,0x23,0xe4,0x51,0xa2,0x7d,0x59,0xf0,0xd9,0x4c, -0x5d,0x56,0x18,0xa0,0xd2,0x20,0xfc,0x80,0x78,0xd2,0x6f,0xd0, -0xa3,0xb0,0xfc,0xca,0xc0,0xa7,0x10,0xdf,0xf4,0x08,0x9f,0xb6, -0x2d,0x3e,0xff,0x13,0xa3,0x98,0xc9,0xca,0xe1,0x89,0xfd,0x5d, -0x65,0xe1,0xb8,0xe4,0x60,0xe5,0xe0,0x7b,0x45,0x41,0x34,0xb3, -0xe4,0x4f,0x89,0xd0,0xce,0x9e,0x1a,0x29,0xbf,0xa7,0x9b,0xed, -0xb2,0xd1,0xdd,0x2e,0xbf,0xc3,0x8f,0x8f,0x7e,0x27,0x57,0x9e, -0xe2,0x7d,0x90,0xf1,0x53,0x55,0x1f,0x5f,0x0f,0x4b,0xa3,0x55, -0xbf,0x69,0x9b,0x80,0x51,0xea,0x6b,0xcd,0xbc,0x1d,0xce,0xaa, -0x42,0x58,0x7b,0xc3,0xc8,0x68,0xf7,0x53,0x6a,0xe0,0x21,0x54, -0xe1,0xc9,0xdd,0xd3,0x3e,0x27,0xa7,0x6a,0x66,0x77,0x44,0xd8, -0xcf,0x7c,0x27,0xa7,0xd1,0xf8,0x7a,0x92,0x8f,0x27,0x00,0xb9, -0x77,0x35,0x19,0x41,0x69,0x83,0x21,0xac,0x9f,0xce,0x8a,0x6a, -0x76,0x50,0x89,0x35,0x6c,0x37,0x94,0xdf,0x8f,0x39,0xf4,0x86, -0xe0,0x98,0x06,0xd0,0xfa,0x6e,0x97,0xe5,0xff,0x9f,0x42,0xd6, -0x48,0x3c,0xab,0xe8,0x35,0x2e,0xee,0xb2,0xf9,0x6e,0x79,0xa3, -0xbb,0x96,0xa7,0xcd,0x69,0x9c,0x7d,0x7d,0x66,0xbc,0x9a,0x11, -0xde,0xdd,0x1b,0x9a,0xff,0xf3,0x67,0x80,0x5c,0x15,0x00,0x92, -0xc2,0x8b,0x1a,0xf8,0xe1,0x91,0xb4,0x96,0xe2,0xc1,0x58,0x1b, -0x38,0x8a,0x59,0x44,0x02,0x88,0x0e,0xcf,0x4e,0xa6,0x09,0x2e, -0xf8,0x8a,0xbd,0xfb,0xbf,0xb3,0x5b,0x84,0x2c,0x37,0xcf,0xe1, -0x83,0xa5,0x87,0x7c,0xd0,0x22,0xc0,0x47,0x07,0xfd,0xd5,0xa0, -0x0f,0xeb,0x78,0x9b,0xb7,0xf7,0x6c,0xa5,0xce,0x8d,0x5a,0x14, -0x9d,0x58,0x01,0xf4,0x83,0x6d,0x93,0xdf,0xfb,0x3d,0x8a,0x3c, -0x04,0x5f,0x9e,0xdd,0xad,0x9a,0xea,0xb0,0xd4,0xda,0x30,0x88, -0x05,0x7e,0xa8,0x86,0x48,0x81,0x10,0x0b,0xa9,0xbb,0x48,0x9e, -0xf5,0x10,0x8b,0x27,0xf0,0xa2,0xb1,0x3d,0x76,0xbb,0xbf,0x1b, -0xa5,0x1f,0x8d,0x4f,0x1a,0x30,0xf9,0xae,0xd8,0xc0,0x2f,0x0e, -0x2c,0x52,0x24,0x14,0xfb,0x7c,0xfe,0x2c,0x70,0x56,0x2c,0xf8, -0x24,0x10,0x9d,0x2c,0x14,0xc2,0xe9,0xbc,0x5a,0x69,0xa1,0x8f, -0x18,0xac,0xfc,0x03,0xde,0x81,0x16,0x06,0x3c,0xb6,0xfa,0x6a, -0x5c,0x9c,0x79,0xab,0xe1,0x2d,0xc6,0x09,0xbf,0xaf,0x9c,0x4c, -0x8c,0xef,0x3e,0x58,0x58,0xa7,0x6c,0xdc,0x66,0x63,0xef,0x31, -0x32,0x2d,0xfd,0x24,0xf9,0x72,0xb9,0xf2,0x6e,0xa1,0xca,0x59, -0xf1,0x39,0x6e,0x9e,0x9c,0xe8,0x09,0x9f,0xb4,0x95,0x36,0x5c, -0xd5,0x92,0xb1,0x56,0x9b,0x58,0x01,0x6a,0xbb,0x75,0xde,0x8d, -0x44,0x5b,0x6d,0x4a,0xc6,0x51,0x7e,0xc1,0x71,0xad,0xb6,0xfa, -0x76,0xc2,0xb6,0xaa,0xaf,0xa7,0x2b,0x6b,0x65,0xb9,0x05,0x81, -0x7b,0x46,0x8a,0x11,0xed,0x61,0xcb,0x1e,0xe2,0x49,0xb6,0x07, -0x14,0x6d,0xf1,0x00,0x20,0xae,0x08,0xb0,0x8a,0x8f,0x42,0x12, -0x1d,0xec,0x59,0xe8,0x56,0x99,0xf5,0xa8,0x02,0x14,0xdd,0xdf, -0x8f,0x91,0xea,0x05,0xe4,0x3b,0x74,0x74,0xbb,0x31,0x14,0xa8, -0x8c,0x9d,0x14,0x4d,0x8c,0x9c,0x25,0x67,0xc9,0xf6,0xf3,0xcf, -0x0f,0x8c,0xb0,0x5c,0x10,0xaa,0x0e,0x8d,0x50,0x8b,0x98,0x24, -0x57,0xcc,0xee,0x6d,0xef,0x9e,0xc9,0xdf,0xa1,0xfe,0x2f,0x6a, -0xc3,0xf4,0xef,0xe5,0x09,0xc5,0xe3,0xb2,0x64,0x27,0xd7,0x0b, -0xd6,0x0f,0xd2,0x92,0xa0,0x64,0x14,0x2a,0xd5,0x45,0x05,0x69, -0x7d,0x70,0xc6,0xf2,0xb7,0x1f,0xf6,0xe5,0xdd,0x96,0xe7,0x6f, -0xaa,0xcd,0xde,0xec,0xe5,0xde,0x9c,0xeb,0x3a,0xca,0x42,0xea, -0x6f,0x9d,0x57,0x1d,0x08,0xe0,0x04,0x16,0xe7,0x82,0x08,0x8a, -0xea,0x8a,0x11,0x80,0xbc,0x96,0x83,0xf2,0x35,0x7b,0xf0,0x9b, -0x66,0xfa,0xb8,0xee,0xe0,0xcc,0x7a,0x38,0x93,0xde,0x05,0x6f, -0x9d,0xc2,0x2a,0xac,0x91,0xbe,0xdc,0x42,0xf6,0x34,0x65,0x0e, -0xd7,0xec,0x43,0xba,0xc5,0x93,0x64,0xc8,0x03,0x1a,0xa5,0xb5, -0x17,0xaf,0x86,0x5e,0xf3,0x04,0xbc,0x80,0x50,0xa2,0x39,0x99, -0xc3,0x12,0x5e,0xbd,0x80,0xfc,0xa5,0x07,0x75,0x75,0xea,0x73, -0x7b,0xa0,0x75,0xe1,0x9c,0x62,0xb4,0x26,0xf2,0x26,0x9e,0x3a, -0x23,0xb8,0x69,0x7c,0xde,0xce,0xa1,0xfc,0xcd,0x0a,0x7d,0x3d, -0xcf,0x3d,0xc1,0xce,0x7a,0x59,0xa2,0xef,0xe5,0xcd,0x6a,0xbb, -0xed,0xa4,0x1d,0xfa,0x8b,0x3e,0xfa,0xf0,0xc3,0x73,0xc3,0xe8, -0xa4,0x53,0x7e,0x3f,0x53,0x60,0x1d,0x85,0x9f,0x29,0x94,0xce, -0xf2,0xe9,0x3e,0xb8,0xbb,0xdd,0xc1,0x9f,0xbb,0x7b,0x73,0xa9, -0x72,0xe6,0xbb,0xb5,0xd0,0xe8,0xd0,0x0a,0x86,0x4d,0x2f,0xce, -0x8b,0x00,0x6a,0xbc,0xd0,0x39,0xc7,0x01,0x5b,0x7f,0x88,0xa1, -0x8d,0xcf,0xe0,0x2e,0xd1,0x8a,0x4f,0xc7,0xa5,0x94,0x0c,0xd0, -0x32,0x4a,0xfd,0x8b,0x40,0x74,0xf2,0x3e,0x01,0xf9,0x53,0x4a, -0x02,0x5c,0x7b,0x47,0x7c,0x00,0x19,0x07,0x30,0x91,0x65,0xa3, -0x41,0x57,0x72,0xb7,0xdb,0x69,0x10,0xf3,0x71,0x40,0x01,0x52, -0xbc,0x24,0x6f,0xc2,0x59,0x3d,0xc3,0x97,0xbd,0x34,0x21,0xbe, -0x52,0xc7,0x4c,0xb3,0xd1,0xf1,0x20,0x3f,0x3f,0x5c,0xe8,0x3c, -0x43,0x4b,0xd4,0xfc,0x4b,0xed,0x1e,0xab,0x0c,0xf6,0xcf,0x29, -0x65,0xc7,0x73,0x26,0x4e,0xc5,0x24,0x0b,0x4d,0x81,0xe0,0x7c, -0x68,0x99,0x02,0xfd,0x04,0xb4,0xe5,0x9f,0x4e,0xcc,0x39,0xfc, -0x39,0x27,0xc7,0xc9,0xa9,0xe7,0x04,0x7c,0xd4,0xc0,0x3b,0x59, -0x46,0x4c,0x89,0x4e,0x91,0x62,0xbc,0x59,0x82,0x51,0x79,0x03, -0x84,0x61,0x40,0x29,0x4a,0x20,0xb3,0x2e,0xaa,0x88,0xd0,0x10, -0x51,0x6b,0xbd,0x86,0x47,0x34,0x19,0x7b,0xa1,0x23,0xf2,0xb4, -0xe5,0x3d,0xa1,0x61,0x36,0x5d,0xf5,0x6f,0x16,0x6c,0x5c,0x80, -0xee,0xcc,0xb6,0xf4,0x79,0x6b,0xe9,0xf3,0x96,0xd2,0x43,0xb2, -0xcf,0x46,0x28,0xf0,0x2f,0x90,0xb1,0x73,0x2d,0xab,0x46,0x21, -0x38,0x58,0xc0,0x50,0x31,0x58,0x55,0x52,0xcb,0x9f,0x2c,0x1a, -0x5c,0x5b,0x3a,0x8c,0x76,0x67,0x42,0xcc,0x0d,0xcc,0x18,0x90, -0x37,0xe1,0x67,0x41,0xed,0xf2,0x70,0xea,0x92,0x59,0xb9,0x01, -0xde,0x6a,0x2b,0xd4,0x45,0x70,0xf3,0x6d,0xdf,0x57,0xc9,0xf5, -0xd7,0xce,0x88,0xe2,0x51,0x37,0x12,0x77,0x8a,0xc0,0xdb,0xf4, -0x52,0x1d,0xb5,0x54,0x48,0x13,0x43,0x1b,0xca,0x24,0xcb,0x55, -0x1b,0xc2,0x5d,0x62,0x7d,0x85,0xcd,0xd9,0x0d,0x2b,0x34,0x20, -0xc6,0xea,0xec,0xa0,0xb5,0x57,0xd3,0xba,0x37,0xc0,0x09,0xe5, -0x52,0x0b,0x45,0xc9,0x46,0x1b,0x72,0x34,0x8a,0x14,0xf8,0x26, -0x29,0x3e,0xff,0x6c,0x80,0x82,0x80,0x01,0x4e,0xb8,0x67,0x23, -0x8d,0xe4,0x47,0xdd,0xcf,0x85,0xe3,0xab,0xe8,0xa0,0x1b,0x00, -0x47,0xb0,0x44,0xd8,0xfd,0x8d,0xdf,0xa5,0x66,0x14,0x7f,0x40, -0x22,0x1e,0x87,0x1b,0x9b,0x1b,0x36,0xd2,0xef,0x1a,0x1f,0xc5, -0xd9,0xbb,0x25,0x32,0x06,0xfa,0x6e,0xf8,0x83,0x81,0x86,0x68, -0xb4,0x41,0x6f,0xa2,0x98,0x2a,0xd8,0xa1,0xf7,0x06,0x42,0xcb, -0x64,0x59,0xb3,0x02,0x7e,0x3a,0x0a,0x5b,0xb3,0x6f,0x47,0x9d, -0x9e,0xd8,0x79,0xb3,0x1b,0x9b,0x13,0xd8,0x7d,0x5b,0x71,0x21, -0x83,0xf6,0x08,0x27,0xce,0x3a,0xe5,0xe3,0xa2,0xd3,0x34,0x63, -0x9b,0x58,0x30,0x70,0x2e,0x23,0x9a,0xa1,0x73,0x07,0x21,0x17, -0xe4,0x50,0xf5,0xb1,0xff,0xbd,0xe4,0x13,0xcd,0x3a,0x1d,0x6a, -0xc8,0x2b,0x3f,0x8a,0xc4,0x61,0x52,0xe8,0xac,0x88,0x54,0x62, -0x1c,0x5a,0x3f,0x1b,0x41,0x16,0x37,0x97,0x47,0xcd,0x35,0x35, -0xce,0xa3,0xa6,0x22,0x84,0xcf,0x51,0x29,0x75,0xd0,0x7e,0xea, -0xa6,0x12,0x99,0x50,0x43,0xb3,0xae,0x04,0x10,0xb0,0x2f,0x01, -0x74,0x4e,0x46,0xf8,0x33,0x87,0x06,0xbb,0xdd,0xb7,0x18,0x53, -0x1a,0x23,0x4b,0x5e,0x56,0x70,0x9a,0xc3,0x2f,0xf6,0x57,0xa6, -0x30,0x47,0xc6,0x77,0x1d,0x76,0x68,0xbe,0xae,0x5f,0x61,0xc4, -0xd9,0xf5,0xf2,0x16,0xfd,0xdc,0x8e,0x28,0xc8,0x45,0x9a,0x3c, -0x7b,0x0f,0x33,0xf5,0xee,0x88,0xca,0x9a,0xa3,0x9b,0xc5,0xba, -0xbc,0x58,0xbe,0x42,0x2b,0xb7,0xd9,0x11,0x12,0x56,0x59,0xd7, -0x18,0xd7,0xfc,0x28,0xe9,0x61,0x75,0xdb,0xb4,0x33,0x46,0x74, -0xed,0x5b,0xbb,0x3e,0x7b,0xd8,0x9e,0x39,0x3e,0x45,0x67,0xee, -0x01,0x3d,0x5a,0x58,0x5d,0x46,0xdb,0xf4,0x3e,0xe6,0xa6,0x27, -0xf4,0x06,0x37,0xf4,0xab,0xb7,0x9e,0xc7,0x27,0x64,0x4b,0x52, -0xb2,0xdb,0x85,0x4d,0x3d,0x1e,0xa0,0x4f,0xdc,0xeb,0xc2,0x81, -0xe5,0x1c,0x00,0xa9,0x11,0x7a,0x0f,0xde,0x76,0xca,0x56,0x57, -0xae,0x30,0x3e,0x0b,0x3f,0xc8,0x1a,0x27,0x77,0xb1,0x48,0xd2, -0x1b,0x1e,0xfd,0xdc,0xa3,0xae,0xa1,0x08,0xe4,0x49,0x42,0x9b, -0xae,0x0d,0xac,0x14,0x68,0x0d,0x10,0x3f,0x41,0xb4,0x23,0x3f, -0xcc,0xcc,0x76,0xfb,0x97,0xe0,0x6b,0x30,0x88,0x63,0x23,0x35, -0x42,0x50,0x52,0xe3,0xb8,0x06,0xe9,0x62,0x79,0x61,0x4d,0xf4, -0xe2,0x74,0x42,0x41,0xa9,0x16,0xce,0x9e,0x36,0x96,0x70,0xeb, -0x3c,0x80,0xfd,0x78,0x20,0xd8,0x56,0xaf,0x90,0x21,0xf3,0x48, -0xad,0x6f,0xed,0x17,0x01,0xa0,0x5f,0x16,0x8d,0x40,0x51,0x47, -0x76,0xd8,0x14,0xf1,0xdf,0xf3,0x36,0x26,0xbd,0x16,0x11,0xbf, -0xcc,0x64,0xd0,0x0d,0x5d,0x97,0x0d,0x3d,0xc6,0x48,0x5a,0x0a, -0x77,0xc3,0xc6,0xac,0xbf,0x2e,0xee,0xbc,0x58,0x66,0xf9,0x57, -0x27,0x46,0x62,0x6e,0xd0,0x09,0x22,0xf7,0x0d,0x5b,0xf9,0x2c, -0x92,0x5b,0x03,0x0b,0xb2,0xf9,0xc4,0x67,0xb7,0xd4,0x60,0x30, -0xbf,0x4b,0x4e,0x93,0x1c,0x0e,0xaf,0xeb,0xdc,0xf7,0x40,0x37, -0x84,0x23,0x7a,0x7f,0x27,0x39,0x6a,0xc9,0x87,0xe4,0x9e,0x4d, -0x5e,0x97,0x6f,0xab,0xe5,0x4d,0x2d,0x88,0x0b,0xea,0xfe,0x73, -0x5f,0xa1,0xdd,0xce,0x79,0x23,0xe6,0x77,0x64,0x1a,0xe1,0x56, -0x33,0x19,0xe0,0x05,0x9e,0xc3,0xd6,0x53,0x19,0x4d,0x3d,0x0f, -0x5a,0x26,0x72,0xa9,0x2f,0x26,0x85,0x84,0x94,0xf8,0x72,0xb2, -0xdd,0xf2,0xaf,0xaf,0xe0,0x57,0x92,0x64,0x87,0xab,0x27,0xff, -0x2c,0x12,0x35,0x77,0x1d,0x3f,0x98,0xa8,0x57,0x20,0x36,0x88, -0x9e,0xea,0xfa,0xd1,0xa3,0x97,0xb7,0xd8,0x28,0x56,0x1d,0x4c, -0xbe,0x04,0x1e,0xc2,0x46,0x1a,0x1f,0x3d,0x92,0x80,0x65,0x9b, -0x64,0xb1,0xb9,0x72,0x9d,0x0f,0xac,0x29,0xad,0xf9,0x22,0x1b, -0x59,0x30,0xb6,0xdb,0x20,0x7a,0x82,0x9a,0xdd,0xea,0xc0,0xbf, -0x9c,0x14,0x3d,0x3b,0xf2,0x91,0x0e,0x5c,0x93,0xfe,0x0c,0xd5, -0x07,0x59,0xfe,0xe0,0x5e,0x4a,0x0f,0xd8,0xba,0xce,0xb0,0xdd, -0x64,0x39,0x9b,0xf9,0x29,0x1a,0x64,0x03,0xea,0xdb,0x36,0xff, -0xdb,0x44,0x50,0xf0,0x6f,0xad,0x15,0x72,0xfd,0x69,0x4d,0xbb, -0x5b,0xa1,0xdc,0xa9,0xad,0x4b,0x8c,0x28,0x62,0xaf,0xef,0x2e, -0x80,0x33,0x9b,0x9b,0xc5,0xaf,0x28,0x0c,0xcd,0xc4,0x88,0x1f, -0x80,0x10,0x8b,0x72,0x98,0x95,0xa1,0x8f,0x50,0xda,0xe2,0x08, -0xef,0x2c,0xcf,0xd8,0xbe,0x46,0xb8,0x63,0xe4,0x31,0x32,0x1e, -0x28,0xfa,0xbf,0x44,0xa5,0x36,0xf7,0x01,0xd2,0x12,0x9f,0xbe, -0xb9,0x05,0x4d,0xa6,0x37,0x17,0x08,0x1a,0x67,0x1f,0xad,0x79, -0x14,0x0e,0xdb,0xe5,0x6b,0xb2,0x7b,0xe0,0x3e,0x4b,0xec,0x10, -0x44,0x56,0x39,0xe6,0xb2,0xd9,0x71,0x94,0x6e,0x5d,0x4f,0xdb, -0xcc,0xa8,0x8d,0x54,0x32,0x16,0x7a,0x5b,0x3b,0x2a,0x11,0x93, -0xe3,0x17,0xe8,0x5b,0xa9,0x8f,0xc3,0xdd,0xa1,0xe5,0x52,0x70, -0x2b,0x4f,0x4a,0x44,0xa1,0x4c,0xd6,0x00,0x6b,0xea,0xa8,0x11, -0x09,0x1b,0xfd,0xe7,0x6d,0x25,0x5b,0xee,0xe0,0x42,0x8a,0x48, -0xfb,0xc0,0x1d,0x8b,0x36,0x27,0x1a,0xfc,0xf6,0xc8,0x44,0x1e, -0x78,0x18,0xf2,0x92,0x4d,0xbf,0x9a,0x9a,0x59,0xa6,0xa0,0x15, -0x48,0x2f,0xe5,0x7a,0x51,0x38,0x53,0xd3,0xb1,0x2d,0xc1,0xc1, -0x71,0x6c,0x90,0x12,0x2a,0x08,0x52,0x94,0x56,0x89,0xde,0xf4, -0x0f,0x35,0x6d,0x59,0xd2,0xf3,0xda,0x89,0x4c,0x8f,0xe8,0x49, -0x7f,0x34,0x7d,0xb6,0x9d,0x7a,0x0a,0xe3,0xf6,0xe1,0x4b,0xa7, -0x5e,0x78,0x79,0x5b,0xc3,0x5e,0xe1,0x34,0x24,0xa8,0x0f,0xdf, -0x7b,0xa4,0x09,0x35,0x83,0x2f,0x5c,0x60,0x80,0x4d,0x40,0x57, -0xc8,0x5e,0xe9,0x05,0x25,0x35,0xc0,0xe4,0x70,0x24,0x87,0x82, -0x46,0xc9,0x1b,0xc2,0x9e,0x78,0xeb,0x49,0xb5,0xe1,0x73,0x80, -0x5c,0x74,0x84,0xea,0x2c,0x8c,0xcb,0x2c,0x5d,0xe4,0xfa,0x63, -0x24,0x72,0x54,0x0f,0xa5,0x84,0x24,0x28,0x33,0x92,0x5e,0xc4, -0x93,0x34,0x8f,0x9a,0x18,0xd9,0x47,0xc5,0x24,0xfb,0x45,0x98, -0x4d,0xa9,0xdd,0xee,0x89,0x15,0x89,0xec,0x3a,0xe4,0xe1,0xe5, -0xc9,0xbd,0xd6,0x0a,0xad,0x85,0x4f,0xe1,0x50,0x9b,0x7c,0x76, -0xa8,0x3c,0x2f,0xb1,0x63,0x4a,0xd4,0x45,0xec,0x40,0xff,0x67, -0x58,0x15,0x1f,0x69,0xec,0xc9,0xc0,0x71,0xd7,0x68,0xeb,0x6d, -0x7b,0x08,0x19,0xf8,0x5c,0x8c,0xd7,0xed,0x89,0xf1,0xfb,0xed, -0x0d,0x6c,0xcf,0xbd,0xe4,0x38,0xc9,0xf1,0x1d,0x04,0x0c,0xcc, -0x11,0xef,0x44,0xe4,0xf0,0x71,0x7b,0x35,0xdd,0xd8,0xb0,0x57, -0xbc,0x4d,0x1b,0xb2,0xa5,0xe3,0x07,0x97,0xd1,0x93,0xb6,0x2c, -0x68,0x17,0x12,0xc3,0x0e,0x8f,0x99,0xe0,0x8b,0x2a,0xb7,0xd3, -0xf5,0xac,0x48,0xb0,0x46,0x54,0xe0,0xf8,0xcb,0xcc,0x2c,0x2f, -0x49,0xe0,0x4b,0xc4,0xda,0x10,0x07,0x84,0xfd,0xf9,0x91,0xba, -0xa9,0x47,0x9e,0x26,0x6c,0x64,0xd4,0xba,0x2c,0x3a,0x71,0x70, -0xcb,0x0f,0xc6,0x99,0x69,0x84,0x98,0x21,0x69,0x14,0x83,0xbe, -0xd1,0x2f,0x0a,0xdb,0x65,0xea,0xcd,0x74,0xbd,0x41,0x73,0xf4, -0x82,0xc7,0x09,0x03,0x90,0x11,0x8d,0x12,0x4f,0xec,0x4b,0x5a, -0xe4,0x19,0x06,0xa5,0x88,0xc0,0xa2,0x05,0x50,0xf0,0xa8,0x0f, -0xf2,0x2e,0x73,0x53,0xb3,0xbd,0x7a,0xd1,0x01,0x90,0xbb,0xdd, -0x0e,0xd7,0x41,0x61,0x98,0x5b,0x23,0xe1,0x96,0xa1,0x12,0x91, -0x1f,0xc0,0x1c,0xf2,0x4f,0x2b,0x2f,0xb2,0xe4,0x88,0xff,0xb8, -0x38,0x35,0xdc,0x10,0x5b,0x54,0xed,0xe7,0x9b,0x78,0xb2,0x52, -0x59,0x3a,0x3e,0x7d,0x77,0x06,0x43,0x42,0x0c,0xfa,0x54,0xe1, -0x0b,0x6e,0xf3,0xf7,0x49,0xa1,0x46,0x3e,0x1d,0xca,0xe9,0x76, -0x03,0xf4,0xec,0xec,0x8e,0x80,0x30,0x53,0xd5,0xb1,0xe2,0x91, -0x47,0xe3,0x89,0xd7,0xb9,0xa4,0x58,0xd3,0x9a,0x89,0x52,0x51, -0xb7,0xab,0x48,0xe1,0x61,0x7a,0xa1,0x74,0xb8,0x8e,0x1f,0x4c, -0x27,0x4a,0xa1,0x70,0x3a,0x8d,0xf0,0x36,0xe4,0xd2,0x42,0x01, -0x61,0xed,0x9c,0x17,0x61,0x68,0x1d,0xf4,0x43,0xd0,0x70,0x3b, -0x83,0x89,0xc4,0x1a,0xdc,0x5b,0xe2,0x01,0x37,0x54,0xd8,0xd6, -0xba,0x5d,0x19,0x8d,0xb3,0x0f,0x1c,0xdb,0xcc,0x09,0x4f,0x4f, -0xaf,0xe7,0x15,0x77,0xb7,0xcd,0x34,0x63,0x30,0x10,0xea,0xd2, -0x81,0x77,0x82,0x51,0xce,0x10,0x85,0xe8,0x86,0x90,0x66,0x36, -0xf4,0x50,0x30,0x51,0xdd,0x6e,0xaf,0x87,0xf5,0xb8,0x25,0xf5, -0x1c,0xb8,0x8b,0x47,0x4e,0x91,0x7d,0xc8,0x97,0xc4,0x91,0x3c, -0x56,0x9b,0xa8,0xb7,0x8b,0x9e,0x5d,0x14,0xef,0xf8,0xfe,0x38, -0xe1,0x30,0x8c,0x55,0xd4,0x16,0xbb,0x28,0xb3,0xfe,0x42,0x4d, -0x74,0x65,0x1e,0x1a,0x07,0xec,0x44,0xce,0x07,0x9d,0xf4,0x8f, -0x23,0x04,0x05,0xa3,0x8f,0xa0,0xf1,0x8e,0xd2,0x78,0x27,0xa6, -0xf1,0xed,0xb6,0xc3,0xc8,0x83,0xae,0xbc,0x71,0x13,0x68,0xfe, -0x88,0x83,0x6f,0x6f,0xc4,0x0e,0xa9,0x84,0xca,0x8c,0x70,0x2b, -0x0a,0x2d,0x38,0x58,0xdb,0xdd,0x0f,0x73,0x8f,0x89,0xa5,0x31, -0x55,0x09,0xa7,0xdb,0x6e,0x4f,0x08,0x51,0x97,0x97,0xff,0x55, -0x38,0x1f,0xfe,0xbe,0x4f,0xbf,0x4f,0x8b,0x13,0x7c,0xe6,0x32, -0x96,0x83,0x59,0x0c,0x75,0xa1,0x09,0xf9,0x6e,0x65,0xfd,0x0a, -0x03,0x94,0x4b,0xf4,0x3d,0xbe,0x26,0x1a,0xf3,0xdf,0x89,0xc4, -0x76,0x70,0xfe,0x45,0x63,0x95,0x64,0x7d,0x14,0xc5,0xa7,0x86, -0x04,0xc6,0xc4,0x9a,0x89,0x72,0x76,0xc4,0x15,0x50,0x0f,0xc2, -0xbf,0x9c,0x59,0x86,0x8b,0x6a,0x3d,0xba,0xf4,0x02,0x34,0xe6, -0x97,0x0b,0x17,0xc6,0x09,0x93,0xeb,0x42,0x7a,0x95,0x4b,0x2c, -0x03,0xfb,0xbb,0x96,0x9e,0x98,0x08,0xc0,0xfe,0xd5,0xb4,0xfe, -0xe9,0x76,0xf1,0xb3,0xbc,0x3e,0x94,0xb6,0xc1,0x9b,0x8d,0x3e, -0x25,0xd2,0x63,0x35,0x7b,0x67,0xc3,0x01,0x5e,0x4a,0xa1,0x46, -0x70,0x47,0x2b,0x6e,0x4b,0x0c,0xa9,0xd9,0xbb,0xa2,0x19,0xc3, -0xc4,0x79,0xd5,0x66,0xe4,0x3b,0x39,0x86,0x62,0x7e,0x84,0x47, -0xfa,0xf4,0x0a,0xed,0xb2,0x3d,0x76,0x57,0x97,0xb2,0x57,0x9d, -0x20,0x20,0x35,0x95,0x5b,0xe0,0x91,0x97,0x27,0x0f,0x5f,0xb8, -0xdd,0xe4,0xfb,0x86,0xa8,0x11,0x95,0x68,0x68,0x78,0x89,0x4c, -0x77,0xfb,0xd6,0xa4,0x58,0x7d,0xe3,0x8b,0xd8,0x71,0xad,0x2d, -0xb6,0x41,0x36,0x0c,0xe3,0x6f,0xb9,0x09,0xfd,0x30,0x82,0x0f, -0xc4,0xbf,0x72,0x3e,0xf7,0xea,0xd8,0x4c,0x15,0x49,0x41,0x8b, -0x0e,0xfe,0x18,0xae,0xa8,0xf2,0xfd,0xf8,0xfc,0xd8,0x32,0x71, -0xc8,0x2b,0x46,0x74,0x80,0x66,0x1b,0x1c,0x26,0x3b,0xb4,0xf7, -0xdb,0x20,0x21,0x80,0x23,0x64,0x4a,0x54,0x42,0x01,0xa2,0x54, -0x07,0x91,0x6a,0xf0,0x4c,0xc7,0x5a,0x60,0x23,0x8b,0xd9,0xd1, -0x63,0xfd,0xf5,0x07,0xe7,0xe2,0xa0,0xa9,0x58,0xec,0xc2,0x4b, -0x99,0x4e,0x41,0xbd,0x73,0x3a,0xe3,0x3d,0xfd,0x04,0xc6,0x24, -0xad,0x7d,0xa4,0xb1,0x4a,0x4b,0x74,0xab,0x64,0x70,0xf0,0x9c, -0x82,0xa4,0x07,0xaa,0x2a,0x27,0x65,0x52,0xdb,0x18,0x28,0x05, -0xc0,0x98,0x4f,0x17,0xaf,0xf6,0x80,0x80,0x59,0x2e,0x4e,0xbf, -0x73,0xd0,0xe1,0xc3,0x09,0xe6,0xd2,0x69,0xe2,0x10,0x1f,0xa1, -0xd6,0x81,0x8b,0x50,0x53,0xd4,0x57,0x81,0xff,0xfc,0x81,0x63, -0xa2,0xd2,0xdb,0xf7,0xd0,0xcc,0x70,0xb6,0x3c,0x92,0xb0,0xe6, -0xf8,0x59,0x84,0xd6,0x33,0x2d,0x87,0x20,0x98,0xf5,0x1c,0xfb, -0x4f,0x5a,0xf5,0xd0,0x09,0x67,0xe5,0x94,0x43,0x10,0x7b,0x67, -0x52,0xea,0x40,0x7f,0x44,0x20,0xda,0xfc,0xa2,0x60,0xac,0x9c, -0xc8,0x9e,0x4c,0x65,0x15,0xeb,0x98,0x85,0xf2,0x78,0x36,0x84, -0x33,0x22,0xd0,0x86,0xa7,0x7d,0xf4,0xa2,0xb6,0xb3,0x6b,0x75, -0xa0,0x22,0xcd,0xdc,0x1d,0x32,0x4c,0x18,0x3f,0xc9,0x14,0xf3, -0x18,0x7e,0xad,0xa5,0xbe,0x52,0xbf,0xcb,0xf9,0xf2,0x42,0x1e, -0xf9,0x8c,0x12,0x90,0xd5,0x5a,0x7b,0x5b,0xfc,0xdd,0xed,0xe2, -0xbf,0x22,0xb4,0xd3,0xc9,0x81,0xa9,0x68,0xb6,0x33,0xeb,0xe5, -0xb2,0xd1,0x51,0x18,0xe3,0x47,0x2e,0xb2,0xf5,0x4d,0xec,0x0f, -0x95,0x6d,0x7d,0x89,0xda,0x05,0xcf,0x47,0xd8,0xbe,0xc5,0x86, -0xdc,0x45,0x96,0x4d,0xa2,0xad,0xbf,0xd3,0x11,0xa2,0x27,0x25, -0x2b,0xfd,0x44,0x93,0xd7,0xed,0xf6,0x9f,0x9c,0x3c,0x3d,0x27, -0xa1,0x01,0x8e,0xb7,0x62,0xb8,0x72,0xc8,0x04,0x56,0x0d,0x5a, -0xe8,0xc9,0x14,0xf4,0x4a,0xad,0x3f,0xa9,0x0a,0x3e,0x43,0xce, -0xf6,0x3f,0x6d,0x73,0x61,0x35,0x23,0x07,0x0e,0x01,0xcd,0xf8, -0xce,0x56,0xed,0x21,0x07,0x1e,0xb5,0x6a,0xda,0x26,0x6c,0x41, -0xd5,0x5a,0x48,0x2d,0x9a,0x76,0x46,0x7f,0x1d,0x1a,0x82,0x7f, -0x6b,0x1f,0x25,0xd8,0x96,0x58,0x54,0x0c,0x1a,0x97,0x11,0x97, -0xd7,0xab,0xcd,0xfb,0xb8,0xf9,0xdf,0xa3,0x49,0xd7,0xd7,0x08, -0x74,0x34,0xa7,0xc9,0x7f,0x4f,0xb6,0xdb,0x2f,0x9a,0x41,0x36, -0xbf,0x6c,0xac,0x86,0x66,0x20,0x47,0x39,0x8d,0xb5,0x0f,0xbb, -0xe3,0x8b,0x48,0x7d,0x1a,0x80,0xa8,0xee,0x0d,0x7b,0x51,0xed, -0xc1,0x96,0xf8,0x58,0xf5,0x5b,0x1e,0xfa,0xdb,0x19,0x9a,0xb2, -0x7d,0x15,0xd9,0x95,0xab,0xbd,0x22,0x3b,0x71,0xb5,0x52,0xcc, -0xef,0x8b,0x06,0xee,0xc5,0xe1,0xf6,0x96,0x46,0x14,0x9d,0x1b, -0x1f,0x4f,0x7b,0xd7,0xca,0x33,0x50,0x77,0xd3,0x68,0xf9,0x00, -0x14,0xd0,0x21,0x36,0x15,0x74,0x87,0xd1,0x27,0x49,0xd7,0x93, -0xd2,0x45,0x67,0x0b,0x63,0x65,0xfb,0x72,0x60,0xba,0x58,0xa0, -0x71,0x1a,0xb5,0x0d,0xe1,0xe3,0x5a,0x74,0x4f,0xb0,0x2f,0x5e, -0x78,0xac,0x84,0xc4,0xc0,0x9d,0x19,0xa9,0x2b,0x3e,0x58,0x85, -0x64,0x00,0x89,0xd6,0x6d,0x34,0x66,0x85,0xb4,0x22,0xba,0xd8, -0x01,0x36,0x56,0xfe,0xfa,0x7b,0x9a,0x32,0x71,0xa4,0xf2,0xf1, -0xc9,0xa9,0x26,0x8d,0xf4,0x47,0x4f,0x02,0x1e,0x5b,0x31,0x78, -0xc7,0xaf,0xee,0xfe,0x3e,0xd8,0x3d,0x07,0xf3,0xa1,0xc8,0x15, -0xe8,0x53,0x5e,0x3c,0xc8,0xfc,0xe2,0x7a,0xe5,0x17,0xc8,0x7c, -0x92,0x07,0xdd,0x2f,0x67,0xb3,0x3f,0xda,0xfb,0xe0,0xf7,0xf7, -0x3e,0xff,0x5d,0xd3,0xe6,0xe1,0xda,0xc3,0xc1,0x87,0xd1,0x3d, -0x3c,0x3e,0xae,0xe0,0x88,0x35,0xfc,0x14,0x08,0x5f,0xfd,0xaf, -0x85,0xb0,0xd7,0xab,0xbe,0x16,0xe1,0xf8,0x13,0xa0,0xdc,0xed, -0xe8,0x32,0xb3,0x02,0xa9,0xf7,0x8e,0xde,0xc6,0xcc,0x3b,0xa2, -0x10,0x3c,0x5f,0xbe,0xc3,0xdf,0x97,0x20,0x6e,0xe0,0x5f,0xf4, -0x89,0xb9,0x5d,0xae,0x67,0xf8,0xbb,0xba,0x9e,0xbe,0xc2,0xc4, -0x5d,0x16,0x1c,0x21,0x41,0xcc,0x6e,0x3e,0x64,0x50,0x65,0xae, -0x03,0x7e,0x38,0x17,0x5b,0x00,0xa1,0xb9,0xdc,0x1c,0x68,0x21, -0x78,0x86,0x00,0x9a,0x90,0xb3,0x49,0x61,0x6d,0x33,0xf8,0xb3, -0x21,0x59,0x73,0x7c,0x6f,0x0d,0x51,0xe3,0x05,0x09,0xc2,0x83, -0x4e,0x39,0x0f,0x62,0x4b,0x61,0x92,0x44,0xbd,0x0f,0x42,0x42, -0x34,0x03,0xdf,0x77,0x24,0xf2,0xbd,0x7b,0x8d,0x04,0x0e,0xf9, -0xf4,0xb7,0x25,0xd0,0x4d,0xa6,0xef,0x83,0x04,0x47,0x15,0xe9, -0x48,0x4e,0x13,0x5e,0x84,0x37,0x2a,0x4a,0x07,0x44,0x2e,0xe2, -0x4e,0x54,0x7e,0x88,0x23,0x9a,0x43,0x81,0x22,0x8f,0xc6,0x11, -0x64,0x0e,0xdb,0x46,0x94,0xc6,0x11,0x78,0x3f,0x2d,0xb8,0x9b, -0xbd,0xc8,0xe7,0xb6,0xc5,0xd4,0x5d,0x77,0xc5,0xc5,0xe6,0xaa, -0x08,0xb7,0xc9,0x5f,0x7d,0x77,0x0e,0x0c,0x7d,0x66,0x8f,0xea, -0xee,0xa9,0x7e,0xbf,0x46,0x7c,0xa2,0xa7,0x5b,0x13,0xf7,0xe9, -0x3b,0x87,0x82,0xc0,0xec,0xdb,0xc5,0xc8,0x53,0x63,0xf8,0xdb, -0x3e,0xc6,0x89,0x91,0x69,0xb8,0x8c,0x7d,0x7a,0xb4,0xed,0xce, -0xdb,0xcf,0x1c,0x27,0x79,0x32,0x29,0xbc,0x04,0x0b,0x99,0xa4, -0xb1,0x0d,0x47,0xd1,0xb0,0xe8,0xd0,0xfc,0x4d,0xd3,0xaa,0x41, -0xb3,0xc8,0x07,0x14,0x83,0x9a,0xf8,0x0e,0xb8,0x26,0x7a,0x9d, -0xbd,0x88,0x6c,0x8e,0x76,0xf2,0x12,0x1a,0xbb,0xd3,0xc1,0x41, -0x67,0x53,0xcd,0x8b,0xfb,0xff,0x89,0x7f,0xd0,0xdf,0x9b,0x45, -0x96,0x1a,0x15,0xcb,0xec,0xf4,0x2d,0x09,0x18,0x23,0xf6,0x2d, -0x7c,0x52,0xc1,0x2d,0x99,0xe0,0xdf,0x37,0xf4,0x20,0x2e,0x2a, -0xe2,0xef,0xbf,0xe8,0x8f,0x5f,0xe4,0x7f,0x3a,0x1b,0x9f,0xf5, -0xcd,0xe4,0x1e,0xb6,0xe3,0x3b,0xd1,0x07,0xc3,0xe7,0x9b,0x3f, -0x3f,0xdb,0xbc,0xba,0x99,0xae,0xa7,0xf0,0xb3,0x9c,0x49,0xd0, -0xb9,0x3b,0xd2,0x5b,0x42,0xbf,0xc4,0x31,0xf8,0x14,0x59,0xe3, -0x6f,0x14,0xd5,0x88,0x63,0x00,0x30,0xb8,0xc4,0x87,0xcd,0x67, -0x52,0x2f,0x83,0xa7,0xfc,0x22,0x15,0x05,0xda,0x7d,0xa0,0x38, -0x7a,0xd9,0x88,0x7d,0xe2,0xd9,0x6c,0xec,0x7d,0x92,0x0b,0x2b, -0xf2,0xdb,0xfa,0xfc,0x3c,0x31,0xac,0x8d,0x67,0x9b,0xe9,0xc5, -0x1b,0x79,0x39,0xd9,0x75,0x26,0xf4,0xe9,0x0b,0x06,0x1c,0x4d, -0xe9,0x24,0x8c,0xb1,0x12,0x3f,0xa4,0x8f,0x1d,0xa0,0x19,0x23, -0xbd,0x97,0xe7,0x42,0x97,0x8b,0x61,0x0c,0xda,0xac,0xb0,0xd3, -0x8c,0xd7,0x8a,0x47,0x9a,0xde,0x3b,0x81,0x1c,0xbb,0x05,0x47, -0xeb,0x9b,0x68,0x14,0x11,0xd8,0xd8,0xca,0x60,0x14,0xd0,0x21, -0x76,0x92,0xe5,0xf8,0x0f,0x56,0xb6,0x24,0x5d,0xf0,0x53,0xc1, -0x36,0x72,0x73,0xf0,0x85,0xbc,0x2c,0x87,0x03,0x77,0xcf,0xf6, -0x4f,0xe6,0x20,0xa8,0xab,0xf0,0xbc,0x65,0xe8,0xf1,0x4c,0x8d, -0xef,0x45,0x5f,0x75,0xa1,0xef,0x6c,0xf2,0x6b,0xbe,0x34,0x6a, -0x9e,0x18,0xce,0x8f,0xbc,0x30,0xa9,0xd3,0xdf,0x89,0x59,0x9e, -0x35,0x6e,0x16,0x39,0xa2,0x87,0xdc,0x9d,0x59,0xf8,0xc7,0xc9, -0x86,0x22,0x25,0x42,0x1a,0xac,0x9e,0xc5,0xf2,0x96,0x1b,0xb4, -0x03,0xee,0x0c,0xb2,0xcc,0x5e,0x60,0xff,0xc1,0xa6,0xe8,0x7d, -0xf9,0xaa,0xde,0xdf,0x4c,0xa7,0xe3,0x3d,0xd3,0xbb,0xff,0xad, -0xdb,0xf5,0xa1,0x80,0x5d,0x11,0xc5,0x32,0x41,0xdb,0xd0,0xb0, -0xa4,0x1a,0x48,0xe5,0x01,0xc6,0x0c,0x84,0x93,0xdc,0x92,0x19, -0x61,0x3f,0xa8,0xe6,0xd4,0x47,0xb9,0x3f,0x43,0x6e,0x31,0x68, -0x36,0x0c,0xeb,0x62,0xbe,0xac,0x01,0x90,0xe6,0xd8,0x6a,0xe7, -0xa8,0x64,0x23,0x10,0x8b,0x51,0xf4,0xdc,0x5f,0xa7,0x46,0x16, -0xc1,0x6a,0x59,0x17,0x07,0x06,0x88,0x8e,0x01,0x7b,0x97,0x73, -0x1d,0x8f,0xde,0x76,0xbe,0xdd,0x06,0x78,0xc8,0x81,0xa8,0x84, -0xa4,0xd4,0x74,0x59,0x56,0xd6,0x90,0x1e,0x7e,0x42,0x0b,0xe6, -0xe8,0xf5,0x39,0x48,0x72,0x2f,0x8f,0x89,0x4d,0x29,0xbd,0xde, -0xa4,0x66,0x65,0x1a,0x61,0x1b,0x83,0x3a,0x0b,0x9e,0xd1,0xda, -0x19,0xaf,0x56,0xbd,0xa5,0x1c,0x5b,0xd0,0x62,0x19,0xe3,0x06, -0x47,0x94,0x20,0x7b,0x33,0x9a,0x4a,0xf3,0xbd,0x49,0xd3,0xa8, -0x7a,0xd7,0x4e,0x77,0x58,0xd9,0x2a,0xc5,0xf7,0x2c,0xff,0x1d, -0x47,0x93,0x3a,0x70,0x64,0x1f,0xb5,0xda,0xb8,0xfd,0xbf,0xd4, -0xbd,0x79,0x7f,0x1b,0x47,0x92,0x28,0xf8,0xff,0xfb,0x14,0x60, -0x0d,0x07,0x5d,0x65,0x14,0x41,0x52,0x76,0xf7,0x74,0x17,0x5d, -0xc2,0xa3,0x29,0xa9,0xad,0x19,0x5d,0x23,0xd2,0xed,0x9e,0x05, -0x61,0x3d,0x90,0x00,0x49,0xc8,0x20,0x00,0xa3,0x40,0x1d,0x4d, -0xe0,0xbb,0x6f,0x9c,0x99,0x91,0x59,0x05,0x4a,0xee,0x37,0xfb, -0x76,0xf7,0x67,0x8b,0xc8,0xca,0x23,0xf2,0xce,0x8c,0x88,0x8c, -0x43,0xe1,0x4d,0x66,0xc7,0xcb,0xe5,0xf0,0xb3,0x2e,0x22,0x75, -0xa9,0xcb,0x1c,0xbf,0x22,0xca,0x44,0x94,0xd6,0x7b,0x52,0xdd, -0x21,0xf6,0x18,0x14,0x20,0x4c,0x84,0xd7,0x57,0x21,0x30,0xda, -0x6d,0x09,0x58,0x45,0x5b,0x59,0x72,0x40,0x9d,0xa5,0x19,0x39, -0xfe,0x45,0x9d,0x1f,0x5d,0x73,0xe8,0x08,0x2b,0x1f,0x8e,0x1a, -0xee,0x83,0x50,0x27,0xb1,0x42,0x69,0xc0,0xad,0x1b,0x29,0xde, -0x2b,0x6e,0x79,0x48,0x2f,0x6e,0x87,0xbf,0x8e,0xb9,0x1f,0x7e, -0x5f,0x3a,0x46,0xb7,0x37,0x94,0xaf,0x51,0x83,0xc2,0x3b,0x99, -0x1b,0x02,0x79,0xaa,0x50,0xc6,0x70,0x38,0xf1,0x31,0x0b,0x87, -0x14,0xda,0x0b,0x33,0x87,0x77,0xe3,0x7d,0xa3,0x73,0x36,0xc4, -0xcb,0x98,0xfa,0x89,0x76,0x24,0xbf,0x74,0xfc,0xa0,0x4b,0x67, -0x26,0x8b,0xc3,0xd3,0x1c,0xc7,0xee,0x35,0xf9,0xce,0x2b,0xa2, -0xef,0xda,0x8e,0xce,0x42,0x07,0xe5,0x80,0x52,0x9e,0xe2,0xcd, -0x68,0x62,0xb8,0x29,0x79,0xe0,0x2e,0xbe,0x99,0x19,0xc2,0x42, -0x30,0x4d,0x8f,0xd6,0x5e,0xf2,0x65,0xe9,0xe5,0xb3,0xe5,0x5d, -0xd3,0x8d,0xaa,0x00,0x65,0x81,0x5b,0x41,0x55,0xb6,0xac,0x59, -0x69,0xcc,0x68,0x22,0x42,0x21,0x56,0x46,0x30,0x73,0x85,0x09, -0xbf,0x89,0x38,0xf9,0x93,0x9c,0xb0,0xa4,0xaf,0x82,0x24,0x59, -0x37,0x8c,0xaa,0x34,0xb7,0x44,0x14,0x17,0xa4,0xb0,0x7d,0xb2, -0xce,0x36,0x8c,0xda,0x7c,0x4d,0xb9,0xf8,0xed,0x5f,0xea,0x84, -0xf5,0xff,0xb5,0x03,0xd0,0x50,0xf3,0xef,0x28,0xbd,0xa5,0xfe, -0x7f,0x6a,0x00,0x6d,0x4b,0x24,0xef,0xff,0xe0,0x06,0xfd,0x73, -0xf3,0x11,0x8b,0x45,0xe8,0xa4,0xc8,0x08,0x7e,0x61,0x89,0xe8, -0x38,0xa7,0xd1,0x8a,0x5c,0xaf,0xef,0x37,0x99,0x61,0x24,0xaa, -0x38,0xbd,0xc3,0x56,0xbf,0x0a,0x6a,0xc4,0x8d,0xc4,0xf2,0x8a, -0xe1,0x3e,0x58,0x5e,0x19,0x5f,0xd2,0xc7,0xc9,0xd5,0x12,0x3e, -0x92,0x8c,0xdf,0x13,0x04,0x44,0x28,0x9f,0xaf,0xb1,0x3f,0x87, -0x36,0x0f,0x8b,0xe0,0xc4,0x11,0xe2,0xd5,0x88,0x0a,0x90,0x2c, -0x5e,0x20,0x81,0x85,0x0e,0xf1,0xdc,0xee,0x16,0x51,0x78,0x97, -0x83,0xc6,0x36,0x0f,0x11,0x6d,0xbc,0xa5,0xdd,0xe9,0xb8,0x60, -0x0c,0xe7,0x6a,0x26,0xd3,0xe0,0x30,0x52,0xfa,0xe4,0x8b,0x9b, -0x24,0xb4,0xc8,0x1e,0xb0,0xf3,0x1e,0x8e,0x59,0x73,0x7f,0xa2, -0x6e,0x3d,0x9f,0x01,0x09,0x32,0xd5,0xc5,0x18,0x0a,0xde,0x65, -0x84,0xcb,0x5a,0x2c,0x02,0xed,0xf9,0xee,0xc4,0x54,0x07,0xf7, -0x6b,0xcb,0x7d,0x98,0x47,0xa5,0x2d,0x9d,0x64,0x7a,0x20,0x6d, -0xc1,0x0b,0x16,0x7d,0xd0,0x2f,0x59,0xbd,0xa1,0x7e,0xfd,0xda, -0x13,0xd4,0x13,0x2e,0x21,0xf6,0xe8,0x54,0x83,0x50,0x4e,0xc2, -0xbf,0xb9,0x41,0x98,0x64,0x39,0x81,0xee,0x4c,0x0a,0xf8,0x48, -0xc9,0xae,0xca,0x92,0xed,0xb6,0xe9,0x1b,0x8d,0xf3,0x05,0xfe, -0x10,0x4e,0x41,0x19,0xf1,0x6e,0x26,0x8d,0x99,0x5e,0x5f,0xbf, -0xd1,0x72,0x45,0x13,0x32,0x62,0x5a,0x94,0x91,0xb5,0xcc,0x68, -0x5b,0xa2,0x1f,0x30,0xd9,0x98,0x8a,0xc3,0xe9,0xd3,0x2b,0xb2, -0x43,0xee,0x96,0xc6,0x75,0x15,0xe1,0x4f,0x7f,0x89,0x90,0x23, -0xe8,0x17,0x93,0xa3,0x65,0x69,0x74,0xcf,0x63,0x14,0x6a,0xbd, -0xde,0x91,0x1b,0x19,0x71,0x1f,0x20,0x7a,0xb9,0x10,0x0a,0x39, -0x60,0xf7,0x43,0x80,0xfa,0x9e,0xee,0x70,0xa5,0x5c,0xb0,0x24, -0x6e,0x46,0xf0,0xe2,0x3c,0x72,0xe7,0x83,0x11,0x40,0xcc,0xfd, -0x5b,0x02,0x2d,0x6c,0x34,0x94,0x35,0x3b,0x9a,0x95,0xb3,0xe0, -0xd9,0x80,0x44,0x61,0x4c,0xb5,0x33,0x91,0xbc,0x80,0x85,0xc2, -0x55,0xcf,0xfc,0xaa,0x87,0xa9,0x3f,0xb2,0x4f,0x31,0x55,0x99, -0x0c,0x2f,0x2e,0x96,0xeb,0xe1,0x72,0x35,0xb9,0x9c,0x8e,0xd7, -0xc3,0x6a,0x02,0x47,0xcd,0xf0,0x6e,0x34,0x99,0xaf,0x2f,0x46, -0x93,0xf5,0xe5,0x70,0xf6,0x61,0x58,0x91,0x3f,0x2d,0xfa,0x33, -0x9d,0x54,0xab,0xf5,0x68,0x0c,0x74,0xcd,0xb4,0x5a,0x5f,0x4d, -0xae,0x2f,0x87,0xf4,0x06,0x83,0xc1,0xbb,0xe5,0x78,0x7d,0x35, -0x9f,0xa3,0xb3,0x1d,0x7e,0x25,0x58,0xdf,0x30,0x4b,0x09,0x1f, -0x52,0xd7,0xb7,0x63,0x4c,0x98,0x0d,0x3f,0xac,0xe7,0x77,0x2b, -0xb4,0xe4,0xb6,0x58,0xce,0xaf,0x51,0xb5,0x64,0x5d,0x8d,0xa9, -0xb7,0xeb,0xea,0xee,0x16,0x72,0x7e,0x5e,0xaf,0x26,0xb7,0xe3, -0xf5,0x07,0x68,0xc6,0x3c,0x41,0x0b,0x70,0xd0,0xc5,0x31,0x0f, -0x79,0xb9,0x2f,0xa7,0xd1,0xf9,0xa8,0x43,0xc6,0x3c,0xf1,0xe2, -0x5d,0xa3,0x49,0xbd,0x04,0x8d,0xb3,0xcd,0xe6,0xd5,0xcd,0xe4, -0x96,0x65,0x75,0xac,0x78,0xea,0xf7,0x64,0x48,0xd3,0x75,0x18, -0x2d,0x15,0x9e,0x9f,0x57,0xfb,0x8f,0x07,0x62,0x71,0x6f,0x29, -0x7e,0x2d,0x7e,0xf6,0x16,0x45,0xf7,0x7f,0x39,0xaf,0x3a,0xfb, -0xf9,0xf2,0xd3,0xcd,0xea,0x76,0x7a,0x36,0xbc,0x2e,0xf7,0x01, -0xc8,0x0e,0x1b,0x9c,0x5b,0xae,0x2f,0xe7,0xd3,0xf5,0xf8,0xf6, -0x02,0x96,0xc7,0xcd,0x72,0x3d,0xb9,0xbd,0x5e,0xb3,0x69,0x3a, -0x68,0x28,0x75,0x73,0xb8,0x86,0xed,0x39,0xbc,0xcd,0x52,0x34, -0x65,0x57,0x0c,0x3a,0x59,0xff,0x97,0xc7,0x83,0x6f,0xb2,0xf3, -0xfd,0xc7,0xfb,0xd7,0x93,0x7c,0x29,0x86,0x32,0x10,0xa4,0xa4, -0x43,0x45,0xab,0x8b,0xf9,0x08,0xfa,0xf7,0x3d,0xfd,0x92,0x2d, -0x3b,0xa8,0x18,0xbe,0xd7,0xed,0x7f,0xe9,0x9d,0x7f,0xec,0x1c, -0x21,0x4f,0x63,0xfe,0x1c,0xdf,0xb0,0x25,0x01,0xfa,0x54,0x5d, -0x2e,0x27,0x8b,0xd5,0xba,0x5a,0x7d,0x86,0x79,0xc3,0xca,0x33, -0x28,0x78,0x3b,0x9c,0x4d,0x16,0x77,0x53,0x7a,0xd5,0x7c,0xb7, -0x24,0x7e,0x27,0x3e,0xce,0x91,0xc4,0x21,0x71,0x54,0x94,0x05, -0xba,0x26,0xa6,0x28,0x1b,0xce,0x93,0x17,0xb5,0x72,0x5f,0x02, -0xe7,0x15,0xea,0xca,0xf7,0x7f,0x29,0x07,0xeb,0x12,0xc2,0xfa, -0xe2,0xd6,0xc5,0x0a,0x96,0x5c,0xad,0x00,0xdc,0x5d,0x9f,0xef, -0x43,0xd6,0xf7,0xc3,0x0f,0xc3,0xf5,0xf8,0xf2,0x76,0x98,0x71, -0x6a,0x98,0xef,0xe5,0xb0,0x22,0xe8,0xbf,0xac,0x96,0x77,0x63, -0xc8,0xdf,0xfd,0x06,0xbb,0x7c,0x89,0x96,0xb2,0x4e,0x29,0x13, -0x8d,0xf7,0x37,0xdf,0xef,0xa0,0x82,0x7e,0xff,0xe4,0xc9,0xf1, -0xd9,0xf1,0x79,0x7f,0xbd,0xb7,0x97,0xad,0x31,0x62,0x70,0x3e, -0xc0,0xf0,0x63,0xc8,0xb1,0x0b,0xd3,0xfc,0x71,0x39,0x5c,0xa0, -0x53,0xa9,0x7b,0x7e,0xfc,0x2b,0xfa,0x87,0xb9,0xda,0x59,0x70, -0x86,0xc7,0xcb,0x3f,0x68,0xe8,0x0f,0x8f,0x61,0x92,0xbd,0x85, -0x85,0x01,0x10,0xf8,0xd7,0x70,0xde,0x71,0xa9,0xab,0xc9,0x78, -0x3a,0x02,0x44,0x97,0xf3,0xf8,0xaf,0x41,0x8e,0x33,0xcd,0x79, -0xe0,0x02,0xe1,0x64,0x0a,0x0c,0x72,0x9a,0x5a,0x4e,0x62,0x5f, -0xcf,0x9c,0xaa,0x61,0xe4,0x9e,0xc0,0x6a,0xe2,0x0c,0x2b,0x1c, -0x79,0x4e,0x97,0x20,0x24,0x2f,0x8b,0xfe,0x23,0x97,0xc6,0xf3, -0x2d,0x59,0x28,0x68,0xb2,0xc2,0x32,0x6b,0xc8,0xeb,0x32,0x42, -0x32,0x6d,0x2f,0x2e,0xed,0xbe,0x6c,0x5d,0xd0,0x8e,0x6f,0x6b, -0xe5,0x57,0x4b,0xa9,0x6f,0xf9,0xb8,0xa1,0xd2,0x77,0x70,0xf2, -0x0d,0x61,0xf0,0xf4,0x08,0x56,0xbd,0x2c,0x5c,0x76,0xa7,0xe3, -0xe5,0x04,0x0e,0x80,0x7f,0x00,0x39,0x71,0x80,0x92,0x4b,0x49, -0x32,0xa0,0x8e,0xfe,0xfd,0x7b,0x36,0xa7,0x95,0x88,0x5d,0xad, -0x01,0x1c,0x65,0xc3,0xab,0xf1,0xb3,0xe5,0xf0,0x9a,0x6c,0xcb, -0x31,0xe7,0xfb,0xd4,0x44,0x39,0xbb,0x73,0x59,0x7e,0x25,0x51, -0x4f,0x26,0x1f,0x4a,0x5b,0x2a,0x36,0x0f,0xc7,0x0f,0xdf,0x0c, -0xea,0x29,0xf3,0x74,0x53,0x36,0x23,0x90,0x1d,0xc9,0x9a,0xe8, -0xc2,0x92,0x60,0xe6,0xb5,0x89,0x40,0xdd,0x1e,0xfd,0xe4,0x5d, -0xe6,0xbe,0xf0,0xb0,0x72,0x5f,0x3a,0x80,0x3e,0x82,0x4f,0x36, -0x9f,0x1d,0x27,0xd6,0x83,0xba,0xf1,0x09,0xa3,0xbc,0xce,0x08, -0x0c,0x9f,0x16,0xd9,0xac,0x46,0x84,0x52,0x0d,0x2f,0x51,0x1d, -0x41,0xd0,0x93,0xe6,0x9c,0x6a,0x7c,0xc7,0x1b,0x6d,0x31,0x7c, -0xd9,0xd4,0xd3,0xa7,0xf2,0x82,0x9b,0xc9,0xa0,0xa5,0x69,0x8d, -0x68,0x8d,0x14,0x28,0xdd,0xf8,0xcb,0x88,0x22,0x63,0x97,0x14, -0x12,0xb9,0x7e,0xc4,0xa6,0x51,0x20,0x88,0xbe,0xdc,0xc3,0x4d, -0xe5,0xac,0xb3,0xd3,0x30,0x04,0xd8,0x3a,0xae,0x0f,0xe2,0x2e, -0x38,0xee,0xf0,0xb3,0x20,0x29,0xb3,0x54,0x20,0x91,0x64,0xd6, -0x28,0xa4,0xb2,0xe2,0x88,0xa1,0x23,0xb0,0xa9,0x1c,0x8b,0x9c, -0xd1,0x10,0x4d,0x50,0xca,0xe9,0x48,0x6d,0x43,0x1f,0x0c,0x18, -0xd7,0xc3,0xcc,0xca,0xc9,0xc3,0x02,0x79,0x63,0x87,0xb3,0xee, -0xf8,0xb7,0xf4,0x00,0x3a,0x3b,0x9d,0xcf,0xc6,0xe9,0xce,0x41, -0x76,0x54,0x27,0xe6,0xdb,0x6d,0x84,0x15,0xda,0x82,0xd3,0xaa, -0xa8,0xbf,0x84,0x4c,0x46,0xac,0x3f,0x15,0xbc,0x21,0xfc,0xee, -0x28,0x42,0xae,0x8d,0xd4,0x8a,0xb1,0x6b,0xe6,0x99,0x31,0x8d, -0xe2,0x01,0x86,0xc7,0xb1,0x71,0xf3,0x49,0x03,0x63,0xf9,0x29, -0x3c,0x03,0x74,0x21,0xc4,0x73,0x10,0x2e,0xb2,0x78,0x1e,0x7a, -0x5f,0x39,0x01,0x04,0xba,0x71,0x0a,0x8a,0x06,0x08,0xca,0xc8, -0x70,0x74,0x38,0x43,0x72,0xb4,0x05,0x52,0xfb,0x57,0x4a,0x10, -0x54,0x29,0x3e,0x7b,0x71,0x50,0x31,0x45,0xf7,0x6d,0x27,0x3f, -0x2b,0xa8,0x98,0x8c,0x01,0xc5,0x6c,0x64,0xed,0xc5,0x9d,0x26, -0xce,0xae,0xeb,0x6a,0xb9,0xa5,0xf3,0x47,0xbf,0x7f,0x11,0x7a, -0x10,0xbd,0xda,0x60,0x14,0xae,0x49,0x77,0xb3,0xb0,0x51,0x11, -0xcf,0x95,0xd6,0x18,0xec,0xce,0x78,0xd0,0x62,0xda,0x8a,0xe0, -0x26,0x78,0x48,0xa1,0x2c,0x56,0xd0,0x1e,0x91,0x0d,0xfb,0x79, -0xb2,0xba,0x61,0x36,0x8d,0xa5,0x98,0x00,0x32,0x0c,0x10,0x72, -0x61,0x68,0xa8,0xb6,0xb5,0x63,0x34,0xbf,0x7d,0x89,0xe8,0x81, -0x13,0x50,0xad,0x72,0x7c,0x07,0x0d,0xc9,0x3e,0x12,0xac,0xa6, -0xec,0x91,0x86,0x69,0x14,0xf7,0x97,0x38,0x2a,0xe3,0xc3,0x26, -0x38,0xb7,0x99,0x44,0x65,0xea,0xfe,0xff,0xfd,0x96,0x05,0x5b, -0xdb,0x71,0xf8,0x02,0x6a,0x18,0x85,0x51,0x28,0xfd,0xf7,0xb4, -0xf4,0x30,0x6e,0xa9,0x99,0x75,0x3e,0x59,0xa2,0x88,0x2d,0x2d, -0xa1,0xea,0x87,0x57,0x01,0x1d,0xf6,0x7f,0xae,0xf6,0x80,0xa4, -0xc0,0x96,0xb0,0xf5,0xc8,0x06,0xee,0xe5,0xaf,0xe3,0xf1,0xe2, -0x09,0xd0,0x02,0x91,0xac,0x2b,0x3e,0x83,0x58,0x27,0x49,0xc2, -0xad,0x66,0x1f,0x49,0xe9,0x8e,0x77,0xe5,0xb9,0x85,0x58,0x8e, -0x0d,0x16,0x20,0x3d,0xab,0x55,0x31,0x21,0x36,0x0e,0xa5,0xa0, -0xf4,0x99,0x05,0x11,0x49,0xcc,0x94,0xa2,0x7c,0x8d,0x18,0x62, -0xc8,0x58,0x00,0x31,0x18,0x07,0x07,0xad,0xdd,0x8e,0x5f,0x68, -0x28,0x6f,0x70,0x65,0x30,0xe9,0x45,0x86,0x05,0xfe,0x3a,0x9d, -0x5f,0x0c,0xa7,0x4f,0xd1,0x16,0x80,0xa9,0x21,0x4f,0x18,0xc3, -0x4d,0xea,0x55,0x05,0x86,0x37,0xb9,0x35,0xc1,0xd9,0x53,0x13, -0x19,0xfb,0x3d,0x23,0x49,0x59,0x1b,0xec,0xcc,0x3d,0x34,0x1a, -0xf4,0x44,0x14,0x5f,0x4f,0x7c,0x01,0xd5,0xda,0x1a,0x18,0x3d, -0xe5,0x71,0x21,0x44,0xa8,0x72,0x55,0x44,0xbc,0x20,0x9e,0xc0, -0x24,0x73,0x46,0x95,0x39,0xbb,0x5a,0x3c,0x3d,0x88,0x2e,0x2e, -0xba,0x83,0x7d,0xd7,0x91,0xa8,0x3c,0x9e,0x8d,0x9e,0x7e,0xa0, -0xd5,0x3c,0x92,0x19,0x72,0x31,0xde,0x0e,0xa9,0x8d,0x15,0x7f, -0x60,0x41,0x5c,0x6f,0xe7,0xb0,0xf8,0x02,0x30,0x2d,0x16,0xc7, -0xf7,0x82,0x72,0x45,0x2d,0x9d,0xf7,0x47,0x74,0xfd,0x87,0xf7, -0x2c,0x63,0x16,0x74,0x80,0x7f,0xa9,0x47,0xb8,0xb7,0xf0,0xe6, -0xf8,0xdf,0x40,0x0f,0x03,0xb4,0x03,0x10,0x14,0xe4,0x0d,0xd6, -0x9f,0xab,0x10,0x59,0xaa,0x61,0x90,0x99,0x51,0x33,0x0b,0x56, -0x10,0xb3,0xf2,0x9c,0x61,0x36,0x2f,0x8d,0x6c,0x49,0x70,0xc0, -0xf9,0x33,0x6f,0xe2,0x92,0x44,0x48,0xea,0x2f,0xd7,0x54,0xe7, -0x7a,0x6d,0x29,0x56,0xe6,0x52,0x71,0xeb,0x1d,0x0b,0xa5,0x99, -0xb8,0x68,0xb7,0x2d,0x25,0xff,0x70,0xc1,0x1a,0xe5,0x0e,0x85, -0x6b,0x71,0x21,0x08,0xc1,0xda,0xfb,0xa9,0x12,0xe2,0x80,0xb2, -0x8f,0x2f,0x5d,0x72,0x9f,0xe9,0x9a,0xac,0xa6,0x74,0x3e,0xc8, -0x70,0xd8,0x71,0x30,0xd9,0x51,0x91,0x1b,0x1e,0xe5,0x11,0x00, -0xf9,0xb3,0x7b,0x08,0xa4,0xd4,0xee,0xa3,0xc7,0x49,0x46,0x76, -0xd4,0x49,0xbd,0x47,0xdf,0xf0,0xec,0x46,0xa6,0xe9,0x6a,0xd8, -0xc2,0xe9,0x17,0xf7,0x70,0x1e,0x4e,0x12,0xb7,0x85,0xb7,0x69, -0x79,0x60,0xac,0xc3,0x6c,0x98,0xff,0xd3,0x44,0x1b,0x70,0x91, -0x2f,0x21,0xf8,0x06,0xdf,0xa8,0xad,0x53,0x8f,0x6c,0x35,0x20, -0x5a,0xd2,0x22,0x55,0x39,0xa0,0x04,0xff,0x1c,0x5a,0x46,0x6b, -0x44,0xd6,0xa7,0xa0,0x3a,0x5c,0xb6,0x8b,0x7c,0x45,0xc6,0x7a, -0x90,0xd1,0x04,0x3d,0x52,0x2e,0xa6,0xbb,0xfe,0xfa,0x94,0x73, -0xd0,0x80,0x24,0x10,0x6f,0x0b,0xa5,0x3e,0xe2,0x2b,0x4d,0x95, -0x13,0xa3,0xcb,0xf0,0x48,0x1f,0x78,0xd2,0x08,0xdd,0xc2,0x43, -0x31,0xcd,0xa4,0x54,0xc3,0x95,0x39,0x23,0x17,0x4d,0xb8,0x97, -0xb9,0x91,0x5f,0x7a,0xf9,0x12,0x88,0xf6,0xa5,0x1d,0x8a,0x4a, -0x87,0x8c,0x55,0x2e,0x54,0x61,0x22,0x6a,0x3c,0x47,0x54,0xf3, -0x62,0x88,0x3a,0xc7,0xa4,0x39,0x74,0x09,0x55,0xbf,0x83,0x0b, -0x0b,0x26,0x59,0x5c,0x38,0xf5,0x07,0xac,0x34,0x43,0x0c,0x3d, -0x56,0x52,0x25,0x85,0xce,0x9b,0x61,0xc5,0x7c,0x95,0x2a,0xaf, -0xe4,0x17,0xcd,0xbb,0x2b,0xa5,0xdd,0xf0,0xae,0x5d,0x09,0x87, -0x3a,0x9f,0xbc,0x9a,0x9f,0xe0,0x49,0x56,0x4e,0xf7,0x0e,0x79, -0x71,0x94,0x58,0x05,0xf2,0x69,0x1f,0x44,0xaf,0x65,0xd6,0xe1, -0x3c,0xf0,0x91,0x68,0x84,0xee,0xf0,0x71,0x39,0x6d,0x78,0x0c, -0x97,0xd9,0x8f,0x76,0x33,0xb1,0x99,0xa8,0x7a,0xb8,0x4b,0x94, -0x29,0x65,0x77,0xef,0x43,0x44,0x24,0x89,0xb8,0x7b,0x32,0x04, -0x7a,0x84,0x34,0x1f,0x47,0x1f,0xf9,0x46,0xa1,0xe5,0x4e,0xee, -0x90,0xec,0x63,0x83,0xd0,0x93,0x80,0x37,0x8d,0x7d,0x8f,0x08, -0x0f,0xdc,0xd8,0xa9,0x39,0xee,0x32,0x7a,0x11,0xb8,0x0a,0xb0, -0xb0,0xda,0x64,0x31,0x8d,0x3a,0x85,0x7a,0x74,0xbc,0x75,0xb8, -0x2e,0xee,0xe0,0x4a,0x75,0x1c,0x10,0x2e,0xdb,0x44,0xac,0x22, -0x4a,0xc7,0xe4,0x13,0xcd,0x69,0xa9,0x70,0xec,0xdb,0x0f,0x29, -0x09,0x6b,0xbc,0x27,0x05,0x9c,0x3d,0x4c,0x5f,0x39,0x15,0x12, -0x50,0x99,0xb8,0xd4,0xc6,0x16,0x97,0xf4,0xb7,0x7e,0xa7,0xf3, -0x3a,0x4a,0x56,0xcb,0x24,0xd3,0xd5,0x63,0x1f,0x53,0xe4,0x40, -0x72,0x6b,0xc9,0x21,0x41,0xaa,0x2a,0xc0,0x2b,0x2f,0x33,0x8b, -0xb0,0x14,0x30,0xce,0xd1,0x93,0x9c,0x88,0xa4,0xf7,0xe8,0x17, -0x25,0xe0,0x77,0xba,0xfa,0x50,0x76,0x1a,0x13,0x83,0xbb,0x95, -0x96,0x36,0x6c,0x78,0xf4,0x5a,0xec,0x81,0xbb,0x0e,0xf0,0x7b, -0x92,0xae,0x77,0x69,0x26,0x95,0xf1,0x78,0x5a,0xe6,0xa6,0x49, -0xa6,0xbd,0x79,0x08,0x54,0x80,0x29,0xa1,0xf4,0x24,0xeb,0xe1, -0x3b,0xc4,0xeb,0xe5,0xb1,0xa7,0xc5,0x39,0x95,0x09,0xb5,0x42, -0x23,0xa8,0xb2,0x49,0x26,0xb6,0x8d,0xa4,0x7d,0x24,0xbb,0x81, -0x2e,0x1d,0xa4,0x65,0xfd,0x70,0x30,0xf6,0x0e,0xe3,0xc9,0x37, -0x63,0xad,0x9d,0x59,0xc2,0x0e,0x80,0xcd,0xaf,0x23,0x8b,0xd8, -0xa1,0xaf,0xc0,0x0e,0xa6,0xd6,0x81,0xc2,0x57,0x9e,0xfd,0x2a, -0x2f,0x44,0x88,0x97,0x8a,0x1c,0x3c,0xe2,0x6a,0x7a,0x95,0xbe, -0x43,0x74,0x45,0x06,0xea,0xda,0x21,0xba,0x98,0x23,0xc6,0x91, -0xf1,0x0c,0x99,0xb1,0x36,0x0e,0x43,0xab,0x96,0x97,0xca,0x8f, -0x1a,0xbe,0x1f,0x7e,0x4a,0xef,0xef,0x96,0xd3,0x42,0x53,0xd8, -0xd8,0x7a,0xf2,0xd7,0xa7,0x67,0x09,0x61,0x44,0x67,0xf4,0x29, -0x53,0x91,0x0f,0xab,0xcf,0xb3,0xcb,0x02,0x16,0x3a,0x57,0x89, -0xa1,0x84,0xcc,0x52,0x55,0x09,0x09,0xcf,0x2a,0xfb,0xd1,0xb7, -0x28,0x95,0x1e,0x90,0xa0,0x8c,0x0b,0x3a,0x25,0xaf,0x19,0x13, -0x36,0x72,0x2f,0x46,0xf6,0x6b,0x0c,0xab,0x19,0x11,0x18,0x94, -0x68,0x0b,0xb6,0x07,0xe1,0x85,0x01,0x96,0x6a,0x1e,0xc8,0x48, -0x7c,0x80,0xef,0xce,0xb3,0x79,0x91,0x70,0x28,0x51,0x8a,0x16, -0xa3,0x24,0x98,0xe4,0xf6,0x96,0x28,0x12,0x26,0x24,0x35,0xf6, -0x98,0xe8,0xba,0x84,0xc8,0xbb,0x44,0xaf,0x57,0x64,0xa1,0x25, -0xe6,0xaa,0x4d,0xe2,0xb7,0xcf,0xf9,0x72,0x72,0x3d,0x99,0x0d, -0xa7,0x0f,0xbc,0x80,0xfa,0xeb,0xc6,0xd2,0x10,0x95,0x98,0xb7, -0x66,0xa1,0x3d,0x6a,0x41,0x19,0x0b,0x0a,0x92,0x8e,0x40,0xc9, -0x89,0x6e,0x3d,0x1e,0x61,0xe4,0xe3,0xd2,0x23,0x2d,0x55,0x39, -0x51,0x33,0x04,0x38,0x34,0xcc,0xfe,0x71,0x8c,0x34,0x15,0xad, -0x61,0x28,0x48,0xa4,0xf4,0xb5,0xcd,0x03,0x7e,0xe3,0x43,0x2e, -0x10,0xdc,0x5c,0x81,0xeb,0x41,0x71,0x1e,0xcc,0xb2,0x26,0xdb, -0xe4,0x4c,0xb6,0xbc,0x54,0x46,0xf4,0x03,0x3f,0xfe,0x7d,0x89, -0x88,0x20,0xc7,0x2a,0xb0,0x0d,0x84,0x73,0xcc,0xca,0xdf,0x4c, -0x8a,0xe4,0x93,0x1c,0x96,0xab,0x4b,0x98,0xcc,0xde,0x0c,0xaf, -0xfd,0xd1,0xf3,0x25,0x02,0xf1,0xc8,0xb3,0x3c,0x2d,0x36,0xfb, -0x47,0x3a,0xc7,0xdc,0x0d,0xa7,0xb2,0xb2,0x8c,0xa4,0xc0,0xc5, -0x58,0x47,0x71,0x93,0xef,0x93,0xce,0xd8,0xaa,0x98,0x74,0x12, -0xc0,0x23,0x7b,0xd4,0x44,0xc6,0x13,0x29,0x48,0x4c,0xda,0x1d, -0x34,0x69,0x6a,0x18,0xe7,0x06,0x21,0xe4,0x76,0xa2,0xce,0x3b, -0x7e,0x5a,0xee,0x7a,0x40,0xe8,0x31,0x58,0x9b,0x6a,0xa8,0x3e, -0xb4,0xa6,0x18,0xf5,0x69,0xc6,0x07,0x33,0x0d,0xa7,0x3b,0x19, -0xa2,0xc4,0x13,0xd5,0xcb,0xaa,0x91,0xea,0xd6,0xbc,0x61,0x8d, -0x7c,0x0f,0xc7,0x26,0xe3,0x03,0xd3,0x4c,0x55,0x29,0x67,0x39, -0x35,0x39,0xb3,0x73,0x55,0x5a,0x7a,0xdf,0x12,0xcd,0x7c,0x18, -0xfa,0x8c,0x44,0x3b,0x77,0x3a,0x93,0xcc,0xc2,0x85,0xc8,0x76, -0xfb,0x6a,0xf2,0xe9,0x44,0x87,0xf5,0x79,0x55,0xdd,0x8d,0x2b, -0x3e,0x0a,0xa3,0x8c,0x34,0xcd,0xc1,0x22,0x43,0x31,0xcf,0xfa, -0x3a,0xc3,0xb6,0xdb,0x06,0x9a,0x30,0xe9,0x9d,0xfa,0xd6,0x06, -0x3d,0xb4,0x1f,0x2e,0x9f,0xf4,0xf7,0x4b,0xdd,0xc2,0x6d,0x4a, -0x59,0x4f,0xe6,0x8b,0xcf,0xd4,0x8c,0x2d,0x3d,0x20,0x8b,0x03, -0x51,0x4e,0xda,0x3c,0x5c,0x91,0x53,0xd1,0xdf,0x36,0xf4,0xf6, -0xaa,0x37,0x79,0x8c,0x1f,0xe3,0x90,0x55,0x12,0x6c,0xb7,0x1d, -0xde,0x56,0xed,0xf6,0x16,0x06,0x4a,0x50,0xab,0x1d,0x41,0x36, -0x2d,0x81,0xa4,0x0a,0x35,0x02,0x75,0xc5,0x0c,0x16,0x15,0x1e, -0x03,0x5e,0x11,0x5b,0x6f,0x4e,0x3e,0xea,0x20,0xdd,0x1f,0x8e, -0xef,0x73,0xee,0xb4,0x3a,0xf1,0x5c,0xdd,0x2e,0x72,0xb4,0xa6, -0x4f,0xb7,0x39,0xf1,0x96,0xf3,0x69,0x20,0xc5,0x40,0x6f,0x59, -0x4d,0x6f,0x58,0xce,0xd0,0x25,0xb6,0xb1,0x72,0xc2,0xd1,0x5e, -0x0c,0xd8,0xf1,0xf7,0xe9,0x46,0xc6,0x80,0x57,0x82,0xc5,0xf4, -0x84,0x5f,0x0d,0x51,0x6d,0x4c,0x9f,0x75,0x60,0x63,0xc8,0x3e, -0x08,0xd0,0x1a,0xaa,0x21,0x0f,0xc9,0x75,0x66,0x97,0x15,0x7c, -0x0a,0xa9,0xd1,0x0a,0x43,0x6c,0x33,0x18,0x46,0xf8,0x6e,0x17, -0x25,0xfc,0x5b,0xaf,0xb1,0x2b,0x01,0x83,0x56,0x8d,0x7c,0x36, -0xbe,0xab,0xe1,0xb8,0x94,0x11,0x9d,0x2c,0x07,0xd8,0x56,0x32, -0x99,0x46,0x50,0x9f,0xc6,0xfa,0x50,0x74,0xe0,0x48,0xee,0xae, -0xbe,0x2d,0xe2,0x98,0x9b,0x23,0x0b,0x93,0x01,0x4e,0x47,0xb8, -0x51,0x0f,0x91,0xd5,0x1d,0xca,0xfb,0x68,0x90,0xbf,0xe7,0x52, -0x07,0x03,0xb6,0xd7,0x28,0x1d,0xf4,0xf6,0x53,0x88,0x33,0xf1, -0xcf,0x33,0x0b,0x84,0xf7,0x47,0xc3,0x2e,0xd2,0x1b,0xc1,0x40, -0xb9,0xe7,0xb2,0x06,0x00,0x7e,0x98,0xf0,0xe5,0x08,0xce,0xd2, -0xa8,0x19,0xb4,0xd0,0x32,0xa7,0x1e,0x2a,0x48,0xe6,0x0e,0x39, -0x96,0x58,0xaf,0x59,0x12,0xc0,0xb4,0xa2,0xe7,0xde,0x91,0x21, -0x8b,0x0c,0x55,0x53,0xb6,0x83,0x02,0x06,0x00,0xff,0x59,0x32, -0xe1,0xbd,0xc8,0x81,0x8c,0x43,0x19,0x2b,0xc5,0xc4,0x69,0xec, -0x6a,0xd8,0x2f,0x3d,0x92,0x46,0x25,0xd0,0xee,0xa5,0x22,0xbc, -0x80,0x3b,0x72,0xed,0x0d,0x74,0x47,0x8d,0xa3,0xc8,0xbd,0x25, -0x0d,0x82,0x86,0x15,0x8d,0xcd,0x35,0x0f,0x19,0xf4,0x6d,0xd0, -0xba,0x32,0x49,0x8e,0xc2,0x1e,0xd1,0x54,0x87,0x15,0x04,0xe9, -0x19,0xe6,0xa7,0x77,0x64,0xbf,0x16,0xd8,0xa2,0xcb,0x97,0xe7, -0x92,0x37,0xcc,0x91,0x6c,0x18,0x38,0xce,0x10,0x4a,0x54,0x95, -0xc3,0x46,0x9c,0xfd,0x57,0xd9,0x4c,0x72,0xf5,0x29,0xda,0x0a, -0xcb,0x38,0x35,0x34,0x48,0x95,0x8b,0x1e,0x29,0x12,0x62,0x9f, -0x9e,0xf0,0x36,0x90,0x42,0x7c,0xc4,0xd3,0x5a,0xa0,0xac,0xe2, -0x49,0x04,0x2d,0x27,0xef,0xb8,0xf3,0x6b,0xbd,0xde,0x3b,0xf4, -0xc7,0x84,0x95,0x5c,0x36,0x67,0x1c,0x22,0xe6,0x4e,0xf1,0xe6, -0x6b,0x11,0x18,0xf2,0x86,0x22,0x4d,0xad,0x1d,0x0f,0x9c,0xc3, -0x1f,0xfc,0x0a,0x2d,0x3e,0xeb,0x69,0x64,0xe4,0xd0,0xe5,0x1b, -0xfc,0xbd,0xf6,0x09,0x92,0xc4,0xb5,0x7f,0x8d,0x24,0xa1,0x26, -0x19,0x92,0x44,0x49,0xa3,0x46,0x6f,0x29,0x7c,0x09,0x60,0x0b, -0x91,0x85,0x2c,0xfc,0xb1,0xf8,0xfc,0x47,0x9e,0xe9,0x62,0xd5, -0xf0,0x36,0x41,0x56,0xbd,0x26,0x23,0x42,0x14,0x09,0x33,0x9e, -0xcc,0xd0,0xa2,0xdc,0x70,0xfa,0x1f,0xe3,0xcf,0x46,0x41,0x08, -0xf5,0xd3,0xc4,0x62,0x92,0x8e,0x1f,0xd9,0xc9,0x1a,0x01,0x10, -0x38,0x1e,0xc5,0xcb,0x73,0xb4,0x04,0x82,0xc4,0x9c,0x2c,0xf2, -0x0e,0x9d,0x60,0x35,0xaa,0xb2,0xae,0xba,0x12,0x19,0x30,0xf5, -0xf5,0x62,0xc8,0xf4,0xba,0x48,0x7d,0xeb,0xdd,0x42,0xf2,0x51, -0xb2,0x3c,0x61,0x8a,0xd1,0x17,0x1a,0x8a,0xb2,0x99,0x2e,0x0c, -0xa8,0x63,0xe5,0x64,0xa4,0x96,0x87,0x26,0xa3,0x01,0xbb,0x0f, -0xc7,0x78,0x6e,0x04,0x63,0x27,0x38,0x10,0xad,0x09,0xb3,0xd3, -0x35,0x5e,0x5a,0xc7,0xa6,0x7c,0x7a,0x41,0xc3,0x85,0x6b,0xe5, -0xc6,0xd0,0x11,0x66,0x9c,0x60,0xb0,0x08,0x1a,0x61,0x82,0x7a, -0x43,0x16,0x94,0x45,0x61,0x0f,0x5b,0x02,0x6d,0xe6,0x31,0x6a, -0xb9,0xa8,0x70,0x44,0x7b,0x92,0x5c,0xeb,0x56,0x61,0x8d,0xfb, -0x71,0x95,0xd6,0xbe,0x1f,0x11,0x16,0x81,0x4f,0x9b,0xa6,0x8c, -0xa9,0x81,0xc8,0x3a,0x86,0x41,0x1d,0x82,0x5d,0x20,0x28,0x6e, -0xc6,0xe8,0xf9,0x48,0x75,0x4b,0x47,0x28,0x6d,0x2e,0x12,0x73, -0x2c,0xf2,0x8a,0x9c,0x85,0x53,0x94,0xb8,0xaa,0x50,0x96,0xef, -0xe4,0xf4,0x34,0x5f,0x0e,0xa7,0x8b,0x9b,0x61,0xb9,0x4f,0x3f, -0xe7,0x69,0xff,0x97,0x6c,0xf0,0xcd,0x39,0x49,0x4a,0xcd,0xe1, -0x3e,0x98,0xac,0x3e,0x97,0xfb,0x12,0x38,0xaf,0xbe,0x41,0x71, -0x2a,0xce,0x82,0x22,0x50,0x0b,0xd1,0x9f,0x45,0xc9,0xac,0xd5, -0x7c,0xb1,0x06,0xc2,0xe9,0x66,0xb5,0xbe,0x98,0xaf,0x56,0xf3, -0xdb,0xf5,0x74,0x7c,0xb5,0x22,0x27,0xa8,0xa3,0x49,0x05,0x77, -0xe2,0xe7,0xea,0x23,0x5c,0xac,0x90,0x71,0x86,0x84,0x05,0x5d, -0x09,0x69,0x6f,0x67,0xef,0xb2,0x3f,0x1e,0x0e,0xb2,0x2e,0x09, -0x91,0xdd,0x0e,0x97,0x40,0x78,0x41,0x1e,0x0e,0xa0,0xcc,0xd8, -0xdd,0x2d,0x9a,0x30,0x5e,0x95,0x35,0x5f,0xbc,0x44,0x92,0x41, -0x32,0xfa,0x77,0xee,0x7e,0x93,0xed,0xaa,0x2c,0x1c,0x44,0x41, -0x05,0x8b,0x4f,0x0f,0x97,0xe8,0xed,0x2c,0x3e,0x65,0xfd,0xe1, -0xde,0x3f,0xfe,0x75,0xd0,0x71,0x45,0x97,0xe3,0xe9,0xab,0xbb, -0xc8,0x81,0x33,0xf9,0xc1,0xcd,0xca,0xa8,0xbc,0x94,0xc0,0xa9, -0x90,0xce,0x95,0xf7,0x3f,0xbc,0x7e,0xf2,0x5f,0x40,0x33,0x4f, -0xe7,0x97,0xbf,0x02,0x15,0x7c,0x59,0x55,0xa7,0x37,0xf3,0x8f, -0xe5,0xbd,0x0e,0x11,0x50,0xce,0x17,0xd5,0x7c,0x0a,0xd3,0x99, -0xe4,0x1f,0x26,0xd5,0xe4,0x62,0x02,0xdd,0xfa,0x5c,0x24,0xec, -0x30,0x06,0x1d,0xc5,0x10,0x9c,0x00,0xc2,0xab,0x39,0x0c,0xc9, -0xf4,0x6c,0x39,0x9c,0x55,0xb0,0xf4,0x6f,0xcb,0x7b,0x98,0x5d, -0x98,0xf7,0x53,0x9c,0x8d,0xd9,0x75,0x71,0x90,0x5f,0xc1,0x71, -0xf6,0xf3,0x18,0x07,0xbd,0xf8,0xee,0xe0,0x80,0xca,0xf0,0x8a, -0x2c,0xfb,0xc9,0xd9,0x7c,0x01,0xcd,0x7c,0x8b,0x89,0xf0,0xfb, -0x03,0xcd,0x09,0x04,0x5e,0xc0,0xac,0xa0,0x6c,0x56,0x55,0xbd, -0x59,0xc2,0x9a,0xfb,0x84,0x88,0x5f,0xf2,0x33,0x39,0x57,0x81, -0xd4,0xd7,0xf0,0xef,0xe5,0xfc,0x1f,0xf0,0xf7,0xb6,0x4a,0x06, -0x0d,0xfa,0x81,0x50,0x2e,0xb2,0x2f,0xf9,0xf5,0x6f,0x3f,0xcc, -0x76,0x36,0x65,0x70,0x55,0xc2,0xc5,0x9b,0x57,0xbc,0x1e,0x6f, -0x51,0x2a,0x8e,0x9e,0x81,0x8e,0xac,0x1c,0x0e,0x5f,0x15,0x6c, -0x4e,0x9c,0x0e,0x45,0xce,0x5e,0xba,0x85,0x2c,0xa7,0x3c,0xaa, -0xbe,0x91,0x79,0xf4,0x9a,0x3f,0x76,0x00,0x4c,0x4c,0x07,0x38, -0xac,0x9c,0x32,0x29,0x74,0xc4,0x37,0x08,0xd1,0xdb,0x9d,0x43, -0x69,0x87,0xd1,0xc7,0x76,0x2e,0xae,0xa8,0xc1,0x3b,0x0d,0x72, -0x4c,0x54,0xa4,0xd6,0xb3,0xa2,0xa9,0x12,0x7c,0xb2,0x70,0x59, -0x6a,0x4f,0x16,0x8f,0x0f,0x51,0xc0,0x1e,0x96,0x4c,0xc3,0xd3, -0x39,0x46,0xff,0x38,0x19,0xc9,0xf3,0x1c,0x31,0xde,0x61,0xd9, -0x34,0x3d,0xf1,0x07,0x39,0xd1,0x61,0xe0,0xfc,0xfa,0x7a,0x6a, -0x9f,0xc0,0x57,0x64,0x6d,0x1c,0x07,0x1e,0x5d,0x7d,0x96,0xce, -0xe1,0xa7,0x97,0x0b,0xc7,0x1c,0x0f,0xc8,0x99,0x64,0xf7,0x29, -0x96,0xe9,0x51,0xbe,0x62,0x52,0xfd,0x48,0xeb,0x97,0xeb,0x73, -0x8a,0x71,0xfc,0x06,0x81,0x8d,0x49,0x75,0x2c,0x24,0x0e,0xdb, -0x9d,0xe2,0xab,0x43,0x03,0x5f,0xa4,0xaa,0x7e,0x9c,0xcf,0x7f, -0xad,0x8a,0x7b,0x39,0x6e,0x8a,0xfb,0x9a,0x49,0x9c,0x1c,0xb5, -0xa7,0xef,0xd0,0x62,0x2f,0xde,0x0d,0xfe,0x43,0xa5,0xe6,0xf9, -0x48,0x13,0xea,0x4c,0xc0,0x78,0xef,0xa6,0x09,0x59,0x0d,0x5d, -0xf5,0x92,0xc3,0x04,0xa5,0xd2,0x37,0x68,0x97,0x0d,0xb7,0xd9, -0xdd,0xed,0x05,0x5a,0xce,0xbd,0x84,0xed,0x79,0x3b,0x3b,0x99, -0xdf,0xcd,0x56,0xa2,0x5f,0x3f,0x7d,0x2d,0x2d,0xd9,0x09,0xf6, -0x1a,0x7c,0xe1,0x8b,0xe2,0x8f,0xee,0x6b,0xee,0xb3,0xcd,0x97, -0x70,0x8a,0xce,0x48,0xdd,0xf6,0xe3,0x64,0x34,0xff,0x48,0xa1, -0x7f,0x90,0x56,0x3f,0x85,0xe6,0xf3,0x5b,0x32,0x98,0x4d,0x1b, -0x70,0xbe,0x80,0xde,0x26,0x57,0xd3,0xf9,0x70,0x95,0xc4,0x02, -0x8b,0x90,0xe1,0x19,0x26,0xf4,0x12,0x0d,0x25,0x45,0x42,0xcb, -0x8d,0x3f,0x36,0xbc,0x5c,0x8b,0x6d,0xdb,0x2b,0x87,0x71,0x5d, -0x0e,0x69,0x9c,0x18,0x87,0xfe,0xb6,0xce,0xd5,0xf8,0x73,0x3d, -0x8a,0x3e,0x09,0xb2,0x1b,0x55,0xbe,0x1a,0x6f,0x70,0x6e,0x88, -0x9f,0x47,0xd2,0xbf,0x0e,0xbd,0xb8,0x1d,0x4f,0x89,0x72,0xa2, -0x25,0xce,0x6d,0x2a,0x3d,0x10,0x32,0x04,0x6f,0xf3,0x4b,0xb7, -0xfb,0x0a,0x08,0xad,0xf2,0x6d,0x4d,0x2b,0xe1,0x7a,0x1e,0xcd, -0x97,0x18,0x4b,0xa8,0x3d,0x81,0x74,0x6d,0x00,0xda,0x84,0x1a, -0x65,0x60,0xd3,0x02,0x62,0x06,0xa3,0x17,0xaf,0xd0,0x68,0x07, -0x36,0xdf,0xfa,0xa8,0x4c,0xf0,0xda,0xed,0x04,0x16,0x1e,0x20, -0xbc,0xfa,0x49,0x5a,0x09,0x14,0x26,0x86,0x9f,0x3c,0x66,0xca, -0x00,0x67,0xc1,0xb1,0x00,0x39,0x0b,0x6a,0x25,0x37,0xe2,0x48, -0xbc,0x42,0x06,0xef,0x46,0xb9,0x7f,0x4c,0x14,0x03,0x9c,0xac, -0xf6,0xc0,0xf7,0x8f,0x7d,0xce,0x45,0x6c,0x89,0xdb,0x8a,0x39, -0x90,0xc6,0x3c,0xcc,0xbe,0xc1,0xd0,0xa3,0x41,0x67,0x31,0x5c, -0x56,0xbc,0x14,0xd2,0xc6,0xd3,0x26,0xa3,0x69,0x2b,0x93,0x19, -0xad,0x6e,0xf2,0x30,0xc2,0xd2,0x03,0xf2,0x76,0xa5,0x09,0xae, -0x11,0xe8,0x26,0xea,0x95,0x7b,0x47,0x4e,0x35,0x5d,0xec,0xbf, -0xda,0xe1,0xe4,0x0d,0x13,0x4c,0x21,0x15,0xeb,0x94,0xc9,0x02, -0x1d,0x5f,0xc5,0x2b,0x19,0x4e,0x98,0x25,0x71,0xa9,0xe8,0xbc, -0x46,0x64,0x79,0xc7,0x35,0xe3,0x40,0x8c,0x3a,0x7a,0x7b,0xda, -0xf8,0xbc,0x81,0x92,0xac,0xb3,0x51,0x42,0x3a,0x2d,0x7e,0x34, -0x4b,0x98,0x93,0x9b,0xf1,0x72,0x82,0x98,0xbc,0xce,0x54,0x15, -0xce,0x14,0x8f,0x16,0xcf,0x55,0xa5,0x73,0x65,0xf7,0x43,0x16, -0x4c,0x3b,0x7b,0x43,0xb1,0x55,0x50,0x5e,0xfb,0x10,0x4d,0x5b, -0xb5,0x69,0x97,0x11,0x3c,0xbd,0x33,0xf8,0xdd,0xf6,0x8e,0xea, -0xfa,0xba,0xad,0xe2,0x0c,0x2a,0xff,0xb7,0xed,0x0e,0xbf,0xeb, -0xfe,0x1b,0xb6,0xc8,0x96,0x9d,0x80,0x9e,0x40,0xe2,0x9d,0x70, -0xa0,0x43,0x8b,0x9d,0xb7,0xc3,0x2b,0xf9,0xed,0xa1,0x4c,0x23, -0x27,0x63,0x86,0x76,0xf0,0x09,0xc3,0xc1,0x35,0x48,0x1e,0x41, -0x30,0x15,0x91,0xfb,0x3a,0xee,0xa3,0xa0,0x6a,0x09,0xe2,0xe8, -0x24,0xa7,0xc3,0x9d,0xda,0xb1,0x5e,0xd3,0x4f,0x0f,0x96,0xfb, -0x6d,0x69,0x76,0x09,0x94,0xcf,0xb8,0xa5,0x64,0xf6,0xca,0xf0, -0x6f,0x61,0x3d,0xc3,0xaa,0xba,0xc4,0x02,0x68,0x4b,0x1e,0xb9, -0x5c,0xe8,0xa1,0x24,0xc3,0x3f,0x70,0xcc,0x7e,0xb4,0x92,0x91, -0xd4,0x0b,0x91,0x10,0x72,0xaf,0x71,0xfc,0x8c,0xed,0xce,0x4b, -0x7e,0xf9,0x98,0x8e,0x00,0x9b,0x21,0x6a,0x5c,0xbb,0x25,0xc5, -0x32,0x48,0x92,0xd5,0xe6,0xa7,0x8c,0x23,0xf2,0x38,0xa2,0x94, -0x32,0x72,0x9c,0xd0,0x1d,0xa7,0x4f,0x80,0xfc,0x18,0x41,0x0d, -0xc2,0xfa,0xd1,0x24,0x6e,0xd6,0x58,0x5f,0x1d,0xa8,0x36,0xc0, -0x7a,0xa0,0x80,0x1b,0x59,0xac,0xca,0xc1,0xd4,0x9e,0xc8,0xd5, -0x4a,0x3b,0xb6,0x97,0x3a,0x5c,0x2b,0x76,0x09,0x22,0xe5,0xb7, -0x14,0x94,0x39,0x87,0x63,0x07,0xf5,0xf4,0x68,0x19,0x34,0x78, -0xcc,0xc9,0xdf,0x85,0x17,0x39,0xdc,0x9b,0xab,0x9b,0xfc,0x76, -0x32,0xfb,0x99,0x03,0xc3,0x4f,0x1c,0xd0,0x5c,0xa5,0xcb,0x4f, -0xbc,0xe6,0x00,0x09,0xa4,0x01,0x92,0xd4,0x9e,0x06,0xb0,0x5d, -0x6a,0x85,0x94,0xdc,0x74,0xa8,0xc2,0x9c,0x66,0xe0,0xc1,0xf0, -0x0f,0xe5,0xf5,0x6b,0x4c,0xed,0xac,0x48,0x01,0x34,0x1d,0x80, -0x87,0xd8,0x12,0x9d,0x24,0x7d,0x2d,0xa7,0x02,0xf6,0xb2,0x51, -0xb5,0x8b,0x30,0xc7,0xcc,0xd0,0x2e,0xcc,0x5d,0xc0,0xe7,0xa3, -0x76,0x5b,0x88,0xa2,0x50,0x4b,0x8e,0x46,0xa8,0x24,0x08,0xdd, -0x70,0xb4,0x24,0xb2,0x36,0x78,0x1a,0xaf,0x63,0x19,0x66,0x8b, -0x52,0x2d,0x64,0xec,0x62,0x30,0xaa,0x52,0xa1,0xcd,0xf2,0xb1, -0x09,0xa4,0x6b,0x42,0x04,0x5b,0x03,0xac,0x57,0xb8,0xf1,0x8e, -0x89,0x62,0x17,0x82,0x5d,0x58,0x30,0x28,0xbb,0x42,0xf3,0x0b, -0x9d,0xfe,0xd2,0x1a,0x64,0x8e,0x9f,0x29,0xf3,0x3b,0x96,0x1c, -0xd2,0xaa,0xf9,0xb2,0x82,0xff,0x91,0x3e,0xfa,0xe7,0x57,0xda, -0x57,0x2f,0x24,0x35,0xa3,0x8f,0x9e,0x9f,0xb8,0x7f,0x66,0x7f, -0x0a,0x42,0x60,0x62,0xb6,0x2c,0x8e,0x1d,0x47,0x81,0x87,0xeb, -0x03,0xbb,0x23,0x93,0x28,0x3d,0xe3,0x26,0x90,0xce,0xe8,0x2d, -0x5f,0xc3,0xd2,0xd7,0x72,0x89,0x2e,0xc4,0x2a,0xcd,0x87,0x51, -0x58,0x3d,0x47,0x94,0xb5,0x31,0xa5,0x68,0x41,0xf3,0x38,0x4b, -0x82,0x88,0xf1,0xe9,0xe4,0x1f,0x63,0x35,0x4b,0x07,0xe8,0xf5, -0xf8,0x96,0x10,0xec,0xdc,0xf5,0xa2,0xbb,0x00,0x4a,0x73,0x8a, -0xc0,0x3b,0x88,0x22,0xd8,0xf2,0xcd,0x15,0x73,0x44,0x26,0xe7, -0x3a,0x21,0xed,0xc3,0xbb,0xd5,0x9c,0xf1,0xf6,0xf0,0x6d,0xbb, -0x9f,0xdc,0x8c,0x85,0xca,0xa5,0x85,0x08,0xd4,0xad,0x97,0xaa, -0x11,0x97,0x5c,0x8d,0xd7,0x5f,0xf9,0x00,0x7d,0xa1,0xd8,0x73, -0xb4,0xef,0x7b,0xfa,0xfc,0xd1,0x9d,0x5f,0x5d,0x01,0x7e,0x41, -0x2b,0x19,0xc6,0xcf,0xb0,0x38,0x78,0x26,0x62,0x94,0x2c,0x91, -0x1c,0x89,0x23,0x92,0xba,0x98,0x59,0x2a,0x65,0x66,0x41,0x5e, -0xa7,0xe7,0xae,0xa5,0x8a,0xd7,0x4b,0x26,0x36,0x62,0xf4,0x03, -0xd5,0x19,0xbe,0x98,0xc7,0x39,0x42,0xae,0x6a,0xdd,0x0d,0x28, -0x05,0x92,0x3d,0xe2,0x1d,0x46,0x31,0xf4,0xd8,0x65,0xd7,0xbc, -0x73,0xe8,0x3f,0x16,0xa7,0x8c,0x1f,0xc6,0x8c,0x0a,0xd6,0x80, -0xf5,0x86,0x77,0xf4,0xc6,0xf4,0x60,0xc3,0x62,0x3c,0xf1,0x62, -0xfe,0x09,0xd6,0x11,0x20,0xc6,0x68,0xff,0x70,0x8e,0x5e,0xb2, -0xf6,0x20,0xca,0xbc,0x2e,0xf9,0xd1,0x74,0x59,0x13,0x43,0xb7, -0xeb,0x6f,0x71,0x40,0x4c,0xaf,0x18,0xbc,0x10,0x69,0x01,0x46, -0x45,0xab,0x41,0x53,0x1e,0x5a,0x0f,0xde,0x0e,0xa4,0x64,0xe6, -0x69,0x4e,0xfd,0x85,0x50,0xdb,0x2a,0xbd,0xfa,0xe6,0x11,0x25, -0x65,0x7f,0x1c,0x48,0x0c,0x3b,0xf0,0xe8,0x75,0x0f,0x0e,0xbf, -0x31,0x08,0x0b,0x33,0xa1,0xba,0xbb,0x87,0x59,0x27,0x49,0x0a, -0xb7,0x04,0x91,0x7a,0x4d,0x92,0xad,0xb3,0x69,0xe6,0xd1,0x9c, -0x3b,0xb9,0x6d,0x47,0x7d,0x5b,0x2b,0x05,0x5b,0xd6,0x70,0x22, -0x86,0x09,0xbb,0x0f,0xb9,0x83,0xa9,0x66,0x4b,0x3a,0x87,0x07, -0x07,0xdf,0x30,0xd2,0x9f,0x64,0xd8,0x1e,0x31,0xdf,0x53,0x86, -0x87,0x76,0x43,0xef,0xd7,0xeb,0x2a,0xf8,0x4a,0x92,0x23,0x8e, -0x40,0xd2,0xb8,0x3c,0xcc,0xb9,0xc6,0xc7,0xe5,0x21,0x26,0x95, -0x42,0x27,0xa0,0x85,0x4a,0xfb,0xcc,0xb8,0x9c,0xdc,0xa6,0xe2, -0x94,0xde,0xfb,0x56,0xc4,0x16,0x92,0x78,0x8c,0x1c,0xa6,0x31, -0x1f,0x15,0x2d,0x50,0x37,0xc5,0xa7,0x09,0x83,0x4a,0xe4,0xb4, -0x11,0xca,0x24,0xec,0xc9,0x4e,0x43,0x57,0x1c,0x79,0x22,0xdf, -0x25,0xb7,0x81,0x97,0x86,0x64,0xe9,0x35,0x37,0x53,0x06,0x32, -0x2b,0x38,0x19,0x8d,0x40,0x75,0x34,0xce,0xaf,0xdd,0x06,0x5d, -0x1b,0x5d,0xcd,0x40,0x2c,0x4e,0x90,0x87,0xfa,0x92,0xd0,0x03, -0x62,0xf3,0x35,0xac,0xec,0x5b,0x9f,0xfa,0x55,0xab,0xdb,0x2d, -0xb2,0xda,0x19,0x75,0xef,0x18,0x94,0x2c,0x4e,0xbd,0xe7,0xf8, -0x94,0xcc,0x45,0xee,0xf3,0xc6,0x34,0x15,0x26,0x03,0x77,0xf2, -0x6c,0xea,0xaf,0x88,0x74,0x21,0x78,0x0f,0xb5,0x9e,0xd3,0xb8, -0x88,0xe3,0xe4,0x90,0x5f,0x11,0x3f,0x73,0xca,0xec,0x4b,0x73, -0xc0,0x2f,0x60,0x47,0xd6,0x0f,0x78,0x8c,0x7d,0xf0,0x80,0xaf, -0x77,0xd9,0xed,0xe5,0x80,0x60,0x21,0xf0,0xf1,0xed,0xeb,0x80, -0x28,0xc3,0x8b,0x8e,0x47,0xd7,0xf4,0x34,0xe3,0xfa,0xe9,0xae, -0x73,0x1b,0xd7,0x8f,0xc6,0x26,0xb3,0xdc,0xaf,0xc5,0xd2,0x77, -0xd5,0x58,0xe7,0xf3,0xd2,0xd4,0x36,0xb6,0xcb,0xfc,0xe2,0x87, -0x50,0x20,0x73,0x33,0x7d,0x5f,0x1e,0xc8,0xc1,0xc4,0x91,0x7c, -0x0e,0x43,0x6c,0x5d,0x12,0x5d,0xd7,0x13,0xf3,0xf3,0x5e,0x53, -0x76,0xa4,0xff,0x90,0x5b,0x8f,0x9b,0xc2,0x50,0x97,0x96,0x49, -0xd4,0x95,0x65,0x61,0x49,0xc9,0xda,0x8d,0xb7,0xc9,0x9b,0x7a, -0x42,0x5c,0x70,0x6b,0x2a,0xd2,0x76,0x65,0x67,0x7b,0xdf,0x55, -0x29,0x2b,0x14,0x70,0xe3,0x85,0x87,0x07,0xd1,0x62,0x38,0xc2, -0x17,0x6d,0x0c,0xf2,0x2d,0x52,0x24,0x34,0x18,0x56,0x36,0x6d, -0x41,0xbc,0xef,0xbc,0xba,0xbb,0x82,0x9f,0xa6,0xd5,0x83,0xc9, -0x1d,0x4e,0x86,0x65,0xc4,0x2f,0x6b,0x35,0x39,0x72,0x63,0x06, -0x34,0xe7,0x2c,0x63,0xa4,0xf9,0x50,0xe6,0x7a,0x55,0xd5,0x4d, -0x22,0x51,0xa9,0x1e,0x4b,0xed,0xd2,0xab,0x06,0x3a,0x0c,0xca, -0x0a,0x91,0x05,0x3f,0xfa,0x4e,0x05,0xed,0x05,0x92,0xb6,0xc2, -0x31,0xf5,0xfb,0x93,0x81,0x6b,0x12,0x55,0x41,0x92,0xf8,0x12, -0xda,0x7b,0xe4,0xc2,0x07,0x8e,0xac,0x53,0x50,0x9b,0x4d,0x1e, -0xd0,0x11,0x0c,0x39,0xab,0x1f,0x18,0x51,0xcf,0x91,0x85,0x52, -0xd6,0x6e,0xfb,0x4c,0x1e,0x94,0x96,0x8f,0x0e,0xca,0xfd,0x7f, -0x7d,0x74,0x80,0xb6,0x0b,0x2e,0x96,0x40,0x91,0x42,0xe6,0xfd, -0xf3,0xfe,0xf9,0x00,0x1f,0x7d,0x4e,0xde,0xbe,0x78,0x06,0x5f, -0xcb,0xde,0xf9,0x0c,0xd3,0xd9,0xc0,0x28,0xcc,0x22,0x72,0x19, -0x2b,0xd6,0xe4,0xe7,0xb8,0x35,0x1b,0x35,0x5e,0x93,0xe9,0xd2, -0x35,0x99,0x1f,0x5d,0xa3,0x69,0x53,0xd6,0xec,0x97,0x72,0x24, -0xf0,0x4b,0x85,0xd8,0x54,0x01,0xbf,0x19,0xaf,0xf1,0x15,0x9c, -0x8c,0x1a,0xfc,0x3a,0xfe,0x7c,0x3d,0x9e,0x65,0xfb,0x93,0x86, -0x27,0x8b,0x4a,0x15,0x33,0x1a,0xf8,0xe5,0x92,0x9b,0x94,0xe2, -0xd5,0x48,0x9f,0xe4,0xe6,0x67,0x07,0x5c,0xbc,0x61,0xd4,0x36, -0xd5,0xb6,0x48,0xbb,0x46,0x5f,0x6e,0x49,0x08,0x48,0xab,0x81, -0x43,0x41,0x74,0x24,0x35,0xc9,0x31,0xa7,0x5d,0xe6,0x9e,0x13, -0xa0,0x55,0xbb,0x58,0x9a,0xc2,0x92,0xba,0x9b,0x26,0xfb,0x88, -0x58,0x1b,0xf3,0x19,0xb1,0x29,0x18,0x0a,0x58,0xf8,0x6c,0x54, -0x7a,0x27,0xe0,0xc4,0x4f,0xaa,0xd4,0xba,0xff,0x47,0x22,0xc0, -0x8f,0x35,0x2f,0x13,0xa7,0x84,0xf8,0x8a,0xe9,0x8a,0x9d,0x68, -0x1a,0x25,0x17,0x6b,0x28,0xaa,0x56,0xa7,0x08,0x12,0xec,0x3c, -0x60,0xc8,0xc1,0x94,0x43,0xdd,0xcf,0x60,0xe4,0x26,0x62,0x32, -0x44,0x9c,0xe8,0x06,0xea,0xb8,0xe8,0x1f,0x36,0x8d,0x3c,0x91, -0x41,0x14,0xbb,0xff,0x8b,0xde,0x8b,0x90,0x7f,0xd3,0x33,0xb2, -0xc8,0xc8,0x97,0xb3,0xfb,0x57,0x27,0xef,0x9e,0x3c,0xca,0x30, -0x13,0xdc,0xf1,0xce,0x91,0xf1,0xe3,0x6f,0x6c,0x5c,0xca,0x79, -0x72,0xbe,0x3c,0x9f,0x25,0x78,0x37,0x17,0xbf,0xbf,0x08,0xcb, -0x89,0x1a,0x9c,0x94,0x16,0x9c,0x3f,0xf9,0x86,0xf9,0x0a,0xad, -0x59,0x60,0x78,0x28,0xca,0xc0,0x7a,0x42,0xa1,0x58,0x16,0x1c, -0x68,0x3e,0x2f,0x2c,0x75,0x8f,0xe8,0x79,0xdd,0x93,0xba,0x32, -0x03,0x9f,0x34,0x69,0x56,0x18,0xb6,0x6f,0x0f,0x10,0x49,0x46, -0xd3,0xab,0xbe,0xbe,0x3c,0x0d,0xca,0xf1,0xec,0x12,0x26,0xf9, -0xa7,0xb7,0xcf,0x91,0x79,0x03,0xe7,0xfd,0x6c,0x85,0xb5,0x00, -0xd6,0x09,0x78,0x5e,0x43,0x9a,0x28,0xe0,0x10,0x6f,0xdb,0x37, -0x3b,0x62,0xfc,0xd9,0x14,0x23,0x63,0x7d,0x3a,0x5e,0xad,0xd0, -0x84,0x94,0xbb,0xf2,0x6c,0x64,0xd7,0x8e,0x42,0x1e,0x4d,0xe8, -0x10,0x0d,0x8d,0x8b,0x99,0x3b,0x2f,0xfd,0x3d,0xa9,0xde,0x4c, -0x87,0x93,0x19,0x9b,0x5a,0x83,0x3c,0x99,0xbd,0x14,0x86,0x96, -0xa2,0x42,0xcb,0x6d,0x6e,0x33,0xb0,0x3a,0x8e,0x74,0x24,0xf3, -0xce,0x8c,0x78,0xd4,0x91,0x77,0x36,0xcc,0x48,0x50,0xef,0x0d, -0x4e,0x54,0xa5,0xf7,0xc5,0x50,0xce,0xc7,0x81,0x9d,0x2f,0x9c, -0x1e,0x4f,0x1a,0x75,0xdf,0xcf,0x27,0xb3,0x34,0x69,0x5b,0xf1, -0xed,0x47,0x07,0xe8,0x44,0xd5,0x5c,0x83,0xd8,0xb8,0xe4,0x62, -0x7a,0xb7,0x6c,0x91,0x87,0x05,0xfe,0x0b,0x95,0xd2,0xef,0xfc, -0x6e,0xd5,0x02,0xfc,0x7f,0x84,0xfe,0xf8,0xe0,0xd0,0x69,0x55, -0x97,0xcb,0xf9,0x74,0xda,0xba,0x9b,0x51,0xe4,0xe5,0x74,0x72, -0xf9,0x6b,0x6b,0x74,0x31,0xe5,0xc0,0xed,0xfc,0xae,0x1a,0x8f, -0xe6,0x1f,0x67,0x1c,0xba,0x5b,0xf0,0x2f,0xe2,0xb7,0x1c,0x82, -0xc0,0x52,0x42,0x00,0x97,0x02,0x63,0x14,0x1e,0xe0,0xe0,0x74, -0x3c,0x84,0x8c,0x97,0x37,0xc3,0xd9,0xf5,0x58,0x4c,0x42,0xb5, -0x78,0x8f,0xb7,0x60,0x05,0x10,0x5c,0xf8,0x25,0x0f,0xca,0x18, -0x00,0xf0,0xe4,0x0e,0xa4,0x25,0x12,0x48,0x70,0x2c,0xdd,0x25, -0xe6,0x2e,0xdb,0x4a,0x78,0xd7,0xa5,0xbe,0x49,0x90,0x05,0x2d, -0x63,0xc9,0xc0,0xd5,0xde,0x43,0x0f,0x58,0xd3,0x5f,0x9f,0x99, -0x49,0xc8,0x41,0x0b,0xb1,0x0c,0x37,0x5c,0xaf,0xd7,0xd7,0x63, -0x66,0x77,0x46,0x46,0xf5,0x6e,0xb0,0xdb,0xbe,0xb2,0xab,0xd9, -0x6b,0xf8,0x86,0x92,0xaf,0xef,0x56,0xd1,0xa9,0xed,0x06,0x44, -0x32,0x65,0x5d,0x3f,0x30,0x29,0x15,0x58,0xaf,0x25,0x65,0x43, -0xb7,0x1e,0x2e,0xd8,0x17,0xf3,0xcb,0x37,0x78,0xd5,0xe6,0xf2, -0x41,0x87,0x1c,0x7d,0xbc,0x03,0x7c,0xe9,0xd2,0xed,0x49,0xb4, -0x4f,0x9a,0x71,0xfc,0x92,0x16,0x2e,0xdc,0x88,0x3d,0xb8,0x1c, -0xc9,0xa7,0xc7,0xfe,0xbf,0x74,0xc9,0xd6,0x2f,0xdc,0x0f,0xfb, -0x69,0xbf,0xd7,0x1e,0x64,0xef,0xca,0xfe,0x2f,0xed,0xc1,0x37, -0x98,0x81,0xec,0x0f,0xd1,0x3d,0xd9,0xfd,0xa6,0x07,0x38,0x42, -0xeb,0x7c,0x35,0x40,0xc1,0x0b,0x3c,0x52,0xd0,0x16,0xcf,0xb2, -0xb7,0xbb,0x7f,0x7d,0x9b,0x2f,0xd1,0x01,0xc8,0xf4,0x0d,0x1a, -0x6f,0xbe,0x9c,0x4f,0xf9,0x86,0x1c,0x5e,0xc0,0x4c,0xaf,0x87, -0x8b,0x05,0xfe,0xdb,0x43,0xcd,0x0b,0xbc,0x58,0xbb,0x9d,0x3d, -0xba,0x10,0x2b,0x36,0x76,0x34,0xa5,0xab,0x76,0xfd,0x71,0x32, -0x42,0xc3,0xb2,0x05,0xd9,0x1c,0x9e,0xab,0xd4,0x1a,0x81,0xf9, -0xeb,0xd3,0xb3,0xf5,0x8f,0x4f,0x8f,0x9f,0x90,0x08,0xc7,0xc2, -0xd7,0x70,0xbe,0x7f,0xbe,0x0f,0x31,0x77,0x4b,0xaa,0xae,0x7f, -0xfe,0x11,0x20,0x0f,0x3a,0x45,0x86,0xa6,0x6e,0x20,0x09,0xdb, -0xb8,0xdf,0xfb,0x97,0x02,0x1a,0x09,0x31,0x45,0x8a,0xd6,0x8d, -0xd6,0xf0,0xdf,0x7e,0xfe,0x0e,0xd7,0xaf,0x31,0x73,0x88,0x9f, -0x39,0xed,0x27,0xb6,0x2d,0x0d,0x88,0xd3,0x0a,0xb9,0xfa,0x88, -0x8e,0xd2,0xd7,0x70,0x3a,0x65,0x6c,0x21,0xf9,0x66,0x3f,0xe9, -0xb2,0xe6,0x58,0x9a,0x7c,0x23,0xca,0x89,0x76,0xe8,0x4b,0xef, -0x07,0x05,0xe0,0xf9,0xd7,0x9b,0x20,0xcf,0x36,0x1b,0x2c,0xc3, -0x24,0x0b,0xe6,0x91,0x60,0xa0,0xe3,0xca,0xa0,0x74,0x2d,0xc7, -0xc6,0x2e,0x84,0x12,0xc7,0x83,0x5f,0xee,0x82,0x8c,0xa1,0xaf, -0x2b,0x72,0x5a,0x17,0x0e,0x80,0x31,0x04,0xb7,0x9c,0xb2,0x71, -0x9e,0xca,0x28,0xce,0x35,0xd9,0x55,0x86,0x8c,0xed,0x36,0x0d, -0xa6,0xbe,0x5a,0xd2,0x87,0x3c,0x07,0x10,0x7a,0xe1,0x36,0x13, -0x23,0x6a,0xc6,0x9c,0x1b,0x8c,0xee,0xac,0x1a,0xf3,0x8b,0xae, -0x37,0xc8,0x0c,0xd4,0x41,0x89,0x1d,0x70,0x6f,0x70,0x2d,0x8f, -0x97,0x40,0xda,0x63,0x24,0x23,0x8c,0x71,0x39,0xc8,0xc9,0x8e, -0x65,0x20,0x2d,0xc7,0x2f,0xd1,0xba,0xc4,0xb0,0x49,0x3d,0x40, -0xb8,0x99,0x39,0xc8,0xdd,0x05,0xc5,0xfd,0xcc,0x80,0xe4,0x92, -0x9e,0x96,0xd2,0x73,0xfe,0x31,0x0f,0x74,0x05,0xc7,0x00,0x21, -0xe2,0x64,0x8d,0x65,0x18,0x34,0x81,0xdf,0x57,0x93,0x37,0xaf, -0x4f,0xcf,0x12,0xd1,0x6f,0xf3,0x12,0xdd,0x35,0xed,0x1e,0x1c, -0x65,0x52,0xec,0x71,0xe2,0x5e,0xac,0xd7,0x83,0xc2,0xba,0xac, -0xe7,0x23,0x35,0xc2,0xa5,0x3d,0x42,0x15,0x11,0x37,0x3f,0x3a, -0x76,0x67,0xe2,0xc2,0x89,0xbf,0x4a,0x6f,0xc8,0xc0,0xab,0xdc, -0xc5,0xb6,0x4f,0x13,0xb6,0x2f,0xe4,0x34,0x58,0x3d,0x12,0x50, -0x8d,0x51,0x82,0x38,0x84,0x9d,0x85,0xe6,0xa8,0xd1,0x80,0x9e, -0x49,0x85,0x76,0x21,0x41,0x89,0x72,0x5e,0x6e,0xf4,0xda,0x6d, -0xd7,0xca,0xf7,0xbf,0xfd,0xfd,0xc7,0xb7,0xe8,0x02,0x73,0x75, -0x57,0x65,0xf7,0xd4,0x24,0xba,0x6d,0xdc,0x4b,0x95,0x02,0x83, -0xb5,0x48,0x79,0xbb,0x16,0xba,0x14,0xcc,0x29,0x05,0x9d,0x9e, -0xe5,0xac,0xf2,0x1e,0x52,0xe2,0x74,0x65,0xa3,0xf3,0x3e,0xa0, -0xc7,0x39,0x4c,0xa4,0x39,0x06,0x4f,0xa4,0x69,0xf2,0x49,0xee, -0xf9,0x35,0xd7,0x1d,0xc9,0x04,0xe9,0x17,0xea,0x1b,0x05,0x74, -0x3c,0xa1,0x83,0xe6,0xbe,0x60,0xbf,0x7c,0xe6,0x08,0x0f,0x0f, -0xee,0xb9,0x78,0x3a,0x85,0xf8,0x58,0xd9,0xa9,0x4f,0x8f,0x96, -0x79,0x02,0xe4,0x78,0xc4,0x2b,0xb8,0x1d,0xaf,0x6e,0xe6,0x23, -0xad,0xa5,0xcf,0x9f,0x83,0x70,0x0f,0xd2,0x1d,0xe3,0x86,0x8b, -0x5b,0xb5,0xd5,0x24,0x0d,0x66,0xce,0x74,0x0d,0xf2,0x33,0xb9, -0x2b,0xea,0xd6,0x36,0x41,0x24,0xf9,0x46,0xbf,0xae,0xf3,0xed, -0x2b,0x93,0x5b,0xe5,0xd7,0xa6,0x5b,0xa9,0x64,0x1d,0x00,0x68, -0x56,0x1a,0xc7,0x42,0xc1,0x37,0xca,0xd2,0xb0,0x33,0xa6,0xe2, -0x80,0x34,0xa4,0x5e,0xce,0x47,0xe8,0xe1,0x8b,0xbc,0xc1,0x8f, -0x57,0xc3,0x6b,0xfc,0xb5,0x68,0x57,0x41,0xf5,0x07,0x57,0x99, -0xd1,0x7d,0x9b,0x54,0x18,0x3b,0x2d,0xc2,0xab,0x86,0x51,0x78, -0x7b,0x04,0xf6,0x0f,0x07,0x99,0xd3,0x86,0x43,0x1b,0xf3,0x73, -0x6c,0x26,0x09,0xb9,0xc2,0xa7,0xe8,0xcb,0x39,0x7b,0xf4,0xbc, -0xeb,0xf0,0xc8,0x9a,0x70,0x95,0xfb,0x9f,0xf6,0x3e,0x7e,0xfc, -0xb8,0x87,0x4f,0xba,0x7b,0xd0,0x1c,0x46,0x3b,0x47,0x47,0x88, -0x9b,0x2c,0x91,0x0a,0xfd,0xe9,0xec,0xd9,0xde,0x9f,0x13,0x11, -0x90,0x45,0xa9,0x99,0x6f,0x92,0x42,0x2f,0x08,0xf6,0x83,0x43, -0x2e,0x6c,0xf6,0x17,0x88,0x0e,0x26,0x6c,0x9f,0x80,0x63,0x78, -0x6b,0xa3,0x93,0xb0,0xb0,0xbe,0xdb,0x69,0xde,0xa2,0x0c,0x9f, -0x30,0xfd,0x7d,0x45,0x32,0x7a,0x26,0x03,0xc6,0x48,0x0e,0x34, -0x0f,0x27,0x32,0xc4,0xc6,0xde,0xe8,0x3d,0xc2,0xc4,0xd2,0xfb, -0x5c,0x1d,0xd5,0xb4,0xcf,0x90,0xa8,0xf4,0xfe,0xc6,0x6d,0xb7, -0x67,0x64,0x9c,0x8d,0x8b,0x24,0x1a,0xf9,0xf7,0x97,0x2f,0x12, -0x69,0xbb,0xdd,0x85,0x5c,0x07,0x60,0x1c,0x78,0x2d,0x62,0x4f, -0xa9,0x11,0x49,0x21,0x8f,0xac,0xa7,0x74,0x1f,0xe4,0xd4,0xb9, -0x16,0x75,0x0e,0xc7,0x95,0x3f,0xb1,0x56,0x27,0x49,0x44,0xe7, -0xcb,0xbf,0x9f,0xbe,0x7e,0x25,0x89,0xd8,0xcf,0x20,0x0d,0xea, -0xdf,0xe4,0x57,0x40,0xa2,0xbd,0xe6,0x87,0x62,0x5e,0x08,0x3a, -0x49,0xe4,0x2b,0x60,0xe3,0x96,0xca,0xdd,0x22,0xb6,0x47,0x8f, -0x1c,0x68,0x5a,0x40,0x5e,0xfc,0x4c,0x22,0x7a,0xb4,0xf9,0x79, -0x2d,0x9a,0xa0,0x14,0x6b,0xc0,0xfc,0x33,0x0f,0xab,0x30,0x05, -0x1a,0x72,0x8a,0x19,0xfa,0x8c,0xdb,0xf5,0x46,0xd1,0x87,0x02, -0x90,0xf0,0xb3,0xb9,0xfb,0xac,0x5e,0x2f,0xcf,0x1c,0x2e,0x91, -0x7a,0x24,0x83,0x6f,0x7a,0x97,0xf4,0x60,0x29,0x8f,0x8c,0x70, -0xa9,0x22,0x38,0x26,0xf4,0x6d,0xfd,0x5e,0x63,0x5b,0x74,0x5d, -0xc8,0x01,0x3a,0x1b,0xe2,0xfe,0x3b,0xa5,0x0f,0x3a,0x58,0x75, -0x7e,0x2b,0x71,0x4f,0x55,0xa9,0x15,0x02,0x39,0x16,0x75,0x5b, -0xe7,0x84,0x5f,0xfb,0x8b,0xfa,0x56,0xf6,0xae,0x9c,0xcc,0x08, -0xab,0x8c,0x81,0x1f,0x3d,0x22,0x57,0xe1,0x43,0xe1,0x70,0xc3, -0x6f,0xf9,0x28,0xc7,0x77,0x3e,0xc0,0x03,0xcf,0xe0,0x07,0x6d, -0xad,0xa3,0x08,0xce,0x19,0x47,0xa5,0x36,0x29,0xf3,0x38,0x97, -0x3f,0x9d,0x5c,0xf5,0x3f,0x72,0x5b,0x79,0xc5,0x95,0xd2,0x72, -0xe4,0xcb,0xe7,0x7a,0x77,0x0c,0x47,0x9f,0x4f,0xa9,0x46,0x6e, -0x1f,0x60,0xed,0xdf,0x15,0x07,0xbe,0xb7,0xf4,0x92,0xa9,0x37, -0x25,0x8c,0xe1,0x8f,0x24,0x5a,0xfd,0x56,0x93,0x53,0xb9,0x6a, -0x7c,0x01,0x54,0x51,0x67,0x48,0xe5,0xa3,0x03,0xb8,0xc0,0xbf, -0x3d,0x38,0x78,0xcc,0x11,0xeb,0xf5,0xb7,0x07,0xe8,0x36,0x8c, -0xbf,0x7a,0x69,0xd5,0x9d,0x5c,0xe9,0xc9,0x86,0x7e,0xf5,0x25, -0x58,0x72,0xcb,0x60,0x89,0xbc,0x0d,0x3a,0x91,0x26,0x2f,0xe0, -0x28,0xdc,0xd3,0x12,0x80,0x2a,0xdc,0xfa,0xc2,0xb2,0xce,0xec, -0x61,0xd9,0x27,0x51,0xef,0x9f,0xde,0xbe,0x18,0x94,0x9a,0xd1, -0x17,0xd9,0x5e,0x09,0x9e,0xaf,0x8d,0xb0,0x31,0xa1,0x09,0x66, -0x96,0x3f,0x0a,0xba,0xe5,0x56,0x44,0x09,0xfb,0xd0,0x4c,0x7a, -0x32,0x9b,0xcb,0xc1,0x93,0x64,0xc5,0xb7,0x5f,0x5b,0x64,0x75, -0xeb,0xfa,0x5b,0x98,0x7c,0x7c,0x55,0xd3,0x11,0x93,0x56,0x6e, -0xf0,0x33,0x5b,0xd8,0x65,0xee,0xd2,0x8a,0xd2,0xf5,0x69,0xe2, -0xe9,0x36,0xa2,0xe5,0x6a,0x22,0x79,0xf9,0x9a,0x16,0x51,0x44, -0x06,0xb5,0x73,0x4e,0x5f,0x43,0x9e,0xea,0xbc,0xee,0xf8,0xc8, -0x4c,0x56,0xb0,0xf6,0x60,0xcc,0x08,0x84,0xae,0x01,0x97,0x5a, -0x1e,0xa0,0x32,0x3c,0xcf,0x82,0xc4,0x58,0xd4,0xa5,0x6b,0x60, -0xa4,0xf1,0x76,0x21,0xcf,0xc3,0x5a,0x5f,0x27,0x49,0x7c,0x6b, -0x7b,0xb0,0xfe,0xa1,0xc6,0xf1,0x08,0x91,0xa2,0xf9,0xf4,0x03, -0x5b,0x2b,0xd3,0x8b,0x56,0xfd,0xad,0xf4,0x75,0xab,0x9a,0xbe, -0x08,0xc2,0x54,0x18,0x00,0x88,0xad,0x36,0x97,0xb7,0x38,0x1a, -0x95,0xa6,0x5e,0x0e,0xc2,0xfe,0x9c,0xa0,0x46,0x8f,0x0f,0xea, -0xdc,0x60,0xd8,0x6c,0xd4,0xab,0xc9,0x72,0xcc,0x6a,0x2c,0x30, -0x36,0xaa,0x6b,0x0e,0x6b,0xc4,0xb9,0x75,0x10,0x12,0xd9,0x77, -0x31,0xc0,0xc5,0x0a,0x8b,0xa5,0x69,0xbb,0xcc,0x78,0x28,0xae, -0xa1,0x0d,0x54,0xcc,0xf3,0x89,0x76,0x13,0xeb,0xff,0xba,0x4e, -0x0e,0xb2,0xb0,0xb1,0xe9,0x03,0xad,0x8d,0x50,0x49,0x85,0x05, -0x20,0xf6,0xf6,0x9c,0x80,0x39,0x4e,0xaa,0x7b,0x75,0x60,0x65, -0x8f,0xa0,0x3c,0x61,0xa5,0xb0,0x4a,0x36,0x35,0xca,0x81,0x08, -0xa8,0x54,0x0e,0x70,0xa4,0x57,0x98,0x6e,0xf1,0xb8,0x99,0x26, -0xc9,0x2f,0x5a,0xbb,0x39,0x12,0x4b,0xfb,0xa8,0x60,0x9d,0xeb, -0x36,0x6e,0x3e,0x27,0x83,0xa3,0xd7,0x76,0xda,0x1f,0xb5,0x71, -0xc1,0xbc,0x8a,0x78,0x64,0x77,0x8b,0x14,0x90,0x34,0xbd,0x63, -0xf2,0x68,0x74,0x4b,0xe7,0x64,0x03,0xd6,0x72,0x5e,0x1f,0x48, -0x9f,0x4e,0xbe,0xbf,0x83,0xb2,0x46,0x7f,0x39,0x4e,0x61,0xde, -0x9a,0x7b,0xd3,0x8a,0x92,0x9d,0x50,0x3b,0x8d,0x75,0xae,0x2b, -0x5d,0x1b,0xae,0x4b,0x22,0xad,0xaf,0x12,0xcd,0x72,0x22,0x00, -0xab,0x34,0x41,0x26,0x49,0xeb,0x76,0x7c,0x3b,0x5f,0x7e,0x4e, -0x82,0xa5,0x5d,0x99,0xe5,0x4f,0x56,0x86,0x96,0x63,0x68,0x55, -0xb5,0x92,0x81,0x2a,0x6b,0x31,0x6c,0x03,0xfa,0x7e,0x93,0xf3, -0xc5,0x87,0x07,0xe0,0xf2,0xf8,0x02,0xaf,0xb3,0xe4,0x72,0x08, -0xb5,0x20,0x73,0x9b,0x37,0x56,0x79,0xef,0xaf,0x2b,0xb8,0xa3, -0x6a,0x87,0x77,0x61,0x59,0xac,0x7c,0x3d,0x93,0x65,0x6b,0xe4, -0x73,0x3e,0x92,0xe3,0x76,0x4c,0x24,0xf9,0x4e,0x34,0x7d,0x19, -0xfb,0x17,0x0a,0xe2,0x50,0xf6,0x8f,0x8a,0x97,0xca,0xcc,0x61, -0xf6,0x40,0xe3,0x92,0xc9,0x8e,0xb2,0x28,0xbe,0x4f,0x65,0xeb, -0x46,0x9c,0xd8,0x4f,0x59,0xff,0xd1,0x40,0x81,0x47,0xc5,0xa0, -0xe9,0x51,0x89,0x4d,0xc0,0x2e,0xa7,0x52,0xcc,0x30,0xa7,0xe0, -0x46,0x2c,0x8d,0x84,0x43,0x51,0x35,0x3c,0x72,0xb8,0x21,0xe8, -0x35,0x76,0x41,0x9c,0x48,0x54,0x38,0xaa,0x66,0x7e,0x9a,0x95, -0x45,0x48,0xe4,0x8a,0x04,0x5f,0x67,0xdb,0x3c,0x62,0xb2,0x22, -0x02,0xba,0x6c,0xc7,0x6c,0x0d,0x73,0xde,0x9f,0x32,0x47,0x71, -0x7b,0xd2,0x7a,0x4d,0x75,0x86,0x19,0xac,0x98,0xaf,0x92,0xb7, -0xc8,0x2c,0x44,0x4f,0xc8,0x2f,0x61,0xc7,0x12,0x61,0xe2,0x11, -0x5d,0x4b,0xfe,0x69,0x8b,0xaa,0xee,0xad,0x64,0xa4,0x13,0x45, -0xa1,0xf8,0x85,0xeb,0xcb,0xdf,0x0e,0xc5,0xe5,0x1d,0x52,0x33, -0xb8,0x90,0x30,0x02,0xd7,0xd3,0x63,0x5e,0x4d,0xe4,0x18,0x07, -0x92,0x90,0xe3,0x8c,0x49,0x1e,0x46,0x1f,0xa3,0x07,0x65,0x3f, -0x8e,0x41,0x2d,0x19,0x0e,0x89,0x0f,0x7e,0xbe,0x34,0x86,0xd3, -0x8f,0xc3,0xcf,0x15,0x82,0xef,0xdb,0x5b,0x64,0x10,0x99,0xed, -0x1b,0xe2,0xce,0x08,0xd5,0x41,0xe4,0x22,0xbc,0x67,0x1b,0x4c, -0xb3,0xe1,0x94,0x6e,0xcd,0x2a,0xb8,0x2e,0x79,0x47,0x39,0x50, -0x7a,0x8c,0xb5,0xdb,0x2e,0xd8,0x25,0xc8,0xa9,0x03,0x90,0xe5, -0x84,0x0e,0xa3,0x16,0x85,0x8b,0x61,0xc3,0x24,0x64,0x89,0x40, -0xef,0x0f,0xa0,0x12,0x6f,0x27,0x30,0xeb,0xd4,0x66,0xcf,0xd9, -0x28,0x6b,0x17,0x0d,0x20,0xea,0x7a,0x3d,0x0a,0x66,0xc1,0x5f, -0x58,0x8b,0x24,0x30,0x82,0xc1,0xe1,0xab,0x21,0x3a,0x55,0xe8, -0xe2,0xb9,0x9e,0x22,0xba,0x0e,0x9d,0xe8,0xd2,0x8f,0xa5,0x72, -0xf1,0xf2,0x37,0x5c,0x7a,0x64,0xca,0xe6,0x41,0x8c,0x72,0x3d, -0xf3,0x88,0xce,0xed,0x24,0xfb,0xfb,0x78,0x6c,0xd1,0x43,0x9b, -0xde,0x12,0x5d,0xa6,0xdb,0xd7,0x6b,0xfd,0x66,0x76,0x80,0x8f, -0xe7,0x98,0x9c,0x11,0x27,0x66,0x6c,0x5a,0x71,0x16,0x1f,0x0f, -0x08,0xf6,0x37,0x49,0xe4,0x30,0x9b,0xbd,0x08,0xa4,0xa4,0x84, -0xb6,0x04,0xbc,0xee,0x23,0xea,0x6b,0xb3,0x1e,0xfb,0x20,0x17, -0x87,0x34,0xdd,0xcb,0xe5,0x1c,0x28,0xee,0xf9,0x2d,0x50,0xc0, -0x70,0xf0,0x2f,0x22,0xc6,0x24,0x8d,0x40,0xc4,0x91,0xcc,0x83, -0x42,0xe5,0x4e,0xba,0x43,0xa5,0xf4,0x25,0xf9,0x70,0x00,0x9b, -0x3e,0xea,0x7c,0xbb,0xcd,0x69,0x8f,0x6a,0x69,0x8f,0x06,0x5a, -0x6b,0xff,0x5b,0x14,0x5c,0x4f,0x6e,0x56,0xab,0x45,0x81,0xc2, -0x03,0x0a,0xad,0xf7,0xe7,0x83,0xe2,0xbb,0xef,0xbe,0x45,0x21, -0xfc,0x90,0x77,0x10,0xe5,0x8f,0xaa,0x74,0xc5,0xa8,0xc1,0x23, -0x32,0x80,0x49,0x2e,0x0b,0x95,0xc3,0xe0,0xfd,0x5b,0x78,0x77, -0x55,0x92,0x4f,0xc6,0xb5,0x0c,0x5e,0x7a,0x05,0x79,0x0d,0x1f, -0x9c,0x32,0x34,0x33,0xb3,0x00,0x4c,0xe1,0x8b,0x54,0x24,0xe0, -0x48,0x2a,0x01,0xce,0x0b,0x37,0xf7,0xf7,0x83,0xb2,0x8a,0x30, -0xfe,0xc8,0x5c,0xfe,0x30,0x3d,0x7c,0x51,0x87,0x68,0xd0,0x81, -0x17,0x6a,0x62,0x94,0xa6,0xd3,0xf1,0xd2,0x20,0x4d,0x48,0x0d, -0xb2,0xdd,0xdc,0xd2,0xe3,0x1f,0x98,0xd3,0x9f,0x16,0x0b,0x67, -0xc0,0xa0,0x42,0xef,0xda,0xca,0xca,0xdf,0xf1,0x6c,0x7d,0xf1, -0xf1,0xc5,0x3e,0x79,0x1d,0x0e,0x53,0xd2,0xb2,0x08,0x0a,0xd1, -0x11,0x27,0xa3,0x17,0x66,0xeb,0xf0,0xb4,0xc9,0x63,0x86,0xc8, -0xc2,0x48,0x8e,0xac,0x97,0xb4,0x01,0xa1,0xec,0xa1,0x27,0x3b, -0x1e,0x5e,0x51,0x8a,0xe5,0x2f,0x5a,0x6a,0xa4,0x98,0x8c,0x8e, -0xb9,0x69,0x5e,0x70,0x77,0x2e,0xd5,0xa5,0xb2,0x07,0xa3,0x21, -0xbf,0x0f,0x01,0xe9,0x4a,0x76,0x0f,0xdf,0x95,0x49,0xc7,0x3f, -0xb1,0x74,0x3a,0x59,0xa1,0x39,0x3b,0x5f,0xd5,0xaa,0xa4,0x06, -0x00,0xdb,0x14,0x10,0x91,0x0f,0xd2,0x81,0x30,0x2f,0x7c,0x00, -0x45,0x57,0x5c,0x9a,0x3c,0xbf,0x72,0x74,0xe5,0xde,0xe9,0x04, -0x60,0x27,0xf9,0x83,0x90,0x3c,0xbb,0x2e,0x20,0x0a,0x1f,0xac, -0xe0,0x15,0x1c,0x74,0x7b,0xe4,0xf8,0x33,0x69,0x2e,0x0d,0xbd, -0x49,0xfd,0xe6,0xf0,0xb3,0x89,0x5f,0x86,0xe9,0xb6,0x83,0xc3, -0xef,0x0f,0x29,0x93,0x92,0x6d,0xaf,0x5f,0x40,0xed,0x61,0xb6, -0x24,0x0f,0x4b,0xe5,0xdb,0x0a,0x1d,0x13,0xab,0x2e,0xb1,0x07, -0x1e,0x59,0x74,0xaf,0x44,0x29,0x1c,0x5f,0xa0,0x6d,0xca,0xa0, -0xb7,0x35,0xa5,0x83,0xcf,0x3e,0xc8,0xe4,0x08,0xa2,0x7b,0x49, -0xde,0x82,0x19,0x15,0x26,0x60,0x27,0x39,0x6a,0xfd,0x56,0x1e, -0x74,0x0f,0x48,0x3c,0x32,0x2b,0x3c,0x30,0x28,0x3b,0xf0,0xbe, -0x6e,0x61,0x3d,0x2a,0x0b,0x66,0x4b,0xcb,0x27,0xb9,0xcb,0xa2, -0xa6,0x73,0xaa,0x2e,0x1b,0xc3,0x42,0xb6,0x35,0xad,0x5d,0xff, -0xc9,0x96,0xd8,0x62,0xfa,0x47,0x48,0x96,0xac,0xe4,0xe1,0xf6, -0xe7,0x43,0x70,0x40,0xc8,0xad,0x99,0x1d,0x79,0xa4,0x95,0x62, -0x12,0xeb,0x99,0x97,0x49,0xb0,0x43,0xa6,0x12,0xe1,0x57,0xef, -0xc6,0xe2,0x70,0xc3,0x3d,0x80,0x46,0xa6,0xae,0xa5,0x9e,0x9d, -0xf3,0xf0,0x69,0xe6,0xb9,0x5b,0xb5,0xd3,0x2c,0xbb,0xaf,0x71, -0x77,0x0e,0xbf,0x9a,0xd2,0xf4,0xdc,0x7d,0x4b,0xb7,0xc1,0x64, -0x20,0x03,0x18,0x27,0x5f,0xa8,0x23,0x7c,0x9d,0x09,0x38,0x51, -0x28,0x67,0xa4,0x1c,0x2a,0x2b,0xf6,0x68,0xc6,0x29,0x91,0xfc, -0xf8,0x94,0xee,0x00,0x65,0xfc,0x1c,0x58,0x49,0x3b,0x3d,0x46, -0x52,0x21,0x07,0x31,0x44,0xfe,0x08,0x29,0xc9,0xfc,0xfb,0x20, -0x99,0x7f,0x55,0x5c,0x2c,0x23,0xfb,0x6d,0xad,0xf1,0x11,0x61, -0x2e,0x7b,0x30,0xe0,0xd9,0x86,0x4d,0x6f,0x68,0x44,0xf2,0x6a, -0xde,0x72,0x43,0xe8,0x5f,0xc8,0xa8,0x89,0x84,0x48,0xb3,0x89, -0xb6,0x90,0x51,0xe8,0x1f,0xf3,0xc2,0x57,0x04,0x14,0x09,0x21, -0xda,0xd3,0x11,0xf3,0xee,0xe5,0xc0,0xd9,0xab,0x20,0xa0,0xc8, -0xc1,0x2d,0xb6,0x3f,0x51,0x6c,0x83,0x1b,0xbe,0x63,0x24,0xc4, -0x19,0xb6,0x4f,0x05,0x86,0xd2,0x74,0x4c,0x75,0xae,0x57,0xf8, -0xe6,0x9e,0xe9,0x9d,0xb7,0x02,0xbe,0xf8,0x96,0x78,0x74,0x9f, -0xd2,0x14,0xff,0x69,0xcf,0xa7,0x04,0xec,0x73,0xa9,0xad,0xd1, -0xff,0x4a,0xc4,0x02,0x27,0xfe,0xb5,0x80,0x30,0xd8,0xb9,0x3c, -0xc5,0x05,0xdd,0x37,0x46,0x3d,0x18,0xeb,0x84,0xbf,0x9b,0xa8, -0xdf,0x6e,0x4f,0xa4,0xce,0xa0,0x9f,0xc7,0x89,0xb3,0x7b,0x7f, -0x53,0x59,0x89,0x16,0x8d,0xdd,0x39,0x8c,0xd0,0x26,0x4a,0xe3, -0x57,0x48,0x7a,0x31,0xd1,0x9b,0x1e,0x73,0x86,0xf5,0xba,0xc5, -0xd3,0x5c,0x2f,0x1d,0x34,0x06,0xb0,0x88,0x72,0xf3,0x98,0xe2, -0x89,0xe4,0xdf,0xb0,0xf1,0xcb,0x59,0xd8,0x4f,0xf0,0x2b,0xc9, -0xc8,0x8e,0xf2,0x36,0x6d,0x1a,0x59,0xac,0xf7,0x55,0x60,0xc5, -0xfe,0x9d,0x59,0x46,0x5c,0xcf,0xd6,0x67,0x72,0x67,0x43,0x85, -0x03,0xbc,0xa1,0x89,0xe3,0xd8,0xe5,0x98,0x13,0x7e,0xa9,0xc1, -0xd1,0xe0,0x1c,0xfa,0x74,0x13,0x65,0x70,0x10,0xaa,0xe5,0xa5, -0x62,0x1f,0x1c,0x31,0x27,0x11,0x97,0xd2,0x7d,0xd1,0x01,0x44, -0x9b,0x93,0x25,0x54,0x4a,0xd3,0xee,0x49,0x45,0x07,0x66,0x76, -0x9f,0x4a,0x08,0xf9,0x88,0x5c,0xd0,0x9f,0x5b,0xeb,0xf5,0x3e, -0x42,0x1c,0x8f,0xd6,0x7a,0x68,0xee,0x0b,0x16,0x14,0xe7,0x24, -0x55,0xdb,0xaf,0x6d,0x06,0x5b,0x72,0xe1,0x64,0x6b,0x9f,0xbd, -0x16,0x15,0x58,0xdf,0xa9,0xc4,0xfc,0xa5,0x8c,0x33,0x01,0x71, -0x6d,0xd7,0x69,0x48,0x1f,0x1d,0x1c,0xc0,0xfe,0x17,0xfe,0x1c, -0x4a,0x1f,0xe2,0xdc,0x86,0xe6,0x7b,0xcd,0x82,0x08,0x3d,0xe1, -0x45,0xd4,0x9d,0x4e,0xa9,0x6b,0x18,0x77,0x2c,0xf5,0x27,0xce, -0x0e,0x69,0x4a,0x88,0x38,0xe7,0x7c,0x3a,0x72,0x1c,0x1a,0x94, -0x3a,0x5b,0xe2,0x91,0xb1,0x28,0xf7,0xd3,0x32,0x3b,0xef,0xa5, -0xbd,0xb2,0xbd,0xde,0xcd,0xd6,0xe7,0xbd,0xf3,0xde,0xfe,0x51, -0xfd,0x08,0xa1,0xbc,0x45,0xa2,0xdd,0xe0,0xf7,0xb4,0x85,0xc2, -0x2b,0x22,0x51,0x45,0xf7,0x34,0x6a,0xeb,0xec,0x2e,0xe6,0x8b, -0x34,0xf3,0x2c,0x3d,0x36,0xad,0x02,0x68,0x5b,0x88,0xb5,0x59, -0x82,0xb6,0xaf,0x80,0x06,0xb8,0x10,0xdd,0x8b,0xe8,0xd6,0xcd, -0x8e,0x8d,0xa2,0x37,0xb2,0x85,0xdd,0x78,0xce,0x48,0xa5,0x7b, -0x63,0xe2,0x7b,0x30,0x68,0xea,0x2b,0xd2,0xe9,0x84,0x13,0xe9, -0xe3,0x12,0x05,0x20,0x67,0x8e,0x85,0x77,0xc2,0x6a,0x7f,0xe3, -0x25,0x75,0x19,0xd5,0x0c,0x61,0x51,0x53,0x1d,0x3b,0x82,0xe9, -0xf2,0x40,0x2a,0xfa,0x0d,0xeb,0x1d,0x90,0x52,0xf8,0x4b,0xea, -0xfc,0x91,0x53,0x3e,0xc1,0xdf,0x76,0xd2,0x00,0xc7,0x62,0x6b, -0xa0,0x4e,0x38,0xe3,0x8b,0xaf,0xa6,0x24,0xcd,0x19,0x54,0x2a, -0xef,0xd4,0x09,0xfe,0x9a,0x8b,0x4b,0x1a,0x0c,0xf0,0x79,0x50, -0xca,0x1a,0x8a,0x95,0xda,0xfe,0x6b,0xc7,0x74,0xca,0x1a,0x44, -0x0f,0xa3,0x1c,0x59,0x2f,0x8a,0x48,0x11,0x31,0x0b,0x62,0xdc, -0xb0,0xf5,0xaa,0xbe,0x06,0x07,0xa5,0x09,0x7b,0x3a,0x80,0xca, -0x21,0x29,0x90,0x74,0x6c,0xb3,0x1c,0xc8,0x1d,0x4b,0x5a,0x34, -0xd1,0x2a,0x3a,0xfa,0x9e,0x50,0xa1,0x72,0x24,0xe8,0x18,0x40, -0x64,0x1c,0x57,0xee,0x9f,0xbe,0x9c,0x7d,0xfc,0xba,0x6a,0xc4, -0x05,0xbc,0x52,0x50,0xbc,0x16,0xfc,0x2a,0x46,0xa4,0x2d,0x18, -0xc4,0x4e,0xd2,0xfa,0x38,0xac,0xd0,0x11,0x21,0xad,0x2d,0x7a, -0x8a,0xaa,0x95,0x87,0xa1,0xdf,0x44,0x88,0x73,0xc9,0x57,0xb8, -0x5d,0x83,0x25,0xbf,0x08,0xf7,0x2d,0xf8,0x41,0xde,0x14,0x19, -0xb6,0x39,0xaa,0xcc,0xcb,0xb0,0x6c,0xf2,0x80,0x9b,0x64,0x0a, -0x35,0x02,0xb5,0xdb,0xa1,0x0a,0xd3,0x68,0x16,0xc2,0xc5,0x12, -0xef,0xb2,0x68,0x19,0x84,0x47,0x01,0x99,0x39,0xb3,0x33,0xd2, -0x30,0x48,0x8e,0x50,0x36,0xeb,0xcf,0x34,0x09,0xd6,0xbb,0xf9, -0x4a,0x9b,0xc6,0xb8,0x01,0xa8,0xed,0x94,0xbf,0xfe,0x37,0xc6, -0x88,0x98,0xaa,0x71,0xf0,0xa9,0xf9,0xe9,0x66,0xe9,0x9a,0x9d, -0xc3,0xc7,0x29,0xeb,0x53,0xc0,0xe1,0x0a,0x1f,0xcf,0x47,0xe5, -0x01,0xfe,0xbe,0x9e,0xfd,0x44,0x47,0x2f,0xe3,0xf8,0xf2,0x8e, -0x7f,0x4c,0xf4,0xfe,0xdf,0x59,0x74,0xd5,0x88,0xf0,0xf0,0xb9, -0xf3,0xeb,0xf8,0x33,0x11,0x01,0xf0,0x8b,0x44,0x8b,0xad,0x26, -0xb3,0x1f,0xc8,0xf1,0x1d,0x44,0x47,0xfa,0x51,0x93,0x6c,0x2d, -0x14,0x6a,0xac,0xb9,0x57,0x5b,0xce,0x3b,0xec,0x6c,0x86,0xe5, -0x3e,0xda,0x6d,0xb1,0x1c,0xb9,0x82,0xd3,0x78,0xb8,0x1c,0xc1, -0xf2,0xc0,0x43,0x9a,0x23,0x05,0x10,0x46,0x6d,0x8a,0x5a,0xbe, -0x60,0x38,0x9a,0x84,0x80,0xb1,0x4d,0x69,0xdd,0x5c,0xc4,0x7c, -0x59,0x95,0x3b,0x3b,0xb6,0x30,0x9c,0x5c,0x1f,0x27,0xab,0x9b, -0x93,0xe5,0x78,0x04,0xcb,0x74,0x02,0x34,0x48,0xc2,0x63,0x12, -0x8c,0x76,0xad,0x2e,0x67,0xe3,0x0e,0xea,0x8c,0x20,0xe6,0x21, -0xf8,0x46,0x1c,0x2d,0x42,0xcd,0x76,0x02,0xdc,0xac,0x6e,0xfc, -0x7d,0xae,0x8f,0xfa,0xba,0x6e,0x9b,0x51,0x2e,0xa1,0x2c,0x1d, -0x19,0xc7,0x65,0xd8,0xe6,0x58,0x3e,0xc1,0x76,0x95,0x32,0x30, -0x4c,0x78,0xde,0x55,0x68,0xe9,0xeb,0x76,0xdc,0x83,0xb8,0xee, -0x7c,0x31,0x9e,0xa5,0x8e,0x91,0xc8,0x8c,0x1b,0x42,0xc4,0x72, -0x9f,0x31,0x47,0xf7,0x00,0x55,0xf5,0x71,0xbe,0x1c,0x65,0xc5, -0x83,0x85,0xf0,0x98,0x83,0x0c,0x2c,0x9b,0x92,0x19,0x02,0xd9, -0x47,0x42,0x08,0x28,0xcb,0xd2,0x44,0xa1,0xff,0x76,0xcf,0x01, -0x6f,0xb7,0xa9,0x86,0x88,0x81,0xde,0x1c,0x6b,0x18,0xe7,0x11, -0x06,0xbd,0x5e,0x2b,0xbd,0x9d,0xfc,0x7d,0x4f,0x88,0xf1,0xf1, -0x68,0x8f,0x2c,0x4a,0x23,0x67,0xf0,0x81,0xd4,0x32,0xf9,0xfb, -0xcb,0x17,0x3f,0xae,0x56,0x0b,0x49,0x30,0xae,0x2b,0xa8,0x33, -0x4a,0xeb,0x63,0x83,0x1a,0x28,0x7d,0x43,0xe7,0x2b,0x59,0xb8, -0x5c,0x66,0xf7,0x1b,0xce,0x8e,0x62,0x78,0x11,0x2b,0x85,0x1c, -0x89,0xae,0xc9,0xa4,0x81,0x97,0xf0,0x6a,0xc2,0x4b,0x59,0x0b, -0x93,0xde,0xa4,0x6b,0xaf,0x7d,0x0d,0x72,0x21,0xd4,0x6a,0xb4, -0x69,0xe4,0x64,0xfa,0x3c,0x62,0x8b,0x8f,0xfd,0xd8,0x20,0x8b, -0xae,0x9a,0xac,0xe6,0x3d,0x98,0x97,0x11,0x94,0xa5,0xf1,0xaf, -0x63,0xae,0x4e,0x2e,0x79,0xbe,0xa8,0x9d,0x4b,0xed,0xb6,0x30, -0xec,0x82,0xe3,0x85,0x21,0xc2,0x49,0xa9,0x1d,0xfb,0x6e,0x27, -0x6e,0x0c,0xcf,0xb6,0xb2,0x32,0x90,0x62,0x76,0x77,0x8c,0x7b, -0x99,0xbb,0xab,0xa8,0x50,0xf3,0x80,0x50,0x52,0xe3,0x3b,0x54, -0x9a,0xe5,0x35,0x24,0x89,0x2b,0xf7,0xe2,0x52,0x46,0xc6,0xa4, -0x22,0xe3,0x9b,0x65,0x9c,0xc3,0x73,0x08,0x44,0x26,0xc0,0x37, -0x05,0x3f,0x3d,0x3f,0xc0,0x8a,0x1e,0x24,0x1b,0x2f,0x9f,0xa0, -0x87,0x21,0x32,0xf1,0xcd,0xb2,0xed,0x1d,0x3e,0x7a,0xf4,0xad, -0x13,0xc4,0xf0,0xc2,0x09,0x8f,0x0e,0xbe,0x03,0x9c,0x84,0xc3, -0x61,0xd3,0x7a,0x80,0xde,0x17,0xdf,0x1d,0x7c,0xb7,0x91,0x3a, -0x91,0xab,0x72,0x35,0xff,0x74,0x4c,0xe8,0xfe,0xd3,0x4f,0x48, -0x86,0xe3,0xc3,0xc4,0xbd,0xa7,0x0b,0x54,0xd6,0x73,0x8f,0x59, -0x30,0x0d,0x99,0x37,0x46,0xda,0xc6,0x65,0x97,0x81,0x6e,0x5a, -0x69,0xf1,0xe8,0x13,0x33,0x85,0xce,0x83,0x5e,0x7d,0xa1,0xf5, -0x0c,0x57,0xc6,0x91,0x8a,0x45,0xca,0xab,0xa2,0xec,0x74,0xe8, -0x92,0x6b,0x58,0x4a,0xa9,0x5d,0x44,0xb0,0x83,0xed,0x27,0xae, -0x09,0xa1,0x5c,0xf9,0x42,0xca,0xba,0xac,0xdc,0x90,0xc6,0x70, -0xe0,0xd6,0x6f,0x5a,0x8c,0xce,0x04,0x0a,0x25,0x37,0xad,0x73, -0xdf,0x54,0x47,0x58,0xa1,0x4b,0xf4,0x1a,0x81,0xe4,0x77,0x9b, -0xcb,0xb7,0x85,0x3a,0xba,0xfa,0xf4,0x6a,0xfe,0x91,0x9e,0xf6, -0xb1,0xc3,0xcb,0xab,0x4f,0x2b,0xaf,0xc8,0xc6,0x56,0xd6,0xd6, -0x68,0xef,0x6c,0x8d,0x06,0xce,0x48,0x64,0xfe,0xea,0xd3,0xac, -0x66,0x4c,0xb0,0x57,0x88,0x3d,0xc1,0x75,0x16,0x5b,0x24,0x64, -0x63,0x84,0xc6,0x90,0xe1,0xf2,0x6e,0x56,0xee,0xc3,0x61,0x75, -0x37,0x26,0xcd,0x3c,0x00,0x39,0x9c,0x4d,0x6e,0x09,0xeb,0xf7, -0x2c,0xbd,0xb2,0x2f,0x26,0x89,0x5d,0xd4,0x20,0x5f,0x7d,0x1c, -0x8f,0x67,0xf4,0x02,0x8d,0x32,0x92,0x7d,0xa3,0xf3,0x08,0x9b, -0xde,0xfa,0x5f,0x9a,0x8d,0xf2,0xbb,0xd9,0x64,0xc5,0x05,0x58, -0x73,0x4c,0x2c,0xca,0x62,0x84,0xcd,0x2f,0x1a,0x8d,0xdc,0x27, -0x6b,0xa0,0x4a,0xa4,0xf1,0x4a,0x02,0x81,0x1a,0xe5,0x29,0xbd, -0xe6,0x03,0x92,0xd3,0xe1,0x94,0xf5,0xfa,0x00,0x08,0xde,0xe1, -0x14,0x59,0x73,0xb7,0xc3,0x4f,0xcf,0xa1,0x85,0x43,0x16,0xae, -0x78,0x44,0x56,0xfe,0x08,0x30,0x1b,0x29,0x9b,0x8d,0xca,0x8e, -0x3e,0x2d,0x51,0xc3,0x4a,0xf3,0x96,0x54,0x33,0x3e,0x45,0x0a, -0xb6,0xa8,0xce,0xc4,0x26,0xa7,0xf0,0x2f,0x19,0xe2,0x9a,0x90, -0x45,0x8d,0x21,0x9e,0xc0,0xdc,0x10,0xa3,0x97,0xca,0xad,0x74, -0x6a,0xbd,0x38,0xc1,0xeb,0x35,0xd4,0xbb,0x5e,0x1f,0x1e,0x8d, -0xe6,0x2d,0x6e,0x27,0xfd,0x05,0x8a,0xa8,0xfb,0xc7,0x84,0xbb, -0xb2,0xcf,0x51,0x79,0x60,0xb7,0x25,0x06,0x45,0x39,0x3b,0x58, -0x7d,0x76,0xf4,0x11,0xc8,0x71,0xa4,0xd2,0xa1,0x10,0x34,0x89, -0x03,0x76,0x84,0xf6,0x45,0x84,0xb1,0xdd,0x46,0xa3,0xf2,0x94, -0xdc,0x6e,0xef,0xed,0x05,0xc3,0x93,0x6d,0xb8,0x00,0x8d,0x83, -0x9f,0xa5,0x2e,0xf7,0x89,0xfe,0x4a,0x0c,0x8e,0x9b,0x7b,0x5f, -0xe3,0x66,0xa4,0xfa,0x8d,0xe6,0xc2,0x10,0xf9,0x80,0x7f,0xce, -0x3b,0x04,0x16,0xda,0x0c,0x1c,0x8e,0x78,0xac,0x8b,0xaa,0x0c, -0x45,0x88,0x5d,0x7c,0x7e,0x2f,0x0b,0xaa,0x08,0x56,0x92,0xd5, -0x52,0x68,0x10,0xf0,0xc7,0x1c,0x81,0x7c,0x3f,0x15,0xa1,0xbf, -0x25,0x33,0xe4,0x0b,0xfe,0xa0,0xbf,0x46,0x85,0xe8,0x48,0x95, -0x6a,0x69,0x58,0x89,0xe6,0x45,0x6f,0x3f,0xec,0x8a,0x8d,0x33, -0x7b,0xfb,0x8e,0x28,0xe5,0x4f,0x59,0x8e,0xe8,0x6f,0xa7,0x93, -0x61,0x0e,0xce,0xd6,0xa7,0x28,0xbf,0x1d,0x44,0x25,0x3c,0xfc, -0x24,0x8d,0x8c,0x30,0x0a,0x86,0xbc,0xba,0x99,0x5c,0x99,0xe3, -0x6e,0xe3,0xb5,0x54,0xfc,0x18,0x38,0xce,0xaa,0x38,0xcf,0xc8, -0xee,0x25,0xd0,0x6b,0xd8,0xa6,0x75,0x98,0x45,0x53,0xae,0x80, -0xea,0xb1,0xbc,0x5a,0xda,0x90,0x25,0xfd,0xcd,0xe9,0x6f,0x97, -0x9e,0xa6,0x89,0xe7,0x78,0x0f,0x84,0x2a,0x9a,0x90,0x45,0x5d, -0x81,0x82,0xb3,0x4c,0x60,0xbd,0x6c,0xb1,0x58,0x45,0xa3,0x8a, -0x3b,0x7f,0x3c,0xac,0x50,0x56,0x89,0x56,0x2c,0x7b,0x5c,0x74, -0x46,0x7d,0xc5,0xe3,0xbe,0x8e,0x64,0x2e,0x56,0x20,0x31,0x7f, -0xc9,0x3f,0xe8,0x11,0xe9,0x23,0x59,0xdf,0x60,0x59,0xfc,0x50, -0x5e,0x8a,0x23,0x79,0xa5,0x8a,0x0a,0xe9,0x47,0x39,0x5e,0xe8, -0x80,0x60,0x78,0xb0,0x6c,0xb1,0x1d,0xf4,0xe1,0x56,0xf9,0x97, -0x77,0x3a,0x19,0x1c,0x88,0xf9,0x49,0x6c,0xe3,0xcc,0x8d,0xcc, -0x82,0x75,0x99,0x5d,0x37,0x9c,0x3e,0xb4,0x58,0x31,0x73,0x06, -0xcc,0x7a,0xde,0x94,0x19,0xfb,0x22,0x8e,0x40,0x38,0x8b,0xef, -0x3e,0xcb,0x26,0x87,0x83,0xd9,0x6c,0x85,0xf1,0xf2,0x12,0x30, -0x43,0x39,0x51,0x87,0x15,0xe2,0x60,0x5f,0xd9,0x18,0x8e,0x99, -0x57,0x25,0x15,0x2b,0xed,0x48,0x76,0x47,0x77,0x7c,0x0e,0x38, -0x63,0xc6,0x34,0xec,0x7d,0x33,0x13,0x03,0xad,0x3a,0x6f,0x2c, -0xf8,0x8d,0xa6,0x1e,0xe4,0x87,0xcd,0x39,0x60,0x03,0x5a,0x00, -0x38,0x47,0xa9,0xce,0xcc,0x9e,0x9f,0x41,0x38,0x40,0xb0,0x79, -0x1d,0x1f,0x13,0x82,0x03,0x54,0x7c,0x21,0x7e,0xda,0x6c,0x94, -0x77,0x4b,0xd5,0xf5,0x4b,0x6a,0x86,0x37,0x28,0x39,0x86,0x0a, -0x26,0xa2,0x72,0x13,0x51,0x7d,0x79,0x22,0x5c,0x16,0x11,0x27, -0x89,0xf7,0x43,0x17,0x17,0xbf,0xd9,0x1e,0x51,0x72,0x1e,0x01, -0x2e,0xef,0x9d,0xc3,0xf0,0xd0,0x58,0x04,0x1d,0x0a,0x6a,0xd2, -0xad,0x82,0xf4,0x50,0xed,0xd8,0x9f,0xff,0xfd,0x95,0x83,0x08, -0xc7,0x89,0x8f,0x57,0x5b,0x09,0x6c,0x32,0x3b,0x8e,0xb7,0xa5, -0x7a,0x29,0xd7,0xb0,0xe5,0xaa,0xf2,0x39,0x51,0x3a,0x25,0xe7, -0xbc,0x40,0x11,0x93,0x3d,0x24,0xb2,0x41,0x86,0x11,0x3d,0xfe, -0x29,0x0e,0xb2,0xa2,0xb9,0x6d,0x91,0xc1,0x18,0xe9,0x9f,0xea, -0xdb,0x7c,0xa2,0x59,0x0b,0x5a,0xb5,0x3d,0x49,0x0a,0x17,0xf5, -0xce,0xa6,0x5b,0x7a,0x1b,0xdb,0x30,0x34,0xc0,0x1a,0xac,0x10, -0x9a,0xd4,0xac,0xb7,0xed,0xda,0x35,0xc3,0xc2,0x41,0x58,0x5d, -0x1d,0x7f,0x69,0x6e,0x1b,0x86,0xd2,0x65,0xde,0xd8,0xc5,0x23, -0xcb,0x8c,0x75,0x6d,0xcf,0xe0,0xe4,0x6b,0x4e,0x21,0x8b,0x59, -0xf7,0x4d,0x03,0x69,0xba,0xec,0x8d,0xa9,0x9a,0xc8,0xc0,0x8f, -0xeb,0x97,0x9a,0x06,0x67,0x7f,0x1e,0x5b,0x3f,0x41,0x14,0x13, -0x70,0x43,0x44,0x32,0xe1,0x07,0xd1,0xcc,0x26,0x43,0x57,0xc4, -0x7f,0xa8,0xaa,0x67,0xb3,0x32,0xd2,0xbc,0x3d,0xda,0xaa,0x89, -0x5b,0x2d,0xc6,0x63,0x77,0x1f,0xd4,0x5e,0x15,0x45,0xb6,0x08, -0xf3,0xc0,0xa9,0x5f,0x37,0x17,0x8c,0x09,0x3d,0xaa,0xb1,0x59, -0x01,0x91,0x55,0x75,0xf9,0xb6,0x1b,0xa7,0xd7,0xe3,0xd9,0xb3, -0x4f,0x2c,0x0a,0x88,0xbe,0x97,0x9a,0xab,0x0e,0x55,0x79,0x55, -0xa5,0xe9,0x0a,0x88,0x95,0xb3,0x79,0x11,0x35,0x7b,0x35,0xdf, -0xda,0x72,0xf1,0x8d,0x4c,0x9c,0x7f,0xb5,0x53,0x9c,0xd1,0xe6, -0x72,0x96,0x81,0xf3,0x03,0xb5,0x52,0xcc,0x0e,0xa8,0x5d,0x3b, -0x9d,0x09,0xe2,0xd5,0x7c,0xb3,0xad,0x95,0x82,0x8f,0x8f,0x43, -0xbc,0x68,0x4b,0x6e,0xbe,0x1f,0xd0,0x91,0xa5,0x67,0x9a,0x3f, -0xc5,0x4f,0xd1,0x5c,0x67,0x33,0x33,0x70,0x82,0x42,0x01,0xc7, -0xbe,0x42,0x48,0x5b,0xa6,0x27,0x1f,0xcd,0x3d,0xe6,0x16,0xdd, -0x86,0xd8,0xae,0xd2,0xa5,0xf2,0x84,0x44,0xfa,0x61,0x9b,0xdc, -0x56,0x98,0x1d,0x19,0x68,0xa8,0x7d,0x38,0xa9,0x6e,0x2c,0x50, -0x04,0x08,0x1b,0x70,0xbe,0x48,0xc9,0xf6,0x74,0x4a,0xfd,0x70, -0xdb,0x96,0x1d,0xb0,0xb1,0x59,0x0b,0x2e,0x4b,0x26,0x98,0xc2, -0x42,0xce,0x3b,0x4f,0xbd,0x22,0x13,0x95,0x0b,0x64,0x6e,0x56, -0x97,0xa8,0x1f,0x92,0xa0,0x30,0x5e,0xdf,0x4d,0x76,0x59,0x5c, -0x94,0x2d,0xb5,0x65,0xec,0xe8,0x20,0x79,0x0b,0xad,0x08,0x65, -0x3a,0xc9,0x03,0xc3,0xf2,0x3f,0x29,0xef,0x35,0x5c,0x0d,0x4f, -0x67,0x23,0xe5,0xd8,0xcc,0x17,0x14,0xed,0xfb,0x4f,0x57,0x93, -0x4f,0x55,0xe3,0xb2,0x10,0x3c,0x12,0x86,0x89,0x8f,0xa1,0xaa, -0x52,0x85,0xa8,0xbd,0xae,0x09,0xa7,0x89,0xd5,0x5f,0xc9,0x57, -0x9a,0xc6,0xf8,0x20,0xe5,0x65,0x1b,0xbe,0x46,0x34,0xdd,0xa7, -0xc3,0xe9,0xa2,0xe2,0x3c,0x72,0xff,0xf2,0x38,0x88,0x3f,0x8a, -0xab,0x4f,0x49,0xde,0x1f,0x28,0xb2,0xd5,0xe0,0xec,0x7e,0x34, -0xe6,0xe1,0xdd,0x39,0x10,0x2c,0x5b,0x8e,0x6e,0x3e,0xb8,0xe0, -0x6f,0x27,0xf1,0xd4,0x67,0xc2,0x94,0xaf,0x97,0x58,0xa4,0xaf, -0xdc,0x4a,0xd6,0xf9,0x65,0xc0,0x6e,0x34,0xc9,0x71,0x25,0xc6, -0x09,0x22,0xde,0x6e,0x9b,0x0f,0x1a,0x2c,0x24,0xd4,0x64,0xb4, -0x53,0x93,0x66,0xcc,0x31,0x50,0x84,0x7a,0x7b,0xf8,0x12,0x30, -0xa4,0x98,0xf9,0x99,0x87,0xeb,0xde,0x0a,0xde,0x41,0x2e,0xb9, -0x1b,0xce,0xdb,0x30,0xc6,0x91,0x37,0x1e,0x8a,0x56,0xe8,0x78, -0x4c,0xef,0xb0,0x47,0x76,0x66,0xfc,0xb9,0x41,0x0a,0xb2,0xd1, -0x60,0x39,0x63,0xc9,0x69,0x98,0xe8,0x37,0x83,0xae,0x8d,0xdc, -0x8d,0xff,0xa1,0x8c,0x2d,0x11,0x41,0x97,0x63,0x6e,0x5a,0x8e, -0xfe,0xae,0x53,0xc9,0xb3,0x5e,0xef,0x68,0x31,0xc7,0xaa,0x96, -0x24,0xde,0x77,0x24,0xd2,0x87,0xae,0x5c,0x79,0x4b,0x6d,0x91, -0x5f,0x36,0xeb,0xc5,0x2a,0xac,0xe2,0x52,0x79,0x60,0x9d,0x70, -0x73,0xb6,0xcc,0x74,0xce,0x7d,0xa0,0xe1,0x35,0x6b,0x26,0x11, -0xfb,0xc1,0xb5,0x04,0x5e,0x4c,0x83,0xe6,0xd5,0x24,0x44,0x1e, -0x65,0xec,0xd1,0x5f,0x99,0x99,0xe2,0x80,0xe6,0x8c,0x9c,0x73, -0xc8,0xa1,0x01,0xab,0x56,0x0a,0x47,0xc3,0x40,0x8b,0x3e,0x40, -0x2f,0x81,0x5e,0x30,0x41,0x01,0x50,0x8f,0xf1,0x18,0x6b,0x96, -0xff,0xce,0xd5,0x21,0xfe,0xfa,0x1b,0x57,0x84,0xb7,0xf1,0xbd, -0x6d,0x45,0xe0,0x45,0xb8,0x6d,0x05,0xf8,0xa5,0x7a,0xd0,0x4c, -0xef,0x52,0x1d,0x6e,0x53,0xd8,0x2f,0xd7,0xd1,0x86,0x48,0xd3, -0x57,0x3d,0xc3,0xcc,0xe0,0x6e,0x6a,0x1a,0xd5,0xf7,0xd5,0x14, -0xf0,0x8d,0x27,0xf3,0x8f,0xb3,0x82,0x6f,0x70,0x46,0x43,0xe0, -0x02,0xc7,0xf8,0x9f,0x16,0x1a,0x4b,0x58,0x89,0xc4,0x9e,0xb1, -0xeb,0x01,0x49,0x11,0xfc,0x25,0xcb,0xf1,0x1a,0x7f,0x3e,0xf3, -0x36,0xfe,0x19,0xd2,0x86,0xe2,0x5f,0xdf,0xad,0x4c,0x02,0x01, -0xe3,0x04,0x81,0xe5,0xd3,0x04,0xdc,0x26,0x76,0x37,0xc9,0x7c, -0x87,0x07,0x7c,0x4d,0x3e,0x88,0xeb,0x04,0xa8,0x0a,0x73,0x2c, -0xbe,0x88,0xa1,0x50,0x86,0x2d,0x15,0x5c,0x09,0x01,0x01,0x57, -0x13,0x63,0x50,0x0d,0x66,0x0c,0x18,0x81,0xaa,0x5d,0xcf,0x14, -0x9d,0xa1,0x4f,0x02,0x11,0xf6,0xbb,0x42,0x77,0xc4,0x57,0xb3, -0x76,0x5b,0x29,0xf0,0x86,0xb7,0x77,0x2a,0x03,0xc7,0x1e,0xb5, -0x41,0xe9,0xbc,0xc2,0x36,0xa9,0xb0,0x10,0xf8,0xd7,0xda,0xd2, -0x71,0xa0,0x38,0x29,0xd3,0xbc,0xee,0xfa,0x86,0x9e,0x38,0x02, -0xb2,0xf4,0x04,0xc2,0xfc,0xea,0xaa,0x77,0x50,0x78,0x8b,0xf2, -0xd2,0x37,0x9b,0xbb,0x67,0x3f,0x0a,0xfb,0x81,0xe7,0xbb,0xa1, -0x35,0xb0,0xb5,0x55,0x2f,0x8e,0xe8,0xdb,0x12,0x83,0x22,0x4e, -0xf6,0xae,0xd9,0xc4,0xc8,0x3d,0x66,0x97,0xb3,0xd3,0x05,0xc9, -0xec,0x77,0xc6,0xaa,0x5f,0x12,0xc3,0x67,0x1f,0x7e,0xa3,0xb9, -0x6e,0xfc,0xd5,0x01,0xcf,0xed,0x47,0x59,0xb7,0xc3,0x68,0xdf, -0x9c,0xb9,0x38,0xbe,0x37,0x73,0xc8,0x1e,0x25,0xae,0xae,0xe6, -0x93,0xdb,0x25,0x67,0xa4,0xee,0x65,0xd0,0x7e,0x62,0xb7,0xdc, -0xa3,0xa9,0xc5,0xa1,0x65,0xbd,0xb9,0xc5,0xba,0x40,0xcb,0xe4, -0x34,0xa5,0xb5,0xb4,0xee,0x1f,0x51,0x52,0x18,0x76,0xf9,0xbc, -0x4a,0x17,0xdf,0x50,0xf0,0xcd,0xf3,0x6c,0xff,0x91,0xa7,0x2a, -0xe4,0xfc,0x35,0xe6,0x48,0x3e,0xc5,0x94,0x32,0x11,0xd2,0x3e, -0x19,0x8a,0xd8,0x87,0x2e,0x31,0x39,0x46,0x2a,0x6e,0x8d,0x87, -0x39,0x7a,0x71,0xc1,0xd3,0x8b,0xf8,0xe6,0x81,0x3d,0x9c,0xa3, -0xc9,0xf7,0xd1,0xc9,0x0a,0xe7,0x18,0xc5,0x94,0x7a,0x44,0xca, -0x05,0x81,0x0f,0xcd,0x2e,0x0a,0xaf,0x56,0x0c,0x6b,0x94,0x3b, -0x2c,0xf7,0xf6,0xe0,0xa8,0x3c,0x0a,0x40,0xba,0xdd,0x41,0xc4, -0x2b,0x8a,0x15,0xe5,0xdc,0x0e,0xff,0xa0,0x1f,0xf4,0xec,0xd6, -0x1a,0x0b,0xa2,0x4f,0x20,0xe8,0xb8,0x05,0x6e,0xda,0xa4,0x02, -0xe2,0xd1,0x71,0x16,0x6f,0xb5,0xf2,0x13,0x73,0x46,0xd2,0xcc, -0x82,0x25,0x1f,0x4d,0x68,0xc2,0xec,0xf0,0xdb,0x3c,0xca,0x68, -0xc7,0x51,0x9e,0x14,0x14,0x65,0x78,0x3e,0x42,0x41,0xd8,0xe7, -0x52,0x36,0x0d,0xc7,0xbf,0x01,0x7a,0x16,0xd4,0x49,0x08,0xaa, -0x7d,0xdf,0x40,0x74,0xd1,0x01,0x93,0x0a,0x32,0x7d,0xc6,0x60, -0xb7,0xc1,0x79,0xbc,0x99,0x80,0xc0,0x9d,0xce,0x3f,0x16,0x7f, -0x3a,0x38,0x80,0xc3,0xb7,0x5a,0x15,0x28,0x92,0xe6,0xb8,0x24, -0xe4,0x67,0x28,0x64,0x0d,0xf8,0xf7,0x9c,0xdf,0x6d,0xcd,0x52, -0x0e,0xdb,0xd1,0x16,0x7b,0x96,0x2a,0xd3,0x89,0xde,0xe8,0xc2, -0xe5,0xd5,0x64,0x17,0x44,0xae,0xe2,0x2b,0xa6,0xac,0x37,0x99, -0x2c,0x86,0x80,0x9a,0x64,0x3b,0x98,0xbe,0x3a,0xa7,0xc6,0x0f, -0x28,0x6b,0x6c,0x82,0x2a,0xf3,0x87,0x1e,0xb1,0x37,0xad,0x9f, -0x0f,0xef,0x49,0x38,0x72,0xe3,0xee,0x0e,0x08,0xae,0x09,0xf9, -0x55,0x6c,0x4c,0xd3,0xed,0x77,0x62,0x91,0x4e,0xd4,0x88,0xe2, -0x68,0x4e,0xbe,0x42,0x51,0x6a,0x27,0xbf,0x98,0x7f,0x2a,0xef, -0x91,0x56,0x41,0x86,0xf7,0xd5,0xaa,0x80,0x91,0xa6,0x3e,0x89, -0x97,0x75,0x74,0x82,0x6f,0xfd,0x21,0x06,0x26,0xe6,0x49,0xad, -0x6a,0x7e,0x99,0x39,0x3a,0x8b,0xc0,0xa2,0x34,0x67,0x2c,0x09, -0xaa,0xa3,0x61,0xfd,0x62,0x53,0x13,0x68,0xe8,0x7b,0xa9,0x75, -0x59,0x76,0x3d,0x5e,0xfd,0x80,0x9e,0x38,0xe0,0x9c,0x39,0x99, -0x4e,0xa0,0xf0,0x5b,0xb8,0xbc,0x9a,0x1c,0x97,0xc1,0xfc,0x62, -0xeb,0xb7,0x17,0x42,0x65,0x23,0xe8,0x63,0x49,0xd6,0xaa,0xf1, -0xd1,0x8f,0x5a,0x9b,0x53,0x77,0xa1,0x64,0x17,0x7e,0x3b,0xf8, -0x1c,0xd8,0x5d,0x0c,0xaf,0xc7,0xff,0xc5,0x43,0x46,0xd2,0xac, -0x4f,0x89,0x8b,0xa4,0x7c,0x99,0x6c,0x4f,0x9b,0xdb,0xbd,0x24, -0xd8,0x67,0x28,0xbe,0x76,0x90,0xf1,0x80,0x21,0x20,0x0c,0x78, -0x48,0x7f,0x6f,0x86,0x44,0xc6,0xc5,0x63,0x50,0x18,0x89,0xb0, -0x36,0x19,0x02,0x72,0x6b,0x5a,0x96,0xcc,0xbd,0x9b,0xc9,0x2d, -0xac,0xf5,0x89,0x18,0xee,0x53,0x9f,0x68,0x86,0x9b,0xc7,0x06, -0x51,0x35,0x25,0xc9,0x8e,0x12,0x7c,0xac,0x9c,0x5c,0x92,0x6e, -0x94,0xb3,0x77,0x6b,0x4c,0xac,0x3a,0x53,0xb2,0x65,0xb2,0x1c, -0x4f,0x49,0x81,0x3d,0xe1,0xf5,0x02,0x08,0x2b,0x74,0x19,0x99, -0xe0,0x6c,0xa9,0xfe,0x6e,0x49,0xf3,0x2c,0xef,0xa9,0x6c,0x9a, -0x1e,0x22,0xb9,0xa5,0xa5,0x24,0x4b,0x1f,0xd2,0x4c,0xac,0xf5, -0x22,0x83,0xab,0xd6,0x3c,0xd2,0x97,0x96,0x0c,0xc4,0xe7,0xaa, -0xe5,0x20,0xe3,0xbb,0x24,0xf6,0x70,0x89,0x86,0x1d,0xc7,0x6a, -0xbe,0xb7,0x4c,0xbd,0x73,0x33,0xd3,0x23,0x24,0x2d,0x26,0x9f, -0xc6,0x23,0x1b,0xe7,0x0f,0x4d,0xf5,0x01,0xc9,0x7c,0xcc,0xbc, -0xef,0x5a,0x66,0x9a,0x30,0xc8,0x1e,0xef,0x1d,0xca,0xab,0xd1, -0x3d,0x71,0xfe,0x5d,0x95,0xf7,0x9b,0xa3,0x5a,0x3b,0x7a,0xa9, -0xcd,0xa1,0x7d,0xf7,0x56,0x79,0x73,0x1e,0xbc,0xd2,0xe4,0xc2, -0x75,0xa7,0x83,0x19,0xc4,0x93,0x9d,0xfb,0x22,0x95,0x12,0xc6, -0x2a,0xb7,0x6b,0x67,0x86,0x6f,0x9b,0x5a,0xb4,0x96,0x81,0x56, -0x18,0xad,0xcc,0x46,0x84,0x81,0x0e,0x1d,0xa3,0x85,0xee,0xf4, -0x71,0x86,0xea,0x5c,0x78,0xe2,0xa7,0x11,0xb6,0x0e,0xd3,0x9d, -0x4e,0xb5,0x10,0xe9,0x5d,0xc6,0x50,0x31,0x6c,0xe3,0xf7,0x5c, -0x29,0xda,0x52,0xdc,0xfe,0xb8,0xfc,0x94,0x8d,0xed,0xeb,0x73, -0x1a,0xb4,0xdf,0xa6,0x18,0x10,0xb4,0x99,0xa4,0x8f,0x59,0x9e, -0xa0,0x91,0xc2,0xeb,0xc4,0x7b,0x16,0xe9,0x69,0x31,0x4a,0x30, -0x8d,0x67,0x3c,0xbc,0xd0,0x29,0xc0,0x25,0xc4,0x51,0x1e,0x05, -0x31,0xac,0x3d,0xe7,0x23,0xcf,0xdc,0x5c,0xa8,0xca,0xc2,0xc7, -0x9e,0xe0,0xd0,0xd4,0xa0,0x37,0xc4,0x45,0xcd,0xf9,0x23,0x67, -0x9e,0xaa,0x2c,0xf5,0x07,0x0e,0x4e,0x65,0xc3,0xb8,0xd5,0xf8, -0xc0,0xd6,0xec,0xc9,0x76,0x7f,0xe8,0x24,0x2b,0x52,0xdb,0x1a, -0x79,0xb9,0x31,0x31,0xb0,0xd2,0x04,0x8a,0x49,0xf2,0x82,0x6e, -0xce,0x53,0xad,0x2d,0x83,0xe7,0x3b,0x5b,0x21,0x43,0xb3,0xbc, -0x41,0xcf,0x6c,0x36,0x07,0x2c,0x0b,0x7a,0x4f,0x73,0x6d,0xbb, -0x55,0x03,0xcd,0xa6,0x90,0x61,0x2d,0xb0,0x2d,0x64,0x62,0xbf, -0x06,0x10,0x68,0xaa,0xbf,0x02,0x04,0x2e,0x05,0x0f,0x43,0x4e, -0xf0,0xb9,0x6b,0xc5,0x5e,0xdc,0xac,0xbd,0xda,0x60,0xb3,0x25, -0x62,0xf2,0x4f,0xb8,0xa3,0x27,0xf7,0xdc,0x37,0x62,0xaf,0xd6, -0xac,0x6d,0x20,0xc8,0x97,0xa1,0x88,0x69,0xe4,0xb6,0xb9,0x5f, -0x69,0xa3,0x57,0x9f,0x9a,0x1f,0x9e,0xcd,0x07,0x74,0x2a,0x6c, -0x36,0x4f,0x47,0x35,0xce,0xaf,0x4e,0x2e,0xa9,0xaa,0xea,0xd9, -0xbf,0x65,0xb4,0x83,0xab,0x22,0x0b,0x5a,0xd7,0xb0,0x16,0xf8, -0xc3,0x98,0xeb,0xfb,0x8a,0x96,0x37,0x91,0xf8,0xee,0x62,0x2c, -0x12,0x73,0x73,0x26,0xb9,0xbb,0x7b,0x39,0x5e,0xee,0x66,0x6b, -0x4d,0x5b,0x2c,0x96,0xb1,0x25,0x76,0xa2,0x0b,0xe0,0x4c,0xda, -0xff,0xaf,0x7d,0x35,0x36,0x0d,0xd1,0xe6,0xf9,0xa1,0x66,0x75, -0xcd,0x58,0xe3,0xfd,0xb2,0xff,0x48,0xa9,0xeb,0x83,0xda,0xc9, -0x0d,0x71,0x8a,0xc0,0x4d,0x45,0xe4,0x8d,0xaa,0x07,0x59,0x7b, -0xd8,0x18,0x24,0x38,0x31,0x0c,0xff,0xf8,0x71,0x19,0x2d,0x6b, -0x75,0xb7,0x0d,0x95,0xb6,0x97,0x1d,0xaf,0xea,0x47,0x2a,0x00, -0x1c,0x66,0x82,0xfe,0x4f,0x7b,0x5e,0xaa,0x29,0x33,0x88,0x46, -0x4a,0x8e,0xa4,0x72,0xcc,0x00,0xbf,0x45,0x3d,0xd3,0x19,0x52, -0x28,0x59,0x50,0x01,0x1a,0x28,0xc8,0x59,0xd0,0x18,0x10,0x7c, -0xdf,0xe9,0x9a,0xdf,0x48,0xf1,0xae,0x14,0xcf,0xa6,0xf8,0x26, -0x74,0xae,0x51,0x68,0xc7,0x16,0xe2,0x20,0x25,0xe6,0x9a,0x04, -0xa6,0xf7,0xb8,0xbc,0x33,0xa2,0x4e,0x6e,0xce,0x93,0x0e,0xe5, -0x13,0x95,0x04,0xb6,0x48,0x87,0x0f,0xf2,0xf3,0xbb,0x95,0x26, -0x1a,0x98,0x42,0x2a,0x3c,0x25,0x1f,0x1f,0x18,0xfb,0x2a,0x32, -0x05,0xab,0x71,0x66,0x0d,0xf0,0x86,0xb6,0xe2,0x49,0x97,0x37, -0x80,0x9c,0x92,0xdd,0xef,0xb8,0xcb,0xe4,0x8e,0xd7,0x57,0x61, -0x9e,0xa5,0x1c,0x83,0x9d,0xc1,0xa9,0xbb,0xaf,0x30,0xb7,0xd4, -0x25,0x4e,0xc0,0xd4,0x15,0xe0,0xce,0x41,0x4f,0x4e,0x15,0xe8, -0x17,0x9f,0x74,0x46,0x51,0xe2,0x4b,0xab,0x92,0x46,0xc4,0x34, -0x1e,0xd6,0xd0,0x51,0x6c,0x34,0xd0,0x2e,0x51,0xf6,0x0d,0xb2, -0x75,0xc9,0x25,0x8c,0x90,0xf2,0xc8,0x0e,0x8a,0xbf,0x94,0x91, -0x9b,0xc6,0x5e,0xaa,0xa4,0x40,0x0d,0xb9,0x27,0x6a,0xff,0x76, -0xf8,0x89,0x91,0x49,0x74,0x5e,0x4e,0xea,0x0b,0xb0,0xca,0x04, -0x1a,0x52,0x11,0x71,0x94,0xc9,0xcb,0xe7,0x47,0x90,0xb7,0x21, -0x2a,0x68,0x5f,0x96,0x15,0x35,0x87,0x8a,0xbd,0xf8,0xc8,0xa6, -0x11,0x12,0x7f,0x34,0xc1,0x93,0x6e,0x34,0x7e,0xea,0xd7,0x86, -0xa3,0xdc,0x22,0xe8,0x89,0x7d,0x7f,0xa3,0x2d,0xa9,0x49,0x6e, -0x07,0x78,0x07,0x66,0x5c,0x81,0x8a,0xbb,0xef,0xca,0x39,0x9b, -0x27,0x3a,0x69,0x9e,0x45,0xc5,0xd0,0x50,0xf4,0x15,0x7f,0xbb, -0xc3,0xdb,0x91,0x0d,0x0b,0x20,0x8d,0x4a,0x13,0x36,0x91,0x83, -0x2f,0x2b,0x0d,0xce,0x7a,0x38,0x33,0xb4,0x43,0xc5,0x1a,0x7d, -0x1e,0x8e,0xc8,0x75,0xb2,0xbc,0xd2,0x67,0x76,0x0f,0x48,0xd5, -0xb8,0x85,0x0f,0x44,0x97,0xa8,0xf6,0xab,0xc6,0xe4,0x6e,0x27, -0xb3,0xa7,0xcb,0x25,0x5a,0x35,0xbb,0x9b,0x7a,0x0e,0x7e,0xc4, -0x7e,0xb9,0x85,0xd5,0x08,0x27,0x32,0xa2,0x8a,0x68,0x3b,0xc7, -0xed,0x13,0xbc,0xad,0xd9,0x50,0x75,0x99,0xf4,0x93,0x8e,0x40, -0xe9,0xf1,0x4f,0x07,0xd6,0x37,0x2a,0x48,0x77,0xb0,0x4c,0x27, -0x19,0xb4,0xd0,0x04,0xe1,0xed,0x02,0xf1,0x68,0x03,0xe1,0x70, -0xe0,0x62,0x8f,0x97,0xd7,0x95,0x35,0x73,0x4a,0x6f,0x59,0x93, -0xab,0xcf,0x86,0x88,0xbe,0x78,0xef,0x9a,0x68,0xd1,0x5c,0x88, -0xee,0xc1,0x1f,0xc0,0x02,0xd8,0x46,0x4c,0xea,0x6d,0x6b,0xec, -0xb7,0xce,0xef,0xfb,0xe7,0xd5,0xf9,0xe9,0x00,0x6d,0x20,0xa3, -0xbe,0xf6,0xa4,0xfa,0x49,0x07,0x85,0x4b,0x26,0x6e,0x90,0x12, -0x48,0x14,0x08,0x0a,0xb3,0x40,0xad,0xd8,0xae,0x6b,0x0b,0xc5, -0xb3,0xb3,0x3d,0x19,0x94,0x52,0x3c,0x19,0x68,0x2f,0x7c,0xcd, -0xe7,0xf7,0xe7,0xa3,0xce,0xf9,0x66,0xff,0xda,0x5c,0x60,0x28, -0x8c,0x2b,0x8f,0xa7,0xcb,0x6b,0x61,0xff,0x77,0x28,0x56,0x6c, -0xd9,0x1e,0xe6,0x7b,0x87,0xee,0x38,0x60,0xe6,0xfb,0xa3,0xef, -0xed,0x08,0xc9,0xd9,0xd4,0x43,0x2e,0x42,0x69,0x13,0xfa,0x92, -0x7b,0x90,0x9b,0xa1,0x81,0x4c,0x59,0x0f,0xfe,0x34,0x0f,0x4d, -0xef,0xa1,0xb1,0xa1,0xa2,0x8d,0x63,0xa3,0x40,0x8b,0xd5,0xfc, -0xdf,0x2b,0xa9,0x24,0x13,0x33,0x3f,0x59,0xae,0xe3,0x22,0xbf, -0x9d,0xe4,0x7c,0x46,0xf6,0x36,0xf6,0xf7,0x49,0x47,0x09,0x79, -0x31,0xd7,0x40,0x4d,0x2d,0xdf,0x03,0xf6,0xb3,0xbc,0xde,0x4f, -0x3a,0x1f,0xc6,0xcb,0x8a,0x9e,0x69,0x61,0x7b,0x75,0x92,0xfd, -0xfa,0x3a,0xda,0xf7,0xeb,0x28,0x9f,0x94,0x8f,0x8e,0x26,0xdf, -0xc7,0x27,0x35,0x7b,0x5f,0x8e,0x2a,0x46,0x63,0x4a,0x93,0x5e, -0xd2,0x83,0xd2,0x6d,0x34,0xcb,0xb0,0x00,0xc8,0x93,0xbd,0x47, -0x5b,0xcd,0xcd,0xfb,0x49,0xf6,0xcb,0x73,0x02,0x87,0x8e,0x13, -0xc8,0x19,0x7f,0x6c,0x91,0x09,0x33,0x9d,0x7a,0x38,0x0b,0xdc, -0x3e,0x12,0xeb,0xf1,0x2f,0x26,0xbf,0x8e,0x79,0x99,0xa2,0x3f, -0x5a,0xe6,0x10,0x5f,0xbc,0x5f,0xaf,0xdd,0xa9,0x8c,0x69,0xc2, -0x3b,0xd9,0x39,0x3c,0x12,0xc7,0xd4,0xf8,0x1a,0x85,0xeb,0x57, -0x3a,0x23,0xf5,0x1d,0x96,0x54,0xd8,0x88,0x79,0xc8,0xd4,0xef, -0x1c,0x14,0x6a,0xac,0x1e,0xc1,0xad,0xd7,0x3b,0xd1,0x66,0x80, -0x8b,0x0b,0xed,0x7d,0x28,0x8f,0xb2,0xc6,0x2f,0xd7,0xeb,0xcd, -0xdb,0x2f,0xe6,0xd0,0xde,0x21,0x79,0x73,0xc4,0x25,0xee,0x3a, -0x06,0x4b,0xfd,0x29,0xde,0xd4,0x10,0x9b,0x4f,0x48,0x6c,0x74, -0xbe,0x54,0xbb,0x9d,0x5e,0x83,0x08,0x7a,0x8b,0xc5,0xf0,0x3d, -0x36,0x6c,0x4a,0x66,0x74,0x8b,0xf0,0x3b,0x71,0x7c,0x5f,0xb8, -0x44,0x21,0x1e,0xd0,0x54,0xae,0xdb,0x7d,0xe2,0x51,0xaf,0x1f, -0xd8,0xfd,0x9b,0x61,0xf5,0xfa,0xe3,0x4c,0xdd,0x2a,0x92,0xcd, -0xac,0x76,0x5b,0x5b,0x22,0x46,0x11,0xc4,0x18,0x02,0x64,0x27, -0x35,0xa5,0x1c,0x33,0xf1,0xeb,0x2e,0x37,0xac,0x2b,0xbd,0x60, -0x88,0xf2,0xb1,0x53,0x96,0x12,0xca,0x4c,0x6c,0x5a,0xeb,0xa4, -0x03,0x14,0xcf,0xb1,0xf6,0xad,0x3c,0x38,0x82,0xbf,0xdf,0x9b, -0x19,0x84,0x4f,0x58,0x90,0x5f,0xdd,0xc8,0x68,0x8c,0xfe,0xb7, -0x7a,0xad,0xe8,0xf9,0xc5,0x7b,0x3f,0x87,0x15,0x69,0x1f,0xfd, -0xc7,0xf8,0x73,0xc5,0x6b,0x53,0x66,0x0d,0xf9,0xf2,0x4e,0x0c, -0xf6,0x6b,0xea,0xc7,0x32,0xcc,0x98,0xb6,0x55,0x51,0x6c,0x45, -0x4a,0x19,0xb5,0x65,0x73,0x4a,0x35,0x6f,0x59,0x3c,0xa6,0xea, -0xaa,0x8c,0xda,0xc8,0xde,0xd9,0xbf,0x27,0xd8,0x66,0x8f,0x3f, -0x38,0x02,0xb8,0x5d,0x69,0x14,0xd8,0x48,0x85,0x69,0x9e,0x6f, -0xd7,0x72,0x8c,0xc7,0xcd,0x58,0xdc,0x2d,0x28,0xb8,0x67,0xb3, -0xfa,0xd5,0xc7,0x58,0x02,0x99,0x68,0xf3,0xd9,0x8c,0x57,0x0c, -0xb3,0xf9,0x67,0xd0,0x88,0x9f,0x26,0x23,0x43,0xe9,0x8d,0x26, -0xd7,0x93,0x95,0x9c,0xef,0x77,0x93,0x51,0xf0,0xb6,0x7b,0x44, -0x87,0x83,0x3c,0xf3,0xe6,0x94,0x13,0xf3,0xe8,0xcb,0x29,0x6a, -0xb3,0xa3,0x19,0xae,0xe3,0x55,0x0a,0x44,0xeb,0x1f,0xff,0xad, -0x2c,0x29,0x8b,0x32,0x5c,0x7d,0xce,0x32,0x39,0x4e,0x72,0x04, -0xce,0x9e,0x20,0x12,0x12,0x87,0xf8,0xcb,0xc1,0xce,0xd6,0xfc, -0x7c,0x7c,0x77,0xaf,0x96,0xf3,0xdb,0x13,0xa9,0x24,0xa5,0xbc, -0x9d,0xc3,0x2c,0x04,0x64,0x2b,0x39,0x48,0x36,0x1e,0x94,0x13, -0x30,0x86,0xe8,0xb0,0x8c,0x59,0x6d,0xe3,0xd5,0x8f,0xc3,0xea, -0x06,0xa6,0x92,0xe6,0x1d,0xee,0xbb,0x1b,0xba,0x9d,0x77,0x77, -0x6f,0x38,0xba,0xbc,0x29,0xe4,0x31,0x38,0x88,0xf6,0x10,0x84, -0x5f,0x33,0xaa,0xe4,0x84,0xb9,0x29,0x21,0xe8,0x33,0xea,0xcc, -0xea,0x8e,0xf5,0x38,0x43,0x88,0x29,0xc0,0x1f,0xd8,0xe3,0x50, -0xb4,0xdd,0xb6,0xa7,0x58,0xd3,0x0c,0x43,0x26,0xda,0x43,0xe2, -0x96,0x19,0x51,0x3c,0xd3,0x0d,0x48,0x85,0x6e,0xe4,0xf0,0x63, -0x0e,0x7c,0xbe,0x31,0xfc,0x33,0x19,0x32,0xe6,0x9e,0x73,0x64, -0x7e,0x78,0x90,0xd9,0x9c,0xe4,0x4b,0x5a,0x78,0x2d,0x91,0xc3, -0x47,0xe9,0x2a,0xdc,0x2d,0xa9,0x04,0x0d,0xee,0xb5,0xc9,0xef, -0xdd,0x71,0x59,0x70,0xf1,0x4d,0x96,0x29,0xe6,0xea,0x97,0xdf, -0x1c,0xe9,0xba,0x7b,0x53,0x23,0x69,0x2f,0xc2,0xb6,0xdd,0x75, -0xf5,0xec,0xfa,0x54,0xea,0xe2,0x33,0xe7,0x5c,0xaa,0x8e,0xf2, -0x79,0xb2,0x16,0x86,0xc2,0x5e,0x71,0x1e,0x39,0x08,0xca,0x1a, -0x24,0x21,0xf4,0x41,0x65,0xcb,0x3e,0xf9,0x42,0xc9,0x9d,0xad, -0x25,0x45,0x72,0x2d,0x6c,0x2e,0xf3,0x19,0x29,0xae,0xe1,0xb1, -0xbb,0x06,0x43,0x30,0x97,0xb0,0xf2,0x46,0xbf,0x59,0xb6,0x94, -0x38,0x9d,0x0c,0x4b,0xc5,0x97,0x69,0xbd,0xa7,0xf8,0xaa,0x1f, -0x96,0xe9,0x73,0x03,0x5b,0x98,0x34,0xc0,0xa2,0x82,0x8d,0x89, -0x28,0xa3,0x1c,0x28,0x31,0x32,0xb1,0x05,0x08,0xa5,0x7d,0x15, -0x94,0x67,0xc1,0x4a,0x77,0x80,0xea,0xd4,0x49,0xad,0x0f,0xa2, -0x4a,0xd4,0xdc,0x00,0x4e,0xfc,0xaa,0x16,0x18,0xac,0xe7,0xde, -0xdf,0x4c,0x7c,0x0d,0x2b,0x91,0xc2,0x5f,0xea,0x59,0x83,0xbf, -0x86,0xd3,0xf1,0x52,0x12,0xcc,0x2b,0x65,0x30,0xa3,0x97,0x70, -0x35,0x35,0x03,0xde,0x1d,0x43,0xde,0x63,0xb6,0x03,0x44,0xdf, -0x1f,0x09,0x35,0xb5,0x03,0x83,0x8a,0x33,0xa6,0xac,0xeb,0x1a, -0x26,0x0c,0xc8,0x55,0x7d,0xd8,0xb3,0x10,0x27,0x9a,0x54,0x6a, -0xad,0x04,0x71,0x33,0xbf,0x26,0x59,0xe8,0x16,0x7f,0x1c,0x6f, -0x6e,0xbd,0xa6,0x4f,0xec,0x17,0x05,0xd0,0xe5,0x82,0x81,0x84, -0x8c,0xc2,0xed,0x98,0x15,0x8b,0x5a,0xd3,0x35,0x1d,0x1d,0x79, -0x0d,0xe7,0x18,0x0b,0xf3,0x4c,0x27,0x15,0x7b,0x8d,0xc0,0x82, -0x7c,0x57,0x37,0xdf,0x99,0xb5,0x42,0x78,0xe6,0x49,0x39,0x7b, -0x76,0x5d,0x4e,0xef,0x46,0xe3,0x0a,0xce,0x58,0x58,0x74,0xb9, -0x19,0xb2,0xbd,0xc3,0x9d,0x52,0x2d,0x45,0xf8,0x44,0x5b,0x32, -0x4e,0xe3,0xa7,0x4f,0xf8,0x52,0x0b,0x13,0x99,0x73,0xc9,0x63, -0x22,0x69,0xa8,0x8f,0x8c,0xc7,0x3b,0x42,0xf7,0x31,0x83,0x45, -0x03,0x28,0x1b,0x1a,0xf0,0xc3,0x14,0xbc,0xed,0x95,0x62,0x3a, -0xd2,0xd6,0xf9,0x96,0x50,0x9e,0xb7,0x64,0x20,0x45,0x5a,0x63, -0x18,0x28,0x7c,0x45,0x87,0x8d,0xe5,0xe4,0x80,0x08,0x23,0xcf, -0x24,0xdc,0x8c,0x48,0x76,0x2a,0x8f,0xf6,0x0e,0x2c,0xcb,0xcf, -0x69,0x35,0xbf,0x5b,0x5e,0x8e,0x73,0x18,0xb8,0xd5,0x64,0xc6, -0xd2,0xa1,0x74,0xe7,0xeb,0x7e,0xe0,0xf4,0x0c,0xe9,0x02,0x5e, -0xc7,0x12,0x21,0xa6,0xa1,0x66,0xd7,0x2f,0x99,0x2c,0x4f,0x2e, -0x17,0x1f,0xd1,0x93,0xc5,0xc9,0x70,0xf6,0x87,0x15,0x81,0xde, -0x69,0xbd,0x1c,0xfe,0x0a,0xeb,0x12,0x3f,0x26,0xe3,0xaa,0x05, -0x9b,0x97,0x61,0xb6,0xe6,0xcb,0x16,0xc1,0x82,0x06,0x57,0x2b, -0xb4,0x48,0x5b,0xc1,0x3a,0x25,0xf3,0x0f,0x95,0xea,0x9f,0x77, -0x19,0x47,0x88,0x9b,0xc5,0x95,0xc3,0x68,0xda,0x84,0x7a,0x4b, -0x26,0x51,0x43,0x4e,0xa9,0x58,0x6b,0x38,0x1b,0xb5,0x4c,0x41, -0x18,0xee,0xb1,0x5c,0x40,0xb0,0xde,0xa4,0x46,0x3d,0xd0,0xb4, -0x9b,0xf7,0xa6,0x80,0xcc,0xab,0x88,0x71,0xf8,0x49,0xe7,0xbc, -0x76,0xd6,0x6d,0x21,0x56,0x61,0xf2,0x63,0x4d,0x94,0x1a,0x59, -0xd1,0x52,0x8c,0xc1,0x64,0xf6,0x98,0x83,0xee,0x1f,0x93,0xda, -0x8c,0x0f,0x88,0xb0,0x9a,0xcf,0x46,0xe8,0xc1,0x26,0x8b,0xf1, -0x65,0xe9,0x51,0x9c,0xb1,0xb4,0x4d,0xc3,0x88,0xec,0xc8,0xe2, -0x13,0xa6,0xf6,0x1b,0x67,0xfc,0xcb,0x47,0x96,0xb2,0x7e,0xf8, -0x87,0x35,0xb4,0xed,0xf8,0xf5,0x6c,0x5e,0xbb,0xe0,0xfa,0x03, -0xa4,0xe1,0xe9,0x02,0x6a,0xca,0x8a,0x14,0xac,0x49,0xc4,0x67, -0x27,0xd4,0xb9,0x45,0x9e,0xb0,0x3b,0xf2,0xb7,0x95,0x0b,0x92, -0xbb,0x92,0xab,0x70,0xb7,0xb3,0x44,0x10,0xcb,0xb4,0xb9,0x6d, -0xf7,0x80,0xba,0xb8,0x4d,0x65,0x32,0x19,0xc4,0xf1,0x06,0x0e, -0xa8,0xf9,0xc7,0x13,0x2a,0xb4,0xbc,0xcc,0x09,0xff,0x83,0x3f, -0x88,0xc2,0x91,0x6d,0xec,0x78,0xec,0x97,0x97,0x19,0xfc,0x6b, -0xa6,0x55,0x92,0xdd,0x5d,0x54,0x5a,0x41,0x3b,0xc5,0xd5,0xdd, -0x05,0x5c,0xf5,0xe9,0x41,0xfe,0x88,0xda,0xa7,0xa8,0x1e,0x14, -0x95,0xa9,0xd1,0x46,0x59,0xfc,0x6e,0xfc,0xdb,0xdd,0x70,0x0a, -0xb4,0xc8,0x61,0x3e,0x7f,0x44,0xbb,0x64,0x4e,0xf4,0xf8,0x23, -0x25,0xde,0x49,0xab,0x94,0x29,0xfc,0x72,0x7e,0xc8,0x02,0xbb, -0x41,0x86,0x43,0x22,0x8a,0x51,0xf3,0x72,0x7e,0x08,0xf7,0x10, -0xfa,0x4e,0xb0,0xa5,0x3d,0xe9,0x8f,0x2b,0x0e,0xff,0x9d,0x8e, -0x57,0xf9,0xea,0xd0,0xc9,0xb4,0x1d,0xe6,0xab,0x47,0xee,0xe3, -0x11,0x99,0xdc,0x83,0x16,0xac,0x1e,0x05,0xe8,0xce,0x21,0x5b, -0x8c,0x70,0xec,0x80,0xc3,0x4c,0x0e,0x1a,0x9a,0x66,0xfc,0x74, -0xec,0x32,0x8e,0xc1,0x21,0x98,0x1f,0xfa,0x99,0xc7,0x46,0xf9, -0x2f,0xde,0xab,0x32,0xd5,0x50,0x1a,0x48,0x4e,0xf7,0xf5,0xc8, -0xc1,0x82,0xf2,0x9e,0x9d,0x44,0x00,0xfc,0x27,0x43,0x90,0xcb, -0xf9,0xd0,0x9c,0x70,0x50,0xde,0xb2,0x41,0x0e,0x83,0x2f,0x4e, -0x93,0x3e,0x3c,0x32,0x0c,0x12,0x1e,0x96,0x52,0xe6,0x5e,0x69, -0xd4,0x43,0xbc,0x00,0x12,0x37,0xbf,0x48,0x31,0x11,0xb5,0xd4, -0x6e,0x07,0x9c,0x90,0x43,0x9e,0x5e,0x1e,0x22,0x37,0x9f,0x4c, -0x2b,0xcf,0x1f,0x49,0x62,0x54,0x15,0x2f,0x8d,0x9d,0x83,0x8d, -0xad,0xef,0x11,0xd6,0xb7,0xc3,0x19,0xb6,0xad,0xb6,0x86,0xc6, -0x48,0x1d,0x3b,0x81,0x79,0xb7,0xa0,0x85,0xb5,0x46,0xe8,0xf2, -0xd8,0x04,0xb3,0x6a,0x47,0x04,0x12,0x52,0x65,0x19,0x1d,0xaa, -0x00,0x11,0x4d,0x82,0x84,0xef,0x3d,0x57,0x42,0xf8,0x3b,0xc8, -0xa0,0x11,0x8e,0xc4,0xd7,0x0c,0x84,0x6b,0xc3,0x46,0xa3,0xec, -0xed,0x46,0xce,0xd1,0xe8,0x26,0x3c,0xcc,0xe9,0xe7,0x11,0x23, -0x11,0xc6,0xc1,0x1e,0xa6,0xa9,0x1b,0x35,0xe2,0x6a,0x32,0xe6, -0x11,0xe4,0xb6,0xf4,0x22,0x66,0x41,0x0e,0x26,0x12,0x72,0x15, -0x6b,0x38,0x3f,0x0f,0x40,0x06,0x40,0x82,0x2c,0x24,0x46,0xe3, -0x20,0x5d,0x88,0x3f,0xab,0x2b,0x27,0x11,0x8b,0xce,0xb4,0x3f, -0x87,0x8c,0x65,0xf5,0xda,0xf5,0xa8,0x17,0x54,0x2c,0x14,0xe4, -0xa3,0xac,0x70,0xd8,0x96,0x9d,0x2a,0x00,0x88,0xfe,0xf9,0xdc, -0xe5,0x0a,0x9b,0x92,0x37,0x46,0xef,0x6a,0x56,0xb8,0x5a,0x94, -0x39,0x5b,0x27,0xa6,0xe2,0xea,0xa1,0x98,0xa0,0x96,0xd4,0x5e, -0x0f,0xa1,0x69,0xdc,0xb4,0x71,0xe8,0x8c,0xa2,0xd8,0x52,0x30, -0xdb,0x34,0x3c,0x46,0x7f,0xa1,0x56,0xa3,0x3d,0x05,0x09,0x54, -0x19,0xc6,0x5b,0xd6,0x06,0xb3,0x78,0xdf,0x32,0xcb,0x78,0x19, -0x3a,0x05,0x65,0xc7,0xa9,0xf4,0x29,0x23,0xb6,0xff,0xcb,0xf9, -0x6e,0x47,0x9e,0x63,0x71,0x63,0xe0,0x3b,0xa4,0x5f,0xfc,0x85, -0xdb,0xf0,0xde,0x71,0x68,0x99,0xec,0xfe,0xfc,0xfc,0xd5,0x93, -0xd7,0x3f,0x8b,0xbf,0xd0,0x76,0x5b,0xc9,0x02,0x75,0x90,0x2e, -0xb9,0x9e,0xbc,0x3e,0xf9,0xe9,0xe5,0xd3,0x57,0x67,0xc4,0x87, -0xa6,0x13,0x45,0x1d,0xb6,0xa7,0x9c,0xe1,0xf4,0xe4,0xf5,0x9b, -0xa7,0x09,0xe1,0x63,0x71,0xf3,0x09,0x5d,0x5e,0x40,0x13,0x57, -0x9f,0x1f,0xa2,0x59,0x21,0x57,0xcf,0x37,0xb6,0xce,0xf8,0xcf, -0xc3,0xc1,0x10,0x88,0xbd,0xa4,0xd5,0x4a,0x0a,0x7e,0x1b,0xf2, -0xec,0xaf,0xe5,0xfc,0x96,0x6a,0x46,0xdb,0x53,0xe6,0xcd,0x42, -0x8e,0x49,0x8a,0xed,0x51,0x0d,0xc4,0x37,0xe0,0x88,0x02,0xff, -0xda,0xb6,0xff,0xc0,0xcf,0x8d,0x4a,0x86,0xc1,0xee,0x95,0x31, -0x3a,0xd8,0x91,0xc1,0x71,0x9b,0x9e,0x66,0xa3,0x9c,0xa2,0xc9, -0xed,0x4b,0x34,0xcf,0x9c,0x24,0x1d,0x41,0x63,0xf9,0xc9,0x6c, -0x27,0x4d,0xae,0xa0,0xa7,0x1f,0xd6,0xeb,0xe4,0x40,0x7e,0xaf, -0xe0,0x18,0x18,0x4b,0x78,0x36,0xd7,0x80,0xfc,0xf6,0x91,0x0a, -0xfa,0xc0,0x18,0x55,0x4b,0x40,0x1c,0x1e,0x05,0x9c,0x01,0xbf, -0x81,0x71,0x3b,0x42,0xb7,0xce,0x86,0xd7,0xea,0x43,0x38,0xbb, -0x97,0x40,0xf9,0xfe,0xb7,0x17,0x40,0x80,0xb8,0xf8,0xee,0xe5, -0x14,0x8d,0x9c,0xb2,0xa5,0x14,0x89,0x64,0x67,0x76,0xc8,0x43, -0x72,0x06,0x52,0x36,0xd8,0x9f,0xb3,0xa7,0x7f,0x3f,0x7b,0xf7, -0xea,0xf5,0x93,0xa7,0xe5,0xb7,0xf4,0x72,0xf8,0x23,0x64,0x53, -0x78,0xb1,0xab,0x3b,0x07,0x9f,0x60,0x31,0xf8,0xd0,0xe1,0x71, -0xff,0x60,0xe0,0x58,0xe9,0xb0,0xb6,0x1c,0xf0,0x9e,0x1f,0x34, -0xad,0x84,0x9f,0xc8,0x31,0x24,0xe6,0xd0,0xf7,0x7f,0x49,0xbf, -0xef,0xff,0xf2,0x78,0xd0,0x79,0x9c,0xed,0x67,0xe8,0x2c,0xc1, -0xbd,0xa1,0xfc,0xf2,0x3d,0xba,0xa6,0x3c,0xdf,0x1b,0x74,0xb2, -0xfd,0xe8,0x9d,0x27,0x77,0x1e,0x8d,0x75,0xcd,0x7d,0x9f,0x74, -0x7c,0x65,0x2e,0x75,0x63,0xfa,0x2d,0x4d,0x6e,0x68,0x92,0xdd, -0x93,0xcb,0xcf,0x4f,0xc6,0x5b,0xbc,0xe5,0xde,0x9b,0x8e,0x8f, -0xb6,0x65,0x32,0xe3,0xec,0xa1,0xd2,0x42,0x04,0xec,0xf4,0x6f, -0xe4,0xcd,0xf7,0x57,0x09,0x38,0x66,0xf1,0x3b,0x87,0x1d,0x23, -0x31,0x87,0x37,0x72,0x44,0x92,0xba,0x22,0x6c,0x45,0x50,0x2c, -0x2d,0xb4,0xad,0xb3,0x56,0x0f,0x14,0x96,0xb2,0x7e,0xc0,0x06, -0x76,0xe0,0x4b,0x8d,0xd4,0xe2,0x25,0x14,0xc7,0xbb,0xac,0xb9, -0xcb,0xae,0x1c,0x99,0x58,0xf3,0x3c,0x26,0x3c,0x7a,0x32,0x7f, -0x42,0x05,0x09,0x52,0xe0,0x70,0x90,0xf5,0xbe,0x04,0x14,0xf2, -0x14,0x80,0xa9,0x29,0x7f,0xbd,0x67,0x1e,0x5d,0xf8,0xce,0xec, -0x69,0x88,0xc9,0x11,0x94,0x15,0x29,0x34,0xaa,0xec,0x3b,0xbe, -0x3c,0xc4,0x0f,0x7c,0x3c,0x9e,0x4f,0x40,0x6a,0x07,0x0c,0xfa, -0xd5,0xdc,0x0d,0xbc,0x63,0xd0,0xb3,0x29,0x94,0xaf,0x22,0xfd, -0x99,0x49,0x1d,0x30,0x8d,0x7a,0x9a,0x9d,0x73,0x5c,0xf9,0x07, -0x41,0xc8,0x23,0x93,0x40,0x35,0x70,0xd3,0xe5,0x39,0x6c,0x39, -0x61,0x99,0x11,0x9c,0xe5,0x9d,0x83,0xac,0x63,0xb2,0x8b,0xe8, -0x42,0x52,0x98,0xd7,0x33,0xcd,0xee,0x73,0x91,0xa8,0x16,0x0a, -0xb6,0x7e,0x05,0xec,0x0f,0x5f,0x02,0xfb,0xc1,0x42,0x64,0xd3, -0x30,0x7a,0x8f,0xf1,0x87,0x73,0x81,0x5c,0x24,0x89,0xc1,0xdc, -0x15,0xca,0xe9,0xf8,0xfa,0x56,0x16,0xbd,0xe7,0xb7,0xd6,0xaa, -0xc0,0x0a,0xfc,0x7e,0xfe,0xd7,0x47,0x7f,0xda,0xbf,0x9e,0xe4, -0x81,0x5f,0xe5,0xfd,0x7f,0xfd,0xf6,0x09,0x45,0x96,0x41,0xe4, -0xa3,0x1f,0x28,0xb2,0x63,0x79,0xdf,0x0d,0xe0,0x17,0x97,0xab, -0xa7,0x14,0x7b,0xba,0x80,0x72,0x55,0xdc,0x92,0x68,0x77,0xda, -0x0a,0xbe,0x3b,0xa0,0x0a,0xfe,0x67,0xd8,0x94,0x63,0x8a,0x2c, -0xc2,0xa6,0x7c,0xb7,0x7f,0x9d,0x03,0x0e,0x1a,0xc4,0x9d,0x50, -0xc6,0x3c,0x8c,0x44,0x27,0xfa,0x51,0x8b,0x7a,0x09,0x44,0x03, -0xc0,0xa0,0x23,0xf2,0x48,0xfb,0x7c,0x06,0xbb,0x50,0x0e,0xd0, -0xfc,0x62,0x3e,0x5f,0xc1,0x85,0x88,0xae,0x46,0x7c,0xbe,0xf0, -0xf8,0xd5,0x23,0x9d,0x25,0xc7,0x09,0xf3,0xe0,0x35,0x20,0xe9, -0x74,0xae,0x43,0x19,0x95,0x22,0xe1,0x27,0xde,0xdc,0x39,0xb0, -0xef,0x4b,0x68,0x40,0x6a,0xe6,0x68,0xc2,0x65,0x76,0x5d,0x40, -0x7e,0xe8,0xc6,0xec,0x7a,0x8f,0x03,0x9f,0xf6,0x5c,0x10,0x95, -0xc9,0xf4,0x6b,0x90,0xbf,0xfa,0xeb,0xbb,0xe3,0x37,0x6f,0xde, -0x9d,0xbc,0x38,0x3e,0x3d,0x7d,0xf7,0xf6,0xe9,0x5f,0x9f,0xfe, -0xfd,0x4d,0xb9,0x7f,0x5e,0xcd,0xae,0xfb,0x05,0x1c,0xd0,0x90, -0x27,0x2d,0xce,0xab,0x6f,0xe8,0xbc,0x1e,0xbd,0x83,0x03,0xfb, -0xa8,0x97,0xf5,0xce,0xab,0x7d,0xc7,0x14,0xa0,0x2a,0x43,0x91, -0xa5,0x8c,0x7c,0xb0,0xab,0x33,0x17,0xf4,0x2f,0xc9,0xfd,0x75, -0x12,0x35,0x40,0x3a,0x49,0x5f,0x7e,0xf8,0xfc,0x7c,0xc4,0x65, -0xb2,0xdc,0xbb,0x9c,0xd1,0xa1,0x87,0xf1,0xcd,0x93,0xf3,0x73, -0x98,0x36,0xed,0x6c,0x97,0x44,0x3c,0x4e,0xc5,0xfb,0xea,0xf1, -0x14,0x7d,0x45,0x69,0x4b,0xb6,0x65,0x49,0x93,0x2e,0xcb,0xc2, -0x64,0xd2,0x10,0x38,0x5c,0xbf,0xb6,0x48,0x47,0xaa,0xff,0xfa, -0x82,0x7d,0x2d,0x38,0xf0,0xc5,0x70,0x27,0x46,0x25,0x23,0x19, -0x25,0x5a,0x06,0x48,0x68,0xf8,0x69,0x16,0x5c,0x7c,0x3a,0xac, -0xc8,0x6f,0x4e,0x99,0xb4,0x60,0xaf,0x4b,0xb5,0x2e,0xb6,0x03, -0xb1,0x39,0x3b,0x1b,0x6a,0x98,0x48,0x36,0x06,0xe5,0x32,0x67, -0xec,0x96,0xa8,0x97,0xfa,0x4a,0xca,0x71,0xb0,0xa6,0xca,0x54, -0x3d,0x18,0xf1,0x7d,0xe4,0xc5,0x32,0xaa,0x0e,0x6e,0x16,0xd8, -0x17,0x80,0xf5,0x46,0x63,0x30,0x5c,0x01,0x7e,0x76,0x71,0xb7, -0xb2,0xcb,0x00,0xe3,0xb2,0x7b,0xd3,0x9b,0x76,0x9b,0xd7,0x04, -0x26,0x90,0x93,0x77,0x34,0x42,0xba,0xbd,0x1d,0x94,0x4d,0x3d, -0xc0,0xe3,0xc9,0x6f,0x21,0xb9,0x2d,0x95,0xd6,0x36,0x45,0xaf, -0xcf,0xbf,0x03,0x20,0x48,0x2c,0x8d,0xe3,0x4a,0x84,0xf5,0x54, -0x2a,0x10,0xf6,0x83,0x66,0x28,0x43,0x29,0xe7,0x2d,0x08,0x99, -0x5b,0x8e,0x93,0xd9,0x7b,0x9a,0xfa,0x54,0x6e,0xce,0xd5,0xf0, -0xba,0xf4,0xa8,0x13,0x72,0x07,0x65,0xcd,0xf7,0x12,0x0d,0x25, -0x45,0x13,0xea,0x77,0x14,0xb3,0x0e,0x2f,0xb0,0x3d,0x8b,0x11, -0x0c,0xf9,0xf1,0x62,0xd1,0x3a,0x9e,0x92,0x15,0xb5,0x96,0x6b, -0xe7,0x62,0x3c,0x6a,0xa1,0xd5,0x4d,0x12,0x9d,0x6d,0xc9,0x20, -0xb4,0xfe,0x70,0x7f,0xb0,0xf9,0x43,0x92,0x43,0x2b,0xb2,0x8d, -0xf4,0xb0,0x94,0x5f,0x32,0xa6,0x24,0x61,0xf7,0x20,0xd9,0x4f, -0x76,0x17,0xcb,0xf9,0x07,0xd4,0xe5,0xf4,0x73,0xa7,0x51,0xd9, -0xbd,0x86,0x78,0x26,0xd2,0x64,0x77,0x09,0xd5,0x4b,0x5d,0x49, -0xee,0x8e,0xa7,0x41,0x56,0x03,0x0c,0x07,0x8f,0xa8,0x44,0xe8, -0x08,0x95,0x6c,0xbd,0xec,0xb9,0x0e,0x98,0x4e,0x80,0xe7,0x18, -0x73,0x02,0x0c,0xe9,0x87,0xf9,0xaf,0x63,0x6c,0x1a,0xd6,0x46, -0x54,0x0b,0x0c,0x42,0x58,0x75,0xb2,0x8b,0xca,0x7f,0x13,0xb2, -0xb9,0xb1,0xab,0x25,0x31,0x2c,0x2a,0x4b,0xd6,0x80,0x34,0x02, -0xd0,0xb6,0xe6,0x52,0x2c,0xd7,0x32,0x6a,0x26,0x02,0xad,0x71, -0x43,0xbe,0xee,0x2e,0x13,0x7a,0x66,0x11,0xe8,0x4c,0x93,0x5a, -0xb5,0xad,0x4c,0x43,0x99,0x02,0x75,0x73,0xc9,0x75,0xe2,0xa6, -0x17,0xe8,0xdd,0x31,0xc9,0xc2,0x8d,0xc8,0xd8,0x02,0x22,0x5d, -0x52,0x74,0x83,0xa7,0xef,0x93,0xa7,0xcf,0x9e,0xbe,0x7d,0xf7, -0xc3,0xeb,0xd7,0x67,0xa7,0x67,0x6f,0x8f,0xe1,0xf0,0xfd,0xa5, -0x1e,0xb9,0xb3,0xaf,0xc3,0xc4,0xb2,0x6a,0xed,0xf6,0x4e,0x3d, -0x13,0x53,0x90,0x22,0x56,0x47,0xe7,0x5d,0xcf,0x2c,0x6c,0x14, -0x8c,0x37,0x69,0xa5,0x09,0xbb,0x6d,0x5e,0x87,0x49,0x46,0x6d, -0xe4,0x5a,0x43,0xab,0x8d,0xb0,0x82,0x1b,0xb6,0x0a,0xbd,0xad, -0xbe,0xd4,0x1d,0xe5,0xce,0x07,0x13,0x6b,0xb0,0x7c,0x11,0x9a, -0xd3,0x15,0x43,0x37,0x9d,0x44,0x6e,0xd0,0x72,0x86,0x69,0xf1, -0xc6,0x09,0xd9,0x7a,0xb2,0x69,0x36,0xfc,0x75,0xfc,0x8e,0x69, -0x01,0x32,0xa7,0x3a,0x46,0xa7,0x41,0x38,0x0d,0xde,0xcd,0xad, -0xc4,0x94,0x2e,0x04,0x87,0xd9,0xbb,0x24,0x0f,0xba,0x7a,0xfa, -0xea,0xf8,0x3f,0x9e,0xbe,0x3b,0x39,0x3e,0x7d,0x2a,0x67,0xa5, -0x6f,0xe1,0x14,0x08,0x54,0x24,0x54,0xe7,0x0e,0xed,0x48,0x21, -0xdc,0x73,0xd0,0x48,0xae,0x8a,0x33,0x85,0x0e,0x9b,0x36,0x11, -0x57,0xe5,0xdf,0x19,0x9f,0x51,0xe1,0xda,0x32,0x90,0x7f,0x14, -0x89,0xe0,0x5e,0xca,0x47,0x8b,0x4a,0x41,0x8e,0x03,0xdf,0xba, -0x57,0xb3,0x9c,0x17,0x66,0xf1,0xef,0xff,0x89,0xb9,0xde,0x38, -0xfd,0x4f,0x5e,0xd6,0xf8,0x2e,0x85,0xa2,0xa2,0xe3,0x65,0x2d, -0x83,0x4f,0x72,0xcb,0xad,0x96,0xc7,0xed,0x02,0x79,0x6d,0x1f, -0x8f,0xc8,0x23,0x74,0x3d,0x9b,0x49,0x85,0x39,0x92,0x74,0xbc, -0x29,0xb8,0x8b,0xf2,0x4e,0x94,0xb0,0x41,0x7d,0x14,0xfa,0xa7, -0xff,0x0f,0x1f,0xc8,0x4a,0xb6,0x44,0x20,0xe7,0x21,0xff,0xff, -0x40,0x4e,0xf6,0x10,0x2d,0x19,0x01,0xad,0x2d,0x64,0xc0,0xb8, -0x80,0x5b,0x9c,0xe1,0x49,0x6d,0x70,0xb2,0x0a,0x8d,0xf3,0x1f, -0x2f,0x49,0x02,0x2f,0x17,0x9f,0x73,0xc3,0x4a,0x5e,0x71,0x76, -0x50,0x0c,0x2f,0x3e,0x7a,0x87,0xcb,0xf1,0x6f,0x78,0xee,0x0a, -0xa3,0x87,0x4f,0x55,0x7c,0x17,0xba,0x3f,0xdc,0xf0,0x32,0x82, -0x05,0xd5,0x4b,0x04,0x0e,0x84,0xd1,0x8f,0xc9,0x64,0x89,0xc6, -0xc2,0x8f,0x3c,0x0f,0xa9,0xa1,0x09,0xcf,0x66,0xbe,0x11,0xec, -0xd6,0x83,0xa8,0x8f,0xe3,0xd9,0x6c,0xbe,0x92,0x97,0x25,0x2d, -0xdf,0x94,0x8a,0x8c,0x65,0x26,0x57,0xb0,0xd5,0x78,0x7f,0x2e, -0xaf,0x91,0x45,0xd7,0x47,0x59,0x45,0x15,0x46,0x83,0xe3,0xc5, -0x77,0x39,0x12,0x6a,0xe4,0x9a,0xd1,0xf3,0x6c,0x6b,0xe8,0x64, -0x17,0xf2,0xd6,0x35,0x7c,0x27,0x48,0xb6,0x5c,0x37,0x48,0x05, -0x40,0x2c,0xc9,0x42,0x1a,0x43,0x76,0x5d,0x19,0x02,0x7e,0xc5, -0x48,0x0a,0x9f,0x13,0xb0,0xb8,0xa0,0xe7,0xd7,0xb4,0x4b,0x98, -0x93,0x34,0x5c,0xdd,0xe4,0xb8,0x2b,0x9e,0xcd,0xce,0xe6,0x74, -0xaa,0xf3,0x04,0x60,0x7c,0xe6,0x1f,0xa3,0xed,0x73,0x05,0x49, -0x25,0x95,0x98,0x41,0x29,0xdb,0x2e,0x9c,0x46,0xe8,0xb5,0xe9, -0xb9,0x30,0x14,0x51,0xd4,0x0f,0x2d,0x53,0x94,0x46,0xe4,0x89, -0x84,0xa0,0x20,0xf8,0x78,0x42,0xef,0x5f,0x48,0x08,0x8b,0x74, -0x53,0x4e,0x8f,0xdd,0xf4,0x08,0x9a,0xc6,0x50,0xb2,0xe0,0x55, -0x63,0x27,0x68,0x28,0x8e,0x7b,0x28,0x37,0x4b,0x3c,0x53,0x0b, -0x82,0xde,0x6a,0x8b,0x50,0x7e,0x0c,0x7d,0x2f,0xf0,0xf1,0xf7, -0x62,0x4e,0x06,0x8d,0x45,0xf0,0xf8,0xde,0x10,0x3e,0xd5,0xdd, -0x92,0xa8,0x57,0x9e,0x99,0xab,0x21,0x0e,0xef,0x67,0xfb,0x38, -0xdf,0x17,0x67,0x89,0xa9,0x0b,0x97,0x92,0x0b,0x50,0x8e,0x8d, -0xa3,0x8a,0x08,0x8e,0xfc,0x88,0x38,0x73,0x22,0x3b,0x23,0xc9, -0x79,0x9e,0xb2,0x3c,0xe1,0x63,0x36,0xc9,0x63,0x49,0x65,0xc1, -0x14,0x0c,0x7b,0x22,0xd0,0x40,0x90,0xe5,0x8d,0x56,0xb1,0x67, -0x57,0x93,0x6b,0x23,0xf7,0xa5,0x29,0xed,0xb6,0x1e,0xe7,0x11, -0xa3,0x9f,0xae,0x21,0x76,0x00,0x8d,0xc9,0xd2,0x03,0x36,0xd0, -0x2c,0xcd,0x95,0x24,0x19,0x01,0xa3,0xff,0xe3,0x1f,0xc5,0x11, -0x21,0x78,0x31,0xc4,0x95,0x24,0x08,0xc9,0x52,0x95,0x2e,0xd8, -0xf5,0xc6,0x4b,0xfa,0x78,0x40,0x2a,0x87,0x41,0xfc,0xa7,0xd8, -0xda,0xf0,0x45,0x60,0x11,0xe3,0xfd,0x93,0x0c,0xd2,0x7e,0x0c, -0xd9,0xb1,0x79,0x1d,0x76,0xa3,0x73,0xbd,0xd9,0xb0,0x97,0x51, -0xee,0xba,0x9c,0x1e,0x22,0x12,0x6e,0x10,0x84,0x0c,0x90,0xa0, -0x39,0x94,0x84,0x93,0x84,0x57,0x81,0x3f,0x47,0x68,0x03,0x7e, -0x18,0x4e,0xa6,0x88,0x15,0xec,0xb4,0xfe,0x6b,0x7e,0xd7,0x1a, -0x03,0x2e,0x37,0x86,0xa9,0x98,0x54,0xd5,0x62,0x8c,0x8e,0x07, -0x00,0xb1,0x1b,0xcb,0xcc,0xb4,0x70,0x68,0xf0,0xa9,0x1a,0xf6, -0x06,0xee,0xd5,0xd5,0xbc,0x85,0x46,0x78,0x5b,0x93,0x55,0xb7, -0xf5,0xfc,0x0a,0x26,0xe1,0x7a,0x52,0xc1,0xe0,0xe0,0x03,0xf7, -0x50,0x8b,0xf0,0xe0,0x02,0x90,0xe1,0xaa,0xf5,0x19,0xe0,0xa3, -0xb7,0xa9,0xc9,0xd5,0x67,0x82,0x3a,0x22,0x53,0x98,0x40,0x4a, -0xe3,0x5b,0xf8,0xb0,0xa2,0xb8,0x6a,0x0c,0x93,0x3b,0x72,0xbc, -0xef,0x2e,0x9f,0x73,0x8a,0xc2,0xb9,0xd1,0x23,0xd7,0x23,0x77, -0xb3,0x1f,0xa6,0x73,0xf1,0x43,0xc2,0x4b,0xa2,0xb4,0x73,0x14, -0x60,0x64,0x9c,0x90,0xc4,0x63,0x58,0xde,0xbf,0x33,0x50,0x0b, -0x13,0xce,0xdf,0x39,0xf8,0x85,0x0b,0xb9,0x25,0x58,0xb8,0xb5, -0x88,0xcd,0x2b,0xd4,0xac,0x08,0xcd,0x5c,0x11,0x36,0xc2,0x21, -0xb6,0x89,0x66,0x20,0x03,0x27,0xb4,0x77,0xb6,0x66,0x95,0x74, -0x34,0x93,0x32,0x5e,0x7e,0x98,0x5c,0x8e,0xb7,0xe6,0x94,0xf4, -0x44,0x64,0x19,0xb6,0xe6,0xa3,0x54,0xd4,0xcb,0x9d,0x53,0xd7, -0x57,0x5b,0x33,0x6a,0x06,0x08,0x0a,0x16,0x9d,0x64,0xde,0xaa, -0x70,0x54,0x4c,0x10,0xcb,0x37,0xda,0xb3,0x3c,0xd1,0x55,0x80, -0x9d,0x64,0x6b,0xa8,0x61,0x09,0x8e,0x6c,0x2e,0x60,0xd0,0x88, -0xb0,0x90,0x4f,0x68,0x2e,0x38,0x82,0xa9,0xb8,0x44,0xfd,0xe7, -0x5a,0x39,0xc2,0x88,0x4d,0x21,0x97,0x93,0xab,0x83,0x35,0x53, -0xf0,0x4f,0x68,0xb1,0xf3,0x02,0xa7,0xdb,0x1f,0x2f,0x3a,0xff, -0x8c,0x25,0x72,0xa2,0x7a,0x2a,0x95,0x3c,0x7a,0x26,0xa1,0x79, -0x6d,0x0c,0xa5,0xee,0x90,0x8a,0xb7,0x2d,0x69,0xa1,0x78,0xce, -0xed,0xdd,0xc5,0x74,0x52,0xdd,0x3c,0x05,0x7c,0x6b,0x39,0x1b, -0x4e,0x8f,0xdf,0x3c,0x4f,0xe5,0xc0,0x04,0x9c,0x9f,0x91,0x30, -0xf9,0xce,0xef,0x1d,0xf9,0x58,0xb8,0x50,0x8e,0xaf,0xf4,0x05, -0xfe,0x11,0x94,0xad,0xe0,0x9f,0x9c,0x5f,0x0a,0x0b,0xfe,0x51, -0xba,0x43,0x70,0x16,0x65,0x02,0x28,0x09,0xed,0x31,0xb3,0x90, -0x38,0xca,0x51,0x38,0xb1,0x20,0x3b,0xf0,0x78,0xd3,0x14,0xf8, -0x47,0xde,0x51,0x44,0x7d,0x20,0xd7,0xb7,0x92,0x42,0x03,0xb9, -0xca,0x2f,0x16,0x1a,0xc8,0x8d,0xec,0xa1,0x55,0x52,0xf0,0x3c, -0xdf,0xc2,0x85,0x72,0x7d,0x65,0x71,0x0a,0x0b,0x46,0x21,0xa2, -0xf0,0xc1,0x5c,0x65,0x17,0x9c,0x10,0x43,0xae,0x12,0x7f,0x85, -0x06,0x72,0x27,0xe9,0x55,0xb8,0x50,0x2e,0x98,0x8b,0xca,0xe0, -0xe7,0xbb,0xbb,0x7c,0x62,0x16,0xfc,0x93,0x8b,0x3e,0x43,0x21, -0xbf,0x39,0x3f,0xbd,0x8b,0x5c,0x46,0xee,0x58,0xfb,0x85,0x0b, -0xe5,0x77,0xe8,0x55,0x93,0xe2,0x5c,0xc8,0xd9,0x00,0xaa,0xd0, -0x36,0xcf,0x1d,0x0a,0xc0,0x15,0x07,0xc4,0x3d,0xe0,0x99,0xe4, -0x73,0xb8,0xdc,0x7a,0x33,0xd3,0x03,0x48,0x90,0x17,0x09,0x59, -0x32,0x2e,0x3f,0xb6,0x0f,0x2d,0xdb,0xb2,0xa0,0x28,0x49,0x57, -0x4f,0x1b,0xd8,0x06,0x53,0x89,0xde,0xe5,0x74,0xdd,0x0e,0xd9, -0xa6,0x06,0x00,0x8a,0x7a,0x30,0x83,0xfc,0x6b,0xe9,0x72,0x53, -0x97,0x23,0x88,0x77,0x4f,0xc2,0xcd,0x97,0x75,0xdd,0xde,0x4b, -0xef,0x87,0x05,0xa2,0xd3,0xc7,0xb3,0xcb,0x9b,0xf9,0xf2,0x89, -0x46,0xc3,0x32,0x5c,0xdc,0xe1,0xc1,0x04,0x7f,0x7d,0x24,0x0a, -0xbc,0x01,0x26,0x3c,0x8c,0xe3,0xd1,0x9d,0x11,0x2e,0xe0,0x5b, -0x1f,0x25,0xbe,0xe2,0xf8,0xc7,0x44,0x13,0x5f,0xac,0xe0,0x1f, -0x13,0x8d,0x9a,0x65,0x05,0xfd,0xf5,0x91,0xac,0xab,0x5e,0xf0, -0x8f,0x8f,0x9e,0x5d,0xff,0x80,0x1b,0x80,0x7f,0xe2,0x68,0x7c, -0xe2,0x29,0x7c,0x30,0x4e,0x3e,0x13,0x3d,0x9f,0x22,0xfc,0xb4, -0xd9,0x4e,0x90,0x35,0x56,0xc8,0x6f,0x2d,0x01,0x5d,0x28,0x16, -0x26,0x5c,0xcb,0xf0,0x7a,0x34,0x2a,0x7c,0x30,0x48,0xae,0x16, -0x05,0xfd,0x0d,0xcb,0xcc,0x87,0xbf,0x16,0xf2,0x1b,0x24,0xf8, -0xf3,0xd7,0x7e,0xd8,0x2c,0xcf,0x70,0xd4,0xf9,0xc7,0x46,0xff, -0x08,0x33,0x5c,0xf0,0x8f,0x8d,0x7e,0x7e,0x55,0xe0,0x9f,0x20, -0x8a,0x85,0x12,0x0b,0x17,0x0a,0x13,0x27,0xab,0x82,0x7f,0x6c, -0xf4,0xab,0xf9,0x0c,0x47,0x0e,0xb1,0x94,0x22,0xf8,0xb2,0x99, -0xde,0x4c,0xef,0x96,0xc3,0xe9,0xe4,0x1f,0x98,0xc5,0x85,0x6d, -0x86,0xb7,0x80,0x6e,0x0c,0x11,0x3c,0x07,0x6c,0xd2,0xe9,0xcd, -0xfc,0x63,0xc1,0x3f,0x41,0x34,0xad,0x10,0xf9,0x0d,0x12,0x3e, -0x4e,0x60,0x0b,0x16,0x1a,0xa8,0x27,0xfd,0x7c,0x43,0x53,0xe6, -0x3f,0xea,0x59,0x9e,0x88,0xad,0x9c,0xe8,0xdb,0x66,0x7c,0xcd, -0x86,0x13,0x0a,0x17,0xb2,0x89,0xe4,0x82,0x46,0xc7,0xd2,0x7f, -0xd8,0x2c,0xb0,0xaf,0xc7,0xb8,0x32,0xe9,0xd7,0x26,0xbc,0x80, -0x8b,0xb3,0xe0,0x9f,0x60,0xfe,0xc9,0x2f,0x42,0xa1,0x01,0x9f, -0xa4,0x84,0xa5,0xa2,0x3d,0xa3,0x70,0x5c,0x1f,0x48,0xa4,0x87, -0xa5,0x42,0x7e,0x5d,0xc2,0xc6,0x1e,0x06,0xb3,0xeb,0x63,0xe5, -0xdc,0x1e,0x4f,0x27,0x43,0xdf,0xc9,0x2a,0xcc,0x45,0x9e,0x44, -0x4d,0x62,0x5e,0x3f,0x7e,0xee,0x01,0x21,0xc1,0x23,0xe5,0x94, -0x14,0x57,0x8b,0xdd,0x63,0xf3,0xa5,0xe7,0x50,0xae,0x48,0x0b, -0x26,0x07,0xd8,0x4b,0xbe,0x7b,0x01,0x28,0x34,0x20,0x54,0xc5, -0xee,0x0f,0x1c,0xf0,0x29,0xe4,0xd5,0xf1,0x99,0x60,0x6d,0xbb, -0x27,0xe6,0xcb,0xe4,0xf1,0xdb,0x67,0xf7,0xa4,0x86,0xb1,0xe4, -0xbb,0xca,0x79,0x2d,0x76,0xd5,0x1e,0x8f,0x4f,0x1b,0xab,0xa7, -0x8e,0x1f,0xc9,0x7b,0x05,0x40,0x78,0x1a,0xc5,0xf8,0xbc,0x82, -0x57,0xed,0x3e,0x0b,0x50,0xa9,0x7c,0x97,0xac,0x3d,0x2d,0xe6, -0x74,0xd6,0xec,0x3e,0xf7,0x1f,0x3e,0x07,0xe9,0xf2,0xed,0xa2, -0x23,0x9c,0x30,0xee,0x07,0xb8,0xab,0x10,0x77,0xa0,0x24,0xf9, -0xf0,0x39,0x54,0xba,0xbb,0xd8,0x55,0xa7,0xe8,0x36,0xed,0x1a, -0xa3,0xaf,0x7d,0x0c,0x3d,0x45,0x17,0xbb,0x6f,0xf0,0xc7,0xc7, -0x3a,0x1e,0x6a,0xb1,0xfb,0x56,0x83,0x3e,0xf5,0xb7,0x62,0xf7, -0x3f,0xfd,0x57,0x05,0xe8,0xee,0xee,0xe9,0xe5,0x38,0x88,0x79, -0x02,0xa7,0xf7,0x35,0xf5,0xec,0xd4,0x7f,0x98,0x1c,0xb3,0xc9, -0xd5,0x15,0x0e,0xca,0x29,0x07,0x7c,0x8a,0xaa,0x59,0xd2,0x9c, -0x15,0xbb,0x67,0xf6,0xd3,0xe4,0x62,0xf7,0x25,0x90,0xce,0x01, -0x9f,0xc2,0x77,0x72,0xb1,0xcb,0xd2,0x2f,0x3e,0x7e,0xf7,0x6e, -0x39,0xfd,0x69,0x35,0x01,0x1c,0x6b,0x77,0xf7,0x27,0x09,0x6a, -0xea,0x06,0xd9,0xab,0x1e,0xc7,0x7b,0xff,0xdb,0x2b,0xb8,0xc4, -0x9e,0x8f,0x1c,0x21,0xd8,0xe9,0xbc,0xff,0xed,0xf9,0xc8,0x48, -0x66,0x01,0xfd,0x30,0x3d,0x51,0x66,0xa2,0x97,0x49,0x0f,0x38, -0x85,0x6f,0x9e,0x9e,0x3c,0x3f,0x7e,0xf1,0xee,0xe4,0xc7,0xe3, -0xb7,0xa7,0x35,0x66,0xe1,0x3b,0xcf,0x81,0xcc,0x85,0x71,0xc8, -0x1a,0xd9,0x9e,0x88,0xa7,0xcf,0x9e,0x63,0x18,0x1a,0x6f,0xe0, -0x05,0x47,0x6e,0xfc,0x43,0xcb,0xcb,0xd7,0xff,0xd7,0xbb,0x1f, -0x8f,0x4f,0xfe,0x43,0xeb,0x01,0xd2,0xf1,0x1f,0xbb,0x87,0xf6, -0x8d,0x71,0x1b,0x9f,0x8c,0x48,0xa1,0xd1,0xa4,0x5a,0x60,0xca, -0x19,0x29,0xc1,0xd3,0x32,0x45,0x04,0xac,0xca,0x99,0x11,0xf3, -0xfc,0xea,0xd5,0xfc,0xd8,0x89,0x2f,0xdd,0x1b,0xfd,0x2e,0x04, -0x41,0x30,0x53,0xf2,0xc1,0x2e,0x06,0x75,0xc7,0xab,0x9c,0x14, -0x09,0x50,0x68,0x1b,0x02,0x2f,0x98,0xbf,0xe2,0x78,0xe9,0xe8, -0xd8,0x92,0x13,0x29,0x28,0xc9,0x14,0x5e,0x8e,0x67,0x24,0x20, -0x5f,0x9a,0x46,0x90,0x6f,0xfa,0xe1,0x6d,0xaf,0x6f,0x6d,0x3f, -0x73,0x75,0x83,0x82,0x22,0x07,0xe4,0x45,0x87,0xce,0x9b,0xd2, -0x76,0x05,0xe5,0xf7,0x76,0x1a,0xba,0xa0,0x16,0x5f,0x19,0x08, -0x72,0x8e,0x8e,0xb0,0x52,0x15,0x7f,0xa6,0x18,0xb4,0xd1,0x42, -0x91,0xfc,0x30,0x91,0xb9,0x1e,0x95,0x07,0xbe,0x4f,0x25,0x1b, -0x33,0xa1,0x52,0x2e,0xf2,0xb1,0xe6,0x3c,0xd2,0x40,0xa7,0x43, -0x20,0xa3,0x47,0x21,0x48,0xed,0x6b,0x8e,0x41,0xe6,0xbb,0xd0, -0xd3,0xa7,0x03,0xf1,0xbe,0x2c,0xe7,0x09,0xe0,0x6a,0x28,0xc8, -0xbb,0x9c,0x03,0x95,0x59,0xf8,0xfe,0xee,0xb8,0xa0,0x19,0x58, -0x68,0xa3,0x19,0xda,0x32,0xd5,0xb1,0x2d,0xdd,0x43,0xa0,0x44, -0xa4,0x99,0x1a,0x54,0x3b,0x32,0x05,0x1e,0x7b,0x48,0x47,0x3e, -0x08,0xbd,0xa0,0x01,0x21,0x9a,0x4a,0xec,0x5d,0x28,0x9c,0xbe, -0xcf,0x66,0x94,0x78,0xd5,0x39,0xe0,0xbf,0xff,0xb6,0xcd,0x90, -0x38,0x3d,0x51,0xdb,0x6c,0x35,0x0b,0xe7,0x41,0x62,0x00,0x70, -0x57,0xbf,0xd6,0x6b,0x1b,0x9f,0x9b,0x55,0xe9,0xf3,0x04,0x45, -0xf3,0xd8,0x94,0xa8,0x29,0x12,0xef,0x98,0xe0,0x99,0x55,0xc2, -0x56,0x30,0x91,0x73,0x65,0xa1,0xf8,0x13,0x2d,0x3c,0xea,0x6b, -0x43,0x56,0x11,0x1a,0x16,0xe9,0x34,0x05,0xdf,0x6e,0x27,0xdf, -0x27,0x3b,0x66,0x82,0x44,0xb2,0x55,0xf8,0x44,0xbc,0x66,0x94, -0xd3,0x3c,0x9b,0x03,0x72,0x0c,0xd4,0xf1,0x8b,0xf9,0x9c,0xf4, -0x13,0xee,0x16,0x5a,0x73,0xd5,0xfa,0x30,0x19,0xb6,0x2a,0x79, -0x59,0xae,0x6b,0x25,0xb4,0x2e,0x3e,0x0b,0xac,0x9d,0xd6,0xe9, -0x78,0x5c,0xb4,0x44,0x5d,0x1c,0xee,0xba,0x58,0x59,0x7c,0xb8, -0x98,0xec,0x47,0x5c,0xf2,0x24,0xd0,0xcf,0x8e,0x06,0x68,0xd3, -0xd4,0x2d,0x79,0x28,0x9d,0x7c,0xd8,0xea,0x82,0x18,0xd2,0x00, -0x2c,0xfc,0xed,0x92,0x55,0x92,0x1f,0xcf,0x5e,0xbe,0x28,0x59, -0xe0,0xac,0xfd,0x2f,0x87,0x7f,0x3a,0x38,0xfa,0x7e,0x9f,0x84, -0xcf,0xf4,0x0d,0x3b,0xc7,0x9c,0xd6,0x03,0x2f,0x7e,0x1b,0x87, -0xb9,0xf2,0x20,0x70,0x3c,0x1a,0xa1,0x59,0x7e,0xb1,0xeb,0x81, -0x79,0x68,0x85,0x52,0x9c,0xf8,0x7e,0x5a,0x0e,0xaf,0xed,0x76, -0x8c,0xda,0xa7,0x57,0xfe,0x33,0xc9,0x96,0xa2,0xe1,0x5e,0x09, -0xab,0x1c,0x1c,0xfb,0x0e,0x21,0xa9,0xbd,0xa6,0x5a,0xdd,0xc0, -0x44,0x4b,0xea,0x84,0xa4,0xf2,0xdc,0xba,0x0a,0x97,0x0e,0xcb, -0xec,0x21,0x14,0x7a,0xf5,0x8b,0x8a,0x3e,0x19,0x0f,0xe1,0x76, -0xf7,0x65,0x39,0x96,0x8f,0x72,0x7c,0x65,0xf1,0xef,0xc2,0x46, -0xbf,0x23,0x6f,0xde,0xfe,0xd4,0x52,0x7c,0xde,0x3d,0x9a,0x7c, -0xaf,0x39,0xac,0x06,0x48,0x50,0xa3,0xdb,0xe8,0x93,0x41,0xad, -0x51,0xaf,0xaf,0xdc,0xce,0x60,0x03,0x1f,0x57,0xb3,0xfc,0x6e, -0xe6,0x96,0x9c,0x0a,0xc9,0x8b,0x58,0x98,0x4d,0x69,0x5c,0xdc, -0x70,0xf1,0xa1,0xe0,0x31,0x2c,0x6f,0x8e,0xff,0x17,0x88,0x48, -0xb3,0xd6,0x68,0x3e,0x0e,0xd6,0x32,0xb1,0x1f,0xff,0x97,0xae, -0xf4,0xff,0xe5,0x58,0x90,0xf2,0x7e,0x3c,0xfe,0x40,0x32,0x2e, -0xdc,0xc2,0xa7,0xec,0x76,0xf8,0x74,0x85,0x9e,0x96,0xb5,0xa9, -0x09,0x67,0x49,0x32,0x71,0xb0,0xf7,0x60,0x5e,0xce,0x02,0xb0, -0x9d,0x33,0x3e,0xab,0xa1,0x49,0xb6,0x76,0x9c,0x14,0x18,0xc3, -0x35,0x82,0x1a,0xf8,0x2d,0xa7,0x78,0xae,0xc6,0xbb,0x71,0xc6, -0xe8,0xd8,0x46,0xa4,0x1e,0x3d,0x15,0x3d,0x9b,0x85,0x83,0x68, -0x4b,0xa1,0x61,0x71,0xd2,0x91,0x61,0xd0,0x64,0x7c,0x7b,0xb0, -0xf1,0x22,0x15,0xfc,0xb4,0xe7,0xdc,0x2d,0xf9,0xaa,0xb9,0x36, -0xdb,0xd4,0xab,0x59,0x86,0xae,0x49,0xbe,0xae,0x7a,0xa9,0xa9, -0xb1,0xfa,0xac,0xb0,0xba,0x5e,0x36,0x85,0x44,0x06,0xa0,0x9e, -0x4d,0x56,0x5b,0x2a,0xf5,0x95,0x6a,0x5c,0x5c,0x88,0x73,0xe8, -0xe7,0x23,0x27,0x0a,0x01,0x88,0x18,0x9b,0xb2,0x31,0x73,0x02, -0xfb,0x95,0x70,0xc1,0xbe,0xcb,0x3e,0xc0,0xe3,0xd6,0x66,0x61, -0x83,0x10,0x08,0xd7,0x09,0x67,0x52,0xf3,0xeb,0x25,0xe9,0x79, -0x9e,0x2f,0x01,0x79,0x33,0x3e,0xb2,0x80,0xba,0x6e,0xb6,0x83, -0x58,0xee,0x6c,0x57,0xef,0xe3,0x76,0xbb,0xa1,0x0c,0xda,0x7b, -0x36,0x37,0x76,0x5e,0xdb,0x2a,0x99,0x1b,0xd5,0x7a,0xb3,0xf2, -0x68,0x00,0x8c,0x89,0xdb,0x78,0x44,0x1b,0x97,0x6b,0x24,0x39, -0xfe,0xcf,0x0e,0xec,0x7a,0xbd,0x77,0xe8,0x44,0x22,0x63,0xad, -0xe2,0x5e,0x30,0x26,0xeb,0x75,0x1a,0xb7,0xd9,0xd7,0xea,0xf1, -0xe9,0x2f,0xcd,0x64,0x79,0xbf,0x09,0xf3,0x18,0x45,0x71,0x7d, -0xd4,0x2f,0x6c,0x7a,0x38,0xf6,0xac,0x37,0x16,0x9f,0x99,0x76, -0xb5,0x45,0x23,0x43,0x16,0xf0,0x1f,0xda,0xf7,0xec,0x5b,0x1b, -0xd9,0x9c,0x84,0x46,0x96,0xf1,0x28,0x20,0x40,0x89,0x29,0x77, -0x34,0x17,0x3e,0xfc,0x59,0xf1,0x58,0x2c,0x3e,0x41,0xa3,0xd3, -0x7f,0x65,0x20,0xbe,0x0c,0xa9,0xa8,0x88,0x56,0x17,0x99,0x79, -0x40,0x8d,0x41,0x72,0x46,0x1a,0x16,0x59,0xaf,0xbf,0xd8,0x48, -0xb6,0xe6,0x8f,0xe3,0xa7,0xcd,0x60,0xdf,0x0a,0x7e,0x00,0xd9, -0x8b,0xa7,0x68,0xd7,0x48,0x03,0xdc,0x0e,0x81,0xac,0xa2,0x4e, -0x64,0xaa,0xb5,0xa9,0xe2,0x9d,0x01,0xc1,0x1d,0xa9,0xcd,0x00, -0x88,0x20,0x79,0xd8,0xda,0xaa,0xfc,0x71,0x58,0x11,0x7b,0xcb, -0xb5,0x51,0xcf,0x69,0x27,0x1a,0xb1,0x55,0x2c,0xcd,0x48,0x8f, -0xf5,0xcf,0x67,0xe7,0xab,0x01,0x0a,0x90,0xb5,0xac,0xab,0x74, -0x28,0xa9,0xe0,0xa8,0xc0,0x63,0xab,0x80,0x6a,0xcf,0x99,0xb0, -0x09,0x97,0x15,0xb7,0x09,0x65,0x4d,0x7c,0xd8,0x9b,0x2b,0xf0, -0x71,0x8d,0x47,0xa9,0x26,0x7b,0xd1,0x1e,0x2f,0x63,0x07,0x38, -0xcf,0x6d,0xfa,0x4f,0x74,0xc9,0x09,0x2c,0x42,0x49,0x82,0xe1, -0x2a,0x21,0x01,0x3d,0xcc,0x13,0x48,0x87,0x38,0x2c,0xe3,0xff, -0xa1,0x9e,0x6d,0x99,0x3c,0x97,0xc1,0xef,0xf2,0xb8,0xf3,0x8d, -0xfd,0x8e,0xfa,0xb4,0xa5,0x2f,0x8c,0x31,0x21,0xc3,0xc1,0x89, -0xa6,0x5a,0x64,0xdb,0x0f,0x78,0x55,0x3a,0x39,0x57,0xaf,0xf1, -0xbd,0xe3,0xb7,0x9a,0x4b,0x15,0x3d,0x0d,0xa3,0xf4,0xe6,0x60, -0xf5,0x54,0xe8,0xb5,0x62,0xdb,0x7f,0x18,0x8a,0x74,0x61,0x23, -0x30,0x84,0x0b,0x61,0xeb,0x02,0xe1,0x5a,0x76,0x2e,0x5c,0xc3, -0xec,0x1c,0x17,0x29,0xba,0xde,0x64,0x1f,0x71,0xae,0xe7,0x56, -0x2e,0xc6,0xef,0xe2,0xdd,0xa4,0x93,0x8a,0x84,0x84,0xe5,0xe6, -0xa2,0x35,0x25,0xfb,0x15,0xd7,0xd9,0x0c,0x8d,0xc8,0x7a,0x39, -0xe8,0x1a,0x68,0x4f,0x27,0x90,0xf8,0x97,0xb2,0x6c,0x50,0xd9, -0x10,0xb3,0xc5,0x46,0xc0,0x92,0x74,0xe9,0x45,0xa8,0x26,0x43, -0x4b,0x3a,0x1c,0xab,0x14,0x33,0xce,0x97,0xc8,0x98,0x07,0x82, -0x6f,0x2c,0xa2,0x6b,0x95,0xf2,0x32,0xab,0xd6,0x72,0x14,0xd7, -0xb2,0x10,0x53,0xab,0x9b,0x40,0x16,0x44,0x54,0x73,0x90,0xed, -0x48,0x26,0x38,0xeb,0xa8,0xc3,0x85,0xcf,0x51,0xfe,0xf0,0xfa, -0xf5,0x8b,0xa7,0xc7,0xaf,0xde,0x1d,0x9f,0x9d,0xbd,0xa5,0xab, -0x3d,0x94,0xb6,0x72,0x17,0x9a,0x29,0xd3,0x6e,0x6b,0xa1,0xa7, -0x2f,0x9e,0xa2,0xe2,0xd3,0xa9,0x2e,0x13,0xb7,0xd2,0x06,0x24, -0x3f,0xaa,0x05,0x0c,0x2b,0x88,0xa9,0x19,0x83,0xac,0xb9,0xe6, -0x31,0x9a,0x20,0x57,0xb0,0xc9,0x50,0x86,0x98,0xa1,0xa0,0x84, -0xb8,0xe2,0x3f,0xd0,0x18,0x2c,0xe9,0x57,0xf8,0xcb,0xb8,0xe5, -0x1a,0xa2,0xad,0xbc,0x29,0xa7,0x73,0xaf,0xfe,0x26,0xda,0x43, -0x78,0x8d,0x52,0x34,0x1a,0x8f,0x47,0xb1,0x8a,0xe1,0x35,0x71, -0x02,0x1d,0xb8,0x28,0xbe,0x0e,0xef,0x12,0xa9,0xd6,0xe9,0x0f, -0x77,0x17,0x68,0xad,0x71,0xe7,0xc0,0x01,0x54,0x17,0xb0,0xa9, -0xfd,0x2c,0x05,0xe8,0xf2,0x52,0xe8,0x3a,0x6f,0xa5,0x34,0xb3, -0xef,0x89,0x52,0xc8,0xfb,0xb7,0xc5,0x4f,0x24,0x04,0x58,0x0b, -0x83,0xbf,0xcb,0xa6,0x0e,0x1f,0x7d,0xdd,0x28,0xc4,0x90,0x51, -0x8c,0x5c,0xca,0x88,0x25,0xe3,0x0f,0x44,0x92,0xe5,0xdb,0xb2, -0x1f,0x6e,0x38,0x85,0x8e,0x95,0x30,0xad,0x2e,0x1f,0xd2,0x0c, -0x64,0xbd,0x3e,0x28,0xcb,0xda,0xa4,0x6c,0xf2,0x90,0x3e,0xe8, -0xb3,0xa3,0x1e,0x19,0x45,0xc4,0x9b,0x43,0xfb,0xf7,0xaa,0x24, -0x18,0xac,0x26,0x14,0x75,0xfc,0xf3,0xe3,0xf2,0xb6,0x9a,0x8c, -0x7b,0xcd,0xeb,0x02,0x39,0x5b,0xcd,0x53,0x6f,0x93,0x1a,0x7a, -0x47,0xc2,0x35,0x45,0x6a,0x31,0xfd,0x08,0x76,0x40,0x05,0xc4, -0xd0,0xc3,0xc4,0x3a,0x7c,0xef,0x2d,0xcc,0x6e,0x05,0xef,0x9f, -0xd4,0x75,0x51,0x52,0xfc,0x0e,0xbb,0xf1,0xf6,0x8d,0x9c,0x82, -0x14,0xca,0x4d,0x9d,0xa9,0xb7,0x23,0xd6,0x26,0x54,0x7b,0xd0, -0x4e,0x10,0x4d,0xb2,0xa8,0xb3,0x46,0xfc,0xee,0xd5,0xcd,0xb1, -0x90,0xf6,0x6e,0x60,0x1a,0x29,0xeb,0xd5,0xa2,0xd2,0xac,0xc0, -0xb8,0x40,0xb7,0xb8,0x5e,0xb0,0x74,0xa6,0xa9,0x38,0x3b,0xca, -0x67,0x49,0x23,0xd0,0x3e,0x63,0xe7,0x57,0x6b,0x76,0x09,0xcd, -0x13,0xbc,0x44,0x51,0x74,0x24,0xa2,0xbc,0xf0,0x2b,0x1b,0xc5, -0x60,0x2f,0xa2,0x77,0xec,0x95,0x33,0x50,0x14,0x21,0x99,0xbe, -0x71,0xaa,0x2a,0xb7,0x22,0x32,0x03,0xd4,0x16,0xb9,0x03,0x87, -0xbd,0xf8,0x64,0x7c,0x39,0x25,0x59,0xb7,0xa3,0xad,0x06,0x68, -0xae,0x66,0xbd,0x54,0x0a,0xa2,0x5f,0x05,0x09,0xe2,0xe5,0xae, -0xb1,0x44,0xbf,0x79,0xc3,0xac,0x0c,0x1d,0xf3,0x36,0x58,0x36, -0x3c,0x3d,0x7b,0xfb,0xfc,0xcd,0xbb,0x93,0xd7,0x2f,0xe9,0xdc, -0x64,0xc1,0x5f,0x6e,0x46,0xc9,0xe5,0x44,0xad,0xef,0x19,0x9c, -0xc8,0x6f,0xff,0x7a,0xea,0x35,0x1a,0x24,0x17,0x6a,0xf8,0x31, -0x56,0xc2,0x39,0xde,0x9d,0xbe,0x79,0xf1,0xfc,0x2c,0xb3,0x5a, -0x54,0xd7,0xd9,0x3d,0x8a,0x12,0x6a,0x8d,0x9c,0xcf,0x64,0x80, -0x65,0x8d,0x13,0xb3,0xac,0xd0,0x83,0x81,0xdc,0x0a,0xd2,0x15, -0xbe,0xb4,0x67,0xa2,0xfa,0x07,0x34,0x97,0xef,0x70,0xa9,0x1d, -0x77,0x06,0xfe,0x88,0x36,0xc6,0xb1,0x2b,0x5d,0xef,0xf7,0x0e, -0x73,0x2b,0x86,0x79,0x35,0xeb,0x63,0xfa,0x20,0x4f,0xae,0x66, -0xd0,0x51,0x33,0x8a,0x6c,0x51,0xf2,0x80,0x86,0x1e,0xe6,0x3f, -0x2c,0x44,0xd9,0xc9,0x2e,0xb6,0x94,0x88,0xef,0x8f,0x58,0xbc, -0xfd,0x6c,0x8e,0x12,0x06,0x86,0xd5,0x2e,0x4c,0x0f,0x21,0x15, -0x46,0xf2,0x9e,0x52,0x17,0x5d,0x33,0xf4,0x8d,0xa3,0xdb,0x0c, -0x7d,0xd1,0x73,0x5a,0x2f,0x98,0x31,0xb4,0xcf,0xe6,0x80,0x66, -0x8e,0xd2,0xd2,0x28,0x03,0xf6,0x7f,0x58,0x2d,0x46,0x7d,0xc7, -0x0b,0xe4,0xa9,0xde,0x31,0x36,0x10,0x3a,0x8e,0x96,0x14,0x6f, -0x79,0xc0,0xc7,0x65,0x6c,0xb4,0x9e,0xbf,0x4a,0x0d,0x3d,0xf7, -0x72,0xfd,0x24,0xde,0x34,0x11,0xf7,0x4d,0x5a,0x28,0xdf,0x71, -0x1f,0xdd,0x5d,0x74,0xbc,0xcd,0xac,0x24,0x27,0x43,0xa6,0xdc, -0xa4,0xc5,0x35,0x9a,0xbb,0x4e,0xf4,0xa5,0x47,0x84,0xe9,0x6e, -0xef,0xaa,0x95,0x98,0x89,0x6d,0x61,0xe9,0x96,0x08,0x70,0xb5, -0x58,0x8a,0xcf,0x89,0xb1,0xa9,0xbd,0x31,0x29,0xcd,0xe4,0x2a, -0xe9,0xe9,0x68,0xd4,0xe9,0xdd,0xd5,0xd5,0xe4,0xd3,0xc0,0xb5, -0xfb,0x9d,0xd1,0x4b,0x16,0x81,0x4b,0x2b,0xa3,0x69,0x84,0x20, -0xc3,0xd1,0xbb,0xdf,0x25,0x4f,0xb5,0x9a,0x6b,0x13,0xd8,0x78, -0x23,0x91,0xb1,0x54,0xcd,0x35,0xab,0x3c,0xad,0x9f,0x7d,0x5b, -0x51,0xdf,0xca,0xd1,0x79,0x21,0x10,0xa7,0x76,0xe0,0x8c,0x94, -0x4d,0x9a,0x46,0xd8,0x82,0x0f,0xde,0xc4,0x58,0x95,0xc3,0xa2, -0x95,0x4d,0x95,0x87,0xe6,0xce,0xb2,0xc0,0x96,0x01,0x55,0x12, -0x40,0xa8,0x0f,0xab,0x92,0xfa,0xca,0x35,0xaf,0xa5,0x78,0x88, -0xa8,0x70,0x4b,0xef,0x67,0xa9,0x8c,0xcf,0x2b,0x7a,0xc9,0xc2, -0xd8,0x67,0xea,0xa2,0x6b,0x39,0x71,0x0f,0x9d,0xf5,0x85,0x85, -0xde,0x9f,0x4d,0xd1,0x68,0x46,0xb3,0x1c,0x4b,0xe3,0xa4,0x94, -0x16,0x0c,0xad,0xb5,0xa3,0x5a,0x4c,0xec,0x43,0x09,0x33,0x38, -0x99,0x45,0xed,0xcc,0xf3,0x48,0x55,0x45,0x2b,0xc8,0x2d,0x38, -0xa3,0xd8,0xd2,0x5c,0x4a,0x7a,0x48,0x06,0x92,0x61,0xd5,0xe8, -0x16,0x2d,0x6c,0x95,0x1b,0x8b,0x40,0xa3,0xcc,0xa7,0xa8,0x53, -0xc4,0xc7,0x0b,0xd9,0xbd,0x32,0x62,0x99,0xb1,0xfa,0x6b,0x90, -0xbd,0xae,0x8b,0x81,0xac,0x04,0x84,0x3e,0x56,0xf8,0x34,0xa6, -0x92,0x9a,0xdd,0x87,0x49,0x70,0x9d,0x49,0x12,0x1e,0x84,0x24, -0x32,0x65,0x1f,0x0c,0x5c,0x29,0x2f,0x59,0xfc,0x6c,0x56,0x86, -0x02,0x4f,0x92,0xe7,0xc8,0x37,0xd9,0x4b,0xfa,0x89,0x29,0x87, -0x7a,0x67,0x9f,0xcd,0xba,0x4e,0xe6,0x36,0xd3,0x6c,0x2e,0xc9, -0x4b,0x8d,0x1a,0x96,0xb9,0x91,0x5b,0xf5,0x19,0xad,0xac,0x29, -0x32,0xd5,0x27,0x93,0xd2,0x44,0x39,0x62,0x71,0x22,0xf2,0xe3, -0xf7,0x1e,0x12,0x99,0xc4,0x08,0x44,0x89,0x07,0xee,0xb0,0x6c, -0x5e,0x96,0xbe,0x1c,0x2a,0x5b,0x1f,0x69,0x9e,0xbe,0x89,0x3f, -0x1c,0x0c,0xe4,0x49,0xcd,0x49,0x20,0x9b,0xd4,0x47,0x03,0xb5, -0x74,0x64,0x8e,0x61,0x19,0xbe,0x5e,0x24,0x1e,0xd9,0x70,0xe2, -0xd2,0x42,0xd3,0x19,0x71,0xd7,0xe3,0x3f,0x5b,0xde,0xde,0x85, -0xb2,0x02,0x54,0xa4,0xdc,0x08,0xc7,0xf1,0xf9,0x1d,0xd5,0xe5, -0x44,0xc0,0x65,0x22,0x44,0x4d,0xcf,0x6a,0x2d,0x8c,0xbb,0x62, -0x21,0xb8,0xdd,0x1e,0xa3,0x23,0xae,0xcb,0x5f,0xdb,0xed,0xbd, -0x43,0x40,0xc0,0xf9,0xc3,0xb1,0x8d,0x5c,0x3e,0x04,0x0a,0x64, -0x6a,0xd7,0xd8,0x4c,0x4e,0x3a,0x92,0xdb,0x5d,0xd0,0xfe,0x02, -0xe1,0x1a,0x97,0x28,0x8c,0xfa,0x6c,0x38,0x21,0x09,0xeb,0x79, -0xcb,0x1c,0x99,0x2a,0x39,0x0d,0xf7,0x4a,0x6b,0x74,0x37,0x86, -0xd4,0xe2,0x7c,0x46,0x1a,0x1f,0xaa,0x6f,0xcb,0xa0,0x01,0xd5, -0xd7,0x2a,0x21,0x88,0xde,0x2d,0x32,0x2f,0x12,0x5d,0xc7,0x07, -0x58,0xbc,0xd4,0xe1,0x05,0x24,0xcf,0xe2,0x05,0xfd,0x2d,0x75, -0x7c,0x2a,0xd7,0x83,0x39,0xcb,0x68,0x6b,0x52,0x91,0x58,0xb0, -0xde,0x66,0xf2,0xb9,0xfa,0x26,0x1a,0xb5,0x0f,0x9f,0xbf,0x3a, -0x3d,0x3b,0x7e,0x75,0xf6,0xfc,0xf8,0xec,0xf9,0xab,0xbf,0x6e, -0xbb,0x59,0x2f,0x47,0x63,0x54,0x0b,0x3e,0x99,0x2c,0xd1,0x61, -0xcf,0xd2,0x8b,0x87,0x7f,0x86,0xf3,0x03,0xf0,0xb1,0x02,0x87, -0x24,0x21,0x05,0x0e,0x51,0x1e,0x6f,0x7d,0xbf,0x87,0x1c,0x2d, -0x27,0x82,0x5b,0xab,0x7a,0x63,0x0c,0x2a,0x50,0x39,0xd5,0x0b, -0xb4,0xcd,0xce,0x1b,0x9a,0x1c,0x34,0xb8,0x29,0x83,0xde,0x52, -0x16,0xd0,0xe6,0x0a,0x1f,0x84,0xa7,0x9f,0xef,0x59,0x55,0x84, -0xdf,0xf9,0xad,0x0d,0x4e,0x5e,0xc7,0x80,0xbe,0x91,0x01,0x17, -0x92,0xca,0x14,0x72,0x5e,0x55,0x46,0x88,0x9f,0x8c,0xaf,0x55, -0x88,0x32,0x2b,0x36,0x68,0xb1,0x74,0x76,0xf6,0x59,0x1e,0xa8, -0x1f,0x54,0x45,0x4b,0xe5,0xb8,0x50,0xf7,0x9f,0x7c,0x64,0xb0, -0xa1,0x06,0xcd,0x83,0xc7,0x44,0xcd,0xc9,0x30,0x22,0x71,0x5b, -0xe6,0x63,0xb2,0xfa,0x15,0x90,0xcc,0xe4,0xf9,0x0c,0xee,0x07, -0x14,0xba,0x12,0xd5,0x48,0x36,0x79,0xf0,0xeb,0x78,0xb6,0xd3, -0x7a,0xfa,0x69,0x01,0x11,0xb0,0x82,0x65,0x14,0x58,0x41,0x60, -0x58,0xb5,0xb8,0x16,0xd6,0xe8,0xa1,0x49,0x23,0x5e,0x34,0xf6, -0x8b,0xb7,0x38,0x77,0xbd,0xdd,0xe6,0xdf,0x26,0xab,0x4c,0x3d, -0x4e,0x22,0xf6,0x70,0x61,0x16,0x25,0x59,0x87,0xd8,0x54,0x24, -0x31,0x97,0x7a,0xdc,0x1b,0x68,0x8d,0xab,0x59,0x89,0xd8,0x34, -0x0d,0xc0,0x20,0xa3,0x21,0xee,0xed,0x1d,0x16,0x43,0x6f,0xd9, -0x27,0xbb,0x47,0xe1,0xdc,0xd6,0x41,0xa1,0xb7,0x12,0xdc,0xae, -0x47,0x14,0x75,0x68,0xa2,0x86,0x7a,0x4a,0x52,0xca,0xa3,0x7a, -0x0a,0xcd,0x0f,0xda,0x97,0xe0,0x1c,0xdf,0x6e,0xcf,0xc1,0xbf, -0x8f,0x34,0xe7,0x77,0x5f,0xce,0xc9,0xbf,0xdf,0x6a,0x89,0x3f, -0x7e,0x7d,0x09,0xfe,0xfd,0x4e,0x4b,0xfe,0xe9,0xf7,0x97,0xe4, -0xdf,0x3f,0x2a,0x84,0x7f,0xfb,0xe7,0x21,0xf0,0xef,0x9f,0x14, -0xd2,0x9f,0xff,0xf7,0x21,0xf1,0xef,0xbf,0x29,0xc4,0xbf,0xfc, -0xf7,0x41,0xe4,0xdf,0x3f,0x2b,0xe4,0xc3,0x83,0xff,0x7e,0xd0, -0xfc,0xfb,0x97,0x01,0xba,0xd1,0x65,0x99,0x4f,0x57,0x45,0x64, -0xd0,0xa9,0x0a,0x8f,0x0b,0x8f,0x46,0x23,0xcd,0x1f,0x1c,0x18, -0x8a,0xcd,0xe5,0x0c,0x6a,0x3c,0x62,0x0b,0x1e,0x27,0x1e,0xdd, -0xb6,0x28,0xa4,0xe3,0x95,0x98,0x74,0xef,0xc4,0xb3,0x74,0x26, -0x0a,0xcf,0xe8,0xdd,0x19,0xff,0xf6,0xf1,0x8f,0xbf,0x16,0x0b, -0x4a,0xf1,0x45,0x1c,0x42,0x4d,0xe6,0x05,0x0d,0xd4,0xb0,0x3d, -0x82,0xa5,0x70,0xf3,0x5d,0x93,0xa5,0x1f,0x4e,0x86,0x3f,0x0d, -0xca,0x00,0x3e,0x60,0x3f,0x0b,0x2d,0x26,0x4a,0x66,0xf7,0x0c, -0x52,0x14,0x3d,0x72,0x33,0x48,0x85,0x09,0xa3,0xa0,0x99,0x39, -0x36,0x72,0x3d,0x42,0x0b,0x0d,0xe4,0x70,0xe6,0x14,0x91,0xb5, -0x88,0x26,0xe2,0xac,0x49,0x91,0x2c,0xc6,0xec,0xd7,0xeb,0xc6, -0xab,0x91,0x99,0x04,0x1b,0x12,0x49,0x0a,0xee,0x93,0x92,0x3d, -0xeb,0x1b,0x10,0x65,0xe2,0x55,0x54,0xf0,0xf2,0xc0,0xd3,0x3f, -0xc0,0x76,0x69,0x98,0x85,0xc5,0x93,0x07,0x0d,0x2c,0x9d,0x14, -0x7e,0xe1,0xe8,0x9f,0x22,0xa4,0xee,0x35,0xda,0xab,0x1d,0x85, -0xe9,0x8a,0x05,0x38,0x5d,0xa3,0x30,0x59,0x62,0x55,0xc1,0x28, -0x4c,0x94,0x57,0x4b,0xa7,0x55,0x14,0xa6,0x6a,0x74,0x96,0x3b, -0xea,0xaa,0x70,0xa1,0x8d,0x1f,0x05,0x45,0x4b,0xca,0x70,0xec, -0x77,0x6b,0x4a,0xfb,0x11,0x16,0x13,0x64,0xb7,0x4a,0x7b,0x5b, -0xae,0xb4,0xbb,0xd9,0x02,0xf1,0xae,0x9f,0x66,0xbf,0xce,0xe6, -0x1f,0xfd,0x4c,0x6f,0xc5,0x2c,0xf0,0xf1,0xd3,0x52,0x8e,0x38, -0x73,0x31,0x1d,0x55,0x06,0x39,0xbe,0xdc,0xe6,0x20,0xbb,0xb1, -0x0f,0xc0,0xa3,0xec,0x9f,0x20,0x1e,0xc6,0xe8,0x4d,0xf6,0x78, -0x39,0x7e,0x89,0xe0,0x5b,0x58,0x42,0xd3,0x4d,0x01,0xba,0x1a, -0x8d,0xa8,0xb5,0xed,0x84,0x5e,0xc8,0x59,0xde,0x56,0x11,0x7a, -0xa3,0x46,0xe5,0x21,0x3b,0x8c,0x9a,0xc7,0x1f,0x73,0x8d,0x42, -0xda,0x42,0xf6,0xa2,0xd7,0x47,0x4e,0x00,0x74,0xe2,0x29,0x1b, -0x2f,0x28,0x77,0x0e,0x8e,0x88,0x9f,0x39,0x9a,0x54,0x18,0x71, -0x6c,0xf3,0xd8,0x43,0xaf,0xb9,0xf0,0xe1,0x86,0xb9,0xa1,0x44, -0x65,0xf7,0x13,0x11,0xef,0x45,0x9b,0x0d,0x2a,0xe6,0xac,0x56, -0x1e,0xc4,0xe4,0x83,0x67,0x79,0x88,0x42,0xab,0xcb,0x68,0xa4, -0x9a,0x43,0x0c,0xfa,0x19,0x8a,0x8f,0x71,0xb7,0x52,0xb6,0xe9, -0xec,0x0d,0x42,0x13,0x5f,0xbc,0x36,0xd2,0x90,0xa9,0xc1,0x10, -0x0b,0x97,0x58,0xaf,0x93,0x21,0x1a,0xa8,0x0c,0xed,0x8a,0xd9, -0xb7,0x24,0x64,0xb5,0x0a,0x74,0x27,0x23,0xa6,0xf6,0xa0,0x0d, -0xdb,0x87,0xc6,0x42,0x46,0x76,0x3c,0xbd,0xc5,0x23,0x10,0x10, -0x48,0xed,0x0d,0x1e,0x5f,0x37,0x29,0x0a,0x24,0x55,0x37,0x3d, -0xa8,0xe2,0xb6,0xdc,0x66,0x27,0x07,0x73,0x64,0xe8,0x9d,0x4a, -0x1d,0xa5,0xc2,0x1a,0x9f,0xff,0x6d,0x32,0xfe,0x88,0xb6,0x1d, -0xb0,0x60,0x34,0x04,0x0d,0x70,0xaa,0x1f,0x3e,0xd3,0xcb,0x1b, -0x83,0xda,0x02,0x8b,0x9c,0x8d,0x02,0xf5,0x80,0x99,0xda,0x6d, -0x19,0x7e,0xef,0x4b,0xed,0x20,0x3f,0xc8,0x8a,0xc6,0xd8,0x8d, -0x78,0xd7,0x62,0x53,0x84,0x9a,0x45,0x23,0x9c,0x6e,0x7c,0xc3, -0xfa,0x80,0x6a,0xdc,0x9c,0x8a,0xe9,0xf1,0xb4,0xfe,0x4c,0x13, -0x8f,0xd9,0xc6,0x1e,0x3d,0x16,0x80,0xb3,0x65,0x9e,0x72,0xfb, -0xc8,0x35,0x01,0x85,0x36,0x46,0xac,0x44,0x94,0x0d,0x6a,0xfe, -0x9f,0x50,0xb8,0xde,0x49,0xb6,0x9b,0x15,0xa6,0x0e,0xc7,0x5f, -0xdf,0xad,0x70,0x53,0xa1,0x0b,0x36,0xd4,0xc1,0x40,0xab,0x1b, -0xb8,0x1d,0x91,0xe6,0x71,0x98,0x05,0x71,0x75,0x9a,0x6c,0x65, -0x1e,0x66,0x9e,0x5e,0x41,0x5b,0xb7,0x35,0x58,0x27,0xa8,0xb8, -0xb6,0xb7,0x97,0xa3,0x33,0x9a,0x2d,0xa9,0x2c,0xea,0xdc,0x90, -0xa8,0xea,0x6f,0xee,0x3d,0x17,0xdb,0xf4,0x50,0xbe,0x05,0x3a, -0x41,0x48,0x0d,0xf5,0x8e,0x7d,0xef,0x92,0xcf,0xa1,0x94,0x2e, -0xd1,0xd0,0x5e,0xe1,0x1b,0x7e,0x13,0x57,0xf7,0xd6,0x28,0x40, -0x2d,0x72,0xfb,0xd9,0xfd,0x8e,0x1f,0xa7,0x9b,0xf1,0xe5,0xaf, -0xa9,0x7f,0x0b,0x59,0x40,0xa9,0x67,0x33,0x23,0x48,0xc7,0x11, -0xd9,0x3d,0xff,0xa6,0x64,0xa8,0x0c,0x57,0x11,0x83,0x2a,0x3d, -0xd4,0x94,0x40,0xe5,0xce,0x9d,0xf6,0x26,0xf0,0x96,0x32,0x59, -0x8e,0x7f,0x5a,0x4e,0x59,0x67,0x06,0xaa,0x43,0x8e,0xbd,0x4c, -0x29,0x44,0xef,0x94,0x88,0xdc,0x75,0xef,0x96,0x53,0x74,0x13, -0x9e,0x86,0x89,0x26,0xcd,0x3d,0x63,0xdc,0x29,0x28,0x95,0xa4, -0x33,0x0d,0x9e,0x4a,0x14,0x54,0x22,0xa1,0xd4,0x43,0x20,0xf9, -0x38,0x96,0x97,0x9f,0x5e,0x91,0x87,0xca,0x7c,0x39,0xfc,0xa8, -0x72,0xa7,0x6e,0x37,0x23,0x42,0xab,0xeb,0x57,0x4d,0x81,0xb8, -0x13,0x0d,0x0a,0x21,0x2a,0xa0,0xf1,0xf2,0x69,0xc6,0x57,0x53, -0x7c,0x4c,0x4e,0xee,0xc3,0xa3,0x64,0x1b,0x97,0x23,0x19,0x0f, -0x93,0xfe,0x64,0x7c,0x85,0xfe,0xc4,0x49,0xbf,0x9f,0x5a,0x3d, -0xa9,0x5e,0xce,0x2f,0x7f,0x2d,0xc5,0x95,0xd2,0x96,0x35,0x06, -0xa4,0xee,0x03,0x0b,0x07,0x79,0x8c,0x04,0x6b,0x77,0x77,0xfb, -0xae,0x28,0xb7,0x27,0xe5,0x52,0x78,0x32,0xbb,0xac,0x27,0x72, -0xfd,0x66,0x6b,0x6f,0x69,0x62,0xa7,0xb3,0x61,0x38,0x80,0x5a, -0x4e,0xae,0x3e,0xa3,0xba,0xd7,0xab,0x79,0x1d,0x5c,0xe5,0x41, -0xa9,0x7e,0xe8,0xef,0x5b,0x9b,0x0f,0xec,0xc4,0x9e,0x42,0x84, -0x63,0xf3,0xc1,0x7d,0x46,0x96,0xd7,0xb5,0xf6,0xcd,0x11,0xfb, -0xa2,0x76,0x4b,0x3e,0x97,0x66,0xb8,0x61,0x1d,0x8e,0x46,0x6f, -0x28,0xaa,0x0c,0x6e,0x7c,0xf7,0x14,0xe4,0x1f,0xe2,0x0d,0x14, -0x58,0xe7,0x76,0x97,0x1e,0x1e,0x1c,0xd8,0x0d,0xaa,0x75,0x70, -0x5b,0x00,0x5a,0x7e,0x35,0xe3,0x86,0x44,0x7b,0xc3,0x1f,0xb2, -0xcb,0xf1,0x55,0x7e,0x01,0x17,0x9f,0x1a,0xe9,0x72,0x17,0x0a, -0x0b,0x91,0x60,0x52,0x82,0xf7,0x1d,0x3d,0xe7,0x8d,0xb0,0x2c, -0x5d,0xb2,0xba,0x35,0x7c,0xdb,0xe1,0x43,0xb3,0x11,0x77,0x04, -0xbe,0xe9,0x37,0xaa,0xb9,0xc4,0x78,0xee,0xa3,0x5e,0x16,0x51, -0x0e,0x04,0xa4,0x27,0xb3,0xee,0x92,0x9e,0x40,0xee,0xc9,0xb7, -0xbe,0x2f,0x9e,0x12,0xef,0x86,0x0e,0xe2,0x24,0xc9,0x11,0x74, -0x91,0x6a,0x16,0x1c,0x83,0x7a,0xba,0xed,0x2c,0x99,0x35,0x4b, -0x13,0x1c,0x83,0x64,0x5b,0x3c,0x5a,0x16,0xd6,0xca,0x53,0x37, -0x6c,0xfa,0xbc,0x49,0x10,0xed,0xe0,0x70,0x13,0x82,0xe1,0x2d, -0xcd,0xc8,0xb8,0x01,0x64,0x26,0xca,0xc6,0xd9,0xd8,0x70,0xa9, -0xeb,0x75,0x50,0xd8,0x9a,0x28,0xfc,0x37,0x14,0x45,0xfb,0x43, -0x22,0x4b,0xab,0x7e,0x96,0x21,0x65,0xe3,0x62,0x51,0xcf,0x13, -0xb7,0x3f,0x4d,0xd4,0x7c,0xe6,0x0e,0xd1,0xa6,0x8d,0x22,0xad, -0x08,0xca,0xe2,0x0b,0x73,0x34,0x09,0xed,0xb6,0x48,0x24,0x89, -0x2a,0x35,0x80,0x4d,0x13,0xb8,0x5d,0xd0,0x9d,0x2e,0x22,0x73, -0xf6,0xac,0xce,0xcc,0x14,0xc2,0x3d,0x7e,0x49,0x91,0xbd,0x86, -0xf2,0x3e,0x35,0x86,0x50,0x84,0xbb,0x24,0x8d,0xe0,0x47,0x3d, -0x3d,0x30,0x31,0x6e,0x44,0xa2,0x3d,0xe9,0x74,0xc7,0xe5,0x4c, -0xc1,0x29,0xff,0x11,0x67,0x28,0x7a,0x11,0xa2,0x59,0xdb,0xb6, -0x1e,0x74,0xd5,0xe2,0x57,0x2f,0x9c,0xa2,0x5f,0x50,0x41,0xa3, -0xea,0x9d,0x17,0xe7,0xfb,0xe7,0xfb,0xfd,0x5f,0xce,0xf7,0x07, -0xdf,0xb0,0xfb,0xc0,0x24,0xf1,0xbb,0xf0,0x04,0xf5,0x40,0xc8, -0x12,0x4b,0xee,0x3f,0xf9,0x61,0xa5,0x84,0x35,0x7a,0x49,0xdf, -0x6f,0x86,0xa4,0xb0,0x64,0xda,0x98,0x66,0x3c,0x95,0x97,0x52, -0x3c,0x34,0xdf,0x62,0xdd,0xb6,0x52,0x06,0x55,0xd4,0xa2,0x0f, -0x56,0xc7,0xe7,0x70,0x3e,0x61,0x43,0xe3,0xa4,0xf6,0x32,0x53, -0xfe,0xb3,0xb9,0xcc,0xa4,0x06,0x44,0x8a,0xa3,0xe6,0x11,0x4a, -0x52,0x6b,0x73,0xbd,0xa8,0xad,0xb5,0x06,0x44,0xa5,0x1a,0x8f, -0x5a,0x62,0xe7,0xc7,0x8c,0x07,0xcb,0xf4,0x99,0xd2,0x91,0x6b, -0x13,0x4e,0x29,0x4d,0x06,0xe4,0xbb,0xb2,0xa7,0x12,0x8e,0xf4, -0x82,0xa7,0x68,0x20,0x97,0x3d,0x94,0x1c,0xf0,0xb8,0x90,0x45, -0xb4,0xbb,0xd9,0xb8,0xba,0x1c,0x2e,0xc6,0x02,0x4a,0x3c,0x20, -0xe0,0x9b,0xd6,0x81,0xda,0x5f,0x3f,0x32,0x8d,0x92,0xd7,0xcc, -0x50,0x96,0xa4,0x9e,0xbe,0x1d,0x2a,0xbb,0x9e,0x3c,0xcc,0xbc, -0x1d,0x1f,0x53,0x7a,0x53,0x77,0xda,0x5a,0x1f,0xcc,0x52,0x40, -0xd3,0x54,0x75,0x92,0xf2,0x88,0x78,0x18,0x49,0xc7,0xaf,0x93, -0x4e,0x82,0x72,0xf0,0xf8,0x66,0x56,0x9e,0xdd,0xdc,0xe5,0xad, -0x83,0xc3,0xd6,0xbf,0x0f,0x67,0xad,0xc3,0xbf,0xfc,0xdb,0x41, -0xeb,0xe0,0xa0,0xa0,0xff,0x5b,0x7f,0x7d,0x79,0x66,0x1c,0x55, -0x3a,0x0b,0xe1,0x76,0xb1,0x94,0x0d,0xab,0x20,0xae,0x3d,0xe9, -0x48,0x04,0x43,0x80,0xaa,0x6b,0xcd,0x51,0x25,0xb5,0xce,0x61, -0x6e,0xa1,0x3f,0xfe,0xee,0xe0,0x2f,0x7f,0x02,0x12,0x00,0x71, -0xcf,0x8f,0x43,0x14,0x2a,0xe6,0x51,0x68,0xfd,0x41,0x8d,0x69, -0xfe,0x01,0x6e,0xcb,0xaa,0x9a,0x5c,0x4c,0x3f,0xb3,0xea,0xc8, -0x78,0x85,0xf6,0x70,0xe6,0x1f,0xc6,0xcb,0x2b,0x24,0xcf,0x46, -0xad,0x8b,0xf1,0xe5,0x10,0x7d,0xb5,0x4e,0x56,0xad,0x8f,0x68, -0xd2,0x66,0x3e,0x87,0xd1,0x5c,0x5e,0x8f,0x5b,0xa9,0xd6,0xcf, -0x55,0x75,0x92,0xd6,0xe3,0x16,0x56,0xd7,0xba,0xf8,0xbc,0x1a, -0x57,0xd9,0x0e,0x72,0x1b,0x78,0xc7,0x8f,0x10,0x4f,0xb2,0xf7, -0x2d,0x8a,0x42,0xa1,0x4c,0x0f,0x19,0x76,0xe4,0x5b,0xf4,0xf9, -0xc8,0xe9,0xc4,0x6d,0xc3,0x49,0x72,0x97,0xd5,0x22,0xb2,0xe6, -0x14,0x11,0x01,0xab,0x08,0x3b,0xeb,0xbb,0x62,0x83,0xfc,0x61, -0x42,0x63,0xc3,0xed,0x42,0x2b,0xfe,0x31,0x8a,0x67,0x80,0xa0, -0xc0,0x9c,0xfb,0xb2,0x3d,0x14,0x81,0x40,0xdf,0xd1,0x11,0x17, -0xf6,0x7c,0xb8,0x08,0xa6,0xa4,0x0f,0x7a,0xe9,0x96,0x96,0x6b, -0x86,0x00,0x23,0x75,0x60,0x1f,0xea,0x0d,0x71,0x29,0xf0,0x19, -0xb9,0xd8,0x39,0x34,0xb4,0x46,0xac,0x06,0x4e,0x8c,0xa5,0x2d, -0xbc,0x83,0x6b,0xfc,0x91,0xeb,0x04,0x83,0xce,0xcc,0x66,0x23, -0x03,0xc1,0xd3,0x75,0x5e,0x2d,0xdc,0xab,0xfd,0x8e,0x3f,0x3a, -0x9a,0xd0,0x95,0xd9,0x42,0x15,0x5a,0x3a,0xb2,0x51,0x2d,0x3d, -0x68,0x73,0x93,0x59,0x2b,0xab,0xd9,0xce,0xaf,0x77,0xcf,0x47, -0x62,0x06,0x23,0xd4,0xcd,0xbd,0x82,0xdd,0x8b,0x06,0xa1,0x57, -0xf8,0x60,0x48,0x3f,0x3b,0x25,0xc5,0x3d,0x9d,0xe1,0x91,0x03, -0x63,0x3a,0x1d,0x43,0xb0,0xa7,0x81,0xb2,0xa4,0x4c,0x26,0x89, -0x23,0xba,0xb3,0xac,0x08,0x63,0x72,0xa0,0xc0,0x7f,0x4d,0x25, -0x31,0xe7,0x5f,0x98,0x0f,0x1f,0x9b,0x6b,0x3d,0x99,0x0b,0x95, -0x61,0x42,0x97,0xc5,0x11,0x0d,0x59,0x46,0xa5,0x51,0x82,0xee, -0x29,0x65,0x44,0xd1,0xc3,0xa7,0xdc,0x76,0x17,0xb9,0x53,0xba, -0x58,0x74,0xfd,0xa5,0xd1,0x36,0xdc,0x5d,0xf8,0x3c,0x59,0x6e, -0xb3,0xbb,0x30,0x56,0xad,0xd9,0x33,0xd2,0x5a,0x94,0x51,0x44, -0xf7,0x23,0x14,0xac,0x19,0xe7,0xb2,0x63,0x8e,0x06,0xba,0x26, -0x93,0x11,0xf9,0x67,0xe2,0x62,0xce,0x3e,0xd7,0x50,0xac,0xad, -0xae,0x86,0xf8,0xdc,0x96,0xe4,0x02,0x97,0x15,0xd2,0xaa,0xc9, -0x3f,0xc6,0xa8,0x3b,0x0c,0x88,0x4d,0x55,0x8a,0x02,0x05,0xdc, -0x4f,0x32,0x73,0xf9,0xfd,0x64,0x54,0x48,0x7e,0x34,0x67,0xc9, -0xba,0x1c,0x00,0x61,0x31,0xbc,0x9c,0xac,0x3e,0x97,0x92,0xad, -0xdd,0x96,0x40,0x57,0x53,0xd6,0x6b,0xb6,0x9e,0xd3,0x7d,0x79, -0xfc,0xf7,0x77,0x7f,0x3b,0x7e,0xf1,0xd3,0xd3,0x7c,0xba,0xbc, -0x43,0xd6,0x33,0x96,0x77,0xa3,0xcf,0x7c,0x05,0x9d,0x45,0xcb, -0xd3,0xe2,0x1e,0xf7,0xa5,0xee,0x41,0x79,0x8f,0x36,0x5d,0x9a, -0xe4,0xc9,0x08,0xd9,0x58,0xde,0xd1,0xb8,0x95,0x52,0x07,0xbd, -0xf7,0x01,0x5a,0x67,0x3f,0xcb,0x7b,0xf8,0x8b,0x02,0x90,0x9e, -0x43,0xa9,0x8b,0x51,0x8b,0x87,0x32,0xc1,0xea,0xa0,0x82,0x84, -0xce,0x0a,0x75,0x04,0xc3,0x6a,0x30,0x38,0x6a,0x70,0x36,0x46, -0x9a,0x2c,0x39,0x46,0x3f,0xd6,0x21,0x68,0xb7,0x69,0xc7,0xb0, -0x52,0x9b,0x5b,0xbc,0x5d,0xd2,0xbc,0x11,0x5b,0xc0,0xf4,0xbe, -0x60,0x3b,0xf5,0x40,0x77,0x10,0x8b,0x71,0x0d,0xdd,0xda,0x01, -0xd7,0xa2,0x8d,0x28,0x1c,0x7f,0x35,0x78,0x8d,0xc5,0x6b,0x5f, -0x33,0xd8,0x8d,0xe9,0xe6,0x4c,0x53,0x69,0x6f,0xb9,0x9c,0xda, -0x3f,0xbb,0x4f,0x5d,0xce,0x99,0xec,0x42,0x1f,0x91,0x5b,0x28, -0x72,0x10,0xdb,0xe6,0x68,0x9c,0xeb,0x0f,0x8d,0xed,0xde,0x5e, -0xa6,0xfd,0x3a,0x9e,0x4e,0xad,0xb3,0x13,0x5d,0x99,0xb2,0x9c, -0x9b,0xd6,0x5a,0xb0,0xcc,0x36,0xb9,0xe8,0xc0,0xd5,0x80,0xe8, -0x92,0x84,0xed,0x40,0x41,0x05,0x45,0x1f,0xd2,0xaa,0x68,0x71, -0x22,0x37,0xff,0x6a,0xde,0xe0,0x7b,0xc5,0xef,0x27,0x02,0x98, -0xdf,0x63,0xf3,0x0a,0xfc,0xb3,0xd1,0x27,0x1d,0x06,0x65,0x8c, -0x15,0xda,0x4d,0xdd,0x45,0xb8,0x31,0xca,0x4e,0x71,0x75,0xdf, -0x0b,0x0c,0xc8,0xa8,0xcf,0xd0,0xc3,0x80,0xee,0xf5,0x7b,0x2c, -0xe5,0x77,0x13,0xbf,0x34,0x61,0x5c,0xca,0x6c,0xf5,0xab,0xf9, -0x26,0x0f,0x6a,0x0e,0xce,0x79,0x07,0xa5,0x79,0x73,0x86,0x45, -0xed,0xcd,0xd7,0x68,0x26,0x23,0xbe,0xff,0x82,0x63,0xcc,0xdc, -0x73,0x36,0xde,0x33,0x4d,0x83,0x8b,0x26,0x51,0xb3,0x1c,0x55, -0x12,0xdd,0x61,0xa1,0xd5,0x28,0x63,0x6d,0x8a,0xa8,0x1e,0x6b, -0x1e,0x06,0x17,0x89,0xbe,0x9f,0xb9,0xd8,0x24,0x17,0xd1,0xe1, -0x77,0x4f,0x9e,0xbf,0x7d,0x7a,0x72,0xf6,0xfc,0x6f,0x4f,0x9d, -0x39,0xf9,0x5f,0xce,0xab,0x6f,0x9c,0x45,0x99,0x73,0xb1,0x27, -0x3f,0x42,0x0f,0x20,0x68,0x51,0xfe,0xbc,0xea,0xa4,0xdd,0x6f, -0xb2,0xdd,0xfd,0x9c,0x6d,0x97,0xd7,0xcb,0xa7,0x36,0x7b,0xda, -0x2b,0xce,0x8b,0xb4,0xff,0xcb,0x11,0x84,0xb3,0xde,0x51,0x2f, -0xdb,0xcf,0x87,0x48,0x04,0x9d,0x0e,0x81,0xd6,0x9b,0xfc,0x83, -0xc8,0xe3,0x9f,0x6f,0x80,0x94,0x24,0xfb,0x13,0x54,0x77,0xca, -0xb4,0xd7,0xfa,0x6a,0xb5,0x58,0xdf,0x0e,0x27,0xd3,0xd5,0x7c, -0x7d,0x05,0x7d,0xcd,0x8a,0xfd,0x7c,0x72,0x7b,0x7d,0xba,0xbc, -0xfc,0xca,0xb2,0x5c,0x68,0x8d,0xab,0xbe,0x98,0xdc,0x0e,0xaf, -0xc7,0xe7,0xfb,0xfb,0xf9,0xd3,0xbf,0x61,0xa7,0x7f,0x3c,0x7e, -0xf5,0xe4,0xc5,0xd3,0xb7,0xa4,0xac,0xe2,0xfb,0x9d,0xce,0x67, -0xfd,0xe1,0xde,0x3f,0x06,0xdf,0xac,0xd1,0x60,0xd6,0x90,0x86, -0x1a,0x3a,0xaa,0xef,0x28,0x32,0x22,0xa5,0xb9,0xec,0xd9,0xb4, -0xde,0x13,0x6f,0x7e,0x87,0xcd,0x7a,0xc8,0x67,0x3c,0xb7,0x0e, -0x69,0x67,0x73,0xcf,0xa9,0xb7,0xfc,0x1a,0x17,0x31,0x16,0xf8, -0xdc,0xfd,0x97,0x07,0xb3,0xda,0xf8,0x8c,0x0a,0x57,0x42,0x90, -0x49,0x88,0x1a,0x14,0x92,0x51,0x23,0x40,0x56,0x6e,0xb4,0xc3, -0xcb,0x22,0x94,0x5d,0x4d,0x6a,0x46,0x76,0x9a,0xe4,0x59,0xeb, -0xa6,0x78,0xd4,0xe4,0x80,0x5b,0x76,0x75,0xb9,0xc6,0x86,0xb6, -0x79,0xd0,0xf5,0x51,0x43,0x6e,0x78,0x00,0xb3,0xdc,0x8d,0xcd, -0x8e,0xd7,0x0a,0x1d,0x19,0xc1,0x3b,0x97,0x98,0xf5,0x3c,0x88, -0x7b,0xb1,0xbc,0x56,0xa8,0xcc,0xac,0xcf,0xb5,0x29,0x76,0xdc, -0x47,0x57,0xb2,0xb5,0xdb,0x3e,0x0a,0x8f,0x78,0x74,0xae,0x17, -0xe7,0x29,0x6b,0xa0,0x28,0x6b,0x66,0x0c,0x2e,0x76,0x17,0xcb, -0xc9,0x7c,0x89,0x18,0x45,0x3d,0x0a,0x28,0x03,0x93,0x91,0x68, -0xdb,0xf0,0x73,0xbd,0x0e,0x97,0x95,0x0a,0x56,0x96,0xb5,0x98, -0xf5,0xda,0x36,0x4e,0x95,0xd7,0x6c,0x1f,0x6a,0x90,0x90,0xba, -0xbd,0x5c,0x95,0xf5,0xa8,0xf5,0x1a,0x7d,0x6f,0xfb,0xf9,0x64, -0x06,0x8c,0x19,0x2d,0xff,0x08,0x11,0x2f,0x05,0x7a,0x8b,0x30, -0xbd,0xaf,0x36,0x83,0x2c,0x5a,0xbe,0x3c,0xfb,0x11,0x4c,0x9d, -0x44,0xaf,0x61,0x2f,0x06,0x6d,0xad,0xbc,0x7c,0x6d,0xd3,0x65, -0x62,0x63,0x92,0x1f,0x2c,0x1f,0x38,0x5c,0xdc,0x52,0x03,0x10, -0x40,0x66,0x9b,0x6d,0xa9,0xaa,0x8c,0x92,0x00,0x3b,0x73,0x3b, -0x14,0xce,0xc3,0x2a,0x22,0xc5,0xf6,0x7c,0xd2,0x9c,0x87,0x0e, -0xac,0xdf,0xd1,0x9e,0x87,0xc0,0x04,0x0d,0x7a,0x20,0x63,0xf4, -0xa2,0x6b,0x77,0xbc,0x31,0x89,0xd5,0x7c,0x00,0x24,0x64,0xff, -0x0a,0x7f,0x03,0x3b,0x51,0x18,0x41,0x86,0xac,0xd8,0xc6,0xbf, -0xd3,0x96,0xcc,0x23,0x6f,0x00,0x9e,0xca,0x4b,0xd0,0x48,0x15, -0xfe,0x38,0xbb,0x50,0xcd,0x7e,0x00,0xcc,0x41,0x63,0x1a,0x57, -0x3f,0x75,0xd8,0x30,0x57,0x64,0xbe,0x4a,0xac,0x6b,0x59,0x53, -0x63,0xe6,0x11,0xda,0x12,0x89,0xd0,0x18,0x63,0xa2,0xca,0xd9, -0x3c,0x8b,0x5e,0x11,0xd1,0x5b,0x80,0x5a,0x6c,0x24,0xb5,0xd9, -0x7c,0xe5,0x0c,0xc9,0x3d,0x9b,0xe5,0xb7,0xc3,0x4f,0x6f,0x64, -0x2f,0xe7,0x93,0xeb,0xd9,0x7c,0xe9,0x2d,0xb8,0xc1,0xe6,0xb0, -0xe5,0xac,0x59,0x1a,0x66,0xa0,0x22,0x6f,0xd6,0xe6,0x50,0x4d, -0xd1,0x20,0x32,0xf3,0xcf,0x5e,0x61,0x33,0x3c,0xe7,0x10,0x3e, -0xd5,0x47,0xe1,0xb7,0x65,0xe9,0xdc,0x33,0x8b,0x26,0x96,0x7e, -0xb2,0xdf,0x29,0xf1,0xf7,0x75,0x7e,0xda,0xd9,0x47,0xc6,0x51, -0x00,0x52,0x5d,0xe2,0x63,0x6e,0x6d,0x0b,0x39,0x80,0xee,0x7e, -0x44,0x1b,0xff,0xc9,0xf7,0xd5,0x62,0x38,0x7b,0xfc,0xfd,0x3e, -0xfd,0x24,0x99,0xd3,0x1c,0x45,0x51,0xc0,0x0d,0x53,0x5c,0x08, -0x6e,0x5e,0x41,0xc1,0x17,0x70,0x08,0x3e,0x9b,0x95,0x16,0xfc, -0x43,0xc3,0x18,0x26,0x3d,0x34,0xa8,0x35,0xbb,0xd7,0x62,0x5e, -0x1f,0x4d,0xb8,0x9c,0xcc,0x67,0x33,0xc8,0x86,0x6f,0x42,0xfe, -0x7a,0xe5,0xf4,0x84,0x7e,0x12,0x2f,0x71,0xbe,0x8b,0xc7,0x34, -0x56,0x57,0x86,0x45,0x7b,0x35,0xeb,0xfc,0x98,0xcc,0x7a,0x81, -0xe1,0xc4,0x14,0x61,0xa3,0x45,0x40,0xdd,0xc1,0x6d,0x16,0x4f, -0xa7,0xb1,0x75,0x79,0xfa,0x93,0xc1,0x51,0x7a,0x18,0xcd,0xd9, -0x7a,0xfd,0x97,0x28,0x26,0x43,0x86,0x9b,0x82,0x1d,0xff,0x96, -0x4e,0x32,0xf5,0x71,0xc1,0xbd,0xca,0xc5,0x8b,0x85,0xba,0x55, -0x18,0x5e,0x79,0x85,0x75,0x57,0x90,0x3c,0x0c,0xc9,0x28,0x44, -0xc3,0xd5,0x6e,0x87,0xdf,0xa6,0x10,0x43,0xce,0xa3,0x79,0x45, -0x0b,0xbe,0x41,0x84,0x8c,0xb2,0x2f,0xe7,0x42,0x99,0x0f,0xda, -0x27,0xea,0xa0,0x89,0x4e,0xe9,0xdd,0xb9,0xbf,0xa1,0xfb,0x5f, -0xe3,0xf1,0xf1,0x80,0x73,0xfa,0x0c,0x8d,0x2e,0xde,0x82,0xd5, -0x86,0x83,0x87,0x2f,0x08,0xd1,0x42,0x33,0xce,0x49,0x1e,0xde, -0xbc,0x01,0xd4,0x7a,0x57,0x1d,0xf8,0x00,0xe2,0x05,0x4a,0x61, -0x9f,0x99,0x0a,0xfd,0xac,0x73,0x69,0x31,0x12,0xc6,0x61,0xda, -0xb9,0x14,0xc1,0x87,0x13,0x05,0x6d,0x69,0xe4,0xf2,0x4f,0xf2, -0x19,0x52,0x58,0x17,0xdc,0x2f,0xac,0x12,0xb1,0x2b,0x95,0x71, -0x86,0x15,0xa7,0x2d,0xa9,0x2d,0xb8,0xb0,0x94,0xa8,0xe5,0xc9, -0x17,0xea,0xd2,0x3b,0x20,0x33,0x06,0x34,0xa5,0x56,0x55,0x01, -0x9c,0x19,0xc0,0xa1,0x35,0x1b,0x02,0xeb,0xcf,0x06,0xb9,0xef, -0x94,0x96,0xec,0x43,0xad,0x03,0xdb,0xc3,0x30,0xc1,0x17,0xe8, -0xa5,0x3e,0xcc,0xfe,0x31,0x7a,0xa9,0x1f,0x87,0x52,0x1c,0xbc, -0xcc,0xc6,0x1f,0x53,0x27,0x00,0x19,0x17,0x80,0x73,0x31,0x38, -0xa2,0xc2,0x0d,0xe1,0x81,0x01,0x42,0x11,0x43,0xae,0x8f,0x73, -0x69,0xa0,0xfb,0xe5,0x52,0xcf,0xb7,0x5e,0xef,0xd4,0x66,0xb8, -0xdd,0xb6,0x0b,0xac,0xe7,0x21,0xa5,0x76,0xaa,0xcd,0x2c,0xd3, -0xac,0x07,0xab,0xc6,0xdb,0x0e,0x0a,0x56,0x4e,0x7c,0xd2,0xd1, -0x26,0xd5,0x25,0xe5,0xb3,0xd6,0xc6,0xcc,0x9d,0x91,0x51,0x9e, -0xee,0xee,0xae,0x8f,0x21,0x35,0x6d,0x5b,0x5f,0x1a,0xe5,0xce, -0xb5,0x3a,0x7a,0xdf,0x73,0x66,0x75,0xc8,0xcc,0x75,0x2d,0x6f, -0xad,0x26,0xc9,0x9e,0x01,0x32,0x98,0x36,0x0c,0x63,0xd0,0xd1, -0xac,0xf8,0xca,0x41,0x73,0x0f,0x2c,0x0d,0xfb,0x4c,0xa6,0xd9, -0x9d,0x4f,0xfe,0xc3,0x6c,0x58,0x63,0x85,0xeb,0x41,0x60,0x1b, -0xbd,0x25,0xb6,0x6c,0x5c,0x8f,0xdd,0xe6,0xf8,0xa6,0x58,0xe5, -0xbc,0xce,0x9f,0x21,0x20,0x09,0xd3,0x33,0x2e,0xbf,0x84,0xd5, -0x36,0x28,0xec,0x29,0x2a,0x46,0x22,0xab,0xc7,0xde,0x27,0x97, -0x21,0xa0,0x2e,0x11,0x73,0x31,0xa6,0x98,0x2b,0xbb,0x73,0x73, -0x00,0xcd,0xf5,0x1e,0x94,0xe5,0xa4,0x67,0x8e,0xb1,0xc2,0xf7, -0x2a,0x3e,0xd0,0xec,0x7e,0x35,0x98,0xbd,0x38,0x1c,0x24,0x51, -0x28,0x5f,0xdb,0xd9,0x9c,0xcc,0xa1,0x99,0x7e,0xda,0xea,0xb9, -0xee,0xad,0xe7,0x6a,0x46,0xde,0x62,0x83,0x73,0xc0,0xd7,0xcd, -0x18,0x89,0xee,0xb5,0xf1,0xf2,0x96,0xcd,0x09,0xee,0x18,0xf8, -0x81,0xad,0xb4,0x2d,0x09,0xda,0x6e,0xac,0xa9,0x68,0x3c,0xf7, -0xc3,0xfc,0xf6,0xf0,0x69,0xdc,0xec,0x45,0xb0,0x24,0x75,0x12, -0xed,0xa1,0x89,0x11,0x51,0x82,0xe9,0x62,0x66,0xd7,0x40,0x69, -0xc2,0x28,0xe0,0xa9,0xc5,0xd7,0x6b,0x53,0xc2,0xc9,0x4d,0xf8, -0xbc,0xbd,0xe8,0xae,0xa1,0x81,0xb4,0xf7,0x5b,0xd3,0xa2,0xa8, -0xaf,0xc6,0x07,0xef,0x35,0x52,0xf6,0x23,0xff,0xad,0xee,0x36, -0xcd,0x9d,0xdf,0xd8,0x00,0xed,0x60,0x68,0x2f,0x87,0x0b,0xf2, -0xf1,0x06,0xd4,0x03,0xfe,0x1c,0x89,0x62,0x89,0xc3,0x4d,0xee, -0x45,0x3d,0x64,0x68,0xcc,0x6a,0xdb,0x85,0xe3,0x82,0xaf,0x90, -0xaf,0x82,0x16,0xa7,0x9d,0x6b,0xd8,0x77,0x72,0x76,0x07,0xc6, -0x42,0xb2,0x3c,0x79,0x9a,0x3c,0x8c,0x02,0xea,0xee,0xc4,0xf6, -0xb0,0xd9,0x95,0x19,0x77,0xe3,0x5a,0xed,0x95,0x30,0x13,0x3a, -0x27,0xc3,0x21,0x15,0xf7,0xca,0xf8,0xbe,0x7b,0x0f,0xf7,0xdd, -0xfb,0xf7,0x25,0xa7,0xc2,0x82,0xa4,0x5f,0xdd,0x9d,0xef,0xdf, -0x3f,0x7e,0x7f,0xf4,0x5e,0xf1,0x35,0x2c,0x75,0x8a,0x92,0x3a, -0x04,0x15,0xbf,0x9e,0xce,0x46,0x14,0xe6,0xe7,0x77,0x8c,0x11, -0x40,0xfd,0xf7,0x83,0x3c,0xdd,0x41,0x3b,0x11,0xeb,0x35,0xfe, -0x7d,0x5c,0xfe,0x79,0xbd,0x26,0xef,0x78,0xec,0xd1,0x63,0x32, -0x1e,0x21,0xca,0x4d,0x44,0xbe,0xf3,0xad,0x67,0xda,0x5c,0x36, -0x8d,0x14,0x69,0x5f,0xa1,0xbf,0x40,0x64,0x56,0xfd,0xf0,0xfc, -0xd5,0x93,0xe7,0xaf,0xfe,0xca,0xde,0xc3,0x7c,0x41,0x07,0xd5, -0x47,0xc9,0xdb,0x75,0xfa,0xa7,0xda,0xe0,0xee,0x1d,0xee,0x94, -0xfc,0x9e,0x6d,0xb3,0xb3,0x83,0x1c,0x79,0x73,0xa7,0xfe,0x26, -0xa8,0x24,0xce,0xf9,0x82,0x8c,0xac,0xee,0xf0,0x47,0x74,0x69, -0x64,0x87,0xa6,0x9c,0x45,0xe3,0xc3,0xee,0x21,0xa5,0x19,0x07, -0xec,0xd1,0x4b,0x0b,0x67,0x9d,0x64,0x3c,0x1b,0x25,0xc6,0x8d, -0x64,0x63,0xbe,0x3f,0xa1,0xa7,0xc9,0x87,0x46,0x26,0xee,0x9b, -0xae,0xd7,0xfe,0x8c,0x35,0xd0,0x5c,0x9b,0x2a,0x8d,0xe1,0xa7, -0x7a,0xb2,0x91,0x84,0x53,0xd4,0x6e,0xb3,0x34,0x48,0x49,0x59, -0x7b,0x0d,0x0e,0x85,0x69,0xe9,0x00,0xd5,0xec,0x4e,0x69,0x66, -0x4b,0x3c,0x42,0x1d,0x4b,0xef,0xf8,0x30,0x6f,0xb0,0x98,0x43, -0x5b,0x73,0xa6,0xf3,0x13,0xb4,0x02,0xa5,0x5c,0x60,0xbb,0x60, -0x56,0x63,0x83,0xda,0x70,0x12,0xa3,0x5d,0x2d,0x8b,0x99,0x55, -0xf1,0xb6,0xed,0x33,0xde,0x02,0xc8,0xb7,0x79,0x60,0xef,0xe4, -0x5b,0xd7,0xb3,0x3c,0xcb,0x39,0x13,0x53,0x7c,0x5d,0xba,0x13, -0xc2,0xb1,0x2f,0x3d,0x0a,0x0e,0x83,0x87,0x32,0xd7,0x3e,0x82, -0x44,0x5d,0xd9,0xc3,0x65,0x33,0x7b,0xb8,0xe6,0xe4,0x32,0xdb, -0x3a,0xbd,0xea,0xdc,0xf2,0x8b,0xfd,0x3d,0x79,0xf8,0xac,0x88, -0x87,0x9e,0xa7,0x9e,0x80,0x7f,0x8b,0x8c,0x29,0xdf,0x61,0x17, -0xd2,0xc5,0x48,0xb9,0x58,0x12,0xa5,0xc3,0x25,0x0e,0x06,0xaa, -0x26,0x77,0x74,0xb1,0x1c,0x0f,0x7f,0x55,0xed,0x36,0x68,0x22, -0x5a,0xdd,0x68,0x9c,0xcb,0xe8,0x0a,0xf5,0xa4,0x78,0x00,0xe3, -0xcf,0x05,0x12,0x3c,0x32,0x76,0x5b,0x98,0xf3,0x3c,0x7a,0x11, -0x10,0xf6,0x28,0x8a,0x07,0xc0,0xc3,0x23,0x79,0xf8,0xe5,0x91, -0x7c,0xf9,0x4f,0x8f,0x24,0x4c,0x53,0x16,0x50,0x64,0x6a,0xc3, -0xce,0xa3,0x19,0xd5,0x7c,0xb9,0x4a,0x2f,0x3e,0x2b,0xf4,0x80, -0x41,0xe8,0xf5,0x02,0x96,0xf3,0xbb,0xc5,0xe9,0xe5,0x50,0x78, -0x1a,0x6e,0xad,0xea,0x3a,0xf5,0xc4,0x14,0x61,0x57,0xa3,0xf1, -0x62,0x75,0x83,0x18,0xd6,0x95,0x3f,0x8b,0x84,0xe1,0x71,0x33, -0xac,0xdc,0x8e,0x6d,0x88,0xf2,0xf9,0xb3,0xec,0x7e,0x34,0x27, -0xad,0x7a,0xba,0x8a,0x44,0x29,0x46,0x10,0x0a,0xa7,0x13,0x03, -0xf3,0x0a,0xed,0x25,0xb5,0x18,0x41,0x59,0x50,0xa5,0xd3,0xdd, -0x28,0x39,0x6b,0xdf,0xca,0xab,0x34,0xc4,0x40,0x1b,0xf9,0x8e, -0x45,0x4b,0xc1,0x7f,0xb8,0x3f,0x84,0x58,0xca,0xd1,0x4d,0x1a, -0x3a,0x75,0x14,0x33,0x00,0x70,0x36,0x1f,0x68,0x71,0xbb,0x4d, -0x1d,0xef,0x74,0xf2,0xe6,0x5c,0x08,0x53,0xf2,0xec,0xed,0x31, -0xda,0x67,0xf0,0x18,0x8e,0x90,0x0a,0x61,0xd1,0x9e,0x4e,0x2e, -0x50,0xb8,0x7f,0xf3,0xf1,0x06,0x39,0x5d,0x54,0xea,0xf1,0x81, -0x58,0xd8,0x8d,0x8b,0x2a,0xc6,0x62,0x08,0x2f,0x6b,0x2a,0x87, -0xa6,0x4f,0xb5,0x17,0x18,0x81,0xf9,0x99,0xfc,0x9f,0xa2,0x8a, -0x07,0xe1,0xcd,0x0d,0x33,0xda,0xcc,0xce,0x51,0x8e,0x00,0xe3, -0x33,0x9e,0x95,0x57,0xc5,0xb6,0x7b,0x4b,0xbf,0x66,0xbc,0x51, -0xb5,0x86,0x7a,0x04,0x3b,0xfb,0x22,0x78,0xc3,0x4d,0xf8,0x22, -0x4a,0x6c,0xd0,0xce,0x5c,0x79,0x90,0xc7,0x75,0xd4,0xf8,0xfd, -0x6f,0x27,0x8c,0xb7,0xa1,0xf4,0xb8,0xda,0xc4,0x7e,0xcb,0xc2, -0x86,0x4d,0xfc,0x06,0xd8,0xa6,0x3c,0x78,0x15,0x0a,0x50,0xa0, -0x73,0xcb,0x55,0xd3,0xc0,0x2d,0xd0,0x12,0xa7,0x5d,0xf8,0x98, -0xbb,0x7c,0x60,0x0e,0x10,0x58,0x1d,0x0e,0x49,0x6c,0x6c,0x7f, -0x55,0xc0,0xd4,0x17,0x16,0xe5,0x85,0x6f,0x2c,0x03,0x8d,0x8a, -0x6b,0x87,0xa8,0x07,0xab,0x6f,0xee,0x07,0xc3,0x7a,0xa8,0x01, -0x90,0x1c,0xb6,0x60,0x8e,0xe6,0x7e,0x02,0x6b,0x75,0xde,0x58, -0x1f,0xbe,0x11,0x70,0xb9,0xdd,0xb1,0x75,0xce,0xcc,0x57,0x28, -0xac,0x9d,0xe5,0x04,0x35,0x4e,0xd8,0xa7,0x5c,0x29,0x66,0x3c, -0x59,0xc0,0x63,0x38,0x45,0x41,0x5b,0x6b,0x20,0x43,0x40,0x65, -0x6c,0xcf,0xef,0x28,0xf9,0x05,0x10,0x05,0x31,0xb9,0x27,0x49, -0xc6,0x68,0x38,0x39,0x9a,0x2c,0xc5,0xec,0x67,0xa6,0xfd,0xd1, -0x7c,0x72,0xa7,0x1c,0x02,0x76,0xfb,0x8b,0xe6,0x82,0x41,0xab, -0x35,0x28,0x70,0xe0,0x99,0x64,0xbe,0x69,0x1a,0x08,0xaa,0x81, -0xc6,0x72,0x73,0xb4,0xb3,0xfd,0x08,0xe0,0x20,0x45,0x1b,0x87, -0xd2,0x88,0xd0,0xa8,0x61,0xbe,0x23,0x8d,0xd8,0x51,0xd0,0x5b, -0x4e,0xbf,0xcb,0x15,0x3b,0xd9,0xf4,0x85,0xf9,0xa0,0xcb,0xd5, -0xa7,0x20,0x59,0x38,0x77,0x13,0xc7,0xe7,0x5d,0xde,0x82,0x2d, -0xf9,0x87,0x55,0xeb,0x62,0xcc,0x27,0xdf,0x4e,0xa2,0x1e,0x52, -0x0c,0x41,0x60,0xed,0xfd,0xb0,0xa9,0x19,0xf7,0x0a,0xc2,0xca, -0xb6,0x3a,0x01,0x30,0x52,0xdc,0x4f,0xb4,0x92,0x25,0xdc,0x70, -0x05,0x67,0x5e,0x52,0x38,0xf3,0x3d,0x65,0xe5,0xb5,0xf2,0xa5, -0xb5,0x41,0xf6,0xa9,0x22,0x2b,0x37,0x5b,0x98,0x11,0x7c,0x72, -0x78,0x06,0xe7,0x57,0xb0,0xfd,0xf8,0x78,0x71,0xec,0x4d,0x62, -0xe6,0xc9,0x49,0xe8,0xcf,0x1c,0xbd,0xbe,0x2a,0xcb,0x2c,0x2f, -0xcb,0x52,0x6b,0xea,0x05,0x47,0x4b,0x51,0xdd,0x0c,0xa7,0xd3, -0xf9,0xc7,0x93,0xf9,0xe2,0x73,0x1a,0x1e,0x3a,0x21,0x33,0x41, -0x9c,0xd5,0xa6,0x9e,0x0f,0x1b,0xe4,0x66,0x62,0x0e,0x3a,0xaf, -0xad,0x53,0x1a,0xcf,0xb5,0x16,0xc0,0x3d,0xaf,0x08,0x99,0x21, -0x0e,0x4c,0x44,0x41,0xbe,0x78,0x7d,0x72,0xfc,0x22,0x90,0x1a, -0x48,0xfb,0xff,0xb3,0x6c,0x0f,0xb2,0xf4,0xbc,0xd7,0xcb,0xf0, -0xf3,0xfc,0xe3,0x37,0xf8,0xbb,0xbb,0x9f,0xf3,0x3b,0x41,0xc0, -0xa6,0xe2,0xa8,0xf5,0x9a,0x3e,0xbd,0x83,0xfa,0x2d,0x55,0x8a, -0x0b,0x5d,0x2b,0x06,0x39,0xc1,0x47,0xae,0xb9,0xcc,0xca,0x2b, -0xa7,0x11,0x8a,0x74,0x0c,0xab,0x79,0x73,0x0d,0x7f,0x45,0xf5, -0x4d,0xae,0x7e,0xbc,0x12,0xef,0xeb,0xbe,0xb4,0xbc,0x86,0xd8, -0xbe,0x64,0x64,0x49,0x7b,0xa8,0x24,0x99,0x62,0x8a,0xd2,0x52, -0x42,0x94,0xdc,0x96,0xa4,0x8d,0xa8,0x18,0x10,0x9a,0x1e,0xd1, -0xfc,0x87,0x03,0xa5,0x92,0xa5,0xbb,0xbb,0x13,0xee,0x16,0x3a, -0x65,0x82,0x83,0xa5,0xea,0x3b,0x68,0x03,0xb4,0xb2,0x32,0xee, -0x68,0x85,0x04,0x85,0x49,0xea,0xe4,0x7f,0x26,0x85,0xcc,0xc9, -0xfc,0x02,0x75,0x59,0xf9,0x76,0xa7,0x5c,0x6e,0x24,0x44,0x4e, -0x8c,0xc0,0x59,0xa0,0xbc,0xa4,0x85,0x22,0x82,0xfa,0x05,0xc2, -0x92,0xbd,0xba,0x53,0x1e,0x88,0xa5,0x02,0xa5,0x99,0x1e,0x21, -0x96,0x5c,0x1e,0x14,0x78,0x8a,0x41,0xdb,0x37,0xb9,0x34,0xca, -0x9f,0xa5,0x06,0x58,0x66,0x31,0xdc,0xa4,0x4c,0x0a,0xd4,0xa1, -0x93,0xb1,0x83,0x53,0x27,0x2e,0x2a,0xca,0x24,0x6e,0xde,0x4a, -0x7e,0xc8,0xab,0x55,0xe1,0xe7,0xb3,0x74,0x79,0xbb,0x80,0xba, -0x03,0xbe,0xba,0x5e,0xd7,0xf4,0x9b,0xdd,0x82,0x28,0x6b,0x1d, -0x71,0xa5,0x83,0x46,0xe7,0xf1,0xf1,0x37,0x83,0xf6,0x12,0x74, -0x38,0x02,0x9f,0x7e,0x82,0xeb,0xaf,0x42,0xaf,0x77,0x82,0xeb, -0xdd,0x55,0xea,0xdc,0x3d,0x3a,0xff,0xd8,0x0b,0xc4,0x6c,0x8f, -0x8b,0x92,0xf3,0xd0,0x24,0x1e,0xdd,0xad,0x5b,0x8c,0x5d,0x82, -0xb3,0x62,0xc1,0xef,0x6b,0x7c,0xb5,0x45,0x2b,0x93,0x94,0x98, -0x28,0x23,0xc3,0x6b,0x2e,0xee,0x6c,0xaf,0xf9,0x9c,0x40,0xe0, -0xc5,0x75,0xe3,0x5d,0x1f,0x64,0x70,0xcd,0xec,0x7d,0xb1,0xc1, -0x6c,0xaf,0xc0,0xcd,0xa0,0xad,0x3d,0xb7,0xed,0xdb,0x0e,0x28, -0xcb,0x6c,0xc6,0x4d,0xb0,0xc6,0xda,0x49,0xf1,0xe5,0xe5,0x53, -0x6b,0x99,0x57,0x1d,0x14,0xe3,0x11,0xc1,0x28,0x44,0x63,0xa4, -0x96,0x19,0x36,0x52,0xaf,0x9a,0xab,0x68,0xbe,0x3a,0x27,0xd5, -0xe5,0x82,0xec,0xc3,0xc0,0x7e,0x9c,0x8c,0x5a,0x72,0x0a,0xb4, -0xa8,0xf2,0x96,0x3f,0x87,0x50,0x48,0x26,0x58,0x40,0xb0,0xb4, -0xba,0xad,0x27,0x2e,0xbd,0x68,0xdd,0x77,0xbb,0x5d,0xf4,0x69, -0x5d,0x40,0xe2,0x23,0x58,0x5d,0xf0,0x89,0xfe,0xad,0x1f,0x5a, -0x3f,0xfe,0x68,0xcc,0x7d,0x4d,0xec,0x58,0x13,0x8d,0x1b,0xd5, -0x5d,0xd7,0x59,0xeb,0xd5,0x0d,0xa9,0x0d,0xd2,0x3a,0xaa,0xa9, -0xa2,0x99,0x9f,0x87,0xe6,0x2b,0xca,0x7b,0x3e,0x61,0x0a,0x79, -0x1c,0x94,0x9b,0xa5,0x70,0x57,0x0c,0xdd,0x41,0x55,0x21,0xd7, -0xa4,0xe1,0xc5,0xd6,0xee,0xd3,0xcd,0x91,0xaf,0xa5,0x6c,0x92, -0x73,0xc9,0xe1,0xb8,0x2c,0x4b,0x2b,0xf7,0x62,0x3a,0xc1,0x77, -0x5b,0x3f,0x94,0x84,0x19,0x58,0x67,0xaa,0xce,0x70,0x9c,0x11, -0x1b,0x30,0x00,0x9c,0x3d,0x8e,0xdd,0xc0,0xd4,0x32,0xe2,0x56, -0x21,0xd0,0x00,0xc5,0x6a,0x80,0x6f,0xc5,0x82,0x7c,0xea,0x71, -0x85,0x82,0xa9,0x6c,0xe8,0x87,0x87,0xac,0xdf,0x9c,0x6d,0x50, -0x36,0x80,0x44,0x19,0x47,0x7e,0xc0,0x33,0xb8,0x7a,0xfc,0x16, -0x88,0xec,0x05,0x46,0x3d,0x4c,0x2e,0xe4,0xe5,0x07,0x74,0xd1, -0x6e,0x48,0x18,0x71,0x9a,0x62,0xe2,0xed,0x76,0x84,0x4d,0x85, -0xc9,0x16,0xa9,0x7a,0x48,0x3c,0x28,0x27,0xc5,0x47,0xb8,0x05, -0xcf,0x86,0xd7,0xa9,0x29,0x43,0xab,0xf2,0xe1,0x27,0x1c,0xf7, -0x06,0xfe,0x95,0xcf,0x38,0x6e,0x5c,0x0c,0x09,0xf1,0xd0,0xc0, -0xf8,0x6c,0xff,0x9f,0x1a,0x19,0xf7,0x1a,0xe5,0x16,0x45,0x88, -0x43,0x7b,0x09,0x18,0x43,0x78,0x7a,0x96,0x5f,0xe3,0x66,0x9e, -0x46,0x0f,0x59,0x8c,0x34,0xe9,0x7b,0x8c,0xb2,0x69,0xca,0xbd, -0x9a,0xa4,0xbb,0x5f,0x3d,0x88,0x8c,0x9b,0x21,0xc3,0x4f,0x38, -0x90,0xc2,0x93,0x88,0x25,0x8b,0xb7,0x9d,0x53,0x9c,0xba,0x8a, -0xbd,0x7d,0x72,0xb4,0x95,0xae,0x28,0x23,0xdc,0x75,0x37,0xb2, -0xad,0x6e,0xb2,0x3a,0x3d,0x4e,0x0f,0x6e,0x1b,0xcd,0xdd,0xf0, -0x0e,0x1c,0x10,0xee,0xb2,0x7a,0x6a,0x8f,0x65,0x46,0xa4,0xc3, -0xcb,0x16,0xfa,0x5c,0x28,0xd4,0x11,0x3c,0x1d,0x98,0x03,0x0b, -0x70,0x2e,0x4b,0xfd,0x06,0x29,0xe3,0xd9,0x28,0x62,0x65,0x05, -0x02,0x27,0x86,0xd3,0x61,0x19,0x0f,0x4d,0x14,0xb5,0x5b,0x0f, -0x5e,0x25,0xad,0x36,0xb5,0x8f,0xeb,0xc2,0x88,0x19,0x5f,0x69, -0x68,0x49,0x36,0x5c,0x17,0xa6,0x99,0xfc,0xf6,0x8e,0xa4,0x3f, -0x09,0xd8,0xbc,0x9a,0x3f,0xb9,0x5b,0x4c,0x27,0x97,0x88,0x0b, -0x26,0x72,0xbd,0x8d,0x5a,0xf2,0x08,0xbf,0x6d,0xda,0xfd,0xaa, -0x0b,0x26,0xd9,0x58,0x3b,0x0a,0xab,0xc7,0xea,0x42,0x81,0x11, -0x3b,0x00,0x28,0xd6,0x22,0x35,0x3b,0xf1,0x96,0xad,0x0b,0xce, -0x5f,0x5d,0xf9,0xc3,0x10,0x0d,0xa4,0x78,0xcd,0xc6,0x31,0x46, -0xf0,0x32,0x0b,0x37,0x66,0x24,0xcb,0x99,0x6f,0x1f,0x56,0xbf, -0x47,0xb3,0xe0,0xe6,0x7a,0x62,0x5f,0x84,0xeb,0x91,0xeb,0xf5, -0xfd,0x26,0x6f,0x98,0x89,0x3f,0x98,0x7b,0xe9,0x95,0x68,0xe5, -0x5d,0x36,0x5e,0x4c,0x46,0x2a,0x33,0x28,0x31,0xd8,0x3a,0x49, -0x5f,0x51,0xd6,0x0e,0xf3,0xf6,0x3e,0xfb,0xbd,0xb6,0x65,0x3d, -0x69,0x86,0x8a,0x2c,0xce,0x34,0x1d,0x6d,0xdb,0xda,0xd8,0x90, -0xd7,0x57,0x5c,0x3f,0xe5,0xea,0x5b,0x21,0x4f,0xd4,0xfd,0x5a, -0x59,0x86,0x1d,0xe8,0xa5,0x7e,0x77,0x7d,0xed,0x8e,0xfc,0x5d, -0x67,0x59,0xe4,0x14,0xed,0x64,0x7e,0xcb,0x4e,0xdb,0x5a,0xb5, -0x39,0x2d,0xd0,0xdd,0x87,0x05,0x16,0x4d,0x03,0xf9,0x41,0xc9, -0x2c,0x9b,0xb2,0xb4,0x0d,0x41,0x46,0xa9,0x9c,0x95,0x3f,0x03, -0x05,0x93,0x06,0x7c,0x4a,0x75,0x98,0xe8,0xcc,0xa2,0xb8,0x5e, -0x87,0x10,0xb3,0x86,0x43,0xd4,0x89,0x4d,0x36,0x5c,0x4b,0x70, -0xac,0xc6,0xc3,0x5f,0x3b,0xb0,0xdb,0xed,0x38,0x86,0x09,0xa2, -0xac,0x30,0x63,0x2f,0x0d,0xb4,0xee,0xdd,0x6c,0xb3,0xc8,0xec, -0xec,0x0a,0xb9,0x90,0x69,0x38,0xfc,0x5d,0xf4,0xa7,0x91,0xa2, -0xc9,0xf4,0xdf,0xdb,0xf0,0x2c,0x10,0xea,0x76,0xc3,0x81,0x47, -0x76,0xc3,0xda,0x95,0xe4,0xa4,0x7e,0xbd,0x6d,0x5f,0xb5,0xb5, -0x8b,0xb0,0xe1,0xc2,0x17,0xa3,0x71,0x0d,0x92,0xee,0xbe,0x49, -0xbd,0x7a,0x5c,0x78,0xc2,0x05,0xab,0x26,0x2b,0xea,0xd9,0xe3, -0xfa,0x46,0xe3,0x99,0x3e,0x28,0xa9,0xda,0x4b,0x7c,0x48,0x07, -0x92,0xe5,0xde,0x7e,0x44,0xed,0x36,0x36,0xbd,0x8f,0x27,0x93, -0x7d,0x10,0x8a,0x0f,0x9b,0x08,0x7c,0x27,0x11,0xaf,0x84,0xc1, -0xcc,0x06,0x6b,0x5b,0xc1,0xe1,0xc2,0x3e,0xdc,0xf1,0xdf,0x72, -0x6d,0xaf,0xd7,0x87,0xf8,0x5c,0x69,0x96,0x82,0x93,0x61,0x68, -0xa6,0xde,0x56,0x8b,0xe9,0x12,0xc5,0x96,0xb5,0xc7,0x4d,0x64, -0x1a,0x9b,0x35,0xbf,0x19,0xc2,0xf7,0xf8,0xd3,0xf0,0x72,0x35, -0xfd,0xdc,0x82,0xa5,0xd8,0x42,0xe6,0xa0,0x73,0x2f,0xd8,0x22, -0xd3,0xb4,0x21,0xca,0x96,0x90,0x05,0x82,0x2d,0x9b,0x2f,0x98, -0x2b,0xbb,0x44,0x08,0xaf,0x80,0x0b,0xe8,0xcc,0xce,0x1f,0x90, -0x59,0x78,0xd0,0x14,0xf7,0x9b,0xcd,0x91,0x91,0x20,0x32,0x68, -0x8b,0x58,0x60,0xae,0x8b,0x8f,0xd8,0x7a,0xec,0x63,0xdd,0x82, -0x2c,0xeb,0x4f,0x3a,0x87,0x79,0x0d,0xf9,0xd9,0xc3,0xe8,0x8c, -0x2e,0xc6,0xa0,0x15,0x10,0x75,0x3b,0x5e,0x5e,0x8f,0x6d,0xa4, -0x70,0x22,0x6b,0x6c,0xca,0xb0,0x60,0x23,0x8e,0xc5,0x0f,0x50, -0xf5,0x8d,0x1b,0xad,0x0a,0x72,0xde,0x55,0x5b,0xbd,0x3f,0x2d, -0xa7,0xd9,0xff,0xc9,0x3d,0xa9,0x0b,0x9e,0x18,0xfa,0xdb,0x96, -0x7b,0x20,0x80,0x25,0x75,0x9c,0xf9,0x06,0xa7,0x0d,0xe3,0xdf, -0x30,0xfa,0x13,0x0b,0x26,0xdf,0xbe,0xa9,0xc3,0x57,0xa7,0xda, -0x51,0xd7,0x3c,0xe8,0x47,0x6c,0xbe,0xfa,0xaa,0xae,0xe6,0x62, -0xc9,0xa4,0x5a,0xe2,0x03,0x67,0x4b,0x63,0xcd,0xfe,0xec,0x62, -0x88,0x59,0xcf,0x3c,0x79,0xb1,0x5e,0xe2,0xb6,0xe7,0xc2,0x62, -0x2a,0xc4,0xa1,0x29,0x21,0xc4,0x16,0x3e,0x6f,0x69,0xb0,0xf9, -0xa9,0xe9,0x77,0x10,0x5f,0xc1,0x55,0xb2,0xb1,0x2b,0x8c,0xef, -0x2e,0x79,0xa2,0x8d,0xc4,0xd8,0x48,0xa4,0x32,0x46,0x2e,0x5e, -0xa2,0xc9,0xc4,0xdb,0xe1,0xa7,0xb4,0x76,0xef,0x35,0x60,0xe0, -0x4e,0x86,0x3b,0x16,0x77,0xad,0xa3,0x9d,0xed,0x76,0x2d,0xaa, -0x6b,0x45,0x93,0x23,0xf9,0xb6,0xb2,0x01,0x27,0x12,0x1a,0x3b, -0x90,0x38,0xf6,0x85,0x37,0xf6,0x45,0xd2,0x0b,0x0e,0x98,0xf3, -0x83,0x65,0xaf,0x9c,0x65,0xac,0x87,0xe4,0x4e,0x68,0x74,0x9d, -0x74,0x16,0x90,0x3a,0x4e,0x78,0x49,0x1d,0x47,0x96,0x65,0x19, -0x8b,0x1c,0xe8,0x58,0xa0,0xea,0xb4,0x13,0x5b,0x63,0x4d,0x6a, -0x28,0xf4,0x65,0x95,0xb8,0xec,0x01,0xc2,0xb9,0x32,0x0a,0x65, -0x68,0x2d,0xd2,0xe8,0xc4,0x65,0x5f,0x41,0xf9,0xe1,0x7e,0xd8, -0x42,0xfc,0xe5,0xa9,0x19,0x08,0x6b,0x16,0x64,0xbd,0x36,0x09, -0x4d,0xf4,0x17,0x5a,0x51,0xdf,0x69,0x50,0xc7,0x72,0xe6,0x50, -0x74,0xa8,0x33,0xd6,0x44,0xf6,0x23,0x6a,0x55,0xd3,0x4a,0x79, -0x43,0xf4,0x31,0xf9,0xbd,0xd0,0x9d,0x45,0x38,0x0d,0x44,0x73, -0x16,0x66,0x32,0xf0,0x35,0xcc,0xcc,0x6f,0xac,0xf6,0xa5,0x0f, -0x28,0x2e,0xe2,0x0b,0x7a,0x60,0x32,0x7f,0xb7,0xa1,0x93,0xe7, -0x6d,0xd7,0xc4,0x08,0x6d,0x8d,0x2d,0x2f,0xc5,0xa1,0xf9,0xf2, -0x12,0x93,0x4a,0xf8,0xe5,0xa7,0xaa,0x1c,0x92,0x29,0x06,0x7e, -0x25,0xc6,0x61,0xce,0x14,0xa5,0x5f,0xee,0x35,0x69,0x64,0x0d, -0x46,0xf2,0xfb,0x2f,0xa9,0x89,0x27,0xbb,0xc9,0x0e,0xfa,0x83, -0xf4,0x6f,0xb7,0x38,0x9a,0xcb,0x4b,0x52,0xc5,0xd6,0x97,0xc6, -0x4e,0x99,0x26,0xd5,0xea,0xf3,0x74,0x8c,0xd6,0x15,0x21,0xa1, -0x97,0x1c,0x25,0x05,0xe2,0xd3,0x1d,0xcd,0x99,0xe5,0x54,0x6d, -0x35,0x5e,0x79,0x8d,0xfd,0x1c,0x0e,0x00,0x69,0x39,0xe7,0x41, -0xe6,0x9c,0xb6,0x07,0x12,0x9a,0xdb,0x43,0x92,0x47,0x09,0x57, -0x93,0x36,0x6b,0x46,0x88,0x98,0x19,0xd4,0xd8,0x97,0xec,0x83, -0x32,0xb5,0x5f,0x3d,0xfb,0x81,0x88,0x7f,0x01,0x18,0x46,0x87, -0x8b,0x15,0xae,0x27,0x58,0x83,0xd7,0xaa,0x58,0xa1,0x35,0x24, -0x4e,0xca,0x1b,0x63,0xd1,0x30,0x4c,0xe2,0x80,0xec,0x32,0x00, -0x3f,0x6c,0x40,0xf8,0xc2,0x08,0x34,0x58,0x26,0x5f,0xaf,0xa9, -0x6d,0xc6,0x78,0x80,0x4c,0x1e,0x47,0x05,0x23,0xb4,0xc9,0x6a, -0xfa,0x1b,0xcd,0xb7,0x62,0x7e,0x31,0x86,0x91,0x74,0x89,0xaf, -0xb6,0xdd,0x82,0x7c,0xef,0x04,0xcf,0xb3,0xf5,0x4b,0x88,0x9f, -0x67,0xaf,0xe0,0x48,0x6e,0x80,0x17,0xc4,0x9f,0x98,0x27,0x60, -0xbc,0x5e,0xd8,0x47,0x45,0x7f,0x10,0x35,0xe7,0xe4,0x01,0x22, -0x0b,0xf9,0x4f,0x64,0xe2,0xb2,0x01,0x2b,0xa8,0x9c,0xb3,0xfb, -0x11,0xec,0xd9,0x0f,0xe3,0xd1,0x69,0x90,0xcf,0x18,0xad,0xa8, -0x01,0xc9,0xef,0x3d,0x29,0x5e,0x04,0x9c,0x34,0x18,0x38,0x89, -0xf0,0x2c,0x75,0x56,0xfd,0x27,0x76,0x3c,0x05,0x05,0x51,0x61, -0x09,0xe3,0xcc,0x16,0xb5,0xc4,0x45,0xbd,0xd6,0x00,0xc9,0xea, -0x3d,0x9c,0x9e,0x36,0xcc,0x4c,0x56,0x3c,0x5c,0xe6,0xc8,0xe9, -0xc2,0x37,0xd1,0x6c,0xa4,0xf1,0x47,0xe7,0x35,0xaa,0xef,0x61, -0xe0,0x6c,0x09,0x08,0xf8,0x78,0xf4,0x76,0x5c,0xcd,0xef,0x96, -0x97,0x54,0xab,0x6d,0x21,0x0c,0x12,0x6a,0x05,0x16,0xa1,0x92, -0xe0,0x26,0xeb,0x56,0x77,0x97,0x97,0x63,0xd8,0x62,0x5e,0xbb, -0x82,0xa9,0x0a,0x7d,0xd3,0x08,0x11,0x9a,0x00,0x63,0xf5,0x84, -0xcb,0x11,0x59,0xb3,0xa2,0x72,0x8d,0x24,0x92,0xc2,0x6c,0x98, -0xbd,0x80,0x48,0x7a,0x98,0x12,0x52,0x28,0xff,0xbf,0x27,0x81, -0x1a,0x46,0x81,0x30,0x08,0x3b,0x61,0x47,0xb5,0xe1,0x36,0xf4, -0x4d,0xc0,0xaf,0x08,0xb6,0xf8,0x56,0x92,0x29,0xff,0x5a,0xaa, -0xa7,0x3e,0xcf,0x0f,0x50,0x34,0x22,0xe6,0x55,0x2b,0xc1,0x64, -0x8b,0x9d,0x92,0xad,0xc7,0x44,0x03,0x57,0x42,0x27,0x9a,0x04, -0xdc,0xcd,0x11,0xa1,0x2e,0x39,0x9a,0x8e,0x88,0x2c,0xdf,0x72, -0x8c,0x95,0xbf,0x4f,0x7e,0x6d,0x0b,0xba,0x1e,0x0e,0x6c,0x7d, -0x02,0x8d,0x76,0x6a,0xa3,0x46,0x13,0x09,0x72,0x4e,0xb2,0x7b, -0x12,0x3a,0xb4,0x2b,0x0f,0xb9,0xdb,0xa6,0x04,0xe0,0x4d,0xf1, -0xb9,0x89,0x77,0xe7,0xb6,0xa3,0x78,0xbb,0x62,0x29,0x0a,0xe9, -0x9a,0xe7,0xa1,0xfa,0xd1,0x7f,0xe4,0x4e,0x71,0xef,0x4a,0x95, -0x30,0x0e,0x42,0xb3,0x7d,0xa2,0x9e,0xcb,0xe1,0x04,0xbe,0x50, -0xad,0xd1,0x7a,0x46,0x8c,0x79,0xeb,0x7b,0xd4,0x90,0xc3,0xbc, -0x48,0x36,0x17,0x6f,0xba,0x3e,0x8e,0x9a,0x1b,0x00,0xfb,0x78, -0xeb,0xd2,0xc2,0xd7,0x44,0x85,0xb6,0x8d,0x41,0x16,0x6c,0xa5, -0xa8,0xe9,0xca,0xfd,0x6b,0xae,0xd9,0x37,0x35,0xdb,0xba,0xf8, -0xd2,0x6d,0x97,0x75,0xba,0xf5,0x72,0x7d,0x48,0xbe,0xca,0xb0, -0xc7,0xbf,0x36,0xdf,0xc6,0x5f,0xd6,0x7c,0xc5,0x89,0xa9,0x64, -0x23,0x2e,0x56,0x2d,0xe6,0xb3,0x0a,0x4f,0x0b,0x00,0x73,0x33, -0x1e,0xa2,0x37,0x43,0x84,0x70,0x35,0xb9,0x76,0xd6,0xf1,0xeb, -0xc7,0xe1,0x7c,0x38,0x0a,0xdc,0x12,0x61,0x44,0x4b,0x0f,0x30, -0xb1,0x92,0xcf,0x40,0xd0,0xae,0x2a,0xe1,0x7f,0x5a,0x23,0x93, -0x37,0xf5,0x3e,0xd3,0x1e,0xd9,0xd2,0x8d,0x7b,0xd7,0x8d,0x5e, -0xea,0xd7,0x0c,0xe1,0xe5,0xa2,0x9a,0x1b,0xc5,0xce,0xdc,0x0c, -0x99,0x48,0xab,0x9c,0x15,0xa7,0x99,0xca,0xb2,0x62,0xdb,0x74, -0x1a,0x51,0x92,0x7f,0x76,0x66,0x1f,0xea,0xe5,0xe6,0x6b,0xf2, -0x78,0x3c,0xd1,0xcb,0x91,0xa7,0xc3,0xfc,0xc2,0x09,0x6c,0x5c, -0x38,0x4a,0x6a,0x6f,0xe8,0x82,0x86,0x8a,0xad,0x71,0x84,0x3e, -0xde,0x0c,0x57,0x64,0xb0,0x6c,0x32,0xbf,0xab,0x9a,0x78,0x41, -0x4e,0x84,0x14,0xee,0xe8,0x5a,0xbe,0x2d,0x57,0xe6,0xed,0xdd, -0x74,0x35,0x19,0x91,0xb4,0xf8,0x4b,0x0c,0x2e,0xa6,0x63,0x23, -0x08,0xdf,0xea,0xc3,0x0a,0xc9,0xf1,0x3a,0x1c,0x40,0x83,0x7e, -0x45,0xf9,0x70,0xbc,0x52,0xef,0x1f,0x6d,0x5a,0x24,0xe7,0xf1, -0x2d,0x3a,0x59,0x88,0x6b,0x8a,0x2d,0x77,0xd0,0x27,0x35,0xde, -0x32,0x2d,0xcc,0x53,0xba,0x21,0xdc,0xbf,0x46,0x31,0x61,0x05, -0x79,0xd4,0xfe,0x92,0xcb,0x09,0x87,0x6c,0x20,0xea,0x85,0x99, -0xc8,0x8d,0x5c,0x90,0xc7,0xd8,0x17,0x11,0x72,0xf1,0x5e,0x47, -0xbe,0x38,0xc8,0x63,0x7b,0x2b,0x91,0x94,0x37,0xad,0x55,0x23, -0x9b,0xc4,0x52,0xe9,0x6a,0x56,0x80,0x34,0x4d,0x51,0x5a,0x4e, -0x64,0x87,0x54,0xe6,0x42,0xa2,0x13,0x12,0xd5,0x3b,0xd2,0x4c, -0x5c,0x79,0xd0,0xb6,0xe8,0x8d,0xaf,0x19,0x8a,0xab,0x25,0xa3, -0x77,0x3f,0x07,0x3c,0x14,0xa5,0x0a,0xe0,0xd6,0xc4,0xef,0x66, -0x72,0xef,0x38,0xb5,0x0c,0x95,0xbf,0xdb,0x04,0xe4,0x8d,0xc7, -0x56,0x51,0x56,0x00,0x86,0xd3,0xab,0x39,0x38,0x75,0x0d,0xd6, -0xc5,0x91,0x15,0x9d,0x7c,0xc2,0xbd,0xfa,0x23,0x6b,0x28,0xd5, -0xb3,0xad,0xd7,0xc9,0xf3,0x97,0x7f,0x05,0x62,0x36,0xd2,0xa7, -0x83,0xe3,0x3f,0x01,0x1a,0x6b,0x5b,0x99,0x19,0x5a,0x09,0x69, -0x4c,0xcc,0x7a,0x84,0x55,0xbf,0x7d,0x7a,0xfa,0xfa,0xa7,0xb7, -0x27,0x4f,0xdf,0xfd,0xf4,0xf6,0x45,0xc1,0x26,0xe2,0x82,0x25, -0xf5,0x7b,0xf5,0x96,0x9c,0x74,0xe6,0x03,0xab,0x4b,0xc9,0x67, -0xe2,0xe8,0x86,0xb3,0x88,0x9b,0x8f,0x77,0xd5,0x82,0xc9,0xf1, -0x19,0x31,0x3c,0xfe,0x6f,0xee,0xde,0x7e,0xbf,0x8d,0xdc,0x58, -0x14,0xfc,0xfb,0xde,0xfb,0x12,0x54,0x1f,0x85,0xee,0x1e,0x42, -0x94,0x28,0x4f,0x72,0x92,0xe6,0xb4,0x79,0x34,0xb2,0x1c,0x3b, -0xb1,0x2c,0x1d,0x49,0x93,0x93,0x44,0xe2,0x38,0x94,0xd8,0x92, -0x68,0x53,0xa4,0x86,0x1f,0xb2,0x1d,0x91,0xef,0xb2,0x7f,0xec, -0x93,0xdc,0x27,0xdb,0xfa,0x02,0x50,0xe8,0x6e,0x4a,0xb2,0x93, -0xbd,0x39,0xbb,0x33,0x3f,0x8b,0x8d,0xaf,0x42,0x01,0x28,0x14, -0x0a,0x40,0xa1,0x2a,0x3a,0xde,0x7b,0xbb,0xb7,0x7b,0x42,0x11, -0x61,0xfb,0x57,0x4c,0xcc,0x69,0x3e,0x24,0x28,0x30,0x31,0x45, -0x0f,0x13,0xf9,0xf7,0xec,0x1a,0xc4,0x58,0x0b,0xfd,0x99,0x7f, -0xd3,0xc1,0x0a,0x7c,0x33,0xeb,0x05,0x36,0xef,0x37,0x6b,0xc2, -0x94,0x84,0xc9,0xeb,0xa9,0xc7,0xd5,0xb6,0x57,0x4f,0x05,0x34, -0xf0,0xfd,0xc8,0x64,0xd0,0x0a,0x26,0xe2,0x69,0xd8,0xab,0x6d, -0xf2,0x83,0x41,0x15,0x01,0x5b,0xec,0x62,0x54,0x76,0xbf,0xa4, -0xde,0x5b,0x69,0xe5,0x4a,0xde,0x0d,0xd2,0xc9,0x58,0x75,0x17, -0x41,0x3b,0xc6,0x37,0xec,0x2b,0x9b,0xd4,0xd6,0xec,0x30,0xa0, -0xf9,0x43,0xe2,0x53,0xaf,0x4f,0xf6,0xdf,0xd6,0x5e,0x1e,0xec, -0xb3,0x47,0x69,0xdf,0x5b,0xd3,0x1a,0x4c,0xae,0x1a,0xba,0x1b, -0x19,0x92,0x91,0xd7,0x66,0xad,0x76,0x38,0xcc,0xf1,0x65,0x13, -0x5a,0x7a,0xc5,0x3e,0x86,0xb9,0x55,0x43,0x2c,0x09,0x54,0x0c, -0xbb,0xaf,0x6b,0xf4,0x82,0x06,0xb1,0x17,0xc0,0x8c,0x3f,0x92, -0x71,0x14,0x58,0x82,0x6b,0xe8,0x58,0x7a,0x44,0x51,0x89,0x8d, -0x6b,0x46,0x05,0x9e,0x53,0x56,0x41,0x15,0x7b,0x52,0x6b,0x5b, -0x66,0xc5,0x1c,0xe3,0x46,0x9b,0x02,0xeb,0x52,0x65,0xb3,0x20, -0xc9,0xae,0xad,0xb1,0xd6,0x9c,0xa5,0x7c,0x68,0xb3,0xa5,0x18, -0x97,0x9d,0x76,0x93,0x84,0x2c,0xe4,0x03,0x08,0x3c,0x03,0x2e, -0x0d,0x4d,0xbd,0x5e,0x8c,0x11,0x1b,0x48,0xa2,0x83,0x2b,0xea, -0xc5,0xc9,0xd3,0xd8,0x0d,0xc3,0x9a,0xe6,0x81,0x67,0x57,0xf4, -0x6c,0x90,0x04,0x4e,0x48,0x57,0xee,0x95,0x84,0xd2,0x60,0x47, -0x70,0x44,0xe6,0x06,0xf1,0x4a,0xe6,0x9d,0x63,0xc5,0xa4,0xa5, -0x7e,0x89,0xbe,0x49,0x24,0xbf,0xcd,0x96,0x15,0xcb,0xf1,0x05, -0x32,0x7e,0xb1,0x1b,0x80,0x62,0xba,0xc8,0xd7,0xa2,0xaf,0x99, -0x55,0xc1,0x14,0x66,0x8f,0xb5,0x23,0xe9,0x06,0xaf,0xc8,0x95, -0x89,0x0b,0x1d,0x5f,0x3a,0x8a,0x2d,0x94,0xc3,0x9d,0x44,0x65, -0x5d,0xe2,0x77,0xc4,0x65,0x6b,0x34,0xba,0x99,0xb4,0xdc,0xbd, -0x28,0xfe,0x90,0x0d,0xcc,0x87,0xed,0xec,0x43,0x43,0xb5,0x6b, -0xa3,0x85,0x8f,0x85,0xab,0x50,0xb0,0x4f,0x85,0xa1,0x08,0xe0, -0x81,0xa1,0xed,0x4e,0x50,0xc7,0x87,0x6e,0x16,0x86,0xb7,0xbb, -0xa9,0x98,0x59,0x2c,0xe4,0x6b,0x57,0xc0,0xdf,0xc8,0x02,0x2c, -0x58,0xe3,0x74,0xc9,0x3d,0x56,0xaf,0xcb,0x72,0x26,0xa3,0x4c, -0xf2,0x56,0x2c,0xcd,0xa9,0x1c,0x3e,0xbe,0x3c,0xbc,0x9c,0xf4, -0xae,0x42,0x07,0x01,0xac,0x66,0x60,0x0d,0x46,0xbf,0x92,0xf4, -0x18,0x36,0xa1,0xf2,0x89,0x6e,0x52,0xf2,0x51,0x9f,0x6b,0xa8, -0x84,0x6c,0x09,0xe8,0x94,0xf7,0x0d,0xcd,0xfc,0xf3,0x6d,0x6f, -0xd4,0x1f,0x77,0x2b,0x87,0xa1,0x98,0xc9,0xf5,0xfd,0xc7,0xac, -0x65,0x3e,0x7e,0x5c,0x45,0x45,0xed,0x8f,0x1f,0x5f,0x7c,0x6c, -0x7f,0xb4,0xef,0xb2,0xed,0xd1,0x6f,0x89,0x26,0x3f,0x76,0xdb, -0xb2,0x7b,0xb1,0xf2,0x90,0xb5,0x52,0x8a,0x86,0x71,0x2b,0x5a, -0x64,0xb3,0x39,0x6f,0xfa,0x65,0x88,0xcb,0x0a,0xca,0xb7,0xa4, -0x53,0x9a,0x4d,0x82,0x6f,0xd6,0x22,0x13,0x98,0xfe,0xd0,0x20, -0x2b,0x3a,0x2f,0x12,0x4e,0xcf,0x86,0xb9,0xd6,0x0b,0x0d,0x12, -0x7b,0x5d,0x98,0x85,0x78,0x3f,0x6a,0xfa,0x2c,0xdb,0x1e,0x9a, -0x72,0x4d,0x77,0xbf,0xee,0x0e,0x9e,0xfc,0x4d,0xbe,0x5b,0xd8, -0xcd,0xba,0xb5,0x9e,0x93,0x2a,0x43,0x22,0x10,0x04,0x41,0x25, -0xb9,0xb7,0x5f,0x68,0x01,0x88,0xbf,0x04,0xfd,0x17,0x5b,0xf5, -0xba,0xb5,0x88,0xe5,0xed,0xef,0x84,0xb8,0x1a,0x07,0x67,0x09, -0xdb,0x30,0x26,0xd5,0x7f,0xac,0x22,0x05,0x64,0x65,0x5d,0xff, -0x13,0x2a,0x9b,0xe6,0x55,0xf6,0x72,0xcd,0x27,0x7c,0x39,0x85, -0x7d,0xe4,0x1e,0x70,0xa8,0x67,0x75,0xe4,0x90,0xf3,0x25,0x99, -0x84,0xce,0x47,0xe8,0xbb,0x75,0x36,0x69,0xc1,0x22,0x3e,0xd9, -0x56,0x0f,0xc4,0x48,0x89,0x91,0x32,0x4e,0x5b,0x19,0x66,0x10, -0x33,0xf7,0x9b,0x67,0xd3,0xc6,0x66,0x22,0x29,0xdb,0x98,0xb2, -0x1d,0xa4,0xa0,0x83,0x9f,0x7c,0x92,0x3a,0x6f,0xc6,0x64,0xf6, -0x43,0xe0,0x68,0xab,0x1f,0xf7,0x36,0x07,0xa5,0x65,0x92,0x03, -0xef,0x8c,0x3e,0x40,0x91,0x0f,0x52,0x64,0xdb,0xf1,0x17,0xe6, -0x6e,0x9c,0x59,0x72,0x6f,0x03,0xb3,0x48,0x70,0x5b,0x34,0x18, -0xcd,0xf3,0x1a,0xd5,0xdb,0x66,0xdc,0x59,0xe6,0xa0,0x5c,0xee, -0xea,0x85,0x53,0xf0,0x65,0xb7,0xbe,0x65,0x90,0x27,0x52,0x2c, -0xa5,0x82,0x44,0x18,0xb1,0xed,0xae,0xf9,0x84,0xf8,0x74,0xd8, -0xf1,0x72,0x21,0xc0,0xa5,0x13,0xca,0xcc,0x19,0x82,0xb1,0x2a, -0xf4,0xad,0x80,0x92,0xa5,0xc9,0x3a,0x8c,0x8b,0xc4,0x5a,0x5f, -0x73,0xdd,0x13,0x54,0xa1,0x20,0x8f,0xa3,0x14,0xd7,0x05,0xe9, -0x60,0x4d,0x9e,0xff,0x5a,0x91,0x15,0xa8,0xc1,0x58,0xd1,0xcf, -0x9c,0xf3,0xf3,0xfb,0x3f,0xe6,0x5f,0xb2,0x8a,0xc7,0xf8,0x61, -0x9b,0x70,0xe1,0x22,0x77,0xab,0xbe,0x10,0x08,0x03,0x85,0x76, -0xc3,0x04,0xbb,0x55,0xb6,0x98,0xfd,0x9b,0x20,0x5f,0x88,0x9b, -0xa3,0xef,0x39,0x6c,0xa6,0x8e,0x9f,0xbe,0x9c,0x6c,0x13,0x52, -0xf7,0x88,0x27,0x2b,0x64,0x71,0x65,0x41,0xc4,0x58,0x55,0x3a, -0x9b,0x8e,0x7a,0x1f,0xf3,0xf7,0xe4,0xe1,0x0c,0x51,0x8b,0x36, -0xd0,0xb1,0x89,0xeb,0x04,0x25,0x08,0x87,0x8d,0x01,0x49,0x26, -0xda,0xd1,0x92,0xb2,0x37,0x7c,0x80,0xf4,0x20,0x1b,0x89,0x30, -0x9d,0x77,0x10,0x44,0x2e,0xd0,0x39,0xa1,0x5d,0x0b,0xb2,0x38, -0xa1,0xc7,0x21,0xf3,0x86,0xf4,0xf0,0xd6,0x72,0x3c,0xbc,0xb3, -0xbe,0x05,0x0c,0x19,0x09,0x08,0x32,0xe3,0xfe,0x44,0xd5,0x5e, -0xaf,0xaf,0x05,0xc9,0xf2,0x2a,0x6b,0xb5,0x79,0x45,0xa4,0x42, -0x8f,0xdd,0x8a,0xe2,0x0f,0x18,0x43,0x44,0xcb,0x16,0x71,0x61, -0xe4,0xb2,0x68,0x3e,0xc2,0xdd,0x62,0x1a,0x35,0x02,0x68,0xd0, -0xbb,0x8e,0xab,0x40,0x3b,0xd6,0x5a,0xd8,0xf2,0xf9,0x70,0x98, -0x65,0x76,0xf6,0x94,0xbd,0x3f,0x14,0x08,0x49,0x6c,0x3a,0xe3, -0x34,0x72,0x6c,0x29,0xad,0x9a,0x64,0xee,0x79,0x97,0x75,0xe5, -0x5f,0xdc,0x04,0x48,0x21,0x17,0xd1,0x0e,0x44,0x4c,0x77,0x10, -0xab,0xa4,0x4c,0x22,0xac,0xc0,0x2f,0x11,0xbb,0x3e,0x93,0xc1, -0x79,0xd4,0x8e,0x27,0xf0,0x5a,0x01,0x16,0xf2,0xdb,0x4b,0xfd, -0xc2,0x91,0x1d,0x69,0x15,0x77,0x2b,0xd3,0x8a,0xed,0xca,0xb4, -0xb8,0x5f,0x31,0x43,0xe7,0xf5,0xa6,0x88,0x76,0x41,0xd2,0xa6, -0xa1,0x02,0x41,0xdb,0x5b,0xcd,0x09,0x9c,0xc3,0xa0,0x93,0xa4, -0x6a,0xb7,0x72,0xea,0xe8,0xca,0x97,0x11,0x61,0x7d,0xb1,0x40, -0x17,0xb6,0xa4,0x3a,0x29,0x97,0x91,0xe8,0x69,0x89,0x9d,0xbc, -0xd0,0xfe,0xee,0xf8,0xcb,0xcd,0xf9,0x78,0x98,0xc5,0xeb,0xca, -0x35,0x98,0x48,0x4c,0x22,0xd6,0xc4,0x11,0x3e,0xd3,0xd5,0x3b, -0x92,0xa6,0x2a,0x89,0x56,0x49,0x40,0xd2,0x10,0x30,0x41,0x2e, -0x17,0x4d,0x77,0x7f,0xa5,0xbb,0xa3,0x2c,0xba,0xbf,0x8f,0xc8, -0xf4,0xb8,0x85,0x05,0x54,0xbf,0x5c,0x46,0xe8,0x32,0x40,0x0a, -0x76,0x06,0x50,0x6e,0x86,0xdb,0x4b,0x6f,0x40,0xcc,0xaa,0x00, -0xda,0x33,0x31,0x77,0x01,0xe4,0xbc,0xdb,0x9c,0xdd,0x9f,0xdd, -0x6f,0x5e,0x19,0x05,0x38,0xf1,0x89,0xcb,0x25,0x24,0xb9,0x0a, -0x60,0xf8,0x0b,0xa6,0x67,0xb2,0xcd,0x9f,0xd9,0x18,0xcc,0xe9, -0xce,0xc6,0x5f,0xbb,0x9b,0xce,0xae,0x37,0x6f,0x23,0xb5,0x95, -0xea,0xb2,0x24,0x12,0xba,0x90,0xbd,0x80,0xc0,0x90,0x0c,0xb7, -0x90,0x25,0x17,0x8b,0xc1,0xe1,0xd1,0xde,0xab,0x37,0x7f,0x96, -0x0d,0x2b,0x6a,0xc9,0x25,0x81,0xe1,0x6b,0x7b,0x1a,0x58,0x70, -0x82,0xa9,0x5e,0xf0,0xa3,0xc5,0xeb,0xdd,0x77,0x27,0x47,0xf4, -0xa0,0x13,0xed,0x3a,0x9f,0x1d,0x37,0x92,0x18,0x56,0xe9,0xde, -0x14,0xad,0x57,0x9f,0x7d,0x42,0x53,0xd4,0xd6,0xae,0xb3,0xb5, -0x27,0x5b,0x70,0xc3,0x73,0xe1,0x9d,0xf5,0x26,0xf7,0xde,0x02, -0x1d,0x9d,0x90,0xc8,0x8d,0xac,0xaa,0x91,0x37,0x97,0xa9,0x8a, -0x91,0x6d,0xa1,0x02,0xf3,0x80,0xfd,0x55,0xeb,0x21,0xa3,0xca, -0xfc,0xa9,0x78,0x58,0x2a,0x19,0x49,0xc8,0xdd,0x3b,0xc4,0x6a, -0x8f,0xc7,0x62,0xdf,0x49,0xf9,0x1c,0x66,0x52,0xb9,0x1c,0xf0, -0x7b,0x63,0xf7,0xc2,0xdd,0x03,0x22,0xbf,0xff,0xa4,0xde,0xe1, -0xe3,0x84,0x95,0xba,0xee,0x4c,0x34,0x48,0xf7,0xd6,0x55,0xc1, -0x76,0xef,0x65,0x8d,0x07,0xa2,0x14,0xd7,0x4b,0x5a,0x3a,0xba, -0xa3,0x3b,0xba,0x03,0x55,0x02,0xf9,0x36,0x9f,0xa1,0x0d,0x09, -0xfd,0x40,0x2a,0x68,0xdc,0x1a,0xea,0x22,0x48,0x2e,0xeb,0x0d, -0xa4,0x90,0x6e,0x9c,0x41,0xd0,0x57,0x41,0xf7,0x15,0xb2,0x79, -0x2f,0xac,0x81,0xbd,0x69,0xef,0x65,0xba,0xdc,0xf3,0xaa,0xf9, -0x74,0x54,0xb5,0xc6,0xf1,0x30,0x5b,0xc7,0x44,0x37,0xde,0x91, -0x7c,0xd0,0x80,0x92,0xbf,0x0b,0xf5,0x34,0x1f,0x0f,0x62,0xf8, -0xe5,0xe0,0x2e,0x39,0x63,0xae,0x41,0xad,0xe3,0xc9,0x4c,0x91, -0xb9,0xdc,0xbc,0xf6,0xa6,0xfc,0xda,0x74,0xad,0xf6,0x6e,0x5c, -0x63,0xb8,0x35,0xae,0xd5,0xfa,0xc6,0xed,0xd7,0xee,0x06,0xbd, -0xda,0xdf,0xb8,0xea,0xbf,0x35,0x23,0xdd,0xe0,0xc5,0x42,0x0d, -0x35,0x11,0xbe,0x6a,0x4a,0x3b,0x7c,0x8f,0xe6,0x53,0xba,0x59, -0xc1,0xdd,0xb4,0x23,0xbb,0x65,0x60,0xa5,0xde,0xee,0x28,0x1f, -0xf3,0x0c,0xe3,0x88,0xba,0x40,0xec,0x81,0x93,0x31,0xe7,0x99, -0x34,0xb4,0x84,0xbf,0x57,0x58,0xb4,0x56,0xd5,0x45,0xbe,0x67, -0xfc,0x04,0x83,0x60,0x79,0x4e,0x69,0x87,0x9a,0xe2,0x19,0x94, -0xfc,0xc8,0x38,0x7f,0xa0,0x49,0xd0,0x3e,0x7a,0x5d,0xfa,0x9a, -0x6f,0x8d,0x62,0xb9,0x3d,0xe2,0x59,0x28,0x32,0xa3,0x19,0x18, -0xca,0xd3,0x57,0xfe,0x0c,0x24,0x5f,0x27,0x76,0x56,0xd1,0x39, -0xc2,0x7a,0xd3,0x3a,0x1b,0x45,0x89,0x76,0x65,0x39,0x42,0xf5, -0x00,0xbc,0x2b,0x54,0x1e,0xb1,0x52,0xc8,0x02,0x55,0x28,0x5f, -0xb7,0xa4,0x24,0x40,0x99,0x9c,0x19,0xad,0x41,0xc2,0x76,0x0d, -0xb2,0x52,0x1a,0x6b,0xef,0x92,0xdc,0x14,0x33,0x7e,0xb4,0xf4, -0x75,0xd4,0x77,0x23,0x8b,0x4c,0x8d,0xf4,0x7e,0x52,0x15,0x9b, -0x91,0x57,0xcf,0x44,0x1a,0x95,0x48,0x92,0xef,0x10,0x69,0xca, -0xef,0x79,0x1a,0x06,0x3d,0x22,0x01,0xe0,0xa3,0x99,0xe3,0xa6, -0x36,0x43,0x47,0x3e,0xbc,0xe5,0xc2,0x92,0x85,0xdf,0x60,0xdd, -0xf0,0xb0,0xd0,0xe0,0xbd,0x07,0x5c,0x39,0x1e,0x09,0xf1,0xe6, -0x8e,0xcf,0x76,0xea,0x7b,0x8d,0xa0,0x82,0x94,0x41,0x16,0x04, -0x7d,0x0e,0x75,0x80,0x46,0xda,0x33,0xe8,0x0d,0x00,0x6d,0x71, -0xc4,0x78,0x6f,0xe0,0xae,0x09,0x2f,0x47,0x53,0x65,0xb2,0xfb, -0x95,0x17,0xb0,0xa0,0x41,0xf0,0x27,0xc8,0x9c,0xa4,0x6e,0xb8, -0x2f,0xb5,0xfb,0x4a,0x94,0xa1,0xc8,0x2b,0x07,0xc8,0x20,0x41, -0x7e,0x71,0x46,0xa3,0x56,0x3e,0xe0,0xd7,0xa2,0xae,0x82,0xbe, -0x36,0xe6,0xbe,0x6e,0x0e,0xbe,0xc8,0xb6,0xb7,0x60,0x1b,0xfd, -0x7c,0x6b,0xeb,0x05,0x47,0xa8,0x29,0xf2,0x7a,0x36,0xbb,0x2d, -0xac,0x96,0x7f,0x38,0x3e,0x78,0xf7,0xfe,0xf8,0x64,0xe7,0xe8, -0x44,0x2c,0x37,0x9c,0x9d,0x2e,0xce,0xee,0x4f,0x7f,0x3e,0xbb, -0xef,0x26,0x9b,0x86,0x52,0xf7,0xde,0xbd,0xcc,0x36,0x4f,0xcf, -0x96,0x67,0xdd,0x2e,0xdb,0x70,0x38,0x3c,0x3a,0x38,0x41,0x6b, -0x54,0x90,0xc6,0x4b,0x34,0x16,0x4d,0xce,0xba,0x67,0xcb,0x67, -0xa6,0x73,0x36,0xda,0x34,0xbb,0x07,0xef,0x4e,0xf0,0xdc,0xf9, -0xe4,0x2f,0x87,0x7b,0xef,0x77,0x0e,0x0f,0xdf,0xbe,0xd9,0xdd, -0xa1,0xec,0x08,0x2f,0xbb,0xa7,0x77,0xb1,0x30,0x8b,0x36,0x50, -0xd9,0x04,0x68,0x18,0x27,0xd8,0x80,0x95,0x19,0x37,0x3f,0x4c, -0xc7,0xa3,0x36,0x6a,0x95,0xc1,0xb6,0x3e,0x9b,0xcf,0x2e,0x37, -0x7e,0x1b,0xa1,0x0b,0x13,0x7a,0x58,0x2d,0x42,0xaf,0x0b,0xdd, -0xbb,0x41,0x39,0x92,0xfb,0xdb,0xf4,0xd4,0x3f,0x4c,0xc6,0x5e, -0x2b,0x39,0x5c,0xa0,0x58,0x54,0x90,0xc4,0xbe,0xc6,0x3f,0x4a, -0xd8,0x28,0x34,0x0a,0x05,0x0e,0xe3,0xbb,0x87,0x0f,0xcc,0xa5, -0xbc,0xed,0x97,0x20,0x92,0x81,0x5e,0x4e,0xc6,0x37,0x7f,0x98, -0x5a,0x04,0x12,0x1e,0xbe,0x65,0xd7,0x28,0x5c,0xc9,0x89,0x96, -0x46,0x55,0xe1,0x69,0x1f,0xbd,0x01,0xbc,0x35,0x20,0x26,0xb2, -0xc3,0x94,0x74,0x66,0x63,0x06,0x99,0xa4,0x7d,0x00,0x65,0x67, -0x09,0xba,0x30,0xb8,0x19,0x8f,0xd2,0xfb,0x9d,0x0b,0x64,0x7d, -0xe5,0x8e,0x34,0x35,0x3c,0xf5,0xde,0x84,0x06,0x0e,0xe0,0xfb, -0xbb,0xcd,0xef,0xa0,0x2f,0x51,0x1f,0x3a,0x7d,0x70,0x80,0x0c, -0x7a,0x20,0x7a,0x24,0x07,0x2e,0xed,0x0f,0xe7,0x59,0x9a,0xcf, -0xd3,0xc9,0x25,0x3b,0x84,0xa3,0xcd,0x6e,0xf4,0xe7,0xe3,0xa3, -0x57,0x1b,0x27,0x07,0x7f,0xdc,0x7b,0x17,0x51,0x1a,0x4f,0x50, -0x49,0xdb,0x50,0xa9,0x4b,0x3e,0x9b,0xc7,0x36,0x8d,0x27,0xec, -0x0b,0x00,0xbd,0x09,0xb2,0x1d,0x7d,0x9f,0x42,0x07,0x34,0xf6, -0xe6,0xfe,0xcd,0xca,0x12,0x15,0x39,0xc8,0x27,0x45,0xb0,0x28, -0xa0,0x2a,0xd9,0x8f,0xbd,0x8b,0x8f,0x64,0x8a,0x30,0x5a,0x3f, -0x67,0x77,0x62,0x64,0xcd,0x3e,0x70,0xda,0x52,0xb0,0x67,0xff, -0x0b,0x1b,0xcd,0xf7,0x02,0x9c,0xb2,0x66,0xef,0x97,0x18,0x05, -0xdc,0x58,0xd0,0x26,0x00,0x1c,0x58,0xa5,0xff,0xc5,0x28,0xa9, -0xcf,0x01,0x54,0xa7,0x57,0x04,0x90,0xec,0xd7,0x90,0xfb,0x3a, -0x56,0x52,0x28,0x73,0x2a,0x3b,0x29,0x9c,0x76,0x83,0xf3,0xab, -0x7e,0xab,0x74,0x09,0x9d,0xea,0x03,0xf1,0x9f,0x34,0xe4,0x73, -0x36,0x91,0x2e,0x4a,0x5d,0xd6,0x66,0xa8,0x20,0xd1,0x2c,0xd5, -0xa8,0x9c,0xe3,0x7b,0x6e,0xe5,0x4a,0x0b,0xdb,0xea,0x60,0x44, -0xba,0xfe,0x0b,0x0c,0x10,0x91,0x3c,0x06,0x93,0x82,0x66,0xb0, -0xe5,0xe1,0x17,0xc5,0x46,0xa2,0xf9,0xbb,0xca,0x05,0x97,0x03, -0xc2,0x60,0xda,0x85,0xa5,0xd5,0x8f,0x09,0x47,0xbc,0x1a,0xc9, -0x5c,0x42,0xc9,0xfe,0x55,0x31,0x0d,0xa7,0x74,0x00,0x2e,0xb3, -0x29,0xb0,0x4b,0xc3,0x95,0x62,0x2d,0x0b,0x92,0xed,0xd2,0x72, -0xca,0xbf,0xdd,0x30,0xd5,0x9e,0xf5,0x17,0x32,0xe1,0x35,0x0d, -0x69,0xa5,0xe7,0x6a,0x3e,0x18,0xb7,0x32,0xbd,0x0c,0xa2,0x61, -0xc8,0x55,0xc8,0x15,0x99,0x66,0x96,0x25,0xba,0x91,0x71,0x39, -0xb5,0xaf,0x33,0x19,0x2e,0xdb,0x5f,0x6d,0x05,0xc0,0x67,0xf2, -0x91,0x4d,0x66,0x2f,0x2a,0x46,0xad,0x98,0x02,0xeb,0x86,0x2c, -0x4b,0x25,0x5d,0xd8,0xd9,0xaa,0x11,0xf1,0x25,0xc2,0x78,0x8f, -0x55,0x62,0x8d,0x65,0x48,0xf8,0x0d,0xcc,0x4d,0xe2,0x5b,0x74, -0x6e,0x1a,0x74,0x06,0x39,0x23,0xf3,0x00,0xef,0xab,0xfb,0x46, -0x89,0x40,0x41,0x69,0xd6,0x9f,0x0b,0x3a,0x0e,0x01,0x2a,0x4c, -0xd0,0xdd,0x98,0x2b,0x1b,0x64,0x4c,0xd0,0xf2,0x52,0x65,0x6d, -0xfe,0xd0,0x75,0x45,0x33,0xda,0xbe,0x82,0xd3,0x00,0x9f,0x6e, -0xa6,0xba,0x33,0x4c,0xf1,0x9e,0x83,0x6d,0xd1,0xa5,0xec,0x67, -0xa1,0xa7,0x83,0xc5,0x8e,0x17,0x10,0x37,0xe8,0xc5,0x14,0x53, -0x5e,0x17,0xab,0xf2,0x72,0xd2,0xd2,0x2e,0x27,0x59,0x30,0xe3, -0x42,0xee,0xd2,0xf6,0x3b,0x5d,0x08,0x9a,0x30,0xb1,0x40,0x57, -0x59,0x81,0x3b,0x30,0x89,0x64,0x73,0x34,0xfc,0x56,0x41,0x3a, -0x74,0xbe,0x82,0xcb,0x01,0x2b,0x4b,0xa8,0xb3,0x43,0x60,0x1d, -0xd0,0x2d,0x07,0xf4,0x60,0x3f,0x56,0x6a,0x54,0x1d,0xcb,0x43, -0xad,0xe3,0xdc,0x38,0x39,0x95,0xe4,0x70,0xc9,0x59,0x2c,0x5c, -0xb3,0xc3,0x84,0xae,0x12,0x30,0x5d,0xd5,0x6e,0xb6,0x4f,0x35, -0x34,0x3f,0x3e,0x05,0x68,0x7a,0x48,0x1d,0x0c,0x71,0x35,0x48, -0xc7,0x54,0xd6,0x9b,0xba,0x56,0x2a,0x26,0x26,0x66,0xfb,0x29, -0xec,0xb6,0x36,0xf3,0xe6,0x5f,0x90,0xef,0x66,0x21,0x17,0x96, -0x8c,0x5a,0x2e,0x2c,0x48,0xd6,0x15,0x9c,0x98,0x2a,0x57,0x6c, -0xd8,0xfb,0xfb,0x53,0xe0,0x12,0x7f,0x52,0x58,0x62,0x91,0x7c, -0x88,0x6d,0xf9,0x63,0x24,0xfa,0xa8,0x1b,0xb8,0x7f,0x8d,0xf4, -0xc4,0x90,0x92,0x64,0x66,0xb2,0x8a,0xc3,0x2d,0x43,0x6f,0x83, -0x52,0x3b,0xda,0x49,0xda,0x9d,0xe4,0xb4,0x93,0xc4,0x9d,0x33, -0x89,0x3b,0x3e,0x97,0xeb,0xea,0x72,0xbe,0x15,0x10,0xb2,0x95, -0x45,0xcc,0x14,0x48,0x17,0x3a,0x44,0x51,0xef,0x4b,0x2d,0x62, -0x37,0x67,0xd7,0xb9,0x58,0xbc,0xd7,0x13,0xa3,0x3c,0x8b,0x12, -0x60,0xa1,0xd7,0x20,0x4a,0x65,0xa7,0xc1,0x08,0x7b,0xb3,0x23, -0x5d,0x03,0x3b,0xed,0x9b,0xc1,0x14,0xc8,0xf8,0x97,0xe6,0x27, -0x84,0x2a,0x63,0x4e,0x4c,0xc8,0x1b,0xab,0x23,0x37,0x46,0x7d, -0x2d,0x90,0x28,0xc5,0x40,0x1f,0x9b,0xdc,0xeb,0x50,0x53,0x66, -0xdd,0x62,0x51,0x11,0xb9,0x87,0x3b,0xd5,0x04,0xdf,0x70,0x01, -0x7e,0x4e,0x47,0xb8,0x22,0xa3,0x59,0x59,0xd8,0x14,0xb2,0x73, -0x9b,0x8b,0xb5,0x71,0x6c,0x58,0x9d,0xd7,0xb6,0x2a,0x64,0x33, -0xab,0xcb,0x82,0x84,0xc0,0x85,0x03,0xbd,0x5b,0x1c,0x09,0x54, -0xe8,0xa5,0x14,0xab,0x0c,0xcb,0xf2,0x41,0x31,0xba,0x6d,0xbb, -0x5a,0x7e,0x65,0x14,0x09,0x80,0x2b,0xe2,0xee,0xbe,0x6c,0x26, -0xd1,0xf4,0xaf,0x74,0xcf,0x1f,0x00,0x2a,0xa9,0x86,0x26,0x78, -0x52,0xbe,0x42,0x1e,0x62,0x89,0x66,0x95,0x7c,0x44,0x1b,0x65, -0x86,0xbd,0xb4,0x1f,0x7c,0xb4,0xf0,0x38,0x1a,0xf4,0x3c,0xe3, -0xff,0x65,0x5c,0xd4,0x9b,0x1b,0x3a,0xcc,0x3e,0xbe,0x1e,0x4f, -0x66,0x6c,0x32,0x72,0x1a,0xd3,0x35,0x25,0x91,0xad,0x3b,0xfb, -0x30,0x85,0xcd,0x38,0x89,0xa3,0x72,0xcc,0xe9,0x92,0xd0,0x41, -0xbe,0x65,0x78,0xf6,0x15,0x07,0x89,0xad,0xc1,0x4a,0x42,0x36, -0x2b,0xee,0x79,0x25,0x48,0xe9,0xd4,0x09,0xca,0xa5,0xf0,0x0f, -0x35,0x58,0x82,0xd7,0x40,0x25,0xcc,0x50,0x95,0x85,0xb8,0xe3, -0x3f,0x82,0x21,0x75,0xdd,0x3f,0x80,0x26,0x01,0x48,0x69,0x87, -0x57,0x40,0x78,0x05,0xcf,0x51,0x72,0x87,0x17,0x66,0xfb,0xa8, -0x8f,0x5d,0x18,0x37,0xcb,0x9d,0xc4,0x4d,0x3b,0xbf,0x5c,0x01, -0xe6,0x57,0xda,0xfd,0x77,0xd8,0xc5,0x25,0xec,0xd9,0xa8,0x45, -0xa7,0x45,0x38,0x55,0x67,0x21,0x02,0xb4,0x9b,0xb0,0xcb,0x5b, -0xab,0x14,0x81,0x8b,0x2b,0x4e,0x37,0xba,0xb2,0x3b,0x64,0xe2, -0xf0,0xaa,0xd1,0x02,0x38,0x04,0x11,0xdc,0xb3,0xac,0xdf,0x5e, -0xf7,0x90,0x63,0xe8,0x38,0x3e,0x3c,0x4b,0xb4,0x26,0xd2,0x53, -0xc0,0x27,0xf7,0x1c,0xf6,0x2f,0x66,0x25,0x7d,0x0b,0xf9,0x54, -0xa9,0x0f,0xc8,0x69,0x35,0xb0,0x7c,0x7b,0xdf,0x98,0xaa,0x08, -0xe4,0x04,0x49,0xcc,0xbb,0x9b,0x42,0x6d,0x69,0x58,0x69,0xfa, -0xe0,0xe2,0x9a,0xf2,0xcf,0x32,0x68,0x0b,0xf6,0xdb,0x61,0x6e, -0x9d,0x63,0x5b,0xc7,0xa5,0xfd,0xcf,0x99,0x3d,0xa2,0xe3,0x47, -0x4a,0xb7,0xb9,0xf6,0x9f,0xed,0x26,0x63,0x1b,0x9f,0x78,0x66, -0x90,0xbd,0x5e,0xaf,0xcc,0xe7,0x1e,0x7f,0xf7,0x3f,0x9b,0x56, -0xe2,0xdd,0xa8,0xb2,0x1b,0x52,0x7c,0xe7,0x74,0x6b,0x6c,0x4b, -0x71,0xc9,0xa1,0xef,0xd8,0xcd,0xf0,0xcc,0xf5,0x82,0x44,0x20, -0xd9,0x66,0xe7,0xf3,0xc1,0xb0,0x8f,0x4f,0xa3,0xbc,0x48,0x65, -0xc5,0x87,0x5b,0xf2,0xac,0x47,0x8a,0x83,0xd5,0xf8,0x58,0x25, -0x71,0x12,0xfb,0x02,0x46,0x55,0xec,0x0a,0x53,0x8c,0x80,0x71, -0x93,0x5a,0x08,0x79,0x25,0x4d,0x51,0x18,0x17,0x13,0x95,0x2c, -0xd7,0xab,0xd1,0xef,0xf7,0x4e,0xd8,0x94,0x9b,0x97,0x18,0x51, -0x02,0xc0,0x2c,0xfe,0xf8,0x50,0x17,0xa4,0xab,0x03,0x17,0x4a, -0xfd,0x81,0x4a,0x58,0x5b,0x27,0x0c,0x5b,0xf9,0x98,0x1e,0x84, -0x25,0xdc,0xbd,0x89,0xf5,0x4c,0x4d,0xfd,0x2c,0x9e,0x64,0xf1, -0xb5,0x19,0x4e,0x13,0xe3,0x7d,0xf3,0xf9,0x3c,0x09,0x9d,0xf9, -0xfb,0x30,0xf5,0x4c,0xa2,0x5d,0xc9,0xfa,0xe8,0xa7,0x74,0x98, -0x2f,0xd2,0xb6,0x46,0x70,0x55,0x65,0x9d,0xc2,0x4c,0xf2,0x49, -0x78,0x0f,0xa3,0x42,0x5b,0xe8,0x99,0xfe,0x56,0x97,0x25,0x23, -0xed,0xe9,0xca,0xf2,0x66,0x7b,0x6b,0xcb,0xdc,0x2f,0xed,0x03, -0xa6,0x80,0xbf,0xc8,0xa0,0x57,0x0b,0x96,0x1e,0x39,0x21,0x68, -0x39,0xf5,0x08,0x45,0x7e,0xa4,0x42,0xc7,0x10,0x91,0xf7,0x29, -0xae,0xe8,0x04,0x59,0xf6,0x46,0x6f,0xaa,0x65,0x3d,0x1b,0x6d, -0xa7,0x33,0x3d,0x5a,0x2b,0x2f,0x67,0x8e,0xd2,0x09,0x71,0xa6, -0x6d,0xba,0x95,0x91,0x6f,0x69,0x02,0xa4,0xb6,0x45,0xef,0x7c, -0x56,0xe1,0xad,0xf4,0x98,0x34,0x7e,0x63,0x2e,0x53,0xf9,0x68, -0x97,0xcf,0x02,0xc4,0x82,0xf2,0x1d,0x1b,0xc2,0x74,0x32,0x21, -0xf1,0x6c,0x1e,0x3c,0xbe,0x67,0x5f,0x2c,0xac,0xfd,0x62,0xfa, -0xe9,0xfa,0x57,0x53,0x0c,0xd2,0xd7,0xa0,0xee,0x1b,0xef,0xc8, -0xec,0x71,0x26,0xe7,0x80,0x77,0x6c,0xf8,0xd2,0x4e,0xc8,0x7c, -0x44,0xae,0x2a,0x26,0x83,0xff,0x9c,0xe7,0x13,0x7e,0x75,0xdb, -0x88,0xb2,0xa8,0x51,0x88,0xbf,0x4b,0x58,0x31,0x14,0xfb,0xbf, -0x11,0x6f,0xb4,0xb2,0x0c,0x3e,0xfc,0x5d,0x42,0x27,0x4a,0x3a, -0xf0,0x27,0x8d,0xea,0x51,0xd2,0x60,0xe8,0xac,0x72,0x03,0x61, -0x7b,0x44,0xe1,0xe6,0x48,0x56,0x70,0x4f,0xcc,0x6e,0x21,0x13, -0x53,0x25,0xdb,0x56,0x38,0x80,0xad,0x3a,0xdf,0xab,0x14,0x81, -0x95,0xe7,0xdc,0x32,0x60,0x2f,0xe7,0xda,0xe3,0xdd,0x8a,0x92, -0x9d,0xf0,0x69,0x7f,0x45,0x8e,0xb4,0xe4,0x4d,0xb6,0x22,0x93, -0x7e,0xbc,0xfd,0xd0,0x99,0xe3,0x43,0xad,0x30,0xe2,0x98,0xd1, -0x9e,0xc2,0x21,0x0c,0x10,0x6b,0xff,0x0f,0x60,0xdf,0xae,0xec, -0x3e,0xbb,0xba,0x20,0x56,0x06,0x66,0xbc,0xc5,0x29,0xad,0x10, -0x38,0xdd,0xd9,0x84,0x45,0x3b,0xb6,0x5b,0x1c,0x97,0x27,0xf1, -0x87,0x89,0x24,0xe3,0x3f,0x19,0x8c,0x3a,0x00,0x14,0x40,0xa8, -0xbd,0x6c,0x2a,0x97,0x20,0x3c,0xf3,0xad,0x10,0x53,0x23,0xe8, -0x9d,0xc8,0x44,0xbc,0x03,0x85,0x0f,0x5c,0xbb,0xe1,0x07,0x0f, -0xc1,0x91,0x2c,0x1f,0x10,0x1f,0x23,0x3c,0x10,0x87,0xac,0xc0, -0xe0,0xdc,0x9b,0x62,0x77,0xc3,0x60,0x3f,0x38,0x3e,0xb8,0x6e, -0x7c,0xed,0xd9,0xdb,0xaa,0x9b,0x46,0x75,0x8a,0x6c,0x2f,0x38, -0xb5,0x0b,0x54,0x7f,0x40,0xec,0xce,0x84,0xe5,0xbe,0xda,0x65, -0xf2,0x1a,0x12,0xd4,0x00,0x55,0xa7,0x2f,0xf4,0xe7,0xd7,0x47, -0xee,0x54,0x99,0x25,0x00,0x0b,0xa7,0xd9,0x1b,0x5d,0xcd,0x87, -0xbd,0x09,0x39,0xac,0x3c,0x87,0x72,0x53,0xa3,0x75,0x57,0x5c, -0x36,0x6b,0x49,0x82,0xf5,0x48,0x2f,0xc6,0x43,0x77,0x29,0x02, -0xdc,0x80,0xf5,0x2e,0xba,0x45,0x51,0xfc,0x51,0x54,0x5e,0x12, -0x26,0xbe,0xe6,0x49,0xef,0x93,0xbd,0x05,0x76,0x56,0x42,0x0e, -0xa5,0x3e,0x25,0x0b,0xd3,0x88,0xa1,0x28,0x45,0xd2,0xf5,0x78, -0x94,0xdb,0x07,0x99,0x93,0xc1,0xed,0x2c,0x53,0x40,0x8a,0x9a, -0x37,0x9c,0x03,0x86,0x10,0x0b,0x89,0x9f,0x01,0xed,0x51,0x5e, -0x17,0x3d,0x1f,0xf7,0xbf,0x58,0xfd,0x52,0xd2,0xfb,0xe5,0xc2, -0x5c,0xb6,0x5e,0x27,0x79,0x3c,0x71,0x17,0xb5,0x9c,0xd8,0x24, -0xfd,0xda,0x88,0x2e,0x54,0x3e,0xf4,0xee,0x7a,0x52,0x9f,0x91, -0xd4,0xe9,0xe4,0x02,0xd9,0xa9,0x41,0xa5,0xb8,0x8e,0xc4,0x8d, -0x47,0x80,0x62,0xff,0x0b,0x0a,0x99,0x39,0xec,0x5d,0x47,0x57, -0xb9,0x46,0x68,0x13,0x5f,0x0b,0xe6,0xfd,0x05,0xea,0xcb,0x20, -0xd5,0x6e,0xf2,0x65,0x92,0x94,0xa5,0x92,0xc7,0x58,0x32,0x61, -0x84,0xac,0xeb,0x84,0x64,0x99,0x3a,0xf0,0x08,0x21,0x73,0x21, -0xde,0x52,0xaa,0xbc,0xa6,0xd4,0x66,0xad,0xeb,0xac,0xdb,0x2c, -0x05,0x96,0xc5,0xdb,0x56,0xb5,0x60,0x93,0x1d,0x1d,0x3b,0x9c, -0xee,0xee,0xd3,0xae,0xd3,0xc5,0x05,0x3a,0x58,0x99,0xd5,0xdd, -0x04,0x67,0x97,0xc9,0x1c,0x3b,0x29,0x1f,0xd5,0xea,0x71,0xe0, -0x5f,0x52,0xff,0xbb,0x4f,0x10,0x65,0x3f,0x5f,0x4f,0xea,0x75, -0xf8,0xd3,0xec,0x9d,0xa3,0xd7,0x98,0x82,0x81,0x08,0xec,0x37, -0x0b,0xcc,0xe1,0xf6,0xc8,0x4e,0x8a,0x96,0x7a,0x21,0xbc,0x0c, -0xc9,0x4c,0x34,0x5b,0x8e,0xf7,0x8e,0xfe,0xb4,0x77,0xf4,0x7e, -0x7f,0xe7,0x64,0xf7,0x35,0x3e,0x27,0x8b,0xa2,0x12,0x9d,0x76, -0x13,0x34,0xe7,0x2e,0x8d,0x78,0x03,0x4b,0x7b,0x40,0xec,0x30, -0xcd,0x46,0x17,0xf9,0x30,0x76,0xe9,0x89,0x6f,0x54,0x06,0x4d, -0x60,0x53,0xa6,0xd2,0xe2,0x08,0x9d,0xd2,0x83,0x68,0x6b,0x31, -0xe9,0x58,0x74,0x3b,0x20,0x7c,0xa5,0xdf,0x6f,0x7d,0x6f,0xf7, -0x26,0x92,0xbf,0xb5,0xbd,0xfd,0x9c,0x34,0xc0,0x20,0x00,0x59, -0x5c,0xb2,0x6d,0xf5,0x23,0xfb,0x47,0xcf,0x21,0xd6,0xd7,0x6d, -0xc7,0x1d,0xcc,0x67,0xa8,0xab,0xa1,0xb8,0x6b,0x3c,0x1a,0x8f, -0x6f,0x79,0xb1,0x67,0x70,0xb4,0x13,0xf0,0x25,0x07,0xa3,0x8b, -0x72,0x21,0x7a,0x91,0x10,0x93,0x54,0x81,0x33,0x00,0xb6,0x7f, -0xb6,0x00,0x84,0x20,0x5e,0x98,0xb0,0x3e,0x31,0x94,0xa3,0x5f, -0xd1,0xd7,0x92,0x16,0xbc,0xe9,0x67,0xd1,0xfb,0xa8,0xe1,0xc6, -0x11,0xef,0x1f,0xe6,0xb8,0x5c,0x35,0x1a,0xe8,0xb8,0x4c,0xd6, -0xc8,0xe7,0xbf,0x49,0xda,0x2e,0xc7,0xa9,0x2f,0xab,0x36,0xf6, -0x7c,0xf3,0x5b,0x99,0xa9,0xe9,0xee,0x7d,0x59,0xaf,0xcf,0x8f, -0x8f,0xe6,0x3b,0x9e,0xfd,0xd1,0x3d,0xef,0xee,0xce,0xdb,0xb7, -0x3f,0xee,0xec,0xfe,0x11,0x58,0x61,0x89,0xa1,0x36,0xa3,0x86, -0x07,0xaf,0xd4,0x35,0x1e,0xaa,0xbf,0xb3,0x92,0x72,0x51,0xee, -0x5e,0x5d,0x2e,0x49,0x1f,0x21,0xf9,0xc5,0x62,0x63,0xdb,0x3d, -0x5d,0xa8,0x84,0xb3,0x54,0xba,0xd3,0x44,0x91,0xf9,0xa7,0x1a, -0x30,0xed,0x36,0x4e,0x30,0xd8,0xae,0x07,0x13,0x1e,0x15,0xa5, -0x1e,0x39,0x1c,0xf6,0xea,0xe3,0x3c,0x47,0xa7,0xb9,0x9d,0xda, -0x2c,0xce,0x2b,0x95,0xe9,0x25,0x4d,0xe5,0x47,0x58,0x22,0x50, -0xdb,0xf7,0x19,0x4e,0x15,0xcd,0xfe,0x02,0x71,0xc9,0xde,0x57, -0x61,0x1e,0xf4,0xc4,0x36,0x1c,0x1e,0x85,0x29,0x40,0x70,0xd3, -0x01,0xf6,0x92,0xcd,0x79,0x1a,0x91,0xc0,0xba,0x21,0xfa,0x82, -0xe2,0xbb,0x04,0x4f,0xb2,0xdf,0xe2,0x68,0xf6,0xae,0x72,0x15, -0x45,0x0a,0x09,0x64,0xda,0x7f,0x00,0x35,0xc2,0xd7,0xdb,0xde, -0x74,0xb6,0xb1,0x3f,0xee,0x93,0xc7,0x3e,0x08,0x1f,0xe2,0xab, -0x91,0x5e,0x84,0x02,0x6d,0x50,0x2d,0x08,0xf5,0x45,0x14,0x81, -0x7f,0x38,0x6b,0x3f,0x1a,0xa3,0xe2,0x25,0xa4,0x7a,0x72,0x60, -0x9b,0x15,0xb6,0xc9,0xe6,0x6b,0x7b,0x47,0x31,0x41,0x4d,0x0d, -0xb9,0x71,0x61,0x23,0xa0,0x6c,0xb1,0x88,0xb4,0x87,0x96,0x62, -0x9e,0xf3,0x41,0x92,0xa1,0x71,0x63,0xde,0xc1,0x1d,0xef,0x19, -0x77,0x47,0x47,0xa4,0x41,0x2a,0xbe,0xff,0x2d,0xa0,0x01,0x52, -0x5b,0x71,0x19,0x00,0x5c,0xb1,0x54,0xf1,0xa4,0x1e,0x29,0x4b, -0xd7,0x23,0xd9,0x74,0x54,0x16,0x6e,0xed,0x98,0xb8,0x40,0xd4, -0xc0,0x55,0x08,0xdf,0x1d,0x90,0x83,0x3a,0x61,0xb4,0x2f,0xb6, -0x12,0x3a,0x3b,0xb6,0x6c,0x37,0x0b,0xb8,0x72,0x1c,0xae,0x39, -0x76,0xc5,0x4a,0xd8,0xbe,0x9d,0x84,0xea,0x75,0xf9,0xa0,0xbd, -0x79,0x18,0x2a,0x00,0xd0,0xaf,0xea,0xd6,0xd5,0xcb,0xd7,0x82, -0x5e,0x8d,0xd6,0x18,0x46,0xed,0x5d,0xa5,0xf9,0x8b,0x9a,0xbb, -0xac,0x7d,0xa0,0x33,0x15,0x1f,0xf6,0xe9,0x47,0x1b,0x9d,0x58, -0xe7,0xe4,0xb9,0xc7,0x6e,0xfe,0x55,0xbc,0xa8,0x95,0xfa,0x7a, -0x1e,0x86,0xe8,0xf3,0x69,0x78,0x2e,0xb6,0xa0,0xa4,0xea,0x9c, -0xfb,0x17,0x55,0xc3,0x9d,0x27,0x7f,0x2f,0xcd,0x8a,0xcf,0xfd, -0xb2,0x87,0xfe,0xe9,0x85,0x96,0x05,0xca,0x0f,0xca,0xd1,0x76, -0xcc,0xeb,0xde,0x5d,0xee,0x3d,0x6b,0x98,0x59,0xf0,0x7e,0xd3, -0xbf,0x99,0xa1,0x76,0x93,0x93,0x4a,0xec,0x58,0xfe,0xb8,0x1c, -0xa1,0xa6,0x29,0x6f,0xb2,0xb2,0x2d,0x63,0xb7,0xf3,0x46,0xde, -0x60,0x21,0x00,0xfb,0x0a,0xf1,0xba,0x37,0x0d,0x1e,0xb4,0x66, -0x6b,0x2d,0xc3,0x86,0x31,0x71,0xb3,0x2a,0x8f,0x91,0xd8,0xa9, -0x67,0x42,0x4e,0x32,0x7d,0x7d,0x0c,0xc7,0xee,0x9a,0x55,0xff, -0xcb,0xee,0x4e,0x0c,0xb7,0xc5,0x16,0xad,0x30,0xbf,0xeb,0x5f, -0xe3,0x21,0x36,0x14,0x90,0xb7,0xec,0x52,0x30,0xe9,0xf0,0xae, -0x6c,0x2d,0xf3,0xd9,0xe8,0x95,0x9f,0xdd,0xf2,0x13,0x50,0xd6, -0xe2,0xe3,0x9d,0x23,0x76,0x80,0xcf,0x1b,0x9e,0x0f,0x5c,0x8e, -0xac,0x03,0x0c,0xe8,0x9f,0xac,0x50,0xf4,0x21,0x3c,0x5c,0xdf, -0xa2,0x1e,0xd3,0xe5,0x08,0x9f,0xe9,0x65,0xbe,0x8b,0x6d,0x62, -0xc3,0x35,0xea,0xed,0xaa,0xde,0xdd,0x4a,0x52,0xdb,0x20,0xee, -0xdc,0x47,0x1b,0x43,0xda,0xb7,0x58,0x8b,0x75,0xb2,0x88,0x96, -0xc8,0x65,0x24,0xb9,0xa8,0x24,0x00,0xe7,0x55,0xa0,0x50,0x67, -0xcb,0xbe,0xb9,0x4b,0x0a,0xd4,0x63,0xeb,0x94,0xe1,0x6d,0xd9, -0x77,0xcf,0x8a,0x0e,0xfd,0xdb,0x67,0xa6,0x06,0x5c,0x0b,0x50, -0xaa,0xae,0x91,0x47,0x3c,0xf5,0x66,0x1d,0xf0,0xa4,0xa7,0xdf, -0x67,0xa3,0x63,0xb2,0xca,0x57,0x93,0x3a,0xe6,0xbd,0x61,0x6d, -0x6f,0x7a,0xd1,0xbb,0xc5,0x17,0xe5,0xf6,0x09,0xf4,0x54,0x17, -0xc4,0x27,0xcf,0xb3,0xeb,0x1e,0x29,0xf0,0xa2,0xe1,0x09,0x74, -0x1f,0x58,0xb3,0x6f,0xce,0x6b,0x5e,0x09,0x77,0x5a,0xc3,0xdd, -0x75,0xad,0x57,0x93,0x46,0xf0,0xbc,0xc5,0x97,0xe8,0xd6,0xd3, -0x56,0xb3,0x56,0x3b,0xce,0xf3,0x1a,0xee,0x4c,0xd3,0xcd,0x4d, -0xd8,0xd8,0x4d,0xed,0x86,0xef,0xc3,0xb4,0x39,0x9e,0x5c,0x6d, -0x02,0x16,0x9b,0xa3,0xab,0x26,0xcf,0x4f,0x9a,0x3f,0xb2,0xa5, -0x59,0x2b,0xcf,0xb4,0xc5,0xa2,0x38,0x6a,0x9d,0x98,0x31,0xb4, -0x6f,0x18,0x65,0xea,0x00,0x31,0x85,0x66,0xb2,0x70,0x5a,0xd2, -0x43,0x12,0x99,0x9a,0xd8,0xc9,0xd6,0x68,0x62,0xf1,0x79,0x6e, -0x64,0x8b,0x82,0x38,0xc8,0x8a,0xd0,0x34,0x78,0x3c,0xa2,0xe8, -0x86,0x9e,0x15,0x51,0x39,0xc2,0x81,0x27,0x5a,0xce,0xc2,0xc1, -0xec,0x14,0x6c,0x7e,0xc5,0x61,0x32,0x15,0x49,0x52,0xca,0x44, -0x1d,0x77,0xc0,0x35,0xb1,0xe2,0x4e,0x46,0xf0,0x17,0x0b,0x82, -0xab,0x1e,0xeb,0x50,0x38,0x8a,0xd0,0x46,0x1e,0x52,0xa2,0xd7, -0xd6,0xbe,0x15,0xb7,0x76,0x84,0x07,0x9f,0xc1,0x11,0x34,0x5a, -0x60,0xb1,0x93,0xf0,0xa1,0x31,0xc6,0xf8,0x47,0x10,0xd4,0x75, -0x7c,0x80,0x16,0xb9,0xb7,0x35,0x13,0x59,0xed,0x41,0xee,0x02, -0xba,0xca,0xaa,0x48,0x8f,0xa2,0x26,0xac,0xee,0xfd,0x6c,0xa6, -0xcd,0x24,0x08,0xc1,0x91,0xf5,0x2c,0x6a,0x23,0xe4,0xf3,0x82, -0x71,0x92,0xb4,0xcb,0xef,0x76,0xb8,0x1e,0x5c,0x99,0x65,0xf6, -0x52,0x39,0xf8,0x66,0xe6,0x48,0x7f,0xf1,0xfd,0x8e,0xb5,0xec, -0x50,0x58,0xb9,0x78,0x3e,0xeb,0x77,0x27,0x96,0x81,0x16,0xa6, -0xbc,0x7f,0x87,0x62,0xef,0x73,0xed,0x65,0xda,0x8a,0xb7,0x30, -0xc1,0xce,0xdc,0xa9,0xaf,0xfa,0xa5,0xac,0xfa,0x75,0x4c,0x45, -0x29,0xb5,0x60,0xe9,0x32,0xc1,0x51,0xcd,0x5b,0xd4,0x5e,0xcf, -0x2b,0x4f,0x69,0x4a,0x00,0xef,0x07,0xfd,0x34,0xca,0x47,0x1b, -0x73,0x90,0xfe,0xde,0xfd,0xb4,0xff,0x23,0xec,0x08,0x5f,0x1d, -0x1c,0xc1,0xa6,0xf0,0x38,0xbd,0x7f,0xb9,0xb7,0xfb,0x66,0x7f, -0xe7,0xed,0xfb,0xe3,0xbd,0xc3,0x34,0x6a,0x46,0xe6,0xf7,0x47, -0x07,0x3f,0x1d,0x72,0xc8,0x44,0xe6,0x70,0xe7,0xe4,0x64,0xef, -0xe8,0xdd,0x71,0x7a,0x7a,0x7f,0x33,0x18,0xc1,0x54,0x4a,0x5b, -0x06,0x3e,0x5e,0x4d,0x7a,0x17,0xe9,0x16,0xda,0x43,0xa5,0xaf, -0xe7,0xb8,0x91,0x3e,0x9c,0xe4,0x29,0x88,0x87,0xf0,0x75,0x3c, -0xbf,0xc4,0xaf,0x51,0x7e,0x45,0x71,0x1b,0xf4,0x29,0x91,0x57, -0xc7,0xf8,0x8a,0xf7,0xb9,0x19,0xca,0x07,0x5e,0x41,0x16,0x01, -0x6f,0x3b,0xc0,0xdb,0x0e,0xf0,0xff,0xfe,0xbf,0xab,0x40,0xc7, -0x18,0x6d,0x81,0x27,0x15,0xd0,0xbb,0x66,0xf7,0xa7,0xa3,0xa3, -0xbd,0x77,0xbb,0x7f,0x79,0x7f,0xfc,0x97,0x7d,0x34,0xf1,0xb8, -0x34,0x2f,0x77,0x4e,0xf6,0x4e,0xde,0xec,0xef,0xf9,0x3e,0xd8, -0x3f,0x78,0x77,0xf2,0x3a,0x8d,0xfe,0xd0,0x1b,0xcd,0x7b,0x93, -0x2f,0xe6,0x55,0x7e,0x3e,0xa1,0x8f,0xfd,0xde,0xe4,0xe2,0xda, -0xec,0xdc,0x4e,0x06,0x43,0xf8,0xfe,0x62,0xfe,0x30,0x1f,0xe5, -0xf0,0x67,0xf8,0xc5,0xec,0xcc,0xaf,0x60,0x66,0x9a,0x63,0xa0, -0xcc,0x1c,0xfd,0x87,0x98,0x83,0x8b,0xd9,0x18,0x7f,0xdf,0x8d, -0xef,0x38,0xe2,0x65,0x7e,0x41,0x1f,0x91,0x55,0x7d,0x37,0xc0, -0xc5,0x8f,0x5f,0x1f,0x1c,0x9d,0xf8,0xda,0xb0,0x26,0xac,0x04, -0xab,0xb0,0x15,0x20,0x7c,0x04,0x8f,0xb0,0x11,0x2a,0x42,0x44, -0x60,0x01,0x9c,0x97,0x3b,0x7f,0x49,0xa3,0x63,0x98,0xe3,0x50, -0x66,0x7f,0x4c,0x3f,0x27,0x20,0xc7,0xe1,0xef,0x7f,0xe5,0xfd, -0x11,0x7f,0x9d,0x5c,0xcf,0x27,0xf4,0xf1,0x6a,0x32,0xc0,0x9f, -0x63,0x90,0x87,0x27,0xf0,0x51,0xc6,0xc8,0x82,0x43,0x58,0x08, -0x08,0x81,0x60,0x71,0x2c,0x89,0xc5,0x82,0x12,0x3b,0xfb,0x87, -0xfb,0x40,0x0f,0xd1,0xce,0x3e,0x6e,0x1d,0xf6,0xa3,0xae,0xb9, -0xc9,0xfb,0x83,0xf9,0x4d,0x1a,0xed,0xef,0xef,0xd7,0xfa,0xa6, -0xf6,0xa5,0x76,0x9d,0xde,0xdc,0xa4,0xd3,0x69,0xad,0x07,0x39, -0xa6,0x78,0x3e,0x09,0xdc,0x67,0x7f,0xb3,0xbf,0xf9,0x85,0x93, -0x30,0xfe,0x12,0x38,0xd6,0x4b,0x9c,0xff,0xd1,0x1e,0xfc,0x67, -0x6a,0xfb,0xb6,0x30,0x1e,0x50,0x8c,0xae,0x38,0x49,0x45,0x72, -0x1d,0x2e,0x5a,0x62,0x09,0xb8,0x44,0x12,0x7c,0x9b,0xf1,0x64, -0x80,0x0a,0xb8,0x0a,0x0f,0xca,0xe9,0x63,0x21,0x6a,0x69,0x6e, -0x87,0xf3,0x49,0x6f,0xb8,0xdb,0x53,0x4f,0xad,0x47,0xf3,0x1b, -0x37,0x0b,0x5b,0xf8,0x5c,0x74,0x7e,0xd3,0x89,0x60,0x2b,0x0d, -0x0d,0x18,0x83,0xdc,0x3c,0x01,0x31,0x57,0x89,0xca,0x7c,0x17, -0x71,0xd8,0x9b,0x5d,0x03,0xe7,0x84,0x65,0xdb,0x4b,0x72,0x39, -0x3d,0xc3,0x47,0x66,0x34,0xbb,0xb6,0x9d,0xb7,0x19,0xa1,0xdd, -0x5d,0x9b,0xe4,0x5e,0x4d,0x6f,0x6c,0xb4,0x13,0x1b,0x89,0x1c, -0xd7,0x5d,0x70,0x1c,0x73,0x64,0xac,0x12,0xdd,0x0d,0x95,0x83, -0xc2,0xcc,0x18,0x40,0x2b,0x25,0x53,0x64,0xcb,0xf6,0x86,0x68, -0x7c,0xfe,0x41,0x79,0xc4,0xcf,0xf4,0x79,0x10,0xbb,0x5b,0xc6, -0x8b,0xbf,0x36,0xe4,0xc2,0x9b,0x6e,0x7b,0x80,0xe4,0x1e,0x3e, -0x71,0xfc,0x35,0x3a,0x3e,0x75,0x0f,0x9f,0x24,0x2f,0x74,0x27, -0x9a,0x14,0x11,0x07,0xc9,0xbf,0xee,0x82,0xcc,0xf2,0x72,0xef, -0xd5,0xce,0x4f,0x6f,0x4f,0xde,0x1f,0x02,0x49,0x1d,0x9f,0x5a, -0x20,0xf2,0x02,0xa1,0x80,0xdf,0xce,0xed,0xed,0x0a,0x14,0x81, -0xe5,0xbc,0x5e,0x85,0x20,0xf4,0x66,0x56,0xe1,0xc3,0xdc,0x7b, -0x80,0xe6,0x7c,0xd3,0x1c,0x27,0x2e,0xbf,0x98,0xf8,0x63,0xfe, -0x85,0xb4,0xd8,0xbc,0x4f,0x6c,0xdb,0xaa,0xde,0xf4,0x01,0x60, -0xbf,0xee,0xd2,0x8e,0xcc,0xf8,0x8a,0xeb,0x75,0xe8,0xe6,0xb5, -0xcc,0x47,0x04,0xe6,0x75,0x15,0x82,0x90,0xad,0xe1,0x83,0x6a, -0x60,0xce,0xf3,0xab,0xc1,0x88,0x0e,0xe9,0x63,0xfa,0x34,0x9f, -0xae,0xc7,0xc3,0xdc,0x10,0x61,0x7d,0x1a,0xa8,0x9b,0x84,0xad, -0x2c,0xa3,0x24,0x27,0x61,0x53,0xf6,0xa4,0xc3,0x91,0xf2,0xce, -0x85,0xe2,0xac,0xc0,0x98,0x3a,0x20,0x4a,0x5b,0x03,0xd6,0xd0, -0xdb,0xd7,0xd0,0x4a,0xea,0x42,0x79,0x3f,0x87,0xd2,0x67,0x70, -0x43,0xf6,0x6f,0x91,0x25,0x2a,0xbc,0x3c,0xa3,0xe8,0x8e,0x28, -0x82,0xa8,0xd7,0x36,0x24,0xbd,0x16,0x40,0x93,0x1e,0x3f,0x81, -0xf6,0x77,0x8e,0xbe,0x48,0x58,0x7b,0xe8,0x7f,0x1f,0xc8,0xb5, -0xd1,0x0a,0xf4,0x4a,0x50,0xf3,0xec,0x47,0x3c,0x45,0xab,0x86, -0x87,0xb2,0xc0,0x96,0x09,0xf0,0xde,0x8c,0x0a,0x61,0x04,0xba, -0xad,0x5f,0x53,0xbe,0x95,0x83,0xce,0xd7,0xb3,0x9b,0xe1,0xaf, -0x91,0xd8,0x7a,0xb7,0xb7,0x58,0x87,0x39,0x87,0x3f,0xb0,0x7c, -0xa0,0x21,0x6c,0x6b,0x9c,0x02,0xad,0x53,0xfe,0x1a,0x2d,0xdb, -0xf8,0xb4,0xcc,0x7f,0x3a,0x0b,0x02,0x02,0xe1,0xdd,0x18,0xdb, -0x9e,0xf9,0x5e,0x90,0xf8,0x44,0xf4,0xef,0x79,0x63,0x12,0x68, -0xe5,0x38,0x2b,0x60,0xf2,0x5e,0x2a,0x98,0xa4,0xf2,0xe2,0x48, -0x2e,0x6c,0x67,0x18,0x9f,0x29,0x62,0x09,0x6a,0x35,0x34,0x1f, -0xf0,0xc2,0xd7,0x5d,0xb1,0x49,0x11,0x67,0xf3,0xc8,0x9e,0xf0, -0x3a,0xe9,0xeb,0x76,0x76,0x7d,0x3b,0xb9,0xfc,0x1c,0x99,0x67, -0xd6,0x51,0x1f,0x00,0xa9,0x45,0x64,0xe5,0xa9,0x76,0x33,0x00, -0x59,0x19,0x04,0x7b,0x84,0x52,0xbb,0xa5,0xf6,0x42,0x12,0xc8, -0x63,0xcd,0x67,0x74,0x64,0x16,0xd4,0x9e,0xb4,0xf5,0xec,0x95, -0x8a,0x2d,0xfe,0x46,0xf4,0x8b,0xe8,0xb5,0xb4,0x8d,0x73,0x3d, -0x82,0xc6,0x59,0x63,0x15,0xca,0x88,0x21,0x4a,0x04,0x9e,0xe8, -0x8c,0xa7,0x78,0x41,0x51,0x88,0xc9,0x0a,0x3e,0x1e,0x65,0x6a, -0xcf,0xc6,0x6e,0x5e,0x4b,0x7e,0x4e,0x48,0x0c,0x4d,0x6b,0x3b, -0xaa,0xf0,0xdd,0x01,0x0a,0x6f,0x94,0x78,0xaa,0xca,0x90,0xa4, -0xce,0xe2,0x03,0xaa,0xe9,0x66,0x8a,0xa9,0x2b,0x6c,0x93,0x46, -0xcc,0x35,0xe0,0x75,0x72,0x83,0x3f,0xc9,0x02,0x35,0x82,0xb0, -0x38,0xf7,0xce,0xa7,0x38,0x74,0x41,0x8f,0x35,0x3c,0x68,0xef, -0x23,0xd9,0xb5,0x72,0x92,0xd3,0xe3,0xfb,0x0a,0x52,0xe9,0x51, -0x17,0x93,0xa9,0x3c,0xee,0x6d,0x99,0xa5,0x31,0x27,0x54,0xd0, -0x07,0x51,0x46,0xb2,0xa6,0x65,0xff,0xd8,0x17,0xcf,0x04,0x60, -0x45,0x79,0x4f,0xe8,0x86,0x13,0x0b,0x50,0xc2,0xf6,0xc4,0xaa, -0x24,0xce,0x42,0x29,0xb2,0x58,0xc8,0x47,0x2a,0xd9,0x1b,0xbe, -0x6e,0xd8,0x28,0xaf,0xc4,0x5a,0x51,0xf5,0x43,0xb5,0x72,0xf9, -0x34,0x9c,0x80,0xc8,0xc9,0x1a,0x80,0x43,0x98,0xd7,0xca,0xfd, -0x15,0x9c,0x00,0xc6,0xea,0x1c,0xb6,0x93,0x9a,0x19,0xe0,0xf8, -0x59,0x66,0xf0,0xa4,0x29,0xee,0x66,0xae,0x85,0x40,0x87,0x4c, -0xe6,0x91,0x99,0x8f,0xc7,0x85,0xe3,0xf9,0x0c,0x0b,0x3c,0x30, -0x7a,0x8b,0xc5,0x83,0x1d,0x64,0x04,0x0a,0xb6,0x03,0xa1,0x00, -0x69,0xc3,0x4a,0x11,0x40,0xf6,0x8b,0x52,0x47,0x81,0xf2,0x8d, -0x34,0x61,0x76,0x67,0xa7,0x81,0x18,0x60,0x27,0x4c,0xc4,0x89, -0x11,0x30,0x9a,0xb0,0xfa,0xd5,0xfc,0xe6,0x7a,0xfa,0x04,0x7e, -0x83,0x38,0x55,0xf1,0x1b,0x35,0x20,0x01,0xb3,0x09,0x2b,0x0f, -0x7b,0xfd,0xff,0x8f,0xec,0xa3,0x11,0x7b,0xa8,0x1d,0xdf,0x29, -0x8a,0x9f,0x20,0x80,0x47,0x38,0x89,0xdb,0x86,0xda,0xc5,0xd8, -0x92,0x71,0x96,0x85,0x0b,0x34,0xad,0xf9,0x8f,0x4e,0x9c,0x37, -0xe5,0xc5,0x54,0xcf,0x9f,0xe2,0x62,0x5a,0x31,0xef,0xe4,0xe1, -0x31,0x2d,0x11,0xfe,0xe1,0xf1,0x57,0x2d,0xae,0x8f,0xf1,0x4c, -0x2b,0x23,0x4b,0xb9,0xca,0x96,0x3e,0x8d,0x1f,0x59,0xbe,0xd2, -0x50,0xdd,0x5f,0xcd,0x88,0x9e,0xca,0x89,0xec,0x54,0x11,0x6d, -0xcc,0x5b,0xb1,0x13,0x50,0xf5,0x58,0xdb,0xda,0xb7,0x40,0x33, -0x2e,0x36,0x5f,0x77,0x25,0xa8,0xe3,0x10,0x20,0xae,0x1b,0xf0, -0x8d,0xfa,0xa4,0x65,0xd8,0xe1,0xd1,0xb9,0x56,0x76,0xe3,0x94, -0x4e,0x58,0x67,0x1a,0x87,0xe1,0xcc,0xc3,0xb6,0x26,0x70,0x8a, -0xd3,0x90,0x67,0xe7,0xb2,0x70,0x5c,0x21,0xb7,0xce,0xfa,0x46, -0xc1,0x77,0x2c,0x5e,0x2d,0x11,0xe1,0xec,0xa3,0x51,0xe7,0xb5, -0x16,0x8f,0xb6,0x4a,0x77,0xe8,0xdf,0x0a,0xb5,0x39,0xfc,0xad, -0xd2,0xa2,0x24,0x74,0x14,0xbb,0xcb,0x38,0x4e,0x2e,0x02,0x7c, -0xbc,0x4c,0x1b,0x5f,0xa1,0xd7,0x0e,0x20,0x3b,0x81,0x25,0xd8, -0x14,0x0d,0x90,0x5d,0x81,0x1b,0x32,0x2a,0xce,0x60,0x6d,0x64, -0xe1,0x7e,0x21,0x78,0x55,0xa9,0xf4,0x66,0xa6,0x23,0xb2,0x07, -0x65,0x1f,0x5e,0x8a,0xb6,0x87,0xbe,0x6d,0x50,0x0f,0x27,0x9d, -0x1a,0x8d,0x94,0x0a,0xec,0x41,0xab,0x9b,0x07,0x32,0x4e,0x6c, -0x7b,0x79,0x97,0x6e,0x21,0xe3,0x31,0x69,0x0a,0x8a,0x59,0x40, -0xd1,0x5d,0x06,0x78,0xbd,0xfe,0x05,0x48,0xe3,0x71,0xe4,0x18, -0x37,0x67,0x17,0xfd,0xe3,0xc8,0xb8,0xf8,0x26,0x73,0x24,0x18, -0x4f,0x81,0xc4,0x06,0x7b,0x9c,0x3f,0x1c,0x5b,0xdd,0x3e,0x59, -0x79,0x55,0xd2,0x35,0x5a,0x36,0x72,0x37,0x56,0x4d,0x1b,0x13, -0xe3,0xf1,0xf7,0x00,0xaf,0xcb,0x70,0xea,0x85,0x37,0xea,0x6d, -0xd7,0x89,0x1d,0x3b,0x13,0x33,0xb5,0x2d,0xf0,0xe5,0x80,0x93, -0x5a,0x78,0x20,0x96,0xa3,0x08,0xa9,0xb1,0xc8,0x6c,0x37,0x35, -0x61,0x20,0x50,0x4b,0xac,0x53,0xdc,0x05,0xa4,0xab,0x79,0x1a, -0x8b,0x29,0x5c,0xb5,0x63,0x19,0xaa,0xe6,0xb0,0xa6,0x0a,0xe6, -0x96,0xf8,0xae,0xa3,0x5b,0x68,0x9d,0xdf,0x31,0x4c,0x5c,0x50, -0x14,0xd3,0x54,0xbd,0x2d,0xd2,0x43,0xac,0x63,0x84,0xd9,0x69, -0x34,0x92,0x80,0x04,0x9a,0x40,0x2f,0x11,0xd9,0x13,0x55,0x04, -0x44,0xc6,0x4b,0x59,0x2d,0x94,0x3e,0x9b,0x17,0xb3,0xc9,0x90, -0x8c,0x80,0x49,0xf8,0x26,0x9f,0xf5,0x28,0xbc,0xbd,0x96,0x71, -0xcc,0xa7,0xeb,0xc1,0x85,0x3a,0xc1,0xc8,0x87,0x37,0xd6,0x55, -0x03,0xa7,0xcf,0x80,0x5b,0xe7,0xb3,0xa4,0x1d,0xf5,0xd0,0xdc, -0x95,0x57,0x7b,0x80,0x8c,0xd6,0x32,0x30,0x3f,0x4d,0x44,0x15, -0x63,0x8e,0x04,0xb6,0x18,0x18,0x81,0xdc,0x82,0x3d,0xf5,0x1a, -0xa6,0x65,0xf0,0xcf,0x59,0x41,0x4e,0xd0,0x12,0xbe,0xf8,0xc6, -0x27,0x26,0x7e,0x3e,0x25,0xf2,0xa1,0x3c,0x68,0xa2,0x8c,0x4d, -0x69,0xe1,0xed,0x2a,0x76,0x05,0xec,0x30,0x88,0x7a,0x2a,0xfa, -0x48,0x4a,0x26,0x6d,0xf9,0xc0,0xd6,0x02,0x10,0xb6,0xef,0xc6, -0xf8,0x47,0x09,0xfa,0x5e,0xf4,0x70,0x5c,0x87,0xd0,0x4c,0x47, -0xb5,0xb8,0xc3,0x09,0x45,0xc0,0x94,0xc7,0x6d,0x3d,0x27,0xde, -0x72,0x9c,0xe4,0x88,0x43,0x54,0xd6,0x0a,0x94,0x8c,0x76,0xff, -0x4b,0x43,0xaa,0x0b,0x84,0xef,0x0b,0xe4,0x2d,0x81,0x09,0xb5, -0xdb,0x4e,0xa3,0xcb,0xcb,0x8d,0xdf,0xfc,0xf6,0xfb,0xed,0xad, -0xdf,0x6e,0x84,0xb5,0x47,0x5d,0xbc,0x79,0x42,0x6d,0xc2,0x8a, -0x79,0xba,0x96,0x79,0x3a,0xf1,0x8a,0x3c,0x84,0x57,0xc5,0xa4, -0xc6,0x2b,0x6b,0x97,0x67,0x8c,0xa8,0x09,0xdf,0xf0,0xc7,0x61, -0xf9,0x27,0x62,0x20,0xa8,0x31,0x53,0x51,0x99,0x4e,0x7e,0x1a, -0x8b,0xc1,0xe3,0x70,0xf2,0x9b,0x8b,0x22,0x5c,0x19,0x64,0x62, -0xb5,0x14,0xdd,0x28,0x58,0x7d,0xe6,0x47,0xda,0x92,0x18,0x5e, -0x6a,0xdb,0x8f,0x9a,0xc8,0x42,0x12,0x63,0x6e,0x96,0x95,0xc1, -0xb4,0xcb,0x23,0x27,0x6d,0x34,0x0f,0xf0,0xd7,0xe5,0xe3,0x2f, -0x46,0xfa,0x83,0x2b,0xd2,0x0d,0x83,0x61,0x63,0x53,0x84,0x04, -0x60,0x97,0x75,0x81,0xb2,0x2d,0x77,0xb3,0xa0,0x8a,0xb0,0xa1, -0xdc,0x15,0x98,0x87,0xda,0x48,0x62,0x4e,0x50,0xdc,0x3f,0x17, -0xe6,0x06,0xc5,0xb9,0x2b,0x1c,0x5d,0x6f,0xbd,0x2e,0xe0,0x2a, -0x7a,0x02,0xf6,0xea,0x41,0xde,0x46,0xe3,0x71,0xeb,0x63,0x5f, -0x45,0x03,0xab,0x17,0x99,0x12,0x0d,0x74,0xca,0xa3,0x22,0x59, -0xd3,0xf8,0x31,0x1a,0x0f,0xbb,0xe6,0xc1,0x61,0x24,0xd5,0x92, -0x8a,0xbe,0xa3,0x7b,0x74,0xdd,0x17,0x7a,0xea,0x15,0xee,0x64, -0xae,0x0a,0xf2,0x4d,0x3f,0x3f,0x9f,0x5f,0xa1,0x28,0x3c,0xcd, -0x87,0x97,0xb4,0xaf,0x68,0x8b,0xb9,0x0f,0x88,0xdf,0x1b,0xf5, -0xce,0x87,0x79,0x5f,0x3d,0x98,0x1b,0xf6,0xae,0x2a,0x64,0x0f, -0x8a,0xee,0xc4,0x0c,0x0b,0x03,0x22,0x77,0x50,0x44,0x41,0xe6, -0x28,0x1b,0xdb,0xb2,0x56,0x87,0x1c,0x9a,0xf8,0xfa,0xb3,0xc7, -0xef,0x23,0xf1,0x75,0x99,0xab,0x10,0xbe,0x9d,0xa5,0xa3,0xf1, -0x65,0x8d,0x32,0xa0,0x09,0xe9,0xc9,0x15,0x5e,0x72,0x5d,0x7c, -0xec,0xc0,0x57,0x86,0xa1,0x1b,0x90,0x12,0x7a,0x57,0x39,0xea, -0x04,0x00,0x8b,0x77,0xe9,0xee,0x2c,0x4e,0xe5,0x49,0x3a,0x7c, -0xd7,0x8c,0x4a,0x3c,0x2a,0x9a,0x54,0x79,0x1a,0xae,0x64,0x5a, -0xf8,0x62,0x47,0x47,0x47,0x6f,0xb9,0xf6,0xac,0xba,0xa0,0xcd, -0xd4,0x88,0x52,0x8e,0x21,0x6b,0x42,0x09,0xee,0x2a,0xb4,0x3e, -0xe5,0x68,0x3a,0x1e,0xe6,0x30,0x30,0xf1,0x8c,0xb4,0x36,0xe5, -0xf1,0x39,0x46,0x66,0x56,0x55,0x58,0xc2,0xf4,0x20,0x6e,0x38, -0xbe,0x22,0xe7,0x2b,0x14,0x73,0x8a,0x65,0x60,0xd1,0x92,0x60, -0x13,0x12,0x17,0x0b,0x54,0x27,0x74,0x36,0xfa,0x30,0x37,0x6f, -0x69,0x3a,0x85,0x99,0x0a,0x68,0x54,0x3d,0x12,0xa8,0x78,0xce, -0x47,0x63,0x80,0xd9,0x45,0xb1,0xa1,0x30,0x3c,0xc8,0x5e,0x54, -0x3d,0xb1,0x20,0x83,0xed,0x04,0x11,0x3b,0xd5,0x60,0x5a,0x18, -0xb9,0x8d,0xd6,0x04,0xd0,0x08,0x98,0x8f,0x58,0x8a,0x0e,0x2c, -0x26,0xa4,0xaa,0x4b,0x22,0x34,0x1b,0x85,0x42,0xd9,0xe5,0x38, -0x88,0xc6,0x08,0x88,0xff,0xd4,0x9b,0x8c,0x82,0x78,0x8c,0x80, -0x78,0xd2,0xc9,0x0d,0x12,0x28,0x06,0xd5,0x94,0x91,0x22,0xd3, -0x42,0x4f,0x5c,0xba,0xfe,0xa4,0xbc,0x94,0xc7,0x1d,0x3a,0xeb, -0x1d,0x0f,0xa5,0xd4,0xeb,0x97,0xb6,0xa5,0x38,0x67,0x02,0xeb, -0x54,0x71,0x68,0xa1,0x2a,0x1f,0x4d,0xe7,0x93,0xfc,0xb8,0x77, -0x99,0xef,0xd3,0x55,0x1b,0x19,0x42,0xa5,0x97,0x8c,0x78,0xbb, -0xe4,0xd5,0x00,0xd8,0xc0,0xbe,0xb2,0x0b,0x66,0x6d,0xec,0xdb, -0x03,0x0c,0xe2,0x29,0xee,0xf4,0x62,0x9a,0x5f,0x5c,0x0e,0xfb, -0x91,0x79,0x76,0x94,0xb3,0xd5,0x56,0x3c,0xad,0x08,0x8a,0xd7, -0x2e,0x07,0xf9,0xb0,0x8f,0x26,0x17,0x76,0x78,0xb9,0x0e,0x54, -0x1c,0x06,0x53,0x65,0x27,0x7e,0xad,0xe6,0xf1,0xa0,0xbb,0xee, -0x67,0x45,0xe4,0x6c,0x47,0x20,0x42,0x55,0x6d,0x93,0xc7,0x32, -0xe3,0xf3,0x0f,0x55,0xcd,0x82,0x68,0x60,0xe3,0xe7,0x1f,0x9a, -0xda,0x70,0x5d,0x86,0xb7,0x14,0xab,0x1b,0x37,0x82,0x9d,0x87, -0x6e,0xdb,0x2b,0x67,0x85,0xe9,0x5b,0x1a,0x14,0xad,0x6a,0x10, -0xe0,0xa0,0x76,0xab,0xf9,0x67,0xd6,0x87,0xba,0x98,0xde,0x2a, -0x5e,0x34,0x98,0xc6,0x64,0x17,0xc9,0x32,0x21,0xd4,0x32,0xa2, -0x08,0xc7,0x4c,0x2e,0xf4,0x0d,0xca,0xa7,0xde,0xa3,0xf9,0xf1, -0xa2,0x61,0x57,0x97,0xb9,0xcd,0xf3,0x8f,0xf1,0x40,0x54,0x14, -0xe6,0x37,0xd9,0x60,0xb1,0x68,0xb9,0x87,0x5e,0xa4,0xea,0x03, -0xb1,0x3f,0x28,0x45,0xaa,0x0e,0x7d,0xcb,0x39,0x9a,0xcb,0x91, -0xa4,0x6b,0x2d,0x6d,0xaf,0xea,0xdd,0x1c,0x49,0x0e,0xd1,0x73, -0xcf,0x19,0xae,0x5f,0x64,0xd1,0x56,0x54,0xaf,0x47,0xbf,0x8b, -0x5e,0x64,0xda,0x21,0xe4,0x60,0x4a,0xa6,0x59,0xa7,0xb7,0xa8, -0x7f,0xeb,0x4b,0x44,0x35,0x7c,0x06,0x78,0x0d,0xdb,0x98,0xb3, -0x89,0xfd,0xfa,0x1f,0x2e,0x6a,0x64,0xbf,0xfe,0x97,0xfd,0xf8, -0xdf,0xff,0x17,0x7d,0x69,0xb8,0x6f,0x50,0xff,0xb0,0x84,0x44, -0x0f,0x91,0xf8,0x3b,0x21,0x01,0xcc,0x0b,0x63,0x76,0x30,0xe6, -0xaf,0x12,0x13,0xbd,0xb7,0x10,0xd7,0x4b,0x00,0x61,0x1c,0x0f, -0x6e,0xd1,0xde,0xc7,0x58,0xb7,0x2d,0xda,0xb0,0x25,0x1a,0xf2, -0xa1,0x3b,0x40,0x19,0x14,0x43,0x92,0x63,0xd6,0x45,0x2c,0x81, -0xb5,0xb7,0x50,0xb1,0x22,0xb9,0x87,0x3f,0xa8,0x51,0x81,0x6f, -0xef,0x51,0x05,0x8d,0x19,0xf1,0xf0,0x78,0x36,0xc9,0xfc,0x4a, -0x47,0xd9,0x61,0xcd,0x98,0xc2,0x72,0x41,0xdf,0x0d,0xa8,0xb9, -0xc1,0x63,0x10,0xd5,0x4e,0xd1,0x1b,0x7c,0x59,0xcf,0x8b,0xc0, -0x37,0xa2,0x2e,0xb9,0xaf,0x44,0xbd,0xad,0x76,0x15,0xe5,0x03, -0x01,0xb2,0x0a,0xca,0x5b,0xfc,0xa8,0xc9,0xaa,0x04,0xf4,0x5b, -0x23,0xb5,0xa5,0xe1,0xfc,0x06,0x95,0x50,0x70,0x0a,0x78,0xd2, -0xaf,0x9d,0xde,0x6f,0x2f,0xbb,0xcd,0x88,0x39,0x9e,0x61,0x74, -0x59,0xe9,0x48,0xbf,0x97,0xed,0xf5,0xa5,0x2f,0xfc,0xf6,0x69, -0x44,0x11,0x78,0xa6,0x41,0x28,0xf2,0xbd,0x5a,0x9b,0xfe,0x6a, -0x52,0x13,0x0b,0x01,0x17,0xd7,0xda,0xd8,0x5d,0x91,0xfa,0x12, -0xba,0xee,0x89,0x9a,0x15,0x3d,0x9f,0x70,0x3d,0x0d,0x48,0x68, -0x3b,0x1d,0x68,0x24,0xf8,0xdd,0xeb,0x8c,0xe8,0x9e,0x8b,0xe6, -0x54,0xb4,0x5e,0x0f,0x87,0x97,0xf3,0x95,0x60,0xd4,0xc8,0x7c, -0x66,0x81,0x0e,0xea,0x75,0xce,0x8d,0x30,0xa4,0x7a,0x29,0x0e, -0x74,0x85,0xe0,0x19,0x88,0xc5,0x5b,0x42,0x62,0xe5,0xbf,0x55, -0xc6,0x93,0x8f,0x95,0x0b,0x95,0x2c,0x16,0xab,0x2a,0x01,0xc2, -0xcb,0xd1,0x0d,0xcc,0xc3,0x95,0x90,0xfb,0x80,0xb6,0xa2,0xc1, -0xc8,0x1e,0x3f,0xa3,0x79,0x49,0xbc,0xdc,0x8d,0x60,0x01,0x61, -0x6a,0x6a,0x2c,0x65,0x88,0x5a,0xdf,0xf1,0x87,0xd8,0x3e,0x17, -0xbf,0x26,0x94,0x89,0x15,0x51,0x69,0xbc,0x53,0xc9,0x84,0x1a, -0xf5,0x29,0x88,0x73,0x97,0xa3,0xb4,0x7c,0x54,0xc7,0x79,0x96, -0xcb,0x02,0x71,0xf0,0x1c,0xf5,0xb4,0x81,0x1c,0xea,0xe5,0x78, -0x66,0xb0,0x6d,0xac,0xe4,0xc9,0x0a,0xea,0x64,0xff,0xe8,0xe2, -0x9a,0x6d,0x54,0x3e,0x4a,0x3a,0xf8,0x62,0x58,0xf4,0x3f,0x03, -0x62,0x31,0x8a,0x50,0x1c,0x77,0x28,0x50,0x4d,0x3b,0x91,0x4c, -0xe8,0x47,0x8d,0xb1,0xc9,0xa4,0x34,0x55,0x8e,0xe3,0x64,0xa4, -0x9f,0x90,0x80,0x24,0x0f,0x39,0xac,0x70,0x58,0x0b,0x62,0x2e, -0x5c,0xa0,0x6b,0x7c,0x34,0x1c,0xe2,0xe7,0x72,0x02,0x8e,0x31, -0x55,0x9f,0xdc,0xfb,0x96,0x67,0x54,0xb3,0xbd,0x22,0x93,0x1a, -0x37,0x98,0x09,0xb4,0x04,0xaf,0x30,0xcf,0x96,0x09,0x72,0x59, -0xdd,0x4a,0x57,0x8d,0xb8,0x93,0x60,0x52,0x0b,0x19,0xb0,0x90, -0x8a,0xcb,0x0a,0xe4,0xe0,0x8d,0xd6,0x97,0x07,0x9f,0xaa,0x5d, -0x62,0x4f,0x1c,0x1c,0xee,0x1d,0xed,0x9c,0x1c,0x1c,0x1d,0x17, -0x0d,0xbd,0x52,0x96,0x24,0x21,0x08,0xcd,0x4b,0xb1,0x7c,0xdf, -0x44,0x72,0xc9,0x5c,0x19,0xb6,0x31,0xda,0xf5,0x33,0x95,0x2d, -0xba,0x66,0xfc,0x03,0x12,0x1b,0xa5,0xe3,0x0a,0x29,0x8a,0x8d, -0x0e,0x9a,0x5c,0xac,0x3a,0x92,0x23,0xc9,0xc8,0x1a,0xc6,0x15, -0xea,0x13,0xf3,0xb0,0x3a,0x69,0x69,0xee,0x7b,0xc0,0xc8,0xae, -0x14,0xb1,0x52,0x72,0x78,0x00,0x3c,0x55,0x05,0x19,0x03,0x51, -0x9e,0x26,0x3e,0x07,0xd4,0xac,0x67,0x06,0x5b,0xdf,0x57,0x14, -0x8b,0x96,0xc6,0xcb,0x33,0xc7,0xd2,0x38,0x75,0x1f,0xaa,0x95, -0xf1,0xc4,0x69,0x2d,0x93,0xaa,0x79,0x26,0xb9,0x1b,0x2d,0xce, -0xaf,0xa6,0x83,0x2b,0x56,0x98,0x53,0x72,0x21,0xf5,0xcb,0x7c, -0x6c,0x1f,0x53,0x94,0x26,0x0b,0x90,0xae,0x57,0xac,0xc6,0xdc, -0x38,0xa5,0x26,0xbd,0x4f,0x5c,0x34,0xa3,0xa2,0x26,0x47,0xb5, -0x56,0x3a,0x6e,0x7e,0x80,0x37,0x97,0x27,0x19,0x92,0x82,0x03, -0x45,0xb3,0x85,0x01,0xb1,0xa4,0x39,0x17,0xe2,0x66,0xd3,0x6d, -0x9f,0x8b,0xca,0xc9,0x8c,0x5d,0x4b,0x26,0xd8,0xaf,0x93,0x36, -0xe4,0x91,0xe7,0x5a,0x9b,0xa7,0x67,0xfd,0xde,0xc6,0x65,0xf7, -0xfe,0xfb,0xe5,0xe6,0x00,0x66,0x6d,0x15,0x2f,0x9b,0x8f,0x06, -0x78,0xc3,0x54,0xe3,0x1a,0x6b,0xa7,0x67,0x67,0xf3,0xa8,0x71, -0x8d,0x0b,0x64,0x37,0x92,0x39,0xd0,0xc8,0xbe,0x37,0x53,0xc1, -0x8d,0x71,0x6c,0xa2,0x05,0x47,0xd8,0xee,0x4e,0x76,0xf1,0x28, -0x92,0x16,0xc6,0x37,0x23,0xb4,0x51,0xfa,0xd9,0xb4,0x7e,0xa3, -0x9d,0x0f,0xc0,0x9e,0x37,0xdb,0x3b,0xde,0xdd,0x39,0xdc,0x3b, -0xbd,0xb8,0xee,0xb6,0x2d,0x14,0x88,0xee,0xc0,0xbf,0x14,0x84, -0x05,0xd7,0x65,0x4b,0xbb,0x5a,0x44,0x67,0x67,0xdc,0x62,0x9b, -0xb4,0xe5,0x98,0x3c,0xf4,0x1e,0xf7,0x74,0x12,0x48,0x5c,0x8d, -0x47,0xb8,0xad,0xeb,0x5b,0x69,0x46,0xca,0x3f,0x0f,0xb3,0x5f, -0xce,0x83,0x07,0x63,0x72,0x16,0x34,0x75,0xc3,0xe3,0x98,0xbe, -0xee,0xd0,0x9f,0x46,0xe2,0xb0,0x1e,0x15,0x91,0x09,0x49,0x61, -0xb9,0x40,0x6b,0xda,0x8b,0x05,0xf2,0x64,0x46,0x17,0x15,0xef, -0xad,0x26,0x3e,0x4d,0x6d,0x54,0xc4,0x27,0x79,0x33,0x03,0xe1, -0xa3,0x92,0x84,0xca,0x6c,0xd0,0x32,0xda,0x29,0xf4,0x5b,0xf4, -0x2c,0x4a,0x12,0x45,0xcf,0xd0,0x87,0x6a,0x0d,0xf6,0x1c,0x1b, -0xf9,0x37,0x4a,0x00,0x49,0x61,0x69,0x8c,0x13,0x2e,0x6e,0xe5, -0x0f,0x55,0xd8,0xf1,0xfe,0x44,0xad,0x41,0x06,0x05,0xe8,0xe8, -0xde,0x20,0xa0,0x08,0xad,0xaf,0x63,0x2b,0x4e,0xb7,0xba,0x76, -0x12,0x8b,0x0b,0x8e,0x53,0x19,0x1d,0xbb,0xb4,0x92,0x0a,0xb3, -0xe2,0x68,0xb8,0xef,0xe7,0x60,0xf0,0x0e,0x00,0x11,0xd4,0x28, -0xc4,0x11,0x0c,0xce,0xf2,0xb4,0xdb,0x34,0xed,0xb4,0x13,0x09, -0x73,0x0c,0xc7,0x9c,0x75,0xfc,0x69,0xcc,0xa1,0x97,0x69,0x78, -0x09,0xc5,0xf4,0xd4,0x70,0x63,0x01,0xdb,0xd3,0x48,0xda,0xbf, -0xec,0xa6,0x06,0x1f,0x00,0x19,0x1b,0xcd,0xaf,0x2a,0xdd,0x8b, -0x79,0xbc,0xbf,0xe5,0x7c,0x2e,0xc9,0x9a,0x5b,0xb2,0x1c,0xc1, -0xd2,0x65,0x69,0x41,0xb8,0xb7,0x39,0xac,0x11,0x3c,0x36,0x4c, -0x72,0xbd,0x0d,0xd4,0xd3,0xe0,0xae,0x06,0x3a,0x78,0x0e,0xa1, -0x6d,0x0e,0x6e,0xe3,0xe3,0x01,0xc5,0xdb,0x61,0xb6,0x40,0xc4, -0x76,0x10,0xb3,0x8d,0x51,0xcf,0x83,0xa8,0xe7,0xdd,0x36,0x44, -0x75,0xaa,0x78,0x66,0xd0,0x17,0xcf,0x89,0xc6,0x47,0xcf,0x97, -0x6e,0x4a,0x3f,0x4f,0x20,0xbc,0xfd,0x78,0xc9,0x6d,0x2e,0xb9, -0xed,0x4b,0x6e,0x63,0xc9,0xc7,0x0b,0x72,0x39,0x35,0x08,0xa7, -0x26,0xb5,0x83,0xd0,0xd8,0xe0,0x8e,0x67,0x80,0x2d,0x54,0x1b, -0xd0,0x13,0x09,0xe4,0x2b,0xd8,0xa3,0xc2,0x34,0x82,0xaf,0x19, -0x1e,0x32,0x4e,0x7a,0x10,0x9c,0xd4,0x22,0x2e,0x21,0xe5,0x5a, -0xc9,0x52,0x66,0x0b,0xcc,0x48,0x7b,0xbd,0x49,0x48,0x15,0xac, -0x4e,0x4f,0x78,0xab,0x48,0xd6,0x5e,0xd7,0x2f,0x07,0xc3,0x19, -0x3e,0xba,0x0e,0xf6,0x8d,0xaa,0xfa,0x9b,0xe9,0x15,0x4f,0x4f, -0xe7,0xd3,0x34,0x90,0xfb,0xa7,0x5f,0x46,0xb3,0xde,0x67,0x90, -0xfb,0x8f,0xe9,0xc3,0x0a,0xfe,0x27,0x58,0x42,0x8c,0x8c,0xa3, -0xdc,0xef,0x36,0x01,0xec,0xa9,0xf2,0x92,0x5c,0xba,0x05,0x3b, -0x81,0xe7,0xcb,0x6e,0xcd,0xba,0xc4,0xc3,0xec,0xa7,0xc0,0xa4, -0x71,0x6f,0xe0,0x27,0x82,0x71,0xa8,0x34,0x2d,0x8f,0xa7,0xe8, -0x02,0xff,0x57,0x39,0x92,0xc2,0x3e,0x95,0xb0,0xe2,0x37,0x83, -0x5b,0x99,0x75,0x88,0x63,0x75,0xfc,0xaa,0x5a,0x37,0xcf,0xc7, -0x78,0xbb,0xa8,0x46,0x00,0xe4,0x04,0x44,0x3f,0x2f,0x6e,0xd2, -0xf5,0x53,0x09,0xe9,0x75,0x98,0xf8,0x85,0x6d,0x72,0xde,0x32, -0xf9,0xb6,0xc9,0x9f,0x9b,0xfc,0x7b,0x42,0x22,0x40,0xe0,0xc5, -0x96,0xd8,0x33,0xd3,0x9c,0x62,0xab,0x6b,0x66,0x8a,0x17,0xe0, -0xc2,0x38,0xcb,0xb2,0xbc,0x05,0x6b,0x17,0xfc,0x6c,0xf3,0xcf, -0x73,0xfe,0xf9,0x7e,0xb1,0x58,0xcb,0x5b,0x78,0xe5,0xb2,0x8d, -0x7f,0x9e,0xe3,0x9f,0xef,0x13,0x8d,0x92,0x50,0x85,0xde,0x69, -0x73,0xc3,0x42,0xcc,0x3c,0x16,0x25,0xac,0x83,0x16,0x76,0x62, -0x24,0x21,0xa8,0xc6,0xf3,0xae,0x7a,0x5d,0x53,0xae,0xb8,0x40, -0xe4,0x85,0x15,0x93,0x65,0x38,0x9d,0x94,0x62,0xf9,0x07,0xc7, -0x06,0x47,0x00,0x78,0xe0,0x32,0xbf,0xc9,0xa1,0x76,0xd8,0xdb, -0x5a,0x24,0x0b,0x6b,0x36,0xc0,0x9f,0xbb,0xa1,0x31,0xd2,0x16, -0xa4,0x1f,0xd8,0xca,0xe6,0x2d,0x5c,0xb1,0x8d,0xb0,0x71,0x0f, -0x77,0x3e,0xea,0x4d,0xbe,0xa0,0xf7,0xc4,0x91,0x99,0x0c,0xae, -0xae,0xbd,0xb9,0x84,0xa7,0x08,0x81,0x97,0x41,0xb4,0x00,0x58, -0x92,0x23,0x7d,0x32,0xf1,0x9f,0x52,0x4c,0xd3,0x06,0xf5,0x56, -0x05,0x26,0x99,0xd4,0x3c,0xcc,0x2f,0x81,0x96,0x07,0xfd,0xfe, -0x30,0xff,0x06,0x14,0xb0,0x74,0x90,0xd0,0x61,0x50,0x41,0x1c, -0xe3,0x51,0x14,0x56,0x1d,0x96,0x08,0xc3,0x21,0x59,0xaf,0x33, -0x00,0x15,0xb1,0xba,0x15,0xe7,0x03,0xdd,0x88,0x7f,0x46,0x1f, -0x12,0xa0,0x72,0x47,0x16,0x50,0x5c,0x8d,0x11,0x3d,0x0b,0xa6, -0x13,0xcc,0x38,0x78,0xea,0x27,0x91,0x78,0x46,0xdc,0x4e,0xca, -0x73,0x0d,0xa8,0x96,0x48,0x23,0x5a,0xa2,0xd6,0x3d,0xfc,0x6b, -0xc3,0x3f,0x5a,0xd8,0x7c,0x59,0x39,0x30,0x26,0x06,0xb9,0x8b, -0x76,0x04,0xf1,0x22,0x6c,0x4d,0x88,0x11,0x0a,0x24,0x89,0x57, -0x65,0x57,0xa5,0xe4,0x98,0xcb,0xc7,0xc0,0x2c,0x4e,0xab,0xbb, -0xc4,0xe2,0xcb,0x3b,0x02,0x76,0xfb,0x55,0x82,0xc4,0x9e,0xbf, -0x82,0x66,0xa9,0xea,0x4e,0x07,0x28,0x46,0x4a,0x08,0x0d,0xfe, -0xd0,0xa3,0x60,0x17,0x13,0xd4,0x17,0xba,0xf5,0x52,0xfa,0x23, -0xef,0x83,0x46,0x86,0x5e,0xc6,0x68,0x84,0x94,0x53,0x0f,0x10, -0x83,0x58,0xf0,0x5a,0x13,0xa1,0xc6,0x76,0xc8,0x22,0x4a,0x5c, -0x8f,0x60,0x99,0x36,0x15,0x0c,0x29,0xc6,0x6e,0xb9,0x0c,0x57, -0x88,0x33,0xd3,0x63,0x61,0xe3,0x0a,0x5e,0xce,0x04,0x3e,0x09, -0x04,0xb2,0x5c,0xc5,0x9e,0x25,0xd2,0x8d,0xc4,0xf4,0xd5,0x88, -0x07,0xba,0x0a,0xaf,0x14,0xf1,0x92,0xa3,0xf2,0x37,0x64,0xee, -0x26,0xab,0x1a,0x0d,0x58,0x45,0x6f,0xe7,0xea,0xb9,0x28,0xdf, -0x30,0x50,0x64,0x57,0x86,0x86,0x6b,0xd2,0xc3,0xe2,0xef,0x15, -0x38,0x0d,0x46,0x23,0xec,0x71,0x77,0x0e,0x5f,0x3c,0x75,0x9f, -0x7a,0x63,0x21,0x65,0x59,0xdb,0x22,0xba,0x5c,0x4a,0x95,0x6c, -0xb3,0x49,0x8d,0x81,0xee,0x37,0x1d,0xef,0x6e,0x9b,0x68,0x63, -0xca,0x9e,0x18,0xd5,0x38,0xeb,0x68,0xde,0x90,0xe0,0xc4,0x32, -0x6a,0xa0,0x85,0x4f,0xc5,0x16,0xf3,0x42,0x67,0x66,0x11,0x3e, -0x34,0xa5,0xd9,0xc9,0xb0,0x0a,0x1c,0xf9,0xe6,0x76,0x38,0x40, -0x6f,0xae,0xae,0x9e,0xda,0xf9,0x7c,0x56,0x71,0xb4,0xb8,0xa5, -0x57,0x72,0x3c,0x5e,0xac,0x5d,0xf4,0x46,0xb4,0x5a,0x9c,0xe7, -0x52,0x9a,0xfc,0xa3,0xbb,0x15,0x83,0x10,0xf5,0xd8,0xf9,0xfb, -0x1d,0xf1,0xe7,0x5e,0xe6,0x92,0x82,0xa0,0x64,0x10,0x8e,0xa8, -0x33,0x3b,0xd6,0x98,0x10,0xbb,0x29,0xf1,0x6a,0xe9,0x22,0x61, -0xd5,0xaa,0x8f,0x86,0xe3,0xab,0x01,0x94,0x3c,0x38,0x5a,0xd5, -0x4b,0x1d,0xea,0x25,0x2e,0xa8,0x70,0xae,0x20,0xcc,0x4e,0xf5, -0xba,0xe0,0xca,0x24,0xa8,0x1e,0xe6,0xfb,0xd7,0xc9,0x21,0xa9, -0xeb,0x18,0xde,0x8d,0x25,0xc5,0x36,0x28,0x1c,0xab,0xe6,0xb3, -0x24,0xef,0xbc,0x7b,0xb9,0x7a,0x3e,0x7f,0xd5,0x84,0xd6,0x00, -0x93,0x65,0x09,0x0f,0x8a,0xbf,0x2f,0x32,0x95,0x5f,0xe6,0x20, -0x1a,0xcc,0x56,0x12,0x1b,0xec,0xa2,0xc8,0xd7,0xda,0x93,0xeb, -0x4d,0x4c,0xd8,0x09,0xbe,0x82,0x62,0xd5,0x93,0x9c,0x5f,0xac, -0xf6,0x86,0x2b,0x29,0x3d,0x83,0x15,0x61,0x0d,0xff,0x64,0xf2, -0x99,0x3d,0x8e,0x8d,0xaf,0xb0,0x84,0x8b,0xae,0xb1,0x88,0x4d, -0xaf,0xdf,0x1f,0x90,0x07,0xec,0x15,0xb8,0xfc,0x00,0xf5,0xbf, -0x80,0x7f,0x3f,0x20,0x22,0x2f,0x9e,0x80,0x87,0xae,0xac,0x84, -0x89,0xaf,0xad,0x8a,0x32,0xe4,0x5d,0x31,0xf9,0x84,0x20,0x8c, -0x42,0x54,0x1a,0x11,0x79,0x0d,0x6c,0x27,0x0f,0x55,0x5f,0x84, -0xe1,0xbd,0xae,0x05,0x88,0x14,0xb3,0x55,0xa1,0x33,0x17,0x6e, -0x14,0x62,0xf1,0x1d,0x60,0xb1,0x09,0xff,0x7e,0xf5,0x18,0x26, -0x52,0x7c,0x05,0x02,0x73,0x35,0xcd,0xa9,0x48,0xdb,0x09,0x32, -0xb6,0xb5,0x49,0xe7,0x76,0x32,0xb8,0xa1,0x6c,0x69,0x61,0x54, -0xd0,0x79,0x62,0xc7,0xd5,0xfb,0xd7,0xbd,0xa3,0x83,0x72,0xbd, -0xc5,0x32,0x6b,0x58,0xc6,0xca,0xa0,0xe5,0xdc,0xae,0x2e,0xb5, -0x73,0xb0,0x51,0x62,0x85,0x88,0x42,0xb8,0x11,0xb0,0x20,0x63, -0x00,0x29,0xd1,0x59,0xb0,0x8e,0x1b,0x2b,0x43,0x83,0x80,0xe3, -0x0f,0x0e,0x6c,0xb1,0x53,0x55,0xac,0x87,0xb6,0x06,0x5f,0xe6, -0x17,0xc3,0x1e,0x9b,0xa2,0x8f,0xcb,0xd9,0xef,0x55,0x76,0x76, -0x11,0x25,0x99,0xee,0xcb,0x2b,0x75,0xdb,0x66,0x74,0x0d,0x94, -0x88,0x70,0xb1,0x40,0x6e,0xdf,0xb3,0x2d,0x52,0xab,0x58,0xb8, -0x4b,0x90,0x7d,0x45,0x2c,0xf9,0x9c,0x10,0x98,0xd9,0x08,0x98, -0x6f,0xf9,0xa4,0x37,0x44,0xbd,0x2d,0x77,0xb2,0x34,0xa2,0xbb, -0x50,0x7e,0xd7,0xdd,0xc6,0x40,0xe6,0x7b,0xcb,0x40,0xd3,0xf1, -0x18,0x1f,0xcf,0xe9,0x63,0xba,0xab,0xc6,0xa5,0x89,0x1e,0x88, -0xc7,0xae,0x1f,0xa5,0xf7,0x77,0x7b,0xc3,0xa1,0x8d,0x34,0xee, -0x6d,0xb9,0x7c,0x90,0x71,0xa6,0x24,0x05,0x70,0x21,0x10,0x9b, -0x6c,0xcb,0x85,0xdd,0x46,0x27,0xe2,0x16,0x26,0x0c,0x39,0xdd, -0x15,0x3c,0xa5,0x38,0xdd,0x8a,0xef,0xb0,0x11,0x5b,0x5f,0x3c, -0x38,0xc3,0xdc,0x3f,0x3c,0x38,0x3e,0x7e,0xf3,0xe3,0xdb,0x3d, -0xaf,0x00,0x20,0x39,0x03,0x59,0xcf,0xc3,0x61,0x94,0x44,0x3a, -0xc2,0x78,0x37,0x86,0xbc,0xe5,0x2e,0x1e,0xa1,0x53,0x1e,0x75, -0x84,0xbe,0x4a,0xee,0x57,0xcb,0x2c,0xe9,0xe6,0x54,0x1d,0xa3, -0x2f,0x16,0x42,0x47,0xd5,0x8b,0x72,0xc5,0xe1,0x3a,0xe5,0x63, -0x59,0xb9,0xb0,0xde,0xcb,0x21,0x7b,0x25,0x40,0xc6,0xb9,0x70, -0xe8,0xee,0xbb,0x43,0x8f,0x8a,0x7b,0x2b,0x49,0x62,0xc9,0xab, -0x51,0x28,0xf6,0xfa,0x77,0xfa,0x3c,0xab,0xf0,0x40,0xf8,0xc1, -0x1d,0x0f,0x89,0xf6,0xe6,0xd6,0x8c,0x71,0xe8,0x83,0x24,0x10, -0x28,0xa5,0x8a,0x20,0xda,0xdd,0xed,0x77,0xe2,0xbb,0xac,0xac, -0xa3,0x80,0x8e,0x7e,0x99,0x02,0xef,0xea,0xf5,0x3b,0xb1,0x3b, -0x13,0xdf,0x66,0x77,0xe8,0x45,0xe6,0x2e,0x1a,0x80,0x60,0x8f, -0xa6,0x2d,0x9a,0x10,0xf0,0x2f,0x7d,0xcc,0x6d,0xc1,0x3a,0x39, -0xfa,0xe1,0xba,0xe7,0x4c,0xf0,0x89,0x5a,0x59,0xf0,0x81,0x41, -0xf8,0x48,0xbc,0x97,0x83,0x07,0x6f,0x37,0x82,0x26,0xa2,0x37, -0xb1,0x87,0x5a,0x53,0xa5,0x6c,0x11,0xf6,0x06,0x35,0x4a,0x79, -0x53,0x0d,0x47,0x28,0x98,0x8c,0x97,0x23,0x3b,0xfd,0x58,0x33, -0xdf,0xab,0x03,0xf1,0x1e,0x00,0x0f,0xcc,0x13,0x58,0xac,0xd5, -0x79,0x0f,0xef,0x13,0xfa,0xe3,0xda,0x4a,0x61,0xba,0x4d,0xb6, -0x3a,0x1c,0xaf,0x33,0x51,0x52,0x1e,0xed,0x24,0x5a,0x29,0x74, -0x86,0x5b,0x86,0xae,0xe3,0x0f,0x01,0xa2,0x9d,0x20,0x14,0x42, -0x48,0x39,0xf4,0x55,0xbb,0x0c,0x5d,0x5e,0x7c,0xb3,0x8f,0x0e, -0x67,0x13,0xf4,0x06,0x16,0x4c,0x3f,0xcb,0xb5,0x58,0xb5,0x0a, -0xc6,0x9a,0xb2,0x59,0xad,0x2a,0xff,0x6d,0xf9,0x0e,0x6f,0x4e, -0x52,0x4a,0xa1,0xfa,0xf0,0x10,0x8a,0x7e,0x5b,0xf2,0xbb,0x2d, -0xbf,0xcf,0xe5,0xf7,0xfb,0x2e,0xdd,0xd8,0x38,0x9a,0x94,0x95, -0x2a,0xbb,0x6b,0xff,0x53,0xc8,0xd2,0x6d,0x5a,0x95,0x54,0x59, -0x5e,0xb0,0xb4,0x57,0xf7,0x57,0x7c,0x97,0x00,0xe4,0xb2,0x6b, -0x17,0x8b,0xb5,0x2d,0xa2,0x8c,0x6e,0x35,0x65,0x84,0x85,0xc3, -0x59,0xef,0x61,0x0a,0xe1,0xd8,0x70,0x62,0xdc,0xa7,0x5b,0x94, -0xd0,0x4f,0xad,0xae,0xb5,0x95,0x2c,0x9f,0x40,0x5a,0x8f,0x32, -0x12,0x4f,0x60,0xd0,0x6c,0xba,0x26,0x21,0x4a,0x51,0xa8,0x05, -0xd4,0x02,0x99,0x0a,0xb8,0x4e,0xcb,0x1b,0xd3,0xff,0xe9,0xf4, -0x12,0x21,0x3b,0x4c,0x76,0x59,0x46,0xf1,0xfe,0xc7,0x9d,0xc2, -0xa8,0xb6,0xe8,0x19,0x69,0xd7,0x7f,0x3b,0xfb,0xff,0xe4,0x7c, -0x9f,0x57,0x74,0xf9,0xf2,0x81,0x2e,0x2f,0x6c,0xf0,0x91,0x91, -0xf0,0x9a,0xcf,0xbb,0x44,0x10,0x17,0xfc,0xd1,0xa7,0xeb,0x2d, -0xd8,0x36,0xb5,0xbd,0xf9,0xb3,0x60,0xac,0x1c,0x2e,0x72,0x0c, -0x0f,0xe1,0xd4,0xd9,0x93,0x4b,0x99,0xb5,0x24,0x1c,0xfa,0x07, -0x87,0x6c,0xf9,0xd4,0x21,0xe3,0xbe,0x42,0xaf,0xb2,0x3c,0x66, -0x1e,0xc5,0xe2,0xe9,0x8e,0x4d,0xc9,0x5c,0x16,0x3c,0xdb,0xe1, -0xf2,0xa7,0x36,0xae,0x49,0x4c,0xd2,0x85,0xa8,0x29,0xe1,0x39, -0x9f,0x57,0x0f,0x83,0x72,0x4f,0x1a,0x57,0x7f,0x02,0x85,0x32, -0x2c,0x73,0x60,0x60,0xe5,0x5b,0xf6,0x98,0x36,0x0b,0xb4,0xcb, -0x8c,0xdf,0xdb,0x67,0xea,0x3c,0xc1,0x68,0x2e,0x9d,0x05,0x3c, -0xdb,0x28,0x89,0x23,0xd3,0xe2,0x87,0x51,0x6b,0x6f,0xa6,0x17, -0x62,0xa3,0xe4,0xd8,0x4c,0x1f,0x4e,0xd9,0x71,0x40,0x99,0xb0, -0x13,0x2b,0x4c,0xdc,0x76,0x37,0xc4,0x43,0xd7,0xac,0x2b,0xd3, -0xf0,0xd5,0xf1,0xcb,0x23,0x87,0xd7,0xf7,0x74,0x95,0x43,0x3d, -0xc1,0x17,0x3c,0x5b,0xe8,0xe1,0x9a,0xe9,0xd0,0x89,0xe9,0x49, -0x5a,0x38,0x82,0x43,0xfb,0x82,0x5b,0x6b,0x85,0xeb,0x86,0xd2, -0x41,0x79,0x6f,0xa4,0xce,0xb2,0x79,0x62,0x88,0x1c,0x8c,0x0c, -0xd8,0x12,0xad,0x93,0x75,0xd7,0x82,0x70,0x81,0xa4,0x5d,0xb2, -0x8d,0xe0,0x74,0x6d,0x11,0xc0,0x8a,0x4b,0xe8,0x96,0xef,0x1a, -0x44,0xb4,0xd9,0x9f,0xc4,0x90,0x38,0xe9,0x1a,0x7a,0xf2,0xc5, -0xd9,0x23,0x5c,0x24,0xb0,0xb6,0xd1,0x24,0x6b,0x1b,0x5b,0x96, -0x3b,0x3a,0x03,0x59,0x4c,0xce,0x38,0x91,0x2b,0xd5,0x47,0x6d, -0x76,0x7b,0xf0,0x6f,0xeb,0x6b,0x8b,0x85,0x55,0xd2,0xf6,0x40, -0x77,0x9c,0x80,0x1a,0xc9,0x03,0x6d,0x15,0x89,0x8b,0x88,0xca, -0x02,0x53,0xca,0xe6,0xca,0x54,0x3c,0x19,0x95,0xd0,0x11,0x64, -0x64,0x42,0x64,0x25,0xbb,0x20,0x8d,0x11,0xda,0x9a,0x7a,0xf3, -0xc6,0x16,0xea,0xef,0xab,0xfd,0xb9,0xf0,0xd2,0x24,0x49,0x6e, -0x81,0x5a,0x92,0xb8,0x28,0x26,0x2c,0xee,0xb4,0x07,0x77,0x67, -0xb0,0xe2,0x8e,0x7c,0x94,0x8f,0xb9,0x3d,0x24,0x62,0xd9,0xe9, -0xf9,0xd5,0x3d,0xe4,0xdb,0xea,0x06,0xcb,0x7e,0xa8,0x4b,0x93, -0xe9,0x2d,0x82,0xfb,0xbd,0x15,0xd7,0x21,0xff,0x16,0x72,0xd5, -0x16,0xfe,0xd9,0xc6,0x3f,0xcf,0xf1,0xcf,0xf7,0x7e,0x9c,0x4b, -0x22,0x9a,0x42,0x85,0x4a,0xbb,0xfa,0x57,0x65,0x69,0x3d,0x9e, -0x65,0xfb,0xf1,0x2c,0xcf,0x1f,0xcf,0xe2,0xb1,0x5e,0x25,0x87, -0x09,0x11,0x91,0xdb,0x0a,0x24,0x56,0xe8,0x9d,0x8c,0xd3,0xea, -0x75,0x71,0x47,0x5c,0xd0,0x2b,0xc2,0x26,0x26,0x1d,0x4e,0x63, -0x21,0xcc,0xe9,0x0b,0xb3,0x93,0x7e,0x01,0x83,0x76,0xbf,0x18, -0x5e,0x68,0xfb,0x8b,0xe2,0xd2,0xd8,0x26,0xca,0x2f,0x8e,0x13, -0x08,0x4d,0x12,0x42,0x1b,0x72,0xf4,0x51,0x24,0x42,0x07,0x3b, -0x76,0x7e,0x89,0x38,0xc6,0x68,0x5a,0x53,0x12,0xd3,0x57,0x10, -0x67,0x62,0x0a,0x38,0xb1,0x80,0x8f,0xe3,0x55,0xaf,0xb3,0xf3, -0x81,0xac,0x88,0x5f,0xc1,0xb6,0x41,0xb9,0x4d,0xad,0xff,0xae, -0x6d,0xda,0xfe,0xf6,0x36,0x6d,0xff,0x77,0x6d,0xd3,0xf3,0x6f, -0x6f,0xd3,0xf3,0xff,0xae,0x6d,0xfa,0xfe,0xdb,0xdb,0xf4,0xfd, -0x7f,0xcb,0x36,0x49,0x00,0xbd,0x59,0x3f,0xfa,0xa1,0xb6,0x2f, -0xee,0x3c,0x85,0x56,0x5f,0x3c,0x4e,0x71,0x0c,0x19,0x44,0x66, -0x9b,0x4a,0x46,0x88,0x8b,0x1c,0x8b,0xcc,0x1d,0x24,0xc1,0x69, -0x8a,0xe4,0x3c,0xc5,0xa4,0xae,0xec,0x04,0x09,0xb3,0x3f,0xe6, -0x5f,0xa6,0xa5,0x55,0xdb,0x26,0xbc,0xb5,0x86,0x31,0x39,0xe8, -0xc4,0xd1,0xcb,0x18,0xa5,0xbc,0xcb,0x51,0xf6,0x9b,0x17,0x61, -0xd6,0x4e,0x71,0x75,0xb1,0xc9,0xa7,0xc2,0xeb,0xe8,0xbb,0xa5, -0xbe,0xb7,0xd5,0xf7,0x73,0xf5,0x0d,0x63,0x69,0xdb,0x5b,0x3c, -0xdd,0x09,0x8e,0x4e,0xd0,0x09,0x3a,0xc8,0x18,0xb0,0x73,0x87, -0xcf,0x6c,0x65,0xf5,0x20,0x72,0x28,0xe0,0x5f,0x19,0xb2,0x88, -0x54,0x9e,0x42,0x29,0x62,0xa1,0x64,0xa4,0x05,0x39,0x2b,0x08, -0xfb,0xe6,0xc5,0xc0,0x6d,0x11,0x90,0x5c,0xfc,0xc1,0x28,0xea, -0x12,0x66,0x11,0xe9,0x25,0x1b,0x1c,0x96,0xda,0x6d,0xfb,0x6c, -0x14,0x39,0x27,0xb3,0x16,0x8a,0x5f,0xd1,0x50,0xe0,0x12,0x9f, -0x1f,0xab,0x16,0x42,0xbf,0x0e,0x22,0xf4,0x46,0x16,0xc1,0x98, -0x4d,0x6b,0xb0,0x32,0xd1,0xa2,0x55,0x5b,0x2c,0x6a,0x1c,0x72, -0xc8,0x27,0x35,0x7b,0x4c,0x06,0x95,0x0f,0x33,0xfc,0x3b,0xcd, -0xa2,0x06,0xeb,0xca,0x75,0xa2,0x69,0x94,0x3e,0x8b,0xe3,0x8f, -0xf5,0xfa,0xc7,0x22,0xb1,0x45,0xcf,0x1a,0x50,0x5f,0x03,0xd5, -0xe8,0x3a,0x1f,0xd3,0x69,0xf2,0x2c,0x69,0x3c,0x3b,0x75,0x91, -0xdd,0x67,0x8d,0x08,0x5b,0xd3,0x00,0x04,0x6a,0x80,0x41,0xbd, -0x5e,0x9b,0xf2,0x01,0x40,0xed,0x1e,0xa3,0x9f,0xd5,0x30,0x7e, -0x8d,0xe7,0x27,0xaa,0xfe,0xc3,0xb6,0x13,0x53,0xa0,0x58,0xed, -0x96,0xb0,0x88,0xf0,0x0b,0x67,0x52,0x4d,0x61,0xeb,0xe2,0x0b, -0xf3,0x12,0xca,0xca,0xf1,0x40,0x7b,0x99,0x70,0xa6,0x25,0x67, -0x9d,0x66,0x68,0x1e,0xe1,0xce,0x45,0x2d,0x5d,0xcf,0xb8,0x76, -0x47,0x78,0x7b,0x6c,0x5f,0xe2,0xc4,0x11,0xda,0xe9,0xfe,0x18, -0x51,0x2e,0xb2,0x52,0x6b,0x4d,0x62,0x56,0x58,0x8b,0xc4,0x3c, -0xf6,0x85,0x57,0xd5,0x84,0xcb,0x2e,0x47,0xea,0x8d,0xe2,0x21, -0xaa,0x0f,0x15,0x5e,0x29,0xb2,0xeb,0xa9,0xfb,0x65,0xe8,0x3e, -0x9a,0xb7,0x20,0x81,0xc1,0x02,0xff,0xaa,0x50,0xb4,0xb2,0x6c, -0x52,0xd9,0xc8,0x44,0x8e,0xbc,0x62,0xfa,0x69,0x80,0x6a,0xad, -0x62,0x70,0x94,0xa2,0xf0,0x8d,0x83,0x35,0x45,0x9a,0x6a,0x37, -0x52,0xc5,0xa1,0xc5,0xdc,0xec,0x07,0xee,0x14,0x3e,0xbb,0xa9, -0xff,0xcc,0x44,0x49,0x0c,0xad,0xf6,0xae,0xb5,0x4c,0x11,0x97, -0x26,0xb2,0x87,0x36,0xd5,0x63,0xb1,0x71,0x35,0x41,0x11,0xeb, -0xa4,0xd7,0x46,0xe1,0xa1,0xd3,0x32,0x78,0x57,0xb6,0xfe,0x9f, -0xab,0x9c,0x9f,0x04,0x16,0x1d,0xca,0x56,0xa3,0x2b,0x0d,0x38, -0x14,0xb3,0xb9,0xae,0xfa,0xc5,0x3a,0x1a,0x72,0xc5,0xac,0x1d, -0xf3,0xf0,0x29,0xad,0x7f,0x6b,0xeb,0xd2,0x97,0x15,0x60,0x75, -0x0b,0x1c,0x68,0x3c,0xb5,0x3f,0x19,0x5c,0x7c,0x34,0x65,0x2c, -0x5c,0x66,0xe7,0x02,0x49,0x9c,0x2e,0xdc,0x95,0x2d,0x6b,0x2f, -0x8b,0x99,0x61,0x5b,0x48,0x79,0x27,0x79,0x6f,0x3a,0x1e,0x29, -0x17,0x38,0xe2,0xf4,0x86,0x62,0xd5,0x69,0x18,0x5d,0x52,0xd0, -0x8a,0x35,0xb5,0x0f,0x63,0xc5,0xc3,0x9b,0x75,0xef,0x76,0x61, -0xdf,0x42,0xa3,0x15,0x75,0xf2,0x53,0x63,0x3d,0x4c,0xb9,0x82, -0x9d,0xd3,0x6e,0x7a,0xbf,0x2c,0xbe,0xa8,0xb4,0xc9,0xa6,0xb8, -0x59,0x62,0x4b,0xfe,0xce,0xdf,0x82,0x41,0x0b,0x14,0x76,0x1f, -0x55,0x5e,0x52,0xb9,0xc9,0x53,0x76,0x38,0x5d,0x92,0xc1,0x13, -0xb6,0x85,0x8f,0xa9,0xea,0x14,0xd8,0x6c,0x6c,0x08,0x78,0xf2, -0xfc,0x16,0x78,0xeb,0xb3,0xd9,0xd1,0x83,0x90,0xab,0xc7,0x77, -0xd7,0x03,0x15,0x15,0xdc,0xfc,0xb9,0xde,0x44,0x05,0x4c,0xd4, -0x0a,0xbc,0xb0,0xaf,0xb7,0x57,0x56,0x69,0x8a,0x3e,0xf2,0x96, -0xae,0xcb,0x8b,0x06,0x91,0xb8,0x1d,0x36,0xbf,0x11,0xc7,0x44, -0xea,0xe1,0xaa,0x1b,0xa8,0x7b,0xeb,0x86,0x30,0x14,0x44,0x80, -0xb5,0x4b,0xa1,0xd0,0xf3,0x05,0xac,0xeb,0x1c,0xdd,0xb6,0x30, -0xfd,0x52,0xc5,0xe7,0x0f,0x38,0x1e,0x08,0xc2,0x7b,0x87,0x70, -0x47,0x0f,0x96,0x6c,0xf5,0xeb,0x72,0xbb,0xe1,0x77,0xc6,0xfe, -0xc5,0x1c,0x73,0xb1,0xb4,0x37,0xcc,0x6c,0x53,0x7c,0x16,0xbc, -0x9f,0xe0,0x83,0x07,0x1a,0x7f,0x1b,0x4f,0xde,0xe4,0xec,0x77, -0x4b,0x7d,0x6f,0x93,0x2b,0xf3,0xa5,0xb8,0x5e,0x4d,0x4b,0xe3, -0x58,0x31,0x02,0x7a,0xc8,0x60,0xe8,0x47,0xe3,0xd9,0xe0,0xf2, -0x4b,0xaa,0x69,0xf3,0x6a,0x42,0x56,0x79,0x9e,0xdc,0x71,0xff, -0x47,0x7a,0x45,0x35,0xd9,0xe3,0x48,0x6d,0x17,0x12,0x4a,0xef, -0xb1,0xcb,0xd2,0x12,0xaf,0xc2,0xb7,0x71,0xf4,0x6b,0x4b,0x31, -0x03,0x13,0xff,0x16,0x40,0x8e,0x6e,0x86,0x7f,0x22,0x7f,0x3c, -0x7d,0xcb,0x67,0x4a,0xa6,0xfc,0xd1,0xde,0x36,0x17,0x71,0xdd, -0x19,0x2b,0x0f,0xf6,0x8e,0xf9,0x75,0xec,0x57,0x5a,0xe0,0x5c, -0x89,0x40,0x72,0x96,0xa9,0x13,0x0f,0x8f,0x86,0x25,0x4f,0x4a, -0x7c,0x10,0xe2,0xd0,0xc5,0x03,0xa3,0x26,0x5c,0x2d,0x2b,0x8d, -0xf3,0x23,0xa8,0x49,0x17,0x24,0x1d,0xf9,0x48,0x43,0x2e,0x99, -0x78,0x7a,0xf8,0x46,0xc4,0x0e,0x55,0xdf,0x66,0x15,0xb4,0xa4, -0xf0,0x63,0x82,0x0b,0xd0,0x0b,0x46,0xa6,0xa3,0x43,0xe5,0x1e, -0x74,0x30,0x15,0xae,0x95,0xa8,0xb9,0xeb,0x5b,0x26,0xd2,0x8e, -0x25,0x56,0x3a,0xd7,0x3e,0x2d,0x8c,0x75,0xa1,0x83,0xab,0x9a, -0xd5,0x95,0x63,0x49,0x9e,0x99,0x5f,0x5f,0x3e,0x91,0x75,0xc3, -0xb1,0x3c,0x13,0x51,0x03,0xa2,0x32,0xc9,0x06,0xb6,0xb5,0x94, -0x4b,0x62,0xb5,0xbc,0x46,0xc0,0xa8,0x20,0xf4,0xa5,0xb2,0xb4, -0x57,0x0f,0xe9,0x7d,0x74,0x9e,0x26,0x99,0x91,0x0a,0x75,0xf4, -0xab,0x26,0x40,0xdb,0xfb,0x89,0xa3,0x3c,0x9d,0x02,0x45,0x31, -0xf1,0xa6,0x21,0x65,0x70,0x64,0xb1,0x6d,0x0e,0x83,0x6b,0x1a, -0x91,0x70,0xe9,0x36,0x83,0xe9,0x51,0x80,0x86,0xc5,0xfc,0x60, -0x3e,0xbb,0x9d,0xb3,0x52,0x40,0x1b,0x29,0xa6,0x10,0xef,0x5a, -0xe8,0xdc,0x98,0x7a,0xb2,0x08,0x28,0x97,0x5a,0xa1,0xdb,0x9e, -0x1b,0xbc,0x94,0x70,0xc2,0x9c,0x86,0x8a,0xcf,0x07,0x4a,0x53, -0x98,0x93,0x58,0x12,0xef,0x54,0x44,0xc6,0x65,0x21,0xb7,0xdc, -0xd5,0xaa,0x95,0x6a,0x85,0xa5,0x97,0xba,0xd5,0x48,0xd2,0x1b, -0x5e,0x44,0x34,0x49,0x1f,0x86,0x56,0xa2,0x8e,0xb2,0x9c,0x40, -0x19,0x2a,0x3b,0x1f,0x95,0x3c,0x56,0xa1,0x53,0x28,0xa0,0x30, -0x42,0xb3,0xd2,0x28,0xa0,0x3c,0xe8,0xdf,0x24,0xe8,0x4b,0x3f, -0x59,0x92,0x0e,0xdf,0x1e,0xad,0x60,0xd0,0x0f,0x91,0x62,0xd5, -0x8a,0x52,0xa0,0xca,0x8b,0x50,0x2c,0x44,0x21,0xa4,0x40,0x8c, -0x6e,0x7d,0x2c,0xf3,0x4d,0x31,0x3b,0xf1,0xf0,0xca,0xf1,0xb5, -0x08,0xfe,0xab,0x98,0x71,0x55,0xc3,0xd1,0x9d,0x45,0x61,0xcc, -0xcc,0x13,0x56,0x46,0x71,0x10,0xfb,0x0d,0xcb,0x23,0xe9,0xde, -0x7d,0xd3,0xaa,0x58,0x9a,0xc1,0x15,0x8d,0x34,0xae,0x0b,0x9e, -0xbc,0x28,0x16,0x11,0x7a,0x6a,0xf7,0x7f,0x2b,0x3a,0x4f,0x5b, -0x0a,0x8b,0x58,0x7d,0xd3,0x12,0xf8,0xc4,0x15,0xb0,0x7a,0x12, -0x5d,0x4a,0xb7,0x57,0x73,0x10,0x24,0x00,0xd8,0x52,0xe0,0x0f, -0x5a,0x07,0x2a,0x90,0x73,0xb1,0xf4,0x37,0xac,0x86,0x95,0xdb, -0x8e,0x87,0x6b,0x0d,0xa1,0xba,0x69,0xa2,0x01,0xf9,0x6e,0x66, -0x50,0xa1,0xcc,0x51,0x81,0x87,0x96,0x27,0x2a,0x58,0x87,0x74, -0x21,0xcb,0xd1,0xec,0xdf,0xdc,0xca,0xda,0xfc,0x43,0x13,0x2c, -0xc5,0x3f,0x78,0x73,0x8f,0xd7,0xc2,0xda,0xa8,0xe4,0x91,0xdd, -0x24,0x17,0xce,0x33,0x4e,0x4e,0xde,0x66,0xad,0x2d,0x65,0xb0, -0x8a,0x9f,0x4b,0x65,0x37,0xf2,0x6a,0x4a,0xed,0xe3,0xc5,0xaa, -0x28,0xdb,0xe7,0x3a,0x99,0xad,0x74,0x30,0xe5,0xcc,0xd0,0xf8, -0xe7,0xf7,0x58,0x8b,0x2c,0xcf,0xf0,0x59,0xb0,0xc3,0x64,0xfd, -0xd7,0xae,0x72,0x2f,0xe5,0x7c,0x4f,0x39,0xbb,0x90,0xfe,0xf0, -0xc0,0x96,0xad,0x72,0x37,0x25,0x7e,0xa8,0xa4,0x94,0x92,0x45, -0xa8,0x39,0xee,0xbc,0x62,0xd0,0x27,0x85,0xbd,0x9f,0x06,0xfd, -0xd8,0x9b,0x26,0x46,0xb3,0x64,0x62,0xdb,0x96,0x8d,0xef,0x59, -0x43,0xb7,0x64,0x6a,0x0c,0xdd,0xbb,0x49,0x18,0x8b,0x1e,0x0f, -0xce,0x87,0xb8,0x6b,0xb3,0x85,0x27,0xf9,0x5d,0x21,0xea,0x02, -0x3d,0x6e,0xa2,0x9f,0xb4,0x20,0xe2,0xa4,0x37,0x18,0xb2,0x53, -0x48,0xb2,0x1a,0x1a,0xe1,0xdf,0xa8,0x2b,0x59,0xb0,0xdf,0xe9, -0xd3,0xe2,0xd4,0x87,0x5e,0x9f,0x8c,0xbf,0xc0,0xf6,0x72,0xad, -0xe5,0xcc,0xe0,0xe2,0x59,0xc7,0x7f,0xce,0x73,0xf4,0x26,0xdd, -0x75,0xc8,0x8f,0xa7,0xb3,0x97,0x34,0x48,0xc5,0x94,0xe1,0x60, -0x3a,0xcb,0x47,0x88,0xfd,0xbd,0x33,0x86,0x3b,0x98,0x92,0xc3, -0x94,0x1f,0x07,0x24,0x93,0x62,0x4a,0xc1,0x0a,0xff,0x21,0xf6, -0x45,0x4c,0x3d,0x52,0x32,0x5f,0xc7,0x1d,0x65,0x9f,0xdb,0x15, -0x68,0x08,0x8d,0x23,0x21,0x4d,0xc3,0xd0,0x91,0xed,0x90,0x21, -0x39,0xf5,0xc3,0xb3,0x43,0x4b,0xe9,0x51,0x85,0x31,0xb8,0xa4, -0x5d,0x8e,0xcb,0xe8,0xaf,0xba,0x1a,0x1d,0xe6,0xbd,0x09,0xe3, -0x15,0x9e,0xff,0x48,0xf6,0xd0,0x89,0x02,0x9a,0x5a,0x1d,0x0c, -0xf3,0x93,0xf1,0x2b,0x3a,0x6a,0x33,0x64,0xce,0xc8,0x9a,0x5b, -0xf2,0x8e,0xb1,0xdc,0x42,0xda,0x9b,0x02,0xb9,0xcc,0x76,0x26, -0x57,0xfc,0xb2,0x8c,0xb2,0x1b,0x7d,0x30,0x88,0x86,0x03,0xff, -0x0b,0xe9,0xe0,0x4f,0xa4,0xff,0xbd,0xe4,0xca,0xc9,0x25,0x04, -0x39,0x89,0xbd,0x57,0x06,0x86,0x52,0x39,0xda,0x1a,0xe5,0x9f, -0xfc,0xb2,0x2e,0x7d,0xce,0x48,0x90,0x37,0x56,0x43,0x04,0xe1, -0x6c,0xed,0x70,0x7a,0x27,0xa6,0x58,0xb2,0x8f,0xc9,0x60,0x28, -0xac,0x68,0x83,0x3f,0x6d,0xb4,0x26,0x87,0x12,0x81,0xb8,0x4c, -0x45,0xf2,0xa8,0xa6,0x9a,0x24,0x8d,0x09,0x31,0x7d,0x08,0xb2, -0x34,0x14,0xa5,0x5a,0x4a,0xf4,0xe9,0x91,0x54,0x4d,0x09,0xe7, -0x55,0xc2,0x91,0x8e,0xc4,0x2f,0x74,0xc6,0x90,0x2c,0x25,0x52, -0xcd,0x3a,0x97,0xcf,0xcd,0x3d,0x1b,0xa1,0x27,0x9f,0x8d,0xf3, -0x53,0x2d,0x88,0xf1,0x73,0xcd,0xf5,0xc3,0x8a,0x69,0x8a,0x39, -0x4d,0x71,0xde,0x76,0xe2,0x62,0x96,0xaa,0xea,0x4d,0x69,0x76, -0xd3,0x97,0xb3,0x1c,0xfe,0x00,0x1b,0x50,0x3d,0xb2,0x34,0xdc, -0x52,0x4f,0x2f,0x14,0xdc,0x03,0xd2,0xb5,0x3d,0x25,0xca,0x36, -0x7b,0xf2,0x42,0xc1,0xfa,0x48,0x1e,0xdb,0x21,0x41,0xce,0xaa, -0xc9,0xde,0x01,0x88,0xe8,0x2b,0xc2,0x87,0x56,0xa8,0x7d,0x36, -0x95,0x69,0x63,0x7b,0xd6,0xc8,0x47,0x76,0x7f,0x39,0x4a,0x5d, -0x65,0xf8,0x12,0x39,0xd5,0x44,0x8f,0xf0,0x53,0xf8,0x87,0xfe, -0xf6,0x52,0x07,0x3b,0xff,0x25,0x5d,0x5b,0x0b,0x11,0x5b,0x8a, -0x51,0x74,0x27,0x5c,0x58,0x98,0xf2,0x88,0x8b,0x83,0x64,0x45, -0xce,0x23,0x6b,0xf3,0x88,0x92,0x63,0x64,0xc3,0xb0,0x00,0x09, -0x7a,0x4d,0xed,0x84,0x0c,0x08,0x0f,0x51,0x1a,0x0f,0xfb,0xf8, -0x43,0x2d,0x4a,0xee,0x2d,0x60,0xbe,0x09,0x42,0x53,0x36,0x97, -0xb1,0x3d,0x4a,0xb7,0xae,0xc7,0x6a,0x16,0xf3,0x7a,0x1d,0xda, -0xe2,0xf4,0x6b,0xc4,0x7a,0xe4,0x64,0x70,0x85,0xfb,0x5a,0xc0, -0xcd,0x57,0xfa,0xf4,0xfa,0x7d,0x71,0xf2,0x15,0xcb,0x46,0xbc, -0xab,0x72,0xf2,0x40,0x1c,0x91,0x2f,0xea,0x98,0xbe,0xed,0x18, -0x38,0x53,0x74,0xac,0xd5,0x87,0xaa,0x6d,0x95,0x63,0x06,0xfc, -0x5d,0xa0,0xb8,0xd7,0xfd,0x16,0x82,0xf6,0x47,0xfb,0x50,0x3d, -0x42,0x70,0xbb,0xe3,0xe1,0x30,0xa7,0xfc,0x9e,0xf4,0x50,0xad, -0xc8,0x0d,0x9a,0x72,0xde,0x58,0x28,0x40,0x94,0x01,0xb5,0x70, -0x13,0x81,0xab,0x40,0xb9,0xdf,0x7b,0xf5,0x70,0xd6,0x09,0x82, -0x44,0xf1,0x10,0xf8,0x31,0xa7,0x57,0x0f,0x83,0xa9,0x28,0x11, -0xdb,0x62,0x49,0x42,0xb1,0x74,0x98,0xfd,0x76,0xf0,0x31,0x57, -0x09,0xf7,0xdc,0x6d,0xf3,0x7c,0x0d,0x5d,0xaf,0xd0,0x9b,0xaa, -0x21,0xe5,0x43,0x05,0x1d,0x49,0x0a,0x13,0xb0,0xa3,0xe5,0x62, -0xd4,0x66,0xb0,0xbe,0xec,0xb6,0xc4,0x4a,0xe5,0xcb,0x7c,0x46, -0x3a,0x5a,0x8d,0x46,0x62,0x1c,0x76,0x99,0xad,0xd4,0x3a,0x3b, -0x73,0x70,0xd6,0x50,0xb7,0x5f,0x72,0x91,0x51,0xa1,0x10,0x86, -0x29,0x56,0xe3,0x11,0x70,0xc5,0x12,0x67,0x69,0x05,0x6f,0x43, -0x5d,0xb4,0x1c,0x38,0x5a,0x00,0xa7,0x83,0x2e,0xd7,0x65,0x43, -0x0f,0xd5,0x86,0x9e,0x90,0x54,0x56,0xb1,0x50,0x52,0xd1,0x5d, -0xdc,0xd9,0x55,0xfd,0x75,0xe0,0x14,0x1b,0x3d,0xca,0x0f,0xf7, -0xd1,0x16,0xb5,0x03,0xc6,0x11,0xd7,0x73,0x37,0x4a,0xae,0xe7, -0x2a,0x4e,0xeb,0xa1,0x5e,0x57,0x5c,0xf7,0x55,0x45,0xd6,0x8e, -0x6b,0x1a,0x5e,0x20,0xe8,0x9e,0xc0,0xf0,0x83,0x7d,0x41,0x37, -0x0e,0x41,0x76,0x58,0xc9,0x5c,0xa3,0x1e,0xce,0x59,0x6e,0x30, -0xe9,0x42,0xbb,0xd2,0x2f,0xfc,0x28,0xde,0x17,0xb3,0xea,0xee, -0xb0,0x55,0x24,0x0f,0xb5,0xb1,0x5e,0x5f,0x7b,0xa4,0xb7,0x5c, -0xbd,0x1b,0x1b,0xd6,0xa9,0x73,0x80,0x3c,0x4c,0x5c,0x7a,0xd8, -0xa3,0x46,0xda,0x42,0xd4,0x63,0x6c,0xe3,0xca,0xad,0x0b,0x2d, -0xdd,0xda,0x84,0xe5,0xca,0x49,0xbe,0x63,0x79,0x89,0x65,0x08, -0x6e,0x7a,0xba,0x5e,0xe5,0x07,0x22,0x4b,0xb1,0xbf,0xcb,0x51, -0x2e,0x93,0x33,0xec,0x5a,0x40,0x05,0x48,0xcd,0xb1,0x0b,0x2b, -0x90,0x91,0x3a,0x5e,0x30,0x9d,0xd9,0xa2,0xcc,0x03,0xf4,0xda, -0xd6,0x87,0x51,0x62,0x13,0xb8,0x92,0x4f,0x95,0xd8,0x1d,0x37, -0x0c,0xaf,0x0a,0x79,0x73,0x53,0x34,0xcb,0x49,0xd9,0xed,0x63, -0x09,0x58,0x0c,0x8d,0x5b,0x2f,0x85,0x43,0xf4,0x07,0xe8,0x48, -0x80,0x1f,0x2a,0xb1,0x15,0x5d,0x7c,0xcb,0xf8,0xa6,0xff,0x19, -0x7f,0xf6,0xa7,0x57,0xe6,0x41,0xc9,0xec,0x69,0x12,0x99,0x99, -0xc1,0x96,0x0b,0x76,0x4f,0x86,0xcd,0x73,0x73,0x47,0x12,0x22, -0x6f,0xc7,0x74,0x0b,0xa5,0xa4,0xf5,0x48,0x1a,0x02,0x4b,0x66, -0x7f,0x4c,0x17,0x1f,0x84,0x21,0x99,0xe7,0x65,0xfc,0x32,0x86, -0xd2,0xf6,0x68,0x38,0x43,0x3c,0x74,0xde,0xc9,0xb9,0xf8,0x46, -0x35,0x56,0x99,0x44,0x95,0x51,0x6d,0xf8,0xcb,0x5e,0x27,0x61, -0xbd,0x85,0x5a,0x61,0xe6,0x78,0x81,0xcd,0x82,0x73,0xab,0x16, -0x19,0x48,0x13,0x36,0x69,0xe3,0x2c,0x06,0x62,0x4b,0x67,0x83, -0x51,0xa1,0x54,0x97,0xe7,0x94,0x13,0xbb,0xdc,0x72,0xf7,0x46, -0x9e,0x42,0xe8,0x0e,0x34,0x96,0xaa,0xc8,0x43,0x0e,0xd9,0x46, -0x93,0x34,0xfc,0xc4,0x49,0xc7,0x48,0x35,0xf3,0x5f,0x3a,0xf4, -0xbe,0x53,0xfc,0x32,0xd0,0xb0,0x26,0x69,0xc4,0x76,0xe9,0xbc, -0x88,0x20,0x87,0x89,0xa5,0x78,0xcc,0x4e,0x06,0x86,0x7a,0xef, -0x64,0xf7,0x6b,0x43,0x04,0x08,0xa7,0xaf,0xf4,0xf8,0x96,0xf1, -0xf5,0x67,0xae,0x6e,0x58,0xc3,0xbf,0xd8,0x53,0x6c,0x46,0x80, -0x93,0x2e,0x47,0x0a,0x9f,0x2c,0xcb,0xb4,0xd4,0x25,0x67,0x97, -0x44,0x80,0xb6,0x95,0xe6,0xd7,0x2f,0x80,0x2c,0xf0,0x01,0x29, -0x91,0x5b,0xf6,0xfd,0x06,0x04,0x1d,0x55,0x9c,0x72,0x6c,0x17, -0x64,0x87,0x62,0x14,0x49,0x0d,0x4c,0x9c,0x99,0x92,0xcf,0x04, -0x41,0xd4,0x27,0x88,0x40,0x04,0xac,0x45,0x0d,0x1f,0xd5,0xc4, -0x9d,0xd0,0x62,0xe1,0xc3,0xca,0xf7,0x68,0xea,0x62,0x05,0x68, -0x23,0x8b,0xda,0xb2,0x2e,0x20,0x14,0x71,0x9a,0xca,0x2d,0x6e, -0x40,0x12,0x73,0x07,0x95,0x44,0xfd,0x56,0xc2,0x9c,0xef,0x48, -0x18,0x64,0xf2,0x28,0xd5,0xd1,0xc3,0x67,0x7a,0x21,0xe8,0x09, -0xce,0xc9,0xdc,0x8b,0x85,0x44,0xa2,0xfa,0x36,0x11,0x7f,0xbd, -0xee,0xb3,0x29,0x59,0x3e,0x21,0xda,0x6c,0x97,0x73,0x97,0x60, -0xeb,0x42,0xed,0xc4,0xce,0x2c,0x97,0xce,0x7b,0x18,0x79,0x81, -0x60,0x53,0x47,0xf4,0xd6,0x0e,0x30,0x25,0xfa,0x40,0x83,0x2b, -0xb3,0xe1,0xc6,0x86,0x6c,0xa1,0xf5,0xc6,0xb6,0x74,0x26,0x43, -0xc6,0x86,0x61,0x5e,0xcb,0x7e,0xda,0x19,0x4a,0xaf,0x91,0xde, -0x38,0x76,0x05,0xfa,0x06,0x46,0xd5,0x8f,0x7e,0xb3,0xb6,0x73, -0x3e,0x26,0xf3,0x3b,0x6b,0x67,0xcc,0xf3,0x60,0xea,0xd4,0x2e, -0xd1,0x6b,0x30,0x2e,0x50,0x68,0xb0,0x07,0xfb,0xbb,0xf6,0x6b, -0x55,0x36,0xad,0x91,0x67,0x59,0xe2,0x30,0x3c,0x24,0x76,0x30, -0x12,0xfb,0x8a,0x82,0x50,0x5e,0x2c,0x4a,0x3c,0x83,0x85,0x1a, -0x8d,0x7c,0xbb,0xc0,0xbc,0x02,0xe6,0x52,0x4c,0x13,0x9e,0x12, -0x3f,0x36,0xbc,0xc8,0x9f,0xf9,0x10,0x44,0x33,0xe8,0xe0,0x48, -0x62,0x8d,0xd8,0x22,0xf6,0x6a,0xe1,0xd0,0xc4,0x39,0x95,0xf3, -0x67,0x39,0x1c,0x10,0x45,0x1e,0x6d,0x84,0x5d,0x0a,0x79,0x97, -0x6b,0xea,0xe8,0x85,0xfc,0x8a,0x17,0x28,0x2b,0x93,0x4a,0xe3, -0x8a,0xa4,0xf2,0x31,0x51,0x52,0x80,0x40,0xdb,0xbf,0x6a,0x08, -0x94,0x54,0x3e,0x55,0xf2,0xc7,0x54,0x3e,0x0e,0xad,0x8e,0x95, -0x22,0xab,0xcf,0xa7,0x02,0x5c,0xca,0x71,0x1e,0x92,0x8a,0xac, -0xde,0x2f,0x57,0xe0,0x14,0x9e,0x96,0xfd,0x53,0x4e,0xc7,0x70, -0xdc,0x71,0x19,0x4a,0xb5,0xc2,0xd4,0xa4,0xf8,0x6a,0xd4,0x1f, -0xe4,0x4c,0x12,0xde,0x6e,0xb9,0xc7,0xa7,0x5e,0x2d,0x28,0x04, -0x51,0x75,0x76,0x14,0xfa,0x22,0x58,0x2f,0x91,0x3a,0xa4,0x5b, -0x33,0xfa,0x7c,0x1b,0xa1,0x28,0xf1,0xe1,0x92,0xf5,0x7a,0xa5, -0x97,0x83,0xa4,0x7c,0x96,0xe7,0x1f,0x35,0x22,0xf2,0x4a,0x0e, -0x28,0xa2,0x5f,0x2d,0x28,0x84,0xc5,0xe9,0x3d,0x60,0xa9,0xa0, -0x3b,0xf9,0xaf,0x05,0x52,0x03,0x65,0x76,0x74,0x4f,0x6b,0x3f, -0xc3,0x79,0x78,0x62,0xca,0x0d,0xf0,0x7d,0xc0,0x01,0xb0,0x8e, -0xca,0x16,0x3b,0x58,0x72,0x6c,0x58,0x75,0x9b,0x41,0x37,0x46, -0xeb,0x7a,0x23,0x4a,0xc6,0xca,0xfd,0x4e,0x94,0xf6,0x93,0x10, -0xd5,0x7f,0xeb,0x0e,0x8f,0x8a,0x87,0x9c,0xa7,0x98,0xde,0xd5, -0x36,0xc3,0x7d,0x66,0xef,0x5f,0xb1,0x90,0x3d,0x2b,0x00,0xc5, -0x65,0x32,0x8c,0x92,0x25,0xc9,0x62,0xa2,0xf7,0xd8,0x61,0xc6, -0x53,0x6b,0xec,0x2f,0x8c,0xf6,0x8d,0xe8,0x7a,0xf2,0xbe,0x19, -0xcc,0xc2,0xa6,0x56,0xb5,0xd0,0x0c,0xac,0xbc,0x99,0xdf,0xdc, -0xce,0x48,0x20,0x56,0x67,0x3e,0xd3,0xd9,0xf8,0x16,0xb7,0x10, -0xbd,0x2b,0x71,0xba,0xdf,0x32,0xe4,0x14,0x22,0x23,0xb4,0x52, -0xea,0x3f,0x5e,0xc9,0x68,0x40,0xe4,0x99,0x6a,0xa1,0x94,0x66, -0xac,0x25,0x80,0x5b,0x4b,0x13,0x3a,0x58,0xd1,0xb9,0xd9,0xfd, -0x4b,0xd1,0x1f,0x05,0x15,0x2a,0x46,0xa2,0x6d,0x30,0xd7,0x0b, -0x3b,0xf8,0xce,0x96,0xdd,0xa5,0xc7,0xa7,0x94,0xa1,0xeb,0x0d, -0xd9,0x9b,0x96,0x13,0x60,0x0b,0xe3,0x62,0x4f,0x41,0x0a,0xc3, -0xb7,0x58,0x50,0xd7,0x18,0x71,0xe5,0xc3,0x6b,0x2e,0xb5,0x37, -0x73,0xcf,0x72,0xa5,0x13,0x0b,0x23,0x1d,0x0a,0x9f,0xb2,0x1f, -0x1f,0x14,0x07,0x0f,0xf7,0xd7,0x48,0xd8,0x55,0x09,0xf2,0xf4, -0x96,0x0e,0x19,0x75,0xeb,0x92,0x62,0x27,0x5b,0x8d,0x6f,0x42, -0xf2,0x91,0xa9,0x45,0x3b,0xbc,0x02,0xa6,0xa8,0x00,0x7e,0x91, -0xc7,0x03,0x83,0xc6,0x7e,0x61,0x73,0xe8,0xe4,0xe5,0xa9,0x6f, -0x68,0x41,0xfa,0xe0,0xc3,0xa6,0x76,0x50,0xb1,0xf1,0x8b,0x5e, -0x15,0xed,0x0d,0xcb,0x64,0xa7,0x77,0x1e,0xe1,0x3e,0x82,0xf6, -0x3e,0xf6,0xfb,0x41,0xb2,0x93,0x3c,0xff,0x3a,0x42,0xd2,0x82, -0x58,0x15,0x9d,0xb8,0x0d,0x9c,0xa3,0x35,0x2f,0xf0,0x95,0xa8, -0x8d,0x9f,0xef,0x5a,0x8a,0x1a,0x3e,0x46,0x4c,0xc3,0x22,0x1d, -0x0d,0x9f,0x42,0x3f,0x4f,0x21,0x91,0xe1,0xe3,0xd4,0xf1,0xff, -0x1d,0xe9,0x38,0x20,0xd3,0x25,0x9d,0x1d,0xfa,0x95,0xc4,0x5f, -0x87,0x94,0x5d,0x01,0x05,0xaa,0xc9,0xc7,0x17,0xf9,0xcb,0x7c, -0x98,0xc3,0x9c,0x2b,0xf9,0xfe,0x3f,0xde,0xdd,0x7b,0xbf,0x7b, -0xf0,0xee,0x64,0xef,0xcf,0x27,0xc7,0x99,0x0e,0xb4,0x45,0xbf, -0x82,0x3d,0xa6,0x4c,0x86,0x64,0xfa,0x14,0xfb,0x36,0x3b,0x8d, -0xf0,0xf8,0x22,0xea,0x1a,0x95,0xfa,0xe3,0xb0,0x77,0xf1,0x91, -0x53,0xbb,0x2c,0x47,0x56,0x16,0x7d,0xfa,0x3d,0x69,0x65,0x71, -0xaf,0xd7,0x54,0x4a,0x93,0x5b,0xbc,0x4a,0x94,0xbe,0xa9,0x56, -0x5f,0xbc,0x5c,0xab,0x4b,0x2b,0xdc,0xdf,0x0e,0xe9,0x8a,0x6f, -0xbd,0x3f,0xbe,0x98,0xb3,0x73,0xbe,0xf0,0x4a,0x17,0xdd,0x6f, -0xfe,0x34,0x1b,0x0c,0xa7,0xfa,0xde,0x16,0xca,0x18,0x57,0xc2, -0xa8,0x6b,0x5c,0x97,0x3b,0xd0,0x1d,0x13,0x37,0xb2,0x37,0xbc, -0x93,0x11,0x87,0xc9,0x3f,0x79,0xaf,0x9d,0x3c,0x36,0xb0,0x5f, -0x96,0x1c,0x1d,0x0f,0xa7,0x39,0x98,0x1e,0xc3,0x5c,0x3d,0xa0, -0x03,0xf9,0xd8,0x97,0x4c,0xd7,0xd6,0x5c,0xa0,0x89,0x6e,0xd3, -0xc4,0xae,0xf4,0x8d,0x3d,0x15,0xf7,0xd7,0x7d,0xa4,0xef,0x24, -0x9d,0xb0,0xc3,0xae,0x3d,0x7e,0xfc,0x72,0x38,0x86,0x69,0xf6, -0x45,0xf9,0x0b,0x37,0x23,0x8f,0x57,0xa6,0xea,0xb7,0x3a,0x03, -0xe8,0xd5,0xd8,0xef,0x98,0xc9,0xa3,0x98,0xf8,0x09,0x41,0x9b, -0xda,0x38,0xa9,0x90,0x93,0x8c,0xb2,0xaa,0x71,0xb6,0xfc,0x64, -0xe4,0x59,0x89,0xeb,0x94,0xaa,0xfc,0xa8,0xa6,0xea,0x9b,0x9a, -0xdc,0xbb,0x8a,0xb6,0xbc,0xa5,0x77,0x89,0x4b,0x2a,0x6b,0x76, -0x63,0xfd,0xc4,0x9a,0x5d,0xfe,0x95,0x35,0xb7,0xa4,0x66,0x4b, -0x86,0x1c,0xaf,0x9f,0x51,0x8d,0x70,0x27,0x9a,0xbf,0x1e,0x0f, -0x61,0xa2,0x9e,0x7c,0xb9,0xcd,0xc9,0x79,0xa1,0x98,0x06,0x77, -0x91,0x9e,0xac,0x67,0x93,0x39,0xb0,0x3c,0x39,0x69,0xb5,0xa2, -0xf0,0x7c,0x84,0x9a,0x15,0x27,0x2a,0xa9,0xe2,0x15,0x88,0x2e, -0xe9,0x75,0x53,0xb0,0x36,0x98,0x0c,0xbe,0x2a,0x75,0xcf,0x89, -0x2c,0x87,0xb0,0x31,0x55,0xc9,0xfc,0x72,0xff,0xe0,0xb2,0xaa, -0xaa,0x55,0x68,0xa1,0xff,0xdd,0x4a,0x58,0x0f,0xbc,0x5e,0x79, -0x00,0x98,0xa2,0x2c,0x0d,0x57,0x99,0xc1,0xc3,0xec,0x3b,0x53, -0x7a,0x5f,0x62,0xc2,0xae,0xb3,0x7e,0x9e,0xac,0x4b,0x9c,0xf3, -0x2f,0x84,0x52,0xe1,0x50,0x9a,0x7c,0x42,0x75,0x38,0x8d,0x9d, -0x3d,0xa5,0xa4,0x1d,0x89,0xab,0x8a,0x2a,0x6d,0x75,0x01,0xa6, -0x17,0xfe,0xd4,0x42,0x8c,0x91,0x00,0x33,0xd8,0x99,0xcd,0x50, -0x3e,0xa3,0x47,0xf1,0x8c,0x52,0xad,0xc7,0x67,0x6c,0x78,0x2c, -0x31,0x10,0xb3,0xed,0x92,0xbf,0x59,0xdb,0xe5,0x0f,0x32,0xcb, -0xdb,0xae,0x11,0xba,0x72,0x4a,0x51,0x6e,0x06,0x22,0x22,0xef, -0x7c,0x75,0xfc,0x62,0xa1,0x43,0xfa,0xc5,0x2f,0x7a,0xaa,0x2f, -0x64,0x4e,0x2a,0x08,0xa4,0xad,0x6e,0x0e,0xd7,0xec,0xf1,0x5f, -0x50,0xa8,0xa2,0xc5,0x98,0x6d,0x55,0x73,0x47,0xe3,0xd1,0x06, -0xc3,0xf3,0x2d,0xef,0x71,0x9b,0x47,0x33,0x58,0x7b,0x7e,0x99, -0x0f,0x28,0xb1,0x27,0x56,0xda,0xd3,0xa0,0x1b,0xb8,0xe9,0x4a, -0xe9,0xef,0x93,0x1e,0xbb,0x70,0x52,0xf8,0xd1,0x17,0x02,0x85, -0xa9,0xfb,0xe5,0x3c,0x17,0xca,0x51,0x9a,0x9f,0x3e,0x52,0x7b, -0x35,0xd3,0xb0,0x78,0x56,0xa2,0x53,0xce,0x8e,0xce,0xbe,0x82, -0x1a,0x53,0x9d,0x27,0x78,0x29,0x29,0x71,0x4c,0x86,0x21,0x36, -0x7e,0xfc,0x74,0xfc,0x62,0xa1,0x43,0x15,0xe3,0x17,0x00,0xa9, -0x68,0x51,0xfb,0x1f,0x26,0x6f,0x55,0xb8,0x5e,0x5f,0xd5,0x59, -0x7a,0x0a,0x54,0x60,0xb1,0xa2,0xa3,0xc8,0x06,0x34,0x32,0x98, -0x2c,0x90,0x42,0x9a,0x47,0x7b,0xc7,0x07,0x3f,0x1d,0x41,0xcc, -0x4f,0x47,0x6f,0x13,0x36,0xc2,0xfe,0xc0,0x1a,0x14,0x74,0x41, -0x65,0x1f,0x54,0x90,0xe8,0x68,0x9a,0x83,0xd0,0x35,0x04,0x2a, -0xfd,0x71,0x38,0xbe,0xf8,0x08,0xad,0x19,0xc2,0x66,0x00,0x09, -0xcf,0xb2,0xf5,0x1a,0x3a,0x42,0x20,0x77,0xe4,0x64,0xdc,0x8d, -0x2b,0xad,0x9d,0x7f,0x21,0x38,0x56,0xb2,0xaa,0xdd,0x12,0x0e, -0xcd,0x5a,0x0d,0x50,0x15,0x0a,0x0d,0x1a,0xae,0x0e,0x89,0x97, -0x2b,0x5a,0xfb,0xfa,0x64,0xff,0xad,0x45,0x1b,0x7d,0xdb,0x1e, -0xf7,0xd0,0x33,0xe7,0xdf,0x41,0x5c,0x0b,0x5a,0x56,0xd1,0x8a, -0xf9,0x68,0xda,0xbb,0x54,0x33,0x0d,0xd1,0x87,0xa9,0x36,0x07, -0x81,0x98,0x6c,0x70,0x60,0xaa,0x9e,0x65,0x14,0xb6,0xec,0x25, -0xe2,0x7b,0xaa,0xa0,0xc6,0x92,0x01,0x91,0x7f,0x6a,0x75,0x6d, -0x27,0xe8,0x20,0xf9,0xc5,0xd1,0xfa,0x54,0xea,0x45,0x5b,0xc5, -0x71,0x88,0x88,0xcf,0x8a,0xb7,0x1a,0x3a,0x2b,0x5f,0xa6,0x57, -0xcf,0xce,0xac,0x62,0x19,0x4d,0x0c,0x93,0x75,0xe6,0x1f,0x84, -0x09,0x9d,0x97,0x46,0xa1,0x5b,0x55,0xbe,0xba,0x26,0x0b,0x35, -0x04,0xb2,0x7b,0x7c,0xfc,0x0f,0xc3,0x00,0x42,0xfa,0x87,0x61, -0xfc,0xe1,0x1f,0x47,0x43,0xcf,0xc1,0x4a,0x60,0xab,0x70,0x4f, -0xcc,0xbd,0xac,0xb6,0xa9,0xfc,0x1a,0xcf,0xf9,0x52,0xff,0x69, -0x84,0x2b,0xa7,0xf2,0xbb,0x2c,0xee,0x5e,0x0a,0xaa,0x9a,0xb9, -0xf8,0xc0,0x04,0x21,0x8e,0x24,0x81,0xbc,0xe8,0x13,0xf3,0x51, -0x41,0xdf,0x41,0x58,0x13,0xe9,0x5e,0x22,0x0a,0x02,0xbd,0x53, -0xb8,0xd4,0x22,0xbd,0x9a,0xf3,0x5a,0x90,0x17,0x25,0x4b,0x2f, -0xca,0xab,0x7c,0xc4,0xb9,0xa4,0x0a,0xe0,0x9c,0xd3,0x41,0x6e, -0x55,0xba,0x39,0x33,0x7b,0xa1,0xb6,0x21,0x74,0x8d,0xac,0x53, -0xe8,0x8e,0x44,0x85,0xd7,0x02,0xa3,0x2c,0xbf,0x7d,0xa1,0xd3, -0xaa,0xd6,0x60,0x5a,0x48,0x3f,0xe2,0xe3,0x60,0x64,0x40,0x17, -0x33,0xbb,0x8c,0xce,0x7b,0xc3,0xda,0x1e,0x7a,0x5f,0xc1,0xd9, -0xdb,0x1f,0xe7,0x6c,0x1a,0x68,0x3a,0xbf,0xbd,0x1d,0x4f,0x66, -0xb5,0x37,0x74,0xc3,0x9c,0xcf,0xd0,0x75,0xdf,0x70,0x3c,0xc9, -0x27,0xb5,0x3b,0xd8,0x54,0xe3,0x88,0xfc,0x50,0xfb,0x1d,0xce, -0x6a,0x86,0x5a,0x43,0x9f,0xe6,0xc0,0xf4,0xfe,0x32,0x9e,0x93, -0xb1,0xe3,0xcb,0xc1,0x67,0x12,0xcf,0x90,0x3d,0xa2,0x49,0x56, -0x64,0x0c,0xd7,0x79,0x0d,0xeb,0xab,0xfd,0xb0,0x06,0xa8,0x22, -0xdf,0x23,0x56,0xf3,0x82,0x64,0x01,0x4c,0x1b,0xdf,0xa2,0xf7, -0x81,0x2f,0xc0,0x6b,0x6b,0x38,0xf7,0x5c,0xb7,0x00,0xd8,0xe3, -0x1c,0x33,0xcf,0x6e,0xd3,0xcd,0x4d,0x88,0x9d,0x5a,0x57,0xc7, -0x1f,0xa6,0xcd,0xf1,0xe4,0x6a,0x13,0x50,0xdf,0xc4,0x23,0xf9, -0x29,0xf2,0xe8,0xf1,0x04,0x70,0x99,0x20,0xc3,0x61,0x6f,0x98, -0x80,0xaa,0xe5,0x6c,0x90,0x9e,0xd1,0x45,0xa3,0x26,0xd1,0xa4, -0x0d,0xd1,0xb0,0x29,0x2a,0x79,0x54,0x55,0xf6,0x63,0x84,0x28, -0x30,0xa3,0x50,0x70,0xa6,0xc7,0xd5,0x46,0x52,0x06,0x4f,0xcf, -0x61,0x1e,0x45,0xe7,0x98,0xcd,0x8a,0xc8,0x41,0x1e,0x89,0xb4, -0x64,0xb8,0x58,0xc4,0xba,0xca,0x02,0x74,0x2f,0xf8,0x7f,0xb1, -0x96,0x23,0x0b,0x2f,0x6f,0x83,0x8a,0xc8,0x3d,0x13,0xea,0xc6, -0x51,0x2c,0x91,0x29,0xc0,0x0c,0x81,0xf0,0x61,0xb8,0x5c,0x0c, -0x4d,0xb1,0x05,0xea,0x12,0xc1,0xbd,0xda,0xa2,0x24,0x6b,0x9d, -0x09,0x0d,0x56,0x50,0xd8,0xaa,0x8b,0x75,0x38,0xbc,0xc2,0x28, -0xbc,0x7d,0xb3,0x1e,0x34,0x85,0x2b,0xe7,0x72,0x05,0x9b,0xee, -0x7c,0xd8,0x41,0x49,0x99,0xc2,0x5b,0x31,0x90,0x42,0xb7,0x18, -0xdd,0x5b,0xf2,0xed,0x14,0x59,0x99,0x6a,0x9a,0xf6,0xf9,0xaf, -0x26,0x03,0xf5,0xee,0x66,0x34,0xbf,0x11,0x45,0x0d,0x7f,0xf6, -0x47,0x6e,0xc6,0xbc,0xa3,0x3d,0x4a,0x41,0xc2,0x39,0xbd,0x80, -0xaf,0xe1,0x2e,0xdd,0x16,0x10,0x6e,0xef,0x7b,0xd3,0xf7,0x51, -0x83,0xf2,0x27,0xdd,0xac,0x70,0xcd,0xa0,0x7b,0x50,0xd5,0x23, -0x57,0x13,0x05,0x70,0xd0,0xa6,0xf7,0xc2,0x9f,0xab,0x20,0x86, -0x03,0xae,0x3a,0xd3,0xc3,0x95,0xc7,0x1b,0x45,0xc0,0x04,0x74, -0x05,0x9e,0x21,0x54,0xbb,0x49,0x2a,0x81,0x5c,0x12,0x15,0x85, -0xfc,0x99,0x5f,0xd0,0xaf,0x7a,0xfe,0x6e,0x9d,0x0b,0x6b,0x26, -0xea,0xb9,0x26,0xa7,0x7a,0xb6,0x29,0x0f,0x8c,0x73,0x88,0x9d, -0xb0,0xb3,0x7b,0x43,0xbb,0x6c,0x3e,0x9f,0x3c,0x66,0xde,0x84, -0xea,0x2e,0xbd,0x51,0x7f,0x32,0x1e,0xf4,0x51,0x71,0x2b,0x8e, -0x37,0x25,0x54,0x8b,0xcf,0xfa,0x8d,0x64,0xb3,0x99,0x7f,0xce, -0x2f,0x62,0x3f,0x68,0xb6,0x9a,0xe6,0xa8,0x77,0x37,0xb8,0x42, -0xcf,0x81,0xe8,0xb6,0x37,0x69,0x82,0xb8,0x32,0xd9,0xb9,0x22, -0x1d,0x09,0x3c,0xc0,0x4c,0x4e,0x5b,0xb0,0x64,0x9d,0x8f,0x3f, -0xe7,0x79,0xb6,0xf9,0x23,0xfe,0x6c,0x0e,0x9a,0x33,0xbc,0xaa, -0x79,0x1c,0x80,0xb1,0x0d,0x08,0xd8,0x38,0x79,0x07,0xe6,0xc6, -0x1c,0xe5,0x57,0xf9,0xe7,0x6c,0xf3,0xe7,0x78,0x7f,0xfc,0xf7, -0xc5,0xa7,0xfc,0xfc,0xe3,0x60,0xb6,0x38,0x58,0xdc,0x4c,0x93, -0xb8,0x93,0x9d,0xee,0x6c,0xfc,0xb5,0x9b,0x6c,0x42,0xd5,0xfd, -0x2f,0xc7,0xb3,0x2f,0xc3,0x3c,0x73,0x1c,0x10,0xa3,0xea,0xf5, -0x20,0xd8,0x9c,0x62,0x16,0x20,0xf7,0xde,0x68,0x3a,0xa0,0x0b, -0x6b,0xbc,0xe1,0x00,0xa1,0x88,0x79,0xde,0x94,0x1c,0x93,0x5d, -0xc6,0x0e,0x98,0x7f,0xef,0x8b,0xe6,0xc3,0x90,0x73,0xfb,0x24, -0x7b,0x88,0x91,0x29,0x24,0xb9,0xf7,0x30,0x2f,0xea,0xa6,0xaa, -0x91,0xe0,0x33,0x25,0x7c,0xf7,0x1c,0xc4,0xea,0x80,0x77,0xb3, -0xd7,0xc2,0x9d,0xf8,0x4f,0xb7,0xb0,0xa7,0x20,0xda,0x4b,0x1a, -0x55,0xd9,0x5a,0x89,0x9c,0x85,0xe8,0x44,0x60,0x7e,0x01,0xfc, -0xe8,0xbf,0xa8,0xb7,0x0e,0x6e,0x7b,0x17,0xc0,0xc5,0x22,0x8d, -0x7f,0xbd,0x1e,0x71,0x57,0xe2,0x4d,0x9d,0xee,0x8f,0x35,0x24, -0x76,0x1b,0x0e,0x8a,0x2c,0x16,0x1a,0x78,0x23,0x3a,0xa9,0xce, -0x96,0x04,0x1d,0x0a,0xe0,0x5c,0xf0,0x41,0x68,0x3b,0x95,0xb9, -0x12,0xb3,0x26,0x04,0x8a,0x9b,0x70,0x87,0x66,0xbd,0xee,0xeb, -0xc0,0x6b,0x38,0xd5,0x80,0xc1,0x54,0xf6,0x0a,0x15,0x23,0xdf, -0xe4,0x26,0x7b,0xc4,0x03,0x5c,0x9f,0x50,0xd2,0x21,0xe9,0x5c, -0x67,0xdc,0xc3,0x84,0x45,0xd3,0x0f,0xe9,0x5a,0xbc,0x66,0x49, -0x5d,0xa2,0x16,0x8b,0x62,0x0c,0x5d,0xfb,0x1d,0xa3,0x1d,0xc0, -0xc5,0xe2,0xfb,0x17,0xae,0x61,0x34,0x6f,0x12,0x03,0xe2,0xfc, -0x35,0x2b,0xb5,0xa5,0xd1,0x78,0xe4,0x43,0xd8,0x21,0x02,0x08, -0xe4,0xaf,0x35,0x87,0x9d,0x16,0x5e,0x16,0x8b,0xca,0xe8,0x17, -0xff,0x4e,0x60,0xf7,0x90,0x03,0xa8,0xfb,0xdb,0x3b,0x62,0x17, -0x78,0x28,0x41,0xee,0x2c,0x60,0x0b,0x4c,0x51,0xf5,0xfa,0xef, -0x60,0x2f,0x8c,0x22,0x96,0xf5,0x0b,0xc4,0x0a,0xb6,0x3f,0x59, -0x91,0x29,0xd6,0xac,0x44,0xee,0x47,0x44,0x0b,0xbb,0x3f,0xb8, -0xdb,0x1b,0xde,0xf8,0xd9,0x77,0x01,0xc4,0x39,0xcb,0xf7,0xd8, -0x78,0x5d,0x1c,0x41,0x32,0x9a,0x1c,0x2f,0x17,0xcf,0xa0,0xa9, -0x51,0x83,0xbe,0x71,0x7e,0x31,0x98,0xa5,0x3e,0xbc,0x0f,0xf3, -0x2f,0xd1,0x3a,0x52,0xea,0xaa,0xa1,0x0d,0x28,0xd0,0x36,0xef, -0x62,0x3b,0x2b,0xe2,0x41,0x54,0x41,0xf5,0xbc,0xbb,0x3c,0x85, -0xe9,0xae,0xc9,0x2e,0x0d,0xd8,0xa5,0x22,0xa3,0x54,0x7d,0x2b, -0x1a,0x49,0xfd,0x67,0x28,0x69,0x9f,0x0c,0x6e,0xf2,0xf1,0x7c, -0xf6,0x24,0x43,0x26,0xee,0x09,0x52,0xb4,0xfe,0xcb,0x57,0x18, -0x36,0x91,0x62,0x66,0xfd,0x97,0x2a,0x23,0x27,0xfe,0x90,0x8e, -0x31,0xc1,0x57,0x27,0xfd,0x7c,0xd8,0x43,0xe3,0x41,0xe8,0x39, -0x64,0x07,0xef,0x85,0xc4,0xb0,0x3d,0xe7,0x78,0xd3,0x77,0x76, -0x28,0xb2,0xf5,0x5f,0x9a,0xf6,0x5d,0xa4,0x35,0x95,0x55,0xb4, -0x69,0x61,0xa6,0x1f,0x07,0xb7,0x04,0x45,0x39,0x10,0xd6,0xb0, -0xeb,0xf5,0x35,0x15,0x74,0x8a,0x94,0xb6,0xb2,0x6c,0xb5,0xce, -0x03,0x5e,0x62,0x95,0x4c,0x3b,0x5c,0x8e,0x02,0x0d,0xc1,0xa2, -0x95,0x0e,0x98,0x2f,0x0f,0xaa,0x11,0x88,0xb2,0xab,0x2d,0x36, -0x3d,0xf5,0x16,0xbc,0x1c,0x4a,0xdd,0xa5,0x6b,0x53,0xa8,0xb2, -0xc1,0x97,0x68,0xc9,0x92,0x7b,0x30,0x31,0x15,0x65,0xb3,0x72, -0x2f,0x4e,0x4f,0x3d,0xe4,0xcc,0x9b,0xf8,0x28,0x9a,0x04,0xc1, -0x9c,0x6a,0x1b,0x2d,0x65,0x9a,0x17,0x78,0x08,0x34,0xcc,0x8a, -0x56,0x55,0xbc,0x00,0xc4,0x61,0x90,0x1d,0xcb,0xc8,0xd0,0xcc, -0xb1,0xb0,0x3b,0xf1,0xc3,0x8d,0xb6,0x3d,0x18,0x71,0x8d,0x79, -0x9f,0x1c,0xc9,0x3f,0xa1,0xb7,0x4c,0x38,0x82,0x82,0x71,0x5c, -0x91,0x35,0x21,0xd7,0x5d,0x46,0xc2,0xc1,0x44,0x59,0xff,0x49, -0xee,0x34,0xaa,0xa7,0x8a,0x36,0xb5,0x61,0xcb,0xa8,0xcb,0x0f, -0x16,0x78,0xe5,0x3c,0x1f,0xdf,0x6f,0x43,0x9c,0xed,0xc8,0x56, -0xeb,0x05,0x31,0x2f,0xe0,0x91,0x10,0x8b,0x76,0x97,0x80,0x97, -0xbf,0xc3,0x2d,0xd6,0x34,0x9f,0xed,0xcc,0x80,0xb5,0x9f,0xcf, -0x67,0x20,0xca,0x61,0xb9,0xc8,0xe0,0xdf,0xc4,0x58,0x18,0x3a, -0x37,0xa7,0x3c,0x19,0x04,0x61,0xd4,0xb9,0xc7,0x40,0x5a,0x01, -0xc9,0xd0,0xa9,0xfc,0xc5,0x78,0x58,0x4c,0xb4,0xf1,0xe6,0x7a, -0x3c,0x9d,0x95,0x4a,0x42,0xdc,0xb2,0x0a,0x1c,0x91,0x51,0x18, -0xbf,0x92,0xe7,0xf6,0x60,0x60,0xf9,0x0d,0x09,0xde,0x26,0xb9, -0x67,0xa7,0xbc,0x22,0xe1,0x6e,0x81,0xf6,0x79,0x84,0xe4,0xda, -0x96,0xdd,0xaa,0x38,0xb3,0x33,0xf2,0x6b,0xf4,0xd5,0x97,0xb6, -0xc8,0xf2,0xcb,0x1c,0xc4,0xbb,0x9f,0xec,0xcd,0x95,0xec,0x7d, -0x4a,0xcf,0x63,0x7c,0xbe,0x8e,0x7f,0x6d,0x6b,0xa3,0xb0,0xda, -0xd4,0x07,0x0b,0x9b,0x25,0xdb,0x41,0xe8,0x5e,0xdf,0xb6,0xc2, -0x45,0xba,0x2d,0x14,0xf6,0x54,0x90,0x83,0xba,0x2e,0x64,0xce, -0xff,0x45,0x4d,0xae,0x24,0x38,0x92,0xcd,0xf1,0x71,0x13,0x65, -0x51,0xa7,0xdd,0xeb,0xaf,0xc8,0xca,0x95,0x2b,0xb4,0x7e,0xcb, -0x5f,0x01,0x5d,0x5e,0xe1,0x85,0x39,0xab,0x95,0x98,0x4b,0x36, -0x01,0xa5,0x74,0xc9,0xb8,0x40,0xf3,0xd2,0xda,0x86,0x82,0x5c, -0x8d,0xe9,0xfc,0x12,0x57,0x1a,0x9b,0x99,0xb7,0xd8,0x14,0x97, -0x45,0x5c,0x63,0x64,0x2f,0x81,0x19,0x78,0x66,0x3f,0x56,0xbd, -0x77,0x2d,0xbf,0x63,0x2d,0x1b,0x07,0xe3,0x4d,0x99,0x45,0x2c, -0x38,0x14,0x54,0x58,0xc1,0x16,0x05,0x6f,0xa6,0xa5,0x51,0x11, -0x5f,0xa7,0x5f,0x7c,0x89,0x8c,0xfd,0x62,0x04,0x13,0x95,0xa7, -0x4f,0x27,0x39,0xf8,0xb7,0x9c,0x66,0xed,0x99,0xf1,0x6f,0x39, -0x9d,0x4d,0x28,0xe3,0xdf,0x72,0xda,0x70,0x70,0x03,0x62,0xda, -0x38,0x32,0xf2,0x51,0x91,0xc3,0x6e,0x51,0x20,0x8f,0xfd,0x2c, -0xe7,0x12,0x95,0x6b,0xc3,0xbf,0xe5,0xf4,0xf1,0x04,0x8f,0xef, -0xa0,0x89,0xf2,0x51,0xce,0x31,0x47,0x39,0x9c,0xeb,0x71,0x9f, -0x92,0xab,0xe8,0xa8,0xec,0x95,0x75,0x57,0x56,0xec,0x7e,0x7e, -0x2a,0xe5,0x0d,0x93,0x1b,0x7c,0xa9,0xc6,0x7e,0xd8,0x13,0x71, -0xcd,0xce,0xa6,0xb7,0x28,0xa3,0x3b,0x7d,0xa7,0x90,0x18,0x42, -0xce,0xfb,0xe8,0x14,0x87,0x6c,0xaa,0xb7,0xc5,0xca,0x9b,0x8f, -0x6c,0x5e,0x5c,0xe7,0x15,0x16,0x0c,0xec,0xde,0xe5,0x43,0xb6, -0xd5,0xfe,0xf0,0x83,0xca,0x2e,0xd7,0xb1,0x1f,0xf8,0x2a,0x76, -0xcd,0xa7,0x9c,0x7e,0xe8,0x5a,0xbb,0x13,0x4e,0x02,0x94,0x8f, -0xad,0xa5,0x91,0x29,0xad,0x91,0x2f,0x58,0x7d,0x63,0x37,0xdf, -0x14,0x79,0x3e,0x1e,0x0f,0xf3,0x1e,0xc4,0x41,0x0d,0xad,0x2c, -0x0b,0x0a,0xb9,0xef,0x2c,0x78,0x22,0x46,0x86,0xe0,0xef,0x0b, -0xe7,0x0e,0xa2,0x60,0xef,0xd2,0x97,0xb2,0xed,0xb1,0x56,0xe5, -0x9e,0x04,0x8c,0x9c,0x2f,0xc4,0x11,0x3b,0x1d,0xc3,0xdd,0xd5, -0x5b,0x24,0x19,0xde,0x5d,0x19,0x8c,0xc7,0x97,0x22,0x61,0xb4, -0xf3,0xb2,0x4b,0x45,0x5e,0x6c,0xb4,0x96,0x3c,0x57,0xf3,0xde, -0x04,0xf6,0x7c,0x15,0x55,0x55,0x3d,0x0e,0xc4,0x14,0xd8,0x68, -0xad,0xd1,0x1d,0xa1,0xf2,0x49,0xbc,0xe5,0xfa,0x97,0xe1,0x39, -0x30,0x7e,0x77,0x97,0xb4,0x43,0x63,0x7e,0xe4,0x89,0x24,0xec, -0x59,0x0a,0x09,0x85,0xa7,0x95,0x76,0xfe,0x5c,0xdf,0x78,0x3c, -0x79,0x74,0xf8,0x85,0x65,0x94,0x86,0x75,0x70,0x43,0x82,0x0c, -0x0f,0x41,0xb2,0x43,0xa0,0x2c,0xdb,0xff,0x51,0x1e,0x2f,0x01, -0xb2,0xd8,0x1f,0xeb,0xe8,0x24,0x8b,0xa3,0x7d,0xd3,0xeb,0x75, -0xdf,0xe8,0x53,0x4e,0x14,0x2f,0x26,0x89,0xa3,0x35,0x47,0x7d, -0x84,0x0c,0x4d,0x84,0x28,0xd5,0x8f,0xde,0x06,0x3f,0xa0,0x01, -0x6d,0x65,0x31,0x1f,0x4d,0x5e,0x7a,0xb0,0x83,0x22,0xc4,0xb6, -0x83,0x18,0x9a,0x23,0x04,0x31,0x65,0xd9,0x2e,0x99,0x4d,0x94, -0x79,0xfa,0xf4,0x0e,0xf7,0x65,0xb2,0xfb,0x75,0x15,0x5a,0x86, -0xdd,0xad,0x8c,0xa8,0x63,0x37,0xa9,0x9a,0xa0,0xab,0x32,0xb4, -0xe2,0xdf,0x59,0x0b,0xb5,0xd2,0x7d,0x16,0x7e,0xae,0x25,0x8b, -0xed,0xec,0x1a,0x33,0xb7,0xd5,0x9c,0x66,0xd7,0x94,0xd5,0x47, -0x52,0xd2,0x31,0x77,0xee,0xe0,0x4c,0x20,0x92,0xb9,0x4a,0xb2, -0x86,0x13,0x83,0xc0,0x16,0x56,0x1c,0xb9,0x73,0x72,0x7f,0x63, -0xfd,0xcf,0x44,0x46,0xfc,0xfe,0x30,0x4e,0x64,0xc3,0x76,0x15, -0x66,0x6d,0xc5,0x55,0x3c,0xab,0x29,0xd6,0xa6,0xfa,0x52,0x0a, -0x14,0xec,0x4e,0xb2,0x5b,0x0b,0x3b,0x02,0xcc,0xb1,0x41,0x62, -0x39,0xed,0x1a,0xe6,0x8f,0xfc,0xfa,0x55,0xb1,0x46,0x6f,0xaf, -0x8f,0xdd,0x61,0x01,0x73,0x6c,0x17,0x79,0xae,0x7b,0xb7,0x63, -0xe1,0x31,0x32,0x72,0xe2,0x67,0xd7,0x01,0x49,0x53,0x46,0x11, -0xc2,0x15,0x15,0x55,0x99,0x2e,0x7a,0x43,0x6b,0x50,0x80,0x8e, -0xe0,0xa7,0x99,0x44,0x36,0xdf,0xfd,0xb4,0xff,0xe3,0xde,0xd1, -0xfb,0x57,0x07,0x47,0xfb,0x3b,0x27,0xc7,0x25,0x1f,0x91,0xbd, -0x1b,0xb4,0x4f,0xe8,0xd6,0xe8,0xe3,0x2f,0x37,0xe7,0x63,0xa7, -0xd3,0x54,0xd3,0x9b,0xf7,0x42,0x16,0x7c,0x02,0x19,0xc4,0x64, -0x52,0x73,0x73,0xf7,0xa7,0xa3,0xa3,0xbd,0x77,0xbb,0x7f,0x79, -0x7f,0xfc,0x97,0xfd,0xc4,0x70,0xac,0xb8,0x3c,0x97,0xda,0x6c, -0xce,0xc3,0x9d,0x93,0x93,0xbd,0xa3,0x77,0xc7,0x68,0xc5,0xcf, -0xc6,0xfd,0xfe,0xe8,0xe0,0xa7,0xc3,0xf7,0xc7,0x7b,0x87,0x2e, -0xe6,0xe5,0xde,0xee,0x9b,0xfd,0x9d,0xb7,0x14,0xb7,0x9d,0x80, -0x68,0x73,0x3b,0x44,0x3f,0xe0,0x9b,0x67,0xf3,0xad,0xad,0x9d, -0xef,0x37,0xaf,0x8a,0xc8,0xab,0x8e,0xd2,0x0b,0xf7,0x3f,0xd4, -0x4d,0x0c,0xc8,0x5c,0xa2,0xa7,0x6c,0x08,0x1f,0x0f,0xfe,0xee, -0x49,0x32,0x68,0xa1,0xcd,0x58,0x6c,0xe1,0xd6,0x53,0x5b,0x18, -0x54,0xa1,0xbd,0x99,0x56,0xd4,0x02,0x74,0x8e,0x57,0x45,0xe6, -0x6a,0x32,0x9e,0xdf,0x1e,0xe7,0xb7,0xb0,0x11,0xbb,0x18,0xdc, -0xf4,0x86,0xf8,0x19,0xa2,0xca,0x1e,0xea,0x7b,0xef,0xa4,0x64, -0xb2,0x58,0xe0,0x93,0x78,0x7c,0xda,0x95,0xdb,0x28,0x61,0x76, -0x51,0x44,0x32,0x03,0xe4,0xc6,0x0b,0x92,0xc1,0x5d,0x9e,0x6d, -0xbd,0xe0,0x1c,0x6d,0xfe,0xc9,0xf6,0xd1,0x02,0x74,0xef,0x7c, -0x6a,0x0b,0xf2,0xc3,0xea,0xf9,0xcd,0xf1,0x6c,0x92,0x71,0x54, -0x23,0x8a,0xa4,0x6d,0x79,0xff,0x04,0x17,0x4f,0x08,0x83,0xc0, -0x4d,0xee,0x73,0xe9,0x40,0xe9,0xf3,0xed,0x78,0x94,0x93,0x77, -0x14,0x3c,0x22,0xda,0x68,0xe1,0xbb,0x51,0x02,0xe0,0xbd,0xd3, -0xe3,0x2d,0x33,0xfb,0xcb,0xa4,0x63,0x52,0x49,0x66,0x05,0xb9, -0xcd,0xf8,0xf4,0xac,0x7f,0xd6,0xec,0x36,0x92,0x3c,0xde,0xe8, -0x24,0x7c,0xf6,0x9c,0xb4,0x6f,0xf8,0x3d,0x5e,0xb4,0x11,0x89, -0x16,0xde,0xe9,0x76,0x17,0xd5,0x25,0xf0,0xeb,0x79,0xf7,0x85, -0xee,0x91,0x46,0xab,0x23,0x18,0x47,0x5b,0x51,0x1a,0x07,0xc8, -0x72,0x42,0x88,0xe7,0x16,0x69,0x0e,0xa8,0xa8,0x44,0x43,0x43, -0xff,0xbf,0xdc,0x72,0x58,0xe9,0xeb,0xf5,0x96,0x74,0x18,0x4c, -0x91,0x22,0x60,0x88,0x05,0x41,0xe1,0xd5,0xe0,0x33,0x4c,0xa8, -0x60,0x84,0x94,0xf3,0x3b,0x1b,0xff,0x36,0x1f,0x65,0xb1,0xb4, -0x9b,0x0d,0x6e,0x2b,0x3a,0xc1,0x23,0x73,0x54,0x3f,0x49,0xdc, -0x2a,0xa6,0xa6,0x6a,0x00,0x19,0xb1,0x50,0x61,0x1e,0xbe,0x9b, -0xc1,0x28,0xe6,0x8f,0xde,0xe7,0x58,0xc8,0x08,0x23,0x5f,0x41, -0x4e,0xa3,0x10,0x48,0x8c,0x4b,0xec,0x7d,0xc6,0x44,0xb9,0xfa, -0xbf,0x1d,0x7f,0x62,0x40,0xf0,0x11,0xb7,0xb6,0x42,0x6a,0x0b, -0x08,0x05,0x48,0x73,0xd4,0x17,0x52,0xf9,0x0e,0x72,0x27,0x9b, -0xf0,0xa7,0xad,0xdb,0x49,0xa2,0x95,0xd0,0x52,0x45,0x43,0xcd, -0xa7,0x6b,0x58,0x40,0x33,0x9b,0x1b,0x66,0x52,0xdb,0x95,0x74, -0x91,0xdc,0x19,0x82,0xda,0x14,0xd5,0xa1,0x69,0x4e,0x64,0x16, -0xfb,0xe1,0x15,0x62,0x66,0xc2,0x48,0x8a,0x43,0x02,0xa4,0x1a, -0xac,0xcf,0x91,0x8c,0x8b,0x36,0xe8,0x6f,0x82,0x2f,0xc5,0x32, -0x9d,0xbe,0xc1,0xc9,0xc1,0x1b,0x7a,0xc8,0x23,0xba,0x89,0x68, -0xf1,0x35,0x86,0xe0,0xc6,0x20,0xf9,0x15,0xe5,0xab,0xd7,0xd1, -0x57,0xcb,0xa0,0x40,0x0a,0x8d,0xcc,0x4e,0xd9,0xc4,0x84,0xf1, -0x5c,0x95,0x48,0x3b,0x03,0xf6,0x40,0x38,0xc8,0x00,0x22,0xc8, -0x2c,0x1a,0x0d,0x5f,0x5b,0x80,0x1c,0x54,0x3b,0xfc,0x67,0xd4, -0x4b,0xcd,0x73,0xfd,0x2c,0xd0,0x7f,0xd0,0xa3,0xdc,0x76,0x33, -0xa0,0x81,0x73,0xa8,0xad,0xd3,0x60,0x0e,0x6e,0xa1,0xec,0x16, -0xc6,0x15,0x30,0xf1,0xbc,0xaa,0xe1,0xea,0x71,0x97,0x18,0xe1, -0xdc,0x58,0xfa,0xad,0xbd,0x75,0xa3,0xed,0xf9,0x53,0xc7,0x0e, -0xe7,0x28,0xbf,0x3a,0x9c,0xe4,0xa9,0x0d,0x42,0x9f,0x41,0x30, -0x31,0xaa,0x90,0x46,0x20,0x48,0xa8,0x86,0x76,0x3c,0xbf,0xd4, -0xd0,0x20,0x68,0x0b,0x7d,0x18,0xc3,0xfc,0x81,0xc9,0xa7,0xdc, -0x6a,0xf6,0xfa,0x9e,0x37,0x9b,0xfe,0xe0,0x6a,0x30,0x9b,0x1a, -0x10,0xef,0x6e,0xe4,0x15,0x4d,0x7e,0x05,0x4c,0x90,0x7a,0x95, -0xd8,0x29,0x59,0x22,0x80,0xa8,0x0d,0xda,0x5c,0x66,0x1b,0xf0, -0x27,0xa1,0x2f,0x9e,0x0a,0x38,0x83,0x6c,0xa7,0x33,0xac,0x76, -0x42,0xa9,0x5b,0x9c,0xec,0x2e,0x0b,0x07,0x04,0x09,0x52,0xb0, -0x80,0xf4,0x9e,0x2f,0xbb,0xc1,0x65,0x13,0xb4,0x9a,0x70,0x85, -0x05,0x95,0x65,0x68,0xe8,0x07,0xb1,0x8e,0x41,0xe7,0x0f,0x53, -0x9c,0x1e,0xe3,0xcb,0xcb,0x69,0x3e,0x13,0xb4,0xad,0xe3,0x27, -0x8a,0xcb,0xf8,0x67,0xb1,0xd8,0xf2,0x27,0x06,0x7d,0x67,0x3f, -0x88,0xc5,0x1c,0x0c,0x9f,0xe2,0x1d,0x2a,0xe0,0x88,0x4f,0x13, -0xbc,0xf7,0x59,0x2e,0xfc,0x62,0x6b,0xb1,0xa0,0x9c,0x2f,0x36, -0x38,0x22,0xb1,0x6f,0xa7,0x1b,0x02,0x9e,0x2d,0x26,0xcb,0x83, -0xe7,0x8d,0xd6,0x76,0x26,0xf1,0xee,0x8d,0x75,0x6b,0x1b,0x47, -0xc0,0xf6,0x34,0x4b,0x7f,0x84,0x39,0xa1,0xbc,0x0c,0x9b,0x07, -0x7c,0x33,0x68,0xe1,0xf5,0x78,0x32,0x7b,0x05,0x14,0x50,0xde, -0x61,0x63,0x6e,0xbb,0x18,0x3f,0xdc,0x22,0x32,0x6b,0xe3,0x76, -0xf2,0xb1,0x83,0xd9,0x89,0x8e,0x5f,0x1f,0x1c,0x9d,0x70,0xbe, -0x94,0xef,0xaa,0x03,0xf1,0x60,0x7a,0x0a,0x25,0xbb,0xa7,0x04, -0xb7,0xab,0x10,0xc5,0x43,0xce,0xbf,0xc2,0x8a,0x22,0x98,0xfa, -0x3e,0xfd,0x3b,0x9a,0x7d,0xdb,0x68,0x7d,0xd7,0xb7,0x9a,0x0c, -0x90,0x11,0xe3,0x0e,0xa8,0x47,0x62,0xea,0x87,0x7e,0xde,0xc7, -0xb2,0x19,0xc6,0xbf,0xc8,0xb6,0x3a,0x51,0x23,0x4a,0x81,0xca, -0xdc,0x6c,0xb1,0x19,0x1a,0x99,0xef,0x34,0x64,0xca,0xa7,0x54, -0x00,0xf2,0x5f,0x0e,0xc7,0x63,0xd8,0x8c,0x44,0x17,0xf9,0x60, -0x18,0x75,0x63,0x8c,0xde,0xfc,0xcd,0x56,0x02,0x62,0x56,0x23, -0x2c,0x41,0xeb,0x3d,0x26,0xff,0x8a,0x93,0x95,0xb1,0xf0,0x9b, -0xdb,0x1b,0x85,0xbc,0xef,0x46,0x6b,0x91,0x5a,0xf0,0x7f,0x3d, -0x9e,0x4f,0xa6,0x71,0xf2,0x43,0x6b,0xbb,0x63,0xc5,0x9e,0x9d, -0xfd,0xc3,0xfd,0x63,0xf2,0x66,0x1f,0x44,0xb4,0xba,0xe1,0x38, -0x16,0xc5,0x37,0x97,0x88,0x87,0x3f,0x7c,0xe7,0x76,0x32,0x7e, -0x09,0x19,0x63,0xde,0x4f,0x29,0xb1,0xa1,0xed,0xae,0x59,0x39, -0x49,0xe4,0x87,0xa3,0xf7,0x6f,0x8e,0x0f,0x7e,0xfb,0x9b,0xad, -0xd6,0xfb,0xe3,0x93,0x23,0x7b,0x03,0x05,0x00,0x48,0xc9,0x9a, -0x20,0xa1,0xb3,0xb2,0xbf,0x23,0xca,0xb0,0x94,0xcc,0xfe,0xbe, -0x3f,0x18,0xc1,0x2f,0xd1,0x14,0x5b,0x88,0x60,0x99,0xe2,0xb7, -0xdd,0x0e,0xb5,0x0e,0xc6,0xe3,0xa7,0x93,0xdd,0x57,0xf3,0xe1, -0xf0,0x2f,0xb0,0x3b,0x49,0x6d,0x9c,0x8d,0xa0,0x83,0xec,0x07, -0x0a,0x52,0xcf,0xb8,0x52,0x14,0x62,0x51,0xe6,0xf4,0x77,0x68, -0x68,0x44,0xf0,0xc0,0xeb,0x76,0x1b,0xdb,0xe0,0x8f,0x16,0x3a, -0xf2,0x62,0xec,0xaa,0x52,0x5b,0x5d,0x98,0xf8,0x1e,0x69,0xb6, -0x02,0x44,0x63,0xe4,0x73,0xe3,0xbd,0xbb,0x0f,0x6d,0x77,0x93, -0x8d,0x96,0x0a,0x3f,0xef,0xca,0x72,0x7f,0xad,0x2a,0xf8,0x1e, -0x96,0xda,0xad,0x64,0x83,0xd1,0x32,0x37,0x2a,0xe5,0xd7,0x36, -0x05,0x30,0x32,0x53,0x95,0xf0,0x1b,0x4a,0x30,0x37,0x53,0x2d, -0x12,0xb4,0xf2,0xe7,0xdf,0xd1,0x29,0xec,0xab,0xe1,0xb8,0x37, -0x8b,0xa3,0xad,0x66,0xd4,0x90,0xfc,0xff,0x4e,0xf9,0xbd,0x9b, -0x3c,0xdf,0x81,0xaa,0x11,0xd7,0xe6,0xc6,0x4c,0x01,0x26,0xb7, -0xd1,0x2e,0x11,0x3c,0xce,0x74,0xd8,0x12,0x8c,0x32,0x2a,0x00, -0x9c,0xf5,0xef,0xbf,0x5f,0x26,0x1b,0x1d,0xf8,0x38,0xeb,0xbb, -0xdf,0xb8,0x93,0x9e,0xb8,0xaf,0xb4,0x53,0xf5,0x79,0xd6,0x24, -0x11,0x33,0xe9,0xe0,0xff,0xf1,0x5f,0x17,0xf1,0x69,0x63,0xa3, -0x9b,0x70,0xb2,0xcd,0x86,0x49,0xeb,0x9b,0xa5,0x8d,0x83,0x9a, -0x13,0xd6,0xe0,0x9b,0xe8,0x55,0xcc,0x0a,0x82,0x31,0xd2,0x2a, -0x67,0x94,0x9d,0x14,0x48,0x34,0x37,0xb0,0x5f,0x9c,0xdf,0xbc, -0x64,0x85,0x37,0x4e,0xb3,0x1b,0x96,0x97,0x3b,0x27,0x7b,0x27, -0x6f,0xf6,0xf7,0xec,0x96,0xe5,0x94,0xd3,0xa1,0xe7,0xf8,0xc3, -0xf8,0xfb,0x68,0xe4,0x29,0x68,0x0d,0x02,0x29,0x5c,0xf6,0x39, -0xd0,0x23,0x6f,0xde,0xfd,0x9e,0x75,0x2c,0x28,0xbd,0x83,0xa3, -0x45,0x5f,0x69,0x69,0x5a,0x51,0x34,0xd0,0xc9,0x54,0x18,0x42, -0x00,0xd0,0x4d,0x19,0xc9,0x05,0xfb,0x0a,0x15,0x54,0x4c,0x40, -0x44,0x0a,0x42,0xae,0x9d,0xf0,0xac,0xc4,0x46,0xd8,0x06,0xbc, -0x3f,0x3e,0x7c,0xfb,0xe6,0x84,0x75,0x20,0xa4,0xc3,0xb8,0xa7, -0xd0,0x7f,0x0f,0x76,0x91,0x3c,0xac,0xa3,0x80,0xf4,0x61,0xcb, -0x8a,0x31,0x99,0x2c,0xe6,0xe3,0x5b,0x72,0x92,0x5d,0x5a,0xf3, -0x5d,0x3e,0xf2,0xb1,0x5c,0x72,0x4a,0x40,0x20,0x4b,0x07,0xa0, -0xa3,0x00,0x3f,0xe1,0xdd,0x34,0x6e,0x8d,0xec,0x72,0xd4,0xb9, -0x94,0xd1,0x5d,0x35,0x22,0xd6,0x44,0xb6,0xdb,0xb0,0xc6,0x3f, -0x3f,0x5b,0x3c,0x5b,0x4f,0x60,0xc7,0x8a,0x72,0xbb,0x8b,0x7e, -0xf6,0x0c,0x63,0x9e,0x45,0xf4,0x10,0x1b,0x80,0xab,0xd5,0xc1, -0x1f,0x6f,0x57,0x9c,0x0b,0x5b,0x67,0xcb,0x76,0x92,0xb0,0xc5, -0x04,0x8e,0x25,0xc3,0xc6,0xca,0x6b,0xbe,0x3e,0x07,0xaf,0x00, -0x45,0xf7,0xf8,0x7c,0x5a,0x1e,0x1e,0x29,0x53,0x02,0xdd,0xcd, -0x5a,0x6a,0xe2,0x18,0x3b,0xb0,0x14,0x6a,0xd3,0xb3,0x42,0x28, -0x4b,0x53,0x9e,0xa1,0x98,0x15,0x05,0x28,0xb5,0x43,0x7f,0x71, -0xc5,0xa2,0xc4,0xe6,0x94,0x9e,0x0b,0x6e,0x09,0x02,0xa9,0x8e, -0xa4,0x28,0xc3,0x31,0x62,0xe9,0x21,0xb5,0x5b,0x54,0x33,0x32, -0xe3,0x39,0x3d,0x7c,0x23,0x4b,0x2e,0x04,0x53,0xe7,0xec,0x58, -0xa4,0x7c,0x54,0x4a,0x51,0x3f,0x6c,0xe8,0x38,0x34,0x5d,0x42, -0x19,0x83,0xd8,0x84,0xb1,0x81,0x55,0x52,0x9e,0x26,0x09,0x72, -0xf1,0x20,0x00,0xd8,0x60,0x04,0x47,0x41,0x64,0xe2,0xde,0x2a, -0xe1,0xd5,0x2c,0x4b,0x98,0x98,0x8c,0x4f,0x2e,0x9d,0x4f,0xe6, -0xb9,0x1e,0xe8,0xe0,0x02,0x41,0x54,0x5a,0x57,0xdd,0x04,0x4c, -0x41,0xe6,0x38,0xb4,0x27,0x49,0x06,0x1f,0xa2,0x42,0xe6,0x03, -0x04,0xa0,0xd6,0x46,0x3c,0x72,0xed,0xc9,0x91,0x6b,0xcb,0x8c, -0xb7,0xfd,0x79,0x3e,0x9f,0x7f,0x06,0x40,0x4a,0xbe,0x43,0xb1, -0x74,0x16,0x64,0x41,0x87,0xaf,0x0c,0x08,0x47,0x1b,0xf7,0x16, -0x98,0x27,0x51,0x07,0xbc,0x96,0x03,0x6f,0x2d,0xd5,0x15,0x17, -0xe1,0xb6,0xeb,0x71,0xc1,0x8c,0xa6,0x9f,0x4f,0x2f,0xac,0x2f, -0x05,0x47,0xbc,0x3f,0xf2,0x31,0x69,0xac,0x12,0x3b,0xbe,0xe1, -0xe6,0x5c,0x39,0xd0,0xb9,0xb9,0x8d,0xcf,0x4d,0x2f,0x59,0xd2, -0xe9,0xfd,0xb2,0xd0,0xe4,0x3c,0xbe,0x6b,0x99,0xbb,0x6d,0xd1, -0x55,0x68,0x39,0x2b,0x3a,0x2d,0x33,0xdb,0x76,0x81,0x6d,0xb7, -0xb2,0xb4,0xb2,0x6c,0xb6,0xdd,0xd1,0x67,0xef,0x2d,0x94,0x36, -0x5b,0xd9,0x5d,0xab,0x70,0xd0,0x7f,0xb7,0x9d,0xdd,0x6d,0x87, -0x71,0x10,0xd9,0x42,0x81,0x75,0xbb,0xb3,0x95,0xde,0x6d,0xbf, -0xb8,0x6b,0x75,0x36,0x5a,0x69,0x2b,0x49,0x67,0xdb,0x2f,0x66, -0xfc,0xbd,0x7c,0xec,0x7e,0x06,0xd3,0x83,0x8e,0x0e,0x93,0x83, -0x24,0xe7,0x63,0x25,0x2c,0xd0,0x09,0x82,0xe9,0x69,0x10,0xec, -0x86,0xe4,0x02,0x02,0xc8,0x6d,0x58,0x5c,0x9b,0x0b,0xb6,0x10, -0xc5,0xe7,0x8b,0x1d,0x08,0xd4,0x72,0x43,0xb9,0xd7,0x65,0x58, -0x2c,0xac,0x6a,0xab,0x37,0x17,0x2a,0xb3,0xdd,0x03,0x81,0x6e, -0x8c,0x41,0x20,0xcd,0x7c,0x31,0x7f,0x88,0x0f,0x4b,0xdb,0x46, -0x75,0x0a,0xad,0x2b,0xbe,0xe6,0x55,0xd9,0x8c,0x8b,0x53,0xa9, -0xbc,0xff,0x41,0x2c,0x5a,0x09,0x4b,0xea,0xa2,0x4c,0xeb,0x91, -0x4a,0x4c,0x99,0x26,0x57,0x52,0x19,0x12,0x13,0x5e,0x72,0xf6, -0x08,0x58,0x7c,0x9e,0x90,0x4a,0x87,0x23,0xcf,0xa5,0x37,0xa4, -0x47,0x83,0xb5,0x3b,0xbe,0x55,0xce,0x94,0x83,0x03,0x61,0xe7, -0x47,0x19,0xf3,0x58,0xd7,0xdb,0x78,0x28,0xac,0x18,0x82,0x4f, -0xc7,0xf1,0x89,0xab,0xe7,0x0e,0x7f,0x86,0x93,0x3e,0x38,0xf0, -0xbc,0x7a,0x39,0x98,0xe4,0xa4,0xac,0x84,0xa6,0x72,0xf8,0x4b, -0x1d,0xe7,0x3a,0x2b,0x4b,0x3e,0x91,0xed,0x44,0x71,0x20,0xbb, -0x1f,0x0e,0x46,0x1f,0x53,0x17,0x86,0x15,0xc9,0x7d,0xa3,0x6a, -0x0d,0x69,0xb1,0x67,0xe5,0x28,0x18,0xcf,0x9d,0xdd,0xc8,0xd8, -0xeb,0x70,0x0f,0xdc,0x63,0x86,0xbb,0x24,0x54,0x7f,0x9f,0xa0, -0xc9,0xb3,0x89,0xf5,0x4b,0x6a,0x60,0x7b,0x3d,0xd1,0x0f,0x6d, -0x67,0xe3,0xab,0xab,0x61,0xfe,0x27,0x7c,0x84,0xb7,0x3b,0x9d, -0xc2,0xc6,0x9c,0x3e,0x0d,0xbd,0xca,0x23,0x8d,0x03,0x72,0x71, -0xfb,0xc7,0x9c,0x94,0x91,0x8a,0x71,0x59,0x39,0xaa,0x03,0x14, -0xd4,0x98,0x8e,0x7a,0x1f,0xf3,0xf7,0xb4,0x63,0x2b,0xe7,0x30, -0x90,0x03,0xd7,0x14,0xeb,0x97,0x16,0x1a,0x85,0x16,0x2a,0x77, -0x87,0x3d,0xa8,0xdf,0x22,0xd0,0x79,0xf3,0xee,0x4f,0x3b,0x6f, -0xdf,0xbc,0x7c,0xbf,0xfb,0x76,0xe7,0xf8,0x38,0x55,0xdf,0x49, -0xa3,0x02,0xb7,0x26,0x6c,0xbd,0x0a,0x00,0x74,0xf1,0x00,0x58, -0x25,0x80,0xa5,0x3d,0xa3,0x66,0x4b,0x07,0xfc,0x50,0x1d,0xbb, -0x30,0xb3,0x58,0x72,0x54,0x9c,0xa0,0x16,0xb8,0xed,0xd4,0x08, -0x4b,0x44,0xe8,0x9a,0x1d,0xa4,0x1c,0xea,0xf0,0xd9,0x64,0x68, -0xe4,0x4d,0xe3,0x2e,0x1e,0xbd,0xc3,0x4a,0x46,0x7e,0x0c,0xa6, -0x24,0x63,0x36,0xd7,0x29,0x40,0x36,0x6f,0x19,0xca,0x54,0x56, -0x53,0x48,0xc2,0x4d,0x6c,0x46,0xc3,0x23,0x56,0xb7,0xe4,0xfb, -0x0a,0x01,0x1b,0xce,0xe3,0x4c,0xba,0x71,0xf0,0x76,0x32,0x98, -0xce,0x06,0x6c,0xae,0x9c,0x63,0xa8,0x66,0x1f,0x14,0x54,0xb0, -0x84,0x6f,0x52,0x73,0x1d,0x7b,0x8b,0xeb,0x27,0x71,0xcd,0x79, -0x42,0xf7,0xdd,0x78,0x78,0xf4,0xe6,0xf8,0xe4,0xcd,0xbb,0x3d, -0xe9,0x32,0x53,0x20,0x13,0x7c,0xc9,0x7c,0x59,0x00,0xe5,0x6f, -0x59,0xa5,0x6d,0x78,0x75,0xcc,0x8d,0xe4,0x19,0x68,0xa3,0x6d, -0xdb,0xb9,0xc9,0x20,0xeb,0x10,0xa4,0xc2,0xe3,0xbc,0x20,0x4f, -0x22,0x87,0x59,0xa7,0x41,0x6c,0x37,0xb3,0x10,0xff,0xe7,0x52, -0xb0,0x11,0x52,0x7a,0x0c,0x21,0x5b,0x73,0x15,0xcc,0xcc,0x42, -0x45,0x0f,0x52,0xa4,0x66,0x55,0x91,0xcb,0x58,0x49,0x96,0x07, -0xd7,0xf3,0xf4,0x5f,0xc8,0x6c,0x9f,0x27,0x30,0xf2,0x9a,0x4e, -0xb2,0x00,0x60,0xc7,0x0e,0x7f,0x07,0x7d,0xe0,0xe3,0x71,0x21, -0x8b,0x61,0xf7,0xde,0x84,0xe5,0x32,0xb4,0x10,0x6b,0x3b,0xd1, -0xa7,0x9b,0x12,0xb8,0x40,0x99,0x20,0x80,0x6b,0x27,0xb4,0xeb, -0x02,0x24,0x74,0xc2,0x32,0x63,0xdc,0x4f,0x0b,0x25,0xba,0xac, -0x44,0x4a,0xa5,0x12,0xca,0x08,0x9d,0xaf,0xf1,0xe1,0x26,0xba, -0xc1,0xfc,0x25,0xb0,0x94,0x14,0x6b,0xe2,0xdf,0xd8,0x08,0xe6, -0x02,0x6a,0xfe,0x56,0x72,0x9b,0xe4,0x31,0xe2,0x4d,0xcc,0x0a, -0x5c,0xc9,0x2e,0x7a,0x91,0x34,0x4b,0xdd,0x1f,0xd0,0xfe,0x23, -0x83,0x40,0xf3,0xc1,0xde,0x0e,0x60,0x4f,0x04,0x0d,0x58,0x85, -0x3f,0x75,0x02,0xdf,0xfa,0x8c,0x2e,0x86,0x73,0x58,0xb4,0x0a, -0xdd,0x24,0x52,0x06,0x1b,0xee,0x58,0xd5,0x98,0x5f,0xac,0xed, -0x76,0x5d,0x69,0xa3,0x51,0x6a,0x61,0xeb,0x1f,0x68,0x21,0xb3, -0x8e,0xa4,0xfd,0x8b,0x37,0x1b,0xe5,0x86,0x32,0x18,0x85,0x56, -0x71,0x14,0xb6,0x96,0x8a,0xf0,0x5e,0x12,0x23,0xd2,0x26,0x5c, -0x2a,0x98,0x79,0x81,0x89,0x78,0xee,0xf2,0xf2,0xcd,0xd1,0xc9, -0x5f,0x2c,0x6b,0x09,0x38,0xdb,0x56,0x89,0xb3,0xb5,0x8a,0x6d, -0xa3,0xaa,0x63,0x3d,0x0b,0x0e,0x6d,0xe6,0x47,0xf0,0xd1,0xd5, -0xae,0x66,0x75,0x4f,0xe1,0xb4,0x34,0xf7,0xdd,0xc4,0x7c,0x80, -0xcf,0x28,0xec,0x62,0xba,0x20,0x57,0x26,0x26,0xf6,0xd0,0x6c, -0x51,0xe1,0xba,0x5d,0xdf,0x2a,0x71,0x8a,0xbc,0x79,0xbe,0xe3, -0x97,0xed,0xf2,0x5c,0x5a,0x42,0x77,0x62,0x03,0x96,0xdf,0x42, -0xf9,0x35,0x1d,0x36,0xb5,0x6f,0x70,0xff,0x43,0x6f,0x16,0xd9, -0x08,0x92,0x5e,0xff,0xcd,0x05,0x2e,0x53,0xd6,0x21,0xa4,0x72, -0x97,0xa0,0x6d,0x00,0x55,0xba,0xc2,0x73,0xab,0x21,0x9a,0x09, -0x4b,0xda,0x7e,0x17,0x81,0x60,0x61,0xa9,0x3a,0x99,0x0c,0x6e, -0x00,0xe1,0x93,0xc8,0x1d,0x35,0x67,0x78,0x52,0x6c,0xb5,0x94, -0xa8,0x62,0xa0,0xb0,0x81,0x98,0x87,0x5d,0x73,0x87,0xcf,0x53, -0xad,0xc9,0xab,0x6a,0xe6,0x02,0x48,0xd1,0xb6,0x8c,0xbd,0xad, -0x4f,0xc8,0xd0,0xb8,0xf3,0x6a,0x69,0xd5,0xe6,0xad,0x8e,0x7c, -0x92,0x58,0x54,0x01,0x8e,0xc4,0x79,0x33,0x5c,0xfe,0xf2,0x4f, -0x94,0x5e,0x59,0x2b,0xf8,0x6d,0x45,0xdb,0x25,0x03,0x32,0x2f, -0xfe,0x7a,0x40,0x33,0xda,0x19,0xc9,0x4d,0xac,0x32,0x2d,0x9b, -0xfb,0x02,0x91,0xb1,0xad,0xd1,0xf9,0x98,0x7f,0xe9,0x8f,0x3f, -0x8d,0x94,0x36,0xa0,0xe8,0xf8,0x8b,0xb6,0x09,0xeb,0xf7,0x37, -0xe1,0x6b,0x17,0x5f,0x45,0xfe,0x0e,0x37,0x3d,0x10,0x58,0x2c, -0xe0,0xcf,0x8b,0xd6,0xaf,0xeb,0xf5,0xd6,0xef,0x5e,0xb8,0x70, -0xf6,0xfc,0xdf,0xeb,0xf5,0xef,0xb7,0x5e,0x70,0x8e,0xa0,0x1d, -0x64,0x6b,0x4e,0x57,0x2c,0x2f,0x14,0x7c,0x47,0x98,0x8a,0x0e, -0xbc,0xed,0x41,0x22,0x8e,0xa0,0x2e,0x49,0x91,0xb5,0x0b,0xec, -0xc5,0xa0,0x8a,0x64,0xc9,0x43,0x04,0x53,0xb4,0x1f,0x76,0x9c, -0x26,0x15,0x4b,0xec,0x85,0xf1,0x4f,0x3a,0x51,0x94,0x16,0xe3, -0xec,0x7b,0x38,0xba,0x07,0xfa,0x13,0xf3,0x2f,0x10,0xc1,0xf9, -0x00,0x4a,0xa2,0x33,0xa1,0xb7,0x43,0xb9,0x90,0x17,0x36,0x97, -0x6b,0x37,0x3c,0x57,0x68,0x99,0xb4,0x38,0xbb,0xf4,0x9c,0x43, -0x07,0x2d,0x57,0x64,0xd4,0x14,0x4f,0xe5,0x38,0xae,0x13,0x7b, -0x8a,0xb3,0x3c,0x34,0x92,0x4a,0x23,0x32,0xe0,0xc2,0xf9,0xd2, -0x87,0xf3,0xc1,0x8a,0xe5,0xbd,0xf7,0x12,0xa1,0x4a,0x1a,0x4c, -0x0b,0x3e,0x89,0xf3,0xd7,0xbc,0x74,0xc1,0xfe,0xf3,0xd9,0x66, -0xdc,0xfc,0x2e,0x81,0xbf,0xa7,0x57,0x83,0x9b,0xee,0x77,0xc9, -0xfa,0xa6,0x3a,0x8d,0xe3,0x36,0xe3,0xc9,0xdf,0x51,0x7e,0xb5, -0xf7,0xf9,0xd6,0x1d,0x26,0x1b,0x77,0x8e,0x6c,0x8a,0x1d,0xf6, -0x80,0xaf,0x2c,0xea,0x2a,0x0b,0xd6,0xbe,0x77,0x4b,0xd2,0x47, -0x21,0xa8,0x71,0x39,0x38,0xff,0x60,0x8d,0x8e,0x91,0xc9,0x40, -0x89,0xa6,0xc3,0x8a,0x35,0x9f,0x67,0xb1,0x50,0x01,0xea,0x67, -0x79,0xb6,0x6b,0x7d,0xcd,0xb8,0x21,0x8c,0x12,0x08,0x8c,0x79, -0x40,0x22,0x13,0x41,0x23,0xc9,0x4e,0x34,0x5a,0x17,0xc0,0xd7, -0xb3,0xe7,0x79,0xad,0x27,0x8d,0xaf,0x9d,0xcf,0x67,0xb5,0x4f, -0xbd,0x29,0x5a,0x05,0x69,0xd6,0x44,0x7d,0x3a,0xad,0xdd,0x6f, -0x2f,0x23,0xa7,0xa0,0xe1,0x2b,0x35,0xd3,0x59,0x8f,0xac,0xa2, -0x9e,0xf4,0xae,0xec,0x26,0x28,0xd1,0xce,0x9e,0x83,0xce,0xc0, -0x02,0xf6,0x49,0x21,0x0f,0x30,0x1f,0x63,0xce,0x9c,0x11,0xc0, -0x62,0x27,0x59,0x9e,0xc6,0x4e,0x77,0x57,0x65,0x4a,0x8c,0x10, -0xec,0xfe,0x60,0x24,0x47,0x56,0x7c,0x69,0x62,0x83,0x74,0x92, -0x57,0xca,0x63,0x20,0x9d,0x4d,0x5d,0x3f,0x36,0xac,0x6b,0x21, -0x69,0xd7,0xeb,0x7c,0x14,0x2a,0x57,0x9a,0xae,0x9a,0x4a,0xf2, -0x76,0xa9,0x45,0xc2,0xad,0x24,0x72,0x9d,0xdb,0x4d,0x87,0x65, -0xbb,0xa2,0x17,0xca,0xc8,0x27,0x2b,0x3a,0xb5,0x22,0x27,0x99, -0x2b,0x92,0xfe,0xe8,0x7d,0x0e,0xfa,0xcc,0x06,0x83,0x3e,0x73, -0x79,0x60,0x4a,0x7c,0xfe,0x27,0xf4,0xd9,0x0b,0x57,0x4d,0x75, -0x9f,0xd9,0xd4,0xa7,0xf5,0x99,0xca,0xfd,0x48,0x9f,0x95,0x90, -0x5f,0xd9,0x67,0xe5,0x9c,0x25,0x45,0xac,0x6f,0x12,0x02,0x06, -0xac,0x44,0xfb,0x2d,0x65,0xab,0xe6,0x42,0x15,0x13,0x61,0x5b, -0x96,0x61,0xf7,0x3b,0x0b,0x70,0x18,0xb7,0x58,0xc8,0xbd,0xc9, -0xd1,0xde,0xef,0xf7,0xfe,0x7c,0xf8,0x28,0x87,0xb6,0x8a,0xe3, -0xd8,0xbb,0x5e,0x5c,0xea,0xe0,0xe2,0x9b,0x12,0x40,0x31,0xa4, -0xad,0xee,0xbf,0x44,0x24,0xa9,0x1c,0x2f,0x07,0x2e,0xe4,0xe3, -0xab,0x86,0x62,0x05,0xa7,0x0d,0xdb,0x87,0x4b,0x5d,0x14,0x35, -0x58,0x54,0x13,0x7e,0x00,0x74,0xef,0xf8,0x40,0xa6,0x70,0x73, -0x89,0xc8,0x00,0xbe,0x9a,0x8c,0xa1,0xcc,0x0b,0x6e,0xff,0x8a, -0xf9,0xfb,0xd4,0x79,0xfe,0x94,0x19,0xfe,0x94,0xb9,0x5d,0x31, -0xab,0x81,0x7a,0xdd,0x6c,0x2e,0xb7,0x1c,0x12,0x71,0x1a,0x7f, -0xdb,0x04,0xc6,0x99,0xbb,0x6a,0xce,0x3e,0x75,0xb6,0x3e,0x65, -0x9e,0x3e,0x65,0x86,0xaa,0x96,0xff,0x03,0xa4,0xb3,0x58,0xb8, -0xab,0xc0,0x27,0xce,0x81,0x07,0x84,0x94,0x55,0xb4,0xed,0x79, -0xc7,0x7c,0x32,0xfc,0x26,0xc6,0xf1,0xad,0x5c,0xa3,0x2d,0x8f, -0xa4,0x1e,0x15,0x5d,0x8a,0xbd,0xf2,0xd3,0xd1,0xdb,0xa7,0x32, -0x08,0x32,0x7b,0xf4,0x48,0xcf,0x70,0x9e,0x82,0xe8,0x56,0x3d, -0x6c,0x1a,0xdb,0x4a,0xae,0x17,0x64,0xf0,0x5d,0x9b,0xdf,0xf4, -0x06,0xff,0x82,0xce,0xa5,0x6a,0xbf,0xba,0x7b,0xf7,0xf6,0x77, -0xde,0x3c,0xb9,0x83,0xa9,0x8a,0x47,0xbb,0xd8,0xe6,0x7a,0x52, -0x27,0x87,0x58,0x57,0x76,0x73,0x21,0x8b,0xba,0x09,0xeb,0xf5, -0x07,0xe3,0x47,0xfb,0x0b,0x16,0x3b,0xb5,0xdf,0x66,0x51,0x82, -0x8f,0x1b,0xdd,0x81,0x28,0xc4,0xc1,0x9c,0x81,0xc8,0xc8,0x28, -0xd7,0x66,0xc1,0x9e,0x6a,0x38,0xb8,0xf8,0x18,0x99,0xf2,0xce, -0x07,0xad,0xdb,0x90,0xda,0x39,0x1a,0xaf,0xf9,0x8a,0x0d,0x2e, -0xe1,0xe1,0x76,0xb9,0xb6,0xdd,0xe5,0x0d,0x96,0x52,0x78,0x77, -0x25,0xda,0xe5,0xba,0x79,0x2d,0xcc,0xb2,0xc2,0x3e,0x6b,0xc9, -0x4b,0xd0,0xfa,0xf8,0x1c,0xba,0xf3,0x2e,0x8f,0x23,0xca,0x16, -0x05,0xb5,0xa9,0x0e,0x25,0x60,0xe7,0xe3,0xcf,0x4f,0xe8,0x53, -0x31,0x55,0x95,0xff,0xc9,0xe3,0x86,0x27,0x03,0x12,0x61,0x2e, -0x7b,0xb0,0xf7,0x0e,0xd2,0x5e,0xb9,0x98,0xb6,0xbb,0xd7,0x72, -0x00,0x12,0x32,0x18,0x60,0xa1,0x21,0x7d,0xb9,0x3c,0x1e,0x12, -0x66,0x52,0x70,0xe9,0xc4,0xf0,0xc1,0x21,0xfa,0x8a,0xe1,0x28, -0xf7,0xe8,0xc3,0xc3,0x52,0x31,0x02,0xa5,0xae,0xff,0xaa,0xc5, -0x40,0xc6,0x2f,0x73,0xbd,0xb0,0x7c,0x92,0xa0,0xa5,0x4b,0x77, -0x5c,0xd9,0xd4,0x77,0x93,0xe6,0xf9,0x17,0x78,0x14,0xe6,0xaf, -0xb2,0x58,0x81,0x2f,0x1f,0x86,0x0f,0x14,0xe9,0x86,0x02,0xb6, -0x6c,0x74,0x6e,0xc6,0xaa,0x77,0xa6,0x64,0x4b,0xe8,0xde,0xde, -0x51,0xa5,0x74,0x45,0x45,0xf7,0x5b,0xde,0x50,0x4e,0x89,0x66, -0xd4,0x55,0x94,0x00,0x26,0xdf,0x15,0xe2,0x77,0x88,0xbd,0x89, -0x24,0xf7,0xb1,0xc5,0x05,0xba,0x61,0x6d,0x6b,0xb1,0x90,0xe1, -0x23,0xe5,0xf5,0x5f,0x6d,0x43,0xa4,0xc3,0xd5,0xb9,0x41,0xaa, -0xd7,0xd7,0xe4,0x31,0x5a,0xe0,0x83,0x0d,0x32,0xe8,0x23,0x41, -0x89,0x34,0xee,0x34,0x50,0x6a,0x4c,0x24,0x3b,0x1b,0x53,0x92, -0x48,0x7d,0xd1,0xee,0x41,0x50,0xd7,0x11,0x96,0x3c,0xa1,0x74, -0xda,0xe5,0x10,0xc7,0x77,0x44,0x81,0x5c,0x65,0xd5,0xfa,0x7c, -0xb6,0xe6,0x22,0x1c,0x97,0xf0,0x04,0x20,0xab,0xb2,0xdc,0x7b, -0xdf,0x6a,0xaa,0x9c,0xbd,0x72,0x95,0x08,0xd1,0xae,0xad,0x45, -0x49,0xe0,0xa0,0xcd,0xe7,0x67,0x15,0x05,0x86,0x8d,0xf7,0x55, -0x05,0x6d,0x1e,0x9b,0x51,0x29,0xf4,0x18,0xf4,0xe4,0x5c,0xaf, -0x4b,0x19,0xa6,0xcf,0x8f,0x34,0x69,0x24,0xc6,0x55,0xb9,0x2c, -0x20,0xa3,0xfc,0x48,0x79,0xdb,0x5f,0x6d,0x19,0x6f,0x76,0xf0, -0x84,0xdd,0xc3,0xc6,0xb8,0x4d,0x99,0x64,0xd8,0xc8,0x6e,0xc8, -0xd9,0x08,0x76,0x30,0xfd,0x2b,0x48,0x4d,0x9f,0x66,0xf8,0x1a, -0xc8,0xa3,0xb7,0x23,0x79,0x7c,0x36,0x41,0x17,0x47,0x01,0x3e, -0x11,0xd3,0x61,0xf8,0xb6,0x17,0x22,0x90,0x84,0xf8,0x4b,0x44, -0xdd,0x71,0x3f,0x6b,0xd5,0x39,0xa6,0xdd,0xaa,0x43,0x10,0x9f, -0xdb,0xd9,0x3c,0x78,0x32,0x34,0xee,0x2b,0x52,0xee,0x38,0x02, -0x58,0x85,0x5a,0xaa,0x49,0x6d,0x55,0x26,0x71,0x9f,0xce,0x83, -0xaa,0x8c,0xce,0x86,0xf7,0x6d,0x24,0x26,0xb8,0x97,0xb9,0x9e, -0x9d,0x59,0x0d,0xcf,0xa2,0xda,0x81,0xc4,0x77,0x44,0xcd,0x33, -0xd0,0xd2,0x48,0x45,0x27,0xd0,0xbf,0xc0,0xfd,0x06,0x78,0xca, -0xa0,0x8e,0x83,0x77,0xd3,0x1b,0xc1,0x84,0x7e,0x5b,0x81,0x65, -0x05,0x40,0x80,0xe5,0x55,0xbe,0xc8,0xee,0xd0,0xe6,0x95,0x3a, -0xa9,0xbf,0x76,0x45,0xb8,0x40,0x13,0x2d,0x5f,0xee,0x5e,0xf7, -0x26,0x78,0xe2,0x1a,0x3f,0xdf,0x5e,0x5c,0x5c,0x93,0xf6,0x03, -0x06,0x59,0x51,0x62,0x09,0x88,0x58,0x1c,0x7e,0xaa,0x68,0xd9, -0xa3,0x38,0xf4,0x36,0xfe,0xfe,0x35,0x38,0x6c,0x3c,0x7f,0x5e, -0xaf,0x46,0xa2,0x1d,0x0d,0x90,0x14,0xa3,0x37,0x51,0xd8,0xf1, -0xe4,0x13,0xca,0x76,0x4e,0xa1,0xb3,0xd4,0x60,0x14,0x9a,0xc0, -0x22,0x22,0x9a,0x62,0x30,0x1f,0x7e,0x79,0x3b,0x98,0xc1,0xcf, -0x7f,0xce,0xf3,0xc9,0x17,0x23,0x6f,0x7a,0xf7,0xc7,0xfd,0xf9, -0x30,0x37,0x23,0xc0,0x01,0x4d,0x69,0xbd,0x37,0xa4,0x28,0x06, -0x7c,0x80,0x35,0xc6,0x0c,0x4e,0x6e,0x0c,0xe1,0xaf,0x71,0xa6, -0x8c,0x4b,0xf4,0x66,0x53,0x0c,0x1d,0x70,0x69,0x7f,0xd3,0xa3, -0x2b,0x74,0x4e,0xca,0x95,0x65,0xd6,0xe0,0x81,0x84,0x4d,0x18, -0x44,0xaf,0x56,0x41,0x44,0x76,0xbf,0x04,0x5e,0x3d,0x1f,0xf4, -0xb3,0xd3,0x68,0x2b,0x32,0xf2,0xaf,0xdb,0xc6,0xe6,0x88,0xc5, -0x2c,0xfc,0x5c,0x61,0x2e,0xcb,0x19,0xb9,0xaa,0x36,0x90,0xc5, -0x2f,0xb8,0xc8,0xc2,0x0e,0xce,0x4e,0x0f,0x12,0x1a,0x82,0xfa, -0x38,0x67,0x9b,0xcd,0xef,0xda,0xb5,0xc9,0x5d,0xfa,0x2d,0xd0, -0x13,0x83,0xae,0x4e,0x9b,0xf2,0xbc,0x9f,0x6e,0xfd,0x44,0xc7, -0x47,0xc5,0x89,0xd9,0xcf,0xc1,0x4d,0x85,0xcd,0x3e,0xa1,0x19, -0xd5,0xc3,0x90,0xaf,0xb3,0x52,0xb6,0x17,0x9a,0x14,0x11,0x9e, -0x8f,0xba,0xe8,0x6e,0x46,0xd4,0x26,0x97,0xe9,0x57,0x15,0x75, -0x64,0xfd,0xf3,0xd9,0xf4,0xbb,0xcd,0x50,0xc1,0x12,0x62,0xd6, -0x29,0x4a,0x20,0xe3,0xab,0x50,0x47,0x3e,0xd9,0xef,0x5e,0x60, -0x4f,0x7a,0x44,0xed,0xa1,0xac,0xb3,0x45,0xc8,0x61,0x77,0xd5, -0x64,0x4b,0x76,0x24,0x22,0xf5,0x02,0x96,0x13,0xf2,0x88,0xf9, -0xbd,0x23,0xd6,0x1c,0xa1,0x61,0x45,0x98,0x1e,0xa5,0xa4,0x8e, -0x7f,0x46,0x50,0x4a,0x43,0x3d,0xfe,0x46,0xb1,0xc2,0x24,0x2d, -0xc6,0xa8,0x3e,0x5a,0x81,0x75,0x09,0x5b,0x17,0xa1,0xd0,0xf6, -0xf0,0x68,0x7c,0x8f,0xdf,0xed,0xfc,0x71,0xef,0xfd,0xee,0xce, -0xf1,0x9e,0x6c,0xb6,0x32,0xc7,0xaf,0xc4,0x10,0x65,0x06,0x42, -0xd2,0x70,0x98,0x46,0xad,0xe6,0x76,0x73,0x6b,0x63,0x72,0xd1, -0xdc,0x46,0x8b,0xbb,0x1f,0xc6,0x93,0xb4,0x85,0x27,0x44,0xf0, -0xbb,0x6d,0xfa,0xe3,0x59,0x8a,0x3a,0x04,0x52,0x5b,0x74,0xde, -0x9b,0xe4,0xd7,0xbd,0x51,0x7f,0x03,0x28,0xf0,0x86,0x5e,0x79, -0xe1,0x29,0x78,0xb4,0x84,0x49,0xbe,0x8b,0x0e,0xc3,0xb2,0x3f, -0xfc,0x27,0x4e,0xf6,0xe6,0x05,0x05,0xee,0x31,0x9e,0x2c,0x00, -0x4a,0x74,0xfe,0xf9,0x16,0x0a,0x8f,0x51,0xdc,0xdb,0x88,0x1a, -0xb1,0xd5,0x3a,0x4e,0xec,0xd3,0x88,0x38,0x81,0x02,0x6f,0x60, -0x41,0x43,0xd1,0x89,0x2e,0x8e,0xec,0x8d,0x10,0xba,0xcf,0xe5, -0x39,0xea,0x2c,0x9d,0x14,0xb3,0x94,0x06,0x9f,0x8d,0x7b,0x5f, -0x8e,0xfc,0xad,0x51,0xb1,0x48,0x2c,0x39,0x70,0xef,0x58,0x1e, -0x86,0x8a,0xf2,0xb3,0x19,0x34,0x4c,0x6e,0xb4,0xd0,0x66,0x95, -0xcd,0xb1,0x34,0xbc,0x6a,0x3e,0x86,0x73,0x45,0xae,0x27,0xa0, -0x5d,0x51,0xea,0x6b,0x31,0xef,0xe7,0x2b,0x31,0x3f,0x3e,0xdc, -0xdb,0x7d,0xb3,0xf3,0xf6,0xfd,0xee,0xeb,0x9d,0xa3,0x63,0x47, -0x2c,0xf1,0xe9,0x59,0x7a,0xb6,0x71,0xf6,0xbe,0xdb,0x88,0x9b, -0x09,0x6a,0x3b,0xef,0x1f,0xfc,0xf5,0xfd,0xeb,0x9d,0xdd,0x3f, -0xba,0x1c,0x3f,0xdf,0x8c,0xff,0x1e,0x5b,0xdb,0x7b,0xcc,0xe4, -0x0b,0x3c,0x98,0x23,0x81,0x0f,0xf3,0xf0,0x1f,0x3a,0x33,0xf8, -0x42,0x0e,0xca,0xd3,0x39,0xb9,0x74,0xf7,0xed,0x40,0xd4,0xfd, -0xcd,0xf3,0x64,0x70,0x75,0x45,0x1a,0xcf,0xe4,0x78,0x0e,0x77, -0x65,0xf8,0x4b,0xb7,0xe6,0x64,0x20,0x8a,0x1f,0x73,0x4f,0xd8, -0x3b,0x40,0x84,0x1a,0x76,0xa8,0x62,0x83,0x27,0xb6,0xaa,0xe3, -0x7b,0x68,0x4d,0x0e,0x9d,0xa1,0x4f,0x99,0xd6,0xd0,0x3e,0x96, -0x40,0x4e,0x52,0x76,0xe2,0x84,0xbb,0xba,0x97,0x07,0xfb,0xbb, -0x6c,0x42,0xfd,0xed,0xb8,0xd7,0xcf,0xfb,0x91,0xb1,0x99,0xa4, -0x15,0xd6,0x4c,0x0c,0xe5,0x46,0x53,0xd7,0x3e,0x07,0x74,0xa6, -0x5d,0x90,0x8a,0x9e,0x3f,0x79,0xbb,0x55,0x16,0x73,0x49,0x31, -0xcc,0x0f,0x1f,0xe9,0xc3,0x59,0x15,0x8a,0x08,0xb8,0x07,0x49, -0x8a,0xa7,0x72,0xc2,0x2b,0x02,0xae,0x01,0x09,0xb7,0x11,0x75, -0x61,0xd6,0xe5,0xbf,0x28,0x5f,0xf0,0x2c,0x13,0x3a,0x95,0x6e, -0x08,0xa1,0x6e,0x36,0x8f,0x01,0xd5,0xc3,0x7e,0xa4,0xba,0x49, -0xaa,0xe3,0xa8,0xe1,0xa2,0x05,0x2d,0xe9,0x4b,0x71,0xf5,0x02, -0x33,0x1f,0xf1,0x48,0x69,0x0d,0x46,0x75,0xc6,0x14,0x97,0x67, -0xf8,0x35,0xec,0x1a,0x86,0x82,0xf4,0xb5,0x34,0x3f,0x1e,0x1c, -0xbc,0xdd,0xdb,0x79,0xf7,0x7e,0xe7,0xe4,0xe4,0x08,0x0d,0x5b, -0xd9,0xf6,0x45,0x37,0xf3,0xe1,0x6c,0x00,0xe3,0x21,0x1b,0xbf, -0x1c,0x7d,0x93,0xd3,0xfe,0xd5,0xf4,0x07,0x53,0xb2,0xd6,0x6a, -0x70,0x6c,0x0e,0x46,0xc3,0x2f,0x86,0x6d,0xd6,0x43,0x0c,0x30, -0xd0,0x51,0x24,0xaf,0x48,0x61,0xed,0x4d,0x4a,0x5a,0xfd,0xba, -0xba,0x53,0xbf,0x38,0x72,0x6a,0x57,0x54,0x17,0x58,0x06,0xb1, -0x59,0xf7,0xde,0xee,0xed,0xef,0xbd,0x3b,0x39,0x0e,0xb0,0x63, -0x95,0x79,0x46,0x0d,0x6a,0x25,0x59,0x1f,0x0f,0xc5,0x80,0xcd, -0xf5,0xcc,0xf9,0x7c,0x36,0x83,0x30,0x6e,0xa8,0x0d,0xce,0x1f, -0x74,0x3d,0xf2,0x14,0xa4,0x6c,0x4d,0xa7,0x7e,0x6d,0xb0,0x88, -0xad,0x6d,0x2d,0x13,0xa7,0xea,0x71,0xdf,0xef,0xcd,0x7a,0x29, -0x53,0x15,0xb0,0xc1,0x9e,0x19,0x8c,0xae,0xf3,0x09,0x04,0xfa, -0x2f,0x7d,0xc2,0x1b,0x1d,0xc7,0x6e,0xba,0x56,0xaf,0x17,0x15, -0x45,0x1c,0x4b,0x40,0xcb,0xcb,0x68,0x7c,0x2e,0x71,0x6a,0x82, -0xc3,0x7c,0x22,0x95,0x78,0x95,0x4e,0x63,0x0d,0x03,0x7d,0x6b, -0x25,0xce,0x28,0x91,0xe3,0x89,0x68,0x33,0xab,0xcc,0xa1,0xd8, -0x1c,0x51,0xc8,0xe1,0xbc,0x75,0x2d,0x4a,0x5d,0xe2,0xeb,0x6d, -0xda,0x87,0x08,0x9a,0xaf,0x25,0x68,0x2e,0x20,0xaa,0x12,0xa2, -0xbd,0x9a,0x17,0xb7,0x5a,0x99,0xb7,0xde,0x4a,0x10,0xe9,0x5d, -0x4b,0xa0,0x08,0xe3,0x66,0xa7,0x9d,0x98,0xbf,0x75,0x06,0xc4, -0x20,0xd2,0xad,0xf7,0xd6,0xb1,0x13,0x5b,0xcd,0xac,0x8a,0x95, -0xbd,0xa3,0xbd,0x27,0x92,0xf2,0x51,0x6f,0x3e,0x1b,0x47,0x09, -0x1d,0x5f,0x62,0xf4,0x62,0xe1,0xa4,0x04,0x55,0x28,0xac,0xd3, -0x82,0xe8,0xb8,0x9d,0x2a,0x4a,0x3c,0x04,0x62,0x59,0x51,0x5a, -0x88,0x9d,0xb6,0xa7,0x0f,0x77,0x4a,0xb0,0x1b,0xeb,0x57,0x5a, -0xe6,0x85,0x7e,0xab,0x9e,0x5a,0x94,0xbd,0x6b,0x9f,0x95,0x14, -0xfa,0x30,0x94,0x56,0xac,0x4b,0xab,0x58,0x1f,0x77,0xd2,0xb0, -0x4e,0x71,0x9d,0x47,0x40,0xfd,0x37,0xb3,0xfc,0x86,0xab,0x64, -0xe7,0xfc,0xf8,0x64,0x3c,0xbf,0x18,0x5c,0x0e,0xf2,0x7e,0x27, -0xac,0x33,0xf5,0xfb,0x75,0xb9,0x7e,0x0a,0xaa,0xc1,0x45,0xc0, -0x75,0x8a,0xb6,0xcf,0xc6,0x6e,0xfe,0x02,0x50,0x78,0x2f,0x57, -0x28,0xdc,0x32,0xab,0x28,0xb0,0x58,0x94,0x95,0xec,0xe8,0x28, -0xa3,0xd0,0xfa,0xd5,0xd5,0x73,0x86,0xb6,0x2d,0x69,0x33,0x5e, -0xa9,0x8c,0x3c,0x2a,0xd0,0x81,0x59,0x55,0x2a,0x83,0xd9,0xf6, -0x5e,0x3a,0x58,0x51,0x0b,0x42,0x8a,0x38,0x20,0xb4,0x44,0x87, -0x7a,0xe3,0xdb,0x87,0x87,0xdf,0x49,0xde,0x21,0xfe,0xc5,0x0e, -0xa5,0x58,0x73,0x87,0xe6,0x80,0xb7,0x9c,0xb0,0xca,0x69,0x4b, -0xe2,0x8d,0x7a,0x69,0x73,0x6b,0x34,0x0a,0x70,0x90,0xe6,0x2a, -0x56,0x24,0x87,0x65,0xf0,0x44,0x20,0x7b,0x77,0xf0,0x72,0xef, -0xfd,0xc9,0x5f,0x0e,0xe1,0xcf,0xde,0x9f,0x4f,0xde,0x1f,0x1e, -0x1d,0x1c,0xee,0x1d,0x9d,0xfc,0xe5,0x54,0x0b,0xb5,0x78,0x08, -0xdc,0x6d,0xaf,0xd4,0x5b,0xeb,0x58,0x60,0x56,0x12,0x3e,0xb5, -0x11,0xdd,0x34,0x8a,0xfc,0xf0,0xba,0xd8,0xb0,0x39,0x24,0x29, -0xac,0x40,0x43,0x2d,0xce,0xb2,0x99,0x88,0x57,0x21,0xdc,0xea, -0x66,0xb0,0x72,0x80,0x14,0x76,0x42,0xde,0x68,0x56,0x65,0x7b, -0x0e,0xd9,0xb0,0x4d,0x74,0x2e,0x0a,0xfb,0x96,0x07,0xc0,0x3d, -0x04,0x02,0xdb,0x22,0x22,0x49,0x64,0xa4,0x9f,0x9b,0xeb,0xfd, -0x3b,0x7c,0x76,0x28,0xc1,0x65,0x4c,0xec,0xa1,0x3c,0xfe,0x9e, -0x1d,0x96,0xfb,0x92,0x4d,0xfc,0x1c,0xc3,0x6a,0xb5,0x7b,0x82, -0x2c,0xc7,0x6d,0xa6,0x1c,0xc7,0xf7,0x9c,0xce,0xae,0xe2,0x96, -0x5c,0xa7,0x10,0xae,0x90,0x66,0xdc,0xf1,0x38,0xad,0xa4,0x4a, -0xb0,0xe1,0x88,0xe4,0x9e,0x7f,0x9b,0x56,0x16,0xc0,0x63,0x53, -0x04,0xc5,0xf2,0x8e,0x24,0xde,0xb1,0x1a,0xa2,0x84,0xd8,0xe0, -0x17,0xbf,0x29,0x97,0xcc,0xa2,0x41,0x41,0xf7,0xf0,0x1c,0xb5, -0x2c,0xec,0x97,0x98,0x25,0x06,0x21,0xcb,0x26,0xd1,0x74,0xfe, -0x57,0xf6,0x53,0x01,0x38,0x8d,0x3c,0xee,0x07,0xb5,0xf5,0x08, -0x43,0x0e,0xf0,0xd0,0x14,0xe3,0xd4,0x2f,0x1a,0x2e,0xaa,0x3d, -0xf8,0xc1,0x07,0xf4,0x73,0x12,0x59,0xfc,0xf3,0xde,0x70,0x7c, -0x11,0xfb,0x2c,0xf4,0xa2,0xa4,0x54,0x9f,0x34,0x61,0x69,0x94, -0x98,0x2c,0xeb,0x68,0x51,0xa0,0x96,0xb9,0xac,0xde,0x9c,0x5d, -0xb5,0xd0,0x89,0xe1,0xb6,0xf5,0x31,0x86,0xc6,0x9a,0xa0,0xb5, -0xf1,0x76,0x96,0x5d,0x8e,0xdc,0x13,0xba,0xcb,0xd1,0x5a,0x96, -0x85,0x6b,0x6d,0x10,0xe9,0xe5,0x84,0x0e,0x42,0x4c,0x09,0xa2, -0xf6,0xa5,0x23,0x5d,0x28,0x87,0xbd,0x98,0x27,0xe1,0x37,0x6b, -0xfc,0x9c,0x46,0x09,0x9b,0xd6,0x60,0xd7,0xe5,0x28,0xb3,0xd0, -0x51,0x92,0x48,0x40,0xa2,0x67,0x59,0x95,0x9c,0x2e,0xb6,0x84, -0x7b,0x22,0x08,0x31,0x98,0x45,0xb1,0x2a,0x17,0x44,0x53,0x4e, -0x36,0x47,0xa5,0x7d,0xf7,0x3b,0x43,0x4f,0x4c,0x00,0xd0,0x4e, -0x98,0x35,0xe6,0xc3,0x07,0x7b,0x77,0xe5,0xb0,0xee,0x38,0x6b, -0x2a,0x0a,0x41,0x83,0x4f,0xc0,0x54,0x90,0x2c,0x44,0x7d,0x78, -0xf1,0xc1,0xdb,0x85,0x72,0x73,0x3b,0xb3,0xe8,0x7c,0x60,0xa4, -0xb9,0xa7,0xdb,0x8a,0xee,0xf8,0x7c,0xa3,0xe1,0x4a,0xa4,0xee, -0x6b,0x19,0x78,0x1e,0x58,0xdd,0x55,0x85,0x7e,0x91,0x2a,0x74, -0x6b,0xb5,0x64,0xc9,0xab,0x99,0x12,0x23,0x8f,0x5c,0x04,0x88, -0xb2,0x48,0x6d,0xa9,0x26,0x3d,0xa3,0x1c,0xb6,0xd6,0xc6,0xa3, -0x57,0xa5,0x8d,0xa4,0x99,0x8f,0xc4,0xb1,0x85,0x1b,0x63,0xbb, -0x8a,0xe8,0x14,0xd1,0xc3,0xd3,0x9b,0x41,0xdc,0x6d,0x02,0xae, -0xe8,0x3e,0x83,0xa3,0xff,0x0d,0x17,0x8e,0xb2,0xbf,0x0c,0xf4, -0x64,0xf1,0x37,0x7b,0x4a,0xfd,0xb7,0xda,0x78,0x52,0xfb,0x1b, -0x3b,0xf6,0x04,0x94,0xff,0x86,0x36,0x43,0x80,0xa0,0xf1,0x52, -0x2b,0x92,0xcb,0x67,0x4c,0x9b,0x0a,0xe1,0xec,0xf1,0xb1,0xc2, -0x31,0x94,0xcc,0xbd,0x28,0xca,0x59,0x22,0x34,0x7e,0x8d,0x96, -0x81,0x1f,0xcc,0xcb,0x59,0xac,0x75,0xea,0xe9,0x62,0xf1,0x04, -0xc0,0x46,0x70,0xb8,0x5f,0xda,0x2a,0x1e,0x2e,0x26,0x75,0x58, -0x74,0xc4,0x50,0x2b,0xc2,0x70,0xa6,0x8b,0x25,0x2b,0x03,0x4e, -0xfc,0x78,0xd2,0x49,0x9c,0xec,0x3c,0x6a,0x7a,0xe7,0x41,0xde, -0xa6,0xee,0x5d,0x87,0xbc,0x1a,0x4d,0x59,0xcb,0x77,0xca,0xae, -0xa7,0x48,0x71,0xd2,0xa6,0x30,0xbf,0xbf,0x19,0xcf,0xa7,0x79, -0x8e,0xce,0x49,0xc4,0x44,0x21,0x3e,0x50,0xc7,0xb8,0x61,0xde, -0xbb,0xcb,0x25,0xce,0xf9,0x72,0x83,0x4d,0x0f,0x80,0x0c,0x6d, -0xa1,0xdb,0x68,0x65,0x6e,0x5c,0xe2,0xe9,0xb1,0xdd,0x4b,0x89, -0x3c,0x1c,0xb3,0xe9,0xec,0xc2,0xfb,0x4f,0x7a,0x68,0x87,0x7a, -0xca,0xd9,0xef,0x60,0xe2,0xf7,0xdc,0xa2,0xdf,0xe9,0xb9,0x53, -0x12,0xab,0x7d,0x89,0x3b,0xb0,0xdb,0xec,0xbc,0x5e,0x3f,0x97, -0x77,0x52,0xc8,0x1b,0xdd,0x43,0x3b,0x28,0x0d,0xc9,0x8b,0xc5, -0x5a,0xbc,0x46,0xbf,0x68,0x7e,0x0a,0x3e,0x1c,0x40,0x4c,0xa1, -0x8a,0x1c,0xc2,0x9d,0x30,0x18,0x43,0xee,0x24,0xed,0xad,0xc2, -0xbb,0x5e,0x6f,0xfd,0xa6,0xbe,0x32,0x95,0x0a,0x27,0xfa,0xe8, -0x85,0x5a,0x87,0x8e,0x03,0xd8,0xaf,0xe9,0x79,0x16,0x60,0x8d, -0xac,0xee,0x1c,0x1b,0x5c,0xb6,0x46,0xb8,0x6c,0xeb,0x21,0xb3, -0xe7,0xb3,0x14,0x77,0xd3,0xbb,0xcd,0xee,0xfd,0xe8,0xa4,0x3c, -0x52,0x63,0xd4,0x97,0xf6,0xe3,0x68,0x63,0xef,0x60,0x48,0x97, -0xed,0x60,0xf6,0x5a,0x20,0x0c,0xba,0x52,0x31,0x5c,0xbb,0xe2, -0x9d,0xe4,0x43,0x34,0xac,0x23,0x8a,0xe2,0x12,0x3a,0xa1,0x0c, -0xed,0x78,0x4d,0xc2,0xa8,0xe0,0x4c,0x1f,0xda,0x65,0xab,0xeb, -0x53,0xf1,0xca,0x2b,0x59,0xf0,0x21,0x28,0x93,0x3b,0x57,0xc9, -0x3e,0xe7,0x96,0x22,0x4c,0x97,0xcf,0xf8,0x42,0xb6,0xc3,0x25, -0x13,0x53,0xec,0x9e,0x4a,0x52,0x5f,0xda,0x58,0xb9,0xb7,0x1e, -0x61,0x35,0x68,0x0b,0x47,0x18,0xdd,0xc1,0xe5,0xa5,0x91,0xb3, -0xe4,0xff,0x1a,0xcc,0xae,0xcb,0x22,0x80,0x24,0xe2,0x68,0xc9, -0x22,0x49,0xd7,0x6d,0xe2,0x87,0x3e,0x7c,0xb0,0x47,0x43,0x1a, -0xac,0xdd,0x56,0x64,0x72,0x53,0x16,0xcf,0x36,0xe5,0xc0,0x48, -0x43,0xf6,0x63,0x30,0xa2,0x8a,0xa8,0x92,0x8e,0x78,0xeb,0x47, -0x5f,0x6a,0x93,0xd9,0x8f,0xf9,0x25,0xf2,0x0d,0x4c,0x37,0x94, -0xdc,0xd4,0x0e,0x70,0x53,0xc9,0x2b,0x40,0x77,0x51,0x66,0xe0, -0xbc,0x0e,0x05,0x2a,0x44,0xe2,0x1c,0x76,0x01,0x49,0x15,0x50, -0xa4,0x62,0x43,0x4f,0x93,0x5c,0x92,0x1f,0x90,0xe6,0xbc,0x5c, -0x62,0xca,0x30,0xf0,0x15,0x41,0x51,0x80,0xaf,0xd7,0x2d,0x54, -0x51,0xa8,0x91,0xcc,0x78,0x55,0x2b,0x09,0x7c,0x08,0x81,0xc3, -0xf7,0xf8,0xe9,0xb7,0xaf,0x1f,0xef,0x39,0x60,0xa7,0x7b,0x7b, -0x9b,0x8f,0xfa,0x15,0x9b,0x1d,0xea,0xd1,0x8a,0x01,0x18,0x85, -0x3d,0x4f,0xf0,0x92,0xfb,0xb8,0x0a,0x75,0xe0,0x23,0x55,0xd1, -0x5a,0x79,0x87,0x6a,0xe7,0x7e,0x67,0x48,0xd8,0xcb,0xb7,0x30, -0x1e,0x0f,0x20,0xc5,0x06,0x70,0xcb,0x60,0x3d,0xa1,0xb9,0xb4, -0xcb,0xc1,0x64,0x3a,0x23,0xf0,0xed,0x27,0x37,0xc5,0x8b,0x8a, -0x8a,0x80,0x28,0x8d,0x89,0x81,0xae,0x6b,0x0d,0xba,0x0d,0x2c, -0x23,0x88,0xb1,0x4c,0xf4,0xf6,0x2b,0x93,0xf3,0x41,0x97,0x82, -0xd6,0xda,0xc4,0xc0,0xf7,0x8a,0xb9,0xc0,0x9f,0x64,0x90,0xbb, -0x44,0x9c,0x16,0x8c,0x27,0x50,0x1b,0x13,0xf4,0xa5,0x23,0x12, -0x39,0x36,0xaa,0xa0,0x8b,0xca,0x19,0xf7,0xb5,0x98,0xd1,0xe5, -0x76,0xa1,0xc6,0xde,0x25,0xf2,0xd1,0xf2,0xd8,0xe5,0x9f,0xf6, -0xf4,0x5c,0x09,0x06,0xea,0x01,0xce,0x50,0x35,0x6e,0x1e,0x54, -0x91,0x05,0x7c,0xc5,0xe4,0x2f,0x4c,0x6d,0x7b,0x97,0x2f,0x2c, -0x6e,0x47,0x82,0x46,0xdd,0xe0,0x07,0xe2,0x1f,0xa7,0xf2,0xc3, -0x3f,0x4e,0x2d,0x35,0xd9,0x4a,0x60,0x38,0x3d,0xfb,0xec,0x1e, -0x25,0x50,0x67,0xf3,0xd1,0x68,0x3a,0xd4,0x06,0xb2,0xb5,0x70, -0xd3,0x50,0x02,0x07,0xb2,0x8c,0xcf,0xdd,0x09,0xd1,0x2d,0xa3, -0x98,0x94,0xcb,0x2f,0xa5,0xbb,0x57,0x70,0xb1,0xd5,0x63,0xe1, -0x8d,0xce,0x11,0x19,0xb4,0x50,0x4c,0x90,0x1e,0x77,0xa2,0x87, -0x80,0xa6,0x57,0x55,0xa4,0x90,0x57,0x51,0x8b,0x3a,0xbf,0xc1, -0x1c,0x32,0x98,0x76,0x60,0x8a,0xf7,0x75,0xa5,0x1c,0x6e,0xaf, -0x98,0x0f,0xfd,0xeb,0x6f,0x35,0xb4,0x6d,0xac,0x1c,0xef,0x17, -0x6f,0x00,0xc7,0x35,0xe4,0x14,0x37,0x0e,0xbf,0x76,0xc2,0x85, -0x6e,0x82,0x02,0xae,0xca,0x1b,0xd8,0x10,0x0e,0xaa,0x58,0x4f, -0x49,0xd5,0x4a,0x1d,0x31,0x09,0x76,0xd3,0x1f,0xbf,0x9c,0xf4, -0xae,0x70,0xd7,0x1f,0xab,0xae,0xbe,0x18,0x8e,0x47,0xb9,0x3d, -0x1a,0xc6,0x6f,0x7b,0xc3,0x21,0x82,0x6a,0xb9,0x2a,0x5a,0x7c, -0x11,0x8c,0x71,0x42,0x7b,0x41,0x2c,0x8d,0x9f,0x22,0xaa,0x93, -0x5b,0xa8,0x53,0x07,0x4c,0x96,0x7a,0x84,0x96,0xb9,0x2f,0xc8, -0x53,0xf0,0xf9,0x8f,0x07,0x88,0x66,0x3a,0x1b,0xdf,0xe2,0x89, -0x4f,0xef,0x8a,0x1e,0xa6,0x52,0xe4,0xd2,0xbf,0xa8,0x16,0x34, -0x4c,0x78,0xc3,0x34,0x62,0x4b,0x54,0x41,0x2b,0x08,0x77,0xe2, -0x97,0xff,0xc0,0x36,0x1b,0xff,0x3c,0xf7,0xd6,0x5a,0xf8,0x10, -0xaa,0x7a,0x63,0x57,0xda,0x8c,0xc6,0x77,0xc1,0x9e,0x52,0x6f, -0xf8,0x18,0x2c,0x9f,0x56,0x84,0x6e,0x0c,0xa5,0x90,0x70,0x6e, -0x39,0xb8,0x48,0x3c,0x63,0xa0,0xe5,0x53,0x6c,0xed,0xad,0x86, -0xac,0x9f,0x30,0x85,0x48,0x61,0x01,0xb9,0x78,0x37,0xc5,0x5e, -0x68,0x9e,0x03,0xf9,0x95,0xee,0xf4,0x9a,0xe3,0x51,0x39,0xe7, -0x7c,0xb4,0x22,0xef,0xe5,0x25,0x08,0x07,0xc0,0x3e,0xae,0xf7, -0x7b,0xb7,0xfa,0x5e,0xf0,0x76,0xae,0x26,0x23,0x6e,0xf5,0xe5, -0xa4,0x86,0x1a,0x80,0xee,0x91,0xfe,0x98,0x7f,0xc1,0x78,0x77, -0xe9,0x83,0x47,0x62,0x41,0x89,0xc0,0xeb,0x75,0x58,0xa4,0xbc, -0xd4,0x7c,0xcc,0xbf,0xe8,0xfb,0x3a,0x2a,0x82,0x4f,0x27,0x83, -0x62,0xb6,0x93,0xe4,0x25,0xbf,0xcd,0xd4,0x35,0x72,0x3e,0x43, -0x2b,0xd2,0xab,0x77,0xef,0x77,0x8e,0x7e,0x7f,0x9c,0x6d,0xfe, -0x6c,0x81,0x9f,0x4d,0xbf,0x3b,0xfd,0xf9,0x2c,0xee,0x7e,0x77, -0x16,0xc3,0x67,0x0c,0xdf,0x49,0xf7,0xbb,0xe4,0x2c,0xd9,0xbc, -0x31,0x9c,0x99,0x6d,0x67,0x65,0x9b,0x66,0x53,0x22,0x32,0xd2, -0x83,0x88,0xdf,0xa3,0x71,0xdc,0xe3,0x46,0x27,0x39,0x6b,0xb1, -0x12,0x04,0xda,0xfd,0x3a,0x7c,0xbf,0x7b,0xb0,0xcf,0x17,0x5a, -0x9b,0x71,0x7c,0xb6,0x89,0x6a,0x23,0xeb,0xc9,0x02,0xbf,0xbe, -0x3b,0x3d,0x9b,0x9e,0x1d,0x77,0xbf,0xeb,0x9c,0x7d,0x77,0xb6, -0x89,0x77,0xb7,0x37,0xde,0x31,0x7e,0xe1,0x86,0x56,0x5d,0xd7, -0x98,0x75,0xf6,0x6b,0x54,0xbc,0xc4,0xb5,0xd1,0x98,0x83,0x5d, -0x5f,0x39,0x7f,0xa2,0xe4,0xe3,0x93,0xbf,0x03,0x57,0x9e,0xd6, -0x4b,0x86,0xf8,0x19,0xb7,0x9c,0x05,0xb7,0xca,0x26,0x74,0x6c, -0xe1,0x97,0xc2,0xc0,0x7d,0x86,0x7d,0xb1,0x4a,0x8e,0x29,0xa2, -0x0d,0xef,0x3c,0xac,0x2d,0x17,0x3b,0xf5,0x7a,0xd4,0x8c,0xd6, -0x28,0x5d,0x59,0x9d,0x11,0xe7,0x9d,0x41,0x3b,0xf0,0x01,0xe0, -0xec,0x02,0x50,0x70,0xef,0xff,0xd0,0xa7,0x26,0x29,0xef,0xd5, -0x2c,0x62,0x35,0xfb,0x9c,0xaf,0xf6,0x09,0xb6,0x06,0xb5,0x67, -0xcd,0x67,0xb5,0xab,0xf1,0xac,0xf6,0xec,0x7e,0x6b,0xf9,0xac, -0x19,0xf1,0xdc,0x6f,0x17,0x1b,0x43,0x73,0xdf,0xdb,0xea,0xef, -0xe2,0x4b,0x59,0x53,0x72,0xf8,0x81,0x91,0xce,0xd1,0x47,0xe8, -0xbe,0x43,0x1e,0xf3,0xea,0x9e,0x93,0x28,0xa7,0xf5,0xca,0xfb, -0xbb,0x12,0xb7,0xe5,0xf5,0x8a,0xa5,0x16,0xd3,0x07,0xfe,0x2c, -0xdb,0x6a,0x0c,0x93,0x00,0x47,0x5f,0xf5,0x3a,0xfd,0x9c,0xd2, -0x5f,0x6b,0x4b,0xb4,0xd5,0x35,0x7e,0x71,0xcc,0x42,0xe1,0x88, -0x3c,0xee,0x39,0x20,0x52,0x9c,0xfd,0xd4,0xb8,0x22,0x5c,0xe9, -0x3b,0xbf,0x0e,0x65,0x95,0x05,0xd4,0x42,0xc5,0xcf,0xcb,0xdb, -0x85,0x4d,0x45,0xb5,0x10,0x44,0x65,0xcb,0x82,0x50,0xb1,0x4e, -0xdc,0x48,0x60,0xb3,0xeb,0x75,0xdb,0x63,0x31,0x06,0xcd,0x16, -0x69,0x43,0x18,0xde,0x2b,0x97,0x7a,0x8d,0x3b,0x2a,0xbc,0xef, -0x89,0x1f,0x04,0x54,0x29,0x86,0x56,0xf5,0xbe,0xb8,0xd1,0x9d, -0xa9,0x33,0x9c,0x72,0x2e,0x25,0xaf,0x95,0x60,0x12,0x3d,0xd2, -0xfa,0xc9,0x00,0x5d,0xd8,0x3b,0xa4,0x73,0x51,0x49,0xc7,0x7d, -0xa6,0x81,0x1a,0x6e,0x21,0xd1,0xab,0xc3,0x6a,0xbb,0x38,0xa1, -0x66,0x30,0x15,0x79,0xa8,0x0b,0xb4,0x28,0xf9,0x2f,0xc4,0xba, -0xa4,0x1a,0xfd,0x28,0xe2,0xe2,0x0a,0x96,0xe5,0x80,0x65,0x77, -0xd9,0x35,0xeb,0x78,0x94,0x33,0x18,0x96,0xf8,0x9b,0x44,0x47, -0x49,0x7b,0x7d,0x97,0x3f,0x2d,0x7f,0xf3,0x3a,0x73,0x9e,0xcf, -0xf1,0x4e,0xe8,0xf0,0x68,0xef,0xd5,0x9b,0x3f,0x7b,0x55,0x98, -0xf8,0x33,0xa9,0xcb,0xbc,0xef,0x2e,0xf0,0x4e,0x5f,0xbe,0x93, -0xcd,0x81,0xf9,0xf3,0xeb,0x23,0xab,0x04,0xf4,0xe7,0xfd,0xb7, -0xaf,0x67,0xb3,0xdb,0x23,0xf6,0x75,0xb4,0x58,0x68,0x73,0x00, -0x93,0x2f,0x4e,0x1d,0x1e,0xf6,0x0c,0xec,0xa7,0xee,0xcf,0x72, -0x8a,0x1e,0xed,0x4f,0x3f,0xdf,0x0c,0xb7,0xa9,0xfc,0xc9,0xc9, -0x61,0xf3,0x37,0xcd,0xad,0xc8,0x39,0x45,0x6b,0x25,0xf7,0xcb, -0xaf,0x2a,0xfd,0x5c,0x97,0xde,0xfe,0xca,0xd2,0xbe,0xe4,0x73, -0x2c,0x19,0x3c,0xba,0x5e,0x47,0x3f,0xc3,0x3f,0xf6,0x2e,0x3e, -0xc2,0x8e,0x8e,0xdf,0x5d,0x7f,0xbe,0x46,0xff,0x76,0x27,0xe4, -0xcd,0x98,0x1f,0x0e,0x95,0x8f,0x7d,0xc3,0x4e,0x41,0x87,0xc3, -0xb8,0x56,0xc1,0x9c,0xb9,0x1d,0x0f,0x2b,0xd6,0x22,0x95,0x04, -0xeb,0xd1,0xf1,0xde,0xd1,0x9f,0xf6,0x8e,0xde,0xef,0xef,0x9c, -0xec,0xbe,0xc6,0x41,0x38,0xfd,0x39,0xed,0x36,0x92,0x14,0x17, -0xc3,0xf8,0xec,0x53,0x23,0xbd,0xdf,0x32,0xad,0xe5,0xd9,0xa7, -0xef,0xfe,0x23,0xe9,0xc4,0x67,0xf7,0x9d,0xd3,0xb3,0x4f,0x67, -0xcd,0x0d,0x58,0x7f,0x97,0xb0,0xa0,0xa6,0xf1,0xe9,0xd6,0xc6, -0xef,0xba,0x68,0x0f,0x14,0xd6,0x4c,0x58,0x8c,0x3b,0xff,0x06, -0xab,0x31,0x7c,0x77,0x00,0x0c,0x7e,0xc2,0xf7,0xbf,0xe1,0xeb, -0x79,0x34,0x08,0x6a,0x0e,0x77,0x4e,0x5e,0xeb,0x7a,0x38,0xf7, -0x8a,0xcc,0x2f,0xf7,0x5e,0xed,0xfc,0xf4,0xf6,0xe4,0xfd,0xe1, -0xc1,0x11,0xaa,0x9b,0x90,0xff,0xe5,0xdf,0x6e,0x19,0xfc,0x9d, -0xa6,0xdf,0x7f,0xff,0xdc,0x5c,0x42,0xc4,0x76,0x6b,0xc9,0x36, -0x27,0x91,0x04,0x8a,0xcd,0xb4,0xf1,0x40,0x93,0x6f,0xe5,0x13, -0x65,0xa3,0xf3,0xde,0xe8,0xea,0xcd,0xe8,0xf5,0xec,0x66,0xf8, -0x6b,0xe7,0x07,0x8b,0xc4,0xa4,0x62,0xa6,0x15,0xa9,0xe5,0x82, -0xf7,0xeb,0xeb,0x78,0x21,0xf6,0x6b,0xf4,0x89,0xb8,0xbe,0x2e, -0xdb,0x76,0x0c,0xf4,0xce,0xa7,0x90,0x33,0xb5,0x88,0x88,0x6d, -0xe0,0x68,0x7d,0x9d,0x13,0x22,0x72,0x8e,0xe6,0x39,0x02,0xba, -0x65,0x93,0xd2,0xa5,0x5b,0x34,0x48,0x73,0x77,0x68,0xb2,0x88, -0xa2,0xc7,0x36,0xef,0x15,0xc0,0x77,0x2e,0x2b,0xbb,0x62,0x01, -0x2b,0x63,0x59,0x9b,0x04,0xf5,0x3a,0x15,0x45,0x9f,0x20,0x71, -0x3f,0x47,0x85,0xc4,0x9f,0x8e,0xde,0xe0,0x74,0x25,0x3b,0xfe, -0xde,0x0e,0x2e,0xec,0x33,0xad,0xf5,0x82,0xc5,0xc2,0xc5,0x4a, -0x69,0x71,0x3b,0x62,0xcd,0xe2,0x92,0xe5,0x7d,0x5e,0x93,0x51, -0xbe,0x53,0xf6,0x6f,0x81,0xf3,0xd8,0xd6,0x50,0xfa,0xd2,0x7b, -0x70,0x2b,0xf7,0x88,0x4d,0xc2,0x9b,0x06,0x74,0xe3,0x56,0xce, -0x81,0xd1,0x90,0x8a,0x04,0x5f,0x55,0x1e,0xa2,0x23,0x32,0xb5, -0x70,0x5d,0x48,0x3d,0xf6,0x79,0x7a,0xf8,0xcc,0xdb,0x75,0x37, -0xf9,0x4a,0x91,0x59,0x1b,0x6d,0xa2,0x71,0x3d,0xbc,0xc0,0x72, -0x82,0x50,0x87,0x60,0x41,0x42,0x03,0x3f,0xd0,0xe7,0x30,0x35, -0x5c,0xbb,0x96,0xc6,0xb0,0xa1,0x5b,0x15,0x7e,0xea,0x74,0x2f, -0xfe,0x6f,0x8a,0xce,0xde,0xd9,0xfd,0x4d,0x6d,0x2b,0x0d,0x07, -0x90,0x01,0x90,0x3f,0x96,0x5a,0x2b,0xa5,0x01,0xb7,0xea,0xe9, -0x94,0x82,0xb2,0x98,0xca,0xc8,0x8f,0x71,0x41,0x80,0xe6,0xc7, -0x4f,0x92,0xc7,0x99,0x43,0x5a,0x73,0x37,0x86,0xbe,0x34,0x49, -0x72,0xe1,0xfc,0x88,0xa3,0xc1,0x74,0x82,0x8d,0xbc,0x22,0xb6, -0x92,0xd7,0xe8,0xa8,0xcc,0xf9,0xa7,0x47,0x27,0xe8,0x74,0x8b, -0xe4,0x8a,0xfd,0x9b,0x0c,0x78,0xf2,0xb7,0x1a,0xee,0xf3,0x6a, -0x37,0x73,0xc8,0x4e,0x46,0x1f,0x58,0xf9,0x1f,0x6f,0x99,0x7a, -0xe4,0xa9,0x08,0xb5,0xca,0x23,0x25,0xe3,0x11,0xd2,0xfc,0xb3, -0x0c,0x3d,0xc9,0xf8,0x3e,0x53,0xdb,0x23,0x6b,0x42,0xc7,0x27, -0x76,0xd4,0xe6,0xc0,0x41,0x3c,0xe5,0x9f,0x6e,0x5a,0x19,0xab, -0x4a,0x2f,0xc3,0xde,0xc6,0x15,0x6a,0x4c,0xe6,0x2b,0x99,0x18, -0x91,0x5a,0x57,0x92,0x0a,0x26,0x46,0xc6,0x7b,0x2d,0xb7,0x93, -0xba,0xac,0xeb,0x2d,0xd0,0x25,0x03,0x6a,0xd2,0xf0,0x3d,0x22, -0x4d,0x4d,0x7e,0x06,0x56,0x64,0x4c,0xec,0xd1,0x3f,0x31,0xa8, -0x9e,0xb0,0x73,0x72,0x70,0x04,0xec,0x2d,0xc2,0xa6,0x47,0x15, -0xe0,0xf9,0x38,0x05,0xbd,0x66,0xe7,0x15,0xc9,0xe8,0x4c,0x2c, -0xa2,0x67,0x63,0x55,0x89,0xc0,0x1c,0xbd,0x82,0x0b,0xed,0xef, -0xd1,0x2a,0xba,0xa2,0x5f,0xe7,0xeb,0xdc,0x68,0x7b,0x8f,0xbd, -0xac,0x17,0xf8,0x41,0x37,0xe7,0xd9,0x79,0x18,0xe1,0xef,0x2d, -0x7b,0x49,0xc7,0x07,0xce,0x93,0x4e,0xaf,0x71,0x9e,0xf6,0xd2, -0x20,0xea,0xdc,0x2b,0x1f,0x2d,0xd1,0xc0,0xe0,0x3f,0x03,0x81, -0x38,0xc0,0xa0,0x97,0x6e,0x25,0x1b,0x71,0xa1,0xd2,0x2d,0x58, -0x04,0xa3,0xef,0x9e,0x52,0x5b,0x00,0xfa,0xbb,0xb0,0x26,0x80, -0xb1,0xf9,0xd5,0x30,0x36,0x4b,0x30,0x7e,0xf5,0xd5,0x30,0x7e, -0x55,0x82,0xf1,0xf3,0x57,0xc3,0xf8,0xb9,0x04,0x23,0x8b,0x84, -0x0e,0xb2,0x2c,0xfb,0x6a,0x70,0x78,0x57,0x58,0x04,0xb8,0xf6, -0x0d,0x70,0xd6,0x2a,0xe0,0x7c,0x13,0x3a,0x15,0xd8,0x7c,0x03, -0x32,0x25,0x28,0x3f,0x7c,0x35,0x90,0x1f,0x4a,0x30,0x5e,0x7c, -0x35,0x8c,0x17,0x65,0x3c,0xbe,0xbe,0x35,0x3f,0x94,0x5b,0xf3, -0xe2,0xeb,0xa1,0xbc,0x28,0x43,0xa9,0xd7,0xbf,0x1a,0x4a,0xbd, -0x5e,0x82,0x02,0x32,0xc1,0xd7,0x42,0x59,0x2c,0xca,0xb8,0x7c, -0x3d,0x2a,0x65,0x4c,0x9e,0x00,0x23,0x2c,0x14,0x66,0x0b,0x92, -0x88,0xf4,0x56,0x01,0x74,0xdc,0x38,0x2c,0xb3,0x5c,0x9a,0xbd, -0xe3,0xdd,0x9d,0xc3,0xbd,0xec,0x7e,0x94,0x46,0x67,0x68,0x70, -0x0c,0x7e,0x2e,0x41,0x64,0x82,0x1f,0x90,0xf8,0x67,0x69,0xf4, -0x3f,0x22,0x73,0x97,0x46,0xff,0x2b,0x42,0x83,0xe8,0x29,0xfc, -0x33,0xcf,0xa2,0x67,0x29,0xfc,0xa3,0xd3,0x39,0xf4,0x92,0x39, -0xda,0xb5,0x0f,0x44,0xd6,0xa7,0x17,0xa5,0x55,0x06,0xa2,0x50, -0xba,0xdf,0xdd,0x7b,0xbf,0x7b,0xf0,0x0e,0x15,0xe2,0x60,0x99, -0x41,0x05,0xa8,0x34,0x42,0x61,0x35,0x32,0xbb,0xc7,0xc7,0x69, -0x74,0x81,0xaf,0x2f,0x7f,0x3a,0x82,0x48,0x32,0xb1,0x70,0xb4, -0x77,0x7c,0xf0,0xd3,0x11,0x14,0xa1,0x28,0xf4,0x33,0x3b,0x9f, -0x5c,0xe4,0x28,0xaa,0x9a,0x3f,0x40,0xee,0x0f,0xd3,0x68,0xd9, -0x2e,0xf8,0x73,0xad,0xdc,0xde,0x15,0x1c,0x9d,0xea,0x3c,0x6c, -0x21,0x1e,0xb2,0x68,0x87,0x64,0x55,0x19,0x68,0x01,0x55,0xde, -0x8f,0xb2,0xa8,0x19,0x19,0x6d,0x85,0x3e,0xbb,0xff,0x02,0xff, -0xa5,0xca,0x7d,0x4b,0x64,0xfd,0x4c,0x44,0xe6,0xfb,0xc4,0xac, -0x4c,0xdb,0xc6,0xed,0xed,0x16,0x64,0x58,0x91,0xde,0x4a,0xcc, -0x3e,0xfc,0x97,0x86,0x9e,0x53,0xa2,0xfd,0xf1,0x08,0xe4,0x47, -0x4a,0xab,0x4e,0x22,0xa0,0x92,0x58,0x48,0xd9,0x26,0xa0,0x55, -0x29,0x2d,0x4c,0xe9,0xf7,0x83,0x24,0xf6,0x72,0xb0,0x0d,0xf1, -0x15,0xd1,0x90,0xfd,0xf5,0xeb,0x20,0x9e,0xfc,0x64,0x50,0xfe, -0xca,0x78,0x28,0x70,0x7d,0x5d,0x59,0xc0,0x6c,0xa0,0x17,0x99, -0xca,0xb4,0x16,0xa7,0xdd,0xdc,0x84,0x38,0x0f,0x46,0xa8,0x81, -0x4c,0x75,0xad,0x48,0x81,0xda,0xa6,0xd3,0x20,0xe9,0x38,0xc7, -0x4b,0x2b,0x2e,0xb4,0x22,0x85,0x0a,0x4d,0x0b,0x00,0x87,0xc3, -0xc1,0xd4,0x66,0x78,0x9e,0x98,0x3d,0xf8,0xaf,0xd8,0xef,0x2f, -0x7b,0x5f,0x22,0x4a,0xa9,0x4a,0xe0,0xf7,0xc6,0xa9,0xf7,0xca, -0x62,0xfe,0x9a,0x86,0x3e,0x66,0x96,0xa6,0xec,0x75,0x01,0x0f, -0x82,0x3b,0xe9,0xe9,0xcf,0x5f,0xf6,0xfb,0xaf,0xaf,0x6f,0xa6, -0xbd,0xbf,0xee,0x3d,0x83,0xfd,0xed,0x02,0xe2,0x9e,0x51,0xfc, -0xb3,0xee,0xe2,0xd9,0xb3,0xe4,0xbb,0x67,0x14,0xb5,0xd7,0x58, -0x7c,0x69,0x2c,0xf6,0x1b,0x8b,0x7e,0x63,0xf1,0xba,0xb1,0xb8, -0x6e,0x2c,0x6e,0x1a,0x8b,0x69,0x63,0xd1,0x5b,0xfc,0x35,0x49, -0x70,0x2b,0xbb,0x69,0x02,0x2f,0x13,0x78,0x16,0xdd,0x6f,0xac, -0x6f,0xb6,0xbd,0x33,0x97,0x95,0xa4,0x5f,0x70,0xc6,0xeb,0x1c, -0x2c,0xbb,0xf8,0xa4,0xe8,0x47,0xd7,0x65,0x99,0xfb,0x07,0xa4, -0x81,0x35,0xfd,0x60,0xae,0x92,0x10,0xca,0x55,0x21,0x43,0xd8, -0x19,0x5d,0x5c,0x8f,0x27,0xce,0xce,0x80,0x83,0xa5,0x8c,0x06, -0xec,0x45,0x46,0x8e,0x79,0xca,0xa7,0x58,0x6c,0x30,0xa0,0xa4, -0xfe,0x4f,0xcf,0xbd,0xd1,0x31,0x36,0xdb,0x4c,0x56,0xe6,0x93, -0xc9,0x94,0x83,0x75,0x00,0x8e,0x5b,0xc8,0x50,0x35,0x20,0x2e, -0xf8,0xe2,0x86,0x5d,0x2a,0x86,0xe2,0xe8,0xcd,0x1e,0xec,0x58, -0x3e,0x47,0x49,0xa2,0xae,0x80,0x03,0xdb,0x05,0xfe,0x74,0xb2, -0xca,0xbe,0x84,0x28,0xed,0x68,0xad,0x7a,0x41,0x02,0x96,0x1a, -0x56,0xd9,0x09,0x2f,0xc6,0xd8,0x36,0x28,0x6c,0xff,0x46,0x57, -0x4e,0xa3,0x7c,0x67,0x38,0xe8,0x79,0x93,0x0c,0x53,0xfd,0x18, -0x46,0x2b,0xfe,0x6b,0x7b,0xf2,0xe3,0x5b,0x3a,0xe2,0xc3,0xfa, -0xe8,0xd4,0x8d,0xd5,0xc8,0x44,0x21,0x18,0x04,0x17,0x9b,0xc3, -0x2a,0x45,0x4e,0x6e,0x7a,0xc3,0xc1,0xdf,0xf3,0xbe,0xb7,0x30, -0xfe,0xce,0xc6,0xc5,0xfc,0xf0,0xcf,0x81,0x6a,0xaf,0xc6,0xec, -0xd4,0x03,0xea,0x96,0xdf,0xa9,0xde,0xdf,0x4e,0x06,0xe3,0x09, -0xec,0x6f,0xd2,0xd6,0xd6,0x56,0x79,0x68,0xcb,0x4e,0x17,0xaa, -0x6c,0x44,0x54,0xbc,0xff,0xf7,0x75,0x96,0x5e,0xf7,0xf8,0x91, -0xb7,0xe8,0x9b,0xb5,0x35,0x67,0x50,0x85,0xfe,0xf3,0xda,0x3f, -0xa7,0x11,0xec,0x54,0x81,0x38,0xe0,0x2f,0x94,0x80,0x0f,0x1a, -0x27,0x05,0xd4,0xf7,0xe6,0xbf,0xa2,0xd3,0x7e,0xf7,0xbb,0x27, -0x98,0xd0,0x08,0xed,0x1d,0x78,0xc8,0xa5,0x9e,0xa1,0x1f,0x3b, -0x63,0xc2,0x1e,0xe2,0x1c,0x86,0xa7,0x94,0xbb,0x9d,0x07,0x82, -0xf1,0x59,0xa8,0xe3,0x6d,0xa8,0x6b,0x2d,0x0b,0x58,0x7f,0x94, -0xde,0x02,0x7a,0x76,0xaf,0xcc,0x82,0xf3,0x5e,0x20,0xb4,0xcc, -0x2d,0x71,0xca,0x34,0x90,0x8f,0x21,0xf3,0xbe,0x3e,0x68,0xad, -0xe7,0xf2,0xe9,0x70,0x3b,0xb4,0x6a,0xaf,0x59,0x8d,0xa0,0x0c, -0x03,0xb8,0x4e,0x86,0xd3,0x23,0xf7,0x8c,0x8a,0xd9,0x0f,0xbe, -0x0d,0x73,0xdd,0xff,0x8a,0xef,0x79,0x7c,0xc7,0x0f,0xa6,0xef, -0xae,0xb4,0xaf,0xb2,0x47,0x2e,0x7d,0x4a,0x00,0x41,0xc0,0xa2, -0x57,0xb7,0x64,0x10,0xde,0x14,0xb8,0x99,0x7b,0xc2,0x15,0x62, -0xbf,0x7a,0x32,0xe0,0xb5,0x79,0x71,0xc4,0x11,0xf2,0x9e,0x1a, -0x75,0x05,0x97,0xdf,0xfa,0xd0,0x98,0xb2,0x17,0x3f,0xd1,0xb4, -0x08,0x58,0x4c,0xd9,0xb6,0xad,0xe5,0x52,0x15,0xfc,0xa8,0x53, -0xcd,0xa4,0x52,0xab,0x6e,0x88,0x48,0x5a,0x53,0x3b,0xcb,0x76, -0x85,0x76,0xa0,0xc2,0x16,0xdf,0x69,0x47,0xd3,0xf9,0xf9,0xcd, -0x00,0xfa,0xb2,0x1a,0xa7,0x44,0xb7,0x8e,0x38,0xea,0x7a,0x1f, -0xbb,0x70,0xfc,0x25,0xb0,0xda,0xe1,0x6e,0x0f,0x82,0x0e,0xab, -0x78,0xcc,0xfb,0xb5,0xd5,0x2f,0xe5,0x2a,0x82,0x5f,0x9d,0x78, -0x7b,0xd3,0x44,0xcc,0x1a,0xb5,0x07,0x9c,0x00,0x98,0x1e,0x4e, -0xf0,0xac,0xb8,0x00,0xb1,0xf9,0xfe,0x36,0x25,0xa2,0x2b,0x6d, -0x92,0x1c,0x78,0x44,0x29,0x4e,0x0d,0x23,0x47,0x68,0x5b,0xde, -0x58,0x3f,0x5b,0xa9,0x7f,0xb4,0x6f,0xc2,0x42,0x05,0x4b,0xf8, -0xb1,0xa2,0x15,0xb3,0x1a,0x15,0x77,0x84,0x62,0x31,0xc9,0x3f, -0xcf,0x70,0x95,0x54,0x28,0xea,0x69,0x2e,0xf3,0xdf,0x3f,0x01, -0xe2,0x29,0xd4,0x71,0xa5,0x6e,0xbf,0xc4,0xc1,0x24,0x49,0x8c, -0x5e,0xe7,0x77,0x76,0xa3,0x65,0x92,0x06,0x19,0x96,0x5d,0xb6, -0xea,0xed,0xa7,0x55,0xd5,0xac,0x8d,0x71,0x99,0x7c,0xf5,0x78, -0x36,0x94,0xcc,0xbc,0x89,0x38,0xbc,0x0d,0xb8,0x9c,0xdd,0x2e, -0xf0,0x6c,0x9f,0xfe,0x4c,0x57,0x5d,0x3f,0x1c,0x37,0x92,0x38, -0x95,0x2b,0x07,0xbc,0x71,0x58,0xa0,0x55,0xde,0xb3,0x4f,0xff, -0xb6,0x96,0x36,0x3b,0x8d,0xac,0xfe,0xab,0xff,0x58,0x3b,0xdb, -0x38,0xdb,0xec,0xf2,0x0d,0x82,0x36,0x92,0x06,0x55,0xe0,0x23, -0xeb,0xde,0xc6,0xdf,0xa1,0x70,0xf3,0xfd,0xaf,0x1a,0x1b,0xdd, -0xc6,0x7f,0xa8,0x18,0x08,0x9e,0x35,0x25,0xdc,0xbd,0xdf,0x36, -0xbf,0x59,0xae,0x3b,0xd1,0xcd,0x41,0x40,0x35,0x82,0xb3,0x8d, -0xc5,0x19,0xd5,0x0d,0x12,0x1f,0xfc,0x81,0x88,0x26,0xfc,0x05, -0x69,0x84,0x95,0x0a,0x06,0xd6,0x06,0x57,0x46,0x66,0xe1,0xd2, -0xc0,0x36,0x9c,0x6c,0x79,0xd2,0x82,0x05,0x50,0xba,0x04,0xd0, -0x96,0xfd,0x0c,0xd9,0x4e,0x4b,0x43,0x8b,0x74,0x86,0xec,0xa6, -0xa5,0xa1,0xf5,0x34,0x63,0x6d,0x7f,0xa5,0x25,0x23,0x60,0xe6, -0x7a,0xd0,0xef,0xe7,0xac,0xc2,0x23,0x8f,0x70,0x45,0xc7,0x87, -0x26,0x1c,0x7f,0xc3,0x98,0xe7,0x33,0xd1,0xf2,0x21,0xd4,0xfd, -0xb8,0x01,0x9b,0x95,0xfb,0x26,0x62,0xd7,0x6c,0xbc,0x4e,0x73, -0x5c,0x49,0x75,0x86,0xed,0x2a,0xac,0x4c,0xed,0x45,0xf6,0x35, -0x72,0x1a,0x75,0x46,0x57,0xfb,0xe3,0x3e,0x6a,0x10,0x3c,0xb6, -0x68,0x8a,0xad,0xb2,0x0b,0x9a,0x61,0xb1,0xeb,0x51,0xf5,0x42, -0x99,0xa6,0x2f,0x69,0x41,0x77,0x17,0x0b,0x97,0x81,0x5f,0x39, -0x7d,0x85,0x01,0x3e,0x98,0x21,0x5d,0xa3,0xdc,0x8f,0x90,0x45, -0x85,0x3b,0x5c,0xf6,0x22,0x13,0xf8,0x25,0xa1,0x04,0xb1,0xb6, -0x1f,0x99,0xd0,0x2a,0x3d,0xa5,0x59,0x1b,0xf4,0xb0,0x41,0xf5, -0x86,0xec,0x29,0x85,0x8c,0xd5,0x47,0xe6,0x1d,0x37,0xde,0xaf, -0x30,0xd8,0xbf,0xbc,0x06,0x42,0xef,0xe6,0x9f,0x2f,0xf2,0x5b, -0xbe,0x7a,0x22,0xd5,0xb0,0x60,0xa5,0x54,0xab,0x27,0xcb,0xec, -0x6a,0x0c,0xb8,0xad,0xa5,0xf2,0x86,0x4a,0x1b,0x5b,0xd4,0x58, -0x27,0x5a,0xde,0x18,0xc0,0xbf,0xce,0xb5,0xcc,0x7a,0xd5,0x6d, -0xf4,0xbf,0xcc,0xb7,0x0c,0x9f,0xde,0x3b,0xf3,0x6f,0x19,0xdb, -0xf2,0x6c,0xbe,0xeb,0xbd,0x13,0x35,0x93,0x1b,0x1c,0xb6,0x15, -0x69,0x62,0xe3,0x0d,0x0d,0xc4,0x70,0x84,0xb7,0x1a,0xe7,0xe3, -0x10,0xf6,0x2e,0xd9,0x76,0xb7,0x6b,0x9a,0x4a,0xd4,0xbe,0x0b, -0x38,0xc6,0x39,0x37,0x90,0xd2,0xd6,0xef,0x06,0x07,0x95,0xd3, -0x18,0x8e,0xa0,0xf7,0xd9,0xeb,0x6e,0x65,0x63,0x79,0x8f,0x69, -0xed,0xf7,0xde,0xe9,0xd3,0xba,0x35,0xc5,0x8c,0xf1,0xc8,0xa2, -0xe9,0xe3,0x18,0x32,0xf8,0xbc,0x4d,0xe8,0xc6,0xc1,0xd5,0x88, -0x1e,0xbb,0xf8,0x0c,0x25,0x9b,0xe0,0x3c,0x87,0xe9,0x66,0x7a, -0xc4,0x25,0x58,0x25,0x08,0x26,0x3d,0x9a,0x43,0x61,0x75,0x1f, -0x58,0x70,0x40,0x1a,0x1f,0x6d,0x70,0x06,0x54,0x21,0xd0,0xd6, -0xc0,0x5b,0xcb,0xc8,0x04,0x18,0x05,0x86,0xc0,0x2d,0x7d,0x24, -0xf6,0x22,0x49,0xcc,0xf8,0x21,0x93,0x6a,0x87,0x32,0x40,0xb6, -0xee,0x55,0xb6,0xf5,0x3b,0xfa,0x88,0x06,0xc2,0xcf,0xb5,0xc7, -0xdc,0x00,0x89,0xeb,0x1f,0xae,0xce,0xba,0x01,0x6a,0xaf,0x97, -0x94,0x3d,0x9e,0xe0,0x79,0x87,0x61,0x3c,0xe2,0xfe,0xc5,0xcd, -0x09,0xeb,0xa4,0xe4,0x9e,0x6b,0x3d,0x2d,0x67,0xe9,0xae,0x65, -0xd9,0x9a,0x64,0x43,0x66,0x28,0x54,0x1e,0xaf,0x2e,0x50,0xaf, -0x7f,0x95,0x9f,0x17,0x8f,0xf4,0x4a,0x52,0xc3,0x17,0xdf,0x25, -0x3f,0x27,0x49,0x31,0xdb,0x56,0x08,0xa6,0x55,0xf0,0x94,0x92, -0x98,0xd5,0x28,0xbb,0x16,0x16,0x7b,0xf4,0x01,0xbe,0xf4,0x04, -0xb7,0x2a,0xc5,0x6e,0x26,0x04,0x13,0xa7,0x40,0xb6,0xc2,0x3b, -0x49,0xe5,0x34,0xd4,0xf3,0xb4,0x92,0x81,0x3d,0xc9,0x7f,0x89, -0xaa,0xd9,0xd9,0xb0,0x2c,0x59,0x7c,0x2d,0x72,0x24,0xd6,0x36, -0x0d,0xd1,0x40,0x57,0xb8,0x1a,0xcd,0xad,0x12,0x9a,0xad,0x6a, -0x34,0x9f,0xe6,0xf6,0xa5,0xda,0xab,0x8b,0x7a,0x48,0xa7,0xb9, -0x5f,0xa8,0x0c,0xec,0xb4,0x6e,0xed,0x9e,0xbe,0xc4,0x46,0xbd, -0x63,0x91,0xb8,0xc4,0x61,0xb9,0xad,0x9e,0xfb,0xd8,0x05,0x4e, -0x36,0xc0,0x61,0xfd,0x15,0x8c,0xd5,0xe3,0xe2,0xbc,0x68,0x90, -0x56,0x90,0x77,0xfe,0x61,0x35,0x6f,0x70,0xca,0x15,0x16,0x4d, -0x88,0x5c,0x92,0x3b,0x10,0x7a,0xd0,0x83,0xdb,0x0b,0xac,0xa7, -0xbd,0x1e,0x1c,0x70,0x54,0x1a,0xa4,0xf1,0x4c,0x54,0x10,0x26, -0x13,0x10,0x6c,0x07,0xb4,0xdc,0x70,0xbf,0x45,0x95,0xa5,0xa2, -0x68,0x71,0xd4,0x0c,0xfa,0x9f,0x33,0x65,0x80,0x54,0xd4,0x9c, -0x51,0x0d,0xba,0x08,0x93,0x21,0xb6,0xa1,0xc0,0xc6,0x46,0x5b, -0x94,0xa0,0x7d,0xc9,0x53,0x88,0xef,0x5a,0x1b,0xee,0x2b,0xfd, -0xbb,0x14,0x3d,0x7f,0xc8,0x28,0x68,0x1b,0xaa,0x68,0x1e,0x08, -0x76,0x14,0x5d,0x3b,0x34,0x4a,0xba,0x2f,0xed,0x8f,0xad,0xc0, -0x77,0xea,0x16,0x0b,0x13,0xfd,0x4c,0x7e,0xae,0xf1,0x5a,0xc0, -0x6f,0xb8,0x4b,0x12,0xd1,0xd3,0xe4,0xc2,0xa9,0x33,0xd8,0x08, -0x65,0x71,0x90,0x28,0x12,0x77,0x92,0x97,0x76,0x57,0xc8,0x31, -0xad,0x6e,0xc8,0xf4,0xdb,0x97,0x6e,0x03,0xa6,0x7c,0xac,0x39, -0x38,0xa1,0x55,0xda,0xca,0x7d,0xdc,0xe5,0x8a,0x1d,0x9c,0x87, -0x41,0xbb,0x2e,0xb4,0x7c,0x49,0x64,0x59,0x79,0x96,0x2a,0xd2, -0xf0,0xb7,0x08,0xc3,0x95,0x24,0x5f,0x30,0x30,0xeb,0xce,0xe3, -0x9c,0xb1,0xc9,0xa6,0x45,0x87,0xb0,0x4b,0x9c,0x79,0xa0,0xa7, -0x8c,0xe1,0xa3,0x42,0xfb,0x4d,0x60,0xb0,0x99,0x69,0x5e,0xce, -0xbc,0x6c,0x3d,0xc0,0x9d,0xda,0x32,0x55,0x1e,0xb6,0x71,0x1d, -0x94,0xa2,0xb5,0x2e,0xb4,0x79,0x2d,0xda,0xf4,0xb8,0x20,0x55, -0x9a,0xb9,0xb6,0x45,0xd9,0x86,0xb5,0x35,0xbd,0xf1,0x48,0xce, -0x92,0x29,0xf9,0xa2,0xe1,0xdf,0xbb,0x55,0x46,0xae,0xe7,0xa3, -0xe9,0xf5,0xe0,0x72,0xa6,0x33,0x14,0x6c,0x9b,0xfa,0x6a,0x02, -0xae,0x21,0xd9,0x4b,0x0e,0x77,0x78,0xcb,0x0e,0xd4,0x83,0x63, -0xfb,0xa4,0xd1,0xf9,0x5a,0x22,0xf2,0xca,0x62,0x9b,0xd6,0xa7, -0x8d,0x58,0x46,0x14,0x42,0xc1,0x9a,0x51,0xcf,0x49,0x7c,0x6b, -0xb2,0x8b,0xf6,0x7a,0xbd,0xc2,0xbb,0x4d,0xe2,0xce,0x50,0xb0, -0xcc,0x62,0x01,0x93,0xdc,0x50,0xcf,0xa8,0xd1,0x75,0x2d,0x73, -0xce,0xab,0xd4,0xf3,0x48,0x97,0x4a,0x87,0x28,0xc4,0xd6,0x5d, -0x94,0x3c,0xd2,0x76,0x78,0x94,0x8d,0x44,0x09,0xeb,0x42,0x98, -0x3c,0x4c,0xda,0x95,0xd5,0x32,0x21,0x27,0x4a,0x95,0xce,0x01, -0x78,0x23,0xf4,0xb5,0xce,0x22,0x58,0xd1,0xf6,0x4e,0x1b,0x59, -0xf4,0x46,0xc4,0x94,0x7a,0x0b,0x4d,0xff,0xdd,0x83,0x77,0xc7, -0x27,0x3b,0xef,0x4e,0xde,0xc3,0xe6,0xe3,0xa7,0x3d,0x75,0xc4, -0x81,0x3a,0x3c,0x0b,0x52,0xd6,0x59,0xa0,0x51,0xca,0xf5,0x4d, -0x18,0x6b,0x6a,0xef,0x43,0x83,0xfd,0xf0,0x11,0xfd,0xec,0x76, -0x68,0xe0,0xdf,0x8e,0xbe,0x7c,0xa9,0xac,0x9f,0x8d,0xc4,0x4b, -0xd6,0xa6,0xd4,0xab,0xbc,0x0a,0x87,0x13,0x5a,0x9f,0xd3,0x5b, -0xe3,0xe3,0x15,0x7c,0xe5,0x4f,0xd2,0xdf,0xab,0xf8,0x42,0xc5, -0xe5,0x80,0x2d,0xf5,0xc0,0xc5,0x80,0xad,0xf0,0x2e,0x9c,0x15, -0x3f,0x0e,0x46,0x8a,0x67,0x69,0xe7,0xae,0x0f,0x9e,0xbf,0x97, -0x84,0x78,0xdc,0x94,0xe3,0xf3,0x19,0xb4,0xb8,0x98,0x34,0xfb, -0xbc,0x5f,0xb0,0x11,0xd6,0x17,0x10,0x56,0x96,0x98,0x2a,0xec, -0x31,0xa5,0x84,0xbc,0xad,0x04,0x0f,0x20,0xe2,0xd2,0xab,0x9f, -0x28,0x4a,0x5d,0x5b,0x12,0x69,0xc9,0x49,0x7e,0x73,0x8b,0xfa, -0xb7,0xc1,0x79,0x8b,0xd6,0xcb,0x0d,0x2c,0x16,0xbb,0xe8,0xa7, -0x5d,0xc9,0xf0,0x2b,0x4b,0x57,0xe8,0xd5,0x28,0xd3,0x30,0x02, -0x8b,0x52,0x72,0xdd,0xa0,0x1a,0x67,0x31,0x4b,0xbc,0xed,0x94, -0x27,0xf6,0x5d,0x50,0x65,0x89,0x1b,0x86,0xd0,0xa3,0x27,0xf4, -0x21,0x8d,0x7d,0x57,0x3a,0x0c,0xd5,0x6e,0x83,0xce,0x42,0xf5, -0x86,0xe0,0x24,0xe4,0x89,0x9d,0xf3,0xed,0x04,0x81,0x28,0xac, -0x26,0x0a,0x4c,0x5d,0xd9,0x28,0xbc,0x60,0x25,0x1c,0xc9,0x46, -0xe7,0x64,0x0e,0x8b,0x37,0x15,0x50,0x8e,0x04,0x5d,0x6b,0x77, -0x03,0x7b,0xef,0x59,0xc1,0xfc,0x7b,0xc4,0x8b,0x96,0x64,0x3b, -0xe8,0xf7,0x57,0xe7,0x84,0xc4,0xc8,0xf8,0xbc,0x78,0x10,0xbf, -0x3a,0x33,0xa6,0xd2,0x3d,0x3c,0xe6,0x1e,0xf7,0x3e,0x56,0x4f, -0xb5,0xfb,0x47,0xee,0x7f,0xd5,0x2c,0x16,0x38,0x91,0xf2,0x26, -0x51,0xf9,0x10,0x01,0xbb,0xff,0x82,0x72,0x26,0x7c,0xcb,0xea, -0x25,0x42,0x35,0xde,0x65,0xc6,0xc8,0x46,0xfc,0xbc,0x6f,0x56, -0x94,0x29,0xa3,0xff,0x88,0xa4,0x0f,0xa7,0xb7,0x21,0xb1,0x94, -0x8f,0x2f,0x8b,0xa7,0x96,0x9e,0xd1,0xe6,0xcf,0xab,0xae,0x7f, -0x9c,0x37,0xbe,0x8b,0xe9,0x2d,0x79,0xdc,0xa4,0x8a,0xe8,0x76, -0xa3,0xfa,0x26,0x98,0xaf,0x9e,0x6b,0xfd,0xf3,0x21,0x7f,0x90, -0x7d,0x09,0xb4,0x9d,0xc1,0x5f,0xf3,0xdb,0x9a,0xb3,0x38,0x51, -0xb3,0x16,0x29,0xf8,0x03,0x3b,0xa7,0xe6,0x4d,0x53,0xd4,0xbc, -0xed,0x8a,0x9a,0xb8,0x27,0xc4,0x5f,0x00,0x00,0x7f,0xe9,0xe0, -0xa5,0xc6,0x67,0xba,0xb0,0xa9,0xb8,0x98,0x4f,0x6b,0xe7,0xc3, -0xf9,0x24,0xaa,0x34,0x74,0x32,0x72,0x77,0xa5,0xfe,0x82,0x14, -0x0f,0x90,0x56,0x5e,0x97,0x8e,0xe4,0xaa,0xb4,0xd0,0xcc,0xd3, -0xa0,0x78,0xd7,0xab,0x13,0xe8,0xf7,0x6b,0xd5,0x7e,0xfb,0x57, -0xf1,0xab,0xcb,0x91,0x3d,0xac,0xa2,0xab,0xcc,0xb0,0x82,0x44, -0xbb,0x68,0x2c,0x58,0xdb,0x2b,0x5d,0xed,0xaf,0xf2,0x1f,0x71, -0x69,0x2b,0x87,0xad,0x1f,0xe6,0xe4,0x1b,0xb3,0x25,0x71,0x65, -0x7c,0xdb,0x22,0x97,0xa5,0x57,0x6f,0x2e,0x03,0xd2,0xb1,0x0f, -0xf7,0x54,0xd3,0x24,0xca,0x91,0xce,0x6c,0xd2,0x1b,0x4d,0xc9, -0xb7,0x6c,0x1a,0xb9,0xd3,0xda,0x80,0x9e,0x60,0x14,0x6f,0x06, -0xa3,0xde,0x10,0xe9,0x55,0x39,0x61,0x78,0x44,0x9f,0xc2,0x78, -0xc0,0xe5,0x8e,0x74,0x67,0xbf,0xf6,0x8c,0xd7,0xf7,0x25,0xd9, -0x2c,0xb0,0x17,0x92,0x14,0x38,0xc6,0xbc,0x85,0xad,0xac,0x65, -0xfa,0x6f,0x2e,0x4b,0x6c,0x4b,0x03,0x00,0x49,0xdc,0x36,0xb8, -0x49,0x34,0x18,0xeb,0x54,0xb1,0x45,0x21,0x21,0x65,0xc2,0x4a, -0xd5,0x8b,0x5b,0x4c,0x17,0x68,0xda,0xad,0x55,0xac,0xb3,0xe8, -0x82,0xde,0x73,0x29,0x63,0x13,0x14,0xcf,0x6c,0x1b,0x40,0x26, -0x45,0x4d,0x6e,0xd7,0x41,0x2a,0x8f,0x32,0xec,0x30,0xe4,0x17, -0x55,0x1a,0x45,0x8a,0x73,0xcf,0x34,0xe5,0xbd,0x99,0x44,0xe6, -0x85,0xbb,0x44,0x17,0x93,0x2c,0xed,0x95,0x3a,0x4e,0xfa,0x37, -0xec,0x48,0x38,0x20,0x13,0xbc,0xb6,0xc2,0x23,0xfa,0x99,0x2c, -0x70,0xa4,0x9a,0x47,0xe7,0xf8,0xa4,0x57,0x73,0xfc,0xff,0x90, -0xf7,0xa7,0xdb,0x6d,0x1c,0xc9,0xa2,0x28,0xfc,0xff,0x3c,0x85, -0x88,0xed,0xa6,0xab,0x80,0xc4,0x50,0xa0,0x24,0xdb,0x80,0x8a, -0x38,0xb2,0x2c,0xbb,0x7d,0x8e,0x3c,0x6c,0x49,0xee,0xb6,0x9b, -0x82,0xb9,0x30,0x91,0x84,0x05,0x02,0x30,0x0a,0xa0,0x00,0x92, -0xf8,0x56,0xef,0xb5,0xce,0xea,0x27,0xf8,0xde,0x64,0xf7,0xfe, -0xd5,0xef,0x20,0xbf,0xd2,0x8d,0x29,0xc7,0x2a,0x80,0x54,0x0f, -0x77,0xdf,0xbb,0xee,0xb2,0x45,0x54,0x4e,0x91,0x91,0x91,0x91, -0x99,0x91,0x99,0x91,0x11,0x03,0x9c,0xa2,0x30,0xac,0x1f,0x4a, -0x29,0x87,0xb5,0x72,0xab,0x19,0x42,0x53,0x3e,0x2c,0xe5,0x41, -0x32,0xef,0xb0,0x4c,0x4b,0x94,0xbb,0x04,0xba,0x77,0x30,0xcf, -0x80,0xcd,0x5c,0xfe,0x2b,0xe4,0xd6,0xfb,0xf1,0x61,0x66,0x2e, -0xb0,0xb3,0xc5,0x00,0xb6,0x03,0xda,0x4b,0x8c,0x10,0x44,0xf6, -0x04,0x90,0xa6,0x66,0x53,0x34,0xea,0x6b,0xb2,0x70,0x90,0x9e, -0x84,0xa0,0x2d,0x4f,0x6e,0x83,0x49,0xc5,0xa8,0x8c,0xa2,0xda, -0xc5,0xd3,0xc5,0x47,0xbe,0xb1,0x15,0x31,0x18,0x4a,0x49,0x12, -0xb0,0xdc,0x8e,0x5b,0x5d,0x3a,0x73,0x1c,0x2d,0xd0,0xa0,0x1c, -0x32,0xd2,0x6a,0xfe,0xa2,0x97,0x2d,0x05,0x49,0x31,0x02,0xe8, -0x8a,0xda,0x2e,0x40,0xe4,0x36,0x27,0xe8,0xb1,0xab,0x13,0x4f, -0xce,0x63,0xe3,0xa0,0xf6,0x82,0xa1,0xe2,0xa5,0x87,0xf9,0x19, -0xc8,0xff,0xd8,0xfa,0x3e,0x34,0x48,0x4c,0xa0,0xa9,0xf0,0x69, -0xf6,0xd2,0x6a,0x60,0x46,0x4c,0x72,0x57,0xeb,0x69,0x31,0x10, -0xeb,0x43,0x17,0xe3,0x8c,0xf7,0xf8,0x5f,0x0f,0xd3,0x4a,0xc5, -0xa3,0x41,0x1b,0x72,0x75,0x98,0xa1,0x50,0xf6,0xc0,0x42,0xea, -0x86,0x0c,0x8b,0xb7,0x7c,0xfe,0xda,0xc6,0xb5,0x6c,0x35,0x18, -0x8c,0xd0,0xab,0x89,0xae,0x01,0xb8,0x68,0x3e,0x9b,0x66,0xac, -0xad,0xe4,0xd5,0x92,0xa6,0x5e,0x2d,0xa2,0x80,0x33,0x7a,0xc7, -0xc4,0x71,0xc7,0x68,0xdb,0x65,0x9e,0x48,0x67,0xc9,0x8f,0xd1, -0x5d,0x1d,0x95,0x23,0xbc,0x86,0x10,0xd0,0x92,0x87,0xb0,0x1f, -0xc9,0x72,0x97,0x69,0x45,0x6e,0xd4,0xfb,0xbc,0x83,0xbd,0x61, -0x59,0xcd,0x8c,0xb0,0x20,0x5b,0x4d,0x9b,0xdc,0x89,0xe2,0xd8, -0x63,0x15,0xcf,0x2e,0xae,0xc7,0xe5,0xb8,0x59,0x76,0xc3,0x87, -0x87,0x07,0xde,0x6e,0x2a,0xc8,0xeb,0x0d,0xf4,0xa0,0xf9,0x50, -0xe4,0x12,0x97,0xf5,0x8f,0xc6,0x1e,0x91,0xc4,0xca,0x76,0xec, -0xed,0xd2,0xcc,0x18,0xe4,0x93,0x9f,0x1a,0x9d,0xbf,0x47,0xc1, -0x51,0xf7,0xae,0x1e,0x45,0x0f,0x31,0x3b,0x7b,0x64,0x6b,0x2b, -0x2a,0xc2,0x46,0xde,0x21,0x22,0x42,0x71,0x6b,0x2f,0x18,0x3b, -0xa3,0x8e,0x97,0xf7,0x95,0x38,0xf7,0x69,0xe2,0xb8,0xb3,0x56, -0x96,0x3f,0x0b,0xcb,0x6a,0x5c,0x15,0x2b,0x47,0x41,0xbd,0xdf, -0xce,0xa6,0x28,0xbb,0xe3,0x0d,0xd5,0x8e,0xea,0xdd,0x99,0xd3, -0x5d,0xd2,0xa9,0xf8,0xf7,0x20,0x69,0x91,0xb8,0xe4,0x2d,0x05, -0xa2,0xc5,0xa9,0x76,0x6e,0xe8,0x38,0x83,0xf2,0x37,0x76,0x64, -0x65,0xfb,0xe5,0xd3,0x67,0xcf,0xd3,0xfa,0xcd,0xb6,0xae,0x2d, -0x93,0x78,0x5a,0x19,0xf7,0x71,0xd9,0xc4,0x0a,0x5f,0x78,0x51, -0x69,0xa6,0xd7,0x01,0xf6,0xa8,0x7a,0x77,0x31,0x9a,0x9a,0x28, -0x5e,0xfb,0x31,0x2a,0xf0,0xa7,0x16,0xa4,0xc6,0x68,0xfb,0x0b, -0xc4,0x79,0x99,0xc5,0xe9,0xfb,0xf6,0xb6,0x41,0xc0,0x32,0xcf, -0xcb,0xb0,0x80,0x27,0xe3,0x23,0x9c,0x0c,0x21,0x34,0x59,0x02, -0x41,0xba,0xde,0x7b,0xb5,0xb9,0xec,0xcf,0x26,0xde,0x4e,0xb4, -0xe6,0x24,0x00,0x9f,0x8f,0xa6,0xc3,0xa2,0x4c,0x26,0x3a,0xc2, -0x87,0x47,0x7f,0x04,0xd0,0x69,0xfd,0x67,0xac,0x21,0x42,0x9d, -0xe3,0x2c,0xee,0x44,0x35,0x3c,0x4c,0x31,0x72,0x37,0xad,0x58, -0x76,0x1d,0x33,0x57,0x93,0xca,0x58,0x50,0x16,0x05,0x4c,0x82, -0xc5,0x07,0x23,0x7e,0x12,0xcc,0xe3,0xd4,0x82,0x40,0xe3,0xc0, -0x64,0x30,0xee,0x34,0x4a,0x98,0xad,0xe4,0x79,0xd8,0x20,0x45, -0x68,0xbc,0xb7,0xaf,0x02,0xf3,0x77,0xd3,0x1d,0xc4,0x3d,0xf1, -0xe0,0xb1,0x2b,0x23,0x8d,0x3f,0x55,0x5d,0xd8,0x00,0xb2,0xab, -0xe1,0xd0,0x96,0xec,0x65,0x04,0x7b,0x7b,0x93,0xdb,0x60,0x44, -0x4c,0xe5,0xf6,0x41,0xc5,0xf0,0x47,0x05,0xaf,0xed,0xb9,0x57, -0x2b,0x86,0xcc,0xb1,0x33,0xbc,0xf7,0x5c,0x7a,0x38,0x9e,0x41, -0x5d,0x3e,0x30,0xb0,0xad,0x19,0x14,0xf6,0xd5,0x22,0xa7,0x67, -0x68,0x5f,0x98,0x3e,0xd1,0x0a,0x29,0xb5,0xe5,0xf6,0x56,0x8e, -0x4c,0x64,0x5c,0xd4,0xe6,0x34,0xaa,0x9e,0x69,0x47,0xb0,0x55, -0x46,0x30,0x8e,0x5d,0xb6,0x3a,0xa1,0xa4,0x6e,0x30,0x04,0x0e, -0xd0,0x0d,0x8f,0xb3,0xdd,0x11,0x17,0x65,0xde,0xd1,0x82,0x76, -0x18,0x16,0xcb,0xd4,0xf3,0x72,0x34,0x1f,0xf5,0x7c,0xbd,0x17, -0xd9,0xcb,0x14,0xca,0xfe,0x94,0xa6,0xec,0x16,0x80,0x6c,0x21, -0x7f,0x75,0xfa,0xf2,0xf9,0x37,0xdf,0xfd,0xe1,0xf9,0x17,0x69, -0xe9,0xa3,0x8f,0x6c,0xa8,0x64,0xc0,0xe7,0xdc,0xea,0x70,0x74, -0x49,0xd3,0xe8,0x03,0xb7,0x10,0xfb,0xa5,0x35,0x9c,0x24,0xec, -0xc6,0xf6,0x5e,0x3b,0x06,0xa8,0x7e,0xf0,0xf6,0xf3,0x0d,0xd0, -0x56,0xd9,0x4f,0x51,0x9a,0x97,0x88,0xaf,0x87,0x5f,0x4e,0xed, -0x37,0x9d,0x88,0xba,0x11,0xdf,0xf5,0x7f,0x81,0xe0,0xe4,0x22, -0x53,0x0b,0xf8,0x47,0xbd,0xf3,0x35,0xbd,0x9e,0x3c,0x1b,0x03, -0x0c,0x60,0x54,0x27,0x64,0x99,0x54,0xab,0x2e,0x08,0x3d,0xd8, -0x67,0x7a,0xea,0x30,0xb1,0xf1,0xc1,0x9e,0x95,0x71,0x98,0xbf, -0xc9,0x2a,0xe3,0x29,0xfc,0x89,0x6a,0xe5,0x4e,0x4c,0xea,0x58, -0x59,0x85,0x50,0x80,0xdf,0xfe,0x86,0x12,0x30,0x53,0x19,0xf5, -0xbf,0x62,0x7a,0xe1,0xf9,0xe5,0xf4,0x05,0x3d,0xb8,0x49,0x6f, -0x3e,0x1a,0x0f,0x5b,0x62,0x81,0x86,0xdc,0xbe,0x1f,0x10,0x6c, -0x51,0x70,0xf0,0x3b,0x2a,0x2a,0x8d,0x03,0x5f,0xe7,0x16,0x25, -0xe4,0x5b,0x3c,0x38,0xc6,0x87,0xb2,0x1f,0x9f,0x8e,0x41,0x86, -0x3a,0xc5,0xa8,0xd3,0x01,0x1e,0x41,0x10,0xa1,0x4f,0x4f,0x98, -0x9e,0x0f,0xfa,0x9b,0x07,0xa7,0xe3,0xe1,0x69,0xf7,0x63,0xf2, -0x8b,0xee,0x1a,0x43,0xb1,0xe0,0xe8,0xaa,0x10,0xc8,0x96,0x1a, -0xbf,0xf1,0x0b,0x13,0x68,0x76,0x9d,0xde,0x90,0xb8,0x87,0x6e, -0x5c,0x27,0x0a,0x7b,0x4b,0x6f,0xa2,0x6d,0x7c,0xec,0x76,0x60, -0xea,0xda,0xe8,0xe1,0x5e,0x52,0xbe,0xcb,0x0b,0xaf,0xa7,0x60, -0x22,0x74,0x69,0x78,0xe2,0x25,0x92,0xad,0x16,0x9f,0xc8,0x27, -0x41,0xbf,0x6b,0xc3,0xe1,0x6e,0x1e,0x71,0x0d,0x98,0xb2,0x61, -0xc8,0x10,0x7f,0xcd,0xa4,0x6e,0x09,0xf4,0xac,0x15,0x85,0x9c, -0x57,0xd0,0x12,0xd3,0x06,0x6d,0x67,0x48,0x4e,0x11,0x03,0x26, -0xf5,0x4a,0xba,0xed,0xde,0x8a,0x5b,0xff,0x14,0xba,0xc3,0x30, -0x5e,0xd4,0x69,0x45,0x27,0x6f,0x3e,0x7a,0xf3,0x0e,0x1f,0x81, -0xbc,0x89,0xcc,0x37,0x30,0x99,0x22,0x03,0x44,0x3a,0x1c,0xc7, -0xc8,0x72,0xfb,0x99,0x6a,0x3c,0x64,0xb6,0x12,0xbe,0xf9,0x18, -0x19,0x67,0x07,0x0f,0x7d,0xfc,0x20,0xbb,0x98,0xad,0x26,0x43, -0x7a,0x69,0x0d,0x4c,0x67,0x68,0x8a,0xcf,0xad,0x3f,0x8e,0x4e, -0x01,0xdf,0x53,0xf5,0xe0,0x94,0x9a,0x78,0x1a,0x7f,0xec,0x30, -0xa8,0xca,0x71,0x1b,0xb4,0x47,0xec,0x28,0xdb,0xae,0x49,0xed, -0xfb,0x7d,0xc3,0x79,0x5e,0xf7,0x1a,0x1e,0xe4,0xc7,0x28,0x20, -0xbe,0x7d,0x0e,0xf3,0xf3,0xdb,0x6f,0xd0,0x98,0xea,0xd6,0x3f, -0x4a,0x78,0x66,0xd0,0x8e,0x70,0xf0,0x5b,0xd1,0xde,0xc4,0xbb, -0xb6,0x40,0xc5,0x12,0x34,0x1a,0xb7,0x61,0x5b,0x38,0xd8,0xa1, -0xec,0x2f,0xdd,0x39,0x11,0x50,0x96,0x41,0x4d,0xf7,0x29,0x0b, -0x10,0xba,0x37,0x53,0x7d,0x44,0x88,0x14,0x87,0xc7,0xb3,0x55, -0x46,0xb6,0x78,0xf4,0xe4,0x86,0x17,0xc2,0x58,0x83,0x83,0xb3, -0x0d,0x7f,0xb7,0x20,0xb3,0x4b,0xdd,0xb6,0x75,0xc1,0xf8,0x62, -0xfc,0x76,0xe4,0xe2,0x1b,0xfb,0x55,0xa5,0x36,0xe8,0x0d,0x27, -0xe7,0xfb,0xf6,0x36,0x64,0x51,0x7e,0x99,0x7f,0x77,0x76,0x9e, -0x3a,0x83,0x0a,0x01,0x3b,0xc7,0x5c,0xb7,0x83,0x99,0xfd,0x0c, -0xfc,0x04,0x12,0x3b,0x1f,0x1e,0x96,0x3e,0x2a,0x1d,0xe0,0xc8, -0xb4,0x86,0x0c,0x40,0x94,0xf7,0x60,0x8b,0xcf,0x47,0xc8,0xdd, -0x0e,0x12,0xd0,0x2d,0x0d,0x88,0xe5,0x58,0xb1,0xd3,0x2b,0x69, -0x90,0x4b,0x3a,0x50,0x3c,0xfd,0xfb,0x54,0x95,0xc4,0x1d,0x45, -0x78,0xe4,0x37,0xda,0xe2,0xc7,0x9f,0xdd,0x2d,0xd2,0x5f,0x36, -0x63,0x8e,0xd6,0xaa,0x6c,0x49,0xdc,0x92,0x78,0x60,0x3a,0x94, -0xb5,0xe5,0x47,0x8a,0x3f,0x1e,0xe6,0x15,0xa7,0x00,0x1b,0x13, -0x33,0x44,0x76,0x69,0x9f,0x9b,0xfc,0x94,0xcb,0xde,0x21,0x59, -0x4d,0xc1,0x38,0x26,0x86,0x4b,0xdd,0xcc,0x27,0x26,0xb5,0xab, -0xc4,0x58,0xc1,0xae,0x64,0x97,0x1d,0x9d,0xf8,0x94,0xb9,0xd8, -0x27,0xa3,0x34,0x8a,0xd3,0x8c,0x85,0x07,0x17,0xc2,0x1e,0x2c, -0x79,0xf2,0xb1,0xe6,0x33,0x5d,0xc0,0x76,0x70,0x12,0xe8,0xf8, -0x86,0x7e,0x0e,0x0f,0xe9,0x87,0xe5,0x71,0xb6,0x42,0x15,0x79, -0xcd,0xe0,0xe4,0xb1,0xc6,0x96,0xef,0xb1,0xfc,0xc9,0x6d,0xb8, -0x9a,0xe3,0xf3,0xbf,0xd2,0x17,0x2b,0x74,0x66,0x04,0xb2,0x51, -0x46,0x76,0xe6,0x1f,0x2c,0x28,0x17,0xcc,0x5b,0xbd,0xc5,0x88, -0x8c,0xd6,0xf4,0x26,0x28,0x56,0x0f,0x6b,0x0f,0x7e,0xc8,0x46, -0x0f,0x3e,0xd6,0x2b,0xa5,0x3b,0x83,0x3d,0x58,0xce,0x1e,0xb0, -0x27,0x93,0xcd,0x83,0xd5,0x74,0x0c,0x9b,0x49,0x9c,0x9e,0xb3, -0xda,0x83,0x97,0x02,0xab,0xf5,0x00,0x66,0x36,0xf5,0xc0,0x54, -0x85,0xc9,0xa2,0x58,0xe8,0xcc,0x83,0x96,0x24,0xed,0x62,0xea, -0xde,0x80,0x44,0x60,0x32,0x6d,0x77,0x76,0xd0,0x41,0xb2,0x75, -0x86,0xa2,0x4b,0x97,0x78,0x1f,0xdf,0xf0,0x70,0x8c,0x0a,0x78, -0x86,0x38,0x33,0x38,0x29,0x62,0x0a,0xcb,0xec,0x95,0xd9,0xdd, -0x80,0x1f,0x5f,0x60,0x3a,0x58,0x4f,0x78,0x56,0xec,0x14,0x07, -0x49,0xd2,0xa5,0xc1,0x29,0x56,0x4c,0xd3,0x8a,0x0c,0x44,0xb5, -0x6f,0xb4,0x16,0x8f,0xd2,0x9b,0x7f,0xc9,0x30,0x65,0x22,0x15, -0xf6,0x92,0x0a,0x58,0x53,0x8e,0x73,0xf9,0x30,0xc8,0x69,0xa3, -0x59,0x50,0x52,0x77,0x41,0x68,0x0f,0x67,0x0f,0x4c,0x82,0xfe, -0xf0,0x2c,0x80,0xbe,0xbb,0xc0,0x33,0x1e,0x9d,0x84,0xca,0x0e, -0xfc,0xe5,0x12,0x34,0x6e,0x07,0x48,0xa4,0x06,0x16,0x1d,0xd7, -0x70,0x47,0x92,0x29,0xb4,0xa0,0xbf,0xe8,0x4c,0x49,0xec,0x49, -0xba,0x78,0xc1,0x6e,0xc6,0x5b,0xb7,0xa4,0xd8,0x94,0x4c,0x4c, -0xb2,0x61,0xef,0x5d,0x87,0xde,0x6d,0x9b,0xb0,0x4b,0xde,0x0a, -0x85,0x38,0xa7,0x44,0x91,0x08,0xe7,0x9e,0xce,0xbb,0xc2,0x99, -0x1b,0x4f,0x56,0x62,0x52,0xf4,0x1f,0x92,0x4f,0x43,0xe6,0xe6, -0x68,0xb4,0xcd,0x6e,0x17,0x8e,0x6a,0xe2,0x65,0xbb,0x1c,0x0f, -0xd1,0x64,0xff,0x41,0x94,0x83,0x7b,0x7b,0x1b,0x82,0xf3,0x91, -0x9a,0x0d,0x87,0x41,0x31,0xbc,0xbb,0x41,0x6c,0xc4,0xcd,0x74, -0x1c,0x72,0xc9,0xed,0x2d,0xef,0x82,0xf6,0x5d,0x0b,0x14,0xde, -0x01,0xdc,0xb7,0xc3,0x38,0xb7,0xc3,0x7f,0xa9,0x53,0x53,0xc8, -0x2d,0x94,0x19,0x45,0x12,0x9f,0x3b,0x3c,0x20,0xd2,0xf7,0x92, -0x97,0xfe,0x3a,0xe6,0x06,0xbd,0xb9,0x29,0x98,0x8d,0x61,0x2b, -0xeb,0x89,0x67,0x6e,0x5e,0x7b,0xc4,0xf6,0xea,0x62,0xf6,0xee, -0x03,0x2f,0xb6,0x3e,0xf0,0xe1,0x67,0x8d,0x2b,0xc9,0xdd,0x26, -0x69,0xb0,0x27,0xe1,0xcd,0x4e,0xa7,0xe4,0x5c,0x04,0x97,0x5a, -0x25,0x7d,0x25,0x5f,0xea,0x5a,0xa3,0xb6,0xd3,0xf3,0xea,0x05, -0x9a,0x14,0x30,0xd7,0xe3,0xbf,0x1f,0x07,0x37,0x2f,0xff,0x8a, -0x76,0x60,0x25,0x1f,0xd2,0x0e,0x83,0x79,0xcb,0x6b,0xd2,0xbe, -0x76,0x90,0xbf,0xb3,0xbf,0x43,0xa1,0xa5,0x90,0xec,0x08,0xcb, -0x3b,0xf7,0xa0,0x98,0x0c,0xdd,0x5c,0xf3,0x57,0x7c,0x63,0x3e, -0x49,0x9f,0x8b,0x3f,0xd9,0xc9,0xb5,0x8e,0xd6,0x4b,0x8e,0x89, -0xf2,0x28,0xa0,0xc8,0x5b,0x9a,0x3d,0x4c,0x19,0xa0,0x63,0x6b, -0xaa,0x97,0x74,0x17,0x94,0x01,0xea,0x78,0x76,0x63,0xa7,0xed, -0x82,0xc1,0x16,0x8f,0x67,0x48,0x70,0x78,0x45,0x56,0xb2,0x3e, -0xf4,0x96,0xd5,0x3f,0x09,0x75,0x94,0xe0,0x18,0x9c,0xf7,0x9a, -0xd3,0x3e,0xae,0x09,0x35,0xbb,0xf1,0x34,0x0f,0x0f,0x25,0xa1, -0x0f,0xee,0xd4,0x9a,0xd3,0xa0,0x9f,0x39,0xcf,0x39,0xe9,0xf2, -0x6b,0xc4,0x8e,0x97,0x5e,0x9b,0x43,0x9b,0xcc,0x38,0x66,0xd4, -0x26,0xa1,0x15,0x75,0x10,0x3e,0x8c,0xd0,0x97,0x64,0x0c,0x4b, -0xee,0xc8,0x40,0x3c,0xd1,0x25,0x68,0xca,0x20,0x81,0xdf,0xeb, -0x5a,0x53,0x3e,0xc7,0x87,0xae,0xc3,0xa4,0xf1,0x38,0xf5,0xe1, -0x18,0xa3,0xc8,0xe3,0xe3,0x31,0x59,0x46,0xf6,0x93,0x4f,0xc6, -0x5d,0xf7,0x4a,0x2b,0x10,0x42,0xc2,0x46,0x90,0x07,0xa5,0x30, -0x12,0x5f,0xaf,0xe4,0x70,0x57,0x51,0x01,0x51,0xd2,0x3c,0x01, -0x99,0xfe,0x27,0xa5,0x03,0x71,0xc6,0x89,0x4a,0xba,0x3b,0x33, -0x75,0x4a,0x68,0x74,0x2e,0xe7,0x3e,0xbd,0xc6,0x17,0x15,0x56, -0x46,0x2a,0xea,0x0f,0xd7,0x18,0x4b,0x90,0xe8,0x77,0x62,0xc1, -0x8d,0x55,0x40,0x51,0xda,0x2e,0x79,0x71,0xb1,0xca,0x43,0xad, -0x39,0x77,0xd1,0x5e,0x66,0x67,0xdd,0x81,0x66,0x3d,0x77,0x6f, -0x32,0xf9,0xa2,0x27,0x2d,0x00,0x26,0x7c,0x98,0x23,0x3e,0x23, -0xe3,0x02,0xca,0x5d,0x6a,0xd9,0x34,0xc5,0xf0,0xed,0x75,0x36, -0x87,0x49,0xe3,0xc1,0x1e,0x80,0x32,0xf5,0xf1,0x2c,0x7c,0xd7, -0x15,0xc8,0xde,0xb3,0xc9,0x47,0x8d,0x86,0x1d,0x8c,0x3f,0xbb, -0xa3,0x71,0xcf,0x19,0x65,0xb6,0x57,0xb5,0x61,0xbf,0x16,0xb4, -0xe5,0x20,0x7d,0xa5,0x63,0x5b,0xd0,0x4d,0xef,0xce,0x82,0xee, -0x2f,0xd4,0xdd,0xd9,0x98,0xd2,0x6e,0xe3,0xed,0xa7,0xc6,0x4d, -0x3b,0xf6,0x33,0x16,0x22,0x64,0x66,0xe3,0x37,0xcc,0xff,0xaf, -0x22,0x27,0x8c,0xb5,0x34,0x08,0xe7,0x08,0x05,0x71,0x7f,0x17, -0x55,0x2c,0x5f,0xef,0xbc,0xe3,0x73,0x67,0x6e,0x0b,0xd1,0x7b, -0xe0,0xef,0xb4,0x90,0x1f,0x80,0xd8,0x98,0xd4,0xf9,0xce,0x4d, -0xec,0x85,0xc7,0xde,0x99,0xf7,0x46,0x7f,0xe0,0x98,0x2d,0x70, -0x86,0x71,0x4e,0x60,0xf4,0xd4,0xf9,0xd0,0x66,0x89,0x7d,0x23, -0xc6,0x46,0x4b,0x38,0x23,0x2b,0xda,0x67,0x83,0xc5,0x78,0xee, -0x5f,0x2c,0x04,0xca,0x21,0x4e,0xe3,0xdc,0x84,0xe2,0x87,0xb4, -0xf7,0x3e,0xff,0x67,0xd3,0x22,0x78,0xd7,0x51,0x07,0xa1,0x43, -0x43,0x2e,0xa5,0xa9,0x7d,0x35,0x2b,0x1e,0x25,0x39,0xe5,0x87, -0xc5,0x84,0x93,0xf0,0x85,0x14,0x94,0x4a,0xed,0x11,0x1a,0xdd, -0x98,0xb4,0x7d,0xec,0x80,0x03,0x96,0x91,0x53,0x56,0xb1,0x6f, -0x41,0x99,0x4c,0xbe,0x63,0x9f,0x85,0x05,0x4f,0x1f,0x1c,0xfc, -0xb7,0x7a,0xfe,0xf4,0x88,0xe3,0x28,0xc6,0xe4,0x94,0xc9,0x8c, -0xb2,0x8b,0x68,0x95,0xc9,0xad,0xcb,0x77,0xdf,0xbf,0xfe,0xfa, -0xbb,0x6f,0x5f,0xf9,0x2f,0xb4,0xf1,0x6a,0x20,0xea,0xb4,0xde, -0x64,0x95,0x5e,0xa6,0xaf,0x0a,0xe2,0x0e,0xc7,0x9c,0x2f,0x66, -0xab,0xb9,0xbd,0x2b,0x40,0x53,0xb3,0xf0,0x01,0xeb,0x08,0x06, -0xcd,0x49,0x2f,0xfd,0x7d,0x33,0xec,0x96,0xc9,0xc6,0x8f,0x18, -0x9a,0x0f,0x52,0xfc,0xf3,0x5f,0x2f,0xfe,0x0d,0xbd,0x06,0x77, -0x2e,0x2a,0xb8,0xee,0xe0,0x9e,0x82,0xb0,0x42,0x1d,0x70,0xd8, -0x70,0x7c,0x63,0x5e,0xb3,0xdc,0x78,0x8f,0xc3,0xc4,0xb2,0xc6, -0xbe,0x77,0xd5,0x27,0x25,0x26,0x25,0x90,0xcd,0x3c,0xd6,0xe8, -0x06,0xa2,0x90,0xf3,0x82,0xd8,0x3c,0x39,0x96,0xe7,0xc5,0x96, -0xc4,0x66,0x78,0xc8,0x70,0x91,0x2b,0x74,0x6d,0x40,0x84,0x3b, -0x56,0xad,0xa6,0x6f,0xa7,0xb3,0x77,0x53,0x09,0xa1,0x41,0x33, -0x76,0x7a,0x25,0xbe,0x2a,0xf5,0x49,0xeb,0xb9,0x6d,0x91,0xd7, -0x3e,0x5c,0xcf,0xce,0x48,0xc7,0xb6,0x3f,0x5b,0x4d,0x87,0x7c, -0xfd,0x93,0xd9,0x77,0xa0,0x98,0x3a,0x9e,0x8e,0x1d,0xed,0x1c, -0x07,0xd4,0xa9,0xb2,0x88,0x9c,0xfa,0x98,0x9c,0xc6,0x37,0x5e, -0x9d,0xc5,0x85,0xd2,0x9d,0xe5,0x53,0x1f,0xda,0x96,0x31,0x01, -0xa9,0x5e,0x92,0x43,0x11,0xcc,0x36,0x57,0x2e,0x22,0xf1,0x0d, -0x9e,0x53,0xab,0xfb,0x9c,0xca,0xbc,0xb3,0x32,0x93,0x05,0x0a, -0x32,0xf2,0xa4,0xcd,0xab,0xd7,0x2c,0xd4,0x87,0x87,0x7e,0xbc, -0x36,0xb3,0x1e,0x0b,0x62,0x1c,0xde,0x81,0x9b,0x36,0xc5,0xcb, -0xc9,0x56,0x89,0x4e,0x8e,0x27,0x73,0xa8,0xdf,0x85,0xb7,0xf8, -0x1a,0xc0,0x77,0x60,0x3f,0xb8,0x58,0x99,0xe7,0x16,0x1a,0xa7, -0x5c,0x0e,0x0f,0x35,0x66,0x25,0x69,0xd7,0x1f,0xd0,0x43,0x73, -0xe7,0x41,0xa9,0xe2,0x5c,0xd5,0xc4,0x95,0xd2,0x83,0x4e,0xa9, -0xed,0xb7,0x1c,0x49,0x65,0xcb,0x38,0x13,0xae,0xb8,0x8a,0x8a, -0xbc,0xec,0x4e,0x7a,0x58,0x30,0x20,0x34,0x1a,0xcf,0x29,0x69, -0xe1,0x8a,0x54,0xaa,0xa5,0x19,0x86,0x70,0xbb,0x9e,0x29,0x59, -0xfa,0x85,0x07,0x7e,0xfa,0xba,0x49,0x9f,0xd6,0xec,0x7c,0x48, -0xb6,0x93,0x5e,0x34,0xdc,0xe3,0xfb,0x6c,0x4d,0xe4,0x29,0x9c, -0x79,0xd0,0xff,0x6a,0x3c,0x3d,0x9f,0x8c,0x24,0x27,0xb7,0xcb, -0xa8,0x35,0xd9,0xee,0x95,0x14,0x7a,0xab,0x76,0xe3,0x75,0xbb, -0x3c,0xb2,0x0e,0x6f,0xfa,0x0d,0x33,0x14,0xf3,0x48,0xdb,0xc2, -0x73,0x39,0xce,0x3c,0xfc,0xe9,0x44,0x1f,0xc6,0xdf,0x3e,0xe6, -0x3c,0x4e,0x0c,0x30,0xf1,0x0a,0x6e,0xdf,0x0d,0x8d,0xf0,0x79, -0xd8,0xee,0x0e,0x8d,0x5b,0x9e,0x17,0x5a,0x03,0xc7,0x2b,0xd8, -0xc9,0xd7,0x08,0xeb,0x7a,0xcb,0x69,0x57,0x21,0xe7,0xdb,0x47, -0x5b,0x01,0xca,0xd8,0xe7,0xbc,0x3d,0xf1,0x7b,0x7c,0x87,0x4e, -0xf2,0x07,0x92,0xc7,0xeb,0x05,0x77,0xa1,0x88,0xf2,0xcd,0x88, -0x59,0xb1,0xd9,0x70,0xc8,0x37,0x62,0x6b,0xac,0x90,0x47,0xec, -0xfb,0x30,0x3c,0x49,0x42,0xb0,0xed,0xc1,0x1e,0xbe,0xc0,0x1b, -0xcb,0x0c,0x35,0xf1,0x1e,0x88,0xb3,0x99,0xdc,0x63,0xb6,0xb6, -0xbf,0x37,0xd3,0x88,0xa1,0x8b,0xa5,0xa8,0xc4,0x43,0xc8,0x55, -0x4c,0xdf,0xe1,0xd9,0x38,0xb5,0x0a,0x75,0x54,0x25,0x69,0x30, -0xba,0x0e,0x8e,0xa9,0x91,0x3b,0x95,0x55,0x60,0x89,0xec,0x4d, -0xb2,0x48,0x37,0x49,0x85,0x48,0xde,0xde,0x9a,0xb4,0x94,0x4c, -0xe2,0x84,0x19,0x72,0x8f,0x60,0x43,0x0e,0xfd,0xc0,0xee,0xa6, -0x3d,0x1f,0x9e,0x8f,0xca,0xf5,0xde,0x3f,0x4e,0xa1,0xc3,0x43, -0x82,0x97,0x77,0xfd,0x4c,0x6e,0x02,0xf3,0x7c,0x42,0xb9,0x03, -0xce,0x10,0x99,0x6d,0x0f,0x63,0x98,0xac,0x9a,0x12,0xd4,0x12, -0xae,0xee,0x2b,0x14,0xa9,0xc8,0x06,0x99,0x13,0x96,0x6f,0x35, -0x5a,0xd3,0x83,0xf4,0xf3,0xef,0xd9,0x8f,0x87,0x0e,0x4a,0x8d, -0x41,0x58,0x91,0x74,0xf6,0xc2,0xbd,0x5d,0xe4,0xa8,0xaf,0xe9, -0xc4,0x99,0xcf,0x9d,0x75,0xc3,0xe9,0xca,0x4e,0x63,0xa9,0xa7, -0xc6,0x49,0xaf,0x0f,0x52,0x84,0x83,0x47,0x96,0xde,0x94,0x4a, -0x2d,0x74,0x77,0x18,0x20,0x9b,0x81,0xc4,0x09,0x72,0x92,0xf3, -0x92,0x3a,0x7c,0x5a,0xcd,0x37,0x15,0x19,0xaf,0x82,0xd9,0x97, -0x32,0x0b,0xc7,0xb4,0x29,0xc2,0xdb,0x28,0x3c,0x3a,0x47,0x48, -0x9d,0x8c,0xbc,0xf6,0xe2,0x1d,0x07,0xaf,0x00,0x59,0xcc,0xcf, -0xbd,0x32,0x31,0xf5,0x4a,0x1a,0x70,0xfa,0x40,0x60,0xb4,0x4c, -0x0f,0x12,0xbc,0x77,0x36,0xbe,0xc0,0x51,0xb3,0x16,0x25,0xc3, -0x2f,0xa7,0x87,0x87,0xfa,0xa5,0xa1,0xc5,0x22,0xa6,0xd5,0xc2, -0x94,0x75,0x87,0xdc,0x49,0x37,0x36,0xfe,0xd0,0x08,0xc2,0xd7, -0x72,0xbb,0x6a,0x03,0x4f,0x2c,0x20,0x7d,0x2e,0x64,0x13,0x2b, -0x95,0x78,0xe2,0x68,0x6d,0xf0,0x4b,0x0e,0x5b,0xe0,0xc4,0xe6, -0xec,0xba,0xf8,0xb3,0xfc,0xcf,0x18,0x0b,0xcb,0x68,0xab,0xdc, -0xc5,0x85,0xc5,0x4f,0xe9,0xae,0x66,0x38,0x6d,0x75,0xaf,0xa6, -0xe4,0xf6,0x29,0x7d,0x9b,0xbb,0x71,0x2e,0xb8,0x90,0xca,0xb7, -0x83,0x7b,0xe0,0x44,0xf7,0xd1,0xc4,0xe8,0xaf,0x70,0xf2,0x5b, -0x7b,0x23,0xd5,0x92,0x6b,0xa6,0x80,0x41,0x52,0xe2,0xbc,0xcf, -0x37,0x61,0x23,0x49,0x87,0x3d,0x72,0x32,0xa7,0x2e,0xbb,0x9d, -0x04,0x50,0xf0,0x9d,0xec,0xfd,0x33,0xe3,0x39,0x5a,0xc8,0xa7, -0xee,0xc8,0x36,0xb1,0x31,0xd0,0x5a,0xf3,0x8f,0x99,0x2b,0xdd, -0x3e,0x92,0x15,0x43,0xba,0xa9,0x53,0xd8,0x5d,0x2d,0xbd,0x35, -0xf3,0x62,0xe3,0x03,0xfb,0x16,0xc3,0x5c,0x3c,0x4b,0x79,0xf7, -0xe9,0x3d,0x5e,0xf9,0xdc,0x6c,0xdb,0x26,0x50,0xcc,0x44,0x86, -0x71,0x52,0x1f,0x05,0x53,0x0c,0x8d,0x8a,0x17,0x62,0xe7,0x33, -0x8d,0x03,0x32,0x4d,0xd3,0x42,0xc4,0xdb,0x2e,0x83,0x39,0xdf, -0xb7,0xb7,0x3a,0xb0,0xa5,0xe9,0x21,0x1d,0x8e,0x33,0xd8,0xbb, -0xe6,0x3a,0x96,0x27,0x8f,0x94,0xff,0x06,0xcf,0x37,0x73,0xf3, -0x8a,0x1c,0x83,0xe8,0xfb,0xe3,0x9d,0x24,0xd6,0xfc,0x97,0xe3, -0x37,0xae,0x4d,0x00,0x6b,0x04,0x5b,0x06,0x53,0xd6,0xc1,0xd0, -0x7d,0x0c,0x3c,0x64,0x77,0x2e,0xda,0x4b,0x82,0x43,0x92,0x8e, -0xc7,0x55,0x30,0xaf,0x99,0xc7,0xe9,0x88,0x21,0xf0,0x2b,0x57, -0x06,0x1f,0xa6,0xa6,0x03,0x87,0x42,0xdb,0xb8,0xe5,0xd1,0x6b, -0x3f,0xb4,0x4e,0x21,0x38,0xd8,0xd3,0xc7,0xce,0x64,0x9d,0x36, -0xdc,0xc9,0x3c,0xcd,0x31,0xb5,0x4c,0x46,0x4e,0x9e,0x63,0x5b, -0xb8,0x6d,0x3f,0x71,0x7c,0x23,0x25,0xc2,0xc1,0x19,0x02,0x3c, -0xb1,0x45,0xbc,0x21,0xb4,0x7f,0xbc,0x79,0x2b,0x05,0x1f,0x6e, -0x30,0x62,0x4f,0x52,0x0b,0xaf,0x13,0xf9,0x8b,0x58,0xaa,0x6f, -0x40,0x5a,0x50,0x9a,0x8a,0xea,0x37,0xad,0x35,0x3a,0xfd,0x89, -0x62,0x31,0x2a,0x4b,0x54,0x2a,0x85,0x03,0x5a,0xf8,0xac,0xe5, -0x32,0x13,0xc5,0x6c,0xc3,0xd5,0x31,0x3d,0xf1,0x2b,0x2e,0x42, -0x97,0x1d,0x03,0xfb,0xe5,0x42,0x19,0x45,0x8e,0xa6,0x7c,0x60, -0xfa,0x80,0x19,0x6d,0xfb,0x84,0xf5,0xe6,0xaa,0xf1,0xa8,0x1b, -0x50,0x23,0x28,0x8c,0x97,0x9c,0x41,0x4d,0xd4,0xba,0x83,0xb0, -0xcb,0x40,0x0a,0x2f,0x44,0xc8,0x27,0x5e,0x11,0xa8,0x10,0x52, -0x1c,0xbb,0xb2,0x00,0xed,0xf1,0x55,0xa0,0xdf,0xe0,0x11,0x7b, -0x8f,0x72,0x03,0xe7,0x73,0xb3,0x6b,0x25,0x84,0x80,0x48,0xb9, -0x66,0x33,0x84,0xa4,0x0b,0x5b,0x1e,0x17,0x19,0x3f,0x9b,0x6e, -0x63,0xd0,0xd3,0x9a,0x42,0x52,0x2f,0x07,0x0f,0x0f,0x1d,0x38, -0xac,0xb9,0x5c,0x54,0xca,0x2b,0x13,0x87,0x80,0xc7,0x43,0x0b, -0x15,0xed,0x47,0xb9,0x20,0x71,0x8f,0x90,0xcb,0x6e,0x33,0x7b, -0x44,0xc5,0xa3,0x40,0x3d,0xd6,0x2d,0x44,0x2b,0x83,0xba,0x70, -0xc3,0x9d,0x58,0x50,0x87,0x99,0xd1,0x03,0x20,0xc8,0x88,0xb4, -0xc1,0x73,0xd0,0xb5,0x93,0x5e,0x67,0xe4,0xf4,0x2e,0x47,0xb5, -0xf4,0xa1,0xa7,0x94,0x08,0x47,0x61,0x4c,0x2d,0xb4,0xed,0x11, -0xc6,0xb2,0x88,0x85,0x08,0x30,0x91,0xf7,0xd1,0x33,0x2b,0x1a, -0x6f,0x76,0x3e,0xf0,0xa4,0xd0,0x96,0x0b,0x48,0xc1,0xc4,0x69, -0x30,0xb1,0xb3,0x66,0x80,0xc1,0xd6,0xa3,0x79,0xc7,0x25,0x2a, -0x39,0xc0,0x33,0x3a,0x3f,0xad,0x5d,0x43,0x47,0x46,0xba,0xbe, -0x19,0xf2,0x38,0x51,0x22,0xad,0x8c,0x55,0xa9,0xb4,0xc3,0xe6, -0x79,0x83,0x22,0xce,0x35,0x1e,0x3a,0x36,0x36,0x95,0xe9,0x4d, -0x29,0xad,0x54,0xed,0x5d,0x33,0xa9,0x3b,0xad,0xc7,0x05,0xf3, -0x17,0x82,0x44,0xee,0xca,0x41,0x35,0xb6,0x4a,0x48,0x55,0x9b, -0xed,0x8e,0x48,0x4f,0xe3,0x13,0x00,0xd1,0xcf,0xcd,0x9d,0x01, -0xc7,0x71,0xce,0x62,0x9d,0xa0,0x8f,0x36,0xeb,0xee,0xab,0xd5, -0xcd,0x3d,0x72,0xfa,0x20,0xea,0x65,0x0f,0x4e,0xa9,0x07,0x4f, -0xe3,0x0e,0xa6,0x3f,0x10,0xc5,0xdb,0xb8,0x23,0x9a,0xb7,0x79, -0xb5,0x5d,0x5f,0xf9,0x36,0x30,0x7b,0x67,0x1b,0xe0,0xd9,0xbc, -0xf3,0x26,0xec,0x98,0x1f,0x22,0x1b,0x31,0x45,0xab,0x74,0x17, -0x78,0xe4,0x52,0x46,0xe8,0x32,0xda,0xe1,0x3a,0xf5,0x31,0xed, -0x52,0x9c,0xa4,0x47,0x5d,0x65,0x44,0x5a,0x1f,0xa4,0xf6,0xe1, -0x25,0x62,0x55,0x58,0x5f,0x47,0x57,0xd7,0x32,0xb5,0x49,0xde, -0x2c,0xcc,0xfc,0x49,0x57,0xb4,0xcd,0xa5,0xce,0x4f,0x45,0x01, -0x53,0xab,0xbe,0x76,0xbc,0xdc,0x9f,0x76,0x63,0xf2,0x16,0x9d, -0x5f,0xd7,0xd2,0x93,0x13,0x33,0xaa,0xfc,0xfd,0xa8,0x16,0x3d, -0xb6,0xdd,0x6e,0xdb,0xce,0x05,0x78,0xb8,0xaa,0x5f,0xeb,0xd9, -0x58,0x31,0x05,0x1a,0x3b,0x27,0xc0,0x39,0xcb,0x03,0x7c,0x1e, -0x5e,0x90,0x05,0xed,0x9d,0xf9,0xeb,0xa8,0xb9,0xf4,0xf9,0x87, -0x8f,0x00,0xdc,0x8d,0xb2,0x55,0x45,0xe5,0x58,0xdd,0x50,0x96, -0x12,0x9d,0xed,0xaf,0xbb,0x21,0xd6,0xda,0xd2,0x56,0x07,0xaf, -0x68,0x97,0x6a,0xb6,0x9f,0xde,0x7e,0x53,0xd4,0x07,0x50,0x92, -0x3e,0xe9,0xde,0x4b,0x64,0xf3,0x06,0xf4,0xbd,0x64,0xb6,0x40, -0x64,0xbb,0x43,0xa2,0xe0,0xc5,0x3a,0xf9,0xd0,0xc5,0x1a,0x1a, -0x15,0x79,0x34,0x2b,0x58,0xb7,0x8d,0x98,0xe3,0xae,0x60,0x46, -0x87,0xd1,0x2b,0xcd,0x87,0x66,0x7a,0xd0,0xa0,0x02,0x6c,0x7e, -0xcf,0x18,0x6b,0x76,0x8e,0xef,0xda,0x78,0x3b,0x5a,0xda,0x8c, -0xbc,0x05,0xe8,0x6c,0x94,0x1c,0x35,0x48,0x77,0xb7,0x5d,0xb8, -0x97,0x20,0xa5,0xee,0xd8,0xdf,0xbf,0xd3,0x06,0xed,0xc1,0x5e, -0xb8,0xa8,0x5e,0xc9,0xba,0xff,0xc6,0x9e,0x55,0xc1,0xa6,0x6f, -0xcb,0xdb,0x2d,0xa1,0x4b,0xc1,0x61,0xa2,0x02,0xb9,0x3f,0x25, -0x0c,0x98,0x75,0xfc,0x3d,0x22,0x16,0x2c,0x79,0xe9,0xe4,0xad, -0x56,0x27,0x99,0xed,0xe3,0x07,0x53,0xcd,0x6f,0x6e,0xe1,0x4e, -0xff,0x43,0x68,0xc8,0x18,0x8a,0x91,0xc2,0xe2,0x5d,0x24,0xf9, -0xa9,0x13,0x72,0xdc,0x49,0xd8,0xbb,0xb8,0x65,0x5f,0x45,0x05, -0x87,0x73,0x8e,0x95,0x20,0xef,0xf4,0x95,0x7f,0xfc,0x63,0x4e, -0x8e,0x8b,0xb7,0x62,0x74,0x0d,0xed,0xdd,0x59,0xfa,0x3a,0x07, -0xdc,0xce,0x91,0xbf,0x35,0x95,0xe7,0xde,0x95,0xe9,0xd2,0xe6, -0x44,0x81,0x2f,0x83,0x75,0xc8,0x59,0xb2,0xb4,0x22,0x93,0x3e, -0xbd,0x73,0x2e,0xd6,0x0e,0x12,0xe5,0x4b,0x61,0xda,0x87,0x7a, -0xe0,0xc5,0x43,0x78,0xca,0x1e,0x70,0xc6,0x2a,0xdc,0x45,0xdd, -0xa3,0x24,0x4d,0x1e,0x58,0xd6,0xbf,0xc3,0x2b,0x96,0x03,0xd1, -0x61,0x3c,0x2b,0x93,0x02,0xc9,0xcc,0x7b,0x4c,0x1d,0x81,0xe9, -0xe3,0x54,0x87,0x72,0xca,0x53,0xc2,0xd7,0x3a,0x1d,0xb5,0xa7, -0xb4,0x65,0x1f,0x4b,0x62,0x47,0x28,0xb5,0x90,0x46,0xbf,0x46, -0x63,0xcd,0x4d,0x00,0xc5,0xb9,0x50,0xc0,0xfb,0x4d,0xf7,0x5a, -0x53,0xed,0xba,0x5e,0xb5,0x87,0x3c,0xda,0x2d,0x84,0xb6,0x28, -0x67,0x8c,0xaf,0xbd,0x94,0x88,0x98,0x17,0x16,0x9d,0xfe,0x87, -0xbb,0x6c,0xed,0xe5,0x2e,0x12,0x8a,0x6c,0xe3,0x05,0x55,0xca, -0x75,0xe0,0x95,0x73,0x9a,0x28,0x1c,0xbe,0x6d,0x7b,0xf0,0x3c, -0x83,0x6b,0x39,0x94,0x82,0x5b,0x0c,0xc7,0x06,0x9b,0x3e,0x66, -0x28,0x28,0x72,0x1f,0xcb,0x7a,0xb9,0x62,0x51,0xf1,0xa5,0x15, -0x0e,0x30,0xcb,0xd4,0x1d,0xff,0xf4,0x7b,0x94,0xbf,0x34,0x8b, -0x5b,0xba,0x1b,0x3a,0xc1,0x1d,0x4a,0x61,0x66,0xef,0x26,0xae, -0x20,0x87,0x7b,0x07,0xc7,0x5a,0x13,0x8c,0xcc,0x07,0x5b,0x02, -0xd3,0x37,0xf3,0xaf,0xec,0x00,0xbf,0x31,0x37,0xd6,0xda,0xbc, -0xbb,0xbd,0x27,0xde,0xa3,0x4b,0xb0,0xdf,0xc6,0x5c,0x4e,0xb7, -0xc4,0xbd,0x56,0xa3,0x8e,0x91,0xab,0x98,0xfb,0xda,0x1d,0xa3, -0x7d,0x56,0x8c,0x57,0x74,0x6d,0x2f,0xb7,0xe7,0x17,0xe8,0x8a, -0x8d,0xc0,0xf9,0x65,0xe2,0xed,0x7d,0xcd,0x09,0x59,0x22,0x93, -0x08,0x5c,0x12,0x8f,0xf3,0x8e,0x4d,0x68,0xb1,0x72,0x6b,0x66, -0x04,0xa3,0x24,0xe7,0x4c,0x98,0x1c,0xc7,0x36,0xc1,0x7c,0x88, -0x20,0x61,0x49,0xa2,0xf1,0x3b,0x51,0x94,0xcb,0xb9,0x20,0x45, -0xb7,0x0e,0x66,0x0e,0x30,0x1a,0x10,0x9d,0xd1,0x8e,0x6d,0xf3, -0x41,0xe2,0xde,0x42,0xa6,0x7e,0x47,0xfb,0xa6,0xd6,0x3a,0xde, -0xc2,0xe0,0x25,0x85,0x6f,0xa1,0x51,0x21,0x98,0x9e,0x44,0xe7, -0x29,0x2d,0xef,0xa2,0xe5,0x97,0x35,0x83,0xe1,0xc3,0x43,0xdb, -0xe5,0xa7,0x48,0x40,0x39,0xf4,0xb2,0x0a,0x13,0xf6,0x95,0x75, -0xab,0x30,0xd9,0x61,0x9b,0xbb,0xed,0xae,0xee,0x42,0xc0,0x01, -0xa2,0xd5,0xdc,0x33,0x5f,0xa9,0xda,0x28,0x1f,0xed,0xd2,0xa4, -0xda,0xc2,0x14,0x0d,0x92,0xe2,0xff,0xfa,0xf7,0xd5,0x88,0x9c, -0x57,0xcc,0x57,0xfd,0xc9,0x38,0xbb,0x78,0xbe,0x86,0x3c,0x90, -0xe3,0xe9,0xf7,0x5f,0x47,0x20,0xd9,0xaf,0x26,0x3d,0x98,0x83, -0x82,0x75,0x09,0x9f,0xfc,0xf7,0x86,0x9e,0x6c,0x2f,0x59,0xd1, -0xc2,0x84,0xc9,0xa6,0xfa,0xb3,0xd9,0x12,0x2a,0xef,0x91,0x15, -0x8e,0x88,0xbd,0x98,0x2b,0x03,0x44,0x49,0x19,0x2d,0xa8,0x3a, -0xe0,0xf9,0x26,0xf0,0x02,0x2a,0x29,0xc5,0x46,0xdd,0xe1,0xe3, -0x27,0xd4,0xc4,0x07,0xe4,0x77,0x9a,0x95,0xbc,0xd0,0x85,0xdf, -0xf1,0xff,0xc4,0xd7,0x7e,0xd0,0x9d,0x0f,0x4a,0x3f,0xbc,0xfe, -0xb2,0xfa,0x69,0xa9,0x7d,0x32,0x3d,0x7f,0xf3,0xa6,0x45,0x86, -0xd5,0xba,0xea,0x44,0xdb,0x58,0x83,0x4f,0x64,0xbd,0xaa,0x13, -0x5e,0x3b,0x81,0x9a,0xfe,0x54,0x35,0x1b,0x4d,0xb1,0xa8,0xbb, -0x7e,0x23,0x5b,0x52,0x98,0x49,0xa6,0xa3,0x07,0x07,0xe3,0x4b, -0xf4,0xb9,0xdc,0x9b,0x2e,0xdb,0x5b,0xaa,0x0b,0x67,0x72,0x93, -0x85,0x1f,0xae,0x6d,0x9f,0xd4,0x09,0xdd,0xe3,0x8f,0x9d,0x9b, -0x4c,0x21,0x81,0x34,0xdc,0x31,0x0f,0x77,0x53,0x5a,0xe1,0x91, -0x3f,0x75,0x54,0x89,0xdd,0xe7,0x6a,0x0f,0x84,0xf2,0x7e,0x5e, -0x13,0xeb,0xa3,0x8f,0x8c,0x97,0x43,0x9d,0xa3,0x14,0xb7,0x75, -0xea,0xe5,0x6c,0xb8,0x9a,0x8c,0xf8,0x95,0xbd,0x24,0x2a,0xb4, -0x46,0x5c,0xea,0x02,0x51,0xc5,0x01,0x89,0x53,0x50,0x39,0xd6, -0x9b,0xcc,0xcb,0xff,0x5f,0xf3,0xd6,0x98,0xe4,0xe5,0xff,0xaf, -0xce,0x75,0xe8,0x68,0x4a,0x3e,0xb4,0x17,0xe3,0x57,0xa3,0x73, -0xea,0x3d,0xd2,0x73,0x91,0x39,0xca,0x24,0x32,0x73,0xa1,0x0a, -0x3d,0x4c,0x79,0xc6,0x2c,0x43,0xfd,0x77,0xcd,0xc7,0xf5,0xf3, -0x31,0xba,0xb3,0x74,0x23,0x8f,0xbe,0xa0,0xc8,0xd4,0x8b,0x6c, -0x7e,0x4e,0x91,0x95,0x92,0x73,0x6b,0x5b,0x00,0x7e,0x0e,0x42, -0x3b,0xc5,0xbe,0x9a,0x43,0xb9,0x2c,0xc4,0xc4,0x75,0xf5,0x8d, -0x88,0x3a,0x15,0x3c,0x6c,0x50,0x05,0xff,0xd3,0x47,0xe5,0x29, -0x45,0xb6,0x7c,0x54,0x1e,0xd6,0xcf,0x81,0x3e,0x7e,0xdc,0x33, -0xca,0xa8,0xfc,0x48,0x00,0x19,0x62,0xd4,0x29,0x41,0x34,0x00, -0xf4,0x1a,0xf2,0x72,0xb6,0x82,0x61,0xa5,0x35,0x0a,0x95,0xb8, -0x88,0xce,0x44,0x91,0x49,0xc7,0xa7,0x26,0x03,0x45,0xeb,0x5c, -0xa9,0xfe,0x20,0xbb,0x23,0x94,0xb4,0x5a,0x4c,0xbe,0x47,0x1f, -0xd0,0xa4,0xe5,0x6f,0x6b,0x91,0xee,0xd6,0x1e,0x68,0xd0,0xd7, -0x3a,0xb9,0x8a,0x16,0x65,0xdd,0x4c,0xb1,0x7f,0x26,0x57,0xa9, -0x06,0x46,0x17,0xec,0x22,0x96,0x0c,0x2d,0xc2,0x81,0x23,0x99, -0x38,0x46,0x16,0xbc,0x21,0x6d,0xb1,0xb5,0x11,0x63,0x27,0x3d, -0x15,0x6f,0x3b,0x80,0x58,0x51,0x4d,0x02,0xc4,0x68,0xb1,0xbb, -0xb1,0x81,0xce,0x3f,0x5b,0xfe,0x18,0x67,0x5f,0xfa,0xd2,0x1c, -0x88,0x85,0xce,0x0e,0x03,0x4f,0x2c,0x00,0x19,0x36,0x0a,0xe2, -0x09,0x6b,0xfc,0x12,0xf8,0xf0,0x10,0xfa,0x57,0x2e,0xcf,0x1c, -0x2f,0xe7,0xec,0x4b,0x94,0x5b,0xc7,0x69,0xd9,0xaa,0x0f,0x83, -0x30,0x42,0xbb,0xff,0x2c,0xde,0x11,0x5c,0x4b,0x4a,0x21,0xf9, -0x4b,0x31,0xab,0xf4,0x35,0x2e,0x38,0x68,0x4a,0x1d,0x24,0x2e, -0x6b,0x30,0x4a,0xe8,0xa1,0x23,0x6a,0x7a,0xb8,0xe5,0x7b,0xc4, -0x18,0x9d,0x43,0x0d,0x0c,0xfa,0x36,0x7d,0xc9,0xc7,0x80,0x0b, -0x64,0x10,0xba,0x33,0x66,0x56,0x71,0x3d,0xdb,0x4b,0xa7,0x39, -0xa4,0xfe,0xe2,0xf9,0x97,0x4f,0x7f,0x78,0xf1,0xfa,0xf4,0xe9, -0x33,0x3a,0x17,0x34,0xdd,0x1a,0xd0,0xd9,0x21,0x31,0x47,0x28, -0x6b,0x1c,0xf1,0xa2,0x97,0x7d,0x3e,0x1b,0x6e,0xd0,0x66,0xef, -0xf7,0xdf,0xbd,0x7a,0x7d,0xfb,0xfd,0x0f,0xf0,0x0f,0x3d,0xeb, -0xc7,0x1f,0xd5,0xc7,0x62,0x30,0x86,0x77,0xa9,0x97,0xa3,0xe5, -0xc5,0x6c,0x18,0xb7,0x75,0x5b,0x4e,0xa6,0xb4,0xf9,0xb3,0xa0, -0x13,0xd5,0x6b,0xaa,0xde,0x91,0xea,0x3d,0x14,0xbb,0x8b,0x48, -0x66,0xb1,0xb3,0xa5,0xc8,0x20,0x13,0xf3,0x06,0xf1,0xd0,0x7e, -0xef,0xf1,0x0f,0x5b,0xec,0x9a,0xa2,0xf7,0x50,0x03,0x48,0x7b, -0x47,0xec,0x3c,0xfe,0xa8,0x45,0x3f,0xcd,0x16,0xbb,0x81,0x37, -0x6c,0xd2,0x6b,0xc6,0xe4,0x7e,0x0a,0xe1,0x03,0x2a,0x58,0x79, -0x0a,0xf8,0x38,0xc5,0xcd,0x06,0xc5,0x2d,0x95,0xa0,0xd6,0x81, -0xce,0x93,0x28,0xa9,0xb6,0x29,0x99,0x4d,0x4a,0x53,0xa7,0x1c, -0x19,0x17,0xf6,0x1e,0x90,0x8e,0x85,0xd1,0x12,0x92,0x76,0x18, -0x85,0xa4,0x65,0x90,0x62,0xa0,0x6d,0xf1,0x8f,0xef,0x7b,0x89, -0x17,0xff,0xf5,0xfe,0x22,0x10,0x95,0xfa,0xbd,0x21,0x90,0x28, -0x73,0x0f,0x72,0x57,0x73,0x7c,0x33,0xfc,0xd0,0x78,0xb1,0xcf, -0x1e,0x9c,0x70,0x0d,0x8a,0x29,0xfe,0x40,0x93,0xfc,0x01,0xa1, -0xdc,0x55,0x74,0x44,0x7b,0xd3,0xd8,0xda,0x12,0x25,0x95,0xa3, -0x3b,0x71,0xdf,0x38,0xfb,0x7a,0x9a,0xc1,0x32,0x37,0x18,0x3d, -0xeb,0x4d,0x26,0x29,0x82,0x03,0x21,0x97,0x63,0x66,0x67,0x86, -0x8b,0x65,0xaf,0xef,0xe7,0xa6,0xe6,0xb6,0x84,0x57,0x44,0xb9, -0xa3,0x73,0xd2,0x6d,0xb1,0x41,0x6d,0x61,0x7f,0xcc,0x13,0x2b, -0x5c,0x63,0x40,0x3a,0x3d,0x1b,0x9f,0xa3,0xbe,0xc8,0x22,0x3f, -0xb6,0x52,0x0d,0xc6,0x46,0x1d,0x1e,0xe6,0xe3,0x6a,0xba,0xe8, -0xed,0xed,0xee,0x71,0x6a,0xe0,0x93,0x8b,0x8c,0x7f,0xa0,0x12, -0x2a,0x7f,0x7b,0x6b,0x4f,0x83,0xfc,0x41,0x56,0x38,0x8d,0x95, -0xb8,0x67,0xd8,0x4a,0x01,0x4c,0x4a,0x42,0x17,0x1b,0xb6,0xf5, -0xe8,0xb8,0xc8,0x52,0x87,0xa7,0x37,0x3b,0x57,0x90,0x52,0x96, -0x4d,0x26,0xc9,0x9a,0x3a,0x49,0xd1,0xb4,0x51,0x03,0x99,0xe8, -0x07,0xbd,0x20,0x38,0x60,0x94,0x9d,0x30,0x76,0xce,0xf2,0x35, -0x46,0x14,0x67,0xa3,0x58,0x46,0x69,0xac,0x93,0x68,0x0e,0x62, -0x4f,0x7e,0xb3,0xcb,0x71,0x36,0x4a,0x49,0x48,0x70,0x2a,0x88, -0x6b,0x4b,0x34,0x51,0x55,0x60,0x4a,0x4f,0x4f,0x03,0xa9,0x99, -0x1b,0xa9,0x4e,0x0d,0x88,0xe7,0xe0,0x8f,0x24,0x88,0x47,0xb7, -0xc4,0x20,0xb8,0xf1,0xd2,0x02,0x8e,0x56,0x13,0xa2,0x84,0x83, -0xf4,0xe0,0xc0,0xe7,0xb0,0x78,0xf7,0xb8,0x19,0x9c,0x9d,0xe3, -0xb0,0xc1,0x5e,0xc3,0x9b,0x0b,0x9d,0xe3,0xc1,0x80,0x70,0x5e, -0x2d,0xc8,0x6f,0x4a,0xed,0x81,0x19,0x57,0x1a,0x45,0x1c,0x5d, -0xa8,0x53,0xde,0xc3,0x27,0xfd,0x53,0x1a,0x38,0xfa,0x9e,0x03, -0x83,0x78,0xb5,0x11,0x70,0x79,0x89,0x34,0xcd,0x60,0xad,0x9f, -0xf5,0x7f,0x21,0xf5,0xf4,0x42,0xe4,0x73,0xd9,0x40,0x8a,0xf3, -0xe1,0x44,0xee,0x79,0x43,0xda,0x30,0xb3,0x38,0xd1,0xcc,0x50, -0x17,0xb5,0x03,0xe5,0x5c,0x8f,0x8f,0x1d,0xbc,0x11,0x46,0xa9, -0xf8,0xe6,0xa9,0xc5,0x9e,0xef,0xec,0x5a,0x27,0x07,0x18,0x86, -0xdc,0xa9,0xfc,0x9a,0x8d,0xa6,0x24,0x23,0xa1,0x26,0x57,0xe4, -0xcc,0x40,0x45,0x32,0x97,0xdc,0xde,0xe2,0x8e,0x3a,0x16,0xd6, -0x36,0x9d,0x89,0x02,0xfb,0x68,0x91,0xc5,0x2a,0xb7,0xf4,0xa5, -0x7e,0x4e,0xc7,0x50,0x55,0x6e,0xd9,0x2c,0xaa,0x76,0xc4,0x63, -0x8d,0x2b,0x75,0x6c,0x1a,0xfe,0x0a,0x15,0x20,0xed,0x6c,0xdc, -0x56,0x98,0xaf,0x60,0x16,0xd9,0x39,0xf2,0x1d,0x6b,0x5d,0xde, -0x0c,0x26,0x04,0x11,0xa3,0x5d,0x39,0x42,0xa9,0x1c,0xa6,0x89, -0x36,0xdc,0xae,0x34,0xfd,0x71,0x6f,0xbb,0x9c,0xe1,0x8e,0x05, -0x24,0x6d,0xb6,0x29,0xec,0x2c,0x91,0x32,0x4f,0x7b,0x6b,0xa2, -0x27,0xeb,0xc8,0xc8,0x83,0x59,0x80,0x97,0x1b,0x9d,0x53,0xaf, -0x2e,0x02,0xc0,0x2c,0xa3,0x3c,0x2e,0x01,0x21,0x98,0xfb,0x52, -0x7f,0x61,0xd6,0x95,0xd1,0x93,0x05,0xbf,0xc6,0xb6,0x15,0x58, -0xa0,0x9c,0x69,0xe7,0xed,0x2d,0x47,0xe0,0x83,0x1e,0xd3,0x1e, -0xdc,0x32,0x3a,0x6b,0xfc,0x70,0x38,0xc6,0x8f,0x1e,0xcf,0x32, -0x5f,0x18,0xf9,0x55,0x20,0x16,0x49,0x9e,0x3b,0x25,0xc3,0x1d, -0xb0,0x8c,0x14,0x63,0xa9,0xba,0x65,0xf7,0xe8,0x9e,0xb0,0x93, -0xde,0x80,0x34,0xd7,0xba,0x61,0xa9,0xa4,0x55,0xfa,0xea,0xf9, -0xeb,0xd2,0x56,0x65,0xbd,0xab,0x91,0x8d,0x43,0x81,0x06,0x22, -0x7f,0xc5,0x4d,0x83,0x9f,0x53,0xc9,0x90,0xc3,0x6d,0xb1,0x9c, -0x1c,0xd9,0x0c,0x5f,0x3c,0x7f,0xf1,0xfc,0xf5,0x73,0x28,0x58, -0xe2,0x67,0x04,0xa5,0x7c,0xd2,0x56,0x21,0x6b,0x9a,0x3d,0x1a, -0x1d,0x40,0xc9,0x68,0x35,0x91,0x12,0x96,0xf6,0x9b,0x68,0x0e, -0x2a,0x1c,0x9c,0x26,0x0e,0x03,0xca,0xb2,0x41,0x6a,0xe7,0x0f, -0x1d,0x25,0x6e,0xf0,0x6d,0x4f,0xc0,0x1c,0x02,0xf4,0x44,0x81, -0x49,0x48,0x2f,0xd7,0x8d,0x14,0x87,0xa9,0xb1,0x91,0xd3,0x49, -0x92,0xb4,0x8c,0x99,0xde,0xb8,0xeb,0x44,0xcb,0xb1,0x60,0x44, -0x8b,0x85,0xb0,0x0d,0x77,0xc2,0x0f,0x5a,0x55,0x1a,0xb7,0x5c, -0xbc,0xb5,0xc2,0x83,0x10,0xe7,0x31,0xcc,0x0a,0x5f,0x54,0xe9, -0xbc,0xa4,0x7f,0x76,0x66,0xf6,0x31,0xca,0x6e,0x4f,0x28,0xde, -0xdd,0xad,0x98,0xd5,0x13,0x22,0xc5,0x7e,0x77,0xfd,0xcd,0x1f, -0xeb,0xce,0xde,0x99,0x10,0x89,0x6f,0x0e,0x1c,0x8f,0x1c,0xa5, -0x9f,0xdf,0xa0,0x4b,0xf1,0x52,0xcc,0x82,0x29,0xe7,0x38,0x3c, -0xa4,0x5f,0xcf,0x75,0x47,0x29,0xfa,0xf9,0xf6,0x04,0x32,0xbf, -0x79,0xd3,0x8d,0x61,0x0b,0x46,0x19,0x2a,0xa5,0xe8,0xcd,0x9b, -0x3f,0xde,0x7e,0x14,0xeb,0xe2,0xb8,0xae,0xc1,0x48,0x33,0x58, -0xb1,0x18,0x85,0xcf,0x5b,0x70,0x9d,0xc5,0x86,0xad,0xe8,0x18, -0x46,0x76,0x79,0xb0,0xed,0xc7,0xed,0x20,0x6c,0x11,0xf5,0xf8, -0xb3,0x6b,0xa6,0xfb,0x3c,0xd7,0x69,0xa7,0x6d,0xcc,0xa9,0xa1, -0x05,0x4d,0xdd,0x52,0x34,0x7c,0x55,0x61,0xf2,0x74,0x38,0xfd, -0x44,0x47,0x74,0x5b,0xfc,0x98,0x48,0xc6,0x88,0x8d,0x77,0x96, -0x1a,0xad,0x3d,0x8e,0xdb,0x5d,0xd6,0x27,0x61,0xbf,0x47,0x9d, -0xc8,0x76,0x5c,0x5a,0xb8,0x8b,0xcf,0xb5,0xd5,0xa5,0x24,0x90, -0x4f,0xd7,0x66,0x29,0xa8,0x4a,0xe7,0xe8,0xc5,0xdc,0xc0,0xad, -0x94,0x3e,0x4a,0xd0,0x00,0xea,0x3e,0x40,0x51,0xbd,0x13,0xef, -0x01,0x66,0x48,0x45,0x17,0xe6,0x6a,0x02,0x4b,0xcb,0x78,0x7a, -0xfe,0x6a,0xd2,0xcb,0x2e,0x46,0x30,0x8b,0xf5,0xc6,0xe6,0xb4, -0xa1,0x54,0x87,0xad,0x1d,0x46,0x38,0x3b,0x3b,0x0c,0xb6,0xfc, -0x32,0x15,0x8c,0xdb,0x16,0x77,0x65,0xbd,0xf2,0x51,0x9d,0x1c, -0xb1,0x17,0x24,0xbd,0xa9,0x45,0x9d,0xf4,0xcd,0xbb,0x4a,0xf4, -0xd1,0xed,0x9b,0x4e,0x1c,0x43,0xbe,0x1a,0x64,0xe4,0xd1,0x51, -0x2b,0xca,0xff,0xe6,0x4d,0x0d,0x32,0xd5,0x31,0x97,0x66,0x83, -0xf9,0xee,0x9d,0xad,0xcf,0x21,0x24,0xe2,0xdd,0xde,0xca,0xe0, -0x13,0x29,0x2c,0xf5,0x42,0xc4,0x5f,0x5e,0x8c,0xb3,0xed,0xa5, -0xd3,0x40,0x15,0x4c,0xbb,0xdb,0xae,0x3d,0x7d,0xe3,0x9f,0x9a, -0x39,0xd9,0xdb,0x71,0x34,0x15,0x1c,0x48,0x99,0x8d,0xab,0x38, -0x1b,0x8c,0xe6,0x5a,0xa9,0x1d,0xe4,0x47,0x7b,0xd6,0xc2,0x33, -0x3b,0x3e,0x24,0x97,0x4f,0xe7,0x80,0x70,0xab,0x6e,0xcc,0x94, -0xd3,0xe2,0xe2,0xa8,0x37,0xc9,0x10,0xec,0xce,0xf8,0x23,0x9a, -0x9d,0x60,0x04,0x5c,0x8d,0x59,0xdb,0xde,0xa4,0xe0,0x4c,0x26, -0xbc,0x83,0x9f,0x78,0x75,0xa0,0x8f,0x21,0x60,0x4d,0x9f,0x66, -0x63,0x3a,0xf6,0xc4,0x58,0x7a,0xb3,0xfb,0xb5,0x8d,0xfc,0x86, -0x38,0x08,0x90,0x4c,0x6f,0x66,0x8b,0xf1,0xf9,0x98,0x96,0x98, -0xe5,0x45,0x8b,0xc0,0x2c,0x46,0xe7,0xa3,0xf5,0x9c,0xbe,0xb7, -0xac,0xdc,0x0e,0x19,0x6b,0x6f,0xc5,0x0e,0x99,0xb4,0x0c,0x53, -0x53,0xfc,0x63,0x3b,0x3a,0x3a,0x89,0xe2,0x5a,0x37,0xc6,0x29, -0xe0,0xcd,0x1b,0xe4,0x76,0x27,0xe9,0x4d,0x3d,0xee,0xb4,0xd0, -0x83,0x6f,0x1c,0x9d,0xbc,0xe9,0xdc,0xbe,0x29,0x77,0xe3,0x0e, -0x64,0x74,0x46,0x7f,0x86,0x3c,0x49,0x0a,0x10,0xfa,0x71,0x83, -0x55,0x8e,0xa0,0xf7,0x61,0x56,0x1d,0x4c,0x34,0x5c,0x59,0x5d, -0x04,0x35,0x67,0xd2,0x52,0xb9,0x38,0xb5,0x6d,0xed,0xf9,0xc9, -0x0d,0x14,0xbb,0xff,0xb6,0xd5,0xe0,0xc1,0xef,0x81,0xfe,0xc4, -0x17,0x1d,0x88,0x46,0x4a,0x7f,0x49,0xb9,0xbb,0x54,0xaa,0x44, -0x3a,0x19,0x95,0x7f,0x29,0x25,0x86,0xa1,0xd9,0x69,0xb9,0x29, -0x14,0xdd,0x82,0xf1,0x52,0x89,0x10,0x21,0xd8,0xb8,0xa0,0xed, -0xc9,0x4e,0x09,0x60,0x44,0x27,0x3f,0xd7,0xd1,0x41,0x31,0xa4, -0x95,0x62,0xa7,0x8c,0xb8,0xf9,0x70,0xe9,0xf7,0xa6,0xfe,0x11, -0x52,0xc6,0x52,0x10,0xbb,0xa8,0xc6,0xfd,0x91,0x7a,0xb3,0x3c, -0xce,0xd8,0xcb,0x0b,0x98,0x54,0xd0,0xd1,0x8a,0xe9,0xd6,0x4e, -0x69,0x5c,0x42,0x24,0xb0,0x98,0x3d,0x2c,0xa1,0xc5,0x84,0x0e, -0xc3,0xd0,0x64,0xaa,0x2b,0x6b,0x61,0x67,0x63,0x0e,0xda,0x4f, -0x70,0xde,0x13,0x8c,0xed,0x9a,0x13,0x94,0xc5,0x08,0xad,0x46, -0x7f,0x37,0x7d,0x35,0xea,0x2d,0x06,0x17,0x2c,0x0c,0x60,0x3e, -0x5a,0x59,0x71,0x5d,0xf1,0x39,0x90,0xa1,0xc5,0xb2,0xee,0xb2, -0xe4,0xc5,0xce,0x1b,0x90,0xbd,0x52,0x9a,0x97,0x30,0x8d,0x6a, -0xb1,0x06,0x6d,0x3a,0x14,0x94,0x93,0xa6,0x86,0xf2,0x12,0xe3, -0x16,0xb7,0xb4,0x5e,0x6a,0x0b,0x86,0x2e,0x48,0x17,0x53,0x8e, -0x7d,0x3d,0x13,0xce,0x75,0x70,0x73,0x4b,0x68,0x1c,0xb5,0x4c, -0x8f,0x84,0x91,0xb3,0xc2,0x19,0xc8,0xc9,0x8b,0x77,0x63,0xd7, -0x6d,0x95,0x88,0x9b,0x37,0x4e,0x66,0xa2,0x22,0x29,0x16,0x99, -0x6d,0xa0,0x03,0xe3,0x23,0x10,0x43,0xf0,0xce,0x6e,0x31,0x9b, -0xb1,0x8d,0x6e,0x3c,0x3f,0xc6,0x5b,0x7e,0xba,0xe1,0x56,0x98, -0x80,0xa3,0x99,0x37,0xbd,0x7c,0xb4,0x5c,0x12,0xe7,0xf8,0x33, -0xf2,0xf4,0xbb,0xd3,0x75,0x40,0xe0,0x14,0xc0,0x54,0xa0,0x0c, -0x78,0xe5,0x02,0x87,0xad,0x80,0x32,0x80,0x55,0xb1,0x0f,0x01, -0xf2,0x11,0x60,0xa6,0x13,0x3e,0x70,0xa2,0xd9,0x86,0xe6,0x07, -0x98,0x6c,0x00,0xa6,0x30,0x08,0xf6,0x25,0xcf,0x05,0x24,0x2b, -0xe1,0xa7,0x73,0x56,0x85,0x27,0x4d,0x9c,0xc0,0xcc,0x1a,0xeb, -0x9b,0x65,0x1c,0x83,0xa4,0xbc,0x97,0xba,0xc9,0xec,0x66,0x4c, -0xcc,0x9d,0x1e,0x5c,0x7a,0xb9,0xad,0x3d,0x13,0xd2,0x01,0x4a, -0x2f,0x1d,0xb5,0x9f,0xe3,0x71,0xbb,0x52,0x19,0x1b,0x64,0xe4, -0xe1,0x06,0x5a,0x44,0x42,0x89,0xa1,0x84,0x73,0xf3,0xf4,0x1c, -0x97,0x3e,0x98,0x51,0x67,0x67,0x0f,0x2e,0x4f,0xc6,0xdd,0xce, -0x70,0x94,0x3b,0xee,0xc6,0xf8,0xb8,0x85,0x7f,0xdb,0x74,0xa0, -0x70,0x85,0x17,0x5b,0x91,0x5d,0x39,0x6a,0x53,0xfd,0x6a,0xc4, -0x70,0x09,0x27,0xda,0x49,0x79,0x35,0x87,0x6d,0xe3,0x88,0x4f, -0x20,0xb5,0xb9,0xfb,0xf5,0x92,0xcd,0x20,0x4b,0x2c,0x29,0x5c, -0xa6,0xdc,0x25,0x35,0xb1,0xda,0x4e,0xa6,0xe7,0x58,0x43,0x96, -0x6d,0x8b,0xd5,0x3e,0xe2,0x0c,0x30,0x7b,0x61,0xa4,0x0e,0x1e, -0x1e,0xca,0x6b,0x35,0xca,0x83,0x9c,0x2c,0x9d,0x4a,0x99,0x6c, -0x18,0xc4,0x98,0x03,0xca,0xe2,0x8f,0x53,0x88,0x05,0x32,0x0e, -0x46,0x2f,0x29,0x96,0x95,0x90,0xf5,0xca,0x29,0x10,0x09,0x1a, -0xed,0x7f,0x9d,0x44,0x8f,0x81,0x62,0x65,0x79,0xac,0x86,0x5e, -0xb2,0x7b,0x43,0x58,0x45,0xd0,0xbe,0x3b,0x65,0xfa,0x81,0x28, -0x50,0x22,0x8c,0x70,0x2f,0x8d,0x70,0x6f,0x6f,0x29,0x04,0xc4, -0x74,0xaa,0x27,0xf7,0xa3,0xfb,0x20,0xb1,0xd1,0xf9,0x57,0xa8, -0x01,0x59,0x22,0xd3,0x56,0x0c,0x53,0xf9,0xa4,0x23,0xc4,0x15, -0xd3,0x4f,0x9a,0xac,0x07,0x3c,0xb9,0x07,0x7c,0x45,0x9d,0x1f, -0x05,0x49,0x71,0xc7,0x8c,0x0e,0x22,0x9b,0x7b,0x21,0x1a,0xe6, -0x55,0x0e,0x69,0xe2,0xb8,0x96,0x11,0x29,0x23,0x37,0xd2,0xcc, -0xd5,0x71,0xcb,0x82,0x05,0x49,0x25,0x84,0x94,0xeb,0xb6,0x00, -0x89,0xd8,0x89,0x90,0x6a,0xe2,0xd8,0x81,0x4e,0xfb,0x78,0x9e, -0x66,0xd0,0x9c,0x42,0x70,0x78,0x14,0x6b,0xeb,0x8f,0xbe,0xe5, -0x06,0xe5,0x9a,0x61,0x10,0x6d,0x3f,0xbb,0xb3,0x14,0x0c,0x69, -0x5f,0x6e,0xb6,0xb7,0x5a,0x18,0xe3,0xdc,0x85,0xc2,0x98,0xd5, -0x64,0xea,0xa6,0x86,0xc8,0x2c,0x58,0x75,0xcc,0xf4,0x42,0x2f, -0x31,0x39,0xb2,0x65,0x23,0xc7,0xd3,0xab,0xd9,0x5b,0xab,0xc7, -0xa4,0xac,0x00,0x6e,0xae,0x71,0x08,0x2b,0x1d,0x8a,0x3b,0xce, -0x3e,0xdf,0x44,0xa2,0x67,0xd8,0xf0,0xd6,0xc7,0xeb,0x14,0x7c, -0xd9,0x1b,0x02,0x46,0x2b,0x16,0x1e,0xec,0x1f,0x78,0x27,0x53, -0x50,0x81,0xa4,0xb8,0x25,0x9d,0x6f,0xbf,0x26,0x97,0xc4,0x69, -0xe0,0xc0,0xcb,0x75,0xce,0xe1,0xc2,0x2e,0x68,0xb6,0x54,0x49, -0xa0,0x27,0xe4,0xab,0xe1,0x75,0x71,0xed,0xa6,0xbe,0xd4,0xba, -0xec,0x70,0x93,0x77,0xb9,0xee,0xd8,0x75,0xd8,0x18,0x5e,0xc3, -0xe0,0xd1,0x17,0x88,0x9c,0x45,0x38,0xc6,0x46,0x8b,0xad,0xf6, -0x51,0x8e,0xfe,0xc4,0x9f,0xbd,0xc9,0x24,0xd2,0xcf,0xb3,0x72, -0x75,0x4a,0xc2,0x0d,0xcd,0x8a,0xc1,0x24,0x28,0x4d,0xb7,0x14, -0x20,0x56,0x15,0x1e,0xa4,0x29,0xc9,0x9d,0xa3,0xbc,0x29,0xe9, -0x8e,0x39,0x49,0x66,0x12,0x3e,0xca,0x71,0xe7,0x12,0xd7,0x0e, -0xbc,0x9c,0x29,0x15,0x63,0x76,0x0f,0xf0,0x74,0x56,0xe6,0x00, -0x97,0x23,0x23,0x34,0xa2,0xe4,0x48,0xe6,0x76,0x1d,0xb8,0x11, -0x37,0xe5,0xd8,0x18,0xd6,0x61,0x0f,0x46,0x1f,0x8b,0x35,0xce, -0xf9,0x1f,0xaf,0xb8,0x07,0xe2,0x6f,0x53,0x16,0xa6,0xb4,0x60, -0x79,0xce,0x4d,0x2b,0x22,0xdb,0x40,0x21,0x56,0x91,0xd7,0x9b, -0x13,0x96,0xd9,0xe4,0xf2,0xa7,0x65,0x67,0x85,0xfc,0x34,0x64, -0x64,0x1a,0x3b,0x7b,0xc9,0xf5,0x8c,0x36,0x88,0x6d,0x96,0x2c, -0xae,0x4c,0x47,0xdf,0xde,0x8a,0x78,0x86,0xab,0x38,0xf9,0x1d, -0x77,0xaa,0x96,0x58,0x83,0xc1,0xcd,0xd6,0x85,0x7f,0xb3,0x75, -0x1f,0x20,0xbb,0x33,0x34,0xaf,0xe8,0x1a,0xa7,0x1b,0xe7,0x28, -0xcf,0xee,0x44,0x34,0x19,0x25,0x33,0x89,0xd5,0xda,0xdf,0x59, -0xcb,0xdd,0x36,0x67,0xbc,0xa1,0x57,0x63,0x9a,0x40,0xc9,0x78, -0x65,0x2c,0x07,0x7c,0x62,0xd4,0xeb,0x9c,0x1f,0x45,0xd0,0x43, -0x47,0xd6,0x1b,0xa2,0x18,0x22,0x77,0x2a,0x01,0x6d,0x0b,0x9c, -0x37,0x33,0xb5,0x72,0x5c,0x27,0x35,0xe1,0xd4,0xcd,0x7b,0x92, -0x20,0x76,0x16,0xb2,0xb3,0x29,0xa5,0x93,0xe0,0xb0,0xca,0x6f, -0x8c,0x32,0x3f,0x0a,0xef,0x62,0x9a,0xc3,0x29,0xc4,0x9e,0x7a, -0xa9,0x18,0x7b,0x1a,0x2d,0xf1,0x6d,0x54,0x7e,0x95,0xc5,0x7e, -0xb9,0x61,0x92,0xb7,0x84,0xaf,0x58,0x34,0x70,0x3d,0xa6,0x78, -0xc5,0x1a,0xde,0x90,0x42,0xb3,0x6a,0x4f,0xb3,0xcd,0x74,0x10, -0x39,0x12,0x0e,0x8c,0x6f,0x4d,0x6c,0x37,0x2f,0xa9,0xe8,0x68, -0x06,0x0a,0x86,0x9e,0x5b,0x5a,0xf0,0xda,0x76,0x73,0x7b,0x5a, -0x6a,0xa1,0xb3,0xb3,0xb5,0x12,0xb4,0xa7,0xc9,0x47,0x3a,0x6a, -0x5b,0xe7,0xfa,0x7e,0x7a,0x8e,0x6a,0xb2,0xfa,0x08,0x95,0xc1, -0xef,0xf4,0x88,0x65,0xad,0xdf,0xa8,0x7d,0x56,0x04,0x43,0xc7, -0x58,0xbe,0xbb,0x85,0x0f,0xf7,0x92,0xb5,0xcb,0xef,0x42,0x60, -0x7f,0x8a,0x95,0xd4,0x4c,0xcb,0x1c,0xff,0x38,0xd8,0xc4,0xff, -0x76,0x77,0x54,0xa1,0xc4,0x2b,0xd3,0x99,0xcc,0xd9,0x05,0xb3, -0xa7,0x13,0x96,0x99,0xdd,0x2e,0x62,0x1c,0x06,0xf1,0x37,0x58, -0x57,0x70,0x17,0x60,0x96,0x96,0x3d,0xbe,0xa3,0xc4,0x88,0xeb, -0x7d,0xbc,0x46,0x31,0xf1,0x14,0x1b,0x4e,0xa5,0xb7,0x1a,0xce, -0xda,0xb5,0xd3,0xd2,0xab,0x71,0xf7,0xc4,0x56,0xee,0xa1,0xba, -0xd4,0xfa,0x7d,0x22,0x50,0x8e,0xbb,0x27,0x4d,0xaf,0x50,0xc4, -0x47,0x1d,0x6c,0xb7,0x33,0x34,0x35,0xb2,0xfb,0x3b,0xac,0xaa, -0xb9,0xe6,0xc8,0x34,0xb5,0x32,0x17,0x1a,0xc1,0x60,0xdf,0xf4, -0x26,0x6b,0xea,0xb0,0x7a,0x94,0x87,0x64,0x75,0xcd,0x73,0x49, -0x4f,0xb3,0x80,0xb3,0x4e,0x8a,0xf2,0xe0,0x45,0xab,0x41,0x4b, -0x48,0x2b,0x0e,0x52,0x5d,0x3f,0x9d,0xae,0x56,0x65,0xbe,0x80, -0x25,0xe0,0x07,0x94,0xdd,0x62,0x57,0x04,0xee,0xb5,0x0a,0xfd, -0x60,0x91,0x4b,0xf4,0xc0,0x09,0x56,0x90,0xf1,0xca,0xf3,0x85, -0xa5,0x02,0xf7,0x5a,0x5b,0x79,0x6a,0x9e,0x63,0xa5,0xed,0x5d, -0xbe,0xe6,0x76,0xfa,0xb8,0x6b,0xbb,0x06,0x80,0x8a,0xc4,0x14, -0x1e,0x59,0xb1,0xd2,0x23,0x0c,0x95,0x17,0xb9,0x67,0xc3,0x0b, -0x10,0xde,0x14,0xda,0x8b,0x12,0x0a,0xee,0xb8,0x3e,0xd9,0x75, -0xd9,0x62,0x44,0xbd,0x34,0x77,0x30,0x7e,0xc7,0x05,0x8b,0x16, -0xff,0x9d,0x34,0x8e,0x68,0x07,0x6a,0x8b,0xca,0xbb,0xf5,0xd1, -0x81,0xe5,0xec,0x7f,0x65,0xb3,0xa9,0xb8,0xd5,0x64,0x31,0x86, -0x54,0xf3,0xd2,0x02,0x4d,0x3d,0x4c,0xb5,0x6a,0x7a,0x73,0xbd, -0x42,0x08,0xfd,0x4a,0xca,0x3f,0x13,0x45,0x9f,0xa4,0x0e,0xc0, -0x5c,0x7e,0x7d,0xda,0x52,0xb4,0xea,0x90,0xe9,0x7a,0xa7,0xec, -0xd0,0x3a,0xc1,0xe5,0x55,0x06,0xdd,0xf6,0x38,0xab,0x4d,0xec, -0x07,0x6b,0xb2,0xdd,0xe1,0x23,0x9f,0xd5,0xf2,0x2e,0x27,0x8f, -0x03,0x87,0xc1,0xad,0x9d,0xdb,0xee,0x87,0x1e,0x46,0xef,0xd4, -0x93,0xcc,0xd3,0xf2,0xd9,0x6c,0xf6,0x76,0x8c,0xb6,0xf8,0xf3, -0x4a,0x8f,0x03,0x9b,0xe4,0x9f,0x57,0x81,0xe4,0xfb,0x2e,0xf3, -0x1d,0xe4,0x3a,0xc7,0x4d,0x92,0xea,0x1e,0x44,0xa3,0x2c,0x23, -0x33,0x37,0xda,0x7e,0xe1,0xfd,0xa3,0xe4,0x13,0x04,0x84,0xbd, -0x49,0x29,0x84,0x7c,0xb5,0x6a,0xf3,0xf9,0x92,0x1e,0xbb,0x8a, -0xe4,0x82,0x19,0xdf,0xd3,0x82,0x38,0xab,0xeb,0xac,0x49,0x02, -0x01,0x70,0x88,0xe0,0x01,0x95,0x3c,0xf2,0xf6,0xc8,0x03,0xa5, -0x42,0xd6,0xf5,0xdc,0xb8,0x1f,0xf0,0x21,0xcc,0x33,0xb7,0x04, -0x2e,0x9b,0x85,0x95,0x8b,0xc6,0x80,0xb4,0x0a,0xaf,0xc4,0x5a, -0xf9,0x9b,0xa6,0x1c,0xb8,0xb8,0xe3,0xe1,0x93,0xaf,0xaf,0x25, -0xd2,0x9f,0x97,0x0d,0xd7,0x14,0xa9,0x28,0xbe,0x31,0x35,0x26, -0x01,0x85,0xd3,0x1c,0xa2,0xc5,0x74,0xf1,0x40,0x43,0x9b,0x7d, -0x28,0xa6,0xd9,0x6e,0x7f,0x14,0x65,0x89,0x3b,0x45,0xa8,0xb6, -0xfc,0xf6,0x15,0x15,0x74,0x69,0xc6,0x33,0x1d,0x19,0x2c,0xf7, -0xb9,0x45,0xc0,0xa0,0xfa,0x93,0x43,0x23,0xd2,0x86,0x5a,0x4d, -0x9f,0xe3,0x71,0x1f,0xb4,0xbf,0xe8,0x8a,0xd8,0x20,0xed,0x4c, -0x53,0x26,0xce,0x88,0xb1,0x9a,0x52,0xbd,0xe1,0xf0,0x7b,0x18, -0x8e,0x5f,0x4e,0xc3,0x57,0x99,0x32,0xbb,0xef,0x21,0x6d,0x1e, -0xe9,0x83,0xd4,0x2f,0xa5,0xfd,0x66,0xf8,0xbd,0xe4,0xe7,0xe1, -0x0d,0x6f,0x10,0xe7,0x0e,0x8b,0xc2,0x0c,0xba,0x2b,0x35,0x31, -0xfc,0x8d,0x2b,0x3f,0x33,0x45,0x1d,0x97,0x28,0xb6,0xe4,0xf2, -0x05,0x7d,0x7e,0x1e,0x80,0x03,0x37,0xd6,0xd0,0xb6,0x05,0x93, -0xc3,0x2b,0x08,0xb1,0x52,0xb4,0x99,0x2c,0x5c,0x3b,0x9c,0x8c, -0x85,0x96,0xa6,0x51,0xdd,0xc0,0xa4,0xca,0xdb,0x3a,0xed,0x20, -0x4e,0x67,0xe6,0x27,0x88,0xae,0x62,0x4b,0xc7,0xac,0x50,0x8b, -0xd9,0x25,0x2e,0x11,0xfa,0x24,0x89,0xd5,0x4d,0xd5,0x7c,0xe5, -0x43,0xd5,0xee,0x9d,0x3c,0x88,0xa9,0xbf,0xca,0x18,0xeb,0x7b, -0xa2,0xb9,0xe0,0x63,0x25,0x7c,0xeb,0x01,0xc0,0x07,0x03,0xff, -0xb4,0x3b,0xc1,0xcb,0xde,0xdb,0x11,0x5a,0x33,0xca,0x8c,0xff, -0x7d,0x35,0xeb,0xff,0x82,0xcc,0xcb,0xb6,0xd2,0x32,0xf4,0x8b, -0xcb,0x3b,0x54,0x55,0x92,0x47,0xf8,0x69,0xa3,0x3d,0x7e,0xc2, -0x76,0xcd,0xf4,0xbb,0x33,0xb4,0x41,0xd1,0xff,0xe5,0x84,0x22, -0x4f,0xc6,0x5d,0xbc,0x77,0xd7,0x94,0x83,0x78,0x2b,0x9a,0xa3, -0x7c,0xfb,0x3d,0x3d,0xb3,0x8a,0xf0,0x53,0x81,0x78,0x31,0x9c, -0xf8,0x13,0x34,0xa6,0xd2,0x09,0x2b,0x3e,0x30,0x5f,0xf6,0xce, -0x15,0xfc,0x23,0x33,0x5d,0xb8,0x03,0x82,0x89,0xb4,0xb7,0xd8, -0xb0,0x5b,0x57,0x8e,0x36,0xe4,0xb4,0x3e,0x0f,0x25,0x45,0x7c, -0x2b,0x3c,0xd7,0x36,0xc1,0x25,0xba,0x4b,0x6f,0x60,0xdb,0xd9, -0xb2,0x37,0x78,0x5b,0x43,0xf6,0x45,0xbb,0x75,0xe3,0x29,0xc8, -0x6d,0x23,0x93,0xd5,0x4d,0xec,0xb6,0x63,0xc2,0xe9,0xf9,0x74, -0x88,0x18,0xa1,0xf5,0x18,0x27,0x35,0x6e,0xeb,0x4b,0x32,0x4e, -0xcf,0xd5,0x76,0x78,0xe8,0x66,0xc7,0xbb,0xee,0x73,0x7e,0x79, -0x19,0x00,0x35,0x48,0x53,0x0b,0xd3,0xab,0xd9,0x78,0x98,0x83, -0x75,0x7b,0x7b,0x70,0x40,0xc9,0x9c,0xe9,0xf6,0x96,0x41,0xd3, -0x9a,0xa6,0xcb,0x93,0xd4,0x42,0xd6,0x49,0x59,0xbb,0x3b,0x5b, -0x9a,0xc3,0xda,0xa7,0xaf,0x5f,0xbf,0x14,0x73,0x03,0xe1,0x0d, -0x3d,0xad,0x13,0xc3,0xd9,0xaa,0x3f,0x19,0xfd,0xfb,0x6a,0xb6, -0x24,0x3d,0x00,0xb4,0x82,0x44,0x6f,0xc2,0xdc,0x98,0xd5,0xf4, -0x57,0x1b,0x72,0x47,0x4d,0xae,0x30,0x60,0x17,0x96,0x46,0x85, -0xcb,0x5f,0xdd,0x20,0x88,0x9c,0x84,0xaa,0x4c,0xc1,0x7c,0x01, -0xf2,0x7c,0xba,0x1c,0x2f,0x71,0xe2,0x32,0xc7,0xb1,0xc2,0x25, -0xec,0x98,0xe2,0xf0,0xd0,0x0b,0xea,0x86,0x8b,0x3d,0x69,0xe6, -0x8f,0xe0,0x78,0x4b,0xa8,0xec,0x30,0x93,0xe6,0xf5,0xf9,0x2c, -0x43,0x6e,0xbe,0x0f,0x37,0x11,0xdb,0x60,0x7e,0xe9,0x4f,0xb9, -0xf4,0x6b,0x43,0xd4,0x71,0xda,0x90,0x6e,0x3e,0x81,0x10,0x2c, -0x54,0x52,0x06,0xd3,0xaa,0x55,0xba,0x3a,0xe2,0x5c,0xfc,0xd2, -0x75,0x1c,0x82,0x18,0x1f,0xa7,0x90,0xde,0x1e,0x43,0x5e,0xdd, -0x36,0x90,0x8b,0x6d,0x43,0xf1,0x08,0x8c,0xc1,0x93,0x5d,0x7b, -0xa7,0x30,0x96,0xe3,0x85,0x49,0x7b,0x37,0x81,0xf6,0xf2,0x29, -0x17,0xb3,0x29,0xbe,0x95,0xa7,0xdb,0x1d,0x1c,0x6e,0x34,0x7c, -0x2d,0x3b,0xe6,0x8f,0x2e,0x1e,0x70,0x2d,0x3e,0x7a,0xdd,0x6d, -0x9b,0x0a,0xd3,0xa8,0xa3,0x0a,0x70,0x86,0xf6,0xc7,0x0f,0x79, -0x3d,0x82,0x31,0x50,0x38,0x80,0x62,0x2c,0x4e,0x08,0x14,0xeb, -0x92,0xd4,0xca,0xf1,0x93,0x37,0x68,0x7f,0x18,0xf5,0x2f,0xb2, -0x72,0xa9,0xe2,0x96,0xae,0x94,0x4e,0x7e,0x3e,0xee,0x96,0x8f, -0x61,0xfa,0x19,0xbb,0x47,0x64,0xbd,0x89,0x58,0x70,0x36,0x17, -0x9e,0x78,0x4a,0xba,0xe4,0xbb,0x03,0xae,0xe4,0xd9,0x77,0xdf, -0x7c,0xf3,0xfc,0xdb,0xd7,0x9a,0xe5,0x4b,0xde,0x45,0xff,0xb3, -0x2f,0x9e,0xbe,0x7e,0xea,0x25,0x19,0x36,0xa3,0x36,0x5a,0xea, -0x53,0x30,0x0a,0x78,0x93,0xaa,0x46,0xd3,0x9e,0xac,0x62,0xbb, -0x7b,0x7a,0xd0,0x6f,0xc9,0xf1,0x10,0x8f,0x48,0x40,0x1d,0xf5, -0xdd,0x59,0x54,0x7a,0x72,0x50,0xad,0x96,0xe2,0x8e,0x18,0x8e, -0xf3,0xd3,0xaa,0xd5,0x63,0x40,0x88,0x82,0xc4,0x5c,0x91,0x41, -0x66,0x76,0xc9,0x47,0x24,0x41,0x04,0x4d,0xa7,0x72,0x2f,0x8d, -0xc2,0xe2,0x43,0xf1,0x10,0x86,0x3a,0xc6,0x9a,0xf8,0x36,0x95, -0x4d,0x11,0x1c,0xc5,0x4a,0xba,0x13,0x9f,0x4c,0x7c,0xfe,0xfc, -0xab,0xaf,0xbf,0xfd,0xea,0xf4,0xf9,0xb7,0x5f,0x9c,0xbe,0x7e, -0xfa,0xd5,0x73,0x21,0x0d,0x6b,0x57,0x61,0x71,0xc0,0x94,0x4f, -0x6a,0x09,0x16,0x1f,0x2d,0x4a,0x66,0x6d,0xc3,0x44,0xe9,0x73, -0xe0,0xa2,0x3a,0xd9,0x76,0x46,0xa3,0x6b,0x9e,0xe0,0x9a,0x08, -0xdd,0x25,0x3e,0x34,0x97,0xac,0x79,0x44,0x9d,0x8c,0x0e,0x8a, -0xe6,0x34,0xd9,0xc1,0xf1,0xd5,0xeb,0xa7,0x2f,0x5f,0xff,0x13, -0xb1,0x0c,0xe1,0x29,0x6f,0xc5,0x72,0x31,0xe5,0x4f,0x18,0x36, -0x05,0x1d,0xfc,0x04,0x16,0xd3,0xff,0xc1,0x77,0x62,0xc0,0xb6, -0x0d,0x36,0x11,0xd1,0xc1,0x3c,0xad,0x00,0xa3,0x86,0xf4,0x24, -0x0d,0x42,0x93,0xb3,0x54,0x0a,0xf3,0x89,0x47,0xb8,0x0f,0x67, -0x62,0x7c,0x45,0x41,0xb0,0x49,0xc4,0xd7,0x4a,0xd8,0x19,0xec, -0xf8,0x96,0xe3,0x6b,0x57,0x09,0x5b,0x3f,0x32,0x7b,0x7d,0x01, -0x82,0x84,0x24,0x2f,0x1e,0xbc,0xeb,0x65,0x0f,0x60,0xd6,0xed, -0x4f,0x48,0xd9,0x9a,0xf2,0x3c,0x58,0x42,0x8e,0xb3,0x19,0xfa, -0x4b,0x03,0xcc,0x1e,0xd0,0x32,0x8c,0x06,0x6a,0xa8,0x79,0xa8, -0x84,0x5d,0x22,0xae,0x64,0xc1,0x94,0x08,0xb3,0x75,0xc7,0x50, -0xec,0x3e,0xd9,0x29,0x58,0x11,0x8c,0xbb,0xcc,0xf1,0x70,0x38, -0x9a,0x7e,0xbf,0x18,0x01,0x59,0xa7,0xa3,0xc5,0xef,0x5f,0x7f, -0xf3,0x42,0xb4,0xd0,0x8d,0xa2,0xca,0x13,0x54,0x50,0x39,0x9c, -0x2c,0xdb,0x38,0xbc,0xfd,0xec,0xaf,0xe9,0x16,0xd8,0x46,0x22, -0x31,0xe4,0x40,0x06,0x17,0xa6,0xf0,0xc1,0xda,0x0e,0x1c,0x82, -0xfa,0x0e,0xa9,0xbe,0xde,0xe5,0xbc,0xed,0x4c,0x35,0xdf,0x7e, -0xf7,0xed,0xe9,0xd3,0x17,0xdf,0xff,0xfe,0xe9,0xb7,0x3f,0x7c, -0xf3,0xfc,0xe5,0xd7,0xcf,0x72,0x6b,0xb0,0x07,0xb2,0x74,0xf8, -0x6f,0xe2,0x63,0x84,0x3a,0xed,0x19,0xd4,0x4d,0x6a,0x71,0x95, -0x52,0xbb,0xe4,0xea,0xe0,0x38,0x4d,0xb3,0x91,0xc7,0x14,0x79, -0x8e,0x91,0xbe,0xd0,0xf5,0x4a,0xfa,0xeb,0x8f,0x8b,0x31,0x9e, -0x24,0xf6,0x57,0x67,0xb2,0x04,0x9e,0x4f,0x41,0x6c,0x26,0x3b, -0x08,0xab,0xa5,0x59,0xfd,0x50,0x51,0x18,0xf3,0x28,0xf8,0x47, -0xe2,0x85,0xf1,0xf7,0x4b,0x6b,0xae,0x15,0x52,0x71,0x49,0x75, -0x17,0xde,0x1b,0x88,0x28,0x5e,0x43,0xd1,0xcb,0x3c,0xd5,0x95, -0x5f,0x2c,0x20,0x95,0x76,0x71,0x8c,0x0a,0xe5,0xe5,0xef,0xdb, -0xdb,0xe4,0x00,0x7b,0xd4,0x17,0x85,0x50,0xc5,0x0e,0x90,0xa5, -0x31,0x84,0x58,0x33,0xf4,0xe0,0xf0,0x88,0xdd,0xa6,0x17,0xde, -0x13,0xd3,0x36,0x0e,0xef,0x41,0xf2,0x78,0x92,0xa5,0x12,0x5d, -0xe7,0x53,0x92,0x4c,0x26,0xac,0xd4,0xb7,0x5a,0x8c,0x9d,0x30, -0xcc,0xe5,0x07,0x30,0x2b,0x1f,0x70,0x37,0xf1,0x3c,0xf3,0xc3, -0xcb,0xaf,0xf5,0x0c,0xa3,0x11,0x7c,0x20,0x08,0x92,0x4d,0x0d, -0xfc,0xf8,0x38,0x2d,0x7d,0xcc,0x5f,0x85,0x4c,0x25,0x99,0x20, -0x0f,0x4a,0x3d,0xf8,0x4d,0x54,0xed,0x94,0xea,0xc7,0xa5,0x56, -0x09,0x56,0x04,0xd8,0x26,0x80,0x2c,0xe0,0x91,0x7f,0x2f,0xc9, -0xef,0x47,0xc6,0xba,0x4b,0x47,0x8a,0xc2,0xaa,0x50,0x54,0x4a, -0x53,0xdd,0x67,0x91,0x61,0x14,0xc0,0x81,0x26,0x13,0x47,0xd9, -0x58,0xa6,0x39,0xa9,0xac,0xa0,0x79,0x9c,0x23,0x96,0xd3,0xc2, -0x60,0x6e,0x29,0x7a,0x1d,0xab,0x73,0xa0,0xab,0x0a,0xfd,0x6d, -0xc5,0x15,0x9a,0x3c,0xa8,0x1f,0x81,0x3f,0x9d,0xfb,0xb5,0xdc, -0xee,0xa2,0x98,0xf1,0x63,0x62,0x6b,0x73,0x3d,0xa5,0xc2,0x19, -0x3d,0xad,0xff,0xfc,0x84,0x5d,0x24,0xbc,0x6b,0x55,0xbb,0x95, -0x38,0x62,0x3f,0x08,0x12,0xa4,0x40,0x39,0xc5,0x0c,0x9d,0x16, -0xea,0xd6,0x9d,0xfc,0x5c,0xea,0x96,0x4b,0xe4,0x72,0xe1,0xe3, -0x93,0x9f,0x3f,0xee,0x96,0x3f,0x8e,0x6f,0x41,0x66,0x79,0x93, -0x41,0xd1,0xb8,0x13,0x97,0xd9,0xdf,0x73,0x9d,0xfc,0x3e,0x1f, -0xd7,0x95,0xbf,0xca,0x49,0x5d,0x20,0xf9,0x38,0x15,0xb2,0xc4, -0x53,0x57,0x8e,0xc0,0x9e,0xd6,0x2d,0x3a,0x21,0x02,0x88,0x1f, -0x22,0x01,0x55,0x09,0x1a,0x1c,0xf3,0x31,0xc6,0x00,0x32,0x91, -0xc1,0x86,0x34,0x19,0xc3,0xf5,0x13,0x51,0xa8,0xab,0xe2,0xe5, -0xdf,0xa0,0x57,0x57,0xbe,0x34,0x95,0xd6,0x51,0x7c,0x21,0x6f, -0x10,0x20,0xab,0x00,0x54,0x57,0x9e,0xc2,0xc4,0x37,0x27,0x14, -0xf3,0xe6,0x84,0xf2,0x74,0xbb,0x98,0xc7,0x0e,0x11,0x7c,0xa2, -0x18,0x9d,0x2d,0xe7,0xb7,0xa8,0x2c,0x90,0x75,0xe2,0x16,0x10, -0xa0,0x7e,0x7b,0xd9,0x1b,0x4f,0x96,0xb3,0xd6,0xed,0x72,0x34, -0x69,0xdd,0xfe,0x5b,0x5c,0x1f,0xab,0x1d,0x53,0x27,0x52,0xe3, -0xe7,0x37,0xff,0x56,0xfd,0xff,0xdd,0x3e,0xb8,0x3d,0x20,0x2d, -0x44,0x77,0xbf,0x94,0xea,0x4d,0x6d,0xa9,0xb7,0x18,0xf5,0x54, -0x7f,0x81,0x06,0xa7,0xd4,0xc5,0x42,0x8d,0x2f,0xcf,0xd5,0xbb, -0xfe,0xa2,0x14,0x2b,0x7f,0xd3,0xf6,0xb9,0xbb,0x4f,0xb4,0xa5, -0xa1,0x14,0x59,0x6c,0x51,0xc3,0xa1,0x1a,0x2e,0xd5,0x04,0xb6, -0x0b,0x6a,0xd9,0x9f,0x0d,0x37,0x6a,0x39,0x54,0xcb,0xb3,0xd9, -0x6c,0xa9,0x96,0x17,0xf0,0xff,0xa8,0x07,0xc1,0x3c,0xd4,0xaf, -0xbd,0x3d,0xa5,0x05,0xbb,0x98,0xab,0xc5,0x32,0x97,0xdb,0xe4, -0xf3,0x0f,0xc7,0xf1,0xea,0x7b,0x1f,0xd8,0x7d,0x2d,0x09,0x36, -0xc0,0x77,0x42,0xf6,0x0a,0x2b,0x4b,0xc4,0xe1,0x10,0x8d,0xcd, -0x29,0x34,0xfb,0x36,0x98,0xc0,0x46,0x2b,0x43,0x27,0x73,0x04, -0x9a,0x36,0x73,0x6a,0xd0,0x63,0x23,0x30,0x03,0xba,0xa5,0xc2, -0x4b,0x61,0x35,0x1c,0xc3,0xef,0xf8,0x4a,0x0d,0x27,0x8a,0xde, -0x67,0x8d,0xf0,0x47,0xe7,0x43,0xca,0x41,0xbe,0x8b,0x44,0x5d, -0x34,0xd5,0xc5,0x91,0xba,0x78,0xa8,0x2e,0x1e,0xa9,0x8b,0xc7, -0x8a,0x9f,0x20,0xa9,0x0b,0xa6,0x3a,0x76,0xd8,0x14,0xf1,0x98, -0x2b,0xc0,0x68,0x05,0x7b,0xd4,0x2b,0x05,0xfd,0x38,0x07,0x60, -0xec,0xa2,0x46,0x65,0x6c,0xd2,0x08,0x66,0x2a,0x90,0x3e,0xd4, -0x6a,0x82,0xb3,0x96,0xbf,0x93,0xff,0x50,0x6a,0xda,0x46,0xab, -0x5e,0x1f,0x18,0xa7,0x37,0x58,0xcc,0xa6,0x9b,0x4b,0xd5,0x57, -0xfd,0xe1,0x18,0xfe,0xcd,0x54,0x7f,0x7c,0x4e,0x1c,0x35,0xc6, -0x86,0xa3,0x1b,0x68,0x6a,0xee,0xd9,0x54,0x8d,0x2e,0xa1,0x61, -0x78,0x67,0x4f,0x6c,0x86,0x98,0xbf,0xed,0x0f,0xc5,0xb2,0x37, -0xb5,0xa1,0xb7,0x78,0xab,0x7e,0x55,0x8b,0x55,0x7f,0xa3,0x88, -0x05,0x54,0xa6,0x32,0x90,0x14,0x54,0x76,0x89,0x7b,0x99,0x6c, -0xde,0x9b,0x2a,0x14,0xe5,0xde,0x8e,0xf0,0x67,0x36,0x3d,0x57, -0x20,0xdb,0xc1,0x3f,0xe0,0xb9,0x31,0x6c,0x6f,0x97,0x4b,0xb5, -0x52,0x30,0xe5,0x61,0x1b,0x83,0x05,0xd4,0xb2,0x96,0xa6,0x0b, -0xda,0x2e,0x60,0xc3,0x78,0xce,0xa8,0xc8,0x93,0xc2,0x1d,0x34, -0x3e,0xaf,0x04,0x64,0xdc,0xc1,0xab,0xa8,0x72,0xcf,0xcb,0xa2, -0x45,0xa1,0x0f,0x9b,0x1f,0xec,0x3f,0x7c,0x16,0x83,0x34,0xba, -0x58,0x8c,0xce,0xd4,0x04,0x9a,0x33,0x1c,0x65,0x03,0x95,0x2d, -0x06,0x6a,0x95,0x8d,0x80,0x1e,0x1a,0x3d,0x2e,0x9e,0xc7,0x4d, -0x43,0x76,0xba,0x84,0x3a,0x64,0x02,0xcb,0x0c,0xfc,0x5d,0xaa, -0xde,0x7a,0x0c,0x48,0x9f,0xc3,0x20,0x9d,0x2d,0x54,0x7f,0x46, -0x4e,0x82,0x07,0xa3,0xc9,0x64,0x8e,0x6f,0x91,0x80,0x7a,0xf8, -0x0d,0x44,0x1d,0xd0,0x37,0xda,0xcd,0x53,0x78,0x53,0x46,0x73, -0xc1,0x8c,0xfe,0x66,0xf4,0x07,0xc9,0x8e,0xb7,0x1e,0xbd,0xc1, -0x12,0x4f,0x17,0x17,0xc3,0x8c,0x98,0xf7,0x0c,0xc4,0x29,0xe1, -0xc7,0x0c,0x7e,0xc7,0xe7,0x17,0x4b,0x6a,0xca,0x04,0x30,0x55, -0x17,0x08,0x77,0xa4,0xc6,0x19,0xf6,0x2b,0xc5,0xe0,0x9f,0x55, -0xef,0x7c,0xa4,0xa6,0x33,0x6a,0xf0,0x74,0xf6,0x6e,0xd1,0x43, -0x3b,0x36,0x13,0xf8,0x77,0xa5,0xf0,0x58,0x95,0xfe,0x60,0x6d, -0x8b,0x15,0xba,0x28,0x14,0xa3,0xf5,0x74,0xf3,0x82,0x28,0x66, -0x17,0x3d,0x0c,0x33,0x17,0xf4,0x90,0x3b,0x56,0x97,0x97,0x78, -0xd4,0x03,0x81,0xf3,0x11,0xcc,0x34,0xe3,0x25,0xf0,0x38,0xaa, -0x8b,0x12,0xdd,0x80,0x08,0x2c,0xd2,0x5c,0x31,0x2e,0xef,0xc6, -0xc3,0xe5,0x05,0xf2,0x86,0x11,0x68,0xd3,0x5d,0x86,0xa8,0x60, -0x00,0x15,0x1a,0x9d,0xd0,0x2b,0x65,0x49,0x9d,0x74,0x63,0xb6, -0x4f,0xe2,0xae,0xc4,0x76,0x21,0xb6,0xc2,0x56,0x41,0x61,0xb4, -0xff,0x31,0xc1,0x95,0xb6,0x84,0x37,0xa1,0xbe,0x41,0x14,0x5c, -0xb1,0x5f,0x7c,0xfd,0xed,0xff,0xfe,0xe9,0xf4,0x87,0x97,0x2f, -0xec,0x54,0x9e,0x5f,0x08,0x22,0x59,0x09,0xe2,0xce,0xc9,0xd3, -0xea,0x9f,0x7a,0xd5,0xeb,0x46,0xf5,0xb3,0xda,0xe9,0xef,0x2a, -0xb0,0xfc,0xfd,0xcf,0xf8,0xcd,0xab,0x32,0x4c,0xfe,0xd9,0x9b, -0xda,0x9b,0xf6,0x1b,0xf5,0x26,0x7a,0x13,0xbf,0xb9,0x79,0xb3, -0x7d,0xf3,0xe4,0xcd,0x71,0xb7,0xae,0xbe,0x79,0xfa,0xf5,0x8b, -0xd7,0xdf,0xd9,0x45,0x46,0xe0,0xd4,0xdb,0xa1,0x6e,0x03,0x4a, -0xfb,0x42,0x5a,0x3a,0xd7,0x38,0xa0,0xbd,0x90,0x73,0x94,0xd0, -0x36,0x86,0x3c,0xf1,0x5d,0x09,0x8c,0xea,0x45,0xef,0x1d,0x1d, -0x2f,0xf0,0x86,0xfa,0xa4,0xab,0xde,0x91,0x44,0x91,0x16,0x08, -0x19,0x24,0x9f,0xe0,0xc3,0x58,0x7c,0x09,0xc4,0x77,0x01,0x74, -0xf4,0x92,0xbf,0x75,0x11,0x0c,0x50,0x75,0xca,0xe4,0xae,0x71, -0x64,0x2a,0x69,0x6d,0xde,0xd1,0x42,0xf5,0x22,0x68,0x86,0x14, -0x8c,0xdb,0x31,0xbe,0x64,0xd1,0x1b,0x55,0xa5,0xed,0x5f,0xa6, -0xc6,0x55,0x3e,0x3f,0x8c,0x4a,0x4b,0x42,0x0d,0x7a,0xb7,0x83, -0xf6,0xc5,0x58,0x49,0x8a,0x4f,0x8e,0xb8,0x35,0xb2,0x4f,0xc4, -0xca,0x8c,0x8e,0xfb,0x38,0x76,0xdb,0x52,0x43,0xfe,0xc6,0x67, -0x33,0xba,0x04,0x9f,0xc2,0x95,0x7a,0x25,0x27,0x53,0xec,0x83, -0xcb,0xed,0xa1,0xbd,0x6e,0xc2,0xb7,0x3b,0xa6,0x00,0x8e,0x7e, -0x80,0x15,0x13,0xb9,0x2d,0x1a,0xb4,0xc1,0xad,0x84,0x9b,0x73, -0xdd,0xab,0x21,0xee,0x7c,0xe8,0x61,0x25,0xbc,0xed,0xee,0xe3, -0xf2,0xe1,0x91,0x7b,0x0a,0x66,0xf7,0x9d,0x47,0xa7,0xd3,0xd5, -0x65,0x1f,0xfa,0x72,0x6d,0x6d,0x8f,0xd1,0x2b,0xac,0x35,0x88, -0xfd,0xe3,0xec,0xdb,0xde,0xb7,0x90,0xb2,0x75,0xf3,0x5f,0x8f, -0xc5,0xba,0x64,0x34,0x34,0x45,0x86,0x82,0xa9,0x97,0x71,0x81, -0x57,0xea,0xa7,0xa8,0x3b,0x76,0x3e,0x5a,0xbc,0x1a,0xf4,0x60, -0x00,0xad,0xad,0x09,0xbc,0xb7,0x69,0xd2,0x5e,0x97,0xdf,0xfe, -0x2e,0x69,0xc7,0x6f,0xcb,0x69,0x62,0x8e,0xd1,0xdf,0x7a,0x30, -0x68,0x42,0x8b,0x48,0x5f,0xde,0xa1,0xfa,0xcd,0x72,0xb1,0xb1, -0x80,0xd8,0xed,0xb7,0x93,0xfc,0x1d,0x3d,0x55,0xae,0xf1,0x05, -0x93,0x79,0xa7,0x46,0x0a,0xb0,0xe6,0x75,0x0f,0x3d,0x6b,0xe1, -0x07,0xc9,0x2d,0x5b,0x94,0x8d,0x06,0xc2,0x22,0x7c,0x39,0x5a, -0xe0,0x5a,0xdb,0x3a,0x48,0x80,0xa6,0x03,0x62,0x47,0xd4,0x66, -0xf1,0x40,0xa4,0xb6,0xdc,0xd6,0x43,0x1a,0x67,0xf1,0xf8,0xc6, -0x8b,0x7a,0x35,0x5a,0x86,0x51,0x8b,0x11,0xed,0x3e,0x65,0xda, -0x13,0x23,0x05,0x62,0xaf,0x22,0xa7,0xa2,0xe4,0x9e,0x40,0x73, -0x9e,0x1a,0x5f,0x29,0x49,0x39,0x63,0x1a,0x21,0xf6,0x2e,0x72, -0x60,0xff,0xc6,0x19,0x3a,0x5c,0x8d,0x5c,0xe0,0x78,0x78,0x5c, -0x01,0x2b,0xce,0x16,0xaf,0x36,0x97,0xfd,0xd9,0x24,0xe2,0x57, -0xde,0x62,0x6a,0x03,0x75,0x9a,0xe5,0xa2,0x92,0xe3,0xcd,0xab, -0x00,0x3c,0xf3,0xc5,0x3f,0x29,0xfe,0xb1,0x2f,0xda,0x6a,0xcb, -0xd9,0x0f,0x73,0x20,0xc9,0x33,0xdc,0x98,0xc5,0xf4,0x90,0xd8, -0xe1,0xeb,0x24,0x6e,0xbb,0x9e,0x51,0xa7,0xa9,0xa9,0x1c,0x66, -0xf1,0xb3,0xf1,0xda,0x9a,0x9d,0xf7,0x5e,0x15,0xcc,0x29,0x91, -0x0e,0xad,0x73,0x05,0x4e,0xc6,0x5d,0xaa,0x85,0x4e,0x9e,0x4d, -0xbe,0x3c,0xc2,0x36,0xcd,0x6f,0x3b,0x6a,0x43,0x84,0xfd,0x82, -0x16,0x92,0xa8,0xc3,0x77,0xc4,0x9f,0xa2,0x6b,0xeb,0x65,0xa4, -0x83,0xae,0x39,0x1a,0x4a,0xb0,0x2c,0x3e,0x51,0xd7,0xe9,0x64, -0x9c,0xc1,0x4a,0x8f,0x8b,0xeb,0x38,0xad,0x26,0xd0,0xe8,0x6b, -0xdd,0x48,0x68,0xe0,0x93,0x69,0x3b,0x8e,0x26,0xe9,0x35,0x9a, -0x2d,0x9c,0x4d,0x61,0x76,0x9c,0x48,0xaf,0xd2,0x2b,0xd5,0x7c, -0x9f,0xea,0x3a,0xf9,0x86,0x56,0x43,0xa6,0xa3,0x6f,0x09,0x7c, -0xce,0x16,0x85,0xf1,0xd8,0x99,0x19,0x51,0x17,0x25,0xdc,0x6a, -0xae,0xd7,0x2c,0xba,0x0a,0xd1,0xe5,0xf4,0x3d,0xc1,0x24,0xf5, -0x21,0x91,0x7e,0x33,0x71,0x83,0x31,0xe8,0x12,0x18,0xe1,0x78, -0xd2,0xec,0xa0,0x2d,0xf4,0x1a,0x1a,0xfc,0xc6,0x37,0x1b,0xf8, -0xc5,0x36,0xde,0x2d,0x82,0xe6,0xab,0x96,0x4d,0xc6,0x30,0x2d, -0xc2,0x5c,0xeb,0xc4,0xe9,0xb3,0xc2,0x49,0x1c,0xa3,0x2e,0x12, -0x8c,0xb6,0x28,0x2c,0x30,0xae,0xe0,0x31,0x63,0x80,0x9a,0xd8, -0x01,0x9e,0x8a,0x89,0x79,0x49,0x04,0x5c,0x4c,0x61,0xba,0x30, -0x0a,0x4a,0x65,0xd2,0x20,0x75,0x03,0x18,0xeb,0x34,0x7c,0xe4, -0x67,0x3a,0x74,0xab,0x88,0x58,0x5e,0xe7,0x53,0x0c,0x70,0x1d, -0xfe,0xc8,0xa3,0x6f,0xe8,0xe8,0xe1,0x51,0x8d,0xe9,0x3a,0x0f, -0x52,0xf2,0x65,0x5f,0xb1,0xa5,0x03,0xcb,0x1c,0x99,0x42,0x7e, -0xe6,0xf2,0xed,0x2c,0x05,0xc4,0x28,0xc7,0x73,0x0a,0xc7,0xa3, -0x34,0x33,0x3d,0x97,0x07,0xf6,0x85,0x66,0x51,0xbd,0xb0,0x6b, -0xa0,0xba,0x0d,0x9a,0x01,0x74,0x58,0xf1,0x80,0x0b,0xfb,0x4e, -0xf3,0xa0,0xce,0x76,0x62,0x32,0xd0,0xdd,0xe6,0x1e,0xa6,0x0f, -0x39,0xb2,0x36,0x3b,0xb3,0x9c,0x05,0xdc,0x7b,0x6d,0xb8,0xf7, -0x3a,0x3f,0x99,0x8d,0x12,0x9e,0xc1,0xc9,0x12,0x6a,0x23,0x1d, -0x25,0x6e,0xdb,0x2d,0x51,0x20,0xde,0x93,0x10,0x94,0x4e,0x81, -0x12,0xa6,0xb7,0x4e,0x30,0x17,0x4c,0xc6,0x5d,0x3b,0x72,0xdc, -0xba,0xb7,0x67,0xa8,0x3c,0x3a,0xd9,0xdc,0xd8,0xb2,0x8d,0xed, -0x76,0x6b,0x8a,0x7b,0xb4,0x65,0x3b,0x76,0x88,0x21,0x6d,0xcc, -0xec,0x1b,0x33,0x4c,0x5b,0xf5,0x79,0x2d,0xe2,0x24,0xe5,0x66, -0xff,0x5e,0xaf,0x08,0xe2,0x61,0x23,0x2b,0x86,0x2a,0x5f,0x78, -0x35,0x25,0x1f,0xe6,0x28,0x54,0xe7,0xb7,0x0f,0xa5,0x74,0x96, -0x8e,0xfe,0x68,0xe5,0x2f,0xb1,0x0c,0x70,0x03,0x50,0x8c,0x32, -0xdd,0x51,0xff,0xd3,0xc9,0xe4,0x9f,0x8e,0x82,0x0b,0x73,0x3f, -0x16,0x64,0xe6,0xdf,0xd1,0xd7,0xb1,0xf3,0x27,0xa6,0x7c,0xbb, -0x42,0x9d,0xbf,0x1b,0xf1,0x69,0x88,0x03,0x1c,0x77,0x47,0xe3, -0x3e,0xea,0xed,0xd3,0x48,0xdf,0xe6,0xb2,0x7f,0xfb,0x6a,0x47, -0x01,0x48,0xe0,0x25,0x88,0x76,0x0e,0xf4,0x49,0x0a,0x97,0x01, -0x8c,0x67,0x33,0xb2,0xa7,0xb1,0xd4,0x50,0x60,0x86,0xf0,0xeb, -0xd4,0x06,0x33,0x0a,0x4b,0xd9,0xda,0xdd,0x72,0xbb,0xaa,0x2e, -0x04,0xf5,0xa5,0xbf,0xca,0xaf,0xe5,0x70,0xbf,0x78,0x29,0x60, -0xe7,0x32,0xeb,0xce,0x6e,0x0a,0xb5,0x76,0xb4,0x62,0xbd,0xa3, -0x5a,0x6a,0xc1,0x3f,0x56,0xf1,0x4e,0x4a,0xe7,0x71,0xd9,0x49, -0x99,0xb5,0x79,0xd9,0x37,0xe5,0xb5,0xbe,0x36,0xcd,0x6a,0xa8, -0x4e,0x39,0x3e,0xdb,0x70,0xc3,0x14,0xa3,0xc0,0x2a,0x2a,0xb6, -0x64,0xc7,0x32,0x42,0x4b,0x7f,0xb6,0x0a,0xf8,0xb9,0xb0,0x9c, -0x25,0x42,0xcb,0x0d,0xb6,0x82,0x6c,0xb6,0xb7,0x5b,0x6e,0xd0, -0x97,0x55,0x61,0x87,0xdb,0x9b,0x83,0xe4,0x63,0x27,0x0e,0xd8, -0xe3,0x2c,0xc6,0x97,0x91,0x73,0x71,0xf1,0x26,0xab,0xe0,0xd5, -0xc5,0x83,0x52,0xbc,0x63,0x78,0xd0,0x2c,0x83,0x6f,0x97,0xb8, -0xcd,0x46,0x28,0x77,0x2f,0x92,0x3b,0xad,0x9f,0x6f,0xdf,0x00, -0xa4,0xb8,0x54,0x01,0x32,0xa1,0x5d,0xdd,0x99,0xee,0x7c,0x7a, -0x06,0x8d,0x69,0xc6,0x48,0xc1,0xfe,0x7a,0x8a,0x47,0xa2,0x24, -0x3a,0x23,0xc3,0x4a,0x6e,0xd5,0x44,0x2f,0x1b,0x58,0x14,0x16, -0x0a,0xe6,0x94,0x90,0xb1,0x05,0xc4,0x97,0xde,0x1b,0x07,0x03, -0x42,0xed,0xe7,0xb5,0xa2,0x0a,0x80,0x3f,0xac,0xc8,0x29,0x64, -0xd5,0x36,0x38,0x80,0xa8,0x31,0xec,0x1a,0xe7,0x51,0x51,0x13, -0xad,0xba,0xc8,0x94,0x0b,0xcb,0xe2,0xb7,0x73,0xe2,0x63,0x46, -0x09,0x1a,0xd0,0x0a,0x68,0xb2,0x9f,0xac,0x58,0x67,0x64,0x0d, -0xd6,0x2d,0x48,0x76,0xdd,0xd9,0xc9,0xb9,0x5d,0xfb,0x14,0x0f, -0xdc,0xa4,0x53,0x50,0x19,0x21,0xc5,0x88,0x1a,0x95,0x7a,0x01, -0x22,0x4b,0xec,0x29,0x6b,0x0d,0x50,0x65,0x4e,0x86,0xfe,0xc0, -0x93,0x8a,0x58,0x1b,0x9d,0x0b,0x9f,0xbb,0xf3,0x41,0x89,0x40, -0x95,0xc8,0x59,0x1a,0x1b,0x9f,0xef,0x44,0x8b,0x11,0xdd,0xe5, -0x6b,0x5f,0x03,0x74,0x53,0x98,0x2d,0xa3,0x41,0x8c,0x26,0x8d, -0x86,0x23,0x7f,0x42,0x11,0x00,0xca,0x65,0xfb,0x41,0x05,0xd8, -0xba,0xc2,0xe2,0x58,0xdc,0xba,0x67,0x19,0x33,0x32,0x16,0x23, -0x1a,0x15,0xf1,0xee,0xd5,0x0b,0x0f,0xfb,0x1c,0x76,0x35,0xcf, -0x41,0xdc,0xf7,0xd2,0x98,0xc7,0x5d,0x42,0x28,0x42,0x68,0x62, -0x76,0x82,0xc1,0x3a,0x42,0x79,0x72,0x8b,0x00,0x15,0x04,0xfc, -0xbd,0x52,0x61,0xcd,0x01,0x90,0xbf,0x7f,0x22,0xdf,0x8d,0x67, -0x6b,0x1f,0x3a,0x6b,0x07,0x15,0x67,0xf7,0xae,0xe7,0x48,0x43, -0x8e,0xdd,0xd3,0xa1,0x87,0x77,0xcb,0x23,0xc5,0x8e,0x7e,0x98, -0xe7,0x09,0xe2,0xaa,0xb5,0x49,0xaa,0x74,0x82,0xe8,0xf5,0x61, -0x1b,0x58,0xf7,0x69,0x9b,0xcb,0x19,0x50,0xfe,0xc4,0xbc,0xf4, -0x5e,0x8d,0xf2,0x99,0xff,0x2e,0x0a,0xe7,0x90,0x68,0x39,0x15, -0xad,0x8b,0x08,0xe7,0xb6,0x62,0x37,0xed,0x42,0xa4,0x5a,0x61, -0x93,0x76,0xcd,0x11,0x78,0x4e,0x29,0x5a,0xc0,0x7b,0xe4,0x2f, -0x4c,0xa6,0xe5,0xaa,0x15,0xed,0x5a,0x11,0x63,0x59,0xa2,0x0a, -0x05,0xb3,0xe2,0x73,0xd1,0xdd,0x42,0x51,0xb1,0x78,0xb7,0x13, -0x0a,0xf1,0x25,0x96,0x27,0x48,0x3f,0xbc,0xfc,0x9a,0x4f,0x0d, -0x76,0x0d,0x5f,0x7c,0x5a,0xf2,0x2d,0xcc,0x09,0x62,0x2d,0x4f, -0xae,0xd9,0x4f,0x29,0xfe,0xf4,0xb4,0x45,0xef,0x5e,0x77,0x14, -0x95,0x83,0xd6,0x50,0x64,0x7d,0xb0,0x47,0x1c,0xe5,0xe7,0x98, -0x19,0xb3,0x84,0x29,0xb8,0x73,0x6a,0x99,0x2d,0x96,0x68,0x8b, -0xa0,0xb7,0xa0,0x7e,0x19,0x98,0x4f,0x03,0x38,0xdc,0x39,0x91, -0xf5,0x1b,0x9d,0x0b,0x7b,0xa6,0x97,0x0d,0x46,0x53,0x3c,0x8f, -0x77,0x55,0xb7,0x54,0xdf,0x02,0x38,0x3c,0xec,0x77,0x6c,0x91, -0xa8,0x57,0xd3,0x4d,0x57,0x7d,0xf3,0x19,0xb7,0x0e,0x7a,0xd5, -0x83,0xfe,0x2e,0x34,0x47,0x78,0xdf,0x2f,0xfb,0x0f,0xe8,0xb1, -0x09,0x5e,0x41,0xd8,0x65,0xf5,0x17,0x98,0xb4,0x2f,0xd9,0x2f, -0x9e,0xd9,0xdf,0x5d,0x1e,0xff,0xd2,0xfe,0x45,0xfc,0xbd,0xd0, -0x02,0x88,0xeb,0x0a,0x65,0x91,0x8c,0x27,0xbf,0x74,0x65,0x6f, -0x78,0xee,0xfa,0x73,0x99,0x8a,0x7b,0x05,0x5a,0x88,0x38,0x09, -0x55,0xf2,0x0e,0x0f,0x75,0xb5,0xbc,0x42,0x8d,0xd5,0x2f,0x66, -0xed,0xda,0xbb,0xf3,0xe1,0x67,0x5d,0x26,0x5c,0xb8,0xb1,0x32, -0xa9,0x2a,0x5f,0xd4,0xd9,0x61,0x99,0x94,0x7d,0x55,0xa1,0x5d, -0x9d,0xc5,0xf2,0xf3,0x11,0xb4,0x7b,0x14,0x51,0x8c,0x1c,0x70, -0x34,0xd4,0x2f,0x8d,0xdc,0x6a,0x3b,0xa4,0x86,0xdc,0x04,0x04, -0xa2,0x62,0x40,0x9f,0x1a,0xeb,0xc6,0x87,0x24,0x42,0x92,0xfe, -0x72,0x90,0xfe,0x82,0x4a,0x6b,0xbf,0x34,0xd2,0x5f,0xd4,0xb8, -0x91,0x36,0x62,0x35,0x3e,0x4e,0xc7,0x18,0x05,0x21,0x3c,0xb4, -0x68,0x1f,0x78,0x24,0x6c,0x20,0x0d,0x41,0xb8,0x69,0xa0,0x74, -0xa3,0xf1,0xc0,0x0c,0xbb,0xba,0x9c,0x5e,0xde,0x2e,0x56,0xf3, -0xa5,0x7d,0xe5,0xf7,0x96,0x6c,0x98,0x01,0xcf,0xd0,0x0b,0x48, -0xb2,0x46,0x77,0x7a,0xda,0xc6,0x04,0x04,0x8d,0xbf,0x35,0xb4, -0x6d,0x76,0x35,0xda,0x01,0x12,0x37,0xe8,0x78,0x00,0xaa,0xcf, -0x3e,0xe8,0x5a,0x73,0xb5,0x70,0x27,0xf1,0xd9,0xf4,0xa5,0xf8, -0x58,0xe2,0x4a,0x53,0x3b,0x5f,0xb6,0xf1,0x8c,0xc7,0xd9,0x02, -0xd0,0x9e,0xfd,0x85,0x80,0x12,0xc0,0x6a,0x52,0xfb,0xc8,0xbc, -0x99,0xb5,0x45,0x9d,0xd5,0x72,0x36,0x7d,0x0a,0x22,0x8b,0x86, -0x8e,0x57,0x48,0xe6,0x98,0x06,0x7b,0x9f,0x6c,0x4e,0x5a,0x33, -0xbb,0xa8,0x7c,0x2c,0x18,0xd5,0x90,0x01,0xa9,0x7e,0x36,0xf1, -0x82,0xa2,0x4f,0x01,0x0a,0xce,0xfc,0x4e,0xd8,0xa4,0xba,0x8d, -0x10,0x38,0x35,0x47,0x4f,0x16,0x1f,0xd9,0xce,0xd0,0x92,0x65, -0xef,0x43,0x16,0x23,0xdf,0xc2,0xcf,0x29,0xd2,0xee,0xe4,0xe7, -0x5a,0x37,0x10,0xbc,0x89,0x31,0xd1,0xf0,0x8f,0x3d,0xe0,0xd4, -0xe7,0xa6,0x84,0x29,0x7a,0x77,0xe2,0x53,0x19,0x9c,0x6d,0x2f, -0xc5,0x3d,0x4c,0x5c,0x40,0xdc,0x9d,0x84,0xd5,0x2e,0xbe,0x76, -0x12,0x97,0x35,0x4e,0x68,0x9d,0x28,0x21,0x9e,0xa5,0x0a,0x11, -0x62,0x4c,0x8b,0x89,0x55,0xc7,0x43,0x7b,0x5f,0x48,0xee,0x34, -0x60,0x09,0x5d,0x51,0x7b,0x7c,0x8c,0xdc,0x4b,0xc7,0xdb,0x54, -0xd2,0xd3,0xcf,0x13,0xcf,0x67,0x3c,0x1f,0x87,0x20,0xbe,0xa4, -0x58,0x76,0x01,0x4e,0x14,0x31,0xe3,0x8c,0x12,0x34,0x54,0x0e, -0x15,0x22,0xc1,0x10,0x60,0x0c,0x75,0x74,0x0f,0x75,0x88,0x51, -0x5a,0xba,0xf7,0xcd,0x79,0x5d,0x47,0xce,0x6e,0x5b,0xa6,0xe3, -0x76,0x32,0xbb,0x21,0xa1,0x61,0xb0,0x7d,0x27,0x53,0x62,0xe2, -0xca,0x1e,0x42,0xd9,0x73,0x23,0x5b,0xee,0xa4,0xd1,0xd5,0x63, -0x90,0xe7,0xed,0x36,0x9e,0x66,0x69,0xf8,0x45,0xc2,0x48,0xd1, -0x41,0xd4,0x6c,0xbb,0x6b,0xd0,0x6b,0x4a,0x14,0xa2,0xcc,0x3c, -0xf3,0x01,0x8d,0xcc,0x4d,0x77,0xd2,0x48,0x7d,0xb6,0x86,0x78, -0x2e,0x46,0x13,0x7a,0xfe,0x83,0x22,0x28,0x7d,0xbd,0xa6,0xc4, -0xb6,0x84,0xa0,0xef,0x74,0x8e,0x54,0x0e,0xe3,0x6e,0x6f,0x3f, -0x3d,0x94,0xb8,0x1a,0x2f,0x6b,0xa3,0x2f,0x44,0x52,0xf8,0x7e, -0xc6,0xb3,0x92,0x3e,0x9d,0x84,0x0d,0xc4,0x44,0xc6,0x2d,0x9f, -0xe3,0x85,0x42,0x02,0xd1,0xe3,0x74,0xb8,0xe8,0x9d,0xbf,0x5a, -0xcd,0xc9,0xe1,0x9f,0xf3,0x72,0x2d,0x2d,0xd5,0x30,0x25,0x93, -0x94,0x6a,0xa9,0xf2,0xa0,0x52,0xf1,0x0a,0x7d,0x3d,0x54,0xcb, -0xd9,0x6a,0x70,0x81,0x9c,0x90,0x96,0xcc,0x27,0x6f,0x45,0x64, -0xbd,0xa0,0xcb,0xbb,0xb4,0xe4,0x04,0x24,0x99,0x60,0x73,0xa2, -0xf9,0x94,0xa4,0xc1,0x64,0x0c,0x93,0x6d,0x89,0x7e,0x24,0xea, -0x1d,0x32,0x86,0x9c,0xb9,0x01,0x0e,0x7c,0xc9,0x16,0xa3,0x67, -0x0c,0x53,0xad,0x2a,0x3e,0x2d,0xa6,0x4c,0x4e,0xf5,0xfb,0xb2, -0x19,0x44,0x76,0x65,0x62,0xc5,0x87,0xd4,0x91,0xce,0x44,0x2c, -0xe3,0x7d,0x82,0xb4,0x39,0xa5,0xc0,0x89,0x4f,0x60,0x4a,0x31, -0x8a,0x64,0xbb,0x73,0xa4,0x25,0x74,0x22,0xe1,0xdc,0x6a,0xeb, -0x0e,0x78,0x86,0xe4,0x70,0xd7,0x8a,0xb3,0x33,0xe8,0xad,0x77, -0x35,0x7a,0xf3,0x0d,0x49,0x8a,0x1f,0xa9,0x53,0x04,0xd1,0x8c, -0x5f,0xbe,0xef,0xa9,0x89,0xb1,0x55,0x5e,0x05,0xc0,0x71,0x0e, -0x48,0x47,0xd0,0xdb,0x75,0x16,0xaf,0x08,0x8f,0x2d,0x7a,0x70, -0x80,0xe6,0x2f,0x5f,0x8f,0x2f,0x47,0xa8,0x9f,0x07,0xb1,0xaa, -0x11,0xee,0x29,0x2f,0x67,0xab,0x6c,0xf4,0xfd,0x0c,0xd6,0xd6, -0x48,0x6c,0x26,0xc3,0x80,0x41,0x97,0x51,0x35,0xf6,0x13,0x38, -0x7c,0x8d,0x9d,0x49,0x8f,0xc5,0x46,0x69,0x18,0x09,0x83,0x9f, -0x67,0xc1,0xec,0xea,0x3c,0x35,0xc5,0x6b,0xb3,0x77,0xf0,0xf7, -0xd5,0x1f,0xbe,0x92,0x8e,0xb8,0xbd,0x35,0x49,0x78,0x37,0x05, -0x79,0x45,0x7e,0x86,0x2c,0x54,0xb3,0xdc,0x6d,0xe1,0x67,0x9a, -0x4f,0x8d,0xe8,0x2d,0x45,0xe3,0x58,0x23,0x7b,0xda,0x5f,0x9d, -0x3f,0x7c,0xd8,0xf8,0xf4,0x08,0x50,0x32,0x7c,0x57,0x63,0xa5, -0x8a,0x1f,0x6f,0x6f,0xc3,0xa8,0x9f,0xd0,0x3e,0x3c,0xa0,0x67, -0x99,0xb5,0x84,0xea,0x5c,0xa5,0x58,0x7b,0x20,0x2d,0x41,0x2a, -0x9a,0xe4,0xa0,0x2d,0xf5,0xcd,0x5c,0x06,0x6c,0xab,0xd4,0xeb, -0x67,0xb3,0x09,0x3d,0x99,0x5d,0xc2,0xe4,0x8a,0xee,0x79,0xcf, -0x96,0xf0,0x73,0x09,0xe3,0x76,0x3c,0x6d,0xa1,0x19,0x38,0xd2, -0x39,0x81,0x2f,0x56,0x44,0x69,0x31,0x97,0x6c,0x55,0xc9,0x38, -0x1b,0x29,0xc9,0xa9,0xc3,0xf2,0x12,0xdb,0x05,0xd4,0xc2,0x7b, -0x6c,0x18,0xf5,0xaf,0xa0,0x85,0xa3,0xe9,0xb3,0xd7,0xdf,0x40, -0xdb,0x72,0xcd,0x4a,0x0f,0x22,0x28,0x50,0x3b,0x03,0xb2,0xc1, -0x0f,0x4a,0x79,0x40,0x12,0xe3,0x6a,0xd4,0x0a,0x8b,0x7e,0xa9, -0x4e,0x44,0xf4,0xab,0xad,0xa1,0x93,0xe6,0xbd,0xf3,0xd1,0x8f, -0x8a,0xc3,0x1b,0x09,0xff,0x14,0xb7,0x9c,0x1c,0xc0,0x4a,0xd0, -0x31,0x6e,0x1e,0x8e,0xf9,0x29,0xe6,0xa8,0x94,0x13,0x60,0xa5, -0x5d,0x8c,0xd7,0xaf,0x51,0xb6,0x42,0x87,0x29,0x96,0x43,0x82, -0x46,0xa0,0x2d,0xa5,0x11,0x1a,0xaf,0x8d,0x63,0x75,0x22,0xb5, -0x68,0xd8,0x5d,0xb6,0xf4,0x87,0x63,0xd0,0x2b,0xfe,0x39,0xaa, -0x0b,0x01,0xfd,0x9e,0x51,0xc5,0x2f,0xb1,0x63,0xf4,0xf4,0x7c, -0x62,0x10,0xac,0x62,0xb9,0x1a,0x52,0xbe,0x6a,0x0b,0x73,0xda, -0x0b,0x88,0x54,0x06,0x6f,0xce,0x08,0x1d,0x95,0xcb,0xf7,0x7a, -0x36,0xef,0xfa,0x6b,0xcb,0xf9,0xd4,0xb9,0xe0,0x5f,0x1f,0x37, -0x3a,0x49,0xab,0x71,0xbc,0xee,0x54,0xe1,0xc7,0xcb,0xd8,0x1b, -0xcc,0x32,0x2f,0x67,0xd2,0x69,0xb4,0xaa,0x09,0x64,0xfd,0xed, -0xcf,0xad,0x6f,0xd0,0x08,0xa0,0xe4,0xf0,0x4b,0x65,0xe3,0x69, -0x50,0xea,0xa2,0x37,0x39,0x83,0x22,0x54,0xb4,0x2a,0x01,0x2e, -0xcf,0x79,0x7d,0xf4,0xc6,0xd3,0x0b,0x5b,0x3e,0x8a,0xd6,0x29, -0x65,0x1d,0x81,0xd4,0xb5,0x8e,0xe3,0x6a,0x52,0x5f,0xc7,0xf5, -0x66,0x70,0xd6,0x9a,0xed,0x29,0x51,0x29,0x28,0x01,0xbc,0xb9, -0xab,0x44,0xb3,0x4c,0xb5,0xc4,0xf5,0x68,0x0d,0x22,0xbb,0x57, -0xea,0xa2,0x87,0x9d,0xec,0xb6,0x4e,0x17,0xa4,0xc8,0x7a,0x33, -0x8e,0xcb,0x6b,0xaf,0xc4,0x33,0x54,0xbb,0x0a,0xef,0x8e,0x2f, -0xb2,0x49,0x74,0xa1,0x32,0x35,0xf1,0xce,0x73,0x21,0xe1,0xf7, -0x26,0xc1,0xcb,0x6f,0xa3,0xf9,0x34,0xea,0x22,0xbd,0x60,0x71, -0x37,0x4b,0xd9,0x38,0x76,0x6d,0x92,0x4e,0xc2,0x1a,0x4e,0x17, -0xe7,0x7d,0x5d,0xca,0x24,0x5d,0x45,0xd6,0xda,0xf2,0xc5,0xf1, -0xd1,0xe3,0x46,0xe7,0xa2,0x9a,0xc2,0x0f,0x74,0x3f,0x3d,0x0d, -0xa9,0x60,0x20,0x56,0x8f,0x21,0xd8,0xb9,0x4c,0x2a,0xd1,0x65, -0xb3,0x7a,0x99,0xc4,0xe5,0x8b,0x3a,0x64,0x49,0x3e,0xa5,0xd8, -0x66,0xab,0xf9,0x30,0x48,0x8e,0x20,0xa6,0x7a,0x11,0x63,0xa6, -0xcb,0xc4,0xe2,0x71,0xe5,0xd6,0x46,0x54,0x22,0x05,0xb9,0xa8, -0xf9,0xe8,0x51,0x19,0x93,0xc4,0x6b,0x70,0xa2,0x2e,0x9b,0x46, -0x95,0x39,0x65,0x25,0x93,0x8b,0x18,0x18,0x2d,0xba,0xf8,0x1d, -0xa1,0xf3,0x04,0xb0,0xac,0x20,0x92,0xd0,0x1a,0x49,0xcf,0x30, -0xbd,0x71,0x9c,0xc1,0xdf,0x0c,0xd8,0x2b,0x69,0x41,0x33,0xd3, -0xc6,0xf1,0x04,0xc2,0x13,0x0a,0x4f,0x00,0x68,0x5a,0x7b,0x74, -0x9c,0x4e,0x3a,0x93,0x72,0x94,0x54,0xb2,0xb8,0x35,0xa9,0x64, -0xd5,0x49,0x39,0x53,0x97,0x49,0xda,0x2c,0x4f,0xaa,0x97,0x4d, -0x5c,0x4f,0x91,0x48,0x88,0x67,0x25,0x69,0x42,0xbb,0x09,0x63, -0xfa,0x5b,0xc5,0x70,0xd0,0xf9,0x03,0xec,0x85,0x41,0x41,0xaf, -0x99,0x04,0xbf,0xd7,0x6c,0x7e,0xbf,0xd7,0x06,0xe9,0x60,0x47, -0xaf,0x0d,0x26,0xe8,0xbe,0x38,0xa8,0x45,0x53,0x04,0xfb,0x87, -0xf6,0x94,0x14,0x1e,0x60,0x78,0x80,0x61,0x28,0x88,0x85,0x26, -0x8a,0x48,0x8c,0xe3,0xf1,0xa2,0x9c,0x92,0xc2,0xcd,0x70,0x0c, -0xd3,0x56,0x5c,0x1e,0x28,0xc3,0xa2,0x17,0x10,0xf2,0xb1,0xe4, -0xa2,0xee,0xd9,0x84,0xb4,0xea,0x85,0x49,0xf0,0xf2,0xdb,0xe8, -0x1b,0xdd,0x06,0xd9,0x7a,0xa5,0x3d,0xfe,0xe8,0xa7,0xfd,0xb0, -0x06,0x22,0xb3,0x94,0xc2,0x3e,0xdf,0xa4,0xd1,0xa4,0x92,0x3c, -0x8e,0xeb,0x49,0xf2,0x58,0xad,0xd3,0x4d,0xa5,0x57,0x7f,0xd4, -0x68,0xa8,0xeb,0x74,0x53,0xed,0xd7,0x9b,0x0d,0xb3,0x27,0x5f, -0xa7,0x52,0x7e,0xbd,0xb9,0x86,0x31,0x57,0x96,0xd0,0x8f,0x6a, -0xe3,0x26,0x6c,0x4c,0xc2,0x4f,0x00,0xc2,0x49,0xb8,0x36,0x09, -0x7f,0xd2,0x7d,0x0d,0x3f,0x90,0x42,0x9f,0x47,0x35,0x60,0xdb, -0x87,0x8f,0x1e,0xc2,0x78,0xaf,0x26,0xb5,0x47,0x47,0x9f,0x24, -0x47,0x9f,0x3e,0x2a,0x6f,0xaa,0xb5,0x87,0x9f,0x7d,0xfa,0xe8, -0x28,0x79,0x58,0xbe,0x26,0xda,0xea,0xec,0xd5,0xda,0x67,0x8f, -0x3f,0x6b,0x3e,0x7e,0x5c,0x86,0x59,0xa1,0xf6,0xe9,0x27,0x8f, -0x1b,0x49,0xe3,0xd3,0xf2,0xa6,0x52,0x6b,0x3c,0x4c,0x1e,0x3d, -0x7a,0x1c,0x64,0xae,0x35,0x20,0xee,0xe1,0xd1,0x43,0x00,0x5d, -0x6b,0x36,0x1e,0x36,0x9a,0x8f,0x3e,0x83,0xbc,0x09,0x44,0x7f, -0xd2,0x6c,0x3e,0x6a,0x42,0xee,0x5c,0x2f,0x10,0x7f,0xf9,0x3d, -0x31,0x81,0x99,0x59,0x18,0x8f,0x27,0x4b,0x98,0xb5,0x9a,0x51, -0x5f,0xf5,0xa8,0x5d,0xc3,0xd1,0x39,0x2c,0x3f,0x99,0x74,0xee, -0xaf,0x8b,0x65,0xd4,0x2b,0xf7,0x2a,0xfd,0x72,0x1f,0xb6,0xb5, -0x71,0x4b,0xca,0x35,0xea,0x0d,0x85,0xff,0x26,0xf9,0xfa,0x98, -0xa6,0x76,0x96,0x06,0x44,0x1f,0x7f,0xfa,0xd9,0x51,0xe3,0xe8, -0x61,0x67,0x5d,0x86,0xff,0x5a,0xd1,0xba,0xfa,0xb0,0xde,0xfc, -0x2c,0xae,0x7f,0x52,0xfb,0xe4,0xd3,0x4f,0x1a,0x47,0x9f,0x78, -0x20,0xb0,0xa9,0xc8,0x3c,0x1e,0x88,0x46,0xe3,0xd3,0x4f,0x1f, -0x3d,0xee,0x10,0x4a,0xf3,0xd9,0xbb,0x68,0xad,0x92,0xfa,0x51, -0xdc,0xd2,0x00,0x80,0x76,0x08,0x31,0x07,0x07,0x49,0xb6,0xd8, -0x35,0x55,0x00,0x31,0x1b,0x47,0x8d,0x87,0xc7,0xe9,0xa2,0x93, -0x34,0x6b,0x9f,0x35,0xcb,0x8b,0x16,0x12,0xf2,0x51,0xd9,0xd4, -0xb2,0x80,0x5a,0x9a,0xb5,0x87,0x71,0x15,0xa3,0x03,0xca,0x02, -0xe8,0x6f,0x59,0xa5,0xcd,0x7f,0xc1,0xa4,0xc7,0x3e,0x46,0x1e, -0x1f,0x03,0x1b,0x42,0x55,0x87,0x12,0xfa,0xd4,0x06,0x72,0xc0, -0x3c,0xab,0x14,0x3e,0x30,0xaf,0x9e,0x8a,0xfb,0x96,0x4a,0x2a, -0x5b,0xa8,0xf3,0xfc,0x38,0x7b,0x69,0x12,0xbc,0xfc,0x36,0x5a, -0x2e,0xb3,0x53,0xbe,0x48,0xaf,0x9d,0xa7,0xe7,0xc5,0xe3,0x0c, -0x6a,0x38,0xbd,0x18,0xad,0xa3,0x2b,0x53,0x43,0xf2,0xf8,0xf8, -0xaa,0x53,0x6a,0x94,0x2a,0x44,0xa9,0xcb,0xde,0x1a,0x76,0xed, -0x57,0xa8,0x96,0x25,0x8d,0x80,0x11,0xc8,0xab,0xf0,0x25,0x4c, -0x0d,0xd0,0xe4,0x30,0x31,0x07,0x9e,0xed,0xf3,0xb3,0x0f,0x50, -0x05,0x11,0x0a,0xd6,0x19,0x39,0x22,0xc1,0x59,0x9c,0xdf,0x53, -0x2f,0xd0,0x35,0x35,0xfc,0xeb,0xf3,0x13,0x63,0x98,0x6e,0xeb, -0xd1,0x49,0xaf,0x7a,0xdd,0xad,0xc4,0x6f,0x22,0x7c,0xf5,0xfa, -0x26,0xae,0x8f,0xd9,0xc0,0x2c,0x43,0x8a,0x63,0xf1,0xa4,0x04, -0x53,0xf6,0x65,0x72,0xd2,0xec,0x3a,0x4f,0xef,0x61,0xb6,0x26, -0x57,0xb9,0xf8,0xb8,0xa7,0x04,0xb5,0x95,0x5a,0x7a,0xa5,0x80, -0x35,0x91,0xd0,0xf9,0x72,0x32,0xeb,0x2d,0xa1,0x28,0x4a,0xe2, -0xca,0x8f,0x81,0x82,0x75,0xf4,0x91,0xe9,0xc7,0x36,0x39,0x36, -0x26,0x27,0x46,0x25,0x68,0x85,0x81,0x29,0x13,0x84,0x69,0xa9, -0x74,0xa8,0xc0,0x2e,0x4e,0x49,0x76,0xa6,0x40,0x3d,0x5a,0x58, -0xd5,0x27,0xf6,0x94,0x8d,0x0e,0xc9,0xe9,0x5c,0x44,0x37,0xbf, -0x83,0x15,0xd3,0x69,0xd0,0x82,0x8f,0xe0,0xcf,0xf9,0xa7,0x8f, -0x56,0x5b,0x49,0xfd,0x92,0x73,0x1e,0x1e,0x96,0xfe,0x0d,0xcd, -0x68,0x73,0xc8,0x2a,0xda,0xc1,0x4a,0xf0,0xd0,0x46,0xf3,0x89, -0x64,0x27,0x5a,0x04,0xf9,0x92,0x58,0x2d,0x2a,0xc0,0x45,0xe7, -0x41,0x7c,0x33,0x56,0xe7,0x15,0x60,0xaa,0x7e,0x10,0x7f,0x14, -0xab,0x7e,0x25,0x05,0x1c,0x3e,0x09,0x61,0xe3,0x01,0x80,0x8e, -0x71,0x14,0xfa,0x14,0x94,0x38,0xcf,0xc7,0x1f,0xa9,0x47,0xb1, -0x05,0x6e,0xe3,0x1f,0xa9,0x4f,0x40,0x58,0x5e,0xb0,0x49,0xde, -0xaf,0x61,0x97,0x03,0xc3,0xf8,0x31,0x42,0x30,0x11,0xe7,0x14, -0xd1,0xb7,0x11,0x7d,0x8c,0x88,0x95,0x1d,0x4b,0x79,0x16,0x45, -0xbe,0x90,0x71,0x43,0xbe,0xc1,0x40,0x00,0x02,0xee,0x4e,0x0d, -0x9b,0x2f,0xea,0x29,0x72,0xfa,0x39,0xff,0xf4,0xe9,0x07,0x9f, -0xbb,0x8a,0x1c,0x87,0x83,0x84,0x8b,0xab,0x61,0x0a,0x81,0x2a, -0x94,0x01,0xd1,0x22,0x82,0xcf,0x0a,0x7c,0x82,0x18,0x69,0x74, -0x9d,0x3a,0x51,0x86,0x12,0xc6,0xa4,0x33,0xac,0x9b,0xe4,0x16, -0x7c,0x83,0x4c,0xc4,0xe5,0x62,0x75,0x91,0x2e,0x50,0x1d,0x79, -0xdd,0x89,0xce,0xab,0xfd,0xb8,0x3e,0xac,0x44,0xfd,0xe3,0xf3, -0xce,0xe3,0x56,0x23,0x6e,0x9d,0x4b,0x42,0xbf,0xba,0xc0,0x84, -0x66,0x2b,0x5a,0x54,0xcf,0xf1,0xeb,0xa1,0x82,0xe5,0x1b,0xa4, -0x9e,0x16,0xae,0xf6,0x30,0x73,0x67,0xe9,0x04,0x0f,0xe1,0x12, -0x12,0x6e,0x2e,0x88,0xd9,0x2e,0x8a,0xc5,0x44,0x6c,0x3e,0xce, -0xc8,0x7a,0x9a,0xd1,0x0c,0x87,0x13,0xfd,0x02,0x09,0xeb,0x84, -0xcf,0x91,0xae,0x4e,0xb8,0xcf,0xfb,0x34,0x5a,0x70,0xf5,0xcc, -0x1e,0xd5,0x1e,0x26,0xcd,0x87,0xb0,0x8e,0x95,0x17,0x95,0xda, -0xd1,0xa3,0x4f,0x1e,0x7d,0xf2,0x38,0x29,0x9f,0x57,0x6a,0x20, -0x08,0x3e,0x3c,0xfa,0xe4,0x51,0x19,0x5b,0x24,0x0b,0x72,0xcc, -0x2b,0xb2,0x2d,0xd9,0x4c,0x9a,0x8f,0x3f,0x69,0x7e,0x86,0x25, -0x3f,0x49,0x1e,0x25,0x8f,0x9a,0x4d,0x2c,0xd9,0xf8,0xa4,0x99, -0x78,0x05,0x61,0xb7,0x75,0xed,0x17,0x6c,0x24,0x9f,0x1d,0x1d, -0x1d,0x51,0xc1,0x24,0xf9,0x2c,0xf9,0x8c,0xca,0x7d,0xf6,0x08, -0x57,0x80,0xc4,0x29,0xf8,0x27,0xab,0x73,0xc6,0x32,0x0c,0x48, -0x12,0xb0,0x76,0xc3,0x3c,0x0e,0x82,0x44,0x19,0x56,0xae,0x4d, -0xac,0x9a,0xf8,0xb5,0xa9,0x86,0x6b,0xad,0x25,0x89,0x96,0xe1, -0x99,0x27,0xe2,0x27,0x29,0x2c,0xe6,0x20,0x11,0x74,0x16,0x75, -0x5a,0x68,0x5a,0x66,0x81,0x89,0x00,0x1b,0x5c,0x5a,0xea,0xb4, -0xf0,0x28,0x5c,0x6b,0x8a,0xe7,0x46,0x19,0xfd,0x03,0x66,0xbe, -0xb3,0xd4,0x99,0x79,0x06,0x1a,0xe3,0xd2,0xef,0x70,0x0c,0x0f, -0xf4,0x30,0x1b,0x58,0x43,0xea,0x1d,0x77,0xe5,0xab,0xc1,0x0a, -0x77,0x16,0xb7,0xce,0xbc,0x8a,0xe8,0x1b,0x0f,0x95,0xf7,0x5c, -0x14,0x5e,0x75,0xae,0x0a,0xee,0xee,0xae,0xfc,0x43,0x8f,0xf1, -0x70,0x84,0xaf,0x14,0x37,0xae,0xd6,0xb6,0xbf,0x2c,0x5f,0x2c, -0x5e,0x03,0xbc,0xbc,0x6d,0x5a,0x03,0x1a,0x95,0xe1,0x2f,0xc7, -0x97,0x23,0xcc,0xe6,0x5c,0x40,0x49,0xb6,0x26,0xb4,0x31,0xbc, -0x26,0x83,0xf9,0x2b,0x8f,0xae,0x06,0x81,0xe2,0xac,0x00,0x49, -0x0d,0x58,0xfd,0x21,0x06,0x13,0x19,0x2f,0xbf,0x66,0x8d,0xa0, -0x45,0x61,0x1b,0x36,0xe4,0xae,0x02,0x37,0xce,0xc9,0x3f,0xa6, -0xe9,0xbb,0x08,0xb6,0xe8,0x89,0x0f,0x53,0x96,0x2b,0xf4,0xb1, -0xf0,0xeb,0x0a,0xed,0x8a,0x70,0x90,0x0c,0xa3,0xf3,0xe7,0xe1, -0xa1,0x4e,0xd2,0xa0,0xf9,0x91,0x35,0xa7,0x1e,0xa7,0xc0,0x87, -0x87,0x87,0x47,0x0d,0xd8,0xae,0x50,0xc4,0xed,0x2d,0xf0,0x32, -0xaa,0x62,0x53,0x88,0x75,0x1f,0xc5,0x66,0xaa,0xb1,0xfb,0x4b, -0xe7,0xa3,0x80,0xb9,0x12,0xc8,0x8e,0xd6,0x79,0xa8,0x6a,0x49, -0xc6,0x6d,0x6d,0x01,0xf4,0x91,0x35,0x1b,0x0f,0x1f,0x34,0xb6, -0x26,0x07,0xda,0xb9,0x73,0x21,0x62,0x55,0x62,0x45,0x6f,0x17, -0x14,0x53,0x2d,0x4d,0x0a,0x17,0x0b,0xb4,0x8c,0x63,0xb4,0x49, -0x87,0x47,0x35,0xa3,0x15,0x5d,0xea,0xd3,0xad,0x5a,0x36,0x9a, -0x0e,0x61,0xb9,0x9e,0x2f,0x66,0xe7,0x0b,0x32,0x99,0x57,0xc2, -0x4a,0xe1,0x87,0xe0,0xe2,0x2b,0x74,0x7e,0x53,0xc4,0xce,0xf2, -0x08,0x36,0x5d,0xa6,0xfc,0xf8,0xcd,0x8b,0xdf,0x2f,0x97,0xf3, -0x97,0x1c,0x65,0xba,0xc6,0x74,0xb9,0x0c,0x99,0x03,0x7b,0xa4, -0xf5,0xe3,0x17,0xb3,0xcb,0xde,0x78,0x2a,0x25,0x6e,0x6f,0x4b, -0x20,0x3a,0x5c,0x3c,0x5b,0x8c,0x88,0x9f,0x7b,0x93,0xac,0x44, -0x7e,0xd2,0x24,0xf1,0xa0,0xfe,0x33,0xf9,0x78,0xc3,0xfd,0x62, -0x2b,0xee,0xe0,0x23,0x9b,0xba,0x75,0x9c,0x73,0x7b,0x1b,0x79, -0xb8,0x78,0x90,0x63,0x55,0x62,0x0b,0x81,0x0e,0xc0,0x8e,0xee, -0x67,0x4e,0x49,0x6d,0x90,0x5d,0x5d,0x09,0xf7,0xb4,0x6c,0x3c, -0xf9,0x1c,0xc6,0x8e,0x1e,0xf1,0xb1,0xa1,0x6f,0x7b,0x44,0x33, -0x0d,0xe4,0x79,0x85,0x79,0x8e,0x8f,0x90,0x95,0x84,0x03,0x0d, -0xa1,0x00,0x8c,0xa6,0xab,0x2d,0x4e,0xc7,0x9d,0x7b,0xae,0x27, -0x28,0x88,0x9c,0x65,0x7a,0x58,0xc3,0x28,0xe8,0xe4,0xa2,0xcb, -0x08,0x05,0x39,0xc4,0x39,0x5a,0xa0,0x24,0xee,0x8b,0xbe,0x24, -0xd9,0x50,0x42,0x05,0xcd,0xfa,0x2e,0x67,0x2f,0xf0,0x2d,0x36, -0xab,0xfc,0xe7,0x5c,0x34,0x3e,0x69,0x76,0x84,0x11,0x44,0xf3, -0x22,0x72,0xd5,0x2c,0xe4,0x16,0xcb,0xcf,0xe1,0x85,0x38,0x23, -0x54,0x84,0xd8,0xc5,0x8c,0xa3,0x99,0x22,0x0c,0x96,0x3e,0x82, -0x21,0x0e,0x9d,0xc8,0x9b,0x54,0x8c,0x02,0x21,0xea,0x76,0x78, -0xe0,0x5b,0x3a,0x1f,0xd7,0xe3,0xf1,0xe6,0xbd,0xeb,0xf2,0x4a, -0xb1,0xba,0x10,0xc1,0x76,0xe3,0x7d,0xf8,0xbb,0x60,0x9b,0x74, -0x03,0x65,0xab,0x4e,0x4a,0x20,0x46,0xe2,0xb8,0x9b,0x65,0xcb, -0x52,0xd7,0x18,0x01,0x30,0x10,0xf4,0xfb,0x11,0xc8,0x7c,0xc2, -0xdf,0xdd,0x02,0xeb,0x37,0x58,0x3b,0x0e,0x60,0xb9,0xaa,0x42, -0xd6,0xd0,0xb9,0xb5,0x06,0x7e,0xd1,0x2d,0x2c,0x3e,0x30,0x52, -0xba,0x6c,0x1a,0xd4,0x49,0xce,0x4b,0x9d,0x19,0x16,0x66,0xcb, -0x7b,0xaf,0x08,0x58,0xd4,0x5d,0x0d,0x08,0x14,0xb9,0x60,0xe4, -0x55,0xc0,0x8c,0x8b,0xf9,0xc8,0x54,0x87,0x33,0x3c,0x1e,0xf6, -0x73,0x97,0xea,0x9e,0x83,0xe9,0xa1,0x37,0x40,0x87,0x79,0x38, -0x88,0x85,0x93,0x60,0xd4,0xcb,0x57,0x8d,0xd3,0x4c,0xee,0x4a, -0x49,0x95,0xeb,0xe5,0x92,0xad,0x20,0x1b,0x2d,0x65,0x36,0xf8, -0x3d,0x95,0x88,0xc3,0xfb,0x5c,0xed,0x3c,0x70,0x57,0x01,0x1e, -0x30,0x1e,0x0b,0xdb,0x7b,0x7e,0x12,0xec,0xed,0xea,0x67,0x5a, -0x75,0x05,0x93,0x09,0x2c,0xcf,0xdf,0xdc,0x99,0x62,0x18,0x99, -0x9b,0x7d,0x90,0xba,0x4c,0x45,0x97,0x74,0xc1,0xaa,0x84,0x3c, -0xe8,0x06,0x74,0x39,0x4d,0xe9,0xc3,0x43,0xec,0x4e,0x34,0x83, -0xca,0xd3,0xb6,0xed,0x3f,0x8a,0xe4,0x29,0xdd,0x74,0xb4,0x9e, -0x3e,0x6e,0xac,0x16,0x08,0xbe,0xf3,0x30,0xb3,0x8a,0x7d,0x43, -0x51,0xb3,0x8b,0x44,0x7e,0xf6,0x71,0xa8,0x8d,0x8e,0x98,0xa8, -0xff,0xb0,0xb3,0x79,0x44,0xb2,0xb7,0x55,0x62,0x76,0xc4,0xad, -0xd7,0x9f,0x2d,0x8a,0x0c,0x38,0x69,0x18,0x94,0x1e,0x49,0x01, -0xba,0x93,0xa7,0x57,0x55,0x58,0x9f,0x79,0xfc,0x00,0xf3,0x7a, -0x49,0x33,0x8a,0x46,0xbd,0x03,0x39,0x5a,0x58,0x01,0x6e,0xcb, -0x58,0x68,0x38,0x3d,0x1b,0xaf,0x9f,0xe9,0x96,0x19,0x42,0xc4, -0xa1,0x6c,0x51,0x9c,0xcd,0xec,0xc0,0x53,0x5b,0x87,0x9e,0x13, -0xec,0x1c,0x4e,0xfe,0x84,0x8b,0xc9,0x98,0xa6,0x94,0xaa,0x57, -0x9c,0x16,0x5f,0xa3,0xb5,0x74,0x1e,0xff,0x2c,0x1b,0xd8,0x60, -0x71,0x9a,0x2d,0x47,0x73,0x7d,0x51,0x3a,0xc3,0x9b,0x49,0x89, -0xc7,0x07,0xd8,0x11,0xa9,0x08,0xf4,0x36,0x36,0x36,0x7b,0x37, -0xc2,0xec,0x55,0xc8,0xda,0xa6,0xa4,0xe3,0xe6,0xc3,0x0e,0x7a, -0x5f,0x18,0x4f,0xc9,0x6b,0x3e,0x46,0xd1,0x09,0x23,0x3e,0x1e, -0xd6,0xd7,0x67,0xa6,0xf4,0x92,0x23,0x48,0x20,0xf3,0x62,0x52, -0xe7,0xb2,0xcd,0xc3,0x8c,0xeb,0x8f,0x9d,0x12,0xa4,0xcb,0x82, -0x76,0x0f,0x71,0xab,0x93,0x8f,0x4d,0x6c,0xce,0xb3,0x05,0xaa, -0xcf,0x7a,0xe0,0x82,0x7e,0xe0,0x04,0xad,0x3b,0xaa,0x89,0xc4, -0x75,0xa2,0xa5,0x02,0xf1,0x55,0x55,0xf0,0xe4,0xa5,0x79,0x3c, -0xc5,0x4b,0x34,0x22,0x0e,0x4c,0x1f,0x47,0x14,0xc4,0x12,0xe9, -0x17,0x68,0x80,0x1a,0xc8,0x13,0xb9,0x48,0xb3,0x2a,0x4d,0xcd, -0x4c,0x4d,0xb6,0xaa,0x20,0x07,0x06,0x52,0x84,0x53,0x21,0xd8, -0x05,0xd8,0x72,0xbf,0x98,0xfe,0xb2,0xd5,0x91,0xd6,0x48,0x00, -0xcf,0x76,0x1c,0xb0,0xc3,0x6a,0x84,0xb3,0x4b,0x3b,0xc8,0xd2, -0x8e,0xa1,0xf4,0x71,0x5a,0x84,0x08,0x5f,0x13,0x7a,0xd1,0x67, -0x93,0x55,0x76,0x91,0xcb,0xec,0xe8,0x73,0xbd,0xab,0x16,0x41, -0xca,0xd3,0x22,0x07,0x03,0xdd,0x2e,0x58,0x5d,0xa6,0x77,0x45, -0x9c,0xca,0xbc,0x67,0x54,0xd5,0x96,0x0d,0xb5,0x4c,0x0a,0x5a, -0x48,0x16,0x00,0xd2,0xa4,0xde,0x68,0x2f,0x93,0x76,0xbc,0x4c, -0x18,0xe9,0x0e,0xe4,0x5d,0x36,0x3a,0xcb,0x06,0xd5,0x94,0x42, -0x34,0xfe,0xb6,0xf2,0xc5,0x4d,0x52,0x04,0x1f,0x98,0xf6,0x44, -0x68,0xc1,0x7d,0xc3,0x71,0x31,0x56,0x1d,0x2d,0x1b,0x10,0x8e, -0x29,0xb7,0xbb,0xaf,0x74,0x00,0xbe,0xee,0x8d,0x27,0x29,0x22, -0x0a,0x51,0xfe,0x8e,0x8c,0x8e,0x35,0xf8,0x05,0x22,0xaa,0x8e, -0x89,0xef,0xa3,0xd4,0xec,0x1d,0x93,0x86,0x3a,0xe2,0xa3,0xca, -0x5e,0x3f,0x8b,0x3e,0xad,0xa2,0xda,0x8c,0xb6,0x2a,0x84,0xaf, -0x60,0x5b,0xe3,0xe3,0x4f,0xed,0x9c,0xe0,0x6c,0xc7,0xea,0x6f, -0x1d,0x1d,0x4c,0x27,0xbe,0xfc,0x76,0xab,0x32,0x7a,0x9f,0xd9, -0x1a,0x6e,0x0b,0x70,0x39,0x9d,0x2f,0x46,0x83,0x71,0x86,0xa5, -0xd6,0x6a,0x6e,0xca,0xcd,0xab,0xd1,0x9a,0x37,0x98,0x83,0xd1, -0x58,0x0e,0x94,0x27,0xb3,0xf3,0x68,0x1d,0xd7,0xe9,0xfb,0xc5, -0xb7,0x09,0x8c,0xc4,0xe0,0xc2,0x4b,0x20,0xe2,0xc2,0xac,0xef, -0xd6,0x9d,0xf3,0xde,0xf0,0x8c,0xb3,0x37,0x64,0x2f,0x01,0xf9, -0xc8,0x57,0xab,0x4b,0xd4,0x7f,0x54,0x33,0xad,0xa8,0x3b,0xab, -0x65,0x69,0xaf,0xd2,0x57,0xfd,0xab,0x74,0x5d,0xed,0xa9,0x1e, -0xfe,0xf4,0xaf,0xda,0xb3,0xda,0x32,0xed,0x55,0x7b,0x57,0x95, -0xa8,0x0f,0x41,0x1f,0x97,0xf3,0xd1,0x0c,0xc6,0x3e,0x48,0xcb, -0x97,0x5f,0x8d,0x66,0xb0,0xec,0x2f,0x36,0xd1,0xb9,0x7c,0x38, -0x0f,0x17,0x75,0xd4,0xe1,0x61,0x61,0x11,0x5c,0xf6,0x42,0x97, -0x95,0xba,0x08,0xbd,0x1c,0x8b,0xf7,0x14,0x3c,0xf1,0x72,0x76, -0x0b,0xaa,0xdf,0x81,0xf0,0x8b,0xf1,0x74,0x14,0x91,0x99,0x05, -0x90,0xb4,0xd1,0xe5,0x83,0xd5,0xb6,0x9b,0xcc,0x32,0xb4,0x19, -0xcc,0xe6,0xc4,0x75,0x06,0xfd,0x3a,0xd4,0x29,0xa2,0x4f,0x09, -0xb8,0x00,0xcd,0x11,0x46,0xc5,0x08,0x6d,0x56,0x90,0x25,0xb6, -0xc8,0xbc,0x95,0xb0,0x25,0x5d,0x20,0x27,0x63,0xfb,0x36,0xb4, -0x36,0x17,0x95,0x07,0x9d,0x8a,0x8f,0xe8,0x9d,0x50,0xe2,0x85, -0x9a,0x5d,0xb4,0xee,0xef,0xd4,0xf7,0x9c,0xf6,0x28,0xc5,0xcd, -0xfd,0x7e,0x36,0xd9,0x9c,0x93,0x3b,0xd6,0x7c,0x8b,0x45,0xf3, -0x72,0x57,0xfb,0xfc,0x86,0xcd,0x19,0x50,0xd0,0xb6,0xbd,0xa4, -0x75,0x9b,0xa8,0x12,0x07,0x69,0x81,0x55,0x8c,0x37,0x5a,0xcc, -0x79,0x89,0x7e,0x37,0xb9,0x26,0x7b,0x20,0x80,0x53,0x02,0xeb, -0x60,0xbc,0xff,0x9b,0xfa,0xed,0x2f,0xf1,0xcd,0xfb,0xbf,0xb9, -0x97,0x6b,0x10,0x95,0xfe,0xf6,0x97,0xb2,0x8d,0xa8,0x37,0x2b, -0xbf,0xfd,0xb9,0xfe,0x90,0x0e,0xd3,0x86,0xef,0xff,0x96,0xbe, -0xff,0x5b,0xf5,0xfd,0xdf,0x1a,0x40,0xc9,0x0c,0x72,0x9a,0x0b, -0x3a,0x00,0x84,0x46,0x46,0x75,0x14,0xde,0xcc,0x61,0xd4,0xdb, -0x94,0x22,0x1b,0x65,0xfa,0x51,0xab,0x94,0x8a,0x35,0xca,0xf4, -0x53,0x79,0x5b,0x36,0xe5,0x01,0x32,0x6c,0xf3,0x53,0x89,0xc1, -0xe2,0x18,0xd3,0x0e,0x1b,0xb3,0xba,0xa4,0xb7,0x20,0xce,0xdd, -0xd1,0x95,0x5a,0xc1,0x14,0x0e,0x18,0x01,0x62,0x8c,0x54,0x83, -0x2b,0x61,0x74,0x1a,0x8c,0x00,0xed,0xfa,0x21,0x53,0x03,0x9a, -0x07,0x7f,0x4c,0x03,0x1a,0x92,0xcb,0xad,0x88,0xd9,0xc8,0x8a, -0x64,0x42,0xa6,0x7c,0x0e,0x43,0x49,0xaa,0x3e,0x42,0xf0,0x80, -0x44,0xec,0xd0,0x4e,0x23,0xe4,0x90,0x29,0x8d,0x10,0x03,0x20, -0x71,0x9c,0xa7,0x71,0xac,0x71,0x76,0x69,0x88,0x52,0x9f,0xad, -0x59,0x18,0xd5,0x15,0x18,0xdd,0x0e,0x95,0x06,0x06,0x07,0x46, -0x58,0x7a,0x00,0x6c,0x30,0xca,0xa0,0xb2,0x28,0x9b,0xa3,0x97, -0x1c,0xd4,0x88,0xbf,0x61,0xaa,0xa4,0x26,0x06,0x07,0x0c,0xa0, -0x68,0xc3,0x34,0x64,0xc2,0x8e,0xd6,0x1a,0x1d,0x94,0x62,0xbb, -0x10,0x7b,0xd0,0x89,0x42,0xf4,0x31,0xca,0x6d,0x4b,0x77,0x37, -0x5a,0x5f,0xcc,0x96,0xbe,0x3e,0x39,0xe0,0x52,0xee,0xc3,0x9f, -0x4a,0x0f,0xb0,0x80,0xaf,0x04,0xbf,0x9a,0xf8,0xd5,0xdc,0x03, -0xe6,0xd9,0x62,0x96,0x65,0x2e,0xa0,0x13,0x29,0xde,0xec,0x56, -0xa5,0x38,0xb4,0x49,0xbe,0x1a,0x18,0xd7,0xe0,0x54,0x25,0x5f, -0x09,0xc6,0x25,0x9c,0xba,0xa7,0x1e,0x54,0xef,0xa5,0x5a,0xb0, -0x58,0x25,0xc5,0xdc,0x0a,0xcb,0xe1,0xa7,0x54,0x80,0x9f,0xfb, -0x50,0x65,0x8b,0x11,0x57,0xfc,0xa2,0xd4,0x08,0xd8,0x27,0x1c, -0x81,0xc8,0xbc,0x55,0xf2,0x9d,0x38,0xdf,0x80,0xf8,0xdb,0x3d, -0x40,0xbf,0xc5,0xe5,0x6d,0x32,0xbe,0x1e,0x45,0x43,0xad,0x4a, -0x69,0x6f,0x53,0x87,0x08,0x15,0xff,0x54,0x86,0x08,0x13,0xff, -0xc0,0x57,0x13,0xbf,0x70,0x36,0xc4,0x94,0x7a,0x3a,0x51,0x18, -0xcf,0xbf,0x4d,0xfc,0xcd,0x4f,0x89,0x9a,0x3d,0x22,0x53,0xad, -0xc1,0xde,0x19,0x99,0x26,0x91,0xb8,0xc8,0x04,0xe4,0xe2,0x89, -0xd4,0x64,0x6c,0x2c,0xd4,0x9f,0x6f,0x95,0xa9,0xe8,0x39,0x3e, -0x0a,0xf1,0xd8,0xc3,0x88,0x1f,0x48,0xff,0x2a,0x92,0x3f,0x7e, -0xf2,0xfe,0xbf,0x0e,0x0f,0x9d,0xf8,0x04,0xe3,0x13,0x8a,0xcf, -0xd3,0x6b,0x84,0x3e,0x11,0xc6,0xc3,0x3d,0x13,0xa1,0x78,0x3a, -0x09,0x98,0xdf,0xd3,0x43,0x68,0x17,0x41,0xfb,0xf1,0xa7,0x3f, -0x45,0x1f,0x3e,0x32,0xf2,0x13,0x78,0x0e,0xe8,0x5a,0x6d,0xd4, -0x75,0x7c,0x43,0xca,0xa1,0x6e,0x86,0x3f,0x36,0x54,0x10,0xf3, -0x63,0xa3,0x92,0x46,0xeb,0x6a,0x2e,0x96,0xee,0x1a,0xf6,0x17, -0xfd,0x09,0x8b,0x6e,0xc2,0xa2,0x3f,0xdd,0xa7,0xe8,0x9f,0xb0, -0xe8,0x75,0x58,0xf4,0x4f,0x45,0x45,0x77,0x36,0xf6,0x85,0x5d, -0xf7,0xef,0xbd,0x62,0xdd,0xa3,0xa3,0xd4,0x3a,0x2d,0xe8,0x92, -0x4d,0x5a,0xd0,0x29,0xd7,0xfe,0x02,0xf6,0x2e,0x75,0x38,0xda, -0x8e,0xa4,0xe8,0x5d,0xba,0x69,0x94,0xaf,0xab,0xd7,0x8d,0xf2, -0x26,0x2e,0xbf,0xab,0x40,0x18,0x3e,0xd7,0xd5,0x35,0x44,0x4a, -0x18,0x3e,0x37,0x55,0xc8,0xb4,0x86,0x30,0x20,0x00,0x1f,0x15, -0x08,0x6d,0x2a,0xd7,0x98,0x25,0xe4,0x9c,0x3f,0x26,0x95,0xf4, -0x5d,0xae,0x1b,0x31,0xb2,0x1c,0xad,0x1b,0x15,0xf8,0x97,0xae, -0x79,0xa7,0xe2,0x75,0x0b,0x67,0xd8,0x40,0x86,0x4d,0x23,0xdd, -0xe4,0x33,0xfc,0x89,0x33,0x5c,0x43,0x86,0xeb,0x46,0x7a,0x9d, -0xcf,0x60,0x99,0xab,0xa1,0x36,0x0d,0x75,0xdd,0x90,0xf3,0x71, -0x09,0x85,0x68,0xee,0x58,0x1a,0x3f,0x7c,0xe0,0xac,0x1b,0x85, -0x1d,0xd2,0x28,0xec,0x11,0x7f,0x3d,0x0c,0x9b,0x90,0x5b,0x8c, -0xef,0x6e,0xe2,0x5e,0xf6,0x23,0x71,0xea,0xa6,0xb8,0x92,0x22, -0xd0,0x3b,0x81,0xfd,0x3d,0xd2,0xd7,0xbf,0x94,0x97,0x07,0x6b, -0xcb,0xb4,0x6a,0xb0,0xb1,0x1c,0xab,0x06,0xd7,0x96,0x5d,0xd5, -0xa5,0xb3,0x62,0x0c,0xd6,0xe5,0xc1,0xba,0x32,0xd8,0x94,0x07, -0x9b,0xca,0xe0,0xba,0x3c,0xb8,0x8e,0x41,0x86,0xf3,0x79,0x19, -0xa4,0xb6,0xcb,0xc3,0x43,0x1c,0xf6,0xa4,0x39,0xb9,0x8a,0xeb, -0x97,0xfe,0xa8,0xb9,0x04,0x09,0x2d,0x64,0xa4,0x1f,0x9b,0x95, -0xf4,0x0a,0x40,0xe7,0x38,0x9a,0xe3,0x37,0x39,0x46,0xe6,0xf8, -0xeb,0x30,0xfe,0xff,0x61,0xe3,0xc6,0x4a,0x98,0x1f,0x3c,0x82, -0x48,0x76,0x54,0x22,0x1b,0xde,0xc9,0xe3,0xff,0xad,0x23,0xae, -0xa0,0xf1,0x39,0x84,0xef,0x29,0xa7,0xee,0x2c,0x57,0x3c,0x30, -0x77,0xd0,0xa5,0x70,0x58,0xfa,0xc7,0x34,0x8b,0xd5,0xc8,0x9c, -0xad,0x1e,0x14,0xac,0x3f,0x93,0xf1,0x5c,0xef,0xf1,0xc4,0x41, -0x62,0xa6,0xe4,0xd9,0x07,0xbe,0x93,0x98,0xd3,0x40,0xfe,0x7a, -0x4a,0x16,0x33,0x1d,0x6f,0x91,0xc1,0x26,0x30,0x5b,0xf5,0xd9, -0x33,0x53,0x97,0x0a,0xa1,0x45,0x5b,0x54,0x8e,0x17,0x80,0xf9, -0xdb,0x0c,0xed,0xfe,0x91,0x0c,0xd7,0x45,0xd1,0xd4,0x38,0x7c, -0x34,0x17,0xe8,0x4f,0xd2,0x86,0xbc,0xa3,0x9a,0xaa,0x79,0x43, -0xa7,0xa3,0xb4,0x39,0x4f,0x4c,0x68,0x4a,0xd5,0x14,0x0b,0x4d, -0xf3,0x06,0xe4,0x04,0x10,0x85,0x5b,0x6b,0xc7,0x5c,0x95,0xb1, -0x49,0x15,0xec,0xa6,0x23,0xa7,0xd6,0x71,0x37,0xa6,0x9a,0x1b, -0x28,0x56,0xe9,0x53,0xa5,0xfc,0x1e,0x5a,0x5f,0xd7,0x92,0x03, -0x8b,0xf4,0x86,0xe0,0xb4,0x10,0x0f,0xfc,0xc8,0x5a,0xda,0x27, -0x26,0xb9,0x3e,0x67,0x87,0xfa,0x57,0xe3,0x6c,0xbc,0x1c,0x0d, -0x5b,0x07,0x89,0xc2,0x3e,0xdc,0xa0,0x33,0x44,0xa1,0x25,0xf9, -0x7f,0xef,0xe7,0xa1,0x9c,0xcc,0x01,0x13,0x06,0xd1,0x2b,0x28, -0x9f,0xd8,0xf2,0xc9,0xb6,0xdd,0x63,0x3f,0xeb,0x69,0x5f,0xc7, -0xb2,0xe5,0xa4,0x5e,0x4c,0xdd,0xc4,0x81,0x7e,0xac,0x2c,0xb2, -0x89,0xad,0x26,0xe9,0xee,0xc7,0x34,0xd9,0x81,0x69,0x11,0x88, -0x22,0x4c,0x1b,0x2e,0xa6,0xea,0x7e,0x98,0xe2,0x3d,0x15,0x85, -0xf1,0x61,0xaf,0x3c,0xd4,0x1d,0xd9,0xc1,0x64,0x99,0x19,0xc6, -0xcd,0xdb,0x67,0xe3,0xc5,0x00,0x6d,0xe4,0x45,0x02,0xf1,0xce, -0x7c,0x18,0x1f,0x9b,0xfa,0xc5,0x44,0xdf,0x8d,0x6b,0xda,0x8c, -0x50,0x4f,0xc3,0x71,0x31,0xa5,0x98,0xd0,0xc6,0x19,0xc6,0xa1, -0xfd,0x2f,0x2e,0xc3,0x7f,0x0f,0xe8,0xc7,0xb0,0x9f,0x78,0x1a, -0x12,0x82,0xf1,0x0f,0x1b,0xcf,0x4c,0x05,0x0b,0x60,0xbb,0x76, -0x9b,0x71,0xd0,0x2e,0x10,0x29,0xdd,0x38,0x16,0x14,0xba,0xb6, -0xf1,0x2d,0xa2,0xc9,0xa2,0x13,0xe9,0x14,0x94,0xf5,0x13,0x16, -0xda,0xb6,0x44,0x9b,0x2b,0x33,0x79,0xa4,0xee,0xc2,0x61,0x32, -0x9c,0xdd,0x58,0x0f,0x8b,0xba,0x2a,0x53,0x92,0x3a,0xcc,0xc4, -0x42,0x7f,0xea,0x04,0x6a,0x64,0xec,0x16,0xd5,0x3d,0xe0,0x8e, -0xbb,0xf1,0x13,0xae,0xda,0x75,0xc1,0x93,0x1b,0x83,0xf6,0xfd, -0x46,0x66,0x46,0x21,0x86,0x68,0x20,0xb2,0x67,0x0a,0xc7,0x85, -0xad,0xd7,0x26,0xe5,0x52,0x41,0xa2,0x12,0x3b,0x6f,0xb5,0x8b, -0x88,0x45,0xaa,0x14,0x45,0x88,0xdf,0x84,0x54,0x5b,0x8c,0xae, -0x84,0x74,0xce,0x64,0xe2,0x37,0xa8,0x5a,0x1d,0x1f,0x43,0x3b, -0x3f,0xac,0x51,0xdb,0x7b,0x36,0xca,0x22,0xa0,0xaa,0xfb,0x5a, -0x85,0xf9,0xb6,0x61,0x24,0xf5,0x9c,0x2a,0x64,0x84,0xed,0xbb, -0x0b,0xf4,0xc0,0x79,0x10,0x74,0x7a,0xe1,0x91,0x61,0x81,0x3c, -0xb9,0x63,0x64,0xd1,0x8d,0x33,0xdb,0x1d,0x4c,0xe9,0x3b,0x37, -0xba,0xa6,0xaa,0x4f,0xef,0xd5,0x7b,0x9c,0x8e,0x7c,0x2f,0x67, -0x84,0x3d,0xbe,0x1d,0xe8,0x4b,0xc2,0xb8,0xab,0xfa,0xd4,0xaa, -0xb4,0x07,0x99,0xfb,0xed,0x20,0xb9,0x61,0x93,0x8b,0xd1,0x63, -0xf2,0xff,0x01,0x9a,0x85,0x26,0x97,0x31,0x06,0xd7,0x58,0x6f, -0x6d,0x33,0xe3,0x3b,0xaf,0x29,0xb5,0x98,0x2d,0xfd,0xd5,0xcf, -0x54,0x31,0x77,0x85,0x5b,0xfb,0x04,0x8c,0x0b,0x48,0x7c,0xdb, -0xad,0x1b,0xe2,0x98,0x09,0xe4,0xf0,0xc9,0xb0,0x80,0x35,0x74, -0x57,0x73,0x81,0x6e,0xfd,0xaa,0xe8,0xb4,0xf4,0x8e,0xea,0xb0, -0xab,0x04,0x86,0xa9,0xc9,0x72,0x9a,0x81,0xe7,0x0c,0xf9,0x1b, -0x9e,0x6d,0x2d,0x87,0x12,0x71,0x08,0x8e,0x93,0xcb,0x2f,0xca, -0x1b,0x87,0xb0,0xa0,0x2d,0x14,0x9c,0xd3,0x52,0x2a,0xee,0x11, -0x34,0xf6,0xa8,0x42,0xca,0x13,0xfc,0x09,0xc5,0xc8,0x33,0xbc, -0xa2,0x06,0x61,0xd6,0x17,0x3e,0x71,0xf6,0x35,0x6c,0xe1,0xec, -0x44,0xbc,0xa2,0x4e,0x5b,0x14,0xb9,0xd9,0x3c,0xe9,0xfa,0xa5, -0xb8,0x4d,0x16,0x55,0x8c,0xe3,0x27,0x6f,0x4a,0x7f,0xa2,0xba, -0x70,0x0e,0x28,0x95,0x6b,0x3b,0xfe,0xac,0xc9,0x66,0xf3,0x34, -0xf5,0x32,0x52,0x94,0xd4,0xfc,0x4a,0x64,0xa4,0xb4,0xbf,0x3a, -0x3b,0x83,0x34,0xfe,0x81,0x44,0x2e,0xa3,0x93,0xcd,0x24,0x79, -0x16,0x31,0xb9,0xd0,0xbc,0xa5,0x92,0x83,0x70,0x26,0xde,0x82, -0x6c,0x54,0x50,0x6b,0x68,0xc5,0x9e,0xd2,0x78,0x4b,0x0e,0xa9, -0xb6,0xf8,0x46,0xf0,0xf1,0xa0,0xe2,0x28,0x33,0xa4,0x56,0x79, -0x29,0x8c,0x2e,0x2f,0xee,0xbc,0xa0,0x28,0x9e,0xdf,0x42,0xd9, -0xc9,0xcc,0xda,0x77,0x8a,0x4f,0xd3,0xe3,0xe4,0xf0,0xb0,0xc9, -0x78,0x1f,0x1e,0x7a,0x54,0x30,0x0d,0xb5,0x31,0x48,0x08,0xad, -0xdb,0xe2,0xa5,0x64,0x17,0xe3,0x33,0x74,0x60,0x84,0xcf,0x46, -0x77,0x16,0x17,0x0b,0x24,0xce,0xfc,0x20,0x49,0x6c,0x09,0x37, -0x89,0xc5,0x6b,0xa2,0x91,0x8d,0x85,0xe4,0x44,0x67,0x62,0xf1, -0x54,0x4f,0x21,0x86,0x46,0xe8,0x1a,0x1e,0x99,0x76,0xf8,0x4c, -0xcf,0x23,0xc2,0xc5,0xfc,0xb2,0x70,0x19,0x99,0xf9,0x85,0xee, -0x69,0x4c,0x00,0x19,0x8a,0xa4,0x67,0x2d,0x49,0x99,0x61,0x44, -0xe9,0x2d,0xf3,0xa5,0x84,0x64,0x2d,0xf9,0x55,0xee,0xe5,0x8a, -0xab,0x30,0x1a,0x8e,0xc9,0x97,0x6c,0x4a,0x1c,0x65,0x15,0x0d, -0x2b,0x35,0x63,0xc4,0x26,0xe0,0x3e,0x44,0x06,0x81,0xa1,0x1d, -0x8a,0xf6,0x52,0x8d,0x6b,0x96,0x34,0xb8,0xd7,0xd9,0x2a,0x7b, -0x37,0xb3,0x0f,0x91,0x10,0x09,0xdb,0x34,0x0f,0x09,0xdd,0x3e, -0x83,0xc4,0xf0,0xa8,0x76,0x39,0x5a,0x9c,0x8f,0xcc,0xd6,0x42, -0xde,0xa4,0xfa,0x02,0x99,0xde,0x17,0x51,0x55,0x5f,0x4f,0xf5, -0xfe,0x26,0xec,0x15,0x8d,0x6b,0xdc,0xce,0xfc,0x51,0xd6,0xd9, -0xb7,0x39,0x72,0x59,0x65,0x26,0x08,0xdf,0xb9,0x49,0x6a,0x05, -0xb9,0xd0,0x9c,0x6a,0xd1,0x5c,0xe4,0xae,0xfb,0x3c,0x88,0xf1, -0x8f,0xb3,0xbe,0xab,0xfc,0xb0,0x89,0x73,0x9d,0xc1,0xa3,0xc9, -0x10,0x4d,0x77,0x1b,0xc2,0xda,0x2a,0xda,0x26,0x8d,0xdc,0xce, -0xd9,0xd1,0x97,0x85,0x22,0xe1,0xdf,0x85,0x61,0x31,0x82,0xdb, -0xad,0xe2,0xc9,0x2e,0x75,0x48,0xfa,0x39,0xc5,0x18,0xab,0x07, -0xfe,0x0c,0x6b,0x87,0x1a,0x17,0x34,0x73,0x09,0xc6,0x17,0xaf, -0xf6,0xfe,0x68,0xb6,0xfb,0x4e,0xfd,0x4c,0xde,0x9b,0xf0,0x8e, -0x93,0x42,0x18,0x21,0x4e,0x37,0xe2,0xb0,0x9c,0xd7,0xc5,0xcc, -0xba,0x6f,0xb9,0xb1,0x63,0xd5,0x23,0x2f,0x64,0xd2,0x16,0x6b, -0x61,0xbe,0x38,0xe9,0xd2,0x30,0xc1,0x11,0x1e,0x9e,0x86,0xf0, -0x7a,0xed,0xae,0x84,0x5b,0x33,0xd6,0x87,0x6c,0x18,0x43,0x68, -0xe6,0x56,0x20,0x6e,0x64,0x90,0x92,0x54,0x97,0x9d,0x61,0x19, -0x3b,0x9e,0xa6,0xa8,0x9f,0x38,0x1b,0xea,0xa3,0xa2,0x69,0xf1, -0x3c,0x96,0x9a,0x0e,0x28,0x82,0x78,0x58,0x07,0x13,0x2d,0x47, -0x99,0x19,0x76,0x87,0x24,0x88,0xe3,0xc3,0xbd,0xbc,0x88,0x22, -0x90,0xee,0x78,0x1a,0xc0,0x35,0xe1,0x49,0xa3,0x43,0x97,0x15, -0xfc,0x88,0xb8,0xfa,0xfe,0xbf,0x5a,0xf2,0x85,0xb1,0x71,0x35, -0x8a,0xfa,0x69,0xdf,0xcb,0xdd,0x2f,0xcc,0xdd,0xf7,0xd7,0xfc, -0xe2,0xb1,0xcf,0xb3,0x8e,0x1e,0xf0,0xfc,0xce,0x69,0xb4,0x18, -0xe3,0x19,0x91,0x15,0xc2,0xd0,0x84,0xd3,0x64,0x32,0x9a,0x18, -0x51,0x4c,0xe9,0x3c,0x7c,0xbb,0x94,0x9e,0x98,0xd3,0x21,0x9d, -0x10,0xab,0xaa,0x39,0x57,0xb2,0x71,0xb4,0xde,0x81,0xdc,0xfb, -0x14,0xdd,0x47,0x82,0x50,0x8b,0x0a,0xcf,0xb8,0x2e,0x37,0x8a, -0xee,0x75,0x51,0x9b,0x2e,0x38,0x93,0x50,0x53,0x3d,0x68,0x0b, -0x0d,0x67,0xd3,0x1a,0x2f,0x19,0x50,0x1e,0xbe,0xa4,0xd9,0xda, -0x11,0x11,0x2e,0xcd,0x46,0x8b,0x5a,0xd2,0x48,0x45,0x66,0xa1, -0xfb,0x5a,0x8e,0x62,0xa1,0xd3,0x84,0x92,0xae,0x5c,0xc5,0x16, -0xdc,0xc4,0x36,0xe2,0x82,0xeb,0x5c,0x88,0xfc,0x25,0x4d,0x70, -0x6f,0xfa,0x0b,0x6c,0x30,0x2f,0xd1,0x56,0x12,0x6a,0x5d,0x89, -0xd0,0x86,0xd5,0xfd,0xc2,0xb2,0xc5,0x0e,0x29,0xd7,0xab,0x2e, -0x38,0xae,0xcd,0xdf,0xb0,0x3b,0xf7,0xef,0xbd,0x29,0xe9,0xf2, -0x70,0xd7,0x99,0x5b,0x2f,0xbc,0x34,0x3f,0xfe,0xed,0xcf,0xc1, -0xbd,0xbb,0x73,0x3a,0xb4,0xe7,0x26,0x3d,0xb8,0x7b,0x57,0xfb, -0xae,0xea,0xe3,0xd8,0xe9,0xd9,0x4a,0xea,0x22,0xd3,0x81,0xf4, -0x0a,0x66,0x82,0x4d,0x5f,0xa7,0xd9,0xaa,0x36,0xe3,0xf2,0x6f, -0x7f,0x6e,0x41,0xd8,0x43,0xf9,0x67,0x68,0xc3,0x71,0xea,0x84, -0x6c,0x80,0x3b,0xb7,0xb7,0x18,0xa4,0xc5,0x97,0xbb,0xb9,0x0b, -0x6d,0xee,0x3b,0x7b,0xac,0xe1,0x27,0xc4,0xf6,0xf0,0x39,0x7f, -0x49,0x0a,0xb5,0xf0,0xd2,0x49,0xf3,0xb9,0xb8,0xa1,0xd9,0x51, -0xaf,0x3f,0x08,0x14,0x15,0xdd,0x0d,0xd9,0x05,0xc8,0x55,0xfc, -0xf6,0x17,0x6c,0x53,0xe4,0x51,0x41,0xc8,0x54,0x4d,0x5a,0x09, -0xe9,0x03,0xd0,0xad,0xa8,0x5b,0x94,0x2e,0x66,0x23,0x3d,0x22, -0x8f,0x09,0xc6,0xed,0xad,0x19,0xa1,0x69,0x4a,0x31,0xc0,0x76, -0xf0,0x17,0x58,0xeb,0xf6,0x16,0x7f,0x61,0x22,0x40,0x4d,0x47, -0x19,0x6e,0x7e,0xf7,0xe8,0x2a,0x93,0x56,0x35,0x21,0x8f,0x81, -0x07,0x68,0x3f,0xad,0x0f,0x6c,0xf1,0xb6,0x2d,0x5a,0x14,0xae, -0xe2,0x84,0xaf,0x52,0x21,0xc3,0x68,0xce,0x07,0xac,0x32,0x9b, -0xc1,0x1c,0x74,0x6c,0x99,0xe1,0xf6,0xd6,0x0b,0x1e,0x1e,0x92, -0x91,0x8f,0x80,0xf5,0xe2,0x9f,0x93,0x43,0xc1,0xae,0x70,0xc6, -0x7c,0xea,0x20,0xec,0xcb,0x94,0xac,0x56,0x84,0x22,0x31,0x0d, -0x62,0x7c,0x4e,0x86,0xe8,0xd1,0xb3,0x32,0x89,0xb8,0x63,0x15, -0x2a,0x58,0xcc,0x79,0x6f,0x92,0x14,0x2c,0x47,0x09,0x40,0x4f, -0xe4,0x6c,0x17,0x42,0x40,0x9d,0xe4,0xb8,0x81,0xa6,0x22,0xaa, -0x30,0xc8,0x70,0x34,0x9a,0x91,0x07,0x29,0x38,0x2e,0xe3,0xb6, -0x3b,0x16,0x21,0x17,0x5e,0x3f,0x77,0xa2,0xdc,0x4e,0x96,0xd1, -0xc6,0x09,0xa4,0x82,0x35,0xd4,0x9b,0x00,0x56,0x9b,0x94,0xe0, -0xdf,0x38,0xd4,0x6b,0xca,0xa4,0xd4,0x7e,0xd1,0xc2,0x6d,0x57, -0x1e,0x40,0x12,0x00,0xd0,0xe8,0x48,0x3c,0x13,0xa2,0x11,0xb7, -0xb0,0xae,0x83,0x34,0xc5,0x12,0x87,0x87,0xc4,0x31,0xbf,0xfd, -0x19,0x38,0xca,0x69,0x6d,0xa3,0x8a,0x79,0xf8,0xd2,0x9d,0x82, -0x98,0xb9,0x51,0x7e,0xff,0x5f,0x72,0xaf,0xad,0x69,0x82,0x20, -0x4c,0xae,0x84,0x72,0x25,0x94,0x0b,0xdb,0xbf,0xa3,0xcb,0xbf, -0xd6,0x03,0x40,0x13,0x4a,0xe9,0x9e,0xf8,0x17,0xd2,0x44,0xb7, -0x3d,0x4f,0x1c,0x1c,0x14,0x94,0x29,0x25,0x14,0x88,0xcf,0xb0, -0x9c,0x15,0x4a,0x76,0x31,0x18,0xe3,0x81,0xf9,0x85,0x49,0xb7, -0x5c,0x4d,0xc1,0x6b,0xb6,0x66,0x95,0x52,0x76,0x88,0x10,0xf7, -0xa3,0xce,0x8d,0xb9,0x07,0x12,0xf5,0xa6,0x04,0xc7,0x32,0xe4, -0x3a,0x45,0xde,0x75,0xae,0x78,0x1a,0x55,0xec,0x95,0x76,0xa8, -0x4b,0x61,0x33,0xc7,0xc7,0xc0,0xb5,0x66,0x6d,0x88,0x22,0x6f, -0x19,0x2c,0xb3,0x7e,0x43,0xe2,0xae,0x4d,0xb0,0x43,0x74,0xef, -0x90,0x92,0xb8,0xea,0x14,0x49,0x74,0x91,0x60,0xe9,0xf4,0x8a, -0x40,0xb0,0x1e,0xb9,0x6b,0x4e,0x52,0x76,0xf0,0x89,0x5b,0xce, -0x58,0xb9,0x1f,0x89,0x44,0x3a,0x47,0x17,0xef,0x6a,0x39,0x43, -0x67,0x50,0x62,0x7e,0xd1,0x9f,0x4c,0x78,0x79,0x64,0x0d,0x75, -0xcc,0x1b,0xc3,0xa2,0x6b,0xf2,0x96,0x79,0x2c,0x86,0x3c,0x81, -0xa3,0x1f,0xd7,0xe3,0x20,0xba,0x51,0x14,0x59,0x9c,0x15,0x62, -0xf3,0xac,0x06,0x91,0xd5,0x42,0xb0,0x45,0xb1,0xd5,0x1d,0xb9, -0xab,0x85,0xa0,0x05,0x63,0xe3,0xc5,0xd8,0xf4,0x39,0x36,0x19, -0x55,0x68,0x96,0x33,0xd4,0xa1,0x81,0x6e,0x97,0xd9,0x2e,0xd5, -0x29,0x4f,0x28,0x85,0xd7,0x0b,0x58,0xc9,0xdb,0x1e,0x79,0xb2, -0x7a,0x33,0x57,0x53,0x76,0x6f,0xd2,0x50,0x4e,0x3e,0xe9,0x0d, -0x52,0xa8,0x4e,0xe8,0xb5,0x42,0x99,0x16,0x7b,0x1b,0x8f,0x54, -0x27,0xa3,0x08,0xaf,0x37,0xf1,0xc5,0xa1,0xb5,0xd0,0x62,0x0e, -0x15,0xf9,0x4c,0xcd,0xe1,0x70,0xb9,0xf3,0x2c,0xbb,0x12,0xd5, -0xf1,0x60,0xe1,0x5a,0xf4,0x0e,0x4f,0x30,0xec,0x89,0x22,0x0c, -0xa8,0x86,0xba,0xc2,0xff,0x1b,0x3c,0x84,0xf7,0xae,0x32,0x90, -0x2b,0xbd,0x6a,0xa0,0x33,0xd6,0xdd,0x4b,0x8b,0x7f,0x62,0xd9, -0xe4,0x0c,0x49,0x2a,0x3b,0x1e,0x75,0x95,0xfa,0x4d,0x51,0x83, -0x94,0xfc,0xcb,0xbd,0xa4,0x16,0x77,0xae,0x3a,0x8d,0x16,0x7a, -0xad,0x93,0xd4,0xd6,0x55,0x47,0x42,0x95,0xa8,0x71,0xfc,0xfe, -0x6f,0xb2,0x44,0xc5,0x94,0x46,0x46,0x0b,0x0e,0xb8,0x19,0x30, -0x0d,0x0b,0x72,0x57,0xee,0x01,0xab,0x3b,0x43,0x5e,0xc1,0xc4, -0x7f,0x85,0x19,0x19,0xb1,0xd4,0x88,0x23,0x22,0x68,0x09,0xa6, -0xb1,0xda,0x75,0xe3,0xe8,0xe4,0x6a,0xc6,0x64,0xbc,0x6b,0x57, -0xae,0x44,0xe7,0x8a,0x75,0x75,0x09,0xa9,0xd3,0xbd,0xff,0x2f, -0xa9,0x84,0x34,0xea,0x20,0x64,0xa9,0x61,0x72,0xd9,0x1c,0x28, -0x8d,0x32,0xd2,0xb1,0x4c,0xe2,0xea,0xaa,0x53,0x7c,0xb0,0x50, -0xdc,0x24,0x41,0x23,0x3f,0x6a,0x38,0xbb,0xa9,0x8c,0x6c,0x23, -0x68,0x9b,0x5b,0x7b,0x08,0x73,0x0f,0x28,0x85,0x07,0x18,0xae, -0x78,0xd5,0x34,0x43,0x75,0x3a,0x5b,0xfe,0x7e,0x74,0x39,0xe6, -0x03,0x8b,0xc3,0x43,0xdd,0x8f,0x0e,0x37,0xfc,0x7c,0x25,0x16, -0x0f,0xdb,0x83,0xc3,0x41,0xe3,0xf6,0xf6,0x20,0x5a,0xee,0x6a, -0x22,0x3e,0x29,0xc3,0x67,0xa2,0x9a,0x50,0x2e,0x4f,0x15,0x93, -0x2c,0x1c,0x99,0x72,0xfc,0xbb,0xd4,0x67,0xbf,0xb9,0xf4,0x44, -0xd2,0x93,0xee,0xae,0x86,0xb6,0x42,0x79,0xe8,0x1e,0x85,0xee, -0xb5,0x9a,0x87,0xc8,0xc1,0x96,0xfc,0xe0,0x0a,0xc4,0x65,0x21, -0xd9,0xdd,0xfc,0x9a,0xa0,0x2d,0xc7,0x82,0xde,0x0b,0x18,0xce, -0xeb,0xa9,0x44,0xe1,0x68,0x82,0xf9,0x21,0x1d,0x14,0x4a,0x04, -0x38,0x96,0x0a,0xda,0x23,0x20,0xf8,0x34,0x6a,0x97,0x54,0x40, -0xf1,0xb7,0x38,0x64,0x0f,0x0f,0x81,0xbd,0x9f,0x3c,0x49,0x5c, -0x01,0xc1,0xf6,0x31,0xbe,0x44,0x58,0xbe,0x93,0xb7,0x08,0xf3, -0x5e,0x6e,0x07,0x83,0x17,0xc8,0xf3,0x7e,0x3e,0xba,0x1f,0xab, -0x29,0xcc,0x39,0x89,0x6a,0xe0,0x66,0x7d,0xda,0xdc,0xb1,0xf5, -0x99,0xf7,0xa0,0x34,0x64,0x6d,0x16,0xe4,0x40,0xc5,0xdd,0x69, -0x13,0xd2,0x10,0x14,0xa4,0x4f,0x89,0xd3,0x87,0x23,0x40,0xed, -0x72,0x3c,0xed,0xa1,0xda,0x0a,0x14,0xab,0x62,0x5a,0x19,0xff, -0xd0,0x64,0xe4,0x24,0xcb,0x7d,0xec,0x01,0xa0,0x7f,0x78,0xd8, -0xe3,0x73,0xcc,0x24,0x1d,0x2c,0xca,0x58,0xac,0xee,0x64,0x54, -0x83,0x66,0x5a,0xc5,0xf8,0x24,0x88,0x9f,0x26,0xeb,0x9d,0x98, -0x4f,0x13,0xc2,0xec,0x69,0x2e,0x99,0xb5,0x6f,0x21,0x79,0x00, -0x03,0xf6,0xf3,0x5d,0xc9,0x4d,0xa8,0x34,0xbf,0xcb,0x43,0xe5, -0xdf,0xa7,0xea,0x73,0xde,0xd6,0xad,0x52,0xaa,0x9f,0x4c,0x67, -0xe6,0x29,0xf3,0x54,0xad,0x62,0xb5,0x5a,0x15,0xa6,0xad,0x30, -0x0d,0xe6,0x91,0x77,0xe5,0x77,0xd5,0xd5,0xaa,0x9c,0xdb,0xd8, -0x72,0xf9,0xa7,0x68,0xb9,0xad,0xfd,0x3f,0x48,0x43,0xa4,0x71, -0x8c,0x73,0x25,0x8f,0x75,0x47,0x4b,0x0b,0x22,0xd5,0xaf,0x3b, -0x9a,0xb0,0x52,0x51,0xf5,0x5d,0x75,0x19,0xd7,0x57,0xab,0xd8, -0x39,0x07,0xf0,0xda,0xf2,0x2b,0x54,0x62,0x01,0x58,0xf5,0xde, -0x5f,0x63,0x85,0xdd,0xa2,0xbd,0x31,0xfc,0x4a,0xed,0xbd,0x26, -0x79,0xb6,0xc7,0xc7,0x28,0x09,0x6b,0x3e,0xa3,0x64,0x87,0xa7, -0x55,0x28,0x85,0x92,0x02,0x34,0x6e,0x25,0x8f,0x69,0xf3,0x10, -0x5d,0xa7,0x28,0xa6,0x6a,0x29,0x1a,0x8b,0x5c,0x33,0xe1,0xde, -0xff,0x95,0x4e,0x33,0x68,0xdb,0xc4,0xa7,0x09,0xce,0x6e,0xea, -0xaf,0x66,0xf3,0xa4,0x9c,0x03,0x2a,0xc8,0x43,0xfb,0x4b,0x4c, -0x96,0x45,0x0d,0xa2,0x0e,0x0f,0xa1,0x7e,0xd8,0x1a,0x73,0x6d, -0x28,0xfc,0x8a,0xac,0x4e,0xe8,0x5c,0xc7,0x06,0x42,0x87,0xb2, -0x77,0x44,0x90,0x3c,0x6e,0xfc,0xfc,0x2b,0xa0,0xfa,0xc4,0xca, -0x42,0xbf,0xa2,0x20,0x24,0xbb,0x1a,0xcc,0xd5,0x42,0x71,0x13, -0xfe,0x34,0x9e,0xa4,0x98,0xf3,0xf0,0x90,0xf2,0x23,0xe4,0x16, -0x62,0x70,0xfc,0xdb,0x9f,0x7f,0x46,0xc1,0x15,0x53,0x1b,0x94, -0x0a,0xd2,0x52,0x4a,0xd2,0x2a,0x75,0xd1,0xaf,0xc9,0xbe,0x3e, -0xa9,0x48,0x9f,0xd8,0xc7,0x59,0xf9,0x7e,0x49,0xb0,0x63,0x4e, -0x7e,0x55,0xf9,0x9e,0x49,0xe2,0xae,0x3b,0x13,0x38,0xa2,0x00, -0x1f,0x8d,0x79,0xc2,0x02,0x4b,0x49,0xad,0xdf,0xfe,0x5c,0xe5, -0x2f,0xf2,0x77,0x9b,0xea,0x8d,0x72,0x75,0x81,0x02,0x03,0x46, -0xdd,0xa6,0xd0,0xb0,0xbf,0x1d,0xa3,0x29,0x63,0x0e,0x02,0x5f, -0x41,0xea,0x6f,0x7f,0x91,0xd4,0x87,0x40,0x0b,0x27,0xf5,0xd3, -0x98,0x00,0xd1,0xc5,0xd0,0x60,0x61,0x45,0x7b,0x11,0xca,0xdc, -0xb5,0x05,0xc6,0xf3,0x71,0x43,0x79,0x2b,0x99,0xed,0xec,0xc1, -0x02,0xe5,0x4e,0xf7,0x48,0xdd,0xd0,0x8d,0xa4,0x3c,0x57,0x9a, -0x97,0x06,0x3c,0x2e,0xbb,0xba,0x6d,0x01,0x09,0xf1,0x76,0xfb, -0x6a,0xdf,0xc5,0xb6,0x4b,0x9b,0x13,0x90,0xae,0x19,0x68,0xb7, -0x75,0x82,0x52,0x32,0x07,0xe0,0x6b,0x87,0xe4,0xf9,0x1c,0x3d, -0xdb,0x2e,0x45,0xf7,0x6d,0x9d,0xa8,0x4d,0xe2,0xfa,0x95,0x99, -0x2d,0xf0,0x6c,0x7c,0x6e,0x77,0x1b,0x79,0x8d,0xf5,0x39,0x72, -0xd9,0x9a,0x99,0xcc,0x64,0x83,0xed,0x7f,0xa3,0x75,0xd4,0x0a, -0x32,0x25,0xb9,0x4c,0xcd,0x56,0xe2,0x66,0x4a,0xba,0xd5,0x4d, -0x1e,0x52,0xd2,0x6a,0xb4,0xdc,0xf0,0x51,0xab,0xe9,0xb2,0x0a, -0x69,0x0b,0x79,0xda,0xf4,0x12,0xf7,0x3d,0xa9,0x3e,0x44,0x72, -0x36,0xad,0xf4,0xa9,0x73,0xae,0xac,0xce,0xa7,0x4d,0x12,0x0d, -0x7a,0xa9,0x34,0xbc,0xa7,0x60,0x36,0x1d,0xf4,0x75,0xb0,0xaf, -0xec,0x2e,0x73,0xd0,0x03,0x11,0x6d,0xd0,0xef,0x0c,0x7a,0xd5, -0x41,0xbf,0xd5,0xa0,0x67,0xc4,0x1d,0xf3,0xde,0xa2,0xc5,0xef, -0x8a,0x3b,0x46,0x9d,0xbf,0xd5,0xd4,0x11,0xa2,0xc7,0xdf,0x32, -0xef,0x35,0x0a,0x24,0x76,0x83,0xca,0x70,0x9d,0x9a,0x7c,0x6a, -0xb8,0x49,0x4d,0x05,0x6a,0x99,0x42,0x57,0x27,0xdd,0xdc,0xa6, -0x77,0xb8,0x0e,0x5e,0x0e,0x0c,0x37,0x44,0xd0,0x35,0x8c,0xeb, -0x1e,0x8d,0xeb,0x1e,0x8d,0xeb,0x35,0x4c,0x2f,0x1b,0x8a,0x4b, -0x28,0x0e,0x67,0x82,0x4d,0xd2,0xca,0x71,0xc6,0x6b,0x60,0x0d, -0xa9,0x7e,0xad,0x96,0xf6,0xf1,0x9d,0x9b,0xa3,0xc7,0xbd,0xab, -0xaa,0xbb,0xb3,0x6c,0x1a,0x8c,0x36,0x3a,0x29,0xdf,0x05,0x04, -0x7b,0x1f,0x81,0x60,0x96,0x0e,0x09,0x51,0x4f,0x70,0x0e,0x44, -0x02,0x10,0xea,0x95,0x25,0x3d,0xf3,0x58,0x2b,0xa4,0x02,0x21, -0x2e,0x31,0x9b,0x98,0x44,0x24,0x32,0x85,0xce,0x0f,0x58,0x96, -0xf4,0x34,0x64,0x2d,0x6f,0x58,0x38,0x04,0xb9,0x40,0x5a,0x44, -0xb5,0xb4,0x50,0xfd,0xa3,0x40,0xef,0x63,0x4c,0x17,0x7a,0xe6, -0x9e,0xc1,0x6a,0xb4,0xbc,0x9b,0x16,0x1c,0xe4,0xab,0x4d,0x8a, -0xcc,0xab,0x5c,0xc5,0x43,0x5d,0xa0,0x8f,0x27,0xea,0x20,0xf1, -0x7b,0x27,0xfb,0x57,0xba,0x60,0x2f,0xbd,0xc2,0xeb,0x7a,0x74, -0xfc,0x50,0xa9,0xfc,0x12,0xf7,0x21,0xf8,0x0b,0x3f,0xbd,0x81, -0xee,0x20,0x7e,0x3a,0xde,0x1c,0x1e,0x8e,0x33,0x34,0x25,0x4b, -0x62,0xd1,0x3c,0xc6,0x66,0x56,0x2a,0xef,0xa6,0xad,0x3e,0xe7, -0x0a,0x92,0x9f,0x40,0x72,0xb5,0xfa,0x6e,0x4a,0x6a,0x34,0xd2, -0x52,0x3c,0xfc,0x7a,0xe7,0x78,0x60,0x70,0x0a,0x0c,0xcc,0x35, -0x8e,0x61,0x34,0x3c,0xde,0xd0,0x7c,0x16,0x57,0xa3,0x81,0x13, -0x6f,0xf8,0x2f,0x0e,0xc4,0xb6,0x7b,0x9e,0x4e,0xf4,0x50,0x17, -0x28,0x61,0xbb,0x73,0xf6,0x8c,0x02,0x04,0x78,0x33,0xe4,0x08, -0x82,0x9d,0x6e,0x62,0xc0,0x3c,0x82,0x12,0x92,0xea,0x82,0x46, -0xc1,0xdf,0x1f,0xc1,0x52,0x3b,0x90,0xe0,0x67,0x67,0xbe,0x88, -0x6f,0x86,0xb3,0x70,0x43,0x8e,0xe3,0xb5,0x77,0x7b,0x7b,0x84, -0x3f,0x30,0x34,0x5a,0xc0,0xbc,0xbd,0xe3,0xa4,0x03,0x03,0x00, -0xe6,0x9f,0x36,0x2b,0x4b,0x01,0x4e,0x51,0xaf,0x62,0x00,0x55, -0x1e,0xc6,0xbf,0x7b,0x88,0xe8,0xf4,0x92,0x0f,0xd9,0xe4,0x7b, -0xba,0x41,0x6b,0xb5,0x71,0x8c,0x31,0xa6,0x6b,0xe8,0xaa,0x75, -0x72,0x8c,0xbe,0x7b,0x37,0xc7,0xe9,0x06,0x42,0x1b,0x08,0x6d, -0x82,0xc2,0x5c,0x2a,0x07,0x27,0xa7,0x4f,0x84,0x91,0x77,0x6a, -0xff,0x60,0xf4,0xf7,0xee,0xad,0x19,0xee,0xc0,0x02,0x75,0x13, -0xbc,0xcb,0x54,0x67,0xe3,0x45,0xb6,0x44,0xf5,0xbf,0xab,0x53, -0xdc,0xf8,0xaf,0x4f,0xd3,0xcd,0x29,0x1d,0xf9,0x15,0x28,0x09, -0xe9,0x3b,0x69,0xba,0xff,0x16,0xf8,0xd1,0xfa,0xf4,0x54,0x6d, -0x4e,0x4f,0x61,0x2f,0x7b,0x7a,0x0a,0xd2,0x3e,0xfc,0xeb,0x7b, -0x37,0xae,0x35,0xbe,0xad,0x8c,0x42,0x45,0x8e,0x20,0x97,0xd6, -0x9c,0x89,0xb5,0xca,0xa8,0xa3,0x6e,0x80,0x40,0xf7,0xbd,0x79, -0x75,0x90,0x41,0x12,0x3a,0x16,0xe4,0xaa,0xb9,0x09,0xe8,0x9b, -0xa7,0x3f,0x2a,0x63,0x83,0xae,0x30,0x15,0x35,0xef,0x37,0xff, -0x18,0x88,0x8d,0x78,0x64,0xb8,0x4a,0x73,0xdd,0x49,0x2e,0x71, -0x75,0x97,0x38,0xba,0x53,0x90,0xa6,0x7b,0x23,0x06,0x92,0xa6, -0x6b,0x24,0x6a,0xba,0x41,0xaa,0xc2,0x2e,0x4d,0x7a,0x09,0xe6, -0x97,0x5d,0xba,0x07,0x05,0x4c,0x62,0x8f,0xcd,0xae,0xb0,0x63, -0x42,0x4d,0x1f,0xc2,0x86,0x34,0x26,0x79,0xc4,0x9e,0xac,0xb1, -0x23,0xbb,0xaa,0x9f,0x12,0x32,0x6d,0x6f,0x95,0xea,0x44,0x57, -0xa7,0x30,0x7c,0xef,0x53,0x75,0x4f,0x3f,0x2a,0x8c,0x73,0x49, -0x24,0x75,0xd3,0x8d,0xaf,0xba,0x72,0x76,0xac,0x76,0x77,0xc9, -0xfb,0xfb,0x03,0x10,0x5f,0x3f,0xa4,0xa1,0xb6,0xd8,0x76,0xcd, -0x94,0x23,0xc2,0xa5,0x57,0x7b,0x94,0x7f,0x84,0x6d,0x91,0xbe, -0x8a,0x1a,0x0e,0x5f,0x4c,0x66,0x86,0x66,0xea,0xb1,0x7e,0x49, -0x94,0xcf,0xb2,0x7b,0x15,0x1f,0xfe,0xb9,0x9a,0x40,0x1a,0x83, -0xd4,0xc7,0x60,0x97,0x7a,0x8f,0x90,0xa3,0xb1,0x4b,0x9f,0xc7, -0x80,0x33,0x6d,0xfc,0x60,0x1d,0x1d,0x7f,0xdd,0x3c,0x41,0xb1, -0x12,0x7b,0x95,0xa3,0x53,0x51,0xfe,0x0a,0x95,0x6b,0x44,0xba, -0x4d,0x03,0x85,0x9d,0x76,0xc4,0xc5,0x6e,0x6f,0x25,0x43,0xec, -0xf6,0x7d,0xa0,0xd6,0x32,0xfe,0x17,0x68,0xe0,0x48,0xb5,0x9e, -0xb8,0xf2,0x8f,0x3d,0xab,0x28,0xd6,0x99,0x29,0xd0,0xea,0x31, -0x0a,0x7f,0xdb,0xed,0xdd,0xea,0x30,0x5a,0x84,0x9a,0xae,0x60, -0xe9,0x1c,0x4d,0x67,0x78,0x90,0x40,0x7e,0x5d,0x49,0x57,0xd0, -0x4a,0x82,0x36,0x09,0x45,0x42,0xbd,0x0f,0x6e,0x1c,0x43,0x45, -0x97,0x7a,0xf3,0xbf,0xba,0xac,0x3b,0xf9,0x68,0x8b,0x6d,0x83, -0xb8,0x98,0x42,0xcc,0xea,0x98,0xd4,0xda,0xe4,0xa4,0x23,0x69, -0x63,0x18,0x25,0x4b,0x3a,0xb0,0x4a,0x57,0xb1,0xd1,0xb7,0x5e, -0x3d,0xc1,0x18,0x27,0xe3,0x93,0x25,0x89,0x9b,0x24,0xdc,0x61, -0xc6,0xdd,0x2f,0x58,0x80,0xaa,0xb3,0x4c,0x24,0x61,0x4f,0x60, -0xc7,0x58,0x6f,0x1d,0x4d,0x7b,0x3c,0xd8,0xfb,0xd1,0x1a,0xa7, -0x91,0x35,0xad,0xbf,0xfa,0xd9,0xb5,0x28,0xe7,0xc1,0xe2,0x63, -0xbe,0x22,0x81,0x22,0x11,0xf6,0x4d,0x8f,0x0f,0x55,0x17,0x60, -0xe0,0xb0,0x44,0xf7,0x4c,0x84,0xad,0x26,0x56,0x02,0xac,0xa0, -0x01,0xd3,0xf1,0x20,0x9a,0x2f,0x66,0xa8,0x68,0xfe,0x74,0xa9, -0x2f,0x67,0x1a,0x69,0x83,0xb6,0xf1,0xbf,0xfd,0xb9,0x7e,0x04, -0x72,0xa0,0x56,0x6b,0xe1,0x6c,0x50,0xfc,0x9b,0xd5,0x92,0xfc, -0x8d,0xd9,0x82,0x6a,0x9e,0x5e,0x46,0x7c,0x0a,0x60,0x77,0x1f, -0xb0,0x0e,0xca,0x9d,0xb9,0x63,0x72,0xee,0x74,0xb7,0x99,0x33, -0x82,0x90,0x9e,0xa2,0x1c,0x0c,0x35,0x27,0x9f,0x32,0x12,0xa7, -0x28,0x3d,0x73,0x38,0x6e,0x9d,0xc0,0xdf,0x32,0x66,0xab,0xe3, -0x41,0xbb,0x04,0x12,0x0c,0x74,0x61,0xbe,0x28,0x6e,0x1f,0x9d, -0x38,0x3e,0x5d,0x8c,0x7a,0x06,0x41,0xdb,0x57,0x20,0xff,0xbe, -0xeb,0x2d,0x86,0xee,0x36,0xfe,0xb7,0xff,0x70,0x4e,0x79,0x9e, -0x55,0x9b,0xe5,0x69,0xd9,0x7b,0x34,0x5c,0xd7,0xd7,0x10,0x27, -0xbf,0xfd,0x87,0x7b,0x97,0x56,0x4e,0xa7,0xb1,0xfa,0xed,0x3f, -0x1a,0x55,0x1d,0x2d,0x77,0x1f,0xec,0xb7,0xa1,0x50,0xb5,0x65, -0x9a,0x46,0x1c,0x5f,0xf1,0x2e,0xee,0xe2,0x7a,0x53,0x3d,0x4b, -0x93,0x8a,0xa8,0x93,0x44,0x80,0x42,0x95,0xbf,0xa9,0x02,0x17, -0x3d,0x8b,0x8d,0x6e,0x4a,0x31,0xc3,0x70,0xc3,0x1a,0xa7,0x9b, -0x14,0x31,0xdc,0xb4,0xf3,0x2f,0xc7,0xd7,0x8a,0xd2,0x01,0xa0, -0x79,0x29,0x0e,0xad,0x8f,0xd6,0xe5,0x75,0x85,0x12,0xca,0x9c, -0x0c,0xd4,0x98,0xc6,0x75,0x44,0x29,0x46,0x92,0x4b,0xa5,0x79, -0x15,0x28,0x00,0xfc,0xf4,0x6e,0x6b,0x18,0x84,0x5b,0x50,0x44, -0xe6,0xae,0x0a,0xbe,0x9b,0xa4,0xa7,0x92,0x1b,0x85,0x2f,0x0a, -0xf1,0xd9,0xdc,0x46,0x1e,0xca,0xe2,0x31,0x80,0x3c,0xf7,0x6b, -0x07,0xa5,0xc3,0xb7,0x7e,0x45,0x35,0xe4,0x5e,0xf6,0x01,0xc4, -0x54,0xea,0x80,0x4a,0xb0,0x1e,0x15,0x16,0xd9,0xfb,0xb6,0x4e, -0x30,0x2a,0x78,0xe4,0x8a,0xe5,0xc9,0x41,0x47,0x56,0xdc,0x60, -0x4e,0xfb,0xb1,0x71,0xbc,0x66,0xeb,0x4a,0x61,0x7c,0xba,0x86, -0x51,0x7d,0x9c,0x4f,0x48,0x0a,0xb3,0x27,0x98,0x3d,0x17,0xfd, -0x53,0x03,0x77,0x67,0xf9,0xec,0x3f,0xe1,0x33,0x4c,0xb5,0xc9, -0x43,0xff,0xa9,0x10,0xfa,0x4f,0x49,0xba,0x29,0xd0,0x75,0xc3, -0x74,0x91,0x7e,0xc3,0x07,0x09,0xd4,0x58,0xd1,0x2d,0x27,0x39, -0xb1,0xf4,0x4d,0x49,0xad,0x55,0x09,0xfe,0xdb,0xf0,0x51,0x3c, -0xdf,0x28,0x16,0x3d,0x2f,0x60,0xa6,0xd9,0x07,0x01,0xdd,0xff, -0xa0,0xe9,0x96,0xf0,0xc5,0x40,0xd1,0x5b,0x85,0x3c,0x9c,0x17, -0x16,0x4e,0xe1,0x66,0x21,0x07,0x3a,0x97,0x4b,0xaf,0xb1,0x01, -0xe0,0x3f,0x95,0xf8,0x59,0xaa,0xd3,0xbe,0x34,0x47,0x2b,0xb9, -0x49,0x7d,0x58,0x43,0x9b,0xd6,0xac,0x3e,0x09,0x92,0x0f,0x57, -0xba,0x4b,0xf0,0x72,0x38,0xaf,0xb8,0xe1,0x94,0x6d,0x7b,0x6f, -0xb1,0x4c,0x80,0x04,0xfa,0xce,0xd2,0xaa,0x5d,0xc2,0x57,0x71, -0xa1,0x82,0x9e,0x90,0xcb,0x57,0x3e,0xf0,0x6b,0x15,0xcc,0xfe, -0xf7,0xa1,0xcf,0xa9,0xee,0xe3,0xad,0xd8,0x9b,0x75,0x51,0x81, -0x95,0x5b,0x08,0xaf,0xdf,0xf3,0x58,0x63,0xbb,0xfa,0x45,0x03, -0xed,0xdd,0x4a,0x25,0xb3,0x22,0x59,0x5a,0x73,0xbe,0xad,0x15, -0x5c,0xa4,0xa2,0xdd,0xfc,0x1d,0xdc,0x7e,0x8b,0xe1,0xe4,0xcb, -0x86,0x2a,0x55,0x38,0xae,0x52,0xea,0xd9,0x4f,0x27,0xf6,0x41, -0xe3,0x41,0xa2,0x92,0x07,0x98,0x11,0x56,0x93,0x7b,0xe7,0xb5, -0x59,0xaf,0x4b,0x85,0x68,0x3d,0xf3,0xec,0x69,0xb8,0x33,0x8b, -0x6b,0x93,0x22,0xff,0xdc,0x1b,0x22,0x37,0x2a,0x67,0xd1,0xe2, -0x4f,0x79,0xf1,0xc6,0xad,0xe4,0x0e,0x23,0x11,0x66,0x91,0x19, -0xae,0x53,0x9c,0xb4,0xf1,0x18,0x10,0x9f,0xb6,0x9b,0xe7,0xf0, -0x64,0x0a,0x65,0x5d,0x1e,0xae,0x2b,0xc3,0x0d,0x9e,0x72,0xe5, -0x1e,0xa7,0x27,0x95,0xf4,0x9a,0xde,0x8f,0xa3,0x93,0x9d,0xa2, -0xc7,0xe3,0xd7,0xf4,0x78,0x7c,0x53,0x90,0x8a,0x2f,0xc7,0xaf, -0xd5,0x6e,0xca,0xe8,0xd5,0xc3,0xb1,0xb3,0xd0,0x2e,0xc8,0x7d, -0xd7,0xd2,0xf1,0xec,0x0e,0xe3,0x07,0x7b,0x6b,0xbe,0x93,0xb8, -0x9e,0x09,0x84,0x82,0xea,0x76,0x55,0xb2,0x17,0xf0,0x7d,0x96, -0xdf,0xff,0xde,0x5e,0xbb,0x76,0x96,0xf9,0x02,0x7b,0x05,0x1a, -0x78,0x91,0xcd,0x02,0x0d,0xba,0xc8,0x6e,0xc1,0x51,0xf9,0x43, -0x18,0x62,0x87,0x3c,0xf1,0xaf,0x64,0x8c,0x40,0xe0,0x88,0xb7, -0x45,0x99,0xff,0x11,0xa9,0xe3,0xd9,0x0c,0xb6,0x76,0x6b,0x76, -0x46,0x87,0x4f,0x71,0x0b,0x17,0x67,0x49,0xac,0xa1,0x4f,0xb4, -0xd7,0x33,0x39,0x8e,0x90,0xb8,0xde,0x62,0x40,0xe6,0x6f,0x9c, -0x79,0x5c,0x81,0xf8,0xfc,0x7f,0xee,0x58,0xad,0x8b,0x40,0x7e, -0xe0,0x42,0xad,0x41,0x60,0xeb,0x05,0xc4,0x3f,0xbc,0x46,0x6b, -0x98,0x64,0xf8,0xee,0x7b,0xa0,0x4f,0xe4,0x2c,0xd3,0x72,0x5f, -0x06,0xcb,0xf1,0xff,0x07,0xd7,0x60,0x69,0xfc,0x69,0xb8,0xd4, -0xca,0xa3,0x8f,0xbb,0x16,0x48,0xc8,0xdd,0xbb,0x9c,0xa3,0x92, -0x10,0x6f,0x06,0x7d,0x33,0xf8,0x9c,0xc4,0x45,0x8b,0x59,0x70, -0x9d,0x4a,0xc1,0x3c,0xab,0xb8,0x7b,0xd8,0xc2,0x23,0xe3,0x35, -0x2b,0x21,0xeb,0x7a,0x72,0xa7,0xc7,0x0e,0xa5,0x0a,0x5f,0x8f, -0xf2,0x38,0x72,0xf6,0x7e,0x79,0xf5,0x77,0xf3,0x1a,0x06,0x76, -0xb8,0x1a,0x51,0xfd,0x20,0x54,0xaa,0x7d,0x21,0x6c,0x4a,0x97, -0x93,0xf4,0x4c,0xa0,0xa1,0xfa,0xa4,0xbb,0x06,0xf8,0xe1,0x9d, -0x22,0xce,0x35,0x74,0xf3,0x22,0xba,0xde,0xbd,0x46,0x3a,0xa0, -0xb3,0x44,0xfc,0x45,0x3b,0x5d,0xf8,0xdb,0xec,0xa2,0xfb,0x90, -0x2f,0x46,0xf3,0xe5,0x85,0xe0,0x1d,0x12,0x03,0xe1,0x17,0x0e, -0x84,0xa0,0xfd,0xf3,0xb0,0xed,0xc1,0x81,0xb3,0xfb,0x32,0xd5, -0x7b,0x8c,0xea,0x83,0xc1,0x6c,0x4c,0x46,0x93,0x10,0x3c,0xd0, -0xf3,0x41,0x7a,0xc4,0x74,0xa9,0xeb,0x59,0x59,0x89,0x65,0xb3, -0x25,0x33,0x1f,0xd0,0xa2,0x91,0x12,0xbd,0x1a,0x29,0xd1,0x0c, -0x48,0xb1,0xbb,0x43,0x8b,0xde,0xc9,0xde,0xa7,0x17,0x78,0x7e, -0x27,0x0b,0x28,0x50,0x1f,0x56,0x86,0x35,0xe5,0xe9,0x9d,0x6b, -0xa8,0x1e,0x5b,0x96,0x8a,0xa1,0xc5,0x19,0x06,0xed,0x42,0xc5, -0xba,0x19,0x0b,0x83,0x81,0x86,0xfb,0xcf,0x3b,0x53,0x35,0x9c, -0xe1,0x1d,0x2f,0x7a,0xf8,0x07,0x4f,0x2c,0xef,0x98,0x4d,0xbc, -0xe7,0x7b,0x05,0x70,0xcc,0x97,0x95,0x98,0x75,0xae,0x6d,0x6e, -0xc0,0xef,0xe9,0x0b,0xbc,0xbb,0x27,0xdd,0xe9,0x5e,0xa2,0xfa, -0xa8,0x08,0xa4,0x86,0x6e,0x17,0x18,0x69,0x24,0xd1,0xe2,0x48, -0x82,0xf2,0xc8,0xb0,0x99,0x3a,0x62,0x08,0x9d,0xf7,0x35,0x8f, -0x1f,0x96,0xdf,0xff,0xb5,0x79,0x78,0x48,0xe5,0xab,0x55,0x7d, -0x75,0xd7,0x6b,0x54,0x10,0x34,0x70,0x53,0x05,0xe1,0x03,0x3b, -0x55,0xa0,0x12,0xd7,0xbe,0x92,0xf8,0x37,0xab,0x0c,0xca,0x03, -0xd4,0xd5,0x6c,0x8a,0x52,0x04,0xd9,0xc6,0xab,0xa7,0x97,0xa8, -0x3b,0xde,0xb4,0x8a,0x12,0x56,0x63,0x02,0x75,0x82,0xe8,0x61, -0x01,0xb4,0xa8,0x82,0x1a,0x27,0x75,0xf1,0xed,0x62,0x1d,0xaa, -0x79,0x93,0x44,0x13,0x81,0x03,0xbf,0x37,0x65,0x12,0x68,0xf2, -0x24,0x30,0x84,0x88,0x75,0x93,0x9b,0xd7,0x4c,0x37,0x4d,0x6a, -0xdf,0x75,0x8a,0x2d,0x83,0x68,0x6c,0xe6,0xa6,0xd9,0x8e,0x86, -0xd7,0xe5,0xe1,0x75,0x1d,0x5a,0x09,0x6d,0xbc,0xbd,0x35,0x48, -0xb0,0x34,0xd6,0x64,0x3a,0x34,0x63,0xc8,0x50,0x85,0x3d,0xe4, -0x71,0xed,0x08,0x6f,0x19,0xb3,0x6f,0xc6,0xd3,0x2f,0xc6,0x19, -0x60,0x33,0x18,0x1d,0xf7,0x1a,0xe5,0x5e,0x52,0xe9,0x37,0xca, -0xfd,0xa4,0x32,0x68,0x94,0x07,0x49,0x4c,0xb6,0xdc,0xef,0xec, -0xa0,0x26,0x20,0x8a,0x14,0x50,0x3d,0xa0,0x05,0x7a,0x36,0xba, -0x54,0x03,0xbf,0x8f,0x82,0x69,0x09,0x0b,0x58,0x96,0xd1,0x90, -0x2d,0x18,0xbc,0xbe,0xbd,0xb3,0xdf,0xe5,0xa9,0x33,0x34,0x36, -0x85,0x45,0xd8,0x6f,0x8a,0x55,0x72,0x39,0x6a,0xb8,0x2a,0x28, -0x66,0xfc,0xa6,0xc9,0xe3,0x90,0x27,0x6b,0xc6,0x7a,0xf1,0xbd, -0x4e,0x1e,0x23,0x03,0x2a,0x42,0x14,0x4e,0xcb,0xa7,0x74,0x81, -0x9d,0x3c,0x36,0xed,0x12,0x17,0x67,0xc4,0x40,0x90,0x85,0xfc, -0x6a,0x84,0xec,0x2f,0x6b,0xc7,0xd2,0xb8,0x79,0xd5,0x4c,0xcd, -0x5e,0x36,0x59,0xb2,0xd8,0xb3,0xcf,0xfc,0x9e,0x59,0xe7,0x15, -0xe5,0xb0,0xeb,0xa9,0x6c,0x6b,0x79,0x06,0x09,0x57,0xdd,0xc2, -0x73,0x61,0xcd,0x83,0x27,0x6b,0xd7,0xc3,0xdf,0xc6,0x09,0xe0, -0xa9,0x70,0xce,0x87,0xb7,0x3b,0x06,0x4d,0x23,0x52,0x71,0x30, -0x57,0xd4,0xb2,0x34,0x5c,0xe1,0x0d,0x4c,0x93,0xb3,0x50,0x76, -0xf6,0x19,0xc8,0xb5,0xdd,0xba,0x71,0x3b,0x78,0xab,0x0c,0x98, -0x02,0x19,0xd7,0x9c,0x46,0x5b,0x4a,0xf9,0x3a,0x49,0xf9,0xf3, -0xea,0xbc,0xae,0xa7,0xe4,0xd9,0xc6,0x05,0xd6,0x7c,0xf7,0x9d, -0x77,0x3b,0xa2,0x8c,0x83,0x07,0x69,0xee,0x78,0xc2,0x95,0x9e, -0x0e,0x5e,0xb2,0x29,0x09,0x8a,0x23,0x3d,0x0f,0xdb,0x64,0x8e, -0x4b,0xba,0x1e,0x6b,0x9f,0x98,0x9c,0x6f,0x2b,0xef,0xff,0xba, -0x56,0xef,0xff,0xba,0xa9,0x9a,0x8c,0xae,0x2d,0x52,0x39,0xed, -0xbf,0xb3,0x6a,0x7d,0x2d,0x60,0x50,0xa8,0x02,0xd8,0xb8,0xfe, -0x56,0x45,0x2e,0x68,0x88,0x10,0x25,0xd9,0xc3,0xc3,0x13,0x17, -0x59,0xcd,0x43,0x2e,0xb2,0x9a,0x95,0xbc,0xf9,0x1f,0x9f,0x93, -0xde,0x78,0x35,0xa7,0xc1,0x65,0xc9,0xc2,0x8b,0xa5,0x10,0x29, -0xe7,0xff,0x15,0x84,0x84,0xf7,0x7f,0xfd,0xed,0x2f,0xf0,0xe7, -0xfd,0x7f,0x02,0x16,0xd2,0xa9,0x7c,0x81,0x37,0xc2,0x7b,0xaa, -0xbc,0x00,0x46,0xad,0x85,0x86,0xc0,0xee,0x95,0xb3,0xb0,0x09, -0x57,0x68,0x52,0xba,0xa9,0x48,0x0c,0x19,0x72,0x85,0xd6,0xf7, -0x26,0xe3,0x21,0xf6,0x81,0xa7,0x14,0x62,0x63,0x6f,0x3c,0x31, -0x17,0xe6,0x4a,0x61,0x52,0xca,0x81,0x97,0xd7,0xc2,0xf2,0xec, -0x3c,0xc4,0xf6,0x3a,0xef,0x22,0x38,0x28,0xc6,0x17,0x94,0xd7, -0x7e,0x25,0xfd,0x27,0x00,0x4c,0xe2,0x07,0x8d,0xe8,0x40,0x4a, -0x3e,0x59,0xfb,0xac,0xb1,0x16,0xb6,0x40,0xe3,0xcf,0xc9,0x23, -0x98,0xc5,0xd3,0x87,0x9f,0xc2,0xdc,0x9e,0x36,0x1f,0xe1,0xf4, -0x4e,0xb7,0x39,0xf0,0x87,0x94,0x3f,0x1b,0x44,0x64,0x0e,0xfd, -0x27,0xfc,0xe0,0x44,0x89,0xd7,0xbd,0x3b,0x5e,0xf5,0x28,0x74, -0x0a,0xa3,0x33,0x68,0x77,0x5f,0x8a,0x73,0xe1,0x03,0x63,0xba, -0xa2,0xb4,0xd7,0x7b,0xae,0xbb,0x25,0x67,0x68,0xe8,0xc9,0xcf, -0xb4,0x6d,0xb6,0x5a,0xce,0x57,0xcb,0x3b,0x89,0xae,0xd7,0x9a, -0xfc,0x15,0xd4,0x4b,0x1e,0x29,0x91,0xa0,0x1f,0x05,0x94,0x37, -0x9b,0x12,0xc1,0x5e,0x57,0x48,0x26,0x39,0xbc,0x3a,0x1a,0x66, -0xdf,0xe3,0xa0,0x6b,0xdb,0x77,0xaf,0xd5,0x43,0xd3,0x90,0x95, -0x85,0x4e,0x3b,0x91,0x2d,0x7f,0xaa,0x76,0x10,0x36,0x6e,0xe5, -0x5f,0xd6,0x38,0xe5,0x2a,0xa7,0xb1,0x37,0x21,0xb8,0xac,0xca, -0x26,0x1d,0x28,0x5f,0x0e,0x6b,0xe9,0x86,0x7b,0xa1,0xed,0xe4, -0x3f,0xb5,0xfd,0x7c,0xda,0xc9,0x2b,0x5e,0x9e,0x8a,0x8e,0xff, -0x29,0xeb,0xf8,0xab,0x53,0x79,0x3b,0x70,0x2a,0x6f,0x07,0x5a, -0x2e,0x73,0xe4,0x51,0x65,0x28,0x1e,0xae,0xe4,0xb4,0xe0,0x7e, -0x68,0xbe,0x05,0x62,0x28,0x99,0x5a,0xe2,0xd6,0x5b,0x0f,0x0c, -0x2d,0x10,0xa4,0xbd,0x7a,0x2f,0x50,0x6b,0x00,0x45,0xb2,0x18, -0xfe,0x26,0x5d,0x0b,0x95,0x94,0x46,0x7c,0x62,0xf2,0x94,0x73, -0x2f,0xb0,0x30,0xac,0x10,0xec,0xef,0x8e,0x1e,0xbb,0x02,0x0a, -0x0e,0x3a,0xac,0x25,0x8c,0xb6,0x95,0xbe,0xff,0x9b,0x3b,0xb7, -0xb2,0x05,0x78,0x33,0xab,0xba,0xc8,0xc8,0x9c,0x79,0x3f,0x64, -0xfe,0xba,0x03,0x1d,0x1a,0xf7,0x45,0x08,0xd1,0x4c,0x70,0xaa, -0x0d,0x30,0x34,0x3b,0xa7,0xb0,0x19,0x0d,0xf2,0xb4,0x1a,0x2e, -0xda,0x7f,0xf5,0x11,0x47,0xc0,0x7e,0xf8,0xfd,0x7f,0xfa,0x4d, -0xb1,0x3e,0x74,0x6c,0xa3,0xc2,0xb1,0x8a,0x7e,0xdf,0x44,0x66, -0x2b,0xc5,0x6a,0xe7,0x52,0x9d,0x9a,0x45,0x58,0x1c,0x4d,0xa1, -0x70,0x65,0x5d,0x84,0xb9,0x53,0xb3,0xbe,0x0a,0x95,0x18,0x73, -0xb7,0xce,0xbf,0xba,0x41,0xfb,0x44,0x0b,0x3d,0xcb,0x7c,0xb0, -0x64,0x74,0xb7,0x20,0x24,0x1b,0x65,0x4f,0x14,0x62,0xb2,0x5a, -0xf6,0xd9,0x25,0x0c,0x85,0x08,0x8f,0x7e,0x5d,0x91,0x02,0x60, -0x6f,0x7a,0x4e,0x26,0xcf,0xbc,0x87,0x78,0x72,0x80,0x91,0x2b, -0xa4,0xc7,0xeb,0x4b,0xb3,0x02,0x07,0xa5,0x8e,0x7f,0xfb,0x73, -0x07,0x5f,0x04,0xfc,0x1f,0x7c,0xd5,0x86,0xea,0xea,0xef,0xff, -0x56,0x81,0xc0,0x0e,0x70,0xbb,0x16,0xf2,0x1b,0xb3,0x4a,0x23, -0x04,0x88,0xbe,0xbd,0xc5,0xf8,0x4e,0x20,0x15,0x04,0x50,0xde, -0xff,0x8d,0xe0,0x98,0xb3,0x5d,0x1d,0xff,0xdb,0x5f,0xde,0xff, -0x67,0x64,0x81,0x9b,0x49,0x34,0x2c,0xd7,0xca,0xd7,0xb4,0x03, -0x44,0x6b,0x07,0x39,0x72,0x0d,0x94,0xab,0xee,0x97,0x41,0x4d, -0x39,0x33,0x6d,0xfe,0x2b,0x48,0xa0,0x59,0x4a,0x14,0xb9,0x93, -0xa0,0x3b,0x29,0x6a,0x6a,0xa2,0x8d,0x80,0x44,0xa6,0xfb,0xb1, -0x32,0xfb,0x21,0x89,0xd7,0x43,0x61,0x67,0xa9,0x2a,0x93,0x7b, -0xb1,0xab,0xf5,0x3b,0xa8,0xe7,0x1c,0xf6,0x05,0x6c,0x54,0x68, -0x1e,0x17,0x0d,0x29,0x7b,0xef,0x42,0xd9,0xc4,0xc1,0xc6,0x7d, -0x27,0xad,0x23,0xaf,0x43,0xaf,0x16,0xd7,0x98,0x80,0x55,0x57, -0xd6,0xf4,0x4a,0x19,0xbe,0x0a,0xf4,0x18,0x36,0x9c,0xeb,0xfd, -0x7f,0x56,0xdf,0x72,0xae,0xf7,0xff,0xa9,0xd6,0xba,0x64,0xf5, -0x5a,0x97,0xb4,0xf6,0xf0,0xdf,0xea,0x02,0x95,0x8d,0x2e,0x20, -0x3a,0x1b,0x52,0xc8,0xc1,0xf8,0xaf,0xe2,0x72,0xc3,0x46,0x13, -0xce,0x7f,0x15,0xd3,0x21,0x34,0x91,0x3a,0xb9,0x51,0x8e,0x15, -0x90,0x6e,0x6e,0x88,0xde,0xd5,0x3f,0x45,0x6f,0x55,0xff,0x25, -0x54,0x04,0xfa,0x98,0xe6,0xee,0xa3,0x62,0xe5,0x3a,0x4f,0xc5, -0xca,0xdb,0x9d,0x54,0x04,0x12,0x9b,0xce,0x41,0xfd,0x90,0x9d, -0xfc,0xb4,0xf3,0x95,0x89,0x59,0x04,0xa4,0xf1,0x45,0x6f,0x5c, -0x16,0xb6,0x51,0x12,0x97,0xdb,0xcb,0xde,0xa5,0xe3,0x8d,0xee, -0xbf,0x9c,0x37,0xd5,0xa6,0xd6,0xb6,0x78,0xca,0x86,0xe2,0x1d, -0x02,0xe2,0x3f,0x8b,0x21,0xa1,0x2b,0x1a,0x2c,0x14,0xbd,0x5a, -0x87,0x0a,0x76,0x24,0x2f,0x67,0xf8,0x54,0xd7,0x79,0x0e,0xb2, -0x9c,0x1d,0x63,0x91,0x16,0xfe,0x39,0x86,0x54,0x90,0x75,0xf1, -0xb3,0xe2,0xe0,0xf0,0xdb,0xff,0xc1,0x57,0x9a,0x54,0xa7,0xdc, -0xfc,0xba,0x69,0x58,0x97,0x3c,0x93,0xa9,0x3d,0x2a,0x93,0xf7, -0xb2,0xb6,0x67,0x69,0x47,0x2d,0x09,0xeb,0xb6,0x57,0x2b,0x54, -0xd5,0x82,0xaa,0x97,0xed,0x65,0x35,0xa5,0x32,0xc5,0x06,0xf3, -0x86,0xe1,0x5b,0xa7,0x13,0x68,0x43,0x35,0x5b,0x58,0x45,0xa6, -0x65,0x6c,0xc3,0x48,0xf7,0x65,0xdc,0x8d,0x03,0xeb,0xa1,0x3b, -0xfa,0xd8,0xd0,0x44,0xf6,0xa8,0x7c,0x4c,0x97,0x3b,0x67,0xe7, -0xd4,0x36,0xbd,0xd2,0x48,0x21,0x77,0x98,0xc1,0xb1,0x1f,0xc3, -0x7b,0xc2,0x1e,0xc9,0xc9,0x43,0x31,0x81,0xce,0xfa,0xfe,0x6d, -0x6d,0xec,0x89,0xfc,0x8e,0x3c,0x69,0x74,0xaa,0x94,0xab,0x45, -0x7f,0xe3,0x4a,0x93,0xf1,0xff,0xfe,0xeb,0xea,0xfb,0xff,0x8a, -0x7f,0x17,0x99,0x60,0x7e,0xef,0x7f,0xbe,0x00,0xb6,0x1d,0xac, -0x26,0xa3,0x1f,0xa3,0x0d,0x6a,0x87,0xaa,0xa1,0xdc,0x94,0xa2, -0xf3,0xbb,0xda,0x02,0x9d,0xb0,0x72,0x02,0x40,0xc2,0x34,0x6d, -0x9c,0x6a,0x93,0xe4,0x59,0xd1,0xba,0xba,0xda,0xd4,0x2e,0x7b, -0x73,0xbb,0xc1,0x33,0xdb,0x3b,0x96,0x3e,0x0b,0x28,0x68,0xd0, -0xf8,0x09,0xcf,0xea,0xd6,0x80,0xc6,0x5a,0xbb,0xbe,0x32,0x68, -0x50,0x02,0xa1,0xb1,0x36,0x68,0xac,0x0b,0xd0,0x70,0x36,0x93, -0x3e,0x1a,0xeb,0xbd,0x68,0x64,0xb3,0xd5,0x62,0x30,0x72,0x3d, -0x87,0xd5,0x38,0xca,0xb7,0xf5,0x0d,0x02,0x18,0xfa,0x3c,0x74, -0x72,0x71,0x54,0x5e,0xec,0x70,0x86,0xbc,0xf7,0x9e,0x8b,0x46, -0xfb,0xc6,0xb1,0x56,0xb1,0x69,0xc0,0x50,0xdf,0x38,0x3a,0x73, -0x18,0x31,0xd8,0x38,0x26,0x30,0x36,0xb8,0x41,0xdc,0x38,0x56, -0x36,0x30,0xe2,0x2d,0x5a,0x5d,0xdf,0x34,0x2c,0xeb,0xae,0xa1, -0xd8,0xdb,0x8d,0x13,0x89,0x39,0x29,0x72,0x9d,0x40,0x64,0xe2, -0xe4,0xc4,0xe2,0x1b,0x27,0x92,0x72,0x26,0xe8,0xc4,0x5d,0x78, -0x85,0xe6,0x3a,0xe7,0x6e,0xfc,0xe8,0xf4,0xa2,0x07,0x13,0x37, -0xd7,0x8d,0xcf,0xc2,0x2a,0x58,0x0b,0x96,0x77,0x93,0xe8,0x34, -0x1b,0xb7,0x9e,0xb0,0x3b,0xaf,0x5b,0x83,0x4f,0xb1,0xff,0xfc, -0xce,0xba,0x6f,0x93,0x51,0xf2,0xb9,0x6d,0xd8,0xb2,0x9c,0x0e, -0xe3,0xf2,0x5b,0xf5,0xd4,0x46,0x0d,0xab,0x4b,0x8c,0x59,0xa7, -0x4f,0xcb,0xd0,0xe4,0xca,0xe7,0xf0,0x17,0x08,0x89,0xa1,0x0d, -0x85,0x80,0x69,0xaf,0x21,0x94,0x51,0x08,0xa8,0x54,0x34,0xcb, -0xab,0x75,0xec,0x0a,0xee,0x4e,0xd2,0xb5,0xb2,0xad,0x44,0x1d, -0xc0,0x4d,0x79,0x13,0xc7,0x9e,0x50,0x9f,0x7f,0x39,0x7d,0xb2, -0x6e,0x78,0xe7,0x8d,0x0d,0x2f,0xbf,0x66,0x48,0xa7,0xcd,0xe8, -0x00,0x9a,0x8f,0x78,0x87,0x2e,0x29,0x72,0x3c,0xc3,0x3b,0x93, -0x3b,0x54,0x50,0x9c,0x15,0x33,0x67,0xd8,0xcb,0x77,0xe6,0x50, -0xb0,0xa0,0x2e,0xed,0x81,0x7f,0xe4,0xdb,0xf9,0x8f,0xe9,0x71, -0x2a,0xad,0xee,0xff,0x7f,0x6d,0x7b,0x88,0x27,0xc4,0xb6,0x87, -0xdd,0xab,0x15,0x4c,0xe6,0xc5,0x7e,0x4b,0x96,0xa1,0xbd,0xff, -0x25,0x90,0x72,0x59,0xd1,0xf1,0x62,0x30,0x4c,0x54,0x3c,0x79, -0xdd,0x2e,0x4b,0x7d,0x90,0x4f,0x3b,0xa8,0xe2,0x5c,0x15,0xd7, -0x5a,0x8b,0xc9,0xa5,0xee,0xb2,0x22,0xa5,0xef,0xad,0x24,0x87, -0xa4,0xf9,0x2d,0xd8,0xed,0x26,0x21,0xf5,0x37,0x26,0x79,0x5d, -0xd6,0x02,0xfa,0x36,0x02,0x2b,0x6a,0x05,0x35,0x99,0xbe,0xdb, -0x06,0xc9,0x05,0x6a,0x0f,0x45,0xe5,0x8b,0x0b,0xe9,0x7b,0xeb, -0xbc,0x77,0xb6,0xeb,0xf1,0xe5,0x6a,0x79,0x01,0xab,0x1b,0x1d, -0x3e,0x28,0x5e,0x14,0x2c,0x2b,0xd9,0x74,0x5f,0xf8,0x72,0x7b, -0xdd,0xd8,0x68,0xf3,0x5b,0xf7,0x36,0x25,0x90,0x11,0xe5,0xe6, -0xbe,0x31,0x1e,0xbb,0xde,0x96,0x0b,0xbc,0x3d,0xbc,0xf5,0x74, -0x8a,0xbb,0x46,0x13,0x5c,0xa3,0xb0,0x4f,0x8b,0x37,0xcd,0x8f, -0x4c,0x35,0x48,0xa9,0x35,0xd1,0x6f,0xff,0x41,0xec,0x32,0xb0, -0x9d,0x33,0x20,0x84,0x07,0x16,0xdf,0x41,0x5c,0xa4,0xfd,0x8b, -0xdc,0x35,0x50,0xbf,0xfd,0x07,0x22,0x3b,0xd0,0xe6,0xa4,0x08, -0xbd,0xff,0x38,0x3c,0x24,0x49,0x71,0x50,0x07,0xe0,0x20,0xd2, -0x19,0x1c,0x8b,0xb5,0xad,0x9f,0xcd,0xa6,0xe4,0xee,0x71,0xf2, -0x61,0xda,0xd6,0xde,0x55,0x1b,0x24,0x6a,0x83,0x5c,0x78,0xe1, -0xd6,0x68,0x7d,0xc9,0xb3,0x26,0x3a,0xc2,0x5c,0x12,0xc1,0xa7, -0x71,0x91,0x12,0xf6,0xb4,0x8c,0xb4,0xfd,0xd2,0xd3,0xc0,0xa6, -0x38,0x23,0xcf,0xe7,0x2d,0x09,0x3a,0x34,0x0e,0x8d,0xd7,0xa0, -0x15,0x26,0x34,0x14,0x58,0xf9,0xed,0x2f,0x75,0xbc,0xf1,0x99, -0x92,0x45,0x29,0x34,0xff,0x96,0x74,0x3c,0x45,0x6e,0xbe,0x1b, -0x42,0x37,0x98,0x5c,0x45,0xdd,0x37,0xd2,0x54,0x37,0xc9,0x4b, -0x8a,0xa8,0x2f,0xd9,0x12,0x93,0xfa,0x52,0x4f,0x00,0x5e,0xeb, -0x50,0x31,0xdc,0xd1,0xeb,0x9e,0x82,0x1c,0x7a,0x3f,0xd5,0xee, -0x2f,0xab,0x1b,0xe8,0x41,0x1c,0x00,0xd9,0x39,0x10,0x43,0x5b, -0x7a,0xd2,0x8c,0xe2,0xaa,0x71,0xef,0x57,0x00,0xd7,0x8b,0x1d, -0x12,0xc0,0xa0,0xf6,0x25,0xf4,0xbf,0x4a,0xea,0xd3,0x58,0xf7, -0x8d,0x55,0xff,0x36,0x9b,0xe8,0xcb,0xd1,0x62,0x80,0x37,0x25, -0x3b,0x15,0xf1,0xc7,0x3c,0xe3,0x2f,0xef,0xcd,0x1c,0x5f,0x55, -0xed,0x2e,0x2f,0xdf,0xd5,0x5f,0x7d,0x50,0x57,0xef,0xee,0x3f, -0xe9,0xb7,0x6a,0xd0,0x6f,0xf8,0x53,0xa5,0xa2,0x5f,0x49,0x4f, -0xd5,0xa7,0x15,0x9c,0x37,0xc3,0xf7,0xcc,0x53,0x7e,0x0e,0x5e, -0x78,0x1a,0xd3,0xba,0x6f,0xf7,0x7d,0x75,0x97,0x5e,0xfe,0x57, -0xd5,0xfb,0x74,0xad,0xd3,0x2d,0xb9,0x6e,0xd0,0xfd,0x93,0x3f, -0x21,0x32,0x2c,0x5a,0xc0,0xf9,0x05,0x5e,0xe2,0x34,0xa0,0xef, -0x0b,0x6e,0xe1,0xf4,0x8b,0xab,0xa7,0x2b,0xd8,0x7f,0x15,0x1c, -0xdd,0x9b,0x9c,0x8a,0x0f,0x81,0x2f,0xf9,0x30,0x58,0xd9,0xb3, -0xdc,0x4b,0x7b,0xae,0xeb,0xde,0x2d,0x5c,0x3a,0x27,0xdc,0xba, -0x0b,0x2e,0xc3,0x93,0x64,0xae,0xff,0x8a,0x27,0x64,0x39,0x22, -0xbc,0x74,0xce,0x07,0x75,0xc1,0x2b,0x34,0x1d,0xda,0xd1,0x78, -0x76,0x5c,0xd8,0xf4,0xf8,0x2a,0x6e,0x5d,0xb6,0xae,0xb6,0xea, -0xb2,0xe8,0x88,0x59,0xd7,0x61,0x92,0xfe,0x19,0xf5,0xec,0x38, -0xbd,0xe7,0x9a,0x6c,0x62,0x51,0x55,0xf8,0x48,0x12,0xab,0x61, -0xbf,0x17,0x52,0x95,0xbe,0x8d,0xd0,0xae,0x84,0x7f,0xfb,0x73, -0x99,0x17,0x29,0x9c,0xf3,0x0c,0xe6,0x51,0xdc,0x76,0x10,0x3a, -0x39,0xa1,0xab,0xc1,0xb7,0x64,0x14,0xa8,0xfa,0xb6,0xab,0x28, -0x5c,0xe1,0x70,0xe5,0x6d,0x17,0x77,0x79,0xf4,0x2c,0x53,0x57, -0x82,0xa6,0xad,0xf8,0x4a,0xcc,0xb4,0xd7,0x6f,0x0b,0xb7,0xb1, -0xf8,0x8e,0x1c,0xa5,0xe3,0xd1,0x37,0x01,0x4b,0x3a,0x82,0xaf, -0xac,0xae,0xde,0xca,0xe9,0x8d,0x10,0xe4,0xd7,0x28,0xa9,0x7c, -0x0e,0x23,0x35,0xa9,0x7e,0x4e,0xef,0x66,0x42,0x09,0x8c,0x38, -0xf9,0x2f,0xb2,0xa0,0xc9,0x0a,0x1e,0xf0,0x73,0x76,0x75,0x7e, -0x8a,0xc2,0x83,0x73,0x36,0xed,0x4c,0x4b,0x94,0x42,0xfe,0xe3, -0x6d,0x9c,0x3c,0x84,0x73,0x1e,0x15,0xdb,0xd7,0x09,0xee,0xfe, -0x26,0xbc,0x1b,0x06,0xd9,0x08,0x68,0x42,0x87,0x1b,0x40,0x12, -0xd9,0xbc,0x87,0x76,0xce,0xc7,0xf2,0x25,0xce,0x6d,0xb1,0x6a, -0xfd,0x0e,0xff,0x0c,0xb7,0x7d,0xa7,0x84,0x06,0x14,0x5e,0xc7, -0xea,0x6c,0xe3,0x46,0x6c,0xac,0x6f,0xdb,0xd1,0x19,0xa0,0x2d, -0x6e,0xf2,0xe9,0x80,0x7c,0x48,0x80,0xf0,0x5d,0xff,0x38,0xee, -0x70,0x25,0xf2,0x76,0xb7,0x72,0xb6,0x76,0xf3,0x41,0xba,0xaa, -0x9c,0x6d,0x82,0xa8,0x6e,0xdc,0x92,0x42,0x8c,0x0a,0x5e,0xd5, -0x69,0x22,0x58,0x9c,0xed,0xcb,0x2f,0x3f,0xaf,0xcd,0x1a,0x5a, -0x3b,0x37,0x61,0xad,0xa9,0x4f,0x1e,0x7f,0xb6,0x7a,0x8b,0x6b, -0xfa,0xe6,0x47,0xb5,0x71,0x83,0x3f,0x29,0x69,0x21,0xf9,0xfb, -0x5e,0xac,0x46,0xa1,0x21,0x13,0x9d,0x11,0xf7,0x0c,0xbd,0x85, -0x9b,0xfa,0xbf,0x47,0x9b,0xd4,0xdd,0x82,0xbc,0x1d,0x6d,0x74, -0xa7,0xa4,0xb5,0x4f,0x5c,0xfb,0xd5,0xb5,0xf5,0x7d,0x2f,0x95, -0x4e,0x49,0x6f,0x2b,0x6e,0xad,0x59,0xf3,0xb3,0xb6,0xb9,0x5f, -0xc1,0x8d,0x29,0xb8,0x91,0x82,0xba,0x55,0xf7,0x2a,0xae,0x33, -0x6b,0x20,0x12,0x16,0x50,0x2e,0x41,0xee,0x05,0x2e,0xa0,0x51, -0x49,0x17,0x2a,0xa5,0x29,0xfa,0xa9,0x9e,0x9d,0x3d,0x38,0xed, -0xb8,0x40,0x4f,0x5b,0xd1,0x0e,0xa2,0xdb,0x53,0xbd,0xd9,0x22, -0xab,0xe1,0xb9,0xc0,0x29,0x9b,0xa4,0xf3,0x7b,0x25,0x26,0xda, -0x33,0xee,0x7e,0xe5,0xd2,0x04,0xdd,0x2d,0xf7,0x42,0x5f,0x67, -0xd6,0xd4,0x90,0x30,0x83,0x2a,0x1c,0xef,0x3f,0xba,0xc7,0x15, -0x9e,0x89,0x13,0x97,0xd3,0xbc,0x4c,0x49,0x71,0x26,0x6e,0x8f, -0x1e,0xe3,0x37,0xfe,0x20,0x60,0xce,0x7e,0x51,0x8a,0xf7,0x14, -0x7d,0x46,0xee,0xb8,0x43,0x00,0x3b,0x6b,0xa8,0x94,0xfe,0x54, -0x2a,0x84,0xf6,0x6a,0x39,0x9a,0x1b,0x28,0xa1,0x51,0x6f,0x67, -0x48,0xa2,0x9e,0x1a,0x85,0xd9,0xf4,0xf8,0xf2,0x22,0x3d,0x21, -0x1d,0x35,0x7c,0x10,0x85,0x4a,0x6a,0xc6,0x01,0xce,0x9c,0xa4, -0x3f,0x9a,0xdb,0x7e,0x5f,0x52,0x64,0x21,0xa7,0x12,0xcd,0x7d, -0x07,0x42,0x38,0xed,0x96,0xfe,0xc0,0x05,0xcd,0x0c,0x40,0x8e, -0x34,0xfc,0xd2,0x73,0x72,0x43,0x4b,0x71,0x7a,0xb4,0xef,0x6c, -0xc4,0xe7,0x23,0xc0,0x7e,0xf4,0xaf,0x6a,0xca,0x1f,0xb0,0x29, -0x6e,0x23,0x12,0xc8,0xaf,0x31,0x34,0x73,0xd8,0xbd,0x10,0x7d, -0x7a,0x86,0x3e,0x3c,0xfe,0x95,0x24,0x0f,0xbc,0x35,0xe5,0x28, -0x7d,0x37,0x9e,0xcf,0x7a,0xe4,0xd6,0x7c,0xf2,0xdd,0x7c,0xa4, -0x17,0x22,0xb3,0x0e,0xdd,0x14,0xcd,0xd8,0x4f,0x1e,0x76,0x76, -0xf2,0x9e,0xac,0x03,0xe4,0xc6,0xd8,0xe6,0xf9,0x3d,0x1a,0x86, -0xd3,0x0a,0x4e,0x59,0x2d,0x9b,0x8c,0x07,0xa3,0x48,0x7b,0x63, -0xb3,0x9e,0xf5,0x54,0x01,0x56,0xaf,0xf1,0x14,0x13,0x0d,0xa3, -0x04,0x98,0xed,0x6f,0x8b,0x37,0x62,0xee,0x68,0xcd,0xd1,0xdd, -0xad,0x69,0x14,0xb6,0x26,0x72,0x17,0x4a,0x93,0x53,0x2f,0x78, -0xfb,0x5b,0x73,0x22,0xf9,0x7d,0x12,0x34,0xbb,0x5d,0x7d,0x46, -0x2b,0xa8,0x9f,0x18,0x7a,0x76,0xe3,0x7b,0xb6,0xfd,0x5f,0xdc, -0x6a,0x81,0xfe,0x0f,0x77,0x55,0x00,0x6f,0x29,0x85,0x49,0x48, -0xd5,0x01,0x8d,0x6c,0x22,0x06,0x23,0x75,0xc4,0x41,0x1a,0xe4, -0x10,0x1b,0x9c,0x3b,0xd3,0x2b,0xcd,0xf8,0xae,0x59,0x93,0x6e, -0x0d,0x7e,0x5d,0xf5,0x86,0xe9,0x7e,0x50,0x3c,0x3a,0x4b,0x25, -0xf4,0xfd,0xe8,0x0c,0xd9,0xd4,0x74,0x94,0x5a,0x36,0x4c,0x19, -0xb2,0xa6,0x93,0xa2,0x29,0xcb,0x71,0x9a,0xa0,0x20,0x8e,0x15, -0xa0,0x61,0x55,0x80,0x51,0x49,0x4b,0xff,0x5e,0xaa,0xb0,0x59, -0xb1,0x66,0x99,0xfc,0x1e,0xd4,0x8f,0x62,0x7a,0xef,0xc7,0x66, -0xc4,0x28,0x32,0x31,0x91,0x34,0xbb,0xf2,0x47,0xd2,0x75,0x6a, -0xc7,0xc0,0x18,0xed,0xc2,0x05,0x88,0x1e,0xc3,0xf6,0x79,0x69, -0x31,0x49,0x1c,0x24,0xe7,0x63,0x2c,0x53,0xa9,0x28,0xc1,0xe3, -0x19,0x56,0xd9,0x20,0xf3,0x54,0xf8,0x57,0x23,0xd1,0x20,0xf3, -0x54,0xe4,0x5a,0x52,0xa3,0x85,0x66,0x89,0x9d,0x1c,0x88,0xe6, -0xd2,0xa4,0x7b,0x18,0x3a,0x5e,0x23,0x9a,0xed,0xf1,0x93,0x00, -0xb9,0x36,0xd5,0x8e,0x8e,0xf5,0x3c,0xa4,0x1c,0x7c,0xc7,0x5d, -0x8d,0xdd,0xab,0x0f,0xac,0x79,0x2b,0x74,0x16,0x77,0x28,0x6e, -0x05,0xed,0x80,0xf0,0x15,0xa0,0x71,0x48,0x77,0x8a,0xb3,0x64, -0x9f,0x18,0xd8,0x13,0x02,0xee,0xcc,0xab,0x7b,0x87,0xe1,0xae, -0xd1,0x60,0x56,0x81,0x79,0xc3,0xf4,0x18,0x0a,0xf4,0xbd,0x14, -0x76,0x29,0x3a,0x13,0xac,0x99,0xf3,0xc4,0x65,0xaf,0xa6,0xd3, -0xdb,0xc0,0x4a,0xe1,0xf2,0x61,0x56,0x07,0xe0,0x8a,0x84,0x8a, -0x36,0x9d,0x32,0x48,0x58,0x4d,0x7e,0x96,0xea,0x7b,0xe5,0x68, -0x8e,0x46,0x41,0xab,0xd4,0xeb,0xb1,0xe2,0x70,0x42,0x61,0xa0, -0xa9,0x5d,0x3e,0x74,0xb9,0xc2,0xa6,0x7e,0xde,0xcb,0xc6,0x99, -0x59,0xdd,0xc8,0xf8,0x8f,0x37,0xc5,0xdc,0x6f,0xcc,0x15,0x34, -0x07,0x19,0xda,0x36,0x1e,0xdf,0xc9,0x8c,0xf5,0x43,0x99,0x31, -0x31,0xf2,0x3a,0x45,0xb3,0x2c,0xfc,0x7f,0x64,0x33,0x27,0xb4, -0x08,0x42,0xfa,0x26,0x3d,0xd9,0xd0,0xed,0xd1,0x06,0x87,0x1e, -0xce,0x9e,0xb2,0xa8,0x42,0x7e,0x7a,0xd6,0x0d,0x3f,0x2f,0x4a, -0xee,0x24,0xf6,0xc5,0x6c,0xf9,0x30,0x0a,0xdb,0xf6,0xf9,0xe8, -0x7a,0x3c,0x5a,0x1c,0x41,0x7d,0x31,0x15,0xbb,0x77,0xfe,0x4d, -0xcc,0xfc,0x5f,0xb0,0x42,0x4c,0xab,0xb8,0x3a,0x63,0x7f,0xa5, -0xd8,0x61,0x63,0xa7,0x93,0xe6,0x6b,0xed,0xb8,0x06,0x3f,0xb9, -0xd0,0x98,0x97,0x94,0x8d,0x4d,0xd9,0x98,0x94,0xa4,0xeb,0x2d, -0x32,0x0e,0x0a,0x34,0xc1,0x00,0x10,0xc4,0x24,0xd8,0x75,0x89, -0x7f,0x36,0x2b,0x46,0x00,0x1d,0xe6,0xe3,0xfb,0x48,0x5e,0x04, -0xdf,0x91,0x12,0x0a,0xba,0xfc,0xe1,0xdd,0x5d,0x6e,0xd8,0x7f, -0x2c,0x5d,0x62,0xf6,0xb2,0x01,0x0b,0x40,0x1f,0x37,0xb8,0x2b, -0xc5,0xe3,0xe2,0x51,0x9e,0x5c,0x01,0x8d,0x1c,0xc2,0x30,0xf9, -0x4d,0x2b,0x3f,0xa0,0xa7,0x69,0xac,0x7f,0x40,0x4f,0xc7,0xaa, -0x5a,0x1d,0x9b,0x01,0xf8,0x7f,0x63,0x7f,0xde,0xaf,0xc3,0x82, -0xcd,0x83,0x21,0x3f,0x01,0x1c,0x17,0xd3,0xfe,0x32,0x9d,0x56, -0x1e,0x52,0x0f,0x70,0x07,0x30,0xfd,0x1f,0x7a,0xcd,0xfb,0xdd, -0xf4,0xbe,0x3d,0x90,0x9e,0xfc,0x2b,0xc7,0x99,0xa1,0xfe,0x65, -0x11,0x7a,0xff,0xbd,0xf4,0x5f,0x4d,0x87,0x93,0x51,0x6e,0x01, -0x20,0xa7,0xa2,0x69,0x20,0xfc,0x92,0x35,0x41,0xeb,0x1c,0x49, -0x2d,0x69,0xda,0xd3,0x73,0xa0,0x9e,0xfd,0x4c,0x98,0x5e,0x0a, -0xe9,0xf0,0xb4,0xcb,0xf6,0x4b,0xf1,0x45,0x94,0x8d,0x22,0x43, -0xa4,0xec,0x27,0xd1,0xcc,0x36,0xee,0x8a,0x90,0x8e,0xeb,0x53, -0xda,0xd3,0xa4,0x82,0x59,0x99,0xf7,0x6f,0x76,0x15,0xa2,0xd7, -0xd5,0xcb,0xf2,0x10,0x7a,0x05,0x97,0x3e,0x27,0x5f,0x12,0xe4, -0xdb,0x50,0xbe,0x8d,0x6b,0xf7,0xb5,0x70,0x91,0x28,0x24,0x13, -0x75,0xb1,0x6b,0xfb,0x14,0xb5,0x3c,0xca,0x68,0x4f,0xad,0x82, -0x6a,0x1b,0x65,0x34,0xaa,0x56,0x41,0x8d,0x0d,0xf8,0x6a,0xe2, -0xd7,0x11,0x7e,0x1d,0x15,0x6f,0xb6,0x73,0x9d,0xc6,0x96,0x07, -0xed,0x74,0xf7,0xec,0xbe,0xec,0x95,0xa8,0x0f,0xe2,0xc6,0x04, -0x35,0xef,0xef,0x9f,0xbd,0xf9,0x61,0xd0,0x9b,0x1f,0x06,0xfd, -0xe8,0xc3,0xa0,0x1f,0xa1,0x45,0xbd,0xe2,0x4d,0xec,0x64,0x36, -0x1f,0x89,0xc7,0x72,0x6d,0x64,0x73,0x9e,0x58,0x49,0xa1,0x8e, -0x21,0x23,0x47,0x14,0x82,0xf8,0x72,0x3c,0x05,0x39,0xff,0x8b, -0x31,0x5a,0xc9,0x18,0x4d,0x07,0xa3,0xac,0x70,0x3f,0xfc,0x4b, -0x38,0x20,0x60,0x12,0x3a,0xe9,0x06,0x42,0x76,0xe2,0x48,0x41, -0xc3,0x14,0x5d,0x61,0xa4,0x3b,0x70,0x25,0x96,0xff,0xa5,0x1d, -0x5f,0x02,0xab,0xa7,0xd1,0xb0,0x12,0x0d,0x8b,0x72,0x6a,0x61, -0x49,0x06,0x45,0x85,0xec,0xd3,0xd7,0x9b,0xe6,0xf6,0x00,0x0b, -0x0f,0xd5,0x65,0x61,0xb3,0xbe,0x99,0x4d,0x67,0xcb,0xd9,0x74, -0x14,0xc8,0x79,0xb6,0x55,0x43,0x7a,0xdc,0x96,0x79,0x42,0xde, -0x65,0x7a,0x1f,0xc2,0xf0,0xa4,0x9c,0xa3,0x88,0x6e,0x53,0x51, -0x53,0xec,0x9a,0xe3,0xb4,0xc5,0xba,0xfd,0x19,0xd2,0x0d,0x14, -0xb5,0xe7,0x92,0xd2,0xd2,0x46,0x2b,0xea,0xe1,0x77,0xb7,0x3e, -0x54,0x7d,0x89,0x84,0xcf,0x2c,0x95,0xf7,0x8f,0x2a,0x3b,0xfe, -0x0c,0x8f,0x71,0xd3,0xa3,0xf2,0xb0,0x6e,0xaf,0x93,0xf0,0x5d, -0x1d,0x42,0xc9,0xca,0x3d,0x25,0x90,0xb2,0x72,0x3f,0xe6,0xc9, -0xde,0xce,0x35,0x80,0x65,0x96,0x6a,0xac,0x8c,0x3d,0xc9,0x5f, -0x14,0x14,0x88,0x69,0xb2,0xf2,0x92,0x7a,0xeb,0x08,0x26,0xaa, -0x2a,0x27,0x01,0x4b,0x3d,0x2e,0x47,0x49,0x05,0xab,0x29,0xe3, -0x9f,0x38,0x0e,0xa5,0xd7,0xec,0xf6,0xb6,0x41,0x68,0x94,0xf1, -0xeb,0x9e,0xc2,0xaa,0xee,0xaf,0x1d,0xe7,0x70,0xff,0xfc,0x6d, -0xf1,0x2e,0x06,0xc9,0x5d,0x77,0x5c,0x9e,0x5e,0xac,0x26,0x93, -0x67,0xcf,0xfe,0x18,0x8d,0x13,0x35,0x6e,0xaa,0xf1,0x91,0xd2, -0xde,0x23,0xe4,0x7d,0xe4,0x50,0x8d,0xd4,0x99,0x69,0x65,0x7a, -0x75,0x32,0x4e,0x70,0xc3,0x40,0x9e,0xeb,0xfa,0xe9,0x92,0x6d, -0x29,0x43,0x6c,0xb3,0xab,0x06,0x1c,0x3b,0x74,0x62,0x8f,0xba, -0x6a,0xc4,0xb1,0x67,0x1c,0x1b,0x9d,0x55,0xfb,0x68,0x20,0xb7, -0xda,0x8b,0xab,0xd1,0x90,0xbe,0x47,0xf0,0x7d,0x9c,0xb3,0xf1, -0x72,0x79,0x2a,0x0f,0x7f,0xd9,0x1e,0x60,0x34,0x57,0xae,0xcf, -0x44,0xd7,0xd8,0xee,0xdc,0x18,0xd5,0x7d,0xe2,0x18,0xd8,0x2d, -0xf3,0xbe,0xad,0x97,0x9b,0x1b,0x7c,0xd8,0xda,0x65,0x02,0x36, -0xd4,0x98,0x8d,0x46,0xed,0x2b,0x92,0xfe,0x8f,0xd8,0xb2,0xfd, -0xba,0x99,0xa4,0x43,0x31,0xd2,0xbc,0x7e,0x78,0xc4,0x36,0xa5, -0xd7,0x30,0x73,0x25,0xfc,0x4c,0x7e,0x73,0xc4,0xf6,0xee,0x37, -0x94,0x8f,0xed,0x30,0x6f,0x28,0x1f,0x7e,0x1f,0xa9,0x15,0x6c, -0xb0,0xa0,0x5c,0x99,0x34,0xb1,0x8e,0xe2,0xea,0x06,0xbf,0x51, -0xf5,0xea,0x08,0xaf,0x72,0x31,0x04,0x35,0x54,0x31,0x07,0x40, -0x30,0x17,0x49,0xeb,0xa4,0xb2,0xea,0x61,0x0a,0xd4,0x83,0x5f, -0x90,0x96,0xbb,0xee,0x34,0x2d,0x11,0x51,0x6b,0x30,0x9b,0xd1, -0x16,0x70,0x39,0xca,0xb4,0x3e,0xa3,0x13,0xc5,0x7d,0xe6,0x46, -0x38,0xdf,0x66,0xac,0x6b,0xa3,0xd8,0x07,0x91,0x31,0xbf,0x7d, -0x7b,0x6b,0x0c,0x6f,0x17,0x10,0xf3,0x6a,0xb6,0x00,0x7e,0x1b, -0xbf,0x1e,0x65,0xd9,0x68,0x32,0xe9,0x59,0x8e,0xc4,0x9b,0x9a, -0x7e,0x6f,0xf0,0x96,0x51,0x79,0x05,0xbc,0x9a,0xa5,0x64,0x25, -0x53,0x5f,0xd9,0x78,0xda,0x7e,0x57,0x6a,0xac,0xfb,0xf7,0x66, -0x3c,0x1d,0x8e,0xd6,0xad,0xb1,0x5a,0xb7,0xae,0x48,0xfe,0x80, -0x1f,0x68,0xfc,0x36,0xae,0x65,0xe8,0x3d,0xd3,0x14,0xf1,0x56, -0xee,0xda,0xe6,0x49,0xbf,0xb6,0x41,0xef,0x79,0xf0,0x79,0x8c, -0x9f,0xf8,0xb5,0x86,0xc8,0x35,0x47,0xae,0x8f,0xf1,0x33,0x69, -0x35,0xb6,0xb1,0xea,0xcf,0x96,0xcb,0xd9,0x25,0xa2,0xc4,0xf7, -0x3a,0xea,0xf9,0xf0,0x7c,0x84,0x06,0x40,0x05,0x41,0x74,0x10, -0x3a,0x3a,0x5b,0xe2,0x93,0x74,0x7a,0x48,0xb6,0x18,0x9f,0x5f, -0xd8,0x10,0xce,0x9e,0xae,0x96,0x99,0x2e,0x5c,0x93,0x32,0xa9, -0x89,0x18,0x2c,0x46,0x40,0x90,0xdf,0xf7,0x26,0x67,0x18,0xc5, -0xc6,0x33,0x4b,0x93,0x52,0x6c,0xea,0xab,0x69,0xc8,0x1f,0x52, -0x46,0xaa,0xa9,0x2d,0xd2,0x1c,0x98,0x3c,0xe0,0xda,0x24,0x0d, -0x0b,0x3a,0x90,0xf0,0xcf,0x6a,0xca,0x12,0xe9,0xee,0x6c,0x1a, -0x56,0xbc,0x55,0x3e,0x76,0x96,0x0a,0x23,0x08,0x29,0x1c,0xaa, -0xa6,0x13,0x31,0xa6,0x65,0xa2,0x5b,0x6c,0xab,0x74,0xb4,0x58, -0x42,0xcf,0x52,0x9b,0x26,0x42,0x5a,0xee,0x81,0x2d,0x50,0x35, -0x1b,0xb9,0xc6,0x04,0x26,0x7d,0x75,0x01,0xd0,0x2e,0x46,0xd0, -0x02,0xfa,0x86,0xf6,0x4e,0xfa,0xb5,0x85,0xc2,0x3f,0x10,0x77, -0x31,0xa2,0x2f,0xf8,0xdd,0x52,0x67,0x91,0x35,0x38,0x5b,0x7c, -0xce,0x6c,0x77,0x31,0xca,0x35,0xbf,0x3d,0x9c,0x61,0x34,0x02, -0x14,0x63,0xd5,0x17,0xa3,0x83,0x3c,0x29,0x0f,0x0f,0xbf,0x02, -0xf6,0xe6,0xe0,0x77,0x67,0x90,0x47,0xcd,0xed,0x3b,0x62,0x2e, -0x0f,0x8c,0x33,0x1c,0x4d,0x6c,0x9d,0x1a,0x5f,0xc2,0x0a,0x30, -0xc5,0x3f,0x84,0x69,0x6d,0x82,0xdf,0x48,0x0c,0xf1,0xf1,0x42, -0x50,0xfd,0x82,0x06,0x72,0x6d,0xc1,0x0d,0xda,0x95,0x3c,0xe1, -0xe4,0x97,0xa3,0x73,0x48,0x2b,0xcc,0xc4,0x37,0xec,0x52,0x63, -0x87,0x46,0x5e,0xcd,0x61,0x79,0x49,0xa8,0x2d,0x08,0xc2,0x09, -0x04,0xb1,0x73,0xba,0x82,0xd5,0x3f,0x0d,0x6e,0x30,0x41,0x7c, -0x37,0x9f,0xcf,0x32,0xc8,0x66,0xea,0x83,0x11,0xad,0x90,0xc4, -0xe9,0x4d,0x7f,0x8c,0xb3,0xb0,0xad,0x32,0x4b,0x54,0xd6,0x94, -0xed,0xca,0xe8,0x1d,0xf6,0x4c,0x7a,0xc3,0x40,0x5b,0x37,0x93, -0x16,0xa4,0x2e,0x5a,0x59,0x73,0xab,0x46,0x73,0x0c,0xfa,0x5c, -0x04,0x7b,0x93,0xac,0x59,0x5b,0x57,0xb3,0xa4,0xb6,0xc6,0x5d, -0x09,0x04,0x36,0x18,0xd8,0xa8,0x1e,0x24,0x0d,0xd7,0xc7,0x8d, -0xce,0x70,0xdd,0x42,0x5b,0xfb,0x3d,0x48,0x1d,0x6e,0x30,0xbc, -0x81,0xf0,0xc6,0x5c,0x10,0x71,0x85,0xb5,0x41,0x8a,0x20,0xd0, -0xb6,0x03,0x96,0x86,0x2d,0x46,0xa5,0xf6,0xa8,0xec,0x19,0x9d, -0x42,0x88,0xc7,0x00,0xa5,0x13,0xe9,0x32,0x3d,0x3c,0x50,0x92, -0xef,0x3e,0x00,0xaf,0x43,0x35,0x06,0x5e,0x1d,0x6a,0x8f,0x5b, -0x91,0x4d,0xb7,0x79,0x7b,0x90,0x54,0x1f,0x6e,0xbc,0xbc,0x00, -0x5f,0x82,0x5b,0xbe,0x26,0xf6,0x49,0x34,0x9a,0x24,0x6a,0x34, -0x11,0x22,0x8d,0x92,0x14,0xc2,0x44,0x7c,0x35,0x6a,0xc2,0x77, -0x93,0xbe,0xc9,0x2b,0xca,0x28,0xb9,0xbd,0x3d,0x18,0x35,0x6f, -0x6f,0x47,0x89,0x74,0x0c,0x70,0x67,0x3a,0x6a,0x9a,0x40,0xec, -0x74,0x2e,0x1d,0x8b,0x0d,0x53,0xc8,0xda,0x2b,0x43,0x96,0x7e, -0x15,0xbe,0xfa,0xf8,0xd5,0x6b,0x7b,0xb6,0x70,0xe3,0x27,0xc9, -0xa8,0x9a,0x34,0x72,0x45,0x47,0x13,0x90,0x17,0xd6,0xa8,0x03, -0x19,0x41,0xc9,0x81,0xc0,0x68,0xe2,0x17,0x00,0x8a,0x41,0xcc, -0xdb,0x70,0xa2,0x44,0xf5,0xaa,0x3a,0x5b,0x0f,0x13,0x47,0xc9, -0x22,0x75,0xd0,0x84,0xb6,0x2c,0x5c,0x4c,0xdb,0x90,0x0e,0xd3, -0x3c,0xc4,0xd6,0x36,0xd8,0x1e,0xf8,0xc1,0x96,0xc0,0xcf,0xe1, -0x21,0x86,0xd6,0x94,0xb4,0xee,0x00,0x71,0x52,0xa2,0x0f,0x00, -0x03,0x8a,0x53,0xa8,0x89,0xa1,0x26,0x9f,0xfb,0xc9,0x90,0x46, -0x9e,0x4d,0x11,0xd9,0xe3,0x74,0x64,0xea,0xa8,0xad,0xcd,0x13, -0x18,0x9b,0xeb,0xf0,0x10,0x26,0xe0,0x14,0xea,0x9a,0xd4,0xd8, -0xa4,0xf2,0x81,0x9f,0xb8,0x70,0x12,0x3b,0x48,0x8c,0xd6,0xcd, -0xba,0x85,0x90,0x61,0xe9,0xc2,0x06,0x6f,0x65,0x68,0x7d,0x77, -0xe6,0x0e,0x2b,0x25,0x53,0xd4,0x48,0x8f,0x28,0xb5,0x9c,0xcd, -0x71,0x84,0x38,0xe8,0x28,0x17,0xd7,0x39,0xac,0x61,0x92,0x05, -0xd0,0x84,0x0e,0x29,0x40,0x51,0x06,0x97,0xee,0x19,0xda,0xdd, -0x17,0x61,0x1b,0xe4,0x23,0x9f,0x02,0xe8,0x76,0x03,0x98,0x51, -0x8c,0x9d,0x6c,0x60,0xcf,0x0e,0x63,0x47,0x57,0xb8,0x81,0xa1, -0x85,0x31,0x6b,0x13,0xb3,0x56,0x67,0x3d,0x58,0x31,0x1b,0xaa, -0xd7,0x9f,0x5d,0x8d,0xd8,0x2d,0x81,0x5f,0x15,0x70,0xf9,0x93, -0xc6,0xed,0x6d,0x18,0x87,0x2e,0x67,0xb9,0x0c,0x01,0x80,0x9a, -0xb0,0x07,0xfa,0x30,0xb2,0xe6,0xb0,0x37,0xa0,0x04,0xa8,0xa7, -0x02,0xb5,0x97,0x31,0x37,0x8c,0x07,0x45,0x90,0xd0,0x41,0x05, -0xa5,0x1e,0xd0,0x4f,0xcc,0x15,0xdf,0xde,0x46,0x04,0x26,0x01, -0x71,0xfd,0x00,0xbf,0xb4,0xb1,0x96,0x2c,0x35,0x98,0x56,0x0d, -0x45,0x27,0x40,0x39,0x06,0x82,0x35,0xc2,0x78,0x9e,0x63,0xb5, -0x30,0xfe,0xe1,0x77,0x33,0x8f,0x9f,0x40,0x31,0xfc,0xc0,0x7d, -0x40,0x13,0x53,0x60,0x08,0x67,0x15,0xcc,0x0a,0xff,0xe2,0x62, -0x34,0x58,0xf5,0x88,0x5f,0x61,0x00,0xab,0xd5,0x06,0x50,0x5b, -0xaf,0x0c,0x2d,0x50,0xcb,0x84,0x48,0xb8,0x99,0xa0,0x67,0x27, -0x9f,0x74,0xcb,0xa3,0x74,0x33,0xb1,0xc4,0x15,0x9c,0x96,0x49, -0x79,0x99,0x1c,0x2f,0x9b,0xe5,0x65,0xb3,0xb2,0x3c,0x2a,0x2f, -0x8f,0xe4,0xd0,0xdf,0xeb,0x5b,0xa6,0x5d,0x8b,0x6b,0x87,0x59, -0x70,0xca,0xc6,0xdb,0x0a,0xd6,0x62,0x85,0xc0,0x63,0x5a,0x89, -0x6b,0xa3,0xf9,0x09,0xcd,0xba,0x29,0xc6,0x29,0x1d,0xb5,0x6b, -0x8e,0xe6,0x09,0xfa,0xf0,0x50,0xcb,0x70,0x04,0x13,0x8d,0xc1, -0x89,0x62,0xbc,0x33,0x57,0x2b,0x43,0xf0,0x34,0xc3,0x26,0xca, -0xbc,0x8b,0xac,0x53,0xdb,0x78,0x1a,0x89,0x39,0xb3,0x7d,0x30, -0x2e,0x9e,0x5f,0xc1,0x0e,0xe5,0xdf,0x57,0xa3,0xd5,0xc8,0x8a, -0x5e,0xa1,0x24,0x70,0xc1,0x0d,0x51,0xb3,0xb3,0xb3,0x6c,0xb4, -0xa4,0x25,0x96,0x05,0x09,0x6e,0x0b,0x04,0x37,0x80,0xd5,0x22, -0x65,0x52,0x56,0x38,0x9b,0xe7,0x3b,0x1c,0x21,0xa7,0xb6,0x2a, -0x92,0x7b,0xd4,0x84,0x2c,0xb5,0xeb,0x7b,0x8c,0xc9,0xf1,0x18, -0xaf,0x86,0xf4,0xaa,0xb3,0x5e,0x52,0x2a,0x6c,0xf5,0x88,0xad, -0x23,0x5d,0xc9,0x31,0x26,0xf1,0xe7,0xed,0xad,0xa9,0x39,0xb5, -0xb1,0x87,0xff,0x17,0x75,0xef,0xb6,0xdd,0xb8,0x91,0x2c,0x88, -0x3e,0x9f,0xf9,0x0a,0x17,0xf7,0xb6,0x36,0x20,0x26,0x29,0x80, -0xba,0x93,0x86,0xb8,0xdc,0x6e,0x7b,0xba,0x66,0x7c,0x3b,0x2e, -0xf7,0xb4,0x6d,0x16,0x5b,0x0b,0x24,0x21,0x09,0xe2,0xd5,0x04, -0x55,0x22,0xab,0xc4,0xbd,0xf6,0xac,0x35,0x6b,0x7f,0xc1,0xf9, -0x95,0x7e,0xea,0xb7,0xf3,0x01,0xf6,0x2f,0x9d,0xb8,0xe4,0x15, -0x17,0x8a,0x55,0x76,0xf7,0x9c,0xe9,0x76,0x89,0x40,0x66,0x64, -0x64,0x64,0x64,0x64,0xe4,0x2d,0x10,0x71,0xc0,0x3d,0xcc,0x80, -0x4c,0x67,0x73,0xed,0x73,0x54,0xe5,0x2d,0xd5,0x98,0x2d,0xe8, -0x1a,0x3a,0x15,0x01,0xae,0x70,0x4a,0xd6,0x10,0x0e,0xe9,0x59, -0x19,0xe1,0x99,0x45,0xf6,0xc1,0xc1,0x04,0x37,0xe0,0x2f,0x40, -0x3e,0x28,0x76,0x4a,0x07,0x72,0x75,0x15,0x21,0xe0,0x4f,0xa6, -0x8b,0xd5,0xa6,0x24,0xa0,0x1c,0xda,0x9c,0xe6,0x70,0x4b,0xb4, -0x5b,0x81,0xd4,0x53,0xde,0x6f,0xa1,0x8c,0xc8,0x01,0xf6,0x11, -0x7d,0x28,0xbf,0x4a,0xdd,0x4c,0xf8,0x9c,0xa0,0x63,0xcd,0x1f, -0x5b,0x31,0x75,0xa3,0xef,0xf3,0x7c,0x92,0x4c,0xf3,0x95,0xe0, -0xa1,0xbc,0x5c,0x62,0xae,0xdb,0x08,0xa0,0x99,0x0c,0xda,0x96, -0xde,0xa9,0x1f,0x40,0xb6,0xa0,0x09,0xcb,0x78,0xb8,0xfa,0x2a, -0x2d,0x8b,0xa6,0x97,0x6f,0xb8,0x3c,0x95,0xdd,0x6e,0x3b,0x7a, -0x25,0x88,0xeb,0x7c,0xcf,0x17,0xf9,0x75,0x4e,0xc4,0x09,0x76, -0x29,0x2d,0x6c,0x30,0x65,0xc3,0xae,0x12,0xbf,0x06,0x81,0xc5, -0x29,0x2c,0xa1,0x97,0xf8,0x67,0xc2,0x8f,0xf4,0xcc,0x4b,0x9d, -0xf9,0x12,0xb7,0x20,0xa8,0xef,0xc5,0x0a,0x7a,0x47,0x2c,0xc4, -0x1b,0x98,0x38,0x17,0x53,0x9c,0xf2,0xab,0xaa,0xe8,0x20,0x27, -0x2d,0xaa,0xa9,0x57,0x3d,0x8c,0xdf,0x68,0x2a,0xb5,0x79,0x85, -0xe7,0x1e,0xbe,0xaf,0x30,0x82,0xd2,0x2a,0x2b,0x2b,0x73,0x61, -0x56,0x35,0x48,0x70,0x72,0xd5,0xe9,0x28,0xda,0x26,0xe3,0xe0, -0x40,0x65,0xac,0xed,0x02,0x20,0xde,0xd8,0x44,0x77,0xa5,0x4d, -0xeb,0x71,0x4f,0xc2,0xfb,0xc4,0x89,0xdc,0x22,0xdb,0xc3,0x42, -0xb4,0x19,0xcb,0x65,0xf0,0x12,0x54,0x66,0x27,0x11,0xad,0xc3, -0x99,0x71,0x1e,0xb2,0x4d,0xe3,0x54,0xcc,0xac,0xdc,0x3c,0x25, -0xb9,0x9d,0x13,0x6b,0x16,0x42,0xac,0x0b,0xa3,0x73,0x2c,0xaa, -0x41,0xaf,0xb2,0x0a,0xf9,0x2e,0xf3,0x60,0xb8,0x4a,0xd2,0xac, -0x44,0x1b,0xf3,0x82,0x96,0xb5,0xfa,0x23,0x22,0x6f,0xa1,0x29, -0xc6,0xe0,0x1d,0xc8,0x07,0x23,0x05,0x7b,0xb4,0x60,0xf9,0x01, -0x2d,0xd0,0x15,0x2c,0x89,0x50,0x68,0x41,0x91,0x56,0x0d,0xb3, -0x83,0xde,0x5d,0x02,0xa9,0xe2,0x14,0x14,0xe5,0x4a,0x86,0x90, -0x67,0x99,0xb0,0x72,0xf5,0x78,0xc4,0x88,0x23,0x45,0x89,0x91, -0x5c,0xad,0x96,0x94,0x65,0x59,0xd6,0xb2,0x28,0x44,0x66,0x7b, -0x24,0x0b,0xc2,0x40,0x2b,0x17,0x31,0x2e,0xfc,0x26,0x42,0x30, -0xa9,0x47,0x98,0x19,0x6a,0x56,0x25,0x04,0xbc,0x3a,0xa3,0x27, -0xde,0xc6,0xfa,0x39,0xa0,0xa5,0x06,0x5a,0xda,0x40,0xba,0xca, -0x52,0x91,0xc1,0x44,0xae,0xdf,0x81,0x93,0x3d,0x36,0x8d,0x40, -0x74,0xb1,0x59,0xcd,0x0d,0x2e,0xfa,0x70,0xec,0x79,0xa8,0x2b, -0x22,0x1c,0x01,0xd8,0x5a,0x52,0x1e,0xd0,0x2e,0x56,0x20,0x00, -0x8e,0x72,0x52,0x1c,0x2c,0x00,0xb2,0xd7,0x40,0x59,0x4c,0x4b, -0xa4,0x2c,0x27,0x66,0x6e,0xa3,0x13,0x51,0xb5,0x7a,0x58,0x4c, -0xfb,0xd8,0xfc,0xe2,0xb8,0xda,0x63,0x60,0x55,0x8e,0xac,0x8a, -0xa1,0x05,0x6d,0xc4,0x61,0xb5,0x63,0x04,0x7c,0xd8,0x10,0x20, -0xbc,0x64,0x0c,0x3d,0x29,0x11,0xba,0xfc,0x99,0x82,0x4f,0xc2, -0x5e,0x72,0x8a,0xd0,0x29,0x2b,0x4d,0x6d,0xf4,0xf5,0xda,0x4a, -0x8b,0x58,0xc9,0xe9,0x1a,0x9a,0x93,0xac,0x96,0x49,0xf2,0x19, -0x06,0x4a,0x59,0x39,0xa6,0xa1,0xcd,0xf5,0x73,0xe0,0x8e,0x91, -0x68,0x73,0x53,0x0d,0xfe,0x35,0x86,0x8c,0xd4,0x27,0x37,0x93, -0x24,0xbe,0x69,0xbf,0xc0,0x10,0x8d,0xa3,0x24,0x6b,0x2b,0xe3, -0x6f,0xde,0x68,0xcb,0xc3,0x9b,0x8d,0xdc,0x6f,0x57,0x62,0xc4, -0xe8,0x4c,0x2b,0xef,0x86,0x70,0x48,0x2f,0x7b,0xec,0x91,0x0f, -0x35,0xc6,0x0b,0x0c,0x64,0x9c,0x4b,0x97,0x5f,0x48,0xae,0x23, -0xdc,0x66,0xaf,0xc3,0xfa,0xba,0x85,0xdf,0xd0,0xd2,0xdb,0x26, -0xac,0xa3,0x2b,0xbf,0xe1,0x5d,0x3a,0x19,0x2d,0x93,0x59,0x84, -0x65,0x9b,0x44,0x5c,0x47,0xa5,0x51,0x04,0x95,0x2a,0x1a,0x2c, -0x20,0x59,0x65,0xb6,0x06,0xdc,0x06,0x23,0x85,0x55,0x79,0xb6, -0x74,0xd8,0xc7,0x72,0x4c,0xb0,0x53,0xba,0xb5,0x4f,0xe9,0x16, -0xd5,0x9d,0x6d,0x08,0x87,0xd5,0x9a,0xde,0xf1,0x3e,0xa5,0x8f, -0xfb,0x4c,0xb3,0x64,0x96,0xcb,0x78,0xcb,0x48,0xf9,0xbb,0xdb, -0x01,0x1f,0x8b,0xc6,0xf4,0x21,0x36,0xbe,0xc1,0xe8,0x57,0xcf, -0x03,0xf9,0x85,0xfa,0x32,0x8a,0x61,0x6b,0x19,0xdf,0xc2,0xcf, -0x2d,0x6c,0xa2,0xe0,0x67,0x20,0x06,0xcb,0x68,0xd0,0x5c,0x36, -0x60,0xe1,0x32,0xb8,0x85,0xa7,0xdb,0x46,0x7c,0x2b,0x06,0x03, -0x78,0x1a,0x34,0xe2,0x41,0xe1,0xa3,0x6d,0xed,0x22,0xab,0xf6, -0x2f,0x64,0xb2,0x00,0xd8,0xaf,0xef,0x92,0x35,0x1f,0x11,0x2c, -0x69,0xe6,0x8f,0x97,0xf5,0xc1,0xf2,0x10,0x06,0x51,0x55,0xfe, -0x6d,0x7d,0x70,0xbb,0x2b,0x7f,0x50,0x1f,0x0c,0x30,0xbf,0xb2, -0xb1,0xdf,0x0c,0xee,0x65,0xbc,0x63,0xf9,0x2d,0x89,0x48,0xa3, -0x77,0x5b,0x31,0x84,0x3f,0xb4,0x20,0x1b,0x7f,0x94,0xce,0x3e, -0x8a,0x7d,0xfa,0x19,0x74,0xd3,0xde,0x98,0x2e,0xed,0x2c,0x04, -0x5e,0x0c,0x69,0x62,0x00,0x7f,0xfc,0xf6,0x10,0xb3,0xf1,0xdd, -0x14,0x1d,0x70,0xd1,0x18,0x43,0x64,0x63,0x2e,0x01,0x96,0xb1, -0x42,0x97,0x48,0x7d,0x02,0xc4,0xaa,0xf0,0x43,0x5c,0x09,0x3a, -0xac,0x6c,0xc1,0xd7,0x0f,0xd3,0x01,0x45,0xca,0x34,0x07,0xd9, -0x83,0x46,0x14,0x47,0xf5,0x58,0x14,0x59,0xfd,0x11,0x5e,0x8f, -0xad,0x2a,0x71,0xbd,0x5a,0xa1,0x4f,0x55,0xc3,0x8d,0xa9,0x48, -0xc5,0xbd,0x98,0x89,0xb9,0xc8,0x30,0x6e,0x4e,0x16,0xe2,0x1f, -0xbc,0x0a,0xfc,0x19,0xcd,0x2a,0x90,0xe4,0xb8,0x8e,0xd6,0x84, -0x03,0xfa,0xeb,0xe2,0xba,0x9e,0x11,0x61,0xcd,0x09,0x6c,0xa7, -0x5f,0xe2,0x61,0x7c,0x14,0x50,0x78,0xc6,0x69,0x54,0x0e,0x97, -0xac,0x93,0x21,0x4a,0x17,0xae,0xf7,0x51,0x09,0x43,0x09,0xd8, -0x07,0xc9,0x90,0x74,0xcd,0xec,0x61,0x90,0x31,0x71,0x19,0xd1, -0x21,0x21,0x40,0x6b,0xff,0xcc,0x20,0xef,0xd2,0xb6,0xb6,0x00, -0x59,0xb7,0xf1,0x76,0x75,0x0b,0x83,0x9f,0xf3,0xd8,0xaf,0x1d, -0x34,0xe1,0x19,0x0a,0xa9,0x45,0x59,0xf0,0xc9,0xc0,0x7c,0xdd, -0x51,0x56,0xbf,0x8f,0x77,0x9c,0x68,0x04,0xfd,0xb3,0x8e,0x93, -0xb5,0xb3,0x51,0x31,0x86,0xa3,0x9f,0x99,0xad,0xcc,0x3c,0xfa, -0x19,0x2f,0x3b,0xe7,0xcd,0x75,0x44,0xd7,0xc0,0xa4,0xcf,0xe6, -0x4d,0xca,0xe3,0xe3,0xd1,0xac,0x07,0xaf,0x78,0x09,0x4a,0x04, -0xe1,0x4b,0x03,0xc3,0x68,0x42,0x09,0xa1,0xf7,0x65,0x90,0x88, -0xc1,0x51,0xef,0x23,0x00,0xec,0xcc,0x64,0xf8,0xca,0x9f,0x7b, -0xf7,0x7d,0x00,0x6e,0x70,0x80,0xb9,0x42,0xe9,0xba,0x42,0xec, -0xa2,0x69,0x55,0xa0,0x89,0xac,0xa0,0xf1,0x2e,0x5d,0xf4,0xd0, -0x47,0x94,0xb9,0x9a,0xfa,0xd5,0xd5,0xd4,0xc3,0x1d,0xf4,0xfe, -0x6c,0x6f,0x37,0xc5,0xac,0xd1,0x10,0x90,0xca,0xdf,0x63,0x21, -0x9f,0x4a,0x45,0x7e,0x11,0xc3,0x34,0xfd,0xc5,0x64,0x1e,0xaf, -0xbc,0xa9,0xb4,0xbf,0xd7,0x09,0x73,0xdc,0x50,0x50,0x7f,0x52, -0x3c,0x50,0x1f,0x78,0x2e,0x8d,0xc7,0xdc,0x96,0x74,0x0d,0xd5, -0xed,0xfd,0x5a,0x40,0xd4,0xa9,0x61,0x89,0xc7,0x5d,0xfa,0x3b, -0x0d,0x85,0x39,0xe8,0x77,0xa9,0x8f,0x83,0x3e,0x9e,0x6f,0x15, -0xc7,0xe0,0x1c,0x9e,0xeb,0xb5,0xda,0xd6,0x2f,0xd9,0x52,0x0e, -0x2c,0xdf,0x03,0x52,0x2b,0xd8,0x31,0x4d,0xb3,0x9e,0x94,0x1e, -0x5f,0x12,0x6a,0xa9,0x08,0xf3,0xa1,0x51,0xe5,0xf8,0x96,0xd1, -0xbc,0xe4,0x46,0xf3,0x06,0xc4,0x18,0xb4,0x7a,0xea,0x7c,0xd2, -0x22,0x25,0xba,0xd1,0x48,0xaf,0xa2,0xe0,0xe0,0xe0,0x85,0x77, -0x53,0x80,0x81,0xea,0x09,0x91,0xdf,0x31,0x9a,0xac,0xaa,0xc6, -0x4f,0x97,0xcb,0x78,0x63,0x14,0x4a,0x2a,0xc8,0x2c,0x6b,0x88, -0x7f,0x66,0x71,0xa4,0xbf,0xfe,0x9a,0xe1,0x2c,0xa1,0x9e,0xe5, -0xb7,0xad,0xb8,0xf9,0xd4,0x00,0x2a,0x57,0xdd,0xd5,0x03,0x4b, -0x02,0xc9,0x93,0xb5,0x36,0x92,0x73,0xd5,0x72,0x8a,0x6a,0x19, -0x6f,0xe0,0xa5,0x18,0xc4,0x12,0x7e,0x88,0x07,0x09,0x98,0x2d, -0xd3,0x07,0x76,0x3a,0x96,0xa8,0x52,0xcf,0x4e,0xad,0x04,0xbd, -0x46,0x56,0x54,0x2a,0xe9,0x24,0xce,0x92,0xeb,0xe1,0x24,0x9e, -0x2e,0xbc,0x9b,0xa2,0x2f,0x2d,0x23,0x0f,0xc1,0x55,0xb4,0xea, -0x06,0xed,0xd5,0x55,0x14,0x76,0xc3,0xf6,0x0d,0xe4,0x94,0xe0, -0x59,0x26,0xf4,0x49,0xe1,0x6e,0x4c,0x61,0xe3,0x06,0x2d,0x9b, -0xca,0xcb,0xdf,0x4c,0x70,0xa6,0xdb,0x55,0x1e,0x17,0x4a,0xcd, -0xd3,0xab,0x55,0xf7,0xc6,0x6b,0xc1,0xac,0xd9,0x6e,0x01,0xba, -0x16,0x5a,0x72,0xe7,0x27,0x50,0xc2,0x88,0x0b,0x0d,0xab,0xee, -0xd5,0xe1,0xaa,0xa4,0xf5,0x0f,0x83,0x74,0xe8,0x02,0x55,0x82, -0xbd,0x9c,0x7d,0xf3,0xb0,0xf2,0x38,0x64,0x1e,0xb2,0xc4,0x39, -0x57,0x46,0xe6,0x98,0x03,0x69,0x32,0x2f,0x68,0x45,0x80,0x0b, -0x0f,0x41,0xf1,0xac,0xb3,0xf3,0x5f,0x64,0xe6,0x89,0x6c,0xc2, -0xea,0xb8,0x7d,0x7c,0xe8,0xad,0x1a,0xab,0x96,0x5f,0x5f,0x1d, -0x37,0x9a,0xe7,0xa7,0x7e,0xb1,0x5e,0xbc,0xfa,0xf6,0x92,0x5d, -0x1c,0x35,0x5f,0xaf,0x8b,0xa4,0x8c,0x09,0xec,0xf2,0xc2,0xf0, -0xdf,0xf8,0xd0,0x38,0x94,0x5f,0xf9,0x17,0xcb,0x24,0xeb,0x45, -0x59,0x0d,0x2d,0x11,0x06,0x48,0x71,0xe8,0x97,0x94,0x61,0x7f, -0x43,0xc5,0xaa,0xe8,0x3c,0x14,0xba,0xfc,0x70,0x55,0x56,0x11, -0xce,0x6c,0xc0,0xff,0x58,0x5d,0x13,0x64,0x9d,0x8a,0x6f,0xcc, -0x3e,0x69,0xa1,0x5d,0x7f,0xd4,0x3c,0x39,0xf5,0x45,0xe1,0xfb, -0xb3,0x2c,0x5a,0x1c,0xfd,0xfa,0xbf,0x2c,0x27,0x31,0xe1,0x51, -0xec,0xa3,0x05,0x0e,0xac,0x45,0x21,0xeb,0xc4,0x2f,0x53,0x72, -0x61,0x3d,0x3e,0xb4,0x9a,0xd6,0x80,0xb6,0xad,0xac,0xcf,0x5d, -0xa1,0x9d,0x99,0x7f,0xf8,0xeb,0xff,0x3a,0x5a,0x94,0x71,0x95, -0xb6,0x34,0xc6,0xca,0x25,0x83,0xd5,0x53,0x16,0x85,0xcd,0xf3, -0x20,0x3c,0xbd,0x28,0xad,0x0d,0xe5,0xca,0xf3,0xb2,0x7a,0xe8, -0x1f,0x22,0xe2,0x32,0x94,0xb0,0x20,0x1c,0x3a,0x0c,0x3c,0x6a, -0x81,0x50,0x80,0xa4,0x9c,0x37,0x4f,0xcf,0x5a,0xa7,0x28,0x98, -0xed,0x56,0x2e,0x0d,0xa8,0x84,0x6a,0x4f,0x29,0x15,0xdd,0x9a, -0xc0,0x4f,0xbb,0x25,0xdf,0x5d,0xa8,0x56,0xb3,0x65,0x81,0x5d, -0x1e,0x03,0xa0,0x93,0x7b,0xe6,0x64,0x5f,0x9c,0x00,0x40,0x95, -0xca,0xfc,0xd3,0x70,0x62,0xad,0xbf,0xef,0xf0,0x4d,0xae,0xbf, -0xf1,0x59,0xad,0xbf,0xef,0x40,0x79,0xde,0x89,0x78,0x08,0x3f, -0x43,0x11,0x4f,0x50,0x95,0x8a,0xc1,0x1d,0x68,0xd1,0xbb,0x46, -0x0c,0xea,0x72,0x08,0x4f,0xc3,0x46,0x3c,0x14,0x83,0x09,0x6a, -0xd6,0x46,0xac,0x3d,0xd7,0xa6,0xd9,0xd7,0xf1,0xd7,0xde,0x60, -0x88,0x6e,0xba,0x00,0x2c,0x40,0x1c,0x9c,0x16,0x0f,0xfd,0x2e, -0x94,0x6a,0xc3,0xaf,0x90,0x50,0x77,0x7e,0x17,0xfe,0x20,0xd0, -0x9d,0x02,0xba,0x43,0xa0,0xbb,0x36,0xfc,0xb6,0x07,0x77,0x57, -0xe1,0x45,0xd0,0x1d,0xdc,0x35,0xa2,0xe3,0xb3,0xa0,0xdd,0x80, -0x97,0xab,0x01,0x7e,0x3f,0x3b,0xb8,0xab,0x63,0x4a,0x69,0x57, -0x41,0x63,0xa1,0x1d,0xd7,0x93,0x18,0xf6,0x15,0x77,0xf5,0xc1, -0x1d,0x8c,0xe1,0x78,0x58,0x1f,0x0c,0xf1,0x77,0x52,0x1f,0x4c, -0x0e,0x79,0x72,0xac,0xe4,0x4e,0xe6,0x70,0x27,0xb3,0xb8,0x93, -0x15,0xb8,0x93,0xc1,0x4f,0x56,0xc2,0x9d,0x0c,0x9e,0xb2,0x46, -0x9c,0x55,0x72,0x27,0x23,0xee,0x64,0xd8,0xf0,0x4c,0x35,0x3c, -0xc3,0x86,0x67,0xed,0x38,0xfb,0x47,0x73,0x27,0x9b,0x5c,0xd3, -0xae,0x4b,0x71,0x27,0xab,0x0f,0xb2,0x3d,0xb9,0xf3,0x65,0x6c, -0xef,0xdd,0x88,0xc7,0x92,0x3b,0xf8,0xac,0xb8,0xc3,0xfc,0x88, -0x71,0xfe,0x8d,0xf5,0xde,0x4d,0x71,0x42,0x0c,0x62,0x78,0x8a, -0x1b,0x71,0xbc,0xcf,0xde,0x0d,0xeb,0x07,0xd4,0x4c,0x30,0x13, -0x08,0x88,0xeb,0x83,0x18,0x7f,0xe5,0xee,0x6b,0x17,0xc1,0xdf, -0xf1,0x3e,0x2d,0xb7,0x79,0x29,0xe3,0x8c,0xbd,0xad,0xc3,0x4d, -0x4c,0x6e,0x90,0x1b,0x2f,0xa5,0xd2,0x97,0xe9,0x32,0x88,0x7a, -0x53,0x68,0xdf,0xb4,0x39,0xe8,0x8b,0x65,0x88,0x2f,0x43,0x78, -0x19,0xf5,0xc5,0x78,0x1d,0xd9,0x05,0x8c,0xf7,0xb5,0x25,0x3a, -0xce,0x7a,0xeb,0x64,0xfe,0x71,0xbe,0x82,0x64,0x28,0x8f,0xde, -0xb3,0x2a,0x8a,0xd9,0xa9,0x9f,0xcd,0xa7,0x03,0xfc,0xaa,0x7e, -0x19,0x0a,0x28,0xd6,0x18,0xbf,0xc5,0x20,0xe9,0x41,0x67,0x89, -0x16,0xa6,0x87,0x4b,0xb4,0x3d,0xfd,0x64,0x19,0xd2,0x73,0xc0, -0x81,0x58,0x39,0x07,0x4d,0x26,0x29,0x85,0x9e,0xc6,0x6b,0xfe, -0x79,0x8b,0x3f,0xbe,0x20,0x17,0xfb,0xd2,0x4d,0xae,0x37,0x5e, -0x77,0x2d,0x3f,0x00,0x54,0x46,0x10,0x0e,0xdf,0x0e,0xd8,0xd0, -0xa0,0x4a,0x04,0x55,0xe8,0xf8,0xb6,0x62,0x64,0xc6,0xef,0x03, -0x70,0x25,0x01,0xae,0xdc,0xf4,0x39,0x83,0x1d,0x4e,0xf4,0xc6, -0x6b,0x68,0xad,0x4a,0x1a,0x27,0x8f,0xd1,0x78,0x63,0xd7,0x3a, -0x7e,0x0b,0xd9,0x36,0xd6,0x76,0x50,0xde,0x17,0xc8,0xbd,0x67, -0x6c,0xa3,0xcb,0x0b,0xda,0x1e,0xf1,0xa4,0x8f,0x07,0xc7,0x33, -0x59,0xae,0x86,0xd8,0x58,0xd5,0x8c,0xf1,0x6a,0x15,0xbf,0x8a, -0x8a,0xc6,0x14,0xeb,0x1a,0x7e,0xa1,0xeb,0xca,0x6b,0x51,0x9d, -0x85,0x76,0x8a,0x63,0x87,0xc8,0x7a,0x34,0x26,0x3a,0x09,0x05, -0xbf,0xa0,0xd1,0x62,0x95,0x14,0x7f,0xaf,0x85,0x4f,0xaf,0x74, -0x67,0x66,0xa3,0x2c,0x3e,0xc5,0x01,0x68,0x04,0x14,0x86,0xe4, -0x1f,0xdc,0x94,0x01,0x5a,0x89,0x46,0x9f,0x5a,0xae,0x41,0x56, -0x83,0xe8,0x0f,0xd6,0xeb,0x12,0x73,0xa5,0x1f,0xf0,0x25,0x66, -0xc9,0xe7,0x47,0x4c,0xc7,0x1e,0x12,0x8f,0x98,0x4a,0x4f,0x63, -0x4a,0x23,0xb7,0x23,0x63,0x4a,0xc4,0x47,0x63,0x65,0x0a,0xad, -0x7a,0x11,0xad,0xd8,0x26,0x6f,0x85,0xcd,0xa3,0x37,0xd8,0x09, -0x79,0xca,0xd3,0x83,0x71,0xa0,0x51,0xe3,0x30,0xcd,0x68,0x8b, -0xcd,0x0f,0x7e,0xcd,0xde,0x70,0x87,0xb0,0xd3,0x2e,0xdd,0x99, -0x51,0x2d,0x82,0x2a,0xf1,0xb7,0x02,0x20,0x8f,0x77,0x40,0x86, -0x04,0x49,0xc1,0x7c,0xfd,0xb6,0x22,0x8c,0x28,0x2a,0x21,0xa8, -0xbe,0x1a,0xd4,0x91,0x8a,0xb6,0xca,0x03,0x82,0x96,0xf1,0x8b, -0x68,0x39,0xe8,0x7a,0xcb,0xb8,0xb1,0x1c,0x90,0xce,0x5d,0x0e, -0x48,0xc3,0xb6,0x97,0x83,0xc6,0x32,0xc6,0x14,0x1c,0x68,0xb1, -0xd4,0xba,0xf6,0x81,0x01,0x3d,0xc9,0xcf,0x8a,0xea,0x35,0x66, -0xaa,0x67,0x35,0xb6,0xd1,0xaa,0x22,0x7c,0x19,0x43,0x47,0x10, -0xc9,0xcb,0x81,0x3e,0x27,0x28,0x60,0xaa,0x2f,0x99,0x5c,0xe8, -0xa8,0x17,0xd1,0xe3,0xa0,0x5b,0x5d,0x37,0x76,0xdd,0x0f,0xfb, -0x55,0xfd,0x18,0x43,0x6f,0x43,0xd5,0xed,0xc7,0x92,0x9a,0x25, -0x9e,0xfa,0xa3,0xac,0x78,0xcc,0x1d,0x3e,0x66,0xbe,0x8e,0xb9, -0xc3,0xc7,0xdc,0xe1,0xb3,0xa8,0x50,0x9a,0xbc,0xa8,0x3c,0xd3, -0xed,0xb3,0xc6,0x49,0x15,0x69,0x54,0x9b,0x18,0x9b,0x8e,0x9f, -0x55,0x37,0x83,0x68,0x11,0x63,0xdd,0xf5,0x5e,0xa8,0xc9,0x0c, -0x25,0x89,0x7e,0x49,0xfb,0x98,0xc2,0xfa,0x58,0xb6,0xcf,0x9c, -0xc2,0x88,0xfc,0x7e,0x0d,0x87,0xe1,0xdc,0x7c,0x43,0x32,0xeb, -0x7c,0xd0,0xf6,0xf9,0x61,0xf6,0xcc,0x61,0x5b,0x34,0x68,0x78, -0x78,0xdc,0xe6,0x77,0xc3,0x23,0x0f,0x66,0x4b,0xbf,0x1d,0x18, -0x52,0xb4,0x03,0x4a,0x6f,0x0d,0x39,0x87,0x83,0x1d,0xb8,0x3f, -0xa3,0x9d,0xe2,0x07,0xa0,0xfe,0xc8,0x32,0x44,0xd7,0x3b,0xe7, -0x50,0xc8,0x1a,0xf3,0x7b,0xb7,0x49,0xbc,0x99,0x3f,0xac,0xae, -0x07,0xf4,0x79,0x0f,0x45,0x8b,0x9b,0xa4,0xb3,0xb1,0xe1,0x58, -0xa6,0x62,0x22,0x8d,0xa5,0x33,0x4c,0xb4,0x0e,0xe1,0x77,0x76, -0x7b,0x29,0x26,0x43,0xf2,0x71,0xea,0x20,0xfa,0x12,0x16,0xd9, -0x2b,0xd0,0xa7,0xd3,0xf9,0xec,0x53,0xfc,0x1a,0x00,0x7d,0xb6, -0x10,0x26,0x2c,0x6d,0xdc,0xa8,0x50,0x52,0xbf,0x43,0x3f,0x2f, -0xa2,0x08,0x30,0x41,0xa7,0x50,0x85,0xf4,0x17,0x83,0x6a,0xc3, -0xce,0x43,0xd8,0x5f,0x1e,0x52,0x86,0xb9,0x1d,0x1f,0xe7,0xbe, -0x1d,0x4d,0xf0,0xca,0x84,0x11,0xa9,0x0f,0xf4,0xf9,0xbc,0x66, -0x2c,0x02,0xae,0x1b,0xc9,0x87,0x7f,0x12,0xb7,0xfb,0x41,0x61, -0x35,0x67,0x54,0x2b,0x32,0xba,0x66,0x30,0xec,0x89,0x55,0x3a, -0x7d,0xe0,0x41,0x28,0xf9,0x32,0x41,0xa2,0x67,0x1f,0xbb,0xf2, -0xc5,0xd7,0xd0,0xf2,0x20,0x12,0x51,0xd1,0x7d,0x46,0xa4,0x5a, -0xca,0x18,0xf8,0x27,0x47,0x62,0x69,0x61,0x9d,0x58,0x4d,0x7a, -0x59,0x5f,0x90,0x58,0xc1,0x3e,0x3a,0x8e,0xa2,0x68,0xa0,0x36, -0xd2,0xb1,0xe6,0x6a,0x8c,0xb7,0x37,0x59,0xa1,0x57,0x0d,0x13, -0x70,0x19,0xf9,0x1c,0x0c,0xcc,0x62,0x84,0x27,0x62,0x6c,0xf2, -0xbc,0x8d,0x8a,0x45,0x03,0x3b,0x29,0xbb,0x83,0x76,0x8e,0x28, -0x9d,0xcc,0xfe,0xb8,0x54,0xc4,0x40,0x20,0x10,0x26,0x9b,0x72, -0xf6,0xc3,0xaa,0xc7,0xb1,0x2e,0x5d,0xc6,0x21,0x68,0xf0,0x30, -0xf9,0xe3,0x32,0xbe,0x25,0xb1,0xc2,0xbb,0xad,0x51,0xf3,0x26, -0x5d,0x27,0xa3,0xa7,0xa8,0xb5,0x13,0x1e,0x24,0xc8,0x82,0x3e, -0x88,0x1a,0xe7,0x95,0xe0,0x5f,0xcd,0x1f,0xb2,0x64,0xfe,0x06, -0xf4,0x84,0x8d,0xfe,0x44,0x80,0x08,0xc2,0xca,0x13,0x6d,0x9b, -0x9a,0x0b,0x58,0x4b,0xe6,0x2e,0xd3,0x8a,0x18,0x1e,0x56,0x6e, -0x8d,0xa7,0x95,0xf0,0x9f,0x0e,0x87,0x0f,0xd3,0x07,0x9a,0x21, -0xf1,0x54,0x06,0xb6,0x0a,0x8b,0xbb,0x58,0x0c,0xef,0x70,0xbc, -0x4a,0x33,0xfe,0x21,0x9e,0xbf,0x0f,0x37,0x6c,0x62,0x87,0x40, -0x4d,0xce,0x86,0xd4,0x17,0xf4,0x8a,0x37,0x76,0xfa,0x13,0xbe, -0x21,0x5f,0xdc,0x45,0x94,0x43,0x8f,0x82,0xef,0xcc,0xf4,0x19, -0xbb,0xad,0x56,0x87,0x9c,0x85,0x27,0x6e,0x2c,0xfe,0x43,0x0e, -0xb2,0x5d,0x41,0xe2,0x30,0x47,0x9f,0xb0,0xc8,0xa9,0x47,0x43, -0xf9,0x24,0x86,0x6b,0xf3,0x72,0x08,0x0f,0x6b,0x20,0xdf,0x4d, -0xd9,0xf8,0xba,0x31,0xd2,0x01,0xb3,0x6e,0xc9,0xd3,0x93,0x95, -0xde,0x5c,0x4b,0xf7,0xa5,0xb0,0x8e,0x18,0xcd,0xa7,0x9e,0xdf, -0x68,0x9e,0x0a,0x2b,0x7b,0x53,0xcc,0xe6,0x9d,0xd2,0x38,0x22, -0x4a,0x0f,0x25,0xa5,0x3d,0xab,0x0c,0x5d,0x3f,0xf4,0x3b,0x0e, -0xe9,0x26,0xfb,0x33,0xe6,0xed,0x98,0x1a,0x31,0x3e,0xb4,0x49, -0xa1,0x56,0x38,0x49,0x9b,0x2d,0x63,0x59,0x47,0xc3,0xf5,0x91, -0x85,0x50,0xf2,0x65,0x13,0x0d,0x37,0x76,0x72,0x99,0x14,0xdc, -0xa5,0xc9,0x32,0x5e,0x0e,0xef,0x36,0xdf,0x71,0x28,0x8a,0x39, -0x5d,0x6b,0x09,0x9d,0x6c,0x6d,0xd7,0x54,0xb4,0x8a,0x3c,0x88, -0xa8,0xe1,0x97,0x11,0x30,0xe7,0xab,0x0b,0x43,0x78,0x7c,0x13, -0x4f,0x1e,0x12,0x98,0x64,0x19,0x96,0x05,0x21,0x92,0x05,0x65, -0x1a,0x4e,0x0c,0xb6,0x52,0xd0,0xf8,0xbe,0xc4,0x0c,0x09,0xb5, -0x93,0xe4,0xcf,0x64,0x7d,0xce,0x45,0xb3,0x22,0x62,0x67,0xc9, -0xff,0x81,0xe4,0x39,0xc5,0x88,0xe0,0x9d,0x65,0x5e,0xe1,0xe7, -0x1f,0xce,0x24,0xcb,0x85,0x1a,0xf1,0x1e,0x85,0xa9,0x4d,0xa4, -0x88,0x6d,0x4f,0x46,0xcd,0x69,0xb2,0xc4,0xaf,0x2c,0x68,0x78, -0x38,0x1f,0xa4,0xb0,0x3e,0x37,0xdf,0x1e,0xb2,0x96,0x57,0x6d, -0x7b,0x7a,0xea,0xc1,0x42,0xc4,0x29,0x40,0x59,0xfa,0x12,0x9d, -0xe7,0xdf,0xb6,0x9c,0x2a,0x78,0xf6,0x6d,0x13,0xc8,0x76,0x8b, -0x4b,0xa6,0x32,0x5a,0x41,0xaf,0x0d,0xc7,0xbb,0xee,0xf8,0x6d, -0xb8,0x5d,0x97,0xfb,0x36,0x1c,0x1e,0xcc,0x8e,0xc8,0x23,0x01, -0xea,0xa3,0x0d,0x45,0xb7,0x84,0xdf,0x68,0x47,0x89,0xe5,0x28, -0x59,0xfe,0x31,0xb9,0x89,0x1f,0x26,0x2b,0xe9,0x62,0xce,0x92, -0x40,0x72,0xee,0x6d,0x79,0x7f,0xab,0x6e,0xc9,0x37,0x64,0x88, -0xf9,0x53,0xb2,0x9c,0x2b,0x47,0x75,0x52,0x43,0xdd,0x47,0xf4, -0x95,0x25,0x79,0x7e,0x0b,0xfa,0x4a,0x2b,0x01,0x69,0xf4,0xbd, -0xf7,0x3d,0x7e,0x50,0xbd,0x09,0x7a,0xf7,0xfd,0x48,0x7b,0xb4, -0xdc,0x04,0x95,0xb5,0x7c,0x15,0xaf,0xe9,0x02,0xd0,0x8b,0xf1, -0xba,0xc2,0x54,0x32,0x26,0x77,0x15,0xf7,0xa0,0x23,0xdf,0x44, -0x94,0x25,0xbf,0x5e,0x9e,0xf1,0x9b,0x5a,0x83,0xa8,0xdb,0x19, -0x6f,0x2c,0xa1,0x52,0x8a,0x1c,0x74,0xf5,0x06,0xd4,0xe0,0x7d, -0x94,0x42,0xe1,0xb1,0x9e,0x9e,0xee,0x2b,0x89,0xf8,0x2e,0x19, -0x3d,0x0c,0x93,0x57,0x0f,0x53,0xa7,0x53,0x96,0x94,0xea,0xe5, -0x60,0x01,0x4a,0x04,0xd5,0x5c,0x43,0x24,0x0b,0x61,0xd0,0x2c, -0xea,0x05,0xff,0x5e,0x5a,0xb4,0x61,0xee,0xbe,0x5d,0xc6,0xd3, -0x3f,0xa4,0xb3,0x57,0x00,0x0b,0x5a,0xce,0xa3,0xee,0x11,0x34, -0x1a,0x32,0xf7,0x9c,0xa7,0x50,0xe4,0x0b,0x9c,0x9a,0x64,0x01, -0x3e,0xfa,0x4e,0xd2,0x89,0xa7,0x1d,0x18,0x32,0x0e,0xd5,0xc9, -0xfc,0x25,0xe6,0x97,0x5f,0xb7,0xea,0xf9,0x23,0xee,0x67,0x30, -0x5b,0x7e,0x47,0xd6,0xd8,0xed,0x83,0xa8,0x4e,0x19,0xb8,0xd7, -0x98,0x46,0x0c,0x44,0x5f,0x97,0xa1,0x27,0xd1,0x1b,0x96,0x80, -0x35,0x7d,0x33,0x7e,0xd3,0x5b,0xf7,0xa3,0xe9,0xe1,0xba,0x3e, -0x28,0xbf,0xa9,0xca,0x57,0xfc,0x1d,0x49,0xa6,0xdb,0xf4,0x1e, -0x8e,0x70,0x58,0x51,0xcb,0x54,0x0c,0xf8,0x83,0x6b,0x6d,0xf9, -0x5a,0xca,0x55,0xb4,0xbf,0x78,0x95,0xc0,0xb8,0x8d,0x4b,0x3e, -0x32,0x93,0x4b,0xbe,0x68,0x20,0x9f,0xba,0x61,0xbb,0x74,0xe9, -0x81,0x48,0xbe,0x44,0x93,0x36,0x5e,0x80,0xd2,0xb4,0xec,0x98, -0xaf,0xa8,0x37,0x7d,0xfd,0x24,0xdf,0x0f,0x0e,0xd4,0x93,0x3a, -0xc1,0xb7,0xac,0x57,0xda,0x54,0x94,0x90,0x37,0x57,0x77,0xcb, -0x24,0x1e,0x55,0xd5,0xfd,0x1d,0xd9,0x19,0x99,0xca,0x67,0x62, -0xdf,0xea,0x61,0x63,0x99,0xa3,0xc0,0x37,0x24,0xa0,0x0b,0x91, -0xfd,0x89,0x78,0x95,0xa0,0xe2,0x65,0x53,0x1f,0x8c,0xa7,0x03, -0x1c,0xdb,0xc9,0x0a,0x74,0x12,0xba,0x8b,0x0e,0xb3,0xc2,0xc1, -0x0c,0x31,0x73,0xd7,0x31,0x5c,0x01,0x63,0x88,0x4a,0x09,0xd1, -0xcd,0x80,0xb5,0x8e,0xa2,0x87,0xd6,0x4a,0x14,0x54,0x93,0xe8, -0xe4,0x3a,0x8d,0xb7,0xb8,0x8a,0x95,0xa8,0x66,0xf1,0x74,0x9e, -0xe5,0x4e,0xc9,0x9a,0xeb,0xc6,0xa0,0x5c,0x6b,0x2b,0x99,0x28, -0x94,0x01,0x78,0x98,0xc1,0x2a,0xcb,0xfc,0x31,0x49,0x16,0x49, -0xa1,0x1a,0x0e,0x2f,0x3b,0xe0,0xdf,0xaa,0xa2,0x64,0x47,0xc4, -0xae,0xe0,0xb8,0x17,0xf4,0x27,0x99,0xba,0xc0,0x1b,0x32,0x35, -0xa2,0xdc,0xc5,0x32,0x79,0x93,0xc2,0xea,0xf5,0x55,0x3a,0x80, -0x95,0xc1,0xed,0xef,0xdb,0x57,0x0a,0x39,0xad,0x18,0x38,0x3c, -0x9f,0xd3,0x7d,0x94,0x6c,0xf7,0x10,0x13,0xc6,0xc5,0x9d,0xc2, -0x7e,0x0e,0x17,0x81,0x74,0xb4,0x35,0x5d,0x69,0x53,0xb6,0x56, -0x33,0x8b,0x9e,0x5f,0x6d,0x51,0x21,0xe3,0x56,0x77,0xdb,0xc8, -0x34,0x90,0xd9,0x2b,0xae,0xc3,0xef,0x50,0xcf,0xd0,0x43,0x19, -0x67,0xa0,0x55,0x39,0x5e,0xc8,0xfb,0xf8,0x92,0x36,0xf2,0x28, -0x62,0x44,0x18,0x19,0x76,0x92,0x4e,0xeb,0x11,0xd5,0x23,0xd3, -0xa6,0xf3,0x91,0x4a,0xa0,0xbf,0x75,0x2e,0xcd,0x26,0xb8,0x75, -0x8f,0x49,0x51,0x89,0xfc,0x56,0xd9,0xb2,0xaf,0x60,0x57,0xe3, -0xa9,0x1d,0x28,0xdb,0xe5,0x11,0x1a,0xff,0x9d,0x4a,0x8c,0xd4, -0x83,0xa4,0x8b,0x06,0x84,0x19,0xed,0x1c,0xf1,0x92,0x5b,0x4f, -0x25,0x8f,0x3c,0xb6,0xc4,0xa3,0x73,0x95,0x86,0x2e,0xcc,0xef, -0x7e,0x47,0x27,0x18,0x32,0x79,0x46,0x60,0x76,0xe1,0x73,0xc3, -0x49,0x93,0x4d,0xe4,0xf6,0xf2,0xbe,0xdc,0xe1,0x09,0x25,0x19, -0x96,0x54,0x35,0x54,0xef,0x9b,0xdf,0xa4,0x53,0x6e,0xa7,0xa2, -0x44,0x0f,0x1f,0xc8,0x91,0x1a,0x4c,0x13,0xa9,0x34,0xbb,0x75, -0x1e,0xd0,0x2d,0x82,0xb5,0xd5,0x43,0x59,0xe5,0x0b,0x9c,0xbd, -0xf3,0x4b,0x54,0xb9,0x34,0x6d,0x0c,0xaa,0x97,0xa8,0x58,0xee, -0x25,0x1b,0xa4,0xea,0x23,0x68,0xbc,0x1b,0xa4,0x8c,0x6b,0xfc, -0x12,0xa3,0x63,0xbf,0x44,0x03,0x31,0x90,0xaf,0x28,0xe6,0x51, -0xac,0x5f,0x29,0x77,0x28,0x86,0x76,0xee,0xa0,0x92,0x52,0x3e, -0x6a,0xe1,0x1b,0xa7,0x5d,0xe8,0xab,0x49,0x96,0xb6,0xb6,0x99, -0x21,0x7b,0xb4,0x8e,0xa4,0x36,0xc3,0x8f,0x80,0x06,0xcd,0x0d, -0x3c,0x6e,0xc4,0x88,0x0c,0x0e,0xeb,0x83,0xe6,0x52,0x2a,0xd6, -0xe6,0xe5,0xe5,0xe5,0xe1,0x68,0x09,0xff,0x5d,0x59,0xdf,0x03, -0x55,0x52,0xca,0x83,0x58,0x9f,0xe6,0x28,0xa8,0x01,0x7f,0x61, -0x40,0x89,0xeb,0xaf,0xd2,0x99,0x31,0x3d,0xa1,0x4e,0x5c,0x37, -0xe8,0x67,0x29,0x30,0xcf,0x87,0xbf,0xb1,0x0c,0x4f,0x85,0x2b, -0x00,0x86,0xa8,0x6b,0x88,0x78,0xed,0x8b,0x4d,0x11,0xc7,0x46, -0xe1,0xd8,0x10,0x8e,0x4d,0x11,0xc7,0xa6,0xae,0x21,0x00,0x07, -0x7a,0x80,0xe3,0xcd,0x84,0xab,0x15,0x7c,0x52,0x92,0xf6,0x26, -0x5c,0x1a,0xbc,0xf2,0xfe,0x9c,0x9d,0x2d,0xa0,0xbd,0xde,0x18, -0x54,0x25,0xb5,0x25,0x3c,0x0a,0x98,0xe4,0x06,0x3e,0x6d,0x54, -0xd2,0x46,0x25,0x91,0xdb,0x1e,0xc2,0x07,0x6a,0xea,0xf3,0x78, -0x78,0xe7,0xb9,0x4c,0xc3,0xed,0x8e,0x2f,0x62,0xb9,0xbb,0xc7, -0xdb,0x90,0x26,0x14,0x24,0xbb,0x4f,0x58,0xfc,0x07,0x82,0x99, -0x17,0xc3,0x04,0x88,0x1e,0x51,0xbd,0x81,0x04,0x84,0x05,0x32, -0x74,0x20,0x9a,0x82,0xc2,0xaf,0x81,0x1b,0x20,0x5c,0x8b,0xb5, -0xba,0x3a,0x31,0x68,0xf5,0x85,0x5b,0xe5,0xb7,0x93,0x98,0x05, -0x4a,0x0c,0x7d,0x59,0x6e,0xe8,0x8b,0x0a,0x39,0x87,0x9c,0xd8, -0x16,0xb3,0x61,0x39,0xe0,0x10,0x24,0x4b,0x0c,0x9c,0xc1,0x00, -0x1a,0xf6,0x98,0x96,0xed,0xf4,0xb9,0x56,0x25,0x09,0xfa,0x5c, -0x43,0x3a,0x7b,0x43,0x49,0x65,0x3b,0x48,0xfc,0xba,0x38,0x0a, -0xe9,0xfc,0xec,0x3e,0xb2,0xc7,0x4e,0xe7,0xfe,0x05,0x2c,0xed, -0x3a,0xf7,0xd1,0xbd,0x5d,0x5d,0x16,0x42,0x3d,0x18,0xda,0xbd, -0x4a,0xfa,0xef,0xa1,0x31,0xfe,0x3b,0xae,0x20,0xec,0xf0,0x87, -0x5e,0xfc,0x39,0x25,0xa5,0x11,0xd3,0xc6,0x91,0xdd,0xda,0xce, -0x18,0x2a,0xba,0xb7,0x12,0x0e,0x0e,0x5e,0x54,0xa2,0x1f,0x03, -0xfa,0xce,0x38,0x1a,0x5b,0xe0,0xd0,0x00,0xa0,0xa9,0x43,0xe8, -0xbb,0x5e,0xd6,0xba,0xca,0xc2,0xa7,0x27,0x68,0x59,0x94,0xb5, -0x0e,0x0e,0xa0,0xef,0x3e,0x81,0x7e,0xee,0x56,0x0d,0xf7,0xe8, -0x9e,0xe2,0x22,0x94,0xe4,0x45,0x63,0xe4,0x76,0xda,0x68,0xf8, -0xed,0x42,0x6b,0x4d,0xb7,0x0d,0xa0,0xaf,0x54,0xef,0xfa,0x5b, -0x79,0x3e,0xe5,0xa1,0xd0,0xd6,0x69,0x1c,0x1d,0xb5,0xf0,0xac, -0xca,0x43,0x91,0xad,0x6f,0x54,0xc2,0x12,0x26,0x40,0xdb,0x20, -0x0e,0xfb,0xce,0x3a,0x7a,0x1a,0xc2,0x68,0x8d,0xf0,0x94,0x08, -0x46,0x5c,0x34,0xdc,0x08,0x15,0x5e,0x0d,0x87,0xd9,0x70,0x09, -0xc9,0xcb,0xba,0xb9,0x43,0x04,0xe0,0x43,0xf8,0x57,0x07,0x60, -0xf8,0xdd,0xf8,0x7e,0x87,0x87,0x20,0x86,0xe7,0xda,0x35,0x24, -0xfe,0x3c,0xa3,0xd3,0xf2,0xd2,0x23,0x75,0x35,0x64,0xa4,0x3a, -0xe1,0x59,0xcf,0xe8,0x44,0xeb,0x9d,0xa4,0xb5,0x6a,0x61,0x68, -0xaa,0x91,0x88,0x46,0xc9,0x24,0x59,0x25,0x1f,0xe5,0xf0,0x89, -0x62,0x32,0xa2,0xad,0xc2,0x68,0xae,0x28,0xd9,0x6c,0x5e,0x6c, -0xc4,0xf8,0xb9,0xd5,0x19,0x2b,0xb8,0x88,0x8e,0xb0,0xf8,0x99, -0xe7,0x4f,0x8c,0x40,0xad,0x92,0x36,0x9c,0xb4,0x3c,0xc4,0xc3, -0x2e,0xa5,0xa9,0xcc,0x27,0x7e,0xe4,0x9f,0x2d,0xbf,0x98,0x51, -0x9e,0xdc,0x2b,0x08,0xb4,0x57,0x70,0x4c,0x67,0x55,0xa3,0x6c, -0x6d,0xc1,0x73,0xc7,0x80,0x66,0x09,0xe8,0x6b,0x51,0x3a,0x8d, -0x60,0xab,0x46,0x03,0x3c,0x9d,0x5c,0x3f,0x3d,0x8d,0x36,0x52, -0x81,0x8e,0xd0,0xe2,0x41,0x96,0x1a,0x46,0xd6,0x6c,0xd2,0x19, -0xc5,0x87,0xd1,0x28,0x16,0xa3,0x01,0xfc,0x0c,0x3a,0xbc,0x01, -0x6d,0x9e,0xd6,0x01,0x47,0x63,0x14,0xfb,0x47,0x5e,0xeb,0x70, -0x04,0xd2,0xbc,0xb1,0x2e,0xa7,0xad,0xbb,0x18,0xc8,0x8c,0x0f, -0x01,0xb4,0x0e,0x30,0x0d,0x2c,0x12,0xc1,0xc3,0x21,0x15,0x85, -0x1c,0x5f,0x15,0xef,0x80,0x1c,0x02,0xd5,0xeb,0x3a,0xd5,0x8b, -0xd5,0xa2,0x08,0x43,0xca,0x06,0x53,0x36,0x0d,0x48,0x59,0xcb, -0xc8,0x04,0x12,0x70,0x34,0x50,0x10,0x65,0x8c,0x19,0x4e,0x1e, -0x32,0x18,0xf8,0x3f,0x6a,0x3e,0x5a,0x36,0x59,0x72,0xef,0xaa, -0x17,0x9a,0xe5,0x07,0x4f,0x2c,0x12,0x80,0xbd,0x94,0xf1,0x12, -0xff,0x0f,0x45,0xfc,0xba,0x9f,0xe5,0xa1,0x85,0xb9,0xaa,0x12, -0x2e,0x7e,0x18,0x77,0x54,0xc3,0x7a,0x2b,0x02,0xff,0x28,0x27, -0x1e,0x3b,0xea,0xfc,0x9d,0xb6,0xc3,0xe5,0x78,0xad,0x4d,0xb2, -0xdf,0xae,0x1a,0x9c,0xb2,0xc0,0xef,0xbb,0x37,0xae,0x40,0xef, -0x6c,0x99,0xab,0x49,0xc2,0xc5,0xe5,0x34,0x5e,0x7c,0x1b,0x8f, -0xbe,0x86,0x5d,0x89,0x24,0x4a,0x7f,0xdd,0x2a,0xc7,0xed,0xa6, -0x2d,0x47,0xeb,0x48,0x26,0x8d,0x70,0x58,0xc8,0xc7,0x4d,0xa9, -0x3a,0x33,0x78,0xe5,0x7e,0x28,0x1e,0x8d,0xf4,0x96,0x6e,0x1d, -0xc9,0xd5,0x8f,0x4c,0xc5,0xaf,0x4b,0x36,0x91,0x5c,0xcd,0xa8, -0xb4,0x80,0xfc,0xfc,0xc9,0xea,0x1a,0x2a,0x15,0xdd,0xaf,0x99, -0x42,0x23,0x59,0x0a,0x04,0xdd,0x14,0xd3,0x8f,0x2d,0x6d,0x8c, -0x1b,0xc0,0x2a,0x0f,0xb8,0x07,0xba,0x68,0xb4,0x86,0xc9,0x00, -0xf0,0x06,0xbe,0x80,0x44,0xfc,0xa6,0x0d,0xb4,0xd1,0x68,0x83, -0x89,0x1b,0x4c,0x44,0xbf,0x09,0xe8,0x34,0x01,0x9b,0xca,0xad, -0xcc,0x37,0x90,0xee,0x90,0x65,0x0c,0x8f,0xd1,0x7c,0x1a,0xa7, -0x33,0x15,0x38,0x13,0x6f,0x22,0x39,0x05,0x89,0xcf,0xf0,0xdb, -0x38,0xf9,0xca,0x3f,0x05,0x17,0x47,0x1f,0x21,0xcc,0x15,0x15, -0xec,0xf2,0xed,0x26,0x95,0xea,0xb7,0x7b,0xf8,0x23,0xf8,0xbe, -0xb3,0x58,0x39,0x9f,0x33,0xd1,0xa3,0xb1,0x72,0xa4,0xa8,0x2e, -0x74,0x98,0xc5,0xb4,0x75,0x0b,0x29,0x1e,0xcd,0xba,0x36,0xf9, -0x16,0x88,0x97,0xf7,0x82,0x87,0x59,0xd7,0x83,0x74,0xc2,0x8e, -0xc0,0xb8,0x01,0x82,0x4f,0xd4,0x9c,0x8b,0x67,0x3b,0x84,0x04, -0x33,0xe2,0x21,0x72,0x00,0x3c,0xc3,0x12,0xf9,0x84,0xfe,0xd9, -0x52,0x3b,0xd6,0x84,0xa7,0x0f,0xe4,0xd4,0x69,0xdc,0xae,0x60, -0x91,0xa9,0xf7,0x00,0x6f,0xf9,0x68,0x8c,0x44,0xef,0x0c,0x57, -0x11,0x92,0x56,0x7c,0x56,0x7b,0x02,0x91,0xe2,0x17,0x28,0x69, -0x18,0xe5,0x3a,0x02,0x3d,0x4d,0x4a,0xa2,0x52,0xfa,0x36,0x4b, -0xbf,0x99,0x3e,0x5a,0x07,0x57,0xeb,0x90,0x94,0x7e,0x94,0x06, -0x88,0x08,0x5d,0x95,0x01,0xe0,0x1a,0xc5,0x88,0xfc,0xf0,0x46, -0xe8,0x97,0x0b,0x53,0x7c,0x61,0x90,0x45,0x58,0x7f,0xf3,0x66, -0x32,0xc7,0x80,0x24,0x81,0xc9,0x09,0x65,0x0e,0x9d,0x75,0x52, -0x08,0x45,0xca,0xa8,0x68,0x0c,0xc5,0x2d,0xa0,0xf8,0xa4,0xba, -0xa3,0xe1,0xa5,0xfb,0x8e,0xf0,0xb6,0xab,0x6e,0xf3,0x65,0xad, -0x47,0x54,0x90,0x42,0xa2,0x6e,0x05,0xd6,0x57,0x59,0x80,0x89, -0xb1,0xe1,0xb7,0x6d,0x87,0x8e,0x97,0x32,0x08,0x74,0x09,0x99, -0x68,0x06,0xfd,0x41,0x52,0x42,0xee,0x6f,0xe1,0x0f,0x9e,0x90, -0x8f,0xcd,0xb6,0xc7,0xe9,0x22,0xc6,0xa5,0x74,0x5d,0x83,0xd7, -0xcc,0x92,0x95,0xe3,0xfe,0x27,0x5a,0xb4,0xb0,0x7f,0xe8,0x59, -0xf5,0x30,0x3b,0xda,0xf7,0x9b,0xca,0xf2,0xdd,0x67,0x4c,0x11, -0xe3,0x2b,0xe4,0xfa,0x74,0xec,0x1f,0x8d,0x3b,0xfe,0x03,0xdf, -0x97,0x97,0x8a,0xf0,0x7d,0x23,0xd4,0x42,0x7c,0x8f,0xde,0xf9, -0x52,0x06,0x2e,0x8a,0x32,0x41,0xca,0xc7,0xbe,0x5f,0x2a,0xcd, -0x7c,0x0f,0x01,0x73,0xaa,0xfc,0x56,0x56,0x32,0x6f,0x2d,0x42, -0x58,0xb3,0x34,0x42,0x6d,0x53,0x0a,0x08,0xbc,0x07,0xfc,0x53, -0x21,0xf3,0x65,0xbc,0xb7,0x39,0x4f,0xdf,0x10,0xd8,0xf1,0xa8, -0x13,0x19,0xed,0x88,0x7d,0x84,0x53,0xe9,0xbd,0xb8,0x7f,0xd5, -0xea,0x96,0x74,0x7a,0xbb,0xa0,0x2c,0xdc,0x9e,0x8f,0xa8,0xfe, -0x6e,0xa9,0xbd,0x4a,0xbb,0xdc,0x42,0x46,0x35,0x1d,0xe6,0x91, -0xc5,0x03,0x59,0x93,0xbc,0x87,0x70,0xc1,0x8b,0x55,0x88,0xa1, -0x65,0x51,0xb7,0x90,0x6b,0x56,0x24,0x63,0x6e,0x19,0xfe,0x32, -0x93,0xcc,0x38,0x61,0x62,0x20,0x81,0x36,0x1c,0xfc,0xc6,0x75, -0x75,0x1c,0x15,0x9c,0x8f,0x65,0x66,0x6e,0x4f,0x09,0x18,0xde, -0xb7,0x5c,0x55,0x53,0x0a,0x6c,0xc9,0x98,0x2c,0x09,0x69,0xc3, -0xb0,0x1c,0xa0,0x96,0xb9,0xe4,0x0b,0xdd,0x93,0xa0,0xd8,0x59, -0x89,0x08,0x4b,0xa3,0xef,0x87,0x98,0x41,0xd7,0x36,0x2e,0x4a, -0x72,0x50,0x91,0x9d,0x6d,0x19,0x3e,0x7b,0x02,0x59,0xfb,0x4d, -0x67,0xc8,0x14,0x2d,0x75,0x75,0xcb,0x49,0x24,0xf6,0xa3,0x8f, -0x41,0x1d,0xfa,0x28,0x49,0xa1,0x4a,0xcb,0xa2,0xf9,0xec,0x42, -0x68,0x17,0x70,0xd0,0x5a,0x19,0x0a,0xf9,0x2a,0x1d,0x8e,0x33, -0x83,0x76,0xea,0x04,0xa0,0xb1,0x06,0xdf,0xf7,0x08,0xa7,0x04, -0x74,0xea,0xdb,0xc5,0xbf,0x40,0x5b,0x54,0x4b,0x1c,0xa6,0x82, -0x82,0x40,0xae,0x76,0xa1,0xe2,0x32,0x1a,0x9f,0x2a,0xa1,0xd0, -0xa2,0x3e,0xde,0x83,0xa8,0xaf,0xad,0xb9,0x70,0x6a,0x09,0x8b, -0xee,0x84,0xf9,0xc2,0x0a,0xc7,0x54,0x85,0xe5,0x39,0xbd,0xb2, -0xb5,0x10,0x57,0xe9,0x26,0x69,0x56,0xc0,0x56,0xa5,0x9c,0x54, -0x62,0x52,0x60,0x67,0x8b,0x1a,0xd5,0x57,0x93,0xbf,0x24,0x3d, -0xf0,0x62,0x55,0x8f,0xa6,0x06,0x48,0x40,0xde,0xff,0x74,0x25, -0x03,0x0a,0xed,0xb3,0xd7,0x0a,0xc5,0x39,0xb7,0xa4,0x5b,0x78, -0xd9,0xa5,0x11,0xc2,0xf2,0xb2,0x74,0xcd,0x54,0x59,0xe0,0x1d, -0x7f,0xe2,0x37,0x85,0xb9,0x6a,0x1a,0x85,0x01,0x9f,0x0a,0x25, -0x1c,0xd8,0xae,0x7c,0x5d,0x29,0xb2,0x45,0x3c,0x8b,0x18,0x04, -0x57,0xbe,0xf2,0x89,0x16,0x97,0xc9,0x22,0xd2,0x5f,0xc8,0x84, -0xd2,0x74,0x8f,0xe7,0x7d,0x7d,0x81,0x8a,0xa5,0x8f,0xa6,0xfa, -0xe6,0x34,0xc4,0x6f,0x4f,0x93,0xe5,0x32,0x9a,0x1e,0x61,0x0e, -0xcd,0xf3,0xea,0x38,0x34,0x3c,0xbd,0x8a,0x20,0xab,0x8b,0x69, -0x87,0x40,0x5c,0xbb,0x79,0xec,0xa4,0x9c,0xb6,0xf1,0xa3,0x15, -0x4c,0x40,0xcf,0xb8,0x94,0xd4,0xf2,0x85,0xa6,0x27,0x32,0xab, -0x08,0x9d,0x66,0xad,0x26,0x84,0x6e,0x42,0x64,0xd1,0xa9,0x13, -0x2d,0xc8,0xba,0x0c,0xe1,0xae,0x4a,0xb4,0xfa,0x91,0xf5,0xba, -0x93,0xdb,0x59,0x59,0x5f,0xb3,0x6e,0x92,0xa1,0x01,0x47,0xc7, -0xe2,0xf9,0x5e,0xda,0xdd,0xa5,0x15,0x43,0x93,0xa6,0x52,0x1d, -0x2a,0x3f,0x6a,0x94,0xf5,0xc6,0x7e,0x12,0x65,0x3a,0xab,0xde, -0x0c,0x42,0xcb,0xab,0x78,0x93,0xab,0xf2,0xf8,0xa7,0xcb,0x3f, -0x30,0x76,0x16,0x74,0x8a,0x01,0xc8,0x39,0xe5,0x7a,0x99,0x98, -0x7d,0xb9,0xe5,0x42,0x57,0xdc,0x8a,0x3b,0x3c,0xdf,0xd5,0xf7, -0xd2,0xb9,0x9c,0xa7,0xa7,0x5a,0x13,0xa3,0x80,0xa8,0x36,0x34, -0x5a,0x87,0x5e,0xed,0x63,0x74,0xfa,0x75,0x0f,0x52,0x73,0xdf, -0x37,0xf6,0xad,0x7e,0xbb,0x26,0x00,0x54,0x43,0xd6,0x6b,0x37, -0xe5,0x83,0x10,0xda,0x2c,0x47,0xf3,0x20,0xce,0x60,0x07,0x88, -0xae,0x39,0xd2,0x37,0x6a,0x36,0xb6,0x23,0x09,0x02,0xa4,0x31, -0x75,0x55,0x70,0x5d,0xcd,0x39,0x58,0x80,0x77,0x83,0xf6,0xda, -0x6f,0x37,0x74,0x12,0xba,0x51,0x0c,0xda,0x8d,0xb5,0x1d,0x43, -0x16,0x6b,0xb1,0x08,0xc1,0x81,0xb1,0xb6,0x7c,0x1d,0xdb,0x58, -0x31,0x8f,0x88,0xd2,0x48,0x75,0x0a,0xe0,0xac,0x5e,0x04,0x48, -0x9d,0xc8,0x04,0xfb,0xdb,0x9d,0x13,0xbf,0x5d,0xf7,0xa3,0xe4, -0x84,0x04,0xa2,0xc2,0x1f,0xb0,0x0a,0x50,0x8d,0x88,0xd6,0x30, -0xbe,0xae,0xc8,0xad,0x18,0x61,0x65,0x1c,0x5e,0xe9,0x2a,0x81, -0x6d,0x84,0x80,0x62,0x5f,0x2e,0x71,0xf2,0x8b,0x05,0x6c,0xb5, -0xa9,0x7e,0x57,0x24,0x39,0x82,0xac,0x5f,0xe7,0x6a,0x95,0x2b, -0xc6,0x42,0x2d,0x08,0x5d,0x3e,0x65,0xc9,0x53,0x0f,0x48,0x1b, -0x45,0x65,0xea,0x58,0xe3,0x12,0x4e,0xaf,0xb5,0x6d,0xd1,0x42, -0xe5,0xfe,0x75,0x72,0x1b,0x63,0xae,0x1e,0x25,0x2e,0x61,0x54, -0x81,0xda,0x5f,0xd1,0xa6,0x6b,0x44,0x98,0xa1,0x3b,0xd4,0x6a, -0xaf,0x7c,0x9e,0x97,0xae,0xbf,0x76,0x6a,0x66,0x2e,0x02,0xbb, -0x97,0x87,0xc8,0x68,0xe5,0x37,0x46,0x57,0xc3,0xd6,0xc6,0x52, -0x02,0x28,0x30,0x0f,0x38,0x8e,0x2c,0x45,0x49,0x26,0x2e,0xf8, -0x41,0x78,0x84,0x9c,0xfa,0x38,0xec,0xb6,0x88,0x65,0x78,0xc8, -0x98,0x66,0xec,0x20,0xdd,0xbb,0x6f,0xa4,0xbe,0x8c,0x35,0xc2, -0x8c,0xe0,0x3b,0xe1,0xce,0xbd,0x3c,0xc0,0x36,0x76,0xcd,0xf8, -0x55,0xfc,0xb8,0x33,0x86,0x34,0xa2,0x4c,0xc5,0x5d,0x79,0xf4, -0x52,0xff,0x70,0xec,0x77,0x0a,0x89,0xfe,0x56,0x7f,0x7a,0x5f, -0xcc,0x43,0xdc,0xe8,0x6e,0xdd,0xa0,0x9f,0xc1,0x86,0x64,0x7c, -0x15,0x74,0xc6,0x8d,0x46,0x79,0x05,0xea,0x68,0x9d,0x32,0x7b, -0x69,0xff,0x93,0x07,0x22,0x50,0xde,0x71,0x70,0x11,0x79,0x80, -0xcb,0x20,0xb8,0x43,0xba,0x7a,0xd3,0xb9,0x07,0x84,0x9c,0x22, -0x81,0x32,0xf9,0x21,0xff,0xbd,0x1e,0x5e,0x94,0xbe,0x6b,0x4d, -0x35,0xd3,0x6a,0x18,0x5d,0xbb,0xe4,0xe5,0xd6,0x2f,0x2c,0x6c, -0xe6,0xb7,0x8c,0xa1,0x53,0xfc,0x90,0x55,0xea,0xd5,0xa8,0x08, -0xdc,0x36,0x71,0x1f,0x5f,0xa8,0xb8,0x8f,0x0c,0x0c,0x33,0xa2, -0x2e,0xe5,0x6a,0x69,0x5f,0xce,0xf1,0x7a,0x7b,0x1b,0xaf,0xbd, -0x66,0x28,0x66,0x47,0x96,0xd8,0xc1,0x2e,0x54,0xd2,0x29,0x6e, -0x22,0x2d,0xef,0x5e,0x12,0xa1,0x83,0xd3,0x96,0xb1,0x8b,0x42, -0x3f,0xa2,0x51,0xc3,0x4a,0x24,0xc9,0x2a,0xee,0x2c,0x2d,0xf3, -0xaf,0x23,0xec,0x9e,0x1b,0x12,0xb4,0x91,0x5f,0x4f,0x7c,0x1f, -0x36,0xb8,0xb2,0x81,0x90,0xd0,0xc6,0x0f,0xeb,0xf6,0x5d,0x0b, -0x6a,0x5d,0x4e,0xa0,0x68,0x42,0x5d,0xa6,0xd2,0xb7,0xf9,0xf9, -0xb5,0x6c,0xe5,0x57,0x7a,0x84,0xa0,0x14,0x24,0x4c,0xf4,0x8b, -0xf9,0x0c,0x0d,0x16,0x0b,0xb3,0x44,0x85,0x2e,0x86,0xb2,0x0b, -0xd2,0xa7,0x34,0x05,0xc3,0x4b,0x64,0xa3,0xfd,0x16,0x30,0x2b, -0x94,0xa8,0x4e,0x1e,0x07,0x85,0xec,0xf0,0x48,0x03,0xec,0xde, -0xc9,0x39,0x0a,0x7d,0xf0,0xfb,0x68,0xf4,0xbd,0x15,0x38,0xb6, -0xac,0x4a,0x83,0xff,0x9f,0xb5,0x4d,0xb1,0x99,0x94,0x5f,0x14, -0xb9,0x4b,0x76,0x8d,0x4a,0x75,0xd0,0xbe,0xb3,0xe4,0x0e,0x21, -0x88,0xd6,0xcf,0x8b,0x41,0xf5,0x04,0xe7,0xf6,0x82,0x2a,0xb0, -0xef,0x28,0xb2,0xd6,0x01,0x72,0x14,0xe5,0xa5,0xfd,0x37,0x0c, -0x20,0x6a,0x64,0xd1,0x19,0xc1,0xda,0x72,0x14,0xb1,0xee,0x9a, -0xb5,0x4e,0x63,0x2d,0x12,0xf9,0xd1,0x27,0x2d,0x94,0x0a,0x0e, -0x0a,0x18,0xf1,0x5c,0x06,0x56,0xb4,0x77,0x81,0xcb,0x1d,0x83, -0x92,0x8f,0xbc,0x3c,0x8f,0x0c,0xdc,0x29,0xc2,0xed,0xda,0x87, -0xa5,0x25,0xef,0xe6,0x60,0x31,0xc9,0x08,0x9a,0xa0,0x33,0x19, -0x22,0x43,0x08,0xd9,0x7a,0x9e,0x52,0x60,0x20,0xf9,0x8d,0xd0, -0xff,0xd8,0x3e,0x7a,0xb2,0xce,0xc0,0x71,0x37,0x90,0x79,0xea, -0xac,0xdc,0x3a,0x17,0x35,0x96,0xc1,0xf6,0x09,0x56,0xce,0x42, -0x3a,0xb5,0x8e,0x51,0x01,0x43,0x9d,0x76,0x32,0xe9,0x96,0x95, -0x07,0x51,0x24,0x37,0xba,0xf4,0xf7,0x0f,0xf1,0x6c,0xd4,0x29, -0x91,0x5a,0x47,0x0c,0x77,0x4e,0x39,0x04,0xde,0x91,0xa5,0xc8, -0xb1,0x2c,0xba,0x32,0x9a,0x25,0x8f,0xc8,0xe1,0xaf,0xe2,0x85, -0xfe,0x84,0x66,0xad,0xc3,0x51,0xad,0x73,0xf7,0x9c,0xcc,0xa7, -0xbb,0x38,0xf3,0xd6,0x29,0xb9,0x0c,0x01,0x7e,0x5a,0xbc,0x4b, -0x5d,0xe6,0xa5,0xbe,0xab,0xc4,0x7a,0x8a,0xe1,0x7d,0xb9,0x07, -0x62,0x49,0x92,0xa9,0xb1,0xff,0x5b,0x4e,0x8c,0x14,0x8b,0x60, -0x1d,0xca,0xf8,0xa2,0x77,0x30,0x4b,0xca,0x8d,0x7b,0xdc,0xd6, -0x25,0xb7,0x6a,0xbc,0x14,0x0f,0x96,0xbe,0xe5,0x2f,0xbb,0xac, -0xab,0x3e,0x7d,0x4f,0x54,0xee,0x64,0x82,0x73,0x23,0xb9,0x79, -0xe6,0x2b,0x97,0xb5,0xbe,0x6d,0x59,0x93,0x8b,0x34,0xdc,0x1a, -0x7b,0xf8,0xde,0xe0,0xaf,0xbf,0x8e,0xcc,0xb5,0x6a,0x28,0x72, -0x17,0x00,0xbe,0xba,0x6e,0xd2,0x6c,0xe3,0xe6,0xb1,0x98,0x39, -0xc0,0xb0,0x38,0xf0,0x94,0xd4,0xcc,0x17,0xfe,0x51,0xab,0x6d, -0xc9,0x90,0xc4,0x72,0xd4,0x62,0xa1,0xdc,0xc9,0x1c,0x6e,0x74, -0x19,0x8b,0x1c,0xde,0x60,0xe9,0x32,0xd6,0x88,0xf9,0xc3,0x2a, -0x59,0x7e,0xbb,0x27,0x9f,0x0a,0x0e,0x39,0x3e,0x39,0x86,0x7c, -0x1b,0x45,0xa4,0x39,0x40,0x1f,0xbe,0xf3,0x89,0x38,0xb1,0xf2, -0x13,0xc9,0x59,0x66,0xb2,0xcc,0x69,0x58,0xcc,0x6e,0xc8,0xb4, -0x52,0xa6,0xbb,0x9c,0x96,0x95,0xd4,0x5b,0x87,0x0e,0xf9,0x65, -0x5c,0xb7,0xd8,0x6a,0xc3,0x2a,0xce,0x72,0x95,0x07,0x07,0xac, -0x20,0x72,0xe7,0xfb,0xc4,0x72,0x2a,0xea,0x85,0xaa,0x4e,0xbf, -0xd0,0x03,0xc4,0xda,0x67,0x3b,0x80,0x4e,0x9d,0xfe,0xcf,0xeb, -0x05,0x6b,0xdb,0xf1,0x01,0x1d,0x42,0x27,0x44,0xf3,0x65,0x64, -0x4a,0x36,0xca,0x0b,0xfa,0xf6,0xf9,0x51,0x49,0xff,0x59,0x1e, -0x17,0x08,0xe3,0x11,0xba,0x53,0xdc,0xb7,0x07,0xad,0xc2,0xf9, -0xce,0x2c,0xf6,0xa6,0xe9,0xa7,0xbd,0xc6,0x54,0xc9,0x0c,0xad, -0xf3,0x1c,0x60,0xde,0xf1,0xed,0x98,0xd0,0xe5,0x96,0x50,0x29, -0x53,0xfe,0x6c,0x79,0xbf,0xa5,0x40,0xf9,0xbc,0xea,0xae,0x20, -0xd5,0x5e,0xb3,0x64,0x5a,0xfe,0xf9,0x21,0x9e,0xad,0x30,0x70, -0x88,0x5d,0xbe,0xf2,0x8e,0x67,0x0c,0x4a,0xe8,0xe7,0xc8,0x9e, -0x50,0x69,0xde,0x41,0xdb,0xf9,0xec,0x6e,0x3e,0x19,0x65,0xfc, -0xd1,0xc3,0xf8,0x93,0x9f,0x3b,0xbe,0x49,0xec,0x8d,0x61,0x87, -0x86,0x5b,0x9a,0x7c,0x65,0xe3,0xa3,0x9f,0xdd,0x49,0xa6,0xfa, -0xbc,0x84,0xfd,0x9a,0xac,0xa3,0xfa,0xda,0xef,0xbe,0x99,0xa7, -0xa3,0x8f,0x02,0x9e,0x04,0x7a,0xe6,0xda,0xcb,0x54,0x28,0xd6, -0x7e,0x9f,0x26,0x62,0x93,0xf4,0xec,0xec,0xfb,0xfc,0x85,0xc9, -0x4d,0x3a,0x41,0xeb,0xf3,0xe2,0xfe,0xe8,0x05,0x13,0x37,0xf2, -0x55,0x58,0x1f,0xa0,0x29,0xce,0x86,0xc9,0x4c,0xea,0x8c,0x7f, -0xc2,0xe5,0x8a,0xe2,0x6c,0x56,0x22,0x2a,0x86,0x09,0xe6,0xaa, -0x03,0x77,0x1a,0x79,0xa9,0x34,0xf7,0x4b,0x1b,0xd9,0xc5,0xb4, -0x3a,0xf8,0xe6,0x06,0x72,0x44,0x70,0xb5,0xe9,0xf6,0x82,0xa3, -0x40,0xc0,0xbf,0x7e,0xbb,0x87,0xc1,0x46,0xac,0x0e,0xde,0xe0, -0xc7,0x14,0xe6,0x76,0x7e,0xf3,0x89,0xc9,0x53,0x4d,0xb0,0xa1, -0x35,0x6c,0xde,0x9a,0xa1,0xbf,0xaf,0xd4,0x97,0x8b,0xed,0xee, -0x2b,0x04,0x2e,0xf3,0x16,0xa4,0x2a,0x40,0xff,0xa5,0x79,0x49, -0x2f,0x5f,0x80,0x96,0x7d,0xd3,0x9e,0xda,0x67,0xe4,0xe3,0xf5, -0xa1,0xb7,0x6e,0xac,0x03,0x58,0x6c,0xda,0x1f,0xe1,0x98,0x71, -0xa3,0x1c,0x82,0xac,0x9d,0x71,0x73,0x44,0x61,0xb3,0x02,0x34, -0x5f,0xb0,0x93,0x61,0xe1,0xc6,0x03,0x81,0x46,0xdb,0x5a,0xa4, -0x1f,0x2e,0xb7,0xeb,0x00,0x06,0x0b,0xfb,0x89,0xc5,0x87,0xb5, -0xe1,0xb2,0x2d,0x45,0x14,0x02,0x38,0xec,0xff,0x03,0x44,0xf2, -0x03,0x84,0x6c,0x13,0xa1,0x98,0x81,0x84,0xb5,0x37,0x47,0xe3, -0x75,0x1d,0x48,0xeb,0x6d,0xc4,0xa6,0x1e,0xc2,0xcb,0x7b,0x4a, -0x46,0xae,0x97,0x77,0x0b,0x86,0x16,0xcd,0x2a,0x25,0x98,0x17, -0x8d,0xf5,0x55,0xb4,0xee,0xe6,0xd5,0x8f,0xba,0x75,0xf7,0xfb, -0x6d,0x56,0x50,0xdb,0x9d,0x5d,0xb7,0xeb,0x78,0x54,0xc2,0x5e, -0x97,0x6f,0xd2,0x73,0x7d,0xb4,0x0b,0x11,0x83,0x5e,0x97,0x2e, -0x9b,0x3f,0xa0,0x7f,0xe4,0x78,0xa5,0xe1,0xae,0xac,0x17,0x36, -0xfb,0x0f,0xda,0x0a,0x3e,0x6f,0xf3,0x17,0xe6,0xba,0x40,0x2a, -0x6d,0x45,0xbc,0xc2,0x11,0x8e,0xce,0xd1,0x9d,0x52,0x5f,0x2b, -0x7e,0xab,0x3c,0x75,0xec,0xa2,0xde,0x85,0xce,0x90,0xfc,0xd5, -0xef,0xef,0x21,0xf5,0xce,0xb9,0x8a,0x42,0xe0,0x6b,0xd4,0xa6, -0xaf,0x34,0xf2,0x0f,0x3e,0x53,0x71,0x30,0xfc,0xae,0xc7,0x2a, -0x1a,0xf3,0x73,0x3d,0x96,0xef,0x00,0x53,0xd4,0xed,0xad,0x37, -0xb7,0xd7,0xf1,0x72,0xf8,0x72,0x36,0x4b,0x96,0xdf,0xc5,0xa3, -0xf4,0x21,0x73,0xbe,0x1d,0x4d,0x4d,0x7a,0x59,0xb1,0x6f,0x70, -0xd5,0x58,0x52,0x6c,0x6e,0xd2,0xcb,0x8a,0x61,0xb8,0x86,0xd5, -0xa7,0xb3,0xdb,0x89,0xfb,0xe9,0x75,0xa6,0x93,0xcb,0x0a,0x7d, -0x3e,0x1b,0x15,0x8b,0x24,0x32,0xb1,0x50,0x00,0x59,0x89,0x04, -0xc0,0xf2,0xaa,0x10,0xfa,0x19,0xdf,0xc5,0x52,0xc4,0xa5,0xc1, -0x9f,0x75,0x24,0x6b,0x4c,0xb4,0xe2,0x05,0x2f,0xf9,0x99,0xbe, -0x61,0x90,0x8f,0xa1,0x8e,0x84,0x89,0xac,0xa0,0xcf,0x8c,0x85, -0x02,0x8a,0x96,0x87,0xda,0x91,0x63,0x2c,0xbd,0x97,0xe0,0xd5, -0xe7,0xd2,0x38,0x30,0x8c,0x73,0xa1,0xc9,0x4b,0x1a,0x9d,0xc4, -0xde,0x62,0x39,0xc7,0xef,0xdf,0xd3,0xb9,0x3d,0x86,0x28,0x47, -0x7e,0xcd,0xac,0x35,0x5d,0x72,0x3b,0x25,0x03,0xbf,0x77,0xf2, -0x49,0xf9,0x37,0xfa,0xaa,0x66,0x5f,0xd9,0x5b,0x08,0x25,0x67, -0xd0,0x3f,0x98,0x8c,0x64,0x2c,0xbe,0xb4,0x41,0xbf,0x93,0x8b, -0xf2,0x42,0x89,0xc0,0xb2,0x99,0x32,0x65,0x6b,0x3f,0xd5,0xd8, -0x55,0x3d,0x47,0x7f,0x45,0xbb,0x6b,0x45,0x49,0xa4,0xdc,0xb7, -0x67,0x81,0x79,0x0c,0xad,0xd8,0xe7,0xd6,0x97,0xdc,0xe2,0x66, -0x4d,0x6e,0x96,0xa9,0x61,0xd2,0x6e,0xee,0x66,0xe3,0x24,0x6d, -0x30,0x09,0xe6,0xc6,0x35,0xc6,0x8e,0x59,0x87,0xdd,0xe2,0x30, -0x58,0x93,0xe9,0x9a,0x46,0x11,0x22,0x8a,0x30,0xda,0x20,0xfc, -0xa6,0x0c,0x7e,0xe3,0xc0,0x6f,0x64,0x00,0x5d,0xb4,0x30,0x4f, -0x6e,0x40,0x92,0x46,0x4d,0x18,0x53,0x13,0x0f,0xbd,0x99,0x89, -0x11,0x7f,0x31,0x0e,0x22,0x91,0xfa,0x5d,0xcd,0x10,0x8e,0xcc, -0x0a,0x8b,0x5c,0xa0,0xde,0x06,0x06,0x20,0x98,0x1b,0xeb,0xd0, -0x80,0x5c,0x6a,0x5f,0xb9,0xb4,0x09,0x65,0x59,0x28,0x19,0xe6, -0x4b,0x42,0xb9,0x7c,0x5a,0xdf,0x57,0x91,0x58,0x03,0xed,0x63, -0xda,0xd3,0x7d,0x5f,0xce,0xe5,0x9c,0xa0,0x99,0x82,0xa6,0x9c, -0x96,0x19,0xe5,0xb1,0x53,0xbd,0xab,0x2b,0x5f,0x76,0xb5,0x4f, -0xc7,0x54,0x81,0x1d,0x80,0xf7,0x07,0xb2,0x7a,0xb4,0xdf,0x37, -0x68,0x66,0xb9,0x71,0x12,0x7f,0x14,0x92,0x8f,0xec,0xff,0xee, -0xc1,0x31,0x22,0xb1,0x01,0x39,0xe4,0xac,0x9d,0xfb,0xdf,0x93, -0x8d,0x6d,0x25,0xda,0x1c,0x27,0x9b,0x12,0x19,0xb5,0x41,0xc4, -0x97,0x11,0x06,0xf5,0x97,0x72,0x19,0x35,0xcf,0x3b,0x7a,0x4a, -0x48,0xf0,0xf3,0xa3,0xbd,0xa6,0x5f,0x32,0xe8,0x84,0xe9,0x17, -0xcb,0xf8,0xed,0x75,0xb8,0x15,0x5c,0x3a,0xd8,0xbb,0xb8,0x2e, -0x1b,0xa8,0xb2,0xe1,0x9e,0x65,0x4b,0xea,0xdd,0xec,0x57,0x14, -0xfd,0x2a,0x98,0xd2,0x1b,0x5d,0x7a,0x4f,0xaa,0x37,0x86,0xea, -0x8d,0xa2,0x7a,0xb3,0x27,0xd5,0x65,0xf5,0xaa,0x4e,0xdf,0x6f, -0xe5,0x24,0x81,0x15,0x16,0xf9,0x2e,0x51,0x95,0x9a,0x78,0xed, -0x42,0x97,0x13,0x21,0x73,0x71,0x17,0xa9,0x8b,0xbb,0xeb,0xae, -0x8d,0xf4,0xba,0xed,0x55,0xc8,0xe4,0x4b,0xc7,0xbb,0x35,0x1e, -0x67,0x5f,0xfb,0x4f,0x4f,0x05,0xa1,0xf5,0xf7,0x10,0x4d,0xa5, -0x37,0xf1,0xf4,0xd6,0x96,0x57,0x1b,0x64,0x48,0x51,0x74,0xbb, -0xa0,0xb6,0xdb,0x28,0xc6,0xcc,0x0c,0xb7,0x35,0x92,0x27,0x4a, -0xc2,0xf7,0xe2,0x87,0x02,0x56,0xec,0x95,0xef,0x8c,0xaa,0x30, -0xf5,0x0c,0xef,0xe6,0xcb,0x51,0xc9,0xc4,0xbe,0x2c,0x9f,0xd3, -0x61,0x9e,0xbd,0x9d,0xcf,0xe2,0xc9,0xb7,0x66,0xa6,0x18,0x19, -0xe7,0x07,0xe4,0xaf,0x68,0x53,0x8c,0x57,0xaf,0x4a,0xf1,0x3c, -0x6d,0x95,0xb5,0xe7,0xbc,0xfc,0x5d,0x86,0xb4,0xe6,0x8e,0x0c, -0x8c,0x3c,0xd4,0x26,0x0d,0xa9,0xdb,0x0d,0x7b,0x77,0x0a,0x97, -0x0c,0x93,0xf5,0xa8,0x6c,0xa2,0x96,0x3a,0xa1,0xe7,0x4e,0xd3, -0xce,0xec,0xdc,0xdf,0x16,0x28,0xce,0x36,0xd3,0xc1,0x7c,0xf2, -0x0a,0xf7,0x28,0x9a,0xb2,0xb3,0x93,0x0a,0xb0,0xef,0x41,0xcc, -0x34,0x58,0x8d,0xdd,0x2f,0xd7,0x2a,0x60,0x3f,0x63,0xe7,0xcc, -0x19,0x60,0x96,0xbe,0x47,0xad,0x0f,0x82,0x30,0xf5,0xe8,0xd7, -0xff,0x50,0x5a,0xbc,0xf6,0x55,0x20,0x6a,0xf5,0x65,0xbd,0xf6, -0x29,0xfd,0xe5,0xe7,0x8f,0x82,0x8f,0x42,0x11,0x7e,0x84,0x39, -0x8d,0xea,0x2c,0x78,0xfb,0xc9,0x25,0x81,0x1c,0x23,0xa6,0xc4, -0xd9,0xdb,0xe5,0xfc,0x61,0x91,0xc1,0x32,0xd1,0x59,0x4c,0x3e, -0x0c,0x86,0x93,0x38,0xcb,0x54,0xae,0x53,0x04,0xba,0x6c,0x35, -0xc7,0xd1,0xe4,0x0b,0xce,0x6e,0xa6,0x23,0x58,0x9a,0xcb,0x97, -0x8a,0x7a,0xae,0x57,0x8f,0x49,0xa2,0x6b,0x9b,0xc5,0x53,0xe9, -0x28,0x44,0x50,0xba,0x74,0xf2,0x3e,0x8a,0x34,0xc2,0x8e,0x45, -0x4c,0x32,0xe1,0x2e,0xbf,0x4e,0xf0,0xbb,0x3a,0x89,0xa2,0x64, -0x68,0x13,0x42,0x33,0xcd,0xd3,0xc7,0x27,0x64,0xf7,0xc4,0x9f, -0xba,0xd9,0xd4,0x5c,0xf7,0xd2,0x51,0xbf,0x49,0x75,0xd3,0x65, -0x0a,0x11,0xc4,0x24,0x12,0x16,0x9e,0x7e,0xf9,0x93,0x7d,0x2e, -0x8c,0xd3,0xff,0xf5,0x35,0xe1,0xf3,0xfd,0x6d,0x9b,0xc1,0x22, -0xab,0x88,0xe5,0x96,0xd7,0xfe,0x8e,0xef,0xd9,0x5a,0xb9,0x70, -0xde,0x2b,0x8f,0xcd,0xb9,0x64,0xbd,0xf2,0xcc,0x47,0xdd,0x6c, -0x4a,0x38,0xa0,0xef,0x66,0xd1,0x8b,0xa5,0x35,0x46,0xd8,0x47, -0x2b,0x80,0x7f,0x36,0x9f,0xd1,0x2e,0x71,0x50,0xe2,0xeb,0x96, -0x90,0x52,0xd0,0x1d,0xc9,0x20,0xe8,0xba,0x74,0x76,0x97,0x2c, -0x53,0x69,0xcc,0x36,0x99,0x0f,0xc7,0x51,0x09,0xf1,0x18,0xed, -0xad,0x98,0x1a,0xbd,0x8b,0x87,0x68,0x08,0xd0,0x0e,0xc4,0x70, -0xfe,0x30,0x5b,0x51,0x8c,0x6b,0x03,0x10,0x21,0x36,0x6c,0x36, -0x85,0x32,0x34,0xe9,0x32,0xd4,0x7b,0x3a,0x45,0x4d,0x49,0x95, -0x37,0xf1,0xa5,0x53,0x52,0x32,0x7a,0x47,0x1c,0x6b,0x9b,0xbb, -0x33,0x81,0xa0,0x6d,0xfc,0x23,0xd0,0x47,0x77,0x5b,0x61,0xc0, -0x17,0xfc,0xbc,0x31,0xde,0xe8,0x24,0x7a,0x13,0xa3,0x07,0xf6, -0xa8,0x62,0x92,0x65,0xc2,0x56,0xd4,0xeb,0x58,0x4f,0x93,0x68, -0x47,0x17,0x2d,0x88,0xd5,0x3a,0x33,0x84,0xe2,0x0b,0x50,0xcc, -0xf6,0x8a,0x9b,0x1c,0xc0,0xe9,0x74,0x65,0x92,0x80,0x48,0x98, -0x13,0x57,0xe9,0xa8,0x8b,0x67,0xe8,0x9e,0xdf,0xf6,0xac,0x64, -0x1c,0x22,0xa6,0x75,0xcd,0x04,0x03,0x42,0x1d,0x1c,0xe4,0x53, -0x78,0x3b,0x64,0x89,0x1f,0xad,0x08,0x2d,0x28,0x96,0x1e,0xf5, -0x91,0xa9,0xa6,0x13,0x67,0x21,0x96,0xa4,0x77,0x52,0x34,0xf3, -0x72,0x8c,0x88,0x7c,0xa8,0x10,0xcb,0xc3,0xd2,0x96,0x96,0x9f, -0x4a,0xf6,0xc8,0xa6,0x49,0xb5,0xe9,0xe9,0x29,0xf4,0xbb,0x21, -0x7d,0x85,0x4b,0xcc,0xb8,0x56,0x96,0x86,0x08,0xc4,0xec,0xa5, -0x0e,0xf0,0x05,0x1f,0x9b,0xd8,0x92,0x6b,0xe3,0xa1,0x0b,0x52, -0x8b,0x01,0x2f,0x22,0x60,0x81,0x6f,0x7d,0xfe,0x64,0xc5,0x2e, -0x5c,0x45,0xb2,0xd0,0x91,0xea,0x19,0x8c,0x99,0x0b,0xdd,0xe9, -0xad,0xd0,0x4c,0x4a,0x11,0xad,0xfd,0x26,0x05,0x1d,0x5f,0xa6, -0xf5,0x1a,0x8d,0x59,0xdf,0x6a,0xa6,0x31,0x09,0xa3,0xb8,0x08, -0xde,0x1e,0x2c,0x47,0x27,0x8f,0x39,0x86,0x33,0x79,0xbe,0x3a, -0x18,0xb2,0x3a,0x1f,0xd3,0x65,0xa7,0x37,0x1a,0x46,0x74,0xba, -0xf2,0xab,0x5a,0x25,0xb4,0x6d,0xe7,0x2b,0x5b,0x67,0xc4,0x88, -0x70,0xcb,0x93,0x99,0xab,0x59,0xb0,0xb5,0x91,0x4d,0x9b,0x16, -0x66,0x3b,0xd5,0x95,0x67,0x27,0x47,0x31,0x4e,0x32,0x06,0x4f, -0xf6,0x25,0x27,0x24,0x6f,0xaf,0x22,0x2a,0xdd,0x75,0x24,0xb8, -0x41,0x69,0x7e,0x49,0x77,0xf3,0x4d,0x7b,0x49,0x7f,0x6f,0x45, -0xc0,0x09,0xc5,0xa9,0x32,0x5e,0xa7,0xd9,0x0f,0x9e,0xd6,0xd8, -0x62,0x8d,0xfb,0xd2,0x89,0x9a,0xb1,0x57,0xab,0xa5,0x74,0xca, -0x8b,0x27,0x1b,0x35,0x51,0x3c,0x99,0x77,0x5c,0xf6,0xae,0xd1, -0x76,0xa9,0x26,0x02,0xbf,0x96,0xfb,0xa0,0x54,0xd5,0xf4,0xa3, -0x55,0xd3,0xe6,0xc3,0x6b,0xc2,0x39,0x72,0x43,0x75,0x15,0x6a, -0x1a,0x61,0x4c,0x9e,0x87,0xd5,0x50,0x29,0xd7,0x6b,0xba,0xc1, -0xff,0x23,0x05,0x0a,0xc9,0x2d,0xb6,0xae,0xc2,0x2e,0xa6,0x37, -0xff,0xfc,0xfd,0x67,0xe5,0x6b,0x13,0x73,0x3f,0x45,0xd7,0x45, -0x8e,0x62,0x06,0x6e,0xf2,0x37,0x13,0x30,0x22,0x71,0xcc,0xc4, -0x13,0xba,0x31,0x13,0xd2,0x9f,0x89,0x75,0xbd,0x43,0x37,0x63, -0x23,0xfd,0x2d,0xd5,0x28,0x88,0x08,0x9c,0x93,0xc4,0x28,0x8c, -0x38,0x04,0xad,0x37,0x0a,0x84,0x65,0x43,0x1c,0x36,0x30,0xff, -0x0a,0xff,0x34,0x46,0x41,0x77,0x14,0xb4,0x47,0xa1,0xa1,0x80, -0x8c,0x12,0x19,0xa7,0xf5,0xa9,0x19,0xa5,0x48,0xec,0x52,0xf7, -0x8e,0xe8,0xf3,0x0e,0x44,0x82,0xf1,0x6c,0xf0,0x0b,0x36,0xfc, -0x64,0x42,0xe3,0x51,0x55,0xe3,0xf2,0x76,0x5c,0x82,0xcb,0xc6, -0x52,0x67,0x82,0xc7,0x79,0x1c,0x6c,0xc9,0xb1,0x02,0x09,0x0b, -0xc5,0x68,0x65,0x4d,0x13,0x44,0xe4,0x2a,0xf0,0x49,0xf4,0xe8, -0xde,0x0a,0xbf,0xc9,0x5e,0x5d,0x85,0x64,0xab,0xd8,0xc1,0xd8, -0xc5,0x38,0x81,0xf8,0x33,0xe9,0x20,0x1a,0x05,0xf4,0xe3,0xd1, -0xea,0xe9,0x89,0xe0,0xa5,0x2b,0x54,0xd5,0x79,0x75,0xca,0xe6, -0x6b,0x49,0x02,0x45,0x66,0x69,0xb3,0x48,0x83,0x6c,0xef,0xb2, -0xda,0x60,0x11,0xe0,0x73,0x8d,0x21,0xf9,0x31,0x0d,0x5a,0x2d, -0x37,0xef,0x24,0x13,0x22,0x4b,0xc2,0xe8,0xf2,0x17,0x7e,0x6d, -0x26,0x51,0xba,0xc4,0x0c,0x8f,0x20,0x7d,0x2b,0x69,0x53,0xe0, -0xc1,0xab,0x44,0xb8,0xbd,0xc1,0xeb,0xc4,0x89,0x41,0x8a,0x44, -0x6e,0xb7,0xd4,0x6f,0x7c,0xe5,0xc1,0x7d,0x28,0x38,0x85,0x04, -0x28,0xa2,0xbf,0x32,0x05,0x19,0x4b,0xdc,0x95,0xef,0xdc,0x8b, -0x52,0x8e,0x54,0x29,0xf3,0x05,0x9d,0x26,0x94,0x73,0xf0,0x29, -0x2f,0xc1,0xd4,0x62,0xca,0xf6,0x6d,0xf2,0x99,0x18,0xa4,0x1c, -0xdf,0x98,0x90,0xd2,0xa2,0x94,0xe5,0x13,0x14,0x11,0x57,0x0a, -0x44,0x96,0x91,0x04,0x23,0x09,0x2e,0x85,0x92,0x71,0x9b,0xb9, -0x46,0xd3,0x08,0xcc,0xe3,0xa6,0x55,0x8f,0x43,0x2a,0x3f,0x4d, -0x56,0x77,0xf3,0x51,0x71,0x43,0xa2,0x64,0xfc,0x37,0xf5,0x26, -0x21,0xe1,0x1a,0xa8,0x43,0xc7,0x7e,0xf3,0xba,0xa2,0x3b,0x8b, -0x64,0x4a,0x6b,0x4e,0x0c,0xc1,0xc9,0x9f,0x56,0x6a,0x08,0x65, -0xe7,0xc9,0xa9,0xda,0x33,0xeb,0x42,0xdc,0x08,0x8e,0x45,0xa6, -0x8f,0xe0,0xee,0xa3,0x40,0x1d,0x79,0x1d,0x9f,0x47,0x51,0xa4, -0x90,0x91,0xc7,0xd0,0xcf,0x60,0x7a,0xfa,0x74,0xe5,0xa5,0x3e, -0x7d,0xf7,0x81,0xe5,0x78,0x20,0x68,0x20,0x13,0xdb,0xec,0x1e, -0xd7,0x16,0xd2,0x87,0xab,0xb7,0x88,0x5c,0x12,0xbf,0x8d,0x47, -0x59,0x6f,0xe8,0xe2,0x06,0xbc,0xe8,0xf8,0x10,0x1d,0x75,0x7b, -0xe5,0x59,0xbe,0xa0,0x08,0x52,0x36,0x22,0xc0,0x22,0x0b,0xdc, -0x30,0xff,0x79,0x29,0xbc,0xe8,0xd6,0xc8,0xc0,0x6c,0xd8,0xad, -0x7d,0x04,0xbb,0xe7,0xa0,0xd6,0x26,0x43,0x3d,0x8b,0xe2,0xa1, -0x0c,0x1f,0x66,0xee,0xae,0xf7,0x69,0x8e,0x04,0xd2,0x5f,0x3f, -0x90,0xe5,0xba,0xa1,0x55,0xae,0x45,0x94,0x60,0xf0,0x47,0x19, -0x14,0x49,0xcc,0x6c,0xcd,0x19,0x85,0xda,0xbc,0xbe,0xdb,0xb4, -0xc3,0xcb,0x20,0x10,0x53,0x58,0x2b,0x8f,0xda,0xa1,0xf8,0x13, -0xfc,0x7e,0x05,0xff,0x5e,0xc1,0xbf,0x2f,0xe1,0xdf,0x4f,0x7c, -0xda,0x46,0xd1,0xb5,0xb8,0xdd,0x84,0xce,0x1b,0x09,0x55,0xa9, -0xa4,0x09,0xa6,0x44,0xb2,0x15,0x7f,0x11,0x49,0x1a,0x5d,0xdb, -0x33,0xf2,0x8d,0x5c,0x5b,0xd4,0x52,0x10,0x16,0xfc,0x3a,0xb6, -0xf9,0xa7,0x08,0xfe,0x7d,0x1c,0xb6,0xea,0x61,0xeb,0x10,0x56, -0x7e,0x6c,0x68,0x42,0xd2,0xff,0x53,0xc4,0x9d,0x36,0x6a,0xfe, -0x44,0x91,0x20,0x91,0xad,0xb0,0x4e,0xb3,0x24,0x56,0x28,0xcd, -0xcd,0x03,0xfa,0xa7,0xae,0x95,0xd7,0x96,0xcf,0x7a,0x8f,0x7a, -0x4f,0x75,0x76,0x47,0xc4,0xce,0x64,0xf5,0x05,0xe0,0xfe,0x91, -0xbe,0x24,0x6b,0x6e,0x04,0x7a,0xfe,0xbc,0xf7,0xdb,0xb5,0x47, -0x45,0x57,0xed,0x2f,0xf4,0xf4,0xf4,0x54,0xfb,0x33,0x3d,0xf8, -0x5d,0xaf,0xa2,0xa0,0x9c,0x68,0x4a,0x72,0x24,0x0e,0x28,0xd9, -0x7c,0xac,0x9f,0xf9,0x1f,0x9f,0xd7,0xcf,0xa1,0x85,0x7f,0x69, -0x30,0xa6,0xdb,0x64,0xf5,0xc7,0x78,0xe3,0xf9,0xf5,0x53,0xc8, -0x69,0x23,0x08,0xe6,0xfe,0x39,0x9f,0x8b,0xe5,0xf0,0xae,0xbf, -0xac,0x8a,0x51,0x73,0x0a,0xff,0x46,0x86,0x80,0x3f,0xcd,0x1f, -0x96,0x19,0xf2,0xb4,0x6e,0xdd,0x29,0x03,0xfb,0x8e,0xc2,0x00, -0x3f,0xf5,0x6e,0x7e,0x55,0x87,0x97,0x8f,0xe1,0x05,0x9e,0x5f, -0xc1,0xbf,0x2f,0x7d,0xa1,0x18,0x87,0x18,0xae,0xa9,0x9a,0xad, -0xbc,0xda,0x69,0xae,0xe6,0x1c,0x3a,0xb1,0xcc,0x0a,0x40,0xf6, -0xb9,0x82,0x2d,0x6a,0x00,0x29,0x1d,0x14,0xf6,0x21,0x27,0x20, -0xf7,0xee,0xc0,0x5f,0xc9,0xc0,0x83,0x39,0xe1,0x15,0x53,0x57, -0x80,0x38,0xe4,0x1d,0x2e,0xdb,0xef,0xaf,0xa2,0xa9,0x14,0xa7, -0x46,0x48,0x0e,0xf7,0xca,0x75,0x43,0xbd,0xee,0x0b,0xd2,0x1c, -0x43,0x2a,0xb6,0x2a,0x0c,0x65,0x82,0x58,0xb8,0x02,0xbd,0xcc, -0x7a,0x2b,0x8c,0x58,0x59,0x54,0x13,0xdd,0xb2,0xe2,0xed,0x55, -0x5f,0xbc,0x58,0xc0,0xee,0xf3,0x3e,0xe2,0x75,0x84,0x69,0xa4, -0xff,0x49,0xa0,0xa9,0xdc,0xaa,0x90,0x83,0x43,0x3d,0x2c,0x2c, -0x4a,0xef,0x01,0x91,0x01,0x95,0x2c,0xbe,0xaf,0xd2,0xaa,0xdf, -0x25,0xb4,0x37,0x37,0x5e,0x4c,0x51,0x8d,0x7f,0x97,0xdc,0x7e, -0xbe,0x5e,0x78,0xb5,0xbf,0x7a,0xdd,0x76,0xad,0x4e,0xf9,0x74, -0xa9,0x48,0xdf,0x31,0xfe,0xfc,0x30,0x87,0x71,0x20,0x95,0xc5, -0x53,0x8d,0xd6,0x92,0xa2,0x96,0xd6,0x4a,0xd6,0x79,0x5c,0xc5, -0x97,0xf3,0xf9,0xf8,0x61,0xa1,0xaa,0x51,0x9d,0x05,0xf8,0x2c, -0x1b,0x51,0x75,0x4f,0xc2,0x75,0xb9,0x37,0x55,0x00,0xa9,0xcf, -0x10,0xf0,0xaa,0x0a,0x44,0xe9,0xcb,0xf9,0x63,0xb2,0xfc,0x2c, -0x26,0x83,0xaa,0x54,0xeb,0x3a,0x00,0xac,0xa2,0x01,0x5d,0x74, -0xf0,0x29,0xcc,0x4d,0x3a,0x99,0x88,0xc7,0x74,0x04,0x4a,0x84, -0x3d,0x5a,0xa4,0xb7,0xb3,0x28,0xb8,0xe2,0x13,0x95,0x5a,0x03, -0xd4,0x6a,0x4d,0xcd,0x1d,0x1e,0xe6,0x75,0x1b,0x94,0xd5,0xe6, -0x0d,0x64,0x1d,0x72,0x99,0xba,0x9c,0x3c,0x29,0x75,0x0b,0x25, -0xea,0x1e,0xa1,0xbf,0x52,0x31,0x0f,0xa1,0x91,0x1c,0xe3,0x8f, -0x92,0x1b,0x9c,0x0c,0x2a,0x9a,0x59,0x88,0xf4,0xf8,0x75,0x46, -0xd6,0x96,0x5a,0xb4,0x62,0x00,0xfc,0x25,0x49,0xc6,0xa3,0x78, -0xf3,0xe9,0x60,0xb0,0x4c,0xde,0x38,0xf2,0x91,0x92,0x87,0x2e, -0x82,0xd4,0x00,0xdf,0x25,0x76,0x60,0xd1,0x0e,0xeb,0xf4,0x32, -0x28,0x8a,0xc0,0x29,0x5b,0x63,0x66,0x06,0xcb,0xc8,0x76,0x26, -0xd5,0xd5,0x63,0xb1,0x38,0xf7,0x2d,0x1d,0x1a,0xa3,0x79,0x8d, -0xdb,0x35,0xd0,0x37,0xf5,0x99,0x71,0x3d,0xec,0xb7,0x1b,0xe1, -0xee,0x96,0xed,0x68,0xd3,0x73,0xad,0xf9,0xe0,0x76,0xfc,0xbe, -0x2d,0x90,0xd1,0x2f,0x2a,0xda,0xc1,0x0b,0xf6,0x9d,0x4d,0xd1, -0x20,0x15,0xad,0x11,0x38,0xb9,0x97,0xb4,0x88,0xc8,0x7c,0x2f, -0x6a,0x99,0xd4,0x57,0xb0,0xfe,0xac,0x66,0xfc,0xef,0x40,0x70, -0x91,0xd8,0x3f,0x7f,0x30,0xb1,0x5f,0xcd,0xff,0xe9,0xc4,0xfe, -0xe5,0x3d,0x89,0x05,0x12,0x57,0x77,0xbb,0x47,0xe8,0xd4,0x80, -0xec,0x24,0xd6,0x85,0xdb,0x9b,0xe2,0x69,0x19,0x82,0xdf,0x49, -0xca,0xa9,0x75,0x3b,0xdb,0xf5,0x7c,0x8b,0x7e,0x43,0x5b,0x7e, -0xa7,0x56,0x7c,0x89,0xab,0x94,0x04,0x17,0x3f,0xf9,0xa6,0x98, -0xa3,0xf8,0xfc,0xb2,0x23,0xb7,0x3e,0x6f,0x0e,0xf5,0x82,0xc6, -0xf3,0x0d,0x82,0x9d,0x15,0xfe,0x51,0x1d,0x29,0x7c,0x50,0x85, -0xeb,0xf7,0xaf,0xf0,0x7b,0x78,0xff,0xf0,0x0a,0x7f,0x78,0x9f, -0x0a,0xcd,0x4a,0xf2,0x1f,0xa9,0xf9,0x4e,0x8a,0x12,0xb2,0x79, -0xcf,0xf1,0xf9,0x8f,0x27,0xb2,0x55,0x46,0xa4,0x2a,0x9f,0xac, -0x17,0xf1,0x6c,0x44,0x44,0x10,0xc1,0xfb,0x0b,0xed,0x4f,0xf3, -0x59,0x45,0x67,0x1e,0xfd,0xb5,0x57,0x6f,0xf4,0x5f,0x8f,0xde, -0x9d,0x6c,0xff,0xf5,0xa8,0xb9,0x42,0x27,0xd6,0x72,0x01,0x53, -0x4a,0xdf,0xa9,0xaf,0xb6,0x1f,0x3f,0x45,0x75,0x85,0x0b,0x52, -0x4b,0xab,0xb6,0xe8,0xb5,0xae,0xcd,0xea,0xde,0xe8,0xea,0xec, -0xa2,0x8b,0x5b,0xbc,0x76,0x2b,0x39,0xae,0x12,0x0a,0xd2,0x16, -0xff,0x8c,0x19,0xb1,0x84,0xe5,0xd3,0x08,0xd9,0x0a,0x6b,0xca, -0x7d,0xf9,0xfb,0xc7,0x7f,0xe8,0xc4,0x52,0x4e,0xe4,0xe8,0x3d, -0x85,0x17,0x68,0xfc,0xe6,0xe6,0x1f,0x2f,0xc1,0xc7,0x45,0x4a, -0xef,0xdf,0x93,0x52,0xdc,0x3b,0xb6,0x4e,0xfe,0xe9,0x0c,0xfd, -0xd3,0xfb,0xce,0xd6,0xe9,0xec,0x01,0x06,0xcc,0x3f,0x9d,0xce, -0xaf,0xde,0x93,0xce,0x57,0xc9,0x10,0x96,0x3e,0xff,0x7c,0x3a, -0x5f,0xbd,0x37,0x3f,0x27,0x13,0x34,0x39,0xfe,0xc7,0x13,0x5b, -0x22,0xa3,0x5f,0xbe,0x27,0xb1,0x9f,0x4e,0xbf,0x9d,0xe6,0x89, -0x74,0xe9,0x88,0x01,0xc2,0x5a,0x73,0x94,0x12,0x12,0xb5,0xfc, -0xdc,0x3a,0xa4,0xe3,0x5c,0x5f,0xc3,0x06,0x12,0x6f,0x18,0x91, -0x40,0xd8,0xf0,0x96,0xce,0xa0,0x6f,0x49,0xb5,0x73,0xe5,0x6f, -0xa3,0x11,0xd6,0x85,0x53,0x37,0x26,0xb3,0x31,0x07,0x4c,0xc0, -0x6f,0xb3,0xe8,0xed,0x55,0xc0,0x1b,0xd4,0x7a,0x4d,0xbc,0xbd, -0x8b,0xfe,0xfd,0xdf,0xf9,0x93,0xbf,0x78,0x90,0x79,0x6f,0xfd, -0x23,0x8c,0xdb,0xf8,0x76,0x1a,0x59,0x49,0x1f,0x9f,0xe9,0xc0, -0x2e,0x6f,0xb3,0x7a,0x71,0x3f,0xfc,0xf6,0x4e,0xd4,0x82,0x9a, -0x68,0xf9,0x65,0x79,0x53,0x99,0x57,0xb5,0xc0,0x48,0x57,0xc9, -0x32,0x9e,0x7c,0x9b,0x2c,0x87,0xe4,0x30,0xa2,0xbc,0xab,0x17, -0x9c,0xbd,0xb3,0xaf,0x0d,0xcc,0x9e,0x3b,0x1e,0xa7,0x7b,0x4b, -0x7b,0x97,0x9b,0xf1,0x67,0xbc,0x8b,0x78,0xe6,0xc8,0xb8,0xea, -0x4c,0xfb,0xbf,0x94,0x1c,0x6a,0x97,0x1c,0x68,0xf3,0x8d,0x11, -0x24,0x54,0x5d,0x4d,0xe8,0xd3,0xc7,0xa8,0xea,0x2c,0x7b,0xbf, -0x53,0xd5,0x5d,0x67,0xef,0xe6,0xee,0x8a,0x4b,0xab,0x42,0xfa, -0x46,0x0c,0xf2,0x0f,0x0e,0xf8,0x48,0xae,0x82,0xcc,0xc2,0x19, -0x1d,0xa3,0x53,0xaf,0x95,0xc7,0x72,0x9c,0xfe,0x32,0x9b,0x7f, -0x4d,0x2e,0x4a,0xdc,0x8b,0x35,0xaa,0x71,0x35,0x7f,0xf9,0xea, -0x1b,0xb5,0x8a,0x2c,0x22,0xe0,0xaf,0x17,0xa4,0x57,0x04,0xbe, -0x1e,0xc8,0xb4,0xf5,0x78,0xe5,0x37,0x0e,0xd2,0x29,0xc2,0x3a, -0x77,0x1b,0xff,0x15,0x5f,0x2f,0xb0,0x77,0x12,0x36,0xd1,0x90, -0x47,0x3c,0x55,0x0e,0xa6,0x38,0x5e,0x54,0x44,0x2e,0xa4,0xd8, -0x2c,0x22,0xb5,0x7d,0x15,0xda,0x44,0xa2,0xbf,0xac,0x4c,0x16, -0xd0,0xac,0x4d,0xa3,0xa8,0x08,0xa4,0x2c,0xd2,0x7a,0xb2,0x3d, -0xcd,0x0d,0x36,0x6e,0x54,0xe5,0x18,0x89,0xa9,0x71,0xbf,0xe0, -0xb6,0xdd,0x5b,0x1c,0x87,0xa7,0xc7,0x67,0xc9,0xd9,0xd6,0x97, -0x2d,0xea,0xb5,0xfa,0xfd,0x76,0xda,0x95,0xc8,0x7b,0x4c,0xd1, -0x51,0x91,0x8c,0x5e,0xda,0x08,0xfb,0x9f,0x94,0xa5,0xf7,0x8f, -0xb8,0x50,0x17,0x20,0xda,0x69,0xbf,0xdd,0x73,0x80,0x6c,0x15, -0xfe,0x1c,0xd5,0x16,0x49,0xfb,0xba,0x09,0x72,0xea,0xa2,0x1d, -0xd1,0x3f,0xc0,0xc5,0xc4,0x5a,0x7b,0x2f,0x70,0xd3,0x7d,0x79, -0x8a,0x99,0x23,0xa1,0xc2,0x9b,0x83,0xba,0x1d,0x13,0xd9,0x38, -0xb5,0x7d,0x4b,0xe2,0xeb,0x22,0x19,0x39,0xd2,0xae,0xbe,0xb3, -0xc3,0xa4,0x83,0x83,0x17,0xaa,0xa8,0x89,0xe3,0x65,0xf6,0x57, -0xa6,0xe5,0x74,0x31,0x0c,0xca,0x8d,0x2b,0x50,0xbe,0x9d,0x69, -0x4c,0x73,0xd3,0x9d,0x0f,0x25,0x95,0x27,0xb2,0x2a,0x67,0x3d, -0x2c,0x11,0x11,0xcf,0x3a,0x8a,0x80,0x6e,0x71,0x60,0x84,0x81, -0xdf,0xae,0xf1,0x0c,0x6b,0x0c,0xca,0x14,0xfc,0xc1,0x41,0xb1, -0x80,0xca,0x33,0xe7,0xac,0x94,0x7d,0x70,0xa0,0x39,0x14,0x71, -0x0a,0x7d,0xc2,0x0b,0x4d,0x51,0xaf,0xe8,0x02,0xb7,0xdc,0x11, -0x86,0xed,0x96,0x0e,0x4b,0x5c,0x85,0x05,0x6f,0xaf,0xe6,0x4e, -0xaf,0x63,0x33,0x5c,0xb7,0x4c,0xdd,0x47,0x20,0x98,0xdf,0xf1, -0x95,0x6a,0xcc,0xb1,0x58,0xde,0xe0,0xdb,0xba,0x30,0xef,0x25, -0xf6,0xd9,0x8a,0x8c,0xcd,0x40,0x75,0x3d,0xaa,0x2b,0x9d,0x8a, -0xb6,0x6d,0xcd,0x3a,0xbf,0xc2,0x8d,0x49,0x4e,0xc6,0xaa,0x5d, -0x32,0xb9,0xa2,0xf0,0x3b,0xf5,0x76,0x09,0xbc,0xa6,0xb8,0x9d, -0x13,0xe2,0x83,0x83,0xde,0x3b,0x7a,0xd0,0x8d,0xda,0x12,0xd1, -0xfd,0x0f,0x91,0x8a,0xdc,0xf8,0x30,0x1a,0xb9,0x84,0xb3,0x5a, -0x71,0xe3,0x48,0x09,0xaf,0x10,0x55,0x37,0x6c,0x13,0xbf,0x76, -0x39,0x77,0x31,0x17,0xd9,0x3c,0x79,0x3d,0xfb,0x21,0x58,0x61, -0x42,0x52,0xfe,0x4b,0x72,0xf3,0xd2,0xfb,0xbb,0x2f,0xc9,0xb5, -0x68,0xe5,0xdc,0xb5,0xc8,0xa4,0x8a,0x02,0x5f,0xd8,0x9e,0x96, -0xb2,0xf2,0xcb,0x79,0x73,0xa7,0x92,0x46,0xea,0xf0,0x46,0x7f, -0xb2,0x79,0xa3,0x92,0x40,0xef,0x77,0x5e,0xdc,0x00,0x1b,0xb9, -0x4c,0xc7,0x37,0x39,0x8d,0x46,0xaa,0x7b,0xf1,0x06,0x7a,0x81, -0x21,0xdc,0x8b,0xf8,0xfb,0x0c,0x6a,0xc3,0xcb,0x9f,0x24,0x33, -0xf4,0xff,0xb7,0x57,0xdf,0x7c,0x2d,0x17,0x1d,0x32,0xab,0xb9, -0x4c,0xb2,0xc5,0x7c,0x96,0x25,0xdf,0x43,0xb7,0xb9,0x8d,0xba, -0x5b,0x4d,0x27,0x06,0x03,0xd9,0x00,0x93,0x95,0x02,0x2e,0x45, -0xe6,0x43,0xd2,0xe7,0xcd,0xe1,0x32,0x81,0xaa,0x79,0x8a,0x71, -0xdd,0x26,0x34,0xd9,0xe4,0x89,0xac,0x39,0xed,0x12,0x83,0xf9, -0x68,0x23,0xbf,0x96,0x97,0xa5,0xc9,0x22,0x74,0xbd,0x7a,0x88, -0x27,0x5f,0x2c,0x63,0xfe,0xfc,0xa3,0x9c,0x38,0x52,0xb4,0xc7, -0xd1,0x3b,0xb4,0x58,0x47,0x8b,0xc9,0xda,0x71,0xf3,0xb8,0x79, -0x56,0xdb,0x76,0xb0,0x4b,0x9a,0xb3,0xf9,0xe3,0xd3,0x93,0xa7, -0x1e,0x8b,0x22,0x53,0x57,0x9d,0xb7,0xe5,0x4b,0x66,0x20,0x8a, -0x22,0x1b,0xbe,0x42,0xd7,0x5e,0x51,0xaf,0xcf,0x3e,0xbe,0x84, -0x4a,0x36,0x08,0x60,0x76,0x75,0x3e,0x8d,0x33,0xa5,0xd8,0x34, -0x8f,0xf2,0x49,0xc8,0x54,0x23,0x23,0xf5,0x60,0x27,0x7e,0x3e, -0x49,0xa6,0x52,0x4f,0x68,0x66,0xe4,0xf2,0x10,0xfc,0x11,0x44, -0x13,0xc8,0xe7,0x9f,0x8e,0x5c,0x47,0x52,0x95,0x5e,0x11,0x19, -0x3b,0xe5,0xa7,0x88,0xc1,0x3e,0xae,0xb1,0xd1,0x6c,0x0f,0xad, -0xbc,0xb7,0xc3,0x78,0x35,0xbc,0x43,0xcf,0x3b,0x55,0xcd,0x31, -0x12,0x88,0xef,0xea,0xd2,0x95,0x41,0xcd,0x05,0x58,0xea,0x77, -0xd2,0x46,0xa3,0xe3,0xab,0x10,0x90,0x04,0xdc,0x33,0xc2,0x47, -0xe9,0xdb,0xad,0x5a,0xed,0xba,0x52,0x21,0x69,0xf4,0x6a,0xa3, -0xf4,0x4d,0xcd,0x6f,0x66,0xab,0xcd,0x84,0xae,0xa9,0xbf,0x5d, -0xce,0x61,0x0f,0xb1,0xda,0x78,0xb5,0xf9,0x22,0x1e,0xa6,0xab, -0x4d,0x0d,0x6f,0xcb,0x6b,0xbe,0x22,0x1a,0xfd,0x32,0x48,0xbb, -0x84,0xe3,0xeb,0x84,0x91,0x5c,0x2f,0x94,0x65,0x77,0xa4,0x79, -0xd4,0x54,0x4c,0xd0,0x79,0xc2,0x2a,0x00,0x15,0x7d,0xba,0x82, -0x75,0xec,0xe0,0x81,0xf5,0x7f,0x01,0x51,0xd3,0x86,0xa8,0x2a, -0xf9,0xf5,0xab,0xe7,0xcb,0x7e,0xfd,0x8a,0x14,0x0c,0x36,0xaf, -0x94,0xcc,0xcf,0x5e,0xbd,0x7a,0x85,0x99,0x7f,0x4c,0x86,0x13, -0x19,0x3e,0xd1,0x25,0x99,0x8b,0x5a,0x9c,0x89,0x8a,0xf8,0x6c, -0xc6,0x75,0x9e,0x25,0xc9,0xf4,0xb6,0x65,0xc7,0xfd,0xae,0xa2, -0x8d,0xd6,0xf7,0x5c,0x06,0xba,0x8e,0x1d,0x22,0xf6,0x68,0xbb, -0xb5,0xf7,0x81,0xde,0x94,0x5b,0xac,0x67,0x6a,0xfc,0xfa,0x95, -0x55,0x67,0xa1,0x98,0xae,0x7a,0x07,0x0b,0xca,0x1a,0x28,0x16, -0xcb,0x74,0xbe,0xc4,0x0f,0x67,0xdf,0x95,0x31,0xb5,0xaa,0x99, -0xa6,0xd8,0x76,0x6b,0x3b,0x5b,0x30,0x55,0x38,0x41,0x09,0xaf, -0x62,0x3c,0x1f,0x88,0xaf,0x06,0x5d,0xfc,0x1b,0x0d,0xba,0x41, -0x3b,0x38,0x0a,0x90,0xde,0x26,0x8c,0xc2,0x9d,0x65,0xb1,0x10, -0x94,0x45,0x14,0xf8,0x37,0x8a,0xad,0xb2,0x53,0x7b,0xe1,0x4c, -0xc3,0x4a,0xdc,0xf0,0x20,0x40,0x77,0xa9,0xf2,0xb2,0xdd,0x89, -0xd0,0xca,0xa1,0x94,0xa2,0x82,0x97,0x27,0x5e,0x19,0xd1,0x55, -0x3c,0x2c,0x6c,0x3d,0x69,0xf9,0x14,0xeb,0x30,0xae,0xb0,0xdc, -0x05,0xba,0x63,0x98,0x50,0xe2,0x88,0xad,0x66,0x3b,0xa6,0x04, -0xda,0x28,0x12,0xfc,0xc0,0x85,0x47,0x9b,0xfe,0x38,0x1a,0xb0, -0x7f,0xc6,0xaa,0x1a,0x6e,0x98,0xc5,0x4c,0xbd,0x2a,0xce,0xa6, -0xdd,0x7b,0xd5,0xb8,0xb3,0xbc,0xa6,0x40,0x31,0x73,0x2b,0xa3, -0x96,0xfe,0xff,0x97,0x6d,0xd0,0xd1,0xff,0x9b,0xd9,0x66,0x51, -0xe0,0xb0,0x2d,0xc9,0x7d,0xa9,0x5f,0xe0,0xdc,0xf0,0x77,0xe1, -0xdd,0xb0,0x94,0x7b,0xc3,0x7d,0xf9,0xe7,0x99,0x5e,0x17,0x83, -0xab,0x21,0x99,0xd7,0x0d,0xfc,0xdd,0xcc,0x1c,0xfe,0x23,0xd9, -0x59,0x45,0x90,0xfc,0x74,0x2b,0x16,0xc3,0x3e,0xb1,0x37,0x7b, -0x98,0x56,0xf2,0x36,0x23,0x6b,0x27,0x9b,0xad,0x1c,0xf0,0xb3, -0x8a,0xb7,0x36,0x8d,0xbc,0x67,0x8d,0xa3,0xba,0x66,0x12,0xac, -0x78,0xb2,0x3a,0x36,0xca,0x18,0x08,0xe7,0x61,0x2b,0xdb,0xa3, -0xcb,0xaa,0x93,0x00,0x1e,0x51,0x49,0x5c,0xad,0x89,0x72,0x84, -0x4f,0x31,0x7c,0x89,0x32,0xd6,0xdc,0xa7,0x01,0xa0,0x97,0xa5, -0x0d,0xb4,0x33,0xb0,0xbc,0x69,0x1d,0xfa,0xae,0x31,0xf5,0x8f, -0xea,0xf5,0xfb,0x92,0xb6,0xd8,0xc5,0x76,0x75,0x8f,0x8b,0x46, -0x59,0x52,0x75,0xa7,0xea,0x5b,0x09,0x61,0xf9,0x0d,0x32,0x6d, -0xe4,0xa8,0xc7,0x42,0xee,0xe6,0xfe,0x14,0xb9,0x21,0x9e,0x61, -0x33,0x7a,0xb8,0xa8,0x87,0xe2,0xce,0x76,0x61,0xf5,0x27,0x5f, -0xbc,0x89,0xea,0x0c,0xd7,0x23,0x37,0x28,0x49,0xf4,0xa7,0x86, -0x36,0x2a,0x4a,0xba,0x6f,0xea,0xc9,0xa1,0xa7,0xf2,0xfb,0x8d, -0x37,0x7e,0xfb,0x8d,0xe4,0xee,0x28,0x2d,0xe5,0x6f,0xc5,0x41, -0x09,0x05,0x21,0xe4,0x55,0x19,0x73,0x9e,0x8e,0x9d,0x7c,0x5f, -0xd8,0x69,0xd2,0x99,0x90,0xe6,0x92,0xcc,0x35,0xa1,0xa1,0x8c, -0xb3,0x24,0xce,0x28,0x71,0x2c,0xd4,0x3c,0xf5,0x6d,0x2e,0xf1, -0x89,0xda,0x7c,0x69,0x28,0xd5,0x34,0xbe,0x9b,0x24,0x37,0xab, -0xb6,0x35,0xc1,0xad,0x61,0xf2,0x16,0x77,0x29,0x6b,0x81,0x52, -0x97,0x62,0x93,0x79,0xa9,0xb7,0xb1,0x13,0xc8,0xba,0x4b,0x23, -0x1d,0x31,0xbd,0x73,0x97,0x5e,0x4d,0xe6,0x1d,0xee,0x87,0x69, -0x3a,0x8a,0x26,0xf3,0x3a,0x24,0x5d,0x5d,0x85,0x1d,0xd5,0xeb, -0x22,0xee,0x41,0x46,0x5f,0xc0,0x1f,0xff,0x93,0x75,0x17,0x10, -0xc3,0x53,0x3d,0x6c,0x03,0x1a,0x78,0xd8,0xea,0x2f,0x9e,0xb6, -0x62,0x89,0xf1,0xa6,0xfe,0xd9,0x64,0xae,0x3f,0x29,0x23,0xb4, -0xcb,0xd4,0xb5,0x15,0xb5,0x16,0x99,0xdb,0xad,0xda,0x8b,0x68, -0x8e,0x5b,0xdc,0x2f,0x3d,0x60,0x84,0xed,0x8b,0x06,0xc1,0x20, -0x5f,0x91,0x55,0xb8,0x89,0x7d,0x63,0xfa,0xcf,0xe0,0xa2,0xe8, -0x5b,0x0e,0x24,0xf1,0x87,0x94,0xd5,0xdd,0xc3,0xcd,0x8d,0x3d, -0x1e,0x72,0x01,0xe3,0x57,0x22,0x85,0x81,0xee,0xcc,0x03,0x53, -0x50,0x2f,0x51,0xf0,0xc4,0xfe,0xd0,0x62,0x58,0xe4,0x4e,0x3d, -0xff,0x70,0xda,0x68,0x88,0x95,0x1c,0xd5,0xd3,0xbe,0x50,0x0f, -0x7a,0x9c,0xeb,0xc1,0x1a,0xad,0xdc,0xed,0x03,0x12,0x01,0x2b, -0xb4,0xa9,0xb3,0x6e,0xe5,0x81,0x41,0xfe,0x62,0x6c,0x23,0x44, -0x0c,0xd0,0x44,0x49,0x4a,0x05,0xc9,0x82,0x59,0xc9,0xa6,0x45, -0x65,0x61,0x95,0xb2,0x6e,0x2e,0x0b,0x09,0x7a,0x07,0xa3,0x80, -0x98,0x8a,0x38,0x5d,0x66,0x95,0x8c,0x58,0x04,0xd2,0x54,0xd5, -0xe6,0x05,0x28,0xbf,0x45,0xa8,0x03,0xe9,0x0b,0xc6,0x60,0x68, -0x09,0xae,0x66,0xb0,0xc8,0x9b,0xf9,0x6c,0xbe,0x4a,0xb9,0x48, -0x4e,0x6f,0x11,0x44,0x0b,0xab,0x24,0xe8,0x39,0x8b,0x24,0x84, -0x22,0x7a,0xde,0xa6,0x0b,0x7b,0x1f,0x8b,0x9f,0x29,0x7a,0xb3, -0xa2,0x86,0x95,0xc6,0xa3,0xbd,0x7e,0xc7,0x09,0x52,0x88,0xb6, -0x38,0xe4,0x63,0x4a,0x17,0xc0,0x45,0x35,0xe0,0xfc,0x52,0x7a, -0x76,0x86,0x47,0x9b,0xd6,0x29,0xfb,0x98,0x98,0x1a,0x0f,0xdb, -0x33,0x50,0xec,0x80,0x08,0xc9,0x40,0x58,0xa4,0xdc,0x80,0xcf, -0x38,0x3a,0x11,0xa8,0x67,0xc8,0xeb,0xdd,0xf7,0x0d,0x5d,0xf0, -0x62,0xed,0x11,0xb1,0x24,0xb5,0x86,0xbe,0x6a,0x5a,0xcc,0xed, -0xeb,0x92,0x69,0x0c,0x3b,0x02,0xfb,0xac,0x19,0x9b,0x6c,0xe2, -0x22,0xc8,0x6c,0x2a,0x3d,0x4e,0x36,0xb6,0xe7,0x9b,0x58,0xaa, -0x6b,0x4a,0xb6,0x5a,0x0e,0xef,0x68,0xbf,0x8b,0xf9,0x98,0xc5, -0x76,0xf4,0xf0,0xa4,0xa7,0x03,0x4c,0x25,0x84,0xac,0x9d,0x4b, -0x50,0xca,0x8c,0x72,0xa4,0x72,0x6e,0x20,0xb4,0xf0,0xde,0x83, -0x2c,0xe3,0xcd,0x82,0x33,0x79,0x85,0x35,0x03,0xd2,0x4b,0xd1, -0xab,0x9c,0x72,0xfc,0x32,0x97,0x2b,0x78,0x07,0x19,0x6d,0xfd, -0xf9,0x64,0x5b,0xd5,0xb7,0xd5,0x15,0x4a,0x68,0x39,0xb5,0x2c, -0x6f,0xf3,0x03,0xc8,0x1c,0x40,0x51,0x9f,0x99,0x6d,0x67,0x73, -0x38,0x9f,0xc1,0x76,0x5b,0xb2,0xba,0x27,0x07,0x67,0xc6,0xac, -0xce,0xb9,0x30,0x32,0x21,0xe0,0xa4,0x6f,0x5b,0x90,0xc3,0x52, -0xfd,0x49,0x8e,0x1e,0xc3,0xa2,0x02,0x6d,0x51,0xde,0x7c,0x11, -0x29,0x4c,0xe8,0x34,0x12,0xc3,0x7e,0x84,0x47,0xe8,0xff,0xc4, -0xf1,0x07,0x49,0x35,0xac,0xee,0x96,0xf3,0x47,0x3a,0x6c,0xfb, -0x1c,0x0f,0x03,0xbc,0x5a,0x3a,0xbb,0x21,0xd7,0xf3,0x7c,0xb4, -0x54,0xe3,0x03,0x9c,0x7b,0x19,0x3c,0x0b,0xa8,0x1f,0xa3,0x6a, -0xe3,0xaf,0x67,0xf0,0xa4,0xf2,0x36,0x59,0xbe,0xa2,0x73,0x41, -0x7d,0xb9,0x4a,0x68,0x7d,0xbd,0xd2,0xa2,0xca,0x30,0xbc,0x27, -0x56,0xcd,0xbf,0x09,0xfd,0x06,0x57,0x04,0x49,0x8b,0x0f,0xef, -0x3e,0xb2,0x9c,0x7e,0xe2,0x87,0x1f,0x57,0x08,0xde,0xf1,0xf9, -0xf0,0x8a,0x7a,0xe8,0xfe,0x68,0x6c,0xaf,0x57,0x8a,0x45,0x3e, -0x29,0x2f,0x62,0x9f,0x94,0xc9,0x8d,0x8c,0x35,0xd4,0xe7,0x03, -0xf4,0x24,0x20,0x27,0x18,0xc7,0xec,0x1a,0x89,0xe7,0x5c,0x10, -0x18,0xa8,0x6a,0x36,0xc4,0xe3,0x61,0xce,0xf3,0x39,0x63,0xe7, -0x07,0xb8,0xca,0x34,0xdb,0x24,0x6d,0x2d,0xfa,0x2d,0x59,0x94, -0x24,0xd8,0xf0,0x9c,0xe4,0x4a,0x3c,0x5a,0x70,0xa3,0x56,0xe1, -0x4f,0xf2,0x99,0x0e,0xf1,0xee,0x2e,0xce,0xda,0x76,0xed,0xf6, -0xe9,0x19,0x94,0xc0,0xa0,0xae,0x37,0xe9,0xba,0x2e,0xab,0xc2, -0x2d,0xf9,0x56,0xdc,0x26,0xab,0xf2,0x32,0x98,0xdd,0x2b,0x14, -0x44,0x47,0x61,0xb9,0x12,0xaa,0x8d,0xbb,0xcb,0xf1,0xc7,0xc7, -0xe8,0xc4,0x6d,0x3a,0x7f,0x93,0x94,0x57,0x09,0xcf,0x51,0xa1, -0xa4,0xb0,0xc8,0x3d,0x38,0x90,0xdf,0xd0,0x52,0x25,0x4c,0x0e, -0x2a,0x96,0x76,0xce,0x2b,0x84,0xd2,0x4e,0x16,0x49,0xa5,0xdd, -0xe3,0xbf,0x73,0x95,0xd5,0xd6,0x17,0xac,0xa7,0x58,0xa3,0xe4, -0xd1,0x1a,0x0d,0xf5,0x1c,0x62,0xc5,0x13,0x5b,0x6d,0xe9,0x2f, -0xab,0x95,0xba,0x92,0x5a,0x24,0x5f,0x8b,0xa5,0xa8,0xf6,0xad, -0x66,0x97,0xf6,0x62,0xbe,0x63,0xbd,0x5a,0x6b,0x49,0x54,0x6d, -0x7b,0x91,0x99,0x93,0x43,0xac,0x12,0x35,0xb9,0xfd,0xc1,0x44, -0xe0,0x47,0x91,0xdb,0x3f,0x98,0x71,0x70,0x70,0x63,0x1d,0xf1, -0x60,0x11,0x63,0x93,0x80,0x1e,0xa5,0x54,0x4f,0xf9,0x5b,0x73, -0x00,0x6c,0x30,0x44,0xb5,0xd7,0xeb,0x20,0xa8,0x89,0x02,0x5a, -0xb7,0x22,0x97,0x0c,0x5c,0x88,0xcd,0x92,0xcc,0xb9,0xc4,0xd0, -0x27,0xe8,0xb8,0x52,0x87,0x7f,0x78,0x1c,0xcb,0xda,0x55,0x8c, -0x92,0x05,0x6e,0x96,0xf1,0xdb,0x50,0x7c,0xba,0x8a,0xa8,0xd3, -0xdd,0x0f,0xa4,0x96,0xf3,0xc9,0xe4,0x61,0xd1,0xe5,0x1f,0xf9, -0xe9,0x37,0xd4,0xc0,0x18,0xfc,0x36,0xae,0xdc,0xff,0x07,0x75, -0x5b,0xd7,0x5a,0xca,0x9b,0x54,0xfc,0x92,0x17,0x92,0xed,0x99, -0x85,0x32,0xe4,0xe0,0xc5,0x31,0x03,0x5b,0x05,0xd9,0xf3,0x65, -0x7b,0x7c,0x64,0x1c,0xd1,0xd5,0x23,0x1a,0xeb,0xf5,0xbe,0x04, -0xfe,0xc3,0x06,0xbd,0xd6,0x58,0x9a,0x48,0xee,0xcd,0xe4,0x2e, -0x27,0xb2,0xa0,0xc8,0xf6,0x46,0xd5,0x8c,0xc8,0xa4,0xca,0x32, -0x5b,0x3e,0xdf,0xef,0xda,0x12,0x29,0x95,0x4d,0xdb,0xc6,0x91, -0x59,0x38,0x44,0x8f,0x21,0x1c,0xbd,0x83,0x8c,0xed,0x2a,0xcc, -0xf2,0x9d,0xfc,0x46,0x61,0x0b,0x23,0x5b,0x3a,0x19,0x07,0x23, -0xf7,0xdf,0x49,0x2d,0xe9,0xe0,0xb7,0xfb,0x4a,0xf2,0x86,0x3b, -0x0b,0x03,0x0b,0xa9,0x3a,0xde,0x6d,0xf7,0x45,0xde,0x53,0xe9, -0xfd,0xa8,0x12,0xb3,0x1e,0x7f,0xdc,0x5c,0x35,0xaa,0xb8,0x06, -0x5f,0xf6,0x97,0xb9,0x8f,0x91,0x63,0x06,0x71,0xed,0x25,0x47, -0x00,0x47,0x42,0xce,0xfb,0x44,0x98,0x22,0x51,0x46,0xb0,0x07, -0xe5,0xaf,0xe9,0x5f,0x8b,0xa5,0xe5,0x63,0x1b,0xfe,0xe1,0xec, -0xf2,0x8e,0xc5,0xa4,0x64,0x68,0x67,0x6d,0x45,0x1d,0x83,0xea, -0x16,0x42,0x1b,0x65,0x75,0xb6,0xb4,0x96,0x9e,0x8d,0x4a,0x38, -0x2f,0x26,0xa7,0x45,0x83,0x26,0x6b,0x41,0x96,0x66,0xba,0xf8, -0x31,0x32,0x2e,0x78,0x70,0x08,0x1a,0x79,0xef,0x58,0xf1,0x5a, -0x2d,0xb4,0x15,0x16,0x82,0xb8,0x53,0x2b,0x8f,0x44,0xd9,0x23, -0xba,0xf6,0xe2,0x58,0x45,0x57,0x02,0x54,0xba,0xb0,0x90,0x93, -0xbb,0x02,0x77,0x05,0xe6,0xc9,0xaf,0xad,0xb0,0x7b,0x14,0x02, -0x83,0x03,0xc7,0x54,0xc9,0x56,0xce,0xe8,0xfc,0x91,0x4f,0x80, -0x12,0x5c,0x37,0xc4,0x2c,0x08,0x96,0xa3,0x64,0x99,0xe7,0x15, -0xa9,0x33,0x73,0x4c,0xd1,0x8f,0x08,0x2a,0x8f,0xe8,0x7f,0xe4, -0x96,0xb9,0x45,0x54,0x12,0xa2,0x50,0x9a,0xd9,0x5c,0x72,0x02, -0xf0,0x91,0xca,0xb1,0xa0,0xf9,0xdc,0x2b,0xc9,0x9f,0x29,0x4a, -0x7b,0x1e,0x48,0x97,0x9a,0xe3,0x55,0xb2,0xea,0xd0,0xfa,0x11, -0x33,0xcd,0x76,0x25,0xbf,0xad,0xa2,0xfd,0x12,0xae,0x9f,0xa0, -0x68,0x33,0x1e,0x8d,0x3c,0xad,0x3a,0xf4,0x69,0x55,0xb2,0x72, -0x57,0x1e,0x80,0x39,0xb7,0xf2,0x70,0xd7,0x03,0xee,0x84,0x4e, -0x79,0x66,0xf5,0x01,0x55,0x54,0x95,0x2b,0x59,0x47,0xbc,0xe1, -0xb1,0xfd,0x22,0x10,0x15,0x8b,0x09,0x17,0x03,0x3b,0x3d,0x29, -0x41,0x21,0x1c,0x2a,0xdc,0x45,0x05,0xd7,0xf1,0x1b,0x57,0x00, -0xfb,0xcc,0xfe,0xbb,0x26,0x61,0x87,0x3e,0x5f,0xba,0x6d,0x79, -0xbf,0x89,0x98,0x0b,0xd9,0x53,0x31,0x4e,0xc0,0x74,0x46,0x91, -0xdc,0xc5,0x6f,0xd2,0xf9,0x12,0xc7,0xb0,0x0e,0xb8,0x68,0xe4, -0x87,0x6d,0x9e,0x60,0x54,0x3f,0x2c,0x87,0xd6,0x95,0x39,0x5f, -0xee,0xc3,0xd4,0xc5,0x33,0x97,0xbb,0xd5,0x50,0x93,0x12,0x97, -0x95,0x06,0x5d,0xd6,0xc6,0x14,0x76,0xd9,0xb4,0x53,0x60,0x03, -0x00,0xa7,0x06,0xf9,0x23,0xcb,0x18,0x39,0x63,0x20,0x7d,0x4c, -0xf3,0x26,0x99,0x8d,0xe6,0xcb,0x6f,0xa9,0xb1,0xd8,0x01,0xb5, -0xc7,0x64,0x30,0x4e,0x57,0x35,0x51,0x9b,0x66,0xf8,0x67,0xfe, -0x16,0xfe,0x7e,0x45,0x7f,0xe7,0xf0,0xef,0x9b,0x5a,0x1f,0x17, -0x09,0xa3,0x34,0x5b,0xe0,0xe5,0xa6,0xb3,0x50,0x50,0x8e,0x29, -0x55,0x9e,0x32,0x5d,0x94,0xef,0x66,0x76,0x2e,0x6f,0xa3,0x82, -0xeb,0x15,0x9a,0xa7,0x72,0xae,0xc9,0x87,0x8c,0xa7,0x5e,0x8d, -0xad,0x8b,0x4c,0xe0,0x3b,0x6b,0xf9,0x62,0xed,0x0d,0x6d,0x0f, -0x6e,0x74,0x47,0x89,0xb7,0xbe,0xc9,0x2c,0x91,0xb7,0xb2,0x29, -0x59,0xa5,0x68,0x5f,0xbe,0xb5,0x66,0xcd,0xa7,0x1b,0xb5,0xa8, -0x56,0xa3,0x4f,0xbf,0xaf,0xa2,0x00,0x63,0xc2,0x63,0x0a,0x5f, -0xdb,0x19,0xd3,0x50,0xb4,0x08,0xa1,0xdb,0xd1,0x5c,0x46,0x40, -0x5f,0xb5,0x93,0xe3,0xae,0x8a,0x83,0xd0,0x4f,0x5a,0x5d,0x1a, -0x16,0x08,0xd3,0x6f,0xca,0x7b,0x3f,0xbf,0x5d,0x4c,0x33,0xc4, -0x22,0x31,0xad,0xd2,0xeb,0x0a,0xc8,0x60,0x3b,0x1c,0x0d,0x4b, -0x9e,0xe7,0x01,0x8d,0x96,0x75,0x1a,0x4f,0xa0,0x4d,0xbe,0x79, -0x9c,0xe9,0xfb,0x6a,0xa2,0xef,0xe0,0xa0,0xa4,0x4e,0xc4,0xe6, -0xdb,0x43,0x71,0xcb,0x87,0x00,0xc8,0x7f,0x32,0xf9,0x11,0xe6, -0x4b,0x5c,0x6b,0x63,0x6d,0xd4,0xb0,0x1d,0x32,0x51,0xc2,0x61, -0xcc,0xc4,0xda,0xeb,0xd7,0xff,0x7a,0x50,0xf3,0xb5,0xf4,0x99, -0xac,0xe8,0xa8,0xf7,0xfa,0xf5,0xeb,0xbf,0xbe,0xfe,0xd7,0xd7, -0x87,0xaf,0xeb,0xaf,0xbb,0xaf,0x9f,0x5e,0xf7,0x5e,0xf7,0x5f, -0x7b,0xaf,0xfd,0xd7,0xcd,0xd7,0xef,0x5e,0x6f,0xfb,0x47,0xb7, -0x74,0x83,0x2a,0x9d,0xa5,0xc1,0x10,0x6b,0x5e,0xf3,0x4d,0xea, -0xf5,0x75,0x37,0xb7,0xcf,0x14,0xba,0xe7,0xf5,0xe2,0x48,0xc3, -0x46,0x3a,0x6f,0xdb,0xde,0x51,0x4c,0x1f,0x91,0x49,0x56,0x21, -0x1b,0x4d,0xb6,0x5c,0x14,0xa9,0xcc,0xbe,0x41,0x6a,0xd2,0xf8, -0xc2,0x97,0x4c,0x46,0x2c,0x06,0x09,0xe3,0x6f,0x6f,0xd6,0xfc, -0xf9,0x21,0x59,0x6e,0x5e,0x11,0x08,0x54,0x97,0xf9,0x56,0x91, -0x4f,0xa1,0x33,0xf7,0x29,0x05,0x70,0x6e,0xc1,0xaf,0x50,0xf4, -0x93,0x65,0x54,0x34,0xad,0xe8,0xe9,0xc1,0xfe,0x8a,0x1c,0xe2, -0x95,0x58,0x5f,0xc0,0x70,0xa7,0xe2,0x99,0x42,0x5f,0xf3,0xfb, -0x79,0xd4,0xd6,0xcc,0x3b,0x13,0x99,0xe3,0xc8,0xce,0x26,0x40, -0xae,0xf4,0x01,0x62,0xdb,0x29,0x71,0x1f,0xf7,0x2a,0x7d,0xfb, -0x76,0x92,0xa0,0xc3,0x84,0xdd,0x4c,0x62,0x38,0x4a,0xea,0x05, -0xfd,0xa7,0x27,0x76,0xda,0xf0,0x3c,0x9b,0xb8,0x5c,0xf3,0x61, -0x96,0x02,0xbb,0x5e,0xe1,0x22,0xcd,0xc2,0x54,0xe4,0x57,0x16, -0xc9,0x02,0xb9,0xf6,0xfb,0xbc,0x08,0x90,0xae,0x8e,0xca,0xad, -0xb6,0x74,0xf6,0x77,0xf3,0xb9,0x51,0xab,0x3a,0xf5,0x5b,0xcb, -0x80,0xc2,0xa0,0x32,0xca,0x09,0xa7,0xbc,0x52,0xf8,0x66,0x81, -0x2f,0x8a,0x28,0x5e,0x82,0x3c,0x0c,0xc8,0x3b,0x9f,0x80,0x07, -0xf2,0xa5,0xc5,0x6f,0xf4,0xa8,0xf2,0x68,0x3e,0x55,0xc5,0x22, -0xbb,0x96,0xeb,0x4c,0x8b,0x9d,0xc2,0xaa,0xb7,0x58,0xf7,0x7c, -0xd6,0x4a,0x0a,0x43,0x6b,0xe8,0x7b,0x3c,0x46,0x7d,0xa7,0x11, -0xf3,0xd4,0xab,0x5e,0xd1,0x01,0xae,0xae,0x14,0xcd,0xb1,0x40, -0x96,0xd0,0x86,0x27,0x62,0x07,0x82,0x84,0x0a,0x03,0xb4,0x5b, -0x59,0xee,0xc9,0xee,0x8c,0x9d,0x11,0xe6,0x26,0x04,0xf2,0x11, -0xc6,0x39,0xb8,0x50,0xea,0x7a,0xa6,0x0a,0x59,0x3b,0x01,0xa8, -0x16,0xec,0xf6,0x23,0xa8,0xf8,0x74,0x70,0x50,0x53,0x19,0xe8, -0xce,0x82,0x93,0x14,0x2e,0x5d,0xc6,0xf5,0x17,0xe6,0xc3,0x3e, -0xd5,0xa9,0x9a,0xd4,0xe3,0xb6,0x44,0x06,0x34,0x89,0xce,0xa0, -0x2c,0xe9,0x56,0x57,0x72,0xcb,0x7b,0xf6,0x7d,0xfb,0x92,0x94, -0xc1,0x7b,0x74,0xa7,0x82,0x70,0x3a,0x49,0xec,0xdf,0x21,0xcc, -0xb7,0x52,0x81,0xd0,0x26,0x5e,0xfb,0x75,0x4e,0xb9,0xf4,0x90, -0x6b,0xc7,0x32,0xe7,0x94,0x36,0x97,0xd5,0x80,0x9b,0x65,0xbc, -0x86,0x89,0xde,0x65,0x6f,0x6e,0xdb,0xb5,0xbb,0xd5,0x6a,0xd1, -0x3e,0x3a,0x7a,0x7c,0x7c,0x6c,0x3e,0x1e,0x37,0xe7,0xcb,0xdb, -0xa3,0x56,0x10,0x04,0x47,0x90,0x57,0x13,0x6b,0x34,0x04,0x2c, -0x03,0x09,0x2f,0x2f,0x2f,0x8f,0x28,0x17,0x80,0x26,0xe9,0x6c, -0x5c,0x0d,0x84,0xb9,0x00,0x54,0x8e,0xe7,0x87,0xaf,0xbe,0x44, -0xb0,0x8b,0x23,0x72,0x18,0x81,0xe6,0x40,0x04,0x3a,0xcb,0x2a, -0xe9,0xa2,0xdc,0xa3,0xda,0x96,0x0e,0x60,0x60,0x76,0xe3,0xc5, -0x67,0xdb,0x6a,0x97,0xf8,0xf9,0x21,0x9e,0xa4,0x37,0x9b,0xb6, -0x63,0x28,0xa4,0x16,0x30,0xf8,0x6c,0x16,0x30,0x6d,0x58,0xc0, -0xc8,0x53,0x20,0xcc,0xd0,0x1f,0x0b,0xf0,0x42,0xc6,0xca,0xc9, -0xad,0x58,0x78,0xd5,0x93,0xcb,0xa0,0x8f,0x5f,0x84,0x45,0x49, -0x7e,0x15,0xc1,0xf8,0xfc,0xee,0x3b,0x6a,0xa9,0x4d,0x73,0x8f, -0xb3,0xfa,0x6c,0x0c,0xd5,0x46,0xc4,0x5b,0xfe,0x5b,0x35,0x38, -0xd0,0x99,0x5c,0xb9,0xad,0x57,0xd9,0x19,0x7d,0x8b,0x92,0x6b, -0x4c,0xa8,0x99,0x5c,0x0c,0x63,0x48,0x5e,0x49,0xf2,0xf1,0xc9, -0x18,0x74,0x52,0x3b,0x89,0xd5,0x4d,0xc9,0x56,0xe6,0x26,0x31, -0xa0,0x49,0xd4,0x76,0x49,0x4c,0x6f,0x1d,0x23,0x2f,0x8f,0x59, -0x43,0xf6,0x5d,0x06,0xd2,0x6f,0x17,0x40,0x19,0x9b,0x74,0xd5, -0x2e,0x77,0x1c,0x94,0x44,0xb4,0x90,0x6b,0x55,0x67,0x04,0x93, -0x0f,0x3d,0xde,0x41,0x21,0x98,0xdc,0x2b,0xf9,0xee,0x0a,0xcc, -0xde,0x18,0x55,0xa0,0xb0,0x18,0x56,0xa9,0x7f,0x68,0xfd,0x64, -0xbb,0x8c,0xfe,0xdd,0xb9,0x0c,0xda,0xc3,0x33,0x92,0x04,0xc5, -0xa6,0x9e,0x0f,0x5c,0x9b,0xa4,0x2b,0xef,0xe8,0xaf,0x4f,0xaf, -0xb3,0xfa,0xd1,0xad,0xef,0xe7,0x6d,0x45,0x78,0x4f,0x49,0x9c, -0x24,0x0a,0xbf,0xd4,0x56,0x9d,0xda,0x00,0xe4,0xc6,0x7b,0x21, -0x77,0x6c,0xf3,0xd9,0x2a,0x4e,0x67,0x19,0xd5,0x42,0xc7,0x6f, -0xf2,0x1b,0x88,0x70,0x6b,0x1d,0xfd,0x6b,0x7c,0x4e,0xcf,0xd4, -0x08,0x7b,0xcd,0xb7,0xb1,0x3a,0x8c,0x94,0xfc,0x91,0x6e,0x69, -0x10,0x3b,0x7f,0xd1,0x2c,0xd9,0xaa,0x6a,0x92,0x5d,0xf3,0x22, -0xd8,0xbf,0x97,0x25,0xe6,0xdf,0xd8,0xd1,0x0a,0xcb,0x3e,0x7d, -0x4d,0x36,0x83,0xcf,0x98,0x16,0xf2,0xc7,0x70,0x85,0x4d,0x19, -0xf0,0xe5,0xf8,0x6a,0xe6,0x74,0xfd,0x0b,0xb7,0xeb,0x5b,0x57, -0x33,0xd0,0x26,0xcc,0xe7,0x5a,0x8d,0x27,0x1c,0x85,0xf6,0x39, -0x56,0x10,0x61,0x1a,0x9a,0x79,0xa1,0xde,0xfa,0xaa,0x55,0x0e, -0x53,0x70,0xef,0x03,0xf4,0x98,0xd9,0x40,0x1a,0xa3,0x42,0xe7, -0x7e,0x36,0x9f,0x2e,0xa0,0x6b,0x47,0x64,0x94,0xea,0x39,0x82, -0x88,0xf3,0x34,0x82,0x28,0x65,0x45,0x47,0x43,0x3c,0x3a,0x3b, -0xaa,0x3e,0x20,0xfe,0x19,0x96,0x33,0xb5,0x65,0xdc,0xab,0xe4, -0xfc,0x62,0x87,0x5d,0xe7,0xfb,0x0f,0x33,0x9b,0x3c,0x6e,0x5a, -0x0f,0xd3,0xd5,0x85,0xeb,0x1e,0xb2,0xa7,0xe8,0xf9,0x8d,0xc2, -0xa7,0xd1,0xec,0x23,0x7d,0x68,0x04,0x1f,0x55,0x1c,0x21,0x15, -0x3e,0xa0,0x32,0x15,0x3e,0xef,0x6d,0x5a,0x1e,0x19,0x49,0xb7, -0xbb,0xa5,0x0e,0x40,0x3b,0x05,0xff,0xcc,0xbc,0x41,0x7e,0xd3, -0xad,0xd5,0xda,0x6f,0xb6,0x6d,0xf9,0x96,0xc7,0x5b,0x28,0x05, -0xb2,0xd1,0xde,0x95,0x2f,0x2f,0x81,0xda,0x56,0xd7,0xd8,0xf9, -0x55,0xac,0xc1,0x35,0xc6,0xff,0x5e,0xd6,0x50,0xdc,0x99,0x3f, -0x7d,0xff,0xd5,0x97,0xef,0xc1,0x18,0x53,0xa6,0x8c,0x2d,0x26, -0xb7,0x8c,0x29,0x3a,0xb7,0x72,0xea,0x5f,0x2c,0x92,0x59,0x6e, -0x5a,0x32,0xdb,0x5e,0x9e,0xb0,0x6d,0x35,0x88,0x26,0xf9,0xf3, -0xa5,0x9c,0xb7,0xa9,0x26,0xce,0xf4,0x8a,0xbb,0x34,0xca,0x65, -0xfc,0x9f,0xe1,0xf7,0x05,0x3c,0x8f,0x97,0x32,0x07,0x0f,0xf9, -0x2b,0x08,0x84,0x09,0xc7,0xf9,0xa4,0x90,0x86,0xc0,0x20,0x81, -0x21,0xb8,0x3f,0x9d,0x0c,0x5e,0xb1,0x15,0x93,0xb8,0xf6,0x69, -0x0c,0xd3,0xf2,0x07,0x2a,0xb0,0xa3,0x35,0xb2,0xbe,0xf7,0x6e, -0x2a,0x1f,0xff,0x96,0x39,0xca,0xd3,0xa2,0x98,0x93,0x38,0x5e, -0xb6,0xf3,0x2a,0xc0,0xda,0xe5,0xf1,0xe3,0xc1,0x01,0xff,0x4a, -0xbc,0xdc,0x07,0x74,0x2a,0x55,0x4d,0x02,0x6e,0x0f,0x72,0x23, -0x44,0xd0,0xc5,0xb2,0xbe,0x41,0xa2,0x23,0x4f,0xde,0x29,0xd1, -0xdf,0x3f,0x52,0xb4,0x1e,0x5a,0x0f,0x0b,0xbd,0xd1,0xc0,0xc4, -0xdc,0x6e,0x06,0x36,0x43,0x1a,0x5e,0x25,0xcd,0x82,0x48,0x47, -0xb0,0xc3,0x00,0x53,0xe2,0x61,0x81,0x9f,0x20,0xd1,0x87,0x28, -0x8e,0x45,0x11,0xde,0xf5,0x26,0xcb,0xb2,0xf4,0x75,0xba,0xca, -0x27,0xcf,0xe8,0xc8,0x8e,0xa8,0x76,0xee,0x2e,0x91,0x30,0xba, -0x25,0xe3,0x0b,0x45,0xcb,0x09,0x1d,0xb6,0xba,0x3c,0x47,0x15, -0xd6,0x76,0x36,0xb4,0x70,0x92,0x8b,0x18,0xfb,0x6e,0xd2,0xec, -0xb4,0xf4,0x4e,0x2d,0xbf,0xe5,0xf2,0x73,0x14,0x50,0xdc,0x63, -0x85,0xc3,0xef,0xea,0xa6,0x90,0x7d,0x14,0x3c,0xb4,0x73,0xe0, -0xce,0x55,0x23,0xed,0xd2,0x0c,0x79,0xfa,0xce,0x9f,0x91,0xb9, -0xa4,0x4e,0xcb,0x48,0xd5,0x9d,0xa1,0x3b,0xcc,0xf4,0x0f,0x9b, -0xc2,0x0a,0x6e,0x4e,0x8e,0x0a,0xfb,0x52,0x16,0x3d,0xca,0x58, -0x5d,0xa6,0x08,0x17,0xc5,0x4d,0x3d,0x49,0x49,0xdb,0xe5,0xb3, -0xdb,0xfe,0xa7,0x27,0xcf,0x74,0x32,0x62,0x72,0x46,0x2b,0x96, -0xd4,0x1e,0xfd,0x09,0x99,0x9f,0xeb,0xb5,0x22,0x7b,0x08,0x2c, -0xcf,0x73,0x1e,0x0b,0x55,0x9c,0x42,0x03,0xee,0xea,0x3e,0xca, -0xe4,0xf6,0xdb,0xe9,0x29,0xbd,0x2f,0x37,0xf6,0xe4,0x06,0x5d, -0xc0,0xf8,0x5c,0x91,0x28,0xf2,0x1a,0x53,0xbb,0x5e,0x39,0xd7, -0x44,0x09,0x83,0xfd,0xf6,0x7b,0xb1,0x8a,0x0d,0xd4,0xa7,0xf2, -0x2a,0x6c,0x9f,0xa2,0x36,0x75,0xb2,0xb8,0xba,0x49,0x2b,0x6d, -0xfc,0xd6,0x20,0x6d,0x32,0xbd,0x91,0x45,0xb6,0x35,0x7a,0xed, -0x83,0x06,0x0b,0xc2,0x4e,0xd6,0x15,0xd8,0x89,0xf9,0x53,0x0a, -0x46,0xc9,0x52,0x6f,0xb0,0x2b,0x05,0xc2,0xe9,0x16,0x7e,0x56, -0x14,0x12,0x5c,0xe1,0xe7,0x2f,0x01,0xad,0x43,0x1f,0x79,0x06, -0x63,0x9f,0xd9,0x94,0x06,0x40,0x7f,0x67,0xed,0x74,0x8c,0xd6, -0x71,0x8e,0xdd,0x82,0xbe,0xaf,0xed,0x7c,0x2b,0x8f,0x71,0x08, -0x85,0xea,0x53,0x73,0xec,0xe5,0x98,0x01,0x6e,0xa5,0x09,0x4d, -0x92,0x3b,0x7d,0xa2,0x24,0x0f,0x4f,0x00,0x25,0xbf,0x9d,0x93, -0x1a,0x4c,0xc7,0x76,0x16,0x52,0xb1,0x41,0x55,0xcb,0x19,0xc7, -0xc4,0xde,0xa8,0x79,0x6e,0x50,0x2a,0xf7,0x07,0x96,0xf2,0x10, -0xf9,0x4e,0xb1,0x75,0x5d,0x89,0xf9,0x7d,0x15,0x4a,0xe3,0x41, -0x9b,0x3b,0x8f,0x5b,0x5b,0x9c,0x05,0x29,0x7d,0xab,0xba,0x98, -0x9a,0x57,0x02,0x04,0xc9,0x0a,0x66,0xc7,0x24,0x67,0x7f,0x59, -0xb1,0xcf,0x3a,0x50,0xaf,0xc0,0xcd,0x31,0xa6,0x24,0xbe,0x5d, -0x95,0x5f,0x39,0xc7,0xb2,0x9d,0xbd,0x75,0x21,0x4e,0xef,0xb9, -0x53,0xc8,0xaa,0xb3,0x48,0xd3,0x79,0x7a,0x5f,0xc8,0xe5,0x41, -0x9e,0x24,0x62,0x47,0x4e,0x54,0x84,0x60,0xae,0xc3,0x3e,0x9f, -0x0c,0x72,0xc7,0x93,0xd3,0xab,0xfb,0x0e,0xfa,0x90,0xfd,0xc7, -0x1c,0x37,0x07,0xf9,0xc3,0x4d,0xd4,0x27,0xe8,0xfb,0xb6,0x30, -0x2c,0x98,0xd4,0xea,0x93,0x4b,0x00,0xc9,0x1d,0x0b,0xa3,0x46, -0xfc,0xc0,0x63,0x61,0xb2,0x56,0x28,0xbb,0x4e,0x7d,0xfe,0xfc, -0xd6,0x1c,0xff,0x5b,0x87,0xb8,0x76,0x13,0x51,0x99,0xe0,0x5e, -0x4c,0xb5,0xad,0xd3,0x68,0xe0,0x59,0x60,0x99,0x26,0x40,0xb8, -0x83,0x03,0xfc,0xfb,0x22,0x62,0x6d,0x80,0xcf,0xaf,0xd2,0xc1, -0x04,0x76,0xa5,0x9c,0x61,0x8f,0x33,0x77,0x1d,0x4a,0x3c,0xc2, -0x2f,0x9b,0xb9,0x3a,0xe7,0xfc,0x96,0xcd,0x12,0x2a,0x8e,0x29, -0xe6,0xf6,0xba,0x7a,0x08,0xdb,0x79,0xfc,0xac,0x14,0xcf,0xc3, -0xcd,0x73,0x6e,0xcd,0x0c,0x25,0x3e,0xd3,0x79,0x15,0xfb,0x9c, -0xe7,0x39,0x28,0xd9,0xc5,0x66,0x3c,0x56,0xbd,0x8e,0x11,0x02, -0xf5,0x8c,0x57,0xd9,0x71,0xb8,0x1a,0xb6,0x88,0x07,0x61,0x19, -0xc4,0xc3,0x71,0xe9,0x0d,0x11,0x07,0x55,0x22,0x32,0xcd,0x2e, -0x42,0x47,0x4e,0x52,0x45,0x77,0x1e,0x95,0x57,0x2f,0x98,0xb1, -0x54,0x19,0x1d,0x6c,0x34,0x75,0x9b,0x99,0x33,0x79,0x8b,0x45, -0x92,0x48,0x5d,0x35,0x33,0x12,0xc1,0x31,0xc2,0xa6,0x62,0x68, -0xc6,0x7b,0x92,0x4a,0x0e,0x4c,0x17,0xab,0x12,0x97,0x06,0x2f, -0xac,0xed,0x5f,0x55,0x51,0x92,0xbf,0x52,0xa1,0xaf,0x52,0x0a, -0x55,0x97,0x16,0x95,0xc3,0xda,0x1c,0x51,0x6a,0xf1,0xc7,0xeb, -0x72,0x94,0x4d,0xb5,0x69,0x83,0xe7,0xad,0xe5,0x32,0xab,0x52, -0x4c,0xd3,0xb7,0x0e,0xb1,0x7c,0x72,0x16,0x74,0x76,0xed,0x8d, -0xea,0xf5,0xd9,0x16,0x86,0x43,0xf1,0x5e,0x90,0xe7,0xcd,0x6f, -0xdd,0x6b,0x40,0xeb,0x82,0xb0,0x6a,0xa6,0x15,0x45,0xa8,0xa6, -0xf3,0x91,0x76,0x55,0x0d,0x62,0x47,0x9e,0xda,0x7c,0xef,0xda, -0x99,0xef,0x2c,0xcf,0x22,0xb0,0x43,0x3c,0x76,0x96,0x9e,0x49, -0xb3,0xd4,0x0a,0x01,0xd9,0x59,0x96,0xc4,0xbe,0x7a,0x44,0xec, -0x2c,0x4b,0x1d,0x5a,0xdd,0xd7,0xbb,0xcb,0x56,0xde,0xce,0x2a, -0x09,0x2d,0xdc,0xd0,0x3e,0x2c,0x76,0xce,0xa8,0x62,0xc7,0xc5, -0xab,0x2c,0x5a,0x98,0xe1,0x78,0x79,0x21,0xde,0x77,0xa2,0xcc, -0x27,0xfc,0x0e,0xb7,0xb1,0x92,0x42,0x5c,0x44,0x96,0x5f,0xcc, -0x16,0x16,0xce,0x3b,0x2e,0x69,0xab,0xee,0x61,0x7f,0xf3,0x35, -0x6c,0xbe,0x17,0xf7,0x39,0xd3,0x29,0xfd,0xfc,0xa3,0xec,0x20, -0x87,0x70,0xbf,0xb4,0x67,0x44,0x3a,0xe3,0xf0,0x77,0x9e,0x27, -0xe5,0xbf,0xe2,0x97,0x95,0x57,0x9e,0xa9,0x9a,0x98,0x6d,0x25, -0x6a,0xd3,0xbd,0x3a,0x4e,0x65,0x30,0x58,0x55,0xe2,0x25,0x07, -0x64,0x7b,0x39,0x7a,0x7a,0xaa,0xd7,0xdd,0x9c,0x91,0x2b,0x89, -0x56,0x2d,0x65,0x18,0x9e,0x9e,0xde,0x51,0x54,0x38,0xe5,0x9b, -0x04,0x3d,0x43,0x61,0x6c,0x38,0x80,0xc5,0xdf,0xeb,0xe1,0xc3, -0x20,0x1d,0xbe,0x9c,0x7d,0xf3,0x20,0x43,0x5a,0x61,0x0c,0x0e, -0x15,0x15,0xae,0x75,0x1a,0x6c,0x77,0x49,0xfa,0x0e,0x51,0xee, -0xfc,0xf6,0x0b,0xeb,0xd1,0x8e,0xc0,0x7c,0x56,0xc0,0x3c,0xb1, -0x7b,0x5d,0x67,0x85,0x74,0xd4,0xe4,0x62,0x54,0xc7,0xea,0x93, -0x43,0x10,0x8c,0xe5,0xc3,0xa2,0x6c,0x9b,0x50,0x71,0xd0,0xae, -0x8b,0xf8,0x5b,0xa3,0xf1,0x23,0x67,0xd5,0xc0,0x13,0x90,0xe4, -0x4e,0xf1,0xde,0x00,0x77,0xf2,0x1a,0xa5,0xa7,0xf4,0xa8,0xb2, -0xf8,0xe1,0x4b,0x4b,0x6d,0x6d,0x59,0xd0,0x10,0x25,0xc6,0x41, -0xee,0x0e,0x8e,0x4a,0xf4,0x6d,0xea,0x1c,0x93,0x86,0x19,0xed, -0x6b,0x2d,0x0a,0xf5,0xea,0xa3,0x94,0xd2,0xac,0x6b,0xdb,0xf4, -0x70,0xe9,0x22,0xb9,0x66,0xc1,0xf2,0xe1,0xf4,0x16,0x26,0x62, -0x34,0xdb,0x31,0xb6,0x39,0x25,0x56,0x51,0x7e,0x85,0x7d,0x4e, -0xa5,0x55,0xa1,0x18,0xc6,0x0b,0xa0,0x32,0xf9,0xa0,0xbb,0x35, -0xb6,0x3f,0xe3,0xbb,0x35,0x85,0x30,0x7a,0x11,0xb2,0xf0,0x4b, -0xc4,0x64,0xd4,0x87,0x70,0x15,0xc2,0x43,0xeb,0x3f,0x82,0x24, -0x13,0xc4,0x9e,0x7c,0xe9,0x1b,0x53,0xc2,0x1d,0x77,0x6b,0xde, -0xcc,0xbe,0xcb,0xed,0xc1,0x36,0x12,0xb6,0x78,0x75,0x32,0x0f, -0x84,0x01,0x34,0x6b,0x5e,0x77,0x24,0x3a,0xa0,0xea,0x99,0xdb, -0xa2,0x4a,0xc6,0x94,0x74,0x03,0x00,0x7f,0x41,0xfb,0xad,0x2c, -0x62,0xb3,0x73,0xef,0xdd,0x74,0xfe,0x90,0x25,0xa4,0x54,0xdb, -0x35,0x7a,0x9e,0xbf,0x49,0x96,0x35,0x41,0x8f,0x93,0x24,0x7e, -0x93,0xa8,0xe4,0x87,0x55,0x6d,0xeb,0xf6,0x92,0xc1,0x56,0xf2, -0x09,0x80,0xff,0xae,0x86,0x4d,0x1a,0xcb,0xb8,0x28,0xaa,0xbb, -0x49,0x3b,0x94,0x61,0x50,0x27,0x75,0xbe,0xf9,0xce,0x86,0x0c, -0x21,0xaf,0x47,0xcb,0xf8,0x96,0x4d,0xc5,0xa2,0xe7,0x8d,0xeb, -0xf8,0x32,0x56,0xd4,0x80,0xe0,0x25,0x17,0xaa,0xd1,0xcc,0x60, -0x50,0xbd,0x1c,0xc1,0x62,0x12,0x1b,0x8f,0x8d,0xb2,0xf7,0x43, -0x74,0xdb,0x6d,0x19,0xb9,0xa3,0x85,0x32,0xc2,0x7c,0x8b,0xc1, -0xc6,0x4d,0xbe,0xc6,0xf6,0x8a,0x2c,0x7f,0x3d,0x8b,0xcd,0x04, -0x7e,0x3d,0x78,0xb8,0x3d,0x39,0x09,0x2e,0x8e,0xa3,0xa3,0xbf, -0x24,0x83,0xff,0x9e,0xae,0xa4,0xaf,0x6e,0x73,0x99,0x3a,0x8b, -0xdf,0xa4,0xb7,0xb4,0xa3,0x42,0x2a,0x3f,0xbd,0x45,0xe9,0x47, -0x0f,0x2f,0x44,0xd6,0x6a,0xfe,0xe0,0xda,0xff,0x99,0x8a,0x65, -0xd6,0xee,0x29,0x53,0x95,0xcf,0x53,0xa9,0x10,0xfb,0x0a,0x4d, -0x57,0xeb,0x0a,0x95,0xe3,0x3a,0xa7,0xa4,0x54,0x79,0xed,0x80, -0x1c,0x88,0xaa,0xf8,0xc1,0x80,0xfa,0x33,0x64,0xcc,0x6d,0xa6, -0x23,0xa8,0x3a,0xbd,0x49,0x61,0x54,0x51,0xb6,0x95,0xc0,0xf1, -0xdc,0xb7,0x7e,0xbb,0xc7,0x5e,0x2f,0x94,0x3d,0x77,0x13,0x3b, -0xa7,0xf4,0x93,0x28,0xcc,0x50,0xd7,0x5f,0x90,0xc3,0xe2,0x08, -0x8c,0x9c,0x51,0x11,0x29,0xaa,0xf8,0xee,0x53,0x36,0x55,0xc8, -0x41,0x47,0x39,0xdf,0x24,0xd8,0x9e,0x44,0x31,0x31,0x1d,0xd9, -0x46,0x86,0x32,0x78,0xe6,0xf0,0x0e,0x3f,0xaf,0x1c,0x7d,0xcf, -0x5c,0x41,0xff,0x58,0x86,0xf8,0x5c,0xf9,0xc5,0x5c,0x4e,0x50, -0xac,0x23,0xdd,0x98,0xc3,0x8a,0xe3,0x32,0xd3,0x57,0x0e,0x17, -0x74,0x13,0x17,0x1a,0x78,0x61,0x33,0x0c,0x83,0x8a,0x6e,0xd1, -0x10,0x73,0xeb,0x70,0x80,0x63,0x5c,0xc2,0x0c,0xaa,0x2a,0x15, -0x38,0x5e,0x04,0x0c,0x87,0xb2,0xe0,0xd2,0xe6,0x7b,0x32,0x00, -0x1a,0xa9,0xeb,0xa3,0x28,0x4f,0x30,0xb5,0xf8,0x25,0x06,0x83, -0x5a,0x47,0x0b,0x74,0xa4,0x3e,0x5f,0xa6,0xb7,0xe9,0xec,0x9a, -0x7c,0xf4,0x6d,0x20,0x29,0x34,0x49,0x21,0xec,0x9d,0x80,0x0e, -0x60,0xcd,0x53,0x34,0x5a,0x3f,0x8d,0x36,0x42,0xe6,0x44,0x0b, -0xc6,0x73,0xed,0xbd,0x43,0x45,0xd4,0xae,0x31,0xdb,0xd7,0x6d, -0xc4,0x58,0xe7,0x78,0x75,0x88,0x70,0xd3,0x46,0x7c,0x2a,0x21, -0x84,0x1a,0xd6,0xed,0xd1,0x1a,0xea,0x69,0x8f,0x36,0x76,0x94, -0x4a,0x68,0x12,0x91,0xfc,0x88,0xdd,0x89,0xf4,0xd7,0x6b,0xcd, -0x5a,0x1d,0x91,0x4a,0x53,0x03,0x48,0x06,0x98,0x5c,0xaa,0xc0, -0xc7,0xef,0x60,0xa4,0xe1,0x92,0x50,0x12,0x8a,0xca,0x46,0xf6, -0xaa,0xf4,0x01,0x1b,0x45,0xf4,0xfa,0x3d,0x3b,0x78,0x2d,0xa1, -0x1b,0x10,0x63,0xc8,0x4c,0x64,0x97,0x8c,0x17,0x28,0x4b,0xd2, -0xe2,0x51,0x5d,0xc0,0x51,0x92,0x73,0x3e,0x4d,0x88,0x18,0x79, -0x73,0x7e,0xa3,0x3e,0x12,0xb0,0x2e,0x09,0xad,0x7a,0xa3,0x1c, -0x55,0xaa,0x1b,0x22,0x94,0x47,0x6a,0x87,0xbc,0x2e,0x96,0xe9, -0x5d,0xe6,0x28,0x13,0xd8,0xa8,0xd5,0x65,0xb2,0xe1,0x7f,0x65, -0xaf,0x32,0x1b,0x60,0xdb,0xfe,0xe8,0x4e,0xbe,0xac,0x89,0xfc, -0x22,0x83,0x49,0x5c,0x8a,0x1c,0xa6,0x2e,0x71,0x58,0x1c,0xb9, -0xfa,0xf9,0x61,0xb1,0x58,0x26,0x59,0xe6,0xf9,0x1d,0x26,0xaa, -0xeb,0xa9,0xa0,0x8b,0xf4,0xaa,0xce,0x8e,0x0a,0x5c,0x91,0x50, -0x3d,0xfe,0x6d,0xae,0x6d,0x01,0x94,0x69,0x1b,0x4b,0x02,0xfb, -0x7e,0x5b,0x95,0x08,0x04,0x80,0x14,0xfb,0x8f,0x86,0x09,0xf6, -0x20,0x9f,0xa4,0x93,0x29,0xbd,0x22,0xf5,0x8f,0xf2,0xa3,0x05, -0x12,0x0f,0x21,0xc5,0xd4,0x2a,0x25,0xb4,0x04,0xf8,0x92,0xb7, -0x6c,0x85,0xaf,0x15,0x4c,0x64,0x06,0x22,0x1a,0xe0,0xcd,0xe7, -0x0b,0xa1,0x66,0x12,0xc1,0x6a,0x09,0xf9,0x57,0x93,0xcf,0x0f, -0x8b,0x9a,0x6f,0x29,0x1f,0xab,0xb0,0x54,0x3e,0xc2,0x51,0x22, -0x82,0x35,0x97,0xc4,0xc0,0xca,0x03,0x49,0xd1,0xb6,0x99,0x50, -0xbc,0x29,0xc9,0xd2,0x23,0x7d,0x97,0xb3,0x5c,0x09,0xbb,0xa6, -0x6e,0x03,0xc6,0xd1,0xab,0xf5,0xd1,0x0c,0xb3,0x81,0x38,0x23, -0x70,0xae,0x96,0x13,0xd9,0xaf,0xff,0xc1,0xb7,0xb4,0xdf,0xbe, -0x14,0xbf,0xfe,0xaf,0xa8,0x75,0xf8,0xeb,0x7f,0x88,0xbb,0x78, -0x72,0x03,0xc9,0xbf,0xfe,0xc7,0x51,0x4b,0xfc,0xf2,0xb7,0x28, -0x4c,0x1a,0x67,0xf0,0xdb,0x8a,0x7e,0xf9,0xdb,0xe1,0x2f,0x7f, -0xc3,0x09,0x11,0xe3,0xe9,0xc3,0x6a,0x1d,0x21,0xc2,0x8b,0x80, -0xd6,0x94,0xc9,0xed,0x32,0x81,0x79,0x08,0x5e,0x8f,0x00,0xc3, -0xaf,0xff,0x93,0x91,0xbe,0xfa,0xbf,0xbf,0xfb,0xbe,0x05,0x6f, -0xad,0x08,0xff,0x9e,0x44,0x27,0x38,0xe7,0xd1,0x32,0x7c,0x31, -0x47,0x47,0xd9,0x3f,0xcd,0xe7,0xd6,0x51,0xfd,0x22,0x10,0x8b, -0xd0,0x52,0x66,0x16,0xa0,0xa7,0x9c,0x3e,0x47,0xab,0xc3,0x57, -0x14,0x38,0x55,0x1e,0xa9,0x0f,0xe7,0xd9,0xdd,0x32,0xc0,0x3e, -0xc7,0x27,0x6f,0x19,0xf8,0xe2,0x21,0x7a,0x0c,0x8e,0x3c,0xa8, -0xf4,0x70,0x14,0xfa,0x87,0x1e,0x43,0x1c,0xe2,0x16,0x23,0x9e, -0xdd,0x41,0xfa,0x61,0x56,0x07,0xb0,0x06,0x2e,0x4c,0xd2,0x19, -0x15,0x51,0x4c,0xef,0x3d,0xac,0x83,0xfa,0xc3,0x21,0x68,0xa9, -0x87,0x0d,0x3d,0x6c,0xc4,0x63,0x70,0xc8,0x08,0x8e,0x54,0x15, -0x0a,0x81,0x72,0x8e,0x4c,0x85,0x56,0xaa,0xd0,0x4a,0x16,0xa2, -0xe6,0x27,0xeb,0x05,0x81,0x03,0x2c,0xb9,0x1e,0x5f,0x07,0xd1, -0x22,0x40,0x59,0x07,0x48,0x7c,0x02,0xa5,0xf8,0x48,0x0f,0x2d, -0x48,0x5a,0x87,0xd1,0x22,0xe4,0x4c,0x7a,0xc2,0x4c,0x7a,0x68, -0xa1,0xea,0x8c,0x20,0xbf,0x01,0x08,0x50,0x4f,0x03,0x40,0x03, -0x30,0x88,0x51,0x0b,0x34,0x33,0x54,0x5c,0x1f,0x6d,0xb0,0xd6, -0x51,0x68,0x45,0xdf,0x1f,0xb5,0x7c,0x31,0x08,0x22,0xef,0x31, -0x3c,0x7c,0x0c,0x1b,0x40,0xd0,0x63,0x50,0x87,0x1e,0x38,0x84, -0xf4,0x23,0xaf,0x05,0x6f,0x87,0x92,0x41,0x62,0x10,0x3a,0x50, -0x0d,0x1b,0x2a,0xd4,0x50,0x4b,0x79,0x99,0x3f,0x99,0xdf,0x7a, -0xa6,0x96,0x41,0x70,0x38,0x08,0xea,0xa1,0xdf,0x18,0x00,0xdf, -0x97,0x61,0x29,0x48,0x78,0x38,0x08,0x09,0x04,0xa3,0x20,0x2e, -0xa3,0x65,0xd8,0x58,0x06,0xe2,0x55,0x04,0x3d,0xf8,0xf4,0xa4, -0xe1,0x1f,0xc3,0xa3,0x47,0xe8,0x87,0xa3,0x5f,0xff,0xa7,0x36, -0xac,0x35,0x9d,0xaf,0x43,0x43,0x83,0x28,0x1e,0x1f,0xbe,0x12, -0x56,0x96,0xbb,0xa6,0x78,0xeb,0x88,0x93,0x25,0x49,0x98,0xe1, -0xdd,0xfa,0xef,0x6e,0x59,0x05,0xca,0xf1,0x6d,0x46,0xba,0x54, -0x82,0xd0,0xc5,0x0a,0xd7,0x35,0x16,0xf9,0xcb,0x5d,0x92,0x4c, -0x40,0x2d,0xe2,0xb3,0x5c,0x78,0x3c,0x62,0x92,0x04,0xd7,0x2a, -0xc0,0xca,0x02,0xb5,0x08,0x33,0x0d,0xad,0x4c,0x46,0x83,0xc9, -0x70,0x92,0x0e,0xc7,0xb2,0xb8,0x7a,0x95,0x85,0x8d,0xa6,0xb0, -0x94,0x06,0xe4,0x99,0x89,0x11,0x0d,0x61,0xdd,0x95,0x43,0xcf, -0xa3,0x59,0xfb,0x4d,0x9a,0x3c,0x36,0xd7,0xfe,0x11,0xfd,0x8e, -0x85,0x47,0xf3,0x36,0xbd,0x6c,0x54,0xa2,0xb5,0x9a,0xa0,0x05, -0x98,0x37,0xd1,0x38,0x26,0x80,0xe2,0x90,0xa1,0xea,0x8c,0x49, -0x4c,0x00,0x81,0x93,0xb4,0xb1,0xca,0x93,0xc7,0xda,0xef,0xe7, -0xf8,0x21,0x12,0x83,0x48,0xab,0x8e,0x78,0xed,0x59,0xae,0x8c, -0x51,0x70,0xb5,0xb5,0x87,0x9d,0x0e,0x62,0x9c,0x39,0x11,0xd6, -0x55,0xd8,0x6a,0x40,0xb9,0x10,0x40,0xd6,0x24,0x52,0x14,0x0a, -0x26,0xa7,0xce,0x6b,0x13,0xa4,0x93,0x53,0x36,0x75,0x5e,0x9a, -0x20,0x99,0x06,0x11,0x70,0x9a,0xdc,0x77,0xf8,0xef,0xd6,0xe1, -0xc1,0xc1,0x3a,0xd4,0x6e,0xc2,0x03,0xe9,0x7f,0x38,0xb7,0xc2, -0xd5,0xba,0xd3,0x5b,0xe7,0x38,0xb8,0x65,0x48,0x28,0xc8,0xee, -0xca,0x7d,0x5f,0x6c,0x00,0xe5,0x46,0xa3,0xdc,0x54,0xa0,0xd4, -0x5f,0xf8,0x7a,0x9b,0x1c,0xff,0x25,0xca,0x8d,0x41,0xb9,0x75, -0x84,0x51,0xf6,0xb6,0x47,0xea,0xd8,0x7f,0xc7,0xdf,0xe0,0xc9, -0x79,0x4d,0xe7,0x3b,0xc1,0xbc,0x31,0xd5,0x14,0xd0,0x8d,0x17, -0x85,0xa2,0x35,0xf6,0x89,0xdc,0x96,0xd2,0xa1,0xf9,0xdd,0xee, -0xc9,0xde,0x56,0x3d,0x9c,0x47,0x4e,0xeb,0xb0,0x2a,0x82,0xe4, -0x42,0xc9,0x5a,0x68,0xea,0xa1,0x53,0xb6,0xfe,0x54,0x4b,0x91, -0x50,0xd8,0xfd,0xad,0x66,0x4f,0xb9,0x32,0xf0,0xa1,0xfb,0x85, -0xdd,0xae,0xeb,0xea,0x65,0xa1,0x1a,0x69,0x7a,0x35,0x83,0xd1, -0x7e,0xd9,0xe1,0x7d,0x6e,0xf0,0xb5,0xf5,0xaa,0x51,0x4e,0xcb, -0xbf,0x61,0xc1,0x98,0xe3,0xcb,0x35,0xaf,0x14,0xed,0x25,0xe2, -0x6f,0x5b,0x09,0xee,0xb7,0x62,0x33,0x5a,0xe6,0x8d,0xad,0x79, -0xa0,0x65,0x72,0xa1,0x36,0x89,0xb4,0xae,0xc8,0xb3,0x78,0xbf, -0x55,0x5c,0xf9,0x31,0x99,0x3c,0x46,0xb5,0x78,0xe1,0x88,0xed, -0x75,0x7e,0x7f,0xa5,0x72,0xed,0x70,0xef,0x09,0x51,0x96,0xc8, -0x2d,0x89,0xd9,0xae,0xea,0xed,0x52,0x2e,0xf8,0x02,0xc9,0x6e, -0x10,0x29,0xe1,0x65,0xa0,0xe2,0x79,0x03,0xc6,0x21,0xb7,0x76, -0x51,0x78,0xea,0xa0,0x58,0x90,0x05,0xe4,0xa5,0x4d,0xbd,0xf4, -0x6c,0xb8,0xbe,0x61,0x14,0x0e,0x49,0xbd,0x3b,0xb6,0xb4,0x9d, -0x69,0x83,0x8a,0x3f,0xcb,0x1b,0xc4,0xa8,0x62,0xc7,0x28,0xaf, -0xc9,0x64,0x62,0xfe,0xcb,0x74,0x8b,0x0e,0x09,0x81,0xa1,0x4e, -0x6d,0x82,0x28,0xd8,0xb1,0xcd,0x1b,0xc3,0x32,0x81,0x2e,0xaa, -0xcd,0x79,0xb0,0x72,0x9e,0xa8,0x98,0x62,0x7d,0xa8,0x7a,0x1a, -0x04,0x57,0x00,0xd3,0xa0,0x2c,0x0a,0xef,0x2e,0x77,0x80,0x2b, -0xbd,0xad,0xb5,0x84,0x04,0x88,0xb1,0xb7,0xa0,0xfd,0x8e,0x52, -0xf1,0x2d,0x39,0x09,0xf8,0x22,0xaf,0xa5,0xf5,0x31,0xc8,0xb7, -0x4b,0x5e,0x5b,0x27,0x37,0xf1,0xc3,0x84,0x42,0xd8,0xb8,0xe3, -0x57,0x53,0x10,0x01,0x3d,0x3a,0xa4,0xad,0x4b,0xf2,0x55,0x58, -0x42,0xde,0xcf,0xfa,0x85,0xb6,0x8a,0x3c,0x07,0xfc,0x6c,0xef, -0x4c,0x7f,0xa6,0x3d,0x69,0xca,0x0e,0x8d,0x02,0x7b,0xe9,0xb3, -0xdd,0x16,0x14,0x90,0xe5,0x8f,0x6d,0x82,0xcb,0x4a,0x31,0x09, -0x45,0xb5,0xf4,0xa9,0xf0,0xc2,0x0e,0x99,0xaa,0x13,0xa1,0x28, -0xf5,0x12,0x7e,0xd6,0xb0,0x08,0x35,0x99,0x69,0x1f,0x33,0x6c, -0x9e,0x86,0x36,0x53,0xa9,0x5f,0x26,0x81,0x3f,0x58,0x26,0xf1, -0xb8,0xc3,0xae,0xdc,0x26,0x41,0x34,0x09,0xf1,0x64,0x70,0x22, -0x39,0xa0,0x5a,0x03,0x4b,0x30,0xf3,0x48,0xab,0xc0,0x06,0x2d, -0x14,0xfd,0x43,0x9d,0x5c,0x77,0x21,0x42,0x82,0x08,0x6d,0x08, -0xd6,0xfb,0x61,0xa4,0x59,0x74,0x70,0x60,0xad,0x07,0x15,0xd4, -0x91,0xce,0xf6,0x91,0x2a,0x58,0x52,0x60,0x45,0x75,0xaa,0xd4, -0x87,0x05,0xbf,0x47,0x68,0xeb,0x54,0x05,0xbc,0xf7,0x91,0xe8, -0x1e,0x34,0x04,0x81,0x26,0x1a,0x68,0x42,0x40,0x13,0x0d,0xa4, -0x17,0x09,0x44,0xc2,0x21,0x8f,0x62,0x47,0x1a,0x70,0x67,0xe5, -0x48,0x15,0x76,0xcc,0xf3,0xea,0x1f,0x97,0xfa,0x5a,0x69,0xe6, -0x04,0xff,0x77,0x19,0xa1,0xd2,0x9d,0xc2,0xb3,0x03,0xd5,0x5c, -0xe7,0x55,0x68,0x1d,0x75,0xf3,0x4d,0x0e,0x32,0xcd,0x20,0xde, -0x3e,0xea,0xd5,0x1e,0x29,0x71,0x3d,0x3b,0xa9,0x2d,0x9f,0x9c, -0x9e,0x56,0xbb,0x97,0xa6,0x55,0xcb,0x45,0x67,0x5a,0xab,0x9a, -0xb0,0xb8,0x3b,0xc4,0xfb,0xcd,0x5b,0xa6,0x65,0xe4,0x7b,0x41, -0x8f,0xbb,0x40,0x9f,0x5f,0xec,0x75,0x86,0x26,0x74,0xd3,0x23, -0xb3,0xe3,0xa5,0x25,0xb1,0x75,0xb4,0xa1,0x78,0x11,0xe9,0x8d, -0x70,0x1e,0xa2,0x7a,0x7e,0x34,0xac,0x35,0xf3,0xa3,0xe6,0xad, -0x9c,0x20,0xed,0xcb,0x08,0x39,0xe4,0x5d,0x76,0xbb,0xbd,0x62, -0xb9,0xa0,0xdb,0xf7,0x1c,0x64,0x9f,0x19,0x54,0xcf,0x2d,0xbb, -0xe7,0x52,0x7b,0x9f,0xa1,0xbc,0x6c,0xe5,0x7b,0x2b,0x67,0x95, -0x63,0x8a,0x60,0x88,0xb1,0x65,0x77,0x38,0xa1,0xb8,0x3e,0x53, -0x3c,0xc6,0xf7,0x72,0x99,0x7e,0xbb,0xe2,0x56,0xcc,0xdc,0x9a, -0x96,0x12,0xe8,0x8b,0x1c,0xa2,0x28,0xe3,0x88,0x66,0x58,0x87, -0xb5,0xe9,0xca,0x43,0xd1,0xc8,0x2f,0xca,0xa5,0x38,0x0d,0xaa, -0x67,0x95,0x8e,0x39,0x87,0x1e,0xd2,0x65,0xc5,0xd3,0x93,0x59, -0xd8,0x20,0x81,0x1d,0xad,0x4a,0x82,0xa7,0x27,0xcf,0xbc,0x98, -0xf9,0x9d,0x4a,0xfb,0xbe,0xd6,0x4c,0xa4,0x0a,0x17,0x30,0x8f, -0xb6,0x44,0x33,0x08,0x5a,0x87,0xf9,0xbd,0xde,0x1f,0x93,0xc9, -0x2a,0xf6,0x7c,0xbf,0x7c,0x0a,0x44,0x64,0x26,0x65,0x87,0xde, -0xca,0x2d,0x46,0xf1,0x20,0xdb,0x10,0x47,0x86,0x30,0xe6,0x98, -0x57,0x6f,0x08,0xf7,0xeb,0x65,0x0a,0x43,0x6f,0x33,0xc1,0x5e, -0xf7,0x2d,0x7c,0x31,0x36,0xdb,0x6f,0xd9,0x86,0x23,0x4a,0xf8, -0xf2,0xeb,0x56,0xa7,0xa4,0x3f,0xcb,0x38,0xa3,0x47,0x74,0x76, -0x97,0xde,0x90,0x4b,0x26,0xca,0xa4,0x58,0x39,0x63,0xbf,0x11, -0xb6,0x2d,0xf7,0xc6,0x63,0x9f,0x3e,0x62,0x2d,0xac,0x14,0x5c, -0xce,0x54,0x2d,0xa1,0x35,0x53,0x04,0x77,0x70,0x5e,0xb8,0x84, -0x9e,0x3a,0xc4,0x3a,0x10,0xeb,0x50,0x6c,0x02,0xd8,0x94,0xd1, -0xde,0x25,0x7a,0xb7,0x6e,0xc3,0x0b,0xfc,0x1b,0xb7,0xc3,0xad, -0x20,0x83,0x93,0xde,0xe5,0x59,0x00,0x12,0x15,0xc8,0x99,0xe8, -0x73,0x1d,0x6d,0xc7,0xe9,0xe3,0x97,0xec,0x21,0x72,0x63,0x1d, -0xf1,0x59,0xd7,0x0b,0xd6,0x2e,0x9f,0xd5,0x95,0x7e,0xb4,0xb3, -0x1e,0x16,0x91,0x3a,0xef,0x93,0xc9,0xd6,0x99,0x9f,0x7d,0x1b, -0xc1,0x99,0x15,0x47,0x92,0x98,0x29,0xac,0x9d,0x9e,0x90,0x5b, -0x37,0xbd,0xd7,0xd2,0x0b,0x62,0x4c,0x90,0x3e,0x42,0xf4,0x20, -0xa3,0x93,0x8c,0xb2,0x2f,0x46,0x9e,0x93,0x20,0x64,0x5f,0x48, -0x8b,0xeb,0x4e,0xb9,0x15,0x43,0xd7,0xd2,0x93,0x28,0x62,0x96, -0x6d,0x04,0x6c,0x7e,0xf9,0x7b,0x29,0xbb,0x79,0x76,0xfd,0xd8, -0x35,0x74,0x6b,0x73,0x7d,0x8d,0xee,0x7f,0x56,0xd7,0xd7,0x4f, -0x4f,0x6e,0x57,0x95,0xa9,0x3d,0xa8,0xe2,0x31,0x49,0x66,0x1e, -0xb5,0xbc,0x5d,0xc4,0x8a,0xab,0xa4,0x75,0x84,0x9d,0x2c,0xd7, -0x82,0xf4,0x08,0x6b,0xc4,0xe1,0x1a,0x96,0x80,0xb0,0x16,0x19, -0x6e,0xa2,0xd1,0x06,0x7e,0x29,0xc8,0x5b,0xee,0x6c,0xd1,0xeb, -0x79,0xc3,0xfc,0x6e,0x5f,0x78,0xc3,0xfc,0x6e,0x1d,0xd6,0x9b, -0xea,0xdc,0x44,0xe8,0x12,0xa1,0x2a,0x12,0x5a,0x65,0x42,0x55, -0x28,0x34,0xa5,0xe0,0xd1,0x78,0x02,0xb2,0x77,0x28,0x14,0x27, -0x30,0x4a,0xe1,0x11,0x7d,0x8f,0xae,0x8f,0x26,0xbd,0x56,0xbf, -0xe3,0x72,0x28,0x52,0x02,0x0d,0x95,0xd2,0xc1,0xcc,0x18,0xb8, -0x05,0x75,0xd1,0x89,0xcc,0x18,0xb8,0x36,0xde,0xe6,0x55,0xcd, -0x56,0xf5,0x83,0x9a,0x25,0x1d,0x7e,0x15,0x87,0x1b,0xe8,0xfb, -0x92,0x3a,0xcb,0xfa,0xe2,0xd9,0xa1,0x4b,0x96,0x92,0x24,0x90, -0x7a,0xfc,0x1a,0xa1,0xbc,0xde,0x71,0x36,0xad,0x5a,0x59,0xbf, -0xe6,0x9b,0xa1,0x3a,0x9e,0xee,0x43,0xeb,0xe4,0x61,0x89,0x30, -0xc7,0x19,0x88,0xdd,0x2f,0x1c,0x55,0x70,0xa5,0x04,0xf3,0x7e, -0x15,0x4a,0x3c,0x1b,0x7e,0xd8,0x40,0x95,0xf5,0xeb,0x62,0x75, -0x8a,0x0c,0x53,0xcb,0xe7,0xb9,0xd8,0x07,0xbb,0xea,0xb2,0x0b, -0xf0,0x45,0xce,0x75,0xb7,0x4a,0xf3,0xb4,0x7b,0xcc,0x02,0x62, -0x40,0x5f,0x56,0x6f,0x21,0x90,0x34,0xb0,0x4e,0xdc,0xaf,0x7a, -0x09,0x7b,0x7d,0x70,0x50,0x86,0x9b,0x73,0x55,0xd3,0x1c,0x63, -0xcb,0x9d,0x6d,0x42,0xc8,0x0a,0x94,0x98,0x27,0x11,0x5a,0x71, -0x35,0xde,0xee,0xc0,0xb6,0x0e,0xa3,0xb7,0xa0,0xc6,0xa3,0xb7, -0x2a,0xf8,0x56,0x89,0x1a,0x67,0xdc,0x6b,0xf9,0xd4,0xdc,0xec, -0x87,0x79,0x83,0x98,0x37,0xfb,0x60,0xde,0x84,0xf6,0x5d,0x08, -0xe9,0xdf,0xc2,0x5d,0x48,0xe9,0x9a,0x40,0x54,0xf5,0x25,0x5e, -0x4a,0x85,0x47,0x1c,0xde,0xac,0x78,0x6c,0x1c,0x01,0x5e,0x9a, -0xce,0x6a,0xae,0x45,0x44,0xb7,0x70,0xca,0x4c,0xb5,0x14,0xad, -0x98,0x1a,0x7a,0x32,0x1e,0x21,0xc0,0x8f,0x87,0x9e,0x9b,0xf0, -0x15,0x1a,0x23,0x85,0xad,0xa0,0x1d,0xc2,0xa0,0xac,0x71,0x55, -0x7e,0x1b,0x6a,0x35,0x33,0xe9,0x87,0x56,0x6d,0x6e,0xc7,0x09, -0x0d,0x41,0x6d,0xe5,0x35,0x97,0xaa,0xe8,0x43,0x9a,0xb1,0x8a, -0xd3,0xc9,0x96,0x3c,0x9f,0x7d,0x45,0xa6,0x05,0xe9,0x3a,0x99, -0xbc,0x1a,0xa2,0xa3,0xc0,0x1a,0xad,0xa0,0x3f,0x9b,0x4f,0xe6, -0x96,0xe5,0xad,0x89,0x5c,0x5a,0x61,0xe2,0xb5,0xbc,0x1d,0x78, -0x7e,0xbd,0x56,0xa3,0x9e,0xbd,0xcb,0x2c,0x53,0xa9,0x3b,0x91, -0x09,0x7d,0xe4,0xfd,0x51,0x59,0xc8,0x89,0xee,0x5d,0xce,0x39, -0xf2,0x9f,0xb2,0x09,0x0e,0x5b,0x40,0xe3,0xdd,0x35,0xef,0x04, -0xec,0x9b,0xe1,0xdf,0xc4,0x47,0x03,0x3c,0xa8,0x47,0xc6,0xd4, -0xaf,0xd5,0xea,0x77,0x42,0xa6,0x00,0xa4,0xe0,0x02,0x04,0x84, -0x05,0x21,0xb3,0x9e,0x89,0xfa,0xc4,0xc8,0x13,0xa4,0xda,0xde, -0x8a,0xb0,0x16,0xcb,0xb4,0x58,0x7e,0x8b,0x48,0xed,0xee,0xe4, -0xa0,0x9b,0x03,0xf2,0xbe,0xef,0x78,0x12,0x35,0x5e,0x21,0x23, -0xbd,0x6a,0x6b,0x9e,0x17,0x9c,0x69,0x77,0x41,0xee,0x7d,0x49, -0x1b,0x2b,0xff,0x3b,0xf9,0x01,0x2c,0xff,0x4c,0x8e,0xc6,0x6c, -0x57,0xe7,0xd4,0x37,0x8a,0x97,0xe3,0xdf,0xb9,0xb6,0xf1,0x21, -0xd7,0x57,0x52,0x1b,0x70,0xb0,0xdc,0xf7,0x13,0x40,0x21,0x7b, -0x4b,0xe9,0x66,0xd3,0xb8,0xbb,0xa1,0xd3,0xd3,0xc3,0xf7,0xee, -0xe9,0x21,0xf7,0xf4,0x50,0xf5,0xf4,0x90,0x7b,0x3a,0x0f,0xf7, -0x65,0x3c,0x40,0xb8,0x49,0x3c,0x90,0xb0,0x13,0x80,0x8b,0xe1, -0xdf,0x80,0x3a,0x5c,0xa5,0x7b,0x77,0x91,0x94,0x08,0x48,0xe1, -0x37,0x92,0xcc,0x3b,0xdf,0x6f,0x2e,0x01,0xfa,0x96,0x4a,0xf8, -0xb9,0xd2,0x58,0x12,0xc5,0x65,0xe8,0x8a,0xcb,0x30,0x27,0x2e, -0xc3,0x9d,0xe2,0x32,0xdc,0x53,0x5c,0x64,0x7d,0x36,0x4b,0x87, -0xe6,0xba,0x26,0x0c,0x02,0xc9,0xdf,0xba,0x6c,0xd6,0x7f,0x3f, -0x2c,0xf8,0x52,0xa0,0x5e,0x96,0x7e,0x09,0x9c,0x7a,0xab,0xc5, -0x66,0x47,0xad,0xf1,0xda,0x53,0x75,0x36,0x3e,0xa0,0xce,0x6a, -0xe1,0x19,0x4e,0xa8,0x17,0xdc,0x3a,0xa5,0xf0,0xb0,0xbe,0x20, -0x11,0x02,0x18,0x2b,0xd8,0x9d,0xb0,0x9d,0xcc,0x96,0x8a,0xd0, -0xa4,0x52,0x34,0xbc,0x09,0x74,0xec,0x04,0x3a,0x76,0x82,0x1d, -0x9b,0x87,0x33,0xa2,0x66,0xc1,0x82,0xc0,0x36,0xef,0xb4,0x62, -0x21,0xa1,0x99,0x28,0xa1,0x99,0x90,0xd0,0x4c,0x40,0x68,0x26, -0x46,0xcc,0xbc,0xfa,0x44,0xd4,0x63,0x51,0x1f,0x18,0x41,0x21, -0x8e,0x45,0xe1,0x85,0x90,0xcf,0x3f,0x44,0xcd,0xcb,0xd3,0xe0, -0xe4,0x5c,0xbd,0xff,0x18,0x85,0xea,0xf1,0xa7,0x28,0x6c,0x06, -0x17,0x17,0x17,0xc7,0x32,0xc1,0x11,0x30,0x68,0xc8,0x0e,0x01, -0xb3,0xa1,0x9f,0x13,0x30,0xa4,0xf3,0x43,0x44,0x8a,0xe1,0x62, -0xfe,0x19,0x70,0x67,0x3b,0xf5,0xee,0x14,0x30,0x53,0xeb,0xfb, -0x88,0xd4,0xb3,0x75,0x56,0x0a,0x18,0xa2,0xd5,0xda,0x69,0x52, -0x82,0xc7,0x2d,0xba,0x14,0xb7,0xcf,0x89,0xd6,0x32,0x27,0x32, -0xdf,0xdd,0x92,0x68,0x61,0x25,0x4b,0x10,0x85,0x25,0x88,0xc2, -0x52,0x8a,0x82,0x33,0x0f,0x2d,0xe5,0x3c,0x24,0x8c,0xce,0x54, -0x40,0xde,0xbf,0xff,0xfb,0x52,0xfc,0xfb,0xbf,0xdf,0xc2,0x3f, -0x4b,0x64,0x20,0xc3,0xe9,0x7a,0xa8,0xa8,0xaa,0xeb,0xff,0x4b, -0x0e,0xbc,0xbc,0xef,0xf7,0x98,0x16,0xa8,0xee,0x25,0x6f,0x0b, -0x81,0x17,0xfc,0x00,0x2c,0xe1,0x07,0x0c,0xbe,0x76,0xac,0xbf, -0xf5,0x59,0x3e,0x3d,0xdd,0x3e,0x3d,0x0d,0xba,0xde,0xf2,0xe0, -0x20,0xbd,0xc2,0x4f,0x1e,0x97,0x51,0xea,0x8b,0x5b,0x7c,0x83, -0x3f,0xde,0x2d,0xbe,0x0d,0xf0,0x0d,0xe3,0x6b,0x0d,0xf0,0x4d, -0x36,0x57,0x8b,0x5d,0xeb,0xf4,0x14,0xda,0xec,0x2d,0x61,0x7e, -0xf3,0x45,0x3e,0xf5,0xb6,0x34,0x75,0x80,0xa9,0xbe,0xe6,0x5c, -0x8a,0x76,0xeb,0x2c,0x10,0x0e,0x03,0x7e,0xf3,0xe4,0xc8,0xdd, -0xe2,0xc9,0x19,0x71,0xe9,0x0b,0xf3,0x72,0x6b,0xbf,0x0c,0xfc, -0x92,0xda,0x9d,0xb5,0x8d,0x2d,0x8e,0x72,0x1d,0xe2,0x49,0x06, -0x4b,0xf6,0xda,0x62,0xed,0xe0,0xa9,0x5e,0x50,0xd5,0xfe,0xa5, -0x56,0x57,0xf8,0x92,0xb5,0xc4,0xe7,0x17,0x92,0x6e,0x8b,0x49, -0xae,0x84,0x5d,0x93,0x7b,0x36,0x6d,0x6e,0x1c,0x63,0x04,0xd4, -0x01,0x7a,0xf6,0x0f,0x4f,0xcf,0x2f,0x5b,0xc7,0xa0,0x84,0x30, -0xc8,0xd4,0xcf,0x0f,0xb0,0x8e,0x4c,0x57,0x90,0x7a,0x76,0x72, -0x72,0x72,0x7c,0x7e,0x2a,0xe2,0x9f,0x1f,0xe2,0xf6,0xd9,0xe9, -0xe9,0x31,0x3f,0x4e,0x63,0x20,0x33,0x69,0x5f,0x1c,0x5f,0x5c, -0x9c,0x9e,0x9d,0x88,0xf8,0xed,0xc3,0x92,0x51,0x9c,0x84,0x00, -0x3c,0x48,0xd2,0x5b,0x2c,0x1b,0x86,0x97,0xad,0xb3,0x40,0x0c, -0xd2,0xec,0x67,0xac,0xe1,0xec,0xfc,0x3c,0x68,0x9d,0x9c,0x88, -0xc1,0x24,0x1e,0x8e,0x61,0xf5,0x0f,0xbf,0xb3,0xe1,0x5d,0x32, -0x8a,0x27,0xd3,0xf9,0x6c,0x44,0xf9,0xad,0xe0,0x04,0x8a,0x23, -0x3d,0xd8,0xfd,0xf8,0x00,0xeb,0xd7,0x49,0xb2,0x6a,0x5f,0x06, -0xa7,0xa7,0xad,0xa0,0x25,0x06,0xcb,0xf9,0xe3,0xac,0x1d,0x06, -0x17,0xad,0x93,0xd6,0x31,0xa0,0x7a,0x58,0x4e,0x36,0x8f,0xf3, -0x39,0x94,0x3e,0x39,0xbd,0x3c,0x6b,0x1d,0x87,0x62,0x18,0xc3, -0x42,0x96,0x50,0x9c,0xb5,0xce,0xce,0x4e,0x5b,0x17,0x82,0xf6, -0xd5,0xcb,0x04,0x56,0xb4,0x44,0xf0,0xf1,0x69,0x0b,0x92,0xe6, -0x43,0x3a,0x85,0x68,0x87,0xc7,0xe7,0x17,0x97,0x27,0xe7,0x81, -0x18,0xce,0x97,0xf1,0x04,0x89,0x38,0x39,0x69,0x9d,0xb7,0xf0, -0x75,0x76,0x33,0x99,0x3f,0x26,0x4b,0xc6,0x75,0x7a,0x19,0x5e, -0x5e,0x84,0x94,0x9c,0xa5,0x93,0x31,0x51,0x7b,0x0a,0xd8,0xc4, -0x70,0x99,0x4e,0xb3,0x39,0xd0,0x04,0xe5,0x8e,0x51,0xaf,0x0e, -0x37,0xf1,0x4c,0xb2,0x0a,0xa5,0x92,0xb9,0x7b,0x7c,0x49,0x2f, -0x94,0x77,0x7c,0x7a,0xde,0x3a,0xa6,0xd7,0xdb,0xf9,0x64,0x94, -0xcc,0x96,0x48,0x7e,0x2b,0xb8,0x6c,0x5d,0x4a,0xa8,0xdb,0x65, -0xbc,0x69,0x87,0xf0,0xbf,0xcb,0x20,0x3c,0x97,0x29,0x49,0x82, -0xdf,0x91,0x9c,0x01,0x7e,0xf9,0x9e,0x83,0x18,0xdf,0xc5,0xe3, -0x14,0xd0,0x9c,0x1c,0x1f,0xb7,0x4e,0x19,0xcd,0x14,0xad,0xe7, -0x56,0x71,0xfb,0x32,0x0c,0x2e,0xcf,0x4e,0xb8,0xc6,0xf9,0x24, -0x7d,0x93,0x30,0xb6,0xd3,0xd3,0xcb,0xf3,0xcb,0x4b,0x06,0x9d, -0x73,0x90,0x69,0x6c,0xfd,0x39,0xf0,0x59,0xa6,0xa1,0x55,0x1c, -0xf0,0x3a,0x38,0x09,0x82,0xb0,0x45,0x69,0xcb,0x64,0x44,0xe8, -0x60,0x2a,0xa3,0xf7,0x8c,0xfa,0x0e,0x7a,0xfe,0x38,0xb8,0x38, -0x09,0xb9,0x5c,0x96,0xc4,0x5c,0x01,0x08,0xc3,0x25,0x70,0x8d, -0x13,0x91,0xd9,0xc4,0x8a,0x93,0xf3,0xe3,0x93,0xe3,0x93,0x73, -0x93,0x4a,0xad,0x45,0xce,0x9d,0x5c,0x9e,0xda,0xa9,0x89,0x9b, -0x0a,0xa3,0xe0,0xe7,0x87,0x79,0x0a,0x9d,0x78,0xda,0xba,0x3c, -0xe1,0x34,0x25,0x1c,0x67,0x97,0x97,0xa7,0xc8,0xbb,0x24,0x59, -0x2c,0xd0,0x85,0x21,0xb4,0x23,0x3c,0xbb,0xc4,0x4a,0x20,0x25, -0x1b,0x6f,0xb8,0xe2,0xcb,0xf0,0x14,0x26,0xdd,0x74,0x4a,0x15, -0x9e,0x5d,0x82,0x0c,0x9d,0x9d,0xf2,0x7b,0x62,0xbd,0xcf,0x47, -0xb7,0xb2,0xcf,0x5b,0x41,0x70,0x0c,0x2d,0x10,0x37,0x29,0xec, -0x4a,0x97,0x29,0xc8,0x6c,0x88,0x0c,0x0a,0x4f,0xce,0x04,0x48, -0x06,0x48,0x8b,0x1a,0x23,0x20,0x09,0x97,0xc0,0x34,0xfc,0x88, -0x29,0x5b,0xc9,0xae,0x6a,0x9d,0x1d,0x5f,0x9c,0xb4,0xc4,0x0d, -0x9e,0x37,0xa6,0x31,0x51,0x14,0x5e,0x82,0x48,0xdc,0xa2,0x13, -0xb7,0xc1,0x7c,0x39,0x47,0x81,0x01,0x59,0x83,0xf1,0x71,0x7b, -0x37,0xcf,0x56,0x0a,0xd7,0x71,0x78,0x06,0xa0,0x02,0x25,0x03, -0x0b,0xc1,0x0b,0x60,0xb6,0xe4,0xe4,0xe4,0xb8,0x75,0x19,0x62, -0x12,0x36,0x02,0x6a,0x08,0xb1,0x2b,0xb8,0xce,0xe3,0xd6,0xf9, -0xd9,0x05,0x3f,0x6f,0x92,0x09,0xc8,0x2e,0xd0,0x7b,0x12,0x1c, -0xc3,0xc8,0x11,0xd4,0x44,0x05,0x7d,0x37,0x9f,0x25,0x9b,0x51, -0xf2,0x28,0x07,0x2c,0x50,0x70,0x37,0x5f,0x29,0xbe,0x1d,0x5f, -0x9c,0x9f,0x04,0x18,0xb3,0x2b,0x8d,0x67,0xd8,0xdb,0xe1,0xf1, -0xc9,0xe9,0xc5,0x69,0xeb,0x84,0x92,0x6e,0xe7,0xc4,0xc5,0xe3, -0x63,0x80,0x78,0x33,0x5f,0x6e,0xa8,0xed,0x40,0x20,0xec,0xe4, -0x59,0xfc,0x4e,0xcf,0x2f,0x80,0xe4,0x40,0x4c,0x62,0xfc,0x32, -0x20,0x59,0x42,0x4a,0x78,0xdc,0x42,0xc9,0x50,0x29,0xc0,0xd9, -0xec,0x8e,0xca,0x1d,0x1f,0x03,0xbb,0x27,0xf1,0xe3,0x8c,0xa9, -0xbf,0x00,0x59,0xbe,0x3c,0x3f,0x13,0x93,0x04,0x24,0x0a,0x24, -0xef,0xe6,0x06,0x05,0x0b,0x79,0x0b,0x3a,0x46,0x4c,0x70,0x7e, -0xe3,0xa1,0x04,0x63,0x09,0x44,0xfc,0x84,0x93,0xe4,0xa8,0x3d, -0x3d,0x3f,0x03,0xb2,0xce,0x64,0x1a,0x0e,0xb2,0x10,0x98,0x0b, -0x12,0x7e,0xc9,0x49,0x9a,0x81,0x8a,0x31,0xa0,0xd7,0x2e,0x5a, -0x48,0x16,0xe5,0xd2,0x78,0x83,0xc1,0xdc,0x3a,0x86,0x81,0x29, -0x93,0x58,0x82,0x2f,0x2f,0x60,0xd0,0xe9,0xa4,0x3c,0x94,0x62, -0xda,0xe9,0xc5,0xc9,0x99,0xa4,0x51,0x8d,0x08,0x48,0x84,0xee, -0x68,0xc9,0x44,0x35,0x24,0x5a,0xe1,0x49,0xeb,0xe2,0x52,0x56, -0xab,0x04,0x13,0x12,0x82,0xe3,0x13,0x59,0x8b,0x19,0x12,0xe7, -0x17,0xc7,0xa0,0x79,0x8f,0x9d,0xe4,0x24,0x9f,0xbc,0x82,0xdd, -0xbe,0x64,0x0b,0x10,0x01,0x43,0x8b,0xd3,0x75,0x33,0xa1,0x7b, -0xc2,0x0b,0x4c,0x9c,0xa2,0x0e,0x6b,0x5d,0x04,0xf4,0x28,0xe5, -0x05,0x44,0x09,0xbb,0x12,0xa3,0xa7,0xcf,0x88,0x25,0xa7,0x67, -0xa0,0x08,0x95,0xda,0xd0,0x22,0x0b,0xca,0x7e,0x0e,0x4d,0x42, -0xdd,0x79,0x16,0x5c,0x08,0x0c,0x7f,0xf8,0x30,0xb5,0x66,0x01, -0x10,0x9a,0xf3,0xe3,0x56,0x4b,0x66,0xc8,0xa1,0x73,0x2a,0x5f, -0x95,0x16,0x69,0xb5,0x42,0x94,0x6c,0x99,0xba,0x78,0x58,0x2e, -0x26,0x09,0x0c,0x5c,0xd0,0xd1,0x30,0xe7,0x70,0xa2,0xe6,0xd2, -0xf1,0xe5,0xf9,0x05,0xc8,0x82,0x4a,0xd6,0xaa,0xe3,0x22,0xb8, -0x38,0x3f,0x07,0xee,0xc9,0xf4,0x05,0x4e,0x95,0x5c,0xe2,0xec, -0x24,0x04,0x89,0xe0,0x74,0xa3,0x28,0x4e,0x40,0x36,0x8f,0x03, -0x05,0xcf,0xca,0x82,0x65,0x1a,0x56,0xe2,0xe1,0xf9,0x31,0x06, -0xf6,0x9b,0x19,0xc1,0x02,0x06,0xc0,0xd0,0x82,0xc4,0xd9,0x0a, -0x3d,0x58,0x4d,0x71,0x06,0x6b,0x85,0x17,0xa7,0x80,0x20,0xcd, -0x56,0x9b,0xe5,0x3c,0x53,0x93,0x18,0x16,0x9d,0x0f,0x87,0x71, -0x96,0xce,0x64,0x4a,0xeb,0x52,0xcc,0xe2,0x37,0xf1,0xfd,0x5c, -0xeb,0x84,0xb3,0x8b,0x33,0x90,0x5b,0x48,0x04,0xa1,0x81,0x49, -0x08,0x04,0x10,0x7d,0x83,0x43,0x0e,0xa8,0xe2,0xd3,0x53,0x4c, -0x00,0x4d,0x4c,0x63,0xf2,0x18,0xa4,0x9e,0xde,0x46,0xcb,0x78, -0xd0,0x3e,0x0f,0x4e,0x2e,0xce,0x41,0x99,0x19,0x95,0x0c,0xaa, -0x0d,0x06,0x3c,0xbf,0x13,0xf9,0xa0,0x13,0x2e,0x8f,0x61,0x22, -0x55,0xbc,0x3d,0x39,0x86,0x01,0x00,0x5d,0xbf,0x88,0x27,0x89, -0xa5,0x2a,0x4e,0xcf,0x4e,0xcf,0xa1,0xa9,0x9c,0x4c,0x6c,0x02, -0x75,0xda,0x82,0xe1,0xc4,0x49,0x86,0x4f,0x20,0x3b,0xad,0x4b, -0xe8,0x0b,0x4a,0xb6,0xd8,0x74,0x72,0x7c,0x01,0xaa,0xe6,0x18, -0x92,0x17,0xf1,0x26,0x86,0x96,0x2d,0x78,0xe0,0x06,0xe7,0xe7, -0x62,0x81,0x87,0xdc,0x8b,0x87,0x9b,0x1b,0x6a,0x2b,0xfc,0xff, -0x18,0xc3,0xf8,0x3d,0xa0,0xbe,0x38,0xbb,0x00,0xb5,0x2f,0xd4, -0xd8,0x38,0x0b,0x03,0x90,0xa1,0xc5,0xe4,0x61,0x8a,0x73,0x74, -0xeb,0xe4,0xec,0x18,0x0a,0xcf,0x1f,0x47,0x52,0xc9,0x42,0xdd, -0x30,0x47,0xc0,0x48,0x94,0x22,0x81,0x52,0x76,0x0e,0x23,0x59, -0x36,0x14,0x84,0x06,0xc8,0x05,0xd6,0x6f,0xe4,0xcc,0xdf,0x82, -0xd9,0xf3,0x14,0x26,0x95,0xe5,0x7c,0x13,0xb3,0xe4,0xc3,0x88, -0x3a,0xc3,0x09,0x21,0x8b,0x47,0xa3,0x49,0xc2,0x60,0xd0,0x8f, -0x20,0xf7,0xe7,0x42,0x8f,0x46,0x50,0x73,0x30,0x70,0xe1,0x7d, -0x36,0x52,0x98,0xce,0x82,0x63,0x28,0x79,0x22,0x8c,0xd8,0x05, -0xa7,0x90,0x74,0x8e,0x09,0xd9,0x1d,0x0c,0x20,0x6a,0x2c,0xd0, -0x7b,0x21,0xb2,0x34,0x99,0xcd,0x60,0x44,0x00,0xc0,0xd9,0x39, -0x08,0x26,0xac,0x00,0xde,0xa0,0x72,0x03,0x25,0xdf,0x42,0xfd, -0xe0,0x8c,0x64,0x58,0x83,0x18,0x91,0x3d,0xbb,0x3c,0x0f,0x82, -0x33,0x99,0xc2,0xc3,0xfa,0x18,0x7a,0x0f,0x3a,0xcf,0x1a,0xd1, -0x2a,0x65,0x26,0x87,0xec,0xe9,0x25,0xf4,0x9a,0x23,0xde,0xa7, -0x27,0xc1,0xb9,0x30,0x83,0xfd,0xe4,0x0c,0x96,0x0b,0x17,0x68, -0x31,0x30,0x43,0x8d,0x04,0xc3,0x02,0x5f,0x12,0xd0,0x84,0xd0, -0xa4,0xcb,0x33,0x5a,0x60,0xae,0x80,0x99,0xa0,0x6d,0x40,0x9a, -0x60,0x91,0xb2,0x9a,0x4f,0xe3,0xd5,0x9c,0xf4,0xfb,0x39,0xcc, -0xde,0xc2,0x1a,0x23,0xad,0x53,0x10,0xf1,0x33,0x21,0xa7,0x52, -0x10,0x1a,0x98,0x74,0x2f,0xce,0xc4,0xe3,0x5d,0x12,0xaf,0x68, -0x0d,0x77,0x8c,0x2d,0x32,0x53,0xdd,0x39,0x4c,0x22,0xfc,0x9a, -0x4d,0xe7,0x63,0xb5,0xcc,0x03,0x51,0xb7,0x74,0xce,0x19,0x5e, -0xf7,0xf1,0xbb,0x12,0x3c,0xe8,0xfb,0xe0,0xfc,0x84,0x3f,0x59, -0xd3,0xeb,0xd0,0x9d,0x71,0xa8,0x1c,0x40,0x15,0xbf,0x8c,0x13, -0xbf,0xe6,0x40,0xb2,0xca,0x17,0x23,0xb9,0x2c,0x27,0x14,0xec, -0x33,0x43,0x3e,0x62,0xea,0xfa,0x8e,0x52,0xe0,0x87,0x42,0x0b, -0xc1,0x23,0x9b,0x3e,0xac,0x36,0x54,0x68,0x94,0xbd,0xb1,0x82, -0xba,0x24,0xa0,0x1b,0x53,0xba,0xec,0x04,0x1d,0xc9,0xb1,0x65, -0xcc,0x6d,0x70,0xf6,0xc6,0x83,0xc5,0x26,0xc8,0xdd,0xa3,0x30, -0x8e,0x26,0x4a,0x43,0xe8,0xa9,0xec,0x08,0x61,0xe1,0x1f,0x5b, -0x27,0xd1,0x22,0x9c,0xc9,0x41,0xaa,0x08,0x99,0xaa,0xc7,0x60, -0x54,0x9b,0x2d,0x80,0x68,0x62,0xd1,0x7d,0x6e,0x03,0x08,0x36, -0xc9,0x16,0xf3,0x19,0xec,0x3c,0xf9,0x86,0xc3,0xc3,0xb2,0xd7, -0x7e,0x57,0x25,0xb7,0x71,0x7b,0x31,0xfa,0x4e,0x01,0xa1,0x77, -0x6f,0x80,0xd8,0x0a,0x59,0x0d,0x82,0xfb,0x8e,0xe9,0x2e,0xc3, -0x61,0xa8,0x82,0x24,0x5b,0x99,0x1d,0x4d,0xf6,0xa6,0xc9,0x1b, -0x5c,0x99,0xa3,0xeb,0xfd,0x1e,0x5d,0xa0,0x58,0xf6,0x86,0x4e, -0x75,0x37,0xc5,0x6f,0xa2,0xde,0x0f,0xb3,0xb8,0xf1,0x2d,0x1b, -0x32,0x10,0x1a,0x90,0xe6,0xef,0x24,0xd9,0x26,0x9e,0xcd,0x23, -0x5b,0xff,0x52,0x2e,0xfb,0x17,0x6b,0xde,0xcf,0x53,0xab,0x5f, -0xfd,0x3c,0x0e,0x76,0x8d,0x8a,0xbe,0x2b,0x0d,0x9a,0xe4,0x0b, -0xca,0xe2,0xcf,0x04,0x29,0xab,0xfb,0x6f,0xb5,0x7f,0xab,0xe3, -0x93,0x0e,0xf0,0x70,0xf4,0xba,0x76,0x74,0x2b,0xfe,0xad,0x56, -0xfb,0x37,0xbf,0x0e,0x99,0x6d,0xcc,0xa4,0x7b,0x73,0x55,0x9a, -0xb6,0xeb,0xdf,0x25,0xb7,0x9f,0xaf,0x17,0xde,0xbf,0xf5,0xa0, -0xb8,0xa6,0xa1,0x5e,0x7b,0x3d,0xeb,0xe3,0x57,0x90,0x2a,0x81, -0xe3,0x93,0xa9,0xb7,0x5c,0x6c,0xb2,0x3c,0x77,0xac,0xaf,0x6f, -0x99,0x2d,0x74,0x89,0x39,0x2f,0xc0,0x01,0x73,0x32,0x09,0xa2, -0x39,0x0e,0xc2,0x98,0x92,0xc1,0xd6,0x5c,0x99,0x42,0xcd,0x39, -0xb1,0x21,0xcf,0x01,0x62,0xa2,0xfa,0x0b,0x55,0xa0,0x36,0xaa, -0x89,0x9a,0x84,0x7c,0x57,0xab,0x6b,0xfe,0x3a,0x7e,0x03,0x52, -0xcd,0xb8,0xff,0xf6,0xea,0x9b,0xaf,0x9b,0xfc,0x65,0xaf,0xf6, -0x2d,0x5d,0xaf,0xb5,0x3f,0x1a,0xf5,0x6a,0xf5,0xb4,0x5e,0xeb, -0xd7,0xb6,0xb2,0x3b,0x00,0x2b,0x64,0x6c,0x6b,0x7e,0x67,0x1e, -0xdd,0x74,0xf3,0xf4,0x29,0x41,0xf1,0x62,0xea,0x5d,0xdc,0xec, -0xb7,0x63,0xf2,0x00,0x63,0xb7,0xad,0xc8,0x07,0xcb,0xd4,0x75, -0x9c,0xc8,0x40,0xab,0x2f,0xaf,0xa2,0xaf,0x55,0x5b,0x3f,0xff, -0xe6,0x0b,0x34,0xd0,0x4c,0xe6,0x13,0x95,0x02,0x8f,0xd1,0x8b, -0x50,0x7c,0xfe,0xcd,0x97,0x1c,0x89,0x32,0x7a,0x49,0x1f,0x29, -0x9f,0xa0,0x09,0x27,0x76,0xb6,0xd5,0x13,0xf7,0xbe,0x1d,0xbd, -0xf6,0x1e,0x7d,0xac,0x7c,0xf2,0x35,0xf9,0x51,0x2e,0x05,0x4f, -0x7d,0xaa,0xfe,0xf8,0xe4,0x45,0x49,0x1e,0x5a,0x65,0xb0,0xd1, -0x61,0xbd,0x9e,0x6e,0x5f,0x46,0x69,0xbd,0x45,0xf5,0x0f,0x4b, -0x41,0x55,0xcf,0x86,0xc7,0x50,0xcf,0xb0,0xeb,0x11,0xd1,0x81, -0x08,0x83,0xb2,0x6a,0xeb,0x2d,0xff,0xe0,0xa0,0x5e,0x7f,0xe9, -0xb7,0x29,0x1f,0x63,0x9b,0x33,0xbc,0x2f,0x08,0xd6,0x38,0x38, -0xbf,0xaf,0x87,0xc0,0x59,0x23,0xcf,0x35,0x96,0xe7,0x7f,0x63, -0xff,0xdd,0x9d,0xaf,0xaf,0x5e,0xca,0xd0,0xc8,0x45,0xaa,0x5e, -0xd6,0xeb,0x78,0x5f,0xce,0xc1,0xce,0xa9,0x1a,0x9f,0x2b,0xe9, -0x28,0x6b,0x52,0xa6,0xd5,0xdf,0x45,0xea,0x4b,0xf4,0xbb,0x04, -0x94,0x8a,0x7a,0x5d,0xc5,0xde,0x84,0x82,0x43,0xe0,0x97,0x33, -0x2e,0x7c,0xfc,0xae,0x35,0x9d,0x3d,0x68,0x77,0xee,0xf9,0x66, -0x88,0x97,0x8d,0xb1,0xf6,0x52,0x90,0xcf,0x54,0xce,0xc8,0x31, -0xfc,0x31,0x10,0x83,0x3d,0x8d,0xa6,0x72,0x20,0x09,0xf8,0xb3, -0x44,0x31,0xea,0xf5,0xc5,0xd7,0x4c,0x9c,0x74,0x8a,0xf9,0x92, -0x6c,0x11,0x83,0x8e,0xb7,0x8a,0xa4,0x20,0xf9,0x40,0x14,0x0a, -0x8f,0x91,0x80,0x18,0x1d,0xd7,0xac,0x28,0xf9,0xcb,0x83,0x83, -0x95,0xca,0x8f,0xd9,0x71,0xc2,0x0a,0xcd,0xd9,0x64,0xd9,0x8e, -0xf7,0xe2,0xe6,0xe9,0x09,0x03,0x9e,0x7b,0xb1,0x98,0x01,0xe3, -0x30,0xc2,0x39,0x56,0xcc,0xa0,0xb1,0x26,0x1d,0xd3,0x58,0xc4, -0x59,0x31,0x45,0xf6,0xb0,0xc8,0x48,0x9a,0x38,0xf8,0x6b,0x9a, -0xb1,0xc3,0x3b,0x4c,0x46,0x63,0x4f,0xdf,0x1a,0xf9,0x5a,0x2f, -0x66,0x5c,0x8a,0xc4,0xea,0x26,0x4d,0x26,0xa3,0x57,0x4e,0xec, -0x2e,0x41,0x69,0x76,0xe0,0x29,0x22,0xa9,0x30,0x17,0x93,0x76, -0x55,0x8d,0xa6,0x32,0x68,0x51,0x89,0xa9,0x0a,0x29,0x39,0x68, -0xa4,0x17,0xff,0xe9,0x89,0xb1,0x72,0xcb,0x34,0x00,0x86,0xfd, -0x62,0x00,0x9c,0xa6,0x7b,0x12,0xe6,0x39,0x3d,0xdd,0x97,0xa1, -0x6d,0xa9,0x1d,0xae,0xd6,0xb1,0x55,0xbe,0x8d,0xcd,0xf8,0x73, -0xc3,0xca,0x34,0x84,0xa5,0xe6,0xa1,0x24,0xd7,0xdf,0xf7,0xb7, -0xc5,0x99,0xc1,0x57,0xda,0xe9,0xf5,0xac,0xe6,0xdb,0x5d,0xe1, -0xaa,0x1b,0xee,0x0e,0x9b,0x6d,0xa6,0x2d,0xdf,0xa1,0xc5,0x63, -0x0e,0x09,0x1d,0x39,0x0f,0x61,0x95,0xc1,0x8b,0x0d,0x54,0x7e, -0xa2,0x86,0x02,0x77,0x04,0x89,0xf4,0x31,0x7c,0x73,0x65,0xe5, -0xfe,0x5f,0x2a,0x77,0x15,0x0f,0x1a,0x59,0x42,0xbe,0xb9,0x92, -0x51,0x83,0xc3,0x7c,0xd5,0xf4,0xa7,0xf8,0x68,0x77,0xb5,0xbc, -0x86,0x79,0xf2,0x21,0xf9,0x53,0x12,0x93,0x9b,0x22,0x2b,0xe9, -0xfb,0x38,0x9d,0x98,0x24,0x32,0xb9,0x01,0x04,0x26,0x65,0xc5, -0x46,0x81,0x26,0x21,0x86,0xb6,0xf1,0xb7,0x1b,0xfc,0x7e,0xb3, -0x94,0x2e,0x7d,0xd9,0x8c,0xb3,0x2c,0x8a,0x0e,0xe7,0xe0,0xfc, -0x40,0xb3,0xf5,0xa7,0xb3,0x74,0x4a,0x86,0x6e,0x5f,0x60,0xd1, -0x9a,0xdf,0x07,0x71,0x28,0xba,0xe5,0xb2,0x0c,0x12,0x55,0xa2, -0x08,0xcf,0x7d,0x8a,0x00,0x41,0x35,0x17,0x7d,0x79,0xb1,0x33, -0x15,0x58,0xd0,0x26,0xb3,0x4a,0xd7,0x12,0xec,0x38,0x82,0x00, -0x31,0xf6,0xfb,0x31,0xbd,0x62,0x09,0xcb,0x38,0x9f,0x59,0x47, -0xa6,0xce,0x98,0x53,0x97,0x68,0xa9,0x52,0xed,0x85,0xac,0xad, -0x6b,0x25,0x57,0x2f,0x64,0xdd,0x86,0x5e,0xdd,0xe8,0x93,0x95, -0x6d,0xa7,0xc8,0xe5,0x6e,0x31,0x89,0x7c,0xc7,0x45,0x94,0xd8, -0x2e,0xf6,0x14,0x67,0x94,0xf4,0x57,0x3e,0x43,0xf5,0x1a,0x68, -0x8f,0x7c,0xbf,0x45,0x8e,0xf5,0x68,0x3e,0xd7,0x2f,0xe2,0xe0, -0xab,0x26,0xab,0x6b,0x4d,0x21,0x0e,0xa9,0xcc,0x91,0xbd,0x31, -0xa1,0x79,0x83,0xc7,0x33,0xf6,0xf1,0xb3,0x06,0x85,0xcd,0xfe, -0xd8,0xb3,0xb0,0x67,0x8f,0x49,0xb2,0xf0,0xcc,0xe1,0x32,0x8f, -0x81,0xeb,0x51,0x32,0x04,0x51,0x98,0x90,0x5b,0x82,0xa8,0xd6, -0xa4,0x60,0xac,0x32,0x6b,0x75,0x37,0x7f,0xc0,0x0d,0x58,0xf6, -0x2a,0x51,0x0e,0xee,0x70,0x38,0x18,0x00,0xf2,0x57,0x82,0xc7, -0xdf,0xbd,0x63,0x71,0xdc,0xb7,0x32,0x86,0x0f,0xcb,0x65,0x32, -0x1b,0x6e,0x58,0xfe,0xa2,0xda,0xbf,0x5a,0xa5,0xac,0x30,0x6e, -0x1b,0x34,0xd0,0x83,0x7f,0x31,0xfc,0xbb,0x81,0x7f,0x0b,0xf8, -0x37,0x83,0x7f,0xff,0xef,0xdf,0xf0,0x23,0x60,0xf8,0x07,0xff, -0x8d,0x31,0xb4,0x1b,0xfc,0xfb,0xaf,0xf0,0xef,0x7b,0xf8,0xf7, -0x2d,0xfc,0xfb,0x1c,0xfe,0xfd,0x04,0xff,0x7e,0xac,0xf5,0x9b, -0x1c,0x03,0xd2,0xc1,0x4d,0x21,0x62,0xed,0x84,0xbc,0x8f,0xe6, -0xc5,0x12,0x1a,0x9d,0xc1,0xbb,0x0a,0x4d,0x12,0x38,0x5e,0x44, -0x41,0x1a,0x83,0x2b,0xf5,0x44,0xbf,0x87,0x11,0xac,0xb9,0x4c, -0x31,0x1d,0x52,0x00,0xaf,0xa7,0xe6,0x0f,0x33,0x15,0x2e,0xc1, -0x30,0x40,0x83,0x16,0x6a,0xa4,0x80,0xd8,0x61,0xdd,0x32,0xfc, -0x0c,0x93,0x46,0xd8,0xaa,0x1b,0x5b,0x53,0xd2,0xb0,0xca,0xd4, -0x34,0x0c,0x10,0x5e,0xdf,0xc8,0x35,0x5a,0x27,0xd6,0x8d,0xcb, -0x89,0x38,0x3e,0xb4,0x10,0x01,0xd5,0x51,0xda,0x85,0x55,0x48, -0x1b,0x97,0x88,0x47,0xc7,0x78,0x0f,0x53,0xe4,0x7b,0xef,0xa2, -0x9e,0x1e,0x1d,0xb3,0xf7,0x08,0x22,0xde,0xfa,0x1c,0xda,0x0e, -0xe2,0xc5,0xa6,0xaa,0xdc,0xbc,0xf5,0xa1,0x37,0x33,0xd7,0x32, -0xb0,0xf9,0xc7,0x76,0x1c,0xcd,0xda,0x36,0x08,0x4b,0x65,0x7e, -0x3a,0xcc,0x16,0xd0,0x70,0xb4,0x5b,0x57,0x81,0xb5,0x31,0xe8, -0x9e,0xe1,0xd3,0x12,0x5d,0x93,0x26,0x43,0x0b,0x0c,0x26,0x3b, -0xd8,0x10,0x11,0x5c,0x2f,0x04,0xb5,0x54,0xfb,0xa8,0x26,0xe2, -0x49,0x7a,0x3b,0x93,0x69,0x2d,0x4c,0xbb,0xaa,0xc1,0x96,0x5e, -0x27,0x1d,0x63,0x12,0xa4,0xb0,0xb0,0x71,0xda,0x09,0xa7,0xbd, -0xb5,0xb0,0x9d,0xf6,0xc5,0x63,0x3a,0x5a,0xdd,0x45,0x75,0x7e, -0x3f,0xeb,0x8b,0xe1,0x7c,0x3a,0x8d,0x65,0xf6,0x79,0xdf,0xf4, -0x92,0x4c,0xba,0xe8,0x73,0xe0,0x3c,0x7e,0xbb,0x94,0xc6,0xb0, -0x30,0x3c,0xb3,0x87,0x1b,0x0a,0x5f,0x5c,0x13,0x32,0xf2,0x39, -0x2c,0x4c,0x3b,0xd9,0x63,0x8a,0xa6,0xa8,0xb6,0x94,0x18,0x84, -0x75,0xfd,0xe8,0x86,0x65,0x14,0x1e,0x91,0x08,0xc4,0x06,0x35, -0x58,0x77,0xe1,0xf3,0xc1,0x41,0x2d,0xc2,0x67,0x6a,0x35,0xae, -0xbb,0xb8,0x11,0xf4,0x07,0xa0,0x24,0x37,0x00,0x86,0xc9,0x07, -0x00,0x6a,0x56,0x23,0xb2,0x45,0x81,0x93,0x40,0x0a,0x4e,0x7c, -0x15,0xe4,0x0f,0xd4,0x66,0x96,0xc0,0xf8,0x6a,0x73,0xab,0x61, -0xa9,0x47,0x8d,0xab,0xdd,0xd6,0x3a,0xbc,0xbe,0xa5,0xfc,0x8f, -0x6b,0x6d,0xd9,0xca,0x20,0xd0,0xed,0xfc,0xb8,0x26,0x61,0x6f, -0x1c,0xd8,0xc5,0x4e,0xd8,0xa5,0x03,0x3b,0x80,0x7a,0xf1,0x77, -0x2e,0x7f,0xd7,0xf2,0xf7,0x87,0x5a,0xbb,0xf6,0x2f,0xd8,0x5c, -0xee,0x3e,0x8c,0xd7,0x24,0x95,0x46,0xc0,0x1e,0x7d,0x9a,0xab, -0xf9,0x97,0x78,0xad,0xf3,0x19,0x40,0xe3,0xbc,0x40,0xa5,0x86, -0xb2,0xf4,0xa8,0xd6,0xd6,0x3d,0x10,0x58,0xfd,0x17,0xd8,0x55, -0x67,0x8a,0xcc,0x46,0xa8,0x49,0xdb,0xda,0x95,0x76,0x55,0x95, -0x40,0xcb,0xbf,0x96,0xd1,0x52,0xa5,0xd9,0x7c,0x98,0x50,0xa5, -0x5b,0xa4,0xa7,0x27,0x5d,0x3b,0x5a,0xd0,0x4b,0xce,0x72,0x78, -0x8f,0x17,0x91,0x2d,0x14,0x35,0xe5,0x57,0xaa,0x6b,0xe8,0xd5, -0x03,0x3c,0xb4,0x86,0x77,0x68,0xab,0x8d,0xb6,0x57,0x4b,0x64, -0x39,0x10,0x96,0x1b,0xf9,0xe8,0x3b,0x42,0x66,0x5d,0xdc,0x1b, -0x2c,0x81,0xa3,0x7c,0x64,0x28,0x48,0x4b,0xc9,0xc3,0x6b,0x46, -0x7e,0xca,0x09,0x1f,0x7e,0x18,0x60,0x67,0xc9,0x4f,0x08,0x68, -0xd2,0x78,0xcb,0x82,0xf3,0x96,0x85,0x94,0x5e,0x0a,0x66,0xba, -0x26,0x78,0x88,0xec,0x97,0x83,0x03,0x4a,0xfa,0x38,0x94,0x2b, -0xdf,0x5a,0x8d,0x70,0xcd,0x92,0xdb,0x18,0x17,0x31,0x91,0xd4, -0xb4,0x4f,0x4f,0xb8,0xed,0x90,0x4a,0x37,0xb8,0x0a,0x8f,0x38, -0xba,0x83,0xd4,0xb3,0xbc,0xa0,0x12,0xb5,0x46,0x0d,0xcd,0x25, -0x6f,0x67,0xb8,0x97,0x09,0xae,0xa8,0x53,0xe5,0xa7,0x6f,0xac, -0xe7,0x73,0x7a,0xbf,0xa0,0x7c,0x3b,0x8c,0x4e,0x86,0x15,0x67, -0xa3,0x55,0xa6,0x58,0x89,0xaf,0xca,0xa2,0xee,0xe5,0xef,0xed, -0xe4,0x0c,0x40,0xd0,0x12,0x01,0x72,0xa6,0x0c,0x39,0xce,0x25, -0xec,0xf8,0x79,0x12,0x67,0xab,0x97,0x76,0xb0,0x4e,0xe9,0x8c, -0x2f,0xb8,0x4a,0x39,0x0a,0x77,0x3b,0x1f,0xa3,0x95,0x82,0x5b, -0xc5,0x37,0x68,0x14,0x80,0x40,0x20,0x8b,0xe5,0x93,0x74,0x3d, -0x5f,0x90,0xb6,0x9d,0x2f,0xec,0x5e,0x71,0x7c,0xff,0xd9,0x73, -0xb5,0x0a,0xdf,0xc0,0xb4,0xf2,0x72,0x4c,0x0a,0xbb,0x5c,0x9c, -0xd5,0x65,0x34,0x06,0xf9,0x46,0xf4,0xa8,0x17,0x8f,0x05,0xa0, -0x1b,0xb4,0x55,0xef,0xa9,0x4f,0xc7,0xc4,0x02,0xf6,0x0d,0xb8, -0x10,0x20,0xb5,0x73,0x25,0xcf,0xb9,0x8c,0xbb,0x6f,0x59,0x13, -0x2b,0x25,0x89,0x2d,0x94,0x6b,0x6f,0xa4,0xdb,0x6f,0x83,0x58, -0x28,0x03,0xfd,0x67,0x9a,0x20,0xab,0xaa,0xab,0xa6,0x08,0x45, -0x4c,0x5d,0xb6,0x84,0xcf,0x23,0x23,0xce,0xe7,0x16,0x08,0xaf, -0xf6,0x89,0x56,0xa9,0x5d,0x5d,0x80,0x00,0xeb,0x12,0x61,0x1b, -0x66,0x15,0x0d,0xa2,0x2a,0x71,0x41,0xdb,0xb5,0xbf,0x16,0x40, -0x9c,0x1e,0x94,0xdf,0x63,0x5e,0x45,0xa1,0x5f,0x2f,0xaf,0xc5, -0x02,0x97,0xbc,0xd3,0xcc,0xd4,0xfc,0xe5,0xba,0x14,0x09,0xf2, -0xd0,0xb4,0xce,0x02,0xba,0xcd,0xaf,0xdf,0x30,0x2a,0xa9,0xd7, -0x6d,0x7b,0xbd,0xbf,0xbe,0x43,0x87,0x97,0xbd,0x4f,0xae,0xa2, -0xbf,0xf6,0x31,0xd4,0x40,0xaf,0xfe,0xba,0xf1,0x11,0x25,0xfd, -0xeb,0xbf,0xe0,0x4f,0x00,0xff,0x5e,0x8f,0xea,0xf0,0x57,0xe0, -0x53,0xb3,0xd1,0xe5,0xb7,0x5e,0xdc,0x78,0xfb,0x31,0x00,0x1c, -0xa5,0x22,0xa7,0x15,0xb4,0xd5,0xc1,0xa0,0x5d,0xe2,0x31,0x65, -0xad,0xad,0x1f,0xbc,0x16,0xcc,0xfd,0xc3,0x32,0x18,0xce,0x6f, -0xde,0x2c,0xe7,0xd3,0xcf,0xe4,0x61,0x02,0x2d,0x14,0xe6,0xcf, -0x20,0xbc,0x00,0x98,0xf5,0x33,0x30,0xe1,0x19,0x00,0xfd,0xf0, -0x3c,0x10,0xbc,0xfc,0x79,0xb1,0x50,0xf3,0xc7,0x56,0xdc,0x5a, -0x45,0xc4,0xc2,0x29,0xf4,0xad,0x5e,0xb5,0x2d,0x00,0x30,0xa9, -0x06,0xfc,0x7c,0xbd,0x98,0xcf,0xf0,0x58,0x3a,0x9e,0x10,0xe8, -0x4d,0x35,0xe8,0x17,0xb0,0xe6,0x1a,0x11,0xd0,0xb2,0x14,0xc8, -0x5b,0x9b,0x55,0xe4,0xba,0x74,0x05,0x89,0xc0,0xbe,0xaf,0x51, -0x55,0x28,0xf9,0xd2,0x92,0x87,0x5e,0x58,0xc7,0xe5,0xe5,0xa9, -0x4f,0x38,0x64,0x10,0x68,0x67,0x3c,0x45,0xd6,0x11,0x3b,0x39, -0x82,0x29,0xac,0xee,0xe5,0x47,0x23,0x85,0xf4,0x2f,0x79,0x50, -0x17,0x33,0xd4,0x3e,0x2f,0x5f,0x53,0x7e,0x92,0x30,0x27,0x74, -0x52,0x67,0xf2,0x91,0xce,0x8a,0xfd,0xcc,0x06,0xe2,0xb6,0x04, -0x77,0x2f,0xe8,0x77,0xd2,0xab,0xe0,0xe0,0xe0,0xf6,0x2a,0xe8, -0xf8,0x2b,0x2b,0xbc,0xb6,0xad,0x11,0x1b,0xd1,0xad,0x48,0xeb, -0xb7,0xa0,0x1c,0x4a,0x71,0xdc,0x47,0x78,0xa0,0xe6,0x7f,0x5c, -0xd5,0x26,0x13,0xe0,0xbb,0x89,0x1f,0xd2,0xe1,0x29,0xb7,0x3a, -0x80,0xd8,0xb1,0x39,0x02,0xee,0x42,0x4f,0xde,0x26,0x73,0x8e, -0xad,0x7d,0x0d,0xe3,0xd7,0xf1,0x7b,0xfc,0x2e,0x6b,0xc3,0x92, -0x0b,0xfe,0x39,0x71,0xcf,0x37,0xb4,0x65,0x23,0xd8,0x57,0x0f, -0x53,0x6f,0xc3,0xc6,0x46,0x2b,0x8d,0xe0,0xfb,0x64,0xba,0xf0, -0x85,0x0d,0x62,0xe7,0x28,0x4b,0x5e,0xfe,0x51,0x61,0x79,0xd8, -0x0b,0xff,0xaa,0x1e,0x39,0xa0,0x2b,0x76,0xbe,0x9f,0xe5,0x52, -0xe9,0xf3,0x8d,0x64,0x55,0x08,0x95,0x94,0xb1,0x4d,0xd9,0x2a, -0x0a,0x64,0x3c,0xf4,0x6f,0x6e,0xda,0x15,0x76,0xe3,0x99,0xd1, -0x4a,0x1a,0xb1,0x3a,0xbf,0xa2,0x84,0x0e,0x33,0x06,0x8f,0x9b, -0x93,0xd8,0x72,0x29,0x23,0x83,0x1a,0x9b,0x50,0xd7,0x9c,0x40, -0x5e,0x0e,0x01,0xfe,0x9a,0xe1,0xbf,0xa1,0xc4,0xef,0xc9,0x70, -0xcb,0x0d,0xeb,0x28,0x23,0x27,0xd3,0xca,0xa5,0x5b,0x51,0xa6, -0x67,0x01,0xf5,0x0b,0x35,0xb6,0x9d,0x52,0xff,0x35,0x99,0x4f, -0x93,0xd5,0x72,0x53,0x00,0xd3,0xed,0x2b,0xab,0x21,0x7a,0xf7, -0x45,0x12,0xa3,0xcf,0x48,0x2b,0x9c,0x3b,0x27,0x58,0x2d,0x2b, -0xaf,0x47,0xc2,0x21,0x73,0x28,0xc1,0xaa,0x51,0x48,0xa4,0x9f, -0xcd,0x27,0xf2,0xd3,0xd2,0x42,0x30,0x68,0x83,0x5d,0x9f,0xfa, -0x71,0x99,0x2c,0x92,0x8d,0x56,0xef,0xca,0x01,0xad,0x7a,0xcf, -0x87,0x35,0xdf,0x45,0x1c,0x7e,0x9a,0x5f,0x46,0x20,0x09,0x79, -0xb1,0x18,0x73,0xe4,0xd5,0xe2,0x2e,0xb1,0x19,0x52,0xa0,0x58, -0x3d,0x35,0x33,0x82,0x44,0xcd,0x4c,0xcb,0x9b,0x1d,0x45,0x38, -0x41,0x35,0x6d,0x08,0xfb,0x1c,0x98,0x20,0xe3,0x15,0xb4,0x4e, -0x23,0x63,0xd7,0x37,0x32,0xf6,0x35,0x3a,0x41,0xe3,0xa7,0x50, -0x3f,0xb5,0xd0,0x1d,0xec,0x57,0xb0,0xa2,0x4d,0x9f,0xab,0x4d, -0x7f,0x0d,0x6f,0xea,0x29,0xab,0x5a,0x32,0xd6,0x4a,0xca,0xf1, -0x56,0x52,0x6d,0x01,0x90,0xab,0x83,0xf7,0xa3,0xf8,0xcb,0x74, -0x96,0xf0,0x94,0xb6,0x83,0x62,0xa7,0x3b,0xb0,0x84,0xb7,0x83, -0x55,0x22,0x50,0x8c,0xd8,0x0b,0xf7,0xef,0xc5,0x8d,0x22,0x8d, -0x15,0x8c,0x21,0xfa,0xbe,0x9d,0x4f,0x36,0xb7,0x3b,0x05,0xdf, -0xc1,0x27,0xc1,0x77,0x35,0xdb,0xf4,0xfe,0x73,0x98,0xff,0x21, -0x2d,0x56,0x14,0x56,0x34,0x1a,0xa8,0x53,0xc3,0xe8,0xbd,0x86, -0xbd,0x1c,0x9d,0xa9,0x21,0xd1,0xa4,0x68,0xdf,0xd3,0x3a,0x65, -0xaf,0xc1,0x6f,0xc0,0x5d,0x02,0xb7,0x4a,0x97,0xc7,0x00,0x9e, -0xd7,0xe4,0xb6,0x29,0x2d,0xe2,0x44,0x18,0x98,0xb2,0x22,0x5c, -0x9e,0x58,0x13,0x80,0x6a,0x88,0x05,0xe5,0x0b,0xb7,0x88,0xa3, -0x70,0x65,0x9a,0x0d,0xf2,0x1d,0x88,0x2b,0x62,0xb6,0x67,0x19, -0x3b,0x1f,0x66,0xdb,0x9c,0x06,0xd2,0xb2,0x22,0xb1,0xd5,0xa3, -0x93,0xc3,0x5f,0xff,0x63,0xcb,0x0e,0x55,0x29,0x22,0x30,0x3a, -0x24,0x9c,0xd0,0x60,0x88,0x97,0x6e,0xca,0xe7,0xb3,0x91,0x7e, -0x5f,0x70,0x17,0x32,0x50,0x39,0x76,0x49,0x5c,0x53,0x7e,0xb2, -0x6e,0xd3,0xd5,0xd4,0x15,0x44,0xf9,0x02,0x98,0xb8,0x55,0xe8, -0xb1,0xc6,0xdc,0x67,0xbb,0xd4,0x2c,0xfa,0xd0,0x3e,0x57,0x4f, -0x27,0xdf,0xb0,0xe0,0x0a,0x1f,0xbb,0xd8,0xbe,0x3a,0x3e,0xb5, -0xf1,0xcf,0xf3,0x54,0x34,0x65,0x53,0x9d,0x34,0xed,0xbd,0x16, -0x1b,0x6f,0x7a,0x7f,0x80,0x0b,0xd6,0xac,0xd4,0x39,0x1c,0x6b, -0xb4,0x5f,0xfe,0x2e,0x7e,0xfd,0x4f,0x90,0x07,0x34,0x82,0x92, -0x97,0x4c,0xf4,0x1c,0xf5,0x7e,0xf9,0x7b,0x10,0x41,0xee,0x2f, -0x7f,0x0f,0xe1,0xa7,0xef,0x03,0x58,0x70,0xf5,0xeb,0x7f,0xc2, -0x86,0x0f,0x1e,0x22,0x28,0x03,0x09,0xf0,0x1e,0x52,0x42,0x88, -0x09,0x96,0x0b,0x37,0x20,0xf0,0x5b,0x1b,0x3d,0xbb,0xbd,0x91, -0xf4,0x0e,0xf1,0xeb,0xdb,0x2c,0x8d,0x67,0x1e,0xd4,0x81,0x7c, -0x92,0x7e,0x15,0x01,0xd2,0x7a,0xe3,0xb0,0x4e,0x8b,0x40,0x05, -0xdd,0x80,0xd5,0xdc,0xa4,0x80,0xe1,0xb3,0xe5,0x3c,0xcb,0xc8, -0x81,0xa2,0x2f,0x92,0x9f,0x1f,0x70,0x79,0x07,0x6b,0xfd,0xa8, -0xc7,0xe0,0xa8,0x9d,0x1b,0xf2,0x11,0x54,0x36,0xfc,0x97,0xce, -0x6e,0x54,0x00,0xfe,0x72,0x54,0x06,0x89,0xe0,0x82,0x7e,0x27, -0x0f,0xf8,0x35,0xa5,0xa7,0x6f,0x13,0xcf,0x60,0xf3,0x4b,0x30, -0x93,0x60,0xa7,0x43,0xd8,0x7a,0x58,0x70,0x3c,0x5c,0x7e,0xf9, -0x3b,0xb0,0xb4,0xf1,0xcb,0xdf,0xaf,0x05,0xac,0xf2,0x7e,0xf9, -0xfb,0x55,0xd0,0x0d,0xdb,0x30,0xf8,0x7f,0xf9,0x7b,0x1a,0x19, -0x58,0xfc,0x2c,0x7a,0xa4,0x5d,0x4c,0x1e,0x66,0x64,0x41,0x3e, -0x05,0x94,0xc8,0x1e,0x3e,0x43,0x8a,0x07,0x99,0x07,0xe5,0xfd, -0xab,0xf0,0x22,0x40,0x6e,0xd9,0x10,0x7f,0x05,0xde,0xa7,0x57, -0xd9,0x21,0xd4,0x72,0x70,0x80,0x3f,0x57,0xf0,0xee,0x33,0x37, -0x7f,0xfd,0x4f,0xa7,0xa2,0xd0,0xae,0xa8,0x03,0x99,0x4e,0xbf, -0xa6,0xbe,0xf6,0x67,0x84,0x04,0xe2,0x9f,0xfa,0xf1,0x59,0xe0, -0x7f,0x0c,0x7f,0x1a,0xe8,0x07,0xf3,0x7d,0x6a,0x6d,0xec,0xa8, -0x16,0xc5,0x2b,0xd5,0xf2,0xa5,0xaa,0x7d,0x56,0xec,0x3a,0x76, -0xfd,0x5d,0xa8,0x18,0xaa,0xec,0x82,0x08,0x4f,0x12,0xa0,0x25, -0x10,0xc8,0x1e,0xfb,0x2d,0xc4,0xb3,0x38,0x16,0x6a,0xbf,0xad, -0x32,0x28,0xbd,0x0c,0x2c,0x20,0x30,0x78,0xbd,0x82,0x87,0xa0, -0x8b,0x29,0x80,0xde,0x64,0x09,0x6a,0x61,0x68,0x50,0x22,0x30, -0xa4,0x5c,0xff,0xbe,0x14,0x30,0x2b,0xec,0x11,0x4b,0xae,0x95, -0x00,0xea,0x1a,0xb2,0xdd,0x81,0x47,0xfa,0x02,0x06,0x3a,0x0d, -0x7d,0xa9,0x17,0xf4,0x80,0x74,0x41,0x41,0x89,0x78,0x72,0xec, -0x63,0x8c,0x1e,0xac,0x97,0x5f,0x42,0x7c,0x09,0x85,0x8d,0x82, -0x7d,0x81,0x2c,0xf2,0xde,0x3c,0x70,0x3d,0xe2,0x0c,0x75,0x6b, -0xd0,0x1a,0x39,0xef,0xc0,0x23,0xa9,0xbc,0x82,0xd8,0x76,0xe1, -0xa1,0xee,0xf1,0x20,0x00,0x81,0x6a,0x37,0x50,0xb4,0xda,0xf0, -0xce,0x4d,0x86,0xb2,0xd8,0x42,0xc0,0x0c,0xbf,0xbf,0xfe,0x67, -0xa7,0xa0,0xf2,0x64,0xb5,0x22,0xaf,0x72,0x5c,0x0a,0x15,0x4f, -0x4a,0x75,0xab,0x97,0x03,0x96,0x5c,0xb1,0x1a,0x76,0x7d,0x4d, -0x04,0x14,0x67,0x08,0x02,0x15,0x76,0xa3,0xa0,0x95,0xfe,0xd5, -0x2f,0x7f,0x93,0x5d,0xd7,0xa0,0x7e,0x86,0x66,0xc2,0x16,0xb7, -0x9a,0xcd,0x05,0xa6,0xba,0x62,0xa0,0x4e,0x20,0xe0,0xb9,0x81, -0x85,0xfd,0x4f,0x02,0x10,0xf3,0x10,0x07,0x21,0x4a,0xa6,0x29, -0xc6,0x51,0xa7,0x92,0xef,0x48,0x97,0x7b,0xf6,0xc7,0x7d,0x31, -0xba,0xd6,0x1a,0x38,0xde,0xb6,0x1f,0x53,0xd8,0x13,0xce,0x08, -0xd6,0x5b,0x33,0x3d,0xe6,0x02,0x5e,0x92,0xfa,0x49,0xa4,0xe8, -0xec,0x9a,0xa4,0xf5,0xc1,0xc1,0xda,0x64,0xb4,0xd7,0x9f,0xa8, -0xac,0xa7,0x27,0x95,0xf8,0xc9,0x9a,0xac,0xb6,0xb0,0x01,0x30, -0x6a,0xb1,0x11,0xf0,0x8b,0xfa,0xed,0x5a,0x68,0x66,0x42,0xab, -0x05,0xf3,0x8b,0x2b,0xcf,0xf8,0x87,0x65,0x2e,0x7a,0xc7,0xd3, -0x3d,0x0b,0x9d,0x99,0xec,0xf5,0x93,0x9e,0xee,0xe5,0x6f,0xe5, -0x74,0x6f,0x8b,0xb0,0xee,0x52,0x29,0xd8,0x66,0x76,0xd5,0x32, -0x62,0xe5,0xe0,0x0c,0x2b,0xc5,0x41,0x12,0x4a,0xeb,0x23,0x5b, -0xfe,0x4c,0x95,0xb8,0x2f,0x2a,0x5f,0x12,0x94,0x94,0x60,0xa9, -0x29,0x0e,0xae,0x3c,0x55,0xa6,0xb5,0x2e,0x55,0xaa,0xcd,0xc1, -0x55,0x71,0x91,0xd1,0xcd,0xc9,0x1d,0x76,0x00,0xbc,0xa2,0xae, -0xbc,0x0c,0x7c,0x1a,0x59,0x00,0x05,0x12,0xda,0xe5,0xa4,0x76, -0xe3,0x97,0xbf,0x5d,0x71,0xa2,0xd4,0xb1,0x8d,0xcb,0x60,0x87, -0xb0,0xc2,0xd2,0x22,0x7f,0x97,0x20,0xf7,0x9f,0xfe,0x3b,0xc4, -0x88,0xf5,0x36,0x88,0x06,0xc4,0x15,0x1e,0x29,0x5c,0x64,0x13, -0xe4,0xae,0x2e,0xd5,0xe6,0x9b,0x1a,0xc7,0x33,0xe3,0x2c,0x92, -0x0b,0x11,0x13,0xdd,0x62,0xa6,0x17,0x27,0x56,0x64,0x35,0x29, -0xe3,0x26,0x8c,0x0b,0x7e,0x52,0x17,0x8a,0x58,0x16,0xc7,0xd9, -0x1e,0x66,0x04,0x74,0x8e,0xd6,0x8b,0xfb,0xda,0xe7,0xd9,0x40, -0x65,0xa7,0x78,0xc5,0x68,0xc4,0x1f,0x07,0x86,0x88,0xfd,0xa7, -0x27,0x37,0x31,0xc4,0xc4,0xae,0xc7,0x83,0x11,0x47,0x90,0xc0, -0x34,0xa5,0xa4,0x29,0x21,0xc6,0x04,0xe0,0x1b,0xfe,0x46,0x94, -0x2b,0x38,0x77,0xa0,0x73,0xcb,0xc1,0x03,0x04,0x47,0x3b,0xa3, -0x36,0xd3,0x29,0x0d,0x97,0xa2,0x81,0x69,0x12,0x74,0x8a,0x18, -0x88,0x41,0x92,0xad,0x60,0x63,0x70,0x84,0xe6,0x53,0x12,0x54, -0x07,0xe2,0x43,0x07,0x6f,0xb1,0x4c,0xed,0xcd,0xb0,0x9d,0x51, -0xda,0x01,0x1c,0x82,0x1b,0x2b,0x33,0xa0,0xb1,0xa8,0x98,0x22, -0x45,0x47,0x88,0xed,0xc0,0xaa,0xaf,0x10,0x37,0x1d,0xe0,0x43, -0x15,0x23,0x9a,0x82,0x02,0xa2,0x8a,0xd6,0x7c,0x44,0xeb,0xd6, -0x56,0x08,0x19,0xb3,0x8f,0x3d,0x5b,0x01,0x45,0x51,0x84,0xd2, -0xf1,0xf4,0xc4,0x8f,0xd0,0xdd,0xdd,0x5e,0x2f,0x00,0x42,0x03, -0x74,0x0e,0xa1,0x9f,0xfa,0xed,0x5e,0x4f,0x2a,0x02,0x48,0x95, -0xaa,0xa0,0xdf,0xdf,0x6e,0x69,0xc1,0x4d,0xf2,0x80,0xce,0x3a, -0x96,0xf3,0x74,0x54,0xdc,0xa6,0xa8,0xb5,0x97,0x04,0xf8,0x4b, -0x10,0xe5,0x53,0xc2,0x7c,0xca,0x0f,0x05,0x98,0x1f,0x0b,0x29, -0x3f,0x15,0x52,0x7e,0x28,0xe0,0xf9,0xb1,0x90,0xf2,0x53,0xb1, -0xae,0x56,0xa1,0x54,0x21,0xe5,0xa7,0xd6,0x33,0xdb,0x2a,0x05, -0x28,0x2d,0x99,0x8b,0x75,0x88,0x4d,0xb1,0x16,0xf1,0xb6,0x58, -0x8f,0x98,0x46,0xeb,0xc3,0x75,0x7d,0x73,0xb8,0xa9,0xbf,0x3d, -0x7c,0xab,0x86,0xe9,0x2f,0x7f,0x6b,0x5d,0xe1,0xd0,0x2e,0x22, -0x0e,0x4b,0x10,0x87,0x25,0x88,0x41,0x71,0xff,0xed,0xaa,0xc0, -0xf7,0x52,0x94,0x41,0x09,0xca,0xa0,0x04,0x25,0x68,0x05,0x87, -0x58,0x41,0x54,0xfa,0x5d,0x2d,0x35,0xed,0x1e,0x4f,0xaa,0xab, -0x78,0xd6,0xf2,0x36,0x62,0xed,0x5b,0x4b,0x45,0x3a,0x65,0xcd, -0xd2,0x99,0xf7,0xf6,0xc8,0x38,0x7a,0x9c,0xfa,0x36,0x48,0xdf, -0xd9,0x97,0x1a,0xf1,0xc9,0xf3,0xfc,0x2f,0x61,0x3e,0xe5,0x87, -0x02,0xcc,0x8f,0x85,0x94,0x9f,0x0a,0x29,0x3f,0x14,0xf0,0xfc, -0x58,0x48,0xf9,0xa9,0x58,0x57,0xab,0x50,0xaa,0x90,0xf2,0x53, -0x21,0x45,0x6f,0x99,0xcd,0x4e,0x57,0xee,0x8c,0x6d,0xa8,0x6f, -0x73,0x33,0x67,0x2e,0xfb,0xcb,0xc2,0x44,0x5a,0x02,0xb0,0x6b, -0x5e,0xcd,0x81,0x17,0x77,0xaa,0x2a,0xe7,0xd9,0x3d,0xf3,0xde, -0x98,0x34,0xcd,0xe6,0x48,0x73,0x38,0x49,0x17,0x9f,0xda,0xdb, -0x22,0x2b,0x9d,0xac,0xb1,0x96,0x6c,0xee,0x53,0x06,0x8b,0xe8, -0xaa,0xf2,0x5e,0x1a,0x7f,0x5a,0xa2,0xd7,0xf8,0xf5,0x3f,0x44, -0x03,0xa3,0x02,0xf4,0x7d,0x1b,0x9e,0x3d,0x17,0x7d,0xf5,0xe9, -0x0f,0x51,0x98,0x5c,0xaa,0xfd,0xb6,0xc9,0xc8,0x47,0x77,0x5c, -0x07,0xe8,0xcb,0x0d,0x3d,0xba,0x85,0x82,0xf5,0x80,0x40,0x60, -0x61,0x95,0x78,0xc7,0xe9,0xd6,0x41,0xd2,0xc3,0x6a,0xf1,0x60, -0x4e,0x6b,0x38,0x1b,0xcd,0x1d,0xe8,0xa1,0xf9,0x06,0xb6,0xa1, -0x23,0x0c,0xe8,0x24,0x11,0x46,0xd4,0x6c,0x59,0x48,0xb8,0x40, -0x81,0x7c,0xdf,0x8a,0x84,0x2a,0x6b,0xef,0xe7,0x64,0x09,0x10, -0x46,0x85,0x36,0x7b,0xeb,0x20,0x22,0x07,0x49,0xe4,0xd5,0x4a, -0x3d,0xc3,0xbc,0xb2,0x0e,0x23,0x72,0x99,0x44,0xe9,0xea,0x19, -0x27,0xc5,0x2a,0xca,0x15,0xe1,0xec,0x0b,0xd4,0xd4,0xe0,0xc3, -0xa4,0x41,0x0c,0x83,0x29,0x83,0x58,0x86,0x13,0x86,0x8e,0x30, -0xaa,0xc1,0x9a,0xdc,0x16,0xaf,0xc7,0x71,0x32,0xb4,0x37,0x3c, -0x3c,0x8a,0xf5,0x54,0x8f,0xcc,0x67,0xe9,0xf0,0x73,0xd8,0xd6, -0x4f,0x3e,0x75,0x4e,0xc2,0xf2,0x67,0x60,0x04,0xe7,0xd9,0x2f, -0xba,0x10,0xda,0xb5,0x2e,0xe3,0xc7,0xa8,0x34,0x53,0xe9,0xf6, -0x78,0x32,0xc0,0x80,0x53,0xa5,0x4e,0x85,0x8a,0x74,0x78,0x80, -0x71,0xbe,0x42,0x4f,0xa9,0x40,0xb4,0xc0,0xc0,0xda,0xec,0xb1, -0xca,0xeb,0x35,0x9a,0x67,0xe2,0xf8,0xa2,0x79,0xce,0x31,0x87, -0xe3,0xc9,0x04,0xf6,0x41,0x5e,0xaf,0x75,0xd9,0x3c,0x15,0x27, -0xa7,0xcd,0x53,0x48,0x66,0x7b,0x88,0x30,0x38,0x0f,0xd8,0x96, -0xcb,0xd4,0xfe,0xe7,0x2c,0x2e,0x3d,0xea,0xd1,0xb9,0xf6,0x81, -0xa6,0x14,0x4b,0xe7,0xc4,0x1b,0xfb,0xcd,0x49,0x08,0xfb,0x4e, -0xc8,0x23,0x9e,0xfb,0xe9,0xf3,0xed,0x93,0x0b,0xde,0x1d,0xad, -0xc5,0xc6,0x67,0x15,0x84,0x16,0xd9,0x93,0x38,0x1b,0xc7,0xf9, -0x0c,0x58,0x59,0xdd,0xc5,0x8f,0x71,0x9a,0xe6,0x33,0xb6,0xda, -0xdf,0xa5,0x83,0x53,0x58,0x68,0x84,0x55,0x52,0x01,0x45,0x4e, -0xa3,0x61,0x1d,0xc1,0xf0,0xd1,0x73,0x9c,0xc6,0x6f,0x19,0x1d, -0x56,0xb7,0xc4,0xe9,0x05,0xb1,0xd4,0xe2,0xf4,0xe9,0xa9,0x38, -0x83,0x24,0x59,0xef,0x1e,0x38,0xcf,0x5d,0x9c,0xc7,0x22,0xbc, -0x6c,0x5e,0xba,0x38,0x2f,0x44,0x78,0xd1,0x97,0x6d,0x7e,0xc5, -0x12,0x2f,0x37,0x38,0xd6,0x55,0xf3,0xc6,0xe7,0xb4,0xa8,0x07, -0xcf,0x96,0xb8,0xeb,0xce,0x93,0x3e,0xf5,0xed,0xa0,0x59,0xb9, -0xce,0x1c,0x47,0x92,0x43,0x4d,0xe5,0xe8,0x6d,0xa5,0x53,0xb4, -0xeb,0x3a,0x48,0x5d,0x47,0x9e,0xdb,0xeb,0x8d,0x15,0x79,0x38, -0x1e,0x43,0xef,0x7b,0x6e,0xf7,0x37,0x56,0xe4,0xeb,0x78,0xdc, -0x51,0x1e,0xff,0xaf,0xa2,0x66,0xd8,0x3a,0x38,0x68,0xb6,0x8e, -0x4f,0xae,0x36,0xb0,0x15,0xbc,0x8a,0x1a,0xcd,0x93,0xd6,0xe9, -0xc1,0x41,0xa3,0xd9,0x0a,0x4f,0xae,0xd6,0x5d,0xee,0x8c,0x36, -0x01,0x9e,0x9d,0xe5,0x20,0x01,0x04,0x21,0xc3,0xf0,0x14,0x20, -0x99,0xc5,0x6d,0x49,0xa1,0x2f,0xdb,0xe7,0x34,0x6b,0x2b,0x4c, -0xf3,0xf3,0xb7,0x92,0xfc,0x2e,0xfd,0x0d,0x32,0x0e,0xc9,0x5c, -0xcd,0x05,0x5e,0x6f,0x49,0x40,0x29,0x26,0x12,0x86,0x5f,0xf2, -0x20,0x4c,0x92,0x04,0xe1,0x97,0x1c,0x88,0x64,0x44,0x69,0xff, -0x39,0x54,0xc8,0xf3,0x09,0x92,0x76,0xbb,0x66,0x3b,0xdd,0xae, -0xce,0x4a,0xdf,0x8a,0xe2,0xa9,0xb8,0x8b,0x5b,0xdd,0xc6,0xb9, -0x98,0x75,0xaa,0x83,0xd7,0xdc,0xdc,0x99,0xe5,0x40,0x25,0x62, -0xeb,0x54,0xc4,0xc5,0x6d,0x67,0x38,0xe8,0xed,0x73,0x14,0xbd, -0x9a,0xd8,0x89,0x9f,0xf7,0xbf,0x05,0xec,0x9c,0x5c,0xc0,0x4d, -0xc9,0xdb,0xca,0x85,0x48,0x9e,0xe7,0xf6,0x9e,0x3c,0xcf,0x77, -0x27,0x2f,0xc7,0xfb,0x7d,0xf6,0xf2,0xa5,0x75,0x95,0xb4,0xc6, -0xc9,0x29,0xab,0x87,0xdb,0xb4,0xb5,0xa5,0xdb,0x98,0x07,0xee, -0x35,0x33,0x2b,0x11,0x37,0x56,0x24,0xd7,0x8a,0x08,0x37,0x4d, -0x0a,0x71,0x0e,0x4e,0xd6,0xea,0xb7,0x8b,0x78,0xdc,0x31,0xb7, -0xbf,0xef,0x49,0x57,0xf5,0x18,0x6a,0xf8,0xbd,0x79,0x7c,0x7a, -0x68,0xa8,0xb1,0x60,0x54,0x4d,0x46,0x3f,0x95,0x68,0x2c,0xdf, -0x10,0x2a,0x35,0xdb,0xb6,0xac,0xa8,0x43,0x68,0x7a,0xe3,0xbd, -0xc8,0x53,0xa9,0x3e,0xeb,0x29,0xa9,0xa2,0x53,0xae,0x40,0xd7, -0x91,0x74,0xde,0xc9,0xab,0x99,0x8e,0x8b,0x80,0xcd,0xf1,0x8b, -0xd8,0xae,0x7d,0x6b,0x25,0x08,0xcb,0x93,0x35,0xe8,0xc8,0xd3, -0x53,0xf4,0x72,0x0a,0x2a,0xf0,0xf8,0xe2,0x10,0x1d,0xae,0xae, -0xeb,0x2a,0xad,0x2e,0xd3,0x70,0x6e,0x67,0x5d,0x63,0xcd,0x14, -0x3e,0xeb,0x05,0x7b,0x4e,0x54,0xba,0xcb,0xd4,0x87,0x15,0x1c, -0x07,0xe7,0x12,0x59,0x10,0x02,0xb2,0x12,0x0a,0x5a,0x50,0x5b, -0xfd,0x97,0xbf,0x21,0x4c,0xd8,0xa2,0x47,0x24,0x83,0x94,0xf2, -0xe1,0xb8,0xc1,0x19,0xa0,0xac,0xe9,0x79,0x37,0x31,0xd6,0xa4, -0xac,0xb4,0xa4,0x4b,0x4c,0x2b,0x50,0x2d,0xc3,0x9a,0x4a,0x88, -0xa1,0x3a,0x15,0x31,0x67,0x67,0x16,0x35,0x30,0x37,0xbc,0x27, -0x35,0x5a,0x12,0x0a,0x92,0x2b,0x57,0x49,0xf6,0x66,0x70,0x01, -0x5b,0xc6,0x4f,0xdd,0x8b,0x4a,0x95,0x24,0x6f,0x7d,0xed,0xe4, -0x3f,0xd0,0xfd,0x99,0xd9,0x14,0x9a,0xb4,0x1f,0x4b,0xd2,0xcc, -0x36,0xd0,0x82,0x0b,0xf3,0xd5,0xa8,0xb9,0xff,0x77,0xbb,0xcb, -0xcc,0xd1,0x1f,0x05,0xf9,0x1a,0x8b,0x7b,0x2a,0x95,0xb3,0xef, -0xee,0xec,0x79,0x4c,0xf9,0x5b,0x49,0x9d,0xee,0xdc,0x4c,0xe6, -0x29,0xcb,0x5d,0x0a,0x94,0xb6,0xe7,0xa8,0x65,0x19,0xb1,0x18, -0xc6,0x5a,0x6c,0x74,0x33,0x4a,0xb6,0xbe,0xcf,0x73,0x35,0x97, -0x68,0xc1,0x39,0x55,0x7f,0xa6,0x37,0xe1,0x85,0xca,0x3f,0xdb, -0xbd,0xf3,0xb6,0x41,0x2a,0x77,0xdf,0x79,0xa0,0x3d,0x76,0xe0, -0x76,0x91,0xf2,0xde,0xf9,0xec,0x3d,0x77,0xe2,0x0e,0x46,0xdd, -0x7b,0xe5,0xcd,0x7c,0x6f,0x22,0x4c,0xd3,0xab,0x8a,0xe6,0x64, -0x28,0xc7,0x0e,0x73,0xb5,0x4d,0x0a,0x07,0xad,0xfc,0x8c,0xd6, -0x9f,0x26,0xab,0xbb,0xf9,0x48,0x7f,0x48,0x58,0xd8,0x4f,0xdb, -0x2b,0x46,0x53,0x5c,0xda,0x26,0x60,0x85,0x3a,0x51,0xaf,0xf6, -0xd4,0x69,0x2c,0x45,0xf8,0x55,0xf8,0x35,0x58,0x6f,0xdc,0x8f, -0x64,0x22,0x3c,0x6a,0x73,0x44,0x95,0xbd,0x35,0x72,0xab,0xd3, -0x6c,0x4b,0xc3,0xb2,0x35,0x24,0x5b,0xea,0xed,0xb3,0x22,0xb4, -0x21,0x9f,0x59,0xe3,0xd9,0xa0,0xcf,0xad,0xd7,0xf2,0xb0,0xcf, -0xac,0xbf,0x5c,0x7a,0xf7,0x59,0x45,0x95,0x94,0x90,0xeb,0x21, -0xb5,0x9b,0xc5,0x7e,0x2f,0xb7,0x59,0x80,0x8c,0xbc,0x0d,0x8b, -0xb2,0x45,0xf4,0x6a,0x0a,0xcc,0xc4,0x03,0x27,0x06,0x7e,0x17, -0x8f,0xd2,0x87,0x0c,0x6d,0xee,0x67,0xf8,0x0d,0xa8,0xbd,0xda, -0xe6,0x2c,0xaf,0x6e,0xbd,0xa8,0x20,0x9e,0xae,0x0b,0x79,0x5f, -0x0c,0xe3,0xe1,0x5d,0xf2,0x4a,0x9e,0x63,0x58,0x2f,0x7c,0x96, -0x01,0x7b,0x5f,0x2b,0x2d,0x82,0x3e,0x46,0xa2,0xf8,0xcd,0x73, -0x2a,0xf6,0xfd,0xf2,0xd3,0x60,0xab,0x38,0xd6,0xe6,0xd0,0xba, -0x4c,0x32,0x8a,0x0f,0xe1,0xb8,0x92,0xa0,0x28,0x0b,0xea,0x48, -0xc4,0xaa,0x9b,0xb6,0xe8,0xc8,0x28,0xde,0x5d,0x33,0x25,0x50, -0x46,0xe1,0x14,0x0e,0x71,0x6e,0x4d,0x36,0x15,0xc2,0x62,0x4a, -0x74,0xd2,0x3c,0xd5,0x07,0x01,0xa4,0xad,0xf7,0xb0,0x2a,0xb2, -0x74,0x7c,0xd5,0x11,0xb8,0xcb,0xa7,0x5c,0x39,0xdf,0x2f,0x99, -0x2d,0xb6,0xd4,0xb4,0x1d,0x57,0x06,0xb9,0x53,0x9d,0xff,0xe3, -0x6e,0x05,0x2a,0x59,0xa2,0x14,0xa1,0x61,0x8b,0x41,0xd6,0xed, -0x15,0x68,0x38,0x7a,0xf6,0x20,0xf9,0xc7,0x12,0x98,0x7e,0xfe, -0x1c,0xf8,0xa7,0xb0,0x88,0x3b,0x2c,0x94,0x2b,0x39,0xfe,0x2e, -0xc2,0x14,0x71,0x07,0x45,0xdc,0x41,0xa1,0x5c,0xc9,0x81,0x7c, -0x11,0xa6,0xdf,0xd6,0x57,0x07,0x52,0x42,0xf2,0x96,0x4f,0xd5, -0x32,0xaa,0xd6,0x6e,0x51,0xc9,0xda,0x2d,0x6a,0x78,0x25,0xab, -0xc1,0x12,0x48,0x79,0xd7,0xf9,0x7e,0x5d,0xca,0x65,0xa1,0x43, -0x7b,0xbd,0x3d,0xd7,0x9c,0xb0,0x44,0xde,0x73,0xd5,0xd9,0x57, -0x8c,0x30,0x1a,0x60,0xbf,0x7d,0x9c,0x43,0x6d,0xe4,0x59,0xc5, -0xaf,0xfd,0xee,0xb5,0x6c,0x1a,0x7d,0x7f,0xa6,0xaa,0xfc,0xd6, -0x69,0xdf,0x35,0x19,0x6b,0xab,0x6f,0x15,0x84,0xd4,0x54,0x7e, -0xdb,0x20,0x52,0x63,0x98,0x35,0xcf,0x9e,0xee,0xfe,0x6d,0x35, -0x25,0xe3,0x0d,0xa8,0x44,0x24,0xcc,0x9a,0xc4,0x89,0x0b,0x0f, -0x37,0x37,0xc9,0xb2,0x9d,0x4b,0xfd,0x8c,0xe1,0x71,0xe3,0x69, -0x26,0x8b,0x17,0xef,0x39,0x59,0x58,0xcf,0xbe,0x69,0x9d,0x84, -0x57,0x3c,0xb7,0x54,0xe7,0x7e,0x4c,0xb7,0x0a,0x94,0xcc,0x63, -0xd7,0xdd,0xeb,0xb6,0xb7,0x63,0xfe,0x82,0x06,0xe1,0x3f,0xac, -0xdb,0x6f,0x5b,0x39,0x05,0x11,0xf0,0xf2,0xe7,0xc5,0x40,0xbb, -0xea,0x08,0xf2,0x71,0x64,0x8e,0x94,0xad,0x8e,0x57,0x2c,0x34, -0x73,0x49,0x01,0xe6,0xab,0x87,0x55,0x2c,0xdd,0x44,0xb9,0xa0, -0x32,0x43,0x9f,0xc7,0x27,0x3f,0x3f,0xa4,0x4b,0xc8,0x8a,0x67, -0xb7,0x0f,0x93,0x78,0x59,0x7d,0x20,0xef,0x10,0x4d,0x29,0xb9, -0xa2,0xb9,0x63,0xf9,0x5c,0xae,0x3a,0x2d,0x2d,0xcf,0x55,0x0d, -0xa0,0x63,0x5c,0x67,0x68,0xf0,0xc1,0xae,0xb5,0xe6,0x80,0x45, -0xdb,0x63,0xbc,0x1c,0xb9,0xa7,0xad,0x6a,0xde,0xb5,0x8c,0x8e, -0xe5,0x89,0xb0,0x7b,0xb0,0x6a,0x9b,0x37,0xba,0x07,0xab,0x56, -0x8e,0x2f,0x72,0x85,0x22,0xeb,0xb2,0x33,0x57,0xaa,0x22,0xcb, -0x38,0x3c,0x41,0x22,0x54,0xa3,0x55,0xeb,0x64,0xa3,0x00,0x35, -0x1a,0xed,0xd9,0x34,0xc9,0x0c,0x87,0x1c,0x99,0xa8,0xa2,0x16, -0xb6,0xba,0x12,0xa8,0x65,0x03,0xb5,0x41,0xd9,0x49,0xce,0xec, -0x3e,0x97,0xae,0xe4,0x54,0xc9,0x71,0xef,0x3f,0x93,0x61,0x8a, -0x7a,0xbd,0x42,0x57,0x1a,0xeb,0x3b,0xc9,0xb4,0xfd,0x04,0x68, -0x98,0x2e,0x87,0xf6,0x09,0x99,0x25,0x38,0x9c,0x25,0xaf,0xff, -0x64,0xb8,0x92,0x92,0xb1,0x2d,0x43,0xc9,0xbb,0x21,0xe4,0xdd, -0xd5,0xa7,0x8c,0x6a,0x2e,0x2a,0x7a,0xb7,0xc1,0xc8,0x73,0xdc, -0x53,0xa9,0x6e,0xdf,0x06,0xea,0x98,0x5d,0x90,0x43,0x64,0xe3, -0xab,0xc6,0x8e,0x3a,0x4e,0x4b,0x48,0x36,0xf3,0x10,0xa5,0x9b, -0x14,0xfa,0x5c,0x90,0x4c,0x56,0xd6,0x4a,0xee,0x31,0xdd,0x17, -0xeb,0x7c,0x6f,0xac,0x73,0x7d,0x80,0x9f,0xb6,0xc9,0xf8,0xba, -0x72,0x7b,0x5f,0xb3,0xbb,0xa5,0xdd,0x43,0xd4,0x7d,0x8e,0x6b, -0x4f,0x16,0x2b,0x76,0xd8,0x6b,0x15,0xaf,0x9e,0xaf,0xfb,0xcc, -0x89,0xe9,0x99,0xbe,0x1a,0x24,0x96,0x7f,0x58,0xfc,0x78,0x2e, -0x6b,0x22,0xc8,0x4b,0x5c,0x44,0xc4,0x7e,0xa8,0x2c,0x4a,0xf5, -0x7a,0x8f,0x90,0x58,0xd7,0xc9,0x1e,0xdb,0x13,0x45,0x75,0x36, -0x69,0x50,0xdd,0xa2,0xdb,0xe2,0x4a,0xb9,0x24,0x89,0x8a,0x68, -0x8a,0xde,0xf3,0xa4,0x78,0x2f,0xaa,0xa8,0x06,0x9b,0x20,0xdb, -0x27,0xc2,0xb5,0x5f,0x4a,0x95,0x86,0x70,0x79,0xe5,0x5d,0x5a, -0xf7,0x92,0x2a,0x7e,0xa2,0x21,0x95,0xcc,0x12,0xd9,0x79,0xd4, -0x2f,0xff,0xcf,0x2f,0x7f,0x8f,0xc8,0x54,0xaa,0x11,0xff,0x7f, -0xd4,0xfd,0x6b,0x77,0x1b,0x47,0x92,0x20,0x0c,0x7f,0x9f,0x5f, -0x21,0xe2,0x6d,0xf3,0xad,0x22,0x0a,0x60,0x15,0xa8,0x8b,0x05, -0xb8,0x88,0x63,0xcb,0xf6,0x58,0xbb,0xbe,0xe8,0xb1,0xd4,0x6e, -0xa9,0xd1,0x68,0x1e,0x10,0x28,0x8a,0x25,0x81,0x00,0x8c,0x02, -0x28,0x94,0x45,0x9c,0x33,0x7b,0x76,0xff,0x52,0x7f,0x7a,0xbe, -0xed,0x0f,0x98,0xfe,0x4b,0x4f,0xdc,0xf2,0x56,0x17,0x00,0x94, -0x3d,0xb3,0xb3,0xa7,0xdb,0x22,0x2a,0x33,0x32,0x32,0x32,0x32, -0x32,0x32,0x32,0x33,0x32,0x92,0x5f,0xfd,0xb4,0x7d,0xc1,0x43, -0x72,0x41,0x72,0xd3,0x22,0x72,0xb2,0xb2,0xd3,0xb2,0x74,0x46, -0x68,0xd8,0x13,0x24,0x9d,0x79,0xf8,0x85,0xfa,0x21,0x33,0xc9, -0xf0,0x21,0xc9,0x00,0x80,0x88,0x35,0x30,0x7c,0x10,0xac,0x4e, -0x44,0x50,0x4a,0x24,0xc8,0xc8,0x86,0x8c,0x04,0x32,0xb2,0x21, -0x4d,0x24,0x31,0xcb,0x5f,0xc5,0x78,0xa5,0x78,0x78,0xa1,0x08, -0xcb,0x9c,0x08,0x9d,0xfe,0xc9,0xaa,0xa9,0x12,0xc3,0x13,0xae, -0xa4,0xc5,0x54,0x9d,0x08,0xa4,0x90,0x0e,0x90,0x8a,0x5e,0x81, -0x6b,0x4a,0xa9,0x02,0x9c,0xe6,0xf5,0x5b,0x7c,0x2d,0x7e,0xbc, -0xae,0xd1,0x46,0x3a,0x57,0x4f,0xb4,0x32,0x0c,0x0b,0x97,0x8b, -0xdc,0xe1,0x88,0x1b,0x00,0x99,0x67,0x47,0x09,0x94,0x14,0xeb, -0x74,0x9b,0xdf,0x08,0x37,0x2f,0xe6,0x81,0x0d,0xff,0xf5,0x6b, -0xff,0xe4,0xeb,0xd7,0x01,0x58,0xa6,0xf0,0x8b,0x2e,0x27,0xbf, -0xf6,0x55,0xec,0xa9,0x8a,0x12,0x60,0xd1,0x7f,0xfd,0x06,0x4a, -0xbc,0x09,0xde,0x40,0x89,0x37,0x5c,0xe2,0x8d,0xbf,0xab,0xc8, -0x06,0x16,0x01,0x38,0x84,0x36,0xe8,0x52,0x01,0xbf,0xda,0x57, -0xe9,0x14,0x0f,0x60,0x2b,0x06,0xaa,0xde,0x5e,0xdc,0x7c,0x06, -0xd4,0x9c,0xff,0xfb,0x3f,0xd4,0xc3,0xe7,0x3b,0x2b,0xc8,0xa1, -0x82,0x1c,0x2a,0xc8,0xd1,0x61,0x03,0x7e,0x95,0x2a,0xc8,0xcb, -0x15,0xe4,0x9f,0x01,0xf1,0xa6,0x02,0x50,0x88,0xa4,0xc7,0x80, -0xc2,0x4d,0x88,0xcc,0x78,0x4d,0x8f,0xf9,0xe5,0x21,0x36,0xf3, -0x4d,0x18,0x80,0xca,0x0c,0x5e,0x07,0x6f,0xf0,0xd9,0x5f,0x8c, -0xf1,0x9d,0xc7,0xd0,0x9a,0xaf,0x5f,0xc7,0x4f,0x43,0xe0,0x41, -0x0c,0x13,0xb5,0xa5,0xe1,0x3a,0x66,0xfd,0xad,0x3b,0x92,0x76, -0x67,0xaa,0x3c,0x0e,0xa4,0x8b,0xdc,0x00,0x5e,0x15,0x93,0xb2, -0x08,0x40,0x5d,0xdf,0xdb,0xb3,0x25,0x46,0x0a,0x34,0x15,0xcf, -0xd7,0x2b,0xac,0xa3,0x5c,0xf5,0x4e,0xcd,0xfe,0x1a,0x24,0x43, -0x73,0xfc,0x8d,0xf7,0x26,0xf2,0xdb,0x19,0xc6,0xc9,0xf7,0x60, -0x64,0x41,0x5e,0xe4,0x5b,0xb7,0x78,0x75,0x06,0xc0,0x85,0x55, -0x19,0x3e,0xcc,0x11,0x16,0x45,0xc9,0xe1,0x0f,0x97,0x99,0x42, -0x37,0xa3,0x77,0xf3,0xa5,0x1c,0x4a,0x5c,0xf8,0x78,0x43,0xdb, -0xfa,0xec,0x5a,0x70,0x56,0x86,0xc3,0x07,0x0b,0xc1,0x61,0xba, -0xf8,0xb5,0xe5,0x38,0xf3,0xda,0x72,0x96,0x79,0x63,0x39,0xd1, -0xbc,0x31,0x8e,0x33,0x20,0x30,0xe7,0xaf,0xd1,0xd3,0xee,0x22, -0x06,0xd1,0x81,0xc2,0x28,0x43,0x11,0x2c,0x77,0xa0,0xc0,0xf9, -0x1b,0xce,0x00,0x39,0x82,0xd2,0x28,0x50,0x94,0x61,0x88,0x33, -0x27,0x7a,0x76,0x78,0x92,0xc1,0xe0,0x35,0x16,0x80,0x75,0xe4, -0x6b,0x2c,0x82,0xab,0xc4,0xca,0x86,0x1e,0xd6,0x1e,0xdb,0x11, -0xc8,0x71,0xfe,0xb1,0xda,0x63,0x39,0x02,0xc1,0x30,0x38,0xdf, -0x30,0xd9,0xf8,0xba,0x65,0x18,0xe3,0xc8,0x20,0xb2,0xf3,0xf0, -0x3c,0xe7,0x0c,0x7c,0xf0,0x32,0x8c,0x71,0x98,0x1c,0xd8,0x9e, -0xa2,0x9f,0x90,0x55,0x04,0xc3,0x70,0x7d,0x8a,0x4c,0xbc,0x84, -0x72,0x5a,0x22,0xe4,0xa3,0x28,0x0f,0x94,0x5c,0x96,0x86,0x97, -0x87,0x56,0xe9,0xc1,0x48,0xe7,0xa3,0x45,0x18,0xec,0xfc,0x30, -0xa0,0x46,0x05,0xcd,0x02,0xed,0xf9,0xf5,0x9b,0x72,0xe7,0x1c, -0x8e,0x7e,0xa2,0x4e,0x2e,0x27,0x79,0x05,0x7a,0xd0,0x34,0x93, -0xdc,0x41,0x7f,0x4f,0xab,0xc2,0x36,0x0f,0x02,0xed,0x08,0xaa, -0xd1,0xbd,0xf6,0xf8,0xe1,0x52,0xf4,0x25,0xcf,0x4b,0xb9,0x6f, -0x3c,0x7e,0xdc,0xd4,0x74,0x64,0xf0,0xba,0x02,0xc5,0x1b,0x52, -0x94,0x88,0xe2,0x4d,0x05,0x8a,0xd7,0xa4,0x52,0x2d,0x14,0x56, -0xfb,0x2c,0xd3,0xa4,0x7a,0xb8,0x0f,0x06,0x74,0x75,0xa9,0xf5, -0x34,0xe4,0x73,0x47,0xfc,0x78,0x1a,0xca,0x49,0xa3,0x3d,0xde, -0x15,0xe0,0xe7,0x94,0xc9,0x80,0x9f,0x53,0xa1,0xa1,0x3d,0xf3, -0x26,0xa3,0xd5,0x97,0xcb,0x71,0xcd,0xc4,0xcb,0x99,0x45,0x25, -0x59,0xa7,0x76,0x07,0xd9,0x7c,0xbd,0x1c,0x27,0x17,0x77,0x77, -0xfc,0xa3,0x7a,0x49,0x20,0xaf,0x5a,0x03,0x14,0xff,0xa8,0x86, -0x12,0x43,0x5a,0x30,0xaa,0x32,0x01,0x7f,0x23,0x4f,0xf9,0x97, -0x7a,0x22,0x1b,0x3d,0x26,0xe9,0x97,0x99,0x6b,0x98,0xf6,0x0a, -0x3b,0xae,0xe8,0xdf,0xa6,0x20,0xbc,0x3f,0x90,0x7c,0x1a,0x5e, -0x42,0x81,0xd0,0x7c,0x90,0x70,0x0a,0xac,0x6a,0xe8,0x45,0xf5, -0x96,0x0a,0x2e,0x70,0xba,0x17,0xba,0x06,0xbf,0xcb,0xd0,0x56, -0x9d,0xc2,0x96,0x83,0xea,0x14,0x58,0xcd,0xe4,0x03,0xeb,0x64, -0x68,0xab,0xce,0x8a,0x71,0xb8,0x4b,0x6f,0x71,0xa9,0x6e,0x68, -0x30,0x68,0xa9,0xb4,0x0d,0x7f,0x73,0x04,0x66,0x77,0x78,0x71, -0xcb,0xc5,0x5e,0x03,0x32,0x60,0x61,0x4d,0x29,0x89,0xae,0x09, -0xce,0xb8,0x0a,0x90,0x92,0xe8,0x2e,0xdb,0x35,0x69,0x12,0x68, -0x68,0xcf,0xe6,0x2c,0x43,0xed,0xbf,0x98,0xac,0xa2,0x1b,0x15, -0x8b,0x39,0xe7,0xfe,0x3a,0xd5,0x05,0xab,0xf7,0xa6,0x2e,0x9f, -0xcd,0x9b,0x52,0x95,0x07,0xb8,0x9f,0x74,0xa8,0x3c,0xfa,0x2d, -0xbd,0x59,0xaf,0xae,0x47,0x53,0xe3,0xa2,0x5a,0xcc,0xb1,0x6d, -0xb8,0xec,0xdf,0xff,0x5f,0x32,0xff,0x5d,0xdb,0x93,0xd6,0x1a, -0x9d,0x53,0x8f,0x96,0x08,0x0a,0x02,0xe3,0xdd,0xa8,0x82,0xff, -0xfc,0x1f,0xba,0x40,0x87,0x43,0x11,0x92,0x03,0xfd,0x3f,0xff, -0x07,0x9e,0xad,0xfb,0x3d,0xbd,0x2b,0x58,0x26,0xe6,0x1e,0xdb, -0x73,0xe5,0xd2,0x85,0x1d,0xba,0x32,0x80,0x73,0x7b,0xdc,0xca, -0x4d,0x59,0x9b,0xac,0x0e,0xe4,0x05,0x47,0x54,0xe6,0x76,0xe1, -0xba,0xcc,0xca,0xd3,0x3b,0x03,0xc7,0xc7,0xe3,0x53,0xbd,0x9a, -0x1b,0xf3,0x23,0x51,0x3a,0x66,0x7d,0x25,0x07,0x34,0x11,0x9f, -0xc8,0x03,0x55,0x7e,0x07,0x17,0x14,0x88,0xeb,0xad,0xfc,0x6c, -0x3e,0xbb,0xe2,0xfb,0xce,0xf7,0xf0,0x56,0xd6,0x85,0xaa,0xbc, -0x95,0x75,0x66,0xc9,0x2d,0x7a,0x7f,0x23,0x2b,0x1d,0xa3,0x6b, -0xda,0x56,0xcc,0xb6,0xfb,0xf7,0xed,0x6c,0x7e,0x83,0xd9,0xf7, -0x93,0xef,0xe8,0xd4,0x24,0x6e,0x03,0xbd,0xc8,0x36,0x3d,0xa6, -0xd1,0xde,0xa3,0x97,0x54,0x99,0x02,0xf5,0x2a,0x59,0x8d,0xcd, -0x9b,0x64,0x39,0xc6,0xbd,0xeb,0xd2,0x06,0x27,0xef,0x81,0x51, -0x25,0x83,0x4d,0xa0,0x46,0x14,0x90,0xc5,0xeb,0xc7,0x64,0x43, -0x6b,0xbf,0x16,0xb4,0xed,0xea,0x9f,0xff,0x36,0x34,0xbe,0xe8, -0x0a,0x61,0x3d,0xad,0x0a,0xe2,0x45,0x89,0x66,0x95,0x53,0xa0, -0x59,0x25,0xdb,0x9c,0x9e,0x2f,0x57,0xd7,0x73,0xb0,0x7b,0x16, -0xd7,0x3b,0xb9,0x6d,0x78,0xac,0x18,0x0b,0x43,0xc3,0x30,0xd6, -0xc1,0x72,0x0f,0xe6,0xda,0xe5,0x0a,0xc4,0xda,0x59,0x6e,0xa4, -0x1e,0xd0,0xc0,0x07,0x50,0x5c,0x29,0x1f,0xae,0xd6,0xdb,0xad, -0xf4,0xb0,0x8b,0x20,0xd9,0x52,0x79,0x6e,0xd5,0xf7,0x68,0xa6, -0x53,0xb0,0xd0,0x4e,0x27,0xcf,0xf1,0x00,0xa1,0x65,0xec,0x0f, -0x87,0xc8,0x95,0xb5,0x9f,0x84,0x76,0x5a,0x3a,0xbb,0xf6,0x36, -0x72,0xd5,0x18,0x55,0x5c,0xce,0xc7,0xc0,0xa4,0xc6,0xb5,0x5a, -0xcb,0x7d,0x3c,0x19,0x85,0x6c,0x84,0xf5,0x2d,0xc1,0x2b,0xd7, -0xfd,0x29,0x22,0x58,0xc6,0x52,0x68,0x77,0x19,0x40,0xe6,0xed, -0x1b,0x8e,0xb5,0xd5,0xce,0x6e,0xdf,0x9a,0x5f,0xed,0xea,0x3d, -0x04,0xa4,0x03,0x72,0x2f,0x30,0xd7,0x79,0x58,0x44,0x4f,0xe7, -0x2a,0xd7,0x6c,0x60,0xce,0x97,0x26,0x0a,0x1e,0xe6,0x8c,0x28, -0x48,0xb4,0x82,0xfb,0x9e,0x52,0x82,0x06,0xe7,0xb4,0xc6,0xd3, -0x79,0x96,0x4c,0x1a,0x65,0x88,0x67,0x94,0x81,0xef,0xc1,0x2c, -0xec,0x4c,0x5c,0x73,0x05,0x0d,0x4c,0x6d,0x71,0xb0,0xc2,0x46, -0x31,0xf7,0x2b,0x4a,0x16,0x18,0x8a,0x64,0x58,0x02,0xf9,0x92, -0xe2,0x1b,0x5e,0x42,0x7f,0x65,0x76,0xd6,0x57,0x98,0x10,0x34, -0x28,0xbd,0x35,0x5f,0x24,0xb3,0x46,0x29,0xf7,0x27,0x48,0x55, -0x10,0x15,0xa4,0x13,0x8c,0x50,0x7e,0xb9,0x9e,0x4d,0xa6,0x89, -0x93,0x4b,0x29,0xc1,0x78,0x44,0xcb,0x8b,0xa9,0x9d,0xf5,0x4c, -0xd2,0x82,0x86,0xca,0x2d,0x13,0xa0,0x60,0x98,0x06,0x0d,0x57, -0x41,0x86,0x82,0x14,0x4a,0x40,0x8f,0xce,0x57,0xf3,0x99,0x43, -0xcb,0x0f,0x92,0xc6,0x8f,0xd5,0x54,0x76,0xe2,0xce,0x87,0x6b, -0x38,0x7c,0x1c,0x24,0xc4,0x3a,0xb1,0xcd,0x94,0xc4,0xa7,0x42, -0xd2,0x9f,0x4e,0xf9,0x7d,0x11,0x00,0xc0,0x41,0x5e,0x90,0x17, -0x62,0xd5,0x57,0xc9,0x6f,0x69,0xb2,0x8c,0xf0,0x0c,0xa1,0x73, -0x7a,0x16,0x44,0xf0,0x1f,0xbf,0x65,0x5e,0x01,0xd5,0xe1,0xb7, -0xce,0xcf,0x08,0xb2,0x16,0xea,0x8c,0xa1,0x1e,0x0b,0xbe,0xc7, -0x43,0xf1,0x50,0xbb,0x69,0x5f,0xe3,0x31,0xb5,0x6e,0x0a,0x8e, -0xf5,0x74,0x8c,0xdb,0x6f,0x7a,0x6d,0x88,0x00,0xde,0x04,0x46, -0x39,0xb9,0x2b,0xe3,0x0f,0xfd,0xe0,0x8d,0xb8,0x2a,0x0f,0x86, -0xd4,0x0a,0x55,0x36,0x98,0x04,0x69,0xf0,0x4e,0xae,0x91,0x6d, -0x3a,0x41,0xde,0x09,0xd6,0xc1,0x6d,0x30,0x0a,0xb2,0x45,0x70, -0xb5,0xb1,0x1e,0xe9,0x41,0x65,0x71,0x95,0xdb,0x09,0xb9,0x1f, -0xcc,0x62,0xab,0x8a,0x60,0x01,0x7f,0xe3,0x59,0x2b,0x62,0x93, -0x97,0x2e,0x68,0xc3,0xa4,0x3d,0x7e,0x8f,0x3f,0xae,0x63,0x0a, -0x28,0x02,0x28,0xe3,0xd8,0x6a,0xf5,0xeb,0xe3,0xe3,0xdc,0x4d, -0x79,0xe3,0x2b,0xd2,0x08,0x37,0xbf,0x39,0x81,0x4e,0x73,0x78, -0x39,0x59,0x67,0x0d,0xcc,0x25,0x6c,0x95,0xc6,0xc7,0x47,0x83, -0xe6,0xd5,0xa6,0x8d,0xe1,0xe1,0x79,0xe9,0x37,0x21,0x24,0x78, -0x65,0x39,0xf5,0x83,0xe6,0x55,0x6e,0x67,0x41,0xd2,0x90,0xfd, -0xf1,0xd2,0x38,0xd2,0xe8,0x34,0x57,0xa1,0x10,0x46,0x1e,0xd0, -0x9f,0xd7,0x43,0x8a,0x4b,0xed,0x66,0xc7,0xb1,0x9b,0x7f,0x7c, -0x6c,0xe7,0x87,0x6e,0xf1,0x90,0xee,0x6a,0x5f,0xc7,0xa9,0xaa, -0x35,0xd4,0xb5,0xa6,0x47,0x71,0x7c,0x0d,0x99,0x79,0x14,0xbb, -0x15,0xb4,0x5c,0xfc,0xb8,0xbb,0xe5,0xd6,0xd0,0x72,0x6b,0x10, -0xde,0x33,0x33,0x3e,0xd2,0xf9,0x49,0xd7,0xbe,0xf9,0x8a,0xfb, -0x63,0x18,0x5c,0x66,0x92,0x6c,0xba,0xe9,0xd6,0x67,0x4a,0xa4, -0x0c,0xdd,0x87,0x77,0xcf,0x55,0xd4,0x3a,0x91,0x4f,0x62,0x5a, -0x97,0xfc,0x77,0xeb,0x07,0x23,0x0e,0x31,0x80,0x87,0xa5,0x9c, -0x16,0xdc,0x5a,0x29,0x84,0x3f,0x58,0x43,0x8f,0x21,0x6b,0x51, -0x2e,0xa4,0x9d,0x28,0x95,0xef,0x14,0x60,0xaa,0x00,0x47,0xb1, -0x95,0x44,0xd8,0x08,0xd0,0x6e,0xeb,0x6d,0x55,0x5b,0x73,0x17, -0xa0,0x82,0x5b,0x1d,0x03,0xf0,0xae,0x12,0x83,0x0b,0x50,0xc5, -0xef,0x93,0x4d,0xd4,0xcc,0xa3,0x93,0x3c,0x3a,0x8f,0x37,0x9d, -0x93,0x4d,0xa7,0x99,0x77,0x4e,0xf2,0x8e,0xdc,0xad,0x32,0x8d, -0x88,0x5b,0x51,0x4f,0xbd,0x82,0xb2,0x95,0xbc,0xb5,0xc9,0xdb, -0x8e,0x8a,0x4d,0x04,0xf6,0xa4,0xc0,0xb4,0x77,0xf4,0xf8,0x09, -0x8d,0x13,0xee,0xb4,0x6b,0x9f,0xee,0xe1,0xbf,0x03,0xe9,0xe8, -0x10,0xd7,0xde,0xf9,0x52,0xf2,0x9d,0xa0,0x3b,0x6f,0x45,0x74, -0xf3,0x51,0x17,0x29,0xe4,0x03,0x82,0x66,0x93,0x7b,0x36,0x5b, -0xc4,0x0c,0x27,0xf1,0x0f,0xa8,0x2b,0xde,0x11,0x52,0xbc,0xce, -0x50,0x2c,0xf9,0x45,0x28,0x0f,0xe8,0xf4,0x8e,0x78,0x06,0xbe, -0xb9,0x40,0x8d,0xf2,0xec,0xd9,0x5f,0xb8,0xba,0x41,0xb6,0x68, -0x75,0x64,0x54,0xe3,0xef,0x48,0xc9,0x9b,0xc6,0xa0,0x87,0xa8, -0xdf,0xf3,0x5b,0xad,0x6c,0xd1,0x53,0xb0,0xcd,0xe6,0x30,0x2e, -0xc0,0xca,0x6d,0xbd,0x69,0x8e,0xc3,0x99,0x47,0x04,0xe2,0xec, -0xa5,0xe7,0xd0,0xf6,0x56,0x2b,0xf5,0x31,0x8f,0x5b,0x48,0xa3, -0x98,0x51,0xa5,0xc3,0xa1,0x6f,0xee,0x10,0x4e,0xf3,0xad,0xbe, -0xe5,0xae,0x95,0x0a,0xef,0x10,0x6a,0x8d,0xd2,0xab,0xdb,0xed, -0x20,0x6d,0xa9,0x09,0xee,0x7a,0xf8,0xdd,0xde,0x1c,0xb8,0x53, -0x1d,0x5f,0x04,0x08,0xef,0x77,0x37,0x5b,0xfa,0xd1,0xce,0x0f, -0x2b,0x98,0xeb,0x82,0x39,0x17,0xd4,0xbb,0x18,0xda,0x7b,0x75, -0xb7,0xe7,0x03,0x36,0x6d,0x7d,0x39,0x9e,0x8e,0xb2,0xcc,0xce, -0x0f,0x54,0x8f,0x09,0x12,0xfd,0x5a,0xb1,0xe3,0xdc,0x60,0x6f, -0x61,0x94,0x41,0xe3,0x02,0x21,0x96,0x6f,0xf1,0x60,0xd8,0xab, -0x2b,0x55,0x70,0xde,0xb4,0x1e,0xe0,0x51,0x31,0xe8,0xf8,0x95, -0x6f,0x9e,0x25,0xf8,0xcd,0xea,0xc1,0x0c,0x85,0x87,0x0a,0x86, -0x2a,0x20,0x1d,0x46,0x9a,0x90,0xdc,0x94,0xf3,0x9a,0x7c,0x64, -0xab,0x4f,0x74,0x4f,0x2e,0xcd,0x95,0x99,0xf6,0xa3,0x13,0x04, -0xd9,0xd6,0xb6,0xbb,0xc2,0xa7,0xf3,0xbd,0x44,0x54,0x0b,0xc6, -0x55,0x94,0x6d,0x62,0x0c,0x37,0x10,0xda,0x14,0x92,0x58,0x16, -0xfb,0xf0,0xee,0xce,0x7b,0x8f,0x21,0x34,0xbc,0xc7,0xfc,0xd4, -0xf4,0x88,0x6e,0x70,0x82,0x2e,0xad,0x6c,0xc7,0x38,0xd6,0x94, -0x53,0xc4,0x9c,0x13,0x8a,0x98,0xb1,0x69,0xc6,0x14,0xb5,0xa3, -0x49,0xa1,0x33,0x4e,0xc6,0x41,0x4e,0x09,0x11,0x26,0x44,0x98, -0xd0,0x53,0xcb,0x41,0xbc,0xf6,0x72,0xf2,0x7e,0xb8,0xab,0x9d, -0x78,0xa9,0xda,0x6c,0xc8,0xad,0x65,0xef,0x4b,0x87,0xd9,0x9d, -0x2d,0xd6,0x2b,0x98,0xe3,0x6f,0x82,0x4b,0x68,0xf7,0x24,0x10, -0x13,0xa7,0x80,0x8e,0xed,0x2c,0x5d,0xba,0x82,0x3f,0xad,0xea, -0x12,0x1c,0x80,0x76,0x64,0xf1,0x4c,0xd8,0x40,0x04,0x50,0xe5, -0xb1,0x60,0x95,0xc3,0x36,0x0c,0xba,0xce,0xdf,0xb2,0x65,0x16, -0x3a,0xec,0xa2,0x22,0x03,0xef,0x86,0x7f,0xa8,0xca,0x99,0x6a, -0x1f,0xa5,0xe6,0x1d,0xea,0x5a,0x50,0x61,0x5f,0xdc,0xf4,0xfc, -0x89,0x80,0xbf,0x1b,0x16,0xf9,0xf3,0x7c,0x96,0x81,0xa5,0xef, -0x4d,0x02,0x9c,0xc8,0xfa,0x5e,0x75,0xee,0x98,0x72,0xef,0xee, -0x14,0x41,0xac,0x6c,0x8a,0xa0,0x2b,0x5c,0x81,0x8c,0x57,0x1e, -0x72,0x0f,0xe1,0x4d,0x03,0x18,0xde,0x57,0xe1,0x64,0xab,0xd1, -0x1f,0x1f,0xdf,0x17,0x3d,0xac,0x59,0x7b,0xff,0xc2,0x2d,0x2e, -0x14,0x96,0x0f,0x14,0x1a,0xe0,0xf9,0xa5,0x72,0xd3,0x92,0x64, -0xa3,0x49,0xf0,0xdd,0x9c,0xf5,0x6c,0x94,0x57,0x59,0x8b,0x28, -0x14,0xc9,0x04,0xe3,0xaf,0x68,0xdb,0xc9,0x39,0xd9,0xd5,0x4b, -0xc6,0x21,0x4c,0xf4,0xab,0x65,0x4a,0xd3,0x95,0xfd,0x0c,0x95, -0x6a,0xc1,0xed,0x7c,0x09,0x26,0x78,0xfa,0x2a,0xc9,0xb2,0x64, -0x4a,0x9b,0xbc,0xda,0xac,0xd4,0xc8,0x60,0x2a,0xa7,0xba,0x06, -0x49,0x7b,0x99,0xbc,0x45,0x37,0xa8,0x29,0x6b,0xff,0xa1,0xc4, -0x70,0x56,0xf3,0xad,0xce,0x5f,0x4a,0x3e,0x3e,0x44,0x45,0x45, -0xcb,0x46,0x3c,0x26,0xe3,0x23,0x74,0x64,0xc9,0xda,0x06,0x51, -0x30,0xde,0xc4,0xb7,0x38,0xb1,0x8f,0x73,0xf8,0x0b,0xb2,0x88, -0x90,0xe5,0xf2,0xb7,0x50,0x54,0x1c,0x71,0x2c,0xe3,0x08,0x0b, -0xb6,0xc6,0x30,0x85,0xe1,0x50,0x1d,0xe7,0xaa,0xfe,0xfb,0xd8, -0x45,0xfa,0x7a,0x07,0xc6,0xb1,0xbe,0x89,0xa9,0xb8,0x0a,0xaf, -0xd3,0xbb,0x81,0x99,0xf7,0x1d,0xcc,0xcc,0x9a,0xa5,0x62,0xb2, -0xde,0x52,0x35,0x28,0xc2,0xfc,0xb7,0x19,0x0d,0xa9,0xf1,0x1a, -0xce,0x74,0xab,0x70,0xdc,0xf4,0x2a,0x4f,0xa9,0xd6,0x0a,0x40, -0x20,0x64,0x11,0xa0,0xef,0xc3,0xb3,0xa1,0x2f,0x22,0xc7,0xa6, -0xf5,0x8e,0x4e,0xbf,0xaf,0xe5,0xff,0xd7,0x38,0x4a,0x1e,0xd7, -0xd8,0xf7,0x57,0x25,0x03,0x5f,0x96,0x07,0xae,0x79,0x2f,0x89, -0xe6,0xed,0x80,0x19,0xd9,0x41,0xda,0x42,0xd6,0xd6,0x53,0x7c, -0x4f,0x13,0x5f,0x82,0x9b,0xd7,0x08,0xac,0x30,0xc7,0x16,0x57, -0x3a,0xe0,0x8a,0x82,0xac,0x83,0xab,0x22,0x5c,0x12,0xc1,0xda, -0xa8,0xd3,0x8b,0xa0,0x15,0x30,0xd3,0x1d,0x1f,0x27,0xed,0x4b, -0xb0,0x4f,0xfa,0x5e,0x16,0xc1,0x77,0xb2,0x68,0x2f,0x01,0x92, -0x7f,0xc1,0x94,0xae,0x53,0xa7,0x3a,0x75,0xe9,0x07,0x52,0xb6, -0x8f,0xe6,0x7d,0x16,0xf5,0xb3,0xa8,0x9d,0x77,0x5b,0x7f,0x45, -0x5b,0x1e,0x94,0x77,0x0b,0x10,0x9e,0x50,0x1d,0x71,0xd6,0xe9, -0x67,0x1d,0xc8,0xfb,0x2b,0x1a,0xae,0x3a,0xab,0x03,0x68,0x37, -0xaa,0xe0,0x06,0x0b,0xe6,0xaa,0xe0,0xe8,0x84,0x48,0x94,0x82, -0x1b,0x28,0x98,0x77,0x4c,0x56,0x87,0x17,0xac,0xb7,0xb0,0x36, -0xe5,0xb3,0xec,0xe0,0x16,0x16,0xa0,0xb4,0xc8,0x1b,0xf6,0x94, -0x28,0xd4,0x8e,0xcb,0x08,0xa0,0xfd,0xa0,0x0c,0xb6,0xac,0x00, -0xdb,0x1a,0xc0,0x58,0xfd,0x70,0xc5,0x4b,0x52,0xd5,0xa0,0x85, -0x61,0xaa,0x3b,0x54,0x86,0xab,0xf9,0x46,0xc3,0x80,0xc6,0xa6, -0xb2,0x42,0x1c,0x4c,0xb7,0x05,0x37,0x99,0xba,0xa1,0x0b,0x26, -0x4f,0x42,0x8f,0xb8,0xb2,0x23,0x8e,0xc2,0x25,0x67,0x3b,0xbb, -0x06,0x35,0xd6,0x3d,0x18,0x81,0xc5,0x41,0x3f,0x2e,0x61,0x40, -0x28,0xc5,0x47,0x38,0x4b,0x8e,0x3b,0x13,0xf3,0x1e,0xe6,0x51, -0x7a,0x77,0xc7,0xc5,0x26,0xaa,0x3c,0x95,0x19,0xa4,0x30,0x71, -0x0d,0x8d,0x27,0x8f,0x29,0xab,0xab,0x15,0x02,0xa1,0xe0,0xd6, -0xe6,0x67,0x59,0x79,0x15,0x58,0x39,0x8b,0xdd,0xa6,0xa1,0xbc, -0x1f,0xcd,0x7c,0x17,0xab,0x68,0x1a,0x90,0x9d,0xd6,0x5f,0x87, -0x01,0xfe,0xc5,0x3f,0xea,0x5f,0x48,0xa3,0xe0,0xa8,0x47,0xde, -0xec,0xbc,0x23,0xc1,0x35,0x31,0x1c,0xa4,0xea,0x91,0x60,0x11, -0xa9,0x4a,0x68,0x89,0xd9,0xd1,0x5f,0xe4,0x82,0x11,0x2f,0x42, -0xf1,0xd1,0x80,0x1f,0xbc,0x46,0x5d,0x44,0xb2,0x3e,0x83,0x1f, -0xbc,0x0e,0x5b,0x74,0x64,0xbd,0x05,0x3f,0x20,0x65,0xb2,0x31, -0xb7,0x34,0x37,0x9d,0x16,0x2e,0x4a,0x27,0x79,0x9c,0x77,0x5a, -0x39,0x3d,0x58,0x69,0x6e,0x70,0xe6,0xfe,0x17,0xff,0xfe,0x0f, -0x26,0x09,0xf2,0xa3,0xf3,0x3c,0xec,0xb7,0xfe,0xda,0xfd,0x6b, -0xaf,0xd4,0xb0,0x9c,0x9a,0x92,0x0f,0xad,0xe8,0xa2,0xff,0x38, -0x9f,0x6c,0x54,0x54,0x92,0x4d,0x74,0xbe,0xa9,0x2c,0xba,0x61, -0x96,0x6c,0x80,0x17,0x5c,0x54,0xea,0x82,0x31,0xd7,0xda,0x84, -0xfe,0x89,0x47,0x44,0xf9,0xe7,0x42,0x26,0x7c,0x47,0xad,0x3c, -0xf4,0xfb,0x7f,0xc5,0x91,0xf8,0x5b,0x6c,0x53,0xda,0x9a,0x6c, -0x7a,0xfa,0xfb,0x37,0x24,0xbc,0xef,0xd6,0x15,0x9e,0x4f,0xf2, -0x3e,0x8c,0xfd,0x1c,0xe9,0xec,0x7a,0xbf,0xe1,0x83,0x06,0x5e, -0x2e,0xaf,0x8c,0xd5,0x36,0x08,0x2f,0xf6,0x73,0x30,0x1b,0xf1, -0x83,0xf2,0x8b,0xdb,0x07,0x56,0x1e,0x59,0x83,0x9e,0x1e,0xb6, -0xe9,0xb0,0xbc,0xd9,0x60,0x65,0xaa,0xeb,0x98,0xac,0x41,0x7b, -0xae,0xd0,0x64,0xfb,0x97,0x56,0x36,0x55,0x74,0x89,0xa2,0x76, -0xad,0xa5,0xe6,0x25,0x99,0xb0,0xba,0x9e,0x24,0xdc,0x67,0xbd, -0x25,0x45,0x68,0xc9,0xa5,0x8a,0xdf,0x63,0xd5,0xa5,0x8b,0xe7, -0xa6,0x78,0x55,0x7c,0xa5,0x3d,0x77,0xfd,0xad,0x26,0x1f,0x1f, -0x0f,0xac,0x2f,0x52,0x64,0xd6,0x67,0x67,0x48,0x93,0x0f,0xdf, -0x2d,0xb9,0xf0,0x4b,0xac,0xd2,0xc2,0xc6,0x9e,0x4e,0xa1,0x15, -0xe6,0x28,0x54,0xdb,0x17,0xc6,0x03,0xaa,0xa3,0xbd,0x9e,0x7a, -0x36,0xa6,0xc2,0x8a,0x0d,0x2f,0xc3,0xb3,0xae,0xa7,0xbf,0x1d, -0xfc,0xdb,0x31,0x7f,0xc9,0xb2,0x50,0x6f,0xde,0x31,0x0b,0x0c, -0x2f,0xb2,0xf4,0xb7,0xc3,0xfc,0x21,0xca,0xcc,0xf3,0x2e,0x80, -0x15,0xec,0x45,0x7d,0x01,0x92,0xed,0xb0,0xc8,0x65,0x89,0xa9, -0x0e,0x04,0xe8,0xbd,0xe5,0x6d,0x58,0x69,0xb0,0xd0,0x91,0xc7, -0x3e,0x6b,0x85,0x11,0x0d,0x86,0xf7,0x35,0x5b,0xfe,0xab,0x1a, -0x2c,0x7b,0xad,0xec,0x1a,0xa3,0x65,0x1a,0x17,0xe7,0xf2,0x60, -0x19,0x17,0xe7,0xed,0x1e,0xb1,0x74,0x30,0x1d,0x0e,0x96,0x43, -0x58,0xc8,0x5a,0x5f,0x31,0xff,0x5e,0x0e,0xe1,0x13,0xe3,0x7b, -0x11,0x5b,0x65,0x23,0x91,0xfd,0x34,0xba,0x44,0xf8,0x74,0x28, -0xce,0x18,0xfc,0xb9,0x84,0x2e,0xf0,0x55,0x2f,0x98,0xde,0x35, -0xeb,0x86,0x42,0x0f,0xe3,0xc6,0xde,0xfe,0x4d,0x60,0xc7,0x17, -0xc8,0x2c,0x67,0x18,0x87,0x09,0x69,0x19,0x54,0xf5,0xc2,0x7e, -0x31,0x90,0xc5,0xad,0x2d,0x0b,0xb2,0x68,0xf5,0x3e,0xb9,0xdb, -0xfc,0x36,0xe6,0xc3,0xc2,0xad,0x8e,0x74,0xd1,0x7c,0xae,0x20, -0x2b,0x36,0x99,0x87,0x55,0x24,0xa1,0x68,0x3e,0x61,0xd8,0xab, -0x8f,0x76,0xe8,0x2c,0xaa,0x0e,0x4d,0x05,0xcd,0x73,0xdf,0xdd, -0xdc,0x91,0xd4,0x9f,0x16,0x8b,0x79,0x96,0xae,0x92,0xf8,0xe3, -0xb4,0xdb,0x58,0x36,0x82,0x65,0xb7,0x31,0x6d,0x6c,0xf5,0x91, -0xc2,0xaf,0xeb,0xd1,0x64,0xb5,0x4c,0x92,0xe2,0x92,0xc2,0x3e, -0x12,0xb0,0x96,0x17,0x0a,0x5c,0x3a,0x53,0xa7,0xa7,0xb3,0x0c, -0xef,0xde,0xcc,0x80,0x17,0xe8,0x85,0xec,0x14,0x46,0x65,0x9a, -0x66,0x3f,0x8e,0x7e,0x84,0xbe,0x38,0x3e,0x96,0x9f,0xb9,0x8f, -0x3b,0x90,0x68,0xb4,0x8c,0xe4,0x75,0xf4,0xd9,0x26,0x9e,0xb5, -0x37,0xc1,0x2c,0x87,0x3f,0xb9,0x52,0x9c,0x47,0xf1,0x6c,0xe3, -0xdb,0x06,0xc1,0x6c,0xd3,0xda,0xf8,0x4d,0xf3,0x99,0xb7,0xc0, -0x42,0x68,0x87,0x91,0xcf,0x14,0x3c,0xbb,0x4e,0xa7,0x93,0x4a, -0x32,0x8c,0xba,0x9d,0x71,0x9c,0x91,0x19,0x4f,0x7e,0x3d,0xa8, -0x14,0x6b,0x54,0xdf,0x7c,0xdf,0xd9,0xc5,0xc6,0x25,0x82,0x19, -0x52,0xe7,0x20,0x0d,0xf6,0xd5,0xca,0xc6,0x08,0x56,0x01,0x65, -0xa1,0x92,0x3c,0x50,0xd5,0x4c,0xc4,0x4e,0xd9,0x87,0xa0,0xc0, -0xe2,0x7a,0x48,0x5e,0xb8,0x6c,0xe2,0xf6,0xa3,0x13,0xb0,0x5d, -0x9a,0x60,0xfe,0x07,0x59,0x4e,0x5f,0x79,0xd4,0x44,0x5a,0x97, -0xe9,0xdb,0xeb,0x55,0xbc,0x39,0x8f,0xb3,0x4d,0x70,0x39,0x5f, -0xad,0xe6,0x37,0x71,0x0e,0x1f,0x39,0x0c,0x07,0x8f,0xbf,0xbf, -0xf8,0x22,0xf2,0x9b,0x04,0xd6,0xe3,0x9e,0xc1,0x90,0x7a,0x30, -0x5b,0xb5,0x67,0xf3,0x09,0xad,0xaf,0x41,0x63,0x98,0x0f,0xbd, -0x85,0xa4,0x84,0xe2,0x47,0x7c,0xe7,0xcb,0x97,0x8a,0xfa,0xb8, -0x66,0xd9,0x74,0x71,0x75,0xa2,0xaa,0xeb,0xe3,0xfa,0x27,0xef, -0xe2,0x4a,0x27,0x0f,0x76,0x48,0xcc,0x96,0xc7,0xf6,0x26,0x0b, -0xf2,0x0c,0x26,0x81,0x19,0x64,0x5e,0x40,0xee,0x05,0x64,0xc3, -0x5f,0xf8,0x6f,0xef,0x08,0x37,0xe2,0x03,0x26,0x1b,0x94,0x8e, -0x09,0xfd,0x45,0x4c,0x55,0xc0,0x17,0x56,0x03,0x5f,0x1d,0xfd, -0xa4,0xf9,0x86,0x3e,0x2f,0xe2,0x96,0xb7,0x21,0xb0,0x0b,0xbe, -0x7e,0xbb,0xa1,0x79,0x25,0xa7,0x7f,0xdd,0x25,0x2f,0xc5,0x0d, -0x5e,0x95,0x2c,0x31,0xa3,0x25,0x26,0xed,0xcd,0x17,0x80,0x0b, -0xc3,0x9a,0x02,0x36,0xf8,0x02,0x0b,0xb7,0x9d,0x7f,0x91,0x53, -0x52,0x4e,0x49,0x39,0x26,0x6d,0xce,0xa1,0x6e,0x84,0xea,0x18, -0xa8,0xf3,0x9c,0x92,0xf2,0x8e,0x40,0x6d,0x44,0x0f,0x53,0x7e, -0xae,0x3f,0x72,0xdf,0x39,0x1e,0xd3,0x44,0xb0,0x51,0x71,0x11, -0x83,0x02,0xf3,0x1c,0xb5,0x95,0x63,0x5a,0x4e,0x0b,0x96,0x1e, -0x50,0x75,0xbe,0x51,0xe4,0x6d,0xd0,0x65,0x1a,0x12,0x72,0x45, -0x5c,0x0e,0x09,0x9b,0x0b,0x8b,0x34,0x82,0xb8,0xb0,0x08,0x23, -0x08,0x21,0x85,0x32,0xe5,0x37,0xa4,0x73,0xff,0x81,0xc8,0x77, -0x2e,0x5a,0xd8,0x75,0x64,0xd9,0x5f,0x80,0x15,0x7d,0xd1,0x9b, -0x6c,0xc8,0x04,0xee,0x10,0x8f,0x9b,0x13,0x94,0x0f,0xec,0x1c, -0xf8,0x99,0x93,0xf2,0x5a,0xce,0xe7,0xab,0x1a,0xb9,0xc2,0x3e, -0xc5,0x6c,0x7c,0xab,0x3c,0xb6,0x17,0x50,0x22,0x49,0x98,0x07, -0xc2,0x44,0x8d,0xa6,0x88,0xc5,0x81,0x6a,0x6b,0x51,0x7e,0xf0, -0x51,0x36,0x44,0x74,0x9b,0x82,0x62,0x34,0xa8,0xae,0x54,0xc4, -0x12,0x53,0xf3,0x2f,0x08,0xe1,0x5d,0x11,0x78,0x05,0x16,0x9e, -0x47,0xc8,0xbe,0x03,0x39,0xe3,0x93,0x13,0x99,0x48,0x1c,0xa2, -0xa4,0x07,0x36,0xb4,0x98,0xca,0xe9,0xdf,0x22,0x32,0x3c,0xf0, -0x60,0x8d,0x40,0x52,0xa6,0x56,0x7d,0xaa,0xc1,0x7a,0x09,0x0a, -0x12,0x99,0xb3,0x29,0x12,0x4f,0x58,0x55,0x21,0xc8,0x96,0x5f, -0xac,0x41,0x91,0x0c,0x0e,0x39,0x0b,0xe1,0xb0,0xd7,0xab,0xd2, -0x1b,0xdd,0x7e,0x5f,0x07,0xe1,0x35,0x4c,0x78,0x46,0xb0,0xaf, -0xb5,0xdf,0x75,0x31,0xe7,0x4d,0x70,0x06,0xd3,0x37,0x63,0x24, -0xe1,0xe0,0x61,0xc6,0x63,0x0e,0xfe,0xc5,0xc7,0xbe,0xf5,0xbc, -0x21,0xb3,0x20,0x2c,0x00,0x54,0xd2,0x7d,0x56,0x00,0xaa,0x0c, -0x2d,0x01,0x34,0x82,0x7b,0xac,0x01,0x0c,0x82,0xdc,0x42,0x70, -0x8f,0xdb,0x1e,0xca,0x9c,0x47,0xf5,0x86,0xad,0x83,0xc6,0x76, -0xa8,0x1f,0x68,0x93,0xe6,0x3e,0x66,0xbc,0x6f,0x11,0xa3,0x64, -0x88,0xfd,0x79,0x2d,0x1b,0x9e,0x6c,0xf7,0xa1,0x45,0xea,0xc1, -0x46,0xfa,0x6e,0x42,0x21,0x25,0x14,0xc2,0x2c,0xaa,0x76,0x90, -0x44,0xab,0xdf,0x80,0xd6,0xc2,0x16,0x39,0x7c,0x76,0x65,0xb9, -0xf9,0xfe,0xfc,0xf6,0x92,0x5e,0x16,0x74,0x52,0x0a,0x30,0xfc, -0x82,0x5a,0x01,0xec,0x27,0xe5,0x81,0x6b,0x43,0xfe,0xb8,0xbe, -0xb9,0xa4,0x4d,0x9c,0x8b,0x52,0x62,0x01,0x92,0x1d,0xdf,0x0b, -0x90,0x9c,0xa8,0x8c,0x22,0x2b,0xe3,0x62,0xc6,0x88,0x4f,0x07, -0xad,0xe6,0xb0,0xef,0xf5,0xbb,0x7f,0x9b,0x34,0xff,0xd6,0xee, -0xff,0x6d,0x72,0x72,0x47,0x7f,0x9a,0x3e,0xa4,0x0d,0x92,0x6f, -0x86,0x94,0x4f,0xef,0x65,0x9e,0xbe,0xed,0xb9,0x35,0x16,0xea, -0x72,0xe9,0x41,0xff,0xa1,0x41,0xc5,0x15,0x45,0xe5,0xb8,0x7d, -0xa9,0xc6,0x61,0x83,0x9f,0x2e,0xc4,0xe7,0x45,0x57,0xf8,0x92, -0xdd,0xf2,0xed,0xe5,0xc5,0x6c,0x74,0x93,0x64,0xf8,0xe4,0x9d, -0x87,0x47,0x13,0xa7,0x7f,0xf7,0xfe,0x7f,0x77,0x90,0xfc,0x37, -0xef,0xee,0x3a,0x9b,0xfe,0xcd,0xf3,0xc5,0x25,0xe5,0x92,0x5e, -0xbe,0x73,0x19,0xdd,0xad,0x6a,0x7f,0xf7,0x12,0x4d,0x07,0xf2, -0xa5,0x87,0x9a,0x01,0xe2,0xd9,0x7c,0x3a,0x5f,0x56,0x14,0x6e, -0xb0,0x17,0x34,0xd3,0x42,0xd6,0x74,0x3b,0xcd,0xd8,0xaa,0x2e, -0x55,0x46,0xc9,0xdd,0xaa,0x2e,0xec,0x56,0xf5,0x96,0x4f,0x3c, -0xd8,0x0e,0x0b,0xbd,0x46,0x58,0xe2,0x0a,0xcc,0xaa,0xcf,0x92, -0x51,0x96,0x5c,0x4c,0xf8,0x2d,0xe2,0x6a,0xcf,0x2e,0xe5,0xcd, -0x45,0xe7,0x63,0x08,0x5e,0xe1,0xb7,0x65,0x63,0xa1,0x7d,0x16, -0x9d,0x88,0x1f,0x24,0xcc,0x15,0x4f,0x1c,0x2a,0x18,0xcc,0xde, -0x06,0xe3,0xf5,0x65,0x3a,0xde,0x01,0x45,0xf9,0x5b,0xbc,0x60, -0xb9,0x03,0x28,0xc3,0x7b,0xc0,0xc9,0x66,0xb1,0x03,0x04,0x72, -0xd5,0xf5,0xd7,0x5d,0xb5,0x11,0x00,0xe0,0xc2,0xf7,0x86,0x81, -0x2e,0x5d,0x9a,0xbf,0x83,0xcb,0xd1,0xf8,0xbd,0x4e,0xc4,0x0f, -0x7a,0x14,0x61,0xbc,0x0b,0x25,0x03,0xc8,0x4b,0xa1,0x94,0x72, -0x03,0xd3,0xae,0xe6,0x66,0x23,0x65,0x4f,0x2e,0x1d,0x87,0x63, -0xbe,0x5e,0xe9,0x1a,0xe4,0x3a,0x5d,0x00,0x40,0x2d,0x48,0x6f, -0x58,0x19,0xf4,0x3a,0x4f,0xd0,0x80,0xd4,0x16,0x62,0xb0,0x27, -0xdb,0x02,0x01,0x02,0xeb,0x15,0x90,0x02,0x24,0xbe,0x5f,0x8a, -0xa3,0x8f,0xba,0x57,0x63,0xc0,0x61,0xa2,0x1e,0xf2,0xc7,0xdf, -0xbc,0xbe,0xc5,0x77,0x97,0x5b,0x0d,0x0c,0x54,0x8b,0x6e,0x04, -0x7d,0xca,0x70,0x1f,0x5b,0xee,0x62,0x5a,0x70,0x53,0x99,0x8f, -0x6f,0x2a,0x77,0xb1,0xa9,0x3a,0xf4,0x56,0x2c,0xe4,0xf0,0x53, -0xd9,0xfc,0x4e,0xb6,0x23,0x4f,0x37,0xb1,0xcd,0x2f,0x02,0xbb, -0x61,0x30,0xcd,0x2a,0xdd,0x6b,0xd3,0xd1,0xcd,0xc2,0xbb,0xf1, -0xd4,0xed,0x14,0x9a,0xbe,0x79,0xa0,0xe9,0x83,0x76,0x3e,0x23, -0xe5,0xc5,0xa5,0xd6,0xe9,0x41,0x44,0xaf,0xb8,0x16,0x46,0xcf, -0x77,0xe3,0x69,0x61,0xec,0x40,0x4a,0x11,0x26,0x2b,0xc1,0x64, -0x45,0x98,0xef,0x47,0x45,0x9d,0x0d,0x29,0x45,0xbd,0x4e,0xaf, -0xa8,0x14,0x74,0x06,0xa6,0x21,0x5c,0x45,0xe4,0x34,0xe6,0x27, -0x77,0x0f,0xa9,0xe5,0xc9,0x7c,0x4c,0x6d,0x69,0x8f,0xf1,0x5a, -0x49,0xf2,0xcd,0x34,0xc1,0xaf,0x1f,0x5f,0xa2,0x47,0xe9,0x2c, -0x6b,0xab,0xf7,0xb7,0x6f,0xdf,0x06,0xf8,0x7c,0xba,0x52,0x8f, -0xbb,0x91,0x6b,0x23,0x5f,0x25,0xbc,0x6d,0x67,0xc9,0xea,0xcb, -0x15,0x7c,0x5d,0xae,0x57,0x89,0xd7,0xd0,0x65,0x1b,0x81,0x80, -0xf4,0x58,0x0d,0xbf,0x35,0x68,0xdb,0x97,0xd0,0x35,0xbf,0x8c, -0xa6,0xe8,0xc1,0x9e,0xcd,0xa7,0xe9,0x84,0xa2,0x77,0xaa,0x4d, -0x32,0x09,0xf8,0x62,0xe2,0xb8,0xad,0xfa,0x2b,0x18,0x10,0x80, -0x6c,0xd3,0xb5,0xd3,0x9f,0x6b,0xa7,0x52,0x5f,0x91,0x47,0x1a, -0xa9,0x22,0x4c,0x9b,0x7e,0xa5,0xb8,0xac,0xd0,0x1a,0x26,0xe8, -0x65,0xa3,0xc9,0xcf,0xb0,0xaa,0x84,0x66,0xc3,0x97,0xc0,0x08, -0x92,0xc3,0x5f,0x90,0x9c,0xbd,0x4f,0x3e,0xbc,0x56,0xa9,0xf8, -0x81,0x69,0x14,0xac,0x52,0xa5,0xe1,0x07,0x24,0x36,0xf4,0xfe, -0x40,0x89,0xec,0xf8,0xe3,0xa8,0x1b,0x05,0x97,0xdd,0x30,0x18, -0xc3,0x7f,0x13,0xf8,0x9d,0xc0,0xdf,0xab,0x6e,0xb8,0x2d,0xcc, -0x7a,0xaf,0x74,0x6f,0xb8,0xb2,0xa0,0xd3,0x51,0x1e,0xa6,0xa3, -0x1c,0xc6,0xbb,0x38,0xe6,0xf2,0x47,0x9b,0x5d,0x49,0x2b,0x74, -0xb8,0x4e,0xa1,0x0d,0x24,0xe3,0xb6,0x80,0x81,0x66,0x68,0xdd, -0x25,0x9b,0x35,0xbc,0x1d,0xe5,0xee,0xd6,0x10,0x8c,0x3e,0x55, -0xe7,0x9a,0x2e,0xb8,0xa6,0x17,0x18,0xd9,0x8d,0xca,0xe0,0x1e, -0xb8,0x6f,0x07,0x1b,0xcb,0xb6,0x36,0x61,0xe3,0xeb,0xf9,0x72, -0x52,0x79,0xd1,0x6e,0x99,0x30,0x88,0x44,0xdc,0x78,0x0f,0xeb, -0xd4,0x4d,0x48,0xee,0x92,0xa0,0x30,0xe8,0xe5,0xe0,0x0c,0xdb, -0x48,0xbf,0x5e,0xae,0x6f,0x88,0x58,0xfa,0xa0,0x07,0xe0,0xcd, -0xd1,0xd3,0xcc,0xd7,0x05,0x38,0x47,0xdc,0x9d,0xa8,0x66,0x53, -0x8a,0x7e,0xbd,0x27,0x9f,0x39,0xf2,0x6f,0x30,0x4e,0x14,0x1b, -0xf2,0x07,0x13,0xa7,0x07,0x48,0xdc,0x34,0x63,0x16,0x41,0x3c, -0x2d,0x7b,0x37,0xec,0x69,0x0a,0x64,0x35,0x56,0xa8,0x4f,0x31, -0x48,0x93,0xe3,0x07,0xef,0x9b,0xf1,0x86,0x1d,0xcf,0xe6,0xcb, -0xd5,0xbf,0x52,0xed,0xc7,0xc7,0x56,0x89,0xfa,0xd3,0x31,0x53, -0xc0,0xd3,0xd5,0x0e,0x46,0x43,0xc3,0x85,0xc1,0x25,0x1d,0x5e, -0x23,0xdc,0x4b,0xc5,0x26,0x72,0x62,0xb0,0xd0,0x97,0xce,0xb4, -0xe8,0x08,0x6d,0xb2,0xa7,0x5a,0x8d,0xce,0x33,0xad,0x87,0x9a, -0xcd,0x07,0xd5,0x0c,0x75,0xbf,0x8f,0xbd,0x7f,0xfe,0xaf,0x96, -0x3c,0xa4,0x7e,0x32,0xc3,0x38,0xe4,0x9b,0x4a,0xbe,0x86,0xf1, -0xc6,0x61,0x2c,0xbf,0x32,0x96,0xc6,0x86,0x54,0x5a,0xc9,0xbf, -0x8b,0x1d,0xf2,0x07,0x13,0x62,0x7b,0x70,0xab,0x7a,0x01,0xbf, -0x27,0x90,0x30,0x42,0x7c,0x23,0xb0,0xb5,0x9b,0xf1,0xed,0xc9, -0xfb,0x9e,0x96,0x12,0x00,0x68,0xc2,0xa4,0xd4,0x04,0x98,0xf8, -0x23,0xfb,0x5e,0x4e,0x52,0xec,0x23,0xf9,0xfd,0x0e,0x3d,0xea, -0x96,0xab,0x2f,0xc9,0x5b,0x73,0x14,0x06,0xc9,0x6c,0x22,0xbf, -0x23,0x76,0x4b,0xee,0xde,0x6e,0xb7,0x1a,0x95,0x83,0xc3,0x94, -0xdb,0x58,0xe5,0x36,0x52,0xcc,0xdb,0xe0,0x11,0xd6,0xe9,0xfb, -0x2d,0xfa,0x1b,0x09,0x3f,0xb6,0xbc,0x5b,0x60,0x58,0x81,0xdf, -0xef,0xe2,0xb4,0xc0,0x05,0xb9,0xf2,0x68,0xda,0xc0,0x4d,0x78, -0xa7,0x36,0x7f,0xad,0x9c,0x77,0x94,0x93,0x0e,0x7b,0x72,0xc7, -0xaf,0x4d,0x95,0xeb,0x9b,0x98,0xec,0x6d,0x7d,0x7c,0xcc,0xe2, -0x2e,0x6e,0x2c,0x16,0xe0,0x17,0x36,0x5c,0x5f,0x6d,0x35,0x73, -0xa2,0xda,0x69,0x96,0x9b,0x94,0x5d,0x95,0xeb,0xdc,0x3a,0x54, -0x77,0x1e,0xfd,0x2d,0x4a,0xc9,0x33,0xaa,0xe6,0xf8,0x78,0x99, -0x90,0x34,0xb9,0x31,0x0e,0x29,0xe5,0xa3,0x50,0xb2,0x5b,0xda, -0x18,0x8f,0xe7,0x8d,0xda,0x36,0xb1,0xcd,0x51,0xdb,0x69,0xd6, -0x69,0x27,0xf0,0x2e,0x5d,0x88,0xcb,0x22,0x04,0x0a,0x25,0xad, -0xcf,0x09,0xa3,0x0c,0x76,0x11,0xdb,0x60,0x16,0x98,0x11,0xe5, -0x0e,0x9a,0xc0,0x50,0x11,0xb0,0x92,0xfa,0x88,0xe1,0xb7,0xa8, -0x13,0xe3,0x50,0xdf,0x43,0xc3,0x1c,0x99,0x91,0x0e,0x8b,0x2a, -0x33,0x8b,0x65,0xfc,0xc4,0xb8,0x1d,0xcb,0x3f,0xf5,0xae,0x16, -0xab,0x24,0xd1,0x47,0x64,0x9f,0x50,0x92,0xdf,0x65,0xb8,0x2d, -0x7f,0xb6,0x15,0x19,0x07,0x55,0xa8,0x80,0x37,0x3b,0xd0,0x2b, -0xe1,0x14,0xfc,0x86,0x27,0x87,0x55,0x61,0xc1,0xef,0xaa,0xc5, -0x80,0xd9,0x15,0x69,0x8e,0x1f,0x5e,0x97,0x29,0xa2,0xab,0x2b, -0xd6,0xa3,0x41,0xec,0xaa,0xb8,0x3b,0x0f,0xaf,0x47,0xe0,0x55, -0x25,0x46,0xaa,0xed,0x9a,0x18,0x48,0x55,0x33,0x2e,0x54,0x61, -0x22,0x66,0x51,0xc6,0xdd,0x9d,0x99,0xd6,0x04,0xab,0x2a,0x59, -0x64,0x82,0x2e,0xc9,0x19,0x4e,0x49,0xa7,0x69,0xf6,0x74,0x0a, -0xda,0x64,0x5c,0x1d,0x30,0x66,0x99,0x2c,0xd6,0x53,0xb0,0xe7, -0x71,0xfb,0xb8,0x3c,0xfb,0xe3,0x2a,0x0b,0x37,0x81,0x71,0x47, -0x8c,0x2c,0x3b,0x4c,0xe0,0xad,0xf2,0xa3,0x38,0xe6,0x32,0xb2, -0xad,0x48,0x39,0xe3,0x4d,0x0b,0x13,0xdb,0x18,0x77,0x40,0x52, -0x72,0x4e,0x01,0x7b,0x7b,0x16,0x47,0xd6,0x53,0x4e,0x93,0xcd, -0xc9,0x64,0xd3,0x9c,0xe4,0x27,0x13,0xde,0x18,0x5e,0x5d,0x27, -0xab,0x91,0x3e,0xcf,0x9f,0xcc,0xd4,0x5b,0x17,0x8c,0xe5,0x1a, -0x07,0x2f,0xa4,0xc2,0xff,0xd5,0x18,0x23,0xb4,0x8b,0x4d,0x2b, -0x9e,0xa0,0x37,0x26,0x7f,0xe5,0xf0,0x95,0xc3,0xd7,0x51,0xb8, -0x75,0x88,0x3d,0x3e,0x4e,0xb3,0x6f,0xd3,0x59,0x0a,0x86,0x1a, -0x60,0x76,0x50,0x53,0xfe,0x33,0x1b,0xff,0x0e,0xc4,0x5b,0xb1, -0x40,0x8f,0x2c,0xaa,0xac,0xa8,0x39,0x93,0xe5,0xe8,0xed,0xcd, -0xfc,0x36,0xc1,0xfd,0x4f,0xc0,0x4c,0xd6,0x06,0xac,0x97,0xc0, -0xbe,0x06,0x86,0x00,0x12,0xf3,0x9d,0x07,0xd4,0x25,0x14,0xf3, -0xf5,0x06,0x2d,0x5b,0xe2,0x22,0x14,0x0f,0x46,0xd3,0xc5,0xf5, -0x28,0x50,0xf7,0xec,0x33,0x7a,0x3b,0x07,0x45,0x0f,0xd5,0x0d, -0x56,0x97,0x71,0x49,0x54,0x3b,0x84,0x0a,0x71,0x02,0x34,0x98, -0x50,0x30,0x75,0x37,0x68,0xea,0x69,0x04,0x0d,0x58,0x6e,0xbe, -0x87,0x3f,0x30,0xf7,0x34,0x30,0xbe,0xd0,0x6f,0x49,0x3c,0x88, -0x82,0x68,0x18,0x5c,0x2d,0x53,0x22,0x35,0x6e,0x3f,0xa5,0xb3, -0xc1,0xaf,0xd5,0x8d,0x7f,0x63,0xa9,0x11,0xfa,0xef,0xad,0x3c, -0x02,0x7c,0x29,0x64,0x54,0x01,0xaa,0x3c,0xa1,0x30,0x6e,0x9d, -0x85,0x78,0x46,0x7c,0x8b,0x26,0x6c,0x3b,0x0a,0xa8,0x67,0xe3, -0xf6,0xe7,0xc4,0x4a,0xb2,0xa7,0xd4,0xd9,0xb0,0xea,0x4a,0xe6, -0x05,0xd2,0x6c,0xcb,0x28,0xf4,0xa0,0x47,0xdc,0x38,0x01,0x6a, -0x9f,0xe2,0x79,0x52,0xf8,0x48,0x1d,0x41,0x32,0x17,0xa1,0x79, -0x9e,0xc4,0x59,0xc0,0x96,0x32,0xef,0xba,0xf4,0x6f,0x1c,0x82, -0xc9,0x71,0x14,0x92,0x6d,0xfd,0x2b,0x98,0x87,0xf3,0x20,0x0b, -0x56,0xc1,0x34,0x78,0x4f,0xe7,0x1a,0x33,0x92,0x5d,0xed,0x63, -0x7c,0xe3,0xda,0xb1,0x6a,0xe7,0xfe,0x46,0x76,0xee,0xe7,0xb1, -0x32,0x59,0x83,0x2c,0x9e,0xb7,0xd5,0x34,0x07,0x3f,0x65,0x3e, -0xdc,0xc4,0xd0,0xc3,0xad,0x0c,0x7a,0x39,0x87,0x5f,0x39,0xfc, -0xca,0x03,0x6f,0xaa,0x9e,0x3e,0xc3,0xfb,0x25,0xd3,0x98,0x5b, -0xa2,0x7b,0x13,0x90,0x9d,0x78,0x90,0x6c,0x06,0xc4,0xd4,0xf7, -0x5b,0xba,0xdb,0xd1,0x3c,0x3e,0x9d,0x06,0x9b,0x93,0x78,0x1a, -0xe4,0xf8,0x0f,0x56,0x00,0x08,0xbd,0xf7,0x71,0xd6,0xfe,0x90, -0xe0,0x51,0xce,0x29,0xac,0x58,0xf9,0x57,0x53,0x25,0x81,0x0d, -0x89,0xd5,0xc7,0x38,0x04,0x80,0x9a,0x26,0x17,0x88,0x5a,0xef, -0x41,0x06,0xda,0x79,0x13,0xd3,0x69,0xac,0x41,0x22,0x93,0x23, -0xbd,0xe4,0xd3,0x33,0x6f,0x28,0x26,0x83,0x70,0x78,0x8a,0x4f, -0xd1,0xd1,0xef,0x08,0x7f,0x93,0xa1,0xff,0xde,0xf7,0xed,0x6d, -0xf4,0x79,0xac,0x4e,0x9b,0x82,0x39,0x56,0x03,0x16,0x0d,0xfc, -0xf5,0xa1,0xd6,0x39,0x56,0xe3,0xe5,0xf0,0x99,0xc3,0x27,0xd6, -0xc5,0x32,0x41,0xc5,0x0b,0xa2,0xf3,0xe5,0x18,0x16,0x9f,0x6b, -0x5a,0x57,0xfd,0x1a,0x17,0x4f,0x3e,0x49,0x21,0xa1,0x0b,0x33, -0x0d,0x07,0x11,0x7d,0xdf,0xb1,0x16,0x3d,0x43,0x06,0xc6,0x6d, -0xda,0x24,0x93,0xbb,0xbb,0x5f,0xf9,0xf8,0xc0,0x53,0x6a,0x6d, -0xee,0x2b,0x4f,0x1b,0x53,0xce,0xa1,0x9e,0xca,0xf5,0x3d,0x20, -0x1f,0xba,0x73,0xb1,0xc1,0x06,0xe0,0x8f,0xdc,0xef,0x62,0x5a, -0x2b,0xf6,0x30,0xb5,0x45,0xff,0xc6,0xd8,0x46,0xff,0x44,0x0d, -0x21,0x04,0xe5,0xfc,0x9c,0xf2,0xb1,0x5c,0x6e,0xe5,0xfb,0x3d, -0x16,0x52,0x14,0x6b,0x25,0xa5,0x3c,0x2c,0x2d,0x31,0xc5,0x83, -0x62,0x96,0x7e,0x1e,0x1c,0x87,0xd9,0x0a,0x04,0xba,0xe1,0x82, -0x7e,0x97,0x3e,0x15,0x9a,0x82,0xff,0xc6,0x2e,0x34,0x0c,0xaa, -0xd1,0x88,0xab,0x00,0xa3,0x71,0x37,0xb4,0x77,0x4e,0x88,0x08, -0xa9,0x91,0xe0,0x97,0x4d,0xca,0xd7,0xa5,0x88,0x22,0xfb,0x28, -0xd2,0x25,0x2a,0x22,0x6b,0x6c,0xfa,0x9b,0x6e,0xd3,0x21,0x58, -0x41,0xab,0x3a,0x4d,0x04,0x93,0x12,0x09,0x16,0x55,0x5a,0x99, -0x1d,0x4c,0x95,0x2e,0x71,0x10,0x55,0x0a,0x5a,0x51,0xa5,0xb5, -0xee,0x41,0xf5,0x69,0x68,0x83,0x54,0x25,0x29,0x84,0xa2,0x68, -0x0f,0x42,0x27,0xb0,0x7b,0x09,0x97,0xd9,0x4c,0x6a,0x50,0xfa, -0xfb,0xa0,0x2a,0x14,0xb0,0x41,0x26,0x29,0x0a,0x1b,0x4f,0x01, -0x07,0xe1,0x62,0x50,0x83,0x89,0xbe,0x15,0x1e,0x56,0xed,0x07, -0xe1,0xd9,0x20,0x0e,0x82,0xef,0x73,0xa9,0xcd,0x79,0x08,0xed, -0x0d,0xbb,0x1b,0xf2,0xef,0xe3,0xb1,0x49,0xd3,0xa5,0x1a,0x9c, -0x32,0x77,0xda,0x93,0xc8,0x86,0x36,0x5e,0xdb,0xf4,0x64,0xa0, -0x67,0x66,0x29,0xdf,0x57,0xd4,0xf1,0x20,0x56,0x83,0x86,0x9e, -0x43,0xa8,0xb2,0xb2,0xc8,0xd9,0x83,0x96,0xd4,0x80,0x69,0x86, -0x71,0x5f,0x68,0x52,0x36,0xfb,0x2a,0xb0,0xe4,0x40,0x0d,0x7e, -0x89,0x47,0x14,0xea,0x97,0x97,0xfa,0xb4,0x0e,0x0e,0x6e,0x74, -0x92,0xb5,0xd5,0x42,0x17,0x2d,0x2c,0x27,0x0e,0x03,0x83,0x77, -0xe6,0x74,0x3d,0x43,0x5f,0x4d,0x99,0x1b,0x27,0x86,0xe1,0x72, -0x34,0x9b,0xcc,0x6f,0x3c,0xff,0x84,0x46,0xac,0x26,0x54,0xd7, -0xcd,0x0e,0x22,0x1a,0x27,0x53,0x6a,0x68,0x1c,0x0c,0xe9,0x2e, -0x9c,0xba,0xb6,0x66,0xd7,0xad,0x36,0x52,0xde,0xd1,0x0c,0xca, -0xf9,0x64,0x70,0xa9,0x69,0xf4,0xdd,0xb0,0x67,0xe0,0xd5,0x74, -0xea,0xf8,0x2f,0xab,0x89,0xd5,0x0f,0x6c,0x40,0x59,0xc0,0xb9, -0x80,0x5c,0xda,0xdf,0x9a,0xad,0x41,0x55,0x20,0x1d,0x92,0x51, -0x85,0x3b,0x43,0x3a,0x11,0xcc,0x80,0x9d,0x53,0x7f,0xf0,0x41, -0xcd,0x82,0xc1,0xb5,0x9a,0x03,0x4b,0x9e,0x9a,0xce,0xbc,0xc3, -0x57,0x09,0xc1,0xbe,0x90,0x29,0x18,0xe0,0x76,0x98,0x0f,0x0d, -0x3e,0xe2,0xb2,0xc2,0xc2,0x4a,0x03,0x40,0x22,0xd5,0x4f,0x41, -0xae,0x3e,0x87,0x7e,0x55,0x29,0xe6,0x05,0x95,0x92,0x95,0xbf, -0x2a,0xc5,0x9f,0x50,0xaa,0xd9,0xd4,0xbc,0x65,0xd2,0x28,0x45, -0x98,0xc8,0x29,0xa5,0xa6,0x59,0x53,0x23,0x8b,0x15,0x4e,0x79, -0x54,0x0b,0xba,0x86,0x8b,0x0c,0x37,0x36,0x8d,0xe0,0x03,0x0c, -0x01,0x05,0x91,0x33,0x44,0x6e,0x41,0xe4,0x8d,0xe0,0xda,0x82, -0x58,0x08,0x12,0x99,0x43,0x4d,0xba,0x14,0x95,0xb9,0x93,0x2e, -0x05,0x28,0x0b,0x08,0x45,0xac,0x42,0x67,0xd9,0x1a,0xdd,0x2f, -0x72,0xda,0x36,0x9f,0xe2,0xa6,0x0d,0x6a,0xf9,0x85,0xe9,0xce, -0x48,0x0b,0x8e,0x1a,0x95,0x58,0x6c,0x24,0x48,0x9f,0x36,0xe0, -0x76,0xd0,0xa7,0xd4,0x7f,0x89,0x3e,0xdb,0xfa,0x63,0xfa,0x14, -0xe8,0x7d,0xe8,0x73,0xb0,0xd8,0x48,0x8c,0xb9,0x55,0x47,0x9d, -0x65,0x8c,0x39,0x9e,0xca,0x5c,0x88,0xa8,0xe2,0xdf,0x16,0x3d, -0x46,0x22,0x6a,0x1c,0x5b,0xac,0xd2,0xfc,0xd3,0x35,0xeb,0xf5, -0x12,0xc7,0x59,0xf2,0x54,0xed,0x39,0x1b,0x15,0xef,0xb5,0x23, -0xdf,0x68,0xd5,0xf9,0x62,0x0f,0x74,0xa8,0x81,0x71,0x09,0x55, -0x75,0x24,0x09,0xc9,0x77,0x77,0x1e,0xe5,0x82,0x3e,0xbf,0x4c, -0xae,0x61,0x7a,0x9a,0x2f,0x09,0xdc,0xf3,0x25,0x92,0xb0,0x13, -0x86,0xa2,0x8d,0x62,0x8c,0xd9,0xa4,0xd4,0x79,0x01,0xdd,0x08, -0x0a,0x16,0xec,0xd7,0x2a,0xdf,0x80,0x6b,0x48,0x59,0x0a,0x9a, -0x1c,0x58,0xa1,0xec,0x40,0x03,0xb9,0x30,0x60,0xaa,0xa6,0xc3, -0x34,0x23,0x14,0x37,0xf3,0x75,0x96,0x00,0xc2,0x65,0x0d,0x92, -0x1f,0x54,0xbe,0x6f,0x81,0xaf,0xeb,0x28,0xff,0x41,0xb2,0x7d, -0xee,0x68,0xa4,0xcf,0x0f,0x6e,0xe7,0xe9,0xe4,0x41,0xe8,0x77, -0xf1,0x8b,0x76,0x0f,0x96,0x09,0x6e,0x86,0xf2,0xc4,0xc7,0xab, -0xcd,0xa0,0x01,0xe2,0x64,0x5c,0x1a,0xeb,0x96,0x8c,0x71,0x27, -0x2c,0xd6,0x68,0x2f,0x14,0xe3,0xa8,0x67,0xf6,0x26,0xae,0xd3, -0x64,0x39,0x5a,0x8e,0xaf,0xf3,0x9a,0xfd,0x89,0xf1,0x7a,0x29, -0xfb,0x13,0xc1,0x24,0x59,0x80,0x8e,0xe6,0x95,0x01,0xdf,0x58, -0x41,0xbf,0xbb,0x2c,0xa6,0x3f,0x80,0x9a,0x1b,0xa3,0x11,0x06, -0xa6,0x10,0xfb,0x9e,0xe1,0x52,0x9e,0x3e,0x63,0x0b,0x13,0xcf, -0x24,0xb4,0x97,0x11,0x30,0x3e,0xd0,0x49,0x33,0xc6,0xa9,0x3d, -0x70,0xcc,0x4c,0x8d,0xae,0x71,0xb4,0x26,0x1a,0x93,0xae,0x6c, -0xab,0xba,0x71,0xc4,0xdd,0xd2,0x91,0x00,0xe1,0x6e,0xea,0xe5, -0xc6,0x24,0x56,0x6d,0x60,0x94,0x38,0x94,0xde,0x49,0x23,0x70, -0x57,0x60,0xb4,0xc4,0x55,0x3c,0x91,0x3a,0x56,0x57,0x17,0x83, -0xdb,0x66,0x3c,0xe1,0xad,0xca,0x1e,0x6e,0x26,0x1d,0x1f,0x8f, -0x79,0x63,0x14,0xff,0xf1,0x79,0x1f,0x19,0xa9,0x44,0x02,0xe8, -0x23,0xbe,0x15,0xa7,0xc6,0x8a,0xac,0xa6,0x04,0xd1,0xa8,0xe5, -0xcd,0xdd,0x5d,0xe8,0xdb,0xbb,0x2a,0xf6,0xe6,0x2c,0x95,0xb5, -0xb8,0x6f,0xf1,0x1d,0x1b,0xed,0xb0,0x00,0xdb,0xcf,0x3a,0x88, -0xbf,0x0d,0x1f,0xb1,0x6f,0x14,0x27,0x0d,0x23,0x89,0x8d,0x25, -0x7e,0x41,0xd3,0x55,0xb5,0xaa,0x2c,0xb1,0x4c,0xb4,0x8f,0x6a, -0xe0,0xed,0xfd,0xda,0x55,0xc9,0xb2,0xe0,0xd6,0x34,0x55,0x63, -0xc0,0xcd,0x1b,0xa2,0x4f,0x76,0x2b,0x14,0x06,0xd5,0x8b,0x93, -0x20,0x54,0xbd,0xc7,0x2b,0x2f,0xde,0x98,0xe7,0x18,0xf2,0x22, -0xf1,0x1a,0xd9,0x4b,0x48,0x0f,0x34,0xb7,0x2a,0x00,0x9e,0x69, -0xde,0x11,0x4d,0x15,0x10,0xbf,0x90,0x14,0x08,0x11,0x3a,0x95, -0xc4,0xe1,0xf0,0x2d,0x4b,0x58,0xa1,0xe9,0xa2,0xbc,0x3f,0xb9, -0x35,0x09,0x46,0x86,0x0f,0x5c,0x50,0x08,0xb4,0x83,0x53,0xa5, -0xda,0x78,0xb9,0x4d,0x07,0x21,0x65,0x50,0x07,0x23,0x25,0xd9, -0xe8,0x44,0x2c,0xec,0x57,0x1c,0xe6,0xc6,0xff,0x5a,0x09,0x0d, -0x39,0xf7,0x85,0x3e,0x7b,0xe0,0x99,0xd2,0xf6,0x76,0x28,0x8c, -0xb9,0x55,0xea,0x2e,0xc8,0xaa,0x8c,0x75,0x12,0xa5,0x4d,0x40, -0x51,0x54,0x77,0xc9,0xbd,0xd6,0x2d,0xe4,0x4f,0x4c,0xbb,0x9d, -0xb1,0xd1,0x35,0x18,0x5c,0x9a,0xbf,0x36,0x18,0xfa,0x99,0x7f, -0xe6,0x31,0xa4,0xee,0x1e,0x27,0x2c,0x84,0x74,0x39,0x9c,0x76, -0x2f,0x68,0x07,0x65,0x13,0x1b,0x01,0xee,0x4f,0x36,0xa7,0xe6, -0xab,0xab,0x6f,0xec,0x6b,0xfa,0xc7,0xb1,0x3d,0x80,0xa0,0x15, -0xf1,0x98,0x61,0x4f,0xb8,0x49,0x78,0xfe,0x34,0xb1,0xb7,0x2f, -0x91,0x5c,0xcf,0xec,0xea,0xd6,0x8c,0xf2,0xc9,0xfd,0x47,0xb9, -0xd1,0x85,0xb4,0xf8,0xb8,0x19,0xa1,0x37,0x28,0x57,0x67,0x91, -0x68,0x0e,0x84,0xa3,0xe6,0xc4,0x90,0xa5,0x7b,0x8b,0x0f,0x24, -0xcd,0xd8,0xb4,0x04,0xd8,0xf1,0xfa,0x37,0x37,0xa6,0xac,0x8e, -0xa4,0x67,0xaf,0xc3,0x40,0x99,0xf7,0x6a,0x83,0xcb,0xb4,0x19, -0x01,0x7c,0x7b,0x4c,0x9b,0x89,0xa9,0x62,0xb2,0xf2,0xec,0x6d, -0x55,0x73,0x8e,0x2d,0x94,0xde,0x7f,0xff,0x44,0x17,0x55,0x7b, -0x28,0x15,0x9a,0xe0,0x67,0x9e,0x8e,0x35,0xa8,0x35,0x58,0x1c, -0xe9,0x4e,0xab,0xb7,0xfa,0x21,0xdd,0xba,0xcf,0x43,0x82,0x50, -0x75,0xe7,0xd8,0xba,0x38,0x69,0x6b,0x58,0x28,0x4d,0xcc,0xa5, -0x30,0x33,0xcd,0xaa,0x67,0xd8,0xcc,0xb1,0x67,0xdf,0xfc,0xac, -0x79,0xea,0xc2,0x00,0xd0,0xc1,0x70,0x15,0x3a,0x75,0x6e,0xda, -0x57,0x3f,0x6a,0x50,0xa9,0x6c,0xbf,0x35,0xc2,0xc0,0x68,0x6d, -0xb0,0x2e,0x59,0x9b,0x64,0x12,0x52,0xc7,0x9c,0xf9,0x5b,0xae, -0xe2,0x7e,0x4f,0x9c,0x54,0x68,0x76,0x4d,0xcd,0xf9,0x3a,0x69, -0xcb,0xd8,0x52,0xc4,0xd0,0x70,0xd4,0xe2,0x5f,0xb1,0x22,0xee, -0x6b,0x3e,0xc1,0xc2,0x52,0x77,0x2b,0xd7,0x07,0x2b,0xdb,0x96, -0xfc,0x82,0xc5,0x67,0xb7,0x12,0x92,0xea,0xd0,0xde,0xe8,0xf8, -0xf7,0xdd,0x50,0x05,0xb1,0x82,0xae,0xcc,0x9c,0xb7,0x3a,0x2a, -0x4f,0xe5,0x65,0x08,0x4c,0x7a,0x08,0x8e,0x46,0xf7,0x47,0x44, -0xd3,0x55,0x38,0x59,0x19,0x4c,0x62,0x4d,0x88,0x73,0x8c,0x6d, -0x9d,0x62,0xc3,0xd8,0xc7,0x03,0x0d,0x34,0x3a,0xc7,0x2c,0xf0, -0xac,0x5d,0xc5,0x93,0xb3,0x30,0xa3,0xb9,0x5c,0xb0,0x70,0xc6, -0xe6,0x84,0x3b,0xfe,0xe7,0xff,0xd2,0x23,0x21,0x4d,0x3e,0x45, -0xfb,0x43,0x31,0xad,0xf7,0x11,0xc5,0xbd,0x27,0x39,0x42,0xc0, -0xd3,0x1b,0x95,0x37,0x64,0x1e,0x86,0xc5,0xc0,0x2b,0x5c,0x3a, -0x85,0x31,0xea,0xb6,0x1e,0x84,0x4f,0x43,0x0b,0x36,0xf5,0x4d, -0xb8,0x8a,0x56,0xb4,0xcb,0xe2,0xf8,0xe3,0xd6,0x32,0x92,0x39, -0x64,0x57,0xd5,0xa8,0xa6,0x1c,0x92,0x28,0x96,0x76,0x71,0x14, -0x48,0x96,0xe9,0xbe,0xd1,0x2d,0x52,0xcb,0xc3,0x9b,0xd0,0xa8, -0x01,0x2e,0xf7,0xb7,0x18,0x49,0xdd,0xd5,0xe8,0x49,0xe1,0x06, -0xb8,0x41,0x3c,0xd8,0xd8,0x38,0x6f,0xe9,0x8e,0x43,0x31,0x85, -0xef,0x53,0xd3,0x35,0xec,0x2c,0xe6,0x1b,0xdc,0x16,0x84,0xdc, -0x76,0xe2,0x06,0xf5,0x54,0x63,0xce,0xda,0x8b,0x64,0x79,0x83, -0x3e,0x64,0x9c,0x22,0xc5,0x35,0xbd,0x16,0x80,0x20,0x10,0x80, -0x9e,0xda,0x08,0x9a,0x07,0xf3,0xab,0xab,0x2c,0x01,0x58,0xfe, -0x5b,0x5b,0x67,0x30,0x53,0xad,0xd7,0xfb,0x44,0xfc,0x8d,0x81, -0xb4,0xac,0x63,0x22,0x7b,0x93,0x0b,0xbf,0xc9,0x77,0xc9,0xe0, -0xd4,0x65,0xd0,0x33,0x65,0x2e,0x95,0xa2,0x9e,0x08,0x74,0x58, -0x2e,0x8e,0x70,0xe5,0x07,0x76,0xb8,0xb3,0x6a,0x2c,0xec,0xdf, -0x32,0x6f,0xea,0x2b,0xe0,0xad,0x48,0x4a,0x07,0xe6,0x9a,0x3e, -0x63,0xd3,0xd7,0xdd,0xf0,0x36,0x9a,0xad,0xed,0x6d,0x57,0x51, -0xb9,0x8a,0xaf,0xc4,0x8f,0xea,0xfa,0x09,0x13,0xbf,0x16,0x27, -0x4a,0x26,0xb7,0x04,0x42,0xa9,0x7f,0x4d,0x96,0x73,0xf4,0x36, -0x2d,0xe5,0xae,0xe5,0xae,0x83,0x9d,0x28,0xb7,0x14,0xec,0x24, -0x1d,0xff,0x89,0x23,0x61,0x09,0x7d,0x87,0x2b,0x0b,0x3c,0xb8, -0xa0,0xa2,0xa2,0x2f,0xb2,0x2d,0x7f,0xb6,0xb9,0x59,0x07,0x21, -0x62,0xd0,0x9a,0x8d,0xf1,0x2a,0xc6,0x64,0xe4,0x48,0xba,0x61, -0x47,0xd2,0x5a,0xb6,0x09,0x59,0x84,0x5d,0x53,0xc5,0xac,0x3c, -0x8c,0x2c,0x86,0x3d,0x90,0x2e,0x16,0xa9,0x5a,0xc2,0x4c,0x67, -0x29,0xb2,0x28,0x45,0xd1,0x65,0x79,0xa7,0xfc,0xb6,0x73,0x33, -0xfd,0x37,0x55,0x7e,0xa3,0x8a,0xe6,0x87,0x15,0xcd,0x4d,0xd1, -0x5c,0x73,0x63,0xbd,0x3a,0xac,0x30,0x02,0xea,0xe2,0xf0,0x21, -0x08,0x8a,0x9a,0xd3,0xea,0x21,0xdb,0x35,0x1a,0xa3,0xfd,0xb0, -0xe7,0x73,0xc5,0x7d,0x66,0xda,0x17,0x76,0x2e,0x8f,0x81,0x55, -0x6a,0x14,0x66,0x01,0xf9,0x0f,0xa3,0xcd,0x73,0x56,0x0c,0x19, -0xfa,0x14,0xd6,0x81,0xfd,0x9c,0x4c,0xd6,0xe3,0xe4,0xe5,0xfa, -0xa6,0x64,0x7a,0xcc,0x76,0x05,0xb7,0x80,0x9a,0x31,0xb4,0x05, -0xfe,0xb9,0xa4,0xf0,0x3e,0xf3,0x05,0x86,0x5f,0xe2,0xdb,0x80, -0x21,0x7e,0xd2,0x86,0x1e,0x27,0x64,0x26,0x30,0x9b,0xd9,0x83, -0x7b,0x17,0xa7,0xca,0x31,0x8e,0xa1,0xce,0xa1,0x50,0xdf,0x83, -0x7f,0x9a,0x31,0x52,0x8c,0x9a,0x03,0xd1,0xf0,0x06,0xc3,0x3b, -0xbc,0xe9,0xc3,0x70,0x26,0x5b,0xb0,0x6b,0x08,0x35,0x40,0x55, -0xba,0x79,0x9d,0x45,0x1e,0x77,0x41,0x7c,0x78,0x69,0x8b,0xd3, -0x8b,0x3c,0x36,0x17,0x6e,0xcb,0xb6,0x97,0xc1,0xb5,0x0d,0x1a, -0xe2,0xb2,0xdd,0xa8,0x1c,0x71,0x32,0xa6,0xd8,0xff,0xbd,0x42, -0xee,0x75,0x77,0x67,0xe9,0xf4,0x7a,0xbe,0x4e,0x56,0xab,0x12, -0x21,0x46,0xf7,0x17,0xba,0x9b,0xef,0xe0,0x69,0x7d,0xce,0x3d, -0x3b,0x18,0x92,0x1c,0xd0,0xf3,0x24,0x55,0x07,0x19,0x2a,0xe8, -0xe5,0x5c,0x98,0x8f,0x41,0xfd,0x40,0x25,0x8b,0xf5,0xc5,0xea, -0xb7,0x37,0x3f,0x07,0x1c,0xb0,0x2e,0x42,0x4c,0x73,0x16,0x19, -0x39,0xaf,0xf0,0xb7,0x45,0x8c,0x79,0x88,0x67,0x26,0x08,0xda, -0x92,0x9e,0xf0,0x4f,0x3b,0x8a,0xf7,0x79,0xb8,0x0d,0x3e,0xa4, -0x6f,0xdf,0x4e,0x6b,0x5b,0x05,0xb3,0x03,0x45,0xc5,0xc9,0xce, -0x70,0x9d,0x0a,0x2a,0x39,0x2c,0x34,0x73,0xa3,0x9a,0x09,0x0d, -0xd6,0x1e,0x65,0xa6,0x71,0x39,0x46,0x09,0x89,0x61,0x6a,0x0a, -0x69,0xef,0x2a,0x2a,0xb7,0x34,0x8b,0x2c,0x39,0xcb,0xa2,0x62, -0x5b,0x35,0x18,0xbd,0x75,0x0b,0x2b,0x60,0x89,0xe9,0x08,0x7f, -0x5b,0x74,0xbb,0xca,0x5c,0x80,0x44,0x50,0x74,0xa5,0xcd,0xce, -0x62,0xcf,0x41,0xd2,0xd2,0x5f,0x2d,0xbe,0x84,0x75,0xea,0x75, -0x60,0x8d,0xea,0xf7,0xd2,0xf3,0xf7,0x50,0xf4,0xbd,0x9f,0x9d, -0x35,0xa5,0xc8,0x7b,0xa7,0xc8,0x7b,0xab,0xc8,0x64,0xd3,0xcb, -0x3a,0x20,0xce,0x67,0x27,0x0e,0xee,0x2d,0x33,0x78,0xde,0xa2, -0x58,0x3d,0x9d,0xd3,0x2c,0x02,0xcc,0x5d,0xe8,0xc0,0xf0,0x7c, -0x8e,0x87,0x0f,0x61,0x5c,0xd7,0x29,0x2d,0x60,0x8a,0xdd,0x11, -0xc9,0x66,0x31,0xb2,0xdf,0xb8,0xbe,0xa7,0x78,0xbd,0x8f,0xa3, -0xd3,0xd9,0xef,0x12,0x2b,0x58,0x74,0xcf,0x9d,0xbd,0x7b,0x84, -0x72,0x40,0x4e,0xe3,0x79,0x79,0x8f,0xbe,0x04,0x15,0xbf,0xaf, -0x91,0x42,0xa7,0xb9,0xbf,0xc1,0xa4,0x51,0x3d,0xdb,0xe0,0x74, -0xc2,0x77,0x3f,0xf4,0xaa,0x38,0x5b,0x61,0x20,0xec,0x9b,0x4a, -0x0b,0x55,0xe7,0x8a,0x95,0x6a,0xf6,0x52,0x61,0x39,0x1b,0x6c, -0x02,0xf8,0x97,0xc6,0x5d,0x71,0x45,0x4a,0xdf,0xcb,0x80,0x43, -0xd9,0x91,0x12,0x89,0xe9,0xdf,0xa5,0xb5,0xdc,0xe7,0x32,0x68, -0x61,0xae,0xae,0x97,0x49,0x76,0x3d,0xc7,0xed,0x60,0x40,0x38, -0x73,0xa0,0xa8,0x98,0x05,0x2b,0xce,0xeb,0x62,0xff,0xea,0x0e, -0x33,0x28,0x74,0x78,0x30,0xe8,0xb5,0xab,0x65,0xf2,0xeb,0x3a, -0x99,0x8d,0xf3,0x7e,0xd4,0x85,0x1e,0xa4,0xad,0x8c,0x9b,0x9e, -0x0f,0x95,0x60,0x45,0x1c,0x94,0x60,0x88,0x8d,0xc0,0x0d,0x1e, -0x83,0x62,0x90,0x36,0x41,0x4a,0x3d,0x4c,0x77,0x93,0xc1,0xd6, -0xc3,0xc4,0x9c,0x77,0x51,0x6e,0xce,0x43,0xbf,0xe8,0xa0,0xb2, -0xb1,0x16,0x6f,0x9b,0x73,0x6e,0x34,0x08,0xd2,0xf1,0xf1,0xe6, -0x0b,0xf9,0xc0,0x98,0xb6,0x9e,0xa6,0x00,0x4f,0x34,0x52,0x0a, -0x56,0x67,0xea,0x01,0xbe,0x46,0xc1,0x0d,0x45,0xe6,0xa3,0xda, -0x9a,0xf1,0x7b,0xfa,0x61,0x02,0x77,0xda,0xfb,0x2d,0x88,0x86, -0xec,0x44,0xdd,0x58,0x8c,0x35,0xc1,0x1d,0xa0,0x16,0x84,0xcc, -0x7b,0x67,0x87,0x52,0xfa,0xf5,0x67,0x62,0x2f,0x73,0xbd,0x2a, -0xff,0xab,0x74,0xf6,0x12,0xaa,0x79,0x9b,0x64,0x66,0x1b,0x53, -0xb2,0xee,0xbd,0x48,0x5c,0xd2,0x1e,0xa1,0x94,0x16,0xdb,0x6f, -0xb9,0x35,0x49,0x3c,0xdb,0x1c,0x86,0xd0,0xb4,0xc8,0xba,0xb4, -0x6e,0x21,0xe7,0x7c,0x1b,0x39,0x89,0xea,0x41,0xb8,0x85,0x1b, -0xa5,0xa3,0xdb,0x8d,0xd9,0x40,0x20,0xf4,0xf6,0x85,0xab,0x0a, -0xbe,0x7d,0x8b,0xee,0x49,0x0c,0x19,0x6c,0xfc,0x6d,0xb7,0x96, -0x56,0xae,0xcf,0xa6,0xd5,0x74,0xe5,0x81,0x5e,0x27,0xba,0xe7, -0x8f,0x1c,0x16,0xeb,0x0c,0x0b,0xb9,0xbd,0xdd,0xe4,0xc6,0xa9, -0xb0,0xc6,0x3d,0x47,0xa4,0xa0,0xf1,0xae,0x92,0xae,0xd2,0x65, -0xb6,0xfa,0xcb,0x68,0xfa,0x9e,0xf7,0x56,0x17,0x60,0x07,0xa4, -0xf3,0x75,0xf6,0x32,0xbd,0x9c,0xea,0x8b,0x49,0x35,0xbb,0x8e, -0x72,0x7d,0x85,0x12,0x2f,0x10,0xf5,0xfe,0x3d,0x48,0xeb,0xcc, -0x46,0xd5,0x44,0xdb,0xed,0xbc,0xf1,0x1a,0x10,0x31,0x94,0x60, -0xf6,0x49,0x43,0x8c,0x72,0x36,0x4e,0x32,0x8a,0x9d,0xaf,0xf3, -0x9d,0x3d,0xcc,0xb1,0x5b,0x02,0xc6,0xa8,0x69,0x15,0x23,0x76, -0x2a,0xf3,0x0d,0xc2,0xd1,0x62,0x01,0x26,0x20,0xed,0xcc,0x96, -0xe1,0x34,0x98,0xef,0xa6,0x73,0x4d,0x3d,0x23,0x1e,0xd8,0xf6, -0x97,0xd7,0xe9,0xd5,0x8a,0x77,0x6a,0xc9,0x14,0xbe,0x49,0x27, -0x1c,0xbf,0x02,0x03,0x4b,0x18,0xba,0x99,0x51,0x78,0x8d,0x6b, -0x9a,0xde,0x34,0xc7,0xa5,0x24,0xbf,0x57,0xe8,0x81,0xbe,0xa7, -0xf6,0x10,0x29,0x3f,0x2e,0x64,0xbb,0xf8,0xb2,0x64,0x31,0x5a, -0x8e,0xcc,0x4e,0x79,0xb1,0x37,0xa5,0xcb,0xda,0x37,0xf3,0x49, -0xec,0xa0,0x6d,0x29,0x72,0xfd,0xae,0x5b,0x9d,0x4a,0xe7,0xa3, -0xac,0x02,0x3e,0xf4,0xa2,0xfc,0xe3,0x88,0xb3,0x6e,0x20,0x80, -0xe6,0x9c,0xcf,0x26,0x46,0x2a,0x61,0x94,0xc8,0x6e,0xbe,0x11, -0x36,0x85,0x76,0xd3,0xdb,0x7d,0x0c,0x70,0xd0,0x96,0xbe,0xde, -0xce,0xdf,0x34,0xed,0x2a,0x80,0x4f,0x4a,0xc6,0x2c,0x92,0x9c, -0x2d,0x7c,0xfb,0x89,0x4e,0x23,0x4d,0x55,0x0d,0x34,0xe2,0x84, -0x8e,0x3c,0xa5,0x81,0xa6,0x46,0x46,0x86,0x82,0x14,0xdc,0xa6, -0x0b,0x3e,0x80,0xbc,0x9d,0xab,0x1f,0x65,0xfe,0x42,0xe6,0x0d, -0x93,0xcb,0x47,0x96,0x6d,0x7b,0xd0,0x64,0x80,0x01,0xb0,0x98, -0x96,0x04,0x19,0xa0,0x02,0x74,0x76,0x0a,0xe0,0x04,0xbc,0x0e, -0x0c,0xa4,0xcc,0xad,0x94,0x1e,0xd6,0xeb,0x8a,0xfa,0xcf,0xe8, -0xa8,0xe2,0x41,0xba,0x4f,0x64,0xba,0x99,0xdf,0x27,0x57,0x98, -0xb7,0xc0,0xbc,0x9b,0xe3,0x63,0xf8,0xd5,0xf3,0x91,0xcc,0x2a, -0xa8,0x39,0x62,0x98,0x17,0x31,0x08,0xfa,0xf9,0x82,0x32,0x85, -0x12,0x3d,0x62,0x89,0x17,0xc4,0x23,0x8b,0x72,0x25,0x61,0x20, -0xc8,0xa6,0xc9,0x22,0xd9,0xc0,0x07,0x5b,0xf4,0xa0,0x50,0x40, -0xe4,0x11,0x0e,0x72,0x3c,0x73,0xab,0xff,0x81,0xbc,0xc5,0x9d, -0xa4,0x2f,0xa5,0x72,0x2a,0x4c,0x04,0x18,0xcd,0x60,0xe8,0xc1, -0xe3,0x07,0x5c,0xdb,0x51,0xff,0x65,0x73,0xf5,0x13,0x93,0x61, -0x4d,0x57,0x60,0x33,0x42,0x16,0x7b,0x07,0xa1,0xe6,0x6e,0x67, -0x20,0x94,0xdd,0x63,0x3d,0x62,0xea,0x51,0x1d,0xc3,0xf0,0x10, -0x56,0x83,0xa3,0xe5,0x31,0x9a,0xc4,0xd4,0x60,0x1b,0x07,0xd0, -0x85,0x6c,0x21,0xfe,0xa6,0x8b,0x12,0x36,0xdd,0x35,0x84,0xec, -0xa6,0x88,0x6c,0x11,0x38,0x34,0x22,0xb2,0x05,0x20,0xbb,0x09, -0x9c,0x1e,0xd2,0xf7,0x42,0x55,0xea,0x76,0xff,0x19,0x11,0x1d, -0x0f,0xc6,0xea,0xdc,0xa7,0xa0,0x5e,0x29,0xc0,0x08,0xbd,0xe5, -0xc1,0xa7,0x89,0xe6,0x76,0x73,0xe5,0xac,0x65,0xc6,0x70,0xfc, -0x51,0x91,0xd0,0x55,0xa0,0x20,0x15,0xb0,0xda,0x00,0xe2,0xf1, -0x1a,0xe7,0x35,0x4e,0xe4,0xf0,0x83,0xfa,0x0a,0xfe,0xb2,0x75, -0xd0,0x2d,0x6a,0xe1,0x6a,0xcd,0xc6,0xc0,0xcd,0xa8,0x1b,0xe2, -0xae,0xbd,0x99,0x74,0xe8,0x14,0x34,0xb0,0x54,0x06,0xd1,0xdc, -0xa2,0x20,0x25,0xae,0xb6,0xa7,0x78,0x64,0xc0,0xf1,0xc2,0x18, -0x78,0x99,0x20,0x87,0x24,0x2a,0x4a,0xa9,0x7b,0x6e,0xe6,0xd9, -0x4a,0x85,0x29,0x3a,0xb4,0x1c,0x09,0x09,0x17,0x9c,0x24,0x49, -0x71,0xcc,0xd5,0x97,0xfb,0x1a,0x80,0x13,0x2c,0xb5,0x09,0x63, -0x24,0x14,0xbd,0xea,0xcd,0x68,0xc2,0x14,0x26,0x04,0x6f,0x78, -0x6d,0xa2,0x98,0x7e,0xb7,0x37,0xf6,0x88,0xa3,0xa4,0x00,0x21, -0x11,0x26,0x8f,0x62,0xac,0x9f,0x0f,0x62,0xef,0xee,0xa2,0x5e, -0xc9,0xd8,0xaa,0xea,0x6e,0xec,0xba,0x97,0xe9,0x6f,0xd6,0x59, -0x0f,0x9f,0x89,0xf2,0xac,0x70,0xa2,0x5d,0x04,0xcb,0x47,0xbd, -0x72,0xaa,0x08,0x8d,0x9e,0x26,0xab,0xe4,0x81,0x11,0x0c,0xeb, -0x34,0xc8,0xc6,0x15,0xcb,0xb9,0x31,0x5d,0x08,0x54,0xb1,0x2d, -0x6b,0xd1,0x9f,0xe6,0xd1,0x8e,0x1a,0xee,0x71,0x74,0xc9,0x5b, -0x51,0x78,0x00,0xe6,0xb7,0xc5,0xc1,0x03,0x7f,0x07,0x86,0x8f, -0xa5,0x1e,0x53,0x19,0xf6,0x65,0x12,0xc5,0xa7,0xf8,0x48,0x33, -0x96,0xa3,0xa1,0x18,0x34,0x87,0x9d,0xba,0x18,0xf8,0x4d,0xc0, -0x71,0x4e,0x4c,0xd2,0x36,0xa8,0x88,0xb0,0xb2,0xcf,0x3b,0x9e, -0xa8,0xe2,0x58,0x29,0x72,0xc0,0xea,0x0b,0x62,0xdd,0xb7,0x14, -0x40,0x85,0x8f,0x5a,0x79,0x74,0xa9,0x62,0xf7,0xaf,0xe4,0xa8, -0xb6,0x12,0x4c,0xa6,0x98,0x2d,0xbb,0xce,0x73,0xb1,0x4c,0xdd, -0x51,0x6e,0xdd,0xa9,0xcf,0x82,0x0e,0x7d,0x0e,0x3b,0x0e,0x77, -0x55,0x5d,0x95,0x8b,0x6b,0xb0,0x14,0x66,0xe1,0x13,0x8e,0xeb, -0xac,0xaf,0x2f,0xa5,0x74,0x2b,0x36,0xa1,0x05,0x86,0xff,0x54, -0x84,0xae,0xe0,0x8c,0xad,0x0e,0x0c,0xb5,0x89,0xe9,0x4f,0x1e, -0x87,0xc1,0x81,0x5a,0x96,0x6e,0x4f,0x2d,0xe3,0xe6,0xd2,0x13, -0x1f,0x28,0xdf,0xd9,0xff,0xab,0x2a,0x6b,0x9d,0xa2,0x01,0x67, -0x44,0x6f,0xe2,0xc9,0x10,0xdf,0x60,0x94,0x23,0xd3,0xa5,0xba, -0x6e,0x7b,0xe2,0x49,0x2b,0xa2,0xae,0x76,0x48,0xe8,0x9c,0x10, -0x9d,0xcb,0xd3,0x0f,0x81,0xfe,0x09,0x36,0xdb,0x69,0xe7,0xd0, -0xc9,0x81,0xc9,0x6e,0xc6,0x93,0xe5,0x27,0x93,0x7b,0xaf,0x9a, -0x5a,0x54,0xd3,0xb6,0xa4,0xd2,0x10,0xa5,0xbe,0xf9,0xcf,0x25, -0x3f,0x80,0x2e,0xbc,0x86,0xff,0x74,0xab,0xe5,0x26,0xde,0x8e, -0x76,0x5b,0xda,0x84,0x4b,0x05,0x07,0x29,0x95,0x42,0xcb,0xc8, -0x5d,0x4d,0x5f,0x8f,0xad,0x74,0x99,0x18,0xbf,0xbf,0x47,0xf8, -0x24,0x82,0xbc,0x08,0x16,0x74,0x20,0xc0,0xa3,0x97,0x30,0x30, -0x89,0x87,0xe1,0x10,0x58,0x89,0xc4,0x74,0x77,0x57,0xf9,0x84, -0xeb,0x45,0xb7,0xa9,0xaa,0x11,0x81,0xe6,0x8a,0x4a,0x77,0x6c, -0x77,0x3e,0x9f,0x2c,0xc0,0x1a,0x95,0xbe,0x50,0xbb,0xcb,0xbb, -0x63,0xfc,0xbe,0x46,0x1b,0x8c,0xa7,0x6b,0x7c,0x73,0xae,0x52, -0x21,0x48,0x9e,0xd1,0x09,0xca,0x86,0xc0,0x30,0x6d,0xc1,0xbd, -0x14,0xc4,0x26,0x0e,0xef,0x63,0x0f,0xed,0xf4,0x8c,0x32,0xeb, -0xa1,0xc2,0x62,0xa8,0xaf,0x3c,0xa6,0x4c,0x4d,0xd2,0x84,0xd7, -0x7a,0xc9,0xe3,0xab,0x29,0xb0,0x04,0xf3,0xc6,0xc0,0xf8,0x5d, -0x85,0xc9,0x6e,0x71,0x1f,0x96,0x56,0x75,0x4b,0x40,0xcc,0xf7, -0xbb,0xa1,0x42,0x1e,0x3a,0x19,0x62,0x50,0x56,0xda,0x4a,0x52, -0x39,0xd9,0xac,0x6c,0x74,0x15,0xad,0x22,0x81,0x60,0x1b,0x99, -0x41,0xfe,0x18,0x4b,0xe6,0x8f,0x30,0x5c,0xb4,0xb1,0xc1,0x6a, -0xb9,0x64,0x68,0xb0,0xba,0x56,0xf7,0x75,0x7d,0x65,0x6a,0xfc, -0x0e,0xd3,0xc5,0x8b,0x5a,0x82,0xb4,0xcf,0x29,0xa7,0xfc,0xd5, -0x8d,0x7c,0x83,0xfe,0xbf,0x84,0xdd,0x22,0x1d,0xf7,0xfb,0x4c, -0x17,0x41,0xe2,0x5a,0x2f,0x1a,0xf3,0xef,0x36,0x60,0x34,0xfa, -0x2a,0x1b,0x46,0x55,0xf3,0xc7,0x98,0x31,0xe5,0xaa,0x0e,0xb2, -0x64,0xa4,0x58,0x8d,0xfa,0xc2,0x0e,0xb9,0x19,0x2d,0xaa,0xbd, -0x58,0x28,0x34,0x8e,0xde,0xec,0x7b,0x6f,0x76,0x28,0x78,0xab, -0x0c,0x5f,0xf3,0x51,0xdb,0xf6,0x05,0x4d,0xa2,0x1f,0xf8,0xc1, -0x67,0x8b,0xbc,0xd2,0xee,0x9c,0x48,0xca,0x89,0x17,0x9e,0xbf, -0xef,0x87,0xdd,0xf7,0xe2,0xe4,0xcd,0xcf,0x23,0xf1,0x45,0x14, -0xfc,0x89,0xce,0xc2,0xe7,0x31,0xfe,0x02,0x20,0x7a,0xc2,0xc8, -0xd0,0xf6,0xeb,0x7a,0xb4,0x4c,0xaf,0xf2,0xfd,0xda,0xce,0xd9, -0x00,0x2a,0xee,0xfe,0x98,0x82,0x41,0x36,0xc6,0x87,0x24,0x67, -0xc1,0x12,0x23,0xee,0xc1,0x9c,0x20,0xde,0xe7,0xcb,0xf9,0x07, -0x3c,0x4a,0x58,0x02,0x97,0xd2,0x19,0xce,0x1b,0x1a,0x85,0x7a, -0xa2,0xe7,0x12,0x16,0x47,0x18,0x03,0x35,0x58,0xc7,0x0d,0x4a, -0xc3,0xb0,0x70,0x18,0xe1,0xaa,0x8f,0xa8,0xda,0x93,0x4d,0xb7, -0x31,0x29,0xa7,0xe6,0x5d,0x06,0x6e,0x39,0x79,0xd1,0xb1,0x59, -0x59,0x68,0x40,0x85,0x86,0xad,0x83,0x14,0xb4,0x10,0x27,0x04, -0x02,0x20,0x0f,0x98,0x51,0x6f,0x69,0x3a,0x25,0x73,0x73,0x22, -0x40,0x96,0x57,0x2a,0x35,0xaa,0x2d,0x2f,0x4a,0x79,0xb3,0x58, -0x97,0x51,0x5c,0x39,0x0f,0x7b,0x3e,0x82,0xd0,0xb9,0x04,0x77, -0x9e,0x86,0xa1,0x87,0x8b,0x0c,0x86,0x66,0x6c,0x7a,0x2e,0x68, -0xa8,0x5e,0x69,0x1c,0x71,0x7b,0xee,0xee,0x3c,0x62,0x6b,0xfc, -0x61,0x0e,0x0b,0x63,0x50,0x3b,0x1f,0x82,0xb5,0xef,0x7f,0x11, -0x23,0xcb,0xfa,0x86,0xd6,0xf6,0x62,0xbe,0x50,0x8c,0x24,0x78, -0x98,0x3f,0x54,0x05,0xad,0x98,0x28,0x41,0x00,0xae,0x44,0x3b, -0x90,0x12,0x36,0x6a,0x66,0x70,0x14,0xf9,0xc0,0xfd,0x5a,0xfe, -0x10,0xb5,0xf2,0x82,0x92,0x69,0xba,0xee,0x38,0xbf,0x67,0xf2, -0x8f,0x8f,0xbd,0xca,0x0a,0xc2,0x1a,0x24,0xbe,0x76,0x18,0xd6, -0x4e,0x81,0x8a,0x09,0xf6,0xbe,0x20,0x86,0x9a,0x7b,0xff,0x87, -0x8a,0x6b,0x51,0x4c,0xeb,0xe5,0x93,0x25,0xf8,0xf7,0xc8,0x48, -0x41,0x04,0xb8,0x37,0x8a,0x22,0x52,0x23,0x13,0xac,0xc4,0x38, -0xe1,0xb7,0x22,0x77,0x25,0x59,0x0f,0x14,0xa1,0x43,0x78,0x5e, -0x92,0xcb,0x9a,0x2e,0xe8,0x95,0xbb,0x40,0xd8,0x6d,0x77,0x81, -0x2d,0x82,0x5a,0x91,0x2d,0x83,0x4c,0x63,0x0a,0x96,0xec,0x64, -0xb0,0x04,0x11,0xa2,0xf1,0x2c,0xaa,0xcd,0x54,0xaa,0x2f,0xb9, -0x2f,0x31,0x91,0xde,0x27,0x44,0x2d,0x05,0xad,0xc2,0x32,0xe7, -0x4b,0xf9,0x11,0x2f,0x81,0xd2,0xf3,0x25,0xbb,0x1b,0x10,0xd2, -0xa5,0x39,0xd5,0xcb,0x4e,0xe2,0x2c,0x58,0x9f,0xc4,0xeb,0x40, -0x96,0x74,0x68,0xee,0xaf,0x4f,0x10,0xec,0x84,0xa6,0xa7,0xd3, -0x2c,0xc8,0x4e,0x29,0x25,0x9d,0x71,0x0a,0x98,0x53,0x40,0xce, -0xb6,0xec,0x0b,0x6f,0x09,0xe8,0x15,0x68,0x79,0x91,0x8f,0x79, -0x99,0x70,0x30,0x1f,0x89,0xb5,0x18,0x0b,0x81,0x7e,0xe4,0xc1, -0x6d,0xbc,0xee,0x2f,0x31,0x2c,0x9d,0x1e,0x69,0xa7,0x6b,0x30, -0xc0,0x50,0xfe,0xd6,0x71,0x2c,0x1d,0xc2,0x8c,0xf2,0x08,0xf9, -0xdd,0xdd,0xed,0xb9,0x1a,0x50,0x74,0xd5,0x43,0x6b,0x1f,0x7d, -0x85,0x9f,0x99,0x82,0xe1,0x07,0x62,0xbe,0xb4,0x9f,0xc3,0xbf, -0x93,0x3c,0xbe,0x45,0xa7,0xf6,0x79,0x5b,0xbd,0x64,0xa2,0x87, -0xe8,0xa6,0x29,0xf5,0xb4,0x36,0xc1,0xad,0x50,0x33,0x67,0x5a, -0x6e,0x81,0x16,0xbf,0x37,0x6f,0xff,0x86,0x67,0x9e,0x58,0xb4, -0x19,0x97,0x8a,0x70,0x4b,0x9a,0x80,0x5e,0x48,0x69,0xc5,0xb7, -0xfc,0x2e,0x49,0x15,0xd5,0x1b,0x9b,0xea,0xcd,0x5e,0xaa,0x37, -0x80,0x36,0x27,0xaa,0xf3,0x02,0xd5,0x79,0x53,0x55,0x97,0xef, -0xa0,0x3a,0xa2,0xa6,0x0b,0xd5,0x76,0x11,0x69,0x86,0xa6,0x7a, -0x83,0x54,0x6f,0x9d,0x03,0x3a,0x72,0x62,0xb2,0x77,0x11,0x58, -0x9e,0x93,0xec,0xee,0xce,0xbe,0x11,0x53,0xd8,0x2e,0x55,0x4b, -0x7d,0x5e,0xdf,0x87,0x81,0x5e,0xe1,0xd3,0x8f,0x89,0x8e,0x25, -0x21,0xdf,0x3a,0x9e,0x44,0xa0,0xb0,0x1f,0x1f,0x97,0xae,0x75, -0xa8,0x7d,0x4c,0xd2,0x1b,0x03,0xfc,0x50,0xc5,0x37,0x27,0x82, -0x86,0xcd,0x47,0x51,0x1a,0x9e,0xc2,0xd5,0x57,0x43,0xb0,0xab, -0xd5,0xa1,0xc2,0x85,0x19,0x39,0x3d,0xcb,0xc9,0xa0,0x71,0xf9, -0xe2,0x8e,0xe4,0xec,0x5b,0xdd,0x06,0xc4,0x7c,0xee,0x1e,0xfa, -0x69,0x1b,0x94,0x6a,0x9d,0x47,0xd1,0x86,0xe0,0xa3,0x60,0x85, -0x02,0x8f,0x5f,0x8c,0x26,0x3f,0x62,0x26,0x53,0x84,0x5d,0x46, -0x71,0x3c,0xcd,0x2c,0x16,0xd0,0xf0,0xa3,0xf3,0xbb,0xa8,0x69, -0x82,0x82,0x3c,0x32,0xa3,0x59,0x30,0xdd,0xff,0x6a,0x81,0x14, -0xf4,0xad,0xdd,0x2e,0xc4,0x53,0x15,0x2d,0xca,0xda,0x5f,0x9a, -0x7c,0x5b,0x5e,0xe4,0x2d,0x62,0xf5,0xf6,0x02,0xe3,0xe0,0xf3, -0x08,0x63,0x46,0x98,0x5b,0x64,0x64,0xb9,0x2e,0xfa,0x75,0x7c, -0x60,0x9c,0xdd,0xaa,0x6c,0x5e,0xab,0x95,0xce,0xcd,0x17,0xfd, -0xc1,0x22,0xa0,0xff,0x0d,0xbb,0x0b,0xdf,0xbe,0x1c,0x32,0x79, -0x36,0xa7,0x40,0xba,0x2b,0x37,0x86,0x51,0x3d,0xee,0x8d,0xbf, -0xdd,0xf5,0x92,0x8c,0xf0,0x85,0x63,0x52,0x42,0xe5,0x66,0xa7, -0x62,0xa2,0x2c,0x72,0x1d,0x3b,0xcb,0xaf,0x6d,0xa1,0xb3,0x73, -0xe6,0xd1,0x63,0x9d,0xca,0x03,0xc0,0xef,0x5b,0xdc,0xed,0x9a, -0x96,0x12,0x00,0xfa,0x5a,0x0e,0x36,0x01,0xfd,0x8f,0x04,0x4b, -0xb5,0x8e,0x36,0x0f,0xd4,0x87,0xea,0x44,0xd3,0x9b,0x2c,0x9e, -0x87,0xb9,0x3d,0x10,0xe8,0xa6,0x6f,0x64,0xb9,0x2b,0x2e,0x1e, -0x5a,0x52,0x28,0xf5,0x48,0x3c,0x3f,0x4c,0x25,0x22,0xbd,0x07, -0xba,0xd2,0x13,0xec,0x26,0x30,0x63,0x0f,0x07,0x80,0x11,0x46, -0xca,0xb7,0x1a,0x40,0xf2,0x7f,0xa0,0xdf,0x06,0x82,0x5a,0x82, -0x4d,0x09,0x06,0x15,0x0d,0xad,0x83,0x30,0x11,0xe4,0xa6,0xd9, -0x68,0x18,0x5c,0x98,0xb4,0x6f,0xb7,0x16,0x45,0xbf,0xb0,0xc6, -0xe1,0x78,0x0a,0x31,0x2c,0x8e,0x61,0x7a,0x9d,0x9a,0x25,0xf3, -0xff,0xfe,0x47,0xf0,0xcf,0xff,0xa9,0xde,0x04,0x2b,0x12,0xa0, -0x44,0xab,0x73,0x8e,0x07,0xc6,0xff,0xfc,0x9f,0x31,0xd8,0x96, -0x11,0xfd,0xfe,0xdf,0xff,0x40,0x7b,0xcf,0x5a,0x29,0x71,0x90, -0x08,0x50,0xe9,0xbd,0xc9,0xfc,0xc1,0x26,0xee,0x9c,0x38,0x51, -0x1c,0x60,0x0e,0xce,0x2b,0xd2,0x96,0x2a,0x04,0x51,0xef,0x03, -0x98,0x2d,0x89,0x77,0xb4,0xbc,0xbb,0x5b,0x9e,0x47,0x7a,0x98, -0xfe,0xef,0x7f,0x34,0xff,0xf9,0x3f,0x4f,0x36,0x27,0x46,0xdf, -0xb4,0x04,0xc9,0x74,0xfe,0xd6,0x5b,0xfa,0xa7,0x4b,0xb0,0x6a, -0x02,0xf8,0xfd,0x63,0xa1,0x55,0x4c,0x8f,0xb4,0x59,0xb7,0xbe, -0xcd,0x8d,0x97,0x7b,0x3a,0x93,0x33,0xeb,0x96,0x4e,0xaf,0x18, -0x59,0xcc,0x7d,0x2f,0x2e,0xd9,0x2c,0x3c,0x45,0x37,0x56,0x99, -0x2e,0x3f,0xa4,0xb3,0xef,0x40,0xd3,0x98,0x2a,0x6f,0xca,0xd1, -0xc9,0xac,0x63,0x6c,0x79,0x9b,0x59,0x3d,0xe6,0x98,0x35,0x63, -0x87,0x17,0xda,0x22,0x3a,0xbd,0xd9,0x72,0x9c,0x51,0x9a,0x66, -0xf0,0x8e,0x85,0x7a,0x3d,0x1e,0xbf,0x2f,0x66,0x60,0xc6,0x9a, -0xe8,0xab,0x57,0xd3,0xf9,0x7c,0xe9,0x84,0x57,0x1e,0x27,0xe9, -0xd4,0x4e,0xa0,0x2b,0x1a,0x54,0xb4,0xcd,0x81,0xad,0xab,0xc3, -0x61,0x33,0x72,0x86,0xf0,0xf0,0x19,0xf9,0x61,0xc0,0xff,0xba, -0xe1,0x5a,0x71,0x6d,0x61,0x88,0xc3,0x3e,0xd8,0x89,0x0e,0xba, -0xa8,0x50,0x3b,0xac,0x5b,0xa0,0xb9,0x60,0xc8,0x72,0x25,0x20, -0x4c,0x21,0x2c,0x27,0x02,0x9c,0xa4,0xc2,0xa1,0x5f,0x68,0x2b, -0x94,0xff,0x16,0xfb,0x8b,0x22,0x88,0x5d,0xd1,0x2f,0xaf,0xd1, -0x0e,0x93,0x06,0x6d,0x43,0x6b,0x98,0x1f,0x81,0x25,0x3f,0x26, -0x6f,0x61,0x84,0xdd,0x26,0x8a,0x25,0xe5,0xb1,0xd5,0x22,0x76, -0x23,0x7b,0xbc,0x16,0xe8,0x57,0x66,0x54,0x1d,0x18,0x21,0x21, -0x38,0x8b,0x7f,0x0b,0x58,0x3a,0xec,0x68,0x2d,0x64,0x97,0x5b, -0x1b,0x44,0xcc,0x47,0x9b,0x6b,0x28,0xc6,0x95,0x88,0x4c,0x3d, -0xc0,0x26,0x10,0xb8,0xf9,0x0c,0x9f,0xc8,0x6a,0x3f,0xb2,0x0b, -0xf3,0x8b,0xcc,0xd3,0x5d,0x84,0x08,0x88,0x07,0x0b,0xf5,0x8f, -0xab,0x6e,0x83,0x5c,0xb3,0x1a,0xc1,0xa8,0x3b,0x18,0x0c,0xf1, -0x4e,0x8b,0xc1,0x35,0x86,0x1e,0x7d,0x3b,0x5f,0xe6,0x51,0xb8, -0x8b,0x1c,0x85,0xce,0x57,0x6e,0xd2,0x67,0x17,0xa6,0x60,0x15, -0xba,0xce,0xa7,0xa2,0xeb,0xd4,0xa0,0xbb,0xfc,0x64,0x7c,0x97, -0xd5,0x08,0xc7,0x9f,0x8c,0x70,0x6c,0x64,0xd4,0xe2,0xde,0xa0, -0x13,0x3e,0xee,0x74,0x1e,0x43,0x47,0x3f,0x7e,0xf2,0xf0,0x21, -0x7c,0x04,0x9d,0xa7,0x9d,0x87,0x8f,0x3e,0xff,0x3c,0x88,0x1e, -0x86,0x67,0x0f,0x9f,0x74,0x3e,0x0f,0x9e,0x3e,0xe9,0xc0,0xf7, -0xa3,0xe0,0x69,0xf4,0xf4,0x49,0x74,0x06,0xe2,0xfe,0xf0,0x69, -0xf8,0xe4,0xec,0x2c,0x0c,0x3e,0x3f,0x7b,0xf4,0xe8,0x49,0x04, -0x09,0x9d,0xb3,0xc7,0x4f,0xa3,0xcf,0x1f,0x07,0xd1,0xa3,0x47, -0x8f,0xa3,0x27,0x8f,0x86,0xca,0x7b,0x7f,0xf9,0xf6,0x92,0xe3, -0x3e,0x93,0xd4,0x5b,0x4c,0x36,0xb5,0x46,0x0f,0x1f,0x3d,0x7c, -0xf8,0xd0,0xaa,0x1e,0x7e,0x3c,0x7a,0xfa,0xf8,0x49,0xc7,0xd0, -0x11,0x86,0x80,0xfb,0xa9,0x45,0x10,0x82,0x84,0x4f,0x1e,0x76, -0x34,0x65,0x51,0xe7,0xe9,0xa3,0x47,0x9f,0x3f,0x8e,0x0c,0x89, -0x1d,0x48,0x8e,0x10,0xad,0xa2,0x35,0x7a,0xdc,0x39,0x7b,0x18, -0x3d,0x7d,0x68,0x88,0x3e,0x0b,0x9f,0x76,0x3e,0x0f,0x9f,0x58, -0xd4,0x03,0x1d,0x9f,0x3f,0xfa,0xfc,0xa9,0x6a,0x06,0xd4,0xfc, -0x30,0x0a,0xa1,0x8a,0xbd,0xed,0xb9,0x8c,0x07,0x67,0x48,0xd2, -0x93,0x27,0xc1,0xa3,0xb3,0xa7,0x50,0xf8,0x69,0xf0,0x24,0x7c, -0x18,0x3e,0x81,0xbf,0x51,0xd8,0x79,0xfc,0xb0,0x03,0xd8,0x1f, -0x3f,0x8a,0x9e,0x86,0x4f,0x9f,0x04,0x4f,0x3b,0xd1,0xe3,0x47, -0x40,0x47,0x14,0x3d,0x8d,0x1e,0x45,0x11,0xd4,0x02,0x04,0x3d, -0x7e,0x7c,0xf6,0x18,0x72,0xc2,0xce,0xd3,0xa7,0x67,0x40,0xd0, -0xc3,0xce,0xe3,0xcf,0x43,0xa4,0x03,0xc8,0x82,0x66,0xe0,0x0f, -0x28,0x7b,0xd6,0x09,0x3e,0x7f,0xfc,0xf8,0x71,0xf4,0x18,0x72, -0xa0,0xd0,0x63,0x68,0x1a,0xf2,0xe4,0xe1,0xd3,0xc7,0x0f,0xcf, -0x10,0xe4,0xc9,0x93,0x33,0x60,0x1b,0x34,0xe9,0xc9,0xe3,0xcf, -0x21,0x21,0xfc,0xfc,0xec,0xe1,0x59,0xe7,0x21,0xe2,0xef,0x7c, -0xfe,0x08,0xd1,0x01,0x3b,0x9f,0x3e,0x7e,0xf4,0x70,0x6f,0x73, -0xc6,0xd0,0x9c,0xce,0xc3,0x87,0xc0,0x37,0x68,0x06,0x32,0x2b, -0x44,0x56,0x84,0x8f,0x1f,0x77,0x90,0xda,0x10,0xf2,0x90,0xc5, -0x80,0xf2,0x51,0xf8,0x08,0x2a,0x42,0x92,0x9e,0x3c,0x7e,0x88, -0x3f,0x3a,0x8f,0x3a,0x8f,0x9e,0xe2,0x8f,0xb3,0x87,0xd0,0x69, -0xc1,0x59,0xe7,0xd1,0x59,0xf8,0xe4,0x71,0xf0,0xe4,0xf1,0xa3, -0xce,0xc3,0x27,0x88,0xe5,0x71,0xf8,0x24,0x0c,0xa1,0xcc,0x59, -0x14,0x02,0x82,0x87,0x90,0xf3,0x14,0xa8,0x43,0x15,0x7a,0xf6, -0x14,0xb0,0x76,0xb8,0x37,0xb0,0x1d,0xd1,0xc3,0x33,0xe0,0xdb, -0xd3,0x08,0x19,0x77,0x06,0xbc,0x0d,0x9e,0x7e,0xfe,0xf8,0x73, -0xa8,0x11,0xd9,0x73,0xf6,0xf0,0x73,0x60,0x75,0xf4,0xb0,0xf3, -0xe4,0x49,0xf8,0x79,0x54,0xd1,0x1e,0xa3,0xf9,0x60,0x09,0x00, -0x93,0x49,0x65,0x18,0x6e,0xad,0x75,0x14,0x0c,0xaa,0x9d,0x81, -0xa3,0xef,0x38,0xe7,0xb7,0x03,0x4a,0xff,0x96,0x78,0x61,0x95, -0xc6,0xd4,0x4e,0xc3,0xbb,0x50,0x68,0x20,0x6f,0xd0,0x7e,0x34, -0x2c,0x23,0x51,0x33,0xe2,0x2e,0x1c,0x0a,0xc6,0xb3,0x4b,0xdf, -0xbe,0x85,0xf5,0xec,0xb8,0x72,0x53,0x18,0xd2,0x95,0xa5,0x11, -0xc6,0xe4,0xdf,0xfa,0x33,0x1d,0xc5,0x55,0x5f,0x04,0x0e,0x96, -0x51,0x0c,0x06,0xdb,0x1e,0xa0,0x51,0x18,0xef,0xbb,0x9c,0xdc, -0x9c,0xf0,0xd3,0x3c,0x50,0x3d,0xfb,0xbb,0x63,0xf4,0xe8,0xdd, -0x97,0x90,0xcb,0x45,0x26,0xa3,0xd8,0x1b,0x85,0xe7,0xa3,0x08, -0x5d,0xbe,0x46,0xf1,0x28,0xc4,0x9a,0x47,0x11,0xa2,0x9a,0x8c, -0x80,0x8c,0xa8,0x35,0x02,0x03,0x6f,0x72,0x15,0xff,0xf3,0xdf, -0xce,0x27,0xa3,0x7e,0x23,0x6c,0x74,0x1b,0x51,0x23,0x18,0x87, -0x6c,0xb8,0x8c,0xe7,0x99,0x87,0x00,0x99,0x7c,0x67,0x30,0xa7, -0xe3,0xf7,0x38,0xb2,0xf2,0xc1,0x5a,0xcc,0x22,0x2b,0x3f,0xb2, -0xee,0xfd,0x9d,0xc7,0x86,0xa4,0x1f,0x46,0x9b,0xfe,0x32,0xec, -0x37,0x7e,0x08,0x83,0x46,0x73,0x19,0x35,0x1b,0x5f,0xf2,0x1f, -0xf9,0x7a,0x10,0x3e,0x00,0xe9,0x7e,0x80,0x99,0xad,0x9d,0xb9, -0xf8,0xc9,0x38,0x42,0x86,0x0a,0x19,0x2a,0x14,0xa8,0x50,0x70, -0xec,0xca,0xc5,0xcf,0xbf,0x42,0x5b,0xff,0x08,0x5a,0x00,0x0f, -0x35,0x0b,0xbf,0x4e,0xc6,0x52,0x5d,0x74,0x92,0x85,0x15,0xe5, -0x1a,0xcd,0xc9,0x15,0x7c,0x47,0x0f,0x18,0x38,0xd2,0xc0,0xf0, -0xeb,0x7b,0x24,0x4b,0xa7,0x85,0x94,0x56,0x6a,0x80,0x20,0xc0, -0x1f,0x08,0x1c,0x6a,0x60,0xd5,0xa0,0xdf,0x4d,0x45,0x18,0x84, -0x0d,0xc4,0xf5,0x2f,0x1c,0xb7,0xcc,0xc8,0xbb,0xd5,0x95,0xcf, -0x4d,0x6a,0x60,0x09,0xbb,0x05,0xf1,0x93,0x49,0xb5,0x6f,0x69, -0x1b,0x80,0x97,0x3a,0xd1,0x5c,0xdc,0x36,0xb9,0xdf,0x48,0x52, -0x4f,0xaf,0x99,0xc6,0x6d,0x9b,0x14,0x73,0xdf,0x77,0xc7,0xb2, -0xaa,0x40,0xbb,0xf2,0x58,0xbf,0xa5,0xbb,0xe6,0x7e,0xd7,0xca, -0xde,0x62,0x4a,0xdb,0x6e,0xc9,0x41,0x15,0x14,0x9a,0x5e,0xac, -0xc0,0xca,0xe6,0x0a,0xaa,0xee,0x81,0xef,0xc2,0xef,0x32,0xae, -0x88,0xde,0xbe,0x13,0x8e,0xd8,0xcb,0x77,0xc2,0x77,0xe1,0xb6, -0x99,0x5e,0xc4,0x6c,0xee,0x87,0x23,0xde,0x31,0x94,0x5b,0xce, -0x53,0x47,0x2f,0x93,0x36,0x8c,0xbd,0xbd,0xda,0xb0,0xb9,0x57, -0x15,0xe2,0xc9,0x33,0x28,0xa9,0xbd,0xda,0x70,0xb7,0xe6,0x03, -0x2c,0x25,0xe5,0x27,0xb2,0x33,0x30,0x9a,0xca,0x3f,0x59,0x06, -0x46,0x4f,0xc1,0xd7,0x90,0x5a,0x68,0x56,0x24,0x76,0xf1,0xb8, -0x75,0x3d,0x9a,0x5e,0xfd,0xf3,0xdf,0x02,0x47,0x83,0xc5,0xff, -0xfc,0x5f,0xad,0x7f,0xff,0x47,0x4f,0xa6,0x0c,0x5c,0x01,0x90, -0xd3,0x87,0x6b,0xac,0xd3,0xa1,0x3d,0x64,0xd9,0xaf,0xab,0x79, -0xd6,0xef,0x9f,0xa9,0x84,0x56,0x93,0x1a,0x0b,0x88,0x11,0xfd, -0xde,0x28,0xbf,0x3a,0xf9,0xa2,0x3f,0xfc,0xec,0x39,0xfd,0xcc, -0x9d,0xfc,0x9c,0xf2,0xb7,0x81,0x55,0xc1,0xcb,0x55,0xb2,0xf8, -0x2a,0x81,0xd5,0x54,0xa2,0xee,0x38,0xc6,0x85,0x5c,0x3a,0xae, -0x2f,0x16,0xa1,0xc4,0xba,0x12,0x8c,0xcf,0x4c,0x96,0xc9,0xa8, -0x66,0x92,0x25,0x5e,0x25,0x23,0x27,0x8e,0xd9,0xd6,0x2e,0x57, -0xc3,0x31,0x42,0x59,0x40,0x50,0xc3,0x31,0x8d,0x05,0x38,0x46, -0xbf,0x35,0xc7,0xe4,0x8b,0xfe,0xd8,0xe3,0x9f,0xd3,0x43,0x17, -0x2c,0x64,0x38,0x7b,0x18,0x73,0x46,0xe4,0xc2,0x45,0x0c,0xc7, -0x3d,0x40,0x3f,0x73,0x07,0x20,0xe7,0x7c,0x6b,0xb8,0x72,0xb2, -0x5b,0x5d,0x2e,0xd5,0xe9,0x91,0xc7,0xa9,0x6e,0x5d,0x39,0xd7, -0xa5,0xf9,0x55,0xff,0x02,0x09,0xe5,0x18,0x47,0x9b,0x4c,0x3f, -0xba,0xc0,0x63,0x43,0xc2,0x43,0x93,0x63,0xcd,0xaa,0x90,0x27, -0xf1,0xa2,0xad,0x20,0x35,0x38,0x6f,0x64,0xed,0x45,0xd8,0x44, -0x63,0x27,0x6b,0x2f,0x03,0xf8,0x88,0xe0,0x1f,0x30,0x10,0xe0, -0x9f,0xd0,0x6f,0x7a,0x09,0x58,0x6f,0xd3,0xcc,0x83,0xc2,0x7e, -0x7f,0xbc,0x5e,0xde,0x26,0x36,0x18,0xfd,0x08,0xfd,0x6e,0x21, -0x63,0x05,0x3f,0x56,0x98,0x41,0x68,0xe5,0x0b,0x93,0x01,0xed, -0x8a,0xd0,0x72,0x01,0x9d,0xa3,0x31,0xf9,0x38,0x03,0x59,0x87, -0x89,0x8a,0xfc,0x2c,0x99,0x5e,0x05,0x57,0x81,0x69,0xb5,0x64, -0xc4,0x57,0x12,0xa9,0x00,0xf3,0xd9,0x99,0x48,0xbc,0x08,0xad, -0x74,0x05,0x8c,0xd9,0xae,0x01,0x56,0x09,0xb3,0xdb,0x00,0x3b, -0xac,0x88,0xb0,0xf7,0xe3,0xb2,0xbb,0x84,0x2a,0xf1,0xfd,0x8e, -0x51,0x84,0x2f,0x77,0x2c,0xc2,0xee,0x60,0x79,0xe2,0xd8,0x58, -0xf2,0x29,0x26,0x16,0xbe,0xab,0xef,0x82,0x44,0x2e,0x48,0xe4, -0x0f,0xad,0x33,0x14,0xe9,0x1d,0xfb,0x36,0x37,0xc8,0x6b,0x18, -0xc7,0x97,0xf0,0xef,0xf1,0x31,0xfc,0x8e,0xe8,0x77,0xb4,0x75, -0xcc,0xda,0x65,0xb0,0x08,0xf4,0x25,0x69,0xb2,0x13,0xd8,0x14, -0x10,0x2b,0xe1,0x41,0xd3,0x1b,0x9d,0xff,0xf3,0xdf,0x7c,0xb1, -0x15,0x16,0xa6,0x30,0xf7,0xdb,0x32,0x84,0x86,0x80,0xc1,0x0b, -0xb4,0x6a,0x24,0xff,0x0f,0xd8,0x45,0x58,0x76,0x11,0x6d,0xad, -0xa7,0x42,0x90,0x2f,0xf6,0xb3,0x1c,0x98,0x20,0x52,0xb8,0x74, -0x4c,0x07,0x92,0xea,0x3f,0xce,0x6e,0xe0,0xc7,0x0b,0x8a,0x6e, -0x79,0xbb,0xe6,0xc5,0x65,0xd5,0x74,0x2e,0x0f,0x2a,0x28,0x17, -0x3c,0xf5,0x66,0xc3,0xda,0x79,0xce,0x60,0xe7,0x4c,0xae,0xd9, -0x50,0xc6,0x2a,0xef,0x97,0x08,0x56,0xe1,0xcf,0x41,0x58,0x0d, -0x2f,0xcb,0x58,0xe5,0xdd,0x13,0x45,0xeb,0x1f,0x65,0x79,0x28, -0x9a,0x2d,0xdb,0x83,0x6b,0xf8,0x43,0xac,0x0f,0xc1,0x6e,0xec, -0x0f,0xf3,0x84,0x04,0x2a,0x43,0x98,0x03,0xde,0xce,0x0b,0x1b, -0x63,0x5a,0x24,0x55,0xa6,0xe5,0x7b,0x08,0x63,0x9c,0x23,0xcf, -0x16,0x1c,0x0d,0x17,0x67,0xb1,0x04,0xa0,0x2d,0x64,0xdc,0xc4, -0xde,0x22,0x6c,0xe7,0xcd,0xc5,0x59,0x3b,0x47,0xc3,0x64,0x11, -0x0f,0x40,0xc0,0x3f,0x6e,0xba,0x90,0xba,0x09,0xf2,0xee,0xcd, -0x96,0x3e,0xce,0xd4,0xc7,0xe2,0xcc,0x78,0x90,0xc6,0x0b,0x5a, -0x5d,0x2f,0x96,0x73,0x7c,0xe9,0x10,0x28,0xf2,0x03,0xd4,0xaa, -0x8b,0x41,0x38,0x6c,0x36,0x9e,0xe1,0x8f,0x08,0x7e,0xe0,0xc8, -0x18,0x74,0xd4,0x8f,0xb3,0xe1,0x81,0x83,0xc4,0x60,0x55,0xc2, -0xae,0x9a,0xfb,0x42,0xe7,0xe8,0x55,0x97,0xe4,0x94,0x04,0x74, -0x77,0xe0,0xa2,0xa2,0x80,0xe2,0x6b,0xc3,0x82,0xc9,0xc8,0xa8, -0xc6,0x5d,0x14,0xd3,0x9d,0xb1,0xb4,0x4b,0x62,0xea,0xe0,0x56, -0x92,0xaa,0x71,0x5b,0x8d,0x3d,0x08,0xbf,0x05,0xbf,0xb1,0xf0, -0x9a,0x64,0x83,0xbb,0x24,0x4a,0x35,0xb6,0x88,0x96,0xb4,0x02, -0xb8,0xe7,0x1f,0xd4,0x15,0x16,0xd0,0x45,0x5c,0xd1,0xae,0x52, -0x57,0xdd,0xb3,0xc9,0x16,0x7a,0xaf,0x40,0x04,0x5b,0x4a,0x86, -0x14,0x87,0x3b,0xfe,0x1e,0xae,0x64,0xf9,0xcd,0xe5,0xbc,0x7a, -0x78,0x71,0x96,0x1d,0xb8,0x49,0xd5,0xcc,0x39,0x1c,0xfb,0x85, -0xde,0xb0,0x73,0x07,0x15,0x47,0x83,0x31,0x80,0xcf,0xe8,0x79, -0x4a,0x9f,0x8e,0x85,0x8b,0xa0,0x5b,0x75,0xda,0x19,0x3b,0x45, -0x5e,0x41,0x0a,0x1f,0x74,0x3b,0xc9,0xe8,0x11,0xa8,0x8f,0x54, -0x28,0xa5,0x4d,0x65,0x0f,0x13,0x4a,0xa9,0xc5,0x12,0x49,0xc6, -0x01,0x02,0x09,0x59,0x5b,0xf9,0xba,0xef,0x39,0x77,0x35,0x46, -0x3e,0xf2,0xe6,0x2f,0x67,0xb5,0x21,0xbc,0xd3,0xf1,0x43,0xe4, -0x71,0xcf,0x0a,0x86,0x05,0xe3,0xe5,0x3c,0xb3,0x2e,0x3a,0x70, -0xe0,0x76,0x5e,0x87,0x99,0x83,0x33,0x4c,0x3d,0x7d,0x64,0x3c, -0x75,0x51,0x03,0xb5,0xce,0x4e,0x78,0x5a,0x6f,0xc1,0x9f,0xef, -0xf8,0xcf,0x2f,0x2a,0xf9,0x3b,0x9a,0xed,0x7f,0xd1,0x99,0x94, -0xf8,0x0b,0x25,0xea,0x2c,0x05,0xd8,0xb2,0xb3,0xb8,0x38,0x18, -0x69,0x28,0x45,0x37,0x73,0x3b,0x22,0x85,0x45,0x5a,0x5e,0xa4, -0x0d,0x23,0x6a,0x38,0x5d,0x3b,0x9a,0x9d,0x85,0xf8,0x2c,0xfc, -0x26,0x5e,0xe6,0x15,0x59,0xba,0x1d,0xbc,0x17,0x93,0xf3,0x36, -0xc9,0x06,0xb7,0x40,0x1a,0x68,0xa5,0xa0,0xb5,0x92,0x93,0x36, -0x6d,0xe9,0x54,0x24,0x8a,0x3c,0x1a,0x92,0x43,0xd8,0x55,0x60, -0x96,0x61,0xd5,0xf7,0xca,0x1c,0xc2,0x8f,0x07,0xb6,0x6d,0xa4, -0xc1,0x84,0x03,0x8d,0xd5,0x32,0xa5,0xc5,0x41,0x6b,0x32,0xff, -0x60,0x3f,0x64,0x6a,0xd5,0xba,0x29,0x72,0xc2,0x15,0x65,0x48, -0x3e,0x03,0x2e,0xe4,0xf1,0x72,0x73,0x52,0xce,0xb1,0x48,0x54, -0x2d,0x56,0x6c,0x10,0xae,0x68,0x06,0xc8,0xb7,0x4b,0xd6,0x7a, -0xf1,0x1f,0x4f,0x54,0xcb,0xa5,0xaa,0x40,0x94,0xd0,0xc4,0xf1, -0x3a,0x8c,0xaa,0xc1,0x91,0x9d,0xc5,0x05,0x45,0xf2,0x3e,0xc9, -0x33,0xcf,0x77,0xde,0xa3,0x24,0xef,0xaf,0xe7,0xb3,0xeb,0x64, -0x99,0xae,0x9e,0x4f,0x82,0xaa,0xe4,0xa0,0x4c,0xa1,0xd5,0xba, -0x33,0x3f,0x28,0x0b,0x17,0xe7,0xaf,0x46,0x33,0xef,0x2c,0xc4, -0x06,0xd2,0x5c,0x30,0xe3,0x0b,0x30,0x06,0xff,0x0b,0xf5,0x38, -0x27,0x7a,0xc5,0xba,0x35,0x4f,0xf8,0x8e,0x42,0x05,0x2c,0x69, -0x36,0x6a,0x58,0x32,0x65,0x75,0xeb,0x66,0xd5,0x54,0xd1,0x4e, -0x6e,0x16,0xab,0xbc,0xa6,0x1c,0xe5,0xd5,0x15,0x44,0x9f,0x92, -0x9a,0x72,0x1c,0x8b,0xb8,0xba,0x98,0xd6,0xaa,0xe5,0x62,0x98, -0xa5,0x5f,0x70,0x2d,0xc4,0xa9,0xd5,0xe0,0xf5,0xfa,0xb0,0xba, -0xeb,0xfa,0xba,0xa4,0x85,0xd7,0x43,0x5f,0x75,0x49,0xee,0xda, -0xc4,0xfc,0x8c,0x8e,0x56,0x36,0xdc,0xd6,0x25,0xc8,0x3c,0x9c, -0x1a,0x57,0xb7,0xaf,0xb6,0xd9,0x54,0x43,0xb1,0x3d,0x78,0x4d, -0xdd,0x5e,0x55,0xe2,0xd2,0x4e,0xfb,0x16,0x05,0xe9,0x24,0xa6, -0x30,0xe8,0xe9,0xc4,0x7a,0xe3,0x73,0x30,0xec,0xa9,0xb2,0x0e, -0x1b,0x2f,0x54,0xaa,0x41,0xdd,0x53,0x6e,0x02,0xf2,0x9a,0x06, -0x21,0x73,0x1f,0xd2,0xf8,0xa8,0x11,0xcb,0x53,0x88,0x6a,0x79, -0x3b,0x18,0x9a,0xe2,0x9c,0x82,0xa5,0x31,0x5e,0x96,0x78,0x3e, -0x52,0x62,0xd1,0x69,0x93,0x64,0x82,0x72,0xd0,0x03,0x1d,0x1d, -0xd0,0xb8,0x3d,0xb1,0xa2,0x89,0xe7,0x5f,0xe3,0x3e,0x75,0x71, -0x81,0x31,0x59,0x2e,0x2e,0xf0,0x21,0x53,0xdf,0xef,0x7b,0x0d, -0x95,0xd0,0x48,0x39,0x26,0xb7,0xc1,0xa1,0x61,0x63,0xe7,0xab, -0x30,0x76,0xf0,0x2e,0x8b,0x2a,0x01,0x48,0x81,0x75,0x02,0x6d, -0x40,0x2e,0x2e,0x06,0xe9,0x64,0x68,0x9e,0x25,0xd5,0x2d,0x27, -0x3f,0x2a,0x90,0x0c,0x27,0x9d,0x6e,0x5a,0x58,0x97,0xcd,0x2c, -0xe1,0xd0,0xbc,0x83,0x6a,0xac,0x07,0x76,0x2b,0x7b,0xfe,0xcb, -0xe9,0xf4,0xd0,0xce,0xcf,0xe4,0xba,0xbb,0x34,0xc2,0xe0,0xfc, -0x44,0x81,0xf8,0x92,0xf7,0x0d,0x0e,0x96,0x89,0x7b,0x77,0xbb, -0x84,0x81,0x36,0x1d,0xff,0xd1,0x1a,0xc1,0x35,0xec,0xd7,0x8d, -0x3d,0x4c,0x34,0x82,0x43,0xe4,0xf4,0x3d,0x92,0x38,0x8f,0x15, -0x66,0x43,0xe5,0xfb,0x2f,0xe6,0x20,0x9c,0x5a,0x16,0x25,0x7f, -0xf0,0x1e,0x45,0xb4,0x5e,0x7a,0xde,0xa3,0xf4,0x98,0xbc,0x3a, -0x81,0xf9,0x3d,0xb2,0x71,0x95,0x4e,0x9d,0x8b,0x6d,0xfc,0x5d, -0x10,0x0b,0xfe,0x57,0xc9,0x84,0xe9,0x75,0xe3,0x69,0x77,0xa4, -0x7c,0xec,0xb8,0x3c,0x8c,0x19,0x41,0xec,0xc8,0x03,0xa7,0xa9, -0x3a,0x6c,0x49,0x08,0x0b,0x82,0xa0,0xdc,0x89,0x3e,0x49,0x3b, -0x84,0x45,0x29,0x51,0xb1,0xb8,0x4a,0xda,0x81,0x29,0xa9,0xef, -0x77,0xdf,0xbc,0xd4,0x6b,0x3d,0x56,0xb0,0x8f,0xdf,0x32,0x42, -0xea,0x99,0xbe,0xfa,0x90,0xd8,0x21,0xdf,0xe9,0x01,0x76,0x4a, -0x93,0xa0,0x66,0x7a,0x90,0xf5,0x6a,0x66,0x9a,0x2f,0x3a,0x7d, -0x82,0x40,0x72,0x3c,0xbf,0x42,0xb8,0xb9,0x0a,0x5a,0x9f,0xd0, -0xbb,0xf0,0xce,0x14,0x73,0x91,0xa0,0xfb,0x3c,0x3f,0x31,0x42, -0x6e,0x95,0x04,0x5c,0x79,0x75,0xad,0x16,0xf3,0x32,0xa1,0xe7, -0x16,0x09,0x79,0xf5,0x3d,0xb5,0xda,0xa2,0x99,0x10,0x25,0x4d, -0xde,0xd6,0xf3,0x69,0xb4,0x5a,0x2d,0x5d,0x36,0xfd,0xf8,0x92, -0x63,0x63,0xd9,0x67,0xd5,0x00,0x44,0x9e,0xad,0x30,0xe8,0xe9, -0x21,0x70,0xa2,0xcc,0xbc,0x7a,0xce,0x24,0x96,0xc0,0x7f,0x7c, -0x59,0x53,0x00,0x32,0xf8,0x15,0xfc,0xc5,0x68,0x0c,0x02,0x81, -0x3f,0xa7,0x73,0x90,0x90,0x02,0x8e,0x57,0x48,0xbc,0x67,0xf6, -0x1e,0x99,0x93,0x97,0x7d,0x85,0xbf,0xeb,0x5d,0x36,0xe3,0x46, -0xa3,0xe8,0x49,0x98,0x06,0x23,0xee,0xdc,0xb7,0xf6,0xcb,0xec, -0x44,0xa3,0xee,0xec,0x23,0xc0,0x03,0x23,0x28,0x8d,0x2d,0x0f, -0x35,0xda,0xe7,0x34,0xc8,0x56,0x42,0x7b,0x56,0xc4,0x12,0xa4, -0x90,0x47,0xcf,0x3e,0x57,0x90,0x0b,0x4d,0xab,0x27,0xf8,0xc7, -0x97,0xf7,0x21,0xb9,0x96,0x4b,0xbf,0xb7,0x15,0x75,0x88,0xad, -0x86,0x81,0xb6,0x2f,0x0f,0x08,0xe5,0x92,0x08,0xc2,0xf1,0x00, -0x67,0x6e,0x12,0x16,0x9f,0xd0,0x63,0x0b,0x39,0x27,0xe0,0xe4, -0x01,0x7d,0x98,0x60,0xbd,0x08,0x25,0x47,0xbf,0x9a,0xd4,0xd8, -0x7a,0x2d,0x3f,0x8e,0xb9,0x5c,0x7f,0x52,0xfd,0xc8,0x7b,0xb7, -0xe0,0x4d,0x88,0xd0,0x31,0x3f,0xe4,0x8f,0x9b,0xd4,0x74,0x25, -0x88,0xe9,0xe9,0x55,0xa9,0x8e,0x0b,0x1a,0x08,0x3c,0x20,0x1b, -0x48,0x6c,0xbb,0xd1,0xb4,0x85,0xdd,0xe2,0x42,0xdf,0xea,0xcb, -0xae,0xfe,0xbd,0x7b,0x04,0xbd,0x2a,0x6b,0x1b,0xc0,0x2c,0xfa, -0xa6,0x42,0xa6,0xf5,0x66,0xe2,0x15,0x2b,0x06,0x77,0xc7,0x22, -0xa8,0x91,0x5e,0xe3,0x68,0x0a,0x6a,0x75,0xbf,0x98,0x5e,0x51, -0x6f,0x56,0xcb,0xe8,0x7d,0x09,0xa8,0x95,0xc5,0x83,0x69,0xaa, -0x15,0x3a,0x21,0x93,0x9c,0x88,0xf7,0x76,0x2a,0xa1,0xe5,0xbe, -0x2c,0x74,0xe3,0xa7,0x77,0x60,0xb6,0xca,0xed,0x0d,0x66,0xe2, -0x1d,0x0b,0xc5,0x62,0x99,0xce,0x97,0x78,0xd0,0x68,0x47,0x24, -0x07,0x60,0x5b,0x17,0x52,0x82,0x28,0x38,0xc0,0xb9,0x48,0x96, -0xab,0xbc,0xa8,0x10,0x09,0x86,0x5d,0xa3,0x2a,0xb4,0x83,0x46, -0xb9,0x43,0x39,0x00,0xe9,0x1f,0xd2,0xd9,0x64,0xfe,0x01,0x7b, -0xe5,0xd9,0xfc,0x66,0x01,0x1c,0x9d,0xbc,0xc4,0x82,0x66,0x92, -0xf1,0x31,0x4f,0x91,0xf0,0x0b,0xdf,0x20,0xae,0x56,0x7a,0xee, -0x58,0xaa,0xd3,0x18,0xd4,0xb0,0xcc,0xa0,0x34,0xca,0xcf,0x30, -0x66,0xab,0x5e,0x1a,0xaf,0xf0,0xff,0x06,0x1d,0x72,0x76,0x3e, -0xa3,0x10,0x5d,0x8d,0x8c,0x1a,0x6f,0x6c,0x18,0x22,0xec,0x23, -0xbb,0x86,0x73,0x94,0xc4,0x46,0x83,0xad,0x0d,0x85,0x5a,0x69, -0x18,0xdf,0x10,0xa3,0xf3,0xa8,0xb3,0x07,0xea,0x4b,0x82,0x6b, -0xba,0xaa,0x46,0x65,0x02,0xde,0x4a,0x5b,0xc2,0x51,0x08,0xdc, -0x54,0x16,0x25,0xe9,0x7a,0xab,0xcb,0xf6,0x48,0x4e,0xc5,0xd8, -0xe7,0x91,0x5f,0x2b,0x3e,0x07,0xe9,0x80,0xdf,0xd3,0xe3,0xbb, -0x07,0x65,0x75,0xbf,0x5e,0xb9,0xfd,0xba,0xad,0x33,0x8b,0xce, -0xf0,0xee,0xa1,0x61,0xae,0x1f,0xd8,0x43,0xd2,0x61,0xa4,0x69, -0xeb,0x0e,0x33,0x2d,0xd9,0xd8,0xa7,0x52,0x6c,0x77,0xec,0xed, -0x2f,0x2c,0xd5,0x90,0x8e,0x2a,0x40,0x41,0x4e,0x7d,0x6d,0x3c, -0x4a,0x2b,0x1c,0x0c,0xa8,0x0d,0x64,0xb2,0xe1,0x5b,0xcb,0xd6, -0x56,0x40,0x69,0x30,0x2e,0x7a,0x04,0xeb,0x5a,0x5e,0xed,0xf1, -0x7c,0x3d,0x5b,0x7d,0xd1,0x41,0xd6,0xf0,0x3c,0xce,0x51,0xe4, -0x28,0x18,0xc3,0xf1,0xf1,0x42,0x2a,0xa6,0xb0,0x85,0xd4,0x84, -0x5d,0x16,0x59,0x32,0xca,0x92,0x12,0x4b,0x0e,0xb4,0x59,0xa3, -0x7d,0x36,0x2b,0x22,0xef,0x7a,0x15,0x8b,0x0a,0xfd,0xf2,0x94, -0xbc,0xd4,0x44,0x90,0x55,0x97,0x1d,0x78,0xc7,0xab,0xc2,0xce, -0x3d,0xd0,0x40,0xa5,0xe6,0xf1,0x2b,0x18,0x7e,0x3d,0x13,0x26, -0xf8,0xee,0xfb,0x81,0x5c,0xa8,0xa1,0xa7,0x22,0xf2,0xc9,0xad, -0xfb,0xc8,0x89,0x6c,0x1f,0xbc,0xab,0x27,0x96,0xc9,0xb0,0x5f, -0x8a,0xa9,0x59,0xba,0x6e,0xbb,0x9e,0xfd,0x1e,0xd9,0xa1,0xcc, -0x60,0xfc,0xfb,0xb9,0xb1,0x2e,0x86,0x6d,0xf8,0x3f,0xc6,0x10, -0x45,0x89,0xbe,0x72,0x1b,0x05,0xfb,0xb9,0x63,0xd8,0x53,0x2c, -0xe6,0x1f,0xcc,0x29,0x55,0xf1,0x5e,0x66,0x61,0x83,0x0d,0xa3, -0x68,0xbb,0x6e,0x9a,0x66,0xab,0x64,0xa6,0xd6,0xdc,0x16,0xc3, -0x6a,0xcc,0x5c,0xb6,0x53,0x69,0x6f,0x31,0xae,0xdc,0x14,0x4e, -0xd5,0xc6,0x63,0x65,0xf6,0xf3,0x49,0xaf,0x3a,0x39,0x4e,0x27, -0x07,0x8d,0x1e,0xee,0x83,0x2a,0x1c,0xb5,0x3b,0x2d,0xe6,0x88, -0xac,0x4e,0x42,0xfd,0xca,0xfd,0xed,0x38,0x35,0xfb,0xdc,0x95, -0x14,0xab,0x5f,0x1c,0xb6,0xf4,0xb0,0xb1,0x4f,0xe7,0x6d,0x7b, -0x77,0x87,0x7a,0x9e,0xb5,0xe5,0x4a,0x0f,0x30,0xde,0xdd,0x95, -0x92,0x50,0x15,0x4d,0x52,0x30,0x1a,0x57,0xa8,0x9c,0xe5,0x15, -0x63,0x7e,0xfa,0xdf,0xa7,0xc7,0x20,0xdd,0x0e,0xde,0xba,0x56, -0x40,0xed,0xb4,0x53,0xb1,0xf9,0x6c,0x5d,0xb8,0xaa,0xd8,0x94, -0x71,0x36,0xe8,0x42,0xbd,0x43,0x97,0x4e,0xa2,0xb8,0xd9,0x2c, -0x6c,0xdf,0x3b,0x3b,0x38,0xc1,0xff,0xb1,0x9d,0x17,0x8b,0x97, -0xf1,0x4f,0x97,0x78,0x3c,0xdb,0x1e,0x2f,0x13,0x34,0xf8,0xaa, -0x3b,0x24,0x1c,0xfa,0x56,0x33,0x59,0x37,0x35,0x63,0x3b,0x45, -0xc6,0x60,0xc5,0x66,0xac,0xde,0x89,0x8d,0xec,0xcd,0xb4,0xe2, -0x6e,0xda,0x81,0x5b,0x69,0x91,0xe5,0xd8,0xb7,0x49,0xb3,0x6a, -0xf7,0x79,0xc8,0xf0,0xde,0xfa,0x1f,0xdf,0xf2,0xb4,0x5d,0x98, -0xa5,0xf1,0xfa,0xa5,0x5e,0x3a,0x06,0xf4,0x2c,0x2a,0x8b,0x2d, -0xcf,0xbe,0x7c,0x0f,0x5b,0x7a,0xf1,0xe2,0x02,0x1f,0x76,0x5d, -0x5d,0x5c,0xdc,0xdd,0x51,0x32,0x67,0x46,0x85,0xcc,0x58,0x2e, -0x02,0xcd,0x17,0x78,0x5f,0x1a,0xf1,0xab,0x20,0x5b,0xf8,0x9b, -0x6c,0xaf,0xac,0xcf,0x25,0xe9,0x15,0x6d,0xf7,0x43,0x26,0x52, -0x4e,0xa2,0xd2,0x5f,0x2a,0xb5,0x73,0xe1,0x77,0x05,0x52,0x6e, -0xb7,0xf9,0x5d,0x83,0x91,0x40,0xe5,0x22,0x9b,0xa9,0x8d,0x13, -0x2e,0xec,0x1a,0x38,0xa9,0x9c,0xb2,0xbb,0x62,0xcb,0x97,0xb2, -0x6b,0x21,0x26,0xb8,0xf8,0xad,0xd9,0xd4,0xf6,0x1a,0x84,0xb2, -0xe1,0xb7,0x51,0xa1,0x78,0xd4,0x24,0x61,0x13,0xf3,0xe2,0x1b, -0x5c,0x4d,0x10,0x6d,0xed,0x04,0x7f,0x7a,0xf8,0x76,0x74,0x06, -0xc6,0xa5,0xd7,0x78,0x0b,0xc3,0x55,0x9a,0x06,0xe5,0x69,0x9f, -0xa0,0x31,0x9e,0x8e,0xb2,0x0c,0xd2,0x05,0x29,0x1b,0xf7,0x8d, -0x39,0xac,0x0c,0x81,0x92,0x46,0xf0,0xef,0xff,0x10,0xac,0x1b, -0xd2,0xbf,0xf6,0x51,0x0d,0x57,0x01,0xe9,0x9e,0x5f,0x59,0x4e, -0x6d,0x9a,0x31,0x82,0x3f,0x2f,0x26,0xb8,0xcd,0x50,0x46,0x51, -0x2e,0x1b,0xa9,0x28,0xfc,0xea,0x42,0x07,0xc5,0x7d,0xf7,0x54, -0x23,0x41,0xff,0x5c,0x17,0x38,0xa2,0xdb,0x44,0x3c,0xc1,0xf7, -0xf5,0x08,0x4a,0xea,0xf4,0x16,0x74,0xf5,0x53,0x98,0x01,0xbd, -0x00,0x2a,0xcb,0x6b,0x60,0x62,0x89,0x0b,0x0a,0x91,0x7b,0x08, -0x45,0x08,0xc0,0x92,0xd7,0xec,0xd5,0x48,0xd0,0x73,0xb5,0x61, -0xf1,0x5d,0x67,0x90,0x59,0x2c,0xc1,0xbd,0x00,0xc6,0x74,0x0a, -0x83,0xc9,0x08,0x50,0xe5,0xf1,0x8f,0x10,0x6b,0x78,0x55,0x04, -0x22,0xfb,0x9c,0x98,0xae,0x32,0xb8,0x12,0x32,0xdc,0x3d,0x23, -0x33,0x0c,0x59,0x57,0x23,0x97,0x21,0x90,0xfa,0x1a,0x85,0xfa, -0xec,0x43,0x8a,0xaa,0x1e,0xd6,0x17,0xc0,0x3c,0xff,0xe3,0x18, -0xcc,0xc5,0x06,0xbf,0xd2,0xd2,0xe8,0x3a,0xc3,0x5a,0xfb,0xe7, -0x81,0x26,0x78,0x1d,0xe8,0x06,0x0b,0x73,0xf3,0x4e,0x23,0x20, -0x4f,0xdd,0x57,0x29,0xc6,0x20,0xfc,0x2d,0xb1,0x28,0x54,0x20, -0x8d,0x40,0x9b,0x23,0x0e,0x68,0x10,0xfa,0x4d,0xac,0xea,0x85, -0xc4,0x57,0xb4,0x58,0x25,0x45,0x37,0x80,0x3d,0xf4,0x77,0x57, -0xe5,0x16,0xb0,0xe1,0x0f,0xae,0x17,0xd1,0x48,0xa1,0x49,0x8e, -0xe3,0xe8,0x49,0x94,0xdc,0x98,0x01,0x83,0xd9,0xad,0xd1,0x0c, -0xbd,0xd9,0x20,0xf3,0x26,0x9d,0x4c,0xa6,0xd8,0x67,0x46,0x0a, -0x55,0xd9,0x06,0xf9,0x8c,0xa9,0xa7,0x16,0xe8,0xe4,0x9c,0xbc, -0x93,0x55,0xc5,0xcd,0xc6,0x2f,0xe1,0x77,0x0a,0x00,0xfd,0xc9, -0x7e,0x29,0x00,0xf8,0xbd,0x4b,0x98,0x39,0xde,0xf7,0xa8,0x37, -0x56,0xf3,0xc5,0xbd,0xbb,0xa2,0xb5,0xbf,0x2f,0x5a,0xde,0x61, -0x5c,0x39,0xa4,0x3b,0xaa,0xaa,0xdb,0xd5,0x1f,0x87,0xd7,0x5d, -0xec,0x92,0xf0,0x8f,0xea,0x90,0xd6,0xde,0x1e,0x69,0xd5,0x77, -0x09,0x06,0xd2,0xdb,0xd1,0x27,0x6f,0x4a,0x7d,0xb2,0x39,0xa0, -0x4f,0x36,0xbf,0xbb,0x4f,0x0a,0x35,0x58,0x1d,0x14,0x56,0xf7, -0xc9,0xc1,0xf5,0x99,0xae,0x0b,0x2b,0x86,0xc9,0x59,0xa7,0xbe, -0x57,0xc8,0x74,0xac,0xed,0x92,0x62,0x2f,0x04,0x76,0x2f,0x7d, -0x17,0xfe,0x62,0x77,0xc9,0x77,0x3b,0xbb,0x84,0x22,0x1c,0xde, -0xbb,0x4f,0xf6,0x77,0xc9,0xef,0x51,0x59,0xf7,0xed,0x8e,0xc3, -0xea,0xfa,0xf4,0xbe,0x60,0x73,0xbe,0xbe,0x37,0xee,0xd7,0x19, -0x6e,0x5f,0xe0,0x11,0x86,0x98,0x43,0x04,0xf5,0xd5,0x68,0x26, -0xa1,0x80,0x30,0x82,0x4f,0x21,0xc3,0xa3,0x48,0x95,0xc6,0xd0, -0x34,0x0f,0x41,0x32,0x24,0x24,0x34,0x27,0x9b,0xad,0x35,0x19, -0xf3,0x66,0x9d,0x63,0x66,0x6e,0x6c,0xa3,0xa3,0x1a,0x80,0x17, -0x51,0x3b,0xb1,0xb0,0x69,0xba,0x0f,0x95,0x63,0x7c,0x81,0x39, -0xb4,0x03,0x46,0x36,0x68,0x6d,0xfb,0x8e,0xe3,0x43,0x94,0x23, -0x0e,0xf0,0xa4,0x6b,0x4b,0xa9,0x3c,0x51,0xf6,0x13,0x65,0xb8, -0xf2,0x13,0x3f,0x0e,0x1c,0x8e,0xc3,0xb7,0x25,0x15,0xf1,0x59, -0xc1,0xda,0x8c,0x07,0x51,0x38,0x0c,0x8c,0x69,0x4b,0xf6,0xac, -0xde,0xc8,0xda,0xe0,0xc6,0x24,0x51,0x75,0x98,0x53,0x24,0x81, -0x6e,0x02,0x2c,0x27,0xf6,0xf3,0x96,0x3e,0xda,0xd2,0x84,0x83, -0xb0,0x08,0xec,0x06,0xb7,0x9a,0xad,0x36,0x73,0x63,0xb3,0x3e, -0x86,0x4e,0xe9,0xd6,0xf2,0x82,0xab,0x66,0x14,0x52,0x37,0xaf, -0x0a,0xca,0xbb,0x8b,0x15,0x6e,0xa2,0x2e,0x6b,0x34,0x80,0x60, -0x75,0xb3,0x2d,0xec,0xbf,0xdc,0xe3,0x25,0x49,0x0b,0x7e,0x63, -0xe1,0xfd,0x45,0x1e,0x94,0xd4,0x38,0x65,0x79,0x71,0x30,0x4e, -0x11,0x22,0x07,0xa9,0xa4,0x59,0x58,0x4b,0x61,0x3a,0x77,0x07, -0x89,0x99,0xf5,0x5d,0x1d,0x13,0x37,0x37,0x05,0xe9,0x6a,0xea, -0xa2,0x14,0x39,0x51,0x2a,0x77,0x0a,0x49,0xf5,0x2e,0xa2,0x83, -0xda,0x55,0xaa,0xbb,0x1e,0xbb,0x4b,0xd5,0x61,0x72,0xe6,0x36, -0x44,0x61,0x77,0x92,0x2d,0xd6,0xbd,0xa8,0x88,0x5d,0xb5,0xb3, -0x47,0x54,0x81,0xa6,0xdd,0x25,0x2f,0x54,0x34,0x66,0xd3,0x25, -0xeb,0xcb,0x49,0x7a,0x9b,0x4e,0xaa,0x36,0xc0,0x8b,0x88,0x8f, -0x8f,0xb1,0x18,0x17,0x76,0x2f,0x32,0x16,0x07,0x42,0xac,0xac, -0xf3,0xa0,0x3c,0x84,0xe2,0x8f,0x60,0x2b,0x76,0x23,0x0e,0x06, -0x0c,0x7f,0x19,0x12,0x7e,0xa0,0xbd,0xd2,0x8d,0xb6,0xca,0x6b, -0xf3,0x72,0xb9,0xce,0xae,0x2b,0x97,0xf9,0x94,0x53,0xbf,0xce, -0x2f,0xad,0xec,0xd5,0x3c,0x43,0x2f,0xed,0x24,0xcb,0x16,0xed, -0x21,0xe1,0x22,0x0b,0xd4,0xa3,0x99,0x85,0x5a,0x1f,0x92,0xcb, -0xf7,0xe9,0xaa,0xb5,0x1a,0x2d,0x5a,0xd7,0x40,0xdb,0x14,0xe9, -0x6b,0x8d,0xe7,0x53,0x9a,0x95,0x96,0x6f,0x2f,0x47,0x5e,0x18, -0xd0,0xff,0xfc,0x06,0x6d,0x32,0x35,0x6e,0xe6,0xeb,0x2c,0x41, -0x37,0x5c,0xa6,0xb5,0x11,0xd0,0x1f,0x9a,0xbf,0x18,0x60,0x35, -0x5f,0x8f,0xf9,0xbb,0x02,0x22,0xb8,0x1c,0x8d,0xdf,0xbf,0xe5, -0xd0,0x54,0xee,0x22,0xd2,0x64,0xd8,0x0b,0xc9,0x9e,0x49,0x2e, -0x2d,0x21,0x31,0xd8,0x5d,0x69,0x09,0xe9,0xa0,0x91,0x46,0xde, -0xa6,0x59,0x7a,0x99,0x4e,0x69,0x75,0xdb,0xb8,0x06,0x5b,0x34, -0x99,0x99,0xcc,0xf1,0x7a,0x99,0x51,0x63,0xc9,0xff,0xfb,0x7a, -0x94,0x2e,0x61,0x1a,0x76,0x49,0x83,0x99,0x1a,0xea,0xb6,0xc9, -0x3a,0x8c,0x16,0x5d,0xae,0x58,0x15,0x2e,0xcc,0x65,0x79,0xba, -0x4c,0xc8,0x43,0xd4,0xad,0x90,0x13,0x55,0x85,0xfc,0x95,0x05, -0xf6,0x7d,0xcf,0x1e,0x27,0xca,0x06,0x80,0x59,0xeb,0xab,0xe4, -0x02,0x79,0x6f,0x8b,0xb4,0x95,0xe7,0xf7,0x06,0x17,0xc5,0xbb, -0xeb,0xdb,0x12,0xc5,0x15,0xe6,0x80,0xc8,0x38,0xf5,0xed,0x33, -0x02,0x1b,0x4c,0x86,0xdb,0x1a,0x8e,0x6c,0xaa,0x50,0x9c,0x0e, -0x92,0x0f,0xc3,0x3f,0x9d,0xc2,0x02,0x3a,0x5b,0x41,0x6a,0xbf, -0x75,0xc6,0x91,0x87,0x2d,0x33,0xaa,0xa2,0xd0,0xdf,0x07,0xb3, -0x6c,0x58,0x5f,0xe8,0x43,0x3a,0x59,0x81,0xc8,0x3d,0x56,0xdf, -0xd7,0x09,0x59,0x9f,0x98,0xb0,0x4b,0x1a,0x14,0xe3,0x04,0x06, -0xf7,0x59,0xa7,0xa3,0x5c,0x44,0x97,0x1d,0x83,0x3d,0xbf,0xdf, -0x98,0xcd,0x67,0x49,0x83,0x2a,0x94,0xde,0xa3,0x17,0xdb,0xde, -0x56,0xee,0xae,0xbc,0xb5,0x85,0xbd,0x12,0xc2,0x64,0xfb,0x3d, -0x0a,0xec,0x59,0xb1,0xed,0xb2,0x29,0x63,0x31,0x3c,0x55,0x36, -0x60,0xa1,0xf1,0xca,0x16,0x6c,0xe9,0x7c,0x68,0xde,0x64,0x39, -0xfa,0xf0,0xda,0x13,0x52,0x61,0x99,0x92,0xd7,0x54,0x98,0xd7, -0x56,0x98,0x97,0x2b,0x54,0xdc,0xad,0xad,0xf1,0x8d,0x55,0x23, -0xa7,0xe8,0x04,0xdb,0x69,0x48,0x65,0xa1,0x76,0xab,0x1c,0x08, -0x5c,0x9d,0x71,0x92,0xa9,0x12,0x60,0xca,0xa5,0x63,0xfc,0x46, -0x73,0xf3,0x0d,0x8d,0xbd,0x41,0xf3,0x34,0x31,0x12,0xc6,0xcb, -0xcb,0x5c,0x67,0xfd,0x3d,0xb3,0xb3,0xfc,0x46,0x99,0xa2,0xd7, -0x36,0x49,0xb6,0x26,0xd0,0x5d,0xa0,0x2a,0xc2,0x26,0x57,0x69, -0x8d,0xa0,0x3d,0xa3,0x68,0xa1,0x41,0x3b,0x3b,0x77,0x86,0x84, -0xf4,0x95,0x2a,0x0f,0xbc,0xb3,0x50,0x15,0xe9,0x78,0xb3,0x9b, -0x0e,0xe8,0x99,0x7c,0x2f,0x1d,0x89,0xd0,0xf1,0xc1,0xa5,0x43, -0x75,0x61,0x6e,0x08,0xc9,0xab,0x08,0x31,0x4a,0xdc,0x9e,0x95, -0xde,0x27,0x39,0xce,0x05,0x90,0x74,0xd6,0x89,0xe9,0xb0,0x14, -0xa7,0x19,0xbc,0x61,0xf0,0x8c,0xbd,0x99,0x81,0xfa,0xb7,0x6f, -0x61,0xf6,0xbd,0xbb,0xf3,0xc6,0xa4,0x95,0x38,0x3e,0x20,0x58, -0x8b,0x90,0x8a,0x0f,0xc8,0xc6,0x86,0x03,0x2a,0x15,0x48,0x88, -0x0d,0x39,0x81,0x42,0x11,0x77,0xe8,0xe0,0x86,0x6a,0x78,0xb1, -0xa4,0x3f,0x32,0xff,0x7a,0xf6,0x3b,0x6f,0x50,0xf7,0x7a,0x51, -0x4b,0x10,0x26,0x0a,0x3e,0x7c,0x16,0x56,0x91,0xd1,0xac,0x24, -0xa3,0x59,0x49,0x46,0x78,0x08,0x15,0xc4,0x2e,0x56,0xca,0x7c, -0x5a,0x4e,0xef,0xf6,0xe1,0x35,0x24,0x9c,0x3f,0xe5,0xca,0x9e, -0x1f,0x20,0xc4,0x04,0x03,0xd4,0xf3,0x7b,0xdd,0x14,0xcd,0x38, -0x25,0xd6,0x37,0xe5,0x19,0x79,0x0c,0xaf,0xca,0x69,0x91,0x49, -0xc3,0xa0,0x6f,0x16,0x67,0x75,0x33,0x47,0xd3,0xd5,0x7f,0x4f, -0xf2,0xbe,0xb0,0xda,0xf0,0x7c,0xe0,0x19,0xe1,0x6a,0x9a,0x96, -0xe2,0x5a,0xcf,0x33,0xbd,0xdd,0xcc,0xed,0x9c,0xa1,0x6f,0x7a, -0x49,0x73,0xa7,0xa9,0xe9,0xfb,0x82,0x51,0xa3,0x90,0x58,0x1c, -0xd3,0x0c,0x53,0x80,0x91,0x06,0x04,0xa4,0x43,0xbf,0x6b,0x09, -0x81,0x28,0x5d,0x68,0xc2,0xeb,0xe3,0x63,0x64,0x44,0xc4,0x65, -0x82,0x0d,0x58,0x1b,0xa8,0x9d,0xf4,0x10,0x54,0x6c,0x0a,0x4d, -0x91,0x37,0x6e,0x91,0x3c,0x88,0x4c,0x91,0x37,0x6e,0x11,0xfa, -0xa5,0x33,0x31,0x8f,0x98,0x75,0xe1,0x7d,0xc4,0x13,0xb0,0x6e, -0x43,0xac,0x14,0x8c,0x08,0xd9,0x55,0x4c,0xed,0xf3,0x1c,0xdd, -0x95,0x59,0xb1,0xb1,0xb5,0xfb,0xd6,0xae,0x98,0x0f,0x4a,0xc4, -0x7f,0xe5,0x26,0x9d,0xe1,0xb3,0xd2,0xf5,0xfb,0xe7,0x40,0x7f, -0xa8,0x1f,0x7a,0xc5,0x48,0x46,0x4a,0x71,0xea,0x18,0xe4,0xb1, -0xb0,0x32,0x1d,0x06,0x3c,0x70,0xe3,0xb4,0x2f,0x3c,0xed,0x4a, -0x27,0xf0,0x4d,0xbd,0x44,0x0f,0xd7,0x44,0xf5,0x9f,0x3e,0xf5, -0x36,0x02,0xbe,0x0c,0x5b,0xb1,0x42,0x0d,0xf5,0xb5,0x28,0x42, -0x6f,0x53,0xa5,0x00,0x6f,0xd2,0x59,0xec,0x41,0x0d,0xcf,0xa6, -0xa3,0x9b,0x45,0x77,0x43,0x7f,0x7c,0x13,0x45,0x7a,0x19,0x06, -0x26,0x52,0x71,0x24,0x72,0x88,0x2f,0xca,0x76,0xd5,0x4f,0x2d, -0x86,0x7d,0x8c,0x4a,0xed,0x01,0x60,0x53,0x57,0xe8,0x37,0xe9, -0x39,0x01,0x11,0x42,0x2b,0x52,0x77,0x5c,0x57,0x41,0xe7,0x44, -0x84,0x25,0x1d,0xb6,0x20,0xc9,0xf7,0x89,0xc2,0x73,0x55,0xae, -0x4f,0x4f,0x6d,0x13,0x97,0x81,0x6e,0x5d,0x4d,0x17,0x53,0x4d, -0x9b,0x34,0x3f,0x8e,0x10,0xf4,0xee,0x4e,0x73,0x0a,0xbe,0x47, -0x9b,0xbe,0xa7,0x19,0xfa,0x35,0x1d,0x37,0x90,0x2c,0x2a,0xde, -0x5a,0x49,0x06,0x0f,0xd5,0xa8,0xb1,0x20,0x12,0x8c,0xde,0xde, -0xbd,0x9d,0xa7,0x93,0x07,0x61,0x61,0xc8,0xa3,0xed,0xe3,0x7f, -0xb4,0x46,0x3f,0x6a,0xe3,0xdd,0x96,0x78,0xd5,0x7c,0x77,0xb8, -0x1d,0x12,0x18,0xbb,0x1f,0xd6,0x1f,0x93,0xbc,0x6c,0x6e,0x32, -0xd8,0x07,0x63,0xbd,0x23,0x61,0xca,0x36,0x67,0xe7,0x29,0x9d, -0xb5,0x5e,0x94,0x33,0xc8,0xa0,0xaf,0x2e,0x43,0x59,0xe8,0x27, -0x54,0xca,0x91,0x89,0xa1,0x32,0xa3,0x50,0x09,0x89,0xd0,0xcf, -0xf4,0x9c,0x1e,0xf2,0xab,0x62,0x70,0xe2,0x4e,0xa5,0x6c,0xde, -0xb0,0x7c,0x04,0xac,0x07,0xd5,0x3d,0x6e,0x3a,0x79,0xa7,0x72, -0xaf,0xd4,0x55,0x68,0xc5,0x13,0xad,0x1a,0x95,0xc6,0x65,0xf4, -0x31,0x27,0xce,0xaf,0x44,0x13,0xdb,0xb1,0xc3,0x9c,0xa5,0x94, -0x94,0x52,0x6a,0x27,0xb6,0x6a,0x41,0xeb,0x7c,0x7d,0xe3,0x59, -0x6a,0x2c,0x3e,0x3a,0xfd,0xbb,0x37,0xbb,0xcb,0x7c,0x65,0x73, -0xa8,0x1c,0xd0,0x4d,0x1b,0xa3,0xba,0x08,0x2c,0xb9,0xfb,0x50, -0x05,0x96,0x9b,0x79,0xc6,0xae,0x8a,0xcc,0xf6,0x04,0xcc,0x6a, -0x35,0xf3,0xdb,0x4c,0x8b,0xd5,0x7c,0x74,0x81,0x89,0x2f,0xd7, -0x8b,0x05,0x20,0xcc,0x3c,0xa5,0xbf,0xcb,0xb3,0xce,0x07,0x87, -0x41,0xe2,0x5d,0x57,0xd9,0x6f,0xf2,0x59,0xd1,0x75,0xf4,0xe1, -0xa3,0xdf,0x88,0x66,0x31,0xbf,0x7b,0x38,0x79,0x45,0x42,0x91, -0xf5,0x3f,0x54,0x4b,0x8f,0x1e,0x1b,0xd5,0x22,0xa4,0x3a,0xdc, -0xef,0xd6,0x48,0x6c,0xa1,0x7c,0x41,0x6c,0x75,0x71,0x18,0x77, -0x34,0xdc,0x96,0xeb,0x05,0xae,0x90,0xac,0x51,0x76,0xd2,0xf0, -0xed,0x2c,0xcd,0x70,0xbf,0x3c,0xdd,0x81,0x7d,0xa2,0x13,0x2b, -0x26,0x39,0x93,0x1d,0x0d,0xf9,0x41,0x76,0x7c,0x0d,0x4d,0x75, -0x26,0x4d,0x09,0xc9,0x26,0x6e,0x9e,0x7e,0x28,0x75,0x74,0x90, -0xe4,0x90,0xfe,0xf7,0x59,0x31,0x5d,0xec,0x80,0x78,0xa0,0x27, -0x69,0x50,0xef,0x36,0x11,0xb9,0x49,0xcf,0xad,0xda,0x87,0x15, -0x73,0x35,0x14,0xac,0xa0,0x19,0x8a,0x29,0x07,0x16,0xc7,0x68, -0x00,0xf5,0x2c,0x73,0x33,0x97,0x51,0x8f,0x3d,0xf8,0xbd,0x7a, -0x05,0x46,0x6a,0xe8,0x40,0x0d,0x76,0xa0,0xb2,0xb2,0x85,0xde, -0xcd,0xf2,0x2b,0x55,0x03,0x6f,0x9c,0x6f,0xfd,0xc0,0x52,0xb9, -0xa4,0x28,0x1c,0x85,0x1e,0x38,0x1a,0x3f,0x50,0xfe,0x34,0x3c, -0x6a,0xbe,0x56,0x4e,0x35,0x84,0x22,0xb0,0xf1,0x06,0xca,0x38, -0x30,0x7a,0xc8,0x0f,0x36,0x3c,0x3d,0xe4,0xfc,0x47,0xea,0x89, -0x07,0x61,0x60,0xba,0x47,0xbe,0x78,0x3e,0xc5,0xf7,0x06,0x72, -0xfd,0x4b,0x56,0xf3,0xb1,0xbd,0x7e,0xfe,0x99,0xd3,0xac,0x29, -0x5c,0xf4,0xfd,0xad,0xb3,0x83,0x5b,0xbb,0xf9,0x53,0xd2,0x69, -0x85,0x68,0x88,0xac,0x34,0xa2,0xf8,0xe3,0x46,0xdb,0x10,0x79, -0x57,0x48,0x0d,0x52,0x77,0xee,0x0b,0xde,0x75,0x1d,0x66,0x6d, -0xa5,0x74,0xd9,0x47,0x44,0xb0,0xf6,0x0a,0xee,0x21,0x92,0x5c, -0xe3,0x46,0xd5,0x2f,0x6e,0x53,0xd9,0xb7,0x4f,0xc5,0xf1,0xd4, -0xd9,0x3c,0xb2,0x9a,0xea,0xce,0xd1,0x42,0x56,0x3b,0x75,0x3b, -0x57,0xa7,0xbf,0xd3,0x5d,0xa3,0x52,0x36,0xba,0x7b,0x54,0x4a, -0xbe,0x4b,0xa4,0xb6,0xbe,0x72,0xe6,0xa5,0xe4,0x6e,0x99,0x20, -0x92,0xb3,0xa2,0x13,0xf9,0x97,0xcb,0xe5,0x28,0x57,0xf6,0xb6, -0x62,0x7d,0x1b,0xd6,0xf1,0x79,0x35,0x64,0x5e,0x80,0xcc,0xb5, -0xe3,0x96,0xdb,0xde,0xb2,0xd1,0xe2,0x78,0xaa,0xbb,0x8d,0x8d, -0xf0,0xe9,0x09,0x72,0x50,0xcf,0x0b,0xe9,0x79,0x9c,0x53,0x7a, -0xbd,0x0d,0x6c,0x6c,0xde,0xed,0xd6,0xb7,0x7c,0x81,0x0f,0xeb, -0x90,0xa8,0xa6,0x43,0x22,0xe8,0x90,0x43,0xea,0xdc,0x31,0xff, -0xfb,0x5d,0x6f,0x97,0x06,0xf8,0x7d,0xd8,0xc9,0x11,0x99,0x87, -0x9d,0x75,0xfc,0xf5,0xdb,0x8e,0x0d,0xe7,0x4d,0xfc,0xdb,0xce, -0xd1,0x7c,0xb4,0xf9,0xe2,0x8b,0xe8,0xee,0x28,0x1f,0x32,0x5a, -0xbf,0xbb,0x51,0x15,0xe4,0x87,0x55,0x90,0xdf,0xb7,0x82,0x5c, -0x55,0x30,0x26,0x6d,0x73,0x58,0x2b,0xc0,0xe6,0x80,0x7f,0x45, -0x3f,0x1d,0xfd,0x46,0x93,0x8c,0xf9,0x82,0x55,0x5b,0x77,0xd3, -0x37,0xd9,0x5d,0x9c,0x28,0x4c,0xbe,0xaf,0xdb,0x86,0x68,0x06, -0x0c,0x27,0xe5,0x87,0xba,0x60,0x37,0x57,0x2b,0x0c,0x7e,0x5a, -0x4c,0xb4,0x1b,0x0b,0xa6,0x4d,0x25,0x0d,0xa8,0x30,0xd8,0x44, -0x41,0x0e,0x3a,0x33,0x0a,0x56,0x75,0xfe,0xdd,0x44,0xb7,0xb7, -0x09,0x63,0xa2,0x77,0x13,0xc5,0x48,0x29,0xed,0x69,0x41,0xda, -0x26,0x94,0xc4,0x4d,0x44,0xfb,0x21,0xae,0x98,0x0e,0xa8,0x02, -0xc8,0x07,0x23,0xe0,0x36,0x59,0xae,0xa4,0x0c,0xfc,0xe3,0x53, -0x19,0x6f,0x13,0xf9,0xf8,0x9e,0xdf,0xf9,0x06,0x83,0xba,0xae, -0x62,0x04,0x87,0xfc,0x08,0x33,0x61,0xdc,0x00,0xdc,0x91,0x65, -0x1f,0xdc,0xdd,0x6d,0xa2,0x23,0x6b,0xf3,0x00,0x57,0xa1,0x7a, -0x4a,0xe0,0x9a,0x64,0xb3,0x2d,0x17,0x62,0x73,0x21,0x76,0xc3, -0x69,0x79,0x48,0x94,0x47,0x71,0x1e,0xd1,0xb2,0xde,0x1d,0x3b, -0x03,0xe2,0x03,0xe4,0x1b,0x62,0xb1,0x0c,0xfc,0xe3,0x53,0x19, -0x2f,0x47,0x62,0xf3,0xf0,0x3c,0x67,0x62,0x11,0x1c,0xf2,0x23, -0xcc,0x44,0x62,0x73,0x20,0x36,0xb7,0x88,0xcd,0xa3,0x23,0x6b, -0x4f,0x03,0x89,0xd5,0x33,0x16,0xd7,0xe4,0xeb,0x1e,0x65,0x0e, -0xdb,0xd4,0xf4,0x89,0x51,0x76,0x8a,0xe2,0xb3,0x93,0x86,0x22, -0x63,0x41,0xba,0x30,0x83,0x32,0xe7,0xe5,0x4b,0x77,0x80,0xfe, -0xde,0xd5,0x0f,0x8a,0xa9,0x2e,0x7d,0xd8,0xf6,0x22,0x7d,0xc8, -0xa6,0x32,0x7d,0x06,0xd2,0x85,0x19,0x94,0x99,0xad,0xe8,0x51, -0x3c,0xd7,0xdf,0xbb,0x58,0x8f,0xa4,0xd3,0x90,0x40,0x21,0xc8, -0xa1,0x8a,0x01,0x0a,0x35,0x18,0x6a,0x30,0x28,0x44,0x2e,0x70, -0x28,0x29,0xae,0x9b,0x81,0x5b,0xfd,0xba,0x80,0xb3,0xfe,0xbb, -0xbb,0xf3,0x76,0x5a,0x1d,0x7b,0xa6,0x0d,0xe9,0xe1,0xad,0xbd, -0xa8,0x2c,0x57,0x79,0x74,0x04,0xf4,0x9b,0x2e,0x8c,0xad,0x0e, -0xbc,0xbb,0x3b,0x3a,0x02,0xda,0x73,0x2b,0xd3,0x70,0x8f,0x9f, -0xc9,0xe0,0x47,0x34,0xd8,0xa0,0x22,0x95,0x1b,0x34,0xe6,0xb3, -0x86,0xef,0x1c,0xa0,0x59,0x47,0x07,0xf1,0x47,0x7c,0x21,0x25, -0x6b,0x89,0x9e,0x0e,0x40,0x35,0x27,0x1f,0xf4,0x57,0xe6,0xe4, -0x7d,0x70,0xf2,0x66,0xf0,0x39,0xfb,0x90,0x25,0x26,0x01,0xca, -0xce,0x92,0xcc,0x2a,0x9d,0x14,0x20,0xb2,0x0f,0x2e,0x84,0x8e, -0xec,0x69,0x2b,0xd7,0x78,0x30,0x68,0xcc,0xf0,0x0c,0x07,0x9d, -0x33,0xe0,0xbf,0x0f,0x68,0xee,0xd2,0x3f,0x94,0x42,0xff,0x7c, -0x68,0x40,0xb7,0x12,0x04,0xff,0x9a,0x11,0x2c,0x46,0xee,0xe6, -0x38,0x26,0x29,0xdf,0xe1,0xa3,0xbf,0x1f,0xa9,0x16,0x3a,0x04, -0xf8,0x1a,0x6f,0x6d,0x4a,0x3e,0xa4,0xe4,0x2f,0x25,0x46,0xd1, -0xa0,0xf1,0x72,0x3d,0x9b,0xa0,0xad,0xdc,0xf8,0x61,0x2e,0x3f, -0x5e,0xad,0x93,0x8c,0x7f,0xfd,0x25,0x99,0xcc,0xd4,0xef,0x57, -0xd7,0xc0,0x35,0xfe,0xf9,0xed,0x32,0xe5,0x1f,0x2f,0x61,0x31, -0xba,0xc4,0x9f,0xf4,0xca,0x3d,0x56,0x74,0xb1,0x5e,0x8d,0xad, -0x00,0x1e,0x1f,0xc1,0xa4,0xc6,0xaa,0x2b,0xde,0x72,0x66,0x0b, -0x0e,0xaf,0x50,0xfd,0xf9,0xd5,0x33,0x84,0xc1,0x40,0x20,0x04, -0x9e,0xef,0x87,0xce,0x05,0xf8,0x5b,0x10,0xad,0x37,0x20,0xbd, -0xfb,0x4a,0x28,0x38,0x29,0xf6,0xdd,0x1c,0xda,0xb2,0xaf,0x0c, -0x01,0x49,0x81,0x1f,0xd2,0xe9,0x34,0xe5,0x67,0xf0,0xf7,0x96, -0xb3,0x61,0x75,0xf1,0xd9,0x1a,0xd6,0x5c,0xfb,0x4b,0x12,0x98, -0x2a,0x34,0x9f,0xad,0xae,0xf7,0x16,0x41,0x20,0x29,0xf0,0xf2, -0x30,0xfa,0x5e,0x3a,0xa4,0xbd,0x02,0x81,0xd8,0x59,0x02,0x01, -0x2c,0xd8,0xdf,0x60,0xf9,0xc5,0x61,0x30,0x2b,0x4a,0x85,0x5b, -0xbe,0x57,0xf2,0xd3,0x55,0x3d,0x4a,0x01,0x40,0x94,0x59,0x59, -0x38,0x94,0x88,0x2e,0xac,0xd8,0x1e,0x4a,0x40,0xac,0x28,0xc1, -0xed,0x0b,0x6b,0xa9,0x21,0x88,0xf2,0xc3,0xf0,0xe4,0x3b,0xd1, -0x54,0x09,0x54,0x1d,0x2e,0x05,0xbb,0x13,0x61,0x49,0xd4,0xea, -0xb0,0x11,0xe0,0x4e,0x54,0x75,0x42,0x58,0x87,0xd1,0x86,0xdf, -0x83,0xb8,0x24,0x9e,0xf5,0x38,0x09,0x74,0x37,0xba,0xa2,0xe0, -0xd6,0x22,0x43,0xc0,0x9d,0xa8,0x2a,0x44,0xba,0x0e,0xd9,0xcb, -0x03,0x1a,0x5a,0x14,0xf6,0x4a,0x5c,0x08,0x54,0x8b,0x45,0x4f, -0x26,0x6e,0x39,0xd2,0xaf,0x46,0xef,0x69,0x4d,0xcb,0x6f,0xdf, -0x60,0x26,0x62,0x8d,0x1b,0x9f,0x8d,0x1e,0x7c,0x76,0xf9,0xe0, -0xb3,0xe4,0xc1,0x67,0xaf,0x1f,0x7c,0xf6,0xa6,0x51,0x01,0x07, -0x30,0x37,0xa7,0x9f,0x4d,0x4e,0xcb,0xb9,0x82,0xe1,0xbb,0xee, -0x67,0x3f,0x74,0x3f,0x7b,0xd9,0xb0,0xb5,0xf9,0x1f,0xa7,0xc7, -0x6d,0xac,0x5f,0x5e,0x5e,0xe2,0xeb,0xd5,0x74,0xe9,0x44,0x6a, -0x60,0xf4,0x8c,0x9b,0xf1,0x32,0x46,0x46,0xc6,0x98,0x2c,0x24, -0x37,0xd8,0xc1,0x58,0xf4,0xbf,0x8d,0x66,0xeb,0xd1,0x92,0xea, -0x4c,0x60,0xc6,0xe3,0x9f,0x3f,0xe0,0x1b,0x57,0xf0,0xf7,0xcb, -0xc5,0x32,0x9d,0xd2,0x37,0xa6,0xfe,0xb7,0x35,0xcd,0x76,0xff, -0x6d,0x3d,0xc5,0xaf,0x2f,0xd7,0x6f,0xd7,0x19,0xee,0x89,0xbc, -0x4c,0x16,0xab,0x84,0x5e,0x33,0x0b,0x1a,0x3f,0x8d,0x57,0x73, -0xfe,0xf5,0xe3,0xfc,0x56,0x25,0x7e,0x9d,0x8c,0xf9,0x67,0xa1, -0xfa,0x62,0x2b,0x80,0x14,0x26,0x83,0x29,0xe0,0xfa,0xed,0xda, -0xb9,0x72,0xae,0x9b,0x2b,0xe6,0x2a,0xb9,0x3a,0xae,0x89,0x27, -0x3c,0xac,0xa4,0x9d,0xa3,0xf1,0xa4,0x6a,0xa4,0x85,0x36,0x68, -0x38,0xb3,0x59,0x42,0xc7,0xce,0xda,0x7f,0x81,0xcf,0xe3,0xb9, -0x20,0x70,0x98,0x73,0x03,0x71,0xc4,0xe7,0x81,0x83,0x0f,0x69, -0x70,0xd2,0x36,0x70,0xb0,0xc8,0xfe,0x33,0x48,0xad,0x80,0xeb, -0x49,0x8d,0x12,0xac,0xd9,0xd0,0xf3,0x9b,0x02,0x5b,0x40,0xe1, -0x10,0xe2,0x96,0x90,0x0b,0x6a,0xaa,0x45,0x59,0x6c,0x7f,0xb1, -0xab,0xa6,0x0b,0xd0,0x86,0x99,0xde,0x05,0xc2,0xa9,0xdf,0x05, -0x84,0x26,0xee,0x65,0x0d,0x8d,0x27,0x80,0x9b,0x25,0x1f,0x1e, -0x88,0x11,0xe1,0x75,0x92,0xb3,0x20,0xb4,0x5e,0x9c,0xc8,0xf7, -0x34,0x37,0x50,0x49,0x32,0x1d,0xea,0x6f,0xb6,0x2b,0xf0,0x3b, -0xdf,0xc7,0x4d,0x02,0x75,0xcb,0x1d,0xca,0x45,0x86,0x6e,0x45, -0x16,0x0f,0x69,0x50,0x5a,0x1f,0x65,0xc6,0xb8,0x0c,0x44,0x90, -0x4a,0xfe,0xfd,0x74,0xf5,0xc6,0xb1,0xce,0x0d,0xd3,0x6c,0xc1, -0xa3,0x0e,0xe0,0xbc,0x9e,0xfd,0xf8,0x19,0x3f,0x87,0x45,0x19, -0x2d,0x04,0x69,0x3d,0x4e,0x1e,0x9e,0xe8,0x56,0xba,0xf3,0x39, -0xb4,0x80,0xba,0xb1,0x22,0xc3,0xf7,0x4f,0x3f,0x7f,0xfc,0x30, -0x79,0x24,0xd7,0x3e,0x5d,0x1b,0x52,0x3f,0x57,0x6b,0xd1,0x98, -0xe3,0xe9,0x24,0xf5,0x3e,0xb4,0x6b,0x35,0xff,0x7e,0xfe,0x21, -0x59,0x3e,0x1b,0x65,0x78,0xd4,0x92,0xc6,0x4f,0x5a,0x69,0x4f, -0xc7,0x22,0xb9,0xa5,0x78,0xac,0x84,0x73,0x00,0xc0,0xc3,0x03, -0x87,0x92,0x57,0x3d,0x94,0xfc,0x9a,0x9e,0x6c,0x59,0xdf,0x60, -0x3b,0x36,0x53,0xff,0xb3,0x27,0x87,0x0d,0xb2,0x2a,0xb1,0x78, -0x72,0x62,0x31,0x57,0x0a,0x94,0x85,0x44,0x09,0x76,0xb9,0x8b, -0x34,0x1d,0x95,0x7d,0x55,0xec,0x7b,0x2e,0xd3,0x44,0xae,0x32, -0xdd,0xa7,0x4f,0xa8,0x3d,0xf9,0x51,0x1c,0xa7,0x3e,0x05,0x5f, -0xd4,0xec,0x6b,0xe2,0x5a,0x20,0x56,0xac,0x73,0xa5,0x49,0xe7, -0x93,0xe0,0x69,0x98,0x92,0xd4,0x31,0x1c,0xd7,0x0d,0xc8,0xfe, -0x13,0x5b,0xb5,0xb5,0x06,0xd0,0x87,0x24,0x79,0xaf,0x2b,0xc9, -0x68,0x6e,0x73,0xf2,0xb2,0x42,0x66,0x61,0xe4,0x10,0x88,0x33, -0xc2,0x04,0xac,0x3c,0xc8,0x10,0x54,0x46,0x99,0x0b,0xcb,0x89, -0x1a,0xee,0x4a,0xbd,0x68,0x67,0x4f,0xca,0x8e,0x39,0x00,0x65, -0x7e,0x4e,0x0a,0x00,0x3f,0x27,0x9e,0x3d,0x51,0xfb,0xf6,0x00, -0xfa,0x7e,0x3e,0x7f,0xbf,0x5e,0x14,0x0a,0x70,0x62,0x7d,0x21, -0x9e,0xcf,0xf6,0xd4,0xe3,0x4c,0x7a,0x15,0xc5,0x0f,0xac,0xb9, -0x06,0x0d,0xcd,0xab,0xbb,0x28,0xe0,0x79,0xbf,0x50,0xe0,0x90, -0x3a,0x2b,0x0b,0xee,0x6f,0x71,0x79,0xa2,0xaf,0x44,0x71,0x30, -0x05,0x35,0xa8,0x16,0xc9,0x12,0x8f,0xab,0x80,0x92,0xd3,0xbf, -0x7f,0x76,0x5a,0x30,0xcf,0x5e,0x8c,0x26,0x59,0xfc,0xb1,0xd1, -0x6a,0x74,0x1b,0x8d,0xe0,0xa2,0xdb,0x78,0xd0,0x08,0xc2,0x6e, -0x23,0x6c,0x6c,0x0b,0x70,0x00,0x34,0xea,0x56,0xfb,0x36,0x56, -0xb2,0x7d,0x30,0xd1,0x03,0x6b,0xb8,0x0d,0xbe,0xdc,0x53,0xb4, -0x00,0x7e,0xb9,0x0b,0xbc,0xdc,0x54,0x2e,0x2c,0x93,0x28,0x14, -0xff,0x6a,0x6f,0xf1,0x52,0x91,0x71,0xd7,0x6d,0xae,0x57,0x6d, -0x0a,0x03,0x53,0x2d,0xdc,0xc1,0xa2,0x84,0x5d,0x33,0xd5,0x9b, -0x18,0xcd,0x1b,0x2c,0x82,0x0e,0x28,0xda,0xe4,0xd3,0x8b,0x7e, -0x77,0xaf,0xa2,0xb2,0x05,0x20,0x65,0x9f,0x7f,0x4a,0xd9,0xcf, -0xa2,0xce,0xdd,0x5d,0xd4,0x11,0x14,0xef,0x0e,0x45,0x11,0x35, -0x2b,0x34,0x26,0x12,0x72,0x06,0x58,0xbe,0xbf,0x17,0x21,0xee, -0x7e,0x84,0xa0,0xb8,0xb9,0x1f,0x0a,0xee,0xde,0x66,0x24,0xcd, -0xf8,0xe1,0x9e,0x04,0xc8,0xb6,0x86,0x94,0x5e,0x54,0x4a,0x95, -0xc5,0xb3,0xf3,0x38,0xea,0xf4,0x1b,0x2f,0x7e,0x80,0xa1,0xf4, -0xe5,0x0f,0x30,0x80,0x5e,0xde,0xab,0xba,0x97,0x56,0x53,0xb1, -0xba,0x3f,0x1f,0x5c,0xba,0x42,0xfb,0x33,0xd3,0x11,0xcd,0x87, -0x5a,0xaa,0x65,0x3b,0xea,0x2f,0xf7,0xad,0xe7,0x66,0x5e,0x55, -0xcf,0x66,0xff,0x00,0xf2,0x83,0xd7,0xbb,0x81,0x78,0x84,0xe5, -0xf7,0x62,0x9b,0xb1,0xa6,0x3f,0x8b,0xc2,0x50,0x88,0x79,0xf3, -0xc9,0x28,0x92,0x87,0x80,0xe2,0x21,0xa0,0xf8,0xab,0x26,0x15, -0x2d,0xca,0xa0,0xf1,0x59,0xa3,0xbc,0x37,0x04,0x89,0x5b,0xa3, -0x27,0x17,0xb0,0xba,0x48,0x96,0xa4,0x27,0x9d,0xa4,0xbf,0xc0, -0x44,0xad,0xb5,0x23,0x68,0xc2,0xaa,0x4c,0xd0,0x78,0x4e,0xf2, -0x0f,0x46,0xc7,0x81,0x36,0x2b,0x67,0x59,0xfa,0x8a,0x12,0xbf, -0xc7,0x10,0x66,0x09,0x36,0x04,0x14,0x94,0x93,0x03,0xbd,0x1c, -0x24,0xe5,0xa4,0xef,0xdc,0x24,0x94,0xe0,0xce,0x43,0x50,0x15, -0x55,0xa9,0xef,0x4a,0xc5,0xc5,0xc4,0xf8,0xbe,0x40,0x99,0x35, -0x62,0x83,0x9b,0x0a,0xaa,0xe5,0x4d,0xed,0x1f,0x8a,0xc5,0x68, -0x9c,0xc1,0xf8,0x72,0x92,0xbf,0xbc,0x79,0x71,0x03,0x43,0xc8, -0x49,0x93,0x31,0x02,0x63,0xa3,0xc4,0x44,0xc6,0xcd,0xbb,0x09, -0x20,0xf4,0x55,0x4c,0x96,0xea,0xff,0x52,0x57,0x98,0x77,0x20, -0x2c,0x49,0xb6,0x58,0x4b,0x1b,0xd1,0xaf,0xab,0x72,0x50,0x68, -0x41,0x66,0x9d,0x1c,0xe2,0xce,0x1b,0x37,0x4d,0x49,0x99,0x25, -0x59,0x94,0xfe,0x57,0x25,0x5e,0x2e,0xee,0x14,0xec,0xdc,0xd1, -0xf4,0x05,0x4f,0xdd,0x46,0xc8,0xf8,0x69,0x74,0x9a,0xca,0xff, -0x96,0x9d,0xfc,0x6d,0xd2,0x34,0xf3,0xf9,0x08,0x18,0xa6,0x4d, -0x05,0x0e,0xe1,0x3f,0xba,0xe9,0xc2,0x90,0xb8,0xe9,0x46,0xc6, -0xdc,0x16,0xe1,0xb7,0x6d,0x4c,0x49,0xfa,0xf3,0x6a,0xec,0x98, -0x85,0x9c,0xfa,0x3c,0x9b,0x97,0xe1,0xbc,0xc6,0x67,0x6f,0x5a, -0x9f,0xdd,0xb4,0x3e,0x9b,0xbc,0x52,0x1b,0x3b,0xed,0xcf,0xbe, -0xff,0x6b,0xa3,0x54,0x4b,0x0a,0xa5,0xdd,0x3d,0x26,0x58,0x56, -0x3d,0x7f,0xf9,0x13,0xc7,0x48,0x3b,0x3e,0x6e,0xe2,0xf2,0x99, -0x66,0xba,0x46,0x27,0x0c,0xc3,0x56,0x18,0xc1,0xff,0x5f,0x85, -0x61,0x97,0xfe,0xdf,0x86,0x24,0x40,0xda,0x2f,0x51,0xf4,0x23, -0xbd,0xbd,0xdc,0x2d,0xa5,0x07,0x35,0x90,0x6d,0xe2,0xa9,0x15, -0x65,0x99,0x43,0xb4,0xc9,0xaa,0x00,0x56,0x64,0x9a,0x0c,0xc9, -0x51,0x6b,0x81,0x34,0xfb,0x71,0xf4,0x23,0xdb,0xfc,0x7d,0xf2, -0x7d,0xe4,0xb5,0x57,0x5d,0x35,0xab,0x39,0x37,0x2c,0x2e,0x01, -0xe8,0x2c,0x6d,0x97,0xf3,0x40,0x39,0x74,0x0b,0xc6,0xde,0x66, -0x88,0x92,0x33,0x7b,0x25,0x87,0x69,0xa7,0x90,0x56,0x5a,0xcb, -0x95,0x16,0x85,0x28,0xaa,0xce,0x2a,0x1a,0xe7,0x46,0x17,0x59, -0xed,0xb2,0xb0,0xb0,0x77,0x60,0x76,0xe3,0xfd,0x42,0x8b,0xac, -0x15,0x0e,0x7d,0x17,0x96,0x2d,0x6a,0x63,0xd7,0x59,0xe3,0x30, -0x60,0x79,0x8d,0x73,0x43,0xaa,0xe1,0x53,0x78,0x84,0x1b,0x06, -0x45,0x1e,0x41,0xda,0xa7,0xf1,0xa8,0x80,0xec,0x50,0x1e,0x99, -0x73,0x11,0xbf,0xd0,0x22,0xc3,0x23,0xfe,0xae,0x6c,0xb6,0xcb, -0x23,0x01,0x2c,0xf3,0xe8,0x1a,0x94,0xf4,0x41,0xbb,0x55,0x08, -0x80,0x73,0x59,0x5c,0xb7,0x8f,0x72,0xfa,0x38,0xec,0x55,0xb0, -0xf2,0xec,0x71,0xf2,0xe8,0xc4,0x2b,0x32,0x13,0x53,0x5b,0x0a, -0x25,0x5e,0xed,0x96,0x5f,0x9f,0xc4,0x60,0xaa,0xe2,0x13,0x38, -0xac,0xce,0xba,0x7c,0x87,0x1b,0x86,0xbb,0xf8,0x55,0xc1,0x2e, -0x97,0xb3,0x04,0x54,0x21,0x7b,0x38,0x63,0xfd,0x41,0x3b,0xa4, -0xa4,0x5a,0x0e,0xdc,0x1e,0x65,0xa3,0xb5,0xb0,0x33,0x78,0xe8, -0x96,0x9e,0x3a,0x57,0xf3,0xdd,0x66,0x58,0xf2,0x46,0x67,0x17, -0x15,0x2d,0x2d,0x48,0x1b,0x81,0x69,0xa6,0x38,0xf3,0x02,0x79, -0xe7,0xe3,0x2b,0x8c,0x59,0x3c,0x00,0xed,0x11,0x3c,0x4a,0xf0, -0x21,0x6b,0xf8,0xe7,0x0c,0x8c,0x27,0x18,0x2a,0xf0,0xf7,0x51, -0xf0,0x14,0xfe,0x8b,0x3e,0x87,0x7f,0xb0,0x6b,0xf1,0x59,0x6b, -0xf8,0xb7,0x13,0xe1,0xef,0x87,0x67,0x1d,0xf8,0x97,0xf6,0xe5, -0x82,0xe8,0x49,0x07,0x33,0x1e,0x87,0x0f,0x29,0xff,0xd1,0xd3, -0x4e,0xf2,0x38,0x78,0xf2,0xe4,0xc9,0x63,0xf8,0x73,0x16,0x3d, -0x82,0xb2,0x8f,0xcd,0xb6,0x38,0xd5,0x4b,0x73,0xee,0x0f,0xc9, -0xea,0x7a,0x3e,0xc1,0x73,0x67,0x57,0x83,0x04,0xd1,0x30,0x28, -0x26,0x3d,0x2a,0x27,0x45,0x15,0x69,0x67,0xa1,0x95,0xc6,0x63, -0xcd,0xc1,0x26,0x49,0x8f,0x2a,0xa0,0x2a,0xd2,0x1c,0x6c,0x28, -0x5f,0x0e,0x2e,0x4a,0x38,0x2b,0x26,0x3c,0x2e,0x15,0xe9,0x58, -0x29,0x68,0x98,0x44,0x85,0x6f,0x3b,0x1f,0x77,0x83,0x5c,0x82, -0xc9,0x4c,0x2c,0xa7,0xd8,0xf5,0xe2,0x26,0x58,0x10,0x0d,0xab, -0x58,0xfc,0xad,0x2c,0xfa,0x35,0x8b,0x95,0xc5,0x0e,0x33,0x7f, -0x43,0x82,0xc2,0xad,0x13,0x83,0xcb,0x64,0x7f,0xd5,0xf0,0x2b, -0xef,0x40,0xda,0xe2,0x59,0x55,0xee,0xf2,0xc1,0x67,0x93,0xea, -0xa2,0xd1,0x51,0x6c,0xad,0x88,0x2b,0x0b,0x8f,0x6a,0x0b,0x9b, -0xa5,0xce,0xf1,0xf1,0x01,0x88,0x9e,0x3f,0xf8,0x6c,0xb1,0x03, -0x91,0x52,0x38,0x95,0x45,0xc1,0x10,0xda,0xd5,0x76,0x3d,0x1b, -0x54,0x14,0xc6,0x93,0xb1,0xfa,0xa2,0x66,0xb2,0xad,0x28,0x0a, -0x86,0xd7,0xce,0x5a,0x9d,0x53,0xfd,0x1d,0x7d,0x1d,0x3b,0x19, -0xdf,0xba,0xeb,0xb3,0x92,0x60,0x18,0x4b,0xaf,0x34,0x2c,0xdb, -0xa5,0x1d,0x7e,0x33,0xf9,0xbb,0xc1,0x1a,0x4a,0x0b,0x33,0xca, -0xf7,0xca,0x21,0x26,0x6a,0xeb,0xaa,0x6d,0x8d,0xbf,0x2d,0xeb, -0x2c,0x9b,0x17,0xf1,0x47,0x52,0x6c,0xd6,0x4b,0x44,0xe8,0x1a, -0x19,0x64,0xab,0xf9,0x02,0xfe,0x49,0xec,0x35,0x23,0xab,0x40, -0xaf,0xc9,0x10,0x4d,0x03,0xa2,0x5e,0xb0,0x37,0x35,0xd0,0x2a, -0xd7,0x5a,0x12,0x52,0x22,0x9e,0x18,0x8b,0xc2,0xaa,0xe7,0x19, -0xa2,0xd2,0xb4,0xdc,0xa8,0xda,0x07,0x37,0x83,0x90,0xf6,0xbd, -0x83,0x1b,0x74,0x5c,0xdf,0xfa,0x25,0xcc,0x75,0xe3,0x54,0x8c, -0xf4,0x1d,0x43,0x95,0x21,0x76,0x8d,0x56,0xcb,0x4f,0xa3,0xa6, -0xf4,0xfe,0x31,0x6b,0x9c,0x64,0x6a,0x50,0xec,0x19,0xb9,0xca, -0x6d,0xc6,0x0c,0xde,0xbd,0x18,0xf7,0x0c,0x61,0xcb,0x45,0xa6, -0x0e,0xc1,0xee,0x81,0xec,0x78,0xbc,0x54,0xa3,0xd8,0x3d,0x9c, -0x1d,0x67,0x96,0x6a,0x04,0xbb,0x07,0x75,0xc9,0x5b,0xe7,0x5f, -0x4a,0x03,0x5b,0x8b,0xc6,0xfe,0x61,0x6d,0xa4,0xc8,0x89,0x82, -0x5f,0x21,0xbf,0xe5,0x91,0x4d,0xb2,0xe9,0x8c,0x6e,0x32,0x23, -0x7e,0xe7,0x08,0x37,0x35,0xd6,0x34,0x4a,0xde,0x5e,0xc2,0x30, -0x73,0x00,0xb0,0xb9,0x5e,0xe2,0xdb,0x5d,0x66,0xfc,0x2c,0x93, -0x5f,0xd7,0x49,0xb6,0xd2,0x95,0xcb,0x37,0x5e,0xb8,0x58,0xcc, -0x67,0x59,0xf2,0x0a,0x0a,0xd2,0x58,0x6a,0xbf,0xcb,0xec,0x60, -0x9e,0xeb,0xe5,0x34,0xc0,0x90,0x38,0x78,0xa9,0xdb,0xa6,0x1c, -0x2a,0xa0,0xbc,0x06,0xba,0x57,0xa4,0x63,0xda,0x74,0x3e,0xc5, -0xa2,0xe4,0xcd,0x80,0x3f,0x4c,0x31,0xa2,0xec,0x7a,0x75,0x33, -0xbd,0x07,0x5a,0x6c,0xc8,0x29,0x96,0x21,0x7c,0xf8,0xa3,0x80, -0x6f,0x73,0x33,0xfd,0xa4,0x86,0xbe,0xfe,0xe1,0x7b,0x6a,0xe7, -0xd6,0xf3,0x8b,0xbe,0xf8,0x37,0xf3,0xc9,0x1a,0xdf,0x7d,0x9e, -0xbd,0x5d,0x4f,0xa1,0x1f,0xf9,0x13,0x43,0x11,0x80,0x9a,0xe3, -0xb4,0x46,0x80,0x11,0x45,0x39,0x03,0xd7,0xef,0x18,0x04,0x64, -0xe9,0x35,0x7e,0xb6,0x21,0x0a,0x48,0x9f,0xcd,0x67,0x57,0xe9, -0xdb,0xf5,0x32,0x59,0xc6,0x1f,0xb7,0x3d,0xf3,0xd5,0x4e,0x67, -0xa9,0xe5,0xbd,0x3c,0xa7,0xb8,0xa2,0xc1,0x98,0xf2,0xad,0x1b, -0xd2,0x16,0x6a,0xf8,0x49,0xcf,0x7c,0x7a,0x0c,0x14,0xfc,0xe9, -0x7a,0xb5,0x5a,0x04,0xc4,0x49,0x8d,0x55,0x1e,0x8e,0x13,0x48, -0xac,0x51,0xd8,0x70,0xc1,0x9e,0xf8,0x17,0xfc,0x7a,0x9b,0x55, -0xc0,0xd0,0x0b,0x39,0x5c,0x9c,0x23,0x0c,0x1b,0x98,0x01,0xe4, -0x0c,0x7b,0x1c,0x9d,0x78,0x31,0x5a,0x8e,0x6e,0xb2,0xf8,0x82, -0xdd,0xaf,0x27,0xde,0x47,0xf1,0x13,0x93,0x0c,0x21,0xa5,0x3d, -0xe1,0x3b,0xcd,0x3f,0x33,0xf3,0x5f,0x50,0x1e,0xbf,0x64,0xd1, -0xbe,0x21,0x31,0x76,0x8f,0x92,0x87,0x7e,0x70,0xd1,0x4e,0xb3, -0x6f,0xc8,0x41,0xd5,0xc6,0x87,0x4f,0x0e,0xf1,0xf3,0xdb,0x4e, -0x2d,0xaa,0x81,0x44,0x99,0x50,0x0a,0xe5,0x5f,0x8e,0xae,0x12, -0xcf,0xae,0xc5,0xef,0x97,0x87,0x1d,0xb1,0xcd,0xd3,0x0d,0xe0, -0x30,0xd3,0x1f,0x81,0x8d,0x5d,0xf8,0x0f,0x1d,0xf9,0xbb,0xf6, -0x7a,0x62,0x74,0x58,0x39,0x5c,0xc6,0x8c,0x70,0x27,0x64,0x84, -0x18,0xb6,0xbe,0xa6,0x90,0x1f,0x90,0x05,0xba,0xb4,0x81,0xde, -0x00,0x6d,0x85,0xa1,0x23,0x92,0x11,0x7a,0xdc,0xcc,0x17,0x74, -0x5c,0x83,0x31,0x3c,0x97,0xa3,0x71,0xd2,0x18,0xf6,0x9c,0xd6, -0x58,0x02,0xb2,0x48,0x38,0x68,0xac,0x26,0x08,0xa3,0xa5,0xcf, -0x56,0xa3,0x74,0x96,0x79,0x56,0x05,0x81,0x06,0x74,0x59,0x2c, -0xb3,0xff,0xe8,0x32,0x9b,0x4f,0x41,0x51,0xbf,0x18,0xad,0x60, -0xf5,0x36,0x8b,0x4f,0xff,0x8e,0xcd,0xca,0xfa,0xdd,0xbf,0x9d, -0xfe,0xed,0xf4,0x34,0x35,0xb5,0x7f,0x29,0x80,0x7f,0x5e,0x4e, -0xcb,0x9b,0x44,0x2a,0xa6,0x97,0x6c,0x59,0x15,0x90,0xf2,0xa5, -0xb4,0x4c,0xc5,0xfc,0x17,0x94,0x97,0x40,0x06,0x22,0xc3,0x8e, -0xfe,0xf3,0x0c,0x04,0x04,0xf4,0xdc,0xc4,0x73,0x33,0xfd,0x7e, -0xa3,0xd1,0x75,0x93,0x02,0x1e,0x19,0xb8,0x0c,0xfc,0x4a,0x30, -0x68,0x72,0x60,0x49,0x2e,0x69,0x9a,0xa4,0x42,0x65,0xd0,0x28, -0x75,0x79,0xce,0x02,0xee,0x9b,0xdf,0xed,0x6c,0x7d,0xc9,0x94, -0x7a,0x61,0x60,0x25,0xf3,0x3d,0x81,0x56,0xe4,0x77,0x4d,0x62, -0xc0,0xa1,0x90,0xa5,0x0a,0x90,0x83,0xe5,0xe8,0xdb,0x34,0x99, -0x42,0xa7,0x96,0x93,0xee,0xee,0x06,0x43,0x8b,0xf4,0x6f,0x2c, -0x60,0x9b,0x7c,0x2b,0xbd,0xd8,0x04,0x1b,0xbf,0x0b,0xe9,0xd2, -0x21,0x43,0xed,0x3b,0xe8,0x46,0x97,0x9a,0x52,0xc6,0xdd,0x1d, -0x8c,0x56,0x43,0xd3,0xd7,0xa5,0x82,0x9a,0x32,0x12,0xee,0x12, -0x45,0xe5,0x9a,0x6e,0x25,0xb8,0xae,0x4d,0xcf,0x87,0x74,0x75, -0x8d,0x20,0x82,0x3e,0x73,0x74,0x9c,0x25,0xbc,0x82,0x2d,0xc3, -0xe4,0xda,0xb6,0x18,0xf1,0x49,0x66,0xe3,0xf9,0x24,0x79,0x3e, -0xc9,0xaa,0x05,0x48,0x67,0xfb,0xfd,0xa3,0xb0,0x5b,0x4c,0xb4, -0x7b,0x42,0xe3,0xd1,0x74,0x31,0x98,0xff,0xb1,0x54,0x13,0xff, -0x2a,0x72,0xda,0x51,0x6a,0xf1,0x8e,0x3c,0xe0,0x37,0x8c,0xf5, -0x2e,0x30,0x7d,0x31,0xcf,0xf8,0xef,0x9a,0xfe,0x70,0xbc,0x1d, -0xfc,0x35,0x9e,0xdf,0xc0,0x1a,0x14,0x7e,0x55,0x74,0x8c,0x5b, -0x8f,0xa6,0x96,0x54,0x60,0x14,0xd0,0x1f,0x09,0xb1,0x7e,0xa3, -0x74,0xcb,0x30,0x10,0xfd,0xcc,0xb9,0x77,0x77,0x0c,0x6c,0x26, -0x01,0x9b,0x73,0x82,0xa1,0x2f,0xa5,0x39,0xb2,0x74,0x83,0x29, -0x6a,0xf8,0x5d,0x04,0xe6,0x2b,0x66,0x8c,0x44,0x03,0x32,0xf2, -0xa8,0xeb,0x94,0x13,0x98,0x40,0xe6,0x19,0xc9,0x33,0x93,0x8b, -0xe8,0xe3,0x8f,0xbb,0xe6,0x07,0x86,0x19,0x32,0xfe,0x6c,0xeb, -0x8b,0x5c,0x09,0x63,0x96,0x87,0xb2,0xbd,0x28,0x4c,0xa0,0x68, -0xf1,0x0c,0xaa,0x32,0xb5,0x66,0x48,0x48,0x11,0x4d,0xfd,0x35, -0x27,0xd4,0x0d,0x08,0x01,0x17,0x28,0x85,0xef,0x90,0xfa,0xdd, -0x26,0x1e,0x54,0x42,0x52,0x99,0x59,0x3f,0xdd,0x26,0xcb,0x65, -0x6a,0x43,0x17,0xd3,0x0b,0x8a,0xe8,0x87,0x62,0xb1,0xd2,0x98, -0x47,0x81,0x9a,0x9b,0x7c,0x3d,0xe1,0x0d,0xe4,0x79,0x18,0x63, -0x12,0xeb,0xb9,0x42,0xa1,0x9b,0x31,0x76,0xaf,0xc1,0x73,0x76, -0x23,0x30,0x78,0x60,0x22,0x37,0x1f,0x22,0x6a,0x3c,0x0c,0xc0, -0xa8,0xaf,0x6b,0x92,0x29,0xe1,0xaa,0x98,0x52,0x8d,0x71,0x41, -0xce,0x02,0xbb,0x31,0x59,0x82,0xde,0xa9,0xa2,0xab,0xee,0xee, -0x6a,0x2a,0x93,0x46,0x1d,0xb9,0x63,0xe4,0xa2,0x7d,0x85,0x37, -0x51,0x18,0x85,0x91,0x66,0xdc,0xd5,0x55,0xb2,0x00,0xbf,0xdd, -0x99,0x36,0x8e,0xe3,0x2a,0x1b,0x87,0x9e,0x5a,0x90,0xca,0xc1, -0x62,0x78,0x86,0x11,0xe4,0xe7,0xcb,0xb8,0x94,0x72,0x77,0xc7, -0xd1,0xad,0xad,0x4e,0xfb,0xb3,0x01,0x37,0x93,0x07,0x3d,0xfc, -0x93,0x5e,0x79,0x40,0xf1,0xf5,0x48,0x99,0x74,0x16,0x9e,0x6f, -0x47,0xf8,0x4e,0x24,0x3f,0xf1,0xe3,0xfb,0xab,0xeb,0xe5,0xfc, -0x03,0xed,0x61,0x7f,0xb3,0x5c,0xce,0xc1,0x66,0xfd,0xf3,0xcf, -0xdf,0x3f,0x80,0xf9,0xfa,0xfa,0x01,0x5f,0xa7,0x4d,0x26,0x0f, -0xd2,0x6c,0xf6,0xff,0x5f,0xa1,0x9d,0x95,0xc2,0xea,0xb3,0xd0, -0xc7,0x16,0xc1,0xfc,0x2e,0x8f,0xdd,0x1d,0x96,0x81,0xec,0xce, -0x40,0xa5,0x0c,0x18,0x6e,0xe9,0xa4,0xdb,0x80,0x0a,0x82,0x25, -0x46,0xed,0xeb,0x36,0xe8,0x4f,0x23,0xe0,0x67,0x5e,0x94,0x85, -0xdb,0x6d,0xb8,0xdf,0x8d,0xc0,0xc2,0xf4,0x6c,0x3e,0x55,0x6f, -0xd1,0x36,0x2a,0x93,0x1b,0xb0,0x52,0x98,0xcd,0xe6,0x33,0x58, -0x95,0x4c,0x9f,0x43,0x6d,0x17,0x17,0xf6,0x77,0x23,0x48,0x56, -0xa3,0xb7,0x4e,0xd1,0x6f,0x20,0x01,0x2f,0xfa,0x4c,0xaf,0xbe, -0x4f,0x67,0xef,0xbb,0x8d,0xeb,0x65,0x72,0xd5,0xc0,0x5b,0x12, -0x5d,0xb6,0xd5,0xe0,0x9f,0xef,0xd3,0x8c,0xbf,0xf0,0x47,0x83, -0xf4,0x78,0x03,0xfe,0x69,0xb0,0x66,0x6f,0xe0,0xbf,0x0d,0xa5, -0xd6,0x95,0x5c,0x07,0xa8,0x0f,0xba,0x62,0xe8,0x91,0x75,0xd7, -0x15,0x23,0x2f,0x10,0xab,0xaf,0x6b,0xcc,0x3f,0xba,0x42,0x8e, -0xed,0x5e,0xa1,0x23,0x35,0x54,0x64,0x19,0xfe,0xd0,0xfd,0x54, -0xb7,0x9b,0x44,0x54,0x7c,0x33,0x4d,0xd0,0x9d,0x9d,0x88,0x91, -0xdf,0x8d,0x60,0x34,0x99,0x58,0xa0,0x3c,0x42,0xba,0x8d,0xaa, -0x54,0xaa,0xe9,0x05,0xf1,0x5a,0xb7,0xd0,0x7c,0x02,0x23,0xa7, -0x73,0xbc,0x10,0x45,0x7f,0x1a,0x41,0x3a,0xc9,0xb0,0xef,0xb2, -0x86,0xad,0x32,0x7f,0x2e,0x75,0xbe,0xb5,0x14,0xcb,0xaa,0x4d, -0x9b,0xb2,0xc0,0x68,0x2d,0x53,0x07,0x11,0x18,0x64,0xae,0xf0, -0xa1,0x3b,0x31,0xa6,0xbf,0x9a,0x63,0xfb,0xed,0xc7,0x41,0x21, -0x31,0x00,0xd9,0xbe,0x09,0xac,0x37,0x5e,0x16,0xfc,0x54,0x53, -0x8a,0xb1,0x3f,0x11,0xa0,0x9d,0xc1,0xea,0x15,0x77,0x0c,0x41, -0x0b,0xa5,0x13,0x0a,0xe9,0x19,0x63,0x99,0xd2,0xb2,0x09,0x57, -0x6b,0xe9,0x68,0xea,0x19,0x04,0xd6,0xca,0x09,0x13,0x61,0x2c, -0x72,0xf9,0x01,0x7e,0x0d,0xf1,0xc6,0x96,0x42,0xe8,0x64,0x6c, -0x75,0x45,0x83,0x8b,0xf6,0x74,0x04,0x86,0xa9,0x85,0x72,0xc8, -0x5a,0xca,0x6d,0xe1,0x5b,0x69,0xe1,0xb7,0xcb,0xf9,0x4d,0x5d, -0x1b,0x0f,0x6f,0x9d,0x5a,0xdd,0xd2,0x03,0x0c,0x54,0xb4,0xd0, -0x56,0x83,0xa4,0xa6,0x81,0xf8,0xf2,0x59,0x55,0xcb,0x7c,0xd3, -0x34,0xbb,0x77,0x9e,0x97,0xba,0x86,0x3a,0x25,0x9d,0x14,0xa5, -0xc2,0xed,0xc9,0x5a,0x49,0xc0,0xd7,0x42,0x14,0x86,0x12,0xa3, -0x9e,0x57,0x70,0x89,0xf9,0xe3,0xd6,0x55,0xe4,0xe9,0xde,0xda, -0x7c,0x6b,0xf2,0xf9,0x05,0x55,0xe4,0xf3,0x89,0x5b,0xc3,0x73, -0x13,0x49,0xae,0x71,0x14,0xc7,0x9c,0x74,0x7c,0x5c,0x98,0x4f, -0x04,0x52,0xd2,0xe9,0xf9,0x3b,0x49,0xb2,0x59,0x06,0xa3,0xbb, -0x92,0x67,0xeb,0xf2,0x42,0xe7,0x50,0xa6,0x29,0xed,0x16,0x68, -0x44,0x25,0xde,0x41,0xad,0x7f,0x34,0xf3,0x9c,0x5a,0xad,0xe9, -0x2f,0x4b,0x9e,0x59,0x2a,0xb9,0xda,0xa2,0x2f,0x00,0x81,0x5d, -0x1f,0x75,0xab,0xb3,0xec,0x99,0xb2,0x80,0xd9,0xb5,0x6e,0x8a, -0x0d,0x29,0xd2,0x51,0x3d,0xf8,0x6c,0x90,0x3a,0x06,0x51,0x6c, -0x21,0x1b,0x15,0x26,0x0f,0x6a,0xf9,0x62,0x83,0x0e,0x03,0x98, -0xa8,0xd7,0x54,0xa8,0x28,0x61,0xde,0xd8,0x61,0x80,0x33,0xad, -0x55,0xc9,0xbc,0x3b,0x9e,0x15,0x5a,0x7b,0x9e,0xa6,0x2d,0x30, -0x5a,0x4a,0x8e,0x2d,0xe3,0xa3,0x94,0x71,0x77,0x57,0xbd,0x07, -0x42,0x1b,0x1c,0xae,0xf2,0x2f,0x60,0x34,0x7c,0x51,0x49,0x15, -0xca,0xbf,0x50,0x46,0x83,0xba,0xd6,0xb0,0x85,0xff,0x39,0x1e, -0x35,0x8f,0x93,0x05,0x42,0xef,0xa8,0x5d,0x35,0xf4,0x0a,0xc6, -0x95,0x2c,0x09,0xec,0x92,0x3b,0x73,0xad,0x26,0x27,0x3c,0x91, -0x9a,0x9d,0x14,0x8a,0xab,0x4e,0x3b,0x6d,0xca,0xc4,0x97,0xdd, -0x29,0x69,0xda,0x47,0x29,0xd1,0x55,0x25,0x05,0xac,0xeb,0x82, -0x77,0x65,0x61,0xe3,0xb6,0xb0,0x44,0xa6,0xa6,0x23,0x35,0x89, -0x45,0x26,0xd6,0xb4,0xd0,0x55,0x17,0x87,0xd3,0xbf,0x93,0xde, -0x40,0xb5,0xce,0xa2,0xa7,0xae,0x02,0x3c,0xe9,0x29,0x76,0xe2, -0xb7,0x7b,0x68,0xbd,0x7f,0x33,0xad,0x12,0x85,0x3d,0x19,0xb4, -0x6d,0x2b,0xba,0xbc,0x98,0x6e,0x75,0xb6,0xef,0x2c,0x00,0xbf, -0x29,0x22,0x38,0x84,0xca,0x52,0xad,0xb5,0xf4,0xcd,0x67,0x5f, -0x25,0x57,0xf3,0x65,0x82,0x83,0xd5,0x32,0x9c,0xd2,0xdf,0x12, -0x3d,0xf8,0x77,0x80,0x14,0x44,0x54,0x93,0x81,0x1f,0x76,0x23, -0x7e,0xda,0x51,0x8b,0x99,0xd2,0xe7,0xd6,0x26,0xf9,0x01,0xe4, -0x21,0x7c,0xb1,0x31,0x3b,0x9b,0xf1,0xbb,0x1a,0xf0,0x09,0xa4, -0xef,0x21,0xda,0x60,0x47,0xd3,0x16,0xcf,0x0c,0x9e,0xf3,0xf6, -0x46,0xec,0x88,0x82,0x9a,0x56,0xaf,0xe7,0xeb,0xe9,0xe4,0xe5, -0xe8,0x36,0x61,0x73,0x38,0xae,0x49,0x77,0x04,0x49,0x56,0x90, -0xa1,0x5d,0x19,0x83,0x4d,0x93,0xac,0x7e,0x77,0xcd,0x6c,0xb4, -0x48,0x46,0xbf,0x8e,0x08,0x63,0x59,0xe3,0xaa,0x49,0x57,0x68, -0x6d,0x04,0xcb,0x86,0x1c,0xe7,0x6f,0x69,0x0f,0xe7,0xab,0xf9, -0x7c,0x9a,0x8c,0x74,0xad,0x18,0xa8,0x6b,0x1f,0x7a,0x8d,0x99, -0xcb,0x6c,0x8b,0x26,0xf7,0xfa,0xea,0x2a,0xdd,0x54,0xcf,0xda, -0x9c,0x27,0x9e,0x81,0x4e,0x5a,0x59,0xe9,0xbd,0x64,0x3c,0xf6, -0x7e,0x28,0x27,0x95,0x8c,0x1c,0x06,0xd4,0xf9,0x2e,0x39,0x3a, -0xac,0xaf,0xb5,0x0b,0x62,0xa7,0xd9,0x7b,0x3c,0xb0,0x08,0x92, -0x65,0xd2,0x2b,0x03,0x51,0x78,0xe4,0x90,0x8f,0x10,0xbf,0x5c, -0xbe,0xc5,0x6a,0x96,0xf8,0x43,0x1e,0x32,0xcc,0xcc,0x1a,0x93, -0x03,0x28,0x59,0xb5,0xf0,0x2b,0x0c,0x1d,0x58,0xfb,0x97,0x62, -0xcd,0xd8,0x50,0x1a,0x77,0xd7,0xb3,0x93,0x55,0x45,0x81,0x53, -0x87,0x06,0xe6,0x68,0xd1,0x48,0xdc,0xab,0x3d,0x6d,0x1d,0x20, -0xd0,0xb0,0x57,0x04,0xc5,0xf7,0x00,0x0f,0x2e,0x8d,0xaf,0xe6, -0x05,0x45,0x70,0xde,0xb2,0xd1,0x8c,0x1a,0xcf,0x31,0xc2,0xa5, -0x3d,0x86,0xb5,0x3d,0x6b,0x37,0xc1,0xc7,0xed,0x16,0x7c,0x6d, -0xcd,0x4e,0xb4,0x19,0xc2,0x7a,0x80,0xe6,0x4d,0x0c,0xa0,0x24, -0xbd,0xc4,0x6b,0x41,0x8b,0x13,0xae,0xe0,0x07,0x57,0xe6,0x10, -0x64,0x57,0xbf,0x0a,0xf4,0x51,0x88,0x05,0x0a,0xc8,0x7f,0x98, -0x4f,0x92,0xe9,0xef,0xc4,0x1b,0x31,0xde,0x22,0x23,0xab,0x16, -0x3b,0x56,0xf3,0x79,0x70,0x06,0x96,0xaa,0x12,0x5f,0xc4,0x43, -0x3a,0x88,0xca,0x0e,0x03,0x09,0x95,0x48,0x35,0xd9,0xcb,0xd5, -0x22,0x8e,0xe3,0x63,0x59,0xd4,0x15,0x33,0xac,0xd8,0x5f,0x26, -0xd5,0xff,0x68,0xe3,0xb5,0x7b,0xc9,0xa1,0xdf,0x02,0xc2,0x25, -0xdf,0x2e,0x05,0xec,0x59,0xb0,0xfb,0x98,0xb0,0x75,0xad,0x37, -0x36,0xed,0xaa,0x95,0x8c,0x0d,0x61,0xaf,0x0b,0xec,0xf4,0x92, -0x01,0x22,0x08,0xcd,0xe2,0x19,0x63,0xd9,0x54,0xda,0x1b,0x1a, -0x72,0x5a,0x38,0xf2,0x29,0x6d,0xae,0xe1,0x01,0x2b,0xf6,0x1d, -0x1e,0x10,0x95,0x76,0xe8,0x60,0x3a,0xe9,0x59,0x19,0x6d,0x27, -0x34,0x01,0x1f,0x04,0xc7,0xd6,0x80,0xe2,0x83,0x5f,0x3b,0xc2, -0x06,0xa7,0x89,0x1c,0x08,0x25,0xd5,0x08,0x79,0xbb,0xac,0x38, -0x9f,0x8d,0xd7,0x4b,0x4c,0x36,0xef,0x71,0x0a,0x18,0x0c,0xf0, -0x9e,0xe4,0xf5,0x7c,0x49,0xe3,0xf1,0xad,0x4a,0x04,0xf2,0x23, -0x96,0xbf,0x03,0x8b,0x9e,0x8a,0xe5,0x8c,0xbb,0x5b,0xa7,0xa3, -0x10,0x2a,0xd4,0x18,0xe6,0x7a,0x49,0x5b,0xa0,0x35,0xe4,0xeb, -0xf3,0xea,0x22,0xe9,0x72,0xc6,0x8d,0x27,0xff,0x7a,0xfb,0x1b, -0x7e,0xcb,0x3e,0xff,0x87,0xeb,0xd1,0x8a,0x76,0xf3,0x02,0xeb, -0x70,0x54,0xda,0xc9,0x87,0xd4,0xfa,0x8c,0xc9,0x94,0xa2,0xd9, -0xc0,0x6e,0x4e,0xd5,0x11,0x42,0x9b,0x4f,0x41,0x7c,0x65,0x2f, -0x17,0x51,0xd9,0xa7,0x07,0x15,0xb8,0x24,0xdb,0xef,0x21,0x71, -0x66,0xc2,0x95,0x03,0x6a,0x43,0x6c,0x5f,0xd0,0x0f,0x1a,0xcf, -0xaf,0x5a,0x3f,0xce,0x67,0x49,0xeb,0x07,0xda,0xff,0x1b,0xc6, -0xb4,0x47,0x69,0xe7,0xda,0x19,0x3c,0x1d,0x80,0x30,0x72,0x4c, -0x45,0x3c,0xeb,0xd4,0x3d,0xa7,0x78,0x0f,0xb9,0xcd,0x18,0x19, -0xd4,0x6f,0x9c,0x36,0x9a,0xf8,0x03,0x2f,0x33,0x52,0xaa,0x4d, -0x30,0xcf,0xac,0x77,0x77,0x90,0x57,0xe5,0x69,0x60,0x81,0x5a, -0xee,0x06,0x1f,0xd5,0x86,0xa8,0x8d,0xa9,0x78,0xec,0xe7,0x7d, -0xbc,0x91,0x8d,0xc7,0x7f,0xfd,0xe6,0x55,0xa3,0xb0,0xa2,0x28, -0xac,0x37,0x40,0x8b,0xe0,0x6e,0xeb,0x1f,0x88,0x0e,0x37,0x68, -0x0f,0x43,0xf7,0xe2,0xcf,0x87,0xa0,0x9b,0x1f,0xdc,0xda,0x17, -0x3f,0xbd,0x3c,0x00,0xa1,0x6c,0x15,0x1f,0x86,0xf2,0xeb,0x6f, -0xbe,0xff,0xe6,0xd5,0x37,0xfb,0x91,0xd2,0x7e,0xf3,0x61,0x28, -0xbf,0xfb,0xe6,0xcb,0xaf,0xf7,0x23,0xe4,0x4d,0xeb,0xc3,0x30, -0xbe,0xfa,0xf9,0xcb,0x67,0x07,0xd0,0xa8,0x76,0xbe,0x0f,0x43, -0xfa,0xd3,0x8b,0x57,0xcf,0x7f,0xfa,0xf1,0xe5,0x01,0x1d,0x44, -0xdb,0xe7,0x07,0xf6,0xd0,0x97,0xaf,0x9e,0x7d,0xb7,0x17,0xe5, -0x56,0x1c,0x26,0xf0,0x8c,0xc4,0xd5,0xe9,0x98,0x62,0x05,0xc6, -0xc2,0x33,0x15,0x4b,0xad,0x05,0x6e,0x36,0x8d,0xce,0x0a,0x95, -0x8c,0xa8,0x2f,0x2e,0x90,0x60,0x1a,0x8f,0x66,0x0f,0x76,0x99, -0x4c,0xd6,0x6a,0xe4,0xd9,0x6a,0xdd,0xe8,0x66,0x8d,0x6c,0x34, -0x5e,0xdf,0x58,0xbb,0xbf,0xf8,0x0b,0xbd,0x14,0xf0,0xef,0x4b, -0xd9,0x78,0x8b,0xb9,0x8e,0x76,0xe5,0x2e,0x9f,0x6c,0x17,0xa5, -0x57,0x9e,0x5d,0x84,0x0e,0x95,0xdc,0x62,0x8e,0x37,0x88,0x0b, -0xec,0x5b,0x4b,0x38,0x95,0xd8,0x13,0x4a,0x62,0x3b,0x71,0xab, -0x02,0x1c,0xdb,0x99,0x03,0xb7,0x9e,0xf2,0xcc,0x22,0xf6,0xce, -0x61,0xb0,0x55,0x87,0x41,0x43,0xf5,0x54,0x79,0x76,0x58,0x85, -0x00,0x38,0xec,0xc1,0x3f,0xa0,0xb5,0x85,0xd0,0x66,0x8c,0x1a, -0x14,0x92,0xda,0xef,0xe6,0xe9,0xcc,0x6b,0x04,0x0d,0x9f,0x9f, -0x9f,0xd3,0x4c,0x7f,0x3e,0xe9,0xf1,0x9f,0x02,0xb7,0x0b,0xdb, -0x8b,0xfd,0x52,0x5f,0x54,0xed,0x2c,0x8a,0x45,0x5c,0x82,0x2d, -0x42,0x94,0xb6,0xa3,0xcd,0xde,0xb2,0x4b,0x79,0xa1,0x2f,0xb5, -0x77,0x43,0x9f,0x7f,0xfd,0xf9,0xe7,0xe7,0xcf,0xe6,0x37,0x60, -0xf1,0x80,0x6c,0x29,0x14,0x5d,0xf9,0xab,0x9f,0x98,0x46,0x49, -0x6b,0x22,0x32,0xc1,0xec,0xce,0x7a,0xca,0x79,0x67,0x5b,0x94, -0xfd,0xab,0x04,0x46,0xa5,0xe3,0xb1,0xa3,0xe6,0x75,0x9c,0x90, -0xb8,0x67,0x94,0xa7,0x4e,0xfd,0x54,0x86,0xb0,0xd0,0x28,0x01, -0xe4,0x46,0x51,0xf9,0x40,0x92,0xea,0xed,0xb3,0x36,0x3d,0x37, -0x8c,0x54,0xf1,0x60,0x7e,0x3b,0x9d,0x5f,0x8e,0xa6,0x6c,0x7b, -0xa9,0x77,0xf0,0xcb,0x5e,0x79,0x1c,0x00,0xb9,0x02,0x96,0xd7, -0xc1,0xed,0x3f,0x61,0x8c,0xf8,0x41,0x83,0x26,0x44,0x10,0x87, -0x3f,0xfd,0x6a,0x39,0xfd,0xf1,0x2c,0xf9,0xa7,0x5f,0x2d,0x17, -0x3e,0x7e,0x2b,0xfc,0x65,0xb2,0xbc,0x4d,0xc7,0xe8,0x6d,0xea, -0xa0,0xd4,0x86,0x9f,0x06,0x5f,0x3a,0x26,0x34,0x6a,0x16,0x8f, -0xf5,0x00,0x6f,0xdd,0xcb,0xd2,0x5f,0x06,0x52,0xfd,0x16,0x33, -0x8f,0x9c,0x98,0xed,0xec,0xdd,0xa0,0xa5,0xa3,0xc5,0x21,0xd8, -0x3b,0x14,0x81,0x11,0x18,0xfa,0xdd,0x68,0x36,0x99,0x02,0x63, -0x54,0x5f,0x06,0x26,0x8d,0x75,0xcf,0x1e,0xe4,0x55,0xc7,0x8e, -0x1a,0x7f,0x55,0xe6,0xb7,0xc2,0x88,0x83,0xb0,0xd3,0xc9,0xa4, -0x46,0x87,0x67,0x5a,0x85,0x15,0x88,0xac,0xde,0x98,0x77,0x06, -0x63,0x1b,0x2f,0x7d,0x49,0x29,0xbc,0x6c,0x5a,0xcc,0x06,0x1b, -0x4f,0xd3,0x28,0x2b,0x5d,0x3b,0x3b,0x43,0x8b,0x76,0xa4,0x41, -0xe4,0xb3,0xaa,0x12,0xf6,0x78,0x53,0xf5,0x28,0xf5,0x5c,0xa8, -0xca,0x02,0xe2,0x2f,0x1b,0x88,0x3b,0xff,0xf8,0xb8,0x66,0xd7, -0x46,0xf6,0x82,0xb4,0xe1,0x0b,0x49,0xe6,0x24,0xc1,0xd5,0x1b, -0x9c,0xab,0x30,0x62,0xa5,0xd5,0xd3,0x82,0x82,0x2b,0xb1,0x1b, -0x84,0x57,0xb6,0x8a,0x2e,0xda,0xeb,0x19,0x4a,0xaf,0x04,0x01, -0xcc,0xe0,0xc7,0x22,0x1d,0xbf,0xaf,0x3f,0xbe,0x1d,0xa8,0x93, -0xfe,0xe2,0xd1,0x3e,0x86,0x8c,0x55,0xaa,0xc9,0x72,0x86,0x2b, -0x78,0x04,0xc4,0x82,0x5f,0x46,0x42,0x3d,0x65,0x7e,0xaf,0xa4, -0x17,0x15,0x53,0x7c,0xc3,0x43,0x73,0x2a,0xe9,0xb9,0xf5,0x04, -0x1a,0xb8,0xac,0x5f,0x35,0xdb,0x1c,0x44,0xfa,0xac,0xae,0x1a, -0x13,0x82,0xef,0x91,0xe1,0xc2,0xea,0x29,0x76,0xbf,0x79,0xde, -0xbc,0x1f,0x06,0xfb,0x45,0x50,0xda,0x5a,0xd1,0xca,0x05,0x64, -0x50,0xf3,0x90,0x34,0x43,0x3a,0x31,0xc6,0x83,0xe5,0x2e,0x5c, -0xc1,0x28,0x7d,0xe6,0xea,0x2a,0x28,0xab,0xe9,0xb6,0x82,0x32, -0x55,0xe2,0x7b,0xaa,0x76,0x95,0x96,0x67,0xb6,0x8d,0xc7,0x4c, -0xd9,0x0a,0x7a,0x30,0x0c,0xd4,0x06,0x8e,0x85,0x4e,0xc6,0x5a, -0x01,0x25,0x9d,0xbf,0x99,0xfd,0xa2,0xc1,0xd0,0x34,0x44,0x25, -0xd6,0xb3,0x0f,0xe7,0xfc,0x98,0x8c,0x2c,0x7b,0x8d,0x8e,0x6e, -0xed,0xf2,0x64,0xac,0x79,0x56,0xb4,0x53,0x6c,0x7e,0x99,0xec, -0x71,0x71,0x93,0xc3,0x21,0x9f,0x95,0x80,0xdb,0x07,0x74,0xac, -0xbb,0xb3,0x13,0x8c,0x90,0x99,0xe3,0xdb,0xfb,0x77,0xc3,0x7f, -0x48,0xc5,0x65,0x06,0x98,0xea,0xdd,0xa6,0xbb,0xe5,0x5e,0x80, -0xb6,0x49,0x71,0x6e,0xe3,0xbf,0xce,0xee,0x90,0x16,0x11,0xc9, -0xa4,0x5e,0x50,0x1a,0x52,0xd2,0x9e,0xa1,0x5e,0x96,0xdf,0xbe, -0xfa,0x81,0x9a,0xac,0x00,0xf7,0xaf,0xc9,0xaa,0x04,0xb6,0x63, -0xd6,0xac,0xb4,0x21,0x9d,0xed,0x4b,0x87,0x52,0x7c,0x95,0x48, -0xaa,0xc6,0x0d,0x94,0x1d,0x34,0x82,0xe2,0xc3,0x88,0xf2,0xbe, -0x26,0xc2,0x30,0xc6,0x82,0xd6,0x7e,0x92,0x74,0x53,0x2b,0xb9, -0x4a,0xc0,0x14,0x9a,0xc4,0x7f,0xfa,0xb5,0x4d,0xbf,0x3d,0x9f, -0xb6,0x3f,0xbe,0x5c,0xbe,0xcd,0xcc,0x36,0x73,0xcf,0xde,0x32, -0x5a,0x5d,0x27,0x33,0xcf,0x3e,0x68,0x70,0x36,0x44,0x4a,0x32, -0x6e,0x44,0x5c,0xe1,0xc5,0x88,0x85,0x58,0x1e,0xcf,0xc2,0x95, -0x43,0x66,0x0f,0x13,0x24,0x8e,0x26,0x24,0xab,0xc3,0xc3,0x40, -0xd1,0x47,0xfb,0x37,0xd3,0x5b,0x72,0xbd,0xf7,0xb7,0x45,0x01, -0x51,0x1d,0xad,0xa1,0x15,0x43,0x90,0xe0,0xfb,0xf6,0x84,0x5f, -0xe2,0x1a,0xf4,0xaf,0x67,0xac,0xc9,0x0a,0x96,0xed,0xe3,0x4f, -0x55,0x2b,0x06,0x88,0x71,0xf8,0x9f,0xd7,0x16,0xa9,0xba,0x88, -0x3e,0x50,0x67,0xe5,0x81,0x73,0x0a,0x5f,0xb1,0x59,0xd9,0x2f, -0xb5,0x42,0x7b,0x54,0x69,0x1c,0x1f,0xed,0x3b,0x0a,0x5d,0xaf, -0x54,0x82,0xea,0x08,0xf8,0x59,0x2a,0x5b,0xe1,0xae,0x96,0xe9, -0xc2,0xb2,0xaa,0xbc,0xc2,0xb6,0x3f,0x90,0xcc,0x0b,0x87,0xc0, -0x32,0x06,0x28,0xb1,0xd6,0x18,0x40,0x07,0x40,0xdf,0x7e,0x12, -0x0d,0xec,0xc0,0x67,0xeb,0x6c,0x35,0xbf,0xf9,0x49,0xed,0x8d, -0x49,0x2d,0x64,0x21,0x8d,0xdd,0x2c,0x6d,0xed,0x51,0x72,0xc1, -0x5c,0x94,0xfd,0xee,0x01,0x7b,0xe7,0x89,0x63,0x9e,0x5c,0xb9, -0x10,0xaf,0xd4,0x61,0xe0,0x5c,0xaa,0xf0,0x3f,0xea,0x32,0x93, -0x39,0x3e,0xe4,0x4a,0x68,0x6d,0x28,0x98,0xff,0x47,0x99,0xf2, -0xf1,0x98,0x4e,0x75,0x8d,0xf8,0x61,0xa8,0x52,0xe8,0xe3,0x38, -0x46,0xb4,0x7d,0xe5,0x06,0xd8,0xc5,0x2f,0xf2,0xc2,0x8c,0x09, -0x51,0x13,0xbf,0xdb,0xab,0xf9,0x9f,0x17,0x0b,0xe5,0x1a,0xda, -0xb3,0xd1,0xc6,0x44,0xfa,0x51,0x1c,0x3b,0xe8,0x8f,0x8f,0xb9, -0x2d,0xc5,0xf4,0xbe,0xcb,0x85,0x6e,0xf9,0xbe,0x08,0xeb,0x63, -0x3a,0xa0,0x77,0xf7,0x3a,0xac,0x3e,0xac,0xe2,0x27,0xbd,0xaf, -0xe2,0x15,0x8e,0xf8,0x5d,0x0c,0xca,0xad,0x87,0xac,0x13,0x6c, -0xa1,0xb1,0xc4,0x6d,0x3e,0x11,0x01,0x0e,0xd5,0xb8,0xbb,0x22, -0xe6,0x2f,0xd7,0xfa,0xaf,0xdf,0xbc,0xfa,0xfe,0xf9,0xcb,0x57, -0xaa,0x38,0x2d,0x36,0x2a,0x16,0x02,0xed,0xc9,0x5c,0x41,0x96, -0x0b,0x1b,0x71,0xaa,0x5f,0x07,0x78,0x57,0x62,0xe4,0xea,0x4d, -0x18,0x80,0x4d,0x75,0xb6,0xeb,0x1d,0x67,0xc3,0xf6,0x2a,0x8d, -0x96,0x6f,0xd8,0xfb,0xc9,0xc2,0x70,0xb0,0x91,0x16,0x38,0xc5, -0x82,0x03,0x91,0xf0,0xaa,0xae,0x76,0x52,0x2d,0x5a,0x01,0x66, -0xaa,0x1f,0x5b,0x93,0xab,0x9e,0xf1,0xf7,0x1f,0xfb,0x6b,0x5f, -0x98,0xa3,0x48,0x2c,0x8a,0x60,0x8a,0x77,0x74,0xe9,0x84,0xe8, -0xa0,0x95,0x6a,0xd1,0x91,0xd8,0xdd,0x13,0x81,0xf9,0x53,0xe3, -0x3b,0xd0,0x5d,0xaa,0x7a,0x6d,0xa3,0xb1,0xd0,0x72,0xc2,0x9a, -0x9e,0x0f,0x40,0xef,0x38,0xbd,0x0e,0x2b,0x6e,0x6a,0x1a,0x84, -0x5b,0xab,0xfd,0xf7,0xb6,0x22,0x80,0x87,0x87,0x51,0xa3,0x87, -0x11,0x45,0xe2,0x92,0x51,0x60,0x5a,0x78,0x20,0x16,0x6e,0x4d, -0xe5,0x88,0xba,0x1f,0x2e,0xd0,0x47,0x1a,0x0f,0xfc,0xfe,0x54, -0x2c,0x73,0x8b,0x1c,0xfc,0xf8,0x44,0x3c,0xac,0x56,0x35,0x26, -0xd6,0xba,0x9f,0x88,0x0b,0xf5,0x98,0xc6,0x84,0x1f,0x9f,0x88, -0x87,0x36,0xca,0x35,0x22,0xfa,0xfa,0x44,0x4c,0xb2,0x3f,0xae, -0x71,0xc9,0xf7,0xa7,0xf2,0x1c,0xb7,0xc5,0x0d,0xd3,0x47,0x35, -0x32,0x50,0x31,0xfd,0x5a,0xb9,0x55,0xe7,0xdb,0x26,0x5f,0x6b, -0x86,0x20,0xe3,0x1d,0xae,0x5a,0xe5,0x54,0xbd,0x52,0x28,0x2c, -0xe2,0xee,0xad,0x97,0xc2,0xdf,0xa9,0x97,0x7e,0xc7,0x70,0x0e, -0xff,0x20,0xb1,0x97,0xc7,0xed,0xfe,0xaf,0x94,0x57,0x73,0xef, -0xc0,0x56,0x11,0x92,0xf4,0x7f,0xd3,0x18,0xb8,0xbf,0x66,0xfe, -0x2a,0x7f,0x3e,0xf9,0x8f,0xd1,0xca,0x22,0x10,0xbf,0x6f,0x58, -0x86,0xb5,0xc3,0x52,0x68,0xf7,0x52,0x5c,0x60,0xfc,0xfa,0xa2, -0xda,0x28,0xe4,0x0d,0x7e,0x65,0x5f,0x01,0xac,0x0e,0xd7,0x46, -0x4f,0x96,0x16,0x8b,0x59,0xeb,0xb2,0x52,0xf7,0x43,0xe1,0x4d, -0xc9,0xf6,0x2c,0x1c,0x80,0xd1,0xd0,0x7c,0x35,0x7f,0xb1,0xe6, -0xe7,0x58,0x07,0x50,0x64,0x18,0xec,0x58,0xcd,0x69,0x70,0x94, -0x40,0xaf,0x80,0xbc,0xb0,0xcc,0x43,0x0e,0x24,0x4b,0xe2,0x1a, -0x55,0x3b,0x4b,0x3e,0xb0,0xa7,0xc4,0x0e,0x3b,0x91,0x49,0xf3, -0x7b,0x0a,0x96,0x08,0x8a,0x0d,0xa6,0xf2,0xe2,0x57,0x41,0xda, -0x01,0xa9,0xd4,0xda,0xcb,0x59,0x65,0xa2,0x5b,0x8a,0xc9,0x39, -0x7c,0x8d,0xe9,0x6c,0xa6,0x50,0xf4,0x3e,0xb5,0xfa,0x43,0x6c, -0x5f,0xc3,0xea,0xcd,0x72,0xc8,0x15,0x87,0x26,0xb5,0x69,0x6f, -0x16,0x92,0x82,0x55,0x07,0xfa,0x1b,0xd5,0x3a,0x81,0x7f,0x0a, -0x56,0x72,0xc7,0x88,0x55,0x7a,0x5b,0x3a,0xc4,0x6b,0x7c,0xf3, -0x6a,0xf4,0xb6,0x61,0x3d,0x03,0xb0,0x1a,0x1d,0x1f,0x8b,0x73, -0x04,0x7e,0xd4,0x8f,0x18,0x04,0x12,0xaf,0x07,0x5a,0x04,0x1b, -0x0e,0x38,0x63,0x87,0xb6,0x00,0xaa,0xa4,0xb9,0x28,0x66,0x55, -0x7b,0x2a,0x73,0xb3,0x86,0xd3,0x37,0xbc,0xd0,0xbb,0xa2,0xe2, -0xf4,0x83,0x8f,0x86,0xf8,0xf4,0x09,0xff,0xfd,0x16,0xd3,0x01, -0x1b,0xfe,0xbe,0xbb,0xe3,0x8a,0xf6,0x59,0xef,0x81,0xdc,0x64, -0xdd,0xed,0x8f,0xee,0xf1,0x43,0xf7,0x9a,0xf5,0x56,0x6d,0xb6, -0x27,0x37,0xb9,0xa0,0xe8,0x76,0xe3,0x97,0xed,0x02,0xa2,0xc8, -0x5f,0xda,0x0e,0x1d,0xe2,0xc9,0xa1,0x02,0x63,0xa8,0xb5,0x9c, -0xfa,0x5e,0x58,0xce,0x35,0xbb,0x9b,0x43,0x7d,0x63,0xf9,0xde, -0x28,0xf5,0xe6,0x15,0x07,0xa0,0x19,0x06,0x12,0x9d,0x02,0xc5, -0xca,0x48,0x09,0xf6,0x2b,0x75,0x6e,0xbc,0x4f,0xaa,0x8b,0x4c, -0xa8,0x10,0x40,0x18,0x2d,0xe3,0x04,0x1f,0x3b,0xa6,0x3a,0x2e, -0x38,0x20,0x11,0x62,0xa9,0x76,0x3d,0xde,0xdb,0x65,0x95,0x73, -0x7f,0xbf,0x62,0xd9,0xb5,0x0f,0x53,0x71,0xfd,0xc7,0x5b,0x26, -0x87,0x49,0x0c,0x6d,0xcd,0x74,0x6b,0x2b,0x65,0x5c,0xd4,0x63, -0x04,0x09,0x32,0x50,0x64,0x83,0x6c,0x06,0x11,0x27,0x9c,0x3c, -0x7f,0x3f,0x0d,0xb5,0x2c,0xd8,0xbd,0x67,0x55,0x6b,0xfd,0x91, -0x64,0x3b,0x44,0x1c,0xc8,0x07,0x9f,0x78,0xf0,0x89,0xb5,0x0a, -0xab,0xdc,0x7a,0x69,0x24,0x57,0x6b,0xec,0x71,0xb5,0x73,0xbf, -0x81,0x38,0x8a,0x61,0x45,0x87,0xf1,0x49,0xfe,0x58,0xcd,0x8e, -0x7d,0xf9,0x6d,0x79,0x17,0x87,0x7a,0x57,0xc6,0x26,0x46,0x4c, -0xb8,0x97,0x86,0x03,0x3a,0x5e,0x58,0x81,0x01,0x44,0x6b,0x20, -0x4f,0x45,0x73,0x1d,0x24,0x85,0xfa,0x38,0x7f,0x8f,0x62,0xa4, -0xad,0x9e,0xcb,0x77,0x78,0x53,0x87,0x31,0xf3,0xac,0x20,0xc9, -0x7b,0xf4,0x49,0x0f,0x3d,0x3e,0x7f,0x22,0xdf,0x4d,0x4f,0x4a, -0x90,0xcf,0xba,0xe0,0x2c,0xed,0x46,0x2a,0x18,0x76,0x8a,0x3b, -0x4c,0xb5,0x4a,0x99,0xfa,0x89,0xcd,0x55,0xae,0x99,0xa5,0x5c, -0x83,0xf9,0xfb,0x67,0x12,0x87,0x28,0xbe,0x97,0x7e,0xa3,0x85, -0xcd,0xef,0x9e,0xb5,0xc9,0xd1,0xa4,0xaf,0xb7,0x02,0x75,0xc9, -0x03,0xe6,0x9e,0x3f,0x64,0x14,0xff,0x47,0x6b,0xba,0xfb,0x8f, -0xf0,0x4a,0x35,0x78,0x08,0x1a,0xb5,0xd1,0x1d,0xd0,0x08,0xdf, -0xd5,0xa9,0xbf,0x5f,0x13,0x14,0xb6,0x8a,0x4d,0x87,0x5b,0xce, -0xa4,0x4e,0x4c,0xa5,0xc2,0xe4,0xec,0x07,0x26,0xe4,0x81,0x41, -0xa3,0x4f,0xa8,0x8b,0xf1,0x17,0xcc,0x94,0x6c,0x82,0x36,0x97, -0x8a,0xab,0x41,0x65,0x8c,0x1f,0xde,0x2c,0xaf,0x24,0xc9,0xf6, -0xc0,0xfd,0xd8,0x78,0xdd,0xfa,0xee,0xd5,0xab,0x17,0x2d,0xae, -0xad,0xa5,0x30,0x37,0xf4,0xcd,0xf7,0xd8,0x92,0xcc,0xbe,0x72, -0x5f,0xec,0xea,0xa4,0xad,0x6f,0x9d,0xae,0x17,0x5d,0x62,0x2b, -0x28,0xed,0xef,0xb1,0x64,0x6c,0xe2,0xaa,0xac,0x18,0x72,0x11, -0x76,0xf7,0x9f,0x07,0xce,0xe7,0x10,0x58,0x2e,0x36,0x8b,0x19, -0xde,0xae,0x5c,0xf8,0xdd,0xff,0x78,0x22,0xfe,0x0b,0x90,0xa0, -0x8a,0xc9,0xf6,0xca,0x3e,0x8a,0x0e,0x9c,0xe0,0xdc,0xb5,0xe7, -0xb7,0x4e,0x24,0x9b,0xda,0xe3,0x08,0x7b,0x1e,0x94,0xc3,0x08, -0x3e,0xc7,0xe1,0x81,0xab,0xe6,0x42,0xf9,0x2a,0xaf,0x43,0xdd, -0x8d,0xc1,0x4f,0xa8,0x4c,0x85,0x71,0x38,0xb0,0x3e,0x6b,0x67, -0xf4,0x13,0x2a,0xa3,0xe3,0xaa,0x43,0x6b,0xb2,0x76,0x91,0x78, -0xe1,0x23,0x47,0x3c,0xf7,0xae,0x94,0x06,0xbc,0x6d,0x59,0x10, -0xa2,0x72,0x8d,0xf6,0x56,0xd3,0x27,0x34,0x8e,0x43,0x60,0x1c, -0xd8,0x3a,0x67,0x37,0xea,0x13,0x2a,0x93,0x10,0x1b,0x07,0xd6, -0x56,0xd8,0x5b,0xfa,0x84,0xfa,0x74,0x08,0x8f,0x43,0x7b,0xcf, -0xde,0x81,0xf2,0x3e,0xb5,0xe7,0x38,0x4e,0x88,0x5b,0x65,0x4d, -0xe7,0xb9,0xe7,0x7a,0x87,0x9c,0xe8,0xed,0x27,0xa0,0x1a,0x49, -0x0d,0x01,0xbb,0xbc,0xfc,0x28,0x8e,0x4d,0xf1,0x22,0xb1,0x5c, -0xa3,0x90,0xb5,0x7c,0x21,0xfe,0x91,0x7c,0x9a,0xed,0x1c,0xa4, -0x31,0xd1,0x08,0x7b,0xee,0xa7,0x59,0xd2,0x3b,0x93,0x52,0xe5, -0xfe,0x1b,0xaf,0x0f,0x80,0x00,0x0c,0x7e,0x55,0x77,0x24,0x5a, -0xa0,0x95,0xcd,0x4e,0x76,0xf1,0x34,0x95,0x5e,0x55,0x8b,0x93, -0xe5,0xac,0x6d,0x2e,0xa4,0xd8,0x77,0x4b,0x3e,0xee,0x74,0x4a, -0xa7,0xe2,0x7c,0x57,0x2e,0x50,0x90,0x2e,0xab,0x54,0x81,0x02, -0xcb,0xa8,0x9c,0xc5,0x37,0xb3,0x47,0xe0,0x32,0xcb,0xba,0x26, -0xa3,0xe3,0x39,0x9a,0x94,0xeb,0xd2,0x0d,0x9c,0x36,0x9f,0xff, -0xf6,0xc8,0xb2,0xe3,0xf3,0xdf,0xda,0x79,0xbd,0xc0,0xa2,0xee, -0xd5,0x21,0x23,0xa0,0x50,0xa8,0x92,0x9d,0x5b,0x23,0x68,0xe8, -0xf3,0xef,0x78,0xd7,0x7e,0xeb,0x5e,0xaf,0x32,0x11,0x32,0x67, -0xc9,0x07,0xb9,0x81,0xe5,0x9c,0xf8,0x56,0xf9,0xfc,0x2a,0x56, -0x15,0x9d,0x84,0x35,0x8a,0x40,0xff,0xf2,0xad,0x48,0x9c,0x9e, -0x9d,0xba,0xd3,0x03,0xd8,0x00,0x72,0xb0,0x48,0x06,0xc3,0x00, -0x32,0x66,0xbe,0xa7,0x5b,0x06,0x75,0x5e,0xce,0x83,0x52,0xc6, -0xb0,0x62,0x0f,0x48,0x5f,0x3a,0x53,0x2e,0xc7,0x41,0xb1,0x45, -0x52,0xb3,0x0a,0x42,0xaa,0x76,0x8e,0x89,0x35,0x07,0xb8,0xd8, -0xaa,0x8d,0x66,0x5d,0xce,0x74,0x87,0x2a,0x5d,0xdb,0x41,0xe5, -0xc2,0x05,0x07,0x5d,0x45,0x1b,0xef,0x8c,0x2b,0xa0,0x82,0x9b, -0x6e,0x35,0x50,0x8d,0xb3,0x6e,0x35,0x70,0xa5,0xcb,0x6e,0x6d, -0xe5,0x65,0xc7,0xdd,0x6a,0xd0,0xf2,0x92,0x85,0x9d,0x68,0xa9, -0x64,0x39,0xb3,0xcc,0x8d,0xba,0x5d,0x8d,0x6a,0x1c,0x96,0xb3, -0x5a,0x11,0xd3,0xd6,0x19,0x59,0x75,0x22,0x59,0x35,0x0e,0xb6, -0xc3,0xad,0x5f,0x11,0x34,0x77,0x39,0x9f,0xf3,0x96,0x7d,0xb0, -0xc0,0xd7,0xe7,0xe6,0x6b,0xba,0xae,0xb9,0xcc,0xc6,0xf3,0x65, -0x12,0x63,0x66,0xfb,0x22,0xb8,0x84,0xca,0xde,0x53,0x00,0xdc, -0x80,0xb6,0xc8,0x5f,0xa0,0x07,0x5a,0xd1,0x1b,0x2d,0x80,0x15, -0x39,0x67,0xf0,0xea,0xdf,0xca,0x41,0x29,0xe1,0x2c,0x25,0x2f, -0x56,0x26,0xf9,0xdd,0x19,0xb4,0xe4,0x87,0x17,0x90,0x67,0x9b, -0x9d,0x4a,0x09,0x28,0xd9,0xe3,0xd1,0xca,0x4e,0xe7,0x94,0x40, -0xbf,0x52,0xa3,0x88,0x30,0x8f,0xd3,0x5c,0x8f,0xb2,0x9f,0x3e, -0xcc,0x5e,0x70,0x54,0xa4,0xdc,0x00,0xb8,0xe9,0xc1,0x8c,0x9e, -0xbb,0xf9,0x96,0x1f,0x50,0xb5,0x6b,0x90,0x37,0x55,0x05,0xe0, -0x87,0xd1,0xc2,0xce,0xbc,0x19,0x2d,0x24,0xe3,0x67,0xba,0xf7, -0x63,0xe7,0xf1,0x4d,0x20,0x27,0xfb,0xe7,0xf4,0xed,0xf5,0xaa, -0x0c,0x43,0xc9,0x8a,0x82,0x74,0x0a,0x2b,0x54,0x87,0x00,0x4a, -0x91,0xec,0x6f,0x40,0xf8,0x73,0x3b,0x17,0x47,0x83,0xa2,0xfe, -0xe5,0xfc,0xc6,0x65,0x1a,0x7c,0x4b,0xd6,0x73,0xe8,0xd4,0xcd, -0x4f,0x57,0x76,0x6e,0xca,0x49,0x02,0xf0,0xfd,0x08,0xf7,0x58, -0x4a,0x40,0x53,0x93,0xac,0x30,0xc9,0x35,0x55,0xee,0x7e,0x09, -0x7d,0x20,0x79,0xff,0x3d,0xc9,0x33,0xd5,0xfd,0x18,0xa8,0x58, -0x92,0xbf,0x82,0xaa,0x62,0x2d,0x04,0x24,0xf1,0xc1,0x45,0x75, -0x5c,0x51,0xf8,0xfd,0x20,0x9d,0xe1,0x60,0x18,0x27,0xf3,0xab, -0x07,0x17,0x7d,0x48,0xa0,0x7b,0x61,0x6e,0xaa,0xbc,0x07,0xfe, -0x61,0x39,0x5a,0x2c,0x92,0x09,0xee,0x54,0xa9,0xfd,0x00,0x8c, -0xf7,0xfa,0xe0,0x82,0x70,0x6e,0x7b,0x8d,0xb5,0xba,0x7a,0xdc, -0x38,0x8a,0x51,0xda,0xa0,0x70,0xb2,0x59,0xcc,0x97,0xab,0xac, -0xef,0x55,0x65,0x72,0x68,0xe8,0xe3,0x63,0x09,0x11,0x2d,0xb0, -0x78,0x25,0x87,0x7f,0xc5,0x6e,0x46,0x7c,0xe1,0x07,0xf2,0xb3, -0x7d,0x01,0x1f,0x5d,0x1e,0x30,0xf1,0x45,0x70,0xd1,0xfe,0xe5, -0x9b,0x9f,0x5f,0x3e,0xff,0xe9,0xc7,0xb8,0x11,0xb5,0x1f,0xb5, -0x3b,0x0d,0xb2,0x33,0xd0,0x4f,0x2d,0x66,0x77,0x35,0xf8,0x23, -0x92,0xe5,0x30,0x22,0xa0,0x47,0xb2,0x24,0x16,0x0e,0x06,0xd1, -0xa6,0xab,0x22,0xa8,0x8d,0x8e,0xb0,0x95,0x3e,0x7e,0xd8,0x92, -0x7a,0x7c,0x0c,0xa9,0x1a,0x53,0x1c,0x3b,0x99,0xbe,0x95,0xe7, -0x95,0x10,0xf7,0xd4,0x8d,0x2e,0x84,0xe2,0x58,0x07,0x80,0xa0, -0x69,0xbe,0xcc,0xc5,0xe3,0x34,0x0e,0x03,0x81,0x30,0xd9,0x3d, -0xfe,0x73,0x9e,0xf6,0xd2,0x66,0x13,0x09,0x53,0x35,0x88,0x47, -0x2a,0x57,0x83,0x9b,0x9d,0x83,0x74,0x18,0xa4,0xf8,0x03,0x63, -0x2b,0x8a,0x32,0x91,0x8b,0x67,0xec,0x1e,0xaf,0xea,0x41,0x99, -0x89,0x25,0xc6,0x35,0x82,0x07,0x56,0xc5,0x98,0x78,0xff,0x9a, -0xb1,0x14,0x54,0x3f,0x0c,0xe4,0x47,0x0d,0x15,0x3d,0x3a,0x68, -0x88,0x31,0x12,0x07,0xa9,0xde,0x7d,0x7d,0x22,0xfb,0x84,0x14, -0x1f,0xd7,0x38,0xab,0x63,0x37,0xc5,0xc8,0xa1,0xbe,0xe4,0x75, -0xb5,0xd2,0xe0,0x7e,0xc2,0x3a,0x54,0x1f,0x41,0x62,0x5f,0xd2, -0xca,0x7d,0xd3,0xf5,0xc8,0xa5,0x11,0x2b,0xb7,0xbd,0x5d,0xd7, -0x49,0x40,0xa3,0x36,0x98,0xa6,0x1c,0x42,0x85,0x6a,0xe1,0xbb, -0xdf,0xd5,0x3c,0x28,0x15,0x92,0x6d,0x6d,0x2c,0x27,0x37,0x24, -0x59,0x07,0x51,0xe4,0x9c,0xb8,0xc1,0x7a,0xea,0x01,0x8e,0x13, -0x0c,0xe6,0xfd,0x60,0x84,0xa3,0x9b,0x0c,0xb3,0x07,0xb3,0xf9, -0x03,0x89,0xa2,0xc7,0xc1,0xbc,0x1b,0x3d,0x75,0xdb,0x91,0x64, -0x79,0x3a,0xa1,0xd8,0xd0,0xb3,0x77,0xf5,0xec,0xbb,0x81,0x15, -0xb9,0xcb,0x43,0x41,0x58,0x0a,0xba,0x71,0xde,0xe9,0x89,0xd8, -0x13,0x3f,0x61,0x00,0xc2,0xbf,0x14,0xd8,0xc2,0x56,0xa7,0xcc, -0x54,0xa1,0x41,0xf1,0x95,0xb3,0x7c,0xe3,0xb6,0x86,0xb5,0x61, -0xcc,0x3b,0xa1,0x42,0x4d,0xb9,0x25,0x9e,0x83,0xb4,0x31,0x35, -0x7d,0x83,0xd5,0x73,0x68,0xf7,0xbb,0x15,0x39,0x7c,0x01,0xf3, -0x80,0xde,0x52,0xd8,0x11,0x53,0x5c,0xdd,0x5b,0xc4,0xa0,0x52, -0xc9,0xae,0x47,0x45,0x54,0x3a,0x73,0x0c,0xcf,0x1c,0xfc,0xe0, -0x48,0x3e,0xad,0xf8,0xa0,0x18,0x1a,0x87,0x63,0x84,0x5a,0x1d, -0xab,0xad,0x60,0xc4,0xb4,0x0d,0x2e,0xec,0x89,0x47,0x7a,0x6f, -0xf9,0x9f,0xd4,0x69,0x54,0xa7,0xdd,0x73,0x4c,0x84,0xdb,0x7d, -0x94,0xf6,0x07,0xf5,0x21,0xe1,0xaa,0xef,0x48,0x37,0x9b,0xd7, -0x83,0x65,0x5d,0x07,0x0d,0xe3,0x5f,0x47,0xa0,0x22,0x95,0x7a, -0xac,0x52,0x59,0xbd,0xb2,0xba,0xda,0x1e,0x2c,0x20,0xf0,0x9b, -0x0a,0xf6,0x49,0x5b,0xb5,0x5a,0x5c,0x7e,0xd8,0x55,0xbf,0x82, -0x43,0x65,0x88,0x94,0x2e,0xa2,0x1b,0x56,0x08,0x92,0x93,0xf9, -0x87,0x48,0x13,0x06,0xd9,0xa5,0x75,0xf0,0xea,0x5e,0xba,0x53, -0x47,0xba,0x9b,0xe5,0xfb,0x55,0x1d,0xef,0x83,0x1f,0xa6,0xe4, -0xfa,0x1e,0x57,0x20,0x63,0x06,0x5a,0xd7,0x65,0xf3,0x40,0x2b, -0x3f,0xa6,0x9a,0x4c,0xad,0x8b,0x36,0x87,0xd0,0xfd,0x63,0x75, -0x3e,0xdb,0x71,0x32,0x3d,0x73,0x45,0x7a,0x76,0xa6,0xcf,0xbe, -0xc9,0xf9,0x54,0xfd,0x7f,0x20,0x37,0x8e,0x8f,0x9d,0x89,0x82, -0x43,0x29,0xda,0xd3,0x00,0xe9,0x83,0x1d,0x7a,0x5b,0xb3,0x40, -0xef,0x2b,0x09,0xdd,0x07,0x75,0xda,0xd1,0x81,0x64,0x6e,0x75, -0x3d,0x48,0x10,0xd9,0xb5,0xd0,0x37,0xb8,0x44,0xdc,0x67,0x20, -0x49,0xca,0xdd,0x9d,0xad,0x20,0xf0,0x3c,0x65,0x95,0xae,0x72, -0x75,0xb0,0x48,0xf2,0x70,0x14,0xd6,0xf5,0x5b,0xd7,0xb2,0xaf, -0xb9,0xd7,0x98,0x02,0xd5,0x69,0x94,0xd1,0xd7,0xe9,0x9f,0x3e, -0x65,0x63,0xed,0x4a,0x3c,0xf9,0xcf,0xf1,0xf1,0xa1,0x73,0x77, -0x9f,0xa5,0xb8,0x2b,0xd6,0x0b,0x0e,0xd9,0x23,0xc6,0xa1,0x9e, -0x87,0x98,0x21,0xcf,0xd0,0xee,0x47,0xd6,0xcd,0xf2,0x3f,0x92, -0x75,0xd1,0x1e,0xd6,0xe1,0xea,0x83,0x39,0x47,0xf5,0x2b,0xc6, -0x61,0x72,0x5f,0xa5,0xfe,0x3e,0xb6,0x09,0x29,0x40,0xac,0xd0, -0x74,0x30,0xdf,0x84,0x61,0x46,0x0b,0x58,0x7c,0x33,0xe1,0xd7, -0xc8,0x7e,0x19,0x4f,0xd7,0x93,0xc4,0xe5,0xdb,0x0a,0x66,0xba, -0xc4,0x50,0x61,0xda,0x7f,0x14,0x75,0x9d,0xe5,0x15,0x37,0x5f, -0x16,0x56,0x9a,0x03,0x92,0xd9,0x6f,0x45,0x64,0xc3,0xab,0x7c, -0x4f,0xd0,0x76,0xab,0xd5,0x9f,0xae,0x8e,0x9f,0x73,0x89,0x63, -0x06,0xdf,0xd2,0xe8,0x48,0x67,0xb7,0xf3,0xf7,0x05,0x2a,0xed, -0xab,0x67,0x23,0xbc,0x62,0x96,0xd5,0xdc,0x80,0x4c,0x69,0xbb, -0x9c,0x42,0x33,0xe9,0x0d,0x36,0x29,0x6c,0xe2,0x58,0xa0,0x59, -0x5a,0x4f,0x94,0xc7,0x45,0xe5,0x49,0x81,0x2e,0xe5,0xa9,0xeb, -0x66,0xbe,0xb9,0x6a,0x06,0xdd,0x80,0x94,0xf8,0x4c,0xf4,0x62, -0xba,0xb6,0x0f,0x68,0x11,0x3b,0xbd,0x59,0x73,0x60,0x9d,0xcc, -0x08,0x7a,0x32,0x86,0xf1,0x7d,0xb8,0x4e,0x96,0x05,0x1e,0x8c, -0x56,0x2b,0x0c,0x8e,0x95,0x2e,0xb3,0x95,0x13,0xd9,0x8c,0x5f, -0xa6,0xa1,0x5c,0xbf,0x4f,0xd9,0x6a,0x28,0x0d,0x86,0xdd,0x8b, -0x01,0xa7,0x34,0x70,0x1a,0x6b,0x74,0x1b,0xac,0xdc,0x1a,0xc3, -0x4a,0x5a,0xac,0xa5,0x0a,0x18,0x42,0x0f,0x18,0x25,0x4c,0xf0, -0xf4,0x83,0x88,0x03,0x0b,0xc1,0x50,0x2a,0x36,0x8c,0x1e,0x3c, -0x47,0x21,0xd3,0x8e,0x55,0xfd,0xa5,0x86,0x7e,0x8b,0x72,0x6a, -0xa2,0xd5,0x32,0x9c,0xa7,0x03,0x64,0xd4,0xe6,0x80,0x55,0xa3, -0x56,0xbc,0x18,0x46,0x4c,0xc5,0x28,0x9c,0x93,0x3f,0x06,0x9a, -0x00,0xe1,0x50,0x96,0x7b,0xf0,0x8b,0x05,0x97,0x8d,0x8c,0x2f, -0x1e,0x3f,0x7a,0x74,0xf6,0x48,0x19,0x5f,0xf4,0x84,0x29,0xd4, -0x27,0x9d,0x8a,0x9f,0xb4,0x86,0xea,0x95,0x2b,0x60,0x26,0x63, -0xa6,0x14,0x6e,0x45,0xa7,0xa1,0xad,0x42,0x3e,0x8e,0xe7,0x37, -0x8b,0xf5,0x2a,0x99,0x74,0x31,0x87,0xc7,0x29,0xfd,0xd4,0x37, -0x57,0x0f,0xd1,0x05,0x7c,0xeb,0x87,0x11,0x69,0x0d,0xd0,0x3f, -0x50,0x15,0xb0,0xa8,0xf6,0x54,0xf9,0x73,0xa6,0xac,0xad,0xbe, -0xc1,0xc6,0x54,0xb4,0x32,0x75,0x8c,0x40,0xd3,0xad,0x7e,0x6c, -0xcd,0xfc,0xc9,0x97,0xe6,0xa8,0x57,0xd2,0xd9,0x7f,0x6e,0xaf, -0xa4,0xb3,0x4f,0xe9,0x95,0x07,0xb5,0xbd,0x62,0x3a,0xe5,0xff, -0x64,0x9f,0x7c,0xf1,0x47,0xf6,0x49,0x76,0xbd,0xbe,0xba,0x9a, -0x26,0x85,0xcd,0x26,0x6a,0xfd,0x68,0x36,0x61,0x2a,0xe2,0x30, -0x10,0xb0,0x89,0x65,0xd8,0xd5,0x34,0xdb,0xc7,0x57,0x16,0xd1, -0xd8,0xc5,0x3f,0xf3,0x1b,0x8f,0x30,0x34,0x9b,0xbe,0x46,0xc1, -0xa6,0x75,0x2b,0x1a,0xc6,0x3a,0x05,0x41,0x87,0x81,0xfb,0xc9, -0x3a,0x62,0x6b,0xca,0x11,0xb5,0xa3,0x9b,0x45,0x81,0xd8,0x60, -0x16,0xbc,0x5d,0x8f,0x96,0x26,0xda,0x7a,0x71,0xd5,0xf5,0x45, -0xe7,0xee,0x8e,0x20,0x70,0x92,0x1d,0x68,0xba,0x8c,0xe4,0xb4, -0x22,0x1f,0x74,0x9c,0xe2,0x04,0x31,0x80,0xb7,0x53,0xbd,0x30, -0x50,0xa3,0x1b,0x1f,0x4f,0x52,0x4f,0x4c,0x4d,0xe9,0xc1,0xf5, -0xe7,0xca,0x1c,0xa8,0x51,0xc6,0xce,0xec,0xc1,0x59,0x7d,0xee, -0x9e,0xba,0x7d,0x3d,0x7e,0x3f,0x6c,0xb8,0xc5,0x19,0x37,0x9b, -0x2f,0x57,0x5f,0x15,0xac,0x13,0xd5,0xad,0xf6,0x4a,0x53,0x11, -0xe1,0xd2,0x24,0xf5,0x99,0xc9,0x8a,0x66,0x16,0xaf,0x6e,0x02, -0xa9,0xb0,0x24,0x1c,0x49,0xa2,0xec,0x2e,0x03,0x8d,0x97,0x58, -0x67,0x3a,0xea,0x1e,0x6a,0xb4,0x6e,0x7d,0x6a,0x8c,0x71,0x96, -0x9d,0x26,0x57,0xab,0x60,0x49,0xeb,0x57,0x9e,0x8a,0x63,0x4c, -0x69,0x2b,0xc4,0xc1,0x65,0x4c,0x99,0x3a,0x01,0x87,0xec,0x08, -0xe6,0x8c,0x4b,0xd2,0x0e,0xa3,0xf3,0xcb,0xbb,0x3b,0x9e,0x9a, -0x30,0x16,0xa9,0x1e,0xb2,0x08,0x75,0x79,0x3e,0xb2,0xf2,0x2e, -0xb5,0x92,0x55,0x67,0x02,0x54,0x0f,0x4b,0x1f,0x57,0x41,0xbf, -0x41,0xc4,0x1a,0xbc,0x5f,0x23,0xfd,0xfb,0x76,0x39,0x5f,0x2f, -0x0c,0xeb,0x2f,0x93,0xeb,0xd1,0x6d,0x6a,0x45,0x45,0xde,0xd7, -0x29,0x6a,0x30,0x6e,0xb5,0x7e,0x8b,0xd9,0x22,0x22,0xd8,0xbe, -0x31,0x1f,0xbb,0x3b,0x7b,0x6d,0xa7,0x5a,0xd1,0x6b,0xeb,0x03, -0x4c,0x3c,0xd2,0x7b,0xaa,0x15,0xa2,0x2a,0xd0,0xb0,0x08,0x0a, -0x0b,0x1d,0x12,0x3c,0x6a,0x3c,0x48,0x1e,0xfd,0x75,0x3c,0x9c, -0xdd,0x52,0x1f,0x3d,0x7e,0x96,0xc5,0x64,0xf8,0x62,0xed,0xd2, -0xb4,0xde,0xb5,0x7e,0xe3,0x0e,0x47,0x61,0x65,0x75,0xc1,0xac, -0x3f,0xa4,0x1e,0x1b,0x91,0x52,0x09,0x68,0x91,0xae,0x67,0xab, -0x5d,0xc5,0xf1,0xd6,0xf3,0x0e,0xfa,0x9a,0x4d,0x87,0xc2,0x88, -0x90,0xa2,0x9c,0x26,0x13,0xb2,0x46,0x4d,0xf7,0xd3,0x06,0x5f, -0xb0,0x7b,0x75,0x20,0xfd,0xab,0xf5,0x7a,0x7d,0x17,0x9b,0x6d, -0x14,0x65,0x27,0xb1,0xf5,0x5a,0xbb,0x41,0x8b,0xd7,0x6e,0x3e, -0x80,0x02,0xbe,0x06,0x81,0x8d,0x89,0x16,0xb5,0xe3,0x82,0x29, -0xe7,0x90,0xd9,0x93,0x37,0xb2,0xd2,0x09,0x68,0x81,0x0f,0x4d, -0x4a,0x3e,0x3f,0x87,0x01,0x51,0x89,0x92,0x50,0x0c,0x00,0x78, -0xe8,0x7f,0xc1,0xf2,0x88,0xf8,0xe1,0xbb,0x19,0x75,0xa9,0x0e, -0xf8,0xa9,0xc7,0xcb,0xfc,0x03,0x2a,0xdd,0xd5,0xbc,0x10,0x77, -0xb3,0xa0,0xb7,0xfa,0xee,0x34,0xdd,0xb7,0xcc,0x6a,0xfc,0xee, -0xd6,0x6d,0x9f,0xf7,0x8d,0x3e,0xb2,0x96,0x54,0x5d,0x7d,0xcb, -0x9e,0x0a,0x0f,0x86,0xa4,0xf7,0xd3,0xdf,0x8a,0x53,0x54,0x69, -0xad,0x11,0xd6,0x57,0x64,0x7e,0x76,0xad,0x9d,0x28,0xb5,0xfb, -0x44,0xd6,0x26,0xd8,0xb7,0x30,0x67,0xe1,0xc1,0x3b,0xfc,0x59, -0x8d,0xec,0xb5,0x03,0x4b,0x40,0x71,0x9a,0xe1,0x8a,0x29,0x4f, -0x99,0xca,0x9c,0x34,0x53,0xd3,0x0d,0xb3,0x3a,0x1c,0x76,0x9d, -0x75,0x06,0xe2,0xc2,0xc9,0x84,0x57,0x29,0xbc,0xbb,0xb4,0xa7, -0xaa,0x8a,0xf2,0xb6,0x24,0xb4,0xbc,0x42,0xc5,0xb0,0xe8,0xf2, -0x09,0x3f,0x1e,0x4e,0xfd,0x51,0xed,0x70,0x6a,0x8c,0x2a,0x1a, -0xa5,0xa7,0x4a,0x07,0x72,0x16,0x84,0xbe,0x6c,0xa0,0x10,0x87, -0x61,0x1d,0x89,0xdb,0xe0,0x93,0xe5,0x7c,0x71,0xef,0x66,0x57, -0x34,0x73,0x4b,0xba,0xe0,0x66,0x31,0x1a,0x17,0x1b,0x5a,0xde, -0x8c,0x61,0x24,0x96,0xa8,0xb1,0xba,0xbf,0x9a,0xe2,0xab,0x8e, -0x33,0x3b,0x12,0x3e,0x98,0x4b,0x60,0x76,0x40,0xc5,0xf3,0x0f, -0xc1,0x7c,0xbd,0x82,0x4f,0x43,0x14,0x27,0x53,0xe0,0x61,0xde, -0xe7,0x20,0x68,0x3d,0x04,0x28,0xb8,0xf9,0x78,0xb4,0x12,0xc3, -0x93,0x4b,0x07,0x04,0xa4,0x96,0xf2,0x5c,0xa2,0x68,0x36,0x14, -0xa2,0xa4,0xfb,0x77,0x77,0x9c,0x22,0xc6,0x8c,0x32,0x21,0xa4, -0xfe,0x3e,0xea,0x53,0xb7,0x0e,0x06,0xe8,0x4a,0x73,0x64,0xb2, -0x28,0xb4,0xa2,0xcb,0x7f,0x0b,0xda,0x58,0x32,0x51,0xfd,0x97, -0x98,0xc1,0x4c,0x13,0x34,0x66,0x0a,0x94,0x4a,0x9c,0x5c,0x7c, -0x3b,0x96,0x96,0x9d,0x29,0x2c,0x73,0xd7,0x3b,0xfa,0x63,0x92, -0x5e,0x5d,0xc1,0xb2,0x6d,0x06,0x16,0x96,0x20,0xa8,0x5a,0x88, -0x47,0x2c,0x37,0xeb,0x59,0xfa,0x2b,0x87,0xe3,0xfa,0x75,0x5d, -0x1a,0x94,0x69,0xf6,0x92,0x74,0x76,0x85,0x6e,0x76,0xec,0x2f, -0x05,0x27,0xd1,0xe1,0xe9,0x15,0x61,0x5d,0x44,0x6b,0x71,0x83, -0x4d,0x7e,0xc4,0x47,0x11,0x6f,0xf1,0xa8,0x81,0x6a,0x69,0x78, -0x54,0x5d,0x42,0x44,0x69,0xcb,0x86,0xd3,0xcd,0x5e,0x68,0x90, -0x25,0xc9,0xac,0x68,0x3a,0x0b,0xce,0x9a,0xe9,0x5d,0x93,0xdc, -0xa7,0x84,0xe3,0x63,0x44,0x31,0xc0,0x7f,0xcc,0x18,0x8c,0xc5, -0xa4,0xe8,0xda,0x2f,0xa5,0x02,0x84,0x88,0x02,0x88,0x10,0x7d, -0xda,0xbd,0x1d,0x38,0x3b,0x9d,0x3c,0xb2,0x79,0xab,0xdb,0x3e, -0xf8,0x12,0xbe,0xdb,0x8e,0x5e,0x56,0xf7,0x61,0x57,0x78,0x5a, -0x54,0xac,0x1e,0x3b,0x92,0xa1,0x4e,0xcf,0x43,0x64,0xc5,0xd0, -0xe6,0x22,0x05,0x62,0x21,0xad,0xaa,0x77,0x5f,0x22,0xcb,0x68, -0x95,0x61,0x2b,0x15,0x72,0x71,0xc3,0x2d,0xe0,0xba,0xed,0xc8, -0xc7,0x83,0x11,0x11,0x5b,0xd1,0x4f,0x56,0xd7,0xc9,0xd2,0x36, -0xc8,0x65,0x5f,0x89,0xd2,0x03,0x42,0x70,0x1e,0x87,0x5b,0xd9, -0x7d,0x31,0x52,0x59,0x4f,0xb3,0x33,0xb6,0x8d,0x17,0x40,0xad, -0x00,0x97,0x1a,0xc3,0x92,0x51,0xbd,0x64,0xb0,0xdf,0x39,0xa0, -0x86,0xa8,0x01,0x8a,0xd4,0xfd,0x96,0x2e,0xec,0xce,0x50,0xd6, -0x83,0x4c,0x77,0xb4,0xe3,0xe1,0x29,0x3b,0xdf,0x90,0xd0,0xe0, -0xfc,0x86,0x2f,0x7e,0x20,0x1e,0xf4,0x90,0x96,0x4b,0x3c,0xbe, -0x60,0x8d,0x23,0xe7,0x35,0x78,0x9a,0xec,0x1e,0x1d,0x0b,0xe8, -0x20,0xc5,0x1b,0xbd,0x25,0xe4,0x8d,0x66,0x6a,0xc7,0x0d,0x51, -0x92,0x33,0xbf,0x74,0xb7,0xca,0x71,0x1d,0xa0,0x1f,0x2c,0xd4, -0x07,0x60,0xb4,0x3a,0x90,0xc5,0xc6,0x56,0x5b,0x43,0x96,0xed, -0x6c,0x4e,0xb6,0x11,0xb4,0xf2,0x64,0x9b,0x91,0x2a,0xbb,0x0e, -0xc1,0xf0,0x28,0x5b,0x5e,0x42,0x84,0x9f,0x5d,0x37,0x07,0xe6, -0xe2,0x61,0xac,0x7e,0x47,0xc3,0xde,0xbf,0x38,0xb4,0x6f,0x03, -0x2d,0x21,0x25,0x35,0xb3,0xa2,0xf0,0xee,0xa2,0x43,0x4c,0x1b, -0x58,0x3c,0xd4,0x32,0xa3,0x57,0xf0,0x04,0x70,0xac,0x35,0x3c, -0x84,0xb7,0x11,0x34,0x66,0xeb,0x9b,0xcb,0x64,0x69,0x3c,0x2b, -0x74,0xae,0x3a,0xc9,0x89,0x1d,0x83,0xd4,0xa2,0xc4,0x17,0x23, -0x2e,0x45,0x05,0x80,0x09,0xfd,0xb4,0x0b,0xb5,0x43,0xcd,0xe7, -0x5a,0x69,0x58,0xeb,0x56,0x26,0xa0,0xa9,0xf1,0x77,0xd5,0xaf, -0xad,0xf6,0x95,0xd0,0xbb,0xb3,0x4c,0x72,0xcd,0xfe,0xac,0xaf, -0x17,0xd8,0x16,0x90,0xe7,0xf2,0x86,0x3a,0xb2,0xe4,0x7e,0x50, -0xa0,0x57,0x37,0xb1,0xa7,0x57,0x68,0x62,0xac,0x3c,0xdf,0xc1, -0x7f,0x8c,0x56,0xb3,0x9b,0xf7,0x60,0xef,0xb3,0xf1,0xce,0xee, -0x20,0x58,0xa0,0xa7,0xdb,0x68,0xf9,0xef,0xa8,0x76,0xda,0x35, -0xaa,0xb6,0x5a,0x60,0x8a,0x4c,0x85,0xb6,0x5f,0x2a,0xe5,0x19, -0xc2,0xba,0xd5,0x99,0xc6,0xce,0x4f,0x63,0x8d,0x07,0x0b,0x74, -0x5d,0xe9,0x68,0xb5,0x7a,0x07,0x33,0x6a,0x89,0x4f,0x0e,0xd8, -0x6f,0x56,0x8f,0x96,0x60,0xae,0xac,0xe6,0x0b,0xf8,0x27,0x59, -0xf8,0x1f,0x4b,0x1b,0x20,0x71,0x04,0xd3,0x1e,0x02,0xc4,0x04, -0x7b,0x77,0x17,0x06,0xf4,0x23,0x0e,0x7d,0x2a,0x62,0x0e,0xaa, -0xff,0x3f,0xde,0x8e,0x65,0xb9,0x71,0xdc,0x78,0xce,0x5f,0x58, -0x4c,0x56,0x22,0x43,0xda,0xb2,0xe7,0x92,0x8c,0x38,0x1c,0x55, -0x6a,0xb2,0x5b,0xb5,0x97,0x39,0x64,0x73,0x93,0x14,0x15,0x29, -0x53,0x1e,0x6e,0xe9,0x65,0x49,0xce,0x78,0xca,0xd4,0xbf,0xa7, -0x9f,0x78,0xf0,0xe1,0x71,0xed,0xa6,0xf6,0x22,0x42,0x40,0xa3, -0xd1,0x00,0x1a,0x8d,0x46,0xa3,0x01,0xcc,0xde,0x2d,0xea,0xfa, -0x2e,0x6d,0x48,0x17,0xc3,0x4c,0x14,0x58,0x95,0xd5,0x26,0x24, -0x74,0xd7,0x84,0x25,0x1a,0x53,0xb1,0xa0,0xe7,0x25,0xd5,0x3d, -0xda,0x8a,0x98,0xbe,0x96,0x74,0x31,0x5c,0x71,0xff,0x9c,0x46, -0x04,0x83,0x67,0xce,0xe3,0x78,0x91,0x69,0x0d,0xe0,0x37,0xce, -0x10,0x99,0x11,0x2a,0x08,0xc5,0x5a,0xda,0xaa,0xf5,0xf4,0x00, -0x6f,0x71,0xbb,0xef,0x1d,0xec,0x56,0xfe,0x42,0x1c,0x4d,0xeb, -0x49,0x01,0x2b,0xc5,0x7b,0xcb,0x06,0xe8,0x6a,0x35,0x1c,0x22, -0x2c,0x67,0xd7,0x7e,0xc7,0x78,0x6d,0x6e,0x1b,0x23,0x92,0x9e, -0x50,0xf7,0xca,0x78,0x7e,0xa3,0xd4,0xd1,0x36,0x10,0x3c,0xea, -0xda,0x23,0x4e,0xad,0x89,0xaa,0x7f,0xf7,0x81,0x08,0x76,0xab, -0x8a,0xf8,0xc3,0xa6,0x8b,0x17,0x41,0x19,0x8b,0x25,0xd5,0x87, -0x69,0xb5,0x0b,0xbd,0x87,0x93,0x4a,0xfd,0xae,0xa2,0x22,0xa0, -0x7c,0xe5,0x3d,0x53,0x40,0x65,0xda,0xbf,0x29,0x3b,0xeb,0x6e, -0xd6,0xec,0x9b,0x0b,0xa0,0x4d,0x78,0xba,0x86,0xd3,0x91,0xda, -0x0e,0x15,0x98,0xef,0x4d,0x24,0x48,0x05,0xe4,0x40,0xa4,0xec, -0x39,0x41,0xaf,0x70,0x48,0x37,0xd0,0x10,0xdd,0x85,0xb6,0x48, -0x80,0x47,0xbc,0x15,0x13,0xb5,0xf5,0x77,0xb6,0x74,0xac,0x52, -0xd1,0x56,0x67,0x1c,0x9a,0xc9,0xc9,0xf3,0x0d,0x34,0x13,0x21, -0xc8,0x3c,0xff,0x68,0x6c,0xbd,0x32,0x1d,0x18,0xd3,0x4f,0x08, -0x74,0x26,0x1a,0xa7,0x08,0x48,0xdd,0xc8,0x5a,0x6f,0xd7,0x0a, -0xf2,0xab,0xed,0xd3,0xe9,0x7c,0x55,0x94,0x57,0x87,0x1c,0x0f, -0xf4,0x1a,0xea,0xaf,0xd0,0x15,0xfd,0x14,0xf8,0x96,0x22,0xc2, -0x68,0x35,0x8b,0x75,0xf4,0x82,0x26,0xc5,0xb5,0xbd,0x64,0x84, -0xfe,0x91,0x49,0x01,0x55,0xfe,0xe2,0x57,0xb2,0xc8,0x97,0xdb, -0xbd,0xb7,0xb4,0x26,0x4e,0x07,0x41,0xf5,0x45,0x1d,0xc9,0xc9, -0x21,0xc2,0xde,0x9c,0xc9,0x49,0xa0,0x58,0x72,0xc0,0xdd,0x0b, -0x6d,0x3a,0xce,0xa2,0x71,0x8a,0xa1,0x1a,0xed,0x2b,0x0d,0x69, -0x75,0x23,0xb4,0xd3,0x90,0x57,0x06,0x59,0x69,0x30,0xc4,0x36, -0x24,0x13,0xca,0x3a,0x7a,0x49,0xb0,0x50,0x67,0xdc,0x97,0x1b, -0xd7,0x3e,0x41,0x95,0xf8,0x9a,0x57,0xe7,0xef,0xef,0xf5,0x29, -0x11,0xa7,0xf2,0xfc,0xef,0x6a,0x5b,0xc2,0xb2,0x25,0x7c,0x95, -0x47,0xe8,0x68,0x37,0x6f,0xd9,0x71,0x09,0x5c,0xfc,0xda,0x7d, -0x64,0x89,0x59,0xd2,0xae,0x76,0x90,0x38,0xc9,0xbe,0x4c,0x66, -0x44,0xdc,0xdd,0xe2,0x35,0x1e,0x43,0xa9,0x42,0x88,0x91,0x31, -0xce,0xe7,0x4d,0xb3,0x7f,0xb0,0x60,0x7d,0x68,0x40,0xb7,0x16, -0xec,0xa0,0x17,0xfd,0x2e,0x39,0x73,0x7d,0x32,0x39,0x8d,0xce, -0xbe,0xcb,0xa0,0xe0,0x49,0x46,0xe8,0x43,0x09,0x65,0x78,0xa7, -0x02,0x89,0xfb,0xfc,0x5c,0x7a,0x42,0xd6,0xe4,0x12,0x48,0x60, -0xd8,0xfc,0x1e,0xbd,0x88,0xf1,0x3c,0xe8,0xf4,0x96,0x5c,0x49, -0xff,0x09,0x99,0xfc,0xb2,0xda,0xd2,0xc0,0x25,0x2f,0xba,0x74, -0x0c,0x46,0x3a,0xb2,0xb0,0xff,0x9a,0x29,0xc2,0x54,0x4b,0xae, -0xeb,0x46,0xd1,0x74,0x14,0x15,0x68,0x37,0xa4,0x41,0x36,0xdd, -0x86,0xdf,0x82,0xea,0x8c,0xd4,0x61,0xfb,0x5c,0x87,0x90,0x70, -0xad,0x50,0xee,0x2d,0x6d,0xb4,0xf8,0x33,0x63,0xdd,0xce,0x7d, -0xc9,0xed,0xc7,0xcc,0xe0,0x98,0x86,0x2b,0x28,0xf0,0xa8,0x3c, -0x21,0xf5,0x8b,0x7a,0x1a,0x15,0x8a,0xfa,0x5e,0xad,0xa3,0x89, -0x64,0xb5,0x35,0x3a,0x1f,0xf3,0x6a,0xa3,0xad,0x89,0x2f,0x58, -0x09,0x6e,0x87,0x17,0xa9,0x47,0x12,0x43,0x55,0x64,0x4d,0xb5, -0xc4,0x76,0x38,0x09,0xac,0x3a,0xb9,0xa3,0xda,0x6e,0xcb,0xfb, -0x2a,0xd7,0x5b,0xa7,0x04,0x35,0xf3,0x87,0xd2,0x85,0x91,0x30, -0x9f,0x6c,0x0f,0x89,0xef,0x6d,0xe4,0xf4,0xcb,0x6b,0xcd,0x65, -0xb2,0xdb,0x5e,0xeb,0x64,0x22,0x8e,0x84,0xc5,0x93,0x82,0x5d, -0x9b,0x9c,0x29,0xd2,0xfa,0x11,0x13,0xa7,0xbd,0x95,0xa7,0xce, -0x44,0x90,0x68,0x12,0x7a,0xad,0x6f,0xaa,0x68,0xdd,0x1d,0x7a, -0x1b,0x5f,0x4e,0x28,0x7f,0x06,0x16,0x33,0xd9,0x86,0xc3,0x81, -0xe0,0x33,0x37,0xa8,0x6a,0x0f,0xf5,0x77,0x05,0x8d,0xf9,0x48, -0x91,0xd9,0x9d,0xb7,0xde,0x92,0xdd,0x0e,0xdb,0xb7,0x3a,0xcb, -0xca,0x58,0xd4,0x9a,0x1c,0x17,0x92,0xb6,0x10,0x82,0x74,0x12, -0x8d,0x93,0x90,0x20,0x6f,0xc9,0x8f,0xed,0x15,0xc1,0xc8,0xb7, -0xdc,0x52,0x4b,0x91,0x47,0x1f,0x51,0x80,0xfe,0xe0,0x4d,0x76, -0x21,0x17,0xf1,0x8e,0xdd,0x0d,0x47,0x78,0x92,0xd0,0xb2,0xcf, -0x1e,0x59,0xd3,0x13,0xf1,0x93,0x53,0xa6,0x20,0x6b,0x4e,0xa8, -0x5c,0x38,0xda,0xe9,0xf6,0xee,0x63,0x21,0xee,0x7c,0xd9,0xba, -0xee,0xb7,0x63,0xb1,0xdb,0x60,0xc0,0xca,0x9b,0x45,0x71,0x05, -0x04,0xab,0x79,0x54,0xa5,0x23,0x4b,0x34,0xae,0x02,0x5b,0xd4, -0x2c,0x5c,0x3d,0x0c,0x16,0x85,0x44,0x5d,0xbe,0xf6,0xf8,0x96, -0x78,0x34,0xf1,0x44,0x79,0xab,0x4f,0xae,0x99,0x93,0x3f,0xdc, -0x4d,0xfb,0xbb,0x41,0x3d,0x6a,0xb0,0x08,0x72,0x85,0xb4,0x07, -0x01,0x9c,0xb7,0x15,0x49,0x79,0x60,0x4f,0x73,0x10,0x70,0xa2, -0x0c,0xd1,0x96,0x74,0x97,0xa7,0x61,0xf0,0xf3,0x8e,0x5e,0xb1, -0x97,0x47,0xdc,0x02,0x96,0x7f,0x84,0x7d,0xb6,0x48,0x1b,0x0e, -0x18,0x88,0x85,0xa7,0x58,0xf5,0x27,0x19,0x0e,0xc9,0xf7,0x92, -0x2c,0x40,0xf8,0x5f,0x5b,0x03,0x63,0x2f,0xe6,0x92,0xdb,0x86, -0x66,0xd3,0xeb,0x81,0xde,0x76,0xe7,0x94,0xb5,0xfe,0x5b,0xec, -0x0b,0x66,0xa5,0x9e,0xb9,0xbe,0xe8,0xa9,0xeb,0xcf,0x72,0x62, -0xb5,0xaf,0x3a,0xfe,0x1e,0x8a,0x38,0xff,0x1b,0x08,0x22,0x40, -0xa4,0x67,0xe6,0xf8,0xc3,0x1b,0xc2,0x9c,0x97,0xbf,0x00,0x4c, -0xfc,0x8c,0xca,0xe3,0xb9,0x87,0x32,0x6b,0xcc,0xf8,0x4d,0x7e, -0xfb,0x62,0xbc,0xf0,0x08,0xc8,0x24,0x94,0xb6,0x4c,0x17,0x4a, -0x02,0x96,0xc3,0x8e,0x46,0xcd,0x06,0xa3,0x49,0x16,0x35,0xc7, -0x5e,0x2e,0x71,0xd6,0x30,0x52,0xea,0x02,0xb8,0x85,0xf2,0xb4, -0xd9,0x85,0xa3,0x69,0x6f,0x97,0x1d,0x12,0xe9,0x26,0x85,0xee, -0x5d,0x1a,0xd2,0x50,0x7b,0x54,0x1d,0xab,0x37,0xf2,0x89,0x7f, -0x1a,0x08,0x12,0x34,0x0f,0xbb,0x1d,0xf7,0x07,0x24,0x53,0xa2, -0x91,0x38,0x7e,0x85,0x9e,0x23,0xcc,0xcb,0xf3,0xa2,0xd6,0xe2, -0x53,0x0a,0x1d,0x95,0xa7,0x73,0x7e,0xda,0x1b,0xbf,0xd5,0x94, -0x47,0x55,0xa1,0xb3,0x3b,0xa6,0x00,0xda,0x6e,0xb4,0x0d,0x49, -0x36,0xe6,0xc3,0x37,0xd6,0x5c,0x4d,0x3b,0xd2,0x4b,0x82,0x87, -0x6f,0x34,0x0f,0x6c,0xab,0x26,0xcb,0xfc,0x7e,0xf2,0x3a,0x3b, -0xd4,0xd8,0x13,0x89,0x60,0xa4,0x13,0x26,0xb7,0x0e,0xe2,0xac, -0x7a,0xc4,0x14,0xea,0x81,0xdd,0x3f,0xa4,0x37,0xcd,0x8e,0xbd, -0xe9,0x56,0xf6,0x8c,0xef,0xe8,0x62,0x67,0xe9,0x42,0x2f,0xa3, -0x74,0xd3,0xe7,0xdc,0xb0,0x43,0x1b,0x92,0x8d,0xfd,0x49,0xf2, -0xb5,0x24,0xff,0x0e,0xdc,0x6c,0xb4,0x77,0x92,0xe8,0x66,0x25, -0x69,0xde,0xee,0x3c,0x49,0x9b,0xbf,0x1d,0xcf,0x23,0xbb,0xef, -0x45,0xd3,0xa8,0xc6,0xcc,0x7c,0x8e,0xe9,0xd1,0x31,0x61,0x25, -0x45,0x92,0xff,0x72,0xc6,0xcb,0x25,0x0a,0xfa,0xb0,0x1f,0x83, -0xe3,0xa1,0x70,0x75,0x0b,0x42,0x3f,0xaf,0xeb,0xbb,0x31,0xc4, -0xde,0x8d,0x0b,0x7b,0x4a,0x00,0x22,0x39,0x60,0x40,0x29,0x5f, -0x9a,0x7b,0xa7,0xbd,0xa0,0xb9,0xf2,0x2c,0x37,0xa7,0xbd,0xa2, -0xa4,0x68,0x26,0x17,0x59,0x61,0x93,0xd9,0x9e,0x02,0x2a,0xd5, -0xe9,0x33,0x5e,0xda,0xad,0x07,0x01,0xa5,0x2f,0x69,0x89,0x6a, -0x52,0x07,0x8d,0xe4,0x22,0xb2,0x0e,0x7b,0xa7,0xaf,0xd5,0x19, -0x18,0xc1,0xc0,0x82,0xba,0x98,0x9f,0xca,0x60,0xc6,0xb3,0xd2, -0x15,0x67,0x5b,0x04,0x13,0x4b,0xb9,0xdc,0xaf,0x58,0xe0,0x85, -0xe0,0x0e,0xe4,0x67,0x32,0x85,0x3a,0x90,0x83,0x2c,0xce,0xa7, -0x05,0xfc,0x16,0x13,0xe0,0x8b,0x7c,0xaa,0x0d,0x33,0x81,0x4f, -0x5c,0xf8,0x99,0x51,0xa3,0x84,0xac,0x5e,0x9c,0x3c,0x09,0x6c, -0x30,0xc6,0x1d,0xf9,0xfe,0x55,0x3e,0xfc,0xf8,0x7c,0x70,0x0a, -0xbd,0x91,0x97,0x58,0xa0,0xa9,0x38,0x34,0x1c,0xe6,0x37,0x7c, -0x50,0x14,0xe3,0x38,0x84,0x71,0x5b,0x18,0x19,0xa8,0xb8,0x13, -0xa8,0xf9,0x83,0x29,0xd5,0xc3,0x6e,0x7f,0x2c,0xf1,0xce,0x73, -0x4c,0xb2,0xff,0xd0,0xd2,0x1a,0xc8,0x64,0x6d,0x0c,0xbe,0xd0, -0xb7,0xad,0xb8,0xc2,0xb6,0x6e,0xc3,0x06,0xc7,0x1c,0xe4,0xcf, -0x13,0x6a,0x34,0xa4,0xa4,0x99,0x9c,0x85,0x70,0x9c,0x62,0x8a, -0x46,0x42,0xc1,0x2e,0xd0,0x9f,0xd0,0x94,0x96,0xa3,0x6c,0x38, -0x9d,0x8f,0x4f,0xf4,0xae,0x7c,0x41,0x71,0x85,0x1b,0x47,0x6e, -0x37,0x18,0x8d,0xae,0x37,0x9f,0xc8,0x5b,0x6e,0x10,0x7a,0x53, -0x04,0xa5,0xc2,0xfc,0x40,0x5f,0x97,0xe3,0xf2,0x4f,0xc6,0xb9, -0xce,0x00,0x17,0x02,0x5c,0x34,0x81,0x39,0xc1,0xd6,0x5b,0x2a, -0xca,0xfb,0x53,0x91,0x0c,0x19,0xfe,0x5b,0x30,0xeb,0x92,0x33, -0xc0,0x6d,0x62,0x3d,0xd6,0xb1,0x75,0xb5,0x5b,0x69,0x9c,0x2f, -0x82,0x2c,0x73,0xd8,0x12,0x25,0x10,0xe6,0xc9,0x55,0x1d,0x90, -0xac,0x14,0x09,0xb5,0x16,0x6d,0x80,0x4c,0xd7,0x18,0x77,0x7d, -0x6d,0x15,0xfb,0xf2,0x31,0xcc,0x67,0x18,0xb9,0x48,0x0a,0xf9, -0xfa,0x83,0x39,0x4a,0xa3,0x94,0x9f,0x4b,0x6b,0x7a,0xbe,0x62, -0xdf,0xb0,0xe6,0x95,0x8b,0xde,0x45,0x64,0xc4,0x71,0x32,0x50, -0xec,0x9c,0x5c,0x48,0x32,0x95,0x85,0xf2,0x18,0x8a,0xa2,0x4f, -0xa3,0xa4,0x28,0x22,0x37,0x6d,0xac,0xaf,0x18,0xca,0xa8,0x4c, -0x29,0xaf,0xb0,0xe5,0x29,0xc2,0x01,0xd7,0x46,0xb2,0x69,0x83, -0x61,0xe4,0xc5,0xbc,0x7c,0x26,0xed,0xbb,0x3f,0x84,0xb6,0xb1, -0xe9,0x8f,0xa8,0x16,0x74,0x5f,0xd9,0x8f,0x8f,0x4f,0x9e,0x9b, -0x42,0x52,0xd8,0xc9,0xe0,0x91,0x24,0xdc,0x6c,0xa1,0xdb,0xbf, -0xe2,0x51,0x99,0xb5,0x94,0x5b,0xe3,0xaa,0xa1,0x9d,0x4d,0x3d, -0xe7,0x0b,0x67,0xde,0xf5,0x16,0x39,0xe1,0xfa,0x64,0xca,0xfc, -0xa0,0x03,0xa0,0x63,0xb6,0x33,0xb5,0x57,0x3d,0xb7,0xc3,0xb5, -0x98,0xc9,0x93,0xe3,0xe9,0x5d,0x73,0xc7,0x20,0x1c,0xec,0xf1, -0x56,0xb7,0xbb,0x01,0x17,0xb7,0xdb,0xdf,0x97,0xa8,0x78,0x4b, -0xcd,0xd8,0x11,0xc6,0x3b,0xe7,0xdb,0xd4,0xe2,0x19,0x4f,0x9b, -0x1f,0x7d,0x39,0x4a,0x66,0xb8,0xc4,0x4e,0x54,0xbd,0xce,0x35, -0x99,0xb7,0x1c,0xb8,0x24,0xf2,0x92,0x83,0xf1,0x0b,0x08,0x92, -0x40,0xc7,0x18,0x04,0xb9,0x08,0x08,0xb0,0x48,0x85,0x00,0x8a, -0x47,0xf8,0xb0,0xb4,0x73,0x1f,0x7d,0xd8,0xd1,0xd0,0x58,0xce, -0x82,0xea,0x14,0xc4,0x7c,0xb7,0x45,0x27,0x0d,0x6d,0xba,0xb3, -0xcc,0xd4,0x8e,0x73,0xc6,0xc1,0x22,0xb8,0xb0,0x57,0x97,0xe3, -0xb0,0x60,0xd7,0x40,0xa0,0x73,0x78,0x49,0xaf,0xb6,0xfc,0xc0, -0x76,0x62,0x80,0x45,0x96,0x65,0x40,0x5b,0xd3,0x81,0x66,0x32, -0xd2,0x72,0x7c,0x33,0x86,0x21,0xe5,0x4a,0x99,0x4e,0xc4,0x36, -0x63,0xa6,0x62,0x16,0xe7,0x69,0x26,0xf7,0x27,0xdc,0x85,0xef, -0x51,0x23,0x34,0x55,0x9c,0x8f,0x07,0x15,0x88,0x93,0xcf,0x21, -0x5d,0x71,0xf7,0xd3,0x66,0x9f,0xcb,0x02,0x8d,0x7b,0x11,0x52, -0xfa,0x75,0x11,0xee,0x0c,0xeb,0xc3,0x3c,0x20,0xe7,0x24,0xce, -0x28,0x53,0xd5,0x6b,0xfd,0x3f,0xb8,0xad,0x6b,0x09,0xdd,0xc1, -0x9c,0xd1,0x9a,0xe4,0xfa,0x79,0x0b,0xdf,0x2b,0x7f,0xcd,0x6d, -0x2a,0x33,0x64,0x98,0x27,0xa8,0xbb,0xc1,0x5d,0x15,0x0d,0x33, -0x40,0x0f,0xf5,0x9f,0x53,0xf0,0x2f,0x0e,0x30,0x34,0x11,0x0c, -0x66,0xde,0xed,0xf1,0xe2,0x05,0x50,0xc0,0x3a,0x5e,0xe4,0xbf, -0x92,0x03,0xe2,0xed,0xbb,0x16,0xe4,0x7d,0x68,0x6b,0x70,0xee, -0x73,0xbc,0x35,0xae,0xcd,0xb4,0x38,0xb7,0x50,0xbb,0x0e,0x27, -0x11,0x63,0x59,0x58,0xad,0x9e,0xb6,0x7c,0x62,0x3f,0xf4,0x5d, -0x7e,0x69,0x89,0xb8,0x93,0x15,0x19,0x81,0xe1,0xea,0xb0,0xdb, -0xb7,0xce,0xee,0x86,0x13,0xa0,0xec,0xd5,0xdd,0xef,0x9d,0xd7, -0xd2,0xb7,0xd5,0x2e,0x01,0xdc,0x8d,0x7e,0x80,0x18,0xe0,0x64, -0x3c,0xb4,0x40,0xe9,0xd5,0x0e,0x77,0xe6,0xe0,0x13,0x13,0x2d, -0xeb,0xcd,0x1e,0xc8,0xa4,0xa0,0xf8,0x2e,0x47,0x7f,0x45,0xe0, -0x6b,0x84,0xb8,0x53,0xaf,0x64,0x6e,0x14,0xbc,0x7e,0xe1,0xa5, -0x3c,0xad,0xf2,0x43,0x39,0x79,0x09,0x86,0xc1,0x24,0x18,0xa2, -0x9d,0x0d,0x24,0xc0,0x07,0x0c,0x6f,0xce,0x18,0xfc,0x88,0xc1, -0x07,0x0c,0x8e,0x82,0x11,0x04,0x1f,0x9f,0xf6,0x14,0x3f,0xc2, -0xf8,0x3f,0x3f,0xbf,0xfb,0x5b,0x0a,0xc3,0x39,0x35,0x08,0x6f, -0x9e,0x76,0x8c,0x31,0xd3,0xf5,0x6e,0x68,0xd3,0x38,0x25,0x72, -0x48,0x00,0x51,0x53,0x3e,0x43,0xc3,0x2b,0x19,0xb8,0xea,0x66, -0xf1,0x03,0xb3,0x74,0x10,0xcb,0x2a,0xb8,0x85,0x41,0x5e,0x1f, -0x0d,0x22,0x94,0x26,0x49,0xf0,0x10,0x44,0x89,0x96,0xeb,0xa1, -0x08,0x3b,0x50,0x28,0xa0,0x22,0xa9,0x11,0x4b,0xc4,0x58,0x70, -0x02,0x13,0xa1,0xc9,0x40,0x10,0xad,0xf0,0xae,0x3c,0xd4,0xd3, -0x41,0x4b,0x3d,0xa6,0xe3,0xee,0xb0,0xe2,0xe0,0x6a,0xf4,0x19, -0x47,0x4e,0x83,0x60,0x02,0x44,0xc7,0x02,0x72,0x73,0x2c,0x0f, -0x9b,0x1c,0x56,0x16,0x5e,0x53,0x28,0x46,0xa7,0x30,0xbc,0xde, -0xca,0x4e,0xa1,0x5a,0x11,0x05,0x9c,0x51,0x3a,0x9e,0xe2,0x21, -0x31,0xe5,0xd8,0x18,0x75,0xc0,0x81,0x00,0xc0,0x9b,0x5e,0x69, -0x84,0xf9,0x93,0x2b,0xa4,0x44,0xde,0xa8,0x4d,0xad,0x97,0x29, -0x27,0xcf,0x34,0xe3,0x22,0xfd,0x9e,0xaf,0xba,0x19,0xbc,0x88, -0x75,0xe2,0x1c,0xe4,0x78,0xae,0x9a,0x22,0x8b,0xda,0xd8,0x31, -0x43,0xf0,0x8a,0xa8,0x31,0xdd,0xa8,0xbd,0x2f,0x5b,0xca,0x75, -0x4a,0xb8,0xcc,0xa3,0x10,0xf4,0x92,0xd9,0x58,0x9c,0x35,0xa6, -0x23,0xd7,0x12,0xe9,0xdd,0x6b,0xf1,0x16,0x93,0xa4,0x1e,0x96, -0xc0,0x8a,0x90,0x6d,0xce,0xb1,0xd5,0x2d,0xc5,0x48,0x7b,0xd1, -0xc7,0x90,0xab,0xfb,0x4f,0xe8,0x58,0x5c,0x1e,0x61,0xd0,0xab, -0xcf,0xd9,0xcf,0x8e,0x4c,0x04,0xb1,0xb4,0xae,0x9e,0xf5,0xc9, -0xdd,0x2c,0x8e,0x4d,0x86,0x38,0x08,0x0c,0x31,0x04,0x34,0xe5, -0x0f,0x40,0x4c,0x2a,0x3a,0xbb,0x70,0x2e,0xb7,0x87,0x0d,0xdd, -0x49,0x73,0x3e,0x03,0xaf,0xe0,0x00,0xc1,0x06,0x85,0x98,0xc9, -0xf8,0xc3,0x0f,0xe1,0x6c,0x7e,0x9a,0xff,0xb2,0x88,0xa7,0xd1, -0x0f,0x1f,0xc7,0x0f,0xbc,0xdc,0x3c,0xec,0x37,0x92,0x9a,0x35, -0x92,0x65,0x5c,0x40,0xca,0xb5,0x9f,0xc2,0xd5,0xd8,0xed,0xe9, -0x39,0xf5,0x6c,0x1c,0xde,0x44,0xff,0x19,0x0b,0x34,0x14,0x48, -0x23,0x7c,0x04,0x03,0x60,0x3e,0x87,0x00,0xfc,0x40,0xe8,0x88, -0xd7,0x5a,0x62,0x60,0x07,0x01,0xd4,0x21,0xfe,0x04,0x5f,0x7c, -0x4d,0x6a,0xfe,0xf4,0xee,0xf6,0xdd,0xdf,0xe1,0x0f,0x7f,0x25, -0xe2,0xbd,0x44,0xbc,0x0f,0x2e,0x82,0xf7,0x98,0x8d,0xe7,0xf3, -0x7a,0x54,0xcf,0x8f,0xf5,0x7c,0x57,0xcf,0xcf,0x35,0x67,0xe4, -0xcf,0xfb,0xf1,0x43,0x6a,0xab,0xee,0x98,0x57,0x51,0x5e,0xd2, -0xbd,0xb1,0x27,0x69,0x0e,0xf5,0xb2,0x42,0x81,0x9f,0x6a,0xa4, -0x77,0x77,0xd8,0xc5,0xc0,0x76,0xb4,0x26,0x4b,0x20,0x1a,0x37, -0x72,0x9b,0x94,0x08,0x8c,0x59,0xa8,0xb9,0x44,0xd2,0xc0,0x92, -0x9a,0x9b,0x27,0x92,0x75,0x5e,0x62,0x21,0x9c,0x66,0x7f,0x0d, -0x4c,0x3b,0xae,0x05,0xb3,0x70,0x05,0x50,0xfd,0x17,0x91,0x63, -0xe6,0x10,0x0d,0x2f,0x30,0x83,0xe5,0xf2,0x10,0x67,0xa3,0x20, -0xc5,0x46,0x30,0x12,0x43,0x48,0x6f,0xc8,0x08,0x69,0x63,0x97, -0x1f,0x12,0x2d,0x3d,0xd9,0xaf,0xd7,0x27,0xe7,0xd8,0x25,0xa3, -0x8f,0x33,0x42,0xcb,0x06,0x0e,0x39,0x00,0xc0,0x70,0x56,0x38, -0x71,0xbf,0xf5,0x88,0x23,0xe4,0x8b,0x58,0x58,0xc6,0x08,0x22, -0xa1,0x03,0x97,0x30,0x52,0x4a,0x30,0x8a,0xe7,0xbb,0x10,0x6f, -0xa6,0xcd,0x42,0x85,0x07,0xa1,0x0b,0xda,0x21,0x8a,0xa2,0xe9, -0x68,0x84,0xb6,0x15,0x8a,0x45,0x98,0x28,0x02,0xe8,0x11,0xb5, -0x85,0xa9,0x48,0x2f,0x32,0x07,0xa6,0x81,0x11,0x10,0x09,0x1e, -0x6d,0x04,0x0f,0x49,0x0a,0x4c,0x1c,0x6b,0x4a,0x0c,0x2c,0x2d, -0x2d,0xad,0x5d,0xc0,0x0d,0x11,0x53,0xa5,0x74,0x2d,0x48,0x7f, -0xf0,0x98,0x91,0x87,0xc5,0xb0,0xda,0x0d,0xb0,0x55,0x95,0x17, -0x1b,0xdc,0x65,0xd2,0xfe,0x43,0x07,0xd7,0x90,0x94,0xd5,0x97, -0x4b,0xf4,0x82,0x65,0x4a,0xe6,0xe0,0x02,0x7f,0x22,0xd3,0xcf, -0x72,0x73,0x75,0x02,0x54,0x64,0xa3,0x11,0x7c,0x7e,0x6d,0xbd, -0x50,0x88,0xec,0x02,0x82,0xbc,0xda,0x79,0xba,0x10,0x91,0x0d, -0xe0,0x4d,0xf3,0xda,0x68,0x04,0x6b,0xcd,0xd4,0x2d,0x50,0x05, -0xf8,0xf2,0x80,0xd1,0xe9,0xf9,0xf8,0xed,0x85,0x87,0x10,0x0d, -0x01,0x23,0xd3,0x3b,0x6a,0x83,0xb6,0x07,0xe0,0xcf,0x65,0x20, -0xe4,0x46,0x97,0x15,0xb6,0x44,0x08,0x52,0x9a,0xf7,0x1c,0x4a, -0x35,0x88,0x08,0xfb,0x93,0x15,0x83,0xae,0x5f,0x37,0xb6,0x67, -0x2c,0x88,0xaf,0x5c,0x5f,0xf2,0xf8,0x6b,0x0f,0x74,0xef,0x61, -0x3f,0xce,0xe1,0xc8,0x62,0xc9,0x6b,0xb6,0x84,0x35,0xbf,0x16, -0x6d,0x74,0x77,0xe0,0x8a,0xde,0x4a,0xe0,0x5c,0xef,0xf7,0x42, -0x90,0x28,0x22,0xb2,0x14,0x7e,0xc9,0x5b,0xb3,0x95,0xb6,0x1b, -0x1f,0x04,0x20,0x88,0xd0,0x5c,0xf7,0xd2,0x9a,0x6d,0xfd,0x47, -0x43,0x96,0x04,0x3e,0xf5,0xf3,0x92,0xe1,0x30,0x95,0xa9,0x31, -0xc4,0x6b,0x8f,0xe4,0x99,0xbe,0x3d,0x3e,0xa1,0x4d,0x4f,0x63, -0x26,0xd0,0x5b,0xff,0x45,0xc7,0x58,0x08,0x9d,0xbe,0x54,0x6b, -0x14,0xb2,0x68,0x3e,0xc7,0x0f,0x3d,0xc6,0x4a,0x9a,0x09,0xa7, -0xb4,0x16,0x6a,0xbc,0x4f,0x88,0xda,0x81,0x73,0xfb,0xd4,0x5b, -0xa7,0x4e,0x5c,0x3d,0x78,0x13,0xa7,0xbd,0x9f,0x02,0x31,0xaa, -0x1f,0x39,0x9e,0xa2,0xb5,0x13,0xa7,0xd0,0x38,0xc8,0x10,0xe5, -0x70,0xa8,0x34,0xf2,0xff,0xba,0xbe,0x1d,0xb8,0x0b,0xf2,0xba, -0xe6,0xeb,0x68,0xe9,0x1c,0xdb,0xed,0xa2,0x3d,0xeb,0xd2,0xa2, -0xe4,0x62,0x9a,0x85,0xcd,0xdb,0x50,0x61,0x1c,0x00,0xd8,0x00, -0x84,0xfb,0xff,0x57,0x6b,0x6f,0x7b,0xc4,0x21,0xc3,0xab,0xb0, -0xd7,0x24,0x4e,0xd5,0x45,0xed,0x12,0xcb,0xb0,0x53,0x4e,0xf2, -0x42,0xbd,0x3d,0x69,0x17,0xe4,0x30,0x06,0xee,0xcd,0xf2,0x5a, -0xa5,0x71,0xdc,0xaf,0x01,0x2c,0xe5,0xa2,0xee,0x61,0x09,0x8c, -0xd2,0xff,0x05,0x00,0x00,0xff,0xff,0x09,0x6f,0xc0,0x81,0x67, -0xfd,0x08,0x00, - })) + var empty [0]byte + sx := (*reflect.StringHeader)(unsafe.Pointer(&_scripts_stats_modules_js)) + b := empty[:] + bx := (*reflect.SliceHeader)(unsafe.Pointer(&b)) + bx.Data = sx.Data + bx.Len = len(_scripts_stats_modules_js) + bx.Cap = bx.Len + + gz, err := gzip.NewReader(bytes.NewBuffer(b)) if err != nil { panic("Decompression failed: " + err.Error()) } - var b bytes.Buffer - io.Copy(&b, gz) + var buf bytes.Buffer + io.Copy(&buf, gz) gz.Close() - return b.Bytes() + return buf.Bytes() } + func init() { go_bindata["/scripts/stats-modules.js"] = scripts_stats_modules_js } diff --git a/mod/dashboard/resources/scripts-stats-scripts.js.go b/mod/dashboard/resources/scripts-stats-scripts.js.go index 98c7386738d..e446b2dce02 100644 --- a/mod/dashboard/resources/scripts-stats-scripts.js.go +++ b/mod/dashboard/resources/scripts-stats-scripts.js.go @@ -4,2745 +4,36 @@ import ( "bytes" "compress/gzip" "io" + "reflect" + "unsafe" ) +var _scripts_stats_scripts_js = "\x1f\x8b\x08\x00\x00\x09\x6e\x88\x00\xff\xec\xbd\xfb\x7f\xdb\xb6\x92\x28\xfe\xfb\xfd\x2b\x1c\x9e\xbb\x2e\x69\x41\xb2\xa4\x34\x7d\x50\x65\xf4\x49\xdb\xf4\x9c\xdc\x6f\xfa\xd8\x36\xa7\xe7\x74\x15\x6d\x3e\x94\x44\xcb\x8c\x65\x51\x21\x69\x9b\x8c\xad\xfd\xdb\xbf\xf3\xc0\x93\xa2\x64\xa7\xaf\xbb\x7b\x77\x4f\x4f\x64\x02\x18\x0c\x06\x83\x01\x30\x00\x06\x83\xeb\x65\x74\x76\xb5\x9e\x97\x69\xb6\xf6\x17\x8f\x45\x99\x6d\xb2\xb7\x45\xb6\x0e\x6e\x55\xec\xd1\xf5\xf2\x4d\x52\xcc\xe3\x4d\xf2\xa6\x28\x73\xbf\x0a\x6e\xf3\xa4\xbc\xca\xd7\x47\x55\x2f\x4f\x36\xab\x78\x9e\xf8\x26\xf9\x4d\x9e\x08\xef\x7f\x0f\x5e\xbf\xfe\xc8\x0b\xb6\x36\x86\x32\x87\x40\x5c\x26\xdf\xaf\xff\x91\xe5\x0b\xbf\x10\xab\x64\x2d\xf2\xe4\x3a\xb8\xbd\x8e\xf3\xa3\xf9\xba\x8c\xfa\x50\xf6\x45\x54\xf4\x8a\xcd\x2a\x2d\x7d\x2b\xcb\x9b\x1b\xc8\x01\x88\x83\x91\x2c\xb8\x88\x20\xe3\xd8\x47\x70\xf8\x07\x54\x5c\x27\x79\x91\xf8\x41\xd0\x3b\x4b\x57\x65\x92\xfb\xba\x42\x37\x9a\x58\x28\xa1\x13\xdd\xf4\xa0\xd4\x65\x79\x8e\x85\x3f\x8d\x20\x6a\x1b\x98\xdc\x21\xe2\xfa\x70\x04\xa2\x90\x71\xe3\xa2\xf7\x36\x4b\xd7\xbe\xe7\x05\xbd\x32\x4f\x2f\x19\xe3\xa4\x3f\xed\x15\xab\x14\x98\xd4\xc7\x4c\x2e\x53\x6e\xf2\xb4\x4c\xfc\xcb\x62\x09\x5c\x58\xf6\xe6\xd9\xfa\x2c\x5d\xf6\xd2\xe2\xbb\x6c\x91\x8c\x37\x79\x36\x4f\x0a\x60\x47\xb9\x48\xf2\xbc\xa7\x41\x3b\xde\xeb\xb5\x17\x84\x00\x5c\x64\xab\xa4\xb7\xca\x96\x84\xc0\xc1\x7b\x19\x5f\x24\xc0\xbe\x24\xf1\x17\x9a\xfe\x45\xef\xcd\x9b\xeb\x25\x46\xbe\x79\x13\x3d\xea\x8b\x45\x6f\x0d\xc5\x14\xa6\xf1\x35\x24\xb1\x1e\xc0\x28\xdd\x2f\xcf\xd3\x42\x4c\xa6\xc1\x56\x2c\x1a\x0d\xaa\x41\xf2\x2c\x2b\x05\x7d\x72\x6b\xa6\x62\x1e\x61\x5c\xef\x3a\x5e\x5d\x25\x85\x58\x47\xf3\xf1\x5c\x72\x29\xec\x8f\xce\xb2\xdc\x27\xe8\xde\xe6\xaa\x38\xa7\xdc\x81\x48\xa3\xfe\x68\xfd\x34\x1d\x75\x3a\x69\xe0\x62\x9f\x4f\xd2\xa9\xc4\xae\x04\x80\x42\x0e\x35\x8b\xb8\x8c\xdf\x2c\xae\x40\x74\x50\x62\xb0\xda\x4c\xc8\x5a\x54\xd1\x62\x94\x9e\x81\x3c\x01\x63\x9f\xe5\x79\x5c\x43\x62\x80\x24\x54\xd1\x64\x8a\xc5\x02\x79\x0b\x49\x9c\xa6\xa0\x62\xd2\x5a\x10\x03\x31\x41\x30\x4a\x56\x45\x72\xa4\xb0\x7e\x3f\x7b\x9b\xcc\x4b\x44\x7b\x5b\x45\xb7\x5b\xaa\x5f\x7a\x94\x02\xcb\x83\x0a\xc0\xa3\x3d\x58\xb6\xaa\x1b\x39\x15\x59\x65\xf1\xe2\xcd\x79\x5c\xfc\x90\x67\x65\x36\xcf\x56\xfe\x55\xbe\xb2\x5b\x86\xd2\x37\x32\xf1\xc7\xe7\xbd\x32\x29\x4a\x82\xd9\xc5\x92\x16\xdf\xa4\xab\xc4\x41\xd0\x8f\xa2\x08\xc2\xbd\x74\xbd\x48\xaa\xef\x91\x7e\x06\x05\xb1\x4f\x54\x89\x2d\x98\xaa\xf3\x1c\xd1\x88\x79\xbc\x5a\xcd\xe2\xf9\x05\x89\x2b\x4a\x9e\xf7\xf2\xfb\x67\x5f\x87\x47\x5e\x07\x0b\x11\x8b\xc7\x3d\x05\xa9\xa5\x0a\x64\x17\xba\x79\xb1\x41\x12\x8a\xcd\xf1\xb1\x8f\x7f\x22\xfc\xe9\xe1\x0f\x48\x71\xf2\x2a\xa9\x40\x00\x14\x6e\x93\x63\xdb\x42\x09\x12\xea\xe3\xcf\x1e\x5a\x8e\xbe\x79\xf1\xf2\x39\x12\x84\x30\xc1\x88\x84\x60\x51\x45\x18\x3a\x5c\xe9\x11\x80\x3d\x8d\xfa\x40\x20\x46\x73\x06\xee\xb7\x6d\xe0\x52\x5e\x82\x00\x28\x7d\x77\x95\xe6\x89\xef\x9d\x15\x1e\x0e\x26\xf1\xe2\x9b\x5d\x12\x5b\x5a\xb8\x2c\x37\x07\x58\x7a\xf4\xb7\x57\xaf\x7e\x50\x7c\xa5\x5a\x40\x39\x91\x2e\x0b\x73\x53\x69\xef\xae\x64\xf3\x1b\x86\xe7\xaa\x17\x6e\xb2\x02\x84\x1b\x25\x2f\x5a\x27\x37\x47\x5f\x5e\x9d\x9d\xc1\xc8\xb6\x89\x61\xb0\x7b\xb1\x2e\x11\xae\x77\x0e\xe4\xc2\xe0\x37\xf1\x60\x34\x29\x93\x75\xd9\xe5\x6a\x79\x53\x31\xe8\x07\xd8\xdb\x8a\x1e\x60\xf4\xa0\x45\xb2\xdc\x73\xda\x34\xb8\x75\xda\x6b\x7d\xb5\x02\xb9\x41\x6e\x70\x0e\x2c\xd5\xca\x00\x73\x46\x05\xc3\xdb\x06\xfa\x1e\x24\x08\xa0\x2c\xc0\x9f\x4e\x54\x49\x46\x5a\x59\x93\xf5\xc2\xca\x69\x95\x83\x65\x50\x75\x50\x30\x90\xb8\x77\x0f\x23\x8e\xe9\x7a\xd7\x03\xc4\xbe\xdb\x12\x71\x95\x16\x3f\x5d\xcd\x16\xe9\x75\xba\x48\x7c\x98\xc3\xa0\xcd\xca\x74\x7e\x51\x88\xcb\xe0\x16\x7a\x77\x71\x35\xa3\x20\x0e\x14\x97\xc7\xc7\xf4\x2d\x09\x7e\x3a\x80\x09\x12\xfa\x39\x32\x5a\x81\x89\xb7\xe2\x5a\x80\x2c\x03\x23\x23\x85\x1d\x71\x3e\xa7\x28\xc6\xdf\x5b\x64\x97\x31\x4c\x12\x01\x8e\x78\xdd\x01\x8c\x3d\x36\x56\xb1\x88\x7c\x0a\x4f\x06\xd3\x2e\x7f\xf4\xa7\xc1\x69\xa7\x73\x89\xa3\xd2\x17\xeb\x11\x0d\x5c\x6f\xa3\xcb\x51\xb7\xfb\xf6\x69\x7f\x14\xf8\xd7\x51\x87\xe1\xd2\x69\xf7\xed\xc9\x22\x78\x1a\x31\x01\x90\xef\xf8\x58\x11\x26\x07\xb3\x80\x46\xa6\x6e\x37\x15\x6f\x61\xb0\xed\x74\xde\x7e\x01\x95\xb2\x31\x74\x10\xc3\x17\x12\xc1\x60\x3a\x0a\x9a\x08\xd4\x90\xa5\xe2\x77\xb9\x69\xd5\x97\x6b\xca\xb2\x58\x94\x71\x5e\x46\x1c\x03\xa4\x89\x02\xd4\x0c\x15\xe4\x3f\x92\x03\x5d\x28\x56\x15\x02\x30\x4f\x29\xe3\x78\x42\x7f\x28\xd7\x34\x9c\xe0\x1f\x41\x31\xd3\x76\x02\x7e\x8c\xd7\x4b\xd9\x9e\x7a\xe4\x63\xee\xe7\x98\xc2\xf4\x8d\x77\x62\x60\xc2\xde\xdb\x6c\x04\x06\xad\xb6\x53\xe0\xcb\x78\x96\xac\x08\x74\xe1\x67\x79\x0a\x39\xc4\x0a\xa3\x0a\x91\xad\x16\x84\x47\x40\xef\xe3\x8f\x22\x7d\x9f\x88\x4d\x0c\x93\x13\x7e\x45\xdf\xc6\xe5\x79\xef\x32\xae\x7c\x8a\xef\x07\x1d\x48\x12\xbe\xb7\x4a\xce\x4a\x0f\xc6\x69\xc6\x76\x77\xe7\x41\x75\x4d\x38\x80\x26\x43\xf8\x13\x90\x9e\x40\xb8\x69\x00\x3b\xcb\xca\x32\xbb\x34\x51\x63\x9c\xa2\x12\x26\x8f\xe9\xea\xc1\xe4\xb1\x49\xf2\x32\x85\xfe\x06\x00\x49\x2e\x6e\xab\x50\xd3\x5a\x87\xb7\x34\x69\x87\x58\xc6\x16\x7a\xce\xa1\xec\x57\x1b\xe8\x8e\x09\xe6\xd7\x55\x74\xf3\x8b\x78\x95\x2e\xd7\x2a\xca\x9b\x53\x79\xde\x56\xcc\xe2\x22\x59\xa5\xeb\x44\xa5\x48\x5e\x7e\x29\xa3\x27\x4c\xfb\x14\xca\x0f\xc2\x87\x55\xc0\x29\xb5\xd6\xf5\x79\x70\x05\x1a\xf9\x75\x7d\x9c\x0a\x48\x32\x9f\x61\x9c\xa6\x71\xa7\x32\xde\x65\xba\x58\xac\x12\x0f\xa9\xdf\x11\x97\x57\xd8\x6d\x5c\x71\xe1\xb1\xa3\x5d\x5a\x64\xef\x81\xf2\xa2\x7b\xe4\x62\xdc\x1d\x84\x83\x11\xc9\xd5\xe0\x14\xa7\x79\xfc\x1c\xdf\x2f\x1f\xb7\xcb\x3c\xbb\xda\x00\xd1\x71\x7e\xd1\xa3\x6f\x98\x15\xd2\xe5\x79\xe9\x89\xcb\xab\x55\x19\x76\xb1\xf0\x6d\xd8\x02\x76\x93\x2e\x60\xaa\x70\xa1\x14\x17\x97\xeb\x13\x66\xe5\x07\x09\xa8\x1c\x6b\x1e\x24\x9f\x7d\x68\xa6\x21\xb7\x97\xdd\xc6\x3b\x28\x0e\xc9\xe8\x43\x71\x24\x55\x5a\xda\x18\x1a\x62\xb9\x9f\x6c\x53\x4e\xc5\xe5\xec\x13\xcd\x03\x64\xb7\xe1\x30\x84\xdc\x4f\xb6\x0d\xdd\x26\x8f\xa5\x1c\xe9\x2c\x79\x84\x28\x41\x03\x9e\xc8\xce\xce\x8a\xa4\x64\x29\x04\xb1\x8e\xfe\xe3\x3f\x7c\x9f\x52\x70\x82\xe2\x0f\x9c\xa0\x86\xb0\x0a\x22\x19\x6d\xb6\xb6\x2b\xb3\x2c\xa3\x3b\x22\xb0\x23\xca\x76\x9d\x80\x96\x83\x7c\x01\xaa\xb6\xf6\xa8\x03\x92\xc7\x44\xc3\xe0\xb3\x5e\xae\x12\xc3\xbf\x2d\x0e\xef\x0f\x47\xec\xa0\xaa\x9d\xe2\x1c\xc4\xdd\xcf\x11\xf5\x0e\x63\xbf\xa6\x59\xcd\xe5\x2c\xcf\x74\x92\xb5\xa6\x7b\x6f\x60\x1e\x18\xf9\xf7\xf0\x4e\x8d\xfb\x30\xec\x53\xd7\x02\x05\x0a\xb2\x45\xf7\x33\xd3\xfb\xd6\xeb\xa8\x96\xea\x78\xc2\xeb\x60\xee\x8e\xf7\x73\xff\x6f\x2a\x7e\x00\xf1\x3f\x73\x7c\x88\xd0\x0c\x20\xec\x6c\x7f\xeb\xff\x6c\x43\xff\x8d\x81\x64\x85\x76\xd9\xd8\x23\xda\x24\x83\xf0\x7b\xbb\xc3\x9f\xbf\x13\xa0\x0f\x8b\xda\x4b\x41\xe3\x89\x28\x01\xbf\x54\x5d\xb3\x88\x02\xe3\xdb\x6d\x48\x00\xdc\x0c\x11\x7e\xf7\x68\x04\x5a\x24\x67\x3d\x8e\x14\x5c\xcd\x66\x1a\x33\x9c\x86\xa8\xc8\x1a\xae\x04\x8f\x6d\x91\x3d\xd0\x8d\x8a\x9b\xb4\x9c\x9f\x3b\x6b\x39\x29\xf7\xc0\x73\x59\x74\x97\x73\x90\x32\x50\x4c\x38\x92\x43\x53\x09\xc2\x8b\x5d\xd0\xeb\x64\x7b\x81\x0e\x5a\x24\xdc\x88\x61\xd6\xab\xa2\xae\xa2\xb7\x57\x83\xf6\x35\x83\x95\xc2\xc5\x88\x40\x72\x1a\x6d\x09\x86\x68\xec\xec\x03\x94\x4d\x4d\x90\x7d\x4a\xe6\x0a\xc8\x0c\x36\x28\x8a\x80\x05\xd7\x75\x20\x80\x41\x31\x0e\xda\x26\xbd\xbf\x25\x7e\x83\x8a\x8b\x7f\x60\xa1\x04\xe3\xd7\x26\x5b\xe9\x06\xca\xf6\x4c\x64\x7a\xcf\xe0\xb6\xac\x37\x20\x3c\xf9\x15\xcc\x7b\x82\xb2\xc7\x00\x7d\x9d\x84\x8f\x06\xe2\x22\xa9\x43\xb9\x20\x30\x82\x12\xde\xd2\x20\x19\xde\x16\x65\x9e\x5d\x58\x8a\x80\xda\x05\xc1\x32\x7a\x38\xa4\x7d\x95\xad\xb2\x7c\x2b\x18\xee\x1f\xc8\xa0\x03\xc0\x94\xbe\x15\xd9\x26\x9e\xa7\xa5\xee\xb4\x83\xa4\xfb\xc9\x76\x2b\x70\x3c\x0c\x6f\xdb\xd3\x58\x6e\x77\x53\xb7\xf8\xbf\xb6\xba\x93\xde\xb7\xc3\x80\x12\x86\x97\x06\x03\xfa\xf7\x30\x00\x56\x8a\xab\x03\x35\xa2\x72\x24\x0f\xce\x60\x95\x76\x1f\xe8\x37\x00\xc3\x90\x3f\x61\x7f\x7e\x00\xf4\x4f\x34\x51\xb7\xb1\x45\x60\x75\x90\xc2\x64\xb5\x08\x3d\x52\x9f\xbc\xdf\x97\x8f\x30\x08\x3f\x88\x85\x6d\x7c\x3b\xc4\x0c\xc0\xfb\x40\x46\x48\x48\x66\x02\x42\xff\x83\xfa\xd4\x7d\xf0\x0c\x05\x39\x0e\xb5\x1e\xc0\xca\x96\x7b\xa0\x1e\xac\x55\x47\x97\xf3\x24\x3b\x16\x73\xdb\x78\xf9\xb5\x5c\x57\xba\xcc\xc4\xb1\x77\xa7\x43\xb6\x30\x53\xcf\x7a\xbd\x27\xd6\x64\x87\x81\x83\xfd\x13\x7f\x1e\xdc\x3f\xf1\x87\xfb\xe7\xfe\x9a\xac\x92\x25\xcc\x97\x7f\xca\xdc\x00\x8b\xad\x68\x78\xe2\x26\x41\xdc\x22\x5d\x2f\xc5\xb2\x1a\xc8\x29\x62\x96\x5d\xad\x17\xc5\xd8\x0e\xf4\xaa\x41\xd8\x07\x90\xe1\x21\x90\x61\x68\xcf\x3a\xab\x1b\xd0\x9c\xa8\x28\x99\x4e\xd1\x7e\xd0\xf1\x33\xfe\x1c\xf7\x43\x5c\x17\x8a\xd5\x79\x03\x90\x47\x78\x86\xe4\x6f\x09\xaa\x66\x2d\x1c\xc4\x7b\x4f\x68\x14\xa7\x3f\x3c\xe5\xad\x6e\x84\x82\x8f\x56\xe7\x7b\xa6\xa4\x4e\x04\xf5\x94\x53\x43\x77\x75\xd3\x3e\x27\x21\xd0\x50\xce\x30\xdb\x06\xb7\x1a\x1a\xef\x6e\x83\xb5\xb4\xed\x4f\xf5\xe5\x2c\x53\x0b\x66\x44\xc5\xab\xe2\xe2\x3c\xde\x24\xd4\x9b\xa4\x24\x49\xd5\x08\x8a\x28\x22\x2a\xb1\x59\x1a\xad\x75\x60\x82\x26\x90\x1e\xa9\x46\xb7\x34\x21\x93\x92\xcc\x73\xf3\x77\xf1\x25\x22\xb5\x3a\x51\xc0\x25\x99\x7c\x18\xd2\x19\x31\xb0\x3f\x27\x52\xa7\x33\x62\x40\xe5\xc3\xef\x03\x05\x52\x7d\x4c\x89\x14\xd4\x45\x52\x68\x6f\xe6\x16\x06\xaa\x41\x93\x8e\x4d\xce\x96\x91\xe9\x65\x9c\x3e\xba\x6f\x34\xbd\x6f\x46\xb6\x97\x1c\xf6\x52\xc9\x1e\xea\xa0\x60\x67\x7c\xb3\x07\x63\x9d\xd6\x3e\x02\x3b\xc9\xfb\x86\x5d\x07\x48\x8d\xb5\x3b\xa3\x25\x6a\x39\xad\x43\xe5\x1f\xa9\x00\xd8\x62\x5c\x3c\xb8\x1d\x0a\x82\xff\xf0\x96\x90\xd5\xe2\xfe\x27\x17\xdb\x72\x8c\x66\x01\xf2\xb8\x34\x4f\x09\x0e\x6d\x72\x03\x07\x48\x96\x6d\x76\x32\x05\x3f\x61\xf4\x96\xfa\x5c\x4b\x22\xb5\x86\x3b\xe2\x9b\xd4\xfd\x03\xbd\x85\xc1\x24\xfe\xa1\x8d\x70\xfd\x92\x6a\xad\x95\xb0\x3f\xb2\x33\x34\x9a\x80\xff\x86\x0f\x69\x83\x66\x8f\x59\xed\xd1\xe5\x74\xda\xfe\x1e\xb3\x72\xb5\x35\x47\x9f\xd0\xe9\xb4\x31\xb5\xdb\x4f\x74\xba\xda\x5f\x6b\x57\xec\xfe\xf8\x4e\xf3\xd7\x3c\x5e\xa4\xb4\xd1\xfa\xc0\x06\xcb\x61\x2d\xf6\x41\xea\x4b\x63\xc8\xba\x69\x4a\xe9\x52\x52\x20\x25\xf4\x7c\x67\xd0\x51\x00\x7f\x93\x63\xce\x6e\x67\x50\x10\x2c\xea\x87\x3b\x85\x0b\xfb\xc7\x77\x8b\xf3\x3f\xb1\x5b\xb4\x8b\xbe\x1c\x7f\x35\x8d\x43\x68\x85\x85\x09\xfd\x09\x7d\x42\x12\x42\xa1\x36\x15\xbb\x65\xd2\xf8\x83\x3a\x00\x72\xff\x7a\x19\xdd\xa2\x7d\x01\x34\x50\xe8\x0d\x7a\x8f\x7b\x43\x4f\x2c\x1e\x87\x96\xa1\x45\xa8\x3e\x80\xa8\x0c\x04\x05\xf4\xcf\x88\x77\x21\x7a\x74\xcc\x8b\x6d\xd4\x53\x29\x23\x6b\x93\xc2\x9c\xdc\x67\xb3\xb7\xfa\x9c\x03\xbe\xa3\x28\x52\xdb\x18\x90\xb0\x15\x94\xe7\x1b\x09\xdc\x9a\xcb\x9b\x64\x94\xe1\x48\x41\x4d\xbd\x28\x52\x65\xf6\xf0\x30\xcd\x46\x25\x89\x3c\x88\x88\x61\x0e\xa2\xa1\xc3\xf8\x88\x7e\x55\xe8\xee\xee\x20\x52\x82\x39\x88\xf3\xbb\xab\xcb\x59\x92\x1f\x26\x8d\x61\x0e\xa2\xf9\x32\xcb\x56\x49\x7c\x0f\xaf\x24\xd0\x41\x44\xaf\xf2\x24\x69\x6f\x27\xcb\x1e\x01\xa3\x8f\x8f\xe1\xd7\xb2\xd0\xa0\xfc\xeb\x46\x6d\x0a\x95\xbb\x53\x50\xfa\xac\x49\xa6\x06\x78\xf4\x88\x21\xd2\x05\x74\x58\x10\xcc\xc8\x3e\x8a\xd5\x76\x07\x66\xf3\xb8\x41\xa3\x3a\xd9\xac\xc4\x1a\x75\xd2\x34\x1a\xa0\xed\x4a\x14\xe7\xcb\xab\x4b\x40\xa8\x8e\x2b\x47\x68\x05\xc3\x06\x13\xb7\x95\x49\x9d\xa4\x53\xb6\xf1\x80\xbc\x68\x06\x51\x05\x80\x74\x82\xa1\x69\x54\xf1\xdf\xad\x91\x56\xa2\x42\x9b\x46\xb4\x33\xeb\xff\xfc\xf4\xfd\x77\x3d\x3a\xb9\xf6\xe9\xb3\x20\x6e\xa7\x67\xcc\x3b\x66\x36\xf5\x65\x93\xfd\x4c\x67\x3e\x93\x16\x45\xde\xeb\xd7\x3d\x2f\x80\xe5\xcb\xc6\x98\xf7\xd8\xe6\x31\x12\x0a\x60\xc8\x2a\x68\x71\x35\x4f\x0c\x60\x2c\x66\x1a\x34\x96\xd5\x07\x95\x3e\x9e\xc4\xe6\xe0\xb2\x13\x41\xe6\xce\x0c\xd6\x14\xe9\x59\x89\x27\xbc\x31\x9d\x9b\xf6\xe2\xcd\x66\x55\x13\x06\x11\x6f\xd9\x94\x06\xe8\x8d\xe7\x68\xde\x93\xe5\x0e\xc9\x74\xda\x33\xb2\x45\xe4\x1b\x93\x7a\x77\x87\x47\xe0\x51\x74\x36\x3e\x0b\xad\x8e\x08\x29\xb8\x37\x1c\x29\x1e\x40\x38\xd0\xa7\xd4\xe3\x96\x86\x2f\x76\x6a\x57\x09\xc3\xae\x6a\x72\x36\xdd\x8a\x2a\xd8\x86\x6d\x32\x83\x89\x44\xfe\x3c\xbb\x84\x06\x89\x4b\xbb\x02\x50\x9b\xd2\x3a\xb1\x9a\xe8\xa3\xdc\xeb\x2c\x5d\x90\xed\x09\x42\x20\xb1\xf0\x07\x92\x61\x6d\x84\x1f\xc8\x0c\xea\x08\x94\xdf\x6d\x20\xc5\x93\x68\x20\x71\x79\x5d\xdc\xdb\x3e\x9b\xf4\xa7\x63\xa8\x73\x77\x20\xce\xa2\x33\x69\xa8\x31\x08\x82\xd0\xeb\xa8\x64\xe2\x49\x23\x99\x4e\x29\xf8\x30\xbb\x08\xec\x46\x80\x72\x70\x81\xe7\xb4\xb6\x32\x20\x3a\x13\x95\xa8\xd9\xa8\x87\x8c\x85\x90\xca\xa6\xbd\x10\x5a\x0b\x9c\x51\x0a\x1a\x2b\x55\xd1\x99\x1f\x07\x02\x7a\x9d\x0f\x6d\x5e\x3f\xad\x02\xa6\x9d\xb6\xee\x97\x6b\xec\x1f\x00\x5f\x3d\xad\x03\xd5\x22\x1c\xab\x7a\x45\x5f\xb2\xf8\x72\x13\xb5\x0a\xe0\xec\x69\x8c\x47\x29\xf1\xd3\xd9\x78\x80\x6a\x0e\x0f\x15\x7b\xc1\xe3\xee\x8c\x25\x8e\x06\xdc\x96\x56\x45\xb1\x7a\x14\x55\x63\x6b\x4c\xaa\x82\x71\x15\x4e\xaa\x69\x38\x99\x52\x5e\xb6\xe4\x6a\xcb\x6c\x6f\x9b\x57\x20\x89\x8f\x1c\x2c\xc7\xc7\x95\xcc\x3b\x56\x1f\x21\x0f\x3c\xd0\x87\xf7\xa2\xd3\x34\x78\x13\xaf\x53\x91\x44\x70\x8e\xa0\xe3\x4d\xbd\xd0\x2d\x72\xdc\x18\x12\xaa\xc0\xe9\x1d\x88\xe5\x23\xaf\xd3\x34\x62\xec\x40\x24\x50\x42\xb6\x35\x8e\xf9\x62\x74\xea\xff\xfb\xdd\xe4\xdf\x5f\xbf\x9e\x06\x1f\x9d\xd2\x84\x0b\xaa\x90\x5b\x73\xcc\x43\x91\x13\x1e\xe8\x64\x98\xc7\x3a\x4c\x60\x11\x83\x2f\x6d\xb1\x86\xb1\x54\xed\xab\x75\xfa\xee\xca\x1a\xe6\xc8\x20\xe6\x0c\xad\x5f\x60\x69\x57\x90\x20\xd9\x1c\x24\x7b\x97\xbb\x3b\xe8\x3b\xf8\xa5\xcc\x8e\x18\x29\x96\x1e\x9d\xdd\xdd\x59\xe3\xbc\xc2\x13\xc9\xbf\x77\x77\x16\x89\xd7\xca\xda\xcd\x60\x32\x26\x77\x20\xab\x18\x8f\x86\x69\x0a\x89\x31\x98\x0a\xbe\xe8\x1f\x1f\xab\x58\x65\x4c\x22\x6b\x26\xa3\xa9\x72\x97\xe9\xfa\x05\xe6\x69\x56\xef\x83\xaa\xd5\x1d\xec\x54\x4b\x57\x01\x6d\xb9\xfa\x02\x8a\x51\xe4\xf6\x81\xdc\xeb\x08\x22\x68\x92\x7a\x40\xdd\x00\xf4\xe9\x35\x8c\x0d\x88\x03\x18\x02\xf8\x52\x5d\x15\x08\x71\x35\xe2\xea\x4f\xa9\x46\x5c\x35\xaa\x11\x57\x0f\xad\xc6\xf5\x53\x00\xc6\x6a\x00\x8e\x7d\xd5\x50\xf6\xb3\xd6\x00\x2d\xa4\x9d\xd1\x26\x2b\x04\x5a\xd5\x8a\x64\xb5\x4a\x37\x45\x2a\xf7\x56\x71\x92\xd7\x93\x3b\xd4\x55\xce\x22\x11\x1a\xae\xaa\xa1\x6a\xa4\xb2\x44\xea\xe3\xee\xce\xeb\xf5\x7a\x1e\x75\xa5\x95\xb1\x6a\xe9\xcb\xc2\xba\x0a\x4e\xf1\x47\x6d\x5d\xa2\x19\x98\xbd\x25\x29\x4b\x50\xe0\x1d\x1f\x29\x1c\xb7\xda\x0e\x8b\x41\x10\x16\x72\x5c\x07\xac\xdd\x55\x10\xf0\xc6\xa5\xdc\x40\x0f\x29\x20\xf7\xd9\x43\x22\x6c\xc0\x94\xcd\x93\x74\xe5\xaf\xf0\xc4\x7a\x35\xe4\x98\xb3\x55\x06\xcd\x82\x51\x92\x81\x07\xca\xb5\x8a\xed\x63\x28\xe8\x3c\x84\xda\xe1\x0e\xb9\x43\xa0\x57\x1d\xc3\xdd\x5b\xa8\x53\xa6\x55\xe4\x96\x47\xaf\x16\x4b\x69\x18\xc3\x26\xaf\xaf\xfa\xfd\xfe\xe7\xf4\xfb\x8c\x7e\xbf\xa4\xdf\xaf\xe8\xf7\x6b\xfc\x1d\xf6\xf1\xf7\x19\xfc\x0e\x3e\xf9\x0c\x7f\x3f\xeb\x3f\x7f\x7d\x35\x84\x64\xfa\x1d\xd0\xef\x90\x7e\x1f\xd3\xef\xc7\xf4\xfb\x84\x7e\x3f\xa1\xdf\x4f\xe9\xf7\x33\xfa\xfd\x9c\x7e\x9f\xe1\xef\xf0\x1b\xfc\x7d\x42\xbf\x43\x4a\x1d\x42\xea\x63\xc2\xfc\xcd\xf3\x6f\xbe\x99\x06\xa7\x23\x36\x77\x34\xb2\x29\xad\xa0\xa5\x4d\xb4\x37\xf9\x39\x59\xc6\x47\x2f\xb3\xe5\xf4\xc8\xeb\x90\x81\x33\x69\xab\x68\xec\xd7\xc8\x03\x3f\x91\x04\x7f\x9e\xe7\x12\x5c\x38\xd6\xd5\xc2\xbb\x82\xfe\x7c\x06\xcb\xc0\x85\xf7\x28\xc2\x15\x55\x76\x76\x04\xcb\x57\x54\x46\xe8\x8f\xaf\x4b\xe0\x75\x72\x74\x6b\x05\xa4\x45\x76\xd4\x86\x24\xa9\x36\x30\xe9\xe3\xc9\xe9\x79\x8a\x16\x17\x14\x7a\x14\x45\xf2\xcb\x42\x82\x0b\xd1\xbf\xff\xf8\x32\xf2\x3c\x2b\xb2\xb8\x5e\xe2\xae\x6f\x01\xcb\xc7\x24\xfa\x48\x2e\x14\x23\x58\x28\x0e\xbc\xa3\xea\x72\xb5\x2e\x22\xb2\xf5\x0c\x4f\x4f\x6f\x6e\x6e\x7a\x37\x8f\x7b\x59\xbe\x3c\xc5\x16\x3a\x85\x9c\x12\x24\xac\x60\x7d\x7b\xd1\x06\x38\xf8\xfc\xf3\xcf\x4f\x29\xd5\xfb\xc8\x2a\x34\xbe\x82\x25\x67\xbc\x78\xb1\xc6\x73\x93\x27\x36\x35\xb8\xa4\x8f\x6e\xcb\xf4\x32\x81\x15\x6a\x0f\xff\x72\x9c\xb8\x2a\xe7\x6e\x4c\x0f\x62\x6c\x16\xe5\xb0\x88\x80\xa5\xca\x2d\x2e\xb5\x79\x1b\x04\x96\x0c\xf0\xfd\x55\xbc\x09\xbd\xd9\x55\x59\x7a\xbc\xba\xf7\x8a\x78\x5d\x74\x8b\x24\x4f\xcf\x3c\xb3\xa8\x1f\x0c\x6c\x5c\x78\x50\x14\xdd\xf2\x99\x45\xa8\x4e\xbd\xd9\x4c\x2a\x1c\xf4\x85\x3c\xa4\x09\x1f\x0b\x7d\xfc\x14\x7a\x7f\x01\x9e\x78\x62\x99\xa7\x0b\x15\xb1\xf8\x0c\xff\xe3\x7c\x0e\x90\x7b\xaa\xaa\x62\xf5\xe9\x14\x10\xae\x4f\x92\xe5\x37\xd1\xf8\x89\xd8\x39\x35\x05\xba\xdd\x48\xb7\x7a\x66\x83\xdc\x14\x2d\xb7\xb4\x5b\x00\x5d\xa4\xce\xd6\x37\x32\x61\xb5\x90\x70\xdf\xf3\xfe\xe3\xe3\x27\x36\xcb\x78\x17\xc6\x30\x8d\xcf\x6f\xd4\x5e\xa5\xc3\xb4\x96\xfd\x2b\x20\xef\xb3\xcf\x3e\xf3\x44\xcb\x76\x15\x70\xc0\xdd\x12\x0b\x07\x9f\x08\x67\x17\x0d\xb0\xf7\xc5\x6a\x87\x9f\x2b\x97\x51\x7d\x13\xe1\xd6\xdd\x6c\x5c\x86\x3c\x0d\x08\x67\xab\x52\x1f\x89\x72\xb4\xac\xfc\x67\xc2\xda\xcd\x0e\xbd\x79\x9a\xcf\x11\xc4\xec\x62\x87\x4f\xfa\xc2\xda\xb5\x56\x15\xdc\xd9\xa4\x0e\x07\xbf\x7b\x2b\xd9\xcd\x32\x47\xb4\xd1\x6d\xbe\x9c\x85\x93\xc1\xf0\x33\x21\xff\x4d\xb1\x32\xe1\x04\x68\x84\xff\xa6\xe2\x7c\xbe\x0a\x27\xf8\xfd\x04\x03\x85\x0c\xf4\x9e\x4c\x9d\x0e\x86\xc6\x37\xd1\x2d\x8e\xf2\xcb\x2c\xaf\x07\xfd\x70\xe2\xfd\x65\x70\xf6\xe9\xa7\xb3\x8f\x3d\xe1\xfd\xe5\xec\xec\xd3\xb3\x7e\x82\x5f\xc3\x79\xdc\x1f\xce\xf1\x6b\xf1\xc9\xf0\xd3\xe1\x67\xf8\xf5\xf9\xc7\x9f\x7c\x3a\x5b\xe0\xd7\x67\xf3\x27\x9f\x7c\x3c\xc3\xaf\xe4\xf1\xa7\x9f\xce\x87\xf8\xf5\xe9\x19\xfe\x87\x5f\xb3\xf9\x6c\x31\xa4\xb8\xc1\xa7\xb3\x64\x7e\xe6\x4d\x85\x2a\x70\xd8\x28\x30\x4e\xe6\x9f\x26\x9f\xb9\x45\x9f\x9d\xcd\x66\x9f\x7e\xe6\x12\xf1\xf9\x67\x8b\xb3\xcf\x62\x97\x9c\xb3\xb3\xcf\x3f\xfb\xfc\x13\x97\xb0\xf9\x93\x59\x7f\xf1\xc4\x25\x71\xfe\xf1\xe7\xf3\xcf\x3f\x76\x89\x3d\xfb\x74\xf6\xc9\xa2\x41\xf6\xfc\x53\xfc\xcf\xad\xc0\x62\xb6\x98\x7d\xb6\xb0\xaa\x82\xa5\x25\x8b\x38\x79\x02\x95\xa2\xc3\x91\x02\x2a\xa4\x24\xc7\x9b\xe7\x59\x51\xc0\xdf\x45\x1a\x5f\x66\xb8\x9f\xe9\x15\xef\xae\xe2\x1c\x93\x60\x19\x41\x06\x5f\xdd\x45\x76\xb3\xb6\xc3\x57\x1b\x8f\x9b\xe8\x4b\x3a\xab\xb5\x6f\xd2\xe0\xbc\x3c\x6b\xc4\xc2\x82\x8c\x26\x09\x28\x30\xce\xfd\x40\xcc\xe4\xa4\x01\x2b\x02\x4a\xdd\x0a\xbd\xdf\x17\xc9\xd3\x5f\x1d\xa1\x54\x3b\xb3\x23\x48\x58\x5a\x2e\xef\x10\xca\x6a\x10\x75\x78\x9b\xab\xf7\xed\xb3\x7f\xbe\xf9\xf9\xd9\xcb\xbf\x3f\x17\x94\x52\xef\x4d\xa9\x86\x51\x77\x4f\x9e\x7d\x29\x16\xc5\x3d\x9c\x4b\xcc\x1a\x69\x20\xea\x81\xa8\x86\xa2\x1e\xee\x10\x06\x69\x8a\x92\x7a\xa0\x8b\x06\x58\x55\x56\x3d\xdc\x41\x0e\x83\x97\x85\x5c\xd4\x66\x3f\xe2\x0b\x89\x16\x54\x61\x5d\x00\x2c\xb9\xbf\x90\x45\xa8\x68\x2c\x2c\x10\xd5\x53\x59\x9c\x86\x1e\x12\xf4\x53\x59\xb4\x86\x1e\x22\x74\x93\x08\x98\xd2\x63\x7b\xa7\x6c\xd1\xac\x5a\x37\x5a\xa8\xaa\xe9\xcf\x6a\xd8\xd1\xb1\xfa\xd3\xc6\x9a\x63\x4b\x1f\x68\x47\x4b\x45\x95\x51\x81\xe6\x5f\x33\xad\x56\x69\xd5\xd0\x52\x76\x65\x4c\xa0\x19\xdc\x48\xaa\x87\xbb\x75\xa5\xe3\xfb\x95\xb3\x76\x58\x54\x62\x51\x37\xc9\x83\x1a\x55\x56\x45\x2b\x45\x1a\x7c\xd7\x56\xad\xeb\x96\x6a\x97\x0e\x76\xea\x51\x82\x9a\x96\x0e\x2d\xb2\x42\x92\x99\x15\x9c\x86\x7b\x37\x6b\x8e\x83\x0f\x15\x37\xaf\xa2\xaa\x5b\x9d\x00\x54\xa7\x3e\x81\x78\x31\xaf\xa3\x1a\x22\xe0\xb3\x5b\x63\xb4\x00\x16\x4a\x5a\x41\x1e\xe5\x27\xc8\xa5\x8c\x05\x09\xac\x65\x6c\x3d\x1c\xd9\x75\x93\xbd\x14\x45\xcf\x07\x3c\x27\xd0\xbc\x80\xf4\x04\xaa\x36\xaf\x90\x14\x88\xe8\x60\x3c\x46\xd4\xbb\x60\xc3\x26\xd8\xd0\x05\x1b\x36\xb1\x0d\x5b\xb1\x0d\x9b\xd8\x86\x16\x36\x9b\x9f\x34\x88\x38\x63\x8d\xda\x2b\x82\x9a\xb7\x74\x92\x19\x09\xd2\x0c\x18\xd0\xd2\x55\x66\x24\x49\x00\x32\x6c\xe9\x30\x33\x92\x25\x00\x19\xb6\x74\x9b\x59\xab\x34\x25\xeb\xf9\x2a\x2b\x92\xa2\x95\x3e\x39\x0a\x02\x95\x44\x94\x0a\x0e\x9f\x52\x49\x32\x58\x53\x6a\xad\x52\x6b\x4a\xad\x87\x76\x21\x74\x7e\x55\x24\xf3\x72\x5f\x31\x8f\x54\x15\xbe\xc0\x72\xee\xee\x64\xb1\x4f\xb1\x18\x19\xaa\x31\xad\x56\x69\x35\xa6\x41\x7d\xec\x52\xf0\xa2\x54\x9c\xae\x8b\xd6\xe1\xe8\x91\xaf\xc7\xa3\xbb\x3b\x3d\xd8\xdc\xdd\xe9\xe1\x08\x3e\x15\xd3\x1c\xac\x6c\xb4\xb3\x6f\x10\x18\x76\x25\x52\x3b\x8b\x34\xee\xd9\x93\xa7\x96\x79\x6a\xc8\xc3\x73\xc9\xd6\xa7\x0d\x4f\x75\x04\x6b\x67\xd4\x27\x88\x4a\xaf\xf3\xb1\x04\x39\x5b\xa5\x8b\xc8\xc3\xf8\x37\xb4\x85\xa6\x20\xde\xa4\x8b\x4e\x87\x3b\x38\xcd\x57\xf8\x83\x26\xcb\xa0\xb1\xc5\xb9\xc7\x09\x78\x4d\x87\xae\x4e\x29\x99\xeb\xeb\xc1\xc9\x4c\x01\x7d\x3d\x2a\xf5\xb7\xd2\xe0\x47\x4e\x81\xaa\xa8\x43\x93\x20\x5d\x23\x32\x27\x06\x6c\xf4\x45\x4a\x97\xcb\x0e\x22\x85\x37\xae\x6e\xa5\x4e\x6c\x03\x87\xf4\xbb\x55\x82\xab\x4a\x06\x96\xa9\x05\xb6\x55\xef\xa8\x8f\x4b\xd7\x79\xbc\xbe\x8e\x0b\xb5\x4c\xa4\x00\x1b\x3d\xb7\xf1\x95\x4f\x2e\x30\x59\x8e\x6e\x57\x39\xac\x97\xca\x1f\x20\x42\xcc\xcf\xaf\xd6\x17\x85\x20\x90\x85\xdc\x79\xc3\x7d\x45\x42\xb6\xa1\xe1\x8e\x96\xfe\x81\xbe\x6c\x9d\xa3\x49\x36\x68\x33\x7f\xf9\xcb\xff\x1e\x78\x81\x3c\xb6\xc8\xd1\x30\x3b\xd0\x3b\xdd\x7a\xa7\xe9\xad\x2c\xe0\x07\xc6\x8f\x1b\x80\xb8\xc9\x43\x88\x9d\x43\x9c\xb4\xd3\x09\x6e\x2d\xc2\x08\x04\x37\xb3\x39\x7f\x64\x25\xe9\x52\xe4\x35\x67\x9b\xb2\xe1\x14\xaf\x7f\x03\x6d\x5d\x9b\xb4\xc7\xd3\xc0\xa1\x23\x9a\xd8\xe8\xe6\xe7\x71\xfe\xac\xf4\xfb\x81\xd9\xaa\x7c\x0b\x74\xbe\x45\x42\x39\x97\x22\x15\xa3\x9e\xbe\x1d\xbd\x05\x5a\x99\x61\x11\xfd\xf9\x66\x95\xc5\xa5\xcf\xa0\x93\xb7\x50\x16\x2c\xd8\xe3\xef\xf8\xaa\xe3\x22\xb8\xbb\xb3\x8b\x66\x29\x90\x49\x23\x9e\x6b\x2e\x2f\x71\x62\xb7\xa1\xf0\xd6\x76\x99\xbd\xcc\x6e\x92\xfc\xab\x18\x2f\x87\x0b\x09\xf5\x92\x08\x89\xe6\x97\xf2\x6b\x22\xe3\x69\x47\xdf\x29\x47\x1d\x0b\x3d\x75\x72\x06\x7a\xbf\x18\x3a\xc1\x85\xa9\xa1\x93\x69\x84\x09\x4f\x2f\x46\x17\x9d\xc8\xcd\xcc\xe2\xc1\x55\x98\x34\xe8\x9d\xe2\xd8\x34\xd7\x7c\x90\x08\xb9\xa9\x2e\xc4\x45\xc7\xc5\xa4\xee\x30\xdb\x18\xed\x8c\xfa\xaa\x1f\x03\x18\x2b\x83\x45\x1e\xdf\x3c\xcb\xe7\xfe\x12\x27\x6b\xe0\x4a\x96\x2f\x0a\x39\xc4\x88\x95\x28\xcd\xd9\x61\x5e\x45\x9c\x8a\xd2\x9a\xd7\x2a\x30\x80\x40\x56\xaa\xd0\x10\x97\x42\x39\x2c\x6c\x64\xf8\x31\xe8\xe6\x37\x49\xb2\x51\xe1\x8f\xa7\x22\xd1\x78\x9e\x40\x40\xe3\xf9\x04\x20\x93\x65\x11\xc5\xf9\xfc\x55\xf6\x53\xb2\xa4\x93\x47\x3f\xa9\x00\x44\xe4\x15\x14\xc8\x88\x19\x1d\x16\x49\xca\x05\x5d\x33\x4f\xbf\xc0\x9c\x7a\x9b\x13\x05\x9f\xf4\xf5\xe4\x7d\x54\x30\xa2\x57\xd9\x97\xc9\xfb\x14\xf4\x5d\x3e\xba\xa3\x9b\xa7\x98\x07\xb7\x5e\x47\xcb\xde\x8c\x12\xbf\xba\xca\xaf\x93\x57\x99\x84\x59\x0a\x88\x0d\x24\x27\x68\x02\x87\x30\x54\xbd\xbb\xc2\x04\xba\xcf\xb9\x93\x3a\x54\xa9\x8f\xdb\x52\x3f\x56\xa9\x4f\x30\xd5\x32\xf4\x20\x38\x6c\x83\x9d\x16\xf8\x6f\xc0\xfd\x3f\x83\xc1\x6e\xad\x90\xcb\xad\x75\x02\x05\xf6\x9f\x22\xab\x44\x56\xd3\xc1\x00\x24\xe2\x99\x53\x84\x4e\x20\xd8\x10\x40\x9f\x88\x07\xc2\x41\xf9\x55\x3c\x3f\x4f\x26\x12\x7e\xaa\x76\xd6\x0f\x80\xd0\x38\x05\xc3\x8e\x2c\xf4\xc4\x27\x3d\xf8\x87\x17\xa7\x83\xcf\xfa\xa4\x18\xbf\x81\x44\xad\x1b\x43\xf7\x06\xb1\x28\x74\x1c\xea\xd0\xb8\xed\x0e\x32\x41\xe1\x78\x56\xf8\x39\xac\x7b\x40\x2c\x4c\xb8\xe6\xc1\x70\x83\xed\x8d\x59\x4f\xfc\x0c\x34\xeb\xe0\xa4\xf7\xa4\xc3\xf8\x21\xa2\xee\xd6\x18\x21\x36\xb5\x01\x92\x71\x5d\x0d\x24\x73\x89\xcd\x2a\xda\x54\x27\x9b\xea\x14\xca\x3a\x81\xe2\x3a\x9b\xfa\x64\x53\x43\xa8\x3e\xc1\xc2\x36\xab\xa7\xa8\x72\x6e\xe4\x81\x41\xf1\x2e\x2f\x21\x00\x44\x55\x27\xd1\x66\x05\xb4\xe1\x1f\xa6\x29\xee\xf7\x65\x79\xa7\xd0\x0e\x71\x7f\x10\x71\x61\x14\x1a\xf4\xa3\xae\x0a\xd6\x10\x1c\x68\xd0\x5a\x54\xfd\x08\xf2\x9e\x64\x55\x07\x32\x9d\x64\xb5\xa8\x21\x62\xc0\x11\x03\x8a\x00\xad\x04\x21\x18\xa0\xc6\x65\x01\xa6\x73\x72\x8d\xb7\xb1\x41\x9f\xaf\xfa\xc1\x89\xfc\xdb\xf1\x61\x69\x57\x63\x98\xff\x8a\xe2\x2c\x9e\x97\x59\xfe\xa6\x78\x17\x0d\x4e\x17\xdd\xde\xf0\xc9\xa8\xff\xd4\x44\xe2\x19\xb4\x81\xe8\x73\x75\x64\x8c\x55\x6f\x03\x83\x67\x23\xd8\x01\x23\x12\x35\x93\x1d\xea\xc8\x1f\x8c\xa2\x9a\x47\xbd\x27\x40\x54\xbf\x03\x5a\xbc\x4a\xd2\x44\xd5\x9c\x5a\xf7\x3b\xa0\xc6\x77\x74\x2a\x57\x01\x74\x9c\xbe\x6c\xf6\x32\x5e\x0f\x01\xaa\x5b\xcf\x81\x53\xdd\x6a\x8e\x69\x03\x27\x6d\x40\x69\x03\x99\xf6\x06\x24\x14\xd6\x4a\x03\x50\x2d\xfb\x50\x4d\x8e\x38\x3e\x1e\x44\x11\x11\x3d\xe6\x88\x4e\x34\x3c\x91\xe2\x19\x72\xcc\xd3\xfe\xf1\x71\x5f\x02\xd1\x5a\x01\x23\xbb\x06\xcc\x68\x2b\x72\x00\x28\xac\x25\xaa\x96\x50\xce\x76\xea\x43\xd5\x64\xbe\x4e\xaf\xdf\x1f\x04\x41\x60\xb4\x26\x1a\x60\x14\x12\xa5\xd4\x70\xdf\xc1\x55\x5e\xbf\x93\x9e\x48\x34\x0a\x08\xaa\xf5\x98\x52\xfc\xb4\x33\x08\x9a\xa9\x23\xc6\x8c\x4e\x3c\x26\xd5\x1c\x18\x0b\xe0\xb8\x3b\xf1\x58\x0e\x08\x2c\x78\xb2\xb3\xe9\xe3\xfb\x03\x1d\x39\x6a\x4e\xa7\x8d\x21\xcf\x87\x75\xde\x1c\x57\xcb\xa8\x17\x0f\xda\x4a\xb9\x7f\xa8\x69\xa0\x6c\x1f\x6c\x0e\x02\x35\x7a\xdd\x89\xec\x75\xaa\x8f\x53\x3f\xeb\x47\x2a\x54\x59\xbd\x0e\xd3\xf8\xab\x6f\x8f\x3d\x7a\x90\xea\xdb\xa3\x54\x5f\x0d\x53\x03\x1b\x76\xa0\x60\x07\x36\x2c\xed\x6c\xbc\x39\x8f\x57\x67\x24\xdb\x52\x0a\x15\xe8\x9b\xf3\xa1\x01\x86\x74\x09\x0a\x79\xa2\xcf\x4e\x1f\x9f\x68\x20\xf3\x75\x6a\xe1\x96\xb0\x30\x16\xcc\xab\x8e\xa4\xbe\x5b\x4a\xd8\x3e\x0e\x0a\xf3\x5a\x0e\x82\xfd\x4e\x79\x22\x21\x44\xf5\xd8\xc0\x0f\x44\xfd\xd8\x40\xd1\x56\x43\xf5\xb8\xa3\x70\xd0\x26\x43\xfd\xb8\xab\xf2\x0e\x46\x0f\x69\x85\x68\x42\x43\xd3\x80\xc7\xa6\x81\xa0\x91\x69\xc0\x43\x13\x84\x30\x6d\xc8\x69\x43\x4e\x1b\x72\xda\x90\xd3\x1e\x73\xda\x63\x4e\x7b\xcc\x69\x8f\x2d\x8f\x09\x7c\x64\x03\x8a\x0b\x6a\xfa\xac\xbb\x59\x8b\x13\x51\x26\x97\x9b\x7f\xd2\xef\x2f\xf4\xfb\x15\xac\x7c\xf3\x6c\xf5\x4f\x3b\xf0\x0b\x2c\x48\x93\xeb\x34\xbb\x2a\x22\x9a\xa5\xf1\x54\xbb\x86\x7f\x73\x09\x6b\x3e\x7f\x81\x4f\xb9\x21\x8a\xce\x48\xf4\xa6\xe9\x48\x99\x06\xe1\xcd\x8a\x15\x0c\x91\x42\x45\xa0\x9d\x50\x89\x11\xa8\x6f\x2d\xd3\x35\x2e\x15\x7c\x33\x54\xec\x5f\xc9\x90\x3d\x8e\x3c\x62\x96\x95\x31\x8b\x19\x0e\xe3\x79\xbb\x3c\x79\xf6\xc2\xaa\xa3\x16\x37\xa8\x1e\xd5\x26\x04\xea\xd1\xb2\x87\x4b\xda\x57\x99\x5f\x75\x56\x90\xe4\xea\x10\xa8\xd2\xd8\x97\x66\x5e\x02\x2e\x07\xd5\xaf\xc6\x74\xde\xa4\xea\x83\x72\xff\xad\x41\xc7\x07\x65\xbe\xf6\xc2\xfa\xd7\x17\xfd\x33\xe4\xfe\xd5\x99\x2f\xef\x6b\x8b\xcb\xec\xfa\xa1\xb8\xbe\x3d\xdc\x16\x1f\x80\x69\xee\x85\xd4\x15\x22\xe8\xeb\x12\x05\xa8\xc2\xd4\x2f\xa2\x5a\x47\x81\x42\xac\x85\xde\x00\x3e\x9e\x1a\xf9\x37\xb0\x1f\x4f\x45\x63\x0d\xe1\x9b\x2c\x83\x29\xd1\x64\xa8\xed\x94\x1a\x33\xa4\x28\x74\x1d\xd9\x41\x21\x8a\x48\x69\x56\xa2\xe3\xb0\xd1\x60\x73\xa0\x14\x5e\x8d\xd5\x49\xb5\x06\x00\x18\x1c\x23\x0e\xd6\xf4\xf7\x17\x9b\x41\x5f\xd9\xac\x7e\x62\xb3\xda\x66\x4a\x1b\x4b\x0e\x30\xc4\x61\xc7\x03\x98\xd1\xd6\x92\x16\x0b\x7e\x15\x03\x9a\x92\x50\xec\x4a\xc2\xe3\x5d\x49\xf8\xd8\xaa\xf4\xf0\xa4\xea\xee\x94\x01\xb1\xb5\x8a\xfd\x65\xb7\xe2\x7b\xaa\x77\x48\x3e\x0e\x49\xc2\x3d\x55\x7c\x90\x9c\x38\x92\xd0\x22\xe6\x83\x56\x31\x1f\x4e\x0f\x49\xcd\x4f\x8a\x99\x3b\xac\xfc\x43\x19\xb9\x5f\xae\x76\x98\xd8\x20\xfe\xf7\x94\xac\x76\x86\xb6\xb1\xd3\xa0\xb7\x99\xf7\xee\x83\x25\xf1\x01\x8d\xb5\xec\xbd\xbb\x8a\x17\x79\x5c\xa6\xf3\xc3\x3c\xfc\x20\x4e\x3d\x98\x0f\x76\xfd\xfe\xf5\x21\xc2\xd1\x46\xef\xaf\x6f\xdd\x07\xb6\xc2\x6f\xaf\x5b\xb9\xdb\x76\x83\xdd\xb6\x83\x92\xd8\xd8\x3b\x52\x1a\x16\x6e\x49\x5e\xc6\xa8\xd4\x9c\x4e\xfe\xf5\xdd\xab\x72\x7a\x1a\x8c\x7d\xd3\xbe\x56\xa3\x06\xa1\x47\x8e\x43\xac\x9c\x16\x24\x76\x25\x47\x9f\x73\xba\x93\xad\xdc\x01\x9e\x77\x0d\x3c\xa0\x93\x7d\x40\x9f\x0c\x1c\xc5\x31\x9a\xb7\x69\x90\x91\xd5\x81\x7f\x2f\xf9\xfb\x10\xa9\xff\xcd\xed\xf9\xaa\x29\xab\x83\xa6\xac\x0e\x3f\x74\x20\xfb\x4f\xd3\x0f\x63\x2f\xb4\xf6\x7b\x79\x8a\x15\x93\xd6\xd1\x4f\xd8\x13\xbc\x35\x00\x19\xbd\xc0\x68\x05\x9d\xca\xea\x9e\x9f\x4e\x3b\x80\x75\x6a\xef\x21\x0b\x4b\x0d\xfc\xc4\x51\x03\x3f\x75\x86\xc1\x67\x7f\x10\x79\x0e\x71\x4d\xd2\x1c\xca\xf6\x10\xf6\x5e\x1a\xa0\xfe\x9b\x17\x2e\x7b\x74\xf6\xc9\xab\x97\xad\x5e\x2e\xc9\x7c\x6a\xcb\x40\xb6\x85\x3e\x76\xf7\xb1\xac\xc6\x8e\x6f\x41\x27\x48\x3b\x3b\xbd\x7f\xdc\x7a\xed\xcf\x59\x62\xfd\xda\x05\xd5\x87\x2c\x9f\x3e\x60\xb1\xf4\x21\x4b\xa3\x0f\x58\x08\xfd\xfa\x65\xcf\xaf\x5d\xe4\xfc\xd9\x4b\x9a\xff\xca\xab\x8f\xff\x6a\x8b\x86\xff\x8e\xfa\xfd\x7f\x37\x75\xfc\xff\x8e\x52\xfd\x3f\x6a\xf1\x7f\x36\xb5\xf8\xff\x2d\xe5\xf6\x8f\x50\x51\x9d\xf3\xf0\xdf\x43\x3b\x75\x74\x53\xc5\xc5\x0f\xd1\x4a\x7f\x67\x8a\x2c\x7a\x2c\x6a\x3e\x50\x11\xbd\x47\xf7\xb4\x4f\xbf\x93\x98\x5c\x39\x69\x87\x5b\x14\x40\x43\x02\x4c\x8a\x16\x8f\xf1\xda\x4a\x8f\xc0\x82\x5e\xd5\x7e\x99\xba\xda\x06\xbd\x7a\xd0\x9e\x56\x63\x5a\x7f\x4f\x5a\x67\x21\x4d\xdb\xb6\xfa\x3e\x5b\x66\x3b\x24\x3c\x3e\xc6\x82\x1d\x17\x85\x4e\x7a\x20\xf8\x36\x6b\xd6\x2b\x93\x35\xde\xa6\x91\x19\x64\xc8\xd7\xf1\x81\xb0\x2a\x6a\x6a\x8f\xfb\xd5\x7b\x6a\x8f\x49\xaa\xf6\x04\x76\xb8\xf6\x7b\x2b\xbf\xa7\x5e\x88\xf2\x83\xea\x45\x19\x5a\xea\x65\x55\x81\x6c\xea\xe4\x81\x92\xd0\x46\x4b\xd1\xed\x65\x38\x14\x2b\xf8\x87\x17\x20\xae\xe1\xdf\x3c\xfc\x44\x14\xe1\xc7\xe2\x1d\xfc\x2b\x21\x3e\x0e\x3f\xdd\x8a\x3c\x89\x26\xa7\xfe\xe4\xdb\x97\x7f\xfb\xf9\xab\x9f\xfe\xf5\xd5\xb3\x7f\xbb\x5c\x9d\x5f\xcf\x8b\x77\x65\xfc\x7e\x1a\x9c\x2e\xc5\xe9\x5f\xfe\xf2\x97\x53\x71\xea\xbf\x5e\x04\x5d\x0c\xbe\x2e\xee\xc4\x1d\xc6\x4d\x5b\xcc\x1d\xd0\x54\xae\xed\xb8\x0b\xe3\xf1\x18\x53\x3a\x82\x30\xb6\x7d\x18\xa9\xfc\x94\x90\xb9\x56\x48\xbf\x40\x15\x1e\x5b\x85\xfc\x47\xf6\x86\x50\xae\xd0\xb0\x45\x43\xfc\x21\x1e\x84\xe4\xeb\x46\x1a\x3e\x4a\x1b\x3d\x74\x25\x56\xb4\x1a\xe9\x01\xc9\xb4\x48\x5b\x8a\x8c\x5b\xbe\x02\x5e\x57\x77\x77\xb8\x42\xca\x7a\x35\x7e\xa4\x79\x84\x6d\xb2\x4e\xf2\x1f\xe3\x45\x7a\x55\x60\x5c\x86\x71\xd9\x55\x69\xc7\x15\x71\x04\xed\x41\x1e\xca\x9f\xa1\x9d\x32\xc4\x05\x5d\x65\xb2\x31\x14\x09\x25\x03\xfd\x2d\x89\x23\xe7\xb4\x4b\xe0\x9d\xf9\x34\x1f\x9b\x03\x0b\xd0\x2e\x43\xec\x7b\x3c\xb4\xa4\x39\x94\x05\xf8\x04\x1d\x93\xa9\xd8\x2c\xc7\x28\x48\x18\x60\xac\xbd\xfc\xb4\x2c\x12\xcb\x73\x53\x5b\xe7\x1a\x78\x46\x76\x8c\x63\x3f\x9b\x90\xf7\x40\xe6\xfa\xc2\x9b\xde\xdd\xed\x44\xb1\xf5\x1d\xe1\xe1\x5c\x74\x16\x8f\xed\xa2\x70\x37\x32\x08\xe0\x28\xba\xaa\x0b\x82\x90\x8f\xf9\xac\x83\x70\xba\x9a\x63\x68\x72\x4f\xfd\xc8\xa4\x49\x94\xfa\x40\xfc\x7d\x12\x29\x6a\xc9\xcf\x35\xff\xa1\xab\x48\xd4\x6c\xdc\x66\x68\x93\xc7\xea\x42\xc6\x8e\xde\xee\xee\xd4\x8d\x8f\x48\xc7\xa9\x13\x71\xc7\x20\x03\x90\x9d\x2a\xeb\x04\x8b\xb3\xb9\xe8\x0b\x6d\xb6\xc0\x4c\xb7\xd8\x2b\x8f\x2e\x8d\x67\x3e\xc6\xcf\xab\x60\xbe\x60\x12\xee\x94\xf2\x24\x00\x89\x28\xa3\xc7\x27\xb9\x75\x2c\xd5\x2d\x45\xdd\xcd\x03\xeb\xf8\xac\x9b\xb7\xc6\x94\x76\x4c\xa7\x3d\xc6\xc9\xd5\xd9\xc1\x8c\x31\x9d\xbc\x99\xab\x25\xa6\x6c\x96\xde\x12\xe3\x52\xc8\x98\xed\xf9\x48\x5d\xaf\x09\x95\xed\x91\x61\x84\x3f\x3c\x29\xe3\xf5\xe3\x3e\xca\x50\x15\xe5\x35\x87\x84\x2d\xfa\xdd\xbc\x76\xa9\x22\xa3\x32\x13\x81\xc5\xd5\x2e\x4d\x3b\xcb\x31\xbe\xd5\x13\x96\x8d\xe2\xa1\xd0\xa8\x84\x96\xc0\x9b\x8c\xe8\x55\x80\xf9\x0d\x12\x57\xba\x5a\xaa\x73\x17\x28\x54\x16\x55\xa6\x16\xf8\xf5\x98\x4c\xab\xf2\xea\x84\x02\x84\xd4\xd4\xa1\xd3\x52\x87\x6e\x0b\xd5\x18\xd7\x5a\xf2\xd5\xe6\x57\x95\xdb\xc6\xbb\x36\x6e\x61\xdc\x76\xcf\xa8\x81\x23\xab\xec\xa1\xed\x33\xe4\x26\xfa\xb0\x51\xc3\xb5\x9d\xee\x59\x13\x32\xbd\x3e\x62\x8d\x24\x9b\xc6\x14\xfd\x47\x92\x61\xcd\x9f\xf7\x91\xc1\x57\x09\xf7\x12\xb2\xba\xc1\x51\xa6\xd4\xd7\x0d\xc5\x6a\xae\x23\xbe\x8a\x37\x23\xe6\x3d\x25\xc9\xf1\x6c\x75\x33\x5e\xdd\x84\xcd\x7b\xb5\x06\x4c\x36\x17\x64\x56\x19\xe6\xe3\xd5\xbc\x3d\x03\x00\x89\x1d\x5e\x59\xf6\x1f\x57\x2b\x43\xbf\x9a\xf3\x06\x7a\xd2\x1b\xa8\x59\xaf\x1a\xea\xb1\xb6\x1a\x8e\xf1\x27\xac\xc8\xa2\x45\xc5\xd6\x18\x5b\x0f\xc3\x7a\xf0\x9f\xa0\xc2\xee\xfc\x69\x7a\x00\xde\x48\xb3\x85\x9d\xee\xa6\xb9\x66\xcd\x98\xe7\xfb\x35\x1b\xea\x0b\x60\x89\xd3\xa8\x9b\x78\x0e\x95\x81\x0a\x92\x07\x33\xa8\x94\xe5\xab\x54\x57\x91\xac\xf6\x35\xab\xc9\x9f\x58\xa4\x26\x20\xe9\x22\x6c\x3c\x08\xf5\xb7\xe8\x03\xfb\xe0\x93\x1f\x1d\x5a\xdd\xdd\x49\xd7\xa7\x32\x0c\x7f\x96\xbd\xe5\x2a\x9b\xc5\xab\x67\xab\xcd\x79\x4c\xb0\x27\x7a\x46\x43\x90\xef\x2d\xa4\x56\x18\x2b\x8a\xc1\x9f\xca\x7a\x85\xd6\xbd\xab\x0c\xed\x7c\x32\x22\x5d\xa5\xa1\x83\x22\xed\x9a\x14\xda\x8d\xb9\xeb\x37\x5b\x30\xb8\xa7\x79\x56\x37\x68\xde\x77\x90\x50\xc6\x66\x93\xea\xc4\x20\x41\x1c\xd1\x24\x57\xb2\x43\xd8\x52\x03\x0d\xd0\x94\x08\xdf\x15\xb2\xe0\x1e\xf9\xb8\x5e\xbe\x84\xef\xaf\xe3\x82\xa6\x68\xca\x85\x01\xf6\xae\x14\x38\x00\x7c\xdd\xd8\x01\xe3\x28\x54\xda\x34\xd9\xc0\xc9\xa0\x45\x94\x9e\x01\x8f\xa5\x28\x15\xf3\x64\x9d\xe8\x9d\x7b\x76\x2c\x84\xef\xdf\xa1\x9c\x68\xcf\x42\xe4\x7c\x03\x01\x7b\x24\x3e\xbb\x3b\xee\x18\x6d\x43\xe0\x6e\xbb\xff\x88\xb1\xde\xdd\xc9\xb5\xb2\xb9\x06\xe5\x5b\x7e\x84\x03\x90\xa9\x16\x11\x27\x87\xbd\xf8\xd3\x20\xff\x47\x9c\x03\x9b\x64\xe3\x83\x48\x3b\xe4\xe9\xfb\x0c\x99\xed\xbc\x57\x58\xdd\x05\xf5\xa6\x1b\x71\xce\x7d\xc2\x26\x5f\x9f\x2f\xec\xab\xee\x99\xaf\x86\x52\xa8\xe8\x81\x7a\x66\xba\x92\x48\xe3\xae\xda\x8e\x12\x4d\x37\xac\x30\x70\x1e\x29\xc7\xc8\xa4\xbd\xdf\xdb\x43\x23\x12\x66\xa9\x24\x8e\x7c\xab\xf3\x07\x7f\x6c\xff\xa4\x36\x90\xcc\x83\xae\xea\x37\x86\x9a\xe0\x7f\x7a\xed\x6f\xeb\xb5\x2e\x7f\x6d\xf3\x7f\xea\x01\x30\x39\x7e\x70\x0f\x68\x11\x7e\x73\x11\xfa\xcf\xeb\x00\xbf\x71\x0e\xff\xe0\x3e\xf1\x3f\x92\xf9\x7b\x49\xe6\x07\xa8\x2e\xce\xec\xd3\x10\xde\x17\x97\xf1\xf2\xc3\xa5\x97\x2b\x96\xe4\x74\x1d\xfa\xcf\x91\xd6\xac\x97\x22\xa9\xc7\xc7\xf2\xa3\x77\x95\xaf\x68\x65\x0e\x7f\x51\x65\xe7\xd8\x48\x51\xd6\xc3\xa7\x13\xb9\x72\x04\x02\x1a\x96\x95\x8f\xa3\xf8\x7e\x84\x9c\x73\x50\x6e\x46\xd6\xf0\xbf\x53\x5e\xeb\xb4\xb0\x03\x65\xe6\x8b\x0a\x77\x6e\x2a\xda\xb2\xf1\x95\x7b\x29\x22\x98\xdc\xcd\x8e\x6f\x4e\x87\xca\xf7\x8a\x1d\x1b\xe2\xa5\x0c\xcc\x59\xcb\x9c\xd2\xa7\x09\xc1\x28\x07\xb5\xe3\x73\xcc\x6c\x3d\xe7\x63\xa5\x20\x02\x45\x0c\xf2\x20\x59\xec\x74\x11\x29\xba\xd4\x79\x75\x57\x0d\xc6\xf8\x81\x3e\x5c\x7a\x46\x2e\x24\x22\xa1\xc7\xbe\x86\xf4\xe0\x73\x9c\xbf\xc3\xe4\x7f\xf3\xa7\x0d\x78\x30\x5f\x66\xf4\xe4\x62\x8f\xcb\x22\xb7\x42\xec\xfc\x2f\xc3\x7e\x82\xbe\x7e\xc9\xc3\x8c\x6a\x11\xf5\xb8\x92\x4c\x53\xbe\x66\x2c\x96\x03\x44\x8c\x6c\x9d\x25\x65\x3a\xf7\x3e\x78\x48\x84\x72\x70\xad\x3e\x06\xd2\x8a\xf8\x3a\xa1\xde\x6c\x6c\x3b\xe4\xe8\x2c\xc5\x01\xb7\x1c\xe8\x5a\x97\x2f\x73\x9d\x38\x97\xbb\x50\x9d\x59\x68\x7d\x66\x41\x59\x42\xdf\xc8\x21\x3e\x0c\x41\xe9\xb6\x88\x51\x24\x7d\x81\xe2\xf0\xe7\xa9\x2c\x24\x39\x19\xf1\x94\xee\xfb\xfd\x2a\xad\x65\xf0\x87\x4d\x01\x07\x26\x00\x4e\xd8\xa1\x1f\x7b\x1d\xb5\xc9\xf1\x31\x0e\xfc\x45\x99\xe5\xd0\x98\x8d\xfe\xa2\xf4\x7c\xf4\xd4\x6a\x9c\xce\xaa\x1d\xe6\x66\x47\x6a\xae\x0e\x30\x57\x73\x85\xd0\x28\x40\x29\xec\x0f\x2b\x60\xb7\x9b\x02\x2f\x0f\xf4\x25\x7b\x41\xd1\x9f\x1e\x58\x2f\xd8\x94\x6a\x78\x3b\xd8\xa4\xfb\xaf\xf8\x12\xc8\x03\x47\x12\x5a\x14\xf1\xd3\x21\x71\x95\xa0\x9f\x42\x74\x68\x55\x88\x65\x25\x96\xb5\x40\x37\xac\x6f\xc5\x65\xcb\x88\xe2\x4c\x5a\xb4\x94\xda\xb7\x7e\x91\xce\x2f\x9d\xa1\x47\xdb\x32\x61\x46\x2a\xdd\x0c\x3f\x48\x86\x7c\xe1\x05\x7d\x83\xbd\xc0\x78\x74\xa7\xa9\x48\x93\x69\x1c\x32\xa9\xcb\x4a\x26\x50\x9f\x5d\xd6\x32\x44\x5a\x58\xeb\x68\x40\x35\x54\x47\x9d\xc7\xc7\x3b\x96\x60\x5d\x00\xe8\x22\x04\xde\x89\xbf\x8c\x90\x2c\x45\xfe\xe5\xd3\xb7\xf8\x8c\x6b\xe0\xe1\x53\xb7\x38\x67\x60\xe2\xe4\xed\x94\x5e\x67\x59\xc5\x75\x92\xa3\xd7\x4e\x39\x77\x22\x5f\x80\x27\x12\x44\x71\x65\xf4\xbf\xf8\x45\x59\x44\xcd\x94\x3a\x0c\x52\x25\x34\xb1\x58\xa0\x36\x32\x83\x6b\x3f\x99\x8f\x7e\x15\x99\x06\xb3\x64\xff\x7d\x14\x4a\x30\x1b\xc7\x5e\x16\xcb\x26\x70\xba\xb8\x11\x65\x33\xd6\xf1\x0b\x6f\xda\x53\x2d\x86\x7a\xe9\x62\xac\xdd\x66\x2c\x19\x42\xe8\xe9\x29\xa4\xf0\x76\xd7\xc1\xc6\x52\x6c\x6c\x17\xc3\x37\xd1\x4c\x3d\x09\x04\x9a\xc8\x4c\xbf\xfa\x23\xa4\xa7\x96\x0e\x88\xd3\xe0\xe4\x46\x48\xdf\x2c\x10\xac\x07\x27\xe7\x82\xbd\xb1\x50\xea\x10\x53\x87\x3a\x75\x08\xa9\x58\x58\x04\x6a\x6b\x9e\x40\x1d\x5f\x92\x67\x0e\xfd\x00\x83\xed\xa0\x89\x1f\xe3\xdd\xb0\x93\x0c\xdb\xcf\x31\x84\x9b\x1d\x05\x71\xe2\x71\x3c\xf9\x4b\xfb\x09\x00\x7c\x84\x82\xee\xa2\x5e\x5c\x52\x41\x76\xc2\xa1\x4e\x3a\x31\x9b\x75\xf8\x94\xce\x2f\xdc\xb1\x01\x55\x11\x6e\x04\x1c\x1e\xc8\x51\xf4\xce\x10\x71\x77\xc7\x71\xaa\x15\x1f\xd9\x41\xed\x97\x45\xb6\xa5\x9c\x88\x1f\x0d\x46\xf2\x81\x66\x2e\x87\x07\x98\xf3\xb4\x14\xe4\xcd\x08\xc6\x95\xb7\x2d\x63\xca\x79\xbc\x06\xed\xcc\x1a\x52\x52\x77\xd4\xe8\x76\xd3\xa7\x51\x7f\xd4\x3a\x6a\x2c\x9c\xee\xbf\xb8\xbf\xfb\xb3\x5b\x25\xe8\xdc\x2d\xbd\x0f\x1f\x79\xc6\x82\xf8\x29\x6a\x22\x31\x6a\xf4\x3c\x15\xdf\xb3\x9e\x6f\x80\x1e\xf6\x08\xef\x56\x43\x3d\x23\x59\x97\x1e\xf2\x5c\x23\x91\xfc\xee\x2e\x90\x57\xdd\x05\x4e\x74\xaa\xa1\x4c\x17\x40\x36\x8d\xec\x2e\xa1\x1f\x52\x6e\x14\x37\x46\xdc\x38\x9d\x41\x86\x57\x00\x5d\xd0\x11\x8a\xd8\x6d\xdb\xf0\xd1\xc0\x15\x02\xcc\x84\xef\xd7\xb7\xc0\x92\xa7\xdd\x96\x79\xc2\x69\xd7\x4c\xcc\x44\x4a\x2d\x34\x80\x3a\x2f\x7b\x6f\xd0\x68\x3a\x83\x9a\x57\x27\x3a\x24\x6a\xf3\x8d\x73\x40\xcb\x0e\x9b\x6a\x4f\x52\x3f\x1b\xfb\x6b\xb3\x48\x75\x66\xdc\x6a\x83\xf9\x73\x47\xd2\x60\x3c\x39\x3e\xa6\x57\xf8\x71\x88\x30\x35\x50\x2c\xcd\x46\x8a\xe6\x46\xe5\xf1\xb0\xe3\x57\x56\x7c\xb6\x2b\xb5\xaa\xab\xcd\xa2\xc6\x54\x8e\x7e\xe6\x1e\xed\x90\x3d\x7e\x34\x08\x1f\xcc\x35\xdd\xb0\x31\xd3\xcc\xdd\x04\x35\xa4\xb1\x2a\x0c\xda\x36\x70\xeb\x87\x23\xce\x7f\xb5\xfa\xad\x98\xe6\x07\xd5\x8f\x24\x77\xbf\x3a\x66\xd5\x58\xb9\x2d\x3a\x2c\xf1\xd6\x8c\x83\x0a\xe8\xdf\xd2\xa6\x40\x11\xe7\xb2\x9e\x72\x9a\x6a\x38\xc6\xf1\xec\x11\x4d\x46\xf6\x99\x8d\x66\x4d\xa4\xe6\x3c\xd2\x70\x45\x79\xb9\xe1\xeb\xb7\xe2\x11\xac\x1b\xe2\xa8\xdb\xb2\xf4\x10\xae\x0f\xb6\xa6\xff\xb5\xc0\x3e\x68\x17\xec\x36\xe2\x04\xc6\x14\xf4\x52\xb6\xac\x3b\xbe\xe3\x96\x2d\x40\x97\x11\x94\x42\x17\xa6\x09\xc0\x71\xd3\xa6\x67\x0a\xab\x21\x37\x15\xe4\x62\x0b\x16\x7d\x54\x17\x35\x8e\xea\x6c\x93\x90\xb6\x74\x69\x25\x42\x07\xa2\x91\x6f\x4c\x30\xcc\xe9\xe9\x63\x50\xf2\xe9\xa4\x99\xeb\x06\x9f\xfe\xe3\xbe\xbb\x04\xd3\xdc\x72\xaf\x2e\x6b\xb1\x89\x6e\xe9\x35\x19\xd9\x6a\x02\x47\xbf\x70\xc7\x2f\xd7\xa3\xfe\x56\xd0\xba\xbb\x3d\x09\x0f\xe3\x42\x4b\x88\x32\x61\xbb\x39\x44\x17\xc4\x3f\x64\x30\xe6\xbe\x58\xf3\x89\xdd\xd8\x6f\x9c\xde\x89\x1d\x10\xb2\x15\xa1\x31\x97\x8d\x61\x6c\x09\xbd\x0f\xb9\x73\xb4\x59\x1c\x46\x1e\xe7\xf3\xfd\x84\xdb\x46\x35\x36\x16\x8a\x44\xc0\x2d\xdb\xec\xec\x25\xce\x3a\x72\x2e\xf6\x61\xa0\x63\xdd\xbd\x24\x38\xa6\x2e\x7b\x30\xb0\xf1\xc9\x7e\x1c\x0d\xe3\x94\x76\x2c\x5b\x65\xad\x84\x1a\xa8\x7a\x30\x5d\xaf\x83\xb8\x9e\x6a\x39\xa7\xaa\xc5\xc6\x5a\x3a\x5a\x9d\xd6\x92\xbf\x98\x50\x2d\x2e\x25\x13\xf9\xc5\xe7\xd0\x5e\x72\x72\xb4\xa4\x5e\x25\x18\x62\x03\x96\x45\xb5\x22\x62\x21\x53\xbb\xdb\xfc\x04\x92\xda\xf0\x91\xb2\xa9\x77\x89\x84\x24\x49\x91\x26\x24\x76\x55\x0a\x70\x1d\x86\x33\x55\x4b\xad\xd1\x71\x2d\xd5\xf4\xc6\x95\x53\x93\x01\xd5\x89\x46\x4e\x6b\x3e\x99\xcb\x5a\xb9\xf1\x1b\xbb\x5e\x6e\x12\x47\xca\x9a\xb9\x49\x18\x15\x70\x25\x1b\x09\x57\xe8\x2b\x92\xea\xeb\x26\x60\x54\x20\xab\xee\xa6\x50\x5c\x20\xe4\x98\x1d\xca\xbf\xdb\x86\x35\xd9\x8f\x6a\xf5\xd9\x66\x50\x46\x7e\xb2\xbf\x22\x40\x9f\xe1\xc5\xbc\x94\xaf\x47\x2c\x92\xeb\x74\x9e\xfc\x90\x56\xc9\xea\x47\x9c\x83\xa2\x9b\x74\xbd\xc8\x6e\x7a\xcd\xf8\xbb\xbb\x81\xc0\x55\x53\xba\x5e\xfe\x84\x6a\x18\x03\x03\x9a\xde\x4d\x32\xbb\x48\xcb\x2f\xad\x34\x3b\x17\x42\x5c\x66\xef\x0f\x26\x17\x87\x52\xb3\x43\x89\xb3\x3d\x69\x03\x41\x13\x6a\xd4\xac\xc5\xe9\x4e\x15\x70\xce\x6a\x42\xc1\x64\xbd\x03\xc7\xdc\xba\x89\x24\xbf\xe5\x1b\xde\x2a\x28\xdf\xef\x96\x21\x58\x7e\x3c\x2b\xcb\x3c\x9d\x5d\xa1\xff\x78\x02\xf5\xc4\xcd\x89\x9c\xe3\x5b\x81\x18\x81\x27\xce\x9b\x50\xb8\x6b\x24\xbd\x32\xde\x74\xbc\x4d\xe5\xb9\x29\xd2\xf9\xe2\x39\x25\x69\x4f\x69\x88\xc2\x4c\xe2\xe9\x3a\x2d\xd3\x78\x05\x33\xb5\x3e\x1e\xa0\xe6\xc7\xd9\x1a\x99\x68\x4e\x05\xb8\x92\xdf\x7d\xff\xf5\xb3\x9f\xfe\x86\xae\xff\x30\x15\xe8\x54\xc9\x63\xdf\x05\xb7\x5f\xa5\xc0\xcc\xec\xe1\xd0\xc0\x53\xf4\xdd\x1d\xe3\x83\xc1\xcd\xcd\xcd\xe7\x0d\x8e\xd3\x44\x89\x62\xe5\x42\x40\xc2\x56\x99\xe9\x41\xd3\x18\x99\xfb\x20\xba\xdc\x2c\x91\x4d\xda\xc3\x29\x73\x91\xec\xa7\x0f\x24\xfe\xc1\x84\x49\x58\xa2\xe8\x3e\x4a\x82\x5b\x28\x69\x2f\x52\x48\x7d\x40\x7e\x5b\xbd\x53\x8b\xbf\x05\x2b\x12\xfc\x6e\x6e\xf3\x5e\xdf\xcc\xd5\x36\x7c\xb5\x8b\x21\xf7\x4e\xc9\x18\xc0\xbc\xc7\x4b\xe3\x70\x30\x0a\x66\xd6\xd2\x92\x52\x69\x1d\x4a\x5f\xb4\x1f\xac\xf5\x2c\x6b\x53\x22\x29\x2d\x3a\xf4\x13\x62\x47\x14\x1a\xeb\xc7\x99\x38\x71\xe7\xe9\x28\x56\xd5\x2d\x77\xb3\xec\xce\xba\xbd\x8e\x8e\x45\xc2\xbd\x90\x13\x4f\x5a\xf7\xa2\xe5\xba\x37\xc5\x77\xbe\x1c\x96\x04\x21\xf2\xc2\xf6\xa5\xa4\x2a\xb2\x74\x6c\x2d\x66\xb3\xac\xa2\x2d\xe7\x91\x63\xf1\x0e\x2b\x02\x4e\x69\x63\x33\xfb\x65\xf6\x77\x0f\xe5\xc8\x3c\x10\xf3\xa1\x47\x61\xfa\x5b\xcb\xbf\x6a\x03\x87\x02\x6a\x07\x87\x4d\x44\xd3\x0d\x7e\x61\x02\xa9\xb3\x79\xcb\xb4\x41\x42\xf9\x06\xe4\x88\x6f\x66\x72\x30\x59\xd9\xa1\xf4\x72\x89\x87\x3f\x51\xdf\xf6\x14\xae\xf7\xbd\x0e\xb8\x49\x35\x43\x91\x29\x31\x59\x09\x39\x9a\x12\xa5\xf8\x40\x00\x8d\x4d\xba\x60\x00\x78\x94\xac\x02\xcb\x8d\x2a\x7b\xcb\x64\xc7\xa7\x68\x06\x9f\xac\x90\x17\x00\x23\x3f\x51\xf9\xf0\x6c\x3b\x6b\x2f\xe8\xe1\x83\x35\x3e\xba\x26\x55\x54\xc9\x74\x3a\x3a\x43\xf7\xca\x9b\x0d\x3e\x04\x2d\xb3\x41\x86\x18\x46\x67\x08\xae\x62\x72\xc7\x2e\xd1\xa8\xd1\x17\x9f\x62\x24\x7f\xa3\x78\x34\xea\x4b\x8f\xd6\x79\x82\x16\x90\xfe\x4e\x6d\x1c\x77\xd3\x04\x63\x6a\x7f\xa0\xea\x72\xd4\x27\x00\x8e\x51\xa3\x3d\x43\x73\x9c\x7c\x4f\x21\x82\xbf\xba\xad\xa4\xff\xa4\x26\x83\x74\xb2\x8a\x68\xf0\x48\xcd\x5f\x5c\x73\x35\x6f\xe1\x9f\x0e\x60\xef\xe1\x49\x15\x7d\xd0\xb9\xa2\x62\x90\x9e\xb9\xe8\x2f\xa5\x97\xd9\x86\xfe\xf2\x11\xa2\xf4\x35\x37\x32\x72\x25\x8b\x59\x67\x0b\xbc\xb9\x00\xbd\x1e\xef\x1a\xe1\x7a\xd0\x1b\x2e\x3c\xc9\x4b\x04\x94\xcb\xe2\x08\x97\xdf\x4d\x0d\x46\xe6\x36\xc0\x01\xce\xfa\x26\x2b\x74\xc1\x57\xd8\x9f\x61\x10\xbb\xf4\x0b\x7c\xde\x00\xf4\xf9\xe2\x44\xd5\x43\x7e\x02\xa5\x7a\x3b\xa9\x65\x9a\x34\xc8\x77\x3c\x49\xcf\x99\x64\xd3\x90\x34\xa5\x2a\xd1\x2a\xab\xb1\x6f\xd5\xb7\x64\x77\xe4\x41\xa8\xe3\x1c\xa7\xd4\xab\x04\x2f\x23\xec\xf3\xa0\x0c\x0d\x66\x43\xa3\x94\x42\x7b\x93\x6e\x5c\xec\xcd\x23\x7b\xa8\x2b\x79\xf4\x30\x8a\x79\x82\x89\x96\xfd\x96\x65\xe1\x6c\x36\x14\xcb\x48\x93\x48\xef\xd2\x3b\x32\x26\x6e\x22\x4b\x32\x5b\x64\x02\x14\x22\x5b\x50\x5b\x84\x01\x86\x1e\x1e\x00\x19\x8e\x77\x12\xe5\x96\xa8\xda\x94\x04\x10\x7b\x00\x6d\xce\x0a\x3c\x90\x25\x71\x4e\x67\x2b\x5d\xdd\xa2\x5d\x59\x1a\x1d\x25\xb3\x24\xc8\x1d\x78\x79\xfa\x32\x0b\xb8\xb6\x74\xa2\x67\x76\x17\xad\x72\x87\xf7\x14\x3c\x9b\x69\x0f\xe2\x30\xf2\x0e\x41\xe4\xfc\x5f\x4b\xcb\x30\x08\x9c\x7d\x7e\x61\x33\x84\x66\x13\xab\xe9\x30\xaf\x23\x69\x2d\x76\x7c\xd8\x87\x69\x1c\xe6\x6d\x16\x15\x34\xe7\x52\x8d\xab\x27\x84\x74\xa2\xc0\xa6\x23\x3d\x7c\x93\x17\x3f\xb2\xbc\xd8\x29\x48\xd8\x5b\x38\xce\x06\xe6\x3d\x93\xb1\x35\xdd\x8e\xd5\xc4\x6c\x4f\xc1\xe1\x6c\xeb\xe0\xbe\xbb\x73\xe7\xd7\x5d\x39\x7e\x56\xd4\xeb\x79\x43\x98\x99\x11\xee\x41\x9c\xae\xd6\x9b\x18\x73\xbc\x49\x17\xc7\xc7\xd4\x62\xaf\xd2\xcb\x24\xbb\x42\xa7\xd0\x4d\x80\x40\xec\xc6\xa1\x60\xa8\x1c\x4e\x9f\x93\x80\xfc\x21\xc9\x10\x0b\xe8\xd2\x65\x72\xb4\x8b\x66\x2b\x9e\xf4\x9d\xca\x68\x3b\x12\x53\x95\xab\x3c\xe5\x8a\x5c\xe5\xab\x86\x29\x0c\x4b\x87\x41\x2b\xbb\x30\xe5\xb7\xd5\x0a\x03\x21\xc7\x81\x4e\x34\xd8\x79\x45\x6a\xec\xcb\x8c\xc9\x0d\x30\xe1\xdd\x55\x0a\x62\x68\x66\xbf\x17\xbc\xec\xc4\x17\x51\xe9\x05\x38\x40\x82\x94\x99\x97\x29\x93\x3c\x17\xf4\xea\x9c\x6a\x63\x88\x18\xfb\xea\x65\x2c\x4c\x56\xd7\x50\x40\x63\x65\x83\x91\x22\x9f\xd3\x7b\x72\xc2\xb6\x1f\x89\x1e\xf5\xc5\x0e\xbd\x5d\xa4\x97\x33\x6f\x03\x95\x9f\xf4\x23\xde\x13\x40\x2b\x9b\x9d\x07\xad\x3a\x48\x1f\xa3\xce\xd6\xa4\x9f\xd8\xef\xa0\xd0\x23\x5f\xbe\xf7\xf2\xfb\x67\x5f\x1f\xbd\xf8\xf6\xd9\x5f\x9f\x87\x47\x5e\x87\x4c\x77\x1e\x44\x4d\xa3\xa5\x49\xfc\x64\x73\xcb\xed\x34\xaa\x9e\x41\xb8\xd5\x59\xdc\x95\xfa\xdf\xe4\x99\x4e\xdb\x42\x3d\xb9\x86\x79\x00\x9f\xe1\xa2\x17\x6b\xe5\xa9\x5c\x84\xdf\xfa\x05\x44\x7c\x15\x56\x35\x73\xff\x69\x3a\xc6\xc4\x90\x20\xd4\xc3\x6c\x29\xef\x54\x9e\x37\xcb\x01\x35\xea\x12\x9a\x7d\xa5\x34\x3c\x3e\x33\xb1\xd5\x3a\x99\x85\x9c\xc9\x27\x2b\xf9\xc2\x81\x99\x16\x51\xc1\x62\x14\x32\x89\xbe\x7d\x46\x6a\x6b\x82\xfa\xa8\x47\x2b\x82\xf7\x6a\x80\xa8\xbd\xd0\x3b\xbb\x5a\xeb\x73\xf5\x3a\x77\xb6\xdf\xaf\xf9\xed\xa8\x7d\x4e\xc6\x5d\x5d\x09\x1f\xa9\x86\x7f\x6c\x22\x67\xab\x95\x76\x49\x00\x1b\x97\x6a\x38\x61\x59\xc7\x86\x2a\x7a\xa0\x5c\x3c\x8f\xe7\xe7\x66\x44\xe0\xe7\x0a\x94\x1a\xb5\x58\x3c\x47\xb8\x97\x69\x51\x82\x9c\xe4\x94\x6a\x75\xa0\xeb\x32\xb8\xd5\x8c\x99\xd0\x28\xac\x06\xdf\xb8\x14\x98\xbc\x0d\xb6\xbb\x9a\x87\xaa\x84\x46\x44\x5a\xa3\x33\xfb\x37\xeb\x69\x67\xa7\xe6\xb2\x1e\xae\x63\x91\xd0\x9b\x9c\xee\xbb\xca\x4a\x8b\xe1\x4c\xf4\xeb\x68\x32\x14\xe3\x3c\x06\xa1\x44\xa8\xf1\x04\x91\x56\x0c\x64\xba\xf5\xb2\x30\x3e\x48\xea\x9f\xb7\x3d\x74\x7c\xa1\xe9\x3a\x9f\x5c\x4c\x5b\x00\x2a\xeb\x25\x64\x72\x1c\x5f\xd1\xc3\xc6\x71\xc0\x8f\x1b\xf3\x93\xaa\xd4\x54\xd1\xc4\xbb\xcc\xae\x8a\x44\xbe\x9c\x44\xdf\x57\x1b\x7c\x6e\x09\x7a\xcd\x05\x3e\xb7\x34\x5b\xa9\xcf\x9b\xf3\x24\x59\xc1\x5f\x20\x4c\x82\xc3\x17\xac\xfc\x68\x1d\x00\x9f\x94\x8f\x30\x30\xe4\xf4\x61\x52\xd1\x68\xea\xc8\x15\x04\x62\xcf\x19\x8e\xc1\x24\x25\xd4\xfa\xd0\xf8\x12\x25\x55\x8e\xcb\xc4\x05\x87\xd7\x96\x00\x53\x13\xc4\xdb\x0d\x2d\xa1\x1b\x25\xf1\x31\x46\x53\xb5\x9b\x45\x90\xd8\x2b\xd1\xd9\x73\xd9\x53\x9a\x0f\x24\x7c\xb5\xc2\x63\x7a\x52\x6f\xf0\xa0\x05\x81\xe6\x14\xf5\xcf\xee\x8c\x75\x9d\xda\x8a\xfc\x05\x22\x51\xed\x89\x23\x7b\x8c\x11\x1b\x0e\xf2\x61\x95\x11\x1c\x3e\x11\x42\xbc\xa2\x16\x95\xd1\x9e\x6a\xa5\x3e\xe9\x91\x6e\x83\x26\x25\x52\x18\x89\x49\x16\x2f\x88\x57\x66\xaa\x89\xe5\xe0\x64\x81\x21\x67\x18\xca\x19\xf9\x36\x62\xd3\x02\x7b\x9d\xe4\x2e\xca\x6d\x93\xa7\x80\xae\xad\xf1\x24\xda\x07\x17\xdf\x54\xf2\xbe\xfe\xfe\xdb\x6f\x11\xfc\xa7\x79\x9e\xad\x56\x7b\xc5\xc3\x96\x3c\x16\x13\x0b\x07\x42\x44\x8e\xdc\x09\xdd\xe6\x8d\x36\x69\xf6\x48\x96\x4a\xdc\x0e\x05\x0e\x1a\x83\x4a\x65\x28\xd9\xe2\x77\x09\x9d\x2c\xa9\x0e\x6f\xb4\x46\x1a\x53\xb1\x54\xea\x86\x86\x34\xfb\x51\x20\x22\x4c\xe6\x64\xe2\x70\x06\x8b\xcc\xf4\x47\x3d\x66\x87\x44\x25\x0d\xe7\xd1\x39\x3f\xde\x0e\xfa\xb7\xfc\xa2\x37\xbc\xcf\xe5\xc3\x2a\x98\x3b\xb4\xcb\x08\xe5\xdf\x96\xd1\x34\x3b\x3b\xfb\xcd\x74\x31\x09\xc8\xba\x73\xb3\x6b\x96\x1a\x96\xa9\x63\x7d\x62\x18\xcd\x90\x11\xbd\x53\x83\x96\x11\x8c\x02\x38\x6e\x31\xf3\x51\xa4\xa6\xd0\xbb\xbb\x73\x7a\xbb\x04\x86\xbd\x54\x0c\x82\xc0\x7e\x18\x6a\x7b\x70\x39\xda\x98\x11\xe4\x94\xd4\x5c\x6b\x3b\xf3\x0a\xf4\xce\xe6\xea\xd0\x3a\xf3\x25\xb3\x38\xae\xb8\x2c\x0c\x3a\xee\xbe\x45\x07\xe2\x6a\x59\x72\x60\xb4\xb3\xe4\xd0\xc7\xd1\xce\xba\xa3\xe5\x48\x5a\xb5\x8a\xd4\xa5\x8a\x6b\xf5\x30\x6a\x41\xaf\x24\xef\xbb\x4a\x5f\xf9\xe6\x52\x39\x9b\xa9\x9a\x4d\xba\xda\x49\xab\xdd\xb4\x73\x37\xb1\x63\x5d\xd5\xb1\xb6\xf9\x50\x29\x72\x6f\xad\x47\xea\x1e\xf8\xa0\xdf\x0f\xf9\xd3\x82\xc7\x5b\xd8\x0e\xe2\xc6\x5d\x70\x0b\x14\x77\xf7\xd1\x5d\x83\x32\xcc\x82\x76\xda\x08\x94\x33\x65\xd9\x15\x2d\xa8\xde\xe3\x45\xa8\x1e\x6b\x19\xe3\xe3\x2b\xb4\x13\xa9\xae\x9b\x83\x20\x65\x81\x6d\x47\x05\x48\x7f\xc8\xf1\x05\xa2\x86\x35\x15\x62\xb7\x92\xd1\xde\x84\x24\x9f\xa2\x8a\x09\x26\x4f\xb9\xe0\x28\x93\x21\xae\x2d\xc5\x8d\x3d\xb4\xb9\x45\xb3\x3f\xcc\x34\x96\x0f\x1d\xe1\xf1\x04\xac\x9a\x10\x61\x92\x97\x35\xa9\xb4\xc2\x5b\x67\x6b\x18\xb2\xe9\x42\x42\x68\x01\xf2\x46\x9a\x03\x0b\x63\xb9\x32\x6c\x83\x8e\x22\x9b\xfa\xcd\xfc\x0a\xf4\xf2\x45\x72\x56\x4c\x54\xe2\x94\x89\x90\xe4\x79\xa0\x81\xfb\x7f\xf1\x3a\x2a\xb5\xe3\x05\x6a\x4b\x69\x0f\x49\x04\xd9\xf1\x24\x51\xce\x25\xe3\xb9\xbf\xcf\x03\xc3\x48\x9d\x75\x58\x67\x38\xa5\xda\x6f\xa2\xa7\x19\xd5\xde\xb7\xd7\xa9\x3a\x10\xd1\xa9\x6d\x4a\x9c\x7c\x0b\x0f\x0f\x24\xdf\xd0\x8d\xcd\x2c\x08\xda\xfd\x9f\x40\x8b\xee\x9a\xa9\x9a\x0b\xbe\x23\x84\x25\x14\xfb\x1d\x77\x98\x67\xb1\x02\xed\xac\x43\xc9\xac\x0c\x8f\x7b\x9f\x86\x96\xff\x8e\x7d\xc4\xca\xa2\xd4\xbe\xc8\xb6\xdd\x69\xc9\x7d\x24\x23\xec\x9f\x44\xb2\x2e\xaa\x85\x64\xc9\xf7\xdf\xb9\x9d\x6d\x17\x16\x52\x25\xd8\xa1\x4a\xba\xaa\xb0\x1d\xe1\x83\x9a\x95\x99\x83\x34\x0b\xbc\x42\x70\xb6\xae\x6f\x49\xad\x3d\x6d\xc4\xdf\x92\x2a\x77\x72\xcd\xdd\xc3\x56\x28\xb5\x91\x6b\x0d\x75\x8d\x3b\xda\xfe\xfe\xbb\xd9\x6d\x7c\xaa\x06\x9e\xd0\x8f\x44\x36\xe8\x85\xa4\xba\x3d\xa9\x1a\x7a\x62\xf7\x86\x58\x3b\x16\x0b\xd4\x5c\x1b\x0b\x9a\xae\x3c\xfe\xac\x4e\xcc\xa5\xb5\xf4\xe3\x94\x6f\xbb\xa8\x23\xe4\xdf\xf3\x12\x10\xd4\xe8\xc3\xef\xff\x40\xed\x7f\xe5\xd5\x9f\xf6\x7d\x17\xba\xef\xb4\xcb\xcc\xef\x7e\xf2\x0f\x3f\x05\x2e\xbc\xf3\x3c\x39\xf3\x04\x6d\x9a\xb4\xcb\x7c\xb5\x4f\xdc\xeb\x43\x92\x7e\x73\x50\xc0\xed\x1e\xe7\xdc\xd0\x91\xf3\xb2\x2f\x4d\xdb\x70\xba\x18\x5b\xdf\x1d\xef\xc8\x0b\x3d\x8f\xee\xb4\x60\xe4\xcf\x31\x7a\xc2\x28\xc7\x4e\xa8\x09\xc4\x2f\x32\x8f\xed\x80\x01\x51\xc2\xab\x2c\xe9\xc6\xe6\x73\xf7\x9e\x9f\x36\xb7\xc3\xb3\xfe\x23\x4f\x15\x70\x77\xd7\x0a\x19\xb8\x46\x7c\x7b\xfb\x00\xda\x09\xab\x7b\x3d\x0b\x7d\xb1\x07\x16\x70\xd2\x10\x8f\x02\x74\x65\x49\x5f\x5e\x9a\x34\x2e\x2f\x4d\x05\x0a\x42\xe4\xc1\x82\xad\x21\x33\x5e\xef\xf3\xe4\xd2\x0b\xdb\x45\xcd\xeb\x3d\x7e\x82\xa9\xed\x63\x07\x34\x7d\x67\xb1\xbf\xf5\x3b\x8b\x76\x01\x58\x40\xe2\x9e\x24\xa4\xbf\x1b\xaf\xe7\xe7\x19\x2c\xf0\xa8\x02\x81\x88\xc7\xf7\x0c\x01\xf2\x42\x94\xd7\x89\xb1\xd5\x9c\x51\x20\x94\x67\x7e\xa8\xb8\xb4\xe6\x0e\x88\x2f\x6d\x25\x20\x95\x98\xb6\x0f\x07\xa4\xcb\x2a\x20\xd1\xa4\xb1\xaf\xcb\x48\xde\x07\xda\xa3\xcb\x78\xd8\xea\xf2\x25\x7b\x7d\xe5\x8c\x54\x1a\xdb\xea\x1f\x6d\xdd\x7f\xef\xe1\xb0\x81\xff\xcd\x6c\xd9\x32\xd6\x35\xee\xbb\xb7\x95\xf5\xc1\x9d\x97\x8e\x54\xca\x78\x29\xf0\x24\x52\xac\x0f\x5a\xc1\xd2\xfe\x29\x5f\x7a\xfa\x16\xd4\x66\x37\x9e\x8f\x77\xdf\x78\xc2\x45\xd6\xb8\x45\xd4\x92\x6f\x03\xe3\x57\x5a\x35\xb2\xb1\x5d\x55\x5c\xc6\x11\x06\xc7\x13\xeb\x9c\x64\x1a\xda\xe6\xfd\xb0\x48\xd6\xa7\x28\xc6\x8a\x3d\x42\xa3\xf9\x31\x2b\xca\xa4\xcd\x8b\x65\xbe\x29\xa2\xa5\x3a\x2f\x9d\x9f\xa7\xab\x05\x6e\xe0\x17\x62\x9d\x95\x7f\x8d\xbc\x25\xde\x64\x41\x12\x36\x91\xbf\x89\x10\x78\x42\xd4\x75\x06\xd3\x60\x6c\x36\x49\x37\xe4\xaf\x4b\x9e\x72\x2f\xf5\x01\x77\x8a\x4f\x8c\x2f\xbd\xce\x51\xa7\x43\x4c\xc0\xf3\x0f\x7c\x6a\x15\x74\xe8\x8d\x01\x09\x44\x11\xa1\x32\x7d\xf4\x14\x7a\x01\x96\x75\x89\xf7\x34\xf4\x61\x7b\x21\xcf\xd7\xe9\x40\x40\x24\xd1\x65\xf3\x58\x1d\xb2\xe8\xc5\x2b\x52\x3d\xf6\x37\x2c\xc3\xbe\xb7\xc9\xa8\xf2\x5d\xde\xd1\xa2\x3d\x8d\x02\x70\xf4\x12\x67\x0f\x0d\x96\x42\x6a\xc1\xd3\x7b\x83\xcb\x3f\x94\x11\xbd\xf4\x81\x05\x02\xbd\x94\xa9\x53\xe8\xd8\x20\xd1\xd5\x45\xc5\x6a\xe7\x48\x1f\x6d\xbc\x96\x64\x5d\x81\x2f\x84\x1e\x22\xe6\x72\xe7\xbc\xff\x92\xc9\x43\x84\x01\x35\xc3\x58\xc6\x10\x9e\x20\x74\x98\x43\x6c\x43\x12\x7a\x4e\x91\x04\xaf\xba\x4c\x60\x67\x17\x9b\x43\xf7\xd7\x6c\xd1\x53\x0b\x44\xbc\x93\xb6\x89\x0e\x4a\x29\x0c\xd7\x5c\x18\xb5\x37\xdf\x40\x09\xc4\x1c\xda\xb1\x45\xb0\xa2\xb9\x5a\x22\xca\xe5\xa3\xb9\xa1\x46\x05\x92\x51\xfc\x1c\xf7\x2c\xde\xbc\xc1\x46\x7f\xf3\x46\xca\xb4\xba\x93\xe6\xa6\x35\xef\xa6\xd1\x95\x36\x17\xc4\xbd\xda\x56\x38\x27\x00\x08\x89\x52\x59\x01\x1d\x0f\xb9\xd9\xd5\x7e\x01\x4d\x73\xd2\xda\x6a\x80\x02\xf4\x0d\x2f\xc0\xdf\xe9\x08\x0f\x09\x79\xe3\xd9\x17\xbd\x5a\x2f\xa2\x99\x43\x5d\xc4\xa1\xaf\xc1\x10\x92\xdf\x72\xff\xec\x57\x53\xb9\xe7\x3a\xda\x3e\x7c\xd6\xb5\x34\x89\x92\x63\x00\xe9\x56\x3a\x6b\xe4\x35\xa9\x71\xb3\x39\xf7\x03\xb3\xf8\xdb\xf1\xbe\x59\x91\x9f\xcd\x9a\x3c\x6a\xd6\xd2\x9e\xd8\x81\xd4\x9e\x2a\x11\x12\x00\x6d\x85\x59\xc1\x48\x8d\x3d\xa0\x6d\x32\x9f\xdd\xd6\xd1\x2e\x8a\xf4\x8d\x26\x87\x28\x7c\xad\x59\xdf\xa5\xbe\x45\x2d\x24\xf4\xc8\xcb\xa1\x27\x58\x17\xd6\x3a\x87\x20\x25\x38\xf4\xa0\x66\xde\x56\xf0\x7e\x46\x74\x8b\xfb\x15\x21\xef\x5a\x08\xeb\x6a\x31\x47\x75\xe5\x5d\x6a\x4f\xde\xd2\x45\xd4\xf8\x57\x85\xe9\xb6\xae\x8a\xec\xca\x59\xcb\xb9\xf5\xab\x13\x1b\x98\xbe\x8a\x37\x3a\x09\xb9\x31\x8f\x37\x2a\x09\xed\x4d\x74\x1a\x9a\x02\x92\xad\x9b\x9d\xca\xb6\x7c\x0e\x0c\x5f\x75\xe3\x5b\xe1\x54\xac\x2a\x4f\xd6\x94\x36\x73\x22\x75\xc6\xc2\x95\x57\xc3\xf0\xed\xd5\x06\x7a\x5e\xa2\x0c\xa7\xe9\xa6\x54\x8b\xd7\x4a\xb2\x95\x86\x7f\xd2\x38\x1a\xbd\xf5\x48\x6b\x68\xfe\xc3\xe6\xcf\xb9\x36\xea\xce\xb5\x41\x77\x69\x8c\xb9\xe5\x81\x28\x4e\x87\xc9\x22\xbc\xa5\x62\x1e\xf5\xb9\x10\xbc\x73\x40\x94\x85\xf4\x2b\xee\xb1\x59\xf7\xc9\x77\x1a\x6f\x42\xd0\x9d\x11\x6d\xb6\xae\x52\x88\x6a\xba\x4d\x22\x2d\xd7\x4d\x96\xb9\x65\xb9\xae\x62\x37\x4d\xcb\x75\x95\xe0\xd8\x77\x73\x02\x4d\x23\xc2\xb2\xee\xe6\x68\x2c\xd1\x13\x96\x6d\x37\x47\xe3\xa7\x27\x6c\xcb\x6e\x8e\xa7\x6f\x4f\x48\xc3\x6e\xaa\x2f\xfe\x68\x93\x6e\xec\x07\x6d\xf6\xdc\x87\x0d\xf6\x70\xe6\xb3\xce\x72\x0f\xda\xef\xe1\xfe\x59\x74\xbb\xfd\xc3\xac\xf7\x1a\xe6\x4b\x68\xc1\xd7\x7e\x58\xab\x77\x6f\xbd\xa6\x2d\x1d\xd5\xc7\xcd\xa4\x26\x73\x48\xd9\x6b\x9e\x67\x2a\xaf\xb1\xd8\x3a\xcf\xef\x69\xa7\xe7\x54\xf2\xd7\xda\xea\x11\x7d\xbf\xaf\xb9\x9d\x65\x04\xc7\x59\xf6\xe9\xf0\xba\x10\x54\xe5\x15\x22\xbd\xbf\xf1\x41\xa7\x0b\xbf\xc5\xa0\x8d\xc7\xa1\xaf\x51\x24\x1b\xc2\xbe\xc4\xf7\xa4\x95\x46\x47\xbc\x82\x75\xdb\x2a\x32\x32\x0c\x33\x97\x19\xde\x62\x74\xff\x40\x92\x4d\x13\x89\x94\x30\x8c\xb0\x2c\x26\xd0\x8b\xed\xc2\x9c\x6e\x63\xaa\x11\x3c\x7d\x86\x48\xd1\xc9\xe5\x06\x96\x55\xe8\x27\x42\x23\x4d\xd7\x05\xac\xb5\x24\x52\xe1\x85\x67\x69\x5e\x94\x5d\xd2\x9b\x3c\x34\xe7\x02\x7a\x23\xca\x6b\x99\xa1\xae\x9c\xfb\xce\xca\x10\x15\x88\xc0\xae\x9e\x2e\x20\x8e\xdd\x44\x40\xde\x1d\x5b\xd4\x9d\xbc\x46\x65\x6f\xc9\xdc\x50\x51\x39\xd2\x51\xa2\xd3\x85\xba\xf4\x71\x16\x01\xbb\x26\xe9\x62\x8a\x60\x0d\x13\xd1\x60\x04\x71\x5c\xd4\x6d\x35\x08\x51\x37\xa9\xf0\x8a\x36\x7f\x0d\x45\xcd\x71\x35\xba\x6d\xe2\xaf\xe1\x56\xde\xdd\xc6\x8c\x56\xdd\x31\x4e\xd5\x18\x01\xe9\x62\x37\x83\xee\xd4\x95\x61\x65\x5a\xa3\x2a\x14\xc9\x75\x57\x53\x5e\xab\x4b\x6b\x4e\xdc\x2a\x3e\x61\xb6\x2e\xdd\xff\xde\x03\x4f\x69\x64\x64\xd1\x72\xd6\x7b\xd8\x5e\xd2\x1c\x47\x58\x5e\x8a\xc6\x72\x78\xc1\x8c\xa4\xcb\xfa\x0d\xa3\x75\xb9\xe0\xe7\x85\xab\x36\xb4\x64\xcc\x5d\xb5\xc9\x69\x3a\x84\xaf\x4d\xc9\xac\xf2\x76\x69\xa4\xa2\x0c\xa1\x96\x49\x27\x59\xb0\xa3\x7a\x2f\xe8\x58\x0e\x0f\x92\xf8\xb0\xcc\x3e\x39\xb3\x6e\xda\xb7\xfa\xa4\xd0\x06\xfe\x7c\x2e\x04\xd8\xd8\xdc\x9f\x3a\x24\x4d\x1e\xc6\xfa\x5f\x9f\x00\x52\x26\x3e\xf5\xe3\x69\x9f\xcf\x97\xc7\x06\x94\x30\x86\x44\x21\xc3\x71\xbd\x6d\x43\x18\xa2\x9c\x0c\x09\x25\x08\x6f\x7b\x34\xd2\x1e\x62\x29\x29\x97\xff\x0f\x35\x94\xd4\xbc\xf9\x20\x2b\x49\x2e\x64\xc7\xf0\x0b\x91\x1d\xb2\xfa\x82\x74\x99\xec\x3b\x67\xcf\x3b\x56\x47\xbb\x76\x1c\x6c\xaf\x61\x99\x6e\x30\xdf\xa5\x19\x87\x5a\x5b\x8d\x30\x12\xc6\x31\xf7\xa6\x06\x35\x05\xf1\x1f\xbd\xa7\x34\xce\xf1\xe3\xd2\x9c\xe3\x13\x93\xed\xcd\x90\xff\x67\xcd\xd4\x1e\xaa\xf1\x68\x6b\xb5\xa6\xa2\x73\xc0\x68\x6d\x57\x9f\x69\x53\x82\x3e\xc4\x5c\xed\x7f\x2c\xc3\x7e\xb5\x65\xd8\xef\x6b\x97\x22\x16\xd9\xe5\xce\x35\x0b\x68\x55\xdd\x9c\x55\xb4\xdf\x26\x05\x15\x9c\xd0\x0c\x01\xd6\xc0\xa2\x08\xd9\x3e\xd0\xee\x05\xaa\x08\x84\xec\x9a\x1d\xd2\x29\x77\x85\xc3\xd0\x1f\x69\xfe\x72\x1f\x13\xfe\x18\xf3\x18\x1f\x6b\xcc\x7a\x42\x4b\xa5\x29\x17\xd5\xdb\x35\xa3\x69\xda\xd1\xb8\xc6\x25\xb4\x75\xcb\xd6\x25\x64\xfb\x0c\xfd\x05\x66\xb0\x67\xb6\x4d\x96\x63\xf8\x8c\xaa\x58\xf1\x2a\x4f\x12\x8e\x1e\x5f\x2f\xdf\x5c\xc6\x17\xc9\x9b\x12\xa3\x5c\x24\x1a\x0a\x87\x5c\x8a\x27\xe5\x11\x26\x0c\x50\x0d\x28\x7c\x19\x6f\x1a\x79\x82\x26\x25\x0e\x19\x57\x97\x6a\x6e\x53\x77\xbe\x01\x30\xb4\x12\x42\xfa\xdd\x36\x91\x20\x25\x06\x11\x4d\xa6\x9a\x14\x6e\x7a\xd0\x16\x16\x91\x9b\x89\xe0\x70\xcf\x10\xff\x4e\x14\x3c\x59\x3e\xa1\xeb\x68\xb3\x6b\x18\x90\xd5\xc9\xa2\x47\xb6\x15\x05\xbf\xa8\xe1\xaf\xc9\xf3\x95\x56\x2b\x54\x2a\xb4\x51\xd4\xc2\xa5\x39\x3d\xe1\xa7\x48\x52\x0d\xb6\x30\xf5\xb8\x44\xcd\xd1\x9a\x38\xce\x5a\x9d\x92\x39\x2d\x45\xf9\x6c\xf3\xcc\x80\x1a\xc1\x62\x8e\xbb\xe8\xc3\x90\xdc\x28\xd5\x17\xf0\x11\x80\x5a\x9c\x2b\x45\xdb\x52\x63\x7a\xf3\x61\xd2\xa7\xf7\x20\xa6\xfc\x2a\x04\xfe\x50\x6b\xee\x6a\x9e\xb8\x85\x03\x35\xf5\x58\xcc\xb3\xb3\x23\xf4\xdc\x04\x85\x4c\x16\xd3\x70\xb1\xb5\xda\xdb\x35\x62\x47\xf3\x76\x1c\x1d\x70\xf3\x50\xdf\x10\x00\x5a\xde\x20\x18\xf4\xc6\xe2\x07\xec\xd6\xa0\xd3\xd2\x15\x82\x31\xfc\x84\xad\x36\xf2\xd8\x5c\xcd\xcb\x00\x72\xc1\x95\x94\x1a\x5f\x5a\x7c\x93\xae\x12\x1f\xcf\x66\xc7\x2a\xee\x0c\x62\x42\x5d\x60\x59\x6e\x46\x90\x03\x41\x0c\x5d\xdb\x64\x55\xa0\xba\xca\x30\xd5\x79\xde\x48\x26\x9b\x5c\x95\xbc\x91\x04\xff\xf8\x3c\x3a\xfd\xf7\xc9\xb3\xee\xbf\xc5\xdd\xf7\xd3\xce\xeb\xf0\xf5\xe9\xeb\xd3\x53\x61\x97\xaa\xaa\x16\x79\x44\xc3\xe9\xa9\x37\x52\x4c\xca\x13\xd7\xd2\xdf\xb2\xa6\x88\x17\xd4\xfe\x02\x17\xbf\xb1\x52\x99\x70\x80\xe1\x88\xe3\x63\xfe\x4b\x83\xce\xdd\x9d\xf7\xb6\xc8\xd6\xde\xc8\x92\x15\x09\x27\x6d\x23\x1d\x5c\xa2\x81\x82\xfc\x7f\x1c\x1f\xd3\x9f\x9f\x49\xae\x6d\x70\x4e\x96\xd2\x66\x59\x9e\x34\x80\xb1\x18\xa9\xb9\x43\x5d\xcd\xd2\x96\xe3\xc5\x06\xdd\x07\x16\xae\x50\x41\x8c\xe5\x60\x02\xf0\x49\x4b\xce\x62\x02\x29\xd3\x29\x5a\x3c\x62\x4f\x6f\x8c\x51\xc0\xbc\x37\x04\xcd\x42\x8c\x20\xe3\x09\xa6\x4c\x69\x14\x12\x58\x8e\xd8\x50\x56\xeb\xbc\xcd\xcd\xb4\x03\xc8\x4b\x4b\x72\x65\x85\x2e\xc5\xe7\xf0\xe3\xb8\x14\x67\x91\xde\x79\xbd\x93\x46\x0a\x4a\xc4\x1e\x8f\x7b\xd9\x98\x93\xab\x2a\xa1\x31\x42\x6d\x67\x63\xbd\x70\xd7\x7a\x1a\x6d\xe0\xd7\x37\xe1\x60\x84\x54\x1c\x1f\xab\x71\x45\x36\xc6\x57\xd9\x0a\x27\x24\xea\x80\x8a\xdc\x47\x7d\xb9\xc7\x2d\xdb\x17\x47\x7b\xc9\xbc\xe8\x76\x7d\x75\x39\x83\xe9\x19\x38\xc6\x5f\xc2\x9b\x65\xd9\x2a\x89\xd7\x1e\xc6\xc9\x6f\x41\x7b\xa7\x5f\xc3\x0f\x37\xad\x9e\xa3\x25\xc6\x1e\xca\x92\x3b\x5f\x38\x52\xb8\x88\xfe\xcf\x4f\xdf\x7f\xc7\x79\x65\x93\x38\x08\x8c\x5c\xc9\xf3\x5d\xdc\x86\xc0\x56\x8c\xe7\xf3\xa4\x28\x80\x65\x8d\xf4\x00\x86\x17\x90\xaf\xad\x50\x04\xcc\x8b\xeb\xe6\x7c\xc5\x05\xc3\x1c\x0d\x69\x6d\x45\x5b\xb9\xcb\x03\xb9\xcb\xfb\x73\x67\x9b\xec\x00\x03\xf4\x63\x38\x0a\x2e\x30\x33\x29\xdf\x1e\xf2\x5e\x41\x0a\x72\xe8\x68\x95\xce\xf2\x38\xaf\xf1\xd0\xf0\x88\xaf\xe8\xc0\xca\x02\x74\x38\x1a\x47\xca\x1d\x2e\xa2\x5a\x8e\x9e\x0b\xda\x99\x79\x96\xc0\xbc\x98\x27\x63\x04\xf2\xf1\xa7\xec\xc1\x2f\x3a\x9f\x9c\xb8\x10\xd3\x60\xac\x68\x53\x51\x7e\x49\x2a\xbf\x0a\x16\xa1\xb9\xeb\xe4\xbd\x58\x83\xc8\xa5\x8b\x23\x4d\x35\x63\xc5\x1b\x46\x2e\x5e\xa4\x3c\x08\x1b\x54\x5d\x26\xc5\xf9\x01\x92\x30\x19\xe8\x99\x68\x82\x30\x02\xa8\xd9\x03\x38\xfd\x30\xc2\x30\x13\x53\x65\x72\x7d\x9b\x16\x05\xcc\x4f\x26\x97\xa4\xfe\x28\x83\xd5\x33\xc0\xe3\x40\x03\xea\x55\x09\xda\xaa\x47\x1c\xb7\x5a\x1e\xba\xe0\xbd\xa2\x3f\x69\xb6\x9a\x6e\xaf\x05\xac\x59\xaf\x97\x88\xe4\xcd\x1b\xbc\x8a\xb5\xd0\xfa\x91\x1c\x8a\x75\x18\x8d\x66\xe5\xa7\x87\x62\x8f\x83\x7d\x4f\x75\x67\xf9\x97\x23\xa9\x9c\xc8\x1a\x6b\x29\xda\xd2\xf5\x78\x87\x66\xcf\x53\x5c\x6a\xa4\x73\x75\x88\xe6\xbc\xae\xb4\xda\x15\xa8\x9f\x20\x7e\xda\xbd\xf0\x59\x9a\xac\x16\x2d\xfe\x85\x11\x8e\x95\x76\x86\x80\x91\x0f\xfb\xaf\x62\x03\x26\x6f\x79\x7d\xcc\xe9\x96\x48\x13\x49\x3d\x19\x6d\x94\x1f\x0c\xa3\xf2\x6b\x34\x20\x86\xd0\x1b\x50\x06\x42\xe9\x98\x6a\x28\xc1\x93\x12\x80\xd8\xaa\x3f\x8a\x2d\xf3\x6c\x53\x37\x57\x6b\x67\x39\x68\xf9\xf6\x30\xe4\x21\x28\x48\x81\xa6\x53\xc4\xf2\x49\x67\xca\xee\x2a\x6b\x0e\xcf\xcc\xe3\x13\x78\x71\x0e\x11\x43\xac\x90\x3b\x1a\x71\x71\x77\xc7\x28\x9d\x29\x64\x1f\x3f\x17\x13\xce\x07\x7c\x9c\x46\x80\x6e\xa2\xd9\x6a\x44\xcb\x3c\x77\x87\x94\xf5\xa8\x26\x2e\xff\x0c\xef\x9a\xb5\xe4\x64\xaa\xd2\x56\x70\xf6\xdf\xd0\x00\x36\x9e\xf8\x20\x8e\xf8\xbe\xfc\x56\x6b\xe1\xb3\x5e\xad\x42\x4c\x29\xd6\x20\x9e\xd1\x14\x0d\x43\xd8\xd5\x8a\x84\x15\x53\x86\x51\x56\x9e\xe3\x82\x8a\xba\xab\x3b\x57\x37\x2d\x00\xe4\x91\xf3\x48\x4d\xc9\xb0\x18\x48\xe3\xe5\xf1\x71\x0a\xeb\xb6\xb7\xe8\x2c\x1d\xa7\xd3\x6c\x92\x5d\x95\x9b\xab\x92\x4e\x18\xa6\x7a\x7a\xd7\xd1\x74\xa8\xc1\xf1\x43\x3c\x83\x66\x72\xb8\x4f\x64\xa6\x27\x70\x34\xf5\x04\x22\x90\x45\x8b\x4b\x8c\x1e\x0d\x04\x63\x53\x47\xc0\xb1\x3e\xee\x9d\x79\x7a\x46\xa6\xea\x4f\xbc\x9b\xb4\x3c\xf7\xac\x5b\x57\xa6\xb5\x19\x31\x28\x07\x08\xb8\xe5\x3f\x3d\x2c\x20\x5b\xc7\xd6\x9a\xcf\x6c\x29\xa8\xe2\x2b\x37\x8b\xa4\xc5\x6c\xa0\xc4\x1b\x7b\x8d\x48\x9a\x1c\xc3\x04\xbb\x77\xc4\x50\xa3\x57\x1e\x64\x20\xe3\xe4\x02\x9f\x72\x65\x68\xf8\x96\x51\xb8\x53\x6d\x17\x69\x5a\xff\x2c\x9e\xbb\x7e\x5e\x8c\x85\x23\xa6\xd8\xad\x3f\x7b\x2b\x2e\x70\x80\x11\x17\xb0\x0c\xe1\xc7\x5f\xc4\x85\xb8\xb8\x56\x12\x71\x0b\x84\x86\x9e\x27\x90\xde\x70\x22\xcd\xfe\xf1\x6b\x2b\x80\x0a\x5a\x18\xc7\xa0\x9b\xca\x06\xe3\xd4\x91\x74\x96\x8a\x79\x5c\x3f\x8a\xe8\x9c\x56\x5e\x53\xc3\xd9\xad\x89\x9c\x97\xa8\x7d\x55\x08\xf4\xb5\x72\xcc\x0b\x31\x5a\xad\xf3\xa2\x18\x4f\x17\x20\xe1\xf8\x18\x7f\x11\x8d\x2c\x35\x90\x24\x3f\x54\xcb\xbc\x00\x90\x0b\x35\x44\x63\xed\x23\xcf\x1b\x5d\x7c\x61\x11\x0d\xb0\x17\xc1\xc5\x35\xd5\x03\xf8\xed\x4b\xb1\x0d\x38\x1b\x57\xe3\xe2\x3a\xa0\xcc\x9d\xc8\xbf\x78\xda\x1f\x7b\x77\x6c\xcc\x29\x8d\xee\x20\x75\x84\x15\xa5\x06\x03\xa8\xa9\x3c\x5a\x40\xdb\xc7\xd9\xdb\xe3\x63\x97\x1d\x1a\x8a\x19\x43\x2d\x42\xac\xe1\x16\x62\xee\x50\x16\xa6\xcf\x6a\x0c\x9a\x78\x25\x23\x18\xa1\x22\x76\x8b\x5e\x5a\x81\x53\x81\xcd\x16\x0b\x89\xc5\x16\x62\x28\x26\xe1\x46\x89\x64\x6a\x4b\xd7\x43\x92\x90\x67\x08\xee\x5c\x34\x27\xe1\x22\xc9\x8e\x6c\x49\x96\xa9\x14\xaf\xc7\xaf\x8b\x96\x89\x87\xf2\x6f\xf9\x4f\x8f\xb0\x9b\xb5\xb7\x59\x72\x63\xbc\x5e\xa0\x23\xc5\x41\x6f\x06\x2b\x6f\x37\xbb\xd5\x17\xd2\x55\xb9\x67\xab\x9d\x93\xdc\x09\x1d\xbd\x60\x8e\xad\x8c\xec\x4e\x93\x05\x8f\x94\x4c\xdc\x72\x71\x2a\x4d\x60\xbd\xd2\xd9\x8a\x71\x76\x1f\x28\x89\x96\x5c\xdf\x38\xe9\x63\xfc\x45\x45\x8b\xb4\x91\x5e\x52\x6d\x72\xb9\x0d\xc1\x38\xb7\xfa\xaf\xae\xcb\x2a\x2e\xcb\x64\xdd\x5e\x19\x4e\x73\x6b\x83\x91\xac\x73\xe1\x3e\xa7\x03\xcb\xd1\x2b\x9a\x5f\xd0\xeb\x19\xe2\x97\x6d\x6e\xdf\xf6\x93\x5d\x48\x4a\x56\xe3\x2c\x48\xe3\x31\x9b\x37\x8c\x71\x44\xeb\x7e\xd3\x4b\x9c\xa5\x01\xe9\x33\x16\x81\x40\xb5\x55\xc5\x6c\xd5\xbe\x7e\xc7\x04\x6b\xe0\x62\x6f\x4c\x30\x73\xad\x65\x2f\x40\x89\xbc\x6c\xa8\x04\xe6\x28\xcb\x1e\x07\xec\x99\x8b\x0e\x41\xac\xd5\xa6\x36\x94\x92\xd7\x53\x6e\x75\xa7\x33\xd5\xa7\x29\x92\x8e\x4a\x88\xe4\xed\x48\x4f\x63\x20\xe2\x53\x49\x02\xce\x62\x3a\x9e\xb2\x4f\x23\x25\xe9\x98\xe8\xf3\x41\x96\xdd\x63\x33\xed\x7c\x86\x63\x5d\x8d\x4f\xe8\xdc\x18\x50\x33\x1d\x0d\xa0\xf0\xe3\x31\xa6\xd0\xa3\x3f\x9e\xb5\x00\x5d\x2d\x76\xb5\x93\xfd\x5a\x49\x60\x15\x23\x79\xb9\xdb\x4d\x01\xe7\x96\x7e\xff\xa4\x49\xce\x14\x68\xcb\x49\x3e\x4f\xf6\x08\x0a\xa4\xb0\x74\x2f\x66\x6a\xbf\x6e\x15\xd7\x80\x99\x6d\xc7\xec\xdd\x3d\x7b\x53\x8f\xf7\xa6\xa5\x6e\x84\x86\x43\x17\xc5\xf1\xf1\x62\x36\xa1\x2f\xa0\x4d\xe2\xa0\xa0\xaf\xe3\x01\x52\xe2\x46\x8b\x33\x9f\x6d\xf0\x64\xef\xc1\xa3\x36\x72\x5a\xb4\x2e\x94\xaa\xcf\xb0\x25\xde\x7d\x36\x1d\x42\x21\xc8\x36\xbe\xdc\x15\xa2\xe5\x03\x45\xe3\x1a\x5b\x02\x70\xcd\x24\x69\xac\xfb\xe0\xe7\xd7\xd0\x9f\xe2\x35\x70\x63\x48\x46\x54\x17\x30\xf7\xf0\x73\xca\x03\x31\x3f\x47\x45\x38\xea\x3e\x26\x5f\x6b\x92\x94\xe8\x49\x9f\xf7\x26\xa3\x89\xb2\x33\x51\x86\x24\x53\x41\x8b\x39\xbc\x20\x7f\x96\xa7\xc4\x4f\x34\x13\x39\x4f\x40\xab\x17\xde\x32\x8f\xaf\xc9\x7e\x8d\x1f\x21\xf1\xec\x85\xf5\x3c\x61\xc6\xb8\x4b\xbd\x9f\x12\x3a\x2a\xd0\x32\xc1\x20\x56\x8a\xa0\xac\x5b\xfe\xd3\xdc\x6f\x7d\xef\xee\xb3\x16\xef\x5d\x68\xa7\xee\x7b\x74\x77\x07\xc6\xe3\x2d\x1c\xb3\xd3\x4a\xb0\x63\xfa\x0d\x5b\x14\x7c\xab\xc1\x15\x0e\xdf\x0e\x04\xbb\xf4\x68\xde\x1f\xa0\x47\xc3\xfc\x06\x7a\x14\x0e\xdf\x0e\x34\xe8\x91\x8d\xbf\x87\x12\x99\xfa\x2b\x69\xe0\xdc\x3e\xff\x69\x94\x6b\x49\x9a\x6d\x85\x90\xdb\xfe\x7f\x14\x00\x7e\xaa\xdc\x2c\x7a\xbb\xe3\x04\x1f\x2b\x13\x90\x3c\xe6\x6a\xd1\xc2\x99\x2e\x4e\xc7\xb3\x2f\xc6\xb9\xd5\xa4\x59\xe3\xc6\xe5\xd5\x2a\x6e\xb1\x46\x6a\xae\x6d\xb1\xbe\xdc\xc9\x70\x66\x8e\x2c\x13\x1b\xa1\x90\xec\x5d\xcf\xc2\xcc\xb4\xda\x59\xfd\x91\x59\x3a\x7d\x4d\x23\x44\x29\x2d\x17\xb0\x04\x9d\x01\x37\xed\x9c\xcd\x3f\x28\x86\xc7\xef\xa8\xc1\x0f\x1b\x33\x9d\xb6\x2b\xaa\xb6\xea\x83\x34\x8a\x3d\x3a\x89\xae\xd2\xc3\x75\x92\x06\xf6\xad\xbd\x67\xb2\x4c\xb2\xd6\xb1\x18\xe2\x7d\xf5\x50\x69\x49\xfb\xaa\x79\xf6\x96\xf7\x5f\x23\xef\x32\xc9\xe7\x71\xa9\xcc\x71\x70\x9c\x03\xf0\x89\x81\x98\xe2\x50\x17\x97\x0e\xe7\x57\x90\xd1\x0e\xab\x79\xb0\x0a\xf1\x92\x10\xcc\x85\x68\xed\x8a\x0b\x93\x7b\xb6\x1a\x56\xab\x68\xb2\xa2\xa0\x20\x1d\x26\x98\x8a\x8a\xb7\x37\xfc\xd5\xca\x6c\x62\xaa\x09\xbc\x9a\x46\x55\x4d\xa7\x74\x2a\xa6\xa6\x98\xc1\xd4\x6e\x2f\x28\xab\x47\x55\x69\x17\x2e\x22\xac\x67\x71\xc0\x9c\xcc\x93\x0a\x66\x52\x60\x42\xdc\x90\x8b\x10\x05\xb9\xd9\xcf\x22\x3d\xdf\xa0\x0c\x1c\xa5\x6b\xe4\x74\x80\xd0\xb2\x2f\x40\x90\xbf\xb4\x76\x01\x54\xdc\xd7\xd9\x70\xee\xdd\xdb\xd5\x34\xc6\xa8\x12\x56\x41\x15\x9a\xe5\x6c\xb0\xcb\x61\x35\x57\x76\xfd\x1a\x63\x20\xb7\x61\x73\x64\x21\xb2\x28\x6b\x5c\xee\xcd\xca\xe2\xb0\x37\xeb\x9f\xa3\x8f\xa8\xe2\x94\x0f\x78\x9e\x30\xe5\xb5\x4b\xe1\x91\x7b\x42\xdb\xba\x52\xdf\x1c\x83\x8f\x4d\x9e\xcc\xd3\x82\xe7\x55\x74\xc0\x49\x6f\xed\x9b\x99\x14\x9a\xb7\x27\x31\xf2\x1f\x01\x31\x8d\x9e\x46\x16\xf2\x4d\xe3\x48\x19\xcd\x12\x42\x37\x89\x61\x5d\x64\x04\xc5\x97\x09\xaa\xcf\xa1\xdf\xcf\xf6\xbe\x08\x50\xb4\x65\xeb\xf4\x32\xee\xbc\xaa\xb3\x91\xd5\x34\xdb\x45\xdf\xdb\xd9\x9a\x3d\x09\x73\x4d\x23\x49\xae\x2f\x0b\x93\xa7\x15\x4e\x47\x6a\xed\x27\x10\xd7\xd2\x55\x28\xd6\xee\x2d\xf0\x49\x27\x02\x9b\x87\xf7\x98\xcd\x4e\x77\x19\x49\x22\x6d\x2e\x6e\xdc\x4e\x64\x35\x4a\xef\x77\xea\x50\x2d\x85\x06\xad\x1d\x8c\x1d\x1c\xec\xe9\x27\x56\x23\xfe\x67\xe9\x2b\xe6\x18\x1b\x34\x96\xd6\xc9\x82\x6e\x84\xc8\xf1\xb9\x88\x8a\xec\x8a\xf4\xf8\x40\x94\xd2\x82\x0d\xbe\xa5\x80\x78\xdf\x7a\x9d\xa2\xc7\xb7\x0e\x8b\x5e\xdd\xf1\x5e\x7a\x9d\x52\x86\x61\x68\x36\x0b\xdd\xf9\x55\x7e\x7d\x1f\x4e\xbc\xf3\x0a\x99\xbb\x80\x10\xef\xbc\x42\x7e\xf8\xac\x45\x0a\xb1\xec\x46\xe0\xc4\x5f\x54\x74\xc3\x34\xad\xad\xa8\xba\xbb\xa8\xf6\xd2\xf3\x95\xd7\xf1\x31\x22\xad\x02\x8a\xf4\x31\x36\xad\x03\xba\x36\xea\x23\xa9\x14\x10\x14\xa8\xbb\x04\x76\xb4\xa7\x0e\x6a\xef\xf1\x9f\xf7\xd5\xe3\x32\xa2\x32\x01\x49\x70\x3a\x3c\x40\xd9\xa5\x09\x1d\xa9\x50\x29\x43\x07\x29\xf8\xe5\x41\x14\xd4\x98\xf9\x30\x05\x2a\x7c\xe9\x94\x79\xb9\x9f\x02\x94\x19\x6b\xf1\x4f\x43\x1d\x5d\x07\x2a\x26\xf4\x67\x3a\xda\x67\xd3\xa1\xc6\xa1\xc6\x00\x44\xa3\xcf\x02\x05\x54\x2f\xb7\x08\x51\x24\xaf\x6f\x70\xfd\xdc\xb3\x0d\x8e\xf3\xf4\xa9\x84\x93\xc8\x71\x98\xc8\x12\x12\xf5\x86\xed\xe3\x25\x93\x1d\xdd\x9a\x3b\x35\x24\xa4\x21\xfd\x0a\xc5\xea\x50\xb7\xba\x8e\xfa\x67\x4b\xdc\x2f\x3a\xee\x97\xed\xc8\x5a\x62\xb0\x2b\x19\xd3\xcb\x60\xb0\x30\x6b\x29\x4a\x82\x18\x5a\x2c\xd2\x2b\x1c\x17\xca\x63\x46\x7b\x0e\x95\xd8\xc8\x23\x79\xb4\x67\x08\x6a\xe1\xa0\x5a\x42\x58\xc5\x32\x27\xf7\xa0\x68\xe1\xf3\x2e\x8a\x3f\x67\x18\x33\x05\x9a\x81\x6c\x93\xb6\x6f\x40\x40\xbc\x25\xab\x72\x77\xaa\xf5\xa0\x90\xc9\x64\xef\x34\x2c\x8d\x71\x44\xdb\x08\xe2\x22\xf2\x93\xf5\xa2\x4b\x81\xe0\x14\x0d\xe9\xae\x2e\x7d\xb5\xcf\x4d\x7b\x52\x38\xb5\x81\x82\xb1\xe4\xb2\xd4\x36\xfb\xc8\xda\x25\x3d\x3e\x26\x40\xde\x24\xb5\xac\x21\x67\xee\x53\x71\xc5\x24\x9e\x76\xe5\xd7\x0c\x77\x92\x39\xd7\x0e\xaf\xe4\x65\xcf\xc5\x48\x6e\x97\xc9\x19\x68\x11\x99\xbd\x3f\x99\xa2\xba\x1a\x51\x4f\x0a\xce\x34\x8a\x9b\x89\x50\x3d\x95\xd4\x89\x16\x27\x17\x76\x57\xcc\xd6\x49\x8b\x0e\x3d\xd8\x2a\xa7\x43\xd0\x69\x08\x77\xd4\x17\x80\x26\x1a\xaa\x77\x71\x78\x4b\xda\x3a\x08\x32\x14\xc8\x1b\x81\xac\x6e\x09\x55\x38\xdd\x04\xe4\xb8\xad\x71\x40\x95\x34\x36\x9f\x67\xee\xe6\xf3\x4c\x00\x08\x3e\x34\x92\x1c\x9e\x84\x65\x22\xad\xa7\x77\x65\x38\xc4\x5a\x68\x44\x86\x38\x6b\xff\x43\xc7\x99\xf2\xa9\xd6\xb2\xb6\x27\x06\x80\x1e\x46\xd2\xc8\x54\x9d\x2c\x2b\x65\x19\x63\xd6\x7f\xc0\x36\x85\x46\x25\xba\x48\xfe\x9c\x7e\xa5\x8a\xb3\x6c\xee\xf0\xa8\xa7\xdd\x32\x9e\x0e\x81\xb8\x67\xc9\x1d\x6b\x1c\x1b\xae\x8d\xc9\x54\x20\xbc\xcb\x74\x8d\x17\x7e\x67\x75\x40\xe9\x64\xc6\x34\x41\x93\xea\x74\xfd\x02\xe5\x5a\x5a\x20\x50\x13\x4c\xa7\xbc\x4f\x0d\xc8\xbc\xcb\xb8\x6a\xcf\x17\x57\x87\xf3\x25\x30\x02\xab\x22\x6f\xf5\xa9\xbf\x7d\x6a\x75\xa0\x03\xc6\x2c\x1e\xf8\x6c\xd5\x4c\x7e\xce\xe0\xf3\x69\x3c\xee\x0e\xc2\xf8\xe9\x6c\x3c\x08\xfb\xa0\x16\x5b\x24\xfd\xc7\x7f\xf8\xb4\xad\xce\xfd\xfd\x74\x08\xc4\x90\x91\x1d\x6f\x8a\x83\x2a\xa3\xf7\x72\x81\x1e\xce\x68\xd1\x02\x00\xb8\x8a\xc5\x3f\x03\xb3\x20\xd4\xfd\x6e\x56\xf3\x86\x07\x6f\x2a\xb4\x1c\xec\xeb\xe1\x05\xb1\xf5\x66\x75\x9b\x36\x0b\xb1\x95\x20\x80\x2d\xff\x69\xee\x5f\x34\xb6\x9f\xdd\xf1\x3d\x70\xb2\x5a\x62\xe1\xf4\x48\x5b\x2a\x90\xb9\x2c\x14\x4a\x22\x94\x35\x26\xc5\x8e\x95\x09\xac\x3a\xa4\xa0\x1b\xe5\xe6\xf4\x08\xd9\xd4\x72\xe0\xd1\x38\x2b\x50\x63\x1e\x0f\xbd\xe9\x68\x0f\xe7\xec\xe1\xd7\x61\x4f\x61\xb3\x87\xcc\x2d\x2f\x71\xc1\x80\x2b\xb1\x82\xcf\x33\xb7\xf2\x57\x57\xb8\x8c\xe7\xed\x7a\x32\xa5\xd8\x67\xb8\x57\xe5\x9b\xba\x1f\xa9\x8d\x89\xbe\xa0\x88\x81\x8e\xa0\xe1\xf0\xcd\xbc\x56\x11\xf3\x5a\x14\x49\x9e\xc2\xa4\x44\x98\x0a\xf7\x74\x86\x5e\xb9\xa4\x64\xed\x48\x0e\x19\xe8\xcb\x8d\x3f\xc7\xe5\xf5\x42\x40\x69\x35\xfa\x62\x58\xa0\x89\x31\x6e\x70\xf1\xd7\x84\x69\x9a\x46\x35\x5a\xed\x58\x51\x03\x88\xea\x34\x22\xe7\x35\xc2\x75\x6a\x10\xe6\x6d\xe0\x73\xd9\x3c\x19\xd8\xce\x9b\x2c\x52\xd9\x81\xa5\x7e\x7d\xf7\x42\x6c\x44\xa5\x4e\x81\x9a\x03\x02\xb9\x70\xe0\xe3\x4a\xae\xf3\x64\xaa\x8e\xa7\x9d\x43\x1d\x75\x40\xcd\x50\xd6\xc9\x51\xfb\xc9\x17\x9d\x20\xdb\xc3\x8f\x9e\x04\xf5\x83\xb8\xcd\xab\xf7\x4c\x89\x74\x4a\x59\x85\x14\xf4\xe3\xc9\x5b\xc8\x51\x87\xf2\x5d\x0e\x0e\xbe\x0f\x53\x41\xec\x09\x31\x0c\xfd\x5f\xb6\x07\x65\xa5\x83\x3c\x28\x5c\xa2\x3b\x30\xb6\xf4\xaa\x2f\x66\xbd\x8a\x46\x92\x5e\xf5\x14\x3f\xf1\xeb\x3d\x44\xbe\xe7\xc8\xf7\x4f\xf1\x93\x86\x18\x51\x45\x8c\x10\xfd\x66\x54\x64\xb1\x84\xd5\xb8\x20\x16\xc8\xa2\x34\x0b\x2e\xe8\x60\x9c\xd6\xe1\x2a\xd7\xc5\x54\x6c\x7a\x15\x8c\xf6\x15\xb3\x66\x94\x7e\xe1\x08\xd1\x28\xe0\xe0\x24\xed\x74\xa6\x9a\x07\x6f\xa1\xe6\x38\xbe\x41\xe1\xaa\xd0\x4e\x34\xa0\xea\x8d\x36\x40\x5e\x7a\x38\x1b\x64\x8a\xde\x8a\x1d\x88\x0d\x33\xe8\xc3\x48\x70\x9a\xbf\xfd\xf8\x85\xfb\x9d\x14\x29\x77\x74\xa4\x61\x01\x94\x7f\x79\xcd\xb7\xcd\x24\x4a\xef\x13\xa9\x3b\x8b\x5e\x96\x2f\x92\xdc\x33\x67\x79\x75\x3f\x24\x27\x66\xf5\x00\xb7\x31\xc5\xbc\x0e\xbd\x79\x6d\xf4\x12\x2a\xbe\xc7\x85\xef\xd1\x38\x76\x29\x93\xfa\x32\xe5\xdd\xf2\x1f\xf5\x4a\xd4\x1e\x1c\x2d\x55\x70\x91\x1c\xde\xe8\x20\xa0\x87\x6f\xd3\x33\xce\xad\xfc\x38\xa8\x74\x40\x43\xfc\x4e\x4a\x87\xcd\x0d\x67\xc0\x2d\xdb\x4d\xa6\xe4\x85\x22\xeb\xf0\x99\x0c\x65\xae\xf1\x3a\x26\x28\x05\xa0\x5a\x44\x9d\xc1\x29\x74\xfa\xb8\x8a\xba\xf8\x01\xca\x3a\x8e\x01\x49\xbc\x86\x3f\xdf\x0e\xd1\xe8\x02\x95\x02\xff\x41\x73\x93\x34\x8c\xa0\x37\x9d\x1d\x5b\x16\x6b\xde\xb7\x8c\x36\xae\x29\x1e\xb5\x6f\xa0\xe3\xe9\x35\x54\x19\xe9\xb9\x0e\xc4\xf5\x53\xa0\x07\x83\x40\x15\x04\x81\xa6\x4e\x24\x49\x8e\xae\xbb\x48\x1c\x51\x08\xca\x37\x46\x9d\xfa\x69\x67\x10\x00\xb1\x32\x7c\xe2\x33\x4c\x30\xfa\x76\x78\x1a\x41\x71\x5d\xf9\xb6\xed\xee\x04\x7b\xbb\xa5\x2a\xe0\xb0\xca\xda\x10\x3e\xa6\x8f\xb4\xb2\x3d\x09\x59\x4e\xcf\x2f\x37\x78\xdf\xc3\xaa\xc2\xd3\xa7\x03\x73\x40\xce\xf9\xa6\x76\xfa\xbf\x0c\xc7\xb2\x62\xa1\xcf\x1f\xdd\xc1\xb4\x23\xa3\x02\x98\x2b\x4c\xee\x79\x76\xb5\x2e\xa7\x48\xa4\x85\x31\x05\x74\xf0\x63\xc5\xc4\x15\x0a\x41\x65\x62\x80\x25\xd3\x08\x7e\x6c\x3a\x90\x0a\xe2\x8d\x75\x78\x8f\xae\xdc\xe6\x20\xcf\xdf\x0e\xad\xbc\xe5\x22\xb9\x9e\x46\xe6\xc1\xce\x6f\x81\xa2\xb8\x28\xd0\x91\x18\x36\x1a\x35\xf8\xbd\xed\x2d\xae\xc9\x82\xaa\xa5\x22\xd7\x3b\x15\xb9\xde\xa9\xc8\xf5\x4e\x45\xae\x5b\x2a\x72\xdd\x5a\x91\xeb\x83\x15\xd1\xed\x78\xdd\x6c\xa1\x66\x93\x05\xe2\xb0\x78\xca\x06\x93\xbd\xeb\x7a\xf4\xbf\xd4\x9e\xa7\x33\xb5\x97\x85\x63\xb7\xb2\xdb\x51\xac\xeb\x08\x3b\x5d\x85\xbb\x27\x1b\xc8\xf2\x9a\xab\x6d\xfc\xe5\xb6\xc1\xc5\x21\x53\x6e\x2f\x13\x89\xef\x30\xda\xe2\x1f\x0f\xfb\x51\x48\xeb\x08\xec\xd0\x21\xad\x0c\xb0\xf7\xc0\x0a\xf2\xea\xd2\xa3\x3e\x03\x91\x78\x03\x40\x28\x8e\xa2\xd9\x05\x7f\xa1\x8b\x16\xe0\x27\xae\x36\xe1\x8f\x27\xcb\x0a\xd5\x32\xc1\x1e\xcc\x61\x46\x95\x94\x98\x85\x66\x66\xdd\xdf\x90\x89\x18\x77\x77\x07\xc4\x52\x96\x2d\xff\x79\xc8\xc2\xb3\x7d\x04\x37\x28\x24\x43\x5a\x56\xb9\x32\x65\xe6\xc2\xff\x39\x0b\x42\xbb\x48\x33\x36\xa3\x0d\x38\x1e\x34\xb4\x8d\xce\x32\xad\x69\xf1\x41\x8b\x81\x87\x99\x7d\xf0\xfe\x0b\x8f\xb8\xd2\x06\xa4\x79\x8b\x8f\x72\x72\x12\x1b\x83\xf1\x3b\x56\xca\xa6\xcd\xae\x3c\x75\x06\xdb\x8a\xf0\xbe\xad\x49\xb5\xfa\x80\x3c\xa8\xc5\xd2\xb5\x38\xd3\x81\x20\x96\x0d\x10\x61\x84\x87\xbf\xe8\xc4\x59\xb2\x2d\xa9\x91\x89\x90\x23\x5a\x50\x40\x79\x07\x50\xa1\x89\x05\x48\x6e\x91\x9d\xed\xce\x1d\xf5\x46\xf1\x31\xd0\x56\xf5\xad\x07\x39\xb2\xfb\x6d\xa5\xed\x52\x6b\x5f\xbb\xd7\xb8\x84\x9d\x9e\x09\xe8\x25\xe9\xfc\x02\x0d\x4a\xc8\x1e\x00\x8f\xc7\xf8\x46\xb2\xa5\x15\x59\x27\xbb\x62\x01\x01\x89\x74\x01\x31\x12\xaf\xee\x54\xb2\x06\xf7\x9b\x91\x48\xc0\xad\xfa\xf8\x95\xdd\x49\xa3\x39\xac\x12\x49\xb0\x87\x2b\x45\x0a\xef\x56\x7f\xfe\x49\x9d\xaf\xc9\x17\xbb\x03\x02\xb2\xb8\x4c\x9a\xe7\x8e\x87\xc6\xdb\xc8\x53\xb9\x3c\xd0\x99\x8b\x94\x4f\x1a\xd9\x0d\xad\x48\x56\xab\x74\x53\xa4\x00\xd4\xeb\xf5\x3c\x71\x03\xca\xf0\x97\x79\x12\x5f\x44\xe4\x09\xea\x32\x2d\xa3\x41\xbf\x2f\x74\xb1\xf7\x1c\xe7\x93\x3f\x1a\x80\x51\xf0\xbe\xda\x52\x65\x5c\xba\x78\x53\x8e\x26\xc0\x3a\xec\x8f\x8b\x29\xf9\x35\xb5\x8f\x22\x15\xc6\x5f\x2d\x22\x9c\x7d\xab\xbf\x76\x9a\xd2\xc5\x14\x4b\xbb\xbb\xb6\x9c\xcc\x17\x9d\x71\x85\x97\x73\xf5\xa6\x3f\x26\x75\x50\x73\x68\xc9\xa8\xb9\x6f\x4e\x52\xb3\xc2\x5a\x35\xc8\x54\xf8\x68\xcb\xad\x9b\xca\xda\x99\x34\xc6\x3c\x3a\x15\x8d\xa0\x3d\xaf\x0d\x01\x32\x7d\x46\x8d\xdb\x32\xd9\x58\x2d\xff\x68\xd6\x92\xdb\xc8\xe0\xd5\x3a\x7d\x77\xd5\xbe\x35\xc8\x49\x3b\x17\xb1\xad\x68\xb9\x75\xe7\x6e\xc1\x57\xda\xcc\x53\x8f\x21\x19\x49\x01\x28\x89\xf6\x8d\x1b\xde\x15\x43\x89\xa6\xa0\xbe\x15\xca\xf8\x3f\x70\x7f\x8b\x33\x6d\xe5\x5f\xe7\xca\x47\x65\x8b\x41\xd5\x80\x34\x8c\xe0\x47\x99\x5b\x19\x01\x49\x1f\xbc\x19\x36\x52\x4e\x23\xe7\x57\x79\x2e\x80\xed\xb4\x5d\xe2\xec\x83\x75\x71\xe4\xb4\xae\x7e\x44\xe9\x28\xed\x40\x8b\x27\x1b\x9e\xbe\xe8\x9e\x26\xe4\xc6\xdd\x15\x04\x95\x5b\x1e\x18\x65\x5b\xa3\x77\xde\xe2\x83\x9b\x50\x80\x5c\x7e\xa3\xf5\x6a\xaa\xac\xd8\x11\x78\xab\xf6\x24\x6f\x65\x24\x02\xf3\xe5\x4c\x1a\xbb\x87\x02\xcb\x8c\x06\xaa\x01\xa0\xba\x8d\xd1\x7e\xed\x0e\xf6\xd0\xe3\xd3\xf5\x56\x10\x1c\xe6\x6c\x83\xc3\x78\x03\x67\xb1\x19\x24\x73\xbe\xca\xae\xda\xed\x91\x29\xa5\xa9\x74\x18\x23\xd4\x74\x9d\xe2\xbd\xbc\x78\x29\x6f\xf0\xe2\xe4\xbe\xa0\x2d\xab\x0b\x9e\x1e\x6d\xd5\x84\xce\xa0\xf1\x6b\xd2\x9f\x9e\x0e\xf1\x14\x9a\x02\x03\x0c\xb4\xaa\x18\x97\x8e\x82\x41\x77\x66\xf5\x86\x66\xeb\x66\x90\x72\x17\x0d\x1d\xaf\xab\xbf\x63\x1c\xf3\x79\x97\x0b\x09\x6d\xb3\x8f\x36\x57\x71\x2f\xc8\x38\x9a\xb7\x7e\xb4\x4e\x72\xb1\xa3\x66\x20\x22\x3c\xe5\x21\x15\xc4\xa3\x3d\x75\xf8\x24\xd3\x37\x6d\x4b\x05\x4b\x4c\x98\xe5\x60\x42\xdf\x4d\xac\x31\xb1\xd6\x3b\xd3\x0f\xd3\xdf\xc8\xa5\x35\x79\xd6\xd3\x6e\xb1\x7d\xed\x1f\xbb\x27\x9d\x36\xf3\x9f\x80\x5a\x55\xae\x36\x7a\xc0\x24\x72\x05\x29\xb8\xb9\x02\x65\xc3\x7b\x40\x4b\xe2\x66\x97\x5e\x28\x27\x9f\xc3\x1c\xf5\xa4\xdf\x9f\xb2\xa7\xbf\x0f\xd2\x85\x14\x81\x6d\x07\x5d\x1f\x6f\x05\x93\xdb\x92\xd8\xdf\x5a\x36\xba\xe4\xe4\xd9\xd3\x3e\xc9\xe5\xf7\x3f\xa4\x57\xb8\x7b\xb4\x29\x72\x53\xe0\x29\x3a\xd8\x59\x74\x28\x71\x72\xf1\xa1\x17\xbb\x67\x63\x54\xfb\x9e\xeb\xfd\xad\x71\x66\xdb\xe4\x83\xba\x2c\x87\x39\xb7\xfc\xe7\x7e\x05\xcd\x81\xde\xe5\x54\xe3\x7e\x9e\x4a\xbf\xaf\xd8\x26\x4f\xe5\xf0\x7f\x3d\x32\x33\x46\x5a\x7c\x4f\x77\x62\x21\xed\xf8\xf8\x11\x8d\x9c\x7a\x04\xad\x82\xb1\x56\xa3\x2a\x3c\x7b\x5d\x64\x97\x63\xff\x3a\x82\xff\xab\x60\x60\xde\xa5\xbe\x0e\x42\x3c\x6a\xd9\xdb\x8e\xd7\x78\x94\x43\xab\x6e\xce\xea\x07\x27\xd7\x6a\xb4\x1d\x24\xdd\x27\xc1\xd4\x7e\x46\xbc\xea\xc5\x78\x15\x64\x0d\xb8\x50\xe7\xe7\x32\x75\xd4\xfd\xc5\xc2\xd0\x63\x95\x9c\xfe\x8b\x2a\x0a\x0b\x91\xb0\xfb\xb9\x77\x58\xbd\x25\xa0\x87\x2b\xb7\x8c\x73\x2b\x3f\xfe\xac\x6b\x72\x96\x18\x98\x01\xfe\x7d\xda\xbe\xa2\x84\x78\x35\xb0\x4b\x09\x61\x03\xdd\x04\xfd\xa5\x6d\xc4\x7b\x3e\x55\x9b\x4d\xde\x4f\xc5\x7b\x5c\xec\xbd\xb7\x4f\x8c\x70\xd4\xc4\xdb\x85\xff\x1f\x0d\x6c\xf2\x8e\xdc\xfb\xf6\xc5\x5f\xc1\x86\x61\x12\x1c\x82\xd3\x08\x16\x56\x66\x4b\xc5\xc6\x6b\x5d\x6f\xd5\x63\x32\xab\xac\x80\x62\x2c\x5f\x5f\xbf\xe6\x3a\x03\xae\x45\x30\x05\xa9\x40\x87\x7c\xf1\xd5\xaa\xa4\xab\xce\xe1\x7b\xce\xf7\x2f\x48\xf4\x74\xe7\x08\xeb\xbd\xd1\x72\x80\x01\x74\x29\xaf\x75\x40\xb3\x51\x46\xcc\x7e\x21\x6b\xe0\x9c\x81\x01\x92\x43\xf7\x2c\xdf\x47\x0b\x01\x20\x5b\x41\x70\x12\x69\x3b\xa8\x53\xa2\xc9\xe5\xa8\x4e\x8e\x51\x34\x5e\x3a\x46\x8b\x68\x0d\x89\x75\xb1\xa5\xc6\x5c\x49\x73\xaa\x78\x11\x98\x2c\xaa\x4a\x2d\xd9\x54\xd2\x9e\xac\x24\x60\x7c\xcf\xfc\xd6\x7c\xf7\xc8\xcb\x75\x9b\xb8\x61\x82\x5f\x6c\x92\x39\x79\x75\x16\x64\x3e\x0a\x92\x41\x51\xbc\xcb\xb5\xbb\x6b\x90\x9c\x29\x47\x3f\xe4\x09\x9a\xbe\xa7\x91\xf5\x4d\x0f\x41\xb0\x4f\x3b\x74\x0e\x8d\xbe\x9a\xf1\x8f\xce\x28\x2c\x58\x55\xe4\xd6\x7e\x6d\xa8\x09\x9d\x1a\xca\x74\x9f\x23\x2f\x8e\x18\x79\x7c\x8c\x00\xfc\xed\x33\xd8\x44\x27\x4e\xb9\xec\x5e\x96\xa3\xdf\x4a\x72\xfd\xc8\x9f\x77\x77\xdf\xff\xf8\xe2\xf9\x77\xaf\x08\x94\x9c\xaa\x28\x48\x3a\x22\x61\x48\xfa\xa4\x57\x6a\x28\x89\xfc\x62\xfb\xda\x43\xf6\xdd\x9d\x77\x96\xe3\x8c\x25\x93\x97\x79\xba\xa0\x54\xfc\xc0\x6d\x33\x19\x5f\xa6\x25\xbe\xa0\x85\xe5\xe0\x17\xbb\x4c\xb3\xd3\xd8\x9f\xb2\x7c\x83\x43\xc3\x71\xec\xb8\x11\xb6\x1c\xe9\x34\x11\x68\x94\xf3\x0b\xe5\xca\x05\xf2\x2a\x5d\xdb\x29\x74\x7e\xf1\x0d\x39\x26\x20\x08\xf6\x51\x80\xfe\xf9\xcf\x9a\x91\x41\xd8\xc8\xf6\xd3\xd5\x6c\x91\x5e\xa7\x0b\xae\x4f\xa1\x42\x86\x47\x08\xf4\x03\x4f\xfd\xb2\xca\x3a\x6c\xbf\x0f\x42\xb0\x52\x45\x90\xcf\xb2\x93\xae\x42\x07\xa4\x4e\x1b\x53\xa9\xa8\x4c\xd9\x77\x9b\x1e\xab\x5b\xa3\xe8\xda\x88\x95\x7c\x1b\x74\xa4\xad\x12\xca\x22\xda\x61\x18\x03\x8d\xb8\xc0\x12\x96\x9d\xf8\xff\xe9\x56\xbf\x54\x66\xa3\xfa\x36\x7e\x9b\xe5\x30\x96\x4a\x05\x79\x37\x4d\xbd\xaf\xe4\x24\xa4\x6b\x93\x69\x30\xdd\x4d\x6b\xcb\xf4\x7c\xbd\x50\x59\x86\xd3\x66\x0a\xab\x72\xfa\x89\x01\xa7\x22\xe8\xb7\x74\x55\xfb\xdc\x4b\xc8\x35\xb9\x8b\x5c\x2e\x02\xe8\xd3\x59\x94\xe9\xf4\xb1\xfe\x0c\x27\xfa\x73\x3a\xd2\xa5\x14\x76\x11\x9c\x85\x9d\x2b\x19\x08\xbf\x85\xcb\x45\xa0\x0c\xa1\x01\xa7\xf4\x0e\x83\xe7\xdb\x9b\xe3\xe3\x0d\x03\x8c\x09\x25\xac\x8b\x81\x91\xaf\x50\x50\x34\x90\xbf\x31\xb1\x05\x6a\x18\xa0\xda\xa1\x73\x56\xbc\xc6\xc1\x79\x85\x0d\x81\xcf\x1c\x70\x91\x2c\x85\x97\xc8\xe5\x5d\x8c\x2a\x76\x3f\x46\x0d\x61\x30\xe2\x4b\x9e\x7b\xa9\xc4\xaa\x18\x32\xee\xee\x6e\xb7\x87\x28\x20\x68\x5d\x04\x41\x5b\xdd\xe6\x65\x3c\x4b\x56\x4d\xf0\x15\x46\xda\x88\xa9\xc3\x37\xa1\xe4\xc0\xa2\x81\x70\x04\x7a\x99\xae\x77\xe0\x78\x64\xd2\x60\xa0\xfa\xc5\xe9\xba\x09\x84\x49\x04\x44\x53\x33\x0f\x93\xa4\xbd\xcb\xd7\x98\x50\x85\xa7\xe7\x76\x44\x99\x6d\x42\x7a\x6c\x47\x70\x92\x01\x61\x2f\x08\x0c\x25\x1d\x21\xe4\xee\x86\x29\xce\x01\xf2\x72\x01\xcf\x51\xa4\xd9\x18\xf5\x04\xe7\x24\xe3\x63\xcc\x58\xe3\xe2\xf3\xdf\x8b\xdd\x8b\x53\xe4\x30\x46\x3e\x03\x4e\xdf\x63\xe3\x45\x06\x5f\xda\x7e\xf1\xdd\x5f\x8f\xbe\x79\xf6\xe2\xe5\xf3\xaf\xd1\xe5\xcc\x02\xdf\x89\x0a\x42\x7e\xfb\x15\x51\x4e\x16\x3d\xd6\x22\xaf\x2d\x5f\x5e\xbc\xb0\x2d\x33\x79\xa1\x1e\xd4\x0f\x35\x2c\x0a\xb4\xe1\xe8\x76\xe9\xd0\xe6\xf8\x58\x91\xe9\x4b\x5f\x4e\xec\x89\xf2\x16\xfd\x2f\x87\x54\x0f\x2c\x21\x04\x41\x3b\x5b\x65\x37\xf8\x97\xed\x53\x43\xf4\x7b\x4e\x28\x60\x44\x93\x87\x4f\x87\xe6\x5d\xb6\x3c\x43\xca\x61\xa0\xa0\x7c\xf2\x31\x75\x56\xd5\x88\x31\x98\x2a\x24\x8f\xf0\x5e\x83\xf6\x40\xe5\x2b\xda\xd1\x2d\x95\xeb\x29\xcc\xae\xb2\xf2\x85\xa3\xce\x08\x5d\x58\xe9\xcd\xd4\xe5\xd8\x42\xfb\x40\x58\x48\xb3\x5c\xcb\x16\x4d\xbe\xae\x4b\xd1\x13\x95\x8e\x5e\x23\xdb\x13\xd0\x89\xe4\xc8\xba\xba\x4d\x45\x04\x5b\x58\xf6\x29\xcf\xe1\xa8\xd7\x52\x56\xe4\xa5\xdd\x6a\x46\x88\x30\x05\xab\xbf\xe5\x66\x92\x8d\x64\x3d\x9b\xaf\xda\x4b\x0c\x64\x85\xb6\x62\x07\x41\x64\xeb\x3b\x72\x04\xad\x22\x1e\x34\x15\x2d\xd6\x89\xb9\xca\xae\xd3\xcc\x26\x6e\xa5\x6c\x99\x0c\x46\x77\xe3\xc6\x00\x36\xbd\x87\x2e\x84\xb9\xc6\x07\xd3\xb2\xc9\xb3\x95\x26\x4a\xa1\x75\x53\xc6\xaa\x84\x7b\xf9\x8e\xa9\xff\xea\xfb\xef\x7e\x7a\xf5\x0c\x7b\xf2\xf3\xd0\xa3\xb5\xdf\x73\x4f\xbc\xfc\x6e\x28\x03\xf0\x85\xc1\x41\x5f\x87\x07\x7d\x88\xf8\xfe\xaf\x43\x05\x4e\xdf\x14\x35\xe8\x5b\x71\x10\xf0\xc4\x0f\x2f\x64\xc4\x0f\x2f\x3c\xf1\xd3\xbf\xfe\xf8\x6a\xf0\x46\x21\x96\x21\x8e\xb6\x23\x87\xde\x56\x7c\xf3\xf7\xef\xbe\x7a\xf5\xe2\xfb\xef\xa2\xdb\x78\x56\xc8\x34\xf8\xf2\x44\x3c\xcf\x74\x18\x3e\x21\xa2\xc0\x93\x51\x8e\x28\xf0\x7c\x14\xea\xaf\x23\xca\x58\x46\x0c\xad\x98\x21\xbe\xc0\x91\xae\x64\x0c\x7e\x42\x84\xc6\x4a\x48\x81\x4f\x32\x08\x5f\x1e\x76\xcf\x2c\x97\x11\xf4\x0d\xe3\x58\xb6\x94\x11\xf0\x25\x0f\x65\x29\x48\x27\xb3\x97\x9a\x26\x3a\xb2\xdd\x40\xef\xe6\x20\x7c\xc1\xb8\x47\x6b\x6b\x19\xc3\x01\xdc\xdc\xb8\x5a\x2f\x54\x1c\x9f\x44\x99\x9a\x51\xc5\xf0\x20\x5c\x85\xdf\xe1\x3b\x22\xa6\xa2\x58\xcf\x2d\x8c\xad\x55\x92\x47\xa7\xfe\xe4\xb5\xf7\xfa\xa3\xe9\xdd\xe4\x75\xf4\xfa\x8b\xd7\x4f\x5f\xff\xc7\xeb\xe3\xd7\x77\xaf\xc7\xaf\xc3\xd7\x9d\xd7\xdd\xd7\xa7\xaf\x4f\x5e\xff\xcb\xeb\x47\xaf\xff\xfd\xb5\x78\x3d\x7a\x3d\x79\x3d\x7d\x7d\xfb\x7a\xfb\xda\x7f\x1d\x1c\x4d\x3b\xc1\xe9\xa8\x39\x7c\xca\xed\x88\x52\x5c\xd3\x3e\x61\xf1\x4e\x2c\xde\xc1\xf0\x7e\x91\xac\x0b\x58\xec\xa3\xc7\xd2\xd2\xa7\x72\xd9\x2c\xa4\x78\x07\xab\xc4\x05\xfe\xc8\x2d\x3c\x82\xdc\x31\x54\x63\xcf\x68\x9c\x08\x6b\xc6\x91\xf7\x11\x3d\x10\x35\xfe\xc8\xfb\x88\xfe\x2e\xde\xdd\xdd\x15\xf0\xcf\x57\xc2\x39\x29\x71\x15\xad\x33\x44\x56\x7c\xa0\x85\x85\x81\xae\x35\x5e\x7c\x5d\xea\xf8\xd8\xf3\x71\x4b\xef\x1a\x94\x34\x07\x83\x95\x09\x66\x72\x2a\x0c\xe8\x7e\xb4\x78\x17\x84\x58\x3a\xd6\xe4\x51\xf1\x4e\x6f\xfa\x7d\xa3\x18\x82\xcf\x28\x4a\xd3\x2a\xc1\xab\x4f\xe1\x49\x18\xdf\xeb\xc8\xfa\xbe\xcd\x52\x80\xf4\x82\x8e\x17\x8c\xf0\xb1\x1a\x7b\x3a\x53\xcf\x0f\xb5\xde\x36\xe2\x34\x9e\xe4\x64\xe0\x43\xd7\x5e\xea\xd5\x1c\xb9\xfc\x72\x83\xd6\x0a\x8c\x13\xf0\x3a\x2b\x7f\x98\x75\x95\x9b\xa5\x6d\x21\xb6\x27\x8b\x26\x95\x83\xbc\x1f\x49\xeb\x01\x7c\x39\xd0\x5e\x82\xa1\x13\x52\xb9\x84\x50\xb0\xf4\xf2\x30\x25\xe2\x97\x03\x4d\xb7\x69\x5c\x70\x7c\x05\x87\xd7\x23\xf0\x61\x03\x63\xb8\x89\x9a\x9e\xa3\x91\xee\xfd\xf1\xd3\x41\x4e\x31\x2a\x87\x59\x07\x1e\x1f\xcb\xdc\x3b\x2b\x44\x5b\x8f\xe6\x95\xa0\x81\x6d\xae\x11\x35\x0d\x7b\xd6\x7a\x32\xf5\x7a\xdf\xb2\x4c\xd5\x96\xd7\x5f\xce\xfa\xe7\xfe\xd5\x59\xbb\x96\x6d\x13\x74\x40\x5d\x94\x60\xab\xfb\x74\x4f\x05\x47\x7f\x76\x00\x29\xd6\x01\xe4\x37\x6b\x9a\x80\x1c\xeb\xa2\x5c\xca\x47\x25\x76\x75\x55\x8e\x67\x55\x54\xed\xe8\xb1\xcc\x3a\x3d\x0d\x3d\x96\xdb\x9b\x78\xb9\xf4\x49\xbb\xa1\x37\x87\xc8\xc3\xbf\xe1\x0b\x4f\xa0\x91\x7e\x22\xa0\x90\x3e\x68\x69\xb3\x8f\x72\xb4\xef\xf5\x51\x12\x6e\xec\xe9\x62\x0d\x4e\x2a\xb3\xc7\xcf\x8d\x4b\x30\xbe\x7e\x08\xb1\xa0\x5d\xc4\xbc\x19\xa7\x02\xf7\x60\xb8\x25\xa0\xd0\xc0\xe3\x1a\x81\x42\xe8\x0b\xce\x72\x86\xad\xe3\x48\x67\x11\x65\xd5\xa2\x05\x99\x7c\x23\xfd\x69\x69\x36\x52\x9d\xc0\x4d\x56\x60\xf4\xd7\xda\xc6\x50\x9d\xdf\xf1\x08\x82\x21\xda\xe3\x1a\x2f\x66\xbc\x97\xca\x8f\x01\x5a\xb9\x8c\xb2\xd3\x50\x6e\xb6\xaa\xe1\x28\xa8\xf8\xc0\x2f\x3f\x50\x94\xab\x3e\x51\xe3\x71\x75\xb7\x6e\x03\x17\x8d\xa5\x81\xfd\x0a\x8e\xf6\x38\x4d\xae\xcd\xf9\x6d\x39\xf9\x7e\x4d\x68\xc3\x49\x4b\x67\x39\x7c\x91\x6a\xce\xbb\x44\xe6\x29\x38\x8e\xc4\x2f\x8a\x92\xe2\xc6\xb1\x32\x40\x09\x44\x52\x48\xa4\x30\x79\x2d\xba\x20\x55\x66\x6b\xd5\xa3\xdd\x2b\xbe\xf6\x41\x8a\x21\x79\x6f\x31\xbe\x2a\x33\x4f\x4a\x66\x5a\xc8\xf5\x07\xa6\x2b\x00\xf4\xdb\x3c\xa7\x17\x75\x21\x76\xac\x82\x61\x23\xa3\x3c\x4f\xb0\x32\x1e\xc1\xb7\x1c\x2f\x08\xe2\x3b\xf2\xc0\x41\x10\x63\xf8\x09\x87\x6a\x25\x72\x8b\x6b\xbb\x0d\xaf\xe3\x36\x72\x09\xb7\x51\xcb\xbc\x8d\x53\x2b\x2d\xc5\xed\x67\x95\xd9\xe5\x06\xdd\x37\x23\x37\x04\xf2\xcb\x36\x63\xa5\x9d\xd2\x1c\xa6\x97\x39\xbe\xe5\x81\x2f\x9f\x43\x84\xf5\x4e\x18\x82\xa3\xa5\x1b\xfa\xc5\x25\xc5\x03\xe1\x3a\x91\x47\x27\xe7\x47\xd1\x11\xa9\xdb\x47\xe3\xa3\xdb\xed\x51\x48\x6e\x99\x46\xaf\x41\x83\x52\x5b\xd8\x84\x6c\x77\x0f\x1b\xca\x8b\x10\x33\xc9\x32\x03\xa1\x7b\x46\xc1\xb8\x53\xf4\xa4\xf6\x7a\x7d\x84\x2f\xe3\xc2\x8f\x8c\xf5\xb2\x9e\xd7\x41\xd0\x8e\x07\xc5\xca\x47\xdd\x7f\x4c\xce\x7c\x55\x03\xd0\x01\x46\x64\x94\x27\x9d\xd0\x47\x8f\xfa\xc8\xe7\xa2\x57\x0d\xc9\x50\x0a\xbf\xc6\x8a\x7c\x44\x9f\x9e\x1d\xf9\x59\xaf\x3a\x7a\x7a\x84\x0f\x4b\x07\x47\xb7\x47\xa4\x2d\x01\x76\x08\x8f\xf0\x87\x3f\x87\xf4\x3d\xc4\xca\x8e\x8e\xb6\x23\x4d\x10\xe2\x90\x6f\x9a\x42\x9a\x4f\x30\x5d\x04\x45\x55\x24\xa4\x02\x29\xf1\xf8\xf8\x11\x97\x3e\xa0\x55\xa3\x95\xb7\x72\xf2\x11\x30\xe6\x65\x86\xf7\x6a\x4d\x77\xdd\x42\x77\x4d\x74\xd7\x0d\xba\x6b\xa4\xb5\xe6\x4f\xa2\xbb\xde\x47\x37\xf7\x44\x26\xa0\x66\x02\x6a\x43\x38\xa7\x2a\xca\xeb\x1d\xca\x6b\x27\xa3\x1c\x04\x88\x74\xf4\x3d\x8e\xd7\x6b\x49\xda\x30\x77\xe0\xe6\x25\xea\x27\x1f\xc9\x27\xe4\x40\x7c\x17\x1f\x4d\x83\xa3\x66\x0c\xa0\xa7\xd3\x04\x2f\x10\x8d\xbc\x24\x70\x01\xcb\x9d\xf3\x58\x3c\xf9\x04\x3b\xca\x80\x8a\x51\x99\xd7\xb7\x3b\xea\x24\xa6\x93\x1b\x23\x1a\x9b\xd8\x3f\x03\xf3\x24\xd8\xce\xe3\x12\x66\x9b\x84\xde\xf2\xe1\x2d\x8b\x84\xa6\x37\x58\x72\xf8\x0c\x61\x54\xb2\xdd\x0a\x1a\x3b\xa0\xbc\xa0\x1b\xc6\xa0\xd5\x7a\xf8\x28\x1d\x8e\x0e\x08\x26\x9f\xae\x57\xc1\x40\xcb\x23\x68\x89\x2c\xa1\xf2\x43\x3e\x90\xcb\x4c\x57\x7f\x55\xa2\x7a\x30\x97\x02\xf2\x56\xae\x0c\x59\x7c\xb0\x7d\x74\xef\x74\x11\x33\xd4\x61\x48\x92\x4d\x8c\xa6\x11\xa1\xf8\x0a\x1f\x41\x8a\xf8\x31\x44\x20\x16\xf3\x01\xed\xf2\xc9\x43\x19\x81\x63\x9b\x84\x24\x7c\x80\xa9\x37\x57\xb8\xe8\x15\x25\x2c\xd2\x3b\x9f\xaf\xf0\x79\xbc\xb3\xde\x39\xfd\xce\xe9\x77\x45\x4f\x3f\x50\xec\xdd\x1d\xfe\x29\x5a\x32\x16\x4e\xc6\xa2\x91\x71\xc5\x19\xe3\xdd\x8c\xa0\x2c\x71\xc6\x15\xfd\xc6\xf4\x3b\xd3\x19\x73\xce\xb8\xe4\x3f\xb3\xdd\xfc\xf9\x52\xe6\xcf\xe9\x77\x29\xf3\x2b\xcb\xe2\xc8\x53\x4e\xdf\xae\x78\xc2\xf7\x9c\x9d\xf2\x5c\xa9\x94\xd2\x2a\x13\x27\xef\x32\xf7\x75\x74\xa0\xb4\x59\xc2\xcd\xfb\x0e\xe4\xbb\x23\xdf\xa0\xeb\x49\x7b\x9a\x31\x10\x80\x0b\xd3\xff\xfa\xe3\xf7\x7f\xff\xe1\xcd\xcf\xcf\x7e\xfc\x69\xa2\xd3\xa6\x63\x16\x65\x18\x15\x75\x9c\x9c\x79\xe5\xed\x2a\x18\x23\x97\x6c\x9d\x64\x61\x54\x13\x24\x1a\x71\xc9\xd5\xd3\x74\x82\xeb\xa7\xa9\x17\x98\x0d\xf8\x9c\xf4\x7b\x79\x75\xd5\xa1\x8b\xcf\x86\xc7\x7e\x83\x21\xcd\xc2\x2c\x7b\xab\xf6\xc2\x9a\xcc\x90\x5c\x83\xda\x76\xbc\x09\x8d\xee\x44\x12\x0e\x47\x50\x10\xbd\x20\xa2\x4f\xdb\x0e\x54\x92\xbe\xee\xad\x6a\xe0\x37\x9a\x32\xe0\xd7\x95\x5b\xe9\xb1\xdb\x8d\x74\x15\xe9\x05\x01\x3f\xa3\x26\x7b\x18\x60\x6c\x35\x3e\xc7\x84\xe6\x9b\xf1\xcb\xd6\x83\x29\x8e\xc6\x25\xa0\xca\xae\x90\x93\x99\x33\xc8\xfe\x42\x11\xcc\x5d\x64\xd0\x88\xc9\x90\xed\x2e\x57\x5b\x5e\x87\x3e\x88\xcd\x58\x09\x0e\x11\xc2\x59\xbc\x06\x65\x85\xaf\x8e\x7f\x19\xe3\xaa\x14\x28\xf0\x25\xbf\xf1\xfd\x6e\x63\xd2\x18\x61\x3c\x65\xba\xbc\x5a\x95\xe3\x6b\xe5\x03\x5f\x47\xa1\xd7\x8a\x13\xf9\x7c\xbd\xcc\xcf\xf0\xbc\x22\x1b\x7b\x47\x9d\x23\xaa\x8d\x95\x4f\x2e\xd6\x30\x8f\x90\x83\x88\xe4\xb4\xa2\xe2\x23\x40\xeb\x7d\x44\xf6\xc5\x96\x27\x0f\xd5\x49\x49\x41\xaf\x44\x2d\xde\xcb\x87\xcb\xab\xa8\x1a\xeb\x71\x0e\xd4\x97\x2a\xb0\xce\xe7\x28\x1b\x3f\xd8\x80\x66\x11\x75\x1d\xd5\x0e\x70\xbd\x0f\x78\x30\x15\xef\xdf\x47\xef\x1d\xe0\xf7\xfb\x80\x87\xea\x34\xdd\xe3\x97\x6e\x60\xa1\xe8\x75\x30\xa9\x83\x75\x9a\x54\x40\x2e\xd0\xfb\x7e\x2a\x65\x50\x00\xbb\x3e\x0a\x90\x37\x5e\xf0\x11\x0d\x2e\xa6\x7b\x47\xb7\xac\x33\x0f\x94\xbe\x3c\xe0\xe7\x2c\x59\x04\x78\x7a\xf0\x1a\x91\xd2\xa6\x28\x1c\x18\x3b\x1d\x56\xf8\x9c\x25\x1a\x8b\x46\xfb\x8d\x52\x4a\x62\xa5\x9e\xbf\x77\xb6\x46\xad\x1d\x91\xe6\x0e\x69\x26\xf4\xf6\x2c\x69\x73\xb8\x08\xa6\xf9\x66\x93\x27\xd7\x11\x2b\x6c\x21\x7e\x7b\xc6\xc8\x92\xf5\x33\x3e\x2f\xc6\xad\x8d\x4c\x3d\x1b\xa4\x8a\x15\xe8\xb4\x07\x2f\xa6\xc8\xbf\x77\x77\x0a\x24\xdb\x0a\xb5\x60\xb8\xb5\x37\x4a\x0c\x32\xfa\xb2\x6a\xc0\x2e\x4b\xd2\xb5\x74\xe1\xa7\x41\xd0\x8e\xae\x50\x6f\x0c\x7d\xff\xe3\xd7\x2f\xbe\x7b\xf6\x72\x9c\xe5\xb0\x3e\x88\x57\xe1\xbb\xab\x18\xb7\x74\x63\x7c\x6f\x4c\xe4\xb0\x62\x90\xce\x16\x20\xb3\xa4\xb0\x71\x01\xdd\x1a\x93\x8c\x23\x23\x2e\x0c\xf3\x23\x3d\x7c\x25\xb5\x30\x34\xb7\xd0\x64\x9e\x25\x51\xa7\xe2\x77\x77\x2f\x5f\x7c\xf7\xfc\xd9\x8f\x38\xd9\x3c\x22\xa0\xbb\x3b\x8c\x87\x29\x87\x07\x03\x7a\x88\x89\xdf\x09\x29\xb3\xdc\x3a\x6c\xa5\x64\x96\xd1\xbb\x3b\x74\x22\x61\xc2\x23\x04\xa5\x2d\x2a\x79\x40\xf3\xf7\x75\x9e\xcc\xb3\xe5\x3a\x7d\x9f\x2c\x98\x97\x44\x05\x9e\xd4\xf0\x43\x4f\x7c\xae\x1f\xcd\xd9\x7b\x54\x4f\x3e\x9a\xa7\xca\x67\x92\xb8\x0e\x5c\xce\x77\xb6\x2c\xa8\x51\x85\x52\x4c\xfd\x25\xb3\xad\x56\xb3\x39\xc5\xeb\x60\x3a\x1f\xc3\x99\xb8\x90\x96\xa2\xf8\x0a\xf2\x88\xa3\x09\xbf\x84\xb0\x8f\x55\x29\x26\xe0\xbd\x27\x99\xae\x22\x43\x7b\x55\x26\xe3\x60\x00\x42\xfc\x16\x36\xe9\x33\x95\x18\x14\xeb\xa3\x5a\x09\xae\xee\x1a\xe7\xfc\x5e\xa8\xdb\x1b\x24\x8d\xb9\x5e\xc5\xdb\x57\x92\x67\x93\x9c\x66\x9b\x29\xbb\x56\x0f\x44\xd3\xbf\x89\x33\x9b\xa8\xa9\x56\x7e\x84\xb4\x07\xda\xa3\x01\x55\x67\xc8\x9d\x09\x8f\xed\x1a\x83\x51\x9b\x09\x34\x96\x25\x29\xe7\x77\xcf\x68\x07\x8e\x1d\x85\xc8\x4b\xce\xea\xc2\x1e\xdd\xd6\x73\x98\x27\xf3\xe1\xfd\x1c\xd0\x0e\x9b\x2f\x16\x41\xa3\xe1\xe8\x0a\xd1\x30\x47\xad\x97\xea\x86\xdf\x50\x36\x00\xf7\x1c\x48\x08\xe8\x81\x50\xbe\x32\x6c\xa7\xfd\x40\x31\x3e\xb5\x3d\x02\x28\x5b\x85\x3e\x67\xa0\x3d\x7a\x68\x0a\xe5\x3a\x5b\xc7\xd9\x38\x7e\xc4\x08\x9c\xd0\xda\xf1\x58\x90\x7b\x81\x8c\x5c\xda\x9d\x7f\x9f\x70\x6a\xe0\xa4\x82\x95\x04\x08\x13\xae\xa1\xf1\xa2\x1e\x5e\xcd\x7b\xbf\xaf\xf9\x51\x8a\x6c\x01\x68\x7a\xe0\x6d\xa8\x4e\x2d\x46\xe5\xb6\x88\x9c\x05\xe3\xb3\x56\xb1\x38\x73\x04\x62\x1b\x8c\xa4\x33\xdf\x9d\x6d\x35\xc0\x7b\xd6\xb4\x50\x87\x4a\xe0\x61\x25\xb5\x3c\x19\x5c\x3c\xc6\xb3\x15\x7f\xa2\x63\x84\x8c\x41\xa9\x3a\x0b\xa6\xb4\x5b\x54\x99\x2c\xef\x39\x4b\x5c\xe9\x2c\xef\x39\x4b\x5c\x99\x2c\xf2\x68\xdb\xee\xde\xb4\x86\xc1\x1b\x50\xef\x55\xf7\x9e\xb0\x83\x7f\xf8\x51\x9e\xd3\xe5\x8e\x2c\xa7\xa3\xe2\xe0\x1a\x53\xa8\x9e\xbf\x33\x3c\x68\x17\xf2\x4e\xf7\xb7\x72\x7c\xd8\x08\x40\x5d\x7f\x87\x99\xd0\xe9\xb5\x30\x88\x01\xfe\x07\xd5\x04\x19\x6e\x12\x13\x88\xf7\x91\x2c\x45\x59\x7c\xb6\x55\xef\x5b\x53\xc3\x1d\x82\xbf\x25\x9a\x4d\x1b\x11\xa3\x1a\x55\xf8\xf6\x70\x2d\xbe\x7d\x70\x45\xfa\x20\xcf\xa6\x22\xca\x06\xc7\xc2\xd2\x46\x3c\x49\xc4\x1e\xe2\xe3\xca\x10\xff\x7e\x0f\xf1\x71\x75\x90\x78\x40\xf1\x10\xe2\xfb\x40\xbe\x43\xfc\xfb\x69\xd4\xc0\xa2\xe6\x59\x18\x59\x5e\x7e\xff\x57\xe8\x98\x26\xfc\xea\xc5\xb7\xcf\xef\xee\x1e\x61\xcc\xfb\x24\xcf\x60\xa0\xb4\x2b\x8a\x51\xfc\xe8\xa0\xe4\x8a\x3a\x86\xf4\xfb\x42\x47\x06\xc2\x14\xac\x4e\x2d\x4d\xfa\x7b\xec\x3c\x6d\xd3\x94\xb6\x14\x57\xe3\x1d\x8e\x5c\x38\x4f\xa1\x26\x02\x43\x2c\xe8\x44\x09\x5a\x09\xa8\xec\x13\x3d\x28\x1e\x1f\x37\x47\xc5\xb1\x67\xbe\x41\x59\xa7\x80\x37\xa5\x01\x99\x2a\x9f\x54\x9b\x6c\x4d\xe7\x30\x56\xd5\x7f\xf8\xfe\x1f\xcf\x7f\x54\xb8\x14\x84\x6f\x83\x73\xe6\xf9\x2a\xbe\xdc\x28\x38\x0a\xf8\xf8\xdc\x3e\x4d\xfc\xd0\xe3\xf8\xc1\x6c\x9b\x9f\x63\x5f\xf5\x73\x7a\x3e\x2a\xbd\x64\xda\x11\x78\xaa\x87\x80\xbd\x9a\x89\x02\x40\xbd\x44\x7d\x2b\x0e\xae\xc9\xc5\x8d\x8a\x54\x63\x3e\xc5\x06\xd6\xe0\xde\x54\xe4\xf8\x29\x7d\xe0\xab\x35\xda\xec\x98\xc0\x51\xa6\x00\x3d\xfe\xec\x3c\xd7\x67\xa7\xb2\x22\xff\xc3\x8f\xdf\xff\xf0\xfc\xc7\x57\xbf\x4c\x74\xda\x34\xa0\x02\xfa\x42\x3e\xed\xa7\xe3\xd9\x95\x10\xee\xa2\x3c\x32\x7a\x1b\x25\x39\x99\xf5\xd0\xef\xf2\x84\x58\x42\x30\x64\x51\xa3\x29\xc1\x99\x6a\x84\x25\xee\xc7\xc9\xd6\x63\x12\xc6\x1e\x42\x19\xc3\x58\x7f\xe2\x4b\x86\x26\x5b\x3b\x63\x78\xa4\x62\x35\xc0\x89\x6d\x0c\x0d\x1c\x4d\x23\x03\x02\x1b\x5d\xa1\x3b\x98\x3a\x00\xcd\x7c\x2c\xed\xb2\xa9\x60\x55\x61\x45\x8e\xec\x31\x06\xe2\x58\x9b\xbb\x76\x26\x35\x08\xcb\x59\xd5\x51\xd3\x71\x04\xb9\x6e\xed\x54\x4a\x5f\x85\x58\x9a\xa6\x58\x03\x8f\xe4\x1b\xec\x9e\x90\xab\x85\xc8\x93\x1a\x2c\xd9\x64\x40\x32\x1a\x28\x50\xcf\x89\x3c\xb2\x3f\x40\x81\x8f\x3c\x14\x72\x4f\xb8\xb2\xb1\xbb\xd0\x33\x97\xc3\x69\x79\xe3\x2e\xde\x60\xf5\xb5\xc7\x30\x4b\x5c\xa7\xc9\xcd\x37\x31\xea\xe5\xb5\xed\x85\x8e\x8c\x88\x78\xb3\x9f\x72\xa3\x89\xd1\x15\x3e\x7b\x8a\x5b\xa5\x14\x4f\x5b\x7e\x44\x05\xed\xc5\xab\xbd\x47\x74\x0f\x2f\xbd\x80\x50\xb4\xda\x75\xc4\x78\x2c\x6b\x83\xae\x98\x28\x45\x85\xe4\x0b\xc4\x68\x6c\xa5\xaa\xd5\x76\xe6\xa3\xc0\x43\xf5\x21\xa4\xf2\x15\x36\x8f\x67\xf8\x50\x47\xd9\xa6\xca\x93\x1e\xf7\x2c\xaa\xe5\x08\x8a\x1f\x66\x70\x4e\xe0\x18\x11\x5f\x00\x34\x0b\x5f\x6d\x2c\x86\x84\x7c\x95\xc1\x22\x2c\xbf\x42\xfe\x05\xf4\xdc\xa6\x1b\x17\x59\xec\xa5\xb7\xff\x41\x6f\x31\x31\x76\x2a\x8d\x59\x3f\x9b\xb0\xb0\x05\x93\x18\x3e\xb6\x1a\x25\x84\xc1\x8f\x7c\xdf\x52\x3d\x1a\x36\x74\xf4\x0c\x5e\xd3\x86\x8e\x7e\x03\xde\x20\xf7\x09\x42\xbe\x0e\xda\x30\x7b\x6a\x37\x9b\x92\x16\x66\xc6\xd6\xda\xb7\x97\x09\x74\xd4\x83\xe0\xad\x07\xaf\x1e\x66\x40\xb3\x91\x8b\xe3\xe3\xb2\xa2\xa7\x70\x92\x33\x75\x67\xb8\x22\x1a\xe8\x98\x52\xda\xbb\xf3\x91\x25\x89\x7b\xa4\x76\xdb\x75\xf4\xf3\xef\x5e\x41\xef\xe8\x9b\x88\xbf\xff\xf0\xf5\xb3\x57\xcf\xd9\x8a\x4e\x82\xfc\xf3\xc5\xab\x68\x68\xc2\x5f\x3f\xff\xe6\xd9\xdf\x5f\xbe\x7a\x03\x70\xcf\xa2\xdb\x02\x4d\xad\xd6\xc9\x2a\x1c\x58\x85\xb9\x86\x8b\xd4\xde\xcd\x03\x55\x89\x25\x6a\x45\x6b\xdf\x86\x68\xf8\x90\xba\xbb\x33\x98\xee\xee\x26\x32\xd7\x54\xae\xff\xa3\x48\x46\xe0\x44\x47\x6e\xda\x5c\x00\x8b\x46\xbe\xa6\x95\x3b\xa7\x8f\x99\xde\x5f\xc9\x7a\xfa\x1a\xd7\xd8\xfa\x46\x9f\xa8\xb4\xbf\x26\x01\x7e\x26\x47\x13\xe5\xd8\x09\x35\x81\xf8\x02\xd8\xd8\x0e\x18\x10\xde\xcb\xcc\xf4\xb5\xaa\xb1\xf9\xb4\xb6\xb7\xf8\xd5\x7a\x9d\x12\x74\xbc\x4d\x85\xbe\x6b\x19\xd8\x36\x23\xb7\x20\x03\xab\xb6\x2f\xf0\x65\x95\xb6\xcd\x26\xdc\xed\x94\x16\x02\xfc\x20\x3b\x9a\x0e\xd0\x69\xb3\x34\x18\x90\x0f\xb2\xd3\x06\xed\x81\xd7\xd8\xf9\x89\x5b\x7a\xee\xf2\x27\xdb\x87\xa8\x79\x85\x9e\x8d\x0f\x74\x21\xbd\x86\x75\x86\xc1\xb8\x29\x8e\x8f\x99\x2d\x6c\x3b\x40\x1b\x4d\xf6\xc3\xd8\xf3\xec\xaa\x48\xad\xeb\xd3\xbc\x89\xa9\xcc\x80\xa4\x3f\x31\xb9\xb3\xa9\x8e\xe1\xf1\xa9\x7e\x8e\x8a\xd4\x25\x06\xe9\x43\x87\x6a\xac\xc9\xb2\xad\x21\xd8\x14\x20\x45\x3f\x7e\x7c\x48\x43\x25\x8c\x29\x0d\xf9\x51\xe8\x27\xf5\xf9\x75\x75\xb6\x08\xba\x44\x78\x9e\xd7\x5c\x20\xe2\x72\x87\x4b\x1f\xd9\x46\x00\x0c\x37\xc1\x8c\x53\xf9\x8d\x85\x3a\x75\x2e\xd2\xd9\xca\x95\x55\xae\xdf\x6f\xad\x2e\xd5\x6e\x5f\x8d\x1a\xc4\x1a\xb0\x5d\xfa\xf8\x9d\xef\xa6\x89\x25\x3d\xe8\x43\x0f\xa6\x93\xed\x2b\x09\x91\xc1\xa2\x5c\x10\xa9\x32\xe9\x11\x1e\x55\x18\xbe\x3b\x0e\xbd\x18\x1f\xae\xb3\x7c\xd9\x74\x07\x63\x36\x86\xc5\xc7\x59\x42\xf6\x6b\x64\x3d\x1e\x8e\xef\xaa\x5c\x6e\xf9\x97\x27\x6c\x16\xff\xd4\x11\x7f\x16\x77\x75\xda\x96\xdc\x1c\x39\xdd\x84\x93\xad\xbe\x73\x9d\x16\xf6\x45\x7f\x7a\x8f\x1b\x43\xea\xd8\x7e\x4d\xe5\x91\xd9\x0c\x46\xf3\x43\xdc\xb2\x25\x1e\xf5\xe9\x78\x0e\xd3\xe9\x61\x6e\xae\x78\x60\x1c\xda\x31\xd7\x1b\x66\x82\x6c\xad\x60\x95\xee\x4b\xae\x4f\xb9\x60\x8d\xdc\xa2\x72\x76\x95\xee\x79\xab\x89\x52\x48\xc3\x5e\xcc\x04\x51\xff\x00\xeb\x16\xb2\xa9\xca\xb3\xcb\xb1\xfa\xf0\x5b\x32\xef\xb5\x7a\x41\xc0\x88\xca\xc5\x07\xac\xa9\x6c\x7e\x9e\xdc\xf0\x80\x93\x91\xe1\x7c\xb3\x89\x66\x09\x06\xa2\x94\x57\x38\x81\x5a\x39\xad\x75\x0b\xcd\x65\xc7\xc7\x04\xf6\x57\xec\x40\x76\xe5\xb8\x8c\xad\x5b\x7b\x97\x8a\x5b\x9b\x4a\xfc\xc1\xbd\x6b\x26\x0d\x60\x22\x05\x47\x92\xea\xec\xfd\x4a\xf2\x71\x4d\x03\x4a\x45\x0a\xf2\xfe\x88\x18\x65\xc7\x44\x11\x5e\x73\x6a\xa1\xa1\xad\xaa\x52\x82\xf0\x3a\x3a\x3d\xf6\x87\x47\xe6\xa4\x0c\x0b\x7a\xbe\x00\x13\xce\xf0\x52\xf6\x37\x96\x0e\x81\xda\x41\x20\x77\xf3\x35\x37\xc9\x04\x67\x8d\xb7\x7d\xc9\x14\x67\xa3\xac\x43\x94\xe1\x07\x82\xef\xda\x7d\x50\xb7\xc0\x24\x94\x2c\xea\x9c\xe8\xe9\xe7\xaa\x88\x70\xce\xa7\xc2\xf9\xd2\xe4\x84\xd2\xe8\x3d\x2d\xee\xa3\xf7\x3f\xea\x47\xb5\x30\xd7\xd5\x93\x3a\x4a\xa9\x04\xac\xcd\xd9\x58\xde\x8b\xa4\x00\xbf\x94\x1f\x4c\x43\x45\x08\x55\x9d\x0a\x1a\x3f\x1a\x84\xfc\x28\x98\xd3\x93\xe5\x53\xf7\xb8\xb0\xb5\x89\xa6\x7c\x63\xd2\x68\x42\x56\x63\x84\x39\x56\x8c\x98\xad\xaa\x22\x58\x32\xf1\x8b\xcd\xea\x77\x6b\xf5\x81\x0c\x8b\x88\x65\x38\x5e\x59\x05\x9c\x8d\x55\x28\xa4\xa2\xe4\x83\x65\x8d\x62\x4d\xa7\x81\xf8\x86\xcc\xec\x4a\xb7\x65\x2a\x74\x29\x2e\xb5\xc5\x10\x4f\x55\xf4\x6b\x0f\x34\xac\xab\x93\xfc\xb2\x69\x9f\x5d\x45\x86\x6e\x79\x8f\x11\x86\x1f\x9e\x05\x19\x02\xeb\x6a\x9f\x67\xd2\xe8\xa5\x5e\x6b\x70\x12\x30\x52\x8f\xe6\xf2\x94\x29\xf8\xa2\x4f\xa7\xe6\xd6\x79\xa8\x7d\x08\x35\x8d\x76\x92\xa6\xf4\xba\x2d\x2c\xfb\x88\x5a\x6b\x86\xb4\x67\x55\xb6\x39\x43\xbf\x7e\x58\x15\xaa\x9c\x76\xf1\x89\x55\xb9\x84\xaa\x5c\x06\xce\xf4\x3a\xe5\x21\x87\xc6\x77\x0c\xea\x33\x29\x0d\x20\xec\xe5\x68\x23\xad\x67\xb1\x64\xdb\x68\xa6\xc6\x58\x45\xb7\x52\x70\x57\xeb\x8a\x1f\x56\x82\xfa\xf3\xe0\x22\xc3\x91\x9d\x28\x37\x7d\x62\xbc\x77\xc2\x60\xf8\x8d\x9b\x31\xf8\xd7\x57\x89\x01\xc3\x91\xfd\xa3\x12\x82\xe6\xb4\x22\x75\x17\x8a\x60\x81\x68\xcc\x2f\xcd\xd7\x62\x35\x46\x7e\xfc\x88\x26\x6a\x3d\xd3\x28\x73\x45\x5d\x55\x7b\x1c\xc2\x31\xc8\x18\xaf\x60\xa8\x69\xbc\x72\x66\xbd\x2e\x09\xc9\x3b\x0f\xd7\xed\x18\xdc\x5b\xbe\xb7\x0a\x34\x7c\xe3\xf9\xf1\x6c\x67\x6e\x7c\x8a\xf2\x54\x74\x22\xef\x0e\xfd\x39\x74\x22\xb5\xf9\xde\x78\xa7\x59\x3a\x2a\xe1\x75\x8f\xbb\x0c\x12\xbc\xea\x71\x17\x41\x42\xae\x7c\x1a\x2b\x21\x41\xcb\x1f\x67\x31\xa4\x4a\xa0\xa6\x77\xd4\x8c\x19\xee\xe9\xed\x33\xfa\x5b\x93\x63\x0c\xf3\x28\xc8\x59\x75\x77\xe7\xc3\x6f\x64\xdd\x14\x4c\x0b\x9c\xa9\xc6\xa0\x93\xf8\x79\xf2\xee\x2a\xcd\x13\xdf\x9b\xc7\xeb\xeb\xb8\xf0\x82\xc0\xc7\x1d\x6c\x5c\xa8\x16\x09\xbe\x9a\xef\x7b\xb3\x6c\x51\x7b\x01\x5e\x45\x44\x9b\x74\x0d\xd8\x8b\xcb\x32\x87\xe0\x2a\x2e\x0a\x4f\x78\xd7\xc9\x32\x7e\x73\x9e\x2e\x16\xc9\x5a\xa7\x49\xff\x17\x03\x15\x96\xbb\x9b\x18\x51\xe0\x3a\xc7\xf7\x16\x29\xe8\x55\x31\x3a\x03\x5b\x67\xeb\xc4\x63\x4f\x6c\x78\xd2\xb8\x4c\xca\xaf\x64\x5d\xbc\xe1\xc2\xb3\xb7\xf2\xd0\xd4\xea\x4b\x62\x81\x9f\x09\x0c\x08\x66\x88\x99\x70\xa5\x55\x67\x79\x3e\xe6\x94\x1e\x1a\x8d\xf7\x05\xfd\x17\x84\x3e\x45\x92\x31\x97\x9d\x5b\x64\xd2\x56\xfd\xf8\x18\x77\xa0\xb2\x5e\xb6\x89\xe7\x29\x3e\xbb\xaf\x12\xfe\x81\xd5\x41\xb9\x90\x58\x93\x6a\x83\xb6\x14\x4e\x32\x74\x20\x4e\x75\xe9\x05\x4a\x15\x91\x6c\x04\xca\xcf\xf5\x8c\xb3\x89\x67\x19\xc0\x79\x78\x41\xab\x19\xc5\xef\x96\xfb\x9c\x21\x60\x65\x48\x2f\x63\x1c\x56\xa8\x22\xac\x0b\xdd\x79\x12\x37\x8a\xe6\xde\x9c\xd9\xfa\x71\xc6\xdd\x17\xcf\x78\x80\xb0\x07\x53\x84\xc8\x89\x8b\xac\x6c\x9a\x3e\x61\x51\x65\x10\xef\x12\x47\x4f\xbf\xfc\x51\xc4\x21\xf2\xdf\x42\x5c\x8e\xa2\xef\x12\x57\x01\x61\xd0\x99\xfa\xa2\x86\x8f\x1a\x3f\x6e\xa2\xaa\x93\xa9\x1d\xb4\xbe\x38\x8f\xea\x4e\xa6\x77\xce\xfa\xba\x03\x1b\x19\x44\x13\x96\x1b\x71\xfe\x7b\xc9\xda\x8e\xa8\xa5\x97\xf1\xb2\xc9\xd3\x9b\xc8\x21\xd1\x22\x50\xc0\xa0\xc0\x55\x0a\xba\xbe\x7a\xca\x0a\x17\x79\xbd\x78\x95\x2e\xd7\xe3\x9b\xd3\xa1\xba\x3d\x6a\xc7\x86\xa0\x10\xd5\x98\xb3\x96\x39\x61\x45\xb9\x58\x25\x0c\x33\x83\x49\x04\x79\x3f\x3e\xc7\xcc\xf2\x56\x6a\x23\x05\x10\xec\xe1\x4d\xd5\xb9\x11\x75\xe7\xdc\x6e\x87\xab\x55\xb3\x42\xd5\x40\xd4\x83\x36\x04\x03\xdd\x40\x03\xd5\x42\x6a\xdf\xa3\x1a\x8e\xf1\x27\x84\xbc\x2a\xaa\xc6\xa8\x7a\x18\xd6\x83\x3f\xac\x39\xe2\x7c\xde\xa0\x9d\x1d\x98\x63\x55\xd3\x4a\xa4\xb5\xc8\x2a\x91\xd5\x62\xae\x45\x6b\xae\x65\x2b\xcd\xe1\x2b\x5d\xaf\x93\xfc\xc7\x78\x91\x5e\x15\x18\x97\x61\x5c\x76\x55\xda\x71\x45\x1c\x11\x19\xea\x55\x08\x6a\x93\xf3\x78\x75\xb6\x49\x45\x42\x69\xea\xa9\x07\x3b\xa5\x42\x7f\xc1\xe8\xb5\xb8\xd2\xfe\x8b\x6b\x15\x55\xeb\x28\x72\x04\x54\x44\x93\x22\x06\x54\x53\x51\x44\x45\xdc\x2d\xe2\x7f\x61\x1c\x7a\x9a\xff\xf8\x69\x7a\x7c\x9c\xc4\x4f\x0b\x9c\x38\x71\xaa\xe4\xf4\x80\x73\xb3\x0a\x5a\x04\x96\x77\x75\x99\xd0\x98\x71\x63\x19\x8f\x4a\x41\x15\xa9\xbb\x82\xf8\x4a\x42\x5a\x45\x69\x7e\x02\x9c\x02\x36\xe1\xdf\x3a\x52\x97\xf6\x28\xb5\xc6\x54\x60\x65\x8d\xa9\x35\x57\x4d\x1f\xca\x61\x08\x79\x9d\x55\x01\xd7\x55\x1f\xc7\x61\x48\xa5\xd4\x4e\x9e\x9a\xf2\x20\xc6\x80\x99\xa1\xf3\xd4\x94\x87\x52\x5a\x04\x70\x5e\x75\xa8\xb8\x79\xdd\x21\x14\x14\x86\x0c\x14\xc6\xe3\x8d\x3f\x48\xcc\xf8\x72\x4f\x43\xd2\x72\x51\x8a\xbc\x12\x79\xcd\xce\xa8\x94\xd4\x17\xbc\xf9\x47\x6e\xa0\xd0\xd9\xe2\xce\xa0\x36\x2a\x6e\x52\xb4\x8a\xce\xf8\x5a\x18\xee\x9b\x17\x89\x47\x6f\xcd\x7b\x61\x6e\xb9\x0c\x46\x14\xa7\x4f\x82\xd3\xa1\x28\xa3\xc7\x27\xb9\xb0\x7b\x62\xb7\x14\x35\xfc\xab\x3a\x35\x74\xe7\x32\x18\x91\x23\xc0\x11\x61\x5a\xa4\xf1\x65\x86\x47\x92\x79\xdd\x44\xe6\x0f\x4f\xca\x78\xfd\xb8\x8f\x47\x34\x55\x94\xd7\x1c\x72\x11\x43\x95\xea\x6e\x8e\x23\x05\x7e\x75\xf2\xda\x41\x5e\xbc\xbb\x82\x59\xc9\x0b\xcb\x06\x6a\x40\x18\x95\x40\xa9\x8b\x0a\x31\x21\x22\xc4\xe3\xa0\x29\x71\x83\x15\x64\xb1\xbb\xc8\x6e\xd6\x40\x69\xd5\xa4\x14\xbf\x1e\x03\xd2\x3a\xca\xab\x13\x0a\xec\x20\x3f\x48\xa7\x2e\x00\x4d\x41\x7f\x57\xf4\x0b\x76\x9b\xb3\xdb\x52\xf2\x96\x70\x70\x88\x07\xea\x00\xec\x0f\x92\x53\xd2\xe6\x94\x94\xc2\x1a\xe6\x47\xf6\xce\xc6\xb3\x94\x35\x19\xe1\x86\xf3\x82\xbe\xac\x79\x86\x22\xe9\x8b\x26\x31\xb5\x47\xbd\x67\x53\x1a\x93\x44\xac\xe6\x2c\x31\xb3\xe6\x1f\xb1\x8c\xb4\x96\xac\xf7\x47\x29\x53\xd4\xb2\x6b\xef\x67\xb0\x42\x23\xe7\x67\xcf\x10\x53\x14\xa3\x51\x3f\x79\x57\xe0\xd8\x2f\x25\xd6\x68\x06\x09\xf4\x42\xf2\x2c\x29\xd3\xb9\x07\xba\xc1\x12\x1d\x72\x17\x57\x79\xf2\x8a\xea\x4d\xe0\x00\x04\x4a\x2e\x1f\x60\x59\x33\x6e\x3c\xae\xba\x91\x3b\xdd\xc6\xb0\xfc\xc0\xc8\x40\x90\xa3\x07\x7c\x6b\x66\x5c\xc3\xd8\x7a\xfa\xc4\x9e\x56\x21\xae\x1b\x9d\x87\xd6\x14\x0c\x31\x51\xdd\x85\xd9\xb7\x73\x7e\x3a\xe8\x87\x90\xfc\xf1\x09\x64\x12\x7b\x66\x5a\x18\x93\x48\x12\x8f\x8f\x1f\xa9\x06\xd1\xad\x2a\x9d\xe8\x49\x08\xf5\xde\x12\xbd\x18\x24\x07\x0d\xd9\x36\xc2\x15\x03\x85\x68\xdc\x0f\x07\xd6\x54\x4e\x2b\x3d\x7f\xa9\xda\x3f\x5d\xcf\x57\x57\x8b\xe4\x25\x5f\xee\x62\x29\xc0\x97\x3d\xc8\x65\x12\x7b\x33\x79\xa1\x97\xe0\xea\x52\xef\x52\x5e\x06\xd3\x29\x23\xe5\x89\xf2\x92\x1c\x21\x35\xdf\xdf\x90\x84\x5d\xad\xc9\x37\x22\x7a\x3f\x7a\x3b\x95\xeb\xa7\xc0\xca\xba\xec\x39\x0b\xd8\xd6\xdc\x12\xc6\x46\x80\x7b\xae\x8d\x5a\x18\x9c\x92\xe4\x83\x38\xd5\x4d\x60\x0b\x27\x0f\xd4\x4b\x75\x87\x48\x05\xd5\xcd\x1c\x99\x3d\x5e\x2c\x7c\x09\x23\x54\xa2\xb2\x58\x5f\xf6\x2a\x93\xaf\x76\xb2\xe0\xfa\x67\xeb\x4e\x5c\xfa\xc1\x52\xc0\x47\x6d\xba\x74\xda\x74\xeb\x1c\xe1\x58\x8a\x33\x6d\x11\xe3\x23\xa1\xbc\xfa\xa4\xd5\xe8\xdd\xdd\x65\x52\x9e\x67\x78\xb5\x59\x6f\xc1\xab\xdd\x4d\xb9\x91\xc5\x58\xdf\xe0\xa6\x0c\x28\xef\x18\x35\xf1\x38\x4e\x6d\xd4\xc8\x9d\xf2\x2f\x25\x3f\xe8\xa1\x9d\xc8\xca\x2b\x36\x1c\x6c\xe4\xd3\xf3\xf1\xf1\xf1\x66\xd6\x9b\xaf\x92\x38\xf7\x03\xc9\xe6\x59\x60\x17\x4e\xa4\x72\x1d\x66\x63\x0d\x1a\x3a\xe5\x52\xc5\x04\xf4\xe9\x1d\x5c\x16\x22\x07\xab\x61\x14\xd6\x58\x32\x8a\xce\x3e\x66\x06\xe3\xad\xa4\x80\xff\x00\xbb\x90\x43\x1c\x38\x3e\xb6\x02\xaa\xcc\xbb\x3b\x87\xaa\x91\xd9\x95\xa3\x0d\x63\x87\xc1\xfc\x5a\xb0\x6a\x00\xc3\xf2\xc2\x3a\x77\x41\x89\xd5\x57\x94\xd8\x82\x57\x5f\x51\x22\x83\x5f\xb5\x24\xda\x61\x15\xad\x93\x94\x8c\x92\x15\x89\xbd\xc7\xe7\xf4\x1e\xb3\xc5\xe7\xc8\x7a\x43\x7e\xf4\x31\x03\x31\x46\x5e\x93\x75\xd8\xa3\x5e\xda\x2d\xe8\xbe\x05\x6f\x3b\xf4\xf8\x9d\x54\x8c\x14\x7a\x19\xdf\x4c\x96\x0c\x57\xeb\xd3\x66\x32\xc6\x0b\xb5\x3e\x6c\x26\xd2\x0c\xc1\x0a\xac\x7a\x2b\x0d\xe6\x5e\x9a\x83\xad\x19\xf5\x31\xbe\x07\x09\xba\x49\xa4\xbc\x43\xf8\x8f\xfb\xf6\xf8\x08\x73\xc6\x19\x9f\xe6\x09\xd9\x20\xd1\x2d\x5f\xd5\xe1\x0d\x33\xd6\xd8\x42\xfe\x23\x68\xd5\x16\xd2\xaf\xc0\x75\x67\x88\x3f\x02\x57\x3e\x21\xfe\x40\x45\xe6\x21\xfc\x23\x77\xa3\x21\xf9\xaa\x96\x2b\x5e\xe8\xfc\x58\x99\x50\xd7\x88\xde\x96\x54\x26\x25\xb7\xc8\xd2\xd0\x48\x23\x49\x43\x68\x9a\xc1\xc2\x67\xd1\xb6\x75\xb6\x9a\x92\x35\x5d\x10\x6d\xdb\x6a\x22\xc3\x35\x82\xc2\x2d\x43\x41\xe3\x88\xc0\xed\xa4\xa4\x28\x59\xf0\xf4\x86\x8c\x8c\x3d\x3e\xa6\xe8\xf1\xd5\x66\x81\x1e\xa2\xcd\x66\xe0\x61\x2c\x21\x13\xb1\x03\xbf\x27\x57\xc0\x09\xa6\x3f\x72\x71\xf7\x91\xca\x5d\x45\x6f\x26\x72\xac\xd5\xe1\x68\xbc\x90\xb7\xb6\xf1\xbc\x98\x7e\x47\x0f\xe8\x04\xb4\x99\xef\x6e\x77\x5a\x27\x94\xf2\x92\x0f\x9f\x70\xd1\xd1\xb5\x49\x73\x8f\xae\x29\x39\x92\x07\xd7\x7c\x64\x2d\xab\x30\xa5\x24\xd0\x17\xf0\x0f\xb3\xe9\x7a\x69\x13\xcc\x97\x27\x9b\x1b\x88\x7c\xc8\xc1\x87\x03\x86\x5b\xcc\x6c\x5f\xca\xe9\x1e\x16\xdf\x32\x14\xcd\xc2\x56\xbb\x58\x99\xe4\xee\xcd\xbe\xd6\x31\x87\x5e\xf2\x9c\x6d\xcc\x18\xe9\x24\xa2\x55\x32\xac\x9a\x68\x5c\x61\xb3\x46\x88\x95\x1b\x7a\xa7\x46\x3c\x43\x3f\xac\x5e\x1f\x46\x85\xf8\x0d\xcc\xd8\x5f\x01\x9a\xcc\xf1\xf0\xa9\x59\x11\x3e\xac\x31\x24\xf1\xb9\x68\xb3\x22\x3b\xe7\x37\x8a\x1c\xba\xe8\x43\x3e\x27\xb5\x86\x45\x5f\x5f\x03\x2a\x50\x47\xf0\x1f\x47\xc2\x07\xc4\x39\x12\x6e\x55\x66\x57\xce\xf5\x29\x8a\x0d\x86\x22\x2f\xef\x20\xd9\x07\x30\xb8\x25\x6d\x05\x8d\x4f\xc9\xa6\x83\x38\xaa\x9b\x3a\xf7\x09\xdd\xa0\x30\xb9\xc8\x97\x96\x7d\xe3\xc9\x37\x69\xaa\xce\x64\x8c\xf0\x86\x0e\x45\xd5\x5d\x1e\xbc\x2e\x5a\x91\x93\x2e\x56\x3a\x89\x24\xf6\x41\xe0\x9b\x00\xfa\xc6\x32\xcc\xd2\x50\x5a\x3d\xf5\x1b\x31\x04\x7f\x6d\x7b\x00\xf5\x55\x49\xcc\x78\xe7\x30\x8b\xa2\x76\x0d\xb0\x62\xf4\xdd\x2b\xdb\x25\x8a\x71\x28\xf0\x2d\x22\xd0\xc1\x8e\x7b\x24\xef\xab\x36\xb4\xaa\x69\x83\xbb\x99\xed\x53\x26\xb1\x33\xbc\x5a\xa7\x7c\x3d\x3b\x97\x96\x13\x1e\x4f\xc8\x3f\x86\x3a\x1a\xb3\x2d\x51\x76\x0f\xc8\x0e\x16\xa1\x8f\xc5\xf4\xf9\x21\x06\x5c\x79\x1e\xd9\xb6\x0a\xfb\xfa\x0a\xe6\x97\x1a\x35\x37\xee\xca\xd6\xca\x1f\x24\xc4\xad\x02\x0c\x68\x64\xfd\xb4\x27\x0a\xdf\x09\x53\x8b\xab\x8e\xe3\x80\xba\x22\x62\xc5\xb9\x42\xa2\xfc\x12\x59\x44\x53\x67\x74\x25\x05\x62\x76\x05\x65\x85\x82\xc2\x7d\x35\x5a\x49\x39\x51\xa4\xb4\x88\x09\x43\xda\x52\x62\x01\x3b\x39\x5b\x64\x84\xc7\xd0\x3d\xcd\x68\x55\x4e\x32\xcd\x16\x95\xd1\x43\x46\xba\xe6\x50\xc7\xb6\x0c\xd2\x3e\x6a\xcf\x10\x7a\xdb\x98\xa2\x71\x0e\x94\x33\xa9\x3b\x7f\xde\xdd\x3d\xff\xf6\x87\x57\xbf\xc8\x73\x7f\x82\xe8\xb1\xf9\x03\x2b\x08\x32\x8a\x03\x22\xa9\xd2\x52\x01\xc1\x27\xdf\x26\x97\x25\xe2\x1b\x88\x7a\x2e\xd6\x93\x70\xf0\x00\x5d\x60\x67\x82\x36\x3a\x81\xe2\x94\xad\x5b\x8f\xd2\x2f\x1c\x44\xbb\xea\x44\xc3\x38\x00\x4f\x18\x61\x34\xa3\x6a\x1d\x1f\xd3\x9f\x16\x7d\xc0\xb5\x67\xe0\x03\x48\x27\xee\x91\xb2\x33\x60\x66\xa8\xbf\x07\x54\x8b\x56\x23\x05\x64\x1c\x50\x01\xbf\x07\x72\xd2\x9f\xe3\xe3\x47\x08\x36\xde\xe3\xde\x81\x1a\x2e\x08\x29\xf5\xee\x4e\x56\xbe\xdb\x9d\x4a\x73\x30\x34\xdd\xfe\x83\x0e\x61\xb9\x68\xeb\xa9\x10\xd4\x7a\x1d\x0d\x99\x0e\xe5\x1b\x0f\xad\x38\x0f\x55\xd1\x99\xbd\x3c\x85\x17\x74\xc6\xce\x26\x90\xe6\x98\x5e\x7e\xb0\xb1\x35\xa5\xd1\xd9\x3c\x1f\xd6\xab\x13\x7a\xca\x49\x17\x3a\x29\xe8\xcd\xaf\x66\xe9\xbc\x9b\xae\xbb\xd9\x15\xbd\x40\x8f\xd9\xb8\x99\x60\xad\x81\x36\x1e\x74\x50\xb8\xb5\x5f\x5c\x4b\x36\x7e\xb2\x8a\x37\x45\x62\x9d\x81\xb3\x2d\x10\x1e\xce\x8b\x33\x91\xd0\xe9\x04\xd9\xd0\xd9\xf8\xd8\x0a\x08\xa3\xf9\xbd\x0f\x32\x5a\x59\xd3\x9b\x39\xaa\x0e\x4e\x8d\x44\x51\x42\xef\x78\xd4\x97\x7b\x19\x98\x67\x44\x28\xe8\xad\x11\x17\x45\x20\xed\xd6\x28\xad\x67\xa8\x61\x35\x98\x3e\x71\x53\xe2\x2c\x52\xa4\x77\xd9\x36\xe1\x54\x17\xd6\x7f\x7a\x16\x70\x81\x03\xbe\x56\x81\x55\x3b\x7b\x8a\xbe\x4b\xce\xa2\x01\x68\x67\x8c\x9a\xb8\x76\xa6\x9e\xbe\xa0\xa8\xe6\x91\x3f\x6e\x2a\x60\x02\x0e\x7e\x72\xdc\xa8\xa7\x91\x8c\x91\xae\x41\xf6\x28\xd0\xa3\x01\x88\xfd\xd9\xd8\x27\xbc\x2c\x95\xbc\xda\xd1\x22\x2a\x74\x95\xb9\x70\xe2\x9f\xe6\x45\x10\xca\x2a\x6c\x2d\xd3\xca\x9e\xf1\x6f\x4a\x2c\x6d\xd8\xca\x72\x67\x39\x60\x2c\x6b\xf5\x23\x23\x9b\xc4\x62\x79\xb3\x53\x3f\x52\x42\x0d\x43\x14\x71\x1e\x96\x01\x3a\x69\x46\x6e\x02\x08\x1a\xe1\xc8\x5c\x44\x34\x31\x8b\xde\xfb\xa0\x2a\xc9\x97\x70\x29\x02\xd3\x61\x04\xc1\x78\x68\x03\x0f\x17\x98\xf2\xd1\x8f\xb1\xce\x45\xa9\x21\xdf\xdb\xd9\xa0\x15\x8a\xdd\xe7\x35\x35\x81\x24\x47\xb7\x06\xd9\x53\xd1\x33\x8b\x30\xa7\x12\x8d\xff\x3f\x7b\x4f\xff\xdb\xc6\xad\xe4\xef\xf7\x57\x24\xdb\x20\xdd\x3d\xd1\xb2\xe4\xd4\x07\xdc\xba\x1b\xe1\x5d\x9b\x7e\xe0\xda\xbb\xa2\x4d\xfb\x0e\x30\x8c\x40\xb6\xd6\x91\x1a\x59\xf2\x69\x65\x59\x8e\xbc\xff\xfb\x71\xbe\xc8\xe1\x2e\x25\xcb\xa9\x8b\x1e\xf0\x5e\x81\x3a\x5a\x72\x66\xf8\x3d\x1c\x0e\x87\x33\x76\x1b\xcd\xd8\xca\x09\x41\xd5\x4d\x2f\x99\x01\xe0\xbc\xe5\xb1\x70\x0c\x2a\x58\xe3\x1c\xb9\x12\x77\x64\x9a\xb1\x84\x42\xf1\x2c\x1d\xbf\xc5\x3e\xa1\x95\xe9\x8f\x68\xf0\x79\x7f\xef\xd6\x2c\xc1\x60\x8f\xe8\x05\x44\x83\xdd\x4a\xc1\x72\x68\xe9\x06\x36\xb8\x18\xef\xdd\x0d\x98\xf2\xc5\xcb\x6b\x96\x88\xd3\xaa\x5c\xba\x45\x1a\xae\xab\xbd\xd6\x9e\x5b\x73\xdb\x7b\x46\x4d\x68\xbb\xa2\xb3\x93\xa5\x9b\x94\x85\x7b\x8b\xc2\x2f\xba\x54\x84\x2e\xc7\x66\x54\xb8\x1d\x16\x1b\x8c\xe4\xc1\x3b\x06\x9c\xc2\x01\x3b\x5d\x46\xd8\xa9\x5d\xea\xcc\x38\x63\xc6\xb4\x9e\x01\x7b\x40\x65\xb0\x0a\x32\xeb\x96\x00\xaf\xa0\x0b\xf7\x42\x36\xb2\xcb\xc0\x15\xfe\x3b\x90\xa6\xd4\x03\x7d\x5b\xec\x2f\xf8\x24\x78\x3e\x1d\xd1\x0f\x7c\xb9\x74\xe2\xde\x03\x15\xfa\x71\xfe\x20\x15\xf8\x62\x83\xa9\x79\xe3\x9d\xbc\x21\xb3\xe7\xbc\x7b\xdc\x69\x3c\x54\x4e\xb3\xc3\xa3\xda\x95\x52\x08\x9d\x2c\xdf\x9f\xa4\x42\x8f\x83\x8a\x65\x9c\x42\xc9\xa8\x3d\x76\x0e\xbc\x83\xe6\x23\xf6\xcf\xf8\x6e\x8e\x3a\xe1\x84\x1c\xd7\x90\x87\xeb\x4a\xc0\xd0\xa5\x76\x0a\xef\x75\xc4\x1b\x77\x2b\xc7\x79\xde\x6e\xe6\x38\xef\xdb\x41\x0e\xa5\xa4\xf2\x88\x52\xb2\xbe\xa6\xb7\x92\x80\xb5\x9c\x96\x1e\x01\x1c\x3f\x66\x27\xae\x5e\x4a\xe4\x23\xf9\x8e\x2f\xa5\x0a\x0a\x2f\xd5\xf4\xe3\x0f\x78\xe8\x90\x04\xe6\x8c\xaf\xdc\x1b\x3a\x83\x92\x47\x4a\xe3\x88\xfb\xa1\x77\x93\xa1\x7f\xd8\xcb\x76\xa0\xfa\x6e\x89\xe0\x42\xfb\xd1\x9b\x3d\x3e\xb7\xd8\x45\xc5\x75\xe1\x36\x2a\x00\x10\x50\xc1\x5e\x0c\xa9\xf8\xee\x7e\xa8\x2e\x46\x85\x0f\x70\x14\x69\x00\x42\x92\xf2\x94\x1b\xa6\x09\x22\xbd\x99\x8d\x1a\x8d\xb1\xe3\xd3\xac\xc6\x52\xd6\x8e\x09\xc2\x28\xf8\xb3\x7f\x74\x34\x59\x34\x97\x3c\x7c\x2a\xa3\x91\x7c\x57\x47\xb0\x5c\x66\x1b\xcd\xf5\x6d\x0c\x4d\x32\x5b\x68\xbe\x33\x23\x68\x2e\xb3\x85\xc6\x4f\xc0\xdb\x28\x94\x11\x29\xc6\x76\x50\xb4\x04\x9b\x4e\xd0\xee\xc1\x85\x3d\x55\xfa\x99\xaa\x26\x9e\x9a\x3d\x6a\x0a\xf0\xd0\x21\xa1\xb3\x93\x98\x27\x47\x65\x70\x9e\x3f\xef\x1b\x5f\x89\x7c\x83\x2b\x2b\xe7\x31\xfe\x95\xaa\x44\x35\x0b\x13\x6b\x7e\xc7\x47\xa6\xb3\x31\x37\x8d\xce\xab\x92\xa1\xe9\xd1\x8c\x1b\xc1\x93\x86\x9f\x99\xf4\xf4\x8c\x69\x87\x98\x70\x59\x46\xb3\x75\x83\x61\x43\x0a\x8e\x1a\x82\x13\x08\x22\x01\x13\x1f\x41\x80\x60\xf2\x6f\x8b\x5c\x11\x2e\xb4\x9d\x50\xbc\x0a\x76\xc2\xf0\xf2\x6a\xc2\xf0\x03\x48\xe3\xc3\x89\xf8\x2a\x52\xf0\x10\xff\xed\xa2\x82\x60\xbf\x5c\x7c\xf8\xdb\xe2\xfd\xcd\x95\xed\x2d\x3b\x11\x22\x25\x57\x67\xe1\xda\x81\x97\x7b\xe1\x44\x85\x94\x70\xa1\x60\x4a\xb0\x06\x08\x4b\x26\x1f\x7c\xa9\x99\x0b\x9f\xb8\xe3\x6e\x54\xbc\x01\x38\x1a\xd8\x91\x68\x3c\xda\x81\x24\x1e\xa4\x01\xff\x9b\x3b\x85\x57\xb8\xfb\x52\x9d\xf8\x75\x94\xef\x17\xf6\x04\x42\x11\x2d\xd4\x6f\x8e\x9a\x51\x39\xb6\xe6\xba\x45\x9e\x87\xf3\xb3\x7b\x7b\xa4\x74\xc4\xa8\x95\x6e\xef\x73\xe1\x56\x88\x0a\x56\x20\xec\x72\x67\x6b\x8c\x2f\x13\xec\x88\xa1\x9e\x8a\x6a\x8a\x7f\xa3\xf9\x68\xb3\x7c\xb5\x54\x6d\xa1\x31\x55\xf5\xdf\x92\xb0\xab\x55\x64\xab\x90\x7b\x60\xaa\x47\x34\x96\x01\xfb\x1c\x2e\x6c\x2d\xd2\x51\x57\x5c\x85\xa0\x5c\x89\x0f\x74\x70\x6c\x07\xa7\xc4\x16\x62\x6d\xc8\x4f\x7d\x1c\x29\xa7\xb1\xeb\x9d\x35\x7c\xdc\x7a\x0b\x64\x3b\xe5\x06\x58\x1d\x8b\x58\x2b\x25\x5f\x7f\x2b\x0a\x42\x6b\xd0\xa3\xed\xa0\x30\x6a\x1a\xf4\x15\x83\xb6\xca\x51\x30\x5f\x6c\x23\x67\x61\x35\xad\xe3\xad\xc5\x2e\xe9\x99\xe7\xc8\xf9\x36\xe3\xf7\x6f\xca\xc1\x75\x21\x3b\x64\xc9\x61\x8b\x0a\xfc\x0b\xdf\xff\x42\x8b\x84\x84\xb1\x68\x24\x50\x19\x5a\x09\x61\x40\xc3\x0e\x31\xe1\xc0\x2a\x1d\xd1\xd6\x86\x25\x58\x52\xed\xf2\xd4\xae\x75\xd8\xa5\xfd\x48\x13\x2c\xd3\x66\xc4\x35\x9e\xf7\x68\x2d\xfc\x37\x26\x55\x83\x75\x27\x49\x9a\xd2\x13\xb7\xb1\x51\x13\x4a\xad\x55\xdc\x94\xfd\x6a\x82\xa0\x5c\x11\x42\x6b\x35\x12\x93\x6b\x1f\xbb\xa5\x8a\x0c\x4e\x8c\xb0\xe6\x8d\x2e\xdf\xd1\x54\xb9\x75\x33\x98\xd3\xbe\x55\x77\xf0\x6b\x45\x96\x92\xea\x66\xf8\xa7\xfd\x69\x32\xbc\xa6\x49\x49\x8a\x66\x18\x27\x71\x6d\xe8\xcd\xc4\xf3\x26\x41\xf7\x6a\x53\xef\x75\xb8\xf0\x67\x45\x13\xf6\xa0\xcf\x3c\xb7\xb3\x66\xc6\x38\x7b\xdd\x1f\x74\xee\xf2\x50\x4a\xb4\x02\x8a\xcd\xe8\x0d\x3a\x0e\xff\x74\x76\x16\xc2\x48\x80\x95\x20\x11\x43\xf7\xfd\x0e\xbe\xc8\x82\x2d\x15\x92\xe1\x37\x25\xf3\x1e\x6a\x13\x6d\x31\xd9\xcb\x97\x3c\x13\x1a\x7b\xb5\xe7\xcc\x7e\x67\x46\x22\xc1\x3e\x0c\xce\xf3\xa1\xbf\xea\x48\x44\xad\xfd\x07\xc3\xa3\x74\xf4\x80\xb8\xe4\x5a\x47\x32\xdb\x73\xf1\x39\x41\x46\x45\xa1\x1c\xac\x73\x57\x00\x01\xd4\xad\x20\x5f\xfb\xd7\x9a\x11\x6c\x3f\x76\x68\x61\x79\x1a\x9d\xc8\xfa\x72\xb9\x75\x2b\x52\xda\x23\x56\x31\x21\xf8\x32\x3d\x8d\x58\x99\x2e\xb7\x56\x01\xdf\xf6\x2b\x0d\x41\x99\x67\x10\x5a\x8b\x3e\x26\xd7\x3e\x40\xd3\x7e\x84\x01\x92\xe9\x22\x52\x8b\x2c\xa4\xd6\xdb\xc2\x3e\xed\x5f\x86\x13\xc7\x54\x61\x5e\x44\x5b\xeb\xd2\x5c\x32\x17\x1b\x89\x8c\xb5\x5f\xb9\xa1\x8c\xc7\x05\x37\x04\x3f\x57\x72\x98\x5e\x6f\x0d\xb3\xb5\x67\xd1\x81\x30\x29\x45\x87\x12\xa6\x2f\x3a\x48\xaf\xb7\x86\xec\xda\x7f\x31\x78\x59\xd7\xed\x33\x81\x00\xbc\xd6\x0b\xc1\xa7\xd7\xd1\x18\x60\x8f\x58\x0f\x61\x91\x4e\x7a\x5e\x07\x6b\x40\x17\xd5\x0c\x11\xb6\x5f\x59\x4a\x14\xe7\xc2\xb4\x70\xee\x4a\x53\x89\x5c\x1c\xce\xed\x70\x2f\xc5\xc9\x4e\xd9\x35\x6e\x13\xa1\x3c\x50\xa0\xa7\xfa\x3e\xbb\xa8\xef\x8b\x8b\xfa\x3e\xb9\xae\xef\x3b\x75\x0a\xd9\xcd\x6e\x24\x50\x19\x1b\xbd\x52\x08\x33\xf9\xa0\xa8\x65\x64\xfb\x2a\x61\xcb\xd0\xca\xd6\x51\x71\x76\xb6\x8e\x10\x86\x3f\x43\x2a\x61\xec\x33\x36\x85\x15\x32\xfc\x59\xfb\x0b\x48\xba\xa8\xdc\xa9\x13\x24\x90\xb6\x56\x70\x34\xa9\x2e\x6c\xdf\x97\x29\x79\xdf\x64\xff\xaf\x4a\xfd\xc7\x36\x75\xe4\x4a\xf0\x63\xd9\x4a\x9a\xea\xa4\xe5\xd8\x96\x37\x9e\x4f\x47\x92\xd6\xf2\x91\xf9\x75\xa8\x80\xbc\xac\x58\x6b\xe2\x85\xcd\x62\xfe\x8e\x2a\x4b\xc7\x25\xd8\x94\xd1\x6e\xdf\x80\x33\x6f\x43\x9a\x2f\xf6\x5a\xc2\xaf\x28\x57\x4f\x75\x76\xe2\xeb\x80\xd8\x11\xc1\xf8\x23\xce\xe5\xd3\x1d\x6f\x88\x14\xf8\x0f\x2a\x8e\x45\x37\x38\x7a\xc5\x9e\x26\xd5\xfb\xf0\x0c\x43\x4c\x0e\x52\xd2\x66\x62\x46\xe0\x56\xd1\x2f\xa0\xd0\x46\x3e\x5d\xd3\x71\x08\x3c\xdc\xf0\xbe\xcc\x0e\x0c\xc5\xc1\x16\x12\xc4\xbf\x2d\x17\xa0\x43\x73\x6e\x26\xc6\x47\xb5\xa6\x07\xa4\xf0\x2c\xbd\xb0\x7f\x0e\xfa\x67\x9d\x8f\xf8\xcf\xe1\x51\x07\x82\x3e\x18\xc8\xe9\x14\xe7\x87\x47\xc6\x9e\x23\xc0\x65\x1c\xd6\xd1\x87\xa7\x82\xf0\x87\x5c\x09\x9f\x98\xfa\x0a\x7b\x41\x3c\x08\xfc\x82\xba\x37\xa9\x69\x05\x76\xfb\xe0\xe6\x32\xbd\xac\x8a\xa9\x63\x66\xde\xc1\xcb\xcb\x97\x97\x15\xb9\xbd\x81\x4a\x69\x5b\x7a\x1d\x9e\xe6\x1b\x06\x07\x18\xa6\xcc\x5a\xac\xa0\x5b\x75\xb8\x69\x55\x65\x6a\x84\x6d\xf6\xe4\x5f\xa9\x1b\xeb\x8c\x8f\x9b\x95\xa9\x3e\x16\xac\xee\x38\x41\x06\x08\xcc\xb1\x2a\x3c\x2f\xf4\xb6\xfc\xd5\xc7\x4e\x71\xdc\xb1\xb9\x2f\x5f\x2e\xb9\xca\x91\xea\x22\xa6\x54\x97\x5f\x76\x73\xe8\x52\x33\xe3\x81\x6b\xdc\xfa\x91\x8b\x34\x18\x8b\xea\xe3\x89\xf7\xf0\xbd\x81\x27\xe2\xc0\x84\xde\x63\xac\x78\x52\x8b\x39\xbf\x5f\xe4\xcd\x34\x7f\x2e\x2e\xfd\x72\xad\x06\xcd\xf1\x6f\x6d\xe8\x38\x9d\x06\xe7\xe9\x5c\x05\x2b\x09\xf4\x02\xca\x15\xd1\xbe\xa7\x76\x33\x0a\xcf\x9e\xb5\xb6\x88\x2a\x4e\x29\x0c\xaf\xd9\xe7\x88\xae\x4e\xb3\x0f\x1d\xcd\x9b\x90\x47\xbb\x8e\xdb\x8a\x9f\xed\xc1\xa5\x38\x5a\xea\x72\x4a\x77\x0b\x84\xc0\xea\x7d\xb6\x98\x55\x19\xbf\x50\x02\x58\xbb\xb8\x0b\x84\x15\x19\xa8\xc8\xfd\x81\x77\x75\x15\xe0\xb0\x32\x9a\x29\x9a\x6d\x3c\xb3\xa1\x82\x8d\x6b\x79\x9d\x0e\x56\x81\x33\xe1\x08\x3c\x2f\xd1\x26\xc2\x74\x9b\x16\x79\xaa\x34\xc8\xed\x4a\xd0\xe5\xc6\x9a\xd7\x6f\xe1\xf5\x86\x71\xc0\xbb\x26\x60\xbb\x58\xa1\x48\xf3\xc9\x41\x76\xfa\x26\xd2\xa6\x10\x58\xc1\x3e\x5a\x87\xdc\x8e\xca\x24\xc8\x3c\x6c\xd4\x03\x5e\xbd\x4c\xc9\xa1\x82\x99\x16\x58\x25\x35\xe5\xd6\x9d\x45\x55\xce\xa1\xcb\xdf\xc6\x26\x0b\x9b\xea\xff\x06\xd3\x95\x74\x90\x76\x89\x17\xe1\x0e\x48\xdb\xaa\xfd\x8a\x6e\x81\xe4\x63\x4f\x82\x69\x11\x3b\xe3\x47\x1d\x91\xc4\xad\x2c\x4d\x60\xf1\x4d\xd5\x9f\xba\xaf\x52\xd0\xfe\x28\xe7\xf3\xb7\x9a\x1c\x47\xd1\x72\x3e\x70\x84\x0a\x23\xca\x1c\x10\xbc\xfe\xa2\x97\x60\xfb\x0f\x74\x89\x73\xf1\x78\xc6\x5c\xf1\x54\xe6\x1d\xb6\xb7\xc3\x5f\x67\xf5\x09\x3a\x4d\x0c\xae\x4e\xe9\x28\x67\x2b\xe3\xdc\x8e\x16\x0d\x97\xa4\x99\xd6\x58\x3e\x09\x8b\x85\xcd\x2b\xce\x64\xf1\x09\x59\x31\x29\xb4\x0b\x9a\x83\xfe\x80\xc5\xd5\xbc\x57\x14\x93\x01\x49\xab\x4e\xa0\x6d\xb0\x62\x6a\xcc\x53\xf3\xe2\x86\xb2\x72\xbb\x8e\xd4\x51\x54\x13\xbf\x3d\xc1\x77\xb2\x5f\x99\x85\x3e\xe7\x5b\x4e\x09\x18\xf0\x38\x64\xc0\x88\x26\x0f\x7a\x04\x21\xb6\x90\xe0\xf9\x2d\xcf\x1f\x78\x6f\xdb\x9a\x43\xe6\x92\x31\xc8\x65\x8b\x20\x9e\xb2\xe7\x6d\x88\x4f\x80\x42\xe0\x69\xcf\xf4\xcf\xc8\x58\xa6\x72\x7e\x3b\x9f\x87\x53\x6b\x9b\xe0\x8b\xe6\x09\xb4\x4c\xfa\xc7\x78\x44\x3a\xb1\xe4\x01\x19\xa8\x61\x78\x4b\xfc\xd5\xbd\x99\x55\xe3\xc9\xe5\x12\x0e\xad\xe8\x03\xdb\x81\x50\xb6\xab\xb4\xc0\xa3\x75\x0a\xbc\xff\x6d\x88\x23\x1a\xfc\xc4\x1b\xa0\x0e\x21\xa0\xe2\xdc\xca\x55\x29\x51\x05\xb3\x4c\x0a\x73\xe0\xbe\xe1\xb6\x9b\x3a\xb3\xcd\x90\x61\x0f\x93\xab\x6e\x80\xaa\x3f\x7d\x23\x8b\x15\xe2\xee\x60\x15\x07\x7b\xec\x5e\x46\x76\x04\xd7\x45\x84\xbe\x79\xdf\x64\x99\xec\x46\x74\x6c\xd3\x81\x73\x3a\x46\xf9\xfe\x9a\x1b\x4c\x00\x94\x0c\x66\x9a\xdb\xb6\x35\xde\xff\x8a\xf1\x98\x9a\xcb\x13\xfe\xba\x68\xf5\x8a\xb9\xdc\x26\x83\x16\x5f\x74\xac\x28\xed\xa5\xe6\xfb\xfb\xa5\x54\x43\xa0\x38\xec\xcf\x8e\x11\x72\x1b\x71\xf5\x71\xeb\x1e\xac\x60\x9c\x45\xd8\x93\x0b\x00\xdb\xab\xd8\xa2\xb9\x47\x6b\x1e\x12\x2b\x5a\x80\x7f\xad\x9c\x20\x2d\xda\x29\x28\x60\x38\x6d\xd1\x05\xf8\x87\xf4\x24\x2e\xd2\x4f\x5c\x6f\x94\x48\x66\x26\xf4\x9b\x4e\x5e\xf4\xb1\x52\xb7\xb8\x97\xc1\x0d\x2e\xda\xb1\xa0\xcf\x63\xb9\xfc\x16\x25\xc8\xbc\x79\xc1\x1d\x44\x40\x15\x1f\xba\xed\x7c\x7d\x7b\xac\x6e\x83\x8f\xcf\x74\x1f\xc1\x2d\xad\x92\x45\xe1\x33\x58\x74\xed\x2b\xde\x69\x70\x45\xcc\x1a\x14\xf7\x4b\x14\x25\x27\x41\xb0\xd0\xd8\xa5\x2f\x1d\xae\x2a\x7c\xb8\x8d\x9d\x78\x7f\x0f\x1d\x68\x3f\xb0\xf3\x42\x02\x96\x2c\x98\xf9\xb9\x22\xe4\x0e\x0c\x30\x98\x8f\x23\xef\x05\xcc\xfb\x7b\xa7\x9b\x51\xf1\x51\x06\x2d\x3d\x4a\xde\x10\xf9\x32\xdf\x84\xc6\xbd\x9d\x4a\x0f\x6e\xf8\x7c\xba\x0c\x82\x9b\xf9\x92\x53\x1b\x15\xa0\x77\xcf\xfb\x3d\xba\xe9\xc0\xeb\x3d\x40\x52\xda\x66\xa2\x95\xe5\x90\x5e\x07\xe1\x7c\xf7\x24\x0d\xa0\x42\x1b\xd1\x62\xc4\x21\xa3\xd6\xd1\x7f\xf7\x23\x0e\x90\x4c\x1b\x91\x22\xa4\x21\xbd\x0e\x63\x05\xef\x59\x71\x84\x95\x9a\x13\x62\xac\xea\x98\x53\x07\x91\x80\x9f\xe0\xbe\x51\xc8\xf3\x8d\x63\x10\x29\x78\xcf\xbe\x41\x58\xe9\x1d\xb9\xc8\x6b\xf7\x0f\xdf\xe7\x49\x0f\x31\xeb\xd8\xaf\x8b\x08\x58\xee\x59\x04\xb7\x13\xeb\x26\xca\x73\xe5\x3c\xfe\x8a\xd8\xdd\x0a\x13\x85\x2d\xf7\xc2\x01\x79\x57\xba\x5c\x09\x07\x6c\xec\x51\x57\x64\x9d\xb5\x27\xc6\x77\x45\x41\x74\xe7\xfd\x88\xad\xe4\x8e\x56\x68\xad\xf8\x86\x76\x4b\x9c\xe5\x3d\x2f\xa1\x14\x6f\xf5\xa4\x55\x6a\xbd\x2d\x3c\xf3\x9e\xc3\xac\x98\xb5\xa7\xaf\x52\xeb\xed\x41\x9d\xf7\xbd\x89\xd2\xfc\xdf\x97\x11\xa4\xd7\x5b\xa2\x56\xef\x7b\x53\xe7\x6f\x59\x7c\x1f\xa9\x2b\x96\x78\xe4\xec\xc7\x5e\xb2\x28\xe2\xfa\x52\x85\x89\x6f\xbf\xe7\x20\x00\x77\xd3\x11\xcc\xf1\x62\x23\xb7\x1c\x7c\xb9\xa1\xbc\xa8\xff\x38\x1f\x95\xd3\xf8\x0b\x69\xc8\x49\xf9\xb9\x07\x9c\xb4\xc4\x92\xcb\x3d\xbe\xc2\x8d\x16\xbf\xc8\x95\xba\xca\xa6\x8a\x3e\xef\x37\x8c\xfe\x91\xe6\x2e\xa3\x7f\x2c\xe6\x52\x39\x72\xad\x76\xf5\x9a\xaf\x19\xfc\x31\xe4\x60\xda\xa7\x6a\xb3\xf4\xb6\xb7\xf5\x07\xcc\x0c\x5c\x2b\xf9\x15\xf5\x07\x08\x95\xed\x69\xd3\xe1\x1c\xa2\x65\xa3\x5f\x5c\x70\x46\xdf\xca\xc5\x5f\x0d\x97\x8e\x69\x02\x0a\x8a\x04\x39\xdb\x0c\x1e\x04\xb4\xfc\x4e\x06\xee\x61\x87\xa7\x1f\x10\x08\x22\x0d\xbf\x7c\x89\x05\x90\x42\x20\x85\xfa\x18\x82\x38\xf3\x81\xb4\x1b\xc6\xf8\x04\x1b\x3a\x1a\x07\xbc\xc9\xec\xfa\x66\xe9\x86\x16\x88\x50\x74\xbb\xe5\x9a\x7e\x0c\xe4\x47\x8a\x90\xfa\xc1\x9d\x6a\x24\x8a\xa4\xe0\xd1\xdb\x81\x8c\x4a\xf0\xb2\x58\xce\x2e\x40\xbc\xe5\xc2\xb2\x60\x18\x14\x40\xab\x5a\xf2\x1a\x6e\x6d\xaa\xf9\xcd\xe2\xa2\x6c\x75\x28\x25\x4b\xdc\x3d\x34\xf7\x6a\xb4\x00\x0e\xa6\x08\x34\xa0\x7f\xb8\x73\xf3\x5e\xcb\xad\x27\x84\x17\x40\xc4\x77\x3e\xc4\x04\xe9\x4c\xd0\x9a\xe1\xed\xa2\xe4\x6f\x78\x9d\xfb\xee\x6a\xf8\xa1\x7c\xb7\x84\xb4\x35\xbf\x56\xe2\x61\xe0\x2a\x4d\xce\xa0\x5b\xc3\xb6\x92\x0a\xcd\x35\x92\x22\x25\x6f\xc2\xf9\x55\xc2\x79\x4c\x4d\x64\x46\x22\x58\xa3\x81\x9a\xfd\x1e\xa2\x91\x81\x6a\x04\x19\x57\xa6\x83\xd5\x4f\x43\x7b\x67\x31\x78\x5e\xb8\xbd\xd6\xbb\x0e\x0e\xaf\xe1\x9a\x13\xc6\x50\xdf\xda\x1e\x46\x13\x1f\x2f\x8f\x6d\x51\x14\x7d\x67\x9b\xa2\x18\x5b\x5b\x45\x8c\x2b\x70\x5b\xfe\x20\xc3\x61\x66\x07\x6e\xbe\x35\xc7\x21\xbf\x0d\x8a\x76\xcb\xd9\x38\x1a\x3a\xd3\xeb\x17\x9c\xbc\x57\xbc\xa4\xc8\xf8\xf9\x4a\x37\xdc\x3d\xd8\x93\xe2\xc2\x87\xb0\xa4\xee\xb9\x22\x4c\x72\x95\x2d\x90\x99\xfb\xd5\x1c\x67\x35\x47\x22\x30\xdc\x6f\xba\xdb\x15\x94\xf6\xe6\xdd\x6f\xf5\x62\xd3\xcf\x46\xdb\x45\x85\xf0\x1a\x1c\x08\x19\x3a\x67\x12\x15\x6c\x19\x6c\x10\x5e\x78\xdf\x05\x85\xd4\x83\xc2\xb5\x70\x47\xa9\x4d\x4c\x7b\xfb\x90\xae\xf1\x08\xd4\xa3\x11\xbf\x19\xed\xc7\x4b\x91\xfd\x55\xcd\x6e\x9e\x77\x0d\xaf\xf3\x2a\xcf\x04\xef\xd4\xc8\x2b\x07\xbf\xd1\x77\x3f\x23\x8f\xe5\xc1\xa8\x61\xe3\xed\x18\x52\x78\x5c\x24\x75\xc3\x1d\x93\xdf\x18\x41\xa0\x96\xe6\x7c\x5a\x05\x69\xe5\x34\x0c\x2f\xc3\xbd\x5e\x4e\xf5\xde\x4c\xf3\x92\x87\xb1\xfb\x0e\x4b\x10\x35\x27\x8a\x03\x9c\x41\xb3\x86\x7e\xbf\x53\xac\x42\xbd\xf4\x7c\xc7\xb3\x86\x81\x82\xb5\xa7\xc1\x86\x37\xcb\x39\x5c\x7c\x4b\x99\x72\xf6\x44\x23\x8c\x1e\x89\x25\x3d\xb1\xc1\xe8\xb1\xf1\x43\x4f\xa4\x0b\x17\xc1\x27\x10\x30\xc0\x5b\x5a\xb9\xd0\x69\xe3\xe1\x6c\x34\x0d\x93\x26\x73\xef\x22\xc7\x94\x53\xb7\x85\x4e\x96\x93\xe1\x14\x2e\xcd\xcb\xa9\xe6\xd9\x18\x1b\x67\x97\x7c\xb2\x93\xa7\x6f\x63\x18\xd4\x79\xf6\x94\xc3\x17\x28\xe9\xee\xfe\x35\x32\x68\xed\xda\x4a\x4e\x97\x7c\x49\xfc\x17\xba\x78\x57\xc3\x48\xf5\x4b\xc3\x1d\x60\xb9\x98\x5c\xb8\x35\xe7\x87\x22\xcb\x02\xf6\x45\x35\x78\x04\xc7\xdf\xda\x5a\x82\xb3\xcd\x15\xaf\x5f\xe9\x83\x73\xe5\x8f\x34\x99\x28\x04\x65\x3c\xb6\xe9\x84\xa5\xdb\x7e\xdd\x34\x53\x04\xab\x87\x87\x5a\x7e\xed\x4c\x15\xed\x2f\x89\x24\xc8\x8e\xf8\x00\x7f\xd0\xac\xc7\x27\xad\x06\x6a\x14\x45\x34\xbd\xc0\xb7\xa9\x40\x3b\x6f\xd0\xee\x35\x68\x83\xf1\x43\x80\x0f\x01\x18\x7c\xaf\xa7\xe1\xaa\x02\x46\xe4\x3b\x5f\xcf\x4a\x1e\x2f\x34\x02\x09\x56\x9d\x8b\x83\x05\x59\x8d\x0e\xbe\x16\x93\x4c\xdf\xc1\x52\x4f\xd7\xc1\xe8\x60\x6c\x72\x19\x36\xe3\xcb\x7e\x10\xac\xa5\xd1\xc4\x13\x72\xb1\x35\x2a\x82\x52\xcc\x39\x47\x6e\xa1\xf3\x0b\xf1\x6a\xfb\xaf\x73\xdc\xd7\x78\x7a\x43\xd4\xbe\x9f\xa1\x76\xac\x38\xef\xae\xfb\x5f\xf6\x06\xe4\xc0\xb5\x9c\x4c\xd3\x03\x48\xc9\x3a\x88\x66\x07\x63\x69\x21\xee\x5a\x10\x77\x0a\x62\x01\x34\x8e\x5e\xab\xae\x53\xb0\x1d\xc8\x3b\x50\x79\x1e\xef\x1c\x28\x0b\x1e\x75\x73\x88\x78\x27\x88\x3c\xc9\x05\xd3\xfb\x86\x1e\x15\x1b\x9c\x3c\x53\xb4\x2c\x5b\xf2\xcc\x59\xc8\x54\x3a\x0f\xa7\x50\x73\x3a\x6e\x9f\x32\xc4\x9c\x54\x28\x48\xcd\x32\x0e\xd2\x69\x67\x91\x65\xe1\x86\xd0\x86\xa5\x8c\x83\x74\xd9\x39\xcf\x62\x3c\x4b\xf7\xc9\xbe\xeb\xfb\x71\xfc\x82\x3d\x69\x6d\xd0\xf1\x45\x9e\xc8\x5d\x67\x33\x9d\x3e\x13\x88\xbf\x89\x68\x7a\x5e\x0b\x2c\xfa\xd7\x6b\x0a\x0f\x6e\x9b\x72\x73\x1a\x9d\xb7\x3e\xc4\x35\x04\x0d\x94\xc1\x16\xde\xad\x45\x1f\xb7\x0e\x9f\x5c\x3d\xaa\xc1\xe1\xf2\x13\x4a\xa1\xa4\xc3\xfb\xa7\x17\xda\x94\x51\x60\xa4\xb2\xe2\x39\x9f\x8d\xfd\xa0\x9a\xb8\x61\xca\xee\x9a\x99\xa4\x5a\xbd\x8f\x64\xdb\xd4\xcc\xed\xc6\xcf\x7d\xb6\xec\xcf\x78\x2f\xbf\x7d\x53\x7f\xe4\x9e\x80\xc2\x0d\x63\x10\xb9\xb4\xd1\x1b\x93\x79\x78\x20\x7d\xb4\xc2\x81\xa6\x25\x80\x12\x7c\x40\x5d\x65\x66\x7f\x5c\xff\xc0\xd4\x20\xd6\x10\x7b\x73\xa3\x03\xbd\x0b\x7f\x07\x74\x5a\xc6\x7c\x23\xe3\x23\x37\x8d\x4e\x3f\x9c\x15\xa1\x2d\xc1\xdf\xac\x7c\x2c\x8a\x03\x33\xaa\xcd\xa6\x0e\xf4\x07\x41\xa9\x62\x0e\x12\x91\x1c\x75\xeb\xae\x42\x2d\x12\x7e\xee\xd7\x8f\xf0\x68\x02\xfe\x75\x33\x82\x48\x5d\x29\x39\x94\x37\x18\x39\x30\xca\x7e\x43\xfc\x9d\x8a\x8a\x8c\x41\xa8\x06\x91\xa9\xa3\x45\x65\x89\xb8\x56\xae\xbe\x23\x92\x66\x45\x87\x8e\xdb\x62\x25\x9b\xde\x18\x7e\xfa\x3d\x0f\xbe\xc4\xe4\x4f\x3a\xd8\x85\x9a\xb0\x14\x5d\xd4\x89\xd1\x64\xd5\x85\x98\x12\x49\xe6\x5d\x68\x38\x59\xbc\xc4\x30\xb2\x0a\x4b\x82\x53\x58\xac\x68\x64\x8a\xc4\x45\x9c\xb8\x9e\x93\xb3\x00\x9b\xb1\x28\xa7\x18\xec\xdb\x85\x9d\x30\x2b\xbf\xd4\xd1\xf3\x8a\xae\x17\xa1\x73\x48\x8b\x54\x41\xda\xb3\xdf\xfd\xfd\x2d\x06\xcb\xf3\xc5\x48\xa8\x8b\x00\xb0\x6f\x01\xc7\x0d\x40\xdb\xb2\x05\x28\xc1\x6c\x7d\x60\x1b\x49\xb0\x12\x6e\x19\xa7\xea\x83\x9c\x81\xca\x12\xec\xfe\xcc\xc9\x59\x28\x8c\x43\x7c\x01\x12\x2e\xd4\xb8\xe0\x18\xc8\x10\x79\x31\x3f\x0d\x08\x32\x6c\x93\x1e\x6c\x62\xab\x8c\x27\xcb\x8a\xe7\x46\x40\x1d\xdc\x11\xba\x8f\x2e\x13\xb7\xab\xb7\x7d\x50\xb3\x92\xbe\x2f\xbe\x0b\x09\x64\x54\x34\x16\x2c\x7f\x6c\x6b\x32\xda\xd0\x83\x49\xd5\x38\x58\x2a\xb1\x0b\x39\x96\x5e\x86\x2c\xbe\xb0\xa3\xc7\x16\xf9\xd0\x71\x4f\x40\xd5\x57\x94\xad\xa8\x82\x64\xe3\x96\x65\x84\xe8\xe5\xe5\xc3\x54\x2f\x2f\x1f\x4b\x96\x3d\x55\x85\x22\x9f\xfd\x53\xd8\xff\x41\x8b\x7a\xe2\x0e\xb4\xb8\x0c\xf1\xc0\x0e\x79\xce\x35\xcf\x20\xe2\x68\x23\xd5\x00\xe0\x8d\x95\x22\x01\x05\x81\x46\xf0\x54\xc7\xdc\x4b\xfd\xbe\xbf\x4f\xe9\x8b\x7a\x9a\x5c\x8c\x51\xc8\x2c\x9d\xce\x3e\x25\x49\x7f\x00\x05\x90\x07\x4d\xf8\x25\x83\x02\x59\xec\xfe\x09\xfd\x9f\xa4\xcd\xe1\x26\x7a\xcd\x71\xd6\xa5\x84\xe3\x5c\x67\x18\x71\x56\x00\x55\x59\x98\xcc\x22\x1b\xf6\x20\x85\x7f\x55\x2a\x02\x89\xf3\x1a\xdf\xd5\x1e\xe8\x7d\x5a\x55\x4c\x89\x7d\x7d\xa7\x5e\x33\x99\x89\x38\xa6\x94\x7b\x99\x13\x91\x30\x51\x42\xe0\x3a\x79\x88\x92\xe5\x2b\x73\xa2\x07\xb6\xca\xb3\x07\x1f\x9a\xc7\x6f\xad\xea\xa5\x2e\xab\x80\xa7\x73\x38\x5a\xad\x5c\xfc\x3d\x95\x9c\xe1\x90\xc0\xa7\x03\x90\x04\xca\x82\x5d\x66\xa5\xf9\x03\x25\x52\xe6\x18\x58\xd9\x73\x08\x4c\x07\x40\xb6\x7f\x92\xab\xf9\x4d\x55\x42\x6a\xa2\x62\xdd\xae\x44\x9f\x85\x0a\x9d\x30\xa0\xa7\xe5\x9b\x08\x9e\xf1\x56\xd5\x10\x2c\x29\x93\x06\x18\x1d\xd9\x82\x9a\x47\x95\x74\xb3\x7c\xa2\x82\x58\x82\x6d\x94\x64\x56\x35\x5a\x53\xfd\x12\xc4\x65\x66\xc3\x86\x30\x56\x33\xeb\x8d\x30\x95\x83\x23\xfb\x08\xcb\xf8\x53\xa2\x20\xf7\x28\x80\xf1\xa9\xf8\xc9\x84\x5f\xa0\xe5\xca\x31\xc2\x1e\x5a\xac\x9c\xd5\x86\x3c\x5c\x8a\x19\x97\x90\x36\x55\xa0\x7a\xa1\xf8\xc9\xfb\xab\x5e\x42\xbe\xe4\xa3\x41\x17\xb7\x0f\xeb\xae\x23\xb8\xa2\x8f\x68\x21\xb7\x55\x01\x11\x6c\x77\x66\xda\xe3\x54\xb0\x8a\xe0\x3b\xa8\x55\x8b\x40\x28\x39\xe2\x55\x89\x15\x7b\x87\x6a\x13\xa1\xcf\x01\xfd\xd3\x05\x88\x02\xad\x60\x29\xa1\xd0\x6a\x08\x8c\x34\x3b\x20\x23\x59\xf8\x53\xe7\x74\xf7\xe2\x2a\x82\xab\x09\x6d\xfe\xb8\x90\x88\xca\x3c\x78\xf6\xfd\x67\x57\x88\xe6\xd5\xee\x2a\x85\x5e\x92\x1a\x95\xf1\xa4\x50\xfd\xba\x93\x10\xc6\x7d\x0d\x4e\x44\x86\xe2\xa0\xc2\x5f\x8a\x69\x4b\xce\xcb\xc0\xf6\x0b\x81\xc9\x0c\x0c\x9f\x3f\x21\xa8\xb2\xef\x72\xd7\xa3\x58\x38\x29\xd6\xd9\x9a\xd2\x92\x24\x0b\xe0\xf1\x90\x14\x62\xe2\xc0\x04\x6c\x60\xf3\xb0\x6f\xe7\xe7\xbf\x2b\xd7\x02\xb6\x04\xb4\x93\xb5\xa9\x83\xd4\xfe\xa1\xe9\xe1\x00\xf3\x48\xdf\xe2\x0a\x6d\xf4\x2d\x36\xfa\xfa\xa1\x92\x7c\x5b\xf8\x7e\x50\xce\x1d\x00\xd9\x3a\x76\xcc\xe1\xd8\x41\x9b\x88\xcd\xb7\xa7\x0b\x59\xc3\x73\x3e\x83\xb8\x37\xc2\xab\x6c\xb0\xca\xc5\xa1\x51\x6d\xe6\x78\x06\xe1\x7a\x8d\xe6\xb3\x32\x8f\x09\x1f\xc3\x65\x5d\x87\xeb\x12\x7c\xab\x47\x1c\x8d\xad\x74\x04\x7a\xc5\x6e\x1c\x84\xa6\x02\x41\xc9\xb7\xc9\x3a\x80\x65\x99\x17\xfc\x65\x27\x5f\x4d\x36\x19\x7a\xf4\x02\xce\x9a\x56\xe4\xbe\x6b\x5c\x0e\xad\x38\x54\x55\x1c\x5f\x5c\x3e\xf7\xd0\xe1\x6b\x05\x3e\x8a\xaf\x74\x30\x27\x29\x8b\xf6\x48\x7d\x3e\x76\xf3\xd1\x9f\x2f\x28\xe6\xdd\xff\x3f\x6d\x3f\x3e\xd2\xda\x4f\xc9\xe9\x35\x01\x58\x53\x0a\x08\xe0\x0e\x0d\xa6\xa9\x10\x78\x94\x0e\x5f\x9f\x54\xf7\x38\xa6\xe2\x19\x55\x1d\x50\xf1\x00\xd7\xae\x41\x78\x04\x0d\xcf\x9f\x9f\x74\x6b\xb0\xeb\xd2\x40\x69\x64\x75\x1d\xb6\x5e\x05\x84\x95\x7b\x2a\x4d\xff\x03\x8a\x7e\xad\xda\xdf\x5a\xcb\x98\x76\x2f\xac\xed\x3f\xb4\x72\x7e\xfb\x0c\xfb\x4b\x74\xea\xe1\x59\xe5\x9f\x2a\xf5\x7f\x0c\x95\xfa\x5f\xa6\x3f\xdf\xbe\xa8\x01\x97\x5c\x04\xff\x53\x97\xfb\xc4\xba\xdc\xf6\xcd\xc0\x16\x6d\x95\x46\x62\xb9\x64\x1b\x0a\x65\xb7\x11\xfe\x56\xdd\xcd\x2e\xa2\xce\x62\xc5\x0a\xf1\x76\x38\x01\x97\x06\x3d\xcb\x49\x17\x5d\xd0\x92\xda\x56\x7c\x0f\xb1\x66\xec\x98\x0c\x52\xad\x23\xc9\x9c\x80\xc7\x7a\x15\xbe\x97\xc8\x72\xbb\xa4\xdf\x4e\xae\xe0\x80\x9d\x02\x39\xd3\xef\x91\xc7\xed\x45\x11\x36\xc8\x38\xdd\xd3\x49\xab\xac\xd7\xbd\x01\x55\x25\x77\xa5\xe8\xa6\x05\xf3\xa3\x5a\xbd\xd7\x3d\x01\x51\x8b\xe0\x6a\x04\xee\x3e\x9c\xc8\xd6\x0a\x05\x7d\xdd\xe0\xb8\xb7\x85\x5a\xbe\x9d\xf4\x7a\x00\x3e\xea\x2f\x97\x9d\xeb\x2e\x6f\x39\x10\x49\x4e\xaf\x60\x82\xb1\x7c\xc7\x82\xc8\xf6\x94\x19\xa8\x8b\x57\x05\xcb\x1a\xf7\x8a\x6a\xa8\x98\x28\x90\x29\x60\xe7\x77\x6f\x7f\xfc\xc1\xf6\xe9\xf5\x74\x68\xe7\xe1\xe1\xb3\xf1\xa2\xbc\x2c\x0e\xdf\x9b\xe4\xd9\x7a\x3a\x99\x7d\xc8\xf1\x3b\x91\x59\xf6\xf9\x97\x96\xc0\x33\x62\x66\xc9\xe7\x9d\xdb\xce\xe7\xc9\x33\xe6\x57\xf6\x73\x0c\x9f\x9f\x77\xfc\xce\x60\x81\xc1\x7b\x2b\x18\x73\x97\x9d\xe4\x75\xd2\xb1\x09\x9d\xe4\xcb\x43\xfb\xcf\xeb\xe4\x21\x8d\xbd\x04\x84\x55\xfd\xd2\xe8\x02\xd3\xda\xb8\xb8\x9a\xee\x6a\xca\xf5\xff\xc0\x31\xb9\x5f\x7e\xfb\x36\x15\xad\x73\xee\x52\xbf\xc2\x61\x75\x19\xad\x55\xe2\x61\x94\x5c\xc7\xc0\x58\x4f\xce\x6c\x45\xa8\x36\xcb\xdb\xe2\x16\xde\x8f\xf2\x70\xda\x1f\x0b\x96\x93\x0a\x7c\x57\x4a\x43\x68\xff\xa5\x41\x34\xbc\xb4\x42\xf9\xdf\x4a\xc5\x5c\xc3\x25\xa8\x3b\xec\xdc\x5f\xae\x0b\x8e\x04\xd5\x0c\x3e\x7d\x62\xf3\x20\x72\x1e\xba\x1a\x06\x73\xfd\xb4\x8f\xc1\xa4\xfb\x46\x6a\x61\xb8\xd8\xac\x21\x7d\x77\x25\xd4\xa0\xa5\xd0\xca\x63\x13\x01\x69\x74\xb3\x73\x6c\xb7\x46\x7a\x26\xda\xf2\x58\xb3\x4f\x1a\xa5\x45\x6e\xff\x4c\xac\xe8\x3f\x59\xdd\xfe\x38\x15\xf6\x93\xea\x9f\xf1\xcc\x17\x55\x3f\x7f\x92\xa6\x38\x38\x91\x76\xff\x88\xce\xf8\xb6\xf0\xfa\x61\xbb\x20\x95\x66\xd8\x5c\x17\x5a\x25\x6c\xe5\xb8\xb8\xca\xd7\xac\xe4\x44\x1a\xd4\x8a\xa6\x8d\x59\xfc\x39\x1a\x61\x56\x87\xba\x12\x9b\x33\xc7\x42\x9a\xc8\x86\x34\x5a\x0c\x6f\xd3\x8b\xf1\x70\x01\xd6\x75\x8b\x3b\x7f\x80\xc7\xb4\x94\xb4\x2d\x52\x8a\x91\xb6\xe6\xba\xe1\x5e\x66\xca\x4e\x84\x33\xe9\xfc\x81\xdb\x62\x70\xcc\x37\x16\x84\x6e\x03\xec\x8f\xd4\x5d\x0d\xa8\x1d\x34\x55\x3b\x28\xee\x46\x9b\x06\x05\x4a\xce\x79\x1b\x06\xfb\xc2\x8b\x21\x84\xc9\x2d\x17\x0b\x05\x5b\x82\xe3\x7a\x0b\x0f\x4f\x07\x03\x1d\x0a\x54\x0e\x55\x28\xd0\xf8\xad\x93\x07\x15\x1f\x75\x3a\x7a\x65\x12\x7b\x2c\x28\x2f\x27\xb3\x12\x7c\x03\xc1\xba\x99\x5f\x3e\xb3\xd3\x70\x0e\xca\x96\x01\x94\x90\xcb\x57\x06\x51\x9a\x6f\xa6\x43\xbc\xce\x85\x60\xd9\x49\xb9\xbc\x18\x25\x06\x5e\xc1\x5f\x12\xd9\x34\x79\x63\x93\x7e\x3b\xb2\x89\xc9\x8b\xf1\x72\x79\xad\xb4\xe4\xf8\xad\x06\x07\x02\x0f\xce\x6e\xae\x5d\x74\x7c\xbe\xe2\xb5\x03\x53\x61\x44\x7d\xdb\x8a\xa9\xdd\xc4\x93\x43\xb7\x81\x3d\xeb\xe1\x19\x6d\xe1\x25\xc3\x24\xc9\x53\xc1\x80\xe4\xcb\xc9\x74\xa9\xbd\xc3\x3b\xf5\x6d\x02\x3b\xfa\xaa\xce\x8c\x86\xfe\x7d\x3e\x99\x21\x7d\x15\xf9\xca\xce\xed\xff\x2c\xef\x20\x0e\x01\x6c\x7e\xac\x71\x2f\xa7\x97\xca\xcd\x2c\x7c\x92\xa2\x54\x37\x41\x30\x0c\x66\x5b\xbe\xfe\x13\xda\x3b\x34\xb7\x44\x2a\xdf\x91\xd8\xa7\x91\x5c\xa5\x24\xc9\xb8\xb1\xdd\xeb\xf9\xb5\x65\x0c\x9c\xde\x6c\x8b\x1d\x5a\x24\x0f\x7d\xd8\x2e\x1d\xf6\xfe\xc3\xa4\xd3\xa8\xf9\x4f\x56\x50\x98\xac\x3b\xae\x5a\x41\x0c\x7b\x8a\xa8\x40\x20\xdd\xea\xe6\xbc\xa2\x83\x78\xcf\xf8\x54\x79\x89\x9e\x0d\x5c\x1a\x06\x0d\xac\x5d\x6f\x44\xc4\x4d\x9c\x10\x90\x97\xba\xfa\xa6\xae\xf6\x81\x6d\xaf\x25\x8a\xee\x2d\x43\xd4\x74\x73\xb3\x98\xe6\x0a\x97\xee\x37\x5e\x74\x51\x63\x9c\xb2\xce\x52\x70\xed\xd8\x53\x80\xc4\x3c\xf9\xe9\xd7\xb7\x89\x81\x65\x51\x2e\xaa\x7c\x93\xe0\x06\x3c\x5b\x1e\xbc\xb5\x93\x3f\xc9\x13\xb8\x28\x85\x27\x06\xb6\xe0\xc3\xf5\xc1\xed\xed\xed\x01\xec\xc3\x07\xb6\x2c\x62\xee\xa3\xa4\xae\xa5\x96\x96\xe5\x97\xcb\xd2\x8e\xc2\xb6\xe6\xb5\xeb\x28\x95\xf8\xfa\xcd\x0f\x6f\xde\xbe\x79\x9a\x7a\x04\xd3\xf7\x97\xe5\xd0\xf6\xa8\xfd\xf3\xc8\x09\xec\x50\xe2\xd3\x87\xd7\x51\x63\xee\x00\x52\xd5\x9a\x3d\x9f\x3c\xec\x35\x87\x09\x21\x8a\x76\xa6\xae\x8e\x0e\xe1\xc0\x75\x98\x18\x55\x12\xa5\x63\xc2\x61\xe2\x37\x19\x18\x0a\xa6\x3e\xa2\xd8\xd4\x55\x97\x7b\xd7\x0a\x47\x57\x57\xf3\xd9\x69\xf2\x3f\x07\x3f\x93\x51\x79\x39\x3a\xf8\xfb\x64\x39\x4e\xce\xcc\xc6\x56\x05\xfa\x2c\xe7\xbe\x33\xf6\xdb\x8e\x68\x4e\xcb\xab\xae\xcf\xe2\x7c\x0f\x40\x2b\xe0\x73\xb3\xf7\x3f\x43\xa0\xfc\xc4\x10\x3b\xb4\xcc\x90\x04\xe8\xd4\xb2\xc0\x05\xe4\xfc\xb4\x98\x83\xcb\x4e\x7d\x35\x19\x66\x64\x9b\xf0\xbb\x7b\x3b\x2e\x71\x31\x9b\x8d\x95\x0e\xae\x21\x52\xc9\xaf\x76\x16\x25\xb0\x95\x54\xd4\xee\xee\x78\x79\x35\x4d\x0c\xc8\x7c\x8b\xf9\x74\x6a\x37\xad\x04\x2b\xf4\xd5\x72\x31\x85\x73\xfe\x7c\x39\x2e\x17\xb7\x93\xca\x1e\xf5\x3f\x91\x44\x4d\x0d\xe1\xbc\x54\xe5\x01\x6f\xaf\x2e\xe6\xd7\xd0\x64\x61\xf7\x09\x92\xfc\x0d\xec\x5f\x54\x2b\x11\xca\x10\x8c\x71\x10\x81\x17\xbc\x21\x75\xa4\x9d\x1e\x04\xd6\xe5\xd1\x48\x93\x29\x0e\x9d\x3d\xf0\xc0\x54\xb1\x87\xa0\x1b\x7c\x95\x95\x36\xce\xf2\x54\x46\x97\x80\x91\x14\x3a\x4f\x31\x41\x3a\x4c\x6b\xf1\xa9\x02\xdf\x92\x7b\x35\xbc\x18\x63\x80\x89\xd3\xb3\x66\x12\xdd\xfe\x6c\xa0\xe3\x66\x17\x77\xf9\x66\xb8\x2a\x17\x10\x3b\xb5\x87\x91\x4d\xec\x32\xb5\xbf\xae\xac\xa0\x7b\x75\x03\x9a\xc5\xab\xe1\x9a\x7f\xd9\x56\xce\x46\xc3\xc5\xe8\xeb\x72\x35\xc1\xb5\x0b\xfa\x46\xbc\x31\x53\xe5\x5b\x5e\xf4\xa2\x5b\x82\xbd\x0a\xbb\x88\x99\x4e\xe7\xb7\x76\x9a\xaa\x37\x09\x76\x53\x5e\xd3\x73\xfe\x8c\x18\x19\x2d\x58\x4a\x8f\xd6\x95\x80\x81\x74\x23\xb7\x9a\x6b\x23\x87\xa1\x39\xf7\x9a\x16\x24\xfa\xe5\x39\xfe\x33\x38\xe8\xe7\x94\xf0\x9a\x13\xfa\xb6\xee\x19\xca\x11\xdf\x2e\x86\xd7\x63\x7c\xeb\x10\xc8\x56\x9c\xec\xc7\x13\xe5\x90\x14\xc4\x8f\xe6\xd5\x0e\x8a\x24\x5e\xf4\x21\x99\x8c\xc5\xb0\x72\x9a\x73\x9d\xdf\x03\x41\xe0\x83\x43\x5b\xf3\x05\xf1\xf3\x0d\x4e\x9c\xbc\xd1\x2a\xcb\xfb\x02\xad\xb9\xe8\xc9\xc4\xcc\x41\x84\xb6\x9a\xd4\x07\x76\xf4\x96\xe2\x8a\xf6\x55\xcf\x8c\xe7\x8b\x8f\xf2\xd9\x3f\x66\x07\x4a\x2f\xd2\x68\x2d\xa4\x9c\xec\x40\x61\xf1\x9d\xcf\x76\x1c\xae\x4e\x76\xa0\x4a\x3e\xe1\xee\x71\x0b\xa1\x8e\x22\x17\xc9\x67\x3c\xed\x12\xa3\x01\x7e\x9b\x54\x93\xf3\x89\x15\x17\xee\x0a\x64\x35\x07\x98\x7a\x50\x8d\xc1\xa4\x8b\x01\x97\xc3\xf3\x69\xd9\x02\xc4\xd4\x83\xb1\xe5\x2d\x0e\x10\xb0\xde\x42\xb2\xe6\xd0\x7b\x10\x59\x94\xab\x72\x68\xf9\x83\xa6\x83\xd3\xe0\x91\x74\xb0\x32\x8e\x8a\x5d\xe2\xdf\x35\xee\x35\xfc\x7e\x61\x47\x78\x36\x9a\xdf\xfa\x3e\xd5\x68\x7f\x0f\xaf\x6b\x22\x58\x3c\x7a\x0e\xe9\xc5\x2d\x8a\xd3\xcd\x92\x8d\xae\x7f\x9a\x74\xb1\xae\x78\x39\x67\xf9\xa0\xe5\x3b\xce\xce\xae\x89\x68\xc7\xf8\x38\x33\x8a\x93\xf9\x25\xd8\x2a\x8a\x5d\x6b\xe9\xda\x6a\x34\xaa\x72\x77\x3e\xa3\xf3\x7a\xa4\x4b\x5f\x90\x19\x17\x6e\x97\xcb\xef\x41\xab\x6b\x97\x7d\x1a\x27\x68\x5a\x38\xc7\xbd\x1e\xf2\x75\x27\xad\xc7\xb8\xb6\x74\xe1\x46\xec\x46\xc8\xfb\x2a\x5f\xa7\x1c\x8b\x36\xfd\x0b\xb9\x4f\xe9\x5b\x06\x47\x86\x25\xec\xcf\x0b\x89\xda\xb1\xe5\x4f\xe7\x7b\x14\xce\x8e\xf8\x04\x3e\x3f\x3d\xee\x99\x7e\xaf\x77\x56\x7b\x53\x14\x06\xbe\x6b\x39\x40\x24\x97\x5e\xae\xf7\xd9\x07\x18\x1d\xdd\xb8\x28\x73\x39\x29\xa7\x56\xa4\x42\xe6\x68\xa5\x22\x29\x79\xed\xb0\xd9\x96\x92\x90\x7f\xb4\xf8\xe2\x4e\xf1\xc7\xe1\xda\x36\xa0\x67\x66\x93\x8b\xd2\xbb\x1a\xeb\xed\x2c\x86\x18\x39\xad\xd0\x2e\xef\x06\xaa\xd4\x0b\x08\xf1\x23\xcd\x60\xc7\x50\xce\x75\x99\xed\x46\x6a\xbb\xe9\x97\xff\xee\xfc\x95\x25\x9f\xf5\x7a\x5f\xff\xc7\xd1\x17\x76\x1f\xfd\xec\x9b\x6f\xbe\xea\xf5\x7a\xf0\xeb\xe2\x8b\x5e\xef\xe8\x48\xfd\x82\xfe\x22\x83\x1d\x76\x68\x63\x1b\x48\x11\x97\xe0\x17\xfa\x96\xca\xff\x8d\x36\x9b\xe4\x07\xaa\xdf\xb3\xf4\xaa\xca\x60\x2c\x08\xe1\xce\x21\xd8\x5f\xda\xf1\x0d\x21\x6f\xc8\xc1\x85\x5c\xf7\x27\x68\x25\x47\x76\xdb\xd0\x40\x1f\xe7\xe5\x21\x48\xf2\x74\x93\x6f\xc0\x21\xc7\x36\x18\x30\x04\x69\xd1\xf9\xec\xd5\xab\x57\xb6\xba\x94\x8a\xab\x45\xb2\xfa\xb5\xfd\xcf\x19\x2a\x71\x7b\x20\x7c\xb9\x1d\x17\x30\xc4\x08\x86\xaa\x8e\x78\xf5\xd9\xac\xf3\x8d\xef\x2d\x22\x6a\xa7\xee\xfa\x48\x27\xef\x1c\x62\x73\xe7\x40\xef\x1a\x93\x4e\x62\x5b\x1d\xf4\x6b\x31\xbd\xe2\x7a\xbf\xaa\x0d\xf5\x02\x63\xf2\xf4\xd8\x3d\x97\x6a\x37\x62\xe4\x79\xe1\xd3\xda\xf8\xa9\x8d\xa9\xd1\xff\x90\x34\xe0\xb8\x27\x2d\x90\x31\x82\xf9\x89\xa3\x61\x37\xe1\x93\xff\x0b\x00\x00\xff\xff\xe2\xc7\x3f\x61\x49\xdb\x01\x00" + // scripts_stats_scripts_js returns raw, uncompressed file data. func scripts_stats_scripts_js() []byte { - gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ -0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0xec,0xbd, -0xfb,0x7f,0xdb,0xb6,0x92,0x28,0xfe,0xfb,0xfd,0x2b,0x1c,0x9e, -0xbb,0x2e,0x69,0x41,0xb2,0xa4,0x34,0x7d,0x50,0x65,0xf4,0x49, -0xdb,0xf4,0x9c,0xdc,0x6f,0xfa,0xd8,0x36,0xa7,0xe7,0x74,0x15, -0x6d,0x3e,0x94,0x44,0xcb,0x8c,0x65,0x51,0x21,0x69,0x9b,0x8c, -0xad,0xfd,0xdb,0xbf,0xf3,0xc0,0x93,0xa2,0x64,0xa7,0xaf,0xbb, -0x7b,0x77,0x4f,0x4f,0x64,0x02,0x18,0x0c,0x06,0x83,0x01,0x30, -0x00,0x06,0x83,0xeb,0x65,0x74,0x76,0xb5,0x9e,0x97,0x69,0xb6, -0xf6,0x17,0x8f,0x45,0x99,0x6d,0xb2,0xb7,0x45,0xb6,0x0e,0x6e, -0x55,0xec,0xd1,0xf5,0xf2,0x4d,0x52,0xcc,0xe3,0x4d,0xf2,0xa6, -0x28,0x73,0xbf,0x0a,0x6e,0xf3,0xa4,0xbc,0xca,0xd7,0x47,0x55, -0x2f,0x4f,0x36,0xab,0x78,0x9e,0xf8,0x26,0xf9,0x4d,0x9e,0x08, -0xef,0x7f,0x0f,0x5e,0xbf,0xfe,0xc8,0x0b,0xb6,0x36,0x86,0x32, -0x87,0x40,0x5c,0x26,0xdf,0xaf,0xff,0x91,0xe5,0x0b,0xbf,0x10, -0xab,0x64,0x2d,0xf2,0xe4,0x3a,0xb8,0xbd,0x8e,0xf3,0xa3,0xf9, -0xba,0x8c,0xfa,0x50,0xf6,0x45,0x54,0xf4,0x8a,0xcd,0x2a,0x2d, -0x7d,0x2b,0xcb,0x9b,0x1b,0xc8,0x01,0x88,0x83,0x91,0x2c,0xb8, -0x88,0x20,0xe3,0xd8,0x47,0x70,0xf8,0x07,0x54,0x5c,0x27,0x79, -0x91,0xf8,0x41,0xd0,0x3b,0x4b,0x57,0x65,0x92,0xfb,0xba,0x42, -0x37,0x9a,0x58,0x28,0xa1,0x13,0xdd,0xf4,0xa0,0xd4,0x65,0x79, -0x8e,0x85,0x3f,0x8d,0x20,0x6a,0x1b,0x98,0xdc,0x21,0xe2,0xfa, -0x70,0x04,0xa2,0x90,0x71,0xe3,0xa2,0xf7,0x36,0x4b,0xd7,0xbe, -0xe7,0x05,0xbd,0x32,0x4f,0x2f,0x19,0xe3,0xa4,0x3f,0xed,0x15, -0xab,0x14,0x98,0xd4,0xc7,0x4c,0x2e,0x53,0x6e,0xf2,0xb4,0x4c, -0xfc,0xcb,0x62,0x09,0x5c,0x58,0xf6,0xe6,0xd9,0xfa,0x2c,0x5d, -0xf6,0xd2,0xe2,0xbb,0x6c,0x91,0x8c,0x37,0x79,0x36,0x4f,0x0a, -0x60,0x47,0xb9,0x48,0xf2,0xbc,0xa7,0x41,0x3b,0xde,0xeb,0xb5, -0x17,0x84,0x00,0x5c,0x64,0xab,0xa4,0xb7,0xca,0x96,0x84,0xc0, -0xc1,0x7b,0x19,0x5f,0x24,0xc0,0xbe,0x24,0xf1,0x17,0x9a,0xfe, -0x45,0xef,0xcd,0x9b,0xeb,0x25,0x46,0xbe,0x79,0x13,0x3d,0xea, -0x8b,0x45,0x6f,0x0d,0xc5,0x14,0xa6,0xf1,0x35,0x24,0xb1,0x1e, -0xc0,0x28,0xdd,0x2f,0xcf,0xd3,0x42,0x4c,0xa6,0xc1,0x56,0x2c, -0x1a,0x0d,0xaa,0x41,0xf2,0x2c,0x2b,0x05,0x7d,0x72,0x6b,0xa6, -0x62,0x1e,0x61,0x5c,0xef,0x3a,0x5e,0x5d,0x25,0x85,0x58,0x47, -0xf3,0xf1,0x5c,0x72,0x29,0xec,0x8f,0xce,0xb2,0xdc,0x27,0xe8, -0xde,0xe6,0xaa,0x38,0xa7,0xdc,0x81,0x48,0xa3,0xfe,0x68,0xfd, -0x34,0x1d,0x75,0x3a,0x69,0xe0,0x62,0x9f,0x4f,0xd2,0xa9,0xc4, -0xae,0x04,0x80,0x42,0x0e,0x35,0x8b,0xb8,0x8c,0xdf,0x2c,0xae, -0x40,0x74,0x50,0x62,0xb0,0xda,0x4c,0xc8,0x5a,0x54,0xd1,0x62, -0x94,0x9e,0x81,0x3c,0x01,0x63,0x9f,0xe5,0x79,0x5c,0x43,0x62, -0x80,0x24,0x54,0xd1,0x64,0x8a,0xc5,0x02,0x79,0x0b,0x49,0x9c, -0xa6,0xa0,0x62,0xd2,0x5a,0x10,0x03,0x31,0x41,0x30,0x4a,0x56, -0x45,0x72,0xa4,0xb0,0x7e,0x3f,0x7b,0x9b,0xcc,0x4b,0x44,0x7b, -0x5b,0x45,0xb7,0x5b,0xaa,0x5f,0x7a,0x94,0x02,0xcb,0x83,0x0a, -0xc0,0xa3,0x3d,0x58,0xb6,0xaa,0x1b,0x39,0x15,0x59,0x65,0xf1, -0xe2,0xcd,0x79,0x5c,0xfc,0x90,0x67,0x65,0x36,0xcf,0x56,0xfe, -0x55,0xbe,0xb2,0x5b,0x86,0xd2,0x37,0x32,0xf1,0xc7,0xe7,0xbd, -0x32,0x29,0x4a,0x82,0xd9,0xc5,0x92,0x16,0xdf,0xa4,0xab,0xc4, -0x41,0xd0,0x8f,0xa2,0x08,0xc2,0xbd,0x74,0xbd,0x48,0xaa,0xef, -0x91,0x7e,0x06,0x05,0xb1,0x4f,0x54,0x89,0x2d,0x98,0xaa,0xf3, -0x1c,0xd1,0x88,0x79,0xbc,0x5a,0xcd,0xe2,0xf9,0x05,0x89,0x2b, -0x4a,0x9e,0xf7,0xf2,0xfb,0x67,0x5f,0x87,0x47,0x5e,0x07,0x0b, -0x11,0x8b,0xc7,0x3d,0x05,0xa9,0xa5,0x0a,0x64,0x17,0xba,0x79, -0xb1,0x41,0x12,0x8a,0xcd,0xf1,0xb1,0x8f,0x7f,0x22,0xfc,0xe9, -0xe1,0x0f,0x48,0x71,0xf2,0x2a,0xa9,0x40,0x00,0x14,0x6e,0x93, -0x63,0xdb,0x42,0x09,0x12,0xea,0xe3,0xcf,0x1e,0x5a,0x8e,0xbe, -0x79,0xf1,0xf2,0x39,0x12,0x84,0x30,0xc1,0x88,0x84,0x60,0x51, -0x45,0x18,0x3a,0x5c,0xe9,0x11,0x80,0x3d,0x8d,0xfa,0x40,0x20, -0x46,0x73,0x06,0xee,0xb7,0x6d,0xe0,0x52,0x5e,0x82,0x00,0x28, -0x7d,0x77,0x95,0xe6,0x89,0xef,0x9d,0x15,0x1e,0x0e,0x26,0xf1, -0xe2,0x9b,0x5d,0x12,0x5b,0x5a,0xb8,0x2c,0x37,0x07,0x58,0x7a, -0xf4,0xb7,0x57,0xaf,0x7e,0x50,0x7c,0xa5,0x5a,0x40,0x39,0x91, -0x2e,0x0b,0x73,0x53,0x69,0xef,0xae,0x64,0xf3,0x1b,0x86,0xe7, -0xaa,0x17,0x6e,0xb2,0x02,0x84,0x1b,0x25,0x2f,0x5a,0x27,0x37, -0x47,0x5f,0x5e,0x9d,0x9d,0xc1,0xc8,0xb6,0x89,0x61,0xb0,0x7b, -0xb1,0x2e,0x11,0xae,0x77,0x0e,0xe4,0xc2,0xe0,0x37,0xf1,0x60, -0x34,0x29,0x93,0x75,0xd9,0xe5,0x6a,0x79,0x53,0x31,0xe8,0x07, -0xd8,0xdb,0x8a,0x1e,0x60,0xf4,0xa0,0x45,0xb2,0xdc,0x73,0xda, -0x34,0xb8,0x75,0xda,0x6b,0x7d,0xb5,0x02,0xb9,0x41,0x6e,0x70, -0x0e,0x2c,0xd5,0xca,0x00,0x73,0x46,0x05,0xc3,0xdb,0x06,0xfa, -0x1e,0x24,0x08,0xa0,0x2c,0xc0,0x9f,0x4e,0x54,0x49,0x46,0x5a, -0x59,0x93,0xf5,0xc2,0xca,0x69,0x95,0x83,0x65,0x50,0x75,0x50, -0x30,0x90,0xb8,0x77,0x0f,0x23,0x8e,0xe9,0x7a,0xd7,0x03,0xc4, -0xbe,0xdb,0x12,0x71,0x95,0x16,0x3f,0x5d,0xcd,0x16,0xe9,0x75, -0xba,0x48,0x7c,0x98,0xc3,0xa0,0xcd,0xca,0x74,0x7e,0x51,0x88, -0xcb,0xe0,0x16,0x7a,0x77,0x71,0x35,0xa3,0x20,0x0e,0x14,0x97, -0xc7,0xc7,0xf4,0x2d,0x09,0x7e,0x3a,0x80,0x09,0x12,0xfa,0x39, -0x32,0x5a,0x81,0x89,0xb7,0xe2,0x5a,0x80,0x2c,0x03,0x23,0x23, -0x85,0x1d,0x71,0x3e,0xa7,0x28,0xc6,0xdf,0x5b,0x64,0x97,0x31, -0x4c,0x12,0x01,0x8e,0x78,0xdd,0x01,0x8c,0x3d,0x36,0x56,0xb1, -0x88,0x7c,0x0a,0x4f,0x06,0xd3,0x2e,0x7f,0xf4,0xa7,0xc1,0x69, -0xa7,0x73,0x89,0xa3,0xd2,0x17,0xeb,0x11,0x0d,0x5c,0x6f,0xa3, -0xcb,0x51,0xb7,0xfb,0xf6,0x69,0x7f,0x14,0xf8,0xd7,0x51,0x87, -0xe1,0xd2,0x69,0xf7,0xed,0xc9,0x22,0x78,0x1a,0x31,0x01,0x90, -0xef,0xf8,0x58,0x11,0x26,0x07,0xb3,0x80,0x46,0xa6,0x6e,0x37, -0x15,0x6f,0x61,0xb0,0xed,0x74,0xde,0x7e,0x01,0x95,0xb2,0x31, -0x74,0x10,0xc3,0x17,0x12,0xc1,0x60,0x3a,0x0a,0x9a,0x08,0xd4, -0x90,0xa5,0xe2,0x77,0xb9,0x69,0xd5,0x97,0x6b,0xca,0xb2,0x58, -0x94,0x71,0x5e,0x46,0x1c,0x03,0xa4,0x89,0x02,0xd4,0x0c,0x15, -0xe4,0x3f,0x92,0x03,0x5d,0x28,0x56,0x15,0x02,0x30,0x4f,0x29, -0xe3,0x78,0x42,0x7f,0x28,0xd7,0x34,0x9c,0xe0,0x1f,0x41,0x31, -0xd3,0x76,0x02,0x7e,0x8c,0xd7,0x4b,0xd9,0x9e,0x7a,0xe4,0x63, -0xee,0xe7,0x98,0xc2,0xf4,0x8d,0x77,0x62,0x60,0xc2,0xde,0xdb, -0x6c,0x04,0x06,0xad,0xb6,0x53,0xe0,0xcb,0x78,0x96,0xac,0x08, -0x74,0xe1,0x67,0x79,0x0a,0x39,0xc4,0x0a,0xa3,0x0a,0x91,0xad, -0x16,0x84,0x47,0x40,0xef,0xe3,0x8f,0x22,0x7d,0x9f,0x88,0x4d, -0x0c,0x93,0x13,0x7e,0x45,0xdf,0xc6,0xe5,0x79,0xef,0x32,0xae, -0x7c,0x8a,0xef,0x07,0x1d,0x48,0x12,0xbe,0xb7,0x4a,0xce,0x4a, -0x0f,0xc6,0x69,0xc6,0x76,0x77,0xe7,0x41,0x75,0x4d,0x38,0x80, -0x26,0x43,0xf8,0x13,0x90,0x9e,0x40,0xb8,0x69,0x00,0x3b,0xcb, -0xca,0x32,0xbb,0x34,0x51,0x63,0x9c,0xa2,0x12,0x26,0x8f,0xe9, -0xea,0xc1,0xe4,0xb1,0x49,0xf2,0x32,0x85,0xfe,0x06,0x00,0x49, -0x2e,0x6e,0xab,0x50,0xd3,0x5a,0x87,0xb7,0x34,0x69,0x87,0x58, -0xc6,0x16,0x7a,0xce,0xa1,0xec,0x57,0x1b,0xe8,0x8e,0x09,0xe6, -0xd7,0x55,0x74,0xf3,0x8b,0x78,0x95,0x2e,0xd7,0x2a,0xca,0x9b, -0x53,0x79,0xde,0x56,0xcc,0xe2,0x22,0x59,0xa5,0xeb,0x44,0xa5, -0x48,0x5e,0x7e,0x29,0xa3,0x27,0x4c,0xfb,0x14,0xca,0x0f,0xc2, -0x87,0x55,0xc0,0x29,0xb5,0xd6,0xf5,0x79,0x70,0x05,0x1a,0xf9, -0x75,0x7d,0x9c,0x0a,0x48,0x32,0x9f,0x61,0x9c,0xa6,0x71,0xa7, -0x32,0xde,0x65,0xba,0x58,0xac,0x12,0x0f,0xa9,0xdf,0x11,0x97, -0x57,0xd8,0x6d,0x5c,0x71,0xe1,0xb1,0xa3,0x5d,0x5a,0x64,0xef, -0x81,0xf2,0xa2,0x7b,0xe4,0x62,0xdc,0x1d,0x84,0x83,0x11,0xc9, -0xd5,0xe0,0x14,0xa7,0x79,0xfc,0x1c,0xdf,0x2f,0x1f,0xb7,0xcb, -0x3c,0xbb,0xda,0x00,0xd1,0x71,0x7e,0xd1,0xa3,0x6f,0x98,0x15, -0xd2,0xe5,0x79,0xe9,0x89,0xcb,0xab,0x55,0x19,0x76,0xb1,0xf0, -0x6d,0xd8,0x02,0x76,0x93,0x2e,0x60,0xaa,0x70,0xa1,0x14,0x17, -0x97,0xeb,0x13,0x66,0xe5,0x07,0x09,0xa8,0x1c,0x6b,0x1e,0x24, -0x9f,0x7d,0x68,0xa6,0x21,0xb7,0x97,0xdd,0xc6,0x3b,0x28,0x0e, -0xc9,0xe8,0x43,0x71,0x24,0x55,0x5a,0xda,0x18,0x1a,0x62,0xb9, -0x9f,0x6c,0x53,0x4e,0xc5,0xe5,0xec,0x13,0xcd,0x03,0x64,0xb7, -0xe1,0x30,0x84,0xdc,0x4f,0xb6,0x0d,0xdd,0x26,0x8f,0xa5,0x1c, -0xe9,0x2c,0x79,0x84,0x28,0x41,0x03,0x9e,0xc8,0xce,0xce,0x8a, -0xa4,0x64,0x29,0x04,0xb1,0x8e,0xfe,0xe3,0x3f,0x7c,0x9f,0x52, -0x70,0x82,0xe2,0x0f,0x9c,0xa0,0x86,0xb0,0x0a,0x22,0x19,0x6d, -0xb6,0xb6,0x2b,0xb3,0x2c,0xa3,0x3b,0x22,0xb0,0x23,0xca,0x76, -0x9d,0x80,0x96,0x83,0x7c,0x01,0xaa,0xb6,0xf6,0xa8,0x03,0x92, -0xc7,0x44,0xc3,0xe0,0xb3,0x5e,0xae,0x12,0xc3,0xbf,0x2d,0x0e, -0xef,0x0f,0x47,0xec,0xa0,0xaa,0x9d,0xe2,0x1c,0xc4,0xdd,0xcf, -0x11,0xf5,0x0e,0x63,0xbf,0xa6,0x59,0xcd,0xe5,0x2c,0xcf,0x74, -0x92,0xb5,0xa6,0x7b,0x6f,0x60,0x1e,0x18,0xf9,0xf7,0xf0,0x4e, -0x8d,0xfb,0x30,0xec,0x53,0xd7,0x02,0x05,0x0a,0xb2,0x45,0xf7, -0x33,0xd3,0xfb,0xd6,0xeb,0xa8,0x96,0xea,0x78,0xc2,0xeb,0x60, -0xee,0x8e,0xf7,0x73,0xff,0x6f,0x2a,0x7e,0x00,0xf1,0x3f,0x73, -0x7c,0x88,0xd0,0x0c,0x20,0xec,0x6c,0x7f,0xeb,0xff,0x6c,0x43, -0xff,0x8d,0x81,0x64,0x85,0x76,0xd9,0xd8,0x23,0xda,0x24,0x83, -0xf0,0x7b,0xbb,0xc3,0x9f,0xbf,0x13,0xa0,0x0f,0x8b,0xda,0x4b, -0x41,0xe3,0x89,0x28,0x01,0xbf,0x54,0x5d,0xb3,0x88,0x02,0xe3, -0xdb,0x6d,0x48,0x00,0xdc,0x0c,0x11,0x7e,0xf7,0x68,0x04,0x5a, -0x24,0x67,0x3d,0x8e,0x14,0x5c,0xcd,0x66,0x1a,0x33,0x9c,0x86, -0xa8,0xc8,0x1a,0xae,0x04,0x8f,0x6d,0x91,0x3d,0xd0,0x8d,0x8a, -0x9b,0xb4,0x9c,0x9f,0x3b,0x6b,0x39,0x29,0xf7,0xc0,0x73,0x59, -0x74,0x97,0x73,0x90,0x32,0x50,0x4c,0x38,0x92,0x43,0x53,0x09, -0xc2,0x8b,0x5d,0xd0,0xeb,0x64,0x7b,0x81,0x0e,0x5a,0x24,0xdc, -0x88,0x61,0xd6,0xab,0xa2,0xae,0xa2,0xb7,0x57,0x83,0xf6,0x35, -0x83,0x95,0xc2,0xc5,0x88,0x40,0x72,0x1a,0x6d,0x09,0x86,0x68, -0xec,0xec,0x03,0x94,0x4d,0x4d,0x90,0x7d,0x4a,0xe6,0x0a,0xc8, -0x0c,0x36,0x28,0x8a,0x80,0x05,0xd7,0x75,0x20,0x80,0x41,0x31, -0x0e,0xda,0x26,0xbd,0xbf,0x25,0x7e,0x83,0x8a,0x8b,0x7f,0x60, -0xa1,0x04,0xe3,0xd7,0x26,0x5b,0xe9,0x06,0xca,0xf6,0x4c,0x64, -0x7a,0xcf,0xe0,0xb6,0xac,0x37,0x20,0x3c,0xf9,0x15,0xcc,0x7b, -0x82,0xb2,0xc7,0x00,0x7d,0x9d,0x84,0x8f,0x06,0xe2,0x22,0xa9, -0x43,0xb9,0x20,0x30,0x82,0x12,0xde,0xd2,0x20,0x19,0xde,0x16, -0x65,0x9e,0x5d,0x58,0x8a,0x80,0xda,0x05,0xc1,0x32,0x7a,0x38, -0xa4,0x7d,0x95,0xad,0xb2,0x7c,0x2b,0x18,0xee,0x1f,0xc8,0xa0, -0x03,0xc0,0x94,0xbe,0x15,0xd9,0x26,0x9e,0xa7,0xa5,0xee,0xb4, -0x83,0xa4,0xfb,0xc9,0x76,0x2b,0x70,0x3c,0x0c,0x6f,0xdb,0xd3, -0x58,0x6e,0x77,0x53,0xb7,0xf8,0xbf,0xb6,0xba,0x93,0xde,0xb7, -0xc3,0x80,0x12,0x86,0x97,0x06,0x03,0xfa,0xf7,0x30,0x00,0x56, -0x8a,0xab,0x03,0x35,0xa2,0x72,0x24,0x0f,0xce,0x60,0x95,0x76, -0x1f,0xe8,0x37,0x00,0xc3,0x90,0x3f,0x61,0x7f,0x7e,0x00,0xf4, -0x4f,0x34,0x51,0xb7,0xb1,0x45,0x60,0x75,0x90,0xc2,0x64,0xb5, -0x08,0x3d,0x52,0x9f,0xbc,0xdf,0x97,0x8f,0x30,0x08,0x3f,0x88, -0x85,0x6d,0x7c,0x3b,0xc4,0x0c,0xc0,0xfb,0x40,0x46,0x48,0x48, -0x66,0x02,0x42,0xff,0x83,0xfa,0xd4,0x7d,0xf0,0x0c,0x05,0x39, -0x0e,0xb5,0x1e,0xc0,0xca,0x96,0x7b,0xa0,0x1e,0xac,0x55,0x47, -0x97,0xf3,0x24,0x3b,0x16,0x73,0xdb,0x78,0xf9,0xb5,0x5c,0x57, -0xba,0xcc,0xc4,0xb1,0x77,0xa7,0x43,0xb6,0x30,0x53,0xcf,0x7a, -0xbd,0x27,0xd6,0x64,0x87,0x81,0x83,0xfd,0x13,0x7f,0x1e,0xdc, -0x3f,0xf1,0x87,0xfb,0xe7,0xfe,0x9a,0xac,0x92,0x25,0xcc,0x97, -0x7f,0xca,0xdc,0x00,0x8b,0xad,0x68,0x78,0xe2,0x26,0x41,0xdc, -0x22,0x5d,0x2f,0xc5,0xb2,0x1a,0xc8,0x29,0x62,0x96,0x5d,0xad, -0x17,0xc5,0xd8,0x0e,0xf4,0xaa,0x41,0xd8,0x07,0x90,0xe1,0x21, -0x90,0x61,0x68,0xcf,0x3a,0xab,0x1b,0xd0,0x9c,0xa8,0x28,0x99, -0x4e,0xd1,0x7e,0xd0,0xf1,0x33,0xfe,0x1c,0xf7,0x43,0x5c,0x17, -0x8a,0xd5,0x79,0x03,0x90,0x47,0x78,0x86,0xe4,0x6f,0x09,0xaa, -0x66,0x2d,0x1c,0xc4,0x7b,0x4f,0x68,0x14,0xa7,0x3f,0x3c,0xe5, -0xad,0x6e,0x84,0x82,0x8f,0x56,0xe7,0x7b,0xa6,0xa4,0x4e,0x04, -0xf5,0x94,0x53,0x43,0x77,0x75,0xd3,0x3e,0x27,0x21,0xd0,0x50, -0xce,0x30,0xdb,0x06,0xb7,0x1a,0x1a,0xef,0x6e,0x83,0xb5,0xb4, -0xed,0x4f,0xf5,0xe5,0x2c,0x53,0x0b,0x66,0x44,0xc5,0xab,0xe2, -0xe2,0x3c,0xde,0x24,0xd4,0x9b,0xa4,0x24,0x49,0xd5,0x08,0x8a, -0x28,0x22,0x2a,0xb1,0x59,0x1a,0xad,0x75,0x60,0x82,0x26,0x90, -0x1e,0xa9,0x46,0xb7,0x34,0x21,0x93,0x92,0xcc,0x73,0xf3,0x77, -0xf1,0x25,0x22,0xb5,0x3a,0x51,0xc0,0x25,0x99,0x7c,0x18,0xd2, -0x19,0x31,0xb0,0x3f,0x27,0x52,0xa7,0x33,0x62,0x40,0xe5,0xc3, -0xef,0x03,0x05,0x52,0x7d,0x4c,0x89,0x14,0xd4,0x45,0x52,0x68, -0x6f,0xe6,0x16,0x06,0xaa,0x41,0x93,0x8e,0x4d,0xce,0x96,0x91, -0xe9,0x65,0x9c,0x3e,0xba,0x6f,0x34,0xbd,0x6f,0x46,0xb6,0x97, -0x1c,0xf6,0x52,0xc9,0x1e,0xea,0xa0,0x60,0x67,0x7c,0xb3,0x07, -0x63,0x9d,0xd6,0x3e,0x02,0x3b,0xc9,0xfb,0x86,0x5d,0x07,0x48, -0x8d,0xb5,0x3b,0xa3,0x25,0x6a,0x39,0xad,0x43,0xe5,0x1f,0xa9, -0x00,0xd8,0x62,0x5c,0x3c,0xb8,0x1d,0x0a,0x82,0xff,0xf0,0x96, -0x90,0xd5,0xe2,0xfe,0x27,0x17,0xdb,0x72,0x8c,0x66,0x01,0xf2, -0xb8,0x34,0x4f,0x09,0x0e,0x6d,0x72,0x03,0x07,0x48,0x96,0x6d, -0x76,0x32,0x05,0x3f,0x61,0xf4,0x96,0xfa,0x5c,0x4b,0x22,0xb5, -0x86,0x3b,0xe2,0x9b,0xd4,0xfd,0x03,0xbd,0x85,0xc1,0x24,0xfe, -0xa1,0x8d,0x70,0xfd,0x92,0x6a,0xad,0x95,0xb0,0x3f,0xb2,0x33, -0x34,0x9a,0x80,0xff,0x86,0x0f,0x69,0x83,0x66,0x8f,0x59,0xed, -0xd1,0xe5,0x74,0xda,0xfe,0x1e,0xb3,0x72,0xb5,0x35,0x47,0x9f, -0xd0,0xe9,0xb4,0x31,0xb5,0xdb,0x4f,0x74,0xba,0xda,0x5f,0x6b, -0x57,0xec,0xfe,0xf8,0x4e,0xf3,0xd7,0x3c,0x5e,0xa4,0xb4,0xd1, -0xfa,0xc0,0x06,0xcb,0x61,0x2d,0xf6,0x41,0xea,0x4b,0x63,0xc8, -0xba,0x69,0x4a,0xe9,0x52,0x52,0x20,0x25,0xf4,0x7c,0x67,0xd0, -0x51,0x00,0x7f,0x93,0x63,0xce,0x6e,0x67,0x50,0x10,0x2c,0xea, -0x87,0x3b,0x85,0x0b,0xfb,0xc7,0x77,0x8b,0xf3,0x3f,0xb1,0x5b, -0xb4,0x8b,0xbe,0x1c,0x7f,0x35,0x8d,0x43,0x68,0x85,0x85,0x09, -0xfd,0x09,0x7d,0x42,0x12,0x42,0xa1,0x36,0x15,0xbb,0x65,0xd2, -0xf8,0x83,0x3a,0x00,0x72,0xff,0x7a,0x19,0xdd,0xa2,0x7d,0x01, -0x34,0x50,0xe8,0x0d,0x7a,0x8f,0x7b,0x43,0x4f,0x2c,0x1e,0x87, -0x96,0xa1,0x45,0xa8,0x3e,0x80,0xa8,0x0c,0x04,0x05,0xf4,0xcf, -0x88,0x77,0x21,0x7a,0x74,0xcc,0x8b,0x6d,0xd4,0x53,0x29,0x23, -0x6b,0x93,0xc2,0x9c,0xdc,0x67,0xb3,0xb7,0xfa,0x9c,0x03,0xbe, -0xa3,0x28,0x52,0xdb,0x18,0x90,0xb0,0x15,0x94,0xe7,0x1b,0x09, -0xdc,0x9a,0xcb,0x9b,0x64,0x94,0xe1,0x48,0x41,0x4d,0xbd,0x28, -0x52,0x65,0xf6,0xf0,0x30,0xcd,0x46,0x25,0x89,0x3c,0x88,0x88, -0x61,0x0e,0xa2,0xa1,0xc3,0xf8,0x88,0x7e,0x55,0xe8,0xee,0xee, -0x20,0x52,0x82,0x39,0x88,0xf3,0xbb,0xab,0xcb,0x59,0x92,0x1f, -0x26,0x8d,0x61,0x0e,0xa2,0xf9,0x32,0xcb,0x56,0x49,0x7c,0x0f, -0xaf,0x24,0xd0,0x41,0x44,0xaf,0xf2,0x24,0x69,0x6f,0x27,0xcb, -0x1e,0x01,0xa3,0x8f,0x8f,0xe1,0xd7,0xb2,0xd0,0xa0,0xfc,0xeb, -0x46,0x6d,0x0a,0x95,0xbb,0x53,0x50,0xfa,0xac,0x49,0xa6,0x06, -0x78,0xf4,0x88,0x21,0xd2,0x05,0x74,0x58,0x10,0xcc,0xc8,0x3e, -0x8a,0xd5,0x76,0x07,0x66,0xf3,0xb8,0x41,0xa3,0x3a,0xd9,0xac, -0xc4,0x1a,0x75,0xd2,0x34,0x1a,0xa0,0xed,0x4a,0x14,0xe7,0xcb, -0xab,0x4b,0x40,0xa8,0x8e,0x2b,0x47,0x68,0x05,0xc3,0x06,0x13, -0xb7,0x95,0x49,0x9d,0xa4,0x53,0xb6,0xf1,0x80,0xbc,0x68,0x06, -0x51,0x05,0x80,0x74,0x82,0xa1,0x69,0x54,0xf1,0xdf,0xad,0x91, -0x56,0xa2,0x42,0x9b,0x46,0xb4,0x33,0xeb,0xff,0xfc,0xf4,0xfd, -0x77,0x3d,0x3a,0xb9,0xf6,0xe9,0xb3,0x20,0x6e,0xa7,0x67,0xcc, -0x3b,0x66,0x36,0xf5,0x65,0x93,0xfd,0x4c,0x67,0x3e,0x93,0x16, -0x45,0xde,0xeb,0xd7,0x3d,0x2f,0x80,0xe5,0xcb,0xc6,0x98,0xf7, -0xd8,0xe6,0x31,0x12,0x0a,0x60,0xc8,0x2a,0x68,0x71,0x35,0x4f, -0x0c,0x60,0x2c,0x66,0x1a,0x34,0x96,0xd5,0x07,0x95,0x3e,0x9e, -0xc4,0xe6,0xe0,0xb2,0x13,0x41,0xe6,0xce,0x0c,0xd6,0x14,0xe9, -0x59,0x89,0x27,0xbc,0x31,0x9d,0x9b,0xf6,0xe2,0xcd,0x66,0x55, -0x13,0x06,0x11,0x6f,0xd9,0x94,0x06,0xe8,0x8d,0xe7,0x68,0xde, -0x93,0xe5,0x0e,0xc9,0x74,0xda,0x33,0xb2,0x45,0xe4,0x1b,0x93, -0x7a,0x77,0x87,0x47,0xe0,0x51,0x74,0x36,0x3e,0x0b,0xad,0x8e, -0x08,0x29,0xb8,0x37,0x1c,0x29,0x1e,0x40,0x38,0xd0,0xa7,0xd4, -0xe3,0x96,0x86,0x2f,0x76,0x6a,0x57,0x09,0xc3,0xae,0x6a,0x72, -0x36,0xdd,0x8a,0x2a,0xd8,0x86,0x6d,0x32,0x83,0x89,0x44,0xfe, -0x3c,0xbb,0x84,0x06,0x89,0x4b,0xbb,0x02,0x50,0x9b,0xd2,0x3a, -0xb1,0x9a,0xe8,0xa3,0xdc,0xeb,0x2c,0x5d,0x90,0xed,0x09,0x42, -0x20,0xb1,0xf0,0x07,0x92,0x61,0x6d,0x84,0x1f,0xc8,0x0c,0xea, -0x08,0x94,0xdf,0x6d,0x20,0xc5,0x93,0x68,0x20,0x71,0x79,0x5d, -0xdc,0xdb,0x3e,0x9b,0xf4,0xa7,0x63,0xa8,0x73,0x77,0x20,0xce, -0xa2,0x33,0x69,0xa8,0x31,0x08,0x82,0xd0,0xeb,0xa8,0x64,0xe2, -0x49,0x23,0x99,0x4e,0x29,0xf8,0x30,0xbb,0x08,0xec,0x46,0x80, -0x72,0x70,0x81,0xe7,0xb4,0xb6,0x32,0x20,0x3a,0x13,0x95,0xa8, -0xd9,0xa8,0x87,0x8c,0x85,0x90,0xca,0xa6,0xbd,0x10,0x5a,0x0b, -0x9c,0x51,0x0a,0x1a,0x2b,0x55,0xd1,0x99,0x1f,0x07,0x02,0x7a, -0x9d,0x0f,0x6d,0x5e,0x3f,0xad,0x02,0xa6,0x9d,0xb6,0xee,0x97, -0x6b,0xec,0x1f,0x00,0x5f,0x3d,0xad,0x03,0xd5,0x22,0x1c,0xab, -0x7a,0x45,0x5f,0xb2,0xf8,0x72,0x13,0xb5,0x0a,0xe0,0xec,0x69, -0x8c,0x47,0x29,0xf1,0xd3,0xd9,0x78,0x80,0x6a,0x0e,0x0f,0x15, -0x7b,0xc1,0xe3,0xee,0x8c,0x25,0x8e,0x06,0xdc,0x96,0x56,0x45, -0xb1,0x7a,0x14,0x55,0x63,0x6b,0x4c,0xaa,0x82,0x71,0x15,0x4e, -0xaa,0x69,0x38,0x99,0x52,0x5e,0xb6,0xe4,0x6a,0xcb,0x6c,0x6f, -0x9b,0x57,0x20,0x89,0x8f,0x1c,0x2c,0xc7,0xc7,0x95,0xcc,0x3b, -0x56,0x1f,0x21,0x0f,0x3c,0xd0,0x87,0xf7,0xa2,0xd3,0x34,0x78, -0x13,0xaf,0x53,0x91,0x44,0x70,0x8e,0xa0,0xe3,0x4d,0xbd,0xd0, -0x2d,0x72,0xdc,0x18,0x12,0xaa,0xc0,0xe9,0x1d,0x88,0xe5,0x23, -0xaf,0xd3,0x34,0x62,0xec,0x40,0x24,0x50,0x42,0xb6,0x35,0x8e, -0xf9,0x62,0x74,0xea,0xff,0xfb,0xdd,0xe4,0xdf,0x5f,0xbf,0x9e, -0x06,0x1f,0x9d,0xd2,0x84,0x0b,0xaa,0x90,0x5b,0x73,0xcc,0x43, -0x91,0x13,0x1e,0xe8,0x64,0x98,0xc7,0x3a,0x4c,0x60,0x11,0x83, -0x2f,0x6d,0xb1,0x86,0xb1,0x54,0xed,0xab,0x75,0xfa,0xee,0xca, -0x1a,0xe6,0xc8,0x20,0xe6,0x0c,0xad,0x5f,0x60,0x69,0x57,0x90, -0x20,0xd9,0x1c,0x24,0x7b,0x97,0xbb,0x3b,0xe8,0x3b,0xf8,0xa5, -0xcc,0x8e,0x18,0x29,0x96,0x1e,0x9d,0xdd,0xdd,0x59,0xe3,0xbc, -0xc2,0x13,0xc9,0xbf,0x77,0x77,0x16,0x89,0xd7,0xca,0xda,0xcd, -0x60,0x32,0x26,0x77,0x20,0xab,0x18,0x8f,0x86,0x69,0x0a,0x89, -0x31,0x98,0x0a,0xbe,0xe8,0x1f,0x1f,0xab,0x58,0x65,0x4c,0x22, -0x6b,0x26,0xa3,0xa9,0x72,0x97,0xe9,0xfa,0x05,0xe6,0x69,0x56, -0xef,0x83,0xaa,0xd5,0x1d,0xec,0x54,0x4b,0x57,0x01,0x6d,0xb9, -0xfa,0x02,0x8a,0x51,0xe4,0xf6,0x81,0xdc,0xeb,0x08,0x22,0x68, -0x92,0x7a,0x40,0xdd,0x00,0xf4,0xe9,0x35,0x8c,0x0d,0x88,0x03, -0x18,0x02,0xf8,0x52,0x5d,0x15,0x08,0x71,0x35,0xe2,0xea,0x4f, -0xa9,0x46,0x5c,0x35,0xaa,0x11,0x57,0x0f,0xad,0xc6,0xf5,0x53, -0x00,0xc6,0x6a,0x00,0x8e,0x7d,0xd5,0x50,0xf6,0xb3,0xd6,0x00, -0x2d,0xa4,0x9d,0xd1,0x26,0x2b,0x04,0x5a,0xd5,0x8a,0x64,0xb5, -0x4a,0x37,0x45,0x2a,0xf7,0x56,0x71,0x92,0xd7,0x93,0x3b,0xd4, -0x55,0xce,0x22,0x11,0x1a,0xae,0xaa,0xa1,0x6a,0xa4,0xb2,0x44, -0xea,0xe3,0xee,0xce,0xeb,0xf5,0x7a,0x1e,0x75,0xa5,0x95,0xb1, -0x6a,0xe9,0xcb,0xc2,0xba,0x0a,0x4e,0xf1,0x47,0x6d,0x5d,0xa2, -0x19,0x98,0xbd,0x25,0x29,0x4b,0x50,0xe0,0x1d,0x1f,0x29,0x1c, -0xb7,0xda,0x0e,0x8b,0x41,0x10,0x16,0x72,0x5c,0x07,0xac,0xdd, -0x55,0x10,0xf0,0xc6,0xa5,0xdc,0x40,0x0f,0x29,0x20,0xf7,0xd9, -0x43,0x22,0x6c,0xc0,0x94,0xcd,0x93,0x74,0xe5,0xaf,0xf0,0xc4, -0x7a,0x35,0xe4,0x98,0xb3,0x55,0x06,0xcd,0x82,0x51,0x92,0x81, -0x07,0xca,0xb5,0x8a,0xed,0x63,0x28,0xe8,0x3c,0x84,0xda,0xe1, -0x0e,0xb9,0x43,0xa0,0x57,0x1d,0xc3,0xdd,0x5b,0xa8,0x53,0xa6, -0x55,0xe4,0x96,0x47,0xaf,0x16,0x4b,0x69,0x18,0xc3,0x26,0xaf, -0xaf,0xfa,0xfd,0xfe,0xe7,0xf4,0xfb,0x8c,0x7e,0xbf,0xa4,0xdf, -0xaf,0xe8,0xf7,0x6b,0xfc,0x1d,0xf6,0xf1,0xf7,0x19,0xfc,0x0e, -0x3e,0xf9,0x0c,0x7f,0x3f,0xeb,0x3f,0x7f,0x7d,0x35,0x84,0x64, -0xfa,0x1d,0xd0,0xef,0x90,0x7e,0x1f,0xd3,0xef,0xc7,0xf4,0xfb, -0x84,0x7e,0x3f,0xa1,0xdf,0x4f,0xe9,0xf7,0x33,0xfa,0xfd,0x9c, -0x7e,0x9f,0xe1,0xef,0xf0,0x1b,0xfc,0x7d,0x42,0xbf,0x43,0x4a, -0x1d,0x42,0xea,0x63,0xc2,0xfc,0xcd,0xf3,0x6f,0xbe,0x99,0x06, -0xa7,0x23,0x36,0x77,0x34,0xb2,0x29,0xad,0xa0,0xa5,0x4d,0xb4, -0x37,0xf9,0x39,0x59,0xc6,0x47,0x2f,0xb3,0xe5,0xf4,0xc8,0xeb, -0x90,0x81,0x33,0x69,0xab,0x68,0xec,0xd7,0xc8,0x03,0x3f,0x91, -0x04,0x7f,0x9e,0xe7,0x12,0x5c,0x38,0xd6,0xd5,0xc2,0xbb,0x82, -0xfe,0x7c,0x06,0xcb,0xc0,0x85,0xf7,0x28,0xc2,0x15,0x55,0x76, -0x76,0x04,0xcb,0x57,0x54,0x46,0xe8,0x8f,0xaf,0x4b,0xe0,0x75, -0x72,0x74,0x6b,0x05,0xa4,0x45,0x76,0xd4,0x86,0x24,0xa9,0x36, -0x30,0xe9,0xe3,0xc9,0xe9,0x79,0x8a,0x16,0x17,0x14,0x7a,0x14, -0x45,0xf2,0xcb,0x42,0x82,0x0b,0xd1,0xbf,0xff,0xf8,0x32,0xf2, -0x3c,0x2b,0xb2,0xb8,0x5e,0xe2,0xae,0x6f,0x01,0xcb,0xc7,0x24, -0xfa,0x48,0x2e,0x14,0x23,0x58,0x28,0x0e,0xbc,0xa3,0xea,0x72, -0xb5,0x2e,0x22,0xb2,0xf5,0x0c,0x4f,0x4f,0x6f,0x6e,0x6e,0x7a, -0x37,0x8f,0x7b,0x59,0xbe,0x3c,0xc5,0x16,0x3a,0x85,0x9c,0x12, -0x24,0xac,0x60,0x7d,0x7b,0xd1,0x06,0x38,0xf8,0xfc,0xf3,0xcf, -0x4f,0x29,0xd5,0xfb,0xc8,0x2a,0x34,0xbe,0x82,0x25,0x67,0xbc, -0x78,0xb1,0xc6,0x73,0x93,0x27,0x36,0x35,0xb8,0xa4,0x8f,0x6e, -0xcb,0xf4,0x32,0x81,0x15,0x6a,0x0f,0xff,0x72,0x9c,0xb8,0x2a, -0xe7,0x6e,0x4c,0x0f,0x62,0x6c,0x16,0xe5,0xb0,0x88,0x80,0xa5, -0xca,0x2d,0x2e,0xb5,0x79,0x1b,0x04,0x96,0x0c,0xf0,0xfd,0x55, -0xbc,0x09,0xbd,0xd9,0x55,0x59,0x7a,0xbc,0xba,0xf7,0x8a,0x78, -0x5d,0x74,0x8b,0x24,0x4f,0xcf,0x3c,0xb3,0xa8,0x1f,0x0c,0x6c, -0x5c,0x78,0x50,0x14,0xdd,0xf2,0x99,0x45,0xa8,0x4e,0xbd,0xd9, -0x4c,0x2a,0x1c,0xf4,0x85,0x3c,0xa4,0x09,0x1f,0x0b,0x7d,0xfc, -0x14,0x7a,0x7f,0x01,0x9e,0x78,0x62,0x99,0xa7,0x0b,0x15,0xb1, -0xf8,0x0c,0xff,0xe3,0x7c,0x0e,0x90,0x7b,0xaa,0xaa,0x62,0xf5, -0xe9,0x14,0x10,0xae,0x4f,0x92,0xe5,0x37,0xd1,0xf8,0x89,0xd8, -0x39,0x35,0x05,0xba,0xdd,0x48,0xb7,0x7a,0x66,0x83,0xdc,0x14, -0x2d,0xb7,0xb4,0x5b,0x00,0x5d,0xa4,0xce,0xd6,0x37,0x32,0x61, -0xb5,0x90,0x70,0xdf,0xf3,0xfe,0xe3,0xe3,0x27,0x36,0xcb,0x78, -0x17,0xc6,0x30,0x8d,0xcf,0x6f,0xd4,0x5e,0xa5,0xc3,0xb4,0x96, -0xfd,0x2b,0x20,0xef,0xb3,0xcf,0x3e,0xf3,0x44,0xcb,0x76,0x15, -0x70,0xc0,0xdd,0x12,0x0b,0x07,0x9f,0x08,0x67,0x17,0x0d,0xb0, -0xf7,0xc5,0x6a,0x87,0x9f,0x2b,0x97,0x51,0x7d,0x13,0xe1,0xd6, -0xdd,0x6c,0x5c,0x86,0x3c,0x0d,0x08,0x67,0xab,0x52,0x1f,0x89, -0x72,0xb4,0xac,0xfc,0x67,0xc2,0xda,0xcd,0x0e,0xbd,0x79,0x9a, -0xcf,0x11,0xc4,0xec,0x62,0x87,0x4f,0xfa,0xc2,0xda,0xb5,0x56, -0x15,0xdc,0xd9,0xa4,0x0e,0x07,0xbf,0x7b,0x2b,0xd9,0xcd,0x32, -0x47,0xb4,0xd1,0x6d,0xbe,0x9c,0x85,0x93,0xc1,0xf0,0x33,0x21, -0xff,0x4d,0xb1,0x32,0xe1,0x04,0x68,0x84,0xff,0xa6,0xe2,0x7c, -0xbe,0x0a,0x27,0xf8,0xfd,0x04,0x03,0x85,0x0c,0xf4,0x9e,0x4c, -0x9d,0x0e,0x86,0xc6,0x37,0xd1,0x2d,0x8e,0xf2,0xcb,0x2c,0xaf, -0x07,0xfd,0x70,0xe2,0xfd,0x65,0x70,0xf6,0xe9,0xa7,0xb3,0x8f, -0x3d,0xe1,0xfd,0xe5,0xec,0xec,0xd3,0xb3,0x7e,0x82,0x5f,0xc3, -0x79,0xdc,0x1f,0xce,0xf1,0x6b,0xf1,0xc9,0xf0,0xd3,0xe1,0x67, -0xf8,0xf5,0xf9,0xc7,0x9f,0x7c,0x3a,0x5b,0xe0,0xd7,0x67,0xf3, -0x27,0x9f,0x7c,0x3c,0xc3,0xaf,0xe4,0xf1,0xa7,0x9f,0xce,0x87, -0xf8,0xf5,0xe9,0x19,0xfe,0x87,0x5f,0xb3,0xf9,0x6c,0x31,0xa4, -0xb8,0xc1,0xa7,0xb3,0x64,0x7e,0xe6,0x4d,0x85,0x2a,0x70,0xd8, -0x28,0x30,0x4e,0xe6,0x9f,0x26,0x9f,0xb9,0x45,0x9f,0x9d,0xcd, -0x66,0x9f,0x7e,0xe6,0x12,0xf1,0xf9,0x67,0x8b,0xb3,0xcf,0x62, -0x97,0x9c,0xb3,0xb3,0xcf,0x3f,0xfb,0xfc,0x13,0x97,0xb0,0xf9, -0x93,0x59,0x7f,0xf1,0xc4,0x25,0x71,0xfe,0xf1,0xe7,0xf3,0xcf, -0x3f,0x76,0x89,0x3d,0xfb,0x74,0xf6,0xc9,0xa2,0x41,0xf6,0xfc, -0x53,0xfc,0xcf,0xad,0xc0,0x62,0xb6,0x98,0x7d,0xb6,0xb0,0xaa, -0x82,0xa5,0x25,0x8b,0x38,0x79,0x02,0x95,0xa2,0xc3,0x91,0x02, -0x2a,0xa4,0x24,0xc7,0x9b,0xe7,0x59,0x51,0xc0,0xdf,0x45,0x1a, -0x5f,0x66,0xb8,0x9f,0xe9,0x15,0xef,0xae,0xe2,0x1c,0x93,0x60, -0x19,0x41,0x06,0x5f,0xdd,0x45,0x76,0xb3,0xb6,0xc3,0x57,0x1b, -0x8f,0x9b,0xe8,0x4b,0x3a,0xab,0xb5,0x6f,0xd2,0xe0,0xbc,0x3c, -0x6b,0xc4,0xc2,0x82,0x8c,0x26,0x09,0x28,0x30,0xce,0xfd,0x40, -0xcc,0xe4,0xa4,0x01,0x2b,0x02,0x4a,0xdd,0x0a,0xbd,0xdf,0x17, -0xc9,0xd3,0x5f,0x1d,0xa1,0x54,0x3b,0xb3,0x23,0x48,0x58,0x5a, -0x2e,0xef,0x10,0xca,0x6a,0x10,0x75,0x78,0x9b,0xab,0xf7,0xed, -0xb3,0x7f,0xbe,0xf9,0xf9,0xd9,0xcb,0xbf,0x3f,0x17,0x94,0x52, -0xef,0x4d,0xa9,0x86,0x51,0x77,0x4f,0x9e,0x7d,0x29,0x16,0xc5, -0x3d,0x9c,0x4b,0xcc,0x1a,0x69,0x20,0xea,0x81,0xa8,0x86,0xa2, -0x1e,0xee,0x10,0x06,0x69,0x8a,0x92,0x7a,0xa0,0x8b,0x06,0x58, -0x55,0x56,0x3d,0xdc,0x41,0x0e,0x83,0x97,0x85,0x5c,0xd4,0x66, -0x3f,0xe2,0x0b,0x89,0x16,0x54,0x61,0x5d,0x00,0x2c,0xb9,0xbf, -0x90,0x45,0xa8,0x68,0x2c,0x2c,0x10,0xd5,0x53,0x59,0x9c,0x86, -0x1e,0x12,0xf4,0x53,0x59,0xb4,0x86,0x1e,0x22,0x74,0x93,0x08, -0x98,0xd2,0x63,0x7b,0xa7,0x6c,0xd1,0xac,0x5a,0x37,0x5a,0xa8, -0xaa,0xe9,0xcf,0x6a,0xd8,0xd1,0xb1,0xfa,0xd3,0xc6,0x9a,0x63, -0x4b,0x1f,0x68,0x47,0x4b,0x45,0x95,0x51,0x81,0xe6,0x5f,0x33, -0xad,0x56,0x69,0xd5,0xd0,0x52,0x76,0x65,0x4c,0xa0,0x19,0xdc, -0x48,0xaa,0x87,0xbb,0x75,0xa5,0xe3,0xfb,0x95,0xb3,0x76,0x58, -0x54,0x62,0x51,0x37,0xc9,0x83,0x1a,0x55,0x56,0x45,0x2b,0x45, -0x1a,0x7c,0xd7,0x56,0xad,0xeb,0x96,0x6a,0x97,0x0e,0x76,0xea, -0x51,0x82,0x9a,0x96,0x0e,0x2d,0xb2,0x42,0x92,0x99,0x15,0x9c, -0x86,0x7b,0x37,0x6b,0x8e,0x83,0x0f,0x15,0x37,0xaf,0xa2,0xaa, -0x5b,0x9d,0x00,0x54,0xa7,0x3e,0x81,0x78,0x31,0xaf,0xa3,0x1a, -0x22,0xe0,0xb3,0x5b,0x63,0xb4,0x00,0x16,0x4a,0x5a,0x41,0x1e, -0xe5,0x27,0xc8,0xa5,0x8c,0x05,0x09,0xac,0x65,0x6c,0x3d,0x1c, -0xd9,0x75,0x93,0xbd,0x14,0x45,0xcf,0x07,0x3c,0x27,0xd0,0xbc, -0x80,0xf4,0x04,0xaa,0x36,0xaf,0x90,0x14,0x88,0xe8,0x60,0x3c, -0x46,0xd4,0xbb,0x60,0xc3,0x26,0xd8,0xd0,0x05,0x1b,0x36,0xb1, -0x0d,0x5b,0xb1,0x0d,0x9b,0xd8,0x86,0x16,0x36,0x9b,0x9f,0x34, -0x88,0x38,0x63,0x8d,0xda,0x2b,0x82,0x9a,0xb7,0x74,0x92,0x19, -0x09,0xd2,0x0c,0x18,0xd0,0xd2,0x55,0x66,0x24,0x49,0x00,0x32, -0x6c,0xe9,0x30,0x33,0x92,0x25,0x00,0x19,0xb6,0x74,0x9b,0x59, -0xab,0x34,0x25,0xeb,0xf9,0x2a,0x2b,0x92,0xa2,0x95,0x3e,0x39, -0x0a,0x02,0x95,0x44,0x94,0x0a,0x0e,0x9f,0x52,0x49,0x32,0x58, -0x53,0x6a,0xad,0x52,0x6b,0x4a,0xad,0x87,0x76,0x21,0x74,0x7e, -0x55,0x24,0xf3,0x72,0x5f,0x31,0x8f,0x54,0x15,0xbe,0xc0,0x72, -0xee,0xee,0x64,0xb1,0x4f,0xb1,0x18,0x19,0xaa,0x31,0xad,0x56, -0x69,0x35,0xa6,0x41,0x7d,0xec,0x52,0xf0,0xa2,0x54,0x9c,0xae, -0x8b,0xd6,0xe1,0xe8,0x91,0xaf,0xc7,0xa3,0xbb,0x3b,0x3d,0xd8, -0xdc,0xdd,0xe9,0xe1,0x08,0x3e,0x15,0xd3,0x1c,0xac,0x6c,0xb4, -0xb3,0x6f,0x10,0x18,0x76,0x25,0x52,0x3b,0x8b,0x34,0xee,0xd9, -0x93,0xa7,0x96,0x79,0x6a,0xc8,0xc3,0x73,0xc9,0xd6,0xa7,0x0d, -0x4f,0x75,0x04,0x6b,0x67,0xd4,0x27,0x88,0x4a,0xaf,0xf3,0xb1, -0x04,0x39,0x5b,0xa5,0x8b,0xc8,0xc3,0xf8,0x37,0xb4,0x85,0xa6, -0x20,0xde,0xa4,0x8b,0x4e,0x87,0x3b,0x38,0xcd,0x57,0xf8,0x83, -0x26,0xcb,0xa0,0xb1,0xc5,0xb9,0xc7,0x09,0x78,0x4d,0x87,0xae, -0x4e,0x29,0x99,0xeb,0xeb,0xc1,0xc9,0x4c,0x01,0x7d,0x3d,0x2a, -0xf5,0xb7,0xd2,0xe0,0x47,0x4e,0x81,0xaa,0xa8,0x43,0x93,0x20, -0x5d,0x23,0x32,0x27,0x06,0x6c,0xf4,0x45,0x4a,0x97,0xcb,0x0e, -0x22,0x85,0x37,0xae,0x6e,0xa5,0x4e,0x6c,0x03,0x87,0xf4,0xbb, -0x55,0x82,0xab,0x4a,0x06,0x96,0xa9,0x05,0xb6,0x55,0xef,0xa8, -0x8f,0x4b,0xd7,0x79,0xbc,0xbe,0x8e,0x0b,0xb5,0x4c,0xa4,0x00, -0x1b,0x3d,0xb7,0xf1,0x95,0x4f,0x2e,0x30,0x59,0x8e,0x6e,0x57, -0x39,0xac,0x97,0xca,0x1f,0x20,0x42,0xcc,0xcf,0xaf,0xd6,0x17, -0x85,0x20,0x90,0x85,0xdc,0x79,0xc3,0x7d,0x45,0x42,0xb6,0xa1, -0xe1,0x8e,0x96,0xfe,0x81,0xbe,0x6c,0x9d,0xa3,0x49,0x36,0x68, -0x33,0x7f,0xf9,0xcb,0xff,0x1e,0x78,0x81,0x3c,0xb6,0xc8,0xd1, -0x30,0x3b,0xd0,0x3b,0xdd,0x7a,0xa7,0xe9,0xad,0x2c,0xe0,0x07, -0xc6,0x8f,0x1b,0x80,0xb8,0xc9,0x43,0x88,0x9d,0x43,0x9c,0xb4, -0xd3,0x09,0x6e,0x2d,0xc2,0x08,0x04,0x37,0xb3,0x39,0x7f,0x64, -0x25,0xe9,0x52,0xe4,0x35,0x67,0x9b,0xb2,0xe1,0x14,0xaf,0x7f, -0x03,0x6d,0x5d,0x9b,0xb4,0xc7,0xd3,0xc0,0xa1,0x23,0x9a,0xd8, -0xe8,0xe6,0xe7,0x71,0xfe,0xac,0xf4,0xfb,0x81,0xd9,0xaa,0x7c, -0x0b,0x74,0xbe,0x45,0x42,0x39,0x97,0x22,0x15,0xa3,0x9e,0xbe, -0x1d,0xbd,0x05,0x5a,0x99,0x61,0x11,0xfd,0xf9,0x66,0x95,0xc5, -0xa5,0xcf,0xa0,0x93,0xb7,0x50,0x16,0x2c,0xd8,0xe3,0xef,0xf8, -0xaa,0xe3,0x22,0xb8,0xbb,0xb3,0x8b,0x66,0x29,0x90,0x49,0x23, -0x9e,0x6b,0x2e,0x2f,0x71,0x62,0xb7,0xa1,0xf0,0xd6,0x76,0x99, -0xbd,0xcc,0x6e,0x92,0xfc,0xab,0x18,0x2f,0x87,0x0b,0x09,0xf5, -0x92,0x08,0x89,0xe6,0x97,0xf2,0x6b,0x22,0xe3,0x69,0x47,0xdf, -0x29,0x47,0x1d,0x0b,0x3d,0x75,0x72,0x06,0x7a,0xbf,0x18,0x3a, -0xc1,0x85,0xa9,0xa1,0x93,0x69,0x84,0x09,0x4f,0x2f,0x46,0x17, -0x9d,0xc8,0xcd,0xcc,0xe2,0xc1,0x55,0x98,0x34,0xe8,0x9d,0xe2, -0xd8,0x34,0xd7,0x7c,0x90,0x08,0xb9,0xa9,0x2e,0xc4,0x45,0xc7, -0xc5,0xa4,0xee,0x30,0xdb,0x18,0xed,0x8c,0xfa,0xaa,0x1f,0x03, -0x18,0x2b,0x83,0x45,0x1e,0xdf,0x3c,0xcb,0xe7,0xfe,0x12,0x27, -0x6b,0xe0,0x4a,0x96,0x2f,0x0a,0x39,0xc4,0x88,0x95,0x28,0xcd, -0xd9,0x61,0x5e,0x45,0x9c,0x8a,0xd2,0x9a,0xd7,0x2a,0x30,0x80, -0x40,0x56,0xaa,0xd0,0x10,0x97,0x42,0x39,0x2c,0x6c,0x64,0xf8, -0x31,0xe8,0xe6,0x37,0x49,0xb2,0x51,0xe1,0x8f,0xa7,0x22,0xd1, -0x78,0x9e,0x40,0x40,0xe3,0xf9,0x04,0x20,0x93,0x65,0x11,0xc5, -0xf9,0xfc,0x55,0xf6,0x53,0xb2,0xa4,0x93,0x47,0x3f,0xa9,0x00, -0x44,0xe4,0x15,0x14,0xc8,0x88,0x19,0x1d,0x16,0x49,0xca,0x05, -0x5d,0x33,0x4f,0xbf,0xc0,0x9c,0x7a,0x9b,0x13,0x05,0x9f,0xf4, -0xf5,0xe4,0x7d,0x54,0x30,0xa2,0x57,0xd9,0x97,0xc9,0xfb,0x14, -0xf4,0x5d,0x3e,0xba,0xa3,0x9b,0xa7,0x98,0x07,0xb7,0x5e,0x47, -0xcb,0xde,0x8c,0x12,0xbf,0xba,0xca,0xaf,0x93,0x57,0x99,0x84, -0x59,0x0a,0x88,0x0d,0x24,0x27,0x68,0x02,0x87,0x30,0x54,0xbd, -0xbb,0xc2,0x04,0xba,0xcf,0xb9,0x93,0x3a,0x54,0xa9,0x8f,0xdb, -0x52,0x3f,0x56,0xa9,0x4f,0x30,0xd5,0x32,0xf4,0x20,0x38,0x6c, -0x83,0x9d,0x16,0xf8,0x6f,0xc0,0xfd,0x3f,0x83,0xc1,0x6e,0xad, -0x90,0xcb,0xad,0x75,0x02,0x05,0xf6,0x9f,0x22,0xab,0x44,0x56, -0xd3,0xc1,0x00,0x24,0xe2,0x99,0x53,0x84,0x4e,0x20,0xd8,0x10, -0x40,0x9f,0x88,0x07,0xc2,0x41,0xf9,0x55,0x3c,0x3f,0x4f,0x26, -0x12,0x7e,0xaa,0x76,0xd6,0x0f,0x80,0xd0,0x38,0x05,0xc3,0x8e, -0x2c,0xf4,0xc4,0x27,0x3d,0xf8,0x87,0x17,0xa7,0x83,0xcf,0xfa, -0xa4,0x18,0xbf,0x81,0x44,0xad,0x1b,0x43,0xf7,0x06,0xb1,0x28, -0x74,0x1c,0xea,0xd0,0xb8,0xed,0x0e,0x32,0x41,0xe1,0x78,0x56, -0xf8,0x39,0xac,0x7b,0x40,0x2c,0x4c,0xb8,0xe6,0xc1,0x70,0x83, -0xd6,0xc4,0x27,0x9c,0xfb,0xc4,0xcf,0x40,0xb9,0x0e,0x3a,0x10, -0xc1,0x45,0x40,0x44,0xdd,0x85,0x66,0xdc,0xd4,0x0e,0x10,0xc6, -0x75,0x6d,0x20,0xcc,0x25,0x36,0xab,0x68,0x53,0x9d,0x6c,0xaa, -0x53,0x28,0xeb,0x04,0x8a,0xeb,0x6c,0xea,0x93,0x4d,0x0d,0xa1, -0xfa,0x04,0x0b,0xdb,0xac,0x9e,0xa2,0xca,0xb9,0x91,0x07,0x06, -0xc5,0xbb,0xbc,0x84,0x00,0x10,0x55,0x9d,0x44,0x9b,0x15,0xd0, -0x86,0x7f,0x98,0xa6,0xb8,0xdf,0x8f,0xb8,0xb0,0x53,0x68,0x87, -0xb8,0x3f,0x88,0xb8,0x24,0x0a,0x0d,0xfa,0x51,0x57,0x05,0x6b, -0x08,0x0e,0x34,0x68,0x2d,0xaa,0x7e,0x04,0x79,0x4f,0xb2,0xaa, -0x03,0x99,0x4e,0xb2,0x5a,0xd4,0x10,0x31,0xe0,0x88,0x01,0x45, -0x80,0x56,0x82,0x10,0x0c,0x50,0xe3,0xb2,0x00,0xd3,0x39,0xb9, -0xc6,0xdb,0xd8,0xa0,0xcf,0x57,0xfd,0xe0,0x44,0xfe,0xed,0xf8, -0xb0,0xb4,0xab,0x31,0xcc,0x7f,0x45,0x71,0x16,0xcf,0xcb,0x2c, -0x7f,0x53,0xbc,0x8b,0x06,0xa7,0x8b,0x6e,0x6f,0xf8,0x64,0xd4, -0x7f,0x6a,0x22,0xf1,0x0c,0xda,0x40,0xf4,0xb9,0x3a,0x32,0xc6, -0xaa,0xb7,0x81,0xc1,0xb3,0x11,0xec,0x80,0x11,0x89,0x9a,0xc9, -0x0e,0x75,0xe4,0x0f,0x46,0x51,0xcd,0xb1,0x01,0xfc,0xaa,0xdf, -0x01,0x2d,0x5e,0x25,0x69,0xa2,0x6a,0x4e,0xad,0xfb,0x1d,0x50, -0xe3,0x3b,0x3a,0x95,0xab,0x00,0x3a,0x4e,0x5f,0x36,0x7b,0x19, -0xaf,0x87,0x00,0xd5,0xad,0xe7,0xc0,0xa9,0x6e,0x35,0xc7,0xb4, -0x81,0x93,0x36,0xa0,0xb4,0x81,0x4c,0x7b,0x03,0x12,0x0a,0x6b, -0xa5,0x01,0xa8,0x96,0x7d,0xa8,0x26,0x47,0x1c,0x1f,0x0f,0xa2, -0x88,0x88,0x1e,0x73,0x44,0x27,0x1a,0x9e,0x48,0xf1,0x0c,0x39, -0xe6,0x69,0xff,0xf8,0xb8,0x2f,0x81,0x68,0xad,0x80,0x91,0x5d, -0x03,0x66,0xb4,0x15,0x39,0x00,0x14,0xd6,0x12,0x55,0x4b,0x28, -0x67,0x3b,0xf5,0xa1,0x6a,0x32,0x5f,0xa7,0xd7,0xef,0x0f,0x82, -0x20,0x30,0x5a,0x13,0x0d,0x30,0x0a,0x89,0x52,0x6a,0xb8,0xef, -0xe0,0x2a,0xaf,0xdf,0x49,0x4f,0x24,0x1a,0x05,0x04,0xd5,0x7a, -0x4c,0x29,0x7e,0xda,0x19,0x04,0xcd,0xd4,0x11,0x63,0x46,0x27, -0x1e,0x93,0x6a,0x0e,0x8c,0x05,0x70,0xdc,0x9d,0x78,0x2c,0x07, -0x04,0x16,0x3c,0xd9,0xd9,0xf4,0xf1,0xfd,0x81,0x8e,0x1c,0x35, -0xa7,0xd3,0xc6,0x90,0xe7,0xc3,0x3a,0x6f,0x8e,0xab,0x65,0xd4, -0x8b,0x07,0x6d,0xa5,0xdc,0x3f,0xd4,0x34,0x50,0xb6,0x0f,0x36, -0x07,0x81,0x1a,0xbd,0xee,0x44,0xf6,0x3a,0xd9,0xcf,0x4e,0xa8, -0x9f,0xf5,0x23,0x15,0xaa,0xac,0x5e,0x87,0x69,0xfc,0xd5,0xb7, -0xc7,0x1e,0x3d,0x48,0xf5,0xed,0x51,0xaa,0xaf,0x86,0xa9,0x81, -0x0d,0x3b,0x50,0xb0,0x03,0x1b,0x96,0x76,0x36,0xde,0x9c,0xc7, -0xab,0x33,0x92,0x6d,0x29,0x85,0x0a,0xf4,0xcd,0xf9,0xd0,0x00, -0x43,0xba,0x04,0x85,0x3c,0xd1,0x67,0xa7,0x8f,0x4f,0x34,0x90, -0xf9,0x3a,0xb5,0x70,0x4b,0x58,0x18,0x0b,0xe6,0x55,0x47,0x52, -0xdf,0x2d,0x25,0x6c,0x1f,0x07,0x85,0x79,0xdd,0x91,0xa1,0x4e, -0x29,0xc7,0xbd,0xbe,0xa8,0x1e,0x1b,0xf8,0x81,0xa8,0x1f,0x1b, -0x28,0xda,0x6a,0xa8,0x1e,0x77,0x14,0x0e,0xda,0x64,0xa8,0x1f, -0x77,0x55,0xde,0xc1,0xe8,0x21,0xad,0x10,0x4d,0x68,0x68,0x1a, -0xf0,0xd8,0x34,0x10,0x34,0x32,0x0d,0x78,0x68,0x82,0x10,0xa6, -0x0d,0x39,0x6d,0xc8,0x69,0x43,0x4e,0x1b,0x72,0xda,0x63,0x4e, -0x7b,0xcc,0x69,0x8f,0x39,0xed,0xb1,0xe5,0x31,0x81,0x8f,0x6c, -0x40,0x71,0x41,0x4d,0x9f,0x75,0x37,0x6b,0x71,0x22,0xca,0xe4, -0x72,0xf3,0x4f,0xfa,0xfd,0x85,0x7e,0xbf,0x82,0x95,0x6f,0x9e, -0xad,0xfe,0x69,0x07,0x7e,0x81,0x05,0x69,0x72,0x9d,0x66,0x57, -0x45,0x44,0xb3,0x34,0x9e,0x6a,0xd7,0xf0,0x6f,0x2e,0x61,0xcd, -0xe7,0x2f,0xf0,0x29,0x37,0x44,0xd1,0x19,0x89,0xde,0x34,0x1d, -0x29,0xd3,0x20,0xbc,0x59,0xb1,0x82,0x21,0x52,0xa8,0x08,0xb4, -0x13,0x2a,0x31,0x02,0xf5,0xad,0x65,0xba,0xc6,0xa5,0x82,0x6f, -0x86,0x8a,0xfd,0x2b,0x19,0xb2,0xc7,0x91,0x47,0xcc,0xb2,0x32, -0x66,0x31,0xc3,0x61,0x3c,0x6f,0x97,0x27,0xcf,0x5e,0x58,0x75, -0xd4,0xe2,0x06,0xd5,0xa3,0xda,0x84,0x40,0x3d,0x5a,0xf6,0x70, -0x49,0xfb,0x2a,0xf3,0xab,0xce,0x0a,0x92,0x5c,0x1d,0x02,0x55, -0x1a,0xfb,0xd2,0xcc,0x4b,0xc0,0xe5,0xa0,0xfa,0xd5,0x98,0xce, -0x9b,0x54,0x7d,0x50,0xee,0xbf,0x35,0xe8,0xf8,0xa0,0xcc,0xd7, -0x5e,0x58,0xff,0xfa,0xa2,0x7f,0x86,0xdc,0xbf,0x3a,0xf3,0xe5, -0x7d,0x6d,0x71,0x99,0x5d,0x3f,0x14,0xd7,0xb7,0x87,0xdb,0xe2, -0x03,0x30,0xcd,0xbd,0x90,0xba,0x42,0x04,0x7d,0x5d,0xa2,0x00, -0x55,0x98,0xfa,0x45,0x54,0xeb,0x28,0x50,0x88,0xb5,0xd0,0x1b, -0xc0,0xc7,0x53,0x23,0xff,0x06,0xf6,0xe3,0xa9,0x68,0xac,0x21, -0x7c,0x93,0x65,0x30,0x25,0x9a,0x0c,0xb5,0x9d,0x52,0x63,0x86, -0x14,0x85,0xae,0x23,0x3b,0x28,0x44,0x11,0x29,0xcd,0x4a,0x74, -0x1c,0x36,0x1a,0x6c,0x0e,0x94,0xc2,0xab,0xb1,0x3a,0xa9,0xd6, -0x00,0x00,0x83,0x63,0xc4,0xc1,0x9a,0xfe,0xfe,0x62,0x33,0xe8, -0x2b,0x9b,0xd5,0x4f,0x6c,0x56,0xdb,0x4c,0x69,0x63,0xc9,0x01, -0x86,0x38,0xec,0x78,0x00,0x33,0xda,0x5a,0xd2,0x62,0xc1,0xaf, -0x62,0x40,0x53,0x12,0x8a,0x5d,0x49,0x78,0xbc,0x2b,0x09,0x1f, -0x5b,0x95,0x1e,0x9e,0x54,0xdd,0x9d,0x32,0x20,0xb6,0x56,0xb1, -0xbf,0xec,0x56,0x7c,0x4f,0xf5,0x0e,0xc9,0xc7,0x21,0x49,0xb8, -0xa7,0x8a,0x0f,0x92,0x13,0x47,0x12,0x5a,0xc4,0x7c,0xd0,0x2a, -0xe6,0xc3,0xe9,0x21,0xa9,0xf9,0x49,0x31,0x73,0x87,0x95,0x7f, -0x28,0x23,0xf7,0xcb,0xd5,0x0e,0x13,0x1b,0xc4,0xff,0x9e,0x92, -0xd5,0xce,0xd0,0x36,0x76,0x1a,0xf4,0x36,0xf3,0xde,0x7d,0xb0, -0x24,0x3e,0xa0,0xb1,0x96,0xbd,0x77,0x57,0xf1,0x22,0x8f,0xcb, -0x74,0x7e,0x98,0x87,0x1f,0xc4,0xa9,0x07,0xf3,0xc1,0xae,0xdf, -0xbf,0x3e,0x44,0x38,0xda,0xe8,0xfd,0xf5,0xad,0xfb,0xc0,0x56, -0xf8,0xed,0x75,0x2b,0x77,0xdb,0x6e,0xb0,0xdb,0x76,0x50,0x12, -0x1b,0x7b,0x47,0x4a,0xc3,0xc2,0x2d,0xc9,0xcb,0x18,0x95,0x9a, -0xd3,0xc9,0xbf,0xbe,0x7b,0x55,0x4e,0x4f,0x83,0xb1,0x6f,0xda, -0xd7,0x6a,0xd4,0x20,0xf4,0xc8,0x71,0x88,0x95,0xd3,0x82,0xc4, -0xae,0xe4,0xe8,0x73,0x4e,0x77,0xb2,0x95,0x3b,0xc0,0xf3,0xae, -0x81,0x07,0x74,0xb2,0x0f,0xe8,0x93,0x81,0xa3,0x38,0x46,0xf3, -0x36,0x0d,0x32,0xb2,0x3a,0xf0,0xef,0x25,0x7f,0x1f,0x22,0xf5, -0xbf,0xb9,0x3d,0x5f,0x35,0x65,0x75,0xd0,0x94,0xd5,0xe1,0x87, -0x0e,0x64,0xff,0x69,0xfa,0x61,0xec,0x85,0xd6,0x7e,0x2f,0x4f, -0xb1,0x62,0xd2,0x3a,0xfa,0x09,0x7b,0x82,0xb7,0x06,0x20,0xa3, -0x17,0x18,0xad,0xa0,0x53,0x59,0xdd,0xf3,0xd3,0x69,0x07,0xb0, -0x4e,0xed,0x3d,0x64,0x61,0xa9,0x81,0x9f,0x38,0x6a,0xe0,0xa7, -0xce,0x30,0xf8,0xec,0x0f,0x22,0xcf,0x21,0xae,0x49,0x9a,0x43, -0xd9,0x1e,0xc2,0xde,0x4b,0x03,0xd4,0x7f,0xf3,0xc2,0x65,0x8f, -0xce,0x3e,0x79,0xf5,0xb2,0xd5,0xcb,0x25,0x99,0x4f,0x6d,0x19, -0xc8,0xb6,0xd0,0xc7,0xee,0x3e,0x96,0xd5,0xd8,0xf1,0x2d,0xe8, -0x04,0x69,0x67,0xa7,0xf7,0x8f,0x5b,0xaf,0xfd,0x39,0x4b,0xac, -0x5f,0xbb,0xa0,0xfa,0x90,0xe5,0xd3,0x07,0x2c,0x96,0x3e,0x64, -0x69,0xf4,0x01,0x0b,0xa1,0x5f,0xbf,0xec,0xf9,0xb5,0x8b,0x9c, -0x3f,0x7b,0x49,0xf3,0x5f,0x79,0xf5,0xf1,0x5f,0x6d,0xd1,0xf0, -0xdf,0x51,0xbf,0xff,0xef,0xa6,0x8e,0xff,0xdf,0x51,0xaa,0xff, -0x47,0x2d,0xfe,0xcf,0xa6,0x16,0xff,0xbf,0xa5,0xdc,0xfe,0x11, -0x2a,0xaa,0x73,0x1e,0xfe,0x7b,0x68,0xa7,0x8e,0x6e,0xaa,0xb8, -0xf8,0x21,0x5a,0xe9,0xef,0x4c,0x91,0x45,0x8f,0x45,0xcd,0x07, -0x2a,0xa2,0xf7,0xe8,0x9e,0xf6,0xe9,0x77,0x12,0x93,0x2b,0x27, -0xed,0x70,0x8b,0x02,0x68,0x48,0x80,0x49,0xd1,0xe2,0x31,0x5e, -0x5b,0xe9,0x11,0x58,0xd0,0xab,0xda,0x2f,0x53,0x57,0xdb,0xa0, -0x57,0x0f,0xda,0xd3,0x6a,0x4c,0xeb,0xef,0x49,0xeb,0x2c,0xa4, -0x69,0xdb,0x56,0xdf,0x67,0xcb,0x6c,0x87,0x84,0xc7,0xc7,0x58, -0xb0,0xe3,0xa2,0xd0,0x49,0x0f,0x04,0xdf,0x66,0xcd,0x7a,0x65, -0xb2,0xc6,0xdb,0x34,0x32,0x83,0x0c,0xf9,0x3a,0x3e,0x10,0x56, -0x45,0x4d,0xed,0x71,0xbf,0x7a,0x4f,0xed,0x31,0x49,0xd5,0x9e, -0xc0,0x0e,0xd7,0x7e,0x6f,0xe5,0xf7,0xd4,0x0b,0x51,0x7e,0x50, -0xbd,0x28,0x43,0x4b,0xbd,0xac,0x2a,0x90,0x4d,0x9d,0x3c,0x50, -0x12,0xda,0x68,0x29,0xba,0xbd,0x0c,0x87,0x62,0x05,0xff,0xf0, -0x02,0xc4,0x35,0xfc,0x9b,0x87,0x9f,0x88,0x22,0xfc,0x58,0xbc, -0x83,0x7f,0x25,0xc4,0xc7,0xe1,0xa7,0x5b,0x91,0x27,0xd1,0xe4, -0xd4,0x9f,0x7c,0xfb,0xf2,0x6f,0x3f,0x7f,0xf5,0xd3,0xbf,0xbe, -0x7a,0xf6,0x6f,0x97,0xab,0xf3,0xeb,0x79,0xf1,0xae,0x8c,0xdf, -0x4f,0x83,0xd3,0xa5,0x38,0xfd,0xcb,0x5f,0xfe,0x72,0x2a,0x4e, -0xfd,0xd7,0x8b,0xa0,0x8b,0xc1,0xd7,0xc5,0x9d,0xb8,0xc3,0xb8, -0x69,0x8b,0xb9,0x03,0x9a,0xca,0xb5,0x1d,0x77,0x61,0x3c,0x1e, -0x63,0x4a,0x47,0x10,0xc6,0xb6,0x0f,0x23,0x95,0x9f,0x12,0x32, -0xd7,0x0a,0xe9,0x17,0xa8,0xc2,0x63,0xab,0x90,0xff,0xc8,0xde, -0x10,0xca,0x15,0x1a,0xb6,0x68,0x88,0x3f,0xc4,0x83,0x90,0x7c, -0xdd,0x48,0xc3,0x47,0x69,0xa3,0x87,0xae,0xc4,0x8a,0x56,0x23, -0x3d,0x20,0x99,0x16,0x69,0x4b,0x91,0x71,0xcb,0x57,0xc0,0xeb, -0xea,0xee,0x0e,0x57,0x48,0x59,0xaf,0xc6,0x8f,0x34,0x8f,0xb0, -0x4d,0xd6,0x49,0xfe,0x63,0xbc,0x48,0xaf,0x0a,0x8c,0xcb,0x30, -0x2e,0xbb,0x2a,0xed,0xb8,0x22,0x8e,0xa0,0x3d,0xc8,0x43,0xf9, -0x33,0xb4,0x53,0x86,0xb8,0xa0,0xab,0x4c,0x36,0x86,0x22,0xa1, -0x64,0xa0,0xbf,0x25,0x71,0xe4,0x9c,0x76,0x09,0xbc,0x33,0x9f, -0xe6,0x63,0x73,0x60,0x01,0xda,0x65,0x88,0x7d,0x8f,0x87,0x96, -0x34,0x87,0xb2,0x00,0x9f,0xa0,0x63,0x32,0x15,0x9b,0xe5,0x18, -0x05,0x09,0x03,0x8c,0xb5,0x97,0x9f,0x96,0x45,0x62,0x79,0x6e, -0x6a,0xeb,0x5c,0x03,0xcf,0xc8,0x8e,0x71,0xec,0x67,0x13,0xf2, -0x1e,0xc8,0x5c,0x5f,0x78,0xd3,0xbb,0xbb,0x9d,0x28,0xb6,0xbe, -0x23,0x3c,0x9c,0x8b,0xce,0xe2,0xb1,0x5d,0x14,0xee,0x46,0x06, -0x01,0x1c,0x45,0x57,0x75,0x41,0x10,0xf2,0x31,0x9f,0x75,0x10, -0x4e,0x57,0x73,0x0c,0x4d,0xee,0xa9,0x1f,0x99,0x34,0x89,0x52, -0x1f,0x88,0xbf,0x4f,0x22,0x45,0x2d,0xf9,0xb9,0xe6,0x3f,0x74, -0x15,0x89,0x9a,0x8d,0xdb,0x0c,0x6d,0xf2,0x58,0x5d,0xc8,0xd8, -0xd1,0xdb,0xdd,0x9d,0xba,0xf1,0x11,0xe9,0x38,0x75,0x22,0xee, -0x18,0x64,0x00,0xb2,0x53,0x65,0x9d,0x60,0x71,0x36,0x17,0x7d, -0xa1,0xcd,0x16,0x98,0xe9,0x16,0x7b,0xe5,0xd1,0xa5,0xf1,0xcc, -0xc7,0xf8,0x79,0x15,0xcc,0x17,0x4c,0xc2,0x9d,0x52,0x9e,0x04, -0x20,0x11,0x65,0xf4,0xf8,0x24,0xb7,0x8e,0xa5,0xba,0xa5,0xa8, -0xbb,0x79,0x60,0x1d,0x9f,0x75,0xf3,0xd6,0x98,0xd2,0x8e,0xe9, -0xb4,0xc7,0x38,0xb9,0x3a,0x3b,0x98,0x31,0xa6,0x93,0x37,0x73, -0xb5,0xc4,0x94,0xcd,0xd2,0x5b,0x62,0x5c,0x0a,0x19,0xb3,0x3d, -0x1f,0xa9,0xeb,0x35,0xa1,0xb2,0x3d,0x32,0x8c,0xf0,0x87,0x27, -0x65,0xbc,0x7e,0xdc,0x47,0x19,0xaa,0xa2,0xbc,0xe6,0x90,0xb0, -0x45,0xbf,0x9b,0xd7,0x2e,0x55,0x64,0x54,0x66,0x22,0xb0,0xb8, -0xda,0xa5,0x69,0x67,0x39,0xc6,0xb7,0x7a,0xc2,0xb2,0x51,0x3c, -0x14,0x1a,0x95,0xd0,0x12,0x78,0x93,0x11,0xbd,0x0a,0x30,0xbf, -0x41,0xe2,0x4a,0x57,0x4b,0x75,0xee,0x02,0x85,0xca,0xa2,0xca, -0xd4,0x02,0xbf,0x1e,0x93,0x69,0x55,0x5e,0x9d,0x50,0x80,0x90, -0x9a,0x3a,0x74,0x5a,0xea,0xd0,0x6d,0xa1,0x1a,0xe3,0x5a,0x4b, -0xbe,0xda,0xfc,0xaa,0x72,0xdb,0x78,0xd7,0xc6,0x2d,0x8c,0xdb, -0xee,0x19,0x35,0x70,0x64,0x95,0x3d,0xb4,0x7d,0x86,0xdc,0x44, -0x1f,0x36,0x6a,0xb8,0xb6,0xd3,0x3d,0x6b,0x42,0xa6,0xd7,0x47, -0xac,0x91,0x64,0xd3,0x98,0xa2,0xff,0x48,0x32,0xac,0xf9,0xf3, -0x3e,0x32,0xf8,0x2a,0xe1,0x5e,0x42,0x56,0x37,0x38,0xca,0x94, -0xfa,0xba,0xa1,0x58,0xcd,0x75,0xc4,0x57,0xf1,0x66,0xc4,0xbc, -0xa7,0x24,0x39,0x9e,0xad,0x6e,0xc6,0xab,0x9b,0xb0,0x79,0xaf, -0xd6,0x80,0xc9,0xe6,0x82,0xcc,0x2a,0xc3,0x7c,0xbc,0x9a,0xb7, -0x67,0x00,0x20,0xb1,0xc3,0x2b,0xcb,0xfe,0xe3,0x6a,0x65,0xe8, -0x57,0x73,0xde,0x40,0x4f,0x7a,0x03,0x35,0xeb,0x55,0x43,0x3d, -0xd6,0x56,0xc3,0x31,0xfe,0x84,0x15,0x59,0xb4,0xa8,0xd8,0x1a, -0x63,0xeb,0x61,0x58,0x0f,0xfe,0x13,0x54,0xd8,0x9d,0x3f,0x4d, -0x0f,0xc0,0x1b,0x69,0xb6,0xb0,0xd3,0xdd,0x34,0xd7,0xac,0x19, -0xf3,0x7c,0xbf,0x66,0x43,0x7d,0x01,0x2c,0x71,0x1a,0x75,0x13, -0xcf,0xa1,0x32,0x50,0x41,0xf2,0x60,0x06,0x95,0xb2,0x7c,0x95, -0xea,0x2a,0x92,0xd5,0xbe,0x66,0x35,0xf9,0x13,0x8b,0xd4,0x04, -0x24,0x5d,0x84,0x8d,0x07,0xa1,0xfe,0x16,0x7d,0x60,0x1f,0x7c, -0xf2,0xa3,0x43,0xab,0xbb,0x3b,0xe9,0xfa,0x54,0x86,0xe1,0xcf, -0xb2,0xb7,0x5c,0x65,0xb3,0x78,0xf5,0x6c,0xb5,0x39,0x8f,0x09, -0xf6,0x44,0xcf,0x68,0x08,0xf2,0xbd,0x85,0xd4,0x0a,0x63,0x45, -0x31,0xf8,0x53,0x59,0xaf,0xd0,0xba,0x77,0x95,0xa1,0x9d,0x4f, -0x46,0xa4,0xab,0x34,0x74,0x50,0xa4,0x5d,0x93,0x42,0xbb,0x31, -0x77,0xfd,0x66,0x0b,0x06,0xf7,0x34,0xcf,0xea,0x06,0xcd,0xfb, -0x0e,0x12,0xca,0xd8,0x6c,0x52,0x9d,0x18,0x24,0x88,0x23,0x9a, -0xe4,0x4a,0x76,0x08,0x5b,0x6a,0xa0,0x01,0x9a,0x12,0xe1,0xbb, -0x42,0x16,0xdc,0x23,0x1f,0xd7,0xcb,0x97,0xf0,0xfd,0x75,0x5c, -0xd0,0x14,0x4d,0xb9,0x30,0xc0,0xde,0x95,0x02,0x07,0x80,0xaf, -0x1b,0x3b,0x60,0x1c,0x85,0x4a,0x9b,0x26,0x1b,0x38,0x19,0xb4, -0x88,0xd2,0x33,0xe0,0xb1,0x14,0xa5,0x62,0x9e,0xac,0x13,0xbd, -0x73,0xcf,0x8e,0x85,0xf0,0xfd,0x3b,0x94,0x13,0xed,0x59,0x88, -0x9c,0x6f,0x20,0x60,0x8f,0xc4,0x67,0x77,0xc7,0x1d,0xa3,0x6d, -0x08,0xdc,0x6d,0xf7,0x1f,0x31,0xd6,0xbb,0x3b,0xb9,0x56,0x36, -0xd7,0xa0,0x7c,0xcb,0x8f,0x70,0x00,0x32,0xd5,0x22,0xe2,0xe4, -0xb0,0x17,0x7f,0x1a,0xe4,0xff,0x88,0x73,0x60,0x93,0x6c,0x7c, -0x10,0x69,0x87,0x3c,0x7d,0x9f,0x21,0xb3,0x9d,0xf7,0x0a,0xab, -0xbb,0xa0,0xde,0x74,0x23,0xce,0xb9,0x4f,0xd8,0xe4,0xeb,0xf3, -0x85,0x7d,0xd5,0x3d,0xf3,0xd5,0x50,0x0a,0x15,0x3d,0x50,0xcf, -0x4c,0x57,0x12,0x69,0xdc,0x55,0xdb,0x51,0xa2,0xe9,0x86,0x15, -0x06,0xce,0x23,0xe5,0x18,0x99,0xb4,0xf7,0x7b,0x7b,0x68,0x44, -0xc2,0x2c,0x95,0xc4,0x91,0x6f,0x75,0xfe,0xe0,0x8f,0xed,0x9f, -0xd4,0x06,0x92,0x79,0xd0,0x55,0xfd,0xc6,0x50,0x13,0xfc,0x4f, -0xaf,0xfd,0x6d,0xbd,0xd6,0xe5,0xaf,0x6d,0xfe,0x4f,0x3d,0x00, -0x26,0xc7,0x0f,0xee,0x01,0x2d,0xc2,0x6f,0x2e,0x42,0xff,0x79, -0x1d,0xe0,0x37,0xce,0xe1,0x1f,0xdc,0x27,0xfe,0x47,0x32,0x7f, -0x2f,0xc9,0xfc,0x00,0xd5,0xc5,0x99,0x7d,0x1a,0xc2,0xfb,0xe2, -0x32,0x5e,0x7e,0xb8,0xf4,0x72,0xc5,0x92,0x9c,0xae,0x43,0xff, -0x39,0xd2,0x9a,0xf5,0x52,0x24,0xf5,0xf8,0x58,0x7e,0xf4,0xae, -0xf2,0x15,0xad,0xcc,0xe1,0x2f,0xaa,0xec,0x1c,0x1b,0x29,0xca, -0x7a,0xf8,0x74,0x22,0x57,0x8e,0x40,0x40,0xc3,0xb2,0xf2,0x71, -0x14,0xdf,0x8f,0x90,0x73,0x0e,0xca,0xcd,0xc8,0x1a,0xfe,0x77, -0xca,0x6b,0x9d,0x16,0x76,0xa0,0xcc,0x7c,0x51,0xe1,0xce,0x4d, -0x45,0x5b,0x36,0xbe,0x72,0x2f,0x45,0x04,0x93,0xbb,0xd9,0xf1, -0xcd,0xe9,0x50,0xf9,0x5e,0xb1,0x63,0x43,0xbc,0x94,0x81,0x39, -0x6b,0x99,0x53,0xfa,0x34,0x21,0x18,0xe5,0xa0,0x76,0x7c,0x8e, -0x99,0xad,0xe7,0x7c,0xac,0x14,0x44,0xa0,0x88,0x41,0x1e,0x24, -0x8b,0x9d,0x2e,0x22,0x45,0x97,0x3a,0xaf,0xee,0xaa,0xc1,0x18, -0x3f,0xd0,0x87,0x4b,0xcf,0xc8,0x85,0x44,0x24,0xf4,0xd8,0xd7, -0x90,0x1e,0x7c,0x8e,0xf3,0x77,0x98,0xfc,0x6f,0xfe,0xb4,0x01, -0x0f,0xe6,0xcb,0x8c,0x9e,0x5c,0xec,0x71,0x59,0xe4,0x56,0x88, -0x9d,0xff,0x65,0xd8,0x4f,0xd0,0xd7,0x2f,0x79,0x98,0x51,0x2d, -0xa2,0x1e,0x57,0x92,0x69,0xca,0xd7,0x8c,0xc5,0x72,0x80,0x88, -0x91,0xad,0xb3,0xa4,0x4c,0xe7,0xde,0x07,0x0f,0x89,0x50,0x0e, -0xae,0xd5,0xc7,0x40,0x5a,0x11,0x5f,0x27,0xd4,0x9b,0x8d,0x6d, -0x87,0x1c,0x9d,0xa5,0x38,0xe0,0x96,0x03,0x5d,0xeb,0xf2,0x65, -0xae,0x13,0xe7,0x72,0x17,0xaa,0x33,0x0b,0xad,0xcf,0x2c,0x28, -0x4b,0xe8,0x1b,0x39,0xc4,0x87,0x21,0x28,0xdd,0x16,0x31,0x8a, -0xa4,0x2f,0x50,0x1c,0xfe,0x3c,0x95,0x85,0x24,0x27,0x23,0x9e, -0xd2,0x7d,0xbf,0x5f,0xa5,0xb5,0x0c,0xfe,0xb0,0x29,0xe0,0xc0, -0x04,0xc0,0x09,0x3b,0xf4,0x63,0xaf,0xa3,0x36,0x39,0x3e,0xc6, -0x81,0xbf,0x28,0xb3,0x1c,0x1a,0xb3,0xd1,0x5f,0x94,0x9e,0x8f, -0x9e,0x5a,0x8d,0xd3,0x59,0xb5,0xc3,0xdc,0xec,0x48,0xcd,0xd5, -0x01,0xe6,0x6a,0xae,0x10,0x1a,0x05,0x28,0x85,0xfd,0x61,0x05, -0xec,0x76,0x53,0xe0,0xe5,0x81,0xbe,0x64,0x2f,0x28,0xfa,0xd3, -0x03,0xeb,0x05,0x9b,0x52,0x0d,0x6f,0x07,0x9b,0x74,0xff,0x15, -0x5f,0x02,0x79,0xe0,0x48,0x42,0x8b,0x22,0x7e,0x3a,0x24,0xae, -0x12,0xf4,0x53,0x88,0x0e,0xad,0x0a,0xb1,0xac,0xc4,0xb2,0x16, -0xe8,0x86,0xf5,0xad,0xb8,0x6c,0x19,0x51,0x9c,0x49,0x8b,0x96, -0x52,0xfb,0xd6,0x2f,0xd2,0xf9,0xa5,0x33,0xf4,0x68,0x5b,0x26, -0xcc,0x48,0xa5,0x9b,0xe1,0x07,0xc9,0x90,0x2f,0xbc,0xa0,0x6f, -0xb0,0x17,0x18,0x8f,0xee,0x34,0x15,0x69,0x32,0x8d,0x43,0x26, -0x75,0x59,0xc9,0x04,0xea,0xb3,0xcb,0x5a,0x86,0x48,0x0b,0x6b, -0x1d,0x0d,0xa8,0x86,0xea,0xa8,0xf3,0xf8,0x78,0xc7,0x12,0xac, -0x0b,0x00,0x5d,0x84,0xc0,0x3b,0xf1,0x97,0x11,0x92,0xa5,0xc8, -0xbf,0x7c,0xfa,0x16,0x9f,0x71,0x0d,0x3c,0x7c,0xea,0x16,0xe7, -0x0c,0x4c,0x9c,0xbc,0x9d,0xd2,0xeb,0x2c,0xab,0xb8,0x4e,0x72, -0xf4,0xda,0x29,0xe7,0x4e,0xe4,0x0b,0xf0,0x44,0x82,0x28,0xae, -0x8c,0xfe,0x17,0xbf,0x28,0x8b,0xa8,0x99,0x52,0x87,0x41,0xaa, -0x84,0x26,0x16,0x0b,0xd4,0x46,0x66,0x70,0xed,0x27,0xf3,0xd1, -0xaf,0x22,0xd3,0x60,0x96,0xec,0xbf,0x8f,0x42,0x09,0x66,0xe3, -0xd8,0xcb,0x62,0xd9,0x04,0x4e,0x17,0x37,0xa2,0x6c,0xc6,0x3a, -0x7e,0xe1,0x4d,0x7b,0xaa,0xc5,0x50,0x2f,0x5d,0x8c,0xb5,0xdb, -0x8c,0x25,0x43,0x08,0x3d,0x3d,0x85,0x14,0xde,0xee,0x3a,0xd8, -0x58,0x8a,0x8d,0xed,0x62,0xf8,0x26,0x9a,0xa9,0x27,0x81,0x40, -0x13,0x99,0xe9,0x57,0x7f,0x84,0xf4,0xd4,0xd2,0x01,0x71,0x1a, -0x9c,0xdc,0x08,0xe9,0x9b,0x05,0x82,0xf5,0xe0,0xe4,0x5c,0xb0, -0x37,0x16,0x4a,0x1d,0x62,0xea,0x50,0xa7,0x0e,0x21,0x15,0x0b, -0x8b,0x40,0x6d,0xcd,0x13,0xa8,0xe3,0x4b,0xf2,0xcc,0xa1,0x1f, -0x60,0xb0,0x1d,0x34,0xf1,0x63,0xbc,0x1b,0x76,0x92,0x61,0xfb, -0x39,0x86,0x70,0xb3,0xa3,0x20,0x4e,0x3c,0x8e,0x27,0x7f,0x69, -0x3f,0x01,0x80,0x8f,0x50,0xd0,0x5d,0xd4,0x8b,0x4b,0x2a,0xc8, -0x4e,0x38,0xd4,0x49,0x27,0x66,0xb3,0x0e,0x9f,0xd2,0xf9,0x85, -0x3b,0x36,0xa0,0x2a,0xc2,0x8d,0x80,0xc3,0x03,0x39,0x8a,0xde, -0x19,0x22,0xee,0xee,0x38,0x4e,0xb5,0xe2,0x23,0x3b,0xa8,0xfd, -0xb2,0xc8,0xb6,0x94,0x13,0xf1,0xa3,0xc1,0x48,0x3e,0xd0,0xcc, -0xe5,0xf0,0x00,0x73,0x9e,0x96,0x82,0xbc,0x19,0xc1,0xb8,0xf2, -0xb6,0x65,0x4c,0x39,0x8f,0xd7,0xa0,0x9d,0x59,0x43,0x4a,0xea, -0x8e,0x1a,0xdd,0x6e,0xfa,0x34,0xea,0x8f,0x5a,0x47,0x8d,0x85, -0xd3,0xfd,0x17,0xf7,0x77,0x7f,0x76,0xab,0x04,0x9d,0xbb,0xa5, -0xf7,0xe1,0x23,0xcf,0x58,0x10,0x3f,0x45,0x4d,0x24,0x46,0x8d, -0x9e,0xa7,0xe2,0x7b,0xd6,0xf3,0x0d,0xd0,0xc3,0x1e,0xe1,0xdd, -0x6a,0xa8,0x67,0x24,0xeb,0xd2,0x43,0x9e,0x6b,0x24,0x92,0xdf, -0xdd,0x05,0xf2,0xaa,0xbb,0xc0,0x89,0x4e,0x35,0x94,0xe9,0x02, -0xc8,0xa6,0x91,0xdd,0x25,0xf4,0x43,0xca,0x8d,0xe2,0xc6,0x88, -0x1b,0xa7,0x33,0xc8,0xf0,0x0a,0xa0,0x0b,0x3a,0x42,0x11,0xbb, -0x6d,0x1b,0x3e,0x1a,0xb8,0x42,0x80,0x99,0xf0,0xfd,0xfa,0x16, -0x58,0xf2,0xb4,0xdb,0x32,0x4f,0x38,0xed,0x9a,0x89,0x99,0x48, -0xa9,0x85,0x06,0x50,0xe7,0x65,0xef,0x0d,0x1a,0x4d,0x67,0x50, -0xf3,0xea,0x44,0x87,0x44,0x6d,0xbe,0x71,0x0e,0x68,0xd9,0x61, -0x53,0xed,0x49,0xea,0x67,0x63,0x7f,0x6d,0x16,0xa9,0xce,0x8c, -0x5b,0x6d,0x30,0x7f,0xee,0x48,0x1a,0x8c,0x27,0xc7,0xc7,0xf4, -0x0a,0x3f,0x0e,0x11,0xa6,0x06,0x8a,0xa5,0xd9,0x48,0xd1,0xdc, -0xa8,0x3c,0x1e,0x76,0xfc,0xca,0x8a,0xcf,0x76,0xa5,0x56,0x75, -0xb5,0x59,0xd4,0x98,0xca,0xd1,0xcf,0xdc,0xa3,0x1d,0xb2,0xc7, -0x8f,0x06,0xe1,0x83,0xb9,0xa6,0x1b,0x36,0x66,0x9a,0xb9,0x9b, -0xa0,0x86,0x34,0x56,0x85,0x41,0xdb,0x06,0x6e,0xfd,0x70,0xc4, -0xf9,0xaf,0x56,0xbf,0x15,0xd3,0xfc,0xa0,0xfa,0x91,0xe4,0xee, -0x57,0xc7,0xac,0x1a,0x2b,0xb7,0x45,0x87,0x25,0xde,0x9a,0x71, -0x50,0x01,0xfd,0x5b,0xda,0x14,0x28,0xe2,0x5c,0xd6,0x53,0x4e, -0x53,0x0d,0xc7,0x38,0x9e,0x3d,0xa2,0xc9,0xc8,0x3e,0xb3,0xd1, -0xac,0x89,0xd4,0x9c,0x47,0x1a,0xae,0x28,0x2f,0x37,0x7c,0xfd, -0x56,0x3c,0x82,0x75,0x43,0x1c,0x75,0x5b,0x96,0x1e,0xc2,0xf5, -0xc1,0xd6,0xf4,0xbf,0x16,0xd8,0x07,0xed,0x62,0x83,0x6e,0x42, -0x8a,0x13,0x18,0x53,0xd0,0x4b,0xd9,0xb2,0xee,0xf8,0x8e,0x5b, -0x36,0x72,0x19,0x41,0x29,0x74,0x61,0x9a,0x00,0x1c,0x37,0x6d, -0x7a,0xa6,0xb0,0x1a,0x72,0x53,0x41,0x2e,0xb6,0x60,0xd1,0x47, -0x75,0x51,0xe3,0xa8,0xce,0x36,0x09,0x69,0x4b,0x97,0x56,0x22, -0x74,0x20,0x1a,0xf9,0xc6,0x04,0xc3,0x9c,0x9e,0x3e,0x06,0x25, -0x9f,0x4e,0x9a,0xb9,0x6e,0xf0,0xe9,0x3f,0xee,0xbb,0x4b,0x30, -0xcd,0x2d,0xf7,0xea,0xb2,0x16,0x9b,0xe8,0x96,0x5e,0x93,0x91, -0xad,0x26,0x70,0xf4,0x0b,0x77,0xfc,0x72,0x3d,0xea,0x6f,0x05, -0xad,0xbb,0xdb,0x93,0xf0,0x30,0x2e,0xb4,0x84,0x28,0x13,0xb6, -0x9b,0x43,0x74,0x41,0xfc,0x43,0x06,0x63,0xee,0x8b,0x35,0x9f, -0xd8,0x8d,0xfd,0xc6,0xe9,0x9d,0xd8,0x01,0x21,0x5b,0x11,0x1a, -0x73,0xd9,0x18,0xc6,0x96,0xd0,0xfb,0x90,0x3b,0x47,0x9b,0xc5, -0x61,0xe4,0x71,0x3e,0xdf,0x4f,0xb8,0x6d,0x54,0x63,0x63,0xa1, -0x48,0x04,0xdc,0xb2,0xcd,0xce,0x5e,0xe2,0xac,0x23,0xe7,0x62, -0x1f,0x06,0x3a,0xd6,0xdd,0x4b,0x82,0x63,0xea,0xb2,0x07,0x03, -0x1b,0x9f,0xec,0xc7,0xd1,0x30,0x4e,0x69,0xc7,0xb2,0x55,0xd6, -0x4a,0xa8,0x81,0xaa,0x07,0xd3,0xf5,0x3a,0x88,0xeb,0xa9,0x96, -0x73,0xaa,0x5a,0x6c,0xac,0xa5,0xa3,0xd5,0x69,0x2d,0xf9,0x8b, -0x09,0xd5,0xe2,0x52,0x32,0x91,0x5f,0x7c,0x0e,0xed,0x25,0x27, -0x47,0x4b,0xea,0x55,0x82,0x21,0x36,0x60,0x59,0x54,0x2b,0x22, -0x16,0x32,0xb5,0xbb,0xcd,0x4f,0x20,0xa9,0x0d,0x1f,0x29,0x9b, -0x7a,0x97,0x48,0x48,0x92,0x14,0x69,0x42,0x62,0x57,0xa5,0x00, -0xd7,0x61,0x38,0x53,0xb5,0xd4,0x1a,0x1d,0xd7,0x52,0x4d,0x6f, -0x5c,0x39,0x35,0x19,0x50,0x9d,0x68,0xe4,0xb4,0xe6,0x93,0xb9, -0xac,0x95,0x1b,0xbf,0xb1,0xeb,0xe5,0x26,0x71,0xa4,0xac,0x99, -0x9b,0x84,0x51,0x01,0x57,0xb2,0x91,0x70,0x85,0xbe,0x22,0xa9, -0xbe,0x6e,0x02,0x46,0x05,0xb2,0xea,0x6e,0x0a,0xc5,0x05,0x42, -0x8e,0xd9,0xa1,0xfc,0xbb,0x6d,0x58,0x93,0xfd,0xa8,0x56,0x9f, -0x6d,0x06,0x65,0xe4,0x27,0xfb,0x2b,0x02,0xf4,0x19,0x5e,0xcc, -0x4b,0xf9,0x7a,0xc4,0x22,0xb9,0x4e,0xe7,0xc9,0x0f,0x69,0x95, -0xac,0x7e,0xc4,0x39,0x28,0xba,0x49,0xd7,0x8b,0xec,0xa6,0xd7, -0x8c,0xbf,0xbb,0x1b,0x08,0x5c,0x35,0xa5,0xeb,0xe5,0x4f,0xa8, -0x86,0x31,0x30,0xa0,0xe9,0xdd,0x24,0xb3,0x8b,0xb4,0xfc,0xd2, -0x4a,0xb3,0x73,0x21,0xc4,0x65,0xf6,0xfe,0x60,0x72,0x71,0x28, -0x35,0x3b,0x94,0x38,0xdb,0x93,0x36,0x10,0x34,0xa1,0x46,0xcd, -0x5a,0x9c,0xee,0x54,0x01,0xe7,0xac,0x26,0x14,0x4c,0xd6,0x3b, -0x70,0xcc,0xad,0x9b,0x48,0xf2,0x5b,0xbe,0xe1,0xad,0x82,0xf2, -0xfd,0x6e,0x19,0x82,0xe5,0xc7,0xb3,0xb2,0xcc,0xd3,0xd9,0x15, -0xfa,0x8f,0x27,0x50,0x4f,0xdc,0x9c,0xc8,0x39,0xbe,0x15,0x88, -0x11,0x78,0xe2,0xbc,0x09,0x85,0xbb,0x46,0xd2,0x2b,0xe3,0x4d, -0xc7,0xdb,0x54,0x9e,0x9b,0x22,0x9d,0x2f,0x9e,0x53,0x92,0xf6, -0x94,0x86,0x28,0xcc,0x24,0x9e,0xae,0xd3,0x32,0x8d,0x57,0x30, -0x53,0xeb,0xe3,0x01,0x6a,0x7e,0x9c,0xad,0x91,0x89,0xe6,0x54, -0x80,0x2b,0xf9,0xdd,0xf7,0x5f,0x3f,0xfb,0xe9,0x6f,0xe8,0xfa, -0x0f,0x53,0x81,0x4e,0x95,0x3c,0xf6,0x5d,0x70,0xfb,0x55,0x0a, -0xcc,0xcc,0x1e,0x0e,0x0d,0x3c,0x45,0xdf,0xdd,0x31,0x3e,0x18, -0xdc,0xdc,0xdc,0x7c,0xde,0xe0,0x38,0x4d,0x94,0x28,0x56,0x2e, -0x04,0x24,0x6c,0x95,0x99,0x1e,0x34,0x8d,0x91,0xb9,0x0f,0xa2, -0xcb,0xcd,0x12,0xd9,0xa4,0x3d,0x9c,0x32,0x17,0xc9,0x7e,0xfa, -0x40,0xe2,0x1f,0x4c,0x98,0x84,0x25,0x8a,0xee,0xa3,0x24,0xb8, -0x85,0x92,0xf6,0x22,0x85,0xd4,0x07,0xe4,0xb7,0xd5,0x3b,0xb5, -0xf8,0x5b,0xb0,0x22,0xc1,0xef,0xe6,0x36,0xef,0xf5,0xcd,0x5c, -0x6d,0xc3,0x57,0xbb,0x18,0x72,0xef,0x94,0x8c,0x01,0xcc,0x7b, -0xbc,0x34,0x0e,0x07,0xa3,0x60,0x66,0x2d,0x2d,0x29,0x95,0xd6, -0xa1,0xf4,0x45,0xfb,0xc1,0x5a,0xcf,0xb2,0x36,0x25,0x92,0xd2, -0xa2,0x43,0x3f,0x21,0x76,0x44,0xa1,0xb1,0x7e,0x9c,0x89,0x13, -0x77,0x9e,0x8e,0x62,0x55,0xdd,0x72,0x37,0xcb,0xee,0xac,0xdb, -0xeb,0xe8,0x58,0x24,0xdc,0x0b,0x39,0xf1,0xa4,0x75,0x2f,0x5a, -0xae,0x7b,0x53,0x7c,0xe7,0xcb,0x61,0x49,0x10,0x22,0x2f,0x6c, -0x5f,0x4a,0xaa,0x22,0x4b,0xc7,0xd6,0x62,0x36,0xcb,0x2a,0xda, -0x72,0x1e,0x39,0x16,0xef,0xb0,0x22,0xe0,0x94,0x36,0x36,0xb3, -0x5f,0x66,0x7f,0xf7,0x50,0x8e,0xcc,0x03,0x31,0x1f,0x7a,0x14, -0xa6,0xbf,0xb5,0xfc,0xab,0x36,0x70,0x28,0xa0,0x76,0x70,0xd8, -0x44,0x34,0xdd,0xe0,0x17,0x26,0x90,0x3a,0x9b,0xb7,0x4c,0x1b, -0x24,0x94,0x6f,0x40,0x8e,0xf8,0x66,0x26,0x07,0x93,0x95,0x1d, -0x4a,0x2f,0x97,0x78,0xf8,0x13,0xf5,0x6d,0x4f,0xe1,0x7a,0xdf, -0xeb,0x80,0x9b,0x54,0x33,0x14,0x99,0x12,0x93,0x95,0x90,0xa3, -0x29,0x51,0x8a,0x0f,0x04,0xd0,0xd8,0xa4,0x0b,0x06,0x80,0x47, -0xc9,0x2a,0xb0,0xdc,0xa8,0xb2,0xb7,0x4c,0x76,0x7c,0x8a,0x66, -0xf0,0xc9,0x0a,0x79,0x01,0x30,0xf2,0x13,0x95,0x0f,0xcf,0xb6, -0xb3,0xf6,0x82,0x1e,0x3e,0x58,0xe3,0xa3,0x6b,0x52,0x45,0x95, -0x4c,0xa7,0xa3,0x33,0x74,0xaf,0xbc,0xd9,0xe0,0x43,0xd0,0x32, -0x1b,0x64,0x88,0x61,0x74,0x86,0xe0,0x2a,0x26,0x77,0xec,0x12, -0x8d,0x1a,0x7d,0xf1,0x29,0x46,0xf2,0x37,0x8a,0x47,0xa3,0xbe, -0xf4,0x68,0x9d,0x27,0x68,0x01,0xe9,0xef,0xd4,0xc6,0x71,0x37, -0x4d,0x30,0xa6,0xf6,0x07,0xaa,0x2e,0x47,0x7d,0x02,0xe0,0x18, -0x35,0xda,0x33,0x34,0xc7,0xc9,0xf7,0x14,0x22,0xf8,0xab,0xdb, -0x4a,0xfa,0x4f,0x6a,0x32,0x48,0x27,0xab,0x88,0x06,0x8f,0xd4, -0xfc,0xc5,0x35,0x57,0xf3,0x16,0xfe,0xe9,0x00,0xf6,0x1e,0x9e, -0x54,0xd1,0x07,0x9d,0x2b,0x2a,0x06,0xe9,0x99,0x8b,0xfe,0x52, -0x7a,0x99,0x6d,0xe8,0x2f,0x1f,0x21,0x4a,0x5f,0x73,0x23,0x23, -0x57,0xb2,0x98,0x75,0xb6,0xc0,0x9b,0x0b,0xd0,0xeb,0xf1,0xae, -0x11,0xae,0x07,0xbd,0xe1,0xc2,0x93,0xbc,0x44,0x40,0xb9,0x2c, -0x8e,0x70,0xf9,0xdd,0xd4,0x60,0x64,0x6e,0x03,0x1c,0xe0,0xac, -0x6f,0xb2,0x42,0x17,0x7c,0x85,0xfd,0x19,0x06,0xb1,0x4b,0xbf, -0xc0,0xe7,0x0d,0x40,0x9f,0x2f,0x4e,0x54,0x3d,0xe4,0x27,0x50, -0xaa,0xb7,0x93,0x5a,0xa6,0x49,0x83,0x7c,0xc7,0x93,0xf4,0x9c, -0x49,0x36,0x0d,0x49,0x53,0xaa,0x12,0xad,0xb2,0x1a,0xfb,0x56, -0x7d,0x4b,0x76,0x47,0x1e,0x84,0x3a,0xce,0x71,0x4a,0xbd,0x4a, -0xf0,0x32,0xc2,0x3e,0x0f,0xca,0xd0,0x60,0x36,0x34,0x4a,0x29, -0xb4,0x37,0xe9,0xc6,0xc5,0xde,0x3c,0xb2,0x87,0xba,0x92,0x47, -0x0f,0xa3,0x98,0x27,0x98,0x68,0xd9,0x6f,0x59,0x16,0xce,0x66, -0x43,0xb1,0x8c,0x34,0x89,0xf4,0x2e,0xbd,0x23,0x63,0xe2,0x26, -0xb2,0x24,0xb3,0x45,0x26,0x40,0x21,0xb2,0x05,0xb5,0x45,0x18, -0x60,0xe8,0xe1,0x01,0x90,0xe1,0x78,0x27,0x51,0x6e,0x89,0xaa, -0x4d,0x49,0x00,0xb1,0x07,0xd0,0xe6,0xac,0xc0,0x03,0x59,0x12, -0xe7,0x74,0xb6,0xd2,0xd5,0x2d,0xda,0x95,0xa5,0xd1,0x51,0x32, -0x4b,0x82,0xdc,0x81,0x97,0xa7,0x2f,0xb3,0x80,0x6b,0x4b,0x27, -0x7a,0x66,0x77,0xd1,0x2a,0x77,0x78,0x4f,0xc1,0xb3,0x99,0xf6, -0x20,0x0e,0x23,0xef,0x10,0x44,0xce,0xff,0xb5,0xb4,0x0c,0x83, -0xc0,0xd9,0xe7,0x17,0x36,0x43,0x68,0x36,0xb1,0x9a,0x0e,0xf3, -0x3a,0x92,0xd6,0x62,0xc7,0x87,0x7d,0x98,0xc6,0x61,0xde,0x66, -0x51,0x41,0x73,0x2e,0xd5,0xb8,0x7a,0x42,0x48,0x27,0x0a,0x6c, -0x3a,0xd2,0xc3,0x37,0x79,0xf1,0x23,0xcb,0x8b,0x9d,0x82,0x84, -0xbd,0x85,0xe3,0x6c,0x60,0xde,0x33,0x19,0x5b,0xd3,0xed,0x58, -0x4d,0xcc,0xf6,0x14,0x1c,0xce,0xb6,0x0e,0xee,0xbb,0x3b,0x77, -0x7e,0xdd,0x95,0xe3,0x67,0x45,0xbd,0x9e,0x37,0x84,0x99,0x19, -0xe1,0x1e,0xc4,0xe9,0x6a,0xbd,0x89,0x31,0xc7,0x9b,0x74,0x71, -0x7c,0x4c,0x2d,0xf6,0x2a,0xbd,0x4c,0xb2,0x2b,0x74,0x0a,0xdd, -0x04,0x08,0xc4,0x6e,0x1c,0x0a,0x86,0xca,0xe1,0xf4,0x39,0x09, -0xc8,0x1f,0x92,0x0c,0xb1,0x80,0x2e,0x5d,0x26,0x47,0xbb,0x68, -0xb6,0xe2,0x49,0xdf,0xa9,0x8c,0xb6,0x23,0x31,0x55,0xb9,0xca, -0x53,0xae,0xc8,0x55,0xbe,0x6a,0x98,0xc2,0xb0,0x74,0x18,0xb4, -0xb2,0x0b,0x53,0x7e,0x5b,0xad,0x30,0x10,0x72,0x1c,0xe8,0x44, -0x83,0x9d,0x57,0xa4,0xc6,0xbe,0xcc,0x98,0xdc,0x00,0x13,0xde, -0x5d,0xa5,0x20,0x86,0x66,0xf6,0x7b,0xc1,0xcb,0x4e,0x7c,0x11, -0x95,0x5e,0x80,0x03,0x24,0x48,0x99,0x79,0x99,0x32,0xc9,0x73, -0x41,0xaf,0xce,0xa9,0x36,0x86,0x88,0xb1,0xaf,0x5e,0xc6,0xc2, -0x64,0x75,0x0d,0x05,0x34,0x56,0x36,0x18,0x29,0xf2,0x39,0xbd, -0x27,0x27,0x6c,0xfb,0x91,0xe8,0x51,0x5f,0xec,0xd0,0xdb,0x45, -0x7a,0x39,0xf3,0x36,0x50,0xf9,0x49,0x3f,0xe2,0x3d,0x01,0xb4, -0xb2,0xd9,0x79,0xd0,0xaa,0x83,0xf4,0x31,0xea,0x6c,0x4d,0xfa, -0x89,0xfd,0x0e,0x0a,0x3d,0xf2,0xe5,0x7b,0x2f,0xbf,0x7f,0xf6, -0xf5,0xd1,0x8b,0x6f,0x9f,0xfd,0xf5,0x79,0x78,0xe4,0x75,0xc8, -0x74,0xe7,0x41,0xd4,0x34,0x5a,0x9a,0xc4,0x4f,0x36,0xb7,0xdc, -0x4e,0xa3,0xea,0x19,0x84,0x5b,0x9d,0xc5,0x5d,0xa9,0xff,0x4d, -0x9e,0xe9,0xb4,0x2d,0xd4,0x93,0x6b,0x98,0x07,0xf0,0x19,0x2e, -0x7a,0xb1,0x56,0x9e,0xca,0x45,0xf8,0xad,0x5f,0x40,0xc4,0x57, -0x61,0x55,0x33,0xf7,0x9f,0xa6,0x63,0x4c,0x0c,0x09,0x42,0x3d, -0xcc,0x96,0xf2,0x4e,0xe5,0x79,0xb3,0x1c,0x50,0xa3,0x2e,0xa1, -0xd9,0x57,0x4a,0xc3,0xe3,0x33,0x13,0x5b,0xad,0x93,0x59,0xc8, -0x99,0x7c,0xb2,0x92,0x2f,0x1c,0x98,0x69,0x11,0x15,0x2c,0x46, -0x21,0x93,0xe8,0xdb,0x67,0xa4,0xb6,0x26,0xa8,0x8f,0x7a,0xb4, -0x22,0x78,0xaf,0x06,0x88,0xda,0x0b,0xbd,0xb3,0xab,0xb5,0x3e, -0x57,0xaf,0x73,0x67,0xfb,0xfd,0x9a,0xdf,0x8e,0xda,0xe7,0x64, -0xdc,0xd5,0x95,0xf0,0x91,0x6a,0xf8,0xc7,0x26,0x72,0xb6,0x5a, -0x69,0x97,0x04,0xb0,0x71,0xa9,0x86,0x13,0x96,0x75,0x6c,0xa8, -0xa2,0x07,0xca,0xc5,0xf3,0x78,0x7e,0x6e,0x46,0x04,0x7e,0xae, -0x40,0xa9,0x51,0x8b,0xc5,0x73,0x84,0x7b,0x99,0x16,0x25,0xc8, -0x49,0x4e,0xa9,0x56,0x07,0xba,0x2e,0x83,0x5b,0xcd,0x98,0x09, -0x8d,0xc2,0x6a,0xf0,0x8d,0x4b,0x81,0xc9,0xdb,0x60,0xbb,0xab, -0x79,0xa8,0x4a,0x68,0x44,0xa4,0x35,0x3a,0xb3,0x7f,0xb3,0x9e, -0x76,0x76,0x6a,0x2e,0xeb,0xe1,0x3a,0x16,0x09,0xbd,0xc9,0xe9, -0xbe,0xab,0xac,0xb4,0x18,0xce,0x44,0xbf,0x8e,0x26,0x43,0x31, -0xce,0x63,0x10,0x4a,0x84,0x1a,0x4f,0x10,0x69,0xc5,0x40,0xa6, -0x5b,0x2f,0x0b,0xe3,0x83,0xa4,0xfe,0x79,0xdb,0x43,0xc7,0x17, -0x9a,0xae,0xf3,0xc9,0xc5,0xb4,0x05,0xa0,0xb2,0x5e,0x42,0x26, -0xc7,0xf1,0x15,0x3d,0x6c,0x1c,0x07,0xfc,0xb8,0x31,0x3f,0xa9, -0x4a,0x4d,0x15,0x4d,0xbc,0xcb,0xec,0xaa,0x48,0xe4,0xcb,0x49, -0xf4,0x7d,0xb5,0xc1,0xe7,0x96,0xa0,0xd7,0x5c,0xe0,0x73,0x4b, -0xb3,0x95,0xfa,0xbc,0x39,0x4f,0x92,0x15,0xfc,0x05,0xc2,0x24, -0x38,0x7c,0xc1,0xca,0x8f,0xd6,0x01,0xf0,0x49,0xf9,0x08,0x03, -0x43,0x4e,0x1f,0x26,0x15,0x8d,0xa6,0x8e,0x5c,0x41,0x20,0xf6, -0x9c,0xe1,0x18,0x4c,0x52,0x42,0xad,0x0f,0x8d,0x2f,0x51,0x52, -0xe5,0xb8,0x4c,0x5c,0x70,0x78,0x6d,0x09,0x30,0x35,0x41,0xbc, -0xdd,0xd0,0x12,0xba,0x51,0x12,0x1f,0x63,0x34,0x55,0xbb,0x59, -0x04,0x89,0xbd,0x12,0x9d,0x3d,0x97,0x3d,0xa5,0xf9,0x40,0xc2, -0x57,0x2b,0x3c,0xa6,0x27,0xf5,0x06,0x0f,0x5a,0x10,0x68,0x4e, -0x51,0xff,0xec,0xce,0x58,0xd7,0xa9,0xad,0xc8,0x5f,0x20,0x12, -0xd5,0x9e,0x38,0xb2,0xc7,0x18,0xb1,0xe1,0x20,0x1f,0x56,0x19, -0xc1,0xe1,0x13,0x21,0xc4,0x2b,0x6a,0x51,0x19,0xed,0xa9,0x56, -0xea,0x93,0x1e,0xe9,0x36,0x68,0x52,0x22,0x85,0x91,0x98,0x64, -0xf1,0x82,0x78,0x65,0xa6,0x9a,0x58,0x0e,0x4e,0x16,0x18,0x72, -0x86,0xa1,0x9c,0x91,0x6f,0x23,0x36,0x2d,0xb0,0xd7,0x49,0xee, -0xa2,0xdc,0x36,0x79,0x0a,0xe8,0xda,0x1a,0x4f,0xa2,0x7d,0x70, -0xf1,0x4d,0x25,0xef,0xeb,0xef,0xbf,0xfd,0x16,0xc1,0x7f,0x9a, -0xe7,0xd9,0x6a,0xb5,0x57,0x3c,0x6c,0xc9,0x63,0x31,0xb1,0x70, -0x20,0x44,0xe4,0xc8,0x9d,0xd0,0x6d,0xde,0x68,0x93,0x66,0x8f, -0x64,0xa9,0xc4,0xed,0x50,0xe0,0xa0,0x31,0xa8,0x54,0x86,0x92, -0x2d,0x7e,0x97,0xd0,0xc9,0x92,0xea,0xf0,0x46,0x6b,0xa4,0x31, -0x15,0x4b,0xa5,0x6e,0x68,0x48,0xb3,0x1f,0x05,0x22,0xc2,0x64, -0x4e,0x26,0x0e,0x67,0xb0,0xc8,0x4c,0x7f,0xd4,0x63,0x76,0x48, -0x54,0xd2,0x70,0x1e,0x9d,0xf3,0xe3,0xed,0xa0,0x7f,0xcb,0x2f, -0x7a,0xc3,0xfb,0x5c,0x3e,0xac,0x82,0xb9,0x43,0xbb,0x8c,0x50, -0xfe,0x6d,0x19,0x4d,0xb3,0xb3,0xb3,0xdf,0x4c,0x17,0x93,0x80, -0xac,0x3b,0x37,0xbb,0x66,0xa9,0x61,0x99,0x3a,0xd6,0x27,0x86, -0xd1,0x0c,0x19,0xd1,0x3b,0x35,0x68,0x19,0xc1,0x28,0x80,0xe3, -0x16,0x33,0x1f,0x45,0x6a,0x0a,0xbd,0xbb,0x3b,0xa7,0xb7,0x4b, -0x60,0xd8,0x4b,0xc5,0x20,0x08,0xec,0x87,0xa1,0xb6,0x07,0x97, -0xa3,0x8d,0x19,0x41,0x4e,0x49,0xcd,0xb5,0xb6,0x33,0xaf,0x40, -0xef,0x6c,0xae,0x0e,0xad,0x33,0x5f,0x32,0x8b,0xe3,0x8a,0xcb, -0xc2,0xa0,0xe3,0xee,0x5b,0x74,0x20,0xae,0x96,0x25,0x07,0x46, -0x3b,0x4b,0x0e,0x7d,0x1c,0xed,0xac,0x3b,0x5a,0x8e,0xa4,0x55, -0xab,0x48,0x5d,0xaa,0xb8,0x56,0x0f,0xa3,0x16,0xf4,0x4a,0xf2, -0xbe,0xab,0xf4,0x95,0x6f,0x2e,0x95,0xb3,0x99,0xaa,0xd9,0xa4, -0xab,0x9d,0xb4,0xda,0x4d,0x3b,0x77,0x13,0x3b,0xd6,0x55,0x1d, -0x6b,0x9b,0x0f,0x95,0x22,0xf7,0xd6,0x7a,0xa4,0xee,0x81,0x0f, -0xfa,0xfd,0x90,0x3f,0x2d,0x78,0xbc,0x85,0xed,0x20,0x6e,0xdc, -0x05,0xb7,0x40,0x71,0x77,0x1f,0xdd,0x35,0x28,0xc3,0x2c,0x68, -0xa7,0x8d,0x40,0x39,0x53,0x96,0x5d,0xd1,0x82,0xea,0x3d,0x5e, -0x84,0xea,0xb1,0x96,0x31,0x3e,0xbe,0x42,0x3b,0x91,0xea,0xba, -0x39,0x08,0x52,0x16,0xd8,0x76,0x54,0x80,0xf4,0x87,0x1c,0x5f, -0x20,0x6a,0x58,0x53,0x21,0x76,0x2b,0x19,0xed,0x4d,0x48,0xf2, -0x29,0xaa,0x98,0x60,0xf2,0x94,0x0b,0x8e,0x32,0x19,0xe2,0xda, -0x52,0xdc,0xd8,0x43,0x9b,0x5b,0x34,0xfb,0xc3,0x4c,0x63,0xf9, -0xd0,0x11,0x1e,0x4f,0xc0,0xaa,0x09,0x11,0x26,0x79,0x59,0x93, -0x4a,0x2b,0xbc,0x75,0xb6,0x86,0x21,0x9b,0x2e,0x24,0x84,0x16, -0x20,0x6f,0xa4,0x39,0xb0,0x30,0x96,0x2b,0xc3,0x36,0xe8,0x28, -0xb2,0xa9,0xdf,0xcc,0xaf,0x40,0x2f,0x5f,0x24,0x67,0xc5,0x44, -0x25,0x4e,0x99,0x08,0x49,0x9e,0x07,0x1a,0xb8,0xff,0x17,0xaf, -0xa3,0x52,0x3b,0x5e,0xa0,0xb6,0x94,0xf6,0x90,0x44,0x90,0x1d, -0x4f,0x12,0xe5,0x5c,0x32,0x9e,0xfb,0xfb,0x3c,0x30,0x8c,0xd4, -0x59,0x87,0x75,0x86,0x53,0xaa,0xfd,0x26,0x7a,0x9a,0x51,0xed, -0x7d,0x7b,0x9d,0xaa,0x03,0x11,0x9d,0xda,0xa6,0xc4,0xc9,0xb7, -0xf0,0xf0,0x40,0xf2,0x0d,0xdd,0xd8,0xcc,0x82,0xa0,0xdd,0xff, -0x09,0xb4,0xe8,0xae,0x99,0xaa,0xb9,0xe0,0x3b,0x42,0x58,0x42, -0xb1,0xdf,0x71,0x87,0x79,0x16,0x2b,0xd0,0xce,0x3a,0x94,0xcc, -0xca,0xf0,0xb8,0xf7,0x69,0x68,0xf9,0xef,0xd8,0x47,0xac,0x2c, -0x4a,0xed,0x8b,0x6c,0xdb,0x9d,0x96,0xdc,0x47,0x32,0xc2,0xfe, -0x49,0x24,0xeb,0xa2,0x5a,0x48,0x96,0x7c,0xff,0x9d,0xdb,0xd9, -0x76,0x61,0x21,0x55,0x82,0x1d,0xaa,0xa4,0xab,0x0a,0xdb,0x11, -0x3e,0xa8,0x59,0x99,0x39,0x48,0xb3,0xc0,0x2b,0x04,0x67,0xeb, -0xfa,0x96,0xd4,0xda,0xd3,0x46,0xfc,0x2d,0xa9,0x72,0x27,0xd7, -0xdc,0x3d,0x6c,0x85,0x52,0x1b,0xb9,0xd6,0x50,0xd7,0xb8,0xa3, -0xed,0xef,0xbf,0x9b,0xdd,0xc6,0xa7,0x6a,0xe0,0x09,0xfd,0x48, -0x64,0x83,0x5e,0x48,0xaa,0xdb,0x93,0xaa,0xa1,0x27,0x76,0x6f, -0x88,0xb5,0x63,0xb1,0x40,0xcd,0xb5,0xb1,0xa0,0xe9,0xca,0xe3, -0xcf,0xea,0xc4,0x5c,0x5a,0x4b,0x3f,0x4e,0xf9,0xb6,0x8b,0x3a, -0x42,0xfe,0x3d,0x2f,0x01,0x41,0x8d,0x3e,0xfc,0xfe,0x0f,0xd4, -0xfe,0x57,0x5e,0xfd,0x69,0xdf,0x77,0xa1,0xfb,0x4e,0xbb,0xcc, -0xfc,0xee,0x27,0xff,0xf0,0x53,0xe0,0xc2,0x3b,0xcf,0x93,0x33, -0x4f,0xd0,0xa6,0x49,0xbb,0xcc,0x57,0xfb,0xc4,0xbd,0x3e,0x24, -0xe9,0x37,0x07,0x05,0xdc,0xee,0x71,0xce,0x0d,0x1d,0x39,0x2f, -0xfb,0xd2,0xb4,0x0d,0xa7,0x8b,0xb1,0xf5,0xdd,0xf1,0x8e,0xbc, -0xd0,0xf3,0xe8,0x4e,0x0b,0x46,0xfe,0x1c,0xa3,0x27,0x8c,0x72, -0xec,0x84,0x9a,0x40,0xfc,0x22,0xf3,0xd8,0x0e,0x18,0x10,0x25, -0xbc,0xca,0x92,0x6e,0x6c,0x3e,0x77,0xef,0xf9,0x69,0x73,0x3b, -0x3c,0xeb,0x3f,0xf2,0x54,0x01,0x77,0x77,0xad,0x90,0x81,0x6b, -0xc4,0xb7,0xb7,0x0f,0xa0,0x9d,0xb0,0xba,0xd7,0xb3,0xd0,0x17, -0x7b,0x60,0x01,0x27,0x0d,0xf1,0x28,0x40,0x57,0x96,0xf4,0xe5, -0xa5,0x49,0xe3,0xf2,0xd2,0x54,0xa0,0x20,0x44,0x1e,0x2c,0xd8, -0x1a,0x32,0xe3,0xf5,0x3e,0x4f,0x2e,0xbd,0xb0,0x5d,0xd4,0xbc, -0xde,0xe3,0x27,0x98,0xda,0x3e,0x76,0x40,0xd3,0x77,0x16,0xfb, -0x5b,0xbf,0xb3,0x68,0x17,0x80,0x05,0x24,0xee,0x49,0x42,0xfa, -0xbb,0xf1,0x7a,0x7e,0x9e,0xc1,0x02,0x8f,0x2a,0x10,0x88,0x78, -0x7c,0xcf,0x10,0x20,0x2f,0x44,0x79,0x9d,0x18,0x5b,0xcd,0x19, -0x05,0x42,0x79,0xe6,0x87,0x8a,0x4b,0x6b,0xee,0x80,0xf8,0xd2, -0x56,0x02,0x52,0x89,0x69,0xfb,0x70,0x40,0xba,0xac,0x02,0x12, -0x4d,0x1a,0xfb,0xba,0x8c,0xe4,0x7d,0xa0,0x3d,0xba,0x8c,0x87, -0xad,0x2e,0x5f,0xb2,0xd7,0x57,0xce,0x48,0xa5,0xb1,0xad,0xfe, -0xd1,0xd6,0xfd,0xf7,0x1e,0x0e,0x1b,0xf8,0xdf,0xcc,0x96,0x2d, -0x63,0x5d,0xe3,0xbe,0x7b,0x5b,0x59,0x1f,0xdc,0x79,0xe9,0x48, -0xa5,0x8c,0x97,0x02,0x4f,0x22,0xc5,0xfa,0xa0,0x15,0x2c,0xed, -0x9f,0xf2,0xa5,0xa7,0x6f,0x41,0x6d,0x76,0xe3,0xf9,0x78,0xf7, -0x8d,0x27,0x5c,0x64,0x8d,0x5b,0x44,0x2d,0xf9,0x36,0x30,0x7e, -0xa5,0x55,0x23,0x1b,0xdb,0x55,0xc5,0x65,0x1c,0x61,0x70,0x3c, -0xb1,0xce,0x49,0xa6,0xa1,0x6d,0xde,0x0f,0x8b,0x64,0x7d,0x8a, -0x62,0xac,0xd8,0x23,0x34,0x9a,0x1f,0xb3,0xa2,0x4c,0xda,0xbc, -0x58,0xe6,0x9b,0x22,0x5a,0xaa,0xf3,0xd2,0xf9,0x79,0xba,0x5a, -0xe0,0x06,0x7e,0x21,0xd6,0x59,0xf9,0xd7,0xc8,0x5b,0xe2,0x4d, -0x16,0x24,0x61,0x13,0xf9,0x9b,0x08,0x81,0x27,0x44,0x5d,0x67, -0x30,0x0d,0xc6,0x66,0x93,0x74,0x43,0xfe,0xba,0xe4,0x29,0xf7, -0x52,0x1f,0x70,0xa7,0xf8,0xc4,0xf8,0xd2,0xeb,0x1c,0x75,0x3a, -0xc4,0x04,0x3c,0xff,0xc0,0xa7,0x56,0x41,0x87,0xde,0x18,0x90, -0x40,0x14,0x11,0x2a,0xd3,0x47,0x4f,0xa1,0x17,0x60,0x59,0x97, -0x78,0x4f,0x43,0x1f,0xb6,0x17,0xf2,0x7c,0x9d,0x0e,0x04,0x44, -0x12,0x5d,0x36,0x8f,0xd5,0x21,0x8b,0x5e,0xbc,0x22,0xd5,0x63, -0x7f,0xc3,0x32,0xec,0x7b,0x9b,0x8c,0x2a,0xdf,0xe5,0x1d,0x2d, -0xda,0xd3,0x28,0x00,0x47,0x2f,0x71,0xf6,0xd0,0x60,0x29,0xa4, -0x16,0x3c,0xbd,0x37,0xb8,0xfc,0x43,0x19,0xd1,0x4b,0x1f,0x58, -0x20,0xd0,0x4b,0x99,0x3a,0x85,0x8e,0x0d,0x12,0x5d,0x5d,0x54, -0xac,0x76,0x8e,0xf4,0xd1,0xc6,0x6b,0x49,0xd6,0x15,0xf8,0x42, -0xe8,0x21,0x62,0x2e,0x77,0xce,0xfb,0x2f,0x99,0x3c,0x44,0x18, -0x50,0x33,0x8c,0x65,0x0c,0xe1,0x09,0x42,0x87,0x39,0xc4,0x36, -0x24,0xa1,0xe7,0x14,0x49,0xf0,0xaa,0xcb,0x04,0x76,0x76,0xb1, -0x39,0x74,0x7f,0xcd,0x16,0x3d,0xb5,0x40,0xc4,0x3b,0x69,0x9b, -0xe8,0xa0,0x94,0xc2,0x70,0xcd,0x85,0x51,0x7b,0xf3,0x0d,0x94, -0x40,0xcc,0xa1,0x1d,0x5b,0x04,0x2b,0x9a,0xab,0x25,0xa2,0x5c, -0x3e,0x9a,0x1b,0x6a,0x54,0x20,0x19,0xc5,0xcf,0x71,0xcf,0xe2, -0xcd,0x1b,0x6c,0xf4,0x37,0x6f,0xa4,0x4c,0xab,0x3b,0x69,0x6e, -0x5a,0xf3,0x6e,0x1a,0x5d,0x69,0x73,0x41,0xdc,0xab,0x6d,0x85, -0x73,0x02,0x80,0x90,0x28,0x95,0x15,0xd0,0xf1,0x90,0x9b,0x5d, -0xed,0x17,0xd0,0x34,0x27,0xad,0xad,0x06,0x28,0x40,0xdf,0xf0, -0x02,0xfc,0x9d,0x8e,0xf0,0x90,0x90,0x37,0x9e,0x7d,0xd1,0xab, -0xf5,0x22,0x9a,0x39,0xd4,0x45,0x1c,0xfa,0x1a,0x0c,0x21,0xf9, -0x2d,0xf7,0xcf,0x7e,0x35,0x95,0x7b,0xae,0xa3,0xed,0xc3,0x67, -0x5d,0x4b,0x93,0x28,0x39,0x06,0x90,0x6e,0xa5,0xb3,0x46,0x5e, -0x93,0x1a,0x37,0x9b,0x73,0x3f,0x30,0x8b,0xbf,0x1d,0xef,0x9b, -0x15,0xf9,0xd9,0xac,0xc9,0xa3,0x66,0x2d,0xed,0x89,0x1d,0x48, -0xed,0xa9,0x12,0x21,0x01,0xd0,0x56,0x98,0x15,0x8c,0xd4,0xd8, -0x03,0xda,0x26,0xf3,0xd9,0x6d,0x1d,0xed,0xa2,0x48,0xdf,0x68, -0x72,0x88,0xc2,0xd7,0x9a,0xf5,0x5d,0xea,0x5b,0xd4,0x42,0x42, -0x8f,0xbc,0x1c,0x7a,0x82,0x75,0x61,0xad,0x73,0x08,0x52,0x82, -0x43,0x0f,0x6a,0xe6,0x6d,0x05,0xef,0x67,0x44,0xb7,0xb8,0x5f, -0x11,0xf2,0xae,0x85,0xb0,0xae,0x16,0x73,0x54,0x57,0xde,0xa5, -0xf6,0xe4,0x2d,0x5d,0x44,0x8d,0x7f,0x55,0x98,0x6e,0xeb,0xaa, -0xc8,0xae,0x9c,0xb5,0x9c,0x5b,0xbf,0x3a,0xb1,0x81,0xe9,0xab, -0x78,0xa3,0x93,0x90,0x1b,0xf3,0x78,0xa3,0x92,0xd0,0xde,0x44, -0xa7,0xa1,0x29,0x20,0xd9,0xba,0xd9,0xa9,0x6c,0xcb,0xe7,0xc0, -0xf0,0x55,0x37,0xbe,0x15,0x4e,0xc5,0xaa,0xf2,0x64,0x4d,0x69, -0x33,0x27,0x52,0x67,0x2c,0x5c,0x79,0x35,0x0c,0xdf,0x5e,0x6d, -0xa0,0xe7,0x25,0xca,0x70,0x9a,0x6e,0x4a,0xb5,0x78,0xad,0x24, -0x5b,0x69,0xf8,0x27,0x8d,0xa3,0xd1,0x5b,0x8f,0xb4,0x86,0xe6, -0x3f,0x6c,0xfe,0x9c,0x6b,0xa3,0xee,0x5c,0x1b,0x74,0x97,0xc6, -0x98,0x5b,0x1e,0x88,0xe2,0x74,0x98,0x2c,0xc2,0x5b,0x2a,0xe6, -0x51,0x9f,0x0b,0xc1,0x3b,0x07,0x44,0x59,0x48,0xbf,0xe2,0x1e, -0x9b,0x75,0x9f,0x7c,0xa7,0xf1,0x26,0x04,0xdd,0x19,0xd1,0x66, -0xeb,0x2a,0x85,0xa8,0xa6,0xdb,0x24,0xd2,0x72,0xdd,0x64,0x99, -0x5b,0x96,0xeb,0x2a,0x76,0xd3,0xb4,0x5c,0x57,0x09,0x8e,0x7d, -0x37,0x27,0xd0,0x34,0x22,0x2c,0xeb,0x6e,0x8e,0xc6,0x12,0x3d, -0x61,0xd9,0x76,0x73,0x34,0x7e,0x7a,0xc2,0xb6,0xec,0xe6,0x78, -0xfa,0xf6,0x84,0x34,0xec,0xa6,0xfa,0xe2,0x8f,0x36,0xe9,0xc6, -0x7e,0xd0,0x66,0xcf,0x7d,0xd8,0x60,0x0f,0x67,0x3e,0xeb,0x2c, -0xf7,0xa0,0xfd,0x1e,0xee,0x9f,0x45,0xb7,0xdb,0x3f,0xcc,0x7a, -0xaf,0x61,0xbe,0x84,0x16,0x7c,0xed,0x87,0xb5,0x7a,0xf7,0xd6, -0x6b,0xda,0xd2,0x51,0x7d,0xdc,0x4c,0x6a,0x32,0x87,0x94,0xbd, -0xe6,0x79,0xa6,0xf2,0x1a,0x8b,0xad,0xf3,0xfc,0x9e,0x76,0x7a, -0x4e,0x25,0x7f,0xad,0xad,0x1e,0xd1,0xf7,0xfb,0x9a,0xdb,0x59, -0x46,0x70,0x9c,0x65,0x9f,0x0e,0xaf,0x0b,0x41,0x55,0x5e,0x21, -0xd2,0xfb,0x1b,0x1f,0x74,0xba,0xf0,0x5b,0x0c,0xda,0x78,0x1c, -0xfa,0x1a,0x45,0xb2,0x21,0xec,0x4b,0x7c,0x4f,0x5a,0x69,0x74, -0xc4,0x2b,0x58,0xb7,0xad,0x22,0x23,0xc3,0x30,0x73,0x99,0xe1, -0x2d,0x46,0xf7,0x0f,0x24,0xd9,0x34,0x91,0x48,0x09,0xc3,0x08, -0xcb,0x62,0x02,0xbd,0xd8,0x2e,0xcc,0xe9,0x36,0xa6,0x1a,0xc1, -0xd3,0x67,0x88,0x14,0x9d,0x5c,0x6e,0x60,0x59,0x85,0x7e,0x22, -0x34,0xd2,0x74,0x5d,0xc0,0x5a,0x4b,0x22,0x15,0x5e,0x78,0x96, -0xe6,0x45,0xd9,0x25,0xbd,0xc9,0x43,0x73,0x2e,0xa0,0x37,0xa2, -0xbc,0x96,0x19,0xea,0xca,0xb9,0xef,0xac,0x0c,0x51,0x81,0x08, -0xec,0xea,0xe9,0x02,0xe2,0xd8,0x4d,0x04,0xe4,0xdd,0xb1,0x45, -0xdd,0xc9,0x6b,0x54,0xf6,0x96,0xcc,0x0d,0x15,0x95,0x23,0x1d, -0x25,0x3a,0x5d,0xa8,0x4b,0x1f,0x67,0x11,0xb0,0x6b,0x92,0x2e, -0xa6,0x08,0xd6,0x30,0x11,0x0d,0x46,0x10,0xc7,0x45,0xdd,0x56, -0x83,0x10,0x75,0x93,0x0a,0xaf,0x68,0xf3,0xd7,0x50,0xd4,0x1c, -0x57,0xa3,0xdb,0x26,0xfe,0x1a,0x6e,0xe5,0xdd,0x6d,0xcc,0x68, -0xd5,0x1d,0xe3,0x54,0x8d,0x11,0x90,0x2e,0x76,0x33,0xe8,0x4e, -0x5d,0x19,0x56,0xa6,0x35,0xaa,0x42,0x91,0x5c,0x77,0x35,0xe5, -0xb5,0xba,0xb4,0xe6,0xc4,0xad,0xe2,0x13,0x66,0xeb,0xd2,0xfd, -0xef,0x3d,0xf0,0x94,0x46,0x46,0x16,0x2d,0x67,0xbd,0x87,0xed, -0x25,0xcd,0x71,0x84,0xe5,0xa5,0x68,0x2c,0x87,0x17,0xcc,0x48, -0xba,0xac,0xdf,0x30,0x5a,0x97,0x0b,0x7e,0x5e,0xb8,0x6a,0x43, -0x4b,0xc6,0xdc,0x55,0x9b,0x9c,0xa6,0x43,0xf8,0xda,0x94,0xcc, -0x2a,0x6f,0x97,0x46,0x2a,0xca,0x10,0x6a,0x99,0x74,0x92,0x05, -0x3b,0xaa,0xf7,0x82,0x8e,0xe5,0xf0,0x20,0x89,0x0f,0xcb,0xec, -0x93,0x33,0xeb,0xa6,0x7d,0xab,0x4f,0x0a,0x6d,0xe0,0xcf,0xe7, -0x42,0x80,0x8d,0xcd,0xfd,0xa9,0x43,0xd2,0xe4,0x61,0xac,0xff, -0xf5,0x09,0x20,0x65,0xe2,0x53,0x3f,0x9e,0xf6,0xf9,0x7c,0x79, -0x6c,0x40,0x09,0x63,0x48,0x14,0x32,0x1c,0xd7,0xdb,0x36,0x84, -0x21,0xca,0xc9,0x90,0x50,0x82,0xf0,0xb6,0x47,0x23,0xed,0x21, -0x96,0x92,0x72,0xf9,0xff,0x50,0x43,0x49,0xcd,0x9b,0x0f,0xb2, -0x92,0xe4,0x42,0x76,0x0c,0xbf,0x10,0xd9,0x21,0xab,0x2f,0x48, -0x97,0xc9,0xbe,0x73,0xf6,0xbc,0x63,0x75,0xb4,0x6b,0xc7,0xc1, -0xf6,0x1a,0x96,0xe9,0x06,0xf3,0x5d,0x9a,0x71,0xa8,0xb5,0xd5, -0x08,0x23,0x61,0x1c,0x73,0x6f,0x6a,0x50,0x53,0x10,0xff,0xd1, -0x7b,0x4a,0xe3,0x1c,0x3f,0x2e,0xcd,0x39,0x3e,0x31,0xd9,0xde, -0x0c,0xf9,0x7f,0xd6,0x4c,0xed,0xa1,0x1a,0x8f,0xb6,0x56,0x6b, -0x2a,0x3a,0x07,0x8c,0xd6,0x76,0xf5,0x99,0x36,0x25,0xe8,0x43, -0xcc,0xd5,0xfe,0xc7,0x32,0xec,0x57,0x5b,0x86,0xfd,0xbe,0x76, -0x29,0x62,0x91,0x5d,0xee,0x5c,0xb3,0x80,0x56,0xd5,0xcd,0x59, -0x45,0xfb,0x6d,0x52,0x50,0xc1,0x09,0xcd,0x10,0x60,0x0d,0x2c, -0x8a,0x90,0xed,0x03,0xed,0x5e,0xa0,0x8a,0x40,0xc8,0xae,0xd9, -0x21,0x9d,0x72,0x57,0x38,0x0c,0xfd,0x91,0xe6,0x2f,0xf7,0x31, -0xe1,0x8f,0x31,0x8f,0xf1,0xb1,0xc6,0xac,0x27,0xb4,0x54,0x9a, -0x72,0x51,0xbd,0x5d,0x33,0x9a,0xa6,0x1d,0x8d,0x6b,0x5c,0x42, -0x5b,0xb7,0x6c,0x5d,0x42,0xb6,0xcf,0xd0,0x5f,0x60,0x06,0x7b, -0x66,0xdb,0x64,0x39,0x86,0xcf,0xa8,0x8a,0x15,0xaf,0xf2,0x24, -0xe1,0xe8,0xf1,0xf5,0xf2,0xcd,0x65,0x7c,0x91,0xbc,0x29,0x31, -0xca,0x45,0xa2,0xa1,0x70,0xc8,0xa5,0x78,0x52,0x1e,0x61,0xc2, -0x00,0xd5,0x80,0xc2,0x97,0xf1,0xa6,0x91,0x27,0x68,0x52,0xe2, -0x90,0x71,0x75,0xa9,0xe6,0x36,0x75,0xe7,0x1b,0x00,0x43,0x2b, -0x21,0xa4,0xdf,0x6d,0x13,0x09,0x52,0x62,0x10,0xd1,0x64,0xaa, -0x49,0xe1,0xa6,0x07,0x6d,0x61,0x11,0xb9,0x99,0x08,0x0e,0xf7, -0x0c,0xf1,0xef,0x44,0xc1,0x93,0xe5,0x13,0xba,0x8e,0x36,0xbb, -0x86,0x01,0x59,0x9d,0x2c,0x7a,0x64,0x5b,0x51,0xf0,0x8b,0x1a, -0xfe,0x9a,0x3c,0x5f,0x69,0xb5,0x42,0xa5,0x42,0x1b,0x45,0x2d, -0x5c,0x9a,0xd3,0x13,0x7e,0x8a,0x24,0xd5,0x60,0x0b,0x53,0x8f, -0x4b,0xd4,0x1c,0xad,0x89,0xe3,0xac,0xd5,0x29,0x99,0xd3,0x52, -0x94,0xcf,0x36,0xcf,0x0c,0xa8,0x11,0x2c,0xe6,0xb8,0x8b,0x3e, -0x0c,0xc9,0x8d,0x52,0x7d,0x01,0x1f,0x01,0xa8,0xc5,0xb9,0x52, -0xb4,0x2d,0x35,0xa6,0x37,0x1f,0x26,0x7d,0x7a,0x0f,0x62,0xca, -0xaf,0x42,0xe0,0x0f,0xb5,0xe6,0xae,0xe6,0x89,0x5b,0x38,0x50, -0x53,0x8f,0xc5,0x3c,0x3b,0x3b,0x42,0xcf,0x4d,0x50,0xc8,0x64, -0x31,0x0d,0x17,0x5b,0xab,0xbd,0x5d,0x23,0x76,0x34,0x6f,0xc7, -0xd1,0x01,0x37,0x0f,0xf5,0x0d,0x01,0xa0,0xe5,0x0d,0x82,0x41, -0x6f,0x2c,0x7e,0xc0,0x6e,0x0d,0x3a,0x2d,0x5d,0x21,0x18,0xc3, -0x4f,0xd8,0x6a,0x23,0x8f,0xcd,0xd5,0xbc,0x0c,0x20,0x17,0x5c, -0x49,0xa9,0xf1,0xa5,0xc5,0x37,0xe9,0x2a,0xf1,0xf1,0x6c,0x76, -0xac,0xe2,0xce,0x20,0x26,0xd4,0x05,0x96,0xe5,0x66,0x04,0x39, -0x10,0xc4,0xd0,0xb5,0x4d,0x56,0x05,0xaa,0xab,0x0c,0x53,0x9d, -0xe7,0x8d,0x64,0xb2,0xc9,0x55,0xc9,0x1b,0x49,0xf0,0x8f,0xcf, -0xa3,0xd3,0x7f,0x9f,0x3c,0xeb,0xfe,0x5b,0xdc,0x7d,0x3f,0xed, -0xbc,0x0e,0x5f,0x9f,0xbe,0x3e,0x3d,0x15,0x76,0xa9,0xaa,0x6a, -0x91,0x47,0x34,0x9c,0x9e,0x7a,0x23,0xc5,0xa4,0x3c,0x71,0x2d, -0xfd,0x2d,0x6b,0x8a,0x78,0x41,0xed,0x2f,0x70,0xf1,0x1b,0x2b, -0x95,0x09,0x07,0x18,0x8e,0x38,0x3e,0xe6,0xbf,0x34,0xe8,0xdc, -0xdd,0x79,0x6f,0x8b,0x6c,0xed,0x8d,0x2c,0x59,0x91,0x70,0xd2, -0x36,0xd2,0xc1,0x25,0x1a,0x28,0xc8,0xff,0xc7,0xf1,0x31,0xfd, -0xf9,0x99,0xe4,0xda,0x06,0xe7,0x64,0x29,0x6d,0x96,0xe5,0x49, -0x03,0x18,0x8b,0x91,0x9a,0x3b,0xd4,0xd5,0x2c,0x6d,0x39,0x5e, -0x6c,0xd0,0x7d,0x60,0xe1,0x0a,0x15,0xc4,0x58,0x0e,0x26,0x00, -0x9f,0xb4,0xe4,0x2c,0x26,0x90,0x32,0x9d,0xa2,0xc5,0x23,0xf6, -0xf4,0xc6,0x18,0x05,0xcc,0x7b,0x43,0xd0,0x2c,0xc4,0x08,0x32, -0x9e,0x60,0xca,0x94,0x46,0x21,0x81,0xe5,0x88,0x0d,0x65,0xb5, -0xce,0xdb,0xdc,0x4c,0x3b,0x80,0xbc,0xb4,0x24,0x57,0x56,0xe8, -0x52,0x7c,0x0e,0x3f,0x8e,0x4b,0x71,0x16,0xe9,0x9d,0xd7,0x3b, -0x69,0xa4,0xa0,0x44,0xec,0xf1,0xb8,0x97,0x8d,0x39,0xb9,0xaa, -0x12,0x1a,0x23,0xd4,0x76,0x36,0xd6,0x0b,0x77,0xad,0xa7,0xd1, -0x06,0x7e,0x7d,0x13,0x0e,0x46,0x48,0xc5,0xf1,0xb1,0x1a,0x57, -0x64,0x63,0x7c,0x95,0xad,0x70,0x42,0xa2,0x0e,0xa8,0xc8,0x7d, -0xd4,0x97,0x7b,0xdc,0xb2,0x7d,0x71,0xb4,0x97,0xcc,0x8b,0x6e, -0xd7,0x57,0x97,0x33,0x98,0x9e,0x81,0x63,0xfc,0x25,0xbc,0x59, -0x96,0xad,0x92,0x78,0xed,0x61,0x9c,0xfc,0x16,0xb4,0x77,0xfa, -0x35,0xfc,0x70,0xd3,0xea,0x39,0x5a,0x62,0xec,0xa1,0x2c,0xb9, -0xf3,0x85,0x23,0x85,0x8b,0xe8,0xff,0xfc,0xf4,0xfd,0x77,0x9c, -0x57,0x36,0x89,0x83,0xc0,0xc8,0x95,0x3c,0xdf,0xc5,0x6d,0x08, -0x6c,0xc5,0x78,0x3e,0x4f,0x8a,0x02,0x58,0xd6,0x48,0x0f,0x60, -0x78,0x01,0xf9,0xda,0x0a,0x45,0xc0,0xbc,0xb8,0x6e,0xce,0x57, -0x5c,0x30,0xcc,0xd1,0x90,0xd6,0x56,0xb4,0x95,0xbb,0x3c,0x90, -0xbb,0xbc,0x3f,0x77,0xb6,0xc9,0x0e,0x30,0x40,0x3f,0x86,0xa3, -0xe0,0x02,0x33,0x93,0xf2,0xed,0x21,0xef,0x15,0xa4,0x20,0x87, -0x8e,0x56,0xe9,0x2c,0x8f,0xf3,0x1a,0x0f,0x0d,0x8f,0xf8,0x8a, -0x0e,0xac,0x2c,0x40,0x87,0xa3,0x71,0xa4,0xdc,0xe1,0x22,0xaa, -0xe5,0xe8,0xb9,0xa0,0x9d,0x99,0x67,0x09,0xcc,0x8b,0x79,0x32, -0x46,0x20,0x1f,0x7f,0xca,0x1e,0xfc,0xa2,0xf3,0xc9,0x89,0x0b, -0x31,0x0d,0xc6,0x8a,0x36,0x15,0xe5,0x97,0xa4,0xf2,0xab,0x60, -0x11,0x9a,0xbb,0x4e,0xde,0x8b,0x35,0x88,0x5c,0xba,0x38,0xd2, -0x54,0x33,0x56,0xbc,0x61,0xe4,0xe2,0x45,0xca,0x83,0xb0,0x41, -0xd5,0x65,0x52,0x9c,0x1f,0x20,0x09,0x93,0x81,0x9e,0x89,0x26, -0x08,0x23,0x80,0x9a,0x3d,0x80,0xd3,0x0f,0x23,0x0c,0x33,0x31, -0x55,0x26,0xd7,0xb7,0x69,0x51,0xc0,0xfc,0x64,0x72,0x49,0xea, -0x8f,0x32,0x58,0x3d,0x03,0x3c,0x0e,0x34,0xa0,0x5e,0x95,0xa0, -0xad,0x7a,0xc4,0x71,0xab,0xe5,0xa1,0x0b,0xde,0x2b,0xfa,0x93, -0x66,0xab,0xe9,0xf6,0x5a,0xc0,0x9a,0xf5,0x7a,0x89,0x48,0xde, -0xbc,0xc1,0xab,0x58,0x0b,0xad,0x1f,0xc9,0xa1,0x58,0x87,0xd1, -0x68,0x56,0x7e,0x7a,0x28,0xf6,0x38,0xd8,0xf7,0x54,0x77,0x96, -0x7f,0x39,0x92,0xca,0x89,0xac,0xb1,0x96,0xa2,0x2d,0x5d,0x8f, -0x77,0x68,0xf6,0x3c,0xc5,0xa5,0x46,0x3a,0x57,0x87,0x68,0xce, -0xeb,0x4a,0xab,0x5d,0x81,0xfa,0x09,0xe2,0xa7,0xdd,0x0b,0x9f, -0xa5,0xc9,0x6a,0xd1,0xe2,0x5f,0x18,0xe1,0x58,0x69,0x67,0x08, -0x18,0xf9,0xb0,0xff,0x2a,0x36,0x60,0xf2,0x96,0xd7,0xc7,0x9c, -0x6e,0x89,0x34,0x91,0xd4,0x93,0xd1,0x46,0xf9,0xc1,0x30,0x2a, -0xbf,0x46,0x03,0x62,0x08,0xbd,0x01,0x65,0x20,0x94,0x8e,0xa9, -0x86,0x12,0x3c,0x29,0x01,0x88,0xad,0xfa,0xa3,0xd8,0x32,0xcf, -0x36,0x75,0x73,0xb5,0x76,0x96,0x83,0x96,0x6f,0x0f,0x43,0x1e, -0x82,0x82,0x14,0x68,0x3a,0x45,0x2c,0x9f,0x74,0xa6,0xec,0xae, -0xb2,0xe6,0xf0,0xcc,0x3c,0x3e,0x81,0x17,0xe7,0x10,0x31,0xc4, -0x0a,0xb9,0xa3,0x11,0x17,0x77,0x77,0x8c,0xd2,0x99,0x42,0xf6, -0xf1,0x73,0x31,0xe1,0x7c,0xc0,0xc7,0x69,0x04,0xe8,0x26,0x9a, -0xad,0x46,0xb4,0xcc,0x73,0x77,0x48,0x59,0x8f,0x6a,0xe2,0xf2, -0xcf,0xf0,0xae,0x59,0x4b,0x4e,0xa6,0x2a,0x6d,0x05,0x67,0xff, -0x0d,0x0d,0x60,0xe3,0x89,0x0f,0xe2,0x88,0xef,0xcb,0x6f,0xb5, -0x16,0x3e,0xeb,0xd5,0x2a,0xc4,0x94,0x62,0x0d,0xe2,0x19,0x4d, -0xd1,0x30,0x84,0x5d,0xad,0x48,0x58,0x31,0x65,0x18,0x65,0xe5, -0x39,0x2e,0xa8,0xa8,0xbb,0xba,0x73,0x75,0xd3,0x02,0x40,0x1e, -0x39,0x8f,0xd4,0x94,0x0c,0x8b,0x81,0x34,0x5e,0x1e,0x1f,0xa7, -0xb0,0x6e,0x7b,0x8b,0xce,0xd2,0x71,0x3a,0xcd,0x26,0xd9,0x55, -0xb9,0xb9,0x2a,0xe9,0x84,0x61,0xaa,0xa7,0x77,0x1d,0x4d,0x87, -0x1a,0x1c,0x3f,0xc4,0x33,0x68,0x26,0x87,0xfb,0x44,0x66,0x7a, -0x02,0x47,0x53,0x4f,0x20,0x02,0x59,0xb4,0xb8,0xc4,0xe8,0xd1, -0x40,0x30,0x36,0x75,0x04,0x1c,0xeb,0xe3,0xde,0x99,0xa7,0x67, -0x64,0xaa,0xfe,0xc4,0xbb,0x49,0xcb,0x73,0xcf,0xba,0x75,0x65, -0x5a,0x9b,0x11,0x83,0x72,0x80,0x80,0x5b,0xfe,0xd3,0xc3,0x02, -0xb2,0x75,0x6c,0xad,0xf9,0xcc,0x96,0x82,0x2a,0xbe,0x72,0xb3, -0x48,0x5a,0xcc,0x06,0x4a,0xbc,0xb1,0xd7,0x88,0xa4,0xc9,0x31, -0x4c,0xb0,0x7b,0x47,0x0c,0x35,0x7a,0xe5,0x41,0x06,0x32,0x4e, -0x2e,0xf0,0x29,0x57,0x86,0x86,0x6f,0x19,0x85,0x3b,0xd5,0x76, -0x91,0xa6,0xf5,0xcf,0xe2,0xb9,0xeb,0xe7,0xc5,0x58,0x38,0x62, -0x8a,0xdd,0xfa,0xb3,0xb7,0xe2,0x02,0x07,0x18,0x71,0x01,0xcb, -0x10,0x7e,0xfc,0x45,0x5c,0x88,0x8b,0x6b,0x25,0x11,0xb7,0x40, -0x68,0xe8,0x79,0x02,0xe9,0x0d,0x27,0xd2,0xec,0x1f,0xbf,0xb6, -0x02,0xa8,0xa0,0x85,0x71,0x0c,0xba,0xa9,0x6c,0x30,0x4e,0x1d, -0x49,0x67,0xa9,0x98,0xc7,0xf5,0xa3,0x88,0xce,0x69,0xe5,0x35, -0x35,0x9c,0xdd,0x9a,0xc8,0x79,0x89,0xda,0x57,0x85,0x40,0x5f, -0x2b,0xc7,0xbc,0x10,0xa3,0xd5,0x3a,0x2f,0x8a,0xf1,0x74,0x01, -0x12,0x8e,0x8f,0xf1,0x17,0xd1,0xc8,0x52,0x03,0x49,0xf2,0x43, -0xb5,0xcc,0x0b,0x00,0xb9,0x50,0x43,0x34,0xd6,0x3e,0xf2,0xbc, -0xd1,0xc5,0x17,0x16,0xd1,0x00,0x7b,0x11,0x5c,0x5c,0x53,0x3d, -0x80,0xdf,0xbe,0x14,0xdb,0x80,0xb3,0x71,0x35,0x2e,0xae,0x03, -0xca,0xdc,0x89,0xfc,0x8b,0xa7,0xfd,0xb1,0x77,0xc7,0xc6,0x9c, -0xd2,0xe8,0x0e,0x52,0x47,0x58,0x51,0x6a,0x30,0x80,0x9a,0xca, -0xa3,0x05,0xb4,0x7d,0x9c,0xbd,0x3d,0x3e,0x76,0xd9,0xa1,0xa1, -0x98,0x31,0xd4,0x22,0xc4,0x1a,0x6e,0x21,0xe6,0x0e,0x65,0x61, -0xfa,0xac,0xc6,0xa0,0x89,0x57,0x32,0x82,0x11,0x2a,0x62,0xb7, -0xe8,0xa5,0x15,0x38,0x15,0xd8,0x6c,0xb1,0x90,0x58,0x6c,0x21, -0x86,0x62,0x12,0x6e,0x94,0x48,0xa6,0xb6,0x74,0x3d,0x24,0x09, -0x79,0x86,0xe0,0xce,0x45,0x73,0x12,0x2e,0x92,0xec,0xc8,0x96, -0x64,0x99,0x4a,0xf1,0x7a,0xfc,0xba,0x68,0x99,0x78,0x28,0xff, -0x96,0xff,0xf4,0x08,0xbb,0x59,0x7b,0x9b,0x25,0x37,0xc6,0xeb, -0x05,0x3a,0x52,0x1c,0xf4,0x66,0xb0,0xf2,0x76,0xb3,0x5b,0x7d, -0x21,0x5d,0x95,0x7b,0xb6,0xda,0x39,0xc9,0x9d,0xd0,0xd1,0x0b, -0xe6,0xd8,0xca,0xc8,0xee,0x34,0x59,0xf0,0x48,0xc9,0xc4,0x2d, -0x17,0xa7,0xd2,0x04,0xd6,0x2b,0x9d,0xad,0x18,0x67,0xf7,0x81, -0x92,0x68,0xc9,0xf5,0x8d,0x93,0x3e,0xc6,0x5f,0x54,0xb4,0x48, -0x1b,0xe9,0x25,0xd5,0x26,0x97,0xdb,0x10,0x8c,0x73,0xab,0xff, -0xea,0xba,0xac,0xe2,0xb2,0x4c,0xd6,0xed,0x95,0xe1,0x34,0xb7, -0x36,0x18,0xc9,0x3a,0x17,0xee,0x73,0x3a,0xb0,0x1c,0xbd,0xa2, -0xf9,0x05,0xbd,0x9e,0x21,0x7e,0xd9,0xe6,0xf6,0x6d,0x3f,0xd9, -0x85,0xa4,0x64,0x35,0xce,0x82,0x34,0x1e,0xb3,0x79,0xc3,0x18, -0x47,0xb4,0xee,0x37,0xbd,0xc4,0x59,0x1a,0x90,0x3e,0x63,0x11, -0x08,0x54,0x5b,0x55,0xcc,0x56,0xed,0xeb,0x77,0x4c,0xb0,0x06, -0x2e,0xf6,0xc6,0x04,0x33,0xd7,0x5a,0xf6,0x02,0x94,0xc8,0xcb, -0x86,0x4a,0x60,0x8e,0xb2,0xec,0x71,0xc0,0x9e,0xb9,0xe8,0x10, -0xc4,0x5a,0x6d,0x6a,0x43,0x29,0x79,0x3d,0xe5,0x56,0x77,0x3a, -0x53,0x7d,0x9a,0x22,0xe9,0xa8,0x84,0x48,0xde,0x8e,0xf4,0x34, -0x06,0x22,0x3e,0x95,0x24,0xe0,0x2c,0xa6,0xe3,0x29,0xfb,0x34, -0x52,0x92,0x8e,0x89,0x3e,0x1f,0x64,0xd9,0x3d,0x36,0xd3,0xce, -0x67,0x38,0xd6,0xd5,0xf8,0x84,0xce,0x8d,0x01,0x35,0xd3,0xd1, -0x00,0x0a,0x3f,0x1e,0x63,0x0a,0x3d,0xfa,0xe3,0x59,0x0b,0xd0, -0xd5,0x62,0x57,0x3b,0xd9,0xaf,0x95,0x04,0x56,0x31,0x92,0x97, -0xbb,0xdd,0x14,0x70,0x6e,0xe9,0xf7,0x4f,0x9a,0xe4,0x4c,0x81, -0xb6,0x9c,0xe4,0xf3,0x64,0x8f,0xa0,0x40,0x0a,0x4b,0xf7,0x62, -0xa6,0xf6,0xeb,0x56,0x71,0x0d,0x98,0xd9,0x76,0xcc,0xde,0xdd, -0xb3,0x37,0xf5,0x78,0x6f,0x5a,0xea,0x46,0x68,0x38,0x74,0x51, -0x1c,0x1f,0x2f,0x66,0x13,0xfa,0x02,0xda,0x24,0x0e,0x0a,0xfa, -0x3a,0x1e,0x20,0x25,0x6e,0xb4,0x38,0xf3,0xd9,0x06,0x4f,0xf6, -0x1e,0x3c,0x6a,0x23,0xa7,0x45,0xeb,0x42,0xa9,0xfa,0x0c,0x5b, -0xe2,0xdd,0x67,0xd3,0x21,0x14,0x82,0x6c,0xe3,0xcb,0x5d,0x21, -0x5a,0x3e,0x50,0x34,0xae,0xb1,0x25,0x00,0xd7,0x4c,0x92,0xc6, -0xba,0x0f,0x7e,0x7e,0x0d,0xfd,0x29,0x5e,0x03,0x37,0x86,0x64, -0x44,0x75,0x01,0x73,0x0f,0x3f,0xa7,0x3c,0x10,0xf3,0x73,0x54, -0x84,0xa3,0xee,0x63,0xf2,0xb5,0x26,0x49,0x89,0x9e,0xf4,0x79, -0x6f,0x32,0x9a,0x28,0x3b,0x13,0x65,0x48,0x32,0x15,0xb4,0x98, -0xc3,0x0b,0xf2,0x67,0x79,0x4a,0xfc,0x44,0x33,0x91,0xf3,0x04, -0xb4,0x7a,0xe1,0x2d,0xf3,0xf8,0x9a,0xec,0xd7,0xf8,0x11,0x12, -0xcf,0x5e,0x58,0xcf,0x13,0x66,0x8c,0xbb,0xd4,0xfb,0x29,0xa1, -0xa3,0x02,0x2d,0x13,0x0c,0x62,0xa5,0x08,0xca,0xba,0xe5,0x3f, -0xcd,0xfd,0xd6,0xf7,0xee,0x3e,0x6b,0xf1,0xde,0x85,0x76,0xea, -0xbe,0x47,0x77,0x77,0x60,0x3c,0xde,0xc2,0x31,0x3b,0xad,0x04, -0x3b,0xa6,0xdf,0xb0,0x45,0xc1,0xb7,0x1a,0x5c,0xe1,0xf0,0xed, -0x40,0xb0,0x4b,0x8f,0xe6,0xfd,0x01,0x7a,0x34,0xcc,0x6f,0xa0, -0x47,0xe1,0xf0,0xed,0x40,0x83,0x1e,0xd9,0xf8,0x7b,0x28,0x91, -0xa9,0xbf,0x92,0x06,0xce,0xed,0xf3,0x9f,0x46,0xb9,0x96,0xa4, -0xd9,0x56,0x08,0xb9,0xed,0xff,0x47,0x01,0xe0,0xa7,0xca,0xcd, -0xa2,0xb7,0x3b,0x4e,0xf0,0xb1,0x32,0x01,0xc9,0x63,0xae,0x16, -0x2d,0x9c,0xe9,0xe2,0x74,0x3c,0xfb,0x62,0x9c,0x5b,0x4d,0x9a, -0x35,0x6e,0x5c,0x5e,0xad,0xe2,0x16,0x6b,0xa4,0xe6,0xda,0x16, -0xeb,0xcb,0x9d,0x0c,0x67,0xe6,0xc8,0x32,0xb1,0x11,0x0a,0xc9, -0xde,0xf5,0x2c,0xcc,0x4c,0xab,0x9d,0xd5,0x1f,0x99,0xa5,0xd3, -0xd7,0x34,0x42,0x94,0xd2,0x72,0x01,0x4b,0xd0,0x19,0x70,0xd3, -0xce,0xd9,0xfc,0x83,0x62,0x78,0xfc,0x8e,0x1a,0xfc,0xb0,0x31, -0xd3,0x69,0xbb,0xa2,0x6a,0xab,0x3e,0x48,0xa3,0xd8,0xa3,0x93, -0xe8,0x2a,0x3d,0x5c,0x27,0x69,0x60,0xdf,0xda,0x7b,0x26,0xcb, -0x24,0x6b,0x1d,0x8b,0x21,0xde,0x57,0x0f,0x95,0x96,0xb4,0xaf, -0x9a,0x67,0x6f,0x79,0xff,0x35,0xf2,0x2e,0x93,0x7c,0x1e,0x97, -0xca,0x1c,0x07,0xc7,0x39,0x00,0x9f,0x18,0x88,0x29,0x0e,0x75, -0x71,0xe9,0x70,0x7e,0x05,0x19,0xed,0xb0,0x9a,0x07,0xab,0x10, -0x2f,0x09,0xc1,0x5c,0x88,0xd6,0xae,0xb8,0x30,0xb9,0x67,0xab, -0x61,0xb5,0x8a,0x26,0x2b,0x0a,0x0a,0xd2,0x61,0x82,0xa9,0xa8, -0x78,0x7b,0xc3,0x5f,0xad,0xcc,0x26,0xa6,0x9a,0xc0,0xab,0x69, -0x54,0xd5,0x74,0x4a,0xa7,0x62,0x6a,0x8a,0x19,0x4c,0xed,0xf6, -0x82,0xb2,0x7a,0x54,0x95,0x76,0xe1,0x22,0xc2,0x7a,0x16,0x07, -0xcc,0xc9,0x3c,0xa9,0x60,0x26,0x05,0x26,0xc4,0x0d,0xb9,0x08, -0x51,0x90,0x9b,0xfd,0x2c,0xd2,0xf3,0x0d,0xca,0xc0,0x51,0xba, -0x46,0x4e,0x07,0x08,0x2d,0xfb,0x02,0x04,0xf9,0x4b,0x6b,0x17, -0x40,0xc5,0x7d,0x9d,0x0d,0xe7,0xde,0xbd,0x5d,0x4d,0x63,0x8c, -0x2a,0x61,0x15,0x54,0xa1,0x59,0xce,0x06,0xbb,0x1c,0x56,0x73, -0x65,0xd7,0xaf,0x31,0x06,0x72,0x1b,0x36,0x47,0x16,0x22,0x8b, -0xb2,0xc6,0xe5,0xde,0xac,0x2c,0x0e,0x7b,0xb3,0xfe,0x39,0xfa, -0x88,0x2a,0x4e,0xf9,0x80,0xe7,0x09,0x53,0x5e,0xbb,0x14,0x1e, -0xb9,0x27,0xb4,0xad,0x2b,0xf5,0xcd,0x31,0xf8,0xd8,0xe4,0xc9, -0x3c,0x2d,0x78,0x5e,0x45,0x07,0x9c,0xf4,0xd6,0xbe,0x99,0x49, -0xa1,0x79,0x7b,0x12,0x23,0xff,0x11,0x10,0xd3,0xe8,0x69,0x64, -0x21,0xdf,0x34,0x8e,0x94,0xd1,0x2c,0x21,0x74,0x93,0x18,0xd6, -0x45,0x46,0x50,0x7c,0x99,0xa0,0xfa,0x1c,0xfa,0xfd,0x6c,0xef, -0x8b,0x00,0x45,0x5b,0xb6,0x4e,0x2f,0xe3,0xce,0xab,0x3a,0x1b, -0x59,0x4d,0xb3,0x5d,0xf4,0xbd,0x9d,0xad,0xd9,0x93,0x30,0xd7, -0x34,0x92,0xe4,0xfa,0xb2,0x30,0x79,0x5a,0xe1,0x74,0xa4,0xd6, -0x7e,0x02,0x71,0x2d,0x5d,0x85,0x62,0xed,0xde,0x02,0x9f,0x74, -0x22,0xb0,0x79,0x78,0x8f,0xd9,0xec,0x74,0x97,0x91,0x24,0xd2, -0xe6,0xe2,0xc6,0xed,0x44,0x56,0xa3,0xf4,0x7e,0xa7,0x0e,0xd5, -0x52,0x68,0xd0,0xda,0xc1,0xd8,0xc1,0xc1,0x9e,0x7e,0x62,0x35, -0xe2,0x7f,0x96,0xbe,0x62,0x8e,0xb1,0x41,0x63,0x69,0x9d,0x2c, -0xe8,0x46,0x88,0x1c,0x9f,0x8b,0xa8,0xc8,0xae,0x48,0x8f,0x0f, -0x44,0x29,0x2d,0xd8,0xe0,0x5b,0x0a,0x88,0xf7,0xad,0xd7,0x29, -0x7a,0x7c,0xeb,0xb0,0xe8,0xd5,0x1d,0xef,0xa5,0xd7,0x29,0x65, -0x18,0x86,0x66,0xb3,0xd0,0x9d,0x5f,0xe5,0xd7,0xf7,0xe1,0xc4, -0x3b,0xaf,0x90,0xb9,0x0b,0x08,0xf1,0xce,0x2b,0xe4,0x87,0xcf, -0x5a,0xa4,0x10,0xcb,0x6e,0x04,0x4e,0xfc,0x45,0x45,0x37,0x4c, -0xd3,0xda,0x8a,0xaa,0xbb,0x8b,0x6a,0x2f,0x3d,0x5f,0x79,0x1d, -0x1f,0x23,0xd2,0x2a,0xa0,0x48,0x1f,0x63,0xd3,0x3a,0xa0,0x6b, -0xa3,0x3e,0x92,0x4a,0x01,0x41,0x81,0xba,0x4b,0x60,0x47,0x7b, -0xea,0xa0,0xf6,0x1e,0xff,0x79,0x5f,0x3d,0x2e,0x23,0x2a,0x13, -0x90,0x04,0xa7,0xc3,0x03,0x94,0x5d,0x9a,0xd0,0x91,0x0a,0x95, -0x32,0x74,0x90,0x82,0x5f,0x1e,0x44,0x41,0x8d,0x99,0x0f,0x53, -0xa0,0xc2,0x97,0x4e,0x99,0x97,0xfb,0x29,0x40,0x99,0xb1,0x16, -0xff,0x34,0xd4,0xd1,0x75,0xa0,0x62,0x42,0x7f,0xa6,0xa3,0x7d, -0x36,0x1d,0x6a,0x1c,0x6a,0x0c,0x40,0x34,0xfa,0x2c,0x50,0x40, -0xf5,0x72,0x8b,0x10,0x45,0xf2,0xfa,0x06,0xd7,0xcf,0x3d,0xdb, -0xe0,0x38,0x4f,0x9f,0x4a,0x38,0x89,0x1c,0x87,0x89,0x2c,0x21, -0x51,0x6f,0xd8,0x3e,0x5e,0x32,0xd9,0xd1,0xad,0xb9,0x53,0x43, -0x42,0x1a,0xd2,0xaf,0x50,0xac,0x0e,0x75,0xab,0xeb,0xa8,0x7f, -0xb6,0xc4,0xfd,0xa2,0xe3,0x7e,0xd9,0x8e,0xac,0x25,0x06,0xbb, -0x92,0x31,0xbd,0x0c,0x06,0x0b,0xb3,0x96,0xa2,0x24,0x88,0xa1, -0xc5,0x22,0xbd,0xc2,0x71,0xa1,0x3c,0x66,0xb4,0xe7,0x50,0x89, -0x8d,0x3c,0x92,0x47,0x7b,0x86,0xa0,0x16,0x0e,0xaa,0x25,0x84, -0x55,0x2c,0x73,0x72,0x0f,0x8a,0x16,0x3e,0xef,0xa2,0xf8,0x73, -0x86,0x31,0x53,0xa0,0x19,0xc8,0x36,0x69,0xfb,0x06,0x04,0xc4, -0x5b,0xb2,0x2a,0x77,0xa7,0x5a,0x0f,0x0a,0x99,0x4c,0xf6,0x4e, -0xc3,0xd2,0x18,0x47,0xb4,0x8d,0x20,0x2e,0x22,0x3f,0x59,0x2f, -0xba,0x14,0x08,0x4e,0xd1,0x90,0xee,0xea,0xd2,0x57,0xfb,0xdc, -0xb4,0x27,0x85,0x53,0x1b,0x28,0x18,0x4b,0x2e,0x4b,0x6d,0xb3, -0x8f,0xac,0x5d,0xd2,0xe3,0x63,0x02,0xe4,0x4d,0x52,0xcb,0x1a, -0x72,0xe6,0x3e,0x15,0x57,0x4c,0xe2,0x69,0x57,0x7e,0xcd,0x70, -0x27,0x99,0x73,0xed,0xf0,0x4a,0x5e,0xf6,0x5c,0x8c,0xe4,0x76, -0x99,0x9c,0x81,0x16,0x91,0xd9,0xfb,0x93,0x29,0xaa,0xab,0x11, -0xf5,0xa4,0xe0,0x4c,0xa3,0xb8,0x99,0x08,0xd5,0x53,0x49,0x9d, -0x68,0x71,0x72,0x61,0x77,0xc5,0x6c,0x9d,0xb4,0xe8,0xd0,0x83, -0xad,0x72,0x3a,0x04,0x9d,0x86,0x70,0x47,0x7d,0x01,0x68,0xa2, -0xa1,0x7a,0x17,0x87,0xb7,0xa4,0xad,0x83,0x20,0x43,0x81,0xbc, -0x11,0xc8,0xea,0x96,0x50,0x85,0xd3,0x4d,0x40,0x8e,0xdb,0x1a, -0x07,0x54,0x49,0x63,0xf3,0x79,0xe6,0x6e,0x3e,0xcf,0x04,0x80, -0xe0,0x43,0x23,0xc9,0xe1,0x49,0x58,0x26,0xd2,0x7a,0x7a,0x57, -0x86,0x43,0xac,0x85,0x46,0x64,0x88,0xb3,0xf6,0x3f,0x74,0x9c, -0x29,0x9f,0x6a,0x2d,0x6b,0x7b,0x62,0x00,0xe8,0x61,0x24,0x8d, -0x4c,0xd5,0xc9,0xb2,0x52,0x96,0x31,0x66,0xfd,0x07,0x6c,0x53, -0x68,0x54,0xa2,0x8b,0xe4,0xcf,0xe9,0x57,0xaa,0x38,0xcb,0xe6, -0x0e,0x8f,0x7a,0xda,0x2d,0xe3,0xe9,0x10,0x88,0x7b,0x96,0xdc, -0xb1,0xc6,0xb1,0xe1,0xda,0x98,0x4c,0x05,0xc2,0xbb,0x4c,0xd7, -0x78,0xe1,0x77,0x56,0x07,0x94,0x4e,0x66,0x4c,0x13,0x34,0xa9, -0x4e,0xd7,0x2f,0x50,0xae,0xa5,0x05,0x02,0x35,0xc1,0x74,0xca, -0xfb,0xd4,0x80,0xcc,0xbb,0x8c,0xab,0xf6,0x7c,0x71,0x75,0x38, -0x5f,0x02,0x23,0xb0,0x2a,0xf2,0x56,0x9f,0xfa,0xdb,0xa7,0x56, -0x07,0x3a,0x60,0xcc,0xe2,0x81,0xcf,0x56,0xcd,0xe4,0xe7,0x0c, -0x3e,0x9f,0xc6,0xe3,0xee,0x20,0x8c,0x9f,0xce,0xc6,0x83,0xb0, -0x0f,0x6a,0xb1,0x45,0xd2,0x7f,0xfc,0x87,0x4f,0xdb,0xea,0xdc, -0xdf,0x4f,0x87,0x40,0x0c,0x19,0xd9,0xf1,0xa6,0x38,0xa8,0x32, -0x7a,0x2f,0x17,0xe8,0xe1,0x8c,0x16,0x2d,0x00,0x80,0xab,0x58, -0xfc,0x33,0x30,0x0b,0x42,0xdd,0xef,0x66,0x35,0x6f,0x78,0xf0, -0xa6,0x42,0xcb,0xc1,0xbe,0x1e,0x5e,0x10,0x5b,0x6f,0x56,0xb7, -0x69,0xb3,0x10,0x5b,0x09,0x02,0xd8,0xf2,0x9f,0xe6,0xfe,0x45, -0x63,0xfb,0xd9,0x1d,0xdf,0x03,0x27,0xab,0x25,0x16,0x4e,0x8f, -0xb4,0xa5,0x02,0x99,0xcb,0x42,0xa1,0x24,0x42,0x59,0x63,0x52, -0xec,0x58,0x99,0xc0,0xaa,0x43,0x0a,0xba,0x51,0x6e,0x4e,0x8f, -0x90,0x4d,0x2d,0x07,0x1e,0x8d,0xb3,0x02,0x35,0xe6,0xf1,0xd0, -0x9b,0x8e,0xf6,0x70,0xce,0x1e,0x7e,0x1d,0xf6,0x14,0x36,0x7b, -0xc8,0xdc,0xf2,0x12,0x17,0x0c,0xb8,0x12,0x2b,0xf8,0x3c,0x73, -0x2b,0x7f,0x75,0x85,0xcb,0x78,0xde,0xae,0x27,0x53,0x8a,0x7d, -0x86,0x7b,0x55,0xbe,0xa9,0xfb,0x91,0xda,0x98,0xe8,0x0b,0x8a, -0x18,0xe8,0x08,0x1a,0x0e,0xdf,0xcc,0x6b,0x15,0x31,0xaf,0x45, -0x91,0xe4,0x29,0x4c,0x4a,0x84,0xa9,0x70,0x4f,0x67,0xe8,0x95, -0x4b,0x4a,0xd6,0x8e,0xe4,0x90,0x81,0xbe,0xdc,0xf8,0x73,0x5c, -0x5e,0x2f,0x04,0x94,0x56,0xa3,0x2f,0x86,0x05,0x9a,0x18,0xe3, -0x06,0x17,0x7f,0x4d,0x98,0xa6,0x69,0x54,0xa3,0xd5,0x8e,0x15, -0x35,0x80,0xa8,0x4e,0x23,0x72,0x5e,0x23,0x5c,0xa7,0x06,0x61, -0xde,0x06,0x3e,0x97,0xcd,0x93,0x81,0xed,0xbc,0xc9,0x22,0x95, -0x1d,0x58,0xea,0xd7,0x77,0x2f,0xc4,0x46,0x54,0xea,0x14,0xa8, -0x39,0x20,0x90,0x0b,0x07,0x3e,0xae,0xe4,0x3a,0x4f,0xa6,0xea, -0x78,0xda,0x39,0xd4,0x51,0x07,0xd4,0x0c,0x65,0x9d,0x1c,0xb5, -0x9f,0x7c,0xd1,0x09,0xb2,0x3d,0xfc,0xe8,0x49,0x50,0x3f,0x88, -0xdb,0xbc,0x7a,0xcf,0x94,0x48,0xa7,0x94,0x55,0x48,0x41,0x3f, -0x9e,0xbc,0x85,0x1c,0x75,0x28,0xdf,0xe5,0xe0,0xe0,0xfb,0x30, -0x15,0xc4,0x9e,0x10,0xc3,0xd0,0xff,0x65,0x7b,0x50,0x56,0x3a, -0xc8,0x83,0xc2,0x25,0xba,0x03,0x63,0x4b,0xaf,0xfa,0x62,0xd6, -0xab,0x68,0x24,0xe9,0x55,0x4f,0xf1,0x13,0xbf,0xde,0x43,0xe4, -0x7b,0x8e,0x7c,0xff,0x14,0x3f,0x69,0x88,0x11,0x55,0xc4,0x08, -0xd1,0x6f,0x46,0x45,0x16,0x4b,0x58,0x8d,0x0b,0x62,0x81,0x2c, -0x4a,0xb3,0xe0,0x82,0x0e,0xc6,0x69,0x1d,0xae,0x72,0x5d,0x4c, -0xc5,0xa6,0x57,0xc1,0x68,0x5f,0x31,0x6b,0x46,0xe9,0x17,0x8e, -0x10,0x8d,0x02,0x0e,0x4e,0xd2,0x4e,0x67,0xaa,0x79,0xf0,0x16, -0x6a,0x8e,0xe3,0x1b,0x14,0xae,0x0a,0xed,0x44,0x03,0xaa,0xde, -0x68,0x03,0xe4,0xa5,0x87,0xb3,0x41,0xa6,0xe8,0xad,0xd8,0x81, -0xd8,0x30,0x83,0x3e,0x8c,0x04,0xa7,0xf9,0xdb,0x8f,0x5f,0xb8, -0xdf,0x49,0x91,0x72,0x47,0x47,0x1a,0x16,0x40,0xf9,0x97,0xd7, -0x7c,0xdb,0x4c,0xa2,0xf4,0x3e,0x91,0xba,0xb3,0xe8,0x65,0xf9, -0x22,0xc9,0x3d,0x73,0x96,0x57,0xf7,0x43,0x72,0x62,0x56,0x0f, -0x70,0x1b,0x53,0xcc,0xeb,0xd0,0x9b,0xd7,0x46,0x2f,0xa1,0xe2, -0x7b,0x5c,0xf8,0x1e,0x8d,0x63,0x97,0x32,0xa9,0x2f,0x53,0xde, -0x2d,0xff,0x51,0xaf,0x44,0xed,0xc1,0xd1,0x52,0x05,0x17,0xc9, -0xe1,0x8d,0x0e,0x02,0x7a,0xf8,0x36,0x3d,0xe3,0xdc,0xca,0x8f, -0x83,0x4a,0x07,0x34,0xc4,0xef,0xa4,0x74,0xd8,0xdc,0x70,0x06, -0xdc,0xb2,0xdd,0x64,0x4a,0x5e,0x28,0xb2,0x0e,0x9f,0xc9,0x50, -0xe6,0x1a,0xaf,0x63,0x82,0x52,0x00,0xaa,0x45,0xd4,0x19,0x9c, -0x42,0xa7,0x8f,0xab,0xa8,0x8b,0x1f,0xa0,0xac,0xe3,0x18,0x90, -0xc4,0x6b,0xf8,0xf3,0xed,0x10,0x8d,0x2e,0x50,0x29,0xf0,0x1f, -0x34,0x37,0x49,0xc3,0x08,0x7a,0xd3,0xd9,0xb1,0x65,0xb1,0xe6, -0x7d,0xcb,0x68,0xe3,0x9a,0xe2,0x51,0xfb,0x06,0x3a,0x9e,0x5e, -0x43,0x95,0x91,0x9e,0xeb,0x40,0x5c,0x3f,0x05,0x7a,0x30,0x08, -0x54,0x41,0x10,0x68,0xea,0x44,0x92,0xe4,0xe8,0xba,0x8b,0xc4, -0x11,0x85,0xa0,0x7c,0x63,0xd4,0xa9,0x9f,0x76,0x06,0x01,0x10, -0x2b,0xc3,0x27,0x3e,0xc3,0x04,0xa3,0x6f,0x87,0xa7,0x11,0x14, -0xd7,0x95,0x6f,0xdb,0xee,0x4e,0xb0,0xb7,0x5b,0xaa,0x02,0x0e, -0xab,0xac,0x0d,0xe1,0x63,0xfa,0x48,0x2b,0xdb,0x93,0x90,0xe5, -0xf4,0xfc,0x72,0x83,0xf7,0x3d,0xac,0x2a,0x3c,0x7d,0x3a,0x30, -0x07,0xe4,0x9c,0x6f,0x6a,0xa7,0xff,0xcb,0x70,0x2c,0x2b,0x16, -0xfa,0xfc,0xd1,0x1d,0x4c,0x3b,0x32,0x2a,0x80,0xb9,0xc2,0xe4, -0x9e,0x67,0x57,0xeb,0x72,0x8a,0x44,0x5a,0x18,0x53,0x40,0x07, -0x3f,0x56,0x4c,0x5c,0xa1,0x10,0x54,0x26,0x06,0x58,0x32,0x8d, -0xe0,0xc7,0xa6,0x03,0xa9,0x20,0xde,0x58,0x87,0xf7,0xe8,0xca, -0x6d,0x0e,0xf2,0xfc,0xed,0xd0,0xca,0x5b,0x2e,0x92,0xeb,0x69, -0x64,0x1e,0xec,0xfc,0x16,0x28,0x8a,0x8b,0x02,0x1d,0x89,0x61, -0xa3,0x51,0x83,0xdf,0xdb,0xde,0xe2,0x9a,0x2c,0xa8,0x5a,0x2a, -0x72,0xbd,0x53,0x91,0xeb,0x9d,0x8a,0x5c,0xef,0x54,0xe4,0xba, -0xa5,0x22,0xd7,0xad,0x15,0xb9,0x3e,0x58,0x11,0xdd,0x8e,0xd7, -0xcd,0x16,0x6a,0x36,0x59,0x20,0x0e,0x8b,0xa7,0x6c,0x30,0xd9, -0xbb,0xae,0x47,0xff,0x4b,0xed,0x79,0x3a,0x53,0x7b,0x59,0x38, -0x76,0x2b,0xbb,0x1d,0xc5,0xba,0x8e,0xb0,0xd3,0x55,0xb8,0x7b, -0xb2,0x81,0x2c,0xaf,0xb9,0xda,0xc6,0x5f,0x6e,0x1b,0x5c,0x1c, -0x32,0xe5,0xf6,0x32,0x91,0xf8,0x0e,0xa3,0x2d,0xfe,0xf1,0xb0, -0x1f,0x85,0xb4,0x8e,0xc0,0x0e,0x1d,0xd2,0xca,0x00,0x7b,0x0f, -0xac,0x20,0xaf,0x2e,0x3d,0xea,0x33,0x10,0x89,0x37,0x00,0x84, -0xe2,0x28,0x9a,0x5d,0xf0,0x17,0xba,0x68,0x01,0x7e,0xe2,0x6a, -0x13,0xfe,0x78,0xb2,0xac,0x50,0x2d,0x13,0xec,0xc1,0x1c,0x66, -0x54,0x49,0x89,0x59,0x68,0x66,0xd6,0xfd,0x0d,0x99,0x88,0x71, -0x77,0x77,0x40,0x2c,0x65,0xd9,0xf2,0x9f,0x87,0x2c,0x3c,0xdb, -0x47,0x70,0x83,0x42,0x32,0xa4,0x65,0x95,0x2b,0x53,0x66,0x2e, -0xfc,0x9f,0xb3,0x20,0xb4,0x8b,0x34,0x63,0x33,0xda,0x80,0xe3, -0x41,0x43,0xdb,0xe8,0x2c,0xd3,0x9a,0x16,0x1f,0xb4,0x18,0x78, -0x98,0xd9,0x07,0xef,0xbf,0xf0,0x88,0x2b,0x6d,0x40,0x9a,0xb7, -0xf8,0x28,0x27,0x27,0xb1,0x31,0x18,0xbf,0x63,0xa5,0x6c,0xda, -0xec,0xca,0x53,0x67,0xb0,0xad,0x08,0xef,0xdb,0x9a,0x54,0xab, -0x0f,0xc8,0x83,0x5a,0x2c,0x5d,0x8b,0x33,0x1d,0x08,0x62,0xd9, -0x00,0x11,0x46,0x78,0xf8,0x8b,0x4e,0x9c,0x25,0xdb,0x92,0x1a, -0x99,0x08,0x39,0xa2,0x05,0x05,0x94,0x77,0x00,0x15,0x9a,0x58, -0x80,0xe4,0x16,0xd9,0xd9,0xee,0xdc,0x51,0x6f,0x14,0x1f,0x03, -0x6d,0x55,0xdf,0x7a,0x90,0x23,0xbb,0xdf,0x56,0xda,0x2e,0xb5, -0xf6,0xb5,0x7b,0x8d,0x4b,0xd8,0xe9,0x99,0x80,0x5e,0x92,0xce, -0x2f,0xd0,0xa0,0x84,0xec,0x01,0xf0,0x78,0x8c,0x6f,0x24,0x5b, -0x5a,0x91,0x75,0xb2,0x2b,0x16,0x10,0x90,0x48,0x17,0x10,0x23, -0xf1,0xea,0x4e,0x25,0x6b,0x70,0xbf,0x19,0x89,0x04,0xdc,0xaa, -0x8f,0x5f,0xd9,0x9d,0x34,0x9a,0xc3,0x2a,0x91,0x04,0x7b,0xb8, -0x52,0xa4,0xf0,0x6e,0xf5,0xe7,0x9f,0xd4,0xf9,0x9a,0x7c,0xb1, -0x3b,0x20,0x20,0x8b,0xcb,0xa4,0x79,0xee,0x78,0x68,0xbc,0x8d, -0x3c,0x95,0xcb,0x03,0x9d,0xb9,0x48,0xf9,0xa4,0x91,0xdd,0xd0, -0x8a,0x64,0xb5,0x4a,0x37,0x45,0x0a,0x40,0xbd,0x5e,0xcf,0x13, -0x37,0xa0,0x0c,0x7f,0x99,0x27,0xf1,0x45,0x44,0x9e,0xa0,0x2e, -0xd3,0x32,0x1a,0xf4,0xfb,0x42,0x17,0x7b,0xcf,0x71,0x3e,0xf9, -0xa3,0x01,0x18,0x05,0xef,0xab,0x2d,0x55,0xc6,0xa5,0x8b,0x37, -0xe5,0x68,0x02,0xac,0xc3,0xfe,0xb8,0x98,0x92,0x5f,0x53,0xfb, -0x28,0x52,0x61,0xfc,0xd5,0x22,0xc2,0xd9,0xb7,0xfa,0x6b,0xa7, -0x29,0x5d,0x4c,0xb1,0xb4,0xbb,0x6b,0xcb,0xc9,0x7c,0xd1,0x19, -0x57,0x78,0x39,0x57,0x6f,0xfa,0x63,0x52,0x07,0x35,0x87,0x96, -0x8c,0x9a,0xfb,0xe6,0x24,0x35,0x2b,0xac,0x55,0x83,0x4c,0x85, -0x8f,0xb6,0xdc,0xba,0xa9,0xac,0x9d,0x49,0x63,0xcc,0xa3,0x53, -0xd1,0x08,0xda,0xf3,0xda,0x10,0x20,0xd3,0x67,0xd4,0xb8,0x2d, -0x93,0x8d,0xd5,0xf2,0x8f,0x66,0x2d,0xb9,0x8d,0x0c,0x5e,0xad, -0xd3,0x77,0x57,0xed,0x5b,0x83,0x9c,0xb4,0x73,0x11,0xdb,0x8a, -0x96,0x5b,0x77,0xee,0x16,0x7c,0xa5,0xcd,0x3c,0xf5,0x18,0x92, -0x91,0x14,0x80,0x92,0x68,0xdf,0xb8,0xe1,0x5d,0x31,0x94,0x68, -0x0a,0xea,0x5b,0xa1,0x8c,0xff,0x03,0xf7,0xb7,0x38,0xd3,0x56, -0xfe,0x75,0xae,0x7c,0x54,0xb6,0x18,0x54,0x0d,0x48,0xc3,0x08, -0x7e,0x94,0xb9,0x95,0x11,0x90,0xf4,0xc1,0x9b,0x61,0x23,0xe5, -0x34,0x72,0x7e,0x95,0xe7,0x02,0xd8,0x4e,0xdb,0x25,0xce,0x3e, -0x58,0x17,0x47,0x4e,0xeb,0xea,0x47,0x94,0x8e,0xd2,0x0e,0xb4, -0x78,0xb2,0xe1,0xe9,0x8b,0xee,0x69,0x42,0x6e,0xdc,0x5d,0x41, -0x50,0xb9,0xe5,0x81,0x51,0xb6,0x35,0x7a,0xe7,0x2d,0x3e,0xb8, -0x09,0x05,0xc8,0xe5,0x37,0x5a,0xaf,0xa6,0xca,0x8a,0x1d,0x81, -0xb7,0x6a,0x4f,0xf2,0x56,0x46,0x22,0x30,0x5f,0xce,0xa4,0xb1, -0x7b,0x28,0xb0,0xcc,0x68,0xa0,0x1a,0x00,0xaa,0xdb,0x18,0xed, -0xd7,0xee,0x60,0x0f,0x3d,0x3e,0x5d,0x6f,0x05,0xc1,0x61,0xce, -0x36,0x38,0x8c,0x37,0x70,0x16,0x9b,0x41,0x32,0xe7,0xab,0xec, -0xaa,0xdd,0x1e,0x99,0x52,0x9a,0x4a,0x87,0x31,0x42,0x4d,0xd7, -0x29,0xde,0xcb,0x8b,0x97,0xf2,0x06,0x2f,0x4e,0xee,0x0b,0xda, -0xb2,0xba,0xe0,0xe9,0xd1,0x56,0x4d,0xe8,0x0c,0x1a,0xbf,0x26, -0xfd,0xe9,0xe9,0x10,0x4f,0xa1,0x29,0x30,0xc0,0x40,0xab,0x8a, -0x71,0xe9,0x28,0x18,0x74,0x67,0x56,0x6f,0x68,0xb6,0x6e,0x06, -0x29,0x77,0xd1,0xd0,0xf1,0xba,0xfa,0x3b,0xc6,0x31,0x9f,0x77, -0xb9,0x90,0xd0,0x36,0xfb,0x68,0x73,0x15,0xf7,0x82,0x8c,0xa3, -0x79,0xeb,0x47,0xeb,0x24,0x17,0x3b,0x6a,0x06,0x22,0xc2,0x53, -0x1e,0x52,0x41,0x3c,0xda,0x53,0x87,0x4f,0x32,0x7d,0xd3,0xb6, -0x54,0xb0,0xc4,0x84,0x59,0x0e,0x26,0xf4,0xdd,0xc4,0x1a,0x13, -0x6b,0xbd,0x33,0xfd,0x30,0xfd,0x8d,0x5c,0x5a,0x93,0x67,0x3d, -0xed,0x16,0xdb,0xd7,0xfe,0xb1,0x7b,0xd2,0x69,0x33,0xff,0x09, -0xa8,0x55,0xe5,0x6a,0xa3,0x07,0x4c,0x22,0x57,0x90,0x82,0x9b, -0x2b,0x50,0x36,0xbc,0x07,0xb4,0x24,0x6e,0x76,0xe9,0x85,0x72, -0xf2,0x39,0xcc,0x51,0x4f,0xfa,0xfd,0x29,0x7b,0xfa,0xfb,0x20, -0x5d,0x48,0x11,0xd8,0x76,0xd0,0xf5,0xf1,0x56,0x30,0xb9,0x2d, -0x89,0xfd,0xad,0x65,0xa3,0x4b,0x4e,0x9e,0x3d,0xed,0x93,0x5c, -0x7e,0xff,0x43,0x7a,0x85,0xbb,0x47,0x9b,0x22,0x37,0x05,0x9e, -0xa2,0x83,0x9d,0x45,0x87,0x12,0x27,0x17,0x1f,0x7a,0xb1,0x7b, -0x36,0x46,0xb5,0xef,0xb9,0xde,0xdf,0x1a,0x67,0xb6,0x4d,0x3e, -0xa8,0xcb,0x72,0x98,0x73,0xcb,0x7f,0xee,0x57,0xd0,0x1c,0xe8, -0x5d,0x4e,0x35,0xee,0xe7,0xa9,0xf4,0xfb,0x8a,0x6d,0xf2,0x54, -0x0e,0xff,0xd7,0x23,0x33,0x63,0xa4,0xc5,0xf7,0x74,0x27,0x16, -0xd2,0x8e,0x8f,0x1f,0xd1,0xc8,0xa9,0x47,0xd0,0x2a,0x18,0x6b, -0x35,0xaa,0xc2,0xb3,0xd7,0x45,0x76,0x39,0xf6,0xaf,0x23,0xf8, -0xbf,0x0a,0x06,0xe6,0x5d,0xea,0xeb,0x20,0xc4,0xa3,0x96,0xbd, -0xed,0x78,0x8d,0x47,0x39,0xb4,0xea,0xe6,0xac,0x7e,0x70,0x72, -0xad,0x46,0xdb,0x41,0xd2,0x7d,0x12,0x4c,0xed,0x67,0xc4,0xab, -0x5e,0x8c,0x57,0x41,0xd6,0x80,0x0b,0x75,0x7e,0x2e,0x53,0x47, -0xdd,0x5f,0x2c,0x0c,0x3d,0x56,0xc9,0xe9,0xbf,0xa8,0xa2,0xb0, -0x10,0x09,0xbb,0x9f,0x7b,0x87,0xd5,0x5b,0x02,0x7a,0xb8,0x72, -0xcb,0x38,0xb7,0xf2,0xe3,0xcf,0xba,0x26,0x67,0x89,0x81,0x19, -0xe0,0xdf,0xa7,0xed,0x2b,0x4a,0x88,0x57,0x03,0xbb,0x94,0x10, -0x36,0xd0,0x4d,0xd0,0x5f,0xda,0x46,0xbc,0xe7,0x53,0xb5,0xd9, -0xe4,0xfd,0x54,0xbc,0xc7,0xc5,0xde,0x7b,0xfb,0xc4,0x08,0x47, -0x4d,0xbc,0x5d,0xf8,0xff,0xd1,0xc0,0x26,0xef,0xc8,0xbd,0x6f, -0x5f,0xfc,0x15,0x6c,0x18,0x26,0xc1,0x21,0x38,0x8d,0x60,0x61, -0x65,0xb6,0x54,0x6c,0xbc,0xd6,0xf5,0x56,0x3d,0x26,0xb3,0xca, -0x0a,0x28,0xc6,0xf2,0xf5,0xf5,0x6b,0xae,0x33,0xe0,0x5a,0x04, -0x53,0x90,0x0a,0x74,0xc8,0x17,0x5f,0xad,0x4a,0xba,0xea,0x1c, -0xbe,0xe7,0x7c,0xff,0x82,0x44,0x4f,0x77,0x8e,0xb0,0xde,0x1b, -0x2d,0x07,0x18,0x40,0x97,0xf2,0x5a,0x07,0x34,0x1b,0x65,0xc4, -0xec,0x17,0xb2,0x06,0xce,0x19,0x18,0x20,0x39,0x74,0xcf,0xf2, -0x7d,0xb4,0x10,0x00,0xb2,0x15,0x04,0x27,0x91,0xb6,0x83,0x3a, -0x25,0x9a,0x5c,0x8e,0xea,0xe4,0x18,0x45,0xe3,0xa5,0x63,0xb4, -0x88,0xd6,0x90,0x58,0x17,0x5b,0x6a,0xcc,0x95,0x34,0xa7,0x8a, -0x17,0x81,0xc9,0xa2,0xaa,0xd4,0x92,0x4d,0x25,0xed,0xc9,0x4a, -0x02,0xc6,0xf7,0xcc,0x6f,0xcd,0x77,0x8f,0xbc,0x5c,0xb7,0x89, -0x1b,0x26,0xf8,0xc5,0x26,0x99,0x93,0x57,0x67,0x41,0xe6,0xa3, -0x20,0x19,0x14,0xc5,0xbb,0x5c,0xbb,0xbb,0x06,0xc9,0x99,0x72, -0xf4,0x43,0x9e,0xa0,0xe9,0x7b,0x1a,0x59,0xdf,0xf4,0x10,0x04, -0xfb,0xb4,0x43,0xe7,0xd0,0xe8,0xab,0x19,0xff,0xe8,0x8c,0xc2, -0x82,0x55,0x45,0x6e,0xed,0xd7,0x86,0x9a,0xd0,0xa9,0xa1,0x4c, -0xf7,0x39,0xf2,0xe2,0x88,0x91,0xc7,0xc7,0x08,0xc0,0xdf,0x3e, -0x83,0x4d,0x74,0xe2,0x94,0xcb,0xee,0x65,0x39,0xfa,0xad,0x24, -0xd7,0x8f,0xfc,0x79,0x77,0xf7,0xfd,0x8f,0x2f,0x9e,0x7f,0xf7, -0x8a,0x40,0xc9,0xa9,0x8a,0x82,0xa4,0x23,0x12,0x86,0xa4,0x4f, -0x7a,0xa5,0x86,0x92,0xc8,0x2f,0xb6,0xaf,0x3d,0x64,0xdf,0xdd, -0x79,0x67,0x39,0xce,0x58,0x32,0x79,0x99,0xa7,0x0b,0x4a,0xc5, -0x0f,0xdc,0x36,0x93,0xf1,0x65,0x5a,0xe2,0x0b,0x5a,0x58,0x0e, -0x7e,0xb1,0xcb,0x34,0x3b,0x8d,0xfd,0x29,0xcb,0x37,0x38,0x34, -0x1c,0xc7,0x8e,0x1b,0x61,0xcb,0x91,0x4e,0x13,0x81,0x46,0x39, -0xbf,0x50,0xae,0x5c,0x20,0xaf,0xd2,0xb5,0x9d,0x42,0xe7,0x17, -0xdf,0x90,0x63,0x02,0x82,0x60,0x1f,0x05,0xe8,0x9f,0xff,0xac, -0x19,0x19,0x84,0x8d,0x6c,0x3f,0x5d,0xcd,0x16,0xe9,0x75,0xba, -0xe0,0xfa,0x14,0x2a,0x64,0x78,0x84,0x40,0x3f,0xf0,0xd4,0x2f, -0xab,0xac,0xc3,0xf6,0xfb,0x20,0x04,0x2b,0x55,0x04,0xf9,0x2c, -0x3b,0xe9,0x2a,0x74,0x40,0xea,0xb4,0x31,0x95,0x8a,0xca,0x94, -0x7d,0xb7,0xe9,0xb1,0xba,0x35,0x8a,0xae,0x8d,0x58,0xc9,0xb7, -0x41,0x47,0xda,0x2a,0xa1,0x2c,0xa2,0x1d,0x86,0x31,0xd0,0x88, -0x0b,0x2c,0x61,0xd9,0x89,0xff,0x9f,0x6e,0xf5,0x4b,0x65,0x36, -0xaa,0x6f,0xe3,0xb7,0x59,0x0e,0x63,0xa9,0x54,0x90,0x77,0xd3, -0xd4,0xfb,0x4a,0x4e,0x42,0xba,0x36,0x99,0x06,0xd3,0xdd,0xb4, -0xb6,0x4c,0xcf,0xd7,0x0b,0x95,0x65,0x38,0x6d,0xa6,0xb0,0x2a, -0xa7,0x9f,0x18,0x70,0x2a,0x82,0x7e,0x4b,0x57,0xb5,0xcf,0xbd, -0x84,0x5c,0x93,0xbb,0xc8,0xe5,0x22,0x80,0x3e,0x9d,0x45,0x99, -0x4e,0x1f,0xeb,0xcf,0x70,0xa2,0x3f,0xa7,0x23,0x5d,0x4a,0x61, -0x17,0xc1,0x59,0xd8,0xb9,0x92,0x81,0xf0,0x5b,0xb8,0x5c,0x04, -0xca,0x10,0x1a,0x70,0x4a,0xef,0x30,0x78,0xbe,0xbd,0x39,0x3e, -0xde,0x30,0xc0,0x98,0x50,0xc2,0xba,0x18,0x18,0xf9,0x0a,0x05, -0x45,0x03,0xf9,0x1b,0x13,0x5b,0xa0,0x86,0x01,0xaa,0x1d,0x3a, -0x67,0xc5,0x6b,0x1c,0x9c,0x57,0xd8,0x10,0xf8,0xcc,0x01,0x17, -0xc9,0x52,0x78,0x89,0x5c,0xde,0xc5,0xa8,0x62,0xf7,0x63,0xd4, -0x10,0x06,0x23,0xbe,0xe4,0xb9,0x97,0x4a,0xac,0x8a,0x21,0xe3, -0xee,0xee,0x76,0x7b,0x88,0x02,0x82,0xd6,0x45,0x10,0xb4,0xd5, -0x6d,0x5e,0xc6,0xb3,0x64,0xd5,0x04,0x5f,0x61,0xa4,0x8d,0x98, -0x3a,0x7c,0x13,0x4a,0x0e,0x2c,0x1a,0x08,0x47,0xa0,0x97,0xe9, -0x7a,0x07,0x8e,0x47,0x26,0x0d,0x06,0xaa,0x5f,0x9c,0xae,0x9b, -0x40,0x98,0x44,0x40,0x34,0x35,0xf3,0x30,0x49,0xda,0xbb,0x7c, -0x8d,0x09,0x55,0x78,0x7a,0x6e,0x47,0x94,0xd9,0x26,0xa4,0xc7, -0x76,0x04,0x27,0x19,0x10,0xf6,0x82,0xc0,0x50,0xd2,0x11,0x42, -0xee,0x6e,0x98,0xe2,0x1c,0x20,0x2f,0x17,0xf0,0x1c,0x45,0x9a, -0x8d,0x51,0x4f,0x70,0x4e,0x32,0x3e,0xc6,0x8c,0x35,0x2e,0x3e, -0xff,0xbd,0xd8,0xbd,0x38,0x45,0x0e,0x63,0xe4,0x33,0xe0,0xf4, -0x3d,0x36,0x5e,0x64,0xf0,0xa5,0xed,0x17,0xdf,0xfd,0xf5,0xe8, -0x9b,0x67,0x2f,0x5e,0x3e,0xff,0x1a,0x5d,0xce,0x2c,0xf0,0x9d, -0xa8,0x20,0xe4,0xb7,0x5f,0x11,0xe5,0x64,0xd1,0x63,0x2d,0xf2, -0xda,0xf2,0xe5,0xc5,0x0b,0xdb,0x32,0x93,0x17,0xea,0x41,0xfd, -0x50,0xc3,0xa2,0x40,0x1b,0x8e,0x6e,0x97,0x0e,0x6d,0x8e,0x8f, -0x15,0x99,0xbe,0xf4,0xe5,0xc4,0x9e,0x28,0x6f,0xd1,0xff,0x72, -0x48,0xf5,0xc0,0x12,0x42,0x10,0xb4,0xb3,0x55,0x76,0x83,0x7f, -0xd9,0x3e,0x35,0x44,0xbf,0xe7,0x84,0x02,0x46,0x34,0x79,0xf8, -0x74,0x68,0xde,0x65,0xcb,0x33,0xa4,0x1c,0x06,0x0a,0xca,0x27, -0x1f,0x53,0x67,0x55,0x8d,0x18,0x83,0xa9,0x42,0xf2,0x08,0xef, -0x35,0x68,0x0f,0x54,0xbe,0xa2,0x1d,0xdd,0x52,0xb9,0x9e,0xc2, -0xec,0x2a,0x2b,0x5f,0x38,0xea,0x8c,0xd0,0x85,0x95,0xde,0x4c, -0x5d,0x8e,0x2d,0xb4,0x0f,0x84,0x85,0x34,0xcb,0xb5,0x6c,0xd1, -0xe4,0xeb,0xba,0x14,0x3d,0x51,0xe9,0xe8,0x35,0xb2,0x3d,0x01, -0x9d,0x48,0x8e,0xac,0xab,0xdb,0x54,0x44,0xb0,0x85,0x65,0x9f, -0xf2,0x1c,0x8e,0x7a,0x2d,0x65,0x45,0x5e,0xda,0xad,0x66,0x84, -0x08,0x53,0xb0,0xfa,0x5b,0x6e,0x26,0xd9,0x48,0xd6,0xb3,0xf9, -0xaa,0xbd,0xc4,0x40,0x56,0x68,0x2b,0x76,0x10,0x44,0xb6,0xbe, -0x23,0x47,0xd0,0x2a,0xe2,0x41,0x53,0xd1,0x62,0x9d,0x98,0xab, -0xec,0x3a,0xcd,0x6c,0xe2,0x56,0xca,0x96,0xc9,0x60,0x74,0x37, -0x6e,0x0c,0x60,0xd3,0x7b,0xe8,0x42,0x98,0x6b,0x7c,0x30,0x2d, -0x9b,0x3c,0x5b,0x69,0xa2,0x14,0x5a,0x37,0x65,0xac,0x4a,0xb8, -0x97,0xef,0x98,0xfa,0xaf,0xbe,0xff,0xee,0xa7,0x57,0xcf,0xb0, -0x27,0x3f,0x0f,0x3d,0x5a,0xfb,0x3d,0xf7,0xc4,0xcb,0xef,0x86, -0x32,0x00,0x5f,0x18,0x1c,0xf4,0x75,0x78,0xd0,0x87,0x88,0xef, -0xff,0x3a,0x54,0xe0,0xf4,0x4d,0x51,0x83,0xbe,0x15,0x07,0x01, -0x4f,0xfc,0xf0,0x42,0x46,0xfc,0xf0,0xc2,0x13,0x3f,0xfd,0xeb, -0x8f,0xaf,0x06,0x6f,0x14,0x62,0x19,0xe2,0x68,0x3b,0x72,0xe8, -0x6d,0xc5,0x37,0x7f,0xff,0xee,0xab,0x57,0x2f,0xbe,0xff,0x2e, -0xba,0x8d,0x67,0x85,0x4c,0x83,0x2f,0x4f,0xc4,0xf3,0x4c,0x87, -0xe1,0x13,0x22,0x0a,0x3c,0x19,0xe5,0x88,0x02,0xcf,0x47,0xa1, -0xfe,0x3a,0xa2,0x8c,0x65,0xc4,0xd0,0x8a,0x19,0xe2,0x0b,0x1c, -0xe9,0x4a,0xc6,0xe0,0x27,0x44,0x68,0xac,0x84,0x14,0xf8,0x24, -0x83,0xf0,0xe5,0x61,0xf7,0xcc,0x72,0x19,0x41,0xdf,0x30,0x8e, -0x65,0x4b,0x19,0x01,0x5f,0xf2,0x50,0x96,0x82,0x74,0x32,0x7b, -0xa9,0x69,0xa2,0x23,0xdb,0x0d,0xf4,0x6e,0x0e,0xc2,0x17,0x8c, -0x7b,0xb4,0xb6,0x96,0x31,0x1c,0xc0,0xcd,0x8d,0xab,0xf5,0x42, -0xc5,0xf1,0x49,0x94,0xa9,0x19,0x55,0x0c,0x0f,0xc2,0x55,0xf8, -0x1d,0xbe,0x23,0x62,0x2a,0x8a,0xf5,0xdc,0xc2,0xd8,0x5a,0x25, -0x79,0x74,0xea,0x4f,0x5e,0x7b,0xaf,0x3f,0x9a,0xde,0x4d,0x5e, -0x47,0xaf,0xbf,0x78,0xfd,0xf4,0xf5,0x7f,0xbc,0x3e,0x7e,0x7d, -0xf7,0x7a,0xfc,0x3a,0x7c,0xdd,0x79,0xdd,0x7d,0x7d,0xfa,0xfa, -0xe4,0xf5,0xbf,0xbc,0x7e,0xf4,0xfa,0xdf,0x5f,0x8b,0xd7,0xa3, -0xd7,0x93,0xd7,0xd3,0xd7,0xb7,0xaf,0xb7,0xaf,0xfd,0xd7,0xc1, -0xd1,0xb4,0x13,0x9c,0x8e,0x9a,0xc3,0xa7,0xdc,0x8e,0x28,0xc5, -0x35,0xed,0x13,0x16,0xef,0xc4,0xe2,0x1d,0x0c,0xef,0x17,0xc9, -0xba,0x80,0xc5,0x3e,0x7a,0x2c,0x2d,0x7d,0x2a,0x97,0xcd,0x42, -0x8a,0x77,0xb0,0x4a,0x5c,0xe0,0x8f,0xdc,0xc2,0x23,0xc8,0x1d, -0x43,0x35,0xf6,0x8c,0xc6,0x89,0xb0,0x66,0x1c,0x79,0x1f,0xd1, -0x03,0x51,0xe3,0x8f,0xbc,0x8f,0xe8,0xef,0xe2,0xdd,0xdd,0x5d, -0x01,0xff,0x7c,0x25,0x9c,0x93,0x12,0x57,0xd1,0x3a,0x43,0x64, -0xc5,0x07,0x5a,0x58,0x18,0xe8,0x5a,0xe3,0xc5,0xd7,0xa5,0x8e, -0x8f,0x3d,0x1f,0xb7,0xf4,0xae,0x41,0x49,0x73,0x30,0x58,0x99, -0x60,0x26,0xa7,0xc2,0x80,0xee,0x47,0x8b,0x77,0x41,0x88,0xa5, -0x63,0x4d,0x1e,0x15,0xef,0xf4,0xa6,0xdf,0x37,0x8a,0x21,0xf8, -0x8c,0xa2,0x34,0xad,0x12,0xbc,0xfa,0x14,0x9e,0x84,0xf1,0xbd, -0x8e,0xac,0xef,0xdb,0x2c,0x05,0x48,0x2f,0xe8,0x78,0xc1,0x08, -0x1f,0xab,0xb1,0xa7,0x33,0xf5,0xfc,0x50,0xeb,0x6d,0x23,0x4e, -0xe3,0x49,0x4e,0x06,0x3e,0x74,0xed,0xa5,0x5e,0xcd,0x91,0xcb, -0x2f,0x37,0x68,0xad,0xc0,0x38,0x01,0xaf,0xb3,0xf2,0x87,0x59, -0x57,0xb9,0x59,0xda,0x16,0x62,0x7b,0xb2,0x68,0x52,0x39,0xc8, -0xfb,0x91,0xb4,0x1e,0xc0,0x97,0x03,0xed,0x25,0x18,0x3a,0x21, -0x95,0x4b,0x08,0x05,0x4b,0x2f,0x0f,0x53,0x22,0x7e,0x39,0xd0, -0x74,0x9b,0xc6,0x05,0xc7,0x57,0x70,0x78,0x3d,0x02,0x1f,0x36, -0x30,0x86,0x9b,0xa8,0xe9,0x39,0x1a,0xe9,0xde,0x1f,0x3f,0x1d, -0xe4,0x14,0xa3,0x72,0x98,0x75,0xe0,0xf1,0xb1,0xcc,0xbd,0xb3, -0x42,0xb4,0xf5,0x68,0x5e,0x09,0x1a,0xd8,0xe6,0x1a,0x51,0xd3, -0xb0,0x67,0xad,0x27,0x53,0xaf,0xf7,0x2d,0xcb,0x54,0x6d,0x79, -0xfd,0xe5,0xac,0x7f,0xee,0x5f,0x9d,0xb5,0x6b,0xd9,0x36,0x41, -0x07,0xd4,0x45,0x09,0xb6,0xba,0x4f,0xf7,0x54,0x70,0xf4,0x67, -0x07,0x90,0x62,0x1d,0x40,0x7e,0xb3,0xa6,0x09,0xc8,0xb1,0x2e, -0xca,0xa5,0x7c,0x54,0x62,0x57,0x57,0xe5,0x78,0x56,0x45,0xd5, -0x8e,0x1e,0xcb,0xac,0xd3,0xd3,0xd0,0x63,0xb9,0xbd,0x89,0x97, -0x4b,0x9f,0xb4,0x1b,0x7a,0x73,0x88,0x3c,0xfc,0x1b,0xbe,0xf0, -0x04,0x1a,0xe9,0x27,0x02,0x0a,0xe9,0x83,0x96,0x36,0xfb,0x28, -0x47,0xfb,0x5e,0x1f,0x25,0xe1,0xc6,0x9e,0x2e,0xd6,0xe0,0xa4, -0x32,0x7b,0xfc,0xdc,0xb8,0x04,0xe3,0xeb,0x87,0x10,0x0b,0xda, -0x45,0xcc,0x9b,0x71,0x2a,0x70,0x0f,0x86,0x5b,0x02,0x0a,0x0d, -0x3c,0xae,0x11,0x28,0x84,0xbe,0xe0,0x2c,0x67,0xd8,0x3a,0x8e, -0x74,0x16,0x51,0x56,0x2d,0x5a,0x90,0xc9,0x37,0xd2,0x9f,0x96, -0x66,0x23,0xd5,0x09,0xdc,0x64,0x05,0x46,0x7f,0xad,0x6d,0x0c, -0xd5,0xf9,0x1d,0x8f,0x20,0x18,0xa2,0x3d,0xae,0xf1,0x62,0xc6, -0x7b,0xa9,0xfc,0x18,0xa0,0x95,0xcb,0x28,0x3b,0x0d,0xe5,0x66, -0xab,0x1a,0x8e,0x82,0x8a,0x0f,0xfc,0xf2,0x03,0x45,0xb9,0xea, -0x13,0x35,0x1e,0x57,0x77,0xeb,0x36,0x70,0xd1,0x58,0x1a,0xd8, -0xaf,0xe0,0x68,0x8f,0xd3,0xe4,0xda,0x9c,0xdf,0x96,0x93,0xef, -0xd7,0x84,0x36,0x9c,0xb4,0x74,0x96,0xc3,0x17,0xa9,0xe6,0xbc, -0x4b,0x64,0x9e,0x82,0xe3,0x48,0xfc,0xa2,0x28,0x29,0x6e,0x1c, -0x2b,0x03,0x94,0x40,0x24,0x85,0x44,0x0a,0x93,0xd7,0xa2,0x0b, -0x52,0x65,0xb6,0x56,0x3d,0xda,0xbd,0xe2,0x6b,0x1f,0xa4,0x18, -0x92,0xf7,0x16,0xe3,0xab,0x32,0xf3,0xa4,0x64,0xa6,0x85,0x5c, -0x7f,0x60,0xba,0x02,0x40,0xbf,0xcd,0x73,0x7a,0x51,0x17,0x62, -0xc7,0x2a,0x18,0x36,0x32,0xca,0xf3,0x04,0x2b,0xe3,0x11,0x7c, -0xcb,0xf1,0x82,0x20,0xbe,0x23,0x0f,0x1c,0x04,0x31,0x86,0x9f, -0x70,0xa8,0x56,0x22,0xb7,0xb8,0xb6,0xdb,0xf0,0x3a,0x6e,0x23, -0x97,0x70,0x1b,0xb5,0xcc,0xdb,0x38,0xb5,0xd2,0x52,0xdc,0x7e, -0x56,0x99,0x5d,0x6e,0xd0,0x7d,0x33,0x72,0x43,0x20,0xbf,0x6c, -0x33,0x56,0xda,0x29,0xcd,0x61,0x7a,0x99,0xe3,0x5b,0x1e,0xf8, -0xf2,0x39,0x44,0x58,0xef,0x84,0x21,0x38,0x5a,0xba,0xa1,0x5f, -0x5c,0x52,0x3c,0x10,0xae,0x13,0x79,0x74,0x72,0x7e,0x14,0x1d, -0x91,0xba,0x7d,0x34,0x3e,0xba,0xdd,0x1e,0x85,0xe4,0x96,0x69, -0xf4,0x1a,0x34,0x28,0xb5,0x85,0x4d,0xc8,0x76,0xf7,0xb0,0xa1, -0xbc,0x08,0x31,0x93,0x2c,0x33,0x10,0xba,0x67,0x14,0x8c,0x3b, -0x45,0x4f,0x6a,0xaf,0xd7,0x47,0xf8,0x32,0x2e,0xfc,0xc8,0x58, -0x2f,0xeb,0x79,0x1d,0x04,0xed,0x78,0x50,0xac,0x7c,0xd4,0xfd, -0xc7,0xe4,0xcc,0x57,0x35,0x00,0x1d,0x60,0x44,0x46,0x79,0xd2, -0x09,0x7d,0xf4,0xa8,0x8f,0x7c,0x2e,0x7a,0xd5,0x90,0x0c,0xa5, -0xf0,0x6b,0xac,0xc8,0x47,0xf4,0xe9,0xd9,0x91,0x9f,0xf5,0xaa, -0xa3,0xa7,0x47,0xf8,0xb0,0x74,0x70,0x74,0x7b,0x44,0xda,0x12, -0x60,0x87,0xf0,0x08,0x7f,0xf8,0x73,0x48,0xdf,0x43,0xac,0xec, -0xe8,0x68,0x3b,0xd2,0x04,0x21,0x0e,0xf9,0xa6,0x29,0xa4,0xf9, -0x04,0xd3,0x45,0x50,0x54,0x45,0x42,0x2a,0x90,0x12,0x8f,0x8f, -0x1f,0x71,0xe9,0x03,0x5a,0x35,0x5a,0x79,0x2b,0x27,0x1f,0x01, -0x63,0x5e,0x66,0x78,0xaf,0xd6,0x74,0xd7,0x2d,0x74,0xd7,0x44, -0x77,0xdd,0xa0,0xbb,0x46,0x5a,0x6b,0xfe,0x24,0xba,0xeb,0x7d, -0x74,0x73,0x4f,0x64,0x02,0x6a,0x26,0xa0,0x36,0x84,0x73,0xaa, -0xa2,0xbc,0xde,0xa1,0xbc,0x76,0x32,0xca,0x41,0x80,0x48,0x47, -0xdf,0xe3,0x78,0xbd,0x96,0xa4,0x0d,0x73,0x07,0x6e,0x5e,0xa2, -0x7e,0xf2,0x91,0x7c,0x42,0x0e,0xc4,0x77,0xf1,0xd1,0x34,0x38, -0x6a,0xc6,0x00,0x7a,0x3a,0x4d,0xf0,0x02,0xd1,0xc8,0x4b,0x02, -0x17,0xb0,0xdc,0x39,0x8f,0xc5,0x93,0x4f,0xb0,0xa3,0x0c,0xa8, -0x18,0x95,0x79,0x7d,0xbb,0xa3,0x4e,0x62,0x3a,0xb9,0x31,0xa2, -0xb1,0x89,0xfd,0x33,0x30,0x4f,0x82,0xed,0x3c,0x2e,0x61,0xb6, -0x49,0xe8,0x2d,0x1f,0xde,0xb2,0x48,0x68,0x7a,0x83,0x25,0x87, -0xcf,0x10,0x46,0x25,0xdb,0xad,0xa0,0xb1,0x03,0xca,0x0b,0xba, -0x61,0x0c,0x5a,0xad,0x87,0x8f,0xd2,0xe1,0xe8,0x80,0x60,0xf2, -0xe9,0x7a,0x15,0x0c,0xb4,0x3c,0x82,0x96,0xc8,0x12,0x2a,0x3f, -0xe4,0x03,0xb9,0xcc,0x74,0xf5,0x57,0x25,0xaa,0x07,0x73,0x29, -0x20,0x6f,0xe5,0xca,0x90,0xc5,0x07,0xdb,0x47,0xf7,0x4e,0x17, -0x31,0x43,0x1d,0x86,0x24,0xd9,0xc4,0x68,0x1a,0x11,0x8a,0xaf, -0xf0,0x11,0xa4,0x88,0x1f,0x43,0x04,0x62,0x31,0x1f,0xd0,0x2e, -0x9f,0x3c,0x94,0x11,0x38,0xb6,0x49,0x48,0xc2,0x07,0x98,0x7a, -0x73,0x85,0x8b,0x5e,0x51,0xc2,0x22,0xbd,0xf3,0xf9,0x0a,0x9f, -0xc7,0x3b,0xeb,0x9d,0xd3,0xef,0x9c,0x7e,0x57,0xf4,0xf4,0x03, -0xc5,0xde,0xdd,0xe1,0x9f,0xa2,0x25,0x63,0xe1,0x64,0x2c,0x1a, -0x19,0x57,0x9c,0x31,0xde,0xcd,0x08,0xca,0x12,0x67,0x5c,0xd1, -0x6f,0x4c,0xbf,0x33,0x9d,0x31,0xe7,0x8c,0x4b,0xfe,0x33,0xdb, -0xcd,0x9f,0x2f,0x65,0xfe,0x9c,0x7e,0x97,0x32,0xbf,0xb2,0x2c, -0x8e,0x3c,0xe5,0xf4,0xed,0x8a,0x27,0x7c,0xcf,0xd9,0x29,0xcf, -0x95,0x4a,0x29,0xad,0x32,0x71,0xf2,0x2e,0x73,0x5f,0x47,0x07, -0x4a,0x9b,0x25,0xdc,0xbc,0xef,0x40,0xbe,0x3b,0xf2,0x0d,0xba, -0x9e,0xb4,0xa7,0x19,0x03,0x01,0xb8,0x30,0xfd,0xaf,0x3f,0x7e, -0xff,0xf7,0x1f,0xde,0xfc,0xfc,0xec,0xc7,0x9f,0x26,0x3a,0x6d, -0x3a,0x66,0x51,0x86,0x51,0x51,0xc7,0xc9,0x99,0x57,0xde,0xae, -0x82,0x31,0x72,0xc9,0xd6,0x49,0x16,0x46,0x35,0x41,0xa2,0x11, -0x97,0x5c,0x3d,0x4d,0x27,0xb8,0x7e,0x9a,0x7a,0x81,0xd9,0x80, -0xcf,0x49,0xbf,0x97,0x57,0x57,0x1d,0xba,0xf8,0x6c,0x78,0xec, -0x37,0x18,0xd2,0x2c,0xcc,0xb2,0xb7,0x6a,0x2f,0xac,0xc9,0x0c, -0xc9,0x35,0xa8,0x6d,0xc7,0x9b,0xd0,0xe8,0x4e,0x24,0xe1,0x70, -0x04,0x05,0xd1,0x0b,0x22,0xfa,0xb4,0xed,0x40,0x25,0xe9,0xeb, -0xde,0xaa,0x06,0x7e,0xa3,0x29,0x03,0x7e,0x5d,0xb9,0x95,0x1e, -0xbb,0xdd,0x48,0x57,0x91,0x5e,0x10,0xf0,0x33,0x6a,0xb2,0x87, -0x01,0xc6,0x56,0xe3,0x73,0x4c,0x68,0xbe,0x19,0xbf,0x6c,0x3d, -0x98,0xe2,0x68,0x5c,0x02,0xaa,0xec,0x0a,0x39,0x99,0x39,0x83, -0xec,0x2f,0x14,0xc1,0xdc,0x45,0x06,0x8d,0x98,0x0c,0xd9,0xee, -0x72,0xb5,0xe5,0x75,0xe8,0x83,0xd8,0x8c,0x95,0xe0,0x10,0x21, -0x9c,0xc5,0x6b,0x50,0x56,0xf8,0xea,0xf8,0x97,0x31,0xae,0x4a, -0x81,0x02,0x5f,0xf2,0x1b,0xdf,0xef,0x36,0x26,0x8d,0x11,0xc6, -0x53,0xa6,0xcb,0xab,0x55,0x39,0xbe,0x56,0x3e,0xf0,0x75,0x14, -0x7a,0xad,0x38,0x91,0xcf,0xd7,0xcb,0xfc,0x0c,0xcf,0x2b,0xb2, -0xb1,0x77,0xd4,0x39,0xa2,0xda,0x58,0xf9,0xe4,0x62,0x0d,0xf3, -0x08,0x39,0x88,0x48,0x4e,0x2b,0x2a,0x3e,0x02,0xb4,0xde,0x47, -0x64,0x5f,0x6c,0x79,0xf2,0x50,0x9d,0x94,0x14,0xf4,0x4a,0xd4, -0xe2,0xbd,0x7c,0xb8,0xbc,0x8a,0xaa,0xb1,0x1e,0xe7,0x40,0x7d, -0xa9,0x02,0xeb,0x7c,0x8e,0xb2,0xf1,0x83,0x0d,0x68,0x16,0x51, -0xd7,0x51,0xed,0x00,0xd7,0xfb,0x80,0x07,0x53,0xf1,0xfe,0x7d, -0xf4,0xde,0x01,0x7e,0xbf,0x0f,0x78,0xa8,0x4e,0xd3,0x3d,0x7e, -0xe9,0x06,0x16,0x8a,0x5e,0x07,0x93,0x3a,0x58,0xa7,0x49,0x05, -0xe4,0x02,0xbd,0xef,0xa7,0x52,0x06,0x05,0xb0,0xeb,0xa3,0x00, -0x79,0xe3,0x05,0x1f,0xd1,0xe0,0x62,0xba,0x77,0x74,0xcb,0x3a, -0xf3,0x40,0xe9,0xcb,0x03,0x7e,0xce,0x92,0x45,0x80,0xa7,0x07, -0xaf,0x11,0x29,0x6d,0x8a,0xc2,0x81,0xb1,0xd3,0x61,0x85,0xcf, -0x59,0xa2,0xb1,0x68,0xb4,0xdf,0x28,0xa5,0x24,0x56,0xea,0xf9, -0x7b,0x67,0x6b,0xd4,0xda,0x11,0x69,0xee,0x90,0x66,0x42,0x6f, -0xcf,0x92,0x36,0x87,0x8b,0x60,0x9a,0x6f,0x36,0x79,0x72,0x1d, -0xb1,0xc2,0x16,0xe2,0xb7,0x67,0x8c,0x2c,0x59,0x3f,0xe3,0xf3, -0x62,0xdc,0xda,0xc8,0xd4,0xb3,0x41,0xaa,0x58,0x81,0x4e,0x7b, -0xf0,0x62,0x8a,0xfc,0x7b,0x77,0xa7,0x40,0xb2,0xad,0x50,0x0b, -0x86,0x5b,0x7b,0xa3,0xc4,0x20,0xa3,0x2f,0xab,0x06,0xec,0xb2, -0x24,0x5d,0x4b,0x17,0x7e,0x1a,0x04,0xed,0xe8,0x0a,0xf5,0xc6, -0xd0,0xf7,0x3f,0x7e,0xfd,0xe2,0xbb,0x67,0x2f,0xc7,0x59,0x0e, -0xeb,0x83,0x78,0x15,0xbe,0xbb,0x8a,0x71,0x4b,0x37,0xc6,0xf7, -0xc6,0x44,0x0e,0x2b,0x06,0xe9,0x6c,0x01,0x32,0x4b,0x0a,0x1b, -0x17,0xd0,0xad,0x31,0xc9,0x38,0x32,0xe2,0xc2,0x30,0x3f,0xd2, -0xc3,0x57,0x52,0x0b,0x43,0x73,0x0b,0x4d,0xe6,0x59,0x12,0x75, -0x2a,0x7e,0x77,0xf7,0xf2,0xc5,0x77,0xcf,0x9f,0xfd,0x88,0x93, -0xcd,0x23,0x02,0xba,0xbb,0xc3,0x78,0x98,0x72,0x78,0x30,0xa0, -0x87,0x98,0xf8,0x9d,0x90,0x32,0xcb,0xad,0xc3,0x56,0x4a,0x66, -0x19,0xbd,0xbb,0x43,0x27,0x12,0x26,0x3c,0x42,0x50,0xda,0xa2, -0x92,0x07,0x34,0x7f,0x5f,0xe7,0xc9,0x3c,0x5b,0xae,0xd3,0xf7, -0xc9,0x82,0x79,0x49,0x54,0xe0,0x49,0x0d,0x3f,0xf4,0xc4,0xe7, -0xfa,0xd1,0x9c,0xbd,0x47,0xf5,0xe4,0xa3,0x79,0xaa,0x7c,0x26, -0x89,0xeb,0xc0,0xe5,0x7c,0x67,0xcb,0x82,0x1a,0x55,0x28,0xc5, -0xd4,0x5f,0x32,0xdb,0x6a,0x35,0x9b,0x53,0xbc,0x0e,0xa6,0xf3, -0x31,0x9c,0x89,0x0b,0x69,0x29,0x8a,0xaf,0x20,0x8f,0x38,0x9a, -0xf0,0x4b,0x08,0xfb,0x58,0x95,0x62,0x02,0xde,0x7b,0x92,0xe9, -0x2a,0x32,0xb4,0x57,0x65,0x32,0x0e,0x06,0x20,0xc4,0x6f,0x61, -0x93,0x3e,0x53,0x89,0x41,0xb1,0x3e,0xaa,0x95,0xe0,0xea,0xae, -0x71,0xce,0xef,0x85,0xba,0xbd,0x41,0xd2,0x98,0xeb,0x55,0xbc, -0x7d,0x25,0x79,0x36,0xc9,0x69,0xb6,0x99,0xb2,0x6b,0xf5,0x40, -0x34,0xfd,0x9b,0x38,0xb3,0x89,0x9a,0x6a,0xe5,0x47,0x48,0x7b, -0xa0,0x3d,0x1a,0x50,0x75,0x86,0xdc,0x99,0xf0,0xd8,0xae,0x31, -0x18,0xb5,0x99,0x40,0x63,0x59,0x92,0x72,0x7e,0xf7,0x8c,0x76, -0xe0,0xd8,0x51,0x88,0xbc,0xe4,0xac,0x2e,0xec,0xd1,0x6d,0x3d, -0x87,0x79,0x32,0x1f,0xde,0xcf,0x01,0xed,0xb0,0xf9,0x62,0x11, -0x34,0x1a,0x8e,0xae,0x10,0x0d,0x73,0xd4,0x7a,0xa9,0x6e,0xf8, -0x0d,0x65,0x03,0x70,0xcf,0x81,0x84,0x80,0x1e,0x08,0xe5,0x2b, -0xc3,0x76,0xda,0x0f,0x14,0xe3,0x53,0xdb,0x23,0x80,0xb2,0x55, -0xe8,0x73,0x06,0xda,0xa3,0x87,0xa6,0x50,0xae,0xb3,0x75,0x9c, -0x8d,0xe3,0x47,0x8c,0xc0,0x09,0xad,0x1d,0x8f,0x05,0xb9,0x17, -0xc8,0xc8,0xa5,0xdd,0xf9,0xf7,0x09,0xa7,0x06,0x4e,0x2a,0x58, -0x49,0x80,0x30,0xe1,0x1a,0x1a,0x2f,0xea,0xe1,0xd5,0xbc,0xf7, -0xfb,0x9a,0x1f,0xa5,0xc8,0x16,0x80,0xa6,0x07,0xde,0x86,0xea, -0xd4,0x62,0x54,0x6e,0x8b,0xc8,0x59,0x30,0x3e,0x6b,0x15,0x8b, -0x33,0x47,0x20,0xb6,0xc1,0x48,0x3a,0xf3,0xdd,0xd9,0x56,0x03, -0xbc,0x67,0x4d,0x0b,0x75,0xa8,0x04,0x1e,0x56,0x52,0xcb,0x93, -0xc1,0xc5,0x63,0x3c,0x5b,0xf1,0x27,0x3a,0x46,0xc8,0x18,0x94, -0xaa,0xb3,0x60,0x4a,0xbb,0x45,0x95,0xc9,0xf2,0x9e,0xb3,0xc4, -0x95,0xce,0xf2,0x9e,0xb3,0xc4,0x95,0xc9,0x22,0x8f,0xb6,0xed, -0xee,0x4d,0x6b,0x18,0xbc,0x01,0xf5,0x5e,0x75,0xef,0x09,0x3b, -0xf8,0x87,0x1f,0xe5,0x39,0x5d,0xee,0xc8,0x72,0x3a,0x2a,0x0e, -0xae,0x31,0x85,0xea,0xf9,0x3b,0xc3,0x83,0x76,0x21,0xef,0x74, -0x7f,0x2b,0xc7,0x87,0x8d,0x00,0xd4,0xf5,0x77,0x98,0x09,0x9d, -0x5e,0x0b,0x83,0x18,0xe0,0x7f,0x50,0x4d,0x90,0xe1,0x26,0x31, -0x81,0x78,0x1f,0xc9,0x52,0x94,0xc5,0x67,0x5b,0xf5,0xbe,0x35, -0x35,0xdc,0x21,0xf8,0x5b,0xa2,0xd9,0xb4,0x11,0x31,0xaa,0x51, -0x85,0x6f,0x0f,0xd7,0xe2,0xdb,0x07,0x57,0xa4,0x0f,0xf2,0x6c, -0x2a,0xa2,0x6c,0x70,0x2c,0x2c,0x6d,0xc4,0x93,0x44,0xec,0x21, -0x3e,0xae,0x0c,0xf1,0xef,0xf7,0x10,0x1f,0x57,0x07,0x89,0x07, -0x14,0x0f,0x21,0xbe,0x0f,0xe4,0x3b,0xc4,0xbf,0x9f,0x46,0x0d, -0x2c,0x6a,0x9e,0x85,0x91,0xe5,0xe5,0xf7,0x7f,0x85,0x8e,0x69, -0xc2,0xaf,0x5e,0x7c,0xfb,0xfc,0xee,0xee,0x11,0xc6,0xbc,0x4f, -0xf2,0x0c,0x06,0x4a,0xbb,0xa2,0x18,0xc5,0x8f,0x0e,0x4a,0xae, -0xa8,0x63,0x48,0xbf,0x2f,0x74,0x64,0x20,0x4c,0xc1,0xea,0xd4, -0xd2,0xa4,0xbf,0xc7,0xce,0xd3,0x36,0x4d,0x69,0x4b,0x71,0x35, -0xde,0xe1,0xc8,0x85,0xf3,0x14,0x6a,0x22,0x30,0xc4,0x82,0x4e, -0x94,0xa0,0x95,0x80,0xca,0x3e,0xd1,0x83,0xe2,0xf1,0x71,0x73, -0x54,0x1c,0x7b,0xe6,0x1b,0x94,0x75,0x0a,0x78,0x53,0x1a,0x90, -0xa9,0xf2,0x49,0xb5,0xc9,0xd6,0x74,0x0e,0x63,0x55,0xfd,0x87, -0xef,0xff,0xf1,0xfc,0x47,0x85,0x4b,0x41,0xf8,0x36,0x38,0x67, -0x9e,0xaf,0xe2,0xcb,0x8d,0x82,0xa3,0x80,0x8f,0xcf,0xed,0xd3, -0xc4,0x0f,0x3d,0x8e,0x1f,0xcc,0xb6,0xf9,0x39,0xf6,0x55,0x3f, -0xa7,0xe7,0xa3,0xd2,0x4b,0xa6,0x1d,0x81,0xa7,0x7a,0x08,0xd8, -0xab,0x99,0x28,0x00,0xd4,0x4b,0xd4,0xb7,0xe2,0xe0,0x9a,0x5c, -0xdc,0xa8,0x48,0x35,0xe6,0x53,0x6c,0x60,0x0d,0xee,0x4d,0x45, -0x8e,0x9f,0xd2,0x07,0xbe,0x5a,0xa3,0xcd,0x8e,0x09,0x1c,0x65, -0x0a,0xd0,0xe3,0xcf,0xce,0x73,0x7d,0x76,0x2a,0x2b,0xf2,0x3f, -0xfc,0xf8,0xfd,0x0f,0xcf,0x7f,0x7c,0xf5,0xcb,0x44,0xa7,0x4d, -0x03,0x2a,0xa0,0x2f,0xe4,0xd3,0x7e,0x3a,0x9e,0x5d,0x09,0xe1, -0x2e,0xca,0x23,0xa3,0xb7,0x51,0x92,0x93,0x59,0x0f,0xfd,0x2e, -0x4f,0x88,0x25,0x04,0x43,0x16,0x35,0x9a,0x12,0x9c,0xa9,0x46, -0x58,0xe2,0x7e,0x9c,0x6c,0x3d,0x26,0x61,0xec,0x21,0x94,0x31, -0x8c,0xf5,0x27,0xbe,0x64,0x68,0xb2,0xb5,0x33,0x86,0x47,0x2a, -0x56,0x03,0x9c,0xd8,0xc6,0xd0,0xc0,0xd1,0x34,0x32,0x20,0xb0, -0xd1,0x15,0xba,0x83,0xa9,0x03,0xd0,0xcc,0xc7,0xd2,0x2e,0x9b, -0x0a,0x56,0x15,0x56,0xe4,0xc8,0x1e,0x63,0x20,0x8e,0xb5,0xb9, -0x6b,0x67,0x52,0x83,0xb0,0x9c,0x55,0x1d,0x35,0x1d,0x47,0x90, -0xeb,0xd6,0x4e,0xa5,0xf4,0x55,0x88,0xa5,0x69,0x8a,0x35,0xf0, -0x48,0xbe,0xc1,0xee,0x09,0xb9,0x5a,0x88,0x3c,0xa9,0xc1,0x92, -0x4d,0x06,0x24,0xa3,0x81,0x02,0xf5,0x9c,0xc8,0x23,0xfb,0x03, -0x14,0xf8,0xc8,0x43,0x21,0xf7,0x84,0x2b,0x1b,0xbb,0x0b,0x3d, -0x73,0x39,0x9c,0x96,0x37,0xee,0xe2,0x0d,0x56,0x5f,0x7b,0x0c, -0xb3,0xc4,0x75,0x9a,0xdc,0x7c,0x13,0xa3,0x5e,0x5e,0xdb,0x5e, -0xe8,0xc8,0x88,0x88,0x37,0xfb,0x29,0x37,0x9a,0x18,0x5d,0xe1, -0xb3,0xa7,0xb8,0x55,0x4a,0xf1,0xb4,0xe5,0x47,0x54,0xd0,0x5e, -0xbc,0xda,0x7b,0x44,0xf7,0xf0,0xd2,0x0b,0x08,0x45,0xab,0x5d, -0x47,0x8c,0xc7,0xb2,0x36,0xe8,0x8a,0x89,0x52,0x54,0x48,0xbe, -0x40,0x8c,0xc6,0x56,0xaa,0x5a,0x6d,0x67,0x3e,0x0a,0x3c,0x54, -0x1f,0x42,0x2a,0x5f,0x61,0xf3,0x78,0x86,0x0f,0x75,0x94,0x6d, -0xaa,0x3c,0xe9,0x71,0xcf,0xa2,0x5a,0x8e,0xa0,0xf8,0x61,0x06, -0xe7,0x04,0x8e,0x11,0xf1,0x05,0x40,0xb3,0xf0,0xd5,0xc6,0x62, -0x48,0xc8,0x57,0x19,0x2c,0xc2,0xf2,0x2b,0xe4,0x5f,0x40,0xcf, -0x6d,0xba,0x71,0x91,0xc5,0x5e,0x7a,0xfb,0x1f,0xf4,0x16,0x13, -0x63,0xa7,0xd2,0x98,0xf5,0xb3,0x09,0x0b,0x5b,0x30,0x89,0xe1, -0x63,0xab,0x51,0x42,0x18,0xfc,0xc8,0xf7,0x2d,0xd5,0xa3,0x61, -0x43,0x47,0xcf,0xe0,0x35,0x6d,0xe8,0xe8,0x37,0xe0,0x0d,0x72, -0x9f,0x20,0xe4,0xeb,0xa0,0x0d,0xb3,0xa7,0x76,0xb3,0x29,0x69, -0x61,0x66,0x6c,0xad,0x7d,0x7b,0x99,0x40,0x47,0x3d,0x08,0xde, -0x7a,0xf0,0xea,0x61,0x06,0x34,0x1b,0xb9,0x38,0x3e,0x2e,0x2b, -0x7a,0x0a,0x27,0x39,0x53,0x77,0x86,0x2b,0xa2,0x81,0x8e,0x29, -0xa5,0xbd,0x3b,0x1f,0x59,0x92,0xb8,0x47,0x6a,0xb7,0x5d,0x47, -0x3f,0xff,0xee,0x15,0xf4,0x8e,0xbe,0x89,0xf8,0xfb,0x0f,0x5f, -0x3f,0x7b,0xf5,0x9c,0xad,0xe8,0x24,0xc8,0x3f,0x5f,0xbc,0x8a, -0x86,0x26,0xfc,0xf5,0xf3,0x6f,0x9e,0xfd,0xfd,0xe5,0xab,0x37, -0x00,0xf7,0x2c,0xba,0x2d,0xd0,0xd4,0x6a,0x9d,0xac,0xc2,0x81, -0x55,0x98,0x6b,0xb8,0x48,0xed,0xdd,0x3c,0x50,0x95,0x58,0xa2, -0x56,0xb4,0xf6,0x6d,0x88,0x86,0x0f,0xa9,0xbb,0x3b,0x83,0xe9, -0xee,0x6e,0x22,0x73,0x4d,0xe5,0xfa,0x3f,0x8a,0x64,0x04,0x4e, -0x74,0xe4,0xa6,0xcd,0x05,0xb0,0x68,0xe4,0x6b,0x5a,0xb9,0x73, -0xfa,0x98,0xe9,0xfd,0x95,0xac,0xa7,0xaf,0x71,0x8d,0xad,0x6f, -0xf4,0x89,0x4a,0xfb,0x6b,0x12,0xe0,0x67,0x72,0x34,0x51,0x8e, -0x9d,0x50,0x13,0x88,0x2f,0x80,0x8d,0xed,0x80,0x01,0xe1,0xbd, -0xcc,0x4c,0x5f,0xab,0x1a,0x9b,0x4f,0x6b,0x7b,0x8b,0x5f,0xad, -0xd7,0x29,0x41,0xc7,0xdb,0x54,0xe8,0xbb,0x96,0x81,0x6d,0x33, -0x72,0x0b,0x32,0xb0,0x6a,0xfb,0x02,0x5f,0x56,0x69,0xdb,0x6c, -0xc2,0xdd,0x4e,0x69,0x21,0xc0,0x0f,0xb2,0xa3,0xe9,0x00,0x9d, -0x36,0x4b,0x83,0x01,0xf9,0x20,0x3b,0x6d,0xd0,0x1e,0x78,0x8d, -0x9d,0x9f,0xb8,0xa5,0xe7,0x2e,0x7f,0xb2,0x7d,0x88,0x9a,0x57, -0xe8,0xd9,0xf8,0x40,0x17,0xd2,0x6b,0x58,0x67,0x18,0x8c,0x9b, -0xe2,0xf8,0x98,0xd9,0xc2,0xb6,0x03,0xb4,0xd1,0x64,0x3f,0x8c, -0x3d,0xcf,0xae,0x8a,0xd4,0xba,0x3e,0xcd,0x9b,0x98,0xca,0x0c, -0x48,0xfa,0x13,0x93,0x3b,0x9b,0xea,0x18,0x1e,0x9f,0xea,0xe7, -0xa8,0x48,0x5d,0x62,0x90,0x3e,0x74,0xa8,0xc6,0x9a,0x2c,0xdb, -0x1a,0x82,0x4d,0x01,0x52,0xf4,0xe3,0xc7,0x87,0x34,0x54,0xc2, -0x98,0xd2,0x90,0x1f,0x85,0x7e,0x52,0x9f,0x5f,0x57,0x67,0x8b, -0xa0,0x4b,0x84,0xe7,0x79,0xcd,0x05,0x22,0x2e,0x77,0xb8,0xf4, -0x91,0x6d,0x04,0xc0,0x70,0x13,0xcc,0x38,0x95,0xdf,0x58,0xa8, -0x53,0xe7,0x22,0x9d,0xad,0x5c,0x59,0xe5,0xfa,0xfd,0xd6,0xea, -0x52,0xed,0xf6,0xd5,0xa8,0x41,0xac,0x01,0xdb,0xa5,0x8f,0xdf, -0xf9,0x6e,0x9a,0x58,0xd2,0x83,0x3e,0xf4,0x60,0x3a,0xd9,0xbe, -0x92,0x10,0x19,0x2c,0xca,0x05,0x91,0x2a,0x93,0x1e,0xe1,0x51, -0x85,0xe1,0xbb,0xe3,0xd0,0x8b,0xf1,0xe1,0x3a,0xcb,0x97,0x4d, -0x77,0x30,0x66,0x63,0x58,0x7c,0x9c,0x25,0x64,0xbf,0x46,0xd6, -0xe3,0xe1,0xf8,0xae,0xca,0xe5,0x96,0x7f,0x79,0xc2,0x66,0xf1, -0x4f,0x1d,0xf1,0x67,0x71,0x57,0xa7,0x6d,0xc9,0xcd,0x91,0xd3, -0x4d,0x38,0xd9,0xea,0x3b,0xd7,0x69,0x61,0x5f,0xf4,0xa7,0xf7, -0xb8,0x31,0xa4,0x8e,0xed,0xd7,0x54,0x1e,0x99,0xcd,0x60,0x34, -0x3f,0xc4,0x2d,0x5b,0xe2,0x51,0x9f,0x8e,0xe7,0x30,0x9d,0x1e, -0xe6,0xe6,0x8a,0x07,0xc6,0xa1,0x1d,0x73,0xbd,0x61,0x26,0xc8, -0xd6,0x0a,0x56,0xe9,0xbe,0xe4,0xfa,0x94,0x0b,0xd6,0xc8,0x2d, -0x2a,0x67,0x57,0xe9,0x9e,0xb7,0x9a,0x28,0x85,0x34,0xec,0xc5, -0x4c,0x10,0xf5,0x0f,0xb0,0x6e,0x21,0x9b,0xaa,0x3c,0xbb,0x1c, -0xab,0x0f,0xbf,0x25,0xf3,0x5e,0xab,0x17,0x04,0x8c,0xa8,0x5c, -0x7c,0xc0,0x9a,0xca,0xe6,0xe7,0xc9,0x0d,0x0f,0x38,0x19,0x19, -0xce,0x37,0x9b,0x68,0x96,0x60,0x20,0x4a,0x79,0x85,0x13,0xa8, -0x95,0xd3,0x5a,0xb7,0xd0,0x5c,0x76,0x7c,0x4c,0x60,0x7f,0xc5, -0x0e,0x64,0x57,0x8e,0xcb,0xd8,0xba,0xb5,0x77,0xa9,0xb8,0xb5, -0xa9,0xc4,0x1f,0xdc,0xbb,0x66,0xd2,0x00,0x26,0x52,0x70,0x24, -0xa9,0xce,0xde,0xaf,0x24,0x1f,0xd7,0x34,0xa0,0x54,0xa4,0x20, -0xef,0x8f,0x88,0x51,0x76,0x4c,0x14,0xe1,0x35,0xa7,0x16,0x1a, -0xda,0xaa,0x2a,0x25,0x08,0xaf,0xa3,0xd3,0x63,0x7f,0x78,0x64, -0x4e,0xca,0xb0,0xa0,0xe7,0x0b,0x30,0xe1,0x0c,0x2f,0x65,0x7f, -0x63,0xe9,0x10,0xa8,0x1d,0x04,0x72,0x37,0x5f,0x73,0x93,0x4c, -0x70,0xd6,0x78,0xdb,0x97,0x4c,0x71,0x36,0xca,0x3a,0x44,0x19, -0x7e,0x20,0xf8,0xae,0xdd,0x07,0x75,0x0b,0x4c,0x42,0xc9,0xa2, -0xce,0x89,0x9e,0x7e,0xae,0x8a,0x08,0xe7,0x7c,0x2a,0x9c,0x2f, -0x4d,0x4e,0x28,0x8d,0xde,0xd3,0xe2,0x3e,0x7a,0xff,0xa3,0x7e, -0x54,0x0b,0x73,0x5d,0x3d,0xa9,0xa3,0x94,0x4a,0xc0,0xda,0x9c, -0x8d,0xe5,0xbd,0x48,0x0a,0xf0,0x4b,0xf9,0xc1,0x34,0x54,0x84, -0x50,0xd5,0xa9,0xa0,0xf1,0xa3,0x41,0xc8,0x8f,0x82,0x39,0x3d, -0x59,0x3e,0x75,0x8f,0x0b,0x5b,0x9b,0x68,0xca,0x37,0x26,0x8d, -0x26,0x64,0x35,0x46,0x98,0x63,0xc5,0x88,0xd9,0xaa,0x2a,0x82, -0x25,0x13,0xbf,0xd8,0xac,0x7e,0xb7,0x56,0x1f,0xc8,0xb0,0x88, -0x58,0x86,0xe3,0x95,0x55,0xc0,0xd9,0x58,0x85,0x42,0x2a,0x4a, -0x3e,0x58,0xd6,0x28,0xd6,0x74,0x1a,0x88,0x6f,0xc8,0xcc,0xae, -0x74,0x5b,0xa6,0x42,0x97,0xe2,0x52,0x5b,0x0c,0xf1,0x54,0x45, -0xbf,0xf6,0x40,0xc3,0xba,0x3a,0xc9,0x2f,0x9b,0xf6,0xd9,0x55, -0x64,0xe8,0x96,0xf7,0x18,0x61,0xf8,0xe1,0x59,0x90,0x21,0xb0, -0xae,0xf6,0x79,0x26,0x8d,0x5e,0xea,0xb5,0x06,0x27,0x01,0x23, -0xf5,0x68,0x2e,0x4f,0x99,0x82,0x2f,0xfa,0x74,0x6a,0x6e,0x9d, -0x87,0xda,0x87,0x50,0xd3,0x68,0x27,0x69,0x4a,0xaf,0xdb,0xc2, -0xb2,0x8f,0xa8,0xb5,0x66,0x48,0x7b,0x56,0x65,0x9b,0x33,0xf4, -0xeb,0x87,0x55,0xa1,0xca,0x69,0x17,0x9f,0x58,0x95,0x4b,0xa8, -0xca,0x65,0xe0,0x4c,0xaf,0x53,0x1e,0x72,0x68,0x7c,0xc7,0xa0, -0x3e,0x93,0xd2,0x00,0xc2,0x5e,0x8e,0x36,0xd2,0x7a,0x16,0x4b, -0xb6,0x8d,0x66,0x6a,0x8c,0x55,0x74,0x2b,0x05,0x77,0xb5,0xae, -0xf8,0x61,0x25,0xa8,0x3f,0x0f,0x2e,0x32,0x1c,0xd9,0x89,0x72, -0xd3,0x27,0xc6,0x7b,0x27,0x0c,0x86,0xdf,0xb8,0x19,0x83,0x7f, -0x7d,0x95,0x18,0x30,0x1c,0xd9,0x3f,0x2a,0x21,0x68,0x4e,0x2b, -0x52,0x77,0xa1,0x08,0x16,0x88,0xc6,0xfc,0xd2,0x7c,0x2d,0x56, -0x63,0xe4,0xc7,0x8f,0x68,0xa2,0xd6,0x33,0x8d,0x32,0x57,0xd4, -0x55,0xb5,0xc7,0x21,0x1c,0x83,0x8c,0xf1,0x0a,0x86,0x9a,0xc6, -0x2b,0x67,0xd6,0xeb,0x92,0x90,0xbc,0xf3,0x70,0xdd,0x8e,0xc1, -0xbd,0xe5,0x7b,0xab,0x40,0xc3,0x37,0x9e,0x1f,0xcf,0x76,0xe6, -0xc6,0xa7,0x28,0x4f,0x45,0x27,0xf2,0xee,0xd0,0x9f,0x43,0x27, -0x52,0x9b,0xef,0x8d,0x77,0x9a,0xa5,0xa3,0x12,0x5e,0xf7,0xb8, -0xcb,0x20,0xc1,0xab,0x1e,0x77,0x11,0x24,0xe4,0xca,0xa7,0xb1, -0x12,0x12,0xb4,0xfc,0x71,0x16,0x43,0xaa,0x04,0x6a,0x7a,0x47, -0xcd,0x98,0xe1,0x9e,0xde,0x3e,0xa3,0xbf,0x35,0x39,0xc6,0x30, -0x8f,0x82,0x9c,0x55,0x77,0x77,0x3e,0xfc,0x46,0xd6,0x4d,0xc1, -0xb4,0xc0,0x99,0x6a,0x0c,0x3a,0x89,0x9f,0x27,0xef,0xae,0xd2, -0x3c,0xf1,0xbd,0x79,0xbc,0xbe,0x8e,0x0b,0x2f,0x08,0x7c,0xdc, -0xc1,0xc6,0x85,0x6a,0x91,0xe0,0xab,0xf9,0xbe,0x37,0xcb,0x16, -0xb5,0x17,0xe0,0x55,0x44,0xb4,0x49,0xd7,0x80,0xbd,0xb8,0x2c, -0x73,0x08,0xae,0xe2,0xa2,0xf0,0x84,0x77,0x9d,0x2c,0xe3,0x37, -0xe7,0xe9,0x62,0x91,0xac,0x75,0x9a,0xf4,0x7f,0x31,0x50,0x61, -0xb9,0xbb,0x89,0x11,0x05,0xae,0x73,0x7c,0x6f,0x91,0x82,0x5e, -0x15,0xa3,0x33,0xb0,0x75,0xb6,0x4e,0x3c,0xf6,0xc4,0x86,0x27, -0x8d,0xcb,0xa4,0xfc,0x4a,0xd6,0xc5,0x1b,0x2e,0x3c,0x7b,0x2b, -0x0f,0x4d,0xad,0xbe,0x24,0x16,0xf8,0x99,0xc0,0x80,0x60,0x86, -0x98,0x09,0x57,0x5a,0x75,0x96,0xe7,0x63,0x4e,0xe9,0xa1,0xd1, -0x78,0x5f,0xd0,0x7f,0x41,0xe8,0x53,0x24,0x19,0x73,0xd9,0xb9, -0x45,0x26,0x6d,0xd5,0x8f,0x8f,0x71,0x07,0x2a,0xeb,0x65,0x9b, -0x78,0x9e,0xe2,0xb3,0xfb,0x2a,0xe1,0x1f,0x58,0x1d,0x94,0x0b, -0x89,0x35,0xa9,0x36,0x68,0x4b,0xe1,0x24,0x43,0x07,0xe2,0x54, -0x97,0x5e,0xa0,0x54,0x11,0xc9,0x46,0xa0,0xfc,0x5c,0xcf,0x38, -0x9b,0x78,0x96,0x01,0x9c,0x87,0x17,0xb4,0x9a,0x51,0xfc,0x6e, -0xb9,0xcf,0x19,0x02,0x56,0x86,0xf4,0x32,0xc6,0x61,0x85,0x2a, -0xc2,0xba,0xd0,0x9d,0x27,0x71,0xa3,0x68,0xee,0xcd,0x99,0xad, -0x1f,0x67,0xdc,0x7d,0xf1,0x8c,0x07,0x08,0x7b,0x30,0x45,0x88, -0x9c,0xb8,0xc8,0xca,0xa6,0xe9,0x13,0x16,0x55,0x06,0xf1,0x2e, -0x71,0xf4,0xf4,0xcb,0x1f,0x45,0x1c,0x22,0xff,0x2d,0xc4,0xe5, -0x28,0xfa,0x2e,0x71,0x15,0x10,0x06,0x9d,0xa9,0x2f,0x6a,0xf8, -0xa8,0xf1,0xe3,0x26,0xaa,0x3a,0x99,0xda,0x41,0xeb,0x8b,0xf3, -0xa8,0xee,0x64,0x7a,0xe7,0xac,0xaf,0x3b,0xb0,0x91,0x41,0x34, -0x61,0xb9,0x11,0xe7,0xbf,0x97,0xac,0xed,0x88,0x5a,0x7a,0x19, -0x2f,0x9b,0x3c,0xbd,0x89,0x1c,0x12,0x2d,0x02,0x05,0x0c,0x0a, -0x5c,0xa5,0xa0,0xeb,0xab,0xa7,0xac,0x70,0x91,0xd7,0x8b,0x57, -0xe9,0x72,0x3d,0xbe,0x39,0x1d,0xaa,0xdb,0xa3,0x76,0x6c,0x08, -0x0a,0x51,0x8d,0x39,0x6b,0x99,0x13,0x56,0x94,0x8b,0x55,0xc2, -0x30,0x33,0x98,0x44,0x90,0xf7,0xe3,0x73,0xcc,0x2c,0x6f,0xa5, -0x36,0x52,0x00,0xc1,0x1e,0xde,0x54,0x9d,0x1b,0x51,0x77,0xce, -0xed,0x76,0xb8,0x5a,0x35,0x2b,0x54,0x0d,0x44,0x3d,0x68,0x43, -0x30,0xd0,0x0d,0x34,0x50,0x2d,0xa4,0xf6,0x3d,0xaa,0xe1,0x18, -0x7f,0x42,0xc8,0xab,0xa2,0x6a,0x8c,0xaa,0x87,0x61,0x3d,0xf8, -0xc3,0x9a,0x23,0xce,0xe7,0x0d,0xda,0xd9,0x81,0x39,0x56,0x35, -0xad,0x44,0x5a,0x8b,0xac,0x12,0x59,0x2d,0xe6,0x5a,0xb4,0xe6, -0x5a,0xb6,0xd2,0x1c,0xbe,0xd2,0xf5,0x3a,0xc9,0x7f,0x8c,0x17, -0xe9,0x55,0x81,0x71,0x19,0xc6,0x65,0x57,0xa5,0x1d,0x57,0xc4, -0x11,0x91,0xa1,0x5e,0x85,0xa0,0x36,0x39,0x8f,0x57,0x67,0x9b, -0x54,0x24,0x94,0xa6,0x9e,0x7a,0xb0,0x53,0x2a,0xf4,0x17,0x8c, -0x5e,0x8b,0x2b,0xed,0xbf,0xb8,0x56,0x51,0xb5,0x8e,0x22,0x47, -0x40,0x45,0x34,0x29,0x62,0x40,0x35,0x15,0x45,0x54,0xc4,0xdd, -0x22,0xfe,0x17,0xc6,0xa1,0xa7,0xf9,0x8f,0x9f,0xa6,0xc7,0xc7, -0x49,0xfc,0xb4,0xc0,0x89,0x13,0xa7,0x4a,0x4e,0x0f,0x38,0x37, -0xab,0xa0,0x45,0x60,0x79,0x57,0x97,0x09,0x8d,0x19,0x37,0x96, -0xf1,0xa8,0x14,0x54,0x91,0xba,0x2b,0x88,0xaf,0x24,0xa4,0x55, -0x94,0xe6,0x27,0xc0,0x29,0x60,0x13,0xfe,0xad,0x23,0x75,0x69, -0x8f,0x52,0x6b,0x4c,0x05,0x56,0xd6,0x98,0x5a,0x73,0xd5,0xf4, -0xa1,0x1c,0x86,0x90,0xd7,0x59,0x15,0x70,0x5d,0xf5,0x71,0x1c, -0x86,0x54,0x4a,0xed,0xe4,0xa9,0x29,0x0f,0x62,0x0c,0x98,0x19, -0x3a,0x4f,0x4d,0x79,0x28,0xa5,0x45,0x00,0xe7,0x55,0x87,0x8a, -0x9b,0xd7,0x1d,0x42,0x41,0x61,0xc8,0x40,0x61,0x3c,0xde,0xf8, -0x83,0xc4,0x8c,0x2f,0xf7,0x34,0x24,0x2d,0x17,0xa5,0xc8,0x2b, -0x91,0xd7,0xec,0x8c,0x4a,0x49,0x7d,0xc1,0x9b,0x7f,0xe4,0x06, -0x0a,0x9d,0x2d,0xee,0x0c,0x6a,0xa3,0xe2,0x26,0x45,0xab,0xe8, -0x8c,0xaf,0x85,0xe1,0xbe,0x79,0x91,0x78,0xf4,0xd6,0xbc,0x17, -0xe6,0x96,0xcb,0x60,0x44,0x71,0xfa,0x24,0x38,0x1d,0x8a,0x32, -0x7a,0x7c,0x92,0x0b,0xbb,0x27,0x76,0x4b,0x51,0xc3,0xbf,0xaa, -0x53,0x43,0x77,0x2e,0x83,0x11,0x39,0x02,0x1c,0x11,0xa6,0x45, -0x1a,0x5f,0x66,0x78,0x24,0x99,0xd7,0x4d,0x64,0xfe,0xf0,0xa4, -0x8c,0xd7,0x8f,0xfb,0x78,0x44,0x53,0x45,0x79,0xcd,0x21,0x17, -0x31,0x54,0xa9,0xee,0xe6,0x38,0x52,0xe0,0x57,0x27,0xaf,0x1d, -0xe4,0xc5,0xbb,0x2b,0x98,0x95,0xbc,0xb0,0x6c,0xa0,0x06,0x84, -0x51,0x09,0x94,0xba,0xa8,0x10,0x13,0x22,0x42,0x3c,0x0e,0x9a, -0x12,0x37,0x58,0x41,0x16,0xbb,0x8b,0xec,0x66,0x0d,0x94,0x56, -0x4d,0x4a,0xf1,0xeb,0x31,0x20,0xad,0xa3,0xbc,0x3a,0xa1,0xc0, -0x0e,0xf2,0x83,0x74,0xea,0x02,0xd0,0x14,0xf4,0x77,0x45,0xbf, -0x60,0xb7,0x39,0xbb,0x2d,0x25,0x6f,0x09,0x07,0x87,0x78,0xa0, -0x0e,0xc0,0xfe,0x20,0x39,0x25,0x6d,0x4e,0x49,0x29,0xac,0x61, -0x7e,0x64,0xef,0x6c,0x3c,0x4b,0x59,0x93,0x11,0x6e,0x38,0x2f, -0xe8,0xcb,0x9a,0x67,0x28,0x92,0xbe,0x68,0x12,0x53,0x7b,0xd4, -0x7b,0x36,0xa5,0x31,0x49,0xc4,0x6a,0xce,0x12,0x33,0x6b,0xfe, -0x11,0xcb,0x48,0x6b,0xc9,0x7a,0x7f,0x94,0x32,0x45,0x2d,0xbb, -0xf6,0x7e,0x06,0x2b,0x34,0x72,0x7e,0xf6,0x0c,0x31,0x45,0x31, -0x1a,0xf5,0x93,0x77,0x05,0x8e,0xfd,0x52,0x62,0x8d,0x66,0x90, -0x40,0x2f,0x24,0xcf,0x92,0x32,0x9d,0x7b,0xa0,0x1b,0x2c,0xd1, -0x21,0x77,0x71,0x95,0x27,0xaf,0xa8,0xde,0x04,0x0e,0x40,0xa0, -0xe4,0xf2,0x01,0x96,0x35,0xe3,0xc6,0xe3,0xaa,0x1b,0xb9,0xd3, -0x6d,0x0c,0xcb,0x0f,0x8c,0x0c,0x04,0x39,0x7a,0xc0,0xb7,0x66, -0xc6,0x35,0x8c,0xad,0xa7,0x4f,0xec,0x69,0x15,0xe2,0xba,0xd1, -0x79,0x68,0x4d,0xc1,0x10,0x13,0xd5,0x5d,0x98,0x7d,0x3b,0xe7, -0xa7,0x83,0x7e,0x08,0xc9,0x1f,0x9f,0x40,0x26,0xb1,0x67,0xa6, -0x85,0x31,0x89,0x24,0xf1,0xf8,0xf8,0x91,0x6a,0x10,0xdd,0xaa, -0xd2,0x89,0x9e,0x84,0x50,0xef,0x2d,0xd1,0x8b,0x41,0x72,0xd0, -0x90,0x6d,0x23,0x5c,0x31,0x50,0x88,0xc6,0xfd,0x70,0x60,0x4d, -0xe5,0xb4,0xd2,0xf3,0x97,0xaa,0xfd,0xd3,0xf5,0x7c,0x75,0xb5, -0x48,0x5e,0xf2,0xe5,0x2e,0x96,0x02,0x7c,0xd9,0x83,0x5c,0x26, -0xb1,0x37,0x93,0x17,0x7a,0x09,0xae,0x2e,0xf5,0x2e,0xe5,0x65, -0x30,0x9d,0x32,0x52,0x9e,0x28,0x2f,0xc9,0x11,0x52,0xf3,0xfd, -0x0d,0x49,0xd8,0xd5,0x9a,0x7c,0x23,0xa2,0xf7,0xa3,0xb7,0x53, -0xb9,0x7e,0x0a,0xac,0xac,0xcb,0x9e,0xb3,0x80,0x6d,0xcd,0x2d, -0x61,0x6c,0x04,0xb8,0xe7,0xda,0xa8,0x85,0xc1,0x29,0x49,0x3e, -0x88,0x53,0xdd,0x04,0xb6,0x70,0xf2,0x40,0xbd,0x54,0x77,0x88, -0x54,0x50,0xdd,0xcc,0x91,0xd9,0xe3,0xc5,0xc2,0x97,0x30,0x42, -0x25,0x2a,0x8b,0xf5,0x65,0xaf,0x32,0xf9,0x6a,0x27,0x0b,0xae, -0x7f,0xb6,0xee,0xc4,0xa5,0x1f,0x2c,0x05,0x7c,0xd4,0xa6,0x4b, -0xa7,0x4d,0xb7,0xce,0x11,0x8e,0xa5,0x38,0xd3,0x16,0x31,0x3e, -0x12,0xca,0xab,0x4f,0x5a,0x8d,0xde,0xdd,0x5d,0x26,0xe5,0x79, -0x86,0x57,0x9b,0xf5,0x16,0xbc,0xda,0xdd,0x94,0x1b,0x59,0x8c, -0xf5,0x0d,0x6e,0xca,0x80,0xf2,0x8e,0x51,0x13,0x8f,0xe3,0xd4, -0x46,0x8d,0xdc,0x29,0xff,0x52,0xf2,0x83,0x1e,0xda,0x89,0xac, -0xbc,0x62,0xc3,0xc1,0x46,0x3e,0x3d,0x1f,0x1f,0x1f,0x6f,0x66, -0xbd,0xf9,0x2a,0x89,0x73,0x3f,0x90,0x6c,0x9e,0x05,0x76,0xe1, -0x44,0x2a,0xd7,0x61,0x36,0xd6,0xa0,0xa1,0x53,0x2e,0x55,0x4c, -0x40,0x9f,0xde,0xc1,0x65,0x21,0x72,0xb0,0x1a,0x46,0x61,0x8d, -0x25,0xa3,0xe8,0xec,0x63,0x66,0x30,0xde,0x4a,0x0a,0xf8,0x0f, -0xb0,0x0b,0x39,0xc4,0x81,0xe3,0x63,0x2b,0xa0,0xca,0xbc,0xbb, -0x73,0xa8,0x1a,0x99,0x5d,0x39,0xda,0x30,0x76,0x18,0xcc,0xaf, -0x05,0xab,0x06,0x30,0x2c,0x2f,0xac,0x73,0x17,0x94,0x58,0x7d, -0x45,0x89,0x2d,0x78,0xf5,0x15,0x25,0x32,0xf8,0x55,0x4b,0xa2, -0x1d,0x56,0xd1,0x3a,0x49,0xc9,0x28,0x59,0x91,0xd8,0x7b,0x7c, -0x4e,0xef,0x31,0x5b,0x7c,0x8e,0xac,0x37,0xe4,0x47,0x1f,0x33, -0x10,0x63,0xe4,0x35,0x59,0x87,0x3d,0xea,0xa5,0xdd,0x82,0xee, -0x5b,0xf0,0xb6,0x43,0x8f,0xdf,0x49,0xc5,0x48,0xa1,0x97,0xf1, -0xcd,0x64,0xc9,0x70,0xb5,0x3e,0x6d,0x26,0x63,0xbc,0x50,0xeb, -0xc3,0x66,0x22,0xcd,0x10,0xac,0xc0,0xaa,0xb7,0xd2,0x60,0xee, -0xa5,0x39,0xd8,0x9a,0x51,0x1f,0xe3,0x7b,0x90,0xa0,0x9b,0x44, -0xca,0x3b,0x84,0xff,0xb8,0x6f,0x8f,0x8f,0x30,0x67,0x9c,0xf1, -0x69,0x9e,0x90,0x0d,0x12,0xdd,0xf2,0x55,0x1d,0xde,0x30,0x63, -0x8d,0x2d,0xe4,0x3f,0x82,0x56,0x6d,0x21,0xfd,0x0a,0x5c,0x77, -0x86,0xf8,0x23,0x70,0xe5,0x13,0xe2,0x0f,0x54,0x64,0x1e,0xc2, -0x3f,0x72,0x37,0x1a,0x92,0xaf,0x6a,0xb9,0xe2,0x85,0xce,0x8f, -0x95,0x09,0x75,0x8d,0xe8,0x6d,0x49,0x65,0x52,0x72,0x8b,0x2c, -0x0d,0x8d,0x34,0x92,0x34,0x84,0xa6,0x19,0x2c,0x7c,0x16,0x6d, -0x5b,0x67,0xab,0x29,0x59,0xd3,0x05,0xd1,0xb6,0xad,0x26,0x32, -0x5c,0x23,0x28,0xdc,0x32,0x14,0x34,0x8e,0x08,0xdc,0x4e,0x4a, -0x8a,0x92,0x05,0x4f,0x6f,0xc8,0xc8,0xd8,0xe3,0x63,0x8a,0x1e, -0x5f,0x6d,0x16,0xe8,0x21,0xda,0x6c,0x06,0x1e,0xc6,0x12,0x32, -0x11,0x3b,0xf0,0x7b,0x72,0x05,0x9c,0x60,0xfa,0x23,0x17,0x77, -0x1f,0xa9,0xdc,0x55,0xf4,0x66,0x22,0xc7,0x5a,0x1d,0x8e,0xc6, -0x0b,0x79,0x6b,0x1b,0xcf,0x8b,0xe9,0x77,0xf4,0x80,0x4e,0x40, -0x9b,0xf9,0xee,0x76,0xa7,0x75,0x42,0x29,0x2f,0xf9,0xf0,0x09, -0x17,0x1d,0x5d,0x9b,0x34,0xf7,0xe8,0x9a,0x92,0x23,0x79,0x70, -0xcd,0x47,0xd6,0xb2,0x0a,0x53,0x4a,0x02,0x7d,0x01,0xff,0x30, -0x9b,0xae,0x97,0x36,0xc1,0x7c,0x79,0xb2,0xb9,0x81,0xc8,0x87, -0x1c,0x7c,0x38,0x60,0xb8,0xc5,0xcc,0xf6,0xa5,0x9c,0xee,0x61, -0xf1,0x2d,0x43,0xd1,0x2c,0x6c,0xb5,0x8b,0x95,0x49,0xee,0xde, -0xec,0x6b,0x1d,0x73,0xe8,0x25,0xcf,0xd9,0xc6,0x8c,0x91,0x4e, -0x22,0x5a,0x25,0xc3,0xaa,0x89,0xc6,0x15,0x36,0x6b,0x84,0x58, -0xb9,0xa1,0x77,0x6a,0xc4,0x33,0xf4,0xc3,0xea,0xf5,0x61,0x54, -0x88,0xdf,0xc0,0x8c,0xfd,0x15,0xa0,0xc9,0x1c,0x0f,0x9f,0x9a, -0x15,0xe1,0xc3,0x1a,0x43,0x12,0x9f,0x8b,0x36,0x2b,0xb2,0x73, -0x7e,0xa3,0xc8,0xa1,0x8b,0x3e,0xe4,0x73,0x52,0x6b,0x58,0xf4, -0xf5,0x35,0xa0,0x02,0x75,0x04,0xff,0x71,0x24,0x7c,0x40,0x9c, -0x23,0xe1,0x56,0x65,0x76,0xe5,0x5c,0x9f,0xa2,0xd8,0x60,0x28, -0xf2,0xf2,0x0e,0x92,0x7d,0x00,0x83,0x5b,0xd2,0x56,0xd0,0xf8, -0x94,0x6c,0x3a,0x88,0xa3,0xba,0xa9,0x73,0x9f,0xd0,0x0d,0x0a, -0x93,0x8b,0x7c,0x69,0xd9,0x37,0x9e,0x7c,0x93,0xa6,0xea,0x4c, -0xc6,0x08,0x6f,0xe8,0x50,0x54,0xdd,0xe5,0xc1,0xeb,0xa2,0x15, -0x39,0xe9,0x62,0xa5,0x93,0x48,0x62,0x1f,0x04,0xbe,0x09,0xa0, -0x6f,0x2c,0xc3,0x2c,0x0d,0xa5,0xd5,0x53,0xbf,0x11,0x43,0xf0, -0xd7,0xb6,0x07,0x50,0x5f,0x95,0xc4,0x8c,0x77,0x0e,0xb3,0x28, -0x6a,0xd7,0x00,0x2b,0x46,0xdf,0xbd,0xb2,0x5d,0xa2,0x18,0x87, -0x02,0xdf,0x22,0x02,0x1d,0xec,0xb8,0x47,0xf2,0xbe,0x6a,0x43, -0xab,0x9a,0x36,0xb8,0x9b,0xd9,0x3e,0x65,0x12,0x3b,0xc3,0xab, -0x75,0xca,0xd7,0xb3,0x73,0x69,0x39,0xe1,0xf1,0x84,0xfc,0x63, -0xa8,0xa3,0x31,0xdb,0x12,0x65,0xf7,0x80,0xec,0x60,0x11,0xfa, -0x58,0x4c,0x9f,0x1f,0x62,0xc0,0x95,0xe7,0x91,0x6d,0xab,0xb0, -0xaf,0xaf,0x60,0x7e,0xa9,0x51,0x73,0xe3,0xae,0x6c,0xad,0xfc, -0x41,0x42,0xdc,0x2a,0xc0,0x80,0x46,0xd6,0x4f,0x7b,0xa2,0xf0, -0x9d,0x30,0xb5,0xb8,0xea,0x38,0x0e,0xa8,0x2b,0x22,0x56,0x9c, -0x2b,0x24,0xca,0x2f,0x91,0x45,0x34,0x75,0x46,0x57,0x52,0x20, -0x66,0x57,0x50,0x56,0x28,0x28,0xdc,0x57,0xa3,0x95,0x94,0x13, -0x45,0x4a,0x8b,0x98,0x30,0xa4,0x2d,0x25,0x16,0xb0,0x93,0xb3, -0x45,0x46,0x78,0x0c,0xdd,0xd3,0x8c,0x56,0xe5,0x24,0xd3,0x6c, -0x51,0x19,0x3d,0x64,0xa4,0x6b,0x0e,0x75,0x6c,0xcb,0x20,0xed, -0xa3,0xf6,0x0c,0xa1,0xb7,0x8d,0x29,0x1a,0xe7,0x40,0x39,0x93, -0xba,0xf3,0xe7,0xdd,0xdd,0xf3,0x6f,0x7f,0x78,0xf5,0x8b,0x3c, -0xf7,0x27,0x88,0x1e,0x9b,0x3f,0xb0,0x82,0x20,0xa3,0x38,0x20, -0x92,0x2a,0x2d,0x15,0x10,0x7c,0xf2,0x6d,0x72,0x59,0x22,0xbe, -0x81,0xa8,0xe7,0x62,0x3d,0x09,0x07,0x0f,0xd0,0x05,0x76,0x26, -0x68,0xa3,0x13,0x28,0x4e,0xd9,0xba,0xf5,0x28,0xfd,0xc2,0x41, -0xb4,0xab,0x4e,0x34,0x8c,0x03,0xf0,0x84,0x11,0x46,0x33,0xaa, -0xd6,0xf1,0x31,0xfd,0x69,0xd1,0x07,0x5c,0x7b,0x06,0x3e,0x80, -0x74,0xe2,0x1e,0x29,0x3b,0x03,0x66,0x86,0xfa,0x7b,0x40,0xb5, -0x68,0x35,0x52,0x40,0xc6,0x01,0x15,0xf0,0x7b,0x20,0x27,0xfd, -0x39,0x3e,0x7e,0x84,0x60,0xe3,0x3d,0xee,0x1d,0xa8,0xe1,0x82, -0x90,0x52,0xef,0xee,0x64,0xe5,0xbb,0xdd,0xa9,0x34,0x07,0x43, -0xd3,0xed,0x3f,0xe8,0x10,0x96,0x8b,0xb6,0x9e,0x0a,0x41,0xad, -0xd7,0xd1,0x90,0xe9,0x50,0xbe,0xf1,0xd0,0x8a,0xf3,0x50,0x15, -0x9d,0xd9,0xcb,0x53,0x78,0x41,0x67,0xec,0x6c,0x02,0x69,0x8e, -0xe9,0xe5,0x07,0x1b,0x5b,0x53,0x1a,0x9d,0xcd,0xf3,0x61,0xbd, -0x3a,0xa1,0xa7,0x9c,0x74,0xa1,0x93,0x82,0xde,0xfc,0x6a,0x96, -0xce,0xbb,0xe9,0xba,0x9b,0x5d,0xd1,0x0b,0xf4,0x98,0x8d,0x9b, -0x09,0xd6,0x1a,0x68,0xe3,0x41,0x07,0x85,0x5b,0xfb,0xc5,0xb5, -0x64,0xe3,0x27,0xab,0x78,0x53,0x24,0xd6,0x19,0x38,0xdb,0x02, -0xe1,0xe1,0xbc,0x38,0x13,0x09,0x9d,0x4e,0x90,0x0d,0x9d,0x8d, -0x8f,0xad,0x80,0x30,0x9a,0xdf,0xfb,0x20,0xa3,0x95,0x35,0xbd, -0x99,0xa3,0xea,0xe0,0xd4,0x48,0x14,0x25,0xf4,0x8e,0x47,0x7d, -0xb9,0x97,0x81,0x79,0x46,0x84,0x82,0xde,0x1a,0x71,0x51,0x04, -0xd2,0x6e,0x8d,0xd2,0x7a,0x86,0x1a,0x56,0x83,0xe9,0x13,0x37, -0x25,0xce,0x22,0x45,0x7a,0x97,0x6d,0x13,0x4e,0x75,0x61,0xfd, -0xa7,0x67,0x01,0x17,0x38,0xe0,0x6b,0x15,0x58,0xb5,0xb3,0xa7, -0xe8,0xbb,0xe4,0x2c,0x1a,0x80,0x76,0xc6,0xa8,0x89,0x6b,0x67, -0xea,0xe9,0x0b,0x8a,0x6a,0x1e,0xf9,0xe3,0xa6,0x02,0x26,0xe0, -0xe0,0x27,0xc7,0x8d,0x7a,0x1a,0xc9,0x18,0xe9,0x1a,0x64,0x8f, -0x02,0x3d,0x1a,0x80,0xd8,0x9f,0x8d,0x7d,0xc2,0xcb,0x52,0xc9, -0xab,0x1d,0x2d,0xa2,0x42,0x57,0x99,0x0b,0x27,0xfe,0x69,0x5e, -0x04,0xa1,0xac,0xc2,0xd6,0x32,0xad,0xec,0x19,0xff,0xa6,0xc4, -0xd2,0x86,0xad,0x2c,0x77,0x96,0x03,0xc6,0xb2,0x56,0x3f,0x32, -0xb2,0x49,0x2c,0x96,0x37,0x3b,0xf5,0x23,0x25,0xd4,0x30,0x44, -0x11,0xe7,0x61,0x19,0xa0,0x93,0x66,0xe4,0x26,0x80,0xa0,0x11, -0x8e,0xcc,0x45,0x44,0x13,0xb3,0xe8,0xbd,0x0f,0xaa,0x92,0x7c, -0x09,0x97,0x22,0x30,0x1d,0x46,0x10,0x8c,0x87,0x36,0xf0,0x70, -0x81,0x29,0x1f,0xfd,0x18,0xeb,0x5c,0x94,0x1a,0xf2,0xbd,0x9d, -0x0d,0x5a,0xa1,0xd8,0x7d,0x5e,0x53,0x13,0x48,0x72,0x74,0x6b, -0x90,0x3d,0x15,0x3d,0xb3,0x08,0x73,0x2a,0xd1,0x08,0xd3,0xe8, -0xff,0xcf,0xde,0xf7,0xff,0xb6,0x71,0x2b,0x89,0xff,0xfe,0xf9, -0x2b,0x92,0x6d,0x90,0xee,0x7e,0x44,0xcb,0x52,0x52,0x1f,0x70, -0xeb,0x6e,0x85,0x5e,0x9b,0x7e,0xc1,0xb5,0xf7,0x8a,0x26,0xaf, -0xef,0x00,0xc3,0x08,0x14,0x6b,0x15,0xa9,0x91,0x25,0x9f,0x24, -0xcb,0x72,0xe4,0xfd,0xdf,0x8f,0xf3,0x8d,0x1c,0xee,0x72,0x65, -0x39,0x75,0xd1,0x03,0xde,0x2b,0x50,0x47,0x4b,0xce,0x0c,0xbf, -0x0f,0x87,0xc3,0xe1,0x4c,0xc6,0x56,0x4e,0x08,0xaa,0x6e,0x7a, -0xc9,0x0c,0x00,0xe7,0x2d,0x8f,0x85,0x63,0x50,0xc1,0x1a,0xe7, -0xc8,0x95,0xb8,0x23,0xd3,0x8c,0x25,0x14,0x8a,0x67,0xe9,0xf8, -0x2d,0xf6,0x09,0xad,0x4c,0x7f,0x44,0x83,0xcf,0xbb,0x3b,0xb7, -0x66,0x09,0x06,0x7b,0x44,0x2f,0x20,0x1a,0xec,0x46,0x0a,0x96, -0x43,0x4b,0x37,0xb0,0xc1,0xc5,0x78,0xef,0x6e,0xc0,0x94,0x2f, -0x5e,0x5e,0xb3,0x44,0x9c,0x56,0xe5,0xda,0x2d,0xd2,0x70,0x5d, -0x1d,0xb4,0xf6,0xdc,0x9a,0x6b,0xef,0x19,0x35,0xa1,0xed,0x8a, -0xce,0x4e,0xd7,0x6e,0x52,0x16,0xee,0x2d,0x0a,0xbf,0xe8,0x52, -0x11,0xba,0x1c,0x9b,0x51,0xe1,0x76,0x58,0x6c,0x30,0x92,0x07, -0xef,0x18,0x70,0x0a,0x07,0xec,0x74,0x1d,0x61,0xa7,0x76,0xa9, -0x33,0xe3,0x8c,0x19,0xd3,0x7a,0x06,0xec,0x01,0x95,0xc1,0x2a, -0xc8,0xac,0x2d,0x01,0x5e,0x41,0x17,0xee,0x85,0x6c,0x64,0x97, -0x81,0x2b,0xfc,0xb7,0x20,0x4d,0xa9,0x07,0xfa,0xb6,0xd8,0xd7, -0xf8,0x24,0x78,0x31,0x1b,0xd1,0x0f,0x7c,0xb9,0x74,0xea,0xde, -0x03,0x15,0xfa,0x71,0xfe,0x20,0x15,0xf8,0x62,0x87,0xa9,0x79, -0xed,0x9d,0xbc,0x21,0xb3,0xe7,0xbc,0x7b,0xd2,0xa9,0x3d,0x54, -0x4e,0xb3,0xe3,0x17,0x95,0x2b,0xa5,0x10,0x3a,0x59,0x7e,0x38, -0x49,0x85,0x1e,0x07,0x15,0xcb,0x38,0x85,0x92,0x51,0x7b,0xec, -0x1c,0x78,0x0b,0xcd,0x47,0xec,0x5f,0xf1,0xdd,0x1c,0x75,0xc2, -0x29,0x39,0xae,0x21,0x0f,0xd7,0x2b,0x01,0x43,0x97,0xda,0x29, -0xbc,0xd7,0x11,0x6f,0xdc,0x8d,0x1c,0xe7,0x79,0xbb,0x9e,0xe3, -0xbc,0x6f,0x07,0x39,0x94,0x92,0xca,0x23,0x4a,0xc9,0xfa,0x96, -0xde,0x4a,0x02,0xd6,0x7a,0x56,0x7a,0x04,0x70,0xfc,0x98,0x9d, -0xba,0x7a,0x29,0x91,0x8f,0xe4,0x3b,0xbe,0x94,0x2a,0x28,0xbc, -0x54,0xdd,0x8f,0x3f,0xe0,0xa1,0x43,0x12,0x98,0x33,0xbe,0x72, -0xaf,0xe8,0x0c,0x4a,0x1e,0x29,0x8d,0x23,0xee,0x87,0xde,0x4d, -0x86,0xfe,0x71,0x2f,0xdb,0x83,0xea,0xbb,0x25,0x82,0x0b,0xed, -0x47,0x6f,0xf6,0xf8,0xdc,0x62,0x1f,0x15,0xd7,0x85,0x6d,0x54, -0x00,0x20,0xa0,0x82,0xbd,0x18,0x52,0xf1,0xdd,0x7d,0x5f,0x5d, -0x8c,0x0a,0x1f,0xe0,0x28,0xd2,0x00,0x84,0x24,0xe5,0x29,0x37, -0x4c,0x13,0x44,0x7a,0x35,0x1f,0xd5,0x1a,0x63,0xc7,0xa7,0x5e, -0x8d,0xb5,0xac,0x1d,0x13,0x84,0x51,0xf0,0x67,0xff,0xe8,0x68, -0xb2,0x68,0x2e,0x79,0xf8,0x54,0x46,0x23,0xf9,0xae,0x8e,0x60, -0xb9,0xcc,0x26,0x9a,0xeb,0xdb,0x18,0x9a,0x64,0x36,0xd0,0x7c, -0x67,0x46,0xd0,0x5c,0x66,0x03,0x8d,0x9f,0x80,0x37,0x51,0x28, -0x23,0x52,0x8c,0xed,0xa0,0x68,0x09,0x36,0x9d,0xa0,0xdd,0x83, -0x0b,0x7b,0xaa,0xf4,0x33,0x55,0x4d,0x3c,0x35,0x7b,0xd4,0x14, -0xe0,0xa1,0x43,0x42,0xe7,0xa7,0x31,0x4f,0x8e,0xca,0xe0,0x3c, -0x7f,0xda,0x37,0xbe,0x12,0xf9,0x0e,0x57,0x56,0xce,0x63,0xfc, -0x77,0xaa,0x12,0xd5,0x2c,0x4c,0xac,0xf8,0x1d,0x1f,0x99,0xce, -0xc6,0xdc,0x34,0x3a,0xaf,0x4a,0x86,0xa6,0x47,0x3d,0x6e,0x04, -0x4f,0x1a,0x7e,0x66,0xd2,0xd3,0x33,0xa6,0x19,0x62,0xc2,0x65, -0x19,0xcd,0xd6,0x0d,0x86,0x0d,0x29,0x38,0x6a,0x08,0x4e,0x20, -0x88,0x04,0x4c,0x7c,0x04,0x01,0x82,0xc9,0xdf,0x16,0xb9,0x22, -0x5c,0x68,0x7b,0xa1,0x78,0x15,0xec,0x85,0xe1,0xe5,0x55,0x87, -0xe1,0x07,0x90,0xc6,0x87,0x13,0xf1,0x55,0xa4,0xe0,0x21,0xfe, -0xdb,0x45,0x05,0xc1,0x7e,0xb9,0xf8,0xf0,0xf5,0xf2,0xfd,0xf5, -0xa5,0xed,0x2d,0x3b,0x11,0x22,0x25,0xaf,0xce,0xc3,0xb5,0x03, -0x2f,0xf7,0xc2,0x89,0x0a,0x29,0xe1,0x42,0xc1,0x94,0x60,0x0d, -0x10,0x96,0x4c,0x3e,0xf8,0x52,0x33,0x17,0x3e,0x71,0xc7,0xdd, -0xa9,0x78,0x03,0x70,0x34,0xb0,0x23,0x51,0x7b,0xb4,0x03,0x49, -0x3c,0x48,0x03,0xfe,0x37,0x77,0x0a,0xaf,0x70,0xf7,0xa5,0x3a, -0xf1,0xeb,0x28,0xdf,0x2f,0xec,0x09,0x84,0x22,0x5a,0xa8,0xdf, -0x1c,0x35,0x63,0xe5,0xd8,0x9a,0xeb,0x16,0x79,0x1e,0xce,0xcf, -0xee,0xed,0x91,0xd2,0x11,0xa3,0x56,0xba,0xbd,0xcf,0x85,0x5b, -0x21,0x2a,0x58,0x81,0xb0,0xcb,0x9d,0xad,0x31,0xbe,0x4c,0xb0, -0x23,0x86,0x7a,0x2a,0xaa,0x29,0xfe,0x8d,0xe6,0xa3,0xcd,0xf2, -0xe5,0x5a,0xb5,0x85,0xc6,0x54,0xd5,0xbf,0x25,0x61,0x5f,0xab, -0xc8,0x56,0x21,0xf7,0xc0,0x54,0x8f,0x68,0x2c,0x03,0xf6,0x39, -0x5c,0xd8,0x5a,0xa4,0xa3,0xae,0xb8,0x0a,0x41,0xb9,0x12,0x1f, -0xe8,0xe0,0xd8,0x0e,0xce,0x88,0x2d,0xc4,0xda,0x90,0x9f,0xf9, -0x38,0x52,0x4e,0x63,0xd7,0x3b,0xaf,0xf9,0xb8,0xf5,0x16,0xc8, -0x76,0xca,0x0d,0xb0,0x3a,0x16,0xb1,0x52,0x4a,0xbe,0x7e,0x2b, -0x0a,0x42,0x6b,0xd0,0x17,0xed,0xa0,0x30,0x6a,0x1a,0xf4,0x25, -0x83,0x36,0xca,0x51,0x30,0x5f,0xb4,0x91,0xb3,0xb0,0x9a,0xd6, -0x49,0x6b,0xb1,0x6b,0x7a,0xe6,0x39,0x72,0xbe,0xcd,0xf8,0xfd, -0x9b,0x72,0x70,0x5d,0xc8,0x0e,0x59,0x72,0xd8,0xa2,0x02,0xff, -0xc2,0xf7,0xff,0xa3,0x45,0x42,0xc2,0x58,0x34,0x12,0xa8,0x0c, -0xad,0x84,0x30,0xa0,0x61,0x87,0x98,0x70,0x60,0x95,0x8e,0x68, -0x5b,0xc3,0x12,0x2c,0xa9,0x76,0x79,0x6a,0x57,0x3a,0xec,0xd2, -0x61,0xa4,0x09,0x96,0x69,0x33,0xe2,0x16,0xcf,0x7b,0xb4,0x16, -0xfe,0x86,0x49,0xab,0xc1,0xb6,0x93,0x24,0x75,0xe9,0x89,0xdb, -0x58,0xab,0x09,0xa5,0x56,0x2a,0x6e,0xca,0x61,0x35,0x41,0x50, -0xae,0x08,0xa1,0x35,0x1a,0x89,0xc9,0x95,0x8f,0xdd,0xb2,0x8a, -0x0c,0x4e,0x8c,0xb0,0xe6,0x8d,0x2e,0xdf,0xd1,0x54,0xb9,0x55, -0x3d,0x98,0xd3,0xa1,0x55,0x77,0xf0,0x5b,0x45,0x96,0x92,0xaa, -0x7a,0xf8,0xa7,0xc3,0x69,0x32,0xbc,0xa6,0x49,0x49,0x8a,0x66, -0x18,0x27,0x71,0x6b,0xe8,0xcd,0xc4,0xd3,0x3a,0x41,0xf7,0x6a, -0x53,0xef,0x75,0xb8,0xf0,0xe7,0x45,0x1d,0xf6,0xa8,0xcf,0x3c, -0xb7,0xb3,0x65,0xc6,0x38,0xff,0xaa,0x3f,0xe8,0xdc,0xe6,0xa1, -0x94,0x68,0x05,0x14,0x9b,0xd1,0x1b,0x74,0x1c,0xfe,0xd9,0xfc, -0x3c,0x84,0x91,0x00,0x2b,0x41,0x22,0x86,0xee,0xfb,0x1d,0x7c, -0x91,0x05,0x5b,0x2a,0x24,0xc3,0x6f,0x4a,0xe6,0x3d,0xd4,0x26, -0xda,0x62,0xb2,0xe7,0xcf,0x79,0x26,0xd4,0xf6,0x6a,0xcf,0x99, -0xfd,0xce,0x8c,0x44,0x82,0x7d,0x18,0x9c,0xe7,0x43,0x7f,0x55, -0x91,0x88,0x5a,0x87,0x0f,0x86,0x47,0xe9,0xe8,0x01,0x71,0xc9, -0x95,0x8e,0x64,0x76,0xe0,0xe2,0x73,0x82,0x8c,0x8a,0x42,0x39, -0xd8,0xe6,0xae,0x00,0x02,0xa8,0x1a,0x41,0xbe,0x0e,0xaf,0x35, -0x23,0xd8,0x7e,0xec,0xd0,0xc2,0xf2,0x34,0x3a,0x91,0xf5,0xe5, -0x72,0xab,0x46,0xa4,0xb4,0x07,0xac,0x62,0x42,0xf0,0x65,0x7a, -0x1a,0xb1,0x32,0x5d,0x6e,0xa5,0x02,0xbe,0x1d,0x56,0x1a,0x82, -0x32,0xcf,0x20,0xb4,0x06,0x7d,0x4c,0xae,0x7c,0x80,0xa6,0xc3, -0x08,0x03,0x24,0xd3,0x45,0xa4,0x06,0x59,0x48,0xad,0xda,0xc2, -0x3e,0x1d,0x5e,0x86,0x13,0xc7,0x54,0x61,0x5e,0x44,0xdb,0xea, -0xd2,0x5c,0x32,0x17,0x1b,0x89,0x8c,0x75,0x58,0xb9,0xa1,0x8c, -0xc7,0x05,0xd7,0x04,0x3f,0x57,0x72,0x98,0x5e,0xb5,0x86,0xd9, -0x3a,0xb0,0xe8,0x40,0x98,0x94,0xa2,0x43,0x09,0xd3,0x17,0x1d, -0xa4,0x57,0xad,0x21,0xbb,0x0e,0x5f,0x0c,0x5e,0xd6,0x75,0xfb, -0x4c,0x20,0x00,0x6f,0xf5,0x42,0xf0,0xe9,0x55,0x34,0x06,0xd8, -0x03,0xd6,0x43,0x58,0xa4,0x93,0x9e,0xb7,0xc1,0x1a,0xd0,0x45, -0xd5,0x43,0x84,0x1d,0x56,0x96,0x12,0xc5,0xb9,0x30,0x2d,0x9c, -0xbb,0xd2,0x54,0x22,0x17,0x87,0x73,0x3b,0xdc,0x4b,0x71,0xb2, -0x53,0x76,0x85,0xdb,0x44,0x28,0x0f,0x14,0xe8,0xa9,0xbe,0xcf, -0x2e,0xea,0xfb,0xe2,0xa2,0xbe,0x4f,0xae,0xeb,0xfb,0x4e,0x9d, -0x42,0x76,0xb3,0x3b,0x09,0x54,0xc6,0x46,0xaf,0x14,0xc2,0x4c, -0x3e,0x28,0x6a,0x19,0xd9,0xbe,0x4a,0xd8,0x32,0xb4,0xb2,0x75, -0x54,0x9c,0x9d,0xad,0x23,0x84,0xe1,0xcf,0x90,0x4a,0x18,0xfb, -0x8c,0x4d,0x61,0x85,0x0c,0x7f,0x56,0xfe,0x02,0x92,0x2e,0x2a, -0xf7,0xea,0x04,0x09,0xa4,0xa9,0x15,0x1c,0x4d,0x57,0x17,0xb6, -0xef,0xcb,0x94,0xbc,0x6f,0xb2,0xff,0x57,0xa5,0xfe,0x63,0x9b, -0x3a,0x72,0x25,0xf8,0xb1,0x6c,0x24,0xcd,0x74,0xd2,0x7a,0x62, -0xcb,0x9b,0x2c,0x66,0x23,0x49,0x6b,0xf8,0xc8,0xfc,0x36,0x54, -0x40,0x8e,0x57,0xac,0x35,0xf1,0xc2,0x66,0xb1,0x78,0x4b,0x95, -0xa5,0xe3,0x12,0x6c,0xca,0x68,0xb7,0x6f,0xc0,0x99,0xb7,0x21, -0xcd,0x17,0x7b,0x2d,0xe1,0x57,0x94,0x9b,0xc7,0x3a,0x3b,0xf1, -0x75,0x40,0xec,0x88,0x60,0xfc,0x11,0x67,0xfc,0x78,0xc7,0x1b, -0x22,0x05,0xfe,0x83,0x8a,0x13,0xd1,0x0d,0x8e,0x5e,0xb2,0xa7, -0x49,0xf5,0x3e,0x3c,0xc3,0x10,0x93,0x83,0x94,0xb4,0x99,0x98, -0x11,0xb8,0x55,0xf4,0x0b,0x28,0xb4,0x91,0x4f,0xb7,0x74,0x1c, -0x02,0x0f,0x37,0xbc,0x2f,0xb3,0x03,0x43,0x71,0xb0,0x85,0x04, -0xf1,0x6f,0xc3,0x05,0xe8,0xd0,0xbc,0x33,0x53,0xe3,0xa3,0x5a, -0xd3,0x03,0x52,0x78,0x96,0x5e,0xd8,0x3f,0x47,0xfd,0xf3,0xce, -0x47,0xfc,0xe7,0xf8,0x45,0x07,0x82,0x3e,0x18,0xc8,0xe9,0x14, -0xef,0x8e,0x5f,0x18,0x7b,0x8e,0x00,0x97,0x71,0x58,0x47,0x1f, -0x9e,0x0a,0xc2,0x1f,0x72,0x25,0x7c,0x62,0xea,0x2b,0xec,0x05, -0xf1,0x20,0xf0,0x0b,0xea,0xde,0xa4,0xa6,0x2b,0xb0,0xdb,0x07, -0x37,0x97,0xe9,0x78,0x55,0xcc,0x1c,0x33,0xf3,0x0e,0x5e,0x9e, -0x3f,0x1f,0xaf,0xc8,0xed,0x0d,0x54,0x4a,0xdb,0xd2,0xeb,0xf0, -0x34,0xdf,0x31,0x38,0xc0,0x30,0x65,0xd6,0x62,0x05,0xdd,0xaa, -0xc3,0x4d,0xab,0x2a,0x53,0x23,0x6c,0xb3,0xa7,0xff,0x9f,0xba, -0xb1,0xca,0xf8,0xb8,0xb9,0x32,0xab,0x8f,0x05,0xab,0x3b,0x4e, -0x91,0x01,0x02,0x73,0x5c,0x15,0x9e,0x17,0x7a,0x5b,0xfe,0xd5, -0xc7,0x4e,0x71,0xd2,0xb1,0xb9,0xcf,0x9f,0xaf,0xb9,0xca,0x91, -0xea,0x22,0xa6,0x54,0x97,0x5f,0x76,0x73,0xe8,0x52,0x33,0xe7, -0x81,0xab,0xdd,0xfa,0x91,0x8b,0x34,0x18,0x8b,0xd5,0xc7,0x53, -0xef,0xe1,0x7b,0x07,0x4f,0xc4,0x81,0x09,0xbd,0xc7,0x58,0xf1, -0xa4,0x16,0x73,0x7e,0xbf,0xc8,0x9b,0x69,0xfe,0x54,0x5c,0xfa, -0xe5,0x5a,0x0d,0x9a,0xe3,0xdf,0xca,0xd0,0x71,0x3a,0x0d,0xce, -0xd3,0xb9,0x0a,0x56,0x12,0xe8,0x05,0x94,0x2b,0xa2,0x43,0x4f, -0xed,0x66,0x14,0x9e,0x3d,0x2b,0x6d,0x11,0x55,0x9c,0x51,0x18, -0x5e,0x73,0xc8,0x11,0x5d,0x9d,0x66,0xef,0x3b,0x9a,0xd7,0x21, -0x5f,0xec,0x3b,0x6e,0x2b,0x7e,0x76,0x00,0x97,0xe2,0x68,0xa9, -0xeb,0x19,0xdd,0x2d,0x10,0x02,0xab,0xf7,0xd9,0x62,0x56,0x65, -0xbc,0xa6,0x04,0xb0,0x76,0x71,0x17,0x08,0x1b,0x32,0x50,0x91, -0xfb,0x03,0xef,0xea,0x2a,0xc0,0x61,0x65,0x34,0x53,0x34,0x6d, -0x3c,0xb3,0xa6,0x82,0x8d,0x6b,0x79,0x9d,0x0e,0x56,0x81,0x33, -0xe1,0x08,0x3c,0x2f,0xd1,0x3a,0xc2,0xac,0x4d,0x8b,0x3c,0x53, -0x1a,0xe4,0x66,0x25,0xe8,0x72,0x63,0xcb,0xeb,0xb7,0xf0,0x7a, -0xc3,0x38,0xe0,0x6d,0x1d,0xb0,0x59,0xac,0x50,0xa4,0xf9,0xe4, -0x20,0x3b,0x7d,0x13,0x69,0x53,0x08,0xac,0x60,0x1f,0xac,0x43, -0x6e,0x46,0x65,0x12,0x64,0x1e,0x36,0xea,0x01,0xaf,0x5e,0xa6, -0xe4,0x50,0xc1,0x4c,0x0b,0x6c,0x25,0x35,0xe5,0xd6,0x9d,0x47, -0x55,0xce,0xa1,0xcb,0xdf,0xda,0x26,0x0b,0x9b,0xea,0xff,0x04, -0xd3,0x95,0x74,0x90,0x76,0x89,0x17,0xe1,0x0e,0x48,0xdb,0xaa, -0xfd,0x8a,0x6e,0x81,0xe4,0x63,0x4f,0x82,0x69,0x11,0x3b,0xe3, -0x47,0x1d,0x91,0xc4,0x56,0x96,0x26,0xb0,0xf8,0xa6,0xea,0x4f, -0xdd,0x57,0x29,0x68,0x7f,0x94,0xf3,0xf9,0x5b,0x4d,0x8e,0xa3, -0x68,0x39,0x1f,0x38,0x42,0x85,0x11,0x65,0x0e,0x08,0x5e,0x7f, -0xd1,0x4b,0xb0,0xfd,0x07,0xba,0xc4,0xb9,0x78,0x3c,0x67,0xae, -0x78,0x26,0xf3,0x0e,0xdb,0xdb,0xe1,0xaf,0xf3,0xea,0x14,0x9d, -0x26,0x06,0x57,0xa7,0x74,0x94,0xb3,0x95,0x71,0x6e,0x47,0x8b, -0x9a,0x4b,0xd2,0x4c,0x6b,0x2c,0x1f,0x85,0xc5,0xc2,0xe6,0x15, -0x67,0xb2,0xf8,0x84,0xac,0x98,0x16,0xda,0x05,0xcd,0x51,0x7f, -0xc0,0xe2,0x6a,0xde,0x2b,0x8a,0xe9,0x80,0xa4,0x55,0x27,0xd0, -0xd6,0x58,0x31,0x35,0xe6,0xb1,0x79,0x71,0x4d,0x59,0xd9,0xae, -0x23,0x75,0x14,0xd5,0xc4,0x6f,0x4e,0xf0,0xbd,0xec,0x57,0x66, -0xa1,0xcf,0xf9,0x9e,0x53,0x02,0x06,0x3c,0x09,0x19,0x30,0xa2, -0xc9,0x83,0x1e,0x41,0x88,0x2d,0x24,0x78,0x7e,0xcb,0xf3,0x07, -0xde,0xdb,0x36,0xe6,0x90,0x19,0x33,0x06,0xb9,0x6c,0x11,0xc4, -0x33,0xf6,0xbc,0x0d,0xf1,0x09,0x50,0x08,0x3c,0xeb,0x99,0xfe, -0x39,0x19,0xcb,0xac,0x9c,0xdf,0xce,0xa7,0xe1,0xd4,0x6a,0x13, -0x7c,0xd1,0x3c,0x81,0x96,0x49,0xff,0x04,0x8f,0x48,0xa7,0x96, -0x3c,0x20,0x03,0x35,0x0c,0x6f,0x89,0xbf,0xba,0xd7,0xf3,0xd5, -0x64,0x3a,0x5e,0xc3,0xa1,0x15,0x7d,0x60,0x3b,0x10,0xca,0x76, -0x95,0x16,0x78,0xb4,0x4e,0x81,0xf7,0xbf,0x35,0x71,0x44,0x83, -0x9f,0x7a,0x03,0xd4,0x21,0x04,0x54,0x5c,0x58,0xb9,0x2a,0x25, -0xaa,0x60,0x96,0x49,0x61,0x0e,0xdc,0x37,0xdc,0x76,0x53,0x67, -0x36,0x19,0x32,0xec,0x61,0x72,0xd5,0x0d,0x50,0xd5,0xa7,0x6f, -0x64,0xb1,0x42,0xdc,0x1d,0xac,0xe2,0x60,0x0f,0xdd,0xcb,0xc8, -0x8e,0xe0,0xaa,0x88,0xd0,0x37,0xef,0xeb,0x2c,0x93,0xdd,0x88, -0x4e,0x6c,0x3a,0x70,0x4e,0xc7,0x28,0xdf,0x5f,0x71,0x83,0x09, -0x80,0x92,0xc1,0x4c,0xb3,0x6d,0x5b,0xe3,0xfd,0xaf,0x98,0x4c, -0xa8,0xb9,0x3c,0xe1,0xaf,0x8a,0x46,0xaf,0x98,0x71,0x9b,0x0c, -0x5a,0x7c,0xd1,0xb1,0xa2,0xb4,0x97,0x9a,0xef,0xee,0xd6,0x52, -0x0d,0x81,0xe2,0xb0,0x3f,0x7b,0x46,0xc8,0x6d,0xc4,0xab,0x8f, -0xad,0x7b,0xb0,0x82,0x71,0x16,0x61,0x8f,0x2e,0x00,0xb4,0x57, -0xb1,0x41,0xf3,0x80,0xd6,0xdc,0x27,0x56,0x34,0x00,0xff,0x5a, -0x39,0x41,0x5a,0xb4,0x57,0x50,0xc0,0x70,0xda,0xa2,0x0b,0xf0, -0x0f,0xe9,0x49,0x5c,0xa4,0x9f,0xb8,0xde,0x28,0x91,0xcc,0x4c, -0xe8,0x37,0x9d,0xbc,0xe8,0x63,0xa3,0x6e,0x71,0xc7,0xc1,0x0d, -0x2e,0xda,0xb1,0xa0,0xcf,0x63,0xb9,0xfc,0x16,0x25,0xc8,0xa2, -0x7e,0xc1,0x1d,0x44,0x40,0x15,0x1f,0xba,0xcd,0x7c,0x7d,0x7b, -0xac,0x6e,0x83,0x4f,0xce,0x75,0x1f,0xc1,0x2d,0xad,0x92,0x45, -0xe1,0x33,0x58,0x74,0xcd,0x2b,0xde,0x59,0x70,0x45,0xcc,0x1a, -0x14,0xf7,0x4b,0x14,0x25,0xa7,0x41,0xb0,0xd0,0xd8,0xa5,0x2f, -0x1d,0xae,0x56,0xf8,0x70,0x1b,0x3b,0xf1,0xee,0x0e,0x3a,0xd0, -0x7e,0x60,0xe7,0x85,0x04,0x2c,0x59,0x30,0xf3,0x73,0x45,0xc8, -0x1d,0x18,0x60,0x30,0x1f,0x47,0xde,0x0b,0x98,0x77,0x77,0x4e, -0x37,0xa3,0xe2,0xa3,0x0c,0x1a,0x7a,0x94,0xbc,0x26,0xf2,0x65, -0xbe,0x09,0xb5,0x7b,0x3b,0x95,0x1e,0xdc,0xf0,0xf9,0x74,0x19, -0x04,0x37,0xf3,0x25,0xa7,0x32,0x2a,0x40,0xef,0x81,0xf7,0x7b, -0x74,0xd3,0x81,0xd7,0x7b,0x80,0xa4,0xb4,0xcd,0x44,0x2b,0xcb, -0x21,0xbd,0x0a,0xc2,0xf9,0x1e,0x48,0x1a,0x40,0x85,0x36,0xa2, -0xc5,0x88,0x43,0x46,0xa5,0xa3,0xff,0x1e,0x46,0x1c,0x20,0x99, -0x36,0x22,0x45,0x48,0x43,0x7a,0x15,0xc6,0x0a,0x3e,0xb0,0xe2, -0x08,0x2b,0x35,0x27,0xc4,0x58,0xd5,0x31,0xa7,0x0a,0x22,0x01, -0x3f,0xc2,0x7d,0xa3,0x90,0xe7,0x1b,0xc7,0x20,0x52,0xf0,0x81, -0x7d,0x83,0xb0,0xd2,0x3b,0x72,0x91,0xd7,0xec,0x1f,0xbe,0xcf, -0x93,0x1e,0x62,0xd6,0x71,0x58,0x17,0x11,0xb0,0xdc,0xb3,0x08, -0x6e,0x27,0xd6,0x4d,0x94,0xe7,0xca,0x79,0xf8,0x15,0xb1,0xbb, -0x15,0x26,0x0a,0x2d,0xf7,0xc2,0x01,0x79,0x57,0xba,0x5c,0x09, -0x07,0x6c,0xec,0x41,0x57,0x64,0x9d,0xad,0x27,0xc6,0x77,0x45, -0x41,0x74,0xe7,0xc3,0x88,0x6d,0xe4,0x8e,0x56,0x68,0x6d,0xf8, -0x86,0xb6,0x25,0xce,0xf2,0x81,0x97,0x50,0x8a,0xb7,0x7a,0xd2, -0x2a,0xb5,0x6a,0x0b,0xcf,0x7c,0xe0,0x30,0x2b,0x66,0xed,0xe9, -0xab,0xd4,0xaa,0x3d,0xa8,0xf3,0xa1,0x37,0x51,0x9a,0xff,0xfb, -0x32,0x82,0xf4,0xaa,0x25,0x6a,0xf5,0xa1,0x37,0x75,0xfe,0x96, -0xc5,0xf7,0x91,0xba,0x62,0x89,0x47,0xce,0x7e,0xe8,0x25,0x8b, -0x22,0xae,0x2f,0x55,0x98,0x78,0xfb,0x3d,0x07,0x01,0xb8,0x9b, -0x8e,0x60,0x8e,0x17,0x3b,0xb9,0xe5,0xe0,0xcb,0x0d,0xe5,0x45, -0xfd,0xe7,0xc5,0xa8,0x9c,0xc5,0x5f,0x48,0x43,0x4e,0xca,0xcf, -0x3d,0xe0,0xa4,0x25,0x96,0x5c,0xee,0xf1,0x15,0x6e,0xb4,0xf8, -0x45,0xae,0xd4,0x55,0x36,0x55,0xf4,0x69,0xbf,0x66,0xf4,0x8f, -0x34,0xf7,0x19,0xfd,0x63,0x31,0x63,0xe5,0xc8,0x75,0xb5,0xaf, -0xd7,0x7c,0xcd,0xe0,0x8f,0x21,0x07,0xd3,0x3e,0x55,0x9b,0xa5, -0x37,0xbd,0xad,0xdf,0x63,0x66,0xe0,0x5a,0xc9,0xaf,0xa8,0x3f, -0x40,0xa8,0x6c,0x4f,0x9b,0x0e,0xe7,0x10,0x2d,0x1b,0xfd,0xe2, -0x82,0x33,0xfa,0x46,0x2e,0xfe,0xaa,0xb9,0x74,0x4c,0x13,0x50, -0x50,0x24,0xc8,0xd9,0xe6,0xf0,0x20,0xa0,0xe1,0x77,0x32,0x70, -0x0f,0x3b,0x3c,0xfb,0x80,0x40,0x10,0x69,0xf8,0xf9,0x73,0x2c, -0x80,0x14,0x02,0x29,0xd4,0xc7,0x10,0xc4,0xb9,0x0f,0xa4,0x5d, -0x33,0xc6,0x27,0xd8,0xd0,0xd1,0x38,0xe0,0x4d,0xe7,0x57,0xd7, -0x6b,0x37,0xb4,0x40,0x84,0xa2,0xdb,0xad,0xb7,0xf4,0x63,0x20, -0x3f,0x52,0x84,0xd4,0x0f,0xee,0x54,0x23,0x51,0x24,0x05,0x8f, -0xde,0x0e,0x64,0x54,0x82,0x97,0xc5,0x72,0x7e,0x01,0xe2,0x2d, -0x17,0x96,0x05,0xc3,0xa0,0x00,0x1a,0xd5,0x92,0xd7,0x70,0x5b, -0xb3,0x5a,0x5c,0x2f,0x2f,0xca,0x46,0x87,0x52,0xb2,0xc4,0xdd, -0x43,0x73,0xaf,0x5a,0x0b,0xe0,0x60,0x8a,0x40,0x03,0xfa,0x87, -0x3b,0x37,0xef,0x35,0xdc,0x7a,0x42,0x78,0x01,0x44,0x7c,0xeb, -0x43,0x4c,0x90,0xce,0x04,0xad,0x19,0xde,0x2c,0x4b,0xfe,0x86, -0xd7,0xb9,0x6f,0x2f,0x87,0x1f,0xca,0xb7,0x6b,0x48,0xdb,0xf2, -0x6b,0x25,0x1e,0x06,0xae,0xd2,0xf4,0x1c,0xba,0x35,0x6c,0x2b, -0xa9,0xd0,0x5c,0x23,0x29,0x52,0xf2,0x2e,0x9c,0x5f,0x25,0x9c, -0xc7,0xd4,0x44,0x66,0x24,0x82,0x35,0x1a,0xa8,0xde,0xef,0x21, -0x1a,0x19,0xa8,0x46,0x90,0x71,0x65,0x3a,0x58,0xfd,0x34,0xb4, -0x77,0x1e,0x83,0xe7,0x85,0xdb,0x6b,0xbc,0xeb,0xe0,0xf0,0x1a, -0xae,0x39,0x61,0x0c,0xf5,0xd6,0xf6,0x30,0x9a,0xf8,0x78,0x79, -0x68,0x8b,0xa2,0xe8,0x7b,0xdb,0x14,0xc5,0x68,0x6d,0x15,0x31, -0xae,0xc0,0x6d,0xf9,0xbd,0x0c,0x87,0x99,0x1d,0xb8,0xf9,0xd6, -0x1c,0x87,0xfc,0x36,0x28,0xda,0x0d,0x67,0xe3,0x68,0xe8,0x4c, -0xaf,0x5f,0x70,0xf2,0x5e,0xf2,0x92,0x22,0xe3,0xe7,0x4b,0xdd, -0x70,0xf7,0x60,0x4f,0x8a,0x0b,0x1f,0xc2,0x92,0xba,0xe7,0x92, -0x30,0xc9,0x55,0xb6,0x40,0x66,0xee,0x57,0x7d,0x9c,0xd5,0x1c, -0x89,0xc0,0x70,0xbf,0xe9,0x6e,0x57,0x50,0xda,0x9b,0x77,0xbf, -0xd1,0x8b,0x75,0x3f,0x1b,0x4d,0x17,0x15,0xc2,0x6b,0x70,0x20, -0x64,0xe8,0x9c,0x49,0x54,0xb0,0x65,0xb0,0x41,0x78,0xe1,0x7d, -0x17,0x14,0x52,0x0f,0x0a,0xd7,0xc2,0x1d,0xa5,0x36,0x31,0xed, -0xed,0x43,0xba,0xc6,0x23,0x50,0x8f,0x46,0xfc,0x66,0x34,0x1f, -0x2f,0x45,0xf6,0x57,0x35,0xbb,0x79,0xde,0xd5,0xbc,0xce,0xab, -0x3c,0x13,0xbc,0x53,0x23,0xaf,0x1c,0xfc,0x46,0xdf,0xfd,0x8c, -0x3c,0x96,0x07,0xa3,0x86,0x9d,0xb7,0x63,0x48,0xe1,0x71,0x91, -0xd4,0x0d,0x77,0x4c,0x7e,0x63,0x04,0x81,0x5a,0xea,0xf3,0x69, -0x13,0xa4,0x95,0xb3,0x30,0xbc,0x0c,0xf7,0x7a,0x39,0xd3,0x7b, -0x33,0xcd,0x4b,0x1e,0xc6,0xee,0x5b,0x2c,0x41,0xd4,0x9c,0x28, -0x0e,0x70,0x06,0xcd,0x1a,0xfa,0xfd,0x56,0xb1,0x0a,0xf5,0xd2, -0xf3,0x2d,0xcf,0x1a,0x06,0x0a,0xd6,0x9e,0x06,0x1b,0x5e,0xaf, -0x17,0x70,0xf1,0x2d,0x65,0xca,0xd9,0x13,0x8d,0x30,0x7a,0x24, -0x96,0xf4,0xc4,0x06,0xa3,0xc7,0xc6,0x0f,0x3d,0x91,0x2e,0x5c, -0x04,0x9f,0x40,0xc0,0x00,0x6f,0x69,0xe5,0x52,0xa7,0x4d,0x86, -0xf3,0xd1,0x2c,0x4c,0x9a,0x2e,0xbc,0x8b,0x1c,0x53,0xce,0xdc, -0x16,0x3a,0x5d,0x4f,0x87,0x33,0xb8,0x34,0x2f,0x67,0x9a,0x67, -0x63,0x6c,0x9c,0x7d,0xf2,0xc9,0x5e,0x9e,0xde,0xc6,0x30,0xa8, -0xf3,0xec,0x29,0x87,0x2f,0x50,0xd2,0xfd,0xfd,0x6b,0x64,0xd0, -0x9a,0xb5,0x95,0x9c,0x2e,0xf9,0x92,0xf8,0x2f,0x74,0xf1,0xae, -0x86,0x91,0xea,0x97,0x86,0x3b,0xc0,0x7a,0x39,0xbd,0x70,0x6b, -0xce,0x0f,0x45,0x96,0x05,0xec,0x8b,0x6a,0xf0,0x00,0x8e,0xdf, -0xda,0x5a,0x82,0xb3,0xcd,0x15,0xaf,0x5f,0xe9,0xbd,0x73,0xe5, -0x8f,0x34,0x99,0x28,0x04,0x65,0x3c,0xb4,0xe9,0x84,0xa5,0xdb, -0x7e,0x55,0x37,0x53,0x04,0xab,0x87,0xfb,0x5a,0x7e,0xe5,0x4c, -0x15,0xed,0x2f,0x89,0x24,0xc8,0x8e,0xf8,0x00,0x7f,0x50,0xaf, -0xc7,0x27,0xad,0x06,0x6a,0x14,0x45,0x34,0xbd,0xc0,0xb7,0xa9, -0x40,0x3b,0xaf,0xd1,0xee,0xd5,0x68,0x83,0xf1,0x43,0x80,0x0f, -0x01,0x18,0x7c,0xaf,0xa7,0xe1,0xaa,0x02,0x46,0xe4,0x3b,0x5f, -0xcf,0x4a,0x1e,0x2f,0x34,0x02,0x09,0x56,0x9d,0x8b,0x83,0x05, -0x59,0xb5,0x0e,0xbe,0x12,0x93,0x4c,0xdf,0xc1,0x52,0x4f,0xd7, -0xc1,0xe8,0x60,0x6c,0x3a,0x0e,0x9b,0xf1,0x65,0x3f,0x08,0xd6, -0x52,0x6b,0xe2,0x29,0xb9,0xd8,0x1a,0x15,0x41,0x29,0xe6,0x1d, -0x47,0x6e,0xa1,0xf3,0x0b,0xf1,0x6a,0xfb,0xaf,0x73,0xdc,0x57, -0x7b,0x7a,0x43,0xd4,0x7e,0x9c,0xa3,0x76,0xac,0x78,0xd7,0xdd, -0xf6,0xbf,0xec,0x0d,0xc8,0x81,0x6b,0x39,0x9d,0xa5,0x47,0x90, -0x92,0x75,0x10,0xcd,0x0e,0xc6,0xda,0x42,0xdc,0x36,0x20,0x6e, -0x15,0xc4,0x12,0x68,0xbc,0xf8,0x4a,0x75,0x9d,0x82,0xed,0x40, -0xde,0x91,0xca,0xf3,0x78,0xef,0x80,0xb2,0xe0,0x51,0x37,0x87, -0x88,0xb7,0x82,0xc8,0x93,0x5c,0x30,0xbd,0x6f,0xe8,0x51,0xb1, -0xc3,0xc9,0x33,0x43,0xcb,0xb2,0x35,0xcf,0x9c,0xa5,0x4c,0xa5, -0x77,0xe1,0x14,0xaa,0x4f,0xc7,0xf6,0x29,0x43,0xcc,0x49,0x85, -0x82,0xd4,0x2c,0xe3,0x28,0x9d,0x75,0x96,0x59,0x16,0x6e,0x08, -0x4d,0x58,0xca,0x38,0x4a,0xd7,0x9d,0x77,0x59,0x8c,0x67,0xe9, -0x3e,0x39,0x74,0x7d,0x3f,0x8c,0x5f,0xb0,0x27,0xad,0x1d,0x3a, -0xbe,0xc8,0x13,0xb9,0xeb,0xac,0xa7,0xd3,0x67,0x02,0xf1,0x37, -0x11,0x4d,0xcf,0x6b,0x81,0x45,0xff,0x7a,0x75,0xe1,0xc1,0x6d, -0x53,0x6e,0x4e,0xa3,0xf3,0xd6,0xfb,0xb8,0x86,0xa0,0x81,0x32, -0xd8,0xc2,0xbb,0xb5,0xe8,0xe3,0xd6,0xe1,0x93,0xab,0x07,0x35, -0x38,0x5c,0x7e,0x42,0x29,0x94,0x74,0x78,0xff,0xf4,0x42,0x9b, -0x32,0x0a,0x8c,0x54,0x56,0x3c,0xe7,0xb3,0xb1,0x1f,0x54,0x13, -0x37,0x4c,0xd9,0x5d,0x33,0x93,0xac,0x36,0xef,0x23,0xd9,0x36, -0x35,0x73,0xbb,0xf1,0x53,0x9f,0x2d,0xfb,0x33,0xde,0xcb,0xb7, -0x6f,0xea,0x0f,0xdc,0x13,0x50,0xb8,0x61,0x0c,0x22,0x97,0xd6, -0x7a,0x63,0xba,0x08,0x0f,0xa4,0x0f,0x56,0x38,0xd0,0xb4,0x04, -0x50,0x82,0x0f,0xa8,0xab,0xcc,0xec,0x8f,0xeb,0x1f,0x98,0x1a, -0xc4,0x1a,0x62,0x6f,0x6e,0x74,0xa0,0x77,0xe1,0xef,0x80,0x4e, -0xc3,0x98,0x6f,0x64,0x7c,0xe4,0xa6,0xd1,0xd9,0x87,0xf3,0x22, -0xb4,0x25,0xf8,0xda,0xca,0xc7,0xa2,0x38,0x30,0xa3,0xca,0xec, -0xaa,0x40,0x7f,0x10,0x94,0x2a,0xe6,0x20,0x11,0xc9,0x51,0xb7, -0xee,0x32,0xd4,0x22,0xe1,0xe7,0x61,0xfd,0x08,0x8f,0x26,0xe0, -0x5f,0x37,0x23,0x88,0xd4,0xa5,0x92,0x43,0x79,0x83,0x91,0x03, -0xa3,0xec,0x37,0xc4,0xdf,0xa9,0xa8,0xc8,0x18,0x84,0x6a,0x10, -0x99,0x3a,0x5a,0x54,0x96,0x88,0x6b,0xe5,0xe6,0x07,0x22,0x69, -0x36,0x74,0xe8,0xb8,0x29,0x36,0xb2,0xe9,0x4d,0xe0,0xa7,0xdf, -0xf3,0xe0,0x4b,0x4c,0xfe,0xa4,0x83,0x5d,0xa8,0x09,0x4b,0xd1, -0x45,0x9d,0x18,0x4d,0x37,0x5d,0x88,0x29,0x91,0x64,0xde,0x85, -0x86,0x93,0xc5,0x4b,0x0c,0x23,0xab,0xb0,0x24,0x38,0x85,0xc5, -0x8a,0x46,0xa6,0x48,0x5c,0xc4,0x89,0xab,0x05,0x39,0x0b,0xb0, -0x19,0xcb,0x72,0x86,0xc1,0xbe,0x5d,0xd8,0x09,0xb3,0xf1,0x4b, -0x1d,0x3d,0xaf,0xe8,0x7a,0x11,0x3a,0x87,0xb4,0x48,0x15,0xa4, -0x3d,0xfb,0xdd,0xdd,0xdd,0x60,0xb0,0x3c,0x5f,0x8c,0x84,0xba, -0x08,0x00,0xfb,0x16,0x70,0x52,0x03,0xb4,0x2d,0x5b,0x82,0x12, -0xcc,0xd6,0x07,0xb6,0x91,0x04,0x2b,0xe1,0x96,0x71,0xaa,0x3e, -0xc8,0x19,0xa8,0x2c,0xc1,0xee,0xaf,0x9c,0x9c,0x85,0xc2,0x38, -0xc4,0x17,0x20,0xe1,0x42,0x8d,0x0b,0x8e,0x81,0x0c,0x91,0x17, -0xf3,0xd3,0x80,0x20,0xc3,0xd6,0xe9,0xc1,0x26,0xb6,0xc9,0x78, -0xb2,0x6c,0x78,0x6e,0x04,0xd4,0xc1,0x1d,0xa1,0xfb,0xe8,0x32, -0x71,0xbb,0x7a,0x9b,0x07,0x35,0x2b,0xe9,0xfb,0xe2,0xbb,0x90, -0x40,0x46,0x45,0x13,0xc1,0xf2,0xc7,0xb6,0x3a,0xa3,0x0d,0x3d, -0x98,0xac,0x6a,0x07,0x4b,0x25,0x76,0x21,0xc7,0xd2,0xcb,0x90, -0xc5,0x17,0x76,0xf4,0xd8,0x20,0x1f,0x3a,0xee,0x09,0xa8,0xfa, -0x8a,0xb2,0x15,0x55,0x90,0x6c,0xdc,0xb2,0x8c,0x10,0x1d,0x8f, -0xef,0xa7,0x3a,0x1e,0x3f,0x94,0x2c,0x7b,0xaa,0x0a,0x45,0x3e, -0xfb,0xa7,0xb0,0xff,0x83,0x16,0xf5,0xd4,0x1d,0x68,0x71,0x19, -0xe2,0x81,0x1d,0xf2,0x9c,0x6b,0x9e,0x41,0xc4,0xd1,0x46,0xaa, -0x01,0xc0,0x1b,0x2b,0x45,0x02,0x0a,0x02,0x8d,0xe0,0xa9,0x8e, -0xb9,0x97,0xfa,0x7d,0x77,0x97,0xd2,0x17,0xf5,0x34,0xb9,0x18, -0xa3,0x90,0x59,0x3a,0x9d,0x7d,0x4a,0x92,0xfe,0x00,0x0a,0x20, -0x0f,0x9a,0xf0,0x4b,0x06,0x05,0xb2,0xd8,0xfd,0x13,0xfa,0x3f, -0x49,0xeb,0xc3,0x4d,0xf4,0xea,0xe3,0xac,0x4b,0x09,0xc7,0xb9, -0xca,0x30,0xe2,0xac,0x00,0xaa,0xb2,0x30,0x99,0x45,0x36,0xec, -0x41,0x0a,0xff,0xaa,0x54,0x04,0x12,0xe7,0x35,0xbe,0xab,0xdd, -0xd3,0xfb,0xb4,0xaa,0x98,0x12,0xfb,0xfa,0x4e,0xbd,0x66,0x32, -0x13,0x71,0x4c,0x29,0xf7,0x32,0x27,0x22,0x61,0xa2,0x84,0xc0, -0x75,0xf2,0x10,0x25,0xcb,0x57,0xe6,0x44,0x0f,0x6c,0x95,0x67, -0x0f,0x3e,0x34,0x8f,0xdf,0x5a,0xd5,0x4b,0x5d,0x56,0x01,0xcf, -0x16,0x70,0xb4,0xda,0xb8,0xf8,0x7b,0x2a,0x39,0xc3,0x21,0x81, -0x4f,0x07,0x20,0x09,0x94,0x05,0xbb,0xcc,0x46,0xf3,0x07,0x4a, -0xa4,0xcc,0x09,0xb0,0xb2,0xa7,0x10,0x98,0x0e,0x80,0x6c,0xff, -0x24,0x97,0x8b,0xeb,0x55,0x09,0xa9,0x89,0x8a,0x75,0xbb,0x11, -0x7d,0x16,0x2a,0x74,0xc2,0x80,0x9e,0x96,0x6f,0x22,0x78,0xc6, -0x5b,0x55,0x4d,0xb0,0xa4,0x4c,0x1a,0x60,0x74,0x64,0x0b,0x6a, -0x1e,0x55,0xd2,0xf5,0xfa,0x91,0x0a,0x62,0x09,0xb6,0x56,0x92, -0xd9,0x54,0x68,0x4d,0xf5,0x3a,0x88,0xcb,0xcc,0x86,0x0d,0x61, -0xac,0x66,0xd6,0x1b,0x61,0x2a,0x07,0x47,0xf6,0x11,0x96,0xf1, -0xa7,0x44,0x41,0xee,0x51,0x00,0xe3,0x33,0xf1,0x93,0x09,0xbf, -0x40,0xcb,0x95,0x63,0x84,0x3d,0xb4,0x58,0x39,0xaf,0x0c,0x79, -0xb8,0x14,0x33,0x2e,0x21,0x6d,0x56,0x81,0xea,0x85,0xe2,0x27, -0x1f,0xae,0x7a,0x09,0xf9,0x92,0x8f,0x06,0x5d,0xdc,0xdc,0xaf, -0xbb,0x8e,0xe0,0x8a,0x3e,0xa2,0x81,0xdc,0x54,0x05,0x44,0xb0, -0xdd,0x99,0xe9,0x80,0x53,0xc1,0x26,0x82,0xef,0xa0,0x36,0x0d, -0x02,0xa1,0xe4,0x88,0x57,0x25,0x56,0xec,0x1d,0xaa,0x4d,0x84, -0x3e,0x07,0xf4,0x4f,0x17,0x20,0x0a,0xb4,0x82,0xa5,0x84,0x42, -0xab,0x21,0x30,0xd2,0xec,0x80,0x8c,0x64,0xe1,0x4f,0x95,0xd3, -0xdd,0x8b,0xab,0x08,0xae,0x26,0xb4,0xf9,0xe3,0x42,0x22,0x2a, -0xf3,0xe0,0xd9,0xf7,0x9f,0x5d,0x21,0x9a,0x57,0xfb,0xab,0x14, -0x7a,0x49,0xaa,0x55,0xc6,0x93,0x42,0xf5,0xeb,0x5e,0x42,0x18, -0xf7,0x35,0x38,0x11,0x19,0x8a,0x83,0x0a,0x7f,0x29,0xa6,0x2d, -0x39,0x2f,0x03,0xdb,0x2f,0x04,0x26,0x33,0x30,0x7c,0xfe,0x84, -0xa0,0xca,0xbe,0xcb,0x5d,0x8f,0x62,0xe1,0xa4,0x58,0x67,0x6b, -0x4a,0x4b,0x92,0x2c,0x80,0x27,0x43,0x52,0x88,0x89,0x03,0x13, -0xb0,0x81,0xcd,0xc3,0xbe,0x5d,0xbc,0xfb,0x5d,0xb9,0x16,0xb0, -0x25,0xa0,0x9d,0xac,0x4d,0x1d,0xa4,0xf6,0x0f,0x4d,0x0f,0x07, -0x98,0x47,0xfa,0x16,0x57,0x68,0xad,0x6f,0xb1,0xd1,0x57,0xf7, -0x95,0xe4,0xdb,0xc2,0xf7,0x83,0x72,0xee,0x00,0xc8,0xc6,0xb1, -0x63,0x01,0xc7,0x0e,0xda,0x44,0x6c,0xbe,0x3d,0x5d,0xc8,0x1a, -0x5e,0xf0,0x19,0xc4,0xbd,0x11,0xde,0x64,0x83,0x4d,0x2e,0x0e, -0x8d,0x2a,0xb3,0xc0,0x33,0x08,0xd7,0x6b,0xb4,0x98,0x97,0x79, -0x4c,0xf8,0x18,0xae,0xab,0x2a,0x5c,0x97,0xe0,0x5b,0x3d,0xe2, -0x68,0x6c,0xa3,0x23,0xd0,0x2b,0x76,0xe3,0x20,0x34,0x15,0x08, -0x4a,0xde,0x26,0xeb,0x00,0x96,0x65,0x5e,0xf0,0x97,0x9d,0x7c, -0xd5,0xd9,0x64,0xe8,0xd1,0x0b,0x38,0x6b,0xba,0x22,0xf7,0x5d, -0x93,0x72,0x68,0xc5,0xa1,0xd5,0x8a,0xe3,0x8b,0xcb,0xe7,0x01, -0x3a,0x7c,0xad,0xc0,0x47,0xf1,0x95,0x0e,0xe6,0x24,0x65,0xd1, -0x1e,0xa9,0xcf,0xc7,0x6e,0x3e,0xfa,0xf3,0x05,0xc5,0xbc,0xfb, -0xbf,0xa7,0xed,0xc7,0x47,0x5a,0x87,0x29,0x39,0xbd,0x26,0x00, -0x6b,0x4a,0x01,0x01,0xdc,0xa1,0xc1,0xd4,0x15,0x02,0x0f,0xd2, -0xe1,0xeb,0x93,0xea,0x01,0xc7,0x54,0x3c,0xa3,0xaa,0x03,0x2a, -0x1e,0xe0,0x9a,0x35,0x08,0x8f,0xa0,0xe1,0xf9,0xf3,0x93,0x6e, -0x0d,0xf6,0x5d,0x1a,0x28,0x8d,0xac,0xae,0x43,0xeb,0x55,0x40, -0x58,0xb9,0xc7,0xd2,0xf4,0xdf,0xa3,0xe8,0xd7,0xaa,0xfd,0xd6, -0x5a,0xc6,0xb4,0x7b,0x61,0x6d,0xff,0xa9,0x95,0xf3,0xed,0x33, -0xec,0x2f,0xd1,0xa9,0x87,0x67,0x95,0x7f,0xa9,0xd4,0xff,0x39, -0x54,0xea,0x7f,0x99,0xfe,0xbc,0x7d,0x51,0x03,0x2e,0xb9,0x08, -0xfe,0x97,0x2e,0xf7,0x91,0x75,0xb9,0xcd,0x9b,0x81,0x16,0x6d, -0x95,0x46,0x62,0xb9,0xa4,0x0d,0x85,0xb2,0x9b,0x08,0x5f,0xaf, -0x6e,0xe7,0x17,0x51,0x67,0xb1,0x62,0x85,0x78,0x33,0x9c,0x82, -0x4b,0x83,0x9e,0xe5,0xa4,0xcb,0x2e,0x68,0x49,0x6d,0x2b,0x7e, -0x84,0x58,0x33,0x76,0x4c,0x06,0xa9,0xd6,0x91,0x64,0x4e,0xc0, -0x63,0xbd,0x0a,0xdf,0x4b,0x64,0xb9,0x5d,0xd2,0x6f,0xa6,0x97, -0x70,0xc0,0x4e,0x81,0x9c,0xe9,0xf7,0xc8,0xe3,0xf6,0xb2,0x08, -0x1b,0x64,0x9c,0xee,0xe9,0xb4,0x51,0xd6,0x57,0xbd,0x01,0x55, -0x25,0x77,0xa5,0xe8,0xa6,0x05,0xf3,0x63,0xb5,0x79,0xaf,0x7b, -0x02,0xa2,0x16,0xc1,0xd5,0x08,0xdc,0x7d,0x38,0x91,0xad,0x11, -0x0a,0xfa,0xaa,0xc6,0x71,0x6f,0x0a,0xb5,0x7c,0x3b,0xe9,0xd5, -0x00,0x7c,0xd4,0x8f,0xd7,0x9d,0xab,0x2e,0x6f,0x39,0x10,0x49, -0x4e,0xaf,0x60,0x82,0xb1,0x7c,0xc7,0x82,0xc8,0xf6,0x94,0x19, -0xa8,0x8b,0x57,0x05,0xcb,0x1a,0xf7,0x8a,0x6a,0xa8,0x98,0x28, -0x90,0x29,0x60,0xe7,0x0f,0x6f,0x7e,0xfe,0xc9,0xf6,0xe9,0xd5, -0x6c,0x68,0xe7,0xe1,0xf1,0x93,0xc9,0xb2,0x1c,0x17,0xc7,0xef, -0x4d,0xf2,0x64,0x3b,0x9b,0xce,0x3f,0xe4,0xf8,0x9d,0xc8,0x2c, -0xfb,0xfc,0x4b,0x4b,0xe0,0x09,0x31,0xb3,0xe4,0xf3,0xce,0x4d, -0xe7,0xf3,0xe4,0xc9,0xe7,0x9d,0xcf,0x99,0x65,0xd9,0x94,0x09, -0xa5,0xf8,0xcd,0xc1,0xc2,0x83,0x03,0x57,0xb0,0xe7,0x2e,0x3b, -0xc9,0x57,0x49,0xc7,0x26,0x74,0x92,0x2f,0x8f,0xed,0x3f,0x5f, -0x25,0xf7,0x29,0xed,0x25,0x26,0xac,0xea,0x9a,0x5a,0x2f,0x98, -0xc6,0xde,0xc5,0x35,0x75,0xb7,0x53,0x6e,0x08,0x06,0x8e,0xcf, -0xbd,0xfe,0xed,0xfb,0x54,0x14,0xcf,0xb9,0x4b,0xfd,0x06,0x47, -0xd6,0x65,0x34,0x16,0x8a,0x87,0x51,0xa2,0x1d,0x03,0x63,0x3d, -0x39,0xb3,0x11,0xa4,0xda,0xac,0x6f,0x8a,0x1b,0x78,0x42,0xca, -0x23,0x6a,0x7f,0x2c,0x59,0x54,0x2a,0xf0,0x69,0x29,0x8d,0xa2, -0xfd,0x97,0xc6,0xd1,0xf0,0xea,0x0a,0x8f,0x00,0x56,0x30,0xe6, -0x1a,0xae,0x41,0xe3,0x61,0xa7,0xff,0x7a,0x5b,0x70,0x30,0xa8, -0x7a,0xfc,0xe9,0x53,0x9b,0x07,0xc1,0xf3,0xd0,0xdb,0x30,0x58, -0xec,0xa7,0x7d,0x8c,0x27,0xdd,0x37,0x52,0x0b,0xc3,0xc5,0x66, -0x35,0x01,0xbc,0x2b,0xd1,0x06,0x2d,0x85,0x46,0x1e,0x5b,0x09, -0x48,0xa3,0xeb,0x9d,0x63,0xbb,0x35,0xd2,0x33,0xd1,0x96,0x47, -0x9b,0x5d,0x2b,0x2d,0x72,0x01,0x68,0x62,0x45,0xff,0xc9,0x1a, -0xf7,0x87,0x69,0xb1,0x1f,0x55,0x05,0x8d,0xc7,0xbe,0xa8,0x06, -0xfa,0x93,0x94,0xc5,0xc1,0xa1,0xb4,0xfb,0x47,0xd4,0xc6,0x37, -0x85,0x57,0x11,0xdb,0x05,0xa9,0x94,0xc3,0xe6,0xaa,0xd0,0x5a, -0x61,0x2b,0xca,0xc5,0xb5,0xbe,0x66,0x23,0x87,0xd2,0xa0,0x56, -0x34,0x6d,0xcc,0xf2,0xcf,0x51,0x0a,0xb3,0x46,0xd4,0x95,0x58, -0x9f,0x39,0x16,0xd2,0x44,0xf6,0xa4,0xd1,0x72,0x78,0x93,0x5e, -0x4c,0x86,0x4b,0x30,0xb0,0x5b,0xde,0xfa,0x33,0x3c,0xa6,0xa5, -0xa4,0x70,0x91,0x52,0x8c,0xb4,0x35,0xd7,0x0d,0xf7,0x62,0x53, -0x76,0x2a,0x9c,0x49,0xe7,0x0f,0xdc,0x2e,0x83,0x63,0xbe,0xb3, -0x20,0x74,0x21,0x60,0x7f,0xa4,0xee,0x76,0x40,0x6d,0xa2,0xa9, -0xda,0x44,0x71,0x43,0xda,0xd5,0x28,0x50,0x72,0xce,0x3b,0x31, -0x98,0x18,0x5e,0x0c,0x21,0x52,0x6e,0xb9,0x5c,0x2a,0xd8,0x12, -0x7c,0xd7,0x5b,0x78,0x78,0x3d,0x18,0xa8,0x51,0xa0,0x72,0xa8, -0x45,0x81,0xc6,0xb7,0x4e,0x1e,0xd4,0x7d,0x54,0xe9,0xe8,0xa5, -0x49,0xec,0xc9,0xa0,0x1c,0x4f,0xe7,0x25,0xb8,0x07,0x82,0x75, -0xb3,0x18,0x3f,0xb1,0xd3,0x70,0x01,0xfa,0x96,0x01,0x94,0x90, -0xcb,0x57,0x06,0x81,0x9a,0xaf,0x67,0x43,0xbc,0xd1,0x85,0x78, -0xd9,0x49,0xb9,0xbe,0x18,0x25,0x06,0x1e,0xc2,0x8f,0x89,0x6c, -0x9a,0xbc,0xb2,0x49,0xbf,0xf5,0x6d,0x62,0xf2,0x6c,0xb2,0x5e, -0x5f,0x29,0x45,0x39,0x7e,0xab,0xc1,0x81,0xd8,0x83,0xf3,0xeb, -0x2b,0x17,0x20,0x9f,0x6f,0x79,0xed,0xc0,0xac,0x30,0xa8,0xbe, -0x6d,0xc5,0xcc,0xee,0xe3,0xc9,0xb1,0xdb,0xc3,0x9e,0xf4,0xf0, -0x98,0xb6,0xf4,0xc2,0x61,0x92,0xe4,0xa9,0x60,0x40,0xf2,0x78, -0x3a,0x5b,0x6b,0x07,0xf1,0x4e,0x83,0x9b,0xc0,0xa6,0xbe,0xa9, -0x32,0x43,0x70,0xbf,0x2f,0xa6,0x73,0xa4,0xac,0xc2,0x5e,0xd9, -0x59,0xfd,0x9f,0xe5,0x2d,0x04,0x21,0x80,0x6d,0x8f,0xd5,0xed, -0xe5,0x6c,0xac,0x7c,0xcc,0xc2,0x27,0x69,0x49,0x75,0xe5,0x05, -0xc3,0x60,0xb6,0xe5,0xe8,0xbf,0xa0,0xb1,0x43,0x7d,0x33,0xa4, -0x7a,0x3a,0x12,0x87,0x34,0x8f,0xab,0x94,0x24,0x19,0x37,0xb3, -0x7b,0xb5,0xb8,0xb2,0x2c,0x81,0xd3,0xeb,0x6d,0xb1,0x83,0x8a, -0xe4,0xa1,0xf7,0x9a,0x12,0x9e,0x05,0xe9,0xd4,0xaa,0xfd,0x8b, -0x15,0x11,0xa6,0xdb,0x8e,0xab,0x93,0x10,0x78,0x1f,0xb5,0xad, -0xc5,0x01,0x84,0xbc,0xd4,0x95,0x92,0xba,0x32,0x03,0x73,0x5c, -0x4b,0x1a,0x3d,0x52,0x86,0xa8,0xe9,0xee,0x7a,0x39,0xcb,0x15, -0x2e,0x5d,0x49,0x3c,0xeb,0xa2,0x92,0x37,0x65,0x35,0xa3,0xe0, -0xda,0xb1,0xa2,0x98,0x86,0x79,0xf2,0xcb,0xdf,0x5e,0xbf,0x49, -0x0c,0xcc,0xe3,0x72,0xb9,0xca,0x77,0x09,0xee,0x98,0xf3,0xf5, -0xd1,0x1b,0x3b,0x5b,0x93,0x3c,0x81,0xcb,0x4d,0x78,0x16,0x60, -0x4b,0x3e,0xde,0x1e,0xdd,0xdc,0xdc,0x1c,0xc1,0xc6,0x79,0x64, -0x0b,0x23,0x6e,0x3c,0x4a,0xaa,0x4a,0xaa,0x69,0x79,0x74,0xb9, -0x2e,0x6d,0xe7,0xb5,0xb5,0xaf,0x59,0x49,0xa9,0xc5,0xb7,0xaf, -0x7e,0x7a,0xf5,0xe6,0xd5,0xe3,0xd4,0x23,0x98,0x75,0xaf,0xd7, -0x43,0xdb,0xa5,0xf6,0xcf,0x03,0xe7,0x9d,0x43,0x79,0xc8,0xa8, -0x03,0xd2,0xea,0xf1,0xc6,0xbd,0xe2,0xd0,0x1e,0x44,0xb1,0x48, -0x8e,0x37,0xfd,0x63,0x38,0x24,0x1d,0x27,0x46,0x95,0x44,0xe9, -0x98,0x70,0x9c,0xf8,0x5d,0x01,0x86,0x82,0xa9,0x8f,0x28,0x9e, -0xf4,0xaa,0xcb,0xbd,0x6b,0xa5,0x99,0xcb,0xcb,0xc5,0xfc,0x2c, -0xf9,0xef,0xa3,0x5f,0xc9,0x10,0xbc,0x1c,0x1d,0xfd,0x63,0xba, -0x9e,0x24,0xe7,0x66,0x67,0xab,0x02,0x7d,0x96,0x73,0xdf,0x19, -0xfb,0x6d,0x47,0x34,0xa7,0x55,0x51,0x55,0xe7,0x71,0x46,0x05, -0xa0,0x2b,0x60,0x4c,0xf3,0xf7,0xbf,0x42,0x70,0xfb,0xc4,0x10, -0xff,0xb2,0xdc,0x8b,0x24,0xde,0xd4,0xf2,0xac,0x25,0xe4,0xfc, -0xb2,0x5c,0x80,0x9b,0x4d,0x7d,0x9d,0x18,0x66,0x64,0xbb,0xf0, -0xbb,0x7b,0x33,0x29,0x71,0x0d,0x9a,0x9d,0xdd,0xce,0xaf,0x20, -0xba,0xc8,0xdf,0xed,0x2c,0x4a,0x80,0xf7,0xaf,0xa8,0xdd,0xdd, -0xc9,0xfa,0x72,0x96,0x18,0x10,0xd2,0x96,0x8b,0xd9,0xcc,0xee, -0x32,0x09,0x56,0xe8,0x9b,0xf5,0x72,0x06,0x67,0xf3,0xc5,0x7a, -0x52,0x2e,0x6f,0xa6,0x2b,0x7b,0x3c,0xff,0x44,0x12,0x15,0x35, -0x84,0xf3,0x52,0x95,0x07,0xcc,0x78,0x75,0xb1,0xb8,0x82,0x26, -0x0b,0x7f,0x4e,0x90,0xe4,0x6f,0x60,0xb3,0xa2,0x5a,0x89,0x50, -0x86,0x60,0x8c,0x83,0x08,0x3c,0xd7,0x0d,0xa9,0x23,0xed,0xf4, -0x20,0xb0,0x2e,0x8f,0x46,0x9a,0xcc,0x70,0xe8,0xec,0x21,0x05, -0xa6,0x8a,0x3d,0xb8,0x5c,0xe3,0x4b,0xaa,0xb4,0x76,0xfe,0xa6, -0x32,0xba,0x04,0x8c,0xa4,0xd0,0xe1,0x89,0xe1,0xf4,0xb1,0xad, -0xfd,0xe2,0xc6,0xce,0x80,0xe2,0xec,0xdc,0x3c,0xeb,0x96,0x60, -0xd7,0xc1,0xae,0x54,0x38,0x43,0xd9,0xee,0xdb,0x9d,0x6b,0x4b, -0xcf,0xde,0x33,0xe2,0x1e,0xb4,0x48,0x28,0xbd,0x4e,0x90,0xae, -0x74,0x08,0x1a,0x1c,0xa7,0xd8,0xed,0xf1,0xfb,0xe5,0xf0,0x6a, -0x82,0x56,0xfc,0x81,0xc8,0xc0,0xc9,0xbe,0xd5,0xb8,0xbd,0xa6, -0xb0,0xab,0xd6,0x2f,0x2d,0x70,0xa7,0xf5,0x3b,0x3a,0x89,0x1a, -0x2c,0x5d,0x94,0xb3,0x9c,0xeb,0xf0,0x1e,0x08,0x02,0xb7,0x18, -0xda,0x7d,0x76,0x49,0x6c,0x6f,0x87,0xdd,0x9b,0xd7,0x6b,0x69, -0x59,0x44,0xa0,0x10,0x16,0x15,0x90,0xdc,0xe0,0x8b,0x30,0x52, -0xd1,0xc9,0x78,0x53,0x2e,0xd7,0xe2,0x65,0xf5,0x65,0xcf,0x4c, -0x16,0xcb,0x8f,0xf2,0xd9,0x3f,0x61,0xdf,0x40,0xcf,0xd2,0x68, -0x35,0xa4,0x9c,0xec,0x48,0x61,0xf1,0x75,0x46,0x3b,0x0e,0x57, -0x27,0x3b,0x52,0x25,0x9f,0x72,0xff,0xb8,0xf9,0x52,0x45,0x91, -0x8b,0xe4,0x33,0x98,0xd6,0xf3,0x8b,0xdb,0xc4,0x68,0x80,0xdf, -0xa6,0xab,0xe9,0xbb,0xa9,0xdd,0x0c,0x6f,0x0b,0x5c,0x91,0x47, -0x98,0x7a,0xb4,0x9a,0x80,0xb5,0x12,0x03,0xae,0x87,0xef,0x66, -0x65,0x03,0x10,0x53,0x8f,0x26,0x76,0x09,0x3a,0x40,0xc0,0x7a, -0x03,0xc9,0x9a,0x91,0x1d,0x40,0x64,0x59,0x6e,0xca,0xa1,0x5d, -0x46,0x9a,0x0e,0xce,0x83,0x07,0xd2,0xc1,0xca,0x38,0x2a,0x76, -0x25,0xfc,0x50,0x53,0xd9,0x7b,0xb6,0x6a,0x47,0x78,0x3e,0x5a, -0xdc,0xf8,0x3e,0xd5,0x68,0xff,0x08,0x6f,0x22,0x22,0x58,0x3c, -0x7a,0x0e,0xe9,0xd9,0x0d,0x8a,0x89,0xf5,0x92,0x8d,0xae,0x7f, -0x9a,0x74,0xb1,0xae,0x78,0xef,0x64,0xd9,0x85,0x5d,0x9e,0xce, -0x84,0xac,0x8e,0x68,0xc7,0xf8,0x24,0x33,0x6a,0xc1,0xdb,0x45, -0xd3,0x56,0x14,0x7b,0x8d,0xd2,0xb5,0xd5,0x68,0x54,0xe5,0xee, -0x62,0x4e,0xe7,0xd0,0x48,0x97,0x3e,0x23,0x0b,0x25,0xdc,0x55, -0xd6,0x3f,0x82,0xc2,0xd2,0x2e,0xd4,0x34,0x4e,0xd0,0x34,0x70, -0x4e,0x7a,0x3d,0x64,0x7f,0x4e,0x0a,0x8d,0x31,0x37,0xe9,0xc2, -0x9d,0x98,0x44,0x90,0x63,0x51,0xbe,0x29,0x38,0x11,0x45,0xf1, -0x17,0x72,0x55,0xd0,0xef,0x55,0x6c,0x33,0xc1,0xae,0xaa,0x90, -0xa8,0x1d,0x5b,0xfe,0x74,0x6e,0x35,0xe1,0x4c,0x84,0xaf,0xbb, -0xf3,0xb3,0x13,0x7b,0x52,0xef,0xf5,0xce,0x2b,0x6f,0x65,0xc1, -0xc0,0xb7,0x0d,0xdf,0x7e,0xe4,0xad,0xca,0xf5,0x3e,0xbb,0xb7, -0xa2,0x23,0x09,0x17,0x65,0xc6,0xd3,0x72,0x66,0x25,0x0f,0xe4, -0x67,0x56,0x78,0x90,0x92,0xb7,0x0e,0x9b,0xcd,0x04,0x09,0xf9, -0x67,0x8b,0x2f,0x9e,0x02,0x7f,0x1e,0x6e,0x6d,0x03,0x7a,0x66, -0x3e,0xbd,0x28,0xbd,0x17,0xad,0xde,0xde,0x62,0xe8,0x28,0x46, -0x2b,0xb4,0x0b,0x01,0x99,0xac,0x6c,0xa3,0x4a,0xbd,0x80,0xe8, -0x35,0xd2,0x0c,0xf6,0x79,0xe4,0xbc,0x72,0xd9,0x6e,0xa4,0xb6, -0x9b,0x7e,0xf9,0xef,0xce,0x15,0x57,0xf2,0x59,0xaf,0xf7,0xed, -0x7f,0xbc,0xf8,0xc2,0x6e,0x37,0x9f,0x7d,0xf7,0xdd,0x37,0xbd, -0x5e,0x0f,0x7e,0x5d,0x7c,0xd1,0xeb,0xbd,0x78,0xa1,0x7e,0x41, -0x7f,0x91,0x2d,0x0a,0xfb,0x6a,0xb1,0x0d,0xa4,0x60,0x42,0xf0, -0x0b,0xdd,0x26,0xe5,0xff,0x66,0xa8,0x16,0x3f,0x51,0xfd,0x9e, -0xa4,0xf6,0xd4,0x0c,0x63,0x41,0x08,0xb7,0x0e,0xc1,0xfe,0xd2, -0x3e,0x5d,0x08,0x79,0x47,0xbe,0x1b,0xe4,0x26,0x3b,0x41,0x03, -0x30,0x32,0x49,0x86,0x06,0xfa,0x10,0x26,0xf7,0x41,0x92,0x13, -0x97,0x7c,0x07,0xbe,0x26,0xda,0x60,0xc0,0xc6,0xa1,0x41,0xe7, -0xb3,0x97,0x2f,0x5f,0xda,0xea,0x52,0x2a,0xae,0x16,0xc9,0xea, -0x57,0xf6,0x3f,0x67,0x83,0xc3,0xed,0x81,0xc8,0xdc,0x76,0x5c, -0xc0,0xc6,0x20,0x18,0xaa,0x2a,0xe2,0xb0,0x66,0xb7,0xcd,0x77, -0xbe,0xb7,0x88,0xa8,0x9d,0xba,0xdb,0x17,0x3a,0x79,0xef,0x10, -0x9b,0x5b,0x07,0x7a,0x5b,0x9b,0x74,0x12,0xb6,0xe9,0xa8,0x5f, -0x89,0x55,0x11,0xd7,0xfb,0x65,0x65,0xa8,0x17,0x18,0x93,0xa7, -0xc7,0xfe,0xb9,0x54,0xb9,0x11,0x23,0xa7,0x02,0x9f,0xd6,0xc6, -0x4f,0x6d,0x4c,0x85,0xae,0x75,0xa4,0x01,0x27,0x3d,0x69,0x81, -0x8c,0x11,0xcc,0x4f,0x1c,0x0d,0xbb,0x09,0x9f,0xfe,0x6f,0x00, -0x00,0x00,0xff,0xff,0xdf,0x8a,0x9a,0x2e,0x24,0xda,0x01,0x00, - })) + var empty [0]byte + sx := (*reflect.StringHeader)(unsafe.Pointer(&_scripts_stats_scripts_js)) + b := empty[:] + bx := (*reflect.SliceHeader)(unsafe.Pointer(&b)) + bx.Data = sx.Data + bx.Len = len(_scripts_stats_scripts_js) + bx.Cap = bx.Len + + gz, err := gzip.NewReader(bytes.NewBuffer(b)) if err != nil { panic("Decompression failed: " + err.Error()) } - var b bytes.Buffer - io.Copy(&b, gz) + var buf bytes.Buffer + io.Copy(&buf, gz) gz.Close() - return b.Bytes() + return buf.Bytes() } + func init() { go_bindata["/scripts/stats-scripts.js"] = scripts_stats_scripts_js } diff --git a/mod/dashboard/resources/stats.html.go b/mod/dashboard/resources/stats.html.go index 9dd24ef8825..517d79511c6 100644 --- a/mod/dashboard/resources/stats.html.go +++ b/mod/dashboard/resources/stats.html.go @@ -4,73 +4,36 @@ import ( "bytes" "compress/gzip" "io" + "reflect" + "unsafe" ) +var _stats_html = "\x1f\x8b\x08\x00\x00\x09\x6e\x88\x00\xff\x8c\x54\xcb\x6e\xdb\x30\x10\xbc\xe7\x2b\x36\xba\xd3\x3c\x14\x45\x92\x82\x12\x90\x16\x39\xe4\x56\xa0\x28\xd0\x22\x08\x02\x9a\x5c\x5b\x4c\x25\x2e\x4b\xae\xec\xfa\xef\x4b\x4a\x56\xa2\x3c\x5a\xd8\x07\x6b\xc5\xe5\xcc\x3e\x66\x6c\x75\x6e\xc9\xf0\x21\x20\xb4\xdc\x77\xcd\x99\x3a\x17\xe2\xce\x6d\xa0\x63\xb8\xbd\x81\x8b\xfb\x06\xc6\x8f\x2a\x59\x30\x9d\x4e\xa9\xae\x3c\x89\xc7\x94\x6f\x08\x87\x57\xd3\xe3\x72\x7a\x5c\x54\x0d\xa8\xf3\x3b\xf4\xd6\x6d\xee\x85\x78\x66\x5b\x52\x9d\xc0\xf6\x1f\x9a\xcb\x53\x68\xfe\x85\xdf\xf2\x91\xa2\x1c\x34\xef\xe0\x47\xa0\x10\x2f\xc0\xa5\x0e\x6a\x5b\x82\x1c\xf6\xc8\x1a\x4c\xab\x63\x42\xae\xab\x81\x37\x22\x77\xbb\x48\xb5\xcc\x41\xe0\xef\xc1\xed\xea\xea\x87\xf8\x7e\x2d\xbe\x50\x1f\x34\xbb\x75\x87\x15\x18\xf2\x8c\x3e\xe3\x6e\x6f\x6a\xb4\x5b\x9c\x91\xec\xb8\xc3\x06\xd9\x58\xf8\x1c\x69\x9f\x30\x2a\x39\x9d\x2d\x98\xbd\xee\xb1\xae\x2c\x26\x13\x5d\x60\x47\x7e\xc1\x57\xbd\xbd\xb8\x73\xb8\x0f\x14\x79\x71\x6b\xef\x2c\xb7\xb5\xc5\x9d\x33\x28\xc6\x97\x19\x97\x67\x86\xaf\x9d\x36\x08\x1b\x9d\xb3\xe4\x57\xf9\x0b\xb4\xb7\xa0\x43\xe8\x50\x30\x0d\xa6\x15\x63\x22\xf8\x2d\x38\x0f\xdc\x22\x44\x22\x06\xeb\x22\x1a\xa6\x78\x80\xb2\xac\xb3\x27\x6d\x3a\xe7\x7f\x41\xc4\xae\xae\x12\x1f\x3a\x4c\x2d\x62\xee\xa5\x8d\xb8\x99\x4f\x64\xaf\x9d\x5f\x99\x94\xb7\x7e\xa6\xe4\xbc\x63\xb5\x26\x7b\x00\xbf\x15\xb9\x72\x5d\x95\x9d\x7c\x63\xcd\x69\xd1\xe9\x0b\x7b\x1e\x0b\xaa\x30\xeb\xb8\x1e\x17\xd8\x66\xf4\xa1\x6a\x7e\xd2\x00\x3a\x22\x0c\xc9\xe5\xb6\xb5\x07\x95\x38\x92\xdf\x36\x34\xb0\xd5\x8c\x56\xc9\xe3\x01\x4c\xb8\xb8\xca\x7b\x40\x9d\x10\x94\x3e\x36\x5b\x14\xfd\x24\xe5\x82\x77\x65\xa8\x97\x55\x33\x84\x6d\xd4\x16\xe1\x40\x43\x9c\xe1\x4a\xea\x06\x98\xc0\xf5\x21\xd2\xee\x98\xc3\x3f\x01\xa3\x43\x6f\x70\xa5\x64\x98\x07\x59\x58\xec\x9d\xd1\xae\x9e\x47\x9b\x04\x87\x14\x4d\x9e\x8e\xf6\x18\x1f\x72\xfd\x40\x3e\x6b\x9a\x24\xa6\x8f\x22\xb5\xae\x7f\x0a\x56\xc5\xc5\x79\xac\x11\x74\x1a\xc7\x63\x22\xff\x41\x76\x6e\x3d\x45\xab\x3e\xcb\xf2\x96\xe5\xfd\x8e\xe1\xda\xda\x69\xca\xe4\x18\x81\xe2\xe8\x18\x67\x74\x71\xe8\x6c\x3d\x68\x31\x8b\x30\xfd\x98\x32\xcc\xba\x1d\x38\x5b\xc4\xb5\x0f\x69\x14\xb7\xe8\x5d\x0c\x3b\x09\x3e\x6b\x2d\xf3\xc5\xe6\xd9\x51\xcb\x19\xa6\x38\xc9\x11\x2e\x7a\xb2\x43\x36\xd4\xab\xa6\x4f\x41\x1e\xdf\x5e\x21\x95\x2c\x26\x1c\x5d\x59\xfe\x14\xff\x06\x00\x00\xff\xff\x95\x89\x83\x4d\x24\x05\x00\x00" + // stats_html returns raw, uncompressed file data. func stats_html() []byte { - gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ -0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0x8c,0x54, -0xcb,0x6e,0xdb,0x30,0x10,0xbc,0xe7,0x2b,0x36,0xba,0xd3,0x3c, -0x14,0x45,0x92,0x82,0x12,0x90,0x16,0x39,0xe4,0x56,0xa0,0x28, -0xd0,0x22,0x08,0x02,0x9a,0x5c,0x5b,0x4c,0x25,0x2e,0x4b,0xae, -0xec,0xfa,0xef,0x4b,0x4a,0x56,0xa2,0x3c,0x5a,0xd8,0x07,0x6b, -0xc5,0xe5,0xcc,0x3e,0x66,0x6c,0x75,0x6e,0xc9,0xf0,0x21,0x20, -0xb4,0xdc,0x77,0xcd,0x99,0x3a,0x17,0xe2,0xce,0x6d,0xa0,0x63, -0xb8,0xbd,0x81,0x8b,0xfb,0x06,0xc6,0x8f,0x2a,0x59,0x30,0x9d, -0x4e,0xa9,0xae,0x3c,0x89,0xc7,0x94,0x6f,0x08,0x87,0x57,0xd3, -0xe3,0x72,0x7a,0x5c,0x54,0x0d,0xa8,0xf3,0x3b,0xf4,0xd6,0x6d, -0xee,0x85,0x78,0x66,0x5b,0x52,0x9d,0xc0,0xf6,0x1f,0x9a,0xcb, -0x53,0x68,0xfe,0x85,0xdf,0xf2,0x91,0xa2,0x1c,0x34,0xef,0xe0, -0x47,0xa0,0x10,0x2f,0xc0,0xa5,0x0e,0x6a,0x5b,0x82,0x1c,0xf6, -0xc8,0x1a,0x4c,0xab,0x63,0x42,0xae,0xab,0x81,0x37,0x22,0x77, -0xbb,0x48,0xb5,0xcc,0x41,0xe0,0xef,0xc1,0xed,0xea,0xea,0x87, -0xf8,0x7e,0x2d,0xbe,0x50,0x1f,0x34,0xbb,0x75,0x87,0x15,0x18, -0xf2,0x8c,0x3e,0xe3,0x6e,0x6f,0x6a,0xb4,0x5b,0x9c,0x91,0xec, -0xb8,0xc3,0x06,0xd9,0x58,0xf8,0x1c,0x69,0x9f,0x30,0x2a,0x39, -0x9d,0x2d,0x98,0xbd,0xee,0xb1,0xae,0x2c,0x26,0x13,0x5d,0x60, -0x47,0x7e,0xc1,0x57,0xbd,0xbd,0xb8,0x73,0xb8,0x0f,0x14,0x79, -0x71,0x6b,0xef,0x2c,0xb7,0xb5,0xc5,0x9d,0x33,0x28,0xc6,0x97, -0x19,0x97,0x67,0x86,0xaf,0x9d,0x36,0x08,0x1b,0x9d,0xb3,0xe4, -0x57,0xf9,0x0b,0xb4,0xb7,0xa0,0x43,0xe8,0x50,0x30,0x0d,0xa6, -0x15,0x63,0x22,0xf8,0x2d,0x38,0x0f,0xdc,0x22,0x44,0x22,0x06, -0xeb,0x22,0x1a,0xa6,0x78,0x80,0xb2,0xac,0xb3,0x27,0x6d,0x3a, -0xe7,0x7f,0x41,0xc4,0xae,0xae,0x12,0x1f,0x3a,0x4c,0x2d,0x62, -0xee,0xa5,0x8d,0xb8,0x99,0x4f,0x64,0xaf,0x9d,0x5f,0x99,0x94, -0xb7,0x7e,0xa6,0xe4,0xbc,0x63,0xb5,0x26,0x7b,0x00,0xbf,0x15, -0xb9,0x72,0x5d,0x95,0x9d,0x7c,0x63,0xcd,0x69,0xd1,0xe9,0x0b, -0x7b,0x1e,0x0b,0xaa,0x30,0xeb,0xb8,0x1e,0x17,0xd8,0x66,0xf4, -0xa1,0x6a,0x7e,0xd2,0x00,0x3a,0x22,0x0c,0xc9,0xe5,0xb6,0xb5, -0x07,0x95,0x38,0x92,0xdf,0x36,0x34,0xb0,0xd5,0x8c,0x56,0xc9, -0xe3,0x01,0x4c,0xb8,0xb8,0xca,0x7b,0x40,0x9d,0x10,0x94,0x3e, -0x36,0x5b,0x14,0xfd,0x24,0xe5,0x82,0x77,0x65,0xa8,0x97,0x55, -0x33,0x84,0x6d,0xd4,0x16,0xe1,0x40,0x43,0x9c,0xe1,0x4a,0xea, -0x06,0x98,0xc0,0xf5,0x21,0xd2,0xee,0x98,0xc3,0x3f,0x01,0xa3, -0x43,0x6f,0x70,0xa5,0x64,0x98,0x07,0x59,0x58,0xec,0x9d,0xd1, -0xae,0x9e,0x47,0x9b,0x04,0x87,0x14,0x4d,0x9e,0x8e,0xf6,0x18, -0x1f,0x72,0xfd,0x40,0x3e,0x6b,0x9a,0x24,0xa6,0x8f,0x22,0xb5, -0xae,0x7f,0x0a,0x56,0xc5,0xc5,0x79,0xac,0x11,0x74,0x1a,0xc7, -0x63,0x22,0xff,0x41,0x76,0x6e,0x3d,0x45,0xab,0x3e,0xcb,0xf2, -0x96,0xe5,0xfd,0x8e,0xe1,0xda,0xda,0x69,0xca,0xe4,0x18,0x81, -0xe2,0xe8,0x18,0x67,0x74,0x71,0xe8,0x6c,0x3d,0x68,0x31,0x8b, -0x30,0xfd,0x98,0x32,0xcc,0xba,0x1d,0x38,0x5b,0xc4,0xb5,0x0f, -0x69,0x14,0xb7,0xe8,0x5d,0x0c,0x3b,0x09,0x3e,0x6b,0x2d,0xf3, -0xc5,0xe6,0xd9,0x51,0xcb,0x19,0xa6,0x38,0xc9,0x11,0x2e,0x7a, -0xb2,0x43,0x36,0xd4,0xab,0xa6,0x4f,0x41,0x1e,0xdf,0x5e,0x21, -0x95,0x2c,0x26,0x1c,0x5d,0x59,0xfe,0x14,0xff,0x06,0x00,0x00, -0xff,0xff,0x95,0x89,0x83,0x4d,0x24,0x05,0x00,0x00, - })) + var empty [0]byte + sx := (*reflect.StringHeader)(unsafe.Pointer(&_stats_html)) + b := empty[:] + bx := (*reflect.SliceHeader)(unsafe.Pointer(&b)) + bx.Data = sx.Data + bx.Len = len(_stats_html) + bx.Cap = bx.Len + + gz, err := gzip.NewReader(bytes.NewBuffer(b)) if err != nil { panic("Decompression failed: " + err.Error()) } - var b bytes.Buffer - io.Copy(&b, gz) + var buf bytes.Buffer + io.Copy(&buf, gz) gz.Close() - return b.Bytes() + return buf.Bytes() } + func init() { go_bindata["/stats.html"] = stats_html } diff --git a/mod/dashboard/resources/styles-main.css.go b/mod/dashboard/resources/styles-main.css.go index 8896b86473b..dfc57144c95 100644 --- a/mod/dashboard/resources/styles-main.css.go +++ b/mod/dashboard/resources/styles-main.css.go @@ -4,213 +4,36 @@ import ( "bytes" "compress/gzip" "io" + "reflect" + "unsafe" ) +var _styles_main_css = "\x1f\x8b\x08\x00\x00\x09\x6e\x88\x00\xff\xe4\x5a\xff\x8f\xa3\xb8\x15\xff\x57\xd0\x9e\x56\xb7\x53\x85\x11\x21\x5f\xe6\x16\x74\x52\xf7\xe6\x3a\x6d\xa5\x5e\x55\xe9\xee\x87\x4a\x55\x7f\x70\xc0\x49\xac\x05\x4c\x8d\x33\x5f\x16\xcd\xff\xde\x67\x1b\x83\x0d\x66\x02\x99\xdd\xaa\xba\xc9\x6a\x35\x8e\xb1\x9f\xdf\xf7\xf7\x79\x26\x3b\x9a\x3e\xd5\x39\x62\x07\x52\x44\xc1\xf3\x35\xe6\x49\xea\x27\xb4\xe0\x88\x14\x98\xd5\x3b\x94\x7c\x3e\x30\x7a\x2a\xc4\x64\x46\x59\xf4\xdd\x7e\xbf\x8f\x77\x94\xa5\x98\x45\xcb\xf2\xd1\xab\x68\x46\x52\xef\xbb\x34\x4d\x9b\x59\x9f\xa1\x94\x9c\xaa\x68\x53\x3e\xc2\xcc\xa3\x5f\x1d\x51\x4a\x1f\x22\x76\xd8\xa1\x0f\xc1\x42\xfc\xbb\x5e\xae\x3f\x06\xe1\x95\x17\x78\x82\xc0\x0a\xd6\xed\xe1\x3c\x7f\x8f\x72\x92\x3d\x45\xef\xfe\x82\xb3\x7b\xcc\x49\x82\xbc\xbf\xe3\x13\x7e\xb7\x68\xbf\x2f\x3e\x31\x82\xb2\x45\x85\x8a\xca\xaf\x30\x23\xfb\x98\xde\x63\xb6\xcf\x80\xfa\x91\xa4\x29\x2e\xd4\x79\xe4\x0b\x29\x0e\x51\xc3\x0c\xcc\xc4\x7e\x4e\xbf\xf8\xee\x47\x25\xad\x08\x27\xb4\x88\x18\xce\x10\x27\xf7\x38\x3e\x01\x65\xa0\x9e\xe1\x84\x47\x05\x2d\x70\xec\x3f\xe0\xdd\x67\xc2\xfd\xe1\x03\x41\xd6\x31\x5b\x0d\x26\x9f\x51\xdd\x28\x2f\x5c\xde\x6c\x3f\xdd\xc6\x1c\x3f\x72\x3f\xc5\x09\x65\x48\x9e\xae\x16\x45\x47\x21\xcf\x02\x45\x28\x11\xac\xd4\xfd\x55\x60\x04\xcc\x32\x30\xcb\x33\x29\xca\x13\xff\x17\x7f\x2a\xf1\x8f\x62\xd1\xbf\x6b\x43\xd1\xa4\xa8\x30\x6f\x74\x1b\xc2\x7f\x53\xf1\x9b\x2b\x6d\xba\xa0\x67\xad\xd6\x0a\xa0\x23\x1c\x2d\xc5\xd7\x12\xa5\x29\xa8\xcb\xcf\xf0\x9e\x4b\x6b\xea\x09\x46\x0e\x47\x35\x73\xc4\x72\x18\xc2\xf8\xf9\x18\xd6\x1d\x81\x30\xec\x5b\xf5\x57\x7a\x62\x09\xf6\x7e\x05\xe3\x79\xff\x60\xf4\xdd\x62\x8e\x9d\x25\xa5\x07\x75\xd8\x26\x08\x62\xed\xae\x20\x66\x18\x74\x9c\xb5\xfe\xbb\x3b\x71\x4e\x8b\x3a\x25\x55\x99\xa1\x27\x50\x89\xd0\x9a\xbf\xcb\x68\xf2\xb9\x5d\xbb\x05\xdd\x2c\x05\x9b\x8a\x18\xb8\x03\xec\xc9\x41\x31\x86\x1a\xd6\x5a\x8a\xe6\xec\x35\x9c\x2d\x49\x35\x82\x2f\xaf\xd7\xe1\x0f\x9b\x9b\xe5\x3a\xfc\xa8\x6c\x8a\x32\x72\x28\xa2\x04\x17\x1c\xb3\xf8\xe1\x48\x38\xf6\xab\x12\x25\x18\x2c\xfc\xc0\x50\x19\x83\x7d\x85\x84\x59\xb3\x30\x07\xb7\xcd\x70\x9c\x9c\x58\x05\xde\x51\x52\x22\xf7\x0d\xa2\x8b\x33\x50\x45\x89\x18\x90\xed\x99\x4d\x30\xf8\x15\x1c\x34\xf6\xe9\x70\x6e\x30\xa1\xb5\x6e\xfa\x90\xd3\xed\x02\xe5\x74\xe1\x66\xb3\xd0\xff\xaf\xc3\xab\x85\xdb\x27\xc3\xcd\x95\x65\x35\x73\xec\x57\x39\xca\xb2\xda\x70\xb3\xce\xd4\xde\x32\x18\xf8\xac\x45\xc8\x2f\x19\x01\x96\x9f\x1c\x39\x0c\xac\xf6\xd3\xed\xa7\xd8\xc8\x68\xd2\x7a\x8d\x28\xc2\xad\x56\x67\xb8\xd4\xc4\x75\xac\x0e\xcf\x08\xc3\x9b\x2d\x4a\x9b\x4d\x25\x2d\x45\x6c\x1b\xcb\xa2\xef\x56\xab\x95\x23\x08\xb5\x7c\x4b\x29\xac\x4e\x4f\x68\x07\x9e\x70\xe2\x38\xe6\xb4\x8c\x56\x1f\xe1\xd1\x17\x9f\x40\x3a\x78\x8c\x3e\xc2\xc7\x14\xa4\x17\xc4\x86\x85\x02\xa9\x7a\xa1\xb5\x7e\x4e\xd0\x71\x22\xd3\x90\xc5\xb0\x8f\x19\xa3\xcc\xb3\xe7\x44\x75\x00\x57\xd4\x39\xed\xee\x6e\x75\xbb\x5e\x59\x71\x72\x03\x71\x62\xba\x8f\xd0\xae\xd3\x4f\xd5\x83\xbe\x9f\x36\xb3\xd5\x60\xb2\xc7\x5c\x41\x79\x72\xac\x1f\x48\xca\x8f\x2a\x56\x75\x58\x9a\x61\x21\x83\x67\x4f\x59\x1e\x31\xca\x11\xc7\x1f\xd6\x9b\x14\x1f\xae\x1c\xba\x6d\x12\x81\x50\xb1\xbf\xe9\x12\x83\xcc\x7f\x52\x97\x3d\xe3\x89\x85\x41\xac\xb2\xa1\x30\x97\xcd\x9d\xcd\xaa\x5c\x55\x4b\x52\x37\x37\xfd\xa5\xea\xa9\xe7\x12\x4e\xee\x38\x47\x5c\x2c\xaa\x1b\x3e\x82\x01\x71\xf1\xd4\x49\x7b\x8c\x73\x61\xe0\x3d\x61\x79\x6d\x28\xc4\xa0\x7b\xc4\x08\x7c\x56\xab\x3d\x08\xde\x3b\xaa\xe8\xdc\x42\x6c\x91\xbe\x96\x59\xcf\x8a\x15\x8c\xb1\xa9\x7e\x6d\xdb\x83\x08\x1c\x70\xc5\x0f\x22\x29\x23\xb6\x90\xa2\x02\xbb\x6a\xa0\x32\xfa\x42\xba\xa9\x5f\xc1\xf4\x87\xe0\xfd\x42\x90\xba\x32\xe7\x84\x04\x0b\x81\x5f\xae\xae\x5c\x47\x28\xca\xdd\x49\x82\xba\xa0\xe1\x05\x72\x93\x27\xb6\xdb\x1b\xc1\x6f\xe7\x6f\xba\xf4\xb4\xe1\x06\x2d\xf6\x70\xdb\x9e\x64\x50\x5e\xa2\x92\xd1\x03\x49\xa3\x9f\xff\xf9\xd7\x1c\x1d\xf0\x6f\x3a\x3c\xae\x7f\x21\x09\xa3\x15\xdd\xf3\xeb\x96\x5a\xc5\x11\xe3\xb7\x42\x57\x15\x67\x3f\x7e\x2f\x48\xc2\xe7\xfb\x85\x87\x8b\xd4\x98\x4e\xe5\x07\xa6\xff\xdc\x6c\xfc\x4d\x20\x93\xa0\xcd\x98\x02\x61\xaa\x60\x91\xee\x1c\x38\x1c\x46\x23\x39\x1f\x32\xea\x89\xd3\xee\xfb\xa3\xc6\x76\x3a\xb6\x85\xc3\x19\xbe\x77\xa1\xa7\x09\x9e\x3c\x8e\x76\x19\xae\xbf\x26\x31\x8f\x0b\x17\xf6\x78\xaa\xc0\x5b\x97\x7b\x4e\x65\x89\x59\x82\x2a\x6c\x26\x68\x28\x87\x16\x9c\x90\x58\xa6\xa5\x2e\xe1\x48\x0f\x62\x6b\x08\xa6\xb4\x69\x03\xb2\xa5\x81\x84\x3a\x34\x63\xa1\xb8\x40\x57\x8a\xed\x76\xeb\xe0\x5d\x8d\xd3\xda\x82\x38\x3f\xcc\xe1\x69\x3b\x80\x89\x2e\xae\xb6\x7d\x74\x19\xf4\x81\x91\xd2\x1e\x2d\xbf\x65\x15\x31\x24\x57\x13\x9c\x67\x76\x72\x5b\xad\xde\x8f\x69\xa9\xdd\x51\xbb\x11\xc8\xe8\x7a\xe3\x2c\x51\x6e\x75\x11\x15\x55\xdc\xac\xa0\xe0\x8e\x63\x6c\x0a\xc4\x41\x01\x1a\xdb\xac\x06\xce\xb3\x1b\x6f\x8c\x20\x9d\x57\xdc\x4f\x8e\x24\x4b\x17\xa3\x76\x37\x57\xd5\x96\x7d\x96\x2f\x53\xcf\xd0\x04\xe2\xdd\xa2\x7a\xe8\x21\xfd\xbe\x53\xd7\x2b\x86\xef\x09\x7e\x68\xbe\xa5\xd0\x15\xd0\xc3\x00\x43\x0d\x0b\xb9\xa2\xdb\x65\x9b\x06\x52\xc9\xf0\x72\x43\xa5\xf5\x2b\x9a\xd0\x36\xd4\x28\x20\x7d\xc2\x9f\x60\x94\x43\xd5\xd4\xf1\x23\x03\xc0\x02\x23\x92\x5b\x90\x8d\x42\x4a\x80\xe5\x80\x70\xe3\xb1\x79\xc7\xbe\xf4\xd4\xb4\x83\xcb\x4d\x90\x57\xf1\xf8\x13\xc7\x5e\x4e\x72\xa1\xf7\xfd\xa9\x90\x3e\x14\x61\x48\x48\x80\x23\x7d\x7a\xe2\xf1\xb4\x65\x33\x0c\x65\x7d\xf1\x73\x5c\x55\x50\x6e\x6a\xbb\xdb\x0a\xe7\xd9\xde\x26\xa9\x90\x78\xe5\x21\x2b\x61\xad\xd6\xb6\x95\xdd\x2d\x97\x89\xed\xfa\xde\xad\x86\x19\xa9\x78\x6d\xb9\xce\xec\xdb\x85\xf6\x7a\x42\x96\x34\xb3\x80\x0d\x7c\xd6\x7d\x7e\x33\x54\xb9\x0b\xa7\x8e\x2e\xe3\x4f\x9f\xee\x56\x77\x77\x23\xbb\xd1\x75\x4a\x18\xec\xa4\xd0\x04\xf5\xe0\xf9\xc8\x0e\xce\xd4\x25\x84\x56\x34\x1c\xcc\xb1\x57\xdd\x1f\xea\xe5\x3d\xa9\xc8\x8e\x64\xc2\xc1\xe5\x10\x94\x08\x80\x22\x83\x12\x9e\xbe\xc8\xbc\xa2\x51\x9b\x25\x4e\xa5\x2c\xd9\xce\xbd\x64\x1d\x08\x24\x57\x07\x3f\xe1\x34\xc9\xb1\x79\xa2\xe4\x54\xc0\x97\x49\x7a\x9e\x42\x0b\x2a\xdf\x04\x46\x1a\x6d\x0a\x12\xbd\x26\xbf\xa7\xbb\xd6\xfb\x1b\x9a\x8c\x3e\x54\xad\x19\x9a\x4c\xaf\x1d\xbc\xeb\x1f\x84\x3f\x98\xd3\x8d\x66\xe5\xf0\x8c\x66\xdd\xad\x65\x93\x33\x45\xb7\xe3\x68\x0a\xdd\x4c\x5a\xe1\xea\xc3\x1f\x8c\x74\x95\x93\xfc\x69\x42\xa6\xf1\xa6\x50\x02\xdc\x87\xcf\xd1\x11\x0f\x06\x06\xd2\x41\x0b\xc8\xc5\xdb\x5a\xba\x3a\x63\x0e\x71\x2f\xb0\x4b\x90\xb1\x41\xe7\x9b\x4a\xd4\xb2\x54\x6e\x54\x2b\x77\xbb\x5d\xb3\x0c\x0e\xfb\x26\x26\xe8\x38\x87\x13\x2c\x19\x43\x43\xc6\x1b\x4b\x46\x98\x9c\x25\x22\xac\x9f\x20\xe1\x04\xdf\xf4\xac\x47\x7e\x89\xf8\xb1\x1e\x8a\xa7\xda\x5f\xc1\xbe\xae\xd0\x4a\xe0\x46\x23\xe2\x46\x6e\x33\xac\x07\xb3\x4e\xf5\xe4\xd5\xe8\x57\xc0\xf4\xd3\x8f\xaf\xd0\x3d\x76\x08\xab\x58\xd8\xac\x5b\x69\xfd\x4d\x77\xa9\x20\x65\x1d\xdc\xb6\x3b\x82\x41\x10\xb7\x63\x6a\x78\xb6\x22\x7f\x73\x46\x73\x93\xe8\x59\x06\x54\x74\xb7\xe1\x74\xc2\x46\xc8\x8e\xb1\x29\xb5\xf0\x5a\x7a\x0e\x36\x5f\x56\xa6\x8d\x29\xc6\x5c\x53\xfa\x8b\x89\xdd\xc4\xf7\xf3\x9d\xa9\x72\xe2\xd9\xe8\xc0\xfd\x6a\xc5\xf2\x5c\xb9\x56\xb1\x66\xf5\x5c\x33\x64\xf5\xe0\xef\x62\xc6\x72\xd1\x20\x21\x86\x91\x0d\x7d\xda\x9c\x65\x82\xe4\x5b\x00\x5f\x14\x60\xfd\xe2\xdd\xdf\xc8\x0e\x2b\xec\xe9\xfd\x42\x0b\xfa\x6e\x71\x4b\x4f\x8c\x60\xb6\xc8\xe1\x9b\xbc\x00\x1f\x6b\xdf\xcd\x3b\x72\x60\xc1\x97\xb7\xe4\x43\xf3\x0c\x6e\x35\xf5\x0d\x79\x0c\x98\x54\x40\x83\x0b\x0c\x70\x69\x51\x9b\xaa\xd8\x09\x24\x5a\x65\x4f\xaf\x91\x4e\x74\x8c\xf3\x92\xeb\x2b\x96\xb6\x1d\x31\xb4\x3c\x7c\x37\x31\x06\x40\x67\x8b\xa2\x8e\x9e\x0c\x16\xc6\xd9\xef\x77\x08\x42\x9a\x70\xf3\x3e\xee\x1a\xe4\xe7\x3f\xe6\x18\xa0\xbf\xf7\x21\x47\x8f\xbe\x92\x0e\x90\x6c\xf9\x78\x55\x4f\x38\x51\xc2\xf9\x4e\x25\x97\x8a\x2b\xc9\xc8\x88\xf4\xa5\x66\xbf\x45\x63\x17\x8e\x36\x76\xe1\xff\xae\xb1\x9b\x06\xcc\x3a\x75\x04\x6f\x44\x15\xbd\x4a\xd2\xd6\x8d\x57\x66\x13\xed\x53\xdf\xe8\xae\xe0\xff\x4f\x8f\xa3\x2e\x35\xd0\xec\xef\x48\x21\x5d\x06\x83\x89\x57\x64\xb0\xcd\x19\x7f\x73\xa9\x72\xd3\x16\xde\x73\xbb\xa7\x7a\xeb\xeb\xe9\x5c\x56\x43\xa6\xb9\xce\x3c\xee\x86\x54\x9c\xbc\xbd\xa2\x87\xbd\x0c\x36\x38\x08\x0c\x40\x83\x54\x98\x9b\xa9\x8a\x23\x5e\x79\xc7\xd0\x2c\xaa\x7e\xd7\x29\x88\x57\x13\x88\x37\xf7\x1e\x94\x39\x70\xb1\x7c\xf7\xd7\x75\x6c\xcb\xad\xfd\xea\xd9\x4d\xc7\xba\x4e\xa1\xcc\x07\x88\x97\xbb\x7f\x8c\xa1\x51\x61\xd8\x36\xce\xcb\xb0\xd7\xc1\xaf\xc7\xb8\x75\x9d\xa2\xba\xdf\x5e\xe7\x3b\x7d\xbb\xd1\xfe\xae\x56\xab\x97\x54\xaa\xc6\x07\x40\xab\xc7\xfa\xf2\x5f\x1d\xf5\x1a\xff\x16\xba\x35\x15\x55\x77\xc9\x26\x14\x9f\xc8\x94\x39\x36\x7e\xce\xe5\x3e\x79\xab\x6e\x37\xdb\x1b\x59\x75\xbe\x1c\x31\xf3\xc5\xd4\xd7\xc0\xd7\x8a\x53\xf3\x7d\x46\x86\x38\x2e\x92\xa7\x73\xa9\xcd\x14\x51\xdd\xcd\xfe\x2e\x9b\xb8\xbe\x98\xda\x4d\xff\x73\x82\x98\xaf\x5b\x99\xba\x78\x09\x8c\xbb\x3a\x2b\xb6\x9a\x90\x6f\x7f\xad\x75\xc1\x79\x90\xb4\x5d\x97\xce\xc9\x3a\x08\xc2\xf0\x12\x7a\x14\xea\xe4\xc1\xf5\x6b\x99\xbb\xbb\xdb\x20\x18\xe9\xe1\x5f\x26\x79\x60\x18\x17\x0e\x8a\x41\xf0\xf3\x4f\xe1\x7a\x16\xc5\x1c\x25\x47\xa1\x3f\xf1\xc3\x3a\xf3\xc5\x9c\xf5\x2e\xcc\xf1\x4b\x39\xeb\x7d\xe9\x28\x16\x1c\x39\xb4\x71\x7f\xff\x1e\x65\x27\xec\xcc\x92\x33\xfb\x9e\x41\xa0\x18\xde\x78\x06\xae\x0e\x73\x9b\xca\x44\x2f\xac\x56\x43\x19\xcf\x76\x89\x37\xf6\xbf\x89\x56\x69\xa0\x0d\xa3\x74\x9b\xba\x7c\xcb\x7a\x38\x53\x82\x55\xe5\x82\x1a\xec\xfd\xa1\x76\xdc\x56\xcf\x38\xa8\xeb\x4b\xdf\xa4\xeb\x8d\x5c\x59\xbc\x3d\xdf\xb3\x14\x71\xc6\xfb\x14\x26\x71\x78\xdf\xcc\xae\x6d\x24\xff\x6e\x74\xfa\x9d\x84\x70\xac\x8c\x31\x03\x19\xf5\x11\xbd\xdd\x23\xfc\x37\x00\x00\xff\xff\x6e\xb7\xf9\x89\xec\x2f\x00\x00" + // styles_main_css returns raw, uncompressed file data. func styles_main_css() []byte { - gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ -0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0xe4,0x5a, -0x6d,0x6f,0xe3,0xb8,0x11,0xfe,0x2b,0x42,0x0e,0x8b,0x8b,0x0b, -0x2b,0x90,0xe5,0x97,0xdc,0xca,0x38,0xa0,0x7b,0xb9,0xa6,0x2d, -0xd0,0x2b,0x0a,0xdc,0x7d,0x28,0x50,0xf4,0x03,0x2d,0xd1,0x36, -0xb1,0x92,0xa8,0x52,0x74,0x5e,0x56,0xc8,0x7f,0xef,0x90,0x14, -0x25,0x52,0xa2,0x62,0xc9,0xd9,0x2d,0x8a,0x8b,0x17,0x8b,0xc8, -0x34,0x39,0x9c,0x79,0x38,0x2f,0xcf,0xd0,0xde,0xd1,0xe4,0xb9, -0xca,0x10,0x3b,0x90,0x3c,0x0a,0x5e,0x6e,0x30,0x8f,0x13,0x3f, -0xa6,0x39,0x47,0x24,0xc7,0xac,0xda,0xa1,0xf8,0xf3,0x81,0xd1, -0x53,0x2e,0x06,0x53,0xca,0xa2,0xef,0xf6,0xfb,0xfd,0x76,0x47, -0x59,0x82,0x59,0xb4,0x28,0x9e,0xbc,0x92,0xa6,0x24,0xf1,0xbe, -0x4b,0x92,0xa4,0x1e,0xf5,0x19,0x4a,0xc8,0xa9,0x8c,0xd6,0xc5, -0x13,0x8c,0x3c,0xf9,0xe5,0x11,0x25,0xf4,0x31,0x62,0x87,0x1d, -0xba,0x0e,0xe6,0xe2,0xdf,0xcd,0x62,0xf5,0x31,0x08,0x67,0x5e, -0xe0,0x09,0x01,0x4b,0x98,0xb7,0x87,0xfd,0xfc,0x3d,0xca,0x48, -0xfa,0x1c,0x5d,0xfd,0x05,0xa7,0x0f,0x98,0x93,0x18,0x79,0x7f, -0xc7,0x27,0x7c,0x35,0x6f,0xde,0xcf,0x3f,0x31,0x82,0xd2,0x79, -0x89,0xf2,0xd2,0x2f,0x31,0x23,0xfb,0x2d,0x7d,0xc0,0x6c,0x9f, -0x82,0xf4,0x23,0x49,0x12,0x9c,0xab,0xfd,0xc8,0x17,0x92,0x1f, -0xa2,0x5a,0x19,0x18,0xd9,0xfa,0x19,0xfd,0xe2,0xbb,0x3f,0x2a, -0x68,0x49,0x38,0xa1,0x79,0xc4,0x70,0x8a,0x38,0x79,0xc0,0x2f, -0xa8,0xaa,0xed,0x0c,0x17,0xb7,0x9b,0x4f,0x77,0x5b,0x8e,0x9f, -0xb8,0x9f,0xe0,0x98,0x32,0x24,0x27,0xe6,0x34,0x87,0x49,0xd1, -0x51,0x6c,0x3d,0x47,0x11,0x8a,0xc5,0xaa,0xaa,0x3b,0x0b,0xf0, -0xc2,0x2c,0x05,0x04,0x5f,0x48,0x5e,0x9c,0xf8,0xbf,0xf8,0x73, -0x81,0x7f,0x14,0x93,0xfe,0x5d,0x19,0x98,0x90,0xbc,0xc4,0xbc, -0x86,0x21,0x84,0xff,0x26,0x46,0xeb,0x99,0x46,0x39,0xe8,0x00, -0xdb,0x00,0x06,0xe6,0xe0,0x68,0x21,0xde,0x16,0x28,0x49,0xc0, -0x32,0x3f,0xc5,0x7b,0x2e,0x81,0xd7,0x03,0x8c,0x1c,0x8e,0x6a, -0xe4,0x88,0xe5,0x63,0x08,0xcf,0x2f,0xc7,0xb0,0x6a,0x05,0x84, -0x61,0xf7,0x00,0x7e,0xa5,0x27,0x16,0x63,0xef,0x57,0xc0,0xd9, -0xfb,0x07,0xa3,0x57,0xf3,0x29,0x47,0x22,0x25,0x3d,0xaa,0xcd, -0xd6,0x41,0xb0,0xd5,0x9e,0x05,0x66,0x86,0x41,0xab,0x59,0xe3, -0x6a,0xbb,0x13,0xe7,0x34,0xaf,0x12,0x52,0x16,0x29,0x7a,0x06, -0x48,0x04,0x6a,0xfe,0x2e,0xa5,0xf1,0xe7,0x66,0xee,0x06,0xb0, -0x59,0x08,0x35,0x95,0x30,0x38,0x39,0x58,0x93,0x01,0x30,0x06, -0x0c,0x2b,0x6d,0x45,0xbd,0xf7,0x0a,0xf6,0x96,0xa2,0x6a,0xc3, -0x17,0x37,0xab,0xf0,0x87,0xf5,0xed,0x62,0x15,0x7e,0x54,0x67, -0x8a,0x52,0x72,0xc8,0xa3,0x18,0xe7,0x1c,0xb3,0xed,0xe3,0x91, -0x70,0xec,0x97,0x05,0x8a,0x31,0x9c,0xf0,0x23,0x43,0xc5,0x16, -0xce,0x57,0x58,0x98,0xd6,0x13,0x33,0xf0,0xb0,0x14,0x6f,0xe3, -0x13,0x2b,0xc1,0x3b,0x0a,0x4a,0xe4,0xba,0x5e,0x20,0x70,0x06, -0x50,0x14,0x88,0x81,0xd8,0xce,0xb1,0x09,0x05,0x41,0xb9,0xdd, -0x67,0xc2,0xfd,0x13,0x60,0x05,0x78,0xa5,0x38,0xe6,0xd2,0xa1, -0x94,0x8b,0x3a,0x46,0x4b,0xc7,0x20,0xed,0x8f,0xf5,0x06,0x34, -0xea,0xa6,0x0f,0x39,0xdd,0x2e,0x50,0x4e,0x17,0xae,0xd7,0x73, -0xfd,0xff,0x26,0x9c,0xcd,0xdd,0x3e,0x19,0xae,0x67,0xd6,0xa9, -0x99,0xcf,0x7e,0x99,0xa1,0x34,0xad,0x0c,0x37,0x6b,0x8f,0xda, -0x5b,0x04,0x3d,0x9f,0xb5,0x04,0xf9,0x05,0x23,0xa0,0xf2,0xb3, -0x23,0xdd,0xc0,0xa9,0xfd,0x74,0xf7,0x69,0x6b,0x24,0x1f,0x79, -0x7a,0xb5,0x29,0xc2,0xad,0x96,0x67,0xb4,0xd4,0xc2,0x75,0xac, -0xf6,0xf7,0x08,0xc3,0xdb,0x0d,0x4a,0xea,0x45,0x05,0x2d,0x44, -0x6c,0x1b,0xd3,0xa2,0xef,0x96,0xcb,0xa5,0x23,0x08,0xb5,0x7d, -0x0b,0x69,0xac,0xce,0x24,0x68,0x07,0x9e,0x70,0xe2,0x78,0xcb, -0x69,0x11,0x2d,0x3f,0xc2,0x47,0x5f,0x7c,0x02,0xe9,0xe0,0x29, -0xfa,0x08,0x2f,0xd3,0x90,0x4e,0x10,0x1b,0x27,0x14,0x48,0xe8, -0x05,0x6a,0xdd,0x9c,0xa0,0xe3,0x44,0xa6,0x21,0x4b,0x61,0x1f, -0x33,0x46,0x99,0x67,0x8f,0x89,0x44,0x0e,0xae,0xa8,0x73,0xda, -0xfd,0xfd,0xf2,0x6e,0xb5,0xb4,0xe2,0xe4,0x36,0x08,0x3a,0x72, -0x72,0xca,0xe3,0x63,0xf5,0x48,0x12,0x7e,0x54,0x61,0xa5,0x23, -0xc8,0xf4,0x60,0xe9,0xe7,0x7b,0xca,0xb2,0x88,0x51,0x8e,0x38, -0xbe,0x5e,0xad,0x13,0x7c,0x98,0x39,0x60,0xa8,0x63,0x56,0xa0, -0xe1,0xaf,0xdb,0x18,0x96,0xa9,0x4a,0x9a,0xdd,0xc1,0x59,0x4c, -0x0c,0xb6,0x2a,0x71,0x09,0x64,0x6d,0xed,0x6c,0x55,0xe5,0xac, -0x4a,0x8a,0xba,0xbd,0xed,0x4e,0x55,0x9f,0x7a,0x2e,0xe3,0xe4, -0x8a,0x73,0xc2,0xc5,0xa4,0xaa,0xd6,0x23,0xe8,0x09,0x17,0x9f, -0x3a,0x65,0x0f,0x69,0x2e,0xce,0x62,0x4f,0x58,0x56,0x19,0x80, -0x18,0x72,0x8f,0x18,0x81,0x7b,0x69,0xd8,0x83,0xe0,0x43,0xbf, -0x36,0x4d,0x2e,0x6f,0x96,0xe8,0x1b,0x99,0xa0,0x2c,0xb7,0xc6, -0x18,0x9b,0xf0,0xeb,0xb3,0x3d,0x08,0x1f,0x07,0xaf,0xb9,0x16, -0xf9,0x13,0xb1,0xb9,0x34,0x15,0xd4,0x55,0x0f,0x2a,0xf9,0xce, -0xa5,0x47,0xf9,0x25,0x0c,0x5f,0x07,0x1f,0xe6,0x42,0xd4,0xcc, -0x1c,0x13,0x16,0xcc,0x05,0x2b,0x98,0xcd,0x5c,0x5b,0x28,0xc9, -0xed,0x4e,0x42,0xba,0x90,0xe1,0x05,0x72,0x91,0x27,0x96,0xdb, -0x0b,0x21,0x15,0x4e,0x5f,0x74,0xe9,0x6e,0xfd,0x05,0xda,0xec, -0xfe,0xb2,0x3d,0x49,0xa1,0x12,0x44,0x05,0xa3,0x07,0x92,0x44, -0x3f,0xff,0xf3,0xaf,0x19,0x3a,0xe0,0xdf,0x74,0x78,0xdc,0xfc, -0x42,0x62,0x46,0x4b,0xba,0xe7,0x37,0x8d,0xb4,0x92,0x23,0xc6, -0xef,0x04,0x56,0x25,0x67,0x3f,0x7e,0x2f,0x44,0xc2,0xeb,0xfb, -0xb9,0x87,0xf3,0xc4,0x18,0x4e,0xe4,0x0b,0x86,0xff,0x5c,0x2f, -0xfc,0x4d,0x90,0x88,0xa0,0x49,0x6e,0x82,0xb7,0xa9,0x60,0x91, -0xee,0x1c,0x38,0x1c,0x46,0xf3,0x23,0x1f,0x92,0xdf,0x89,0xd3, -0xf6,0xfd,0x93,0x66,0x4c,0x3a,0xb6,0x85,0xc3,0x19,0xbe,0x77, -0xa1,0xa7,0x09,0x9d,0x3c,0x8e,0x76,0x29,0xae,0xbe,0xa6,0x30, -0x8f,0x0b,0x17,0xf6,0x78,0xa2,0x78,0x56,0x9b,0x7b,0x4e,0x45, -0x81,0x59,0x8c,0x4a,0x6c,0xe6,0x52,0xa8,0x5c,0x56,0xe5,0x97, -0xb4,0xa3,0x91,0x2e,0x99,0x43,0x87,0xb8,0x6a,0xb6,0xa4,0xd0, -0xb4,0xb9,0xd3,0xc2,0x20,0x2d,0x2d,0xf1,0xb0,0x08,0x57,0xa0, -0x93,0xfa,0x66,0xb3,0x71,0xe8,0xae,0x9e,0x93,0xca,0x62,0x23, -0x3f,0x4c,0xd1,0x69,0xd3,0x63,0x74,0x2e,0xad,0x36,0x5d,0x22, -0x18,0x74,0x39,0x8c,0x42,0x8f,0x16,0x7d,0x25,0xd5,0x00,0xe7, -0xa9,0x9d,0x87,0x96,0xcb,0x0f,0x43,0x06,0x35,0x2b,0x2a,0x77, -0x5d,0x1f,0x9c,0x6f,0xec,0x25,0x8a,0x98,0x2e,0x4d,0xa2,0x36, -0x9a,0x75,0x69,0xd1,0xd4,0xa5,0x9e,0x9a,0xa2,0x8e,0x53,0x20, -0x9c,0xb6,0xaa,0x81,0x73,0xef,0xda,0x71,0x22,0xc8,0xbc,0x25, -0xf7,0xe3,0x23,0x49,0x93,0xf9,0xe0,0x11,0x99,0xb3,0x2a,0x0b, -0xca,0xc5,0xeb,0xd2,0x53,0x34,0x42,0x78,0x3b,0xa9,0xea,0x1f, -0x66,0xb7,0xf1,0xd2,0xa5,0x85,0xe1,0x07,0x82,0x1f,0xeb,0x77, -0x09,0x70,0x6d,0x7a,0xe8,0x31,0x93,0x7e,0xcd,0x55,0x72,0xdb, -0xc4,0x50,0x13,0x15,0x19,0x09,0x6e,0x02,0xb2,0x7a,0x43,0x17, -0xd6,0x44,0x05,0x05,0xfe,0x4c,0xf8,0x33,0x3c,0x65,0x50,0xe0, -0xb4,0xab,0x4b,0x5f,0xb5,0x78,0x83,0xd4,0x16,0x6c,0xa3,0x10, -0xbd,0x30,0x1d,0x78,0xe3,0x76,0x68,0xdc,0xb1,0x2e,0x39,0xd5, -0x4d,0xd6,0x62,0x1d,0x64,0xe5,0x76,0xf8,0x13,0xc7,0x5a,0x4e, -0x32,0x81,0xfb,0xfe,0x94,0x4b,0x1f,0x8a,0x30,0xe4,0x0e,0x60, -0x67,0x3e,0x3d,0xf1,0xed,0xb8,0x69,0x13,0x0e,0xca,0x7a,0xe3, -0x67,0xb8,0x2c,0xa1,0x32,0x54,0x76,0x0f,0x13,0x4e,0x3b,0x7b, -0x5b,0xa4,0xe2,0xb7,0xa5,0x87,0xac,0xdc,0xb2,0x5c,0xd9,0xa7, -0xec,0x6e,0x64,0x4c,0x1a,0xd6,0xf5,0x6e,0xf5,0x98,0x92,0x92, -0x57,0x96,0xeb,0x4c,0x6e,0xaf,0x9b,0xfe,0x5c,0x56,0x1f,0xb3, -0xd6,0xf4,0x7c,0xd6,0xbd,0x7f,0xfd,0xa8,0x5a,0x1b,0x9c,0x38, -0xb8,0xfb,0x9f,0x3e,0xdd,0x2f,0xef,0xef,0x07,0x56,0xa3,0x9b, -0x84,0x30,0x58,0x49,0xa1,0xb5,0x70,0x93,0xde,0xee,0x0a,0xce, -0x54,0x6b,0xaf,0x81,0x86,0x8d,0x39,0xf6,0xca,0x87,0x43,0xb5, -0x78,0x20,0x25,0xd9,0x91,0x54,0x38,0xb8,0x7c,0x04,0x10,0xa1, -0xf6,0xa7,0x50,0x6d,0x93,0x57,0x95,0x57,0x32,0x2a,0xb3,0x1a, -0xa9,0x94,0x25,0x9b,0xa4,0xd7,0x4e,0x07,0x02,0xc9,0xd5,0x17, -0x8f,0xd8,0x4d,0x6a,0x6c,0xee,0x28,0x35,0x15,0x4c,0x63,0x14, -0xce,0x63,0x64,0x41,0x91,0x1a,0xa1,0x48,0x8d,0xa6,0x10,0xd1, -0x69,0x9d,0x3b,0xd8,0x35,0xde,0x5f,0xcb,0x64,0xf4,0xb1,0x6c, -0x8e,0xa1,0xce,0xf4,0xda,0xc1,0x5b,0xaa,0x2f,0xfc,0xc1,0x1c, -0xae,0x91,0x95,0x8f,0x67,0x90,0x75,0x37,0x6c,0x75,0xce,0x14, -0x8d,0x89,0xa3,0xd5,0x72,0x2b,0x69,0x85,0xab,0x0f,0x7f,0x30, -0xd2,0x55,0x4e,0xea,0xa7,0x05,0x99,0x87,0x37,0x46,0x12,0x50, -0x34,0x7c,0x4e,0x8e,0xf8,0xa0,0x77,0x40,0x3a,0x68,0x81,0x64, -0x78,0x1b,0x0b,0xab,0x33,0xc7,0x21,0xba,0xed,0x5d,0x8c,0x8c, -0x05,0x3a,0xdf,0x94,0xa2,0x96,0x25,0x72,0xa1,0x9a,0xb9,0xdb, -0xed,0xea,0x69,0xb0,0xd9,0x37,0x39,0x82,0x56,0x73,0xd8,0xc1, -0xb2,0x31,0x34,0x6c,0xbc,0xb5,0x6c,0x84,0xc1,0x49,0x26,0xc2, -0xfc,0x11,0x16,0x8e,0xf0,0x4d,0xcf,0xfa,0xc8,0x2f,0x10,0x3f, -0x56,0x7d,0xf3,0x54,0xa7,0x2a,0xd4,0xd7,0x15,0x5a,0x19,0x5c, -0x23,0x22,0xee,0xb9,0xd6,0xfd,0x7a,0x30,0x69,0x57,0x4f,0x5e, -0x38,0x7e,0x05,0xfa,0x3d,0x7e,0xfb,0x12,0x3d,0x60,0x87,0xb1, -0x4a,0x85,0xf5,0xaa,0xb1,0xd6,0x5f,0xb7,0xfd,0xbf,0xb4,0xb5, -0x77,0xdd,0xec,0x08,0x06,0x21,0xdc,0x8e,0xa9,0xfe,0xde,0x4a, -0xfc,0xed,0x19,0xe4,0x46,0xc9,0xb3,0x0e,0x50,0xc9,0xdd,0x84, -0xe3,0x05,0x1b,0x21,0x3b,0xa4,0xa6,0x44,0xe1,0xad,0xf2,0x1c, -0x6a,0xbe,0x0e,0xa6,0xcd,0x29,0x86,0x5c,0x53,0xfa,0x8b,0xc9, -0xdd,0xc4,0xfb,0xf3,0x4d,0xa4,0x72,0xe2,0xc9,0xec,0xc0,0xfd, -0xdd,0x82,0xe5,0xb9,0x72,0xae,0x52,0xcd,0x6a,0x8f,0x26,0xd8, -0xea,0xc1,0xdf,0xf9,0x84,0xe9,0xa2,0x49,0x42,0x0c,0x23,0x9b, -0xfa,0x34,0x39,0xcb,0x24,0xc9,0x77,0x40,0xbe,0x28,0xd0,0xfa, -0xf9,0xd5,0xdf,0xc8,0x0e,0x2b,0xee,0xe9,0xfd,0x42,0x73,0x7a, -0x35,0xbf,0xa3,0x27,0x46,0x30,0x9b,0x67,0xf0,0x4e,0x5e,0x2b, -0x0f,0x75,0xda,0xe6,0xcd,0x33,0xa8,0xe0,0xcb,0xbb,0xe7,0xfe, -0xf1,0xf4,0xee,0x0a,0xf5,0xbd,0xf3,0x16,0x38,0xa9,0xa0,0x06, -0x17,0x1c,0xc0,0xa5,0x45,0x6d,0x2c,0xb0,0x23,0x44,0x34,0x60, -0x8f,0xaf,0x91,0x4e,0x76,0x8c,0xb3,0x82,0xeb,0xdb,0x90,0xa6, -0x1d,0x31,0x50,0xee,0xdf,0xf8,0x0f,0x11,0xd0,0xc9,0xa6,0xa8, -0xad,0x47,0x93,0x85,0x61,0xf5,0xbb,0x1d,0x82,0xb0,0x26,0x5c, -0x7f,0xd8,0xb6,0x0d,0xf2,0xcb,0x1f,0x33,0x0c,0xd4,0xdf,0xbb, -0xce,0xd0,0x93,0xaf,0xac,0x03,0x26,0x5b,0x3c,0xcd,0xaa,0x11, -0x3b,0x4a,0x3a,0xdf,0x42,0x72,0xa9,0xb9,0x52,0x8c,0x8c,0x48, -0x5f,0x22,0xfb,0x2d,0x1a,0xbb,0x70,0xb0,0xb1,0x0b,0xff,0x77, -0x8d,0xdd,0x38,0x62,0xd6,0xc2,0x11,0xbc,0x13,0x28,0x3a,0x95, -0xa4,0xa9,0x1b,0x6f,0xcc,0x26,0xda,0xa7,0xbe,0xd1,0x5d,0xc1, -0xff,0x1f,0x8e,0x83,0x2e,0xd5,0x43,0xf6,0x77,0x04,0x48,0x9b, -0xc1,0x60,0xe0,0x0d,0x19,0x6c,0x7d,0xc6,0xdf,0x5c,0x50,0xae, -0x9b,0xc2,0x7b,0x6e,0xf5,0x58,0x6f,0x7d,0xbb,0x9c,0xcb,0x6a, -0xc8,0x38,0xd7,0x99,0xa6,0x5d,0x5f,0x8a,0x53,0xb7,0x37,0xf4, -0xb0,0x97,0xd1,0x06,0x87,0x80,0x1e,0x69,0x90,0x80,0xb9,0x95, -0x2a,0x39,0xe2,0xa5,0x77,0x0c,0xcd,0xa2,0xea,0xb7,0x9d,0x82, -0xf8,0x16,0x01,0xf1,0xfa,0xde,0x83,0x32,0x07,0x2f,0x96,0x5f, -0xd3,0xb5,0x1d,0xdb,0x62,0x63,0x7f,0xa1,0xeb,0x96,0x63,0x5d, -0xa7,0x50,0xe6,0x03,0xc5,0xcb,0xdc,0x3f,0x71,0xd0,0xac,0x30, -0x6c,0x1a,0xe7,0x45,0xd8,0xe9,0xe0,0x57,0x43,0xda,0xba,0x76, -0x51,0xdd,0x6f,0xa7,0xf3,0x1d,0xbf,0xdc,0x68,0x7f,0x97,0xcb, -0xe5,0x6b,0x90,0xaa,0xe7,0x03,0xb0,0xd5,0x63,0x75,0xf9,0xcf, -0x6e,0x3a,0x8d,0x7f,0x43,0xdd,0xea,0x8a,0xaa,0xbb,0x64,0x93, -0x8a,0x8f,0x54,0xca,0x7c,0x36,0x7e,0xcf,0xe4,0xde,0x79,0xa3, -0x6e,0x37,0x9b,0x1b,0x59,0xb5,0xbf,0x7c,0x62,0xe6,0x77,0x48, -0x5f,0x83,0x5f,0x2b,0x4d,0xcd,0xef,0x33,0x52,0xc4,0x71,0x1e, -0x3f,0x9f,0x4b,0x6d,0xa6,0x89,0xea,0x6e,0xf6,0x77,0xd9,0xc4, -0x75,0xcd,0xd4,0x6e,0xfa,0x9f,0x13,0xc4,0x7c,0xd5,0xd8,0xd4, -0xc6,0x4b,0x60,0xdc,0xd5,0x59,0xb1,0x55,0x87,0x7c,0xf3,0x1b, -0xa8,0x0b,0xf6,0x83,0xa4,0xed,0xba,0x74,0x8e,0x57,0x41,0x10, -0x86,0x97,0xc8,0xa3,0x50,0x27,0x0f,0xae,0xdf,0xa0,0xdc,0xdf, -0xdf,0x05,0xc1,0x40,0x0f,0xff,0xba,0xc8,0x03,0xc3,0x38,0x77, -0x48,0x0c,0x82,0x9f,0x7f,0x0a,0x57,0x93,0x24,0xd6,0x9e,0xe8, -0x3f,0xa0,0xf4,0x84,0x9d,0x09,0x6b,0x62,0x0b,0xd2,0xf3,0x59, -0xc3,0x31,0xce,0x30,0xc7,0x7e,0x9a,0x51,0x49,0xe1,0x95,0xd9, -0xea,0x51,0x86,0x96,0x5d,0x6d,0x8d,0xf5,0xef,0xa2,0x6b,0xe9, -0xa1,0x61,0x54,0x51,0x13,0xcb,0xf7,0x8c,0xc3,0x99,0x6a,0xa8, -0x8a,0x08,0x94,0x43,0xef,0x0f,0x95,0xe3,0xe2,0x78,0xc2,0x46, -0x6d,0x8b,0xf8,0x2e,0x5d,0x6f,0xe0,0xf6,0xe0,0xfd,0xf9,0x9e, -0x05,0xc4,0x19,0xef,0x53,0xf4,0xc0,0xe1,0x7d,0x13,0x1b,0xa8, -0x81,0xfc,0xbb,0xd6,0xe9,0x77,0x14,0xd9,0xb0,0x32,0xc6,0x04, -0x92,0xd2,0x25,0xd7,0x36,0x5d,0xff,0x6f,0x00,0x00,0x00,0xff, -0xff,0xb7,0x8e,0x2c,0x40,0x78,0x2e,0x00,0x00, - })) + var empty [0]byte + sx := (*reflect.StringHeader)(unsafe.Pointer(&_styles_main_css)) + b := empty[:] + bx := (*reflect.SliceHeader)(unsafe.Pointer(&b)) + bx.Data = sx.Data + bx.Len = len(_styles_main_css) + bx.Cap = bx.Len + + gz, err := gzip.NewReader(bytes.NewBuffer(b)) if err != nil { panic("Decompression failed: " + err.Error()) } - var b bytes.Buffer - io.Copy(&b, gz) + var buf bytes.Buffer + io.Copy(&buf, gz) gz.Close() - return b.Bytes() + return buf.Bytes() } + func init() { go_bindata["/styles/main.css"] = styles_main_css } diff --git a/mod/dashboard/resources/views-browser.html.go b/mod/dashboard/resources/views-browser.html.go index a246ebf1a6f..e8f7c2c405a 100644 --- a/mod/dashboard/resources/views-browser.html.go +++ b/mod/dashboard/resources/views-browser.html.go @@ -4,166 +4,36 @@ import ( "bytes" "compress/gzip" "io" + "reflect" + "unsafe" ) +var _views_browser_html = "\x1f\x8b\x08\x00\x00\x09\x6e\x88\x00\xff\xc4\x58\x7b\x6f\xdb\x46\x12\xff\x3b\xf7\x29\x36\x3c\x20\xb6\x01\x92\xe2\x6b\x49\xca\x91\x14\x5c\x7c\x01\x7c\x38\xa7\x29\xda\xc0\x40\x51\x04\xc1\x8a\x5c\x8b\xac\x29\x92\xe0\xae\x24\xab\xae\xbf\x7b\x67\xb8\x94\x4c\x52\x0f\xdb\x6d\xda\xda\x90\xb8\x8f\xd9\x99\xdf\xcc\xce\xee\xfc\xc4\x51\x9c\x2e\x49\x94\x31\x21\xc6\x1a\x97\x51\x6c\x44\x45\x2e\x59\x9a\xf3\x8a\xd4\xdd\x69\x55\xac\x04\x74\xee\xef\xa3\x22\x5b\xcc\x73\xf1\xf0\x00\xed\xb2\xe2\xcb\x94\xaf\xea\xb6\x60\x4b\xfe\xf0\xa0\x4d\xfe\x45\xe0\x6f\xf4\xda\x30\x54\xa3\xaf\xb7\x2c\xca\x62\xb9\xd1\xda\x74\x0c\x5e\x55\x45\xd5\x2c\x3d\xb6\xca\xc8\x0b\x19\x25\xda\x64\x34\x00\x81\x67\x88\xa3\x13\x3c\x97\x2d\xcd\xf8\xf7\x09\xa6\x56\x55\x2a\x39\x91\x49\x2a\xc8\x92\x65\x0b\xfe\xee\x51\xd9\xf3\x75\xdf\xa4\xd5\xbc\xa7\x7b\x34\x5d\x48\x59\xe4\x9d\x15\xcd\x50\xab\x6d\x88\x39\xcb\xb2\xce\x48\x59\xa5\x73\x56\xad\xc9\x26\xf8\x4a\xf7\x16\xea\x68\xa0\xe4\x26\xfb\x70\xb6\x9a\x86\x31\x79\x71\xdc\x49\x1a\x37\x42\xb8\x87\x7f\xf3\x66\x7c\x40\x6b\xe7\x4f\x78\xf5\xc7\xbc\x51\x39\xfb\x8f\xf8\x43\x5e\xea\x50\xc2\x59\x0c\xfe\x88\x22\x4b\xe3\x36\x56\xd6\xcd\x24\x16\xdd\x6a\x24\x9f\x19\x51\x96\x46\xb7\x63\x0d\xfb\xa7\x67\xcd\x48\x2d\x76\x7f\xc3\x32\xc1\xcf\x4f\x6a\xf1\x38\x15\x6c\x9a\xf1\xf8\xe4\xe1\x67\x9e\x63\xeb\x3d\xc8\x7f\xe9\x67\xac\x58\xce\x08\xf8\x23\xd2\x22\x1f\x6b\xb6\x69\x6b\xe4\x6e\x9e\xe5\xa0\x2b\x91\xb2\x3c\x1f\x0c\x56\xab\x95\xb9\x72\xcd\xa2\x9a\x0d\x1c\xcb\xb2\x06\x20\xdf\x88\x9c\xdf\x65\x69\x7e\xbb\x4f\xd0\x1e\x0e\x87\x83\x7a\x16\x44\xc7\x9a\x55\xde\x69\x64\xad\x9e\x1d\xe3\x70\x81\xc0\xad\xb2\xe4\xff\x11\x25\x8f\xe4\x0f\x4c\xa6\xc5\x58\xbb\xfb\x98\xe6\x3f\xc1\x47\x23\x78\xb9\xbc\x2f\x50\x01\xb1\x48\xe0\x9a\x2e\xf5\x89\x0f\x00\x95\x37\x75\x38\x66\x55\xb1\xc8\x61\xc7\x73\xbe\x22\x3d\x29\xc0\x78\x2e\x4a\x16\xf1\xb1\xb6\xb1\xd3\xf7\xbd\x64\x32\x21\xb0\xfa\x23\x35\x9d\x40\x77\x5d\xd3\x71\xfc\xcc\x71\x4c\xcf\x09\x75\xf5\x88\x6c\x93\xfa\x8e\xae\xbe\x3d\xd3\x1a\xd2\xa6\x4d\x4d\x9f\x06\xba\xa5\xe6\x8d\x96\x94\xa1\xa4\x2c\xa3\x96\xb8\xb2\x03\x33\x00\xd5\x9e\xe9\xd9\x6e\xe2\x85\x26\xb5\xbd\x0e\x86\xc8\x31\x1d\x6b\xa8\x5b\xba\x07\x4a\x82\xa1\x0d\x4f\x4f\xa8\xa6\xe1\xe1\xff\x25\x6a\xf0\x86\x99\x0d\xfa\x2c\x10\xa2\x26\x0d\x9d\xda\xac\x5b\x9b\xb5\x6b\xb3\xb4\x36\xeb\xe9\x96\x69\x81\x50\x6d\xfa\xc2\x75\x4c\x1a\xf8\x80\xda\x05\xb5\x2e\x48\x51\x70\x41\x77\x2d\x93\x82\xb3\x3d\x14\xa0\xca\x72\x5c\x40\xed\x98\x96\xe7\x83\x1a\x77\x08\xcd\x10\x04\x21\x25\xfc\xe1\x15\x06\xc8\xd1\x9d\x00\x4c\x05\x91\x61\x99\x01\xb5\x75\xfc\x86\x75\x76\xe0\x82\x50\xe0\x87\x4d\x1b\x97\x0d\x2f\x10\x4e\x88\x56\x3d\x1a\x02\x04\x30\x0d\x70\x3c\x80\xd3\x0a\xf5\xaf\xda\xa0\xb7\x21\x98\x5d\xed\x3b\x8e\x1d\x3c\x0b\x2c\xc6\x83\xf2\x54\x2a\x3f\x9e\x15\x90\xc7\xa3\xf2\xd7\xe6\xf6\xab\x57\xaf\x5e\x94\xd2\xb0\x41\xcf\x48\xe9\x47\xa9\xfd\x29\x0d\x56\x1f\x33\xd9\xc5\x58\xdb\x7a\xb8\xb4\x61\x47\xc3\x70\xe9\xb8\x4b\x1b\x36\xca\x0d\x22\x4b\x57\xa9\xd6\xa4\x19\xfe\xef\x4b\xbe\x6b\x0f\x52\xc0\x0f\xae\xed\xa1\x69\x5b\xc3\xeb\x30\xc2\x9c\x00\xa9\x76\x4e\xa2\xc9\x0b\xd7\x37\x5d\xbb\xce\x28\x65\x92\xd6\x1a\x36\xf6\xeb\xbd\xed\x20\xb3\x7d\x54\xa8\x8e\x82\x93\x28\x78\x89\xe3\x26\x0d\xbc\x3d\x50\xf6\x99\xbe\xa4\x16\x64\x19\xbd\x84\x54\x74\xec\xe0\x52\x29\x8d\x8c\x66\xb5\xe1\xe9\x8d\xac\xae\x40\xda\x8e\x32\xea\x98\x3e\xa0\xb2\x5d\x73\x03\x40\xef\xc0\xd9\xa2\x3d\x96\x80\xfd\x4b\xbb\x21\x45\x06\x7a\xd8\xbf\x59\xd2\xbc\x5c\x48\x22\xd7\x25\x6c\x95\xe4\x77\xb2\x4e\xc5\x79\x11\xf3\x4c\x2d\xfe\x1e\xd7\xe0\x18\x94\x10\x5e\x8d\x35\xb1\xce\xa3\xab\x22\xc2\x74\xc9\x31\x4f\x25\x9b\xa6\x79\xcc\x21\x51\xc2\x30\xd4\xc8\x60\x6f\xe1\xaf\xbb\xdf\x84\x71\x60\xe5\x6f\x9f\x16\xec\xff\x97\x49\x06\x50\x26\x3f\x42\xbb\xcb\x3f\x8e\x55\xb2\x69\x11\xaf\x8f\x55\xdb\x2c\x15\xfd\x9a\x39\x92\x98\xfb\x24\xe2\x59\x56\xc2\x31\x4d\xf3\x19\x1c\x0f\xad\xee\x63\xb2\x37\x7d\xb5\x3f\x12\x2b\x65\xdd\x54\x2b\xe3\x8e\xf2\x9c\xcd\x79\x53\x4b\xb5\xc9\x77\xd0\x19\x0d\xe4\x61\x69\x29\xb3\xad\xf0\xe7\xcf\x57\x47\x65\x59\x84\x3b\x23\xb6\xf2\x6f\xf2\xa9\x28\xb7\x2b\xa0\xb1\xc5\x35\x92\x18\x82\x96\xa2\x0a\xe3\x5a\xf1\x92\x33\x39\xd6\x6e\xf9\x9a\xa4\x39\xc1\x28\x90\xdf\x48\x51\x81\xb2\xf7\xeb\xf3\x13\x18\x3e\xd1\xb6\x6b\x1a\x00\x13\xb8\xf2\x5a\xa5\x5d\x56\x0b\xa8\xec\x71\x5a\xc1\xbd\x52\x54\x6b\xa8\xea\xb0\xca\x84\xfe\x17\x8d\x24\x15\xbf\x19\x6b\xff\x1e\x2c\x9d\x01\x0c\x8a\xfb\x7b\x9c\x82\x0f\x30\x72\x92\xa4\xb3\x24\x83\x8f\x9c\xb4\x86\x31\xb3\x3b\x0e\x6f\x9c\x06\x83\x62\x95\x02\x13\x22\x78\x89\xbe\x7e\x8d\x2b\xf8\x5d\x99\x56\x75\x6a\x6a\xfd\x00\x2a\xd0\x8d\xab\xb8\xd7\xdb\xf5\xc6\x2a\xe1\xa0\x01\x51\x03\xa9\x92\xe9\x9c\x93\x8a\x67\xa0\x65\xc9\x49\xcc\x24\xc7\x11\xf0\xea\xbe\x6b\xc0\x14\x8b\xa9\x90\x15\xec\xf9\xa9\xa5\x93\xde\x1c\x98\x96\xff\xc3\x53\xf1\xe9\xe6\xf4\xc4\x38\x39\x3b\xc3\x1f\x1c\xe0\x05\x68\xda\xd0\xb6\x43\x60\x62\x7e\xc3\x16\x99\xdc\xd9\xff\xbc\xc8\x01\xde\x9b\x79\xcc\x44\xf2\xb6\xad\x63\x5f\x70\x3a\xfd\xbd\xd9\xdd\xa4\x89\xb6\x23\xb9\x57\x1a\x6e\x03\x2e\xd5\xb9\xdb\x89\x39\xec\xeb\x7e\x2d\xdb\x4a\xd7\x8f\x74\x4d\xfd\xda\x87\x58\xa9\xff\x3f\x5f\xe3\x85\xd2\xad\x8b\x48\x92\xaf\xd8\x9a\x57\x5f\xbf\x35\xdf\x3b\x04\xb9\x57\x2a\x5f\x46\xfd\xe0\xaa\xf6\xdc\x27\xeb\xe4\xa3\xd4\xc1\x3a\xb9\x37\x98\xdb\x0a\xe5\x0d\xa1\x16\x60\x91\xa0\x16\xcd\x0c\xdf\xb4\x5c\xaa\xd7\xdf\x57\x50\x6d\x6c\xa8\x19\x2e\x35\x43\xcb\xed\x4c\x45\x1d\xee\xd7\xe6\x89\x96\xe2\x88\x4f\x31\xc8\x43\xb0\x32\x50\x4f\xd1\x12\xcd\xa0\x54\x39\x2e\xd4\xc5\xfa\xf1\x38\xfe\x14\xf7\xbc\xa0\xb5\x27\x3a\x94\xda\xa1\xe7\xea\x50\x38\x87\xc0\xe1\x9a\x5e\xdb\xd7\x4d\xf5\x3b\x1e\x1d\x07\x2a\x36\x50\x59\xe4\x05\xbe\x93\xb5\xa3\xe3\x01\xe9\x0e\xa8\xee\x01\xd9\xf3\xdb\x13\x7f\x96\x3c\x1f\x0c\x8d\xb1\x8d\xc1\x15\x54\xf6\x20\x74\x11\x15\xa5\xb4\x35\xd1\xec\x8b\xd1\x52\xdb\xb4\x1b\xe3\x17\xb8\xc4\x06\x44\x14\xe9\x8e\xbe\xe9\x05\xa6\xaf\x77\x3c\x3d\x1a\x1c\x45\x17\x0e\xcd\x1e\xb9\x0f\xb7\x37\xcd\xae\xca\x7d\x33\xbb\xa3\xdd\x6a\x35\x90\xd5\xa6\x10\x6d\xab\x0f\xb4\xf1\xac\x1c\x66\x0e\x3b\xbf\x6c\xd5\xeb\x9c\x9d\xea\x0c\xf4\x85\x55\x9c\x91\x32\x83\x13\x95\x14\x59\x8c\x84\xe5\x03\xf2\x16\xc2\xf0\x8a\x26\x58\x76\x09\x9b\xc2\xcf\x62\xc2\xf2\x98\x40\x2d\x54\x6f\x55\x48\xc2\x2b\xde\x66\x3e\x02\xee\xf5\x8c\x5f\xe3\xdc\x0e\xc5\xc1\x7b\x2b\x61\xf9\x0c\xce\xac\x48\x8a\x15\xb2\x0e\x64\x1f\xbd\xdf\x07\x1b\x30\xbd\xf1\xbe\x2b\x7c\x5e\xca\x75\x6f\xf1\x61\x41\x63\xce\x85\x60\x33\xd8\x97\xed\x3b\xad\xaf\xcd\x10\x56\xca\x4e\xd4\xf6\x04\x72\xaf\xe2\x38\x65\x59\x31\x7b\x0e\x04\x25\xf9\x88\x61\x67\x05\xfe\x61\x38\xea\xe0\x02\x91\x80\xdb\x9d\x6d\xde\x5e\xa9\x90\xbd\xdb\xb5\xb2\x0b\xf1\x98\x71\xc5\xee\xc4\x01\xe3\xcf\xe4\x98\x0d\xa3\x54\x8c\x91\x5c\xd4\xc8\xc4\xee\x9b\xab\x8e\x66\xd6\x50\x97\x5f\xd8\x92\x89\xa8\x4a\x4b\x79\xbe\x2c\xd2\xf8\xd4\x3a\x7b\xab\x4d\x2e\x58\x0e\x24\xb0\x43\xc2\x8f\x6d\x41\x2f\xbd\x77\xde\xbc\x34\x8f\xdf\x03\x00\x00\xff\xff\x5c\x14\xa6\x24\xec\x14\x00\x00" + // views_browser_html returns raw, uncompressed file data. func views_browser_html() []byte { - gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ -0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0xc4,0x58, -0x7b,0x6f,0xdb,0x46,0x12,0xff,0x3b,0xf7,0x29,0x36,0x3c,0x20, -0xb6,0x01,0x92,0xe2,0x6b,0x49,0xca,0x91,0x14,0x5c,0x7c,0x01, -0x7c,0x38,0xa7,0x29,0xda,0xc0,0x40,0x51,0x04,0xc1,0x8a,0x5c, -0x8b,0xac,0x29,0x92,0xe0,0xae,0x24,0xab,0xae,0xbf,0x7b,0x67, -0xb8,0x94,0x4c,0x52,0x0f,0xdb,0x6d,0xda,0xda,0x90,0xb8,0x8f, -0xd9,0x99,0xdf,0xcc,0xce,0xee,0xfc,0xc4,0x51,0x9c,0x2e,0x49, -0x94,0x31,0x21,0xc6,0x1a,0x97,0x51,0x6c,0x44,0x45,0x2e,0x59, -0x9a,0xf3,0x8a,0xd4,0xdd,0x69,0x55,0xac,0x04,0x74,0xee,0xef, -0xa3,0x22,0x5b,0xcc,0x73,0xf1,0xf0,0x00,0xed,0xb2,0xe2,0xcb, -0x94,0xaf,0xea,0xb6,0x60,0x4b,0xfe,0xf0,0xa0,0x4d,0xfe,0x45, -0xe0,0x6f,0xf4,0xda,0x30,0x54,0xa3,0xaf,0xb7,0x2c,0xca,0x62, -0xb9,0xd1,0xda,0x74,0x0c,0x5e,0x55,0x45,0xd5,0x2c,0x3d,0xb6, -0xca,0xc8,0x0b,0x19,0x25,0xda,0x64,0x34,0x00,0x81,0x67,0x88, -0xa3,0x13,0x3c,0x97,0x2d,0xcd,0xf8,0xf7,0x09,0xa6,0x56,0x55, -0x2a,0x39,0x91,0x49,0x2a,0xc8,0x92,0x65,0x0b,0xfe,0xee,0x51, -0xd9,0xf3,0x75,0xdf,0xa4,0xd5,0xbc,0xa7,0x7b,0x34,0x5d,0x48, -0x59,0xe4,0x9d,0x15,0xcd,0x50,0xab,0x6d,0x88,0x39,0xcb,0xb2, -0xce,0x48,0x59,0xa5,0x73,0x56,0xad,0xc9,0x26,0xf8,0x4a,0xf7, -0x16,0xea,0x68,0xa0,0xe4,0x26,0xfb,0x70,0xb6,0x9a,0x86,0x31, -0x79,0x71,0xdc,0x49,0x1a,0x37,0x42,0xb8,0x87,0x7f,0xf3,0x66, -0x7c,0x40,0x6b,0xe7,0x4f,0x78,0xf5,0xc7,0xbc,0x51,0x39,0xfb, -0x8f,0xf8,0x43,0x5e,0xea,0x50,0xc2,0x59,0x0c,0xfe,0x88,0x22, -0x4b,0xe3,0x36,0x56,0xd6,0xcd,0x24,0x16,0xdd,0x6a,0x24,0x9f, -0x19,0x51,0x96,0x46,0xb7,0x63,0x0d,0xfb,0xa7,0x67,0xcd,0x48, -0x2d,0x76,0x7f,0xc3,0x32,0xc1,0xcf,0x4f,0x6a,0xf1,0x38,0x15, -0x6c,0x9a,0xf1,0xf8,0xe4,0xe1,0x67,0x9e,0x63,0xeb,0x2b,0x2e, -0xf8,0xd2,0x4f,0x59,0xb1,0x9c,0x11,0x70,0x48,0xa4,0x45,0x3e, -0xd6,0x6c,0xd3,0xd6,0xc8,0xdd,0x3c,0xcb,0x41,0x59,0x22,0x65, -0x79,0x3e,0x18,0xac,0x56,0x2b,0x73,0xe5,0x9a,0x45,0x35,0x1b, -0x38,0x96,0x65,0x0d,0x40,0xbe,0x11,0x39,0xbf,0xcb,0xd2,0xfc, -0x76,0x9f,0xa0,0x3d,0x1c,0x0e,0x07,0xf5,0x2c,0x88,0x8e,0x35, -0xab,0xbc,0xd3,0xc8,0x5a,0x3d,0x3b,0xc6,0xe1,0x06,0x81,0x6b, -0x65,0xc9,0xff,0x23,0x4a,0x1e,0xc9,0x1f,0x98,0x4c,0x8b,0xb1, -0x76,0xf7,0x31,0xcd,0x7f,0x82,0x8f,0x46,0xf0,0x76,0x79,0x5f, -0xa0,0x02,0x62,0x91,0xc0,0x35,0x5d,0xea,0x13,0x1f,0x00,0x2a, -0x77,0xea,0x78,0xcc,0xaa,0x62,0x91,0xc3,0x96,0xe7,0x7c,0x45, -0x7a,0x52,0x80,0xf1,0x5c,0x94,0x2c,0xe2,0x63,0x6d,0x63,0xa7, -0xef,0x7b,0xc9,0x64,0x42,0x60,0xf5,0x47,0x6a,0x3a,0x81,0xee, -0xba,0xa6,0xe3,0xf8,0x99,0xe3,0x98,0x9e,0x13,0xea,0xea,0x11, -0xd9,0x26,0xf5,0x1d,0x5d,0x7d,0x7b,0xa6,0x35,0xa4,0x4d,0x9b, -0x9a,0x3e,0x0d,0x74,0x4b,0xcd,0x1b,0x2d,0x29,0x43,0x49,0x59, -0x46,0x2d,0x71,0x65,0x07,0x66,0x00,0xaa,0x3d,0xd3,0xb3,0xdd, -0xc4,0x0b,0x4d,0x6a,0x7b,0x1d,0x0c,0x91,0x63,0x3a,0xd6,0x50, -0xb7,0x74,0x0f,0x94,0x04,0x43,0x1b,0x9e,0x9e,0x50,0x4d,0xc3, -0xc3,0xff,0x4b,0xd4,0xe0,0x0d,0x33,0x1b,0xf4,0x59,0x20,0x44, -0x4d,0x1a,0x3a,0xb5,0x59,0xb7,0x36,0x6b,0xd7,0x66,0x69,0x6d, -0xd6,0xd3,0x2d,0xd3,0x02,0xa1,0xda,0xf4,0x85,0xeb,0x98,0x34, -0xf0,0x01,0xb5,0x0b,0x6a,0x5d,0x90,0xa2,0xe0,0x82,0xee,0x5a, -0x26,0x05,0x67,0x7b,0x28,0x40,0x95,0xe5,0xb8,0x80,0xda,0x31, -0x2d,0xcf,0x07,0x35,0xee,0x10,0x9a,0x21,0x08,0x42,0x4a,0xf8, -0xc3,0x2b,0x0c,0x90,0xa3,0x3b,0x01,0x98,0x0a,0x22,0xc3,0x32, -0x03,0x6a,0xeb,0xf8,0x0d,0xeb,0xec,0xc0,0x05,0xa1,0xc0,0x0f, -0x9b,0x36,0x2e,0x1b,0x5e,0x20,0x9c,0x10,0xad,0x7a,0x34,0x04, -0x08,0x60,0x1a,0xe0,0x78,0x00,0xa7,0x15,0xea,0x5f,0xb5,0x41, -0x6f,0x43,0x30,0xbb,0xda,0x97,0x1c,0x3b,0x78,0x18,0x58,0x8c, -0x27,0xe5,0xa9,0x54,0x7e,0x3c,0x2c,0x20,0x8f,0x67,0xe5,0xaf, -0xcd,0xed,0x57,0xaf,0x5e,0xbd,0x28,0xa5,0x61,0x83,0x9e,0x91, -0xd2,0x8f,0x52,0xfb,0x53,0x1a,0xac,0x3e,0x66,0xb2,0x8b,0xb1, -0xb6,0xf5,0x70,0x69,0xc3,0x8e,0x86,0xe1,0xd2,0x71,0x97,0x36, -0x6c,0x94,0x1b,0x44,0x96,0xae,0x52,0xad,0x49,0x33,0xfc,0xdf, -0x97,0x7c,0xd7,0x1e,0xa4,0x80,0x1f,0x5c,0xdb,0x43,0xd3,0xb6, -0x86,0xd7,0x61,0x84,0x39,0x01,0x52,0xed,0x9c,0x44,0x93,0x17, -0xae,0x6f,0xba,0x76,0x9d,0x51,0xca,0x24,0xad,0x35,0x6c,0xec, -0xd7,0x7b,0xdb,0x41,0x66,0xfb,0xa8,0x50,0x1d,0x05,0x27,0x51, -0xf0,0x12,0xc7,0x4d,0x1a,0x78,0x7b,0xa0,0xec,0x33,0x7d,0x49, -0x2d,0xc8,0x32,0x7a,0x09,0xa9,0xe8,0xd8,0xc1,0xa5,0x52,0x1a, -0x19,0xcd,0x6a,0xc3,0xd3,0x1b,0x59,0x5d,0x81,0xb4,0x1d,0x65, -0xd4,0x31,0x7d,0x40,0x65,0xbb,0xe6,0x06,0x80,0xde,0x81,0xb3, -0x45,0x7b,0x2c,0x01,0xfb,0xb7,0x76,0xc3,0x8a,0x0c,0xf4,0xb0, -0x7f,0xb3,0xa4,0x79,0xb9,0x90,0x44,0xae,0x4b,0xd8,0x2a,0xc9, -0xef,0x64,0x9d,0x8a,0xf3,0x22,0xe6,0x99,0x5a,0xfc,0x3d,0xae, -0xc1,0x31,0xa8,0x21,0xbc,0x1a,0x6b,0x62,0x9d,0x47,0x57,0x45, -0x84,0xe9,0x92,0x63,0x9e,0x4a,0x36,0x4d,0xf3,0x98,0x43,0xa2, -0x84,0x61,0xa8,0x91,0xc1,0xde,0xca,0x5f,0x77,0xbf,0x09,0xe5, -0xc0,0xd2,0xdf,0x3e,0x2d,0xd8,0xff,0x2f,0x93,0x0c,0xa0,0x4c, -0x7e,0x84,0x76,0x97,0x80,0x1c,0x2b,0x65,0xd3,0x22,0x5e,0x1f, -0x2b,0xb7,0x59,0x2a,0xfa,0x45,0x73,0x24,0x31,0xf7,0x49,0xc4, -0xb3,0xac,0x84,0x63,0x9a,0xe6,0x33,0x38,0x1e,0x5a,0xdd,0xc7, -0x64,0x6f,0xfa,0x6a,0x7f,0x24,0x96,0xca,0xba,0xa9,0x56,0xc6, -0x1d,0xe5,0x39,0x9b,0xf3,0xa6,0x98,0x6a,0x93,0xef,0xa0,0x33, -0x1a,0xc8,0xc3,0xd2,0x52,0x66,0x5b,0xe1,0xcf,0x9f,0xaf,0x8e, -0xca,0xb2,0x08,0x77,0x46,0x6c,0xe5,0xdf,0xe4,0x53,0x51,0x6e, -0x57,0x40,0x63,0x8b,0x6b,0x24,0x31,0x04,0x2d,0x45,0x15,0xc6, -0xb5,0xe2,0x25,0x67,0x72,0xac,0xdd,0xf2,0x35,0x49,0x73,0x82, -0x51,0x20,0xbf,0x91,0xa2,0x02,0x65,0xef,0xd7,0xe7,0x27,0x30, -0x7c,0xa2,0x6d,0xd7,0x34,0x00,0x26,0x70,0xe5,0xb5,0x6a,0xbb, -0xac,0x16,0x50,0xda,0xe3,0xb4,0x82,0x7b,0xa5,0xa8,0xd6,0x50, -0xd6,0x61,0x95,0x09,0xfd,0x2f,0x1a,0x49,0x2a,0x7e,0x33,0xd6, -0xfe,0x3d,0x58,0xda,0x03,0x18,0x14,0xf7,0xf7,0x38,0x05,0x1f, -0xa0,0xe4,0x24,0x49,0x67,0x49,0x06,0x1f,0x39,0x69,0x0d,0x63, -0x66,0x77,0x1c,0xde,0x38,0x0d,0x06,0xc5,0x2a,0x05,0x2a,0x44, -0xf0,0x12,0x7d,0xfd,0x1a,0x57,0xf0,0xbb,0x32,0xad,0xea,0xd4, -0xd4,0xfa,0x01,0x54,0xa0,0x1b,0x57,0x71,0xaf,0xb7,0xeb,0x8d, -0x55,0xc2,0x41,0x03,0xa2,0x06,0x56,0x25,0xd3,0x39,0x27,0x15, -0xcf,0x40,0xcb,0x92,0x93,0x98,0x49,0x8e,0x23,0xe0,0xd5,0x7d, -0xd7,0x80,0x29,0x16,0x53,0x21,0x2b,0xd8,0xf3,0x53,0x4b,0x27, -0xbd,0x39,0x30,0x2d,0xff,0x87,0xa7,0xe2,0xd3,0xcd,0xe9,0x89, -0x71,0x72,0x76,0x86,0xbf,0x38,0xc0,0x0b,0xd0,0xb4,0xe1,0x6d, -0x87,0xc0,0xc4,0xfc,0x86,0x2d,0x32,0xb9,0xb3,0xff,0x79,0x91, -0x03,0xbc,0x37,0xf3,0x98,0x89,0xe4,0x6d,0x5b,0xc7,0xbe,0xe0, -0x74,0xfa,0x7b,0xb3,0xbb,0x49,0x13,0x6d,0x47,0x72,0xaf,0x34, -0xdc,0x06,0x5c,0xaa,0x73,0xb7,0x13,0x73,0xd8,0xd7,0xfd,0x5a, -0xb6,0x95,0xae,0x1f,0xe9,0x9a,0xfb,0xb5,0x0f,0xb1,0x52,0xff, -0x7f,0xbe,0xc6,0x0b,0xa5,0x5b,0x17,0x91,0x25,0x5f,0xb1,0x35, -0xaf,0xbe,0x7e,0x6b,0xbe,0x77,0x08,0x72,0xaf,0x54,0xbe,0x8c, -0xfa,0xc1,0x55,0xed,0xb9,0x4f,0xd6,0xc9,0x47,0xa9,0x83,0x75, -0x72,0x6f,0x30,0xb7,0x15,0xca,0x1b,0x42,0x2d,0xc0,0x22,0x41, -0x2d,0x9a,0x19,0xbe,0x69,0xb9,0x54,0xaf,0xbf,0xaf,0xa0,0xda, -0xd8,0x50,0x33,0x5c,0x6a,0x86,0x96,0xdb,0x99,0x8a,0x3a,0xdc, -0xaf,0xcd,0x13,0x2d,0xc5,0x11,0x9f,0x62,0x90,0x87,0x60,0x65, -0xa0,0x9e,0xa2,0x25,0x9a,0x41,0xa9,0x72,0x5c,0xa8,0x8b,0xf5, -0xe3,0x71,0xfc,0x29,0xee,0x79,0x41,0x6b,0x4f,0x74,0x28,0xb5, -0x43,0xcf,0xd5,0xa1,0x70,0x0e,0x81,0xc3,0x35,0xbd,0xb6,0xaf, -0x9b,0xea,0x77,0x3c,0x3a,0x0e,0x54,0x6c,0xa0,0xb2,0xc8,0x0b, -0x7c,0x27,0x6b,0x47,0xc7,0x03,0xd2,0x1d,0x50,0xdd,0x03,0xb2, -0xe7,0xb7,0x27,0xfe,0x2c,0x79,0x3e,0x18,0x1a,0x63,0x1b,0x83, -0x2b,0xa8,0xec,0x41,0xe8,0x22,0x2a,0x4a,0x69,0x6b,0xa2,0xd9, -0x17,0xa3,0xa5,0xb6,0x69,0x37,0xc6,0x2f,0x70,0x89,0x0d,0x88, -0x28,0xd2,0x1d,0x7d,0xd3,0x0b,0x4c,0x5f,0xef,0x78,0x7a,0x34, -0x38,0x8a,0x2e,0x1c,0x9a,0x3d,0x72,0x1f,0x6e,0x6f,0x9a,0x5d, -0x95,0xfb,0x66,0x76,0x47,0xbb,0xd5,0x6a,0x20,0xab,0x4d,0x21, -0xda,0x56,0x1f,0x68,0xe3,0x59,0x39,0xcc,0x1c,0x76,0x7e,0xda, -0xaa,0xf7,0x39,0x3b,0xd5,0x19,0xe8,0x0b,0xab,0x38,0x23,0x65, -0x06,0x27,0x2a,0x29,0xb2,0x18,0x09,0xcb,0x07,0xe4,0x2d,0x84, -0xe1,0x15,0x4d,0xb0,0xec,0x12,0x36,0x85,0xdf,0xc5,0x84,0xe5, -0x31,0x81,0x5a,0xa8,0x5e,0xab,0x90,0x84,0x57,0xbc,0xcd,0x7c, -0x04,0xdc,0xeb,0x19,0xbf,0xc6,0xb9,0x1d,0x8a,0x83,0xf7,0x56, -0xc2,0xf2,0x19,0x9c,0x59,0x91,0x14,0x2b,0x64,0x1d,0xc8,0x3e, -0x7a,0xbf,0x0f,0x36,0x60,0x7a,0xe3,0x7d,0x57,0xf8,0xbc,0x94, -0xeb,0xde,0xe2,0xc3,0x82,0xc6,0x9c,0x0b,0xc1,0x66,0xb0,0x2f, -0xdb,0x97,0x5a,0x5f,0x9b,0x21,0xac,0x94,0x9d,0xa8,0xed,0x09, -0xe4,0x5e,0xc5,0x71,0xca,0xb2,0x62,0xf6,0x1c,0x08,0x4a,0xf2, -0x11,0xc3,0xce,0x0a,0xfc,0xc3,0x70,0xd4,0xc1,0x05,0x22,0x01, -0xb7,0x3b,0xdb,0xbc,0xbe,0x52,0x21,0x7b,0xb7,0x6b,0x65,0x17, -0xe2,0x31,0xe3,0x8a,0xdd,0x89,0x03,0xc6,0x9f,0xc9,0x31,0x1b, -0x46,0xa9,0x18,0x23,0xb9,0xa8,0x91,0x89,0xdd,0x57,0x57,0x1d, -0xcd,0xac,0xa1,0x2e,0xbf,0xb0,0x25,0x13,0x51,0x95,0x96,0xf2, -0x7c,0x59,0xa4,0xf1,0xa9,0x75,0xf6,0x56,0x9b,0x5c,0xb0,0x1c, -0x48,0x60,0x87,0x84,0x1f,0xdb,0x82,0x5e,0x7a,0xef,0xbc,0x7a, -0x69,0x1e,0xbf,0x07,0x00,0x00,0xff,0xff,0x7c,0x65,0x3f,0x3c, -0xed,0x14,0x00,0x00, - })) + var empty [0]byte + sx := (*reflect.StringHeader)(unsafe.Pointer(&_views_browser_html)) + b := empty[:] + bx := (*reflect.SliceHeader)(unsafe.Pointer(&b)) + bx.Data = sx.Data + bx.Len = len(_views_browser_html) + bx.Cap = bx.Len + + gz, err := gzip.NewReader(bytes.NewBuffer(b)) if err != nil { panic("Decompression failed: " + err.Error()) } - var b bytes.Buffer - io.Copy(&b, gz) + var buf bytes.Buffer + io.Copy(&buf, gz) gz.Close() - return b.Bytes() + return buf.Bytes() } + func init() { go_bindata["/views/browser.html"] = views_browser_html } diff --git a/mod/dashboard/resources/views-stats.html.go b/mod/dashboard/resources/views-stats.html.go index f62994bf692..00c65e039ea 100644 --- a/mod/dashboard/resources/views-stats.html.go +++ b/mod/dashboard/resources/views-stats.html.go @@ -4,92 +4,36 @@ import ( "bytes" "compress/gzip" "io" + "reflect" + "unsafe" ) +var _views_stats_html = "\x1f\x8b\x08\x00\x00\x09\x6e\x88\x00\xff\xd4\x56\x4b\x8f\xdb\x36\x10\xbe\xe7\x57\x10\xec\xc1\x09\x20\xc9\x7a\xf8\xb1\x32\x2c\x03\x4d\x0b\xf4\xd2\xed\xa1\x29\x0a\x14\x45\x0f\x34\x45\x5b\xc4\xd2\x94\x4a\xd2\xd6\x3a\xae\xff\x7b\x87\xa2\xec\xae\x64\xc5\xdd\xa0\x7b\xe9\x3e\x40\x71\x38\xf3\xcd\xa7\xe1\xc7\xa1\x96\x39\x3f\x20\x2a\x88\xd6\x19\x66\x86\xe6\x3e\x2d\xa5\x21\x5c\x32\x85\x9a\xa9\x36\xc4\x68\x74\x3a\xd1\x52\xec\x77\x52\x9f\xcf\xf0\x6c\xc8\x5a\xb0\x5f\xb9\xe6\x6b\x2e\xb8\x39\x9e\xcf\x78\xf5\x0e\xc1\xcf\x0d\xd6\xba\xcc\x8f\xed\xda\xe0\xfa\xa6\x54\x3b\x62\x7c\xcd\x04\xa3\xa6\x54\x2f\x5c\x07\xdd\x2f\x7e\x3e\x37\x6c\x87\xba\xa6\xad\x22\x55\x81\x91\xdc\xfa\x54\x70\xfa\x94\x61\x5d\x94\xf5\x0f\xd6\xf8\xfe\x43\x0f\xb7\xc1\xd6\x87\x2d\x3a\x30\xa5\x79\x29\x33\x1c\x05\x11\x46\xcf\x3b\x21\x21\x53\x61\x4c\xb5\x18\x8f\xeb\xba\x0e\xea\x24\x28\xd5\x76\x1c\x87\x61\x38\x06\xff\xd6\x65\xf1\x2c\xb8\x7c\x1a\x72\x8c\xd2\x34\x1d\x37\xab\xe0\x9a\xe1\xb0\x7a\xc6\xe8\xe8\xc6\x1b\x02\x95\x62\x9a\xa9\x03\xfb\x56\x57\xf0\x02\x3f\x13\xc3\xcb\x0c\x3f\x3f\x72\xf9\x1b\xfc\x63\x74\xe0\xac\xfe\x58\x5a\x10\x14\xa2\x99\xfd\x0b\xc2\x70\x8e\x11\x93\xb6\xf6\xfe\x9a\xd0\xa7\xad\x2a\xf7\x32\xcf\xb0\x64\x35\xea\x79\x01\xcf\x85\xae\x08\x65\x19\xbe\xe4\x19\xaa\x41\x45\x4c\x81\x36\x5c\x88\x0c\x7f\x13\x7d\x0f\xbf\x1f\x31\x02\xc0\xc7\x24\xf4\x01\x28\x8d\xa8\x1f\xcd\x82\x59\x1c\x79\xa1\x9f\x58\xc3\xc4\x8b\x92\x60\x32\x9f\x5c\x66\x6e\xa0\xa1\xd7\xba\xb9\x55\xaf\xb3\xda\x0e\x37\xc9\xb5\xb3\xfb\x9d\x98\x16\xf8\xbb\xd9\x35\x5b\xf2\x90\x78\x93\x06\xdd\x51\xf2\x2e\xdc\x3e\x23\xa0\xe9\x4d\xe6\x41\x9c\xa4\xd4\x4f\x83\x69\x94\x02\xcd\xc8\xce\xa7\xfe\x3c\x98\x47\xb3\xcb\xc4\x0d\x37\x04\x3e\xc5\x61\x30\x79\x00\xd2\x71\x30\x9f\x3d\x00\x6e\xfb\x44\x5b\x2c\xcf\xc5\x79\x0d\xd6\x65\xe2\x86\x4f\x89\xf3\x71\xd9\xbd\x2b\x8f\xcf\x78\x3c\x50\x65\x2b\x9d\xd5\xbb\xae\xb2\xc7\x20\xed\xff\x20\xf6\xe6\x00\xf6\xc5\xfe\x8b\x35\xfe\xff\xc5\x3e\x4d\x83\x49\x3a\xb3\xc3\x34\x4c\xee\x0a\xbe\xe7\xf9\x4a\xd1\x2b\x60\xd0\x10\x76\x74\x71\xff\x00\xd4\x3c\x37\x45\x86\x1d\x38\x46\x05\xe3\xdb\xc2\x40\xd5\x92\x20\x8a\xa2\xc1\x0d\xee\x22\xc6\xe0\x68\x03\xdf\x18\x16\x0e\x41\x92\xc6\x6f\x00\xeb\xe4\x78\x47\x8d\xfd\x69\x5f\x98\xae\xd3\xf6\x20\x8a\x78\xf5\x48\x68\x01\xf7\x06\xfa\x91\x18\x26\xe9\x71\x39\x06\xdb\x7d\x89\x37\x48\xff\x5c\x38\x18\x71\xd8\x5d\xe1\xc2\xfb\x09\xbe\x8e\xa3\xe0\xda\xdc\xa3\x08\xcb\x03\xfc\x9a\x63\x85\x28\x13\xa2\x22\x79\xce\xe5\xb6\x29\xbe\x9d\x5b\x5d\xb5\xf3\x7e\x4c\xc1\x48\x3e\x50\x66\x93\x77\x08\x49\xb2\x63\xbe\x75\x85\xd7\xbc\xd2\xf8\x09\x8c\xcb\xb1\x79\x45\xf8\xb5\x28\xd7\xe2\xf6\xa3\xc0\x72\xcb\x64\x69\xec\x0d\x3c\x04\xaf\x6c\xf3\x50\xac\x62\x04\xd4\xb2\x6b\xf9\x70\x89\xda\x47\x3d\x70\x72\x2e\xbc\x20\x50\xd7\xdc\xd0\x02\xd9\x6e\x72\x32\x6a\xcf\x16\x23\xd1\xbc\xda\xc8\x43\x1b\x22\x34\x5b\xa0\xd1\xa6\x14\xa2\xac\xc1\x74\xfe\xdd\xad\xd9\x97\x45\x59\x76\xc9\x10\xd8\x8a\xfc\xf1\x85\x34\x4d\x2a\xbb\xa7\xd7\x5c\x7e\x5d\x30\xc8\x26\xda\x0a\x9e\x4e\x2f\x61\xce\xe7\x25\x6c\x90\xec\x54\xac\x5d\xf7\xcd\xb1\x82\x36\xf0\xde\x05\x7e\x00\xf5\x83\xe3\x6a\xa0\xff\xde\x49\x9d\xb3\x0d\xd9\x0b\x73\x9b\xf4\xcb\x28\xc3\xbb\xda\x56\xf0\x5f\x12\x77\x2e\x81\x3f\xf7\x44\x31\xd7\xe8\xc1\xba\x40\xa7\xd1\x0b\x3b\x1c\x20\xc6\xe4\x68\x71\xad\x69\x2b\x93\x80\xee\x95\x62\xd2\xa0\x25\x8a\xa7\x1e\xea\x84\x94\x8a\xc8\x2d\xbb\x1b\x33\x0b\x7b\x31\x8a\xe5\x77\x02\x56\x19\x44\xc0\x27\xe0\x6b\x8a\x3a\xa0\x69\xff\x40\xc4\x9e\xbd\xdc\xd2\x7e\x82\xbf\x90\xdc\xef\xd6\x4c\x2d\x22\x04\x9f\x9f\x3b\xfd\xd5\x75\x07\xab\xba\x39\x2d\xb7\x27\x03\x8c\xb6\x01\x0c\x76\x99\xf6\xb1\x1d\xfe\x0e\x00\x00\xff\xff\xe8\x36\xb1\x2a\x35\x0b\x00\x00" + // views_stats_html returns raw, uncompressed file data. func views_stats_html() []byte { - gz, err := gzip.NewReader(bytes.NewBuffer([]byte{ -0x1f,0x8b,0x08,0x00,0x00,0x09,0x6e,0x88,0x00,0xff,0xd4,0x56, -0x5b,0x8f,0xe3,0x34,0x14,0x7e,0xdf,0x5f,0x61,0x99,0x87,0x05, -0x29,0x37,0x27,0xbd,0x4c,0xaa,0xa6,0x12,0x0b,0xe2,0x89,0xe1, -0x81,0x95,0x90,0x78,0x42,0xae,0xe3,0x69,0xac,0x71,0x9d,0x60, -0xbb,0xcd,0xcc,0x96,0xfe,0x77,0x4e,0xe2,0xa4,0x4c,0x93,0x10, -0x2d,0x62,0x5f,0x76,0x2e,0x72,0xce,0xed,0x3b,0x9f,0x4f,0x3f, -0x3b,0xdd,0xe6,0xe2,0x8c,0x98,0xa4,0xc6,0x64,0x98,0x5b,0x96, -0xfb,0xac,0x54,0x96,0x0a,0xc5,0x35,0x6a,0x4d,0x63,0xa9,0x35, -0xe8,0x72,0x61,0xa5,0x3c,0x1d,0x95,0xb9,0x5e,0xe1,0xd9,0xd2, -0xbd,0xe4,0xbf,0x09,0x23,0xf6,0x42,0x0a,0xfb,0x7a,0xbd,0xe2, -0xdd,0x3b,0x04,0x3f,0x23,0xac,0x7d,0x99,0xbf,0x76,0xb1,0xc9, -0xf8,0x53,0xa9,0x8f,0xd4,0xfa,0x86,0x4b,0xce,0x6c,0xa9,0xdf, -0xa4,0x4e,0xa6,0xf7,0x79,0xbe,0xb0,0xfc,0x88,0xee,0x5d,0x07, -0x4d,0xab,0x02,0x23,0x75,0xf0,0x99,0x14,0xec,0x39,0xc3,0xa6, -0x28,0xeb,0x3f,0x5a,0xef,0xb7,0xdf,0x0d,0x80,0x5b,0x70,0x73, -0x3e,0xa0,0x33,0xd7,0x46,0x94,0x2a,0xc3,0x24,0x20,0x18,0xbd, -0x1c,0xa5,0x82,0x56,0x85,0xb5,0xd5,0x26,0x0c,0xeb,0xba,0x0e, -0xea,0x24,0x28,0xf5,0x21,0x8c,0xa3,0x28,0x0a,0x21,0xbf,0x4b, -0xd9,0xbc,0x48,0xa1,0x9e,0xa7,0x12,0x49,0x9a,0xa6,0x61,0x1b, -0x85,0xd4,0x0c,0x47,0xd5,0x0b,0x46,0xaf,0x6e,0x1d,0x11,0xa8, -0x34,0x37,0x5c,0x9f,0xf9,0xf7,0xa6,0x82,0x1d,0xfc,0x4a,0xad, -0x28,0x33,0xfc,0xf2,0x28,0xd4,0xef,0xf0,0x8f,0xd1,0x59,0xf0, -0xfa,0x43,0xd9,0x80,0xa0,0x08,0xad,0x9a,0xbf,0x20,0x8a,0xd6, -0x18,0x71,0xd5,0x0c,0xdf,0xdf,0x53,0xf6,0x7c,0xd0,0xe5,0x49, -0xe5,0x19,0x56,0xbc,0x46,0x83,0x2c,0xe0,0xb9,0x31,0x15,0x65, -0x3c,0xc3,0x7d,0x9f,0xa9,0x19,0x54,0xd4,0x16,0xe8,0x49,0x48, -0x99,0xe1,0x6f,0xc8,0x8f,0xf0,0xfb,0x01,0x23,0x00,0x7c,0x4c, -0x22,0x1f,0x80,0x52,0xc2,0x7c,0xb2,0x0a,0x56,0x31,0xf1,0x22, -0x3f,0x69,0x1c,0x0b,0x8f,0x24,0xc1,0x62,0xbd,0xe8,0x2d,0xb7, -0xb0,0xc8,0xeb,0xd2,0x5c,0xd4,0xbb,0x8b,0x76,0xcb,0xa8,0xb9, -0x71,0x7e,0xff,0xae,0xa6,0x03,0xfe,0x61,0x75,0xeb,0x96,0x3c, -0x24,0xde,0xa2,0x45,0x77,0x94,0xbc,0x9e,0xdb,0x27,0x04,0x34, -0xbd,0xc5,0x3a,0x88,0x93,0x94,0xf9,0x69,0xb0,0x24,0x29,0xd0, -0x24,0x8d,0xbd,0xf4,0xd7,0xc1,0x9a,0xac,0x7a,0xc3,0x2d,0x23, -0x02,0x1f,0xe3,0x28,0x58,0x3c,0x00,0xe9,0x38,0x58,0xaf,0x1e, -0x00,0xb7,0x7b,0x62,0x1d,0x96,0xe7,0xea,0xbc,0x16,0xab,0x37, -0xdc,0xf2,0x31,0x71,0x39,0xae,0xbb,0x77,0xe3,0xf1,0x09,0x87, -0x13,0x53,0x6e,0xa4,0xb3,0x7b,0x77,0x2f,0xed,0x10,0xb4,0xfd, -0x3f,0xd4,0xde,0x9e,0xc0,0x91,0xda,0x5b,0xef,0xd7,0xaf,0xf6, -0x65,0x1a,0x2c,0xd2,0x55,0xb3,0x2c,0xa3,0x64,0x56,0xf1,0x83, -0xcc,0xcf,0x54,0xbd,0x06,0x06,0x2d,0x61,0x47,0x17,0x0f,0x4f, -0x40,0x2d,0x72,0x5b,0x64,0xd8,0x81,0x63,0x54,0x70,0x71,0x28, -0x2c,0x4c,0x2d,0x09,0x08,0x21,0x93,0x9f,0xf0,0x3d,0x62,0x0c, -0x89,0x4d,0xe1,0x17,0x86,0x85,0x53,0x90,0xa4,0xf1,0x17,0x80, -0x75,0x7a,0x9c,0x91,0xe3,0xd0,0x1c,0x2a,0xd3,0xdd,0xb5,0x03, -0x88,0x22,0xde,0xfd,0x54,0x4a,0x59,0xd6,0xf0,0xe6,0xf8,0x99, -0x5a,0xae,0xd8,0xeb,0x36,0x04,0xe7,0xbc,0xc8,0x5b,0xa8,0x7f, -0xde,0x39,0x18,0x09,0xf8,0x78,0xa5,0x2b,0x1f,0x76,0xf8,0x6f, -0x24,0xa5,0x30,0x76,0x96,0x23,0xc4,0x27,0x08,0xb6,0x67,0x08, -0x31,0x2e,0x65,0x45,0xf3,0x5c,0xa8,0x43,0x3b,0xfe,0xc6,0x6e, -0x94,0xd5,0xd9,0xc3,0x9a,0x82,0xd3,0x7c,0x62,0xd0,0x36,0xbf, -0x63,0xa4,0xe8,0x91,0xfb,0x4d,0x2a,0xec,0x73,0xf7,0x48,0x59, -0x01,0x3b,0x46,0xbf,0x80,0x73,0x1b,0xda,0xcf,0x28,0xbf,0x4d, -0xe5,0x36,0xdd,0x61,0x15,0x78,0xc6,0x4c,0xb6,0xb6,0x79,0x0b, -0x4f,0xc1,0xeb,0xe6,0xfe,0xd0,0xbc,0xe2,0x14,0xf4,0xf2,0xd4, -0xcf,0x45,0x28,0xd4,0x3f,0x9b,0x89,0xd3,0xd3,0x31,0xdb,0x5d, -0x2e,0x7d,0x56,0xd0,0xec,0xeb,0x7a,0x9d,0xde,0x44,0x9f,0x3e, -0x19,0x68,0x83,0xa3,0x6b,0xef,0xcf,0x13,0xd5,0xdc,0x5d,0x6d, -0xe0,0xdd,0xa0,0xcb,0xfb,0x37,0x7e,0x10,0x0c,0xe7,0xea,0xfd, -0xe6,0xc6,0x31,0xe8,0xc6,0x12,0xb0,0x93,0xd6,0x5c,0x59,0xb4, -0x45,0xb1,0x87,0xee,0x4a,0x4a,0x4d,0xd5,0x81,0xcf,0xd7,0x2c, -0x07,0x35,0x9a,0xe7,0x73,0x05,0xbb,0x0c,0x91,0x08,0xbe,0xf6, -0x4c,0xdc,0xe3,0xb3,0x9b,0xeb,0x70,0xfc,0x33,0x95,0x27,0xb8, -0x9b,0xde,0x0c,0x71,0xd8,0xe1,0x2f,0xa4,0x4e,0xc7,0x3d,0xd7, -0x1b,0x82,0xe0,0x3b,0xd7,0xd1,0xcc,0x74,0xfa,0x17,0xf9,0x84, -0x56,0x8f,0xe4,0x31,0x96,0x02,0x38,0x1b,0xc5,0x4f,0x9e,0xab, -0xee,0xb1,0x5b,0xfe,0x0e,0x00,0x00,0xff,0xff,0xf4,0x18,0x69, -0x45,0x2a,0x0a,0x00,0x00, - })) + var empty [0]byte + sx := (*reflect.StringHeader)(unsafe.Pointer(&_views_stats_html)) + b := empty[:] + bx := (*reflect.SliceHeader)(unsafe.Pointer(&b)) + bx.Data = sx.Data + bx.Len = len(_views_stats_html) + bx.Cap = bx.Len + + gz, err := gzip.NewReader(bytes.NewBuffer(b)) if err != nil { panic("Decompression failed: " + err.Error()) } - var b bytes.Buffer - io.Copy(&b, gz) + var buf bytes.Buffer + io.Copy(&buf, gz) gz.Close() - return b.Bytes() + return buf.Bytes() } + func init() { go_bindata["/views/stats.html"] = views_stats_html } From 6156d5c790b7a3c112353ade581cbea90cb2d0f6 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 9 Nov 2013 17:55:54 -0800 Subject: [PATCH 231/247] refactor separate etcd index from raft index --- error/error.go | 5 +- server/join_command.go | 20 +-- server/registry.go | 18 +-- server/remove_command.go | 2 +- server/server.go | 8 +- server/v1/get_key_handler.go | 2 +- server/v1/set_key_handler.go | 4 +- server/v1/watch_key_handler.go | 7 +- server/v2/get_handler.go | 36 ++--- server/v2/post_handler.go | 2 +- server/v2/put_handler.go | 8 +- server/v2/tests/delete_handler_test.go | 2 +- server/v2/tests/get_handler_test.go | 23 ++- server/v2/tests/post_handler_test.go | 8 +- server/v2/tests/put_handler_test.go | 22 +-- store/event.go | 11 +- store/event_history.go | 28 +--- store/event_test.go | 24 +-- store/heap_test.go | 4 +- store/node.go | 37 ++--- store/stats_test.go | 38 ++--- store/store.go | 151 ++++++++++--------- store/store_test.go | 200 ++++++++++++------------- store/v2/compare_and_swap_command.go | 3 +- store/v2/create_command.go | 2 +- store/v2/delete_command.go | 2 +- store/v2/set_command.go | 2 +- store/v2/update_command.go | 2 +- store/watcher_hub.go | 64 ++------ store/watcher_test.go | 6 +- 30 files changed, 334 insertions(+), 407 deletions(-) diff --git a/error/error.go b/error/error.go index 22b55906b04..e4555eb6ecc 100644 --- a/error/error.go +++ b/error/error.go @@ -64,16 +64,14 @@ type Error struct { Message string `json:"message"` Cause string `json:"cause,omitempty"` Index uint64 `json:"index"` - Term uint64 `json:"term"` } -func NewError(errorCode int, cause string, index uint64, term uint64) *Error { +func NewError(errorCode int, cause string, index uint64) *Error { return &Error{ ErrorCode: errorCode, Message: errors[errorCode], Cause: cause, Index: index, - Term: term, } } @@ -93,7 +91,6 @@ func (e Error) toJsonString() string { func (e Error) Write(w http.ResponseWriter) { w.Header().Add("X-Etcd-Index", fmt.Sprint(e.Index)) - w.Header().Add("X-Etcd-Term", fmt.Sprint(e.Term)) // 3xx is reft internal error if e.ErrorCode/100 == 3 { http.Error(w, e.toJsonString(), http.StatusInternalServerError) diff --git a/server/join_command.go b/server/join_command.go index 49dab4f2eaa..6e7f7471bdd 100644 --- a/server/join_command.go +++ b/server/join_command.go @@ -14,20 +14,20 @@ func init() { // The JoinCommand adds a node to the cluster. type JoinCommand struct { - MinVersion int `json:"minVersion"` - MaxVersion int `json:"maxVersion"` - Name string `json:"name"` - RaftURL string `json:"raftURL"` - EtcdURL string `json:"etcdURL"` + MinVersion int `json:"minVersion"` + MaxVersion int `json:"maxVersion"` + Name string `json:"name"` + RaftURL string `json:"raftURL"` + EtcdURL string `json:"etcdURL"` } func NewJoinCommand(minVersion int, maxVersion int, name, raftUrl, etcdUrl string) *JoinCommand { return &JoinCommand{ MinVersion: minVersion, MaxVersion: maxVersion, - Name: name, - RaftURL: raftUrl, - EtcdURL: etcdUrl, + Name: name, + RaftURL: raftUrl, + EtcdURL: etcdUrl, } } @@ -54,11 +54,11 @@ func (c *JoinCommand) Apply(server raft.Server) (interface{}, error) { // Check machine number in the cluster if ps.registry.Count() == ps.MaxClusterSize { log.Debug("Reject join request from ", c.Name) - return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMoreMachine, "", server.CommitIndex(), server.Term()) + return []byte{0}, etcdErr.NewError(etcdErr.EcodeNoMoreMachine, "", server.CommitIndex()) } // Add to shared machine registry. - ps.registry.Register(c.Name, c.RaftURL, c.EtcdURL, server.CommitIndex(), server.Term()) + ps.registry.Register(c.Name, c.RaftURL, c.EtcdURL) // Add peer in raft err := server.AddPeer(c.Name, "") diff --git a/server/registry.go b/server/registry.go index 05cccc10c11..b7d23809927 100644 --- a/server/registry.go +++ b/server/registry.go @@ -38,20 +38,20 @@ func NewRegistry(s store.Store) *Registry { } // Adds a node to the registry. -func (r *Registry) Register(name string, peerURL string, url string, commitIndex uint64, term uint64) error { +func (r *Registry) Register(name string, peerURL string, url string) error { r.Lock() defer r.Unlock() // Write data to store. key := path.Join(RegistryKey, name) value := fmt.Sprintf("raft=%s&etcd=%s", peerURL, url) - _, err := r.store.Create(key, value, false, store.Permanent, commitIndex, term) + _, err := r.store.Create(key, value, false, store.Permanent) log.Debugf("Register: %s", name) return err } // Removes a node from the registry. -func (r *Registry) Unregister(name string, commitIndex uint64, term uint64) error { +func (r *Registry) Unregister(name string) error { r.Lock() defer r.Unlock() @@ -59,14 +59,14 @@ func (r *Registry) Unregister(name string, commitIndex uint64, term uint64) erro // delete(r.nodes, name) // Remove the key from the store. - _, err := r.store.Delete(path.Join(RegistryKey, name), false, commitIndex, term) + _, err := r.store.Delete(path.Join(RegistryKey, name), false) log.Debugf("Unregister: %s", name) return err } // Returns the number of nodes in the cluster. func (r *Registry) Count() int { - e, err := r.store.Get(RegistryKey, false, false, 0, 0) + e, err := r.store.Get(RegistryKey, false, false) if err != nil { return 0 } @@ -133,7 +133,7 @@ func (r *Registry) urls(leaderName, selfName string, url func(name string) (stri } // Retrieve a list of all nodes. - if e, _ := r.store.Get(RegistryKey, false, false, 0, 0); e != nil { + if e, _ := r.store.Get(RegistryKey, false, false); e != nil { // Lookup the URL for each one. for _, pair := range e.KVPairs { _, name := filepath.Split(pair.Key) @@ -160,7 +160,7 @@ func (r *Registry) load(name string) { } // Retrieve from store. - e, err := r.store.Get(path.Join(RegistryKey, name), false, false, 0, 0) + e, err := r.store.Get(path.Join(RegistryKey, name), false, false) if err != nil { return } @@ -173,7 +173,7 @@ func (r *Registry) load(name string) { // Create node. r.nodes[name] = &node{ - url: m["etcd"][0], - peerURL: m["raft"][0], + url: m["etcd"][0], + peerURL: m["raft"][0], } } diff --git a/server/remove_command.go b/server/remove_command.go index 42e2c507867..bb1e2bb9696 100644 --- a/server/remove_command.go +++ b/server/remove_command.go @@ -27,7 +27,7 @@ func (c *RemoveCommand) Apply(server raft.Server) (interface{}, error) { ps, _ := server.Context().(*PeerServer) // Remove node from the shared registry. - err := ps.registry.Unregister(c.Name, server.CommitIndex(), server.Term()) + err := ps.registry.Unregister(c.Name) // Delete from stats delete(ps.followersStats.Followers, c.Name) diff --git a/server/server.go b/server/server.go index 48757ad2ba5..e2b54f8b55a 100644 --- a/server/server.go +++ b/server/server.go @@ -234,7 +234,7 @@ func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Reque } if result == nil { - return etcdErr.NewError(300, "Empty result from raft", store.UndefIndex, store.UndefTerm) + return etcdErr.NewError(300, "Empty result from raft", s.Store().Index()) } // response for raft related commands[join/remove] @@ -268,7 +268,7 @@ func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Reque // No leader available. if leader == "" { - return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) + return etcdErr.NewError(300, "", s.Store().Index()) } var url string @@ -317,7 +317,7 @@ func (s *Server) GetVersionHandler(w http.ResponseWriter, req *http.Request) err func (s *Server) GetLeaderHandler(w http.ResponseWriter, req *http.Request) error { leader := s.peerServer.RaftServer().Leader() if leader == "" { - return etcdErr.NewError(etcdErr.EcodeLeaderElect, "", store.UndefIndex, store.UndefTerm) + return etcdErr.NewError(etcdErr.EcodeLeaderElect, "", s.Store().Index()) } w.WriteHeader(http.StatusOK) url, _ := s.registry.PeerURL(leader) @@ -348,7 +348,7 @@ func (s *Server) GetLeaderStatsHandler(w http.ResponseWriter, req *http.Request) leader := s.peerServer.RaftServer().Leader() if leader == "" { - return etcdErr.NewError(300, "", store.UndefIndex, store.UndefTerm) + return etcdErr.NewError(300, "", s.Store().Index()) } hostname, _ := s.registry.ClientURL(leader) redirect(hostname, w, req) diff --git a/server/v1/get_key_handler.go b/server/v1/get_key_handler.go index 53558e142ff..880bf289e36 100644 --- a/server/v1/get_key_handler.go +++ b/server/v1/get_key_handler.go @@ -13,7 +13,7 @@ func GetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { key := "/" + vars["key"] // Retrieve the key from the store. - event, err := s.Store().Get(key, false, false, s.CommitIndex(), s.Term()) + event, err := s.Store().Get(key, false, false) if err != nil { return err } diff --git a/server/v1/set_key_handler.go b/server/v1/set_key_handler.go index 7acfe7ecb21..b1b4390a248 100644 --- a/server/v1/set_key_handler.go +++ b/server/v1/set_key_handler.go @@ -19,13 +19,13 @@ func SetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { // Parse non-blank value. value := req.Form.Get("value") if len(value) == 0 { - return etcdErr.NewError(200, "Set", store.UndefIndex, store.UndefTerm) + return etcdErr.NewError(200, "Set", s.Store().Index()) } // Convert time-to-live to an expiration time. expireTime, err := store.TTL(req.Form.Get("ttl")) if err != nil { - return etcdErr.NewError(202, "Set", store.UndefIndex, store.UndefTerm) + return etcdErr.NewError(202, "Set", s.Store().Index()) } // If the "prevValue" is specified then test-and-set. Otherwise create a new key. diff --git a/server/v1/watch_key_handler.go b/server/v1/watch_key_handler.go index e8db56c30b4..de5ed065620 100644 --- a/server/v1/watch_key_handler.go +++ b/server/v1/watch_key_handler.go @@ -6,7 +6,6 @@ import ( "strconv" etcdErr "github.com/coreos/etcd/error" - "github.com/coreos/etcd/store" "github.com/gorilla/mux" ) @@ -21,14 +20,14 @@ func WatchKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { if req.Method == "POST" { sinceIndex, err = strconv.ParseUint(string(req.FormValue("index")), 10, 64) if err != nil { - return etcdErr.NewError(203, "Watch From Index", store.UndefIndex, store.UndefTerm) + return etcdErr.NewError(203, "Watch From Index", s.Store().Index()) } } // Start the watcher on the store. - c, err := s.Store().Watch(key, false, sinceIndex, s.CommitIndex(), s.Term()) + c, err := s.Store().Watch(key, false, sinceIndex) if err != nil { - return etcdErr.NewError(500, key, store.UndefIndex, store.UndefTerm) + return etcdErr.NewError(500, key, s.Store().Index()) } event := <-c diff --git a/server/v2/get_handler.go b/server/v2/get_handler.go index dba491cbd25..f2d05c507db 100644 --- a/server/v2/get_handler.go +++ b/server/v2/get_handler.go @@ -16,7 +16,6 @@ import ( func GetHandler(w http.ResponseWriter, req *http.Request, s Server) error { var err error var event *store.Event - events := make([]*store.Event, 0) vars := mux.Vars(req) key := "/" + vars["key"] @@ -42,51 +41,36 @@ func GetHandler(w http.ResponseWriter, req *http.Request, s Server) error { if waitIndex != "" { sinceIndex, err = strconv.ParseUint(string(req.FormValue("waitIndex")), 10, 64) if err != nil { - return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index", store.UndefIndex, store.UndefTerm) + return etcdErr.NewError(etcdErr.EcodeIndexNaN, "Watch From Index", s.Store().Index()) } } // Start the watcher on the store. - eventChan, err := s.Store().Watch(key, recursive, sinceIndex, s.CommitIndex(), s.Term()) + eventChan, err := s.Store().Watch(key, recursive, sinceIndex) if err != nil { - return etcdErr.NewError(500, key, store.UndefIndex, store.UndefTerm) + return etcdErr.NewError(500, key, s.Store().Index()) } cn, _ := w.(http.CloseNotifier) closeChan := cn.CloseNotify() - eventLoop: - for { - select { - case <-closeChan: - return nil - case event = <-eventChan: - // for events other than expire, just one event for one watch - // for expire event, we might have a stream of events - // we use a nil item to terminate the expire event stream - if event != nil && event.Action == store.Expire { - events = append(events, event) - } else { - events = append(events, event) - break eventLoop - } - } + select { + case <-closeChan: + return nil + case event = <-eventChan: } } else { //get // Retrieve the key from the store. - event, err = s.Store().Get(key, recursive, sorted, s.CommitIndex(), s.Term()) + event, err = s.Store().Get(key, recursive, sorted) if err != nil { return err } } - var b []byte - - w.Header().Add("X-Etcd-Index", fmt.Sprint(events[0].Index)) - w.Header().Add("X-Etcd-Term", fmt.Sprint(events[0].Term)) + w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) w.WriteHeader(http.StatusOK) - b, _ = json.Marshal(events) + b, _ := json.Marshal(event) w.Write(b) diff --git a/server/v2/post_handler.go b/server/v2/post_handler.go index 4dc98b92510..2338d0250e2 100644 --- a/server/v2/post_handler.go +++ b/server/v2/post_handler.go @@ -15,7 +15,7 @@ func PostHandler(w http.ResponseWriter, req *http.Request, s Server) error { value := req.FormValue("value") expireTime, err := store.TTL(req.FormValue("ttl")) if err != nil { - return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create", store.UndefIndex, store.UndefTerm) + return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Create", s.Store().Index()) } c := s.Store().CommandFactory().CreateCreateCommand(key, value, expireTime, true) diff --git a/server/v2/put_handler.go b/server/v2/put_handler.go index 3afb018b726..5b61e58b39c 100644 --- a/server/v2/put_handler.go +++ b/server/v2/put_handler.go @@ -22,7 +22,7 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { value := req.Form.Get("value") expireTime, err := store.TTL(req.Form.Get("ttl")) if err != nil { - return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", store.UndefIndex, store.UndefTerm) + return etcdErr.NewError(etcdErr.EcodeTTLNaN, "Update", s.Store().Index()) } _, valueOk := req.Form["prevValue"] @@ -59,7 +59,7 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { // bad previous index if err != nil { - return etcdErr.NewError(etcdErr.EcodeIndexNaN, "CompareAndSwap", store.UndefIndex, store.UndefTerm) + return etcdErr.NewError(etcdErr.EcodeIndexNaN, "CompareAndSwap", s.Store().Index()) } } else { prevIndex = 0 @@ -67,7 +67,7 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { if valueOk { if prevValue == "" { - return etcdErr.NewError(etcdErr.EcodePrevValueRequired, "CompareAndSwap", store.UndefIndex, store.UndefTerm) + return etcdErr.NewError(etcdErr.EcodePrevValueRequired, "CompareAndSwap", s.Store().Index()) } } @@ -88,7 +88,7 @@ func CreateHandler(w http.ResponseWriter, req *http.Request, s Server, key, valu func UpdateHandler(w http.ResponseWriter, req *http.Request, s Server, key, value string, expireTime time.Time) error { // Update should give at least one option if value == "" && expireTime.Sub(store.Permanent) == 0 { - return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", store.UndefIndex, store.UndefTerm) + return etcdErr.NewError(etcdErr.EcodeValueOrTTLRequired, "Update", s.Store().Index()) } c := s.Store().CommandFactory().CreateUpdateCommand(key, value, expireTime) diff --git a/server/v2/tests/delete_handler_test.go b/server/v2/tests/delete_handler_test.go index ab710c5ac32..2905c68c484 100644 --- a/server/v2/tests/delete_handler_test.go +++ b/server/v2/tests/delete_handler_test.go @@ -24,6 +24,6 @@ func TestV2DeleteKey(t *testing.T) { resp, err = tests.DeleteForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), url.Values{}) body := tests.ReadBody(resp) assert.Nil(t, err, "") - assert.Equal(t, string(body), `{"action":"delete","key":"/foo/bar","prevValue":"XXX","index":4,"term":0}`, "") + assert.Equal(t, string(body), `{"action":"delete","key":"/foo/bar","prevValue":"XXX","index":2}`, "") }) } diff --git a/server/v2/tests/get_handler_test.go b/server/v2/tests/get_handler_test.go index 3b365872cb1..0f005405563 100644 --- a/server/v2/tests/get_handler_test.go +++ b/server/v2/tests/get_handler_test.go @@ -27,8 +27,7 @@ func TestV2GetKey(t *testing.T) { assert.Equal(t, body["action"], "get", "") assert.Equal(t, body["key"], "/foo/bar", "") assert.Equal(t, body["value"], "XXX", "") - assert.Equal(t, body["index"], 3, "") - assert.Equal(t, body["term"], 0, "") + assert.Equal(t, body["index"], 1, "") }) } @@ -55,7 +54,7 @@ func TestV2GetKeyRecursively(t *testing.T) { assert.Equal(t, body["action"], "get", "") assert.Equal(t, body["key"], "/foo", "") assert.Equal(t, body["dir"], true, "") - assert.Equal(t, body["index"], 4, "") + assert.Equal(t, body["index"], 2, "") assert.Equal(t, len(body["kvs"].([]interface{})), 2, "") kv0 := body["kvs"].([]interface{})[0].(map[string]interface{}) @@ -81,9 +80,11 @@ func TestV2GetKeyRecursively(t *testing.T) { func TestV2WatchKey(t *testing.T) { tests.RunServer(func(s *server.Server) { var body map[string]interface{} + c := make(chan bool) go func() { resp, _ := tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar?wait=true")) body = tests.ReadBodyJSON(resp) + c <- true }() // Make sure response didn't fire early. @@ -98,12 +99,19 @@ func TestV2WatchKey(t *testing.T) { // A response should follow from the GET above. time.Sleep(1 * time.Millisecond) + + select { + case <-c: + + default: + t.Fatal("cannot get watch result") + } + assert.NotNil(t, body, "") assert.Equal(t, body["action"], "set", "") assert.Equal(t, body["key"], "/foo/bar", "") assert.Equal(t, body["value"], "XXX", "") - assert.Equal(t, body["index"], 3, "") - assert.Equal(t, body["term"], 0, "") + assert.Equal(t, body["index"], 1, "") }) } @@ -118,7 +126,7 @@ func TestV2WatchKeyWithIndex(t *testing.T) { var body map[string]interface{} c := make(chan bool) go func() { - resp, _ := tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar?wait=true&waitIndex=4")) + resp, _ := tests.Get(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar?wait=true&waitIndex=2")) body = tests.ReadBodyJSON(resp) c <- true }() @@ -156,7 +164,6 @@ func TestV2WatchKeyWithIndex(t *testing.T) { assert.Equal(t, body["action"], "set", "") assert.Equal(t, body["key"], "/foo/bar", "") assert.Equal(t, body["value"], "YYY", "") - assert.Equal(t, body["index"], 4, "") - assert.Equal(t, body["term"], 0, "") + assert.Equal(t, body["index"], 2, "") }) } diff --git a/server/v2/tests/post_handler_test.go b/server/v2/tests/post_handler_test.go index 8e7e23a8ce0..655278f2224 100644 --- a/server/v2/tests/post_handler_test.go +++ b/server/v2/tests/post_handler_test.go @@ -21,18 +21,18 @@ func TestV2CreateUnique(t *testing.T) { resp, _ := tests.PostForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), nil) body := tests.ReadBodyJSON(resp) assert.Equal(t, body["action"], "create", "") - assert.Equal(t, body["key"], "/foo/bar/3", "") + assert.Equal(t, body["key"], "/foo/bar/1", "") assert.Equal(t, body["dir"], true, "") - assert.Equal(t, body["index"], 3, "") + assert.Equal(t, body["index"], 1, "") // Second POST should add next index to list. resp, _ = tests.PostForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), nil) body = tests.ReadBodyJSON(resp) - assert.Equal(t, body["key"], "/foo/bar/4", "") + assert.Equal(t, body["key"], "/foo/bar/2", "") // POST to a different key should add index to that list. resp, _ = tests.PostForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/baz"), nil) body = tests.ReadBodyJSON(resp) - assert.Equal(t, body["key"], "/foo/baz/5", "") + assert.Equal(t, body["key"], "/foo/baz/3", "") }) } diff --git a/server/v2/tests/put_handler_test.go b/server/v2/tests/put_handler_test.go index 1341d03b302..73f6be91c41 100644 --- a/server/v2/tests/put_handler_test.go +++ b/server/v2/tests/put_handler_test.go @@ -22,7 +22,7 @@ func TestV2SetKey(t *testing.T) { resp, err := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) body := tests.ReadBody(resp) assert.Nil(t, err, "") - assert.Equal(t, string(body), `{"action":"set","key":"/foo/bar","value":"XXX","index":3,"term":0}`, "") + assert.Equal(t, string(body), `{"action":"set","key":"/foo/bar","value":"XXX","index":1}`, "") }) } @@ -42,7 +42,7 @@ func TestV2SetKeyWithTTL(t *testing.T) { // Make sure the expiration date is correct. expiration, _ := time.Parse(time.RFC3339Nano, body["expiration"].(string)) - assert.Equal(t, expiration.Sub(t0) / time.Second, 20, "") + assert.Equal(t, expiration.Sub(t0)/time.Second, 20, "") }) } @@ -110,7 +110,7 @@ func TestV2UpdateKeySuccess(t *testing.T) { v.Set("value", "XXX") resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) tests.ReadBody(resp) - + v.Set("value", "YYY") v.Set("prevExist", "true") resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) @@ -160,7 +160,7 @@ func TestV2UpdateKeyFailOnMissingDirectory(t *testing.T) { // Ensures that a key is set only if the previous index matches. // // $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=XXX -// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY -d prevIndex=3 +// $ curl -X PUT localhost:4001/v2/keys/foo/bar -d value=YYY -d prevIndex=1 // func TestV2SetKeyCASOnIndexSuccess(t *testing.T) { tests.RunServer(func(s *server.Server) { @@ -169,13 +169,13 @@ func TestV2SetKeyCASOnIndexSuccess(t *testing.T) { resp, _ := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) tests.ReadBody(resp) v.Set("value", "YYY") - v.Set("prevIndex", "3") + v.Set("prevIndex", "1") resp, _ = tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) body := tests.ReadBodyJSON(resp) assert.Equal(t, body["action"], "compareAndSwap", "") assert.Equal(t, body["prevValue"], "XXX", "") assert.Equal(t, body["value"], "YYY", "") - assert.Equal(t, body["index"], 4, "") + assert.Equal(t, body["index"], 2, "") }) } @@ -196,8 +196,8 @@ func TestV2SetKeyCASOnIndexFail(t *testing.T) { body := tests.ReadBodyJSON(resp) assert.Equal(t, body["errorCode"], 101, "") assert.Equal(t, body["message"], "Test Failed", "") - assert.Equal(t, body["cause"], "[ != XXX] [10 != 3]", "") - assert.Equal(t, body["index"], 4, "") + assert.Equal(t, body["cause"], "[ != XXX] [10 != 1]", "") + assert.Equal(t, body["index"], 1, "") }) } @@ -236,7 +236,7 @@ func TestV2SetKeyCASOnValueSuccess(t *testing.T) { assert.Equal(t, body["action"], "compareAndSwap", "") assert.Equal(t, body["prevValue"], "XXX", "") assert.Equal(t, body["value"], "YYY", "") - assert.Equal(t, body["index"], 4, "") + assert.Equal(t, body["index"], 2, "") }) } @@ -257,8 +257,8 @@ func TestV2SetKeyCASOnValueFail(t *testing.T) { body := tests.ReadBodyJSON(resp) assert.Equal(t, body["errorCode"], 101, "") assert.Equal(t, body["message"], "Test Failed", "") - assert.Equal(t, body["cause"], "[AAA != XXX] [0 != 3]", "") - assert.Equal(t, body["index"], 4, "") + assert.Equal(t, body["cause"], "[AAA != XXX] [0 != 1]", "") + assert.Equal(t, body["index"], 1, "") }) } diff --git a/store/event.go b/store/event.go index ef007c642ac..50d2872daec 100644 --- a/store/event.go +++ b/store/event.go @@ -14,11 +14,6 @@ const ( Expire = "expire" ) -const ( - UndefIndex = 0 - UndefTerm = 0 -) - type Event struct { Action string `json:"action"` Key string `json:"key, omitempty"` @@ -28,17 +23,15 @@ type Event struct { KVPairs kvPairs `json:"kvs,omitempty"` Expiration *time.Time `json:"expiration,omitempty"` TTL int64 `json:"ttl,omitempty"` // Time to live in second - // The command index of the raft machine when the command is executed + // The index of the etcd state machine when the comment is executed Index uint64 `json:"index"` - Term uint64 `json:"term"` } -func newEvent(action string, key string, index uint64, term uint64) *Event { +func newEvent(action string, key string, index uint64) *Event { return &Event{ Action: action, Key: key, Index: index, - Term: term, } } diff --git a/store/event_history.go b/store/event_history.go index 4d11a8883e8..aaa93d44e0d 100644 --- a/store/event_history.go +++ b/store/event_history.go @@ -12,8 +12,6 @@ type EventHistory struct { Queue eventQueue StartIndex uint64 LastIndex uint64 - LastTerm uint64 - DupCnt uint64 // help to compute the watching point with duplicated indexes in the queue rwl sync.RWMutex } @@ -31,14 +29,9 @@ func (eh *EventHistory) addEvent(e *Event) *Event { eh.rwl.Lock() defer eh.rwl.Unlock() - if e.Index == eh.LastIndex { - eh.DupCnt += 1 - } - eh.Queue.insert(e) eh.LastIndex = e.Index - eh.LastTerm = e.Term eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index @@ -47,7 +40,7 @@ func (eh *EventHistory) addEvent(e *Event) *Event { // scan function is enumerating events from the index in history and // stops till the first point where the key has identified prefix -func (eh *EventHistory) scan(prefix string, index uint64) ([]*Event, *etcdErr.Error) { +func (eh *EventHistory) scan(prefix string, index uint64) (*Event, *etcdErr.Error) { eh.rwl.RLock() defer eh.rwl.RUnlock() @@ -56,7 +49,7 @@ func (eh *EventHistory) scan(prefix string, index uint64) ([]*Event, *etcdErr.Er return nil, etcdErr.NewError(etcdErr.EcodeEventIndexCleared, fmt.Sprintf("the requested history has been cleared [%v/%v]", - eh.StartIndex, index), UndefIndex, UndefTerm) + eh.StartIndex, index), 0) } // the index should locate before the size of the queue minus the duplicate count @@ -66,28 +59,17 @@ func (eh *EventHistory) scan(prefix string, index uint64) ([]*Event, *etcdErr.Er i := eh.Queue.Front - events := make([]*Event, 0) - var eventIndex uint64 - for { e := eh.Queue.Events[i] - if eventIndex != 0 && eventIndex != e.Index { - return events, nil - } - if strings.HasPrefix(e.Key, prefix) && index <= e.Index { // make sure we bypass the smaller one - eventIndex = e.Index - events = append(events, e) + return e, nil } i = (i + 1) % eh.Queue.Capacity if i > eh.Queue.back() { - if eventIndex == 0 { // find nothing, return and watch from current index - return nil, nil - } - return events, nil + return nil, nil } } } @@ -110,8 +92,6 @@ func (eh *EventHistory) clone() *EventHistory { StartIndex: eh.StartIndex, Queue: clonedQueue, LastIndex: eh.LastIndex, - LastTerm: eh.LastTerm, - DupCnt: eh.DupCnt, } } diff --git a/store/event_test.go b/store/event_test.go index aedf7f7da15..8a70d09a2d9 100644 --- a/store/event_test.go +++ b/store/event_test.go @@ -13,7 +13,7 @@ func TestEventQueue(t *testing.T) { // Add for i := 0; i < 200; i++ { - e := newEvent(Create, "/foo", uint64(i), 1) + e := newEvent(Create, "/foo", uint64(i)) eh.addEvent(e) } @@ -35,27 +35,27 @@ func TestScanHistory(t *testing.T) { eh := newEventHistory(100) // Add - eh.addEvent(newEvent(Create, "/foo", 1, 1)) - eh.addEvent(newEvent(Create, "/foo/bar", 2, 1)) - eh.addEvent(newEvent(Create, "/foo/foo", 3, 1)) - eh.addEvent(newEvent(Create, "/foo/bar/bar", 4, 1)) - eh.addEvent(newEvent(Create, "/foo/foo/foo", 5, 1)) + eh.addEvent(newEvent(Create, "/foo", 1)) + eh.addEvent(newEvent(Create, "/foo/bar", 2)) + eh.addEvent(newEvent(Create, "/foo/foo", 3)) + eh.addEvent(newEvent(Create, "/foo/bar/bar", 4)) + eh.addEvent(newEvent(Create, "/foo/foo/foo", 5)) e, err := eh.scan("/foo", 1) - if err != nil || e[0].Index != 1 { - t.Fatalf("scan error [/foo] [1] %v", e[0].Index) + if err != nil || e.Index != 1 { + t.Fatalf("scan error [/foo] [1] %v", e.Index) } e, err = eh.scan("/foo/bar", 1) - if err != nil || e[0].Index != 2 { - t.Fatalf("scan error [/foo/bar] [2] %v", e[0].Index) + if err != nil || e.Index != 2 { + t.Fatalf("scan error [/foo/bar] [2] %v", e.Index) } e, err = eh.scan("/foo/bar", 3) - if err != nil || e[0].Index != 4 { - t.Fatalf("scan error [/foo/bar/bar] [4] %v", e[0].Index) + if err != nil || e.Index != 4 { + t.Fatalf("scan error [/foo/bar/bar] [4] %v", e.Index) } e, err = eh.scan("/foo/bar", 6) diff --git a/store/heap_test.go b/store/heap_test.go index aa0b9caf613..9175feef955 100644 --- a/store/heap_test.go +++ b/store/heap_test.go @@ -14,7 +14,7 @@ func TestHeapPushPop(t *testing.T) { for i := 0; i < 10; i++ { path := fmt.Sprintf("%v", 10-i) m := time.Duration(10 - i) - n := newKV(nil, path, path, 0, 0, nil, "", time.Now().Add(time.Second*m)) + n := newKV(nil, path, path, 0, nil, "", time.Now().Add(time.Second*m)) h.push(n) } @@ -40,7 +40,7 @@ func TestHeapUpdate(t *testing.T) { for i, n := range kvs { path := fmt.Sprintf("%v", 10-i) m := time.Duration(10 - i) - n = newKV(nil, path, path, 0, 0, nil, "", time.Now().Add(time.Second*m)) + n = newKV(nil, path, path, 0, nil, "", time.Now().Add(time.Second*m)) kvs[i] = n h.push(n) } diff --git a/store/node.go b/store/node.go index 6b11727c88f..a0916b81a82 100644 --- a/store/node.go +++ b/store/node.go @@ -8,11 +8,6 @@ import ( etcdErr "github.com/coreos/etcd/error" ) -const ( - normal = iota - removed -) - var Permanent time.Time // Node is the basic element in the store system. @@ -26,7 +21,7 @@ type Node struct { ModifiedIndex uint64 ModifiedTerm uint64 - Parent *Node `json:"-"` // should not encode this field! avoid cyclical dependency. + Parent *Node `json:"-"` // should not encode this field! avoid circular dependency. ExpireTime time.Time ACL string @@ -39,14 +34,12 @@ type Node struct { // newKV creates a Key-Value pair func newKV(store *store, nodePath string, value string, createIndex uint64, - createTerm uint64, parent *Node, ACL string, expireTime time.Time) *Node { + parent *Node, ACL string, expireTime time.Time) *Node { return &Node{ Path: nodePath, CreateIndex: createIndex, - CreateTerm: createTerm, ModifiedIndex: createIndex, - ModifiedTerm: createTerm, Parent: parent, ACL: ACL, store: store, @@ -56,13 +49,12 @@ func newKV(store *store, nodePath string, value string, createIndex uint64, } // newDir creates a directory -func newDir(store *store, nodePath string, createIndex uint64, createTerm uint64, - parent *Node, ACL string, expireTime time.Time) *Node { +func newDir(store *store, nodePath string, createIndex uint64, parent *Node, + ACL string, expireTime time.Time) *Node { return &Node{ Path: nodePath, CreateIndex: createIndex, - CreateTerm: createTerm, Parent: parent, ACL: ACL, ExpireTime: expireTime, @@ -101,7 +93,7 @@ func (n *Node) IsDir() bool { // If the receiver node is not a key-value pair, a "Not A File" error will be returned. func (n *Node) Read() (string, *etcdErr.Error) { if n.IsDir() { - return "", etcdErr.NewError(etcdErr.EcodeNotFile, "", UndefIndex, UndefTerm) + return "", etcdErr.NewError(etcdErr.EcodeNotFile, "", n.store.Index()) } return n.Value, nil @@ -109,14 +101,13 @@ func (n *Node) Read() (string, *etcdErr.Error) { // Write function set the value of the node to the given value. // If the receiver node is a directory, a "Not A File" error will be returned. -func (n *Node) Write(value string, index uint64, term uint64) *etcdErr.Error { +func (n *Node) Write(value string, index uint64) *etcdErr.Error { if n.IsDir() { - return etcdErr.NewError(etcdErr.EcodeNotFile, "", UndefIndex, UndefTerm) + return etcdErr.NewError(etcdErr.EcodeNotFile, "", n.store.Index()) } n.Value = value n.ModifiedIndex = index - n.ModifiedTerm = term return nil } @@ -132,7 +123,7 @@ func (n *Node) ExpirationAndTTL() (*time.Time, int64) { // If the receiver node is not a directory, a "Not A Directory" error will be returned. func (n *Node) List() ([]*Node, *etcdErr.Error) { if !n.IsDir() { - return nil, etcdErr.NewError(etcdErr.EcodeNotDir, "", UndefIndex, UndefTerm) + return nil, etcdErr.NewError(etcdErr.EcodeNotDir, "", n.store.Index()) } nodes := make([]*Node, len(n.Children)) @@ -150,7 +141,7 @@ func (n *Node) List() ([]*Node, *etcdErr.Error) { // On success, it returns the file node func (n *Node) GetChild(name string) (*Node, *etcdErr.Error) { if !n.IsDir() { - return nil, etcdErr.NewError(etcdErr.EcodeNotDir, n.Path, UndefIndex, UndefTerm) + return nil, etcdErr.NewError(etcdErr.EcodeNotDir, n.Path, n.store.Index()) } child, ok := n.Children[name] @@ -168,7 +159,7 @@ func (n *Node) GetChild(name string) (*Node, *etcdErr.Error) { // error will be returned func (n *Node) Add(child *Node) *etcdErr.Error { if !n.IsDir() { - return etcdErr.NewError(etcdErr.EcodeNotDir, "", UndefIndex, UndefTerm) + return etcdErr.NewError(etcdErr.EcodeNotDir, "", n.store.Index()) } _, name := path.Split(child.Path) @@ -176,7 +167,7 @@ func (n *Node) Add(child *Node) *etcdErr.Error { _, ok := n.Children[name] if ok { - return etcdErr.NewError(etcdErr.EcodeNodeExist, "", UndefIndex, UndefTerm) + return etcdErr.NewError(etcdErr.EcodeNodeExist, "", n.store.Index()) } n.Children[name] = child @@ -189,7 +180,7 @@ func (n *Node) Remove(recursive bool, callback func(path string)) *etcdErr.Error if n.IsDir() && !recursive { // cannot delete a directory without set recursive to true - return etcdErr.NewError(etcdErr.EcodeNotFile, "", UndefIndex, UndefTerm) + return etcdErr.NewError(etcdErr.EcodeNotFile, "", n.store.Index()) } if !n.IsDir() { // key-value pair @@ -309,10 +300,10 @@ func (n *Node) UpdateTTL(expireTime time.Time) { // If the node is a key-value pair, it will clone the pair. func (n *Node) Clone() *Node { if !n.IsDir() { - return newKV(n.store, n.Path, n.Value, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime) + return newKV(n.store, n.Path, n.Value, n.CreateIndex, n.Parent, n.ACL, n.ExpireTime) } - clone := newDir(n.store, n.Path, n.CreateIndex, n.CreateTerm, n.Parent, n.ACL, n.ExpireTime) + clone := newDir(n.store, n.Path, n.CreateIndex, n.Parent, n.ACL, n.ExpireTime) for key, child := range n.Children { clone.Children[key] = child.Clone() diff --git a/store/stats_test.go b/store/stats_test.go index 9b3172ad8a7..ad16261b85c 100644 --- a/store/stats_test.go +++ b/store/stats_test.go @@ -10,85 +10,85 @@ import ( // Ensure that a successful Get is recorded in the stats. func TestStoreStatsGetSuccess(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 3, 1) - s.Get("/foo", false, false, 3, 1) + s.Create("/foo", "bar", false, Permanent) + s.Get("/foo", false, false) assert.Equal(t, uint64(1), s.Stats.GetSuccess, "") } // Ensure that a failed Get is recorded in the stats. func TestStoreStatsGetFail(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 3, 1) - s.Get("/no_such_key", false, false, 3, 1) + s.Create("/foo", "bar", false, Permanent) + s.Get("/no_such_key", false, false) assert.Equal(t, uint64(1), s.Stats.GetFail, "") } // Ensure that a successful Create is recorded in the stats. func TestStoreStatsCreateSuccess(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 3, 1) + s.Create("/foo", "bar", false, Permanent) assert.Equal(t, uint64(1), s.Stats.CreateSuccess, "") } // Ensure that a failed Create is recorded in the stats. func TestStoreStatsCreateFail(t *testing.T) { s := newStore() - s.Create("/foo", "", false, Permanent, 3, 1) - s.Create("/foo", "bar", false, Permanent, 4, 1) + s.Create("/foo", "", false, Permanent) + s.Create("/foo", "bar", false, Permanent) assert.Equal(t, uint64(1), s.Stats.CreateFail, "") } // Ensure that a successful Update is recorded in the stats. func TestStoreStatsUpdateSuccess(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 3, 1) - s.Update("/foo", "baz", Permanent, 4, 1) + s.Create("/foo", "bar", false, Permanent) + s.Update("/foo", "baz", Permanent) assert.Equal(t, uint64(1), s.Stats.UpdateSuccess, "") } // Ensure that a failed Update is recorded in the stats. func TestStoreStatsUpdateFail(t *testing.T) { s := newStore() - s.Update("/foo", "bar", Permanent, 4, 1) + s.Update("/foo", "bar", Permanent) assert.Equal(t, uint64(1), s.Stats.UpdateFail, "") } // Ensure that a successful CAS is recorded in the stats. func TestStoreStatsCompareAndSwapSuccess(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 3, 1) - s.CompareAndSwap("/foo", "bar", 0, "baz", Permanent, 4, 1) + s.Create("/foo", "bar", false, Permanent) + s.CompareAndSwap("/foo", "bar", 0, "baz", Permanent) assert.Equal(t, uint64(1), s.Stats.CompareAndSwapSuccess, "") } // Ensure that a failed CAS is recorded in the stats. func TestStoreStatsCompareAndSwapFail(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 3, 1) - s.CompareAndSwap("/foo", "wrong_value", 0, "baz", Permanent, 4, 1) + s.Create("/foo", "bar", false, Permanent) + s.CompareAndSwap("/foo", "wrong_value", 0, "baz", Permanent) assert.Equal(t, uint64(1), s.Stats.CompareAndSwapFail, "") } // Ensure that a successful Delete is recorded in the stats. func TestStoreStatsDeleteSuccess(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 3, 1) - s.Delete("/foo", false, 4, 1) + s.Create("/foo", "bar", false, Permanent) + s.Delete("/foo", false) assert.Equal(t, uint64(1), s.Stats.DeleteSuccess, "") } // Ensure that a failed Delete is recorded in the stats. func TestStoreStatsDeleteFail(t *testing.T) { s := newStore() - s.Delete("/foo", false, 4, 1) + s.Delete("/foo", false) assert.Equal(t, uint64(1), s.Stats.DeleteFail, "") } //Ensure that the number of expirations is recorded in the stats. func TestStoreStatsExpireCount(t *testing.T) { s := newStore() - go mockSyncService(s.deleteExpiredKeys) - s.Create("/foo", "bar", false, time.Now().Add(500*time.Millisecond), 3, 1) + go mockSyncService(s.DeleteExpiredKeys) + s.Create("/foo", "bar", false, time.Now().Add(500*time.Millisecond)) assert.Equal(t, uint64(0), s.Stats.ExpireCount, "") time.Sleep(600 * time.Millisecond) assert.Equal(t, uint64(1), s.Stats.ExpireCount, "") diff --git a/store/store.go b/store/store.go index 5e374f1ff12..f9ebab7ada6 100644 --- a/store/store.go +++ b/store/store.go @@ -19,27 +19,27 @@ const defaultVersion = 2 type Store interface { Version() int CommandFactory() CommandFactory - Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) - Set(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) - Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) - Create(nodePath string, value string, incrementalSuffix bool, expireTime time.Time, - index uint64, term uint64) (*Event, error) + Index() uint64 + Get(nodePath string, recursive, sorted bool) (*Event, error) + Set(nodePath string, value string, expireTime time.Time) (*Event, error) + Update(nodePath string, newValue string, expireTime time.Time) (*Event, error) + Create(nodePath string, value string, incrementalSuffix bool, + expireTime time.Time) (*Event, error) CompareAndSwap(nodePath string, prevValue string, prevIndex uint64, - value string, expireTime time.Time, index uint64, term uint64) (*Event, error) - Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error) - Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error) + value string, expireTime time.Time) (*Event, error) + Delete(nodePath string, recursive bool) (*Event, error) + Watch(prefix string, recursive bool, sinceIndex uint64) (<-chan *Event, error) Save() ([]byte, error) Recovery(state []byte) error TotalTransactions() uint64 JsonStats() []byte - DeleteExpiredKeys(cutoff time.Time, index uint64, term uint64) + DeleteExpiredKeys(cutoff time.Time) } type store struct { Root *Node WatcherHub *watcherHub - Index uint64 - Term uint64 + CurrentIndex uint64 Stats *Stats CurrentVersion int ttlKeyHeap *ttlKeyHeap // need to recovery manually @@ -53,7 +53,7 @@ func New() Store { func newStore() *store { s := new(store) s.CurrentVersion = defaultVersion - s.Root = newDir(s, "/", UndefIndex, UndefTerm, nil, "", Permanent) + s.Root = newDir(s, "/", s.CurrentIndex, nil, "", Permanent) s.Stats = newStats() s.WatcherHub = newWatchHub(1000) s.ttlKeyHeap = newTtlKeyHeap() @@ -65,6 +65,11 @@ func (s *store) Version() int { return s.CurrentVersion } +// Retrieves current of the store +func (s *store) Index() uint64 { + return s.CurrentIndex +} + // CommandFactory retrieves the command factory for the current version of the store. func (s *store) CommandFactory() CommandFactory { return GetCommandFactory(s.Version()) @@ -73,20 +78,20 @@ func (s *store) CommandFactory() CommandFactory { // Get function returns a get event. // If recursive is true, it will return all the content under the node path. // If sorted is true, it will sort the content by keys. -func (s *store) Get(nodePath string, recursive, sorted bool, index uint64, term uint64) (*Event, error) { +func (s *store) Get(nodePath string, recursive, sorted bool) (*Event, error) { s.worldLock.RLock() defer s.worldLock.RUnlock() nodePath = path.Clean(path.Join("/", nodePath)) - n, err := s.internalGet(nodePath, index, term) + n, err := s.internalGet(nodePath) if err != nil { s.Stats.Inc(GetFail) return nil, err } - e := newEvent(Get, nodePath, index, term) + e := newEvent(Get, nodePath, s.CurrentIndex) if n.IsDir() { // node is a directory e.Dir = true @@ -128,13 +133,12 @@ func (s *store) Get(nodePath string, recursive, sorted bool, index uint64, term // Create function creates the Node at nodePath. Create will help to create intermediate directories with no ttl. // If the node has already existed, create will fail. // If any node on the path is a file, create will fail. -func (s *store) Create(nodePath string, value string, unique bool, - expireTime time.Time, index uint64, term uint64) (*Event, error) { +func (s *store) Create(nodePath string, value string, unique bool, expireTime time.Time) (*Event, error) { nodePath = path.Clean(path.Join("/", nodePath)) s.worldLock.Lock() defer s.worldLock.Unlock() - e, err := s.internalCreate(nodePath, value, unique, false, expireTime, index, term, Create) + e, err := s.internalCreate(nodePath, value, unique, false, expireTime, Create) if err == nil { s.Stats.Inc(CreateSuccess) @@ -146,12 +150,12 @@ func (s *store) Create(nodePath string, value string, unique bool, } // Set function creates or replace the Node at nodePath. -func (s *store) Set(nodePath string, value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { +func (s *store) Set(nodePath string, value string, expireTime time.Time) (*Event, error) { nodePath = path.Clean(path.Join("/", nodePath)) s.worldLock.Lock() defer s.worldLock.Unlock() - e, err := s.internalCreate(nodePath, value, false, true, expireTime, index, term, Set) + e, err := s.internalCreate(nodePath, value, false, true, expireTime, Set) if err == nil { s.Stats.Inc(SetSuccess) @@ -163,14 +167,14 @@ func (s *store) Set(nodePath string, value string, expireTime time.Time, index u } func (s *store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint64, - value string, expireTime time.Time, index uint64, term uint64) (*Event, error) { + value string, expireTime time.Time) (*Event, error) { nodePath = path.Clean(path.Join("/", nodePath)) s.worldLock.Lock() defer s.worldLock.Unlock() - n, err := s.internalGet(nodePath, index, term) + n, err := s.internalGet(nodePath) if err != nil { s.Stats.Inc(CompareAndSwapFail) @@ -179,17 +183,20 @@ func (s *store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint if n.IsDir() { // can only test and set file s.Stats.Inc(CompareAndSwapFail) - return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term) + return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, s.CurrentIndex) } // If both of the prevValue and prevIndex are given, we will test both of them. // Command will be executed, only if both of the tests are successful. if (prevValue == "" || n.Value == prevValue) && (prevIndex == 0 || n.ModifiedIndex == prevIndex) { - e := newEvent(CompareAndSwap, nodePath, index, term) + // update etcd index + s.CurrentIndex++ + + e := newEvent(CompareAndSwap, nodePath, s.CurrentIndex) e.PrevValue = n.Value // if test succeed, write the value - n.Write(value, index, term) + n.Write(value, s.CurrentIndex) n.UpdateTTL(expireTime) e.Value = value @@ -202,25 +209,25 @@ func (s *store) CompareAndSwap(nodePath string, prevValue string, prevIndex uint cause := fmt.Sprintf("[%v != %v] [%v != %v]", prevValue, n.Value, prevIndex, n.ModifiedIndex) s.Stats.Inc(CompareAndSwapFail) - return nil, etcdErr.NewError(etcdErr.EcodeTestFailed, cause, index, term) + return nil, etcdErr.NewError(etcdErr.EcodeTestFailed, cause, s.CurrentIndex) } // Delete function deletes the node at the given path. // If the node is a directory, recursive must be true to delete it. -func (s *store) Delete(nodePath string, recursive bool, index uint64, term uint64) (*Event, error) { +func (s *store) Delete(nodePath string, recursive bool) (*Event, error) { nodePath = path.Clean(path.Join("/", nodePath)) s.worldLock.Lock() defer s.worldLock.Unlock() - n, err := s.internalGet(nodePath, index, term) + n, err := s.internalGet(nodePath) if err != nil { // if the node does not exist, return error s.Stats.Inc(DeleteFail) return nil, err } - e := newEvent(Delete, nodePath, index, term) + e := newEvent(Delete, nodePath, s.CurrentIndex) if n.IsDir() { e.Dir = true @@ -240,33 +247,38 @@ func (s *store) Delete(nodePath string, recursive bool, index uint64, term uint6 return nil, err } + // update etcd index + s.CurrentIndex++ + e.Index = s.CurrentIndex + s.WatcherHub.notify(e) s.Stats.Inc(DeleteSuccess) return e, nil } -func (s *store) Watch(prefix string, recursive bool, sinceIndex uint64, index uint64, term uint64) (<-chan *Event, error) { +func (s *store) Watch(prefix string, recursive bool, sinceIndex uint64) (<-chan *Event, error) { prefix = path.Clean(path.Join("/", prefix)) + nextIndex := s.CurrentIndex + 1 + s.worldLock.RLock() defer s.worldLock.RUnlock() - s.Index, s.Term = index, term - var c <-chan *Event var err *etcdErr.Error if sinceIndex == 0 { - c, err = s.WatcherHub.watch(prefix, recursive, index+1) + c, err = s.WatcherHub.watch(prefix, recursive, nextIndex) } else { c, err = s.WatcherHub.watch(prefix, recursive, sinceIndex) } if err != nil { - err.Index = index - err.Term = term + // watchhub do not know the current Index + // we need to attach the currentIndex here + err.Index = s.CurrentIndex return nil, err } @@ -298,52 +310,59 @@ func (s *store) walk(nodePath string, walkFunc func(prev *Node, component string // Update function updates the value/ttl of the node. // If the node is a file, the value and the ttl can be updated. // If the node is a directory, only the ttl can be updated. -func (s *store) Update(nodePath string, newValue string, expireTime time.Time, index uint64, term uint64) (*Event, error) { +func (s *store) Update(nodePath string, newValue string, expireTime time.Time) (*Event, error) { s.worldLock.Lock() defer s.worldLock.Unlock() + + currIndex, nextIndex := s.CurrentIndex, s.CurrentIndex+1 + nodePath = path.Clean(path.Join("/", nodePath)) - n, err := s.internalGet(nodePath, index, term) + n, err := s.internalGet(nodePath) if err != nil { // if the node does not exist, return error s.Stats.Inc(UpdateFail) return nil, err } - e := newEvent(Update, nodePath, s.Index, s.Term) + e := newEvent(Update, nodePath, nextIndex) if len(newValue) != 0 { if n.IsDir() { // if the node is a directory, we cannot update value s.Stats.Inc(UpdateFail) - return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term) + return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, currIndex) } e.PrevValue = n.Value - n.Write(newValue, index, term) + n.Write(newValue, nextIndex) + e.Value = newValue + } else { + // do not update value + e.Value = n.Value } // update ttl n.UpdateTTL(expireTime) - e.Value = newValue - e.Expiration, e.TTL = n.ExpirationAndTTL() s.WatcherHub.notify(e) s.Stats.Inc(UpdateSuccess) + s.CurrentIndex = nextIndex + return e, nil } func (s *store) internalCreate(nodePath string, value string, unique bool, replace bool, - expireTime time.Time, index uint64, term uint64, action string) (*Event, error) { + expireTime time.Time, action string) (*Event, error) { - s.Index, s.Term = index, term + currIndex, nextIndex := s.CurrentIndex, s.CurrentIndex+1 if unique { // append unique item under the node path - nodePath += "/" + strconv.FormatUint(index, 10) + nodePath += "/" + strconv.FormatUint(nextIndex, 10) } nodePath = path.Clean(path.Join("/", nodePath)) @@ -355,11 +374,11 @@ func (s *store) internalCreate(nodePath string, value string, unique bool, repla if err != nil { s.Stats.Inc(SetFail) - err.Index, err.Term = s.Index, s.Term + err.Index = currIndex return nil, err } - e := newEvent(action, nodePath, s.Index, s.Term) + e := newEvent(action, nodePath, nextIndex) n, _ := d.GetChild(newNodeName) @@ -367,25 +386,25 @@ func (s *store) internalCreate(nodePath string, value string, unique bool, repla if n != nil { if replace { if n.IsDir() { - return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, index, term) + return nil, etcdErr.NewError(etcdErr.EcodeNotFile, nodePath, currIndex) } e.PrevValue, _ = n.Read() n.Remove(false, nil) } else { - return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath, index, term) + return nil, etcdErr.NewError(etcdErr.EcodeNodeExist, nodePath, currIndex) } } if len(value) != 0 { // create file e.Value = value - n = newKV(s, nodePath, value, index, term, d, "", expireTime) + n = newKV(s, nodePath, value, nextIndex, d, "", expireTime) } else { // create directory e.Dir = true - n = newDir(s, nodePath, index, term, d, "", expireTime) + n = newDir(s, nodePath, nextIndex, d, "", expireTime) } @@ -399,23 +418,20 @@ func (s *store) internalCreate(nodePath string, value string, unique bool, repla e.Expiration, e.TTL = n.ExpirationAndTTL() } + s.CurrentIndex = nextIndex + s.WatcherHub.notify(e) return e, nil } // InternalGet function get the node of the given nodePath. -func (s *store) internalGet(nodePath string, index uint64, term uint64) (*Node, *etcdErr.Error) { +func (s *store) internalGet(nodePath string) (*Node, *etcdErr.Error) { nodePath = path.Clean(path.Join("/", nodePath)) - // update file system known index and term - if index > s.Index { - s.Index, s.Term = index, term - } - walkFunc := func(parent *Node, name string) (*Node, *etcdErr.Error) { if !parent.IsDir() { - err := etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path, index, term) + err := etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path, s.CurrentIndex) return nil, err } @@ -424,7 +440,7 @@ func (s *store) internalGet(nodePath string, index uint64, term uint64) (*Node, return child, nil } - return nil, etcdErr.NewError(etcdErr.EcodeKeyNotFound, path.Join(parent.Path, name), index, term) + return nil, etcdErr.NewError(etcdErr.EcodeKeyNotFound, path.Join(parent.Path, name), s.CurrentIndex) } f, err := s.walk(nodePath, walkFunc) @@ -436,12 +452,10 @@ func (s *store) internalGet(nodePath string, index uint64, term uint64) (*Node, } // deleteExpiredKyes will delete all -func (s *store) DeleteExpiredKeys(cutoff time.Time, index uint64, term uint64) { +func (s *store) DeleteExpiredKeys(cutoff time.Time) { s.worldLock.Lock() defer s.worldLock.Unlock() - s.Index, s.Term = index, term - for { node := s.ttlKeyHeap.top() if node == nil || node.ExpireTime.After(cutoff) { @@ -451,10 +465,12 @@ func (s *store) DeleteExpiredKeys(cutoff time.Time, index uint64, term uint64) { s.ttlKeyHeap.pop() node.Remove(true, nil) + s.CurrentIndex++ + s.Stats.Inc(ExpireCount) - s.WatcherHub.notify(newEvent(Expire, node.Path, s.Index, s.Term)) + s.WatcherHub.notify(newEvent(Expire, node.Path, s.CurrentIndex)) } - s.WatcherHub.clearPendingWatchers() + } // checkDir function will check whether the component is a directory under parent node. @@ -469,10 +485,10 @@ func (s *store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) { return node, nil } - return nil, etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path, UndefIndex, UndefTerm) + return nil, etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path, s.CurrentIndex) } - n := newDir(s, path.Join(parent.Path, dirName), s.Index, s.Term, parent, parent.ACL, Permanent) + n := newDir(s, path.Join(parent.Path, dirName), s.CurrentIndex, parent, parent.ACL, Permanent) parent.Children[dirName] = n @@ -487,8 +503,7 @@ func (s *store) Save() ([]byte, error) { s.worldLock.Lock() clonedStore := newStore() - clonedStore.Index = s.Index - clonedStore.Term = s.Term + clonedStore.CurrentIndex = s.CurrentIndex clonedStore.Root = s.Root.Clone() clonedStore.WatcherHub = s.WatcherHub.clone() clonedStore.Stats = s.Stats.clone() diff --git a/store/store_test.go b/store/store_test.go index 1fc242d914e..898d510ab26 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -11,8 +11,8 @@ import ( // Ensure that the store can retrieve an existing value. func TestStoreGetValue(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 2, 1) - e, err := s.Get("/foo", false, false, 2, 1) + s.Create("/foo", "bar", false, Permanent) + e, err := s.Get("/foo", false, false) assert.Nil(t, err, "") assert.Equal(t, e.Action, "get", "") assert.Equal(t, e.Key, "/foo", "") @@ -23,14 +23,14 @@ func TestStoreGetValue(t *testing.T) { // Note that hidden files should not be returned. func TestStoreGetDirectory(t *testing.T) { s := newStore() - s.Create("/foo", "", false, Permanent, 2, 1) - s.Create("/foo/bar", "X", false, Permanent, 3, 1) - s.Create("/foo/_hidden", "*", false, Permanent, 4, 1) - s.Create("/foo/baz", "", false, Permanent, 5, 1) - s.Create("/foo/baz/bat", "Y", false, Permanent, 6, 1) - s.Create("/foo/baz/_hidden", "*", false, Permanent, 7, 1) - s.Create("/foo/baz/ttl", "Y", false, time.Now().Add(time.Second*3), 8, 1) - e, err := s.Get("/foo", true, false, 8, 1) + s.Create("/foo", "", false, Permanent) + s.Create("/foo/bar", "X", false, Permanent) + s.Create("/foo/_hidden", "*", false, Permanent) + s.Create("/foo/baz", "", false, Permanent) + s.Create("/foo/baz/bat", "Y", false, Permanent) + s.Create("/foo/baz/_hidden", "*", false, Permanent) + s.Create("/foo/baz/ttl", "Y", false, time.Now().Add(time.Second*3)) + e, err := s.Get("/foo", true, false) assert.Nil(t, err, "") assert.Equal(t, e.Action, "get", "") assert.Equal(t, e.Key, "/foo", "") @@ -53,13 +53,13 @@ func TestStoreGetDirectory(t *testing.T) { // Ensure that the store can retrieve a directory in sorted order. func TestStoreGetSorted(t *testing.T) { s := newStore() - s.Create("/foo", "", false, Permanent, 2, 1) - s.Create("/foo/x", "0", false, Permanent, 3, 1) - s.Create("/foo/z", "0", false, Permanent, 4, 1) - s.Create("/foo/y", "", false, Permanent, 5, 1) - s.Create("/foo/y/a", "0", false, Permanent, 6, 1) - s.Create("/foo/y/b", "0", false, Permanent, 7, 1) - e, err := s.Get("/foo", true, true, 8, 1) + s.Create("/foo", "", false, Permanent) + s.Create("/foo/x", "0", false, Permanent) + s.Create("/foo/z", "0", false, Permanent) + s.Create("/foo/y", "", false, Permanent) + s.Create("/foo/y/a", "0", false, Permanent) + s.Create("/foo/y/b", "0", false, Permanent) + e, err := s.Get("/foo", true, true) assert.Nil(t, err, "") assert.Equal(t, e.KVPairs[0].Key, "/foo/x", "") assert.Equal(t, e.KVPairs[1].Key, "/foo/y", "") @@ -71,7 +71,7 @@ func TestStoreGetSorted(t *testing.T) { // Ensure that the store can create a new key if it doesn't already exist. func TestStoreCreateValue(t *testing.T) { s := newStore() - e, err := s.Create("/foo", "bar", false, Permanent, 2, 1) + e, err := s.Create("/foo", "bar", false, Permanent) assert.Nil(t, err, "") assert.Equal(t, e.Action, "create", "") assert.Equal(t, e.Key, "/foo", "") @@ -81,14 +81,13 @@ func TestStoreCreateValue(t *testing.T) { assert.Nil(t, e.KVPairs, "") assert.Nil(t, e.Expiration, "") assert.Equal(t, e.TTL, 0, "") - assert.Equal(t, e.Index, uint64(2), "") - assert.Equal(t, e.Term, uint64(1), "") + assert.Equal(t, e.Index, uint64(1), "") } // Ensure that the store can create a new directory if it doesn't already exist. func TestStoreCreateDirectory(t *testing.T) { s := newStore() - e, err := s.Create("/foo", "", false, Permanent, 2, 1) + e, err := s.Create("/foo", "", false, Permanent) assert.Nil(t, err, "") assert.Equal(t, e.Action, "create", "") assert.Equal(t, e.Key, "/foo", "") @@ -98,22 +97,21 @@ func TestStoreCreateDirectory(t *testing.T) { // Ensure that the store fails to create a key if it already exists. func TestStoreCreateFailsIfExists(t *testing.T) { s := newStore() - s.Create("/foo", "", false, Permanent, 2, 1) - e, _err := s.Create("/foo", "", false, Permanent, 3, 1) + s.Create("/foo", "", false, Permanent) + e, _err := s.Create("/foo", "", false, Permanent) err := _err.(*etcdErr.Error) assert.Equal(t, err.ErrorCode, etcdErr.EcodeNodeExist, "") assert.Equal(t, err.Message, "Already exists", "") assert.Equal(t, err.Cause, "/foo", "") - assert.Equal(t, err.Index, uint64(3), "") - assert.Equal(t, err.Term, uint64(1), "") + assert.Equal(t, err.Index, uint64(1), "") assert.Nil(t, e, 0, "") } // Ensure that the store can update a key if it already exists. func TestStoreUpdateValue(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 2, 1) - e, err := s.Update("/foo", "baz", Permanent, 3, 1) + s.Create("/foo", "bar", false, Permanent) + e, err := s.Update("/foo", "baz", Permanent) assert.Nil(t, err, "") assert.Equal(t, e.Action, "update", "") assert.Equal(t, e.Key, "/foo", "") @@ -121,17 +119,16 @@ func TestStoreUpdateValue(t *testing.T) { assert.Equal(t, e.PrevValue, "bar", "") assert.Equal(t, e.Value, "baz", "") assert.Equal(t, e.TTL, 0, "") - assert.Equal(t, e.Index, uint64(3), "") - assert.Equal(t, e.Term, uint64(1), "") - e, _ = s.Get("/foo", false, false, 3, 1) + assert.Equal(t, e.Index, uint64(2), "") + e, _ = s.Get("/foo", false, false) assert.Equal(t, e.Value, "baz", "") } // Ensure that the store cannot update a directory. func TestStoreUpdateFailsIfDirectory(t *testing.T) { s := newStore() - s.Create("/foo", "", false, Permanent, 2, 1) - e, _err := s.Update("/foo", "baz", Permanent, 3, 1) + s.Create("/foo", "", false, Permanent) + e, _err := s.Update("/foo", "baz", Permanent) err := _err.(*etcdErr.Error) assert.Equal(t, err.ErrorCode, etcdErr.EcodeNotFile, "") assert.Equal(t, err.Message, "Not A File", "") @@ -142,14 +139,14 @@ func TestStoreUpdateFailsIfDirectory(t *testing.T) { // Ensure that the store can update the TTL on a value. func TestStoreUpdateValueTTL(t *testing.T) { s := newStore() - go mockSyncService(s.deleteExpiredKeys) - s.Create("/foo", "bar", false, Permanent, 2, 1) - _, err := s.Update("/foo", "baz", time.Now().Add(500*time.Millisecond), 3, 1) - e, _ := s.Get("/foo", false, false, 3, 1) + go mockSyncService(s.DeleteExpiredKeys) + s.Create("/foo", "bar", false, Permanent) + _, err := s.Update("/foo", "baz", time.Now().Add(500*time.Millisecond)) + e, _ := s.Get("/foo", false, false) assert.Equal(t, e.Value, "baz", "") time.Sleep(600 * time.Millisecond) - e, err = s.Get("/foo", false, false, 3, 1) + e, err = s.Get("/foo", false, false) assert.Nil(t, e, "") assert.Equal(t, err.(*etcdErr.Error).ErrorCode, etcdErr.EcodeKeyNotFound, "") } @@ -157,15 +154,15 @@ func TestStoreUpdateValueTTL(t *testing.T) { // Ensure that the store can update the TTL on a directory. func TestStoreUpdateDirTTL(t *testing.T) { s := newStore() - go mockSyncService(s.deleteExpiredKeys) - s.Create("/foo", "", false, Permanent, 2, 1) - s.Create("/foo/bar", "baz", false, Permanent, 3, 1) - _, err := s.Update("/foo", "", time.Now().Add(500*time.Millisecond), 3, 1) - e, _ := s.Get("/foo/bar", false, false, 3, 1) + go mockSyncService(s.DeleteExpiredKeys) + s.Create("/foo", "", false, Permanent) + s.Create("/foo/bar", "baz", false, Permanent) + _, err := s.Update("/foo", "", time.Now().Add(500*time.Millisecond)) + e, _ := s.Get("/foo/bar", false, false) assert.Equal(t, e.Value, "baz", "") time.Sleep(600 * time.Millisecond) - e, err = s.Get("/foo/bar", false, false, 3, 1) + e, err = s.Get("/foo/bar", false, false) assert.Nil(t, e, "") assert.Equal(t, err.(*etcdErr.Error).ErrorCode, etcdErr.EcodeKeyNotFound, "") } @@ -173,8 +170,8 @@ func TestStoreUpdateDirTTL(t *testing.T) { // Ensure that the store can delete a value. func TestStoreDeleteValue(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 2, 1) - e, err := s.Delete("/foo", false, 3, 1) + s.Create("/foo", "bar", false, Permanent) + e, err := s.Delete("/foo", false) assert.Nil(t, err, "") assert.Equal(t, e.Action, "delete", "") } @@ -182,8 +179,8 @@ func TestStoreDeleteValue(t *testing.T) { // Ensure that the store can delete a directory if recursive is specified. func TestStoreDeleteDiretory(t *testing.T) { s := newStore() - s.Create("/foo", "", false, Permanent, 2, 1) - e, err := s.Delete("/foo", true, 3, 1) + s.Create("/foo", "", false, Permanent) + e, err := s.Delete("/foo", true) assert.Nil(t, err, "") assert.Equal(t, e.Action, "delete", "") } @@ -191,8 +188,8 @@ func TestStoreDeleteDiretory(t *testing.T) { // Ensure that the store cannot delete a directory if recursive is not specified. func TestStoreDeleteDiretoryFailsIfNonRecursive(t *testing.T) { s := newStore() - s.Create("/foo", "", false, Permanent, 2, 1) - e, _err := s.Delete("/foo", false, 3, 1) + s.Create("/foo", "", false, Permanent) + e, _err := s.Delete("/foo", false) err := _err.(*etcdErr.Error) assert.Equal(t, err.ErrorCode, etcdErr.EcodeNotFile, "") assert.Equal(t, err.Message, "Not A File", "") @@ -202,60 +199,60 @@ func TestStoreDeleteDiretoryFailsIfNonRecursive(t *testing.T) { // Ensure that the store can conditionally update a key if it has a previous value. func TestStoreCompareAndSwapPrevValue(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 2, 1) - e, err := s.CompareAndSwap("/foo", "bar", 0, "baz", Permanent, 3, 1) + s.Create("/foo", "bar", false, Permanent) + e, err := s.CompareAndSwap("/foo", "bar", 0, "baz", Permanent) assert.Nil(t, err, "") assert.Equal(t, e.Action, "compareAndSwap", "") assert.Equal(t, e.PrevValue, "bar", "") assert.Equal(t, e.Value, "baz", "") - e, _ = s.Get("/foo", false, false, 3, 1) + e, _ = s.Get("/foo", false, false) assert.Equal(t, e.Value, "baz", "") } // Ensure that the store cannot conditionally update a key if it has the wrong previous value. func TestStoreCompareAndSwapPrevValueFailsIfNotMatch(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 2, 1) - e, _err := s.CompareAndSwap("/foo", "wrong_value", 0, "baz", Permanent, 3, 1) + s.Create("/foo", "bar", false, Permanent) + e, _err := s.CompareAndSwap("/foo", "wrong_value", 0, "baz", Permanent) err := _err.(*etcdErr.Error) assert.Equal(t, err.ErrorCode, etcdErr.EcodeTestFailed, "") assert.Equal(t, err.Message, "Test Failed", "") assert.Nil(t, e, "") - e, _ = s.Get("/foo", false, false, 3, 1) + e, _ = s.Get("/foo", false, false) assert.Equal(t, e.Value, "bar", "") } // Ensure that the store can conditionally update a key if it has a previous index. func TestStoreCompareAndSwapPrevIndex(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 2, 1) - e, err := s.CompareAndSwap("/foo", "", 2, "baz", Permanent, 3, 1) + s.Create("/foo", "bar", false, Permanent) + e, err := s.CompareAndSwap("/foo", "", 1, "baz", Permanent) assert.Nil(t, err, "") assert.Equal(t, e.Action, "compareAndSwap", "") assert.Equal(t, e.PrevValue, "bar", "") assert.Equal(t, e.Value, "baz", "") - e, _ = s.Get("/foo", false, false, 3, 1) + e, _ = s.Get("/foo", false, false) assert.Equal(t, e.Value, "baz", "") } // Ensure that the store cannot conditionally update a key if it has the wrong previous index. func TestStoreCompareAndSwapPrevIndexFailsIfNotMatch(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 2, 1) - e, _err := s.CompareAndSwap("/foo", "", 100, "baz", Permanent, 3, 1) + s.Create("/foo", "bar", false, Permanent) + e, _err := s.CompareAndSwap("/foo", "", 100, "baz", Permanent) err := _err.(*etcdErr.Error) assert.Equal(t, err.ErrorCode, etcdErr.EcodeTestFailed, "") assert.Equal(t, err.Message, "Test Failed", "") assert.Nil(t, e, "") - e, _ = s.Get("/foo", false, false, 3, 1) + e, _ = s.Get("/foo", false, false) assert.Equal(t, e.Value, "bar", "") } // Ensure that the store can watch for key creation. func TestStoreWatchCreate(t *testing.T) { s := newStore() - c, _ := s.Watch("/foo", false, 0, 0, 1) - s.Create("/foo", "bar", false, Permanent, 2, 1) + c, _ := s.Watch("/foo", false, 0) + s.Create("/foo", "bar", false, Permanent) e := nbselect(c) assert.Equal(t, e.Action, "create", "") assert.Equal(t, e.Key, "/foo", "") @@ -266,8 +263,8 @@ func TestStoreWatchCreate(t *testing.T) { // Ensure that the store can watch for recursive key creation. func TestStoreWatchRecursiveCreate(t *testing.T) { s := newStore() - c, _ := s.Watch("/foo", true, 0, 0, 1) - s.Create("/foo/bar", "baz", false, Permanent, 2, 1) + c, _ := s.Watch("/foo", true, 0) + s.Create("/foo/bar", "baz", false, Permanent) e := nbselect(c) assert.Equal(t, e.Action, "create", "") assert.Equal(t, e.Key, "/foo/bar", "") @@ -276,9 +273,9 @@ func TestStoreWatchRecursiveCreate(t *testing.T) { // Ensure that the store can watch for key updates. func TestStoreWatchUpdate(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 2, 1) - c, _ := s.Watch("/foo", false, 0, 0, 1) - s.Update("/foo", "baz", Permanent, 3, 1) + s.Create("/foo", "bar", false, Permanent) + c, _ := s.Watch("/foo", false, 0) + s.Update("/foo", "baz", Permanent) e := nbselect(c) assert.Equal(t, e.Action, "update", "") assert.Equal(t, e.Key, "/foo", "") @@ -287,9 +284,9 @@ func TestStoreWatchUpdate(t *testing.T) { // Ensure that the store can watch for recursive key updates. func TestStoreWatchRecursiveUpdate(t *testing.T) { s := newStore() - s.Create("/foo/bar", "baz", false, Permanent, 2, 1) - c, _ := s.Watch("/foo", true, 0, 0, 1) - s.Update("/foo/bar", "baz", Permanent, 3, 1) + s.Create("/foo/bar", "baz", false, Permanent) + c, _ := s.Watch("/foo", true, 0) + s.Update("/foo/bar", "baz", Permanent) e := nbselect(c) assert.Equal(t, e.Action, "update", "") assert.Equal(t, e.Key, "/foo/bar", "") @@ -298,9 +295,9 @@ func TestStoreWatchRecursiveUpdate(t *testing.T) { // Ensure that the store can watch for key deletions. func TestStoreWatchDelete(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 2, 1) - c, _ := s.Watch("/foo", false, 0, 0, 1) - s.Delete("/foo", false, 3, 1) + s.Create("/foo", "bar", false, Permanent) + c, _ := s.Watch("/foo", false, 0) + s.Delete("/foo", false) e := nbselect(c) assert.Equal(t, e.Action, "delete", "") assert.Equal(t, e.Key, "/foo", "") @@ -309,9 +306,9 @@ func TestStoreWatchDelete(t *testing.T) { // Ensure that the store can watch for recursive key deletions. func TestStoreWatchRecursiveDelete(t *testing.T) { s := newStore() - s.Create("/foo/bar", "baz", false, Permanent, 2, 1) - c, _ := s.Watch("/foo", true, 0, 0, 1) - s.Delete("/foo/bar", false, 3, 1) + s.Create("/foo/bar", "baz", false, Permanent) + c, _ := s.Watch("/foo", true, 0) + s.Delete("/foo/bar", false) e := nbselect(c) assert.Equal(t, e.Action, "delete", "") assert.Equal(t, e.Key, "/foo/bar", "") @@ -320,9 +317,9 @@ func TestStoreWatchRecursiveDelete(t *testing.T) { // Ensure that the store can watch for CAS updates. func TestStoreWatchCompareAndSwap(t *testing.T) { s := newStore() - s.Create("/foo", "bar", false, Permanent, 2, 1) - c, _ := s.Watch("/foo", false, 0, 0, 1) - s.CompareAndSwap("/foo", "bar", 0, "baz", Permanent, 3, 1) + s.Create("/foo", "bar", false, Permanent) + c, _ := s.Watch("/foo", false, 0) + s.CompareAndSwap("/foo", "bar", 0, "baz", Permanent) e := nbselect(c) assert.Equal(t, e.Action, "compareAndSwap", "") assert.Equal(t, e.Key, "/foo", "") @@ -331,9 +328,9 @@ func TestStoreWatchCompareAndSwap(t *testing.T) { // Ensure that the store can watch for recursive CAS updates. func TestStoreWatchRecursiveCompareAndSwap(t *testing.T) { s := newStore() - s.Create("/foo/bar", "baz", false, Permanent, 2, 1) - c, _ := s.Watch("/foo", true, 0, 0, 1) - s.CompareAndSwap("/foo/bar", "baz", 0, "bat", Permanent, 3, 1) + s.Create("/foo/bar", "baz", false, Permanent) + c, _ := s.Watch("/foo", true, 0) + s.CompareAndSwap("/foo/bar", "baz", 0, "bat", Permanent) e := nbselect(c) assert.Equal(t, e.Action, "compareAndSwap", "") assert.Equal(t, e.Key, "/foo/bar", "") @@ -342,17 +339,18 @@ func TestStoreWatchRecursiveCompareAndSwap(t *testing.T) { // Ensure that the store can watch for key expiration. func TestStoreWatchExpire(t *testing.T) { s := newStore() - go mockSyncService(s.deleteExpiredKeys) - s.Create("/foo", "bar", false, time.Now().Add(500*time.Millisecond), 2, 1) - s.Create("/foofoo", "barbarbar", false, time.Now().Add(500*time.Millisecond), 2, 1) + go mockSyncService(s.DeleteExpiredKeys) + s.Create("/foo", "bar", false, time.Now().Add(500*time.Millisecond)) + s.Create("/foofoo", "barbarbar", false, time.Now().Add(500*time.Millisecond)) - c, _ := s.Watch("/", true, 0, 0, 1) + c, _ := s.Watch("/", true, 0) e := nbselect(c) assert.Nil(t, e, "") time.Sleep(600 * time.Millisecond) e = nbselect(c) assert.Equal(t, e.Action, "expire", "") assert.Equal(t, e.Key, "/foo", "") + c, _ = s.Watch("/", true, 4) e = nbselect(c) assert.Equal(t, e.Action, "expire", "") assert.Equal(t, e.Key, "/foofoo", "") @@ -361,19 +359,19 @@ func TestStoreWatchExpire(t *testing.T) { // Ensure that the store can recover from a previously saved state. func TestStoreRecover(t *testing.T) { s := newStore() - s.Create("/foo", "", false, Permanent, 2, 1) - s.Create("/foo/x", "bar", false, Permanent, 3, 1) - s.Create("/foo/y", "baz", false, Permanent, 4, 1) + s.Create("/foo", "", false, Permanent) + s.Create("/foo/x", "bar", false, Permanent) + s.Create("/foo/y", "baz", false, Permanent) b, err := s.Save() s2 := newStore() s2.Recovery(b) - e, err := s.Get("/foo/x", false, false, 4, 1) + e, err := s.Get("/foo/x", false, false) assert.Nil(t, err, "") assert.Equal(t, e.Value, "bar", "") - e, err = s.Get("/foo/y", false, false, 4, 1) + e, err = s.Get("/foo/y", false, false) assert.Nil(t, err, "") assert.Equal(t, e.Value, "baz", "") } @@ -381,25 +379,25 @@ func TestStoreRecover(t *testing.T) { // Ensure that the store can recover from a previously saved state that includes an expiring key. func TestStoreRecoverWithExpiration(t *testing.T) { s := newStore() - go mockSyncService(s.deleteExpiredKeys) - s.Create("/foo", "", false, Permanent, 2, 1) - s.Create("/foo/x", "bar", false, Permanent, 3, 1) - s.Create("/foo/y", "baz", false, time.Now().Add(5*time.Millisecond), 4, 1) + go mockSyncService(s.DeleteExpiredKeys) + s.Create("/foo", "", false, Permanent) + s.Create("/foo/x", "bar", false, Permanent) + s.Create("/foo/y", "baz", false, time.Now().Add(5*time.Millisecond)) b, err := s.Save() time.Sleep(10 * time.Millisecond) s2 := newStore() - go mockSyncService(s2.deleteExpiredKeys) + go mockSyncService(s2.DeleteExpiredKeys) s2.Recovery(b) time.Sleep(600 * time.Millisecond) - e, err := s.Get("/foo/x", false, false, 4, 1) + e, err := s.Get("/foo/x", false, false) assert.Nil(t, err, "") assert.Equal(t, e.Value, "bar", "") - e, err = s.Get("/foo/y", false, false, 4, 1) + e, err = s.Get("/foo/y", false, false) assert.NotNil(t, err, "") assert.Nil(t, e, "") } @@ -414,9 +412,9 @@ func nbselect(c <-chan *Event) *Event { } } -func mockSyncService(f func(now time.Time, index uint64, term uint64)) { +func mockSyncService(f func(now time.Time)) { ticker := time.Tick(time.Millisecond * 500) for now := range ticker { - f(now, 2, 1) + f(now) } } diff --git a/store/v2/compare_and_swap_command.go b/store/v2/compare_and_swap_command.go index 1de79fb1ff3..fe7572bd7aa 100644 --- a/store/v2/compare_and_swap_command.go +++ b/store/v2/compare_and_swap_command.go @@ -30,8 +30,7 @@ func (c *CompareAndSwapCommand) CommandName() string { func (c *CompareAndSwapCommand) Apply(server raft.Server) (interface{}, error) { s, _ := server.StateMachine().(store.Store) - e, err := s.CompareAndSwap(c.Key, c.PrevValue, c.PrevIndex, - c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) + e, err := s.CompareAndSwap(c.Key, c.PrevValue, c.PrevIndex, c.Value, c.ExpireTime) if err != nil { log.Debug(err) diff --git a/store/v2/create_command.go b/store/v2/create_command.go index e187d99f7f1..a6863ce0042 100644 --- a/store/v2/create_command.go +++ b/store/v2/create_command.go @@ -29,7 +29,7 @@ func (c *CreateCommand) CommandName() string { func (c *CreateCommand) Apply(server raft.Server) (interface{}, error) { s, _ := server.StateMachine().(store.Store) - e, err := s.Create(c.Key, c.Value, c.Unique, c.ExpireTime, server.CommitIndex(), server.Term()) + e, err := s.Create(c.Key, c.Value, c.Unique, c.ExpireTime) if err != nil { log.Debug(err) diff --git a/store/v2/delete_command.go b/store/v2/delete_command.go index 3e8bac81c37..1ac1d398751 100644 --- a/store/v2/delete_command.go +++ b/store/v2/delete_command.go @@ -25,7 +25,7 @@ func (c *DeleteCommand) CommandName() string { func (c *DeleteCommand) Apply(server raft.Server) (interface{}, error) { s, _ := server.StateMachine().(store.Store) - e, err := s.Delete(c.Key, c.Recursive, server.CommitIndex(), server.Term()) + e, err := s.Delete(c.Key, c.Recursive) if err != nil { log.Debug(err) diff --git a/store/v2/set_command.go b/store/v2/set_command.go index 4f6ecf59f1f..c340718531e 100644 --- a/store/v2/set_command.go +++ b/store/v2/set_command.go @@ -29,7 +29,7 @@ func (c *SetCommand) Apply(server raft.Server) (interface{}, error) { s, _ := server.StateMachine().(store.Store) // create a new node or replace the old node. - e, err := s.Set(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) + e, err := s.Set(c.Key, c.Value, c.ExpireTime) if err != nil { log.Debug(err) diff --git a/store/v2/update_command.go b/store/v2/update_command.go index d080ecced44..19eb68622de 100644 --- a/store/v2/update_command.go +++ b/store/v2/update_command.go @@ -27,7 +27,7 @@ func (c *UpdateCommand) CommandName() string { func (c *UpdateCommand) Apply(server raft.Server) (interface{}, error) { s, _ := server.StateMachine().(store.Store) - e, err := s.Update(c.Key, c.Value, c.ExpireTime, server.CommitIndex(), server.Term()) + e, err := s.Update(c.Key, c.Value, c.ExpireTime) if err != nil { log.Debug(err) diff --git a/store/watcher_hub.go b/store/watcher_hub.go index 52af68c2148..b952aec6da3 100644 --- a/store/watcher_hub.go +++ b/store/watcher_hub.go @@ -16,11 +16,9 @@ import ( // event happens between the end of the first watch command and the start // of the second command. type watcherHub struct { - watchers map[string]*list.List - count int64 // current number of watchers. - EventHistory *EventHistory - pendingWatchers map[*list.Element]*list.List - pendingList map[*list.List]string + watchers map[string]*list.List + count int64 // current number of watchers. + EventHistory *EventHistory } // newWatchHub creates a watchHub. The capacity determines how many events we will @@ -29,10 +27,8 @@ type watcherHub struct { // Ideally, it should smaller than 20K/s[max throughput] * 2 * 50ms[RTT] = 2000 func newWatchHub(capacity int) *watcherHub { return &watcherHub{ - watchers: make(map[string]*list.List), - EventHistory: newEventHistory(capacity), - pendingWatchers: make(map[*list.Element]*list.List), - pendingList: make(map[*list.List]string), + watchers: make(map[string]*list.List), + EventHistory: newEventHistory(capacity), } } @@ -41,22 +37,16 @@ func newWatchHub(capacity int) *watcherHub { // If recursive is false, the first change after index at prefix will be sent to the event channel. // If index is zero, watch will start from the current index + 1. func (wh *watcherHub) watch(prefix string, recursive bool, index uint64) (<-chan *Event, *etcdErr.Error) { - events, err := wh.EventHistory.scan(prefix, index) + event, err := wh.EventHistory.scan(prefix, index) if err != nil { return nil, err } - eventChan := make(chan *Event, len(events)+5) // use a buffered channel + eventChan := make(chan *Event, 1) // use a buffered channel - if events != nil { - for _, e := range events { - eventChan <- e - } - - if events[0].Action == Expire { - eventChan <- nil - } + if event != nil { + eventChan <- event return eventChan, nil } @@ -123,16 +113,11 @@ func (wh *watcherHub) notifyWatchers(e *Event, path string, deleted bool) { if w.notify(e, e.Key == path, deleted) { - if e.Action == Expire { - wh.pendingWatchers[curr] = l - wh.pendingList[l] = path - } else { - // if we successfully notify a watcher - // we need to remove the watcher from the list - // and decrease the counter - l.Remove(curr) - atomic.AddInt64(&wh.count, -1) - } + // if we successfully notify a watcher + // we need to remove the watcher from the list + // and decrease the counter + l.Remove(curr) + atomic.AddInt64(&wh.count, -1) } @@ -141,27 +126,6 @@ func (wh *watcherHub) notifyWatchers(e *Event, path string, deleted bool) { } } -func (wh *watcherHub) clearPendingWatchers() { - if len(wh.pendingWatchers) == 0 { // avoid making new maps - return - } - - for e, l := range wh.pendingWatchers { - l.Remove(e) - - if l.Len() == 0 { - path := wh.pendingList[l] - delete(wh.watchers, path) - } - - w, _ := e.Value.(*watcher) - w.eventChan <- nil - } - - wh.pendingWatchers = make(map[*list.Element]*list.List) - wh.pendingList = make(map[*list.List]string) -} - // clone function clones the watcherHub and return the cloned one. // only clone the static content. do not clone the current watchers. func (wh *watcherHub) clone() *watcherHub { diff --git a/store/watcher_test.go b/store/watcher_test.go index c3da475fca7..d1b62b76257 100644 --- a/store/watcher_test.go +++ b/store/watcher_test.go @@ -19,7 +19,7 @@ func TestWatcher(t *testing.T) { // do nothing } - e := newEvent(Create, "/foo/bar", 1, 1) + e := newEvent(Create, "/foo/bar", 1) wh.notify(e) @@ -31,7 +31,7 @@ func TestWatcher(t *testing.T) { c, _ = wh.watch("/foo", false, 2) - e = newEvent(Create, "/foo/bar", 2, 1) + e = newEvent(Create, "/foo/bar", 2) wh.notify(e) @@ -42,7 +42,7 @@ func TestWatcher(t *testing.T) { // do nothing } - e = newEvent(Create, "/foo", 3, 1) + e = newEvent(Create, "/foo", 3) wh.notify(e) From eca433cee5736152609b602bfdea7e5ec3f8f577 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 9 Nov 2013 18:59:43 -0800 Subject: [PATCH 232/247] fix add sync_command.go --- store/v2/sync_command.go | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) create mode 100644 store/v2/sync_command.go diff --git a/store/v2/sync_command.go b/store/v2/sync_command.go new file mode 100644 index 00000000000..4b645907929 --- /dev/null +++ b/store/v2/sync_command.go @@ -0,0 +1,29 @@ +package v2 + +import ( + "time" + + "github.com/coreos/etcd/store" + "github.com/coreos/go-raft" +) + +func init() { + raft.RegisterCommand(&SyncCommand{}) +} + +type SyncCommand struct { + Time time.Time `json:"time"` +} + +// The name of the Sync command in the log +func (c SyncCommand) CommandName() string { + return "etcd:sync" +} + +func (c SyncCommand) Apply(server raft.Server) (interface{}, error) { + + s, _ := server.StateMachine().(store.Store) + s.DeleteExpiredKeys(c.Time) + + return nil, nil +} From d87e0e93d3b14b9e606fbac81448c8981703b4d4 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 9 Nov 2013 19:05:38 -0800 Subject: [PATCH 233/247] fix get return the last modified index of the node --- store/node.go | 1 - store/store.go | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/store/node.go b/store/node.go index a0916b81a82..7ed78303c38 100644 --- a/store/node.go +++ b/store/node.go @@ -19,7 +19,6 @@ type Node struct { CreateIndex uint64 CreateTerm uint64 ModifiedIndex uint64 - ModifiedTerm uint64 Parent *Node `json:"-"` // should not encode this field! avoid circular dependency. diff --git a/store/store.go b/store/store.go index f9ebab7ada6..04bacb7115f 100644 --- a/store/store.go +++ b/store/store.go @@ -91,7 +91,7 @@ func (s *store) Get(nodePath string, recursive, sorted bool) (*Event, error) { return nil, err } - e := newEvent(Get, nodePath, s.CurrentIndex) + e := newEvent(Get, nodePath, n.ModifiedIndex) if n.IsDir() { // node is a directory e.Dir = true From d8e5994c352ae8929b9f89bdcf65f98e90edb8e2 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 9 Nov 2013 20:20:47 -0800 Subject: [PATCH 234/247] feat attach etcd-index,raft-index,raft-term to header --- server/server.go | 7 +++++++ server/v2/get_handler.go | 4 +++- store/node.go | 1 - 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/server/server.go b/server/server.go index e2b54f8b55a..a435fbce077 100644 --- a/server/server.go +++ b/server/server.go @@ -225,6 +225,7 @@ func (s *Server) Close() { } } +// Dispatch command to the current leader func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Request) error { ps := s.peerServer if ps.raftServer.State() == raft.Leader { @@ -252,6 +253,12 @@ func (s *Server) Dispatch(c raft.Command, w http.ResponseWriter, req *http.Reque e, _ := result.(*store.Event) b, _ = json.Marshal(e) + // etcd index should be the same as the event index + // which is also the last modified index of the node + w.Header().Add("X-Etcd-Index", fmt.Sprint(e.Index)) + w.Header().Add("X-Raft-Index", fmt.Sprint(s.CommitIndex())) + w.Header().Add("X-Raft-Term", fmt.Sprint(s.Term())) + if e.IsCreated() { w.WriteHeader(http.StatusCreated) } else { diff --git a/server/v2/get_handler.go b/server/v2/get_handler.go index f2d05c507db..3e8ddee6426 100644 --- a/server/v2/get_handler.go +++ b/server/v2/get_handler.go @@ -68,7 +68,9 @@ func GetHandler(w http.ResponseWriter, req *http.Request, s Server) error { } } - w.Header().Add("X-Etcd-Index", fmt.Sprint(event.Index)) + w.Header().Add("X-Etcd-Index", fmt.Sprint(s.Store().Index())) + w.Header().Add("X-Raft-Index", fmt.Sprint(s.CommitIndex())) + w.Header().Add("X-Raft-Term", fmt.Sprint(s.Term())) w.WriteHeader(http.StatusOK) b, _ := json.Marshal(event) diff --git a/store/node.go b/store/node.go index 7ed78303c38..9a7196bd3f7 100644 --- a/store/node.go +++ b/store/node.go @@ -17,7 +17,6 @@ type Node struct { Path string CreateIndex uint64 - CreateTerm uint64 ModifiedIndex uint64 Parent *Node `json:"-"` // should not encode this field! avoid circular dependency. From 06f1b7f2e8f4ae203adf384a4e77b9dd425bc19e Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 9 Nov 2013 20:49:19 -0800 Subject: [PATCH 235/247] refactor change Index to modifiedIndex --- server/v2/tests/delete_handler_test.go | 2 +- server/v2/tests/get_handler_test.go | 8 +++--- server/v2/tests/post_handler_test.go | 2 +- server/v2/tests/put_handler_test.go | 6 ++--- store/event.go | 34 ++++++++++++++------------ store/event_history.go | 6 ++--- store/event_test.go | 8 +++--- store/node.go | 15 ++++++------ store/store.go | 7 +++--- store/store_test.go | 4 +-- store/watcher.go | 2 +- 11 files changed, 50 insertions(+), 44 deletions(-) diff --git a/server/v2/tests/delete_handler_test.go b/server/v2/tests/delete_handler_test.go index 2905c68c484..997127a9ecc 100644 --- a/server/v2/tests/delete_handler_test.go +++ b/server/v2/tests/delete_handler_test.go @@ -24,6 +24,6 @@ func TestV2DeleteKey(t *testing.T) { resp, err = tests.DeleteForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), url.Values{}) body := tests.ReadBody(resp) assert.Nil(t, err, "") - assert.Equal(t, string(body), `{"action":"delete","key":"/foo/bar","prevValue":"XXX","index":2}`, "") + assert.Equal(t, string(body), `{"action":"delete","key":"/foo/bar","prevValue":"XXX","modifiedIndex":2}`, "") }) } diff --git a/server/v2/tests/get_handler_test.go b/server/v2/tests/get_handler_test.go index 0f005405563..b15195873db 100644 --- a/server/v2/tests/get_handler_test.go +++ b/server/v2/tests/get_handler_test.go @@ -27,7 +27,7 @@ func TestV2GetKey(t *testing.T) { assert.Equal(t, body["action"], "get", "") assert.Equal(t, body["key"], "/foo/bar", "") assert.Equal(t, body["value"], "XXX", "") - assert.Equal(t, body["index"], 1, "") + assert.Equal(t, body["modifiedIndex"], 1, "") }) } @@ -54,7 +54,7 @@ func TestV2GetKeyRecursively(t *testing.T) { assert.Equal(t, body["action"], "get", "") assert.Equal(t, body["key"], "/foo", "") assert.Equal(t, body["dir"], true, "") - assert.Equal(t, body["index"], 2, "") + assert.Equal(t, body["modifiedIndex"], 1, "") assert.Equal(t, len(body["kvs"].([]interface{})), 2, "") kv0 := body["kvs"].([]interface{})[0].(map[string]interface{}) @@ -111,7 +111,7 @@ func TestV2WatchKey(t *testing.T) { assert.Equal(t, body["action"], "set", "") assert.Equal(t, body["key"], "/foo/bar", "") assert.Equal(t, body["value"], "XXX", "") - assert.Equal(t, body["index"], 1, "") + assert.Equal(t, body["modifiedIndex"], 1, "") }) } @@ -164,6 +164,6 @@ func TestV2WatchKeyWithIndex(t *testing.T) { assert.Equal(t, body["action"], "set", "") assert.Equal(t, body["key"], "/foo/bar", "") assert.Equal(t, body["value"], "YYY", "") - assert.Equal(t, body["index"], 2, "") + assert.Equal(t, body["modifiedIndex"], 2, "") }) } diff --git a/server/v2/tests/post_handler_test.go b/server/v2/tests/post_handler_test.go index 655278f2224..856633ef018 100644 --- a/server/v2/tests/post_handler_test.go +++ b/server/v2/tests/post_handler_test.go @@ -23,7 +23,7 @@ func TestV2CreateUnique(t *testing.T) { assert.Equal(t, body["action"], "create", "") assert.Equal(t, body["key"], "/foo/bar/1", "") assert.Equal(t, body["dir"], true, "") - assert.Equal(t, body["index"], 1, "") + assert.Equal(t, body["modifiedIndex"], 1, "") // Second POST should add next index to list. resp, _ = tests.PostForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), nil) diff --git a/server/v2/tests/put_handler_test.go b/server/v2/tests/put_handler_test.go index 73f6be91c41..3ee64260415 100644 --- a/server/v2/tests/put_handler_test.go +++ b/server/v2/tests/put_handler_test.go @@ -22,7 +22,7 @@ func TestV2SetKey(t *testing.T) { resp, err := tests.PutForm(fmt.Sprintf("http://%s%s", s.URL(), "/v2/keys/foo/bar"), v) body := tests.ReadBody(resp) assert.Nil(t, err, "") - assert.Equal(t, string(body), `{"action":"set","key":"/foo/bar","value":"XXX","index":1}`, "") + assert.Equal(t, string(body), `{"action":"set","key":"/foo/bar","value":"XXX","modifiedIndex":1}`, "") }) } @@ -175,7 +175,7 @@ func TestV2SetKeyCASOnIndexSuccess(t *testing.T) { assert.Equal(t, body["action"], "compareAndSwap", "") assert.Equal(t, body["prevValue"], "XXX", "") assert.Equal(t, body["value"], "YYY", "") - assert.Equal(t, body["index"], 2, "") + assert.Equal(t, body["modifiedIndex"], 2, "") }) } @@ -236,7 +236,7 @@ func TestV2SetKeyCASOnValueSuccess(t *testing.T) { assert.Equal(t, body["action"], "compareAndSwap", "") assert.Equal(t, body["prevValue"], "XXX", "") assert.Equal(t, body["value"], "YYY", "") - assert.Equal(t, body["index"], 2, "") + assert.Equal(t, body["modifiedIndex"], 2, "") }) } diff --git a/store/event.go b/store/event.go index 50d2872daec..126ceee4bad 100644 --- a/store/event.go +++ b/store/event.go @@ -15,23 +15,23 @@ const ( ) type Event struct { - Action string `json:"action"` - Key string `json:"key, omitempty"` - Dir bool `json:"dir,omitempty"` - PrevValue string `json:"prevValue,omitempty"` - Value string `json:"value,omitempty"` - KVPairs kvPairs `json:"kvs,omitempty"` - Expiration *time.Time `json:"expiration,omitempty"` - TTL int64 `json:"ttl,omitempty"` // Time to live in second - // The index of the etcd state machine when the comment is executed - Index uint64 `json:"index"` + Action string `json:"action"` + + Key string `json:"key, omitempty"` + Dir bool `json:"dir,omitempty"` + PrevValue string `json:"prevValue,omitempty"` + Value string `json:"value,omitempty"` + KVPairs kvPairs `json:"kvs,omitempty"` + Expiration *time.Time `json:"expiration,omitempty"` + TTL int64 `json:"ttl,omitempty"` // Time to live in second + ModifiedIndex uint64 `json:"modifiedIndex"` } func newEvent(action string, key string, index uint64) *Event { return &Event{ - Action: action, - Key: key, - Index: index, + Action: action, + Key: key, + ModifiedIndex: index, } } @@ -47,6 +47,10 @@ func (e *Event) IsCreated() bool { return false } +func (e *Event) Index() uint64 { + return e.ModifiedIndex +} + // Converts an event object into a response object. func (event *Event) Response() interface{} { if !event.Dir { @@ -55,7 +59,7 @@ func (event *Event) Response() interface{} { Key: event.Key, Value: event.Value, PrevValue: event.PrevValue, - Index: event.Index, + Index: event.ModifiedIndex, TTL: event.TTL, Expiration: event.Expiration, } @@ -80,7 +84,7 @@ func (event *Event) Response() interface{} { Key: kv.Key, Value: kv.Value, Dir: kv.Dir, - Index: event.Index, + Index: event.ModifiedIndex, } } return responses diff --git a/store/event_history.go b/store/event_history.go index aaa93d44e0d..4fd077184ff 100644 --- a/store/event_history.go +++ b/store/event_history.go @@ -31,9 +31,9 @@ func (eh *EventHistory) addEvent(e *Event) *Event { eh.Queue.insert(e) - eh.LastIndex = e.Index + eh.LastIndex = e.Index() - eh.StartIndex = eh.Queue.Events[eh.Queue.Front].Index + eh.StartIndex = eh.Queue.Events[eh.Queue.Front].ModifiedIndex return e } @@ -62,7 +62,7 @@ func (eh *EventHistory) scan(prefix string, index uint64) (*Event, *etcdErr.Erro for { e := eh.Queue.Events[i] - if strings.HasPrefix(e.Key, prefix) && index <= e.Index { // make sure we bypass the smaller one + if strings.HasPrefix(e.Key, prefix) && index <= e.Index() { // make sure we bypass the smaller one return e, nil } diff --git a/store/event_test.go b/store/event_test.go index 8a70d09a2d9..dc30ce44dc2 100644 --- a/store/event_test.go +++ b/store/event_test.go @@ -23,7 +23,7 @@ func TestEventQueue(t *testing.T) { n := eh.Queue.Size for ; n > 0; n-- { e := eh.Queue.Events[i] - if e.Index != uint64(j) { + if e.Index() != uint64(j) { t.Fatalf("queue error!") } j++ @@ -42,19 +42,19 @@ func TestScanHistory(t *testing.T) { eh.addEvent(newEvent(Create, "/foo/foo/foo", 5)) e, err := eh.scan("/foo", 1) - if err != nil || e.Index != 1 { + if err != nil || e.Index() != 1 { t.Fatalf("scan error [/foo] [1] %v", e.Index) } e, err = eh.scan("/foo/bar", 1) - if err != nil || e.Index != 2 { + if err != nil || e.Index() != 2 { t.Fatalf("scan error [/foo/bar] [2] %v", e.Index) } e, err = eh.scan("/foo/bar", 3) - if err != nil || e.Index != 4 { + if err != nil || e.Index() != 4 { t.Fatalf("scan error [/foo/bar/bar] [4] %v", e.Index) } diff --git a/store/node.go b/store/node.go index 9a7196bd3f7..a7fd7853c8e 100644 --- a/store/node.go +++ b/store/node.go @@ -51,13 +51,14 @@ func newDir(store *store, nodePath string, createIndex uint64, parent *Node, ACL string, expireTime time.Time) *Node { return &Node{ - Path: nodePath, - CreateIndex: createIndex, - Parent: parent, - ACL: ACL, - ExpireTime: expireTime, - Children: make(map[string]*Node), - store: store, + Path: nodePath, + CreateIndex: createIndex, + ModifiedIndex: createIndex, + Parent: parent, + ACL: ACL, + ExpireTime: expireTime, + Children: make(map[string]*Node), + store: store, } } diff --git a/store/store.go b/store/store.go index 04bacb7115f..ff78320fe0f 100644 --- a/store/store.go +++ b/store/store.go @@ -220,6 +220,8 @@ func (s *store) Delete(nodePath string, recursive bool) (*Event, error) { s.worldLock.Lock() defer s.worldLock.Unlock() + nextIndex := s.CurrentIndex + 1 + n, err := s.internalGet(nodePath) if err != nil { // if the node does not exist, return error @@ -227,7 +229,7 @@ func (s *store) Delete(nodePath string, recursive bool) (*Event, error) { return nil, err } - e := newEvent(Delete, nodePath, s.CurrentIndex) + e := newEvent(Delete, nodePath, nextIndex) if n.IsDir() { e.Dir = true @@ -249,7 +251,6 @@ func (s *store) Delete(nodePath string, recursive bool) (*Event, error) { // update etcd index s.CurrentIndex++ - e.Index = s.CurrentIndex s.WatcherHub.notify(e) s.Stats.Inc(DeleteSuccess) @@ -488,7 +489,7 @@ func (s *store) checkDir(parent *Node, dirName string) (*Node, *etcdErr.Error) { return nil, etcdErr.NewError(etcdErr.EcodeNotDir, parent.Path, s.CurrentIndex) } - n := newDir(s, path.Join(parent.Path, dirName), s.CurrentIndex, parent, parent.ACL, Permanent) + n := newDir(s, path.Join(parent.Path, dirName), s.CurrentIndex+1, parent, parent.ACL, Permanent) parent.Children[dirName] = n diff --git a/store/store_test.go b/store/store_test.go index 898d510ab26..5ee58649d6f 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -81,7 +81,7 @@ func TestStoreCreateValue(t *testing.T) { assert.Nil(t, e.KVPairs, "") assert.Nil(t, e.Expiration, "") assert.Equal(t, e.TTL, 0, "") - assert.Equal(t, e.Index, uint64(1), "") + assert.Equal(t, e.ModifiedIndex, uint64(1), "") } // Ensure that the store can create a new directory if it doesn't already exist. @@ -119,7 +119,7 @@ func TestStoreUpdateValue(t *testing.T) { assert.Equal(t, e.PrevValue, "bar", "") assert.Equal(t, e.Value, "baz", "") assert.Equal(t, e.TTL, 0, "") - assert.Equal(t, e.Index, uint64(2), "") + assert.Equal(t, e.ModifiedIndex, uint64(2), "") e, _ = s.Get("/foo", false, false) assert.Equal(t, e.Value, "baz", "") } diff --git a/store/watcher.go b/store/watcher.go index 2015d0072da..87b2e155d71 100644 --- a/store/watcher.go +++ b/store/watcher.go @@ -24,7 +24,7 @@ func (w *watcher) notify(e *Event, originalPath bool, deleted bool) bool { // at the file we need to delete. // For example a watcher is watching at "/foo/bar". And we deletes "/foo". The watcher // should get notified even if "/foo" is not the path it is watching. - if (w.recursive || originalPath || deleted) && e.Index >= w.sinceIndex { + if (w.recursive || originalPath || deleted) && e.Index() >= w.sinceIndex { w.eventChan <- e return true } From cb4b6f1fe478b35b0bdfbb05d37344d6db137e46 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 9 Nov 2013 20:52:06 -0800 Subject: [PATCH 236/247] feat add modifiedIndex in kvpair --- store/kv_pairs.go | 13 +++++++------ store/node.go | 10 ++++++---- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/store/kv_pairs.go b/store/kv_pairs.go index f2496d36335..be6c01fb422 100644 --- a/store/kv_pairs.go +++ b/store/kv_pairs.go @@ -6,12 +6,13 @@ import ( // When user list a directory, we add all the node into key-value pair slice type KeyValuePair struct { - Key string `json:"key, omitempty"` - Value string `json:"value,omitempty"` - Dir bool `json:"dir,omitempty"` - Expiration *time.Time `json:"expiration,omitempty"` - TTL int64 `json:"ttl,omitempty"` // Time to live in second - KVPairs kvPairs `json:"kvs,omitempty"` + Key string `json:"key, omitempty"` + Value string `json:"value,omitempty"` + Dir bool `json:"dir,omitempty"` + Expiration *time.Time `json:"expiration,omitempty"` + TTL int64 `json:"ttl,omitempty"` // Time to live in second + KVPairs kvPairs `json:"kvs,omitempty"` + ModifiedIndex uint64 `json:"modifiedIndex,omitempty"` } type kvPairs []KeyValuePair diff --git a/store/node.go b/store/node.go index a7fd7853c8e..a4968e1a04a 100644 --- a/store/node.go +++ b/store/node.go @@ -226,8 +226,9 @@ func (n *Node) Remove(recursive bool, callback func(path string)) *etcdErr.Error func (n *Node) Pair(recurisive, sorted bool) KeyValuePair { if n.IsDir() { pair := KeyValuePair{ - Key: n.Path, - Dir: true, + Key: n.Path, + Dir: true, + ModifiedIndex: n.ModifiedIndex, } pair.Expiration, pair.TTL = n.ExpirationAndTTL() @@ -263,8 +264,9 @@ func (n *Node) Pair(recurisive, sorted bool) KeyValuePair { } pair := KeyValuePair{ - Key: n.Path, - Value: n.Value, + Key: n.Path, + Value: n.Value, + ModifiedIndex: n.ModifiedIndex, } pair.Expiration, pair.TTL = n.ExpirationAndTTL() return pair From 27157e5e784616197dd966e3e5aefa8e9b38f45b Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 9 Nov 2013 21:17:32 -0800 Subject: [PATCH 237/247] fix tests --- .../multi_node_kill_all_and_recovery_test.go | 5 ++--- tests/functional/simple_snapshot_test.go | 13 +++++++++---- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/tests/functional/multi_node_kill_all_and_recovery_test.go b/tests/functional/multi_node_kill_all_and_recovery_test.go index 454b9648c21..4f3c0e7953d 100644 --- a/tests/functional/multi_node_kill_all_and_recovery_test.go +++ b/tests/functional/multi_node_kill_all_and_recovery_test.go @@ -65,8 +65,7 @@ func TestMultiNodeKillAllAndRecovery(t *testing.T) { t.Fatalf("Recovery error: %s", err) } - if result.Index != 18 { - t.Fatalf("recovery failed! [%d/18]", result.Index) + if result.Index != 16 { + t.Fatalf("recovery failed! [%d/16]", result.Index) } } - diff --git a/tests/functional/simple_snapshot_test.go b/tests/functional/simple_snapshot_test.go index e7cce08cc04..e6f8d7729d5 100644 --- a/tests/functional/simple_snapshot_test.go +++ b/tests/functional/simple_snapshot_test.go @@ -3,6 +3,7 @@ package test import ( "io/ioutil" "os" + "strconv" "testing" "time" @@ -52,8 +53,10 @@ func TestSimpleSnapshot(t *testing.T) { t.Fatal("wrong number of snapshot :[1/", len(snapshots), "]") } - if snapshots[0].Name() != "0_503.ss" { - t.Fatal("wrong name of snapshot :[0_503.ss/", snapshots[0].Name(), "]") + index, _ := strconv.Atoi(snapshots[0].Name()[2:5]) + + if index < 507 || index > 510 { + t.Fatal("wrong name of snapshot :", snapshots[0].Name()) } // issue second 501 commands @@ -82,7 +85,9 @@ func TestSimpleSnapshot(t *testing.T) { t.Fatal("wrong number of snapshot :[1/", len(snapshots), "]") } - if snapshots[0].Name() != "0_1004.ss" { - t.Fatal("wrong name of snapshot :[0_1004.ss/", snapshots[0].Name(), "]") + index, _ = strconv.Atoi(snapshots[0].Name()[2:6]) + + if index < 1015 || index > 1018 { + t.Fatal("wrong name of snapshot :", snapshots[0].Name()) } } From e427c85f030cbd9054bc5a3c687fc4d5fd0dc64e Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Sat, 9 Nov 2013 21:31:17 -0800 Subject: [PATCH 238/247] refactor add debug info to remove_node test --- tests/functional/remove_node_test.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/functional/remove_node_test.go b/tests/functional/remove_node_test.go index 09fa747aae8..a9490cdcc62 100644 --- a/tests/functional/remove_node_test.go +++ b/tests/functional/remove_node_test.go @@ -1,6 +1,7 @@ package test import ( + "fmt" "net/http" "os" "testing" @@ -31,6 +32,7 @@ func TestRemoveNode(t *testing.T) { for i := 0; i < 2; i++ { client.Do(rmReq) + fmt.Println("send remove to node3 and wait for its exiting") etcds[2].Wait() resp, err := c.Get("_etcd/machines") @@ -71,6 +73,7 @@ func TestRemoveNode(t *testing.T) { // first kill the node, then remove it, then add it back for i := 0; i < 2; i++ { etcds[2].Kill() + fmt.Println("kill node3 and wait for its exiting") etcds[2].Wait() client.Do(rmReq) From 8b2e1025efecadd2bebbe1c9d8949a5a2411652f Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 11 Nov 2013 21:19:30 -0800 Subject: [PATCH 239/247] style remove the extra space --- store/event.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/store/event.go b/store/event.go index 126ceee4bad..f3d607e0b52 100644 --- a/store/event.go +++ b/store/event.go @@ -15,8 +15,7 @@ const ( ) type Event struct { - Action string `json:"action"` - + Action string `json:"action"` Key string `json:"key, omitempty"` Dir bool `json:"dir,omitempty"` PrevValue string `json:"prevValue,omitempty"` From fe5fb6cfbabf5ab2a082d00769be9de9331ee2e0 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 11 Nov 2013 21:21:50 -0800 Subject: [PATCH 240/247] style naming the initialization fields for sync command --- store/v2/command_factory.go | 4 +++- store/v2/sync_command.go | 1 - 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/store/v2/command_factory.go b/store/v2/command_factory.go index 8332891d66d..77263b8d75b 100644 --- a/store/v2/command_factory.go +++ b/store/v2/command_factory.go @@ -73,5 +73,7 @@ func (f *CommandFactory) CreateCompareAndSwapCommand(key string, value string, p } func (f *CommandFactory) CreateSyncCommand(now time.Time) raft.Command { - return &SyncCommand{time.Now()} + return &SyncCommand{ + Time: time.Now(), + } } diff --git a/store/v2/sync_command.go b/store/v2/sync_command.go index 4b645907929..caf2c37ccf7 100644 --- a/store/v2/sync_command.go +++ b/store/v2/sync_command.go @@ -21,7 +21,6 @@ func (c SyncCommand) CommandName() string { } func (c SyncCommand) Apply(server raft.Server) (interface{}, error) { - s, _ := server.StateMachine().(store.Store) s.DeleteExpiredKeys(c.Time) From 811c577fe8a117d0f7c9390edc86de785c76d128 Mon Sep 17 00:00:00 2001 From: Xiang Li Date: Mon, 11 Nov 2013 21:31:12 -0800 Subject: [PATCH 241/247] test stop mockSync goroutines --- store/stats_test.go | 8 ++++++- store/store_test.go | 51 ++++++++++++++++++++++++++++++++++++++------- 2 files changed, 50 insertions(+), 9 deletions(-) diff --git a/store/stats_test.go b/store/stats_test.go index ad16261b85c..cc5a4afb56b 100644 --- a/store/stats_test.go +++ b/store/stats_test.go @@ -87,7 +87,13 @@ func TestStoreStatsDeleteFail(t *testing.T) { //Ensure that the number of expirations is recorded in the stats. func TestStoreStatsExpireCount(t *testing.T) { s := newStore() - go mockSyncService(s.DeleteExpiredKeys) + + c := make(chan bool) + defer func() { + c <- true + }() + + go mockSyncService(s.DeleteExpiredKeys, c) s.Create("/foo", "bar", false, time.Now().Add(500*time.Millisecond)) assert.Equal(t, uint64(0), s.Stats.ExpireCount, "") time.Sleep(600 * time.Millisecond) diff --git a/store/store_test.go b/store/store_test.go index 5ee58649d6f..01b8bb7bbad 100644 --- a/store/store_test.go +++ b/store/store_test.go @@ -139,7 +139,13 @@ func TestStoreUpdateFailsIfDirectory(t *testing.T) { // Ensure that the store can update the TTL on a value. func TestStoreUpdateValueTTL(t *testing.T) { s := newStore() - go mockSyncService(s.DeleteExpiredKeys) + + c := make(chan bool) + defer func() { + c <- true + }() + go mockSyncService(s.DeleteExpiredKeys, c) + s.Create("/foo", "bar", false, Permanent) _, err := s.Update("/foo", "baz", time.Now().Add(500*time.Millisecond)) e, _ := s.Get("/foo", false, false) @@ -154,7 +160,13 @@ func TestStoreUpdateValueTTL(t *testing.T) { // Ensure that the store can update the TTL on a directory. func TestStoreUpdateDirTTL(t *testing.T) { s := newStore() - go mockSyncService(s.DeleteExpiredKeys) + + c := make(chan bool) + defer func() { + c <- true + }() + go mockSyncService(s.DeleteExpiredKeys, c) + s.Create("/foo", "", false, Permanent) s.Create("/foo/bar", "baz", false, Permanent) _, err := s.Update("/foo", "", time.Now().Add(500*time.Millisecond)) @@ -339,7 +351,13 @@ func TestStoreWatchRecursiveCompareAndSwap(t *testing.T) { // Ensure that the store can watch for key expiration. func TestStoreWatchExpire(t *testing.T) { s := newStore() - go mockSyncService(s.DeleteExpiredKeys) + + stopChan := make(chan bool) + defer func() { + stopChan <- true + }() + go mockSyncService(s.DeleteExpiredKeys, stopChan) + s.Create("/foo", "bar", false, time.Now().Add(500*time.Millisecond)) s.Create("/foofoo", "barbarbar", false, time.Now().Add(500*time.Millisecond)) @@ -379,7 +397,13 @@ func TestStoreRecover(t *testing.T) { // Ensure that the store can recover from a previously saved state that includes an expiring key. func TestStoreRecoverWithExpiration(t *testing.T) { s := newStore() - go mockSyncService(s.DeleteExpiredKeys) + + c := make(chan bool) + defer func() { + c <- true + }() + go mockSyncService(s.DeleteExpiredKeys, c) + s.Create("/foo", "", false, Permanent) s.Create("/foo/x", "bar", false, Permanent) s.Create("/foo/y", "baz", false, time.Now().Add(5*time.Millisecond)) @@ -388,7 +412,13 @@ func TestStoreRecoverWithExpiration(t *testing.T) { time.Sleep(10 * time.Millisecond) s2 := newStore() - go mockSyncService(s2.DeleteExpiredKeys) + + c2 := make(chan bool) + defer func() { + c2 <- true + }() + go mockSyncService(s2.DeleteExpiredKeys, c2) + s2.Recovery(b) time.Sleep(600 * time.Millisecond) @@ -412,9 +442,14 @@ func nbselect(c <-chan *Event) *Event { } } -func mockSyncService(f func(now time.Time)) { +func mockSyncService(f func(now time.Time), c chan bool) { ticker := time.Tick(time.Millisecond * 500) - for now := range ticker { - f(now) + for { + select { + case <-c: + return + case now := <-ticker: + f(now) + } } } From 954217fb739ad5ef9f5c5e00047bafc20288ca4b Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Tue, 12 Nov 2013 19:22:09 -0500 Subject: [PATCH 242/247] Fix v1 cluster migration test. --- server/v1/delete_key_handler.go | 2 +- server/v1/get_key_handler.go | 2 +- server/v1/set_key_handler.go | 2 +- server/v1/watch_key_handler.go | 2 +- server/v2/delete_handler.go | 2 +- server/v2/get_handler.go | 2 +- server/v2/post_handler.go | 2 +- server/v2/put_handler.go | 2 +- tests/functional/v1_migration_test.go | 12 ++++++------ 9 files changed, 14 insertions(+), 14 deletions(-) diff --git a/server/v1/delete_key_handler.go b/server/v1/delete_key_handler.go index 561196b9599..44fe459e494 100644 --- a/server/v1/delete_key_handler.go +++ b/server/v1/delete_key_handler.go @@ -8,7 +8,7 @@ import ( // Removes a key from the store. func DeleteKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) - key := vars["key"] + key := "/" + vars["key"] c := s.Store().CommandFactory().CreateDeleteCommand(key, false) return s.Dispatch(c, w, req) } diff --git a/server/v1/get_key_handler.go b/server/v1/get_key_handler.go index 8664a56ae96..880bf289e36 100644 --- a/server/v1/get_key_handler.go +++ b/server/v1/get_key_handler.go @@ -10,7 +10,7 @@ import ( // Retrieves the value for a given key. func GetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) - key := vars["key"] + key := "/" + vars["key"] // Retrieve the key from the store. event, err := s.Store().Get(key, false, false) diff --git a/server/v1/set_key_handler.go b/server/v1/set_key_handler.go index b3ec7d68bd4..b1b4390a248 100644 --- a/server/v1/set_key_handler.go +++ b/server/v1/set_key_handler.go @@ -12,7 +12,7 @@ import ( // Sets the value for a given key. func SetKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) - key := vars["key"] + key := "/" + vars["key"] req.ParseForm() diff --git a/server/v1/watch_key_handler.go b/server/v1/watch_key_handler.go index 5943924ccfe..de5ed065620 100644 --- a/server/v1/watch_key_handler.go +++ b/server/v1/watch_key_handler.go @@ -13,7 +13,7 @@ import ( func WatchKeyHandler(w http.ResponseWriter, req *http.Request, s Server) error { var err error vars := mux.Vars(req) - key := vars["key"] + key := "/" + vars["key"] // Create a command to watch from a given index (default 0). var sinceIndex uint64 = 0 diff --git a/server/v2/delete_handler.go b/server/v2/delete_handler.go index 17030c75715..7afc02f0412 100644 --- a/server/v2/delete_handler.go +++ b/server/v2/delete_handler.go @@ -8,7 +8,7 @@ import ( func DeleteHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) - key := vars["key"] + key := "/" + vars["key"] recursive := (req.FormValue("recursive") == "true") c := s.Store().CommandFactory().CreateDeleteCommand(key, recursive) diff --git a/server/v2/get_handler.go b/server/v2/get_handler.go index 26128fc95ec..3e8ddee6426 100644 --- a/server/v2/get_handler.go +++ b/server/v2/get_handler.go @@ -18,7 +18,7 @@ func GetHandler(w http.ResponseWriter, req *http.Request, s Server) error { var event *store.Event vars := mux.Vars(req) - key := vars["key"] + key := "/" + vars["key"] // Help client to redirect the request to the current leader if req.FormValue("consistent") == "true" && s.State() != raft.Leader { diff --git a/server/v2/post_handler.go b/server/v2/post_handler.go index 59c11947690..2338d0250e2 100644 --- a/server/v2/post_handler.go +++ b/server/v2/post_handler.go @@ -10,7 +10,7 @@ import ( func PostHandler(w http.ResponseWriter, req *http.Request, s Server) error { vars := mux.Vars(req) - key := vars["key"] + key := "/" + vars["key"] value := req.FormValue("value") expireTime, err := store.TTL(req.FormValue("ttl")) diff --git a/server/v2/put_handler.go b/server/v2/put_handler.go index 19a55de050f..5b61e58b39c 100644 --- a/server/v2/put_handler.go +++ b/server/v2/put_handler.go @@ -15,7 +15,7 @@ func PutHandler(w http.ResponseWriter, req *http.Request, s Server) error { var c raft.Command vars := mux.Vars(req) - key := vars["key"] + key := "/" + vars["key"] req.ParseForm() diff --git a/tests/functional/v1_migration_test.go b/tests/functional/v1_migration_test.go index c99f7711a0e..415ff6dedd5 100644 --- a/tests/functional/v1_migration_test.go +++ b/tests/functional/v1_migration_test.go @@ -87,17 +87,17 @@ func TestV1ClusterMigration(t *testing.T) { time.Sleep(time.Second) } - time.Sleep(120 * time.Second) - // Ensure deleted message is removed. resp, err := tests.Get("http://localhost:4001/v2/keys/message") - tests.ReadBody(resp) + body := tests.ReadBody(resp) assert.Nil(t, err, "") - assert.Equal(t, resp.StatusCode, 404, "") + assert.Equal(t, resp.StatusCode, 400, ) + assert.Equal(t, string(body), `{"errorCode":100,"message":"Key Not Found","cause":"/message","index":11}`+"\n") // Ensure TTL'd message is removed. resp, err = tests.Get("http://localhost:4001/v2/keys/foo") - tests.ReadBody(resp) + body = tests.ReadBody(resp) assert.Nil(t, err, "") - assert.Equal(t, resp.StatusCode, 404, "") + assert.Equal(t, resp.StatusCode, 200, "") + assert.Equal(t, string(body), `{"action":"get","key":"/foo","value":"one","modifiedIndex":9}`) } From df00306a7192ba4d057af17d5d9a846fa6696703 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Tue, 12 Nov 2013 21:28:51 -0500 Subject: [PATCH 243/247] README --- README.md | 271 +++++++++++++++++++++++++++++++++--------------------- 1 file changed, 164 insertions(+), 107 deletions(-) diff --git a/README.md b/README.md index 4f8b521856c..4ebeaac5137 100644 --- a/README.md +++ b/README.md @@ -1,22 +1,27 @@ # etcd + README version 0.2.0 [![Build Status](https://travis-ci.org/coreos/etcd.png)](https://travis-ci.org/coreos/etcd) -A highly-available key value store for shared configuration and service discovery. etcd is inspired by zookeeper and doozer, with a focus on: +A highly-available key value store for shared configuration and service discovery. +etcd is inspired by zookeeper and doozer, with a focus on: * Simple: curl'able user facing API (HTTP+JSON) * Secure: optional SSL client cert authentication * Fast: benchmarked 1000s of writes/s per instance * Reliable: Properly distributed using Raft -Etcd is written in Go and uses the [raft][raft] consensus algorithm to manage a highly-available replicated log. +Etcd is written in Go and uses the [Raft][raft] consensus algorithm to manage a highly-available replicated log. -See [etcdctl][etcdctl] for a simple command line client. Or feel free to just use curl, as in the examples below. +See [etcdctl][etcdctl] for a simple command line client. +Or feel free to just use curl, as in the examples below. [raft]: https://github.com/coreos/go-raft [etcdctl]: http://coreos.com/docs/etcdctl/ + + ## Getting Started ### Getting etcd @@ -25,6 +30,7 @@ The latest release is available as a binary at [Github][github-release]. [github-release]: https://github.com/coreos/etcd/releases/ + ### Building You can build etcd from source: @@ -43,9 +49,11 @@ _NOTE_: you need go 1.1+. Please check your installation with go version ``` + ### Running a single node -These examples will use a single node cluster to show you the basics of the etcd REST API. Lets start etcd: +These examples will use a single node cluster to show you the basics of the etcd REST API. +Let's start etcd: ```sh ./etcd -d node0 -n node0 @@ -55,86 +63,104 @@ This will bring up an etcd node listening on port 4001 for client communication The `-d node0` argument tells etcd to write node configuration, logs and snapshots to the `./node0/` directory. The `-n node0` tells the rest of the cluster that this node is named node0. + + ## Usage ### Setting the value to a key -Let’s set the first key-value pair to the node. In this case the key is `/message` and the value is `Hello world`. +Let’s set the first key-value pair to the node. +In this case the key is `/message` and the value is `Hello world`. ```sh -curl -L http://127.0.0.1:4001/v2/keys/message -XPUT -d value="Hello world" +curl -L http://127.0.0.1:4001/v2/keys/message -X PUT -d value="Hello world" ``` ```json -{"action":"set","key":"/message","value":"Hello world","index":3,"term":0} +{"action":"set","key":"/message","value":"Hello world","modifiedIndex":2} ``` -This response contains five fields. We will introduce three more fields as we try more commands. +This response contains four fields. +We will introduce three more fields as we try more commands. -1. The action of the request; we set the value via a PUT request, thus the action is `set`. +1. The action of the request; we set the value via a `PUT` request, thus the action is `set`. -2. The key of the request; we set `/message` to `Hello world!`, so the key field is `/message`. -Notice we use a file system like structure to represent the key-value pairs. So each key starts with `/`. +2. The key of the request; we set `/message` to `Hello world`, so the key field is `/message`. +We use a file system like structure to represent the key-value pairs so each key starts with `/`. 3. The current value of the key; we set the value to`Hello world`. -4. Index is the unique internal log index of the set request. Requests that change the log index include `set`, `delete`, `update`, `create` and `compareAndSwap`. The `get` and `watch` commands do not change state in the store and so they do not change the index. You may notice that in this example the index is 3, although it is the first request you sent to the server. This is because there are internal commands that also change the state like adding and syncing servers. +4. Modified Index is a unique, monotonically incrementing index created for each change to etcd. +Requests that change the index include `set`, `delete`, `update`, `create` and `compareAndSwap`. +Since the `get` and `watch` commands do not change state in the store, they do not change the index. +You may notice that in this example the index is `2` even though it is the first request you sent to the server. +This is because there are internal commands that also change the state like adding and syncing servers. + ### Get the value of a key -Get the value that we just set in `/message` by issuing a GET: +We can get the value that we just set in `/message` by issuing a `GET` request: ```sh curl -L http://127.0.0.1:4001/v2/keys/message ``` ```json -{"action":"get","key":"/message","value":"Hello world","index":3,"term":0} +{"action":"get","key":"/message","value":"Hello world","modifiedIndex":2} ``` + + ### Changing the value of a key -Change the value of `/message` from `Hello world` to `Hello etcd` with another PUT request to the key: +You can change the value of `/message` from `Hello world` to `Hello etcd` with another `PUT` request to the key: ```sh curl -L http://127.0.0.1:4001/v1/keys/message -XPUT -d value="Hello etcd" ``` ```json -{"action":"set","key":"/message","prevValue":"Hello world","value":"Hello etcd","index":4,"term":0} +{"action":"set","key":"/message","prevValue":"Hello world","value":"Hello etcd","index":3} ``` -Notice that the `prevValue` is set to the previous value of the key - `Hello world`. It is useful when you want to atomically set a value to a key and get its old value. +Notice that the `prevValue` is set to the previous value of the key - `Hello world`. +It is useful when you want to atomically set a value to a key and get its old value. + + ### Deleting a key -Remove the `/message` key with a DELETE: +You can remove the `/message` key with a `DELETE` request: ```sh curl -L http://127.0.0.1:4001/v2/keys/message -XDELETE ``` ```json -{"action":"delete","key":"/message","prevValue":"Hello etcd","index":5,"term":0} +{"action":"delete","key":"/message","prevValue":"Hello etcd","modifiedIndex":4} ``` + ### Using key TTL -Keys in etcd can be set to expire after a specified number of seconds. That is done by setting a TTL (time to live) on the key when you POST: +Keys in etcd can be set to expire after a specified number of seconds. +You can do this by setting a TTL (time to live) on the key when send a `PUT` request: ```sh curl -L http://127.0.0.1:4001/v2/keys/foo -XPUT -d value=bar -d ttl=5 ``` ```json -{"action":"set","key":"/foo","value":"bar","expiration":"2013-10-19T18:44:04.528757176-07:00","ttl":5,"index":6,"term":0} +{"action":"set","key":"/foo","value":"bar","expiration":"2013-11-12T20:21:22.629352334-05:00","ttl":5,"modifiedIndex":5} ``` -Note the last two new fields in response: +Note the two new fields in response: + +1. The `expiration` is the time that this key will expire and be deleted. -1. The expiration is the time that this key will expire and be deleted. +2. The `ttl` is the time to live for the key, in seconds. -2. The ttl is the time to live of the key. +_NOTE_: Keys can only be expired by a cluster leader so if a node gets disconnected from the cluster, its keys will not expire until it rejoins. -Now you can try to get the key by sending: +Now you can try to get the key by sending a `GET` request: ```sh curl -L http://127.0.0.1:4001/v2/keys/foo @@ -143,12 +169,14 @@ curl -L http://127.0.0.1:4001/v2/keys/foo If the TTL has expired, the key will be deleted, and you will be returned a 100. ```json -{"errorCode":100,"message":"Key Not Found","cause":"/foo","index":6,"term":0} +{"errorCode":100,"message":"Key Not Found","cause":"/foo","index":6} ``` + ### Waiting for a change -We can watch for a change and get a notification at a given path or any keys underneath it. +We can watch for a change on a key and receive a notification by using long polling. +This also works for child keys by passing `recursive=true` in curl. In one terminal, we send a get request with `wait=true` : @@ -156,7 +184,7 @@ In one terminal, we send a get request with `wait=true` : curl -L http://127.0.0.1:4001/v2/keys/foo?wait=true ``` -Now, we are waiting for any changes at path `/foo`. +Now we are waiting for any changes at path `/foo`. In another terminal, we set a key `/foo` with value `bar`: @@ -167,10 +195,12 @@ curl -L http://127.0.0.1:4001/v2/keys/foo -XPUT -d value=bar The first terminal should get the notification and return with the same response as the set request. ```json -{"action":"set","key":"/foo","value":"bar","index":7,"term":0} +{"action":"set","key":"/foo","value":"bar","modifiedIndex":7} ``` -However, the watch command can do more than this. Using the the index we can watch for commands that has happened in the past. This is useful for ensuring you don't miss events between watch commands. +However, the watch command can do more than this. +Using the the index we can watch for commands that has happened in the past. +This is useful for ensuring you don't miss events between watch commands. Let's try to watch for the set command of index 7 again: @@ -180,27 +210,30 @@ curl -L http://127.0.0.1:4001/v2/keys/foo?wait=true\&waitIndex=7 The watch command returns immediately with the same response as previous. -### Atomic Compare and Swap + +### Atomic Compare-and-Swap (CAS) Etcd can be used as a centralized coordination service in a cluster and `CompareAndSwap` is the most basic operation to build distributed lock service. -This command will set the value to the key only if the client provided conditions are equal to the current conditions. +This command will set the value of a key only if the client-provided conditions are equal to the current conditions. The current comparable conditions are: -1. `prevValue` previous value of the key: -2. `prevIndex` previous index of the key +1. `prevValue` - checks the previous value of the key. + +2. `prevIndex` - checks the previous index of the key. -3. `prevExist` previous existence of the key: if `prevExist` is true, it is a `update` request; if prevExist is `false`, it is a `create` request. +3. `prevExist` - checks existence of the key: if `prevExist` is true, it is a `update` request; if prevExist is `false`, it is a `create` request. -Here is a simple example. Let's create a key-value pair first: `foo=one`. +Here is a simple example. +Let's create a key-value pair first: `foo=one`. ```sh curl -L http://127.0.0.1:4001/v1/keys/foo -XPUT -d value=one ``` -Let's try an invalid `CompareAndSwap` command. -We can give another parameter prevValue to set command to make it a `CompareAndSwap` command. +Let's try an invalid `CompareAndSwap` command first. +We can provide the `prevValue` parameter to the set command to make it a `CompareAndSwap` command. ```sh curl -L http://127.0.0.1:4001/v1/keys/foo?prevValue=two -XPUT -d value=three @@ -209,12 +242,12 @@ curl -L http://127.0.0.1:4001/v1/keys/foo?prevValue=two -XPUT -d value=three This will try to compare the previous value of the key and the previous value we provided. If they are equal, the value of the key will change to three. ```json -{"errorCode":101,"message":"Test Failed","cause":"[two != one] [0 != 8]","index":9,"term":0} +{"errorCode":101,"message":"Test Failed","cause":"[two != one] [0 != 8]","index":8} ``` -which means `compareAndSwap` failed. +which means `CompareAndSwap` failed. -Let us try a valid one. +Let's try a valid condition: ```sh curl -L http://127.0.0.1:4001/v2/keys/foo?prevValue=one -XPUT -d value=two @@ -223,53 +256,57 @@ curl -L http://127.0.0.1:4001/v2/keys/foo?prevValue=one -XPUT -d value=two The response should be ```json -{"action":"compareAndSwap","key":"/foo","prevValue":"one","value":"two","index":10,"term":0} +{"action":"compareAndSwap","key":"/foo","prevValue":"one","value":"two","modifiedIndex":9} ``` -We successfully changed the value from “one” to “two”, since we give the correct previous value. +We successfully changed the value from “one” to “two” since we gave the correct previous value. + ### Listing a directory -Let us create some keys first. +In etcd we can store two types of things: keys and directories. +Keys store a single string value. +Directories store a set of keys and/or other directories. -We already have `/foo=two` +In this example, let's first create some keys: -We create another one `/foo_dir/foo=bar` +We already have `/foo=two` so now we'll create another one called `/foo_dir/foo` with the value of `bar`: ```sh curl -L http://127.0.0.1:4001/v2/keys/foo_dir/foo -XPUT -d value=bar ``` ```json -{"action":"set","key":"/foo_dir/foo","value":"bar","index":11,"term":0} +{"action":"set","key":"/foo_dir/foo","value":"bar","modifiedIndex":10} ``` -Now list the keys under root `/` +Now we can list the keys under root `/`: ```sh curl -L http://127.0.0.1:4001/v2/keys/ ``` -We should see the response as an array of items +We should see the response as an array of items: ```json -{"action":"get","key":"/","dir":true,"kvs":[{"key":"/foo","value":"two"},{"key":"/foo_dir","dir":true}],"index":11,"term":0} +{"action":"get","key":"/","dir":true,"kvs":[{"key":"/foo","value":"two","modifiedIndex":9},{"key":"/foo_dir","dir":true,"modifiedIndex":10}],"modifiedIndex":0} ``` -which meas `/foo=two` is a key-value pair under `/ and `/foo_dir` is a directory. - -Also we can recursively get all the content under a directory by add `recursive=true`. +Here we can see `/foo` is a key-value pair under `/` and `/foo_dir` is a directory. +We can also recursively get all the contents under a directory by adding `recursive=true`. ```sh curl -L http://127.0.0.1:4001/v2/keys/?recursive=true ``` ```json -{"action":"get","key":"/","dir":true,"kvs":[{"key":"/foo","value":"two"},{"key":"/foo_dir","dir":true,"kvs":[{"key":"/foo_dir/foo","value":"bar"}]}],"index":11,"term":0} +{"action":"get","key":"/","dir":true,"kvs":[{"key":"/foo","value":"two","modifiedIndex":9},{"key":"/foo_dir","dir":true,"kvs":[{"key":"/foo_dir/foo","value":"bar","modifiedIndex":10}],"modifiedIndex":10}],"modifiedIndex":0} ``` + ### Deleting a directory -Let try to delete the directory `/foo_dir`. + +Now let's try to delete the directory `/foo_dir`. To delete a directory, we must add `recursive=true`. @@ -278,61 +315,73 @@ curl -L http://127.0.0.1:4001/v2/keys/foo_dir?recursive=true -XDELETE ``` ```json -{"action":"delete","key":"/foo_dir","dir":true,"index":12,"term":0} +{"action":"delete","key":"/foo_dir","dir":true,"modifiedIndex":11} ``` + ### Creating a hidden node -We can create a hidden key-value pair or directory by add `_` prefix. The hidden item will not be list when using get for a directory. + +We can create a hidden key-value pair or directory by add a `_` prefix. +The hidden item will not be listed when sending a `GET` request for a directory. + +First we'll add a hidden key named `/_message`: ```sh curl -L http://127.0.0.1:4001/v2/keys/_message -XPUT -d value="Hello hidden world" ``` ```json -{"action":"set","key":"/_message","value":"Hello hidden world","index":13,"term":0} +{"action":"set","key":"/_message","value":"Hello hidden world","modifiedIndex":12} ``` + +Next we'll add a regular key named `/message`: + ```sh curl -L http://127.0.0.1:4001/v2/keys/message -XPUT -d value="Hello world" ``` ```json -{"action":"set","key":"/message","value":"Hello world","index":14,"term":0} +{"action":"set","key":"/message","value":"Hello world","modifiedIndex":13} ``` -Let us try to get the root `/` +Now let's try to get a listing of keys under the root directory, `/`: ```sh curl -L http://127.0.0.1:4001/v2/keys/ ``` ```json -{"action":"get","key":"/","dir":true,"kvs":[{"key":"/foo","value":"two"},{"key":"/message","value":"Hello world"}],"index":15,"term":0} +{"action":"get","key":"/","dir":true,"kvs":[{"key":"/foo","value":"two","modifiedIndex":9},{"key":"/message","value":"Hello world","modifiedIndex":13}],"modifiedIndex":0} ``` -We can only get `/message`, but cannot get `/_message`. +Here we see the `/message` key but our hidden `/_message` key is not returned. + ## Advanced Usage ### Transport security with HTTPS -Etcd supports SSL/TLS and client cert authentication for clients to server, as well as server to server communication +Etcd supports SSL/TLS and client cert authentication for clients to server, as well as server to server communication. -First, you need to have a CA cert `clientCA.crt` and signed key pair `client.crt`, `client.key`. This site has a good reference for how to generate self-signed key pairs: +First, you need to have a CA cert `clientCA.crt` and signed key pair `client.crt`, `client.key`. +This site has a good reference for how to generate self-signed key pairs: http://www.g-loaded.eu/2005/11/10/be-your-own-ca/ For testing you can use the certificates in the `fixtures/ca` directory. -Next, lets configure etcd to use this keypair: +Let's configure etcd to use this keypair: ```sh ./etcd -n node0 -d node0 -clientCert=./fixtures/ca/server.crt -clientKey=./fixtures/ca/server.key.insecure -f ``` -`-f` forces new node configuration if existing configuration is found (WARNING: data loss!) -`-clientCert` and `-clientKey` are the key and cert for transport layer security between client and server +There are a few new options we're using: + +* `-f` - forces a new node configuration, even if an existing configuration is found. (WARNING: data loss!) +* `-clientCert` and `-clientKey` specify the location of the cert and key files to be used for for transport layer security between the client and server. -You can now test the configuration using https: +You can now test the configuration using HTTPS: ```sh curl --cacert fixtures/ca/ca.crt https://127.0.0.1:4001/v2/keys/foo -XPUT -d value=bar -v @@ -346,15 +395,17 @@ SSLv3, TLS handshake, Finished (20): ... ``` -And also the response from the etcd server. +And also the response from the etcd server: ```json -{"action":"set","key":"/foo","value":"bar","index":3, "term: 0"} +{"action":"set","key":"/foo","prevValue":"bar","value":"bar","modifiedIndex":3} ``` + ### Authentication with HTTPS client certificates -We can also do authentication using CA certs. The clients will provide their cert to the server and the server will check whether the cert is signed by the CA and decide whether to serve the request. +We can also do authentication using CA certs. +The clients will provide their cert to the server and the server will check whether the cert is signed by the CA and decide whether to serve the request. ```sh ./etcd -n node0 -d node0 -clientCAFile=./fixtures/ca/ca.crt -clientCert=./fixtures/ca/server.crt -clientKey=./fixtures/ca/server.key.insecure -f @@ -382,7 +433,8 @@ We need to give the CA signed cert to the server. curl -L https://127.0.0.1:4001/v1/keys/foo -XPUT -d value=bar -v --key myclient.key --cert myclient.crt -cacert clientCA.crt ``` -You should able to see +You should able to see: + ``` ... SSLv3, TLS handshake, CERT verify (15): @@ -393,14 +445,16 @@ TLS handshake, Finished (20) And also the response from the server: ```json -{"action":"set","key":"/foo","value":"bar","index":3,"term:0"} +{"action":"set","key":"/foo","prevValue":"bar","value":"bar","modifiedIndex":3} ``` + ## Clustering ### Example cluster of three machines -Let's explore the use of etcd clustering. We use go-raft as the underlying distributed protocol which provides consistency and persistence of the data across all of the etcd instances. +Let's explore the use of etcd clustering. +We use Raft as the underlying distributed protocol which provides consistency and persistence of the data across all of the etcd instances. Let start by creating 3 new etcd instances. @@ -410,17 +464,17 @@ We use -s to specify server port and -c to specify client port and -d to specify ./etcd -s 127.0.0.1:7001 -c 127.0.0.1:4001 -d nodes/node1 -n node1 ``` -**Note:** If you want to run etcd on external IP address and still have access locally you need to add `-cl 0.0.0.0` so that it will listen on both external and localhost addresses. +**Note:** If you want to run etcd on an external IP address and still have access locally, you'll need to add `-cl 0.0.0.0` so that it will listen on both external and localhost addresses. A similar argument `-sl` is used to setup the listening address for the server port. -Let the join two more nodes to this cluster using the -C argument: +Let's join two more nodes to this cluster using the `-C` argument: ```sh ./etcd -s 127.0.0.1:7002 -c 127.0.0.1:4002 -C 127.0.0.1:7001 -d nodes/node2 -n node2 ./etcd -s 127.0.0.1:7003 -c 127.0.0.1:4003 -C 127.0.0.1:7001 -d nodes/node3 -n node3 ``` -Get the machines in the cluster: +We can retrieve a list of machines in the cluster using the HTTP API: ```sh curl -L http://127.0.0.1:4001/v1/machines @@ -432,24 +486,23 @@ We should see there are three nodes in the cluster http://127.0.0.1:4001, http://127.0.0.1:4002, http://127.0.0.1:4003 ``` -The machine list is also available via this API: +The machine list is also available via the main key API: ```sh curl -L http://127.0.0.1:4001/v1/keys/_etcd/machines ``` ```json -[{"action":"get","key":"/_etcd/machines/node1","value":"raft=http://127.0.0.1:7001&etcd=http://127.0.0.1:4001&raftVersion=v0.1.1-311-g91cad59","index":4},{"action":"get","key":"/_etcd/machines/node2","value":"raft=http://127.0.0.1:7002&etcd=http://127.0.0.1:4002&raftVersion=v0.1.1-311-g91cad59","index":4},{"action":"get","key":"/_etcd/machines/node3","value":"raft=http://127.0.0.1:7003&etcd=http://127.0.0.1:4003&raftVersion=v0.1.1-311-g91cad59","index":4}] +[{"action":"get","key":"/_etcd/machines/node1","value":"raft=http://127.0.0.1:7001\u0026etcd=http://127.0.0.1:4001","index":1},{"action":"get","key":"/_etcd/machines/node2","value":"raft=http://127.0.0.1:7002\u0026etcd=http://127.0.0.1:4002","index":1},{"action":"get","key":"/_etcd/machines/node3","value":"raft=http://127.0.0.1:7003\u0026etcd=http://127.0.0.1:4003","index":1}] ``` -The key of the machine is based on the ```commit index``` when it was added. The value of the machine is ```hostname```, ```raft port``` and ```client port```. - -Also try to get the current leader in the cluster +We can also get the current leader in the cluster: ``` curl -L http://127.0.0.1:4001/v2/leader ``` -The first server we set up should be the leader, if it has not died during these commands. + +The first server we set up should still be the leader unless it has died during these commands. ``` http://127.0.0.1:7001 @@ -462,21 +515,22 @@ curl -L http://127.0.0.1:4001/v2/keys/foo -XPUT -d value=bar ``` ```json -{"action":"set","key":"/foo","value":"bar","index":5,"term:0"} +{"action":"set","key":"/foo","value":"bar","modifiedIndex":4} ``` + ### Killing Nodes in the Cluster -Let's kill the leader of the cluster and get the value from the other machine: +Now if we kill the leader of the cluster, we can get the value from one of the other two machines: ```sh curl -L http://127.0.0.1:4002/v1/keys/foo ``` -A new leader should have been elected. +We can also see that a new leader has been elected: ``` -curl -L http://127.0.0.1:4001/v1/leader +curl -L http://127.0.0.1:4002/v1/leader ``` ``` @@ -489,17 +543,11 @@ or http://127.0.0.1:7003 ``` -You should be able to see this: - -```json -{"action":"get","key":"/foo","value":"bar","index":5,"term:1"} -``` - -It succeeded! ### Testing Persistence -OK. Next let us kill all the nodes to test persistence. And restart all the nodes use the same command as before. +Next we'll kill all the nodes to test persistence. +Type `CTRL-C` on each terminal and then rerun the same command you used to start each node. Your request for the `foo` key will return the correct value: @@ -508,19 +556,24 @@ curl -L http://127.0.0.1:4002/v1/keys/foo ``` ```json -{"action":"GET","key":"/foo","value":"bar","index":5} +{"action":"get","key":"/foo","value":"bar","index":4} ``` + ### Using HTTPS between servers -In the previous example we showed how to use SSL client certs for client to server communication. Etcd can also do internal server to server communication using SSL client certs. To do this just change the ```-client*``` flags to ```-server*```. +In the previous example we showed how to use SSL client certs for client-to-server communication. +Etcd can also do internal server-to-server communication using SSL client certs. +To do this just change the `-client*` flags to `-server*`. + +If you are using SSL for server-to-server communication, you must use it on all instances of etcd. -If you are using SSL for server to server communication, you must use it on all instances of etcd. ## Contributing See [CONTRIBUTING](https://github.com/coreos/etcd/blob/master/CONTRIBUTING.md) for details on submitting patches and contacting developers via IRC and mailing lists. + ## Libraries and Tools **Tools** @@ -536,7 +589,6 @@ See [CONTRIBUTING](https://github.com/coreos/etcd/blob/master/CONTRIBUTING.md) f - [justinsb/jetcd](https://github.com/justinsb/jetcd) - [diwakergupta/jetcd](https://github.com/diwakergupta/jetcd) - **Python libraries** - [transitorykris/etcd-py](https://github.com/transitorykris/etcd-py) @@ -581,45 +633,49 @@ See [CONTRIBUTING](https://github.com/coreos/etcd/blob/master/CONTRIBUTING.md) f - [mattn/etcdenv](https://github.com/mattn/etcdenv) - "env" shebang with etcd integration - [kelseyhightower/confd](https://github.com/kelseyhightower/confd) - Manage local app config files using templates and data from etcd + ## FAQ ### What size cluster should I use? Every command the client sends to the master is broadcast to all of the followers. -But, the command is not committed until the majority of the cluster machines receive that command. +The command is not committed until the majority of the cluster machines receive that command. -Because of this majority voting property the ideal cluster should be kept small to keep speed up and be made up of an odd number of machines. +Because of this majority voting property, the ideal cluster should be kept small to keep speed up and be made up of an odd number of machines. -Odd numbers are good because if you have 8 machines the majority will be 5 and if you have 9 machines the majority with be 5. +Odd numbers are good because if you have 8 machines the majority will be 5 and if you have 9 machines the majority will still be 5. The result is that an 8 machine cluster can tolerate 3 machine failures and a 9 machine cluster can tolerate 4 nodes failures. And in the best case when all 9 machines are responding the cluster will perform at the speed of the fastest 5 nodes. + ### Why SSLv3 alert handshake failure when using SSL client auth? -The `TLS` pacakge of `golang` checks the key usage of certificate public key before using it. To use the certificate public key to do client auth, we need to add `clientAuth` to `Extended Key Usage` when creating the certificate public key. + +The `crypto/tls` package of `golang` checks the key usage of the certificate public key before using it. +To use the certificate public key to do client auth, we need to add `clientAuth` to `Extended Key Usage` when creating the certificate public key. Here is how to do it: Add the following section to your openssl.cnf: ``` -[ ssl_client ] +[ ssl_client ] ... extendedKeyUsage = clientAuth ... ``` -When creating the cert be sure to reference it in the -extensions flag: +When creating the cert be sure to reference it in the `-extensions` flag: ``` openssl ca -config openssl.cnf -policy policy_anything -extensions ssl_client -out certs/node.crt -infiles node.csr ``` + ## Project Details ### Versioning etcd uses [semantic versioning][semver]. -When we release v1.0.0 of etcd we will promise not to break the "v1" REST API. New minor versions may add additional features to the API however. You can get the version of etcd by issuing a request to /version: @@ -628,10 +684,11 @@ You can get the version of etcd by issuing a request to /version: curl -L http://127.0.0.1:4001/version ``` -During the v0 series of releases we may break the API as we fix bugs and get feedback. +During the pre-v1.0.0 series of releases we may break the API as we fix bugs and get feedback. [semver]: http://semver.org/ + ### License etcd is under the Apache 2.0 license. See the [LICENSE][license] file for details. From ca7e61052d5cdb5015e4873ddf9c2c155f5db264 Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Wed, 13 Nov 2013 06:15:35 -0800 Subject: [PATCH 244/247] refactor(mod): remove the /etcd/ prefix Originally we were going to move all of the API endpoints to /etcd/ so that you could move etcd traffic over a multiplexed HTTP proxy with other service but since we don't have a strong use case for this just remove this for now. --- mod/README.md | 2 +- mod/dashboard/README.md | 2 +- server/server.go | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/mod/README.md b/mod/README.md index 2dde929752b..5fd9c1782a6 100644 --- a/mod/README.md +++ b/mod/README.md @@ -1,7 +1,7 @@ ## Etcd modules etcd modules (mods) are higher order pieces of functionality that only -speak to the client etcd API and are presented in the `/etcd/mod` HTTP path +speak to the client etcd API and are presented in the `/mod` HTTP path of the etcd service. The basic idea is that etcd can ship things like dashboards, master diff --git a/mod/dashboard/README.md b/mod/dashboard/README.md index 1b9da7f86f0..b41fff05061 100644 --- a/mod/dashboard/README.md +++ b/mod/dashboard/README.md @@ -24,4 +24,4 @@ bower install Run etcd like you normally would and afterward browse to: -http://localhost:4001/etcd/mod/dashboard/ +http://localhost:4001/mod/dashboard/ diff --git a/server/server.go b/server/server.go index 898c578a086..541f76c017a 100644 --- a/server/server.go +++ b/server/server.go @@ -123,7 +123,7 @@ func (s *Server) installV2() { func (s *Server) installMod() { r := s.Handler.(*mux.Router) - r.PathPrefix("/etcd/mod").Handler(http.StripPrefix("/etcd/mod", mod.HttpHandler())) + r.PathPrefix("/mod").Handler(http.StripPrefix("/mod", mod.HttpHandler())) } // Adds a v1 server handler to the router. From eecd9f7e35a5d7b5b665b9172732ed4ba364fcd8 Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Wed, 13 Nov 2013 10:16:02 -0800 Subject: [PATCH 245/247] feat(etcd): create unique directory on no-flag case Tell etcd to store the log and configuration into directory in the pattern of `${machineName}.etcd` if no directory is specified. --- etcd.go | 16 +++++++++++----- server/config.go | 1 - 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/etcd.go b/etcd.go index 16124db6d6f..953fadff223 100644 --- a/etcd.go +++ b/etcd.go @@ -47,11 +47,6 @@ func main() { log.Verbose = true } - // Create data directory if it doesn't already exist. - if err := os.MkdirAll(config.DataDir, 0744); err != nil { - log.Fatalf("Unable to create path: %s", err) - } - // Load info object. info, err := config.Info() if err != nil { @@ -66,6 +61,17 @@ func main() { info.Name = host } + // Setup a default directory based on the machine name + if config.DataDir == "" { + config.DataDir = info.Name + ".etcd" + log.Warnf("Using the directory %s as the etcd configuration directory because a directory was not specified. ", config.DataDir) + } + + // Create data directory if it doesn't already exist. + if err := os.MkdirAll(config.DataDir, 0744); err != nil { + log.Fatalf("Unable to create path: %s", err) + } + // Retrieve TLS configuration. tlsConfig, err := info.EtcdTLS.Config() if err != nil { diff --git a/server/config.go b/server/config.go index df87b98946c..8d8adcf828b 100644 --- a/server/config.go +++ b/server/config.go @@ -57,7 +57,6 @@ func NewConfig() *Config { c.SystemPath = DefaultSystemConfigPath c.AdvertisedUrl = "127.0.0.1:4001" c.AdvertisedUrl = "127.0.0.1:4001" - c.DataDir = "." c.MaxClusterSize = 9 c.MaxResultBuffer = 1024 c.MaxRetryAttempts = 3 From b3704a711905b98b78c9fa20aab4443b9db0c141 Mon Sep 17 00:00:00 2001 From: Brandon Philips Date: Wed, 13 Nov 2013 11:15:08 -0800 Subject: [PATCH 246/247] fix(fixtures/ca): make the certificates work Something was wrong with the old certificates. Rebuild them to get them all working. --- README.md | 6 +- fixtures/ca/README | 7 ++- fixtures/ca/ca.crt | 52 ++++++++-------- fixtures/ca/ca.key | 100 +++++++++++++++---------------- fixtures/ca/chain.pem | 61 ------------------- fixtures/ca/chain2.pem | 61 ------------------- fixtures/ca/server-chain.pem | 63 +++++++++++++++++++ fixtures/ca/server.crt | 55 ++++++++--------- fixtures/ca/server.csr | 55 ++++++++--------- fixtures/ca/server.key | 100 +++++++++++++++---------------- fixtures/ca/server.key.insecure | 98 +++++++++++++++--------------- fixtures/ca/server.pem | 14 ----- fixtures/ca/server.pub | 14 ----- fixtures/ca/server.pub.sig | Bin 512 -> 0 bytes fixtures/ca/server2.crt | 48 +++++++-------- fixtures/ca/server2.csr | 48 +++++++-------- fixtures/ca/server2.key | 100 +++++++++++++++---------------- fixtures/ca/server2.key.insecure | 98 +++++++++++++++--------------- fixtures/ca/server2.pub | 14 ----- 19 files changed, 449 insertions(+), 545 deletions(-) delete mode 100644 fixtures/ca/chain.pem delete mode 100644 fixtures/ca/chain2.pem create mode 100644 fixtures/ca/server-chain.pem delete mode 100644 fixtures/ca/server.pem delete mode 100644 fixtures/ca/server.pub delete mode 100644 fixtures/ca/server.pub.sig delete mode 100644 fixtures/ca/server2.pub diff --git a/README.md b/README.md index 4ebeaac5137..bd5f99eb40c 100644 --- a/README.md +++ b/README.md @@ -384,7 +384,7 @@ There are a few new options we're using: You can now test the configuration using HTTPS: ```sh -curl --cacert fixtures/ca/ca.crt https://127.0.0.1:4001/v2/keys/foo -XPUT -d value=bar -v +curl --cacert ./fixtures/ca/server-chain.pem https://127.0.0.1:4001/v2/keys/foo -XPUT -d value=bar -v ``` You should be able to see the handshake succeed. @@ -416,7 +416,7 @@ The clients will provide their cert to the server and the server will check whet Try the same request to this server: ```sh -curl --cacert fixtures/ca/ca.crt https://127.0.0.1:4001/v2/keys/foo -XPUT -d value=bar -v +curl --cacert ./fixtures/ca/server-chain.pem https://127.0.0.1:4001/v2/keys/foo -XPUT -d value=bar -v ``` The request should be rejected by the server. @@ -430,7 +430,7 @@ routines:SSL3_READ_BYTES:sslv3 alert bad certificate We need to give the CA signed cert to the server. ```sh -curl -L https://127.0.0.1:4001/v1/keys/foo -XPUT -d value=bar -v --key myclient.key --cert myclient.crt -cacert clientCA.crt +curl --key ./fixtures/ca/server2.key.insecure --cert ./fixtures/ca/server2.crt --cacert ./fixtures/ca/server-chain.pem -L https://127.0.0.1:4001/v1/keys/foo -XPUT -d value=bar -v ``` You should able to see: diff --git a/fixtures/ca/README b/fixtures/ca/README index 308bcfe508a..37d2e740dad 100644 --- a/fixtures/ca/README +++ b/fixtures/ca/README @@ -1,8 +1,10 @@ -Testing x509 certs for luacrypto +Testing x509 certs for etcd + +The passphrases for the keys are `asdf`. # Make the CA cert openssl genrsa -des3 -out ca.key 4096 -openssl req -new -x509 -days 365 -key ca.key -out ca.crt -extfile openssl.cnf -extensions v3_ca +openssl req -new -x509 -days 365 -key ca.key -out ca.crt -config openssl.cnf -extensions v3_ca # Make server cert and signing request openssl genrsa -des3 -out server.key 4096 @@ -16,3 +18,4 @@ openssl rsa -in server.key -out server.key.insecure # Output "raw" public key from server crt openssl x509 -pubkey -noout -in server.crt > server.pub + diff --git a/fixtures/ca/ca.crt b/fixtures/ca/ca.crt index 1bea67e7079..35e167d8c55 100644 --- a/fixtures/ca/ca.crt +++ b/fixtures/ca/ca.crt @@ -1,31 +1,31 @@ -----BEGIN CERTIFICATE----- -MIIFajCCA1KgAwIBAgIJAL6GUooGHc/oMA0GCSqGSIb3DQEBBQUAMEUxCzAJBgNV +MIIFajCCA1KgAwIBAgIJAKMtnrbM5eQaMA0GCSqGSIb3DQEBBQUAMEUxCzAJBgNV BAYTAkFVMRMwEQYDVQQIDApTb21lLVN0YXRlMSEwHwYDVQQKDBhJbnRlcm5ldCBX -aWRnaXRzIFB0eSBMdGQwHhcNMTMwODExMDUxMTI0WhcNMTQwODExMDUxMTI0WjBF +aWRnaXRzIFB0eSBMdGQwHhcNMTMxMTEzMTkwNzMwWhcNMTQxMTEzMTkwNzMwWjBF MQswCQYDVQQGEwJBVTETMBEGA1UECAwKU29tZS1TdGF0ZTEhMB8GA1UECgwYSW50 ZXJuZXQgV2lkZ2l0cyBQdHkgTHRkMIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIIC -CgKCAgEAyAJNWQmsBtTBPv/jSjFk+EqCZM9zLcnS9P7bg8snLu1RaDS0NA8RjQFm -1fw+fAoNyOJ5X4FdEep6piMcVaYa/xGgls3DVkUytOvJ0BcdUJgrcyH0CDodDhu4 -T/qi1W7I+y3gbjr+VyyBdOSQuybyun9RwRrktcfVDfObaA0AmLt1PtJzMI+tB2As -XRgxPfFLETUTy9nIQc3PQxs11sWeEzvxcVrO595XsumPYZZAan86KNrQzES4r61R -0pOGAIEEfyvT2uU5y7fnFNtRr2xxjdgUj2/ghJX6M49BnYp4edyQuyNQp+weSA6c -3ueTu98gin1vxzMaVJJIaRRerKzekCerXLq3YsFzS7HFzMaR201faPw45b7K83bh -/DJ2wcc8JhyrhnOBM76jCnug4FReiETnCyUAc7fP+iCOCpgCzYky7wi8Jc+MTXWG -RIvpfmcB326gUdyG8n/yvIc95E6ZiQFNx9B75wikaEUcSOkp3pZxG0Fc7l60oe1l -hYpA5kL6YOdaBPSq5y1B6kFT4D6gfLYs+KS3vTWjxeLTpyRhF9eVMdxoOqUviK5X -MVVxc8KkrQbqKQw7VlmqNeA7kIsBGMOfMn3WetRY7pi1OyYMhMr/eG5r9YtaWN6Y -sHicwmyNfVHIi0McJgAS+c+7sAVnGPoHIWUf24xll4z9DUqk65cCAwEAAaNdMFsw -HQYDVR0OBBYEFLOtmFVLJOtj9kytXy/vDJROI8lSMB8GA1UdIwQYMBaAFLOtmFVL -JOtj9kytXy/vDJROI8lSMAwGA1UdEwQFMAMBAf8wCwYDVR0PBAQDAgEGMA0GCSqG -SIb3DQEBBQUAA4ICAQAN8vTepTHSkJNyU1guATIFZTil/RM1dqnC0mM1B6dkkrJj -H9m2wldxuUJJ6PwkLZXHKBBrkHSP+DaFwFD3AkNJQWoslT+o1rU/bI5fSK0QqaD7 -ZHx2BBkvfOsj8as+nzhYArlJy8YH786qG1Gbd8toP3IjzHq2HR58xeiHB0X/rvY0 -sfxPfpP0id52vJyh1Cl91hI8+KYFv3b6pesAG9drYnLZIKh0mAIdpmH5n8V9FQ2y -gkGORvRfBQdA+xTmy1MpQFeXgbE9CLHoHDXmTZneKzxSRSqwoqFxsj1fcqXC87lz -aqJWvnL6iF6exaqV7df4iT6xHp9R7sahRLKbkpe7r/jbcr1i/6aaa7Ve+Z4MtZRd -TcrNerwchF9RborO86baM6gDR3SJ4wCnfyncKFqmGJs1rrzg8gEBddZtzVZiSntt -GMup4eh1Yt/0w/AIvX8nxOUhc9P1zw3Fb80Dd7ucxbKdkOXfqZ/cEm5zyh92HMvd -RqkQee31tENYzjpqx8CXfeZ+B/tHq1baOFv6zM7yJ3Hr9KzPhKhLHXooO+qMNk+g -E5QjY82R6pRSVfVRDbJMEfS7xeJ3qrEU8UueJYx9S7qJSxB3lwunf6T4SJ4vqEmU -fwX4jSahFIUIlXGwfIDqM7P+biIyJS2AaMC5KMcatnyXDNbEZzg1ibhhpHLWIA== +CgKCAgEAs3+PRcgMLMquxhHB/1aN7lytduBGlXTsDNky3IQyQCpqFUdYYc5cS3nL +Y0wtwB4v+yNJ2qNOsYOlSUSSPS1nUxDsHWiMMPC6NxsE34wuf1jYI1UQbQiAEf73 +wB+LMTuv30ZDG9EMfwiHf1VbOGKUwkSeZcMl8EX/DfmJCB9PONFHvlS1yQHnJwqv +SvIw55UgL/7fRvmblqrMsl0g/cveSanT2bGTV6eNYlDcAfw6SsszYbKA+i5zjt9F +puZA+JbqZ2mQ4RNi228ib9qGiS1S1YgyWqiJqGD8I15nvrWV9fA93z+kYekdc+nM +HXtWnd296vfcGQfuRgKAigp0Q2Xr/r6IfT0etMwNWOT/ikAE7l5hA3xUdEba0xdZ +2PYLmrb+5mtPB8uZ8K0JSrZJU70p1hlk644Nw1S6Je5/XfUZFzwLq8OotGnRzD7Y +dyvY/DEDodiQisLtLTCI9z4F7cjadTKwSSq5Yzr8Pdq1PkahBQth1Eq8KvodOXOR +WTVxP+YBYmbbk7EEOSZ8ZI7ppqngS6/pjcjCHobZfzJdfx8YuTrBWUHucYDqeV6r +xYtvlDiOaxyij8vhaAJ7zLfUuVGHKPfLgHZDAH47a+1qnIq5tM2Zv8p9g7wg56UV +aplu4GwBqNrL+5R10P2YuBgrUOZOjIOv0u5pvBjLZpTeal8KI7sCAwEAAaNdMFsw +HQYDVR0OBBYEFOkUWSDlAWoxFoSsbnbEH9GIN8bfMB8GA1UdIwQYMBaAFOkUWSDl +AWoxFoSsbnbEH9GIN8bfMAwGA1UdEwQFMAMBAf8wCwYDVR0PBAQDAgEGMA0GCSqG +SIb3DQEBBQUAA4ICAQBi30INj8fsPRsX9p1vVtljln2dh/iOJ0shL3gz9NmUM0jV +/UchKo5iykVrrCKVlI7DD/QFXXty20x6PnbSUwfhZpJj+AwT9G8pVD/8DBU60+z0 +1zFSUxQ2GN9RDWgoiyz1QZ48n5zl7JVzzvBAf6N3jmdvgmIoKaDpqNLmQxqNsuCW +USMKqvqKwWfqvj8JQNYVmKdDVsoax36glVaj4qHZojul9eWS6SdDOo6a5t/xf0kP +Upxi87dqS4H7qfa6HTVFQhqRL8JuPqTs4csojA6XJt+yYzTfs8Gf3MAyQznuwiWh +E7kIv9lYH5APLW5LXNLizTaZtBS826f05TgBsYuYj3mGeSsr/llP4zb8u7qxL+B3 +0Q6OLK3JtPbwtaHCBxs70HOQzjWjiEF6PE3f1MhvXFjMQmQEgGzCuK69EEUWK2s0 +cIjoTLJxmQ+voWPms39gjstNLeykAygsyaYryGks/YjgchRrTtrOxUCows8knkmB +lve7RC9xW7yQwmWacPq0ndJUL0smdsWODx+L3J0R/YmbjYIO5N8i9YFqSwFLIC2v +ghirHq7EqZbYAaDxS6KvpeVh+f0pC8AC63FLbsp9+SOayoEQJ/gB8f+s3cxV+rNQ +/Z6077QuDgb1gpjCWCHHjMMELtjvy+HNUmgiRfv6a+mtWOS8g5Ii3OUYFVr2kQ== -----END CERTIFICATE----- diff --git a/fixtures/ca/ca.key b/fixtures/ca/ca.key index d32b785641e..0d46106657e 100644 --- a/fixtures/ca/ca.key +++ b/fixtures/ca/ca.key @@ -1,54 +1,54 @@ -----BEGIN RSA PRIVATE KEY----- Proc-Type: 4,ENCRYPTED -DEK-Info: DES-EDE3-CBC,90DB77765661F24C +DEK-Info: DES-EDE3-CBC,5AD5C95753A0AE70 -f0pCWQG7S6EdHqlN0tHfpowMyEFR/WYtQTjYws93+2MBcEDa9fqecN1mqNkskWsA -As1HVdp6o357bJ3VtoFHBRvv4iql6xutkZV8NoVAQseLjjZkYGFbIJ8Dv/O6xnXy -XDqm3WopCQB9hLKjORcwJN3Q9SRKmmRCxkndVcNkCjoY/TdN1jPEmQqtfWfmLy5S -GVmTHmoTPVvJh34mvkUsLrNkQ6XOrIisz93zmFO0VpskvxUmE2lOdLzTALbz2MZP -TJalI2zZSfoU6ZaF8eUG8DA5+3PknGNSRP4ivbA0CYkYn0Wssatf6+YUg1kMJ5+B -nPhh1tTeeE9bMaTEmcrJiDQFqn+EBmzXBzLDxYsUIEzCzcc4bPbvuHpZSThXFTQa -D6bakKXYW/bmO190o3XxpfQkLqzRQLWhURqjx57k2uXdxftqosoyPEcxW+o/L9am -z1PaL0EtpyQUdX0T51vjmOhe5tgtQXNXdcGkL4nkR9p75kVPCKYpPv5PBcNE/PWR -zsCJWVERknr1ZMVWdOfcOGdaINK/WcFSzK2NUpTI1urqPuvXq1Uc0hxYdkWo3GU2 -vKSqYPIOZHC6QcwQILD8xukPMjncf0vA/K1XV5WLfLwQIZLo9XoFVWESr6sOCXTU -qJ/hYB+yUQSOJD0+gBptJLu15z854YpoHoFrHpjzrSaw7NXrNex5cMzjQSNsUzDQ -p0zP3lrnftLUdSZRrPR8Uq2usv3tSRrZmQGU/We26CuajrJVB3uui8uj8i/lABJD -Ikvq/Xse5krjrlX2LTtUpE1YGfl7jE556AuDI46V2e81io0yqMDQiEU22+0REB0s -4u4Rzg8g5XFGt+gnG+RnZ+Z6/o0RgKsmilq82EuHl2WHnmCyTU3tl/t7vlPGkHa9 -D7pvMF3OkW9/tAKISx/KkDEEIlTYAnNjFhA5wd0din0xR1WpeAx6qkEwbFPzBZZi -nhgP22bX66+ST+dQ6h4+jTupiKFJX7uXo7B7ptTO4eo73+5H8xOujdtEaZ2two6a -VYUre+/ZyhmCBwdKqpXcTwXs2osqTrvkgWktVCTQYYIWNbjy+gSvuYWK+RJbzFLc -gwxASDyledtgpC+PhK9O3BxWTwqU+j4YdhjYxX5ZBdVqp5dlQVPN6fseYDjaEvrS -wgPtHpXueOCJv7vhltc5ABS4ZbxSUCQphiXYdof9FVPh9jwYF8SOlP3WWAP5opec -Vk91CWUNcTi1AgLgt/1L/FzOE6bf/x+bWIvOHLZKDO+9hT635NlQ6+HIzPbeLA0P -KksFuQ2WDjh2xHG/RAdNG6eWavzPuAKom+LiEiOQFBDNhnDTaURPSSxPhwQNYaEw -++WupENWp31GLRF1BKmZwP7hgO5fkh3Zy0Ah/fEiGIw59QLjl7bCHNBeGyl1CIKJ -h5v2zFo6PNpd6Hcxj8Wf1WKQcSJgZdZ+QjUbK4pdr1OokulgNZyL04LNndTFqh6z -PrX12xhkjxxv4xlTuyW26VMA9YJYKA+9Pg13LSTFQKklxTg/j1CK88c7IOYk8X2k -mH+fioME9sy5wDvpuAF5ufrANqeUdhn0/MrKExNDVCh4i4BDYJck/qt21w03sdx3 -jCQF32S4ltGoEA6AQCALqgfuSLUUWVuOu71dl9a08zR52MP/2pLnzF+PsHoiih09 -OA+9395WoiRvip5ei+JQGLwXfC8C9zX6kGSPeckju9v/72Gv8M5QxHvAakHZFpxe -FWG3y4BEG6TeNgSAlXYvGwDlpjSF8osjlNWubMYY7pbbM9WhNZ35NUU6fWQ8KtVL -coqQgJc07f3liv+TNiXjNsxVu72rE3L/6efbtwNnCpPO1kRcs0w1q99S/j+kVzAv -8bdfEm8S63m6+OR8AUe0EzJrLu2KA5WqkMUgtDM4Rk8fvcEMhyIGbwyRQEB8NCpT -4MQqz7fpnZZ6YZadQ8F8HI2YeOhl2M8iVWG4BhRX05MEkd9mqf8qvqmEmj2PCkKT -5vAI25LY/lwidvdCpzYOFpSFSQf4YuzHRvYnF8qKCDDdE1x9TAV4DZsjoeRnjgEq -TNXCYMxzyBw8WYxWB3e8emgK/9lHmROzb46LDVixmu9l84dx7S8SL/f+R2K4yqQ0 -jajSzMrDlBUXtjm1RHMhyTZq7PQxQxau49V+N7SCS2eej549XH5pPb2g/LwYUtr4 -UVtYYE5CH7RuS3pBBdXsAE9GnkW7JOXaUt91w831JlLofuZLn1JEGJCoVk4j+fuR -LHuHmKdbPJUacgqMYFf+XvVq7JERnhO0/MeQBoKyYy9C9x1JAPfyuThO5HHEotQa -vnaVt5S2TjTSaKAoiq1x1IkFus1yVyrRNCHIfaOKv/L/ty62Q0LVNX+7xrZrkwPp -KlWIQuhFAGVts8H6dub+M167tatDK0I0V4iHSExKLLJn3EV8yX1/7wwW3qpy6Lmm -Kt+Dme9NpTFXP3ejV3n3tpIxCUsihaeecYB7G6LnVGOMRZ5cDjv6SU0kE6MEwczd -p2tghbaNKAdV2AnhohDxVFTMVcUuMI9qdnixSMIBdx6Nax+JeFAT9dec0Hk+U8x4 -rYOZ6UV5IWrteF+I0r1oD9HL0BL0C8Va6rMq5wDhOfs1Cy5ccek/1HmcYlI5DV9D -JNAQyfH1ut3yFRusS4rgJP6GpfUeJm7EIsY4jI3uI3Z4o1zSDCOCO4mCnXfq3Lgc -Pn6CcQKxNrPCiefs7EOiPRWWbffwlpUakkTyAaU057Ge3lqBUL++bHIP9yek/UEn -26qDVzbiWRFsKWx1wvoXTAURIhAlzdjKwjje5P6GKpJcc3bUw3HsprA9JFmOPlvZ -LKCxk4ew6/7IeGYZT1L1rzziV8DBq8pqio1M0VMTxLnXfc7P9wdHXE9A2wL2dFdJ -gP9utDzrZLvP2sOf/Bz/WJB7IAPMZOTBSlRquKQjDqBpXjHWSEaknLxYxmyF0HN7 -3iutUpLtDR3KebLoW3P0lgcPj8cv6xEyGUDV8tZHkUkw7chGn0LfEQxyVWhcQbYv -69by3FV4wnTATAnWMzoYHCP+e42uLWD/a3WDpsDuasddhyHCAineUHmHMoWliMVY -z7nBoJttya5IiB1oh4ksCGjkCaqPpbtdOqXMc/KBWi75fV8GidFH4UgGPADUKa1Z +NW4oxy0HmyCZQluW6/g8A+LxWALNpscFiFWSVIVaqfOWD1sFZxzsP25pG6/dE3cR +sg5PlVZ2UOlTHoaqmtA21UKDuXnwmczqMrWtLegyz5WQLw2DpfMiEbaMhcwIy60m +yKh9N9Agf2gE+SumpzG0De338zJhcc28ogStr6d4VECbCHqdsj39cF4Os44FH/ns +PT332vLt1umLyYgMhvZWaXZGlYARU51vsB03Jb5vNyZPOfPbEWT6+jIukFWYuBK1 +WeccnYhjtFWjCg9zhOyYLyjb2kPJORvIPBwkjkjhRnFnMyJpRVEZNSseE1dDyzZV +dGgVn4HmydJY4L+Mcaexp2CLAPxGEmv/C6/2sz5sffQYZ3+3BGMjXOUJSTzpaKNj +wSLfZFiY5nZ2TpLucRmlUHU+kJvPmJzPzSPl3QrLVC/Gb0Wy3HbEZl4KWvaoTdIL +aoBFx32j9KvnvfU9x+i2wj1Z+bFig/yH4Sxp+2EpUyE/9Hqg7se14Eg1F04skb+K +Uorz9JELyQ01Mm6gxYhVzL1UPJKfEq7NBwDyFnjSqB2zUFnNnsizYkSgT/wBRo/9 +BtLI0YjXqeq2Y5OEvZ1t6a1XB9gLOx9isyA9GooXR6iaWg+9vwOm2Woo5MgNWA4b +JUYteTqXktdkqMgk7Rzz/xB9U/bAo67gMz3LR9SYYX6YPOEMcTLQfF1njPzD0fCk +WhWTfVOmPNJFoZUe5pfA5uLpNmC8fIDCFYFwRInImKfSd7SBQ70cF9LV8xvjOHCt +Sl2cdipumILO5+UfsN5EAhMPpeh8Xk+b54IAE77tnE+CRi02nw7G+7Jt/QGIgckR +ymc2+JYRdbS/Wd87nY7ipTKspV/1Bjus9cokKD3mWfURPzQOWWNpgRsB+dnkK+HB +muecoNeoWwLQ6Z11wU5B36qVtFj1/wfkMUH8LlBXqfIDHmvH9vNeLOykiZyudn0n +CjN6+hq9Gy49T141lw3Ek9XHo4mly6lQQtFdtNI9/PHyGlctqVIeMZ4WXvE3ohxq +mq0AMWqb39BHXjrno3gru039rhe6Jf/+vhbpsF+ZT1LquptL9x+ggV1U4D7d2/Mw +oY3DKc5jjUYsnJNFWarSc8M0HNaHcYXi6HshfYDqamUVWiQVaw2W4lLti8A+vBuw +NwWHQ0vI8XwIKEiWJ9/e7xSNuHp7SF8zbBKDXgnuzG6V76iASay2xEtpoJUTRMk7 +ckMa+x4IftAr2Z5KuJk9As6mSDIraTkWu7lnkIt9C3dXJR5yOIhF+XkQOoKSEjJ2 +lKh7rkz8O5HkvAubXre64MYvokdxhCFQaHPGcB810+hJQBgAcClWLHoF9acY/3ca +L6tnoGqS+B3ZPugtQqHkLzFLMMB/qI7MUUNp7DcA6P8H35L0fpX3I6mg4cty0G75 +f96KnUQDuHxis9Vqf46dh2vjZXYHAOnjTl/BqxY+vqmzSRblY81Sx+2q1U5u4OvG +WuotIykrAMIxDED66fE+67+kugrX1AMVNCty82BA+xeXim/cExOxnn578hhtIWiI +aBl1zq9KhACnHC7tVNrHXA1auhIdpk0i+WCMnPI5KFfTcSL3/PZaf/3+IagRouNl +pLK6Y3ZGgEbxL+fD7p6PJ4UQUUFV5bKHEo4S6sb0iqYBRZwfQq4VCU1bdtETMc9S +7/0ViTdOiEb37/Wblyqs1Q/aIZ809nPobGxkCY4uYnLZIYdvlxAO7eLS5cLZ3Yln +JFR/be0wQ4hztLgcqyVpr53qiVrlZkuRgwdyBiOp9Lf051M7hsNlEuVy32LFV4or +ED5fskmKhTpsX7gxhXWPqZbKYJo27Ahze2JA/luq5Abav5wq4eXduf4I0SDUlwEV +BLZVq0uIz/lHc1JrfuD+d8mUV8NMUByrKATX2kEccfYgepAs8IcBdXJfYTBmDG+7 +JU2cBbMCV1Ktf0tL8I/njI6QEwcPVjI+2fzKEkV6qz37Gwb6Fe5MT+1KtecuxG4a +vYEKVXYlSEJSxeXDV/Mdb/iXn1CoGG/A/EXirq8GLZUo3J8ftwh2EFkKLjJ/jLdr +RvtJqBg/6AD23DHwD33znpG9wKj8XY5vPJ4+K7xDvURhyqZWt7e6BkeKDaMwFAH6 +Hk2WEy48c7X1PRspOrHpRGZdVOjlElDQWrWr8eDZNl9E6noXuXDLampNp0sURvi/ +py22lh0xAGPxjj+N3ahNqweVt4pRmNmprSIkw7eX4jbe5XTdzp4YNGvA4zUNZqB5 +8WElwbVZG+F3kx8+vJ10z/PSIP1GaF9wk6ChqBNi945A8VBkZkry1W5Km63AQCUW +ejRjYm9TnV2zhIS9eVL80+LSJhyTobZbebQ/1gVck0SIpbm7uyp0mGNxvhtV25SA +t/WIFyOBWQzEDyzmc27VEHtr9mIZi25AiGaktzMfLKEZ+ZSfMKo3udwHc3uKX9fg +iVN/arnKFbtRSYI7DMJsFYLFyLASy23jNn+OhxgNJRGvhmaaHpxLoL0kCBqIIqha +SffOlLhCPxdqsP6UZ0Y8DJdAGx1LeFzaLbizTrAJ2TmYys2cawRbkH9ufaoBWxpx +rKYGii6N0yaqk9IJqz/h/WxPJmGuJF8J4yDi7jlJWTmw/l17G74LMFstD8qrUtSN +Vv4wgEeq6CeqIOCzRpo2E4nJ6uaRRatVyI9sDAb403dgeR76QoM0qhXeVRZgegT7 +iNtI5vhGsSv1xokr1WJ/cyOjnTrcCE7qGH97IMaquIWniMDw6I1vY3eLK6w7gFtD +HmsA/TQ59PgDVHP1wthzSQFpEXu2fOqrShBDF30m0MV5SlemVHxLYLl9ApEDUWbD +hQZP9r1zdWzQHiBrO0tot6SFHi4oSCtBY4cquvFAO/9HSdhlBZgBsBl3MFgc1sBG +E0DqNsHrykoHkSpqSJNNwSz4wxYpGmu1asshv4wBnoG4k4MGDgiuKpQ7BiKfz7EE +89ZARgv9cac43KZAnP3VvPpLURvjjjarQIuCS5M61vEOYr9e77v+YOQmvmcijWmt +lNKeTlZMqYYa0FSDxOQ2WtmTqQXCv5oMrgEo5AZ3WWnDus+5UFFlLmgQ9u6u2QB1 +COpJsJ/4+vfqQ+aqWBPIL639kwb2yqJzi3naAqKk/k8ze0BEQC71oFK+nr7s93Oj -----END RSA PRIVATE KEY----- diff --git a/fixtures/ca/chain.pem b/fixtures/ca/chain.pem deleted file mode 100644 index fa770f86164..00000000000 --- a/fixtures/ca/chain.pem +++ /dev/null @@ -1,61 +0,0 @@ ------BEGIN CERTIFICATE----- -MIIFMDCCAxigAwIBAgIBATANBgkqhkiG9w0BAQUFADBFMQswCQYDVQQGEwJBVTET -MBEGA1UECAwKU29tZS1TdGF0ZTEhMB8GA1UECgwYSW50ZXJuZXQgV2lkZ2l0cyBQ -dHkgTHRkMB4XDTEzMDgxMTA1MTE0OVoXDTE0MDgxMTA1MTE0OVowRTELMAkGA1UE -BhMCQVUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoMGEludGVybmV0IFdp -ZGdpdHMgUHR5IEx0ZDCCAiIwDQYJKoZIhvcNAQEBBQADggIPADCCAgoCggIBAMeY -u5Nby4uG4s5HFuky6eiLf7bMiTl4cDomb6keKMaHuSYN0zEr8jdw4r8TDSguv5MO -E3xWSLvQwLgkOQT0N7opDeEqm5Yj458+M9Z5lqYUSshWpnFNUyu8GBCfARB80YCy -8dLbYp5ORJ2AWzS5fxtp4TXEf8qlrAMoeJz2ZXMvGHMUuv8TFa3KPAajQ5n2DPSu -iFevaDitiRamz6aT48bnOowMO0Enek8UkfpTeR7uh0vOPbWOUIjzuqr7G4MnJkYD -yd7R3KscZN4iXf6NRMj2f6V4PGY0WljUbli/fT7bC8IbBgLkQxT8mO7dcJ9QHKte -Jgdju/9eto3zC3kDC3Yh2RaxfxM1vtmgZm3QT5oz95QjjskzW1gz2giE45wojbOg -nTI+QRtw2EMBX6mzaP/YU6vCvPCqhJ9zrVMsM88EK0TTdkE/NWC25JUmsPXX37Z5 -jFXIFamM0FWE/zhDip8Xfl3yqAM+NVQ5xnmWGMpqyHn/A1EaRv0ASVTMsg413N3j -r815qKy/xSCbyRIFhrBmKkwy0bSZVP+9Y0sC8+bBTZNdbNwrdp0It7BH/xKmG0Ma -TNpAgVdkbnnSt5DlqS91Sbta5i+kHibkFml4KnZ6OvJgQuTbC+UsbpD5B1vM8Rd2 -64xOQmH9KFNgouDhiBOtwAL3bn+76Pdt8OnMbO0JAgMBAAGjKzApMAkGA1UdEwQC -MAAwCwYDVR0PBAQDAgXgMA8GA1UdEQQIMAaHBH8AAAEwDQYJKoZIhvcNAQEFBQAD -ggIBAJIVOwXJkpI9bPRq6FxN7a0/2NcEU5cC9NvtvEZEjgMFkZwiPMsguwHbbsK/ -Hmg+UBo9HdOCleaPq0HhrhqDzqDGzuzCCwqDinYJl/MaTs7dBE5sJw5sCn+oESF0 -5S1rCKvvF82o1KSzj458aTWKYpOJpdJYPVu8QEm9sBPPAFcQHhevFRuVp8QBdRJD -6H4+6b4eZyADL1yM+Txt/ucuyx/6A8S/G+Uqe5Lnh1pvhZXFfWO1UF8QmYNUb0H2 -7soxruLh4k2mwF8MPSmKw8D3k4rCAMZ7W1P6OEV55Jc4OMVQ5es8tRuj9e2SHD0c -gL84rv9lNYfA/4DEKEviJTko+dD/NyIKrZCyc39Q3MmSBR+ekCNRhdCHWL5IyMB9 -o2u5g2ffsKLLjqBNIrOcGQ8vYSTsuX+y1Tonml6FiBHCgtDv7ZcwxXq37jmeorMt -QqpGJsndMObmvTVkYDN8vgEoia/nndhU7SGgi9NIYDLarDzWrU9baLta8Oq7BHaR -oMV44flX7/2Co6SOzK4y2WgQngCUaAxezN0tZPFIhZjwGwc3CbaigIaF8LTKHQ8a -cGIBGQmZ3670IDQ/vgtjHqG6LlMiJ+WR9GtWSJl3cb+4yHM/wu4oFgjYoB1MSWl2 -f5fczxP6ZXwER7NwcRaooJ/0C7XDE7ux2HsN422jgDaGT/Zw ------END CERTIFICATE----- ------BEGIN CERTIFICATE----- -MIIFajCCA1KgAwIBAgIJAL6GUooGHc/oMA0GCSqGSIb3DQEBBQUAMEUxCzAJBgNV -BAYTAkFVMRMwEQYDVQQIDApTb21lLVN0YXRlMSEwHwYDVQQKDBhJbnRlcm5ldCBX -aWRnaXRzIFB0eSBMdGQwHhcNMTMwODExMDUxMTI0WhcNMTQwODExMDUxMTI0WjBF -MQswCQYDVQQGEwJBVTETMBEGA1UECAwKU29tZS1TdGF0ZTEhMB8GA1UECgwYSW50 -ZXJuZXQgV2lkZ2l0cyBQdHkgTHRkMIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIIC -CgKCAgEAyAJNWQmsBtTBPv/jSjFk+EqCZM9zLcnS9P7bg8snLu1RaDS0NA8RjQFm -1fw+fAoNyOJ5X4FdEep6piMcVaYa/xGgls3DVkUytOvJ0BcdUJgrcyH0CDodDhu4 -T/qi1W7I+y3gbjr+VyyBdOSQuybyun9RwRrktcfVDfObaA0AmLt1PtJzMI+tB2As -XRgxPfFLETUTy9nIQc3PQxs11sWeEzvxcVrO595XsumPYZZAan86KNrQzES4r61R -0pOGAIEEfyvT2uU5y7fnFNtRr2xxjdgUj2/ghJX6M49BnYp4edyQuyNQp+weSA6c -3ueTu98gin1vxzMaVJJIaRRerKzekCerXLq3YsFzS7HFzMaR201faPw45b7K83bh -/DJ2wcc8JhyrhnOBM76jCnug4FReiETnCyUAc7fP+iCOCpgCzYky7wi8Jc+MTXWG -RIvpfmcB326gUdyG8n/yvIc95E6ZiQFNx9B75wikaEUcSOkp3pZxG0Fc7l60oe1l -hYpA5kL6YOdaBPSq5y1B6kFT4D6gfLYs+KS3vTWjxeLTpyRhF9eVMdxoOqUviK5X -MVVxc8KkrQbqKQw7VlmqNeA7kIsBGMOfMn3WetRY7pi1OyYMhMr/eG5r9YtaWN6Y -sHicwmyNfVHIi0McJgAS+c+7sAVnGPoHIWUf24xll4z9DUqk65cCAwEAAaNdMFsw -HQYDVR0OBBYEFLOtmFVLJOtj9kytXy/vDJROI8lSMB8GA1UdIwQYMBaAFLOtmFVL -JOtj9kytXy/vDJROI8lSMAwGA1UdEwQFMAMBAf8wCwYDVR0PBAQDAgEGMA0GCSqG -SIb3DQEBBQUAA4ICAQAN8vTepTHSkJNyU1guATIFZTil/RM1dqnC0mM1B6dkkrJj -H9m2wldxuUJJ6PwkLZXHKBBrkHSP+DaFwFD3AkNJQWoslT+o1rU/bI5fSK0QqaD7 -ZHx2BBkvfOsj8as+nzhYArlJy8YH786qG1Gbd8toP3IjzHq2HR58xeiHB0X/rvY0 -sfxPfpP0id52vJyh1Cl91hI8+KYFv3b6pesAG9drYnLZIKh0mAIdpmH5n8V9FQ2y -gkGORvRfBQdA+xTmy1MpQFeXgbE9CLHoHDXmTZneKzxSRSqwoqFxsj1fcqXC87lz -aqJWvnL6iF6exaqV7df4iT6xHp9R7sahRLKbkpe7r/jbcr1i/6aaa7Ve+Z4MtZRd -TcrNerwchF9RborO86baM6gDR3SJ4wCnfyncKFqmGJs1rrzg8gEBddZtzVZiSntt -GMup4eh1Yt/0w/AIvX8nxOUhc9P1zw3Fb80Dd7ucxbKdkOXfqZ/cEm5zyh92HMvd -RqkQee31tENYzjpqx8CXfeZ+B/tHq1baOFv6zM7yJ3Hr9KzPhKhLHXooO+qMNk+g -E5QjY82R6pRSVfVRDbJMEfS7xeJ3qrEU8UueJYx9S7qJSxB3lwunf6T4SJ4vqEmU -fwX4jSahFIUIlXGwfIDqM7P+biIyJS2AaMC5KMcatnyXDNbEZzg1ibhhpHLWIA== ------END CERTIFICATE----- diff --git a/fixtures/ca/chain2.pem b/fixtures/ca/chain2.pem deleted file mode 100644 index 4cfa6604af3..00000000000 --- a/fixtures/ca/chain2.pem +++ /dev/null @@ -1,61 +0,0 @@ ------BEGIN CERTIFICATE----- -MIIFMDCCAxigAwIBAgIBAjANBgkqhkiG9w0BAQUFADBFMQswCQYDVQQGEwJBVTET -MBEGA1UECAwKU29tZS1TdGF0ZTEhMB8GA1UECgwYSW50ZXJuZXQgV2lkZ2l0cyBQ -dHkgTHRkMB4XDTEzMDgxMTA1MzE1OVoXDTE0MDgxMTA1MzE1OVowRTELMAkGA1UE -BhMCQVUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoMGEludGVybmV0IFdp -ZGdpdHMgUHR5IEx0ZDCCAiIwDQYJKoZIhvcNAQEBBQADggIPADCCAgoCggIBALRv -lwJo+lU+QkAol+sFEAyW+qTNxdTKiSqeHhigpbyuo18/ILvXJRRz4Slrx1sqLjf4 -ne00N0wTy3l+DGi9kI39BrVrj8Lg9mTlfJQy/JJSDVIMli/lnkbfjNlsC3miRLmY -YxqKLZJH5onErIR+XCTJ3o4kVk6QMy3oR0LPWWz/cs4PrXNVosL6jl4tTTOyqWAC -4dtDGlDElSFui78KuSQCKO+9sepVvSFXE8Wo32LZWQ1vahg/+J/eagbw6rakl+uu -VJgfin7JH+bFsiBCkOAN4v0QF3JYchMIBeXwQzEq/HpN73Es2wPGuyglB0OGkxpu -nZ0B7bAJSOQMMNL7NkGIu6HNqORt2FzXypiXaIMUCVcIvvf2VqGBpULe+4fdLvbc -Ho/F6MzmmxfDNMwvBb1P+1nPOKc78pKWO2mqN+hOudxTbdzAiYURtjIp6oyEzvl3 -Hdgf3UUVmBQe6jPw9Cm17c7y58icPdRERoxCSdhOfwYFuls/fenPwBhMZ53+cRYG -eP2f3TT6cMzcEUkz2ZIZa4XZp0JCox1yQxy8vrmWfLo9sghqE2iRGWqRKexhr7IM -Iv9Q2wL5qcGaX1wA1gOMfpuqySb6zp8LouVEXAII9RfiRFundqYjJjtZg2sosjRJ -Eab94wfwLu9d0lKZtNBkAh65q02+Ys4Ah94IEu+ZAgMBAAGjKzApMAkGA1UdEwQC -MAAwCwYDVR0PBAQDAgXgMA8GA1UdEQQIMAaHBH8AAAEwDQYJKoZIhvcNAQEFBQAD -ggIBALWlvNm8kp3bdzymrJnvE4+sV5p+2AnKx1yZIs3h62rUM22ROU1wCAoJfyCA -LuXQNaXuWVOWUq96BCzlTScTSa1xhB/vbn9RPJVo0t+uni5fx/Pm2CHLAUijrT8z -10BHbaIPjYnmvz0lkii4Y5+Tb4WQ6yLrUYm2dpLexYwyOUhmGQNGRgY750dwf8Fe -2TBFOV9rkXlhgdopWYPhUv0ZmciYGwrJ2+9jULDFhT+PDrdAjbeDARPKcMi0jpZ1 -zBHyC6lNT167Gdj9LVV6dIFEHozzrqdMecz5CJrgKPL0s5bM88DRssupS1WgT1RG -qGVxfcuBYRLtz10W5+JBXvA3JRHgaPotkqvKsUqeII/nqvu+qSRDnh7O+i1PJUTr -D+5CSMxUK9DvxH1gUYhnQ5asP9PXZxp8hlGGwyDVu2rYTQpDyiJnHGmsWfSZuSOy -W8ViseuFe3WmdsD0wo6VguyPFMHGzh5Sx/onb4eeASz/BtcGYVPApD4WByF9WlVF -Cg3SfvNPj2fvI92DP6KAKtDgOdcHidzwPAh3XCZGikN19Oz3cCYf+AT+s/KNfvMt -B6DplYeleAlKTXYsS4ycGojGp4DpRzrxSb2mhHdHsz51H/gn9+Rgx4+QAIJGKqxk -yNRnW/UpsJbN7G7hI3pgBEFRD+QE4zvGwkn6+SwxxozhtZZ4 ------END CERTIFICATE----- ------BEGIN CERTIFICATE----- -MIIFajCCA1KgAwIBAgIJAL6GUooGHc/oMA0GCSqGSIb3DQEBBQUAMEUxCzAJBgNV -BAYTAkFVMRMwEQYDVQQIDApTb21lLVN0YXRlMSEwHwYDVQQKDBhJbnRlcm5ldCBX -aWRnaXRzIFB0eSBMdGQwHhcNMTMwODExMDUxMTI0WhcNMTQwODExMDUxMTI0WjBF -MQswCQYDVQQGEwJBVTETMBEGA1UECAwKU29tZS1TdGF0ZTEhMB8GA1UECgwYSW50 -ZXJuZXQgV2lkZ2l0cyBQdHkgTHRkMIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIIC -CgKCAgEAyAJNWQmsBtTBPv/jSjFk+EqCZM9zLcnS9P7bg8snLu1RaDS0NA8RjQFm -1fw+fAoNyOJ5X4FdEep6piMcVaYa/xGgls3DVkUytOvJ0BcdUJgrcyH0CDodDhu4 -T/qi1W7I+y3gbjr+VyyBdOSQuybyun9RwRrktcfVDfObaA0AmLt1PtJzMI+tB2As -XRgxPfFLETUTy9nIQc3PQxs11sWeEzvxcVrO595XsumPYZZAan86KNrQzES4r61R -0pOGAIEEfyvT2uU5y7fnFNtRr2xxjdgUj2/ghJX6M49BnYp4edyQuyNQp+weSA6c -3ueTu98gin1vxzMaVJJIaRRerKzekCerXLq3YsFzS7HFzMaR201faPw45b7K83bh -/DJ2wcc8JhyrhnOBM76jCnug4FReiETnCyUAc7fP+iCOCpgCzYky7wi8Jc+MTXWG -RIvpfmcB326gUdyG8n/yvIc95E6ZiQFNx9B75wikaEUcSOkp3pZxG0Fc7l60oe1l -hYpA5kL6YOdaBPSq5y1B6kFT4D6gfLYs+KS3vTWjxeLTpyRhF9eVMdxoOqUviK5X -MVVxc8KkrQbqKQw7VlmqNeA7kIsBGMOfMn3WetRY7pi1OyYMhMr/eG5r9YtaWN6Y -sHicwmyNfVHIi0McJgAS+c+7sAVnGPoHIWUf24xll4z9DUqk65cCAwEAAaNdMFsw -HQYDVR0OBBYEFLOtmFVLJOtj9kytXy/vDJROI8lSMB8GA1UdIwQYMBaAFLOtmFVL -JOtj9kytXy/vDJROI8lSMAwGA1UdEwQFMAMBAf8wCwYDVR0PBAQDAgEGMA0GCSqG -SIb3DQEBBQUAA4ICAQAN8vTepTHSkJNyU1guATIFZTil/RM1dqnC0mM1B6dkkrJj -H9m2wldxuUJJ6PwkLZXHKBBrkHSP+DaFwFD3AkNJQWoslT+o1rU/bI5fSK0QqaD7 -ZHx2BBkvfOsj8as+nzhYArlJy8YH786qG1Gbd8toP3IjzHq2HR58xeiHB0X/rvY0 -sfxPfpP0id52vJyh1Cl91hI8+KYFv3b6pesAG9drYnLZIKh0mAIdpmH5n8V9FQ2y -gkGORvRfBQdA+xTmy1MpQFeXgbE9CLHoHDXmTZneKzxSRSqwoqFxsj1fcqXC87lz -aqJWvnL6iF6exaqV7df4iT6xHp9R7sahRLKbkpe7r/jbcr1i/6aaa7Ve+Z4MtZRd -TcrNerwchF9RborO86baM6gDR3SJ4wCnfyncKFqmGJs1rrzg8gEBddZtzVZiSntt -GMup4eh1Yt/0w/AIvX8nxOUhc9P1zw3Fb80Dd7ucxbKdkOXfqZ/cEm5zyh92HMvd -RqkQee31tENYzjpqx8CXfeZ+B/tHq1baOFv6zM7yJ3Hr9KzPhKhLHXooO+qMNk+g -E5QjY82R6pRSVfVRDbJMEfS7xeJ3qrEU8UueJYx9S7qJSxB3lwunf6T4SJ4vqEmU -fwX4jSahFIUIlXGwfIDqM7P+biIyJS2AaMC5KMcatnyXDNbEZzg1ibhhpHLWIA== ------END CERTIFICATE----- diff --git a/fixtures/ca/server-chain.pem b/fixtures/ca/server-chain.pem new file mode 100644 index 00000000000..c56bb659cba --- /dev/null +++ b/fixtures/ca/server-chain.pem @@ -0,0 +1,63 @@ +-----BEGIN CERTIFICATE----- +MIIFajCCA1KgAwIBAgIJAKMtnrbM5eQaMA0GCSqGSIb3DQEBBQUAMEUxCzAJBgNV +BAYTAkFVMRMwEQYDVQQIDApTb21lLVN0YXRlMSEwHwYDVQQKDBhJbnRlcm5ldCBX +aWRnaXRzIFB0eSBMdGQwHhcNMTMxMTEzMTkwNzMwWhcNMTQxMTEzMTkwNzMwWjBF +MQswCQYDVQQGEwJBVTETMBEGA1UECAwKU29tZS1TdGF0ZTEhMB8GA1UECgwYSW50 +ZXJuZXQgV2lkZ2l0cyBQdHkgTHRkMIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIIC +CgKCAgEAs3+PRcgMLMquxhHB/1aN7lytduBGlXTsDNky3IQyQCpqFUdYYc5cS3nL +Y0wtwB4v+yNJ2qNOsYOlSUSSPS1nUxDsHWiMMPC6NxsE34wuf1jYI1UQbQiAEf73 +wB+LMTuv30ZDG9EMfwiHf1VbOGKUwkSeZcMl8EX/DfmJCB9PONFHvlS1yQHnJwqv +SvIw55UgL/7fRvmblqrMsl0g/cveSanT2bGTV6eNYlDcAfw6SsszYbKA+i5zjt9F +puZA+JbqZ2mQ4RNi228ib9qGiS1S1YgyWqiJqGD8I15nvrWV9fA93z+kYekdc+nM +HXtWnd296vfcGQfuRgKAigp0Q2Xr/r6IfT0etMwNWOT/ikAE7l5hA3xUdEba0xdZ +2PYLmrb+5mtPB8uZ8K0JSrZJU70p1hlk644Nw1S6Je5/XfUZFzwLq8OotGnRzD7Y +dyvY/DEDodiQisLtLTCI9z4F7cjadTKwSSq5Yzr8Pdq1PkahBQth1Eq8KvodOXOR +WTVxP+YBYmbbk7EEOSZ8ZI7ppqngS6/pjcjCHobZfzJdfx8YuTrBWUHucYDqeV6r +xYtvlDiOaxyij8vhaAJ7zLfUuVGHKPfLgHZDAH47a+1qnIq5tM2Zv8p9g7wg56UV +aplu4GwBqNrL+5R10P2YuBgrUOZOjIOv0u5pvBjLZpTeal8KI7sCAwEAAaNdMFsw +HQYDVR0OBBYEFOkUWSDlAWoxFoSsbnbEH9GIN8bfMB8GA1UdIwQYMBaAFOkUWSDl +AWoxFoSsbnbEH9GIN8bfMAwGA1UdEwQFMAMBAf8wCwYDVR0PBAQDAgEGMA0GCSqG +SIb3DQEBBQUAA4ICAQBi30INj8fsPRsX9p1vVtljln2dh/iOJ0shL3gz9NmUM0jV +/UchKo5iykVrrCKVlI7DD/QFXXty20x6PnbSUwfhZpJj+AwT9G8pVD/8DBU60+z0 +1zFSUxQ2GN9RDWgoiyz1QZ48n5zl7JVzzvBAf6N3jmdvgmIoKaDpqNLmQxqNsuCW +USMKqvqKwWfqvj8JQNYVmKdDVsoax36glVaj4qHZojul9eWS6SdDOo6a5t/xf0kP +Upxi87dqS4H7qfa6HTVFQhqRL8JuPqTs4csojA6XJt+yYzTfs8Gf3MAyQznuwiWh +E7kIv9lYH5APLW5LXNLizTaZtBS826f05TgBsYuYj3mGeSsr/llP4zb8u7qxL+B3 +0Q6OLK3JtPbwtaHCBxs70HOQzjWjiEF6PE3f1MhvXFjMQmQEgGzCuK69EEUWK2s0 +cIjoTLJxmQ+voWPms39gjstNLeykAygsyaYryGks/YjgchRrTtrOxUCows8knkmB +lve7RC9xW7yQwmWacPq0ndJUL0smdsWODx+L3J0R/YmbjYIO5N8i9YFqSwFLIC2v +ghirHq7EqZbYAaDxS6KvpeVh+f0pC8AC63FLbsp9+SOayoEQJ/gB8f+s3cxV+rNQ +/Z6077QuDgb1gpjCWCHHjMMELtjvy+HNUmgiRfv6a+mtWOS8g5Ii3OUYFVr2kQ== +-----END CERTIFICATE----- +-----BEGIN CERTIFICATE----- +MIIFfDCCA2SgAwIBAgIBATANBgkqhkiG9w0BAQUFADBFMQswCQYDVQQGEwJBVTET +MBEGA1UECAwKU29tZS1TdGF0ZTEhMB8GA1UECgwYSW50ZXJuZXQgV2lkZ2l0cyBQ +dHkgTHRkMB4XDTEzMTExMzE5MDgxMloXDTE0MTExMzE5MDgxMlowZTELMAkGA1UE +BhMCQVUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoMGEludGVybmV0IFdp +ZGdpdHMgUHR5IEx0ZDEeMBwGA1UEAwwVaHR0cDovLzEyNy4wLjAuMTo0MDAxMIIC +IjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKCAgEA1XBtjDav5Sl3H+/fUcGiQO36 +oqtZG+YuC9D0z0u89Shq+XNs3tRtonGGCyEIrDtI6R7PItMJa6rQ1VuFoMWPrjmF +f5tFemSOZtQx/DF78H+5tWaIBVDA+Kw1zxdqj1n3/AQjAGsSuqhgcaIQQFqTNNtA +tW40048fDh17jWIDB9baF65az2uArq97uS4deDujG3CHV9svO7hoqpzYt039VDKK +4N+dDMUZFqhEmY2MqjyQySY2bd/gsYBjcGWSIuonALactiYDc4zIusAfNptnXycw +K/aQAqDwhwMcQA9L5YKQ5hoUukDTQFbvoNLJ3vNQDI/o8sjCh94EkMuopSp90tJ/ +syTPKRlh/MaGMXvwu8Vab5iPeVop48jTKl3Z+G8NErGM8KKCyd1mQoGisVuIMQPK +uJUi7jOu6wgXlA8ZgUGfSQQDA4v2Q0tV/GlJmvsP5JshA3v7C/sSBY/3AnPHeWTl +ozXlNgXitxps1EwgR2jo+YW2gxrfM//xtgMCjMXjO9g2TsCnWR6j93oXWn88UP/C +eAcyjeTdJjW+piuLdvYOctY6+Yql5gm9Vx0u+w9jTmpzOCoh+9cNtjqmPiOhecFc +Vf71vMf4krMp6lmY/Nq1/km9u0ViNP5CJHk18YXG42vnj7sUgT7WgpLh8g/iPc2p +etH9lMd9te+Jyak/zA8CAwEAAaNXMFUwCQYDVR0TBAIwADALBgNVHQ8EBAMCBeAw +KgYDVR0lAQH/BCAwHgYIKwYBBQUHAwgGCCsGAQUFBwMBBggrBgEFBQcDAjAPBgNV +HREECDAGhwR/AAABMA0GCSqGSIb3DQEBBQUAA4ICAQCmF13fF7kAcgCzRKBCpggz +ObVNSK9btGKUU+xyOZsRgEDWdfeBvG3VzXflLMDv3P45O90CHkcGfckbj1ISctIY +0OSnCn4vmIwnhwUWyYlCagtNB68GVcPH8XLxn/wsnT2MbiA/ohUStKXQtBmBxIUH +rI+7r28qcHW7AnT47G5BbSYzW0xYzlkUwyl8rBxRLDKVTHBJ5GmlREJKdAzKS7CS +6jhcsxa544Rsa7CDvjLKpJO0iploL0/GY+5oj3VdhgdEJgwqwvu3skfd1N8wkxH4 +NQuRmyvXxcMSxKv4vbOOUm4PfOqOwwXiVZLoc29ePUv9zCU2AVdS21DD9zAZeKDb +B87VWnQKO6JUvL5vX7xsMnsSbnLHGA2kPv4IDZ6jKuZdVneM+whDZlBWpHRL2RKX +K0JZICf7+EbmrUW3Rwl+dIaIJ55Ni1rfDSZWeIYKFx04Mod6Wbch7ahb/XVvIDe9 +SFjLfeNj7L/Iz0i+1lTarAMxIRC521IwcobhAxqxYCv3oNf6f+tz8DyCCvsTCc9W +/OLKX7sukh1ohle+0EFrSYpX5PzkHwZRVZjx55KwkIV/KtwBadJv+z6iwW3qOn/A +/1yC8Mbc2TdCaRPwFO80LAg9cz+XfT5vZoQUvOnOxIrhFnasQ/xiovy0zKCr2Fjg +ePQ4BNEN9wt3SsPp8ig39g== +-----END CERTIFICATE----- diff --git a/fixtures/ca/server.crt b/fixtures/ca/server.crt index 7363b399d3d..8c8b58b3310 100644 --- a/fixtures/ca/server.crt +++ b/fixtures/ca/server.crt @@ -1,31 +1,32 @@ -----BEGIN CERTIFICATE----- -MIIFXDCCA0SgAwIBAgIBATANBgkqhkiG9w0BAQUFADBFMQswCQYDVQQGEwJBVTET +MIIFfDCCA2SgAwIBAgIBATANBgkqhkiG9w0BAQUFADBFMQswCQYDVQQGEwJBVTET MBEGA1UECAwKU29tZS1TdGF0ZTEhMB8GA1UECgwYSW50ZXJuZXQgV2lkZ2l0cyBQ -dHkgTHRkMB4XDTEzMDgxMTE1MzIyNloXDTE0MDgxMTE1MzIyNlowRTELMAkGA1UE +dHkgTHRkMB4XDTEzMTExMzE5MDgxMloXDTE0MTExMzE5MDgxMlowZTELMAkGA1UE BhMCQVUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoMGEludGVybmV0IFdp -ZGdpdHMgUHR5IEx0ZDCCAiIwDQYJKoZIhvcNAQEBBQADggIPADCCAgoCggIBAMeY -u5Nby4uG4s5HFuky6eiLf7bMiTl4cDomb6keKMaHuSYN0zEr8jdw4r8TDSguv5MO -E3xWSLvQwLgkOQT0N7opDeEqm5Yj458+M9Z5lqYUSshWpnFNUyu8GBCfARB80YCy -8dLbYp5ORJ2AWzS5fxtp4TXEf8qlrAMoeJz2ZXMvGHMUuv8TFa3KPAajQ5n2DPSu -iFevaDitiRamz6aT48bnOowMO0Enek8UkfpTeR7uh0vOPbWOUIjzuqr7G4MnJkYD -yd7R3KscZN4iXf6NRMj2f6V4PGY0WljUbli/fT7bC8IbBgLkQxT8mO7dcJ9QHKte -Jgdju/9eto3zC3kDC3Yh2RaxfxM1vtmgZm3QT5oz95QjjskzW1gz2giE45wojbOg -nTI+QRtw2EMBX6mzaP/YU6vCvPCqhJ9zrVMsM88EK0TTdkE/NWC25JUmsPXX37Z5 -jFXIFamM0FWE/zhDip8Xfl3yqAM+NVQ5xnmWGMpqyHn/A1EaRv0ASVTMsg413N3j -r815qKy/xSCbyRIFhrBmKkwy0bSZVP+9Y0sC8+bBTZNdbNwrdp0It7BH/xKmG0Ma -TNpAgVdkbnnSt5DlqS91Sbta5i+kHibkFml4KnZ6OvJgQuTbC+UsbpD5B1vM8Rd2 -64xOQmH9KFNgouDhiBOtwAL3bn+76Pdt8OnMbO0JAgMBAAGjVzBVMAkGA1UdEwQC -MAAwCwYDVR0PBAQDAgXgMCoGA1UdJQEB/wQgMB4GCCsGAQUFBwMIBggrBgEFBQcD -AQYIKwYBBQUHAwIwDwYDVR0RBAgwBocEfwAAATANBgkqhkiG9w0BAQUFAAOCAgEA -eM4HmrFKqN3KJK8pC6oVOLvN7BLyy634udGInNmeAbfhDehrSrwqS9zUIJ6EpP4m -rBWP9NFK3pKt2hDhPhGGCyR2LWSJ+jPzpsNcQedGpQm7K+mhWaXsk2+ogzq9Gh51 -dtViSPQWziGVV4bjeM8nwk9f8vV1qNRfu1+kSZz5W58+JtYq6a1yqr8iudhtDVy6 -+yvrzCiMRJ6Oiqen8/5S/9VaaUq5alu8eseNwQ2+PZCBAwMB4UWjRiD+xC1gJo6f -KctVHDLnxUCweMZ+ZQzvi+S/sVIz0UbQ/u2tC2VdNmIlyQPi0RcQ+a423nrDculB -nBHiPbx6uSGK2sS4yiU8v2J/K9RS5m5qi/hJZTv8RRrxG93aIbiD5rjQsN3Tcg8X -IIfU648G2CJq3iH7P1OYrC5P5DriCXnn9higxKNecqN7yZDl+u7NBBFReucLi8Qw -bZlvtsIwumu/Z9mkcVIOxt9ZJgW51uzarozdLZlkFvnLFpuvferRdrb47R/Hj+GT -UVZ8knL4pgT3oXVS7vfyl/X99gZTRg+UaRzIAhGYFiy9RZJ+iG5mjRtrFQVtHjbP -UGiKS/e0GwpM5wFQfIh3pHvmQ67nyhe9xcaf5sLlpgTNIUgkM8TViaVeFVwPIIUS -he8NCYMr31zwHSDl8rrTapldn19XHrhiGnD6xvN38cs= +ZGdpdHMgUHR5IEx0ZDEeMBwGA1UEAwwVaHR0cDovLzEyNy4wLjAuMTo0MDAxMIIC +IjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKCAgEA1XBtjDav5Sl3H+/fUcGiQO36 +oqtZG+YuC9D0z0u89Shq+XNs3tRtonGGCyEIrDtI6R7PItMJa6rQ1VuFoMWPrjmF +f5tFemSOZtQx/DF78H+5tWaIBVDA+Kw1zxdqj1n3/AQjAGsSuqhgcaIQQFqTNNtA +tW40048fDh17jWIDB9baF65az2uArq97uS4deDujG3CHV9svO7hoqpzYt039VDKK +4N+dDMUZFqhEmY2MqjyQySY2bd/gsYBjcGWSIuonALactiYDc4zIusAfNptnXycw +K/aQAqDwhwMcQA9L5YKQ5hoUukDTQFbvoNLJ3vNQDI/o8sjCh94EkMuopSp90tJ/ +syTPKRlh/MaGMXvwu8Vab5iPeVop48jTKl3Z+G8NErGM8KKCyd1mQoGisVuIMQPK +uJUi7jOu6wgXlA8ZgUGfSQQDA4v2Q0tV/GlJmvsP5JshA3v7C/sSBY/3AnPHeWTl +ozXlNgXitxps1EwgR2jo+YW2gxrfM//xtgMCjMXjO9g2TsCnWR6j93oXWn88UP/C +eAcyjeTdJjW+piuLdvYOctY6+Yql5gm9Vx0u+w9jTmpzOCoh+9cNtjqmPiOhecFc +Vf71vMf4krMp6lmY/Nq1/km9u0ViNP5CJHk18YXG42vnj7sUgT7WgpLh8g/iPc2p +etH9lMd9te+Jyak/zA8CAwEAAaNXMFUwCQYDVR0TBAIwADALBgNVHQ8EBAMCBeAw +KgYDVR0lAQH/BCAwHgYIKwYBBQUHAwgGCCsGAQUFBwMBBggrBgEFBQcDAjAPBgNV +HREECDAGhwR/AAABMA0GCSqGSIb3DQEBBQUAA4ICAQCmF13fF7kAcgCzRKBCpggz +ObVNSK9btGKUU+xyOZsRgEDWdfeBvG3VzXflLMDv3P45O90CHkcGfckbj1ISctIY +0OSnCn4vmIwnhwUWyYlCagtNB68GVcPH8XLxn/wsnT2MbiA/ohUStKXQtBmBxIUH +rI+7r28qcHW7AnT47G5BbSYzW0xYzlkUwyl8rBxRLDKVTHBJ5GmlREJKdAzKS7CS +6jhcsxa544Rsa7CDvjLKpJO0iploL0/GY+5oj3VdhgdEJgwqwvu3skfd1N8wkxH4 +NQuRmyvXxcMSxKv4vbOOUm4PfOqOwwXiVZLoc29ePUv9zCU2AVdS21DD9zAZeKDb +B87VWnQKO6JUvL5vX7xsMnsSbnLHGA2kPv4IDZ6jKuZdVneM+whDZlBWpHRL2RKX +K0JZICf7+EbmrUW3Rwl+dIaIJ55Ni1rfDSZWeIYKFx04Mod6Wbch7ahb/XVvIDe9 +SFjLfeNj7L/Iz0i+1lTarAMxIRC521IwcobhAxqxYCv3oNf6f+tz8DyCCvsTCc9W +/OLKX7sukh1ohle+0EFrSYpX5PzkHwZRVZjx55KwkIV/KtwBadJv+z6iwW3qOn/A +/1yC8Mbc2TdCaRPwFO80LAg9cz+XfT5vZoQUvOnOxIrhFnasQ/xiovy0zKCr2Fjg +ePQ4BNEN9wt3SsPp8ig39g== -----END CERTIFICATE----- diff --git a/fixtures/ca/server.csr b/fixtures/ca/server.csr index 324d13347f3..bf05ac4771b 100644 --- a/fixtures/ca/server.csr +++ b/fixtures/ca/server.csr @@ -1,29 +1,30 @@ -----BEGIN CERTIFICATE REQUEST----- -MIIE8DCCAtgCAQAwRTELMAkGA1UEBhMCQVUxEzARBgNVBAgMClNvbWUtU3RhdGUx -ITAfBgNVBAoMGEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDCCAiIwDQYJKoZIhvcN -AQEBBQADggIPADCCAgoCggIBAMeYu5Nby4uG4s5HFuky6eiLf7bMiTl4cDomb6ke -KMaHuSYN0zEr8jdw4r8TDSguv5MOE3xWSLvQwLgkOQT0N7opDeEqm5Yj458+M9Z5 -lqYUSshWpnFNUyu8GBCfARB80YCy8dLbYp5ORJ2AWzS5fxtp4TXEf8qlrAMoeJz2 -ZXMvGHMUuv8TFa3KPAajQ5n2DPSuiFevaDitiRamz6aT48bnOowMO0Enek8UkfpT -eR7uh0vOPbWOUIjzuqr7G4MnJkYDyd7R3KscZN4iXf6NRMj2f6V4PGY0WljUbli/ -fT7bC8IbBgLkQxT8mO7dcJ9QHKteJgdju/9eto3zC3kDC3Yh2RaxfxM1vtmgZm3Q -T5oz95QjjskzW1gz2giE45wojbOgnTI+QRtw2EMBX6mzaP/YU6vCvPCqhJ9zrVMs -M88EK0TTdkE/NWC25JUmsPXX37Z5jFXIFamM0FWE/zhDip8Xfl3yqAM+NVQ5xnmW -GMpqyHn/A1EaRv0ASVTMsg413N3jr815qKy/xSCbyRIFhrBmKkwy0bSZVP+9Y0sC -8+bBTZNdbNwrdp0It7BH/xKmG0MaTNpAgVdkbnnSt5DlqS91Sbta5i+kHibkFml4 -KnZ6OvJgQuTbC+UsbpD5B1vM8Rd264xOQmH9KFNgouDhiBOtwAL3bn+76Pdt8OnM -bO0JAgMBAAGgZjBkBgkqhkiG9w0BCQ4xVzBVMAkGA1UdEwQCMAAwCwYDVR0PBAQD -AgXgMCoGA1UdJQEB/wQgMB4GCCsGAQUFBwMIBggrBgEFBQcDAQYIKwYBBQUHAwIw -DwYDVR0RBAgwBocEfwAAATANBgkqhkiG9w0BAQUFAAOCAgEAL5fYwydR3LaAEX0N -RNYbAyeunF0oYSKFz7vL3XF+nDHN8WBd49wfApBb9VCEcjINRInC7/i57xNlw4Bw -phdQiPGTG4dUdtfMw56n9z47RmZpmMs1sfS3CdPuJFb5NcrLGfBJRjRZ9+UgkbmA -+4LP2QHMi4viS3r2DmRhIrKl8Ov+S2TTyY4QdazK4PjtmtqYrB6XeE4I5cT4UsK2 -6jB32U4JCK7mUwaFenSkMKOSXNKBz4dzM1508WuO8z+lbTQSoUBW6YK89XHzhye3 -URDlLzSQy0BYZ/J4djpEDbJ/t+52vQrXU/mAbSZMuiRaacVs90b2r8MkYqTH/BCM -3e5dtZIk28K27mR54/K0noS46l2TXPbDZIxaCVyaBjw/ogC2FoIEVOcVdISZ7XOj -NigTBW1ndBRqXYpKAAlGVV3dIxe54OH7awt+Arn63S9YXprjXn78N6ohl1OMxG/8 -ES+FAY+0Oly7pOZsbg9W08Ao3CTqW5cobVcQE36ZWO2lahb1w6Ya7B843g7S56WS -GatSli9UHN5bUrb135elwwzDixeb/PPnYqBIGG2hOSZJz5oxwnWqxiGR/pApHuRx -Beta1BwPNZ897jbZ/M+JuOep52OGsZYKdJ1dDICrVdnSmEUPbS6L3vG6ZH1SdAix -LRTenXfl/mlG3QW2aLSn+kjtGW8= +MIIFEDCCAvgCAQAwZTELMAkGA1UEBhMCQVUxEzARBgNVBAgMClNvbWUtU3RhdGUx +ITAfBgNVBAoMGEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDEeMBwGA1UEAwwVaHR0 +cDovLzEyNy4wLjAuMTo0MDAxMIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKC +AgEA1XBtjDav5Sl3H+/fUcGiQO36oqtZG+YuC9D0z0u89Shq+XNs3tRtonGGCyEI +rDtI6R7PItMJa6rQ1VuFoMWPrjmFf5tFemSOZtQx/DF78H+5tWaIBVDA+Kw1zxdq +j1n3/AQjAGsSuqhgcaIQQFqTNNtAtW40048fDh17jWIDB9baF65az2uArq97uS4d +eDujG3CHV9svO7hoqpzYt039VDKK4N+dDMUZFqhEmY2MqjyQySY2bd/gsYBjcGWS +IuonALactiYDc4zIusAfNptnXycwK/aQAqDwhwMcQA9L5YKQ5hoUukDTQFbvoNLJ +3vNQDI/o8sjCh94EkMuopSp90tJ/syTPKRlh/MaGMXvwu8Vab5iPeVop48jTKl3Z ++G8NErGM8KKCyd1mQoGisVuIMQPKuJUi7jOu6wgXlA8ZgUGfSQQDA4v2Q0tV/GlJ +mvsP5JshA3v7C/sSBY/3AnPHeWTlozXlNgXitxps1EwgR2jo+YW2gxrfM//xtgMC +jMXjO9g2TsCnWR6j93oXWn88UP/CeAcyjeTdJjW+piuLdvYOctY6+Yql5gm9Vx0u ++w9jTmpzOCoh+9cNtjqmPiOhecFcVf71vMf4krMp6lmY/Nq1/km9u0ViNP5CJHk1 +8YXG42vnj7sUgT7WgpLh8g/iPc2petH9lMd9te+Jyak/zA8CAwEAAaBmMGQGCSqG +SIb3DQEJDjFXMFUwCQYDVR0TBAIwADALBgNVHQ8EBAMCBeAwKgYDVR0lAQH/BCAw +HgYIKwYBBQUHAwgGCCsGAQUFBwMBBggrBgEFBQcDAjAPBgNVHREECDAGhwR/AAAB +MA0GCSqGSIb3DQEBBQUAA4ICAQCeVkI4JwSyax5kJJ1iljkngfK+BdsrqBJWPITj +mrl9DH7/2ev0GdzHEG3oQrti6vMvsowS6vHtpgc1tsbS9UwDY/N0ZgFWgnb5O5k2 +4zZfGkma3GHCvG9WXsA9+Gs7KedggsXxfnJTLe4B/sZqtRO0dMD/JZTJQ6reuayh +bYvVBVSmittAjsfer+9xuXkHYYAPNYmW52aUN1AhnIsS3TVp1vHcxgNoFOQglN21 +lHwmeh5QbTx/paHFnWLLqLVydbiB/Qzz6x4zsEKESATd02WbN9XKUfGM0G+bG+57 +ErtrU7yzsLjPYYPcP9nYg8dzfdwVgfdjg+yw2hdmkqjDQD3YAmxRcat7uK8htVa0 +z4dfjdNRO3HhSLALKS/Tl9qpLKpEi8/0ByYErJz6i+Xyf4pkdPQcBQKybkFja136 +9xkonhE7DLTo1zQobfAJlnfTMxuJc0mOGvT+DqGSCFmNEl3WgIAgu9m77mp81Bqo +0qwrB3pYSAzL9xHuluwZMn37sdmVFFReEkEaRllRgDTZL9vSQh2yOqtV920083/y +sHPUijSsKysSKz0RuzMBCc3RD07Kcs1TFg/NdZiYKf8V9NDDOgk5LC2Sqoin7v6F +yB4wpnm6RqQ7iSRpp/VBs3PAnK2uJEkoOU5p5jZdQ0IDtesHVzM7bCUIrQIX2pAr +owvMhQ== -----END CERTIFICATE REQUEST----- diff --git a/fixtures/ca/server.key b/fixtures/ca/server.key index 14293421285..9bbdf105abf 100644 --- a/fixtures/ca/server.key +++ b/fixtures/ca/server.key @@ -1,54 +1,54 @@ -----BEGIN RSA PRIVATE KEY----- Proc-Type: 4,ENCRYPTED -DEK-Info: DES-EDE3-CBC,34239066AD971D20 +DEK-Info: DES-EDE3-CBC,E445BB698AB9CDC4 -BEq7ca8qDXV9TYCqW/mFBtgvzU778iPdYJcRWU+T4etNUhcT3dYoANMvUX6/xvOi -UBgNguRfOTRJXRQTmbjLFz9kERVfF9N6ZBaZnCGCURHohK9HOvo37U7r+mmarJH/ -KswF7jQOOSNIY0ikp349NWFj8gYvN+ivGAcEbnW8C5p91nRYwqvtluZ7v+qXhzDV -3elSXL8ahCaqIRFra1WjsR6/LG1pQ86yjDq9twRlNylQA4VVQ3tIXw/i7gdmfZw/ -HxrMBv/ZnkbpLeew3cykmV9NxUO+e1wBIooPvqmYzQIwsXTqtH6BWI29PNE6X8ZL -sVkO1bowK7B0aUKsk7Xlu11CGTEpTv0AZUJXmtcl7T1szDdXHZ917tSFSaKPuvpB -fETWc201PRy28QGD0w8LrXVOtweH1HP7sI5vpcDRhhKTu+sj0VQn29rNImuqeyoM -6XoR9pCUt+iZTDjQ0dNpa2zIKxehrAqWDNfYw8AH7+kBEi1jULnwouQK7y8qQaul -DXhIRqRodO0fDaw0RTmYO9bciqL55N22VBfngG4CNtknaCB1e42cDuFRtyL61hpP -oDS/3u+zbmd1eeMpQl6DD6EjRSIC5apPa3jrVqCZ/vshlX1N2UK0sTWNwK+Jj66L -f8Px/AZzgKIZaKDoIpKSfuKyHo6lDf6qggw2LWy2evLKI7qAtU5vNpAJ5QIPNLu1 -d24tlJYMrHLvqMNpvq2x+CExTcsqpMvUnanl/LjaKw/+aluzqcz00rc14CRGmv9C -B3LjyUXTVNhMw9CGLS5QkFgKQfg2kUNLycy5R1/9Y850aqLKRDV+VxYebepNK3QA -fhzf7sjv5PKeomimsuEJvHKvXcF094bOrxYR+t3ZZxf+nH+51ZY/Z1MKuTyHlECi -db/cMnTNY3MJDj7Si7xmowN4hcA2EZ6xFWQKuLua+0i721ifrx+QTSuja7Nd6moq -qqjMExJCRz5dLdlMANVo2ZqZqm0Au4mBTEXHkvdvxzjx+pElvtadc/IQXx3Waty5 -jlQJfJg3HnMUP355GMQ/Z1/g9unUZw+Uj6/KriEcWYWMniSfsjgp+2CCXasjmdmS -IwFVTy47r+w2IeWgeoK4zatbFjBLMS9O4AnszxQmu9avfo+1Kd2WBgs9kKgAkIfk -HdEfaNFwuzUoGnAzpMKDWOwTXqeNw4Jw8IAh6daTnfS46ZIYYTTjEF4urTV2gI0t -5X5pxf1RzP1Fywo2E6KCCnA2TKic1f1Em9+wR7wfzlAVtqEBaCFBC1BB+k+98bZ/ -NKUVQq/4KnFVD1eScqrj877/Z+UI6xvRMBrKA2lDaE0J+QSJGbFQ0zO42OeLdyl8 -Nv28EnnPOgBV3Fz1QVwJOedYep+u9qYvVRdIY03mSuUUpkyqG/vzgkcYQ4DNEZ+9 -NynKZX7mH5PqtCn53YpukT5dIMmVoMzXdqXzzwaHz6uEtapLCUC54nz7PjG4qAlV -dWSq5EqXQRr/C3PcvgVEh1WfD3mtevjWHfIryrEW4BHQ/D00A6IDfWx9gRyj0+00 -9Day2IEYvVUZ5RBf29cTINjESHOYKHY62S1p/aFMSxALHe6KLVAHDog+dGPl8sbX -Bv0Ze7Jvvv19BIlj6LqCqNjrTSI7ymmuWLiUghewjTfeCEE/HdIu0Xw+Y2m9J6iD -3GFFY/IpLUjnA6Yvi0/GAkWxBQhREppY0mcYv5RQCAGY48yeKSj9+xB4+IffwA8w -luUBbEYgPSCzytDxbkapFfSm0pr6n/44WzTQyQlW2WbMOIc9IHOH1A78curgguOw -d1el66q0UmFK2j2dJq/uimGfKjwVhAn9L1rzl0yhzXLcHs1EYTnKnm2W3yMdkq4s -16hVPx08klq7Nwg8TN7h49ZNwPW8axFOTXzk0OQhSmTUdARtZjk24KJfmINsM54H -54FVzzqvJBdcuEr9N3zBva3JqeSHPKTko69sjN1l/WVxI2Rd0aaBzavQfMtAlx8V -gtYzFAMCSm4SFwIg7aOr9pkNembNClMY0xBm8HaIhZNNctPu9f3A57nV/h2vH7YC -cU3pg4RABMhUnI45KuPXUpcrlw/U6mvJYYqnGS7T74e+e0KrJSebwL+JwQkQO9M0 -JFvm+OEYfJB1jnZXFG8itpanb68EoV5OGvCMenhLzBtE8HXGiG3rqiMsWwvAsnsM -sdV5AF8tI3g73Vc6OhLGH6Cy0yazyEV3FmDUEbNyz706LVL5rT/Rl0hp9IOj7cA8 -3mqC25msKAJxSyFj5ZnnjKakch8iJ3beNXvWCkjusV6cB9pS46RhEkz8Vp5bYNaX -8V4nWPiQz88/L6eS4hFadwjow+KtAwA++2rYzW79SDxX9o2wYMuKTotux7eN+3N3 -8GitvyWAGNanaNlkxwwGbrFYomEgcS+bARZSeDqfTD9AkAnnmRHUAK/lCaUWqP4P -VUdm0wXGs/lDD1YJRqR95Kp8btSAfbS9S6Y1EWpIoL0+ZzxtwiUL/od2xlVNO+CS -cC8U7gt9VpGkpBzR/hLEElHUvqgeMtVeOdvLvvaW7eXX5IrLewkYb7cBjaAFrhdA -o/IUQjjuMdzhTyRay9//uSCloojRWqjDDGg068BXfF+lAi5WpGEuL1zivkqm5dwW -fn1N606kmn5Ja8IlMkqKr0xX04h1RH1/W3czUbF7HmHchmUQOkF7QQceKKrkH+EW -Yxr3rxjV2IW/SjueW1g9MM71IR/ZaMwxz7x3S1cMk1JsA9MtP9ZsGJ7aU0kDZLvh -/0jfyQWcP6gsycmpz2PbuaxHKyPgj/FP+egd1DggKrvtwsZlF3V0wC2GkPyyPCgf -yfnfJgANCARo8Bjqx74Jv1KryGVzvPS+uCghVbuELXLvri6VHoAcu28knkYjHUDu -8saxFdJLPL4GGtJLwON4mGSYiuuYooyowpjajhgHSh/Kg5REijEBZjNeEFVQlxWu -dMcmEBQaqs8e2XoeynS4bvgjQpyZ304FxMNUpxo++oQdZcUm5ZlU5Lh2Urggbjwr -PqOsQbZdkoCpssnwbRkYE9OcCw/CuAS4fkpI5RR8/2WMoxV1ki014caGKlZ7VzzT -paNHYlBA//PtGsuaYptb9b4I8rXw2TCC2rgnfg/WeK7ZyAAxZ3BRv+CyqTEP0tRV +GRzRVv6TB5rdE/8OP9HHqoQRQDCbgXgpB2hs6quyq4eNQSQPQ6IfhFtRJh5Utie7 +flNfN46NqSOAdwehnQffRBUayWpuSQFstNzglnFLK2fESisuxmB2ppnkKwm0+Wz2 +SHY/u/okRPOpGbXy+LV3pomqLe6Q5AaH2BwKfpYtYlCGAQpcErFupwwv5YpkuH+Q +lzfhOZj/Cj/8A4Egsumzl+h7Ikg6vgOw8FHCwq0QFkdyGSUnqeZOCtgMChH3FJis +ubjpZDfSLK8IITiQ7VtzgSTeNziH/YxFQveBqRmDH+a/mqisyyOR9D/LgvCQ13cB +/k241OA+NugxvZ+VBgbGq5u3gfjt9qYVrsea/oeUwR9iD+LSIQToJob9varFKtGA +i7v5h+4He5mqs9+bQabk3J0OpMHxnDR+N99uYFlboQK65t1/mlM52LCMM0o4DlJX +dhtfEz+KEuMUscVgl3DOtkUoY1+Y/Mp5FgQ7omP9vQLyTIGVez6hLRXLMPHIk43+ +5dU6VwzHJMz7lz/bisJmLL/Dy3cM7lYea/PF+hbhbUfeXoDi3OkaCD3ZFnnPBVN5 +nKYEb3ucpKGGiqjYywDYkHltvWfZom8q2q0AZU9xpIepkeO3aVFmN/QQib1LiwR8 +FRdOaJO7sOpyS1XgPW1N+0lllfxCsdDIYzHByauTG+qa95En1sFdmH2yKIHWejuI +DTgOrI3jUCt7o7cxUa3FL6xoMwirgOTq0AJSzoF7sFs4VtBknAA0Vg4EvbKtNBUu +Z2gvpKCgUPwAL8CTZwTWRlZg//BgBEJTPlMKObUQz85iQRMxezvQrnD2VmL1EZJx +qjjBgT0rQFCDUZydM5hXRlEoHJ4lkJ+LhDggcUDhCzehSMYbQ7kBWlfRLZpBXM4o +U/q7ZLZkRphk8GJSOTwv41xkBzq9gDX2bw5eCR70Lh/oBdW+q+06zyCSu9Q/rYZj +8rXJRpqFozr/5A2tZkzg8Eqib255NkSyXY1FhYhhuNkphNoStuhajPudCK7T3Qd9 +qvIiLD1iyXaA7pr/ZInCGXRgE18Whn6A4mde8eKnnJ3il5wDSAeEb2C1Oplu0eto +f3J8l+OmM8f/J1p+2XeE4HMyopsoFTHubvKbvO93u5HQn2+jZJPlsg8nPL06pBqd +FZzArkIWZPCMJWb9z2+xiOzPsptGoT7uesTKv244DN3f6SWIgX1Ye9kRn/vCp+Dd +P85l9VxDcZ1Av3u2WQxMBw7OiMOzHToe+rLUEjbGR2juDArfcyeEfV5aTwXDmdMX +QDAtiBV4Xezv0kg55Y0e85y815BUc+0oOTWTWiP5zgGQGcWKZgPtddUc6YZcwdgC +4AAbdCcclXSImDsn/znsUXs4BYCJNnVr9FBQ8axBNxBUUjFMDr96lCspnOVEVq3E +9aW3JxS4X354W0JSKa8AVjxq0P8XUC1tCcYTf0g4grJoDp+z4c3WgQlrOWzwtj6/ +D/nGIAOUXSt9c4Kn677Q/JPfxuV662sSiDgtMA6tfDkHkjcdvkxmXYiUkjHeU13K +dxWU7LHDA0hhPL3Wuf8mDJb5YNnV5T/MdPJdOGcBjnhhu1pi46ej6xZ+KYHSnfbG +Tp9XU3WV+ltmWEd11xRnrBRa3QkfoIzD8LLxywXECYVk/dWgVJEDY3MzjzZZeNPg +FjiQ2kByEVI/NkG5G7shwqR4MdqLdwB8gRgMpmV9psMnrafKYeKDXW/Dw2FhJ2+k +QqALnkC27aYb8hbW2uJZwCzTCadu6GrHwsZ6RZfl/sq5ptapbbB2QnkvWU7EJnTz +CAjssY6MQ20/8/Y5x6AqT8dnnH/nhSbd3GfA0JSxAeQPD2dw1kXwXNUYZdjfWu86 +ScITEy2HUwnAtlqXXmjxUB1N9eE8nO3tiYPrYzOWMZp5eBzWCE1a6l9W1HsfXvKo +nnOLmEg9SkkLtEqgmtuhOZYAdZIA3T9Xjc9XCHpWpq1fwa+EgXDRhIgJbL9URj72 +pG7rUYSpY96a8h5udOiN2p2NdX+YLG7hfcKciEUk9uvXi/U0965A6xv4G1C9JbBu +X7ROEN7MS3jKZT6Qn4J/h++0h1HYACDZNnuYp7hDSVyQVlCDqa1Wo3Yv5S9hDjEj +NZ8vDHE7adIs5SFgyKDgPPHg/qUZQr11GaT5rvwHhtO1MGF0+q4VPOOqpZpY6iIi +Kz21GQY0E9L2XCndlD9lkRDebQUnJB0PHGCnSqFKHZTLG9idRgB+C2ZAqzATuwS5 +wVxEgvXqQTH2gsCfPfeRZZiduN3ghZQ0ggVCyINIczE2FtmQ6erxGfFtM8kkDRSI +JQkEXvCqz4A2VOYFEZFfF0JP286R2HSMxmWVg+qYKo59et6EQBjPtFrhlQXhWEQ+ +pe3ks+sQO3YvQxRQn03+MHhpMG5QI0zUijnNv3p4PRArDJKMBEq2/03u3AQdaseR +WN8jzYd7d1KjcHCjshppX/FpoDrVu4l+TNvmmKaAoSdN5Dm5GvVhuoY1zhmFm4jo +fJQOgvG76nH2rCKP2dAegYZTkyBzC+/8GoJC6KQeWPKT0uiBqgaP0qzuKsgcABlY +Ydj1/R1k6Z1pogpi8IUUenJvAQADp6JabjqbrFRRfB3fAZ8OGIsk/s6W8t3IlYL+ +srcCrElFmazyTy7y95sBc9oi9wAwBS/BURz47AZnytraxWpx7xS6u1Z9Wt9rfSQw +RFfZ4ZdELZJfMvfXcQdFLZ3LG4fSgEJye+WgWBHTBjnPL6rm3NJ6J1viRvP4tbqv +cFl4Tgr+GhkWpmp01K+kqjzwPWU7CvsYM2Z+bXd7C0aC1OttAly1/s9tp1k0PESz +4y+4ImHwzceslIZiUDsKFOW1+mldM7akSg656LFmURdDcwOEZvnb97YZ9A3Ch4ol +0+2v4cziNNtXx5fOyrNNGUj9bM2+Kv+yvy09uLE+aoS9p+Swx5HeJUVK7aMNGZoH +LjCa1qdcpLtAaSBaagbJ9gTwgbGTZ7dAmJW9kGmN5ImOI5GKSsBf7feqI4J+Mq2P +GTUfTTrBw1/ttCRcnvWlO7aQPZLkest5uq88K3hk1JcGu08pNIeDDWRwLK0cRMxq +1cvlHYStF5YFG/bV/cROcm1uksEhN1mQu5Z4A+mSMz3pF+ZY2oRoZTBW/KngRpPf -----END RSA PRIVATE KEY----- diff --git a/fixtures/ca/server.key.insecure b/fixtures/ca/server.key.insecure index 72db8bd18fe..f9804f4fadd 100644 --- a/fixtures/ca/server.key.insecure +++ b/fixtures/ca/server.key.insecure @@ -1,51 +1,51 @@ -----BEGIN RSA PRIVATE KEY----- -MIIJKgIBAAKCAgEAx5i7k1vLi4bizkcW6TLp6It/tsyJOXhwOiZvqR4oxoe5Jg3T -MSvyN3DivxMNKC6/kw4TfFZIu9DAuCQ5BPQ3uikN4SqbliPjnz4z1nmWphRKyFam -cU1TK7wYEJ8BEHzRgLLx0ttink5EnYBbNLl/G2nhNcR/yqWsAyh4nPZlcy8YcxS6 -/xMVrco8BqNDmfYM9K6IV69oOK2JFqbPppPjxuc6jAw7QSd6TxSR+lN5Hu6HS849 -tY5QiPO6qvsbgycmRgPJ3tHcqxxk3iJd/o1EyPZ/pXg8ZjRaWNRuWL99PtsLwhsG -AuRDFPyY7t1wn1Acq14mB2O7/162jfMLeQMLdiHZFrF/EzW+2aBmbdBPmjP3lCOO -yTNbWDPaCITjnCiNs6CdMj5BG3DYQwFfqbNo/9hTq8K88KqEn3OtUywzzwQrRNN2 -QT81YLbklSaw9dfftnmMVcgVqYzQVYT/OEOKnxd+XfKoAz41VDnGeZYYymrIef8D -URpG/QBJVMyyDjXc3eOvzXmorL/FIJvJEgWGsGYqTDLRtJlU/71jSwLz5sFNk11s -3Ct2nQi3sEf/EqYbQxpM2kCBV2RuedK3kOWpL3VJu1rmL6QeJuQWaXgqdno68mBC -5NsL5SxukPkHW8zxF3brjE5CYf0oU2Ci4OGIE63AAvduf7vo923w6cxs7QkCAwEA -AQKCAgEAlrAifUACTdaKCP32uBxuJ9iZlSKaPz9ES0KVbnKMikYRbD9nwGnTNjQN -nAAAIOQaUiWAZJCn3NPfi6YdPjY6lFtGVUZbrGBwCttlO3kUWVJcmx+ADW45an1Z -FcNVhGMXsDhpBa4HqEii2N38/bNF2SZ4lqVBbXbihIfbd3U2Zl2Z8dgmzUhVR520 -77X25ZezdE8INFsDLjcllmpdvv9MKfCMbQsW+TuaxXVcOEco3Eds8bhFMnq9JogL -1+Y4gS3fYWCe2ZBLLwwCwwnjOLjiw24GNKCvyOGhjArlPC9lmTy6hdLtGOXIF2IU -+9FGo6BR5LbxswaC+mtBG63wbW31CedPjm/3xei8gCGvTsdvPYMbh9gDd4MPTfkU -a1zAhTvDWdw46ld7bLbvjNb9h+uvbyEbd7cpsrFxFsqSjLWf5iAgxiZtfDYZaNzz -GQuit+Q1uN68ULThfrWZDYfnGOX5RR+A8D4pZgcGtTF/ephRzxEIKhpzIrhACKHr -+X1mU7cYbuaVgs278TdP45ZId/02Jfe1hi1vhEHs7upnX0pifBoFWaOg4Qvn8EL0 -b0LvD8HHOqqW33tw9ZB23UXRyg4xdk3aO93Vc6uVBnPMut4SESt7/BJl/NbwAFIo -87hTk7w9a4s1EJNSrlOOiddxc4XP99vFz9ia9+nRsln9R7fxkgECggEBAOjp21OI -7IBi1PvPDJluQmU+BF2ZTCiJ0OwqoRX05Lbv/dqAXyFxC5s1Qw986/Mrvh8FZZ1B -mLj/xzgT3titpkV7BTKb1jEI34SeJW22mH9X0nr5VYEwS+qBXbG4XyTU8skiks7Y -XpsCpAQKCgjChzRJTVZ55+/TDvJ6RpbmKuYrSA+pLe2cIfsDNBNnuIuxLiVh8et/ -C/W6xDlSDBWv4t+oZvai67KA6uss3wSbop21Gr7n5t4X4798MjUl3Sl4mvoTKrhd -X8oZ0t0FuZKNdym62Wjyp/12YRlPFhX72ksaZy/N7g5X3cWnQmKKpDPP+MF6NGtX -Y8wM/A9iP0JrxSkCggEBANthdxPg/XgI7eWhsb/Qz/JGeekFimtnHUCIW85yCH+d -Kd7kYjFYLcpLPUERIxfhT2v+FTURqOkwnEcuqpnOBGcGj1/ZRs7wYKuwVpVZ1omF -Ob8H+mdGHzrPBOZ4FxwHVYYCjaq4Y93worxW/lXWn2t7kUTjnnDQyeCgS7ja8EUg -qTxiN4MIMOOxQkvdW+N/QHiqJbdKSe0pFS4K7LvQR73ALv5qdwN5MpzXFNGHABAP -1QEpcFXWNvdoXM4DXZPVvg2KuhwGG8URorhaF7RKKk1xFbb9X+jZRLExZKHLO+L1 -lB4XyxiWSQkBh6ybickc8CfmVny4HzFIFkFTdMugBOECggEBANYhij72d1hhhKYs -6Mx2jhw+NA1JTrdGXQmC964UA+IcKiqkMtGv+JetFAY9Nz/NS3GBqLY3BI2wuhtY -SVyz6VWfkFvC4d0a50QpkQeZBAKvXxcn+/BV0rW6UcV+WBqonL1GR+pbCj9A4kHE -aQ08qsjrS2rhkNbwF6HdwOAio/YQfKPJSixVivgXLd1ZUlU7g81iiuOTXg+Asb5x -LCMUHWS6kk7V4hOuakvkaPT3kT2krv4sfhhZpkz5hb9PHFFwTCr1TCVL0zEfJLmG -9eFCpfd7jT3rOX7RQtvd1dRIQ50gnRVaIi6VoZKB/4pRJD1uSqi2DVNSeLG0jlgm -XzpVkmECggEAWpCW5vb3zIjrJOQmjAg5AEyF4WOvK/2KfuyL8eLzjTMlaOWhf7tm -U9/Rrr3TXfVeozdmK91ZfMLbkSs7tHjvKlTz9V6uM5naXqZSaB+JSIZeO3Wgsueo -1s9Ft5sV9zUz4jnFoBe06pd/pv7GykrqzyVY6DaLXwlifb/O4sZHcFI2az4kqoxE -Gos/0i/U0krjI60iGtOpRyWxn6tU5YfrRfNDszXiYeWztjm4V3NC3F6c0Xj47gab -9HD59vY+uFwBtHdzs0P2TNml8jMHHB+N7SBlFYDuCiM/j2LTp1NOKri05+NsrX7F -MdmW1/Px4rt2jRIy3BPqlJ6syVUZn0I1IQKCAQEAsLejmPoaWxfDxLxFsdH/bozg -89DFvWRVjAoU1EWmVQ6d1DdS2TkG0iLgomoyg8hI6SmJ1cxPZBA9aFSTnomZizxL -CfeFV0lIyDwHD/d4gtuG2+los9Y5dwLKI2HT0eFumyMIWdtPRelmzS9rJGXEJjKU -YKbDyne6Fn/2HfrUFwVVe1RTg5vN/9nOKQoxP9i9N25GuklHSPRWb0KUSj+qV1rP -SHI2wixMm789GLuWZQaH2LntS0tu9IitrTGccXa+CJev06Pw2tQjWrFBAOdp3cZl -aJXR3MR1b6bQyOUpoL2hcg/jZhQLup/Y7RPNGNqvV5PMpL7YArpBMd6uhWH1Sg== +MIIJKAIBAAKCAgEA1XBtjDav5Sl3H+/fUcGiQO36oqtZG+YuC9D0z0u89Shq+XNs +3tRtonGGCyEIrDtI6R7PItMJa6rQ1VuFoMWPrjmFf5tFemSOZtQx/DF78H+5tWaI +BVDA+Kw1zxdqj1n3/AQjAGsSuqhgcaIQQFqTNNtAtW40048fDh17jWIDB9baF65a +z2uArq97uS4deDujG3CHV9svO7hoqpzYt039VDKK4N+dDMUZFqhEmY2MqjyQySY2 +bd/gsYBjcGWSIuonALactiYDc4zIusAfNptnXycwK/aQAqDwhwMcQA9L5YKQ5hoU +ukDTQFbvoNLJ3vNQDI/o8sjCh94EkMuopSp90tJ/syTPKRlh/MaGMXvwu8Vab5iP +eVop48jTKl3Z+G8NErGM8KKCyd1mQoGisVuIMQPKuJUi7jOu6wgXlA8ZgUGfSQQD +A4v2Q0tV/GlJmvsP5JshA3v7C/sSBY/3AnPHeWTlozXlNgXitxps1EwgR2jo+YW2 +gxrfM//xtgMCjMXjO9g2TsCnWR6j93oXWn88UP/CeAcyjeTdJjW+piuLdvYOctY6 ++Yql5gm9Vx0u+w9jTmpzOCoh+9cNtjqmPiOhecFcVf71vMf4krMp6lmY/Nq1/km9 +u0ViNP5CJHk18YXG42vnj7sUgT7WgpLh8g/iPc2petH9lMd9te+Jyak/zA8CAwEA +AQKCAgBR1Tw7IRCJdT92IDroFqyF5nhM/BM7LiKDZ0clX22ANVHmeEnKmXm7aXky +NSUlG8nVj3ltaapX/HL7Co8OWBDBhM5ZYYfe6ETsyfisL7DMQbxK/5exKggCj8xF +rT2u3pjEqDVfSK4yoLHxf2hptBBymImTxkA8yMfoWodvap+s1sRhhfjNQ/NfhmqS +Ukr8OSlNMPTDS4cth4OhvmccyKsTKBm1JCcLqVn4JOXAVdQTxQriBGOj9s0oYQg/ +JMJF3q67iEhHUgXKvLSNXXHaNvUIN3cxs+P9DgWKTjf7m6HGyiuR/Xfq/UXBilNv +vsGlWHZdiqOOykhDXW00stDjGoqIoVhkz4dxabY7il2BPW0MujdYNELVOJ/ikJ9v +0M+3jgyyS8UDhgBAz6LFYXRkAyREbRi6NwWGT4vWtrfycEd7tO02OZ9v7w9Xb6el +iU/AvNlb5MqfXaMp9gWJR2tDka66cMBhH+9VtMDd2J6Qpfjzwnn6VtEqUvt7Q+KF +LyPb0CtOTzxYuPdCc5ZpIiIiv6iudUXsDOSl9CsXMTGTD0W8g9RQ+GKo703fZWlu +LU9gOWVXq9c4VaR2dpLNjvOPzYxkChCHI5rNtcsnU6x7XumL/n1iJq+3/SMQ5K6t +QSqTXAuMDZkHGhLB38zrQBb4Zg9LsojzEC1AvCrhqvN3nx+UuQKCAQEA/TySUNIr +39j+oQtw0m6icqWDOMTKn6GamDOxq9+9gsNfY1mQt+W2XLYfPgScNXXXaggYxtXU +i4dhEc+f1a8crBSFPPIgVx/ggawTO1jBZAGrrO/ppcePvf8bGmj45ybmiB3NjzhO +Eg50neZ9/ciz0cmx76Flb/2zFto9aj1RhxKpjAQHq8aNdBCanPVuZC7ov2PgBpPI +UU8FZB/pYjVjha+on4+fUG2S3urEwJR6uA9zwSrNsLmJ4VrcqBGN2huAkrsqzUXP +vysrn82Y+sR4q5fmXb3Dn5pFZ3mSuRclnp0ESEFA//U95JTvG1aBtK4UGwzI78ia +FVIqyRqyOZ52xQKCAQEA18SuK51fglgLvdq62GlzteDcZy2GIk8JRh6/rJe3X/E4 +4PgoWYEz++N3SMZsXWaz76KpGaAhirV6l60uy0/iG/yoV6P9amokFELdcWoj9+I2 +OAVPbJrRjZOKFDDA/TwgZIKd7amUArfwwvZLAxCzG77HSaz6/DcxwRriKR32GPQ5 +ty0sfK+xtU4li9kT2iOBHHjyjBcTfN2SjBupwaLQ4jHu37dpfWIEqfk9hEVaTPCh +UejbzKsbEvTg6R5YMDa5JPcf9x2Wxk4ZbC8FPWdCMcXYtprH8pwoHa4hqTgHBN+Z +LNQTgXKqItWPTMWuG14J/l2C71YcwtzCcYUaPfNEwwKCAQAnFqZvG0Hyd4g2S5HK +qZEhqTKsHJQ6N7OpMrGGGi8idA3RRA32lNqlTOddp1CFX/80OrO4XWFFeEwfd7Dw +RutiFHjMg4NCb4Uz/t+pFXYkfa2GMDIciMVDSpFgbjudUn/bGt6T8Nj8KIcPqHhi +KAy5oSx6FKuXsc1nBaDdOUHQW60YE7craKaE99slxyyXAjai9EOsQDt3cX8fiV14 +70zBYe/hUUYCICe/iPV91G1s49W2R2kgkkMaKfBNcQg4Vm5uN73PmasLkxpUvGOU +sab+tZ+1cIk1pZZ49mcTcuM3rHzwukHSQIShN+wAiEXVIdmwozSQ7qH6EIjSKfDA +vBkRAoIBAHRX/zpRT1CvPRWQPbO3mMb3iqCv8WXKjEudBOmBnUVEgtD7vnYUrv0h +eA5rv77VRCzw3pGMwMlUddgXb+X9GwTQRc2MBXc96Fpse49OFjrxZR7r7hm3mUrn +xUqBx25E34qSy6l9COw2VsIpn+T1Oj65rifR+DvLXy6q2kwlda+a8QwOdbB95CrJ +CoHP+V5kSpgZt19GiiGIMB8QQ4a/zjZJim5jLaSIF8+3Ly6FXt2h2rqZ/vrrQFwG +YsgQrqjAuTBveHL9J3GiZx7oc8DaTt0bu3ErIKl2/kKSxF/EcDR2hNehOytPsuG5 +md1hsjHbkTPxJEr9eeCwvMANb0r8Q5UCggEBAOhGwGdM24ULcSLGONg2Qz03bdDP +alDteEfq79X3mEuxn2x7N+tu31sBKCgIlrrTlsG6vWgLebFelAF3Vm/T3Hc6ifUL +yDf7/gkvM+X/+f4XYt2Z5r0u+bVOsAITydDQJhakGfar/0it6URknXr9NJca8tRD +DATfrRA5jHCbu+sEoDLb6m2lGXIFnkeD0rc7tz27inues1sxyTLy6lENAB4c6K2n +oFtigO7F22miqUiVOXAAllJo6yVfwtmbtDkEYTjLZ2oQwpEdyff70f/iDm2+uKRO +eF8kNuKxR2Em03slcfOV6M145fN35Eq5IuOI5N/+SiQhvywbjh6j5DRHlTg= -----END RSA PRIVATE KEY----- diff --git a/fixtures/ca/server.pem b/fixtures/ca/server.pem deleted file mode 100644 index 37e41f5dfd4..00000000000 --- a/fixtures/ca/server.pem +++ /dev/null @@ -1,14 +0,0 @@ ------BEGIN PUBLIC KEY----- -MIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKCAgEAmmCdKFi6X1mpk5AU6EEH -exou9NTRyiQVaHmTQvyPu6rd9krXB47/TgBDXcGIstkhFkGLwAh3cHDEPEF2jEcw -W27S+/MvQdVgC4SJaN83pmk6ZYOvr0AX6zmPschoLxl84AT9xKHhFJuH5X1eCzP5 -DY1rAvNLdB9lFC/DM8m2AySwKHc1kAPhs//j6RPcI8R37yDOEta7e/ikhbAwnOFv -/rs3Aob/nYE0ql2CMpO68uU9vbDYQt2bFdiX/zau402Zi9kU1lAaeNBNM0UP9thU -/SSOYuDFqy+XbRVvItLhjvo5hP46GOw9GLz9ICQQohiXjC33e4Hs8sq2XkM+jYyk -DGRiPEtVwt95x9h/ReCYJowJzJWnaSvQKEPNQaMvwGCV5ZLZ0IlI8cqS3m+ns4ZK -gTQDQjeRJADz0JY3jBpBhLebH2HfrYJGp3EWC7CdhhTvYXN5ZkBK6A7xkzPY0mZj -RAvS5K417LkAc+G0gO6qyJtXplkL4G/Q07Vdt8zc7ZAg5rbGWY83lw8E/7h0Gpu2 -JXfANZzdPKiV5P2tB6ZEwdxTABY/kHEk0u0WoPjqqgNv9I/zwLCbjefon8RcIJ5D -EXXm9DibcaCpRYUkq5redFXDG8VHVzYVce2CJdrko8GvWUIOsAh0Y4CbyrXgXepV -BvDtjEvMJUJ/iI33Ytzi6w0CAwEAAQ== ------END PUBLIC KEY----- diff --git a/fixtures/ca/server.pub b/fixtures/ca/server.pub deleted file mode 100644 index 33361caf5df..00000000000 --- a/fixtures/ca/server.pub +++ /dev/null @@ -1,14 +0,0 @@ ------BEGIN PUBLIC KEY----- -MIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKCAgEAtG+XAmj6VT5CQCiX6wUQ -DJb6pM3F1MqJKp4eGKClvK6jXz8gu9clFHPhKWvHWyouN/id7TQ3TBPLeX4MaL2Q -jf0GtWuPwuD2ZOV8lDL8klINUgyWL+WeRt+M2WwLeaJEuZhjGootkkfmicSshH5c -JMnejiRWTpAzLehHQs9ZbP9yzg+tc1WiwvqOXi1NM7KpYALh20MaUMSVIW6Lvwq5 -JAIo772x6lW9IVcTxajfYtlZDW9qGD/4n95qBvDqtqSX665UmB+Kfskf5sWyIEKQ -4A3i/RAXclhyEwgF5fBDMSr8ek3vcSzbA8a7KCUHQ4aTGm6dnQHtsAlI5Aww0vs2 -QYi7oc2o5G3YXNfKmJdogxQJVwi+9/ZWoYGlQt77h90u9twej8XozOabF8M0zC8F -vU/7Wc84pzvykpY7aao36E653FNt3MCJhRG2MinqjITO+Xcd2B/dRRWYFB7qM/D0 -KbXtzvLnyJw91ERGjEJJ2E5/BgW6Wz996c/AGExnnf5xFgZ4/Z/dNPpwzNwRSTPZ -khlrhdmnQkKjHXJDHLy+uZZ8uj2yCGoTaJEZapEp7GGvsgwi/1DbAvmpwZpfXADW -A4x+m6rJJvrOnwui5URcAgj1F+JEW6d2piMmO1mDayiyNEkRpv3jB/Au713SUpm0 -0GQCHrmrTb5izgCH3ggS75kCAwEAAQ== ------END PUBLIC KEY----- diff --git a/fixtures/ca/server.pub.sig b/fixtures/ca/server.pub.sig deleted file mode 100644 index 24283ce8afd9d8e102e2c14cb62477dd0393290b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 512 zcmV+b0{{I#vaBJH6F^Scg`R$e`9YDG8#{+soOi`sfK^l%%j95`h(*_(QJ``Us#a;5 zh-MX1XHc2CgZGZ|k=^>a$*M4aH|9qS_0Cas-?%7XTK@J3|1x&*AMRvVK|2i0zm#1G z3V4od&7SRqVOqFY5O=j^9TIBq&T77KzP2p~m-H zf;fBsFf=vy(So)4rVaotLcBfQHypSMBqDgz8W@o;R?J_{lct!J1=fO!+V-g%QwLd> z>l~SEiEXh)i{{(0?L7W8_C{l0iD~`EO8wQU0#1-!>BbGkbYLc0k7hHMNTlaed>yOf zGLa$;$me#tEXN;N!^n8ShCy27rT_XJ`ApwLH*uRNYoT|`-u6+l9^n2m%mB!72%HMC zL7XRZo09eNvooJNFkHzJ_gEEU!KFx^%@wT)>6*dwqkL;15g#^93lbzuCJ=(jSzve0 zU+Cnbw|ifEmDj3es(I!)0)Z4j79w#pWvSydVqD9eYYk;xN26&lYN}lpTLro`AQUN6 z=b_Cqq2Q?QeT1k_dH=u~Tb$3cRoQogQK3*^E+?JD*=n~qrVHTShr!T1^DSg60Pt?A zo7Pe+8%7mQ%7l&*=k0z;Gj*&jo}y-_yf6cOP2hS{I9#`*UU*zt!TN-f>tiSqQ*vIH C4)<&T diff --git a/fixtures/ca/server2.crt b/fixtures/ca/server2.crt index c68687389ee..b30b42f5ef2 100644 --- a/fixtures/ca/server2.crt +++ b/fixtures/ca/server2.crt @@ -1,31 +1,31 @@ -----BEGIN CERTIFICATE----- MIIFXDCCA0SgAwIBAgIBAjANBgkqhkiG9w0BAQUFADBFMQswCQYDVQQGEwJBVTET MBEGA1UECAwKU29tZS1TdGF0ZTEhMB8GA1UECgwYSW50ZXJuZXQgV2lkZ2l0cyBQ -dHkgTHRkMB4XDTEzMDgxMTE1MzMxMVoXDTE0MDgxMTE1MzMxMVowRTELMAkGA1UE +dHkgTHRkMB4XDTEzMTExMzE5MTg1MFoXDTE0MTExMzE5MTg1MFowRTELMAkGA1UE BhMCQVUxEzARBgNVBAgMClNvbWUtU3RhdGUxITAfBgNVBAoMGEludGVybmV0IFdp -ZGdpdHMgUHR5IEx0ZDCCAiIwDQYJKoZIhvcNAQEBBQADggIPADCCAgoCggIBALRv -lwJo+lU+QkAol+sFEAyW+qTNxdTKiSqeHhigpbyuo18/ILvXJRRz4Slrx1sqLjf4 -ne00N0wTy3l+DGi9kI39BrVrj8Lg9mTlfJQy/JJSDVIMli/lnkbfjNlsC3miRLmY -YxqKLZJH5onErIR+XCTJ3o4kVk6QMy3oR0LPWWz/cs4PrXNVosL6jl4tTTOyqWAC -4dtDGlDElSFui78KuSQCKO+9sepVvSFXE8Wo32LZWQ1vahg/+J/eagbw6rakl+uu -VJgfin7JH+bFsiBCkOAN4v0QF3JYchMIBeXwQzEq/HpN73Es2wPGuyglB0OGkxpu -nZ0B7bAJSOQMMNL7NkGIu6HNqORt2FzXypiXaIMUCVcIvvf2VqGBpULe+4fdLvbc -Ho/F6MzmmxfDNMwvBb1P+1nPOKc78pKWO2mqN+hOudxTbdzAiYURtjIp6oyEzvl3 -Hdgf3UUVmBQe6jPw9Cm17c7y58icPdRERoxCSdhOfwYFuls/fenPwBhMZ53+cRYG -eP2f3TT6cMzcEUkz2ZIZa4XZp0JCox1yQxy8vrmWfLo9sghqE2iRGWqRKexhr7IM -Iv9Q2wL5qcGaX1wA1gOMfpuqySb6zp8LouVEXAII9RfiRFundqYjJjtZg2sosjRJ -Eab94wfwLu9d0lKZtNBkAh65q02+Ys4Ah94IEu+ZAgMBAAGjVzBVMAkGA1UdEwQC +ZGdpdHMgUHR5IEx0ZDCCAiIwDQYJKoZIhvcNAQEBBQADggIPADCCAgoCggIBALY5 +FfOgbclrkfZS/XpzPJgWZrs/W+zakuoBDtkeTeIdVMk2lNZ//oA4g+eYVPnf0DSJ +oEPvJIOuwjF8b2M3iYR+fwV4iI2NKVEiV5469qtjkSm7OlvNRMAeWztGm8uXI9wg +9MEqVU4SPlpTV7kUE2c/d3jU6ZAjvgBn6sWnRWG39lBGXecqay6PzbAYhlrGH/Ou +PZaL4mc6nE1D7/mdBOo3J+6dNQUzB9FhU/BY55M5c0tSrDGlM9pPE9/p3JXu1dcp +95YtZbUCib/NPVnHGEK5s8LfG6RE8kxxUj6bPjPmyEF+jsgwJb2vdHjOXbwbtsji +WprZ6TrAFfLQWmD9qbsVxvyMlnuzrfPyaREWhgijBADDGBmcL8k/OFH3LqhmwOET +LpGepCV5BbRqD3+tDIQcjjlt1fdJTDu1RuPs921EdHXaQNzTTPbO9DoNpFY0l6Ul +Hjlz8VlnwuEoehd3NDZc7a/b8X6Ry0JwuaymouUJE9GQrGJRJ3imZkwrhhqxiWIt +p6HJKgePD38V1tFbifqjiF/LfhqaZ/oaFU5f3wLbs41BuMOQtEoNsmRCNd2L6BJ+ ++fskqnJfbuw6YRoB7gvoFjyqn5CZuox8ArmOs/oYk2nior2jBEQI+S0JwVquBDvb +vyUnYT+E/mSURUA6kPtHax0kaDZnQijXGc4KwASpAgMBAAGjVzBVMAkGA1UdEwQC MAAwCwYDVR0PBAQDAgXgMCoGA1UdJQEB/wQgMB4GCCsGAQUFBwMIBggrBgEFBQcD AQYIKwYBBQUHAwIwDwYDVR0RBAgwBocEfwAAATANBgkqhkiG9w0BAQUFAAOCAgEA -HpP6XxoL3470/1+9OD2uTZGLSjv5lvC9jaAthRQIuSmyhD10XG2uWl7EULBgPfX2 -QYyCWAu3Upuji7KEoxzoCRmD0WOUL8YaNhJSNPe4+acYg1vJYdWSO47jaIy/l69s -LOry4DCbefw497DxHmbJB0NtxLVFinmkzy+earnm0+Fp1qnTmMSPoupQatGHXgMQ -uHVLnTz3oC4OZVN+yyo629j3SaZKWlZ4MOS+RIAaKMCpT8i+4xBRrBJ1XkZVjaH3 -0PPWBKVT7dLxmybm/hZO7PTJdcbEEzl58lgAj1uw9biLG56/0FJ8ZTDbtEqRoxnO -7EEdEhcSdoghoQEm9LkOX7l4wxM7j+MD8mAn0kyEeh8iFeq/zxYhV4IbRchqPs0k -5GZ9DeHBzbgxSENO0KRQLLTLWjWfeY6XplISCfTp34LY/gXnxU732EhezV3FehXh -AVpDOO0LynBTOKvHaZa+/y7fkynHNr2OgxB4CITNoXoNrJd2PSc/wV8HNwDMVdpf -PdTgx3+UW/g4E9KN4GL8IqTLLWRydHez2mLEL4Wp2DIlKs8WZ9ZMlL49sitarEbP -McgOYeUpzvx4UFwJ/ilQEOQODb7BOrX51Cf8k6dBhBze7sMUxIecu1P8amjDlzoS -nIF5foVRbt8G/nfdV3ygegcLj6nz9ynO5Xp2QUu+E+c= +qZuN6qS3vIXPCOcRAC/3mxLtccfZOg57NKb6NQ1lzkPXdMDp6Bd3TfWyrkj0lm1w +Ya9fb/G0yVGAI11WW4ifML3SF3sKwd5PwiqCZa2kTTxL2Wes7t01jueZ0J7vDecf +n++PNkCZjQ4BoSbnCABNyuHoW3GMdZAnlaVdkZHetRN7gwsNvfiCWFHU/slhjN8K +KgvQi08BWhiIFqUmBhqpDJDSgd8lJi2gBjq4idkYkW1xcpO4Iz+dZRloPB/ZICnl +dt+Err9MAec6XpKdniJFLil2FALMmATEZXnGZMuXezXJEZKYhx9ZHuZZDuNBhD6Y +VbOTrpq9F7oWdzqhUCa4y3HKKPt0ZuvP/0nHnGBe/eOPrJANQYpCHw02AXom67eJ +9PW0PS2YC7O4cy+Y/DOI9FkzRpa+Z6OMSqKu9hkBCLGVad1cb8cLyl0MJpUu2UAV +55ovFT6owhR/ZSIDEX4VeNwX0PsHIXIUDFE5Xp58tB9QbRp9ZA4c4OAQ2eGQ8Grz +v2IxJJgmjBIuAZqdW6KDKy2vc1SQZw3uaFOYUEk3f5uAoTiARIVF2qrVGPDQjGbC +fumWNsjU0ale1EstgS5KdQc2Ox3wHsc84up6QaMfPqKmvSDOd+RgGzRqh5G07bni +BWHaIF0K+tJNnPYiTX7nbP9dP3SAfvsPylUFVj9fiSI= -----END CERTIFICATE----- diff --git a/fixtures/ca/server2.csr b/fixtures/ca/server2.csr index 51bb281106e..659090fe2fe 100644 --- a/fixtures/ca/server2.csr +++ b/fixtures/ca/server2.csr @@ -1,29 +1,29 @@ -----BEGIN CERTIFICATE REQUEST----- MIIE8DCCAtgCAQAwRTELMAkGA1UEBhMCQVUxEzARBgNVBAgMClNvbWUtU3RhdGUx ITAfBgNVBAoMGEludGVybmV0IFdpZGdpdHMgUHR5IEx0ZDCCAiIwDQYJKoZIhvcN -AQEBBQADggIPADCCAgoCggIBALRvlwJo+lU+QkAol+sFEAyW+qTNxdTKiSqeHhig -pbyuo18/ILvXJRRz4Slrx1sqLjf4ne00N0wTy3l+DGi9kI39BrVrj8Lg9mTlfJQy -/JJSDVIMli/lnkbfjNlsC3miRLmYYxqKLZJH5onErIR+XCTJ3o4kVk6QMy3oR0LP -WWz/cs4PrXNVosL6jl4tTTOyqWAC4dtDGlDElSFui78KuSQCKO+9sepVvSFXE8Wo -32LZWQ1vahg/+J/eagbw6rakl+uuVJgfin7JH+bFsiBCkOAN4v0QF3JYchMIBeXw -QzEq/HpN73Es2wPGuyglB0OGkxpunZ0B7bAJSOQMMNL7NkGIu6HNqORt2FzXypiX -aIMUCVcIvvf2VqGBpULe+4fdLvbcHo/F6MzmmxfDNMwvBb1P+1nPOKc78pKWO2mq -N+hOudxTbdzAiYURtjIp6oyEzvl3Hdgf3UUVmBQe6jPw9Cm17c7y58icPdRERoxC -SdhOfwYFuls/fenPwBhMZ53+cRYGeP2f3TT6cMzcEUkz2ZIZa4XZp0JCox1yQxy8 -vrmWfLo9sghqE2iRGWqRKexhr7IMIv9Q2wL5qcGaX1wA1gOMfpuqySb6zp8LouVE -XAII9RfiRFundqYjJjtZg2sosjRJEab94wfwLu9d0lKZtNBkAh65q02+Ys4Ah94I -Eu+ZAgMBAAGgZjBkBgkqhkiG9w0BCQ4xVzBVMAkGA1UdEwQCMAAwCwYDVR0PBAQD +AQEBBQADggIPADCCAgoCggIBALY5FfOgbclrkfZS/XpzPJgWZrs/W+zakuoBDtke +TeIdVMk2lNZ//oA4g+eYVPnf0DSJoEPvJIOuwjF8b2M3iYR+fwV4iI2NKVEiV546 +9qtjkSm7OlvNRMAeWztGm8uXI9wg9MEqVU4SPlpTV7kUE2c/d3jU6ZAjvgBn6sWn +RWG39lBGXecqay6PzbAYhlrGH/OuPZaL4mc6nE1D7/mdBOo3J+6dNQUzB9FhU/BY +55M5c0tSrDGlM9pPE9/p3JXu1dcp95YtZbUCib/NPVnHGEK5s8LfG6RE8kxxUj6b +PjPmyEF+jsgwJb2vdHjOXbwbtsjiWprZ6TrAFfLQWmD9qbsVxvyMlnuzrfPyaREW +hgijBADDGBmcL8k/OFH3LqhmwOETLpGepCV5BbRqD3+tDIQcjjlt1fdJTDu1RuPs +921EdHXaQNzTTPbO9DoNpFY0l6UlHjlz8VlnwuEoehd3NDZc7a/b8X6Ry0Jwuaym +ouUJE9GQrGJRJ3imZkwrhhqxiWItp6HJKgePD38V1tFbifqjiF/LfhqaZ/oaFU5f +3wLbs41BuMOQtEoNsmRCNd2L6BJ++fskqnJfbuw6YRoB7gvoFjyqn5CZuox8ArmO +s/oYk2nior2jBEQI+S0JwVquBDvbvyUnYT+E/mSURUA6kPtHax0kaDZnQijXGc4K +wASpAgMBAAGgZjBkBgkqhkiG9w0BCQ4xVzBVMAkGA1UdEwQCMAAwCwYDVR0PBAQD AgXgMCoGA1UdJQEB/wQgMB4GCCsGAQUFBwMIBggrBgEFBQcDAQYIKwYBBQUHAwIw -DwYDVR0RBAgwBocEfwAAATANBgkqhkiG9w0BAQUFAAOCAgEARkq8KI9Oy6Hjx83L -Kq/+532CEv/vz750uwc3MzejQJYDL9x7tkZR4ujDy1uspCUwAfAmq+EWsj0qMiLS -d1GYEEJAxGsWMxj2dRIgCnbXLsS4r08JMxzPVES1kgpiHw0neuT1q7jlfioqoIUG -WBMKV3PW4JqcxblBUCZzx0hTeEmvKP/aDikyw57kfUHJZG4P/TUzjBOn+afha7Ly -ptpdlNNFEZzmEOttrbu3V/KA2QjQEsN2Q73Sga595OUGaFLFbLtsvQ4Udj16M6gW -Z0Ays4XgjItsODS0W/q/InHKCP0dQyuyIb2PrWF5hZ7XcXb+iNExYN4E/WlG4vF6 -3EVIZPeL4TsPRK4lleQkCDyCfm7Ihrac1fhuzlacSyqz9Fp0fH7COyBltRvjrBPD -eqqUWZ5ZmxzT+GEG8tZJCxmMTNF6Lhqrq5n7E3t7kn73q7ikXjmkaVqlOFKR530z -Q5EXw6IFRrUnmbZGWdIdPMG3W1bkwYV5hhpytr2LsB7C0QZEu0phPje5eSsuh5SY -MX8cVh34T2vVqLURLf53N0cQBOR6UBeTa5YPSTuABlHwsGDEXwQdWGYSVVSFrf+o -d0hdZoAbDwXGGximscpQX8t3nUe3R/3lQtem3dxJ+lGq+rxamEgltXnjoOwSXFaU -djj5eM7u0qX1tjAezwt7OYAHEco= +DwYDVR0RBAgwBocEfwAAATANBgkqhkiG9w0BAQUFAAOCAgEAsjhQOckysiSTZaz/ +Ck5APYE13ckBQOxvHqrCvDQSRMqq/yb/eWtItQhCs9u3fKGFGKCefPYwtVdUj6v2 +g4SpXzBblcXNP8SXEtbcNuG4lPNsTc2YbDxuLWmgvLX8on2nq0f+TcGmtvvmCaLz +tEdLPlQfkzQUlwfaK4kMJ4qzR8eeHxBrhKJyDyzSreZSa/ZXMcKpjJLhXl6oM+ud +cfi0BO/xOKt/MmGHAxTMNrTFV5HcEWzws+r51sCcIV4BwuELA5rZuG0leaj3CDDy +3dPkpLp3HObbX3KV6yyCCfEAN4pyoZQNut8i87FITsYbJJy4ld7/PWTvg/JMk96X +ZACpeI1ERL/Sr3uzFpmlLszwSQ7t/hv5ykFA6d1QLv0u+EX1mL1O/PZ5Lb3moC4t +7ma3CffPqJpEjvvdXtweol62MI4Rr/1sXyZGR+lvkjcoSY5vzv0j7V4w6/e1In3D +nHY4vmpuy65w5gGm/s2y2cXRfCc8P4we0+A+QbNAgt3eicdJmWZGcWwlMQI10Nmq +6Qnp8cpwvFM/iCIajfIAz0fD7G0UEL5ExSFoh6WhFFDOYbTl5MeXUjMaYmemICE2 +r3GVfZxHLvDQzftkq/oSXXiZgLkoYveN0R7VJuUfCgyLOlJL5O8WX2lX8odUKINc +DcZ/FpV+R/u7WGm2j0tvRA4y9Qc= -----END CERTIFICATE REQUEST----- diff --git a/fixtures/ca/server2.key b/fixtures/ca/server2.key index 9d7e1e1fd57..492d86435f8 100644 --- a/fixtures/ca/server2.key +++ b/fixtures/ca/server2.key @@ -1,54 +1,54 @@ -----BEGIN RSA PRIVATE KEY----- Proc-Type: 4,ENCRYPTED -DEK-Info: DES-EDE3-CBC,6357944A31F1FA4D +DEK-Info: DES-EDE3-CBC,F332544C4A2552FC -8JSh0AAkpFI4wbTIxc461YY18lytg4AQHXDj3DU2lmkOcdS5vPQOqqWvqOW7DrmU -FyNkQRke/SyL6ZkcEPAe0wCaRV4KlU49HZOfxMGlJLeRbrSbcVsE/Kk1dJo3VB+x -Dm496BZMqVLrEuXx05kLiy3TUEsMNN3NSyFT/F76ZiV9HJ/ruVv/P5CS1Rwxzju7 -lZM4nEEsFCnGTPjdB/Es9HCjPX4HZmSdTwcU/vDBctS5aj91aE1EBSMdJfSYq2k4 -Ct6tAmOfBoi4qj+6Wr+gmaDyGvJyIscUbaBNXdM530CckkaFbZ00zXIqwnV33oiZ -1CZ6uEd0Uk7Hu0n8lnBClqLJmXVMyxnO2lv2MPAPkBeLK5OEFj8aX9ekZDrkIL7M -VgNgA5UCzmvtiEHNvow3BcEXhRFO1oT3O6AEyHpDUW3Ev7jn3SGg8Dg6suLs6GBc -h9sFynnKO0cPT0tJDBbPNqZwcgktnpIiWHPJutMl6kQxAGDijK3WrGOialvuK99x -rP/IIQm8bzi8eRvCulLtGw1al+yEH5Kwbu804cEgt2GFNXXvTo4M1RDFyxYiD9wz -g+Gs0ZCkwpjasM2iNnURDX/xgZo4o2G/R53221Y47/xVu94DVSTD/AQWvEbgWPga -zV7AQhUHGyoHMt5IyStcx1tobR2cmro4yCq3AN1aQwnUJm8LDY+KJpCAL+t4P61j -hUCrJ6ZyUsQAPnvJmYJ4gkm8e8uNKfB57SONkNw35aE1W0AK8sKY2ax+dcTuEtnd -6Pdh7WtucjMS8BEb2FvO1hOaidZ7aHfkU2QDJRlRsfKM0dfwVRN7BWmHGNPfGt1p -f13Odpsa2OHS5htrcQ1PsxykXdHxrCLKlwz/dA1lu84L2F/Ious0B7rq5+Jf+R6U -/xrzg24X63TYz0ysUfily4h0L1HyDQsseaSHCl5yYVKbjkBmapQwS4DmrlcTUUcF -KA6myKCyRuHsMr8sXo7qUmKhuUIjrgStbHIzueIlan69HdL0c8mhNHlIXP2tWQkI -v4pJXwtHp8sRMtF2Kbf/ZYCa5HKowu2JndFU45O+msWEq0xL0N+r+L16y74CEpuF -6N3aI0UnLdhzVG4ZGCEQ+YYvFySIzP/veGhrJj/UaJyRkf15YEKKs8sULOhVGmdR -LSHBe8yo7i1Vqec5bS6ZrttCNdaNWriylsEPjIZOkrlAAZGYqmwP2PR7NBvnWLX0 -QGIQoPQJuiibM9L9o+DEDDldzxFkWYdjsAO/ROs2bLng6uVU5oNsoTxZZOT2S+KN -eM/TkQasJn27JHAIRb64GFayY6ianh2RJHFOGrVNnZ8aU8SM/eP7r07vYxXgNAA0 -r748yVacwLk7nhq1IXrrcRSNugTXUVgQSQw7VL5+ftxpHIHC5ZNj+NWYB8dAB2m+ -e0+BJhxoEFzNa3PypvLPqhOPs4FshVyzcejrULjEZDJZ59pLHJENu4IS9MV0/spo -fjytvXK07Lk+UpAEbtUOONl9WIV06EAbJjH1Ow6cBGvJVypYqgWfhMH5vLvpID6k -RAKVGU3/J1Sq4J2OcgRQ6g2oQQV4GXwOEL9o+6mesbcGHFiu8YLGriKGul5HxE3k -5sSJUNKna1s6PxL+0vJQLljf9qrNG/bYS6p9jGtd1Wxma7vAEbpUvB644kSEEo1h -1GleOr/8bz7BvMgoLg7pikogqOCRxHBN7AsGXzasF7bQXITIQLv/b46G5auvNECn -mLXQayPx1okwSkYeOkDsoHATiFqmjyE6z5JGVF21z31K85ksxEzB+agchfD28Lme -7GmlvfiEq7eSK/MUhkW+msnuURY6WJpKg+9LOm83sp9OVXDWLWdkurfQTh4iXdPX -zLrQFhiHqmsZtLUUAQE9Hr34lvUwctDm7+LaL8IZgydlNAZks0xUuZhrmHHbbiRj -iuqCHvvVFrlhJPAUsL6ICw1ygUJVo/jXdsky7K5zeeLuPFr2IlrlUiGKgt4ew/xa -x3aFOw7zO5wVFSBjv7AfiImeM6/ke4fR7Jry75KWE2RLDQ8Gcnc8R8bIYUMZJjnV -JCjAb+PJbyfTqth5/epjYw2c5XulxGgQnab+P3gZL3W0GiHggPOviNVx8LPjW9Os -PQsRFGTB7S4lA1GCNRjwUgKdx5SKjNTcmGRltEju95Jdk2mOnI6cqTLiaXix9+fH -WWb9s9TiL6FvaDCEbU4npb0vpkJNtHHudy0x7FdbQhLMtPeVH4qLlDAIHyrrt7g4 -Mzu6LUlQCvvcrB2jfWym+hYCIkIn+MbMK2jaknh0jPS3F72i2wHRd3ykIg5Rdv7T -5OjApmDgMXibTBf2NmQAix/O6PYZp+jFZ/P3KNh7XSDC+Ne6qa8O3cdwaQsNLblt -7e+fTGHJeWnQT5Kk0fY5Q1Lrje0lQ3iZgnUrsNX1wzYwOJtCbKXYC7wgBQmz6/IQ -WSvlSZANwuutp9XdR44J3wbBm9Ux9cTgyL43PHSPTirPdu2QjQKE4A4ryqg+v64G -q4Mal0kMhfSJ1OP7t7dAkMCNjBrsmi9QsBlmSX18BOP3B4TzqiAWIH2NGxnH5MVg -TN/yQiIMyD24Zia7Z/CpMulQWyYKw9PkP7bdfyG65aJ0+LnYRu5MYS9wO694dcUG -kZDyBq866y8GIDVvzWHNIh+iEbvnGvAgIFOT9i983Y/3SqTJA5EJ6iK/1AAsnERd -14gQ/2SybyHZVB6RyPIw3E/QHu8ywNVOHg+pQDecFD0nQWV+l7fhRxn7+w4oLfx9 -PUIA/VFHaxx4pr14oHX1xUMOxuovhEjdXS1JkU6Niib5EqfrL3MDBnoMcH4EiAuT -XfNy5K+uxD2uW0bxq6yjRGTB1yec8kcrcZyArQkAI7mfTOAywSiQyD6O038hQ7Rl -hfMBSNjJiEhEvr4teNK6YPacQvsUEaAK411sHpb+enMR7l75PUazNdLHkVr/4rYo -Z4eL6J3TAVmCN2QKvPDEgz9lr5pl4ogTN5mysPzCc8YtT+B7o096TcD8U+8bstFa -GcTfsEGcxggX321Xc7kbfBPXKS7yq8b9wqwB3zbDTWgf3H/fd2aHozPBFeYWi8iK -6qIM1wiFYYV20So/jF3U8a6EdOjy47JdXXdUONj9Qe+fE+8iNA2x0FzWUJIXy0dI +0qzUXmrtDC8Uj4pnudhKPIHLErzRRgqS5qY6+Q+LNET41cltRat3t1/txN7WVPYa +3AKo1SRb6VyAthcbih4pEPC2rQpjPcPrhuXNP3HeiDxQ9CHcNxniN2HmVMYxf7p9 +WaKSYgfTLP/La3YHlicimDIpmLvPjCzVvEx7bY2zrRU24mmSopG2gsLpnbLUDL2e +U1R5lXKVULaO47hdtwcLQkmGo+b3uW+TAAmn6wTrJbOHKjq2+kFwZuilnmspXPcp +t64OrBbE2jmY2TiBurDhT8HnbRhpeYWn7LqRdYOV9Fsl/3ftXIr3pItUDitiGxkH +aNdFTm7XM2Sw25sS3TKxvksZW4sOMkIOUu/U/LrwQqNjvXMJMFJWRpMR9yDx62uc +yElcz53zAo5yg0q7vNOu4yap0xQdOc5fu7fWDWZO+cfEYFtaSZV1du9QPxTpnSy+ +UQIqFmv4sNcJOl3gdVu2rClKi//jutnjyPkxhoPGpbSxA1WbltcO0c2J1eSJJnd2 +2jWbHFXFpIIp4WLI1TKKMM2ARVL6ITi9L/EL7HNAXh+bw5F1j3KkdsYKDMepgk2b +ArJVPqvbLu8r6AdTtooOXfweJIWIGQNitDx4Ghj2d+XH6StImGBatYoofgKBH4q3 +nvxQRj+lsCX2ChdOD2rB1kyv8ak/6h880qtv3XTlaXzKUZfQ6Okd8/z+//eQcpRe +WY8uXfXWyc/EZ+IHkAp9FCwjlSe2kQpGDmTbXz55gT2fKy/WtJOX7YDEpoUcPjI7 ++efoQYjo62e73/drH2SafQBkoDTfB66+IVO7osGJx7wb5iLENjkZ7mnpLR4oydsr +miF9Iaik4eKzbX2EEXCvVX3/oCG51cdXB3Tun9bnLNZmHCsRFeOjNPTydMA9V729 +Pw07cBCKDMYK5D0QYkRRFWqpCsLiq5GZlEjLoJEjh7FJyQwCYrilfSSanCNk7llk +onCKwh11OrC6Amdq7NvLfhuM0pHkLf+LZsEQ1Z6MbMN6Sm/NGAMFl0SeCQrlZNub +rtOSQ3M6LhzFxWddfxt53MiK3adXRWdqeGBQtQLddj5PPCCJ7SSxFzIhAP6/jo5E +1Tym9BkqoaZhnK7IwGvc3qBlziyvs952OfwkS57YkM5t5DFdJwlVCNUiVvVNJdju +Z0PnRgfxZDUU+IB2h0jplSFpgsHPgZ3nPMYYprPMomj/qH2NTNQFLwf5Wv2eT33+ +AWjdnY4rUf5Up2rRVoPFgx2mVY3t54pLhX7UjcEwXMyKn8/jy/BhtHCRVdN9gHba +sHvicd4tOGepTjZYORDWq1pEJv0uB2qdHoJlA4PoH5HT32b5MW2hVssHyVt0DpLO +JP03k4ZxzA7JFCzjXQJvQfJ5uWTz2d/pHXQN7D+h2JxADhd+NeLvP7hwLZnXCF9z +uz3AmYbFQVWGnUaWuvwPfizqTRuKmbkkuEcVPhoOsrqXSj+K5ulMTOzPmAAku3Ig +UU2yAzCb6Jh/BlqhjreX0DU+ugvUZ0dYk+sGKv61VKA1Ifu315S6RtkowZn5cL/g +AQDc94OQ1kQUe0etnDjTUjW+MswAYrjsHGUa2KCYUwz/X/u4xfbaX/cPRiyM09+U +VbACr1ido9Fb/B71CHyYMi2nx0FYM0LpIfC/336+ONpuidfUwWD3Thia2krEaUQW +0/47ALXa96BVhJ1XWItII+1uW243YnN1W/QKB+fB4IKFqX6qCf7XddabFMWBG4OS +I8Ms2LdDcybIORs7G1iP95+UMljMNo9TFHH7vxEH1RcTRPejjmLk7/AxQzB63hML ++Cf24pD8H2CaHKBet97CEfP5AYk2Y3KcSPm7rmc9bwllFEbIotjz4QEPAOL8A81l +DUPuIaBNUgTzyjFhuzh5CCnseiJBrn4BnKJGeS27AzL2OEzsB6irmbrztAYA5AK8 +D7UrjdoT7f1MSmqf0gyXrxtrOJszdHG1U0HW3pZfJhgiqDVtUaE2ekspu9ji8vbk +s9wtIRWXM9+EXPVGtp2nbJftJjxj+Bj4GignDw3taLfn5nl5pfUbW9EQGBQ10KlU +CNl25Hwl5qcdvQeyfom/lXrUgY4nyGEwv6fqjjklz6N3yQpYYpJdW5kdTue56QDV +MkKeogf8ZgyFqMaEtaMhHNwSW032K3hc2B2GXoaF17KPdb5snEKGN+TXG+2tuV8A +ZBMT7Is+29Kyf750cUUmDlHS21BN9OjJcdZV9vQLbivRUtggfsMKImC0DYcqL4O4 +ToZB2VHiywlZjPLkiJSbx5u91mYCv10x2IPd3N4O15MWVZtp1L9KZEO8Dkn/OMAd +mw6mtFYHCfCnHC8hjvU/cWfvlia1M0gLDweokd3auiN61gQ/1S9THkUMPP/RA7qd +Zpj5UeqoTpLDjcVbYFl6NzkJM2KSbsdIewpqgsZW7PHiwO3c0iiXux8Ueo8nyDc9 +U3+mbYcBHpkd8F2hdyJPi5t9qW4ytI5P5LCA8Cgoj2CvkOLYh7qQFTMay2MEwsm6 +qwwcK8S3Ul4kZKx5KxoZOux4jDM36fR4pf0z+GIrDzfbnonJsv+oqHXGQYs4adwv +4LT71pJ9mein1LGDrRq6Kgvp/Eu4Sr/S3oWjkFKxa/zd2LucKrjt5n9M5dL4NVZK +0xyuoWTWY2lUol0hi53W0otz2hLQPFJllaHKUGISclmVSbHLi/6/Cc9XGALLM1+N +hGyrMDsEJLsgsMNZFm1QDphrGgHo/HdBsQoghiUTxuLS2gIE/845maJYW9F+1YZh +Q3uZKjFpL8MHyLTbP7RXjUFKUkM2lU5JJ9ydr3+W1OkVJENnmLbpalQctkO0W1It +vXU0U8bB4nCtZUO75299m4mUh/lw2Ej/D651+Pbk+iHcO/+p1hjiVfhoLF701JNo +WmEZ2/1ZGBqtO+ON7mlJLWOeYmiLnE/GheVPnFWv7ZFZCLVp8uEG69iK4pm9EH0B +QaqUA42etpo7gHOMfTiOkwvmQDWsiXl4+MSnPo+MBaCFlWor8ZHj5OkDU79bEn1x +dzzjhb3HTEJ9a//RC1vZbpRJuA4eueTrVkAJYcH1JMtnftKFz8qAaxYxsk8lVbnK +TIkxA1CjP9Btoq5a0XNkSRzzTAXo1blYAA/xy6HLbODuwNY4Mym+ZIPR9B6QlYZQ -----END RSA PRIVATE KEY----- diff --git a/fixtures/ca/server2.key.insecure b/fixtures/ca/server2.key.insecure index 0f2b9c0c129..e7024de6048 100644 --- a/fixtures/ca/server2.key.insecure +++ b/fixtures/ca/server2.key.insecure @@ -1,51 +1,51 @@ -----BEGIN RSA PRIVATE KEY----- -MIIJKgIBAAKCAgEAtG+XAmj6VT5CQCiX6wUQDJb6pM3F1MqJKp4eGKClvK6jXz8g -u9clFHPhKWvHWyouN/id7TQ3TBPLeX4MaL2Qjf0GtWuPwuD2ZOV8lDL8klINUgyW -L+WeRt+M2WwLeaJEuZhjGootkkfmicSshH5cJMnejiRWTpAzLehHQs9ZbP9yzg+t -c1WiwvqOXi1NM7KpYALh20MaUMSVIW6Lvwq5JAIo772x6lW9IVcTxajfYtlZDW9q -GD/4n95qBvDqtqSX665UmB+Kfskf5sWyIEKQ4A3i/RAXclhyEwgF5fBDMSr8ek3v -cSzbA8a7KCUHQ4aTGm6dnQHtsAlI5Aww0vs2QYi7oc2o5G3YXNfKmJdogxQJVwi+ -9/ZWoYGlQt77h90u9twej8XozOabF8M0zC8FvU/7Wc84pzvykpY7aao36E653FNt -3MCJhRG2MinqjITO+Xcd2B/dRRWYFB7qM/D0KbXtzvLnyJw91ERGjEJJ2E5/BgW6 -Wz996c/AGExnnf5xFgZ4/Z/dNPpwzNwRSTPZkhlrhdmnQkKjHXJDHLy+uZZ8uj2y -CGoTaJEZapEp7GGvsgwi/1DbAvmpwZpfXADWA4x+m6rJJvrOnwui5URcAgj1F+JE -W6d2piMmO1mDayiyNEkRpv3jB/Au713SUpm00GQCHrmrTb5izgCH3ggS75kCAwEA -AQKCAgBeBBcHJnPpnrseewhNaSHnrXOEE8QVEENQdXrxEiPJoKV3p4kC1yN2+LpF -vubtVZkniN+hDSgS9+15rHgCy2Na2JB6T0VlIZrBD+JNxhNcmmxeaDiJVHeYLjtR -vr5r7mUo34Ij/gOoyNYSyuupTb3tXVIddkmSPgiszu7ynN/Xr3K+c+TIx8I7Hhq4 -b5peaaVfZaERgnFfzE54UQV30bqOTKHP6WOe2nXlvV2MDGX5N47zNS5u3EZL5rQQ -Uc+6wyB1qSxi52xei6WXtUMPFCAw3ot7mLre1b76s59/JCWepOtRPvKYYersmMxE -KqcSRDi5+REfEjYrF2tniAKYrFSkCKQXSOySmSNBcbpvmRhG5l4/s/dAgWuox0ha -qj+VteV8H+CruVEonBFDxiXy9cEQFeRQJU2hP1HighYe4pjfMmA6VggdnTSKaUGq -VLVYiKsEZW6G4tP11q1h8EVANFLUUVjXYLsoROKNai9n+tHRA+VxDbX2cMnbqsb7 -LT9xDpCOjl0cZ2Sw7FF8ENuDttQXr6ehhpUSPk6dNh++g7juCuYHQvceHCL4t+m8 -IIuQxWNMC6kjpOXSDpkedaGPu7OwSpxqTc6HyYb4t3quesTsha+ZELd2txNgrCHj -TfIHnoE7rqmwYAKMlloxk3xWaOzXk5M1JkazpxTHiFd/SpgeQQKCAQEA49Qy8NCJ -VQIJTs54mljKNWEUuWjt48k1K2eQmPsbj/wJHFDwOM2TKktAthJmgW4lmOYHbIB9 -xIRBF/+Auzhcbuh/W7gPatwGD+w36ljluX9NxXrLDxGD8rYCJArr1WE0Elb6fUnN -oNijYlLOSv6BVWLt9l6tdchl62pkhunrp5JmbGxe4npmCfACnR1OqJF/l0CwnQRz -23qR3Zfyaek9sL1MkXp5DIhLrBlWD9b4JU7DwriWZPuTGRKOYiUrkRa4QcVwBRIF -MfmK+pJvJ+X49sIE7bpByCBKyki/8raUBdGJq80iFetvogYGdpld/aLMuni/2EIS -fIrKTjLmyBNjpQKCAQEAyr8w3ye8PhE7TFJjUCgaCcUmrNf//ePcC/wCjzacLgxh -TzIVRAVpE3hp7r6XfSmtEuad2d/3OyIjKckcp+46+YVmhD3L2vv5q7myl1+nWU22 -qnK1nO8xt9W9rH3wn51UjAK31wa3x9OGuu7HJ8UeX+McgSL42uOxI0IEGu9FQts6 -oU8FUmnptdw3xXLKN55yQlwfd+apMZCiVd5RjPusoLeGjfjg/vS4mJMi+MuFued4 -5SE/GqRHwFxT0zNXh73DyvkQ8S/JxZpgdJiWk4tckM3gfAwZdqqnCczIj0XTn612 -SZ+RN9S3J8FlEThXcA4NX/zBb+GGe0QRgyOzIjIJ5QKCAQEArO//dwKkrE1uaU6b -B3ZMj7ZQd+kpYpXx8S+c+DLsGiCTfdYGcufBRQJ4bXyMKMVGbsh1bCwgy5IkoyXE -PtkqeNmtCx8tPM0lIOMLEq1GO8dhbnymNJr0EMGN4HQVzhQJ5b32SDJEj0rCwrje -dNi5ren5feEiRFzI2KkbA7n+smWTr8uXPszwNazlHwQHGDfRpStqpNWjaD+jB5T2 -YuS0ejtHKsrPpe6zmkBlLoLcO92NBXr+VksMvqRyRhe2+VxIo0xOmtqx7NxiXY/Y -Jm4PzKc7/IQ8uL+iZehRI9jphX0nxqxgqkjbpR5zu4Txbr0sMIPGBE8rHzRvGmrS -+Z6WLQKCAQEAsWgu2XWpkB3/5z7ITCFq91WeC+xNwdmaeJohmzNL4jdPBr9qQEUD -ttfMye3YHNtU6I8HXNhPO10Zq7yg6Ija7e++zsRMFugZhhxNm4tFoi0QQ4FwTUw4 -EwZinAbvgJtomcLfHrZwJdh/sh6yAajIdVsDXDQ/0TkfjRx8/xyTXHb8jJ3aqEyp -BksWOh3BuH1auZpmpsIdFpPamIyai2TFnzQ39w7pwe5dgJRvK4jbENrsvIIk96j8 -Z+PiLZJlaw+vvXHHU2RfiBsXf67tQ1nUu5iKb8EenumglECu1j9hd/2O3clUhqgJ -RC5Dw5erWw/QwEco9D9BL6dWM4wPeUKTyQKCAQEA2EXIdcrehSjYRouLoR6j9Io1 -/0yeydM5mSdrwjaQTC7AmPF7tt6BUpth2TfG1hbkuouu6eUNqMiMGs1sLPXEPpvh -geA1vdMzX5c9f/rx14jTRwVZnVoPiapLBNr4Z/Hgojr42vYqqvsrMaC0k8G/xH85 -9oUBKBrSApDycP5Pdufex8TGRHF9NW4VKDdZS86jIhQVfUVgvIacx4D2Y6XHIgtW -3mJvoRvIPLipbeXgJQgNi+RhsFuYkfVerC1O4ZMsIlLFzmpclcF/nmVmb5rF03wx -wJLEKmAS4yAjFtPuhdqbdeDdR2W8SCZDmkrZIiZuCWvHxPByGbxL7gQJlVqpTA== +MIIJJwIBAAKCAgEAtjkV86BtyWuR9lL9enM8mBZmuz9b7NqS6gEO2R5N4h1UyTaU +1n/+gDiD55hU+d/QNImgQ+8kg67CMXxvYzeJhH5/BXiIjY0pUSJXnjr2q2ORKbs6 +W81EwB5bO0aby5cj3CD0wSpVThI+WlNXuRQTZz93eNTpkCO+AGfqxadFYbf2UEZd +5yprLo/NsBiGWsYf8649loviZzqcTUPv+Z0E6jcn7p01BTMH0WFT8FjnkzlzS1Ks +MaUz2k8T3+ncle7V1yn3li1ltQKJv809WccYQrmzwt8bpETyTHFSPps+M+bIQX6O +yDAlva90eM5dvBu2yOJamtnpOsAV8tBaYP2puxXG/IyWe7Ot8/JpERaGCKMEAMMY +GZwvyT84UfcuqGbA4RMukZ6kJXkFtGoPf60MhByOOW3V90lMO7VG4+z3bUR0ddpA +3NNM9s70Og2kVjSXpSUeOXPxWWfC4Sh6F3c0Nlztr9vxfpHLQnC5rKai5QkT0ZCs +YlEneKZmTCuGGrGJYi2nockqB48PfxXW0VuJ+qOIX8t+Gppn+hoVTl/fAtuzjUG4 +w5C0Sg2yZEI13YvoEn75+ySqcl9u7DphGgHuC+gWPKqfkJm6jHwCuY6z+hiTaeKi +vaMERAj5LQnBWq4EO9u/JSdhP4T+ZJRFQDqQ+0drHSRoNmdCKNcZzgrABKkCAwEA +AQKCAgA+zV/mbmVIJR3SMnoQCMVaeWYApO6OrCo0Ihc29z3Kb2d4TapwXv6cvF2h +pRusXtnIMaKdpz8Db2iYW5WcMVjg5CPtA8S0XHFf+CEQdKvtF8zBADk1yIIoYI36 +2PP67+U5Cdaw+GEcHieFQ/IY5HVngTUw3Nh+iAME6su8QVElQ5zNv+K/OBxmmMNA +LMOpZ109w9CQITfvcgDKlF6Rve8itc26bE4Is7S/Efc2/70YPZWh4SVdmt1LITPt +WRFgT0c998XP6WeDQhOtmhPJ7FdNL+lngTNqoySK+gdpcmG2y5Q7Fl4pWoa3YFAZ +Dq65lSejBnhJpE7Ao9EstWhgwywKrKF2TRFWDyBCFbCJ+LdbuwA/0fjMYC1NiyxU +ApSuULGkbg7hCUGOAa8OzbSk1BL55s934cI2bW0NuLQHYcrFcLxjOxGpgt1RWHT9 +9tIeE28oupjMEvlXqIvOy8OWR5nLLhUT8xVyGWRtj/GgyI4GBGtujR2NmYG6xOrj +CThdCqmm1oMdZx4puNPKpG4rXIafP7itOVbWucuAecYNiOD5buVJQLLSHSYpCool +204QPY+jlFlYrXypCKzSdGv0Yx8Z2BryVV930lle0GY1QwJyklk4Qhksl3txXrO1 +89ERklRnqn83WDKDopPKKgveTWMK1VGA8VuWYIDXj/qqklFKMQKCAQEA3dHbJgQ3 +EJvrnK/WL/RPNmfxc7R0oSPDPP691CriXrlKIfs9zHPX/KsdZ8HHDYqR9AehE/xc +6dYsoKtvy9kP/96WDh8TH+Ofx0zlQmYtLRengWTBkHU9ti+1D/cZBSyW5xx5uTKr +lxCJBOk5qn5LJuYaIZpzCKR7FkzjE/H4KGiGnG4aNXwa8jXx0ZP7GGKTuaR1dEa/ +G9Ju25yokhmiAvrQ07qk91lZmOER2wlZdU8LTJMgYnM8bPEoZc10jpLj1q8QSNEw +XL/uvFXsiBh2qNr+11QGzQiKXNPWzGVLtjxzQG//5SIj1gpuTgGO0+sgMFQE/e13 +0utMmgv8DLgEnQKCAQEA0k1AsvR5soPYgZQxztUK/Zrn4HTyjO4PRkObbPAzdNjw +CrLJugSbeNQJ+7mOxR2p09KsDuLcWhhB55pab/SytXZVCr3wxjpQIUxtmybkkOMx +7QPkCi9+rKDE5MSSWfSkGOMhTVV7+1jG2+enIWCQBJg+pEgjC3OypkLTR1jDwsa8 +WgpNTm8RZx79WHi+XMqf9BpgKxTD88wdJJQbOW0sh8/MiQ6rUl4Za64rufglnNmM ++6X2ME9Sz55KvFQqv5Us573lU2z1WYnRoHjt4WbCNMZdBT0pUvN50Lg3rWglu7jY +c+EyWEUd9eMrA2Hk3ZUc8p0diRcWh6d0jrlO3reUfQKCAQBU84j1b0nTb5N1h5YE ++ZDYqkg7YtID4JlmI715ow7c7iNpDjplsbv3RWVWlkzwb7BkAAP9jnnbCC5BPkbr +j+7jtFBNijMd1GQdxOJMYqtMiLGbCYZkF7KRsoWqXpzTcXc9fZdUiQZULX38RoHS +PNn1RMyfL/J8Tdnh+YJB4jqC5z0ebcBV2XjMaEJ7XCwe86nVwBlHdcy9EANq0f1x +LqXwdDRD0khZfnuk5BWdiGAdYC9YnUQa0D1FD7rD+kJ4U+M0FgmriYn3C36X3GRg +3tWa53wP5VtRbMLouCycTPMJEO+mrv4Wt9N3prkF4OzdVkAWoibjRO3N9lV47bwS +9uq1AoIBAEPqmQNyOr8xH0Gxx2ghm1wNo+b0PcTPuPUbLl2/MQ8CZHtABC/j/wXF +jLfT1EzKaKc0+UYRc9JQ1S/jxGM1pmU+IvbGIrUR7gDi+t7Jb7Vu+heuUv4LGqDL +hurOpOkSPdCfwYiFG/YvVIF+TZZU5g3l0Q0jEtZG9iIFoNAA1a/YmMmHXDIBYqBn +/K+OxwOWmJOv1PD00tewSpUek7A3FtOBg2+b4i5Gn3UMGakEf7ko9QPsNBaj931/ +hGlP0UJv/cGVrTMFFDNnc+CcTU6m7f83NKFVgDv+z49dfvWslcsLRjQePTEOmT9o +ruJ3wf7hgijEHt7AKxGCPf090T/SD80CggEAfxDd94aLvdJZWFenilAQsx0Zm6i4 +PMbMN0izQ4DG6Ds5XnUtbPjmomHaW/sqorrpGNoE3hG4sSQKcxjD4TI3Dp6OPjoN +wIuFWjeXDNzGNRBPV5wmm+uKWbMf00sOOa9BvHAzd0yNisMEMGEun4b9wuC/E/IS +XI/cJVoDQUI+dF/q7OS5mLclVnPO6TC3ZT8/JBpBnjs62b8bkqAtLa1TqWJFKnz2 +vz8uyM2o+6zVUqKB7s7vOHwN7nf8EHNTXe/JeZ49qLYgzmBkSjoVLFsAAvZ1IkXW +vETDH3t9bjd7WzZizF0iVBrk0zd5242L150av9AgGSTLljOo7npEkSlzEg== -----END RSA PRIVATE KEY----- diff --git a/fixtures/ca/server2.pub b/fixtures/ca/server2.pub deleted file mode 100644 index 33361caf5df..00000000000 --- a/fixtures/ca/server2.pub +++ /dev/null @@ -1,14 +0,0 @@ ------BEGIN PUBLIC KEY----- -MIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKCAgEAtG+XAmj6VT5CQCiX6wUQ -DJb6pM3F1MqJKp4eGKClvK6jXz8gu9clFHPhKWvHWyouN/id7TQ3TBPLeX4MaL2Q -jf0GtWuPwuD2ZOV8lDL8klINUgyWL+WeRt+M2WwLeaJEuZhjGootkkfmicSshH5c -JMnejiRWTpAzLehHQs9ZbP9yzg+tc1WiwvqOXi1NM7KpYALh20MaUMSVIW6Lvwq5 -JAIo772x6lW9IVcTxajfYtlZDW9qGD/4n95qBvDqtqSX665UmB+Kfskf5sWyIEKQ -4A3i/RAXclhyEwgF5fBDMSr8ek3vcSzbA8a7KCUHQ4aTGm6dnQHtsAlI5Aww0vs2 -QYi7oc2o5G3YXNfKmJdogxQJVwi+9/ZWoYGlQt77h90u9twej8XozOabF8M0zC8F -vU/7Wc84pzvykpY7aao36E653FNt3MCJhRG2MinqjITO+Xcd2B/dRRWYFB7qM/D0 -KbXtzvLnyJw91ERGjEJJ2E5/BgW6Wz996c/AGExnnf5xFgZ4/Z/dNPpwzNwRSTPZ -khlrhdmnQkKjHXJDHLy+uZZ8uj2yCGoTaJEZapEp7GGvsgwi/1DbAvmpwZpfXADW -A4x+m6rJJvrOnwui5URcAgj1F+JEW6d2piMmO1mDayiyNEkRpv3jB/Au713SUpm0 -0GQCHrmrTb5izgCH3ggS75kCAwEAAQ== ------END PUBLIC KEY----- From 256a0428ddef17e789db493f6fd144301f7bdf85 Mon Sep 17 00:00:00 2001 From: Ben Johnson Date: Wed, 13 Nov 2013 21:56:02 -0500 Subject: [PATCH 247/247] CHANGELOG --- CHANGELOG | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) create mode 100644 CHANGELOG diff --git a/CHANGELOG b/CHANGELOG new file mode 100644 index 00000000000..414bdad84cd --- /dev/null +++ b/CHANGELOG @@ -0,0 +1,26 @@ +v0.2 +* Add Compare-and-Swap (CAS) support. +* Support recursive GETs. +* Support fully consistent GETs. +* Allow clients to watch for key expiration. +* Refactor low-level data store. +* Modularize store, server and API code. +* Integrate Gorilla Web Toolkit. +* Add tiered configuration (command line args, env variables, config file). +* Add peer protocol versioning. +* Add rolling upgrade support for future versions. +* Sync key expiration across cluster. +* Significantly improve test coverage. +* Improve migration testing. +* Bug Fixes: https://github.com/coreos/etcd/issues?milestone=1&state=closed + +Contributors: +* Xiang Li (@xiangli-cmu) +* Ben Johnson (@benbjohnson) +* Brandon Philips (@philips) +* Yifan (@yifan-gu) +* Rob Szumski +* Hongchao Deng (@fengjingchao) +* Kelsey Hightower (@kelseyhightower) +* Adrián (@adrianlzt) +* Antonio Terreno (@aterreno)
Machine Name Latency
{{follower.name}}
+
{{machine.name}}(leader)
+
{{machine.name}}
+
-
-
{{follower.latency.current | number:1 }} ms
+
+
{{machine.latency.current | number:1 }} ms
-
+
{{machine.latency.current | number:1 }} ms