Browse Source

Merge pull request #8420 from heyitsanthony/corrupt-alarm

corruption alarm
Anthony Romano 8 years ago
parent
commit
5c975fdb10

+ 2 - 1
Documentation/dev-guide/apispec/swagger/rpc.swagger.json

@@ -1163,7 +1163,8 @@
       "default": "NONE",
       "enum": [
         "NONE",
-        "NOSPACE"
+        "NOSPACE",
+        "CORRUPT"
       ]
     },
     "etcdserverpbAuthDisableRequest": {

+ 9 - 9
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": [

+ 0 - 35
cmd/vendor/github.com/cockroachdb/cmux/buffer.go

@@ -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
-}

+ 0 - 150
cmd/vendor/github.com/cockroachdb/cmux/matchers.go

@@ -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
-			}
-		}
-	}
-}

+ 0 - 0
cmd/vendor/github.com/cockroachdb/cmux/LICENSE → cmd/vendor/github.com/soheilhy/cmux/LICENSE


+ 67 - 0
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()
+}

+ 80 - 21
cmd/vendor/github.com/cockroachdb/cmux/cmux.go → 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)
 }

+ 18 - 0
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

+ 262 - 0
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
+		}
+	}
+}

+ 51 - 45
cmd/vendor/github.com/cockroachdb/cmux/patricia.go → 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)
 }

+ 5 - 0
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

+ 49 - 29
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) {

+ 2 - 2
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) {

+ 3 - 0
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, "")

+ 1 - 1
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"
 )

+ 4 - 0
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.
 `
 )

+ 3 - 0
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.

+ 1 - 0
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,

+ 5 - 2
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:

+ 2 - 0
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

+ 182 - 0
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
+}

+ 1 - 0
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 {

+ 231 - 227
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,
 }

+ 1 - 0
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 {

+ 4 - 0
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
 }
 

+ 2 - 0
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

+ 4 - 2
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

+ 87 - 36
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)
 }
 

+ 217 - 0
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)
+}

+ 0 - 110
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 {