diff --git a/Documentation/dev-guide/apispec/swagger/rpc.swagger.json b/Documentation/dev-guide/apispec/swagger/rpc.swagger.json index 28a88b899fe..10697cc0f02 100644 --- a/Documentation/dev-guide/apispec/swagger/rpc.swagger.json +++ b/Documentation/dev-guide/apispec/swagger/rpc.swagger.json @@ -1163,7 +1163,8 @@ "default": "NONE", "enum": [ "NONE", - "NOSPACE" + "NOSPACE", + "CORRUPT" ] }, "etcdserverpbAuthDisableRequest": { diff --git a/bill-of-materials.json b/bill-of-materials.json index 20fc5200228..cd518ece334 100644 --- a/bill-of-materials.json +++ b/bill-of-materials.json @@ -26,15 +26,6 @@ } ] }, - { - "project": "github.com/cockroachdb/cmux", - "licenses": [ - { - "type": "Apache License 2.0", - "confidence": 1 - } - ] - }, { "project": "github.com/coreos/bbolt", "licenses": [ @@ -264,6 +255,15 @@ } ] }, + { + "project": "github.com/soheilhy/cmux", + "licenses": [ + { + "type": "Apache License 2.0", + "confidence": 1 + } + ] + }, { "project": "github.com/spf13/cobra", "licenses": [ diff --git a/cmd/vendor/github.com/cockroachdb/cmux/buffer.go b/cmd/vendor/github.com/cockroachdb/cmux/buffer.go deleted file mode 100644 index 5c178585363..00000000000 --- a/cmd/vendor/github.com/cockroachdb/cmux/buffer.go +++ /dev/null @@ -1,35 +0,0 @@ -package cmux - -import ( - "bytes" - "io" -) - -// bufferedReader is an optimized implementation of io.Reader that behaves like -// ``` -// io.MultiReader(bytes.NewReader(buffer.Bytes()), io.TeeReader(source, buffer)) -// ``` -// without allocating. -type bufferedReader struct { - source io.Reader - buffer *bytes.Buffer - bufferRead int - bufferSize int -} - -func (s *bufferedReader) Read(p []byte) (int, error) { - // Functionality of bytes.Reader. - bn := copy(p, s.buffer.Bytes()[s.bufferRead:s.bufferSize]) - s.bufferRead += bn - - p = p[bn:] - - // Funtionality of io.TeeReader. - sn, sErr := s.source.Read(p) - if sn > 0 { - if wn, wErr := s.buffer.Write(p[:sn]); wErr != nil { - return bn + wn, wErr - } - } - return bn + sn, sErr -} diff --git a/cmd/vendor/github.com/cockroachdb/cmux/matchers.go b/cmd/vendor/github.com/cockroachdb/cmux/matchers.go deleted file mode 100644 index abc30f6e0ad..00000000000 --- a/cmd/vendor/github.com/cockroachdb/cmux/matchers.go +++ /dev/null @@ -1,150 +0,0 @@ -package cmux - -import ( - "bufio" - "io" - "io/ioutil" - "net/http" - "strings" - - "golang.org/x/net/http2" - "golang.org/x/net/http2/hpack" -) - -// Any is a Matcher that matches any connection. -func Any() Matcher { - return func(r io.Reader) bool { return true } -} - -// PrefixMatcher returns a matcher that matches a connection if it -// starts with any of the strings in strs. -func PrefixMatcher(strs ...string) Matcher { - pt := newPatriciaTreeString(strs...) - return pt.matchPrefix -} - -var defaultHTTPMethods = []string{ - "OPTIONS", - "GET", - "HEAD", - "POST", - "PUT", - "DELETE", - "TRACE", - "CONNECT", -} - -// HTTP1Fast only matches the methods in the HTTP request. -// -// This matcher is very optimistic: if it returns true, it does not mean that -// the request is a valid HTTP response. If you want a correct but slower HTTP1 -// matcher, use HTTP1 instead. -func HTTP1Fast(extMethods ...string) Matcher { - return PrefixMatcher(append(defaultHTTPMethods, extMethods...)...) -} - -const maxHTTPRead = 4096 - -// HTTP1 parses the first line or upto 4096 bytes of the request to see if -// the conection contains an HTTP request. -func HTTP1() Matcher { - return func(r io.Reader) bool { - br := bufio.NewReader(&io.LimitedReader{R: r, N: maxHTTPRead}) - l, part, err := br.ReadLine() - if err != nil || part { - return false - } - - _, _, proto, ok := parseRequestLine(string(l)) - if !ok { - return false - } - - v, _, ok := http.ParseHTTPVersion(proto) - return ok && v == 1 - } -} - -// grabbed from net/http. -func parseRequestLine(line string) (method, uri, proto string, ok bool) { - s1 := strings.Index(line, " ") - s2 := strings.Index(line[s1+1:], " ") - if s1 < 0 || s2 < 0 { - return - } - s2 += s1 + 1 - return line[:s1], line[s1+1 : s2], line[s2+1:], true -} - -// HTTP2 parses the frame header of the first frame to detect whether the -// connection is an HTTP2 connection. -func HTTP2() Matcher { - return hasHTTP2Preface -} - -// HTTP1HeaderField returns a matcher matching the header fields of the first -// request of an HTTP 1 connection. -func HTTP1HeaderField(name, value string) Matcher { - return func(r io.Reader) bool { - return matchHTTP1Field(r, name, value) - } -} - -// HTTP2HeaderField resturns a matcher matching the header fields of the first -// headers frame. -func HTTP2HeaderField(name, value string) Matcher { - return func(r io.Reader) bool { - return matchHTTP2Field(r, name, value) - } -} - -func hasHTTP2Preface(r io.Reader) bool { - var b [len(http2.ClientPreface)]byte - if _, err := io.ReadFull(r, b[:]); err != nil { - return false - } - - return string(b[:]) == http2.ClientPreface -} - -func matchHTTP1Field(r io.Reader, name, value string) (matched bool) { - req, err := http.ReadRequest(bufio.NewReader(r)) - if err != nil { - return false - } - - return req.Header.Get(name) == value -} - -func matchHTTP2Field(r io.Reader, name, value string) (matched bool) { - if !hasHTTP2Preface(r) { - return false - } - - framer := http2.NewFramer(ioutil.Discard, r) - hdec := hpack.NewDecoder(uint32(4<<10), func(hf hpack.HeaderField) { - if hf.Name == name && hf.Value == value { - matched = true - } - }) - for { - f, err := framer.ReadFrame() - if err != nil { - return false - } - - switch f := f.(type) { - case *http2.HeadersFrame: - if _, err := hdec.Write(f.HeaderBlockFragment()); err != nil { - return false - } - if matched { - return true - } - - if f.FrameHeader.Flags&http2.FlagHeadersEndHeaders != 0 { - return false - } - } - } -} diff --git a/cmd/vendor/github.com/cockroachdb/cmux/LICENSE b/cmd/vendor/github.com/soheilhy/cmux/LICENSE similarity index 100% rename from cmd/vendor/github.com/cockroachdb/cmux/LICENSE rename to cmd/vendor/github.com/soheilhy/cmux/LICENSE diff --git a/cmd/vendor/github.com/soheilhy/cmux/buffer.go b/cmd/vendor/github.com/soheilhy/cmux/buffer.go new file mode 100644 index 00000000000..f8cf30a1e66 --- /dev/null +++ b/cmd/vendor/github.com/soheilhy/cmux/buffer.go @@ -0,0 +1,67 @@ +// Copyright 2016 The CMux Authors. 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. + +package cmux + +import ( + "bytes" + "io" +) + +// bufferedReader is an optimized implementation of io.Reader that behaves like +// ``` +// io.MultiReader(bytes.NewReader(buffer.Bytes()), io.TeeReader(source, buffer)) +// ``` +// without allocating. +type bufferedReader struct { + source io.Reader + buffer bytes.Buffer + bufferRead int + bufferSize int + sniffing bool + lastErr error +} + +func (s *bufferedReader) Read(p []byte) (int, error) { + if s.bufferSize > s.bufferRead { + // If we have already read something from the buffer before, we return the + // same data and the last error if any. We need to immediately return, + // otherwise we may block for ever, if we try to be smart and call + // source.Read() seeking a little bit of more data. + bn := copy(p, s.buffer.Bytes()[s.bufferRead:s.bufferSize]) + s.bufferRead += bn + return bn, s.lastErr + } else if !s.sniffing && s.buffer.Cap() != 0 { + // We don't need the buffer anymore. + // Reset it to release the internal slice. + s.buffer = bytes.Buffer{} + } + + // If there is nothing more to return in the sniffed buffer, read from the + // source. + sn, sErr := s.source.Read(p) + if sn > 0 && s.sniffing { + s.lastErr = sErr + if wn, wErr := s.buffer.Write(p[:sn]); wErr != nil { + return wn, wErr + } + } + return sn, sErr +} + +func (s *bufferedReader) reset(snif bool) { + s.sniffing = snif + s.bufferRead = 0 + s.bufferSize = s.buffer.Len() +} diff --git a/cmd/vendor/github.com/cockroachdb/cmux/cmux.go b/cmd/vendor/github.com/soheilhy/cmux/cmux.go similarity index 62% rename from cmd/vendor/github.com/cockroachdb/cmux/cmux.go rename to cmd/vendor/github.com/soheilhy/cmux/cmux.go index 89cc910b024..9de6b0a3c2a 100644 --- a/cmd/vendor/github.com/cockroachdb/cmux/cmux.go +++ b/cmd/vendor/github.com/soheilhy/cmux/cmux.go @@ -1,16 +1,33 @@ +// Copyright 2016 The CMux Authors. 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. + package cmux import ( - "bytes" "fmt" "io" "net" "sync" + "time" ) // Matcher matches a connection based on its content. type Matcher func(io.Reader) bool +// MatchWriter is a match that can also write response (say to do handshake). +type MatchWriter func(io.Writer, io.Reader) bool + // ErrorHandler handles an error and returns whether // the mux should continue serving the listener. type ErrorHandler func(error) bool @@ -44,13 +61,17 @@ func (e errListenerClosed) Timeout() bool { return false } // listener is closed. var ErrListenerClosed = errListenerClosed("mux: listener closed") +// for readability of readTimeout +var noTimeout time.Duration + // New instantiates a new connection multiplexer. func New(l net.Listener) CMux { return &cMux{ - root: l, - bufLen: 1024, - errh: func(_ error) bool { return true }, - donec: make(chan struct{}), + root: l, + bufLen: 1024, + errh: func(_ error) bool { return true }, + donec: make(chan struct{}), + readTimeout: noTimeout, } } @@ -61,27 +82,53 @@ type CMux interface { // // The order used to call Match determines the priority of matchers. Match(...Matcher) net.Listener + // MatchWithWriters returns a net.Listener that accepts only the + // connections that matched by at least of the matcher writers. + // + // Prefer Matchers over MatchWriters, since the latter can write on the + // connection before the actual handler. + // + // The order used to call Match determines the priority of matchers. + MatchWithWriters(...MatchWriter) net.Listener // Serve starts multiplexing the listener. Serve blocks and perhaps // should be invoked concurrently within a go routine. Serve() error // HandleError registers an error handler that handles listener errors. HandleError(ErrorHandler) + // sets a timeout for the read of matchers + SetReadTimeout(time.Duration) } type matchersListener struct { - ss []Matcher + ss []MatchWriter l muxListener } type cMux struct { - root net.Listener - bufLen int - errh ErrorHandler - donec chan struct{} - sls []matchersListener + root net.Listener + bufLen int + errh ErrorHandler + donec chan struct{} + sls []matchersListener + readTimeout time.Duration +} + +func matchersToMatchWriters(matchers []Matcher) []MatchWriter { + mws := make([]MatchWriter, 0, len(matchers)) + for _, m := range matchers { + mws = append(mws, func(w io.Writer, r io.Reader) bool { + return m(r) + }) + } + return mws } func (m *cMux) Match(matchers ...Matcher) net.Listener { + mws := matchersToMatchWriters(matchers) + return m.MatchWithWriters(mws...) +} + +func (m *cMux) MatchWithWriters(matchers ...MatchWriter) net.Listener { ml := muxListener{ Listener: m.root, connc: make(chan net.Conn, m.bufLen), @@ -90,6 +137,10 @@ func (m *cMux) Match(matchers ...Matcher) net.Listener { return ml } +func (m *cMux) SetReadTimeout(t time.Duration) { + m.readTimeout = t +} + func (m *cMux) Serve() error { var wg sync.WaitGroup @@ -124,10 +175,17 @@ func (m *cMux) serve(c net.Conn, donec <-chan struct{}, wg *sync.WaitGroup) { defer wg.Done() muc := newMuxConn(c) + if m.readTimeout > noTimeout { + _ = c.SetReadDeadline(time.Now().Add(m.readTimeout)) + } for _, sl := range m.sls { for _, s := range sl.ss { - matched := s(muc.getSniffer()) + matched := s(muc.Conn, muc.startSniffing()) if matched { + muc.doneSniffing() + if m.readTimeout > noTimeout { + _ = c.SetReadDeadline(time.Time{}) + } select { case sl.l.connc <- muc: case <-donec: @@ -177,13 +235,13 @@ func (l muxListener) Accept() (net.Conn, error) { // MuxConn wraps a net.Conn and provides transparent sniffing of connection data. type MuxConn struct { net.Conn - buf bytes.Buffer - sniffer bufferedReader + buf bufferedReader } func newMuxConn(c net.Conn) *MuxConn { return &MuxConn{ Conn: c, + buf: bufferedReader{source: c}, } } @@ -198,13 +256,14 @@ func newMuxConn(c net.Conn) *MuxConn { // return either err == EOF or err == nil. The next Read should // return 0, EOF. func (m *MuxConn) Read(p []byte) (int, error) { - if n, err := m.buf.Read(p); err != io.EOF { - return n, err - } - return m.Conn.Read(p) + return m.buf.Read(p) +} + +func (m *MuxConn) startSniffing() io.Reader { + m.buf.reset(true) + return &m.buf } -func (m *MuxConn) getSniffer() io.Reader { - m.sniffer = bufferedReader{source: m.Conn, buffer: &m.buf, bufferSize: m.buf.Len()} - return &m.sniffer +func (m *MuxConn) doneSniffing() { + m.buf.reset(false) } diff --git a/cmd/vendor/github.com/soheilhy/cmux/doc.go b/cmd/vendor/github.com/soheilhy/cmux/doc.go new file mode 100644 index 00000000000..aaa8f315899 --- /dev/null +++ b/cmd/vendor/github.com/soheilhy/cmux/doc.go @@ -0,0 +1,18 @@ +// Copyright 2016 The CMux Authors. 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. + +// Package cmux is a library to multiplex network connections based on +// their payload. Using cmux, you can serve different protocols from the +// same listener. +package cmux diff --git a/cmd/vendor/github.com/soheilhy/cmux/matchers.go b/cmd/vendor/github.com/soheilhy/cmux/matchers.go new file mode 100644 index 00000000000..652fd869178 --- /dev/null +++ b/cmd/vendor/github.com/soheilhy/cmux/matchers.go @@ -0,0 +1,262 @@ +// Copyright 2016 The CMux Authors. 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. + +package cmux + +import ( + "bufio" + "crypto/tls" + "io" + "io/ioutil" + "net/http" + "strings" + + "golang.org/x/net/http2" + "golang.org/x/net/http2/hpack" +) + +// Any is a Matcher that matches any connection. +func Any() Matcher { + return func(r io.Reader) bool { return true } +} + +// PrefixMatcher returns a matcher that matches a connection if it +// starts with any of the strings in strs. +func PrefixMatcher(strs ...string) Matcher { + pt := newPatriciaTreeString(strs...) + return pt.matchPrefix +} + +func prefixByteMatcher(list ...[]byte) Matcher { + pt := newPatriciaTree(list...) + return pt.matchPrefix +} + +var defaultHTTPMethods = []string{ + "OPTIONS", + "GET", + "HEAD", + "POST", + "PUT", + "DELETE", + "TRACE", + "CONNECT", +} + +// HTTP1Fast only matches the methods in the HTTP request. +// +// This matcher is very optimistic: if it returns true, it does not mean that +// the request is a valid HTTP response. If you want a correct but slower HTTP1 +// matcher, use HTTP1 instead. +func HTTP1Fast(extMethods ...string) Matcher { + return PrefixMatcher(append(defaultHTTPMethods, extMethods...)...) +} + +// TLS matches HTTPS requests. +// +// By default, any TLS handshake packet is matched. An optional whitelist +// of versions can be passed in to restrict the matcher, for example: +// TLS(tls.VersionTLS11, tls.VersionTLS12) +func TLS(versions ...int) Matcher { + if len(versions) == 0 { + versions = []int{ + tls.VersionSSL30, + tls.VersionTLS10, + tls.VersionTLS11, + tls.VersionTLS12, + } + } + prefixes := [][]byte{} + for _, v := range versions { + prefixes = append(prefixes, []byte{22, byte(v >> 8 & 0xff), byte(v & 0xff)}) + } + return prefixByteMatcher(prefixes...) +} + +const maxHTTPRead = 4096 + +// HTTP1 parses the first line or upto 4096 bytes of the request to see if +// the conection contains an HTTP request. +func HTTP1() Matcher { + return func(r io.Reader) bool { + br := bufio.NewReader(&io.LimitedReader{R: r, N: maxHTTPRead}) + l, part, err := br.ReadLine() + if err != nil || part { + return false + } + + _, _, proto, ok := parseRequestLine(string(l)) + if !ok { + return false + } + + v, _, ok := http.ParseHTTPVersion(proto) + return ok && v == 1 + } +} + +// grabbed from net/http. +func parseRequestLine(line string) (method, uri, proto string, ok bool) { + s1 := strings.Index(line, " ") + s2 := strings.Index(line[s1+1:], " ") + if s1 < 0 || s2 < 0 { + return + } + s2 += s1 + 1 + return line[:s1], line[s1+1 : s2], line[s2+1:], true +} + +// HTTP2 parses the frame header of the first frame to detect whether the +// connection is an HTTP2 connection. +func HTTP2() Matcher { + return hasHTTP2Preface +} + +// HTTP1HeaderField returns a matcher matching the header fields of the first +// request of an HTTP 1 connection. +func HTTP1HeaderField(name, value string) Matcher { + return func(r io.Reader) bool { + return matchHTTP1Field(r, name, func(gotValue string) bool { + return gotValue == value + }) + } +} + +// HTTP1HeaderFieldPrefix returns a matcher matching the header fields of the +// first request of an HTTP 1 connection. If the header with key name has a +// value prefixed with valuePrefix, this will match. +func HTTP1HeaderFieldPrefix(name, valuePrefix string) Matcher { + return func(r io.Reader) bool { + return matchHTTP1Field(r, name, func(gotValue string) bool { + return strings.HasPrefix(gotValue, valuePrefix) + }) + } +} + +// HTTP2HeaderField returns a matcher matching the header fields of the first +// headers frame. +func HTTP2HeaderField(name, value string) Matcher { + return func(r io.Reader) bool { + return matchHTTP2Field(ioutil.Discard, r, name, func(gotValue string) bool { + return gotValue == value + }) + } +} + +// HTTP2HeaderFieldPrefix returns a matcher matching the header fields of the +// first headers frame. If the header with key name has a value prefixed with +// valuePrefix, this will match. +func HTTP2HeaderFieldPrefix(name, valuePrefix string) Matcher { + return func(r io.Reader) bool { + return matchHTTP2Field(ioutil.Discard, r, name, func(gotValue string) bool { + return strings.HasPrefix(gotValue, valuePrefix) + }) + } +} + +// HTTP2MatchHeaderFieldSendSettings matches the header field and writes the +// settings to the server. Prefer HTTP2HeaderField over this one, if the client +// does not block on receiving a SETTING frame. +func HTTP2MatchHeaderFieldSendSettings(name, value string) MatchWriter { + return func(w io.Writer, r io.Reader) bool { + return matchHTTP2Field(w, r, name, func(gotValue string) bool { + return gotValue == value + }) + } +} + +// HTTP2MatchHeaderFieldPrefixSendSettings matches the header field prefix +// and writes the settings to the server. Prefer HTTP2HeaderFieldPrefix over +// this one, if the client does not block on receiving a SETTING frame. +func HTTP2MatchHeaderFieldPrefixSendSettings(name, valuePrefix string) MatchWriter { + return func(w io.Writer, r io.Reader) bool { + return matchHTTP2Field(w, r, name, func(gotValue string) bool { + return strings.HasPrefix(gotValue, valuePrefix) + }) + } +} + +func hasHTTP2Preface(r io.Reader) bool { + var b [len(http2.ClientPreface)]byte + last := 0 + + for { + n, err := r.Read(b[last:]) + if err != nil { + return false + } + + last += n + eq := string(b[:last]) == http2.ClientPreface[:last] + if last == len(http2.ClientPreface) { + return eq + } + if !eq { + return false + } + } +} + +func matchHTTP1Field(r io.Reader, name string, matches func(string) bool) (matched bool) { + req, err := http.ReadRequest(bufio.NewReader(r)) + if err != nil { + return false + } + + return matches(req.Header.Get(name)) +} + +func matchHTTP2Field(w io.Writer, r io.Reader, name string, matches func(string) bool) (matched bool) { + if !hasHTTP2Preface(r) { + return false + } + + done := false + framer := http2.NewFramer(w, r) + hdec := hpack.NewDecoder(uint32(4<<10), func(hf hpack.HeaderField) { + if hf.Name == name { + done = true + if matches(hf.Value) { + matched = true + } + } + }) + for { + f, err := framer.ReadFrame() + if err != nil { + return false + } + + switch f := f.(type) { + case *http2.SettingsFrame: + if err := framer.WriteSettings(); err != nil { + return false + } + case *http2.ContinuationFrame: + if _, err := hdec.Write(f.HeaderBlockFragment()); err != nil { + return false + } + done = done || f.FrameHeader.Flags&http2.FlagHeadersEndHeaders != 0 + case *http2.HeadersFrame: + if _, err := hdec.Write(f.HeaderBlockFragment()); err != nil { + return false + } + done = done || f.FrameHeader.Flags&http2.FlagHeadersEndHeaders != 0 + } + + if done { + return matched + } + } +} diff --git a/cmd/vendor/github.com/cockroachdb/cmux/patricia.go b/cmd/vendor/github.com/soheilhy/cmux/patricia.go similarity index 55% rename from cmd/vendor/github.com/cockroachdb/cmux/patricia.go rename to cmd/vendor/github.com/soheilhy/cmux/patricia.go index 56ec4e7b287..c3e3d85bdea 100644 --- a/cmd/vendor/github.com/cockroachdb/cmux/patricia.go +++ b/cmd/vendor/github.com/soheilhy/cmux/patricia.go @@ -1,3 +1,17 @@ +// Copyright 2016 The CMux Authors. 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. + package cmux import ( @@ -8,12 +22,20 @@ import ( // patriciaTree is a simple patricia tree that handles []byte instead of string // and cannot be changed after instantiation. type patriciaTree struct { - root *ptNode + root *ptNode + maxDepth int // max depth of the tree. } -func newPatriciaTree(b ...[]byte) *patriciaTree { +func newPatriciaTree(bs ...[]byte) *patriciaTree { + max := 0 + for _, b := range bs { + if max < len(b) { + max = len(b) + } + } return &patriciaTree{ - root: newNode(b), + root: newNode(bs), + maxDepth: max + 1, } } @@ -22,17 +44,19 @@ func newPatriciaTreeString(strs ...string) *patriciaTree { for i, s := range strs { b[i] = []byte(s) } - return &patriciaTree{ - root: newNode(b), - } + return newPatriciaTree(b...) } func (t *patriciaTree) matchPrefix(r io.Reader) bool { - return t.root.match(r, true) + buf := make([]byte, t.maxDepth) + n, _ := io.ReadFull(r, buf) + return t.root.match(buf[:n], true) } func (t *patriciaTree) match(r io.Reader) bool { - return t.root.match(r, false) + buf := make([]byte, t.maxDepth) + n, _ := io.ReadFull(r, buf) + return t.root.match(buf[:n], false) } type ptNode struct { @@ -122,52 +146,34 @@ func splitPrefix(bss [][]byte) (prefix []byte, rest [][]byte) { return prefix, rest } -func readBytes(r io.Reader, n int) (b []byte, err error) { - b = make([]byte, n) - o := 0 - for o < n { - nr, err := r.Read(b[o:]) - if err != nil && err != io.EOF { - return b, err +func (n *ptNode) match(b []byte, prefix bool) bool { + l := len(n.prefix) + if l > 0 { + if l > len(b) { + l = len(b) } - - o += nr - - if err == io.EOF { - break - } - } - return b[:o], nil -} - -func (n *ptNode) match(r io.Reader, prefix bool) bool { - if l := len(n.prefix); l > 0 { - b, err := readBytes(r, l) - if err != nil || len(b) != l || !bytes.Equal(b, n.prefix) { + if !bytes.Equal(b[:l], n.prefix) { return false } } - if prefix && n.terminal { + if n.terminal && (prefix || len(n.prefix) == len(b)) { return true } - b := make([]byte, 1) - for { - nr, err := r.Read(b) - if nr != 0 { - break - } - - if err == io.EOF { - return n.terminal - } + if l >= len(b) { + return false + } - if err != nil { - return false - } + nextN, ok := n.next[b[l]] + if !ok { + return false } - nextN, ok := n.next[b[0]] - return ok && nextN.match(r, prefix) + if l == len(b) { + b = b[l:l] + } else { + b = b[l+1:] + } + return nextN.match(b, prefix) } diff --git a/embed/config.go b/embed/config.go index 8e40eb3601f..9f1516b6535 100644 --- a/embed/config.go +++ b/embed/config.go @@ -22,6 +22,7 @@ import ( "net/url" "path/filepath" "strings" + "time" "github.com/coreos/etcd/etcdserver" "github.com/coreos/etcd/pkg/cors" @@ -140,6 +141,10 @@ type Config struct { // auth AuthToken string `json:"auth-token"` + + // Experimental flags + + ExperimentalCorruptCheckTime time.Duration `json:"experimental-corrupt-check-time"` } // configYAML holds the config suitable for yaml parsing diff --git a/embed/etcd.go b/embed/etcd.go index e69adbfd63b..5959f405976 100644 --- a/embed/etcd.go +++ b/embed/etcd.go @@ -16,6 +16,7 @@ package embed import ( "context" + "crypto/tls" "fmt" "io/ioutil" defaultLog "log" @@ -28,6 +29,7 @@ import ( "github.com/coreos/etcd/etcdserver" "github.com/coreos/etcd/etcdserver/api/etcdhttp" "github.com/coreos/etcd/etcdserver/api/v2http" + "github.com/coreos/etcd/etcdserver/api/v3rpc" "github.com/coreos/etcd/pkg/cors" "github.com/coreos/etcd/pkg/debugutil" runtimeutil "github.com/coreos/etcd/pkg/runtime" @@ -35,6 +37,9 @@ import ( "github.com/coreos/etcd/pkg/types" "github.com/coreos/etcd/rafthttp" "github.com/coreos/pkg/capnslog" + + "github.com/soheilhy/cmux" + "google.golang.org/grpc" ) var plog = capnslog.NewPackageLogger("github.com/coreos/etcd", "embed") @@ -146,35 +151,46 @@ func StartEtcd(inCfg *Config) (e *Etcd, err error) { StrictReconfigCheck: cfg.StrictReconfigCheck, ClientCertAuthEnabled: cfg.ClientTLSInfo.ClientCertAuth, AuthToken: cfg.AuthToken, + CorruptCheckTime: cfg.ExperimentalCorruptCheckTime, } if e.Server, err = etcdserver.NewServer(srvcfg); err != nil { return } + // buffer channel so goroutines on closed connections won't wait forever + e.errc = make(chan error, len(e.Peers)+len(e.Clients)+2*len(e.sctxs)) + + e.Server.Start() + // configure peer handlers after rafthttp.Transport started ph := etcdhttp.NewPeerHandler(e.Server) + var peerTLScfg *tls.Config + if !cfg.PeerTLSInfo.Empty() { + if peerTLScfg, err = cfg.PeerTLSInfo.ServerConfig(); err != nil { + return + } + } for _, p := range e.Peers { + gs := v3rpc.Server(e.Server, peerTLScfg) + m := cmux.New(p.Listener) + go gs.Serve(m.Match(cmux.HTTP2())) srv := &http.Server{ - Handler: ph, + Handler: grpcHandlerFunc(gs, ph), ReadTimeout: 5 * time.Minute, ErrorLog: defaultLog.New(ioutil.Discard, "", 0), // do not log user error } - - l := p.Listener - p.serve = func() error { return srv.Serve(l) } + go srv.Serve(m.Match(cmux.Any())) + p.serve = func() error { return m.Serve() } p.close = func(ctx context.Context) error { // gracefully shutdown http.Server // close open listeners, idle connections // until context cancel or time-out + e.stopGRPCServer(gs) return srv.Shutdown(ctx) } } - // buffer channel so goroutines on closed connections won't wait forever - e.errc = make(chan error, len(e.Peers)+len(e.Clients)+2*len(e.sctxs)) - - e.Server.Start() if err = e.serve(); err != nil { return } @@ -190,29 +206,9 @@ func (e *Etcd) Config() Config { func (e *Etcd) Close() { e.closeOnce.Do(func() { close(e.stopc) }) - timeout := 2 * time.Second - if e.Server != nil { - timeout = e.Server.Cfg.ReqTimeout() - } for _, sctx := range e.sctxs { for gs := range sctx.grpcServerC { - ch := make(chan struct{}) - go func() { - defer close(ch) - // close listeners to stop accepting new connections, - // will block on any existing transports - gs.GracefulStop() - }() - // wait until all pending RPCs are finished - select { - case <-ch: - case <-time.After(timeout): - // took too long, manually close open transports - // e.g. watch streams - gs.Stop() - // concurrent GracefulStop should be interrupted - <-ch - } + e.stopGRPCServer(gs) } } @@ -243,6 +239,30 @@ func (e *Etcd) Close() { } } +func (e *Etcd) stopGRPCServer(gs *grpc.Server) { + timeout := 2 * time.Second + if e.Server != nil { + timeout = e.Server.Cfg.ReqTimeout() + } + ch := make(chan struct{}) + go func() { + defer close(ch) + // close listeners to stop accepting new connections, + // will block on any existing transports + gs.GracefulStop() + }() + // wait until all pending RPCs are finished + select { + case <-ch: + case <-time.After(timeout): + // took too long, manually close open transports + // e.g. watch streams + gs.Stop() + // concurrent GracefulStop should be interrupted + <-ch + } +} + func (e *Etcd) Err() <-chan error { return e.errc } func startPeerListeners(cfg *Config) (peers []*peerListener, err error) { diff --git a/embed/serve.go b/embed/serve.go index 3e9b37ea070..6af2374327c 100644 --- a/embed/serve.go +++ b/embed/serve.go @@ -34,8 +34,8 @@ import ( "github.com/coreos/etcd/pkg/debugutil" "github.com/coreos/etcd/pkg/transport" - "github.com/cockroachdb/cmux" gw "github.com/grpc-ecosystem/grpc-gateway/runtime" + "github.com/soheilhy/cmux" "golang.org/x/net/context" "golang.org/x/net/trace" "google.golang.org/grpc" @@ -152,7 +152,7 @@ func (sctx *serveCtx) serve(s *etcdserver.EtcdServer, tlsinfo *transport.TLSInfo } // grpcHandlerFunc returns an http.Handler that delegates to grpcServer on incoming gRPC -// connections or otherHandler otherwise. Copied from cockroachdb. +// connections or otherHandler otherwise. Given in gRPC docs. func grpcHandlerFunc(grpcServer *grpc.Server, otherHandler http.Handler) http.Handler { if otherHandler == nil { return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { diff --git a/etcdmain/config.go b/etcdmain/config.go index 4bc900bc1ed..4b72649b061 100644 --- a/etcdmain/config.go +++ b/etcdmain/config.go @@ -216,6 +216,9 @@ func newConfig() *config { // auth fs.StringVar(&cfg.AuthToken, "auth-token", cfg.AuthToken, "Specify auth token specific options.") + // experimental + fs.DurationVar(&cfg.ExperimentalCorruptCheckTime, "experimental-corrupt-check-time", cfg.ExperimentalCorruptCheckTime, "Duration of time between cluster corruption check passes.") + // ignored for _, f := range cfg.ignored { fs.Var(&flags.IgnoredFlag{Name: f}, f, "") diff --git a/etcdmain/grpc_proxy.go b/etcdmain/grpc_proxy.go index 183b20c19fb..27dcac375d1 100644 --- a/etcdmain/grpc_proxy.go +++ b/etcdmain/grpc_proxy.go @@ -37,8 +37,8 @@ import ( "github.com/coreos/etcd/pkg/transport" "github.com/coreos/etcd/proxy/grpcproxy" - "github.com/cockroachdb/cmux" grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus" + "github.com/soheilhy/cmux" "github.com/spf13/cobra" "google.golang.org/grpc" ) diff --git a/etcdmain/help.go b/etcdmain/help.go index ad4d30240ee..152fbb649e3 100644 --- a/etcdmain/help.go +++ b/etcdmain/help.go @@ -179,5 +179,9 @@ profiling flags: auth flags: --auth-token 'simple' Specify a v3 authentication token type and its options ('simple' or 'jwt'). + +experimental flags: + --experimental-corrupt-check-time '0s' + duration of time between cluster corruption check passes. ` ) diff --git a/etcdserver/api/v3rpc/rpctypes/error.go b/etcdserver/api/v3rpc/rpctypes/error.go index 719ad8157a7..cc8ebbdb852 100644 --- a/etcdserver/api/v3rpc/rpctypes/error.go +++ b/etcdserver/api/v3rpc/rpctypes/error.go @@ -66,6 +66,7 @@ var ( ErrGRPCTimeoutDueToLeaderFail = grpc.Errorf(codes.Unavailable, "etcdserver: request timed out, possibly due to previous leader failure") ErrGRPCTimeoutDueToConnectionLost = grpc.Errorf(codes.Unavailable, "etcdserver: request timed out, possibly due to connection lost") ErrGRPCUnhealthy = grpc.Errorf(codes.Unavailable, "etcdserver: unhealthy cluster") + ErrGRPCCorrupt = grpc.Errorf(codes.DataLoss, "etcdserver: corrupt cluster") errStringToError = map[string]error{ grpc.ErrorDesc(ErrGRPCEmptyKey): ErrGRPCEmptyKey, @@ -114,6 +115,7 @@ var ( grpc.ErrorDesc(ErrGRPCTimeoutDueToLeaderFail): ErrGRPCTimeoutDueToLeaderFail, grpc.ErrorDesc(ErrGRPCTimeoutDueToConnectionLost): ErrGRPCTimeoutDueToConnectionLost, grpc.ErrorDesc(ErrGRPCUnhealthy): ErrGRPCUnhealthy, + grpc.ErrorDesc(ErrGRPCCorrupt): ErrGRPCCorrupt, } // client-side error @@ -162,6 +164,7 @@ var ( ErrTimeoutDueToLeaderFail = Error(ErrGRPCTimeoutDueToLeaderFail) ErrTimeoutDueToConnectionLost = Error(ErrGRPCTimeoutDueToConnectionLost) ErrUnhealthy = Error(ErrGRPCUnhealthy) + ErrCorrupt = Error(ErrGRPCCorrupt) ) // EtcdError defines gRPC server errors. diff --git a/etcdserver/api/v3rpc/util.go b/etcdserver/api/v3rpc/util.go index ecbd1762378..9724b093b69 100644 --- a/etcdserver/api/v3rpc/util.go +++ b/etcdserver/api/v3rpc/util.go @@ -46,6 +46,7 @@ var toGRPCErrorMap = map[error]error{ etcdserver.ErrTimeoutDueToConnectionLost: rpctypes.ErrGRPCTimeoutDueToConnectionLost, etcdserver.ErrUnhealthy: rpctypes.ErrGRPCUnhealthy, etcdserver.ErrKeyNotFound: rpctypes.ErrGRPCKeyNotFound, + etcdserver.ErrCorrupt: rpctypes.ErrGRPCCorrupt, lease.ErrLeaseNotFound: rpctypes.ErrGRPCLeaseNotFound, lease.ErrLeaseExists: rpctypes.ErrGRPCLeaseExist, diff --git a/etcdserver/apply.go b/etcdserver/apply.go index 60ff1092ea2..26b17f14194 100644 --- a/etcdserver/apply.go +++ b/etcdserver/apply.go @@ -577,9 +577,11 @@ func (a *applierV3backend) Alarm(ar *pb.AlarmRequest) (*pb.AlarmResponse, error) break } + plog.Warningf("alarm %v raised by peer %s", m.Alarm, types.ID(m.MemberID)) switch m.Alarm { + case pb.AlarmType_CORRUPT: + a.s.applyV3 = newApplierV3Corrupt(a) case pb.AlarmType_NOSPACE: - plog.Warningf("alarm raised %+v", m) a.s.applyV3 = newApplierV3Capped(a) default: plog.Errorf("unimplemented alarm activation (%+v)", m) @@ -596,7 +598,8 @@ func (a *applierV3backend) Alarm(ar *pb.AlarmRequest) (*pb.AlarmResponse, error) } switch m.Alarm { - case pb.AlarmType_NOSPACE: + case pb.AlarmType_NOSPACE, pb.AlarmType_CORRUPT: + // TODO: check kv hash before deactivating CORRUPT? plog.Infof("alarm disarmed %+v", ar) a.s.applyV3 = a.s.newApplierV3() default: diff --git a/etcdserver/config.go b/etcdserver/config.go index f6ed1f1bae7..1e39369026c 100644 --- a/etcdserver/config.go +++ b/etcdserver/config.go @@ -66,6 +66,8 @@ type ServerConfig struct { ClientCertAuthEnabled bool AuthToken string + + CorruptCheckTime time.Duration } // VerifyBootstrap sanity-checks the initial config for bootstrap case diff --git a/etcdserver/corrupt.go b/etcdserver/corrupt.go new file mode 100644 index 00000000000..b1602de1ebd --- /dev/null +++ b/etcdserver/corrupt.go @@ -0,0 +1,182 @@ +// Copyright 2017 The etcd Authors +// +// 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 etcdserver + +import ( + "time" + + "github.com/coreos/etcd/clientv3" + pb "github.com/coreos/etcd/etcdserver/etcdserverpb" + "github.com/coreos/etcd/mvcc" + "github.com/coreos/etcd/pkg/types" + + "golang.org/x/net/context" +) + +func (s *EtcdServer) monitorKVHash() { + t := s.Cfg.CorruptCheckTime + if t == 0 { + return + } + plog.Infof("enabled corruption checking with %s interval", t) + for { + select { + case <-s.stopping: + return + case <-time.After(t): + } + if !s.isLeader() { + continue + } + if err := s.checkHashKV(); err != nil { + plog.Debugf("check hash kv failed %v", err) + } + } +} + +func (s *EtcdServer) checkHashKV() error { + h, rev, crev, err := s.kv.HashByRev(0) + if err != nil { + plog.Fatalf("failed to hash kv store (%v)", err) + } + resps := []*clientv3.HashKVResponse{} + for _, m := range s.cluster.Members() { + if m.ID == s.ID() { + continue + } + + cli, cerr := clientv3.New(clientv3.Config{Endpoints: m.PeerURLs}) + if cerr != nil { + continue + } + + respsLen := len(resps) + for _, c := range cli.Endpoints() { + ctx, cancel := context.WithTimeout(context.Background(), s.Cfg.ReqTimeout()) + resp, herr := cli.HashKV(ctx, c, rev) + cancel() + if herr == nil { + cerr = herr + resps = append(resps, resp) + break + } + } + cli.Close() + + if respsLen == len(resps) { + plog.Warningf("failed to hash kv for peer %s (%v)", types.ID(m.ID), cerr) + } + } + + ctx, cancel := context.WithTimeout(context.Background(), s.Cfg.ReqTimeout()) + err = s.linearizableReadNotify(ctx) + cancel() + if err != nil { + return err + } + + h2, rev2, crev2, err := s.kv.HashByRev(0) + if err != nil { + plog.Warningf("failed to hash kv store (%v)", err) + return err + } + + alarmed := false + mismatch := func(id uint64) { + if alarmed { + return + } + alarmed = true + a := &pb.AlarmRequest{ + MemberID: uint64(id), + Action: pb.AlarmRequest_ACTIVATE, + Alarm: pb.AlarmType_CORRUPT, + } + s.goAttach(func() { + s.raftRequest(s.ctx, pb.InternalRaftRequest{Alarm: a}) + }) + } + + if h2 != h && rev2 == rev && crev == crev2 { + plog.Warningf("mismatched hashes %d and %d for revision %d", h, h2, rev) + mismatch(uint64(s.ID())) + } + + for _, resp := range resps { + id := resp.Header.MemberId + if resp.Header.Revision > rev2 { + plog.Warningf( + "revision %d from member %v, expected at most %d", + resp.Header.Revision, + types.ID(id), + rev2) + mismatch(id) + } + if resp.CompactRevision > crev2 { + plog.Warningf( + "compact revision %d from member %v, expected at most %d", + resp.CompactRevision, + types.ID(id), + crev2, + ) + mismatch(id) + } + if resp.CompactRevision == crev && resp.Hash != h { + plog.Warningf( + "hash %d at revision %d from member %v, expected hash %d", + resp.Hash, + rev, + types.ID(id), + h, + ) + mismatch(id) + } + } + return nil +} + +type applierV3Corrupt struct { + applierV3 +} + +func newApplierV3Corrupt(a applierV3) *applierV3Corrupt { return &applierV3Corrupt{a} } + +func (a *applierV3Corrupt) Put(txn mvcc.TxnWrite, p *pb.PutRequest) (*pb.PutResponse, error) { + return nil, ErrCorrupt +} + +func (a *applierV3Corrupt) Range(txn mvcc.TxnRead, p *pb.RangeRequest) (*pb.RangeResponse, error) { + return nil, ErrCorrupt +} + +func (a *applierV3Corrupt) DeleteRange(txn mvcc.TxnWrite, p *pb.DeleteRangeRequest) (*pb.DeleteRangeResponse, error) { + return nil, ErrCorrupt +} + +func (a *applierV3Corrupt) Txn(rt *pb.TxnRequest) (*pb.TxnResponse, error) { + return nil, ErrCorrupt +} + +func (a *applierV3Corrupt) Compaction(compaction *pb.CompactionRequest) (*pb.CompactionResponse, <-chan struct{}, error) { + return nil, nil, ErrCorrupt +} + +func (a *applierV3Corrupt) LeaseGrant(lc *pb.LeaseGrantRequest) (*pb.LeaseGrantResponse, error) { + return nil, ErrCorrupt +} + +func (a *applierV3Corrupt) LeaseRevoke(lc *pb.LeaseRevokeRequest) (*pb.LeaseRevokeResponse, error) { + return nil, ErrCorrupt +} diff --git a/etcdserver/errors.go b/etcdserver/errors.go index 09571e56e31..fb93c4b2a1d 100644 --- a/etcdserver/errors.go +++ b/etcdserver/errors.go @@ -35,6 +35,7 @@ var ( ErrTooManyRequests = errors.New("etcdserver: too many requests") ErrUnhealthy = errors.New("etcdserver: unhealthy cluster") ErrKeyNotFound = errors.New("etcdserver: key not found") + ErrCorrupt = errors.New("etcdserver: corrupt cluster") ) type DiscoveryError struct { diff --git a/etcdserver/etcdserverpb/rpc.pb.go b/etcdserver/etcdserverpb/rpc.pb.go index 7e32e0ffe46..97e0c4c49e0 100644 --- a/etcdserver/etcdserverpb/rpc.pb.go +++ b/etcdserver/etcdserverpb/rpc.pb.go @@ -32,15 +32,18 @@ type AlarmType int32 const ( AlarmType_NONE AlarmType = 0 AlarmType_NOSPACE AlarmType = 1 + AlarmType_CORRUPT AlarmType = 2 ) var AlarmType_name = map[int32]string{ 0: "NONE", 1: "NOSPACE", + 2: "CORRUPT", } var AlarmType_value = map[string]int32{ "NONE": 0, "NOSPACE": 1, + "CORRUPT": 2, } func (x AlarmType) String() string { @@ -18445,234 +18448,235 @@ var ( func init() { proto.RegisterFile("rpc.proto", fileDescriptorRpc) } var fileDescriptorRpc = []byte{ - // 3663 bytes of a gzipped FileDescriptorProto + // 3674 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5b, 0x5b, 0x6f, 0x1b, 0xc7, - 0xf5, 0xd7, 0x92, 0x22, 0x29, 0x1e, 0x5e, 0x44, 0x8d, 0x64, 0x9b, 0xa2, 0x6d, 0x59, 0x1e, 0xdf, - 0x64, 0x3b, 0x96, 0x12, 0x25, 0xff, 0xff, 0x83, 0x5b, 0x04, 0x91, 0x25, 0xc6, 0x52, 0x24, 0x4b, - 0xce, 0x4a, 0x56, 0x52, 0x20, 0x28, 0xb1, 0x22, 0xc7, 0xd2, 0x42, 0xe4, 0x2e, 0xb3, 0xbb, 0xa4, + 0x77, 0xd7, 0x92, 0x22, 0x29, 0x1e, 0x5e, 0x44, 0x8d, 0x64, 0x9b, 0xa2, 0x6d, 0x59, 0x1e, 0xdf, + 0x64, 0x3b, 0x16, 0xff, 0x7f, 0x25, 0xed, 0x83, 0x5b, 0x04, 0x91, 0x25, 0xc6, 0x52, 0x24, 0x4b, + 0xca, 0x8a, 0x56, 0x52, 0x20, 0x28, 0xb1, 0x22, 0xc7, 0xd2, 0x42, 0xe4, 0x2e, 0xb3, 0xbb, 0xa4, 0xa5, 0x34, 0x2d, 0x8a, 0x20, 0x41, 0xd1, 0x02, 0x7d, 0x69, 0x1e, 0x7a, 0x7b, 0x2c, 0x8a, 0x22, 0x2f, 0x7d, 0x2b, 0xfa, 0x15, 0x8a, 0xbe, 0xb4, 0x40, 0xbf, 0x40, 0x91, 0xf6, 0xa5, 0xdf, 0xa1, - 0x45, 0x8b, 0xb9, 0xed, 0xce, 0x2e, 0x77, 0x29, 0x25, 0x4c, 0xf2, 0x22, 0xef, 0x9c, 0x39, 0x73, - 0xce, 0x99, 0x33, 0x73, 0xce, 0x99, 0xf9, 0x0d, 0x0d, 0x79, 0xa7, 0xdb, 0x5c, 0xec, 0x3a, 0xb6, - 0x67, 0xa3, 0x22, 0xf1, 0x9a, 0x2d, 0x97, 0x38, 0x7d, 0xe2, 0x74, 0x0f, 0x6a, 0x33, 0x87, 0xf6, - 0xa1, 0xcd, 0x3a, 0x96, 0xe8, 0x17, 0xe7, 0xa9, 0xcd, 0x52, 0x9e, 0xa5, 0x4e, 0xbf, 0xd9, 0x64, - 0x7f, 0xba, 0x07, 0x4b, 0xc7, 0x7d, 0xd1, 0x75, 0x99, 0x75, 0x19, 0x3d, 0xef, 0x88, 0xfd, 0xe9, - 0x1e, 0xb0, 0x7f, 0x44, 0xe7, 0x95, 0x43, 0xdb, 0x3e, 0x6c, 0x93, 0x25, 0xa3, 0x6b, 0x2e, 0x19, - 0x96, 0x65, 0x7b, 0x86, 0x67, 0xda, 0x96, 0xcb, 0x7b, 0xf1, 0x27, 0x1a, 0x94, 0x75, 0xe2, 0x76, - 0x6d, 0xcb, 0x25, 0xeb, 0xc4, 0x68, 0x11, 0x07, 0x5d, 0x05, 0x68, 0xb6, 0x7b, 0xae, 0x47, 0x9c, - 0x86, 0xd9, 0xaa, 0x6a, 0xf3, 0xda, 0xc2, 0xb8, 0x9e, 0x17, 0x94, 0x8d, 0x16, 0xba, 0x0c, 0xf9, - 0x0e, 0xe9, 0x1c, 0xf0, 0xde, 0x14, 0xeb, 0x9d, 0xe0, 0x84, 0x8d, 0x16, 0xaa, 0xc1, 0x84, 0x43, - 0xfa, 0xa6, 0x6b, 0xda, 0x56, 0x35, 0x3d, 0xaf, 0x2d, 0xa4, 0x75, 0xbf, 0x4d, 0x07, 0x3a, 0xc6, - 0x73, 0xaf, 0xe1, 0x11, 0xa7, 0x53, 0x1d, 0xe7, 0x03, 0x29, 0x61, 0x8f, 0x38, 0x1d, 0xfc, 0x71, - 0x06, 0x8a, 0xba, 0x61, 0x1d, 0x12, 0x9d, 0xbc, 0xdf, 0x23, 0xae, 0x87, 0x2a, 0x90, 0x3e, 0x26, - 0xa7, 0x4c, 0x7d, 0x51, 0xa7, 0x9f, 0x7c, 0xbc, 0x75, 0x48, 0x1a, 0xc4, 0xe2, 0x8a, 0x8b, 0x74, - 0xbc, 0x75, 0x48, 0xea, 0x56, 0x0b, 0xcd, 0x40, 0xa6, 0x6d, 0x76, 0x4c, 0x4f, 0x68, 0xe5, 0x8d, - 0x90, 0x39, 0xe3, 0x11, 0x73, 0x56, 0x01, 0x5c, 0xdb, 0xf1, 0x1a, 0xb6, 0xd3, 0x22, 0x4e, 0x35, - 0x33, 0xaf, 0x2d, 0x94, 0x97, 0x6f, 0x2e, 0xaa, 0x0b, 0xb1, 0xa8, 0x1a, 0xb4, 0xb8, 0x6b, 0x3b, - 0xde, 0x0e, 0xe5, 0xd5, 0xf3, 0xae, 0xfc, 0x44, 0x6f, 0x42, 0x81, 0x09, 0xf1, 0x0c, 0xe7, 0x90, - 0x78, 0xd5, 0x2c, 0x93, 0x72, 0xeb, 0x0c, 0x29, 0x7b, 0x8c, 0x59, 0x67, 0xea, 0xf9, 0x37, 0xc2, - 0x50, 0x74, 0x89, 0x63, 0x1a, 0x6d, 0xf3, 0x03, 0xe3, 0xa0, 0x4d, 0xaa, 0xb9, 0x79, 0x6d, 0x61, - 0x42, 0x0f, 0xd1, 0xe8, 0xfc, 0x8f, 0xc9, 0xa9, 0xdb, 0xb0, 0xad, 0xf6, 0x69, 0x75, 0x82, 0x31, - 0x4c, 0x50, 0xc2, 0x8e, 0xd5, 0x3e, 0x65, 0x8b, 0x66, 0xf7, 0x2c, 0x8f, 0xf7, 0xe6, 0x59, 0x6f, - 0x9e, 0x51, 0x58, 0xf7, 0x02, 0x54, 0x3a, 0xa6, 0xd5, 0xe8, 0xd8, 0xad, 0x86, 0xef, 0x10, 0x60, - 0x0e, 0x29, 0x77, 0x4c, 0xeb, 0x89, 0xdd, 0xd2, 0xa5, 0x5b, 0x28, 0xa7, 0x71, 0x12, 0xe6, 0x2c, - 0x08, 0x4e, 0xe3, 0x44, 0xe5, 0x5c, 0x84, 0x69, 0x2a, 0xb3, 0xe9, 0x10, 0xc3, 0x23, 0x01, 0x73, - 0x91, 0x31, 0x4f, 0x75, 0x4c, 0x6b, 0x95, 0xf5, 0x84, 0xf8, 0x8d, 0x93, 0x01, 0xfe, 0x92, 0xe0, - 0x37, 0x4e, 0xc2, 0xfc, 0x78, 0x11, 0xf2, 0xbe, 0xcf, 0xd1, 0x04, 0x8c, 0x6f, 0xef, 0x6c, 0xd7, - 0x2b, 0x63, 0x08, 0x20, 0xbb, 0xb2, 0xbb, 0x5a, 0xdf, 0x5e, 0xab, 0x68, 0xa8, 0x00, 0xb9, 0xb5, - 0x3a, 0x6f, 0xa4, 0xf0, 0x23, 0x80, 0xc0, 0xbb, 0x28, 0x07, 0xe9, 0xcd, 0xfa, 0x77, 0x2a, 0x63, - 0x94, 0x67, 0xbf, 0xae, 0xef, 0x6e, 0xec, 0x6c, 0x57, 0x34, 0x3a, 0x78, 0x55, 0xaf, 0xaf, 0xec, - 0xd5, 0x2b, 0x29, 0xca, 0xf1, 0x64, 0x67, 0xad, 0x92, 0x46, 0x79, 0xc8, 0xec, 0xaf, 0x6c, 0x3d, - 0xab, 0x57, 0xc6, 0xf1, 0xa7, 0x1a, 0x94, 0xc4, 0x7a, 0xf1, 0x98, 0x40, 0xaf, 0x41, 0xf6, 0x88, - 0xc5, 0x05, 0xdb, 0x8a, 0x85, 0xe5, 0x2b, 0x91, 0xc5, 0x0d, 0xc5, 0x8e, 0x2e, 0x78, 0x11, 0x86, - 0xf4, 0x71, 0xdf, 0xad, 0xa6, 0xe6, 0xd3, 0x0b, 0x85, 0xe5, 0xca, 0x22, 0x0f, 0xd8, 0xc5, 0x4d, - 0x72, 0xba, 0x6f, 0xb4, 0x7b, 0x44, 0xa7, 0x9d, 0x08, 0xc1, 0x78, 0xc7, 0x76, 0x08, 0xdb, 0xb1, - 0x13, 0x3a, 0xfb, 0xa6, 0xdb, 0x98, 0x2d, 0x9a, 0xd8, 0xad, 0xbc, 0x81, 0x3f, 0xd3, 0x00, 0x9e, - 0xf6, 0xbc, 0xe4, 0xd0, 0x98, 0x81, 0x4c, 0x9f, 0x0a, 0x16, 0x61, 0xc1, 0x1b, 0x2c, 0x26, 0x88, - 0xe1, 0x12, 0x3f, 0x26, 0x68, 0x03, 0x5d, 0x82, 0x5c, 0xd7, 0x21, 0xfd, 0xc6, 0x71, 0x9f, 0x29, - 0x99, 0xd0, 0xb3, 0xb4, 0xb9, 0xd9, 0x47, 0xd7, 0xa1, 0x68, 0x1e, 0x5a, 0xb6, 0x43, 0x1a, 0x5c, - 0x56, 0x86, 0xf5, 0x16, 0x38, 0x8d, 0xd9, 0xad, 0xb0, 0x70, 0xc1, 0x59, 0x95, 0x65, 0x8b, 0x92, - 0xb0, 0x05, 0x05, 0x66, 0xea, 0x48, 0xee, 0xbb, 0x1b, 0xd8, 0x98, 0x62, 0xc3, 0x06, 0x5d, 0x28, - 0xac, 0xc6, 0xef, 0x01, 0x5a, 0x23, 0x6d, 0xe2, 0x91, 0x51, 0xb2, 0x87, 0xe2, 0x93, 0xb4, 0xea, - 0x13, 0xfc, 0x33, 0x0d, 0xa6, 0x43, 0xe2, 0x47, 0x9a, 0x56, 0x15, 0x72, 0x2d, 0x26, 0x8c, 0x5b, - 0x90, 0xd6, 0x65, 0x13, 0xdd, 0x87, 0x09, 0x61, 0x80, 0x5b, 0x4d, 0x27, 0x6c, 0x9a, 0x1c, 0xb7, - 0xc9, 0xc5, 0x9f, 0xa5, 0x20, 0x2f, 0x26, 0xba, 0xd3, 0x45, 0x2b, 0x50, 0x72, 0x78, 0xa3, 0xc1, - 0xe6, 0x23, 0x2c, 0xaa, 0x25, 0x27, 0xa1, 0xf5, 0x31, 0xbd, 0x28, 0x86, 0x30, 0x32, 0xfa, 0x16, - 0x14, 0xa4, 0x88, 0x6e, 0xcf, 0x13, 0x2e, 0xaf, 0x86, 0x05, 0x04, 0xfb, 0x6f, 0x7d, 0x4c, 0x07, - 0xc1, 0xfe, 0xb4, 0xe7, 0xa1, 0x3d, 0x98, 0x91, 0x83, 0xf9, 0x6c, 0x84, 0x19, 0x69, 0x26, 0x65, - 0x3e, 0x2c, 0x65, 0x70, 0xa9, 0xd6, 0xc7, 0x74, 0x24, 0xc6, 0x2b, 0x9d, 0xaa, 0x49, 0xde, 0x09, - 0x4f, 0xde, 0x03, 0x26, 0xed, 0x9d, 0x58, 0x83, 0x26, 0xed, 0x9d, 0x58, 0x8f, 0xf2, 0x90, 0x13, - 0x2d, 0xfc, 0xc7, 0x14, 0x80, 0x5c, 0x8d, 0x9d, 0x2e, 0x5a, 0x83, 0xb2, 0x23, 0x5a, 0x21, 0x6f, - 0x5d, 0x8e, 0xf5, 0x96, 0x58, 0xc4, 0x31, 0xbd, 0x24, 0x07, 0x71, 0xe3, 0x5e, 0x87, 0xa2, 0x2f, - 0x25, 0x70, 0xd8, 0x6c, 0x8c, 0xc3, 0x7c, 0x09, 0x05, 0x39, 0x80, 0xba, 0xec, 0x1d, 0xb8, 0xe0, - 0x8f, 0x8f, 0xf1, 0xd9, 0xf5, 0x21, 0x3e, 0xf3, 0x05, 0x4e, 0x4b, 0x09, 0xaa, 0xd7, 0x54, 0xc3, - 0x02, 0xb7, 0xcd, 0xc6, 0xb8, 0x6d, 0xd0, 0x30, 0xea, 0x38, 0xa0, 0xf5, 0x92, 0x37, 0xf1, 0xbf, - 0xd2, 0x90, 0x5b, 0xb5, 0x3b, 0x5d, 0xc3, 0xa1, 0xab, 0x91, 0x75, 0x88, 0xdb, 0x6b, 0x7b, 0xcc, - 0x5d, 0xe5, 0xe5, 0x1b, 0x61, 0x89, 0x82, 0x4d, 0xfe, 0xab, 0x33, 0x56, 0x5d, 0x0c, 0xa1, 0x83, - 0x45, 0x79, 0x4c, 0x9d, 0x63, 0xb0, 0x28, 0x8e, 0x62, 0x88, 0x0c, 0xe4, 0x74, 0x10, 0xc8, 0x35, - 0xc8, 0xf5, 0x89, 0x13, 0x94, 0xf4, 0xf5, 0x31, 0x5d, 0x12, 0xd0, 0x5d, 0x98, 0x8c, 0x96, 0x97, - 0x8c, 0xe0, 0x29, 0x37, 0xc3, 0xd5, 0xe8, 0x06, 0x14, 0x43, 0x35, 0x2e, 0x2b, 0xf8, 0x0a, 0x1d, - 0xa5, 0xc4, 0x5d, 0x94, 0x79, 0x95, 0xd6, 0xe3, 0xe2, 0xfa, 0x98, 0xcc, 0xac, 0x17, 0x65, 0x66, - 0x9d, 0x10, 0xa3, 0x44, 0x6e, 0x0d, 0x25, 0x99, 0x37, 0xc2, 0x49, 0x06, 0xbf, 0x01, 0xa5, 0x90, - 0x83, 0x68, 0xdd, 0xa9, 0xbf, 0xfd, 0x6c, 0x65, 0x8b, 0x17, 0xa9, 0xc7, 0xac, 0x2e, 0xe9, 0x15, - 0x8d, 0xd6, 0xba, 0xad, 0xfa, 0xee, 0x6e, 0x25, 0x85, 0x4a, 0x90, 0xdf, 0xde, 0xd9, 0x6b, 0x70, - 0xae, 0x34, 0x7e, 0xec, 0x4b, 0x10, 0x45, 0x4e, 0xa9, 0x6d, 0x63, 0x4a, 0x6d, 0xd3, 0x64, 0x6d, - 0x4b, 0x05, 0xb5, 0x8d, 0x95, 0xb9, 0xad, 0xfa, 0xca, 0x6e, 0xbd, 0x32, 0xfe, 0xa8, 0x0c, 0x45, - 0xee, 0xdf, 0x46, 0xcf, 0xa2, 0xa5, 0xf6, 0x37, 0x1a, 0x40, 0x10, 0x4d, 0x68, 0x09, 0x72, 0x4d, - 0xae, 0xa7, 0xaa, 0xb1, 0x64, 0x74, 0x21, 0x76, 0xc9, 0x74, 0xc9, 0x85, 0x5e, 0x81, 0x9c, 0xdb, - 0x6b, 0x36, 0x89, 0x2b, 0x4b, 0xde, 0xa5, 0x68, 0x3e, 0x14, 0xd9, 0x4a, 0x97, 0x7c, 0x74, 0xc8, - 0x73, 0xc3, 0x6c, 0xf7, 0x58, 0x01, 0x1c, 0x3e, 0x44, 0xf0, 0xe1, 0x5f, 0x6a, 0x50, 0x50, 0x36, - 0xef, 0x97, 0x4c, 0xc2, 0x57, 0x20, 0xcf, 0x6c, 0x20, 0x2d, 0x91, 0x86, 0x27, 0xf4, 0x80, 0x80, - 0xfe, 0x1f, 0xf2, 0x32, 0x02, 0x64, 0x26, 0xae, 0xc6, 0x8b, 0xdd, 0xe9, 0xea, 0x01, 0x2b, 0xde, - 0x84, 0x29, 0xe6, 0x95, 0x26, 0x3d, 0x5c, 0x4b, 0x3f, 0xaa, 0xc7, 0x4f, 0x2d, 0x72, 0xfc, 0xac, - 0xc1, 0x44, 0xf7, 0xe8, 0xd4, 0x35, 0x9b, 0x46, 0x5b, 0x58, 0xe1, 0xb7, 0xf1, 0x5b, 0x80, 0x54, - 0x61, 0xa3, 0x4c, 0x17, 0x97, 0xa0, 0xb0, 0x6e, 0xb8, 0x47, 0xc2, 0x24, 0x7c, 0x1f, 0x4a, 0xb4, - 0xb9, 0xb9, 0x7f, 0x0e, 0x1b, 0xd9, 0xe5, 0x40, 0x72, 0x8f, 0xe4, 0x73, 0x04, 0xe3, 0x47, 0x86, - 0x7b, 0xc4, 0x26, 0x5a, 0xd2, 0xd9, 0x37, 0xba, 0x0b, 0x95, 0x26, 0x9f, 0x64, 0x23, 0x72, 0x65, - 0x98, 0x14, 0x74, 0xff, 0x24, 0xf8, 0x2e, 0x14, 0xf9, 0x1c, 0xbe, 0x6a, 0x23, 0xf0, 0x14, 0x4c, - 0xee, 0x5a, 0x46, 0xd7, 0x3d, 0xb2, 0x65, 0x75, 0xa3, 0x93, 0xae, 0x04, 0xb4, 0x91, 0x34, 0xde, - 0x81, 0x49, 0x87, 0x74, 0x0c, 0xd3, 0x32, 0xad, 0xc3, 0xc6, 0xc1, 0xa9, 0x47, 0x5c, 0x71, 0x61, - 0x2a, 0xfb, 0xe4, 0x47, 0x94, 0x4a, 0x4d, 0x3b, 0x68, 0xdb, 0x07, 0x22, 0xcd, 0xb1, 0x6f, 0xfc, - 0x07, 0x0d, 0x8a, 0xef, 0x18, 0x5e, 0x53, 0x2e, 0x1d, 0xda, 0x80, 0xb2, 0x9f, 0xdc, 0x18, 0x45, - 0xd8, 0x12, 0x29, 0xb1, 0x6c, 0x8c, 0x3c, 0x4a, 0xcb, 0xea, 0x58, 0x6a, 0xaa, 0x04, 0x26, 0xca, - 0xb0, 0x9a, 0xa4, 0xed, 0x8b, 0x4a, 0x25, 0x8b, 0x62, 0x8c, 0xaa, 0x28, 0x95, 0xf0, 0x68, 0x32, - 0x38, 0x7e, 0xf0, 0x5c, 0xf2, 0xab, 0x14, 0xa0, 0x41, 0x1b, 0xbe, 0xe8, 0x89, 0xec, 0x16, 0x94, - 0x5d, 0xcf, 0x70, 0x06, 0xf6, 0x46, 0x89, 0x51, 0xfd, 0x04, 0x7d, 0x07, 0x26, 0xbb, 0x8e, 0x7d, - 0xe8, 0x10, 0xd7, 0x6d, 0x58, 0xb6, 0x67, 0x3e, 0x3f, 0x15, 0x87, 0xda, 0xb2, 0x24, 0x6f, 0x33, - 0x2a, 0xaa, 0x43, 0xee, 0xb9, 0xd9, 0xf6, 0x88, 0xe3, 0x56, 0x33, 0xf3, 0xe9, 0x85, 0xf2, 0xf2, - 0xfd, 0xb3, 0xbc, 0xb6, 0xf8, 0x26, 0xe3, 0xdf, 0x3b, 0xed, 0x12, 0x5d, 0x8e, 0x55, 0x0f, 0x8a, - 0xd9, 0xd0, 0x41, 0xf1, 0x16, 0x40, 0xc0, 0x4f, 0x53, 0xed, 0xf6, 0xce, 0xd3, 0x67, 0x7b, 0x95, - 0x31, 0x54, 0x84, 0x89, 0xed, 0x9d, 0xb5, 0xfa, 0x56, 0x9d, 0xe6, 0x65, 0xbc, 0x24, 0x7d, 0xa3, - 0xfa, 0x10, 0xcd, 0xc2, 0xc4, 0x0b, 0x4a, 0x95, 0xf7, 0xed, 0xb4, 0x9e, 0x63, 0xed, 0x8d, 0x16, - 0xfe, 0x69, 0x0a, 0x4a, 0x62, 0x17, 0x8c, 0xb4, 0x15, 0x55, 0x15, 0xa9, 0x90, 0x0a, 0x7a, 0x2a, - 0xe5, 0xbb, 0xa3, 0x25, 0x0e, 0xbf, 0xb2, 0x49, 0x73, 0x03, 0x5f, 0x6c, 0xd2, 0x12, 0x6e, 0xf5, - 0xdb, 0xb1, 0xe1, 0x9b, 0x89, 0x0d, 0x5f, 0x74, 0x03, 0x4a, 0xfe, 0x6e, 0x33, 0x5c, 0x51, 0x6b, - 0xf3, 0x7a, 0x51, 0x6e, 0x24, 0x4a, 0x43, 0xb7, 0x20, 0x4b, 0xfa, 0xc4, 0xf2, 0xdc, 0x6a, 0x81, - 0x65, 0xdd, 0x92, 0x3c, 0xff, 0xd6, 0x29, 0x55, 0x17, 0x9d, 0xf8, 0xff, 0x60, 0x8a, 0xdd, 0x33, - 0x1e, 0x3b, 0x86, 0xa5, 0x5e, 0x88, 0xf6, 0xf6, 0xb6, 0x84, 0xeb, 0xe8, 0x27, 0x2a, 0x43, 0x6a, - 0x63, 0x4d, 0x4c, 0x34, 0xb5, 0xb1, 0x86, 0x3f, 0xd2, 0x00, 0xa9, 0xe3, 0x46, 0xf2, 0x65, 0x44, - 0xb8, 0x54, 0x9f, 0x0e, 0xd4, 0xcf, 0x40, 0x86, 0x38, 0x8e, 0xed, 0x30, 0xaf, 0xe5, 0x75, 0xde, - 0xc0, 0x37, 0x85, 0x0d, 0x3a, 0xe9, 0xdb, 0xc7, 0x7e, 0x60, 0x70, 0x69, 0x9a, 0x6f, 0xea, 0x26, - 0x4c, 0x87, 0xb8, 0x46, 0xca, 0xfe, 0x77, 0xe0, 0x02, 0x13, 0xb6, 0x49, 0x48, 0x77, 0xa5, 0x6d, - 0xf6, 0x13, 0xb5, 0x76, 0xe1, 0x62, 0x94, 0xf1, 0xeb, 0xf5, 0x11, 0xfe, 0xb6, 0xd0, 0xb8, 0x67, - 0x76, 0xc8, 0x9e, 0xbd, 0x95, 0x6c, 0x1b, 0xcd, 0x8e, 0xc7, 0xe4, 0xd4, 0x15, 0x65, 0x92, 0x7d, - 0xe3, 0xdf, 0x6a, 0x70, 0x69, 0x60, 0xf8, 0xd7, 0xbc, 0xaa, 0x73, 0x00, 0x87, 0x74, 0xfb, 0x90, - 0x16, 0xed, 0xe0, 0x37, 0x74, 0x85, 0xe2, 0xdb, 0x49, 0x13, 0x4c, 0x51, 0xd8, 0x39, 0x23, 0xd6, - 0x9c, 0xfd, 0x71, 0x65, 0x8d, 0xb9, 0x0a, 0x05, 0x46, 0xd8, 0xf5, 0x0c, 0xaf, 0xe7, 0x0e, 0x2c, - 0xc6, 0x0f, 0xc4, 0x16, 0x90, 0x83, 0x46, 0x9a, 0xd7, 0x2b, 0x90, 0x65, 0x87, 0x53, 0x79, 0x34, - 0x8b, 0xdc, 0x06, 0x14, 0x3b, 0x74, 0xc1, 0x88, 0x8f, 0x20, 0xfb, 0x84, 0x21, 0x7a, 0x8a, 0x65, - 0xe3, 0x72, 0x29, 0x2c, 0xa3, 0xc3, 0x71, 0x86, 0xbc, 0xce, 0xbe, 0xd9, 0x49, 0x86, 0x10, 0xe7, - 0x99, 0xbe, 0xc5, 0x4f, 0x4c, 0x79, 0xdd, 0x6f, 0x53, 0x97, 0x35, 0xdb, 0x26, 0xb1, 0x3c, 0xd6, - 0x3b, 0xce, 0x7a, 0x15, 0x0a, 0x5e, 0x84, 0x0a, 0xd7, 0xb4, 0xd2, 0x6a, 0x29, 0x27, 0x12, 0x5f, - 0x9e, 0x16, 0x96, 0x87, 0x7f, 0xa7, 0xc1, 0x94, 0x32, 0x60, 0x24, 0xc7, 0xbc, 0x04, 0x59, 0x8e, - 0x5b, 0x8a, 0xe2, 0x37, 0x13, 0x1e, 0xc5, 0xd5, 0xe8, 0x82, 0x07, 0x2d, 0x42, 0x8e, 0x7f, 0xc9, - 0x63, 0x61, 0x3c, 0xbb, 0x64, 0xc2, 0xb7, 0x60, 0x5a, 0x90, 0x48, 0xc7, 0x8e, 0xdb, 0xdb, 0xcc, - 0xa1, 0xf8, 0x43, 0x98, 0x09, 0xb3, 0x8d, 0x34, 0x25, 0xc5, 0xc8, 0xd4, 0x79, 0x8c, 0x5c, 0x91, - 0x46, 0x3e, 0xeb, 0xb6, 0x94, 0x5a, 0x1d, 0x5d, 0x75, 0x75, 0x45, 0x52, 0x91, 0x15, 0xf1, 0x27, - 0x20, 0x45, 0x7c, 0xa3, 0x13, 0x98, 0x96, 0xdb, 0x61, 0xcb, 0x74, 0xfd, 0x13, 0xdc, 0x07, 0x80, - 0x54, 0xe2, 0x37, 0x6d, 0xd0, 0x1a, 0x79, 0xee, 0x18, 0x87, 0x1d, 0xe2, 0xd7, 0x27, 0x7a, 0x9e, - 0x57, 0x89, 0x23, 0x65, 0xf4, 0x25, 0x98, 0x7a, 0x62, 0xf7, 0x69, 0x6a, 0xa0, 0xd4, 0x20, 0x64, - 0xf8, 0x7d, 0xce, 0x5f, 0x36, 0xbf, 0x4d, 0x95, 0xab, 0x03, 0x46, 0x52, 0xfe, 0x17, 0x0d, 0x8a, - 0x2b, 0x6d, 0xc3, 0xe9, 0x48, 0xc5, 0xaf, 0x43, 0x96, 0xdf, 0x52, 0x04, 0x30, 0x70, 0x3b, 0x2c, - 0x46, 0xe5, 0xe5, 0x8d, 0x15, 0x7e, 0xa7, 0x11, 0xa3, 0xa8, 0xe1, 0xe2, 0xed, 0x60, 0x2d, 0xf2, - 0x96, 0xb0, 0x86, 0x1e, 0x40, 0xc6, 0xa0, 0x43, 0x58, 0x0a, 0x2e, 0x47, 0xef, 0x87, 0x4c, 0x1a, - 0x3b, 0x9c, 0x71, 0x2e, 0xfc, 0x1a, 0x14, 0x14, 0x0d, 0xf4, 0x06, 0xfc, 0xb8, 0x2e, 0x0e, 0x60, - 0x2b, 0xab, 0x7b, 0x1b, 0xfb, 0xfc, 0x62, 0x5c, 0x06, 0x58, 0xab, 0xfb, 0xed, 0x14, 0x7e, 0x57, - 0x8c, 0x12, 0xf9, 0x4e, 0xb5, 0x47, 0x4b, 0xb2, 0x27, 0x75, 0x2e, 0x7b, 0x4e, 0xa0, 0x24, 0xa6, - 0x3f, 0x6a, 0xfa, 0x66, 0xf2, 0x12, 0xd2, 0xb7, 0x62, 0xbc, 0x2e, 0x18, 0xf1, 0x24, 0x94, 0x44, - 0x42, 0x17, 0xfb, 0xef, 0xcf, 0x1a, 0x94, 0x25, 0x65, 0x54, 0x00, 0x53, 0x62, 0x2f, 0xbc, 0x02, - 0xf8, 0xc8, 0xcb, 0x45, 0xc8, 0xb6, 0x0e, 0x76, 0xcd, 0x0f, 0x24, 0xd8, 0x2c, 0x5a, 0x94, 0xde, - 0xe6, 0x7a, 0xf8, 0x8b, 0x8f, 0x68, 0xd1, 0x5b, 0xb8, 0x63, 0x3c, 0xf7, 0x36, 0xac, 0x16, 0x39, - 0x61, 0xe7, 0xc6, 0x71, 0x3d, 0x20, 0xb0, 0x4b, 0xa9, 0x78, 0x19, 0x62, 0x87, 0x45, 0xf5, 0xa5, - 0x68, 0x1a, 0xa6, 0x56, 0x7a, 0xde, 0x51, 0xdd, 0x32, 0x0e, 0xda, 0x32, 0x63, 0xd1, 0x32, 0x4b, - 0x89, 0x6b, 0xa6, 0xab, 0x52, 0xeb, 0x30, 0x4d, 0xa9, 0xc4, 0xf2, 0xcc, 0xa6, 0x92, 0xde, 0x64, - 0x11, 0xd3, 0x22, 0x45, 0xcc, 0x70, 0xdd, 0x17, 0xb6, 0xd3, 0x12, 0x53, 0xf3, 0xdb, 0x78, 0x8d, - 0x0b, 0x7f, 0xe6, 0x86, 0xca, 0xd4, 0x17, 0x95, 0xb2, 0x10, 0x48, 0x79, 0x4c, 0xbc, 0x21, 0x52, - 0xf0, 0x7d, 0xb8, 0x20, 0x39, 0x05, 0xb8, 0x37, 0x84, 0x79, 0x07, 0xae, 0x4a, 0xe6, 0xd5, 0x23, - 0x7a, 0x7b, 0x7a, 0x2a, 0x14, 0x7e, 0x59, 0x3b, 0x1f, 0x41, 0xd5, 0xb7, 0x93, 0x1d, 0x96, 0xed, - 0xb6, 0x6a, 0x40, 0xcf, 0x15, 0x7b, 0x26, 0xaf, 0xb3, 0x6f, 0x4a, 0x73, 0xec, 0xb6, 0x7f, 0x24, - 0xa0, 0xdf, 0x78, 0x15, 0x66, 0xa5, 0x0c, 0x71, 0x8c, 0x0d, 0x0b, 0x19, 0x30, 0x28, 0x4e, 0x88, - 0x70, 0x18, 0x1d, 0x3a, 0xdc, 0xed, 0x2a, 0x67, 0xd8, 0xb5, 0x4c, 0xa6, 0xa6, 0xc8, 0xbc, 0xc0, - 0x77, 0x04, 0x35, 0x4c, 0xad, 0x18, 0x82, 0x4c, 0x05, 0xa8, 0x64, 0xb1, 0x10, 0x94, 0x3c, 0xb0, - 0x10, 0x03, 0xa2, 0xdf, 0x83, 0x39, 0xdf, 0x08, 0xea, 0xb7, 0xa7, 0xc4, 0xe9, 0x98, 0xae, 0xab, - 0xc0, 0x41, 0x71, 0x13, 0xbf, 0x0d, 0xe3, 0x5d, 0x22, 0x72, 0x4a, 0x61, 0x19, 0x2d, 0xf2, 0xf7, - 0xdb, 0x45, 0x65, 0x30, 0xeb, 0xc7, 0x2d, 0xb8, 0x26, 0xa5, 0x73, 0x8f, 0xc6, 0x8a, 0x8f, 0x1a, - 0x25, 0x6f, 0xdd, 0xdc, 0xad, 0x83, 0xb7, 0xee, 0x34, 0x5f, 0x7b, 0x1f, 0xa2, 0x7c, 0x8b, 0x3b, - 0x52, 0xc6, 0xd6, 0x48, 0xb5, 0x62, 0x93, 0xfb, 0xd4, 0x0f, 0xc9, 0x91, 0x84, 0x1d, 0xc0, 0x4c, - 0x38, 0x92, 0x47, 0x4a, 0x63, 0x33, 0x90, 0xf1, 0xec, 0x63, 0x22, 0x93, 0x18, 0x6f, 0x48, 0x83, - 0xfd, 0x30, 0x1f, 0xc9, 0x60, 0x23, 0x10, 0xc6, 0xb6, 0xe4, 0xa8, 0xf6, 0xd2, 0xd5, 0x94, 0x87, - 0x2f, 0xde, 0xc0, 0xdb, 0x70, 0x31, 0x9a, 0x26, 0x46, 0x32, 0x79, 0x9f, 0x6f, 0xe0, 0xb8, 0x4c, - 0x32, 0x92, 0xdc, 0xb7, 0x83, 0x64, 0xa0, 0x24, 0x94, 0x91, 0x44, 0xea, 0x50, 0x8b, 0xcb, 0x2f, - 0x5f, 0xc5, 0x7e, 0xf5, 0xd3, 0xcd, 0x48, 0xc2, 0xdc, 0x40, 0xd8, 0xe8, 0xcb, 0x1f, 0xe4, 0x88, - 0xf4, 0xd0, 0x1c, 0x21, 0x82, 0x24, 0xc8, 0x62, 0x5f, 0xc3, 0xa6, 0x13, 0x3a, 0x82, 0x04, 0x3a, - 0xaa, 0x0e, 0x5a, 0x43, 0x7c, 0x1d, 0xac, 0x21, 0x37, 0xb6, 0x9a, 0x76, 0x47, 0x5a, 0x8c, 0x77, - 0x82, 0xdc, 0x39, 0x90, 0x99, 0x47, 0x12, 0xfc, 0x2e, 0xcc, 0x27, 0x27, 0xe5, 0x51, 0x24, 0xdf, - 0xc3, 0x90, 0xf7, 0x0f, 0x94, 0xca, 0x6f, 0x1f, 0x0a, 0x90, 0xdb, 0xde, 0xd9, 0x7d, 0xba, 0xb2, - 0x5a, 0xaf, 0x68, 0xcb, 0xff, 0x49, 0x43, 0x6a, 0x73, 0x1f, 0x7d, 0x17, 0x32, 0xfc, 0xf1, 0x6f, - 0xc8, 0x8b, 0x6f, 0x6d, 0xd8, 0xfb, 0x26, 0xbe, 0xf2, 0xd1, 0xdf, 0xfe, 0xf9, 0x69, 0xea, 0x22, - 0x9e, 0x5a, 0xea, 0xbf, 0x6a, 0xb4, 0xbb, 0x47, 0xc6, 0xd2, 0x71, 0x7f, 0x89, 0xd5, 0x84, 0x87, - 0xda, 0x3d, 0xb4, 0x0f, 0xe9, 0xa7, 0x3d, 0x0f, 0x25, 0x3e, 0x07, 0xd7, 0x92, 0xdf, 0x3d, 0x71, - 0x8d, 0x49, 0x9e, 0xc1, 0x93, 0xaa, 0xe4, 0x6e, 0xcf, 0xa3, 0x72, 0xfb, 0x50, 0x50, 0x9f, 0x2e, - 0xcf, 0x7c, 0x28, 0xae, 0x9d, 0xfd, 0x2c, 0x8a, 0x31, 0xd3, 0x77, 0x05, 0x5f, 0x52, 0xf5, 0xf1, - 0x17, 0x56, 0x75, 0x3e, 0x7b, 0x27, 0x16, 0x4a, 0x7c, 0x4b, 0xae, 0x25, 0x3f, 0x97, 0xc6, 0xcf, - 0xc7, 0x3b, 0xb1, 0xa8, 0x5c, 0x5b, 0x3c, 0x97, 0x36, 0x3d, 0x74, 0x2d, 0xe6, 0xb9, 0x4c, 0x7d, - 0x18, 0xaa, 0xcd, 0x27, 0x33, 0x08, 0x4d, 0xd7, 0x99, 0xa6, 0xcb, 0xf8, 0xa2, 0xaa, 0xa9, 0xe9, - 0xf3, 0x3d, 0xd4, 0xee, 0x2d, 0x1f, 0x41, 0x86, 0x21, 0xc3, 0xa8, 0x21, 0x3f, 0x6a, 0x31, 0x98, - 0x76, 0xc2, 0x0e, 0x08, 0x61, 0xca, 0x78, 0x96, 0x69, 0x9b, 0xc6, 0x65, 0x5f, 0x1b, 0x03, 0x87, - 0x1f, 0x6a, 0xf7, 0x16, 0xb4, 0x97, 0xb5, 0xe5, 0x7f, 0x8f, 0x43, 0x86, 0xe1, 0x44, 0xa8, 0x0b, - 0x10, 0xc0, 0xa8, 0xd1, 0x79, 0x0e, 0x00, 0xb3, 0xd1, 0x79, 0x0e, 0x22, 0xb0, 0xf8, 0x1a, 0xd3, - 0x3c, 0x8b, 0x67, 0x7c, 0xcd, 0x0c, 0x83, 0x5a, 0x62, 0xb0, 0x1a, 0x75, 0xeb, 0x0b, 0x01, 0x95, - 0xf1, 0x00, 0x43, 0x71, 0x12, 0x43, 0x78, 0x6a, 0x74, 0x9b, 0xc4, 0x60, 0xa9, 0xf8, 0x06, 0x53, - 0x7a, 0x15, 0x57, 0x55, 0xe7, 0x72, 0xbd, 0x0e, 0xe3, 0xa4, 0x8a, 0x3f, 0xd6, 0xa0, 0x1c, 0x86, - 0x44, 0xd1, 0x8d, 0x18, 0xd1, 0x51, 0x64, 0xb5, 0x76, 0x73, 0x38, 0x53, 0xa2, 0x09, 0x5c, 0xff, - 0x31, 0x21, 0x5d, 0x83, 0x72, 0x0a, 0xdf, 0xa3, 0x1f, 0x69, 0x30, 0x19, 0x01, 0x3a, 0x51, 0x9c, - 0x8a, 0x01, 0x18, 0xb5, 0x76, 0xeb, 0x0c, 0x2e, 0x61, 0xc9, 0x1d, 0x66, 0xc9, 0x75, 0x7c, 0x65, - 0xd0, 0x19, 0x9e, 0xd9, 0x21, 0x9e, 0x2d, 0xac, 0xf1, 0x57, 0x82, 0xa3, 0x92, 0xb1, 0x2b, 0x11, - 0x42, 0x39, 0x63, 0x57, 0x22, 0x0c, 0x69, 0x0e, 0x5b, 0x09, 0x8e, 0x45, 0xd2, 0x8d, 0xfe, 0xdf, - 0x34, 0xe4, 0x56, 0xf9, 0xef, 0x0f, 0x91, 0x07, 0x79, 0x1f, 0xff, 0x43, 0x73, 0x71, 0x58, 0x4c, - 0x70, 0x57, 0xa8, 0x5d, 0x4b, 0xec, 0x17, 0xea, 0x6f, 0x33, 0xf5, 0xf3, 0xf8, 0xb2, 0xaf, 0x5e, - 0xfc, 0xce, 0x71, 0x89, 0xdf, 0xfa, 0x97, 0x8c, 0x56, 0x8b, 0x4e, 0xfd, 0x87, 0x1a, 0x14, 0x55, - 0x98, 0x0e, 0x5d, 0x8f, 0x45, 0x81, 0x54, 0xa4, 0xaf, 0x86, 0x87, 0xb1, 0x08, 0xfd, 0x77, 0x99, - 0xfe, 0x1b, 0x78, 0x2e, 0x49, 0xbf, 0xc3, 0xf8, 0xc3, 0x26, 0x70, 0xa0, 0x2d, 0xde, 0x84, 0x10, - 0x8e, 0x17, 0x6f, 0x42, 0x18, 0xa7, 0x3b, 0xdb, 0x84, 0x1e, 0xe3, 0xa7, 0x26, 0x9c, 0x00, 0x04, - 0xb8, 0x1a, 0x8a, 0x75, 0xae, 0x72, 0x7b, 0x8a, 0x06, 0xff, 0x20, 0x24, 0x17, 0xb3, 0xf5, 0x22, - 0xba, 0xdb, 0xa6, 0x4b, 0x93, 0xc0, 0xf2, 0xef, 0xb3, 0x50, 0x78, 0x62, 0x98, 0x96, 0x47, 0x2c, - 0xc3, 0x6a, 0x12, 0x74, 0x08, 0x19, 0x56, 0x1e, 0xa3, 0x19, 0x4f, 0xc5, 0x9b, 0xa2, 0x19, 0x2f, - 0x04, 0xc6, 0xe0, 0x5b, 0x4c, 0xf5, 0x35, 0x5c, 0xf3, 0x55, 0x77, 0x02, 0xf9, 0x4b, 0x0c, 0x48, - 0xa1, 0x53, 0x3e, 0x86, 0xac, 0xc0, 0xe8, 0x23, 0xd2, 0x42, 0x00, 0x4b, 0xed, 0x4a, 0x7c, 0x67, - 0xe2, 0x2e, 0x53, 0x75, 0xb9, 0x8c, 0x99, 0x2a, 0xfb, 0x1e, 0x40, 0x00, 0x13, 0x46, 0xfd, 0x3b, - 0x80, 0x2a, 0xd6, 0xe6, 0x93, 0x19, 0x84, 0xe2, 0x7b, 0x4c, 0xf1, 0x4d, 0x7c, 0x2d, 0x56, 0x71, - 0xcb, 0x1f, 0x40, 0x95, 0x37, 0x61, 0x7c, 0xdd, 0x70, 0x8f, 0x50, 0xa4, 0xfa, 0x29, 0xbf, 0x1d, - 0xa8, 0xd5, 0xe2, 0xba, 0x84, 0xaa, 0x9b, 0x4c, 0xd5, 0x1c, 0x9e, 0x8d, 0x55, 0x75, 0x64, 0xb8, - 0xb4, 0x98, 0x20, 0x13, 0xb2, 0xfc, 0xf7, 0x04, 0x51, 0x77, 0x86, 0x7e, 0x93, 0x10, 0x75, 0x67, - 0xf8, 0x27, 0x08, 0xe7, 0x54, 0xd5, 0x83, 0x09, 0xf9, 0x8a, 0x8f, 0xae, 0x46, 0x96, 0x27, 0xfc, - 0xe2, 0x5f, 0x9b, 0x4b, 0xea, 0x16, 0x0a, 0x17, 0x98, 0x42, 0x8c, 0xaf, 0xc6, 0xaf, 0x9f, 0x60, - 0x7f, 0xa8, 0xdd, 0x7b, 0x59, 0xa3, 0x55, 0x03, 0x02, 0xb8, 0x75, 0x20, 0x48, 0xa2, 0xc8, 0xed, - 0x40, 0x90, 0x0c, 0x20, 0xb5, 0xf8, 0x55, 0xa6, 0xfd, 0x01, 0x5e, 0x88, 0xd5, 0xee, 0x39, 0x86, - 0xe5, 0x3e, 0x27, 0xce, 0x03, 0x8e, 0xab, 0xb9, 0x47, 0x66, 0x97, 0x06, 0xcc, 0x4f, 0x2a, 0x30, - 0x4e, 0x8f, 0xa6, 0xb4, 0x60, 0x07, 0x37, 0xfa, 0xa8, 0x39, 0x03, 0x38, 0x5a, 0xd4, 0x9c, 0x41, - 0x30, 0x20, 0xa6, 0x60, 0xb3, 0xdf, 0x9d, 0x13, 0xc6, 0x45, 0x1d, 0xef, 0x41, 0x41, 0xb9, 0xf7, - 0xa3, 0x18, 0x89, 0x61, 0x94, 0x2e, 0x5a, 0x26, 0x62, 0x40, 0x03, 0x3c, 0xcf, 0x94, 0xd6, 0xf0, - 0x85, 0xb0, 0xd2, 0x16, 0x67, 0xa3, 0x5a, 0x3f, 0x84, 0xa2, 0x0a, 0x10, 0xa0, 0x18, 0xa1, 0x11, - 0x18, 0x30, 0x9a, 0x1d, 0xe3, 0xf0, 0x85, 0x98, 0x34, 0xe1, 0xff, 0xca, 0x5e, 0xf2, 0x52, 0xed, - 0xef, 0x43, 0x4e, 0xc0, 0x06, 0x71, 0xf3, 0x0d, 0x03, 0x87, 0x71, 0xf3, 0x8d, 0x60, 0x0e, 0x31, - 0xa7, 0x3f, 0xa6, 0x96, 0x5e, 0x8f, 0x64, 0x49, 0x12, 0x2a, 0x1f, 0x13, 0x2f, 0x49, 0x65, 0x00, - 0x85, 0x25, 0xa9, 0x54, 0xae, 0xa6, 0x43, 0x55, 0x1e, 0x12, 0x4f, 0x84, 0x94, 0xbc, 0xf7, 0xa1, - 0x04, 0x89, 0x6a, 0xfe, 0xc7, 0xc3, 0x58, 0x12, 0x0f, 0xec, 0x81, 0x56, 0x91, 0xfc, 0xd1, 0xf7, - 0x01, 0x02, 0x8c, 0x23, 0x7a, 0x06, 0x8b, 0x05, 0x4a, 0xa3, 0x67, 0xb0, 0x78, 0x98, 0x24, 0x26, - 0x91, 0x04, 0xca, 0xf9, 0xa5, 0x81, 0xaa, 0xff, 0xb9, 0x06, 0x68, 0x10, 0x13, 0x41, 0xf7, 0xe3, - 0x55, 0xc4, 0x62, 0xb0, 0xb5, 0x97, 0xce, 0xc7, 0x9c, 0x58, 0x2f, 0x02, 0xbb, 0x9a, 0x6c, 0x48, - 0xf7, 0x05, 0xb5, 0xec, 0x13, 0x0d, 0x4a, 0x21, 0x54, 0x05, 0xdd, 0x4e, 0x58, 0xe7, 0x08, 0x8e, - 0x5b, 0xbb, 0x73, 0x26, 0x5f, 0xe2, 0xf9, 0x4c, 0xd9, 0x15, 0xf2, 0x88, 0xfe, 0x63, 0x0d, 0xca, - 0x61, 0x28, 0x06, 0x25, 0x28, 0x18, 0x00, 0x83, 0x6b, 0x0b, 0x67, 0x33, 0x9e, 0x63, 0xb5, 0x82, - 0x53, 0xfb, 0xfb, 0x90, 0x13, 0x08, 0x4e, 0x5c, 0x58, 0x84, 0xb1, 0xe4, 0xb8, 0xb0, 0x88, 0xc0, - 0x3f, 0x49, 0x61, 0xe1, 0xd8, 0x6d, 0xa2, 0x44, 0xa2, 0xc0, 0x79, 0x92, 0x54, 0x0e, 0x8f, 0xc4, - 0x08, 0x48, 0x34, 0x54, 0x65, 0x10, 0x89, 0x12, 0xe5, 0x41, 0x09, 0x12, 0xcf, 0x88, 0xc4, 0x28, - 0x48, 0x94, 0x14, 0x89, 0x4c, 0xab, 0x12, 0x89, 0x01, 0x28, 0x13, 0x17, 0x89, 0x03, 0x48, 0x79, - 0x5c, 0x24, 0x0e, 0xe2, 0x3a, 0x49, 0x6b, 0xcb, 0x94, 0x87, 0x22, 0x71, 0x3a, 0x06, 0xc4, 0x41, - 0x2f, 0x25, 0xf8, 0x34, 0x16, 0x85, 0xaf, 0x3d, 0x38, 0x27, 0xf7, 0xf0, 0x08, 0xe0, 0xab, 0x21, - 0x23, 0xe0, 0xd7, 0x1a, 0xcc, 0xc4, 0xa1, 0x40, 0x28, 0x41, 0x59, 0x02, 0x84, 0x5f, 0x5b, 0x3c, - 0x2f, 0xfb, 0x39, 0xfc, 0xe6, 0xc7, 0xc4, 0xa3, 0xca, 0x9f, 0x3e, 0x9f, 0xd3, 0xfe, 0xfa, 0xf9, - 0x9c, 0xf6, 0xf7, 0xcf, 0xe7, 0xb4, 0x5f, 0xfc, 0x63, 0x6e, 0xec, 0x20, 0xcb, 0xfe, 0xf3, 0xd7, - 0xab, 0xff, 0x0b, 0x00, 0x00, 0xff, 0xff, 0x93, 0x8b, 0xaf, 0x54, 0x83, 0x36, 0x00, 0x00, + 0x45, 0xff, 0x98, 0xdb, 0xee, 0xec, 0x72, 0x97, 0x52, 0xc2, 0x24, 0x2f, 0xf2, 0xce, 0x99, 0x33, + 0xe7, 0x9c, 0x39, 0x33, 0xe7, 0x9c, 0x99, 0xdf, 0xd0, 0x90, 0x77, 0xfa, 0xed, 0xd5, 0xbe, 0x63, + 0x7b, 0x36, 0x2a, 0x12, 0xaf, 0xdd, 0x71, 0x89, 0x33, 0x24, 0x4e, 0xff, 0xb8, 0xb6, 0x70, 0x62, + 0x9f, 0xd8, 0xac, 0xa3, 0x4e, 0xbf, 0x38, 0x4f, 0x6d, 0x91, 0xf2, 0xd4, 0x7b, 0xc3, 0x76, 0x9b, + 0xfd, 0xe9, 0x1f, 0xd7, 0xcf, 0x86, 0xa2, 0xeb, 0x26, 0xeb, 0x32, 0x06, 0xde, 0x29, 0xfb, 0xd3, + 0x3f, 0x66, 0xff, 0x88, 0xce, 0x5b, 0x27, 0xb6, 0x7d, 0xd2, 0x25, 0x75, 0xa3, 0x6f, 0xd6, 0x0d, + 0xcb, 0xb2, 0x3d, 0xc3, 0x33, 0x6d, 0xcb, 0xe5, 0xbd, 0xf8, 0x1b, 0x0d, 0xca, 0x3a, 0x71, 0xfb, + 0xb6, 0xe5, 0x92, 0x2d, 0x62, 0x74, 0x88, 0x83, 0x6e, 0x03, 0xb4, 0xbb, 0x03, 0xd7, 0x23, 0x4e, + 0xcb, 0xec, 0x54, 0xb5, 0x65, 0x6d, 0x65, 0x5a, 0xcf, 0x0b, 0xca, 0x76, 0x07, 0xdd, 0x84, 0x7c, + 0x8f, 0xf4, 0x8e, 0x79, 0x6f, 0x8a, 0xf5, 0xce, 0x70, 0xc2, 0x76, 0x07, 0xd5, 0x60, 0xc6, 0x21, + 0x43, 0xd3, 0x35, 0x6d, 0xab, 0x9a, 0x5e, 0xd6, 0x56, 0xd2, 0xba, 0xdf, 0xa6, 0x03, 0x1d, 0xe3, + 0x8d, 0xd7, 0xf2, 0x88, 0xd3, 0xab, 0x4e, 0xf3, 0x81, 0x94, 0xd0, 0x24, 0x4e, 0x0f, 0x7f, 0x9d, + 0x81, 0xa2, 0x6e, 0x58, 0x27, 0x44, 0x27, 0x9f, 0x0f, 0x88, 0xeb, 0xa1, 0x0a, 0xa4, 0xcf, 0xc8, + 0x05, 0x53, 0x5f, 0xd4, 0xe9, 0x27, 0x1f, 0x6f, 0x9d, 0x90, 0x16, 0xb1, 0xb8, 0xe2, 0x22, 0x1d, + 0x6f, 0x9d, 0x90, 0x86, 0xd5, 0x41, 0x0b, 0x90, 0xe9, 0x9a, 0x3d, 0xd3, 0x13, 0x5a, 0x79, 0x23, + 0x64, 0xce, 0x74, 0xc4, 0x9c, 0x0d, 0x00, 0xd7, 0x76, 0xbc, 0x96, 0xed, 0x74, 0x88, 0x53, 0xcd, + 0x2c, 0x6b, 0x2b, 0xe5, 0xb5, 0xfb, 0xab, 0xea, 0x42, 0xac, 0xaa, 0x06, 0xad, 0x1e, 0xda, 0x8e, + 0xb7, 0x4f, 0x79, 0xf5, 0xbc, 0x2b, 0x3f, 0xd1, 0x87, 0x50, 0x60, 0x42, 0x3c, 0xc3, 0x39, 0x21, + 0x5e, 0x35, 0xcb, 0xa4, 0x3c, 0xb8, 0x44, 0x4a, 0x93, 0x31, 0xeb, 0x4c, 0x3d, 0xff, 0x46, 0x18, + 0x8a, 0x2e, 0x71, 0x4c, 0xa3, 0x6b, 0x7e, 0x61, 0x1c, 0x77, 0x49, 0x35, 0xb7, 0xac, 0xad, 0xcc, + 0xe8, 0x21, 0x1a, 0x9d, 0xff, 0x19, 0xb9, 0x70, 0x5b, 0xb6, 0xd5, 0xbd, 0xa8, 0xce, 0x30, 0x86, + 0x19, 0x4a, 0xd8, 0xb7, 0xba, 0x17, 0x6c, 0xd1, 0xec, 0x81, 0xe5, 0xf1, 0xde, 0x3c, 0xeb, 0xcd, + 0x33, 0x0a, 0xeb, 0x5e, 0x81, 0x4a, 0xcf, 0xb4, 0x5a, 0x3d, 0xbb, 0xd3, 0xf2, 0x1d, 0x02, 0xcc, + 0x21, 0xe5, 0x9e, 0x69, 0xbd, 0xb2, 0x3b, 0xba, 0x74, 0x0b, 0xe5, 0x34, 0xce, 0xc3, 0x9c, 0x05, + 0xc1, 0x69, 0x9c, 0xab, 0x9c, 0xab, 0x30, 0x4f, 0x65, 0xb6, 0x1d, 0x62, 0x78, 0x24, 0x60, 0x2e, + 0x32, 0xe6, 0xb9, 0x9e, 0x69, 0x6d, 0xb0, 0x9e, 0x10, 0xbf, 0x71, 0x3e, 0xc2, 0x5f, 0x12, 0xfc, + 0xc6, 0x79, 0x98, 0x1f, 0xaf, 0x42, 0xde, 0xf7, 0x39, 0x9a, 0x81, 0xe9, 0xbd, 0xfd, 0xbd, 0x46, + 0x65, 0x0a, 0x01, 0x64, 0xd7, 0x0f, 0x37, 0x1a, 0x7b, 0x9b, 0x15, 0x0d, 0x15, 0x20, 0xb7, 0xd9, + 0xe0, 0x8d, 0x14, 0x7e, 0x01, 0x10, 0x78, 0x17, 0xe5, 0x20, 0xbd, 0xd3, 0xf8, 0xbd, 0xca, 0x14, + 0xe5, 0x39, 0x6a, 0xe8, 0x87, 0xdb, 0xfb, 0x7b, 0x15, 0x8d, 0x0e, 0xde, 0xd0, 0x1b, 0xeb, 0xcd, + 0x46, 0x25, 0x45, 0x39, 0x5e, 0xed, 0x6f, 0x56, 0xd2, 0x28, 0x0f, 0x99, 0xa3, 0xf5, 0xdd, 0xd7, + 0x8d, 0xca, 0x34, 0xfe, 0x56, 0x83, 0x92, 0x58, 0x2f, 0x1e, 0x13, 0xe8, 0x3d, 0xc8, 0x9e, 0xb2, + 0xb8, 0x60, 0x5b, 0xb1, 0xb0, 0x76, 0x2b, 0xb2, 0xb8, 0xa1, 0xd8, 0xd1, 0x05, 0x2f, 0xc2, 0x90, + 0x3e, 0x1b, 0xba, 0xd5, 0xd4, 0x72, 0x7a, 0xa5, 0xb0, 0x56, 0x59, 0xe5, 0x01, 0xbb, 0xba, 0x43, + 0x2e, 0x8e, 0x8c, 0xee, 0x80, 0xe8, 0xb4, 0x13, 0x21, 0x98, 0xee, 0xd9, 0x0e, 0x61, 0x3b, 0x76, + 0x46, 0x67, 0xdf, 0x74, 0x1b, 0xb3, 0x45, 0x13, 0xbb, 0x95, 0x37, 0xf0, 0x77, 0x1a, 0xc0, 0xc1, + 0xc0, 0x4b, 0x0e, 0x8d, 0x05, 0xc8, 0x0c, 0xa9, 0x60, 0x11, 0x16, 0xbc, 0xc1, 0x62, 0x82, 0x18, + 0x2e, 0xf1, 0x63, 0x82, 0x36, 0xd0, 0x0d, 0xc8, 0xf5, 0x1d, 0x32, 0x6c, 0x9d, 0x0d, 0x99, 0x92, + 0x19, 0x3d, 0x4b, 0x9b, 0x3b, 0x43, 0x74, 0x17, 0x8a, 0xe6, 0x89, 0x65, 0x3b, 0xa4, 0xc5, 0x65, + 0x65, 0x58, 0x6f, 0x81, 0xd3, 0x98, 0xdd, 0x0a, 0x0b, 0x17, 0x9c, 0x55, 0x59, 0x76, 0x29, 0x09, + 0x5b, 0x50, 0x60, 0xa6, 0x4e, 0xe4, 0xbe, 0xc7, 0x81, 0x8d, 0x29, 0x36, 0x6c, 0xd4, 0x85, 0xc2, + 0x6a, 0xfc, 0x19, 0xa0, 0x4d, 0xd2, 0x25, 0x1e, 0x99, 0x24, 0x7b, 0x28, 0x3e, 0x49, 0xab, 0x3e, + 0xc1, 0x7f, 0xa1, 0xc1, 0x7c, 0x48, 0xfc, 0x44, 0xd3, 0xaa, 0x42, 0xae, 0xc3, 0x84, 0x71, 0x0b, + 0xd2, 0xba, 0x6c, 0xa2, 0xa7, 0x30, 0x23, 0x0c, 0x70, 0xab, 0xe9, 0x84, 0x4d, 0x93, 0xe3, 0x36, + 0xb9, 0xf8, 0xbb, 0x14, 0xe4, 0xc5, 0x44, 0xf7, 0xfb, 0x68, 0x1d, 0x4a, 0x0e, 0x6f, 0xb4, 0xd8, + 0x7c, 0x84, 0x45, 0xb5, 0xe4, 0x24, 0xb4, 0x35, 0xa5, 0x17, 0xc5, 0x10, 0x46, 0x46, 0xbf, 0x03, + 0x05, 0x29, 0xa2, 0x3f, 0xf0, 0x84, 0xcb, 0xab, 0x61, 0x01, 0xc1, 0xfe, 0xdb, 0x9a, 0xd2, 0x41, + 0xb0, 0x1f, 0x0c, 0x3c, 0xd4, 0x84, 0x05, 0x39, 0x98, 0xcf, 0x46, 0x98, 0x91, 0x66, 0x52, 0x96, + 0xc3, 0x52, 0x46, 0x97, 0x6a, 0x6b, 0x4a, 0x47, 0x62, 0xbc, 0xd2, 0xa9, 0x9a, 0xe4, 0x9d, 0xf3, + 0xe4, 0x3d, 0x62, 0x52, 0xf3, 0xdc, 0x1a, 0x35, 0xa9, 0x79, 0x6e, 0xbd, 0xc8, 0x43, 0x4e, 0xb4, + 0xf0, 0x3f, 0xa7, 0x00, 0xe4, 0x6a, 0xec, 0xf7, 0xd1, 0x26, 0x94, 0x1d, 0xd1, 0x0a, 0x79, 0xeb, + 0x66, 0xac, 0xb7, 0xc4, 0x22, 0x4e, 0xe9, 0x25, 0x39, 0x88, 0x1b, 0xf7, 0x3e, 0x14, 0x7d, 0x29, + 0x81, 0xc3, 0x16, 0x63, 0x1c, 0xe6, 0x4b, 0x28, 0xc8, 0x01, 0xd4, 0x65, 0x9f, 0xc0, 0x35, 0x7f, + 0x7c, 0x8c, 0xcf, 0xee, 0x8e, 0xf1, 0x99, 0x2f, 0x70, 0x5e, 0x4a, 0x50, 0xbd, 0xa6, 0x1a, 0x16, + 0xb8, 0x6d, 0x31, 0xc6, 0x6d, 0xa3, 0x86, 0x51, 0xc7, 0x01, 0xad, 0x97, 0xbc, 0x89, 0xff, 0x27, + 0x0d, 0xb9, 0x0d, 0xbb, 0xd7, 0x37, 0x1c, 0xba, 0x1a, 0x59, 0x87, 0xb8, 0x83, 0xae, 0xc7, 0xdc, + 0x55, 0x5e, 0xbb, 0x17, 0x96, 0x28, 0xd8, 0xe4, 0xbf, 0x3a, 0x63, 0xd5, 0xc5, 0x10, 0x3a, 0x58, + 0x94, 0xc7, 0xd4, 0x15, 0x06, 0x8b, 0xe2, 0x28, 0x86, 0xc8, 0x40, 0x4e, 0x07, 0x81, 0x5c, 0x83, + 0xdc, 0x90, 0x38, 0x41, 0x49, 0xdf, 0x9a, 0xd2, 0x25, 0x01, 0x3d, 0x86, 0xd9, 0x68, 0x79, 0xc9, + 0x08, 0x9e, 0x72, 0x3b, 0x5c, 0x8d, 0xee, 0x41, 0x31, 0x54, 0xe3, 0xb2, 0x82, 0xaf, 0xd0, 0x53, + 0x4a, 0xdc, 0x75, 0x99, 0x57, 0x69, 0x3d, 0x2e, 0x6e, 0x4d, 0xc9, 0xcc, 0x7a, 0x5d, 0x66, 0xd6, + 0x19, 0x31, 0x4a, 0xe4, 0xd6, 0x50, 0x92, 0xf9, 0x20, 0x9c, 0x64, 0xf0, 0x07, 0x50, 0x0a, 0x39, + 0x88, 0xd6, 0x9d, 0xc6, 0xc7, 0xaf, 0xd7, 0x77, 0x79, 0x91, 0x7a, 0xc9, 0xea, 0x92, 0x5e, 0xd1, + 0x68, 0xad, 0xdb, 0x6d, 0x1c, 0x1e, 0x56, 0x52, 0xa8, 0x04, 0xf9, 0xbd, 0xfd, 0x66, 0x8b, 0x73, + 0xa5, 0xf1, 0x4b, 0x5f, 0x82, 0x28, 0x72, 0x4a, 0x6d, 0x9b, 0x52, 0x6a, 0x9b, 0x26, 0x6b, 0x5b, + 0x2a, 0xa8, 0x6d, 0xac, 0xcc, 0xed, 0x36, 0xd6, 0x0f, 0x1b, 0x95, 0xe9, 0x17, 0x65, 0x28, 0x72, + 0xff, 0xb6, 0x06, 0x16, 0x2d, 0xb5, 0x7f, 0xa7, 0x01, 0x04, 0xd1, 0x84, 0xea, 0x90, 0x6b, 0x73, + 0x3d, 0x55, 0x8d, 0x25, 0xa3, 0x6b, 0xb1, 0x4b, 0xa6, 0x4b, 0x2e, 0xf4, 0x6b, 0xc8, 0xb9, 0x83, + 0x76, 0x9b, 0xb8, 0xb2, 0xe4, 0xdd, 0x88, 0xe6, 0x43, 0x91, 0xad, 0x74, 0xc9, 0x47, 0x87, 0xbc, + 0x31, 0xcc, 0xee, 0x80, 0x15, 0xc0, 0xf1, 0x43, 0x04, 0x1f, 0xfe, 0x6b, 0x0d, 0x0a, 0xca, 0xe6, + 0xfd, 0x91, 0x49, 0xf8, 0x16, 0xe4, 0x99, 0x0d, 0xa4, 0x23, 0xd2, 0xf0, 0x8c, 0x1e, 0x10, 0xd0, + 0x6f, 0x43, 0x5e, 0x46, 0x80, 0xcc, 0xc4, 0xd5, 0x78, 0xb1, 0xfb, 0x7d, 0x3d, 0x60, 0xc5, 0x3b, + 0x30, 0xc7, 0xbc, 0xd2, 0xa6, 0x87, 0x6b, 0xe9, 0x47, 0xf5, 0xf8, 0xa9, 0x45, 0x8e, 0x9f, 0x35, + 0x98, 0xe9, 0x9f, 0x5e, 0xb8, 0x66, 0xdb, 0xe8, 0x0a, 0x2b, 0xfc, 0x36, 0xfe, 0x08, 0x90, 0x2a, + 0x6c, 0x92, 0xe9, 0xe2, 0x12, 0x14, 0xb6, 0x0c, 0xf7, 0x54, 0x98, 0x84, 0x9f, 0x42, 0x89, 0x36, + 0x77, 0x8e, 0xae, 0x60, 0x23, 0xbb, 0x1c, 0x48, 0xee, 0x89, 0x7c, 0x8e, 0x60, 0xfa, 0xd4, 0x70, + 0x4f, 0xd9, 0x44, 0x4b, 0x3a, 0xfb, 0x46, 0x8f, 0xa1, 0xd2, 0xe6, 0x93, 0x6c, 0x45, 0xae, 0x0c, + 0xb3, 0x82, 0xee, 0x9f, 0x04, 0x3f, 0x85, 0x22, 0x9f, 0xc3, 0x4f, 0x6d, 0x04, 0x9e, 0x83, 0xd9, + 0x43, 0xcb, 0xe8, 0xbb, 0xa7, 0xb6, 0xac, 0x6e, 0x74, 0xd2, 0x95, 0x80, 0x36, 0x91, 0xc6, 0x47, + 0x30, 0xeb, 0x90, 0x9e, 0x61, 0x5a, 0xa6, 0x75, 0xd2, 0x3a, 0xbe, 0xf0, 0x88, 0x2b, 0x2e, 0x4c, + 0x65, 0x9f, 0xfc, 0x82, 0x52, 0xa9, 0x69, 0xc7, 0x5d, 0xfb, 0x58, 0xa4, 0x39, 0xf6, 0x8d, 0xff, + 0x49, 0x83, 0xe2, 0x27, 0x86, 0xd7, 0x96, 0x4b, 0x87, 0xb6, 0xa1, 0xec, 0x27, 0x37, 0x46, 0x11, + 0xb6, 0x44, 0x4a, 0x2c, 0x1b, 0x23, 0x8f, 0xd2, 0xb2, 0x3a, 0x96, 0xda, 0x2a, 0x81, 0x89, 0x32, + 0xac, 0x36, 0xe9, 0xfa, 0xa2, 0x52, 0xc9, 0xa2, 0x18, 0xa3, 0x2a, 0x4a, 0x25, 0xbc, 0x98, 0x0d, + 0x8e, 0x1f, 0x3c, 0x97, 0xfc, 0x4d, 0x0a, 0xd0, 0xa8, 0x0d, 0x3f, 0xf4, 0x44, 0xf6, 0x00, 0xca, + 0xae, 0x67, 0x38, 0x23, 0x7b, 0xa3, 0xc4, 0xa8, 0x7e, 0x82, 0x7e, 0x04, 0xb3, 0x7d, 0xc7, 0x3e, + 0x71, 0x88, 0xeb, 0xb6, 0x2c, 0xdb, 0x33, 0xdf, 0x5c, 0x88, 0x43, 0x6d, 0x59, 0x92, 0xf7, 0x18, + 0x15, 0x35, 0x20, 0xf7, 0xc6, 0xec, 0x7a, 0xc4, 0x71, 0xab, 0x99, 0xe5, 0xf4, 0x4a, 0x79, 0xed, + 0xe9, 0x65, 0x5e, 0x5b, 0xfd, 0x90, 0xf1, 0x37, 0x2f, 0xfa, 0x44, 0x97, 0x63, 0xd5, 0x83, 0x62, + 0x36, 0x74, 0x50, 0x7c, 0x00, 0x10, 0xf0, 0xd3, 0x54, 0xbb, 0xb7, 0x7f, 0xf0, 0xba, 0x59, 0x99, + 0x42, 0x45, 0x98, 0xd9, 0xdb, 0xdf, 0x6c, 0xec, 0x36, 0x68, 0x5e, 0xc6, 0x75, 0xe9, 0x1b, 0xd5, + 0x87, 0x68, 0x11, 0x66, 0xde, 0x52, 0xaa, 0xbc, 0x6f, 0xa7, 0xf5, 0x1c, 0x6b, 0x6f, 0x77, 0xf0, + 0x9f, 0xa7, 0xa0, 0x24, 0x76, 0xc1, 0x44, 0x5b, 0x51, 0x55, 0x91, 0x0a, 0xa9, 0xa0, 0xa7, 0x52, + 0xbe, 0x3b, 0x3a, 0xe2, 0xf0, 0x2b, 0x9b, 0x34, 0x37, 0xf0, 0xc5, 0x26, 0x1d, 0xe1, 0x56, 0xbf, + 0x1d, 0x1b, 0xbe, 0x99, 0xd8, 0xf0, 0x45, 0xf7, 0xa0, 0xe4, 0xef, 0x36, 0xc3, 0x15, 0xb5, 0x36, + 0xaf, 0x17, 0xe5, 0x46, 0xa2, 0x34, 0xf4, 0x00, 0xb2, 0x64, 0x48, 0x2c, 0xcf, 0xad, 0x16, 0x58, + 0xd6, 0x2d, 0xc9, 0xf3, 0x6f, 0x83, 0x52, 0x75, 0xd1, 0x89, 0x7f, 0x0b, 0xe6, 0xd8, 0x3d, 0xe3, + 0xa5, 0x63, 0x58, 0xea, 0x85, 0xa8, 0xd9, 0xdc, 0x15, 0xae, 0xa3, 0x9f, 0xa8, 0x0c, 0xa9, 0xed, + 0x4d, 0x31, 0xd1, 0xd4, 0xf6, 0x26, 0xfe, 0x4a, 0x03, 0xa4, 0x8e, 0x9b, 0xc8, 0x97, 0x11, 0xe1, + 0x52, 0x7d, 0x3a, 0x50, 0xbf, 0x00, 0x19, 0xe2, 0x38, 0xb6, 0xc3, 0xbc, 0x96, 0xd7, 0x79, 0x03, + 0xdf, 0x17, 0x36, 0xe8, 0x64, 0x68, 0x9f, 0xf9, 0x81, 0xc1, 0xa5, 0x69, 0xbe, 0xa9, 0x3b, 0x30, + 0x1f, 0xe2, 0x9a, 0x28, 0xfb, 0x3f, 0x82, 0x6b, 0x4c, 0xd8, 0x0e, 0x21, 0xfd, 0xf5, 0xae, 0x39, + 0x4c, 0xd4, 0xda, 0x87, 0xeb, 0x51, 0xc6, 0x9f, 0xd7, 0x47, 0xf8, 0x77, 0x85, 0xc6, 0xa6, 0xd9, + 0x23, 0x4d, 0x7b, 0x37, 0xd9, 0x36, 0x9a, 0x1d, 0xcf, 0xc8, 0x85, 0x2b, 0xca, 0x24, 0xfb, 0xc6, + 0x7f, 0xaf, 0xc1, 0x8d, 0x91, 0xe1, 0x3f, 0xf3, 0xaa, 0x2e, 0x01, 0x9c, 0xd0, 0xed, 0x43, 0x3a, + 0xb4, 0x83, 0xdf, 0xd0, 0x15, 0x8a, 0x6f, 0x27, 0x4d, 0x30, 0x45, 0x61, 0xe7, 0x82, 0x58, 0x73, + 0xf6, 0xc7, 0x95, 0x35, 0xe6, 0x36, 0x14, 0x18, 0xe1, 0xd0, 0x33, 0xbc, 0x81, 0x3b, 0xb2, 0x18, + 0x7f, 0x24, 0xb6, 0x80, 0x1c, 0x34, 0xd1, 0xbc, 0x7e, 0x0d, 0x59, 0x76, 0x38, 0x95, 0x47, 0xb3, + 0xc8, 0x6d, 0x40, 0xb1, 0x43, 0x17, 0x8c, 0xf8, 0x14, 0xb2, 0xaf, 0x18, 0xa2, 0xa7, 0x58, 0x36, + 0x2d, 0x97, 0xc2, 0x32, 0x7a, 0x1c, 0x67, 0xc8, 0xeb, 0xec, 0x9b, 0x9d, 0x64, 0x08, 0x71, 0x5e, + 0xeb, 0xbb, 0xfc, 0xc4, 0x94, 0xd7, 0xfd, 0x36, 0x75, 0x59, 0xbb, 0x6b, 0x12, 0xcb, 0x63, 0xbd, + 0xd3, 0xac, 0x57, 0xa1, 0xe0, 0x55, 0xa8, 0x70, 0x4d, 0xeb, 0x9d, 0x8e, 0x72, 0x22, 0xf1, 0xe5, + 0x69, 0x61, 0x79, 0xf8, 0x1f, 0x34, 0x98, 0x53, 0x06, 0x4c, 0xe4, 0x98, 0x77, 0x20, 0xcb, 0x71, + 0x4b, 0x51, 0xfc, 0x16, 0xc2, 0xa3, 0xb8, 0x1a, 0x5d, 0xf0, 0xa0, 0x55, 0xc8, 0xf1, 0x2f, 0x79, + 0x2c, 0x8c, 0x67, 0x97, 0x4c, 0xf8, 0x01, 0xcc, 0x0b, 0x12, 0xe9, 0xd9, 0x71, 0x7b, 0x9b, 0x39, + 0x14, 0x7f, 0x09, 0x0b, 0x61, 0xb6, 0x89, 0xa6, 0xa4, 0x18, 0x99, 0xba, 0x8a, 0x91, 0xeb, 0xd2, + 0xc8, 0xd7, 0xfd, 0x8e, 0x52, 0xab, 0xa3, 0xab, 0xae, 0xae, 0x48, 0x2a, 0xb2, 0x22, 0xfe, 0x04, + 0xa4, 0x88, 0x5f, 0x74, 0x02, 0xf3, 0x72, 0x3b, 0xec, 0x9a, 0xae, 0x7f, 0x82, 0xfb, 0x02, 0x90, + 0x4a, 0xfc, 0xa5, 0x0d, 0xda, 0x24, 0x6f, 0x1c, 0xe3, 0xa4, 0x47, 0xfc, 0xfa, 0x44, 0xcf, 0xf3, + 0x2a, 0x71, 0xa2, 0x8c, 0x5e, 0x87, 0xb9, 0x57, 0xf6, 0x90, 0xa6, 0x06, 0x4a, 0x0d, 0x42, 0x86, + 0xdf, 0xe7, 0xfc, 0x65, 0xf3, 0xdb, 0x54, 0xb9, 0x3a, 0x60, 0x22, 0xe5, 0xff, 0xa6, 0x41, 0x71, + 0xbd, 0x6b, 0x38, 0x3d, 0xa9, 0xf8, 0x7d, 0xc8, 0xf2, 0x5b, 0x8a, 0x00, 0x06, 0x1e, 0x86, 0xc5, + 0xa8, 0xbc, 0xbc, 0xb1, 0xce, 0xef, 0x34, 0x62, 0x14, 0x35, 0x5c, 0xbc, 0x1d, 0x6c, 0x46, 0xde, + 0x12, 0x36, 0xd1, 0x33, 0xc8, 0x18, 0x74, 0x08, 0x4b, 0xc1, 0xe5, 0xe8, 0xfd, 0x90, 0x49, 0x63, + 0x87, 0x33, 0xce, 0x85, 0xdf, 0x83, 0x82, 0xa2, 0x81, 0xde, 0x80, 0x5f, 0x36, 0xc4, 0x01, 0x6c, + 0x7d, 0xa3, 0xb9, 0x7d, 0xc4, 0x2f, 0xc6, 0x65, 0x80, 0xcd, 0x86, 0xdf, 0x4e, 0xe1, 0x4f, 0xc5, + 0x28, 0x91, 0xef, 0x54, 0x7b, 0xb4, 0x24, 0x7b, 0x52, 0x57, 0xb2, 0xe7, 0x1c, 0x4a, 0x62, 0xfa, + 0x93, 0xa6, 0x6f, 0x26, 0x2f, 0x21, 0x7d, 0x2b, 0xc6, 0xeb, 0x82, 0x11, 0xcf, 0x42, 0x49, 0x24, + 0x74, 0xb1, 0xff, 0xfe, 0x55, 0x83, 0xb2, 0xa4, 0x4c, 0x0a, 0x60, 0x4a, 0xec, 0x85, 0x57, 0x00, + 0x1f, 0x79, 0xb9, 0x0e, 0xd9, 0xce, 0xf1, 0xa1, 0xf9, 0x85, 0x04, 0x9b, 0x45, 0x8b, 0xd2, 0xbb, + 0x5c, 0x0f, 0x7f, 0xf1, 0x11, 0x2d, 0x7a, 0x0b, 0x77, 0x8c, 0x37, 0xde, 0xb6, 0xd5, 0x21, 0xe7, + 0xec, 0xdc, 0x38, 0xad, 0x07, 0x04, 0x76, 0x29, 0x15, 0x2f, 0x43, 0xec, 0xb0, 0xa8, 0xbe, 0x14, + 0xcd, 0xc3, 0xdc, 0xfa, 0xc0, 0x3b, 0x6d, 0x58, 0xc6, 0x71, 0x57, 0x66, 0x2c, 0x5a, 0x66, 0x29, + 0x71, 0xd3, 0x74, 0x55, 0x6a, 0x03, 0xe6, 0x29, 0x95, 0x58, 0x9e, 0xd9, 0x56, 0xd2, 0x9b, 0x2c, + 0x62, 0x5a, 0xa4, 0x88, 0x19, 0xae, 0xfb, 0xd6, 0x76, 0x3a, 0x62, 0x6a, 0x7e, 0x1b, 0x6f, 0x72, + 0xe1, 0xaf, 0xdd, 0x50, 0x99, 0xfa, 0xa1, 0x52, 0x56, 0x02, 0x29, 0x2f, 0x89, 0x37, 0x46, 0x0a, + 0x7e, 0x0a, 0xd7, 0x24, 0xa7, 0x00, 0xf7, 0xc6, 0x30, 0xef, 0xc3, 0x6d, 0xc9, 0xbc, 0x71, 0x4a, + 0x6f, 0x4f, 0x07, 0x42, 0xe1, 0x8f, 0xb5, 0xf3, 0x05, 0x54, 0x7d, 0x3b, 0xd9, 0x61, 0xd9, 0xee, + 0xaa, 0x06, 0x0c, 0x5c, 0xb1, 0x67, 0xf2, 0x3a, 0xfb, 0xa6, 0x34, 0xc7, 0xee, 0xfa, 0x47, 0x02, + 0xfa, 0x8d, 0x37, 0x60, 0x51, 0xca, 0x10, 0xc7, 0xd8, 0xb0, 0x90, 0x11, 0x83, 0xe2, 0x84, 0x08, + 0x87, 0xd1, 0xa1, 0xe3, 0xdd, 0xae, 0x72, 0x86, 0x5d, 0xcb, 0x64, 0x6a, 0x8a, 0xcc, 0x6b, 0x7c, + 0x47, 0x50, 0xc3, 0xd4, 0x8a, 0x21, 0xc8, 0x54, 0x80, 0x4a, 0x16, 0x0b, 0x41, 0xc9, 0x23, 0x0b, + 0x31, 0x22, 0xfa, 0x33, 0x58, 0xf2, 0x8d, 0xa0, 0x7e, 0x3b, 0x20, 0x4e, 0xcf, 0x74, 0x5d, 0x05, + 0x0e, 0x8a, 0x9b, 0xf8, 0x43, 0x98, 0xee, 0x13, 0x91, 0x53, 0x0a, 0x6b, 0x68, 0x95, 0xbf, 0xdf, + 0xae, 0x2a, 0x83, 0x59, 0x3f, 0xee, 0xc0, 0x1d, 0x29, 0x9d, 0x7b, 0x34, 0x56, 0x7c, 0xd4, 0x28, + 0x79, 0xeb, 0xe6, 0x6e, 0x1d, 0xbd, 0x75, 0xa7, 0xf9, 0xda, 0xfb, 0x10, 0xe5, 0x47, 0xdc, 0x91, + 0x32, 0xb6, 0x26, 0xaa, 0x15, 0x3b, 0xdc, 0xa7, 0x7e, 0x48, 0x4e, 0x24, 0xec, 0x18, 0x16, 0xc2, + 0x91, 0x3c, 0x51, 0x1a, 0x5b, 0x80, 0x8c, 0x67, 0x9f, 0x11, 0x99, 0xc4, 0x78, 0x43, 0x1a, 0xec, + 0x87, 0xf9, 0x44, 0x06, 0x1b, 0x81, 0x30, 0xb6, 0x25, 0x27, 0xb5, 0x97, 0xae, 0xa6, 0x3c, 0x7c, + 0xf1, 0x06, 0xde, 0x83, 0xeb, 0xd1, 0x34, 0x31, 0x91, 0xc9, 0x47, 0x7c, 0x03, 0xc7, 0x65, 0x92, + 0x89, 0xe4, 0x7e, 0x1c, 0x24, 0x03, 0x25, 0xa1, 0x4c, 0x24, 0x52, 0x87, 0x5a, 0x5c, 0x7e, 0xf9, + 0x29, 0xf6, 0xab, 0x9f, 0x6e, 0x26, 0x12, 0xe6, 0x06, 0xc2, 0x26, 0x5f, 0xfe, 0x20, 0x47, 0xa4, + 0xc7, 0xe6, 0x08, 0x11, 0x24, 0x41, 0x16, 0xfb, 0x19, 0x36, 0x9d, 0xd0, 0x11, 0x24, 0xd0, 0x49, + 0x75, 0xd0, 0x1a, 0xe2, 0xeb, 0x60, 0x0d, 0xb9, 0xb1, 0xd5, 0xb4, 0x3b, 0xd1, 0x62, 0x7c, 0x12, + 0xe4, 0xce, 0x91, 0xcc, 0x3c, 0x91, 0xe0, 0x4f, 0x61, 0x39, 0x39, 0x29, 0x4f, 0x22, 0xf9, 0x49, + 0x1d, 0xf2, 0xfe, 0x81, 0x52, 0xf9, 0xed, 0x43, 0x01, 0x72, 0x7b, 0xfb, 0x87, 0x07, 0xeb, 0x1b, + 0x0d, 0xfe, 0xe3, 0x87, 0x8d, 0x7d, 0x5d, 0x7f, 0x7d, 0xd0, 0xac, 0xa4, 0xd6, 0xfe, 0x2f, 0x0d, + 0xa9, 0x9d, 0x23, 0xf4, 0xfb, 0x90, 0xe1, 0x2f, 0x81, 0x63, 0x9e, 0x7f, 0x6b, 0xe3, 0x1e, 0x3b, + 0xf1, 0xad, 0xaf, 0xfe, 0xe3, 0xbf, 0xbf, 0x4d, 0x5d, 0xc7, 0x73, 0xf5, 0xe1, 0xbb, 0x46, 0xb7, + 0x7f, 0x6a, 0xd4, 0xcf, 0x86, 0x75, 0x56, 0x20, 0x9e, 0x6b, 0x4f, 0xd0, 0x11, 0xa4, 0x0f, 0x06, + 0x1e, 0x4a, 0x7c, 0x1b, 0xae, 0x25, 0x3f, 0x82, 0xe2, 0x1a, 0x93, 0xbc, 0x80, 0x67, 0x55, 0xc9, + 0xfd, 0x81, 0x47, 0xe5, 0x0e, 0xa1, 0xa0, 0xbe, 0x63, 0x5e, 0xfa, 0x6a, 0x5c, 0xbb, 0xfc, 0x8d, + 0x14, 0x63, 0xa6, 0xef, 0x16, 0xbe, 0xa1, 0xea, 0xe3, 0xcf, 0xad, 0xea, 0x7c, 0x9a, 0xe7, 0x16, + 0x4a, 0x7c, 0x58, 0xae, 0x25, 0xbf, 0x9d, 0xc6, 0xcf, 0xc7, 0x3b, 0xb7, 0xa8, 0x5c, 0x5b, 0xbc, + 0x9d, 0xb6, 0x3d, 0x74, 0x27, 0xe6, 0xed, 0x4c, 0x7d, 0x25, 0xaa, 0x2d, 0x27, 0x33, 0x08, 0x4d, + 0x77, 0x99, 0xa6, 0x9b, 0xf8, 0xba, 0xaa, 0xa9, 0xed, 0xf3, 0x3d, 0xd7, 0x9e, 0xac, 0x9d, 0x42, + 0x86, 0xc1, 0xc4, 0xa8, 0x25, 0x3f, 0x6a, 0x31, 0x00, 0x77, 0xc2, 0x0e, 0x08, 0x01, 0xcc, 0x78, + 0x91, 0x69, 0x9b, 0xc7, 0x65, 0x5f, 0x1b, 0x43, 0x8a, 0x9f, 0x6b, 0x4f, 0x56, 0xb4, 0x5f, 0x69, + 0x6b, 0xff, 0x3b, 0x0d, 0x19, 0x06, 0x1a, 0xa1, 0x3e, 0x40, 0x80, 0xa9, 0x46, 0xe7, 0x39, 0x82, + 0xd2, 0x46, 0xe7, 0x39, 0x0a, 0xc7, 0xe2, 0x3b, 0x4c, 0xf3, 0x22, 0x5e, 0xf0, 0x35, 0x33, 0x40, + 0xaa, 0xce, 0x30, 0x36, 0xea, 0xd6, 0xb7, 0x02, 0x37, 0xe3, 0xd1, 0x86, 0xe2, 0x24, 0x86, 0xc0, + 0xd5, 0xe8, 0x36, 0x89, 0x01, 0x56, 0xf1, 0x3d, 0xa6, 0xf4, 0x36, 0xae, 0xaa, 0xce, 0xe5, 0x7a, + 0x1d, 0xc6, 0x49, 0x15, 0x7f, 0xad, 0x41, 0x39, 0x8c, 0x8f, 0xa2, 0x7b, 0x31, 0xa2, 0xa3, 0x30, + 0x6b, 0xed, 0xfe, 0x78, 0xa6, 0x44, 0x13, 0xb8, 0xfe, 0x33, 0x42, 0xfa, 0x06, 0xe5, 0x14, 0xbe, + 0x47, 0x7f, 0xa2, 0xc1, 0x6c, 0x04, 0xf5, 0x44, 0x71, 0x2a, 0x46, 0x30, 0xd5, 0xda, 0x83, 0x4b, + 0xb8, 0x84, 0x25, 0x8f, 0x98, 0x25, 0x77, 0xf1, 0xad, 0x51, 0x67, 0x78, 0x66, 0x8f, 0x78, 0xb6, + 0xb0, 0xc6, 0x5f, 0x09, 0x0e, 0x51, 0xc6, 0xae, 0x44, 0x08, 0xf2, 0x8c, 0x5d, 0x89, 0x30, 0xbe, + 0x39, 0x6e, 0x25, 0x38, 0x30, 0x49, 0x37, 0xfa, 0xff, 0xa7, 0x21, 0xb7, 0xc1, 0x7f, 0x8c, 0x88, + 0x3c, 0xc8, 0xfb, 0x60, 0x20, 0x5a, 0x8a, 0x03, 0x66, 0x82, 0x8b, 0x43, 0xed, 0x4e, 0x62, 0xbf, + 0x50, 0xff, 0x90, 0xa9, 0x5f, 0xc6, 0x37, 0x7d, 0xf5, 0xe2, 0x47, 0x8f, 0x75, 0x0e, 0x01, 0xd4, + 0x8d, 0x4e, 0x87, 0x4e, 0xfd, 0x8f, 0x35, 0x28, 0xaa, 0x98, 0x1d, 0xba, 0x1b, 0x0b, 0x09, 0xa9, + 0xb0, 0x5f, 0x0d, 0x8f, 0x63, 0x11, 0xfa, 0x1f, 0x33, 0xfd, 0xf7, 0xf0, 0x52, 0x92, 0x7e, 0x87, + 0xf1, 0x87, 0x4d, 0xe0, 0xa8, 0x5b, 0xbc, 0x09, 0x21, 0x50, 0x2f, 0xde, 0x84, 0x30, 0x68, 0x77, + 0xb9, 0x09, 0x03, 0xc6, 0x4f, 0x4d, 0x38, 0x07, 0x08, 0x40, 0x36, 0x14, 0xeb, 0x5c, 0xe5, 0x2a, + 0x15, 0x0d, 0xfe, 0x51, 0x7c, 0x2e, 0x66, 0xeb, 0x45, 0x74, 0x77, 0x4d, 0x97, 0x26, 0x81, 0xb5, + 0x7f, 0xcc, 0x42, 0xe1, 0x95, 0x61, 0x5a, 0x1e, 0xb1, 0x0c, 0xab, 0x4d, 0xd0, 0x09, 0x64, 0x58, + 0xad, 0x8c, 0x66, 0x3c, 0x15, 0x7c, 0x8a, 0x66, 0xbc, 0x10, 0x32, 0x83, 0x1f, 0x30, 0xd5, 0x77, + 0x70, 0xcd, 0x57, 0xdd, 0x0b, 0xe4, 0xd7, 0x19, 0xaa, 0x42, 0xa7, 0x7c, 0x06, 0x59, 0x01, 0xd8, + 0x47, 0xa4, 0x85, 0xd0, 0x96, 0xda, 0xad, 0xf8, 0xce, 0xc4, 0x5d, 0xa6, 0xea, 0x72, 0x19, 0x33, + 0x55, 0xf6, 0x07, 0x00, 0x01, 0x66, 0x18, 0xf5, 0xef, 0x08, 0xc4, 0x58, 0x5b, 0x4e, 0x66, 0x10, + 0x8a, 0x9f, 0x30, 0xc5, 0xf7, 0xf1, 0x9d, 0x58, 0xc5, 0x1d, 0x7f, 0x00, 0x55, 0xde, 0x86, 0xe9, + 0x2d, 0xc3, 0x3d, 0x45, 0x91, 0xea, 0xa7, 0xfc, 0x90, 0xa0, 0x56, 0x8b, 0xeb, 0x12, 0xaa, 0xee, + 0x33, 0x55, 0x4b, 0x78, 0x31, 0x56, 0xd5, 0xa9, 0xe1, 0xd2, 0x62, 0x82, 0x4c, 0xc8, 0xf2, 0x1f, + 0x17, 0x44, 0xdd, 0x19, 0xfa, 0x81, 0x42, 0xd4, 0x9d, 0xe1, 0xdf, 0x23, 0x5c, 0x51, 0xd5, 0x00, + 0x66, 0xe4, 0x93, 0x3e, 0xba, 0x1d, 0x59, 0x9e, 0xf0, 0xf3, 0x7f, 0x6d, 0x29, 0xa9, 0x5b, 0x28, + 0x5c, 0x61, 0x0a, 0x31, 0xbe, 0x1d, 0xbf, 0x7e, 0x82, 0xfd, 0xb9, 0xf6, 0xe4, 0x57, 0x1a, 0xad, + 0x1a, 0x10, 0x60, 0xaf, 0x23, 0x41, 0x12, 0x85, 0x71, 0x47, 0x82, 0x64, 0x04, 0xb6, 0xc5, 0xef, + 0x32, 0xed, 0xcf, 0xf0, 0x4a, 0xac, 0x76, 0xcf, 0x31, 0x2c, 0xf7, 0x0d, 0x71, 0x9e, 0x71, 0x90, + 0xcd, 0x3d, 0x35, 0xfb, 0x34, 0x60, 0xfe, 0xac, 0x02, 0xd3, 0xf4, 0x9c, 0x4a, 0x0b, 0x76, 0x70, + 0xbd, 0x8f, 0x9a, 0x33, 0x02, 0xaa, 0x45, 0xcd, 0x19, 0x45, 0x06, 0x62, 0x0a, 0x36, 0xfb, 0x11, + 0x3a, 0x61, 0x5c, 0xd4, 0xf1, 0x1e, 0x14, 0x14, 0x10, 0x00, 0xc5, 0x48, 0x0c, 0x43, 0x76, 0xd1, + 0x32, 0x11, 0x83, 0x20, 0xe0, 0x65, 0xa6, 0xb4, 0x86, 0xaf, 0x85, 0x95, 0x76, 0x38, 0x1b, 0xd5, + 0xfa, 0x25, 0x14, 0x55, 0xb4, 0x00, 0xc5, 0x08, 0x8d, 0x60, 0x82, 0xd1, 0xec, 0x18, 0x07, 0x36, + 0xc4, 0xa4, 0x09, 0xff, 0x27, 0xf7, 0x92, 0x97, 0x6a, 0xff, 0x1c, 0x72, 0x02, 0x43, 0x88, 0x9b, + 0x6f, 0x18, 0x45, 0x8c, 0x9b, 0x6f, 0x04, 0x80, 0x88, 0x39, 0xfd, 0x31, 0xb5, 0xf4, 0xae, 0x24, + 0x4b, 0x92, 0x50, 0xf9, 0x92, 0x78, 0x49, 0x2a, 0x03, 0x5c, 0x2c, 0x49, 0xa5, 0x72, 0x4f, 0x1d, + 0xab, 0xf2, 0x84, 0x78, 0x22, 0xa4, 0xe4, 0x25, 0x10, 0x25, 0x48, 0x54, 0xf3, 0x3f, 0x1e, 0xc7, + 0x92, 0x78, 0x60, 0x0f, 0xb4, 0x8a, 0xe4, 0x8f, 0xfe, 0x10, 0x20, 0x00, 0x3c, 0xa2, 0x67, 0xb0, + 0x58, 0xd4, 0x34, 0x7a, 0x06, 0x8b, 0xc7, 0x4c, 0x62, 0x12, 0x49, 0xa0, 0x9c, 0x5f, 0x1a, 0xa8, + 0xfa, 0xbf, 0xd4, 0x00, 0x8d, 0x02, 0x24, 0xe8, 0x69, 0xbc, 0x8a, 0x58, 0x40, 0xb6, 0xf6, 0xce, + 0xd5, 0x98, 0x13, 0xeb, 0x45, 0x60, 0x57, 0x9b, 0x0d, 0xe9, 0xbf, 0xa5, 0x96, 0x7d, 0xa3, 0x41, + 0x29, 0x04, 0xb1, 0xa0, 0x87, 0x09, 0xeb, 0x1c, 0x01, 0x75, 0x6b, 0x8f, 0x2e, 0xe5, 0x4b, 0x3c, + 0x9f, 0x29, 0xbb, 0x42, 0x1e, 0xd1, 0xff, 0x54, 0x83, 0x72, 0x18, 0x97, 0x41, 0x09, 0x0a, 0x46, + 0x90, 0xe1, 0xda, 0xca, 0xe5, 0x8c, 0x57, 0x58, 0xad, 0xe0, 0xd4, 0xfe, 0x39, 0xe4, 0x04, 0x9c, + 0x13, 0x17, 0x16, 0x61, 0x60, 0x39, 0x2e, 0x2c, 0x22, 0x58, 0x50, 0x52, 0x58, 0x38, 0x76, 0x97, + 0x28, 0x91, 0x28, 0x40, 0x9f, 0x24, 0x95, 0xe3, 0x23, 0x31, 0x82, 0x18, 0x8d, 0x55, 0x19, 0x44, + 0xa2, 0x84, 0x7c, 0x50, 0x82, 0xc4, 0x4b, 0x22, 0x31, 0x8a, 0x18, 0x25, 0x45, 0x22, 0xd3, 0xaa, + 0x44, 0x62, 0x80, 0xd0, 0xc4, 0x45, 0xe2, 0x08, 0x6c, 0x1e, 0x17, 0x89, 0xa3, 0x20, 0x4f, 0xd2, + 0xda, 0x32, 0xe5, 0xa1, 0x48, 0x9c, 0x8f, 0x41, 0x74, 0xd0, 0x3b, 0x09, 0x3e, 0x8d, 0x85, 0xe4, + 0x6b, 0xcf, 0xae, 0xc8, 0x3d, 0x3e, 0x02, 0xf8, 0x6a, 0xc8, 0x08, 0xf8, 0x5b, 0x0d, 0x16, 0xe2, + 0x20, 0x21, 0x94, 0xa0, 0x2c, 0x01, 0xcf, 0xaf, 0xad, 0x5e, 0x95, 0xfd, 0x0a, 0x7e, 0xf3, 0x63, + 0xe2, 0x45, 0xe5, 0x5f, 0xbe, 0x5f, 0xd2, 0xfe, 0xfd, 0xfb, 0x25, 0xed, 0x3f, 0xbf, 0x5f, 0xd2, + 0xfe, 0xea, 0xbf, 0x96, 0xa6, 0x8e, 0xb3, 0xec, 0x7f, 0x82, 0xbd, 0xfb, 0x9b, 0x00, 0x00, 0x00, + 0xff, 0xff, 0xdd, 0x84, 0xb6, 0xd7, 0x90, 0x36, 0x00, 0x00, } diff --git a/etcdserver/etcdserverpb/rpc.proto b/etcdserver/etcdserverpb/rpc.proto index be1b9ab15fa..2c18ce101c6 100644 --- a/etcdserver/etcdserverpb/rpc.proto +++ b/etcdserver/etcdserverpb/rpc.proto @@ -852,6 +852,7 @@ message MoveLeaderResponse { enum AlarmType { NONE = 0; // default, used to query if any alarm is active NOSPACE = 1; // space quota is exhausted + CORRUPT = 2; // kv store corruption detected } message AlarmRequest { diff --git a/etcdserver/server.go b/etcdserver/server.go index 8fb830c1a8e..ac7601ecb0e 100644 --- a/etcdserver/server.go +++ b/etcdserver/server.go @@ -524,6 +524,7 @@ func (s *EtcdServer) Start() { s.goAttach(func() { monitorFileDescriptor(s.stopping) }) s.goAttach(s.monitorVersions) s.goAttach(s.linearizableReadLoop) + s.goAttach(s.monitorKVHash) } // start prepares and starts server in a new goroutine. It is no longer safe to @@ -1628,6 +1629,9 @@ func (s *EtcdServer) restoreAlarms() error { if len(as.Get(pb.AlarmType_NOSPACE)) > 0 { s.applyV3 = newApplierV3Capped(s.applyV3) } + if len(as.Get(pb.AlarmType_CORRUPT)) > 0 { + s.applyV3 = newApplierV3Corrupt(s.applyV3) + } return nil } diff --git a/glide.lock b/glide.lock index 32138d23113..528cd4d36fa 100644 --- a/glide.lock +++ b/glide.lock @@ -97,6 +97,8 @@ imports: - xfs - name: github.com/russross/blackfriday version: 4048872b16cc0fc2c5fd9eacf0ed2c2fedaa0c8c +- name: github.com/soheilhy/cmux + version: bb79a83465015a27a175925ebd155e660f55e9f1 - name: github.com/spf13/cobra version: 1c44ec8d3f1552cac48999f9306da23c4d8a288b - name: github.com/spf13/pflag diff --git a/glide.yaml b/glide.yaml index 513d767da07..1fb5602bb16 100644 --- a/glide.yaml +++ b/glide.yaml @@ -1,4 +1,6 @@ package: github.com/coreos/etcd +ignore: +- google.golang.org/appengine import: - package: github.com/bgentry/speakeasy version: v0.1.0 @@ -128,5 +130,5 @@ import: - unicode/norm - package: github.com/russross/blackfriday version: 4048872b16cc0fc2c5fd9eacf0ed2c2fedaa0c8c -ignore: - - google.golang.org/appengine +- package: github.com/soheilhy/cmux + version: v0.1.3 diff --git a/integration/cluster.go b/integration/cluster.go index ed245eca2f2..7dcfdcdd49e 100644 --- a/integration/cluster.go +++ b/integration/cluster.go @@ -31,6 +31,7 @@ import ( "testing" "time" + "github.com/soheilhy/cmux" "golang.org/x/net/context" "google.golang.org/grpc" @@ -475,13 +476,14 @@ type member struct { // ClientTLSInfo enables client TLS when set ClientTLSInfo *transport.TLSInfo - raftHandler *testutil.PauseableHandler - s *etcdserver.EtcdServer - hss []*httptest.Server + raftHandler *testutil.PauseableHandler + s *etcdserver.EtcdServer + serverClosers []func() - grpcServer *grpc.Server - grpcAddr string - grpcBridge *bridge + grpcServer *grpc.Server + grpcServerPeer *grpc.Server + grpcAddr string + grpcBridge *bridge // serverClient is a clientv3 that directly calls the etcdserver. serverClient *clientv3.Client @@ -649,23 +651,80 @@ func (m *member) Launch() error { m.s.SyncTicker = time.NewTicker(500 * time.Millisecond) m.s.Start() + var peerTLScfg *tls.Config + if m.PeerTLSInfo != nil && !m.PeerTLSInfo.Empty() { + if peerTLScfg, err = m.PeerTLSInfo.ServerConfig(); err != nil { + return err + } + } + + if m.grpcListener != nil { + var ( + tlscfg *tls.Config + ) + if m.ClientTLSInfo != nil && !m.ClientTLSInfo.Empty() { + tlscfg, err = m.ClientTLSInfo.ServerConfig() + if err != nil { + return err + } + } + m.grpcServer = v3rpc.Server(m.s, tlscfg) + m.grpcServerPeer = v3rpc.Server(m.s, peerTLScfg) + m.serverClient = v3client.New(m.s) + lockpb.RegisterLockServer(m.grpcServer, v3lock.NewLockServer(m.serverClient)) + epb.RegisterElectionServer(m.grpcServer, v3election.NewElectionServer(m.serverClient)) + go m.grpcServer.Serve(m.grpcListener) + } + m.raftHandler = &testutil.PauseableHandler{Next: etcdhttp.NewPeerHandler(m.s)} + h := (http.Handler)(m.raftHandler) + if m.grpcListener != nil { + h = http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + if r.ProtoMajor == 2 && strings.Contains(r.Header.Get("Content-Type"), "application/grpc") { + m.grpcServerPeer.ServeHTTP(w, r) + } else { + m.raftHandler.ServeHTTP(w, r) + } + }) + } + for _, ln := range m.PeerListeners { - hs := &httptest.Server{ - Listener: ln, - Config: &http.Server{Handler: m.raftHandler}, + cm := cmux.New(ln) + // don't hang on matcher after closing listener + cm.SetReadTimeout(time.Second) + + if m.grpcServer != nil { + grpcl := cm.Match(cmux.HTTP2()) + go m.grpcServerPeer.Serve(grpcl) } - if m.PeerTLSInfo == nil { - hs.Start() - } else { - hs.TLS, err = m.PeerTLSInfo.ServerConfig() - if err != nil { + + // serve http1/http2 rafthttp/grpc + ll := cm.Match(cmux.Any()) + if peerTLScfg != nil { + if ll, err = transport.NewTLSListener(ll, m.PeerTLSInfo); err != nil { return err } - hs.StartTLS() } - m.hss = append(m.hss, hs) + hs := &httptest.Server{ + Listener: ll, + Config: &http.Server{Handler: h, TLSConfig: peerTLScfg}, + TLS: peerTLScfg, + } + hs.Start() + + donec := make(chan struct{}) + go func() { + defer close(donec) + cm.Serve() + }() + closer := func() { + ll.Close() + hs.CloseClientConnections() + hs.Close() + <-donec + } + m.serverClosers = append(m.serverClosers, closer) } for _, ln := range m.ClientListeners { hs := &httptest.Server{ @@ -681,23 +740,12 @@ func (m *member) Launch() error { } hs.StartTLS() } - m.hss = append(m.hss, hs) - } - if m.grpcListener != nil { - var ( - tlscfg *tls.Config - ) - if m.ClientTLSInfo != nil && !m.ClientTLSInfo.Empty() { - tlscfg, err = m.ClientTLSInfo.ServerConfig() - if err != nil { - return err - } + closer := func() { + ln.Close() + hs.CloseClientConnections() + hs.Close() } - m.grpcServer = v3rpc.Server(m.s, tlscfg) - m.serverClient = v3client.New(m.s) - lockpb.RegisterLockServer(m.grpcServer, v3lock.NewLockServer(m.serverClient)) - epb.RegisterElectionServer(m.grpcServer, v3election.NewElectionServer(m.serverClient)) - go m.grpcServer.Serve(m.grpcListener) + m.serverClosers = append(m.serverClosers, closer) } plog.Printf("launched %s (%s)", m.Name, m.grpcAddr) @@ -745,13 +793,16 @@ func (m *member) Close() { m.serverClient = nil } if m.grpcServer != nil { + m.grpcServer.Stop() m.grpcServer.GracefulStop() m.grpcServer = nil + m.grpcServerPeer.Stop() + m.grpcServerPeer.GracefulStop() + m.grpcServerPeer = nil } m.s.HardStop() - for _, hs := range m.hss { - hs.CloseClientConnections() - hs.Close() + for _, f := range m.serverClosers { + f() } } @@ -759,7 +810,7 @@ func (m *member) Close() { func (m *member) Stop(t *testing.T) { plog.Printf("stopping %s (%s)", m.Name, m.grpcAddr) m.Close() - m.hss = nil + m.serverClosers = nil plog.Printf("stopped %s (%s)", m.Name, m.grpcAddr) } diff --git a/integration/v3_alarm_test.go b/integration/v3_alarm_test.go new file mode 100644 index 00000000000..9c735231d1c --- /dev/null +++ b/integration/v3_alarm_test.go @@ -0,0 +1,217 @@ +// Copyright 2017 The etcd Authors +// +// 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 integration + +import ( + "os" + "path/filepath" + "sync" + "testing" + "time" + + "github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes" + pb "github.com/coreos/etcd/etcdserver/etcdserverpb" + "github.com/coreos/etcd/mvcc" + "github.com/coreos/etcd/mvcc/backend" + "github.com/coreos/etcd/pkg/testutil" + "golang.org/x/net/context" +) + +// TestV3StorageQuotaApply tests the V3 server respects quotas during apply +func TestV3StorageQuotaApply(t *testing.T) { + testutil.AfterTest(t) + quotasize := int64(16 * os.Getpagesize()) + + clus := NewClusterV3(t, &ClusterConfig{Size: 2}) + defer clus.Terminate(t) + kvc0 := toGRPC(clus.Client(0)).KV + kvc1 := toGRPC(clus.Client(1)).KV + + // Set a quota on one node + clus.Members[0].QuotaBackendBytes = quotasize + clus.Members[0].Stop(t) + clus.Members[0].Restart(t) + clus.waitLeader(t, clus.Members) + waitForRestart(t, kvc0) + + key := []byte("abc") + + // test small put still works + smallbuf := make([]byte, 1024) + _, serr := kvc0.Put(context.TODO(), &pb.PutRequest{Key: key, Value: smallbuf}) + if serr != nil { + t.Fatal(serr) + } + + // test big put + bigbuf := make([]byte, quotasize) + _, err := kvc1.Put(context.TODO(), &pb.PutRequest{Key: key, Value: bigbuf}) + if err != nil { + t.Fatal(err) + } + + // quorum get should work regardless of whether alarm is raised + _, err = kvc0.Range(context.TODO(), &pb.RangeRequest{Key: []byte("foo")}) + if err != nil { + t.Fatal(err) + } + + // wait until alarm is raised for sure-- poll the alarms + stopc := time.After(5 * time.Second) + for { + req := &pb.AlarmRequest{Action: pb.AlarmRequest_GET} + resp, aerr := clus.Members[0].s.Alarm(context.TODO(), req) + if aerr != nil { + t.Fatal(aerr) + } + if len(resp.Alarms) != 0 { + break + } + select { + case <-stopc: + t.Fatalf("timed out waiting for alarm") + case <-time.After(10 * time.Millisecond): + } + } + + // small quota machine should reject put + if _, err := kvc0.Put(context.TODO(), &pb.PutRequest{Key: key, Value: smallbuf}); err == nil { + t.Fatalf("past-quota instance should reject put") + } + + // large quota machine should reject put + if _, err := kvc1.Put(context.TODO(), &pb.PutRequest{Key: key, Value: smallbuf}); err == nil { + t.Fatalf("past-quota instance should reject put") + } + + // reset large quota node to ensure alarm persisted + clus.Members[1].Stop(t) + clus.Members[1].Restart(t) + clus.waitLeader(t, clus.Members) + + if _, err := kvc1.Put(context.TODO(), &pb.PutRequest{Key: key, Value: smallbuf}); err == nil { + t.Fatalf("alarmed instance should reject put after reset") + } +} + +// TestV3AlarmDeactivate ensures that space alarms can be deactivated so puts go through. +func TestV3AlarmDeactivate(t *testing.T) { + clus := NewClusterV3(t, &ClusterConfig{Size: 3}) + defer clus.Terminate(t) + kvc := toGRPC(clus.RandClient()).KV + mt := toGRPC(clus.RandClient()).Maintenance + + alarmReq := &pb.AlarmRequest{ + MemberID: 123, + Action: pb.AlarmRequest_ACTIVATE, + Alarm: pb.AlarmType_NOSPACE, + } + if _, err := mt.Alarm(context.TODO(), alarmReq); err != nil { + t.Fatal(err) + } + + key := []byte("abc") + smallbuf := make([]byte, 512) + _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: key, Value: smallbuf}) + if err == nil && !eqErrGRPC(err, rpctypes.ErrGRPCNoSpace) { + t.Fatalf("put got %v, expected %v", err, rpctypes.ErrGRPCNoSpace) + } + + alarmReq.Action = pb.AlarmRequest_DEACTIVATE + if _, err = mt.Alarm(context.TODO(), alarmReq); err != nil { + t.Fatal(err) + } + + if _, err = kvc.Put(context.TODO(), &pb.PutRequest{Key: key, Value: smallbuf}); err != nil { + t.Fatal(err) + } +} + +type fakeConsistentIndex struct{ rev uint64 } + +func (f *fakeConsistentIndex) ConsistentIndex() uint64 { return f.rev } + +func TestV3CorruptAlarm(t *testing.T) { + defer testutil.AfterTest(t) + clus := NewClusterV3(t, &ClusterConfig{Size: 3}) + defer clus.Terminate(t) + + var wg sync.WaitGroup + wg.Add(10) + for i := 0; i < 10; i++ { + go func() { + defer wg.Done() + if _, err := clus.Client(0).Put(context.TODO(), "k", "v"); err != nil { + t.Fatal(err) + } + }() + } + wg.Wait() + + // Corrupt member 0 by modifying backend offline. + clus.Members[0].Stop(t) + fp := filepath.Join(clus.Members[0].DataDir, "member", "snap", "db") + be := backend.NewDefaultBackend(fp) + s := mvcc.NewStore(be, nil, &fakeConsistentIndex{13}) + // NOTE: cluster_proxy mode with namespacing won't set 'k', but namespace/'k'. + s.Put([]byte("abc"), []byte("def"), 0) + s.Put([]byte("xyz"), []byte("123"), 0) + s.Compact(5) + s.Commit() + s.Close() + be.Close() + + // Wait for cluster so Puts succeed in case member 0 was the leader. + if _, err := clus.Client(1).Get(context.TODO(), "k"); err != nil { + t.Fatal(err) + } + clus.Client(1).Put(context.TODO(), "xyz", "321") + clus.Client(1).Put(context.TODO(), "abc", "fed") + + // Restart with corruption checking enabled. + clus.Members[1].Stop(t) + clus.Members[2].Stop(t) + for _, m := range clus.Members { + m.CorruptCheckTime = time.Second + m.Restart(t) + } + // Member 0 restarts into split brain. + + resp0, err0 := clus.Client(0).Get(context.TODO(), "abc") + if err0 != nil { + t.Fatal(err0) + } + resp1, err1 := clus.Client(1).Get(context.TODO(), "abc") + if err1 != nil { + t.Fatal(err1) + } + + if resp0.Kvs[0].ModRevision == resp1.Kvs[0].ModRevision { + t.Fatalf("matching ModRevision values") + } + + for i := 0; i < 5; i++ { + presp, perr := clus.Client(0).Put(context.TODO(), "abc", "aaa") + if perr != nil { + if !eqErrGRPC(perr, rpctypes.ErrCorrupt) { + t.Fatalf("expected %v, got %+v (%v)", rpctypes.ErrCorrupt, presp, perr) + } else { + return + } + } + time.Sleep(time.Second) + } + t.Fatalf("expected error %v after %s", rpctypes.ErrCorrupt, 5*time.Second) +} diff --git a/integration/v3_grpc_test.go b/integration/v3_grpc_test.go index 13c2b875636..fb131f090b8 100644 --- a/integration/v3_grpc_test.go +++ b/integration/v3_grpc_test.go @@ -1277,116 +1277,6 @@ func TestV3StorageQuotaAPI(t *testing.T) { } } -// TestV3StorageQuotaApply tests the V3 server respects quotas during apply -func TestV3StorageQuotaApply(t *testing.T) { - testutil.AfterTest(t) - quotasize := int64(16 * os.Getpagesize()) - - clus := NewClusterV3(t, &ClusterConfig{Size: 2}) - defer clus.Terminate(t) - kvc0 := toGRPC(clus.Client(0)).KV - kvc1 := toGRPC(clus.Client(1)).KV - - // Set a quota on one node - clus.Members[0].QuotaBackendBytes = quotasize - clus.Members[0].Stop(t) - clus.Members[0].Restart(t) - clus.waitLeader(t, clus.Members) - waitForRestart(t, kvc0) - - key := []byte("abc") - - // test small put still works - smallbuf := make([]byte, 1024) - _, serr := kvc0.Put(context.TODO(), &pb.PutRequest{Key: key, Value: smallbuf}) - if serr != nil { - t.Fatal(serr) - } - - // test big put - bigbuf := make([]byte, quotasize) - _, err := kvc1.Put(context.TODO(), &pb.PutRequest{Key: key, Value: bigbuf}) - if err != nil { - t.Fatal(err) - } - - // quorum get should work regardless of whether alarm is raised - _, err = kvc0.Range(context.TODO(), &pb.RangeRequest{Key: []byte("foo")}) - if err != nil { - t.Fatal(err) - } - - // wait until alarm is raised for sure-- poll the alarms - stopc := time.After(5 * time.Second) - for { - req := &pb.AlarmRequest{Action: pb.AlarmRequest_GET} - resp, aerr := clus.Members[0].s.Alarm(context.TODO(), req) - if aerr != nil { - t.Fatal(aerr) - } - if len(resp.Alarms) != 0 { - break - } - select { - case <-stopc: - t.Fatalf("timed out waiting for alarm") - case <-time.After(10 * time.Millisecond): - } - } - - // small quota machine should reject put - if _, err := kvc0.Put(context.TODO(), &pb.PutRequest{Key: key, Value: smallbuf}); err == nil { - t.Fatalf("past-quota instance should reject put") - } - - // large quota machine should reject put - if _, err := kvc1.Put(context.TODO(), &pb.PutRequest{Key: key, Value: smallbuf}); err == nil { - t.Fatalf("past-quota instance should reject put") - } - - // reset large quota node to ensure alarm persisted - clus.Members[1].Stop(t) - clus.Members[1].Restart(t) - clus.waitLeader(t, clus.Members) - - if _, err := kvc1.Put(context.TODO(), &pb.PutRequest{Key: key, Value: smallbuf}); err == nil { - t.Fatalf("alarmed instance should reject put after reset") - } -} - -// TestV3AlarmDeactivate ensures that space alarms can be deactivated so puts go through. -func TestV3AlarmDeactivate(t *testing.T) { - clus := NewClusterV3(t, &ClusterConfig{Size: 3}) - defer clus.Terminate(t) - kvc := toGRPC(clus.RandClient()).KV - mt := toGRPC(clus.RandClient()).Maintenance - - alarmReq := &pb.AlarmRequest{ - MemberID: 123, - Action: pb.AlarmRequest_ACTIVATE, - Alarm: pb.AlarmType_NOSPACE, - } - if _, err := mt.Alarm(context.TODO(), alarmReq); err != nil { - t.Fatal(err) - } - - key := []byte("abc") - smallbuf := make([]byte, 512) - _, err := kvc.Put(context.TODO(), &pb.PutRequest{Key: key, Value: smallbuf}) - if err == nil && !eqErrGRPC(err, rpctypes.ErrGRPCNoSpace) { - t.Fatalf("put got %v, expected %v", err, rpctypes.ErrGRPCNoSpace) - } - - alarmReq.Action = pb.AlarmRequest_DEACTIVATE - if _, err = mt.Alarm(context.TODO(), alarmReq); err != nil { - t.Fatal(err) - } - - if _, err = kvc.Put(context.TODO(), &pb.PutRequest{Key: key, Value: smallbuf}); err != nil { - t.Fatal(err) - } -} - func TestV3RangeRequest(t *testing.T) { defer testutil.AfterTest(t) tests := []struct {