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",
       "default": "NONE",
       "enum": [
       "enum": [
         "NONE",
         "NONE",
-        "NOSPACE"
+        "NOSPACE",
+        "CORRUPT"
       ]
       ]
     },
     },
     "etcdserverpbAuthDisableRequest": {
     "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",
 		"project": "github.com/coreos/bbolt",
 		"licenses": [
 		"licenses": [
@@ -264,6 +255,15 @@
 			}
 			}
 		]
 		]
 	},
 	},
+	{
+		"project": "github.com/soheilhy/cmux",
+		"licenses": [
+			{
+				"type": "Apache License 2.0",
+				"confidence": 1
+			}
+		]
+	},
 	{
 	{
 		"project": "github.com/spf13/cobra",
 		"project": "github.com/spf13/cobra",
 		"licenses": [
 		"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
 package cmux
 
 
 import (
 import (
-	"bytes"
 	"fmt"
 	"fmt"
 	"io"
 	"io"
 	"net"
 	"net"
 	"sync"
 	"sync"
+	"time"
 )
 )
 
 
 // Matcher matches a connection based on its content.
 // Matcher matches a connection based on its content.
 type Matcher func(io.Reader) bool
 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
 // ErrorHandler handles an error and returns whether
 // the mux should continue serving the listener.
 // the mux should continue serving the listener.
 type ErrorHandler func(error) bool
 type ErrorHandler func(error) bool
@@ -44,13 +61,17 @@ func (e errListenerClosed) Timeout() bool   { return false }
 // listener is closed.
 // listener is closed.
 var ErrListenerClosed = errListenerClosed("mux: listener closed")
 var ErrListenerClosed = errListenerClosed("mux: listener closed")
 
 
+// for readability of readTimeout
+var noTimeout time.Duration
+
 // New instantiates a new connection multiplexer.
 // New instantiates a new connection multiplexer.
 func New(l net.Listener) CMux {
 func New(l net.Listener) CMux {
 	return &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.
 	// The order used to call Match determines the priority of matchers.
 	Match(...Matcher) net.Listener
 	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
 	// Serve starts multiplexing the listener. Serve blocks and perhaps
 	// should be invoked concurrently within a go routine.
 	// should be invoked concurrently within a go routine.
 	Serve() error
 	Serve() error
 	// HandleError registers an error handler that handles listener errors.
 	// HandleError registers an error handler that handles listener errors.
 	HandleError(ErrorHandler)
 	HandleError(ErrorHandler)
+	// sets a timeout for the read of matchers
+	SetReadTimeout(time.Duration)
 }
 }
 
 
 type matchersListener struct {
 type matchersListener struct {
-	ss []Matcher
+	ss []MatchWriter
 	l  muxListener
 	l  muxListener
 }
 }
 
 
 type cMux struct {
 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 {
 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{
 	ml := muxListener{
 		Listener: m.root,
 		Listener: m.root,
 		connc:    make(chan net.Conn, m.bufLen),
 		connc:    make(chan net.Conn, m.bufLen),
@@ -90,6 +137,10 @@ func (m *cMux) Match(matchers ...Matcher) net.Listener {
 	return ml
 	return ml
 }
 }
 
 
+func (m *cMux) SetReadTimeout(t time.Duration) {
+	m.readTimeout = t
+}
+
 func (m *cMux) Serve() error {
 func (m *cMux) Serve() error {
 	var wg sync.WaitGroup
 	var wg sync.WaitGroup
 
 
@@ -124,10 +175,17 @@ func (m *cMux) serve(c net.Conn, donec <-chan struct{}, wg *sync.WaitGroup) {
 	defer wg.Done()
 	defer wg.Done()
 
 
 	muc := newMuxConn(c)
 	muc := newMuxConn(c)
+	if m.readTimeout > noTimeout {
+		_ = c.SetReadDeadline(time.Now().Add(m.readTimeout))
+	}
 	for _, sl := range m.sls {
 	for _, sl := range m.sls {
 		for _, s := range sl.ss {
 		for _, s := range sl.ss {
-			matched := s(muc.getSniffer())
+			matched := s(muc.Conn, muc.startSniffing())
 			if matched {
 			if matched {
+				muc.doneSniffing()
+				if m.readTimeout > noTimeout {
+					_ = c.SetReadDeadline(time.Time{})
+				}
 				select {
 				select {
 				case sl.l.connc <- muc:
 				case sl.l.connc <- muc:
 				case <-donec:
 				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.
 // MuxConn wraps a net.Conn and provides transparent sniffing of connection data.
 type MuxConn struct {
 type MuxConn struct {
 	net.Conn
 	net.Conn
-	buf     bytes.Buffer
-	sniffer bufferedReader
+	buf bufferedReader
 }
 }
 
 
 func newMuxConn(c net.Conn) *MuxConn {
 func newMuxConn(c net.Conn) *MuxConn {
 	return &MuxConn{
 	return &MuxConn{
 		Conn: c,
 		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 either err == EOF or err == nil.  The next Read should
 // return 0, EOF.
 // return 0, EOF.
 func (m *MuxConn) Read(p []byte) (int, error) {
 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
 package cmux
 
 
 import (
 import (
@@ -8,12 +22,20 @@ import (
 // patriciaTree is a simple patricia tree that handles []byte instead of string
 // patriciaTree is a simple patricia tree that handles []byte instead of string
 // and cannot be changed after instantiation.
 // and cannot be changed after instantiation.
 type patriciaTree struct {
 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{
 	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 {
 	for i, s := range strs {
 		b[i] = []byte(s)
 		b[i] = []byte(s)
 	}
 	}
-	return &patriciaTree{
-		root: newNode(b),
-	}
+	return newPatriciaTree(b...)
 }
 }
 
 
 func (t *patriciaTree) matchPrefix(r io.Reader) bool {
 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 {
 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 {
 type ptNode struct {
@@ -122,52 +146,34 @@ func splitPrefix(bss [][]byte) (prefix []byte, rest [][]byte) {
 	return prefix, rest
 	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
 			return false
 		}
 		}
 	}
 	}
 
 
-	if prefix && n.terminal {
+	if n.terminal && (prefix || len(n.prefix) == len(b)) {
 		return true
 		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"
 	"net/url"
 	"path/filepath"
 	"path/filepath"
 	"strings"
 	"strings"
+	"time"
 
 
 	"github.com/coreos/etcd/etcdserver"
 	"github.com/coreos/etcd/etcdserver"
 	"github.com/coreos/etcd/pkg/cors"
 	"github.com/coreos/etcd/pkg/cors"
@@ -140,6 +141,10 @@ type Config struct {
 	// auth
 	// auth
 
 
 	AuthToken string `json:"auth-token"`
 	AuthToken string `json:"auth-token"`
+
+	// Experimental flags
+
+	ExperimentalCorruptCheckTime time.Duration `json:"experimental-corrupt-check-time"`
 }
 }
 
 
 // configYAML holds the config suitable for yaml parsing
 // configYAML holds the config suitable for yaml parsing

+ 49 - 29
embed/etcd.go

@@ -16,6 +16,7 @@ package embed
 
 
 import (
 import (
 	"context"
 	"context"
+	"crypto/tls"
 	"fmt"
 	"fmt"
 	"io/ioutil"
 	"io/ioutil"
 	defaultLog "log"
 	defaultLog "log"
@@ -28,6 +29,7 @@ import (
 	"github.com/coreos/etcd/etcdserver"
 	"github.com/coreos/etcd/etcdserver"
 	"github.com/coreos/etcd/etcdserver/api/etcdhttp"
 	"github.com/coreos/etcd/etcdserver/api/etcdhttp"
 	"github.com/coreos/etcd/etcdserver/api/v2http"
 	"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/cors"
 	"github.com/coreos/etcd/pkg/debugutil"
 	"github.com/coreos/etcd/pkg/debugutil"
 	runtimeutil "github.com/coreos/etcd/pkg/runtime"
 	runtimeutil "github.com/coreos/etcd/pkg/runtime"
@@ -35,6 +37,9 @@ import (
 	"github.com/coreos/etcd/pkg/types"
 	"github.com/coreos/etcd/pkg/types"
 	"github.com/coreos/etcd/rafthttp"
 	"github.com/coreos/etcd/rafthttp"
 	"github.com/coreos/pkg/capnslog"
 	"github.com/coreos/pkg/capnslog"
+
+	"github.com/soheilhy/cmux"
+	"google.golang.org/grpc"
 )
 )
 
 
 var plog = capnslog.NewPackageLogger("github.com/coreos/etcd", "embed")
 var plog = capnslog.NewPackageLogger("github.com/coreos/etcd", "embed")
@@ -146,35 +151,46 @@ func StartEtcd(inCfg *Config) (e *Etcd, err error) {
 		StrictReconfigCheck:     cfg.StrictReconfigCheck,
 		StrictReconfigCheck:     cfg.StrictReconfigCheck,
 		ClientCertAuthEnabled:   cfg.ClientTLSInfo.ClientCertAuth,
 		ClientCertAuthEnabled:   cfg.ClientTLSInfo.ClientCertAuth,
 		AuthToken:               cfg.AuthToken,
 		AuthToken:               cfg.AuthToken,
+		CorruptCheckTime:        cfg.ExperimentalCorruptCheckTime,
 	}
 	}
 
 
 	if e.Server, err = etcdserver.NewServer(srvcfg); err != nil {
 	if e.Server, err = etcdserver.NewServer(srvcfg); err != nil {
 		return
 		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
 	// configure peer handlers after rafthttp.Transport started
 	ph := etcdhttp.NewPeerHandler(e.Server)
 	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 {
 	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{
 		srv := &http.Server{
-			Handler:     ph,
+			Handler:     grpcHandlerFunc(gs, ph),
 			ReadTimeout: 5 * time.Minute,
 			ReadTimeout: 5 * time.Minute,
 			ErrorLog:    defaultLog.New(ioutil.Discard, "", 0), // do not log user error
 			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 {
 		p.close = func(ctx context.Context) error {
 			// gracefully shutdown http.Server
 			// gracefully shutdown http.Server
 			// close open listeners, idle connections
 			// close open listeners, idle connections
 			// until context cancel or time-out
 			// until context cancel or time-out
+			e.stopGRPCServer(gs)
 			return srv.Shutdown(ctx)
 			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 {
 	if err = e.serve(); err != nil {
 		return
 		return
 	}
 	}
@@ -190,29 +206,9 @@ func (e *Etcd) Config() Config {
 func (e *Etcd) Close() {
 func (e *Etcd) Close() {
 	e.closeOnce.Do(func() { close(e.stopc) })
 	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 _, sctx := range e.sctxs {
 		for gs := range sctx.grpcServerC {
 		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 (e *Etcd) Err() <-chan error { return e.errc }
 
 
 func startPeerListeners(cfg *Config) (peers []*peerListener, err error) {
 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/debugutil"
 	"github.com/coreos/etcd/pkg/transport"
 	"github.com/coreos/etcd/pkg/transport"
 
 
-	"github.com/cockroachdb/cmux"
 	gw "github.com/grpc-ecosystem/grpc-gateway/runtime"
 	gw "github.com/grpc-ecosystem/grpc-gateway/runtime"
+	"github.com/soheilhy/cmux"
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
 	"golang.org/x/net/trace"
 	"golang.org/x/net/trace"
 	"google.golang.org/grpc"
 	"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
 // 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 {
 func grpcHandlerFunc(grpcServer *grpc.Server, otherHandler http.Handler) http.Handler {
 	if otherHandler == nil {
 	if otherHandler == nil {
 		return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
 		return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {

+ 3 - 0
etcdmain/config.go

@@ -216,6 +216,9 @@ func newConfig() *config {
 	// auth
 	// auth
 	fs.StringVar(&cfg.AuthToken, "auth-token", cfg.AuthToken, "Specify auth token specific options.")
 	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
 	// ignored
 	for _, f := range cfg.ignored {
 	for _, f := range cfg.ignored {
 		fs.Var(&flags.IgnoredFlag{Name: f}, f, "")
 		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/pkg/transport"
 	"github.com/coreos/etcd/proxy/grpcproxy"
 	"github.com/coreos/etcd/proxy/grpcproxy"
 
 
-	"github.com/cockroachdb/cmux"
 	grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus"
 	grpc_prometheus "github.com/grpc-ecosystem/go-grpc-prometheus"
+	"github.com/soheilhy/cmux"
 	"github.com/spf13/cobra"
 	"github.com/spf13/cobra"
 	"google.golang.org/grpc"
 	"google.golang.org/grpc"
 )
 )

+ 4 - 0
etcdmain/help.go

@@ -179,5 +179,9 @@ profiling flags:
 auth flags:
 auth flags:
 	--auth-token 'simple'
 	--auth-token 'simple'
 		Specify a v3 authentication token type and its options ('simple' or 'jwt').
 		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")
 	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")
 	ErrGRPCTimeoutDueToConnectionLost = grpc.Errorf(codes.Unavailable, "etcdserver: request timed out, possibly due to connection lost")
 	ErrGRPCUnhealthy                  = grpc.Errorf(codes.Unavailable, "etcdserver: unhealthy cluster")
 	ErrGRPCUnhealthy                  = grpc.Errorf(codes.Unavailable, "etcdserver: unhealthy cluster")
+	ErrGRPCCorrupt                    = grpc.Errorf(codes.DataLoss, "etcdserver: corrupt cluster")
 
 
 	errStringToError = map[string]error{
 	errStringToError = map[string]error{
 		grpc.ErrorDesc(ErrGRPCEmptyKey):      ErrGRPCEmptyKey,
 		grpc.ErrorDesc(ErrGRPCEmptyKey):      ErrGRPCEmptyKey,
@@ -114,6 +115,7 @@ var (
 		grpc.ErrorDesc(ErrGRPCTimeoutDueToLeaderFail):     ErrGRPCTimeoutDueToLeaderFail,
 		grpc.ErrorDesc(ErrGRPCTimeoutDueToLeaderFail):     ErrGRPCTimeoutDueToLeaderFail,
 		grpc.ErrorDesc(ErrGRPCTimeoutDueToConnectionLost): ErrGRPCTimeoutDueToConnectionLost,
 		grpc.ErrorDesc(ErrGRPCTimeoutDueToConnectionLost): ErrGRPCTimeoutDueToConnectionLost,
 		grpc.ErrorDesc(ErrGRPCUnhealthy):                  ErrGRPCUnhealthy,
 		grpc.ErrorDesc(ErrGRPCUnhealthy):                  ErrGRPCUnhealthy,
+		grpc.ErrorDesc(ErrGRPCCorrupt):                    ErrGRPCCorrupt,
 	}
 	}
 
 
 	// client-side error
 	// client-side error
@@ -162,6 +164,7 @@ var (
 	ErrTimeoutDueToLeaderFail     = Error(ErrGRPCTimeoutDueToLeaderFail)
 	ErrTimeoutDueToLeaderFail     = Error(ErrGRPCTimeoutDueToLeaderFail)
 	ErrTimeoutDueToConnectionLost = Error(ErrGRPCTimeoutDueToConnectionLost)
 	ErrTimeoutDueToConnectionLost = Error(ErrGRPCTimeoutDueToConnectionLost)
 	ErrUnhealthy                  = Error(ErrGRPCUnhealthy)
 	ErrUnhealthy                  = Error(ErrGRPCUnhealthy)
+	ErrCorrupt                    = Error(ErrGRPCCorrupt)
 )
 )
 
 
 // EtcdError defines gRPC server errors.
 // 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.ErrTimeoutDueToConnectionLost: rpctypes.ErrGRPCTimeoutDueToConnectionLost,
 	etcdserver.ErrUnhealthy:                  rpctypes.ErrGRPCUnhealthy,
 	etcdserver.ErrUnhealthy:                  rpctypes.ErrGRPCUnhealthy,
 	etcdserver.ErrKeyNotFound:                rpctypes.ErrGRPCKeyNotFound,
 	etcdserver.ErrKeyNotFound:                rpctypes.ErrGRPCKeyNotFound,
+	etcdserver.ErrCorrupt:                    rpctypes.ErrGRPCCorrupt,
 
 
 	lease.ErrLeaseNotFound: rpctypes.ErrGRPCLeaseNotFound,
 	lease.ErrLeaseNotFound: rpctypes.ErrGRPCLeaseNotFound,
 	lease.ErrLeaseExists:   rpctypes.ErrGRPCLeaseExist,
 	lease.ErrLeaseExists:   rpctypes.ErrGRPCLeaseExist,

+ 5 - 2
etcdserver/apply.go

@@ -577,9 +577,11 @@ func (a *applierV3backend) Alarm(ar *pb.AlarmRequest) (*pb.AlarmResponse, error)
 			break
 			break
 		}
 		}
 
 
+		plog.Warningf("alarm %v raised by peer %s", m.Alarm, types.ID(m.MemberID))
 		switch m.Alarm {
 		switch m.Alarm {
+		case pb.AlarmType_CORRUPT:
+			a.s.applyV3 = newApplierV3Corrupt(a)
 		case pb.AlarmType_NOSPACE:
 		case pb.AlarmType_NOSPACE:
-			plog.Warningf("alarm raised %+v", m)
 			a.s.applyV3 = newApplierV3Capped(a)
 			a.s.applyV3 = newApplierV3Capped(a)
 		default:
 		default:
 			plog.Errorf("unimplemented alarm activation (%+v)", m)
 			plog.Errorf("unimplemented alarm activation (%+v)", m)
@@ -596,7 +598,8 @@ func (a *applierV3backend) Alarm(ar *pb.AlarmRequest) (*pb.AlarmResponse, error)
 		}
 		}
 
 
 		switch m.Alarm {
 		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)
 			plog.Infof("alarm disarmed %+v", ar)
 			a.s.applyV3 = a.s.newApplierV3()
 			a.s.applyV3 = a.s.newApplierV3()
 		default:
 		default:

+ 2 - 0
etcdserver/config.go

@@ -66,6 +66,8 @@ type ServerConfig struct {
 	ClientCertAuthEnabled bool
 	ClientCertAuthEnabled bool
 
 
 	AuthToken string
 	AuthToken string
+
+	CorruptCheckTime time.Duration
 }
 }
 
 
 // VerifyBootstrap sanity-checks the initial config for bootstrap case
 // 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")
 	ErrTooManyRequests            = errors.New("etcdserver: too many requests")
 	ErrUnhealthy                  = errors.New("etcdserver: unhealthy cluster")
 	ErrUnhealthy                  = errors.New("etcdserver: unhealthy cluster")
 	ErrKeyNotFound                = errors.New("etcdserver: key not found")
 	ErrKeyNotFound                = errors.New("etcdserver: key not found")
+	ErrCorrupt                    = errors.New("etcdserver: corrupt cluster")
 )
 )
 
 
 type DiscoveryError struct {
 type DiscoveryError struct {

+ 231 - 227
etcdserver/etcdserverpb/rpc.pb.go

@@ -32,15 +32,18 @@ type AlarmType int32
 const (
 const (
 	AlarmType_NONE    AlarmType = 0
 	AlarmType_NONE    AlarmType = 0
 	AlarmType_NOSPACE AlarmType = 1
 	AlarmType_NOSPACE AlarmType = 1
+	AlarmType_CORRUPT AlarmType = 2
 )
 )
 
 
 var AlarmType_name = map[int32]string{
 var AlarmType_name = map[int32]string{
 	0: "NONE",
 	0: "NONE",
 	1: "NOSPACE",
 	1: "NOSPACE",
+	2: "CORRUPT",
 }
 }
 var AlarmType_value = map[string]int32{
 var AlarmType_value = map[string]int32{
 	"NONE":    0,
 	"NONE":    0,
 	"NOSPACE": 1,
 	"NOSPACE": 1,
+	"CORRUPT": 2,
 }
 }
 
 
 func (x AlarmType) String() string {
 func (x AlarmType) String() string {
@@ -18445,234 +18448,235 @@ var (
 func init() { proto.RegisterFile("rpc.proto", fileDescriptorRpc) }
 func init() { proto.RegisterFile("rpc.proto", fileDescriptorRpc) }
 
 
 var fileDescriptorRpc = []byte{
 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,
 	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,
 	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,
 	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 {
 enum AlarmType {
 	NONE = 0; // default, used to query if any alarm is active
 	NONE = 0; // default, used to query if any alarm is active
 	NOSPACE = 1; // space quota is exhausted
 	NOSPACE = 1; // space quota is exhausted
+	CORRUPT = 2; // kv store corruption detected
 }
 }
 
 
 message AlarmRequest {
 message AlarmRequest {

+ 4 - 0
etcdserver/server.go

@@ -524,6 +524,7 @@ func (s *EtcdServer) Start() {
 	s.goAttach(func() { monitorFileDescriptor(s.stopping) })
 	s.goAttach(func() { monitorFileDescriptor(s.stopping) })
 	s.goAttach(s.monitorVersions)
 	s.goAttach(s.monitorVersions)
 	s.goAttach(s.linearizableReadLoop)
 	s.goAttach(s.linearizableReadLoop)
+	s.goAttach(s.monitorKVHash)
 }
 }
 
 
 // start prepares and starts server in a new goroutine. It is no longer safe to
 // 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 {
 	if len(as.Get(pb.AlarmType_NOSPACE)) > 0 {
 		s.applyV3 = newApplierV3Capped(s.applyV3)
 		s.applyV3 = newApplierV3Capped(s.applyV3)
 	}
 	}
+	if len(as.Get(pb.AlarmType_CORRUPT)) > 0 {
+		s.applyV3 = newApplierV3Corrupt(s.applyV3)
+	}
 	return nil
 	return nil
 }
 }
 
 

+ 2 - 0
glide.lock

@@ -97,6 +97,8 @@ imports:
   - xfs
   - xfs
 - name: github.com/russross/blackfriday
 - name: github.com/russross/blackfriday
   version: 4048872b16cc0fc2c5fd9eacf0ed2c2fedaa0c8c
   version: 4048872b16cc0fc2c5fd9eacf0ed2c2fedaa0c8c
+- name: github.com/soheilhy/cmux
+  version: bb79a83465015a27a175925ebd155e660f55e9f1
 - name: github.com/spf13/cobra
 - name: github.com/spf13/cobra
   version: 1c44ec8d3f1552cac48999f9306da23c4d8a288b
   version: 1c44ec8d3f1552cac48999f9306da23c4d8a288b
 - name: github.com/spf13/pflag
 - name: github.com/spf13/pflag

+ 4 - 2
glide.yaml

@@ -1,4 +1,6 @@
 package: github.com/coreos/etcd
 package: github.com/coreos/etcd
+ignore:
+- google.golang.org/appengine
 import:
 import:
 - package: github.com/bgentry/speakeasy
 - package: github.com/bgentry/speakeasy
   version: v0.1.0
   version: v0.1.0
@@ -128,5 +130,5 @@ import:
   - unicode/norm
   - unicode/norm
 - package: github.com/russross/blackfriday
 - package: github.com/russross/blackfriday
   version: 4048872b16cc0fc2c5fd9eacf0ed2c2fedaa0c8c
   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"
 	"testing"
 	"time"
 	"time"
 
 
+	"github.com/soheilhy/cmux"
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
 	"google.golang.org/grpc"
 	"google.golang.org/grpc"
 
 
@@ -475,13 +476,14 @@ type member struct {
 	// ClientTLSInfo enables client TLS when set
 	// ClientTLSInfo enables client TLS when set
 	ClientTLSInfo *transport.TLSInfo
 	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 is a clientv3 that directly calls the etcdserver.
 	serverClient *clientv3.Client
 	serverClient *clientv3.Client
@@ -649,23 +651,80 @@ func (m *member) Launch() error {
 	m.s.SyncTicker = time.NewTicker(500 * time.Millisecond)
 	m.s.SyncTicker = time.NewTicker(500 * time.Millisecond)
 	m.s.Start()
 	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)}
 	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 {
 	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
 				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 {
 	for _, ln := range m.ClientListeners {
 		hs := &httptest.Server{
 		hs := &httptest.Server{
@@ -681,23 +740,12 @@ func (m *member) Launch() error {
 			}
 			}
 			hs.StartTLS()
 			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)
 	plog.Printf("launched %s (%s)", m.Name, m.grpcAddr)
@@ -745,13 +793,16 @@ func (m *member) Close() {
 		m.serverClient = nil
 		m.serverClient = nil
 	}
 	}
 	if m.grpcServer != nil {
 	if m.grpcServer != nil {
+		m.grpcServer.Stop()
 		m.grpcServer.GracefulStop()
 		m.grpcServer.GracefulStop()
 		m.grpcServer = nil
 		m.grpcServer = nil
+		m.grpcServerPeer.Stop()
+		m.grpcServerPeer.GracefulStop()
+		m.grpcServerPeer = nil
 	}
 	}
 	m.s.HardStop()
 	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) {
 func (m *member) Stop(t *testing.T) {
 	plog.Printf("stopping %s (%s)", m.Name, m.grpcAddr)
 	plog.Printf("stopping %s (%s)", m.Name, m.grpcAddr)
 	m.Close()
 	m.Close()
-	m.hss = nil
+	m.serverClosers = nil
 	plog.Printf("stopped %s (%s)", m.Name, m.grpcAddr)
 	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) {
 func TestV3RangeRequest(t *testing.T) {
 	defer testutil.AfterTest(t)
 	defer testutil.AfterTest(t)
 	tests := []struct {
 	tests := []struct {