Browse Source

Merge pull request #8153 from gyuho/leadership-transfer

*: expose Leadership Transfer API to clients
Gyu-Ho Lee 8 years ago
parent
commit
a57405a958

+ 17 - 0
Documentation/dev-guide/api_reference_v3.md

@@ -70,6 +70,7 @@ This is a generated documentation. Please read the proto files for more.
 | Defragment | DefragmentRequest | DefragmentResponse | Defragment defragments a member's backend database to recover storage space. |
 | Hash | HashRequest | HashResponse | Hash returns the hash of the local KV state for consistency checking purpose. This is designed for testing; do not use this in production when there are ongoing transactions. |
 | Snapshot | SnapshotRequest | SnapshotResponse | Snapshot sends a snapshot of the entire backend from a member over a stream to a client. |
+| MoveLeader | MoveLeaderRequest | MoveLeaderResponse | MoveLeader requests current leader node to transfer its leadership to transferee. |
 
 
 
@@ -608,6 +609,22 @@ Empty field.
 
 
 
+##### message `MoveLeaderRequest` (etcdserver/etcdserverpb/rpc.proto)
+
+| Field | Description | Type |
+| ----- | ----------- | ---- |
+| targetID | targetID is the node ID for the new leader. | uint64 |
+
+
+
+##### message `MoveLeaderResponse` (etcdserver/etcdserverpb/rpc.proto)
+
+| Field | Description | Type |
+| ----- | ----------- | ---- |
+| header |  | ResponseHeader |
+
+
+
 ##### message `PutRequest` (etcdserver/etcdserverpb/rpc.proto)
 
 | Field | Description | Type |

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

@@ -934,6 +934,33 @@
         }
       }
     },
+    "/v3alpha/maintenance/transfer-leadership": {
+      "post": {
+        "tags": [
+          "Maintenance"
+        ],
+        "summary": "MoveLeader requests current leader node to transfer its leadership to transferee.",
+        "operationId": "MoveLeader",
+        "parameters": [
+          {
+            "name": "body",
+            "in": "body",
+            "required": true,
+            "schema": {
+              "$ref": "#/definitions/etcdserverpbMoveLeaderRequest"
+            }
+          }
+        ],
+        "responses": {
+          "200": {
+            "description": "(empty)",
+            "schema": {
+              "$ref": "#/definitions/etcdserverpbMoveLeaderResponse"
+            }
+          }
+        }
+      }
+    },
     "/v3alpha/watch": {
       "post": {
         "tags": [
@@ -1803,6 +1830,24 @@
         }
       }
     },
+    "etcdserverpbMoveLeaderRequest": {
+      "type": "object",
+      "properties": {
+        "targetID": {
+          "description": "targetID is the node ID for the new leader.",
+          "type": "string",
+          "format": "uint64"
+        }
+      }
+    },
+    "etcdserverpbMoveLeaderResponse": {
+      "type": "object",
+      "properties": {
+        "header": {
+          "$ref": "#/definitions/etcdserverpbResponseHeader"
+        }
+      }
+    },
     "etcdserverpbPutRequest": {
       "type": "object",
       "properties": {
@@ -2177,7 +2222,7 @@
           "format": "boolean"
         },
         "compact_revision": {
-          "description": "compact_revision is set to the minimum index if a watcher tries to watch\nat a compacted index.\n\nThis happens when creating a watcher at a compacted revision or the watcher cannot\ncatch up with the progress of the key-value store. \n\nThe client should treat the watcher as canceled and should not try to create any\nwatcher with the same start_revision again.",
+          "description": "compact_revision is set to the minimum index if a watcher tries to watch\nat a compacted index.\n\nThis happens when creating a watcher at a compacted revision or the watcher cannot\ncatch up with the progress of the key-value store.\n\nThe client should treat the watcher as canceled and should not try to create any\nwatcher with the same start_revision again.",
           "type": "string",
           "format": "int64"
         },

+ 53 - 0
clientv3/integration/maintenance_test.go

@@ -0,0 +1,53 @@
+// 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 (
+	"context"
+	"testing"
+
+	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
+	"github.com/coreos/etcd/integration"
+	"github.com/coreos/etcd/pkg/testutil"
+)
+
+func TestMaintenanceMoveLeader(t *testing.T) {
+	defer testutil.AfterTest(t)
+
+	clus := integration.NewClusterV3(t, &integration.ClusterConfig{Size: 3})
+	defer clus.Terminate(t)
+
+	oldLeadIdx := clus.WaitLeader(t)
+	targetIdx := (oldLeadIdx + 1) % 3
+	target := uint64(clus.Members[targetIdx].ID())
+
+	cli := clus.Client(targetIdx)
+	_, err := cli.MoveLeader(context.Background(), target)
+	if err != rpctypes.ErrNotLeader {
+		t.Fatalf("error expected %v, got %v", rpctypes.ErrNotLeader, err)
+	}
+
+	cli = clus.Client(oldLeadIdx)
+	_, err = cli.MoveLeader(context.Background(), target)
+	if err != nil {
+		t.Fatal(err)
+	}
+
+	leadIdx := clus.WaitLeader(t)
+	lead := uint64(clus.Members[leadIdx].ID())
+	if target != lead {
+		t.Fatalf("new leader expected %d, got %d", target, lead)
+	}
+}

+ 10 - 0
clientv3/maintenance.go

@@ -28,6 +28,7 @@ type (
 	AlarmResponse      pb.AlarmResponse
 	AlarmMember        pb.AlarmMember
 	StatusResponse     pb.StatusResponse
+	MoveLeaderResponse pb.MoveLeaderResponse
 )
 
 type Maintenance interface {
@@ -51,6 +52,10 @@ type Maintenance interface {
 
 	// Snapshot provides a reader for a snapshot of a backend.
 	Snapshot(ctx context.Context) (io.ReadCloser, error)
+
+	// MoveLeader requests current leader to transfer its leadership to the transferee.
+	// Request must be made to the leader.
+	MoveLeader(ctx context.Context, transfereeID uint64) (*MoveLeaderResponse, error)
 }
 
 type maintenance struct {
@@ -180,3 +185,8 @@ func (m *maintenance) Snapshot(ctx context.Context) (io.ReadCloser, error) {
 	}()
 	return pr, nil
 }
+
+func (m *maintenance) MoveLeader(ctx context.Context, transfereeID uint64) (*MoveLeaderResponse, error) {
+	resp, err := m.remote.MoveLeader(ctx, &pb.MoveLeaderRequest{TargetID: transfereeID}, grpc.FailFast(false))
+	return (*MoveLeaderResponse)(resp), toErr(ctx, err)
+}

+ 92 - 0
e2e/ctl_v3_move_leader_test.go

@@ -0,0 +1,92 @@
+// 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 e2e
+
+import (
+	"context"
+	"fmt"
+	"os"
+	"testing"
+	"time"
+
+	"github.com/coreos/etcd/clientv3"
+	"github.com/coreos/etcd/pkg/testutil"
+	"github.com/coreos/etcd/pkg/types"
+)
+
+func TestCtlV3MoveLeader(t *testing.T) {
+	defer testutil.AfterTest(t)
+
+	epc := setupEtcdctlTest(t, &configNoTLS, true)
+	defer func() {
+		if errC := epc.Close(); errC != nil {
+			t.Fatalf("error closing etcd processes (%v)", errC)
+		}
+	}()
+
+	var leadIdx int
+	var leaderID uint64
+	var transferee uint64
+	for i, ep := range epc.grpcEndpoints() {
+		cli, err := clientv3.New(clientv3.Config{
+			Endpoints:   []string{ep},
+			DialTimeout: 3 * time.Second,
+		})
+		if err != nil {
+			t.Fatal(err)
+		}
+		resp, err := cli.Status(context.Background(), ep)
+		if err != nil {
+			t.Fatal(err)
+		}
+		cli.Close()
+
+		if resp.Header.GetMemberId() == resp.Leader {
+			leadIdx = i
+			leaderID = resp.Leader
+		} else {
+			transferee = resp.Header.GetMemberId()
+		}
+	}
+
+	os.Setenv("ETCDCTL_API", "3")
+	defer os.Unsetenv("ETCDCTL_API")
+	cx := ctlCtx{
+		t:           t,
+		cfg:         configNoTLS,
+		dialTimeout: 7 * time.Second,
+		epc:         epc,
+	}
+
+	tests := []struct {
+		prefixes []string
+		expect   string
+	}{
+		{ // request to non-leader
+			cx.prefixArgs([]string{cx.epc.grpcEndpoints()[(leadIdx+1)%3]}),
+			"no leader endpoint given at ",
+		},
+		{ // request to leader
+			cx.prefixArgs([]string{cx.epc.grpcEndpoints()[leadIdx]}),
+			fmt.Sprintf("Leadership transferred from %s to %s", types.ID(leaderID), types.ID(transferee)),
+		},
+	}
+	for i, tc := range tests {
+		cmdArgs := append(tc.prefixes, "move-leader", types.ID(transferee).String())
+		if err := spawnWithExpect(cmdArgs, tc.expect); err != nil {
+			t.Fatalf("#%d: %v", i, err)
+		}
+	}
+}

+ 23 - 0
etcdctl/README.md

@@ -805,6 +805,29 @@ Prints a line of JSON encoding the database hash, revision, total keys, and size
 +----------+----------+------------+------------+
 ```
 
+### MOVE-LEADER \<hexadecimal-transferee-id\>
+
+MOVE-LEADER transfers leadership from the leader to another member in the cluster.
+
+#### Example
+
+```bash
+# to choose transferee
+transferee_id=$(./etcdctl \
+  --endpoints localhost:12379,localhost:22379,localhost:32379 \
+  endpoint status | grep -m 1 "false" | awk -F', ' '{print $2}')
+echo ${transferee_id}
+# c89feb932daef420
+
+# endpoints should include leader node
+./etcdctl --endpoints ${transferee_ep} move-leader ${transferee_id}
+# Error:  no leader endpoint given at [localhost:22379 localhost:32379]
+
+# request to leader with target node ID
+./etcdctl --endpoints ${leader_ep} move-leader ${transferee_id}
+# Leadership transferred from 45ddc0e800e20b93 to c89feb932daef420
+```
+
 ## Concurrency commands
 
 ### LOCK \<lockname\> [command arg1 arg2 ...]

+ 87 - 0
etcdctl/ctlv3/command/move_leader_command.go

@@ -0,0 +1,87 @@
+// 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 command
+
+import (
+	"fmt"
+	"strconv"
+	"time"
+
+	"github.com/coreos/etcd/clientv3"
+	"github.com/spf13/cobra"
+)
+
+// NewMoveLeaderCommand returns the cobra command for "move-leader".
+func NewMoveLeaderCommand() *cobra.Command {
+	cmd := &cobra.Command{
+		Use:   "move-leader <transferee-member-id>",
+		Short: "Transfers leadership to another etcd cluster member.",
+		Run:   transferLeadershipCommandFunc,
+	}
+	return cmd
+}
+
+// transferLeadershipCommandFunc executes the "compaction" command.
+func transferLeadershipCommandFunc(cmd *cobra.Command, args []string) {
+	if len(args) != 1 {
+		ExitWithError(ExitBadArgs, fmt.Errorf("move-leader command needs 1 argument"))
+	}
+	target, err := strconv.ParseUint(args[0], 16, 64)
+	if err != nil {
+		ExitWithError(ExitBadArgs, err)
+	}
+
+	c := mustClientFromCmd(cmd)
+	eps := c.Endpoints()
+	c.Close()
+
+	ctx, cancel := commandCtx(cmd)
+
+	// find current leader
+	var leaderCli *clientv3.Client
+	var leaderID uint64
+	for _, ep := range eps {
+		cli, err := clientv3.New(clientv3.Config{
+			Endpoints:   []string{ep},
+			DialTimeout: 3 * time.Second,
+		})
+		if err != nil {
+			ExitWithError(ExitError, err)
+		}
+		resp, err := cli.Status(ctx, ep)
+		if err != nil {
+			ExitWithError(ExitError, err)
+		}
+
+		if resp.Header.GetMemberId() == resp.Leader {
+			leaderCli = cli
+			leaderID = resp.Leader
+			break
+		}
+		cli.Close()
+	}
+	if leaderCli == nil {
+		ExitWithError(ExitBadArgs, fmt.Errorf("no leader endpoint given at %v", eps))
+	}
+
+	var resp *clientv3.MoveLeaderResponse
+	resp, err = leaderCli.MoveLeader(ctx, target)
+	cancel()
+	if err != nil {
+		ExitWithError(ExitError, err)
+	}
+
+	display.MoveLeader(leaderID, target, *resp)
+}

+ 6 - 1
etcdctl/ctlv3/command/printer.go

@@ -43,6 +43,7 @@ type printer interface {
 	MemberList(v3.MemberListResponse)
 
 	EndpointStatus([]epStatus)
+	MoveLeader(leader, target uint64, r v3.MoveLeaderResponse)
 
 	Alarm(v3.AlarmResponse)
 	DBStatus(dbstatus)
@@ -104,7 +105,9 @@ func (p *printerRPC) MemberUpdate(id uint64, r v3.MemberUpdateResponse) {
 }
 func (p *printerRPC) MemberList(r v3.MemberListResponse) { p.p((*pb.MemberListResponse)(&r)) }
 func (p *printerRPC) Alarm(r v3.AlarmResponse)           { p.p((*pb.AlarmResponse)(&r)) }
-
+func (p *printerRPC) MoveLeader(leader, target uint64, r v3.MoveLeaderResponse) {
+	p.p((*pb.MoveLeaderResponse)(&r))
+}
 func (p *printerRPC) RoleAdd(_ string, r v3.AuthRoleAddResponse) { p.p((*pb.AuthRoleAddResponse)(&r)) }
 func (p *printerRPC) RoleGet(_ string, r v3.AuthRoleGetResponse) { p.p((*pb.AuthRoleGetResponse)(&r)) }
 func (p *printerRPC) RoleDelete(_ string, r v3.AuthRoleDeleteResponse) {
@@ -145,6 +148,8 @@ func newPrinterUnsupported(n string) printer {
 func (p *printerUnsupported) EndpointStatus([]epStatus) { p.p(nil) }
 func (p *printerUnsupported) DBStatus(dbstatus)         { p.p(nil) }
 
+func (p *printerUnsupported) MoveLeader(leader, target uint64, r v3.MoveLeaderResponse) { p.p(nil) }
+
 func makeMemberListTable(r v3.MemberListResponse) (hdr []string, rows [][]string) {
 	hdr = []string{"ID", "Status", "Name", "Peer Addrs", "Client Addrs"}
 	for _, m := range r.Members {

+ 5 - 0
etcdctl/ctlv3/command/printer_simple.go

@@ -20,6 +20,7 @@ import (
 
 	v3 "github.com/coreos/etcd/clientv3"
 	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
+	"github.com/coreos/etcd/pkg/types"
 )
 
 type simplePrinter struct {
@@ -142,6 +143,10 @@ func (s *simplePrinter) DBStatus(ds dbstatus) {
 	}
 }
 
+func (s *simplePrinter) MoveLeader(leader, target uint64, r v3.MoveLeaderResponse) {
+	fmt.Printf("Leadership transferred from %s to %s\n", types.ID(leader), types.ID(target))
+}
+
 func (s *simplePrinter) RoleAdd(role string, r v3.AuthRoleAddResponse) {
 	fmt.Printf("Role %s created\n", role)
 }

+ 1 - 0
etcdctl/ctlv3/ctl.go

@@ -69,6 +69,7 @@ func init() {
 		command.NewAlarmCommand(),
 		command.NewDefragCommand(),
 		command.NewEndpointCommand(),
+		command.NewMoveLeaderCommand(),
 		command.NewWatchCommand(),
 		command.NewVersionCommand(),
 		command.NewLeaseCommand(),

+ 23 - 1
etcdserver/api/v3rpc/maintenance.go

@@ -20,6 +20,7 @@ import (
 
 	"github.com/coreos/etcd/auth"
 	"github.com/coreos/etcd/etcdserver"
+	"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"
@@ -40,9 +41,14 @@ type Alarmer interface {
 	Alarm(ctx context.Context, ar *pb.AlarmRequest) (*pb.AlarmResponse, error)
 }
 
+type LeaderTransferrer interface {
+	MoveLeader(ctx context.Context, lead, target uint64) error
+}
+
 type RaftStatusGetter interface {
 	Index() uint64
 	Term() uint64
+	ID() types.ID
 	Leader() types.ID
 }
 
@@ -56,11 +62,12 @@ type maintenanceServer struct {
 	kg  KVGetter
 	bg  BackendGetter
 	a   Alarmer
+	lt  LeaderTransferrer
 	hdr header
 }
 
 func NewMaintenanceServer(s *etcdserver.EtcdServer) pb.MaintenanceServer {
-	srv := &maintenanceServer{rg: s, kg: s, bg: s, a: s, hdr: newHeader(s)}
+	srv := &maintenanceServer{rg: s, kg: s, bg: s, a: s, lt: s, hdr: newHeader(s)}
 	return &authMaintenanceServer{srv, s}
 }
 
@@ -147,6 +154,17 @@ func (ms *maintenanceServer) Status(ctx context.Context, ar *pb.StatusRequest) (
 	return resp, nil
 }
 
+func (ms *maintenanceServer) MoveLeader(ctx context.Context, tr *pb.MoveLeaderRequest) (*pb.MoveLeaderResponse, error) {
+	if ms.rg.ID() != ms.rg.Leader() {
+		return nil, rpctypes.ErrGRPCNotLeader
+	}
+
+	if err := ms.lt.MoveLeader(ctx, uint64(ms.rg.Leader()), tr.TargetID); err != nil {
+		return nil, togRPCError(err)
+	}
+	return &pb.MoveLeaderResponse{}, nil
+}
+
 type authMaintenanceServer struct {
 	*maintenanceServer
 	ag AuthGetter
@@ -188,3 +206,7 @@ func (ams *authMaintenanceServer) Hash(ctx context.Context, r *pb.HashRequest) (
 func (ams *authMaintenanceServer) Status(ctx context.Context, ar *pb.StatusRequest) (*pb.StatusResponse, error) {
 	return ams.maintenanceServer.Status(ctx, ar)
 }
+
+func (ams *authMaintenanceServer) MoveLeader(ctx context.Context, tr *pb.MoveLeaderRequest) (*pb.MoveLeaderResponse, error) {
+	return ams.maintenanceServer.MoveLeader(ctx, tr)
+}

+ 3 - 0
etcdserver/api/v3rpc/rpctypes/error.go

@@ -59,6 +59,7 @@ var (
 	ErrGRPCInvalidAuthMgmt      = grpc.Errorf(codes.InvalidArgument, "etcdserver: invalid auth management")
 
 	ErrGRPCNoLeader                   = grpc.Errorf(codes.Unavailable, "etcdserver: no leader")
+	ErrGRPCNotLeader                  = grpc.Errorf(codes.Unavailable, "etcdserver: not leader")
 	ErrGRPCNotCapable                 = grpc.Errorf(codes.Unavailable, "etcdserver: not capable")
 	ErrGRPCStopped                    = grpc.Errorf(codes.Unavailable, "etcdserver: server stopped")
 	ErrGRPCTimeout                    = grpc.Errorf(codes.Unavailable, "etcdserver: request timed out")
@@ -106,6 +107,7 @@ var (
 		grpc.ErrorDesc(ErrGRPCInvalidAuthMgmt):      ErrGRPCInvalidAuthMgmt,
 
 		grpc.ErrorDesc(ErrGRPCNoLeader):                   ErrGRPCNoLeader,
+		grpc.ErrorDesc(ErrGRPCNotLeader):                  ErrGRPCNotLeader,
 		grpc.ErrorDesc(ErrGRPCNotCapable):                 ErrGRPCNotCapable,
 		grpc.ErrorDesc(ErrGRPCStopped):                    ErrGRPCStopped,
 		grpc.ErrorDesc(ErrGRPCTimeout):                    ErrGRPCTimeout,
@@ -153,6 +155,7 @@ var (
 	ErrInvalidAuthMgmt      = Error(ErrGRPCInvalidAuthMgmt)
 
 	ErrNoLeader                   = Error(ErrGRPCNoLeader)
+	ErrNotLeader                  = Error(ErrGRPCNotLeader)
 	ErrNotCapable                 = Error(ErrGRPCNotCapable)
 	ErrStopped                    = Error(ErrGRPCStopped)
 	ErrTimeout                    = Error(ErrGRPCTimeout)

+ 1 - 0
etcdserver/api/v3rpc/util.go

@@ -39,6 +39,7 @@ var toGRPCErrorMap = map[error]error{
 	etcdserver.ErrTooManyRequests: rpctypes.ErrTooManyRequests,
 
 	etcdserver.ErrNoLeader:                   rpctypes.ErrGRPCNoLeader,
+	etcdserver.ErrNotLeader:                  rpctypes.ErrGRPCNotLeader,
 	etcdserver.ErrStopped:                    rpctypes.ErrGRPCStopped,
 	etcdserver.ErrTimeout:                    rpctypes.ErrGRPCTimeout,
 	etcdserver.ErrTimeoutDueToLeaderFail:     rpctypes.ErrGRPCTimeoutDueToLeaderFail,

+ 1 - 0
etcdserver/errors.go

@@ -29,6 +29,7 @@ var (
 	ErrTimeoutLeaderTransfer      = errors.New("etcdserver: request timed out, leader transfer took too long")
 	ErrNotEnoughStartedMembers    = errors.New("etcdserver: re-configuration failed due to not enough started members")
 	ErrNoLeader                   = errors.New("etcdserver: no leader")
+	ErrNotLeader                  = errors.New("etcdserver: not leader")
 	ErrRequestTooLarge            = errors.New("etcdserver: request is too large")
 	ErrNoSpace                    = errors.New("etcdserver: no space")
 	ErrTooManyRequests            = errors.New("etcdserver: too many requests")

+ 2 - 0
etcdserver/etcdserverpb/etcdserver.pb.go

@@ -58,6 +58,8 @@
 		MemberListResponse
 		DefragmentRequest
 		DefragmentResponse
+		MoveLeaderRequest
+		MoveLeaderResponse
 		AlarmRequest
 		AlarmMember
 		AlarmResponse

+ 45 - 0
etcdserver/etcdserverpb/gw/rpc.pb.gw.go

@@ -361,6 +361,19 @@ func request_Maintenance_Snapshot_0(ctx context.Context, marshaler runtime.Marsh
 
 }
 
+func request_Maintenance_MoveLeader_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.MaintenanceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
+	var protoReq etcdserverpb.MoveLeaderRequest
+	var metadata runtime.ServerMetadata
+
+	if err := marshaler.NewDecoder(req.Body).Decode(&protoReq); err != nil {
+		return nil, metadata, grpc.Errorf(codes.InvalidArgument, "%v", err)
+	}
+
+	msg, err := client.MoveLeader(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
+	return msg, metadata, err
+
+}
+
 func request_Auth_AuthEnable_0(ctx context.Context, marshaler runtime.Marshaler, client etcdserverpb.AuthClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
 	var protoReq etcdserverpb.AuthEnableRequest
 	var metadata runtime.ServerMetadata
@@ -1335,6 +1348,34 @@ func RegisterMaintenanceHandler(ctx context.Context, mux *runtime.ServeMux, conn
 
 	})
 
+	mux.Handle("POST", pattern_Maintenance_MoveLeader_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
+		ctx, cancel := context.WithCancel(ctx)
+		defer cancel()
+		if cn, ok := w.(http.CloseNotifier); ok {
+			go func(done <-chan struct{}, closed <-chan bool) {
+				select {
+				case <-done:
+				case <-closed:
+					cancel()
+				}
+			}(ctx.Done(), cn.CloseNotify())
+		}
+		inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req)
+		rctx, err := runtime.AnnotateContext(ctx, req)
+		if err != nil {
+			runtime.HTTPError(ctx, outboundMarshaler, w, req, err)
+		}
+		resp, md, err := request_Maintenance_MoveLeader_0(rctx, inboundMarshaler, client, req, pathParams)
+		ctx = runtime.NewServerMetadataContext(ctx, md)
+		if err != nil {
+			runtime.HTTPError(ctx, outboundMarshaler, w, req, err)
+			return
+		}
+
+		forward_Maintenance_MoveLeader_0(ctx, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
+
+	})
+
 	return nil
 }
 
@@ -1348,6 +1389,8 @@ var (
 	pattern_Maintenance_Hash_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3alpha", "maintenance", "hash"}, ""))
 
 	pattern_Maintenance_Snapshot_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3alpha", "maintenance", "snapshot"}, ""))
+
+	pattern_Maintenance_MoveLeader_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"v3alpha", "maintenance", "transfer-leadership"}, ""))
 )
 
 var (
@@ -1360,6 +1403,8 @@ var (
 	forward_Maintenance_Hash_0 = runtime.ForwardResponseMessage
 
 	forward_Maintenance_Snapshot_0 = runtime.ForwardResponseStream
+
+	forward_Maintenance_MoveLeader_0 = runtime.ForwardResponseMessage
 )
 
 // RegisterAuthHandlerFromEndpoint is same as RegisterAuthHandler but

+ 591 - 295
etcdserver/etcdserverpb/rpc.pb.go

@@ -204,7 +204,7 @@ func (x AlarmRequest_AlarmAction) String() string {
 	return proto.EnumName(AlarmRequest_AlarmAction_name, int32(x))
 }
 func (AlarmRequest_AlarmAction) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptorRpc, []int{41, 0}
+	return fileDescriptorRpc, []int{43, 0}
 }
 
 type ResponseHeader struct {
@@ -2157,6 +2157,39 @@ func (m *DefragmentResponse) GetHeader() *ResponseHeader {
 	return nil
 }
 
+type MoveLeaderRequest struct {
+	// targetID is the node ID for the new leader.
+	TargetID uint64 `protobuf:"varint,1,opt,name=targetID,proto3" json:"targetID,omitempty"`
+}
+
+func (m *MoveLeaderRequest) Reset()                    { *m = MoveLeaderRequest{} }
+func (m *MoveLeaderRequest) String() string            { return proto.CompactTextString(m) }
+func (*MoveLeaderRequest) ProtoMessage()               {}
+func (*MoveLeaderRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{41} }
+
+func (m *MoveLeaderRequest) GetTargetID() uint64 {
+	if m != nil {
+		return m.TargetID
+	}
+	return 0
+}
+
+type MoveLeaderResponse struct {
+	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
+}
+
+func (m *MoveLeaderResponse) Reset()                    { *m = MoveLeaderResponse{} }
+func (m *MoveLeaderResponse) String() string            { return proto.CompactTextString(m) }
+func (*MoveLeaderResponse) ProtoMessage()               {}
+func (*MoveLeaderResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{42} }
+
+func (m *MoveLeaderResponse) GetHeader() *ResponseHeader {
+	if m != nil {
+		return m.Header
+	}
+	return nil
+}
+
 type AlarmRequest struct {
 	// action is the kind of alarm request to issue. The action
 	// may GET alarm statuses, ACTIVATE an alarm, or DEACTIVATE a
@@ -2172,7 +2205,7 @@ type AlarmRequest struct {
 func (m *AlarmRequest) Reset()                    { *m = AlarmRequest{} }
 func (m *AlarmRequest) String() string            { return proto.CompactTextString(m) }
 func (*AlarmRequest) ProtoMessage()               {}
-func (*AlarmRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{41} }
+func (*AlarmRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{43} }
 
 func (m *AlarmRequest) GetAction() AlarmRequest_AlarmAction {
 	if m != nil {
@@ -2205,7 +2238,7 @@ type AlarmMember struct {
 func (m *AlarmMember) Reset()                    { *m = AlarmMember{} }
 func (m *AlarmMember) String() string            { return proto.CompactTextString(m) }
 func (*AlarmMember) ProtoMessage()               {}
-func (*AlarmMember) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{42} }
+func (*AlarmMember) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{44} }
 
 func (m *AlarmMember) GetMemberID() uint64 {
 	if m != nil {
@@ -2230,7 +2263,7 @@ type AlarmResponse struct {
 func (m *AlarmResponse) Reset()                    { *m = AlarmResponse{} }
 func (m *AlarmResponse) String() string            { return proto.CompactTextString(m) }
 func (*AlarmResponse) ProtoMessage()               {}
-func (*AlarmResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{43} }
+func (*AlarmResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{45} }
 
 func (m *AlarmResponse) GetHeader() *ResponseHeader {
 	if m != nil {
@@ -2252,7 +2285,7 @@ type StatusRequest struct {
 func (m *StatusRequest) Reset()                    { *m = StatusRequest{} }
 func (m *StatusRequest) String() string            { return proto.CompactTextString(m) }
 func (*StatusRequest) ProtoMessage()               {}
-func (*StatusRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{44} }
+func (*StatusRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{46} }
 
 type StatusResponse struct {
 	Header *ResponseHeader `protobuf:"bytes,1,opt,name=header" json:"header,omitempty"`
@@ -2271,7 +2304,7 @@ type StatusResponse struct {
 func (m *StatusResponse) Reset()                    { *m = StatusResponse{} }
 func (m *StatusResponse) String() string            { return proto.CompactTextString(m) }
 func (*StatusResponse) ProtoMessage()               {}
-func (*StatusResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{45} }
+func (*StatusResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{47} }
 
 func (m *StatusResponse) GetHeader() *ResponseHeader {
 	if m != nil {
@@ -2321,7 +2354,7 @@ type AuthEnableRequest struct {
 func (m *AuthEnableRequest) Reset()                    { *m = AuthEnableRequest{} }
 func (m *AuthEnableRequest) String() string            { return proto.CompactTextString(m) }
 func (*AuthEnableRequest) ProtoMessage()               {}
-func (*AuthEnableRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{46} }
+func (*AuthEnableRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{48} }
 
 type AuthDisableRequest struct {
 }
@@ -2329,7 +2362,7 @@ type AuthDisableRequest struct {
 func (m *AuthDisableRequest) Reset()                    { *m = AuthDisableRequest{} }
 func (m *AuthDisableRequest) String() string            { return proto.CompactTextString(m) }
 func (*AuthDisableRequest) ProtoMessage()               {}
-func (*AuthDisableRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{47} }
+func (*AuthDisableRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{49} }
 
 type AuthenticateRequest struct {
 	Name     string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
@@ -2339,7 +2372,7 @@ type AuthenticateRequest struct {
 func (m *AuthenticateRequest) Reset()                    { *m = AuthenticateRequest{} }
 func (m *AuthenticateRequest) String() string            { return proto.CompactTextString(m) }
 func (*AuthenticateRequest) ProtoMessage()               {}
-func (*AuthenticateRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{48} }
+func (*AuthenticateRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{50} }
 
 func (m *AuthenticateRequest) GetName() string {
 	if m != nil {
@@ -2363,7 +2396,7 @@ type AuthUserAddRequest struct {
 func (m *AuthUserAddRequest) Reset()                    { *m = AuthUserAddRequest{} }
 func (m *AuthUserAddRequest) String() string            { return proto.CompactTextString(m) }
 func (*AuthUserAddRequest) ProtoMessage()               {}
-func (*AuthUserAddRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{49} }
+func (*AuthUserAddRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{51} }
 
 func (m *AuthUserAddRequest) GetName() string {
 	if m != nil {
@@ -2386,7 +2419,7 @@ type AuthUserGetRequest struct {
 func (m *AuthUserGetRequest) Reset()                    { *m = AuthUserGetRequest{} }
 func (m *AuthUserGetRequest) String() string            { return proto.CompactTextString(m) }
 func (*AuthUserGetRequest) ProtoMessage()               {}
-func (*AuthUserGetRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{50} }
+func (*AuthUserGetRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{52} }
 
 func (m *AuthUserGetRequest) GetName() string {
 	if m != nil {
@@ -2403,7 +2436,7 @@ type AuthUserDeleteRequest struct {
 func (m *AuthUserDeleteRequest) Reset()                    { *m = AuthUserDeleteRequest{} }
 func (m *AuthUserDeleteRequest) String() string            { return proto.CompactTextString(m) }
 func (*AuthUserDeleteRequest) ProtoMessage()               {}
-func (*AuthUserDeleteRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{51} }
+func (*AuthUserDeleteRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{53} }
 
 func (m *AuthUserDeleteRequest) GetName() string {
 	if m != nil {
@@ -2423,7 +2456,7 @@ func (m *AuthUserChangePasswordRequest) Reset()         { *m = AuthUserChangePas
 func (m *AuthUserChangePasswordRequest) String() string { return proto.CompactTextString(m) }
 func (*AuthUserChangePasswordRequest) ProtoMessage()    {}
 func (*AuthUserChangePasswordRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptorRpc, []int{52}
+	return fileDescriptorRpc, []int{54}
 }
 
 func (m *AuthUserChangePasswordRequest) GetName() string {
@@ -2450,7 +2483,7 @@ type AuthUserGrantRoleRequest struct {
 func (m *AuthUserGrantRoleRequest) Reset()                    { *m = AuthUserGrantRoleRequest{} }
 func (m *AuthUserGrantRoleRequest) String() string            { return proto.CompactTextString(m) }
 func (*AuthUserGrantRoleRequest) ProtoMessage()               {}
-func (*AuthUserGrantRoleRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{53} }
+func (*AuthUserGrantRoleRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{55} }
 
 func (m *AuthUserGrantRoleRequest) GetUser() string {
 	if m != nil {
@@ -2474,7 +2507,7 @@ type AuthUserRevokeRoleRequest struct {
 func (m *AuthUserRevokeRoleRequest) Reset()                    { *m = AuthUserRevokeRoleRequest{} }
 func (m *AuthUserRevokeRoleRequest) String() string            { return proto.CompactTextString(m) }
 func (*AuthUserRevokeRoleRequest) ProtoMessage()               {}
-func (*AuthUserRevokeRoleRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{54} }
+func (*AuthUserRevokeRoleRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{56} }
 
 func (m *AuthUserRevokeRoleRequest) GetName() string {
 	if m != nil {
@@ -2498,7 +2531,7 @@ type AuthRoleAddRequest struct {
 func (m *AuthRoleAddRequest) Reset()                    { *m = AuthRoleAddRequest{} }
 func (m *AuthRoleAddRequest) String() string            { return proto.CompactTextString(m) }
 func (*AuthRoleAddRequest) ProtoMessage()               {}
-func (*AuthRoleAddRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{55} }
+func (*AuthRoleAddRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{57} }
 
 func (m *AuthRoleAddRequest) GetName() string {
 	if m != nil {
@@ -2514,7 +2547,7 @@ type AuthRoleGetRequest struct {
 func (m *AuthRoleGetRequest) Reset()                    { *m = AuthRoleGetRequest{} }
 func (m *AuthRoleGetRequest) String() string            { return proto.CompactTextString(m) }
 func (*AuthRoleGetRequest) ProtoMessage()               {}
-func (*AuthRoleGetRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{56} }
+func (*AuthRoleGetRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{58} }
 
 func (m *AuthRoleGetRequest) GetRole() string {
 	if m != nil {
@@ -2529,7 +2562,7 @@ type AuthUserListRequest struct {
 func (m *AuthUserListRequest) Reset()                    { *m = AuthUserListRequest{} }
 func (m *AuthUserListRequest) String() string            { return proto.CompactTextString(m) }
 func (*AuthUserListRequest) ProtoMessage()               {}
-func (*AuthUserListRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{57} }
+func (*AuthUserListRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{59} }
 
 type AuthRoleListRequest struct {
 }
@@ -2537,7 +2570,7 @@ type AuthRoleListRequest struct {
 func (m *AuthRoleListRequest) Reset()                    { *m = AuthRoleListRequest{} }
 func (m *AuthRoleListRequest) String() string            { return proto.CompactTextString(m) }
 func (*AuthRoleListRequest) ProtoMessage()               {}
-func (*AuthRoleListRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{58} }
+func (*AuthRoleListRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{60} }
 
 type AuthRoleDeleteRequest struct {
 	Role string `protobuf:"bytes,1,opt,name=role,proto3" json:"role,omitempty"`
@@ -2546,7 +2579,7 @@ type AuthRoleDeleteRequest struct {
 func (m *AuthRoleDeleteRequest) Reset()                    { *m = AuthRoleDeleteRequest{} }
 func (m *AuthRoleDeleteRequest) String() string            { return proto.CompactTextString(m) }
 func (*AuthRoleDeleteRequest) ProtoMessage()               {}
-func (*AuthRoleDeleteRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{59} }
+func (*AuthRoleDeleteRequest) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{61} }
 
 func (m *AuthRoleDeleteRequest) GetRole() string {
 	if m != nil {
@@ -2566,7 +2599,7 @@ func (m *AuthRoleGrantPermissionRequest) Reset()         { *m = AuthRoleGrantPer
 func (m *AuthRoleGrantPermissionRequest) String() string { return proto.CompactTextString(m) }
 func (*AuthRoleGrantPermissionRequest) ProtoMessage()    {}
 func (*AuthRoleGrantPermissionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptorRpc, []int{60}
+	return fileDescriptorRpc, []int{62}
 }
 
 func (m *AuthRoleGrantPermissionRequest) GetName() string {
@@ -2593,7 +2626,7 @@ func (m *AuthRoleRevokePermissionRequest) Reset()         { *m = AuthRoleRevokeP
 func (m *AuthRoleRevokePermissionRequest) String() string { return proto.CompactTextString(m) }
 func (*AuthRoleRevokePermissionRequest) ProtoMessage()    {}
 func (*AuthRoleRevokePermissionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptorRpc, []int{61}
+	return fileDescriptorRpc, []int{63}
 }
 
 func (m *AuthRoleRevokePermissionRequest) GetRole() string {
@@ -2624,7 +2657,7 @@ type AuthEnableResponse struct {
 func (m *AuthEnableResponse) Reset()                    { *m = AuthEnableResponse{} }
 func (m *AuthEnableResponse) String() string            { return proto.CompactTextString(m) }
 func (*AuthEnableResponse) ProtoMessage()               {}
-func (*AuthEnableResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{62} }
+func (*AuthEnableResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{64} }
 
 func (m *AuthEnableResponse) GetHeader() *ResponseHeader {
 	if m != nil {
@@ -2640,7 +2673,7 @@ type AuthDisableResponse struct {
 func (m *AuthDisableResponse) Reset()                    { *m = AuthDisableResponse{} }
 func (m *AuthDisableResponse) String() string            { return proto.CompactTextString(m) }
 func (*AuthDisableResponse) ProtoMessage()               {}
-func (*AuthDisableResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{63} }
+func (*AuthDisableResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{65} }
 
 func (m *AuthDisableResponse) GetHeader() *ResponseHeader {
 	if m != nil {
@@ -2658,7 +2691,7 @@ type AuthenticateResponse struct {
 func (m *AuthenticateResponse) Reset()                    { *m = AuthenticateResponse{} }
 func (m *AuthenticateResponse) String() string            { return proto.CompactTextString(m) }
 func (*AuthenticateResponse) ProtoMessage()               {}
-func (*AuthenticateResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{64} }
+func (*AuthenticateResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{66} }
 
 func (m *AuthenticateResponse) GetHeader() *ResponseHeader {
 	if m != nil {
@@ -2681,7 +2714,7 @@ type AuthUserAddResponse struct {
 func (m *AuthUserAddResponse) Reset()                    { *m = AuthUserAddResponse{} }
 func (m *AuthUserAddResponse) String() string            { return proto.CompactTextString(m) }
 func (*AuthUserAddResponse) ProtoMessage()               {}
-func (*AuthUserAddResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{65} }
+func (*AuthUserAddResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{67} }
 
 func (m *AuthUserAddResponse) GetHeader() *ResponseHeader {
 	if m != nil {
@@ -2698,7 +2731,7 @@ type AuthUserGetResponse struct {
 func (m *AuthUserGetResponse) Reset()                    { *m = AuthUserGetResponse{} }
 func (m *AuthUserGetResponse) String() string            { return proto.CompactTextString(m) }
 func (*AuthUserGetResponse) ProtoMessage()               {}
-func (*AuthUserGetResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{66} }
+func (*AuthUserGetResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{68} }
 
 func (m *AuthUserGetResponse) GetHeader() *ResponseHeader {
 	if m != nil {
@@ -2721,7 +2754,7 @@ type AuthUserDeleteResponse struct {
 func (m *AuthUserDeleteResponse) Reset()                    { *m = AuthUserDeleteResponse{} }
 func (m *AuthUserDeleteResponse) String() string            { return proto.CompactTextString(m) }
 func (*AuthUserDeleteResponse) ProtoMessage()               {}
-func (*AuthUserDeleteResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{67} }
+func (*AuthUserDeleteResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{69} }
 
 func (m *AuthUserDeleteResponse) GetHeader() *ResponseHeader {
 	if m != nil {
@@ -2738,7 +2771,7 @@ func (m *AuthUserChangePasswordResponse) Reset()         { *m = AuthUserChangePa
 func (m *AuthUserChangePasswordResponse) String() string { return proto.CompactTextString(m) }
 func (*AuthUserChangePasswordResponse) ProtoMessage()    {}
 func (*AuthUserChangePasswordResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptorRpc, []int{68}
+	return fileDescriptorRpc, []int{70}
 }
 
 func (m *AuthUserChangePasswordResponse) GetHeader() *ResponseHeader {
@@ -2755,7 +2788,7 @@ type AuthUserGrantRoleResponse struct {
 func (m *AuthUserGrantRoleResponse) Reset()                    { *m = AuthUserGrantRoleResponse{} }
 func (m *AuthUserGrantRoleResponse) String() string            { return proto.CompactTextString(m) }
 func (*AuthUserGrantRoleResponse) ProtoMessage()               {}
-func (*AuthUserGrantRoleResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{69} }
+func (*AuthUserGrantRoleResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{71} }
 
 func (m *AuthUserGrantRoleResponse) GetHeader() *ResponseHeader {
 	if m != nil {
@@ -2771,7 +2804,7 @@ type AuthUserRevokeRoleResponse struct {
 func (m *AuthUserRevokeRoleResponse) Reset()                    { *m = AuthUserRevokeRoleResponse{} }
 func (m *AuthUserRevokeRoleResponse) String() string            { return proto.CompactTextString(m) }
 func (*AuthUserRevokeRoleResponse) ProtoMessage()               {}
-func (*AuthUserRevokeRoleResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{70} }
+func (*AuthUserRevokeRoleResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{72} }
 
 func (m *AuthUserRevokeRoleResponse) GetHeader() *ResponseHeader {
 	if m != nil {
@@ -2787,7 +2820,7 @@ type AuthRoleAddResponse struct {
 func (m *AuthRoleAddResponse) Reset()                    { *m = AuthRoleAddResponse{} }
 func (m *AuthRoleAddResponse) String() string            { return proto.CompactTextString(m) }
 func (*AuthRoleAddResponse) ProtoMessage()               {}
-func (*AuthRoleAddResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{71} }
+func (*AuthRoleAddResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{73} }
 
 func (m *AuthRoleAddResponse) GetHeader() *ResponseHeader {
 	if m != nil {
@@ -2804,7 +2837,7 @@ type AuthRoleGetResponse struct {
 func (m *AuthRoleGetResponse) Reset()                    { *m = AuthRoleGetResponse{} }
 func (m *AuthRoleGetResponse) String() string            { return proto.CompactTextString(m) }
 func (*AuthRoleGetResponse) ProtoMessage()               {}
-func (*AuthRoleGetResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{72} }
+func (*AuthRoleGetResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{74} }
 
 func (m *AuthRoleGetResponse) GetHeader() *ResponseHeader {
 	if m != nil {
@@ -2828,7 +2861,7 @@ type AuthRoleListResponse struct {
 func (m *AuthRoleListResponse) Reset()                    { *m = AuthRoleListResponse{} }
 func (m *AuthRoleListResponse) String() string            { return proto.CompactTextString(m) }
 func (*AuthRoleListResponse) ProtoMessage()               {}
-func (*AuthRoleListResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{73} }
+func (*AuthRoleListResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{75} }
 
 func (m *AuthRoleListResponse) GetHeader() *ResponseHeader {
 	if m != nil {
@@ -2852,7 +2885,7 @@ type AuthUserListResponse struct {
 func (m *AuthUserListResponse) Reset()                    { *m = AuthUserListResponse{} }
 func (m *AuthUserListResponse) String() string            { return proto.CompactTextString(m) }
 func (*AuthUserListResponse) ProtoMessage()               {}
-func (*AuthUserListResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{74} }
+func (*AuthUserListResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{76} }
 
 func (m *AuthUserListResponse) GetHeader() *ResponseHeader {
 	if m != nil {
@@ -2875,7 +2908,7 @@ type AuthRoleDeleteResponse struct {
 func (m *AuthRoleDeleteResponse) Reset()                    { *m = AuthRoleDeleteResponse{} }
 func (m *AuthRoleDeleteResponse) String() string            { return proto.CompactTextString(m) }
 func (*AuthRoleDeleteResponse) ProtoMessage()               {}
-func (*AuthRoleDeleteResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{75} }
+func (*AuthRoleDeleteResponse) Descriptor() ([]byte, []int) { return fileDescriptorRpc, []int{77} }
 
 func (m *AuthRoleDeleteResponse) GetHeader() *ResponseHeader {
 	if m != nil {
@@ -2892,7 +2925,7 @@ func (m *AuthRoleGrantPermissionResponse) Reset()         { *m = AuthRoleGrantPe
 func (m *AuthRoleGrantPermissionResponse) String() string { return proto.CompactTextString(m) }
 func (*AuthRoleGrantPermissionResponse) ProtoMessage()    {}
 func (*AuthRoleGrantPermissionResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptorRpc, []int{76}
+	return fileDescriptorRpc, []int{78}
 }
 
 func (m *AuthRoleGrantPermissionResponse) GetHeader() *ResponseHeader {
@@ -2910,7 +2943,7 @@ func (m *AuthRoleRevokePermissionResponse) Reset()         { *m = AuthRoleRevoke
 func (m *AuthRoleRevokePermissionResponse) String() string { return proto.CompactTextString(m) }
 func (*AuthRoleRevokePermissionResponse) ProtoMessage()    {}
 func (*AuthRoleRevokePermissionResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptorRpc, []int{77}
+	return fileDescriptorRpc, []int{79}
 }
 
 func (m *AuthRoleRevokePermissionResponse) GetHeader() *ResponseHeader {
@@ -2962,6 +2995,8 @@ func init() {
 	proto.RegisterType((*MemberListResponse)(nil), "etcdserverpb.MemberListResponse")
 	proto.RegisterType((*DefragmentRequest)(nil), "etcdserverpb.DefragmentRequest")
 	proto.RegisterType((*DefragmentResponse)(nil), "etcdserverpb.DefragmentResponse")
+	proto.RegisterType((*MoveLeaderRequest)(nil), "etcdserverpb.MoveLeaderRequest")
+	proto.RegisterType((*MoveLeaderResponse)(nil), "etcdserverpb.MoveLeaderResponse")
 	proto.RegisterType((*AlarmRequest)(nil), "etcdserverpb.AlarmRequest")
 	proto.RegisterType((*AlarmMember)(nil), "etcdserverpb.AlarmMember")
 	proto.RegisterType((*AlarmResponse)(nil), "etcdserverpb.AlarmResponse")
@@ -3742,6 +3777,8 @@ type MaintenanceClient interface {
 	Hash(ctx context.Context, in *HashRequest, opts ...grpc.CallOption) (*HashResponse, error)
 	// Snapshot sends a snapshot of the entire backend from a member over a stream to a client.
 	Snapshot(ctx context.Context, in *SnapshotRequest, opts ...grpc.CallOption) (Maintenance_SnapshotClient, error)
+	// MoveLeader requests current leader node to transfer its leadership to transferee.
+	MoveLeader(ctx context.Context, in *MoveLeaderRequest, opts ...grpc.CallOption) (*MoveLeaderResponse, error)
 }
 
 type maintenanceClient struct {
@@ -3820,6 +3857,15 @@ func (x *maintenanceSnapshotClient) Recv() (*SnapshotResponse, error) {
 	return m, nil
 }
 
+func (c *maintenanceClient) MoveLeader(ctx context.Context, in *MoveLeaderRequest, opts ...grpc.CallOption) (*MoveLeaderResponse, error) {
+	out := new(MoveLeaderResponse)
+	err := grpc.Invoke(ctx, "/etcdserverpb.Maintenance/MoveLeader", in, out, c.cc, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return out, nil
+}
+
 // Server API for Maintenance service
 
 type MaintenanceServer interface {
@@ -3835,6 +3881,8 @@ type MaintenanceServer interface {
 	Hash(context.Context, *HashRequest) (*HashResponse, error)
 	// Snapshot sends a snapshot of the entire backend from a member over a stream to a client.
 	Snapshot(*SnapshotRequest, Maintenance_SnapshotServer) error
+	// MoveLeader requests current leader node to transfer its leadership to transferee.
+	MoveLeader(context.Context, *MoveLeaderRequest) (*MoveLeaderResponse, error)
 }
 
 func RegisterMaintenanceServer(s *grpc.Server, srv MaintenanceServer) {
@@ -3934,6 +3982,24 @@ func (x *maintenanceSnapshotServer) Send(m *SnapshotResponse) error {
 	return x.ServerStream.SendMsg(m)
 }
 
+func _Maintenance_MoveLeader_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
+	in := new(MoveLeaderRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	if interceptor == nil {
+		return srv.(MaintenanceServer).MoveLeader(ctx, in)
+	}
+	info := &grpc.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: "/etcdserverpb.Maintenance/MoveLeader",
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		return srv.(MaintenanceServer).MoveLeader(ctx, req.(*MoveLeaderRequest))
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
 var _Maintenance_serviceDesc = grpc.ServiceDesc{
 	ServiceName: "etcdserverpb.Maintenance",
 	HandlerType: (*MaintenanceServer)(nil),
@@ -3954,6 +4020,10 @@ var _Maintenance_serviceDesc = grpc.ServiceDesc{
 			MethodName: "Hash",
 			Handler:    _Maintenance_Hash_Handler,
 		},
+		{
+			MethodName: "MoveLeader",
+			Handler:    _Maintenance_MoveLeader_Handler,
+		},
 	},
 	Streams: []grpc.StreamDesc{
 		{
@@ -6329,6 +6399,57 @@ func (m *DefragmentResponse) MarshalTo(dAtA []byte) (int, error) {
 	return i, nil
 }
 
+func (m *MoveLeaderRequest) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalTo(dAtA)
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *MoveLeaderRequest) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if m.TargetID != 0 {
+		dAtA[i] = 0x8
+		i++
+		i = encodeVarintRpc(dAtA, i, uint64(m.TargetID))
+	}
+	return i, nil
+}
+
+func (m *MoveLeaderResponse) Marshal() (dAtA []byte, err error) {
+	size := m.Size()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalTo(dAtA)
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *MoveLeaderResponse) MarshalTo(dAtA []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if m.Header != nil {
+		dAtA[i] = 0xa
+		i++
+		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
+		n36, err := m.Header.MarshalTo(dAtA[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n36
+	}
+	return i, nil
+}
+
 func (m *AlarmRequest) Marshal() (dAtA []byte, err error) {
 	size := m.Size()
 	dAtA = make([]byte, size)
@@ -6409,11 +6530,11 @@ func (m *AlarmResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n36, err := m.Header.MarshalTo(dAtA[i:])
+		n37, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n36
+		i += n37
 	}
 	if len(m.Alarms) > 0 {
 		for _, msg := range m.Alarms {
@@ -6467,11 +6588,11 @@ func (m *StatusResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n37, err := m.Header.MarshalTo(dAtA[i:])
+		n38, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n37
+		i += n38
 	}
 	if len(m.Version) > 0 {
 		dAtA[i] = 0x12
@@ -6869,11 +6990,11 @@ func (m *AuthRoleGrantPermissionRequest) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0x12
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Perm.Size()))
-		n38, err := m.Perm.MarshalTo(dAtA[i:])
+		n39, err := m.Perm.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n38
+		i += n39
 	}
 	return i, nil
 }
@@ -6933,11 +7054,11 @@ func (m *AuthEnableResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n39, err := m.Header.MarshalTo(dAtA[i:])
+		n40, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n39
+		i += n40
 	}
 	return i, nil
 }
@@ -6961,11 +7082,11 @@ func (m *AuthDisableResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n40, err := m.Header.MarshalTo(dAtA[i:])
+		n41, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n40
+		i += n41
 	}
 	return i, nil
 }
@@ -6989,11 +7110,11 @@ func (m *AuthenticateResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n41, err := m.Header.MarshalTo(dAtA[i:])
+		n42, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n41
+		i += n42
 	}
 	if len(m.Token) > 0 {
 		dAtA[i] = 0x12
@@ -7023,11 +7144,11 @@ func (m *AuthUserAddResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n42, err := m.Header.MarshalTo(dAtA[i:])
+		n43, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n42
+		i += n43
 	}
 	return i, nil
 }
@@ -7051,11 +7172,11 @@ func (m *AuthUserGetResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n43, err := m.Header.MarshalTo(dAtA[i:])
+		n44, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n43
+		i += n44
 	}
 	if len(m.Roles) > 0 {
 		for _, s := range m.Roles {
@@ -7094,11 +7215,11 @@ func (m *AuthUserDeleteResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n44, err := m.Header.MarshalTo(dAtA[i:])
+		n45, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n44
+		i += n45
 	}
 	return i, nil
 }
@@ -7122,11 +7243,11 @@ func (m *AuthUserChangePasswordResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n45, err := m.Header.MarshalTo(dAtA[i:])
+		n46, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n45
+		i += n46
 	}
 	return i, nil
 }
@@ -7150,11 +7271,11 @@ func (m *AuthUserGrantRoleResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n46, err := m.Header.MarshalTo(dAtA[i:])
+		n47, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n46
+		i += n47
 	}
 	return i, nil
 }
@@ -7178,11 +7299,11 @@ func (m *AuthUserRevokeRoleResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n47, err := m.Header.MarshalTo(dAtA[i:])
+		n48, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n47
+		i += n48
 	}
 	return i, nil
 }
@@ -7206,11 +7327,11 @@ func (m *AuthRoleAddResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n48, err := m.Header.MarshalTo(dAtA[i:])
+		n49, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n48
+		i += n49
 	}
 	return i, nil
 }
@@ -7234,11 +7355,11 @@ func (m *AuthRoleGetResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n49, err := m.Header.MarshalTo(dAtA[i:])
+		n50, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n49
+		i += n50
 	}
 	if len(m.Perm) > 0 {
 		for _, msg := range m.Perm {
@@ -7274,11 +7395,11 @@ func (m *AuthRoleListResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n50, err := m.Header.MarshalTo(dAtA[i:])
+		n51, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n50
+		i += n51
 	}
 	if len(m.Roles) > 0 {
 		for _, s := range m.Roles {
@@ -7317,11 +7438,11 @@ func (m *AuthUserListResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n51, err := m.Header.MarshalTo(dAtA[i:])
+		n52, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n51
+		i += n52
 	}
 	if len(m.Users) > 0 {
 		for _, s := range m.Users {
@@ -7360,11 +7481,11 @@ func (m *AuthRoleDeleteResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n52, err := m.Header.MarshalTo(dAtA[i:])
+		n53, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n52
+		i += n53
 	}
 	return i, nil
 }
@@ -7388,11 +7509,11 @@ func (m *AuthRoleGrantPermissionResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n53, err := m.Header.MarshalTo(dAtA[i:])
+		n54, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n53
+		i += n54
 	}
 	return i, nil
 }
@@ -7416,11 +7537,11 @@ func (m *AuthRoleRevokePermissionResponse) MarshalTo(dAtA []byte) (int, error) {
 		dAtA[i] = 0xa
 		i++
 		i = encodeVarintRpc(dAtA, i, uint64(m.Header.Size()))
-		n54, err := m.Header.MarshalTo(dAtA[i:])
+		n55, err := m.Header.MarshalTo(dAtA[i:])
 		if err != nil {
 			return 0, err
 		}
-		i += n54
+		i += n55
 	}
 	return i, nil
 }
@@ -8224,6 +8345,25 @@ func (m *DefragmentResponse) Size() (n int) {
 	return n
 }
 
+func (m *MoveLeaderRequest) Size() (n int) {
+	var l int
+	_ = l
+	if m.TargetID != 0 {
+		n += 1 + sovRpc(uint64(m.TargetID))
+	}
+	return n
+}
+
+func (m *MoveLeaderResponse) Size() (n int) {
+	var l int
+	_ = l
+	if m.Header != nil {
+		l = m.Header.Size()
+		n += 1 + l + sovRpc(uint64(l))
+	}
+	return n
+}
+
 func (m *AlarmRequest) Size() (n int) {
 	var l int
 	_ = l
@@ -13780,6 +13920,158 @@ func (m *DefragmentResponse) Unmarshal(dAtA []byte) error {
 	}
 	return nil
 }
+func (m *MoveLeaderRequest) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: MoveLeaderRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: MoveLeaderRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field TargetID", wireType)
+			}
+			m.TargetID = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.TargetID |= (uint64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *MoveLeaderResponse) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowRpc
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: MoveLeaderResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: MoveLeaderResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Header", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowRpc
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthRpc
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Header == nil {
+				m.Header = &ResponseHeader{}
+			}
+			if err := m.Header.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipRpc(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthRpc
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
 func (m *AlarmRequest) Unmarshal(dAtA []byte) error {
 	l := len(dAtA)
 	iNdEx := 0
@@ -17294,223 +17586,227 @@ var (
 func init() { proto.RegisterFile("rpc.proto", fileDescriptorRpc) }
 
 var fileDescriptorRpc = []byte{
-	// 3487 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5b, 0xdd, 0x6f, 0x1b, 0xc7,
-	0xb5, 0xd7, 0x92, 0x12, 0x29, 0x1e, 0x7e, 0x88, 0x1e, 0xc9, 0x36, 0xb5, 0xb6, 0x65, 0x79, 0xfc,
-	0x25, 0xdb, 0x89, 0x94, 0x28, 0xb9, 0xf7, 0xc1, 0x37, 0x08, 0xae, 0x2c, 0x31, 0x96, 0x22, 0x59,
-	0x72, 0x56, 0xb2, 0x93, 0x0b, 0x04, 0x97, 0x58, 0x91, 0x63, 0x6a, 0x21, 0x72, 0x97, 0xd9, 0x5d,
-	0xd2, 0x52, 0x6e, 0x2e, 0x70, 0x91, 0x9b, 0xa0, 0x68, 0x81, 0xbe, 0x34, 0x0f, 0xfd, 0x7a, 0x2c,
-	0x8a, 0x22, 0x7f, 0x40, 0xd1, 0x97, 0x3e, 0x17, 0x45, 0x5f, 0x5a, 0xa0, 0xff, 0x40, 0x91, 0xf6,
-	0xdf, 0x28, 0x5a, 0xcc, 0xd7, 0xee, 0xec, 0x72, 0x97, 0x52, 0xc2, 0x26, 0x2f, 0xd6, 0xce, 0x99,
-	0x33, 0xe7, 0x77, 0xe6, 0xcc, 0x9c, 0x73, 0x66, 0xce, 0xd0, 0x50, 0x70, 0x7b, 0xcd, 0xe5, 0x9e,
-	0xeb, 0xf8, 0x0e, 0x2a, 0x11, 0xbf, 0xd9, 0xf2, 0x88, 0x3b, 0x20, 0x6e, 0xef, 0x50, 0x9f, 0x6b,
-	0x3b, 0x6d, 0x87, 0x75, 0xac, 0xd0, 0x2f, 0xce, 0xa3, 0xcf, 0x53, 0x9e, 0x95, 0xee, 0xa0, 0xd9,
-	0x64, 0xff, 0xf4, 0x0e, 0x57, 0x8e, 0x07, 0xa2, 0xeb, 0x0a, 0xeb, 0x32, 0xfb, 0xfe, 0x11, 0xfb,
-	0xa7, 0x77, 0xc8, 0xfe, 0x88, 0xce, 0xab, 0x6d, 0xc7, 0x69, 0x77, 0xc8, 0x8a, 0xd9, 0xb3, 0x56,
-	0x4c, 0xdb, 0x76, 0x7c, 0xd3, 0xb7, 0x1c, 0xdb, 0xe3, 0xbd, 0xf8, 0x73, 0x0d, 0x2a, 0x06, 0xf1,
-	0x7a, 0x8e, 0xed, 0x91, 0x4d, 0x62, 0xb6, 0x88, 0x8b, 0xae, 0x01, 0x34, 0x3b, 0x7d, 0xcf, 0x27,
-	0x6e, 0xc3, 0x6a, 0xd5, 0xb4, 0x45, 0x6d, 0x69, 0xd2, 0x28, 0x08, 0xca, 0x56, 0x0b, 0x5d, 0x81,
-	0x42, 0x97, 0x74, 0x0f, 0x79, 0x6f, 0x86, 0xf5, 0x4e, 0x73, 0xc2, 0x56, 0x0b, 0xe9, 0x30, 0xed,
-	0x92, 0x81, 0xe5, 0x59, 0x8e, 0x5d, 0xcb, 0x2e, 0x6a, 0x4b, 0x59, 0x23, 0x68, 0xd3, 0x81, 0xae,
-	0xf9, 0xc2, 0x6f, 0xf8, 0xc4, 0xed, 0xd6, 0x26, 0xf9, 0x40, 0x4a, 0x38, 0x20, 0x6e, 0x17, 0x7f,
-	0x36, 0x05, 0x25, 0xc3, 0xb4, 0xdb, 0xc4, 0x20, 0x1f, 0xf5, 0x89, 0xe7, 0xa3, 0x2a, 0x64, 0x8f,
-	0xc9, 0x29, 0x83, 0x2f, 0x19, 0xf4, 0x93, 0x8f, 0xb7, 0xdb, 0xa4, 0x41, 0x6c, 0x0e, 0x5c, 0xa2,
-	0xe3, 0xed, 0x36, 0xa9, 0xdb, 0x2d, 0x34, 0x07, 0x53, 0x1d, 0xab, 0x6b, 0xf9, 0x02, 0x95, 0x37,
-	0x22, 0xea, 0x4c, 0xc6, 0xd4, 0x59, 0x07, 0xf0, 0x1c, 0xd7, 0x6f, 0x38, 0x6e, 0x8b, 0xb8, 0xb5,
-	0xa9, 0x45, 0x6d, 0xa9, 0xb2, 0x7a, 0x6b, 0x59, 0x5d, 0x88, 0x65, 0x55, 0xa1, 0xe5, 0x7d, 0xc7,
-	0xf5, 0xf7, 0x28, 0xaf, 0x51, 0xf0, 0xe4, 0x27, 0x7a, 0x07, 0x8a, 0x4c, 0x88, 0x6f, 0xba, 0x6d,
-	0xe2, 0xd7, 0x72, 0x4c, 0xca, 0xed, 0x33, 0xa4, 0x1c, 0x30, 0x66, 0x83, 0xc1, 0xf3, 0x6f, 0x84,
-	0xa1, 0xe4, 0x11, 0xd7, 0x32, 0x3b, 0xd6, 0xc7, 0xe6, 0x61, 0x87, 0xd4, 0xf2, 0x8b, 0xda, 0xd2,
-	0xb4, 0x11, 0xa1, 0xd1, 0xf9, 0x1f, 0x93, 0x53, 0xaf, 0xe1, 0xd8, 0x9d, 0xd3, 0xda, 0x34, 0x63,
-	0x98, 0xa6, 0x84, 0x3d, 0xbb, 0x73, 0xca, 0x16, 0xcd, 0xe9, 0xdb, 0x3e, 0xef, 0x2d, 0xb0, 0xde,
-	0x02, 0xa3, 0xb0, 0xee, 0x25, 0xa8, 0x76, 0x2d, 0xbb, 0xd1, 0x75, 0x5a, 0x8d, 0xc0, 0x20, 0xc0,
-	0x0c, 0x52, 0xe9, 0x5a, 0xf6, 0x13, 0xa7, 0x65, 0x48, 0xb3, 0x50, 0x4e, 0xf3, 0x24, 0xca, 0x59,
-	0x14, 0x9c, 0xe6, 0x89, 0xca, 0xb9, 0x0c, 0xb3, 0x54, 0x66, 0xd3, 0x25, 0xa6, 0x4f, 0x42, 0xe6,
-	0x12, 0x63, 0xbe, 0xd0, 0xb5, 0xec, 0x75, 0xd6, 0x13, 0xe1, 0x37, 0x4f, 0x86, 0xf8, 0xcb, 0x82,
-	0xdf, 0x3c, 0x89, 0xf2, 0xe3, 0x65, 0x28, 0x04, 0x36, 0x47, 0xd3, 0x30, 0xb9, 0xbb, 0xb7, 0x5b,
-	0xaf, 0x4e, 0x20, 0x80, 0xdc, 0xda, 0xfe, 0x7a, 0x7d, 0x77, 0xa3, 0xaa, 0xa1, 0x22, 0xe4, 0x37,
-	0xea, 0xbc, 0x91, 0xc1, 0x8f, 0x00, 0x42, 0xeb, 0xa2, 0x3c, 0x64, 0xb7, 0xeb, 0xff, 0x55, 0x9d,
-	0xa0, 0x3c, 0xcf, 0xeb, 0xc6, 0xfe, 0xd6, 0xde, 0x6e, 0x55, 0xa3, 0x83, 0xd7, 0x8d, 0xfa, 0xda,
-	0x41, 0xbd, 0x9a, 0xa1, 0x1c, 0x4f, 0xf6, 0x36, 0xaa, 0x59, 0x54, 0x80, 0xa9, 0xe7, 0x6b, 0x3b,
-	0xcf, 0xea, 0xd5, 0x49, 0xfc, 0x85, 0x06, 0x65, 0xb1, 0x5e, 0xdc, 0x27, 0xd0, 0x9b, 0x90, 0x3b,
-	0x62, 0x7e, 0xc1, 0xb6, 0x62, 0x71, 0xf5, 0x6a, 0x6c, 0x71, 0x23, 0xbe, 0x63, 0x08, 0x5e, 0x84,
-	0x21, 0x7b, 0x3c, 0xf0, 0x6a, 0x99, 0xc5, 0xec, 0x52, 0x71, 0xb5, 0xba, 0xcc, 0x1d, 0x76, 0x79,
-	0x9b, 0x9c, 0x3e, 0x37, 0x3b, 0x7d, 0x62, 0xd0, 0x4e, 0x84, 0x60, 0xb2, 0xeb, 0xb8, 0x84, 0xed,
-	0xd8, 0x69, 0x83, 0x7d, 0xd3, 0x6d, 0xcc, 0x16, 0x4d, 0xec, 0x56, 0xde, 0xc0, 0x5f, 0x6a, 0x00,
-	0x4f, 0xfb, 0x7e, 0xba, 0x6b, 0xcc, 0xc1, 0xd4, 0x80, 0x0a, 0x16, 0x6e, 0xc1, 0x1b, 0xcc, 0x27,
-	0x88, 0xe9, 0x91, 0xc0, 0x27, 0x68, 0x03, 0x5d, 0x86, 0x7c, 0xcf, 0x25, 0x83, 0xc6, 0xf1, 0x80,
-	0x81, 0x4c, 0x1b, 0x39, 0xda, 0xdc, 0x1e, 0xa0, 0x1b, 0x50, 0xb2, 0xda, 0xb6, 0xe3, 0x92, 0x06,
-	0x97, 0x35, 0xc5, 0x7a, 0x8b, 0x9c, 0xc6, 0xf4, 0x56, 0x58, 0xb8, 0xe0, 0x9c, 0xca, 0xb2, 0x43,
-	0x49, 0xd8, 0x86, 0x22, 0x53, 0x75, 0x2c, 0xf3, 0xdd, 0x0b, 0x75, 0xcc, 0xb0, 0x61, 0xc3, 0x26,
-	0x14, 0x5a, 0xe3, 0x0f, 0x01, 0x6d, 0x90, 0x0e, 0xf1, 0xc9, 0x38, 0xd1, 0x43, 0xb1, 0x49, 0x56,
-	0xb5, 0x09, 0xfe, 0x91, 0x06, 0xb3, 0x11, 0xf1, 0x63, 0x4d, 0xab, 0x06, 0xf9, 0x16, 0x13, 0xc6,
-	0x35, 0xc8, 0x1a, 0xb2, 0x89, 0x1e, 0xc0, 0xb4, 0x50, 0xc0, 0xab, 0x65, 0x53, 0x36, 0x4d, 0x9e,
-	0xeb, 0xe4, 0xe1, 0x2f, 0x33, 0x50, 0x10, 0x13, 0xdd, 0xeb, 0xa1, 0x35, 0x28, 0xbb, 0xbc, 0xd1,
-	0x60, 0xf3, 0x11, 0x1a, 0xe9, 0xe9, 0x41, 0x68, 0x73, 0xc2, 0x28, 0x89, 0x21, 0x8c, 0x8c, 0xfe,
-	0x03, 0x8a, 0x52, 0x44, 0xaf, 0xef, 0x0b, 0x93, 0xd7, 0xa2, 0x02, 0xc2, 0xfd, 0xb7, 0x39, 0x61,
-	0x80, 0x60, 0x7f, 0xda, 0xf7, 0xd1, 0x01, 0xcc, 0xc9, 0xc1, 0x7c, 0x36, 0x42, 0x8d, 0x2c, 0x93,
-	0xb2, 0x18, 0x95, 0x32, 0xbc, 0x54, 0x9b, 0x13, 0x06, 0x12, 0xe3, 0x95, 0x4e, 0x55, 0x25, 0xff,
-	0x84, 0x07, 0xef, 0x21, 0x95, 0x0e, 0x4e, 0xec, 0x61, 0x95, 0x0e, 0x4e, 0xec, 0x47, 0x05, 0xc8,
-	0x8b, 0x16, 0xfe, 0x4d, 0x06, 0x40, 0xae, 0xc6, 0x5e, 0x0f, 0x6d, 0x40, 0xc5, 0x15, 0xad, 0x88,
-	0xb5, 0xae, 0x24, 0x5a, 0x4b, 0x2c, 0xe2, 0x84, 0x51, 0x96, 0x83, 0xb8, 0x72, 0x6f, 0x43, 0x29,
-	0x90, 0x12, 0x1a, 0x6c, 0x3e, 0xc1, 0x60, 0x81, 0x84, 0xa2, 0x1c, 0x40, 0x4d, 0xf6, 0x3e, 0x5c,
-	0x0c, 0xc6, 0x27, 0xd8, 0xec, 0xc6, 0x08, 0x9b, 0x05, 0x02, 0x67, 0xa5, 0x04, 0xd5, 0x6a, 0xaa,
-	0x62, 0xa1, 0xd9, 0xe6, 0x13, 0xcc, 0x36, 0xac, 0x18, 0x35, 0x1c, 0xd0, 0x7c, 0xc9, 0x9b, 0xf8,
-	0x77, 0x59, 0xc8, 0xaf, 0x3b, 0xdd, 0x9e, 0xe9, 0xd2, 0xd5, 0xc8, 0xb9, 0xc4, 0xeb, 0x77, 0x7c,
-	0x66, 0xae, 0xca, 0xea, 0xcd, 0xa8, 0x44, 0xc1, 0x26, 0xff, 0x1a, 0x8c, 0xd5, 0x10, 0x43, 0xe8,
-	0x60, 0x91, 0x1e, 0x33, 0xe7, 0x18, 0x2c, 0x92, 0xa3, 0x18, 0x22, 0x1d, 0x39, 0x1b, 0x3a, 0xb2,
-	0x0e, 0xf9, 0x01, 0x71, 0xc3, 0x94, 0xbe, 0x39, 0x61, 0x48, 0x02, 0xba, 0x07, 0x33, 0xf1, 0xf4,
-	0x32, 0x25, 0x78, 0x2a, 0xcd, 0x68, 0x36, 0xba, 0x09, 0xa5, 0x48, 0x8e, 0xcb, 0x09, 0xbe, 0x62,
-	0x57, 0x49, 0x71, 0x97, 0x64, 0x5c, 0xa5, 0xf9, 0xb8, 0xb4, 0x39, 0x21, 0x23, 0x6b, 0x24, 0x98,
-	0x4c, 0x47, 0x83, 0x09, 0xfe, 0x4f, 0x28, 0x47, 0x0c, 0x41, 0xf3, 0x4b, 0xfd, 0xbd, 0x67, 0x6b,
-	0x3b, 0x3c, 0x19, 0x3d, 0x66, 0xf9, 0xc7, 0xa8, 0x6a, 0x34, 0xa7, 0xed, 0xd4, 0xf7, 0xf7, 0xab,
-	0x19, 0x54, 0x86, 0xc2, 0xee, 0xde, 0x41, 0x83, 0x73, 0x65, 0xf1, 0x5b, 0x81, 0x04, 0x91, 0xcc,
-	0x94, 0x1c, 0x36, 0xa1, 0xe4, 0x30, 0x4d, 0xe6, 0xb0, 0x4c, 0x98, 0xc3, 0xb2, 0x8f, 0x2a, 0x50,
-	0xe2, 0xc6, 0x6b, 0xf4, 0x6d, 0x9a, 0x47, 0x7f, 0xa1, 0x01, 0x84, 0xae, 0x82, 0x56, 0x20, 0xdf,
-	0xe4, 0xc2, 0x6b, 0x1a, 0x8b, 0x34, 0x17, 0x13, 0xd7, 0xc3, 0x90, 0x5c, 0xe8, 0x75, 0xc8, 0x7b,
-	0xfd, 0x66, 0x93, 0x78, 0x32, 0x9f, 0x5d, 0x8e, 0x07, 0x3b, 0x11, 0x8a, 0x0c, 0xc9, 0x47, 0x87,
-	0xbc, 0x30, 0xad, 0x4e, 0x9f, 0x65, 0xb7, 0xd1, 0x43, 0x04, 0x1f, 0xfe, 0xa9, 0x06, 0x45, 0x65,
-	0x67, 0x7e, 0xc3, 0x08, 0x7b, 0x15, 0x0a, 0x4c, 0x07, 0xd2, 0x12, 0x31, 0x76, 0xda, 0x08, 0x09,
-	0xe8, 0xdf, 0xa1, 0x20, 0xb7, 0xb7, 0x0c, 0xb3, 0xb5, 0x64, 0xb1, 0x7b, 0x3d, 0x23, 0x64, 0xc5,
-	0xdb, 0x70, 0x81, 0x59, 0xa5, 0x49, 0x4f, 0xce, 0xd2, 0x8e, 0xea, 0xd9, 0x52, 0x8b, 0x9d, 0x2d,
-	0x75, 0x98, 0xee, 0x1d, 0x9d, 0x7a, 0x56, 0xd3, 0xec, 0x08, 0x2d, 0x82, 0x36, 0x7e, 0x17, 0x90,
-	0x2a, 0x6c, 0x9c, 0xe9, 0xe2, 0x32, 0x14, 0x37, 0x4d, 0xef, 0x48, 0xa8, 0x84, 0x3f, 0x80, 0x12,
-	0x6f, 0x8e, 0x65, 0x43, 0x04, 0x93, 0x47, 0xa6, 0x77, 0xc4, 0x14, 0x2f, 0x1b, 0xec, 0x1b, 0x5f,
-	0x80, 0x99, 0x7d, 0xdb, 0xec, 0x79, 0x47, 0x8e, 0xcc, 0x02, 0xf4, 0xe6, 0x50, 0x0d, 0x69, 0x63,
-	0x21, 0xde, 0x85, 0x19, 0x97, 0x74, 0x4d, 0xcb, 0xb6, 0xec, 0x76, 0xe3, 0xf0, 0xd4, 0x27, 0x9e,
-	0xb8, 0x58, 0x54, 0x02, 0xf2, 0x23, 0x4a, 0xa5, 0xaa, 0x1d, 0x76, 0x9c, 0x43, 0x11, 0x0e, 0xd8,
-	0x37, 0xfe, 0xb5, 0x06, 0xa5, 0xf7, 0x4d, 0xbf, 0x29, 0xad, 0x80, 0xb6, 0xa0, 0x12, 0x04, 0x01,
-	0x46, 0x11, 0xba, 0xc4, 0x52, 0x11, 0x1b, 0x23, 0x8f, 0x9c, 0x32, 0x8b, 0x94, 0x9b, 0x2a, 0x81,
-	0x89, 0x32, 0xed, 0x26, 0xe9, 0x04, 0xa2, 0x32, 0xe9, 0xa2, 0x18, 0xa3, 0x2a, 0x4a, 0x25, 0x3c,
-	0x9a, 0x09, 0xd3, 0x34, 0x77, 0xcb, 0x9f, 0x65, 0x00, 0x0d, 0xeb, 0xf0, 0x75, 0x4f, 0x2e, 0xb7,
-	0xa1, 0xe2, 0xf9, 0xa6, 0xeb, 0x37, 0x62, 0xd7, 0xae, 0x32, 0xa3, 0x06, 0x81, 0xec, 0x2e, 0xcc,
-	0xf4, 0x5c, 0xa7, 0xed, 0x12, 0xcf, 0x6b, 0xd8, 0x8e, 0x6f, 0xbd, 0x38, 0x15, 0x87, 0xbf, 0x8a,
-	0x24, 0xef, 0x32, 0x2a, 0xaa, 0x43, 0xfe, 0x85, 0xd5, 0xf1, 0x89, 0xeb, 0xd5, 0xa6, 0x16, 0xb3,
-	0x4b, 0x95, 0xd5, 0x07, 0x67, 0x59, 0x6d, 0xf9, 0x1d, 0xc6, 0x7f, 0x70, 0xda, 0x23, 0x86, 0x1c,
-	0xab, 0x1e, 0xa8, 0x72, 0x91, 0x03, 0xd5, 0x6d, 0x80, 0x90, 0x9f, 0x46, 0xad, 0xdd, 0xbd, 0xa7,
-	0xcf, 0x0e, 0xaa, 0x13, 0xa8, 0x04, 0xd3, 0xbb, 0x7b, 0x1b, 0xf5, 0x9d, 0x3a, 0x8d, 0x6b, 0x78,
-	0x45, 0xda, 0x46, 0xb5, 0x21, 0x9a, 0x87, 0xe9, 0x97, 0x94, 0x2a, 0xef, 0xa5, 0x59, 0x23, 0xcf,
-	0xda, 0x5b, 0x2d, 0xfc, 0xc3, 0x0c, 0x94, 0xc5, 0x2e, 0x18, 0x6b, 0x2b, 0xaa, 0x10, 0x99, 0x08,
-	0x04, 0x3d, 0xbd, 0xf1, 0xdd, 0xd1, 0x12, 0x87, 0x44, 0xd9, 0xa4, 0xee, 0xce, 0x17, 0x9b, 0xb4,
-	0x84, 0x59, 0x83, 0x36, 0xba, 0x07, 0xd5, 0x26, 0x77, 0xf7, 0x58, 0x4e, 0x32, 0x66, 0x04, 0x5d,
-	0x49, 0x49, 0xe5, 0x60, 0xb7, 0x99, 0x9e, 0xc8, 0x49, 0x05, 0xa3, 0x24, 0x37, 0x12, 0xa5, 0xa1,
-	0xdb, 0x90, 0x23, 0x03, 0x62, 0xfb, 0x5e, 0xad, 0xc8, 0x02, 0x58, 0x59, 0x9e, 0x13, 0xeb, 0x94,
-	0x6a, 0x88, 0x4e, 0xfc, 0x6f, 0x70, 0x81, 0x9d, 0xc7, 0x1f, 0xbb, 0xa6, 0xad, 0x5e, 0x1c, 0x0e,
-	0x0e, 0x76, 0x84, 0xe9, 0xe8, 0x27, 0xaa, 0x40, 0x66, 0x6b, 0x43, 0x4c, 0x34, 0xb3, 0xb5, 0x81,
-	0x3f, 0xd5, 0x00, 0xa9, 0xe3, 0xc6, 0xb2, 0x65, 0x4c, 0xb8, 0x84, 0xcf, 0x86, 0xf0, 0x73, 0x30,
-	0x45, 0x5c, 0xd7, 0x71, 0x99, 0xd5, 0x0a, 0x06, 0x6f, 0xe0, 0x5b, 0x42, 0x07, 0x83, 0x0c, 0x9c,
-	0xe3, 0xc0, 0x31, 0xb8, 0x34, 0x2d, 0x50, 0x75, 0x1b, 0x66, 0x23, 0x5c, 0x63, 0x05, 0xd2, 0xbb,
-	0x70, 0x91, 0x09, 0xdb, 0x26, 0xa4, 0xb7, 0xd6, 0xb1, 0x06, 0xa9, 0xa8, 0x3d, 0xb8, 0x14, 0x67,
-	0xfc, 0x76, 0x6d, 0x84, 0xdf, 0x12, 0x88, 0x07, 0x56, 0x97, 0x1c, 0x38, 0x3b, 0xe9, 0xba, 0xd1,
-	0xe8, 0x78, 0x4c, 0x4e, 0x3d, 0x91, 0x71, 0xd8, 0x37, 0xfe, 0xa5, 0x06, 0x97, 0x87, 0x86, 0x7f,
-	0xcb, 0xab, 0xba, 0x00, 0xd0, 0xa6, 0xdb, 0x87, 0xb4, 0x68, 0x07, 0xbf, 0xc9, 0x2a, 0x94, 0x40,
-	0x4f, 0x1a, 0x60, 0x4a, 0x42, 0xcf, 0x23, 0xc8, 0x3d, 0x61, 0x45, 0x24, 0x65, 0x56, 0x93, 0x72,
-	0x56, 0xb6, 0xd9, 0xe5, 0x57, 0xdb, 0x82, 0xc1, 0xbe, 0x59, 0x7e, 0x25, 0xc4, 0x7d, 0x66, 0xec,
-	0xf0, 0x3c, 0x5e, 0x30, 0x82, 0x36, 0x45, 0x6f, 0x76, 0x2c, 0x62, 0xfb, 0xac, 0x77, 0x92, 0xf5,
-	0x2a, 0x14, 0xbc, 0x0c, 0x55, 0x8e, 0xb4, 0xd6, 0x6a, 0x29, 0xb9, 0x3c, 0x90, 0xa7, 0x45, 0xe5,
-	0xe1, 0x5f, 0x69, 0x70, 0x41, 0x19, 0x30, 0x96, 0xed, 0x5e, 0x81, 0x1c, 0x2f, 0x95, 0x89, 0x3c,
-	0x32, 0x17, 0x1d, 0xc5, 0x61, 0x0c, 0xc1, 0x83, 0x96, 0x21, 0xcf, 0xbf, 0xe4, 0x61, 0x25, 0x99,
-	0x5d, 0x32, 0xe1, 0xdb, 0x30, 0x2b, 0x48, 0xa4, 0xeb, 0x24, 0x6d, 0x13, 0x66, 0x50, 0xfc, 0x09,
-	0xcc, 0x45, 0xd9, 0xc6, 0x9a, 0x92, 0xa2, 0x64, 0xe6, 0x3c, 0x4a, 0xae, 0x49, 0x25, 0x9f, 0xf5,
-	0x5a, 0x4a, 0xda, 0x8b, 0xaf, 0xba, 0xba, 0x22, 0x99, 0xd8, 0x8a, 0x04, 0x13, 0x90, 0x22, 0xbe,
-	0xd3, 0x09, 0xcc, 0xca, 0xed, 0xb0, 0x63, 0x79, 0xc1, 0x61, 0xe8, 0x63, 0x40, 0x2a, 0xf1, 0xbb,
-	0x56, 0x68, 0x83, 0xbc, 0x70, 0xcd, 0x76, 0x97, 0x04, 0xa1, 0x9e, 0x9e, 0x32, 0x55, 0xe2, 0x58,
-	0xc1, 0xf1, 0x8f, 0x1a, 0x94, 0xd6, 0x3a, 0xa6, 0xdb, 0x95, 0x8b, 0xf5, 0x36, 0xe4, 0xf8, 0xf1,
-	0x55, 0x5c, 0x07, 0xef, 0x44, 0xc5, 0xa8, 0xbc, 0xbc, 0xb1, 0xc6, 0x0f, 0xbb, 0x62, 0x14, 0x5d,
-	0x5c, 0x51, 0x31, 0xde, 0x88, 0x55, 0x90, 0x37, 0xd0, 0xab, 0x30, 0x65, 0xd2, 0x21, 0x2c, 0xa0,
-	0x54, 0xe2, 0x17, 0x07, 0x26, 0x8d, 0x1d, 0x35, 0x38, 0x17, 0x7e, 0x13, 0x8a, 0x0a, 0x02, 0xbd,
-	0x0f, 0x3d, 0xae, 0x8b, 0xe3, 0xc4, 0xda, 0xfa, 0xc1, 0xd6, 0x73, 0x7e, 0x4d, 0xaa, 0x00, 0x6c,
-	0xd4, 0x83, 0x76, 0x06, 0x7f, 0x20, 0x46, 0x89, 0x90, 0xa3, 0xea, 0xa3, 0xa5, 0xe9, 0x93, 0x39,
-	0x97, 0x3e, 0x27, 0x50, 0x16, 0xd3, 0x1f, 0x6b, 0x0f, 0xbc, 0x0e, 0x39, 0x26, 0x4f, 0x6e, 0x81,
-	0xf9, 0x04, 0x58, 0x19, 0x2d, 0x38, 0x23, 0x9e, 0x81, 0xf2, 0xbe, 0x6f, 0xfa, 0x7d, 0x4f, 0x6e,
-	0x81, 0x3f, 0x68, 0x50, 0x91, 0x94, 0x71, 0xcb, 0x56, 0xf2, 0xc6, 0xcd, 0x83, 0x70, 0x70, 0xdf,
-	0xbe, 0x04, 0xb9, 0xd6, 0xe1, 0xbe, 0xf5, 0xb1, 0x2c, 0x31, 0x8a, 0x16, 0xa5, 0x77, 0x38, 0x0e,
-	0xaf, 0xf3, 0x8b, 0x16, 0xbd, 0x9e, 0xb9, 0xe6, 0x0b, 0x7f, 0xcb, 0x6e, 0x91, 0x13, 0x76, 0x0a,
-	0x9a, 0x34, 0x42, 0x02, 0xbb, 0x51, 0x89, 0xf7, 0x00, 0x76, 0xf4, 0x51, 0xdf, 0x07, 0x66, 0xe1,
-	0xc2, 0x5a, 0xdf, 0x3f, 0xaa, 0xdb, 0xe6, 0x61, 0x47, 0x06, 0x0d, 0x3c, 0x07, 0x88, 0x12, 0x37,
-	0x2c, 0x4f, 0xa5, 0xd6, 0x61, 0x96, 0x52, 0x89, 0xed, 0x5b, 0x4d, 0x25, 0xc2, 0xc8, 0x3c, 0xa2,
-	0xc5, 0xf2, 0x88, 0xe9, 0x79, 0x2f, 0x1d, 0xb7, 0x25, 0xa6, 0x16, 0xb4, 0xf1, 0x06, 0x17, 0xfe,
-	0xcc, 0x8b, 0x64, 0x8a, 0xaf, 0x2b, 0x65, 0x29, 0x94, 0xf2, 0x98, 0xf8, 0x23, 0xa4, 0xe0, 0x07,
-	0x70, 0x51, 0x72, 0x8a, 0x92, 0xce, 0x08, 0xe6, 0x3d, 0xb8, 0x26, 0x99, 0xd7, 0x8f, 0xe8, 0x5d,
-	0xe0, 0xa9, 0x00, 0xfc, 0xa6, 0x7a, 0x3e, 0x82, 0x5a, 0xa0, 0x27, 0x3b, 0xfa, 0x39, 0x1d, 0x55,
-	0x81, 0xbe, 0x27, 0xf6, 0x4c, 0xc1, 0x60, 0xdf, 0x94, 0xe6, 0x3a, 0x9d, 0x20, 0x2b, 0xd3, 0x6f,
-	0xbc, 0x0e, 0xf3, 0x52, 0x86, 0x38, 0x94, 0x45, 0x85, 0x0c, 0x29, 0x94, 0x24, 0x44, 0x18, 0x8c,
-	0x0e, 0x1d, 0x6d, 0x76, 0x95, 0x33, 0x6a, 0x5a, 0x26, 0x53, 0x53, 0x64, 0x5e, 0xe4, 0x3b, 0x82,
-	0x2a, 0xa6, 0x06, 0x6d, 0x41, 0xa6, 0x02, 0x54, 0xb2, 0x58, 0x08, 0x4a, 0x1e, 0x5a, 0x88, 0x21,
-	0xd1, 0x1f, 0xc2, 0x42, 0xa0, 0x04, 0xb5, 0xdb, 0x53, 0xe2, 0x76, 0x2d, 0xcf, 0x53, 0xea, 0x04,
-	0x49, 0x13, 0xbf, 0x03, 0x93, 0x3d, 0x22, 0x62, 0x4a, 0x71, 0x15, 0x2d, 0xf3, 0x57, 0xbb, 0x65,
-	0x65, 0x30, 0xeb, 0xc7, 0x2d, 0xb8, 0x2e, 0xa5, 0x73, 0x8b, 0x26, 0x8a, 0x8f, 0x2b, 0x25, 0xef,
-	0x90, 0xdc, 0xac, 0xc3, 0x77, 0xc8, 0x2c, 0x5f, 0xfb, 0xa0, 0x60, 0xf5, 0x2e, 0x37, 0xa4, 0xf4,
-	0xad, 0xb1, 0x72, 0xc5, 0x36, 0xb7, 0x69, 0xe0, 0x92, 0x63, 0x09, 0x3b, 0x84, 0xb9, 0xa8, 0x27,
-	0x8f, 0x15, 0xc6, 0xe6, 0x60, 0xca, 0x77, 0x8e, 0x89, 0x0c, 0x62, 0xbc, 0x21, 0x15, 0x0e, 0xdc,
-	0x7c, 0x2c, 0x85, 0xcd, 0x50, 0x18, 0xdb, 0x92, 0xe3, 0xea, 0x4b, 0x57, 0x53, 0x9e, 0x7f, 0x78,
-	0x03, 0xef, 0xc2, 0xa5, 0x78, 0x98, 0x18, 0x4b, 0xe5, 0xe7, 0x7c, 0x03, 0x27, 0x45, 0x92, 0xb1,
-	0xe4, 0xbe, 0x17, 0x06, 0x03, 0x25, 0xa0, 0x8c, 0x25, 0xd2, 0x00, 0x3d, 0x29, 0xbe, 0xfc, 0x2b,
-	0xf6, 0x6b, 0x10, 0x6e, 0xc6, 0x12, 0xe6, 0x85, 0xc2, 0xc6, 0x5f, 0xfe, 0x30, 0x46, 0x64, 0x47,
-	0xc6, 0x08, 0xe1, 0x24, 0x61, 0x14, 0xfb, 0x16, 0x36, 0x9d, 0xc0, 0x08, 0x03, 0xe8, 0xb8, 0x18,
-	0x34, 0x87, 0x04, 0x18, 0xac, 0x21, 0x37, 0xb6, 0x1a, 0x76, 0xc7, 0x5a, 0x8c, 0xf7, 0xc3, 0xd8,
-	0x39, 0x14, 0x99, 0xc7, 0x12, 0xfc, 0x01, 0x2c, 0xa6, 0x07, 0xe5, 0x71, 0x24, 0xdf, 0xc7, 0x50,
-	0x08, 0x0e, 0x94, 0xca, 0x8b, 0x77, 0x11, 0xf2, 0xbb, 0x7b, 0xfb, 0x4f, 0xd7, 0xd6, 0xeb, 0x55,
-	0x6d, 0xf5, 0xef, 0x59, 0xc8, 0x6c, 0x3f, 0x47, 0xff, 0x0d, 0x53, 0xfc, 0xc9, 0x67, 0xc4, 0x3b,
-	0x9f, 0x3e, 0xea, 0x55, 0x0b, 0x5f, 0xfd, 0xf4, 0xcf, 0x7f, 0xfb, 0x22, 0x73, 0x09, 0x5f, 0x58,
-	0x19, 0xbc, 0x61, 0x76, 0x7a, 0x47, 0xe6, 0xca, 0xf1, 0x60, 0x85, 0xe5, 0x84, 0x87, 0xda, 0x7d,
-	0xf4, 0x1c, 0xb2, 0x4f, 0xfb, 0x3e, 0x4a, 0x7d, 0x04, 0xd4, 0xd3, 0x5f, 0xbb, 0xb0, 0xce, 0x24,
-	0xcf, 0xe1, 0x19, 0x55, 0x72, 0xaf, 0xef, 0x53, 0xb9, 0x03, 0x28, 0xaa, 0x0f, 0x56, 0x67, 0x3e,
-	0x0f, 0xea, 0x67, 0x3f, 0x86, 0x61, 0xcc, 0xf0, 0xae, 0xe2, 0xcb, 0x2a, 0x1e, 0x7f, 0x57, 0x53,
-	0xe7, 0x73, 0x70, 0x62, 0xa3, 0xd4, 0x17, 0x44, 0x3d, 0xfd, 0x91, 0x2c, 0x79, 0x3e, 0xfe, 0x89,
-	0x4d, 0xe5, 0x3a, 0xe2, 0x91, 0xac, 0xe9, 0xa3, 0xeb, 0x09, 0xef, 0x28, 0xea, 0x8b, 0x81, 0xbe,
-	0x98, 0xce, 0x20, 0x90, 0x6e, 0x30, 0xa4, 0x2b, 0xf8, 0x92, 0x8a, 0xd4, 0x0c, 0xf8, 0x1e, 0x6a,
-	0xf7, 0x57, 0x8f, 0x60, 0x8a, 0xd5, 0x39, 0x51, 0x43, 0x7e, 0xe8, 0x09, 0x15, 0xda, 0x94, 0x1d,
-	0x10, 0xa9, 0x90, 0xe2, 0x79, 0x86, 0x36, 0x8b, 0x2b, 0x01, 0x1a, 0x2b, 0x75, 0x3e, 0xd4, 0xee,
-	0x2f, 0x69, 0xaf, 0x69, 0xab, 0xff, 0x3f, 0x09, 0x53, 0xac, 0x74, 0x84, 0x7a, 0x00, 0x61, 0x51,
-	0x30, 0x3e, 0xcf, 0xa1, 0x32, 0x63, 0x7c, 0x9e, 0xc3, 0xf5, 0x44, 0x7c, 0x9d, 0x21, 0xcf, 0xe3,
-	0xb9, 0x00, 0x99, 0xfd, 0x9e, 0x60, 0x85, 0x15, 0x89, 0xa8, 0x59, 0x5f, 0x42, 0x51, 0x29, 0xee,
-	0xa1, 0x24, 0x89, 0x91, 0xea, 0x60, 0x7c, 0x9b, 0x24, 0x54, 0x06, 0xf1, 0x4d, 0x06, 0x7a, 0x0d,
-	0xd7, 0x54, 0xe3, 0x72, 0x5c, 0x97, 0x71, 0x52, 0xe0, 0xcf, 0x34, 0xa8, 0x44, 0x0b, 0x7c, 0xe8,
-	0x66, 0x82, 0xe8, 0x78, 0x9d, 0x50, 0xbf, 0x35, 0x9a, 0x29, 0x55, 0x05, 0x8e, 0x7f, 0x4c, 0x48,
-	0xcf, 0xa4, 0x9c, 0xc2, 0xf6, 0xe8, 0x7b, 0x1a, 0xcc, 0xc4, 0xca, 0x76, 0x28, 0x09, 0x62, 0xa8,
-	0x28, 0xa8, 0xdf, 0x3e, 0x83, 0x4b, 0x68, 0x72, 0x97, 0x69, 0x72, 0x03, 0x5f, 0x1d, 0x36, 0x86,
-	0x6f, 0x75, 0x89, 0xef, 0x08, 0x6d, 0x56, 0xff, 0x91, 0x85, 0xfc, 0x3a, 0xff, 0xf1, 0x17, 0xf2,
-	0xa1, 0x10, 0x54, 0xc2, 0xd0, 0x42, 0x52, 0x55, 0x22, 0x3c, 0xb2, 0xeb, 0xd7, 0x53, 0xfb, 0x85,
-	0x0a, 0x77, 0x98, 0x0a, 0x8b, 0xf8, 0x4a, 0xa0, 0x82, 0xf8, 0x91, 0xd9, 0x0a, 0xbf, 0x7c, 0xaf,
-	0x98, 0xad, 0x16, 0x5d, 0x92, 0xff, 0xd3, 0xa0, 0xa4, 0x16, 0xac, 0xd0, 0x8d, 0xc4, 0x7a, 0x88,
-	0x5a, 0xf3, 0xd2, 0xf1, 0x28, 0x16, 0x81, 0x7f, 0x8f, 0xe1, 0xdf, 0xc4, 0x0b, 0x69, 0xf8, 0x2e,
-	0xe3, 0x8f, 0xaa, 0xc0, 0x4b, 0x4e, 0xc9, 0x2a, 0x44, 0x2a, 0x5a, 0xc9, 0x2a, 0x44, 0x2b, 0x56,
-	0x67, 0xab, 0xd0, 0x67, 0xfc, 0x54, 0x85, 0x13, 0x80, 0xb0, 0xc2, 0x84, 0x12, 0x8d, 0xab, 0x5c,
-	0x62, 0xe2, 0x3e, 0x38, 0x5c, 0x9c, 0x4a, 0xd8, 0x01, 0x31, 0xec, 0x8e, 0xe5, 0x51, 0x5f, 0x5c,
-	0xfd, 0xed, 0x24, 0x14, 0x9f, 0x98, 0x96, 0xed, 0x13, 0xdb, 0xb4, 0x9b, 0x04, 0xb5, 0x61, 0x8a,
-	0x65, 0xa9, 0x78, 0xe0, 0x51, 0xcb, 0x3e, 0xf1, 0xc0, 0x13, 0xa9, 0x89, 0xe0, 0xdb, 0x0c, 0xfa,
-	0x3a, 0xd6, 0x03, 0xe8, 0x6e, 0x28, 0x7f, 0x85, 0xd5, 0x33, 0xe8, 0x94, 0x8f, 0x21, 0xc7, 0xeb,
-	0x17, 0x28, 0x26, 0x2d, 0x52, 0xe7, 0xd0, 0xaf, 0x26, 0x77, 0xa6, 0xee, 0x32, 0x15, 0xcb, 0x63,
-	0xcc, 0x14, 0xec, 0x7f, 0x00, 0xc2, 0x82, 0x59, 0xdc, 0xbe, 0x43, 0xf5, 0x35, 0x7d, 0x31, 0x9d,
-	0x41, 0x00, 0xdf, 0x67, 0xc0, 0xb7, 0xf0, 0xf5, 0x44, 0xe0, 0x56, 0x30, 0x80, 0x82, 0x37, 0x61,
-	0x72, 0xd3, 0xf4, 0x8e, 0x50, 0x2c, 0x09, 0x29, 0x6f, 0xbb, 0xba, 0x9e, 0xd4, 0x25, 0xa0, 0x6e,
-	0x31, 0xa8, 0x05, 0x3c, 0x9f, 0x08, 0x75, 0x64, 0x7a, 0x34, 0xa6, 0xa3, 0x3e, 0x4c, 0xcb, 0xf7,
-	0x5a, 0x74, 0x2d, 0x66, 0xb3, 0xe8, 0xdb, 0xae, 0xbe, 0x90, 0xd6, 0x2d, 0x00, 0x97, 0x18, 0x20,
-	0xc6, 0xd7, 0x92, 0x8d, 0x2a, 0xd8, 0x1f, 0x6a, 0xf7, 0x5f, 0xd3, 0x56, 0x7f, 0x50, 0x85, 0x49,
-	0x7a, 0x5e, 0xa2, 0x59, 0x24, 0xbc, 0x66, 0xc6, 0x2d, 0x3c, 0x54, 0xdc, 0x89, 0x5b, 0x78, 0xf8,
-	0x86, 0x9a, 0x90, 0x45, 0xd8, 0x4f, 0x60, 0x09, 0xe3, 0xa2, 0x33, 0xf6, 0xa1, 0xa8, 0x5c, 0x46,
-	0x51, 0x82, 0xc4, 0x68, 0xe9, 0x28, 0x9e, 0x45, 0x12, 0x6e, 0xb2, 0x78, 0x91, 0x81, 0xea, 0xf8,
-	0x62, 0x14, 0xb4, 0xc5, 0xd9, 0x28, 0xea, 0x27, 0x50, 0x52, 0x6f, 0xad, 0x28, 0x41, 0x68, 0xac,
-	0x36, 0x15, 0x8f, 0x15, 0x49, 0x97, 0xde, 0x04, 0xa7, 0x09, 0x7e, 0xf0, 0x2b, 0x79, 0x29, 0xfa,
-	0x47, 0x90, 0x17, 0x77, 0xd9, 0xa4, 0xf9, 0x46, 0xab, 0x59, 0x49, 0xf3, 0x8d, 0x5d, 0x84, 0x13,
-	0x8e, 0x24, 0x0c, 0x96, 0x9e, 0xd9, 0x65, 0x80, 0x16, 0x90, 0x8f, 0x89, 0x9f, 0x06, 0x19, 0xd6,
-	0x67, 0xd2, 0x20, 0x95, 0xfb, 0xd2, 0x48, 0xc8, 0x36, 0xf1, 0xc5, 0x5e, 0x96, 0x97, 0x11, 0x94,
-	0x22, 0x51, 0x8d, 0x86, 0x78, 0x14, 0x4b, 0xea, 0x29, 0x32, 0x44, 0x15, 0xa1, 0x10, 0xfd, 0x2f,
-	0x40, 0x78, 0xf1, 0x8e, 0x1f, 0x0c, 0x12, 0xab, 0x77, 0xf1, 0x83, 0x41, 0xf2, 0xdd, 0x3d, 0xc1,
-	0x83, 0x43, 0x70, 0x7e, 0x92, 0xa5, 0xf0, 0x3f, 0xd6, 0x00, 0x0d, 0x5f, 0xd4, 0xd1, 0x83, 0x64,
-	0x88, 0xc4, 0xc2, 0xa0, 0xfe, 0xca, 0xf9, 0x98, 0x53, 0xa3, 0x67, 0xa8, 0x57, 0x93, 0x0d, 0xe9,
-	0xbd, 0xa4, 0x9a, 0x7d, 0xae, 0x41, 0x39, 0x72, 0xd5, 0x47, 0x77, 0x52, 0xd6, 0x39, 0x56, 0x5c,
-	0xd4, 0xef, 0x9e, 0xc9, 0x97, 0x7a, 0x76, 0x52, 0x76, 0x85, 0x3c, 0x37, 0x7e, 0x5f, 0x83, 0x4a,
-	0xb4, 0x3e, 0x80, 0x52, 0x00, 0x86, 0x2a, 0x94, 0xfa, 0xd2, 0xd9, 0x8c, 0xe7, 0x58, 0xad, 0xf0,
-	0x28, 0xf9, 0x11, 0xe4, 0x45, 0x59, 0x21, 0xc9, 0x2d, 0xa2, 0x05, 0xce, 0x24, 0xb7, 0x88, 0xd5,
-	0x24, 0xd2, 0xdc, 0x82, 0xde, 0xd0, 0x15, 0x4f, 0x14, 0xc5, 0x87, 0x34, 0xc8, 0xd1, 0x9e, 0x18,
-	0xab, 0x5c, 0x8c, 0x84, 0x0c, 0x3d, 0x51, 0x96, 0x1e, 0x50, 0x8a, 0xc4, 0x33, 0x3c, 0x31, 0x5e,
-	0xb9, 0x48, 0xf3, 0x44, 0x86, 0xaa, 0x78, 0x62, 0x58, 0x29, 0x48, 0xf2, 0xc4, 0xa1, 0xf2, 0x6d,
-	0x92, 0x27, 0x0e, 0x17, 0x1b, 0xd2, 0xd6, 0x96, 0x81, 0x47, 0x3c, 0x71, 0x36, 0xa1, 0xb2, 0x80,
-	0x5e, 0x49, 0xb1, 0x69, 0x62, 0x69, 0x58, 0x7f, 0xf5, 0x9c, 0xdc, 0xa3, 0x3d, 0x80, 0xaf, 0x86,
-	0xf4, 0x80, 0x9f, 0x6b, 0x30, 0x97, 0x54, 0x9a, 0x40, 0x29, 0x60, 0x29, 0x75, 0x65, 0x7d, 0xf9,
-	0xbc, 0xec, 0xe7, 0xb0, 0x5b, 0xe0, 0x13, 0x8f, 0xaa, 0xbf, 0xff, 0x6a, 0x41, 0xfb, 0xd3, 0x57,
-	0x0b, 0xda, 0x5f, 0xbe, 0x5a, 0xd0, 0x7e, 0xf2, 0xd7, 0x85, 0x89, 0xc3, 0x1c, 0xfb, 0x7f, 0x28,
-	0x6f, 0xfc, 0x33, 0x00, 0x00, 0xff, 0xff, 0xd1, 0x0e, 0xf0, 0x0b, 0x0e, 0x33, 0x00, 0x00,
+	// 3549 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5b, 0x5f, 0x6f, 0x1b, 0xc7,
+	0xb5, 0xd7, 0x92, 0x22, 0x29, 0x1e, 0xfe, 0x11, 0x35, 0x92, 0x6d, 0x6a, 0x6d, 0xcb, 0xf2, 0xf8,
+	0x9f, 0x6c, 0xc7, 0x52, 0xa2, 0xe4, 0xde, 0x07, 0xdf, 0x20, 0xb8, 0xb2, 0xc4, 0x58, 0x8a, 0x64,
+	0xc9, 0x59, 0xc9, 0x4e, 0x2e, 0x10, 0x5c, 0x62, 0x45, 0x8e, 0xa5, 0x85, 0xc8, 0x5d, 0x66, 0x77,
+	0x49, 0x4b, 0x69, 0x0a, 0x14, 0x69, 0x82, 0xa2, 0x05, 0xfa, 0xd2, 0x3c, 0xf4, 0xdf, 0x63, 0x51,
+	0x14, 0xf9, 0x00, 0x45, 0x3f, 0x40, 0x81, 0xa2, 0xe8, 0x4b, 0x0b, 0xf4, 0x0b, 0x14, 0x69, 0xbf,
+	0x46, 0xd1, 0x62, 0xfe, 0xed, 0xce, 0x2e, 0x77, 0x25, 0x25, 0x6c, 0xf2, 0x62, 0xed, 0x9c, 0x39,
+	0x73, 0x7e, 0x67, 0xce, 0xcc, 0x39, 0x67, 0xe6, 0x0c, 0x0d, 0x45, 0xb7, 0xd7, 0x5a, 0xec, 0xb9,
+	0x8e, 0xef, 0xa0, 0x32, 0xf1, 0x5b, 0x6d, 0x8f, 0xb8, 0x03, 0xe2, 0xf6, 0xf6, 0xf5, 0x99, 0x03,
+	0xe7, 0xc0, 0x61, 0x1d, 0x4b, 0xf4, 0x8b, 0xf3, 0xe8, 0xb3, 0x94, 0x67, 0xa9, 0x3b, 0x68, 0xb5,
+	0xd8, 0x3f, 0xbd, 0xfd, 0xa5, 0xa3, 0x81, 0xe8, 0xba, 0xcc, 0xba, 0xcc, 0xbe, 0x7f, 0xc8, 0xfe,
+	0xe9, 0xed, 0xb3, 0x3f, 0xa2, 0xf3, 0xca, 0x81, 0xe3, 0x1c, 0x74, 0xc8, 0x92, 0xd9, 0xb3, 0x96,
+	0x4c, 0xdb, 0x76, 0x7c, 0xd3, 0xb7, 0x1c, 0xdb, 0xe3, 0xbd, 0xf8, 0x33, 0x0d, 0xaa, 0x06, 0xf1,
+	0x7a, 0x8e, 0xed, 0x91, 0x75, 0x62, 0xb6, 0x89, 0x8b, 0xae, 0x02, 0xb4, 0x3a, 0x7d, 0xcf, 0x27,
+	0x6e, 0xd3, 0x6a, 0xd7, 0xb5, 0x79, 0x6d, 0x61, 0xdc, 0x28, 0x0a, 0xca, 0x46, 0x1b, 0x5d, 0x86,
+	0x62, 0x97, 0x74, 0xf7, 0x79, 0x6f, 0x86, 0xf5, 0x4e, 0x70, 0xc2, 0x46, 0x1b, 0xe9, 0x30, 0xe1,
+	0x92, 0x81, 0xe5, 0x59, 0x8e, 0x5d, 0xcf, 0xce, 0x6b, 0x0b, 0x59, 0x23, 0x68, 0xd3, 0x81, 0xae,
+	0xf9, 0xc2, 0x6f, 0xfa, 0xc4, 0xed, 0xd6, 0xc7, 0xf9, 0x40, 0x4a, 0xd8, 0x23, 0x6e, 0x17, 0x7f,
+	0x9a, 0x83, 0xb2, 0x61, 0xda, 0x07, 0xc4, 0x20, 0x1f, 0xf6, 0x89, 0xe7, 0xa3, 0x1a, 0x64, 0x8f,
+	0xc8, 0x09, 0x83, 0x2f, 0x1b, 0xf4, 0x93, 0x8f, 0xb7, 0x0f, 0x48, 0x93, 0xd8, 0x1c, 0xb8, 0x4c,
+	0xc7, 0xdb, 0x07, 0xa4, 0x61, 0xb7, 0xd1, 0x0c, 0xe4, 0x3a, 0x56, 0xd7, 0xf2, 0x05, 0x2a, 0x6f,
+	0x44, 0xd4, 0x19, 0x8f, 0xa9, 0xb3, 0x0a, 0xe0, 0x39, 0xae, 0xdf, 0x74, 0xdc, 0x36, 0x71, 0xeb,
+	0xb9, 0x79, 0x6d, 0xa1, 0xba, 0x7c, 0x73, 0x51, 0x5d, 0x88, 0x45, 0x55, 0xa1, 0xc5, 0x5d, 0xc7,
+	0xf5, 0x77, 0x28, 0xaf, 0x51, 0xf4, 0xe4, 0x27, 0x7a, 0x1b, 0x4a, 0x4c, 0x88, 0x6f, 0xba, 0x07,
+	0xc4, 0xaf, 0xe7, 0x99, 0x94, 0x5b, 0x67, 0x48, 0xd9, 0x63, 0xcc, 0x06, 0x83, 0xe7, 0xdf, 0x08,
+	0x43, 0xd9, 0x23, 0xae, 0x65, 0x76, 0xac, 0x8f, 0xcc, 0xfd, 0x0e, 0xa9, 0x17, 0xe6, 0xb5, 0x85,
+	0x09, 0x23, 0x42, 0xa3, 0xf3, 0x3f, 0x22, 0x27, 0x5e, 0xd3, 0xb1, 0x3b, 0x27, 0xf5, 0x09, 0xc6,
+	0x30, 0x41, 0x09, 0x3b, 0x76, 0xe7, 0x84, 0x2d, 0x9a, 0xd3, 0xb7, 0x7d, 0xde, 0x5b, 0x64, 0xbd,
+	0x45, 0x46, 0x61, 0xdd, 0x0b, 0x50, 0xeb, 0x5a, 0x76, 0xb3, 0xeb, 0xb4, 0x9b, 0x81, 0x41, 0x80,
+	0x19, 0xa4, 0xda, 0xb5, 0xec, 0x27, 0x4e, 0xdb, 0x90, 0x66, 0xa1, 0x9c, 0xe6, 0x71, 0x94, 0xb3,
+	0x24, 0x38, 0xcd, 0x63, 0x95, 0x73, 0x11, 0xa6, 0xa9, 0xcc, 0x96, 0x4b, 0x4c, 0x9f, 0x84, 0xcc,
+	0x65, 0xc6, 0x3c, 0xd5, 0xb5, 0xec, 0x55, 0xd6, 0x13, 0xe1, 0x37, 0x8f, 0x87, 0xf8, 0x2b, 0x82,
+	0xdf, 0x3c, 0x8e, 0xf2, 0xe3, 0x45, 0x28, 0x06, 0x36, 0x47, 0x13, 0x30, 0xbe, 0xbd, 0xb3, 0xdd,
+	0xa8, 0x8d, 0x21, 0x80, 0xfc, 0xca, 0xee, 0x6a, 0x63, 0x7b, 0xad, 0xa6, 0xa1, 0x12, 0x14, 0xd6,
+	0x1a, 0xbc, 0x91, 0xc1, 0x8f, 0x00, 0x42, 0xeb, 0xa2, 0x02, 0x64, 0x37, 0x1b, 0xff, 0x57, 0x1b,
+	0xa3, 0x3c, 0xcf, 0x1b, 0xc6, 0xee, 0xc6, 0xce, 0x76, 0x4d, 0xa3, 0x83, 0x57, 0x8d, 0xc6, 0xca,
+	0x5e, 0xa3, 0x96, 0xa1, 0x1c, 0x4f, 0x76, 0xd6, 0x6a, 0x59, 0x54, 0x84, 0xdc, 0xf3, 0x95, 0xad,
+	0x67, 0x8d, 0xda, 0x38, 0xfe, 0x5c, 0x83, 0x8a, 0x58, 0x2f, 0xee, 0x13, 0xe8, 0x0d, 0xc8, 0x1f,
+	0x32, 0xbf, 0x60, 0x5b, 0xb1, 0xb4, 0x7c, 0x25, 0xb6, 0xb8, 0x11, 0xdf, 0x31, 0x04, 0x2f, 0xc2,
+	0x90, 0x3d, 0x1a, 0x78, 0xf5, 0xcc, 0x7c, 0x76, 0xa1, 0xb4, 0x5c, 0x5b, 0xe4, 0x0e, 0xbb, 0xb8,
+	0x49, 0x4e, 0x9e, 0x9b, 0x9d, 0x3e, 0x31, 0x68, 0x27, 0x42, 0x30, 0xde, 0x75, 0x5c, 0xc2, 0x76,
+	0xec, 0x84, 0xc1, 0xbe, 0xe9, 0x36, 0x66, 0x8b, 0x26, 0x76, 0x2b, 0x6f, 0xe0, 0x2f, 0x34, 0x80,
+	0xa7, 0x7d, 0x3f, 0xdd, 0x35, 0x66, 0x20, 0x37, 0xa0, 0x82, 0x85, 0x5b, 0xf0, 0x06, 0xf3, 0x09,
+	0x62, 0x7a, 0x24, 0xf0, 0x09, 0xda, 0x40, 0x97, 0xa0, 0xd0, 0x73, 0xc9, 0xa0, 0x79, 0x34, 0x60,
+	0x20, 0x13, 0x46, 0x9e, 0x36, 0x37, 0x07, 0xe8, 0x3a, 0x94, 0xad, 0x03, 0xdb, 0x71, 0x49, 0x93,
+	0xcb, 0xca, 0xb1, 0xde, 0x12, 0xa7, 0x31, 0xbd, 0x15, 0x16, 0x2e, 0x38, 0xaf, 0xb2, 0x6c, 0x51,
+	0x12, 0xb6, 0xa1, 0xc4, 0x54, 0x1d, 0xc9, 0x7c, 0x77, 0x43, 0x1d, 0x33, 0x6c, 0xd8, 0xb0, 0x09,
+	0x85, 0xd6, 0xf8, 0x03, 0x40, 0x6b, 0xa4, 0x43, 0x7c, 0x32, 0x4a, 0xf4, 0x50, 0x6c, 0x92, 0x55,
+	0x6d, 0x82, 0x7f, 0xa2, 0xc1, 0x74, 0x44, 0xfc, 0x48, 0xd3, 0xaa, 0x43, 0xa1, 0xcd, 0x84, 0x71,
+	0x0d, 0xb2, 0x86, 0x6c, 0xa2, 0xfb, 0x30, 0x21, 0x14, 0xf0, 0xea, 0xd9, 0x94, 0x4d, 0x53, 0xe0,
+	0x3a, 0x79, 0xf8, 0x8b, 0x0c, 0x14, 0xc5, 0x44, 0x77, 0x7a, 0x68, 0x05, 0x2a, 0x2e, 0x6f, 0x34,
+	0xd9, 0x7c, 0x84, 0x46, 0x7a, 0x7a, 0x10, 0x5a, 0x1f, 0x33, 0xca, 0x62, 0x08, 0x23, 0xa3, 0xff,
+	0x81, 0x92, 0x14, 0xd1, 0xeb, 0xfb, 0xc2, 0xe4, 0xf5, 0xa8, 0x80, 0x70, 0xff, 0xad, 0x8f, 0x19,
+	0x20, 0xd8, 0x9f, 0xf6, 0x7d, 0xb4, 0x07, 0x33, 0x72, 0x30, 0x9f, 0x8d, 0x50, 0x23, 0xcb, 0xa4,
+	0xcc, 0x47, 0xa5, 0x0c, 0x2f, 0xd5, 0xfa, 0x98, 0x81, 0xc4, 0x78, 0xa5, 0x53, 0x55, 0xc9, 0x3f,
+	0xe6, 0xc1, 0x7b, 0x48, 0xa5, 0xbd, 0x63, 0x7b, 0x58, 0xa5, 0xbd, 0x63, 0xfb, 0x51, 0x11, 0x0a,
+	0xa2, 0x85, 0x7f, 0x97, 0x01, 0x90, 0xab, 0xb1, 0xd3, 0x43, 0x6b, 0x50, 0x75, 0x45, 0x2b, 0x62,
+	0xad, 0xcb, 0x89, 0xd6, 0x12, 0x8b, 0x38, 0x66, 0x54, 0xe4, 0x20, 0xae, 0xdc, 0x5b, 0x50, 0x0e,
+	0xa4, 0x84, 0x06, 0x9b, 0x4d, 0x30, 0x58, 0x20, 0xa1, 0x24, 0x07, 0x50, 0x93, 0xbd, 0x07, 0x17,
+	0x82, 0xf1, 0x09, 0x36, 0xbb, 0x7e, 0x8a, 0xcd, 0x02, 0x81, 0xd3, 0x52, 0x82, 0x6a, 0x35, 0x55,
+	0xb1, 0xd0, 0x6c, 0xb3, 0x09, 0x66, 0x1b, 0x56, 0x8c, 0x1a, 0x0e, 0x68, 0xbe, 0xe4, 0x4d, 0xfc,
+	0x87, 0x2c, 0x14, 0x56, 0x9d, 0x6e, 0xcf, 0x74, 0xe9, 0x6a, 0xe4, 0x5d, 0xe2, 0xf5, 0x3b, 0x3e,
+	0x33, 0x57, 0x75, 0xf9, 0x46, 0x54, 0xa2, 0x60, 0x93, 0x7f, 0x0d, 0xc6, 0x6a, 0x88, 0x21, 0x74,
+	0xb0, 0x48, 0x8f, 0x99, 0x73, 0x0c, 0x16, 0xc9, 0x51, 0x0c, 0x91, 0x8e, 0x9c, 0x0d, 0x1d, 0x59,
+	0x87, 0xc2, 0x80, 0xb8, 0x61, 0x4a, 0x5f, 0x1f, 0x33, 0x24, 0x01, 0xdd, 0x85, 0xc9, 0x78, 0x7a,
+	0xc9, 0x09, 0x9e, 0x6a, 0x2b, 0x9a, 0x8d, 0x6e, 0x40, 0x39, 0x92, 0xe3, 0xf2, 0x82, 0xaf, 0xd4,
+	0x55, 0x52, 0xdc, 0x45, 0x19, 0x57, 0x69, 0x3e, 0x2e, 0xaf, 0x8f, 0xc9, 0xc8, 0x1a, 0x09, 0x26,
+	0x13, 0xd1, 0x60, 0x82, 0xff, 0x17, 0x2a, 0x11, 0x43, 0xd0, 0xfc, 0xd2, 0x78, 0xf7, 0xd9, 0xca,
+	0x16, 0x4f, 0x46, 0x8f, 0x59, 0xfe, 0x31, 0x6a, 0x1a, 0xcd, 0x69, 0x5b, 0x8d, 0xdd, 0xdd, 0x5a,
+	0x06, 0x55, 0xa0, 0xb8, 0xbd, 0xb3, 0xd7, 0xe4, 0x5c, 0x59, 0xfc, 0x66, 0x20, 0x41, 0x24, 0x33,
+	0x25, 0x87, 0x8d, 0x29, 0x39, 0x4c, 0x93, 0x39, 0x2c, 0x13, 0xe6, 0xb0, 0xec, 0xa3, 0x2a, 0x94,
+	0xb9, 0xf1, 0x9a, 0x7d, 0x9b, 0xe6, 0xd1, 0x5f, 0x69, 0x00, 0xa1, 0xab, 0xa0, 0x25, 0x28, 0xb4,
+	0xb8, 0xf0, 0xba, 0xc6, 0x22, 0xcd, 0x85, 0xc4, 0xf5, 0x30, 0x24, 0x17, 0x7a, 0x0d, 0x0a, 0x5e,
+	0xbf, 0xd5, 0x22, 0x9e, 0xcc, 0x67, 0x97, 0xe2, 0xc1, 0x4e, 0x84, 0x22, 0x43, 0xf2, 0xd1, 0x21,
+	0x2f, 0x4c, 0xab, 0xd3, 0x67, 0xd9, 0xed, 0xf4, 0x21, 0x82, 0x0f, 0xff, 0x5c, 0x83, 0x92, 0xb2,
+	0x33, 0xbf, 0x66, 0x84, 0xbd, 0x02, 0x45, 0xa6, 0x03, 0x69, 0x8b, 0x18, 0x3b, 0x61, 0x84, 0x04,
+	0xf4, 0xdf, 0x50, 0x94, 0xdb, 0x5b, 0x86, 0xd9, 0x7a, 0xb2, 0xd8, 0x9d, 0x9e, 0x11, 0xb2, 0xe2,
+	0x4d, 0x98, 0x62, 0x56, 0x69, 0xd1, 0x93, 0xb3, 0xb4, 0xa3, 0x7a, 0xb6, 0xd4, 0x62, 0x67, 0x4b,
+	0x1d, 0x26, 0x7a, 0x87, 0x27, 0x9e, 0xd5, 0x32, 0x3b, 0x42, 0x8b, 0xa0, 0x8d, 0xdf, 0x01, 0xa4,
+	0x0a, 0x1b, 0x65, 0xba, 0xb8, 0x02, 0xa5, 0x75, 0xd3, 0x3b, 0x14, 0x2a, 0xe1, 0xf7, 0xa1, 0xcc,
+	0x9b, 0x23, 0xd9, 0x10, 0xc1, 0xf8, 0xa1, 0xe9, 0x1d, 0x32, 0xc5, 0x2b, 0x06, 0xfb, 0xc6, 0x53,
+	0x30, 0xb9, 0x6b, 0x9b, 0x3d, 0xef, 0xd0, 0x91, 0x59, 0x80, 0xde, 0x1c, 0x6a, 0x21, 0x6d, 0x24,
+	0xc4, 0x3b, 0x30, 0xe9, 0x92, 0xae, 0x69, 0xd9, 0x96, 0x7d, 0xd0, 0xdc, 0x3f, 0xf1, 0x89, 0x27,
+	0x2e, 0x16, 0xd5, 0x80, 0xfc, 0x88, 0x52, 0xa9, 0x6a, 0xfb, 0x1d, 0x67, 0x5f, 0x84, 0x03, 0xf6,
+	0x8d, 0x7f, 0xab, 0x41, 0xf9, 0x3d, 0xd3, 0x6f, 0x49, 0x2b, 0xa0, 0x0d, 0xa8, 0x06, 0x41, 0x80,
+	0x51, 0x84, 0x2e, 0xb1, 0x54, 0xc4, 0xc6, 0xc8, 0x23, 0xa7, 0xcc, 0x22, 0x95, 0x96, 0x4a, 0x60,
+	0xa2, 0x4c, 0xbb, 0x45, 0x3a, 0x81, 0xa8, 0x4c, 0xba, 0x28, 0xc6, 0xa8, 0x8a, 0x52, 0x09, 0x8f,
+	0x26, 0xc3, 0x34, 0xcd, 0xdd, 0xf2, 0x17, 0x19, 0x40, 0xc3, 0x3a, 0x7c, 0xd5, 0x93, 0xcb, 0x2d,
+	0xa8, 0x7a, 0xbe, 0xe9, 0xfa, 0xcd, 0xd8, 0xb5, 0xab, 0xc2, 0xa8, 0x41, 0x20, 0xbb, 0x03, 0x93,
+	0x3d, 0xd7, 0x39, 0x70, 0x89, 0xe7, 0x35, 0x6d, 0xc7, 0xb7, 0x5e, 0x9c, 0x88, 0xc3, 0x5f, 0x55,
+	0x92, 0xb7, 0x19, 0x15, 0x35, 0xa0, 0xf0, 0xc2, 0xea, 0xf8, 0xc4, 0xf5, 0xea, 0xb9, 0xf9, 0xec,
+	0x42, 0x75, 0xf9, 0xfe, 0x59, 0x56, 0x5b, 0x7c, 0x9b, 0xf1, 0xef, 0x9d, 0xf4, 0x88, 0x21, 0xc7,
+	0xaa, 0x07, 0xaa, 0x7c, 0xe4, 0x40, 0x75, 0x0b, 0x20, 0xe4, 0xa7, 0x51, 0x6b, 0x7b, 0xe7, 0xe9,
+	0xb3, 0xbd, 0xda, 0x18, 0x2a, 0xc3, 0xc4, 0xf6, 0xce, 0x5a, 0x63, 0xab, 0x41, 0xe3, 0x1a, 0x5e,
+	0x92, 0xb6, 0x51, 0x6d, 0x88, 0x66, 0x61, 0xe2, 0x25, 0xa5, 0xca, 0x7b, 0x69, 0xd6, 0x28, 0xb0,
+	0xf6, 0x46, 0x1b, 0xff, 0x38, 0x03, 0x15, 0xb1, 0x0b, 0x46, 0xda, 0x8a, 0x2a, 0x44, 0x26, 0x02,
+	0x41, 0x4f, 0x6f, 0x7c, 0x77, 0xb4, 0xc5, 0x21, 0x51, 0x36, 0xa9, 0xbb, 0xf3, 0xc5, 0x26, 0x6d,
+	0x61, 0xd6, 0xa0, 0x8d, 0xee, 0x42, 0xad, 0xc5, 0xdd, 0x3d, 0x96, 0x93, 0x8c, 0x49, 0x41, 0x57,
+	0x52, 0x52, 0x25, 0xd8, 0x6d, 0xa6, 0x27, 0x72, 0x52, 0xd1, 0x28, 0xcb, 0x8d, 0x44, 0x69, 0xe8,
+	0x16, 0xe4, 0xc9, 0x80, 0xd8, 0xbe, 0x57, 0x2f, 0xb1, 0x00, 0x56, 0x91, 0xe7, 0xc4, 0x06, 0xa5,
+	0x1a, 0xa2, 0x13, 0xff, 0x17, 0x4c, 0xb1, 0xf3, 0xf8, 0x63, 0xd7, 0xb4, 0xd5, 0x8b, 0xc3, 0xde,
+	0xde, 0x96, 0x30, 0x1d, 0xfd, 0x44, 0x55, 0xc8, 0x6c, 0xac, 0x89, 0x89, 0x66, 0x36, 0xd6, 0xf0,
+	0x27, 0x1a, 0x20, 0x75, 0xdc, 0x48, 0xb6, 0x8c, 0x09, 0x97, 0xf0, 0xd9, 0x10, 0x7e, 0x06, 0x72,
+	0xc4, 0x75, 0x1d, 0x97, 0x59, 0xad, 0x68, 0xf0, 0x06, 0xbe, 0x29, 0x74, 0x30, 0xc8, 0xc0, 0x39,
+	0x0a, 0x1c, 0x83, 0x4b, 0xd3, 0x02, 0x55, 0x37, 0x61, 0x3a, 0xc2, 0x35, 0x52, 0x20, 0xbd, 0x03,
+	0x17, 0x98, 0xb0, 0x4d, 0x42, 0x7a, 0x2b, 0x1d, 0x6b, 0x90, 0x8a, 0xda, 0x83, 0x8b, 0x71, 0xc6,
+	0x6f, 0xd6, 0x46, 0xf8, 0x4d, 0x81, 0xb8, 0x67, 0x75, 0xc9, 0x9e, 0xb3, 0x95, 0xae, 0x1b, 0x8d,
+	0x8e, 0x47, 0xe4, 0xc4, 0x13, 0x19, 0x87, 0x7d, 0xe3, 0x5f, 0x6b, 0x70, 0x69, 0x68, 0xf8, 0x37,
+	0xbc, 0xaa, 0x73, 0x00, 0x07, 0x74, 0xfb, 0x90, 0x36, 0xed, 0xe0, 0x37, 0x59, 0x85, 0x12, 0xe8,
+	0x49, 0x03, 0x4c, 0x59, 0xe8, 0x79, 0x08, 0xf9, 0x27, 0xac, 0x88, 0xa4, 0xcc, 0x6a, 0x5c, 0xce,
+	0xca, 0x36, 0xbb, 0xfc, 0x6a, 0x5b, 0x34, 0xd8, 0x37, 0xcb, 0xaf, 0x84, 0xb8, 0xcf, 0x8c, 0x2d,
+	0x9e, 0xc7, 0x8b, 0x46, 0xd0, 0xa6, 0xe8, 0xad, 0x8e, 0x45, 0x6c, 0x9f, 0xf5, 0x8e, 0xb3, 0x5e,
+	0x85, 0x82, 0x17, 0xa1, 0xc6, 0x91, 0x56, 0xda, 0x6d, 0x25, 0x97, 0x07, 0xf2, 0xb4, 0xa8, 0x3c,
+	0xfc, 0x1b, 0x0d, 0xa6, 0x94, 0x01, 0x23, 0xd9, 0xee, 0x15, 0xc8, 0xf3, 0x52, 0x99, 0xc8, 0x23,
+	0x33, 0xd1, 0x51, 0x1c, 0xc6, 0x10, 0x3c, 0x68, 0x11, 0x0a, 0xfc, 0x4b, 0x1e, 0x56, 0x92, 0xd9,
+	0x25, 0x13, 0xbe, 0x05, 0xd3, 0x82, 0x44, 0xba, 0x4e, 0xd2, 0x36, 0x61, 0x06, 0xc5, 0x1f, 0xc3,
+	0x4c, 0x94, 0x6d, 0xa4, 0x29, 0x29, 0x4a, 0x66, 0xce, 0xa3, 0xe4, 0x8a, 0x54, 0xf2, 0x59, 0xaf,
+	0xad, 0xa4, 0xbd, 0xf8, 0xaa, 0xab, 0x2b, 0x92, 0x89, 0xad, 0x48, 0x30, 0x01, 0x29, 0xe2, 0x5b,
+	0x9d, 0xc0, 0xb4, 0xdc, 0x0e, 0x5b, 0x96, 0x17, 0x1c, 0x86, 0x3e, 0x02, 0xa4, 0x12, 0xbf, 0x6d,
+	0x85, 0xd6, 0xc8, 0x0b, 0xd7, 0x3c, 0xe8, 0x92, 0x20, 0xd4, 0xd3, 0x53, 0xa6, 0x4a, 0x1c, 0x29,
+	0x38, 0x2e, 0xc1, 0xd4, 0x13, 0x67, 0x40, 0xb6, 0x38, 0x35, 0x74, 0x19, 0x7e, 0xcb, 0x08, 0x96,
+	0x2d, 0x68, 0x53, 0x70, 0x75, 0xc0, 0x48, 0xe0, 0x7f, 0xd6, 0xa0, 0xbc, 0xd2, 0x31, 0xdd, 0xae,
+	0x04, 0x7e, 0x0b, 0xf2, 0xfc, 0xec, 0x2c, 0xee, 0xa2, 0xb7, 0xa3, 0x62, 0x54, 0x5e, 0xde, 0x58,
+	0xe1, 0x27, 0x6d, 0x31, 0x8a, 0x2a, 0x2e, 0xca, 0xd5, 0x6b, 0xb1, 0xf2, 0xf5, 0x1a, 0x7a, 0x00,
+	0x39, 0x93, 0x0e, 0x61, 0xd1, 0xac, 0x1a, 0xbf, 0xb5, 0x30, 0x69, 0xec, 0x9c, 0xc3, 0xb9, 0xf0,
+	0x1b, 0x50, 0x52, 0x10, 0xe8, 0x65, 0xec, 0x71, 0x43, 0x9c, 0x65, 0x56, 0x56, 0xf7, 0x36, 0x9e,
+	0xf3, 0x3b, 0x5a, 0x15, 0x60, 0xad, 0x11, 0xb4, 0x33, 0xf8, 0x7d, 0x31, 0x4a, 0xc4, 0x3b, 0x55,
+	0x1f, 0x2d, 0x4d, 0x9f, 0xcc, 0xb9, 0xf4, 0x39, 0x86, 0x8a, 0x98, 0xfe, 0x48, 0x1b, 0xf0, 0x35,
+	0xc8, 0x33, 0x79, 0x72, 0xff, 0xcd, 0x26, 0xc0, 0xca, 0x50, 0xc5, 0x19, 0xf1, 0x24, 0x54, 0x76,
+	0x7d, 0xd3, 0xef, 0x7b, 0x72, 0xff, 0xfd, 0x49, 0x83, 0xaa, 0xa4, 0x8c, 0x5a, 0x33, 0x93, 0xd7,
+	0x7d, 0x9e, 0x01, 0x82, 0xcb, 0xfe, 0x45, 0xc8, 0xb7, 0xf7, 0x77, 0xad, 0x8f, 0x64, 0x7d, 0x53,
+	0xb4, 0x28, 0xbd, 0xc3, 0x71, 0xf8, 0x23, 0x83, 0x68, 0xd1, 0xbb, 0xa1, 0x6b, 0xbe, 0xf0, 0x37,
+	0xec, 0x36, 0x39, 0x66, 0x47, 0xb0, 0x71, 0x23, 0x24, 0xb0, 0xeb, 0x9c, 0x78, 0x8c, 0x60, 0xe7,
+	0x2e, 0xf5, 0x71, 0x62, 0x1a, 0xa6, 0x56, 0xfa, 0xfe, 0x61, 0xc3, 0x36, 0xf7, 0x3b, 0x32, 0x62,
+	0xe1, 0x19, 0x40, 0x94, 0xb8, 0x66, 0x79, 0x2a, 0xb5, 0x01, 0xd3, 0x94, 0x4a, 0x6c, 0xdf, 0x6a,
+	0x29, 0xe1, 0x4d, 0x26, 0x31, 0x2d, 0x96, 0xc4, 0x4c, 0xcf, 0x7b, 0xe9, 0xb8, 0x6d, 0x31, 0xb5,
+	0xa0, 0x8d, 0xd7, 0xb8, 0xf0, 0x67, 0x5e, 0x24, 0x4d, 0x7d, 0x55, 0x29, 0x0b, 0xa1, 0x94, 0xc7,
+	0xc4, 0x3f, 0x45, 0x0a, 0xbe, 0x0f, 0x17, 0x24, 0xa7, 0xa8, 0x27, 0x9d, 0xc2, 0xbc, 0x03, 0x57,
+	0x25, 0xf3, 0xea, 0x21, 0xbd, 0x88, 0x3c, 0x15, 0x80, 0x5f, 0x57, 0xcf, 0x47, 0x50, 0x0f, 0xf4,
+	0x64, 0xe7, 0x4e, 0xa7, 0xa3, 0x2a, 0xd0, 0xf7, 0xc4, 0x9e, 0x29, 0x1a, 0xec, 0x9b, 0xd2, 0x5c,
+	0xa7, 0x13, 0x1c, 0x09, 0xe8, 0x37, 0x5e, 0x85, 0x59, 0x29, 0x43, 0x9c, 0x08, 0xa3, 0x42, 0x86,
+	0x14, 0x4a, 0x12, 0x22, 0x0c, 0x46, 0x87, 0x9e, 0x6e, 0x76, 0x95, 0x33, 0x6a, 0x5a, 0x26, 0x53,
+	0x53, 0x64, 0x5e, 0xe0, 0x3b, 0x82, 0x2a, 0xa6, 0x66, 0x0c, 0x41, 0xa6, 0x02, 0x54, 0xb2, 0x58,
+	0x08, 0x4a, 0x1e, 0x5a, 0x88, 0x21, 0xd1, 0x1f, 0xc0, 0x5c, 0xa0, 0x04, 0xb5, 0xdb, 0x53, 0xe2,
+	0x76, 0x2d, 0xcf, 0x53, 0x8a, 0x14, 0x49, 0x13, 0xbf, 0x0d, 0xe3, 0x3d, 0x22, 0x62, 0x4a, 0x69,
+	0x19, 0x2d, 0xf2, 0x27, 0xc3, 0x45, 0x65, 0x30, 0xeb, 0xc7, 0x6d, 0xb8, 0x26, 0xa5, 0x73, 0x8b,
+	0x26, 0x8a, 0x8f, 0x2b, 0x25, 0x2f, 0xb0, 0xdc, 0xac, 0xc3, 0x17, 0xd8, 0x2c, 0x5f, 0xfb, 0xa0,
+	0x5a, 0xf6, 0x0e, 0x37, 0xa4, 0xf4, 0xad, 0x91, 0x72, 0xc5, 0x26, 0xb7, 0x69, 0xe0, 0x92, 0x23,
+	0x09, 0xdb, 0x87, 0x99, 0xa8, 0x27, 0x8f, 0x14, 0xc6, 0x66, 0x20, 0xe7, 0x3b, 0x47, 0x44, 0x06,
+	0x31, 0xde, 0x90, 0x0a, 0x07, 0x6e, 0x3e, 0x92, 0xc2, 0x66, 0x28, 0x8c, 0x6d, 0xc9, 0x51, 0xf5,
+	0xa5, 0xab, 0x29, 0x0f, 0x5f, 0xbc, 0x81, 0xb7, 0xe1, 0x62, 0x3c, 0x4c, 0x8c, 0xa4, 0xf2, 0x73,
+	0xbe, 0x81, 0x93, 0x22, 0xc9, 0x48, 0x72, 0xdf, 0x0d, 0x83, 0x81, 0x12, 0x50, 0x46, 0x12, 0x69,
+	0x80, 0x9e, 0x14, 0x5f, 0xfe, 0x13, 0xfb, 0x35, 0x08, 0x37, 0x23, 0x09, 0xf3, 0x42, 0x61, 0xa3,
+	0x2f, 0x7f, 0x18, 0x23, 0xb2, 0xa7, 0xc6, 0x08, 0xe1, 0x24, 0x61, 0x14, 0xfb, 0x06, 0x36, 0x9d,
+	0xc0, 0x08, 0x03, 0xe8, 0xa8, 0x18, 0x34, 0x87, 0x04, 0x18, 0xac, 0x21, 0x37, 0xb6, 0x1a, 0x76,
+	0x47, 0x5a, 0x8c, 0xf7, 0xc2, 0xd8, 0x39, 0x14, 0x99, 0x47, 0x12, 0xfc, 0x3e, 0xcc, 0xa7, 0x07,
+	0xe5, 0x51, 0x24, 0xdf, 0xc3, 0x50, 0x0c, 0x0e, 0x94, 0xca, 0x73, 0x7b, 0x09, 0x0a, 0xdb, 0x3b,
+	0xbb, 0x4f, 0x57, 0x56, 0x1b, 0x35, 0x6d, 0xf9, 0x9f, 0x59, 0xc8, 0x6c, 0x3e, 0x47, 0xff, 0x0f,
+	0x39, 0xfe, 0xde, 0x74, 0xca, 0x23, 0xa3, 0x7e, 0xda, 0x93, 0x1a, 0xbe, 0xf2, 0xc9, 0x5f, 0xff,
+	0xf1, 0x79, 0xe6, 0x22, 0x9e, 0x5a, 0x1a, 0xbc, 0x6e, 0x76, 0x7a, 0x87, 0xe6, 0xd2, 0xd1, 0x60,
+	0x89, 0xe5, 0x84, 0x87, 0xda, 0x3d, 0xf4, 0x1c, 0xb2, 0x4f, 0xfb, 0x3e, 0x4a, 0x7d, 0x81, 0xd4,
+	0xd3, 0x9f, 0xda, 0xb0, 0xce, 0x24, 0xcf, 0xe0, 0x49, 0x55, 0x72, 0xaf, 0xef, 0x53, 0xb9, 0x03,
+	0x28, 0xa9, 0xaf, 0x65, 0x67, 0xbe, 0x4d, 0xea, 0x67, 0xbf, 0xc4, 0x61, 0xcc, 0xf0, 0xae, 0xe0,
+	0x4b, 0x2a, 0x1e, 0x7f, 0xd4, 0x53, 0xe7, 0xb3, 0x77, 0x6c, 0xa3, 0xd4, 0xe7, 0x4b, 0x3d, 0xfd,
+	0x85, 0x2e, 0x79, 0x3e, 0xfe, 0xb1, 0x4d, 0xe5, 0x3a, 0xe2, 0x85, 0xae, 0xe5, 0xa3, 0x6b, 0x09,
+	0x8f, 0x38, 0xea, 0x73, 0x85, 0x3e, 0x9f, 0xce, 0x20, 0x90, 0xae, 0x33, 0xa4, 0xcb, 0xf8, 0xa2,
+	0x8a, 0xd4, 0x0a, 0xf8, 0x1e, 0x6a, 0xf7, 0x96, 0x0f, 0x21, 0xc7, 0x8a, 0xac, 0xa8, 0x29, 0x3f,
+	0xf4, 0x84, 0xf2, 0x70, 0xca, 0x0e, 0x88, 0x94, 0x67, 0xf1, 0x2c, 0x43, 0x9b, 0xc6, 0xd5, 0x00,
+	0x8d, 0xd5, 0x59, 0x1f, 0x6a, 0xf7, 0x16, 0xb4, 0x57, 0xb5, 0xe5, 0xef, 0x8f, 0x43, 0x8e, 0xd5,
+	0xad, 0x50, 0x0f, 0x20, 0xac, 0x48, 0xc6, 0xe7, 0x39, 0x54, 0xe3, 0x8c, 0xcf, 0x73, 0xb8, 0x98,
+	0x89, 0xaf, 0x31, 0xe4, 0x59, 0x3c, 0x13, 0x20, 0xb3, 0x1f, 0x33, 0x2c, 0xb1, 0x0a, 0x15, 0x35,
+	0xeb, 0x4b, 0x28, 0x29, 0x95, 0x45, 0x94, 0x24, 0x31, 0x52, 0x9a, 0x8c, 0x6f, 0x93, 0x84, 0xb2,
+	0x24, 0xbe, 0xc1, 0x40, 0xaf, 0xe2, 0xba, 0x6a, 0x5c, 0x8e, 0xeb, 0x32, 0x4e, 0x0a, 0xfc, 0xa9,
+	0x06, 0xd5, 0x68, 0x75, 0x11, 0xdd, 0x48, 0x10, 0x1d, 0x2f, 0x52, 0xea, 0x37, 0x4f, 0x67, 0x4a,
+	0x55, 0x81, 0xe3, 0x1f, 0x11, 0xd2, 0x33, 0x29, 0xa7, 0xb0, 0x3d, 0xfa, 0x81, 0x06, 0x93, 0xb1,
+	0x9a, 0x21, 0x4a, 0x82, 0x18, 0xaa, 0x48, 0xea, 0xb7, 0xce, 0xe0, 0x12, 0x9a, 0xdc, 0x61, 0x9a,
+	0x5c, 0xc7, 0x57, 0x86, 0x8d, 0xe1, 0x5b, 0x5d, 0xe2, 0x3b, 0x42, 0x9b, 0xe5, 0x7f, 0x65, 0xa1,
+	0xb0, 0xca, 0x7f, 0x79, 0x86, 0x7c, 0x28, 0x06, 0x65, 0x38, 0x34, 0x97, 0x54, 0x12, 0x09, 0x8f,
+	0xec, 0xfa, 0xb5, 0xd4, 0x7e, 0xa1, 0xc2, 0x6d, 0xa6, 0xc2, 0x3c, 0xbe, 0x1c, 0xa8, 0x20, 0x7e,
+	0xe1, 0xb6, 0xc4, 0x2f, 0xdf, 0x4b, 0x66, 0xbb, 0x4d, 0x97, 0xe4, 0x7b, 0x1a, 0x94, 0xd5, 0x6a,
+	0x19, 0xba, 0x9e, 0x58, 0x8c, 0x51, 0x0b, 0x6e, 0x3a, 0x3e, 0x8d, 0x45, 0xe0, 0xdf, 0x65, 0xf8,
+	0x37, 0xf0, 0x5c, 0x1a, 0xbe, 0xcb, 0xf8, 0xa3, 0x2a, 0xf0, 0x7a, 0x57, 0xb2, 0x0a, 0x91, 0x72,
+	0x5a, 0xb2, 0x0a, 0xd1, 0x72, 0xd9, 0xd9, 0x2a, 0xf4, 0x19, 0x3f, 0x55, 0xe1, 0x18, 0x20, 0x2c,
+	0x6f, 0xa1, 0x44, 0xe3, 0x2a, 0x97, 0x98, 0xb8, 0x0f, 0x0e, 0x57, 0xc6, 0x12, 0x76, 0x40, 0x0c,
+	0xbb, 0x63, 0x79, 0xd4, 0x17, 0x97, 0x7f, 0x9f, 0x83, 0xd2, 0x13, 0xd3, 0xb2, 0x7d, 0x62, 0x9b,
+	0x76, 0x8b, 0xa0, 0x03, 0xc8, 0xb1, 0x2c, 0x15, 0x0f, 0x3c, 0x6a, 0xd9, 0x27, 0x1e, 0x78, 0x22,
+	0x35, 0x11, 0x7c, 0x8b, 0x41, 0x5f, 0xc3, 0x7a, 0x00, 0xdd, 0x0d, 0xe5, 0x2f, 0xb1, 0x7a, 0x06,
+	0x9d, 0xf2, 0x11, 0xe4, 0x79, 0xfd, 0x02, 0xc5, 0xa4, 0x45, 0xea, 0x1c, 0xfa, 0x95, 0xe4, 0xce,
+	0xd4, 0x5d, 0xa6, 0x62, 0x79, 0x8c, 0x99, 0x82, 0x7d, 0x07, 0x20, 0xac, 0xd6, 0xc5, 0xed, 0x3b,
+	0x54, 0xdc, 0xd3, 0xe7, 0xd3, 0x19, 0x04, 0xf0, 0x3d, 0x06, 0x7c, 0x13, 0x5f, 0x4b, 0x04, 0x6e,
+	0x07, 0x03, 0x28, 0x78, 0x0b, 0xc6, 0xd7, 0x4d, 0xef, 0x10, 0xc5, 0x92, 0x90, 0xf2, 0xb0, 0xac,
+	0xeb, 0x49, 0x5d, 0x02, 0xea, 0x26, 0x83, 0x9a, 0xc3, 0xb3, 0x89, 0x50, 0x87, 0xa6, 0x47, 0x63,
+	0x3a, 0xea, 0xc3, 0x84, 0x7c, 0x2c, 0x46, 0x57, 0x63, 0x36, 0x8b, 0x3e, 0x2c, 0xeb, 0x73, 0x69,
+	0xdd, 0x02, 0x70, 0x81, 0x01, 0x62, 0x7c, 0x35, 0xd9, 0xa8, 0x82, 0xfd, 0xa1, 0x76, 0xef, 0x55,
+	0x8d, 0x46, 0x54, 0x08, 0x4b, 0x91, 0x43, 0x3b, 0x37, 0x5e, 0xd5, 0x1c, 0xda, 0xb9, 0x43, 0x55,
+	0x4c, 0xfc, 0x3a, 0x43, 0x7f, 0x80, 0x17, 0x12, 0xd1, 0x7d, 0xd7, 0xb4, 0xbd, 0x17, 0xc4, 0x7d,
+	0xc0, 0x6b, 0x4e, 0xde, 0xa1, 0xd5, 0xa3, 0xbb, 0xf8, 0x47, 0x35, 0x18, 0xa7, 0xc7, 0x36, 0x9a,
+	0xcc, 0xc2, 0xdb, 0x6e, 0x5c, 0x9d, 0xa1, 0x1a, 0x53, 0x5c, 0x9d, 0xe1, 0x8b, 0x72, 0x42, 0x32,
+	0x63, 0x3f, 0x03, 0x26, 0x8c, 0x8b, 0x1a, 0xde, 0x87, 0x92, 0x72, 0x27, 0x46, 0x09, 0x12, 0xa3,
+	0x15, 0xac, 0x78, 0x32, 0x4b, 0xb8, 0x50, 0xe3, 0x79, 0x06, 0xaa, 0xe3, 0x0b, 0x51, 0xd0, 0x36,
+	0x67, 0xa3, 0xa8, 0x1f, 0x43, 0x59, 0xbd, 0x3c, 0xa3, 0x04, 0xa1, 0xb1, 0x12, 0x59, 0x3c, 0x64,
+	0x25, 0xdd, 0xbd, 0x13, 0x7c, 0x37, 0xf8, 0xd1, 0xb3, 0xe4, 0xa5, 0xe8, 0x1f, 0x42, 0x41, 0x5c,
+	0xa9, 0x93, 0xe6, 0x1b, 0x2d, 0xaa, 0x25, 0xcd, 0x37, 0x76, 0x1f, 0x4f, 0x38, 0x19, 0x31, 0x58,
+	0x7a, 0x75, 0x90, 0x79, 0x42, 0x40, 0x3e, 0x26, 0x7e, 0x1a, 0x64, 0x58, 0x26, 0x4a, 0x83, 0x54,
+	0xae, 0x6d, 0xa7, 0x42, 0x1e, 0x10, 0x5f, 0xb8, 0x94, 0xbc, 0x13, 0xa1, 0x14, 0x89, 0x6a, 0x50,
+	0xc6, 0xa7, 0xb1, 0xa4, 0x1e, 0x66, 0x43, 0x54, 0x11, 0x91, 0xd1, 0x77, 0x01, 0xc2, 0xfb, 0x7f,
+	0xfc, 0x7c, 0x92, 0x58, 0x44, 0x8c, 0x9f, 0x4f, 0x92, 0x4b, 0x08, 0x09, 0x81, 0x24, 0x04, 0xe7,
+	0x07, 0x6a, 0x0a, 0xff, 0x53, 0x0d, 0xd0, 0x70, 0xbd, 0x00, 0xdd, 0x4f, 0x86, 0x48, 0xac, 0x4f,
+	0xea, 0xaf, 0x9c, 0x8f, 0x39, 0x35, 0x88, 0x87, 0x7a, 0xb5, 0xd8, 0x90, 0xde, 0x4b, 0xaa, 0xd9,
+	0x67, 0x1a, 0x54, 0x22, 0x15, 0x07, 0x74, 0x3b, 0x65, 0x9d, 0x63, 0x35, 0x4e, 0xfd, 0xce, 0x99,
+	0x7c, 0xa9, 0x47, 0x38, 0x65, 0x57, 0xc8, 0xe3, 0xeb, 0x0f, 0x35, 0xa8, 0x46, 0xcb, 0x14, 0x28,
+	0x05, 0x60, 0xa8, 0x50, 0xaa, 0x2f, 0x9c, 0xcd, 0x78, 0x8e, 0xd5, 0x0a, 0x4f, 0xb4, 0x1f, 0x42,
+	0x41, 0x54, 0x37, 0x92, 0xdc, 0x22, 0x5a, 0x67, 0x4d, 0x72, 0x8b, 0x58, 0x69, 0x24, 0xcd, 0x2d,
+	0x5c, 0xa7, 0x43, 0x14, 0x4f, 0x14, 0x35, 0x90, 0x34, 0xc8, 0xd3, 0x3d, 0x31, 0x56, 0x40, 0x39,
+	0x15, 0x32, 0xf4, 0x44, 0x59, 0x01, 0x41, 0x29, 0x12, 0xcf, 0xf0, 0xc4, 0x78, 0x01, 0x25, 0xcd,
+	0x13, 0x19, 0xaa, 0xe2, 0x89, 0x61, 0xc1, 0x22, 0xc9, 0x13, 0x87, 0xaa, 0xc8, 0x49, 0x9e, 0x38,
+	0x5c, 0xf3, 0x48, 0x5b, 0x5b, 0x06, 0x1e, 0xf1, 0xc4, 0xe9, 0x84, 0x02, 0x07, 0x7a, 0x25, 0xc5,
+	0xa6, 0x89, 0x15, 0x6a, 0xfd, 0xc1, 0x39, 0xb9, 0x4f, 0xf7, 0x00, 0xbe, 0x1a, 0xd2, 0x03, 0x7e,
+	0xa9, 0xc1, 0x4c, 0x52, 0x85, 0x04, 0xa5, 0x80, 0xa5, 0x94, 0xb7, 0xf5, 0xc5, 0xf3, 0xb2, 0x9f,
+	0xc3, 0x6e, 0x81, 0x4f, 0x3c, 0xaa, 0xfd, 0xf1, 0xcb, 0x39, 0xed, 0x2f, 0x5f, 0xce, 0x69, 0x7f,
+	0xfb, 0x72, 0x4e, 0xfb, 0xd9, 0xdf, 0xe7, 0xc6, 0xf6, 0xf3, 0xec, 0xff, 0xe2, 0xbc, 0xfe, 0xef,
+	0x00, 0x00, 0x00, 0xff, 0xff, 0x2b, 0x2a, 0x70, 0xa1, 0x12, 0x34, 0x00, 0x00,
 }

+ 20 - 3
etcdserver/etcdserverpb/rpc.proto

@@ -191,6 +191,14 @@ service Maintenance {
         body: "*"
     };
   }
+
+  // MoveLeader requests current leader node to transfer its leadership to transferee.
+  rpc MoveLeader(MoveLeaderRequest) returns (MoveLeaderResponse) {
+      option (google.api.http) = {
+        post: "/v3alpha/maintenance/transfer-leadership"
+        body: "*"
+    };
+  }
 }
 
 service Auth {
@@ -380,7 +388,7 @@ message RangeRequest {
 
   // keys_only when set returns only the keys and not the values.
   bool keys_only = 8;
- 
+
   // count_only when set returns only the count of the keys in the range.
   bool count_only = 9;
 
@@ -558,7 +566,7 @@ message TxnResponse {
 // CompactionRequest compacts the key-value store up to a given revision. All superseded keys
 // with a revision less than the compaction revision will be removed.
 message CompactionRequest {
-  // revision is the key-value store revision for the compaction operation. 
+  // revision is the key-value store revision for the compaction operation.
   int64 revision = 1;
   // physical is set so the RPC will wait until the compaction is physically
   // applied to the local database such that compacted entries are totally
@@ -654,7 +662,7 @@ message WatchResponse {
   // at a compacted index.
   //
   // This happens when creating a watcher at a compacted revision or the watcher cannot
-  // catch up with the progress of the key-value store. 
+  // catch up with the progress of the key-value store.
   //
   // The client should treat the watcher as canceled and should not try to create any
   // watcher with the same start_revision again.
@@ -787,6 +795,15 @@ message DefragmentResponse {
   ResponseHeader header = 1;
 }
 
+message MoveLeaderRequest {
+  // targetID is the node ID for the new leader.
+  uint64 targetID = 1;
+}
+
+message MoveLeaderResponse {
+  ResponseHeader header = 1;
+}
+
 enum AlarmType {
 	NONE = 0; // default, used to query if any alarm is active
 	NOSPACE = 1; // space quota is exhausted

+ 3 - 4
etcdserver/server.go

@@ -932,9 +932,8 @@ func (s *EtcdServer) isLeader() bool {
 	return uint64(s.ID()) == s.Lead()
 }
 
-// transferLeadership transfers the leader to the given transferee.
-// TODO: maybe expose to client?
-func (s *EtcdServer) transferLeadership(ctx context.Context, lead, transferee uint64) error {
+// MoveLeader transfers the leader to the given transferee.
+func (s *EtcdServer) MoveLeader(ctx context.Context, lead, transferee uint64) error {
 	now := time.Now()
 	interval := time.Duration(s.Cfg.TickMs) * time.Millisecond
 
@@ -973,7 +972,7 @@ func (s *EtcdServer) TransferLeadership() error {
 
 	tm := s.Cfg.ReqTimeout()
 	ctx, cancel := context.WithTimeout(s.ctx, tm)
-	err := s.transferLeadership(ctx, s.Lead(), uint64(transferee))
+	err := s.MoveLeader(ctx, s.Lead(), uint64(transferee))
 	cancel()
 	return err
 }

+ 2 - 0
integration/cluster.go

@@ -578,6 +578,8 @@ func (m *member) electionTimeout() time.Duration {
 	return time.Duration(m.s.Cfg.ElectionTicks) * time.Millisecond
 }
 
+func (m *member) ID() types.ID { return m.s.ID() }
+
 func (m *member) DropConnections()    { m.grpcBridge.Reset() }
 func (m *member) PauseConnections()   { m.grpcBridge.Pause() }
 func (m *member) UnpauseConnections() { m.grpcBridge.Unpause() }

+ 0 - 45
integration/cluster_test.go

@@ -526,51 +526,6 @@ func clusterMustProgress(t *testing.T, membs []*member) {
 	}
 }
 
-func TestTransferLeader(t *testing.T) {
-	defer testutil.AfterTest(t)
-
-	clus := NewClusterV3(t, &ClusterConfig{Size: 3})
-	defer clus.Terminate(t)
-
-	oldLeadIdx := clus.WaitLeader(t)
-	oldLeadID := uint64(clus.Members[oldLeadIdx].s.ID())
-
-	// ensure followers go through leader transition while learship transfer
-	idc := make(chan uint64)
-	for i := range clus.Members {
-		if oldLeadIdx != i {
-			go func(m *member) {
-				idc <- checkLeaderTransition(t, m, oldLeadID)
-			}(clus.Members[i])
-		}
-	}
-
-	err := clus.Members[oldLeadIdx].s.TransferLeadership()
-	if err != nil {
-		t.Fatal(err)
-	}
-
-	// wait until leader transitions have happened
-	var newLeadIDs [2]uint64
-	for i := range newLeadIDs {
-		select {
-		case newLeadIDs[i] = <-idc:
-		case <-time.After(time.Second):
-			t.Fatal("timed out waiting for leader transition")
-		}
-	}
-
-	// remaining members must agree on the same leader
-	if newLeadIDs[0] != newLeadIDs[1] {
-		t.Fatalf("expected same new leader %d == %d", newLeadIDs[0], newLeadIDs[1])
-	}
-
-	// new leader must be different than the old leader
-	if oldLeadID == newLeadIDs[0] {
-		t.Fatalf("expected old leader %d != new leader %d", oldLeadID, newLeadIDs[0])
-	}
-}
-
 func TestSpeedyTerminate(t *testing.T) {
 	defer testutil.AfterTest(t)
 	clus := NewClusterV3(t, &ClusterConfig{Size: 3})

+ 108 - 0
integration/v3_leadership_test.go

@@ -0,0 +1,108 @@
+// 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 (
+	"context"
+	"testing"
+	"time"
+
+	"github.com/coreos/etcd/etcdserver/api/v3rpc/rpctypes"
+	pb "github.com/coreos/etcd/etcdserver/etcdserverpb"
+	"github.com/coreos/etcd/pkg/testutil"
+)
+
+func TestMoveLeader(t *testing.T)        { testMoveLeader(t, true) }
+func TestMoveLeaderService(t *testing.T) { testMoveLeader(t, false) }
+
+func testMoveLeader(t *testing.T, auto bool) {
+	defer testutil.AfterTest(t)
+
+	clus := NewClusterV3(t, &ClusterConfig{Size: 3})
+	defer clus.Terminate(t)
+
+	oldLeadIdx := clus.WaitLeader(t)
+	oldLeadID := uint64(clus.Members[oldLeadIdx].s.ID())
+
+	// ensure followers go through leader transition while learship transfer
+	idc := make(chan uint64)
+	for i := range clus.Members {
+		if oldLeadIdx != i {
+			go func(m *member) {
+				idc <- checkLeaderTransition(t, m, oldLeadID)
+			}(clus.Members[i])
+		}
+	}
+
+	target := uint64(clus.Members[(oldLeadIdx+1)%3].s.ID())
+	if auto {
+		err := clus.Members[oldLeadIdx].s.TransferLeadership()
+		if err != nil {
+			t.Fatal(err)
+		}
+	} else {
+		mvc := toGRPC(clus.Client(oldLeadIdx)).Maintenance
+		_, err := mvc.MoveLeader(context.TODO(), &pb.MoveLeaderRequest{TargetID: target})
+		if err != nil {
+			t.Fatal(err)
+		}
+	}
+
+	// wait until leader transitions have happened
+	var newLeadIDs [2]uint64
+	for i := range newLeadIDs {
+		select {
+		case newLeadIDs[i] = <-idc:
+		case <-time.After(time.Second):
+			t.Fatal("timed out waiting for leader transition")
+		}
+	}
+
+	// remaining members must agree on the same leader
+	if newLeadIDs[0] != newLeadIDs[1] {
+		t.Fatalf("expected same new leader %d == %d", newLeadIDs[0], newLeadIDs[1])
+	}
+
+	// new leader must be different than the old leader
+	if oldLeadID == newLeadIDs[0] {
+		t.Fatalf("expected old leader %d != new leader %d", oldLeadID, newLeadIDs[0])
+	}
+
+	// if move-leader were used, new leader must match transferee
+	if !auto {
+		if newLeadIDs[0] != target {
+			t.Fatalf("expected new leader %d != target %d", newLeadIDs[0], target)
+		}
+	}
+}
+
+// TestMoveLeaderError ensures that request to non-leader fail.
+func TestMoveLeaderError(t *testing.T) {
+	defer testutil.AfterTest(t)
+
+	clus := NewClusterV3(t, &ClusterConfig{Size: 3})
+	defer clus.Terminate(t)
+
+	oldLeadIdx := clus.WaitLeader(t)
+	followerIdx := (oldLeadIdx + 1) % 3
+
+	target := uint64(clus.Members[(oldLeadIdx+2)%3].s.ID())
+
+	mvc := toGRPC(clus.Client(followerIdx)).Maintenance
+	_, err := mvc.MoveLeader(context.TODO(), &pb.MoveLeaderRequest{TargetID: target})
+	if !eqErrGRPC(err, rpctypes.ErrGRPCNotLeader) {
+		t.Errorf("err = %v, want %v", err, rpctypes.ErrGRPCNotLeader)
+	}
+}

+ 4 - 0
proxy/grpcproxy/adapter/maintenance_client_adapter.go

@@ -43,6 +43,10 @@ func (s *mts2mtc) Hash(ctx context.Context, r *pb.HashRequest, opts ...grpc.Call
 	return s.mts.Hash(ctx, r)
 }
 
+func (s *mts2mtc) MoveLeader(ctx context.Context, r *pb.MoveLeaderRequest, opts ...grpc.CallOption) (*pb.MoveLeaderResponse, error) {
+	return s.mts.MoveLeader(ctx, r)
+}
+
 func (s *mts2mtc) Snapshot(ctx context.Context, in *pb.SnapshotRequest, opts ...grpc.CallOption) (pb.Maintenance_SnapshotClient, error) {
 	cs := newPipeStream(ctx, func(ss chanServerStream) error {
 		return s.mts.Snapshot(in, &ss2scServerStream{ss})

+ 5 - 0
proxy/grpcproxy/maintenance.go

@@ -72,3 +72,8 @@ func (mp *maintenanceProxy) Status(ctx context.Context, r *pb.StatusRequest) (*p
 	conn := mp.client.ActiveConnection()
 	return pb.NewMaintenanceClient(conn).Status(ctx, r)
 }
+
+func (mp *maintenanceProxy) MoveLeader(ctx context.Context, r *pb.MoveLeaderRequest) (*pb.MoveLeaderResponse, error) {
+	conn := mp.client.ActiveConnection()
+	return pb.NewMaintenanceClient(conn).MoveLeader(ctx, r)
+}